diff --git a/.github/PULL_REQUEST_TEMPLATE b/.github/PULL_REQUEST_TEMPLATE index 5af45d6fa7988..e7ed23dea0f9c 100644 --- a/.github/PULL_REQUEST_TEMPLATE +++ b/.github/PULL_REQUEST_TEMPLATE @@ -7,4 +7,4 @@ (Please explain how this patch was tested. E.g. unit tests, integration tests, manual tests) (If this patch involves UI changes, please attach a screenshot; otherwise, remove this) -Please review http://spark.apache.org/contributing.html before opening a pull request. +Please review https://spark.apache.org/contributing.html before opening a pull request. diff --git a/.gitignore b/.gitignore index 19db7ac277944..d5cf66d1db126 100644 --- a/.gitignore +++ b/.gitignore @@ -77,7 +77,6 @@ target/ unit-tests.log work/ docs/.jekyll-metadata -*.crc # For Hive TempStatsStore/ @@ -95,3 +94,6 @@ spark-warehouse/ *.Rproj.* .Rproj.user + +# For SBT +.jvmopts diff --git a/CONTRIBUTING.md b/CONTRIBUTING.md index 8fdd5aa9e7dfb..925a7457fd656 100644 --- a/CONTRIBUTING.md +++ b/CONTRIBUTING.md @@ -1,12 +1,12 @@ ## Contributing to Spark *Before opening a pull request*, review the -[Contributing to Spark guide](http://spark.apache.org/contributing.html). +[Contributing to Spark guide](https://spark.apache.org/contributing.html). It lists steps that are required before creating a PR. In particular, consider: - Is the change important and ready enough to ask the community to spend time reviewing? - Have you searched for existing, related JIRAs and pull requests? -- Is this a new feature that can stand alone as a [third party project](http://spark.apache.org/third-party-projects.html) ? +- Is this a new feature that can stand alone as a [third party project](https://spark.apache.org/third-party-projects.html) ? - Is the change being proposed clearly explained and motivated? When you contribute code, you affirm that the contribution is your original work and that you diff --git a/LICENSE b/LICENSE index b771bd552b762..150ccc54ec6c2 100644 --- a/LICENSE +++ b/LICENSE @@ -222,7 +222,7 @@ Python Software Foundation License ---------------------------------- pyspark/heapq3.py - +python/docs/_static/copybutton.js BSD 3-Clause ------------ @@ -258,4 +258,4 @@ data/mllib/images/kittens/29.5.a_b_EGDP022204.jpg data/mllib/images/kittens/54893.jpg data/mllib/images/kittens/DP153539.jpg data/mllib/images/kittens/DP802813.jpg -data/mllib/images/multi-channel/chr30.4.184.jpg \ No newline at end of file +data/mllib/images/multi-channel/chr30.4.184.jpg diff --git a/LICENSE-binary b/LICENSE-binary index b94ea90de08be..0d3dcd78fcbd8 100644 --- a/LICENSE-binary +++ b/LICENSE-binary @@ -209,10 +209,10 @@ org.apache.zookeeper:zookeeper oro:oro commons-configuration:commons-configuration commons-digester:commons-digester -com.chuusai:shapeless_2.11 +com.chuusai:shapeless_2.12 com.googlecode.javaewah:JavaEWAH com.twitter:chill-java -com.twitter:chill_2.11 +com.twitter:chill_2.12 com.univocity:univocity-parsers javax.jdo:jdo-api joda-time:joda-time @@ -220,23 +220,23 @@ net.sf.opencsv:opencsv org.apache.derby:derby org.objenesis:objenesis org.roaringbitmap:RoaringBitmap -org.scalanlp:breeze-macros_2.11 -org.scalanlp:breeze_2.11 -org.typelevel:macro-compat_2.11 +org.scalanlp:breeze-macros_2.12 +org.scalanlp:breeze_2.12 +org.typelevel:macro-compat_2.12 org.yaml:snakeyaml org.apache.xbean:xbean-asm5-shaded com.squareup.okhttp3:logging-interceptor com.squareup.okhttp3:okhttp com.squareup.okio:okio -org.apache.spark:spark-catalyst_2.11 -org.apache.spark:spark-kvstore_2.11 -org.apache.spark:spark-launcher_2.11 -org.apache.spark:spark-mllib-local_2.11 -org.apache.spark:spark-network-common_2.11 -org.apache.spark:spark-network-shuffle_2.11 -org.apache.spark:spark-sketch_2.11 -org.apache.spark:spark-tags_2.11 -org.apache.spark:spark-unsafe_2.11 +org.apache.spark:spark-catalyst_2.12 +org.apache.spark:spark-kvstore_2.12 +org.apache.spark:spark-launcher_2.12 +org.apache.spark:spark-mllib-local_2.12 +org.apache.spark:spark-network-common_2.12 +org.apache.spark:spark-network-shuffle_2.12 +org.apache.spark:spark-sketch_2.12 +org.apache.spark:spark-tags_2.12 +org.apache.spark:spark-unsafe_2.12 commons-httpclient:commons-httpclient com.vlkan:flatbuffers com.ning:compress-lzf @@ -260,9 +260,6 @@ net.sf.supercsv:super-csv org.apache.arrow:arrow-format org.apache.arrow:arrow-memory org.apache.arrow:arrow-vector -org.apache.calcite:calcite-avatica -org.apache.calcite:calcite-core -org.apache.calcite:calcite-linq4j org.apache.commons:commons-crypto org.apache.commons:commons-lang3 org.apache.hadoop:hadoop-annotations @@ -287,10 +284,10 @@ org.apache.orc:orc-mapreduce org.mortbay.jetty:jetty org.mortbay.jetty:jetty-util com.jolbox:bonecp -org.json4s:json4s-ast_2.11 -org.json4s:json4s-core_2.11 -org.json4s:json4s-jackson_2.11 -org.json4s:json4s-scalap_2.11 +org.json4s:json4s-ast_2.12 +org.json4s:json4s-core_2.12 +org.json4s:json4s-jackson_2.12 +org.json4s:json4s-scalap_2.12 com.carrotsearch:hppc com.fasterxml.jackson.core:jackson-annotations com.fasterxml.jackson.core:jackson-core @@ -298,14 +295,13 @@ com.fasterxml.jackson.core:jackson-databind com.fasterxml.jackson.dataformat:jackson-dataformat-yaml com.fasterxml.jackson.module:jackson-module-jaxb-annotations com.fasterxml.jackson.module:jackson-module-paranamer -com.fasterxml.jackson.module:jackson-module-scala_2.11 +com.fasterxml.jackson.module:jackson-module-scala_2.12 com.github.mifmif:generex com.google.code.findbugs:jsr305 com.google.code.gson:gson com.google.inject:guice com.google.inject.extensions:guice-servlet com.twitter:parquet-hadoop-bundle -commons-beanutils:commons-beanutils-core commons-cli:commons-cli commons-dbcp:commons-dbcp commons-io:commons-io @@ -372,6 +368,8 @@ org.eclipse.jetty:jetty-servlets org.eclipse.jetty:jetty-util org.eclipse.jetty:jetty-webapp org.eclipse.jetty:jetty-xml +org.scala-lang.modules:scala-xml_2.12 +org.opencypher:okapi-shade core/src/main/java/org/apache/spark/util/collection/TimSort.java core/src/main/resources/org/apache/spark/ui/static/bootstrap* @@ -415,8 +413,7 @@ com.thoughtworks.paranamer:paranamer org.scala-lang:scala-compiler org.scala-lang:scala-library org.scala-lang:scala-reflect -org.scala-lang.modules:scala-parser-combinators_2.11 -org.scala-lang.modules:scala-xml_2.11 +org.scala-lang.modules:scala-parser-combinators_2.12 org.fusesource.leveldbjni:leveldbjni-all net.sourceforge.f2j:arpack_combined_all xmlenc:xmlenc @@ -437,15 +434,15 @@ is distributed under the 3-Clause BSD license. MIT License ----------- -org.spire-math:spire-macros_2.11 -org.spire-math:spire_2.11 -org.typelevel:machinist_2.11 +org.spire-math:spire-macros_2.12 +org.spire-math:spire_2.12 +org.typelevel:machinist_2.12 net.razorvine:pyrolite org.slf4j:jcl-over-slf4j org.slf4j:jul-to-slf4j org.slf4j:slf4j-api org.slf4j:slf4j-log4j12 -com.github.scopt:scopt_2.11 +com.github.scopt:scopt_2.12 core/src/main/resources/org/apache/spark/ui/static/dagre-d3.min.js core/src/main/resources/org/apache/spark/ui/static/*dataTables* @@ -487,6 +484,14 @@ org.glassfish.jersey.core:jersey-server org.glassfish.jersey.media:jersey-media-jaxb +Eclipse Distribution License (EDL) 1.0 +-------------------------------------- + +org.glassfish.jaxb:jaxb-runtime +jakarta.xml.bind:jakarta.xml.bind-api +com.sun.istack:istack-commons-runtime + + Mozilla Public License (MPL) 1.1 -------------------------------- diff --git a/NOTICE-binary b/NOTICE-binary index b707c436983f7..1198e79f36b19 100644 --- a/NOTICE-binary +++ b/NOTICE-binary @@ -792,15 +792,6 @@ Copyright 2005-2006 The Apache Software Foundation Apache Jakarta HttpClient Copyright 1999-2007 The Apache Software Foundation -Calcite Avatica -Copyright 2012-2015 The Apache Software Foundation - -Calcite Core -Copyright 2012-2015 The Apache Software Foundation - -Calcite Linq4j -Copyright 2012-2015 The Apache Software Foundation - Apache HttpClient Copyright 1999-2017 The Apache Software Foundation @@ -1172,3 +1163,18 @@ Copyright 2014 The Apache Software Foundation Apache Mahout (http://mahout.apache.org/) Copyright 2014 The Apache Software Foundation + +scala-xml +Copyright (c) 2002-2019 EPFL +Copyright (c) 2011-2019 Lightbend, Inc. + +scala-xml includes software developed at +LAMP/EPFL (https://lamp.epfl.ch/) and +Lightbend, Inc. (https://www.lightbend.com/). + +Licensed under the Apache License, Version 2.0 (the "License"). +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. \ No newline at end of file diff --git a/R/CRAN_RELEASE.md b/R/CRAN_RELEASE.md index d6084c7a7cc90..4d9b6416c01cb 100644 --- a/R/CRAN_RELEASE.md +++ b/R/CRAN_RELEASE.md @@ -1,3 +1,21 @@ +--- +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. +--- + # SparkR CRAN Release To release SparkR as a package to CRAN, we would use the `devtools` package. Please work with the diff --git a/R/DOCUMENTATION.md b/R/DOCUMENTATION.md index 7314a1fcccda9..fd0c7644189db 100644 --- a/R/DOCUMENTATION.md +++ b/R/DOCUMENTATION.md @@ -1,3 +1,21 @@ +--- +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. +--- + # SparkR Documentation SparkR documentation is generated by using in-source comments and annotated by using diff --git a/R/README.md b/R/README.md index d77a1ecffc99c..c808ca88f72dc 100644 --- a/R/README.md +++ b/R/README.md @@ -17,7 +17,7 @@ export R_HOME=/home/username/R #### Build Spark -Build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run +Build Spark with [Maven](https://spark.apache.org/docs/latest/building-spark.html#buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run ```bash build/mvn -DskipTests -Psparkr package @@ -35,23 +35,15 @@ SparkContext, you can run ./bin/sparkR --master "local[2]" -To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](http://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR` +To set other options like driver memory, executor memory etc. you can pass in the [spark-submit](https://spark.apache.org/docs/latest/submitting-applications.html) arguments to `./bin/sparkR` #### Using SparkR from RStudio -If you wish to use SparkR from RStudio or other R frontends you will need to set some environment variables which point SparkR to your Spark installation. For example -```R -# Set this to where Spark is installed -Sys.setenv(SPARK_HOME="/Users/username/spark") -# This line loads SparkR from the installed directory -.libPaths(c(file.path(Sys.getenv("SPARK_HOME"), "R", "lib"), .libPaths())) -library(SparkR) -sparkR.session() -``` +If you wish to use SparkR from RStudio, please refer [SparkR documentation](https://spark.apache.org/docs/latest/sparkr.html#starting-up-from-rstudio). #### Making changes to SparkR -The [instructions](http://spark.apache.org/contributing.html) for making contributions to Spark also apply to SparkR. +The [instructions](https://spark.apache.org/contributing.html) for making contributions to Spark also apply to SparkR. If you only make R file changes (i.e. no Scala changes) then you can just re-install the R package using `R/install-dev.sh` and test your changes. Once you have made your changes, please include unit tests for them and run existing unit tests using the `R/run-tests.sh` script as described below. @@ -66,7 +58,7 @@ To run one of them, use `./bin/spark-submit `. For example: ```bash ./bin/spark-submit examples/src/main/r/dataframe.R ``` -You can run R unit tests by following the instructions under [Running R Tests](http://spark.apache.org/docs/latest/building-spark.html#running-r-tests). +You can run R unit tests by following the instructions under [Running R Tests](https://spark.apache.org/docs/latest/building-spark.html#running-r-tests). ### Running on YARN diff --git a/R/WINDOWS.md b/R/WINDOWS.md index da668a69b8679..7982451cffc31 100644 --- a/R/WINDOWS.md +++ b/R/WINDOWS.md @@ -1,20 +1,38 @@ +--- +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. +--- + ## Building SparkR on Windows To build SparkR on Windows, the following steps are required -1. Install R (>= 3.1) and [Rtools](http://cran.r-project.org/bin/windows/Rtools/). Make sure to -include Rtools and R in `PATH`. +1. Install R (>= 3.1) and [Rtools](https://cloud.r-project.org/bin/windows/Rtools/). Make sure to +include Rtools and R in `PATH`. Note that support for R prior to version 3.4 is deprecated as of Spark 3.0.0. 2. Install -[JDK8](http://www.oracle.com/technetwork/java/javase/downloads/jdk8-downloads-2133151.html) and set +[JDK8](https://www.oracle.com/technetwork/java/javase/downloads/jdk8-downloads-2133151.html) and set `JAVA_HOME` in the system environment variables. -3. Download and install [Maven](http://maven.apache.org/download.html). Also include the `bin` +3. Download and install [Maven](https://maven.apache.org/download.html). Also include the `bin` directory in Maven in `PATH`. -4. Set `MAVEN_OPTS` as described in [Building Spark](http://spark.apache.org/docs/latest/building-spark.html). +4. Set `MAVEN_OPTS` as described in [Building Spark](https://spark.apache.org/docs/latest/building-spark.html). -5. Open a command shell (`cmd`) in the Spark directory and build Spark with [Maven](http://spark.apache.org/docs/latest/building-spark.html#buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run +5. Open a command shell (`cmd`) in the Spark directory and build Spark with [Maven](https://spark.apache.org/docs/latest/building-spark.html#buildmvn) and include the `-Psparkr` profile to build the R package. For example to use the default Hadoop versions you can run ```bash mvn.cmd -DskipTests -Psparkr package @@ -34,7 +52,7 @@ To run the SparkR unit tests on Windows, the following steps are required —ass 4. Set the environment variable `HADOOP_HOME` to the full path to the newly created `hadoop` directory. -5. Run unit tests for SparkR by running the command below. You need to install the needed packages following the instructions under [Running R Tests](http://spark.apache.org/docs/latest/building-spark.html#running-r-tests) first: +5. Run unit tests for SparkR by running the command below. You need to install the needed packages following the instructions under [Running R Tests](https://spark.apache.org/docs/latest/building-spark.html#running-r-tests) first: ``` .\bin\spark-submit2.cmd --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION index cdaaa6104e6a9..3d31be809be61 100644 --- a/R/pkg/DESCRIPTION +++ b/R/pkg/DESCRIPTION @@ -1,8 +1,8 @@ Package: SparkR Type: Package Version: 3.0.0 -Title: R Frontend for Apache Spark -Description: Provides an R Frontend for Apache Spark. +Title: R Front End for 'Apache Spark' +Description: Provides an R Front end for 'Apache Spark' . Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "shivaram@cs.berkeley.edu"), person("Xiangrui", "Meng", role = "aut", @@ -11,11 +11,11 @@ Authors@R: c(person("Shivaram", "Venkataraman", role = c("aut", "cre"), email = "felixcheung@apache.org"), person(family = "The Apache Software Foundation", role = c("aut", "cph"))) License: Apache License (== 2.0) -URL: http://www.apache.org/ http://spark.apache.org/ -BugReports: http://spark.apache.org/contributing.html +URL: https://www.apache.org/ https://spark.apache.org/ +BugReports: https://spark.apache.org/contributing.html SystemRequirements: Java (== 8) Depends: - R (>= 3.0), + R (>= 3.1), methods Suggests: knitr, diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE index 96ff389faf4a0..f9d9494ca6fa1 100644 --- a/R/pkg/NAMESPACE +++ b/R/pkg/NAMESPACE @@ -28,9 +28,8 @@ importFrom("utils", "download.file", "object.size", "packageVersion", "tail", "u # S3 methods exported export("sparkR.session") -export("sparkR.init") -export("sparkR.stop") export("sparkR.session.stop") +export("sparkR.stop") export("sparkR.conf") export("sparkR.version") export("sparkR.uiWebUrl") @@ -42,9 +41,6 @@ export("sparkR.callJStatic") export("install.spark") -export("sparkRSQL.init", - "sparkRHive.init") - # MLlib integration exportMethods("glm", "spark.glm", @@ -70,7 +66,9 @@ exportMethods("glm", "spark.svmLinear", "spark.fpGrowth", "spark.freqItemsets", - "spark.associationRules") + "spark.associationRules", + "spark.findFrequentSequentialPatterns", + "spark.assignClusters") # Job group lifecycle management methods export("setJobGroup", @@ -150,7 +148,6 @@ exportMethods("arrange", "printSchema", "randomSplit", "rbind", - "registerTempTable", "rename", "repartition", "repartitionByRange", @@ -158,7 +155,6 @@ exportMethods("arrange", "sample", "sample_frac", "sampleBy", - "saveAsParquetFile", "saveAsTable", "saveDF", "schema", @@ -200,6 +196,7 @@ exportMethods("%<=>%", "acos", "add_months", "alias", + "approx_count_distinct", "approxCountDistinct", "approxQuantile", "array_contains", @@ -258,6 +255,7 @@ exportMethods("%<=>%", "dayofweek", "dayofyear", "decode", + "degrees", "dense_rank", "desc", "element_at", @@ -274,6 +272,7 @@ exportMethods("%<=>%", "floor", "format_number", "format_string", + "from_csv", "from_json", "from_unixtime", "from_utc_timestamp", @@ -313,8 +312,10 @@ exportMethods("%<=>%", "lower", "lpad", "ltrim", + "map_concat", "map_entries", "map_from_arrays", + "map_from_entries", "map_keys", "map_values", "max", @@ -339,6 +340,7 @@ exportMethods("%<=>%", "posexplode", "posexplode_outer", "quarter", + "radians", "rand", "randn", "rank", @@ -352,6 +354,8 @@ exportMethods("%<=>%", "row_number", "rpad", "rtrim", + "schema_of_csv", + "schema_of_json", "second", "sha1", "sha2", @@ -385,6 +389,7 @@ exportMethods("%<=>%", "tanh", "toDegrees", "toRadians", + "to_csv", "to_date", "to_json", "to_timestamp", @@ -403,6 +408,7 @@ exportMethods("%<=>%", "weekofyear", "when", "window", + "xxhash64", "year") exportClasses("GroupedData") @@ -413,18 +419,14 @@ export("as.DataFrame", "cacheTable", "clearCache", "createDataFrame", - "createExternalTable", "createTable", "currentDatabase", - "dropTempTable", "dropTempView", - "jsonFile", "listColumns", "listDatabases", "listFunctions", "listTables", "loadDF", - "parquetFile", "read.df", "read.jdbc", "read.json", diff --git a/R/pkg/R/DataFrame.R b/R/pkg/R/DataFrame.R index 34691883bc5a9..6f3c7c120ba3c 100644 --- a/R/pkg/R/DataFrame.R +++ b/R/pkg/R/DataFrame.R @@ -226,7 +226,9 @@ setMethod("showDF", #' show #' -#' Print class and type information of a Spark object. +#' If eager evaluation is enabled and the Spark object is a SparkDataFrame, evaluate the +#' SparkDataFrame and print top rows of the SparkDataFrame, otherwise, print the class +#' and type information of the Spark object. #' #' @param object a Spark object. Can be a SparkDataFrame, Column, GroupedData, WindowSpec. #' @@ -244,11 +246,33 @@ setMethod("showDF", #' @note show(SparkDataFrame) since 1.4.0 setMethod("show", "SparkDataFrame", function(object) { - cols <- lapply(dtypes(object), function(l) { - paste(l, collapse = ":") - }) - s <- paste(cols, collapse = ", ") - cat(paste(class(object), "[", s, "]\n", sep = "")) + allConf <- sparkR.conf() + prop <- allConf[["spark.sql.repl.eagerEval.enabled"]] + if (!is.null(prop) && identical(prop, "true")) { + argsList <- list() + argsList$x <- object + prop <- allConf[["spark.sql.repl.eagerEval.maxNumRows"]] + if (!is.null(prop)) { + numRows <- as.integer(prop) + if (numRows > 0) { + argsList$numRows <- numRows + } + } + prop <- allConf[["spark.sql.repl.eagerEval.truncate"]] + if (!is.null(prop)) { + truncate <- as.integer(prop) + if (truncate > 0) { + argsList$truncate <- truncate + } + } + do.call(showDF, argsList) + } else { + cols <- lapply(dtypes(object), function(l) { + paste(l, collapse = ":") + }) + s <- paste(cols, collapse = ", ") + cat(paste(class(object), "[", s, "]\n", sep = "")) + } }) #' DataTypes @@ -497,32 +521,6 @@ 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. @@ -768,6 +766,13 @@ setMethod("repartition", #' \item{2.} {Return a new SparkDataFrame range partitioned by the given column(s), #' using \code{spark.sql.shuffle.partitions} as number of partitions.} #'} +#' At least one partition-by expression must be specified. +#' When no explicit sort order is specified, "ascending nulls first" is assumed. +#' +#' Note that due to performance reasons this method uses sampling to estimate the ranges. +#' Hence, the output may not be consistent, since sampling can return different values. +#' The sample size can be controlled by the config +#' \code{spark.sql.execution.rangeExchange.sampleSizePerPartition}. #' #' @param x a SparkDataFrame. #' @param numPartitions the number of partitions to use. @@ -822,7 +827,6 @@ setMethod("repartitionByRange", #' toJSON #' #' Converts a SparkDataFrame into a SparkDataFrame of JSON string. -#' #' Each row is turned into a JSON document with columns as different fields. #' The returned SparkDataFrame has a single character column with the name \code{value} #' @@ -932,7 +936,6 @@ setMethod("write.orc", #' path <- "path/to/file.json" #' df <- read.json(path) #' write.parquet(df, "/tmp/sparkr-tmp1/") -#' saveAsParquetFile(df, "/tmp/sparkr-tmp2/") #'} #' @note write.parquet since 1.6.0 setMethod("write.parquet", @@ -943,22 +946,11 @@ setMethod("write.parquet", invisible(handledCallJMethod(write, "parquet", path)) }) -#' @rdname write.parquet -#' @name saveAsParquetFile -#' @aliases saveAsParquetFile,SparkDataFrame,character-method -#' @note saveAsParquetFile since 1.4.0 -setMethod("saveAsParquetFile", - signature(x = "SparkDataFrame", path = "character"), - function(x, path) { - .Deprecated("write.parquet") - write.parquet(x, path) - }) - #' Save the content of SparkDataFrame in a text file at the specified path. #' #' Save the content of the SparkDataFrame in a text file at the specified path. #' The SparkDataFrame must have only one column of string type with the name "value". -#' Each row becomes a new line in the output file. +#' Each row becomes a new line in the output file. The text files will be encoded as UTF-8. #' #' @param x A SparkDataFrame #' @param path The directory where the file is saved @@ -1185,11 +1177,56 @@ setMethod("dim", setMethod("collect", signature(x = "SparkDataFrame"), function(x, stringsAsFactors = FALSE) { + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) + useArrow <- FALSE + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] == "true" + if (arrowEnabled) { + useArrow <- tryCatch({ + checkSchemaInArrow(schema(x)) + TRUE + }, error = function(e) { + warning(paste0("The conversion from Spark DataFrame to R DataFrame was attempted ", + "with Arrow optimization because ", + "'spark.sql.execution.arrow.sparkr.enabled' is set to true; ", + "however, failed, attempting non-optimization. Reason: ", + e)) + FALSE + }) + } + dtypes <- dtypes(x) ncol <- length(dtypes) if (ncol <= 0) { # empty data.frame with 0 columns and 0 rows data.frame() + } else if (useArrow) { + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + read_arrow <- get("read_arrow", envir = asNamespace("arrow"), inherits = FALSE) + # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. + useAsTibble <- exists("as_tibble", envir = asNamespace("arrow")) + + portAuth <- callJMethod(x@sdf, "collectAsArrowToR") + port <- portAuth[[1]] + authSecret <- portAuth[[2]] + conn <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) + output <- tryCatch({ + doServerAuth(conn, authSecret) + arrowTable <- read_arrow(readRaw(conn)) + if (useAsTibble) { + as_tibble <- get("as_tibble", envir = asNamespace("arrow")) + as.data.frame(as_tibble(arrowTable), stringsAsFactors = stringsAsFactors) + } else { + as.data.frame(arrowTable, stringsAsFactors = stringsAsFactors) + } + }, finally = { + close(conn) + }) + return(output) + } else { + stop("'arrow' package should be installed.") + } } else { # listCols is a list of columns listCols <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "dfToCols", x@sdf) @@ -1445,6 +1482,18 @@ dapplyInternal <- function(x, func, schema) { schema <- structType(schema) } + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] == "true" + if (arrowEnabled) { + if (inherits(schema, "structType")) { + checkSchemaInArrow(schema) + } else if (is.null(schema)) { + stop(paste0("Arrow optimization does not support 'dapplyCollect' yet. Please disable ", + "Arrow optimization or use 'collect' and 'dapply' APIs instead.")) + } else { + stop("'schema' should be DDL-formatted string or structType.") + } + } + packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) @@ -2100,6 +2149,11 @@ setMethod("selectExpr", #' Return a new SparkDataFrame by adding a column or replacing the existing column #' that has the same name. #' +#' Note: This method introduces a projection internally. Therefore, calling it multiple times, +#' for instance, via loops in order to add multiple columns can generate big plans which +#' can cause performance issues and even \code{StackOverflowException}. To avoid this, +#' use \code{select} with the multiple columns at once. +#' #' @param x a SparkDataFrame. #' @param colName a column name. #' @param col a Column expression (which must refer only to this SparkDataFrame), or an atomic @@ -2477,8 +2531,9 @@ setMethod("dropDuplicates", #' Column expression. If joinExpr is omitted, the default, inner join is attempted and an error is #' thrown if it would be a Cartesian Product. For Cartesian join, use crossJoin instead. #' @param joinType The type of join to perform, default 'inner'. -#' Must be one of: 'inner', 'cross', 'outer', 'full', 'full_outer', -#' 'left', 'left_outer', 'right', 'right_outer', 'left_semi', or 'left_anti'. +#' Must be one of: 'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer', +#' 'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer', 'semi', +#' 'leftsemi', 'left_semi', 'anti', 'leftanti', 'left_anti'. #' @return A SparkDataFrame containing the result of the join operation. #' @family SparkDataFrame functions #' @aliases join,SparkDataFrame,SparkDataFrame-method @@ -2510,14 +2565,14 @@ setMethod("join", "outer", "full", "fullouter", "full_outer", "left", "leftouter", "left_outer", "right", "rightouter", "right_outer", - "left_semi", "leftsemi", "left_anti", "leftanti")) { + "semi", "left_semi", "leftsemi", "anti", "left_anti", "leftanti")) { joinType <- gsub("_", "", joinType) sdf <- callJMethod(x@sdf, "join", y@sdf, joinExpr@jc, joinType) } else { - stop("joinType must be one of the following types: ", - "'inner', 'cross', 'outer', 'full', 'full_outer',", - "'left', 'left_outer', 'right', 'right_outer',", - "'left_semi', or 'left_anti'.") + stop(paste("joinType must be one of the following types:", + "'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',", + "'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',", + "'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'.")) } } } @@ -2738,15 +2793,29 @@ setMethod("union", dataFrame(unioned) }) -#' unionAll is deprecated - use union instead -#' @rdname union -#' @name unionAll +#' Return a new SparkDataFrame containing the union of rows. +#' +#' This is an alias for \code{union}. +#' +#' @param x a SparkDataFrame. +#' @param y a SparkDataFrame. +#' @return A SparkDataFrame containing the result of the unionAll operation. +#' @family SparkDataFrame functions #' @aliases unionAll,SparkDataFrame,SparkDataFrame-method +#' @rdname unionAll +#' @name unionAll +#' @seealso \link{union} +#' @examples +#'\dontrun{ +#' sparkR.session() +#' df1 <- read.json(path) +#' df2 <- read.json(path2) +#' unionAllDF <- unionAll(df1, df2) +#' } #' @note unionAll since 1.4.0 setMethod("unionAll", signature(x = "SparkDataFrame", y = "SparkDataFrame"), function(x, y) { - .Deprecated("union") union(x, y) }) diff --git a/R/pkg/R/SQLContext.R b/R/pkg/R/SQLContext.R index c819a7d14ae98..69ddaab9eeb86 100644 --- a/R/pkg/R/SQLContext.R +++ b/R/pkg/R/SQLContext.R @@ -37,37 +37,6 @@ getInternalType <- function(x) { stop(paste("Unsupported type for SparkDataFrame:", class(x)))) } -#' Temporary function to reroute old S3 Method call to new -#' This function is specifically implemented to remove SQLContext from the parameter list. -#' It determines the target to route the call by checking the parent of this callsite (say 'func'). -#' The target should be called 'func.default'. -#' We need to check the class of x to ensure it is SQLContext/HiveContext before dispatching. -#' @param newFuncSig name of the function the user should call instead in the deprecation message -#' @param x the first parameter of the original call -#' @param ... the rest of parameter to pass along -#' @return whatever the target returns -#' @noRd -dispatchFunc <- function(newFuncSig, x, ...) { - # When called with SparkR::createDataFrame, sys.call()[[1]] returns c(::, SparkR, createDataFrame) - callsite <- as.character(sys.call(sys.parent())[[1]]) - funcName <- callsite[[length(callsite)]] - f <- get(paste0(funcName, ".default")) - # Strip sqlContext from list of parameters and then pass the rest along. - contextNames <- c("org.apache.spark.sql.SQLContext", - "org.apache.spark.sql.hive.HiveContext", - "org.apache.spark.sql.hive.test.TestHiveContext", - "org.apache.spark.sql.SparkSession") - if (missing(x) && length(list(...)) == 0) { - f() - } else if (class(x) == "jobj" && - any(grepl(paste(contextNames, collapse = "|"), getClassName.jobj(x)))) { - .Deprecated(newFuncSig, old = paste0(funcName, "(sqlContext...)")) - f(...) - } else { - f(x, ...) - } -} - #' return the SparkSession #' @noRd getSparkSession <- function() { @@ -178,6 +147,93 @@ getDefaultSqlSource <- function() { l[["spark.sql.sources.default"]] } +writeToFileInArrow <- function(fileName, rdf, numPartitions) { + requireNamespace1 <- requireNamespace + + # R API in Arrow is not yet released in CRAN. CRAN requires to add the + # package in requireNamespace at DESCRIPTION. Later, CRAN checks if the package is available + # or not. Therefore, it works around by avoiding direct requireNamespace. + # Currently, as of Arrow 0.12.0, it can be installed by install_github. See ARROW-3204. + if (requireNamespace1("arrow", quietly = TRUE)) { + record_batch <- get("record_batch", envir = asNamespace("arrow"), inherits = FALSE) + RecordBatchStreamWriter <- get( + "RecordBatchStreamWriter", envir = asNamespace("arrow"), inherits = FALSE) + FileOutputStream <- get( + "FileOutputStream", envir = asNamespace("arrow"), inherits = FALSE) + + numPartitions <- if (!is.null(numPartitions)) { + numToInt(numPartitions) + } else { + 1 + } + + rdf_slices <- if (numPartitions > 1) { + split(rdf, makeSplits(numPartitions, nrow(rdf))) + } else { + list(rdf) + } + + stream_writer <- NULL + tryCatch({ + for (rdf_slice in rdf_slices) { + batch <- record_batch(rdf_slice) + if (is.null(stream_writer)) { + stream <- FileOutputStream(fileName) + schema <- batch$schema + stream_writer <- RecordBatchStreamWriter(stream, schema) + } + + stream_writer$write_batch(batch) + } + }, + finally = { + if (!is.null(stream_writer)) { + stream_writer$close() + } + }) + + } else { + stop("'arrow' package should be installed.") + } +} + +getSchema <- function(schema, firstRow = NULL, rdd = NULL) { + if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { + if (is.null(firstRow)) { + stopifnot(!is.null(rdd)) + firstRow <- firstRDD(rdd) + } + names <- if (is.null(schema)) { + names(firstRow) + } else { + as.list(schema) + } + if (is.null(names)) { + names <- lapply(1:length(firstRow), function(x) { + paste0("_", as.character(x)) + }) + } + + # SPAKR-SQL does not support '.' in column name, so replace it with '_' + # TODO(davies): remove this once SPARK-2775 is fixed + names <- lapply(names, function(n) { + nn <- gsub("[.]", "_", n) + if (nn != n) { + warning(paste("Use", nn, "instead of", n, "as column name")) + } + nn + }) + + types <- lapply(firstRow, infer_type) + fields <- lapply(1:length(firstRow), function(i) { + structField(names[[i]], types[[i]], TRUE) + }) + schema <- do.call(structType, fields) + } else { + schema + } +} + #' Create a SparkDataFrame #' #' Converts R data.frame or list into SparkDataFrame. @@ -198,42 +254,80 @@ getDefaultSqlSource <- function() { #' df4 <- createDataFrame(cars, numPartitions = 2) #' } #' @name createDataFrame -#' @method createDataFrame default #' @note createDataFrame since 1.4.0 # TODO(davies): support sampling and infer type from NA -createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, - numPartitions = NULL) { +createDataFrame <- function(data, schema = NULL, samplingRatio = 1.0, + numPartitions = NULL) { sparkSession <- getSparkSession() + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] == "true" + useArrow <- FALSE + firstRow <- NULL if (is.data.frame(data)) { - # Convert data into a list of rows. Each row is a list. - - # get the names of columns, they will be put into RDD - if (is.null(schema)) { - schema <- names(data) - } + # get the names of columns, they will be put into RDD + if (is.null(schema)) { + schema <- names(data) + } - # get rid of factor type - cleanCols <- function(x) { - if (is.factor(x)) { - as.character(x) - } else { - x - } + # get rid of factor type + cleanCols <- function(x) { + if (is.factor(x)) { + as.character(x) + } else { + x } + } + data[] <- lapply(data, cleanCols) + + args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) + if (arrowEnabled) { + useArrow <- tryCatch({ + stopifnot(length(data) > 0) + firstRow <- do.call(mapply, append(args, head(data, 1)))[[1]] + schema <- getSchema(schema, firstRow = firstRow) + checkSchemaInArrow(schema) + fileName <- tempfile(pattern = "sparwriteToFileInArrowk-arrow", fileext = ".tmp") + tryCatch({ + writeToFileInArrow(fileName, data, numPartitions) + jrddInArrow <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "readArrowStreamFromFile", + sparkSession, + fileName) + }, + finally = { + # File might not be created. + suppressWarnings(file.remove(fileName)) + }) + TRUE + }, + error = function(e) { + warning(paste0("createDataFrame attempted Arrow optimization because ", + "'spark.sql.execution.arrow.sparkr.enabled' is set to true; however, ", + "failed, attempting non-optimization. Reason: ", + e)) + FALSE + }) + } + if (!useArrow) { + # Convert data into a list of rows. Each row is a list. # drop factors and wrap lists - data <- setNames(lapply(data, cleanCols), NULL) + data <- setNames(as.list(data), NULL) # check if all columns have supported type lapply(data, getInternalType) # convert to rows - args <- list(FUN = list, SIMPLIFY = FALSE, USE.NAMES = FALSE) data <- do.call(mapply, append(args, data)) + if (length(data) > 0) { + firstRow <- data[[1]] + } + } } - if (is.list(data)) { + if (useArrow) { + rdd <- jrddInArrow + } else if (is.list(data)) { sc <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", sparkSession) if (!is.null(numPartitions)) { rdd <- parallelize(sc, data, numSlices = numToInt(numPartitions)) @@ -246,70 +340,34 @@ createDataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, stop(paste("unexpected type:", class(data))) } - if (is.null(schema) || (!inherits(schema, "structType") && is.null(names(schema)))) { - row <- firstRDD(rdd) - names <- if (is.null(schema)) { - names(row) - } else { - as.list(schema) - } - if (is.null(names)) { - names <- lapply(1:length(row), function(x) { - paste("_", as.character(x), sep = "") - }) - } - - # SPAKR-SQL does not support '.' in column name, so replace it with '_' - # TODO(davies): remove this once SPARK-2775 is fixed - names <- lapply(names, function(n) { - nn <- gsub("[.]", "_", n) - if (nn != n) { - warning(paste("Use", nn, "instead of", n, " as column name")) - } - nn - }) - - types <- lapply(row, infer_type) - fields <- lapply(1:length(row), function(i) { - structField(names[[i]], types[[i]], TRUE) - }) - schema <- do.call(structType, fields) - } + schema <- getSchema(schema, firstRow, rdd) stopifnot(class(schema) == "structType") - jrdd <- getJRDD(lapply(rdd, function(x) x), "row") - srdd <- callJMethod(jrdd, "rdd") - sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", - srdd, schema$jobj, sparkSession) + if (useArrow) { + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "toDataFrame", rdd, schema$jobj, sparkSession) + } else { + jrdd <- getJRDD(lapply(rdd, function(x) x), "row") + srdd <- callJMethod(jrdd, "rdd") + sdf <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", "createDF", + srdd, schema$jobj, sparkSession) + } dataFrame(sdf) } -createDataFrame <- function(x, ...) { - dispatchFunc("createDataFrame(data, schema = NULL)", x, ...) -} - #' @rdname createDataFrame #' @aliases createDataFrame -#' @method as.DataFrame default #' @note as.DataFrame since 1.6.0 -as.DataFrame.default <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { +as.DataFrame <- function(data, schema = NULL, samplingRatio = 1.0, numPartitions = NULL) { createDataFrame(data, schema, samplingRatio, numPartitions) } -#' @param ... additional argument(s). -#' @rdname createDataFrame -#' @aliases as.DataFrame -as.DataFrame <- function(data, ...) { - dispatchFunc("as.DataFrame(data, schema = NULL)", data, ...) -} - #' toDF #' #' Converts an RDD to a SparkDataFrame by infer the types. #' #' @param x An RDD -#' #' @rdname SparkDataFrame #' @noRd #' @examples @@ -343,12 +401,10 @@ setMethod("toDF", signature(x = "RDD"), #' path <- "path/to/file.json" #' df <- read.json(path) #' df <- read.json(path, multiLine = TRUE) -#' df <- jsonFile(path) #' } #' @name read.json -#' @method read.json default #' @note read.json since 1.6.0 -read.json.default <- function(path, ...) { +read.json <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the text file path @@ -359,55 +415,6 @@ read.json.default <- function(path, ...) { dataFrame(sdf) } -read.json <- function(x, ...) { - dispatchFunc("read.json(path)", x, ...) -} - -#' @rdname read.json -#' @name jsonFile -#' @method jsonFile default -#' @note jsonFile since 1.4.0 -jsonFile.default <- function(path) { - .Deprecated("read.json") - read.json(path) -} - -jsonFile <- function(x, ...) { - dispatchFunc("jsonFile(path)", x, ...) -} - -#' JSON RDD -#' -#' Loads an RDD storing one JSON object per string as a SparkDataFrame. -#' -#' @param sqlContext SQLContext to use -#' @param rdd An RDD of JSON string -#' @param schema A StructType object to use as schema -#' @param samplingRatio The ratio of simpling used to infer the schema -#' @return A SparkDataFrame -#' @noRd -#' @examples -#'\dontrun{ -#' sparkR.session() -#' rdd <- texFile(sc, "path/to/json") -#' df <- jsonRDD(sqlContext, rdd) -#'} - -# TODO: remove - this method is no longer exported -# TODO: support schema -jsonRDD <- function(sqlContext, rdd, schema = NULL, samplingRatio = 1.0) { - .Deprecated("read.json") - rdd <- serializeToString(rdd) - if (is.null(schema)) { - read <- callJMethod(sqlContext, "read") - # samplingRatio is deprecated - sdf <- callJMethod(read, "json", callJMethod(getJRDD(rdd), "rdd")) - dataFrame(sdf) - } else { - stop("not implemented") - } -} - #' Create a SparkDataFrame from an ORC file. #' #' Loads an ORC file, returning the result as a SparkDataFrame. @@ -434,12 +441,12 @@ read.orc <- function(path, ...) { #' Loads a Parquet file, returning the result as a SparkDataFrame. #' #' @param path path of file to read. A vector of multiple paths is allowed. +#' @param ... additional data source specific named properties. #' @return SparkDataFrame #' @rdname read.parquet #' @name read.parquet -#' @method read.parquet default #' @note read.parquet since 1.6.0 -read.parquet.default <- function(path, ...) { +read.parquet <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the Parquet file path @@ -450,29 +457,11 @@ read.parquet.default <- function(path, ...) { dataFrame(sdf) } -read.parquet <- function(x, ...) { - dispatchFunc("read.parquet(...)", x, ...) -} - -#' @param ... argument(s) passed to the method. -#' @rdname read.parquet -#' @name parquetFile -#' @method parquetFile default -#' @note parquetFile since 1.4.0 -parquetFile.default <- function(...) { - .Deprecated("read.parquet") - read.parquet(unlist(list(...))) -} - -parquetFile <- function(x, ...) { - dispatchFunc("parquetFile(...)", x, ...) -} - #' Create a SparkDataFrame from a text file. #' #' Loads text files and returns a SparkDataFrame whose schema starts with #' a string column named "value", and followed by partitioned columns if -#' there are any. +#' there are any. The text files must be encoded as UTF-8. #' #' Each line in the text file is a new row in the resulting SparkDataFrame. #' @@ -487,9 +476,8 @@ parquetFile <- function(x, ...) { #' df <- read.text(path) #' } #' @name read.text -#' @method read.text default #' @note read.text since 1.6.1 -read.text.default <- function(path, ...) { +read.text <- function(path, ...) { sparkSession <- getSparkSession() options <- varargsToStrEnv(...) # Allow the user to have a more flexible definition of the text file path @@ -500,10 +488,6 @@ read.text.default <- function(path, ...) { dataFrame(sdf) } -read.text <- function(x, ...) { - dispatchFunc("read.text(path)", x, ...) -} - #' SQL Query #' #' Executes a SQL query using Spark, returning the result as a SparkDataFrame. @@ -520,18 +504,13 @@ read.text <- function(x, ...) { #' new_df <- sql("SELECT * FROM table") #' } #' @name sql -#' @method sql default #' @note sql since 1.4.0 -sql.default <- function(sqlQuery) { +sql <- function(sqlQuery) { sparkSession <- getSparkSession() sdf <- callJMethod(sparkSession, "sql", sqlQuery) dataFrame(sdf) } -sql <- function(x, ...) { - dispatchFunc("sql(sqlQuery)", x, ...) -} - #' Create a SparkDataFrame from a SparkSQL table or view #' #' Returns the specified table or view as a SparkDataFrame. The table or view must already exist or @@ -590,9 +569,8 @@ tableToDF <- function(tableName) { #' df4 <- read.df(mapTypeJsonPath, "json", stringSchema, multiLine = TRUE) #' } #' @name read.df -#' @method read.df default #' @note read.df since 1.4.0 -read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { +read.df <- function(path = NULL, source = NULL, schema = NULL, na.strings = "NA", ...) { if (!is.null(path) && !is.character(path)) { stop("path should be character, NULL or omitted.") } @@ -627,22 +605,13 @@ read.df.default <- function(path = NULL, source = NULL, schema = NULL, na.string dataFrame(sdf) } -read.df <- function(x = NULL, ...) { - dispatchFunc("read.df(path = NULL, source = NULL, schema = NULL, ...)", x, ...) -} - #' @rdname read.df #' @name loadDF -#' @method loadDF default #' @note loadDF since 1.6.0 -loadDF.default <- function(path = NULL, source = NULL, schema = NULL, ...) { +loadDF <- function(path = NULL, source = NULL, schema = NULL, ...) { read.df(path, source, schema, ...) } -loadDF <- function(x = NULL, ...) { - dispatchFunc("loadDF(path = NULL, source = NULL, schema = NULL, ...)", x, ...) -} - #' Create a SparkDataFrame representing the database table accessible via JDBC URL #' #' Additional JDBC database connection properties can be set (...) diff --git a/R/pkg/R/WindowSpec.R b/R/pkg/R/WindowSpec.R index ee7f4adf726e6..037809cd0923e 100644 --- a/R/pkg/R/WindowSpec.R +++ b/R/pkg/R/WindowSpec.R @@ -127,6 +127,16 @@ setMethod("orderBy", #' "0" means "current row", while "-1" means the row before the current row, and "5" means the #' fifth row after the current row. #' +#' We recommend users use \code{Window.unboundedPreceding}, \code{Window.unboundedFollowing}, +#' and \code{Window.currentRow} to specify special boundary values, rather than using long values +#' directly. +#' +#' A row based boundary is based on the position of the row within the partition. +#' An offset indicates the number of rows above or below the current row, the frame for the +#' current row starts or ends. For instance, given a row based sliding frame with a lower bound +#' offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from +#' index 4 to index 6. +#' #' @param x a WindowSpec #' @param start boundary start, inclusive. #' The frame is unbounded if this is the minimum long value. @@ -139,7 +149,14 @@ setMethod("orderBy", #' @family windowspec_method #' @examples #' \dontrun{ -#' rowsBetween(ws, 0, 3) +#' id <- c(rep(1, 3), rep(2, 3), 3) +#' desc <- c('New', 'New', 'Good', 'New', 'Good', 'Good', 'New') +#' df <- data.frame(id, desc) +#' df <- createDataFrame(df) +#' w1 <- orderBy(windowPartitionBy('desc'), df$id) +#' w2 <- rowsBetween(w1, 0, 3) +#' df1 <- withColumn(df, "sum", over(sum(df$id), w2)) +#' head(df1) #' } #' @note rowsBetween since 2.0.0 setMethod("rowsBetween", @@ -158,6 +175,19 @@ setMethod("rowsBetween", #' "current row", while "-1" means one off before the current row, and "5" means the five off #' after the current row. #' +#' We recommend users use \code{Window.unboundedPreceding}, \code{Window.unboundedFollowing}, +#' and \code{Window.currentRow} to specify special boundary values, rather than using long values +#' directly. +#' +#' A range-based boundary is based on the actual value of the ORDER BY +#' expression(s). An offset is used to alter the value of the ORDER BY expression, +#' for instance if the current ORDER BY expression has a value of 10 and the lower bound offset +#' is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a +#' number of constraints on the ORDER BY expressions: there can be only one expression and this +#' expression must have a numerical data type. An exception can be made when the offset is +#' unbounded, because no value modification is needed, in this case multiple and non-numeric +#' ORDER BY expression are allowed. +#' #' @param x a WindowSpec #' @param start boundary start, inclusive. #' The frame is unbounded if this is the minimum long value. @@ -170,7 +200,14 @@ setMethod("rowsBetween", #' @family windowspec_method #' @examples #' \dontrun{ -#' rangeBetween(ws, 0, 3) +#' id <- c(rep(1, 3), rep(2, 3), 3) +#' desc <- c('New', 'New', 'Good', 'New', 'Good', 'Good', 'New') +#' df <- data.frame(id, desc) +#' df <- createDataFrame(df) +#' w1 <- orderBy(windowPartitionBy('desc'), df$id) +#' w2 <- rangeBetween(w1, 0, 3) +#' df1 <- withColumn(df, "sum", over(sum(df$id), w2)) +#' head(df1) #' } #' @note rangeBetween since 2.0.0 setMethod("rangeBetween", diff --git a/R/pkg/R/catalog.R b/R/pkg/R/catalog.R index c2d0fc38786be..7641f8a7a0432 100644 --- a/R/pkg/R/catalog.R +++ b/R/pkg/R/catalog.R @@ -17,40 +17,6 @@ # 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 -#' @method createExternalTable default -#' @note createExternalTable since 1.4.0 -createExternalTable.default <- function(tableName, path = NULL, source = NULL, schema = NULL, ...) { - .Deprecated("createTable", old = "createExternalTable") - createTable(tableName, path, source, schema, ...) -} - -createExternalTable <- function(x, ...) { - dispatchFunc("createExternalTable(tableName, path = NULL, source = NULL, ...)", x, ...) -} - #' 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 @@ -116,18 +82,13 @@ createTable <- function(tableName, path = NULL, source = NULL, schema = NULL, .. #' cacheTable("table") #' } #' @name cacheTable -#' @method cacheTable default #' @note cacheTable since 1.4.0 -cacheTable.default <- function(tableName) { +cacheTable <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(handledCallJMethod(catalog, "cacheTable", tableName)) } -cacheTable <- function(x, ...) { - dispatchFunc("cacheTable(tableName)", x, ...) -} - #' Uncache Table #' #' Removes the specified table from the in-memory cache. @@ -145,18 +106,13 @@ cacheTable <- function(x, ...) { #' uncacheTable("table") #' } #' @name uncacheTable -#' @method uncacheTable default #' @note uncacheTable since 1.4.0 -uncacheTable.default <- function(tableName) { +uncacheTable <- function(tableName) { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(handledCallJMethod(catalog, "uncacheTable", tableName)) } -uncacheTable <- function(x, ...) { - dispatchFunc("uncacheTable(tableName)", x, ...) -} - #' Clear Cache #' #' Removes all cached tables from the in-memory cache. @@ -167,48 +123,13 @@ uncacheTable <- function(x, ...) { #' clearCache() #' } #' @name clearCache -#' @method clearCache default #' @note clearCache since 1.4.0 -clearCache.default <- function() { +clearCache <- function() { sparkSession <- getSparkSession() catalog <- callJMethod(sparkSession, "catalog") invisible(callJMethod(catalog, "clearCache")) } -clearCache <- function() { - dispatchFunc("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 -#' @method dropTempTable default -#' @note dropTempTable since 1.4.0 -dropTempTable.default <- function(tableName) { - .Deprecated("dropTempView", old = "dropTempTable") - if (class(tableName) != "character") { - stop("tableName must be a string.") - } - dropTempView(tableName) -} - -dropTempTable <- function(x, ...) { - dispatchFunc("dropTempView(viewName)", x, ...) -} - #' Drops the temporary view with the given view name in the catalog. #' #' Drops the temporary view with the given view name in the catalog. @@ -249,17 +170,12 @@ dropTempView <- function(viewName) { #' tables("hive") #' } #' @name tables -#' @method tables default #' @note tables since 1.4.0 -tables.default <- function(databaseName = NULL) { +tables <- function(databaseName = NULL) { # rename column to match previous output schema withColumnRenamed(listTables(databaseName), "name", "tableName") } -tables <- function(x, ...) { - dispatchFunc("tables(databaseName = NULL)", x, ...) -} - #' Table Names #' #' Returns the names of tables in the given database as an array. @@ -273,9 +189,8 @@ tables <- function(x, ...) { #' tableNames("hive") #' } #' @name tableNames -#' @method tableNames default #' @note tableNames since 1.4.0 -tableNames.default <- function(databaseName = NULL) { +tableNames <- function(databaseName = NULL) { sparkSession <- getSparkSession() callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getTableNames", @@ -283,10 +198,6 @@ tableNames.default <- function(databaseName = NULL) { databaseName) } -tableNames <- function(x, ...) { - dispatchFunc("tableNames(databaseName = NULL)", x, ...) -} - #' Returns the current default database #' #' Returns the current default database. diff --git a/R/pkg/R/context.R b/R/pkg/R/context.R index e99136723f65b..619153645d925 100644 --- a/R/pkg/R/context.R +++ b/R/pkg/R/context.R @@ -29,7 +29,7 @@ getMinPartitions <- function(sc, minPartitions) { #' #' This function reads a text file from HDFS, a local file system (available on all #' nodes), or any Hadoop-supported file system URI, and creates an -#' RDD of strings from it. +#' RDD of strings from it. The text files must be encoded as UTF-8. #' #' @param sc SparkContext to use #' @param path Path of file to read. A vector of multiple paths is allowed. @@ -81,13 +81,33 @@ objectFile <- function(sc, path, minPartitions = NULL) { RDD(jrdd, "byte") } +makeSplits <- function(numSerializedSlices, length) { + # Generate the slice ids to put each row + # For instance, for numSerializedSlices of 22, length of 50 + # [1] 0 0 2 2 4 4 6 6 6 9 9 11 11 13 13 15 15 15 18 18 20 20 22 22 22 + # [26] 25 25 27 27 29 29 31 31 31 34 34 36 36 38 38 40 40 40 43 43 45 45 47 47 47 + # Notice the slice group with 3 slices (ie. 6, 15, 22) are roughly evenly spaced. + # We are trying to reimplement the calculation in the positions method in ParallelCollectionRDD + if (numSerializedSlices > 0) { + unlist(lapply(0: (numSerializedSlices - 1), function(x) { + # nolint start + start <- trunc((as.numeric(x) * length) / numSerializedSlices) + end <- trunc(((as.numeric(x) + 1) * length) / numSerializedSlices) + # nolint end + rep(start, end - start) + })) + } else { + 1 + } +} + #' Create an RDD from a homogeneous list or vector. #' #' This function creates an RDD from a local homogeneous list in R. The elements #' in the list are split into \code{numSlices} slices and distributed to nodes #' in the cluster. #' -#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MB), the function +#' If size of serialized slices is larger than spark.r.maxAllocationLimit or (200MiB), the function #' will write it to disk and send the file name to JVM. Also to make sure each slice is not #' larger than that limit, number of slices may be increased. #' @@ -143,25 +163,7 @@ parallelize <- function(sc, coll, numSlices = 1) { # For large objects we make sure the size of each slice is also smaller than sizeLimit numSerializedSlices <- min(len, max(numSlices, ceiling(objectSize / sizeLimit))) - # Generate the slice ids to put each row - # For instance, for numSerializedSlices of 22, length of 50 - # [1] 0 0 2 2 4 4 6 6 6 9 9 11 11 13 13 15 15 15 18 18 20 20 22 22 22 - # [26] 25 25 27 27 29 29 31 31 31 34 34 36 36 38 38 40 40 40 43 43 45 45 47 47 47 - # Notice the slice group with 3 slices (ie. 6, 15, 22) are roughly evenly spaced. - # We are trying to reimplement the calculation in the positions method in ParallelCollectionRDD - splits <- if (numSerializedSlices > 0) { - unlist(lapply(0: (numSerializedSlices - 1), function(x) { - # nolint start - start <- trunc((as.numeric(x) * len) / numSerializedSlices) - end <- trunc(((as.numeric(x) + 1) * len) / numSerializedSlices) - # nolint end - rep(start, end - start) - })) - } else { - 1 - } - - slices <- split(coll, splits) + slices <- split(coll, makeSplits(numSerializedSlices, len)) # Serialize each slice: obtain a list of raws, or a list of lists (slices) of # 2-tuples of raws @@ -173,13 +175,15 @@ parallelize <- function(sc, coll, numSlices = 1) { if (objectSize < sizeLimit) { jrdd <- callJStatic("org.apache.spark.api.r.RRDD", "createRDDFromArray", sc, serializedSlices) } else { - if (callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc)) { + if (callJStatic("org.apache.spark.api.r.RUtils", "isEncryptionEnabled", sc)) { + connectionTimeout <- as.numeric(Sys.getenv("SPARKR_BACKEND_CONNECTION_TIMEOUT", "6000")) # the length of slices here is the parallelism to use in the jvm's sc.parallelize() parallelism <- as.integer(numSlices) jserver <- newJObject("org.apache.spark.api.r.RParallelizeServer", sc, parallelism) authSecret <- callJMethod(jserver, "secret") port <- callJMethod(jserver, "port") - conn <- socketConnection(port = port, blocking = TRUE, open = "wb", timeout = 1500) + conn <- socketConnection( + port = port, blocking = TRUE, open = "wb", timeout = connectionTimeout) doServerAuth(conn, authSecret) writeToConnection(serializedSlices, conn) jrdd <- callJMethod(jserver, "getResult") diff --git a/R/pkg/R/deserialize.R b/R/pkg/R/deserialize.R index cb03f1667629f..b38d245a0cca7 100644 --- a/R/pkg/R/deserialize.R +++ b/R/pkg/R/deserialize.R @@ -231,6 +231,44 @@ readMultipleObjectsWithKeys <- function(inputCon) { list(keys = keys, data = data) # this is a list of keys and corresponding data } +readDeserializeInArrow <- function(inputCon) { + # This is a hack to avoid CRAN check. Arrow is not uploaded into CRAN now. See ARROW-3204. + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + RecordBatchStreamReader <- get( + "RecordBatchStreamReader", envir = asNamespace("arrow"), inherits = FALSE) + # Arrow drops `as_tibble` since 0.14.0, see ARROW-5190. + useAsTibble <- exists("as_tibble", envir = asNamespace("arrow")) + + + # Currently, there looks no way to read batch by batch by socket connection in R side, + # See ARROW-4512. Therefore, it reads the whole Arrow streaming-formatted binary at once + # for now. + dataLen <- readInt(inputCon) + arrowData <- readBin(inputCon, raw(), as.integer(dataLen), endian = "big") + batches <- RecordBatchStreamReader(arrowData)$batches() + + if (useAsTibble) { + as_tibble <- get("as_tibble", envir = asNamespace("arrow")) + # Read all groupped batches. Tibble -> data.frame is cheap. + lapply(batches, function(batch) as.data.frame(as_tibble(batch))) + } else { + lapply(batches, function(batch) as.data.frame(batch)) + } + } else { + stop("'arrow' package should be installed.") + } +} + +readDeserializeWithKeysInArrow <- function(inputCon) { + data <- readDeserializeInArrow(inputCon) + + keys <- readMultipleObjects(inputCon) + + # Read keys to map with each groupped batch later. + list(keys = keys, data = data) +} + readRowList <- function(obj) { # readRowList is meant for use inside an lapply. As a result, it is # necessary to open a standalone connection for the row and consume diff --git a/R/pkg/R/functions.R b/R/pkg/R/functions.R index 6a8fef5aa7b22..3bd1f544d77a5 100644 --- a/R/pkg/R/functions.R +++ b/R/pkg/R/functions.R @@ -80,6 +80,11 @@ NULL #' \item \code{from_utc_timestamp}, \code{to_utc_timestamp}: time zone to use. #' \item \code{next_day}: day of the week string. #' } +#' @param ... additional argument(s). +#' \itemize{ +#' \item \code{months_between}, this contains an optional parameter to specify the +#' the result is rounded off to 8 digits. +#' } #' #' @name column_datetime_diff_functions #' @rdname column_datetime_diff_functions @@ -112,7 +117,7 @@ NULL #' df <- createDataFrame(cbind(model = rownames(mtcars), mtcars)) #' tmp <- mutate(df, v1 = log(df$mpg), v2 = cbrt(df$disp), #' v3 = bround(df$wt, 1), v4 = bin(df$cyl), -#' v5 = hex(df$wt), v6 = toDegrees(df$gear), +#' v5 = hex(df$wt), v6 = degrees(df$gear), #' v7 = atan2(df$cyl, df$am), v8 = hypot(df$cyl, df$am), #' v9 = pmod(df$hp, df$cyl), v10 = shiftLeft(df$disp, 1), #' v11 = conv(df$hp, 10, 16), v12 = sign(df$vs - 0.5), @@ -187,7 +192,9 @@ NULL #' \itemize{ #' \item \code{to_json}: it is the column containing the struct, array of the structs, #' the map or array of maps. +#' \item \code{to_csv}: it is the column containing the struct. #' \item \code{from_json}: it is the column containing the JSON string. +#' \item \code{from_csv}: it is the column containing the CSV string. #' } #' @param y Column to compute on. #' @param value A value to compute on. @@ -196,11 +203,27 @@ NULL #' \item \code{array_position}: a value to locate in the given array. #' \item \code{array_remove}: a value to remove in the given array. #' } -#' @param ... additional argument(s). In \code{to_json} and \code{from_json}, this contains -#' additional named properties to control how it is converted, accepts the same -#' options as the JSON data source. Additionally \code{to_json} supports the "pretty" -#' option which enables pretty JSON generation. In \code{arrays_zip}, this contains -#' additional Columns of arrays to be merged. +#' @param schema +#' \itemize{ +#' \item \code{from_json}: a structType object to use as the schema to use +#' when parsing the JSON string. Since Spark 2.3, the DDL-formatted string is +#' also supported for the schema. Since Spark 3.0, \code{schema_of_json} or +#' the DDL-formatted string literal can also be accepted. +#' \item \code{from_csv}: a structType object, DDL-formatted string or \code{schema_of_csv} +#' } +#' @param ... additional argument(s). +#' \itemize{ +#' \item \code{to_json}, \code{from_json} and \code{schema_of_json}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the JSON data source. +#' \item \code{to_json}: it supports the "pretty" option which enables pretty +#' JSON generation. +#' \item \code{to_csv}, \code{from_csv} and \code{schema_of_csv}: this contains +#' additional named properties to control how it is converted and accepts the +#' same options as the CSV data source. +#' \item \code{arrays_zip}, this contains additional Columns of arrays to be merged. +#' \item \code{map_concat}, this contains additional Columns of maps to be unioned. +#' } #' @name column_collection_functions #' @rdname column_collection_functions #' @family collection functions @@ -212,7 +235,7 @@ NULL #' head(select(tmp, array_contains(tmp$v1, 21), size(tmp$v1), shuffle(tmp$v1))) #' head(select(tmp, array_max(tmp$v1), array_min(tmp$v1), array_distinct(tmp$v1))) #' head(select(tmp, array_position(tmp$v1, 21), array_repeat(df$mpg, 3), array_sort(tmp$v1))) -#' head(select(tmp, flatten(tmp$v1), reverse(tmp$v1), array_remove(tmp$v1, 21))) +#' head(select(tmp, reverse(tmp$v1), array_remove(tmp$v1, 21))) #' tmp2 <- mutate(tmp, v2 = explode(tmp$v1)) #' head(tmp2) #' head(select(tmp, posexplode(tmp$v1))) @@ -221,15 +244,21 @@ NULL #' head(select(tmp, sort_array(tmp$v1, asc = FALSE))) #' tmp3 <- mutate(df, v3 = create_map(df$model, df$cyl)) #' head(select(tmp3, map_entries(tmp3$v3), map_keys(tmp3$v3), map_values(tmp3$v3))) -#' head(select(tmp3, element_at(tmp3$v3, "Valiant"))) +#' head(select(tmp3, element_at(tmp3$v3, "Valiant"), map_concat(tmp3$v3, tmp3$v3))) #' tmp4 <- mutate(df, v4 = create_array(df$mpg, df$cyl), v5 = create_array(df$cyl, df$hp)) #' head(select(tmp4, concat(tmp4$v4, tmp4$v5), arrays_overlap(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_except(tmp4$v4, tmp4$v5), array_intersect(tmp4$v4, tmp4$v5))) #' head(select(tmp4, array_union(tmp4$v4, tmp4$v5))) -#' head(select(tmp4, arrays_zip(tmp4$v4, tmp4$v5), map_from_arrays(tmp4$v4, tmp4$v5))) +#' head(select(tmp4, arrays_zip(tmp4$v4, tmp4$v5))) #' head(select(tmp, concat(df$mpg, df$cyl, df$hp))) #' tmp5 <- mutate(df, v6 = create_array(df$model, df$model)) -#' head(select(tmp5, array_join(tmp5$v6, "#"), array_join(tmp5$v6, "#", "NULL")))} +#' head(select(tmp5, array_join(tmp5$v6, "#"), array_join(tmp5$v6, "#", "NULL"))) +#' tmp6 <- mutate(df, v7 = create_array(create_array(df$model, df$model))) +#' head(select(tmp6, flatten(tmp6$v7))) +#' tmp7 <- mutate(df, v8 = create_array(df$model, df$cyl), v9 = create_array(df$model, df$hp)) +#' head(select(tmp7, map_from_arrays(tmp7$v8, tmp7$v9))) +#' tmp8 <- mutate(df, v10 = create_array(struct(df$model, df$cyl))) +#' head(select(tmp8, map_from_entries(tmp8$v10)))} NULL #' Window functions for Column operations @@ -311,23 +340,37 @@ setMethod("acos", }) #' @details -#' \code{approxCountDistinct}: Returns the approximate number of distinct items in a group. +#' \code{approx_count_distinct}: Returns the approximate number of distinct items in a group. #' #' @rdname column_aggregate_functions -#' @aliases approxCountDistinct approxCountDistinct,Column-method +#' @aliases approx_count_distinct approx_count_distinct,Column-method #' @examples #' #' \dontrun{ -#' head(select(df, approxCountDistinct(df$gear))) -#' head(select(df, approxCountDistinct(df$gear, 0.02))) +#' head(select(df, approx_count_distinct(df$gear))) +#' head(select(df, approx_count_distinct(df$gear, 0.02))) #' head(select(df, countDistinct(df$gear, df$cyl))) #' head(select(df, n_distinct(df$gear))) #' head(distinct(select(df, "gear")))} +#' @note approx_count_distinct(Column) since 3.0.0 +setMethod("approx_count_distinct", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "approx_count_distinct", x@jc) + column(jc) + }) + +#' @details +#' \code{approxCountDistinct}: Returns the approximate number of distinct items in a group. +#' +#' @rdname column_aggregate_functions +#' @aliases approxCountDistinct approxCountDistinct,Column-method #' @note approxCountDistinct(Column) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc) + .Deprecated("approx_count_distinct") + jc <- callJStatic("org.apache.spark.sql.functions", "approx_count_distinct", x@jc) column(jc) }) @@ -692,6 +735,25 @@ setMethod("hash", column(jc) }) +#' @details +#' \code{xxhash64}: Calculates the hash code of given columns using the 64-bit +#' variant of the xxHash algorithm, and returns the result as a long +#' column. +#' +#' @rdname column_misc_functions +#' @aliases xxhash64 xxhash64,Column-method +#' @note xxhash64 since 3.0.0 +setMethod("xxhash64", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function(x) { + stopifnot(class(x) == "Column") + x@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "xxhash64", jcols) + column(jc) + }) + #' @details #' \code{dayofmonth}: Extracts the day of the month as an integer from a #' given date/timestamp/string. @@ -1642,7 +1704,22 @@ setMethod("tanh", setMethod("toDegrees", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "toDegrees", x@jc) + .Deprecated("degrees") + jc <- callJStatic("org.apache.spark.sql.functions", "degrees", x@jc) + column(jc) + }) + +#' @details +#' \code{degrees}: Converts an angle measured in radians to an approximately equivalent angle +#' measured in degrees. +#' +#' @rdname column_math_functions +#' @aliases degrees degrees,Column-method +#' @note degrees since 3.0.0 +setMethod("degrees", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "degrees", x@jc) column(jc) }) @@ -1656,14 +1733,29 @@ setMethod("toDegrees", setMethod("toRadians", signature(x = "Column"), function(x) { - jc <- callJStatic("org.apache.spark.sql.functions", "toRadians", x@jc) + .Deprecated("radians") + jc <- callJStatic("org.apache.spark.sql.functions", "radians", x@jc) + column(jc) + }) + +#' @details +#' \code{radians}: Converts an angle measured in degrees to an approximately equivalent angle +#' measured in radians. +#' +#' @rdname column_math_functions +#' @aliases radians radians,Column-method +#' @note radians since 3.0.0 +setMethod("radians", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "radians", x@jc) column(jc) }) #' @details #' \code{to_date}: Converts the column into a DateType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' 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 @@ -1718,12 +1810,16 @@ setMethod("to_date", #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts a map into a JSON object -#' df2 <- sql("SELECT map('name', 'Bob')) as people") +#' df2 <- sql("SELECT map('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' #' # Converts an array of maps into a JSON array #' df2 <- sql("SELECT array(map('name', 'Bob'), map('name', 'Alice')) as people") -#' df2 <- mutate(df2, people_json = to_json(df2$people))} +#' df2 <- mutate(df2, people_json = to_json(df2$people)) +#' +#' # Converts a map into a pretty JSON object +#' df2 <- sql("SELECT map('name', 'Bob') as people") +#' df2 <- mutate(df2, people_json = to_json(df2$people, pretty = TRUE))} #' @note to_json since 2.2.0 setMethod("to_json", signature(x = "Column"), function(x, ...) { @@ -1732,10 +1828,30 @@ setMethod("to_json", signature(x = "Column"), column(jc) }) +#' @details +#' \code{to_csv}: Converts a column containing a \code{structType} into a Column of CSV string. +#' Resolving the Column can fail if an unsupported type is encountered. +#' +#' @rdname column_collection_functions +#' @aliases to_csv to_csv,Column-method +#' @examples +#' +#' \dontrun{ +#' # Converts a struct into a CSV string +#' df2 <- sql("SELECT named_struct('date', cast('2000-01-01' as date)) as d") +#' select(df2, to_csv(df2$d, dateFormat = 'dd/MM/yyyy'))} +#' @note to_csv since 3.0.0 +setMethod("to_csv", signature(x = "Column"), + function(x, ...) { + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", "to_csv", x@jc, options) + column(jc) + }) + #' @details #' \code{to_timestamp}: Converts the column into a TimestampType. You may optionally specify #' a format according to the rules in: -#' \url{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}. +#' \url{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}. #' 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 @@ -1989,15 +2105,21 @@ setMethod("levenshtein", signature(y = "Column"), #' are on the same day of month, or both are the last day of month, time of day will be ignored. #' Otherwise, the difference is calculated based on 31 days per month, and rounded to 8 digits. #' +#' @param roundOff an optional parameter to specify if the result is rounded off to 8 digits #' @rdname column_datetime_diff_functions #' @aliases months_between months_between,Column-method #' @note months_between since 1.5.0 setMethod("months_between", signature(y = "Column"), - function(y, x) { + function(y, x, roundOff = NULL) { if (class(x) == "Column") { x <- x@jc } - jc <- callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x) + jc <- if (is.null(roundOff)) { + callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x) + } else { + callJStatic("org.apache.spark.sql.functions", "months_between", y@jc, x, + as.logical(roundOff)) + } column(jc) }) @@ -2036,13 +2158,24 @@ setMethod("pmod", signature(y = "Column"), #' @param rsd maximum estimation error allowed (default = 0.05). #' +#' @rdname column_aggregate_functions +#' @aliases approx_count_distinct,Column-method +#' @note approx_count_distinct(Column, numeric) since 3.0.0 +setMethod("approx_count_distinct", + signature(x = "Column"), + function(x, rsd = 0.05) { + jc <- callJStatic("org.apache.spark.sql.functions", "approx_count_distinct", x@jc, rsd) + column(jc) + }) + #' @rdname column_aggregate_functions #' @aliases approxCountDistinct,Column-method #' @note approxCountDistinct(Column, numeric) since 1.4.0 setMethod("approxCountDistinct", signature(x = "Column"), function(x, rsd = 0.05) { - jc <- callJStatic("org.apache.spark.sql.functions", "approxCountDistinct", x@jc, rsd) + .Deprecated("approx_count_distinct") + jc <- callJStatic("org.apache.spark.sql.functions", "approx_count_distinct", x@jc, rsd) column(jc) }) @@ -2145,7 +2278,7 @@ setMethod("n", signature(x = "Column"), #' \code{date_format}: Converts a date/timestamp/string to a value of string in the format #' specified by the date format given by the second argument. A pattern could be for instance #' \code{dd.MM.yyyy} and could return a string like '18.03.1993'. All -#' pattern letters of \code{java.text.SimpleDateFormat} can be used. +#' pattern letters of \code{java.time.format.DateTimeFormatter} can be used. #' Note: Use when ever possible specialized functions like \code{year}. These benefit from a #' specialized implementation. #' @@ -2159,16 +2292,16 @@ setMethod("date_format", signature(y = "Column", x = "character"), column(jc) }) +setClassUnion("characterOrstructTypeOrColumn", c("character", "structType", "Column")) + #' @details #' \code{from_json}: Parses a column containing a JSON string into a Column of \code{structType} #' with the specified \code{schema} or array of \code{structType} if \code{as.json.array} is set #' to \code{TRUE}. If the string is unparseable, the Column will contain the value NA. #' #' @rdname column_collection_functions -#' @param schema a structType object to use as the schema to use when parsing the JSON string. -#' Since Spark 2.3, the DDL-formatted string is also supported for the schema. #' @param as.json.array indicating if input string is JSON array of objects or a single object. -#' @aliases from_json from_json,Column,characterOrstructType-method +#' @aliases from_json from_json,Column,characterOrstructTypeOrColumn-method #' @examples #' #' \dontrun{ @@ -2176,25 +2309,37 @@ setMethod("date_format", signature(y = "Column", x = "character"), #' df2 <- mutate(df2, d2 = to_json(df2$d, dateFormat = 'dd/MM/yyyy')) #' schema <- structType(structField("date", "string")) #' head(select(df2, from_json(df2$d2, schema, dateFormat = 'dd/MM/yyyy'))) - #' df2 <- sql("SELECT named_struct('name', 'Bob') as people") #' df2 <- mutate(df2, people_json = to_json(df2$people)) #' schema <- structType(structField("name", "string")) #' head(select(df2, from_json(df2$people_json, schema))) -#' head(select(df2, from_json(df2$people_json, "name STRING")))} +#' head(select(df2, from_json(df2$people_json, "name STRING"))) +#' head(select(df2, from_json(df2$people_json, schema_of_json(head(df2)$people_json))))} #' @note from_json since 2.2.0 -setMethod("from_json", signature(x = "Column", schema = "characterOrstructType"), +setMethod("from_json", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), function(x, schema, as.json.array = FALSE, ...) { if (is.character(schema)) { - schema <- structType(schema) + jschema <- structType(schema)$jobj + } else if (class(schema) == "structType") { + jschema <- schema$jobj + } else { + jschema <- schema@jc } if (as.json.array) { - jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", - "createArrayType", - schema$jobj) - } else { - jschema <- schema$jobj + # This case is R-specifically different. Unlike Scala and Python side, + # R side has 'as.json.array' option to indicate if the schema should be + # treated as struct or element type of array in order to make it more + # R-friendly. + if (class(schema) == "Column") { + jschema <- callJStatic("org.apache.spark.sql.api.r.SQLUtils", + "createArrayType", + jschema) + } else { + jschema <- callJStatic("org.apache.spark.sql.types.DataTypes", + "createArrayType", + jschema) + } } options <- varargsToStrEnv(...) jc <- callJStatic("org.apache.spark.sql.functions", @@ -2203,6 +2348,93 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") column(jc) }) +#' @details +#' \code{schema_of_json}: Parses a JSON string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_json schema_of_json,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' json <- "{\"name\":\"Bob\"}" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_json(json)))} +#' @note schema_of_json since 3.0.0 +setMethod("schema_of_json", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_json", + col, options) + column(jc) + }) + +#' @details +#' \code{from_csv}: Parses a column containing a CSV string into a Column of \code{structType} +#' with the specified \code{schema}. +#' If the string is unparseable, the Column will contain the value NA. +#' +#' @rdname column_collection_functions +#' @aliases from_csv from_csv,Column,characterOrstructTypeOrColumn-method +#' @examples +#' +#' \dontrun{ +#' csv <- "Amsterdam,2018" +#' df <- sql(paste0("SELECT '", csv, "' as csv")) +#' schema <- "city STRING, year INT" +#' head(select(df, from_csv(df$csv, schema))) +#' head(select(df, from_csv(df$csv, structType(schema)))) +#' head(select(df, from_csv(df$csv, schema_of_csv(csv))))} +#' @note from_csv since 3.0.0 +setMethod("from_csv", signature(x = "Column", schema = "characterOrstructTypeOrColumn"), + function(x, schema, ...) { + if (class(schema) == "structType") { + schema <- callJMethod(schema$jobj, "toDDL") + } + + if (is.character(schema)) { + jschema <- callJStatic("org.apache.spark.sql.functions", "lit", schema) + } else { + jschema <- schema@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "from_csv", + x@jc, jschema, options) + column(jc) + }) + +#' @details +#' \code{schema_of_csv}: Parses a CSV string and infers its schema in DDL format. +#' +#' @rdname column_collection_functions +#' @aliases schema_of_csv schema_of_csv,characterOrColumn-method +#' @examples +#' +#' \dontrun{ +#' csv <- "Amsterdam,2018" +#' df <- sql("SELECT * FROM range(1)") +#' head(select(df, schema_of_csv(csv)))} +#' @note schema_of_csv since 3.0.0 +setMethod("schema_of_csv", signature(x = "characterOrColumn"), + function(x, ...) { + if (class(x) == "character") { + col <- callJStatic("org.apache.spark.sql.functions", "lit", x) + } else { + col <- x@jc + } + options <- varargsToStrEnv(...) + jc <- callJStatic("org.apache.spark.sql.functions", + "schema_of_csv", + col, options) + column(jc) + }) + #' @details #' \code{from_utc_timestamp}: This is a common function for databases supporting TIMESTAMP WITHOUT #' TIMEZONE. This function takes a timestamp which is timezone-agnostic, and interprets it as a @@ -2227,6 +2459,7 @@ setMethod("from_json", signature(x = "Column", schema = "characterOrstructType") #' @note from_utc_timestamp since 1.5.0 setMethod("from_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { + .Deprecated(msg = "from_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "from_utc_timestamp", y@jc, x) column(jc) }) @@ -2285,6 +2518,7 @@ setMethod("next_day", signature(y = "Column", x = "character"), #' @note to_utc_timestamp since 1.5.0 setMethod("to_utc_timestamp", signature(y = "Column", x = "character"), function(y, x) { + .Deprecated(msg = "to_utc_timestamp is deprecated. See SPARK-25496.") jc <- callJStatic("org.apache.spark.sql.functions", "to_utc_timestamp", y@jc, x) column(jc) }) @@ -2491,7 +2725,7 @@ 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{http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html}{ +#' See \href{https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html}{ #' Customizing Formats} for available options. #' #' @rdname column_datetime_functions @@ -3253,12 +3487,29 @@ setMethod("flatten", column(jc) }) +#' @details +#' \code{map_concat}: Returns the union of all the given maps. +#' +#' @rdname column_collection_functions +#' @aliases map_concat map_concat,Column-method +#' @note map_concat since 3.0.0 +setMethod("map_concat", + signature(x = "Column"), + function(x, ...) { + jcols <- lapply(list(x, ...), function(arg) { + stopifnot(class(arg) == "Column") + arg@jc + }) + jc <- callJStatic("org.apache.spark.sql.functions", "map_concat", jcols) + column(jc) + }) + #' @details #' \code{map_entries}: Returns an unordered array of all entries in the given map. #' #' @rdname column_collection_functions #' @aliases map_entries map_entries,Column-method -#' @note map_entries since 2.4.0 +#' @note map_entries since 3.0.0 setMethod("map_entries", signature(x = "Column"), function(x) { @@ -3281,6 +3532,19 @@ setMethod("map_from_arrays", column(jc) }) +#' @details +#' \code{map_from_entries}: Returns a map created from the given array of entries. +#' +#' @rdname column_collection_functions +#' @aliases map_from_entries map_from_entries,Column-method +#' @note map_from_entries since 3.0.0 +setMethod("map_from_entries", + signature(x = "Column"), + function(x) { + jc <- callJStatic("org.apache.spark.sql.functions", "map_from_entries", x@jc) + column(jc) + }) + #' @details #' \code{map_keys}: Returns an unordered array containing the keys of the map. #' @@ -3325,6 +3589,8 @@ setMethod("element_at", #' @details #' \code{explode}: Creates a new row for each element in the given array or map column. +#' Uses the default column name \code{col} for elements in the array and +#' \code{key} and \code{value} for elements in the map unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases explode explode,Column-method @@ -3385,7 +3651,9 @@ setMethod("sort_array", #' @details #' \code{posexplode}: Creates a new row for each element with position in the given array -#' or map column. +#' or map column. Uses the default column name \code{pos} for position, and \code{col} +#' for elements in the array and \code{key} and \code{value} for elements in the map +#' unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases posexplode posexplode,Column-method @@ -3526,7 +3794,8 @@ setMethod("repeat_string", #' \code{explode}: Creates a new row for each element in the given array or map column. #' Unlike \code{explode}, if the array/map is \code{null} or empty #' then \code{null} is produced. -#' +#' Uses the default column name \code{col} for elements in the array and +#' \code{key} and \code{value} for elements in the map unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases explode_outer explode_outer,Column-method @@ -3551,6 +3820,9 @@ setMethod("explode_outer", #' \code{posexplode_outer}: Creates a new row for each element with position in the given #' array or map column. Unlike \code{posexplode}, if the array/map is \code{null} or empty #' then the row (\code{null}, \code{null}) is produced. +#' Uses the default column name \code{pos} for position, and \code{col} +#' for elements in the array and \code{key} and \code{value} for elements in the map +#' unless specified otherwise. #' #' @rdname column_collection_functions #' @aliases posexplode_outer posexplode_outer,Column-method diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R index 697d124095a75..f849dd172247c 100644 --- a/R/pkg/R/generics.R +++ b/R/pkg/R/generics.R @@ -528,9 +528,6 @@ 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") }) @@ -595,9 +592,6 @@ setGeneric("write.parquet", function(x, path, ...) { standardGeneric("write.parquet") }) -#' @rdname write.parquet -setGeneric("saveAsParquetFile", function(x, path) { standardGeneric("saveAsParquetFile") }) - #' @rdname write.stream setGeneric("write.stream", function(df, source = NULL, outputMode = NULL, ...) { standardGeneric("write.stream") @@ -637,7 +631,7 @@ setGeneric("toRDD", function(x) { standardGeneric("toRDD") }) #' @rdname union setGeneric("union", function(x, y) { standardGeneric("union") }) -#' @rdname union +#' @rdname unionAll setGeneric("unionAll", function(x, y) { standardGeneric("unionAll") }) #' @rdname unionByName @@ -755,6 +749,10 @@ setGeneric("windowOrderBy", function(col, ...) { standardGeneric("windowOrderBy" #' @name NULL setGeneric("add_months", function(y, x) { standardGeneric("add_months") }) +#' @rdname column_aggregate_functions +#' @name NULL +setGeneric("approx_count_distinct", function(x, ...) { standardGeneric("approx_count_distinct") }) + #' @rdname column_aggregate_functions #' @name NULL setGeneric("approxCountDistinct", function(x, ...) { standardGeneric("approxCountDistinct") }) @@ -984,6 +982,10 @@ setGeneric("format_string", function(format, x, ...) { standardGeneric("format_s #' @name NULL setGeneric("from_json", function(x, schema, ...) { standardGeneric("from_json") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("from_csv", function(x, schema, ...) { standardGeneric("from_csv") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("from_unixtime", function(x, ...) { standardGeneric("from_unixtime") }) @@ -1076,6 +1078,10 @@ setGeneric("lpad", function(x, len, pad) { standardGeneric("lpad") }) #' @name NULL setGeneric("ltrim", function(x, trimString) { standardGeneric("ltrim") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_concat", function(x, ...) { standardGeneric("map_concat") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) @@ -1084,6 +1090,10 @@ setGeneric("map_entries", function(x) { standardGeneric("map_entries") }) #' @name NULL setGeneric("map_from_arrays", function(x, y) { standardGeneric("map_from_arrays") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("map_from_entries", function(x) { standardGeneric("map_from_entries") }) + #' @rdname column_collection_functions #' @name NULL setGeneric("map_keys", function(x) { standardGeneric("map_keys") }) @@ -1111,7 +1121,7 @@ setGeneric("month", function(x) { standardGeneric("month") }) #' @rdname column_datetime_diff_functions #' @name NULL -setGeneric("months_between", function(y, x) { standardGeneric("months_between") }) +setGeneric("months_between", function(y, x, ...) { standardGeneric("months_between") }) #' @rdname count setGeneric("n", function(x) { standardGeneric("n") }) @@ -1204,6 +1214,14 @@ setGeneric("rpad", function(x, len, pad) { standardGeneric("rpad") }) #' @name NULL setGeneric("rtrim", function(x, trimString) { standardGeneric("rtrim") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_csv", function(x, ...) { standardGeneric("schema_of_csv") }) + +#' @rdname column_collection_functions +#' @name NULL +setGeneric("schema_of_json", function(x, ...) { standardGeneric("schema_of_json") }) + #' @rdname column_aggregate_functions #' @name NULL setGeneric("sd", function(x, na.rm = FALSE) { standardGeneric("sd") }) @@ -1292,10 +1310,18 @@ setGeneric("substring_index", function(x, delim, count) { standardGeneric("subst #' @name NULL setGeneric("sumDistinct", function(x) { standardGeneric("sumDistinct") }) +#' @rdname column_math_functions +#' @name NULL +setGeneric("degrees", function(x) { standardGeneric("degrees") }) + #' @rdname column_math_functions #' @name NULL setGeneric("toDegrees", function(x) { standardGeneric("toDegrees") }) +#' @rdname column_math_functions +#' @name NULL +setGeneric("radians", function(x) { standardGeneric("radians") }) + #' @rdname column_math_functions #' @name NULL setGeneric("toRadians", function(x) { standardGeneric("toRadians") }) @@ -1308,6 +1334,10 @@ setGeneric("to_date", function(x, format) { standardGeneric("to_date") }) #' @name NULL setGeneric("to_json", function(x, ...) { standardGeneric("to_json") }) +#' @rdname column_collection_functions +#' @name NULL +setGeneric("to_csv", function(x, ...) { standardGeneric("to_csv") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("to_timestamp", function(x, format) { standardGeneric("to_timestamp") }) @@ -1364,6 +1394,10 @@ setGeneric("weekofyear", function(x) { standardGeneric("weekofyear") }) #' @name NULL setGeneric("window", function(x, ...) { standardGeneric("window") }) +#' @rdname column_misc_functions +#' @name NULL +setGeneric("xxhash64", function(x, ...) { standardGeneric("xxhash64") }) + #' @rdname column_datetime_functions #' @name NULL setGeneric("year", function(x) { standardGeneric("year") }) @@ -1453,6 +1487,14 @@ setGeneric("spark.freqItemsets", function(object) { standardGeneric("spark.freqI #' @rdname spark.fpGrowth setGeneric("spark.associationRules", function(object) { standardGeneric("spark.associationRules") }) +#' @rdname spark.prefixSpan +setGeneric("spark.findFrequentSequentialPatterns", + function(data, ...) { standardGeneric("spark.findFrequentSequentialPatterns") }) + +#' @rdname spark.powerIterationClustering +setGeneric("spark.assignClusters", + function(data, ...) { standardGeneric("spark.assignClusters") }) + #' @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/group.R b/R/pkg/R/group.R index f751b952f3915..6e8f4dc3a7907 100644 --- a/R/pkg/R/group.R +++ b/R/pkg/R/group.R @@ -229,6 +229,18 @@ gapplyInternal <- function(x, func, schema) { if (is.character(schema)) { schema <- structType(schema) } + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] == "true" + if (arrowEnabled) { + if (inherits(schema, "structType")) { + checkSchemaInArrow(schema) + } else if (is.null(schema)) { + stop(paste0("Arrow optimization does not support 'gapplyCollect' yet. Please disable ", + "Arrow optimization or use 'collect' and 'gapply' APIs instead.")) + } else { + stop("'schema' should be DDL-formatted string or structType.") + } + } + packageNamesArr <- serialize(.sparkREnv[[".packages"]], connection = NULL) broadcastArr <- lapply(ls(.broadcastNames), diff --git a/R/pkg/R/mllib_classification.R b/R/pkg/R/mllib_classification.R index 2964fdeff0957..3ad824e1e634a 100644 --- a/R/pkg/R/mllib_classification.R +++ b/R/pkg/R/mllib_classification.R @@ -50,7 +50,7 @@ setClass("NaiveBayesModel", representation(jobj = "jobj")) #' #' @param data SparkDataFrame for training. #' @param formula A symbolic description of the model to be fitted. Currently only a few formula -#' operators are supported, including '~', '.', ':', '+', and '-'. +#' operators are supported, including '~', '.', ':', '+', '-', '*', and '^'. #' @param regParam The regularization parameter. Only supports L2 regularization currently. #' @param maxIter Maximum iteration number. #' @param tol Convergence tolerance of iterations. diff --git a/R/pkg/R/mllib_clustering.R b/R/pkg/R/mllib_clustering.R index 900be685824da..8bc15353465d8 100644 --- a/R/pkg/R/mllib_clustering.R +++ b/R/pkg/R/mllib_clustering.R @@ -41,6 +41,12 @@ setClass("KMeansModel", representation(jobj = "jobj")) #' @note LDAModel since 2.1.0 setClass("LDAModel", representation(jobj = "jobj")) +#' S4 class that represents a PowerIterationClustering +#' +#' @param jobj a Java object reference to the backing Scala PowerIterationClustering +#' @note PowerIterationClustering since 3.0.0 +setClass("PowerIterationClustering", slots = list(jobj = "jobj")) + #' Bisecting K-Means Clustering Model #' #' Fits a bisecting k-means clustering model against a SparkDataFrame. @@ -49,7 +55,7 @@ setClass("LDAModel", representation(jobj = "jobj")) #' #' @param data a SparkDataFrame for training. #' @param formula a symbolic description of the model to be fitted. Currently only a few formula -#' operators are supported, including '~', '.', ':', '+', and '-'. +#' operators are supported, including '~', '.', ':', '+', '-', '*', and '^'. #' Note that the response variable of formula is empty in spark.bisectingKmeans. #' @param k the desired number of leaf clusters. Must be > 1. #' The actual number could be smaller if there are no divisible leaf clusters. @@ -610,3 +616,56 @@ setMethod("write.ml", signature(object = "LDAModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' PowerIterationClustering +#' +#' A scalable graph clustering algorithm. Users can call \code{spark.assignClusters} to +#' return a cluster assignment for each input vertex. +#' Run the PIC algorithm and returns a cluster assignment for each input vertex. +#' @param data a SparkDataFrame. +#' @param k the number of clusters to create. +#' @param initMode the initialization algorithm; "random" or "degree" +#' @param maxIter the maximum number of iterations. +#' @param sourceCol the name of the input column for source vertex IDs. +#' @param destinationCol the name of the input column for destination vertex IDs +#' @param weightCol weight column name. If this is not set or \code{NULL}, +#' we treat all instance weights as 1.0. +#' @param ... additional argument(s) passed to the method. +#' @return A dataset that contains columns of vertex id and the corresponding cluster for the id. +#' The schema of it will be: \code{id: integer}, \code{cluster: integer} +#' @rdname spark.powerIterationClustering +#' @aliases spark.assignClusters,SparkDataFrame-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), +#' list(1L, 2L, 1.0), list(3L, 4L, 1.0), +#' list(4L, 0L, 0.1)), +#' schema = c("src", "dst", "weight")) +#' clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") +#' showDF(clusters) +#' } +#' @note spark.assignClusters(SparkDataFrame) since 3.0.0 +setMethod("spark.assignClusters", + signature(data = "SparkDataFrame"), + function(data, k = 2L, initMode = c("random", "degree"), maxIter = 20L, + sourceCol = "src", destinationCol = "dst", weightCol = NULL) { + if (!is.integer(k) || k < 1) { + stop("k should be a number with value >= 1.") + } + if (!is.integer(maxIter) || maxIter <= 0) { + stop("maxIter should be a number with value > 0.") + } + initMode <- match.arg(initMode) + if (!is.null(weightCol) && weightCol == "") { + weightCol <- NULL + } else if (!is.null(weightCol)) { + weightCol <- as.character(weightCol) + } + jobj <- callJStatic("org.apache.spark.ml.r.PowerIterationClusteringWrapper", + "getPowerIterationClustering", + as.integer(k), initMode, + as.integer(maxIter), as.character(sourceCol), + as.character(destinationCol), weightCol) + object <- new("PowerIterationClustering", jobj = jobj) + dataFrame(callJMethod(object@jobj, "assignClusters", data@sdf)) + }) diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R index 4ad34fe82328f..0cc7a16c302dc 100644 --- a/R/pkg/R/mllib_fpm.R +++ b/R/pkg/R/mllib_fpm.R @@ -23,6 +23,12 @@ #' @note FPGrowthModel since 2.2.0 setClass("FPGrowthModel", slots = list(jobj = "jobj")) +#' S4 class that represents a PrefixSpan +#' +#' @param jobj a Java object reference to the backing Scala PrefixSpan +#' @note PrefixSpan since 3.0.0 +setClass("PrefixSpan", slots = list(jobj = "jobj")) + #' FP-growth #' #' A parallel FP-growth algorithm to mine frequent itemsets. @@ -155,3 +161,62 @@ setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"), function(object, path, overwrite = FALSE) { write_internal(object, path, overwrite) }) + +#' PrefixSpan +#' +#' A parallel PrefixSpan algorithm to mine frequent sequential patterns. +#' \code{spark.findFrequentSequentialPatterns} returns a complete set of frequent sequential +#' patterns. +#' For more details, see +#' \href{https://spark.apache.org/docs/latest/mllib-frequent-pattern-mining.html#prefixspan}{ +#' PrefixSpan}. +#' +# Find frequent sequential patterns. +#' @param data A SparkDataFrame. +#' @param minSupport Minimal support level. +#' @param maxPatternLength Maximal pattern length. +#' @param maxLocalProjDBSize Maximum number of items (including delimiters used in the internal +#' storage format) allowed in a projected database before local +#' processing. +#' @param sequenceCol name of the sequence column in dataset. +#' @param ... additional argument(s) passed to the method. +#' @return A complete set of frequent sequential patterns in the input sequences of itemsets. +#' The returned \code{SparkDataFrame} contains columns of sequence and corresponding +#' frequency. The schema of it will be: +#' \code{sequence: ArrayType(ArrayType(T))}, \code{freq: integer} +#' where T is the item type +#' @rdname spark.prefixSpan +#' @aliases findFrequentSequentialPatterns,PrefixSpan,SparkDataFrame-method +#' @examples +#' \dontrun{ +#' df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), +#' list(list(list(1L), list(3L, 2L), list(1L, 2L))), +#' list(list(list(1L, 2L), list(5L))), +#' list(list(list(6L)))), +#' schema = c("sequence")) +#' frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, +#' maxLocalProjDBSize = 32000000L) +#' showDF(frequency) +#' } +#' @note spark.findFrequentSequentialPatterns(SparkDataFrame) since 3.0.0 +setMethod("spark.findFrequentSequentialPatterns", + signature(data = "SparkDataFrame"), + function(data, minSupport = 0.1, maxPatternLength = 10L, + maxLocalProjDBSize = 32000000L, sequenceCol = "sequence") { + if (!is.numeric(minSupport) || minSupport < 0) { + stop("minSupport should be a number with value >= 0.") + } + if (!is.integer(maxPatternLength) || maxPatternLength <= 0) { + stop("maxPatternLength should be a number with value > 0.") + } + if (!is.numeric(maxLocalProjDBSize) || maxLocalProjDBSize <= 0) { + stop("maxLocalProjDBSize should be a number with value > 0.") + } + + jobj <- callJStatic("org.apache.spark.ml.r.PrefixSpanWrapper", "getPrefixSpan", + as.numeric(minSupport), as.integer(maxPatternLength), + as.numeric(maxLocalProjDBSize), as.character(sequenceCol)) + object <- new("PrefixSpan", jobj = jobj) + dataFrame(callJMethod(object@jobj, "findFrequentSequentialPatterns", data@sdf)) + } + ) diff --git a/R/pkg/R/mllib_regression.R b/R/pkg/R/mllib_regression.R index 95c1a29905197..4fabe9a0068fc 100644 --- a/R/pkg/R/mllib_regression.R +++ b/R/pkg/R/mllib_regression.R @@ -44,7 +44,7 @@ setClass("IsotonicRegressionModel", representation(jobj = "jobj")) #' #' @param data a SparkDataFrame for training. #' @param formula a symbolic description of the model to be fitted. Currently only a few formula -#' operators are supported, including '~', '.', ':', '+', and '-'. +#' operators are supported, including '~', '.', ':', '+', '-', '*', and '^'. #' @param family a description of the error distribution and link function to be used in the model. #' This can be a character string naming a family function, a family function or #' the result of a call to a family function. Refer R family at diff --git a/R/pkg/R/mllib_tree.R b/R/pkg/R/mllib_tree.R index 0e60842dd44c8..ff16b436217dc 100644 --- a/R/pkg/R/mllib_tree.R +++ b/R/pkg/R/mllib_tree.R @@ -135,7 +135,7 @@ print.summary.decisionTree <- function(x) { #' #' @param data a SparkDataFrame for training. #' @param formula a symbolic description of the model to be fitted. Currently only a few formula -#' operators are supported, including '~', ':', '+', and '-'. +#' operators are supported, including '~', ':', '+', '-', '*', and '^'. #' @param type type of model, one of "regression" or "classification", to fit #' @param maxDepth Maximum depth of the tree (>= 0). #' @param maxBins Maximum number of bins used for discretizing continuous features and for choosing @@ -157,7 +157,7 @@ print.summary.decisionTree <- function(x) { #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -382,7 +382,7 @@ setMethod("write.ml", signature(object = "GBTClassificationModel", path = "chara #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the @@ -588,7 +588,7 @@ setMethod("write.ml", signature(object = "RandomForestClassificationModel", path #' @param checkpointInterval Param for set checkpoint interval (>= 1) or disable checkpoint (-1). #' Note: this setting will be ignored if the checkpoint directory is not #' set. -#' @param maxMemoryInMB Maximum memory in MB allocated to histogram aggregation. +#' @param maxMemoryInMB Maximum memory in MiB allocated to histogram aggregation. #' @param cacheNodeIds If FALSE, the algorithm will pass trees to executors to match instances with #' nodes. If TRUE, the algorithm will cache node IDs for each instance. Caching #' can speed up training of deeper trees. Users can set how often should the diff --git a/R/pkg/R/serialize.R b/R/pkg/R/serialize.R index 263b9b576c0c5..0d6f32c8f7e1f 100644 --- a/R/pkg/R/serialize.R +++ b/R/pkg/R/serialize.R @@ -220,3 +220,18 @@ writeArgs <- function(con, args) { } } } + +writeSerializeInArrow <- function(conn, df) { + # This is a hack to avoid CRAN check. Arrow is not uploaded into CRAN now. See ARROW-3204. + requireNamespace1 <- requireNamespace + if (requireNamespace1("arrow", quietly = TRUE)) { + write_arrow <- get("write_arrow", envir = asNamespace("arrow"), inherits = FALSE) + + # There looks no way to send each batch in streaming format via socket + # connection. See ARROW-4512. + # So, it writes the whole Arrow streaming-formatted binary at once for now. + writeRaw(conn, write_arrow(df, raw())) + } else { + stop("'arrow' package should be installed.") + } +} diff --git a/R/pkg/R/sparkR.R b/R/pkg/R/sparkR.R index 038fefadaaeff..31b986c326d0c 100644 --- a/R/pkg/R/sparkR.R +++ b/R/pkg/R/sparkR.R @@ -88,49 +88,6 @@ 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 = "", @@ -272,61 +229,6 @@ 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 @@ -367,7 +269,7 @@ sparkRHive.init <- function(jsc = NULL) { #' sparkR.session("yarn-client", "SparkR", "/home/spark", #' list(spark.executor.memory="4g"), #' c("one.jar", "two.jar", "three.jar"), -#' c("com.databricks:spark-avro_2.11:2.0.1")) +#' c("com.databricks:spark-avro_2.12:2.0.1")) #' sparkR.session(spark.master = "yarn-client", spark.executor.memory = "4g") #'} #' @note sparkR.session since 2.0.0 @@ -482,26 +384,11 @@ sparkR.uiWebUrl <- function() { #' setJobGroup("myJobGroup", "My job group description", TRUE) #'} #' @note setJobGroup since 1.5.0 -#' @method setJobGroup default -setJobGroup.default <- function(groupId, description, interruptOnCancel) { +setJobGroup <- function(groupId, description, interruptOnCancel) { sc <- getSparkContext() invisible(callJMethod(sc, "setJobGroup", groupId, description, interruptOnCancel)) } -setJobGroup <- function(sc, groupId, description, interruptOnCancel) { - if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { - .Deprecated("setJobGroup(groupId, description, interruptOnCancel)", - old = "setJobGroup(sc, groupId, description, interruptOnCancel)") - setJobGroup.default(groupId, description, interruptOnCancel) - } else { - # Parameter order is shifted - groupIdToUse <- sc - descriptionToUse <- groupId - interruptOnCancelToUse <- description - setJobGroup.default(groupIdToUse, descriptionToUse, interruptOnCancelToUse) - } -} - #' Clear current job group ID and its description #' #' @rdname clearJobGroup @@ -512,22 +399,11 @@ setJobGroup <- function(sc, groupId, description, interruptOnCancel) { #' clearJobGroup() #'} #' @note clearJobGroup since 1.5.0 -#' @method clearJobGroup default -clearJobGroup.default <- function() { +clearJobGroup <- function() { sc <- getSparkContext() invisible(callJMethod(sc, "clearJobGroup")) } -clearJobGroup <- function(sc) { - if (!missing(sc) && - class(sc) == "jobj" && - any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { - .Deprecated("clearJobGroup()", old = "clearJobGroup(sc)") - } - clearJobGroup.default() -} - - #' Cancel active jobs for the specified group #' #' @param groupId the ID of job group to be cancelled @@ -539,23 +415,11 @@ clearJobGroup <- function(sc) { #' cancelJobGroup("myJobGroup") #'} #' @note cancelJobGroup since 1.5.0 -#' @method cancelJobGroup default -cancelJobGroup.default <- function(groupId) { +cancelJobGroup <- function(groupId) { sc <- getSparkContext() invisible(callJMethod(sc, "cancelJobGroup", groupId)) } -cancelJobGroup <- function(sc, groupId) { - if (class(sc) == "jobj" && any(grepl("JavaSparkContext", getClassName.jobj(sc)))) { - .Deprecated("cancelJobGroup(groupId)", old = "cancelJobGroup(sc, groupId)") - cancelJobGroup.default(groupId) - } else { - # Parameter order is shifted - groupIdToUse <- sc - cancelJobGroup.default(groupIdToUse) - } -} - #' Set a human readable description of the current job. #' #' Set a description that is shown as a job description in UI. diff --git a/R/pkg/R/stats.R b/R/pkg/R/stats.R index 497f18c763048..7252351ebebb2 100644 --- a/R/pkg/R/stats.R +++ b/R/pkg/R/stats.R @@ -109,7 +109,7 @@ setMethod("corr", #' #' Finding frequent items for columns, possibly with false positives. #' Using the frequent element count algorithm described in -#' \url{http://dx.doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. +#' \url{https://doi.org/10.1145/762471.762473}, proposed by Karp, Schenker, and Papadimitriou. #' #' @param x A SparkDataFrame. #' @param cols A vector column names to search frequent items in. @@ -143,7 +143,7 @@ setMethod("freqItems", signature(x = "SparkDataFrame", cols = "character"), #' *exact* rank of x is close to (p * N). More precisely, #' floor((p - err) * N) <= rank(x) <= ceil((p + err) * N). #' This method implements a variation of the Greenwald-Khanna algorithm (with some speed -#' optimizations). The algorithm was first present in [[http://dx.doi.org/10.1145/375663.375670 +#' optimizations). The algorithm was first present in [[https://doi.org/10.1145/375663.375670 #' Space-efficient Online Computation of Quantile Summaries]] by Greenwald and Khanna. #' Note that NA values will be ignored in numerical columns before calculation. For #' columns only containing NA values, an empty list is returned. diff --git a/R/pkg/R/types.R b/R/pkg/R/types.R index ade0f05c02542..55f75508e88ff 100644 --- a/R/pkg/R/types.R +++ b/R/pkg/R/types.R @@ -83,3 +83,38 @@ specialtypeshandle <- function(type) { } returntype } + +# Helper function that checks supported types in Arrow. +checkSchemaInArrow <- function(schema) { + stopifnot(inherits(schema, "structType")) + + requireNamespace1 <- requireNamespace + if (!requireNamespace1("arrow", quietly = TRUE)) { + stop("'arrow' package should be installed.") + } + + # Both cases below produce a corrupt value for unknown reason. It needs to be investigated. + if (any(sapply(schema$fields(), function(x) x$dataType.toString() == "FloatType"))) { + stop("Arrow optimization in R does not support float type yet.") + } + if (any(sapply(schema$fields(), function(x) x$dataType.toString() == "BinaryType"))) { + stop("Arrow optimization in R does not support binary type yet.") + } + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "ArrayType")))) { + stop("Arrow optimization in R does not support array type yet.") + } + + # Arrow optimization in Spark does not yet support both cases below. + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "StructType")))) { + stop("Arrow optimization in R does not support nested struct type yet.") + } + if (any(sapply(schema$fields(), + function(x) startsWith(x$dataType.toString(), + "MapType")))) { + stop("Arrow optimization in R does not support map type yet.") + } +} diff --git a/R/pkg/inst/profile/general.R b/R/pkg/inst/profile/general.R index 8c75c19ca7ac3..3efb460846fc2 100644 --- a/R/pkg/inst/profile/general.R +++ b/R/pkg/inst/profile/general.R @@ -16,6 +16,10 @@ # .First <- function() { + if (utils::compareVersion(paste0(R.version$major, ".", R.version$minor), "3.4.0") == -1) { + warning("Support for R prior to version 3.4 is deprecated since Spark 3.0.0") + } + packageDir <- Sys.getenv("SPARKR_PACKAGE_DIR") dirs <- strsplit(packageDir, ",")[[1]] .libPaths(c(dirs, .libPaths())) diff --git a/R/pkg/inst/profile/shell.R b/R/pkg/inst/profile/shell.R index 8a8111a8c5419..e4e0d032997de 100644 --- a/R/pkg/inst/profile/shell.R +++ b/R/pkg/inst/profile/shell.R @@ -16,6 +16,10 @@ # .First <- function() { + if (utils::compareVersion(paste0(R.version$major, ".", R.version$minor), "3.4.0") == -1) { + warning("Support for R prior to version 3.4 is deprecated since Spark 3.0.0") + } + home <- Sys.getenv("SPARK_HOME") .libPaths(c(file.path(home, "R", "lib"), .libPaths())) Sys.setenv(NOAWT = 1) @@ -29,19 +33,19 @@ sc <- SparkR:::callJStatic("org.apache.spark.sql.api.r.SQLUtils", "getJavaSparkContext", spark) assign("sc", sc, envir = .GlobalEnv) sparkVer <- SparkR:::callJMethod(sc, "version") - cat("\n Welcome to") + cat("\nWelcome to") cat("\n") - cat(" ____ __", "\n") - cat(" / __/__ ___ _____/ /__", "\n") - cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") - cat(" /___/ .__/\\_,_/_/ /_/\\_\\") + cat(" ____ __", "\n") + cat(" / __/__ ___ _____/ /__", "\n") + cat(" _\\ \\/ _ \\/ _ `/ __/ '_/", "\n") + cat(" /___/ .__/\\_,_/_/ /_/\\_\\") if (nchar(sparkVer) == 0) { cat("\n") } else { - cat(" version ", sparkVer, "\n") + cat(" version", sparkVer, "\n") } - cat(" /_/", "\n") + cat(" /_/", "\n") cat("\n") - cat("\n SparkSession available as 'spark'.\n") + cat("\nSparkSession available as 'spark'.\n") } diff --git a/R/pkg/inst/worker/worker.R b/R/pkg/inst/worker/worker.R index c2adf613acb02..80dc4ee634512 100644 --- a/R/pkg/inst/worker/worker.R +++ b/R/pkg/inst/worker/worker.R @@ -49,7 +49,7 @@ compute <- function(mode, partition, serializer, deserializer, key, names(inputData) <- colNames } else { # Check to see if inputData is a valid data.frame - stopifnot(deserializer == "byte") + stopifnot(deserializer == "byte" || deserializer == "arrow") stopifnot(class(inputData) == "data.frame") } @@ -63,7 +63,7 @@ compute <- function(mode, partition, serializer, deserializer, key, output <- split(output, seq(nrow(output))) } else { # Serialize the output to a byte array - stopifnot(serializer == "byte") + stopifnot(serializer == "byte" || serializer == "arrow") } } else { output <- computeFunc(partition, inputData) @@ -76,6 +76,8 @@ outputResult <- function(serializer, output, outputCon) { SparkR:::writeRawSerialize(outputCon, output) } else if (serializer == "row") { SparkR:::writeRowSerialize(outputCon, output) + } else if (serializer == "arrow") { + SparkR:::writeSerializeInArrow(outputCon, output) } else { # write lines one-by-one with flag lapply(output, function(line) SparkR:::writeString(outputCon, line)) @@ -171,6 +173,16 @@ if (isEmpty != 0) { data <- dataWithKeys$data } else if (deserializer == "row") { data <- SparkR:::readMultipleObjects(inputCon) + } else if (deserializer == "arrow" && mode == 2) { + dataWithKeys <- SparkR:::readDeserializeWithKeysInArrow(inputCon) + keys <- dataWithKeys$keys + data <- dataWithKeys$data + } else if (deserializer == "arrow" && mode == 1) { + data <- SparkR:::readDeserializeInArrow(inputCon) + # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html + # rbind.fill might be an anternative to make it faster if plyr is installed. + # Also, note that, 'dapply' applies a function to each partition. + data <- do.call("rbind", data) } # Timing reading input data for execution @@ -181,17 +193,29 @@ if (isEmpty != 0) { colNames, computeFunc, data) } else { # gapply mode + outputs <- list() for (i in 1:length(data)) { # Timing reading input data for execution inputElap <- elapsedSecs() output <- compute(mode, partition, serializer, deserializer, keys[[i]], colNames, computeFunc, data[[i]]) computeElap <- elapsedSecs() - outputResult(serializer, output, outputCon) + if (serializer == "arrow") { + outputs[[length(outputs) + 1L]] <- output + } else { + outputResult(serializer, output, outputCon) + } outputElap <- elapsedSecs() computeInputElapsDiff <- computeInputElapsDiff + (computeElap - inputElap) outputComputeElapsDiff <- outputComputeElapsDiff + (outputElap - computeElap) } + + if (serializer == "arrow") { + # See https://stat.ethz.ch/pipermail/r-help/2010-September/252046.html + # rbind.fill might be an anternative to make it faster if plyr is installed. + combined <- do.call("rbind", outputs) + SparkR:::writeSerializeInArrow(outputCon, combined) + } } } else { output <- compute(mode, partition, serializer, deserializer, NULL, diff --git a/R/pkg/tests/fulltests/test_Serde.R b/R/pkg/tests/fulltests/test_Serde.R index 1525bdb2f5c8b..e01f6ee005218 100644 --- a/R/pkg/tests/fulltests/test_Serde.R +++ b/R/pkg/tests/fulltests/test_Serde.R @@ -138,7 +138,7 @@ test_that("createDataFrame large objects", { enableHiveSupport = FALSE)) sc <- getSparkContext() - actual <- callJStatic("org.apache.spark.api.r.RUtils", "getEncryptionEnabled", sc) + actual <- callJStatic("org.apache.spark.api.r.RUtils", "isEncryptionEnabled", sc) expected <- as.logical(encryptionEnabled) expect_equal(actual, expected) diff --git a/R/pkg/tests/fulltests/test_client.R b/R/pkg/tests/fulltests/test_client.R index de624b572cc2a..9798627ffc551 100644 --- a/R/pkg/tests/fulltests/test_client.R +++ b/R/pkg/tests/fulltests/test_client.R @@ -37,7 +37,7 @@ test_that("multiple packages don't produce a warning", { test_that("sparkJars sparkPackages as character vectors", { args <- generateSparkSubmitArgs("", "", c("one.jar", "two.jar", "three.jar"), "", - c("com.databricks:spark-avro_2.11:2.0.1")) + c("com.databricks:spark-avro_2.12:2.0.1")) expect_match(args, "--jars one.jar,two.jar,three.jar") - expect_match(args, "--packages com.databricks:spark-avro_2.11:2.0.1") + expect_match(args, "--packages com.databricks:spark-avro_2.12:2.0.1") }) diff --git a/R/pkg/tests/fulltests/test_context.R b/R/pkg/tests/fulltests/test_context.R index 288a2714a554e..eb8d2a700e1ea 100644 --- a/R/pkg/tests/fulltests/test_context.R +++ b/R/pkg/tests/fulltests/test_context.R @@ -54,15 +54,6 @@ test_that("Check masked functions", { sort(namesOfMaskedCompletely, na.last = TRUE)) }) -test_that("repeatedly starting and stopping SparkR", { - for (i in 1:4) { - sc <- suppressWarnings(sparkR.init(master = sparkRTestMaster)) - rdd <- parallelize(sc, 1:20, 2L) - expect_equal(countRDD(rdd), 20) - suppressWarnings(sparkR.stop()) - } -}) - test_that("repeatedly starting and stopping SparkSession", { for (i in 1:4) { sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) @@ -101,9 +92,6 @@ test_that("job group functions can be called", { cancelJobGroup("groupId") clearJobGroup() - suppressWarnings(setJobGroup(sc, "groupId", "job description", TRUE)) - suppressWarnings(cancelJobGroup(sc, "groupId")) - suppressWarnings(clearJobGroup(sc)) sparkR.session.stop() }) diff --git a/R/pkg/tests/fulltests/test_mllib_classification.R b/R/pkg/tests/fulltests/test_mllib_classification.R index 023686e75d50a..1f1b187aef567 100644 --- a/R/pkg/tests/fulltests/test_mllib_classification.R +++ b/R/pkg/tests/fulltests/test_mllib_classification.R @@ -299,7 +299,7 @@ test_that("spark.mlp", { df <- read.df(absoluteSparkPath("data/mllib/sample_multiclass_classification_data.txt"), source = "libsvm") model <- spark.mlp(df, label ~ features, blockSize = 128, layers = c(4, 5, 4, 3), - solver = "l-bfgs", maxIter = 100, tol = 0.5, stepSize = 1, seed = 1) + solver = "l-bfgs", maxIter = 100, tol = 0.00001, stepSize = 1, seed = 1) # Test summary method summary <- summary(model) @@ -307,13 +307,13 @@ test_that("spark.mlp", { expect_equal(summary$numOfOutputs, 3) expect_equal(summary$layers, c(4, 5, 4, 3)) expect_equal(length(summary$weights), 64) - expect_equal(head(summary$weights, 5), list(-0.878743, 0.2154151, -1.16304, -0.6583214, 1.009825), + expect_equal(head(summary$weights, 5), list(-24.28415, 107.8701, 16.86376, 1.103736, 9.244488), tolerance = 1e-6) # Test predict method mlpTestDF <- df mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) - expect_equal(head(mlpPredictions$prediction, 6), c("1.0", "0.0", "0.0", "0.0", "0.0", "0.0")) + expect_equal(head(mlpPredictions$prediction, 6), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0")) # Test model save/load if (windows_with_hadoop()) { @@ -348,12 +348,12 @@ test_that("spark.mlp", { # Test random seed # default seed - model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10) + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 100) mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) expect_equal(head(mlpPredictions$prediction, 10), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) # seed equals 10 - model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 10, seed = 10) + model <- spark.mlp(df, label ~ features, layers = c(4, 5, 4, 3), maxIter = 100, seed = 10) mlpPredictions <- collect(select(predict(model, mlpTestDF), "prediction")) expect_equal(head(mlpPredictions$prediction, 10), c("1.0", "1.0", "1.0", "1.0", "0.0", "1.0", "2.0", "2.0", "1.0", "0.0")) diff --git a/R/pkg/tests/fulltests/test_mllib_clustering.R b/R/pkg/tests/fulltests/test_mllib_clustering.R index 4110e13da4948..028ad574b8134 100644 --- a/R/pkg/tests/fulltests/test_mllib_clustering.R +++ b/R/pkg/tests/fulltests/test_mllib_clustering.R @@ -153,7 +153,7 @@ test_that("spark.kmeans", { model <- spark.kmeans(data = training, ~ ., k = 2, maxIter = 10, initMode = "random") sample <- take(select(predict(model, training), "prediction"), 1) expect_equal(typeof(sample$prediction), "integer") - expect_equal(sample$prediction, 1) + expect_equal(sample$prediction, 0) # Test stats::kmeans is working statsModel <- kmeans(x = newIris, centers = 2) @@ -319,4 +319,17 @@ test_that("spark.posterior and spark.perplexity", { expect_equal(length(local.posterior), sum(unlist(local.posterior))) }) +test_that("spark.assignClusters", { + df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) + clusters <- spark.assignClusters(df, initMode = "degree", weightCol = "weight") + expected_result <- createDataFrame(list(list(4L, 1L), list(0L, 0L), + list(1L, 0L), list(3L, 1L), + list(2L, 0L)), + schema = c("id", "cluster")) + expect_equivalent(expected_result, clusters) +}) + sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_fpm.R b/R/pkg/tests/fulltests/test_mllib_fpm.R index d80f66a25de1c..bc1e17538d41a 100644 --- a/R/pkg/tests/fulltests/test_mllib_fpm.R +++ b/R/pkg/tests/fulltests/test_mllib_fpm.R @@ -83,4 +83,21 @@ test_that("spark.fpGrowth", { }) +test_that("spark.prefixSpan", { + df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) + result <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, + maxLocalProjDBSize = 32000000L) + + expected_result <- createDataFrame(list(list(list(list(1L)), 3L), list(list(list(3L)), 2L), + list(list(list(2L)), 3L), list(list(list(1L, 2L)), 3L), + list(list(list(1L), list(3L)), 2L)), + schema = c("sequence", "freq")) + + expect_equivalent(expected_result, result) +}) + sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_mllib_recommendation.R b/R/pkg/tests/fulltests/test_mllib_recommendation.R index 4d919c9d746b0..d50de4123aeb0 100644 --- a/R/pkg/tests/fulltests/test_mllib_recommendation.R +++ b/R/pkg/tests/fulltests/test_mllib_recommendation.R @@ -27,13 +27,13 @@ test_that("spark.als", { list(2, 1, 1.0), list(2, 2, 5.0)) df <- createDataFrame(data, c("user", "item", "score")) model <- spark.als(df, ratingCol = "score", userCol = "user", itemCol = "item", - rank = 10, maxIter = 5, seed = 0, regParam = 0.1) + rank = 10, maxIter = 15, seed = 0, regParam = 0.1) stats <- summary(model) expect_equal(stats$rank, 10) test <- createDataFrame(list(list(0, 2), list(1, 0), list(2, 0)), c("user", "item")) predictions <- collect(predict(model, test)) - expect_equal(predictions$prediction, c(-0.1380762, 2.6258414, -1.5018409), + expect_equal(predictions$prediction, c(0.6324540, 3.6218479, -0.4568263), tolerance = 1e-4) # Test model save/load diff --git a/R/pkg/tests/fulltests/test_mllib_regression.R b/R/pkg/tests/fulltests/test_mllib_regression.R index 23daca75fcc22..b40c4cb9a9694 100644 --- a/R/pkg/tests/fulltests/test_mllib_regression.R +++ b/R/pkg/tests/fulltests/test_mllib_regression.R @@ -102,10 +102,18 @@ test_that("spark.glm and predict", { }) test_that("spark.glm summary", { + # prepare dataset + Sepal.Length <- c(2.0, 1.5, 1.8, 3.4, 5.1, 1.8, 1.0, 2.3) + Sepal.Width <- c(2.1, 2.3, 5.4, 4.7, 3.1, 2.1, 3.1, 5.5) + Petal.Length <- c(1.8, 2.1, 7.1, 2.5, 3.7, 6.3, 2.2, 7.2) + Species <- c("setosa", "versicolor", "versicolor", "versicolor", "virginica", "virginica", + "versicolor", "virginica") + dataset <- data.frame(Sepal.Length, Sepal.Width, Petal.Length, Species, stringsAsFactors = TRUE) + # gaussian family - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species)) - rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = dataset)) # test summary coefficients return matrix type expect_true(class(stats$coefficients) == "matrix") @@ -126,15 +134,15 @@ test_that("spark.glm summary", { out <- capture.output(print(stats)) expect_match(out[2], "Deviance Residuals:") - expect_true(any(grepl("AIC: 59.22", out))) + expect_true(any(grepl("AIC: 35.84", out))) # binomial family - df <- suppressWarnings(createDataFrame(iris)) + df <- suppressWarnings(createDataFrame(dataset)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(spark.glm(training, Species ~ Sepal_Length + Sepal_Width, family = binomial(link = "logit"))) - rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rTraining <- dataset[dataset$Species %in% c("versicolor", "virginica"), ] rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, family = binomial(link = "logit"))) @@ -174,17 +182,17 @@ test_that("spark.glm summary", { expect_equal(stats$aic, rStats$aic) # Test spark.glm works with offset - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(spark.glm(training, Sepal_Width ~ Sepal_Length + Species, family = poisson(), offsetCol = "Petal_Length")) rStats <- suppressWarnings(summary(glm(Sepal.Width ~ Sepal.Length + Species, - data = iris, family = poisson(), offset = iris$Petal.Length))) + data = dataset, family = poisson(), offset = dataset$Petal.Length))) expect_true(all(abs(rStats$coefficients - stats$coefficients) < 1e-3)) # Test summary works on base GLM models - baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = iris) + baseModel <- stats::glm(Sepal.Width ~ Sepal.Length + Species, data = dataset) baseSummary <- summary(baseModel) - expect_true(abs(baseSummary$deviance - 12.19313) < 1e-4) + expect_true(abs(baseSummary$deviance - 11.84013) < 1e-4) # Test spark.glm works with regularization parameter data <- as.data.frame(cbind(a1, a2, b)) @@ -300,11 +308,19 @@ test_that("glm and predict", { }) test_that("glm summary", { + # prepare dataset + Sepal.Length <- c(2.0, 1.5, 1.8, 3.4, 5.1, 1.8, 1.0, 2.3) + Sepal.Width <- c(2.1, 2.3, 5.4, 4.7, 3.1, 2.1, 3.1, 5.5) + Petal.Length <- c(1.8, 2.1, 7.1, 2.5, 3.7, 6.3, 2.2, 7.2) + Species <- c("setosa", "versicolor", "versicolor", "versicolor", "virginica", "virginica", + "versicolor", "virginica") + dataset <- data.frame(Sepal.Length, Sepal.Width, Petal.Length, Species, stringsAsFactors = TRUE) + # gaussian family - training <- suppressWarnings(createDataFrame(iris)) + training <- suppressWarnings(createDataFrame(dataset)) stats <- summary(glm(Sepal_Width ~ Sepal_Length + Species, data = training)) - rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = iris)) + rStats <- summary(glm(Sepal.Width ~ Sepal.Length + Species, data = dataset)) coefs <- stats$coefficients rCoefs <- rStats$coefficients @@ -320,12 +336,12 @@ test_that("glm summary", { expect_equal(stats$aic, rStats$aic) # binomial family - df <- suppressWarnings(createDataFrame(iris)) + df <- suppressWarnings(createDataFrame(dataset)) training <- df[df$Species %in% c("versicolor", "virginica"), ] stats <- summary(glm(Species ~ Sepal_Length + Sepal_Width, data = training, family = binomial(link = "logit"))) - rTraining <- iris[iris$Species %in% c("versicolor", "virginica"), ] + rTraining <- dataset[dataset$Species %in% c("versicolor", "virginica"), ] rStats <- summary(glm(Species ~ Sepal.Length + Sepal.Width, data = rTraining, family = binomial(link = "logit"))) diff --git a/R/pkg/tests/fulltests/test_mllib_tree.R b/R/pkg/tests/fulltests/test_mllib_tree.R index facd3a941cf1f..ad68700c7ff4e 100644 --- a/R/pkg/tests/fulltests/test_mllib_tree.R +++ b/R/pkg/tests/fulltests/test_mllib_tree.R @@ -148,10 +148,10 @@ test_that("spark.randomForest", { model <- spark.randomForest(data, Employed ~ ., "regression", maxDepth = 5, maxBins = 16, numTrees = 20, seed = 123) predictions <- collect(predict(model, data)) - expect_equal(predictions$prediction, c(60.32820, 61.22315, 60.69025, 62.11070, - 63.53160, 64.05470, 65.12710, 64.30450, - 66.70910, 67.86125, 68.08700, 67.21865, - 68.89275, 69.53180, 69.39640, 69.68250), + expect_equal(predictions$prediction, c(60.32495, 61.06495, 60.52120, 61.98500, + 63.64450, 64.21910, 65.00810, 64.30450, + 66.70910, 67.96875, 68.22140, 67.21865, + 68.89275, 69.55900, 69.30160, 69.93050), tolerance = 1e-4) stats <- summary(model) expect_equal(stats$numTrees, 20) diff --git a/R/pkg/tests/fulltests/test_sparkSQL.R b/R/pkg/tests/fulltests/test_sparkSQL.R index 5cc75aa3f3673..fdc7474820659 100644 --- a/R/pkg/tests/fulltests/test_sparkSQL.R +++ b/R/pkg/tests/fulltests/test_sparkSQL.R @@ -106,15 +106,6 @@ 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) }) @@ -221,7 +212,7 @@ test_that("structField type strings", { test_that("create DataFrame from RDD", { rdd <- lapply(parallelize(sc, 1:10), function(x) { list(x, as.character(x)) }) - df <- createDataFrame(rdd, list("a", "b")) + df <- SparkR::createDataFrame(rdd, list("a", "b")) dfAsDF <- as.DataFrame(rdd, list("a", "b")) expect_is(df, "SparkDataFrame") expect_is(dfAsDF, "SparkDataFrame") @@ -287,7 +278,7 @@ test_that("create DataFrame from RDD", { df <- as.DataFrame(cars, numPartitions = 2) expect_equal(getNumPartitions(df), 2) - df <- createDataFrame(cars, numPartitions = 3) + df <- SparkR::createDataFrame(cars, numPartitions = 3) expect_equal(getNumPartitions(df), 3) # validate limit by num of rows df <- createDataFrame(cars, numPartitions = 60) @@ -308,7 +299,7 @@ test_that("create DataFrame from RDD", { sql("CREATE TABLE people (name string, age double, height float)") df <- read.df(jsonPathNa, "json", schema) insertInto(df, "people") - expect_equal(collect(sql("SELECT age from people WHERE name = 'Bob'"))$age, + expect_equal(collect(SparkR::sql("SELECT age from people WHERE name = 'Bob'"))$age, c(16)) expect_equal(collect(sql("SELECT height from people WHERE name ='Bob'"))$height, c(176.5)) @@ -628,14 +619,10 @@ test_that("read/write json files", { jsonPath3 <- tempfile(pattern = "jsonPath3", fileext = ".json") write.json(df, jsonPath3) - # Test read.json()/jsonFile() works with multiple input paths + # Test read.json() works with multiple input paths jsonDF1 <- read.json(c(jsonPath2, jsonPath3)) expect_is(jsonDF1, "SparkDataFrame") expect_equal(count(jsonDF1), 6) - # Suppress warnings because jsonFile is deprecated - jsonDF2 <- suppressWarnings(jsonFile(c(jsonPath2, jsonPath3))) - expect_is(jsonDF2, "SparkDataFrame") - expect_equal(count(jsonDF2), 6) unlink(jsonPath2) unlink(jsonPath3) @@ -655,20 +642,6 @@ test_that("read/write json files - compression option", { unlink(jsonPath) }) -test_that("jsonRDD() on a RDD with json string", { - sqlContext <- suppressWarnings(sparkRSQL.init(sc)) - rdd <- parallelize(sc, mockLines) - expect_equal(countRDD(rdd), 3) - df <- suppressWarnings(jsonRDD(sqlContext, rdd)) - expect_is(df, "SparkDataFrame") - expect_equal(count(df), 3) - - rdd2 <- flatMap(rdd, function(x) c(x, x)) - df <- suppressWarnings(jsonRDD(sqlContext, rdd2)) - expect_is(df, "SparkDataFrame") - expect_equal(count(df), 6) -}) - test_that("test tableNames and tables", { count <- count(listTables()) @@ -683,10 +656,10 @@ test_that("test tableNames and tables", { expect_true("tableName" %in% colnames(tables())) expect_true(all(c("tableName", "database", "isTemporary") %in% colnames(tables()))) - suppressWarnings(registerTempTable(df, "table2")) + createOrReplaceTempView(df, "table2") tables <- listTables() expect_equal(count(tables), count + 2) - suppressWarnings(dropTempTable("table1")) + dropTempView("table1") expect_true(dropTempView("table2")) tables <- listTables() @@ -1406,7 +1379,7 @@ test_that("column operators", { test_that("column functions", { c <- column("a") - c1 <- abs(c) + acos(c) + approxCountDistinct(c) + ascii(c) + asin(c) + atan(c) + c1 <- abs(c) + acos(c) + approx_count_distinct(c) + ascii(c) + asin(c) + atan(c) c2 <- avg(c) + base64(c) + bin(c) + bitwiseNOT(c) + cbrt(c) + ceil(c) + cos(c) c3 <- cosh(c) + count(c) + crc32(c) + hash(c) + exp(c) c4 <- explode(c) + expm1(c) + factorial(c) + first(c) + floor(c) + hex(c) @@ -1415,9 +1388,9 @@ test_that("column functions", { c7 <- mean(c) + min(c) + month(c) + negate(c) + posexplode(c) + quarter(c) c8 <- reverse(c) + rint(c) + round(c) + rtrim(c) + sha1(c) + monotonically_increasing_id() c9 <- signum(c) + sin(c) + sinh(c) + size(c) + stddev(c) + soundex(c) + sqrt(c) + sum(c) - c10 <- sumDistinct(c) + tan(c) + tanh(c) + toDegrees(c) + toRadians(c) + c10 <- sumDistinct(c) + tan(c) + tanh(c) + degrees(c) + radians(c) c11 <- to_date(c) + trim(c) + unbase64(c) + unhex(c) + upper(c) - c12 <- variance(c) + ltrim(c, "a") + rtrim(c, "b") + trim(c, "c") + c12 <- variance(c) + xxhash64(c) + ltrim(c, "a") + rtrim(c, "b") + trim(c, "c") c13 <- lead("col", 1) + lead(c, 1) + lag("col", 1) + lag(c, 1) c14 <- cume_dist() + ntile(1) + corr(c, c1) c15 <- dense_rank() + percent_rank() + rank() + row_number() @@ -1467,6 +1440,14 @@ test_that("column functions", { df5 <- createDataFrame(list(list(a = "010101"))) expect_equal(collect(select(df5, conv(df5$a, 2, 16)))[1, 1], "15") + # Test months_between() + df <- createDataFrame(list(list(a = as.Date("1997-02-28"), + b = as.Date("1996-10-30")))) + result1 <- collect(select(df, alias(months_between(df[[1]], df[[2]]), "month")))[[1]] + expect_equal(result1, 3.93548387) + result2 <- collect(select(df, alias(months_between(df[[1]], df[[2]], FALSE), "month")))[[1]] + expect_equal(result2, 3.935483870967742) + # Test array_contains(), array_max(), array_min(), array_position(), element_at() and reverse() df <- createDataFrame(list(list(list(1L, 2L, 3L)), list(list(6L, 5L, 4L)))) result <- collect(select(df, array_contains(df[[1]], 1L)))[[1]] @@ -1512,6 +1493,13 @@ test_that("column functions", { expected_entries <- list(as.environment(list(x = 1, y = 2))) expect_equal(result, expected_entries) + # Test map_from_entries() + df <- createDataFrame(list(list(list(listToStruct(list(c1 = "x", c2 = 1L)), + listToStruct(list(c1 = "y", c2 = 2L)))))) + result <- collect(select(df, map_from_entries(df[[1]])))[[1]] + expected_entries <- list(as.environment(list(x = 1L, y = 2L))) + expect_equal(result, expected_entries) + # Test array_repeat() df <- createDataFrame(list(list("a", 3L), list("b", 2L))) result <- collect(select(df, array_repeat(df[[1]], df[[2]])))[[1]] @@ -1570,6 +1558,13 @@ test_that("column functions", { result <- collect(select(df, flatten(df[[1]])))[[1]] expect_equal(result, list(list(1L, 2L, 3L, 4L), list(5L, 6L, 7L, 8L))) + # Test map_concat + df <- createDataFrame(list(list(map1 = as.environment(list(x = 1, y = 2)), + map2 = as.environment(list(a = 3, b = 4))))) + result <- collect(select(df, map_concat(df[[1]], df[[2]])))[[1]] + expected_entries <- list(as.environment(list(x = 1, y = 2, a = 3, b = 4))) + expect_equal(result, expected_entries) + # Test map_entries(), map_keys(), map_values() and element_at() df <- createDataFrame(list(list(map = as.environment(list(x = 1, y = 2))))) result <- collect(select(df, map_entries(df$map)))[[1]] @@ -1647,7 +1642,26 @@ test_that("column functions", { expect_equal(collect(select(df, bround(df$x, 0)))[[1]][1], 2) expect_equal(collect(select(df, bround(df$x, 0)))[[1]][2], 4) - # Test to_json(), from_json() + # Test from_csv(), schema_of_csv() + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, alias(from_csv(df$col, "a INT"), "csv"))) + expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, lit("a INT")), "csv"))) + expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, structType("a INT")), "csv"))) + expect_equal(c[[1]][[1]]$a, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv("1")), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) + c <- collect(select(df, alias(from_csv(df$col, schema_of_csv(lit("1"))), "csv"))) + expect_equal(c[[1]][[1]]$`_c0`, 1) + + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_csv("Amsterdam,2018"))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") + c <- collect(select(df, schema_of_csv(lit("Amsterdam,2018")))) + expect_equal(c[[1]], "struct<_c0:string,_c1:int>") + + # Test to_json(), from_json(), schema_of_json() df <- sql("SELECT array(named_struct('name', 'Bob'), named_struct('name', 'Alice')) as people") j <- collect(select(df, alias(to_json(df$people), "json"))) expect_equal(j[order(j$json), ][1], "[{\"name\":\"Bob\"},{\"name\":\"Alice\"}]") @@ -1665,7 +1679,9 @@ test_that("column functions", { expect_equal(j[order(j$json), ][1], "{\"age\":16,\"height\":176.5}") df <- as.DataFrame(j) schemas <- list(structType(structField("age", "integer"), structField("height", "double")), - "age INT, height DOUBLE") + "age INT, height DOUBLE", + schema_of_json("{\"age\":16,\"height\":176.5}"), + schema_of_json(lit("{\"age\":16,\"height\":176.5}"))) for (schema in schemas) { s <- collect(select(df, alias(from_json(df$json, schema), "structcol"))) expect_equal(ncol(s), 1) @@ -1674,6 +1690,12 @@ test_that("column functions", { expect_true(any(apply(s, 1, function(x) { x[[1]]$age == 16 }))) } + df <- as.DataFrame(list(list("col" = "1"))) + c <- collect(select(df, schema_of_json('{"name":"Bob"}'))) + expect_equal(c[[1]], "struct") + c <- collect(select(df, schema_of_json(lit('{"name":"Bob"}')))) + expect_equal(c[[1]], "struct") + # Test to_json() supports arrays of primitive types and arrays df <- sql("SELECT array(19, 42, 70) as age") j <- collect(select(df, alias(to_json(df$age), "json"))) @@ -1687,7 +1709,7 @@ test_that("column functions", { df <- as.DataFrame(list(list("col" = "{\"date\":\"21/10/2014\"}"))) schema2 <- structType(structField("date", "date")) s <- collect(select(df, from_json(df$col, schema2))) - expect_equal(s[[1]][[1]], NA) + expect_equal(s[[1]][[1]]$date, NA) s <- collect(select(df, from_json(df$col, schema2, dateFormat = "dd/MM/yyyy"))) expect_is(s[[1]][[1]]$date, "Date") expect_equal(as.character(s[[1]][[1]]$date), "2014-10-21") @@ -1699,7 +1721,11 @@ test_that("column functions", { # check if array type in string is correctly supported. jsonArr <- "[{\"name\":\"Bob\"}, {\"name\":\"Alice\"}]" df <- as.DataFrame(list(list("people" = jsonArr))) - for (schema in list(structType(structField("name", "string")), "name STRING")) { + schemas <- list(structType(structField("name", "string")), + "name STRING", + schema_of_json("{\"name\":\"Alice\"}"), + schema_of_json(lit("{\"name\":\"Bob\"}"))) + for (schema in schemas) { arr <- collect(select(df, alias(from_json(df$people, schema, as.json.array = TRUE), "arrcol"))) expect_equal(ncol(arr), 1) expect_equal(nrow(arr), 1) @@ -1709,6 +1735,11 @@ test_that("column functions", { expect_equal(arr$arrcol[[1]][[2]]$name, "Alice") } + # Test to_csv() + df <- sql("SELECT named_struct('name', 'Bob') as people") + j <- collect(select(df, alias(to_csv(df$people), "csv"))) + expect_equal(j[order(j$csv), ][1], "Bob") + # Test create_array() and create_map() df <- as.DataFrame(data.frame( x = c(1.0, 2.0), y = c(-1.0, 3.0), z = c(-2.0, 5.0) @@ -1755,9 +1786,9 @@ test_that("column binary mathfunctions", { expect_equal(collect(select(df, shiftRight(df$b, 1)))[4, 1], 4) expect_equal(collect(select(df, shiftRightUnsigned(df$b, 1)))[4, 1], 4) expect_equal(class(collect(select(df, rand()))[2, 1]), "numeric") - expect_equal(collect(select(df, rand(1)))[1, 1], 0.134, tolerance = 0.01) + expect_equal(collect(select(df, rand(1)))[1, 1], 0.636, tolerance = 0.01) expect_equal(class(collect(select(df, randn()))[2, 1]), "numeric") - expect_equal(collect(select(df, randn(1)))[1, 1], -1.03, tolerance = 0.01) + expect_equal(collect(select(df, randn(1)))[1, 1], 1.68, tolerance = 0.01) }) test_that("string operators", { @@ -1874,10 +1905,20 @@ test_that("date functions on a DataFrame", { df2 <- createDataFrame(l2) expect_equal(collect(select(df2, minute(df2$b)))[, 1], c(34, 24)) expect_equal(collect(select(df2, second(df2$b)))[, 1], c(0, 34)) - expect_equal(collect(select(df2, from_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) - expect_equal(collect(select(df2, to_utc_timestamp(df2$b, "JST")))[, 1], - c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) + conf <- callJMethod(sparkSession, "conf") + isUtcTimestampFuncEnabled <- callJMethod(conf, "get", "spark.sql.legacy.utcTimestampFunc.enabled") + callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", "true") + tryCatch({ + # Both from_utc_timestamp and to_utc_timestamp are deprecated as of SPARK-25496 + expect_equal(suppressWarnings(collect(select(df2, from_utc_timestamp(df2$b, "JST"))))[, 1], + c(as.POSIXct("2012-12-13 21:34:00 UTC"), as.POSIXct("2014-12-15 10:24:34 UTC"))) + expect_equal(suppressWarnings(collect(select(df2, to_utc_timestamp(df2$b, "JST"))))[, 1], + c(as.POSIXct("2012-12-13 03:34:00 UTC"), as.POSIXct("2014-12-14 16:24:34 UTC"))) + }, + finally = { + # Reverting the conf back + callJMethod(conf, "set", "spark.sql.legacy.utcTimestampFunc.enabled", isUtcTimestampFuncEnabled) + }) expect_gt(collect(select(df2, unix_timestamp()))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(df2$b)))[1, 1], 0) expect_gt(collect(select(df2, unix_timestamp(lit("2015-01-01"), "yyyy-MM-dd")))[1, 1], 0) @@ -2325,39 +2366,94 @@ test_that("join(), crossJoin() and merge() on a DataFrame", { expect_equal(names(joined2), c("age", "name", "name", "test")) expect_equal(count(joined2), 3) - joined3 <- join(df, df2, df$name == df2$name, "rightouter") + joined3 <- join(df, df2, df$name == df2$name, "right") expect_equal(names(joined3), c("age", "name", "name", "test")) expect_equal(count(joined3), 4) expect_true(is.na(collect(orderBy(joined3, joined3$age))$age[2])) - joined4 <- select(join(df, df2, df$name == df2$name, "outer"), - alias(df$age + 5, "newAge"), df$name, df2$test) - expect_equal(names(joined4), c("newAge", "name", "test")) + joined4 <- join(df, df2, df$name == df2$name, "right_outer") + expect_equal(names(joined4), c("age", "name", "name", "test")) expect_equal(count(joined4), 4) - expect_equal(collect(orderBy(joined4, joined4$name))$newAge[3], 24) + expect_true(is.na(collect(orderBy(joined4, joined4$age))$age[2])) - joined5 <- join(df, df2, df$name == df2$name, "leftouter") + joined5 <- join(df, df2, df$name == df2$name, "rightouter") expect_equal(names(joined5), c("age", "name", "name", "test")) - expect_equal(count(joined5), 3) - expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[1])) + expect_equal(count(joined5), 4) + expect_true(is.na(collect(orderBy(joined5, joined5$age))$age[2])) - joined6 <- join(df, df2, df$name == df2$name, "inner") - expect_equal(names(joined6), c("age", "name", "name", "test")) - expect_equal(count(joined6), 3) - joined7 <- join(df, df2, df$name == df2$name, "leftsemi") - expect_equal(names(joined7), c("age", "name")) - expect_equal(count(joined7), 3) + joined6 <- select(join(df, df2, df$name == df2$name, "outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined6), c("newAge", "name", "test")) + expect_equal(count(joined6), 4) + expect_equal(collect(orderBy(joined6, joined6$name))$newAge[3], 24) - joined8 <- join(df, df2, df$name == df2$name, "left_outer") - expect_equal(names(joined8), c("age", "name", "name", "test")) - expect_equal(count(joined8), 3) - expect_true(is.na(collect(orderBy(joined8, joined8$age))$age[1])) + joined7 <- select(join(df, df2, df$name == df2$name, "full"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined7), c("newAge", "name", "test")) + expect_equal(count(joined7), 4) + expect_equal(collect(orderBy(joined7, joined7$name))$newAge[3], 24) + + joined8 <- select(join(df, df2, df$name == df2$name, "fullouter"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined8), c("newAge", "name", "test")) + expect_equal(count(joined8), 4) + expect_equal(collect(orderBy(joined8, joined8$name))$newAge[3], 24) - joined9 <- join(df, df2, df$name == df2$name, "right_outer") - expect_equal(names(joined9), c("age", "name", "name", "test")) + joined9 <- select(join(df, df2, df$name == df2$name, "full_outer"), + alias(df$age + 5, "newAge"), df$name, df2$test) + expect_equal(names(joined9), c("newAge", "name", "test")) expect_equal(count(joined9), 4) - expect_true(is.na(collect(orderBy(joined9, joined9$age))$age[2])) + expect_equal(collect(orderBy(joined9, joined9$name))$newAge[3], 24) + + joined10 <- join(df, df2, df$name == df2$name, "left") + expect_equal(names(joined10), c("age", "name", "name", "test")) + expect_equal(count(joined10), 3) + expect_true(is.na(collect(orderBy(joined10, joined10$age))$age[1])) + + joined11 <- join(df, df2, df$name == df2$name, "leftouter") + expect_equal(names(joined11), c("age", "name", "name", "test")) + expect_equal(count(joined11), 3) + expect_true(is.na(collect(orderBy(joined11, joined11$age))$age[1])) + + joined12 <- join(df, df2, df$name == df2$name, "left_outer") + expect_equal(names(joined12), c("age", "name", "name", "test")) + expect_equal(count(joined12), 3) + expect_true(is.na(collect(orderBy(joined12, joined12$age))$age[1])) + + joined13 <- join(df, df2, df$name == df2$name, "inner") + expect_equal(names(joined13), c("age", "name", "name", "test")) + expect_equal(count(joined13), 3) + + joined14 <- join(df, df2, df$name == df2$name, "semi") + expect_equal(names(joined14), c("age", "name")) + expect_equal(count(joined14), 3) + + joined14 <- join(df, df2, df$name == df2$name, "leftsemi") + expect_equal(names(joined14), c("age", "name")) + expect_equal(count(joined14), 3) + + joined15 <- join(df, df2, df$name == df2$name, "left_semi") + expect_equal(names(joined15), c("age", "name")) + expect_equal(count(joined15), 3) + + joined16 <- join(df2, df, df2$name == df$name, "anti") + expect_equal(names(joined16), c("name", "test")) + expect_equal(count(joined16), 1) + + joined17 <- join(df2, df, df2$name == df$name, "leftanti") + expect_equal(names(joined17), c("name", "test")) + expect_equal(count(joined17), 1) + + joined18 <- join(df2, df, df2$name == df$name, "left_anti") + expect_equal(names(joined18), c("name", "test")) + expect_equal(count(joined18), 1) + + error_msg <- paste("joinType must be one of the following types:", + "'inner', 'cross', 'outer', 'full', 'fullouter', 'full_outer',", + "'left', 'leftouter', 'left_outer', 'right', 'rightouter', 'right_outer',", + "'semi', 'leftsemi', 'left_semi', 'anti', 'leftanti' or 'left_anti'.") + expect_error(join(df2, df, df2$name == df$name, "invalid"), error_msg) merged <- merge(df, df2, by.x = "name", by.y = "name", all.x = TRUE, all.y = TRUE) expect_equal(count(merged), 4) @@ -2472,6 +2568,7 @@ test_that("union(), unionByName(), rbind(), except(), and intersect() on a DataF expect_is(unioned, "SparkDataFrame") expect_equal(count(unioned), 6) expect_equal(first(unioned)$name, "Michael") + expect_equal(count(arrange(suppressWarnings(union(df, df2)), df$age)), 6) expect_equal(count(arrange(suppressWarnings(unionAll(df, df2)), df$age)), 6) df1 <- select(df2, "age", "name") @@ -2651,17 +2748,14 @@ test_that("read/write Parquet files", { expect_is(df2, "SparkDataFrame") expect_equal(count(df2), 3) - # Test write.parquet/saveAsParquetFile and read.parquet/parquetFile + # Test write.parquet and read.parquet parquetPath2 <- tempfile(pattern = "parquetPath2", fileext = ".parquet") write.parquet(df, parquetPath2) parquetPath3 <- tempfile(pattern = "parquetPath3", fileext = ".parquet") - suppressWarnings(saveAsParquetFile(df, parquetPath3)) + write.parquet(df, parquetPath3) parquetDF <- read.parquet(c(parquetPath2, parquetPath3)) expect_is(parquetDF, "SparkDataFrame") expect_equal(count(parquetDF), count(df) * 2) - parquetDF2 <- suppressWarnings(parquetFile(parquetPath2, parquetPath3)) - expect_is(parquetDF2, "SparkDataFrame") - expect_equal(count(parquetDF2), count(df) * 2) # Test if varargs works with variables saveMode <- "overwrite" @@ -2942,7 +3036,7 @@ test_that("sampleBy() on a DataFrame", { sample <- sampleBy(df, "key", fractions, 0) result <- collect(orderBy(count(groupBy(sample, "key")), "key")) expect_identical(as.list(result[1, ]), list(key = "0", count = 3)) - expect_identical(as.list(result[2, ]), list(key = "1", count = 7)) + expect_identical(as.list(result[2, ]), list(key = "1", count = 8)) }) test_that("approxQuantile() on a DataFrame", { @@ -3470,39 +3564,6 @@ test_that("Window functions on a DataFrame", { expect_equal(result, expected) }) -test_that("createDataFrame sqlContext parameter backward compatibility", { - sqlContext <- suppressWarnings(sparkRSQL.init(sc)) - a <- 1:3 - b <- c("a", "b", "c") - ldf <- data.frame(a, b) - # Call function with namespace :: operator - SPARK-16538 - df <- suppressWarnings(SparkR::createDataFrame(sqlContext, ldf)) - expect_equal(columns(df), c("a", "b")) - expect_equal(dtypes(df), list(c("a", "int"), c("b", "string"))) - expect_equal(count(df), 3) - ldf2 <- collect(df) - expect_equal(ldf$a, ldf2$a) - - df2 <- suppressWarnings(createDataFrame(sqlContext, iris)) - expect_equal(count(df2), 150) - expect_equal(ncol(df2), 5) - - df3 <- suppressWarnings(read.df(sqlContext, jsonPath, "json")) - expect_is(df3, "SparkDataFrame") - expect_equal(count(df3), 3) - - before <- suppressWarnings(createDataFrame(sqlContext, iris)) - after <- suppressWarnings(createDataFrame(iris)) - expect_equal(collect(before), collect(after)) - - # more tests for SPARK-16538 - createOrReplaceTempView(df, "table") - SparkR::listTables() - SparkR::sql("SELECT 1") - suppressWarnings(SparkR::sql(sqlContext, "SELECT * FROM table")) - suppressWarnings(SparkR::dropTempTable(sqlContext, "table")) -}) - test_that("randomSplit", { num <- 4000 df <- createDataFrame(data.frame(id = 1:num)) @@ -3689,7 +3750,7 @@ test_that("catalog APIs, listTables, listColumns, listFunctions", { createOrReplaceTempView(as.DataFrame(cars), "cars") - tb <- listTables() + tb <- SparkR::listTables() expect_equal(nrow(tb), count + 1) tbs <- collect(tb) expect_true(nrow(tbs[tbs$name == "cars", ]) > 0) diff --git a/R/pkg/tests/fulltests/test_sparkSQL_arrow.R b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R new file mode 100644 index 0000000000000..825c7423e1579 --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkSQL_arrow.R @@ -0,0 +1,315 @@ +# +# 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. +# + +library(testthat) + +context("SparkSQL Arrow optimization") + +sparkSession <- sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + +test_that("createDataFrame/collect Arrow optimization", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + expected <- collect(createDataFrame(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(mtcars)), expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("createDataFrame/collect Arrow optimization - many partitions (partition order test)", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(mtcars, numPartitions = 32)), + collect(createDataFrame(mtcars, numPartitions = 1))) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("createDataFrame/collect Arrow optimization - type specification", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L, + f = as.Date("1990-02-24"), + g = as.POSIXct("1990-02-24 12:34:56")))) + + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + conf <- callJMethod(sparkSession, "conf") + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + expected <- collect(createDataFrame(rdf)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_equal(collect(createDataFrame(rdf)), expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization", { + skip_if_not_installed("arrow") + df <- createDataFrame(mtcars) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + ret <- dapply(df, + function(rdf) { + stopifnot(class(rdf) == "data.frame") + rdf + }, + schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- dapply(df, + function(rdf) { + stopifnot(class(rdf) == "data.frame") + # mtcars' hp is more then 50. + stopifnot(all(rdf$hp > 50)) + rdf + }, + schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + expect_equal(count(ret), nrow(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization - type specification", { + skip_if_not_installed("arrow") + # Note that regular dapply() seems not supporting date and timestamps + # whereas Arrow-optimized dapply() does. + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L))) + # numPartitions are set to 8 intentionally to test empty partitions as well. + df <- createDataFrame(rdf, numPartitions = 8) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("dapply() Arrow optimization - type specification (date and timestamp)", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = as.Date("1990-02-24"), + b = as.POSIXct("1990-02-24 12:34:56")))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- dapply(df, function(rdf) { rdf }, schema(df)) + expect_equal(collect(ret), rdf) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization", { + skip_if_not_installed("arrow") + df <- createDataFrame(mtcars) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + ret <- gapply(df, + "gear", + function(key, grouped) { + if (length(key) > 0) { + stopifnot(is.numeric(key[[1]])) + } + stopifnot(class(grouped) == "data.frame") + grouped + }, + schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- gapply(df, + "gear", + function(key, grouped) { + if (length(key) > 0) { + stopifnot(is.numeric(key[[1]])) + } + stopifnot(class(grouped) == "data.frame") + stopifnot(length(colnames(grouped)) == 11) + # mtcars' hp is more then 50. + stopifnot(all(grouped$hp > 50)) + grouped + }, + schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + expect_equal(count(ret), nrow(mtcars)) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization - type specification", { + skip_if_not_installed("arrow") + # Note that regular gapply() seems not supporting date and timestamps + # whereas Arrow-optimized gapply() does. + rdf <- data.frame(list(list(a = 1, + b = "a", + c = TRUE, + d = 1.1, + e = 1L))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "false") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + expected <- collect(ret) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) + + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + actual <- collect(ret) + expect_equal(actual, expected) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("gapply() Arrow optimization - type specification (date and timestamp)", { + skip_if_not_installed("arrow") + rdf <- data.frame(list(list(a = as.Date("1990-02-24"), + b = as.POSIXct("1990-02-24 12:34:56")))) + df <- createDataFrame(rdf) + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + ret <- gapply(df, + "a", + function(key, grouped) { grouped }, schema(df)) + expect_equal(collect(ret), rdf) + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +test_that("Arrow optimization - unsupported types", { + skip_if_not_installed("arrow") + + conf <- callJMethod(sparkSession, "conf") + arrowEnabled <- sparkR.conf("spark.sql.execution.arrow.sparkr.enabled")[[1]] + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", "true") + tryCatch({ + expect_error(checkSchemaInArrow(structType("a FLOAT")), "not support float type") + expect_error(checkSchemaInArrow(structType("a BINARY")), "not support binary type") + expect_error(checkSchemaInArrow(structType("a ARRAY")), "not support array type") + expect_error(checkSchemaInArrow(structType("a MAP")), "not support map type") + expect_error(checkSchemaInArrow(structType("a STRUCT")), + "not support nested struct type") + }, + finally = { + callJMethod(conf, "set", "spark.sql.execution.arrow.sparkr.enabled", arrowEnabled) + }) +}) + +sparkR.session.stop() diff --git a/R/pkg/tests/fulltests/test_sparkSQL_eager.R b/R/pkg/tests/fulltests/test_sparkSQL_eager.R new file mode 100644 index 0000000000000..9b4489a47b655 --- /dev/null +++ b/R/pkg/tests/fulltests/test_sparkSQL_eager.R @@ -0,0 +1,72 @@ +# +# 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. +# + +library(testthat) + +context("test show SparkDataFrame when eager execution is enabled.") + +test_that("eager execution is not enabled", { + # Start Spark session without eager execution enabled + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE) + + df <- createDataFrame(faithful) + expect_is(df, "SparkDataFrame") + expected <- "eruptions:double, waiting:double" + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) + +test_that("eager execution is enabled", { + # Start Spark session with eager execution enabled + sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true") + + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) + + df <- createDataFrame(faithful) + expect_is(df, "SparkDataFrame") + expected <- paste0("(+---------+-------+\n", + "|eruptions|waiting|\n", + "+---------+-------+\n)*", + "(only showing top 20 rows)") + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) + +test_that("eager execution is enabled with maxNumRows and truncate set", { + # Start Spark session with eager execution enabled + sparkConfig <- list(spark.sql.repl.eagerEval.enabled = "true", + spark.sql.repl.eagerEval.maxNumRows = as.integer(5), + spark.sql.repl.eagerEval.truncate = as.integer(2)) + + sparkR.session(master = sparkRTestMaster, enableHiveSupport = FALSE, sparkConfig = sparkConfig) + + df <- arrange(createDataFrame(faithful), "waiting") + expect_is(df, "SparkDataFrame") + expected <- paste0("(+---------+-------+\n", + "|eruptions|waiting|\n", + "+---------+-------+\n", + "| 1.| 43|\n)*", + "(only showing top 5 rows)") + expect_output(show(df), expected) + + # Stop Spark session + sparkR.session.stop() +}) diff --git a/R/pkg/tests/fulltests/test_streaming.R b/R/pkg/tests/fulltests/test_streaming.R index bfb1a046490ec..6f0d2aefee886 100644 --- a/R/pkg/tests/fulltests/test_streaming.R +++ b/R/pkg/tests/fulltests/test_streaming.R @@ -127,6 +127,7 @@ test_that("Specify a schema by using a DDL-formatted string when reading", { expect_false(awaitTermination(q, 5 * 1000)) callJMethod(q@ssq, "processAllAvailable") expect_equal(head(sql("SELECT count(*) FROM people3"))[[1]], 3) + stopQuery(q) expect_error(read.stream(path = parquetPath, schema = "name stri"), "DataType stri is not supported.") diff --git a/R/pkg/vignettes/sparkr-vignettes.Rmd b/R/pkg/vignettes/sparkr-vignettes.Rmd index ad934947437bc..9e48ae34634e2 100644 --- a/R/pkg/vignettes/sparkr-vignettes.Rmd +++ b/R/pkg/vignettes/sparkr-vignettes.Rmd @@ -57,6 +57,20 @@ First, let's load and attach the package. library(SparkR) ``` +```{r, include=FALSE} +# disable eval if java version not supported +override_eval <- tryCatch(!is.numeric(SparkR:::checkJavaVersion()), + error = function(e) { TRUE }, + warning = function(e) { TRUE }) + +if (override_eval) { + opts_hooks$set(eval = function(options) { + options$eval = FALSE + options + }) +} +``` + `SparkSession` is the entry point into SparkR which connects your R program to a Spark cluster. You can create a `SparkSession` using `sparkR.session` and pass in options such as the application name, any Spark packages depended on, etc. We use default settings in which it runs in local mode. It auto downloads Spark package in the background if no previous installation is found. For more details about setup, see [Spark Session](#SetupSparkSession). @@ -205,7 +219,7 @@ SparkR supports operating on a variety of data sources through the `SparkDataFra The general method for creating `SparkDataFrame` from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active Spark Session will be used automatically. SparkR supports reading CSV, JSON and Parquet files natively and through Spark Packages you can find data source connectors for popular file formats like Avro. These packages can be added with `sparkPackages` parameter when initializing SparkSession using `sparkR.session`. ```{r, eval=FALSE} -sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +sparkR.session(sparkPackages = "com.databricks:spark-avro_2.12:3.0.0") ``` We can see how to use data sources using an example CSV input file. For more information please refer to SparkR [read.df](https://spark.apache.org/docs/latest/api/R/read.df.html) API documentation. @@ -535,6 +549,8 @@ SparkR supports the following machine learning models and algorithms. * Latent Dirichlet Allocation (LDA) +* Power Iteration Clustering (PIC) + #### Collaborative Filtering * Alternating Least Squares (ALS) @@ -542,6 +558,7 @@ SparkR supports the following machine learning models and algorithms. #### Frequent Pattern Mining * FP-growth +* PrefixSpan #### Statistics @@ -967,6 +984,18 @@ predicted <- predict(model, df) head(predicted) ``` +#### Power Iteration Clustering + +Power Iteration Clustering (PIC) is a scalable graph clustering algorithm. `spark.assignClusters` method runs the PIC algorithm and returns a cluster assignment for each input vertex. + +```{r} +df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) +head(spark.assignClusters(df, initMode = "degree", weightCol = "weight")) +``` + #### FP-growth `spark.fpGrowth` executes FP-growth algorithm to mine frequent itemsets on a `SparkDataFrame`. `itemsCol` should be an array of values. @@ -998,6 +1027,19 @@ We can make predictions based on the `antecedent`. head(predict(fpm, df)) ``` +#### PrefixSpan + +`spark.findFrequentSequentialPatterns` method can be used to find the complete set of frequent sequential patterns in the input sequences of itemsets. + +```{r} +df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) +head(spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L)) +``` + #### Kolmogorov-Smirnov Test `spark.kstest` runs a two-sided, one-sample [Kolmogorov-Smirnov (KS) test](https://en.wikipedia.org/wiki/Kolmogorov%E2%80%93Smirnov_test). diff --git a/README.md b/README.md index fd8c7f656968e..9759559e6cf6f 100644 --- a/README.md +++ b/README.md @@ -1,24 +1,28 @@ # Apache Spark -Spark is a fast and general cluster computing system for Big Data. It provides +Spark is a unified analytics engine for large-scale data processing. It provides high-level APIs in Scala, Java, Python, and R, and an optimized engine that supports general computation graphs for data analysis. It also supports a rich set of higher-level tools including Spark SQL for SQL and DataFrames, MLlib for machine learning, GraphX for graph processing, -and Spark Streaming for stream processing. +and Structured Streaming for stream processing. - + + +[![Jenkins Build](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7/badge/icon)](https://amplab.cs.berkeley.edu/jenkins/job/spark-master-test-sbt-hadoop-2.7) +[![AppVeyor Build](https://img.shields.io/appveyor/ci/ApacheSoftwareFoundation/spark/master.svg?style=plastic&logo=appveyor)](https://ci.appveyor.com/project/ApacheSoftwareFoundation/spark) +[![PySpark Coverage](https://img.shields.io/badge/dynamic/xml.svg?label=pyspark%20coverage&url=https%3A%2F%2Fspark-test.github.io%2Fpyspark-coverage-site&query=%2Fhtml%2Fbody%2Fdiv%5B1%5D%2Fdiv%2Fh1%2Fspan&colorB=brightgreen&style=plastic)](https://spark-test.github.io/pyspark-coverage-site) ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the [project web page](http://spark.apache.org/documentation.html). +guide, on the [project web page](https://spark.apache.org/documentation.html). This README file only contains basic setup instructions. ## Building Spark -Spark is built using [Apache Maven](http://maven.apache.org/). +Spark is built using [Apache Maven](https://maven.apache.org/). To build Spark and its example programs, run: build/mvn -DskipTests clean package @@ -27,9 +31,9 @@ To build Spark and its example programs, run: You can build Spark using more than one thread by using the -T option with Maven, see ["Parallel builds in Maven 3"](https://cwiki.apache.org/confluence/display/MAVEN/Parallel+builds+in+Maven+3). More detailed documentation is available from the project site, at -["Building Spark"](http://spark.apache.org/docs/latest/building-spark.html). +["Building Spark"](https://spark.apache.org/docs/latest/building-spark.html). -For general development tips, including info on developing Spark using an IDE, see ["Useful Developer Tools"](http://spark.apache.org/developer-tools.html). +For general development tips, including info on developing Spark using an IDE, see ["Useful Developer Tools"](https://spark.apache.org/developer-tools.html). ## Interactive Scala Shell @@ -37,9 +41,9 @@ The easiest way to start using Spark is through the Scala shell: ./bin/spark-shell -Try the following command, which should return 1000: +Try the following command, which should return 1,000,000,000: - scala> sc.parallelize(1 to 1000).count() + scala> spark.range(1000 * 1000 * 1000).count() ## Interactive Python Shell @@ -47,9 +51,9 @@ Alternatively, if you prefer Python, you can use the Python shell: ./bin/pyspark -And run the following command, which should also return 1000: +And run the following command, which should also return 1,000,000,000: - >>> sc.parallelize(range(1000)).count() + >>> spark.range(1000 * 1000 * 1000).count() ## Example Programs @@ -79,7 +83,7 @@ can be run using: ./dev/run-tests Please see the guidance on how to -[run tests for a module, or individual tests](http://spark.apache.org/developer-tools.html#individual-tests). +[run tests for a module, or individual tests](https://spark.apache.org/developer-tools.html#individual-tests). There is also a Kubernetes integration test, see resource-managers/kubernetes/integration-tests/README.md @@ -90,16 +94,16 @@ storage systems. Because the protocols have changed in different versions of Hadoop, you must build Spark against the same version that your cluster runs. Please refer to the build documentation at -["Specifying the Hadoop Version and Enabling YARN"](http://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version-and-enabling-yarn) +["Specifying the Hadoop Version and Enabling YARN"](https://spark.apache.org/docs/latest/building-spark.html#specifying-the-hadoop-version-and-enabling-yarn) for detailed guidance on building for a particular distribution of Hadoop, including building for particular Hive and Hive Thriftserver distributions. ## Configuration -Please refer to the [Configuration Guide](http://spark.apache.org/docs/latest/configuration.html) +Please refer to the [Configuration Guide](https://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. ## Contributing -Please review the [Contribution to Spark guide](http://spark.apache.org/contributing.html) +Please review the [Contribution to Spark guide](https://spark.apache.org/contributing.html) for information on how to get started contributing to the project. diff --git a/appveyor.yml b/appveyor.yml index 7fb45745a036f..8fb090cb0c603 100644 --- a/appveyor.yml +++ b/appveyor.yml @@ -42,9 +42,12 @@ install: # Install maven and dependencies - ps: .\dev\appveyor-install-dependencies.ps1 # Required package for R unit tests - - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='http://cran.us.r-project.org')" + - cmd: R -e "install.packages(c('knitr', 'rmarkdown', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" # Here, we use the fixed version of testthat. For more details, please see SPARK-22817. - - cmd: R -e "devtools::install_version('testthat', version = '1.0.2', repos='http://cran.us.r-project.org')" + # As of devtools 2.1.0, it requires testthat higher then 2.1.1 as a dependency. SparkR test requires testthat 1.0.2. + # Therefore, we don't use devtools but installs it directly from the archive including its dependencies. + - cmd: R -e "install.packages(c('crayon', 'praise', 'R6'), repos='https://cloud.r-project.org/')" + - cmd: R -e "install.packages('https://cloud.r-project.org/src/contrib/Archive/testthat/testthat_1.0.2.tar.gz', repos=NULL, type='source')" - cmd: R -e "packageVersion('knitr'); packageVersion('rmarkdown'); packageVersion('testthat'); packageVersion('e1071'); packageVersion('survival')" build_script: @@ -52,6 +55,10 @@ build_script: environment: NOT_CRAN: true + # See SPARK-27848. Currently installing some dependent packagess causes + # "(converted from warning) unable to identify current timezone 'C':" for an unknown reason. + # This environment variable works around to test SparkR against a higher version. + R_REMOTES_NO_ERRORS_FROM_WARNINGS: true test_script: - cmd: .\bin\spark-submit2.cmd --driver-java-options "-Dlog4j.configuration=file:///%CD:\=/%/R/log4j.properties" --conf spark.hadoop.fs.defaultFS="file:///" R\pkg\tests\run-all.R diff --git a/assembly/README b/assembly/README index d5dafab477410..1fd6d8858348c 100644 --- a/assembly/README +++ b/assembly/README @@ -9,4 +9,4 @@ This module is off by default. To activate it specify the profile in the command If you need to build an assembly for a different version of Hadoop the hadoop-version system property needs to be set as in this example: - -Dhadoop.version=2.7.3 + -Dhadoop.version=2.7.4 diff --git a/assembly/pom.xml b/assembly/pom.xml index b0337e58cca71..ef916fb99a04c 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-assembly_2.11 + spark-assembly_2.12 Spark Project Assembly http://spark.apache.org/ pom @@ -64,6 +64,11 @@ spark-graphx_${scala.binary.version} ${project.version} + + org.apache.spark + spark-graph_${scala.binary.version} + ${project.version} + org.apache.spark spark-sql_${scala.binary.version} diff --git a/bin/docker-image-tool.sh b/bin/docker-image-tool.sh index 228494de6d5a1..68fafbb848001 100755 --- a/bin/docker-image-tool.sh +++ b/bin/docker-image-tool.sh @@ -29,6 +29,20 @@ if [ -z "${SPARK_HOME}" ]; then fi . "${SPARK_HOME}/bin/load-spark-env.sh" +CTX_DIR="$SPARK_HOME/target/tmp/docker" + +function is_dev_build { + [ ! -f "$SPARK_HOME/RELEASE" ] +} + +function cleanup_ctx_dir { + if is_dev_build; then + rm -rf "$CTX_DIR" + fi +} + +trap cleanup_ctx_dir EXIT + function image_ref { local image="$1" local add_repo="${2:-1}" @@ -41,57 +55,154 @@ function image_ref { echo "$image" } +function docker_push { + local image_name="$1" + if [ ! -z $(docker images -q "$(image_ref ${image_name})") ]; then + docker push "$(image_ref ${image_name})" + if [ $? -ne 0 ]; then + error "Failed to push $image_name Docker image." + fi + else + echo "$(image_ref ${image_name}) image not found. Skipping push for this image." + fi +} + +function resolve_file { + local FILE=$1 + if [ -n "$FILE" ]; then + local DIR=$(dirname $FILE) + DIR=$(cd $DIR && pwd) + FILE="${DIR}/$(basename $FILE)" + fi + echo $FILE +} + +# Create a smaller build context for docker in dev builds to make the build faster. Docker +# uploads all of the current directory to the daemon, and it can get pretty big with dev +# builds that contain test log files and other artifacts. +# +# Three build contexts are created, one for each image: base, pyspark, and sparkr. For them +# to have the desired effect, the docker command needs to be executed inside the appropriate +# context directory. +# +# Note: docker does not support symlinks in the build context. +function create_dev_build_context {( + set -e + local BASE_CTX="$CTX_DIR/base" + mkdir -p "$BASE_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$BASE_CTX/kubernetes/dockerfiles" + + cp -r "assembly/target/scala-$SPARK_SCALA_VERSION/jars" "$BASE_CTX/jars" + cp -r "resource-managers/kubernetes/integration-tests/tests" \ + "$BASE_CTX/kubernetes/tests" + + mkdir "$BASE_CTX/examples" + cp -r "examples/src" "$BASE_CTX/examples/src" + # Copy just needed examples jars instead of everything. + mkdir "$BASE_CTX/examples/jars" + for i in examples/target/scala-$SPARK_SCALA_VERSION/jars/*; do + if [ ! -f "$BASE_CTX/jars/$(basename $i)" ]; then + cp $i "$BASE_CTX/examples/jars" + fi + done + + for other in bin sbin data; do + cp -r "$other" "$BASE_CTX/$other" + done + + local PYSPARK_CTX="$CTX_DIR/pyspark" + mkdir -p "$PYSPARK_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$PYSPARK_CTX/kubernetes/dockerfiles" + mkdir "$PYSPARK_CTX/python" + cp -r "python/lib" "$PYSPARK_CTX/python/lib" + cp -r "python/pyspark" "$PYSPARK_CTX/python/pyspark" + + local R_CTX="$CTX_DIR/sparkr" + mkdir -p "$R_CTX/kubernetes" + cp -r "resource-managers/kubernetes/docker/src/main/dockerfiles" \ + "$R_CTX/kubernetes/dockerfiles" + cp -r "R" "$R_CTX/R" +)} + +function img_ctx_dir { + if is_dev_build; then + echo "$CTX_DIR/$1" + else + echo "$SPARK_HOME" + fi +} + function build { local BUILD_ARGS - local IMG_PATH - - if [ ! -f "$SPARK_HOME/RELEASE" ]; then - # Set image build arguments accordingly if this is a source repo and not a distribution archive. - IMG_PATH=resource-managers/kubernetes/docker/src/main/dockerfiles - BUILD_ARGS=( - ${BUILD_PARAMS} - --build-arg - img_path=$IMG_PATH - --build-arg - spark_jars=assembly/target/scala-$SPARK_SCALA_VERSION/jars - --build-arg - k8s_tests=resource-managers/kubernetes/integration-tests/tests - ) - else - # Not passed as an argument to docker, but used to validate the Spark directory. - IMG_PATH="kubernetes/dockerfiles" - BUILD_ARGS=(${BUILD_PARAMS}) + local SPARK_ROOT="$SPARK_HOME" + + if is_dev_build; then + create_dev_build_context || error "Failed to create docker build context." + SPARK_ROOT="$CTX_DIR/base" fi - if [ ! -d "$IMG_PATH" ]; then + # Verify that the Docker image content directory is present + if [ ! -d "$SPARK_ROOT/kubernetes/dockerfiles" ]; then error "Cannot find docker image. This script must be run from a runnable distribution of Apache Spark." fi + + # Verify that Spark has actually been built/is a runnable distribution + # i.e. the Spark JARs that the Docker files will place into the image are present + local TOTAL_JARS=$(ls $SPARK_ROOT/jars/spark-* | wc -l) + TOTAL_JARS=$(( $TOTAL_JARS )) + if [ "${TOTAL_JARS}" -eq 0 ]; then + error "Cannot find Spark JARs. This script assumes that Apache Spark has first been built locally or this is a runnable distribution." + fi + + local BUILD_ARGS=(${BUILD_PARAMS}) + + # If a custom SPARK_UID was set add it to build arguments + if [ -n "$SPARK_UID" ]; then + BUILD_ARGS+=(--build-arg spark_uid=$SPARK_UID) + fi + local BINDING_BUILD_ARGS=( - ${BUILD_PARAMS} + ${BUILD_ARGS[@]} --build-arg base_img=$(image_ref spark) ) - local BASEDOCKERFILE=${BASEDOCKERFILE:-"$IMG_PATH/spark/Dockerfile"} - local PYDOCKERFILE=${PYDOCKERFILE:-"$IMG_PATH/spark/bindings/python/Dockerfile"} - local RDOCKERFILE=${RDOCKERFILE:-"$IMG_PATH/spark/bindings/R/Dockerfile"} - docker build $NOCACHEARG "${BUILD_ARGS[@]}" \ + local BASEDOCKERFILE=${BASEDOCKERFILE:-"kubernetes/dockerfiles/spark/Dockerfile"} + local PYDOCKERFILE=${PYDOCKERFILE:-false} + local RDOCKERFILE=${RDOCKERFILE:-false} + + (cd $(img_ctx_dir base) && docker build $NOCACHEARG "${BUILD_ARGS[@]}" \ -t $(image_ref spark) \ - -f "$BASEDOCKERFILE" . + -f "$BASEDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build Spark JVM Docker image, please refer to Docker build output for details." + fi - docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ - -t $(image_ref spark-py) \ - -f "$PYDOCKERFILE" . + if [ "${PYDOCKERFILE}" != "false" ]; then + (cd $(img_ctx_dir pyspark) && docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ + -t $(image_ref spark-py) \ + -f "$PYDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build PySpark Docker image, please refer to Docker build output for details." + fi + fi - docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ - -t $(image_ref spark-r) \ - -f "$RDOCKERFILE" . + if [ "${RDOCKERFILE}" != "false" ]; then + (cd $(img_ctx_dir sparkr) && docker build $NOCACHEARG "${BINDING_BUILD_ARGS[@]}" \ + -t $(image_ref spark-r) \ + -f "$RDOCKERFILE" .) + if [ $? -ne 0 ]; then + error "Failed to build SparkR Docker image, please refer to Docker build output for details." + fi + fi } function push { - docker push "$(image_ref spark)" - docker push "$(image_ref spark-py)" - docker push "$(image_ref spark-r)" + docker_push "spark" + docker_push "spark-py" + docker_push "spark-r" } function usage { @@ -106,14 +217,18 @@ Commands: Options: -f file Dockerfile to build for JVM based Jobs. By default builds the Dockerfile shipped with Spark. - -p file Dockerfile to build for PySpark Jobs. Builds Python dependencies and ships with Spark. - -R file Dockerfile to build for SparkR Jobs. Builds R dependencies and ships with Spark. + -p file (Optional) Dockerfile to build for PySpark Jobs. Builds Python dependencies and ships with Spark. + Skips building PySpark docker image if not specified. + -R file (Optional) Dockerfile to build for SparkR Jobs. Builds R dependencies and ships with Spark. + Skips building SparkR docker image if not specified. -r repo Repository address. -t tag Tag to apply to the built image, or to identify the image to be pushed. -m Use minikube's Docker daemon. -n Build docker image with --no-cache - -b arg Build arg to build or push the image. For multiple build args, this option needs to - be used separately for each build arg. + -u uid UID to use in the USER directive to set the user the main Spark process runs as inside the + resulting container + -b arg Build arg to build or push the image. For multiple build args, this option needs to + be used separately for each build arg. Using minikube when building images will do so directly into minikube's Docker daemon. There is no need to push the images into minikube in that case, they'll be automatically @@ -127,6 +242,9 @@ Examples: - Build image in minikube with tag "testing" $0 -m -t testing build + - Build PySpark docker image + $0 -r docker.io/myrepo -t v2.3.0 -p kubernetes/dockerfiles/spark/bindings/python/Dockerfile build + - 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 @@ -145,13 +263,14 @@ PYDOCKERFILE= RDOCKERFILE= NOCACHEARG= BUILD_PARAMS= -while getopts f:p:R:mr:t:n:b: option +SPARK_UID= +while getopts f:p:R:mr:t:nb:u: option do case "${option}" in - f) BASEDOCKERFILE=${OPTARG};; - p) PYDOCKERFILE=${OPTARG};; - R) RDOCKERFILE=${OPTARG};; + f) BASEDOCKERFILE=$(resolve_file ${OPTARG});; + p) PYDOCKERFILE=$(resolve_file ${OPTARG});; + R) RDOCKERFILE=$(resolve_file ${OPTARG});; r) REPO=${OPTARG};; t) TAG=${OPTARG};; n) NOCACHEARG="--no-cache";; @@ -160,8 +279,12 @@ do if ! which minikube 1>/dev/null; then error "Cannot find minikube." fi - eval $(minikube docker-env) + if ! minikube status 1>/dev/null; then + error "Cannot contact minikube. Make sure it's running." + fi + eval $(minikube docker-env --shell bash) ;; + u) SPARK_UID=${OPTARG};; esac done diff --git a/bin/load-spark-env.cmd b/bin/load-spark-env.cmd index cefa513b6fb77..5f98cc34b6bab 100644 --- a/bin/load-spark-env.cmd +++ b/bin/load-spark-env.cmd @@ -21,6 +21,7 @@ rem This script loads spark-env.cmd if it exists, and ensures it is only loaded rem spark-env.cmd is loaded from SPARK_CONF_DIR if set, or within the current directory's rem conf\ subdirectory. +set SPARK_ENV_CMD=spark-env.cmd if [%SPARK_ENV_LOADED%] == [] ( set SPARK_ENV_LOADED=1 @@ -28,30 +29,34 @@ if [%SPARK_ENV_LOADED%] == [] ( set SPARK_CONF_DIR=%~dp0..\conf ) - call :LoadSparkEnv + set SPARK_ENV_CMD=%SPARK_CONF_DIR%\%SPARK_ENV_CMD% + if exist %SPARK_ENV_CMD% ( + call %SPARK_ENV_CMD% + ) ) rem Setting SPARK_SCALA_VERSION if not already set. -set ASSEMBLY_DIR2="%SPARK_HOME%\assembly\target\scala-2.11" -set ASSEMBLY_DIR1="%SPARK_HOME%\assembly\target\scala-2.12" - -if [%SPARK_SCALA_VERSION%] == [] ( - - if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( - echo "Presence of build for multiple Scala versions detected." - echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." - exit 1 - ) - if exist %ASSEMBLY_DIR2% ( - set SPARK_SCALA_VERSION=2.11 - ) else ( - set SPARK_SCALA_VERSION=2.12 - ) -) +rem TODO: revisit for Scala 2.13 support +set SPARK_SCALA_VERSION=2.12 +rem if [%SPARK_SCALA_VERSION%] == [] ( +rem set SCALA_VERSION_1=2.12 +rem set SCALA_VERSION_2=2.11 +rem +rem set ASSEMBLY_DIR1=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_1% +rem set ASSEMBLY_DIR2=%SPARK_HOME%\assembly\target\scala-%SCALA_VERSION_2% +rem set ENV_VARIABLE_DOC=https://spark.apache.org/docs/latest/configuration.html#environment-variables +rem if exist %ASSEMBLY_DIR2% if exist %ASSEMBLY_DIR1% ( +rem echo "Presence of build for multiple Scala versions detected (%ASSEMBLY_DIR1% and %ASSEMBLY_DIR2%)." +rem echo "Remove one of them or, set SPARK_SCALA_VERSION=%SCALA_VERSION_1% in %SPARK_ENV_CMD%." +rem echo "Visit %ENV_VARIABLE_DOC% for more details about setting environment variables in spark-env.cmd." +rem echo "Either clean one of them or, set SPARK_SCALA_VERSION in spark-env.cmd." +rem exit 1 +rem ) +rem if exist %ASSEMBLY_DIR1% ( +rem set SPARK_SCALA_VERSION=%SCALA_VERSION_1% +rem ) else ( +rem set SPARK_SCALA_VERSION=%SCALA_VERSION_2% +rem ) +rem ) exit /b 0 - -:LoadSparkEnv -if exist "%SPARK_CONF_DIR%\spark-env.cmd" ( - call "%SPARK_CONF_DIR%\spark-env.cmd" -) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 0b5006dbd63ac..107e7991c28bc 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -26,35 +26,42 @@ if [ -z "${SPARK_HOME}" ]; then source "$(dirname "$0")"/find-spark-home fi +SPARK_ENV_SH="spark-env.sh" if [ -z "$SPARK_ENV_LOADED" ]; then export SPARK_ENV_LOADED=1 export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}"/conf}" - if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then + SPARK_ENV_SH="${SPARK_CONF_DIR}/${SPARK_ENV_SH}" + if [[ -f "${SPARK_ENV_SH}" ]]; then # Promote all variable declarations to environment (exported) variables set -a - . "${SPARK_CONF_DIR}/spark-env.sh" + . ${SPARK_ENV_SH} set +a fi fi # Setting SPARK_SCALA_VERSION if not already set. -if [ -z "$SPARK_SCALA_VERSION" ]; then - - ASSEMBLY_DIR2="${SPARK_HOME}/assembly/target/scala-2.11" - ASSEMBLY_DIR1="${SPARK_HOME}/assembly/target/scala-2.12" - - if [[ -d "$ASSEMBLY_DIR2" && -d "$ASSEMBLY_DIR1" ]]; then - echo -e "Presence of build for multiple Scala versions detected." 1>&2 - echo -e 'Either clean one of them or, export SPARK_SCALA_VERSION in spark-env.sh.' 1>&2 - exit 1 - fi - - if [ -d "$ASSEMBLY_DIR2" ]; then - export SPARK_SCALA_VERSION="2.11" - else - export SPARK_SCALA_VERSION="2.12" - fi -fi +# TODO: revisit for Scala 2.13 support +export SPARK_SCALA_VERSION=2.12 +#if [ -z "$SPARK_SCALA_VERSION" ]; then +# SCALA_VERSION_1=2.12 +# SCALA_VERSION_2=2.11 +# +# ASSEMBLY_DIR_1="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_1}" +# ASSEMBLY_DIR_2="${SPARK_HOME}/assembly/target/scala-${SCALA_VERSION_2}" +# ENV_VARIABLE_DOC="https://spark.apache.org/docs/latest/configuration.html#environment-variables" +# if [[ -d "$ASSEMBLY_DIR_1" && -d "$ASSEMBLY_DIR_2" ]]; then +# echo "Presence of build for multiple Scala versions detected ($ASSEMBLY_DIR_1 and $ASSEMBLY_DIR_2)." 1>&2 +# echo "Remove one of them or, export SPARK_SCALA_VERSION=$SCALA_VERSION_1 in ${SPARK_ENV_SH}." 1>&2 +# echo "Visit ${ENV_VARIABLE_DOC} for more details about setting environment variables in spark-env.sh." 1>&2 +# exit 1 +# fi +# +# if [[ -d "$ASSEMBLY_DIR_1" ]]; then +# export SPARK_SCALA_VERSION=${SCALA_VERSION_1} +# else +# export SPARK_SCALA_VERSION=${SCALA_VERSION_2} +# fi +#fi diff --git a/bin/pyspark b/bin/pyspark index 5d5affb1f97c3..44891aee2e0a3 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -38,26 +38,19 @@ if [[ -n "$IPYTHON" || -n "$IPYTHON_OPTS" ]]; then fi # Default to standard python interpreter unless told otherwise -if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then - PYSPARK_DRIVER_PYTHON="${PYSPARK_PYTHON:-"python"}" -fi - -WORKS_WITH_IPYTHON=$(python -c 'import sys; print(sys.version_info >= (2, 7, 0))') - -# Determine the Python executable to use for the executors: if [[ -z "$PYSPARK_PYTHON" ]]; then - if [[ $PYSPARK_DRIVER_PYTHON == *ipython* && ! $WORKS_WITH_IPYTHON ]]; then - echo "IPython requires Python 2.7+; please install python2.7 or set PYSPARK_PYTHON" 1>&2 - exit 1 - else - PYSPARK_PYTHON=python - fi + PYSPARK_PYTHON=python +fi +if [[ -z "$PYSPARK_DRIVER_PYTHON" ]]; then + PYSPARK_DRIVER_PYTHON=$PYSPARK_PYTHON fi export PYSPARK_PYTHON +export PYSPARK_DRIVER_PYTHON +export PYSPARK_DRIVER_PYTHON_OPTS # Add the PySpark classes to the Python path: export PYTHONPATH="${SPARK_HOME}/python/:$PYTHONPATH" -export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-src.zip:$PYTHONPATH" +export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.8.1-src.zip:$PYTHONPATH" # Load the PySpark shell.py script when ./pyspark is used interactively: export OLD_PYTHONSTARTUP="$PYTHONSTARTUP" @@ -72,6 +65,4 @@ if [[ -n "$SPARK_TESTING" ]]; then exit fi -export PYSPARK_DRIVER_PYTHON -export PYSPARK_DRIVER_PYTHON_OPTS exec "${SPARK_HOME}"/bin/spark-submit pyspark-shell-main --name "PySparkShell" "$@" diff --git a/bin/pyspark2.cmd b/bin/pyspark2.cmd index 15fa910c277b3..479fd464c7d3e 100644 --- a/bin/pyspark2.cmd +++ b/bin/pyspark2.cmd @@ -30,7 +30,7 @@ if "x%PYSPARK_DRIVER_PYTHON%"=="x" ( ) set PYTHONPATH=%SPARK_HOME%\python;%PYTHONPATH% -set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.7-src.zip;%PYTHONPATH% +set PYTHONPATH=%SPARK_HOME%\python\lib\py4j-0.10.8.1-src.zip;%PYTHONPATH% set OLD_PYTHONSTARTUP=%PYTHONSTARTUP% set PYTHONSTARTUP=%SPARK_HOME%\python\pyspark\shell.py diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 5da7d7a430d79..34d04c9856d2c 100644 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -63,7 +63,12 @@ if not "x%JAVA_HOME%"=="x" ( rem The launcher library prints the command to be executed in a single line suitable for being rem executed by the batch interpreter. So read all the output of the launcher into a variable. +:gen set LAUNCHER_OUTPUT=%temp%\spark-class-launcher-output-%RANDOM%.txt +rem SPARK-28302: %RANDOM% would return the same number if we call it instantly after last call, +rem so we should make it sure to generate unique file to avoid process collision of writing into +rem the same file concurrently. +if exist %LAUNCHER_OUTPUT% goto :gen "%RUNNER%" -Xmx128m -cp "%LAUNCH_CLASSPATH%" org.apache.spark.launcher.Main %* > %LAUNCHER_OUTPUT% for /f "tokens=*" %%i in (%LAUNCHER_OUTPUT%) do ( set SPARK_CMD=%%i diff --git a/bin/spark-shell b/bin/spark-shell index 421f36cac3d47..e920137974980 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -32,7 +32,10 @@ if [ -z "${SPARK_HOME}" ]; then source "$(dirname "$0")"/find-spark-home fi -export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options]" +export _SPARK_CMD_USAGE="Usage: ./bin/spark-shell [options] + +Scala REPL options: + -I preload , enforcing line-by-line interpretation" # SPARK-4161: scala does not assume use of the java classpath, # so we need to add the "-Dscala.usejavacp=true" flag manually. We diff --git a/bin/spark-shell2.cmd b/bin/spark-shell2.cmd index aaf71906c6526..549bf43bb6078 100644 --- a/bin/spark-shell2.cmd +++ b/bin/spark-shell2.cmd @@ -20,7 +20,13 @@ rem rem Figure out where the Spark framework is installed call "%~dp0find-spark-home.cmd" -set _SPARK_CMD_USAGE=Usage: .\bin\spark-shell.cmd [options] +set LF=^ + + +rem two empty lines are required +set _SPARK_CMD_USAGE=Usage: .\bin\spark-shell.cmd [options]^%LF%%LF%^%LF%%LF%^ +Scala REPL options:^%LF%%LF%^ + -I ^ preload ^, enforcing line-by-line interpretation rem SPARK-4161: scala does not assume use of the java classpath, rem so we need to add the "-Dscala.usejavacp=true" flag manually. We diff --git a/build/mvn b/build/mvn index 2487b81abb4ea..75feb2f718d12 100755 --- a/build/mvn +++ b/build/mvn @@ -116,7 +116,8 @@ install_zinc() { # the build/ folder install_scala() { # determine the Scala version used in Spark - local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}'` + local scala_binary_version=`grep "scala.binary.version" "${_DIR}/../pom.xml" | head -n1 | awk -F '[<>]' '{print $3}'` + local scala_version=`grep "scala.version" "${_DIR}/../pom.xml" | grep ${scala_binary_version} | head -n1 | awk -F '[<>]' '{print $3}'` local scala_bin="${_DIR}/scala-${scala_version}/bin/scala" local TYPESAFE_MIRROR=${TYPESAFE_MIRROR:-https://downloads.lightbend.com} @@ -133,12 +134,6 @@ install_scala() { # the environment ZINC_PORT=${ZINC_PORT:-"3030"} -# Remove `--force` for backward compatibility. -if [ "$1" == "--force" ]; then - echo "WARNING: '--force' is deprecated and ignored." - shift -fi - # Install the proper version of Scala, Zinc and Maven for the build install_zinc install_scala @@ -153,6 +148,7 @@ if [ -n "${ZINC_INSTALL_FLAG}" -o -z "`"${ZINC_BIN}" -status -port ${ZINC_PORT}` export ZINC_OPTS=${ZINC_OPTS:-"$_COMPILE_JVM_OPTS"} "${ZINC_BIN}" -shutdown -port ${ZINC_PORT} "${ZINC_BIN}" -start -port ${ZINC_PORT} \ + -server 127.0.0.1 -idle-timeout 3h \ -scala-compiler "${SCALA_COMPILER}" \ -scala-library "${SCALA_LIBRARY}" &>/dev/null fi @@ -162,5 +158,12 @@ export MAVEN_OPTS=${MAVEN_OPTS:-"$_COMPILE_JVM_OPTS"} echo "Using \`mvn\` from path: $MVN_BIN" 1>&2 -# Last, call the `mvn` command as usual +# call the `mvn` command as usual +# SPARK-25854 "${MVN_BIN}" -DzincPort=${ZINC_PORT} "$@" +MVN_RETCODE=$? + +# Try to shut down zinc explicitly if the server is still running. +"${ZINC_BIN}" -shutdown -port ${ZINC_PORT} + +exit $MVN_RETCODE diff --git a/build/sbt-launch-lib.bash b/build/sbt-launch-lib.bash index 4732669ee651f..0ed6f8b6d737b 100755 --- a/build/sbt-launch-lib.bash +++ b/build/sbt-launch-lib.bash @@ -56,13 +56,13 @@ acquire_sbt_jar () { wget --quiet ${URL1} -O "${JAR_DL}" &&\ mv "${JAR_DL}" "${JAR}" else - printf "You do not have curl or wget installed, please install sbt manually from http://www.scala-sbt.org/\n" + printf "You do not have curl or wget installed, please install sbt manually from https://www.scala-sbt.org/\n" exit -1 fi fi if [ ! -f "${JAR}" ]; then # We failed to download - printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from http://www.scala-sbt.org/\n" + printf "Our attempt to download sbt locally to ${JAR} failed. Please install sbt manually from https://www.scala-sbt.org/\n" exit -1 fi printf "Launching sbt from ${JAR}\n" diff --git a/common/kvstore/pom.xml b/common/kvstore/pom.xml index 23a0f49206909..f042a12fda3d2 100644 --- a/common/kvstore/pom.xml +++ b/common/kvstore/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-kvstore_2.11 + spark-kvstore_2.12 jar Spark Project Local DB http://spark.apache.org/ diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java index 5ca4371285198..6af45aec3c7b2 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/InMemoryStore.java @@ -21,16 +21,18 @@ import java.util.Collection; import java.util.Collections; import java.util.Iterator; +import java.util.HashSet; import java.util.List; import java.util.NoSuchElementException; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; +import java.util.function.BiConsumer; +import java.util.function.Predicate; import java.util.stream.Collectors; import java.util.stream.Stream; import com.google.common.base.Objects; import com.google.common.base.Preconditions; -import com.google.common.base.Throwables; import org.apache.spark.annotation.Private; @@ -43,7 +45,7 @@ public class InMemoryStore implements KVStore { private Object metadata; - private ConcurrentMap, InstanceList> data = new ConcurrentHashMap<>(); + private InMemoryLists inMemoryLists = new InMemoryLists(); @Override public T getMetadata(Class klass) { @@ -57,13 +59,13 @@ public void setMetadata(Object value) { @Override public long count(Class type) { - InstanceList list = data.get(type); + InstanceList list = inMemoryLists.get(type); return list != null ? list.size() : 0; } @Override public long count(Class type, String index, Object indexedValue) throws Exception { - InstanceList list = data.get(type); + InstanceList list = inMemoryLists.get(type); int count = 0; Object comparable = asKey(indexedValue); KVTypeInfo.Accessor accessor = list.getIndexAccessor(index); @@ -77,29 +79,22 @@ public long count(Class type, String index, Object indexedValue) throws Excep @Override public T read(Class klass, Object naturalKey) { - InstanceList list = data.get(klass); - Object value = list != null ? list.get(naturalKey) : null; + InstanceList list = inMemoryLists.get(klass); + T value = list != null ? list.get(naturalKey) : null; if (value == null) { throw new NoSuchElementException(); } - return klass.cast(value); + return value; } @Override public void write(Object value) throws Exception { - InstanceList list = data.computeIfAbsent(value.getClass(), key -> { - try { - return new InstanceList(key); - } catch (Exception e) { - throw Throwables.propagate(e); - } - }); - list.put(value); + inMemoryLists.write(value); } @Override public void delete(Class type, Object naturalKey) { - InstanceList list = data.get(type); + InstanceList list = inMemoryLists.get(type); if (list != null) { list.delete(naturalKey); } @@ -107,15 +102,28 @@ public void delete(Class type, Object naturalKey) { @Override public KVStoreView view(Class type){ - InstanceList list = data.get(type); - return list != null ? list.view(type) - : new InMemoryView<>(type, Collections.emptyList(), null); + InstanceList list = inMemoryLists.get(type); + return list != null ? list.view() : emptyView(); } @Override public void close() { metadata = null; - data.clear(); + inMemoryLists.clear(); + } + + @Override + public boolean removeAllByIndexValues( + Class klass, + String index, + Collection indexValues) { + InstanceList list = inMemoryLists.get(klass); + + if (list != null) { + return list.countingRemoveAllByIndexValues(index, indexValues) > 0; + } else { + return false; + } } @SuppressWarnings("unchecked") @@ -126,64 +134,150 @@ private static Comparable asKey(Object in) { return (Comparable) in; } - private static class InstanceList { + @SuppressWarnings("unchecked") + private static KVStoreView emptyView() { + return (InMemoryView) InMemoryView.EMPTY_VIEW; + } + + /** + * Encapsulates ConcurrentHashMap so that the typing in and out of the map strictly maps a + * class of type T to an InstanceList of type T. + */ + private static class InMemoryLists { + private final ConcurrentMap, InstanceList> data = new ConcurrentHashMap<>(); + + @SuppressWarnings("unchecked") + public InstanceList get(Class type) { + return (InstanceList) data.get(type); + } + + @SuppressWarnings("unchecked") + public void write(T value) throws Exception { + InstanceList list = + (InstanceList) data.computeIfAbsent(value.getClass(), InstanceList::new); + list.put(value); + } + + public void clear() { + data.clear(); + } + } + + private static class InstanceList { + + /** + * A BiConsumer to control multi-entity removal. We use this in a forEach rather than an + * iterator because there is a bug in jdk8 which affects remove() on all concurrent map + * iterators. https://bugs.openjdk.java.net/browse/JDK-8078645 + */ + private static class CountingRemoveIfForEach implements BiConsumer, T> { + private final ConcurrentMap, T> data; + private final Predicate filter; + + /** + * Keeps a count of the number of elements removed. This count is not currently surfaced + * to clients of KVStore as Java's generic removeAll() construct returns only a boolean, + * but I found it handy to have the count of elements removed while debugging; a count being + * no more complicated than a boolean, I've retained that behavior here, even though there + * is no current requirement. + */ + private int count = 0; + + CountingRemoveIfForEach( + ConcurrentMap, T> data, + Predicate filter) { + this.data = data; + this.filter = filter; + } + + @Override + public void accept(Comparable key, T value) { + if (filter.test(value)) { + if (data.remove(key, value)) { + count++; + } + } + } + + public int count() { return count; } + } private final KVTypeInfo ti; private final KVTypeInfo.Accessor naturalKey; - private final ConcurrentMap, Object> data; - - private int size; + private final ConcurrentMap, T> data; - private InstanceList(Class type) throws Exception { - this.ti = new KVTypeInfo(type); + private InstanceList(Class klass) { + this.ti = new KVTypeInfo(klass); this.naturalKey = ti.getAccessor(KVIndex.NATURAL_INDEX_NAME); this.data = new ConcurrentHashMap<>(); - this.size = 0; } KVTypeInfo.Accessor getIndexAccessor(String indexName) { return ti.getAccessor(indexName); } - public Object get(Object key) { + int countingRemoveAllByIndexValues(String index, Collection indexValues) { + Predicate filter = getPredicate(ti.getAccessor(index), indexValues); + CountingRemoveIfForEach callback = new CountingRemoveIfForEach<>(data, filter); + + data.forEach(callback); + return callback.count(); + } + + public T get(Object key) { return data.get(asKey(key)); } - public void put(Object value) throws Exception { - Preconditions.checkArgument(ti.type().equals(value.getClass()), - "Unexpected type: %s", value.getClass()); - if (data.put(asKey(naturalKey.get(value)), value) == null) { - size++; - } + public void put(T value) throws Exception { + data.put(asKey(naturalKey.get(value)), value); } public void delete(Object key) { - if (data.remove(asKey(key)) != null) { - size--; - } + data.remove(asKey(key)); } public int size() { - return size; + return data.size(); } - @SuppressWarnings("unchecked") - public InMemoryView view(Class type) { - Preconditions.checkArgument(ti.type().equals(type), "Unexpected type: %s", type); - Collection all = (Collection) data.values(); - return new InMemoryView<>(type, all, ti); + public InMemoryView view() { + return new InMemoryView<>(data.values(), ti); + } + + private static Predicate getPredicate( + KVTypeInfo.Accessor getter, + Collection values) { + if (Comparable.class.isAssignableFrom(getter.getType())) { + HashSet set = new HashSet<>(values); + + return (value) -> set.contains(indexValueForEntity(getter, value)); + } else { + HashSet set = new HashSet<>(values.size()); + for (Object key : values) { + set.add(asKey(key)); + } + return (value) -> set.contains(asKey(indexValueForEntity(getter, value))); + } } + private static Object indexValueForEntity(KVTypeInfo.Accessor getter, Object entity) { + try { + return getter.get(entity); + } catch (ReflectiveOperationException e) { + throw new RuntimeException(e); + } + } } private static class InMemoryView extends KVStoreView { + private static final InMemoryView EMPTY_VIEW = + new InMemoryView<>(Collections.emptyList(), null); private final Collection elements; private final KVTypeInfo ti; private final KVTypeInfo.Accessor natural; - InMemoryView(Class type, Collection elements, KVTypeInfo ti) { - super(type); + InMemoryView(Collection elements, KVTypeInfo ti) { this.elements = elements; this.ti = ti; this.natural = ti != null ? ti.getAccessor(KVIndex.NATURAL_INDEX_NAME) : null; @@ -195,34 +289,32 @@ public Iterator iterator() { return new InMemoryIterator<>(elements.iterator()); } - try { - KVTypeInfo.Accessor getter = index != null ? ti.getAccessor(index) : null; - int modifier = ascending ? 1 : -1; + KVTypeInfo.Accessor getter = index != null ? ti.getAccessor(index) : null; + int modifier = ascending ? 1 : -1; - final List sorted = copyElements(); - Collections.sort(sorted, (e1, e2) -> modifier * compare(e1, e2, getter)); - Stream stream = sorted.stream(); + final List sorted = copyElements(); + sorted.sort((e1, e2) -> modifier * compare(e1, e2, getter)); + Stream stream = sorted.stream(); - if (first != null) { - stream = stream.filter(e -> modifier * compare(e, getter, first) >= 0); - } - - if (last != null) { - stream = stream.filter(e -> modifier * compare(e, getter, last) <= 0); - } + if (first != null) { + Comparable firstKey = asKey(first); + stream = stream.filter(e -> modifier * compare(e, getter, firstKey) >= 0); + } - if (skip > 0) { - stream = stream.skip(skip); - } + if (last != null) { + Comparable lastKey = asKey(last); + stream = stream.filter(e -> modifier * compare(e, getter, lastKey) <= 0); + } - if (max < sorted.size()) { - stream = stream.limit((int) max); - } + if (skip > 0) { + stream = stream.skip(skip); + } - return new InMemoryIterator<>(stream.iterator()); - } catch (Exception e) { - throw Throwables.propagate(e); + if (max < sorted.size()) { + stream = stream.limit((int) max); } + + return new InMemoryIterator<>(stream.iterator()); } /** @@ -232,9 +324,10 @@ private List copyElements() { if (parent != null) { KVTypeInfo.Accessor parentGetter = ti.getParentAccessor(index); Preconditions.checkArgument(parentGetter != null, "Parent filter for non-child index."); + Comparable parentKey = asKey(parent); return elements.stream() - .filter(e -> compare(e, parentGetter, parent) == 0) + .filter(e -> compare(e, parentGetter, parentKey) == 0) .collect(Collectors.toList()); } else { return new ArrayList<>(elements); @@ -243,24 +336,23 @@ private List copyElements() { private int compare(T e1, T e2, KVTypeInfo.Accessor getter) { try { - int diff = compare(e1, getter, getter.get(e2)); + int diff = compare(e1, getter, asKey(getter.get(e2))); if (diff == 0 && getter != natural) { - diff = compare(e1, natural, natural.get(e2)); + diff = compare(e1, natural, asKey(natural.get(e2))); } return diff; - } catch (Exception e) { - throw Throwables.propagate(e); + } catch (ReflectiveOperationException e) { + throw new RuntimeException(e); } } - private int compare(T e1, KVTypeInfo.Accessor getter, Object v2) { + private int compare(T e1, KVTypeInfo.Accessor getter, Comparable v2) { try { - return asKey(getter.get(e1)).compareTo(asKey(v2)); - } catch (Exception e) { - throw Throwables.propagate(e); + return asKey(getter.get(e1)).compareTo(v2); + } catch (ReflectiveOperationException e) { + throw new RuntimeException(e); } } - } private static class InMemoryIterator implements KVStoreIterator { diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java index 72d06a8ca8070..ac159eb43182f 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStore.java @@ -18,6 +18,7 @@ package org.apache.spark.util.kvstore; import java.io.Closeable; +import java.util.Collection; import org.apache.spark.annotation.Private; @@ -126,4 +127,9 @@ public interface KVStore extends Closeable { */ long count(Class type, String index, Object indexedValue) throws Exception; + /** + * A cheaper way to remove multiple items from the KVStore + */ + boolean removeAllByIndexValues(Class klass, String index, Collection indexValues) + throws Exception; } diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java index 8ea79bbe160d9..90135268fdef7 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVStoreView.java @@ -38,8 +38,6 @@ @Private public abstract class KVStoreView implements Iterable { - final Class type; - boolean ascending = true; String index = KVIndex.NATURAL_INDEX_NAME; Object first = null; @@ -48,10 +46,6 @@ public abstract class KVStoreView implements Iterable { long skip = 0L; long max = Long.MAX_VALUE; - public KVStoreView(Class type) { - this.type = type; - } - /** * Reverses the order of iteration. By default, iterates in ascending order. */ diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java index 870b484f99068..b8c5fab8709ed 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/KVTypeInfo.java @@ -37,7 +37,7 @@ public class KVTypeInfo { private final Map indices; private final Map accessors; - public KVTypeInfo(Class type) throws Exception { + public KVTypeInfo(Class type) { this.type = type; this.accessors = new HashMap<>(); this.indices = new HashMap<>(); @@ -122,8 +122,9 @@ Accessor getParentAccessor(String indexName) { */ interface Accessor { - Object get(Object instance) throws Exception; + Object get(Object instance) throws ReflectiveOperationException; + Class getType(); } private class FieldAccessor implements Accessor { @@ -135,10 +136,14 @@ private class FieldAccessor implements Accessor { } @Override - public Object get(Object instance) throws Exception { + public Object get(Object instance) throws ReflectiveOperationException { return field.get(instance); } + @Override + public Class getType() { + return field.getType(); + } } private class MethodAccessor implements Accessor { @@ -150,10 +155,14 @@ private class MethodAccessor implements Accessor { } @Override - public Object get(Object instance) throws Exception { + public Object get(Object instance) throws ReflectiveOperationException { return method.invoke(instance); } + @Override + public Class getType() { + return method.getReturnType(); + } } } diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java index 58e2a8f25f34f..2ca4b0b2cb9f9 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDB.java @@ -19,10 +19,7 @@ import java.io.File; import java.io.IOException; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; -import java.util.NoSuchElementException; +import java.util.*; import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.ConcurrentMap; import java.util.concurrent.atomic.AtomicReference; @@ -188,11 +185,11 @@ public void delete(Class type, Object naturalKey) throws Exception { @Override public KVStoreView view(Class type) throws Exception { - return new KVStoreView(type) { + return new KVStoreView() { @Override public Iterator iterator() { try { - return new LevelDBIterator<>(LevelDB.this, this); + return new LevelDBIterator<>(type, LevelDB.this, this); } catch (Exception e) { throw Throwables.propagate(e); } @@ -200,6 +197,26 @@ public Iterator iterator() { }; } + @Override + public boolean removeAllByIndexValues( + Class klass, + String index, + Collection indexValues) throws Exception { + LevelDBTypeInfo.Index naturalIndex = getTypeInfo(klass).naturalIndex(); + boolean removed = false; + KVStoreView view = view(klass).index(index); + + for (Object indexValue : indexValues) { + for (T value: view.first(indexValue).last(indexValue)) { + Object itemKey = naturalIndex.getValue(value); + delete(klass, itemKey); + removed = true; + } + } + + return removed; + } + @Override public long count(Class type) throws Exception { LevelDBTypeInfo.Index idx = getTypeInfo(type).naturalIndex(); diff --git a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java index f62e85d435318..94e8c9fc5796c 100644 --- a/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java +++ b/common/kvstore/src/main/java/org/apache/spark/util/kvstore/LevelDBIterator.java @@ -45,11 +45,11 @@ class LevelDBIterator implements KVStoreIterator { private boolean closed; private long count; - LevelDBIterator(LevelDB db, KVStoreView params) throws Exception { + LevelDBIterator(Class type, LevelDB db, KVStoreView params) throws Exception { this.db = db; this.ascending = params.ascending; this.it = db.db().iterator(); - this.type = params.type; + this.type = type; this.ti = db.getTypeInfo(type); this.index = ti.index(params.index); this.max = params.max; @@ -196,6 +196,7 @@ public synchronized void close() throws IOException { * when Scala wrappers are used, this makes sure that, hopefully, the JNI resources held by * the iterator will eventually be released. */ + @SuppressWarnings("deprecation") @Override protected void finalize() throws Throwable { db.closeIterator(this); @@ -206,47 +207,43 @@ private byte[] loadNext() { return null; } - try { - while (true) { - boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); - if (!hasNext) { - return null; - } + while (true) { + boolean hasNext = ascending ? it.hasNext() : it.hasPrev(); + if (!hasNext) { + return null; + } - Map.Entry nextEntry; - try { - // Avoid races if another thread is updating the DB. - nextEntry = ascending ? it.next() : it.prev(); - } catch (NoSuchElementException e) { - return null; - } + Map.Entry nextEntry; + try { + // Avoid races if another thread is updating the DB. + nextEntry = ascending ? it.next() : it.prev(); + } catch (NoSuchElementException e) { + return null; + } - byte[] nextKey = nextEntry.getKey(); - // Next key is not part of the index, stop. - if (!startsWith(nextKey, indexKeyPrefix)) { - return null; - } + byte[] nextKey = nextEntry.getKey(); + // Next key is not part of the index, stop. + if (!startsWith(nextKey, indexKeyPrefix)) { + return null; + } - // If the next key is an end marker, then skip it. - if (isEndMarker(nextKey)) { - continue; - } + // If the next key is an end marker, then skip it. + if (isEndMarker(nextKey)) { + continue; + } - // If there's a known end key and iteration has gone past it, stop. - if (end != null) { - int comp = compare(nextKey, end) * (ascending ? 1 : -1); - if (comp > 0) { - return null; - } + // If there's a known end key and iteration has gone past it, stop. + if (end != null) { + int comp = compare(nextKey, end) * (ascending ? 1 : -1); + if (comp > 0) { + return null; } + } - count++; + count++; - // Next element is part of the iteration, return it. - return nextEntry.getValue(); - } - } catch (Exception e) { - throw Throwables.propagate(e); + // Next element is part of the iteration, return it. + return nextEntry.getValue(); } } diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java index 9abf26f02f7a7..9e34225e14e18 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/InMemoryStoreSuite.java @@ -19,6 +19,7 @@ import java.util.NoSuchElementException; +import com.google.common.collect.ImmutableSet; import org.junit.Test; import static org.junit.Assert.*; @@ -132,6 +133,51 @@ public void testArrayIndices() throws Exception { assertEquals(o, store.view(ArrayKeyIndexType.class).index("id").first(o.id).iterator().next()); } + @Test + public void testRemoveAll() throws Exception { + KVStore store = new InMemoryStore(); + + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { i, j, 0 }; + o.id = new String[] { "things" }; + store.write(o); + + o = new ArrayKeyIndexType(); + o.key = new int[] { i, j, 1 }; + o.id = new String[] { "more things" }; + store.write(o); + } + } + + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { 2, 2, 2 }; + o.id = new String[] { "things" }; + store.write(o); + + assertEquals(9, store.count(ArrayKeyIndexType.class)); + + + store.removeAllByIndexValues( + ArrayKeyIndexType.class, + KVIndex.NATURAL_INDEX_NAME, + ImmutableSet.of(new int[] {0, 0, 0}, new int[] { 2, 2, 2 })); + assertEquals(7, store.count(ArrayKeyIndexType.class)); + + store.removeAllByIndexValues( + ArrayKeyIndexType.class, + "id", + ImmutableSet.of(new String [] { "things" })); + assertEquals(4, store.count(ArrayKeyIndexType.class)); + + store.removeAllByIndexValues( + ArrayKeyIndexType.class, + "id", + ImmutableSet.of(new String [] { "more things" })); + assertEquals(0, store.count(ArrayKeyIndexType.class)); + } + @Test public void testBasicIteration() throws Exception { KVStore store = new InMemoryStore(); diff --git a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java index 39a952f2b0df9..0b755ba0e8000 100644 --- a/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java +++ b/common/kvstore/src/test/java/org/apache/spark/util/kvstore/LevelDBSuite.java @@ -24,6 +24,7 @@ import java.util.stream.Collectors; import java.util.stream.StreamSupport; +import com.google.common.collect.ImmutableSet; import org.apache.commons.io.FileUtils; import org.iq80.leveldb.DBIterator; import org.junit.After; @@ -198,6 +199,48 @@ public void testUpdate() throws Exception { assertEquals(0, db.count(t.getClass(), "name", "name")); } + @Test + public void testRemoveAll() throws Exception { + for (int i = 0; i < 2; i++) { + for (int j = 0; j < 2; j++) { + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { i, j, 0 }; + o.id = new String[] { "things" }; + db.write(o); + + o = new ArrayKeyIndexType(); + o.key = new int[] { i, j, 1 }; + o.id = new String[] { "more things" }; + db.write(o); + } + } + + ArrayKeyIndexType o = new ArrayKeyIndexType(); + o.key = new int[] { 2, 2, 2 }; + o.id = new String[] { "things" }; + db.write(o); + + assertEquals(9, db.count(ArrayKeyIndexType.class)); + + db.removeAllByIndexValues( + ArrayKeyIndexType.class, + KVIndex.NATURAL_INDEX_NAME, + ImmutableSet.of(new int[] {0, 0, 0}, new int[] { 2, 2, 2 })); + assertEquals(7, db.count(ArrayKeyIndexType.class)); + + db.removeAllByIndexValues( + ArrayKeyIndexType.class, + "id", + ImmutableSet.of(new String[] { "things" })); + assertEquals(4, db.count(ArrayKeyIndexType.class)); + + db.removeAllByIndexValues( + ArrayKeyIndexType.class, + "id", + ImmutableSet.of(new String[] { "more things" })); + assertEquals(0, db.count(ArrayKeyIndexType.class)); + } + @Test public void testSkip() throws Exception { for (int i = 0; i < 10; i++) { diff --git a/common/network-common/pom.xml b/common/network-common/pom.xml index 41fcbf0589499..56d01fa0e8b3d 100644 --- a/common/network-common/pom.xml +++ b/common/network-common/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-network-common_2.11 + spark-network-common_2.12 jar Spark Project Networking http://spark.apache.org/ 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 480b52652de53..d99b9bdbce392 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 @@ -17,9 +17,11 @@ package org.apache.spark.network; +import java.io.Closeable; import java.util.ArrayList; import java.util.List; +import com.codahale.metrics.Counter; import io.netty.channel.Channel; import io.netty.channel.ChannelPipeline; import io.netty.channel.EventLoopGroup; @@ -59,13 +61,14 @@ * channel. As each TransportChannelHandler contains a TransportClient, this enables server * processes to send messages back to the client on an existing channel. */ -public class TransportContext { +public class TransportContext implements Closeable { private static final Logger logger = LoggerFactory.getLogger(TransportContext.class); private final TransportConf conf; private final RpcHandler rpcHandler; private final boolean closeIdleConnections; - private final boolean isClientOnly; + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); /** * Force to create MessageEncoder and MessageDecoder so that we can make sure they will be created @@ -85,7 +88,7 @@ public class TransportContext { // Separate thread pool for handling ChunkFetchRequest. This helps to enable throttling // max number of TransportServer worker threads that are blocked on writing response // of ChunkFetchRequest message back to the client via the underlying channel. - private static EventLoopGroup chunkFetchWorkers; + private final EventLoopGroup chunkFetchWorkers; public TransportContext(TransportConf conf, RpcHandler rpcHandler) { this(conf, rpcHandler, false, false); @@ -117,18 +120,16 @@ public TransportContext( this.conf = conf; this.rpcHandler = rpcHandler; this.closeIdleConnections = closeIdleConnections; - this.isClientOnly = isClientOnly; - - synchronized(TransportContext.class) { - if (chunkFetchWorkers == null && - conf.getModuleName() != null && - conf.getModuleName().equalsIgnoreCase("shuffle") && - !isClientOnly) { - chunkFetchWorkers = NettyUtils.createEventLoop( - IOMode.valueOf(conf.ioMode()), - conf.chunkFetchHandlerThreads(), - "shuffle-chunk-fetch-handler"); - } + + if (conf.getModuleName() != null && + conf.getModuleName().equalsIgnoreCase("shuffle") && + !isClientOnly) { + chunkFetchWorkers = NettyUtils.createEventLoop( + IOMode.valueOf(conf.ioMode()), + conf.chunkFetchHandlerThreads(), + "shuffle-chunk-fetch-handler"); + } else { + chunkFetchWorkers = null; } } @@ -198,9 +199,7 @@ public TransportChannelHandler initializePipeline( // would require more logic to guarantee if this were not part of the same event loop. .addLast("handler", channelHandler); // Use a separate EventLoopGroup to handle ChunkFetchRequest messages for shuffle rpcs. - if (conf.getModuleName() != null && - conf.getModuleName().equalsIgnoreCase("shuffle") - && !isClientOnly) { + if (chunkFetchWorkers != null) { pipeline.addLast(chunkFetchWorkers, "chunkFetchHandler", chunkFetchHandler); } return channelHandler; @@ -221,7 +220,7 @@ private TransportChannelHandler createChannelHandler(Channel channel, RpcHandler TransportRequestHandler requestHandler = new TransportRequestHandler(channel, client, rpcHandler, conf.maxChunksBeingTransferred()); return new TransportChannelHandler(client, responseHandler, requestHandler, - conf.connectionTimeoutMs(), closeIdleConnections); + conf.connectionTimeoutMs(), closeIdleConnections, this); } /** @@ -234,4 +233,14 @@ private ChunkFetchRequestHandler createChunkFetchHandler(TransportChannelHandler } public TransportConf getConf() { return conf; } + + public Counter getRegisteredConnections() { + return registeredConnections; + } + + public void close() { + if (chunkFetchWorkers != null) { + chunkFetchWorkers.shutdownGracefully(); + } + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java index 20d840baeaf6c..b018197deaf2e 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportClient.java @@ -237,11 +237,16 @@ public ByteBuffer sendRpcSync(ByteBuffer message, long timeoutMs) { sendRpc(message, new RpcResponseCallback() { @Override public void onSuccess(ByteBuffer response) { - ByteBuffer copy = ByteBuffer.allocate(response.remaining()); - copy.put(response); - // flip "copy" to make it readable - copy.flip(); - result.set(copy); + try { + ByteBuffer copy = ByteBuffer.allocate(response.remaining()); + copy.put(response); + // flip "copy" to make it readable + copy.flip(); + result.set(copy); + } catch (Throwable t) { + logger.warn("Error in responding PRC callback", t); + result.setException(t); + } } @Override 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 16d242dbb2c47..53835d8304866 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 @@ -84,7 +84,7 @@ private static class ClientPool { private final Class socketChannelClass; private EventLoopGroup workerGroup; - private PooledByteBufAllocator pooledAllocator; + private final PooledByteBufAllocator pooledAllocator; private final NettyMemoryMetrics metrics; public TransportClientFactory( @@ -103,8 +103,13 @@ public TransportClientFactory( ioMode, conf.clientThreads(), conf.getModuleName() + "-client"); - this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + if (conf.sharedByteBufAllocators()) { + this.pooledAllocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), false /* allowCache */); + } else { + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), false /* allowCache */, conf.clientThreads()); + } this.metrics = new NettyMemoryMetrics( this.pooledAllocator, conf.getModuleName() + "-client", conf); } @@ -167,10 +172,13 @@ public TransportClient createClient(String remoteHost, int remotePort) final long preResolveHost = System.nanoTime(); final InetSocketAddress resolvedAddress = new InetSocketAddress(remoteHost, remotePort); final long hostResolveTimeMs = (System.nanoTime() - preResolveHost) / 1000000; + final String resolvMsg = resolvedAddress.isUnresolved() ? "failed" : "succeed"; if (hostResolveTimeMs > 2000) { - logger.warn("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs); + logger.warn("DNS resolution {} for {} took {} ms", + resolvMsg, resolvedAddress, hostResolveTimeMs); } else { - logger.trace("DNS resolution for {} took {} ms", resolvedAddress, hostResolveTimeMs); + logger.trace("DNS resolution {} for {} took {} ms", + resolvMsg, resolvedAddress, hostResolveTimeMs); } synchronized (clientPool.locks[clientIndex]) { diff --git a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java index 596b0ea5dba9b..2f143f77fa4ae 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/client/TransportResponseHandler.java @@ -91,7 +91,7 @@ public void removeRpcRequest(long requestId) { } public void addStreamCallback(String streamId, StreamCallback callback) { - timeOfLastRequestNs.set(System.nanoTime()); + updateTimeOfLastRequest(); streamCallbacks.offer(ImmutablePair.of(streamId, callback)); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java index 3c263783a6104..77b167d15e911 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/AuthClientBootstrap.java @@ -20,6 +20,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.security.GeneralSecurityException; +import java.util.concurrent.TimeoutException; import com.google.common.base.Throwables; import io.netty.buffer.ByteBuf; @@ -82,13 +83,19 @@ public void doBootstrap(TransportClient client, Channel channel) { } catch (RuntimeException e) { // There isn't a good exception that can be caught here to know whether it's really // OK to switch back to SASL (because the server doesn't speak the new protocol). So - // try it anyway, and in the worst case things will fail again. - if (conf.saslFallback()) { - LOG.warn("New auth protocol failed, trying SASL.", e); - doSaslAuth(client, channel); - } else { + // try it anyway, unless it's a timeout, which is locally fatal. In the worst case + // things will fail again. + if (!conf.saslFallback() || e.getCause() instanceof TimeoutException) { throw e; } + + if (LOG.isDebugEnabled()) { + Throwable cause = e.getCause() != null ? e.getCause() : e; + LOG.debug("New auth protocol failed, trying SASL.", cause); + } else { + LOG.info("New auth protocol failed, trying SASL."); + } + doSaslAuth(client, channel); } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java index 2745052265f7f..8995bbc940f63 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java +++ b/common/network-common/src/main/java/org/apache/spark/network/crypto/TransportCipher.java @@ -44,7 +44,8 @@ public class TransportCipher { @VisibleForTesting static final String ENCRYPTION_HANDLER_NAME = "TransportEncryption"; private static final String DECRYPTION_HANDLER_NAME = "TransportDecryption"; - private static final int STREAM_BUFFER_SIZE = 1024 * 32; + @VisibleForTesting + static final int STREAM_BUFFER_SIZE = 1024 * 32; private final Properties conf; private final String cipher; @@ -84,7 +85,8 @@ public byte[] getOutputIv() { return outIv; } - private CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { + @VisibleForTesting + CryptoOutputStream createOutputStream(WritableByteChannel ch) throws IOException { return new CryptoOutputStream(cipher, conf, ch, key, new IvParameterSpec(outIv)); } @@ -104,7 +106,8 @@ public void addToChannel(Channel ch) throws IOException { .addFirst(DECRYPTION_HANDLER_NAME, new DecryptionHandler(this)); } - private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { + @VisibleForTesting + static class EncryptionHandler extends ChannelOutboundHandlerAdapter { private final ByteArrayWritableChannel byteChannel; private final CryptoOutputStream cos; private boolean isCipherValid; @@ -118,7 +121,12 @@ private static class EncryptionHandler extends ChannelOutboundHandlerAdapter { @Override public void write(ChannelHandlerContext ctx, Object msg, ChannelPromise promise) throws Exception { - ctx.write(new EncryptedMessage(this, cos, msg, byteChannel), promise); + ctx.write(createEncryptedMessage(msg), promise); + } + + @VisibleForTesting + EncryptedMessage createEncryptedMessage(Object msg) { + return new EncryptedMessage(this, cos, msg, byteChannel); } @Override @@ -190,12 +198,14 @@ public void channelInactive(ChannelHandlerContext ctx) throws Exception { } } - private static class EncryptedMessage extends AbstractFileRegion { + @VisibleForTesting + static class EncryptedMessage extends AbstractFileRegion { private final boolean isByteBuf; private final ByteBuf buf; private final FileRegion region; private final CryptoOutputStream cos; private final EncryptionHandler handler; + private final long count; private long transferred; // Due to streaming issue CRYPTO-125: https://issues.apache.org/jira/browse/CRYPTO-125, it has @@ -221,11 +231,12 @@ private static class EncryptedMessage extends AbstractFileRegion { this.byteRawChannel = new ByteArrayWritableChannel(STREAM_BUFFER_SIZE); this.cos = cos; this.byteEncChannel = ch; + this.count = isByteBuf ? buf.readableBytes() : region.count(); } @Override public long count() { - return isByteBuf ? buf.readableBytes() : region.count(); + return count; } @Override @@ -277,22 +288,38 @@ public boolean release(int decrement) { public long transferTo(WritableByteChannel target, long position) throws IOException { Preconditions.checkArgument(position == transferred(), "Invalid position."); + if (transferred == count) { + return 0; + } + + long totalBytesWritten = 0L; do { if (currentEncrypted == null) { encryptMore(); } - int bytesWritten = currentEncrypted.remaining(); - target.write(currentEncrypted); - bytesWritten -= currentEncrypted.remaining(); - transferred += bytesWritten; - if (!currentEncrypted.hasRemaining()) { + long remaining = currentEncrypted.remaining(); + if (remaining == 0) { + // Just for safety to avoid endless loop. It usually won't happen, but since the + // underlying `region.transferTo` is allowed to transfer 0 bytes, we should handle it for + // safety. currentEncrypted = null; byteEncChannel.reset(); + return totalBytesWritten; } - } while (transferred < count()); - return transferred; + long bytesWritten = target.write(currentEncrypted); + totalBytesWritten += bytesWritten; + transferred += bytesWritten; + if (bytesWritten < remaining) { + // break as the underlying buffer in "target" is full + break; + } + currentEncrypted = null; + byteEncChannel.reset(); + } while (transferred < count); + + return totalBytesWritten; } private void encryptMore() throws IOException { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java index 7b28a9a969486..a7afbfa8621c8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchFailure.java @@ -33,7 +33,7 @@ public ChunkFetchFailure(StreamChunkId streamChunkId, String errorString) { } @Override - public Type type() { return Type.ChunkFetchFailure; } + public Message.Type type() { return Type.ChunkFetchFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java index 26d063feb5fe3..fe54fcc50dc86 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchRequest.java @@ -32,7 +32,7 @@ public ChunkFetchRequest(StreamChunkId streamChunkId) { } @Override - public Type type() { return Type.ChunkFetchRequest; } + public Message.Type type() { return Type.ChunkFetchRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java index 94c2ac9b20e43..d5c9a9b3202fb 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/ChunkFetchSuccess.java @@ -39,7 +39,7 @@ public ChunkFetchSuccess(StreamChunkId streamChunkId, ManagedBuffer buffer) { } @Override - public Type type() { return Type.ChunkFetchSuccess; } + public Message.Type type() { return Type.ChunkFetchSuccess; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java index be217522367c5..736059fdd1f57 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/Encoders.java @@ -89,4 +89,27 @@ public static String[] decode(ByteBuf buf) { return strings; } } + + /** Integer arrays are encoded with their length followed by integers. */ + public static class IntArrays { + public static int encodedLength(int[] ints) { + return 4 + 4 * ints.length; + } + + public static void encode(ByteBuf buf, int[] ints) { + buf.writeInt(ints.length); + for (int i : ints) { + buf.writeInt(i); + } + } + + public static int[] decode(ByteBuf buf) { + int numInts = buf.readInt(); + int[] ints = new int[numInts]; + for (int i = 0; i < ints.length; i ++) { + ints[i] = buf.readInt(); + } + return ints; + } + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java index 997f74e1a21b4..06dc447309dd9 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/MessageEncoder.java @@ -79,7 +79,7 @@ public void encode(ChannelHandlerContext ctx, Message in, List out) thro // sent. int headerLength = 8 + msgType.encodedLength() + in.encodedLength(); long frameLength = headerLength + (isBodyInFrame ? bodyLength : 0); - ByteBuf header = ctx.alloc().heapBuffer(headerLength); + ByteBuf header = ctx.alloc().buffer(headerLength); header.writeLong(frameLength); msgType.encode(header); in.encode(header); diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java index f7ffb1bd49bb6..1632fb9e03687 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/OneWayMessage.java @@ -34,7 +34,7 @@ public OneWayMessage(ManagedBuffer body) { } @Override - public Type type() { return Type.OneWayMessage; } + public Message.Type type() { return Type.OneWayMessage; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java index a76624ef5dc96..61061903de23f 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcFailure.java @@ -31,7 +31,7 @@ public RpcFailure(long requestId, String errorString) { } @Override - public Type type() { return Type.RpcFailure; } + public Message.Type type() { return Type.RpcFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java index 2b30920f0598d..cc1bb95d2d566 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcRequest.java @@ -38,7 +38,7 @@ public RpcRequest(long requestId, ManagedBuffer message) { } @Override - public Type type() { return Type.RpcRequest; } + public Message.Type type() { return Type.RpcRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java index d73014ecd8506..c03291e9c0b23 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/RpcResponse.java @@ -33,7 +33,7 @@ public RpcResponse(long requestId, ManagedBuffer message) { } @Override - public Type type() { return Type.RpcResponse; } + public Message.Type type() { return Type.RpcResponse; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java index 258ef81c6783d..68fcfa7748611 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamFailure.java @@ -33,7 +33,7 @@ public StreamFailure(String streamId, String error) { } @Override - public Type type() { return Type.StreamFailure; } + public Message.Type type() { return Type.StreamFailure; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java index dc183c043ed9a..1b135af752bd8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamRequest.java @@ -34,7 +34,7 @@ public StreamRequest(String streamId) { } @Override - public Type type() { return Type.StreamRequest; } + public Message.Type type() { return Type.StreamRequest; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java index 50b811604b84b..568108c4fe5e8 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/StreamResponse.java @@ -40,7 +40,7 @@ public StreamResponse(String streamId, long byteCount, ManagedBuffer buffer) { } @Override - public Type type() { return Type.StreamResponse; } + public Message.Type type() { return Type.StreamResponse; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java index fa1d26e76b852..7d21151e01074 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java +++ b/common/network-common/src/main/java/org/apache/spark/network/protocol/UploadStream.java @@ -52,7 +52,7 @@ private UploadStream(long requestId, ManagedBuffer meta, long bodyByteCount) { } @Override - public Type type() { return Type.UploadStream; } + public Message.Type type() { return Type.UploadStream; } @Override public int encodedLength() { diff --git a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java index 7331c2b481fb1..1b03300d948e2 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java +++ b/common/network-common/src/main/java/org/apache/spark/network/sasl/SaslMessage.java @@ -23,6 +23,7 @@ import org.apache.spark.network.buffer.NettyManagedBuffer; import org.apache.spark.network.protocol.Encoders; import org.apache.spark.network.protocol.AbstractMessage; +import org.apache.spark.network.protocol.Message; /** * Encodes a Sasl-related message which is attempting to authenticate using some credentials tagged @@ -46,7 +47,7 @@ class SaslMessage extends AbstractMessage { } @Override - public Type type() { return Type.User; } + public Message.Type type() { return Type.User; } @Override public int encodedLength() { 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 f08d8b0f984cf..94412c4db559f 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 @@ -90,8 +90,10 @@ protected void channelRead0( ManagedBuffer buf; try { streamManager.checkAuthorization(client, msg.streamChunkId.streamId); - streamManager.registerChannel(channel, msg.streamChunkId.streamId); buf = streamManager.getChunk(msg.streamChunkId.streamId, msg.streamChunkId.chunkIndex); + if (buf == null) { + throw new IllegalStateException("Chunk was not found"); + } } catch (Exception e) { logger.error(String.format("Error opening block %s for request from %s", msg.streamChunkId, getRemoteAddress(channel)), e); diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java index 0f6a8824d95e5..67f64d7962035 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/OneForOneStreamManager.java @@ -23,6 +23,7 @@ import java.util.concurrent.ConcurrentHashMap; import java.util.concurrent.atomic.AtomicLong; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.channel.Channel; import org.apache.commons.lang3.tuple.ImmutablePair; @@ -49,7 +50,7 @@ private static class StreamState { final Iterator buffers; // The channel associated to the stream - Channel associatedChannel = null; + final Channel associatedChannel; // Used to keep track of the index of the buffer that the user has retrieved, just to ensure // that the caller only requests each chunk one at a time, in order. @@ -58,9 +59,10 @@ private static class StreamState { // Used to keep track of the number of chunks being transferred and not finished yet. volatile long chunksBeingTransferred = 0L; - StreamState(String appId, Iterator buffers) { + StreamState(String appId, Iterator buffers, Channel channel) { this.appId = appId; this.buffers = Preconditions.checkNotNull(buffers); + this.associatedChannel = channel; } } @@ -71,13 +73,6 @@ public OneForOneStreamManager() { streams = new ConcurrentHashMap<>(); } - @Override - public void registerChannel(Channel channel, long streamId) { - if (streams.containsKey(streamId)) { - streams.get(streamId).associatedChannel = channel; - } - } - @Override public ManagedBuffer getChunk(long streamId, int chunkIndex) { StreamState state = streams.get(streamId); @@ -130,7 +125,10 @@ public void connectionTerminated(Channel channel) { // Release all remaining buffers. while (state.buffers.hasNext()) { - state.buffers.next().release(); + ManagedBuffer buffer = state.buffers.next(); + if (buffer != null) { + buffer.release(); + } } } } @@ -195,11 +193,19 @@ public long chunksBeingTransferred() { * * If an app ID is provided, only callers who've authenticated with the given app ID will be * allowed to fetch from this stream. + * + * This method also associates the stream with a single client connection, which is guaranteed + * to be the only reader of the stream. Once the connection is closed, the stream will never + * be used again, enabling cleanup by `connectionTerminated`. */ - public long registerStream(String appId, Iterator buffers) { + public long registerStream(String appId, Iterator buffers, Channel channel) { long myStreamId = nextStreamId.getAndIncrement(); - streams.put(myStreamId, new StreamState(appId, buffers)); + streams.put(myStreamId, new StreamState(appId, buffers, channel)); return myStreamId; } + @VisibleForTesting + public int numStreamStates() { + return streams.size(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java index c535295831606..e48d27be1126a 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/StreamManager.java @@ -60,16 +60,6 @@ public ManagedBuffer openStream(String streamId) { throw new UnsupportedOperationException(); } - /** - * Associates a stream with a single client connection, which is guaranteed to be the only reader - * of the stream. The getChunk() method will be called serially on this connection and once the - * connection is closed, the stream will never be used again, enabling cleanup. - * - * This must be called before the first getChunk() on the stream, but it may be invoked multiple - * times with the same channel and stream id. - */ - public void registerChannel(Channel channel, long streamId) { } - /** * Indicates that the given channel has been terminated. After this occurs, we are guaranteed not * to read from the associated streams again, so any state can be cleaned up. 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 c824a7b0d4740..31371f6970ffe 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 @@ -21,6 +21,7 @@ import io.netty.channel.SimpleChannelInboundHandler; import io.netty.handler.timeout.IdleState; import io.netty.handler.timeout.IdleStateEvent; +import org.apache.spark.network.TransportContext; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -57,18 +58,21 @@ public class TransportChannelHandler extends SimpleChannelInboundHandler 0; boolean isActuallyOverdue = System.nanoTime() - responseHandler.getTimeOfLastRequestNs() > requestTimeoutNs; if (e.state() == IdleState.ALL_IDLE && isActuallyOverdue) { - if (responseHandler.numOutstandingRequests() > 0) { + if (hasInFlightRequests) { String address = getRemoteAddress(ctx.channel()); logger.error("Connection to {} has been quiet for {} ms while there are outstanding " + "requests. Assuming connection is dead; please adjust spark.network.timeout if " + @@ -176,4 +181,16 @@ public TransportResponseHandler getResponseHandler() { return responseHandler; } + @Override + public void channelRegistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().inc(); + super.channelRegistered(ctx); + } + + @Override + public void channelUnregistered(ChannelHandlerContext ctx) throws Exception { + transportContext.getRegisteredConnections().dec(); + super.channelUnregistered(ctx); + } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java index 3e089b4cae273..0792b58304247 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportRequestHandler.java @@ -140,6 +140,8 @@ private void processStreamRequest(final StreamRequest req) { streamManager.streamSent(req.streamId); }); } else { + // org.apache.spark.repl.ExecutorClassLoader.STREAM_NOT_FOUND_REGEX should also be updated + // when the following error message is changed. respond(new StreamFailure(req.streamId, String.format( "Stream '%s' was not found.", req.streamId))); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java index 9c85ab2f5f06f..8396e691e9db1 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java +++ b/common/network-common/src/main/java/org/apache/spark/network/server/TransportServer.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.TimeUnit; +import com.codahale.metrics.Counter; import com.codahale.metrics.MetricSet; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -53,6 +54,7 @@ public class TransportServer implements Closeable { private ServerBootstrap bootstrap; private ChannelFuture channelFuture; private int port = -1; + private final PooledByteBufAllocator pooledAllocator; private NettyMemoryMetrics metrics; /** @@ -68,6 +70,13 @@ public TransportServer( this.context = context; this.conf = context.getConf(); this.appRpcHandler = appRpcHandler; + if (conf.sharedByteBufAllocators()) { + this.pooledAllocator = NettyUtils.getSharedPooledByteBufAllocator( + conf.preferDirectBufsForSharedByteBufAllocators(), true /* allowCache */); + } else { + this.pooledAllocator = NettyUtils.createPooledByteBufAllocator( + conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); + } this.bootstraps = Lists.newArrayList(Preconditions.checkNotNull(bootstraps)); boolean shouldClose = true; @@ -95,18 +104,15 @@ private void init(String hostToBind, int portToBind) { NettyUtils.createEventLoop(ioMode, conf.serverThreads(), conf.getModuleName() + "-server"); EventLoopGroup workerGroup = bossGroup; - PooledByteBufAllocator allocator = NettyUtils.createPooledByteBufAllocator( - conf.preferDirectBufs(), true /* allowCache */, conf.serverThreads()); - bootstrap = new ServerBootstrap() .group(bossGroup, workerGroup) .channel(NettyUtils.getServerChannelClass(ioMode)) - .option(ChannelOption.ALLOCATOR, allocator) + .option(ChannelOption.ALLOCATOR, pooledAllocator) .option(ChannelOption.SO_REUSEADDR, !SystemUtils.IS_OS_WINDOWS) - .childOption(ChannelOption.ALLOCATOR, allocator); + .childOption(ChannelOption.ALLOCATOR, pooledAllocator); this.metrics = new NettyMemoryMetrics( - allocator, conf.getModuleName() + "-server", conf); + pooledAllocator, conf.getModuleName() + "-server", conf); if (conf.backLog() > 0) { bootstrap.option(ChannelOption.SO_BACKLOG, conf.backLog()); @@ -120,9 +126,15 @@ private void init(String hostToBind, int portToBind) { bootstrap.childOption(ChannelOption.SO_SNDBUF, conf.sendBuf()); } + if (conf.enableTcpKeepAlive()) { + bootstrap.childOption(ChannelOption.SO_KEEPALIVE, true); + } + bootstrap.childHandler(new ChannelInitializer() { @Override protected void initChannel(SocketChannel ch) { + logger.debug("New connection accepted for remote address {}.", ch.remoteAddress()); + RpcHandler rpcHandler = appRpcHandler; for (TransportServerBootstrap bootstrap : bootstraps) { rpcHandler = bootstrap.doBootstrap(ch, rpcHandler); @@ -159,4 +171,8 @@ public void close() { } bootstrap = null; } + + public Counter getRegisteredConnections() { + return context.getRegisteredConnections(); + } } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java index 984575acaf511..6f7925c26094d 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/ByteUnit.java @@ -18,11 +18,11 @@ public enum ByteUnit { BYTE(1), - KiB(1024L), - MiB((long) Math.pow(1024L, 2L)), - GiB((long) Math.pow(1024L, 3L)), - TiB((long) Math.pow(1024L, 4L)), - PiB((long) Math.pow(1024L, 5L)); + KiB(1L << 10), + MiB(1L << 20), + GiB(1L << 30), + TiB(1L << 40), + PiB(1L << 50); ByteUnit(long multiplier) { this.multiplier = multiplier; @@ -50,7 +50,7 @@ public long convertTo(long d, ByteUnit u) { } } - public double toBytes(long d) { + public long toBytes(long d) { if (d < 0) { throw new IllegalArgumentException("Negative size value. Size must be positive: " + d); } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java index 33d6eb4a83a0c..423cc0c70ea02 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/NettyUtils.java @@ -36,6 +36,22 @@ * Utilities for creating various Netty constructs based on whether we're using EPOLL or NIO. */ public class NettyUtils { + + /** + * Specifies an upper bound on the number of Netty threads that Spark requires by default. + * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core + * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes + * at a premium. + * + * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory + * allocation. It can be overridden by setting the number of serverThreads and clientThreads + * manually in Spark's configuration. + */ + private static int MAX_DEFAULT_NETTY_THREADS = 8; + + private static final PooledByteBufAllocator[] _sharedPooledByteBufAllocator = + new PooledByteBufAllocator[2]; + /** Creates a new ThreadFactory which prefixes each thread with the given name. */ public static ThreadFactory createThreadFactory(String threadPoolPrefix) { return new DefaultThreadFactory(threadPoolPrefix, true); @@ -95,6 +111,38 @@ public static String getRemoteAddress(Channel channel) { return ""; } + /** + * Returns the default number of threads for both the Netty client and server thread pools. + * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. + */ + public static int defaultNumThreads(int numUsableCores) { + final int availableCores; + if (numUsableCores > 0) { + availableCores = numUsableCores; + } else { + availableCores = Runtime.getRuntime().availableProcessors(); + } + return Math.min(availableCores, MAX_DEFAULT_NETTY_THREADS); + } + + /** + * Returns the lazily created shared pooled ByteBuf allocator for the specified allowCache + * parameter value. + */ + public static synchronized PooledByteBufAllocator getSharedPooledByteBufAllocator( + boolean allowDirectBufs, + boolean allowCache) { + final int index = allowCache ? 0 : 1; + if (_sharedPooledByteBufAllocator[index] == null) { + _sharedPooledByteBufAllocator[index] = + createPooledByteBufAllocator( + allowDirectBufs, + allowCache, + defaultNumThreads(0)); + } + return _sharedPooledByteBufAllocator[index]; + } + /** * Create a pooled ByteBuf allocator but disables the thread-local cache. Thread-local caches * are disabled for TransportClients because the ByteBufs are allocated by the event loop thread, 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 43a6bc7dc3d06..589dfcbefb6ea 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 @@ -42,6 +42,7 @@ public class TransportConf { private final String SPARK_NETWORK_IO_RETRYWAIT_KEY; private final String SPARK_NETWORK_IO_LAZYFD_KEY; private final String SPARK_NETWORK_VERBOSE_METRICS; + private final String SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY; private final ConfigProvider conf; @@ -64,6 +65,7 @@ public TransportConf(String module, ConfigProvider conf) { SPARK_NETWORK_IO_RETRYWAIT_KEY = getConfKey("io.retryWait"); SPARK_NETWORK_IO_LAZYFD_KEY = getConfKey("io.lazyFD"); SPARK_NETWORK_VERBOSE_METRICS = getConfKey("io.enableVerboseMetrics"); + SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY = getConfKey("io.enableTcpKeepAlive"); } public int getInt(String name, int defaultValue) { @@ -106,8 +108,8 @@ public int numConnectionsPerPeer() { return conf.getInt(SPARK_NETWORK_IO_NUMCONNECTIONSPERPEER_KEY, 1); } - /** Requested maximum length of the queue of incoming connections. Default -1 for no backlog. */ - public int backLog() { return conf.getInt(SPARK_NETWORK_IO_BACKLOG_KEY, -1); } + /** Requested maximum length of the queue of incoming connections. Default is 64. */ + public int backLog() { return conf.getInt(SPARK_NETWORK_IO_BACKLOG_KEY, 64); } /** Number of threads used in the server thread pool. Default to 0, which is 2x#cores. */ public int serverThreads() { return conf.getInt(SPARK_NETWORK_IO_SERVERTHREADS_KEY, 0); } @@ -173,6 +175,14 @@ public boolean verboseMetrics() { return conf.getBoolean(SPARK_NETWORK_VERBOSE_METRICS, false); } + /** + * Whether to enable TCP keep-alive. If true, the TCP keep-alives are enabled, which removes + * connections that are idle for too long. + */ + public boolean enableTcpKeepAlive() { + return conf.getBoolean(SPARK_NETWORK_IO_ENABLETCPKEEPALIVE_KEY, false); + } + /** * Maximum number of retries when binding to a port before giving up. */ @@ -265,6 +275,23 @@ public boolean saslServerAlwaysEncrypt() { return conf.getBoolean("spark.network.sasl.serverAlwaysEncrypt", false); } + /** + * Flag indicating whether to share the pooled ByteBuf allocators between the different Netty + * channels. If enabled then only two pooled ByteBuf allocators are created: one where caching + * is allowed (for transport servers) and one where not (for transport clients). + * When disabled a new allocator is created for each transport servers and clients. + */ + public boolean sharedByteBufAllocators() { + return conf.getBoolean("spark.network.sharedByteBufAllocators.enabled", true); + } + + /** + * If enabled then off-heap byte buffers will be prefered for the shared ByteBuf allocators. + */ + public boolean preferDirectBufsForSharedByteBufAllocators() { + return conf.getBoolean("spark.network.io.preferDirectBufs", true); + } + /** * The commons-crypto configuration for the module. */ @@ -309,8 +336,18 @@ public int chunkFetchHandlerThreads() { } int chunkFetchHandlerThreadsPercent = conf.getInt("spark.shuffle.server.chunkFetchHandlerThreadsPercent", 100); - return (int)Math.ceil( - (this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors()) * - chunkFetchHandlerThreadsPercent/(double)100); + int threads = + this.serverThreads() > 0 ? this.serverThreads() : 2 * NettyRuntime.availableProcessors(); + return (int) Math.ceil(threads * (chunkFetchHandlerThreadsPercent / 100.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 + * job fetching blocks from old version external shuffle service. + */ + public boolean useOldFetchProtocol() { + return conf.getBoolean("spark.shuffle.useOldFetchProtocol", false); } + } diff --git a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java index 8e73ab077a5c1..1980361a15523 100644 --- a/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java +++ b/common/network-common/src/main/java/org/apache/spark/network/util/TransportFrameDecoder.java @@ -19,6 +19,7 @@ import java.util.LinkedList; +import com.google.common.annotations.VisibleForTesting; import com.google.common.base.Preconditions; import io.netty.buffer.ByteBuf; import io.netty.buffer.CompositeByteBuf; @@ -48,14 +49,30 @@ public class TransportFrameDecoder extends ChannelInboundHandlerAdapter { private static final int LENGTH_SIZE = 8; private static final int MAX_FRAME_SIZE = Integer.MAX_VALUE; private static final int UNKNOWN_FRAME_SIZE = -1; + private static final long CONSOLIDATE_THRESHOLD = 20 * 1024 * 1024; private final LinkedList buffers = new LinkedList<>(); private final ByteBuf frameLenBuf = Unpooled.buffer(LENGTH_SIZE, LENGTH_SIZE); + private final long consolidateThreshold; + + private CompositeByteBuf frameBuf = null; + private long consolidatedFrameBufSize = 0; + private int consolidatedNumComponents = 0; private long totalSize = 0; private long nextFrameSize = UNKNOWN_FRAME_SIZE; + private int frameRemainingBytes = UNKNOWN_FRAME_SIZE; private volatile Interceptor interceptor; + public TransportFrameDecoder() { + this(CONSOLIDATE_THRESHOLD); + } + + @VisibleForTesting + TransportFrameDecoder(long consolidateThreshold) { + this.consolidateThreshold = consolidateThreshold; + } + @Override public void channelRead(ChannelHandlerContext ctx, Object data) throws Exception { ByteBuf in = (ByteBuf) data; @@ -123,30 +140,56 @@ private long decodeFrameSize() { private ByteBuf decodeNext() { long frameSize = decodeFrameSize(); - if (frameSize == UNKNOWN_FRAME_SIZE || totalSize < frameSize) { + if (frameSize == UNKNOWN_FRAME_SIZE) { return null; } - // Reset size for next frame. - nextFrameSize = UNKNOWN_FRAME_SIZE; - - Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, "Too large frame: %s", frameSize); - Preconditions.checkArgument(frameSize > 0, "Frame length should be positive: %s", frameSize); + if (frameBuf == null) { + Preconditions.checkArgument(frameSize < MAX_FRAME_SIZE, + "Too large frame: %s", frameSize); + Preconditions.checkArgument(frameSize > 0, + "Frame length should be positive: %s", frameSize); + frameRemainingBytes = (int) frameSize; - // If the first buffer holds the entire frame, return it. - int remaining = (int) frameSize; - if (buffers.getFirst().readableBytes() >= remaining) { - return nextBufferForFrame(remaining); + // If buffers is empty, then return immediately for more input data. + if (buffers.isEmpty()) { + return null; + } + // Otherwise, if the first buffer holds the entire frame, we attempt to + // build frame with it and return. + if (buffers.getFirst().readableBytes() >= frameRemainingBytes) { + // Reset buf and size for next frame. + frameBuf = null; + nextFrameSize = UNKNOWN_FRAME_SIZE; + return nextBufferForFrame(frameRemainingBytes); + } + // Other cases, create a composite buffer to manage all the buffers. + frameBuf = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); } - // Otherwise, create a composite buffer. - CompositeByteBuf frame = buffers.getFirst().alloc().compositeBuffer(Integer.MAX_VALUE); - while (remaining > 0) { - ByteBuf next = nextBufferForFrame(remaining); - remaining -= next.readableBytes(); - frame.addComponent(next).writerIndex(frame.writerIndex() + next.readableBytes()); + while (frameRemainingBytes > 0 && !buffers.isEmpty()) { + ByteBuf next = nextBufferForFrame(frameRemainingBytes); + frameRemainingBytes -= next.readableBytes(); + frameBuf.addComponent(true, next); } - assert remaining == 0; + // If the delta size of frameBuf exceeds the threshold, then we do consolidation + // to reduce memory consumption. + if (frameBuf.capacity() - consolidatedFrameBufSize > consolidateThreshold) { + int newNumComponents = frameBuf.numComponents() - consolidatedNumComponents; + frameBuf.consolidate(consolidatedNumComponents, newNumComponents); + consolidatedFrameBufSize = frameBuf.capacity(); + consolidatedNumComponents = frameBuf.numComponents(); + } + if (frameRemainingBytes > 0) { + return null; + } + + // Reset buf and size for next frame. + ByteBuf frame = frameBuf; + frameBuf = null; + consolidatedFrameBufSize = 0; + consolidatedNumComponents = 0; + nextFrameSize = UNKNOWN_FRAME_SIZE; return frame; } diff --git a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java index 37a8664a52661..5999b6255b37b 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/ChunkFetchIntegrationSuite.java @@ -56,6 +56,7 @@ public class ChunkFetchIntegrationSuite { static final int BUFFER_CHUNK_INDEX = 0; static final int FILE_CHUNK_INDEX = 1; + static TransportContext context; static TransportServer server; static TransportClientFactory clientFactory; static StreamManager streamManager; @@ -117,7 +118,7 @@ public StreamManager getStreamManager() { return streamManager; } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); } @@ -127,6 +128,7 @@ public static void tearDown() { bufferChunk.release(); server.close(); clientFactory.close(); + context.close(); testFile.delete(); } @@ -172,7 +174,7 @@ public void onFailure(int chunkIndex, Throwable e) { for (int chunkIndex : chunkIndices) { client.fetchChunk(STREAM_ID, chunkIndex, callback); } - if (!sem.tryAcquire(chunkIndices.size(), 5, TimeUnit.SECONDS)) { + if (!sem.tryAcquire(chunkIndices.size(), 60, TimeUnit.SECONDS)) { fail("Timeout getting response from the server"); } } 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 2c72c53a33ae8..7e30ed4048ca8 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 @@ -23,6 +23,7 @@ import io.netty.channel.Channel; import org.apache.spark.network.server.ChunkFetchRequestHandler; +import org.junit.Assert; import org.junit.Test; import static org.mockito.Mockito.*; @@ -45,9 +46,8 @@ public void handleChunkFetchRequest() throws Exception { Channel channel = mock(Channel.class); ChannelHandlerContext context = mock(ChannelHandlerContext.class); when(context.channel()) - .thenAnswer(invocationOnMock0 -> { - return channel; - }); + .thenAnswer(invocationOnMock0 -> channel); + List> responseAndPromisePairs = new ArrayList<>(); when(channel.writeAndFlush(any())) @@ -62,41 +62,50 @@ public void handleChunkFetchRequest() throws Exception { List managedBuffers = new ArrayList<>(); managedBuffers.add(new TestManagedBuffer(10)); managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(null); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); TransportClient reverseClient = mock(TransportClient.class); ChunkFetchRequestHandler requestHandler = new ChunkFetchRequestHandler(reverseClient, rpcHandler.getStreamManager(), 2L); RequestMessage request0 = new ChunkFetchRequest(new StreamChunkId(streamId, 0)); requestHandler.channelRead(context, request0); - assert responseAndPromisePairs.size() == 1; - assert responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body() == - managedBuffers.get(0); + Assert.assertEquals(1, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(0), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(0).getLeft())).body()); RequestMessage request1 = new ChunkFetchRequest(new StreamChunkId(streamId, 1)); requestHandler.channelRead(context, request1); - assert responseAndPromisePairs.size() == 2; - assert responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body() == - managedBuffers.get(1); + Assert.assertEquals(2, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(1), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(1).getLeft())).body()); // Finish flushing the response for request0. responseAndPromisePairs.get(0).getRight().finish(true); RequestMessage request2 = new ChunkFetchRequest(new StreamChunkId(streamId, 2)); requestHandler.channelRead(context, request2); - assert responseAndPromisePairs.size() == 3; - assert responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchSuccess; - assert ((ChunkFetchSuccess) (responseAndPromisePairs.get(2).getLeft())).body() == - managedBuffers.get(2); + Assert.assertEquals(3, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof ChunkFetchFailure); + ChunkFetchFailure chunkFetchFailure = + ((ChunkFetchFailure) (responseAndPromisePairs.get(2).getLeft())); + Assert.assertEquals("java.lang.IllegalStateException: Chunk was not found", + chunkFetchFailure.errorString.split("\\r?\\n")[0]); RequestMessage request3 = new ChunkFetchRequest(new StreamChunkId(streamId, 3)); requestHandler.channelRead(context, request3); + Assert.assertEquals(4, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof ChunkFetchSuccess); + Assert.assertEquals(managedBuffers.get(3), + ((ChunkFetchSuccess) (responseAndPromisePairs.get(3).getLeft())).body()); + + RequestMessage request4 = new ChunkFetchRequest(new StreamChunkId(streamId, 4)); + requestHandler.channelRead(context, request4); verify(channel, times(1)).close(); - assert responseAndPromisePairs.size() == 3; + Assert.assertEquals(4, responseAndPromisePairs.size()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java index c0724e018263f..15a28ba249b84 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RequestTimeoutIntegrationSuite.java @@ -48,6 +48,7 @@ */ public class RequestTimeoutIntegrationSuite { + private TransportContext context; private TransportServer server; private TransportClientFactory clientFactory; @@ -79,6 +80,9 @@ public void tearDown() { if (clientFactory != null) { clientFactory.close(); } + if (context != null) { + context.close(); + } } // Basic suite: First request completes quickly, and second waits for longer than network timeout. @@ -106,7 +110,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); @@ -153,7 +157,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); @@ -204,7 +208,7 @@ public StreamManager getStreamManager() { } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); TransportClient client = clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); diff --git a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java index 1f4d75c7e2ec5..117f1e4d00fe3 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/RpcIntegrationSuite.java @@ -44,6 +44,7 @@ public class RpcIntegrationSuite { static TransportConf conf; + static TransportContext context; static TransportServer server; static TransportClientFactory clientFactory; static RpcHandler rpcHandler; @@ -90,7 +91,7 @@ public void receive(TransportClient client, ByteBuffer message) { @Override public StreamManager getStreamManager() { return new OneForOneStreamManager(); } }; - TransportContext context = new TransportContext(conf, rpcHandler); + context = new TransportContext(conf, rpcHandler); server = context.createServer(); clientFactory = context.createClientFactory(); oneWayMsgs = new ArrayList<>(); @@ -160,6 +161,7 @@ public String getID() { public static void tearDown() { server.close(); clientFactory.close(); + context.close(); testData.cleanup(); } @@ -371,23 +373,33 @@ private void assertErrorsContain(Set errors, Set contains) { private void assertErrorAndClosed(RpcResult result, String expectedError) { assertTrue("unexpected success: " + result.successMessages, result.successMessages.isEmpty()); - // we expect 1 additional error, which contains *either* "closed" or "Connection reset" Set errors = result.errorMessages; assertEquals("Expected 2 errors, got " + errors.size() + "errors: " + errors, 2, errors.size()); + // We expect 1 additional error due to closed connection and here are possible keywords in the + // error message. + Set possibleClosedErrors = Sets.newHashSet( + "closed", + "Connection reset", + "java.nio.channels.ClosedChannelException", + "java.io.IOException: Broken pipe" + ); Set containsAndClosed = Sets.newHashSet(expectedError); - containsAndClosed.add("closed"); - containsAndClosed.add("Connection reset"); + containsAndClosed.addAll(possibleClosedErrors); Pair, Set> r = checkErrorsContain(errors, containsAndClosed); - Set errorsNotFound = r.getRight(); - assertEquals(1, errorsNotFound.size()); - String err = errorsNotFound.iterator().next(); - assertTrue(err.equals("closed") || err.equals("Connection reset")); + assertTrue("Got a non-empty set " + r.getLeft(), r.getLeft().isEmpty()); - assertTrue(r.getLeft().isEmpty()); + Set errorsNotFound = r.getRight(); + assertEquals( + "The size of " + errorsNotFound + " was not " + (possibleClosedErrors.size() - 1), + possibleClosedErrors.size() - 1, + errorsNotFound.size()); + for (String err: errorsNotFound) { + assertTrue("Found a wrong error " + err, containsAndClosed.contains(err)); + } } private Pair, Set> checkErrorsContain( diff --git a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java index f3050cb79cdfd..485d8ad554144 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/StreamSuite.java @@ -51,6 +51,7 @@ public class StreamSuite { private static final String[] STREAMS = StreamTestHelper.STREAMS; private static StreamTestHelper testData; + private static TransportContext context; private static TransportServer server; private static TransportClientFactory clientFactory; @@ -93,7 +94,7 @@ public StreamManager getStreamManager() { return streamManager; } }; - TransportContext context = new TransportContext(conf, handler); + context = new TransportContext(conf, handler); server = context.createServer(); clientFactory = context.createClientFactory(); } @@ -103,6 +104,7 @@ public static void tearDown() { server.close(); clientFactory.close(); testData.cleanup(); + context.close(); } @Test 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/TransportClientFactorySuite.java index e95d25fe6ae91..b4caa872928db 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/TransportClientFactorySuite.java @@ -64,6 +64,7 @@ public void setUp() { public void tearDown() { JavaUtils.closeQuietly(server1); JavaUtils.closeQuietly(server2); + JavaUtils.closeQuietly(context); } /** @@ -80,49 +81,50 @@ private void testClientReuse(int maxConnections, boolean concurrent) TransportConf conf = new TransportConf("shuffle", new MapConfigProvider(configMap)); RpcHandler rpcHandler = new NoOpRpcHandler(); - TransportContext context = new TransportContext(conf, rpcHandler); - TransportClientFactory factory = context.createClientFactory(); - Set clients = Collections.synchronizedSet( - new HashSet()); - - AtomicInteger failed = new AtomicInteger(); - Thread[] attempts = new Thread[maxConnections * 10]; - - // Launch a bunch of threads to create new clients. - for (int i = 0; i < attempts.length; i++) { - attempts[i] = new Thread(() -> { - try { - TransportClient client = - factory.createClient(TestUtils.getLocalHost(), server1.getPort()); - assertTrue(client.isActive()); - clients.add(client); - } catch (IOException e) { - failed.incrementAndGet(); - } catch (InterruptedException e) { - throw new RuntimeException(e); + try (TransportContext context = new TransportContext(conf, rpcHandler)) { + TransportClientFactory factory = context.createClientFactory(); + Set clients = Collections.synchronizedSet( + new HashSet()); + + AtomicInteger failed = new AtomicInteger(); + Thread[] attempts = new Thread[maxConnections * 10]; + + // Launch a bunch of threads to create new clients. + for (int i = 0; i < attempts.length; i++) { + attempts[i] = new Thread(() -> { + try { + TransportClient client = + factory.createClient(TestUtils.getLocalHost(), server1.getPort()); + assertTrue(client.isActive()); + clients.add(client); + } catch (IOException e) { + failed.incrementAndGet(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + }); + + if (concurrent) { + attempts[i].start(); + } else { + attempts[i].run(); } - }); + } - if (concurrent) { - attempts[i].start(); - } else { - attempts[i].run(); + // Wait until all the threads complete. + for (Thread attempt : attempts) { + attempt.join(); } - } - // Wait until all the threads complete. - for (Thread attempt : attempts) { - attempt.join(); - } + Assert.assertEquals(0, failed.get()); + Assert.assertTrue(clients.size() <= maxConnections); - Assert.assertEquals(0, failed.get()); - Assert.assertEquals(clients.size(), maxConnections); + for (TransportClient client : clients) { + client.close(); + } - for (TransportClient client : clients) { - client.close(); + factory.close(); } - - factory.close(); } @Test @@ -204,8 +206,8 @@ public Iterable> getAll() { throw new UnsupportedOperationException(); } }); - TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); - try (TransportClientFactory factory = context.createClientFactory()) { + try (TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true); + TransportClientFactory factory = context.createClientFactory()) { TransportClient c1 = factory.createClient(TestUtils.getLocalHost(), server1.getPort()); assertTrue(c1.isActive()); long expiredTime = System.currentTimeMillis() + 10000; // 10 seconds 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 ad640415a8e6d..a43a659048686 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 @@ -21,6 +21,7 @@ import java.util.List; import io.netty.channel.Channel; +import org.junit.Assert; import org.junit.Test; import static org.mockito.Mockito.*; @@ -38,7 +39,7 @@ public class TransportRequestHandlerSuite { @Test - public void handleStreamRequest() throws Exception { + public void handleStreamRequest() { RpcHandler rpcHandler = new NoOpRpcHandler(); OneForOneStreamManager streamManager = (OneForOneStreamManager) (rpcHandler.getStreamManager()); Channel channel = mock(Channel.class); @@ -56,43 +57,56 @@ public void handleStreamRequest() throws Exception { List managedBuffers = new ArrayList<>(); managedBuffers.add(new TestManagedBuffer(10)); managedBuffers.add(new TestManagedBuffer(20)); + managedBuffers.add(null); managedBuffers.add(new TestManagedBuffer(30)); managedBuffers.add(new TestManagedBuffer(40)); - long streamId = streamManager.registerStream("test-app", managedBuffers.iterator()); - streamManager.registerChannel(channel, streamId); + long streamId = streamManager.registerStream("test-app", managedBuffers.iterator(), channel); + + Assert.assertEquals(1, streamManager.numStreamStates()); + TransportClient reverseClient = mock(TransportClient.class); TransportRequestHandler requestHandler = new TransportRequestHandler(channel, reverseClient, rpcHandler, 2L); RequestMessage request0 = new StreamRequest(String.format("%d_%d", streamId, 0)); requestHandler.handle(request0); - assert responseAndPromisePairs.size() == 1; - assert responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body() == - managedBuffers.get(0); + Assert.assertEquals(1, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(0).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(0), + ((StreamResponse) (responseAndPromisePairs.get(0).getLeft())).body()); RequestMessage request1 = new StreamRequest(String.format("%d_%d", streamId, 1)); requestHandler.handle(request1); - assert responseAndPromisePairs.size() == 2; - assert responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body() == - managedBuffers.get(1); + Assert.assertEquals(2, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(1).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(1), + ((StreamResponse) (responseAndPromisePairs.get(1).getLeft())).body()); // Finish flushing the response for request0. responseAndPromisePairs.get(0).getRight().finish(true); - RequestMessage request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); + StreamRequest request2 = new StreamRequest(String.format("%d_%d", streamId, 2)); requestHandler.handle(request2); - assert responseAndPromisePairs.size() == 3; - assert responseAndPromisePairs.get(2).getLeft() instanceof StreamResponse; - assert ((StreamResponse) (responseAndPromisePairs.get(2).getLeft())).body() == - managedBuffers.get(2); + Assert.assertEquals(3, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(2).getLeft() instanceof StreamFailure); + Assert.assertEquals(String.format("Stream '%s' was not found.", request2.streamId), + ((StreamFailure) (responseAndPromisePairs.get(2).getLeft())).error); - // Request3 will trigger the close of channel, because the number of max chunks being - // transferred is 2; RequestMessage request3 = new StreamRequest(String.format("%d_%d", streamId, 3)); requestHandler.handle(request3); + Assert.assertEquals(4, responseAndPromisePairs.size()); + Assert.assertTrue(responseAndPromisePairs.get(3).getLeft() instanceof StreamResponse); + Assert.assertEquals(managedBuffers.get(3), + ((StreamResponse) (responseAndPromisePairs.get(3).getLeft())).body()); + + // Request4 will trigger the close of channel, because the number of max chunks being + // transferred is 2; + RequestMessage request4 = new StreamRequest(String.format("%d_%d", streamId, 4)); + requestHandler.handle(request4); verify(channel, times(1)).close(); - assert responseAndPromisePairs.size() == 3; + Assert.assertEquals(4, responseAndPromisePairs.size()); + + streamManager.connectionTerminated(channel); + Assert.assertEquals(0, streamManager.numStreamStates()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java index c0aa298a4017c..0790f0079c2bd 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthEngineSuite.java @@ -17,16 +17,27 @@ package org.apache.spark.network.crypto; +import java.nio.ByteBuffer; +import java.nio.channels.WritableByteChannel; import java.util.Arrays; import java.util.Map; import java.security.InvalidKeyException; +import java.util.Random; + import static java.nio.charset.StandardCharsets.UTF_8; import com.google.common.collect.ImmutableMap; +import io.netty.buffer.ByteBuf; +import io.netty.buffer.Unpooled; +import io.netty.channel.FileRegion; import org.junit.BeforeClass; import org.junit.Test; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; import static org.junit.Assert.*; +import static org.mockito.Mockito.*; +import org.apache.spark.network.util.ByteArrayWritableChannel; import org.apache.spark.network.util.MapConfigProvider; import org.apache.spark.network.util.TransportConf; @@ -121,4 +132,77 @@ public void testBadKeySize() throws Exception { } } + @Test + public void testEncryptedMessage() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher cipher = server.sessionCipher(); + TransportCipher.EncryptionHandler handler = new TransportCipher.EncryptionHandler(cipher); + + byte[] data = new byte[TransportCipher.STREAM_BUFFER_SIZE + 1]; + new Random().nextBytes(data); + ByteBuf buf = Unpooled.wrappedBuffer(data); + + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(data.length); + TransportCipher.EncryptedMessage emsg = handler.createEncryptedMessage(buf); + while (emsg.transfered() < emsg.count()) { + emsg.transferTo(channel, emsg.transfered()); + } + assertEquals(data.length, channel.length()); + } finally { + client.close(); + server.close(); + } + } + + @Test + public void testEncryptedMessageWhenTransferringZeroBytes() throws Exception { + AuthEngine client = new AuthEngine("appId", "secret", conf); + AuthEngine server = new AuthEngine("appId", "secret", conf); + try { + ClientChallenge clientChallenge = client.challenge(); + ServerResponse serverResponse = server.respond(clientChallenge); + client.validate(serverResponse); + + TransportCipher cipher = server.sessionCipher(); + TransportCipher.EncryptionHandler handler = new TransportCipher.EncryptionHandler(cipher); + + int testDataLength = 4; + FileRegion region = mock(FileRegion.class); + when(region.count()).thenReturn((long) testDataLength); + // Make `region.transferTo` do nothing in first call and transfer 4 bytes in the second one. + when(region.transferTo(any(), anyLong())).thenAnswer(new Answer() { + + private boolean firstTime = true; + + @Override + public Long answer(InvocationOnMock invocationOnMock) throws Throwable { + if (firstTime) { + firstTime = false; + return 0L; + } else { + WritableByteChannel channel = invocationOnMock.getArgument(0); + channel.write(ByteBuffer.wrap(new byte[testDataLength])); + return (long) testDataLength; + } + } + }); + + TransportCipher.EncryptedMessage emsg = handler.createEncryptedMessage(region); + ByteArrayWritableChannel channel = new ByteArrayWritableChannel(testDataLength); + // "transferTo" should act correctly when the underlying FileRegion transfers 0 bytes. + assertEquals(0L, emsg.transferTo(channel, emsg.transfered())); + assertEquals(testDataLength, emsg.transferTo(channel, emsg.transfered())); + assertEquals(emsg.transfered(), emsg.count()); + assertEquals(4, channel.length()); + } finally { + client.close(); + server.close(); + } + } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java index 8751944a1c2a3..2f9dd629df1ba 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/crypto/AuthIntegrationSuite.java @@ -124,6 +124,42 @@ public void testAuthReplay() throws Exception { } } + @Test + public void testLargeMessageEncryption() throws Exception { + // Use a big length to create a message that cannot be put into the encryption buffer completely + final int testErrorMessageLength = TransportCipher.STREAM_BUFFER_SIZE; + ctx = new AuthTestCtx(new RpcHandler() { + @Override + public void receive( + TransportClient client, + ByteBuffer message, + RpcResponseCallback callback) { + char[] longMessage = new char[testErrorMessageLength]; + Arrays.fill(longMessage, 'D'); + callback.onFailure(new RuntimeException(new String(longMessage))); + } + + @Override + public StreamManager getStreamManager() { + return null; + } + }); + ctx.createServer("secret"); + ctx.createClient("secret"); + + try { + ctx.client.sendRpcSync(JavaUtils.stringToBytes("Ping"), 5000); + fail("Should have failed unencrypted RPC."); + } catch (Exception e) { + assertTrue(ctx.authRpcHandler.doDelegate); + assertTrue(e.getMessage() + " is not an expected error", e.getMessage().contains("DDDDD")); + // Verify we receive the complete error message + int messageStart = e.getMessage().indexOf("DDDDD"); + int messageEnd = e.getMessage().lastIndexOf("DDDDD") + 5; + assertEquals(testErrorMessageLength, messageEnd - messageStart); + } + } + private class AuthTestCtx { private final String appId = "testAppId"; @@ -136,10 +172,7 @@ private class AuthTestCtx { volatile AuthRpcHandler authRpcHandler; AuthTestCtx() throws Exception { - Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); - this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); - - RpcHandler rpcHandler = new RpcHandler() { + this(new RpcHandler() { @Override public void receive( TransportClient client, @@ -153,8 +186,12 @@ public void receive( public StreamManager getStreamManager() { return null; } - }; + }); + } + AuthTestCtx(RpcHandler rpcHandler) throws Exception { + Map testConf = ImmutableMap.of("spark.network.crypto.enabled", "true"); + this.conf = new TransportConf("rpc", new MapConfigProvider(testConf)); this.ctx = new TransportContext(conf, rpcHandler); } @@ -196,6 +233,9 @@ void close() { if (server != null) { server.close(); } + if (ctx != null) { + ctx.close(); + } } private SecretKeyHolder createKeyHolder(String secret) { diff --git a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java index 6f15718bd8705..cf2d72f71e8de 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/sasl/SparkSaslSuite.java @@ -347,10 +347,10 @@ public void testRpcHandlerDelegate() throws Exception { verify(handler).getStreamManager(); saslHandler.channelInactive(null); - verify(handler).channelInactive(any(TransportClient.class)); + verify(handler).channelInactive(isNull()); saslHandler.exceptionCaught(null, null); - verify(handler).exceptionCaught(any(Throwable.class), any(TransportClient.class)); + verify(handler).exceptionCaught(isNull(), isNull()); } @Test @@ -365,6 +365,7 @@ private static class SaslTestCtx { final TransportClient client; final TransportServer server; + final TransportContext ctx; private final boolean encrypt; private final boolean disableClientEncryption; @@ -396,7 +397,7 @@ private static class SaslTestCtx { when(keyHolder.getSaslUser(anyString())).thenReturn("user"); when(keyHolder.getSecretKey(anyString())).thenReturn("secret"); - TransportContext ctx = new TransportContext(conf, rpcHandler); + this.ctx = new TransportContext(conf, rpcHandler); this.checker = new EncryptionCheckerBootstrap(SaslEncryption.ENCRYPTION_HANDLER_NAME); @@ -431,6 +432,9 @@ void close() { if (server != null) { server.close(); } + if (ctx != null) { + ctx.close(); + } } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java index c647525d8f1bd..fb3503b783e54 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/server/OneForOneStreamManagerSuite.java @@ -21,6 +21,8 @@ import java.util.List; import io.netty.channel.Channel; +import org.junit.After; +import org.junit.Assert; import org.junit.Test; import org.mockito.Mockito; @@ -29,22 +31,69 @@ public class OneForOneStreamManagerSuite { + List managedBuffersToRelease = new ArrayList<>(); + + @After + public void tearDown() { + managedBuffersToRelease.forEach(managedBuffer -> managedBuffer.release()); + managedBuffersToRelease.clear(); + } + + private ManagedBuffer getChunk(OneForOneStreamManager manager, long streamId, int chunkIndex) { + ManagedBuffer chunk = manager.getChunk(streamId, chunkIndex); + if (chunk != null) { + managedBuffersToRelease.add(chunk); + } + return chunk; + } + @Test - public void managedBuffersAreFeedWhenConnectionIsClosed() throws Exception { + public void testMissingChunk() { OneForOneStreamManager manager = new OneForOneStreamManager(); List buffers = new ArrayList<>(); TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + TestManagedBuffer buffer3 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + // the nulls here are to simulate a file which goes missing before being read, + // just as a defensive measure + buffers.add(null); buffers.add(buffer2); - long streamId = manager.registerStream("appId", buffers.iterator()); + buffers.add(null); + buffers.add(buffer3); Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); - manager.registerChannel(dummyChannel, streamId); + long streamId = manager.registerStream("appId", buffers.iterator(), dummyChannel); + Assert.assertEquals(1, manager.numStreamStates()); + Assert.assertNotNull(getChunk(manager, streamId, 0)); + Assert.assertNull(getChunk(manager, streamId, 1)); + Assert.assertNotNull(getChunk(manager, streamId, 2)); + manager.connectionTerminated(dummyChannel); + + // loaded buffers are not released yet as in production a MangedBuffer returned by getChunk() + // would only be released by Netty after it is written to the network + Mockito.verify(buffer1, Mockito.never()).release(); + Mockito.verify(buffer2, Mockito.never()).release(); + Mockito.verify(buffer3, Mockito.times(1)).release(); + } + @Test + public void managedBuffersAreFreedWhenConnectionIsClosed() { + OneForOneStreamManager manager = new OneForOneStreamManager(); + List buffers = new ArrayList<>(); + TestManagedBuffer buffer1 = Mockito.spy(new TestManagedBuffer(10)); + TestManagedBuffer buffer2 = Mockito.spy(new TestManagedBuffer(20)); + buffers.add(buffer1); + buffers.add(buffer2); + + Channel dummyChannel = Mockito.mock(Channel.class, Mockito.RETURNS_SMART_NULLS); + manager.registerStream("appId", buffers.iterator(), dummyChannel); + Assert.assertEquals(1, manager.numStreamStates()); manager.connectionTerminated(dummyChannel); Mockito.verify(buffer1, Mockito.times(1)).release(); Mockito.verify(buffer2, Mockito.times(1)).release(); + Assert.assertEquals(0, manager.numStreamStates()); } } diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java index 400b385c9703a..f049cad83ff6a 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/NettyMemoryMetricsSuite.java @@ -60,11 +60,14 @@ public void tearDown() { JavaUtils.closeQuietly(clientFactory); clientFactory = null; } - if (server != null) { JavaUtils.closeQuietly(server); server = null; } + if (context != null) { + JavaUtils.closeQuietly(context); + context = null; + } } @Test diff --git a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java index b53e41303751c..4b67aa80351d2 100644 --- a/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java +++ b/common/network-common/src/test/java/org/apache/spark/network/util/TransportFrameDecoderSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.network.util; -import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -28,11 +27,15 @@ import io.netty.channel.ChannelHandlerContext; import org.junit.AfterClass; import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + import static org.junit.Assert.*; import static org.mockito.Mockito.*; public class TransportFrameDecoderSuite { + private static final Logger logger = LoggerFactory.getLogger(TransportFrameDecoderSuite.class); private static Random RND = new Random(); @AfterClass @@ -48,6 +51,69 @@ public void testFrameDecoding() throws Exception { verifyAndCloseDecoder(decoder, ctx, data); } + @Test + public void testConsolidationPerf() throws Exception { + long[] testingConsolidateThresholds = new long[] { + ByteUnit.MiB.toBytes(1), + ByteUnit.MiB.toBytes(5), + ByteUnit.MiB.toBytes(10), + ByteUnit.MiB.toBytes(20), + ByteUnit.MiB.toBytes(30), + ByteUnit.MiB.toBytes(50), + ByteUnit.MiB.toBytes(80), + ByteUnit.MiB.toBytes(100), + ByteUnit.MiB.toBytes(300), + ByteUnit.MiB.toBytes(500), + Long.MAX_VALUE }; + for (long threshold : testingConsolidateThresholds) { + TransportFrameDecoder decoder = new TransportFrameDecoder(threshold); + ChannelHandlerContext ctx = mock(ChannelHandlerContext.class); + List retained = new ArrayList<>(); + when(ctx.fireChannelRead(any())).thenAnswer(in -> { + ByteBuf buf = (ByteBuf) in.getArguments()[0]; + retained.add(buf); + return null; + }); + + // Testing multiple messages + int numMessages = 3; + long targetBytes = ByteUnit.MiB.toBytes(300); + int pieceBytes = (int) ByteUnit.KiB.toBytes(32); + for (int i = 0; i < numMessages; i++) { + try { + long writtenBytes = 0; + long totalTime = 0; + ByteBuf buf = Unpooled.buffer(8); + buf.writeLong(8 + targetBytes); + decoder.channelRead(ctx, buf); + while (writtenBytes < targetBytes) { + buf = Unpooled.buffer(pieceBytes * 2); + ByteBuf writtenBuf = Unpooled.buffer(pieceBytes).writerIndex(pieceBytes); + buf.writeBytes(writtenBuf); + writtenBuf.release(); + long start = System.currentTimeMillis(); + decoder.channelRead(ctx, buf); + long elapsedTime = System.currentTimeMillis() - start; + totalTime += elapsedTime; + writtenBytes += pieceBytes; + } + logger.info("Writing 300MiB frame buf with consolidation of threshold " + threshold + + " took " + totalTime + " milis"); + } finally { + for (ByteBuf buf : retained) { + release(buf); + } + } + } + long totalBytesGot = 0; + for (ByteBuf buf : retained) { + totalBytesGot += buf.capacity(); + } + assertEquals(numMessages, retained.size()); + assertEquals(targetBytes * numMessages, totalBytesGot); + } + } + @Test public void testInterception() throws Exception { int interceptedReads = 3; @@ -69,7 +135,7 @@ public void testInterception() throws Exception { decoder.channelRead(ctx, len); decoder.channelRead(ctx, dataBuf); verify(interceptor, times(interceptedReads)).handle(any(ByteBuf.class)); - verify(ctx).fireChannelRead(any(ByteBuffer.class)); + verify(ctx).fireChannelRead(any(ByteBuf.class)); assertEquals(0, len.refCnt()); assertEquals(0, dataBuf.refCnt()); } finally { diff --git a/common/network-shuffle/pom.xml b/common/network-shuffle/pom.xml index ff717057bb25d..a6d99813a8501 100644 --- a/common/network-shuffle/pom.xml +++ b/common/network-shuffle/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-network-shuffle_2.11 + spark-network-shuffle_2.12 jar Spark Project Shuffle Streaming Service http://spark.apache.org/ diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java similarity index 81% rename from external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala rename to common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java index 47c5187f8751f..01aca7efb12b1 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package.scala +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/Constants.java @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark.streaming +package org.apache.spark.network.shuffle; -/** - * Kafka receiver for spark streaming, - */ -package object kafka +public class Constants { + + public static final String SHUFFLE_SERVICE_FETCH_RDD_ENABLED = + "spark.shuffle.service.fetch.rdd.enabled"; +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java new file mode 100644 index 0000000000000..13f6046dd856b --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExecutorDiskUtils.java @@ -0,0 +1,66 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.spark.network.util.JavaUtils; + +public class ExecutorDiskUtils { + + private static final Pattern MULTIPLE_SEPARATORS = Pattern.compile(File.separator + "{2,}"); + + /** + * Hashes a filename into the corresponding local directory, in a manner consistent with + * Spark's DiskBlockManager.getFile(). + */ + public static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) { + int hash = JavaUtils.nonNegativeHash(filename); + String localDir = localDirs[hash % localDirs.length]; + int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; + return new File(createNormalizedInternedPathname( + localDir, String.format("%02x", subDirId), filename)); + } + + /** + * This method is needed to avoid the situation when multiple File instances for the + * same pathname "foo/bar" are created, each with a separate copy of the "foo/bar" String. + * According to measurements, in some scenarios such duplicate strings may waste a lot + * of memory (~ 10% of the heap). To avoid that, we intern the pathname, and before that + * we make sure that it's in a normalized form (contains no "//", "///" etc.) Otherwise, + * the internal code in java.io.File would normalize it later, creating a new "foo/bar" + * String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File + * uses, since it is in the package-private class java.io.FileSystem. + */ + @VisibleForTesting + static String createNormalizedInternedPathname(String dir1, String dir2, String fname) { + String pathname = dir1 + File.separator + dir2 + File.separator + fname; + Matcher m = MULTIPLE_SEPARATORS.matcher(pathname); + pathname = m.replaceAll("/"); + // A single trailing slash needs to be taken care of separately + if (pathname.length() > 1 && pathname.endsWith("/")) { + pathname = pathname.substring(0, pathname.length() - 1); + } + return pathname.intern(); + } + +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java index 098fa7974b87b..9b7bf254e5b4e 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandler.java @@ -23,12 +23,14 @@ import java.util.HashMap; import java.util.Iterator; import java.util.Map; +import java.util.function.Function; import com.codahale.metrics.Gauge; import com.codahale.metrics.Meter; import com.codahale.metrics.Metric; import com.codahale.metrics.MetricSet; import com.codahale.metrics.Timer; +import com.codahale.metrics.Counter; import com.google.common.annotations.VisibleForTesting; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -47,9 +49,9 @@ /** * RPC Handler for a server which can serve shuffle blocks from outside of an Executor process. * - * Handles registering executors and opening shuffle blocks from them. Shuffle blocks are registered - * with the "one-for-one" strategy, meaning each Transport-layer Chunk is equivalent to one Spark- - * level shuffle block. + * Handles registering executors and opening shuffle or disk persisted RDD blocks from them. + * Blocks are registered with the "one-for-one" strategy, meaning each Transport-layer Chunk + * is equivalent to one block. */ public class ExternalShuffleBlockHandler extends RpcHandler { private static final Logger logger = LoggerFactory.getLogger(ExternalShuffleBlockHandler.class); @@ -65,6 +67,11 @@ public ExternalShuffleBlockHandler(TransportConf conf, File registeredExecutorFi new ExternalShuffleBlockResolver(conf, registeredExecutorFile)); } + @VisibleForTesting + public ExternalShuffleBlockResolver getBlockResolver() { + return blockManager; + } + /** Enables mocking out the StreamManager and BlockManager. */ @VisibleForTesting public ExternalShuffleBlockHandler( @@ -85,21 +92,37 @@ protected void handleMessage( BlockTransferMessage msgObj, TransportClient client, RpcResponseCallback callback) { - if (msgObj instanceof OpenBlocks) { + if (msgObj instanceof FetchShuffleBlocks || msgObj instanceof OpenBlocks) { final Timer.Context responseDelayContext = metrics.openBlockRequestLatencyMillis.time(); try { - OpenBlocks msg = (OpenBlocks) msgObj; - checkAuth(client, msg.appId); - long streamId = streamManager.registerStream(client.getClientId(), - new ManagedBufferIterator(msg.appId, msg.execId, msg.blockIds)); + int numBlockIds; + long streamId; + if (msgObj instanceof FetchShuffleBlocks) { + FetchShuffleBlocks msg = (FetchShuffleBlocks) msgObj; + checkAuth(client, msg.appId); + numBlockIds = 0; + for (int[] ids: msg.reduceIds) { + numBlockIds += ids.length; + } + streamId = streamManager.registerStream(client.getClientId(), + new ManagedBufferIterator(msg, numBlockIds), client.getChannel()); + } else { + // For the compatibility with the old version, still keep the support for OpenBlocks. + OpenBlocks msg = (OpenBlocks) msgObj; + numBlockIds = msg.blockIds.length; + checkAuth(client, msg.appId); + streamId = streamManager.registerStream(client.getClientId(), + new ManagedBufferIterator(msg), client.getChannel()); + } if (logger.isTraceEnabled()) { - logger.trace("Registered streamId {} with {} buffers for client {} from host {}", - streamId, - msg.blockIds.length, - client.getClientId(), - getRemoteAddress(client.getChannel())); + logger.trace( + "Registered streamId {} with {} buffers for client {} from host {}", + streamId, + numBlockIds, + client.getClientId(), + getRemoteAddress(client.getChannel())); } - callback.onSuccess(new StreamHandle(streamId, msg.blockIds.length).toByteBuffer()); + callback.onSuccess(new StreamHandle(streamId, numBlockIds).toByteBuffer()); } finally { responseDelayContext.stop(); } @@ -116,11 +139,22 @@ protected void handleMessage( responseDelayContext.stop(); } + } else if (msgObj instanceof RemoveBlocks) { + RemoveBlocks msg = (RemoveBlocks) msgObj; + checkAuth(client, msg.appId); + int numRemovedBlocks = blockManager.removeBlocks(msg.appId, msg.execId, msg.blockIds); + callback.onSuccess(new BlocksRemoved(numRemovedBlocks).toByteBuffer()); + } else { throw new UnsupportedOperationException("Unexpected message: " + msgObj); } } + @Override + public void exceptionCaught(Throwable cause, TransportClient client) { + metrics.caughtExceptions.inc(); + } + public MetricSet getAllMetrics() { return metrics; } @@ -173,7 +207,8 @@ private void checkAuth(TransportClient client, String appId) { /** * A simple class to wrap all shuffle service wrapper metrics */ - private class ShuffleMetrics implements MetricSet { + @VisibleForTesting + public class ShuffleMetrics implements MetricSet { private final Map allMetrics; // Time latency for open block request in ms private final Timer openBlockRequestLatencyMillis = new Timer(); @@ -181,14 +216,23 @@ private class ShuffleMetrics implements MetricSet { private final Timer registerExecutorRequestLatencyMillis = new Timer(); // Block transfer rate in byte per second private final Meter blockTransferRateBytes = new Meter(); + // Number of active connections to the shuffle service + private Counter activeConnections = new Counter(); + // Number of registered connections to the shuffle service + private Counter registeredConnections = new Counter(); + // Number of exceptions caught in connections to the shuffle service + private Counter caughtExceptions = new Counter(); - private ShuffleMetrics() { + public ShuffleMetrics() { allMetrics = new HashMap<>(); allMetrics.put("openBlockRequestLatencyMillis", openBlockRequestLatencyMillis); allMetrics.put("registerExecutorRequestLatencyMillis", registerExecutorRequestLatencyMillis); allMetrics.put("blockTransferRateBytes", blockTransferRateBytes); allMetrics.put("registeredExecutorsSize", (Gauge) () -> blockManager.getRegisteredExecutorsSize()); + allMetrics.put("numActiveConnections", activeConnections); + allMetrics.put("numRegisteredConnections", registeredConnections); + allMetrics.put("numCaughtExceptions", caughtExceptions); } @Override @@ -200,21 +244,45 @@ public Map getMetrics() { private class ManagedBufferIterator implements Iterator { private int index = 0; - private final String appId; - private final String execId; - private final int shuffleId; - // An array containing mapId and reduceId pairs. - private final int[] mapIdAndReduceIds; - - ManagedBufferIterator(String appId, String execId, String[] blockIds) { - this.appId = appId; - this.execId = execId; + private final Function blockDataForIndexFn; + private final int size; + + ManagedBufferIterator(OpenBlocks msg) { + String appId = msg.appId; + String execId = msg.execId; + String[] blockIds = msg.blockIds; String[] blockId0Parts = blockIds[0].split("_"); - if (blockId0Parts.length != 4 || !blockId0Parts[0].equals("shuffle")) { - throw new IllegalArgumentException("Unexpected shuffle block id format: " + blockIds[0]); + if (blockId0Parts.length == 4 && blockId0Parts[0].equals("shuffle")) { + final int shuffleId = Integer.parseInt(blockId0Parts[1]); + final int[] mapIdAndReduceIds = shuffleMapIdAndReduceIds(blockIds, shuffleId); + size = mapIdAndReduceIds.length; + blockDataForIndexFn = index -> blockManager.getBlockData(appId, execId, shuffleId, + mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + } else if (blockId0Parts.length == 3 && blockId0Parts[0].equals("rdd")) { + final int[] rddAndSplitIds = rddAndSplitIds(blockIds); + size = rddAndSplitIds.length; + blockDataForIndexFn = index -> blockManager.getRddBlockData(appId, execId, + rddAndSplitIds[index], rddAndSplitIds[index + 1]); + } else { + throw new IllegalArgumentException("Unexpected block id format: " + blockIds[0]); } - this.shuffleId = Integer.parseInt(blockId0Parts[1]); - mapIdAndReduceIds = new int[2 * blockIds.length]; + } + + private int[] rddAndSplitIds(String[] blockIds) { + final int[] rddAndSplitIds = new int[2 * blockIds.length]; + for (int i = 0; i < blockIds.length; i++) { + String[] blockIdParts = blockIds[i].split("_"); + if (blockIdParts.length != 3 || !blockIdParts[0].equals("rdd")) { + throw new IllegalArgumentException("Unexpected RDD block id format: " + blockIds[i]); + } + rddAndSplitIds[2 * i] = Integer.parseInt(blockIdParts[1]); + rddAndSplitIds[2 * i + 1] = Integer.parseInt(blockIdParts[2]); + } + return rddAndSplitIds; + } + + private int[] shuffleMapIdAndReduceIds(String[] blockIds, int shuffleId) { + final int[] mapIdAndReduceIds = new int[2 * blockIds.length]; for (int i = 0; i < blockIds.length; i++) { String[] blockIdParts = blockIds[i].split("_"); if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { @@ -227,21 +295,48 @@ private class ManagedBufferIterator implements Iterator { mapIdAndReduceIds[2 * i] = Integer.parseInt(blockIdParts[2]); mapIdAndReduceIds[2 * i + 1] = Integer.parseInt(blockIdParts[3]); } + return mapIdAndReduceIds; + } + + ManagedBufferIterator(FetchShuffleBlocks msg, int numBlockIds) { + final int[] mapIdAndReduceIds = new int[2 * numBlockIds]; + int idx = 0; + for (int i = 0; i < msg.mapIds.length; i++) { + for (int reduceId : msg.reduceIds[i]) { + mapIdAndReduceIds[idx++] = msg.mapIds[i]; + mapIdAndReduceIds[idx++] = reduceId; + } + } + assert(idx == 2 * numBlockIds); + size = mapIdAndReduceIds.length; + blockDataForIndexFn = index -> blockManager.getBlockData(msg.appId, msg.execId, + msg.shuffleId, mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); } @Override public boolean hasNext() { - return index < mapIdAndReduceIds.length; + return index < size; } @Override public ManagedBuffer next() { - final ManagedBuffer block = blockManager.getBlockData(appId, execId, shuffleId, - mapIdAndReduceIds[index], mapIdAndReduceIds[index + 1]); + final ManagedBuffer block = blockDataForIndexFn.apply(index); index += 2; metrics.blockTransferRateBytes.mark(block != null ? block.size() : 0); return block; } } + @Override + public void channelActive(TransportClient client) { + metrics.activeConnections.inc(); + super.channelActive(client); + } + + @Override + public void channelInactive(TransportClient client) { + metrics.activeConnections.dec(); + super.channelInactive(client); + } + } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java index 0b7a27402369d..50f16fc700f12 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolver.java @@ -24,7 +24,6 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Executor; import java.util.concurrent.Executors; -import java.util.regex.Matcher; import java.util.regex.Pattern; import com.fasterxml.jackson.annotation.JsonCreator; @@ -86,6 +85,8 @@ public class ExternalShuffleBlockResolver { private final TransportConf conf; + private final boolean rddFetchEnabled; + @VisibleForTesting final File registeredExecutorFile; @VisibleForTesting @@ -109,6 +110,8 @@ public ExternalShuffleBlockResolver(TransportConf conf, File registeredExecutorF File registeredExecutorFile, Executor directoryCleaner) throws IOException { this.conf = conf; + this.rddFetchEnabled = + Boolean.valueOf(conf.get(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, "false")); this.registeredExecutorFile = registeredExecutorFile; String indexCacheSize = conf.get("spark.shuffle.service.index.cache.size", "100m"); CacheLoader indexCacheLoader = @@ -179,6 +182,18 @@ public ManagedBuffer getBlockData( return getSortBasedShuffleBlockData(executor, shuffleId, mapId, reduceId); } + public ManagedBuffer getRddBlockData( + String appId, + String execId, + int rddId, + int splitIndex) { + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + return getDiskPersistedRddBlockData(executor, rddId, splitIndex); + } /** * Removes our metadata of all executors registered for the given application, and optionally * also deletes the local directories associated with the executors of that application in a @@ -217,22 +232,23 @@ public void applicationRemoved(String appId, boolean cleanupLocalDirs) { } /** - * Removes all the non-shuffle files in any local directories associated with the finished - * executor. + * Removes all the files which cannot be served by the external shuffle service (non-shuffle and + * non-RDD files) in any local directories associated with the finished executor. */ public void executorRemoved(String executorId, String appId) { - logger.info("Clean up non-shuffle files associated with the finished executor {}", executorId); + logger.info("Clean up non-shuffle and non-RDD files associated with the finished executor {}", + executorId); AppExecId fullId = new AppExecId(appId, executorId); final ExecutorShuffleInfo executor = executors.get(fullId); if (executor == null) { // Executor not registered, skip clean up of the local directories. logger.info("Executor is not registered (appId={}, execId={})", appId, executorId); } else { - logger.info("Cleaning up non-shuffle files in executor {}'s {} local dirs", fullId, - executor.localDirs.length); + logger.info("Cleaning up non-shuffle and non-RDD files in executor {}'s {} local dirs", + fullId, executor.localDirs.length); // Execute the actual deletion in a different thread, as it may take some time. - directoryCleaner.execute(() -> deleteNonShuffleFiles(executor.localDirs)); + directoryCleaner.execute(() -> deleteNonShuffleServiceServedFiles(executor.localDirs)); } } @@ -252,24 +268,24 @@ private void deleteExecutorDirs(String[] dirs) { } /** - * Synchronously deletes non-shuffle files in each directory recursively. + * Synchronously deletes files not served by shuffle service in each directory recursively. * Should be executed in its own thread, as this may take a long time. */ - private void deleteNonShuffleFiles(String[] dirs) { - FilenameFilter filter = new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - // Don't delete shuffle data or shuffle index files. - return !name.endsWith(".index") && !name.endsWith(".data"); - } + private void deleteNonShuffleServiceServedFiles(String[] dirs) { + FilenameFilter filter = (dir, name) -> { + // Don't delete shuffle data, shuffle index files or cached RDD files. + return !name.endsWith(".index") && !name.endsWith(".data") + && (!rddFetchEnabled || !name.startsWith("rdd_")); }; for (String localDir : dirs) { try { JavaUtils.deleteRecursively(new File(localDir), filter); - logger.debug("Successfully cleaned up non-shuffle files in directory: {}", localDir); + logger.debug("Successfully cleaned up files not served by shuffle service in directory: {}", + localDir); } catch (Exception e) { - logger.error("Failed to delete non-shuffle files in directory: " + localDir, e); + logger.error("Failed to delete files not served by shuffle service in directory: " + + localDir, e); } } } @@ -281,7 +297,7 @@ public boolean accept(File dir, String name) { */ private ManagedBuffer getSortBasedShuffleBlockData( ExecutorShuffleInfo executor, int shuffleId, int mapId, int reduceId) { - File indexFile = getFile(executor.localDirs, executor.subDirsPerLocalDir, + File indexFile = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.index"); try { @@ -289,7 +305,7 @@ private ManagedBuffer getSortBasedShuffleBlockData( ShuffleIndexRecord shuffleIndexRecord = shuffleIndexInformation.getIndex(reduceId); return new FileSegmentManagedBuffer( conf, - getFile(executor.localDirs, executor.subDirsPerLocalDir, + ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, "shuffle_" + shuffleId + "_" + mapId + "_0.data"), shuffleIndexRecord.getOffset(), shuffleIndexRecord.getLength()); @@ -298,17 +314,16 @@ private ManagedBuffer getSortBasedShuffleBlockData( } } - /** - * Hashes a filename into the corresponding local directory, in a manner consistent with - * Spark's DiskBlockManager.getFile(). - */ - @VisibleForTesting - static File getFile(String[] localDirs, int subDirsPerLocalDir, String filename) { - int hash = JavaUtils.nonNegativeHash(filename); - String localDir = localDirs[hash % localDirs.length]; - int subDirId = (hash / localDirs.length) % subDirsPerLocalDir; - return new File(createNormalizedInternedPathname( - localDir, String.format("%02x", subDirId), filename)); + public ManagedBuffer getDiskPersistedRddBlockData( + ExecutorShuffleInfo executor, int rddId, int splitIndex) { + File file = ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, + "rdd_" + rddId + "_" + splitIndex); + long fileLength = file.length(); + ManagedBuffer res = null; + if (file.exists()) { + res = new FileSegmentManagedBuffer(conf, file, 0, fileLength); + } + return res; } void close() { @@ -321,26 +336,23 @@ void close() { } } - /** - * This method is needed to avoid the situation when multiple File instances for the - * same pathname "foo/bar" are created, each with a separate copy of the "foo/bar" String. - * According to measurements, in some scenarios such duplicate strings may waste a lot - * of memory (~ 10% of the heap). To avoid that, we intern the pathname, and before that - * we make sure that it's in a normalized form (contains no "//", "///" etc.) Otherwise, - * the internal code in java.io.File would normalize it later, creating a new "foo/bar" - * String copy. Unfortunately, we cannot just reuse the normalization code that java.io.File - * uses, since it is in the package-private class java.io.FileSystem. - */ - @VisibleForTesting - static String createNormalizedInternedPathname(String dir1, String dir2, String fname) { - String pathname = dir1 + File.separator + dir2 + File.separator + fname; - Matcher m = MULTIPLE_SEPARATORS.matcher(pathname); - pathname = m.replaceAll("/"); - // A single trailing slash needs to be taken care of separately - if (pathname.length() > 1 && pathname.endsWith("/")) { - pathname = pathname.substring(0, pathname.length() - 1); + public int removeBlocks(String appId, String execId, String[] blockIds) { + ExecutorShuffleInfo executor = executors.get(new AppExecId(appId, execId)); + if (executor == null) { + throw new RuntimeException( + String.format("Executor is not registered (appId=%s, execId=%s)", appId, execId)); + } + int numRemovedBlocks = 0; + for (String blockId : blockIds) { + File file = + ExecutorDiskUtils.getFile(executor.localDirs, executor.subDirsPerLocalDir, blockId); + if (file.delete()) { + numRemovedBlocks++; + } else { + logger.warn("Failed to delete block: " + file.getAbsolutePath()); + } } - return pathname.intern(); + return numRemovedBlocks; } /** Simply encodes an executor's full ID, which is appId + execId. */ diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java index e49e27ab5aa79..2100b89468020 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ExternalShuffleClient.java @@ -19,10 +19,15 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.Arrays; import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.Future; import com.codahale.metrics.MetricSet; import com.google.common.collect.Lists; +import org.apache.spark.network.client.RpcResponseCallback; +import org.apache.spark.network.shuffle.protocol.*; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -33,8 +38,6 @@ import org.apache.spark.network.crypto.AuthClientBootstrap; import org.apache.spark.network.sasl.SecretKeyHolder; import org.apache.spark.network.server.NoOpRpcHandler; -import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; -import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.util.TransportConf; /** @@ -73,7 +76,10 @@ protected void checkInit() { assert appId != null : "Called before init()"; } - @Override + /** + * Initializes the ShuffleClient, specifying this Executor's appId. + * Must be called before any other method on the ShuffleClient. + */ public void init(String appId) { this.appId = appId; TransportContext context = new TransportContext(conf, new NoOpRpcHandler(), true, true); @@ -139,12 +145,47 @@ public void registerWithShuffleServer( String execId, ExecutorShuffleInfo executorInfo) throws IOException, InterruptedException { checkInit(); - try (TransportClient client = clientFactory.createUnmanagedClient(host, port)) { + try (TransportClient client = clientFactory.createClient(host, port)) { ByteBuffer registerMessage = new RegisterExecutor(appId, execId, executorInfo).toByteBuffer(); client.sendRpcSync(registerMessage, registrationTimeoutMs); } } + public Future removeBlocks( + String host, + int port, + String execId, + String[] blockIds) throws IOException, InterruptedException { + checkInit(); + CompletableFuture numRemovedBlocksFuture = new CompletableFuture<>(); + ByteBuffer removeBlocksMessage = new RemoveBlocks(appId, execId, blockIds).toByteBuffer(); + final TransportClient client = clientFactory.createClient(host, port); + client.sendRpc(removeBlocksMessage, new RpcResponseCallback() { + @Override + public void onSuccess(ByteBuffer response) { + try { + BlockTransferMessage msgObj = BlockTransferMessage.Decoder.fromByteBuffer(response); + numRemovedBlocksFuture.complete(((BlocksRemoved) msgObj).numRemovedBlocks); + } catch (Throwable t) { + logger.warn("Error trying to remove RDD blocks " + Arrays.toString(blockIds) + + " via external shuffle service from executor: " + execId, t); + numRemovedBlocksFuture.complete(0); + } finally { + client.close(); + } + } + + @Override + public void onFailure(Throwable e) { + logger.warn("Error trying to remove RDD blocks " + Arrays.toString(blockIds) + + " via external shuffle service from executor: " + execId, e); + numRemovedBlocksFuture.complete(0); + client.close(); + } + }); + return numRemovedBlocksFuture; + } + @Override public void close() { checkInit(); diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java index 30587023877c1..cc11e92067375 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/OneForOneBlockFetcher.java @@ -19,8 +19,11 @@ import java.io.IOException; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; +import java.util.HashMap; +import com.google.common.primitives.Ints; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -31,6 +34,7 @@ import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.server.OneForOneStreamManager; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.TransportConf; @@ -48,7 +52,7 @@ public class OneForOneBlockFetcher { private static final Logger logger = LoggerFactory.getLogger(OneForOneBlockFetcher.class); private final TransportClient client; - private final OpenBlocks openMessage; + private final BlockTransferMessage message; private final String[] blockIds; private final BlockFetchingListener listener; private final ChunkReceivedCallback chunkCallback; @@ -76,12 +80,71 @@ public OneForOneBlockFetcher( TransportConf transportConf, DownloadFileManager downloadFileManager) { this.client = client; - this.openMessage = new OpenBlocks(appId, execId, blockIds); this.blockIds = blockIds; this.listener = listener; this.chunkCallback = new ChunkCallback(); this.transportConf = transportConf; this.downloadFileManager = downloadFileManager; + if (blockIds.length == 0) { + throw new IllegalArgumentException("Zero-sized blockIds array"); + } + if (!transportConf.useOldFetchProtocol() && isShuffleBlocks(blockIds)) { + this.message = createFetchShuffleBlocksMsg(appId, execId, blockIds); + } else { + this.message = new OpenBlocks(appId, execId, blockIds); + } + } + + private boolean isShuffleBlocks(String[] blockIds) { + for (String blockId : blockIds) { + if (!blockId.startsWith("shuffle_")) { + return false; + } + } + return true; + } + + /** + * Analyze the pass in blockIds and create FetchShuffleBlocks message. + * The blockIds has been sorted by mapId and reduceId. It's produced in + * org.apache.spark.MapOutputTracker.convertMapStatuses. + */ + private FetchShuffleBlocks createFetchShuffleBlocksMsg( + String appId, String execId, String[] blockIds) { + int shuffleId = splitBlockId(blockIds[0])[0]; + HashMap> mapIdToReduceIds = new HashMap<>(); + for (String blockId : blockIds) { + int[] blockIdParts = splitBlockId(blockId); + if (blockIdParts[0] != shuffleId) { + throw new IllegalArgumentException("Expected shuffleId=" + shuffleId + + ", got:" + blockId); + } + int mapId = blockIdParts[1]; + if (!mapIdToReduceIds.containsKey(mapId)) { + mapIdToReduceIds.put(mapId, new ArrayList<>()); + } + mapIdToReduceIds.get(mapId).add(blockIdParts[2]); + } + int[] mapIds = Ints.toArray(mapIdToReduceIds.keySet()); + int[][] reduceIdArr = new int[mapIds.length][]; + for (int i = 0; i < mapIds.length; i++) { + reduceIdArr[i] = Ints.toArray(mapIdToReduceIds.get(mapIds[i])); + } + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIdArr); + } + + /** Split the shuffleBlockId and return shuffleId, mapId and reduceId. */ + private int[] splitBlockId(String blockId) { + String[] blockIdParts = blockId.split("_"); + if (blockIdParts.length != 4 || !blockIdParts[0].equals("shuffle")) { + throw new IllegalArgumentException( + "Unexpected shuffle block id format: " + blockId); + } + return new int[] { + Integer.parseInt(blockIdParts[1]), + Integer.parseInt(blockIdParts[2]), + Integer.parseInt(blockIdParts[3]) + }; } /** Callback invoked on receipt of each chunk. We equate a single chunk to a single block. */ @@ -106,11 +169,7 @@ public void onFailure(int chunkIndex, Throwable e) { * {@link StreamHandle}. We will send all fetch requests immediately, without throttling. */ public void start() { - if (blockIds.length == 0) { - throw new IllegalArgumentException("Zero-sized blockIds array"); - } - - client.sendRpc(openMessage.toByteBuffer(), new RpcResponseCallback() { + client.sendRpc(message.toByteBuffer(), new RpcResponseCallback() { @Override public void onSuccess(ByteBuffer response) { try { diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java index f309dda8afca6..6bf3da94030d4 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/RetryingBlockFetcher.java @@ -101,7 +101,7 @@ void createAndStart(String[] blockIds, BlockFetchingListener listener) public RetryingBlockFetcher( TransportConf conf, - BlockFetchStarter fetchStarter, + RetryingBlockFetcher.BlockFetchStarter fetchStarter, String[] blockIds, BlockFetchingListener listener) { this.fetchStarter = fetchStarter; diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java index 62b99c40f61f9..0be5cf5ad922f 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/ShuffleClient.java @@ -25,12 +25,6 @@ /** Provides an interface for reading shuffle files, either from an Executor or external service. */ public abstract class ShuffleClient implements Closeable { - /** - * Initializes the ShuffleClient, specifying this Executor's appId. - * Must be called before any other method on the ShuffleClient. - */ - public void init(String appId) { } - /** * Fetch a sequence of blocks from a remote node asynchronously, * diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java index a68a297519b66..29d7edc258137 100644 --- a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlockTransferMessage.java @@ -31,10 +31,13 @@ * by Spark's NettyBlockTransferService. * * At a high level: - * - OpenBlock is handled by both services, but only services shuffle files for the external - * shuffle service. It returns a StreamHandle. + * - OpenBlock is logically only handled by the NettyBlockTransferService, but for the capability + * for old version Spark, we still keep it in external shuffle service. + * It returns a StreamHandle. * - UploadBlock is only handled by the NettyBlockTransferService. * - RegisterExecutor is only handled by the external shuffle service. + * - RemoveBlocks is only handled by the external shuffle service. + * - FetchShuffleBlocks is handled by both services for shuffle files. It returns a StreamHandle. */ public abstract class BlockTransferMessage implements Encodable { protected abstract Type type(); @@ -42,7 +45,8 @@ public abstract class BlockTransferMessage implements Encodable { /** Preceding every serialized message is its type, which allows us to deserialize it. */ public enum Type { OPEN_BLOCKS(0), UPLOAD_BLOCK(1), REGISTER_EXECUTOR(2), STREAM_HANDLE(3), REGISTER_DRIVER(4), - HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6); + HEARTBEAT(5), UPLOAD_BLOCK_STREAM(6), REMOVE_BLOCKS(7), BLOCKS_REMOVED(8), + FETCH_SHUFFLE_BLOCKS(9); private final byte id; @@ -68,6 +72,9 @@ public static BlockTransferMessage fromByteBuffer(ByteBuffer msg) { case 4: return RegisterDriver.decode(buf); case 5: return ShuffleServiceHeartbeat.decode(buf); case 6: return UploadBlockStream.decode(buf); + case 7: return RemoveBlocks.decode(buf); + case 8: return BlocksRemoved.decode(buf); + case 9: return FetchShuffleBlocks.decode(buf); default: throw new IllegalArgumentException("Unknown message type: " + type); } } diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java new file mode 100644 index 0000000000000..3f04443871b68 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/BlocksRemoved.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** The reply to remove blocks giving back the number of removed blocks. */ +public class BlocksRemoved extends BlockTransferMessage { + public final int numRemovedBlocks; + + public BlocksRemoved(int numRemovedBlocks) { + this.numRemovedBlocks = numRemovedBlocks; + } + + @Override + protected Type type() { return Type.BLOCKS_REMOVED; } + + @Override + public int hashCode() { + return Objects.hashCode(numRemovedBlocks); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("numRemovedBlocks", numRemovedBlocks) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof BlocksRemoved) { + BlocksRemoved o = (BlocksRemoved) other; + return Objects.equal(numRemovedBlocks, o.numRemovedBlocks); + } + return false; + } + + @Override + public int encodedLength() { + return 4; + } + + @Override + public void encode(ByteBuf buf) { + buf.writeInt(numRemovedBlocks); + } + + public static BlocksRemoved decode(ByteBuf buf) { + int numRemovedBlocks = buf.readInt(); + return new BlocksRemoved(numRemovedBlocks); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java new file mode 100644 index 0000000000000..466eeb3e048a8 --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/FetchShuffleBlocks.java @@ -0,0 +1,130 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import java.util.Arrays; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; + +import org.apache.spark.network.protocol.Encoders; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** Request to read a set of blocks. Returns {@link StreamHandle}. */ +public class FetchShuffleBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final int shuffleId; + // The length of mapIds must equal to reduceIds.size(), for the i-th mapId in mapIds, + // it corresponds to the i-th int[] in reduceIds, which contains all reduce id for this map id. + public final int[] mapIds; + public final int[][] reduceIds; + + public FetchShuffleBlocks( + String appId, + String execId, + int shuffleId, + int[] mapIds, + int[][] reduceIds) { + this.appId = appId; + this.execId = execId; + this.shuffleId = shuffleId; + this.mapIds = mapIds; + this.reduceIds = reduceIds; + assert(mapIds.length == reduceIds.length); + } + + @Override + protected Type type() { return Type.FETCH_SHUFFLE_BLOCKS; } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("shuffleId", shuffleId) + .add("mapIds", Arrays.toString(mapIds)) + .add("reduceIds", Arrays.deepToString(reduceIds)) + .toString(); + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + + FetchShuffleBlocks that = (FetchShuffleBlocks) o; + + if (shuffleId != that.shuffleId) return false; + if (!appId.equals(that.appId)) return false; + if (!execId.equals(that.execId)) return false; + if (!Arrays.equals(mapIds, that.mapIds)) return false; + return Arrays.deepEquals(reduceIds, that.reduceIds); + } + + @Override + public int hashCode() { + int result = appId.hashCode(); + result = 31 * result + execId.hashCode(); + result = 31 * result + shuffleId; + result = 31 * result + Arrays.hashCode(mapIds); + result = 31 * result + Arrays.deepHashCode(reduceIds); + return result; + } + + @Override + public int encodedLength() { + int encodedLengthOfReduceIds = 0; + for (int[] ids: reduceIds) { + encodedLengthOfReduceIds += Encoders.IntArrays.encodedLength(ids); + } + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + 4 /* encoded length of shuffleId */ + + Encoders.IntArrays.encodedLength(mapIds) + + 4 /* encoded length of reduceIds.size() */ + + encodedLengthOfReduceIds; + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + buf.writeInt(shuffleId); + Encoders.IntArrays.encode(buf, mapIds); + buf.writeInt(reduceIds.length); + for (int[] ids: reduceIds) { + Encoders.IntArrays.encode(buf, ids); + } + } + + public static FetchShuffleBlocks decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + int shuffleId = buf.readInt(); + int[] mapIds = Encoders.IntArrays.decode(buf); + int reduceIdsSize = buf.readInt(); + int[][] reduceIds = new int[reduceIdsSize][]; + for (int i = 0; i < reduceIdsSize; i++) { + reduceIds[i] = Encoders.IntArrays.decode(buf); + } + return new FetchShuffleBlocks(appId, execId, shuffleId, mapIds, reduceIds); + } +} diff --git a/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java new file mode 100644 index 0000000000000..1c718d307753f --- /dev/null +++ b/common/network-shuffle/src/main/java/org/apache/spark/network/shuffle/protocol/RemoveBlocks.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle.protocol; + +import com.google.common.base.Objects; +import io.netty.buffer.ByteBuf; +import org.apache.spark.network.protocol.Encoders; + +import java.util.Arrays; + +// Needed by ScalaDoc. See SPARK-7726 +import static org.apache.spark.network.shuffle.protocol.BlockTransferMessage.Type; + +/** Request to remove a set of blocks. */ +public class RemoveBlocks extends BlockTransferMessage { + public final String appId; + public final String execId; + public final String[] blockIds; + + public RemoveBlocks(String appId, String execId, String[] blockIds) { + this.appId = appId; + this.execId = execId; + this.blockIds = blockIds; + } + + @Override + protected Type type() { return Type.REMOVE_BLOCKS; } + + @Override + public int hashCode() { + return Objects.hashCode(appId, execId) * 41 + Arrays.hashCode(blockIds); + } + + @Override + public String toString() { + return Objects.toStringHelper(this) + .add("appId", appId) + .add("execId", execId) + .add("blockIds", Arrays.toString(blockIds)) + .toString(); + } + + @Override + public boolean equals(Object other) { + if (other != null && other instanceof RemoveBlocks) { + RemoveBlocks o = (RemoveBlocks) other; + return Objects.equal(appId, o.appId) + && Objects.equal(execId, o.execId) + && Arrays.equals(blockIds, o.blockIds); + } + return false; + } + + @Override + public int encodedLength() { + return Encoders.Strings.encodedLength(appId) + + Encoders.Strings.encodedLength(execId) + + Encoders.StringArrays.encodedLength(blockIds); + } + + @Override + public void encode(ByteBuf buf) { + Encoders.Strings.encode(buf, appId); + Encoders.Strings.encode(buf, execId); + Encoders.StringArrays.encode(buf, blockIds); + } + + public static RemoveBlocks decode(ByteBuf buf) { + String appId = Encoders.Strings.decode(buf); + String execId = Encoders.Strings.decode(buf); + String[] blockIds = Encoders.StringArrays.decode(buf); + return new RemoveBlocks(appId, execId, blockIds); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java index 02e6eb3a4467e..57c1c5e7722ce 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/sasl/SaslIntegrationSuite.java @@ -91,6 +91,7 @@ public static void beforeAll() throws IOException { @AfterClass public static void afterAll() { server.close(); + context.close(); } @After @@ -153,13 +154,14 @@ public void testNoSaslClient() throws IOException, InterruptedException { @Test public void testNoSaslServer() { RpcHandler handler = new TestRpcHandler(); - TransportContext context = new TransportContext(conf, handler); - clientFactory = context.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - try (TransportServer server = context.createServer()) { - clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); - } catch (Exception e) { - assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); + try (TransportContext context = new TransportContext(conf, handler)) { + clientFactory = context.createClientFactory( + Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); + try (TransportServer server = context.createServer()) { + clientFactory.createClient(TestUtils.getLocalHost(), server.getPort()); + } catch (Exception e) { + assertTrue(e.getMessage(), e.getMessage().contains("Digest-challenge format violation")); + } } } @@ -174,18 +176,15 @@ public void testAppIsolation() throws Exception { ExternalShuffleBlockHandler blockHandler = new ExternalShuffleBlockHandler( new OneForOneStreamManager(), blockResolver); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, secretKeyHolder); - TransportContext blockServerContext = new TransportContext(conf, blockHandler); - TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); - TransportClient client1 = null; - TransportClient client2 = null; - TransportClientFactory clientFactory2 = null; - try { + try ( + TransportContext blockServerContext = new TransportContext(conf, blockHandler); + TransportServer blockServer = blockServerContext.createServer(Arrays.asList(bootstrap)); // Create a client, and make a request to fetch blocks from a different app. - clientFactory = blockServerContext.createClientFactory( + TransportClientFactory clientFactory1 = blockServerContext.createClientFactory( Arrays.asList(new SaslClientBootstrap(conf, "app-1", secretKeyHolder))); - client1 = clientFactory.createClient(TestUtils.getLocalHost(), - blockServer.getPort()); + TransportClient client1 = clientFactory1.createClient( + TestUtils.getLocalHost(), blockServer.getPort())) { AtomicReference exception = new AtomicReference<>(); @@ -223,41 +222,33 @@ public void onBlockFetchFailure(String blockId, Throwable t) { StreamHandle stream = (StreamHandle) BlockTransferMessage.Decoder.fromByteBuffer(response); long streamId = stream.streamId; - // Create a second client, authenticated with a different app ID, and try to read from - // the stream created for the previous app. - clientFactory2 = blockServerContext.createClientFactory( - Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); - client2 = clientFactory2.createClient(TestUtils.getLocalHost(), - blockServer.getPort()); - - CountDownLatch chunkReceivedLatch = new CountDownLatch(1); - ChunkReceivedCallback callback = new ChunkReceivedCallback() { - @Override - public void onSuccess(int chunkIndex, ManagedBuffer buffer) { - chunkReceivedLatch.countDown(); - } - @Override - public void onFailure(int chunkIndex, Throwable t) { - exception.set(t); - chunkReceivedLatch.countDown(); - } - }; - - exception.set(null); - client2.fetchChunk(streamId, 0, callback); - chunkReceivedLatch.await(); - checkSecurityException(exception.get()); - } finally { - if (client1 != null) { - client1.close(); - } - if (client2 != null) { - client2.close(); - } - if (clientFactory2 != null) { - clientFactory2.close(); + try ( + // Create a second client, authenticated with a different app ID, and try to read from + // the stream created for the previous app. + TransportClientFactory clientFactory2 = blockServerContext.createClientFactory( + Arrays.asList(new SaslClientBootstrap(conf, "app-2", secretKeyHolder))); + TransportClient client2 = clientFactory2.createClient( + TestUtils.getLocalHost(), blockServer.getPort()) + ) { + CountDownLatch chunkReceivedLatch = new CountDownLatch(1); + ChunkReceivedCallback callback = new ChunkReceivedCallback() { + @Override + public void onSuccess(int chunkIndex, ManagedBuffer buffer) { + chunkReceivedLatch.countDown(); + } + + @Override + public void onFailure(int chunkIndex, Throwable t) { + exception.set(t); + chunkReceivedLatch.countDown(); + } + }; + + exception.set(null); + client2.fetchChunk(streamId, 0, callback); + chunkReceivedLatch.await(); + checkSecurityException(exception.get()); } - blockServer.close(); } } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java index 86c8609e7070b..649c471dc1679 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/BlockTransferMessagesSuite.java @@ -28,6 +28,9 @@ public class BlockTransferMessagesSuite { @Test public void serializeOpenShuffleBlocks() { checkSerializeDeserialize(new OpenBlocks("app-1", "exec-2", new String[] { "b1", "b2" })); + checkSerializeDeserialize(new FetchShuffleBlocks( + "app-1", "exec-2", 0, new int[] {0, 1}, + new int[][] {{ 0, 1 }, { 0, 1, 2 }})); checkSerializeDeserialize(new RegisterExecutor("app-1", "exec-2", new ExecutorShuffleInfo( new String[] { "/local1", "/local2" }, 32, "MyShuffleManager"))); checkSerializeDeserialize(new UploadBlock("app-1", "exec-2", "block-3", new byte[] { 1, 2 }, diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java new file mode 100644 index 0000000000000..e38442327e22d --- /dev/null +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/CleanupNonShuffleServiceServedFilesSuite.java @@ -0,0 +1,256 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.shuffle; + +import java.io.File; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.nio.file.Path; +import java.util.*; +import java.util.concurrent.Executor; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.MoreExecutors; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.spark.network.util.MapConfigProvider; +import org.apache.spark.network.util.TransportConf; + +public class CleanupNonShuffleServiceServedFilesSuite { + + // Same-thread Executor used to ensure cleanup happens synchronously in test thread. + private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); + + private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + + private static Set expectedShuffleFilesToKeep = + ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data"); + + private static Set expectedShuffleAndRddFilesToKeep = + ImmutableSet.of("shuffle_782_450_0.index", "shuffle_782_450_0.data", "rdd_12_34"); + + private TransportConf getConf(boolean isFetchRddEnabled) { + return new TransportConf( + "shuffle", + new MapConfigProvider(ImmutableMap.of( + Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, + Boolean.toString(isFetchRddEnabled)))); + } + + @Test + public void cleanupOnRemovedExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnRemovedExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + + @Test + public void cleanupOnRemovedExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnRemovedExecutor(true, getConf(false), expectedShuffleFilesToKeep); + } + + @Test + public void cleanupOnRemovedExecutorWithoutFilesToKeep() throws IOException { + cleanupOnRemovedExecutor(false, getConf(true), Collections.emptySet()); + } + + private void cleanupOnRemovedExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); + + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); + resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); + resolver.executorRemoved("exec0", "app"); + + assertContainedFilenames(dataContext, expectedFilesKept); + } + + @Test + public void cleanupUsesExecutorWithFilesToKeep() throws IOException { + cleanupUsesExecutor(true); + } + + @Test + public void cleanupUsesExecutorWithoutFilesToKeep() throws IOException { + cleanupUsesExecutor(false); + } + + private void cleanupUsesExecutor(boolean withFilesToKeep) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); + + AtomicBoolean cleanupCalled = new AtomicBoolean(false); + + // Executor which only captures whether it's being used, without executing anything. + Executor dummyExecutor = runnable -> cleanupCalled.set(true); + + ExternalShuffleBlockResolver manager = + new ExternalShuffleBlockResolver(getConf(true), null, dummyExecutor); + + manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); + manager.executorRemoved("exec0", "app"); + + assertTrue(cleanupCalled.get()); + assertStillThere(dataContext); + } + + @Test + public void cleanupOnlyRemovedExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnlyRemovedExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + + @Test + public void cleanupOnlyRemovedExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnlyRemovedExecutor(true, getConf(false), expectedShuffleFilesToKeep); + } + + @Test + public void cleanupOnlyRemovedExecutorWithoutFilesToKeep() throws IOException { + cleanupOnlyRemovedExecutor(false, getConf(true) , Collections.emptySet()); + } + + private void cleanupOnlyRemovedExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { + TestShuffleDataContext dataContext0 = initDataContext(withFilesToKeep); + TestShuffleDataContext dataContext1 = initDataContext(withFilesToKeep); + + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); + resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo(SORT_MANAGER)); + resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo(SORT_MANAGER)); + + + resolver.executorRemoved("exec-nonexistent", "app"); + assertStillThere(dataContext0); + assertStillThere(dataContext1); + + resolver.executorRemoved("exec0", "app"); + assertContainedFilenames(dataContext0, expectedFilesKept); + assertStillThere(dataContext1); + + resolver.executorRemoved("exec1", "app"); + assertContainedFilenames(dataContext0, expectedFilesKept); + assertContainedFilenames(dataContext1, expectedFilesKept); + + // Make sure it's not an error to cleanup multiple times + resolver.executorRemoved("exec1", "app"); + assertContainedFilenames(dataContext0, expectedFilesKept); + assertContainedFilenames(dataContext1, expectedFilesKept); + } + + @Test + public void cleanupOnlyRegisteredExecutorWithFilesToKeepFetchRddEnabled() throws IOException { + cleanupOnlyRegisteredExecutor(true, getConf(true), expectedShuffleAndRddFilesToKeep); + } + + @Test + public void cleanupOnlyRegisteredExecutorWithFilesToKeepFetchRddDisabled() throws IOException { + cleanupOnlyRegisteredExecutor(true, getConf(false), expectedShuffleFilesToKeep); + } + + @Test + public void cleanupOnlyRegisteredExecutorWithoutFilesToKeep() throws IOException { + cleanupOnlyRegisteredExecutor(false, getConf(true), Collections.emptySet()); + } + + private void cleanupOnlyRegisteredExecutor( + boolean withFilesToKeep, + TransportConf conf, + Set expectedFilesKept) throws IOException { + TestShuffleDataContext dataContext = initDataContext(withFilesToKeep); + + ExternalShuffleBlockResolver resolver = + new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); + resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); + + resolver.executorRemoved("exec1", "app"); + assertStillThere(dataContext); + + resolver.executorRemoved("exec0", "app"); + assertContainedFilenames(dataContext, expectedFilesKept); + } + + private static void assertStillThere(TestShuffleDataContext dataContext) { + for (String localDir : dataContext.localDirs) { + assertTrue(localDir + " was cleaned up prematurely", new File(localDir).exists()); + } + } + + private static Set collectFilenames(File[] files) throws IOException { + Set result = new HashSet<>(); + for (File file : files) { + if (file.exists()) { + try (Stream walk = Files.walk(file.toPath())) { + result.addAll(walk + .filter(Files::isRegularFile) + .map(x -> x.toFile().getName()) + .collect(Collectors.toSet())); + } + } + } + return result; + } + + private static void assertContainedFilenames( + TestShuffleDataContext dataContext, + Set expectedFilenames) throws IOException { + Set collectedFilenames = new HashSet<>(); + for (String localDir : dataContext.localDirs) { + File[] dirs = new File[] { new File(localDir) }; + collectedFilenames.addAll(collectFilenames(dirs)); + } + assertEquals(expectedFilenames, collectedFilenames); + } + + private static TestShuffleDataContext initDataContext(boolean withFilesToKeep) + throws IOException { + TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); + dataContext.create(); + if (withFilesToKeep) { + createFilesToKeep(dataContext); + } else { + createRemovableTestFiles(dataContext); + } + return dataContext; + } + + private static void createFilesToKeep(TestShuffleDataContext dataContext) throws IOException { + Random rand = new Random(123); + dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { + "ABC".getBytes(StandardCharsets.UTF_8), + "DEF".getBytes(StandardCharsets.UTF_8)}); + dataContext.insertCachedRddData(12, 34, new byte[] { 42 }); + } + + private static void createRemovableTestFiles(TestShuffleDataContext dataContext) + throws IOException { + dataContext.insertSpillData(); + dataContext.insertBroadcastData(); + dataContext.insertTempShuffleData(); + } +} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java index 7846b71d5a8b1..3d30fd02a8cae 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockHandlerSuite.java @@ -27,7 +27,7 @@ import org.mockito.ArgumentCaptor; import static org.junit.Assert.*; -import static org.mockito.Matchers.any; +import static org.mockito.ArgumentMatchers.any; import static org.mockito.Mockito.*; import org.apache.spark.network.buffer.ManagedBuffer; @@ -38,6 +38,7 @@ import org.apache.spark.network.server.RpcHandler; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.RegisterExecutor; import org.apache.spark.network.shuffle.protocol.StreamHandle; @@ -49,6 +50,10 @@ public class ExternalShuffleBlockHandlerSuite { OneForOneStreamManager streamManager; ExternalShuffleBlockResolver blockResolver; RpcHandler handler; + ManagedBuffer[] blockMarkers = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), + new NioManagedBuffer(ByteBuffer.wrap(new byte[7])) + }; @Before public void beforeEach() { @@ -76,19 +81,72 @@ public void testRegisterExecutor() { assertEquals(1, registerExecutorRequestLatencyMillis.getCount()); } - @SuppressWarnings("unchecked") @Test - public void testOpenShuffleBlocks() { - RpcResponseCallback callback = mock(RpcResponseCallback.class); + public void testCompatibilityWithOldVersion() { + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); + + OpenBlocks openBlocks = new OpenBlocks( + "app0", "exec1", new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }); + checkOpenBlocksReceive(openBlocks, blockMarkers); + + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + verifyOpenBlockLatencyMetrics(); + } + + @Test + public void testFetchShuffleBlocks() { + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(blockMarkers[1]); + + FetchShuffleBlocks fetchShuffleBlocks = new FetchShuffleBlocks( + "app0", "exec1", 0, new int[] { 0 }, new int[][] {{ 0, 1 }}); + checkOpenBlocksReceive(fetchShuffleBlocks, blockMarkers); + + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); + verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + verifyOpenBlockLatencyMetrics(); + } - ManagedBuffer block0Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[3])); - ManagedBuffer block1Marker = new NioManagedBuffer(ByteBuffer.wrap(new byte[7])); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 0)).thenReturn(block0Marker); - when(blockResolver.getBlockData("app0", "exec1", 0, 0, 1)).thenReturn(block1Marker); - ByteBuffer openBlocks = new OpenBlocks("app0", "exec1", - new String[] { "shuffle_0_0_0", "shuffle_0_0_1" }) - .toByteBuffer(); - handler.receive(client, openBlocks, callback); + @Test + public void testOpenDiskPersistedRDDBlocks() { + when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)).thenReturn(blockMarkers[0]); + when(blockResolver.getRddBlockData("app0", "exec1", 0, 1)).thenReturn(blockMarkers[1]); + + OpenBlocks openBlocks = new OpenBlocks( + "app0", "exec1", new String[] { "rdd_0_0", "rdd_0_1" }); + checkOpenBlocksReceive(openBlocks, blockMarkers); + + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 1); + verifyOpenBlockLatencyMetrics(); + } + + @Test + public void testOpenDiskPersistedRDDBlocksWithMissingBlock() { + ManagedBuffer[] blockMarkersWithMissingBlock = { + new NioManagedBuffer(ByteBuffer.wrap(new byte[3])), + null + }; + when(blockResolver.getRddBlockData("app0", "exec1", 0, 0)) + .thenReturn(blockMarkersWithMissingBlock[0]); + when(blockResolver.getRddBlockData("app0", "exec1", 0, 1)) + .thenReturn(null); + + OpenBlocks openBlocks = new OpenBlocks( + "app0", "exec1", new String[] { "rdd_0_0", "rdd_0_1" }); + checkOpenBlocksReceive(openBlocks, blockMarkersWithMissingBlock); + + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 0); + verify(blockResolver, times(1)).getRddBlockData("app0", "exec1", 0, 1); + } + + private void checkOpenBlocksReceive(BlockTransferMessage msg, ManagedBuffer[] blockMarkers) { + when(client.getClientId()).thenReturn("app0"); + + RpcResponseCallback callback = mock(RpcResponseCallback.class); + handler.receive(client, msg.toByteBuffer(), callback); ArgumentCaptor response = ArgumentCaptor.forClass(ByteBuffer.class); verify(callback, times(1)).onSuccess(response.capture()); @@ -101,15 +159,15 @@ public void testOpenShuffleBlocks() { @SuppressWarnings("unchecked") ArgumentCaptor> stream = (ArgumentCaptor>) (ArgumentCaptor) ArgumentCaptor.forClass(Iterator.class); - verify(streamManager, times(1)).registerStream(anyString(), stream.capture()); + verify(streamManager, times(1)).registerStream(anyString(), stream.capture(), + any()); Iterator buffers = stream.getValue(); - assertEquals(block0Marker, buffers.next()); - assertEquals(block1Marker, buffers.next()); + assertEquals(blockMarkers[0], buffers.next()); + assertEquals(blockMarkers[1], buffers.next()); assertFalse(buffers.hasNext()); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 0); - verify(blockResolver, times(1)).getBlockData("app0", "exec1", 0, 0, 1); + } - // Verify open block request latency metrics + private void verifyOpenBlockLatencyMetrics() { Timer openBlockRequestLatencyMillis = (Timer) ((ExternalShuffleBlockHandler) handler) .getAllMetrics() .getMetrics() diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java index 459629c5f05fe..09eb699be305a 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleBlockResolverSuite.java @@ -149,7 +149,7 @@ public void testNormalizeAndInternPathname() { private void assertPathsMatch(String p1, String p2, String p3, String expectedPathname) { String normPathname = - ExternalShuffleBlockResolver.createNormalizedInternedPathname(p1, p2, p3); + ExecutorDiskUtils.createNormalizedInternedPathname(p1, p2, p3); assertEquals(expectedPathname, normPathname); File file = new File(normPathname); String returnedPath = file.getPath(); diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java index 526b96b364473..8a2ac56001635 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleIntegrationSuite.java @@ -17,20 +17,25 @@ package org.apache.spark.network.shuffle; +import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; import java.util.Arrays; import java.util.Collections; +import java.util.HashMap; import java.util.HashSet; import java.util.LinkedList; import java.util.List; import java.util.Random; import java.util.Set; +import java.util.concurrent.Future; import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Sets; +import org.apache.spark.network.buffer.FileSegmentManagedBuffer; +import org.apache.spark.network.server.OneForOneStreamManager; import org.junit.After; import org.junit.AfterClass; import org.junit.BeforeClass; @@ -52,12 +57,23 @@ public class ExternalShuffleIntegrationSuite { private static final String APP_ID = "app-id"; private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; + private static final int RDD_ID = 1; + private static final int SPLIT_INDEX_VALID_BLOCK = 0; + private static final int SPLIT_INDEX_MISSING_FILE = 1; + private static final int SPLIT_INDEX_CORRUPT_LENGTH = 2; + private static final int SPLIT_INDEX_VALID_BLOCK_TO_RM = 3; + private static final int SPLIT_INDEX_MISSING_BLOCK_TO_RM = 4; + // Executor 0 is sort-based static TestShuffleDataContext dataContext0; static ExternalShuffleBlockHandler handler; static TransportServer server; static TransportConf conf; + static TransportContext transportContext; + + static byte[] exec0RddBlockValid = new byte[123]; + static byte[] exec0RddBlockToRemove = new byte[124]; static byte[][] exec0Blocks = new byte[][] { new byte[123], @@ -80,14 +96,40 @@ public static void beforeAll() throws IOException { for (byte[] block: exec1Blocks) { rand.nextBytes(block); } + rand.nextBytes(exec0RddBlockValid); + rand.nextBytes(exec0RddBlockToRemove); dataContext0 = new TestShuffleDataContext(2, 5); dataContext0.create(); dataContext0.insertSortShuffleData(0, 0, exec0Blocks); - - conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); - handler = new ExternalShuffleBlockHandler(conf, null); - TransportContext transportContext = new TransportContext(conf, handler); + dataContext0.insertCachedRddData(RDD_ID, SPLIT_INDEX_VALID_BLOCK, exec0RddBlockValid); + dataContext0.insertCachedRddData(RDD_ID, SPLIT_INDEX_VALID_BLOCK_TO_RM, exec0RddBlockToRemove); + + HashMap config = new HashMap<>(); + config.put("spark.shuffle.io.maxRetries", "0"); + config.put(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, "true"); + conf = new TransportConf("shuffle", new MapConfigProvider(config)); + handler = new ExternalShuffleBlockHandler( + new OneForOneStreamManager(), + new ExternalShuffleBlockResolver(conf, null) { + @Override + public ManagedBuffer getRddBlockData(String appId, String execId, int rddId, int splitIdx) { + ManagedBuffer res; + if (rddId == RDD_ID) { + switch (splitIdx) { + case SPLIT_INDEX_CORRUPT_LENGTH: + res = new FileSegmentManagedBuffer(conf, new File("missing.file"), 0, 12); + break; + default: + res = super.getRddBlockData(appId, execId, rddId, splitIdx); + } + } else { + res = super.getRddBlockData(appId, execId, rddId, splitIdx); + } + return res; + } + }); + transportContext = new TransportContext(conf, handler); server = transportContext.createServer(); } @@ -95,6 +137,7 @@ public static void beforeAll() throws IOException { public static void afterAll() { dataContext0.cleanup(); server.close(); + transportContext.close(); } @After @@ -197,9 +240,55 @@ public void testRegisterInvalidExecutor() throws Exception { @Test public void testFetchWrongBlockId() throws Exception { registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); - FetchResult execFetch = fetchBlocks("exec-1", new String[] { "rdd_1_0_0" }); + FetchResult execFetch = fetchBlocks("exec-1", new String[] { "broadcast_1" }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet("broadcast_1"), execFetch.failedBlocks); + } + + @Test + public void testFetchValidRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String validBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_VALID_BLOCK; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { validBlockId }); + assertTrue(execFetch.failedBlocks.isEmpty()); + assertEquals(Sets.newHashSet(validBlockId), execFetch.successBlocks); + assertBuffersEqual(new NioManagedBuffer(ByteBuffer.wrap(exec0RddBlockValid)), + execFetch.buffers.get(0)); + } + + @Test + public void testFetchDeletedRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String missingBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_MISSING_FILE; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { missingBlockId }); + assertTrue(execFetch.successBlocks.isEmpty()); + assertEquals(Sets.newHashSet(missingBlockId), execFetch.failedBlocks); + } + + @Test + public void testRemoveRddBlocks() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String validBlockIdToRemove = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_VALID_BLOCK_TO_RM; + String missingBlockIdToRemove = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_MISSING_BLOCK_TO_RM; + + try (ExternalShuffleClient client = new ExternalShuffleClient(conf, null, false, 5000)) { + client.init(APP_ID); + Future numRemovedBlocks = client.removeBlocks( + TestUtils.getLocalHost(), + server.getPort(), + "exec-1", + new String[] { validBlockIdToRemove, missingBlockIdToRemove }); + assertEquals(1, numRemovedBlocks.get().intValue()); + } + } + + @Test + public void testFetchCorruptRddBlock() throws Exception { + registerExecutor("exec-1", dataContext0.createExecutorInfo(SORT_MANAGER)); + String corruptBlockId = "rdd_" + RDD_ID +"_" + SPLIT_INDEX_CORRUPT_LENGTH; + FetchResult execFetch = fetchBlocks("exec-1", new String[] { corruptBlockId }); assertTrue(execFetch.successBlocks.isEmpty()); - assertEquals(Sets.newHashSet("rdd_1_0_0"), execFetch.failedBlocks); + assertEquals(Sets.newHashSet(corruptBlockId), execFetch.failedBlocks); } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java index 82caf392b821b..67f79021daf5e 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/ExternalShuffleSecuritySuite.java @@ -41,14 +41,14 @@ public class ExternalShuffleSecuritySuite { TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); TransportServer server; + TransportContext transportContext; @Before public void beforeEach() throws IOException { - TransportContext context = - new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); + transportContext = new TransportContext(conf, new ExternalShuffleBlockHandler(conf, null)); TransportServerBootstrap bootstrap = new SaslServerBootstrap(conf, new TestSecretKeyHolder("my-app-id", "secret")); - this.server = context.createServer(Arrays.asList(bootstrap)); + this.server = transportContext.createServer(Arrays.asList(bootstrap)); } @After @@ -57,6 +57,10 @@ public void afterEach() { server.close(); server = null; } + if (transportContext != null) { + transportContext.close(); + transportContext = null; + } } @Test diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java deleted file mode 100644 index d22f3ace4103b..0000000000000 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/NonShuffleFilesCleanupSuite.java +++ /dev/null @@ -1,221 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.network.shuffle; - -import java.io.File; -import java.io.FilenameFilter; -import java.io.IOException; -import java.nio.charset.StandardCharsets; -import java.util.Random; -import java.util.concurrent.Executor; -import java.util.concurrent.atomic.AtomicBoolean; - -import com.google.common.util.concurrent.MoreExecutors; -import org.junit.Test; -import static org.junit.Assert.assertTrue; - -import org.apache.spark.network.util.MapConfigProvider; -import org.apache.spark.network.util.TransportConf; - -public class NonShuffleFilesCleanupSuite { - - // Same-thread Executor used to ensure cleanup happens synchronously in test thread. - private Executor sameThreadExecutor = MoreExecutors.sameThreadExecutor(); - private TransportConf conf = new TransportConf("shuffle", MapConfigProvider.EMPTY); - private static final String SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager"; - - @Test - public void cleanupOnRemovedExecutorWithShuffleFiles() throws IOException { - cleanupOnRemovedExecutor(true); - } - - @Test - public void cleanupOnRemovedExecutorWithoutShuffleFiles() throws IOException { - cleanupOnRemovedExecutor(false); - } - - private void cleanupOnRemovedExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); - - ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - resolver.executorRemoved("exec0", "app"); - - assertCleanedUp(dataContext); - } - - @Test - public void cleanupUsesExecutorWithShuffleFiles() throws IOException { - cleanupUsesExecutor(true); - } - - @Test - public void cleanupUsesExecutorWithoutShuffleFiles() throws IOException { - cleanupUsesExecutor(false); - } - - private void cleanupUsesExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); - - AtomicBoolean cleanupCalled = new AtomicBoolean(false); - - // Executor which does nothing to ensure we're actually using it. - Executor noThreadExecutor = runnable -> cleanupCalled.set(true); - - ExternalShuffleBlockResolver manager = - new ExternalShuffleBlockResolver(conf, null, noThreadExecutor); - - manager.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - manager.executorRemoved("exec0", "app"); - - assertTrue(cleanupCalled.get()); - assertStillThere(dataContext); - } - - @Test - public void cleanupOnlyRemovedExecutorWithShuffleFiles() throws IOException { - cleanupOnlyRemovedExecutor(true); - } - - @Test - public void cleanupOnlyRemovedExecutorWithoutShuffleFiles() throws IOException { - cleanupOnlyRemovedExecutor(false); - } - - private void cleanupOnlyRemovedExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext0 = initDataContext(withShuffleFiles); - TestShuffleDataContext dataContext1 = initDataContext(withShuffleFiles); - - ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app", "exec0", dataContext0.createExecutorInfo(SORT_MANAGER)); - resolver.registerExecutor("app", "exec1", dataContext1.createExecutorInfo(SORT_MANAGER)); - - - resolver.executorRemoved("exec-nonexistent", "app"); - assertStillThere(dataContext0); - assertStillThere(dataContext1); - - resolver.executorRemoved("exec0", "app"); - assertCleanedUp(dataContext0); - assertStillThere(dataContext1); - - resolver.executorRemoved("exec1", "app"); - assertCleanedUp(dataContext0); - assertCleanedUp(dataContext1); - - // Make sure it's not an error to cleanup multiple times - resolver.executorRemoved("exec1", "app"); - assertCleanedUp(dataContext0); - assertCleanedUp(dataContext1); - } - - @Test - public void cleanupOnlyRegisteredExecutorWithShuffleFiles() throws IOException { - cleanupOnlyRegisteredExecutor(true); - } - - @Test - public void cleanupOnlyRegisteredExecutorWithoutShuffleFiles() throws IOException { - cleanupOnlyRegisteredExecutor(false); - } - - private void cleanupOnlyRegisteredExecutor(boolean withShuffleFiles) throws IOException { - TestShuffleDataContext dataContext = initDataContext(withShuffleFiles); - - ExternalShuffleBlockResolver resolver = - new ExternalShuffleBlockResolver(conf, null, sameThreadExecutor); - resolver.registerExecutor("app", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)); - - resolver.executorRemoved("exec1", "app"); - assertStillThere(dataContext); - - resolver.executorRemoved("exec0", "app"); - assertCleanedUp(dataContext); - } - - private static void assertStillThere(TestShuffleDataContext dataContext) { - for (String localDir : dataContext.localDirs) { - assertTrue(localDir + " was cleaned up prematurely", new File(localDir).exists()); - } - } - - private static FilenameFilter filter = new FilenameFilter() { - @Override - public boolean accept(File dir, String name) { - // Don't delete shuffle data or shuffle index files. - return !name.endsWith(".index") && !name.endsWith(".data"); - } - }; - - private static boolean assertOnlyShuffleDataInDir(File[] dirs) { - for (File dir : dirs) { - assertTrue(dir.getName() + " wasn't cleaned up", !dir.exists() || - dir.listFiles(filter).length == 0 || assertOnlyShuffleDataInDir(dir.listFiles())); - } - return true; - } - - private static void assertCleanedUp(TestShuffleDataContext dataContext) { - for (String localDir : dataContext.localDirs) { - File[] dirs = new File[] {new File(localDir)}; - assertOnlyShuffleDataInDir(dirs); - } - } - - private static TestShuffleDataContext initDataContext(boolean withShuffleFiles) - throws IOException { - if (withShuffleFiles) { - return initDataContextWithShuffleFiles(); - } else { - return initDataContextWithoutShuffleFiles(); - } - } - - private static TestShuffleDataContext initDataContextWithShuffleFiles() throws IOException { - TestShuffleDataContext dataContext = createDataContext(); - createShuffleFiles(dataContext); - createNonShuffleFiles(dataContext); - return dataContext; - } - - private static TestShuffleDataContext initDataContextWithoutShuffleFiles() throws IOException { - TestShuffleDataContext dataContext = createDataContext(); - createNonShuffleFiles(dataContext); - return dataContext; - } - - private static TestShuffleDataContext createDataContext() { - TestShuffleDataContext dataContext = new TestShuffleDataContext(10, 5); - dataContext.create(); - return dataContext; - } - - private static void createShuffleFiles(TestShuffleDataContext dataContext) throws IOException { - Random rand = new Random(123); - dataContext.insertSortShuffleData(rand.nextInt(1000), rand.nextInt(1000), new byte[][] { - "ABC".getBytes(StandardCharsets.UTF_8), - "DEF".getBytes(StandardCharsets.UTF_8)}); - } - - private static void createNonShuffleFiles(TestShuffleDataContext dataContext) throws IOException { - // Create spill file(s) - dataContext.insertSpillData(); - } -} diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java index dc947a619bf02..66633cc7a3595 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/OneForOneBlockFetcherSuite.java @@ -18,6 +18,7 @@ package org.apache.spark.network.shuffle; import java.nio.ByteBuffer; +import java.util.HashMap; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.concurrent.atomic.AtomicInteger; @@ -28,10 +29,10 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; -import static org.mockito.Matchers.anyLong; -import static org.mockito.Matchers.eq; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; +import static org.mockito.ArgumentMatchers.anyLong; +import static org.mockito.ArgumentMatchers.eq; import static org.mockito.Mockito.doAnswer; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.times; @@ -44,6 +45,7 @@ import org.apache.spark.network.client.RpcResponseCallback; import org.apache.spark.network.client.TransportClient; import org.apache.spark.network.shuffle.protocol.BlockTransferMessage; +import org.apache.spark.network.shuffle.protocol.FetchShuffleBlocks; import org.apache.spark.network.shuffle.protocol.OpenBlocks; import org.apache.spark.network.shuffle.protocol.StreamHandle; import org.apache.spark.network.util.MapConfigProvider; @@ -57,20 +59,69 @@ public class OneForOneBlockFetcherSuite { public void testFetchOne() { LinkedHashMap blocks = Maps.newLinkedHashMap(); blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0 }}), + conf); + + verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); + } + + @Test + public void testUseOldProtocol() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[0]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - BlockFetchingListener listener = fetchBlocks(blocks); + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new OpenBlocks("app-id", "exec-id", blockIds), + new TransportConf("shuffle", new MapConfigProvider( + new HashMap() {{ + put("spark.shuffle.useOldFetchProtocol", "true"); + }} + ))); verify(listener).onBlockFetchSuccess("shuffle_0_0_0", blocks.get("shuffle_0_0_0")); } + @Test + public void testFetchThreeShuffleBlocks() { + LinkedHashMap blocks = Maps.newLinkedHashMap(); + blocks.put("shuffle_0_0_0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); + blocks.put("shuffle_0_0_1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); + blocks.put("shuffle_0_0_2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); + + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new FetchShuffleBlocks("app-id", "exec-id", 0, new int[] { 0 }, new int[][] {{ 0, 1, 2 }}), + conf); + + for (int i = 0; i < 3; i ++) { + verify(listener, times(1)).onBlockFetchSuccess( + "shuffle_0_0_" + i, blocks.get("shuffle_0_0_" + i)); + } + } + @Test public void testFetchThree() { LinkedHashMap blocks = Maps.newLinkedHashMap(); blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); blocks.put("b1", new NioManagedBuffer(ByteBuffer.wrap(new byte[23]))); blocks.put("b2", new NettyManagedBuffer(Unpooled.wrappedBuffer(new byte[23]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - BlockFetchingListener listener = fetchBlocks(blocks); + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new OpenBlocks("app-id", "exec-id", blockIds), + conf); for (int i = 0; i < 3; i ++) { verify(listener, times(1)).onBlockFetchSuccess("b" + i, blocks.get("b" + i)); @@ -83,8 +134,13 @@ public void testFailure() { blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); blocks.put("b1", null); blocks.put("b2", null); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - BlockFetchingListener listener = fetchBlocks(blocks); + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new OpenBlocks("app-id", "exec-id", blockIds), + conf); // Each failure will cause a failure to be invoked in all remaining block fetches. verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); @@ -98,8 +154,13 @@ public void testFailureAndSuccess() { blocks.put("b0", new NioManagedBuffer(ByteBuffer.wrap(new byte[12]))); blocks.put("b1", null); blocks.put("b2", new NioManagedBuffer(ByteBuffer.wrap(new byte[21]))); + String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); - BlockFetchingListener listener = fetchBlocks(blocks); + BlockFetchingListener listener = fetchBlocks( + blocks, + blockIds, + new OpenBlocks("app-id", "exec-id", blockIds), + conf); // We may call both success and failure for the same block. verify(listener, times(1)).onBlockFetchSuccess("b0", blocks.get("b0")); @@ -111,7 +172,11 @@ public void testFailureAndSuccess() { @Test public void testEmptyBlockFetch() { try { - fetchBlocks(Maps.newLinkedHashMap()); + fetchBlocks( + Maps.newLinkedHashMap(), + new String[] {}, + new OpenBlocks("app-id", "exec-id", new String[] {}), + conf); fail(); } catch (IllegalArgumentException e) { assertEquals("Zero-sized blockIds array", e.getMessage()); @@ -126,12 +191,15 @@ public void testEmptyBlockFetch() { * * If a block's buffer is "null", an exception will be thrown instead. */ - private static BlockFetchingListener fetchBlocks(LinkedHashMap blocks) { + private static BlockFetchingListener fetchBlocks( + LinkedHashMap blocks, + String[] blockIds, + BlockTransferMessage expectMessage, + TransportConf transportConf) { TransportClient client = mock(TransportClient.class); BlockFetchingListener listener = mock(BlockFetchingListener.class); - String[] blockIds = blocks.keySet().toArray(new String[blocks.size()]); OneForOneBlockFetcher fetcher = - new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener, conf); + new OneForOneBlockFetcher(client, "app-id", "exec-id", blockIds, listener, transportConf); // Respond to the "OpenBlocks" message with an appropriate ShuffleStreamHandle with streamId 123 doAnswer(invocationOnMock -> { @@ -139,7 +207,7 @@ private static BlockFetchingListener fetchBlocks(LinkedHashMap> inte } assertNotNull(stub); - stub.when(fetchStarter).createAndStart(any(), anyObject()); + stub.when(fetchStarter).createAndStart(any(), any()); String[] blockIdArray = blockIds.toArray(new String[blockIds.size()]); new RetryingBlockFetcher(conf, fetchStarter, blockIdArray, listener).start(); } diff --git a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java index 6989c3baf2e28..457805feeac45 100644 --- a/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java +++ b/common/network-shuffle/src/test/java/org/apache/spark/network/shuffle/TestShuffleDataContext.java @@ -22,7 +22,6 @@ import java.io.FileOutputStream; import java.io.IOException; import java.io.OutputStream; -import java.util.UUID; import com.google.common.io.Closeables; import com.google.common.io.Files; @@ -77,9 +76,9 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr try { dataStream = new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); + ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId + ".data")); indexStream = new DataOutputStream(new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); + ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId + ".index"))); long offset = 0; indexStream.writeLong(offset); @@ -97,13 +96,37 @@ public void insertSortShuffleData(int shuffleId, int mapId, byte[][] blocks) thr /** Creates spill file(s) within the local dirs. */ public void insertSpillData() throws IOException { - String filename = "temp_local_" + UUID.randomUUID(); - OutputStream dataStream = null; + String filename = "temp_local_uuid"; + insertFile(filename); + } + + public void insertBroadcastData() throws IOException { + String filename = "broadcast_12_uuid"; + insertFile(filename); + } + + public void insertTempShuffleData() throws IOException { + String filename = "temp_shuffle_uuid"; + insertFile(filename); + } + public void insertCachedRddData(int rddId, int splitId, byte[] block) throws IOException { + String blockId = "rdd_" + rddId + "_" + splitId; + insertFile(blockId, block); + } + + private void insertFile(String filename) throws IOException { + insertFile(filename, new byte[] { 42 }); + } + + private void insertFile(String filename, byte[] block) throws IOException { + OutputStream dataStream = null; + File file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename); + assert(!file.exists()) : "this test file has been already generated"; try { dataStream = new FileOutputStream( - ExternalShuffleBlockResolver.getFile(localDirs, subDirsPerLocalDir, filename)); - dataStream.write(42); + ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, filename)); + dataStream.write(block); } finally { Closeables.close(dataStream, false); } diff --git a/common/network-yarn/pom.xml b/common/network-yarn/pom.xml index a1cf761d12d8b..76a402bb2bd31 100644 --- a/common/network-yarn/pom.xml +++ b/common/network-yarn/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-network-yarn_2.11 + spark-network-yarn_2.12 jar Spark Project YARN Shuffle Service http://spark.apache.org/ @@ -35,7 +35,7 @@ provided ${project.build.directory}/scala-${scala.binary.version}/spark-${project.version}-yarn-shuffle.jar - org/spark_project/ + org/sparkproject/ @@ -128,12 +128,49 @@ - - + + + org.codehaus.mojo + build-helper-maven-plugin + + + regex-property + + regex-property + + + spark.shade.native.packageName + ${spark.shade.packageName} + \. + _ + true + + + + org.apache.maven.plugins maven-antrun-plugin + + unpack + package + + + + + + + + + + + run + + + verify 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 72ae1a1295236..25592e9873ffc 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 @@ -113,6 +113,8 @@ public class YarnShuffleService extends AuxiliaryService { // The actual server that serves shuffle files private TransportServer shuffleServer = null; + private TransportContext transportContext = null; + private Configuration _conf = null; // The recovery path used to shuffle service recovery @@ -170,15 +172,6 @@ protected void serviceInit(Configuration conf) throws Exception { TransportConf transportConf = new TransportConf("shuffle", new HadoopConfigProvider(conf)); blockHandler = new ExternalShuffleBlockHandler(transportConf, registeredExecutorFile); - // register metrics on the block handler into the Node Manager's metrics system. - YarnShuffleServiceMetrics serviceMetrics = - new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); - - MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); - metricsSystem.register( - "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); - logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); - // If authentication is enabled, set up the shuffle server to use a // special RPC handler that filters out unauthenticated fetch requests List bootstraps = Lists.newArrayList(); @@ -193,12 +186,24 @@ protected void serviceInit(Configuration conf) throws Exception { int port = conf.getInt( SPARK_SHUFFLE_SERVICE_PORT_KEY, DEFAULT_SPARK_SHUFFLE_SERVICE_PORT); - TransportContext transportContext = new TransportContext(transportConf, blockHandler); + transportContext = new TransportContext(transportConf, blockHandler); shuffleServer = transportContext.createServer(port, bootstraps); // the port should normally be fixed, but for tests its useful to find an open port port = shuffleServer.getPort(); boundPort = port; String authEnabledString = authEnabled ? "enabled" : "not enabled"; + + // register metrics on the block handler into the Node Manager's metrics system. + blockHandler.getAllMetrics().getMetrics().put("numRegisteredConnections", + shuffleServer.getRegisteredConnections()); + YarnShuffleServiceMetrics serviceMetrics = + new YarnShuffleServiceMetrics(blockHandler.getAllMetrics()); + + MetricsSystemImpl metricsSystem = (MetricsSystemImpl) DefaultMetricsSystem.instance(); + metricsSystem.register( + "sparkShuffleService", "Metrics on the Spark Shuffle Service", serviceMetrics); + logger.info("Registered metrics with Hadoop's DefaultMetricsSystem"); + logger.info("Started YARN shuffle service for Spark on port {}. " + "Authentication is {}. Registered executor file is {}", port, authEnabledString, registeredExecutorFile); @@ -315,6 +320,9 @@ protected void serviceStop() { if (shuffleServer != null) { shuffleServer.close(); } + if (transportContext != null) { + transportContext.close(); + } if (blockHandler != null) { blockHandler.close(); } diff --git a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java index 3e4d479b862b3..0064c1c0295c8 100644 --- a/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java +++ b/common/network-yarn/src/main/java/org/apache/spark/network/yarn/YarnShuffleServiceMetrics.java @@ -96,24 +96,36 @@ public static void collectMetric( } else if (metric instanceof Gauge) { final Object gaugeValue = ((Gauge) metric).getValue(); if (gaugeValue instanceof Integer) { - metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Integer) gaugeValue); + metricsRecordBuilder.addGauge( + getShuffleServiceMetricsInfoForGauge(name), (Integer) gaugeValue); } else if (gaugeValue instanceof Long) { - metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Long) gaugeValue); + metricsRecordBuilder.addGauge( + getShuffleServiceMetricsInfoForGauge(name), (Long) gaugeValue); } else if (gaugeValue instanceof Float) { - metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Float) gaugeValue); + metricsRecordBuilder.addGauge( + getShuffleServiceMetricsInfoForGauge(name), (Float) gaugeValue); } else if (gaugeValue instanceof Double) { - metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfo(name), (Double) gaugeValue); + metricsRecordBuilder.addGauge( + getShuffleServiceMetricsInfoForGauge(name), (Double) gaugeValue); } else { throw new IllegalStateException( "Not supported class type of metric[" + name + "] for value " + gaugeValue); } + } else if (metric instanceof Counter) { + Counter c = (Counter) metric; + long counterValue = c.getCount(); + metricsRecordBuilder.addGauge(getShuffleServiceMetricsInfoForCounter(name), counterValue); } } - private static MetricsInfo getShuffleServiceMetricsInfo(String name) { + private static MetricsInfo getShuffleServiceMetricsInfoForGauge(String name) { return new ShuffleServiceMetricsInfo(name, "Value of gauge " + name); } + private static ShuffleServiceMetricsInfo getShuffleServiceMetricsInfoForCounter(String name) { + return new ShuffleServiceMetricsInfo(name, "Value of counter " + name); + } + private static class ShuffleServiceMetricsInfo implements MetricsInfo { private final String name; diff --git a/common/sketch/pom.xml b/common/sketch/pom.xml index adbbcb1cb3040..3c3c0d2d96a1c 100644 --- a/common/sketch/pom.xml +++ b/common/sketch/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-sketch_2.11 + spark-sketch_2.12 jar Spark Project Sketch http://spark.apache.org/ diff --git a/common/tags/pom.xml b/common/tags/pom.xml index f6627beabe84b..883b73a69c9de 100644 --- a/common/tags/pom.xml +++ b/common/tags/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-tags_2.11 + spark-tags_2.12 jar Spark Project Tags http://spark.apache.org/ diff --git a/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java index 0ecef6db0e039..890f2faca28b0 100644 --- a/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java +++ b/common/tags/src/main/java/org/apache/spark/annotation/DeveloperApi.java @@ -29,6 +29,7 @@ * of the known issue that Scaladoc displays only either the annotation or the comment, whichever * comes first. */ +@Documented @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java b/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java new file mode 100644 index 0000000000000..87e8948f204ff --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Evolving.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. + * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Evolving {} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java index ff8120291455f..96875920cd9c3 100644 --- a/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java +++ b/common/tags/src/main/java/org/apache/spark/annotation/Experimental.java @@ -30,6 +30,7 @@ * of the known issue that Scaladoc displays only either the annotation or the comment, whichever * comes first. */ +@Documented @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java b/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java deleted file mode 100644 index 323098f69c6e1..0000000000000 --- a/common/tags/src/main/java/org/apache/spark/annotation/InterfaceStability.java +++ /dev/null @@ -1,49 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.annotation; - -import java.lang.annotation.Documented; - -/** - * Annotation to inform users of how much to rely on a particular package, - * class or method not changing over time. - */ -public class InterfaceStability { - - /** - * Stable APIs that retain source and binary compatibility within a major release. - * These interfaces can change from one major release to another major release - * (e.g. from 1.0 to 2.0). - */ - @Documented - public @interface Stable {}; - - /** - * APIs that are meant to evolve towards becoming stable APIs, but are not stable APIs yet. - * Evolving interfaces can change from one feature release to another release (i.e. 2.1 to 2.2). - */ - @Documented - public @interface Evolving {}; - - /** - * Unstable APIs, with no guarantee on stability. - * Classes that are unannotated are considered Unstable. - */ - @Documented - public @interface Unstable {}; -} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Private.java b/common/tags/src/main/java/org/apache/spark/annotation/Private.java index 9082fcf0c84bc..a460d608ae16b 100644 --- a/common/tags/src/main/java/org/apache/spark/annotation/Private.java +++ b/common/tags/src/main/java/org/apache/spark/annotation/Private.java @@ -17,10 +17,7 @@ package org.apache.spark.annotation; -import java.lang.annotation.ElementType; -import java.lang.annotation.Retention; -import java.lang.annotation.RetentionPolicy; -import java.lang.annotation.Target; +import java.lang.annotation.*; /** * A class that is considered private to the internals of Spark -- there is a high-likelihood @@ -35,6 +32,7 @@ * of the known issue that Scaladoc displays only either the annotation or the comment, whichever * comes first. */ +@Documented @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Stable.java b/common/tags/src/main/java/org/apache/spark/annotation/Stable.java new file mode 100644 index 0000000000000..b198bfbe91e10 --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Stable.java @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotation; + +import java.lang.annotation.*; + +/** + * Stable APIs that retain source and binary compatibility within a major release. + * These interfaces can change from one major release to another major release + * (e.g. from 1.0 to 2.0). + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Stable {} diff --git a/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java b/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java new file mode 100644 index 0000000000000..88ee72125b23f --- /dev/null +++ b/common/tags/src/main/java/org/apache/spark/annotation/Unstable.java @@ -0,0 +1,30 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * Unstable APIs, with no guarantee on stability. + * Classes that are unannotated are considered Unstable. + */ +@Documented +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Unstable {} diff --git a/common/unsafe/pom.xml b/common/unsafe/pom.xml index 62c493a5e1ed8..93a4f67fd23f2 100644 --- a/common/unsafe/pom.xml +++ b/common/unsafe/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-unsafe_2.11 + spark-unsafe_2.12 jar Spark Project Unsafe http://spark.apache.org/ @@ -89,6 +89,11 @@ commons-lang3 test + + org.apache.commons + commons-text + test + target/scala-${scala.binary.version}/classes diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java index aca6fca00c48b..dc8d6e3a2f0ba 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/Platform.java @@ -19,10 +19,10 @@ import java.lang.reflect.Constructor; import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.nio.ByteBuffer; -import sun.misc.Cleaner; import sun.misc.Unsafe; public final class Platform { @@ -44,27 +44,61 @@ public final class Platform { public static final int DOUBLE_ARRAY_OFFSET; private static final boolean unaligned; + + // Access fields and constructors once and store them, for performance: + + private static final Constructor DBB_CONSTRUCTOR; + private static final Field DBB_CLEANER_FIELD; static { - boolean _unaligned; - String arch = System.getProperty("os.arch", ""); - if (arch.equals("ppc64le") || arch.equals("ppc64")) { - // Since java.nio.Bits.unaligned() doesn't return true on ppc (See JDK-8165231), but - // ppc64 and ppc64le support it - _unaligned = true; + try { + Class cls = Class.forName("java.nio.DirectByteBuffer"); + Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); + constructor.setAccessible(true); + Field cleanerField = cls.getDeclaredField("cleaner"); + cleanerField.setAccessible(true); + DBB_CONSTRUCTOR = constructor; + DBB_CLEANER_FIELD = cleanerField; + } catch (ClassNotFoundException | NoSuchMethodException | NoSuchFieldException e) { + throw new IllegalStateException(e); + } + } + + // Split java.version on non-digit chars: + private static final int majorVersion = + Integer.parseInt(System.getProperty("java.version").split("\\D+")[0]); + + private static final Method CLEANER_CREATE_METHOD; + static { + // The implementation of Cleaner changed from JDK 8 to 9 + String cleanerClassName; + if (majorVersion < 9) { + cleanerClassName = "sun.misc.Cleaner"; } else { + cleanerClassName = "jdk.internal.ref.Cleaner"; + } + try { + Class cleanerClass = Class.forName(cleanerClassName); + Method createMethod = cleanerClass.getMethod("create", Object.class, Runnable.class); + // Accessing jdk.internal.ref.Cleaner should actually fail by default in JDK 9+, + // unfortunately, unless the user has allowed access with something like + // --add-opens java.base/java.lang=ALL-UNNAMED If not, we can't really use the Cleaner + // hack below. It doesn't break, just means the user might run into the default JVM limit + // on off-heap memory and increase it or set the flag above. This tests whether it's + // available: try { - Class bitsClass = - Class.forName("java.nio.Bits", false, ClassLoader.getSystemClassLoader()); - Method unalignedMethod = bitsClass.getDeclaredMethod("unaligned"); - unalignedMethod.setAccessible(true); - _unaligned = Boolean.TRUE.equals(unalignedMethod.invoke(null)); - } catch (Throwable t) { - // We at least know x86 and x64 support unaligned access. - //noinspection DynamicRegexReplaceableByCompiledPattern - _unaligned = arch.matches("^(i[3-6]86|x86(_64)?|x64|amd64|aarch64)$"); + createMethod.invoke(null, null, null); + } catch (IllegalAccessException e) { + // Don't throw an exception, but can't log here? + createMethod = null; + } catch (InvocationTargetException ite) { + // shouldn't happen; report it + throw new IllegalStateException(ite); } + CLEANER_CREATE_METHOD = createMethod; + } catch (ClassNotFoundException | NoSuchMethodException e) { + throw new IllegalStateException(e); } - unaligned = _unaligned; + } /** @@ -155,22 +189,33 @@ public static long reallocateMemory(long address, long oldSize, long newSize) { } /** - * Uses internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's - * MaxDirectMemorySize limit (the default limit is too low and we do not want to require users - * to increase it). + * Allocate a DirectByteBuffer, potentially bypassing the JVM's MaxDirectMemorySize limit. */ - @SuppressWarnings("unchecked") public static ByteBuffer allocateDirectBuffer(int size) { try { - Class cls = Class.forName("java.nio.DirectByteBuffer"); - Constructor constructor = cls.getDeclaredConstructor(Long.TYPE, Integer.TYPE); - constructor.setAccessible(true); - Field cleanerField = cls.getDeclaredField("cleaner"); - cleanerField.setAccessible(true); + if (CLEANER_CREATE_METHOD == null) { + // Can't set a Cleaner (see comments on field), so need to allocate via normal Java APIs + try { + return ByteBuffer.allocateDirect(size); + } catch (OutOfMemoryError oome) { + // checkstyle.off: RegexpSinglelineJava + throw new OutOfMemoryError("Failed to allocate direct buffer (" + oome.getMessage() + + "); try increasing -XX:MaxDirectMemorySize=... to, for example, your heap size"); + // checkstyle.on: RegexpSinglelineJava + } + } + // Otherwise, use internal JDK APIs to allocate a DirectByteBuffer while ignoring the JVM's + // MaxDirectMemorySize limit (the default limit is too low and we do not want to + // require users to increase it). long memory = allocateMemory(size); - ByteBuffer buffer = (ByteBuffer) constructor.newInstance(memory, size); - Cleaner cleaner = Cleaner.create(buffer, () -> freeMemory(memory)); - cleanerField.set(buffer, cleaner); + ByteBuffer buffer = (ByteBuffer) DBB_CONSTRUCTOR.newInstance(memory, size); + try { + DBB_CLEANER_FIELD.set(buffer, + CLEANER_CREATE_METHOD.invoke(null, buffer, (Runnable) () -> freeMemory(memory))); + } catch (IllegalAccessException | InvocationTargetException e) { + freeMemory(memory); + throw new IllegalStateException(e); + } return buffer; } catch (Exception e) { throwException(e); @@ -254,4 +299,36 @@ public static void throwException(Throwable t) { DOUBLE_ARRAY_OFFSET = 0; } } + + // This requires `majorVersion` and `_UNSAFE`. + static { + boolean _unaligned; + String arch = System.getProperty("os.arch", ""); + if (arch.equals("ppc64le") || arch.equals("ppc64") || arch.equals("s390x")) { + // Since java.nio.Bits.unaligned() doesn't return true on ppc (See JDK-8165231), but + // ppc64 and ppc64le support it + _unaligned = true; + } else { + try { + Class bitsClass = + Class.forName("java.nio.Bits", false, ClassLoader.getSystemClassLoader()); + if (_UNSAFE != null && majorVersion >= 9) { + // Java 9/10 and 11/12 have different field names. + Field unalignedField = + bitsClass.getDeclaredField(majorVersion >= 11 ? "UNALIGNED" : "unaligned"); + _unaligned = _UNSAFE.getBoolean( + _UNSAFE.staticFieldBase(unalignedField), _UNSAFE.staticFieldOffset(unalignedField)); + } else { + Method unalignedMethod = bitsClass.getDeclaredMethod("unaligned"); + unalignedMethod.setAccessible(true); + _unaligned = Boolean.TRUE.equals(unalignedMethod.invoke(null)); + } + } catch (Throwable t) { + // We at least know x86 and x64 support unaligned access. + //noinspection DynamicRegexReplaceableByCompiledPattern + _unaligned = arch.matches("^(i[3-6]86|x86(_64)?|x64|amd64|aarch64)$"); + } + } + unaligned = _unaligned; + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java similarity index 80% rename from sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java rename to common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java index f0f66bae245fd..481ea89090b2a 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UTF8StringBuilder.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/UTF8StringBuilder.java @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst.expressions.codegen; +package org.apache.spark.unsafe; -import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.types.UTF8String; @@ -34,7 +33,18 @@ public class UTF8StringBuilder { public UTF8StringBuilder() { // Since initial buffer size is 16 in `StringBuilder`, we set the same size here - this.buffer = new byte[16]; + this(16); + } + + public UTF8StringBuilder(int initialSize) { + if (initialSize < 0) { + throw new IllegalArgumentException("Size must be non-negative"); + } + if (initialSize > ARRAY_MAX) { + throw new IllegalArgumentException( + "Size " + initialSize + " exceeded maximum size of " + ARRAY_MAX); + } + this.buffer = new byte[initialSize]; } // Grows the buffer by at least `neededSize` @@ -72,6 +82,17 @@ public void append(String value) { append(UTF8String.fromString(value)); } + public void appendBytes(Object base, long offset, int length) { + grow(length); + Platform.copyMemory( + base, + offset, + buffer, + cursor, + length); + cursor += length; + } + public UTF8String build() { return UTF8String.fromBytes(buffer, 0, totalSize()); } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java index be62e40412f83..546e8780a6606 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/UnsafeAlignedOffset.java @@ -39,7 +39,9 @@ public static int getSize(Object object, long offset) { case 8: return (int)Platform.getLong(object, offset); default: + // checkstyle.off: RegexpSinglelineJava throw new AssertionError("Illegal UAO_SIZE"); + // checkstyle.on: RegexpSinglelineJava } } @@ -52,7 +54,9 @@ public static void putSize(Object object, long offset, int value) { Platform.putLong(object, offset, value); break; default: + // checkstyle.off: RegexpSinglelineJava throw new AssertionError("Illegal UAO_SIZE"); + // checkstyle.on: RegexpSinglelineJava } } } diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java index 2cd39bd60c2ac..305cc1c5d1115 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/array/LongArray.java @@ -23,7 +23,7 @@ /** * An array of long values. Compared with native JVM arrays, this: *
    - *
  • supports using both in-heap and off-heap memory
  • + *
  • supports using both on-heap and off-heap memory
  • *
  • has no bound checking, and thus can crash the JVM process when assert is turned off
  • *
*/ diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java index 74ebc87dc978c..897b8a2b7ec50 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/memory/MemoryLocation.java @@ -21,7 +21,7 @@ /** * A memory location. Tracked either by a memory address (with off-heap allocation), - * or by an offset from a JVM object (in-heap allocation). + * or by an offset from a JVM object (on-heap allocation). */ public class MemoryLocation { diff --git a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java index 621f2c6bf3777..908ff1983e6be 100644 --- a/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java +++ b/common/unsafe/src/main/java/org/apache/spark/unsafe/types/CalendarInterval.java @@ -18,6 +18,7 @@ package org.apache.spark.unsafe.types; import java.io.Serializable; +import java.util.Locale; import java.util.regex.Matcher; import java.util.regex.Pattern; @@ -53,8 +54,8 @@ private static String unitRegex(String unit) { private static Pattern yearMonthPattern = Pattern.compile("^(?:['|\"])?([+|-])?(\\d+)-(\\d+)(?:['|\"])?$"); - private static Pattern dayTimePattern = - Pattern.compile("^(?:['|\"])?([+|-])?(\\d+) (\\d+):(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); + private static Pattern dayTimePattern = Pattern.compile( + "^(?:['|\"])?([+|-])?((\\d+) )?((\\d+):)?(\\d+):(\\d+)(\\.(\\d+))?(?:['|\"])?$"); private static Pattern quoteTrimPattern = Pattern.compile("^(?:['|\"])?(.*?)(?:['|\"])?$"); @@ -66,6 +67,10 @@ private static long toLong(String s) { } } + /** + * Convert a string to CalendarInterval. Return null if the input string is not a valid interval. + * This method is case-sensitive and all characters in the input string should be in lower case. + */ public static CalendarInterval fromString(String s) { if (s == null) { return null; @@ -87,6 +92,26 @@ public static CalendarInterval fromString(String s) { } } + /** + * Convert a string to CalendarInterval. Unlike fromString, this method is case-insensitive and + * will throw IllegalArgumentException when the input string is not a valid interval. + * + * @throws IllegalArgumentException if the string is not a valid internal. + */ + public static CalendarInterval fromCaseInsensitiveString(String s) { + if (s == null || s.trim().isEmpty()) { + throw new IllegalArgumentException("Interval cannot be null or blank."); + } + String sInLowerCase = s.trim().toLowerCase(Locale.ROOT); + String interval = + sInLowerCase.startsWith("interval ") ? sInLowerCase : "interval " + sInLowerCase; + CalendarInterval cal = fromString(interval); + if (cal == null) { + throw new IllegalArgumentException("Invalid interval: " + s); + } + return cal; + } + public static long toLongWithRange(String fieldName, String s, long minValue, long maxValue) throws IllegalArgumentException { long result = 0; @@ -130,11 +155,25 @@ public static CalendarInterval fromYearMonthString(String s) throws IllegalArgum } /** - * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn * * adapted from HiveIntervalDayTime.valueOf */ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumentException { + return fromDayTimeString(s, "day", "second"); + } + + /** + * Parse dayTime string in form: [-]d HH:mm:ss.nnnnnnnnn and [-]HH:mm:ss.nnnnnnnnn + * + * adapted from HiveIntervalDayTime.valueOf. + * Below interval conversion patterns are supported: + * - DAY TO (HOUR|MINUTE|SECOND) + * - HOUR TO (MINUTE|SECOND) + * - MINUTE TO SECOND + */ + public static CalendarInterval fromDayTimeString(String s, String from, String to) + throws IllegalArgumentException { CalendarInterval result = null; if (s == null) { throw new IllegalArgumentException("Interval day-time string was null"); @@ -147,12 +186,42 @@ public static CalendarInterval fromDayTimeString(String s) throws IllegalArgumen } else { try { int sign = m.group(1) != null && m.group(1).equals("-") ? -1 : 1; - long days = toLongWithRange("day", m.group(2), 0, Integer.MAX_VALUE); - long hours = toLongWithRange("hour", m.group(3), 0, 23); - long minutes = toLongWithRange("minute", m.group(4), 0, 59); - long seconds = toLongWithRange("second", m.group(5), 0, 59); + long days = m.group(2) == null ? 0 : toLongWithRange("day", m.group(3), + 0, Integer.MAX_VALUE); + long hours = 0; + long minutes; + long seconds = 0; + if (m.group(5) != null || from.equals("minute")) { // 'HH:mm:ss' or 'mm:ss minute' + hours = toLongWithRange("hour", m.group(5), 0, 23); + minutes = toLongWithRange("minute", m.group(6), 0, 59); + seconds = toLongWithRange("second", m.group(7), 0, 59); + } else if (m.group(8) != null){ // 'mm:ss.nn' + minutes = toLongWithRange("minute", m.group(6), 0, 59); + seconds = toLongWithRange("second", m.group(7), 0, 59); + } else { // 'HH:mm' + hours = toLongWithRange("hour", m.group(6), 0, 23); + minutes = toLongWithRange("second", m.group(7), 0, 59); + } // Hive allow nanosecond precision interval - long nanos = toLongWithRange("nanosecond", m.group(7), 0L, 999999999L); + String nanoStr = m.group(9) == null ? null : (m.group(9) + "000000000").substring(0, 9); + long nanos = toLongWithRange("nanosecond", nanoStr, 0L, 999999999L); + switch (to) { + case "hour": + minutes = 0; + seconds = 0; + nanos = 0; + break; + case "minute": + seconds = 0; + nanos = 0; + break; + case "second": + // No-op + break; + default: + throw new IllegalArgumentException( + String.format("Cannot support (interval '%s' %s to %s) expression", s, from, to)); + } result = new CalendarInterval(0, sign * ( days * MICROS_PER_DAY + hours * MICROS_PER_HOUR + minutes * MICROS_PER_MINUTE + seconds * MICROS_PER_SECOND + nanos / 1000L)); @@ -319,6 +388,8 @@ public String toString() { appendUnit(sb, rest / MICROS_PER_MILLI, "millisecond"); rest %= MICROS_PER_MILLI; appendUnit(sb, rest, "microsecond"); + } else if (months == 0) { + sb.append(" 0 microseconds"); } return sb.toString(); 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 3a3bfc4a94bb3..30b884c5fa9c6 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 @@ -32,6 +32,7 @@ import com.google.common.primitives.Ints; import org.apache.spark.unsafe.Platform; +import org.apache.spark.unsafe.UTF8StringBuilder; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.unsafe.hash.Murmur3_x86_32; @@ -529,26 +530,35 @@ private UTF8String copyUTF8String(int start, int end) { return UTF8String.fromBytes(newBytes); } + /** + * Trims space characters (ASCII 32) from both ends of this string. + * + * @return this string with no spaces at the start or end + */ public UTF8String trim() { int s = 0; // skip all of the space (0x20) in the left side while (s < this.numBytes && getByte(s) == 0x20) s++; if (s == this.numBytes) { - // empty string + // Everything trimmed return EMPTY_UTF8; } // skip all of the space (0x20) in the right side int e = this.numBytes - 1; while (e > s && getByte(e) == 0x20) e--; + if (s == 0 && e == numBytes - 1) { + // Nothing trimmed + return this; + } return copyUTF8String(s, e); } /** - * Based on the given trim string, trim this string starting from both ends - * This method searches for each character in the source string, removes the character if it is - * found in the trim string, stops at the first not found. It calls the trimLeft first, then - * trimRight. It returns a new string in which both ends trim characters have been removed. + * Trims instances of the given trim string from both ends of this string. + * * @param trimString the trim character string + * @return this string with no occurrences of the trim string at the start or end, or `null` + * if `trimString` is `null` */ public UTF8String trim(UTF8String trimString) { if (trimString != null) { @@ -558,24 +568,32 @@ public UTF8String trim(UTF8String trimString) { } } + /** + * Trims space characters (ASCII 32) from the start of this string. + * + * @return this string with no spaces at the start + */ public UTF8String trimLeft() { int s = 0; // skip all of the space (0x20) in the left side while (s < this.numBytes && getByte(s) == 0x20) s++; + if (s == 0) { + // Nothing trimmed + return this; + } if (s == this.numBytes) { - // empty string + // Everything trimmed return EMPTY_UTF8; - } else { - return copyUTF8String(s, this.numBytes - 1); } + return copyUTF8String(s, this.numBytes - 1); } /** - * Based on the given trim string, trim this string starting from left end - * This method searches each character in the source string starting from the left end, removes - * the character if it is in the trim string, stops at the first character which is not in the - * trim string, returns the new string. + * Trims instances of the given trim string from the start of this string. + * * @param trimString the trim character string + * @return this string with no occurrences of the trim string at the start, or `null` + * if `trimString` is `null` */ public UTF8String trimLeft(UTF8String trimString) { if (trimString == null) return null; @@ -597,34 +615,43 @@ public UTF8String trimLeft(UTF8String trimString) { } srchIdx += searchCharBytes; } - + if (srchIdx == 0) { + // Nothing trimmed + return this; + } if (trimIdx >= numBytes) { - // empty string + // Everything trimmed return EMPTY_UTF8; - } else { - return copyUTF8String(trimIdx, numBytes - 1); } + return copyUTF8String(trimIdx, numBytes - 1); } + /** + * Trims space characters (ASCII 32) from the end of this string. + * + * @return this string with no spaces at the end + */ public UTF8String trimRight() { int e = numBytes - 1; // skip all of the space (0x20) in the right side while (e >= 0 && getByte(e) == 0x20) e--; - + if (e == numBytes - 1) { + // Nothing trimmed + return this; + } if (e < 0) { - // empty string + // Everything trimmed return EMPTY_UTF8; - } else { - return copyUTF8String(0, e); } + return copyUTF8String(0, e); } /** - * Based on the given trim string, trim this string starting from right end - * This method searches each character in the source string starting from the right end, - * removes the character if it is in the trim string, stops at the first character which is not - * in the trim string, returns the new string. + * Trims instances of the given trim string from the end of this string. + * * @param trimString the trim character string + * @return this string with no occurrences of the trim string at the end, or `null` + * if `trimString` is `null` */ public UTF8String trimRight(UTF8String trimString) { if (trimString == null) return null; @@ -658,12 +685,15 @@ public UTF8String trimRight(UTF8String trimString) { numChars --; } + if (trimEnd == numBytes - 1) { + // Nothing trimmed + return this; + } if (trimEnd < 0) { - // empty string + // Everything trimmed return EMPTY_UTF8; - } else { - return copyUTF8String(0, trimEnd); } + return copyUTF8String(0, trimEnd); } public UTF8String reverse() { @@ -973,12 +1003,29 @@ public UTF8String[] split(UTF8String pattern, int limit) { } public UTF8String replace(UTF8String search, UTF8String replace) { - if (EMPTY_UTF8.equals(search)) { + // This implementation is loosely based on commons-lang3's StringUtils.replace(). + if (numBytes == 0 || search.numBytes == 0) { + return this; + } + // Find the first occurrence of the search string. + int start = 0; + int end = this.find(search, start); + if (end == -1) { + // Search string was not found, so string is unchanged. return this; } - String replaced = toString().replace( - search.toString(), replace.toString()); - return fromString(replaced); + // At least one match was found. Estimate space needed for result. + // The 16x multiplier here is chosen to match commons-lang3's implementation. + int increase = Math.max(0, replace.numBytes - search.numBytes) * 16; + final UTF8StringBuilder buf = new UTF8StringBuilder(numBytes + increase); + while (end != -1) { + buf.appendBytes(this.base, this.offset + start, end - start); + buf.append(replace); + start = end + search.numBytes; + end = this.find(search, start); + } + buf.appendBytes(this.base, this.offset + start, numBytes - start); + return buf.build(); } // TODO: Need to use `Code Point` here instead of Char in case the character longer than 2 bytes diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java index 9e69e264ff287..c307d74e0ba07 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/CalendarIntervalSuite.java @@ -41,6 +41,9 @@ public void equalsTest() { public void toStringTest() { CalendarInterval i; + i = new CalendarInterval(0, 0); + assertEquals("interval 0 microseconds", i.toString()); + i = new CalendarInterval(34, 0); assertEquals("interval 2 years 10 months", i.toString()); @@ -101,6 +104,31 @@ public void fromStringTest() { assertNull(fromString(input)); } + @Test + public void fromCaseInsensitiveStringTest() { + for (String input : new String[]{"5 MINUTES", "5 minutes", "5 Minutes"}) { + assertEquals(fromCaseInsensitiveString(input), new CalendarInterval(0, 5L * 60 * 1_000_000)); + } + + for (String input : new String[]{null, "", " "}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("cannot be null or blank")); + } + } + + for (String input : new String[]{"interval", "interval1 day", "foo", "foo 1 day"}) { + try { + fromCaseInsensitiveString(input); + fail("Expected to throw an exception for the invalid input"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Invalid interval")); + } + } + } + @Test public void fromYearMonthStringTest() { String input; @@ -134,7 +162,8 @@ public void fromDayTimeStringTest() { assertEquals(fromDayTimeString(input), i); input = "10 0:12:0.888"; - i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE); + i = new CalendarInterval(0, 10 * MICROS_PER_DAY + 12 * MICROS_PER_MINUTE + + 888 * MICROS_PER_MILLI); assertEquals(fromDayTimeString(input), i); input = "-3 0:0:0"; @@ -156,6 +185,14 @@ public void fromDayTimeStringTest() { } catch (IllegalArgumentException e) { assertTrue(e.getMessage().contains("not match day-time format")); } + + try { + input = "5 1:12:20"; + fromDayTimeString(input, "hour", "microsecond"); + fail("Expected to throw an exception for the invalid convention type"); + } catch (IllegalArgumentException e) { + assertTrue(e.getMessage().contains("Cannot support (interval")); + } } @Test diff --git a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java index cf9cc6b1800a9..bc75fa9e724a0 100644 --- a/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java +++ b/common/unsafe/src/test/java/org/apache/spark/unsafe/types/UTF8StringSuite.java @@ -403,6 +403,44 @@ public void split() { new UTF8String[]{fromString("ab"), fromString("def,ghi,")})); } + @Test + public void replace() { + assertEquals( + fromString("re123ace"), + fromString("replace").replace(fromString("pl"), fromString("123"))); + assertEquals( + fromString("reace"), + fromString("replace").replace(fromString("pl"), fromString(""))); + assertEquals( + fromString("replace"), + fromString("replace").replace(fromString(""), fromString("123"))); + // tests for multiple replacements + assertEquals( + fromString("a12ca12c"), + fromString("abcabc").replace(fromString("b"), fromString("12"))); + assertEquals( + fromString("adad"), + fromString("abcdabcd").replace(fromString("bc"), fromString(""))); + // tests for single character search and replacement strings + assertEquals( + fromString("AbcAbc"), + fromString("abcabc").replace(fromString("a"), fromString("A"))); + assertEquals( + fromString("abcabc"), + fromString("abcabc").replace(fromString("Z"), fromString("A"))); + // Tests with non-ASCII characters + assertEquals( + fromString("花ab界"), + fromString("花花世界").replace(fromString("花世"), fromString("ab"))); + assertEquals( + fromString("a水c"), + fromString("a火c").replace(fromString("火"), fromString("水"))); + // Tests for a large number of replacements, triggering UTF8StringBuilder resize + assertEquals( + fromString("abcd").repeat(17), + fromString("a").repeat(17).replace(fromString("a"), fromString("abcd"))); + } + @Test public void levenshteinDistance() { assertEquals(0, EMPTY_UTF8.levenshteinDistance(EMPTY_UTF8)); diff --git a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala index 9656951810daf..fdb81a06d41c9 100644 --- a/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala +++ b/common/unsafe/src/test/scala/org/apache/spark/unsafe/types/UTF8StringPropertyCheckSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.unsafe.types -import org.apache.commons.lang3.StringUtils +import org.apache.commons.text.similarity.LevenshteinDistance import org.scalacheck.{Arbitrary, Gen} import org.scalatest.prop.GeneratorDrivenPropertyChecks // scalastyle:off @@ -232,7 +232,7 @@ class UTF8StringPropertyCheckSuite extends FunSuite with GeneratorDrivenProperty test("levenshteinDistance") { forAll { (one: String, another: String) => assert(toUTF8(one).levenshteinDistance(toUTF8(another)) === - StringUtils.getLevenshteinDistance(one, another)) + LevenshteinDistance.getDefaultInstance.apply(one, another)) } } diff --git a/conf/log4j.properties.template b/conf/log4j.properties.template index ec1aa187dfb32..e91595dd324b0 100644 --- a/conf/log4j.properties.template +++ b/conf/log4j.properties.template @@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: log4j.logger.org.apache.spark.repl.Main=WARN # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=WARN -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.sparkproject.jetty=WARN +log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO log4j.logger.org.apache.parquet=ERROR diff --git a/conf/metrics.properties.template b/conf/metrics.properties.template index 4c008a13607c2..23407e1f70755 100644 --- a/conf/metrics.properties.template +++ b/conf/metrics.properties.template @@ -64,6 +64,10 @@ # "/metrics/applications/json" endpoints can be sent separately to get # metrics snapshots of the master instance and applications. This # MetricsServlet does not have to be configured. +# 6. The metrics system can also be configured using Spark configuration +# parameters. The relevant parameter names are formed by adding the +# prefix "spark.metrics.conf." to the configuration entries detailed in +# this file (see examples below). ## List of available common sources and their properties. @@ -172,6 +176,14 @@ # Unit of the polling period for the Slf4jSink #*.sink.slf4j.unit=minutes +# Example configuration for Graphite sink +#*.sink.graphite.class=org.apache.spark.metrics.sink.GraphiteSink +#*.sink.graphite.host= +#*.sink.graphite.port= +#*.sink.graphite.period=10 +#*.sink.graphite.unit=seconds +#*.sink.graphite.prefix= + # Enable JvmSource for instance master, worker, driver and executor #master.source.jvm.class=org.apache.spark.metrics.source.JvmSource @@ -179,4 +191,4 @@ #driver.source.jvm.class=org.apache.spark.metrics.source.JvmSource -#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource +#executor.source.jvm.class=org.apache.spark.metrics.source.JvmSource \ No newline at end of file diff --git a/core/benchmarks/CoalescedRDDBenchmark-results.txt b/core/benchmarks/CoalescedRDDBenchmark-results.txt new file mode 100644 index 0000000000000..dd63b0adea4f2 --- /dev/null +++ b/core/benchmarks/CoalescedRDDBenchmark-results.txt @@ -0,0 +1,40 @@ +================================================================================================ +Coalesced RDD , large scale +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_201-b09 on Windows 10 10.0 +Intel64 Family 6 Model 63 Stepping 2, GenuineIntel +Coalesced RDD: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Coalesce Num Partitions: 100 Num Hosts: 1 346 364 24 0.3 3458.9 1.0X +Coalesce Num Partitions: 100 Num Hosts: 5 258 264 6 0.4 2579.0 1.3X +Coalesce Num Partitions: 100 Num Hosts: 10 242 249 7 0.4 2415.2 1.4X +Coalesce Num Partitions: 100 Num Hosts: 20 237 242 7 0.4 2371.7 1.5X +Coalesce Num Partitions: 100 Num Hosts: 40 230 231 1 0.4 2299.8 1.5X +Coalesce Num Partitions: 100 Num Hosts: 80 222 233 14 0.4 2223.0 1.6X +Coalesce Num Partitions: 500 Num Hosts: 1 659 665 5 0.2 6590.4 0.5X +Coalesce Num Partitions: 500 Num Hosts: 5 340 381 47 0.3 3395.2 1.0X +Coalesce Num Partitions: 500 Num Hosts: 10 279 307 47 0.4 2788.3 1.2X +Coalesce Num Partitions: 500 Num Hosts: 20 259 261 2 0.4 2591.9 1.3X +Coalesce Num Partitions: 500 Num Hosts: 40 241 250 15 0.4 2406.5 1.4X +Coalesce Num Partitions: 500 Num Hosts: 80 235 237 3 0.4 2349.9 1.5X +Coalesce Num Partitions: 1000 Num Hosts: 1 1050 1053 4 0.1 10503.2 0.3X +Coalesce Num Partitions: 1000 Num Hosts: 5 405 407 2 0.2 4049.5 0.9X +Coalesce Num Partitions: 1000 Num Hosts: 10 320 322 2 0.3 3202.7 1.1X +Coalesce Num Partitions: 1000 Num Hosts: 20 276 277 0 0.4 2762.3 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 40 257 260 5 0.4 2571.2 1.3X +Coalesce Num Partitions: 1000 Num Hosts: 80 245 252 13 0.4 2448.9 1.4X +Coalesce Num Partitions: 5000 Num Hosts: 1 3099 3145 55 0.0 30988.6 0.1X +Coalesce Num Partitions: 5000 Num Hosts: 5 1037 1050 20 0.1 10374.4 0.3X +Coalesce Num Partitions: 5000 Num Hosts: 10 626 633 8 0.2 6261.8 0.6X +Coalesce Num Partitions: 5000 Num Hosts: 20 426 431 5 0.2 4258.6 0.8X +Coalesce Num Partitions: 5000 Num Hosts: 40 328 341 22 0.3 3275.4 1.1X +Coalesce Num Partitions: 5000 Num Hosts: 80 272 275 4 0.4 2721.4 1.3X +Coalesce Num Partitions: 10000 Num Hosts: 1 5516 5526 9 0.0 55156.8 0.1X +Coalesce Num Partitions: 10000 Num Hosts: 5 1956 1992 48 0.1 19560.9 0.2X +Coalesce Num Partitions: 10000 Num Hosts: 10 1045 1057 18 0.1 10447.4 0.3X +Coalesce Num Partitions: 10000 Num Hosts: 20 637 658 24 0.2 6373.2 0.5X +Coalesce Num Partitions: 10000 Num Hosts: 40 431 448 15 0.2 4312.9 0.8X +Coalesce Num Partitions: 10000 Num Hosts: 80 326 328 2 0.3 3263.4 1.1X + + diff --git a/core/benchmarks/KryoBenchmark-results.txt b/core/benchmarks/KryoBenchmark-results.txt new file mode 100644 index 0000000000000..91e22f3afc14f --- /dev/null +++ b/core/benchmarks/KryoBenchmark-results.txt @@ -0,0 +1,29 @@ +================================================================================================ +Benchmark Kryo Unsafe vs safe Serialization +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + +Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +basicTypes: Int with unsafe:true 138 / 149 7.2 138.0 1.0X +basicTypes: Long with unsafe:true 168 / 173 6.0 167.7 0.8X +basicTypes: Float with unsafe:true 153 / 174 6.5 153.1 0.9X +basicTypes: Double with unsafe:true 161 / 185 6.2 161.1 0.9X +Array: Int with unsafe:true 2 / 3 409.7 2.4 56.5X +Array: Long with unsafe:true 4 / 5 232.5 4.3 32.1X +Array: Float with unsafe:true 3 / 4 367.3 2.7 50.7X +Array: Double with unsafe:true 4 / 5 228.5 4.4 31.5X +Map of string->Double with unsafe:true 38 / 45 26.5 37.8 3.7X +basicTypes: Int with unsafe:false 176 / 187 5.7 175.9 0.8X +basicTypes: Long with unsafe:false 191 / 203 5.2 191.2 0.7X +basicTypes: Float with unsafe:false 166 / 176 6.0 166.2 0.8X +basicTypes: Double with unsafe:false 174 / 190 5.7 174.3 0.8X +Array: Int with unsafe:false 19 / 26 52.9 18.9 7.3X +Array: Long with unsafe:false 27 / 31 37.7 26.5 5.2X +Array: Float with unsafe:false 8 / 10 124.3 8.0 17.2X +Array: Double with unsafe:false 12 / 13 83.6 12.0 11.5X +Map of string->Double with unsafe:false 38 / 42 26.1 38.3 3.6X + + diff --git a/core/benchmarks/KryoSerializerBenchmark-results.txt b/core/benchmarks/KryoSerializerBenchmark-results.txt new file mode 100644 index 0000000000000..c3ce336d93241 --- /dev/null +++ b/core/benchmarks/KryoSerializerBenchmark-results.txt @@ -0,0 +1,12 @@ +================================================================================================ +Benchmark KryoPool vs "pool of 1" +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_131-b11 on Mac OS X 10.14 +Intel(R) Core(TM) i7-4770HQ CPU @ 2.20GHz +Benchmark KryoPool vs "pool of 1": Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +KryoPool:true 2682 / 3425 0.0 5364627.9 1.0X +KryoPool:false 8176 / 9292 0.0 16351252.2 0.3X + + diff --git a/core/benchmarks/XORShiftRandomBenchmark-results.txt b/core/benchmarks/XORShiftRandomBenchmark-results.txt new file mode 100644 index 0000000000000..1140489e4a7f3 --- /dev/null +++ b/core/benchmarks/XORShiftRandomBenchmark-results.txt @@ -0,0 +1,44 @@ +================================================================================================ +Pseudo random +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextInt: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 1362 / 1362 73.4 13.6 1.0X +XORShiftRandom 227 / 227 440.6 2.3 6.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextLong: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 2732 / 2732 36.6 27.3 1.0X +XORShiftRandom 629 / 629 159.0 6.3 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextDouble: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 2730 / 2730 36.6 27.3 1.0X +XORShiftRandom 629 / 629 159.0 6.3 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +nextGaussian: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +java.util.Random 10288 / 10288 9.7 102.9 1.0X +XORShiftRandom 6351 / 6351 15.7 63.5 1.6X + + +================================================================================================ +hash seed +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Hash seed: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +XORShiftRandom.hashSeed 1193 / 1195 8.4 119.3 1.0X + + diff --git a/core/pom.xml b/core/pom.xml index eff3aa1d19423..4446dbdb5ed06 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-core_2.11 + spark-core_2.12 core @@ -33,6 +33,10 @@ Spark Project Core http://spark.apache.org/ + + com.thoughtworks.paranamer + paranamer + org.apache.avro avro @@ -56,7 +60,7 @@ org.apache.xbean - xbean-asm6-shaded + xbean-asm7-shaded org.apache.hadoop @@ -216,10 +220,18 @@ commons-net commons-net + + org.scala-lang.modules + scala-xml_${scala.binary.version} + org.scala-lang scala-library + + org.scala-lang + scala-reflect + org.json4s json4s-jackson_${scala.binary.version} @@ -305,6 +317,33 @@ selenium-htmlunit-driver test + + + net.sourceforge.htmlunit + htmlunit + test + + + net.sourceforge.htmlunit + htmlunit-core-js + test + + + + org.apache.httpcomponents + httpclient + test + + + org.apache.httpcomponents + httpmime + test + + + org.apache.httpcomponents + httpcore + test + xml-apis @@ -339,7 +378,7 @@ net.razorvine pyrolite - 4.13 + 4.30 net.razorvine @@ -350,7 +389,7 @@ net.sf.py4j py4j - 0.10.7 + 0.10.8.1 org.apache.spark @@ -364,6 +403,13 @@ tests test + + org.apache.spark + spark-network-shuffle_${scala.binary.version} + ${project.version} + tests + test + - - ${hive.group} - hive-exec - provided - - - ${hive.group} - hive-metastore - provided - - - org.apache.thrift - libthrift - provided - - - org.apache.thrift - libfb303 - provided - - target/scala-${scala.binary.version}/classes diff --git a/core/src/main/java/org/apache/spark/ExecutorPlugin.java b/core/src/main/java/org/apache/spark/ExecutorPlugin.java index ec0b57f1a2819..f86520c81df33 100644 --- a/core/src/main/java/org/apache/spark/ExecutorPlugin.java +++ b/core/src/main/java/org/apache/spark/ExecutorPlugin.java @@ -20,18 +20,18 @@ import org.apache.spark.annotation.DeveloperApi; /** - * A plugin which can be automaticaly instantiated within each Spark executor. Users can specify + * A plugin which can be automatically instantiated within each Spark executor. Users can specify * plugins which should be created with the "spark.executor.plugins" configuration. An instance * of each plugin will be created for every executor, including those created by dynamic allocation, * before the executor starts running any tasks. * * The specific api exposed to the end users still considered to be very unstable. We will - * hopefully be able to keep compatability by providing default implementations for any methods + * hopefully be able to keep compatibility by providing default implementations for any methods * added, but make no guarantees this will always be possible across all Spark releases. * * Spark does nothing to verify the plugin is doing legitimate things, or to manage the resources * it uses. A plugin acquires the same privileges as the user running the task. A bad plugin - * could also intefere with task execution and make the executor fail in unexpected ways. + * could also interfere with task execution and make the executor fail in unexpected ways. */ @DeveloperApi public interface ExecutorPlugin { diff --git a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java b/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java deleted file mode 100644 index 0dd8fafbf2c82..0000000000000 --- a/core/src/main/java/org/apache/spark/api/java/JavaSparkContextVarargsWorkaround.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.api.java; - -import java.util.ArrayList; -import java.util.List; - -// See -// http://scala-programming-language.1934581.n4.nabble.com/Workaround-for-implementing-java-varargs-in-2-7-2-final-tp1944767p1944772.html -abstract class JavaSparkContextVarargsWorkaround { - - @SafeVarargs - public final JavaRDD union(JavaRDD... rdds) { - if (rdds.length == 0) { - throw new IllegalArgumentException("Union called on empty list"); - } - List> rest = new ArrayList<>(rdds.length - 1); - for (int i = 1; i < rdds.length; i++) { - rest.add(rdds[i]); - } - return union(rdds[0], rest); - } - - public JavaDoubleRDD union(JavaDoubleRDD... rdds) { - if (rdds.length == 0) { - throw new IllegalArgumentException("Union called on empty list"); - } - List rest = new ArrayList<>(rdds.length - 1); - for (int i = 1; i < rdds.length; i++) { - rest.add(rdds[i]); - } - return union(rdds[0], rest); - } - - @SafeVarargs - public final JavaPairRDD union(JavaPairRDD... rdds) { - if (rdds.length == 0) { - throw new IllegalArgumentException("Union called on empty list"); - } - List> rest = new ArrayList<>(rdds.length - 1); - for (int i = 1; i < rdds.length; i++) { - rest.add(rdds[i]); - } - return union(rdds[0], rest); - } - - // These methods take separate "first" and "rest" elements to avoid having the same type erasure - public abstract JavaRDD union(JavaRDD first, List> rest); - public abstract JavaDoubleRDD union(JavaDoubleRDD first, List rest); - public abstract JavaPairRDD union(JavaPairRDD first, List> - rest); -} diff --git a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java index f6d1288cb263d..92bf0ecc1b5cb 100644 --- a/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java +++ b/core/src/main/java/org/apache/spark/io/NioBufferedFileInputStream.java @@ -27,7 +27,7 @@ * to read a file to avoid extra copy of data between Java and * native memory which happens when using {@link java.io.BufferedInputStream}. * Unfortunately, this is not something already available in JDK, - * {@link sun.nio.ch.ChannelInputStream} supports reading a file using nio, + * {@code sun.nio.ch.ChannelInputStream} supports reading a file using nio, * but does not support buffering. */ public final class NioBufferedFileInputStream extends InputStream { @@ -130,6 +130,7 @@ public synchronized void close() throws IOException { StorageUtils.dispose(byteBuffer); } + @SuppressWarnings("deprecation") @Override protected void finalize() throws IOException { close(); diff --git a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java index 115e1fbb79a2e..4bfd2d358f36f 100644 --- a/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java +++ b/core/src/main/java/org/apache/spark/memory/MemoryConsumer.java @@ -83,10 +83,10 @@ public void spill() throws IOException { public abstract long spill(long size, MemoryConsumer trigger) throws IOException; /** - * Allocates a LongArray of `size`. Note that this method may throw `OutOfMemoryError` if Spark - * doesn't have enough memory for this allocation, or throw `TooLargePageException` if this - * `LongArray` is too large to fit in a single page. The caller side should take care of these - * two exceptions, or make sure the `size` is small enough that won't trigger exceptions. + * Allocates a LongArray of `size`. Note that this method may throw `SparkOutOfMemoryError` + * if Spark doesn't have enough memory for this allocation, or throw `TooLargePageException` + * if this `LongArray` is too large to fit in a single page. The caller side should take care of + * these two exceptions, or make sure the `size` is small enough that won't trigger exceptions. * * @throws SparkOutOfMemoryError * @throws TooLargePageException @@ -111,7 +111,7 @@ public void freeArray(LongArray array) { /** * Allocate a memory block with at least `required` bytes. * - * @throws OutOfMemoryError + * @throws SparkOutOfMemoryError */ protected MemoryBlock allocatePage(long required) { MemoryBlock page = taskMemoryManager.allocatePage(Math.max(pageSize, required), this); @@ -154,7 +154,9 @@ private void throwOom(final MemoryBlock page, final long required) { taskMemoryManager.freePage(page, this); } taskMemoryManager.showMemoryUsage(); + // checkstyle.off: RegexpSinglelineJava throw new SparkOutOfMemoryError("Unable to acquire " + required + " bytes of memory, got " + got); + // checkstyle.on: RegexpSinglelineJava } } diff --git a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java index d07faf1da1248..1d9391845be5f 100644 --- a/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java +++ b/core/src/main/java/org/apache/spark/memory/TaskMemoryManager.java @@ -85,9 +85,9 @@ public class TaskMemoryManager { /** * Similar to an operating system's page table, this array maps page numbers into base object * pointers, allowing us to translate between the hashtable's internal 64-bit address - * representation and the baseObject+offset representation which we use to support both in- and + * representation and the baseObject+offset representation which we use to support both on- and * off-heap addresses. When using an off-heap allocator, every entry in this map will be `null`. - * When using an in-heap allocator, the entries in this map will point to pages' base objects. + * When using an on-heap allocator, the entries in this map will point to pages' base objects. * Entries are added to this map as new data pages are allocated. */ private final MemoryBlock[] pageTable = new MemoryBlock[PAGE_TABLE_SIZE]; @@ -102,7 +102,7 @@ public class TaskMemoryManager { private final long taskAttemptId; /** - * Tracks whether we're in-heap or off-heap. For off-heap, we short-circuit most of these methods + * Tracks whether we're on-heap or off-heap. For off-heap, we short-circuit most of these methods * without doing any masking or lookups. Since this branching should be well-predicted by the JIT, * this extra layer of indirection / abstraction hopefully shouldn't be too expensive. */ @@ -194,8 +194,10 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("error while calling spill() on " + c, e); + // checkstyle.off: RegexpSinglelineJava throw new SparkOutOfMemoryError("error while calling spill() on " + c + " : " + e.getMessage()); + // checkstyle.on: RegexpSinglelineJava } } } @@ -215,8 +217,10 @@ public long acquireExecutionMemory(long required, MemoryConsumer consumer) { throw new RuntimeException(e.getMessage()); } catch (IOException e) { logger.error("error while calling spill() on " + consumer, e); + // checkstyle.off: RegexpSinglelineJava throw new SparkOutOfMemoryError("error while calling spill() on " + consumer + " : " + e.getMessage()); + // checkstyle.on: RegexpSinglelineJava } } diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java index 4426c7afcebdd..a029931f9e4c0 100644 --- a/core/src/main/java/org/apache/spark/package-info.java +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -16,8 +16,8 @@ */ /** - * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} - * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * Core Spark classes in Scala. A few classes here, such as + * {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the * {@link org.apache.spark.api.java} package contains the main Java API. */ package org.apache.spark; diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java index b020a6d99247b..32b446785a9f0 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriter.java @@ -34,15 +34,15 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import org.apache.spark.internal.config.package$; import org.apache.spark.Partitioner; import org.apache.spark.ShuffleDependency; import org.apache.spark.SparkConf; -import org.apache.spark.TaskContext; -import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; import org.apache.spark.serializer.Serializer; import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.shuffle.IndexShuffleBlockResolver; import org.apache.spark.shuffle.ShuffleWriter; import org.apache.spark.storage.*; @@ -59,9 +59,8 @@ * simultaneously opens separate serializers and file streams for all partitions. As a result, * {@link SortShuffleManager} only selects this write path when *
    - *
  • no Ordering is specified,
  • - *
  • no Aggregator is specified, and
  • - *
  • the number of partitions is less than + *
  • no map-side combine is specified, and
  • + *
  • the number of partitions is less than or equal to * spark.shuffle.sort.bypassMergeThreshold.
  • *
* @@ -79,7 +78,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { private final int numPartitions; private final BlockManager blockManager; private final Partitioner partitioner; - private final ShuffleWriteMetrics writeMetrics; + private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; private final int mapId; private final Serializer serializer; @@ -103,10 +102,10 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { IndexShuffleBlockResolver shuffleBlockResolver, BypassMergeSortShuffleHandle handle, int mapId, - TaskContext taskContext, - SparkConf conf) { + SparkConf conf, + ShuffleWriteMetricsReporter writeMetrics) { // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - this.fileBufferSize = (int) conf.getSizeAsKb("spark.shuffle.file.buffer", "32k") * 1024; + this.fileBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.transferToEnabled = conf.getBoolean("spark.file.transferTo", true); this.blockManager = blockManager; final ShuffleDependency dep = handle.dependency(); @@ -114,7 +113,7 @@ final class BypassMergeSortShuffleWriter extends ShuffleWriter { this.shuffleId = dep.shuffleId(); this.partitioner = dep.partitioner(); this.numPartitions = partitioner.numPartitions(); - this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + this.writeMetrics = writeMetrics; this.serializer = dep.serializer(); this.shuffleBlockResolver = shuffleBlockResolver; } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java index 1c0d664afb138..024756087bf7f 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/ShuffleExternalSorter.java @@ -38,6 +38,7 @@ import org.apache.spark.memory.TooLargePageException; import org.apache.spark.serializer.DummySerializerInstance; import org.apache.spark.serializer.SerializerInstance; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.storage.BlockManager; import org.apache.spark.storage.DiskBlockObjectWriter; import org.apache.spark.storage.FileSegment; @@ -75,7 +76,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { private final TaskMemoryManager taskMemoryManager; private final BlockManager blockManager; private final TaskContext taskContext; - private final ShuffleWriteMetrics writeMetrics; + private final ShuffleWriteMetricsReporter writeMetrics; /** * Force this sorter to spill when there are this many elements in memory. @@ -113,7 +114,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { int initialSize, int numPartitions, SparkConf conf, - ShuffleWriteMetrics writeMetrics) { + ShuffleWriteMetricsReporter writeMetrics) { super(memoryManager, (int) Math.min(PackedRecordPointer.MAXIMUM_PAGE_SIZE_BYTES, memoryManager.pageSizeBytes()), memoryManager.getTungstenMemoryMode()); @@ -128,7 +129,7 @@ final class ShuffleExternalSorter extends MemoryConsumer { (int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()); this.writeMetrics = writeMetrics; this.inMemSorter = new ShuffleInMemorySorter( - this, initialSize, conf.getBoolean("spark.shuffle.sort.useRadixSort", true)); + this, initialSize, (boolean) conf.get(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT())); this.peakMemoryUsedBytes = getMemoryUsage(); this.diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); @@ -144,7 +145,16 @@ final class ShuffleExternalSorter extends MemoryConsumer { */ private void writeSortedFile(boolean isLastFile) { - final ShuffleWriteMetrics writeMetricsToUse; + // This call performs the actual sort. + final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = + inMemSorter.getSortedIterator(); + + // If there are no sorted records, so we don't need to create an empty spill file. + if (!sortedRecords.hasNext()) { + return; + } + + final ShuffleWriteMetricsReporter writeMetricsToUse; if (isLastFile) { // We're writing the final non-spill file, so we _do_ want to count this as shuffle bytes. @@ -156,10 +166,6 @@ private void writeSortedFile(boolean isLastFile) { writeMetricsToUse = new ShuffleWriteMetrics(); } - // This call performs the actual sort. - final ShuffleInMemorySorter.ShuffleSorterIterator sortedRecords = - inMemSorter.getSortedIterator(); - // Small writes to DiskBlockObjectWriter will be fairly inefficient. Since there doesn't seem to // be an API to directly transfer bytes from managed memory to the disk writer, we buffer // data through a byte array. This array does not need to be large enough to hold a single @@ -241,9 +247,14 @@ private void writeSortedFile(boolean isLastFile) { // // Note that we intentionally ignore the value of `writeMetricsToUse.shuffleWriteTime()`. // Consistent with ExternalSorter, we do not count this IO towards shuffle write time. - // This means that this IO time is not accounted for anywhere; SPARK-3577 will fix this. - writeMetrics.incRecordsWritten(writeMetricsToUse.recordsWritten()); - taskContext.taskMetrics().incDiskBytesSpilled(writeMetricsToUse.bytesWritten()); + // SPARK-3577 tracks the spill time separately. + + // This is guaranteed to be a ShuffleWriteMetrics based on the if check in the beginning + // of this method. + writeMetrics.incRecordsWritten( + ((ShuffleWriteMetrics)writeMetricsToUse).recordsWritten()); + taskContext.taskMetrics().incDiskBytesSpilled( + ((ShuffleWriteMetrics)writeMetricsToUse).bytesWritten()); } } diff --git a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java index 4839d04522f10..9d05f03613ce9 100644 --- a/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java +++ b/core/src/main/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriter.java @@ -37,7 +37,7 @@ import org.apache.spark.*; import org.apache.spark.annotation.Private; -import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.internal.config.package$; import org.apache.spark.io.CompressionCodec; import org.apache.spark.io.CompressionCodec$; import org.apache.spark.io.NioBufferedFileInputStream; @@ -47,6 +47,7 @@ import org.apache.spark.network.util.LimitedInputStream; import org.apache.spark.scheduler.MapStatus; import org.apache.spark.scheduler.MapStatus$; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; import org.apache.spark.serializer.SerializationStream; import org.apache.spark.serializer.SerializerInstance; import org.apache.spark.shuffle.IndexShuffleBlockResolver; @@ -55,7 +56,6 @@ import org.apache.spark.storage.TimeTrackingOutputStream; import org.apache.spark.unsafe.Platform; import org.apache.spark.util.Utils; -import org.apache.spark.internal.config.package$; @Private public class UnsafeShuffleWriter extends ShuffleWriter { @@ -73,7 +73,7 @@ public class UnsafeShuffleWriter extends ShuffleWriter { private final TaskMemoryManager memoryManager; private final SerializerInstance serializer; private final Partitioner partitioner; - private final ShuffleWriteMetrics writeMetrics; + private final ShuffleWriteMetricsReporter writeMetrics; private final int shuffleId; private final int mapId; private final TaskContext taskContext; @@ -122,7 +122,8 @@ public UnsafeShuffleWriter( SerializedShuffleHandle handle, int mapId, TaskContext taskContext, - SparkConf sparkConf) throws IOException { + SparkConf sparkConf, + ShuffleWriteMetricsReporter writeMetrics) throws IOException { final int numPartitions = handle.dependency().partitioner().numPartitions(); if (numPartitions > SortShuffleManager.MAX_SHUFFLE_OUTPUT_PARTITIONS_FOR_SERIALIZED_MODE()) { throw new IllegalArgumentException( @@ -138,12 +139,12 @@ public UnsafeShuffleWriter( this.shuffleId = dep.shuffleId(); this.serializer = dep.serializer().newInstance(); this.partitioner = dep.partitioner(); - this.writeMetrics = taskContext.taskMetrics().shuffleWriteMetrics(); + this.writeMetrics = writeMetrics; this.taskContext = taskContext; this.sparkConf = sparkConf; this.transferToEnabled = sparkConf.getBoolean("spark.file.transferTo", true); - this.initialSortBufferSize = sparkConf.getInt("spark.shuffle.sort.initialBufferSize", - DEFAULT_INITIAL_SORT_BUFFER_SIZE); + this.initialSortBufferSize = + (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()); this.inputBufferSizeInBytes = (int) (long) sparkConf.get(package$.MODULE$.SHUFFLE_FILE_BUFFER_SIZE()) * 1024; this.outputBufferSizeInBytes = @@ -281,10 +282,10 @@ void forceSorterToSpill() throws IOException { * @return the partition lengths in the merged file. */ private long[] mergeSpills(SpillInfo[] spills, File outputFile) throws IOException { - final boolean compressionEnabled = sparkConf.getBoolean("spark.shuffle.compress", true); + final boolean compressionEnabled = (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_COMPRESS()); final CompressionCodec compressionCodec = CompressionCodec$.MODULE$.createCodec(sparkConf); final boolean fastMergeEnabled = - sparkConf.getBoolean("spark.shuffle.unsafe.fastMergeEnabled", true); + (boolean) sparkConf.get(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE()); final boolean fastMergeIsSupported = !compressionEnabled || CompressionCodec$.MODULE$.supportsConcatenationOfSerializedStreams(compressionCodec); final boolean encryptionEnabled = blockManager.serializerManager().encryptionEnabled(); diff --git a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java index 5d0555a8c28e1..fcba3b73445c9 100644 --- a/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java +++ b/core/src/main/java/org/apache/spark/storage/TimeTrackingOutputStream.java @@ -21,7 +21,7 @@ import java.io.OutputStream; import org.apache.spark.annotation.Private; -import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter; /** * Intercepts write calls and tracks total time spent writing in order to update shuffle write @@ -30,10 +30,11 @@ @Private public final class TimeTrackingOutputStream extends OutputStream { - private final ShuffleWriteMetrics writeMetrics; + private final ShuffleWriteMetricsReporter writeMetrics; private final OutputStream outputStream; - public TimeTrackingOutputStream(ShuffleWriteMetrics writeMetrics, OutputStream outputStream) { + public TimeTrackingOutputStream( + ShuffleWriteMetricsReporter writeMetrics, OutputStream outputStream) { this.writeMetrics = writeMetrics; this.outputStream = outputStream; } 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 9b6cbab38cbcc..13ca7fb8a3181 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 @@ -31,6 +31,7 @@ import org.apache.spark.SparkEnv; import org.apache.spark.executor.ShuffleWriteMetrics; import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockManager; @@ -158,11 +159,9 @@ public final class BytesToBytesMap extends MemoryConsumer { */ private final Location loc; - private final boolean enablePerfMetrics; + private long numProbes = 0L; - private long numProbes = 0; - - private long numKeyLookups = 0; + private long numKeyLookups = 0L; private long peakMemoryUsedBytes = 0L; @@ -179,8 +178,7 @@ public BytesToBytesMap( SerializerManager serializerManager, int initialCapacity, double loadFactor, - long pageSizeBytes, - boolean enablePerfMetrics) { + long pageSizeBytes) { super(taskMemoryManager, pageSizeBytes, taskMemoryManager.getTungstenMemoryMode()); this.taskMemoryManager = taskMemoryManager; this.blockManager = blockManager; @@ -188,7 +186,6 @@ public BytesToBytesMap( this.loadFactor = loadFactor; this.loc = new Location(); this.pageSizeBytes = pageSizeBytes; - this.enablePerfMetrics = enablePerfMetrics; if (initialCapacity <= 0) { throw new IllegalArgumentException("Initial capacity must be greater than 0"); } @@ -208,14 +205,6 @@ public BytesToBytesMap( TaskMemoryManager taskMemoryManager, int initialCapacity, long pageSizeBytes) { - this(taskMemoryManager, initialCapacity, pageSizeBytes, false); - } - - public BytesToBytesMap( - TaskMemoryManager taskMemoryManager, - int initialCapacity, - long pageSizeBytes, - boolean enablePerfMetrics) { this( taskMemoryManager, SparkEnv.get() != null ? SparkEnv.get().blockManager() : null, @@ -223,8 +212,7 @@ public BytesToBytesMap( initialCapacity, // In order to re-use the longArray for sorting, the load factor cannot be larger than 0.5. 0.5, - pageSizeBytes, - enablePerfMetrics); + pageSizeBytes); } /** @@ -267,33 +255,46 @@ private MapIterator(int numRecords, Location loc, boolean destructive) { } private void advanceToNextPage() { - synchronized (this) { - int nextIdx = dataPages.indexOf(currentPage) + 1; - if (destructive && currentPage != null) { - dataPages.remove(currentPage); - freePage(currentPage); - nextIdx --; - } - if (dataPages.size() > nextIdx) { - currentPage = dataPages.get(nextIdx); - pageBaseObject = currentPage.getBaseObject(); - offsetInPage = currentPage.getBaseOffset(); - recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); - offsetInPage += UnsafeAlignedOffset.getUaoSize(); - } else { - currentPage = null; - if (reader != null) { - handleFailedDelete(); + // SPARK-26265: We will first lock this `MapIterator` and then `TaskMemoryManager` when going + // to free a memory page by calling `freePage`. At the same time, it is possibly that another + // memory consumer first locks `TaskMemoryManager` and then this `MapIterator` when it + // acquires memory and causes spilling on this `MapIterator`. To avoid deadlock here, we keep + // reference to the page to free and free it after releasing the lock of `MapIterator`. + MemoryBlock pageToFree = null; + + try { + synchronized (this) { + int nextIdx = dataPages.indexOf(currentPage) + 1; + if (destructive && currentPage != null) { + dataPages.remove(currentPage); + pageToFree = currentPage; + nextIdx--; } - try { - Closeables.close(reader, /* swallowIOException = */ false); - reader = spillWriters.getFirst().getReader(serializerManager); - recordsInPage = -1; - } catch (IOException e) { - // Scala iterator does not handle exception - Platform.throwException(e); + if (dataPages.size() > nextIdx) { + currentPage = dataPages.get(nextIdx); + pageBaseObject = currentPage.getBaseObject(); + offsetInPage = currentPage.getBaseOffset(); + recordsInPage = UnsafeAlignedOffset.getSize(pageBaseObject, offsetInPage); + offsetInPage += UnsafeAlignedOffset.getUaoSize(); + } else { + currentPage = null; + if (reader != null) { + handleFailedDelete(); + } + try { + Closeables.close(reader, /* swallowIOException = */ false); + reader = spillWriters.getFirst().getReader(serializerManager); + recordsInPage = -1; + } catch (IOException e) { + // Scala iterator does not handle exception + Platform.throwException(e); + } } } + } finally { + if (pageToFree != null) { + freePage(pageToFree); + } } } @@ -341,52 +342,50 @@ public Location next() { } } - public long spill(long numBytes) throws IOException { - synchronized (this) { - if (!destructive || dataPages.size() == 1) { - return 0L; - } + public synchronized long spill(long numBytes) throws IOException { + if (!destructive || dataPages.size() == 1) { + return 0L; + } - updatePeakMemoryUsed(); + updatePeakMemoryUsed(); - // TODO: use existing ShuffleWriteMetrics - ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); + // TODO: use existing ShuffleWriteMetrics + ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - long released = 0L; - while (dataPages.size() > 0) { - MemoryBlock block = dataPages.getLast(); - // The currentPage is used, cannot be released - if (block == currentPage) { - break; - } + long released = 0L; + while (dataPages.size() > 0) { + MemoryBlock block = dataPages.getLast(); + // The currentPage is used, cannot be released + if (block == currentPage) { + break; + } - Object base = block.getBaseObject(); - long offset = block.getBaseOffset(); - int numRecords = UnsafeAlignedOffset.getSize(base, offset); - int uaoSize = UnsafeAlignedOffset.getUaoSize(); - offset += uaoSize; - final UnsafeSorterSpillWriter writer = - new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); - while (numRecords > 0) { - int length = UnsafeAlignedOffset.getSize(base, offset); - writer.write(base, offset + uaoSize, length, 0); - offset += uaoSize + length + 8; - numRecords--; - } - writer.close(); - spillWriters.add(writer); + Object base = block.getBaseObject(); + long offset = block.getBaseOffset(); + int numRecords = UnsafeAlignedOffset.getSize(base, offset); + int uaoSize = UnsafeAlignedOffset.getUaoSize(); + offset += uaoSize; + final UnsafeSorterSpillWriter writer = + new UnsafeSorterSpillWriter(blockManager, 32 * 1024, writeMetrics, numRecords); + while (numRecords > 0) { + int length = UnsafeAlignedOffset.getSize(base, offset); + writer.write(base, offset + uaoSize, length, 0); + offset += uaoSize + length + 8; + numRecords--; + } + writer.close(); + spillWriters.add(writer); - dataPages.removeLast(); - released += block.size(); - freePage(block); + dataPages.removeLast(); + released += block.size(); + freePage(block); - if (released >= numBytes) { - break; - } + if (released >= numBytes) { + break; } - - return released; } + + return released; } @Override @@ -461,15 +460,12 @@ public Location lookup(Object keyBase, long keyOffset, int keyLength, int hash) public void safeLookup(Object keyBase, long keyOffset, int keyLength, Location loc, int hash) { assert(longArray != null); - if (enablePerfMetrics) { - numKeyLookups++; - } + numKeyLookups++; + int pos = hash & mask; int step = 1; while (true) { - if (enablePerfMetrics) { - numProbes++; - } + numProbes++; if (longArray.get(pos * 2) == 0) { // This is a new key. loc.with(pos, hash, false); @@ -716,7 +712,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff final long recordOffset = offset; UnsafeAlignedOffset.putSize(base, offset, klen + vlen + uaoSize); UnsafeAlignedOffset.putSize(base, offset + uaoSize, klen); - offset += (2 * uaoSize); + offset += (2L * uaoSize); Platform.copyMemory(kbase, koff, base, offset, klen); offset += klen; Platform.copyMemory(vbase, voff, base, offset, vlen); @@ -741,7 +737,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff if (numKeys >= growthThreshold && longArray.size() < MAX_CAPACITY) { try { growAndRehash(); - } catch (OutOfMemoryError oom) { + } catch (SparkOutOfMemoryError oom) { canGrowArray = false; } } @@ -757,7 +753,7 @@ public boolean append(Object kbase, long koff, int klen, Object vbase, long voff private boolean acquireNewPage(long required) { try { currentPage = allocatePage(required); - } catch (OutOfMemoryError e) { + } catch (SparkOutOfMemoryError e) { return false; } dataPages.add(currentPage); @@ -784,7 +780,7 @@ private void allocate(int capacity) { assert (capacity >= 0); capacity = Math.max((int) Math.min(MAX_CAPACITY, ByteArrayMethods.nextPowerOf2(capacity)), 64); assert (capacity <= MAX_CAPACITY); - longArray = allocateArray(capacity * 2); + longArray = allocateArray(capacity * 2L); longArray.zeroOut(); this.growthThreshold = (int) (capacity * loadFactor); @@ -858,10 +854,7 @@ public long getPeakMemoryUsedBytes() { /** * Returns the average number of probes per key lookup. */ - public double getAverageProbesPerLookup() { - if (!enablePerfMetrics) { - throw new IllegalStateException(); - } + public double getAvgHashProbeBucketListIterations() { return (1.0 * numProbes) / numKeyLookups; } diff --git a/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java new file mode 100644 index 0000000000000..57d96756c8bee --- /dev/null +++ b/core/src/main/java/org/apache/spark/util/ChildFirstURLClassLoader.java @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util; + +import java.io.IOException; +import java.net.URL; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Enumeration; + +/** + * A mutable class loader that gives preference to its own URLs over the parent class loader + * when loading classes and resources. + */ +public class ChildFirstURLClassLoader extends MutableURLClassLoader { + + static { + ClassLoader.registerAsParallelCapable(); + } + + private ParentClassLoader parent; + + public ChildFirstURLClassLoader(URL[] urls, ClassLoader parent) { + super(urls, null); + this.parent = new ParentClassLoader(parent); + } + + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + try { + return super.loadClass(name, resolve); + } catch (ClassNotFoundException cnf) { + return parent.loadClass(name, resolve); + } + } + + @Override + public Enumeration getResources(String name) throws IOException { + ArrayList urls = Collections.list(super.getResources(name)); + urls.addAll(Collections.list(parent.getResources(name))); + return Collections.enumeration(urls); + } + + @Override + public URL getResource(String name) { + URL url = super.getResource(name); + if (url != null) { + return url; + } else { + return parent.getResource(name); + } + } +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala b/core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java similarity index 64% rename from resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala rename to core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java index 9b8c98cda8da8..a7c775db179b0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/SimpleExtensionService.scala +++ b/core/src/main/java/org/apache/spark/util/MutableURLClassLoader.java @@ -15,20 +15,26 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.util; -import java.util.concurrent.atomic.AtomicBoolean +import java.net.URL; +import java.net.URLClassLoader; -private[spark] class SimpleExtensionService extends SchedulerExtensionService { +/** + * URL class loader that exposes the `addURL` method in URLClassLoader. + */ +public class MutableURLClassLoader extends URLClassLoader { - /** started flag; set in the `start()` call, stopped in `stop()`. */ - val started = new AtomicBoolean(false) + static { + ClassLoader.registerAsParallelCapable(); + } - override def start(binding: SchedulerExtensionServiceBinding): Unit = { - started.set(true) + public MutableURLClassLoader(URL[] urls, ClassLoader parent) { + super(urls, parent); } - override def stop(): Unit = { - started.set(false) + @Override + public void addURL(URL url) { + super.addURL(url); } } diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/java/org/apache/spark/util/ParentClassLoader.java similarity index 65% rename from core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala rename to core/src/main/java/org/apache/spark/util/ParentClassLoader.java index c9b7493fcdc1b..094005c58c788 100644 --- a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala +++ b/core/src/main/java/org/apache/spark/util/ParentClassLoader.java @@ -15,23 +15,28 @@ * limitations under the License. */ -package org.apache.spark.util +package org.apache.spark.util; /** * A class loader which makes some protected methods in ClassLoader accessible. */ -private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { +public class ParentClassLoader extends ClassLoader { - override def findClass(name: String): Class[_] = { - super.findClass(name) + static { + ClassLoader.registerAsParallelCapable(); } - override def loadClass(name: String): Class[_] = { - super.loadClass(name) + public ParentClassLoader(ClassLoader parent) { + super(parent); } - override def loadClass(name: String, resolve: Boolean): Class[_] = { - super.loadClass(name, resolve) + @Override + public Class findClass(String name) throws ClassNotFoundException { + return super.findClass(name); } + @Override + public Class loadClass(String name, boolean resolve) throws ClassNotFoundException { + return super.loadClass(name, resolve); + } } diff --git a/core/src/main/java/org/apache/spark/util/collection/TimSort.java b/core/src/main/java/org/apache/spark/util/collection/TimSort.java index 40b5fb7fe4b49..31428665f9634 100644 --- a/core/src/main/java/org/apache/spark/util/collection/TimSort.java +++ b/core/src/main/java/org/apache/spark/util/collection/TimSort.java @@ -409,10 +409,14 @@ private SortState(Buffer a, Comparator c, int len) { * large) stack lengths for smaller arrays. The "magic numbers" in the * computation below must be changed if MIN_MERGE is decreased. See * the MIN_MERGE declaration above for more information. + * The maximum value of 49 allows for an array up to length + * Integer.MAX_VALUE-4, if array is filled by the worst case stack size + * increasing scenario. More explanations are given in section 4 of: + * http://envisage-project.eu/wp-content/uploads/2015/02/sorting.pdf */ int stackLen = (len < 120 ? 5 : len < 1542 ? 10 : - len < 119151 ? 19 : 40); + len < 119151 ? 24 : 49); runBase = new int[stackLen]; runLen = new int[stackLen]; } @@ -439,15 +443,20 @@ private void pushRun(int runBase, int runLen) { * This method is called each time a new run is pushed onto the stack, * so the invariants are guaranteed to hold for i < stackSize upon * entry to the method. + * + * Thanks to Stijn de Gouw, Jurriaan Rot, Frank S. de Boer, + * Richard Bubel and Reiner Hahnle, this is fixed with respect to + * the analysis in "On the Worst-Case Complexity of TimSort" by + * Nicolas Auger, Vincent Jug, Cyril Nicaud, and Carine Pivoteau. */ private void mergeCollapse() { while (stackSize > 1) { int n = stackSize - 2; - if ( (n >= 1 && runLen[n-1] <= runLen[n] + runLen[n+1]) - || (n >= 2 && runLen[n-2] <= runLen[n] + runLen[n-1])) { + if (n > 0 && runLen[n-1] <= runLen[n] + runLen[n+1] || + n > 1 && runLen[n-2] <= runLen[n] + runLen[n-1]) { if (runLen[n - 1] < runLen[n + 1]) n--; - } else if (runLen[n] > runLen[n + 1]) { + } else if (n < 0 || runLen[n] > runLen[n + 1]) { break; // Invariant is established } mergeAt(n); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java index 0910db22af004..bef1bdadb27aa 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/PrefixComparators.java @@ -69,6 +69,8 @@ public static final class DoublePrefixComparator { * details see http://stereopsis.com/radix.html. */ public static long computePrefix(double value) { + // normalize -0.0 to 0.0, as they should be equal + value = value == -0.0 ? 0.0 : value; // Java's doubleToLongBits already canonicalizes all NaN values to the smallest possible // positive NaN, so there's nothing special we need to do for NaNs. long bits = Double.doubleToLongBits(value); diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java index 5056652a2420b..1b206c11d9a8e 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorter.java @@ -213,14 +213,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { spillWriters.size() > 1 ? " times" : " time"); ShuffleWriteMetrics writeMetrics = new ShuffleWriteMetrics(); - // We only write out contents of the inMemSorter if it is not empty. - if (inMemSorter.numRecords() > 0) { - final UnsafeSorterSpillWriter spillWriter = - new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, - inMemSorter.numRecords()); - spillWriters.add(spillWriter); - spillIterator(inMemSorter.getSortedIterator(), spillWriter); - } + + final UnsafeSorterSpillWriter spillWriter = + new UnsafeSorterSpillWriter(blockManager, fileBufferSizeBytes, writeMetrics, + inMemSorter.numRecords()); + spillWriters.add(spillWriter); + spillIterator(inMemSorter.getSortedIterator(), spillWriter); final long spillSize = freeMemory(); // Note that this is more-or-less going to be a multiple of the page size, so wasted space in @@ -575,19 +573,31 @@ public boolean hasNext() { @Override public void loadNext() throws IOException { - synchronized (this) { - loaded = true; - if (nextUpstream != null) { - // Just consumed the last record from in memory iterator - if (lastPage != null) { - freePage(lastPage); - lastPage = null; + MemoryBlock pageToFree = null; + try { + synchronized (this) { + loaded = true; + if (nextUpstream != null) { + // Just consumed the last record from in memory iterator + if(lastPage != null) { + // Do not free the page here, while we are locking `SpillableIterator`. The `freePage` + // method locks the `TaskMemoryManager`, and it's a bad idea to lock 2 objects in + // sequence. We may hit dead lock if another thread locks `TaskMemoryManager` and + // `SpillableIterator` in sequence, which may happen in + // `TaskMemoryManager.acquireExecutionMemory`. + pageToFree = lastPage; + lastPage = null; + } + upstream = nextUpstream; + nextUpstream = null; } - upstream = nextUpstream; - nextUpstream = null; + numRecords--; + upstream.loadNext(); + } + } finally { + if (pageToFree != null) { + freePage(pageToFree); } - numRecords--; - upstream.loadNext(); } } diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java index 75690ae264838..1a9453a8b3e80 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorter.java @@ -214,7 +214,9 @@ public boolean hasSpaceForAnotherRecord() { public void expandPointerArray(LongArray newArray) { if (newArray.size() < array.size()) { + // checkstyle.off: RegexpSinglelineJava throw new SparkOutOfMemoryError("Not enough memory to grow pointer array"); + // checkstyle.on: RegexpSinglelineJava } Platform.copyMemory( array.getBaseObject(), diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java index fb179d07edebc..a524c4790407d 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillReader.java @@ -21,13 +21,13 @@ import com.google.common.io.Closeables; import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; +import org.apache.spark.internal.config.ConfigEntry; import org.apache.spark.io.NioBufferedFileInputStream; import org.apache.spark.io.ReadAheadInputStream; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.storage.BlockId; import org.apache.spark.unsafe.Platform; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; import java.io.*; @@ -36,9 +36,7 @@ * of the file format). */ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implements Closeable { - private static final Logger logger = LoggerFactory.getLogger(UnsafeSorterSpillReader.class); - private static final int DEFAULT_BUFFER_SIZE_BYTES = 1024 * 1024; // 1 MB - private static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb + public static final int MAX_BUFFER_SIZE_BYTES = 16777216; // 16 mb private InputStream in; private DataInputStream din; @@ -51,7 +49,6 @@ public final class UnsafeSorterSpillReader extends UnsafeSorterIterator implemen private byte[] arr = new byte[1024 * 1024]; private Object baseObject = arr; - private final long baseOffset = Platform.BYTE_ARRAY_OFFSET; private final TaskContext taskContext = TaskContext.get(); public UnsafeSorterSpillReader( @@ -59,28 +56,23 @@ public UnsafeSorterSpillReader( File file, BlockId blockId) throws IOException { assert (file.length() > 0); - long bufferSizeBytes = - SparkEnv.get() == null ? - DEFAULT_BUFFER_SIZE_BYTES: - SparkEnv.get().conf().getSizeAsBytes("spark.unsafe.sorter.spill.reader.buffer.size", - DEFAULT_BUFFER_SIZE_BYTES); - if (bufferSizeBytes > MAX_BUFFER_SIZE_BYTES || bufferSizeBytes < DEFAULT_BUFFER_SIZE_BYTES) { - // fall back to a sane default value - logger.warn("Value of config \"spark.unsafe.sorter.spill.reader.buffer.size\" = {} not in " + - "allowed range [{}, {}). Falling back to default value : {} bytes", bufferSizeBytes, - DEFAULT_BUFFER_SIZE_BYTES, MAX_BUFFER_SIZE_BYTES, DEFAULT_BUFFER_SIZE_BYTES); - bufferSizeBytes = DEFAULT_BUFFER_SIZE_BYTES; - } + final ConfigEntry bufferSizeConfigEntry = + package$.MODULE$.UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE(); + // This value must be less than or equal to MAX_BUFFER_SIZE_BYTES. Cast to int is always safe. + final int DEFAULT_BUFFER_SIZE_BYTES = + ((Long) bufferSizeConfigEntry.defaultValue().get()).intValue(); + int bufferSizeBytes = SparkEnv.get() == null ? DEFAULT_BUFFER_SIZE_BYTES : + ((Long) SparkEnv.get().conf().get(bufferSizeConfigEntry)).intValue(); - final boolean readAheadEnabled = SparkEnv.get() != null && - SparkEnv.get().conf().getBoolean("spark.unsafe.sorter.spill.read.ahead.enabled", true); + final boolean readAheadEnabled = SparkEnv.get() != null && (boolean)SparkEnv.get().conf().get( + package$.MODULE$.UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED()); final InputStream bs = - new NioBufferedFileInputStream(file, (int) bufferSizeBytes); + new NioBufferedFileInputStream(file, bufferSizeBytes); try { if (readAheadEnabled) { this.in = new ReadAheadInputStream(serializerManager.wrapStream(blockId, bs), - (int) bufferSizeBytes); + bufferSizeBytes); } else { this.in = serializerManager.wrapStream(blockId, bs); } @@ -132,7 +124,7 @@ public Object getBaseObject() { @Override public long getBaseOffset() { - return baseOffset; + return Platform.BYTE_ARRAY_OFFSET; } @Override diff --git a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java index 9399024f01783..c1d71a23b1dbe 100644 --- a/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java +++ b/core/src/main/java/org/apache/spark/util/collection/unsafe/sort/UnsafeSorterSpillWriter.java @@ -42,7 +42,10 @@ public final class UnsafeSorterSpillWriter { private final SparkConf conf = new SparkConf(); - /** The buffer size to use when writing the sorted records to an on-disk file */ + /** + * The buffer size to use when writing the sorted records to an on-disk file, and + * this space used by prefix + len + recordLength must be greater than 4 + 8 bytes. + */ private final int diskWriteBufferSize = (int) (long) conf.get(package$.MODULE$.SHUFFLE_DISK_WRITE_BUFFER_SIZE()); diff --git a/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider b/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider new file mode 100644 index 0000000000000..c1f2060cabcff --- /dev/null +++ b/core/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider @@ -0,0 +1,2 @@ +org.apache.spark.deploy.security.HadoopFSDelegationTokenProvider +org.apache.spark.deploy.security.HBaseDelegationTokenProvider diff --git a/core/src/main/resources/org/apache/spark/log4j-defaults.properties b/core/src/main/resources/org/apache/spark/log4j-defaults.properties index 277010015072a..eb12848900b58 100644 --- a/core/src/main/resources/org/apache/spark/log4j-defaults.properties +++ b/core/src/main/resources/org/apache/spark/log4j-defaults.properties @@ -28,8 +28,8 @@ log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: log4j.logger.org.apache.spark.repl.Main=WARN # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=WARN -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.sparkproject.jetty=WARN +log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js index 3c8ddddf07b1e..3798dc47529af 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js +++ b/core/src/main/resources/org/apache/spark/ui/static/additional-metrics.js @@ -83,8 +83,8 @@ $(function() { $(this).parent().find('input[type="checkbox"]').trigger('click'); }); - // Trigger a double click on the span to show full job description. - $(".description-input").dblclick(function() { - $(this).removeClass("description-input").addClass("description-input-full"); + // Show/hide full job description on click event. + $(".description-input").click(function() { + $(this).toggleClass("description-input-full"); }); }); diff --git a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html index 5c91304e49fd7..f2c17aef097a4 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html +++ b/core/src/main/resources/org/apache/spark/ui/static/executorspage-template.html @@ -16,10 +16,10 @@ --> diff --git a/core/src/main/resources/org/apache/spark/ui/static/table.js b/core/src/main/resources/org/apache/spark/ui/static/table.js index 0315ebf5c48a9..fd258d5ee70d9 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/table.js +++ b/core/src/main/resources/org/apache/spark/ui/static/table.js @@ -89,7 +89,7 @@ function onSearchStringChange() { if($(this).attr('id') && $(this).attr('id').match(/thread_[0-9]+_tr/) ) { var children = $(this).children() var found = false - for (i = 0; i < children.length; i++) { + for (var i = 0; i < children.length; i++) { if (children.eq(i).text().toLowerCase().indexOf(searchString) >= 0) { found = true } 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 3bf3e8bfa1f31..10bceae2fbdda 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 @@ -98,12 +98,12 @@ rect.getting-result-time-proportion { cursor: pointer; } -.vis-timeline .vis-item.stage.succeeded { +.vis-timeline .vis-item.stage.complete { background-color: #A0DFFF; border-color: #3EC0FF; } -.vis-timeline .vis-item.stage.succeeded.vis-selected { +.vis-timeline .vis-item.stage.complete.vis-selected { background-color: #A0DFFF; border-color: #3EC0FF; z-index: auto; @@ -130,12 +130,12 @@ rect.getting-result-time-proportion { stroke: #FF4D6D; } -.vis-timeline .vis-item.stage.running { +.vis-timeline .vis-item.stage.active { background-color: #A2FCC0; border-color: #36F572; } -.vis-timeline .vis-item.stage.running.vis-selected { +.vis-timeline .vis-item.stage.active.vis-selected { background-color: #A2FCC0; border-color: #36F572; z-index: auto; diff --git a/core/src/main/resources/org/apache/spark/ui/static/utils.js b/core/src/main/resources/org/apache/spark/ui/static/utils.js index 4f63f6413d6de..6fc34a9e1f7ea 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/utils.js +++ b/core/src/main/resources/org/apache/spark/ui/static/utils.js @@ -18,7 +18,7 @@ // this function works exactly the same as UIUtils.formatDuration function formatDuration(milliseconds) { if (milliseconds < 100) { - return milliseconds + " ms"; + return parseInt(milliseconds).toFixed(1) + " ms"; } var seconds = milliseconds * 1.0 / 1000; if (seconds < 1) { @@ -40,9 +40,9 @@ function formatDuration(milliseconds) { function formatBytes(bytes, type) { if (type !== 'display') return bytes; if (bytes == 0) return '0.0 B'; - var k = 1000; + var k = 1024; var dm = 1; - var sizes = ['B', 'KB', 'MB', 'GB', 'TB', 'PB', 'EB', 'ZB', 'YB']; + var sizes = ['B', 'KiB', 'MiB', 'GiB', 'TiB', 'PiB', 'EiB', 'ZiB', 'YiB']; var i = Math.floor(Math.log(bytes) / Math.log(k)); return parseFloat((bytes / Math.pow(k, i)).toFixed(dm)) + ' ' + sizes[i]; } @@ -74,3 +74,114 @@ function getTimeZone() { return new Date().toString().match(/\((.*)\)/)[1]; } } + +function formatLogsCells(execLogs, type) { + if (type !== 'display') return Object.keys(execLogs); + if (!execLogs) return; + var result = ''; + $.each(execLogs, function (logName, logUrl) { + result += '' + }); + return result; +} + +function getStandAloneAppId(cb) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + cb(appId); + return; + } + // Looks like Web UI is running in standalone mode + // Let's get application-id using REST End Point + $.getJSON(location.origin + "/api/v1/applications", function(response, status, jqXHR) { + if (response && response.length > 0) { + var appId = response[0].id; + cb(appId); + return; + } + }); +} + +// This function is a helper function for sorting in datatable. +// When the data is in duration (e.g. 12ms 2s 2min 2h ) +// It will convert the string into integer for correct ordering +function ConvertDurationString(data) { + data = data.toString(); + var units = data.replace(/[\d\.]/g, '' ) + .replace(' ', '') + .toLowerCase(); + var multiplier = 1; + + switch(units) { + case 's': + multiplier = 1000; + break; + case 'min': + multiplier = 600000; + break; + case 'h': + multiplier = 3600000; + break; + default: + break; + } + return parseFloat(data) * multiplier; +} + +function createTemplateURI(appId, templateName) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var baseURI = words.slice(0, ind + 1).join('/') + '/' + appId + '/static/' + templateName + '-template.html'; + return baseURI; + } + ind = words.indexOf("history"); + if(ind > 0) { + var baseURI = words.slice(0, ind).join('/') + '/static/' + templateName + '-template.html'; + return baseURI; + } + return location.origin + "/static/" + templateName + "-template.html"; +} + +function setDataTableDefaults() { + $.extend($.fn.dataTable.defaults, { + stateSave: true, + lengthMenu: [[20, 40, 60, 100, -1], [20, 40, 60, 100, "All"]], + pageLength: 20 + }); +} + +function formatDate(date) { + if (date <= 0) return "-"; + else return date.split(".")[0].replace("T", " "); +} + +function createRESTEndPointForExecutorsPage(appId) { + var words = document.baseURI.split('/'); + var ind = words.indexOf("proxy"); + if (ind > 0) { + var appId = words[ind + 1]; + var newBaseURI = words.slice(0, ind + 2).join('/'); + return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; + } + ind = words.indexOf("history"); + if (ind > 0) { + var appId = words[ind + 1]; + var attemptId = words[ind + 2]; + var newBaseURI = words.slice(0, ind).join('/'); + if (isNaN(attemptId)) { + return newBaseURI + "/api/v1/applications/" + appId + "/allexecutors"; + } else { + return newBaseURI + "/api/v1/applications/" + appId + "/" + attemptId + "/allexecutors"; + } + } + return location.origin + "/api/v1/applications/" + appId + "/allexecutors"; +} diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui-dataTables.css b/core/src/main/resources/org/apache/spark/ui/static/webui-dataTables.css new file mode 100644 index 0000000000000..f6b4abed21e0d --- /dev/null +++ b/core/src/main/resources/org/apache/spark/ui/static/webui-dataTables.css @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +table.dataTable thead .sorting_asc { background: url('images/sort_asc.png') no-repeat bottom right; } + +table.dataTable thead .sorting_desc { background: url('images/sort_desc.png') no-repeat bottom right; } \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index 4b060b0f4e53e..fe5bb25687af1 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -88,6 +88,10 @@ a.kill-link { float: right; } +a.name-link { + word-wrap: break-word; +} + span.expand-details { font-size: 10pt; cursor: pointer; @@ -256,4 +260,105 @@ a.expandbutton { .paginate_button.active > a { color: #999999; text-decoration: underline; +} + +.title-table { + clear: left; + display: inline-block; +} + +.table-dataTable { + width: 100%; +} + +.container-fluid-div { + width: 200px; +} + +.scheduler-delay-checkbox-div { + width: 120px; +} + +.task-deserialization-time-checkbox-div { + width: 175px; +} + +.shuffle-read-blocked-time-checkbox-div { + width: 187px; +} + +.shuffle-remote-reads-checkbox-div { + width: 157px; +} + +.result-serialization-time-checkbox-div { + width: 171px; +} + +.getting-result-time-checkbox-div { + width: 141px; +} + +.peak-execution-memory-checkbox-div { + width: 170px; +} + +#active-tasks-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#active-tasks-table th:first-child { + border-left: 1px solid #dddddd; +} + +#accumulator-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#accumulator-table th:first-child { + border-left: 1px solid #dddddd; +} + +#summary-executor-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#summary-executor-table th:first-child { + border-left: 1px solid #dddddd; +} + +#summary-metrics-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#summary-metrics-table th:first-child { + border-left: 1px solid #dddddd; +} + +#summary-execs-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#summary-execs-table th:first-child { + border-left: 1px solid #dddddd; +} + +#active-executors-table th { + border-top: 1px solid #dddddd; + border-bottom: 1px solid #dddddd; + border-right: 1px solid #dddddd; +} + +#active-executors-table th:first-child { + border-left: 1px solid #dddddd; } \ No newline at end of file diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.js b/core/src/main/resources/org/apache/spark/ui/static/webui.js index b1254e08fa504..89622106ff1f0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.js +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.js @@ -33,7 +33,7 @@ function collapseTable(thisName, table){ var status = window.localStorage.getItem(thisName) == "true"; status = !status; - thisClass = '.' + thisName + var thisClass = '.' + thisName // Expand the list of additional metrics. var tableDiv = $(thisClass).parent().find('.' + table); @@ -63,6 +63,7 @@ $(function() { collapseTablePageLoad('collapse-aggregated-finishedDrivers','aggregated-finishedDrivers'); collapseTablePageLoad('collapse-aggregated-runtimeInformation','aggregated-runtimeInformation'); collapseTablePageLoad('collapse-aggregated-sparkProperties','aggregated-sparkProperties'); + collapseTablePageLoad('collapse-aggregated-hadoopProperties','aggregated-hadoopProperties'); collapseTablePageLoad('collapse-aggregated-systemProperties','aggregated-systemProperties'); collapseTablePageLoad('collapse-aggregated-classpathEntries','aggregated-classpathEntries'); collapseTablePageLoad('collapse-aggregated-activeJobs','aggregated-activeJobs'); diff --git a/core/src/main/scala/org/apache/spark/Accumulable.scala b/core/src/main/scala/org/apache/spark/Accumulable.scala deleted file mode 100644 index 3092074232d18..0000000000000 --- a/core/src/main/scala/org/apache/spark/Accumulable.scala +++ /dev/null @@ -1,226 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -import java.io.Serializable - -import scala.collection.generic.Growable -import scala.reflect.ClassTag - -import org.apache.spark.scheduler.AccumulableInfo -import org.apache.spark.serializer.JavaSerializer -import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, LegacyAccumulatorWrapper} - - -/** - * A data type that can be accumulated, i.e. has a commutative and associative "add" operation, - * but where the result type, `R`, may be different from the element type being added, `T`. - * - * You must define how to add data, and how to merge two of these together. For some data types, - * such as a counter, these might be the same operation. In that case, you can use the simpler - * [[org.apache.spark.Accumulator]]. They won't always be the same, though -- e.g., imagine you are - * accumulating a set. You will add items to the set, and you will union two sets together. - * - * Operations are not thread-safe. - * - * @param id ID of this accumulator; for internal use only. - * @param initialValue initial value of accumulator - * @param param helper object defining how to add elements of type `R` and `T` - * @param name human-readable name for use in Spark's web UI - * @param countFailedValues whether to accumulate values from failed tasks. This is set to true - * for system and time metrics like serialization time or bytes spilled, - * and false for things with absolute values like number of input rows. - * This should be used for internal metrics only. - * @tparam R the full accumulated data (result type) - * @tparam T partial data that can be added in - */ -@deprecated("use AccumulatorV2", "2.0.0") -class Accumulable[R, T] private ( - val id: Long, - // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile - @transient private val initialValue: R, - param: AccumulableParam[R, T], - val name: Option[String], - private[spark] val countFailedValues: Boolean) - extends Serializable { - - private[spark] def this( - initialValue: R, - param: AccumulableParam[R, T], - name: Option[String], - countFailedValues: Boolean) = { - this(AccumulatorContext.newId(), initialValue, param, name, countFailedValues) - } - - private[spark] def this(initialValue: R, param: AccumulableParam[R, T], name: Option[String]) = { - this(initialValue, param, name, false /* countFailedValues */) - } - - def this(initialValue: R, param: AccumulableParam[R, T]) = this(initialValue, param, None) - - val zero = param.zero(initialValue) - private[spark] val newAcc = new LegacyAccumulatorWrapper(initialValue, param) - newAcc.metadata = AccumulatorMetadata(id, name, countFailedValues) - // Register the new accumulator in ctor, to follow the previous behaviour. - AccumulatorContext.register(newAcc) - - /** - * Add more data to this accumulator / accumulable - * @param term the data to add - */ - def += (term: T) { newAcc.add(term) } - - /** - * Add more data to this accumulator / accumulable - * @param term the data to add - */ - def add(term: T) { newAcc.add(term) } - - /** - * Merge two accumulable objects together - * - * Normally, a user will not want to use this version, but will instead call `+=`. - * @param term the other `R` that will get merged with this - */ - def ++= (term: R) { newAcc._value = param.addInPlace(newAcc._value, term) } - - /** - * Merge two accumulable objects together - * - * Normally, a user will not want to use this version, but will instead call `add`. - * @param term the other `R` that will get merged with this - */ - def merge(term: R) { newAcc._value = param.addInPlace(newAcc._value, term) } - - /** - * Access the accumulator's current value; only allowed on driver. - */ - def value: R = { - if (newAcc.isAtDriverSide) { - newAcc.value - } else { - throw new UnsupportedOperationException("Can't read accumulator value in task") - } - } - - /** - * Get the current value of this accumulator from within a task. - * - * This is NOT the global value of the accumulator. To get the global value after a - * completed operation on the dataset, call `value`. - * - * The typical use of this method is to directly mutate the local value, eg., to add - * an element to a Set. - */ - def localValue: R = newAcc.value - - /** - * Set the accumulator's value; only allowed on driver. - */ - def value_= (newValue: R) { - if (newAcc.isAtDriverSide) { - newAcc._value = newValue - } else { - throw new UnsupportedOperationException("Can't assign accumulator value in task") - } - } - - /** - * Set the accumulator's value. For internal use only. - */ - def setValue(newValue: R): Unit = { newAcc._value = newValue } - - /** - * Set the accumulator's value. For internal use only. - */ - private[spark] def setValueAny(newValue: Any): Unit = { setValue(newValue.asInstanceOf[R]) } - - /** - * Create an [[AccumulableInfo]] representation of this [[Accumulable]] with the provided values. - */ - private[spark] def toInfo(update: Option[Any], value: Option[Any]): AccumulableInfo = { - val isInternal = name.exists(_.startsWith(InternalAccumulator.METRICS_PREFIX)) - new AccumulableInfo(id, name, update, value, isInternal, countFailedValues) - } - - override def toString: String = if (newAcc._value == null) "null" else newAcc._value.toString -} - - -/** - * Helper object defining how to accumulate values of a particular type. An implicit - * AccumulableParam needs to be available when you create [[Accumulable]]s of a specific type. - * - * @tparam R the full accumulated data (result type) - * @tparam T partial data that can be added in - */ -@deprecated("use AccumulatorV2", "2.0.0") -trait AccumulableParam[R, T] extends Serializable { - /** - * Add additional data to the accumulator value. Is allowed to modify and return `r` - * for efficiency (to avoid allocating objects). - * - * @param r the current value of the accumulator - * @param t the data to be added to the accumulator - * @return the new value of the accumulator - */ - def addAccumulator(r: R, t: T): R - - /** - * Merge two accumulated values together. Is allowed to modify and return the first value - * for efficiency (to avoid allocating objects). - * - * @param r1 one set of accumulated data - * @param r2 another set of accumulated data - * @return both data sets merged together - */ - def addInPlace(r1: R, r2: R): R - - /** - * Return the "zero" (identity) value for an accumulator type, given its initial value. For - * example, if R was a vector of N dimensions, this would return a vector of N zeroes. - */ - def zero(initialValue: R): R -} - - -@deprecated("use AccumulatorV2", "2.0.0") -private[spark] class -GrowableAccumulableParam[R : ClassTag, T] - (implicit rg: R => Growable[T] with TraversableOnce[T] with Serializable) - extends AccumulableParam[R, T] { - - def addAccumulator(growable: R, elem: T): R = { - growable += elem - growable - } - - def addInPlace(t1: R, t2: R): R = { - t1 ++= t2 - t1 - } - - def zero(initialValue: R): R = { - // We need to clone initialValue, but it's hard to specify that R should also be Cloneable. - // Instead we'll serialize it to a buffer and load it back. - val ser = new JavaSerializer(new SparkConf(false)).newInstance() - val copy = ser.deserialize[R](ser.serialize(initialValue)) - copy.clear() // In case it contained stuff - copy - } -} diff --git a/core/src/main/scala/org/apache/spark/Accumulator.scala b/core/src/main/scala/org/apache/spark/Accumulator.scala deleted file mode 100644 index 9d5fbefc824ad..0000000000000 --- a/core/src/main/scala/org/apache/spark/Accumulator.scala +++ /dev/null @@ -1,117 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark - -/** - * A simpler value of [[Accumulable]] where the result type being accumulated is the same - * as the types of elements being merged, i.e. variables that are only "added" to through an - * associative and commutative operation and can therefore be efficiently supported in parallel. - * They can be used to implement counters (as in MapReduce) or sums. Spark natively supports - * accumulators of numeric value types, and programmers can add support for new types. - * - * An accumulator is created from an initial value `v` by calling `SparkContext.accumulator`. - * Tasks running on the cluster can then add to it using the `+=` operator. - * However, they cannot read its value. Only the driver program can read the accumulator's value, - * using its [[#value]] method. - * - * The interpreter session below shows an accumulator being used to add up the elements of an array: - * - * {{{ - * scala> val accum = sc.accumulator(0) - * accum: org.apache.spark.Accumulator[Int] = 0 - * - * scala> sc.parallelize(Array(1, 2, 3, 4)).foreach(x => accum += x) - * ... - * 10/09/29 18:41:08 INFO SparkContext: Tasks finished in 0.317106 s - * - * scala> accum.value - * res2: Int = 10 - * }}} - * - * @param initialValue initial value of accumulator - * @param param helper object defining how to add elements of type `T` - * @param name human-readable name associated with this accumulator - * @param countFailedValues whether to accumulate values from failed tasks - * @tparam T result type -*/ -@deprecated("use AccumulatorV2", "2.0.0") -class Accumulator[T] private[spark] ( - // SI-8813: This must explicitly be a private val, or else scala 2.11 doesn't compile - @transient private val initialValue: T, - param: AccumulatorParam[T], - name: Option[String] = None, - countFailedValues: Boolean = false) - extends Accumulable[T, T](initialValue, param, name, countFailedValues) - - -/** - * A simpler version of [[org.apache.spark.AccumulableParam]] where the only data type you can add - * in is the same type as the accumulated value. An implicit AccumulatorParam object needs to be - * available when you create Accumulators of a specific type. - * - * @tparam T type of value to accumulate - */ -@deprecated("use AccumulatorV2", "2.0.0") -trait AccumulatorParam[T] extends AccumulableParam[T, T] { - def addAccumulator(t1: T, t2: T): T = { - addInPlace(t1, t2) - } -} - - -@deprecated("use AccumulatorV2", "2.0.0") -object AccumulatorParam { - - // The following implicit objects were in SparkContext before 1.2 and users had to - // `import SparkContext._` to enable them. Now we move them here to make the compiler find - // them automatically. However, as there are duplicate codes in SparkContext for backward - // compatibility, please update them accordingly if you modify the following implicit objects. - - @deprecated("use AccumulatorV2", "2.0.0") - implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { - def addInPlace(t1: Double, t2: Double): Double = t1 + t2 - def zero(initialValue: Double): Double = 0.0 - } - - @deprecated("use AccumulatorV2", "2.0.0") - implicit object IntAccumulatorParam extends AccumulatorParam[Int] { - def addInPlace(t1: Int, t2: Int): Int = t1 + t2 - def zero(initialValue: Int): Int = 0 - } - - @deprecated("use AccumulatorV2", "2.0.0") - implicit object LongAccumulatorParam extends AccumulatorParam[Long] { - def addInPlace(t1: Long, t2: Long): Long = t1 + t2 - def zero(initialValue: Long): Long = 0L - } - - @deprecated("use AccumulatorV2", "2.0.0") - implicit object FloatAccumulatorParam extends AccumulatorParam[Float] { - def addInPlace(t1: Float, t2: Float): Float = t1 + t2 - def zero(initialValue: Float): Float = 0f - } - - // Note: when merging values, this param just adopts the newer value. This is used only - // internally for things that shouldn't really be accumulated across tasks, like input - // read method, which should be the same across all tasks in the same stage. - @deprecated("use AccumulatorV2", "2.0.0") - private[spark] object StringAccumulatorParam extends AccumulatorParam[String] { - def addInPlace(t1: String, t2: String): String = t2 - def zero(initialValue: String): String = "" - } -} diff --git a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala index 6439ca5db06e9..9f59295059d30 100644 --- a/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala +++ b/core/src/main/scala/org/apache/spark/BarrierCoordinator.scala @@ -19,7 +19,7 @@ package org.apache.spark import java.util.{Timer, TimerTask} import java.util.concurrent.ConcurrentHashMap -import java.util.function.{Consumer, Function} +import java.util.function.Consumer import scala.collection.mutable.ArrayBuffer @@ -202,10 +202,8 @@ private[spark] class BarrierCoordinator( case request @ RequestToSync(numTasks, stageId, stageAttemptId, _, _) => // Get or init the ContextBarrierState correspond to the stage attempt. val barrierId = ContextBarrierId(stageId, stageAttemptId) - states.computeIfAbsent(barrierId, new Function[ContextBarrierId, ContextBarrierState] { - override def apply(key: ContextBarrierId): ContextBarrierState = - new ContextBarrierState(key, numTasks) - }) + states.computeIfAbsent(barrierId, + (key: ContextBarrierId) => new ContextBarrierState(key, numTasks)) val barrierState = states.get(barrierId) barrierState.handleRequest(context, request) diff --git a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala index 90a5c4130f799..c393df8f02226 100644 --- a/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala +++ b/core/src/main/scala/org/apache/spark/BarrierTaskContext.scala @@ -20,13 +20,13 @@ package org.apache.spark import java.util.{Properties, Timer, TimerTask} import scala.concurrent.duration._ -import scala.language.postfixOps import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.{RpcEndpointRef, RpcTimeout} import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util._ @@ -41,14 +41,14 @@ import org.apache.spark.util._ class BarrierTaskContext private[spark] ( taskContext: TaskContext) extends TaskContext with Logging { + import BarrierTaskContext._ + // Find the driver side RPCEndpointRef of the coordinator that handles all the barrier() calls. private val barrierCoordinator: RpcEndpointRef = { val env = SparkEnv.get RpcUtils.makeDriverRef("barrierSync", env.conf, env.rpcEnv) } - private val timer = new Timer("Barrier task timer for barrier() calls.") - // Local barrierEpoch that identify a barrier() call from current task, it shall be identical // with the driver side epoch. private var barrierEpoch = 0 @@ -109,8 +109,8 @@ class BarrierTaskContext private[spark] ( override def run(): Unit = { logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) waiting " + s"under the global sync since $startTime, has been waiting for " + - s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch " + - s"is $barrierEpoch.") + s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " + + s"current barrier epoch is $barrierEpoch.") } } // Log the update of global sync every 60 seconds. @@ -122,18 +122,18 @@ class BarrierTaskContext private[spark] ( barrierEpoch), // Set a fixed timeout for RPC here, so users shall get a SparkException thrown by // BarrierCoordinator on timeout, instead of RPCTimeoutException from the RPC framework. - timeout = new RpcTimeout(31536000 /* = 3600 * 24 * 365 */ seconds, "barrierTimeout")) + timeout = new RpcTimeout(365.days, "barrierTimeout")) barrierEpoch += 1 logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) finished " + "global sync successfully, waited for " + - s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch is " + - s"$barrierEpoch.") + s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " + + s"current barrier epoch is $barrierEpoch.") } catch { case e: SparkException => logInfo(s"Task $taskAttemptId from Stage $stageId(Attempt $stageAttemptNumber) failed " + "to perform global sync, waited for " + - s"${(System.currentTimeMillis() - startTime) / 1000} seconds, current barrier epoch " + - s"is $barrierEpoch.") + s"${MILLISECONDS.toSeconds(System.currentTimeMillis() - startTime)} seconds, " + + s"current barrier epoch is $barrierEpoch.") throw e } finally { timerTask.cancel() @@ -158,8 +158,6 @@ class BarrierTaskContext private[spark] ( override def isInterrupted(): Boolean = taskContext.isInterrupted() - override def isRunningLocally(): Boolean = taskContext.isRunningLocally() - override def addTaskCompletionListener(listener: TaskCompletionListener): this.type = { taskContext.addTaskCompletionListener(listener) this @@ -188,6 +186,8 @@ class BarrierTaskContext private[spark] ( taskContext.getMetricsSources(sourceName) } + override def resources(): Map[String, ResourceInformation] = taskContext.resources() + override private[spark] def killTaskIfInterrupted(): Unit = taskContext.killTaskIfInterrupted() override private[spark] def getKillReason(): Option[String] = taskContext.getKillReason() @@ -234,4 +234,7 @@ object BarrierTaskContext { @Experimental @Since("2.4.0") def get(): BarrierTaskContext = TaskContext.get().asInstanceOf[BarrierTaskContext] + + private val timer = new Timer("Barrier task timer for barrier() calls.") + } diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala index 4d884dec07916..a111a60d1d024 100644 --- a/core/src/main/scala/org/apache/spark/ContextCleaner.scala +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -25,6 +25,7 @@ import scala.collection.JavaConverters._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.util.{AccumulatorContext, AccumulatorV2, ThreadUtils, Utils} @@ -83,8 +84,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { * 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. */ - private val periodicGCInterval = - sc.conf.getTimeAsSeconds("spark.cleaner.periodicGC.interval", "30min") + private val periodicGCInterval = sc.conf.get(CLEANER_PERIODIC_GC_INTERVAL) /** * Whether the cleaning thread will block on cleanup tasks (other than shuffle, which @@ -96,8 +96,7 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { * for instance, when the driver performs a GC and cleans up all broadcast blocks that are no * longer in scope. */ - private val blockOnCleanupTasks = sc.conf.getBoolean( - "spark.cleaner.referenceTracking.blocking", true) + private val blockOnCleanupTasks = sc.conf.get(CLEANER_REFERENCE_TRACKING_BLOCKING) /** * Whether the cleaning thread will block on shuffle cleanup tasks. @@ -109,8 +108,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { * until the real RPC issue (referred to in the comment above `blockOnCleanupTasks`) is * resolved. */ - private val blockOnShuffleCleanupTasks = sc.conf.getBoolean( - "spark.cleaner.referenceTracking.blocking.shuffle", false) + private val blockOnShuffleCleanupTasks = + sc.conf.get(CLEANER_REFERENCE_TRACKING_BLOCKING_SHUFFLE) @volatile private var stopped = false @@ -124,9 +123,8 @@ private[spark] class ContextCleaner(sc: SparkContext) extends Logging { cleaningThread.setDaemon(true) cleaningThread.setName("Spark Context Cleaner") cleaningThread.start() - periodicGCService.scheduleAtFixedRate(new Runnable { - override def run(): Unit = System.gc() - }, periodicGCInterval, periodicGCInterval, TimeUnit.SECONDS) + periodicGCService.scheduleAtFixedRate(() => System.gc(), + periodicGCInterval, periodicGCInterval, TimeUnit.SECONDS) } /** diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 9ea6d2fa2fd95..fb051a8c0db8e 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer -import org.apache.spark.shuffle.ShuffleHandle +import org.apache.spark.shuffle.{ShuffleHandle, ShuffleWriteProcessor} /** * :: DeveloperApi :: @@ -65,6 +65,7 @@ abstract class NarrowDependency[T](_rdd: RDD[T]) extends Dependency[T] { * @param keyOrdering key ordering for RDD's shuffles * @param aggregator map/reduce-side aggregator for RDD's shuffle * @param mapSideCombine whether to perform partial aggregation (also known as map-side combine) + * @param shuffleWriterProcessor the processor to control the write behavior in ShuffleMapTask */ @DeveloperApi class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( @@ -73,7 +74,8 @@ class ShuffleDependency[K: ClassTag, V: ClassTag, C: ClassTag]( val serializer: Serializer = SparkEnv.get.serializer, val keyOrdering: Option[Ordering[K]] = None, val aggregator: Option[Aggregator[K, V, C]] = None, - val mapSideCombine: Boolean = false) + val mapSideCombine: Boolean = false, + val shuffleWriterProcessor: ShuffleWriteProcessor = new ShuffleWriteProcessor) extends Dependency[Product2[K, V]] { if (mapSideCombine) { diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala index 63d87b4cd385c..cb965cb180207 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationClient.scala @@ -23,10 +23,17 @@ package org.apache.spark */ private[spark] trait ExecutorAllocationClient { - /** Get the list of currently active executors */ private[spark] def getExecutorIds(): Seq[String] + /** + * Whether an executor is active. An executor is active when it can be used to execute tasks + * for jobs submitted by the application. + * + * @return whether the executor with the given ID is currently active. + */ + def isExecutorActive(id: String): Boolean + /** * Update the cluster manager on our scheduling needs. Three bits of information are included * to help it make decisions. diff --git a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala index c3e5b96a55884..5114cf70e3f26 100644 --- a/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala +++ b/core/src/main/scala/org/apache/spark/ExecutorAllocationManager.scala @@ -27,9 +27,10 @@ import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL import org.apache.spark.metrics.source.Source import org.apache.spark.scheduler._ -import org.apache.spark.storage.BlockManagerMaster +import org.apache.spark.scheduler.dynalloc.ExecutorMonitor import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} /** @@ -57,7 +58,8 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * a long time to ramp up under heavy workloads. * * The remove policy is simpler: If an executor has been idle for K seconds, meaning it has not - * been scheduled to run any tasks, then it is removed. + * been scheduled to run any tasks, then it is removed. Note that an executor caching any data + * blocks will be removed if it has been idle for more than L seconds. * * There is no retry logic in either case because we make the assumption that the cluster manager * will eventually fulfill all requests it receives asynchronously. @@ -81,13 +83,19 @@ import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} * This is used only after the initial backlog timeout is exceeded * * spark.dynamicAllocation.executorIdleTimeout (K) - - * If an executor has been idle for this duration, remove it + * If an executor without caching any data blocks has been idle for this duration, remove it + * + * spark.dynamicAllocation.cachedExecutorIdleTimeout (L) - + * If an executor with caching data blocks has been idle for more than this duration, + * the executor will be removed + * */ private[spark] class ExecutorAllocationManager( client: ExecutorAllocationClient, listenerBus: LiveListenerBus, conf: SparkConf, - blockManagerMaster: BlockManagerMaster) + cleaner: Option[ContextCleaner] = None, + clock: Clock = new SystemClock()) extends Logging { allocationManager => @@ -100,28 +108,20 @@ private[spark] class ExecutorAllocationManager( private val initialNumExecutors = Utils.getDynamicAllocationInitialExecutors(conf) // How long there must be backlogged tasks for before an addition is triggered (seconds) - private val schedulerBacklogTimeoutS = conf.getTimeAsSeconds( - "spark.dynamicAllocation.schedulerBacklogTimeout", "1s") + private val schedulerBacklogTimeoutS = conf.get(DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT) // Same as above, but used only after `schedulerBacklogTimeoutS` is exceeded - private val sustainedSchedulerBacklogTimeoutS = conf.getTimeAsSeconds( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", s"${schedulerBacklogTimeoutS}s") - - // How long an executor must be idle for before it is removed (seconds) - private val executorIdleTimeoutS = conf.getTimeAsSeconds( - "spark.dynamicAllocation.executorIdleTimeout", "60s") - - private val cachedExecutorIdleTimeoutS = conf.getTimeAsSeconds( - "spark.dynamicAllocation.cachedExecutorIdleTimeout", s"${Integer.MAX_VALUE}s") + private val sustainedSchedulerBacklogTimeoutS = + conf.get(DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT) // During testing, the methods to actually kill and add executors are mocked out - private val testing = conf.getBoolean("spark.dynamicAllocation.testing", false) + private val testing = conf.get(DYN_ALLOCATION_TESTING) // TODO: The default value of 1 for spark.executor.cores works right now because dynamic // allocation is only supported for YARN and the default number of cores per executor in YARN is // 1, but it might need to be attained differently for different cluster managers private val tasksPerExecutorForFullParallelism = - conf.getInt("spark.executor.cores", 1) / conf.getInt("spark.task.cpus", 1) + conf.get(EXECUTOR_CORES) / conf.get(CPUS_PER_TASK) private val executorAllocationRatio = conf.get(DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO) @@ -135,33 +135,22 @@ private[spark] class ExecutorAllocationManager( // is the number of executors we would immediately want from the cluster manager. private var numExecutorsTarget = initialNumExecutors - // Executors that have been requested to be removed but have not been killed yet - private val executorsPendingToRemove = new mutable.HashSet[String] - - // All known executors - private val executorIds = new mutable.HashSet[String] - // A timestamp of when an addition should be triggered, or NOT_SET if it is not set // This is set when pending tasks are added but not scheduled yet private var addTime: Long = NOT_SET - // A timestamp for each executor of when the executor should be removed, indexed by the ID - // This is set when an executor is no longer running a task, or when it first registers - private val removeTimes = new mutable.HashMap[String, Long] - // Polling loop interval (ms) private val intervalMillis: Long = if (Utils.isTesting) { - conf.getLong(TESTING_SCHEDULE_INTERVAL_KEY, 100) + conf.get(TEST_SCHEDULE_INTERVAL) } else { 100 } - // Clock used to schedule when executors should be added and removed - private var clock: Clock = new SystemClock() - // Listener for Spark events that impact the allocation policy val listener = new ExecutorAllocationListener + val executorMonitor = new ExecutorMonitor(conf, client, listenerBus, clock) + // Executor that handles the scheduling task. private val executor = ThreadUtils.newDaemonSingleThreadScheduledExecutor("spark-dynamic-executor-allocation") @@ -188,57 +177,47 @@ private[spark] class ExecutorAllocationManager( */ private def validateSettings(): Unit = { if (minNumExecutors < 0 || maxNumExecutors < 0) { - throw new SparkException("spark.dynamicAllocation.{min/max}Executors must be positive!") + throw new SparkException( + s"${DYN_ALLOCATION_MIN_EXECUTORS.key} and ${DYN_ALLOCATION_MAX_EXECUTORS.key} must be " + + "positive!") } if (maxNumExecutors == 0) { - throw new SparkException("spark.dynamicAllocation.maxExecutors cannot be 0!") + throw new SparkException(s"${DYN_ALLOCATION_MAX_EXECUTORS.key} cannot be 0!") } if (minNumExecutors > maxNumExecutors) { - throw new SparkException(s"spark.dynamicAllocation.minExecutors ($minNumExecutors) must " + - s"be less than or equal to spark.dynamicAllocation.maxExecutors ($maxNumExecutors)!") + throw new SparkException(s"${DYN_ALLOCATION_MIN_EXECUTORS.key} ($minNumExecutors) must " + + s"be less than or equal to ${DYN_ALLOCATION_MAX_EXECUTORS.key} ($maxNumExecutors)!") } if (schedulerBacklogTimeoutS <= 0) { - throw new SparkException("spark.dynamicAllocation.schedulerBacklogTimeout must be > 0!") + throw new SparkException(s"${DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!") } if (sustainedSchedulerBacklogTimeoutS <= 0) { throw new SparkException( - "spark.dynamicAllocation.sustainedSchedulerBacklogTimeout must be > 0!") - } - if (executorIdleTimeoutS < 0) { - throw new SparkException("spark.dynamicAllocation.executorIdleTimeout must be >= 0!") - } - if (cachedExecutorIdleTimeoutS < 0) { - throw new SparkException("spark.dynamicAllocation.cachedExecutorIdleTimeout must be >= 0!") - } - // Require external shuffle service for dynamic allocation - // Otherwise, we may lose shuffle files when killing executors - if (!conf.get(config.SHUFFLE_SERVICE_ENABLED) && !testing) { - throw new SparkException("Dynamic allocation of executors requires the external " + - "shuffle service. You may enable this through spark.shuffle.service.enabled.") - } - if (tasksPerExecutorForFullParallelism == 0) { - throw new SparkException("spark.executor.cores must not be < spark.task.cpus.") + s"s${DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key} must be > 0!") + } + if (!conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + if (conf.get(config.DYN_ALLOCATION_SHUFFLE_TRACKING)) { + logWarning("Dynamic allocation without a shuffle service is an experimental feature.") + } else if (!testing) { + throw new SparkException("Dynamic allocation of executors requires the external " + + "shuffle service. You may enable this through spark.shuffle.service.enabled.") + } } if (executorAllocationRatio > 1.0 || executorAllocationRatio <= 0.0) { throw new SparkException( - "spark.dynamicAllocation.executorAllocationRatio must be > 0 and <= 1.0") + s"${DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO.key} must be > 0 and <= 1.0") } } - /** - * Use a different clock for this allocation manager. This is mainly used for testing. - */ - def setClock(newClock: Clock): Unit = { - clock = newClock - } - /** * Register for scheduler callbacks to decide when to add and remove executors, and start * the scheduling task. */ def start(): Unit = { listenerBus.addToManagementQueue(listener) + listenerBus.addToManagementQueue(executorMonitor) + cleaner.foreach(_.attachListener(executorMonitor)) val scheduleTask = new Runnable() { override def run(): Unit = { @@ -275,8 +254,7 @@ private[spark] class ExecutorAllocationManager( def reset(): Unit = synchronized { addTime = 0L numExecutorsTarget = initialNumExecutors - executorsPendingToRemove.clear() - removeTimes.clear() + executorMonitor.reset() } /** @@ -304,19 +282,13 @@ private[spark] class ExecutorAllocationManager( * This is factored out into its own method for testing. */ private def schedule(): Unit = synchronized { - val now = clock.getTimeMillis - - updateAndSyncNumExecutorsTarget(now) - - val executorIdsToBeRemoved = ArrayBuffer[String]() - removeTimes.retain { case (executorId, expireTime) => - val expired = now >= expireTime - if (expired) { - initializing = false - executorIdsToBeRemoved += executorId - } - !expired + val executorIdsToBeRemoved = executorMonitor.timedOutExecutors() + if (executorIdsToBeRemoved.nonEmpty) { + initializing = false } + + // Update executor target number only after initializing flag is unset + updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) if (executorIdsToBeRemoved.nonEmpty) { removeExecutors(executorIdsToBeRemoved) } @@ -392,7 +364,7 @@ private[spark] class ExecutorAllocationManager( val oldNumExecutorsTarget = numExecutorsTarget // There's no point in wasting time ramping up to the number of executors we already have, so // make sure our target is at least as much as our current allocation: - numExecutorsTarget = math.max(numExecutorsTarget, executorIds.size) + numExecutorsTarget = math.max(numExecutorsTarget, executorMonitor.executorCount) // Boost our target with the number to add for this round: numExecutorsTarget += numExecutorsToAdd // Ensure that our target doesn't exceed what we need at the present moment: @@ -452,7 +424,7 @@ private[spark] class ExecutorAllocationManager( val executorIdsToBeRemoved = new ArrayBuffer[String] logInfo("Request to remove executorIds: " + executors.mkString(", ")) - val numExistingExecutors = allocationManager.executorIds.size - executorsPendingToRemove.size + val numExistingExecutors = executorMonitor.executorCount - executorMonitor.pendingRemovalCount var newExecutorTotal = numExistingExecutors executors.foreach { executorIdToBeRemoved => @@ -462,7 +434,7 @@ private[spark] class ExecutorAllocationManager( } else if (newExecutorTotal - 1 < numExecutorsTarget) { logDebug(s"Not removing idle executor $executorIdToBeRemoved because there are only " + s"$newExecutorTotal executor(s) left (number of executor target $numExecutorsTarget)") - } else if (canBeKilled(executorIdToBeRemoved)) { + } else { executorIdsToBeRemoved += executorIdToBeRemoved newExecutorTotal -= 1 } @@ -481,24 +453,17 @@ private[spark] class ExecutorAllocationManager( client.killExecutors(executorIdsToBeRemoved, adjustTargetNumExecutors = false, countFailures = false, force = false) } + // [SPARK-21834] killExecutors api reduces the target number of executors. // So we need to update the target with desired value. client.requestTotalExecutors(numExecutorsTarget, localityAwareTasks, hostToLocalTaskCount) // reset the newExecutorTotal to the existing number of executors newExecutorTotal = numExistingExecutors if (testing || executorsRemoved.nonEmpty) { - executorsRemoved.foreach { removedExecutorId => - // If it is a cached block, it uses cachedExecutorIdleTimeoutS for timeout - val idleTimeout = if (blockManagerMaster.hasCachedBlocks(removedExecutorId)) { - cachedExecutorIdleTimeoutS - } else { - executorIdleTimeoutS - } - newExecutorTotal -= 1 - logInfo(s"Removing executor $removedExecutorId because it has been idle for " + - s"$idleTimeout seconds (new desired total will be $newExecutorTotal)") - executorsPendingToRemove.add(removedExecutorId) - } + newExecutorTotal -= executorsRemoved.size + executorMonitor.executorsKilled(executorsRemoved) + logInfo(s"Executors ${executorsRemoved.mkString(",")} removed due to idle timeout." + + s"(new desired total will be $newExecutorTotal)") executorsRemoved } else { logWarning(s"Unable to reach the cluster manager to kill executor/s " + @@ -507,70 +472,6 @@ private[spark] class ExecutorAllocationManager( } } - /** - * Request the cluster manager to remove the given executor. - * Return whether the request is acknowledged. - */ - private def removeExecutor(executorId: String): Boolean = synchronized { - val executorsRemoved = removeExecutors(Seq(executorId)) - executorsRemoved.nonEmpty && executorsRemoved(0) == executorId - } - - /** - * Determine if the given executor can be killed. - */ - private def canBeKilled(executorId: String): Boolean = synchronized { - // Do not kill the executor if we are not aware of it (should never happen) - if (!executorIds.contains(executorId)) { - logWarning(s"Attempted to remove unknown executor $executorId!") - return false - } - - // Do not kill the executor again if it is already pending to be killed (should never happen) - if (executorsPendingToRemove.contains(executorId)) { - logWarning(s"Attempted to remove executor $executorId " + - s"when it is already pending to be removed!") - return false - } - - true - } - - /** - * Callback invoked when the specified executor has been added. - */ - private def onExecutorAdded(executorId: String): Unit = synchronized { - if (!executorIds.contains(executorId)) { - executorIds.add(executorId) - // If an executor (call this executor X) is not removed because the lower bound - // has been reached, it will no longer be marked as idle. When new executors join, - // however, we are no longer at the lower bound, and so we must mark executor X - // as idle again so as not to forget that it is a candidate for removal. (see SPARK-4951) - executorIds.filter(listener.isExecutorIdle).foreach(onExecutorIdle) - logInfo(s"New executor $executorId has registered (new total is ${executorIds.size})") - } else { - logWarning(s"Duplicate executor $executorId has registered") - } - } - - /** - * Callback invoked when the specified executor has been removed. - */ - private def onExecutorRemoved(executorId: String): Unit = synchronized { - if (executorIds.contains(executorId)) { - executorIds.remove(executorId) - removeTimes.remove(executorId) - logInfo(s"Existing executor $executorId has been removed (new total is ${executorIds.size})") - if (executorsPendingToRemove.contains(executorId)) { - executorsPendingToRemove.remove(executorId) - logDebug(s"Executor $executorId is no longer pending to " + - s"be removed (${executorsPendingToRemove.size} left)") - } - } else { - logWarning(s"Unknown executor $executorId has been removed!") - } - } - /** * Callback invoked when the scheduler receives new pending tasks. * This sets a time in the future that decides when executors should be added @@ -594,44 +495,8 @@ private[spark] class ExecutorAllocationManager( numExecutorsToAdd = 1 } - /** - * Callback invoked when the specified executor is no longer running any tasks. - * This sets a time in the future that decides when this executor should be removed if - * the executor is not already marked as idle. - */ - private def onExecutorIdle(executorId: String): Unit = synchronized { - if (executorIds.contains(executorId)) { - if (!removeTimes.contains(executorId) && !executorsPendingToRemove.contains(executorId)) { - // Note that it is not necessary to query the executors since all the cached - // blocks we are concerned with are reported to the driver. Note that this - // does not include broadcast blocks. - val hasCachedBlocks = blockManagerMaster.hasCachedBlocks(executorId) - val now = clock.getTimeMillis() - val timeout = { - if (hasCachedBlocks) { - // Use a different timeout if the executor has cached blocks. - now + cachedExecutorIdleTimeoutS * 1000 - } else { - now + executorIdleTimeoutS * 1000 - } - } - val realTimeout = if (timeout <= 0) Long.MaxValue else timeout // overflow - removeTimes(executorId) = realTimeout - logDebug(s"Starting idle timer for $executorId because there are no more tasks " + - s"scheduled to run on the executor (to expire in ${(realTimeout - now)/1000} seconds)") - } - } else { - logWarning(s"Attempted to mark unknown executor $executorId idle") - } - } - - /** - * Callback invoked when the specified executor is now running a task. - * This resets all variables used for removing this executor. - */ - private def onExecutorBusy(executorId: String): Unit = synchronized { - logDebug(s"Clearing idle timer for $executorId because it is now running a task") - removeTimes.remove(executorId) + private case class StageAttempt(stageId: Int, stageAttemptId: Int) { + override def toString: String = s"Stage $stageId (Attempt $stageAttemptId)" } /** @@ -642,30 +507,32 @@ private[spark] class ExecutorAllocationManager( */ private[spark] class ExecutorAllocationListener extends SparkListener { - private val stageIdToNumTasks = new mutable.HashMap[Int, Int] - // Number of running tasks per stage including speculative tasks. + private val stageAttemptToNumTasks = new mutable.HashMap[StageAttempt, Int] + // Number of running tasks per stageAttempt including speculative tasks. // Should be 0 when no stages are active. - private val stageIdToNumRunningTask = new mutable.HashMap[Int, Int] - private val stageIdToTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] - private val executorIdToTaskIds = new mutable.HashMap[String, mutable.HashSet[Long]] - // Number of speculative tasks to be scheduled in each stage - private val stageIdToNumSpeculativeTasks = new mutable.HashMap[Int, Int] - // The speculative tasks started in each stage - private val stageIdToSpeculativeTaskIndices = new mutable.HashMap[Int, mutable.HashSet[Int]] - - // stageId to tuple (the number of task with locality preferences, a map where each pair is a - // node and the number of tasks that would like to be scheduled on that node) map, - // maintain the executor placement hints for each stage Id used by resource framework to better - // place the executors. - private val stageIdToExecutorPlacementHints = new mutable.HashMap[Int, (Int, Map[String, Int])] + private val stageAttemptToNumRunningTask = new mutable.HashMap[StageAttempt, Int] + private val stageAttemptToTaskIndices = new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] + // Number of speculative tasks to be scheduled in each stageAttempt + private val stageAttemptToNumSpeculativeTasks = new mutable.HashMap[StageAttempt, Int] + // The speculative tasks started in each stageAttempt + private val stageAttemptToSpeculativeTaskIndices = + new mutable.HashMap[StageAttempt, mutable.HashSet[Int]] + + // stageAttempt to tuple (the number of task with locality preferences, a map where each pair + // is a node and the number of tasks that would like to be scheduled on that node) map, + // maintain the executor placement hints for each stageAttempt used by resource framework + // to better place the executors. + private val stageAttemptToExecutorPlacementHints = + new mutable.HashMap[StageAttempt, (Int, Map[String, Int])] override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted): Unit = { initializing = false val stageId = stageSubmitted.stageInfo.stageId + val stageAttemptId = stageSubmitted.stageInfo.attemptNumber() + val stageAttempt = StageAttempt(stageId, stageAttemptId) val numTasks = stageSubmitted.stageInfo.numTasks allocationManager.synchronized { - stageIdToNumTasks(stageId) = numTasks - stageIdToNumRunningTask(stageId) = 0 + stageAttemptToNumTasks(stageAttempt) = numTasks allocationManager.onSchedulerBacklogged() // Compute the number of tasks requested by the stage on each host @@ -680,7 +547,7 @@ private[spark] class ExecutorAllocationManager( } } } - stageIdToExecutorPlacementHints.put(stageId, + stageAttemptToExecutorPlacementHints.put(stageAttempt, (numTasksPending, hostToLocalTaskCountPerStage.toMap)) // Update the executor placement hints @@ -690,20 +557,24 @@ private[spark] class ExecutorAllocationManager( override def onStageCompleted(stageCompleted: SparkListenerStageCompleted): Unit = { val stageId = stageCompleted.stageInfo.stageId + val stageAttemptId = stageCompleted.stageInfo.attemptNumber() + val stageAttempt = StageAttempt(stageId, stageAttemptId) allocationManager.synchronized { - stageIdToNumTasks -= stageId - stageIdToNumRunningTask -= stageId - stageIdToNumSpeculativeTasks -= stageId - stageIdToTaskIndices -= stageId - stageIdToSpeculativeTaskIndices -= stageId - stageIdToExecutorPlacementHints -= stageId + // do NOT remove stageAttempt from stageAttemptToNumRunningTasks, + // because the attempt may still have running tasks, + // even after another attempt for the stage is submitted. + stageAttemptToNumTasks -= stageAttempt + stageAttemptToNumSpeculativeTasks -= stageAttempt + stageAttemptToTaskIndices -= stageAttempt + stageAttemptToSpeculativeTaskIndices -= stageAttempt + stageAttemptToExecutorPlacementHints -= stageAttempt // Update the executor placement hints updateExecutorPlacementHints() // If this is the last stage with pending tasks, mark the scheduler queue as empty // This is needed in case the stage is aborted for any reason - if (stageIdToNumTasks.isEmpty && stageIdToNumSpeculativeTasks.isEmpty) { + if (stageAttemptToNumTasks.isEmpty && stageAttemptToNumSpeculativeTasks.isEmpty) { allocationManager.onSchedulerQueueEmpty() } } @@ -711,56 +582,38 @@ private[spark] class ExecutorAllocationManager( override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { val stageId = taskStart.stageId - val taskId = taskStart.taskInfo.taskId + val stageAttemptId = taskStart.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) val taskIndex = taskStart.taskInfo.index - val executorId = taskStart.taskInfo.executorId - allocationManager.synchronized { - if (stageIdToNumRunningTask.contains(stageId)) { - stageIdToNumRunningTask(stageId) += 1 - } - // This guards against the race condition in which the `SparkListenerTaskStart` - // event is posted before the `SparkListenerBlockManagerAdded` event, which is - // possible because these events are posted in different threads. (see SPARK-4951) - if (!allocationManager.executorIds.contains(executorId)) { - allocationManager.onExecutorAdded(executorId) - } - + stageAttemptToNumRunningTask(stageAttempt) = + stageAttemptToNumRunningTask.getOrElse(stageAttempt, 0) + 1 // If this is the last pending task, mark the scheduler queue as empty if (taskStart.taskInfo.speculative) { - stageIdToSpeculativeTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += - taskIndex + stageAttemptToSpeculativeTaskIndices.getOrElseUpdate(stageAttempt, + new mutable.HashSet[Int]) += taskIndex } else { - stageIdToTaskIndices.getOrElseUpdate(stageId, new mutable.HashSet[Int]) += taskIndex + stageAttemptToTaskIndices.getOrElseUpdate(stageAttempt, + new mutable.HashSet[Int]) += taskIndex } if (totalPendingTasks() == 0) { allocationManager.onSchedulerQueueEmpty() } - - // Mark the executor on which this task is scheduled as busy - executorIdToTaskIds.getOrElseUpdate(executorId, new mutable.HashSet[Long]) += taskId - allocationManager.onExecutorBusy(executorId) } } override def onTaskEnd(taskEnd: SparkListenerTaskEnd): Unit = { - val executorId = taskEnd.taskInfo.executorId - val taskId = taskEnd.taskInfo.taskId - val taskIndex = taskEnd.taskInfo.index val stageId = taskEnd.stageId + val stageAttemptId = taskEnd.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) + val taskIndex = taskEnd.taskInfo.index allocationManager.synchronized { - if (stageIdToNumRunningTask.contains(stageId)) { - stageIdToNumRunningTask(stageId) -= 1 - } - // If the executor is no longer running any scheduled tasks, mark it as idle - if (executorIdToTaskIds.contains(executorId)) { - executorIdToTaskIds(executorId) -= taskId - if (executorIdToTaskIds(executorId).isEmpty) { - executorIdToTaskIds -= executorId - allocationManager.onExecutorIdle(executorId) + if (stageAttemptToNumRunningTask.contains(stageAttempt)) { + stageAttemptToNumRunningTask(stageAttempt) -= 1 + if (stageAttemptToNumRunningTask(stageAttempt) == 0) { + stageAttemptToNumRunningTask -= stageAttempt } } - // If the task failed, we expect it to be resubmitted later. To ensure we have // enough resources to run the resubmitted task, we need to mark the scheduler // as backlogged again if it's not already marked as such (SPARK-8366) @@ -769,37 +622,22 @@ private[spark] class ExecutorAllocationManager( allocationManager.onSchedulerBacklogged() } if (taskEnd.taskInfo.speculative) { - stageIdToSpeculativeTaskIndices.get(stageId).foreach {_.remove(taskIndex)} + stageAttemptToSpeculativeTaskIndices.get(stageAttempt).foreach {_.remove(taskIndex)} } else { - stageIdToTaskIndices.get(stageId).foreach {_.remove(taskIndex)} + stageAttemptToTaskIndices.get(stageAttempt).foreach {_.remove(taskIndex)} } } } } - override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { - val executorId = executorAdded.executorId - if (executorId != SparkContext.DRIVER_IDENTIFIER) { - // This guards against the race condition in which the `SparkListenerTaskStart` - // event is posted before the `SparkListenerBlockManagerAdded` event, which is - // possible because these events are posted in different threads. (see SPARK-4951) - if (!allocationManager.executorIds.contains(executorId)) { - allocationManager.onExecutorAdded(executorId) - } - } - } - - override def onExecutorRemoved(executorRemoved: SparkListenerExecutorRemoved): Unit = { - allocationManager.onExecutorRemoved(executorRemoved.executorId) - } - override def onSpeculativeTaskSubmitted(speculativeTask: SparkListenerSpeculativeTaskSubmitted) : Unit = { - val stageId = speculativeTask.stageId - + val stageId = speculativeTask.stageId + val stageAttemptId = speculativeTask.stageAttemptId + val stageAttempt = StageAttempt(stageId, stageAttemptId) allocationManager.synchronized { - stageIdToNumSpeculativeTasks(stageId) = - stageIdToNumSpeculativeTasks.getOrElse(stageId, 0) + 1 + stageAttemptToNumSpeculativeTasks(stageAttempt) = + stageAttemptToNumSpeculativeTasks.getOrElse(stageAttempt, 0) + 1 allocationManager.onSchedulerBacklogged() } } @@ -811,14 +649,14 @@ private[spark] class ExecutorAllocationManager( * Note: This is not thread-safe without the caller owning the `allocationManager` lock. */ def pendingTasks(): Int = { - stageIdToNumTasks.map { case (stageId, numTasks) => - numTasks - stageIdToTaskIndices.get(stageId).map(_.size).getOrElse(0) + stageAttemptToNumTasks.map { case (stageAttempt, numTasks) => + numTasks - stageAttemptToTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) }.sum } def pendingSpeculativeTasks(): Int = { - stageIdToNumSpeculativeTasks.map { case (stageId, numTasks) => - numTasks - stageIdToSpeculativeTaskIndices.get(stageId).map(_.size).getOrElse(0) + stageAttemptToNumSpeculativeTasks.map { case (stageAttempt, numTasks) => + numTasks - stageAttemptToSpeculativeTaskIndices.get(stageAttempt).map(_.size).getOrElse(0) }.sum } @@ -828,18 +666,10 @@ private[spark] class ExecutorAllocationManager( /** * The number of tasks currently running across all stages. + * Include running-but-zombie stage attempts */ def totalRunningTasks(): Int = { - stageIdToNumRunningTask.values.sum - } - - /** - * Return true if an executor is not currently running a task, and false otherwise. - * - * Note: This is not thread-safe without the caller owning the `allocationManager` lock. - */ - def isExecutorIdle(executorId: String): Boolean = { - !executorIdToTaskIds.contains(executorId) + stageAttemptToNumRunningTask.values.sum } /** @@ -853,7 +683,7 @@ private[spark] class ExecutorAllocationManager( def updateExecutorPlacementHints(): Unit = { var localityAwareTasks = 0 val localityToCount = new mutable.HashMap[String, Int]() - stageIdToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) => + stageAttemptToExecutorPlacementHints.values.foreach { case (numTasksPending, localities) => localityAwareTasks += numTasksPending localities.foreach { case (hostname, count) => val updatedCount = localityToCount.getOrElse(hostname, 0) + count @@ -884,8 +714,8 @@ private[spark] class ExecutorAllocationManager( } registerGauge("numberExecutorsToAdd", numExecutorsToAdd, 0) - registerGauge("numberExecutorsPendingToRemove", executorsPendingToRemove.size, 0) - registerGauge("numberAllExecutors", executorIds.size, 0) + registerGauge("numberExecutorsPendingToRemove", executorMonitor.pendingRemovalCount, 0) + registerGauge("numberAllExecutors", executorMonitor.executorCount, 0) registerGauge("numberTargetExecutors", numExecutorsTarget, 0) registerGauge("numberMaxNeededExecutors", maxNumExecutorsNeeded(), 0) } @@ -893,5 +723,4 @@ private[spark] class ExecutorAllocationManager( private object ExecutorAllocationManager { val NOT_SET = Long.MaxValue - val TESTING_SCHEDULE_INTERVAL_KEY = "spark.testing.dynamicAllocation.scheduleInterval" } diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 036c9a60630ea..8230533f9d245 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -89,18 +89,6 @@ trait FutureAction[T] extends Future[T] { */ override def value: Option[Try[T]] - // These two methods must be implemented in Scala 2.12. They're implemented as a no-op here - // and then filled in with a real implementation in the two subclasses below. The no-op exists - // here so that those implementations can declare "override", necessary in 2.12, while working - // in 2.11, where the method doesn't exist in the superclass. - // After 2.11 support goes away, remove these two: - - def transform[S](f: (Try[T]) => Try[S])(implicit executor: ExecutionContext): Future[S] = - throw new UnsupportedOperationException() - - def transformWith[S](f: (Try[T]) => Future[S])(implicit executor: ExecutionContext): Future[S] = - throw new UnsupportedOperationException() - /** * Blocks and returns the result of this job. */ @@ -117,43 +105,6 @@ trait FutureAction[T] extends Future[T] { } -/** - * Scala 2.12 defines the two new transform/transformWith methods mentioned above. Impementing - * these for 2.12 in the Spark class here requires delegating to these same methods in an - * underlying Future object. But that only exists in 2.12. But these methods are only called - * in 2.12. So define helper shims to access these methods on a Future by reflection. - */ -private[spark] object FutureAction { - - private val transformTryMethod = - try { - classOf[Future[_]].getMethod("transform", classOf[(_) => _], classOf[ExecutionContext]) - } catch { - case _: NoSuchMethodException => null // Would fail later in 2.11, but not called in 2.11 - } - - private val transformWithTryMethod = - try { - classOf[Future[_]].getMethod("transformWith", classOf[(_) => _], classOf[ExecutionContext]) - } catch { - case _: NoSuchMethodException => null // Would fail later in 2.11, but not called in 2.11 - } - - private[spark] def transform[T, S]( - future: Future[T], - f: (Try[T]) => Try[S], - executor: ExecutionContext): Future[S] = - transformTryMethod.invoke(future, f, executor).asInstanceOf[Future[S]] - - private[spark] def transformWith[T, S]( - future: Future[T], - f: (Try[T]) => Future[S], - executor: ExecutionContext): Future[S] = - transformWithTryMethod.invoke(future, f, executor).asInstanceOf[Future[S]] - -} - - /** * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. @@ -195,16 +146,10 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: def jobIds: Seq[Int] = Seq(jobWaiter.jobId) override def transform[S](f: (Try[T]) => Try[S])(implicit e: ExecutionContext): Future[S] = - FutureAction.transform( - jobWaiter.completionFuture, - (u: Try[Unit]) => f(u.map(_ => resultFunc)), - e) + jobWaiter.completionFuture.transform((u: Try[Unit]) => f(u.map(_ => resultFunc))) override def transformWith[S](f: (Try[T]) => Future[S])(implicit e: ExecutionContext): Future[S] = - FutureAction.transformWith( - jobWaiter.completionFuture, - (u: Try[Unit]) => f(u.map(_ => resultFunc)), - e) + jobWaiter.completionFuture.transformWith((u: Try[Unit]) => f(u.map(_ => resultFunc))) } @@ -299,10 +244,10 @@ class ComplexFutureAction[T](run : JobSubmitter => Future[T]) def jobIds: Seq[Int] = subActions.flatMap(_.jobIds) override def transform[S](f: (Try[T]) => Try[S])(implicit e: ExecutionContext): Future[S] = - FutureAction.transform(p.future, f, e) + p.future.transform(f) override def transformWith[S](f: (Try[T]) => Future[S])(implicit e: ExecutionContext): Future[S] = - FutureAction.transformWith(p.future, f, e) + p.future.transformWith(f) } diff --git a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala index ab0ae55ed357d..f7e3103f2502a 100644 --- a/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala +++ b/core/src/main/scala/org/apache/spark/HeartbeatReceiver.scala @@ -23,7 +23,8 @@ import scala.collection.mutable import scala.concurrent.Future import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.Network import org.apache.spark.rpc.{RpcCallContext, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerId @@ -74,18 +75,18 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) // executor ID -> timestamp of when the last heartbeat from this executor was received private val executorLastSeen = new mutable.HashMap[String, Long] - // "spark.network.timeout" uses "seconds", while `spark.storage.blockManagerSlaveTimeoutMs` uses - // "milliseconds" - private val executorTimeoutMs = - sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", - s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s")}s") + private val executorTimeoutMs = sc.conf.get(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT) - // "spark.network.timeoutInterval" uses "seconds", while - // "spark.storage.blockManagerTimeoutIntervalMs" uses "milliseconds" - private val timeoutIntervalMs = - sc.conf.getTimeAsMs("spark.storage.blockManagerTimeoutIntervalMs", "60s") - private val checkTimeoutIntervalMs = - sc.conf.getTimeAsSeconds("spark.network.timeoutInterval", s"${timeoutIntervalMs}ms") * 1000 + private val checkTimeoutIntervalMs = sc.conf.get(Network.NETWORK_TIMEOUT_INTERVAL) + + private val executorHeartbeatIntervalMs = sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL) + + require(checkTimeoutIntervalMs <= executorTimeoutMs, + s"${Network.NETWORK_TIMEOUT_INTERVAL.key} should be less than or " + + s"equal to ${config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key}.") + require(executorHeartbeatIntervalMs <= executorTimeoutMs, + s"${config.EXECUTOR_HEARTBEAT_INTERVAL.key} should be less than or " + + s"equal to ${config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT.key}") private var timeoutCheckingTask: ScheduledFuture[_] = null @@ -97,11 +98,9 @@ private[spark] class HeartbeatReceiver(sc: SparkContext, clock: Clock) private val killExecutorThread = ThreadUtils.newDaemonSingleThreadExecutor("kill-executor-thread") override def onStart(): Unit = { - timeoutCheckingTask = eventLoopThread.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - Option(self).foreach(_.ask[Boolean](ExpireDeadHosts)) - } - }, 0, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS) + timeoutCheckingTask = eventLoopThread.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { Option(self).foreach(_.ask[Boolean](ExpireDeadHosts)) }, + 0, checkTimeoutIntervalMs, TimeUnit.MILLISECONDS) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { diff --git a/core/src/main/scala/org/apache/spark/Heartbeater.scala b/core/src/main/scala/org/apache/spark/Heartbeater.scala index 5ba1b9b2d828e..1012755e068d1 100644 --- a/core/src/main/scala/org/apache/spark/Heartbeater.scala +++ b/core/src/main/scala/org/apache/spark/Heartbeater.scala @@ -61,10 +61,17 @@ private[spark] class Heartbeater( /** * Get the current executor level metrics. These are returned as an array, with the index - * determined by MetricGetter.values + * determined by ExecutorMetricType.metricToOffset */ def getCurrentMetrics(): ExecutorMetrics = { - val metrics = ExecutorMetricType.values.map(_.getMetricValue(memoryManager)).toArray + + val metrics = new Array[Long](ExecutorMetricType.numMetrics) + var offset = 0 + ExecutorMetricType.metricGetters.foreach { metric => + val newMetrics = metric.getMetricValues(memoryManager) + Array.copy(newMetrics, 0, metrics, offset, newMetrics.size) + offset += newMetrics.length + } new ExecutorMetrics(metrics) } } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 1c4fa4bc6541f..5c820f5a2ed0f 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -18,7 +18,7 @@ package org.apache.spark import java.io._ -import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor} +import java.util.concurrent.{ConcurrentHashMap, LinkedBlockingQueue, ThreadPoolExecutor, TimeUnit} import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.JavaConverters._ @@ -201,7 +201,7 @@ private class ShuffleStatus(numPartitions: Int) { Utils.tryLogNonFatalError { // Use `blocking = false` so that this operation doesn't hang while trying to send cleanup // RPCs to dead executors. - cachedSerializedBroadcast.destroy(blocking = false) + cachedSerializedBroadcast.destroy() } cachedSerializedBroadcast = null } @@ -227,7 +227,7 @@ private[spark] class MapOutputTrackerMasterEndpoint( case GetMapOutputStatuses(shuffleId: Int) => val hostPort = context.senderAddress.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) - val mapOutputStatuses = tracker.post(new GetMapOutputMessage(shuffleId, context)) + tracker.post(new GetMapOutputMessage(shuffleId, context)) case StopMapOutputTracker => logInfo("MapOutputTrackerMasterEndpoint stopped!") @@ -322,11 +322,10 @@ private[spark] class MapOutputTrackerMaster( extends MapOutputTracker(conf) { // The size at which we use Broadcast to send the map output statuses to the executors - private val minSizeForBroadcast = - conf.getSizeAsBytes("spark.shuffle.mapOutput.minSizeForBroadcast", "512k").toInt + private val minSizeForBroadcast = conf.get(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST).toInt /** Whether to compute locality preferences for reduce tasks */ - private val shuffleLocalityEnabled = conf.getBoolean("spark.shuffle.reduceLocality.enabled", true) + private val shuffleLocalityEnabled = conf.get(SHUFFLE_REDUCE_LOCALITY_ENABLE) // Number of map and reduce tasks above which we do not assign preferred locations based on map // output sizes. We limit the size of jobs for which assign preferred locations as computing the @@ -353,7 +352,7 @@ private[spark] class MapOutputTrackerMaster( // Thread pool used for handling map output status requests. This is a separate thread pool // to ensure we don't block the normal dispatcher threads. private val threadpool: ThreadPoolExecutor = { - val numThreads = conf.getInt("spark.shuffle.mapOutput.dispatcher.numThreads", 8) + val numThreads = conf.get(SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS) val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "map-output-dispatcher") for (i <- 0 until numThreads) { pool.execute(new MessageLoop) @@ -364,9 +363,9 @@ private[spark] class MapOutputTrackerMaster( // Make sure that we aren't going to exceed the max RPC message size by making sure // we use broadcast to send large map output statuses. if (minSizeForBroadcast > maxRpcMessageSize) { - val msg = s"spark.shuffle.mapOutput.minSizeForBroadcast ($minSizeForBroadcast bytes) must " + - s"be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an rpc " + - "message that is too large." + val msg = s"${SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST.key} ($minSizeForBroadcast bytes) " + + s"must be <= spark.rpc.message.maxSize ($maxRpcMessageSize bytes) to prevent sending an " + + "rpc message that is too large." logError(msg) throw new IllegalArgumentException(msg) } @@ -671,7 +670,7 @@ private[spark] class MapOutputTrackerMaster( /** * Executor-side client for fetching map output info from the driver's MapOutputTrackerMaster. * Note that this is not used in local-mode; instead, local-mode Executors access the - * MapOutputTrackerMaster directly (which is possible because the master and worker share a comon + * MapOutputTrackerMaster directly (which is possible because the master and worker share a common * superclass). */ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { @@ -707,7 +706,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { logInfo("Don't have map outputs for shuffle " + shuffleId + ", fetching them") - val startTime = System.currentTimeMillis + val startTimeNs = System.nanoTime() var fetchedStatuses: Array[MapStatus] = null fetching.synchronized { // Someone else is fetching it; wait for them to be done @@ -745,7 +744,7 @@ private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTr } } logDebug(s"Fetching map output statuses for shuffle $shuffleId took " + - s"${System.currentTimeMillis - startTime} ms") + s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms") if (fetchedStatuses != null) { fetchedStatuses diff --git a/core/src/main/scala/org/apache/spark/Partitioner.scala b/core/src/main/scala/org/apache/spark/Partitioner.scala index 515237558fd87..a0cba8ab13fef 100644 --- a/core/src/main/scala/org/apache/spark/Partitioner.scala +++ b/core/src/main/scala/org/apache/spark/Partitioner.scala @@ -51,8 +51,8 @@ object Partitioner { * * When available, we choose the partitioner from rdds with maximum number of partitions. If this * partitioner is eligible (number of partitions within an order of maximum number of partitions - * in rdds), or has partition number higher than default partitions number - we use this - * partitioner. + * in rdds), or has partition number higher than or equal to default partitions number - we use + * this partitioner. * * Otherwise, we'll use a new HashPartitioner with the default partitions number. * @@ -79,9 +79,9 @@ object Partitioner { } // If the existing max partitioner is an eligible one, or its partitions number is larger - // than the default number of partitions, use the existing partitioner. + // than or equal to the default number of partitions, use the existing partitioner. if (hasMaxPartitioner.nonEmpty && (isEligiblePartitioner(hasMaxPartitioner.get, rdds) || - defaultNumPartitions < hasMaxPartitioner.get.getNumPartitions)) { + defaultNumPartitions <= hasMaxPartitioner.get.getNumPartitions)) { hasMaxPartitioner.get.partitioner.get } else { new HashPartitioner(defaultNumPartitions) diff --git a/core/src/main/scala/org/apache/spark/SSLOptions.scala b/core/src/main/scala/org/apache/spark/SSLOptions.scala index 1632e0c69eef5..446a8e5eba73e 100644 --- a/core/src/main/scala/org/apache/spark/SSLOptions.scala +++ b/core/src/main/scala/org/apache/spark/SSLOptions.scala @@ -68,7 +68,7 @@ private[spark] case class SSLOptions( */ def createJettySslContextFactory(): Option[SslContextFactory] = { if (enabled) { - val sslContextFactory = new SslContextFactory() + val sslContextFactory = new SslContextFactory.Server() keyStore.foreach(file => sslContextFactory.setKeyStorePath(file.getAbsolutePath)) keyStorePassword.foreach(sslContextFactory.setKeyStorePassword) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 3cfafeb951105..77db0f5d0eaa7 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -17,8 +17,10 @@ package org.apache.spark -import java.net.{Authenticator, PasswordAuthentication} +import java.io.File import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files +import java.util.Base64 import org.apache.hadoop.io.Text import org.apache.hadoop.security.{Credentials, UserGroupInformation} @@ -26,6 +28,7 @@ import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.sasl.SecretKeyHolder import org.apache.spark.util.Utils @@ -43,7 +46,8 @@ import org.apache.spark.util.Utils */ private[spark] class SecurityManager( sparkConf: SparkConf, - val ioEncryptionKey: Option[Array[Byte]] = None) + val ioEncryptionKey: Option[Array[Byte]] = None, + authSecretFileConf: ConfigEntry[Option[String]] = AUTH_SECRET_FILE) extends Logging with SecretKeyHolder { import SecurityManager._ @@ -52,17 +56,13 @@ private[spark] class SecurityManager( private val WILDCARD_ACL = "*" private val authOn = sparkConf.get(NETWORK_AUTH_ENABLED) - // keep spark.ui.acls.enable for backwards compatibility with 1.0 - private var aclsOn = - sparkConf.getBoolean("spark.acls.enable", sparkConf.getBoolean("spark.ui.acls.enable", false)) + private var aclsOn = sparkConf.get(ACLS_ENABLE) // admin acls should be set before view or modify acls - private var adminAcls: Set[String] = - stringToSet(sparkConf.get("spark.admin.acls", "")) + private var adminAcls: Set[String] = sparkConf.get(ADMIN_ACLS).toSet // admin group acls should be set before view or modify group acls - private var adminAclsGroups : Set[String] = - stringToSet(sparkConf.get("spark.admin.acls.groups", "")) + private var adminAclsGroups: Set[String] = sparkConf.get(ADMIN_ACLS_GROUPS).toSet private var viewAcls: Set[String] = _ @@ -78,11 +78,11 @@ private[spark] class SecurityManager( private val defaultAclUsers = Set[String](System.getProperty("user.name", ""), Utils.getCurrentUserName()) - setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) - setModifyAcls(defaultAclUsers, sparkConf.get("spark.modify.acls", "")) + setViewAcls(defaultAclUsers, sparkConf.get(UI_VIEW_ACLS)) + setModifyAcls(defaultAclUsers, sparkConf.get(MODIFY_ACLS)) - setViewAclsGroups(sparkConf.get("spark.ui.view.acls.groups", "")); - setModifyAclsGroups(sparkConf.get("spark.modify.acls.groups", "")); + setViewAclsGroups(sparkConf.get(UI_VIEW_ACLS_GROUPS)) + setModifyAclsGroups(sparkConf.get(MODIFY_ACLS_GROUPS)) private var secretKey: String = _ logInfo("SecurityManager: authentication " + (if (authOn) "enabled" else "disabled") + @@ -92,25 +92,6 @@ private[spark] class SecurityManager( "; users with modify permissions: " + modifyAcls.toString() + "; groups with modify permissions: " + modifyAclsGroups.toString()) - // Set our own authenticator to properly negotiate user/password for HTTP connections. - // This is needed by the HTTP client fetching from the HttpServer. Put here so its - // only set once. - if (authOn) { - Authenticator.setDefault( - new Authenticator() { - override def getPasswordAuthentication(): PasswordAuthentication = { - var passAuth: PasswordAuthentication = null - val userInfo = getRequestingURL().getUserInfo() - if (userInfo != null) { - val parts = userInfo.split(":", 2) - passAuth = new PasswordAuthentication(parts(0), parts(1).toCharArray()) - } - return passAuth - } - } - ) - } - private val hadoopConf = SparkHadoopUtil.get.newConfiguration(sparkConf) // the default SSL configuration - it will be used by all communication layers unless overwritten private val defaultSSLOptions = @@ -123,23 +104,16 @@ private[spark] class SecurityManager( opts } - /** - * Split a comma separated String, filter out any empty items, and return a Set of strings - */ - private def stringToSet(list: String): Set[String] = { - list.split(',').map(_.trim).filter(!_.isEmpty).toSet - } - /** * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setViewAcls(defaultUsers: Set[String], allowedUsers: String) { - viewAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setViewAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + viewAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing view acls to: " + viewAcls.mkString(",")) } - def setViewAcls(defaultUser: String, allowedUsers: String) { + def setViewAcls(defaultUser: String, allowedUsers: Seq[String]) { setViewAcls(Set[String](defaultUser), allowedUsers) } @@ -147,8 +121,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setViewAclsGroups(allowedUserGroups: String) { - viewAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setViewAclsGroups(allowedUserGroups: Seq[String]) { + viewAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing view acls groups to: " + viewAclsGroups.mkString(",")) } @@ -175,8 +149,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setModifyAcls(defaultUsers: Set[String], allowedUsers: String) { - modifyAcls = (adminAcls ++ defaultUsers ++ stringToSet(allowedUsers)) + def setModifyAcls(defaultUsers: Set[String], allowedUsers: Seq[String]) { + modifyAcls = adminAcls ++ defaultUsers ++ allowedUsers logInfo("Changing modify acls to: " + modifyAcls.mkString(",")) } @@ -184,8 +158,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setModifyAclsGroups(allowedUserGroups: String) { - modifyAclsGroups = (adminAclsGroups ++ stringToSet(allowedUserGroups)); + def setModifyAclsGroups(allowedUserGroups: Seq[String]) { + modifyAclsGroups = adminAclsGroups ++ allowedUserGroups logInfo("Changing modify acls groups to: " + modifyAclsGroups.mkString(",")) } @@ -212,8 +186,8 @@ private[spark] class SecurityManager( * Admin acls should be set before the view or modify acls. If you modify the admin * acls you should also set the view and modify acls again to pick up the changes. */ - def setAdminAcls(adminUsers: String) { - adminAcls = stringToSet(adminUsers) + def setAdminAcls(adminUsers: Seq[String]) { + adminAcls = adminUsers.toSet logInfo("Changing admin acls to: " + adminAcls.mkString(",")) } @@ -221,8 +195,8 @@ private[spark] class SecurityManager( * Admin acls groups should be set before the view or modify acls groups. If you modify the admin * acls groups you should also set the view and modify acls groups again to pick up the changes. */ - def setAdminAclsGroups(adminUserGroups: String) { - adminAclsGroups = stringToSet(adminUserGroups) + def setAdminAclsGroups(adminUserGroups: Seq[String]) { + adminAclsGroups = adminUserGroups.toSet logInfo("Changing admin acls groups to: " + adminAclsGroups.mkString(",")) } @@ -239,6 +213,15 @@ private[spark] class SecurityManager( */ def aclsEnabled(): Boolean = aclsOn + /** + * Checks whether the given user is an admin. This gives the user both view and + * modify permissions, and also allows the user to impersonate other users when + * making UI requests. + */ + def checkAdminPermissions(user: String): Boolean = { + isUserInACL(user, adminAcls, adminAclsGroups) + } + /** * Checks the given user against the view acl and groups list to see if they have * authorization to view the UI. If the UI acls are disabled @@ -252,13 +235,7 @@ private[spark] class SecurityManager( def checkUIViewPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " viewAcls=" + viewAcls.mkString(",") + " viewAclsGroups=" + viewAclsGroups.mkString(",")) - if (!aclsEnabled || user == null || viewAcls.contains(user) || - viewAcls.contains(WILDCARD_ACL) || viewAclsGroups.contains(WILDCARD_ACL)) { - return true - } - val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) - logDebug("userGroups=" + currentUserGroups.mkString(",")) - viewAclsGroups.exists(currentUserGroups.contains(_)) + isUserInACL(user, viewAcls, viewAclsGroups) } /** @@ -274,13 +251,7 @@ private[spark] class SecurityManager( def checkModifyPermissions(user: String): Boolean = { logDebug("user=" + user + " aclsEnabled=" + aclsEnabled() + " modifyAcls=" + modifyAcls.mkString(",") + " modifyAclsGroups=" + modifyAclsGroups.mkString(",")) - if (!aclsEnabled || user == null || modifyAcls.contains(user) || - modifyAcls.contains(WILDCARD_ACL) || modifyAclsGroups.contains(WILDCARD_ACL)) { - return true - } - val currentUserGroups = Utils.getCurrentUserGroups(sparkConf, user) - logDebug("userGroups=" + currentUserGroups) - modifyAclsGroups.exists(currentUserGroups.contains(_)) + isUserInACL(user, modifyAcls, modifyAclsGroups) } /** @@ -294,16 +265,9 @@ private[spark] class SecurityManager( * @return Whether to enable encryption when connecting to services that support it. */ def isEncryptionEnabled(): Boolean = { - sparkConf.get(NETWORK_ENCRYPTION_ENABLED) || sparkConf.get(SASL_ENCRYPTION_ENABLED) + sparkConf.get(Network.NETWORK_CRYPTO_ENABLED) || sparkConf.get(SASL_ENCRYPTION_ENABLED) } - /** - * Gets the user used for authenticating HTTP connections. - * For now use a single hardcoded user. - * @return the HTTP user as a String - */ - def getHttpUser(): String = "sparkHttpUser" - /** * Gets the user used for authenticating SASL connections. * For now use a single hardcoded user. @@ -328,6 +292,7 @@ private[spark] class SecurityManager( .orElse(Option(secretKey)) .orElse(Option(sparkConf.getenv(ENV_AUTH_SECRET))) .orElse(sparkConf.getOption(SPARK_AUTH_SECRET_CONF)) + .orElse(secretKeyFromFile()) .getOrElse { throw new IllegalArgumentException( s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config") @@ -353,20 +318,70 @@ private[spark] class SecurityManager( return } + // TODO: this really should be abstracted somewhere else. val master = sparkConf.get(SparkLauncher.SPARK_MASTER, "") - master match { + val storeInUgi = master match { case "yarn" | "local" | LOCAL_N_REGEX(_) | LOCAL_N_FAILURES_REGEX(_, _) => - // Secret generation allowed here + true + + case k8sRegex() => + // Don't propagate the secret through the user's credentials in kubernetes. That conflicts + // with the way k8s handles propagation of delegation tokens. + false + case _ => require(sparkConf.contains(SPARK_AUTH_SECRET_CONF), s"A secret key must be specified via the $SPARK_AUTH_SECRET_CONF config.") return } - secretKey = Utils.createSecret(sparkConf) - val creds = new Credentials() - creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) - UserGroupInformation.getCurrentUser().addCredentials(creds) + if (sparkConf.get(AUTH_SECRET_FILE_DRIVER).isDefined != + sparkConf.get(AUTH_SECRET_FILE_EXECUTOR).isDefined) { + throw new IllegalArgumentException( + "Invalid secret configuration: Secret files must be specified for both the driver and the" + + " executors, not only one or the other.") + } + + secretKey = secretKeyFromFile().getOrElse(Utils.createSecret(sparkConf)) + + if (storeInUgi) { + val creds = new Credentials() + creds.addSecretKey(SECRET_LOOKUP_KEY, secretKey.getBytes(UTF_8)) + UserGroupInformation.getCurrentUser().addCredentials(creds) + } + } + + private def secretKeyFromFile(): Option[String] = { + sparkConf.get(authSecretFileConf).flatMap { secretFilePath => + sparkConf.getOption(SparkLauncher.SPARK_MASTER).map { + case k8sRegex() => + val secretFile = new File(secretFilePath) + require(secretFile.isFile, s"No file found containing the secret key at $secretFilePath.") + val base64Key = Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + require(!base64Key.isEmpty, s"Secret key from file located at $secretFilePath is empty.") + base64Key + case _ => + throw new IllegalArgumentException( + "Secret keys provided via files is only allowed in Kubernetes mode.") + } + } + } + + private def isUserInACL( + user: String, + aclUsers: Set[String], + aclGroups: Set[String]): Boolean = { + if (user == null || + !aclsEnabled || + aclUsers.contains(WILDCARD_ACL) || + aclUsers.contains(user) || + aclGroups.contains(WILDCARD_ACL)) { + true + } else { + val userGroups = Utils.getCurrentUserGroups(sparkConf, user) + logDebug(s"user $user is in groups ${userGroups.mkString(",")}") + aclGroups.exists(userGroups.contains(_)) + } } // Default SecurityManager only has a single secret key, so ignore appId. @@ -376,8 +391,9 @@ private[spark] class SecurityManager( private[spark] object SecurityManager { + val k8sRegex = "k8s.*".r val SPARK_AUTH_CONF = NETWORK_AUTH_ENABLED.key - val SPARK_AUTH_SECRET_CONF = "spark.authenticate.secret" + val SPARK_AUTH_SECRET_CONF = AUTH_SECRET.key // This is used to set auth secret to an executor's env variable. It should have the same // value as SPARK_AUTH_SECRET_CONF set in SparkConf val ENV_AUTH_SECRET = "_SPARK_AUTH_SECRET" diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index 81aa31d79ba82..24be54ec91828 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -25,9 +25,11 @@ import scala.collection.mutable.LinkedHashSet import org.apache.avro.{Schema, SchemaNormalization} -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.internal.config.Network._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -60,9 +62,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria @transient private lazy val reader: ConfigReader = { val _reader = new ConfigReader(new SparkConfigProvider(settings)) - _reader.bindEnv(new ConfigProvider { - override def get(key: String): Option[String] = Option(getenv(key)) - }) + _reader.bindEnv((key: String) => Option(getenv(key))) _reader } @@ -123,7 +123,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria /** Set JAR files to distribute to the cluster. */ def setJars(jars: Seq[String]): SparkConf = { for (jar <- jars if (jar == null)) logWarning("null jar passed to SparkContext constructor") - set("spark.jars", jars.filter(_ != null).mkString(",")) + set(JARS, jars.filter(_ != null)) } /** Set JAR files to distribute to the cluster. (Java-friendly version.) */ @@ -168,6 +168,15 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } /** Set multiple parameters together */ + def setAll(settings: Iterable[(String, String)]): SparkConf = { + settings.foreach { case (k, v) => set(k, v) } + this + } + + /** + * Set multiple parameters together + */ + @deprecated("Use setAll(Iterable) instead", "3.0.0") def setAll(settings: Traversable[(String, String)]): SparkConf = { settings.foreach { case (k, v) => set(k, v) } this @@ -201,12 +210,12 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria */ def registerKryoClasses(classes: Array[Class[_]]): SparkConf = { val allClassNames = new LinkedHashSet[String]() - allClassNames ++= get("spark.kryo.classesToRegister", "").split(',').map(_.trim) + allClassNames ++= get(KRYO_CLASSES_TO_REGISTER).map(_.trim) .filter(!_.isEmpty) allClassNames ++= classes.map(_.getName) - set("spark.kryo.classesToRegister", allClassNames.mkString(",")) - set("spark.serializer", classOf[KryoSerializer].getName) + set(KRYO_CLASSES_TO_REGISTER, allClassNames.toSeq) + set(SERIALIZER, classOf[KryoSerializer].getName) this } @@ -390,7 +399,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria /** Get an optional value, applying variable substitution. */ private[spark] def getWithSubstitution(key: String): Option[String] = { - getOption(key).map(reader.substitute(_)) + getOption(key).map(reader.substitute) } /** Get all parameters as a list of pairs */ @@ -406,7 +415,6 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria .map { case (k, v) => (k.substring(prefix.length), v) } } - /** * Get a parameter as an integer, falling back to a default if not set * @throws NumberFormatException If the value cannot be interpreted as an integer @@ -491,6 +499,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } + /** * Checks for illegal or deprecated config settings. Throws an exception for the former. Not * idempotent - may mutate this conf object to convert deprecated settings to supported ones. @@ -503,12 +512,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria logWarning(msg) } - val executorOptsKey = "spark.executor.extraJavaOptions" - val executorClasspathKey = "spark.executor.extraClassPath" - val driverOptsKey = "spark.driver.extraJavaOptions" - val driverClassPathKey = "spark.driver.extraClassPath" - val driverLibraryPathKey = "spark.driver.extraLibraryPath" - val sparkExecutorInstances = "spark.executor.instances" + val executorOptsKey = EXECUTOR_JAVA_OPTIONS.key // Used by Yarn in 1.1 and before sys.props.get("spark.driver.libraryPath").foreach { value => @@ -517,7 +521,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria |spark.driver.libraryPath was detected (set to '$value'). |This is deprecated in Spark 1.2+. | - |Please instead use: $driverLibraryPathKey + |Please instead use: ${DRIVER_LIBRARY_PATH.key} """.stripMargin logWarning(warning) } @@ -537,38 +541,13 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } // Validate memory fractions - val deprecatedMemoryKeys = Seq( - "spark.storage.memoryFraction", - "spark.shuffle.memoryFraction", - "spark.shuffle.safetyFraction", - "spark.storage.unrollFraction", - "spark.storage.safetyFraction") - val memoryKeys = Seq( - "spark.memory.fraction", - "spark.memory.storageFraction") ++ - deprecatedMemoryKeys - for (key <- memoryKeys) { + for (key <- Seq(MEMORY_FRACTION.key, MEMORY_STORAGE_FRACTION.key)) { val value = getDouble(key, 0.5) if (value > 1 || value < 0) { throw new IllegalArgumentException(s"$key should be between 0 and 1 (was '$value').") } } - // Warn against deprecated memory fractions (unless legacy memory management mode is enabled) - val legacyMemoryManagementKey = "spark.memory.useLegacyMode" - val legacyMemoryManagement = getBoolean(legacyMemoryManagementKey, false) - if (!legacyMemoryManagement) { - val keyset = deprecatedMemoryKeys.toSet - val detected = settings.keys().asScala.filter(keyset.contains) - if (detected.nonEmpty) { - logWarning("Detected deprecated memory fraction settings: " + - detected.mkString("[", ", ", "]") + ". As of Spark 1.6, execution and storage " + - "memory management are unified. All memory fractions used in the old model are " + - "now deprecated and no longer read. If you wish to use the old memory management, " + - s"you may explicitly enable `$legacyMemoryManagementKey` (not recommended).") - } - } - if (contains("spark.master") && get("spark.master").startsWith("yarn-")) { val warning = s"spark.master ${get("spark.master")} is deprecated in Spark 2.0+, please " + "instead use \"yarn\" with specified deploy mode." @@ -577,26 +556,26 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria case "yarn-cluster" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "cluster") + set(SUBMIT_DEPLOY_MODE, "cluster") case "yarn-client" => logWarning(warning) set("spark.master", "yarn") - set("spark.submit.deployMode", "client") + set(SUBMIT_DEPLOY_MODE, "client") case _ => // Any other unexpected master will be checked when creating scheduler backend. } } - if (contains("spark.submit.deployMode")) { - get("spark.submit.deployMode") match { + if (contains(SUBMIT_DEPLOY_MODE)) { + get(SUBMIT_DEPLOY_MODE) match { case "cluster" | "client" => - case e => throw new SparkException("spark.submit.deployMode can only be \"cluster\" or " + - "\"client\".") + case e => throw new SparkException(s"${SUBMIT_DEPLOY_MODE.key} can only be " + + "\"cluster\" or \"client\".") } } - if (contains("spark.cores.max") && contains("spark.executor.cores")) { - val totalCores = getInt("spark.cores.max", 1) - val executorCores = getInt("spark.executor.cores", 1) + if (contains(CORES_MAX) && contains(EXECUTOR_CORES)) { + val totalCores = getInt(CORES_MAX.key, 1) + val executorCores = get(EXECUTOR_CORES) val leftCores = totalCores % executorCores if (leftCores != 0) { logWarning(s"Total executor cores: ${totalCores} is not " + @@ -605,18 +584,18 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria } } - val encryptionEnabled = get(NETWORK_ENCRYPTION_ENABLED) || get(SASL_ENCRYPTION_ENABLED) + val encryptionEnabled = get(NETWORK_CRYPTO_ENABLED) || get(SASL_ENCRYPTION_ENABLED) require(!encryptionEnabled || get(NETWORK_AUTH_ENABLED), s"${NETWORK_AUTH_ENABLED.key} must be enabled when enabling encryption.") - val executorTimeoutThresholdMs = - getTimeAsSeconds("spark.network.timeout", "120s") * 1000 + val executorTimeoutThresholdMs = get(NETWORK_TIMEOUT) * 1000 val executorHeartbeatIntervalMs = get(EXECUTOR_HEARTBEAT_INTERVAL) + val networkTimeout = NETWORK_TIMEOUT.key // If spark.executor.heartbeatInterval bigger than spark.network.timeout, // it will almost always cause ExecutorLostFailure. See SPARK-22754. require(executorTimeoutThresholdMs > executorHeartbeatIntervalMs, "The value of " + - s"spark.network.timeout=${executorTimeoutThresholdMs}ms must be no less than the value of " + - s"spark.executor.heartbeatInterval=${executorHeartbeatIntervalMs}ms.") + s"${networkTimeout}=${executorTimeoutThresholdMs}ms must be no less than the value of " + + s"${EXECUTOR_HEARTBEAT_INTERVAL.key}=${executorHeartbeatIntervalMs}ms.") } /** @@ -624,7 +603,7 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging with Seria * configuration out for debugging. */ def toDebugString: String = { - getAll.sorted.map{case (k, v) => k + "=" + v}.mkString("\n") + Utils.redact(this, getAll).sorted.map { case (k, v) => k + "=" + v }.mkString("\n") } } @@ -656,7 +635,8 @@ private[spark] object SparkConf extends Logging { DeprecatedConfig("spark.shuffle.service.index.cache.entries", "2.3.0", "Not used anymore. Please use spark.shuffle.service.index.cache.size"), DeprecatedConfig("spark.yarn.credentials.file.retention.count", "2.4.0", "Not used anymore."), - DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore.") + DeprecatedConfig("spark.yarn.credentials.file.retention.days", "2.4.0", "Not used anymore."), + DeprecatedConfig("spark.yarn.services", "3.0.0", "Feature no longer available.") ) Map(configs.map { cfg => (cfg.key -> cfg) } : _*) @@ -671,13 +651,13 @@ private[spark] object SparkConf extends Logging { * TODO: consolidate it with `ConfigBuilder.withAlternative`. */ private val configsWithAlternatives = Map[String, Seq[AlternateConfig]]( - "spark.executor.userClassPathFirst" -> Seq( + EXECUTOR_USER_CLASS_PATH_FIRST.key -> Seq( AlternateConfig("spark.files.userClassPathFirst", "1.3")), - "spark.history.fs.update.interval" -> Seq( + UPDATE_INTERVAL_S.key -> Seq( AlternateConfig("spark.history.fs.update.interval.seconds", "1.4"), AlternateConfig("spark.history.fs.updateInterval", "1.3"), AlternateConfig("spark.history.updateInterval", "1.3")), - "spark.history.fs.cleaner.interval" -> Seq( + CLEANER_INTERVAL_S.key -> Seq( AlternateConfig("spark.history.fs.cleaner.interval.seconds", "1.4")), MAX_LOG_AGE_S.key -> Seq( AlternateConfig("spark.history.fs.cleaner.maxAge.seconds", "1.4")), @@ -685,28 +665,28 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.yarn.applicationMaster.waitTries", "1.3", // Translate old value to a duration, with 10s wait time per try. translation = s => s"${s.toLong * 10}s")), - "spark.reducer.maxSizeInFlight" -> Seq( + REDUCER_MAX_SIZE_IN_FLIGHT.key -> Seq( AlternateConfig("spark.reducer.maxMbInFlight", "1.4")), - "spark.kryoserializer.buffer" -> Seq( + KRYO_SERIALIZER_BUFFER_SIZE.key -> Seq( AlternateConfig("spark.kryoserializer.buffer.mb", "1.4", translation = s => s"${(s.toDouble * 1000).toInt}k")), - "spark.kryoserializer.buffer.max" -> Seq( + KRYO_SERIALIZER_MAX_BUFFER_SIZE.key -> Seq( AlternateConfig("spark.kryoserializer.buffer.max.mb", "1.4")), - "spark.shuffle.file.buffer" -> Seq( + SHUFFLE_FILE_BUFFER_SIZE.key -> Seq( AlternateConfig("spark.shuffle.file.buffer.kb", "1.4")), - "spark.executor.logs.rolling.maxSize" -> Seq( + EXECUTOR_LOGS_ROLLING_MAX_SIZE.key -> Seq( AlternateConfig("spark.executor.logs.rolling.size.maxBytes", "1.4")), - "spark.io.compression.snappy.blockSize" -> Seq( + IO_COMPRESSION_SNAPPY_BLOCKSIZE.key -> Seq( AlternateConfig("spark.io.compression.snappy.block.size", "1.4")), - "spark.io.compression.lz4.blockSize" -> Seq( + IO_COMPRESSION_LZ4_BLOCKSIZE.key -> Seq( AlternateConfig("spark.io.compression.lz4.block.size", "1.4")), - "spark.rpc.numRetries" -> Seq( + RPC_NUM_RETRIES.key -> Seq( AlternateConfig("spark.akka.num.retries", "1.4")), - "spark.rpc.retry.wait" -> Seq( + RPC_RETRY_WAIT.key -> Seq( AlternateConfig("spark.akka.retry.wait", "1.4")), - "spark.rpc.askTimeout" -> Seq( + RPC_ASK_TIMEOUT.key -> Seq( AlternateConfig("spark.akka.askTimeout", "1.4")), - "spark.rpc.lookupTimeout" -> Seq( + RPC_LOOKUP_TIMEOUT.key -> Seq( AlternateConfig("spark.akka.lookupTimeout", "1.4")), "spark.streaming.fileStream.minRememberDuration" -> Seq( AlternateConfig("spark.streaming.minRememberDuration", "1.5")), @@ -714,12 +694,10 @@ private[spark] object SparkConf extends Logging { AlternateConfig("spark.yarn.max.worker.failures", "1.5")), MEMORY_OFFHEAP_ENABLED.key -> Seq( AlternateConfig("spark.unsafe.offHeap", "1.6")), - "spark.rpc.message.maxSize" -> Seq( + RPC_MESSAGE_MAX_SIZE.key -> Seq( AlternateConfig("spark.akka.frameSize", "1.6")), "spark.yarn.jars" -> Seq( AlternateConfig("spark.yarn.jar", "2.0")), - "spark.yarn.access.hadoopFileSystems" -> Seq( - AlternateConfig("spark.yarn.access.namenodes", "2.2")), MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM.key -> Seq( AlternateConfig("spark.reducer.maxReqSizeShuffleToMem", "2.3")), LISTENER_BUS_EVENT_QUEUE_CAPACITY.key -> Seq( @@ -729,9 +707,16 @@ private[spark] object SparkConf extends Logging { EXECUTOR_MEMORY_OVERHEAD.key -> Seq( AlternateConfig("spark.yarn.executor.memoryOverhead", "2.3")), KEYTAB.key -> Seq( - AlternateConfig("spark.yarn.keytab", "2.5")), + AlternateConfig("spark.yarn.keytab", "3.0")), PRINCIPAL.key -> Seq( - AlternateConfig("spark.yarn.principal", "2.5")) + AlternateConfig("spark.yarn.principal", "3.0")), + KERBEROS_RELOGIN_PERIOD.key -> Seq( + AlternateConfig("spark.yarn.kerberos.relogin.period", "3.0")), + KERBEROS_FILESYSTEMS_TO_ACCESS.key -> Seq( + AlternateConfig("spark.yarn.access.namenodes", "2.2"), + AlternateConfig("spark.yarn.access.hadoopFileSystems", "3.0")), + "spark.kafka.consumer.cache.capacity" -> Seq( + AlternateConfig("spark.sql.kafkaConsumerCache.capacity", "3.0")) ) /** @@ -754,7 +739,6 @@ private[spark] object SparkConf extends Logging { */ def isExecutorStartupConf(name: String): Boolean = { (name.startsWith("spark.auth") && name != SecurityManager.SPARK_AUTH_SECRET_CONF) || - name.startsWith("spark.ssl") || name.startsWith("spark.rpc") || name.startsWith("spark.network") || isSparkPortConf(name) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 0a66dae94dbd0..75182b0c9008b 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,7 +25,6 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger, AtomicReferenc import scala.collection.JavaConverters._ import scala.collection.Map -import scala.collection.generic.Growable import scala.collection.mutable.HashMap import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} @@ -46,27 +45,32 @@ import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.{FixedLengthBinaryInputFormat, PortableDataStream, StreamInputFormat, WholeTextFileInputFormat} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec +import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ +import org.apache.spark.resource.{ResourceID, ResourceInformation} +import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, StandaloneSchedulerBackend} +import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend -import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.{AppStatusSource, AppStatusStore} import org.apache.spark.status.api.v1.ThreadStackTrace import org.apache.spark.storage._ import org.apache.spark.storage.BlockManagerMessages.TriggerThreadDump import org.apache.spark.ui.{ConsoleProgressBar, SparkUI} import org.apache.spark.util._ +import org.apache.spark.util.logging.DriverLogger /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * - * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before - * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. - * + * @note Only one `SparkContext` should be active per JVM. You must `stop()` the + * active `SparkContext` before creating a new one. * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. */ @@ -75,14 +79,10 @@ class SparkContext(config: SparkConf) extends Logging { // The call site where this SparkContext was constructed. private val creationSite: CallSite = Utils.getCallSite() - // If true, log warnings instead of throwing exceptions when multiple SparkContexts are active - private val allowMultipleContexts: Boolean = - config.getBoolean("spark.driver.allowMultipleContexts", false) - // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having started construction. // NOTE: this must be placed at the beginning of the SparkContext constructor. - SparkContext.markPartiallyConstructed(this, allowMultipleContexts) + SparkContext.markPartiallyConstructed(this) val startTime = System.currentTimeMillis() @@ -206,6 +206,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _applicationId: String = _ private var _applicationAttemptId: Option[String] = None private var _eventLogger: Option[EventLoggingListener] = None + private var _driverLogger: Option[DriverLogger] = None private var _executorAllocationManager: Option[ExecutorAllocationManager] = None private var _cleaner: Option[ContextCleaner] = None private var _listenerBusStarted: Boolean = false @@ -214,6 +215,7 @@ class SparkContext(config: SparkConf) extends Logging { private var _shutdownHookRef: AnyRef = _ private var _statusStore: AppStatusStore = _ private var _heartbeater: Heartbeater = _ + private var _resources: scala.collection.immutable.Map[String, ResourceInformation] = _ /* ------------------------------------------------------------------------------------- * | Accessors and public fields. These provide access to the internal state of the | @@ -228,13 +230,15 @@ class SparkContext(config: SparkConf) extends Logging { */ def getConf: SparkConf = conf.clone() + def resources: Map[String, ResourceInformation] = _resources + def jars: Seq[String] = _jars def files: Seq[String] = _files def master: String = _conf.get("spark.master") - def deployMode: String = _conf.getOption("spark.submit.deployMode").getOrElse("client") + def deployMode: String = _conf.get(SUBMIT_DEPLOY_MODE) def appName: String = _conf.get("spark.app.name") - private[spark] def isEventLogEnabled: Boolean = _conf.getBoolean("spark.eventLog.enabled", false) + private[spark] def isEventLogEnabled: Boolean = _conf.get(EVENT_LOG_ENABLED) private[spark] def eventLogDir: Option[URI] = _eventLogDir private[spark] def eventLogCodec: Option[String] = _eventLogCodec @@ -372,6 +376,11 @@ class SparkContext(config: SparkConf) extends Logging { throw new SparkException("An application name must be set in your configuration") } + _driverLogger = DriverLogger(_conf) + + val resourcesFileOpt = conf.get(DRIVER_RESOURCES_FILE) + _resources = getOrDiscoverAllResources(_conf, SPARK_DRIVER_PREFIX, resourcesFileOpt) + // log out spark.app.name in the Spark driver logs logInfo(s"Submitted application: $appName") @@ -388,25 +397,24 @@ class SparkContext(config: SparkConf) extends Logging { // Set Spark driver host and port system properties. This explicitly sets the configuration // instead of relying on the default value of the config constant. _conf.set(DRIVER_HOST_ADDRESS, _conf.get(DRIVER_HOST_ADDRESS)) - _conf.setIfMissing("spark.driver.port", "0") + _conf.setIfMissing(DRIVER_PORT, 0) - _conf.set("spark.executor.id", SparkContext.DRIVER_IDENTIFIER) + _conf.set(EXECUTOR_ID, SparkContext.DRIVER_IDENTIFIER) _jars = Utils.getUserJars(_conf) - _files = _conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.nonEmpty)) + _files = _conf.getOption(FILES.key).map(_.split(",")).map(_.filter(_.nonEmpty)) .toSeq.flatten _eventLogDir = if (isEventLogEnabled) { - val unresolvedDir = conf.get("spark.eventLog.dir", EventLoggingListener.DEFAULT_LOG_DIR) - .stripSuffix("/") + val unresolvedDir = conf.get(EVENT_LOG_DIR).stripSuffix("/") Some(Utils.resolveURI(unresolvedDir)) } else { None } _eventLogCodec = { - val compress = _conf.getBoolean("spark.eventLog.compress", false) + val compress = _conf.get(EVENT_LOG_COMPRESS) if (compress && isEventLogEnabled) { Some(CompressionCodec.getCodecName(_conf)).map(CompressionCodec.getShortName) } else { @@ -418,7 +426,8 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the app status store and listener before SparkEnv is created so that it gets // all events. - _statusStore = AppStatusStore.createLiveStore(conf) + val appStatusSource = AppStatusSource.createSource(conf) + _statusStore = AppStatusStore.createLiveStore(conf, appStatusSource) listenerBus.addToStatusQueue(_statusStore.listener.get) // Create the Spark execution environment (cache, map output tracker, etc) @@ -434,14 +443,14 @@ class SparkContext(config: SparkConf) extends Logging { _statusTracker = new SparkStatusTracker(this, _statusStore) _progressBar = - if (_conf.get(UI_SHOW_CONSOLE_PROGRESS) && !log.isInfoEnabled) { + if (_conf.get(UI_SHOW_CONSOLE_PROGRESS)) { Some(new ConsoleProgressBar(this)) } else { None } _ui = - if (conf.getBoolean("spark.ui.enabled", true)) { + if (conf.get(UI_ENABLED)) { Some(SparkUI.create(Some(this), _statusStore, _conf, _env.securityManager, appName, "", startTime)) } else { @@ -453,6 +462,15 @@ class SparkContext(config: SparkConf) extends Logging { _ui.foreach(_.bind()) _hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(_conf) + // Performance optimization: this dummy call to .size() triggers eager evaluation of + // Configuration's internal `properties` field, guaranteeing that it will be computed and + // cached before SessionState.newHadoopConf() uses `sc.hadoopConfiguration` to create + // a new per-session Configuration. If `properties` has not been computed by that time + // then each newly-created Configuration will perform its own expensive IO and XML + // parsing to load configuration defaults and populate its own properties. By ensuring + // that we've pre-computed the parent's properties, the child Configuration will simply + // clone the parent's properties. + _hadoopConfiguration.size() // Add each JAR given through the constructor if (jars != null) { @@ -463,7 +481,7 @@ class SparkContext(config: SparkConf) extends Logging { files.foreach(addFile) } - _executorMemory = _conf.getOption("spark.executor.memory") + _executorMemory = _conf.getOption(EXECUTOR_MEMORY.key) .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) .orElse(Option(System.getenv("SPARK_MEM")) .map(warnSparkMem)) @@ -472,7 +490,7 @@ class SparkContext(config: SparkConf) extends Logging { // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", IS_TESTING.key)) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value } @@ -498,7 +516,9 @@ class SparkContext(config: SparkConf) extends Logging { _heartbeatReceiver.ask[Boolean](TaskSchedulerIsSet) // create and start the heartbeater for collecting memory metrics - _heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, "driver-heartbeater", + _heartbeater = new Heartbeater(env.memoryManager, + () => SparkContext.this.reportHeartBeat(), + "driver-heartbeater", conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) _heartbeater.start() @@ -507,9 +527,9 @@ class SparkContext(config: SparkConf) extends Logging { _taskScheduler.start() _applicationId = _taskScheduler.applicationId() - _applicationAttemptId = taskScheduler.applicationAttemptId() + _applicationAttemptId = _taskScheduler.applicationAttemptId() _conf.set("spark.app.id", _applicationId) - if (_conf.getBoolean("spark.ui.reverseProxy", false)) { + if (_conf.get(UI_REVERSE_PROXY)) { System.setProperty("spark.ui.proxyBase", "/proxy/" + _applicationId) } _ui.foreach(_.setAppId(_applicationId)) @@ -533,7 +553,14 @@ class SparkContext(config: SparkConf) extends Logging { None } - // Optionally scale number of executors dynamically based on workload. Exposed for testing. + _cleaner = + if (_conf.get(CLEANER_REFERENCE_TRACKING)) { + Some(new ContextCleaner(this)) + } else { + None + } + _cleaner.foreach(_.start()) + val dynamicAllocationEnabled = Utils.isDynamicAllocationEnabled(_conf) _executorAllocationManager = if (dynamicAllocationEnabled) { @@ -541,7 +568,7 @@ class SparkContext(config: SparkConf) extends Logging { case b: ExecutorAllocationClient => Some(new ExecutorAllocationManager( schedulerBackend.asInstanceOf[ExecutorAllocationClient], listenerBus, _conf, - _env.blockManager.master)) + cleaner = cleaner)) case _ => None } @@ -550,14 +577,6 @@ class SparkContext(config: SparkConf) extends Logging { } _executorAllocationManager.foreach(_.start()) - _cleaner = - if (_conf.getBoolean("spark.cleaner.referenceTracking", true)) { - Some(new ContextCleaner(this)) - } else { - None - } - _cleaner.foreach(_.start()) - setupAndStartListenerBus() postEnvironmentUpdate() postApplicationStart() @@ -566,10 +585,11 @@ class SparkContext(config: SparkConf) extends Logging { _taskScheduler.postStartHook() _env.metricsSystem.registerSource(_dagScheduler.metricsSource) _env.metricsSystem.registerSource(new BlockManagerSource(_env.blockManager)) + _env.metricsSystem.registerSource(new JVMCPUSource()) _executorAllocationManager.foreach { e => _env.metricsSystem.registerSource(e.executorAllocationManagerSource) } - + appStatusSource.foreach(_env.metricsSystem.registerSource(_)) // Make sure the context is stopped if the user forgets about it. This avoids leaving // unfinished event logs around after the JVM exits cleanly. It doesn't help if the JVM // is killed, though. @@ -825,6 +845,8 @@ class SparkContext(config: SparkConf) extends Logging { /** * Read a text file from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI, and return it as an RDD of Strings. + * The text files must be encoded as UTF-8. + * * @param path path to the text file on a supported file system * @param minPartitions suggested minimum number of partitions for the resulting RDD * @return RDD of lines of the text file @@ -841,6 +863,7 @@ class SparkContext(config: SparkConf) extends Logging { * Read a directory of text files from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI. Each file is read as a single record and returned in a * key-value pair, where the key is the path of each file, the value is the content of each file. + * The text files must be encoded as UTF-8. * *

For example, if you have the following files: * {{{ @@ -1037,7 +1060,7 @@ class SparkContext(config: SparkConf) extends Logging { // See SPARK-11227 for details. FileSystem.getLocal(hadoopConfiguration) - // A Hadoop configuration can be about 10 KB, which is pretty big, so broadcast it. + // A Hadoop configuration can be about 10 KiB, which is pretty big, so broadcast it. val confBroadcast = broadcast(new SerializableConfiguration(hadoopConfiguration)) val setInputPathsFunc = (jobConf: JobConf) => FileInputFormat.setInputPaths(jobConf, path) new HadoopRDD( @@ -1336,76 +1359,6 @@ class SparkContext(config: SparkConf) extends Logging { // Methods for creating shared variables - /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" - * values to using the `+=` method. Only the driver can access the accumulator's `value`. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulator[T](initialValue: T)(implicit param: AccumulatorParam[T]): Accumulator[T] = { - val acc = new Accumulator(initialValue, param) - cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) - acc - } - - /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, with a name for display - * in the Spark UI. Tasks can "add" values to the accumulator using the `+=` method. Only the - * driver can access the accumulator's `value`. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulator[T](initialValue: T, name: String)(implicit param: AccumulatorParam[T]) - : Accumulator[T] = { - val acc = new Accumulator(initialValue, param, Option(name)) - cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) - acc - } - - /** - * Create an [[org.apache.spark.Accumulable]] shared variable, to which tasks can add values - * with `+=`. Only the driver can access the accumulable's `value`. - * @tparam R accumulator result type - * @tparam T type that can be added to the accumulator - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulable[R, T](initialValue: R)(implicit param: AccumulableParam[R, T]) - : Accumulable[R, T] = { - val acc = new Accumulable(initialValue, param) - cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) - acc - } - - /** - * Create an [[org.apache.spark.Accumulable]] shared variable, with a name for display in the - * Spark UI. Tasks can add values to the accumulable using the `+=` operator. Only the driver can - * access the accumulable's `value`. - * @tparam R accumulator result type - * @tparam T type that can be added to the accumulator - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulable[R, T](initialValue: R, name: String)(implicit param: AccumulableParam[R, T]) - : Accumulable[R, T] = { - val acc = new Accumulable(initialValue, param, Option(name)) - cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) - acc - } - - /** - * Create an accumulator from a "mutable collection" type. - * - * Growable and TraversableOnce are the standard APIs that guarantee += and ++=, implemented by - * standard mutable collections. So you can use this with mutable Map, Set, etc. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable: ClassTag, T] - (initialValue: R): Accumulable[R, T] = { - // TODO the context bound (<%) above should be replaced with simple type bound and implicit - // conversion but is a breaking change. This should be fixed in Spark 3.x. - val param = new GrowableAccumulableParam[R, T] - val acc = new Accumulable(initialValue, param) - cleaner.foreach(_.registerAccumulatorForCleanup(acc.newAcc)) - acc - } - /** * Register the given accumulator. * @@ -1821,7 +1774,7 @@ class SparkContext(config: SparkConf) extends Logging { /** * Unpersist an RDD from memory and/or disk storage */ - private[spark] def unpersistRDD(rddId: Int, blocking: Boolean = true) { + private[spark] def unpersistRDD(rddId: Int, blocking: Boolean) { env.blockManager.master.removeRdd(rddId, blocking) persistentRdds.remove(rddId) listenerBus.post(SparkListenerUnpersistRDD(rddId)) @@ -1838,7 +1791,7 @@ class SparkContext(config: SparkConf) extends Logging { * @note A path can be added only once. Subsequent additions of the same path are ignored. */ def addJar(path: String) { - def addJarFile(file: File): String = { + def addLocalJarFile(file: File): String = { try { if (!file.exists()) { throw new FileNotFoundException(s"Jar ${file.getAbsolutePath} not found") @@ -1855,12 +1808,36 @@ class SparkContext(config: SparkConf) extends Logging { } } + def checkRemoteJarFile(path: String): String = { + val hadoopPath = new Path(path) + val scheme = new URI(path).getScheme + if (!Array("http", "https", "ftp").contains(scheme)) { + try { + val fs = hadoopPath.getFileSystem(hadoopConfiguration) + if (!fs.exists(hadoopPath)) { + throw new FileNotFoundException(s"Jar ${path} not found") + } + if (fs.isDirectory(hadoopPath)) { + throw new IllegalArgumentException( + s"Directory ${path} is not allowed for addJar") + } + path + } catch { + case NonFatal(e) => + logError(s"Failed to add $path to Spark environment", e) + null + } + } else { + path + } + } + if (path == null) { logWarning("null specified as parameter to addJar") } else { val key = if (path.contains("\\")) { // For local paths with backslashes on Windows, URI throws an exception - addJarFile(new File(path)) + addLocalJarFile(new File(path)) } else { val uri = new URI(path) // SPARK-17650: Make sure this is a valid URL before adding it to the list of dependencies @@ -1869,12 +1846,12 @@ class SparkContext(config: SparkConf) extends Logging { // A JAR file which exists only on the driver node case null => // SPARK-22585 path without schema is not url encoded - addJarFile(new File(uri.getRawPath)) + addLocalJarFile(new File(uri.getRawPath)) // A JAR file which exists only on the driver node - case "file" => addJarFile(new File(uri.getPath)) + case "file" => addLocalJarFile(new File(uri.getPath)) // A JAR file which exists locally on every worker node case "local" => "file:" + uri.getPath - case _ => path + case _ => checkRemoteJarFile(path) } } if (key != null) { @@ -1936,6 +1913,9 @@ class SparkContext(config: SparkConf) extends Logging { Utils.tryLogNonFatalError { postApplicationEnd() } + Utils.tryLogNonFatalError { + _driverLogger.foreach(_.stop()) + } Utils.tryLogNonFatalError { _ui.foreach(_.stop()) } @@ -2420,7 +2400,9 @@ class SparkContext(config: SparkConf) extends Logging { // Note: this code assumes that the task scheduler has been initialized and has contacted // the cluster manager to get an application ID (in case the cluster manager provides one). listenerBus.post(SparkListenerApplicationStart(appName, Some(applicationId), - startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls)) + startTime, sparkUser, applicationAttemptId, schedulerBackend.getDriverLogUrls, + schedulerBackend.getDriverAttributes)) + _driverLogger.foreach(_.startSync(_hadoopConfiguration)) } /** Post the application end event */ @@ -2434,8 +2416,8 @@ class SparkContext(config: SparkConf) extends Logging { val schedulingMode = getSchedulingMode.toString val addedJarPaths = addedJars.keys.toSeq val addedFilePaths = addedFiles.keys.toSeq - val environmentDetails = SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, - addedFilePaths) + val environmentDetails = SparkEnv.environmentDetails(conf, hadoopConfiguration, + schedulingMode, addedJarPaths, addedFilePaths) val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) listenerBus.post(environmentUpdate) } @@ -2452,7 +2434,7 @@ class SparkContext(config: SparkConf) extends Logging { // In order to prevent multiple SparkContexts from being active at the same time, mark this // context as having finished construction. // NOTE: this must be placed at the end of the SparkContext constructor. - SparkContext.setActiveContext(this, allowMultipleContexts) + SparkContext.setActiveContext(this) } /** @@ -2469,18 +2451,18 @@ object SparkContext extends Logging { private val SPARK_CONTEXT_CONSTRUCTOR_LOCK = new Object() /** - * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. + * The active, fully-constructed SparkContext. If no SparkContext is active, then this is `null`. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK. + * Access to this field is guarded by `SPARK_CONTEXT_CONSTRUCTOR_LOCK`. */ private val activeContext: AtomicReference[SparkContext] = new AtomicReference[SparkContext](null) /** - * Points to a partially-constructed SparkContext if some thread is in the SparkContext + * Points to a partially-constructed SparkContext if another thread is in the SparkContext * constructor, or `None` if no SparkContext is being constructed. * - * Access to this field is guarded by SPARK_CONTEXT_CONSTRUCTOR_LOCK + * Access to this field is guarded by `SPARK_CONTEXT_CONSTRUCTOR_LOCK`. */ private var contextBeingConstructed: Option[SparkContext] = None @@ -2488,24 +2470,16 @@ object SparkContext extends Logging { * Called to ensure that no other SparkContext is running in this JVM. * * Throws an exception if a running context is detected and logs a warning if another thread is - * constructing a SparkContext. This warning is necessary because the current locking scheme + * constructing a SparkContext. This warning is necessary because the current locking scheme * prevents us from reliably distinguishing between cases where another context is being * constructed and cases where another constructor threw an exception. */ - private def assertNoOtherContextIsRunning( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private def assertNoOtherContextIsRunning(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { Option(activeContext.get()).filter(_ ne sc).foreach { ctx => - val errMsg = "Only one SparkContext may be running in this JVM (see SPARK-2243)." + - " To ignore this error, set spark.driver.allowMultipleContexts = true. " + + val errMsg = "Only one SparkContext should be running in this JVM (see SPARK-2243)." + s"The currently running SparkContext was created at:\n${ctx.creationSite.longForm}" - val exception = new SparkException(errMsg) - if (allowMultipleContexts) { - logWarning("Multiple running SparkContexts detected in the same JVM!", exception) - } else { - throw exception - } + throw new SparkException(errMsg) } contextBeingConstructed.filter(_ ne sc).foreach { otherContext => @@ -2514,7 +2488,7 @@ object SparkContext extends Logging { val otherContextCreationSite = Option(otherContext.creationSite).map(_.longForm).getOrElse("unknown location") val warnMsg = "Another SparkContext is being constructed (or threw an exception in its" + - " constructor). This may indicate an error, since only one SparkContext may be" + + " constructor). This may indicate an error, since only one SparkContext should be" + " running in this JVM (see SPARK-2243)." + s" The other SparkContext was created at:\n$otherContextCreationSite" logWarning(warnMsg) @@ -2527,8 +2501,6 @@ object SparkContext extends Logging { * singleton object. Because we can only have one active SparkContext per JVM, * this is useful when applications may wish to share a SparkContext. * - * @note This function cannot be used to create multiple SparkContext instances - * even if multiple contexts are allowed. * @param config `SparkConfig` that will be used for initialisation of the `SparkContext` * @return current `SparkContext` (or a new one if it wasn't created before the function call) */ @@ -2537,7 +2509,7 @@ object SparkContext extends Logging { // from assertNoOtherContextIsRunning within setActiveContext SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { if (activeContext.get() == null) { - setActiveContext(new SparkContext(config), allowMultipleContexts = false) + setActiveContext(new SparkContext(config)) } else { if (config.getAll.nonEmpty) { logWarning("Using an existing SparkContext; some configuration may not take effect.") @@ -2554,14 +2526,12 @@ object SparkContext extends Logging { * * This method allows not passing a SparkConf (useful if just retrieving). * - * @note This function cannot be used to create multiple SparkContext instances - * even if multiple contexts are allowed. * @return current `SparkContext` (or a new one if wasn't created before the function call) */ def getOrCreate(): SparkContext = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { if (activeContext.get() == null) { - setActiveContext(new SparkContext(), allowMultipleContexts = false) + setActiveContext(new SparkContext()) } activeContext.get() } @@ -2576,16 +2546,14 @@ object SparkContext extends Logging { /** * Called at the beginning of the SparkContext constructor to ensure that no SparkContext is - * running. Throws an exception if a running context is detected and logs a warning if another - * thread is constructing a SparkContext. This warning is necessary because the current locking + * running. Throws an exception if a running context is detected and logs a warning if another + * thread is constructing a SparkContext. This warning is necessary because the current locking * scheme prevents us from reliably distinguishing between cases where another context is being * constructed and cases where another constructor threw an exception. */ - private[spark] def markPartiallyConstructed( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private[spark] def markPartiallyConstructed(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - assertNoOtherContextIsRunning(sc, allowMultipleContexts) + assertNoOtherContextIsRunning(sc) contextBeingConstructed = Some(sc) } } @@ -2594,18 +2562,16 @@ object SparkContext extends Logging { * Called at the end of the SparkContext constructor to ensure that no other SparkContext has * raced with this constructor and started. */ - private[spark] def setActiveContext( - sc: SparkContext, - allowMultipleContexts: Boolean): Unit = { + private[spark] def setActiveContext(sc: SparkContext): Unit = { SPARK_CONTEXT_CONSTRUCTOR_LOCK.synchronized { - assertNoOtherContextIsRunning(sc, allowMultipleContexts) + assertNoOtherContextIsRunning(sc) contextBeingConstructed = None activeContext.set(sc) } } /** - * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's + * Clears the active SparkContext metadata. This is called by `SparkContext#stop()`. It's * also called in unit tests to prevent a flood of warnings from test suites that don't / can't * properly clean up their SparkContexts. */ @@ -2618,6 +2584,7 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" + private[spark] val SPARK_SCHEDULER_POOL = "spark.scheduler.pool" private[spark] val RDD_SCOPE_KEY = "spark.rdd.scope" private[spark] val RDD_SCOPE_NO_OVERRIDE_KEY = "spark.rdd.scope.noOverride" @@ -2628,12 +2595,8 @@ object SparkContext extends Logging { */ private[spark] val DRIVER_IDENTIFIER = "driver" - /** - * Legacy version of DRIVER_IDENTIFIER, retained for backwards-compatibility. - */ - private[spark] val LEGACY_DRIVER_IDENTIFIER = "" - private implicit def arrayToArrayWritable[T <: Writable : ClassTag](arr: Traversable[T]) + private implicit def arrayToArrayWritable[T <: Writable : ClassTag](arr: Iterable[T]) : ArrayWritable = { def anyToWritable[U <: Writable](u: U): Writable = u @@ -2720,8 +2683,8 @@ object SparkContext extends Logging { case SparkMasterRegex.LOCAL_N_REGEX(threads) => convertToInt(threads) case SparkMasterRegex.LOCAL_N_FAILURES_REGEX(threads, _) => convertToInt(threads) case "yarn" => - if (conf != null && conf.getOption("spark.submit.deployMode").contains("cluster")) { - conf.getInt("spark.driver.cores", 0) + if (conf != null && conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { + conf.getInt(DRIVER_CORES.key, 0) } else { 0 } @@ -2742,8 +2705,75 @@ object SparkContext extends Logging { // When running locally, don't try to re-execute tasks on failure. val MAX_LOCAL_TASK_FAILURES = 1 + // Ensure that executor's resources satisfies one or more tasks requirement. + def checkResourcesPerTask(clusterMode: Boolean, executorCores: Option[Int]): Unit = { + val taskCores = sc.conf.get(CPUS_PER_TASK) + val execCores = if (clusterMode) { + executorCores.getOrElse(sc.conf.get(EXECUTOR_CORES)) + } else { + executorCores.get + } + + // Number of cores per executor must meet at least one task requirement. + if (execCores < taskCores) { + throw new SparkException(s"The number of cores per executor (=$execCores) has to be >= " + + s"the task config: ${CPUS_PER_TASK.key} = $taskCores when run on $master.") + } + + // Calculate the max slots each executor can provide based on resources available on each + // executor and resources required by each task. + val taskResourceRequirements = parseTaskResourceRequirements(sc.conf) + val executorResourcesAndAmounts = + parseAllResourceRequests(sc.conf, SPARK_EXECUTOR_PREFIX) + .map(request => (request.id.resourceName, request.amount)).toMap + var numSlots = execCores / taskCores + var limitingResourceName = "CPU" + + taskResourceRequirements.foreach { taskReq => + // Make sure the executor resources were specified through config. + val execAmount = executorResourcesAndAmounts.getOrElse(taskReq.resourceName, + throw new SparkException("The executor resource config: " + + ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + + " needs to be specified since a task requirement config: " + + ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + + " was specified") + ) + // Make sure the executor resources are large enough to launch at least one task. + if (execAmount < taskReq.amount) { + throw new SparkException("The executor resource config: " + + ResourceID(SPARK_EXECUTOR_PREFIX, taskReq.resourceName).amountConf + + s" = $execAmount has to be >= the requested amount in task resource config: " + + ResourceID(SPARK_TASK_PREFIX, taskReq.resourceName).amountConf + + s" = ${taskReq.amount}") + } + // Compare and update the max slots each executor can provide. + val resourceNumSlots = execAmount / taskReq.amount + if (resourceNumSlots < numSlots) { + numSlots = resourceNumSlots + limitingResourceName = taskReq.resourceName + } + } + // There have been checks above to make sure the executor resources were specified and are + // large enough if any task resources were specified. + taskResourceRequirements.foreach { taskReq => + val execAmount = executorResourcesAndAmounts(taskReq.resourceName) + if (taskReq.amount * numSlots < execAmount) { + val message = s"The configuration of resource: ${taskReq.resourceName} " + + s"(exec = ${execAmount}, task = ${taskReq.amount}) will result in wasted " + + s"resources due to resource ${limitingResourceName} limiting the number of " + + s"runnable tasks per executor to: ${numSlots}. Please adjust your configuration." + if (Utils.isTesting) { + throw new SparkException(message) + } else { + logWarning(message) + } + } + } + } + master match { case "local" => + checkResourcesPerTask(clusterMode = false, Some(1)) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, 1) scheduler.initialize(backend) @@ -2756,6 +2786,7 @@ object SparkContext extends Logging { if (threadCount <= 0) { throw new SparkException(s"Asked to run locally with $threadCount threads") } + checkResourcesPerTask(clusterMode = false, Some(threadCount)) val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) @@ -2766,12 +2797,14 @@ object SparkContext extends Logging { // local[*, M] means the number of cores on the computer with M failures // local[N, M] means exactly N threads with M failures val threadCount = if (threads == "*") localCpuCount else threads.toInt + checkResourcesPerTask(clusterMode = false, Some(threadCount)) val scheduler = new TaskSchedulerImpl(sc, maxFailures.toInt, isLocal = true) val backend = new LocalSchedulerBackend(sc.getConf, scheduler, threadCount) scheduler.initialize(backend) (backend, scheduler) case SPARK_REGEX(sparkUrl) => + checkResourcesPerTask(clusterMode = true, None) val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) val backend = new StandaloneSchedulerBackend(scheduler, sc, masterUrls) @@ -2779,11 +2812,12 @@ object SparkContext extends Logging { (backend, scheduler) case LOCAL_CLUSTER_REGEX(numSlaves, coresPerSlave, memoryPerSlave) => + checkResourcesPerTask(clusterMode = true, Some(coresPerSlave.toInt)) // Check to make sure memory requested <= memoryPerSlave. Otherwise Spark will just hang. val memoryPerSlaveInt = memoryPerSlave.toInt if (sc.executorMemory > memoryPerSlaveInt) { throw new SparkException( - "Asked to launch cluster with %d MB RAM / worker but requested %d MB/worker".format( + "Asked to launch cluster with %d MiB RAM / worker but requested %d MiB/worker".format( memoryPerSlaveInt, sc.executorMemory)) } @@ -2799,6 +2833,7 @@ object SparkContext extends Logging { (backend, scheduler) case masterUrl => + checkResourcesPerTask(clusterMode = true, None) val cm = getClusterManager(masterUrl) match { case Some(clusterMgr) => clusterMgr case None => throw new SparkException("Could not parse Master URL: '" + master + "'") diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 72123f2232532..4e778a1ddd5fa 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -21,19 +21,22 @@ import java.io.File import java.net.Socket import java.util.Locale +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.util.Properties import com.google.common.collect.MapMaker +import org.apache.hadoop.conf.Configuration import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ -import org.apache.spark.memory.{MemoryManager, StaticMemoryManager, UnifiedMemoryManager} -import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.network.netty.NettyBlockTransferService +import org.apache.spark.memory.{MemoryManager, UnifiedMemoryManager} +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} +import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} +import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{LiveListenerBus, OutputCommitCoordinator} import org.apache.spark.scheduler.OutputCommitCoordinator.OutputCommitCoordinatorEndpoint @@ -49,9 +52,6 @@ import org.apache.spark.util.{RpcUtils, Utils} * including the serializer, RpcEnv, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a global variable, so all the threads can access the same * SparkEnv. It can be accessed by SparkEnv.get (e.g. after creating a SparkContext). - * - * NOTE: This is not intended for external use. This is exposed for Shark and may be made private - * in a future release. */ @DeveloperApi class SparkEnv ( @@ -163,10 +163,10 @@ object SparkEnv extends Logging { mockOutputCommitCoordinator: Option[OutputCommitCoordinator] = None): SparkEnv = { assert(conf.contains(DRIVER_HOST_ADDRESS), s"${DRIVER_HOST_ADDRESS.key} is not set on the driver!") - assert(conf.contains("spark.driver.port"), "spark.driver.port is not set on the driver!") + assert(conf.contains(DRIVER_PORT), s"${DRIVER_PORT.key} is not set on the driver!") val bindAddress = conf.get(DRIVER_BIND_ADDRESS) val advertiseAddress = conf.get(DRIVER_HOST_ADDRESS) - val port = conf.get("spark.driver.port").toInt + val port = conf.get(DRIVER_PORT) val ioEncryptionKey = if (conf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(conf)) } else { @@ -232,8 +232,8 @@ object SparkEnv extends Logging { if (isDriver) { assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") } - - val securityManager = new SecurityManager(conf, ioEncryptionKey) + val authSecretFileConf = if (isDriver) AUTH_SECRET_FILE_DRIVER else AUTH_SECRET_FILE_EXECUTOR + val securityManager = new SecurityManager(conf, ioEncryptionKey, authSecretFileConf) if (isDriver) { securityManager.initializeAuth() } @@ -251,7 +251,7 @@ object SparkEnv extends Logging { // Figure out which port RpcEnv actually bound to in case the original port is 0 or occupied. if (isDriver) { - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) } // Create an instance of the class with the given name, possibly initializing it with our conf @@ -261,7 +261,7 @@ object SparkEnv extends Logging { // SparkConf, then one taking no arguments try { cls.getConstructor(classOf[SparkConf], java.lang.Boolean.TYPE) - .newInstance(conf, new java.lang.Boolean(isDriver)) + .newInstance(conf, java.lang.Boolean.valueOf(isDriver)) .asInstanceOf[T] } catch { case _: NoSuchMethodException => @@ -274,14 +274,13 @@ object SparkEnv extends Logging { } } - // Create an instance of the class named by the given SparkConf property, or defaultClassName + // Create an instance of the class named by the given SparkConf property // if the property is not set, possibly initializing it with our conf - def instantiateClassFromConf[T](propertyName: String, defaultClassName: String): T = { - instantiateClass[T](conf.get(propertyName, defaultClassName)) + def instantiateClassFromConf[T](propertyName: ConfigEntry[String]): T = { + instantiateClass[T](conf.get(propertyName)) } - val serializer = instantiateClassFromConf[Serializer]( - "spark.serializer", "org.apache.spark.serializer.JavaSerializer") + val serializer = instantiateClassFromConf[Serializer](SERIALIZER) logDebug(s"Using serializer: ${serializer.getClass}") val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey) @@ -317,18 +316,12 @@ object SparkEnv extends Logging { val shortShuffleMgrNames = Map( "sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName, "tungsten-sort" -> classOf[org.apache.spark.shuffle.sort.SortShuffleManager].getName) - val shuffleMgrName = conf.get("spark.shuffle.manager", "sort") + val shuffleMgrName = conf.get(config.SHUFFLE_MANAGER) val shuffleMgrClass = shortShuffleMgrNames.getOrElse(shuffleMgrName.toLowerCase(Locale.ROOT), shuffleMgrName) val shuffleManager = instantiateClass[ShuffleManager](shuffleMgrClass) - val useLegacyMemoryManager = conf.getBoolean("spark.memory.useLegacyMode", false) - val memoryManager: MemoryManager = - if (useLegacyMemoryManager) { - new StaticMemoryManager(conf, numUsableCores) - } else { - UnifiedMemoryManager(conf, numUsableCores) - } + val memoryManager: MemoryManager = UnifiedMemoryManager(conf, numUsableCores) val blockManagerPort = if (isDriver) { conf.get(DRIVER_BLOCK_MANAGER_PORT) @@ -336,31 +329,58 @@ object SparkEnv extends Logging { conf.get(BLOCK_MANAGER_PORT) } - val blockTransferService = - new NettyBlockTransferService(conf, securityManager, bindAddress, advertiseAddress, - blockManagerPort, numUsableCores) + val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) + Some(new ExternalShuffleClient(transConf, securityManager, + securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT))) + } else { + None + } val blockManagerMaster = new BlockManagerMaster(registerOrLookupEndpoint( BlockManagerMaster.DRIVER_ENDPOINT_NAME, - new BlockManagerMasterEndpoint(rpcEnv, isLocal, conf, listenerBus)), + new BlockManagerMasterEndpoint( + rpcEnv, + isLocal, + conf, + listenerBus, + if (conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)) { + externalShuffleClient + } else { + None + })), conf, isDriver) + val blockTransferService = + new NettyBlockTransferService(conf, securityManager, bindAddress, advertiseAddress, + blockManagerPort, numUsableCores, blockManagerMaster.driverEndpoint) + // NB: blockManager is not valid until initialize() is called later. - val blockManager = new BlockManager(executorId, rpcEnv, blockManagerMaster, - serializerManager, conf, memoryManager, mapOutputTracker, shuffleManager, - blockTransferService, securityManager, numUsableCores) + val blockManager = new BlockManager( + executorId, + rpcEnv, + blockManagerMaster, + serializerManager, + conf, + memoryManager, + mapOutputTracker, + shuffleManager, + blockTransferService, + securityManager, + externalShuffleClient) val metricsSystem = if (isDriver) { // Don't start metrics system right now for Driver. // We need to wait for the task scheduler to give us an app ID. // Then we can start the metrics system. - MetricsSystem.createMetricsSystem("driver", conf, securityManager) + MetricsSystem.createMetricsSystem(MetricsSystemInstances.DRIVER, conf, securityManager) } else { // We need to set the executor ID before the MetricsSystem is created because sources and // sinks specified in the metrics configuration file will want to incorporate this executor's // ID into the metrics they report. - conf.set("spark.executor.id", executorId) - val ms = MetricsSystem.createMetricsSystem("executor", conf, securityManager) + conf.set(EXECUTOR_ID, executorId) + val ms = MetricsSystem.createMetricsSystem(MetricsSystemInstances.EXECUTOR, conf, + securityManager) ms.start() ms } @@ -407,6 +427,7 @@ object SparkEnv extends Logging { private[spark] def environmentDetails( conf: SparkConf, + hadoopConf: Configuration, schedulingMode: String, addedJars: Seq[String], addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { @@ -421,8 +442,8 @@ object SparkEnv extends Logging { // Spark properties // This includes the scheduling mode whether or not it is configured (used by SparkUI) val schedulerMode = - if (!conf.contains("spark.scheduler.mode")) { - Seq(("spark.scheduler.mode", schedulingMode)) + if (!conf.contains(SCHEDULER_MODE)) { + Seq((SCHEDULER_MODE.key, schedulingMode)) } else { Seq.empty[(String, String)] } @@ -442,9 +463,14 @@ object SparkEnv extends Logging { val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted + // Add Hadoop properties, it will not ignore configs including in Spark. Some spark + // conf starting with "spark.hadoop" may overwrite it. + val hadoopProperties = hadoopConf.asScala + .map(entry => (entry.getKey, entry.getValue)).toSeq.sorted Map[String, Seq[(String, String)]]( "JVM Information" -> jvmInformation, "Spark Properties" -> sparkProperties, + "Hadoop Properties" -> hadoopProperties, "System Properties" -> otherProperties, "Classpath Entries" -> classPaths) } diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index 977a27bdfe1bf..4ad9a0cc4b103 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -37,3 +37,9 @@ private[spark] class SparkDriverExecutionException(cause: Throwable) */ private[spark] case class SparkUserAppException(exitCode: Int) extends SparkException(s"User application exited with $exitCode") + +/** + * Exception thrown when the relative executor to access is dead. + */ +private[spark] case class ExecutorDeadException(message: String) + extends SparkException(message) diff --git a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala index 815237eba0174..555c085d85a1e 100644 --- a/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala +++ b/core/src/main/scala/org/apache/spark/SparkStatusTracker.scala @@ -99,6 +99,7 @@ class SparkStatusTracker private[spark] (sc: SparkContext, store: AppStatusStore /** * Returns information of all known executors, including host, port, cacheSize, numRunningTasks * and memory metrics. + * Note this include information for both the driver and executors. */ def getExecutorInfos: Array[SparkExecutorInfo] = { store.executorList(true).map { exec => diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index 2b939dabb1105..2299c54e2624b 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -20,10 +20,11 @@ package org.apache.spark import java.io.Serializable import java.util.Properties -import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.annotation.{DeveloperApi, Evolving} import org.apache.spark.executor.TaskMetrics import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceInformation import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.{AccumulatorV2, TaskCompletionListener, TaskFailureListener} @@ -96,13 +97,6 @@ abstract class TaskContext extends Serializable { */ def isInterrupted(): Boolean - /** - * Returns true if the task is running locally in the driver program. - * @return false - */ - @deprecated("Local execution was removed, so this always returns false", "2.0.0") - def isRunningLocally(): Boolean - /** * Adds a (Java friendly) listener to be executed on task completion. * This will be called in all situations - success, failure, or cancellation. Adding a listener @@ -183,6 +177,14 @@ abstract class TaskContext extends Serializable { */ def getLocalProperty(key: String): String + /** + * Resources allocated to the task. The key is the resource name and the value is information + * about the resource. Please refer to [[org.apache.spark.resource.ResourceInformation]] for + * specifics. + */ + @Evolving + def resources(): Map[String, ResourceInformation] + @DeveloperApi def taskMetrics(): TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala index 89730424e5acf..516fb95593324 100644 --- a/core/src/main/scala/org/apache/spark/TaskContextImpl.scala +++ b/core/src/main/scala/org/apache/spark/TaskContextImpl.scala @@ -27,6 +27,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.metrics.source.Source +import org.apache.spark.resource.ResourceInformation import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util._ @@ -51,7 +52,8 @@ private[spark] class TaskContextImpl( localProperties: Properties, @transient private val metricsSystem: MetricsSystem, // The default value is only used in tests. - override val taskMetrics: TaskMetrics = TaskMetrics.empty) + override val taskMetrics: TaskMetrics = TaskMetrics.empty, + override val resources: Map[String, ResourceInformation] = Map.empty) extends TaskContext with Logging { @@ -157,8 +159,6 @@ private[spark] class TaskContextImpl( @GuardedBy("this") override def isCompleted(): Boolean = synchronized(completed) - override def isRunningLocally(): Boolean = false - override def isInterrupted(): Boolean = reasonIfKilled.isDefined override def getLocalProperty(key: String): String = localProperties.getProperty(key) diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index c2ebd388a2365..41ae3ae3b758a 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -20,9 +20,11 @@ package org.apache.spark import java.io.{ByteArrayInputStream, File, FileInputStream, FileOutputStream} import java.net.{HttpURLConnection, URI, URL} import java.nio.charset.StandardCharsets +import java.nio.file.{Files => JavaFiles} +import java.nio.file.attribute.PosixFilePermission.{OWNER_EXECUTE, OWNER_READ, OWNER_WRITE} import java.security.SecureRandom import java.security.cert.X509Certificate -import java.util.{Arrays, Properties} +import java.util.{Arrays, EnumSet, Properties} import java.util.concurrent.{TimeoutException, TimeUnit} import java.util.jar.{JarEntry, JarOutputStream} import javax.net.ssl._ @@ -36,8 +38,11 @@ import scala.util.Try import com.google.common.io.{ByteStreams, Files} import org.apache.log4j.PropertyConfigurator +import org.json4s.JsonAST.JValue +import org.json4s.jackson.JsonMethods.{compact, render} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.internal.config._ import org.apache.spark.scheduler._ import org.apache.spark.util.Utils @@ -192,6 +197,20 @@ private[spark] object TestUtils { assert(listener.numSpilledStages == 0, s"expected $identifier to not spill, but did") } + /** + * Asserts that exception message contains the message. Please note this checks all + * exceptions in the tree. + */ + def assertExceptionMsg(exception: Throwable, msg: String): Unit = { + var e = exception + var contains = e.getMessage.contains(msg) + while (e.getCause != null && !contains) { + e = e.getCause + contains = e.getMessage.contains(msg) + } + assert(contains, s"Exception tree doesn't contain the expected message: $msg") + } + /** * Test if a command is available. */ @@ -297,6 +316,22 @@ private[spark] object TestUtils { current ++ current.filter(_.isDirectory).flatMap(recursiveList) } + /** Creates a temp JSON file that contains the input JSON record. */ + def createTempJsonFile(dir: File, prefix: String, jsonValue: JValue): String = { + val file = File.createTempFile(prefix, ".json", dir) + JavaFiles.write(file.toPath, compact(render(jsonValue)).getBytes()) + file.getPath + } + + /** Creates a temp bash script that prints the given output. */ + def createTempScriptWithExpectedOutput(dir: File, prefix: String, output: String): String = { + val file = File.createTempFile(prefix, ".sh", dir) + val script = s"cat < JFunction, Function2 => JFunction2, PairFunction} +import org.apache.spark.api.java.function.{FlatMapFunction, Function => JFunction, + Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.rdd.RDD.rddToPairRDDFunctions @@ -674,8 +675,8 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Pass each value in the key-value pair RDD through a flatMap function without changing the * keys; this also retains the original RDD's partitioning. */ - def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { - def fn: (V) => Iterable[U] = (x: V) => f.call(x).asScala + def flatMapValues[U](f: FlatMapFunction[V, U]): JavaPairRDD[K, U] = { + def fn: (V) => Iterator[U] = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } @@ -951,7 +952,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -968,7 +969,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -984,7 +985,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 41b5cab601c36..6f0182255e5fe 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -17,6 +17,9 @@ package org.apache.spark.api.java +import java.io.{DataInputStream, EOFException, FileInputStream, InputStream} + +import scala.collection.mutable import scala.language.implicitConversions import scala.reflect.ClassTag @@ -213,4 +216,34 @@ object JavaRDD { implicit def fromRDD[T: ClassTag](rdd: RDD[T]): JavaRDD[T] = new JavaRDD[T](rdd) implicit def toRDD[T](rdd: JavaRDD[T]): RDD[T] = rdd.rdd + + private[api] def readRDDFromFile( + sc: JavaSparkContext, + filename: String, + parallelism: Int): JavaRDD[Array[Byte]] = { + readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + } + + private[api] def readRDDFromInputStream( + sc: SparkContext, + in: InputStream, + parallelism: Int): JavaRDD[Array[Byte]] = { + val din = new DataInputStream(in) + try { + val objs = new mutable.ArrayBuffer[Array[Byte]] + try { + while (true) { + val length = din.readInt() + val obj = new Array[Byte](length) + din.readFully(obj) + objs += obj + } + } catch { + case eof: EOFException => // No-op + } + JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) + } finally { + din.close() + } + } } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 91ae1002abd21..5ba821935ac69 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -685,7 +685,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 09c83849e26b2..330c2f6e6117e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -21,6 +21,7 @@ import java.io.Closeable import java.util import java.util.{Map => JMap} +import scala.annotation.varargs import scala.collection.JavaConverters._ import scala.language.implicitConversions import scala.reflect.ClassTag @@ -30,21 +31,20 @@ import org.apache.hadoop.mapred.{InputFormat, JobConf} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.spark._ -import org.apache.spark.AccumulatorParam._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD} +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD} +import org.apache.spark.resource.ResourceInformation /** * A Java-friendly version of [[org.apache.spark.SparkContext]] that returns * [[org.apache.spark.api.java.JavaRDD]]s and works with Java collections instead of Scala ones. * - * Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before - * creating a new one. This limitation may eventually be removed; see SPARK-2243 for more details. + * @note Only one `SparkContext` should be active per JVM. You must `stop()` the + * active `SparkContext` before creating a new one. */ -class JavaSparkContext(val sc: SparkContext) - extends JavaSparkContextVarargsWorkaround with Closeable { +class JavaSparkContext(val sc: SparkContext) extends Closeable { /** * Create a JavaSparkContext that loads settings from system properties (for instance, when @@ -115,6 +115,8 @@ class JavaSparkContext(val sc: SparkContext) def appName: String = sc.appName + def resources: JMap[String, ResourceInformation] = sc.resources.asJava + def jars: util.List[String] = sc.jars.asJava def startTime: java.lang.Long = sc.startTime @@ -168,12 +170,14 @@ class JavaSparkContext(val sc: SparkContext) /** * Read a text file from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI, and return it as an RDD of Strings. + * The text files must be encoded as UTF-8. */ def textFile(path: String): JavaRDD[String] = sc.textFile(path) /** * Read a text file from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI, and return it as an RDD of Strings. + * The text files must be encoded as UTF-8. */ def textFile(path: String, minPartitions: Int): JavaRDD[String] = sc.textFile(path, minPartitions) @@ -184,6 +188,7 @@ class JavaSparkContext(val sc: SparkContext) * Read a directory of text files from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI. Each file is read as a single record and returned in a * key-value pair, where the key is the path of each file, the value is the content of each file. + * The text files must be encoded as UTF-8. * *

For example, if you have the following files: * {{{ @@ -217,6 +222,7 @@ class JavaSparkContext(val sc: SparkContext) * Read a directory of text files from HDFS, a local file system (available on all nodes), or any * Hadoop-supported file system URI. Each file is read as a single record and returned in a * key-value pair, where the key is the path of each file, the value is the content of each file. + * The text files must be encoded as UTF-8. * * @see `wholeTextFiles(path: String, minPartitions: Int)`. */ @@ -507,141 +513,31 @@ class JavaSparkContext(val sc: SparkContext) new JavaNewHadoopRDD(rdd.asInstanceOf[NewHadoopRDD[K, V]]) } - /** Build the union of two or more RDDs. */ - override def union[T](first: JavaRDD[T], rest: java.util.List[JavaRDD[T]]): JavaRDD[T] = { - val rdds: Seq[RDD[T]] = (Seq(first) ++ rest.asScala).map(_.rdd) - implicit val ctag: ClassTag[T] = first.classTag - sc.union(rdds) + /** Build the union of JavaRDDs. */ + @varargs + def union[T](rdds: JavaRDD[T]*): JavaRDD[T] = { + require(rdds.nonEmpty, "Union called on no RDDs") + implicit val ctag: ClassTag[T] = rdds.head.classTag + sc.union(rdds.map(_.rdd)) } - /** Build the union of two or more RDDs. */ - override def union[K, V](first: JavaPairRDD[K, V], rest: java.util.List[JavaPairRDD[K, V]]) - : JavaPairRDD[K, V] = { - val rdds: Seq[RDD[(K, V)]] = (Seq(first) ++ rest.asScala).map(_.rdd) - implicit val ctag: ClassTag[(K, V)] = first.classTag - implicit val ctagK: ClassTag[K] = first.kClassTag - implicit val ctagV: ClassTag[V] = first.vClassTag - new JavaPairRDD(sc.union(rdds)) + /** Build the union of JavaPairRDDs. */ + @varargs + def union[K, V](rdds: JavaPairRDD[K, V]*): JavaPairRDD[K, V] = { + require(rdds.nonEmpty, "Union called on no RDDs") + implicit val ctag: ClassTag[(K, V)] = rdds.head.classTag + implicit val ctagK: ClassTag[K] = rdds.head.kClassTag + implicit val ctagV: ClassTag[V] = rdds.head.vClassTag + new JavaPairRDD(sc.union(rdds.map(_.rdd))) } - /** Build the union of two or more RDDs. */ - override def union(first: JavaDoubleRDD, rest: java.util.List[JavaDoubleRDD]): JavaDoubleRDD = { - val rdds: Seq[RDD[Double]] = (Seq(first) ++ rest.asScala).map(_.srdd) - new JavaDoubleRDD(sc.union(rdds)) + /** Build the union of JavaDoubleRDDs. */ + @varargs + def union(rdds: JavaDoubleRDD*): JavaDoubleRDD = { + require(rdds.nonEmpty, "Union called on no RDDs") + new JavaDoubleRDD(sc.union(rdds.map(_.srdd))) } - /** - * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - */ - @deprecated("use sc().longAccumulator()", "2.0.0") - def intAccumulator(initialValue: Int): Accumulator[java.lang.Integer] = - sc.accumulator(initialValue)(IntAccumulatorParam).asInstanceOf[Accumulator[java.lang.Integer]] - - /** - * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use sc().longAccumulator(String)", "2.0.0") - def intAccumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = - sc.accumulator(initialValue, name)(IntAccumulatorParam) - .asInstanceOf[Accumulator[java.lang.Integer]] - - /** - * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - */ - @deprecated("use sc().doubleAccumulator()", "2.0.0") - def doubleAccumulator(initialValue: Double): Accumulator[java.lang.Double] = - sc.accumulator(initialValue)(DoubleAccumulatorParam).asInstanceOf[Accumulator[java.lang.Double]] - - /** - * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use sc().doubleAccumulator(String)", "2.0.0") - def doubleAccumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = - sc.accumulator(initialValue, name)(DoubleAccumulatorParam) - .asInstanceOf[Accumulator[java.lang.Double]] - - /** - * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - */ - @deprecated("use sc().longAccumulator()", "2.0.0") - def accumulator(initialValue: Int): Accumulator[java.lang.Integer] = intAccumulator(initialValue) - - /** - * Create an [[org.apache.spark.Accumulator]] integer variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use sc().longAccumulator(String)", "2.0.0") - def accumulator(initialValue: Int, name: String): Accumulator[java.lang.Integer] = - intAccumulator(initialValue, name) - - /** - * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - */ - @deprecated("use sc().doubleAccumulator()", "2.0.0") - def accumulator(initialValue: Double): Accumulator[java.lang.Double] = - doubleAccumulator(initialValue) - - - /** - * Create an [[org.apache.spark.Accumulator]] double variable, which tasks can "add" values - * to using the `add` method. Only the master can access the accumulator's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use sc().doubleAccumulator(String)", "2.0.0") - def accumulator(initialValue: Double, name: String): Accumulator[java.lang.Double] = - doubleAccumulator(initialValue, name) - - /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" - * values to using the `add` method. Only the master can access the accumulator's `value`. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulator[T](initialValue: T, accumulatorParam: AccumulatorParam[T]): Accumulator[T] = - sc.accumulator(initialValue)(accumulatorParam) - - /** - * Create an [[org.apache.spark.Accumulator]] variable of a given type, which tasks can "add" - * values to using the `add` method. Only the master can access the accumulator's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulator[T](initialValue: T, name: String, accumulatorParam: AccumulatorParam[T]) - : Accumulator[T] = - sc.accumulator(initialValue, name)(accumulatorParam) - - /** - * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks - * can "add" values with `add`. Only the master can access the accumulable's `value`. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulable[T, R](initialValue: T, param: AccumulableParam[T, R]): Accumulable[T, R] = - sc.accumulable(initialValue)(param) - - /** - * Create an [[org.apache.spark.Accumulable]] shared variable of the given type, to which tasks - * can "add" values with `add`. Only the master can access the accumulable's `value`. - * - * This version supports naming the accumulator for display in Spark's web UI. - */ - @deprecated("use AccumulatorV2", "2.0.0") - def accumulable[T, R](initialValue: T, name: String, param: AccumulableParam[T, R]) - : Accumulable[T, R] = - sc.accumulable(initialValue, name)(param) - /** * Broadcast a read-only variable to the cluster, returning a * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala index 6259bead3ea88..a4817b3cf770d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonHadoopUtil.scala @@ -33,17 +33,17 @@ import org.apache.spark.util.{SerializableConfiguration, Utils} * A trait for use with reading custom classes in PySpark. Implement this trait and add custom * transformation code by overriding the convert method. */ -trait Converter[T, + U] extends Serializable { +trait Converter[-T, +U] extends Serializable { def convert(obj: T): U } private[python] object Converter extends Logging { - def getInstance(converterClass: Option[String], - defaultConverter: Converter[Any, Any]): Converter[Any, Any] = { + def getInstance[T, U](converterClass: Option[String], + defaultConverter: Converter[_ >: T, _ <: U]): Converter[T, U] = { converterClass.map { cc => Try { - val c = Utils.classForName(cc).newInstance().asInstanceOf[Converter[Any, Any]] + val c = Utils.classForName[Converter[T, U]](cc).getConstructor().newInstance() logInfo(s"Loaded converter: $cc") c } match { @@ -176,8 +176,8 @@ private[python] object PythonHadoopUtil { * [[org.apache.hadoop.io.Writable]], into an RDD of base types, or vice versa. */ def convertRDD[K, V](rdd: RDD[(K, V)], - keyConverter: Converter[Any, Any], - valueConverter: Converter[Any, Any]): RDD[(Any, Any)] = { + keyConverter: Converter[K, Any], + valueConverter: Converter[V, Any]): RDD[(Any, Any)] = { rdd.map { case (k, v) => (keyConverter.convert(k), valueConverter.convert(v)) } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8b5a7a9aefea5..5b80e149b38ac 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,10 +24,6 @@ import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.concurrent.Promise -import scala.concurrent.duration.Duration -import scala.language.existentials -import scala.util.Try import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec @@ -39,9 +35,10 @@ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} import org.apache.spark.broadcast.Broadcast import org.apache.spark.input.PortableDataStream import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.network.util.JavaUtils import org.apache.spark.rdd.RDD -import org.apache.spark.security.SocketAuthHelper +import org.apache.spark.security.{SocketAuthHelper, SocketAuthServer, SocketFuncServer} import org.apache.spark.util._ @@ -89,7 +86,7 @@ private[spark] case class PythonFunction( private[spark] case class ChainedPythonFunctions(funcs: Seq[PythonFunction]) /** Thrown for exceptions in user Python code. */ -private[spark] class PythonException(msg: String, cause: Exception) +private[spark] class PythonException(msg: String, cause: Throwable) extends RuntimeException(msg, cause) /** @@ -140,8 +137,9 @@ private[spark] object PythonRDD extends Logging { * (effectively a collect()), but allows you to run on a certain subset of partitions, * or to enable local execution. * - * @return 2-tuple (as a Java array) with the port number of a local socket which serves the - * data collected from this job, and the secret for authentication. + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. */ def runJob( sc: SparkContext, @@ -159,43 +157,86 @@ private[spark] object PythonRDD extends Logging { /** * A helper function to collect an RDD as an iterator, then serve it via socket. * - * @return 2-tuple (as a Java array) with the port number of a local socket which serves the - * data collected from this job, and the secret for authentication. + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. */ def collectAndServe[T](rdd: RDD[T]): Array[Any] = { serveIterator(rdd.collect().iterator, s"serve RDD ${rdd.id}") } + /** + * A helper function to create a local RDD iterator and serve it via socket. Partitions are + * are collected as separate jobs, by order of index. Partition data is first requested by a + * non-zero integer to start a collection job. The response is prefaced by an integer with 1 + * meaning partition data will be served, 0 meaning the local iterator has been consumed, + * and -1 meaning an error occurred during collection. This function is used by + * pyspark.rdd._local_iterator_from_socket(). + * + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. + */ def toLocalIteratorAndServe[T](rdd: RDD[T]): Array[Any] = { - serveIterator(rdd.toLocalIterator, s"serve toLocalIterator") + val handleFunc = (sock: Socket) => { + val out = new DataOutputStream(sock.getOutputStream) + val in = new DataInputStream(sock.getInputStream) + Utils.tryWithSafeFinallyAndFailureCallbacks(block = { + // Collects a partition on each iteration + val collectPartitionIter = rdd.partitions.indices.iterator.map { i => + rdd.sparkContext.runJob(rdd, (iter: Iterator[Any]) => iter.toArray, Seq(i)).head + } + + // Write data until iteration is complete, client stops iteration, or error occurs + var complete = false + while (!complete) { + + // Read request for data, value of zero will stop iteration or non-zero to continue + if (in.readInt() == 0) { + complete = true + } else if (collectPartitionIter.hasNext) { + + // Client requested more data, attempt to collect the next partition + val partitionArray = collectPartitionIter.next() + + // Send response there is a partition to read + out.writeInt(1) + + // Write the next object and signal end of data for this iteration + writeIteratorToStream(partitionArray.toIterator, out) + out.writeInt(SpecialLengths.END_OF_DATA_SECTION) + out.flush() + } else { + // Send response there are no more partitions to read and close + out.writeInt(0) + complete = true + } + } + })(catchBlock = { + // Send response that an error occurred, original exception is re-thrown + out.writeInt(-1) + }, finallyBlock = { + out.close() + in.close() + }) + } + + val server = new SocketFuncServer(authHelper, "serve toLocalIterator", handleFunc) + Array(server.port, server.secret, server) } - def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): - JavaRDD[Array[Byte]] = { - readRDDFromInputStream(sc.sc, new FileInputStream(filename), parallelism) + def readRDDFromFile( + sc: JavaSparkContext, + filename: String, + parallelism: Int): JavaRDD[Array[Byte]] = { + JavaRDD.readRDDFromFile(sc, filename, parallelism) } def readRDDFromInputStream( sc: SparkContext, in: InputStream, parallelism: Int): JavaRDD[Array[Byte]] = { - val din = new DataInputStream(in) - try { - val objs = new mutable.ArrayBuffer[Array[Byte]] - try { - while (true) { - val length = din.readInt() - val obj = new Array[Byte](length) - din.readFully(obj) - objs += obj - } - } catch { - case eof: EOFException => // No-op - } - JavaRDD.fromRDD(sc.parallelize(objs, parallelism)) - } finally { - din.close() - } + JavaRDD.readRDDFromInputStream(sc, in, parallelism) } def setupBroadcast(path: String): PythonBroadcast = { @@ -241,8 +282,8 @@ private[spark] object PythonRDD extends Logging { batchSize: Int): JavaRDD[Array[Byte]] = { val keyClass = Option(keyClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") val valueClass = Option(valueClassMaybeNull).getOrElse("org.apache.hadoop.io.Text") - val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] - val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] + val kc = Utils.classForName[K](keyClass) + val vc = Utils.classForName[V](valueClass) val rdd = sc.sc.sequenceFile[K, V](path, kc, vc, minSplits) val confBroadcasted = sc.sc.broadcast(new SerializableConfiguration(sc.hadoopConfiguration())) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, @@ -309,9 +350,9 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration): RDD[(K, V)] = { - val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] - val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] - val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] + val kc = Utils.classForName[K](keyClass) + val vc = Utils.classForName[V](valueClass) + val fc = Utils.classForName[F](inputFormatClass) if (path.isDefined) { sc.sc.newAPIHadoopFile[K, V, F](path.get, fc, kc, vc, conf) } else { @@ -378,9 +419,9 @@ private[spark] object PythonRDD extends Logging { keyClass: String, valueClass: String, conf: Configuration) = { - val kc = Utils.classForName(keyClass).asInstanceOf[Class[K]] - val vc = Utils.classForName(valueClass).asInstanceOf[Class[V]] - val fc = Utils.classForName(inputFormatClass).asInstanceOf[Class[F]] + val kc = Utils.classForName[K](keyClass) + val vc = Utils.classForName[V](valueClass) + val fc = Utils.classForName[F](inputFormatClass) if (path.isDefined) { sc.sc.hadoopFile(path.get, fc, kc, vc) } else { @@ -405,8 +446,9 @@ private[spark] object PythonRDD extends Logging { * * The thread will terminate after all the data are sent or any exceptions happen. * - * @return 2-tuple (as a Java array) with the port number of a local socket which serves the - * data collected from this job, and the secret for authentication. + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. */ def serveIterator(items: Iterator[_], threadName: String): Array[Any] = { serveToStream(threadName) { out => @@ -426,18 +468,14 @@ private[spark] object PythonRDD extends Logging { * * The thread will terminate after the block of code is executed or any * exceptions happen. + * + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. */ private[spark] def serveToStream( threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = { - val (port, secret) = PythonServer.setupOneConnectionServer(authHelper, threadName) { s => - val out = new BufferedOutputStream(s.getOutputStream()) - Utils.tryWithSafeFinally { - writeFunc(out) - } { - out.close() - } - } - Array(port, secret) + SocketAuthServer.serveToStream(threadName, authHelper)(writeFunc) } private def getMergedConf(confAsMap: java.util.HashMap[String, String], @@ -446,29 +484,33 @@ private[spark] object PythonRDD extends Logging { PythonHadoopUtil.mergeConfs(baseConf, conf) } - private def inferKeyValueTypes[K, V](rdd: RDD[(K, V)], keyConverterClass: String = null, - valueConverterClass: String = null): (Class[_], Class[_]) = { + private def inferKeyValueTypes[K, V, KK, VV](rdd: RDD[(K, V)], keyConverterClass: String = null, + valueConverterClass: String = null): (Class[_ <: KK], Class[_ <: VV]) = { // Peek at an element to figure out key/value types. Since Writables are not serializable, // we cannot call first() on the converted RDD. Instead, we call first() on the original RDD // and then convert locally. val (key, value) = rdd.first() - val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, - new JavaToWritableConverter) + val (kc, vc) = getKeyValueConverters[K, V, KK, VV]( + keyConverterClass, valueConverterClass, new JavaToWritableConverter) (kc.convert(key).getClass, vc.convert(value).getClass) } - private def getKeyValueTypes(keyClass: String, valueClass: String): - Option[(Class[_], Class[_])] = { + private def getKeyValueTypes[K, V](keyClass: String, valueClass: String): + Option[(Class[K], Class[V])] = { for { k <- Option(keyClass) v <- Option(valueClass) } yield (Utils.classForName(k), Utils.classForName(v)) } - private def getKeyValueConverters(keyConverterClass: String, valueConverterClass: String, - defaultConverter: Converter[Any, Any]): (Converter[Any, Any], Converter[Any, Any]) = { - val keyConverter = Converter.getInstance(Option(keyConverterClass), defaultConverter) - val valueConverter = Converter.getInstance(Option(valueConverterClass), defaultConverter) + private def getKeyValueConverters[K, V, KK, VV]( + keyConverterClass: String, + valueConverterClass: String, + defaultConverter: Converter[_, _]): (Converter[K, KK], Converter[V, VV]) = { + val keyConverter = Converter.getInstance(Option(keyConverterClass), + defaultConverter.asInstanceOf[Converter[K, KK]]) + val valueConverter = Converter.getInstance(Option(valueConverterClass), + defaultConverter.asInstanceOf[Converter[V, VV]]) (keyConverter, valueConverter) } @@ -480,7 +522,7 @@ private[spark] object PythonRDD extends Logging { keyConverterClass: String, valueConverterClass: String, defaultConverter: Converter[Any, Any]): RDD[(Any, Any)] = { - val (kc, vc) = getKeyValueConverters(keyConverterClass, valueConverterClass, + val (kc, vc) = getKeyValueConverters[K, V, Any, Any](keyConverterClass, valueConverterClass, defaultConverter) PythonHadoopUtil.convertRDD(rdd, kc, vc) } @@ -491,7 +533,7 @@ private[spark] object PythonRDD extends Logging { * [[org.apache.hadoop.io.Writable]] types already, since Writables are not Java * `Serializable` and we can't peek at them. The `path` can be on any Hadoop file system. */ - def saveAsSequenceFile[K, V, C <: CompressionCodec]( + def saveAsSequenceFile[C <: CompressionCodec]( pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, path: String, @@ -510,7 +552,7 @@ private[spark] object PythonRDD extends Logging { * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of * this RDD. */ - def saveAsHadoopFile[K, V, F <: OutputFormat[_, _], C <: CompressionCodec]( + def saveAsHadoopFile[F <: OutputFormat[_, _], C <: CompressionCodec]( pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, path: String, @@ -528,7 +570,7 @@ private[spark] object PythonRDD extends Logging { val codec = Option(compressionCodecClass).map(Utils.classForName(_).asInstanceOf[Class[C]]) val converted = convertRDD(rdd, keyConverterClass, valueConverterClass, new JavaToWritableConverter) - val fc = Utils.classForName(outputFormatClass).asInstanceOf[Class[F]] + val fc = Utils.classForName[F](outputFormatClass) converted.saveAsHadoopFile(path, kc, vc, fc, new JobConf(mergedConf), codec = codec) } @@ -541,7 +583,7 @@ private[spark] object PythonRDD extends Logging { * `confAsMap` is merged with the default Hadoop conf associated with the SparkContext of * this RDD. */ - def saveAsNewAPIHadoopFile[K, V, F <: NewOutputFormat[_, _]]( + def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, path: String, @@ -569,7 +611,7 @@ private[spark] object PythonRDD extends Logging { * (mapred vs. mapreduce). Keys/values are converted for output using either user specified * converters or, by default, [[org.apache.spark.api.python.JavaToWritableConverter]]. */ - def saveAsHadoopDataset[K, V]( + def saveAsHadoopDataset( pyRDD: JavaRDD[Array[Byte]], batchSerialized: Boolean, confAsMap: java.util.HashMap[String, String], @@ -604,7 +646,7 @@ private[spark] class PythonAccumulatorV2( Utils.checkHost(serverHost) - val bufferSize = SparkEnv.get.conf.getInt("spark.buffer.size", 65536) + val bufferSize = SparkEnv.get.conf.get(BUFFER_SIZE) /** * We try to reuse a single Socket to transfer accumulator updates, as they are all added @@ -659,7 +701,8 @@ private[spark] class PythonAccumulatorV2( private[spark] class PythonBroadcast(@transient var path: String) extends Serializable with Logging { - private var encryptionServer: PythonServer[Unit] = null + private var encryptionServer: SocketAuthServer[Unit] = null + private var decryptionServer: SocketAuthServer[Unit] = null /** * Read data from disks, then copy it to `out` @@ -704,20 +747,40 @@ private[spark] class PythonBroadcast(@transient var path: String) extends Serial } def setupEncryptionServer(): Array[Any] = { - encryptionServer = new PythonServer[Unit]("broadcast-encrypt-server") { + encryptionServer = new SocketAuthServer[Unit]("broadcast-encrypt-server") { override def handleConnection(sock: Socket): Unit = { val env = SparkEnv.get val in = sock.getInputStream() - val dir = new File(Utils.getLocalDir(env.conf)) - val file = File.createTempFile("broadcast", "", dir) - path = file.getAbsolutePath - val out = env.serializerManager.wrapForEncryption(new FileOutputStream(path)) + val abspath = new File(path).getAbsolutePath + val out = env.serializerManager.wrapForEncryption(new FileOutputStream(abspath)) DechunkedInputStream.dechunkAndCopyToOutput(in, out) } } Array(encryptionServer.port, encryptionServer.secret) } + def setupDecryptionServer(): Array[Any] = { + decryptionServer = new SocketAuthServer[Unit]("broadcast-decrypt-server-for-driver") { + override def handleConnection(sock: Socket): Unit = { + val out = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream())) + Utils.tryWithSafeFinally { + val in = SparkEnv.get.serializerManager.wrapForEncryption(new FileInputStream(path)) + Utils.tryWithSafeFinally { + Utils.copyStream(in, out, false) + } { + in.close() + } + out.flush() + } { + JavaUtils.closeQuietly(out) + } + } + } + Array(decryptionServer.port, decryptionServer.secret) + } + + def waitTillBroadcastDataSent(): Unit = decryptionServer.getResult() + def waitTillDataReceived(): Unit = encryptionServer.getResult() } // scalastyle:on no.finalize @@ -792,91 +855,13 @@ private[spark] object DechunkedInputStream { } } -/** - * Creates a server in the jvm to communicate with python for handling one batch of data, with - * authentication and error handling. - */ -private[spark] abstract class PythonServer[T]( - authHelper: SocketAuthHelper, - threadName: String) { - - def this(env: SparkEnv, threadName: String) = this(new SocketAuthHelper(env.conf), threadName) - def this(threadName: String) = this(SparkEnv.get, threadName) - - val (port, secret) = PythonServer.setupOneConnectionServer(authHelper, threadName) { sock => - promise.complete(Try(handleConnection(sock))) - } - - /** - * Handle a connection which has already been authenticated. Any error from this function - * will clean up this connection and the entire server, and get propogated to [[getResult]]. - */ - def handleConnection(sock: Socket): T - - val promise = Promise[T]() - - /** - * Blocks indefinitely for [[handleConnection]] to finish, and returns that result. If - * handleConnection throws an exception, this will throw an exception which includes the original - * exception as a cause. - */ - def getResult(): T = { - getResult(Duration.Inf) - } - - def getResult(wait: Duration): T = { - ThreadUtils.awaitResult(promise.future, wait) - } - -} - -private[spark] object PythonServer { - - /** - * Create a socket server and run user function on the socket in a background thread. - * - * The socket server can only accept one connection, or close if no connection - * in 15 seconds. - * - * The thread will terminate after the supplied user function, or if there are any exceptions. - * - * If you need to get a result of the supplied function, create a subclass of [[PythonServer]] - * - * @return The port number of a local socket and the secret for authentication. - */ - def setupOneConnectionServer( - authHelper: SocketAuthHelper, - threadName: String) - (func: Socket => Unit): (Int, String) = { - val serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) - // Close the socket if no connection in 15 seconds - serverSocket.setSoTimeout(15000) - - new Thread(threadName) { - setDaemon(true) - override def run(): Unit = { - var sock: Socket = null - try { - sock = serverSocket.accept() - authHelper.authClient(sock) - func(sock) - } finally { - JavaUtils.closeQuietly(serverSocket) - JavaUtils.closeQuietly(sock) - } - } - }.start() - (serverSocket.getLocalPort, authHelper.secret) - } -} - /** * Sends decrypted broadcast data to python worker. See [[PythonRunner]] for entire protocol. */ private[spark] class EncryptedPythonBroadcastServer( val env: SparkEnv, val idsAndFiles: Seq[(Long, String)]) - extends PythonServer[Unit]("broadcast-decrypt-server") with Logging { + extends SocketAuthServer[Unit]("broadcast-decrypt-server") with Logging { override def handleConnection(socket: Socket): Unit = { val out = new DataOutputStream(new BufferedOutputStream(socket.getOutputStream())) @@ -914,7 +899,7 @@ private[spark] class EncryptedPythonBroadcastServer( * over a socket. This is used in preference to writing data to a file when encryption is enabled. */ private[spark] abstract class PythonRDDServer - extends PythonServer[JavaRDD[Array[Byte]]]("pyspark-parallelize-server") { + extends SocketAuthServer[JavaRDD[Array[Byte]]]("pyspark-parallelize-server") { def handleConnection(sock: Socket): JavaRDD[Array[Byte]] = { val in = sock.getInputStream() @@ -933,4 +918,3 @@ private[spark] class PythonParallelizeServer(sc: SparkContext, parallelism: Int) PythonRDD.readRDDFromInputStream(sc, input, parallelism) } } - diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala index 6e53a044e9a8c..dc6c59673d142 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala @@ -24,10 +24,12 @@ import java.nio.charset.StandardCharsets.UTF_8 import java.util.concurrent.atomic.AtomicBoolean import scala.collection.JavaConverters._ +import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY +import org.apache.spark.internal.config.{BUFFER_SIZE, EXECUTOR_CORES} +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util._ @@ -44,6 +46,8 @@ private[spark] object PythonEvalType { val SQL_GROUPED_MAP_PANDAS_UDF = 201 val SQL_GROUPED_AGG_PANDAS_UDF = 202 val SQL_WINDOW_AGG_PANDAS_UDF = 203 + val SQL_SCALAR_PANDAS_ITER_UDF = 204 + val SQL_MAP_PANDAS_ITER_UDF = 205 def toString(pythonEvalType: Int): String = pythonEvalType match { case NON_UDF => "NON_UDF" @@ -52,6 +56,8 @@ private[spark] object PythonEvalType { case SQL_GROUPED_MAP_PANDAS_UDF => "SQL_GROUPED_MAP_PANDAS_UDF" case SQL_GROUPED_AGG_PANDAS_UDF => "SQL_GROUPED_AGG_PANDAS_UDF" case SQL_WINDOW_AGG_PANDAS_UDF => "SQL_WINDOW_AGG_PANDAS_UDF" + case SQL_SCALAR_PANDAS_ITER_UDF => "SQL_SCALAR_PANDAS_ITER_UDF" + case SQL_MAP_PANDAS_ITER_UDF => "SQL_MAP_PANDAS_ITER_UDF" } } @@ -70,20 +76,22 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( require(funcs.length == argOffsets.length, "argOffsets should have the same length as funcs") private val conf = SparkEnv.get.conf - private val bufferSize = conf.getInt("spark.buffer.size", 65536) - private val reuseWorker = conf.getBoolean("spark.python.worker.reuse", true) + protected val bufferSize: Int = conf.get(BUFFER_SIZE) + private val reuseWorker = conf.get(PYTHON_WORKER_REUSE) // each python worker gets an equal part of the allocation. the worker pool will grow to the // number of concurrent tasks, which is determined by the number of cores in this executor. - private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY) - .map(_ / conf.getInt("spark.executor.cores", 1)) + private val memoryMb = conf.get(PYSPARK_EXECUTOR_MEMORY).map(_ / conf.get(EXECUTOR_CORES)) // All the Python functions should have the same exec, version and envvars. - protected val envVars = funcs.head.funcs.head.envVars - protected val pythonExec = funcs.head.funcs.head.pythonExec - protected val pythonVer = funcs.head.funcs.head.pythonVer + protected val envVars: java.util.Map[String, String] = funcs.head.funcs.head.envVars + protected val pythonExec: String = funcs.head.funcs.head.pythonExec + protected val pythonVer: String = funcs.head.funcs.head.pythonVer // TODO: support accumulator in multiple UDF - protected val accumulator = funcs.head.funcs.head.accumulator + protected val accumulator: PythonAccumulatorV2 = funcs.head.funcs.head.accumulator + + // Python accumulator is always set in production except in tests. See SPARK-27893 + private val maybeAccumulator: Option[PythonAccumulatorV2] = Option(accumulator) // Expose a ServerSocket to support method calls via socket from Python side. private[spark] var serverSocket: Option[ServerSocket] = None @@ -105,16 +113,19 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (memoryMb.isDefined) { envVars.put("PYSPARK_EXECUTOR_MEMORY_MB", memoryMb.get.toString) } + envVars.put("SPARK_BUFFER_SIZE", bufferSize.toString) val worker: Socket = env.createPythonWorker(pythonExec, envVars.asScala.toMap) - // Whether is the worker released into idle pool - val released = new AtomicBoolean(false) + // Whether is the worker released into idle pool or closed. When any codes try to release or + // close a worker, they should use `releasedOrClosed.compareAndSet` to flip the state to make + // sure there is only one winner that is going to release or close the worker. + val releasedOrClosed = new AtomicBoolean(false) // Start a thread to feed the process input from our parent's iterator val writerThread = newWriterThread(env, worker, inputIterator, partitionIndex, context) context.addTaskCompletionListener[Unit] { _ => writerThread.shutdownOnTaskCompletion() - if (!reuseWorker || !released.get) { + if (!reuseWorker || releasedOrClosed.compareAndSet(false, true)) { try { worker.close() } catch { @@ -131,7 +142,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = newReaderIterator( - stream, writerThread, startTime, env, worker, released, context) + stream, writerThread, startTime, env, worker, releasedOrClosed, context) new InterruptibleIterator(context, stdoutIterator) } @@ -148,7 +159,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[OUT] /** @@ -163,15 +174,15 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( context: TaskContext) extends Thread(s"stdout writer for $pythonExec") { - @volatile private var _exception: Exception = null + @volatile private var _exception: Throwable = null private val pythonIncludes = funcs.flatMap(_.funcs.flatMap(_.pythonIncludes.asScala)).toSet private val broadcastVars = funcs.flatMap(_.funcs.flatMap(_.broadcastVars.asScala)) setDaemon(true) - /** Contains the exception thrown while writing the parent iterator to the Python process. */ - def exception: Option[Exception] = Option(_exception) + /** Contains the throwable thrown while writing the parent iterator to the Python process. */ + def exception: Option[Throwable] = Option(_exception) /** Terminates the writer thread, ignoring any exceptions that may occur due to cleanup. */ def shutdownOnTaskCompletion() { @@ -270,6 +281,16 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( dataOut.writeInt(context.partitionId()) dataOut.writeInt(context.attemptNumber()) dataOut.writeLong(context.taskAttemptId()) + val resources = context.resources() + dataOut.writeInt(resources.size) + resources.foreach { case (k, v) => + PythonRDD.writeUTF(k, dataOut) + PythonRDD.writeUTF(v.name, dataOut) + dataOut.writeInt(v.addresses.size) + v.addresses.foreach { case addr => + PythonRDD.writeUTF(addr, dataOut) + } + } val localProps = context.getLocalProperties.asScala dataOut.writeInt(localProps.size) localProps.foreach { case (k, v) => @@ -345,18 +366,21 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( dataOut.writeInt(SpecialLengths.END_OF_STREAM) dataOut.flush() } catch { - case e: Exception if context.isCompleted || context.isInterrupted => - logDebug("Exception thrown after task completion (likely due to cleanup)", e) - if (!worker.isClosed) { - Utils.tryLog(worker.shutdownOutput()) - } - - case e: Exception => - // We must avoid throwing exceptions here, because the thread uncaught exception handler - // will kill the whole executor (see org.apache.spark.executor.Executor). - _exception = e - if (!worker.isClosed) { - Utils.tryLog(worker.shutdownOutput()) + case t: Throwable if (NonFatal(t) || t.isInstanceOf[Exception]) => + if (context.isCompleted || context.isInterrupted) { + logDebug("Exception/NonFatal Error thrown after task completion (likely due to " + + "cleanup)", t) + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } + } else { + // We must avoid throwing exceptions/NonFatals here, because the thread uncaught + // exception handler will kill the whole executor (see + // org.apache.spark.executor.Executor). + _exception = t + if (!worker.isClosed) { + Utils.tryLog(worker.shutdownOutput()) + } } } } @@ -392,7 +416,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext) extends Iterator[OUT] { @@ -459,13 +483,12 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( val updateLen = stream.readInt() val update = new Array[Byte](updateLen) stream.readFully(update) - accumulator.add(update) + maybeAccumulator.foreach(_.add(update)) } // Check whether the worker is ready to be re-used. if (stream.readInt() == SpecialLengths.END_OF_STREAM) { - if (reuseWorker) { + if (reuseWorker && releasedOrClosed.compareAndSet(false, true)) { env.releasePythonWorker(pythonExec, envVars.asScala.toMap, worker) - released.set(true) } } eos = true @@ -495,7 +518,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( extends Thread(s"Worker Monitor for $pythonExec") { /** How long to wait before killing the python worker if a task cannot be interrupted. */ - private val taskKillTimeout = env.conf.getTimeAsMs("spark.python.task.killTimeout", "2s") + private val taskKillTimeout = env.conf.get(PYTHON_TASK_KILL_TIMEOUT) setDaemon(true) @@ -510,7 +533,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT]( if (!context.isCompleted) { try { // Mimic the task name used in `Executor` to help the user find out the task to blame. - val taskName = s"${context.partitionId}.${context.taskAttemptId} " + + val taskName = s"${context.partitionId}.${context.attemptNumber} " + s"in stage ${context.stageId} (TID ${context.taskAttemptId})" logWarning(s"Incomplete task $taskName interrupted: Attempting to kill Python Worker") env.destroyPythonWorker(pythonExec, envVars.asScala.toMap, worker) @@ -565,9 +588,9 @@ private[spark] class PythonRunner(funcs: Seq[ChainedPythonFunctions]) startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala index cdce371dfcbfa..62d60475985b3 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonUtils.scala @@ -27,12 +27,15 @@ import org.apache.spark.SparkContext import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} private[spark] object PythonUtils { + val PY4J_ZIP_NAME = "py4j-0.10.8.1-src.zip" + /** Get the PYTHONPATH for PySpark, either from SPARK_HOME, if it is set, or from our JAR */ def sparkPythonPath: String = { val pythonPath = new ArrayBuffer[String] for (sparkHome <- sys.env.get("SPARK_HOME")) { pythonPath += Seq(sparkHome, "python", "lib", "pyspark.zip").mkString(File.separator) - pythonPath += Seq(sparkHome, "python", "lib", "py4j-0.10.7-src.zip").mkString(File.separator) + pythonPath += + Seq(sparkHome, "python", "lib", PY4J_ZIP_NAME).mkString(File.separator) } pythonPath ++= SparkContext.jarOfObject(this) pythonPath.mkString(File.pathSeparator) @@ -75,7 +78,11 @@ private[spark] object PythonUtils { jm.asScala.toMap } - def getEncryptionEnabled(sc: JavaSparkContext): Boolean = { + def isEncryptionEnabled(sc: JavaSparkContext): Boolean = { sc.conf.get(org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED) } + + def getBroadcastThreshold(sc: JavaSparkContext): Long = { + sc.conf.get(org.apache.spark.internal.config.BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) + } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala index 6afa37aa36fd3..6c37844a088ce 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala @@ -17,21 +17,23 @@ package org.apache.spark.api.python -import java.io.{DataInputStream, DataOutputStream, EOFException, InputStream, OutputStreamWriter} +import java.io.{DataInputStream, DataOutputStream, EOFException, InputStream} import java.net.{InetAddress, ServerSocket, Socket, SocketException} -import java.nio.charset.StandardCharsets import java.util.Arrays +import java.util.concurrent.TimeUnit +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Python._ import org.apache.spark.security.SocketAuthHelper import org.apache.spark.util.{RedirectThread, Utils} private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String, String]) - extends Logging { + extends Logging { self => import PythonWorkerFactory._ @@ -39,8 +41,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // pyspark/daemon.py (by default) and tell it to fork new workers for our tasks. This daemon // currently only works on UNIX-based systems now because it uses signals for child management, // so we can also fall back to launching workers, pyspark/worker.py (by default) directly. - val useDaemon = { - val useDaemonEnabled = SparkEnv.get.conf.getBoolean("spark.python.use.daemon", true) + private val useDaemon = { + val useDaemonEnabled = SparkEnv.get.conf.get(PYTHON_USE_DAEMON) // This flag is ignored on Windows as it's unable to fork. !System.getProperty("os.name").startsWith("Windows") && useDaemonEnabled @@ -51,44 +53,52 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String // as expert-only option, and shouldn't be used before knowing what it means exactly. // This configuration indicates the module to run the daemon to execute its Python workers. - val daemonModule = SparkEnv.get.conf.getOption("spark.python.daemon.module").map { value => - logInfo( - s"Python daemon module in PySpark is set to [$value] in 'spark.python.daemon.module', " + - "using this to start the daemon up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is enabled and the platform is not Windows.") - value - }.getOrElse("pyspark.daemon") + private val daemonModule = + SparkEnv.get.conf.get(PYTHON_DAEMON_MODULE).map { value => + logInfo( + s"Python daemon module in PySpark is set to [$value] in '${PYTHON_DAEMON_MODULE.key}', " + + "using this to start the daemon up. Note that this configuration only has an effect when " + + s"'${PYTHON_USE_DAEMON.key}' is enabled and the platform is not Windows.") + value + }.getOrElse("pyspark.daemon") // This configuration indicates the module to run each Python worker. - val workerModule = SparkEnv.get.conf.getOption("spark.python.worker.module").map { value => - logInfo( - s"Python worker module in PySpark is set to [$value] in 'spark.python.worker.module', " + - "using this to start the worker up. Note that this configuration only has an effect when " + - "'spark.python.use.daemon' is disabled or the platform is Windows.") - value - }.getOrElse("pyspark.worker") + private val workerModule = + SparkEnv.get.conf.get(PYTHON_WORKER_MODULE).map { value => + logInfo( + s"Python worker module in PySpark is set to [$value] in '${PYTHON_WORKER_MODULE.key}', " + + "using this to start the worker up. Note that this configuration only has an effect when " + + s"'${PYTHON_USE_DAEMON.key}' is disabled or the platform is Windows.") + value + }.getOrElse("pyspark.worker") private val authHelper = new SocketAuthHelper(SparkEnv.get.conf) - var daemon: Process = null + @GuardedBy("self") + private var daemon: Process = null val daemonHost = InetAddress.getByAddress(Array(127, 0, 0, 1)) - var daemonPort: Int = 0 - val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() - val idleWorkers = new mutable.Queue[Socket]() - var lastActivity = 0L + @GuardedBy("self") + private var daemonPort: Int = 0 + @GuardedBy("self") + private val daemonWorkers = new mutable.WeakHashMap[Socket, Int]() + @GuardedBy("self") + private val idleWorkers = new mutable.Queue[Socket]() + @GuardedBy("self") + private var lastActivityNs = 0L new MonitorThread().start() - var simpleWorkers = new mutable.WeakHashMap[Socket, Process]() + @GuardedBy("self") + private val simpleWorkers = new mutable.WeakHashMap[Socket, Process]() - val pythonPath = PythonUtils.mergePythonPaths( + private val pythonPath = PythonUtils.mergePythonPaths( PythonUtils.sparkPythonPath, envVars.getOrElse("PYTHONPATH", ""), sys.env.getOrElse("PYTHONPATH", "")) def create(): Socket = { if (useDaemon) { - synchronized { - if (idleWorkers.size > 0) { + self.synchronized { + if (idleWorkers.nonEmpty) { return idleWorkers.dequeue() } } @@ -117,7 +127,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String socket } - synchronized { + self.synchronized { // Start the daemon if it hasn't been started startDaemon() @@ -163,7 +173,9 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String try { val socket = serverSocket.accept() authHelper.authClient(socket) - simpleWorkers.put(socket, worker) + self.synchronized { + simpleWorkers.put(socket, worker) + } return socket } catch { case e: Exception => @@ -178,7 +190,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } private def startDaemon() { - synchronized { + self.synchronized { // Is it already running? if (daemon != null) { return @@ -278,10 +290,10 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String override def run() { while (true) { - synchronized { - if (lastActivity + IDLE_WORKER_TIMEOUT_MS < System.currentTimeMillis()) { + self.synchronized { + if (IDLE_WORKER_TIMEOUT_NS < System.nanoTime() - lastActivityNs) { cleanupIdleWorkers() - lastActivity = System.currentTimeMillis() + lastActivityNs = System.nanoTime() } } Thread.sleep(10000) @@ -303,7 +315,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } private def stopDaemon() { - synchronized { + self.synchronized { if (useDaemon) { cleanupIdleWorkers() @@ -325,7 +337,7 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String } def stopWorker(worker: Socket) { - synchronized { + self.synchronized { if (useDaemon) { if (daemon != null) { daemonWorkers.get(worker).foreach { pid => @@ -345,8 +357,8 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String def releaseWorker(worker: Socket) { if (useDaemon) { - synchronized { - lastActivity = System.currentTimeMillis() + self.synchronized { + lastActivityNs = System.nanoTime() idleWorkers.enqueue(worker) } } else { @@ -363,5 +375,5 @@ private[spark] class PythonWorkerFactory(pythonExec: String, envVars: Map[String private object PythonWorkerFactory { val PROCESS_WAIT_TIMEOUT_MS = 10000 - val IDLE_WORKER_TIMEOUT_MS = 60000 // kill idle workers after 1 minute + val IDLE_WORKER_TIMEOUT_NS = TimeUnit.MINUTES.toNanos(1) // kill idle workers after 1 minute } diff --git a/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala new file mode 100644 index 0000000000000..f96c5215cf0af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/api/r/BaseRRunner.scala @@ -0,0 +1,345 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.r + +import java.io._ +import java.net.{InetAddress, ServerSocket} +import java.util.Arrays + +import scala.io.Source +import scala.util.Try + +import org.apache.spark._ +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.BUFFER_SIZE +import org.apache.spark.internal.config.R._ +import org.apache.spark.util.Utils + +/** + * A helper class to run R UDFs in Spark. + */ +private[spark] abstract class BaseRRunner[IN, OUT]( + func: Array[Byte], + deserializer: String, + serializer: String, + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + numPartitions: Int, + isDataFrame: Boolean, + colNames: Array[String], + mode: Int) + extends Logging { + protected var bootTime: Double = _ + protected var dataStream: DataInputStream = _ + + def compute( + inputIterator: Iterator[IN], + partitionIndex: Int): Iterator[OUT] = { + // Timing start + bootTime = System.currentTimeMillis / 1000.0 + + // we expect two connections + val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost")) + val listenPort = serverSocket.getLocalPort() + + // The stdout/stderr is shared by multiple tasks, because we use one daemon + // to launch child process as worker. + val errThread = BaseRRunner.createRWorker(listenPort) + + // We use two sockets to separate input and output, then it's easy to manage + // the lifecycle of them to avoid deadlock. + // TODO: optimize it to use one socket + + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + dataStream = try { + val inSocket = serverSocket.accept() + BaseRRunner.authHelper.authClient(inSocket) + newWriterThread(inSocket.getOutputStream(), inputIterator, partitionIndex).start() + + // the socket used to receive the output of task + val outSocket = serverSocket.accept() + BaseRRunner.authHelper.authClient(outSocket) + val inputStream = new BufferedInputStream(outSocket.getInputStream) + new DataInputStream(inputStream) + } finally { + serverSocket.close() + } + + try { + newReaderIterator(dataStream, errThread) + } catch { + case e: Exception => + throw new SparkException("R computation failed with\n " + errThread.getLines(), e) + } + } + + /** + * Creates an iterator that reads data from R process. + */ + protected def newReaderIterator( + dataStream: DataInputStream, errThread: BufferedStreamThread): ReaderIterator + + /** + * Start a thread to write RDD data to the R process. + */ + protected def newWriterThread( + output: OutputStream, + iter: Iterator[IN], + partitionIndex: Int): WriterThread + + abstract class ReaderIterator( + stream: DataInputStream, + errThread: BufferedStreamThread) + extends Iterator[OUT] { + + private var nextObj: OUT = _ + // eos should be marked as true when the stream is ended. + protected var eos = false + + override def hasNext: Boolean = nextObj != null || { + if (!eos) { + nextObj = read() + hasNext + } else { + false + } + } + + override def next(): OUT = { + if (hasNext) { + val obj = nextObj + nextObj = null.asInstanceOf[OUT] + obj + } else { + Iterator.empty.next() + } + } + + /** + * Reads next object from the stream. + * When the stream reaches end of data, needs to process the following sections, + * and then returns null. + */ + protected def read(): OUT + } + + /** + * The thread responsible for writing the iterator to the R process. + */ + abstract class WriterThread( + output: OutputStream, + iter: Iterator[IN], + partitionIndex: Int) + extends Thread("writer for R") { + + private val env = SparkEnv.get + private val taskContext = TaskContext.get() + private val bufferSize = System.getProperty(BUFFER_SIZE.key, + BUFFER_SIZE.defaultValueString).toInt + private val stream = new BufferedOutputStream(output, bufferSize) + protected lazy val dataOut = new DataOutputStream(stream) + protected lazy val printOut = new PrintStream(stream) + + override def run(): Unit = { + try { + SparkEnv.set(env) + TaskContext.setTaskContext(taskContext) + dataOut.writeInt(partitionIndex) + + SerDe.writeString(dataOut, deserializer) + SerDe.writeString(dataOut, serializer) + + dataOut.writeInt(packageNames.length) + dataOut.write(packageNames) + + dataOut.writeInt(func.length) + dataOut.write(func) + + dataOut.writeInt(broadcastVars.length) + broadcastVars.foreach { broadcast => + // TODO(shivaram): Read a Long in R to avoid this cast + dataOut.writeInt(broadcast.id.toInt) + // TODO: Pass a byte array from R to avoid this cast ? + val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] + dataOut.writeInt(broadcastByteArr.length) + dataOut.write(broadcastByteArr) + } + + dataOut.writeInt(numPartitions) + dataOut.writeInt(mode) + + if (isDataFrame) { + SerDe.writeObject(dataOut, colNames, jvmObjectTracker = null) + } + + if (!iter.hasNext) { + dataOut.writeInt(0) + } else { + dataOut.writeInt(1) + } + + writeIteratorToStream(dataOut) + + stream.flush() + } catch { + // TODO: We should propagate this error to the task thread + case e: Exception => + logError("R Writer thread got an exception", e) + } finally { + Try(output.close()) + } + } + + /** + * Writes input data to the stream connected to the R worker. + */ + protected def writeIteratorToStream(dataOut: DataOutputStream): Unit + } +} + +private[spark] object SpecialLengths { + val TIMING_DATA = -1 +} + +private[spark] object RRunnerModes { + val RDD = 0 + val DATAFRAME_DAPPLY = 1 + val DATAFRAME_GAPPLY = 2 +} + +private[spark] class BufferedStreamThread( + in: InputStream, + name: String, + errBufferSize: Int) extends Thread(name) with Logging { + val lines = new Array[String](errBufferSize) + var lineIdx = 0 + override def run() { + for (line <- Source.fromInputStream(in).getLines) { + synchronized { + lines(lineIdx) = line + lineIdx = (lineIdx + 1) % errBufferSize + } + logInfo(line) + } + } + + def getLines(): String = synchronized { + (0 until errBufferSize).filter { x => + lines((x + lineIdx) % errBufferSize) != null + }.map { x => + lines((x + lineIdx) % errBufferSize) + }.mkString("\n") + } +} + +private[r] object BaseRRunner { + // Because forking processes from Java is expensive, we prefer to launch + // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. + // This daemon currently only works on UNIX-based systems now, so we should + // also fall back to launching workers (worker.R) directly. + private[this] var errThread: BufferedStreamThread = _ + private[this] var daemonChannel: DataOutputStream = _ + + private lazy val authHelper = { + val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + new RAuthHelper(conf) + } + + /** + * Start a thread to print the process's stderr to ours + */ + private def startStdoutThread(proc: Process): BufferedStreamThread = { + val BUFFER_SIZE = 100 + val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) + thread.setDaemon(true) + thread.start() + thread + } + + private def createRProcess(port: Int, script: String): BufferedStreamThread = { + // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", + // but kept here for backward compatibility. + val sparkConf = SparkEnv.get.conf + var rCommand = sparkConf.get(SPARKR_COMMAND) + rCommand = sparkConf.get(R_COMMAND).orElse(Some(rCommand)).get + + val rConnectionTimeout = sparkConf.get(R_BACKEND_CONNECTION_TIMEOUT) + val rOptions = "--vanilla" + val rLibDir = RUtils.sparkRPackagePath(isDriver = false) + val rExecScript = rLibDir(0) + "/SparkR/worker/" + script + val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript)) + // Unset the R_TESTS environment variable for workers. + // This is set by R CMD check as startup.Rs + // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) + // and confuses worker script which tries to load a non-existent file + pb.environment().put("R_TESTS", "") + pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(",")) + pb.environment().put("SPARKR_WORKER_PORT", port.toString) + pb.environment().put("SPARKR_BACKEND_CONNECTION_TIMEOUT", rConnectionTimeout.toString) + pb.environment().put("SPARKR_SPARKFILES_ROOT_DIR", SparkFiles.getRootDirectory()) + pb.environment().put("SPARKR_IS_RUNNING_ON_WORKER", "TRUE") + pb.environment().put("SPARKR_WORKER_SECRET", authHelper.secret) + pb.redirectErrorStream(true) // redirect stderr into stdout + val proc = pb.start() + val errThread = startStdoutThread(proc) + errThread + } + + /** + * ProcessBuilder used to launch worker R processes. + */ + def createRWorker(port: Int): BufferedStreamThread = { + val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) + if (!Utils.isWindows && useDaemon) { + synchronized { + if (daemonChannel == null) { + // we expect one connections + val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) + val daemonPort = serverSocket.getLocalPort + errThread = createRProcess(daemonPort, "daemon.R") + // the socket used to send out the input of task + serverSocket.setSoTimeout(10000) + val sock = serverSocket.accept() + try { + authHelper.authClient(sock) + daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) + } finally { + serverSocket.close() + } + } + try { + daemonChannel.writeInt(port) + daemonChannel.flush() + } catch { + case e: IOException => + // daemon process died + daemonChannel.close() + daemonChannel = null + errThread = null + // fail the current task, retry by scheduler + throw e + } + errThread + } + } else { + createRProcess(port, "worker.R") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala index 7ce2581555014..c755dcba6bcea 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackend.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackend.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.r -import java.io.{DataInputStream, DataOutputStream, File, FileOutputStream, IOException} +import java.io.{DataOutputStream, File, FileOutputStream, IOException} import java.net.{InetAddress, InetSocketAddress, ServerSocket, Socket} import java.util.concurrent.TimeUnit @@ -30,10 +30,9 @@ import io.netty.handler.codec.LengthFieldBasedFrameDecoder import io.netty.handler.codec.bytes.{ByteArrayDecoder, ByteArrayEncoder} import io.netty.handler.timeout.ReadTimeoutHandler -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.Utils +import org.apache.spark.internal.config.R._ /** * Netty-based backend server that is used to communicate between R and Java. @@ -48,11 +47,9 @@ private[spark] class RBackend { private[r] val jvmObjectTracker = new JVMObjectTracker def init(): (Int, RAuthHelper) = { - val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) - bossGroup = new NioEventLoopGroup( - conf.getInt("spark.r.numRBackendThreads", SparkRDefaults.DEFAULT_NUM_RBACKEND_THREADS)) + val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) + bossGroup = new NioEventLoopGroup(conf.get(R_NUM_BACKEND_THREADS)) val workerGroup = bossGroup val handler = new RBackendHandler(this) val authHelper = new RAuthHelper(conf) @@ -99,7 +96,7 @@ private[spark] class RBackend { if (bootstrap != null && bootstrap.config().group() != null) { bootstrap.config().group().shutdownGracefully() } - if (bootstrap != null && bootstrap.childGroup() != null) { + if (bootstrap != null && bootstrap.config().childGroup() != null) { bootstrap.config().childGroup().shutdownGracefully() } bootstrap = null @@ -127,9 +124,8 @@ private[spark] object RBackend extends Logging { val listenPort = serverSocket.getLocalPort() // Connection timeout is set by socket client. To make it configurable we will pass the // timeout value to client inside the temp file - val conf = new SparkConf() - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) // tell the R process via temporary file val path = args(0) @@ -147,7 +143,7 @@ private[spark] object RBackend extends Logging { new Thread("wait for socket to close") { setDaemon(true) override def run(): Unit = { - // any un-catched exception will also shutdown JVM + // any uncaught exception will also shutdown JVM val buf = new Array[Byte](1024) // shutdown JVM if R does not connect back in 10 seconds serverSocket.setSoTimeout(10000) diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala index 4162e4a6c7476..8cd95ee653ebe 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendAuthHandler.scala @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets.UTF_8 import io.netty.channel.{Channel, ChannelHandlerContext, SimpleChannelInboundHandler} import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils /** * Authentication handler for connections from the R process. diff --git a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala index 18fc595301f46..f2f81b11fc813 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RBackendHandler.scala @@ -20,15 +20,14 @@ package org.apache.spark.api.r import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.util.concurrent.TimeUnit -import scala.language.existentials - import io.netty.channel.{ChannelHandlerContext, SimpleChannelInboundHandler} import io.netty.channel.ChannelHandler.Sharable import io.netty.handler.timeout.ReadTimeoutException -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.api.r.SerDe._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.R._ import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -97,11 +96,9 @@ private[r] class RBackendHandler(server: RBackend) ctx.write(pingBaos.toByteArray) } } - val conf = new SparkConf() - val heartBeatInterval = conf.getInt( - "spark.r.heartBeatInterval", SparkRDefaults.DEFAULT_HEARTBEAT_INTERVAL) - val backendConnectionTimeout = conf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) + val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + val heartBeatInterval = conf.get(R_HEARTBEAT_INTERVAL) + val backendConnectionTimeout = conf.get(R_BACKEND_CONNECTION_TIMEOUT) val interval = Math.min(heartBeatInterval, backendConnectionTimeout - 1) execService.scheduleAtFixedRate(pingRunner, interval, interval, TimeUnit.SECONDS) diff --git a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala index 1dc61c7eef33c..892e69bfce5ce 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRDD.scala @@ -17,9 +17,8 @@ package org.apache.spark.api.r -import java.io.{DataInputStream, File} +import java.io.{File, OutputStream} import java.net.Socket -import java.nio.charset.StandardCharsets.UTF_8 import java.util.{Map => JMap} import scala.collection.JavaConverters._ @@ -27,11 +26,10 @@ import scala.reflect.ClassTag import org.apache.spark._ import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} -import org.apache.spark.api.python.{PythonRDD, PythonServer} import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.security.SocketAuthHelper +import org.apache.spark.security.SocketAuthServer private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( parent: RDD[T], @@ -45,7 +43,7 @@ private abstract class BaseRRDD[T: ClassTag, U: ClassTag]( override def getPartitions: Array[Partition] = parent.partitions override def compute(partition: Partition, context: TaskContext): Iterator[U] = { - val runner = new RRunner[U]( + val runner = new RRunner[T, U]( func, deserializer, serializer, packageNames, broadcastVars, numPartitions) // The parent may be also an RRDD, so we should launch it first. @@ -104,7 +102,7 @@ private class StringRRDD[T: ClassTag]( lazy val asJavaRDD : JavaRDD[String] = JavaRDD.fromRDD(this) } -private[r] object RRDD { +private[spark] object RRDD { def createSparkContext( master: String, appName: String, @@ -163,7 +161,12 @@ private[r] object RRDD { */ def createRDDFromFile(jsc: JavaSparkContext, fileName: String, parallelism: Int): JavaRDD[Array[Byte]] = { - PythonRDD.readRDDFromFile(jsc, fileName, parallelism) + JavaRDD.readRDDFromFile(jsc, fileName, parallelism) + } + + private[spark] def serveToStream( + threadName: String)(writeFunc: OutputStream => Unit): Array[Any] = { + SocketAuthServer.serveToStream(threadName, new RAuthHelper(SparkEnv.get.conf))(writeFunc) } } @@ -172,23 +175,11 @@ private[r] object RRDD { * over a socket. This is used in preference to writing data to a file when encryption is enabled. */ private[spark] class RParallelizeServer(sc: JavaSparkContext, parallelism: Int) - extends PythonServer[JavaRDD[Array[Byte]]]( - new RSocketAuthHelper(), "sparkr-parallelize-server") { + extends SocketAuthServer[JavaRDD[Array[Byte]]]( + new RAuthHelper(SparkEnv.get.conf), "sparkr-parallelize-server") { override def handleConnection(sock: Socket): JavaRDD[Array[Byte]] = { val in = sock.getInputStream() - PythonRDD.readRDDFromInputStream(sc.sc, in, parallelism) - } -} - -private[spark] class RSocketAuthHelper extends SocketAuthHelper(SparkEnv.get.conf) { - override protected def readUtf8(s: Socket): String = { - val din = new DataInputStream(s.getInputStream()) - val len = din.readInt() - val bytes = new Array[Byte](len) - din.readFully(bytes) - // The R code adds a null terminator to serialized strings, so ignore it here. - assert(bytes(bytes.length - 1) == 0) // sanity check. - new String(bytes, 0, bytes.length - 1, UTF_8) + JavaRDD.readRDDFromInputStream(sc.sc, in, parallelism) } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala index e7fdc3963945a..0327386b45ed5 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RRunner.scala @@ -18,21 +18,14 @@ package org.apache.spark.api.r import java.io._ -import java.net.{InetAddress, ServerSocket} -import java.util.Arrays - -import scala.io.Source -import scala.util.Try import org.apache.spark._ import org.apache.spark.broadcast.Broadcast -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils /** * A helper class to run R UDFs in Spark. */ -private[spark] class RRunner[U]( +private[spark] class RRunner[IN, OUT]( func: Array[Byte], deserializer: String, serializer: String, @@ -42,368 +35,149 @@ private[spark] class RRunner[U]( isDataFrame: Boolean = false, colNames: Array[String] = null, mode: Int = RRunnerModes.RDD) - extends Logging { - private var bootTime: Double = _ - private var dataStream: DataInputStream = _ - val readData = numPartitions match { - case -1 => - serializer match { - case SerializationFormats.STRING => readStringData _ - case _ => readByteArrayData _ + extends BaseRRunner[IN, OUT]( + func, + deserializer, + serializer, + packageNames, + broadcastVars, + numPartitions, + isDataFrame, + colNames, + mode) { + + protected def newReaderIterator( + dataStream: DataInputStream, errThread: BufferedStreamThread): ReaderIterator = { + new ReaderIterator(dataStream, errThread) { + private val readData = numPartitions match { + case -1 => + serializer match { + case SerializationFormats.STRING => readStringData _ + case _ => readByteArrayData _ + } + case _ => readShuffledData _ } - case _ => readShuffledData _ - } - - def compute( - inputIterator: Iterator[_], - partitionIndex: Int): Iterator[U] = { - // Timing start - bootTime = System.currentTimeMillis / 1000.0 - - // we expect two connections - val serverSocket = new ServerSocket(0, 2, InetAddress.getByName("localhost")) - val listenPort = serverSocket.getLocalPort() - - // The stdout/stderr is shared by multiple tasks, because we use one daemon - // to launch child process as worker. - val errThread = RRunner.createRWorker(listenPort) - - // We use two sockets to separate input and output, then it's easy to manage - // the lifecycle of them to avoid deadlock. - // TODO: optimize it to use one socket - - // the socket used to send out the input of task - serverSocket.setSoTimeout(10000) - dataStream = try { - val inSocket = serverSocket.accept() - RRunner.authHelper.authClient(inSocket) - startStdinThread(inSocket.getOutputStream(), inputIterator, partitionIndex) - - // the socket used to receive the output of task - val outSocket = serverSocket.accept() - RRunner.authHelper.authClient(outSocket) - val inputStream = new BufferedInputStream(outSocket.getInputStream) - new DataInputStream(inputStream) - } finally { - serverSocket.close() - } - try { - return new Iterator[U] { - def next(): U = { - val obj = _nextObj - if (hasNext) { - _nextObj = read() - } - obj + private def readShuffledData(length: Int): (Int, Array[Byte]) = { + length match { + case length if length == 2 => + val hashedKey = dataStream.readInt() + val contentPairsLength = dataStream.readInt() + val contentPairs = new Array[Byte](contentPairsLength) + dataStream.readFully(contentPairs) + (hashedKey, contentPairs) + case _ => null } + } - var _nextObj = read() - - def hasNext(): Boolean = { - val hasMore = (_nextObj != null) - if (!hasMore) { - dataStream.close() - } - hasMore + private def readByteArrayData(length: Int): Array[Byte] = { + length match { + case length if length > 0 => + val obj = new Array[Byte](length) + dataStream.readFully(obj) + obj + case _ => null } } - } catch { - case e: Exception => - throw new SparkException("R computation failed with\n " + errThread.getLines()) - } - } - /** - * Start a thread to write RDD data to the R process. - */ - private def startStdinThread( - output: OutputStream, - iter: Iterator[_], - partitionIndex: Int): Unit = { - val env = SparkEnv.get - val taskContext = TaskContext.get() - val bufferSize = System.getProperty("spark.buffer.size", "65536").toInt - val stream = new BufferedOutputStream(output, bufferSize) + private def readStringData(length: Int): String = { + length match { + case length if length > 0 => + SerDe.readStringBytes(dataStream, length) + case _ => null + } + } - new Thread("writer for R") { - override def run(): Unit = { + /** + * Reads next object from the stream. + * When the stream reaches end of data, needs to process the following sections, + * and then returns null. + */ + override protected def read(): OUT = { try { - SparkEnv.set(env) - TaskContext.setTaskContext(taskContext) - val dataOut = new DataOutputStream(stream) - dataOut.writeInt(partitionIndex) - - SerDe.writeString(dataOut, deserializer) - SerDe.writeString(dataOut, serializer) - - dataOut.writeInt(packageNames.length) - dataOut.write(packageNames) - - dataOut.writeInt(func.length) - dataOut.write(func) - - dataOut.writeInt(broadcastVars.length) - broadcastVars.foreach { broadcast => - // TODO(shivaram): Read a Long in R to avoid this cast - dataOut.writeInt(broadcast.id.toInt) - // TODO: Pass a byte array from R to avoid this cast ? - val broadcastByteArr = broadcast.value.asInstanceOf[Array[Byte]] - dataOut.writeInt(broadcastByteArr.length) - dataOut.write(broadcastByteArr) + val length = dataStream.readInt() + + length match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length > 0 => + readData(length).asInstanceOf[OUT] + case length if length == 0 => + // End of stream + eos = true + null.asInstanceOf[OUT] } - - dataOut.writeInt(numPartitions) - dataOut.writeInt(mode) - - if (isDataFrame) { - SerDe.writeObject(dataOut, colNames, jvmObjectTracker = null) - } - - if (!iter.hasNext) { - dataOut.writeInt(0) - } else { - dataOut.writeInt(1) - } - - val printOut = new PrintStream(stream) - - def writeElem(elem: Any): Unit = { - if (deserializer == SerializationFormats.BYTE) { - val elemArr = elem.asInstanceOf[Array[Byte]] - dataOut.writeInt(elemArr.length) - dataOut.write(elemArr) - } else if (deserializer == SerializationFormats.ROW) { - dataOut.write(elem.asInstanceOf[Array[Byte]]) - } else if (deserializer == SerializationFormats.STRING) { - // write string(for StringRRDD) - // scalastyle:off println - printOut.println(elem) - // scalastyle:on println - } - } - - for (elem <- iter) { - elem match { - case (key, innerIter: Iterator[_]) => - for (innerElem <- innerIter) { - writeElem(innerElem) - } - // Writes key which can be used as a boundary in group-aggregate - dataOut.writeByte('r') - writeElem(key) - case (key, value) => - writeElem(key) - writeElem(value) - case _ => - writeElem(elem) - } - } - - stream.flush() } catch { - // TODO: We should propagate this error to the task thread - case e: Exception => - logError("R Writer thread got an exception", e) - } finally { - Try(output.close()) + case eof: EOFException => + throw new SparkException("R worker exited unexpectedly (cranshed)", eof) } } - }.start() - } - - private def read(): U = { - try { - val length = dataStream.readInt() - - length match { - case SpecialLengths.TIMING_DATA => - // Timing data from R worker - val boot = dataStream.readDouble - bootTime - val init = dataStream.readDouble - val broadcast = dataStream.readDouble - val input = dataStream.readDouble - val compute = dataStream.readDouble - val output = dataStream.readDouble - logInfo( - ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + - "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + - "total = %.3f s").format( - boot, - init, - broadcast, - input, - compute, - output, - boot + init + broadcast + input + compute + output)) - read() - case length if length >= 0 => - readData(length).asInstanceOf[U] - } - } catch { - case eof: EOFException => - throw new SparkException("R worker exited unexpectedly (cranshed)", eof) - } - } - - private def readShuffledData(length: Int): (Int, Array[Byte]) = { - length match { - case length if length == 2 => - val hashedKey = dataStream.readInt() - val contentPairsLength = dataStream.readInt() - val contentPairs = new Array[Byte](contentPairsLength) - dataStream.readFully(contentPairs) - (hashedKey, contentPairs) - case _ => null - } - } - - private def readByteArrayData(length: Int): Array[Byte] = { - length match { - case length if length > 0 => - val obj = new Array[Byte](length) - dataStream.readFully(obj) - obj - case _ => null - } - } - - private def readStringData(length: Int): String = { - length match { - case length if length > 0 => - SerDe.readStringBytes(dataStream, length) - case _ => null - } - } -} - -private object SpecialLengths { - val TIMING_DATA = -1 -} - -private[spark] object RRunnerModes { - val RDD = 0 - val DATAFRAME_DAPPLY = 1 - val DATAFRAME_GAPPLY = 2 -} - -private[r] class BufferedStreamThread( - in: InputStream, - name: String, - errBufferSize: Int) extends Thread(name) with Logging { - val lines = new Array[String](errBufferSize) - var lineIdx = 0 - override def run() { - for (line <- Source.fromInputStream(in).getLines) { - synchronized { - lines(lineIdx) = line - lineIdx = (lineIdx + 1) % errBufferSize - } - logInfo(line) } } - def getLines(): String = synchronized { - (0 until errBufferSize).filter { x => - lines((x + lineIdx) % errBufferSize) != null - }.map { x => - lines((x + lineIdx) % errBufferSize) - }.mkString("\n") - } -} - -private[r] object RRunner { - // Because forking processes from Java is expensive, we prefer to launch - // a single R daemon (daemon.R) and tell it to fork new workers for our tasks. - // This daemon currently only works on UNIX-based systems now, so we should - // also fall back to launching workers (worker.R) directly. - private[this] var errThread: BufferedStreamThread = _ - private[this] var daemonChannel: DataOutputStream = _ - - private lazy val authHelper = { - val conf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) - new RAuthHelper(conf) - } - /** - * Start a thread to print the process's stderr to ours - */ - private def startStdoutThread(proc: Process): BufferedStreamThread = { - val BUFFER_SIZE = 100 - val thread = new BufferedStreamThread(proc.getInputStream, "stdout reader for R", BUFFER_SIZE) - thread.setDaemon(true) - thread.start() - thread - } - - private def createRProcess(port: Int, script: String): BufferedStreamThread = { - // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", - // but kept here for backward compatibility. - val sparkConf = SparkEnv.get.conf - var rCommand = sparkConf.get("spark.sparkr.r.command", "Rscript") - rCommand = sparkConf.get("spark.r.command", rCommand) - - val rConnectionTimeout = sparkConf.getInt( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT) - val rOptions = "--vanilla" - val rLibDir = RUtils.sparkRPackagePath(isDriver = false) - val rExecScript = rLibDir(0) + "/SparkR/worker/" + script - val pb = new ProcessBuilder(Arrays.asList(rCommand, rOptions, rExecScript)) - // Unset the R_TESTS environment variable for workers. - // This is set by R CMD check as startup.Rs - // (http://svn.r-project.org/R/trunk/src/library/tools/R/testing.R) - // and confuses worker script which tries to load a non-existent file - pb.environment().put("R_TESTS", "") - pb.environment().put("SPARKR_RLIBDIR", rLibDir.mkString(",")) - pb.environment().put("SPARKR_WORKER_PORT", port.toString) - pb.environment().put("SPARKR_BACKEND_CONNECTION_TIMEOUT", rConnectionTimeout.toString) - pb.environment().put("SPARKR_SPARKFILES_ROOT_DIR", SparkFiles.getRootDirectory()) - pb.environment().put("SPARKR_IS_RUNNING_ON_WORKER", "TRUE") - pb.environment().put("SPARKR_WORKER_SECRET", authHelper.secret) - pb.redirectErrorStream(true) // redirect stderr into stdout - val proc = pb.start() - val errThread = startStdoutThread(proc) - errThread - } - - /** - * ProcessBuilder used to launch worker R processes. + * Start a thread to write RDD data to the R process. */ - def createRWorker(port: Int): BufferedStreamThread = { - val useDaemon = SparkEnv.get.conf.getBoolean("spark.sparkr.use.daemon", true) - if (!Utils.isWindows && useDaemon) { - synchronized { - if (daemonChannel == null) { - // we expect one connections - val serverSocket = new ServerSocket(0, 1, InetAddress.getByName("localhost")) - val daemonPort = serverSocket.getLocalPort - errThread = createRProcess(daemonPort, "daemon.R") - // the socket used to send out the input of task - serverSocket.setSoTimeout(10000) - val sock = serverSocket.accept() - try { - authHelper.authClient(sock) - daemonChannel = new DataOutputStream(new BufferedOutputStream(sock.getOutputStream)) - } finally { - serverSocket.close() + protected override def newWriterThread( + output: OutputStream, + iter: Iterator[IN], + partitionIndex: Int): WriterThread = { + new WriterThread(output, iter, partitionIndex) { + + /** + * Writes input data to the stream connected to the R worker. + */ + override protected def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + def writeElem(elem: Any): Unit = { + if (deserializer == SerializationFormats.BYTE) { + val elemArr = elem.asInstanceOf[Array[Byte]] + dataOut.writeInt(elemArr.length) + dataOut.write(elemArr) + } else if (deserializer == SerializationFormats.ROW) { + dataOut.write(elem.asInstanceOf[Array[Byte]]) + } else if (deserializer == SerializationFormats.STRING) { + // write string(for StringRRDD) + // scalastyle:off println + printOut.println(elem) + // scalastyle:on println } } - try { - daemonChannel.writeInt(port) - daemonChannel.flush() - } catch { - case e: IOException => - // daemon process died - daemonChannel.close() - daemonChannel = null - errThread = null - // fail the current task, retry by scheduler - throw e + + for (elem <- iter) { + elem match { + case (key, innerIter: Iterator[_]) => + for (innerElem <- innerIter) { + writeElem(innerElem) + } + // Writes key which can be used as a boundary in group-aggregate + dataOut.writeByte('r') + writeElem(key) + case (key, value) => + writeElem(key) + writeElem(value) + case _ => + writeElem(elem) + } } - errThread } - } else { - createRProcess(port, "worker.R") } } } diff --git a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala index 9bf35af1da925..311fade127839 100644 --- a/core/src/main/scala/org/apache/spark/api/r/RUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/r/RUtils.scala @@ -22,7 +22,7 @@ import java.util.Arrays import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.api.java.JavaSparkContext -import org.apache.spark.api.python.PythonUtils +import org.apache.spark.internal.config._ private[spark] object RUtils { // Local path where R binary packages built from R source code contained in the spark @@ -60,10 +60,10 @@ private[spark] object RUtils { def sparkRPackagePath(isDriver: Boolean): Seq[String] = { val (master, deployMode) = if (isDriver) { - (sys.props("spark.master"), sys.props("spark.submit.deployMode")) + (sys.props("spark.master"), sys.props(SUBMIT_DEPLOY_MODE.key)) } else { val sparkConf = SparkEnv.get.conf - (sparkConf.get("spark.master"), sparkConf.get("spark.submit.deployMode", "client")) + (sparkConf.get("spark.master"), sparkConf.get(SUBMIT_DEPLOY_MODE)) } val isYarnCluster = master != null && master.contains("yarn") && deployMode == "cluster" @@ -107,5 +107,7 @@ private[spark] object RUtils { } } - def getEncryptionEnabled(sc: JavaSparkContext): Boolean = PythonUtils.getEncryptionEnabled(sc) + def isEncryptionEnabled(sc: JavaSparkContext): Boolean = { + sc.conf.get(org.apache.spark.internal.config.IO_ENCRYPTION_ENABLED) + } } diff --git a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala index 537ab57f9664d..917203831404f 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SerDe.scala +++ b/core/src/main/scala/org/apache/spark/api/r/SerDe.scala @@ -74,9 +74,9 @@ private[spark] object SerDe { jvmObjectTracker: JVMObjectTracker): Object = { dataType match { case 'n' => null - case 'i' => new java.lang.Integer(readInt(dis)) - case 'd' => new java.lang.Double(readDouble(dis)) - case 'b' => new java.lang.Boolean(readBoolean(dis)) + case 'i' => java.lang.Integer.valueOf(readInt(dis)) + case 'd' => java.lang.Double.valueOf(readDouble(dis)) + case 'b' => java.lang.Boolean.valueOf(readBoolean(dis)) case 'c' => readString(dis) case 'e' => readMap(dis, jvmObjectTracker) case 'r' => readBytes(dis) @@ -102,7 +102,7 @@ private[spark] object SerDe { def readBytes(in: DataInputStream): Array[Byte] = { val len = readInt(in) val out = new Array[Byte](len) - val bytesRead = in.readFully(out) + in.readFully(out) out } diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index 24d953e77bfab..0e81ad198db67 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -92,10 +92,9 @@ abstract class Broadcast[T: ClassTag](val id: Long) extends Serializable with Lo /** * Destroy all data and metadata related to this broadcast variable. Use this with caution; * once a broadcast variable has been destroyed, it cannot be used again. - * This method blocks until destroy has completed */ def destroy() { - destroy(blocking = true) + destroy(blocking = false) } /** diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index cbd49e070f2eb..f416be883dcfd 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -18,6 +18,7 @@ package org.apache.spark.broadcast import java.io._ +import java.lang.ref.SoftReference import java.nio.ByteBuffer import java.util.zip.Adler32 @@ -26,7 +27,7 @@ import scala.reflect.ClassTag import scala.util.Random import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.io.CompressionCodec import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ @@ -61,9 +62,11 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) * Value of the broadcast object on executors. This is reconstructed by [[readBroadcastBlock]], * which builds this value by reading blocks from the driver and/or other executors. * - * On the driver, if the value is required, it is read lazily from the block manager. + * On the driver, if the value is required, it is read lazily from the block manager. We hold + * a soft reference so that it can be garbage collected if required, as we can always reconstruct + * in the future. */ - @transient private lazy val _value: T = readBroadcastBlock() + @transient private var _value: SoftReference[T] = _ /** The compression codec to use, or None if compression is disabled */ @transient private var compressionCodec: Option[CompressionCodec] = _ @@ -71,14 +74,14 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) @transient private var blockSize: Int = _ private def setConf(conf: SparkConf) { - compressionCodec = if (conf.getBoolean("spark.broadcast.compress", true)) { + compressionCodec = if (conf.get(config.BROADCAST_COMPRESS)) { Some(CompressionCodec.createCodec(conf)) } else { None } // Note: use getSizeAsKb (not bytes) to maintain compatibility if no units are provided - blockSize = conf.getSizeAsKb("spark.broadcast.blockSize", "4m").toInt * 1024 - checksumEnabled = conf.getBoolean("spark.broadcast.checksum", true) + blockSize = conf.get(config.BROADCAST_BLOCKSIZE).toInt * 1024 + checksumEnabled = conf.get(config.BROADCAST_CHECKSUM) } setConf(SparkEnv.get.conf) @@ -92,8 +95,15 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) /** The checksum for all the blocks. */ private var checksums: Array[Int] = _ - override protected def getValue() = { - _value + override protected def getValue() = synchronized { + val memoized: T = if (_value == null) null.asInstanceOf[T] else _value.get + if (memoized != null) { + memoized + } else { + val newlyRead = readBroadcastBlock() + _value = new SoftReference[T](newlyRead) + newlyRead + } } private def calcChecksum(block: ByteBuffer): Int = { @@ -205,8 +215,8 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) } private def readBroadcastBlock(): T = Utils.tryOrIOException { - TorrentBroadcast.synchronized { - val broadcastCache = SparkEnv.get.broadcastManager.cachedValues + val broadcastCache = SparkEnv.get.broadcastManager.cachedValues + broadcastCache.synchronized { Option(broadcastCache.get(broadcastId)).map(_.asInstanceOf[T]).getOrElse { setConf(SparkEnv.get.conf) @@ -226,10 +236,12 @@ private[spark] class TorrentBroadcast[T: ClassTag](obj: T, id: Long) throw new SparkException(s"Failed to get locally stored broadcast data: $broadcastId") } case None => - logInfo("Started reading broadcast variable " + id) - val startTimeMs = System.currentTimeMillis() + val estimatedTotalSize = Utils.bytesToString(numBlocks * blockSize) + logInfo(s"Started reading broadcast variable $id with $numBlocks pieces " + + s"(estimated total size $estimatedTotalSize)") + val startTimeNs = System.nanoTime() val blocks = readBlocks() - logInfo("Reading broadcast variable " + id + " took" + Utils.getUsedTimeMs(startTimeMs)) + logInfo(s"Reading broadcast variable $id took ${Utils.getUsedTimeNs(startTimeNs)}") try { val obj = TorrentBroadcast.unBlockifyObject[T]( diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index d5145094ec079..ea7c902b1b6bb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -27,7 +27,8 @@ import org.apache.log4j.Logger import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.Network.RPC_ASK_TIMEOUT import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.util.{SparkExitCode, ThreadUtils, Utils} @@ -60,6 +61,10 @@ private class ClientEndpoint( private val lostMasters = new HashSet[RpcAddress] private var activeMasterEndpoint: RpcEndpointRef = null + private def getProperty(key: String, conf: SparkConf): Option[String] = { + sys.props.get(key).orElse(conf.getOption(key)) + } + override def onStart(): Unit = { driverArgs.cmd match { case "launch" => @@ -68,19 +73,20 @@ private class ClientEndpoint( // people call `addJar` assuming the jar is in the same directory. val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" - val classPathConf = "spark.driver.extraClassPath" - val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => + val classPathConf = config.DRIVER_CLASS_PATH.key + val classPathEntries = getProperty(classPathConf, conf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val libraryPathConf = "spark.driver.extraLibraryPath" - val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => + val libraryPathConf = config.DRIVER_LIBRARY_PATH.key + val libraryPathEntries = getProperty(libraryPathConf, conf).toSeq.flatMap { cp => cp.split(java.io.File.pathSeparator) } - val extraJavaOptsConf = "spark.driver.extraJavaOptions" - val extraJavaOpts = sys.props.get(extraJavaOptsConf) + val extraJavaOptsConf = config.DRIVER_JAVA_OPTIONS.key + val extraJavaOpts = getProperty(extraJavaOptsConf, conf) .map(Utils.splitCommandString).getOrElse(Seq.empty) + val sparkJavaOpts = Utils.sparkJavaOpts(conf) val javaOpts = sparkJavaOpts ++ extraJavaOpts val command = new Command(mainClass, @@ -226,8 +232,8 @@ private[spark] class ClientApp extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { val driverArgs = new ClientArguments(args) - if (!conf.contains("spark.rpc.askTimeout")) { - conf.set("spark.rpc.askTimeout", "10s") + if (!conf.contains(RPC_ASK_TIMEOUT)) { + conf.set(RPC_ASK_TIMEOUT, "10s") } Logger.getRootLogger.setLevel(driverArgs.logLevel) diff --git a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala index 178bdcfccb603..5a17a6b6e169c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DependencyUtils.scala @@ -61,11 +61,12 @@ private[deploy] object DependencyUtils extends Logging { hadoopConf: Configuration, secMgr: SecurityManager): String = { val targetDir = Utils.createTempDir() + val userJarName = userJar.split(File.separatorChar).last Option(jars) .map { resolveGlobPaths(_, hadoopConf) .split(",") - .filterNot(_.contains(userJar.split("/").last)) + .filterNot(_.contains(userJarName)) .mkString(",") } .filterNot(_ == "") diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 49a319abb3238..5723b0f69057b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -96,11 +96,13 @@ private[deploy] object DeployMessages { * @param masterWebUiUrl the master Web UI address * @param masterAddress the master address used by the worker to connect. It should be * [[RegisterWorker.masterAddress]]. + * @param duplicate whether it is a duplicate register request from the worker */ case class RegisteredWorker( master: RpcEndpointRef, masterWebUiUrl: String, - masterAddress: RpcAddress) extends DeployMessage with RegisterWorkerResponse + masterAddress: RpcAddress, + duplicate: Boolean) extends DeployMessage with RegisterWorkerResponse case class RegisterWorkerFailed(message: String) extends DeployMessage with RegisterWorkerResponse diff --git a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala index f6b3c37f0fe72..28279fc5f823e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ExternalShuffleService.scala @@ -17,13 +17,14 @@ package org.apache.spark.deploy +import java.io.File import java.util.concurrent.CountDownLatch import scala.collection.JavaConverters._ import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.{config, Logging} -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.network.TransportContext import org.apache.spark.network.crypto.AuthServerBootstrap import org.apache.spark.network.netty.SparkTransportConf @@ -43,24 +44,46 @@ private[deploy] class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityManager) extends Logging { protected val masterMetricsSystem = - MetricsSystem.createMetricsSystem("shuffleService", sparkConf, securityManager) + MetricsSystem.createMetricsSystem(MetricsSystemInstances.SHUFFLE_SERVICE, + sparkConf, securityManager) private val enabled = sparkConf.get(config.SHUFFLE_SERVICE_ENABLED) private val port = sparkConf.get(config.SHUFFLE_SERVICE_PORT) + private val registeredExecutorsDB = "registeredExecutors.ldb" + private val transportConf = SparkTransportConf.fromSparkConf(sparkConf, "shuffle", numUsableCores = 0) private val blockHandler = newShuffleBlockHandler(transportConf) - private val transportContext: TransportContext = - new TransportContext(transportConf, blockHandler, true) + private var transportContext: TransportContext = _ private var server: TransportServer = _ private val shuffleServiceSource = new ExternalShuffleServiceSource + protected def findRegisteredExecutorsDBFile(dbName: String): File = { + val localDirs = sparkConf.getOption("spark.local.dir").map(_.split(",")).getOrElse(Array()) + if (localDirs.length >= 1) { + new File(localDirs.find(new File(_, dbName).exists()).getOrElse(localDirs(0)), dbName) + } else { + logWarning(s"'spark.local.dir' should be set first when we use db in " + + s"ExternalShuffleService. Note that this only affects standalone mode.") + null + } + } + + /** Get blockhandler */ + def getBlockHandler: ExternalShuffleBlockHandler = { + blockHandler + } + /** Create a new shuffle block handler. Factored out for subclasses to override. */ protected def newShuffleBlockHandler(conf: TransportConf): ExternalShuffleBlockHandler = { - new ExternalShuffleBlockHandler(conf, null) + if (sparkConf.get(config.SHUFFLE_SERVICE_DB_ENABLED) && enabled) { + new ExternalShuffleBlockHandler(conf, findRegisteredExecutorsDBFile(registeredExecutorsDB)) + } else { + new ExternalShuffleBlockHandler(conf, null) + } } /** Starts the external shuffle service if the user has configured us to. */ @@ -81,9 +104,12 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana } else { Nil } + transportContext = new TransportContext(transportConf, blockHandler, true) server = transportContext.createServer(port, bootstraps.asJava) shuffleServiceSource.registerMetricSet(server.getAllMetrics) + blockHandler.getAllMetrics.getMetrics.put("numRegisteredConnections", + server.getRegisteredConnections) shuffleServiceSource.registerMetricSet(blockHandler.getAllMetrics) masterMetricsSystem.registerSource(shuffleServiceSource) masterMetricsSystem.start() @@ -104,6 +130,10 @@ class ExternalShuffleService(sparkConf: SparkConf, securityManager: SecurityMana server.close() server = null } + if (transportContext != null) { + transportContext.close() + transportContext = null + } } } diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index c6307da61c7eb..99f841234005e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -26,7 +26,6 @@ import scala.collection.mutable.ListBuffer import scala.concurrent.{Future, Promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ -import scala.language.postfixOps import scala.sys.process._ import org.json4s._ @@ -34,7 +33,7 @@ import org.json4s.jackson.JsonMethods import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.master.RecoveryState -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -60,7 +59,7 @@ import org.apache.spark.util.{ThreadUtils, Utils} private object FaultToleranceTest extends App with Logging { private val conf = new SparkConf() - private val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + private val zkDir = conf.get(config.Deploy.ZOOKEEPER_DIRECTORY).getOrElse("/spark") private val masters = ListBuffer[TestMasterInfo]() private val workers = ListBuffer[TestWorkerInfo]() @@ -77,7 +76,7 @@ private object FaultToleranceTest extends App with Logging { private val containerSparkHome = "/opt/spark" private val dockerMountDir = "%s:%s".format(sparkHome, containerSparkHome) - System.setProperty("spark.driver.host", "172.17.42.1") // default docker host ip + System.setProperty(config.DRIVER_HOST_ADDRESS.key, "172.17.42.1") // default docker host ip private def afterEach() { if (sc != null) { @@ -87,8 +86,8 @@ private object FaultToleranceTest extends App with Logging { terminateCluster() // Clear ZK directories in between tests (for speed purposes) - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/spark_leader") - SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/master_status") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/spark_leader") + SparkCuratorUtil.deleteRecursive(zk, zkDir + "/master_status") } test("sanity-basic") { @@ -112,7 +111,7 @@ private object FaultToleranceTest extends App with Logging { assertValidClusterState() killLeader() - delay(30 seconds) + delay(30.seconds) assertValidClusterState() createClient() assertValidClusterState() @@ -126,12 +125,12 @@ private object FaultToleranceTest extends App with Logging { killLeader() addMasters(1) - delay(30 seconds) + delay(30.seconds) assertValidClusterState() killLeader() addMasters(1) - delay(30 seconds) + delay(30.seconds) assertValidClusterState() } @@ -156,7 +155,7 @@ private object FaultToleranceTest extends App with Logging { killLeader() workers.foreach(_.kill()) workers.clear() - delay(30 seconds) + delay(30.seconds) addWorkers(2) assertValidClusterState() } @@ -174,7 +173,7 @@ private object FaultToleranceTest extends App with Logging { (1 to 3).foreach { _ => killLeader() - delay(30 seconds) + delay(30.seconds) assertValidClusterState() assertTrue(getLeader == masters.head) addMasters(1) @@ -216,7 +215,7 @@ private object FaultToleranceTest extends App with Logging { if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. - System.setProperty("spark.driver.port", "0") + System.setProperty(config.DRIVER_PORT.key, "0") sc = new SparkContext(getMasterUrls(masters), "fault-tolerance", containerSparkHome) } @@ -264,7 +263,7 @@ private object FaultToleranceTest extends App with Logging { } // Avoid waiting indefinitely (e.g., we could register but get no executors). - assertTrue(ThreadUtils.awaitResult(f, 120 seconds)) + assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) } /** @@ -317,7 +316,7 @@ private object FaultToleranceTest extends App with Logging { } try { - assertTrue(ThreadUtils.awaitResult(f, 120 seconds)) + assertTrue(ThreadUtils.awaitResult(f, 2.minutes)) } catch { case e: TimeoutException => logError("Master states: " + masters.map(_.state)) @@ -421,7 +420,7 @@ private object SparkDocker { } dockerCmd.run(ProcessLogger(findIpAndLog _)) - val ip = ThreadUtils.awaitResult(ipPromise.future, 30 seconds) + val ip = ThreadUtils.awaitResult(ipPromise.future, 30.seconds) val dockerId = Docker.getLastProcessId (ip, dockerId, outFile) } diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index be293f88a9d4a..c1866b4c3606e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -51,8 +51,8 @@ class LocalSparkCluster( // Disable REST server on Master in this mode unless otherwise specified val _conf = conf.clone() - .setIfMissing("spark.master.rest.enabled", "false") - .set(config.SHUFFLE_SERVICE_ENABLED.key, "false") + .setIfMissing(config.MASTER_REST_SERVER_ENABLED, false) + .set(config.SHUFFLE_SERVICE_ENABLED, false) /* Start the Master */ val (rpcEnv, webUiPort, _) = Master.startRpcEnvAndEndpoint(localHostname, 0, 0, _conf) diff --git a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala index ccb30e205ca40..f5e8cfff2ad1d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/PythonRunner.scala @@ -60,11 +60,7 @@ object PythonRunner { .javaAddress(localhost) .callbackClient(py4j.GatewayServer.DEFAULT_PYTHON_PORT, localhost, secret) .build() - val thread = new Thread(new Runnable() { - override def run(): Unit = Utils.logUncaughtExceptions { - gatewayServer.start() - } - }) + val thread = new Thread(() => Utils.logUncaughtExceptions { gatewayServer.start() }) thread.setName("py4j-gateway-init") thread.setDaemon(true) thread.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala index e86b362639e57..60ba0470a628a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/RRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/RRunner.scala @@ -25,7 +25,9 @@ import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path import org.apache.spark.{SparkException, SparkUserAppException} -import org.apache.spark.api.r.{RBackend, RUtils, SparkRDefaults} +import org.apache.spark.api.r.{RBackend, RUtils} +import org.apache.spark.internal.config.R._ +import org.apache.spark.internal.config.SUBMIT_DEPLOY_MODE import org.apache.spark.util.RedirectThread /** @@ -43,9 +45,9 @@ object RRunner { val rCommand = { // "spark.sparkr.r.command" is deprecated and replaced by "spark.r.command", // but kept here for backward compatibility. - var cmd = sys.props.getOrElse("spark.sparkr.r.command", "Rscript") - cmd = sys.props.getOrElse("spark.r.command", cmd) - if (sys.props.getOrElse("spark.submit.deployMode", "client") == "client") { + var cmd = sys.props.getOrElse(SPARKR_COMMAND.key, SPARKR_COMMAND.defaultValue.get) + cmd = sys.props.getOrElse(R_COMMAND.key, cmd) + if (sys.props.getOrElse(SUBMIT_DEPLOY_MODE.key, "client") == "client") { cmd = sys.props.getOrElse("spark.r.driver.command", cmd) } cmd @@ -53,7 +55,7 @@ object RRunner { // Connection timeout set by R process on its connection to RBackend in seconds. val backendConnectionTimeout = sys.props.getOrElse( - "spark.r.backendConnectionTimeout", SparkRDefaults.DEFAULT_CONNECTION_TIMEOUT.toString) + R_BACKEND_CONNECTION_TIMEOUT.key, R_BACKEND_CONNECTION_TIMEOUT.defaultValue.get.toString) // Check if the file path exists. // If not, change directory to current working directory for YARN cluster mode diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala index 8247110940dbc..8118c01eb712f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkCuratorUtil.scala @@ -25,6 +25,7 @@ import org.apache.zookeeper.KeeperException import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL private[spark] object SparkCuratorUtil extends Logging { @@ -35,7 +36,7 @@ private[spark] object SparkCuratorUtil extends Logging { def newClient( conf: SparkConf, - zkUrlConf: String = "spark.deploy.zookeeper.url"): CuratorFramework = { + zkUrlConf: String = ZOOKEEPER_URL.key): CuratorFramework = { val ZK_URL = conf.get(zkUrlConf) val zk = CuratorFrameworkFactory.newClient(ZK_URL, ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 4cc0063d010ef..11420bb985520 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream, File, IOException} import java.security.PrivilegedExceptionAction import java.text.DateFormat -import java.util.{Arrays, Comparator, Date, Locale} +import java.util.{Arrays, Date, Locale} import scala.collection.JavaConverters._ import scala.collection.immutable.Map @@ -30,24 +30,21 @@ import scala.util.control.NonFatal import com.google.common.primitives.Longs import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileStatus, FileSystem, Path, PathFilter} +import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.BUFFER_SIZE import org.apache.spark.util.Utils /** - * :: DeveloperApi :: * Contains util methods to interact with Hadoop from Spark. */ -@DeveloperApi -class SparkHadoopUtil extends Logging { +private[spark] class SparkHadoopUtil extends Logging { private val sparkConf = new SparkConf(false).loadFromSystemProperties(true) val conf: Configuration = newConfiguration(sparkConf) UserGroupInformation.setConfiguration(conf) @@ -273,11 +270,8 @@ class SparkHadoopUtil extends Logging { name.startsWith(prefix) && !name.endsWith(exclusionSuffix) } }) - Arrays.sort(fileStatuses, new Comparator[FileStatus] { - override def compare(o1: FileStatus, o2: FileStatus): Int = { - Longs.compare(o1.getModificationTime, o2.getModificationTime) - } - }) + Arrays.sort(fileStatuses, (o1: FileStatus, o2: FileStatus) => + Longs.compare(o1.getModificationTime, o2.getModificationTime)) fileStatuses } catch { case NonFatal(e) => @@ -387,7 +381,7 @@ class SparkHadoopUtil extends Logging { } -object SparkHadoopUtil { +private[spark] object SparkHadoopUtil { private lazy val instance = new SparkHadoopUtil @@ -412,20 +406,6 @@ object SparkHadoopUtil { def get: SparkHadoopUtil = instance - /** - * Given an expiration date for the current set of credentials, calculate the time when new - * credentials should be created. - * - * @param expirationDate Drop-dead expiration date - * @param conf Spark configuration - * @return Timestamp when new credentials should be created. - */ - private[spark] def nextCredentialRenewalTime(expirationDate: Long, conf: SparkConf): Long = { - val ct = System.currentTimeMillis - val ratio = conf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) - (ct + (ratio * (expirationDate - ct))).toLong - } - /** * Returns a Configuration object with Spark configuration applied on top. Unlike * the instance method, this will always return a Configuration instance, and not a @@ -460,7 +440,7 @@ object SparkHadoopUtil { } } appendSparkHadoopConfigs(conf, hadoopConf) - val bufferSize = conf.get("spark.buffer.size", "65536") + val bufferSize = conf.get(BUFFER_SIZE).toString hadoopConf.set("io.file.buffer.size", bufferSize) } } @@ -471,4 +451,39 @@ object SparkHadoopUtil { hadoopConf.set(key.substring("spark.hadoop.".length), value) } } + + // scalastyle:off line.size.limit + /** + * Create a path that uses replication instead of erasure coding (ec), regardless of the default + * configuration in hdfs for the given path. This can be helpful as hdfs ec doesn't support + * hflush(), hsync(), or append() + * https://hadoop.apache.org/docs/r3.0.0/hadoop-project-dist/hadoop-hdfs/HDFSErasureCoding.html#Limitations + */ + // scalastyle:on line.size.limit + def createNonECFile(fs: FileSystem, path: Path): FSDataOutputStream = { + try { + // Use reflection as this uses APIs only available in Hadoop 3 + val builderMethod = fs.getClass().getMethod("createFile", classOf[Path]) + // the builder api does not resolve relative paths, nor does it create parent dirs, while + // the old api does. + if (!fs.mkdirs(path.getParent())) { + throw new IOException(s"Failed to create parents of $path") + } + val qualifiedPath = fs.makeQualified(path) + val builder = builderMethod.invoke(fs, qualifiedPath) + val builderCls = builder.getClass() + // this may throw a NoSuchMethodException if the path is not on hdfs + val replicateMethod = builderCls.getMethod("replicate") + val buildMethod = builderCls.getMethod("build") + val b2 = replicateMethod.invoke(builder) + buildMethod.invoke(b2).asInstanceOf[FSDataOutputStream] + } catch { + case _: NoSuchMethodException => + // No createFile() method, we're using an older hdfs client, which doesn't give us control + // over EC vs. replication. Older hdfs doesn't have EC anyway, so just create a file with + // old apis. + fs.create(path) + } + } + } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 61b379f286802..12a8473b22025 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -18,16 +18,18 @@ package org.apache.spark.deploy import java.io._ -import java.lang.reflect.{InvocationTargetException, Modifier, UndeclaredThrowableException} -import java.net.URL +import java.lang.reflect.{InvocationTargetException, UndeclaredThrowableException} +import java.net.{URI, URL} import java.security.PrivilegedExceptionAction import java.text.ParseException -import java.util.UUID +import java.util.{ServiceLoader, UUID} import scala.annotation.tailrec -import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.util.{Properties, Try} +import org.apache.commons.io.FilenameUtils import org.apache.commons.lang3.StringUtils import org.apache.hadoop.conf.{Configuration => HadoopConfiguration} import org.apache.hadoop.fs.{FileSystem, Path} @@ -50,6 +52,7 @@ import org.apache.spark.api.r.RUtils import org.apache.spark.deploy.rest._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.util._ @@ -95,20 +98,35 @@ private[spark] class SparkSubmit extends Logging { } /** - * Kill an existing submission using the REST protocol. Standalone and Mesos cluster mode only. + * Kill an existing submission. */ private def kill(args: SparkSubmitArguments): Unit = { - new RestSubmissionClient(args.master) - .killSubmission(args.submissionToKill) + if (RestSubmissionClient.supportsRestClient(args.master)) { + new RestSubmissionClient(args.master) + .killSubmission(args.submissionToKill) + } else { + val sparkConf = args.toSparkConf() + sparkConf.set("spark.master", args.master) + SparkSubmitUtils + .getSubmitOperations(args.master) + .kill(args.submissionToKill, sparkConf) + } } /** - * Request the status of an existing submission using the REST protocol. - * Standalone and Mesos cluster mode only. + * Request the status of an existing submission. */ private def requestStatus(args: SparkSubmitArguments): Unit = { - new RestSubmissionClient(args.master) - .requestSubmissionStatus(args.submissionToRequestStatusFor) + if (RestSubmissionClient.supportsRestClient(args.master)) { + new RestSubmissionClient(args.master) + .requestSubmissionStatus(args.submissionToRequestStatusFor) + } else { + val sparkConf = args.toSparkConf() + sparkConf.set("spark.master", args.master) + SparkSubmitUtils + .getSubmitOperations(args.master) + .printSubmissionStatus(args.submissionToRequestStatusFor, sparkConf) + } } /** Print version information to the log. */ @@ -130,17 +148,11 @@ private[spark] class SparkSubmit extends Logging { } /** - * Submit the application using the provided parameters. - * - * This runs in two steps. First, we prepare the launch environment by setting up - * the appropriate classpath, system properties, and application arguments for - * running the child main class based on the cluster manager and the deploy mode. - * Second, we use this launch environment to invoke the main method of the child - * main class. + * Submit the application using the provided parameters, ensuring to first wrap + * in a doAs when --proxy-user is specified. */ @tailrec private def submit(args: SparkSubmitArguments, uninitLog: Boolean): Unit = { - val (childArgs, childClasspath, sparkConf, childMainClass) = prepareSubmitEnvironment(args) def doRunMain(): Unit = { if (args.proxyUser != null) { @@ -149,7 +161,7 @@ private[spark] class SparkSubmit extends Logging { try { proxyUser.doAs(new PrivilegedExceptionAction[Unit]() { override def run(): Unit = { - runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) + runMain(args, uninitLog) } }) } catch { @@ -164,15 +176,10 @@ private[spark] class SparkSubmit extends Logging { } } } else { - runMain(childArgs, childClasspath, sparkConf, childMainClass, args.verbose) + runMain(args, uninitLog) } } - // Let the main class re-initialize the logging system once it starts. - if (uninitLog) { - Logging.uninitialize() - } - // In standalone cluster mode, there are two submission gateways: // (1) The traditional RPC gateway using o.a.s.deploy.Client as a wrapper // (2) The new REST-based gateway introduced in Spark 1.3 @@ -216,7 +223,7 @@ private[spark] class SparkSubmit extends Logging { // Return values val childArgs = new ArrayBuffer[String]() val childClasspath = new ArrayBuffer[String]() - val sparkConf = new SparkConf() + val sparkConf = args.toSparkConf() var childMainClass = "" // Set the cluster manager @@ -307,7 +314,9 @@ private[spark] class SparkSubmit extends Logging { val isMesosCluster = clusterManager == MESOS && deployMode == CLUSTER val isStandAloneCluster = clusterManager == STANDALONE && deployMode == CLUSTER val isKubernetesCluster = clusterManager == KUBERNETES && deployMode == CLUSTER - val isMesosClient = clusterManager == MESOS && deployMode == CLIENT + val isKubernetesClient = clusterManager == KUBERNETES && deployMode == CLIENT + val isKubernetesClusterModeDriver = isKubernetesClient && + sparkConf.getBoolean("spark.kubernetes.submitInDriver", false) if (!isMesosCluster && !isStandAloneCluster) { // Resolve maven dependencies if there are any and add classpath to jars. Add them to py-files @@ -317,9 +326,25 @@ private[spark] class SparkSubmit extends Logging { args.ivySettingsPath) if (!StringUtils.isBlank(resolvedMavenCoordinates)) { - args.jars = mergeFileLists(args.jars, resolvedMavenCoordinates) - if (args.isPython) { - args.pyFiles = mergeFileLists(args.pyFiles, resolvedMavenCoordinates) + // In K8s client mode, when in the driver, add resolved jars early as we might need + // them at the submit time for artifact downloading. + // For example we might use the dependencies for downloading + // files from a Hadoop Compatible fs eg. S3. In this case the user might pass: + // --packages com.amazonaws:aws-java-sdk:1.7.4:org.apache.hadoop:hadoop-aws:2.7.6 + if (isKubernetesClusterModeDriver) { + val loader = getSubmitClassLoader(sparkConf) + for (jar <- resolvedMavenCoordinates.split(",")) { + addJarToClasspath(jar, loader) + } + } else if (isKubernetesCluster) { + // We need this in K8s cluster mode so that we can upload local deps + // via the k8s application, like in cluster mode driver + childClasspath ++= resolvedMavenCoordinates.split(",") + } else { + args.jars = mergeFileLists(args.jars, resolvedMavenCoordinates) + if (args.isPython || isInternal(args.primaryResource)) { + args.pyFiles = mergeFileLists(args.pyFiles, resolvedMavenCoordinates) + } } } @@ -330,23 +355,25 @@ private[spark] class SparkSubmit extends Logging { } } - args.sparkProperties.foreach { case (k, v) => sparkConf.set(k, v) } + // update spark config from args + args.toSparkConf(Option(sparkConf)) val hadoopConf = conf.getOrElse(SparkHadoopUtil.newConfiguration(sparkConf)) val targetDir = Utils.createTempDir() - // assure a keytab is available from any place in a JVM - if (clusterManager == YARN || clusterManager == LOCAL || isMesosClient) { - if (args.principal != null) { - if (args.keytab != null) { - require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") - // Add keytab and principal configurations in sysProps to make them available - // for later use; e.g. in spark sql, the isolated class loader used to talk - // to HiveMetastore will use these settings. They will be set as Java system - // properties and then loaded by SparkConf - sparkConf.set(KEYTAB, args.keytab) - sparkConf.set(PRINCIPAL, args.principal) - UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) - } + // Kerberos is not supported in standalone mode, and keytab support is not yet available + // in Mesos cluster mode. + if (clusterManager != STANDALONE + && !isMesosCluster + && args.principal != null + && args.keytab != null) { + // If client mode, make sure the keytab is just a local path. + if (deployMode == CLIENT && Utils.isLocalUri(args.keytab)) { + args.keytab = new URI(args.keytab).getPath() + } + + if (!Utils.isLocalUri(args.keytab)) { + require(new File(args.keytab).exists(), s"Keytab file: ${args.keytab} does not exist") + UserGroupInformation.loginUserFromKeytab(args.principal, args.keytab) } } @@ -372,6 +399,17 @@ private[spark] class SparkSubmit extends Logging { localPyFiles = Option(args.pyFiles).map { downloadFileList(_, targetDir, sparkConf, hadoopConf, secMgr) }.orNull + + if (isKubernetesClusterModeDriver) { + // Replace with the downloaded local jar path to avoid propagating hadoop compatible uris. + // Executors will get the jars from the Spark file server. + // Explicitly download the related files here + args.jars = renameResourcesToLocalFS(args.jars, localJars) + val localFiles = Option(args.files).map { + downloadFileList(_, targetDir, sparkConf, hadoopConf, secMgr) + }.orNull + args.files = renameResourcesToLocalFS(args.files, localFiles) + } } // When running in YARN, for some remote resources with scheme: @@ -435,7 +473,7 @@ private[spark] class SparkSubmit extends Logging { } if (localPyFiles != null) { - sparkConf.set("spark.submit.pyFiles", localPyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, localPyFiles.split(",").toSeq) } // In YARN mode for an R app, add the SparkR package archive and the R package @@ -509,57 +547,64 @@ private[spark] class SparkSubmit extends Logging { // All cluster managers OptionAssigner(args.master, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = "spark.master"), OptionAssigner(args.deployMode, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.submit.deployMode"), + confKey = SUBMIT_DEPLOY_MODE.key), OptionAssigner(args.name, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = "spark.app.name"), OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, CLIENT, confKey = "spark.jars.ivy"), OptionAssigner(args.driverMemory, ALL_CLUSTER_MGRS, CLIENT, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverExtraClassPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraClassPath"), + confKey = DRIVER_CLASS_PATH.key), OptionAssigner(args.driverExtraJavaOptions, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraJavaOptions"), + confKey = DRIVER_JAVA_OPTIONS.key), OptionAssigner(args.driverExtraLibraryPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, - confKey = "spark.driver.extraLibraryPath"), + confKey = DRIVER_LIBRARY_PATH.key), OptionAssigner(args.principal, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = PRINCIPAL.key), OptionAssigner(args.keytab, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES, confKey = KEYTAB.key), + OptionAssigner(args.pyFiles, ALL_CLUSTER_MGRS, CLUSTER, confKey = SUBMIT_PYTHON_FILES.key), // Propagate attributes for dependency resolution at the driver side - OptionAssigner(args.packages, STANDALONE | MESOS, CLUSTER, confKey = "spark.jars.packages"), - OptionAssigner(args.repositories, STANDALONE | MESOS, CLUSTER, - confKey = "spark.jars.repositories"), - OptionAssigner(args.ivyRepoPath, STANDALONE | MESOS, CLUSTER, confKey = "spark.jars.ivy"), - OptionAssigner(args.packagesExclusions, STANDALONE | MESOS, + OptionAssigner(args.packages, STANDALONE | MESOS | KUBERNETES, + CLUSTER, confKey = "spark.jars.packages"), + OptionAssigner(args.repositories, STANDALONE | MESOS | KUBERNETES, + CLUSTER, confKey = "spark.jars.repositories"), + OptionAssigner(args.ivyRepoPath, STANDALONE | MESOS | KUBERNETES, + CLUSTER, confKey = "spark.jars.ivy"), + OptionAssigner(args.packagesExclusions, STANDALONE | MESOS | KUBERNETES, CLUSTER, confKey = "spark.jars.excludes"), // Yarn only OptionAssigner(args.queue, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.queue"), - OptionAssigner(args.numExecutors, YARN, ALL_DEPLOY_MODES, - confKey = "spark.executor.instances"), - OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.pyFiles"), - OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.jars"), - OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.files"), - OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.archives"), + OptionAssigner(args.pyFiles, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.pyFiles", + mergeFn = Some(mergeFileLists(_, _))), + OptionAssigner(args.jars, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.jars", + mergeFn = Some(mergeFileLists(_, _))), + OptionAssigner(args.files, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.files", + mergeFn = Some(mergeFileLists(_, _))), + OptionAssigner(args.archives, YARN, ALL_DEPLOY_MODES, confKey = "spark.yarn.dist.archives", + mergeFn = Some(mergeFileLists(_, _))), // Other options + OptionAssigner(args.numExecutors, YARN | KUBERNETES, ALL_DEPLOY_MODES, + confKey = EXECUTOR_INSTANCES.key), OptionAssigner(args.executorCores, STANDALONE | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.cores"), + confKey = EXECUTOR_CORES.key), OptionAssigner(args.executorMemory, STANDALONE | MESOS | YARN | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.executor.memory"), + confKey = EXECUTOR_MEMORY.key), OptionAssigner(args.totalExecutorCores, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.cores.max"), + confKey = CORES_MAX.key), OptionAssigner(args.files, LOCAL | STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.files"), - OptionAssigner(args.jars, LOCAL, CLIENT, confKey = "spark.jars"), + confKey = FILES.key), + OptionAssigner(args.jars, LOCAL, CLIENT, confKey = JARS.key), OptionAssigner(args.jars, STANDALONE | MESOS | KUBERNETES, ALL_DEPLOY_MODES, - confKey = "spark.jars"), + confKey = JARS.key), OptionAssigner(args.driverMemory, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.memory"), + confKey = DRIVER_MEMORY.key), OptionAssigner(args.driverCores, STANDALONE | MESOS | YARN | KUBERNETES, CLUSTER, - confKey = "spark.driver.cores"), + confKey = DRIVER_CORES.key), OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER, - confKey = "spark.driver.supervise"), + confKey = DRIVER_SUPERVISE.key), OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, confKey = "spark.jars.ivy"), // An internal option used only for spark-shell to add user jars to repl's classloader, @@ -599,7 +644,13 @@ private[spark] class SparkSubmit extends Logging { (deployMode & opt.deployMode) != 0 && (clusterManager & opt.clusterManager) != 0) { if (opt.clOption != null) { childArgs += (opt.clOption, opt.value) } - if (opt.confKey != null) { sparkConf.set(opt.confKey, opt.value) } + if (opt.confKey != null) { + if (opt.mergeFn.isDefined && sparkConf.contains(opt.confKey)) { + sparkConf.set(opt.confKey, opt.mergeFn.get.apply(sparkConf.get(opt.confKey), opt.value)) + } else { + sparkConf.set(opt.confKey, opt.value) + } + } } } @@ -612,11 +663,11 @@ private[spark] class SparkSubmit extends Logging { // For YARN cluster mode, the jar is already distributed on each node as "app.jar" // For python and R files, the primary resource is already distributed as a regular file if (!isYarnCluster && !args.isPython && !args.isR) { - var jars = sparkConf.getOption("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq.empty) + var jars = sparkConf.get(JARS) if (isUserJar(args.primaryResource)) { jars = jars ++ Seq(args.primaryResource) } - sparkConf.set("spark.jars", jars.mkString(",")) + sparkConf.set(JARS, jars) } // In standalone cluster mode, use the REST client to submit the application (Spark 1.3+). @@ -646,7 +697,8 @@ private[spark] class SparkSubmit extends Logging { } } - if (clusterManager == MESOS && UserGroupInformation.isSecurityEnabled) { + if ((clusterManager == MESOS || clusterManager == KUBERNETES) + && UserGroupInformation.isSecurityEnabled) { setRMPrincipal(sparkConf) } @@ -678,7 +730,7 @@ private[spark] class SparkSubmit extends Logging { // Second argument is main class childArgs += (args.primaryResource, "") if (args.pyFiles != null) { - sparkConf.set("spark.submit.pyFiles", args.pyFiles) + sparkConf.set(SUBMIT_PYTHON_FILES, args.pyFiles.split(",").toSeq) } } else if (args.isR) { // Second argument is main class @@ -697,9 +749,6 @@ private[spark] class SparkSubmit extends Logging { if (args.isPython) { childArgs ++= Array("--primary-py-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.PythonRunner") - if (args.pyFiles != null) { - childArgs ++= Array("--other-py-files", args.pyFiles) - } } else if (args.isR) { childArgs ++= Array("--primary-r-file", args.primaryResource) childArgs ++= Array("--main-class", "org.apache.spark.deploy.RRunner") @@ -725,13 +774,13 @@ private[spark] class SparkSubmit extends Logging { // Ignore invalid spark.driver.host in cluster modes. if (deployMode == CLUSTER) { - sparkConf.remove("spark.driver.host") + sparkConf.remove(DRIVER_HOST_ADDRESS) } // Resolve paths in certain spark properties val pathConfigs = Seq( - "spark.jars", - "spark.files", + JARS.key, + FILES.key, "spark.yarn.dist.files", "spark.yarn.dist.archives", "spark.yarn.dist.jars") @@ -745,25 +794,39 @@ private[spark] class SparkSubmit extends Logging { // Resolve and format python file paths properly before adding them to the PYTHONPATH. // The resolving part is redundant in the case of --py-files, but necessary if the user // explicitly sets `spark.submit.pyFiles` in his/her default properties file. - sparkConf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val resolvedPyFiles = Utils.resolveURIs(pyFiles) - val formattedPyFiles = if (!isYarnCluster && !isMesosCluster) { - PythonRunner.formatPaths(resolvedPyFiles).mkString(",") - } else { - // Ignoring formatting python path in yarn and mesos cluster mode, these two modes - // support dealing with remote python files, they could distribute and add python files - // locally. - resolvedPyFiles - } - sparkConf.set("spark.submit.pyFiles", formattedPyFiles) + val pyFiles = sparkConf.get(SUBMIT_PYTHON_FILES) + val resolvedPyFiles = Utils.resolveURIs(pyFiles.mkString(",")) + val formattedPyFiles = if (deployMode != CLUSTER) { + PythonRunner.formatPaths(resolvedPyFiles).mkString(",") + } else { + // Ignoring formatting python path in yarn and mesos cluster mode, these two modes + // support dealing with remote python files, they could distribute and add python files + // locally. + resolvedPyFiles } + sparkConf.set(SUBMIT_PYTHON_FILES, formattedPyFiles.split(",").toSeq) (childArgs, childClasspath, sparkConf, childMainClass) } + private def renameResourcesToLocalFS(resources: String, localResources: String): String = { + if (resources != null && localResources != null) { + val localResourcesSeq = Utils.stringToSeq(localResources) + Utils.stringToSeq(resources).map { resource => + val filenameRemote = FilenameUtils.getName(new URI(resource).getPath) + localResourcesSeq.find { localUri => + val filenameLocal = FilenameUtils.getName(new URI(localUri).getPath) + filenameRemote == filenameLocal + }.getOrElse(resource) + }.mkString(",") + } else { + resources + } + } + // [SPARK-20328]. HadoopRDD calls into a Hadoop library that fetches delegation tokens with - // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos mode, we - // must trick it into thinking we're YARN. + // renewer set to the YARN ResourceManager. Since YARN isn't configured in Mesos or Kubernetes + // mode, we must trick it into thinking we're YARN. private def setRMPrincipal(sparkConf: SparkConf): Unit = { val shortUserName = UserGroupInformation.getCurrentUser.getShortUserName val key = s"spark.hadoop.${YarnConfiguration.RM_PRINCIPAL}" @@ -771,19 +834,39 @@ private[spark] class SparkSubmit extends Logging { sparkConf.set(key, shortUserName) } + private def getSubmitClassLoader(sparkConf: SparkConf): MutableURLClassLoader = { + val loader = + if (sparkConf.get(DRIVER_USER_CLASS_PATH_FIRST)) { + new ChildFirstURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + } else { + new MutableURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + } + Thread.currentThread.setContextClassLoader(loader) + loader + } + /** - * Run the main method of the child class using the provided launch environment. + * Run the main method of the child class using the submit arguments. + * + * This runs in two steps. First, we prepare the launch environment by setting up + * the appropriate classpath, system properties, and application arguments for + * running the child main class based on the cluster manager and the deploy mode. + * Second, we use this launch environment to invoke the main method of the child + * main class. * * Note that this main class will not be the one provided by the user if we're * running cluster deploy mode or python applications. */ - private def runMain( - childArgs: Seq[String], - childClasspath: Seq[String], - sparkConf: SparkConf, - childMainClass: String, - verbose: Boolean): Unit = { - if (verbose) { + private def runMain(args: SparkSubmitArguments, uninitLog: Boolean): Unit = { + val (childArgs, childClasspath, sparkConf, childMainClass) = prepareSubmitEnvironment(args) + // Let the main class re-initialize the logging system once it starts. + if (uninitLog) { + Logging.uninitialize() + } + + if (args.verbose) { logInfo(s"Main class:\n$childMainClass") logInfo(s"Arguments:\n${childArgs.mkString("\n")}") // sysProps may contain sensitive information, so redact before printing @@ -791,17 +874,7 @@ private[spark] class SparkSubmit extends Logging { logInfo(s"Classpath elements:\n${childClasspath.mkString("\n")}") logInfo("\n") } - - val loader = - if (sparkConf.get(DRIVER_USER_CLASS_PATH_FIRST)) { - new ChildFirstURLClassLoader(new Array[URL](0), - Thread.currentThread.getContextClassLoader) - } else { - new MutableURLClassLoader(new Array[URL](0), - Thread.currentThread.getContextClassLoader) - } - Thread.currentThread.setContextClassLoader(loader) - + val loader = getSubmitClassLoader(sparkConf) for (jar <- childClasspath) { addJarToClasspath(jar, loader) } @@ -812,14 +885,14 @@ private[spark] class SparkSubmit extends Logging { mainClass = Utils.classForName(childMainClass) } catch { case e: ClassNotFoundException => - logWarning(s"Failed to load $childMainClass.", e) + logError(s"Failed to load class $childMainClass.") if (childMainClass.contains("thriftserver")) { logInfo(s"Failed to load main class $childMainClass.") logInfo("You need to build Spark with -Phive and -Phive-thriftserver.") } throw new SparkUserAppException(CLASS_NOT_FOUND_EXIT_STATUS) case e: NoClassDefFoundError => - logWarning(s"Failed to load $childMainClass: ${e.getMessage()}") + logError(s"Failed to load $childMainClass: ${e.getMessage()}") if (e.getMessage.contains("org/apache/hadoop/hive")) { logInfo(s"Failed to load hive class.") logInfo("You need to build Spark with -Phive and -Phive-thriftserver.") @@ -828,12 +901,8 @@ private[spark] class SparkSubmit extends Logging { } val app: SparkApplication = if (classOf[SparkApplication].isAssignableFrom(mainClass)) { - mainClass.newInstance().asInstanceOf[SparkApplication] + mainClass.getConstructor().newInstance().asInstanceOf[SparkApplication] } else { - // SPARK-4170 - if (classOf[scala.App].isAssignableFrom(mainClass)) { - logWarning("Subclasses of scala.App may not work correctly. Use a main() method instead.") - } new JavaMainApplication(mainClass) } @@ -914,6 +983,8 @@ object SparkSubmit extends CommandLineUtils with Logging { override protected def logInfo(msg: => String): Unit = self.logInfo(msg) override protected def logWarning(msg: => String): Unit = self.logWarning(msg) + + override protected def logError(msg: => String): Unit = self.logError(msg) } } @@ -921,6 +992,8 @@ object SparkSubmit extends CommandLineUtils with Logging { override protected def logWarning(msg: => String): Unit = printMessage(s"Warning: $msg") + override protected def logError(msg: => String): Unit = printMessage(s"Error: $msg") + override def doSubmit(args: Array[String]): Unit = { try { super.doSubmit(args) @@ -991,9 +1064,9 @@ private[spark] object SparkSubmitUtils { // Exposed for testing. // These components are used to make the default exclusion rules for Spark dependencies. - // We need to specify each component explicitly, otherwise we miss spark-streaming-kafka-0-8 and - // other spark-streaming utility components. Underscore is there to differentiate between - // spark-streaming_2.1x and spark-streaming-kafka-0-8-assembly_2.1x + // We need to specify each component explicitly, otherwise we miss + // spark-streaming utility components. Underscore is there to differentiate between + // spark-streaming_2.1x and spark-streaming-kafka-0-10-assembly_2.1x val IVY_DEFAULT_EXCLUDES = Seq("catalyst_", "core_", "graphx_", "kvstore_", "launcher_", "mllib_", "mllib-local_", "network-common_", "network-shuffle_", "repl_", "sketch_", "sql_", "streaming_", "tags_", "unsafe_") @@ -1079,7 +1152,7 @@ private[spark] object SparkSubmitUtils { val sp: IBiblioResolver = new IBiblioResolver sp.setM2compatible(true) sp.setUsepoms(true) - sp.setRoot("http://dl.bintray.com/spark-packages/maven") + sp.setRoot("https://dl.bintray.com/spark-packages/maven") sp.setName("spark-packages") cr.add(sp) cr @@ -1340,6 +1413,23 @@ private[spark] object SparkSubmitUtils { } } + private[deploy] def getSubmitOperations(master: String): SparkSubmitOperation = { + val loader = Utils.getContextOrSparkClassLoader + val serviceLoaders = + ServiceLoader.load(classOf[SparkSubmitOperation], loader) + .asScala + .filter(_.supports(master)) + + serviceLoaders.size match { + case x if x > 1 => + throw new SparkException(s"Multiple($x) external SparkSubmitOperations " + + s"clients registered for master url ${master}.") + case 1 => serviceLoaders.headOption.get + case _ => + throw new IllegalArgumentException(s"No external SparkSubmitOperations " + + s"clients found for master url: '$master'") + } + } } /** @@ -1351,4 +1441,14 @@ private case class OptionAssigner( clusterManager: Int, deployMode: Int, clOption: String = null, - confKey: String = null) + confKey: String = null, + mergeFn: Option[(String, String) => String] = None) + +private[spark] trait SparkSubmitOperation { + + def kill(submissionId: String, conf: SparkConf): Unit + + def printSubmissionStatus(submissionId: String, conf: SparkConf): Unit + + def supports(master: String): Boolean +} diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 4cf08a7980f55..ed1324baed0f1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -29,9 +29,10 @@ import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.io.Source import scala.util.Try -import org.apache.spark.{SparkException, SparkUserAppException} +import org.apache.spark.{SparkConf, SparkException, SparkUserAppException} import org.apache.spark.deploy.SparkSubmitAction._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.DYN_ALLOCATION_ENABLED import org.apache.spark.launcher.SparkSubmitArgumentsParser import org.apache.spark.network.util.JavaUtils import org.apache.spark.util.Utils @@ -155,36 +156,36 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(env.get("MASTER")) .orNull driverExtraClassPath = Option(driverExtraClassPath) - .orElse(sparkProperties.get("spark.driver.extraClassPath")) + .orElse(sparkProperties.get(config.DRIVER_CLASS_PATH.key)) .orNull driverExtraJavaOptions = Option(driverExtraJavaOptions) - .orElse(sparkProperties.get("spark.driver.extraJavaOptions")) + .orElse(sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key)) .orNull driverExtraLibraryPath = Option(driverExtraLibraryPath) - .orElse(sparkProperties.get("spark.driver.extraLibraryPath")) + .orElse(sparkProperties.get(config.DRIVER_LIBRARY_PATH.key)) .orNull driverMemory = Option(driverMemory) - .orElse(sparkProperties.get("spark.driver.memory")) + .orElse(sparkProperties.get(config.DRIVER_MEMORY.key)) .orElse(env.get("SPARK_DRIVER_MEMORY")) .orNull driverCores = Option(driverCores) - .orElse(sparkProperties.get("spark.driver.cores")) + .orElse(sparkProperties.get(config.DRIVER_CORES.key)) .orNull executorMemory = Option(executorMemory) - .orElse(sparkProperties.get("spark.executor.memory")) + .orElse(sparkProperties.get(config.EXECUTOR_MEMORY.key)) .orElse(env.get("SPARK_EXECUTOR_MEMORY")) .orNull executorCores = Option(executorCores) - .orElse(sparkProperties.get("spark.executor.cores")) + .orElse(sparkProperties.get(config.EXECUTOR_CORES.key)) .orElse(env.get("SPARK_EXECUTOR_CORES")) .orNull totalExecutorCores = Option(totalExecutorCores) - .orElse(sparkProperties.get("spark.cores.max")) + .orElse(sparkProperties.get(config.CORES_MAX.key)) .orNull name = Option(name).orElse(sparkProperties.get("spark.app.name")).orNull - jars = Option(jars).orElse(sparkProperties.get("spark.jars")).orNull - files = Option(files).orElse(sparkProperties.get("spark.files")).orNull - pyFiles = Option(pyFiles).orElse(sparkProperties.get("spark.submit.pyFiles")).orNull + jars = Option(jars).orElse(sparkProperties.get(config.JARS.key)).orNull + files = Option(files).orElse(sparkProperties.get(config.FILES.key)).orNull + pyFiles = Option(pyFiles).orElse(sparkProperties.get(config.SUBMIT_PYTHON_FILES.key)).orNull ivyRepoPath = sparkProperties.get("spark.jars.ivy").orNull ivySettingsPath = sparkProperties.get("spark.jars.ivySettings") packages = Option(packages).orElse(sparkProperties.get("spark.jars.packages")).orNull @@ -193,11 +194,11 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S repositories = Option(repositories) .orElse(sparkProperties.get("spark.jars.repositories")).orNull deployMode = Option(deployMode) - .orElse(sparkProperties.get("spark.submit.deployMode")) + .orElse(sparkProperties.get(config.SUBMIT_DEPLOY_MODE.key)) .orElse(env.get("DEPLOY_MODE")) .orNull numExecutors = Option(numExecutors) - .getOrElse(sparkProperties.get("spark.executor.instances").orNull) + .getOrElse(sparkProperties.get(config.EXECUTOR_INSTANCES.key).orNull) queue = Option(queue).orElse(sparkProperties.get("spark.yarn.queue")).orNull keytab = Option(keytab) .orElse(sparkProperties.get("spark.kerberos.keytab")) @@ -208,7 +209,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S .orElse(sparkProperties.get("spark.yarn.principal")) .orNull dynamicAllocationEnabled = - sparkProperties.get("spark.dynamicAllocation.enabled").exists("true".equalsIgnoreCase) + sparkProperties.get(DYN_ALLOCATION_ENABLED.key).exists("true".equalsIgnoreCase) // Try to set main class from JAR if no --class argument is given if (mainClass == null && !isPython && !isR && primaryResource != null) { @@ -304,19 +305,12 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S } private def validateKillArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { - error("Killing submissions is only supported in standalone or Mesos mode!") - } if (submissionToKill == null) { error("Please specify a submission to kill.") } } private def validateStatusRequestArguments(): Unit = { - if (!master.startsWith("spark://") && !master.startsWith("mesos://")) { - error( - "Requesting submission statuses is only supported in standalone or Mesos mode!") - } if (submissionToRequestStatusFor == null) { error("Please specify a submission to request status for.") } @@ -511,7 +505,7 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S if (unknownParam != null) { logInfo("Unknown/unsupported param " + unknownParam) } - val command = sys.env.get("_SPARK_CMD_USAGE").getOrElse( + val command = sys.env.getOrElse("_SPARK_CMD_USAGE", """Usage: spark-submit [options] [app arguments] |Usage: spark-submit --kill [submission ID] --master [spark://...] |Usage: spark-submit --status [submission ID] --master [spark://...] @@ -573,30 +567,31 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S | | Spark standalone or Mesos with cluster deploy mode only: | --supervise If given, restarts the driver on failure. + | + | Spark standalone, Mesos or K8s with cluster deploy mode only: | --kill SUBMISSION_ID If given, kills the driver specified. | --status SUBMISSION_ID If given, requests the status of the driver specified. | - | Spark standalone and Mesos only: + | Spark standalone, Mesos and Kubernetes only: | --total-executor-cores NUM Total cores for all executors. | - | Spark standalone and YARN only: - | --executor-cores NUM Number of cores per executor. (Default: 1 in YARN mode, - | or all available cores on the worker in standalone mode) + | Spark standalone, YARN and Kubernetes only: + | --executor-cores NUM Number of cores used by each executor. (Default: 1 in + | YARN and K8S modes, or all available cores on the worker + | in standalone mode). | - | YARN-only: - | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). + | Spark on YARN and Kubernetes only: | --num-executors NUM Number of executors to launch (Default: 2). | If dynamic allocation is enabled, the initial number of | executors will be at least NUM. + | --principal PRINCIPAL Principal to be used to login to KDC. + | --keytab KEYTAB The full path to the file that contains the keytab for the + | principal specified above. + | + | Spark on YARN only: + | --queue QUEUE_NAME The YARN queue to submit to (Default: "default"). | --archives ARCHIVES Comma separated list of archives to be extracted into the | working directory of each executor. - | --principal PRINCIPAL Principal to be used to login to KDC, while running on - | secure HDFS. - | --keytab KEYTAB The full path to the file that contains the keytab for the - | principal specified above. This keytab will be copied to - | the node running the Application Master via the Secure - | Distributed Cache, for renewing the login tickets and the - | delegation tokens periodically. """.stripMargin ) @@ -662,4 +657,10 @@ private[deploy] class SparkSubmitArguments(args: Seq[String], env: Map[String, S private def error(msg: String): Unit = throw new SparkException(msg) + private[deploy] def toSparkConf(sparkConf: Option[SparkConf] = None): SparkConf = { + // either use an existing config or create a new empty one + sparkProperties.foldLeft(sparkConf.getOrElse(new SparkConf())) { + case (conf, (k, v)) => conf.set(k, v) + } + } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala index c23a659e76df1..5f9b18ce01279 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/FsHistoryProvider.scala @@ -34,7 +34,7 @@ import com.fasterxml.jackson.annotation.JsonIgnore import com.google.common.io.ByteStreams import com.google.common.util.concurrent.MoreExecutors import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} -import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.hdfs.protocol.HdfsConstants import org.apache.hadoop.security.AccessControlException import org.fusesource.leveldbjni.internal.NativeDB @@ -42,13 +42,17 @@ import org.fusesource.leveldbjni.internal.NativeDB import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Status._ +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.scheduler._ import org.apache.spark.scheduler.ReplayListenerBus._ import org.apache.spark.status._ import org.apache.spark.status.KVUtils._ import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} -import org.apache.spark.status.config._ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{Clock, SystemClock, ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -86,31 +90,28 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) this(conf, new SystemClock()) } - import config._ import FsHistoryProvider._ // Interval between safemode checks. - private val SAFEMODE_CHECK_INTERVAL_S = conf.getTimeAsSeconds( - "spark.history.fs.safemodeCheck.interval", "5s") + private val SAFEMODE_CHECK_INTERVAL_S = conf.get(History.SAFEMODE_CHECK_INTERVAL_S) // Interval between each check for event log updates - private val UPDATE_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.update.interval", "10s") + private val UPDATE_INTERVAL_S = conf.get(History.UPDATE_INTERVAL_S) // Interval between each cleaner checks for event logs to delete - private val CLEAN_INTERVAL_S = conf.getTimeAsSeconds("spark.history.fs.cleaner.interval", "1d") + private val CLEAN_INTERVAL_S = conf.get(History.CLEANER_INTERVAL_S) // Number of threads used to replay event logs. - private val NUM_PROCESSING_THREADS = conf.getInt(SPARK_HISTORY_FS_NUM_REPLAY_THREADS, - Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) + private val NUM_PROCESSING_THREADS = conf.get(History.NUM_REPLAY_THREADS) - private val logDir = conf.get(EVENT_LOG_DIR) + private val logDir = conf.get(History.HISTORY_LOG_DIR) - private val HISTORY_UI_ACLS_ENABLE = conf.getBoolean("spark.history.ui.acls.enable", false) - private val HISTORY_UI_ADMIN_ACLS = conf.get("spark.history.ui.admin.acls", "") - private val HISTORY_UI_ADMIN_ACLS_GROUPS = conf.get("spark.history.ui.admin.acls.groups", "") - logInfo(s"History server ui acls " + (if (HISTORY_UI_ACLS_ENABLE) "enabled" else "disabled") + - "; users with admin permissions: " + HISTORY_UI_ADMIN_ACLS.toString + - "; groups with admin permissions" + HISTORY_UI_ADMIN_ACLS_GROUPS.toString) + private val historyUiAclsEnable = conf.get(History.HISTORY_SERVER_UI_ACLS_ENABLE) + private val historyUiAdminAcls = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS) + private val historyUiAdminAclsGroups = conf.get(History.HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS) + logInfo(s"History server ui acls " + (if (historyUiAclsEnable) "enabled" else "disabled") + + "; users with admin permissions: " + historyUiAdminAcls.mkString(",") + + "; groups with admin permissions" + historyUiAdminAclsGroups.mkString(",")) private val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) // Visible for testing @@ -185,13 +186,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) * Return a runnable that performs the given operation on the event logs. * This operation is expected to be executed periodically. */ - private def getRunner(operateFun: () => Unit): Runnable = { - new Runnable() { - override def run(): Unit = Utils.tryOrExit { - operateFun() - } - } - } + private def getRunner(operateFun: () => Unit): Runnable = + () => Utils.tryOrExit { operateFun() } /** * Fixed size thread pool to fetch and parse log files. @@ -220,29 +216,25 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // Cannot probe anything while the FS is in safe mode, so spawn a new thread that will wait // for the FS to leave safe mode before enabling polling. This allows the main history server // UI to be shown (so that the user can see the HDFS status). - val initThread = new Thread(new Runnable() { - override def run(): Unit = { - try { - while (isFsInSafeMode()) { - logInfo("HDFS is still in safe mode. Waiting...") - val deadline = clock.getTimeMillis() + - TimeUnit.SECONDS.toMillis(SAFEMODE_CHECK_INTERVAL_S) - clock.waitTillTime(deadline) - } - startPolling() - } catch { - case _: InterruptedException => + val initThread = new Thread(() => { + try { + while (isFsInSafeMode()) { + logInfo("HDFS is still in safe mode. Waiting...") + val deadline = clock.getTimeMillis() + + TimeUnit.SECONDS.toMillis(SAFEMODE_CHECK_INTERVAL_S) + clock.waitTillTime(deadline) } + startPolling() + } catch { + case _: InterruptedException => } }) initThread.setDaemon(true) initThread.setName(s"${getClass().getSimpleName()}-init") initThread.setUncaughtExceptionHandler(errorHandler.getOrElse( - new Thread.UncaughtExceptionHandler() { - override def uncaughtException(t: Thread, e: Throwable): Unit = { - logError("Error initializing FsHistoryProvider.", e) - System.exit(1) - } + (_: Thread, e: Throwable) => { + logError("Error initializing FsHistoryProvider.", e) + System.exit(1) })) initThread.start() initThread @@ -268,17 +260,24 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } // Disable the background thread during tests. - if (!conf.contains("spark.testing")) { + if (!conf.contains(IS_TESTING)) { // A task that periodically checks for event log updates on disk. logDebug(s"Scheduling update thread every $UPDATE_INTERVAL_S seconds") pool.scheduleWithFixedDelay( getRunner(() => checkForLogs()), 0, UPDATE_INTERVAL_S, TimeUnit.SECONDS) - if (conf.getBoolean("spark.history.fs.cleaner.enabled", false)) { + if (conf.get(CLEANER_ENABLED)) { // A task that periodically cleans event logs on disk. pool.scheduleWithFixedDelay( getRunner(() => cleanLogs()), 0, CLEAN_INTERVAL_S, TimeUnit.SECONDS) } + + if (conf.contains(DRIVER_LOG_DFS_DIR) && conf.get(DRIVER_LOG_CLEANER_ENABLED)) { + pool.scheduleWithFixedDelay(getRunner(() => cleanDriverLogs()), + 0, + conf.get(DRIVER_LOG_CLEANER_INTERVAL), + TimeUnit.SECONDS) + } } else { logDebug("Background update thread disabled for testing") } @@ -307,6 +306,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) override def getLastUpdatedTime(): Long = lastScanTime.get() + /** + * Split a comma separated String, filter out any empty items, and return a Sequence of strings + */ + private def stringToSeq(list: String): Seq[String] = { + list.split(',').map(_.trim).filter(!_.isEmpty) + } + override def getAppUI(appId: String, attemptId: Option[String]): Option[LoadedAppUI] = { val app = try { load(appId) @@ -323,13 +329,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val conf = this.conf.clone() val secManager = new SecurityManager(conf) - secManager.setAcls(HISTORY_UI_ACLS_ENABLE) + secManager.setAcls(historyUiAclsEnable) // make sure to set admin acls before view acls so they are properly picked up - secManager.setAdminAcls(HISTORY_UI_ADMIN_ACLS + "," + attempt.adminAcls.getOrElse("")) - secManager.setViewAcls(attempt.info.sparkUser, attempt.viewAcls.getOrElse("")) - secManager.setAdminAclsGroups(HISTORY_UI_ADMIN_ACLS_GROUPS + "," + - attempt.adminAclsGroups.getOrElse("")) - secManager.setViewAclsGroups(attempt.viewAclsGroups.getOrElse("")) + secManager.setAdminAcls(historyUiAdminAcls ++ stringToSeq(attempt.adminAcls.getOrElse(""))) + secManager.setViewAcls(attempt.info.sparkUser, stringToSeq(attempt.viewAcls.getOrElse(""))) + secManager.setAdminAclsGroups(historyUiAdminAclsGroups ++ + stringToSeq(attempt.adminAclsGroups.getOrElse(""))) + secManager.setViewAclsGroups(stringToSeq(attempt.viewAclsGroups.getOrElse(""))) val kvstore = try { diskManager match { @@ -344,10 +350,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) return None } - val ui = SparkUI.create(None, new AppStatusStore(kvstore), conf, secManager, app.info.name, - HistoryServer.getAttemptURI(appId, attempt.info.attemptId), - attempt.info.startTime.getTime(), - attempt.info.appSparkVersion) + val ui = SparkUI.create(None, new HistoryAppStatusStore(conf, kvstore), conf, secManager, + app.info.name, HistoryServer.getAttemptURI(appId, attempt.info.attemptId), + attempt.info.startTime.getTime(), attempt.info.appSparkVersion) loadPlugins().foreach(_.setupUI(ui)) val loadedUI = LoadedAppUI(ui) @@ -449,10 +454,32 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) listing.write(info.copy(lastProcessed = newLastScanTime, fileSize = entry.getLen())) } - if (info.fileSize < entry.getLen()) { + if (shouldReloadLog(info, entry)) { if (info.appId.isDefined && fastInProgressParsing) { // When fast in-progress parsing is on, we don't need to re-parse when the // size changes, but we do need to invalidate any existing UIs. + // Also, we need to update the `lastUpdated time` to display the updated time in + // the HistoryUI and to avoid cleaning the inprogress app while running. + val appInfo = listing.read(classOf[ApplicationInfoWrapper], info.appId.get) + + val attemptList = appInfo.attempts.map { attempt => + if (attempt.info.attemptId == info.attemptId) { + new AttemptInfoWrapper( + attempt.info.copy(lastUpdated = new Date(newLastScanTime)), + attempt.logPath, + attempt.fileSize, + attempt.adminAcls, + attempt.viewAcls, + attempt.adminAclsGroups, + attempt.viewAclsGroups) + } else { + attempt + } + } + + val updatedAppInfo = new ApplicationInfoWrapper(appInfo.info, attemptList) + listing.write(updatedAppInfo) + invalidateUI(info.appId.get, info.attemptId) false } else { @@ -466,8 +493,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the file is currently not being tracked by the SHS, add an entry for it and try // to parse it. This will allow the cleaner code to detect the file as stale later on // if it was not possible to parse it. - listing.write(LogInfo(entry.getPath().toString(), newLastScanTime, None, None, - entry.getLen())) + listing.write(LogInfo(entry.getPath().toString(), newLastScanTime, LogType.EventLogs, + None, None, entry.getLen())) entry.getLen() > 0 } } @@ -481,9 +508,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val tasks = updated.flatMap { entry => try { - val task: Future[Unit] = replayExecutor.submit(new Runnable { - override def run(): Unit = mergeApplicationListing(entry, newLastScanTime, true) - }, Unit) + val task: Future[Unit] = replayExecutor.submit( + () => mergeApplicationListing(entry, newLastScanTime, true)) Some(task -> entry.getPath) } catch { // let the iteration over the updated entries break, since an exception on @@ -510,6 +536,9 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // We don't have read permissions on the log file logWarning(s"Unable to read log $path", e.getCause) blacklist(path) + // SPARK-28157 We should remove this blacklisted entry from the KVStore + // to handle permission-only changes with the same file sizes later. + listing.delete(classOf[LogInfo], path.toString) case e: Exception => logError("Exception while merging application listings", e) } finally { @@ -541,6 +570,24 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } } + private[history] def shouldReloadLog(info: LogInfo, entry: FileStatus): Boolean = { + var result = info.fileSize < entry.getLen + if (!result && info.logPath.endsWith(EventLoggingListener.IN_PROGRESS)) { + try { + result = Utils.tryWithResource(fs.open(entry.getPath)) { in => + in.getWrappedStream match { + case dfsIn: DFSInputStream => info.fileSize < dfsIn.getFileLength + case _ => false + } + } + } catch { + case e: Exception => + logDebug(s"Failed to check the length for the file : ${info.logPath}", e) + } + } + result + } + private def cleanAppData(appId: String, attemptId: Option[String], logPath: String): Unit = { try { val app = load(appId) @@ -706,7 +753,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // listing data is good. invalidateUI(app.info.id, app.attempts.head.info.attemptId) addListing(app) - listing.write(LogInfo(logPath.toString(), scanTime, Some(app.info.id), + listing.write(LogInfo(logPath.toString(), scanTime, LogType.EventLogs, Some(app.info.id), app.attempts.head.info.attemptId, fileStatus.getLen())) // For a finished log, remove the corresponding "in progress" entry from the listing DB if @@ -735,7 +782,8 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) // If the app hasn't written down its app ID to the logs, still record the entry in the // listing db, with an empty ID. This will make the log eligible for deletion if the app // does not make progress after the configured max log age. - listing.write(LogInfo(logPath.toString(), scanTime, None, None, fileStatus.getLen())) + listing.write( + LogInfo(logPath.toString(), scanTime, LogType.EventLogs, None, None, fileStatus.getLen())) } } @@ -757,6 +805,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) */ private[history] def cleanLogs(): Unit = Utils.tryLog { val maxTime = clock.getTimeMillis() - conf.get(MAX_LOG_AGE_S) * 1000 + val maxNum = conf.get(MAX_LOG_NUM) val expired = listing.view(classOf[ApplicationInfoWrapper]) .index("oldestAttempt") @@ -769,23 +818,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) val (remaining, toDelete) = app.attempts.partition { attempt => attempt.info.lastUpdated.getTime() >= maxTime } - - if (remaining.nonEmpty) { - val newApp = new ApplicationInfoWrapper(app.info, remaining) - listing.write(newApp) - } - - toDelete.foreach { attempt => - logInfo(s"Deleting expired event log for ${attempt.logPath}") - val logPath = new Path(logDir, attempt.logPath) - listing.delete(classOf[LogInfo], logPath.toString()) - cleanAppData(app.id, attempt.info.attemptId, logPath.toString()) - deleteLog(logPath) - } - - if (remaining.isEmpty) { - listing.delete(app.getClass(), app.id) - } + deleteAttemptLogs(app, remaining, toDelete) } // Delete log files that don't have a valid application and exceed the configured max age. @@ -794,18 +827,123 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) .reverse() .first(maxTime) .asScala + .filter { l => l.logType == null || l.logType == LogType.EventLogs } .toList stale.foreach { log => if (log.appId.isEmpty) { logInfo(s"Deleting invalid / corrupt event log ${log.logPath}") - deleteLog(new Path(log.logPath)) + deleteLog(fs, new Path(log.logPath)) listing.delete(classOf[LogInfo], log.logPath) } } + + // If the number of files is bigger than MAX_LOG_NUM, + // clean up all completed attempts per application one by one. + val num = listing.view(classOf[LogInfo]).index("lastProcessed").asScala.size + var count = num - maxNum + if (count > 0) { + logInfo(s"Try to delete $count old event logs to keep $maxNum logs in total.") + val oldAttempts = listing.view(classOf[ApplicationInfoWrapper]) + .index("oldestAttempt") + .asScala + oldAttempts.foreach { app => + if (count > 0) { + // Applications may have multiple attempts, some of which may not be completed yet. + val (toDelete, remaining) = app.attempts.partition(_.info.completed) + count -= deleteAttemptLogs(app, remaining, toDelete) + } + } + if (count > 0) { + logWarning(s"Fail to clean up according to MAX_LOG_NUM policy ($maxNum).") + } + } + // Clean the blacklist from the expired entries. clearBlacklist(CLEAN_INTERVAL_S) } + private def deleteAttemptLogs( + app: ApplicationInfoWrapper, + remaining: List[AttemptInfoWrapper], + toDelete: List[AttemptInfoWrapper]): Int = { + if (remaining.nonEmpty) { + val newApp = new ApplicationInfoWrapper(app.info, remaining) + listing.write(newApp) + } + + var countDeleted = 0 + toDelete.foreach { attempt => + logInfo(s"Deleting expired event log for ${attempt.logPath}") + val logPath = new Path(logDir, attempt.logPath) + listing.delete(classOf[LogInfo], logPath.toString()) + cleanAppData(app.id, attempt.info.attemptId, logPath.toString()) + if (deleteLog(fs, logPath)) { + countDeleted += 1 + } + } + + if (remaining.isEmpty) { + listing.delete(app.getClass(), app.id) + } + + countDeleted + } + + /** + * Delete driver logs from the configured spark dfs dir that exceed the configured max age + */ + private[history] def cleanDriverLogs(): Unit = Utils.tryLog { + val driverLogDir = conf.get(DRIVER_LOG_DFS_DIR).get + val driverLogFs = new Path(driverLogDir).getFileSystem(hadoopConf) + val currentTime = clock.getTimeMillis() + val maxTime = currentTime - conf.get(MAX_DRIVER_LOG_AGE_S) * 1000 + val logFiles = driverLogFs.listLocatedStatus(new Path(driverLogDir)) + while (logFiles.hasNext()) { + val f = logFiles.next() + // Do not rely on 'modtime' as it is not updated for all filesystems when files are written to + val deleteFile = + try { + val info = listing.read(classOf[LogInfo], f.getPath().toString()) + // Update the lastprocessedtime of file if it's length or modification time has changed + if (info.fileSize < f.getLen() || info.lastProcessed < f.getModificationTime()) { + listing.write( + info.copy(lastProcessed = currentTime, fileSize = f.getLen())) + false + } else if (info.lastProcessed > maxTime) { + false + } else { + true + } + } catch { + case e: NoSuchElementException => + // For every new driver log file discovered, create a new entry in listing + listing.write(LogInfo(f.getPath().toString(), currentTime, LogType.DriverLogs, None, + None, f.getLen())) + false + } + if (deleteFile) { + logInfo(s"Deleting expired driver log for: ${f.getPath().getName()}") + listing.delete(classOf[LogInfo], f.getPath().toString()) + deleteLog(driverLogFs, f.getPath()) + } + } + + // Delete driver log file entries that exceed the configured max age and + // may have been deleted on filesystem externally. + val stale = listing.view(classOf[LogInfo]) + .index("lastProcessed") + .reverse() + .first(maxTime) + .asScala + .filter { l => l.logType != null && l.logType == LogType.DriverLogs } + .toList + stale.foreach { log => + logInfo(s"Deleting invalid driver log ${log.logPath}") + listing.delete(classOf[LogInfo], log.logPath) + deleteLog(driverLogFs, new Path(log.logPath)) + } + } + /** * Rebuilds the application state store from its event log. */ @@ -962,12 +1100,13 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) throw new NoSuchElementException(s"Cannot find attempt $attemptId of $appId.")) } - private def deleteLog(log: Path): Unit = { + private def deleteLog(fs: FileSystem, log: Path): Boolean = { + var deleted = false if (isBlacklisted(log)) { logDebug(s"Skipping deleting $log as we don't have permissions on it.") } else { try { - fs.delete(log, true) + deleted = fs.delete(log, true) } catch { case _: AccessControlException => logInfo(s"No permission to delete $log, ignoring.") @@ -975,6 +1114,7 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) logError(s"IOException in cleaning $log", ioe) } } + deleted } private def isCompleted(name: String): Boolean = { @@ -984,7 +1124,6 @@ private[history] class FsHistoryProvider(conf: SparkConf, clock: Clock) } private[history] object FsHistoryProvider { - private val SPARK_HISTORY_FS_NUM_REPLAY_THREADS = "spark.history.fs.numReplayThreads" private val APPL_START_EVENT_PREFIX = "{\"Event\":\"SparkListenerApplicationStart\"" @@ -1007,6 +1146,10 @@ private[history] case class FsHistoryProviderMetadata( uiVersion: Long, logDir: String) +private[history] object LogType extends Enumeration { + val DriverLogs, EventLogs = Value +} + /** * Tracking info for event logs detected in the configured log directory. Tracks both valid and * invalid logs (e.g. unparseable logs, recorded as logs with no app ID) so that the cleaner @@ -1015,6 +1158,7 @@ private[history] case class FsHistoryProviderMetadata( private[history] case class LogInfo( @KVIndexParam logPath: String, @KVIndexParam("lastProcessed") lastProcessed: Long, + logType: LogType.Value, appId: Option[String], attemptId: Option[String], fileSize: Long) @@ -1079,11 +1223,16 @@ private[history] class AppListingListener( // Only parse the first env update, since any future changes don't have any effect on // the ACLs set for the UI. if (!gotEnvUpdate) { + def emptyStringToNone(strOption: Option[String]): Option[String] = strOption match { + case Some("") => None + case _ => strOption + } + val allProperties = event.environmentDetails("Spark Properties").toMap - attempt.viewAcls = allProperties.get("spark.ui.view.acls") - attempt.adminAcls = allProperties.get("spark.admin.acls") - attempt.viewAclsGroups = allProperties.get("spark.ui.view.acls.groups") - attempt.adminAclsGroups = allProperties.get("spark.admin.acls.groups") + attempt.viewAcls = emptyStringToNone(allProperties.get(UI_VIEW_ACLS.key)) + attempt.adminAcls = emptyStringToNone(allProperties.get(ADMIN_ACLS.key)) + attempt.viewAclsGroups = emptyStringToNone(allProperties.get(UI_VIEW_ACLS_GROUPS.key)) + attempt.adminAclsGroups = emptyStringToNone(allProperties.get(ADMIN_ACLS_GROUPS.key)) gotEnvUpdate = true checkProgress() diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala new file mode 100644 index 0000000000000..73b2dc26a85dc --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryAppStatusStore.scala @@ -0,0 +1,78 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.history + +import org.apache.spark.SparkConf +import org.apache.spark.executor.ExecutorLogUrlHandler +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History._ +import org.apache.spark.status.AppStatusStore +import org.apache.spark.status.api.v1 +import org.apache.spark.util.kvstore.KVStore + +private[spark] class HistoryAppStatusStore( + conf: SparkConf, + store: KVStore) + extends AppStatusStore(store, None) with Logging { + + private val logUrlPattern: Option[String] = { + val appInfo = super.applicationInfo() + val applicationCompleted = appInfo.attempts.nonEmpty && appInfo.attempts.head.completed + if (applicationCompleted || conf.get(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP)) { + conf.get(CUSTOM_EXECUTOR_LOG_URL) + } else { + None + } + } + + private val logUrlHandler = new ExecutorLogUrlHandler(logUrlPattern) + + override def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { + val execList = super.executorList(activeOnly) + logUrlPattern match { + case Some(pattern) => execList.map(replaceLogUrls(_, pattern)) + case None => execList + } + } + + override def executorSummary(executorId: String): v1.ExecutorSummary = { + val execSummary = super.executorSummary(executorId) + logUrlPattern match { + case Some(pattern) => replaceLogUrls(execSummary, pattern) + case None => execSummary + } + } + + private def replaceLogUrls(exec: v1.ExecutorSummary, urlPattern: String): v1.ExecutorSummary = { + val newLogUrlMap = logUrlHandler.applyPattern(exec.executorLogs, exec.attributes) + replaceExecutorLogs(exec, newLogUrlMap) + } + + private def replaceExecutorLogs( + source: v1.ExecutorSummary, + newExecutorLogs: Map[String, String]): v1.ExecutorSummary = { + new v1.ExecutorSummary(source.id, source.hostPort, source.isActive, source.rddBlocks, + source.memoryUsed, source.diskUsed, source.totalCores, source.maxTasks, source.activeTasks, + source.failedTasks, source.completedTasks, source.totalTasks, source.totalDuration, + source.totalGCTime, source.totalInputBytes, source.totalShuffleRead, + source.totalShuffleWrite, source.isBlacklisted, source.maxMemory, source.addTime, + source.removeTime, source.removeReason, newExecutorLogs, source.memoryMetrics, + source.blacklistedInStages, source.peakMemoryMetrics, source.attributes) + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 32667ddf5c7ea..7a8ab7fddd79f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -27,12 +27,11 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val requestedIncomplete = - Option(UIUtils.stripXSS(request.getParameter("showIncomplete"))).getOrElse("false").toBoolean + val requestedIncomplete = Option(request.getParameter("showIncomplete")) + .getOrElse("false").toBoolean - val allAppsSize = parent.getApplicationList() - .count(isApplicationCompleted(_) != requestedIncomplete) + val displayApplications = parent.getApplicationList() + .exists(isApplicationCompleted(_) != requestedIncomplete) val eventLogsUnderProcessCount = parent.getEventLogsUnderProcess() val lastUpdatedTime = parent.getLastUpdatedTime() val providerConfig = parent.getProviderConfig() @@ -63,9 +62,9 @@ private[history] class HistoryPage(parent: HistoryServer) extends WebUIPage("") } { - if (allAppsSize > 0) { + if (displayApplications) { ++ + request, "/static/dataTables.rowsGroup.js")}> ++

++ ++ diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 56f3f59504a7d..7df36c5aeba07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -28,12 +28,12 @@ import org.eclipse.jetty.servlet.{ServletContextHandler, ServletHolder} import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.history.config.HISTORY_SERVER_UI_PORT import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.{ApiRootResource, ApplicationInfo, UIRoot} import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{ShutdownHookManager, SystemClock, Utils} /** @@ -56,7 +56,7 @@ class HistoryServer( with Logging with UIRoot with ApplicationCacheOperations { // How many applications to retain - private val retainedApplications = conf.getInt("spark.history.retainedApplications", 50) + private val retainedApplications = conf.get(History.RETAINED_APPLICATIONS) // How many applications the summary ui displays private[history] val maxApplications = conf.get(HISTORY_UI_MAX_APPS); @@ -150,17 +150,15 @@ class HistoryServer( ui: SparkUI, completed: Boolean) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") - handlers.synchronized { - ui.getHandlers.foreach(attachHandler) + ui.getHandlers.foreach { handler => + serverInfo.get.addHandler(handler, ui.securityManager) } } /** Detach a reconstructed UI from this server. Only valid after bind(). */ override def detachSparkUI(appId: String, attemptId: Option[String], ui: SparkUI): Unit = { assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") - handlers.synchronized { - ui.getHandlers.foreach(detachHandler) - } + ui.getHandlers.foreach(detachHandler) provider.onUIDetached(appId, attemptId, ui) } @@ -273,14 +271,13 @@ object HistoryServer extends Logging { initSecurity() val securityManager = createSecurityManager(conf) - val providerName = conf.getOption("spark.history.provider") + val providerName = conf.get(History.PROVIDER) .getOrElse(classOf[FsHistoryProvider].getName()) - val provider = Utils.classForName(providerName) + val provider = Utils.classForName[ApplicationHistoryProvider](providerName) .getConstructor(classOf[SparkConf]) .newInstance(conf) - .asInstanceOf[ApplicationHistoryProvider] - val port = conf.get(HISTORY_SERVER_UI_PORT) + val port = conf.get(History.HISTORY_SERVER_UI_PORT) val server = new HistoryServer(conf, provider, securityManager, port) server.bind() @@ -304,11 +301,10 @@ object HistoryServer extends Logging { config.set(SecurityManager.SPARK_AUTH_CONF, "false") } - if (config.getBoolean("spark.acls.enable", config.getBoolean("spark.ui.acls.enable", false))) { - logInfo("Either spark.acls.enable or spark.ui.acls.enable is configured, clearing it and " + - "only using spark.history.ui.acl.enable") - config.set("spark.acls.enable", "false") - config.set("spark.ui.acls.enable", "false") + if (config.get(ACLS_ENABLE)) { + logInfo(s"${ACLS_ENABLE.key} is configured, " + + s"clearing it and only using ${History.HISTORY_SERVER_UI_ACLS_ENABLE.key}") + config.set(ACLS_ENABLE, false) } new SecurityManager(config) @@ -319,10 +315,12 @@ object HistoryServer extends Logging { // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically // occur from the keytab. - if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + if (conf.get(History.KERBEROS_ENABLED)) { // if you have enabled kerberos the following 2 params must be set - val principalName = conf.get("spark.history.kerberos.principal") - val keytabFilename = conf.get("spark.history.kerberos.keytab") + val principalName = conf.get(History.KERBEROS_PRINCIPAL) + .getOrElse(throw new NoSuchElementException(History.KERBEROS_PRINCIPAL.key)) + val keytabFilename = conf.get(History.KERBEROS_KEYTAB) + .getOrElse(throw new NoSuchElementException(History.KERBEROS_KEYTAB.key)) SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala index 080ba12c2f0d1..dec89769c030b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -34,35 +34,21 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin @tailrec private def parse(args: List[String]): Unit = { - if (args.length == 1) { - setLogDirectory(args.head) - } else { - args match { - case ("--dir" | "-d") :: value :: tail => - setLogDirectory(value) - parse(tail) + args match { + case ("--help" | "-h") :: tail => + printUsageAndExit(0) - case ("--help" | "-h") :: tail => - printUsageAndExit(0) + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) - case ("--properties-file") :: value :: tail => - propertiesFile = value - parse(tail) + case Nil => - case Nil => - - case _ => - printUsageAndExit(1) - } + case _ => + printUsageAndExit(1) } } - private def setLogDirectory(value: String): Unit = { - logWarning("Setting log directory through the command line is deprecated as of " + - "Spark 1.1.0. Please set this through spark.history.fs.logDirectory instead.") - conf.set("spark.history.fs.logDirectory", value) - } - // This mutates the SparkConf, so all accesses to it must be made after this line Utils.loadDefaultSparkProperties(conf, propertiesFile) @@ -73,8 +59,6 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin |Usage: HistoryServer [options] | |Options: - | DIR Deprecated; set spark.history.fs.logDirectory directly - | --dir DIR (-d DIR) Deprecated; set spark.history.fs.logDirectory directly | --properties-file FILE Path to a custom Spark properties file. | Default is conf/spark-defaults.conf. | @@ -95,7 +79,7 @@ private[history] class HistoryServerArguments(conf: SparkConf, args: Array[Strin | | spark.history.fs.logDirectory Directory where app logs are stored | (default: file:/tmp/spark-events) - | spark.history.fs.updateInterval How often to reload log data from storage + | spark.history.fs.update.interval How often to reload log data from storage | (in seconds, default: 10) |""".stripMargin) // scalastyle:on println diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala index ad0dd23cb59c8..0a1f33395ad62 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerDiskManager.scala @@ -27,6 +27,7 @@ import org.apache.commons.io.FileUtils import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.History._ import org.apache.spark.status.KVUtils._ import org.apache.spark.util.{Clock, Utils} import org.apache.spark.util.kvstore.KVStore @@ -50,8 +51,6 @@ private class HistoryServerDiskManager( listing: KVStore, clock: Clock) extends Logging { - import config._ - private val appStoreDir = new File(path, "apps") if (!appStoreDir.isDirectory() && !appStoreDir.mkdir()) { throw new IllegalArgumentException(s"Failed to create app directory ($appStoreDir).") diff --git a/core/src/main/scala/org/apache/spark/deploy/history/config.scala b/core/src/main/scala/org/apache/spark/deploy/history/config.scala deleted file mode 100644 index 25ba9edb9e014..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/history/config.scala +++ /dev/null @@ -1,67 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.history - -import java.util.concurrent.TimeUnit - -import org.apache.spark.internal.config.ConfigBuilder -import org.apache.spark.network.util.ByteUnit - -private[spark] object config { - - val DEFAULT_LOG_DIR = "file:/tmp/spark-events" - - val EVENT_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory") - .stringConf - .createWithDefault(DEFAULT_LOG_DIR) - - val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("7d") - - val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path") - .doc("Local directory where to cache application history information. By default this is " + - "not set, meaning all history information will be kept in memory.") - .stringConf - .createOptional - - val MAX_LOCAL_DISK_USAGE = ConfigBuilder("spark.history.store.maxDiskUsage") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("10g") - - val HISTORY_SERVER_UI_PORT = ConfigBuilder("spark.history.ui.port") - .doc("Web UI port to bind Spark History Server") - .intConf - .createWithDefault(18080) - - val FAST_IN_PROGRESS_PARSING = - ConfigBuilder("spark.history.fs.inProgressOptimization.enabled") - .doc("Enable optimized handling of in-progress logs. This option may leave finished " + - "applications that fail to rename their event logs listed as in-progress.") - .booleanConf - .createWithDefault(true) - - val END_EVENT_REPARSE_CHUNK_SIZE = - ConfigBuilder("spark.history.fs.endEventReparseChunkSize") - .doc("How many bytes to parse at the end of log files looking for the end event. " + - "This is used to speed up generation of application listings by skipping unnecessary " + - "parts of event log files. It can be disabled by setting this config to 0.") - .bytesConf(ByteUnit.BYTE) - .createWithDefaultString("1m") - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index e1184248af460..3c0a49e4ab205 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -33,7 +33,11 @@ import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.deploy.rest.StandaloneRestServer import org.apache.spark.internal.Logging -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.rpc._ import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -54,12 +58,12 @@ private[deploy] class Master( // For application IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) - private val WORKER_TIMEOUT_MS = conf.getLong("spark.worker.timeout", 60) * 1000 - private val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) - private val RETAINED_DRIVERS = conf.getInt("spark.deploy.retainedDrivers", 200) - private val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) - private val RECOVERY_MODE = conf.get("spark.deploy.recoveryMode", "NONE") - private val MAX_EXECUTOR_RETRIES = conf.getInt("spark.deploy.maxExecutorRetries", 10) + private val workerTimeoutMs = conf.get(WORKER_TIMEOUT) * 1000 + private val retainedApplications = conf.get(RETAINED_APPLICATIONS) + private val retainedDrivers = conf.get(RETAINED_DRIVERS) + private val reaperIterations = conf.get(REAPER_ITERATIONS) + private val recoveryMode = conf.get(RECOVERY_MODE) + private val maxExecutorRetries = conf.get(MAX_EXECUTOR_RETRIES) val workers = new HashSet[WorkerInfo] val idToApp = new HashMap[String, ApplicationInfo] @@ -82,9 +86,10 @@ private[deploy] class Master( Utils.checkHost(address.host) - private val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) - private val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, - securityMgr) + private val masterMetricsSystem = + MetricsSystem.createMetricsSystem(MetricsSystemInstances.MASTER, conf, securityMgr) + private val applicationMetricsSystem = + MetricsSystem.createMetricsSystem(MetricsSystemInstances.APPLICATIONS, conf, securityMgr) private val masterSource = new MasterSource(this) // After onStart, webUi will be set @@ -111,17 +116,17 @@ private[deploy] class Master( // As a temporary workaround before better ways of configuring memory, we allow users to set // a flag that will perform round-robin scheduling across the nodes (spreading out each app // among all the nodes) instead of trying to consolidate each app onto a small # of nodes. - private val spreadOutApps = conf.getBoolean("spark.deploy.spreadOut", true) + private val spreadOutApps = conf.get(SPREAD_OUT_APPS) // Default maxCores for applications that don't specify it (i.e. pass Int.MaxValue) - private val defaultCores = conf.getInt("spark.deploy.defaultCores", Int.MaxValue) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + private val defaultCores = conf.get(DEFAULT_CORES) + val reverseProxy = conf.get(UI_REVERSE_PROXY) if (defaultCores < 1) { - throw new SparkException("spark.deploy.defaultCores must be positive") + throw new SparkException(s"${DEFAULT_CORES.key} must be positive") } // Alternative application submission gateway that is stable across Spark versions - private val restServerEnabled = conf.getBoolean("spark.master.rest.enabled", false) + private val restServerEnabled = conf.get(MASTER_REST_SERVER_ENABLED) private var restServer: Option[StandaloneRestServer] = None private var restServerBoundPort: Option[Int] = None @@ -138,21 +143,19 @@ private[deploy] class Master( logInfo(s"Running Spark version ${org.apache.spark.SPARK_VERSION}") webUi = new MasterWebUI(this, webUiPort) webUi.bind() - masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort + masterWebUiUrl = s"${webUi.scheme}$masterPublicAddress:${webUi.boundPort}" if (reverseProxy) { - masterWebUiUrl = conf.get("spark.ui.reverseProxyUrl", masterWebUiUrl) + masterWebUiUrl = conf.get(UI_REVERSE_PROXY_URL).orElse(Some(masterWebUiUrl)).get webUi.addProxy() logInfo(s"Spark Master is acting as a reverse proxy. Master, Workers and " + s"Applications UIs are available at $masterWebUiUrl") } - checkForWorkerTimeOutTask = forwardMessageThread.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - self.send(CheckForWorkerTimeOut) - } - }, 0, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + checkForWorkerTimeOutTask = forwardMessageThread.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { self.send(CheckForWorkerTimeOut) }, + 0, workerTimeoutMs, TimeUnit.MILLISECONDS) if (restServerEnabled) { - val port = conf.getInt("spark.master.rest.port", 6066) + val port = conf.get(MASTER_REST_SERVER_PORT) restServer = Some(new StandaloneRestServer(address.host, port, conf, self, masterUrl)) } restServerBoundPort = restServer.map(_.start()) @@ -166,7 +169,7 @@ private[deploy] class Master( applicationMetricsSystem.getServletHandlers.foreach(webUi.attachHandler) val serializer = new JavaSerializer(conf) - val (persistenceEngine_, leaderElectionAgent_) = RECOVERY_MODE match { + val (persistenceEngine_, leaderElectionAgent_) = recoveryMode match { case "ZOOKEEPER" => logInfo("Persisting recovery state to ZooKeeper") val zkFactory = @@ -177,7 +180,7 @@ private[deploy] class Master( new FileSystemRecoveryModeFactory(conf, serializer) (fsFactory.createPersistenceEngine(), fsFactory.createLeaderElectionAgent(this)) case "CUSTOM" => - val clazz = Utils.classForName(conf.get("spark.deploy.recoveryMode.factory")) + val clazz = Utils.classForName(conf.get(RECOVERY_MODE_FACTORY)) val factory = clazz.getConstructor(classOf[SparkConf], classOf[Serializer]) .newInstance(conf, serializer) .asInstanceOf[StandaloneRecoveryModeFactory] @@ -231,7 +234,7 @@ private[deploy] class Master( override def run(): Unit = Utils.tryLogNonFatalError { self.send(CompleteRecovery) } - }, WORKER_TIMEOUT_MS, TimeUnit.MILLISECONDS) + }, workerTimeoutMs, TimeUnit.MILLISECONDS) } case CompleteRecovery => completeRecovery() @@ -247,13 +250,13 @@ private[deploy] class Master( if (state == RecoveryState.STANDBY) { workerRef.send(MasterInStandby) } else if (idToWorker.contains(id)) { - workerRef.send(RegisterWorkerFailed("Duplicate worker ID")) + workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, true)) } else { val worker = new WorkerInfo(id, workerHost, workerPort, cores, memory, workerRef, workerWebUiUrl) if (registerWorker(worker)) { persistenceEngine.addWorker(worker) - workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress)) + workerRef.send(RegisteredWorker(self, masterWebUiUrl, masterAddress, false)) schedule() } else { val workerAddress = worker.endpoint.address @@ -309,8 +312,8 @@ private[deploy] class Master( // Important note: this code path is not exercised by tests, so be very careful when // changing this `if` condition. if (!normalExit - && appInfo.incrementRetryCount() >= MAX_EXECUTOR_RETRIES - && MAX_EXECUTOR_RETRIES >= 0) { // < 0 disables this application-killing path + && appInfo.incrementRetryCount() >= maxExecutorRetries + && maxExecutorRetries >= 0) { // < 0 disables this application-killing path val execs = appInfo.executors.values if (!execs.exists(_.state == ExecutorState.RUNNING)) { logError(s"Application ${appInfo.desc.name} with ID ${appInfo.id} failed " + @@ -366,7 +369,7 @@ private[deploy] class Master( val validExecutors = executors.filter(exec => idToApp.get(exec.appId).isDefined) for (exec <- validExecutors) { - val app = idToApp.get(exec.appId).get + val app = idToApp(exec.appId) val execInfo = app.addExecutor(worker, exec.cores, Some(exec.execId)) worker.addExecutor(execInfo) execInfo.copyState(exec) @@ -868,8 +871,8 @@ private[deploy] class Master( endpointToApp -= app.driver addressToApp -= app.driver.address - if (completedApps.size >= RETAINED_APPLICATIONS) { - val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) + if (completedApps.size >= retainedApplications) { + val toRemove = math.max(retainedApplications / 10, 1) completedApps.take(toRemove).foreach { a => applicationMetricsSystem.removeSource(a.appSource) } @@ -987,14 +990,15 @@ private[deploy] class Master( private def timeOutDeadWorkers() { // Copy the workers into an array so we don't modify the hashset while iterating through it val currentTime = System.currentTimeMillis() - val toRemove = workers.filter(_.lastHeartbeat < currentTime - WORKER_TIMEOUT_MS).toArray + val toRemove = workers.filter(_.lastHeartbeat < currentTime - workerTimeoutMs).toArray for (worker <- toRemove) { if (worker.state != WorkerState.DEAD) { + val workerTimeoutSecs = TimeUnit.MILLISECONDS.toSeconds(workerTimeoutMs) logWarning("Removing %s because we got no heartbeat in %d seconds".format( - worker.id, WORKER_TIMEOUT_MS / 1000)) - removeWorker(worker, s"Not receiving heartbeat for ${WORKER_TIMEOUT_MS / 1000} seconds") + worker.id, workerTimeoutSecs)) + removeWorker(worker, s"Not receiving heartbeat for $workerTimeoutSecs seconds") } else { - if (worker.lastHeartbeat < currentTime - ((REAPER_ITERATIONS + 1) * WORKER_TIMEOUT_MS)) { + if (worker.lastHeartbeat < currentTime - ((reaperIterations + 1) * workerTimeoutMs)) { workers -= worker // we've seen this DEAD worker in the UI, etc. for long enough; cull it } } @@ -1029,8 +1033,8 @@ private[deploy] class Master( case Some(driver) => logInfo(s"Removing driver: $driverId") drivers -= driver - if (completedDrivers.size >= RETAINED_DRIVERS) { - val toRemove = math.max(RETAINED_DRIVERS / 10, 1) + if (completedDrivers.size >= retainedDrivers) { + val toRemove = math.max(retainedDrivers / 10, 1) completedDrivers.trimStart(toRemove) } completedDrivers += driver diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala index 615d2533cf085..cd31bbdcfab59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterArguments.scala @@ -21,6 +21,7 @@ import scala.annotation.tailrec import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.MASTER_UI_PORT import org.apache.spark.util.{IntParam, Utils} /** @@ -53,8 +54,8 @@ private[master] class MasterArguments(args: Array[String], conf: SparkConf) exte // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.master.ui.port")) { - webUiPort = conf.get("spark.master.ui.port").toInt + if (conf.contains(MASTER_UI_PORT.key)) { + webUiPort = conf.get(MASTER_UI_PORT) } @tailrec diff --git a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala index ffdd635be4f5f..470798793cecb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/RecoveryModeFactory.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.RECOVERY_DIRECTORY import org.apache.spark.serializer.Serializer /** @@ -52,11 +53,11 @@ abstract class StandaloneRecoveryModeFactory(conf: SparkConf, serializer: Serial private[master] class FileSystemRecoveryModeFactory(conf: SparkConf, serializer: Serializer) extends StandaloneRecoveryModeFactory(conf, serializer) with Logging { - val RECOVERY_DIR = conf.get("spark.deploy.recoveryDirectory", "") + val recoveryDir = conf.get(RECOVERY_DIRECTORY) def createPersistenceEngine(): PersistenceEngine = { - logInfo("Persisting recovery state to directory: " + RECOVERY_DIR) - new FileSystemPersistenceEngine(RECOVERY_DIR, serializer) + logInfo("Persisting recovery state to directory: " + recoveryDir) + new FileSystemPersistenceEngine(recoveryDir, serializer) } def createLeaderElectionAgent(master: LeaderElectable): LeaderElectionAgent = { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 1e8dabfbe6b0c..47f309144bdc0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -23,11 +23,12 @@ import org.apache.curator.framework.recipes.leader.{LeaderLatch, LeaderLatchList import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_DIRECTORY private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderElectable, conf: SparkConf) extends LeaderLatchListener with LeaderElectionAgent with Logging { - val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" + val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/leader_election" private var zk: CuratorFramework = _ private var leaderLatch: LeaderLatch = _ @@ -38,7 +39,7 @@ private[master] class ZooKeeperLeaderElectionAgent(val masterInstance: LeaderEle private def start() { logInfo("Starting ZooKeeper LeaderElection agent") zk = SparkCuratorUtil.newClient(conf) - leaderLatch = new LeaderLatch(zk, WORKING_DIR) + leaderLatch = new LeaderLatch(zk, workingDir) leaderLatch.addListener(this) leaderLatch.start() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index af850e4871e57..73dd0de017960 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -28,6 +28,7 @@ import org.apache.zookeeper.CreateMode import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.serializer.Serializer @@ -35,22 +36,22 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer extends PersistenceEngine with Logging { - private val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + private val workingDir = conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark") + "/master_status" private val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) override def persist(name: String, obj: Object): Unit = { - serializeIntoFile(WORKING_DIR + "/" + name, obj) + serializeIntoFile(workingDir + "/" + name, obj) } override def unpersist(name: String): Unit = { - zk.delete().forPath(WORKING_DIR + "/" + name) + zk.delete().forPath(workingDir + "/" + name) } override def read[T: ClassTag](prefix: String): Seq[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala + zk.getChildren.forPath(workingDir).asScala .filter(_.startsWith(prefix)).flatMap(deserializeFromFile[T]) } @@ -66,13 +67,13 @@ private[master] class ZooKeeperPersistenceEngine(conf: SparkConf, val serializer } private def deserializeFromFile[T](filename: String)(implicit m: ClassTag[T]): Option[T] = { - val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) + val fileData = zk.getData().forPath(workingDir + "/" + filename) try { Some(serializer.newInstance().deserialize[T](ByteBuffer.wrap(fileData))) } catch { case e: Exception => logWarning("Exception while reading persisted file, deleting", e) - zk.delete().forPath(WORKING_DIR + "/" + filename) + zk.delete().forPath(workingDir + "/" + filename) None } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index fad4e46dc035d..bcd7a7e4ccdb5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -33,8 +33,7 @@ private[ui] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") /** Executor details for a particular application */ def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val appId = UIUtils.stripXSS(request.getParameter("appId")) + val appId = request.getParameter("appId") val state = master.askSync[MasterStateResponse](RequestMasterState) val app = state.activeApps.find(_.id == appId) .getOrElse(state.completedApps.find(_.id == appId).orNull) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index b8afe203fbfa2..6701465c023c7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -57,10 +57,8 @@ private[ui] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { private def handleKillRequest(request: HttpServletRequest, action: String => Unit): Unit = { if (parent.killEnabled && parent.master.securityMgr.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val killFlag = - Option(UIUtils.stripXSS(request.getParameter("terminate"))).getOrElse("false").toBoolean - val id = Option(UIUtils.stripXSS(request.getParameter("id"))) + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val id = Option(request.getParameter("id")) if (id.isDefined && killFlag) { action(id.get) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index e87b2240564bd..be402ae247511 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master.ui import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.Master import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_KILL_ENABLED import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ @@ -34,7 +35,7 @@ class MasterWebUI( requestedPort, master.conf, name = "MasterUI") with Logging { val masterEndpointRef = master.self - val killEnabled = master.conf.getBoolean("spark.ui.killEnabled", true) + val killEnabled = master.conf.get(UI_KILL_ENABLED) initialize() diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala index 31a8e3e60c067..1648ba516d9b6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/RestSubmissionClient.scala @@ -61,8 +61,6 @@ import org.apache.spark.util.Utils private[spark] class RestSubmissionClient(master: String) extends Logging { import RestSubmissionClient._ - private val supportedMasterPrefixes = Seq("spark://", "mesos://") - private val masters: Array[String] = if (master.startsWith("spark://")) { Utils.parseStandaloneMasterUrls(master) } else { @@ -408,6 +406,12 @@ private[spark] class RestSubmissionClient(master: String) extends Logging { } private[spark] object RestSubmissionClient { + + val supportedMasterPrefixes = Seq("spark://", "mesos://") + + // SPARK_HOME and SPARK_CONF_DIR are filtered out because they are usually wrong + // on the remote machine (SPARK-12345) (SPARK-25934) + private val BLACKLISTED_SPARK_ENV_VARS = Set("SPARK_ENV_LOADED", "SPARK_HOME", "SPARK_CONF_DIR") private val REPORT_DRIVER_STATUS_INTERVAL = 1000 private val REPORT_DRIVER_STATUS_MAX_TRIES = 10 val PROTOCOL_VERSION = "v1" @@ -417,11 +421,13 @@ private[spark] object RestSubmissionClient { */ private[rest] def filterSystemEnvironment(env: Map[String, String]): Map[String, String] = { env.filterKeys { k => - // SPARK_HOME is filtered out because it is usually wrong on the remote machine (SPARK-12345) - (k.startsWith("SPARK_") && k != "SPARK_ENV_LOADED" && k != "SPARK_HOME") || - k.startsWith("MESOS_") + (k.startsWith("SPARK_") && !BLACKLISTED_SPARK_ENV_VARS.contains(k)) || k.startsWith("MESOS_") } } + + private[spark] def supportsRestClient(master: String): Boolean = { + supportedMasterPrefixes.exists(master.startsWith) + } } private[spark] class RestSubmissionClientApp extends SparkApplication { @@ -454,5 +460,4 @@ private[spark] class RestSubmissionClientApp extends SparkApplication { val env = RestSubmissionClient.filterSystemEnvironment(sys.env) run(appResource, mainClass, appArgs, conf, env) } - } diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala index 22b65abce611a..f912ed64c80bd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/StandaloneRestServer.scala @@ -23,6 +23,8 @@ import javax.servlet.http.HttpServletResponse import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.{Command, DeployMessages, DriverDescription} import org.apache.spark.deploy.ClientArguments._ +import org.apache.spark.internal.config +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.Utils @@ -132,12 +134,23 @@ private[rest] class StandaloneSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) + val driverDefaultJavaOptions = sparkProperties.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) + // The semantics of "spark.master" and the masterUrl are different. While the + // property "spark.master" could contain all registered masters, masterUrl + // contains only the active master. To make sure a Spark driver can recover + // in a multi-master setup, we use the "spark.master" property while submitting + // the driver. + val masters = sparkProperties.get("spark.master") + val (_, masterPort) = Utils.extractHostPortFromSparkUrl(masterUrl) + val masterRestPort = this.conf.get(config.MASTER_REST_SERVER_PORT) + val updatedMasters = masters.map( + _.replace(s":$masterRestPort", s":$masterPort")).getOrElse(masterUrl) val appArgs = request.appArgs // Filter SPARK_LOCAL_(IP|HOSTNAME) environment variables from being set on the remote system. val environmentVariables = @@ -146,12 +159,14 @@ private[rest] class StandaloneSubmitRequestServlet( // Construct driver description val conf = new SparkConf(false) .setAll(sparkProperties) - .set("spark.master", masterUrl) + .set("spark.master", updatedMasters) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val defaultJavaOpts = driverDefaultJavaOptions.map(Utils.splitCommandString) + .getOrElse(Seq.empty) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaOpts = sparkJavaOpts ++ extraJavaOpts + val javaOpts = sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts val command = new Command( "org.apache.spark.deploy.worker.DriverWrapper", Seq("{{WORKER_URL}}", "{{USER_JAR}}", mainClass) ++ appArgs, // args to the DriverWrapper diff --git a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala index 86ddf954ca128..7f462148c71a1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/rest/SubmitRestProtocolRequest.scala @@ -19,6 +19,7 @@ package org.apache.spark.deploy.rest import scala.util.Try +import org.apache.spark.internal.config import org.apache.spark.util.Utils /** @@ -49,11 +50,11 @@ private[rest] class CreateSubmissionRequest extends SubmitRestProtocolRequest { assertFieldIsSet(appArgs, "appArgs") assertFieldIsSet(environmentVariables, "environmentVariables") assertPropertyIsSet("spark.app.name") - assertPropertyIsBoolean("spark.driver.supervise") - assertPropertyIsNumeric("spark.driver.cores") - assertPropertyIsNumeric("spark.cores.max") - assertPropertyIsMemory("spark.driver.memory") - assertPropertyIsMemory("spark.executor.memory") + assertPropertyIsBoolean(config.DRIVER_SUPERVISE.key) + assertPropertyIsNumeric(config.DRIVER_CORES.key) + assertPropertyIsNumeric(config.CORES_MAX.key) + assertPropertyIsMemory(config.DRIVER_MEMORY.key) + assertPropertyIsMemory(config.EXECUTOR_MEMORY.key) } private def assertPropertyIsSet(key: String): Unit = diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala index 5dcde4ec3a8a4..2e21adac86a15 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HBaseDelegationTokenProvider.scala @@ -17,6 +17,8 @@ package org.apache.spark.deploy.security +import java.io.Closeable + import scala.reflect.runtime.universe import scala.util.control.NonFatal @@ -26,6 +28,7 @@ import org.apache.hadoop.security.token.{Token, TokenIdentifier} import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.security.HadoopDelegationTokenProvider import org.apache.spark.util.Utils private[security] class HBaseDelegationTokenProvider @@ -40,8 +43,8 @@ private[security] class HBaseDelegationTokenProvider try { val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) val obtainToken = mirror.classLoader. - loadClass("org.apache.hadoop.hbase.security.token.TokenUtil"). - getMethod("obtainToken", classOf[Configuration]) + loadClass("org.apache.hadoop.hbase.security.token.TokenUtil") + .getMethod("obtainToken", classOf[Configuration]) logDebug("Attempting to fetch HBase security token.") val token = obtainToken.invoke(null, hbaseConf(hadoopConf)) @@ -50,12 +53,58 @@ private[security] class HBaseDelegationTokenProvider creds.addToken(token.getService, token) } catch { case NonFatal(e) => - logDebug(s"Failed to get token from service $serviceName", e) + logWarning(s"Failed to get token from service $serviceName due to " + e + + s" Retrying to fetch HBase security token with hbase connection parameter.") + // Seems to be spark is trying to get the token from HBase 2.x.x version or above where the + // obtainToken(Configuration conf) API has been removed. Lets try obtaining the token from + // another compatible API of HBase service. + obtainDelegationTokensWithHBaseConn(hadoopConf, creds) } - None } + /** + * Token obtainToken(Configuration conf) is a deprecated + * method and in Hbase 2.0.0 the method is already removed. + * The HBase client API used in below method is introduced from HBase 0.98.9 version, + * to invoke this api first connection object has to be retrieved from ConnectionFactory and the + * same connection can be passed to + * Token obtainToken(Connection conn) API + * + * @param hadoopConf Configuration of current Hadoop Compatible system. + * @param creds Credentials to add tokens and security keys to. + */ + private def obtainDelegationTokensWithHBaseConn( + hadoopConf: Configuration, + creds: Credentials): Unit = { + var hbaseConnection : Closeable = null + try { + val mirror = universe.runtimeMirror(Utils.getContextOrSparkClassLoader) + val connectionFactoryClass = mirror.classLoader + .loadClass("org.apache.hadoop.hbase.client.ConnectionFactory") + .getMethod("createConnection", classOf[Configuration]) + hbaseConnection = connectionFactoryClass.invoke(null, hbaseConf(hadoopConf)) + .asInstanceOf[Closeable] + val connectionParamTypeClassRef = mirror.classLoader + .loadClass("org.apache.hadoop.hbase.client.Connection") + val obtainTokenMethod = mirror.classLoader + .loadClass("org.apache.hadoop.hbase.security.token.TokenUtil") + .getMethod("obtainToken", connectionParamTypeClassRef) + logDebug("Attempting to fetch HBase security token.") + val token = obtainTokenMethod.invoke(null, hbaseConnection) + .asInstanceOf[Token[_ <: TokenIdentifier]] + logInfo(s"Get token from HBase: ${token.toString}") + creds.addToken(token.getService, token) + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token from service $serviceName", e) + } finally { + if (null != hbaseConnection) { + hbaseConnection.close() + } + } + } + override def delegationTokensRequired( sparkConf: SparkConf, hadoopConf: Configuration): Boolean = { @@ -71,7 +120,9 @@ private[security] class HBaseDelegationTokenProvider confCreate.invoke(null, conf).asInstanceOf[Configuration] } catch { case NonFatal(e) => - logDebug("Fail to invoke HBaseConfiguration", e) + // Keep at debug level since this is executed even when HBase tokens are not needed. + // Avoids a noisy warning for users who don't care about HBase. + logDebug("Unable to load HBaseConfiguration.", e) conf } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala index ab8d8d96a9b08..759d857d56e0e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManager.scala @@ -17,76 +17,163 @@ package org.apache.spark.deploy.security +import java.io.File +import java.net.URI +import java.security.PrivilegedExceptionAction +import java.util.ServiceLoader +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import scala.collection.mutable + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem -import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.rpc.RpcEndpointRef +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens +import org.apache.spark.security.HadoopDelegationTokenProvider +import org.apache.spark.ui.UIUtils +import org.apache.spark.util.{ThreadUtils, Utils} /** - * Manages all the registered HadoopDelegationTokenProviders and offer APIs for other modules to - * obtain delegation tokens and their renewal time. By default [[HadoopFSDelegationTokenProvider]], - * [[HiveDelegationTokenProvider]] and [[HBaseDelegationTokenProvider]] will be loaded in if not - * explicitly disabled. + * Manager for delegation tokens in a Spark application. + * + * When delegation token renewal is enabled, this manager will make sure long-running apps can + * run without interruption while accessing secured services. It periodically logs in to the KDC + * with user-provided credentials, and contacts all the configured secure services to obtain + * delegation tokens to be distributed to the rest of the application. * - * Also, each HadoopDelegationTokenProvider is controlled by - * spark.security.credentials.{service}.enabled, and will not be loaded if this config is set to - * false. For example, Hive's delegation token provider [[HiveDelegationTokenProvider]] can be - * enabled/disabled by the configuration spark.security.credentials.hive.enabled. + * New delegation tokens are created once 75% of the renewal interval of the original tokens has + * elapsed. The new tokens are sent to the Spark driver endpoint. The driver is tasked with + * distributing the tokens to other processes that might need them. * - * @param sparkConf Spark configuration - * @param hadoopConf Hadoop configuration - * @param fileSystems Delegation tokens will be fetched for these Hadoop filesystems. + * Renewal can be enabled in two different ways: by providing a principal and keytab to Spark, or by + * enabling renewal based on the local credential cache. The latter has the drawback that Spark + * can't create new TGTs by itself, so the user has to manually update the Kerberos ticket cache + * externally. + * + * This class can also be used just to create delegation tokens, by calling the + * `obtainDelegationTokens` method. This option does not require calling the `start` method nor + * providing a driver reference, but leaves it up to the caller to distribute the tokens that were + * generated. */ private[spark] class HadoopDelegationTokenManager( - sparkConf: SparkConf, - hadoopConf: Configuration, - fileSystems: Configuration => Set[FileSystem]) - extends Logging { + protected val sparkConf: SparkConf, + protected val hadoopConf: Configuration, + protected val schedulerRef: RpcEndpointRef) extends Logging { private val deprecatedProviderEnabledConfigs = List( "spark.yarn.security.tokens.%s.enabled", "spark.yarn.security.credentials.%s.enabled") private val providerEnabledConfig = "spark.security.credentials.%s.enabled" - // Maintain all the registered delegation token providers - private val delegationTokenProviders = getDelegationTokenProviders + private val principal = sparkConf.get(PRINCIPAL).orNull + + // The keytab can be a local: URI for cluster mode, so translate it to a regular path. If it is + // needed later on, the code will check that it exists. + private val keytab = sparkConf.get(KEYTAB).map { uri => new URI(uri).getPath() }.orNull + + require((principal == null) == (keytab == null), + "Both principal and keytab must be defined, or neither.") + + private val delegationTokenProviders = loadProviders() logDebug("Using the following builtin delegation token providers: " + s"${delegationTokenProviders.keys.mkString(", ")}.") - /** Construct a [[HadoopDelegationTokenManager]] for the default Hadoop filesystem */ - def this(sparkConf: SparkConf, hadoopConf: Configuration) = { - this( - sparkConf, - hadoopConf, - hadoopConf => Set(FileSystem.get(hadoopConf).getHomeDirectory.getFileSystem(hadoopConf))) + private var renewalExecutor: ScheduledExecutorService = _ + + /** @return Whether delegation token renewal is enabled. */ + def renewalEnabled: Boolean = sparkConf.get(KERBEROS_RENEWAL_CREDENTIALS) match { + case "keytab" => principal != null + case "ccache" => UserGroupInformation.getCurrentUser().hasKerberosCredentials() + case _ => false } - private def getDelegationTokenProviders: Map[String, HadoopDelegationTokenProvider] = { - val providers = Seq(new HadoopFSDelegationTokenProvider(fileSystems)) ++ - safeCreateProvider(new HiveDelegationTokenProvider) ++ - safeCreateProvider(new HBaseDelegationTokenProvider) + /** + * Start the token renewer. Requires a principal and keytab. Upon start, the renewer will + * obtain delegation tokens for all configured services and send them to the driver, and + * set up tasks to periodically get fresh tokens as needed. + * + * This method requires that a keytab has been provided to Spark, and will try to keep the + * logged in user's TGT valid while this manager is active. + * + * @return New set of delegation tokens created for the configured principal. + */ + def start(): Array[Byte] = { + require(renewalEnabled, "Token renewal must be enabled to start the renewer.") + require(schedulerRef != null, "Token renewal requires a scheduler endpoint.") + renewalExecutor = + ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") - // Filter out providers for which spark.security.credentials.{service}.enabled is false. - providers - .filter { p => isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap + val ugi = UserGroupInformation.getCurrentUser() + if (ugi.isFromKeytab()) { + // In Hadoop 2.x, renewal of the keytab-based login seems to be automatic, but in Hadoop 3.x, + // it is configurable (see hadoop.kerberos.keytab.login.autorenewal.enabled, added in + // HADOOP-9567). This task will make sure that the user stays logged in regardless of that + // configuration's value. Note that checkTGTAndReloginFromKeytab() is a no-op if the TGT does + // not need to be renewed yet. + val tgtRenewalTask = new Runnable() { + override def run(): Unit = { + ugi.checkTGTAndReloginFromKeytab() + } + } + val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) + renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, + TimeUnit.SECONDS) + } + + updateTokensTask() } - private def safeCreateProvider( - createFn: => HadoopDelegationTokenProvider): Option[HadoopDelegationTokenProvider] = { - try { - Some(createFn) - } catch { - case t: Throwable => - logDebug(s"Failed to load built in provider.", t) - None + def stop(): Unit = { + if (renewalExecutor != null) { + renewalExecutor.shutdownNow() } } - def isServiceEnabled(serviceName: String): Boolean = { + /** + * Fetch new delegation tokens for configured services, storing them in the given credentials. + * + * @param creds Credentials object where to store the delegation tokens. + */ + def obtainDelegationTokens(creds: Credentials): Unit = { + val freshUGI = doLogin() + freshUGI.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val (newTokens, _) = obtainDelegationTokens() + creds.addAll(newTokens) + } + }) + } + + /** + * Fetch new delegation tokens for configured services. + * + * @return 2-tuple (credentials with new tokens, time by which the tokens must be renewed) + */ + private def obtainDelegationTokens(): (Credentials, Long) = { + val creds = new Credentials() + val nextRenewal = delegationTokenProviders.values.flatMap { provider => + if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { + provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) + } else { + logDebug(s"Service ${provider.serviceName} does not require a token." + + s" Check your configuration to see if security is disabled or not.") + None + } + }.foldLeft(Long.MaxValue)(math.min) + (creds, nextRenewal) + } + + // Visible for testing. + def isProviderLoaded(serviceName: String): Boolean = { + delegationTokenProviders.contains(serviceName) + } + + protected def isServiceEnabled(serviceName: String): Boolean = { val key = providerEnabledConfig.format(serviceName) deprecatedProviderEnabledConfigs.foreach { pattern => @@ -109,33 +196,103 @@ private[spark] class HadoopDelegationTokenManager( .getOrElse(isEnabledDeprecated) } + private def scheduleRenewal(delay: Long): Unit = { + val _delay = math.max(0, delay) + logInfo(s"Scheduling renewal in ${UIUtils.formatDuration(delay)}.") + + val renewalTask = new Runnable() { + override def run(): Unit = { + updateTokensTask() + } + } + renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) + } + /** - * Get delegation token provider for the specified service. + * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself + * to fetch the next set of tokens when needed. */ - def getServiceDelegationTokenProvider(service: String): Option[HadoopDelegationTokenProvider] = { - delegationTokenProviders.get(service) + private def updateTokensTask(): Array[Byte] = { + try { + val freshUGI = doLogin() + val creds = obtainTokensAndScheduleRenewal(freshUGI) + val tokens = SparkHadoopUtil.get.serialize(creds) + + logInfo("Updating delegation tokens.") + schedulerRef.send(UpdateDelegationTokens(tokens)) + tokens + } catch { + case _: InterruptedException => + // Ignore, may happen if shutting down. + null + case e: Exception => + val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) + logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + + " If this happens too often tasks will fail.", e) + scheduleRenewal(delay) + null + } } /** - * Writes delegation tokens to creds. Delegation tokens are fetched from all registered - * providers. + * Obtain new delegation tokens from the available providers. Schedules a new task to fetch + * new tokens before the new set expires. * - * @param hadoopConf hadoop Configuration - * @param creds Credentials that will be updated in place (overwritten) - * @return Time after which the fetched delegation tokens should be renewed. + * @return Credentials containing the new tokens. */ - def obtainDelegationTokens( - hadoopConf: Configuration, - creds: Credentials): Long = { - delegationTokenProviders.values.flatMap { provider => - if (provider.delegationTokensRequired(sparkConf, hadoopConf)) { - provider.obtainDelegationTokens(hadoopConf, sparkConf, creds) - } else { - logDebug(s"Service ${provider.serviceName} does not require a token." + - s" Check your configuration to see if security is disabled or not.") - None + private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { + ugi.doAs(new PrivilegedExceptionAction[Credentials]() { + override def run(): Credentials = { + val (creds, nextRenewal) = obtainDelegationTokens() + + // Calculate the time when new credentials should be created, based on the configured + // ratio. + val now = System.currentTimeMillis + val ratio = sparkConf.get(CREDENTIALS_RENEWAL_INTERVAL_RATIO) + val delay = (ratio * (nextRenewal - now)).toLong + scheduleRenewal(delay) + creds } - }.foldLeft(Long.MaxValue)(math.min) + }) } -} + private def doLogin(): UserGroupInformation = { + if (principal != null) { + logInfo(s"Attempting to login to KDC using principal: $principal") + require(new File(keytab).isFile(), s"Cannot find keytab at $keytab.") + val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) + logInfo("Successfully logged into KDC.") + ugi + } else if (!SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser())) { + logInfo(s"Attempting to load user's ticket cache.") + val ccache = sparkConf.getenv("KRB5CCNAME") + val user = Option(sparkConf.getenv("KRB5PRINCIPAL")).getOrElse( + UserGroupInformation.getCurrentUser().getUserName()) + UserGroupInformation.getUGIFromTicketCache(ccache, user) + } else { + UserGroupInformation.getCurrentUser() + } + } + + private def loadProviders(): Map[String, HadoopDelegationTokenProvider] = { + val loader = ServiceLoader.load(classOf[HadoopDelegationTokenProvider], + Utils.getContextOrSparkClassLoader) + val providers = mutable.ArrayBuffer[HadoopDelegationTokenProvider]() + + val iterator = loader.iterator + while (iterator.hasNext) { + try { + providers += iterator.next + } catch { + case t: Throwable => + logDebug(s"Failed to load built in provider.", t) + } + } + + // Filter out providers for which spark.security.credentials.{service}.enabled is false. + providers + .filter { p => isServiceEnabled(p.serviceName) } + .map { p => (p.serviceName, p) } + .toMap + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala index 21ca669ea98f0..ac432e7581e90 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProvider.scala @@ -19,9 +19,10 @@ package org.apache.spark.deploy.security import scala.collection.JavaConverters._ import scala.util.Try +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapred.Master import org.apache.hadoop.security.{Credentials, UserGroupInformation} import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier @@ -29,8 +30,9 @@ import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdenti import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.security.HadoopDelegationTokenProvider -private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration => Set[FileSystem]) +private[deploy] class HadoopFSDelegationTokenProvider extends HadoopDelegationTokenProvider with Logging { // This tokenRenewalInterval will be set in the first call to obtainDelegationTokens. @@ -44,29 +46,34 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration hadoopConf: Configuration, sparkConf: SparkConf, creds: Credentials): Option[Long] = { + try { + val fileSystems = HadoopFSDelegationTokenProvider.hadoopFSsToAccess(sparkConf, hadoopConf) + val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fileSystems, creds) - val fsToGetTokens = fileSystems(hadoopConf) - val fetchCreds = fetchDelegationTokens(getTokenRenewer(hadoopConf), fsToGetTokens, creds) + // Get the token renewal interval if it is not set. It will only be called once. + if (tokenRenewalInterval == null) { + tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fileSystems) + } - // Get the token renewal interval if it is not set. It will only be called once. - if (tokenRenewalInterval == null) { - tokenRenewalInterval = getTokenRenewalInterval(hadoopConf, sparkConf, fsToGetTokens) - } + // Get the time of next renewal. + val nextRenewalDate = tokenRenewalInterval.flatMap { interval => + val nextRenewalDates = fetchCreds.getAllTokens.asScala + .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) + .map { token => + val identifier = token + .decodeIdentifier() + .asInstanceOf[AbstractDelegationTokenIdentifier] + identifier.getIssueDate + interval + } + if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) + } - // Get the time of next renewal. - val nextRenewalDate = tokenRenewalInterval.flatMap { interval => - val nextRenewalDates = fetchCreds.getAllTokens.asScala - .filter(_.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier]) - .map { token => - val identifier = token - .decodeIdentifier() - .asInstanceOf[AbstractDelegationTokenIdentifier] - identifier.getIssueDate + interval - } - if (nextRenewalDates.isEmpty) None else Some(nextRenewalDates.min) + nextRenewalDate + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token from service $serviceName", e) + None } - - nextRenewalDate } override def delegationTokensRequired( @@ -94,7 +101,7 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration creds: Credentials): Credentials = { filesystems.foreach { fs => - logInfo("getting token for: " + fs) + logInfo(s"getting token for: $fs with renewer $renewer") fs.addDelegationTokens(renewer, creds) } @@ -108,22 +115,63 @@ private[deploy] class HadoopFSDelegationTokenProvider(fileSystems: Configuration // We cannot use the tokens generated with renewer yarn. Trying to renew // those will fail with an access control issue. So create new tokens with the logged in // user as renewer. - sparkConf.get(PRINCIPAL).flatMap { renewer => - val creds = new Credentials() - fetchDelegationTokens(renewer, filesystems, creds) - - val renewIntervals = creds.getAllTokens.asScala.filter { - _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] - }.flatMap { token => - Try { - val newExpiration = token.renew(hadoopConf) - val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] - val interval = newExpiration - identifier.getIssueDate - logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") - interval - }.toOption + val renewer = UserGroupInformation.getCurrentUser().getUserName() + + val creds = new Credentials() + fetchDelegationTokens(renewer, filesystems, creds) + + val renewIntervals = creds.getAllTokens.asScala.filter { + _.decodeIdentifier().isInstanceOf[AbstractDelegationTokenIdentifier] + }.flatMap { token => + Try { + val newExpiration = token.renew(hadoopConf) + val identifier = token.decodeIdentifier().asInstanceOf[AbstractDelegationTokenIdentifier] + val interval = newExpiration - identifier.getIssueDate + logInfo(s"Renewal interval is $interval for token ${token.getKind.toString}") + interval + }.toOption + } + if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + } +} + +private[deploy] object HadoopFSDelegationTokenProvider { + def hadoopFSsToAccess( + sparkConf: SparkConf, + hadoopConf: Configuration): Set[FileSystem] = { + val filesystemsToAccess = sparkConf.get(KERBEROS_FILESYSTEMS_TO_ACCESS) + + val defaultFS = FileSystem.get(hadoopConf) + val master = sparkConf.get("spark.master", null) + val stagingFS = if (master != null && master.contains("yarn")) { + sparkConf.get(STAGING_DIR).map(new Path(_).getFileSystem(hadoopConf)) + } else { + None + } + + // Add the list of available namenodes for all namespaces in HDFS federation. + // If ViewFS is enabled, this is skipped as ViewFS already handles delegation tokens for its + // namespaces. + val hadoopFilesystems = if (!filesystemsToAccess.isEmpty || defaultFS.getScheme == "viewfs" || + (stagingFS.isDefined && stagingFS.get.getScheme == "viewfs")) { + filesystemsToAccess.map(new Path(_).getFileSystem(hadoopConf)).toSet + } else { + val nameservices = hadoopConf.getTrimmedStrings("dfs.nameservices") + // Retrieving the filesystem for the nameservices where HA is not enabled + val filesystemsWithoutHA = nameservices.flatMap { ns => + Option(hadoopConf.get(s"dfs.namenode.rpc-address.$ns")).map { nameNode => + new Path(s"hdfs://$nameNode").getFileSystem(hadoopConf) + } + } + // Retrieving the filesystem for the nameservices where HA is enabled + val filesystemsWithHA = nameservices.flatMap { ns => + Option(hadoopConf.get(s"dfs.ha.namenodes.$ns")).map { _ => + new Path(s"hdfs://$ns").getFileSystem(hadoopConf) + } } - if (renewIntervals.isEmpty) None else Some(renewIntervals.min) + (filesystemsWithoutHA ++ filesystemsWithHA).toSet } + + hadoopFilesystems ++ stagingFS + defaultFS } } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/README.md b/core/src/main/scala/org/apache/spark/deploy/security/README.md new file mode 100644 index 0000000000000..0d98ce43919e6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/security/README.md @@ -0,0 +1,267 @@ +--- +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. +--- + +# Delegation Token Handling In Spark + +This document aims to explain and demystify delegation tokens as they are used by Spark, since +this topic is generally a huge source of confusion. + + +## What are delegation tokens and why use them? + +Delegation tokens (DTs from now on) are authentication tokens used by some services to replace +Kerberos service tokens. Many services in the Hadoop ecosystem have support for DTs, since they +have some very desirable advantages over Kerberos tokens: + +* No need to distribute Kerberos credentials + +In a distributed application, distributing Kerberos credentials is tricky. Not all users have +keytabs, and when they do, it's generally frowned upon to distribute them over the network as +part of application data. + +DTs allow for a single place (e.g. the Spark driver) to require Kerberos credentials. That entity +can then distribute the DTs to other parts of the distributed application (e.g. Spark executors), +so they can authenticate to services. + +* A single token per service is used for authentication + +If Kerberos authentication were used, each client connection to a server would require a trip +to the KDC and generation of a service ticket. In a distributed system, the number of service +tickets can balloon pretty quickly when you think about the number of client processes (e.g. Spark +executors) vs. the number of service processes (e.g. HDFS DataNodes). That generates unnecessary +extra load on the KDC, and may even run into usage limits set up by the KDC admin. + +* DTs are only used for authentication + +DTs, unlike TGTs, can only be used to authenticate to the specific service for which they were +issued. You cannot use an existing DT to create new DTs or to create DTs for a different service. + +So in short, DTs are *not* Kerberos tokens. They are used by many services to replace Kerberos +authentication, or even other forms of authentication, although there is nothing (aside from +maybe implementation details) that ties them to Kerberos or any other authentication mechanism. + + +## Lifecycle of DTs + +DTs, unlike Kerberos tokens, are service-specific. There is no centralized location you contact +to create a DT for a service. So, the first step needed to get a DT is being able to authenticate +to the service in question. In the Hadoop ecosystem, that is generally done using Kerberos. + +This requires Kerberos credentials to be available somewhere for the application to use. The user +is generally responsible for providing those credentials, which is most commonly done by logging +in to the KDC (e.g. using "kinit"). That generates a (Kerberos) "token cache" containing a TGT +(ticket granting ticket), which can then be used to request service tickets. + +There are other ways of obtaining TGTs, but, ultimately, you need a TGT to bootstrap the process. + +Once a TGT is available, the target service's client library can then be used to authenticate +to the service using the Kerberos credentials, and request the creation of a delegation token. +This token can now be sent to other processes and used to authenticate to different daemons +belonging to that service. + +And thus the first drawback of DTs becomes apparent: you need service-specific logic to create and +use them. + +Spark implements a (somewhat) pluggable, internal DT creation API. Support for new services can be +added by implementing a `HadoopDelegationTokenProvider` that is then called by Spark when generating +delegation tokens for an application. Spark makes the DTs available to code by stashing them in the +`UserGroupInformation` credentials, and it's up to the DT provider and the respective client library +to agree on how to use those tokens. + +Once they are created, the semantics of how DTs operate are also service-specific. But, in general, +they try to follow the semantics of Kerberos tokens: + +* A "renewable period (equivalent to TGT's "lifetime") which is for how long the DT is valid + before it requires renewal. +* A "max lifetime" (equivalent to TGT's "renewable life") which is for how long the DT can be + renewed. + +Once the token reaches its "max lifetime", a new one needs to be created by contacting the +appropriate service, restarting the above process. + + +## DT Renewal, Renewers, and YARN + +This is the most confusing part of DT handling, and part of it is because much of the system was +designed with MapReduce, and later YARN, in mind. + +As seen above, DTs need to be renewed periodically until they finally expire for good. An example of +this is the default configuration of HDFS services: delegation tokens are valid for up to 7 days, +and need to be renewed every 24 hours. If 24 hours pass without the token being renewed, the token +cannot be used anymore. And the token cannot be renewed anymore after 7 days. + +This raises the question: who renews tokens? And for a long time the answer was YARN. + +When YARN applications are submitted, a set of DTs is also submitted with them. YARN takes care +of distributing these tokens to containers (using conventions set by the `UserGroupInformation` +API) and, also, keeping them renewed while the app is running. These tokens are used not just +by the application; they are also used by YARN itself to implement features like log collection +and aggregation. + +But this has a few caveats. + + +1. Who renews the tokens? + +This is handled mostly transparently by the Hadoop libraries in the case of YARN. Some services have +the concept of a token "renewer". This "renewer" is the name of the principal that is allowed to +renew the DT. When submitting to YARN, that will be the principal that the YARN service is running +as, which means that the client application needs to know that information. + +For other resource managers, the renewer mostly does not matter, since there is no service that +is doing the renewal. Except that it sometimes leaks into library code, such as in SPARK-20328. + + +2. What tokens are renewed? + +This is probably the biggest caveat. + +As discussed in the previous section, DTs are service-specific, and require service-specific +libraries for creation *and* renewal. This means that for YARN to be able to renew application +tokens, YARN needs: + +* The client libraries for all the services the application is using +* Information about how to connect to the services the application is using +* Permissions to connect to those services + +In reality, though, most of the time YARN has access to a single HDFS cluster, and that will be +the extent of its DT renewal features. Any other tokens sent to YARN will be distributed to +containers, but will not be renewed. + +This means that those tokens will expire way before their max lifetime, unless some other code +takes care of renewing them. + +Also, not all client libraries even implement token renewal. To use the example of a service +supported by Spark, the `renew()` method of HBase tokens is a no-op. So the only way to "renew" an +HBase token is to create a new one. + + +3. What happens when tokens expire for good? + +The final caveat is that DTs have a maximum life, regardless of renewal. And after that deadline +is met, you need to create new tokens to be able to connect to the services. That means you need +the ability to connect to the service without a delegation token, which requires some form of +authentication aside from DTs. + +This is especially important for long-running applications that run unsupervised. They must be +able to keep on going without having someone logging into a terminal and typing a password every +few days. + + +## DT Renewal in Spark + +Because of the issues explained above, Spark implements a different way of doing renewal. Spark's +solution is a compromise: it targets the lowest common denominator, which is services like HBase +that do not support actual token renewal. + +In Spark, DT "renewal" is enabled by giving the application a Kerberos keytab. A keytab is +basically your Kerberos password written into a plain text file, which is why it's so sensitive: +if anyone is able to get hold of that keytab file, they'll be able to authenticate to any service +as that user, for as long as the credentials stored in the keytab remain valid in the KDC. + +By having the keytab, Spark can indefinitely maintain a valid Kerberos TGT. + +With Kerberos credentials available, Spark will create new DTs for the configured services as old +ones expire. So Spark doesn't renew tokens as explained in the previous section: it will create new +tokens at every renewal interval instead, and distribute those tokens to executors. + +This also has another advantage on top of supporting services like HBase: it removes the dependency +on an external renewal service (like YARN). That way, Spark's renewal feature can be used with +resource managers that are not DT-aware, such as Mesos or Kubernetes, as long as the application +has access to a keytab. + + +## DTs and Proxy Users + +"Proxy users" is Hadoop-speak for impersonation. It allows user A to impersonate user B when +connecting to a service, if that service allows it. + +Spark allows impersonation when submitting applications, so that the whole application runs as +user B in the above example. + +Spark does not allow token renewal when impersonation is on. Impersonation was added in Spark +as a means for services (like Hive or Oozie) to start Spark applications on behalf of users. +That means that those services would provide the Spark launcher code with privileged credentials +and, potentially, user code that will run when the application starts. The user code is not +necessarily under control of the service. + +In that situation, the service credentials should never be made available to the Spark application, +since that would be tantamount to giving your service credentials to unprivileged users. + +The above also implies that running impersonated applications in client mode can be a security +concern, since arbitrary user code would have access to the same local content as the privileged +user. But unlike token renewal, Spark does not prevent that configuration from running. + +When impersonating, the Spark launcher will create DTs for the "proxy" user. In the example +used above, that means that when code authenticates to a service using the DTs, the authenticated +user will be "B", not "A". + +Note that "proxy user" is a very Hadoop-specific concept. It does not apply to OS users (which +is why the client-mode case is an issue) and to services that do not authenticate using Hadoop's +`UserGroupInformation` system. It is generally used in the context of YARN - since an application +submitted as a proxy user will run as that particular user in the YARN cluster, obeying any +Hadoop-to-local-OS-user mapping configured for the service. But the overall support should work +for connecting to other services even when YARN is not being used. + +Also, if writing a new DT provider in Spark, be aware that providers need to explicitly handle +impersonation. If a service does not support impersonation, the provider should either error out or +not generate tokens, depending on what makes more sense in the context. + + +## Externally Generated DTs + +Spark uses the `UserGroupInformation` API to manage the Hadoop credentials. That means that Spark +inherits the feature of loading DTs automatically from a file. The Hadoop classes will load the +token cache pointed at by the `HADOOP_TOKEN_FILE_LOCATION` environment variable, when it's defined. + +In this situation, Spark will not create DTs for the services that already have tokens in the +cache. It may try to get delegation tokens for other services if Kerberos credentials are also +provided. + +This feature is mostly used by services that start Spark on behalf of users. Regular users do not +generally use this feature, given it would require them to figure out how to get those tokens +outside of Spark. + + +## Limitations of DT support in Spark + +There are certain limitations to bear in mind when talking about DTs in Spark. + +The first one is that not all DTs actually expose their renewal period. This is generally a +service configuration that is not generally exposed to clients. For this reason, certain DT +providers cannot provide a renewal period to the Spark code, thus requiring that the service's +configuration is in some way synchronized with another one that does provide that information. + +The HDFS service, which is generally available when DTs are needed in the first place, provides +that information, so in general it's a good idea for all services using DTs to use the same +configuration as HDFS for the renewal period. + +The second one is that Spark doesn't always know what delegation tokens will be needed. For +example, when submitting an application in cluster mode without a keytab, the launcher needs +to create DTs without knowing what the application code will actually be doing. This means that +Spark will try to get as many delegation tokens as is possible based on the configuration +available. That means that if an HBase configuration is available to the launcher but the app +doesn't actually use HBase, a DT will still be generated. The user would have to explicitly +opt out of generating HBase tokens in that case. + +The third one is that it's hard to create DTs "as needed". Without being able to authenticate +to specific services, Spark cannot create DTs, which means that applications submitted in cluster +mode like the above need DTs to be created up front, instead of on demand. + +The advantage, though, is that user code does not need to worry about DTs, since Spark will handle +them transparently when the proper configuration is available. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index a6d13d12fc28d..0c88119441ad3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -31,6 +31,7 @@ import org.apache.spark.deploy.DeployMessages.DriverStateChanged import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Worker.WORKER_DRIVER_TERMINATE_TIMEOUT import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{Clock, ShutdownHookManager, SystemClock, Utils} @@ -57,8 +58,7 @@ private[deploy] class DriverRunner( @volatile private[worker] var finalException: Option[Exception] = None // Timeout to wait for when trying to terminate a driver. - private val DRIVER_TERMINATE_TIMEOUT_MS = - conf.getTimeAsMs("spark.worker.driverTerminateTimeout", "10s") + private val driverTerminateTimeoutMs = conf.get(WORKER_DRIVER_TERMINATE_TIMEOUT) // Decoupled for testing def setClock(_clock: Clock): Unit = { @@ -122,7 +122,7 @@ private[deploy] class DriverRunner( killed = true synchronized { process.foreach { p => - val exitCode = Utils.terminateProcess(p, DRIVER_TERMINATE_TIMEOUT_MS) + val exitCode = Utils.terminateProcess(p, driverTerminateTimeoutMs) if (exitCode.isEmpty) { logWarning("Failed to terminate driver process: " + p + ". This process will likely be orphaned.") @@ -193,8 +193,9 @@ private[deploy] class DriverRunner( CommandUtils.redirectStream(process.getInputStream, stdout) val stderr = new File(baseDir, "stderr") - val formattedCommand = builder.command.asScala.mkString("\"", "\" \"", "\"") - val header = "Launch Command: %s\n%s\n\n".format(formattedCommand, "=" * 40) + val redactedCommand = Utils.redactCommandLineArgs(conf, builder.command.asScala) + .mkString("\"", "\" \"", "\"") + val header = "Launch Command: %s\n%s\n\n".format(redactedCommand, "=" * 40) Files.append(header, stderr, StandardCharsets.UTF_8) CommandUtils.redirectStream(process.getErrorStream, stderr) } @@ -210,8 +211,10 @@ private[deploy] class DriverRunner( val successfulRunDuration = 5 var keepTrying = !killed + val redactedCommand = Utils.redactCommandLineArgs(conf, command.command) + .mkString("\"", "\" \"", "\"") while (keepTrying) { - logInfo("Launch Command: " + command.command.mkString("\"", "\" \"", "\"")) + logInfo("Launch Command: " + redactedCommand) synchronized { if (killed) { return exitCode } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index 8d6a2b80ef5f2..56356f5f27e27 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -22,8 +22,8 @@ import java.io.File import org.apache.commons.lang3.StringUtils import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil, SparkSubmit} -import org.apache.spark.internal.Logging +import org.apache.spark.deploy.{DependencyUtils, SparkHadoopUtil} +import org.apache.spark.internal.{config, Logging} import org.apache.spark.rpc.RpcEnv import org.apache.spark.util._ @@ -43,7 +43,7 @@ object DriverWrapper extends Logging { case workerUrl :: userJar :: mainClass :: extraArgs => val conf = new SparkConf() val host: String = Utils.localHostName() - val port: Int = sys.props.getOrElse("spark.driver.port", "0").toInt + val port: Int = sys.props.getOrElse(config.DRIVER_PORT.key, "0").toInt val rpcEnv = RpcEnv.create("Driver", host, port, conf, new SecurityManager(conf)) logInfo(s"Driver address: ${rpcEnv.address}") rpcEnv.setupEndpoint("workerWatcher", new WorkerWatcher(rpcEnv, workerUrl)) @@ -51,7 +51,7 @@ object DriverWrapper extends Logging { val currentLoader = Thread.currentThread.getContextClassLoader val userJarUrl = new File(userJar).toURI().toURL() val loader = - if (sys.props.getOrElse("spark.driver.userClassPathFirst", "false").toBoolean) { + if (sys.props.getOrElse(config.DRIVER_USER_CLASS_PATH_FIRST.key, "false").toBoolean) { new ChildFirstURLClassLoader(Array(userJarUrl), currentLoader) } else { new MutableURLClassLoader(Array(userJarUrl), currentLoader) @@ -91,7 +91,7 @@ object DriverWrapper extends Logging { val resolvedMavenCoordinates = DependencyUtils.resolveMavenDependencies(packagesExclusions, packages, repositories, ivyRepoPath, Option(ivySettingsPath)) val jars = { - val jarsProp = sys.props.get("spark.jars").orNull + val jarsProp = sys.props.get(config.JARS.key).orNull if (!StringUtils.isBlank(resolvedMavenCoordinates)) { DependencyUtils.mergeFileLists(jarsProp, resolvedMavenCoordinates) } else { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index dc6a3076a5113..6f1484cee586e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -25,9 +25,10 @@ import scala.collection.JavaConverters._ import com.google.common.io.Files import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.{ApplicationDescription, Command, ExecutorState} +import org.apache.spark.deploy.{ApplicationDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages.ExecutorStateChanged import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.util.{ShutdownHookManager, Utils} import org.apache.spark.util.logging.FileAppender @@ -44,6 +45,7 @@ private[deploy] class ExecutorRunner( val memory: Int, val worker: RpcEndpointRef, val workerId: String, + val webUiScheme: String, val host: String, val webUiPort: Int, val publicAddress: String, @@ -76,8 +78,8 @@ private[deploy] class ExecutorRunner( // Shutdown hook that kills actors on shutdown. shutdownHook = ShutdownHookManager.addShutdownHook { () => // It's possible that we arrive here before calling `fetchAndRunExecutor`, then `state` will - // be `ExecutorState.RUNNING`. In this case, we should set `state` to `FAILED`. - if (state == ExecutorState.RUNNING) { + // be `ExecutorState.LAUNCHING`. In this case, we should set `state` to `FAILED`. + if (state == ExecutorState.LAUNCHING) { state = ExecutorState.FAILED } killProcess(Some("Worker shutting down")) } @@ -149,8 +151,9 @@ private[deploy] class ExecutorRunner( val builder = CommandUtils.buildProcessBuilder(subsCommand, new SecurityManager(conf), memory, sparkHome.getAbsolutePath, substituteVariables) val command = builder.command() - val formattedCommand = command.asScala.mkString("\"", "\" \"", "\"") - logInfo(s"Launch command: $formattedCommand") + val redactedCommand = Utils.redactCommandLineArgs(conf, command.asScala) + .mkString("\"", "\" \"", "\"") + logInfo(s"Launch command: $redactedCommand") builder.directory(executorDir) builder.environment.put("SPARK_EXECUTOR_DIRS", appLocalDirs.mkString(File.pathSeparator)) @@ -160,17 +163,17 @@ private[deploy] class ExecutorRunner( // Add webUI log urls val baseUrl = - if (conf.getBoolean("spark.ui.reverseProxy", false)) { + if (conf.get(UI_REVERSE_PROXY)) { s"/proxy/$workerId/logPage/?appId=$appId&executorId=$execId&logType=" } else { - s"http://$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" + s"$webUiScheme$publicAddress:$webUiPort/logPage/?appId=$appId&executorId=$execId&logType=" } builder.environment.put("SPARK_LOG_URL_STDERR", s"${baseUrl}stderr") builder.environment.put("SPARK_LOG_URL_STDOUT", s"${baseUrl}stdout") process = builder.start() val header = "Spark Executor Command: %s\n%s\n\n".format( - formattedCommand, "=" * 40) + redactedCommand, "=" * 40) // Redirect its stdout and stderr to files val stdout = new File(executorDir, "stdout") @@ -180,6 +183,8 @@ private[deploy] class ExecutorRunner( Files.write(header, stderr, StandardCharsets.UTF_8) stderrAppender = FileAppender(process.getErrorStream, stderr, conf) + state = ExecutorState.RUNNING + worker.send(ExecutorStateChanged(appId, execId, state, None, None)) // Wait for it to exit; executor may exit with code 0 (when driver instructs it to shutdown) // or with nonzero exit code val exitCode = process.waitFor() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index d5ea2523c628b..ac7a1b91db6b9 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -37,7 +37,12 @@ import org.apache.spark.deploy.ExternalShuffleService import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.deploy.worker.ui.WorkerWebUI import org.apache.spark.internal.{config, Logging} -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ import org.apache.spark.util.{SparkUncaughtExceptionHandler, ThreadUtils, Utils} @@ -51,6 +56,7 @@ private[deploy] class Worker( workDirPath: String = null, val conf: SparkConf, val securityMgr: SecurityManager, + resourceFileOpt: Option[String] = None, externalShuffleServiceSupplier: Supplier[ExternalShuffleService] = null) extends ThreadSafeRpcEndpoint with Logging { @@ -61,7 +67,7 @@ private[deploy] class Worker( assert (port > 0) // A scheduled executor used to send messages at the specified time. - private val forwordMessageScheduler = + private val forwardMessageScheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("worker-forward-message-scheduler") // A separated thread to clean up the workDir and the directories of finished applications. @@ -72,7 +78,7 @@ private[deploy] class Worker( // For worker and executor IDs private def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss", Locale.US) // Send a heartbeat every (heartbeat timeout) / 4 milliseconds - private val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 + private val HEARTBEAT_MILLIS = conf.get(WORKER_TIMEOUT) * 1000 / 4 // Model retries to connect to the master, after Hadoop's model. // The first six attempts to reconnect are in shorter intervals (between 5 and 15 seconds) @@ -91,27 +97,23 @@ private[deploy] class Worker( private val PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS = (math.round(60 * REGISTRATION_RETRY_FUZZ_MULTIPLIER)) - private val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", false) + private val CLEANUP_ENABLED = conf.get(WORKER_CLEANUP_ENABLED) // How often worker will clean up old app folders - private val CLEANUP_INTERVAL_MILLIS = - conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + private val CLEANUP_INTERVAL_MILLIS = conf.get(WORKER_CLEANUP_INTERVAL) * 1000 // TTL for app folders/data; after TTL expires it will be cleaned up - private val APP_DATA_RETENTION_SECONDS = - conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + private val APP_DATA_RETENTION_SECONDS = conf.get(APP_DATA_RETENTION) - // Whether or not cleanup the non-shuffle files on executor exits. - private val CLEANUP_NON_SHUFFLE_FILES_ENABLED = - conf.getBoolean("spark.storage.cleanupFilesAfterExecutorExit", true) + // Whether or not cleanup the non-shuffle service served files on executor exits. + private val CLEANUP_FILES_AFTER_EXECUTOR_EXIT = + conf.get(config.STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT) - private val testing: Boolean = sys.props.contains("spark.testing") private var master: Option[RpcEndpointRef] = None /** * Whether to use the master address in `masterRpcAddresses` if possible. If it's disabled, Worker * will just use the address received from Master. */ - private val preferConfiguredMasterAddress = - conf.getBoolean("spark.worker.preferConfiguredMasterAddress", false) + private val preferConfiguredMasterAddress = conf.get(PREFER_CONFIGURED_MASTER_ADDRESS) /** * The master address to connect in case of failure. When the connection is broken, worker will * use this address to connect. This is usually just one of `masterRpcAddresses`. However, when @@ -127,11 +129,11 @@ private[deploy] class Worker( private var connected = false private val workerId = generateWorkerId() private val sparkHome = - if (testing) { + if (sys.props.contains(IS_TESTING.key)) { assert(sys.props.contains("spark.test.home"), "spark.test.home is not set!") new File(sys.props("spark.test.home")) } else { - new File(sys.env.get("SPARK_HOME").getOrElse(".")) + new File(sys.env.getOrElse("SPARK_HOME", ".")) } var workDir: File = null @@ -142,10 +144,8 @@ private[deploy] class Worker( val appDirectories = new HashMap[String, Seq[String]] val finishedApps = new HashSet[String] - val retainedExecutors = conf.getInt("spark.worker.ui.retainedExecutors", - WorkerWebUI.DEFAULT_RETAINED_EXECUTORS) - val retainedDrivers = conf.getInt("spark.worker.ui.retainedDrivers", - WorkerWebUI.DEFAULT_RETAINED_DRIVERS) + val retainedExecutors = conf.get(WORKER_UI_RETAINED_EXECUTORS) + val retainedDrivers = conf.get(WORKER_UI_RETAINED_DRIVERS) // The shuffle service is not actually started unless configured. private val shuffleService = if (externalShuffleServiceSupplier != null) { @@ -162,10 +162,11 @@ private[deploy] class Worker( private var connectionAttemptCount = 0 - private val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) + private val metricsSystem = + MetricsSystem.createMetricsSystem(MetricsSystemInstances.WORKER, conf, securityMgr) private val workerSource = new WorkerSource(this) - val reverseProxy = conf.getBoolean("spark.ui.reverseProxy", false) + val reverseProxy = conf.get(UI_REVERSE_PROXY) private var registerMasterFutures: Array[JFuture[_]] = null private var registrationRetryTimer: Option[JScheduledFuture[_]] = None @@ -178,6 +179,9 @@ private[deploy] class Worker( masterRpcAddresses.length // Make sure we can register with all masters at the same time ) + // visible for tests + private[deploy] var resources: Map[String, ResourceInformation] = _ + var coresUsed = 0 var memoryUsed = 0 @@ -210,10 +214,11 @@ private[deploy] class Worker( logInfo("Spark home: " + sparkHome) createWorkDir() startExternalShuffleService() + setupWorkerResources() webUi = new WorkerWebUI(this, workDir, webUiPort) webUi.bind() - workerWebUiUrl = s"http://$publicAddress:${webUi.boundPort}" + workerWebUiUrl = s"${webUi.scheme}$publicAddress:${webUi.boundPort}" registerWithMaster() metricsSystem.registerSource(workerSource) @@ -222,6 +227,16 @@ private[deploy] class Worker( metricsSystem.getServletHandlers.foreach(webUi.attachHandler) } + private def setupWorkerResources(): Unit = { + try { + resources = getOrDiscoverAllResources(conf, SPARK_WORKER_PREFIX, resourceFileOpt) + } catch { + case e: Exception => + logError("Failed to setup worker resources: ", e) + System.exit(1) + } + } + /** * Change to use the new master. * @@ -327,11 +342,9 @@ private[deploy] class Worker( if (connectionAttemptCount == INITIAL_REGISTRATION_RETRIES) { registrationRetryTimer.foreach(_.cancel(true)) registrationRetryTimer = Some( - forwordMessageScheduler.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - self.send(ReregisterWithMaster) - } - }, PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS, + forwardMessageScheduler.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { self.send(ReregisterWithMaster) }, + PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS, PROLONGED_REGISTRATION_RETRY_INTERVAL_SECONDS, TimeUnit.SECONDS)) } @@ -343,7 +356,7 @@ private[deploy] class Worker( } /** - * Cancel last registeration retry, or do nothing if no retry + * Cancel last registration retry, or do nothing if no retry */ private def cancelLastRegistrationRetry(): Unit = { if (registerMasterFutures != null) { @@ -362,12 +375,8 @@ private[deploy] class Worker( registered = false registerMasterFutures = tryRegisterAllMasters() connectionAttemptCount = 0 - registrationRetryTimer = Some(forwordMessageScheduler.scheduleAtFixedRate( - new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - Option(self).foreach(_.send(ReregisterWithMaster)) - } - }, + registrationRetryTimer = Some(forwardMessageScheduler.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { Option(self).foreach(_.send(ReregisterWithMaster)) }, INITIAL_REGISTRATION_RETRY_INTERVAL_SECONDS, INITIAL_REGISTRATION_RETRY_INTERVAL_SECONDS, TimeUnit.SECONDS)) @@ -401,27 +410,32 @@ private[deploy] class Worker( private def handleRegisterResponse(msg: RegisterWorkerResponse): Unit = synchronized { msg match { - case RegisteredWorker(masterRef, masterWebUiUrl, masterAddress) => - if (preferConfiguredMasterAddress) { - logInfo("Successfully registered with master " + masterAddress.toSparkURL) + case RegisteredWorker(masterRef, masterWebUiUrl, masterAddress, duplicate) => + val preferredMasterAddress = if (preferConfiguredMasterAddress) { + masterAddress.toSparkURL } else { - logInfo("Successfully registered with master " + masterRef.address.toSparkURL) + masterRef.address.toSparkURL } + + // there're corner cases which we could hardly avoid duplicate worker registration, + // e.g. Master disconnect(maybe due to network drop) and recover immediately, see + // SPARK-23191 for more details. + if (duplicate) { + logWarning(s"Duplicate registration at master $preferredMasterAddress") + } + + logInfo(s"Successfully registered with master $preferredMasterAddress") registered = true changeMaster(masterRef, masterWebUiUrl, masterAddress) - forwordMessageScheduler.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - self.send(SendHeartbeat) - } - }, 0, HEARTBEAT_MILLIS, TimeUnit.MILLISECONDS) + forwardMessageScheduler.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { self.send(SendHeartbeat) }, + 0, HEARTBEAT_MILLIS, TimeUnit.MILLISECONDS) if (CLEANUP_ENABLED) { logInfo( s"Worker cleanup enabled; old application directories will be deleted in: $workDir") - forwordMessageScheduler.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - self.send(WorkDirCleanup) - } - }, CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) + forwardMessageScheduler.scheduleAtFixedRate( + () => Utils.tryLogNonFatalError { self.send(WorkDirCleanup) }, + CLEANUP_INTERVAL_MILLIS, CLEANUP_INTERVAL_MILLIS, TimeUnit.MILLISECONDS) } val execs = executors.values.map { e => @@ -452,27 +466,41 @@ private[deploy] class Worker( // rpcEndpoint. // Copy ids so that it can be used in the cleanup thread. val appIds = (executors.values.map(_.appId) ++ drivers.values.map(_.driverId)).toSet - val cleanupFuture = concurrent.Future { - val appDirs = workDir.listFiles() - if (appDirs == null) { - throw new IOException("ERROR: Failed to list files in " + appDirs) - } - appDirs.filter { dir => - // the directory is used by an application - check that the application is not running - // when cleaning up - val appIdFromDir = dir.getName - val isAppStillRunning = appIds.contains(appIdFromDir) - dir.isDirectory && !isAppStillRunning && - !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECONDS) - }.foreach { dir => - logInfo(s"Removing directory: ${dir.getPath}") - Utils.deleteRecursively(dir) - } - }(cleanupThreadExecutor) + try { + val cleanupFuture: concurrent.Future[Unit] = concurrent.Future { + val appDirs = workDir.listFiles() + if (appDirs == null) { + throw new IOException("ERROR: Failed to list files in " + appDirs) + } + appDirs.filter { dir => + // the directory is used by an application - check that the application is not running + // when cleaning up + val appIdFromDir = dir.getName + val isAppStillRunning = appIds.contains(appIdFromDir) + dir.isDirectory && !isAppStillRunning && + !Utils.doesDirectoryContainAnyNewFiles(dir, APP_DATA_RETENTION_SECONDS) + }.foreach { dir => + logInfo(s"Removing directory: ${dir.getPath}") + Utils.deleteRecursively(dir) + + // Remove some registeredExecutors information of DB in external shuffle service when + // #spark.shuffle.service.db.enabled=true, the one which comes to mind is, what happens + // if an application is stopped while the external shuffle service is down? + // So then it'll leave an entry in the DB and the entry should be removed. + if (conf.get(config.SHUFFLE_SERVICE_DB_ENABLED) && + conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + shuffleService.applicationRemoved(dir.getName) + } + } + }(cleanupThreadExecutor) - cleanupFuture.failed.foreach(e => - logError("App dir cleanup failed: " + e.getMessage, e) - )(cleanupThreadExecutor) + cleanupFuture.failed.foreach(e => + logError("App dir cleanup failed: " + e.getMessage, e) + )(cleanupThreadExecutor) + } catch { + case _: RejectedExecutionException if cleanupThreadExecutor.isShutdown => + logWarning("Failed to cleanup work dir as executor pool was shutdown") + } case MasterChanged(masterRef, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterRef.address.toSparkURL) @@ -530,6 +558,7 @@ private[deploy] class Worker( memory_, self, workerId, + webUi.scheme, host, webUi.boundPort, publicAddress, @@ -537,12 +566,12 @@ private[deploy] class Worker( executorDir, workerUri, conf, - appLocalDirs, ExecutorState.RUNNING) + appLocalDirs, + ExecutorState.LAUNCHING) executors(appId + "/" + execId) = manager manager.start() coresUsed += cores_ memoryUsed += memory_ - sendToMaster(ExecutorStateChanged(appId, execId, manager.state, None, None)) } catch { case e: Exception => logError(s"Failed to launch executor $appId/$execId for ${appDesc.name}.", e) @@ -555,7 +584,7 @@ private[deploy] class Worker( } } - case executorStateChanged @ ExecutorStateChanged(appId, execId, state, message, exitStatus) => + case executorStateChanged: ExecutorStateChanged => handleExecutorStateChanged(executorStateChanged) case KillExecutor(masterUrl, appId, execId) => @@ -619,7 +648,7 @@ private[deploy] class Worker( override def onDisconnected(remoteAddress: RpcAddress): Unit = { if (master.exists(_.address == remoteAddress) || - masterAddressToConnect.exists(_ == remoteAddress)) { + masterAddressToConnect.contains(remoteAddress)) { logInfo(s"$remoteAddress Disassociated !") masterDisconnected() } @@ -635,15 +664,20 @@ private[deploy] class Worker( val shouldCleanup = finishedApps.contains(id) && !executors.values.exists(_.appId == id) if (shouldCleanup) { finishedApps -= id - appDirectories.remove(id).foreach { dirList => - concurrent.Future { - logInfo(s"Cleaning up local directories for application $id") - dirList.foreach { dir => - Utils.deleteRecursively(new File(dir)) - } - }(cleanupThreadExecutor).failed.foreach(e => - logError(s"Clean up app dir $dirList failed: ${e.getMessage}", e) - )(cleanupThreadExecutor) + try { + appDirectories.remove(id).foreach { dirList => + concurrent.Future { + logInfo(s"Cleaning up local directories for application $id") + dirList.foreach { dir => + Utils.deleteRecursively(new File(dir)) + } + }(cleanupThreadExecutor).failed.foreach(e => + logError(s"Clean up app dir $dirList failed: ${e.getMessage}", e) + )(cleanupThreadExecutor) + } + } catch { + case _: RejectedExecutionException if cleanupThreadExecutor.isShutdown => + logWarning("Failed to cleanup application as executor pool was shutdown") } shuffleService.applicationRemoved(id) } @@ -670,7 +704,7 @@ private[deploy] class Worker( cleanupThreadExecutor.shutdownNow() metricsSystem.report() cancelLastRegistrationRetry() - forwordMessageScheduler.shutdownNow() + forwardMessageScheduler.shutdownNow() registerMasterThreadPool.shutdownNow() executors.values.foreach(_.kill()) drivers.values.foreach(_.kill()) @@ -742,7 +776,8 @@ private[deploy] class Worker( trimFinishedExecutorsIfNecessary() coresUsed -= executor.cores memoryUsed -= executor.memory - if (CLEANUP_NON_SHUFFLE_FILES_ENABLED) { + + if (CLEANUP_FILES_AFTER_EXECUTOR_EXIT) { shuffleService.executorRemoved(executorStateChanged.execId.toString, appId) } case None => @@ -767,7 +802,8 @@ private[deploy] object Worker extends Logging { val conf = new SparkConf val args = new WorkerArguments(argStrings, conf) val rpcEnv = startRpcEnvAndEndpoint(args.host, args.port, args.webUiPort, args.cores, - args.memory, args.masters, args.workDir, conf = conf) + args.memory, args.masters, args.workDir, conf = conf, + resourceFileOpt = conf.get(SPARK_WORKER_RESOURCE_FILE)) // With external shuffle service enabled, if we request to launch multiple workers on one host, // we can only successfully launch the first worker and the rest fails, because with the port // bound, we may launch no more than one external shuffle service on each host. @@ -791,15 +827,16 @@ private[deploy] object Worker extends Logging { masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None, - conf: SparkConf = new SparkConf): RpcEnv = { + conf: SparkConf = new SparkConf, + resourceFileOpt: Option[String] = None): RpcEnv = { // The LocalSparkCluster runs multiple local sparkWorkerX RPC Environments val systemName = SYSTEM_NAME + workerNumber.map(_.toString).getOrElse("") val securityMgr = new SecurityManager(conf) val rpcEnv = RpcEnv.create(systemName, host, port, conf, securityMgr) - val masterAddresses = masterUrls.map(RpcAddress.fromSparkURL(_)) + val masterAddresses = masterUrls.map(RpcAddress.fromSparkURL) rpcEnv.setupEndpoint(ENDPOINT_NAME, new Worker(rpcEnv, webUiPort, cores, memory, - masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr)) + masterAddresses, ENDPOINT_NAME, workDir, conf, securityMgr, resourceFileOpt)) rpcEnv } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 580281288b060..8c87708e960e6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -22,6 +22,7 @@ import java.lang.management.ManagementFactory import scala.annotation.tailrec import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Worker._ import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** @@ -59,9 +60,7 @@ private[worker] class WorkerArguments(args: Array[String], conf: SparkConf) { // This mutates the SparkConf, so all accesses to it must be made after this line propertiesFile = Utils.loadDefaultSparkProperties(conf, propertiesFile) - if (conf.contains("spark.worker.ui.port")) { - webUiPort = conf.get("spark.worker.ui.port").toInt - } + conf.get(WORKER_UI_PORT).foreach { webUiPort = _ } checkWorkerMemory() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index 4fca9342c0378..4e720a759a1bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -33,15 +33,13 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with private val supportedLogTypes = Set("stderr", "stdout") private val defaultBytes = 100 * 1024 - // stripXSS is called first to remove suspicious characters used in XSS attacks def renderLog(request: HttpServletRequest): String = { - val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) - val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) - val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) - val logType = UIUtils.stripXSS(request.getParameter("logType")) - val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) - val byteLength = - Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) .getOrElse(defaultBytes) val logDir = (appId, executorId, driverId) match { @@ -58,15 +56,13 @@ private[ui] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") with pre + logText } - // stripXSS is called first to remove suspicious characters used in XSS attacks def render(request: HttpServletRequest): Seq[Node] = { - val appId = Option(UIUtils.stripXSS(request.getParameter("appId"))) - val executorId = Option(UIUtils.stripXSS(request.getParameter("executorId"))) - val driverId = Option(UIUtils.stripXSS(request.getParameter("driverId"))) - val logType = UIUtils.stripXSS(request.getParameter("logType")) - val offset = Option(UIUtils.stripXSS(request.getParameter("offset"))).map(_.toLong) - val byteLength = - Option(UIUtils.stripXSS(request.getParameter("byteLength"))).map(_.toInt) + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt) .getOrElse(defaultBytes) val (logDir, params, pageName) = (appId, executorId, driverId) match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index ea67b7434a769..96980c3ff0331 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -50,13 +50,10 @@ class WorkerWebUI( addStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE) attachHandler(createServletHandler("/log", (request: HttpServletRequest) => logPage.renderLog(request), - worker.securityMgr, worker.conf)) } } private[worker] object WorkerWebUI { val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_RETAINED_DRIVERS = 1000 - val DEFAULT_RETAINED_EXECUTORS = 1000 } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 48d3630abd1f9..98e5aa6ec0c7c 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -26,11 +26,16 @@ import scala.collection.mutable import scala.util.{Failure, Success} import scala.util.control.NonFatal +import org.json4s.DefaultFormats + import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.rpc._ import org.apache.spark.scheduler.{ExecutorLossReason, TaskDescription} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -44,9 +49,12 @@ private[spark] class CoarseGrainedExecutorBackend( hostname: String, cores: Int, userClassPath: Seq[URL], - env: SparkEnv) + env: SparkEnv, + resourcesFileOpt: Option[String]) extends ThreadSafeRpcEndpoint with ExecutorBackend with Logging { + private implicit val formats = DefaultFormats + private[this] val stopping = new AtomicBoolean(false) var executor: Executor = null @volatile var driver: Option[RpcEndpointRef] = None @@ -55,12 +63,21 @@ private[spark] class CoarseGrainedExecutorBackend( // to be changed so that we don't share the serializer instance across threads private[this] val ser: SerializerInstance = env.closureSerializer.newInstance() + /** + * Map each taskId to the information about the resource allocated to it, Please refer to + * [[ResourceInformation]] for specifics. + * Exposed for testing only. + */ + private[executor] val taskResources = new mutable.HashMap[Long, Map[String, ResourceInformation]] + override def onStart() { logInfo("Connecting to driver: " + driverUrl) + val resources = parseOrFindResources(resourcesFileOpt) rpcEnv.asyncSetupEndpointRefByURI(driverUrl).flatMap { ref => // This is a very fast action so we can use "ThreadUtils.sameThread" driver = Some(ref) - ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls)) + ref.ask[Boolean](RegisterExecutor(executorId, self, hostname, cores, extractLogUrls, + extractAttributes, resources)) }(ThreadUtils.sameThread).onComplete { // This is a very fast action so we can use "ThreadUtils.sameThread" case Success(msg) => @@ -70,12 +87,38 @@ private[spark] class CoarseGrainedExecutorBackend( }(ThreadUtils.sameThread) } + // visible for testing + def parseOrFindResources(resourcesFileOpt: Option[String]): Map[String, ResourceInformation] = { + // only parse the resources if a task requires them + val resourceInfo = if (parseTaskResourceRequirements(env.conf).nonEmpty) { + val resources = getOrDiscoverAllResources(env.conf, SPARK_EXECUTOR_PREFIX, resourcesFileOpt) + if (resources.isEmpty) { + throw new SparkException("User specified resources per task via: " + + s"$SPARK_TASK_PREFIX, but can't find any resources available on the executor.") + } + resources + } else { + if (resourcesFileOpt.nonEmpty) { + logWarning("A resources file was specified but the application is not configured " + + s"to use any resources, see the configs with prefix: ${SPARK_TASK_PREFIX}") + } + Map.empty[String, ResourceInformation] + } + resourceInfo + } + def extractLogUrls: Map[String, String] = { val prefix = "SPARK_LOG_URL_" sys.env.filterKeys(_.startsWith(prefix)) .map(e => (e._1.substring(prefix.length).toLowerCase(Locale.ROOT), e._2)) } + def extractAttributes: Map[String, String] = { + val prefix = "SPARK_EXECUTOR_ATTRIBUTE_" + sys.env.filterKeys(_.startsWith(prefix)) + .map(e => (e._1.substring(prefix.length).toUpperCase(Locale.ROOT), e._2)) + } + override def receive: PartialFunction[Any, Unit] = { case RegisteredExecutor => logInfo("Successfully registered with driver") @@ -95,6 +138,7 @@ private[spark] class CoarseGrainedExecutorBackend( } else { val taskDesc = TaskDescription.decode(data.value) logInfo("Got assigned task " + taskDesc.taskId) + taskResources(taskDesc.taskId) = taskDesc.resources executor.launchTask(this, taskDesc) } @@ -141,7 +185,11 @@ private[spark] class CoarseGrainedExecutorBackend( } override def statusUpdate(taskId: Long, state: TaskState, data: ByteBuffer) { - val msg = StatusUpdate(executorId, taskId, state, data) + val resources = taskResources.getOrElse(taskId, Map.empty[String, ResourceInformation]) + val msg = StatusUpdate(executorId, taskId, state, data, resources) + if (TaskState.isFinished(state)) { + taskResources.remove(taskId) + } driver match { case Some(driverRef) => driverRef.send(msg) case None => logWarning(s"Drop $msg because has not yet connected to driver") @@ -174,33 +222,61 @@ private[spark] class CoarseGrainedExecutorBackend( private[spark] object CoarseGrainedExecutorBackend extends Logging { - private def run( + case class Arguments( driverUrl: String, executorId: String, hostname: String, cores: Int, appId: String, workerUrl: Option[String], - userClassPath: Seq[URL]) { + userClassPath: mutable.ListBuffer[URL], + resourcesFileOpt: Option[String]) + + def main(args: Array[String]): Unit = { + val createFn: (RpcEnv, Arguments, SparkEnv) => + CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => + new CoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, + arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.resourcesFileOpt) + } + run(parseArguments(args, this.getClass.getCanonicalName.stripSuffix("$")), createFn) + System.exit(0) + } + + def run( + arguments: Arguments, + backendCreateFn: (RpcEnv, Arguments, SparkEnv) => CoarseGrainedExecutorBackend): Unit = { Utils.initDaemon(log) SparkHadoopUtil.get.runAsSparkUser { () => // Debug code - Utils.checkHost(hostname) + Utils.checkHost(arguments.hostname) // Bootstrap to fetch the driver's Spark properties. val executorConf = new SparkConf val fetcher = RpcEnv.create( "driverPropsFetcher", - hostname, + arguments.hostname, -1, executorConf, new SecurityManager(executorConf), clientMode = true) - val driver = fetcher.setupEndpointRefByURI(driverUrl) + + var driver: RpcEndpointRef = null + val nTries = 3 + for (i <- 0 until nTries if driver == null) { + try { + driver = fetcher.setupEndpointRefByURI(arguments.driverUrl) + } catch { + case e: Throwable => if (i == nTries - 1) { + throw e + } + } + } + val cfg = driver.askSync[SparkAppConfig](RetrieveSparkAppConfig) - val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", appId)) + val props = cfg.sparkProperties ++ Seq[(String, String)](("spark.app.id", arguments.appId)) fetcher.shutdown() // Create SparkEnv using properties we fetched from the driver. @@ -218,23 +294,24 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { SparkHadoopUtil.get.addDelegationTokens(tokens, driverConf) } - val env = SparkEnv.createExecutorEnv( - driverConf, executorId, hostname, cores, cfg.ioEncryptionKey, isLocal = false) + driverConf.set(EXECUTOR_ID, arguments.executorId) + val env = SparkEnv.createExecutorEnv(driverConf, arguments.executorId, arguments.hostname, + arguments.cores, cfg.ioEncryptionKey, isLocal = false) - env.rpcEnv.setupEndpoint("Executor", new CoarseGrainedExecutorBackend( - env.rpcEnv, driverUrl, executorId, hostname, cores, userClassPath, env)) - workerUrl.foreach { url => + env.rpcEnv.setupEndpoint("Executor", backendCreateFn(env.rpcEnv, arguments, env)) + arguments.workerUrl.foreach { url => env.rpcEnv.setupEndpoint("WorkerWatcher", new WorkerWatcher(env.rpcEnv, url)) } env.rpcEnv.awaitTermination() } } - def main(args: Array[String]) { + def parseArguments(args: Array[String], classNameForEntry: String): Arguments = { var driverUrl: String = null var executorId: String = null var hostname: String = null var cores: Int = 0 + var resourcesFileOpt: Option[String] = None var appId: String = null var workerUrl: Option[String] = None val userClassPath = new mutable.ListBuffer[URL]() @@ -254,6 +331,9 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { case ("--cores") :: value :: tail => cores = value.toInt argv = tail + case ("--resourcesFile") :: value :: tail => + resourcesFileOpt = Some(value) + argv = tail case ("--app-id") :: value :: tail => appId = value argv = tail @@ -269,30 +349,31 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // scalastyle:off println System.err.println(s"Unrecognized options: ${tail.mkString(" ")}") // scalastyle:on println - printUsageAndExit() + printUsageAndExit(classNameForEntry) } } if (driverUrl == null || executorId == null || hostname == null || cores <= 0 || appId == null) { - printUsageAndExit() + printUsageAndExit(classNameForEntry) } - run(driverUrl, executorId, hostname, cores, appId, workerUrl, userClassPath) - System.exit(0) + Arguments(driverUrl, executorId, hostname, cores, appId, workerUrl, + userClassPath, resourcesFileOpt) } - private def printUsageAndExit() = { + private def printUsageAndExit(classNameForEntry: String): Unit = { // scalastyle:off println System.err.println( - """ - |Usage: CoarseGrainedExecutorBackend [options] + s""" + |Usage: $classNameForEntry [options] | | Options are: | --driver-url | --executor-id | --hostname | --cores + | --resourcesFile | --app-id | --worker-url | --user-class-path @@ -300,5 +381,4 @@ private[spark] object CoarseGrainedExecutorBackend extends Logging { // scalastyle:on println System.exit(1) } - } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 61deb543d8747..2c035285c08ff 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -38,6 +38,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} +import org.apache.spark.metrics.source.JVMCPUSource import org.apache.spark.rpc.RpcTimeout import org.apache.spark.scheduler._ import org.apache.spark.shuffle.FetchFailedException @@ -88,17 +89,14 @@ private[spark] class Executor( } // Start worker thread pool + // Use UninterruptibleThread to run tasks so that we can allow running codes without being + // interrupted by `Thread.interrupt()`. Some issues, such as KAFKA-1894, HADOOP-10622, + // will hang forever if some methods are interrupted. private val threadPool = { val threadFactory = new ThreadFactoryBuilder() .setDaemon(true) .setNameFormat("Executor task launch worker-%d") - .setThreadFactory(new ThreadFactory { - override def newThread(r: Runnable): Thread = - // Use UninterruptibleThread to run tasks so that we can allow running codes without being - // interrupted by `Thread.interrupt()`. Some issues, such as KAFKA-1894, HADOOP-10622, - // will hang forever if some methods are interrupted. - new UninterruptibleThread(r, "unused") // thread name will be set by ThreadFactoryBuilder - }) + .setThreadFactory((r: Runnable) => new UninterruptibleThread(r, "unused")) .build() Executors.newCachedThreadPool(threadFactory).asInstanceOf[ThreadPoolExecutor] } @@ -117,6 +115,7 @@ private[spark] class Executor( if (!isLocal) { env.blockManager.initialize(conf.getAppId) env.metricsSystem.registerSource(executorSource) + env.metricsSystem.registerSource(new JVMCPUSource()) env.metricsSystem.registerSource(env.blockManager.shuffleMetricsSource) } @@ -124,7 +123,7 @@ private[spark] class Executor( private val userClassPathFirst = conf.get(EXECUTOR_USER_CLASS_PATH_FIRST) // Whether to monitor killed / interrupted tasks - private val taskReaperEnabled = conf.getBoolean("spark.task.reaper.enabled", false) + private val taskReaperEnabled = conf.get(TASK_REAPER_ENABLED) // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager @@ -163,7 +162,7 @@ private[spark] class Executor( // Max size of direct result. If task result is bigger than this, we use the block manager // to send the result back. private val maxDirectResultSize = Math.min( - conf.getSizeAsBytes("spark.task.maxDirectResultSize", 1L << 20), + conf.get(TASK_MAX_DIRECT_RESULT_SIZE), RpcUtils.maxMessageSizeBytes(conf)) private val maxResultSize = conf.get(MAX_RESULT_SIZE) @@ -173,8 +172,8 @@ private[spark] class Executor( /** * When an executor is unable to send heartbeats to the driver more than `HEARTBEAT_MAX_FAILURES` - * times, it should kill itself. The default value is 60. It means we will retry to send - * heartbeats about 10 minutes because the heartbeat interval is 10s. + * times, it should kill itself. The default value is 60. For example, if max failures is 60 and + * heartbeat interval is 10s, then it will try to send heartbeats for up to 600s (10 minutes). */ private val HEARTBEAT_MAX_FAILURES = conf.get(EXECUTOR_HEARTBEAT_MAX_FAILURES) @@ -190,8 +189,11 @@ private[spark] class Executor( private val HEARTBEAT_INTERVAL_MS = conf.get(EXECUTOR_HEARTBEAT_INTERVAL) // Executor for the heartbeat task. - private val heartbeater = new Heartbeater(env.memoryManager, reportHeartBeat, - "executor-heartbeater", HEARTBEAT_INTERVAL_MS) + private val heartbeater = new Heartbeater( + env.memoryManager, + () => Executor.this.reportHeartBeat(), + "executor-heartbeater", + HEARTBEAT_INTERVAL_MS) // must be initialized before running startDriverHeartbeat() private val heartbeatReceiverRef = @@ -345,10 +347,11 @@ private[spark] class Executor( * 2. Collect accumulator updates * 3. Set the finished flag to true and clear current thread's interrupt status */ - private def collectAccumulatorsAndResetStatusOnFailure(taskStartTime: Long) = { + private def collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs: Long) = { // Report executor runtime and JVM gc time Option(task).foreach(t => { - t.metrics.setExecutorRunTime(System.currentTimeMillis() - taskStartTime) + t.metrics.setExecutorRunTime(TimeUnit.NANOSECONDS.toMillis( + System.nanoTime() - taskStartTimeNs)) t.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) }) @@ -366,7 +369,7 @@ private[spark] class Executor( Thread.currentThread.setName(threadName) val threadMXBean = ManagementFactory.getThreadMXBean val taskMemoryManager = new TaskMemoryManager(env.memoryManager, taskId) - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTimeNs = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -374,7 +377,7 @@ private[spark] class Executor( val ser = env.closureSerializer.newInstance() logInfo(s"Running $taskName (TID $taskId)") execBackend.statusUpdate(taskId, TaskState.RUNNING, EMPTY_BYTE_BUFFER) - var taskStartTime: Long = 0 + var taskStartTimeNs: Long = 0 var taskStartCpu: Long = 0 startGCTime = computeTotalGcTime() @@ -410,7 +413,7 @@ private[spark] class Executor( } // Run the actual task and measure its runtime. - taskStartTime = System.currentTimeMillis() + taskStartTimeNs = System.nanoTime() taskStartCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -419,7 +422,8 @@ private[spark] class Executor( val res = task.run( taskAttemptId = taskId, attemptNumber = taskDescription.attemptNumber, - metricsSystem = env.metricsSystem) + metricsSystem = env.metricsSystem, + resources = taskDescription.resources) threwException = false res } { @@ -428,7 +432,7 @@ private[spark] class Executor( if (freedMemory > 0 && !threwException) { val errMsg = s"Managed memory leak detected; size = $freedMemory bytes, TID = $taskId" - if (conf.getBoolean("spark.unsafe.exceptionOnMemoryLeak", false)) { + if (conf.get(UNSAFE_EXCEPTION_ON_MEMORY_LEAK)) { throw new SparkException(errMsg) } else { logWarning(errMsg) @@ -439,7 +443,7 @@ private[spark] class Executor( val errMsg = s"${releasedLocks.size} block locks were not released by TID = $taskId:\n" + releasedLocks.mkString("[", ", ", "]") - if (conf.getBoolean("spark.storage.exceptionOnPinLeak", false)) { + if (conf.get(STORAGE_EXCEPTION_PIN_LEAK)) { throw new SparkException(errMsg) } else { logInfo(errMsg) @@ -454,7 +458,7 @@ private[spark] class Executor( s"unrecoverable fetch failures! Most likely this means user code is incorrectly " + s"swallowing Spark's internal ${classOf[FetchFailedException]}", fetchFailure) } - val taskFinish = System.currentTimeMillis() + val taskFinishNs = System.nanoTime() val taskFinishCpu = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L @@ -463,22 +467,24 @@ private[spark] class Executor( task.context.killTaskIfInterrupted() val resultSer = env.serializer.newInstance() - val beforeSerialization = System.currentTimeMillis() + val beforeSerializationNs = System.nanoTime() val valueBytes = resultSer.serialize(value) - val afterSerialization = System.currentTimeMillis() + val afterSerializationNs = System.nanoTime() // Deserialization happens in two parts: first, we deserialize a Task object, which // includes the Partition. Second, Task.run() deserializes the RDD and function to be run. - task.metrics.setExecutorDeserializeTime( - (taskStartTime - deserializeStartTime) + task.executorDeserializeTime) + task.metrics.setExecutorDeserializeTime(TimeUnit.NANOSECONDS.toMillis( + (taskStartTimeNs - deserializeStartTimeNs) + task.executorDeserializeTimeNs)) task.metrics.setExecutorDeserializeCpuTime( (taskStartCpu - deserializeStartCpuTime) + task.executorDeserializeCpuTime) // We need to subtract Task.run()'s deserialization time to avoid double-counting - task.metrics.setExecutorRunTime((taskFinish - taskStartTime) - task.executorDeserializeTime) + task.metrics.setExecutorRunTime(TimeUnit.NANOSECONDS.toMillis( + (taskFinishNs - taskStartTimeNs) - task.executorDeserializeTimeNs)) task.metrics.setExecutorCpuTime( (taskFinishCpu - taskStartCpu) - task.executorDeserializeCpuTime) task.metrics.setJvmGCTime(computeTotalGcTime() - startGCTime) - task.metrics.setResultSerializationTime(afterSerialization - beforeSerialization) + task.metrics.setResultSerializationTime(TimeUnit.NANOSECONDS.toMillis( + afterSerializationNs - beforeSerializationNs)) // Expose task metrics using the Dropwizard metrics system. // Update task metrics counters @@ -557,7 +563,7 @@ private[spark] class Executor( case t: TaskKilledException => logInfo(s"Executor killed $taskName (TID $taskId), reason: ${t.reason}") - val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTime) + val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) val serializedTK = ser.serialize(TaskKilled(t.reason, accUpdates, accums)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) @@ -566,7 +572,7 @@ private[spark] class Executor( val killReason = task.reasonIfKilled.getOrElse("unknown reason") logInfo(s"Executor interrupted and killed $taskName (TID $taskId), reason: $killReason") - val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTime) + val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) val serializedTK = ser.serialize(TaskKilled(killReason, accUpdates, accums)) execBackend.statusUpdate(taskId, TaskState.KILLED, serializedTK) @@ -601,7 +607,7 @@ private[spark] class Executor( // the task failure would not be ignored if the shutdown happened because of premption, // instead of an app issue). if (!ShutdownHookManager.inShutdown()) { - val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTime) + val (accums, accUpdates) = collectAccumulatorsAndResetStatusOnFailure(taskStartTimeNs) val serializedTaskEndReason = { try { @@ -664,18 +670,18 @@ private[spark] class Executor( private[this] val taskId: Long = taskRunner.taskId - private[this] val killPollingIntervalMs: Long = - conf.getTimeAsMs("spark.task.reaper.pollingInterval", "10s") + private[this] val killPollingIntervalMs: Long = conf.get(TASK_REAPER_POLLING_INTERVAL) - private[this] val killTimeoutMs: Long = conf.getTimeAsMs("spark.task.reaper.killTimeout", "-1") + private[this] val killTimeoutNs: Long = { + TimeUnit.MILLISECONDS.toNanos(conf.get(TASK_REAPER_KILL_TIMEOUT)) + } - private[this] val takeThreadDump: Boolean = - conf.getBoolean("spark.task.reaper.threadDump", true) + private[this] val takeThreadDump: Boolean = conf.get(TASK_REAPER_THREAD_DUMP) override def run(): Unit = { - val startTimeMs = System.currentTimeMillis() - def elapsedTimeMs = System.currentTimeMillis() - startTimeMs - def timeoutExceeded(): Boolean = killTimeoutMs > 0 && elapsedTimeMs > killTimeoutMs + val startTimeNs = System.nanoTime() + def elapsedTimeNs = System.nanoTime() - startTimeNs + def timeoutExceeded(): Boolean = killTimeoutNs > 0 && elapsedTimeNs > killTimeoutNs try { // Only attempt to kill the task once. If interruptThread = false then a second kill // attempt would be a no-op and if interruptThread = true then it may not be safe or @@ -700,6 +706,7 @@ private[spark] class Executor( if (taskRunner.isFinished) { finished = true } else { + val elapsedTimeMs = TimeUnit.NANOSECONDS.toMillis(elapsedTimeNs) logWarning(s"Killed task $taskId is still running after $elapsedTimeMs ms") if (takeThreadDump) { try { @@ -717,6 +724,7 @@ private[spark] class Executor( } if (!taskRunner.isFinished && timeoutExceeded()) { + val killTimeoutMs = TimeUnit.NANOSECONDS.toMillis(killTimeoutNs) if (isLocal) { logError(s"Killed task $taskId could not be stopped within $killTimeoutMs ms; " + "not killing JVM because we are running in local mode.") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.scala new file mode 100644 index 0000000000000..0ddeef8e9a82d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorLogUrlHandler.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.executor + +import java.util.concurrent.atomic.AtomicBoolean + +import scala.util.matching.Regex + +import org.apache.spark.internal.Logging + +private[spark] class ExecutorLogUrlHandler(logUrlPattern: Option[String]) extends Logging { + import ExecutorLogUrlHandler._ + + private val informedForMissingAttributes = new AtomicBoolean(false) + + def applyPattern( + logUrls: Map[String, String], + attributes: Map[String, String]): Map[String, String] = { + logUrlPattern match { + case Some(pattern) => doApplyPattern(logUrls, attributes, pattern) + case None => logUrls + } + } + + private def doApplyPattern( + logUrls: Map[String, String], + attributes: Map[String, String], + urlPattern: String): Map[String, String] = { + // Relation between pattern {{FILE_NAME}} and attribute {{LOG_FILES}} + // Given that this class don't know which types of log files can be provided + // from resource manager, we require resource manager to provide available types of log + // files, which are encouraged to be same as types of log files provided in original log URLs. + // Once we get the list of log files, we need to expose them to end users as a pattern + // so that end users can compose custom log URL(s) including log file name(s). + val allPatterns = CUSTOM_URL_PATTERN_REGEX.findAllMatchIn(urlPattern).map(_.group(1)).toSet + val allPatternsExceptFileName = allPatterns.filter(_ != "FILE_NAME") + val allAttributeKeys = attributes.keySet + val allAttributeKeysExceptLogFiles = allAttributeKeys.filter(_ != "LOG_FILES") + + if (allPatternsExceptFileName.diff(allAttributeKeysExceptLogFiles).nonEmpty) { + logFailToRenewLogUrls("some of required attributes are missing in app's event log.", + allPatternsExceptFileName, allAttributeKeys) + logUrls + } else if (allPatterns.contains("FILE_NAME") && !allAttributeKeys.contains("LOG_FILES")) { + logFailToRenewLogUrls("'FILE_NAME' parameter is provided, but file information is " + + "missing in app's event log.", allPatternsExceptFileName, allAttributeKeys) + logUrls + } else { + val updatedUrl = allPatternsExceptFileName.foldLeft(urlPattern) { case (orig, patt) => + // we already checked the existence of attribute when comparing keys + orig.replace(s"{{$patt}}", attributes(patt)) + } + + if (allPatterns.contains("FILE_NAME")) { + // allAttributeKeys should contain "LOG_FILES" + attributes("LOG_FILES").split(",").map { file => + file -> updatedUrl.replace("{{FILE_NAME}}", file) + }.toMap + } else { + Map("log" -> updatedUrl) + } + } + } + + private def logFailToRenewLogUrls( + reason: String, + allPatterns: Set[String], + allAttributes: Set[String]): Unit = { + if (informedForMissingAttributes.compareAndSet(false, true)) { + logInfo(s"Fail to renew executor log urls: $reason. Required: $allPatterns / " + + s"available: $allAttributes. Falling back to show app's original log urls.") + } + } +} + +private[spark] object ExecutorLogUrlHandler { + val CUSTOM_URL_PATTERN_REGEX: Regex = "\\{\\{([A-Za-z0-9_\\-]+)\\}\\}".r +} diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala index 2933f3ba6d3b5..f19ac813fde34 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorMetrics.scala @@ -27,17 +27,15 @@ import org.apache.spark.metrics.ExecutorMetricType */ @DeveloperApi class ExecutorMetrics private[spark] extends Serializable { - - // Metrics are indexed by MetricGetter.values - private val metrics = new Array[Long](ExecutorMetricType.values.length) - + // Metrics are indexed by ExecutorMetricType.metricToOffset + private val metrics = new Array[Long](ExecutorMetricType.numMetrics) // the first element is initialized to -1, indicating that the values for the array // haven't been set yet. metrics(0) = -1 - /** Returns the value for the specified metricType. */ - def getMetricValue(metricType: ExecutorMetricType): Long = { - metrics(ExecutorMetricType.metricIdxMap(metricType)) + /** Returns the value for the specified metric. */ + def getMetricValue(metricName: String): Long = { + metrics(ExecutorMetricType.metricToOffset(metricName)) } /** Returns true if the values for the metrics have been set, false otherwise. */ @@ -49,14 +47,14 @@ class ExecutorMetrics private[spark] extends Serializable { } /** - * Constructor: create the ExecutorMetrics with the values specified. + * Constructor: create the ExecutorMetrics with using a given map. * * @param executorMetrics map of executor metric name to value */ private[spark] def this(executorMetrics: Map[String, Long]) { this() - (0 until ExecutorMetricType.values.length).foreach { idx => - metrics(idx) = executorMetrics.getOrElse(ExecutorMetricType.values(idx).name, 0L) + ExecutorMetricType.metricToOffset.foreach { case(name, idx) => + metrics(idx) = executorMetrics.getOrElse(name, 0L) } } @@ -69,9 +67,8 @@ class ExecutorMetrics private[spark] extends Serializable { */ private[spark] def compareAndUpdatePeakValues(executorMetrics: ExecutorMetrics): Boolean = { var updated = false - - (0 until ExecutorMetricType.values.length).foreach { idx => - if (executorMetrics.metrics(idx) > metrics(idx)) { + (0 until ExecutorMetricType.numMetrics).foreach { idx => + if (executorMetrics.metrics(idx) > metrics(idx)) { updated = true metrics(idx) = executorMetrics.metrics(idx) } diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index a8264022a0aff..669ce63325d0e 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -17,12 +17,9 @@ package org.apache.spark.executor -import java.lang.management.ManagementFactory import java.util.concurrent.ThreadPoolExecutor -import javax.management.{MBeanServer, ObjectName} import scala.collection.JavaConverters._ -import scala.util.control.NonFatal import com.codahale.metrics.{Gauge, MetricRegistry} import org.apache.hadoop.fs.FileSystem @@ -76,24 +73,6 @@ class ExecutorSource(threadPool: ThreadPoolExecutor, executorId: String) extends registerFileSystemStat(scheme, "write_ops", _.getWriteOps(), 0) } - // Dropwizard metrics gauge measuring the executor's process CPU time. - // This Gauge will try to get and return the JVM Process CPU time or return -1 otherwise. - // The CPU time value is returned in nanoseconds. - // It will use proprietary extensions such as com.sun.management.OperatingSystemMXBean or - // com.ibm.lang.management.OperatingSystemMXBean, if available. - metricRegistry.register(MetricRegistry.name("jvmCpuTime"), new Gauge[Long] { - val mBean: MBeanServer = ManagementFactory.getPlatformMBeanServer - val name = new ObjectName("java.lang", "type", "OperatingSystem") - override def getValue: Long = { - try { - // return JVM process CPU time if the ProcessCpuTime method is available - mBean.getAttribute(name, "ProcessCpuTime").asInstanceOf[Long] - } catch { - case NonFatal(_) => -1L - } - } - }) - // Expose executor task metrics using the Dropwizard metrics system. // The list is taken from TaskMetrics.scala val METRIC_CPU_TIME = metricRegistry.counter(MetricRegistry.name("cpuTime")) diff --git a/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala new file mode 100644 index 0000000000000..f354d603c2e3d --- /dev/null +++ b/core/src/main/scala/org/apache/spark/executor/ProcfsMetricsGetter.scala @@ -0,0 +1,228 @@ +/* + * 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.executor + +import java.io._ +import java.nio.charset.Charset +import java.nio.file.{Files, Paths} +import java.util.Locale + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Try + +import org.apache.spark.{SparkEnv, SparkException} +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.util.Utils + + +private[spark] case class ProcfsMetrics( + jvmVmemTotal: Long, + jvmRSSTotal: Long, + pythonVmemTotal: Long, + pythonRSSTotal: Long, + otherVmemTotal: Long, + otherRSSTotal: Long) + +// Some of the ideas here are taken from the ProcfsBasedProcessTree class in hadoop +// project. +private[spark] class ProcfsMetricsGetter(procfsDir: String = "/proc/") extends Logging { + private val procfsStatFile = "stat" + private val testing = Utils.isTesting + private val pageSize = computePageSize() + private var isAvailable: Boolean = isProcfsAvailable + private val pid = computePid() + + private lazy val isProcfsAvailable: Boolean = { + if (testing) { + true + } + else { + val procDirExists = Try(Files.exists(Paths.get(procfsDir))).recover { + case ioe: IOException => + logWarning("Exception checking for procfs dir", ioe) + false + } + val shouldLogStageExecutorMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_STAGE_EXECUTOR_METRICS) + val shouldLogStageExecutorProcessTreeMetrics = + SparkEnv.get.conf.get(config.EVENT_LOG_PROCESS_TREE_METRICS) + procDirExists.get && shouldLogStageExecutorProcessTreeMetrics && shouldLogStageExecutorMetrics + } + } + + private def computePid(): Int = { + if (!isAvailable || testing) { + return -1; + } + try { + // This can be simplified in java9: + // https://docs.oracle.com/javase/9/docs/api/java/lang/ProcessHandle.html + val cmd = Array("bash", "-c", "echo $PPID") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } + catch { + case e: SparkException => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped", e) + isAvailable = false + -1 + } + } + + private def computePageSize(): Long = { + if (testing) { + return 4096; + } + try { + val cmd = Array("getconf", "PAGESIZE") + val out = Utils.executeAndGetOutput(cmd) + Integer.parseInt(out.split("\n")(0)) + } catch { + case e: Exception => + logWarning("Exception when trying to compute pagesize, as a" + + " result reporting of ProcessTree metrics is stopped") + isAvailable = false + 0 + } + } + + private def computeProcessTree(): Set[Int] = { + if (!isAvailable || testing) { + return Set() + } + var ptree: Set[Int] = Set() + ptree += pid + val queue = mutable.Queue.empty[Int] + queue += pid + while ( !queue.isEmpty ) { + val p = queue.dequeue() + val c = getChildPids(p) + if (!c.isEmpty) { + queue ++= c + ptree ++= c.toSet + } + } + ptree + } + + private def getChildPids(pid: Int): ArrayBuffer[Int] = { + try { + val builder = new ProcessBuilder("pgrep", "-P", pid.toString) + val process = builder.start() + val childPidsInInt = mutable.ArrayBuffer.empty[Int] + def appendChildPid(s: String): Unit = { + if (s != "") { + logTrace("Found a child pid:" + s) + childPidsInInt += Integer.parseInt(s) + } + } + val stdoutThread = Utils.processStreamByLine("read stdout for pgrep", + process.getInputStream, appendChildPid) + val errorStringBuilder = new StringBuilder() + val stdErrThread = Utils.processStreamByLine( + "stderr for pgrep", + process.getErrorStream, + line => errorStringBuilder.append(line)) + val exitCode = process.waitFor() + stdoutThread.join() + stdErrThread.join() + val errorString = errorStringBuilder.toString() + // pgrep will have exit code of 1 if there are more than one child process + // and it will have a exit code of 2 if there is no child process + if (exitCode != 0 && exitCode > 2) { + val cmd = builder.command().toArray.mkString(" ") + logWarning(s"Process $cmd exited with code $exitCode and stderr: $errorString") + throw new SparkException(s"Process $cmd exited with code $exitCode") + } + childPidsInInt + } catch { + case e: Exception => + logWarning("Exception when trying to compute process tree." + + " As a result reporting of ProcessTree metrics is stopped.", e) + isAvailable = false + mutable.ArrayBuffer.empty[Int] + } + } + + def addProcfsMetricsFromOneProcess( + allMetrics: ProcfsMetrics, + pid: Int): ProcfsMetrics = { + + // The computation of RSS and Vmem are based on proc(5): + // http://man7.org/linux/man-pages/man5/proc.5.html + try { + val pidDir = new File(procfsDir, pid.toString) + def openReader(): BufferedReader = { + val f = new File(new File(procfsDir, pid.toString), procfsStatFile) + new BufferedReader(new InputStreamReader(new FileInputStream(f), Charset.forName("UTF-8"))) + } + Utils.tryWithResource(openReader) { in => + val procInfo = in.readLine + val procInfoSplit = procInfo.split(" ") + val vmem = procInfoSplit(22).toLong + val rssMem = procInfoSplit(23).toLong * pageSize + if (procInfoSplit(1).toLowerCase(Locale.US).contains("java")) { + allMetrics.copy( + jvmVmemTotal = allMetrics.jvmVmemTotal + vmem, + jvmRSSTotal = allMetrics.jvmRSSTotal + (rssMem) + ) + } + else if (procInfoSplit(1).toLowerCase(Locale.US).contains("python")) { + allMetrics.copy( + pythonVmemTotal = allMetrics.pythonVmemTotal + vmem, + pythonRSSTotal = allMetrics.pythonRSSTotal + (rssMem) + ) + } + else { + allMetrics.copy( + otherVmemTotal = allMetrics.otherVmemTotal + vmem, + otherRSSTotal = allMetrics.otherRSSTotal + (rssMem) + ) + } + } + } catch { + case f: IOException => + logWarning("There was a problem with reading" + + " the stat file of the process. ", f) + ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + + private[spark] def computeAllMetrics(): ProcfsMetrics = { + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + val pids = computeProcessTree + var allMetrics = ProcfsMetrics(0, 0, 0, 0, 0, 0) + for (p <- pids) { + allMetrics = addProcfsMetricsFromOneProcess(allMetrics, p) + // if we had an error getting any of the metrics, we don't want to report partial metrics, as + // that would be misleading. + if (!isAvailable) { + return ProcfsMetrics(0, 0, 0, 0, 0, 0) + } + } + allMetrics + } +} + +private[spark] object ProcfsMetricsGetter { + final val pTreeInfo = new ProcfsMetricsGetter +} diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala index 4be395c8358b2..12c4b8f67f71c 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleReadMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.shuffle.ShuffleReadMetricsReporter import org.apache.spark.util.LongAccumulator @@ -123,12 +124,13 @@ class ShuffleReadMetrics private[spark] () extends Serializable { } } + /** * A temporary shuffle read metrics holder that is used to collect shuffle read metrics for each * shuffle dependency, and all temporary metrics will be merged into the [[ShuffleReadMetrics]] at * last. */ -private[spark] class TempShuffleReadMetrics { +private[spark] class TempShuffleReadMetrics extends ShuffleReadMetricsReporter { private[this] var _remoteBlocksFetched = 0L private[this] var _localBlocksFetched = 0L private[this] var _remoteBytesRead = 0L @@ -137,13 +139,13 @@ private[spark] class TempShuffleReadMetrics { private[this] var _fetchWaitTime = 0L private[this] var _recordsRead = 0L - def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched += v - def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched += v - def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v - def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v - def incLocalBytesRead(v: Long): Unit = _localBytesRead += v - def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v - def incRecordsRead(v: Long): Unit = _recordsRead += v + override def incRemoteBlocksFetched(v: Long): Unit = _remoteBlocksFetched += v + override def incLocalBlocksFetched(v: Long): Unit = _localBlocksFetched += v + override def incRemoteBytesRead(v: Long): Unit = _remoteBytesRead += v + override def incRemoteBytesReadToDisk(v: Long): Unit = _remoteBytesReadToDisk += v + override def incLocalBytesRead(v: Long): Unit = _localBytesRead += v + override def incFetchWaitTime(v: Long): Unit = _fetchWaitTime += v + override def incRecordsRead(v: Long): Unit = _recordsRead += v def remoteBlocksFetched: Long = _remoteBlocksFetched def localBlocksFetched: Long = _localBlocksFetched diff --git a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala index ada2e1bc08593..d0b0e7da079c9 100644 --- a/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/ShuffleWriteMetrics.scala @@ -18,6 +18,7 @@ package org.apache.spark.executor import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.util.LongAccumulator @@ -27,7 +28,7 @@ import org.apache.spark.util.LongAccumulator * Operations are not thread-safe. */ @DeveloperApi -class ShuffleWriteMetrics private[spark] () extends Serializable { +class ShuffleWriteMetrics private[spark] () extends ShuffleWriteMetricsReporter with Serializable { private[executor] val _bytesWritten = new LongAccumulator private[executor] val _recordsWritten = new LongAccumulator private[executor] val _writeTime = new LongAccumulator @@ -47,23 +48,13 @@ class ShuffleWriteMetrics private[spark] () extends Serializable { */ def writeTime: Long = _writeTime.sum - private[spark] def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) - private[spark] def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) - private[spark] def incWriteTime(v: Long): Unit = _writeTime.add(v) - private[spark] def decBytesWritten(v: Long): Unit = { + private[spark] override def incBytesWritten(v: Long): Unit = _bytesWritten.add(v) + private[spark] override def incRecordsWritten(v: Long): Unit = _recordsWritten.add(v) + private[spark] override def incWriteTime(v: Long): Unit = _writeTime.add(v) + private[spark] override def decBytesWritten(v: Long): Unit = { _bytesWritten.setValue(bytesWritten - v) } - private[spark] def decRecordsWritten(v: Long): Unit = { + private[spark] override def decRecordsWritten(v: Long): Unit = { _recordsWritten.setValue(recordsWritten - v) } - - // Legacy methods for backward compatibility. - // TODO: remove these once we make this class private. - @deprecated("use bytesWritten instead", "2.0.0") - def shuffleBytesWritten: Long = bytesWritten - @deprecated("use writeTime instead", "2.0.0") - def shuffleWriteTime: Long = writeTime - @deprecated("use recordsWritten instead", "2.0.0") - def shuffleRecordsWritten: Long = recordsWritten - } diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 85b2745a2aec4..ea79c7310349d 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.{ArrayBuffer, LinkedHashMap} import org.apache.spark._ import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.storage.{BlockId, BlockStatus} import org.apache.spark.util._ @@ -202,7 +203,7 @@ class TaskMetrics private[spark] () extends Serializable { } // Only used for test - private[spark] val testAccum = sys.props.get("spark.testing").map(_ => new LongAccumulator) + private[spark] val testAccum = sys.props.get(IS_TESTING.key).map(_ => new LongAccumulator) import InternalAccumulator._ diff --git a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala index ab020aaf6fa4f..5b33c110154d6 100644 --- a/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala +++ b/core/src/main/scala/org/apache/spark/input/PortableDataStream.scala @@ -52,6 +52,18 @@ private[spark] abstract class StreamFileInputFormat[T] val totalBytes = files.filterNot(_.isDirectory).map(_.getLen + openCostInBytes).sum val bytesPerCore = totalBytes / defaultParallelism val maxSplitSize = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + + // For small files we need to ensure the min split size per node & rack <= maxSplitSize + val jobConfig = context.getConfiguration + val minSplitSizePerNode = jobConfig.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERNODE, 0L) + val minSplitSizePerRack = jobConfig.getLong(CombineFileInputFormat.SPLIT_MINSIZE_PERRACK, 0L) + + if (maxSplitSize < minSplitSizePerNode) { + super.setMinSplitSizeNode(maxSplitSize) + } + if (maxSplitSize < minSplitSizePerRack) { + super.setMinSplitSizeRack(maxSplitSize) + } super.setMaxSplitSize(maxSplitSize) } diff --git a/core/src/main/scala/org/apache/spark/internal/Logging.scala b/core/src/main/scala/org/apache/spark/internal/Logging.scala index c0d709ad25f29..0987917bac0e7 100644 --- a/core/src/main/scala/org/apache/spark/internal/Logging.scala +++ b/core/src/main/scala/org/apache/spark/internal/Logging.scala @@ -17,7 +17,10 @@ package org.apache.spark.internal -import org.apache.log4j.{Level, LogManager, PropertyConfigurator} +import scala.collection.JavaConverters._ + +import org.apache.log4j._ +import org.apache.log4j.spi.{Filter, LoggingEvent} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder @@ -143,13 +146,19 @@ trait Logging { // overriding the root logger's config if they're different. val replLogger = LogManager.getLogger(logName) val replLevel = Option(replLogger.getLevel()).getOrElse(Level.WARN) + // Update the consoleAppender threshold to replLevel if (replLevel != rootLogger.getEffectiveLevel()) { if (!silent) { System.err.printf("Setting default log level to \"%s\".\n", replLevel) System.err.println("To adjust logging level use sc.setLogLevel(newLevel). " + "For SparkR, use setLogLevel(newLevel).") } - rootLogger.setLevel(replLevel) + Logging.sparkShellThresholdLevel = replLevel + rootLogger.getAllAppenders().asScala.foreach { + case ca: ConsoleAppender => + ca.addFilter(new SparkShellLoggingFilter()) + case _ => // no-op + } } } // scalastyle:on println @@ -166,6 +175,7 @@ private[spark] object Logging { @volatile private var initialized = false @volatile private var defaultRootLevel: Level = null @volatile private var defaultSparkLog4jConfig = false + @volatile private[spark] var sparkShellThresholdLevel: Level = null val initLock = new Object() try { @@ -192,7 +202,9 @@ private[spark] object Logging { defaultSparkLog4jConfig = false LogManager.resetConfiguration() } else { - LogManager.getRootLogger().setLevel(defaultRootLevel) + val rootLogger = LogManager.getRootLogger() + rootLogger.setLevel(defaultRootLevel) + sparkShellThresholdLevel = null } } this.initialized = false @@ -206,3 +218,31 @@ private[spark] object Logging { "org.slf4j.impl.Log4jLoggerFactory".equals(binderClass) } } + +private class SparkShellLoggingFilter extends Filter { + + /** + * If sparkShellThresholdLevel is not defined, this filter is a no-op. + * If log level of event is not equal to root level, the event is allowed. Otherwise, + * the decision is made based on whether the log came from root or some custom configuration + * @param loggingEvent + * @return decision for accept/deny log event + */ + def decide(loggingEvent: LoggingEvent): Int = { + if (Logging.sparkShellThresholdLevel == null) { + return Filter.NEUTRAL + } + val rootLevel = LogManager.getRootLogger().getLevel() + if (!loggingEvent.getLevel().eq(rootLevel)) { + return Filter.NEUTRAL + } + var logger = loggingEvent.getLogger() + while (logger.getParent() != null) { + if (logger.getLevel() != null) { + return Filter.NEUTRAL + } + logger = logger.getParent() + } + return Filter.DENY + } +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala index f27aca03773a9..68e1994f0f94f 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigBuilder.scala @@ -127,8 +127,9 @@ private[spark] class TypedConfigBuilder[T]( /** Creates a [[ConfigEntry]] that does not have a default value. */ def createOptional: OptionalConfigEntry[T] = { - val entry = new OptionalConfigEntry[T](parent.key, parent._alternatives, converter, - stringConverter, parent._doc, parent._public) + val entry = new OptionalConfigEntry[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, converter, stringConverter, parent._doc, + parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -141,8 +142,9 @@ private[spark] class TypedConfigBuilder[T]( createWithDefaultString(default.asInstanceOf[String]) } else { val transformedDefault = converter(stringConverter(default)) - val entry = new ConfigEntryWithDefault[T](parent.key, parent._alternatives, - transformedDefault, converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefault[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, transformedDefault, converter, + stringConverter, parent._doc, parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -150,8 +152,9 @@ private[spark] class TypedConfigBuilder[T]( /** Creates a [[ConfigEntry]] with a function to determine the default value */ def createWithDefaultFunction(defaultFunc: () => T): ConfigEntry[T] = { - val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._alternatives, defaultFunc, - converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefaultFunction[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, defaultFunc, converter, stringConverter, + parent._doc, parent._public) parent._onCreate.foreach(_ (entry)) entry } @@ -161,8 +164,9 @@ private[spark] class TypedConfigBuilder[T]( * [[String]] and must be a valid value for the entry. */ def createWithDefaultString(default: String): ConfigEntry[T] = { - val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._alternatives, default, - converter, stringConverter, parent._doc, parent._public) + val entry = new ConfigEntryWithDefaultString[T](parent.key, parent._prependedKey, + parent._prependSeparator, parent._alternatives, default, converter, stringConverter, + parent._doc, parent._public) parent._onCreate.foreach(_(entry)) entry } @@ -178,6 +182,8 @@ private[spark] case class ConfigBuilder(key: String) { import ConfigHelpers._ + private[config] var _prependedKey: Option[String] = None + private[config] var _prependSeparator: String = "" private[config] var _public = true private[config] var _doc = "" private[config] var _onCreate: Option[ConfigEntry[_] => Unit] = None @@ -202,24 +208,34 @@ private[spark] case class ConfigBuilder(key: String) { this } + def withPrepended(key: String, separator: String = " "): ConfigBuilder = { + _prependedKey = Option(key) + _prependSeparator = separator + this + } + def withAlternative(key: String): ConfigBuilder = { _alternatives = _alternatives :+ key this } def intConf: TypedConfigBuilder[Int] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toInt, key, "int")) } def longConf: TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toLong, key, "long")) } def doubleConf: TypedConfigBuilder[Double] = { + checkPrependConfig new TypedConfigBuilder(this, toNumber(_, _.toDouble, key, "double")) } def booleanConf: TypedConfigBuilder[Boolean] = { + checkPrependConfig new TypedConfigBuilder(this, toBoolean(_, key)) } @@ -228,20 +244,30 @@ private[spark] case class ConfigBuilder(key: String) { } def timeConf(unit: TimeUnit): TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, timeFromString(_, unit), timeToString(_, unit)) } def bytesConf(unit: ByteUnit): TypedConfigBuilder[Long] = { + checkPrependConfig new TypedConfigBuilder(this, byteFromString(_, unit), byteToString(_, unit)) } def fallbackConf[T](fallback: ConfigEntry[T]): ConfigEntry[T] = { - val entry = new FallbackConfigEntry(key, _alternatives, _doc, _public, fallback) + val entry = new FallbackConfigEntry(key, _prependedKey, _prependSeparator, _alternatives, _doc, + _public, fallback) _onCreate.foreach(_(entry)) entry } def regexConf: TypedConfigBuilder[Regex] = { + checkPrependConfig new TypedConfigBuilder(this, regexFromString(_, this.key), _.toString) } + + private def checkPrependConfig = { + if (_prependedKey.isDefined) { + throw new IllegalArgumentException(s"$key type must be string if prepend used") + } + } } diff --git a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala index ede3ace4f9aac..c5df4c8820098 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/ConfigEntry.scala @@ -28,6 +28,8 @@ package org.apache.spark.internal.config * value declared as a string. * * @param key the key for the configuration + * @param prependedKey the key for the configuration which will be prepended + * @param prependSeparator the separator which is used for prepending * @param valueConverter how to convert a string to the value. It should throw an exception if the * string does not have the required format. * @param stringConverter how to convert a value to a string that the user can use it as a valid @@ -41,6 +43,8 @@ package org.apache.spark.internal.config */ private[spark] abstract class ConfigEntry[T] ( val key: String, + val prependedKey: Option[String], + val prependSeparator: String, val alternatives: List[String], val valueConverter: String => T, val stringConverter: T => String, @@ -54,7 +58,15 @@ private[spark] abstract class ConfigEntry[T] ( def defaultValueString: String protected def readString(reader: ConfigReader): Option[String] = { - alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + val values = Seq( + prependedKey.flatMap(reader.get(_)), + alternatives.foldLeft(reader.get(key))((res, nextKey) => res.orElse(reader.get(nextKey))) + ).flatten + if (values.nonEmpty) { + Some(values.mkString(prependSeparator)) + } else { + None + } } def readFrom(reader: ConfigReader): T @@ -68,13 +80,24 @@ private[spark] abstract class ConfigEntry[T] ( private class ConfigEntryWithDefault[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], _defaultValue: T, valueConverter: String => T, stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(_defaultValue) @@ -86,14 +109,25 @@ private class ConfigEntryWithDefault[T] ( } private class ConfigEntryWithDefaultFunction[T] ( - key: String, - alternatives: List[String], - _defaultFunction: () => T, - valueConverter: String => T, - stringConverter: T => String, - doc: String, - isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + key: String, + prependedKey: Option[String], + prependSeparator: String, + alternatives: List[String], + _defaultFunction: () => T, + valueConverter: String => T, + stringConverter: T => String, + doc: String, + isPublic: Boolean) + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(_defaultFunction()) @@ -106,13 +140,24 @@ private class ConfigEntryWithDefaultFunction[T] ( private class ConfigEntryWithDefaultString[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], _defaultValue: String, valueConverter: String => T, stringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry(key, alternatives, valueConverter, stringConverter, doc, isPublic) { + extends ConfigEntry( + key, + prependedKey, + prependSeparator, + alternatives, + valueConverter, + stringConverter, + doc, + isPublic + ) { override def defaultValue: Option[T] = Some(valueConverter(_defaultValue)) @@ -130,14 +175,23 @@ private class ConfigEntryWithDefaultString[T] ( */ private[spark] class OptionalConfigEntry[T]( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], val rawValueConverter: String => T, val rawStringConverter: T => String, doc: String, isPublic: Boolean) - extends ConfigEntry[Option[T]](key, alternatives, + extends ConfigEntry[Option[T]]( + key, + prependedKey, + prependSeparator, + alternatives, s => Some(rawValueConverter(s)), - v => v.map(rawStringConverter).orNull, doc, isPublic) { + v => v.map(rawStringConverter).orNull, + doc, + isPublic + ) { override def defaultValueString: String = ConfigEntry.UNDEFINED @@ -151,12 +205,22 @@ private[spark] class OptionalConfigEntry[T]( */ private[spark] class FallbackConfigEntry[T] ( key: String, + prependedKey: Option[String], + prependSeparator: String, alternatives: List[String], doc: String, isPublic: Boolean, val fallback: ConfigEntry[T]) - extends ConfigEntry[T](key, alternatives, - fallback.valueConverter, fallback.stringConverter, doc, isPublic) { + extends ConfigEntry[T]( + key, + prependedKey, + prependSeparator, + alternatives, + fallback.valueConverter, + fallback.stringConverter, + doc, + isPublic + ) { override def defaultValueString: String = s"" diff --git a/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala new file mode 100644 index 0000000000000..ceab957b36634 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Deploy.scala @@ -0,0 +1,68 @@ +/* + * 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.internal.config + +private[spark] object Deploy { + val RECOVERY_MODE = ConfigBuilder("spark.deploy.recoveryMode") + .stringConf + .createWithDefault("NONE") + + val RECOVERY_MODE_FACTORY = ConfigBuilder("spark.deploy.recoveryMode.factory") + .stringConf + .createWithDefault("") + + val RECOVERY_DIRECTORY = ConfigBuilder("spark.deploy.recoveryDirectory") + .stringConf + .createWithDefault("") + + val ZOOKEEPER_URL = ConfigBuilder("spark.deploy.zookeeper.url") + .doc(s"When `${RECOVERY_MODE.key}` is set to ZOOKEEPER, this " + + "configuration is used to set the zookeeper URL to connect to.") + .stringConf + .createOptional + + val ZOOKEEPER_DIRECTORY = ConfigBuilder("spark.deploy.zookeeper.dir") + .stringConf + .createOptional + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.deploy.retainedApplications") + .intConf + .createWithDefault(200) + + val RETAINED_DRIVERS = ConfigBuilder("spark.deploy.retainedDrivers") + .intConf + .createWithDefault(200) + + val REAPER_ITERATIONS = ConfigBuilder("spark.dead.worker.persistence") + .intConf + .createWithDefault(15) + + val MAX_EXECUTOR_RETRIES = ConfigBuilder("spark.deploy.maxExecutorRetries") + .intConf + .createWithDefault(10) + + val SPREAD_OUT_APPS = ConfigBuilder("spark.deploy.spreadOut") + .booleanConf + .createWithDefault(true) + + val DEFAULT_CORES = ConfigBuilder("spark.deploy.defaultCores") + .intConf + .createWithDefault(Int.MaxValue) + + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/History.scala b/core/src/main/scala/org/apache/spark/internal/config/History.scala new file mode 100644 index 0000000000000..ca9af316dffd0 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/History.scala @@ -0,0 +1,151 @@ +/* + * 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.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object History { + + val DEFAULT_LOG_DIR = "file:/tmp/spark-events" + + val HISTORY_LOG_DIR = ConfigBuilder("spark.history.fs.logDirectory") + .stringConf + .createWithDefault(DEFAULT_LOG_DIR) + + val SAFEMODE_CHECK_INTERVAL_S = ConfigBuilder("spark.history.fs.safemodeCheck.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("5s") + + val UPDATE_INTERVAL_S = ConfigBuilder("spark.history.fs.update.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("10s") + + val CLEANER_ENABLED = ConfigBuilder("spark.history.fs.cleaner.enabled") + .booleanConf + .createWithDefault(false) + + val CLEANER_INTERVAL_S = ConfigBuilder("spark.history.fs.cleaner.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("1d") + + val MAX_LOG_AGE_S = ConfigBuilder("spark.history.fs.cleaner.maxAge") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("7d") + + val MAX_LOG_NUM = ConfigBuilder("spark.history.fs.cleaner.maxNum") + .doc("The maximum number of log files in the event log directory.") + .intConf + .createWithDefault(Int.MaxValue) + + val LOCAL_STORE_DIR = ConfigBuilder("spark.history.store.path") + .doc("Local directory where to cache application history information. By default this is " + + "not set, meaning all history information will be kept in memory.") + .stringConf + .createOptional + + val MAX_LOCAL_DISK_USAGE = ConfigBuilder("spark.history.store.maxDiskUsage") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("10g") + + val HISTORY_SERVER_UI_PORT = ConfigBuilder("spark.history.ui.port") + .doc("Web UI port to bind Spark History Server") + .intConf + .createWithDefault(18080) + + val FAST_IN_PROGRESS_PARSING = + ConfigBuilder("spark.history.fs.inProgressOptimization.enabled") + .doc("Enable optimized handling of in-progress logs. This option may leave finished " + + "applications that fail to rename their event logs listed as in-progress.") + .booleanConf + .createWithDefault(true) + + val END_EVENT_REPARSE_CHUNK_SIZE = + ConfigBuilder("spark.history.fs.endEventReparseChunkSize") + .doc("How many bytes to parse at the end of log files looking for the end event. " + + "This is used to speed up generation of application listings by skipping unnecessary " + + "parts of event log files. It can be disabled by setting this config to 0.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("1m") + + val DRIVER_LOG_CLEANER_ENABLED = ConfigBuilder("spark.history.fs.driverlog.cleaner.enabled") + .fallbackConf(CLEANER_ENABLED) + + val DRIVER_LOG_CLEANER_INTERVAL = ConfigBuilder("spark.history.fs.driverlog.cleaner.interval") + .fallbackConf(CLEANER_INTERVAL_S) + + val MAX_DRIVER_LOG_AGE_S = ConfigBuilder("spark.history.fs.driverlog.cleaner.maxAge") + .fallbackConf(MAX_LOG_AGE_S) + + val HISTORY_SERVER_UI_ACLS_ENABLE = ConfigBuilder("spark.history.ui.acls.enable") + .booleanConf + .createWithDefault(false) + + val HISTORY_SERVER_UI_ADMIN_ACLS = ConfigBuilder("spark.history.ui.admin.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS = ConfigBuilder("spark.history.ui.admin.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val NUM_REPLAY_THREADS = ConfigBuilder("spark.history.fs.numReplayThreads") + .intConf + .createWithDefaultFunction(() => Math.ceil(Runtime.getRuntime.availableProcessors() / 4f).toInt) + + val RETAINED_APPLICATIONS = ConfigBuilder("spark.history.retainedApplications") + .intConf + .createWithDefault(50) + + val PROVIDER = ConfigBuilder("spark.history.provider") + .stringConf + .createOptional + + val KERBEROS_ENABLED = ConfigBuilder("spark.history.kerberos.enabled") + .booleanConf + .createWithDefault(false) + + val KERBEROS_PRINCIPAL = ConfigBuilder("spark.history.kerberos.principal") + .stringConf + .createOptional + + val KERBEROS_KEYTAB = ConfigBuilder("spark.history.kerberos.keytab") + .stringConf + .createOptional + + val CUSTOM_EXECUTOR_LOG_URL = ConfigBuilder("spark.history.custom.executor.log.url") + .doc("Specifies custom spark executor log url for supporting external log service instead of " + + "using cluster managers' application log urls in the history server. Spark will support " + + "some path variables via patterns which can vary on cluster manager. Please check the " + + "documentation for your cluster manager to see which patterns are supported, if any. " + + "This configuration has no effect on a live application, it only affects the history server.") + .stringConf + .createOptional + + val APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP = + ConfigBuilder("spark.history.custom.executor.log.url.applyIncompleteApplication") + .doc("Whether to apply custom executor log url, as specified by " + + "`spark.history.custom.executor.log.url`, to incomplete application as well. " + + "Even if this is true, this still only affects the behavior of the history server, " + + "not running spark applications.") + .booleanConf + .createWithDefault(true) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala new file mode 100644 index 0000000000000..717a09914a2f5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Kryo.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Kryo { + + val KRYO_REGISTRATION_REQUIRED = ConfigBuilder("spark.kryo.registrationRequired") + .booleanConf + .createWithDefault(false) + + val KRYO_USER_REGISTRATORS = ConfigBuilder("spark.kryo.registrator") + .stringConf + .createOptional + + val KRYO_CLASSES_TO_REGISTER = ConfigBuilder("spark.kryo.classesToRegister") + .stringConf + .toSequence + .createWithDefault(Nil) + + val KRYO_USE_UNSAFE = ConfigBuilder("spark.kryo.unsafe") + .booleanConf + .createWithDefault(false) + + val KRYO_USE_POOL = ConfigBuilder("spark.kryo.pool") + .booleanConf + .createWithDefault(true) + + val KRYO_REFERENCE_TRACKING = ConfigBuilder("spark.kryo.referenceTracking") + .booleanConf + .createWithDefault(true) + + val KRYO_SERIALIZER_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("64k") + + val KRYO_SERIALIZER_MAX_BUFFER_SIZE = ConfigBuilder("spark.kryoserializer.buffer.max") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("64m") + +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Network.scala b/core/src/main/scala/org/apache/spark/internal/config/Network.scala new file mode 100644 index 0000000000000..129e31a82979f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Network.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.internal.config + +import java.util.concurrent.TimeUnit + +private[spark] object Network { + + private[spark] val NETWORK_CRYPTO_SASL_FALLBACK = + ConfigBuilder("spark.network.crypto.saslFallback") + .booleanConf + .createWithDefault(true) + + private[spark] val NETWORK_CRYPTO_ENABLED = + ConfigBuilder("spark.network.crypto.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION = + ConfigBuilder("spark.network.remoteReadNioBufferConversion") + .booleanConf + .createWithDefault(false) + + private[spark] val NETWORK_TIMEOUT = + ConfigBuilder("spark.network.timeout") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("120s") + + private[spark] val NETWORK_TIMEOUT_INTERVAL = + ConfigBuilder("spark.network.timeoutInterval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString(STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL.defaultValueString) + + private[spark] val RPC_ASK_TIMEOUT = + ConfigBuilder("spark.rpc.askTimeout") + .stringConf + .createOptional + + private[spark] val RPC_CONNECT_THREADS = + ConfigBuilder("spark.rpc.connect.threads") + .intConf + .createWithDefault(64) + + private[spark] val RPC_IO_NUM_CONNECTIONS_PER_PEER = + ConfigBuilder("spark.rpc.io.numConnectionsPerPeer") + .intConf + .createWithDefault(1) + + private[spark] val RPC_IO_THREADS = + ConfigBuilder("spark.rpc.io.threads") + .intConf + .createOptional + + private[spark] val RPC_LOOKUP_TIMEOUT = + ConfigBuilder("spark.rpc.lookupTimeout") + .stringConf + .createOptional + + private[spark] val RPC_MESSAGE_MAX_SIZE = + ConfigBuilder("spark.rpc.message.maxSize") + .intConf + .createWithDefault(128) + + private[spark] val RPC_NETTY_DISPATCHER_NUM_THREADS = + ConfigBuilder("spark.rpc.netty.dispatcher.numThreads") + .intConf + .createOptional + + private[spark] val RPC_NUM_RETRIES = + ConfigBuilder("spark.rpc.numRetries") + .intConf + .createWithDefault(3) + + private[spark] val RPC_RETRY_WAIT = + ConfigBuilder("spark.rpc.retry.wait") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("3s") +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Python.scala b/core/src/main/scala/org/apache/spark/internal/config/Python.scala new file mode 100644 index 0000000000000..26a0598f49411 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Python.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object Python { + val PYTHON_WORKER_REUSE = ConfigBuilder("spark.python.worker.reuse") + .booleanConf + .createWithDefault(true) + + val PYTHON_TASK_KILL_TIMEOUT = ConfigBuilder("spark.python.task.killTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("2s") + + val PYTHON_USE_DAEMON = ConfigBuilder("spark.python.use.daemon") + .booleanConf + .createWithDefault(true) + + val PYTHON_DAEMON_MODULE = ConfigBuilder("spark.python.daemon.module") + .stringConf + .createOptional + + val PYTHON_WORKER_MODULE = ConfigBuilder("spark.python.worker.module") + .stringConf + .createOptional + + val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") + .bytesConf(ByteUnit.MiB) + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/R.scala b/core/src/main/scala/org/apache/spark/internal/config/R.scala new file mode 100644 index 0000000000000..26e06a5231c42 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/R.scala @@ -0,0 +1,40 @@ +/* + * 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.internal.config + +private[spark] object R { + + val R_BACKEND_CONNECTION_TIMEOUT = ConfigBuilder("spark.r.backendConnectionTimeout") + .intConf + .createWithDefault(6000) + + val R_NUM_BACKEND_THREADS = ConfigBuilder("spark.r.numRBackendThreads") + .intConf + .createWithDefault(2) + + val R_HEARTBEAT_INTERVAL = ConfigBuilder("spark.r.heartBeatInterval") + .intConf + .createWithDefault(100) + + val SPARKR_COMMAND = ConfigBuilder("spark.sparkr.r.command") + .stringConf + .createWithDefault("Rscript") + + val R_COMMAND = ConfigBuilder("spark.r.command") + .stringConf + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/status/config.scala b/core/src/main/scala/org/apache/spark/internal/config/Status.scala similarity index 72% rename from core/src/main/scala/org/apache/spark/status/config.scala rename to core/src/main/scala/org/apache/spark/internal/config/Status.scala index 67801b8f046f4..3e6a4e9810664 100644 --- a/core/src/main/scala/org/apache/spark/status/config.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/Status.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.status +package org.apache.spark.internal.config import java.util.concurrent.TimeUnit -import org.apache.spark.internal.config._ - -private[spark] object config { +private[spark] object Status { val ASYNC_TRACKING_ENABLED = ConfigBuilder("spark.appStateStore.asyncTracking.enable") .booleanConf @@ -31,6 +29,12 @@ private[spark] object config { .timeConf(TimeUnit.NANOSECONDS) .createWithDefaultString("100ms") + val LIVE_ENTITY_UPDATE_MIN_FLUSH_PERIOD = ConfigBuilder("spark.ui.liveUpdate.minFlushPeriod") + .doc("Minimum time elapsed before stale UI data is flushed. This avoids UI staleness when " + + "incoming task events are not fired frequently.") + .timeConf(TimeUnit.NANOSECONDS) + .createWithDefaultString("1s") + val MAX_RETAINED_JOBS = ConfigBuilder("spark.ui.retainedJobs") .intConf .createWithDefault(1000) @@ -51,4 +55,10 @@ private[spark] object config { .intConf .createWithDefault(Int.MaxValue) + val APP_STATUS_METRICS_ENABLED = + ConfigBuilder("spark.app.status.metrics.enabled") + .doc("Whether Dropwizard/Codahale metrics " + + "will be reported for the status of the running spark app.") + .booleanConf + .createWithDefault(false) } diff --git a/core/src/main/scala/org/apache/spark/internal/config/Streaming.scala b/core/src/main/scala/org/apache/spark/internal/config/Streaming.scala new file mode 100644 index 0000000000000..6e58c090e8126 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Streaming.scala @@ -0,0 +1,68 @@ +/* + * 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.internal.config + +import java.util.concurrent.TimeUnit + +private[spark] object Streaming { + + private[spark] val STREAMING_DYN_ALLOCATION_ENABLED = + ConfigBuilder("spark.streaming.dynamicAllocation.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val STREAMING_DYN_ALLOCATION_TESTING = + ConfigBuilder("spark.streaming.dynamicAllocation.testing") + .booleanConf + .createWithDefault(false) + + private[spark] val STREAMING_DYN_ALLOCATION_MIN_EXECUTORS = + ConfigBuilder("spark.streaming.dynamicAllocation.minExecutors") + .intConf + .checkValue(_ > 0, "The min executor number of streaming dynamic " + + "allocation must be positive.") + .createOptional + + private[spark] val STREAMING_DYN_ALLOCATION_MAX_EXECUTORS = + ConfigBuilder("spark.streaming.dynamicAllocation.maxExecutors") + .intConf + .checkValue(_ > 0, "The max executor number of streaming dynamic " + + "allocation must be positive.") + .createWithDefault(Int.MaxValue) + + private[spark] val STREAMING_DYN_ALLOCATION_SCALING_INTERVAL = + ConfigBuilder("spark.streaming.dynamicAllocation.scalingInterval") + .timeConf(TimeUnit.SECONDS) + .checkValue(_ > 0, "The scaling interval of streaming dynamic " + + "allocation must be positive.") + .createWithDefault(60) + + private[spark] val STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO = + ConfigBuilder("spark.streaming.dynamicAllocation.scalingUpRatio") + .doubleConf + .checkValue(_ > 0, "The scaling up ratio of streaming dynamic " + + "allocation must be positive.") + .createWithDefault(0.9) + + private[spark] val STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO = + ConfigBuilder("spark.streaming.dynamicAllocation.scalingDownRatio") + .doubleConf + .checkValue(_ > 0, "The scaling down ratio of streaming dynamic " + + "allocation must be positive.") + .createWithDefault(0.3) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Tests.scala b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala new file mode 100644 index 0000000000000..21660ab3a9512 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Tests.scala @@ -0,0 +1,56 @@ +/* + * 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.internal.config + +private[spark] object Tests { + + val TEST_USE_COMPRESSED_OOPS_KEY = "spark.test.useCompressedOops" + + val TEST_MEMORY = ConfigBuilder("spark.testing.memory") + .longConf + .createWithDefault(Runtime.getRuntime.maxMemory) + + val TEST_SCHEDULE_INTERVAL = + ConfigBuilder("spark.testing.dynamicAllocation.scheduleInterval") + .longConf + .createWithDefault(100) + + val IS_TESTING = ConfigBuilder("spark.testing") + .booleanConf + .createOptional + + val TEST_NO_STAGE_RETRY = ConfigBuilder("spark.test.noStageRetry") + .booleanConf + .createWithDefault(false) + + val TEST_RESERVED_MEMORY = ConfigBuilder("spark.testing.reservedMemory") + .longConf + .createOptional + + val TEST_N_HOSTS = ConfigBuilder("spark.testing.nHosts") + .intConf + .createWithDefault(5) + + val TEST_N_EXECUTORS_HOST = ConfigBuilder("spark.testing.nExecutorsPerHost") + .intConf + .createWithDefault(4) + + val TEST_N_CORES_EXECUTOR = ConfigBuilder("spark.testing.nCoresPerExecutor") + .intConf + .createWithDefault(2) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/UI.scala b/core/src/main/scala/org/apache/spark/internal/config/UI.scala new file mode 100644 index 0000000000000..a11970ec73d88 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/UI.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.internal.config + +import java.util.concurrent.TimeUnit + +import org.apache.spark.network.util.ByteUnit + +private[spark] object UI { + + val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") + .doc("When true, show the progress bar in the console.") + .booleanConf + .createWithDefault(false) + + val UI_CONSOLE_PROGRESS_UPDATE_INTERVAL = + ConfigBuilder("spark.ui.consoleProgress.update.interval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(200) + + val UI_ENABLED = ConfigBuilder("spark.ui.enabled") + .doc("Whether to run the web UI for the Spark application.") + .booleanConf + .createWithDefault(true) + + val UI_PORT = ConfigBuilder("spark.ui.port") + .doc("Port for your application's dashboard, which shows memory and workload data.") + .intConf + .createWithDefault(4040) + + val UI_FILTERS = ConfigBuilder("spark.ui.filters") + .doc("Comma separated list of filter class names to apply to the Spark Web UI.") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_ALLOW_FRAMING_FROM = ConfigBuilder("spark.ui.allowFramingFrom") + .stringConf + .createOptional + + val UI_REVERSE_PROXY = ConfigBuilder("spark.ui.reverseProxy") + .doc("Enable running Spark Master as reverse proxy for worker and application UIs. " + + "In this mode, Spark master will reverse proxy the worker and application UIs to enable " + + "access without requiring direct access to their hosts. Use it with caution, as worker " + + "and application UI will not be accessible directly, you will only be able to access them" + + "through spark master/proxy public URL. This setting affects all the workers and " + + "application UIs running in the cluster and must be set on all the workers, drivers " + + " and masters.") + .booleanConf + .createWithDefault(false) + + val UI_REVERSE_PROXY_URL = ConfigBuilder("spark.ui.reverseProxyUrl") + .doc("This is the URL where your proxy is running. This URL is for proxy which is running " + + "in front of Spark Master. This is useful when running proxy for authentication e.g. " + + "OAuth proxy. Make sure this is a complete URL including scheme (http/https) and port to " + + "reach your proxy.") + .stringConf + .createOptional + + val UI_KILL_ENABLED = ConfigBuilder("spark.ui.killEnabled") + .doc("Allows jobs and stages to be killed from the web UI.") + .booleanConf + .createWithDefault(true) + + val UI_THREAD_DUMPS_ENABLED = ConfigBuilder("spark.ui.threadDumpsEnabled") + .booleanConf + .createWithDefault(true) + + val UI_X_XSS_PROTECTION = ConfigBuilder("spark.ui.xXssProtection") + .doc("Value for HTTP X-XSS-Protection response header") + .stringConf + .createWithDefaultString("1; mode=block") + + val UI_X_CONTENT_TYPE_OPTIONS = ConfigBuilder("spark.ui.xContentTypeOptions.enabled") + .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") + .booleanConf + .createWithDefault(true) + + val UI_STRICT_TRANSPORT_SECURITY = ConfigBuilder("spark.ui.strictTransportSecurity") + .doc("Value for HTTP Strict Transport Security Response Header") + .stringConf + .createOptional + + val UI_REQUEST_HEADER_SIZE = ConfigBuilder("spark.ui.requestHeaderSize") + .doc("Value for HTTP request header size in bytes.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("8k") + + val UI_TIMELINE_TASKS_MAXIMUM = ConfigBuilder("spark.ui.timeline.tasks.maximum") + .intConf + .createWithDefault(1000) + + val ACLS_ENABLE = ConfigBuilder("spark.acls.enable") + .booleanConf + .createWithDefault(false) + + val UI_VIEW_ACLS = ConfigBuilder("spark.ui.view.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val UI_VIEW_ACLS_GROUPS = ConfigBuilder("spark.ui.view.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS = ConfigBuilder("spark.admin.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val ADMIN_ACLS_GROUPS = ConfigBuilder("spark.admin.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS = ConfigBuilder("spark.modify.acls") + .stringConf + .toSequence + .createWithDefault(Nil) + + val MODIFY_ACLS_GROUPS = ConfigBuilder("spark.modify.acls.groups") + .stringConf + .toSequence + .createWithDefault(Nil) + + val USER_GROUPS_MAPPING = ConfigBuilder("spark.user.groups.mapping") + .stringConf + .createWithDefault("org.apache.spark.security.ShellBasedGroupsMappingProvider") + + val CUSTOM_EXECUTOR_LOG_URL = ConfigBuilder("spark.ui.custom.executor.log.url") + .doc("Specifies custom spark executor log url for supporting external log service instead of " + + "using cluster managers' application log urls in the Spark UI. Spark will support " + + "some path variables via patterns which can vary on cluster manager. Please check the " + + "documentation for your cluster manager to see which patterns are supported, if any. " + + "This configuration replaces original log urls in event log, which will be also effective " + + "when accessing the application on history server. The new log urls must be permanent, " + + "otherwise you might have dead link for executor log urls.") + .stringConf + .createOptional +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/Worker.scala b/core/src/main/scala/org/apache/spark/internal/config/Worker.scala new file mode 100644 index 0000000000000..f1eaae29f18df --- /dev/null +++ b/core/src/main/scala/org/apache/spark/internal/config/Worker.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.internal.config + +import java.util.concurrent.TimeUnit + +private[spark] object Worker { + val SPARK_WORKER_PREFIX = "spark.worker" + + val SPARK_WORKER_RESOURCE_FILE = + ConfigBuilder("spark.worker.resourcesFile") + .internal() + .doc("Path to a file containing the resources allocated to the worker. " + + "The file should be formatted as a JSON array of ResourceAllocation objects. " + + "Only used internally in standalone mode.") + .stringConf + .createOptional + + val WORKER_TIMEOUT = ConfigBuilder("spark.worker.timeout") + .longConf + .createWithDefault(60) + + val WORKER_DRIVER_TERMINATE_TIMEOUT = ConfigBuilder("spark.worker.driverTerminateTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10s") + + val WORKER_CLEANUP_ENABLED = ConfigBuilder("spark.worker.cleanup.enabled") + .booleanConf + .createWithDefault(false) + + val WORKER_CLEANUP_INTERVAL = ConfigBuilder("spark.worker.cleanup.interval") + .longConf + .createWithDefault(60 * 30) + + val APP_DATA_RETENTION = ConfigBuilder("spark.worker.cleanup.appDataTtl") + .longConf + .createWithDefault(7 * 24 * 3600) + + val PREFER_CONFIGURED_MASTER_ADDRESS = ConfigBuilder("spark.worker.preferConfiguredMasterAddress") + .booleanConf + .createWithDefault(false) + + val WORKER_UI_PORT = ConfigBuilder("spark.worker.ui.port") + .intConf + .createOptional + + val WORKER_UI_RETAINED_EXECUTORS = ConfigBuilder("spark.worker.ui.retainedExecutors") + .intConf + .createWithDefault(1000) + + val WORKER_UI_RETAINED_DRIVERS = ConfigBuilder("spark.worker.ui.retainedDrivers") + .intConf + .createWithDefault(1000) + + val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = + ConfigBuilder("spark.worker.ui.compressedLogFileLengthCacheSize") + .intConf + .createWithDefault(100) +} diff --git a/core/src/main/scala/org/apache/spark/internal/config/package.scala b/core/src/main/scala/org/apache/spark/internal/config/package.scala index e8b1d8859cc44..f2b88fe00cdf9 100644 --- a/core/src/main/scala/org/apache/spark/internal/config/package.scala +++ b/core/src/main/scala/org/apache/spark/internal/config/package.scala @@ -20,16 +20,38 @@ package org.apache.spark.internal import java.util.concurrent.TimeUnit import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.metrics.GarbageCollectionMetrics +import org.apache.spark.network.shuffle.Constants import org.apache.spark.network.util.ByteUnit +import org.apache.spark.scheduler.{EventLoggingListener, SchedulingMode} +import org.apache.spark.storage.{DefaultTopologyMapper, RandomBlockReplicationPolicy} +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils +import org.apache.spark.util.collection.unsafe.sort.UnsafeSorterSpillReader.MAX_BUFFER_SIZE_BYTES package object config { + private[spark] val SPARK_DRIVER_PREFIX = "spark.driver" + private[spark] val SPARK_EXECUTOR_PREFIX = "spark.executor" + private[spark] val SPARK_TASK_PREFIX = "spark.task" + + private[spark] val DRIVER_RESOURCES_FILE = + ConfigBuilder("spark.driver.resourcesFile") + .internal() + .doc("Path to a file containing the resources allocated to the driver. " + + "The file should be formatted as a JSON array of ResourceAllocation objects. " + + "Only used internally in standalone mode.") + .stringConf + .createOptional + private[spark] val DRIVER_CLASS_PATH = ConfigBuilder(SparkLauncher.DRIVER_EXTRA_CLASSPATH).stringConf.createOptional private[spark] val DRIVER_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS).stringConf.createOptional + ConfigBuilder(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS) + .withPrepended(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) + .stringConf + .createOptional private[spark] val DRIVER_LIBRARY_PATH = ConfigBuilder(SparkLauncher.DRIVER_EXTRA_LIBRARY_PATH).stringConf.createOptional @@ -37,17 +59,43 @@ package object config { private[spark] val DRIVER_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.driver.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val DRIVER_MEMORY = ConfigBuilder("spark.driver.memory") + private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") + .doc("Number of cores to use for the driver process, only in cluster mode.") + .intConf + .createWithDefault(1) + + private[spark] val DRIVER_MEMORY = ConfigBuilder(SparkLauncher.DRIVER_MEMORY) .doc("Amount of memory to use for the driver process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") private[spark] val DRIVER_MEMORY_OVERHEAD = ConfigBuilder("spark.driver.memoryOverhead") - .doc("The amount of off-heap memory to be allocated per driver in cluster mode, " + + .doc("The amount of non-heap memory to be allocated per driver in cluster mode, " + "in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val DRIVER_LOG_DFS_DIR = + ConfigBuilder("spark.driver.log.dfsDir").stringConf.createOptional + + private[spark] val DRIVER_LOG_LAYOUT = + ConfigBuilder("spark.driver.log.layout") + .stringConf + .createOptional + + private[spark] val DRIVER_LOG_PERSISTTODFS = + ConfigBuilder("spark.driver.log.persistToDfs.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val EVENT_LOG_ENABLED = ConfigBuilder("spark.eventLog.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val EVENT_LOG_DIR = ConfigBuilder("spark.eventLog.dir") + .stringConf + .createWithDefault(EventLoggingListener.DEFAULT_LOG_DIR) + private[spark] val EVENT_LOG_COMPRESS = ConfigBuilder("spark.eventLog.compress") .booleanConf @@ -58,6 +106,11 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_ALLOW_EC = + ConfigBuilder("spark.eventLog.allowErasureCoding") + .booleanConf + .createWithDefault(false) + private[spark] val EVENT_LOG_TESTING = ConfigBuilder("spark.eventLog.testing") .internal() @@ -74,12 +127,38 @@ package object config { .booleanConf .createWithDefault(false) + private[spark] val EVENT_LOG_PROCESS_TREE_METRICS = + ConfigBuilder("spark.eventLog.logStageExecutorProcessTreeMetrics.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS = + ConfigBuilder("spark.eventLog.gcMetrics.youngGenerationGarbageCollectors") + .doc("Names of supported young generation garbage collector. A name usually is " + + " the return of GarbageCollectorMXBean.getName. The built-in young generation garbage " + + s"collectors are ${GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + + private[spark] val EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS = + ConfigBuilder("spark.eventLog.gcMetrics.oldGenerationGarbageCollectors") + .doc("Names of supported old generation garbage collector. A name usually is " + + "the return of GarbageCollectorMXBean.getName. The built-in old generation garbage " + + s"collectors are ${GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS}") + .stringConf + .toSequence + .createWithDefault(GarbageCollectionMetrics.OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS) + private[spark] val EVENT_LOG_OVERWRITE = ConfigBuilder("spark.eventLog.overwrite").booleanConf.createWithDefault(false) private[spark] val EVENT_LOG_CALLSITE_LONG_FORM = ConfigBuilder("spark.eventLog.longForm.enabled").booleanConf.createWithDefault(false) + private[spark] val EXECUTOR_ID = + ConfigBuilder("spark.executor.id").stringConf.createOptional + private[spark] val EXECUTOR_CLASS_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_CLASSPATH).stringConf.createOptional @@ -98,7 +177,10 @@ package object config { ConfigBuilder("spark.executor.heartbeat.maxFailures").internal().intConf.createWithDefault(60) private[spark] val EXECUTOR_JAVA_OPTIONS = - ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS).stringConf.createOptional + ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_JAVA_OPTIONS) + .withPrepended(SparkLauncher.EXECUTOR_DEFAULT_JAVA_OPTIONS) + .stringConf + .createOptional private[spark] val EXECUTOR_LIBRARY_PATH = ConfigBuilder(SparkLauncher.EXECUTOR_EXTRA_LIBRARY_PATH).stringConf.createOptional @@ -106,17 +188,30 @@ package object config { private[spark] val EXECUTOR_USER_CLASS_PATH_FIRST = ConfigBuilder("spark.executor.userClassPathFirst").booleanConf.createWithDefault(false) - private[spark] val EXECUTOR_MEMORY = ConfigBuilder("spark.executor.memory") + private[spark] val EXECUTOR_CORES = ConfigBuilder(SparkLauncher.EXECUTOR_CORES) + .intConf + .createWithDefault(1) + + private[spark] val EXECUTOR_MEMORY = ConfigBuilder(SparkLauncher.EXECUTOR_MEMORY) .doc("Amount of memory to use per executor process, in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createWithDefaultString("1g") private[spark] val EXECUTOR_MEMORY_OVERHEAD = ConfigBuilder("spark.executor.memoryOverhead") - .doc("The amount of off-heap memory to be allocated per executor in cluster mode, " + + .doc("The amount of non-heap memory to be allocated per executor in cluster mode, " + "in MiB unless otherwise specified.") .bytesConf(ByteUnit.MiB) .createOptional + private[spark] val CORES_MAX = ConfigBuilder("spark.cores.max") + .doc("When running on a standalone deploy cluster or a Mesos cluster in coarse-grained " + + "sharing mode, the maximum amount of CPU cores to request for the application from across " + + "the cluster (not from each machine). If not set, the default will be " + + "`spark.deploy.defaultCores` on Spark's standalone cluster manager, or infinite " + + "(all available cores) on Mesos.") + .intConf + .createOptional + private[spark] val MEMORY_OFFHEAP_ENABLED = ConfigBuilder("spark.memory.offHeap.enabled") .doc("If true, Spark will attempt to use off-heap memory for certain operations. " + "If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive.") @@ -133,15 +228,121 @@ package object config { .checkValue(_ >= 0, "The off-heap memory size must not be negative") .createWithDefault(0) - private[spark] val PYSPARK_EXECUTOR_MEMORY = ConfigBuilder("spark.executor.pyspark.memory") - .bytesConf(ByteUnit.MiB) - .createOptional + private[spark] val MEMORY_STORAGE_FRACTION = ConfigBuilder("spark.memory.storageFraction") + .doc("Amount of storage memory immune to eviction, expressed as a fraction of the " + + "size of the region set aside by spark.memory.fraction. The higher this is, the " + + "less working memory may be available to execution and tasks may spill to disk more " + + "often. Leaving this at the default value is recommended. ") + .doubleConf + .checkValue(v => v >= 0.0 && v < 1.0, "Storage fraction must be in [0,1)") + .createWithDefault(0.5) + + private[spark] val MEMORY_FRACTION = ConfigBuilder("spark.memory.fraction") + .doc("Fraction of (heap space - 300MB) used for execution and storage. The " + + "lower this is, the more frequently spills and cached data eviction occur. " + + "The purpose of this config is to set aside memory for internal metadata, " + + "user data structures, and imprecise size estimation in the case of sparse, " + + "unusually large records. Leaving this at the default value is recommended. ") + .doubleConf + .createWithDefault(0.6) + + private[spark] val STORAGE_SAFETY_FRACTION = ConfigBuilder("spark.storage.safetyFraction") + .doubleConf + .createWithDefault(0.9) + + private[spark] val STORAGE_UNROLL_MEMORY_THRESHOLD = + ConfigBuilder("spark.storage.unrollMemoryThreshold") + .doc("Initial memory to request before unrolling any block") + .longConf + .createWithDefault(1024 * 1024) + + private[spark] val STORAGE_REPLICATION_PROACTIVE = + ConfigBuilder("spark.storage.replication.proactive") + .doc("Enables proactive block replication for RDD blocks. " + + "Cached RDD block replicas lost due to executor failures are replenished " + + "if there are any existing available replicas. This tries to " + + "get the replication level of the block to the initial number") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_MEMORY_MAP_THRESHOLD = + ConfigBuilder("spark.storage.memoryMapThreshold") + .doc("Size in bytes of a block above which Spark memory maps when " + + "reading a block from disk. " + + "This prevents Spark from memory mapping very small blocks. " + + "In general, memory mapping has high overhead for blocks close to or below " + + "the page size of the operating system.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("2m") + + private[spark] val STORAGE_REPLICATION_POLICY = + ConfigBuilder("spark.storage.replication.policy") + .stringConf + .createWithDefaultString(classOf[RandomBlockReplicationPolicy].getName) + + private[spark] val STORAGE_REPLICATION_TOPOLOGY_MAPPER = + ConfigBuilder("spark.storage.replication.topologyMapper") + .stringConf + .createWithDefaultString(classOf[DefaultTopologyMapper].getName) + + private[spark] val STORAGE_CACHED_PEERS_TTL = ConfigBuilder("spark.storage.cachedPeersTtl") + .intConf.createWithDefault(60 * 1000) + + private[spark] val STORAGE_MAX_REPLICATION_FAILURE = + ConfigBuilder("spark.storage.maxReplicationFailures") + .intConf.createWithDefault(1) + + private[spark] val STORAGE_REPLICATION_TOPOLOGY_FILE = + ConfigBuilder("spark.storage.replication.topologyFile").stringConf.createOptional + + private[spark] val STORAGE_EXCEPTION_PIN_LEAK = + ConfigBuilder("spark.storage.exceptionOnPinLeak") + .booleanConf + .createWithDefault(false) + + private[spark] val STORAGE_BLOCKMANAGER_TIMEOUTINTERVAL = + ConfigBuilder("spark.storage.blockManagerTimeoutIntervalMs") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("60s") + + private[spark] val STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT = + ConfigBuilder("spark.storage.blockManagerSlaveTimeoutMs") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString(Network.NETWORK_TIMEOUT.defaultValueString) + + private[spark] val STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT = + ConfigBuilder("spark.storage.cleanupFilesAfterExecutorExit") + .doc("Whether or not cleanup the files not served by the external shuffle service " + + "on executor exits.") + .booleanConf + .createWithDefault(true) + + private[spark] val DISKSTORE_SUB_DIRECTORIES = + ConfigBuilder("spark.diskStore.subDirectories") + .doc("Number of subdirectories inside each path listed in spark.local.dir for " + + "hashing Block files into.") + .intConf + .checkValue(_ > 0, "The number of subdirectories must be positive.") + .createWithDefault(64) + + private[spark] val BLOCK_FAILURES_BEFORE_LOCATION_REFRESH = + ConfigBuilder("spark.block.failures.beforeLocationRefresh") + .doc("Max number of failures before this block manager refreshes " + + "the block locations from the driver.") + .intConf + .createWithDefault(5) private[spark] val IS_PYTHON_APP = ConfigBuilder("spark.yarn.isPython").internal() .booleanConf.createWithDefault(false) private[spark] val CPUS_PER_TASK = ConfigBuilder("spark.task.cpus").intConf.createWithDefault(1) + private[spark] val DYN_ALLOCATION_ENABLED = + ConfigBuilder("spark.dynamicAllocation.enabled").booleanConf.createWithDefault(false) + + private[spark] val DYN_ALLOCATION_TESTING = + ConfigBuilder("spark.dynamicAllocation.testing").booleanConf.createWithDefault(false) + private[spark] val DYN_ALLOCATION_MIN_EXECUTORS = ConfigBuilder("spark.dynamicAllocation.minExecutors").intConf.createWithDefault(0) @@ -156,6 +357,37 @@ package object config { ConfigBuilder("spark.dynamicAllocation.executorAllocationRatio") .doubleConf.createWithDefault(1.0) + private[spark] val DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.cachedExecutorIdleTimeout") + .timeConf(TimeUnit.SECONDS) + .checkValue(_ >= 0L, "Timeout must be >= 0.") + .createWithDefault(Integer.MAX_VALUE) + + private[spark] val DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.executorIdleTimeout") + .timeConf(TimeUnit.SECONDS) + .checkValue(_ >= 0L, "Timeout must be >= 0.") + .createWithDefault(60) + + private[spark] val DYN_ALLOCATION_SHUFFLE_TRACKING = + ConfigBuilder("spark.dynamicAllocation.shuffleTracking.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val DYN_ALLOCATION_SHUFFLE_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.shuffleTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(_ >= 0L, "Timeout must be >= 0.") + .createWithDefault(Long.MaxValue) + + private[spark] val DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.schedulerBacklogTimeout") + .timeConf(TimeUnit.SECONDS).createWithDefault(1) + + private[spark] val DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT = + ConfigBuilder("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout") + .fallbackConf(DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT) + private[spark] val LOCALITY_WAIT = ConfigBuilder("spark.locality.wait") .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("3s") @@ -163,6 +395,22 @@ package object config { private[spark] val SHUFFLE_SERVICE_ENABLED = ConfigBuilder("spark.shuffle.service.enabled").booleanConf.createWithDefault(false) + private[spark] val SHUFFLE_SERVICE_FETCH_RDD_ENABLED = + ConfigBuilder(Constants.SHUFFLE_SERVICE_FETCH_RDD_ENABLED) + .doc("Whether to use the ExternalShuffleService for fetching disk persisted RDD blocks. " + + "In case of dynamic allocation if this feature is enabled executors having only disk " + + "persisted blocks are considered idle after " + + "'spark.dynamicAllocation.executorIdleTimeout' and will be released accordingly.") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_SERVICE_DB_ENABLED = + ConfigBuilder("spark.shuffle.service.db.enabled") + .doc("Whether to use db in ExternalShuffleService. Note that this only affects " + + "standalone mode.") + .booleanConf + .createWithDefault(true) + private[spark] val SHUFFLE_SERVICE_PORT = ConfigBuilder("spark.shuffle.service.port").intConf.createWithDefault(7337) @@ -174,6 +422,28 @@ package object config { .doc("Name of the Kerberos principal.") .stringConf.createOptional + private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.kerberos.relogin.period") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("1m") + + private[spark] val KERBEROS_RENEWAL_CREDENTIALS = + ConfigBuilder("spark.kerberos.renewal.credentials") + .doc( + "Which credentials to use when renewing delegation tokens for executors. Can be either " + + "'keytab', the default, which requires a keytab to be provided, or 'ccache', which uses " + + "the local credentials cache.") + .stringConf + .checkValues(Set("keytab", "ccache")) + .createWithDefault("keytab") + + private[spark] val KERBEROS_FILESYSTEMS_TO_ACCESS = + ConfigBuilder("spark.kerberos.access.hadoopFileSystems") + .doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " + + "that hosts fs.defaultFS does not need to be listed here.") + .stringConf + .toSequence + .createWithDefault(Nil) + private[spark] val EXECUTOR_INSTANCES = ConfigBuilder("spark.executor.instances") .intConf .createOptional @@ -184,11 +454,36 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val MAX_TASK_FAILURES = + private[spark] val TASK_MAX_DIRECT_RESULT_SIZE = + ConfigBuilder("spark.task.maxDirectResultSize") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(1L << 20) + + private[spark] val TASK_MAX_FAILURES = ConfigBuilder("spark.task.maxFailures") .intConf .createWithDefault(4) + private[spark] val TASK_REAPER_ENABLED = + ConfigBuilder("spark.task.reaper.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val TASK_REAPER_KILL_TIMEOUT = + ConfigBuilder("spark.task.reaper.killTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(-1) + + private[spark] val TASK_REAPER_POLLING_INTERVAL = + ConfigBuilder("spark.task.reaper.pollingInterval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10s") + + private[spark] val TASK_REAPER_THREAD_DUMP = + ConfigBuilder("spark.task.reaper.threadDump") + .booleanConf + .createWithDefault(true) + // Blacklist confs private[spark] val BLACKLIST_ENABLED = ConfigBuilder("spark.blacklist.enabled") @@ -258,7 +553,7 @@ package object config { private[spark] val LISTENER_BUS_EVENT_QUEUE_CAPACITY = ConfigBuilder("spark.scheduler.listenerbus.eventqueue.capacity") .intConf - .checkValue(_ > 0, "The capacity of listener bus event queue must not be negative") + .checkValue(_ > 0, "The capacity of listener bus event queue must be positive") .createWithDefault(10000) private[spark] val LISTENER_BUS_METRICS_MAX_LISTENER_CLASSES_TIMED = @@ -272,6 +567,10 @@ package object config { .stringConf .createOptional + private[spark] val METRICS_CONF = ConfigBuilder("spark.metrics.conf") + .stringConf + .createOptional + private[spark] val PYSPARK_DRIVER_PYTHON = ConfigBuilder("spark.pyspark.driver.python") .stringConf .createOptional @@ -284,11 +583,6 @@ package object config { private[spark] val HISTORY_UI_MAX_APPS = ConfigBuilder("spark.history.ui.maxApplications").intConf.createWithDefault(Integer.MAX_VALUE) - private[spark] val UI_SHOW_CONSOLE_PROGRESS = ConfigBuilder("spark.ui.showConsoleProgress") - .doc("When true, show the progress bar in the console.") - .booleanConf - .createWithDefault(false) - private[spark] val IO_ENCRYPTION_ENABLED = ConfigBuilder("spark.io.encryption.enabled") .booleanConf .createWithDefault(false) @@ -314,6 +608,17 @@ package object config { .stringConf .createWithDefault(Utils.localCanonicalHostName()) + private[spark] val DRIVER_PORT = ConfigBuilder("spark.driver.port") + .doc("Port of driver endpoints.") + .intConf + .createWithDefault(0) + + private[spark] val DRIVER_SUPERVISE = ConfigBuilder("spark.driver.supervise") + .doc("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.") + .booleanConf + .createWithDefault(false) + private[spark] val DRIVER_BIND_ADDRESS = ConfigBuilder("spark.driver.bindAddress") .doc("Address where to bind network listen sockets on the driver.") .fallbackConf(DRIVER_HOST_ADDRESS) @@ -336,7 +641,7 @@ package object config { private[spark] val IGNORE_MISSING_FILES = ConfigBuilder("spark.files.ignoreMissingFiles") .doc("Whether to ignore missing files. If true, the Spark jobs will continue to run when " + - "encountering missing files and the contents that have been read will still be returned.") + "encountering missing files and the contents that have been read will still be returned.") .booleanConf .createWithDefault(false) @@ -346,7 +651,7 @@ package object config { private[spark] val FILES_MAX_PARTITION_BYTES = ConfigBuilder("spark.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") - .longConf + .bytesConf(ByteUnit.BYTE) .createWithDefault(128 * 1024 * 1024) private[spark] val FILES_OPEN_COST_IN_BYTES = ConfigBuilder("spark.files.openCostInBytes") @@ -354,7 +659,7 @@ package object config { " the same time. This is used when putting multiple files into a partition. It's better to" + " over estimate, then the partitions with small files will be faster than partitions with" + " bigger files.") - .longConf + .bytesConf(ByteUnit.BYTE) .createWithDefault(4 * 1024 * 1024) private[spark] val HADOOP_RDD_IGNORE_EMPTY_SPLITS = @@ -371,7 +676,7 @@ package object config { "a property key or value, the value is redacted from the environment UI and various logs " + "like YARN and event logs.") .regexConf - .createWithDefault("(?i)secret|password".r) + .createWithDefault("(?i)secret|password|token".r) private[spark] val STRING_REDACTION_PATTERN = ConfigBuilder("spark.redaction.string.regex") @@ -381,6 +686,11 @@ package object config { .regexConf .createOptional + private[spark] val AUTH_SECRET = + ConfigBuilder("spark.authenticate.secret") + .stringConf + .createOptional + private[spark] val AUTH_SECRET_BIT_LENGTH = ConfigBuilder("spark.authenticate.secretBitLength") .intConf @@ -396,18 +706,45 @@ package object config { .booleanConf .createWithDefault(false) - private[spark] val NETWORK_ENCRYPTION_ENABLED = - ConfigBuilder("spark.network.crypto.enabled") - .booleanConf - .createWithDefault(false) + private[spark] val AUTH_SECRET_FILE = + ConfigBuilder("spark.authenticate.secret.file") + .doc("Path to a file that contains the authentication secret to use. The secret key is " + + "loaded from this path on both the driver and the executors if overrides are not set for " + + "either entity (see below). File-based secret keys are only allowed when using " + + "Kubernetes.") + .stringConf + .createOptional + + private[spark] val AUTH_SECRET_FILE_DRIVER = + ConfigBuilder("spark.authenticate.secret.driver.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "driver. In Kubernetes client mode it is often useful to set a different secret " + + "path for the driver vs. the executors, since the driver may not be running in " + + "a pod unlike the executors. If this is set, an accompanying secret file must " + + "be specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) + + private[spark] val AUTH_SECRET_FILE_EXECUTOR = + ConfigBuilder("spark.authenticate.secret.executor.file") + .doc("Path to a file that contains the authentication secret to use. Loaded by the " + + "executors only. In Kubernetes client mode it is often useful to set a different " + + "secret path for the driver vs. the executors, since the driver may not be running " + + "in a pod unlike the executors. If this is set, an accompanying secret file must be " + + "specified for the executors. The fallback configuration allows the same path to be " + + "used for both the driver and the executors when running in cluster mode. File-based " + + "secret keys are only allowed when using Kubernetes.") + .fallbackConf(AUTH_SECRET_FILE) private[spark] val BUFFER_WRITE_CHUNK_SIZE = ConfigBuilder("spark.buffer.write.chunkSize") .internal() .doc("The chunk size in bytes during writing out the bytes of ChunkedByteBuffer.") .bytesConf(ByteUnit.BYTE) - .checkValue(_ <= Int.MaxValue, "The chunk size during writing out the bytes of" + - " ChunkedByteBuffer should not larger than Int.MaxValue.") + .checkValue(_ <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "The chunk size during writing out the bytes of ChunkedByteBuffer should" + + s" be less than or equal to ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") .createWithDefault(64 * 1024 * 1024) private[spark] val CHECKPOINT_COMPRESS = @@ -452,17 +789,19 @@ package object config { private[spark] val MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM = ConfigBuilder("spark.maxRemoteBlockSizeFetchToMem") .doc("Remote block will be fetched to disk when size of the block is above this threshold " + - "in bytes. This is to avoid a giant request takes too much memory. We can enable this " + - "config by setting a specific value(e.g. 200m). Note this configuration will affect " + - "both shuffle fetch and block manager remote block fetch. For users who enabled " + - "external shuffle service, this feature can only be worked when external shuffle" + - "service is newer than Spark 2.2.") + "in bytes. This is to avoid a giant request takes too much memory. Note this " + + "configuration will affect both shuffle fetch and block manager remote block fetch. " + + "For users who enabled external shuffle service, this feature can only work when " + + "external shuffle service is at least 2.3.0.") .bytesConf(ByteUnit.BYTE) // fetch-to-mem is guaranteed to fail if the message is bigger than 2 GB, so we might // as well use fetch-to-disk in that case. The message includes some metadata in addition // to the block data itself (in particular UploadBlock has a lot of metadata), so we leave // extra room. - .createWithDefault(Int.MaxValue - 512) + .checkValue( + _ <= Int.MaxValue - 512, + "maxRemoteBlockSizeFetchToMem cannot be larger than (Int.MaxValue - 512) bytes.") + .createWithDefaultString("200m") private[spark] val TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES = ConfigBuilder("spark.taskMetrics.trackUpdatedBlockStatuses") @@ -478,8 +817,9 @@ package object config { "otherwise specified. These buffers reduce the number of disk seeks and system calls " + "made in creating intermediate shuffle files.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, - s"The file buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, + s"The file buffer size must be positive and less than or equal to" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_UNSAFE_FILE_OUTPUT_BUFFER_SIZE = @@ -487,16 +827,18 @@ package object config { .doc("The file system for this buffer size after each partition " + "is written in unsafe shuffle writer. In KiB unless otherwise specified.") .bytesConf(ByteUnit.KiB) - .checkValue(v => v > 0 && v <= Int.MaxValue / 1024, - s"The buffer size must be greater than 0 and less than ${Int.MaxValue / 1024}.") + .checkValue(v => v > 0 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024, + s"The buffer size must be positive and less than or equal to" + + s" ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH / 1024}.") .createWithDefaultString("32k") private[spark] val SHUFFLE_DISK_WRITE_BUFFER_SIZE = ConfigBuilder("spark.shuffle.spill.diskWriteBufferSize") .doc("The buffer size, in bytes, to use when writing the sorted records to an on-disk file.") .bytesConf(ByteUnit.BYTE) - .checkValue(v => v > 0 && v <= Int.MaxValue, - s"The buffer size must be greater than 0 and less than ${Int.MaxValue}.") + .checkValue(v => v > 12 && v <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + s"The buffer size must be greater than 12 and less than or equal to " + + s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") .createWithDefault(1024 * 1024) private[spark] val UNROLL_MEMORY_CHECK_PERIOD = @@ -525,24 +867,6 @@ package object config { .toSequence .createWithDefault(Nil) - private[spark] val UI_X_XSS_PROTECTION = - ConfigBuilder("spark.ui.xXssProtection") - .doc("Value for HTTP X-XSS-Protection response header") - .stringConf - .createWithDefaultString("1; mode=block") - - private[spark] val UI_X_CONTENT_TYPE_OPTIONS = - ConfigBuilder("spark.ui.xContentTypeOptions.enabled") - .doc("Set to 'true' for setting X-Content-Type-Options HTTP response header to 'nosniff'") - .booleanConf - .createWithDefault(true) - - private[spark] val UI_STRICT_TRANSPORT_SECURITY = - ConfigBuilder("spark.ui.strictTransportSecurity") - .doc("Value for HTTP Strict Transport Security Response Header") - .stringConf - .createOptional - private[spark] val EXTRA_LISTENERS = ConfigBuilder("spark.extraListeners") .doc("Class names of listeners to add to SparkContext during initialization.") .stringConf @@ -586,6 +910,106 @@ package object config { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("1h") + private[spark] val SHUFFLE_SORT_INIT_BUFFER_SIZE = + ConfigBuilder("spark.shuffle.sort.initialBufferSize") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v > 0 && v <= Int.MaxValue, + s"The buffer size must be greater than 0 and less than or equal to ${Int.MaxValue}.") + .createWithDefault(4096) + + private[spark] val SHUFFLE_COMPRESS = + ConfigBuilder("spark.shuffle.compress") + .doc("Whether to compress shuffle output. Compression will use " + + "spark.io.compression.codec.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SPILL_COMPRESS = + ConfigBuilder("spark.shuffle.spill.compress") + .doc("Whether to compress data spilled during shuffles. Compression will use " + + "spark.io.compression.codec.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD = + ConfigBuilder("spark.shuffle.spill.initialMemoryThreshold") + .internal() + .doc("Initial threshold for the size of a collection before we start tracking its " + + "memory usage.") + .bytesConf(ByteUnit.BYTE) + .createWithDefault(5 * 1024 * 1024) + + private[spark] val SHUFFLE_SPILL_BATCH_SIZE = + ConfigBuilder("spark.shuffle.spill.batchSize") + .internal() + .doc("Size of object batches when reading/writing from serializers.") + .longConf + .createWithDefault(10000) + + private[spark] val SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD = + ConfigBuilder("spark.shuffle.sort.bypassMergeThreshold") + .doc("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") + .intConf + .createWithDefault(200) + + private[spark] val SHUFFLE_MANAGER = + ConfigBuilder("spark.shuffle.manager") + .stringConf + .createWithDefault("sort") + + private[spark] val SHUFFLE_REDUCE_LOCALITY_ENABLE = + ConfigBuilder("spark.shuffle.reduceLocality.enabled") + .doc("Whether to compute locality preferences for reduce tasks") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST = + ConfigBuilder("spark.shuffle.mapOutput.minSizeForBroadcast") + .doc("The size at which we use Broadcast to send the map output statuses to the executors.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("512k") + + private[spark] val SHUFFLE_MAPOUTPUT_DISPATCHER_NUM_THREADS = + ConfigBuilder("spark.shuffle.mapOutput.dispatcher.numThreads") + .intConf + .createWithDefault(8) + + private[spark] val SHUFFLE_DETECT_CORRUPT = + ConfigBuilder("spark.shuffle.detectCorrupt") + .doc("Whether to detect any corruption in fetched blocks.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_DETECT_CORRUPT_MEMORY = + ConfigBuilder("spark.shuffle.detectCorrupt.useExtraMemory") + .doc("If enabled, part of a compressed/encrypted stream will be de-compressed/de-crypted " + + "by using extra memory to detect early corruption. Any IOException thrown will cause " + + "the task to be retried once and if it fails again with same exception, then " + + "FetchFailedException will be thrown to retry previous stage") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_SYNC = + ConfigBuilder("spark.shuffle.sync") + .doc("Whether to force outstanding writes to disk.") + .booleanConf + .createWithDefault(false) + + private[spark] val SHUFFLE_UNDAFE_FAST_MERGE_ENABLE = + ConfigBuilder("spark.shuffle.unsafe.fastMergeEnabled") + .doc("Whether to perform a fast spill merge.") + .booleanConf + .createWithDefault(true) + + private[spark] val SHUFFLE_SORT_USE_RADIXSORT = + ConfigBuilder("spark.shuffle.sort.useRadixSort") + .doc("Whether to use radix sort for sorting in-memory partition ids. Radix sort is much " + + "faster, but requires additional memory to be reserved memory as pointers are added.") + .booleanConf + .createWithDefault(true) + private[spark] val SHUFFLE_MIN_NUM_PARTS_TO_HIGHLY_COMPRESS = ConfigBuilder("spark.shuffle.minNumPartitionsToHighlyCompress") .internal() @@ -599,7 +1023,7 @@ package object config { .internal() .doc("For testing only, controls the size of chunks when memory mapping a file") .bytesConf(ByteUnit.BYTE) - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) private[spark] val BARRIER_SYNC_TIMEOUT = ConfigBuilder("spark.barrier.sync.timeout") @@ -611,6 +1035,14 @@ package object config { .checkValue(v => v > 0, "The value should be a positive time value.") .createWithDefaultString("365d") + private[spark] val UNSCHEDULABLE_TASKSET_TIMEOUT = + ConfigBuilder("spark.scheduler.blacklist.unschedulableTaskSetTimeout") + .doc("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.") + .timeConf(TimeUnit.SECONDS) + .checkValue(v => v >= 0, "The value should be a non negative time value.") + .createWithDefault(120) + private[spark] val BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL = ConfigBuilder("spark.scheduler.barrier.maxConcurrentTasksCheck.interval") .doc("Time in seconds to wait between a max concurrent tasks check failure and the next " + @@ -638,6 +1070,26 @@ package object config { .checkValue(v => v > 0, "The max failures should be a positive value.") .createWithDefault(40) + private[spark] val UNSAFE_EXCEPTION_ON_MEMORY_LEAK = + ConfigBuilder("spark.unsafe.exceptionOnMemoryLeak") + .internal() + .booleanConf + .createWithDefault(false) + + private[spark] val UNSAFE_SORTER_SPILL_READ_AHEAD_ENABLED = + ConfigBuilder("spark.unsafe.sorter.spill.read.ahead.enabled") + .internal() + .booleanConf + .createWithDefault(true) + + private[spark] val UNSAFE_SORTER_SPILL_READER_BUFFER_SIZE = + ConfigBuilder("spark.unsafe.sorter.spill.reader.buffer.size") + .internal() + .bytesConf(ByteUnit.BYTE) + .checkValue(v => 1024 * 1024 <= v && v <= MAX_BUFFER_SIZE_BYTES, + s"The value must be in allowed range [1,048,576, ${MAX_BUFFER_SIZE_BYTES}].") + .createWithDefault(1024 * 1024) + private[spark] val EXECUTOR_PLUGINS = ConfigBuilder("spark.executor.plugins") .doc("Comma-separated list of class names for \"plugins\" implementing " + @@ -647,4 +1099,275 @@ package object config { .stringConf .toSequence .createWithDefault(Nil) + + private[spark] val CLEANER_PERIODIC_GC_INTERVAL = + ConfigBuilder("spark.cleaner.periodicGC.interval") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("30min") + + private[spark] val CLEANER_REFERENCE_TRACKING = + ConfigBuilder("spark.cleaner.referenceTracking") + .booleanConf + .createWithDefault(true) + + private[spark] val CLEANER_REFERENCE_TRACKING_BLOCKING = + ConfigBuilder("spark.cleaner.referenceTracking.blocking") + .booleanConf + .createWithDefault(true) + + private[spark] val CLEANER_REFERENCE_TRACKING_BLOCKING_SHUFFLE = + ConfigBuilder("spark.cleaner.referenceTracking.blocking.shuffle") + .booleanConf + .createWithDefault(false) + + private[spark] val CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS = + ConfigBuilder("spark.cleaner.referenceTracking.cleanCheckpoints") + .booleanConf + .createWithDefault(false) + + private[spark] val EXECUTOR_LOGS_ROLLING_STRATEGY = + ConfigBuilder("spark.executor.logs.rolling.strategy").stringConf.createWithDefault("") + + private[spark] val EXECUTOR_LOGS_ROLLING_TIME_INTERVAL = + ConfigBuilder("spark.executor.logs.rolling.time.interval").stringConf.createWithDefault("daily") + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_SIZE = + ConfigBuilder("spark.executor.logs.rolling.maxSize") + .stringConf + .createWithDefault((1024 * 1024).toString) + + private[spark] val EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES = + ConfigBuilder("spark.executor.logs.rolling.maxRetainedFiles").intConf.createWithDefault(-1) + + private[spark] val EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION = + ConfigBuilder("spark.executor.logs.rolling.enableCompression") + .booleanConf + .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_ENABLED = ConfigBuilder("spark.master.rest.enabled") + .booleanConf + .createWithDefault(false) + + private[spark] val MASTER_REST_SERVER_PORT = ConfigBuilder("spark.master.rest.port") + .intConf + .createWithDefault(6066) + + private[spark] val MASTER_UI_PORT = ConfigBuilder("spark.master.ui.port") + .intConf + .createWithDefault(8080) + + private[spark] val IO_COMPRESSION_SNAPPY_BLOCKSIZE = + ConfigBuilder("spark.io.compression.snappy.blockSize") + .doc("Block size in bytes used in Snappy compression, in the case when " + + "Snappy compression codec is used. Lowering this block size " + + "will also lower shuffle memory usage when Snappy is used") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_LZ4_BLOCKSIZE = + ConfigBuilder("spark.io.compression.lz4.blockSize") + .doc("Block size in bytes used in LZ4 compression, in the case when LZ4 compression" + + "codec is used. Lowering this block size will also lower shuffle memory " + + "usage when LZ4 is used.") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_CODEC = + ConfigBuilder("spark.io.compression.codec") + .doc("The codec used to compress internal data such as RDD partitions, event log, " + + "broadcast variables and shuffle outputs. By default, Spark provides four codecs: " + + "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + + "the codec") + .stringConf + .createWithDefaultString("lz4") + + private[spark] val IO_COMPRESSION_ZSTD_BUFFERSIZE = + ConfigBuilder("spark.io.compression.zstd.bufferSize") + .doc("Buffer size in bytes used in Zstd compression, in the case when Zstd " + + "compression codec 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") + .bytesConf(ByteUnit.BYTE) + .createWithDefaultString("32k") + + private[spark] val IO_COMPRESSION_ZSTD_LEVEL = + ConfigBuilder("spark.io.compression.zstd.level") + .doc("Compression level for Zstd compression codec. Increasing the compression " + + "level will result in better compression at the expense of more CPU and memory") + .intConf + .createWithDefault(1) + + private[spark] val EVENT_LOG_COMPRESSION_CODEC = + ConfigBuilder("spark.eventLog.compression.codec") + .doc("The codec used to compress event log. By default, Spark provides four codecs: " + + "lz4, lzf, snappy, and zstd. You can also use fully qualified class names to specify " + + "the codec. If this is not given, spark.io.compression.codec will be used.") + .fallbackConf(IO_COMPRESSION_CODEC) + + private[spark] val BUFFER_SIZE = + ConfigBuilder("spark.buffer.size") + .intConf + .checkValue(_ >= 0, "The buffer size must not be negative") + .createWithDefault(65536) + + private[spark] val LOCALITY_WAIT_PROCESS = ConfigBuilder("spark.locality.wait.process") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_NODE = ConfigBuilder("spark.locality.wait.node") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val LOCALITY_WAIT_RACK = ConfigBuilder("spark.locality.wait.rack") + .fallbackConf(LOCALITY_WAIT) + + private[spark] val REDUCER_MAX_SIZE_IN_FLIGHT = ConfigBuilder("spark.reducer.maxSizeInFlight") + .doc("Maximum size of map outputs to fetch simultaneously from each reduce task, " + + "in MiB unless otherwise specified. Since each output requires us to create a " + + "buffer to receive it, this represents a fixed memory overhead per reduce task, " + + "so keep it small unless you have a large amount of memory") + .bytesConf(ByteUnit.MiB) + .createWithDefaultString("48m") + + private[spark] val REDUCER_MAX_REQS_IN_FLIGHT = ConfigBuilder("spark.reducer.maxReqsInFlight") + .doc("This configuration limits the number of remote requests to fetch blocks at " + + "any given point. When the number of hosts in the cluster increase, " + + "it might lead to very large number 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") + .intConf + .createWithDefault(Int.MaxValue) + + private[spark] val BROADCAST_COMPRESS = ConfigBuilder("spark.broadcast.compress") + .doc("Whether to compress broadcast variables before sending them. " + + "Generally a good idea. Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(true) + + private[spark] val BROADCAST_BLOCKSIZE = ConfigBuilder("spark.broadcast.blockSize") + .doc("Size of each piece of a block for TorrentBroadcastFactory, in " + + "KiB unless otherwise specified. Too large a value decreases " + + "parallelism during broadcast (makes it slower); however, " + + "if it is too small, BlockManager might take a performance hit") + .bytesConf(ByteUnit.KiB) + .createWithDefaultString("4m") + + private[spark] val BROADCAST_CHECKSUM = ConfigBuilder("spark.broadcast.checksum") + .doc("Whether to enable checksum for broadcast. If enabled, " + + "broadcasts will include a checksum, which can 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") + .booleanConf.createWithDefault(true) + + private[spark] val BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD = + ConfigBuilder("spark.broadcast.UDFCompressionThreshold") + .doc("The threshold at which user-defined functions (UDFs) and Python RDD commands " + + "are compressed by broadcast in bytes unless otherwise specified") + .bytesConf(ByteUnit.BYTE) + .checkValue(v => v >= 0, "The threshold should be non-negative.") + .createWithDefault(1L * 1024 * 1024) + + private[spark] val RDD_COMPRESS = ConfigBuilder("spark.rdd.compress") + .doc("Whether to compress serialized RDD partitions " + + "(e.g. for StorageLevel.MEMORY_ONLY_SER in Scala " + + "or StorageLevel.MEMORY_ONLY in Python). Can save substantial " + + "space at the cost of some extra CPU time. " + + "Compression will use spark.io.compression.codec") + .booleanConf.createWithDefault(false) + + private[spark] val RDD_PARALLEL_LISTING_THRESHOLD = + ConfigBuilder("spark.rdd.parallelListingThreshold") + .intConf + .createWithDefault(10) + + private[spark] val RDD_LIMIT_SCALE_UP_FACTOR = + ConfigBuilder("spark.rdd.limit.scaleUpFactor") + .intConf + .createWithDefault(4) + + private[spark] val SERIALIZER = ConfigBuilder("spark.serializer") + .stringConf + .createWithDefault("org.apache.spark.serializer.JavaSerializer") + + private[spark] val SERIALIZER_OBJECT_STREAM_RESET = + ConfigBuilder("spark.serializer.objectStreamReset") + .intConf + .createWithDefault(100) + + private[spark] val SERIALIZER_EXTRA_DEBUG_INFO = ConfigBuilder("spark.serializer.extraDebugInfo") + .booleanConf + .createWithDefault(true) + + private[spark] val JARS = ConfigBuilder("spark.jars") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val FILES = ConfigBuilder("spark.files") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val SUBMIT_DEPLOY_MODE = ConfigBuilder("spark.submit.deployMode") + .stringConf + .createWithDefault("client") + + private[spark] val SUBMIT_PYTHON_FILES = ConfigBuilder("spark.submit.pyFiles") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val SCHEDULER_ALLOCATION_FILE = + ConfigBuilder("spark.scheduler.allocation.file") + .stringConf + .createOptional + + private[spark] val SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO = + ConfigBuilder("spark.scheduler.minRegisteredResourcesRatio") + .doubleConf + .createOptional + + private[spark] val SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME = + ConfigBuilder("spark.scheduler.maxRegisteredResourcesWaitingTime") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("30s") + + private[spark] val SCHEDULER_MODE = + ConfigBuilder("spark.scheduler.mode") + .stringConf + .createWithDefault(SchedulingMode.FIFO.toString) + + private[spark] val SCHEDULER_REVIVE_INTERVAL = + ConfigBuilder("spark.scheduler.revive.interval") + .timeConf(TimeUnit.MILLISECONDS) + .createOptional + + private[spark] val SPECULATION_ENABLED = + ConfigBuilder("spark.speculation") + .booleanConf + .createWithDefault(false) + + private[spark] val SPECULATION_INTERVAL = + ConfigBuilder("spark.speculation.interval") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefault(100) + + private[spark] val SPECULATION_MULTIPLIER = + ConfigBuilder("spark.speculation.multiplier") + .doubleConf + .createWithDefault(1.5) + + private[spark] val SPECULATION_QUANTILE = + ConfigBuilder("spark.speculation.quantile") + .doubleConf + .createWithDefault(0.75) + + private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") + .doc("Staging directory used while submitting applications.") + .stringConf + .createOptional + + private[spark] val BUFFER_PAGESIZE = ConfigBuilder("spark.buffer.pageSize") + .doc("The amount of memory used per page in bytes") + .bytesConf(ByteUnit.BYTE) + .createOptional + } diff --git a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala index e6e9c9e328853..854093851f5d0 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/FileCommitProtocol.scala @@ -149,7 +149,7 @@ object FileCommitProtocol extends Logging { logDebug(s"Creating committer $className; job $jobId; output=$outputPath;" + s" dynamic=$dynamicPartitionOverwrite") - val clazz = Utils.classForName(className).asInstanceOf[Class[FileCommitProtocol]] + val clazz = Utils.classForName[FileCommitProtocol](className) // First try the constructor with arguments (jobId: String, outputPath: String, // dynamicPartitionOverwrite: Boolean). // If that doesn't exist, try the one with (jobId: string, outputPath: String). diff --git a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala index 3e60c50ada59b..7477e03bfaa76 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/HadoopMapReduceCommitProtocol.scala @@ -91,7 +91,7 @@ class HadoopMapReduceCommitProtocol( private def stagingDir = new Path(path, ".spark-staging-" + jobId) protected def setupCommitter(context: TaskAttemptContext): OutputCommitter = { - val format = context.getOutputFormatClass.newInstance() + val format = context.getOutputFormatClass.getConstructor().newInstance() // If OutputFormat is Configurable, we should set conf to it. format match { case c: Configurable => c.setConf(context.getConfiguration) diff --git a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala index 9ebd0aa301592..a619f10bbf064 100644 --- a/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/internal/io/SparkHadoopWriter.scala @@ -116,12 +116,14 @@ object SparkHadoopWriter extends Logging { jobTrackerId, commitJobId, sparkPartitionId, sparkAttemptNumber) committer.setupTask(taskContext) - val (outputMetrics, callback) = initHadoopOutputMetrics(context) - // Initiate the writer. config.initWriter(taskContext, sparkPartitionId) var recordsWritten = 0L + // We must initialize the callback for calculating bytes written after the statistic table + // is initialized in FileSystem which is happened in initWriter. + val (outputMetrics, callback) = initHadoopOutputMetrics(context) + // Write all rows in RDD partition. try { val ret = Utils.tryWithSafeFinallyAndFailureCallbacks { @@ -256,7 +258,7 @@ class HadoopMapRedWriteConfigUtil[K, V: ClassTag](conf: SerializableJobConf) private def getOutputFormat(): OutputFormat[K, V] = { require(outputFormat != null, "Must call initOutputFormat first.") - outputFormat.newInstance() + outputFormat.getConstructor().newInstance() } // -------------------------------------------------------------------------- @@ -379,7 +381,7 @@ class HadoopMapReduceWriteConfigUtil[K, V: ClassTag](conf: SerializableConfigura private def getOutputFormat(): NewOutputFormat[K, V] = { require(outputFormat != null, "Must call initOutputFormat first.") - outputFormat.newInstance() + outputFormat.getConstructor().newInstance() } // -------------------------------------------------------------------------- diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 0664c5ac752c1..adbd59c9f03b4 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -22,11 +22,13 @@ import java.util.Locale import com.github.luben.zstd.{ZstdInputStream, ZstdOutputStream} import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} -import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream} +import net.jpountz.lz4.{LZ4BlockInputStream, LZ4BlockOutputStream, LZ4Factory} +import net.jpountz.xxhash.XXHashFactory import org.xerial.snappy.{Snappy, SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -43,11 +45,15 @@ trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream def compressedInputStream(s: InputStream): InputStream + + private[spark] def compressedContinuousInputStream(s: InputStream): InputStream = { + compressedInputStream(s) + } } private[spark] object CompressionCodec { - private val configKey = "spark.io.compression.codec" + private val configKey = IO_COMPRESSION_CODEC.key private[spark] def supportsConcatenationOfSerializedStreams(codec: CompressionCodec): Boolean = { (codec.isInstanceOf[SnappyCompressionCodec] || codec.isInstanceOf[LZFCompressionCodec] @@ -61,7 +67,7 @@ private[spark] object CompressionCodec { "zstd" -> classOf[ZStdCompressionCodec].getName) def getCodecName(conf: SparkConf): String = { - conf.get(configKey, DEFAULT_COMPRESSION_CODEC) + conf.get(IO_COMPRESSION_CODEC) } def createCodec(conf: SparkConf): CompressionCodec = { @@ -72,8 +78,9 @@ private[spark] object CompressionCodec { val codecClass = shortCompressionCodecNames.getOrElse(codecName.toLowerCase(Locale.ROOT), codecName) val codec = try { - val ctor = Utils.classForName(codecClass).getConstructor(classOf[SparkConf]) - Some(ctor.newInstance(conf).asInstanceOf[CompressionCodec]) + val ctor = + Utils.classForName[CompressionCodec](codecClass).getConstructor(classOf[SparkConf]) + Some(ctor.newInstance(conf)) } catch { case _: ClassNotFoundException | _: IllegalArgumentException => None } @@ -112,14 +119,35 @@ private[spark] object CompressionCodec { @DeveloperApi class LZ4CompressionCodec(conf: SparkConf) extends CompressionCodec { + // SPARK-28102: if the LZ4 JNI libraries fail to initialize then `fastestInstance()` calls fall + // back to non-JNI implementations but do not remember the fact that JNI failed to load, so + // repeated calls to `fastestInstance()` will cause performance problems because the JNI load + // will be repeatedly re-attempted and that path is slow because it throws exceptions from a + // static synchronized method (causing lock contention). To avoid this problem, we cache the + // result of the `fastestInstance()` calls ourselves (both factories are thread-safe). + @transient private[this] lazy val lz4Factory: LZ4Factory = LZ4Factory.fastestInstance() + @transient private[this] lazy val xxHashFactory: XXHashFactory = XXHashFactory.fastestInstance() + + private[this] val defaultSeed: Int = 0x9747b28c // LZ4BlockOutputStream.DEFAULT_SEED + override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.lz4.blockSize", "32k").toInt - new LZ4BlockOutputStream(s, blockSize) + val blockSize = conf.get(IO_COMPRESSION_LZ4_BLOCKSIZE).toInt + val syncFlush = false + new LZ4BlockOutputStream( + s, + blockSize, + lz4Factory.fastCompressor(), + xxHashFactory.newStreamingHash32(defaultSeed).asChecksum, + syncFlush) } override def compressedInputStream(s: InputStream): InputStream = { val disableConcatenationOfByteStream = false - new LZ4BlockInputStream(s, disableConcatenationOfByteStream) + new LZ4BlockInputStream( + s, + lz4Factory.fastDecompressor(), + xxHashFactory.newStreamingHash32(defaultSeed).asChecksum, + disableConcatenationOfByteStream) } } @@ -162,7 +190,7 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { } override def compressedOutputStream(s: OutputStream): OutputStream = { - val blockSize = conf.getSizeAsBytes("spark.io.compression.snappy.blockSize", "32k").toInt + val blockSize = conf.get(IO_COMPRESSION_SNAPPY_BLOCKSIZE).toInt new SnappyOutputStream(s, blockSize) } @@ -181,10 +209,10 @@ class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { @DeveloperApi class ZStdCompressionCodec(conf: SparkConf) extends CompressionCodec { - private val bufferSize = conf.getSizeAsBytes("spark.io.compression.zstd.bufferSize", "32k").toInt + private val bufferSize = conf.get(IO_COMPRESSION_ZSTD_BUFFERSIZE).toInt // Default compression level for zstd compression to 1 because it is // fastest of all with reasonably high compression ratio. - private val level = conf.getInt("spark.io.compression.zstd.level", 1) + private val level = conf.get(IO_COMPRESSION_ZSTD_LEVEL) override def compressedOutputStream(s: OutputStream): OutputStream = { // Wrap the zstd output stream in a buffered output stream, so that we can @@ -197,4 +225,12 @@ class ZStdCompressionCodec(conf: SparkConf) extends CompressionCodec { // avoid overhead excessive of JNI call while trying to uncompress small amount of data. new BufferedInputStream(new ZstdInputStream(s), bufferSize) } + + override def compressedContinuousInputStream(s: InputStream): InputStream = { + // SPARK-26283: Enable reading from open frames of zstd (for eg: zstd compressed eventLog + // Reading). By default `isContinuous` is false, and when we try to read from open frames, + // `compressedInputStream` method above throws truncated error exception. This method set + // `isContinuous` true to allow reading from open frames. + new BufferedInputStream(new ZstdInputStream(s).setContinuous(true), bufferSize) + } } diff --git a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala index 1b049b786023a..77bbbd9a934ce 100644 --- a/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala +++ b/core/src/main/scala/org/apache/spark/launcher/LauncherBackend.scala @@ -44,7 +44,7 @@ private[spark] abstract class LauncherBackend { .map(_.toInt) val secret = conf.getOption(LauncherProtocol.CONF_LAUNCHER_SECRET) .orElse(sys.env.get(LauncherProtocol.ENV_LAUNCHER_SECRET)) - if (port != None && secret != None) { + if (port.isDefined && secret.isDefined) { val s = new Socket(InetAddress.getLoopbackAddress(), port.get) connection = new BackendConnection(s) connection.send(new Hello(secret.get, SPARK_VERSION)) @@ -94,11 +94,8 @@ private[spark] abstract class LauncherBackend { protected def onDisconnected() : Unit = { } private def fireStopRequest(): Unit = { - val thread = LauncherBackend.threadFactory.newThread(new Runnable() { - override def run(): Unit = Utils.tryLogNonFatalError { - onStopRequest() - } - }) + val thread = LauncherBackend.threadFactory.newThread( + () => Utils.tryLogNonFatalError { onStopRequest() }) thread.start() } diff --git a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala index f1915857ea43a..50055dcd2954a 100644 --- a/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala +++ b/core/src/main/scala/org/apache/spark/memory/ExecutionMemoryPool.scala @@ -151,7 +151,7 @@ private[memory] class ExecutionMemoryPool( */ def releaseMemory(numBytes: Long, taskAttemptId: Long): Unit = lock.synchronized { val curMem = memoryForTask.getOrElse(taskAttemptId, 0L) - var memoryToFree = if (curMem < numBytes) { + val memoryToFree = if (curMem < numBytes) { logWarning( s"Internal error: release called on $numBytes bytes but task only has $curMem bytes " + s"of memory from the $poolName pool") diff --git a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala index 4fde2d0beaa71..c08b47f99dda3 100644 --- a/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/MemoryManager.scala @@ -41,6 +41,8 @@ private[spark] abstract class MemoryManager( onHeapStorageMemory: Long, onHeapExecutionMemory: Long) extends Logging { + require(onHeapExecutionMemory > 0, "onHeapExecutionMemory must be > 0") + // -- Methods related to memory allocation policies and bookkeeping ------------------------------ @GuardedBy("this") @@ -57,7 +59,7 @@ private[spark] abstract class MemoryManager( protected[this] val maxOffHeapMemory = conf.get(MEMORY_OFFHEAP_SIZE) protected[this] val offHeapStorageMemory = - (maxOffHeapMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong + (maxOffHeapMemory * conf.get(MEMORY_STORAGE_FRACTION)).toLong offHeapExecutionMemoryPool.incrementPoolSize(maxOffHeapMemory - offHeapStorageMemory) offHeapStorageMemoryPool.incrementPoolSize(offHeapStorageMemory) @@ -253,7 +255,7 @@ private[spark] abstract class MemoryManager( } val size = ByteArrayMethods.nextPowerOf2(maxTungstenMemory / cores / safetyFactor) val default = math.min(maxPageSize, math.max(minPageSize, size)) - conf.getSizeAsBytes("spark.buffer.pageSize", default) + conf.get(BUFFER_PAGESIZE).getOrElse(default) } /** diff --git a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala deleted file mode 100644 index a6f7db0600e60..0000000000000 --- a/core/src/main/scala/org/apache/spark/memory/StaticMemoryManager.scala +++ /dev/null @@ -1,145 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.memory - -import org.apache.spark.SparkConf -import org.apache.spark.storage.BlockId - -/** - * A [[MemoryManager]] that statically partitions the heap space into disjoint regions. - * - * The sizes of the execution and storage regions are determined through - * `spark.shuffle.memoryFraction` and `spark.storage.memoryFraction` respectively. The two - * regions are cleanly separated such that neither usage can borrow memory from the other. - */ -private[spark] class StaticMemoryManager( - conf: SparkConf, - maxOnHeapExecutionMemory: Long, - override val maxOnHeapStorageMemory: Long, - numCores: Int) - extends MemoryManager( - conf, - numCores, - maxOnHeapStorageMemory, - maxOnHeapExecutionMemory) { - - def this(conf: SparkConf, numCores: Int) { - this( - conf, - StaticMemoryManager.getMaxExecutionMemory(conf), - StaticMemoryManager.getMaxStorageMemory(conf), - numCores) - } - - // The StaticMemoryManager does not support off-heap storage memory: - offHeapExecutionMemoryPool.incrementPoolSize(offHeapStorageMemoryPool.poolSize) - offHeapStorageMemoryPool.decrementPoolSize(offHeapStorageMemoryPool.poolSize) - - // Max number of bytes worth of blocks to evict when unrolling - private val maxUnrollMemory: Long = { - (maxOnHeapStorageMemory * conf.getDouble("spark.storage.unrollFraction", 0.2)).toLong - } - - override def maxOffHeapStorageMemory: Long = 0L - - override def acquireStorageMemory( - blockId: BlockId, - numBytes: Long, - memoryMode: MemoryMode): Boolean = synchronized { - require(memoryMode != MemoryMode.OFF_HEAP, - "StaticMemoryManager does not support off-heap storage memory") - if (numBytes > maxOnHeapStorageMemory) { - // Fail fast if the block simply won't fit - logInfo(s"Will not store $blockId as the required space ($numBytes bytes) exceeds our " + - s"memory limit ($maxOnHeapStorageMemory bytes)") - false - } else { - onHeapStorageMemoryPool.acquireMemory(blockId, numBytes) - } - } - - override def acquireUnrollMemory( - blockId: BlockId, - numBytes: Long, - memoryMode: MemoryMode): Boolean = synchronized { - require(memoryMode != MemoryMode.OFF_HEAP, - "StaticMemoryManager does not support off-heap unroll memory") - val currentUnrollMemory = onHeapStorageMemoryPool.memoryStore.currentUnrollMemory - val freeMemory = onHeapStorageMemoryPool.memoryFree - // When unrolling, we will use all of the existing free memory, and, if necessary, - // some extra space freed from evicting cached blocks. We must place a cap on the - // amount of memory to be evicted by unrolling, however, otherwise unrolling one - // big block can blow away the entire cache. - val maxNumBytesToFree = math.max(0, maxUnrollMemory - currentUnrollMemory - freeMemory) - // Keep it within the range 0 <= X <= maxNumBytesToFree - val numBytesToFree = math.max(0, math.min(maxNumBytesToFree, numBytes - freeMemory)) - onHeapStorageMemoryPool.acquireMemory(blockId, numBytes, numBytesToFree) - } - - private[memory] - override def acquireExecutionMemory( - numBytes: Long, - taskAttemptId: Long, - memoryMode: MemoryMode): Long = synchronized { - memoryMode match { - case MemoryMode.ON_HEAP => onHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) - case MemoryMode.OFF_HEAP => offHeapExecutionMemoryPool.acquireMemory(numBytes, taskAttemptId) - } - } -} - - -private[spark] object StaticMemoryManager { - - private val MIN_MEMORY_BYTES = 32 * 1024 * 1024 - - /** - * Return the total amount of memory available for the storage region, in bytes. - */ - private def getMaxStorageMemory(conf: SparkConf): Long = { - val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) - val memoryFraction = conf.getDouble("spark.storage.memoryFraction", 0.6) - val safetyFraction = conf.getDouble("spark.storage.safetyFraction", 0.9) - (systemMaxMemory * memoryFraction * safetyFraction).toLong - } - - /** - * Return the total amount of memory available for the execution region, in bytes. - */ - private def getMaxExecutionMemory(conf: SparkConf): Long = { - val systemMaxMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) - - if (systemMaxMemory < MIN_MEMORY_BYTES) { - throw new IllegalArgumentException(s"System memory $systemMaxMemory must " + - s"be at least $MIN_MEMORY_BYTES. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") - } - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") - if (executorMemory < MIN_MEMORY_BYTES) { - throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + - s"$MIN_MEMORY_BYTES. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") - } - } - val memoryFraction = conf.getDouble("spark.shuffle.memoryFraction", 0.2) - val safetyFraction = conf.getDouble("spark.shuffle.safetyFraction", 0.8) - (systemMaxMemory * memoryFraction * safetyFraction).toLong - } - -} diff --git a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala index 78edd2c4d7faa..7282a83f0739f 100644 --- a/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala +++ b/core/src/main/scala/org/apache/spark/memory/UnifiedMemoryManager.scala @@ -18,6 +18,8 @@ package org.apache.spark.memory import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ import org.apache.spark.storage.BlockId /** @@ -44,7 +46,7 @@ import org.apache.spark.storage.BlockId * it if necessary. Cached blocks can be evicted only if actual * storage memory usage exceeds this region. */ -private[spark] class UnifiedMemoryManager private[memory] ( +private[spark] class UnifiedMemoryManager( conf: SparkConf, val maxHeapMemory: Long, onHeapStorageRegionSize: Long, @@ -201,7 +203,7 @@ object UnifiedMemoryManager { conf, maxHeapMemory = maxMemory, onHeapStorageRegionSize = - (maxMemory * conf.getDouble("spark.memory.storageFraction", 0.5)).toLong, + (maxMemory * conf.get(config.MEMORY_STORAGE_FRACTION)).toLong, numCores = numCores) } @@ -209,26 +211,26 @@ object UnifiedMemoryManager { * Return the total amount of memory shared between execution and storage, in bytes. */ private def getMaxMemory(conf: SparkConf): Long = { - val systemMemory = conf.getLong("spark.testing.memory", Runtime.getRuntime.maxMemory) - val reservedMemory = conf.getLong("spark.testing.reservedMemory", - if (conf.contains("spark.testing")) 0 else RESERVED_SYSTEM_MEMORY_BYTES) + val systemMemory = conf.get(TEST_MEMORY) + val reservedMemory = conf.getLong(TEST_RESERVED_MEMORY.key, + if (conf.contains(IS_TESTING)) 0 else RESERVED_SYSTEM_MEMORY_BYTES) val minSystemMemory = (reservedMemory * 1.5).ceil.toLong if (systemMemory < minSystemMemory) { throw new IllegalArgumentException(s"System memory $systemMemory must " + s"be at least $minSystemMemory. Please increase heap size using the --driver-memory " + - s"option or spark.driver.memory in Spark configuration.") + s"option or ${config.DRIVER_MEMORY.key} in Spark configuration.") } // SPARK-12759 Check executor memory to fail fast if memory is insufficient - if (conf.contains("spark.executor.memory")) { - val executorMemory = conf.getSizeAsBytes("spark.executor.memory") + if (conf.contains(config.EXECUTOR_MEMORY)) { + val executorMemory = conf.getSizeAsBytes(config.EXECUTOR_MEMORY.key) if (executorMemory < minSystemMemory) { throw new IllegalArgumentException(s"Executor memory $executorMemory must be at least " + s"$minSystemMemory. Please increase executor memory using the " + - s"--executor-memory option or spark.executor.memory in Spark configuration.") + s"--executor-memory option or ${config.EXECUTOR_MEMORY.key} in Spark configuration.") } } val usableMemory = systemMemory - reservedMemory - val memoryFraction = conf.getDouble("spark.memory.fraction", 0.6) + val memoryFraction = conf.get(config.MEMORY_FRACTION) (usableMemory * memoryFraction).toLong } } diff --git a/core/src/main/scala/org/apache/spark/memory/package.scala b/core/src/main/scala/org/apache/spark/memory/package.scala index 3d00cd9cb6377..7f782193f246f 100644 --- a/core/src/main/scala/org/apache/spark/memory/package.scala +++ b/core/src/main/scala/org/apache/spark/memory/package.scala @@ -61,15 +61,10 @@ package org.apache.spark * }}} * * - * There are two implementations of [[org.apache.spark.memory.MemoryManager]] which vary in how - * they handle the sizing of their memory pools: + * There is one implementation of [[org.apache.spark.memory.MemoryManager]]: * - * - [[org.apache.spark.memory.UnifiedMemoryManager]], the default in Spark 1.6+, enforces soft + * - [[org.apache.spark.memory.UnifiedMemoryManager]] enforces soft * boundaries between storage and execution memory, allowing requests for memory in one region * to be fulfilled by borrowing memory from the other. - * - [[org.apache.spark.memory.StaticMemoryManager]] enforces hard boundaries between storage - * and execution memory by statically partitioning Spark's memory and preventing storage and - * execution from borrowing memory from each other. This mode is retained only for legacy - * compatibility purposes. */ package object memory diff --git a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala index cd10dad25e87b..b830f9f402ebe 100644 --- a/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala +++ b/core/src/main/scala/org/apache/spark/metrics/ExecutorMetricType.scala @@ -19,25 +19,46 @@ package org.apache.spark.metrics import java.lang.management.{BufferPoolMXBean, ManagementFactory} import javax.management.ObjectName +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.SparkEnv +import org.apache.spark.executor.ProcfsMetricsGetter +import org.apache.spark.internal.{config, Logging} import org.apache.spark.memory.MemoryManager /** * Executor metric types for executor-level metrics stored in ExecutorMetrics. */ sealed trait ExecutorMetricType { + private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] + private[spark] def names: Seq[String] +} + +sealed trait SingleValueExecutorMetricType extends ExecutorMetricType { + override private[spark] def names = { + Seq(getClass().getName(). + stripSuffix("$").split("""\.""").last) + } + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val metrics = new Array[Long](1) + metrics(0) = getMetricValue(memoryManager) + metrics + } + private[spark] def getMetricValue(memoryManager: MemoryManager): Long - private[spark] val name = getClass().getName().stripSuffix("$").split("""\.""").last } private[spark] abstract class MemoryManagerExecutorMetricType( - f: MemoryManager => Long) extends ExecutorMetricType { + f: MemoryManager => Long) extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { f(memoryManager) } } private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) - extends ExecutorMetricType { + extends SingleValueExecutorMetricType { private val bean = ManagementFactory.newPlatformMXBeanProxy( ManagementFactory.getPlatformMBeanServer, new ObjectName(mBeanName).toString, classOf[BufferPoolMXBean]) @@ -47,18 +68,97 @@ private[spark] abstract class MBeanExecutorMetricType(mBeanName: String) } } -case object JVMHeapMemory extends ExecutorMetricType { +case object JVMHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getHeapMemoryUsage().getUsed() } } -case object JVMOffHeapMemory extends ExecutorMetricType { +case object JVMOffHeapMemory extends SingleValueExecutorMetricType { override private[spark] def getMetricValue(memoryManager: MemoryManager): Long = { ManagementFactory.getMemoryMXBean.getNonHeapMemoryUsage().getUsed() } } +case object ProcessTreeMetrics extends ExecutorMetricType { + override val names = Seq( + "ProcessTreeJVMVMemory", + "ProcessTreeJVMRSSMemory", + "ProcessTreePythonVMemory", + "ProcessTreePythonRSSMemory", + "ProcessTreeOtherVMemory", + "ProcessTreeOtherRSSMemory") + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val allMetrics = ProcfsMetricsGetter.pTreeInfo.computeAllMetrics() + val processTreeMetrics = new Array[Long](names.length) + processTreeMetrics(0) = allMetrics.jvmVmemTotal + processTreeMetrics(1) = allMetrics.jvmRSSTotal + processTreeMetrics(2) = allMetrics.pythonVmemTotal + processTreeMetrics(3) = allMetrics.pythonRSSTotal + processTreeMetrics(4) = allMetrics.otherVmemTotal + processTreeMetrics(5) = allMetrics.otherRSSTotal + processTreeMetrics + } +} + +case object GarbageCollectionMetrics extends ExecutorMetricType with Logging { + private var nonBuiltInCollectors: Seq[String] = Nil + + override val names = Seq( + "MinorGCCount", + "MinorGCTime", + "MajorGCCount", + "MajorGCTime" + ) + + /* We builtin some common GC collectors which categorized as young generation and old */ + private[spark] val YOUNG_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq( + "Copy", + "PS Scavenge", + "ParNew", + "G1 Young Generation" + ) + + private[spark] val OLD_GENERATION_BUILTIN_GARBAGE_COLLECTORS = Seq( + "MarkSweepCompact", + "PS MarkSweep", + "ConcurrentMarkSweep", + "G1 Old Generation" + ) + + private lazy val youngGenerationGarbageCollector: Seq[String] = { + SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS) + } + + private lazy val oldGenerationGarbageCollector: Seq[String] = { + SparkEnv.get.conf.get(config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS) + } + + override private[spark] def getMetricValues(memoryManager: MemoryManager): Array[Long] = { + val gcMetrics = new Array[Long](names.length) // minorCount, minorTime, majorCount, majorTime + ManagementFactory.getGarbageCollectorMXBeans.asScala.foreach { mxBean => + if (youngGenerationGarbageCollector.contains(mxBean.getName)) { + gcMetrics(0) = mxBean.getCollectionCount + gcMetrics(1) = mxBean.getCollectionTime + } else if (oldGenerationGarbageCollector.contains(mxBean.getName)) { + gcMetrics(2) = mxBean.getCollectionCount + gcMetrics(3) = mxBean.getCollectionTime + } else if (!nonBuiltInCollectors.contains(mxBean.getName)) { + nonBuiltInCollectors = mxBean.getName +: nonBuiltInCollectors + // log it when first seen + logWarning(s"To enable non-built-in garbage collector(s) " + + s"$nonBuiltInCollectors, users should configure it(them) to " + + s"${config.EVENT_LOG_GC_METRICS_YOUNG_GENERATION_GARBAGE_COLLECTORS.key} or " + + s"${config.EVENT_LOG_GC_METRICS_OLD_GENERATION_GARBAGE_COLLECTORS.key}") + } else { + // do nothing + } + } + gcMetrics + } +} + case object OnHeapExecutionMemory extends MemoryManagerExecutorMetricType( _.onHeapExecutionMemoryUsed) @@ -84,8 +184,9 @@ case object MappedPoolMemory extends MBeanExecutorMetricType( "java.nio:type=BufferPool,name=mapped") private[spark] object ExecutorMetricType { - // List of all executor metric types - val values = IndexedSeq( + + // List of all executor metric getters + val metricGetters = IndexedSeq( JVMHeapMemory, JVMOffHeapMemory, OnHeapExecutionMemory, @@ -95,10 +196,21 @@ private[spark] object ExecutorMetricType { OnHeapUnifiedMemory, OffHeapUnifiedMemory, DirectPoolMemory, - MappedPoolMemory + MappedPoolMemory, + ProcessTreeMetrics, + GarbageCollectionMetrics ) - // Map of executor metric type to its index in values. - val metricIdxMap = - Map[ExecutorMetricType, Int](ExecutorMetricType.values.zipWithIndex: _*) + + val (metricToOffset, numMetrics) = { + var numberOfMetrics = 0 + val definedMetricsAndOffset = mutable.LinkedHashMap.empty[String, Int] + metricGetters.foreach { m => + (0 until m.names.length).foreach { idx => + definedMetricsAndOffset += (m.names(idx) -> (idx + numberOfMetrics)) + } + numberOfMetrics += m.names.length + } + (definedMetricsAndOffset, numberOfMetrics) + } } diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index a4056508c181e..b6be8aaefd351 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -26,6 +26,7 @@ import scala.util.matching.Regex import org.apache.spark.SparkConf import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.util.Utils private[spark] class MetricsConfig(conf: SparkConf) extends Logging { @@ -52,7 +53,7 @@ private[spark] class MetricsConfig(conf: SparkConf) extends Logging { // Add default properties in case there's no properties file setDefaultProperties(properties) - loadPropertiesFromFile(conf.getOption("spark.metrics.conf")) + loadPropertiesFromFile(conf.get(METRICS_CONF)) // Also look for the properties in provided Spark configuration val prefix = "spark.metrics.conf." diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 3457a2632277d..c96640a6fab3f 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -22,7 +22,7 @@ import java.util.concurrent.TimeUnit import scala.collection.mutable -import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} +import com.codahale.metrics.{Metric, MetricRegistry} import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{SecurityManager, SparkConf} @@ -94,11 +94,13 @@ private[spark] class MetricsSystem private ( metricsConfig.initialize() - def start() { + def start(registerStaticSources: Boolean = true) { require(!running, "Attempting to start a MetricsSystem that is already running") running = true - StaticSources.allSources.foreach(registerSource) - registerSources() + if (registerStaticSources) { + StaticSources.allSources.foreach(registerSource) + registerSources() + } registerSinks() sinks.foreach(_.start) } @@ -128,7 +130,7 @@ private[spark] class MetricsSystem private ( private[spark] def buildRegistryName(source: Source): String = { val metricsNamespace = conf.get(METRICS_NAMESPACE).orElse(conf.getOption("spark.app.id")) - val executorId = conf.getOption("spark.executor.id") + val executorId = conf.get(EXECUTOR_ID) val defaultName = MetricRegistry.name(source.sourceName) if (instance == "driver" || instance == "executor") { @@ -166,9 +168,7 @@ private[spark] class MetricsSystem private ( def removeSource(source: Source) { sources -= source val regName = buildRegistryName(source) - registry.removeMatching(new MetricFilter { - def matches(name: String, metric: Metric): Boolean = name.startsWith(regName) - }) + registry.removeMatching((name: String, _: Metric) => name.startsWith(regName)) } private def registerSources() { @@ -179,8 +179,8 @@ private[spark] class MetricsSystem private ( sourceConfigs.foreach { kv => val classPath = kv._2.getProperty("class") try { - val source = Utils.classForName(classPath).newInstance() - registerSource(source.asInstanceOf[Source]) + val source = Utils.classForName[Source](classPath).getConstructor().newInstance() + registerSource(source) } catch { case e: Exception => logError("Source class " + classPath + " cannot be instantiated", e) } @@ -195,13 +195,18 @@ private[spark] class MetricsSystem private ( val classPath = kv._2.getProperty("class") if (null != classPath) { try { - val sink = Utils.classForName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) - .newInstance(kv._2, registry, securityMgr) if (kv._1 == "servlet") { - metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + val servlet = Utils.classForName[MetricsServlet](classPath) + .getConstructor( + classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + metricsServlet = Some(servlet) } else { - sinks += sink.asInstanceOf[Sink] + val sink = Utils.classForName[Sink](classPath) + .getConstructor( + classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + sinks += sink } } catch { case e: Exception => @@ -233,3 +238,29 @@ private[spark] object MetricsSystem { new MetricsSystem(instance, conf, securityMgr) } } + +private[spark] object MetricsSystemInstances { + // The Spark standalone master process + val MASTER = "master" + + // A component within the master which reports on various applications + val APPLICATIONS = "applications" + + // A Spark standalone worker process + val WORKER = "worker" + + // A Spark executor + val EXECUTOR = "executor" + + // The Spark driver process (the process in which your SparkContext is created) + val DRIVER = "driver" + + // The Spark shuffle service + val SHUFFLE_SERVICE = "shuffleService" + + // The Spark ApplicationMaster when running on YARN + val APPLICATION_MASTER = "applicationMaster" + + // The Spark cluster scheduler when running on Mesos + val MESOS_CLUSTER = "mesos_cluster" +} diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index ac33e68abb490..21b4dfb26dfa8 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -17,7 +17,6 @@ package org.apache.spark.metrics.sink -import java.net.InetSocketAddress import java.util.{Locale, Properties} import java.util.concurrent.TimeUnit diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 68b58b8490641..bea24ca7807e4 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -51,7 +51,7 @@ private[spark] class MetricsServlet( def getHandlers(conf: SparkConf): Array[ServletContextHandler] = { Array[ServletContextHandler]( createServletHandler(servletPath, - new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr, conf) + new ServletParams(request => getMetricsSnapshot(request), "text/json"), conf) ) } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala new file mode 100644 index 0000000000000..45a4d224d45fe --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/AccumulatorSource.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.metrics.source + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Experimental +import org.apache.spark.util.{AccumulatorV2, DoubleAccumulator, LongAccumulator} + +/** + * AccumulatorSource is a Spark metric Source that reports the current value + * of the accumulator as a gauge. + * + * It is restricted to the LongAccumulator and the DoubleAccumulator, as those + * are the current built-in numerical accumulators with Spark, and excludes + * the CollectionAccumulator, as that is a List of values (hard to report, + * to a metrics system) + */ +private[spark] class AccumulatorSource extends Source { + private val registry = new MetricRegistry + protected def register[T](accumulators: Map[String, AccumulatorV2[_, T]]): Unit = { + accumulators.foreach { + case (name, accumulator) => + val gauge = new Gauge[T] { + override def getValue: T = accumulator.value + } + registry.register(MetricRegistry.name(name), gauge) + } + } + + override def sourceName: String = "AccumulatorSource" + override def metricRegistry: MetricRegistry = registry +} + +@Experimental +class LongAccumulatorSource extends AccumulatorSource + +@Experimental +class DoubleAccumulatorSource extends AccumulatorSource + +/** + * :: Experimental :: + * Metrics source specifically for LongAccumulators. Accumulators + * are only valid on the driver side, so these metrics are reported + * only by the driver. + * Register LongAccumulators using: + * LongAccumulatorSource.register(sc, {"name" -> longAccumulator}) + */ +@Experimental +object LongAccumulatorSource { + def register(sc: SparkContext, accumulators: Map[String, LongAccumulator]): Unit = { + val source = new LongAccumulatorSource + source.register(accumulators) + sc.env.metricsSystem.registerSource(source) + } +} + +/** + * :: Experimental :: + * Metrics source specifically for DoubleAccumulators. Accumulators + * are only valid on the driver side, so these metrics are reported + * only by the driver. + * Register DoubleAccumulators using: + * DoubleAccumulatorSource.register(sc, {"name" -> doubleAccumulator}) + */ +@Experimental +object DoubleAccumulatorSource { + def register(sc: SparkContext, accumulators: Map[String, DoubleAccumulator]): Unit = { + val source = new DoubleAccumulatorSource + source.register(accumulators) + sc.env.metricsSystem.registerSource(source) + } +} diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JVMCPUSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JVMCPUSource.scala new file mode 100644 index 0000000000000..11a3acfea016f --- /dev/null +++ b/core/src/main/scala/org/apache/spark/metrics/source/JVMCPUSource.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.metrics.source + +import java.lang.management.ManagementFactory + +import com.codahale.metrics.{Gauge, MetricRegistry} +import javax.management.{MBeanServer, ObjectName} +import scala.util.control.NonFatal + +private[spark] class JVMCPUSource extends Source { + + override val metricRegistry = new MetricRegistry() + override val sourceName = "JVMCPU" + + // Dropwizard/Codahale metrics gauge measuring the JVM process CPU time. + // This Gauge will try to get and return the JVM Process CPU time or return -1 otherwise. + // The CPU time value is returned in nanoseconds. + // It will use proprietary extensions such as com.sun.management.OperatingSystemMXBean or + // com.ibm.lang.management.OperatingSystemMXBean, if available. + metricRegistry.register(MetricRegistry.name("jvmCpuTime"), new Gauge[Long] { + val mBean: MBeanServer = ManagementFactory.getPlatformMBeanServer + val name = new ObjectName("java.lang", "type", "OperatingSystem") + override def getValue: Long = { + try { + // return JVM process CPU time if the ProcessCpuTime method is available + mBean.getAttribute(name, "ProcessCpuTime").asInstanceOf[Long] + } catch { + case NonFatal(_) => -1L + } + } + }) +} diff --git a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala index e94a01244474c..4993519aa3843 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockDataManager.scala @@ -19,6 +19,7 @@ package org.apache.spark.network import scala.reflect.ClassTag +import org.apache.spark.TaskContext import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.storage.{BlockId, StorageLevel} @@ -58,5 +59,5 @@ trait BlockDataManager { /** * Release locks acquired by [[putBlockData()]] and [[getBlockData()]]. */ - def releaseLock(blockId: BlockId, taskAttemptId: Option[Long]): Unit + def releaseLock(blockId: BlockId, taskContext: Option[TaskContext]): Unit } diff --git a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala index eef8c31e05ab1..51ced697c6afb 100644 --- a/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/BlockTransferService.scala @@ -27,7 +27,7 @@ import scala.reflect.ClassTag import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ShuffleClient} -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, EncryptedManagedBuffer, StorageLevel} import org.apache.spark.util.ThreadUtils private[spark] @@ -104,11 +104,17 @@ abstract class BlockTransferService extends ShuffleClient with Closeable with Lo data match { case f: FileSegmentManagedBuffer => result.success(f) + case e: EncryptedManagedBuffer => + result.success(e) case _ => - val ret = ByteBuffer.allocate(data.size.toInt) - ret.put(data.nioByteBuffer()) - ret.flip() - result.success(new NioManagedBuffer(ret)) + try { + val ret = ByteBuffer.allocate(data.size.toInt) + ret.put(data.nioByteBuffer()) + ret.flip() + result.success(new NioManagedBuffer(ret)) + } catch { + case e: Throwable => result.failure(e) + } } } }, tempFileManager) diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala index 7076701421e2e..b2ab31488e4c1 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockRpcServer.scala @@ -20,7 +20,6 @@ package org.apache.spark.network.netty import java.nio.ByteBuffer import scala.collection.JavaConverters._ -import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark.internal.Logging @@ -30,7 +29,7 @@ import org.apache.spark.network.client.{RpcResponseCallback, StreamCallbackWithI import org.apache.spark.network.server.{OneForOneStreamManager, RpcHandler, StreamManager} import org.apache.spark.network.shuffle.protocol._ import org.apache.spark.serializer.Serializer -import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, ShuffleBlockId, StorageLevel} /** * Serves requests to open blocks by simply registering one chunk per block requested. @@ -59,18 +58,28 @@ class NettyBlockRpcServer( val blocksNum = openBlocks.blockIds.length val blocks = for (i <- (0 until blocksNum).view) yield blockManager.getBlockData(BlockId.apply(openBlocks.blockIds(i))) - val streamId = streamManager.registerStream(appId, blocks.iterator.asJava) + val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, + client.getChannel) logTrace(s"Registered streamId $streamId with $blocksNum buffers") responseContext.onSuccess(new StreamHandle(streamId, blocksNum).toByteBuffer) + case fetchShuffleBlocks: FetchShuffleBlocks => + val blocks = fetchShuffleBlocks.mapIds.zipWithIndex.flatMap { case (mapId, index) => + fetchShuffleBlocks.reduceIds.apply(index).map { reduceId => + blockManager.getBlockData( + ShuffleBlockId(fetchShuffleBlocks.shuffleId, mapId, reduceId)) + } + } + val numBlockIds = fetchShuffleBlocks.reduceIds.map(_.length).sum + val streamId = streamManager.registerStream(appId, blocks.iterator.asJava, + client.getChannel) + logTrace(s"Registered streamId $streamId with $numBlockIds buffers") + responseContext.onSuccess( + new StreamHandle(streamId, numBlockIds).toByteBuffer) + case uploadBlock: UploadBlock => // StorageLevel and ClassTag are serialized as bytes using our JavaSerializer. - val (level: StorageLevel, classTag: ClassTag[_]) = { - serializer - .newInstance() - .deserialize(ByteBuffer.wrap(uploadBlock.metadata)) - .asInstanceOf[(StorageLevel, ClassTag[_])] - } + val (level, classTag) = deserializeMetadata(uploadBlock.metadata) val data = new NioManagedBuffer(ByteBuffer.wrap(uploadBlock.blockData)) val blockId = BlockId(uploadBlock.blockId) logDebug(s"Receiving replicated block $blockId with level ${level} " + @@ -86,12 +95,7 @@ class NettyBlockRpcServer( responseContext: RpcResponseCallback): StreamCallbackWithID = { val message = BlockTransferMessage.Decoder.fromByteBuffer(messageHeader).asInstanceOf[UploadBlockStream] - val (level: StorageLevel, classTag: ClassTag[_]) = { - serializer - .newInstance() - .deserialize(ByteBuffer.wrap(message.metadata)) - .asInstanceOf[(StorageLevel, ClassTag[_])] - } + val (level, classTag) = deserializeMetadata(message.metadata) val blockId = BlockId(message.blockId) logDebug(s"Receiving replicated block $blockId with level ${level} as stream " + s"from ${client.getSocketAddress}") @@ -100,5 +104,12 @@ class NettyBlockRpcServer( blockManager.putBlockDataAsStream(blockId, level, classTag) } + private def deserializeMetadata[T](metadata: Array[Byte]): (StorageLevel, ClassTag[T]) = { + serializer + .newInstance() + .deserialize(ByteBuffer.wrap(metadata)) + .asInstanceOf[(StorageLevel, ClassTag[T])] + } + override def getStreamManager(): StreamManager = streamManager } diff --git a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala index dc55685b1e7bd..b12cd4254f19e 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/NettyBlockTransferService.scala @@ -17,16 +17,19 @@ package org.apache.spark.network.netty +import java.io.IOException import java.nio.ByteBuffer import java.util.{HashMap => JHashMap, Map => JMap} import scala.collection.JavaConverters._ import scala.concurrent.{Future, Promise} import scala.reflect.ClassTag +import scala.util.{Success, Try} import com.codahale.metrics.{Metric, MetricSet} import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.ExecutorDeadException import org.apache.spark.internal.config import org.apache.spark.network._ import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} @@ -36,8 +39,10 @@ import org.apache.spark.network.server._ import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, OneForOneBlockFetcher, RetryingBlockFetcher} import org.apache.spark.network.shuffle.protocol.{UploadBlock, UploadBlockStream} import org.apache.spark.network.util.JavaUtils +import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.serializer.JavaSerializer import org.apache.spark.storage.{BlockId, StorageLevel} +import org.apache.spark.storage.BlockManagerMessages.IsExecutorAlive import org.apache.spark.util.Utils /** @@ -49,7 +54,8 @@ private[spark] class NettyBlockTransferService( bindAddress: String, override val hostName: String, _port: Int, - numCores: Int) + numCores: Int, + driverEndPointRef: RpcEndpointRef = null) extends BlockTransferService { // TODO: Don't use Java serialization, use a more cross-version compatible serialization format. @@ -112,8 +118,20 @@ private[spark] class NettyBlockTransferService( val blockFetchStarter = new RetryingBlockFetcher.BlockFetchStarter { override def createAndStart(blockIds: Array[String], listener: BlockFetchingListener) { val client = clientFactory.createClient(host, port) - new OneForOneBlockFetcher(client, appId, execId, blockIds, listener, - transportConf, tempFileManager).start() + try { + new OneForOneBlockFetcher(client, appId, execId, blockIds, listener, + transportConf, tempFileManager).start() + } catch { + case e: IOException => + Try { + driverEndPointRef.askSync[Boolean](IsExecutorAlive(execId)) + } match { + case Success(v) if v == false => + throw new ExecutorDeadException(s"The relative remote executor(Id: $execId)," + + " which maintains the block data to fetch is dead.") + case _ => throw e + } + } } } @@ -182,5 +200,8 @@ private[spark] class NettyBlockTransferService( if (clientFactory != null) { clientFactory.close() } + if (transportContext != null) { + transportContext.close() + } } } diff --git a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala index 25f7bcb9801b9..c9103045260f2 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/SparkTransportConf.scala @@ -20,7 +20,7 @@ package org.apache.spark.network.netty import scala.collection.JavaConverters._ import org.apache.spark.SparkConf -import org.apache.spark.network.util.{ConfigProvider, TransportConf} +import org.apache.spark.network.util.{ConfigProvider, NettyUtils, TransportConf} /** * Provides a utility for transforming from a SparkConf inside a Spark JVM (e.g., Executor, @@ -28,17 +28,6 @@ import org.apache.spark.network.util.{ConfigProvider, TransportConf} * like the number of cores that are allocated to this JVM. */ object SparkTransportConf { - /** - * Specifies an upper bound on the number of Netty threads that Spark requires by default. - * In practice, only 2-4 cores should be required to transfer roughly 10 Gb/s, and each core - * that we use will have an initial overhead of roughly 32 MB of off-heap memory, which comes - * at a premium. - * - * Thus, this value should still retain maximum throughput and reduce wasted off-heap memory - * allocation. It can be overridden by setting the number of serverThreads and clientThreads - * manually in Spark's configuration. - */ - private val MAX_DEFAULT_NETTY_THREADS = 8 /** * Utility for creating a [[TransportConf]] from a [[SparkConf]]. @@ -47,16 +36,26 @@ object SparkTransportConf { * @param numUsableCores if nonzero, this will restrict the server and client threads to only * use the given number of cores, rather than all of the machine's cores. * This restriction will only occur if these properties are not already set. + * @param role optional role, could be driver, executor, worker and master. Default is + * [[None]], means no role specific configurations. */ - def fromSparkConf(_conf: SparkConf, module: String, numUsableCores: Int = 0): TransportConf = { + def fromSparkConf( + _conf: SparkConf, + module: String, + numUsableCores: Int = 0, + role: Option[String] = None): TransportConf = { val conf = _conf.clone - - // Specify thread configuration based on our JVM's allocation of cores (rather than necessarily - // assuming we have all the machine's cores). - // NB: Only set if serverThreads/clientThreads not already set. - val numThreads = defaultNumThreads(numUsableCores) - conf.setIfMissing(s"spark.$module.io.serverThreads", numThreads.toString) - conf.setIfMissing(s"spark.$module.io.clientThreads", numThreads.toString) + // specify default thread configuration based on our JVM's allocation of cores (rather than + // necessarily assuming we have all the machine's cores). + val numThreads = NettyUtils.defaultNumThreads(numUsableCores) + // override threads configurations with role specific values if specified + // config order is role > module > default + Seq("serverThreads", "clientThreads").foreach { suffix => + val value = role.flatMap { r => conf.getOption(s"spark.$r.$module.io.$suffix") } + .getOrElse( + conf.get(s"spark.$module.io.$suffix", numThreads.toString)) + conf.set(s"spark.$module.io.$suffix", value) + } new TransportConf(module, new ConfigProvider { override def get(name: String): String = conf.get(name) @@ -66,14 +65,4 @@ object SparkTransportConf { } }) } - - /** - * Returns the default number of threads for both the Netty client and server thread pools. - * If numUsableCores is 0, we will use Runtime get an approximate number of available cores. - */ - private def defaultNumThreads(numUsableCores: Int): Int = { - val availableCores = - if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() - math.min(availableCores, MAX_DEFAULT_NETTY_THREADS) - } } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 8058a4d5dbdea..5d0639e92c36a 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -19,6 +19,8 @@ package org.apache import java.util.Properties +import org.apache.spark.util.VersionUtils + /** * Core Spark functionality. [[org.apache.spark.SparkContext]] serves as the main entry point to * Spark, while [[org.apache.spark.rdd.RDD]] is the data type representing a distributed collection, @@ -89,6 +91,7 @@ package object spark { } val SPARK_VERSION = SparkBuildInfo.spark_version + val SPARK_VERSION_SHORT = VersionUtils.shortVersion(SparkBuildInfo.spark_version) val SPARK_BRANCH = SparkBuildInfo.spark_branch val SPARK_REVISION = SparkBuildInfo.spark_revision val SPARK_BUILD_USER = SparkBuildInfo.spark_build_user diff --git a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala index a14bad47dfe10..039dbcbd5e035 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BinaryFileRDD.scala @@ -41,7 +41,7 @@ private[spark] class BinaryFileRDD[T]( // traversing a large number of directories and files. Parallelize it. conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, Runtime.getRuntime.availableProcessors().toString) - val inputFormat = inputFormatClass.newInstance + val inputFormat = inputFormatClass.getConstructor().newInstance() inputFormat match { case configurable: Configurable => configurable.setConf(conf) diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 4574c3724962e..909f58512153b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -20,7 +20,6 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ @@ -143,8 +142,10 @@ class CoGroupedRDD[K: ClassTag]( case shuffleDependency: ShuffleDependency[_, _, _] => // Read map outputs of shuffle + val metrics = context.taskMetrics().createTempShuffleReadMetrics() val it = SparkEnv.get.shuffleManager - .getReader(shuffleDependency.shuffleHandle, split.index, split.index + 1, context) + .getReader( + shuffleDependency.shuffleHandle, split.index, split.index + 1, context, metrics) .read() rddIterators += ((it, depNum)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 94e7d0b38cba3..55c141c2b8a0a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -21,7 +21,6 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ @@ -58,7 +57,7 @@ private[spark] case class CoalescedRDDPartition( val parentPreferredLocations = rdd.context.getPreferredLocs(rdd, p.index).map(_.host) preferredLocation.exists(parentPreferredLocations.contains) } - if (parents.size == 0) 0.0 else (loc.toDouble / parents.size.toDouble) + if (parents.isEmpty) 0.0 else loc.toDouble / parents.size.toDouble } } @@ -91,7 +90,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( pc.coalesce(maxPartitions, prev).zipWithIndex.map { case (pg, i) => val ids = pg.partitions.map(_.index).toArray - new CoalescedRDDPartition(i, prev, ids, pg.prefLoc) + CoalescedRDDPartition(i, prev, ids, pg.prefLoc) } } @@ -116,7 +115,7 @@ private[spark] class CoalescedRDD[T: ClassTag]( /** * Returns the preferred machine for the partition. If split is of type CoalescedRDDPartition, * then the preferred machine will be one which most parent splits prefer too. - * @param partition + * @param partition the partition for which to retrieve the preferred machine, if exists * @return the machine most preferred by split */ override def getPreferredLocations(partition: Partition): Seq[String] = { @@ -156,9 +155,12 @@ private[spark] class CoalescedRDD[T: ClassTag]( private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) extends PartitionCoalescer { - def compare(o1: PartitionGroup, o2: PartitionGroup): Boolean = o1.numPartitions < o2.numPartitions - def compare(o1: Option[PartitionGroup], o2: Option[PartitionGroup]): Boolean = - if (o1 == None) false else if (o2 == None) true else compare(o1.get, o2.get) + + implicit object partitionGroupOrdering extends Ordering[PartitionGroup] { + override def compare(o1: PartitionGroup, o2: PartitionGroup): Int = + java.lang.Integer.compare(o1.numPartitions, o2.numPartitions) + } + val rnd = new scala.util.Random(7919) // keep this class deterministic @@ -178,7 +180,7 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) prev.context.getPreferredLocs(prev, part.index).map(tl => tl.host) } - class PartitionLocations(prev: RDD[_]) { + private class PartitionLocations(prev: RDD[_]) { // contains all the partitions from the previous RDD that don't have preferred locations val partsWithoutLocs = ArrayBuffer[Partition]() @@ -213,15 +215,14 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) } /** - * Sorts and gets the least element of the list associated with key in groupHash + * Gets the least element of the list associated with key in groupHash * The returned PartitionGroup is the least loaded of all groups that represent the machine "key" * * @param key string representing a partitioned group on preferred machine key * @return Option of [[PartitionGroup]] that has least elements for key */ - def getLeastGroupHash(key: String): Option[PartitionGroup] = { - groupHash.get(key).map(_.sortWith(compare).head) - } + def getLeastGroupHash(key: String): Option[PartitionGroup] = + groupHash.get(key).filter(_.nonEmpty).map(_.min) def addPartToPGroup(part: Partition, pgroup: PartitionGroup): Boolean = { if (!initialHash.contains(part)) { @@ -236,12 +237,12 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) * is assigned a preferredLocation. This uses coupon collector to estimate how many * preferredLocations it must rotate through until it has seen most of the preferred * locations (2 * n log(n)) - * @param targetLen + * @param targetLen The number of desired partition groups */ def setupGroups(targetLen: Int, partitionLocs: PartitionLocations) { // deal with empty case, just create targetLen partition groups with no preferred location if (partitionLocs.partsWithLocs.isEmpty) { - (1 to targetLen).foreach(x => groupArr += new PartitionGroup()) + (1 to targetLen).foreach(_ => groupArr += new PartitionGroup()) return } @@ -297,9 +298,8 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) partitionLocs: PartitionLocations): PartitionGroup = { val slack = (balanceSlack * prev.partitions.length).toInt // least loaded pref locs - val pref = currPrefLocs(p, prev).map(getLeastGroupHash(_)).sortWith(compare) - val prefPart = if (pref == Nil) None else pref.head - + val pref = currPrefLocs(p, prev).flatMap(getLeastGroupHash) + val prefPart = if (pref.isEmpty) None else Some(pref.min) val r1 = rnd.nextInt(groupArr.size) val r2 = rnd.nextInt(groupArr.size) val minPowerOfTwo = { @@ -351,7 +351,7 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) val partIter = partitionLocs.partsWithLocs.iterator groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => while (partIter.hasNext && pg.numPartitions == 0) { - var (nxt_replica, nxt_part) = partIter.next() + var (_, nxt_part) = partIter.next() if (!initialHash.contains(nxt_part)) { pg.partitions += nxt_part initialHash += nxt_part @@ -364,7 +364,7 @@ private class DefaultPartitionCoalescer(val balanceSlack: Double = 0.10) val partNoLocIter = partitionLocs.partsWithoutLocs.iterator groupArr.filter(pg => pg.numPartitions == 0).foreach { pg => while (partNoLocIter.hasNext && pg.numPartitions == 0) { - var nxt_part = partNoLocIter.next() + val nxt_part = partNoLocIter.next() if (!initialHash.contains(nxt_part)) { pg.partitions += nxt_part initialHash += nxt_part diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2d66d25ba39fa..483de28d92ab7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -120,7 +120,7 @@ class NewHadoopRDD[K, V]( } override def getPartitions: Array[Partition] = { - val inputFormat = inputFormatClass.newInstance + val inputFormat = inputFormatClass.getConstructor().newInstance() inputFormat match { case configurable: Configurable => configurable.setConf(_conf) @@ -183,7 +183,7 @@ class NewHadoopRDD[K, V]( } } - private val format = inputFormatClass.newInstance + private val format = inputFormatClass.getConstructor().newInstance() format match { case configurable: Configurable => configurable.setConf(conf) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index e68c6b1366c7f..7f8064f01ec45 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -36,6 +36,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SPECULATION_ENABLED import org.apache.spark.internal.io._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.serializer.Serializer @@ -394,7 +395,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is * greater than `p`) would trigger sparse representation of registers, which may reduce the @@ -436,7 +437,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -456,7 +457,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -473,7 +474,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -1011,7 +1012,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) outputFormatClass: Class[_ <: OutputFormat[_, _]], codec: Class[_ <: CompressionCodec]): Unit = self.withScope { saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, - new JobConf(self.context.hadoopConfiguration), Some(codec)) + new JobConf(self.context.hadoopConfiguration), Option(codec)) } /** @@ -1051,7 +1052,7 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) // When speculation is on and output committer class name contains "Direct", we should warn // users that they may loss data if they are using a direct output committer. - val speculationEnabled = self.conf.getBoolean("spark.speculation", false) + val speculationEnabled = self.conf.get(SPECULATION_ENABLED) val outputCommitterClass = hadoopConf.get("mapred.output.committer.class", "") if (speculationEnabled && outputCommitterClass.contains("Direct")) { val warningMessage = diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 02b28b72fb0e7..3b11e82dab196 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -32,7 +32,7 @@ import scala.collection.mutable.ArrayBuffer import scala.io.Source import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkEnv, TaskContext} +import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.Utils @@ -109,11 +109,10 @@ private[spark] class PipedRDD[T: ClassTag]( } val proc = pb.start() - val env = SparkEnv.get val childThreadException = new AtomicReference[Throwable](null) // Start a thread to print the process's stderr to ours - new Thread(s"stderr reader for $command") { + val stderrReaderThread = new Thread(s"${PipedRDD.STDERR_READER_THREAD_PREFIX} $command") { override def run(): Unit = { val err = proc.getErrorStream try { @@ -128,10 +127,11 @@ private[spark] class PipedRDD[T: ClassTag]( err.close() } } - }.start() + } + stderrReaderThread.start() // Start a thread to feed the process input from our parent's iterator - new Thread(s"stdin writer for $command") { + val stdinWriterThread = new Thread(s"${PipedRDD.STDIN_WRITER_THREAD_PREFIX} $command") { override def run(): Unit = { TaskContext.setTaskContext(context) val out = new PrintWriter(new BufferedWriter( @@ -156,7 +156,28 @@ private[spark] class PipedRDD[T: ClassTag]( out.close() } } - }.start() + } + stdinWriterThread.start() + + // interrupts stdin writer and stderr reader threads when the corresponding task is finished. + // Otherwise, these threads could outlive the task's lifetime. For example: + // val pipeRDD = sc.range(1, 100).pipe(Seq("cat")) + // val abnormalRDD = pipeRDD.mapPartitions(_ => Iterator.empty) + // the iterator generated by PipedRDD is never involved. If the parent RDD's iterator takes a + // long time to generate(ShuffledRDD's shuffle operation for example), the stdin writer thread + // may consume significant memory and CPU time even if task is already finished. + context.addTaskCompletionListener[Unit] { _ => + if (proc.isAlive) { + proc.destroy() + } + + if (stdinWriterThread.isAlive) { + stdinWriterThread.interrupt() + } + if (stderrReaderThread.isAlive) { + stderrReaderThread.interrupt() + } + } // Return an iterator that read lines from the process's stdout val lines = Source.fromInputStream(proc.getInputStream)(encoding).getLines @@ -219,4 +240,7 @@ private object PipedRDD { } buf } + + val STDIN_WRITER_THREAD_PREFIX = "stdin writer for" + val STDERR_READER_THREAD_PREFIX = "stderr reader for" } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 743e3441eea55..1b67e9906457d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -36,6 +36,8 @@ import org.apache.spark.Partitioner._ import org.apache.spark.annotation.{DeveloperApi, Experimental, Since} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.RDD_LIMIT_SCALE_UP_FACTOR import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator import org.apache.spark.partial.GroupedCountEvaluator @@ -210,11 +212,11 @@ abstract class RDD[T: ClassTag]( /** * Mark the RDD as non-persistent, and remove all blocks for it from memory and disk. * - * @param blocking Whether to block until all blocks are deleted. + * @param blocking Whether to block until all blocks are deleted (default: false) * @return This RDD. */ - def unpersist(blocking: Boolean = true): this.type = { - logInfo("Removing RDD " + id + " from persistence list") + def unpersist(blocking: Boolean = false): this.type = { + logInfo(s"Removing RDD $id from persistence list") sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE this @@ -1258,7 +1260,7 @@ abstract class RDD[T: ClassTag]( * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * The relative accuracy is approximately `1.054 / sqrt(2^p)`. Setting a nonzero (`sp` is greater * than `p`) would trigger sparse representation of registers, which may reduce the memory @@ -1290,7 +1292,7 @@ abstract class RDD[T: ClassTag]( * * The algorithm used is based on streamlib's implementation of "HyperLogLog in Practice: * Algorithmic Engineering of a State of The Art Cardinality Estimation Algorithm", available - * here. + * here. * * @param relativeSD Relative accuracy. Smaller values create counters that require more space. * It must be greater than 0.000017. @@ -1349,7 +1351,7 @@ abstract class RDD[T: ClassTag]( * an exception if called on an RDD of `Nothing` or `Null`. */ def take(num: Int): Array[T] = withScope { - val scaleUpFactor = Math.max(conf.getInt("spark.rdd.limit.scaleUpFactor", 4), 2) + val scaleUpFactor = Math.max(conf.get(RDD_LIMIT_SCALE_UP_FACTOR), 2) if (num == 0) { new Array[T](0) } else { @@ -1490,45 +1492,21 @@ abstract class RDD[T: ClassTag]( * Save this RDD as a text file, using string representations of elements. */ def saveAsTextFile(path: String): Unit = withScope { - // https://issues.apache.org/jira/browse/SPARK-2075 - // - // NullWritable is a `Comparable` in Hadoop 1.+, so the compiler cannot find an implicit - // Ordering for it and will use the default `null`. However, it's a `Comparable[NullWritable]` - // in Hadoop 2.+, so the compiler will call the implicit `Ordering.ordered` method to create an - // Ordering for `NullWritable`. That's why the compiler will generate different anonymous - // classes for `saveAsTextFile` in Hadoop 1.+ and Hadoop 2.+. - // - // Therefore, here we provide an explicit Ordering `null` to make sure the compiler generate - // same bytecodes for `saveAsTextFile`. - val nullWritableClassTag = implicitly[ClassTag[NullWritable]] - val textClassTag = implicitly[ClassTag[Text]] - val r = this.mapPartitions { iter => - val text = new Text() - iter.map { x => - text.set(x.toString) - (NullWritable.get(), text) - } - } - RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) - .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path) + saveAsTextFile(path, null) } /** * Save this RDD as a compressed text file, using string representations of elements. */ def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = withScope { - // https://issues.apache.org/jira/browse/SPARK-2075 - val nullWritableClassTag = implicitly[ClassTag[NullWritable]] - val textClassTag = implicitly[ClassTag[Text]] - val r = this.mapPartitions { iter => + this.mapPartitions { iter => val text = new Text() iter.map { x => + require(x != null, "text files do not allow null rows") text.set(x.toString) (NullWritable.get(), text) } - } - RDD.rddToPairRDDFunctions(r)(nullWritableClassTag, textClassTag, null) - .saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) + }.saveAsHadoopFile[TextOutputFormat[NullWritable, Text]](path, codec) } /** @@ -1590,8 +1568,8 @@ abstract class RDD[T: ClassTag]( * The checkpoint directory set through `SparkContext#setCheckpointDir` is not used. */ def localCheckpoint(): this.type = RDDCheckpointData.synchronized { - if (conf.getBoolean("spark.dynamicAllocation.enabled", false) && - conf.contains("spark.dynamicAllocation.cachedExecutorIdleTimeout")) { + if (conf.get(DYN_ALLOCATION_ENABLED) && + conf.contains(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT)) { logWarning("Local checkpointing is NOT safe to use with dynamic allocation, " + "which removes executors along with their cached blocks. If you must use both " + "features, you are advised to set `spark.dynamicAllocation.cachedExecutorIdleTimeout` " + diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala index 8273d8a9eb476..d165610291f1d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableCheckpointRDD.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.CHECKPOINT_COMPRESS +import org.apache.spark.internal.config.{BUFFER_SIZE, CHECKPOINT_COMPRESS} import org.apache.spark.io.CompressionCodec import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -176,7 +176,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { val tempOutputPath = new Path(outputDir, s".$finalOutputName-attempt-${ctx.attemptNumber()}") - val bufferSize = env.conf.getInt("spark.buffer.size", 65536) + val bufferSize = env.conf.get(BUFFER_SIZE) val fileOutputStream = if (blockSize < 0) { val fileStream = fs.create(tempOutputPath, false, bufferSize) @@ -222,7 +222,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { sc: SparkContext, partitioner: Partitioner, checkpointDirPath: Path): Unit = { try { val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) - val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val bufferSize = sc.conf.get(BUFFER_SIZE) val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) val fileOutputStream = fs.create(partitionerFilePath, false, bufferSize) val serializer = SparkEnv.get.serializer.newInstance() @@ -249,7 +249,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { sc: SparkContext, checkpointDirPath: String): Option[Partitioner] = { try { - val bufferSize = sc.conf.getInt("spark.buffer.size", 65536) + val bufferSize = sc.conf.get(BUFFER_SIZE) val partitionerFilePath = new Path(checkpointDirPath, checkpointPartitionerFileName) val fs = partitionerFilePath.getFileSystem(sc.hadoopConfiguration) val fileInputStream = fs.open(partitionerFilePath, bufferSize) @@ -287,7 +287,7 @@ private[spark] object ReliableCheckpointRDD extends Logging { context: TaskContext): Iterator[T] = { val env = SparkEnv.get val fs = path.getFileSystem(broadcastedConf.value.value) - val bufferSize = env.conf.getInt("spark.buffer.size", 65536) + val bufferSize = env.conf.get(BUFFER_SIZE) val fileInputStream = { val fileStream = fs.open(path, bufferSize) if (env.conf.get(CHECKPOINT_COMPRESS)) { diff --git a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala index b6d723c682796..7a592ab11ec79 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ReliableRDDCheckpointData.scala @@ -23,6 +23,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS /** * An implementation of checkpointing that writes the RDD data to reliable storage. @@ -58,7 +59,7 @@ private[spark] class ReliableRDDCheckpointData[T: ClassTag](@transient private v val newRDD = ReliableCheckpointRDD.writeRDDToCheckpointDirectory(rdd, cpDir) // Optionally clean our checkpoint files if the reference is out of scope - if (rdd.conf.getBoolean("spark.cleaner.referenceTracking.cleanCheckpoints", false)) { + if (rdd.conf.get(CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS)) { rdd.context.cleaner.foreach { cleaner => cleaner.registerRDDCheckpointDataForCleanup(newRDD, rdd.id) } diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index e8f9b27b7eb55..5ec99b7f4f3ab 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -101,7 +101,9 @@ class ShuffledRDD[K: ClassTag, V: ClassTag, C: ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[(K, C)] = { val dep = dependencies.head.asInstanceOf[ShuffleDependency[K, V, C]] - SparkEnv.get.shuffleManager.getReader(dep.shuffleHandle, split.index, split.index + 1, context) + val metrics = context.taskMetrics().createTempShuffleReadMetrics() + SparkEnv.get.shuffleManager.getReader( + dep.shuffleHandle, split.index, split.index + 1, context, metrics) .read() .asInstanceOf[Iterator[(K, C)]] } diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index a733eaa5d7e53..42d190377f104 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -107,9 +107,14 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( .asInstanceOf[Iterator[Product2[K, V]]].foreach(op) case shuffleDependency: ShuffleDependency[_, _, _] => + val metrics = context.taskMetrics().createTempShuffleReadMetrics() val iter = SparkEnv.get.shuffleManager .getReader( - shuffleDependency.shuffleHandle, partition.index, partition.index + 1, context) + shuffleDependency.shuffleHandle, + partition.index, + partition.index + 1, + context, + metrics) .read() iter.foreach(op) } diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index 60e383afadf1c..36589e93a1c5e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -21,12 +21,12 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer import scala.collection.parallel.ForkJoinTaskSupport -import scala.concurrent.forkjoin.ForkJoinPool import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.Utils +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD +import org.apache.spark.util.{ThreadUtils, Utils} /** * Partition for UnionRDD. @@ -60,7 +60,7 @@ private[spark] class UnionPartition[T: ClassTag]( object UnionRDD { private[spark] lazy val partitionEvalTaskSupport = - new ForkJoinTaskSupport(new ForkJoinPool(8)) + new ForkJoinTaskSupport(ThreadUtils.newForkJoinPool("partition-eval-task-support", 8)) } @DeveloperApi @@ -71,7 +71,7 @@ class UnionRDD[T: ClassTag]( // visible for testing private[spark] val isPartitionListingParallel: Boolean = - rdds.length > conf.getInt("spark.rdd.parallelListingThreshold", 10) + rdds.length > conf.get(RDD_PARALLEL_LISTING_THRESHOLD) override def getPartitions: Array[Partition] = { val parRDDs = if (isPartitionListingParallel) { diff --git a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala index 9f3d0745c33c9..eada762b99c8e 100644 --- a/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/WholeTextFileRDD.scala @@ -44,7 +44,7 @@ private[spark] class WholeTextFileRDD( // traversing a large number of directories and files. Parallelize it. conf.setIfUnset(FileInputFormat.LIST_STATUS_NUM_THREADS, Runtime.getRuntime.availableProcessors().toString) - val inputFormat = inputFormatClass.newInstance + val inputFormat = inputFormatClass.getConstructor().newInstance() inputFormat match { case configurable: Configurable => configurable.setConf(conf) diff --git a/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala index 5e181a9822534..4a6106984a495 100644 --- a/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala +++ b/core/src/main/scala/org/apache/spark/rdd/util/PeriodicRDDCheckpointer.scala @@ -89,7 +89,7 @@ private[spark] class PeriodicRDDCheckpointer[T]( } } - override protected def unpersist(data: RDD[T]): Unit = data.unpersist(blocking = false) + override protected def unpersist(data: RDD[T]): Unit = data.unpersist() override protected def getCheckpointFiles(data: RDD[T]): Iterable[String] = { data.getCheckpointFile.map(x => x) diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala new file mode 100644 index 0000000000000..96aef74b6ddc3 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceInformation.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import scala.util.control.NonFatal + +import org.json4s.{DefaultFormats, Extraction, JValue} +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.SparkException +import org.apache.spark.annotation.Evolving + +/** + * Class to hold information about a type of Resource. A resource could be a GPU, FPGA, etc. + * The array of addresses are resource specific and its up to the user to interpret the address. + * + * One example is GPUs, where the addresses would be the indices of the GPUs + * + * @param name the name of the resource + * @param addresses an array of strings describing the addresses of the resource + */ +@Evolving +class ResourceInformation( + val name: String, + val addresses: Array[String]) extends Serializable { + + override def toString: String = s"[name: ${name}, addresses: ${addresses.mkString(",")}]" + + override def equals(obj: Any): Boolean = { + obj match { + case that: ResourceInformation => + that.getClass == this.getClass && + that.name == name && that.addresses.toSeq == addresses.toSeq + case _ => + false + } + } + + override def hashCode(): Int = Seq(name, addresses.toSeq).hashCode() +} + +private[spark] object ResourceInformation { + + private lazy val exampleJson: String = compact(render( + ResourceInformationJson("gpu", Seq("0", "1")).toJValue)) + + /** + * Parses a JSON string into a [[ResourceInformation]] instance. + */ + def parseJson(json: String): ResourceInformation = { + implicit val formats = DefaultFormats + try { + parse(json).extract[ResourceInformationJson].toResourceInformation + } catch { + case NonFatal(e) => + throw new SparkException(s"Error parsing JSON into ResourceInformation:\n$json\n" + + s"Here is a correct example: $exampleJson.", e) + } + } +} + +/** A case class to simplify JSON serialization of [[ResourceInformation]]. */ +private case class ResourceInformationJson(name: String, addresses: Seq[String]) { + + def toJValue: JValue = { + Extraction.decompose(this)(DefaultFormats) + } + + def toResourceInformation: ResourceInformation = { + new ResourceInformation(name, addresses.toArray) + } +} diff --git a/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala new file mode 100644 index 0000000000000..69265861a9316 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/resource/ResourceUtils.scala @@ -0,0 +1,191 @@ +/* + * 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.resource + +import java.io.File +import java.nio.file.{Files, Paths} + +import scala.util.control.NonFatal + +import org.json4s.DefaultFormats +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils.executeAndGetOutput + +/** + * Resource identifier. + * @param componentName spark.driver / spark.executor / spark.task + * @param resourceName gpu, fpga, etc + */ +private[spark] case class ResourceID(componentName: String, resourceName: String) { + def confPrefix: String = s"$componentName.resource.$resourceName." // with ending dot + def amountConf: String = s"$confPrefix${ResourceUtils.AMOUNT}" + def discoveryScriptConf: String = s"$confPrefix${ResourceUtils.DISCOVERY_SCRIPT}" + def vendorConf: String = s"$confPrefix${ResourceUtils.VENDOR}" +} + +private[spark] case class ResourceRequest( + id: ResourceID, + amount: Int, + discoveryScript: Option[String], + vendor: Option[String]) + +private[spark] case class TaskResourceRequirement(resourceName: String, amount: Int) + +/** + * Case class representing allocated resource addresses for a specific resource. + * Cluster manager uses the JSON serialization of this case class to pass allocated resource info to + * driver and executors. See the ``--resourcesFile`` option there. + */ +private[spark] case class ResourceAllocation(id: ResourceID, addresses: Seq[String]) { + def toResourceInformation: ResourceInformation = { + new ResourceInformation(id.resourceName, addresses.toArray) + } +} + +private[spark] object ResourceUtils extends Logging { + + // config suffixes + val DISCOVERY_SCRIPT = "discoveryScript" + val VENDOR = "vendor" + // user facing configs use .amount to allow to extend in the future, + // internally we currently only support addresses, so its just an integer count + val AMOUNT = "amount" + + def parseResourceRequest(sparkConf: SparkConf, resourceId: ResourceID): ResourceRequest = { + val settings = sparkConf.getAllWithPrefix(resourceId.confPrefix).toMap + val amount = settings.getOrElse(AMOUNT, + throw new SparkException(s"You must specify an amount for ${resourceId.resourceName}") + ).toInt + val discoveryScript = settings.get(DISCOVERY_SCRIPT) + val vendor = settings.get(VENDOR) + ResourceRequest(resourceId, amount, discoveryScript, vendor) + } + + def listResourceIds(sparkConf: SparkConf, componentName: String): Seq[ResourceID] = { + sparkConf.getAllWithPrefix(s"$componentName.resource.").map { case (key, _) => + key.substring(0, key.indexOf('.')) + }.toSet.toSeq.map(name => ResourceID(componentName, name)) + } + + def parseAllResourceRequests( + sparkConf: SparkConf, + componentName: String): Seq[ResourceRequest] = { + listResourceIds(sparkConf, componentName).map { id => + parseResourceRequest(sparkConf, id) + } + } + + def parseTaskResourceRequirements(sparkConf: SparkConf): Seq[TaskResourceRequirement] = { + parseAllResourceRequests(sparkConf, SPARK_TASK_PREFIX).map { request => + TaskResourceRequirement(request.id.resourceName, request.amount) + } + } + + private def parseAllocatedFromJsonFile(resourcesFile: String): Seq[ResourceAllocation] = { + implicit val formats = DefaultFormats + val json = new String(Files.readAllBytes(Paths.get(resourcesFile))) + try { + parse(json).extract[Seq[ResourceAllocation]] + } catch { + case NonFatal(e) => + throw new SparkException(s"Error parsing resources file $resourcesFile", e) + } + } + + private def parseAllocatedOrDiscoverResources( + sparkConf: SparkConf, + componentName: String, + resourcesFileOpt: Option[String]): Seq[ResourceAllocation] = { + val allocated = resourcesFileOpt.toSeq.flatMap(parseAllocatedFromJsonFile) + .filter(_.id.componentName == componentName) + val otherResourceIds = listResourceIds(sparkConf, componentName).diff(allocated.map(_.id)) + allocated ++ otherResourceIds.map { id => + val request = parseResourceRequest(sparkConf, id) + ResourceAllocation(id, discoverResource(request).addresses) + } + } + + private def assertResourceAllocationMeetsRequest( + allocation: ResourceAllocation, + request: ResourceRequest): Unit = { + require(allocation.id == request.id && allocation.addresses.size >= request.amount, + s"Resource: ${allocation.id.resourceName}, with addresses: " + + s"${allocation.addresses.mkString(",")} " + + s"is less than what the user requested: ${request.amount})") + } + + private def assertAllResourceAllocationsMeetRequests( + allocations: Seq[ResourceAllocation], + requests: Seq[ResourceRequest]): Unit = { + val allocated = allocations.map(x => x.id -> x).toMap + requests.foreach(r => assertResourceAllocationMeetsRequest(allocated(r.id), r)) + } + + /** + * Gets all allocated resource information for the input component from input resources file and + * discover the remaining via discovery scripts. + * It also verifies the resource allocation meets required amount for each resource. + * @return a map from resource name to resource info + */ + def getOrDiscoverAllResources( + sparkConf: SparkConf, + componentName: String, + resourcesFileOpt: Option[String]): Map[String, ResourceInformation] = { + val requests = parseAllResourceRequests(sparkConf, componentName) + val allocations = parseAllocatedOrDiscoverResources(sparkConf, componentName, resourcesFileOpt) + assertAllResourceAllocationsMeetRequests(allocations, requests) + val resourceInfoMap = allocations.map(a => (a.id.resourceName, a.toResourceInformation)).toMap + logInfo("==============================================================") + logInfo(s"Resources for $componentName:\n${resourceInfoMap.mkString("\n")}") + logInfo("==============================================================") + resourceInfoMap + } + + // visible for test + private[spark] def discoverResource(resourceRequest: ResourceRequest): ResourceInformation = { + val resourceName = resourceRequest.id.resourceName + val script = resourceRequest.discoveryScript + val result = if (script.nonEmpty) { + val scriptFile = new File(script.get) + // check that script exists and try to execute + if (scriptFile.exists()) { + val output = executeAndGetOutput(Seq(script.get), new File(".")) + ResourceInformation.parseJson(output) + } else { + throw new SparkException(s"Resource script: $scriptFile to discover $resourceName " + + "doesn't exist!") + } + } else { + throw new SparkException(s"User is expecting to use resource: $resourceName but " + + "didn't specify a discovery script!") + } + if (!result.name.equals(resourceName)) { + throw new SparkException(s"Error running the resource discovery script ${script.get}: " + + s"script returned resource name ${result.name} and we were expecting $resourceName.") + } + result + } + + // known types of resources + final val GPU: String = "gpu" + final val FPGA: String = "fpga" +} diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala index 117f51c5b8f2a..f6b20593462cd 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcCallContext.scala @@ -24,7 +24,7 @@ package org.apache.spark.rpc private[spark] trait RpcCallContext { /** - * Reply a message to the sender. If the sender is [[RpcEndpoint]], its [[RpcEndpoint.receive]] + * Reply a message to the sender. If the sender is [[RpcEndpoint]], its `RpcEndpoint.receive` * will be called. */ def reply(response: Any): Unit diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala index 4d39f144dd198..6c4c0383b3c7f 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcEndpointRef.scala @@ -20,7 +20,7 @@ package org.apache.spark.rpc import scala.concurrent.Future import scala.reflect.ClassTag -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.SparkConf import org.apache.spark.internal.Logging import org.apache.spark.util.RpcUtils diff --git a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala index 3dc41f7f12798..770ae2f1dd22f 100644 --- a/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala +++ b/core/src/main/scala/org/apache/spark/rpc/RpcTimeout.scala @@ -52,7 +52,7 @@ private[spark] class RpcTimeout(val duration: FiniteDuration, val timeoutProp: S * * @note This can be used in the recover callback of a Future to add to a TimeoutException * Example: - * val timeout = new RpcTimeout(5 millis, "short timeout") + * val timeout = new RpcTimeout(5.milliseconds, "short timeout") * Future(throw new TimeoutException).recover(timeout.addMessageIfTimeout) */ def addMessageIfTimeout[T]: PartialFunction[Throwable, T] = { diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala index 904c4d02dd2a4..2f923d7902b05 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Dispatcher.scala @@ -24,8 +24,10 @@ import scala.collection.JavaConverters._ import scala.concurrent.Promise import scala.util.control.NonFatal -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_ID +import org.apache.spark.internal.config.Network.RPC_NETTY_DISPATCHER_NUM_THREADS import org.apache.spark.network.client.RpcResponseCallback import org.apache.spark.rpc._ import org.apache.spark.util.ThreadUtils @@ -193,12 +195,22 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte endpoints.containsKey(name) } - /** Thread pool used for dispatching messages. */ - private val threadpool: ThreadPoolExecutor = { + private def getNumOfThreads(conf: SparkConf): Int = { val availableCores = if (numUsableCores > 0) numUsableCores else Runtime.getRuntime.availableProcessors() - val numThreads = nettyEnv.conf.getInt("spark.rpc.netty.dispatcher.numThreads", - math.max(2, availableCores)) + + val modNumThreads = conf.get(RPC_NETTY_DISPATCHER_NUM_THREADS) + .getOrElse(math.max(2, availableCores)) + + conf.get(EXECUTOR_ID).map { id => + val role = if (id == SparkContext.DRIVER_IDENTIFIER) "driver" else "executor" + conf.getInt(s"spark.$role.rpc.netty.dispatcher.numThreads", modNumThreads) + }.getOrElse(modNumThreads) + } + + /** Thread pool used for dispatching messages. */ + private val threadpool: ThreadPoolExecutor = { + val numThreads = getNumOfThreads(nettyEnv.conf) val pool = ThreadUtils.newDaemonFixedThreadPool(numThreads, "dispatcher-event-loop") for (i <- 0 until numThreads) { pool.execute(new MessageLoop) @@ -224,7 +236,15 @@ private[netty] class Dispatcher(nettyEnv: NettyRpcEnv, numUsableCores: Int) exte } } } catch { - case ie: InterruptedException => // exit + case _: InterruptedException => // exit + case t: Throwable => + try { + // Re-submit a MessageLoop so that Dispatcher will still work if + // UncaughtExceptionHandler decides to not kill JVM. + threadpool.execute(new MessageLoop) + } finally { + throw t + } } } } diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala index d32eba64e13e9..44d2622a42f58 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Inbox.scala @@ -106,7 +106,7 @@ private[netty] class Inbox( throw new SparkException(s"Unsupported message $message from ${_sender}") }) } catch { - case NonFatal(e) => + case e: Throwable => context.sendFailure(e) // Throw the exception -- this exception will be caught by the safelyCall function. // The endpoint's onError function will be called. diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala index 47576959322d1..5dce43b7523d9 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/NettyRpcEnv.scala @@ -29,8 +29,10 @@ import scala.reflect.ClassTag import scala.util.{DynamicVariable, Failure, Success, Try} import scala.util.control.NonFatal -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_ID +import org.apache.spark.internal.config.Network._ import org.apache.spark.network.TransportContext import org.apache.spark.network.client._ import org.apache.spark.network.crypto.{AuthClientBootstrap, AuthServerBootstrap} @@ -46,11 +48,15 @@ private[netty] class NettyRpcEnv( host: String, securityManager: SecurityManager, numUsableCores: Int) extends RpcEnv(conf) with Logging { + val role = conf.get(EXECUTOR_ID).map { id => + if (id == SparkContext.DRIVER_IDENTIFIER) "driver" else "executor" + } private[netty] val transportConf = SparkTransportConf.fromSparkConf( - conf.clone.set("spark.rpc.io.numConnectionsPerPeer", "1"), + conf.clone.set(RPC_IO_NUM_CONNECTIONS_PER_PEER, 1), "rpc", - conf.getInt("spark.rpc.io.threads", numUsableCores)) + conf.get(RPC_IO_THREADS).getOrElse(numUsableCores), + role) private val dispatcher: Dispatcher = new Dispatcher(this, numUsableCores) @@ -87,7 +93,7 @@ private[netty] class NettyRpcEnv( // TODO: a non-blocking TransportClientFactory.createClient in future private[netty] val clientConnectionExecutor = ThreadUtils.newDaemonCachedThreadPool( "netty-rpc-connection", - conf.getInt("spark.rpc.connect.threads", 64)) + conf.get(RPC_CONNECT_THREADS)) @volatile private var server: TransportServer = _ @@ -314,6 +320,9 @@ private[netty] class NettyRpcEnv( if (fileDownloadFactory != null) { fileDownloadFactory.close() } + if (transportContext != null) { + transportContext.close() + } } override def deserialize[T](deserializationAction: () => T): T = { diff --git a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala index b7e068aa68357..3db63934813af 100644 --- a/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala +++ b/core/src/main/scala/org/apache/spark/rpc/netty/Outbox.scala @@ -166,7 +166,7 @@ private[netty] class Outbox(nettyEnv: NettyRpcEnv, val address: RpcAddress) { if (_client != null) { message.sendWith(_client) } else { - assert(stopped == true) + assert(stopped) } } catch { case NonFatal(e) => diff --git a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala index 0a5fe5a1d3ee1..bd0fe90b1f3b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AccumulableInfo.scala @@ -22,7 +22,7 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * Information about an [[org.apache.spark.Accumulable]] modified during a task or stage. + * Information about an [[org.apache.spark.util.AccumulatorV2]] modified during a task or stage. * * @param id accumulator ID * @param name accumulator name @@ -47,33 +47,3 @@ case class AccumulableInfo private[spark] ( private[spark] val countFailedValues: Boolean, // TODO: use this to identify internal task metrics instead of encoding it in the name private[spark] val metadata: Option[String] = None) - - -/** - * A collection of deprecated constructors. This will be removed soon. - */ -object AccumulableInfo { - - @deprecated("do not create AccumulableInfo", "2.0.0") - def apply( - id: Long, - name: String, - update: Option[String], - value: String, - internal: Boolean): AccumulableInfo = { - new AccumulableInfo( - id, Option(name), update, Option(value), internal, countFailedValues = false) - } - - @deprecated("do not create AccumulableInfo", "2.0.0") - def apply(id: Long, name: String, update: Option[String], value: String): AccumulableInfo = { - new AccumulableInfo( - id, Option(name), update, Option(value), internal = false, countFailedValues = false) - } - - @deprecated("do not create AccumulableInfo", "2.0.0") - def apply(id: Long, name: String, value: String): AccumulableInfo = { - new AccumulableInfo( - id, Option(name), None, Option(value), internal = false, countFailedValues = false) - } -} diff --git a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala index e2b6df4600590..7cd2b862216ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/AsyncEventQueue.scala @@ -169,7 +169,8 @@ private class AsyncEventQueue( val prevLastReportTimestamp = lastReportTimestamp lastReportTimestamp = System.currentTimeMillis() val previous = new java.util.Date(prevLastReportTimestamp) - logWarning(s"Dropped $droppedCount events from $name since $previous.") + logWarning(s"Dropped $droppedCount events from $name since " + + s"${if (prevLastReportTimestamp == 0) "the application started" else s"$previous"}.") } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala index 803a0a1226d6c..64a02b30ca0d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BarrierJobAllocationFailed.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler import org.apache.spark.SparkException +import org.apache.spark.internal.config.DYN_ALLOCATION_ENABLED /** * Exception thrown when submit a job with barrier stage(s) failing a required check. @@ -51,7 +52,7 @@ private[spark] object BarrierJobAllocationFailed { val ERROR_MESSAGE_RUN_BARRIER_WITH_DYN_ALLOCATION = "[SPARK-24942]: Barrier execution mode does not support dynamic resource allocation for " + "now. You can disable dynamic resource allocation by setting Spark conf " + - "\"spark.dynamicAllocation.enabled\" to \"false\"." + s""""${DYN_ALLOCATION_ENABLED.key}" to "false".""" // Error message when running a barrier stage that requires more slots than current total number. val ERROR_MESSAGE_BARRIER_REQUIRE_MORE_SLOTS_THAN_CURRENT_TOTAL_NUMBER = diff --git a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala index 980fbbe516b91..9e524c52267be 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/BlacklistTracker.scala @@ -146,21 +146,31 @@ private[scheduler] class BlacklistTracker ( nextExpiryTime = math.min(execMinExpiry, nodeMinExpiry) } + private def killExecutor(exec: String, msg: String): Unit = { + allocationClient match { + case Some(a) => + logInfo(msg) + a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, + force = true) + case None => + logInfo(s"Not attempting to kill blacklisted executor id $exec " + + s"since allocation client is not defined.") + } + } + private def killBlacklistedExecutor(exec: String): Unit = { if (conf.get(config.BLACKLIST_KILL_ENABLED)) { - allocationClient match { - case Some(a) => - logInfo(s"Killing blacklisted executor id $exec " + - s"since ${config.BLACKLIST_KILL_ENABLED.key} is set.") - a.killExecutors(Seq(exec), adjustTargetNumExecutors = false, countFailures = false, - force = true) - case None => - logWarning(s"Not attempting to kill blacklisted executor id $exec " + - s"since allocation client is not defined.") - } + killExecutor(exec, + s"Killing blacklisted executor id $exec since ${config.BLACKLIST_KILL_ENABLED.key} is set.") } } + private[scheduler] def killBlacklistedIdleExecutor(exec: String): Unit = { + killExecutor(exec, + s"Killing blacklisted idle executor id $exec because of task unschedulability and trying " + + "to acquire a new executor.") + } + private def killExecutorsOnBlacklistedNode(node: String): Unit = { if (conf.get(config.BLACKLIST_KILL_ENABLED)) { allocationClient match { @@ -450,15 +460,15 @@ private[spark] object BlacklistTracker extends Logging { } } - val maxTaskFailures = conf.get(config.MAX_TASK_FAILURES) + val maxTaskFailures = conf.get(config.TASK_MAX_FAILURES) val maxNodeAttempts = conf.get(config.MAX_TASK_ATTEMPTS_PER_NODE) if (maxNodeAttempts >= maxTaskFailures) { throw new IllegalArgumentException(s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + s"Spark will not be robust to one bad node. Decrease " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index f93d8a8d5de55..5072e617937e5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -21,14 +21,11 @@ import java.io.NotSerializableException import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicInteger -import java.util.function.BiFunction import scala.annotation.tailrec import scala.collection.Map -import scala.collection.mutable.{ArrayStack, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet, ListBuffer} import scala.concurrent.duration._ -import scala.language.existentials -import scala.language.postfixOps import scala.util.control.NonFatal import org.apache.commons.lang3.SerializationUtils @@ -38,6 +35,7 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.network.util.JavaUtils import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.{DeterministicLevel, RDD, RDDCheckpointData} @@ -186,7 +184,7 @@ private[spark] class DAGScheduler( private val closureSerializer = SparkEnv.get.closureSerializer.newInstance() /** If enabled, FetchFailed will not cause stage retry, in order to surface the problem. */ - private val disallowStageRetryForTest = sc.getConf.getBoolean("spark.test.noStageRetry", false) + private val disallowStageRetryForTest = sc.getConf.get(TEST_NO_STAGE_RETRY) /** * Whether to unregister all the outputs on the host in condition that we receive a FetchFailure, @@ -265,12 +263,12 @@ private[spark] class DAGScheduler( // (taskId, stageId, stageAttemptId, accumUpdates) accumUpdates: Array[(Long, Int, Int, Seq[AccumulableInfo])], blockManagerId: BlockManagerId, - // executor metrics indexed by MetricGetter.values + // executor metrics indexed by ExecutorMetricType.values executorUpdates: ExecutorMetrics): Boolean = { listenerBus.post(SparkListenerExecutorMetricsUpdate(execId, accumUpdates, Some(executorUpdates))) blockManagerMaster.driverEndpoint.askSync[Boolean]( - BlockManagerHeartbeat(blockManagerId), new RpcTimeout(600 seconds, "BlockManagerHeartbeat")) + BlockManagerHeartbeat(blockManagerId), new RpcTimeout(10.minutes, "BlockManagerHeartbeat")) } /** @@ -369,9 +367,10 @@ private[spark] class DAGScheduler( * 2. An RDD that depends on multiple barrier RDDs (eg. barrierRdd1.zip(barrierRdd2)). */ private def checkBarrierStageWithRDDChainPattern(rdd: RDD[_], numTasksInStage: Int): Unit = { - val predicate: RDD[_] => Boolean = (r => - r.getNumPartitions == numTasksInStage && r.dependencies.filter(_.rdd.isBarrier()).size <= 1) - if (rdd.isBarrier() && !traverseParentRDDsWithinStage(rdd, predicate)) { + if (rdd.isBarrier() && + !traverseParentRDDsWithinStage(rdd, (r: RDD[_]) => + r.getNumPartitions == numTasksInStage && + r.dependencies.count(_.rdd.isBarrier()) <= 1)) { throw new BarrierJobUnsupportedRDDChainException } } @@ -382,7 +381,8 @@ private[spark] class DAGScheduler( * locations that are still available from the previous shuffle to avoid unnecessarily * regenerating data. */ - def createShuffleMapStage(shuffleDep: ShuffleDependency[_, _, _], jobId: Int): ShuffleMapStage = { + def createShuffleMapStage[K, V, C]( + shuffleDep: ShuffleDependency[K, V, C], jobId: Int): ShuffleMapStage = { val rdd = shuffleDep.rdd checkBarrierStageWithDynamicAllocation(rdd) checkBarrierStageWithNumSlots(rdd) @@ -468,21 +468,21 @@ private[spark] class DAGScheduler( /** Find ancestor shuffle dependencies that are not registered in shuffleToMapStage yet */ private def getMissingAncestorShuffleDependencies( - rdd: RDD[_]): ArrayStack[ShuffleDependency[_, _, _]] = { - val ancestors = new ArrayStack[ShuffleDependency[_, _, _]] + rdd: RDD[_]): ListBuffer[ShuffleDependency[_, _, _]] = { + val ancestors = new ListBuffer[ShuffleDependency[_, _, _]] val visited = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new ArrayStack[RDD[_]] - waitingForVisit.push(rdd) + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += rdd while (waitingForVisit.nonEmpty) { - val toVisit = waitingForVisit.pop() + val toVisit = waitingForVisit.remove(0) if (!visited(toVisit)) { visited += toVisit getShuffleDependencies(toVisit).foreach { shuffleDep => if (!shuffleIdToMapStage.contains(shuffleDep.shuffleId)) { - ancestors.push(shuffleDep) - waitingForVisit.push(shuffleDep.rdd) + ancestors.prepend(shuffleDep) + waitingForVisit.prepend(shuffleDep.rdd) } // Otherwise, the dependency and its ancestors have already been registered. } } @@ -506,17 +506,17 @@ private[spark] class DAGScheduler( rdd: RDD[_]): HashSet[ShuffleDependency[_, _, _]] = { val parents = new HashSet[ShuffleDependency[_, _, _]] val visited = new HashSet[RDD[_]] - val waitingForVisit = new ArrayStack[RDD[_]] - waitingForVisit.push(rdd) + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += rdd while (waitingForVisit.nonEmpty) { - val toVisit = waitingForVisit.pop() + val toVisit = waitingForVisit.remove(0) if (!visited(toVisit)) { visited += toVisit toVisit.dependencies.foreach { case shuffleDep: ShuffleDependency[_, _, _] => parents += shuffleDep case dependency => - waitingForVisit.push(dependency.rdd) + waitingForVisit.prepend(dependency.rdd) } } } @@ -529,10 +529,10 @@ private[spark] class DAGScheduler( */ private def traverseParentRDDsWithinStage(rdd: RDD[_], predicate: RDD[_] => Boolean): Boolean = { val visited = new HashSet[RDD[_]] - val waitingForVisit = new ArrayStack[RDD[_]] - waitingForVisit.push(rdd) + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += rdd while (waitingForVisit.nonEmpty) { - val toVisit = waitingForVisit.pop() + val toVisit = waitingForVisit.remove(0) if (!visited(toVisit)) { if (!predicate(toVisit)) { return false @@ -542,7 +542,7 @@ private[spark] class DAGScheduler( case _: ShuffleDependency[_, _, _] => // Not within the same stage with current rdd, do nothing. case dependency => - waitingForVisit.push(dependency.rdd) + waitingForVisit.prepend(dependency.rdd) } } } @@ -554,7 +554,8 @@ private[spark] class DAGScheduler( val visited = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new ArrayStack[RDD[_]] + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += stage.rdd def visit(rdd: RDD[_]) { if (!visited(rdd)) { visited += rdd @@ -568,15 +569,14 @@ private[spark] class DAGScheduler( missing += mapStage } case narrowDep: NarrowDependency[_] => - waitingForVisit.push(narrowDep.rdd) + waitingForVisit.prepend(narrowDep.rdd) } } } } } - waitingForVisit.push(stage.rdd) while (waitingForVisit.nonEmpty) { - visit(waitingForVisit.pop()) + visit(waitingForVisit.remove(0)) } missing.toList } @@ -691,14 +691,19 @@ private[spark] class DAGScheduler( } val jobId = nextJobId.getAndIncrement() - if (partitions.size == 0) { + if (partitions.isEmpty) { + val time = clock.getTimeMillis() + listenerBus.post( + SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties)) + listenerBus.post( + SparkListenerJobEnd(jobId, time, JobSucceeded)) // Return immediately if the job is running 0 tasks return new JobWaiter[U](this, jobId, 0, resultHandler) } - assert(partitions.size > 0) + assert(partitions.nonEmpty) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val waiter = new JobWaiter(this, jobId, partitions.size, resultHandler) + val waiter = new JobWaiter[U](this, jobId, partitions.size, resultHandler) eventProcessLoop.post(JobSubmitted( jobId, rdd, func2, partitions.toArray, callSite, waiter, SerializationUtils.clone(properties))) @@ -761,12 +766,19 @@ private[spark] class DAGScheduler( callSite: CallSite, timeout: Long, properties: Properties): PartialResult[R] = { + val jobId = nextJobId.getAndIncrement() + if (rdd.partitions.isEmpty) { + // Return immediately if the job is running 0 tasks + val time = clock.getTimeMillis() + listenerBus.post(SparkListenerJobStart(jobId, time, Seq[StageInfo](), properties)) + listenerBus.post(SparkListenerJobEnd(jobId, time, JobSucceeded)) + return new PartialResult(evaluator.currentResult(), true) + } val listener = new ApproximateActionListener(rdd, func, evaluator, timeout) val func2 = func.asInstanceOf[(TaskContext, Iterator[_]) => _] - val partitions = (0 until rdd.partitions.length).toArray - val jobId = nextJobId.getAndIncrement() eventProcessLoop.post(JobSubmitted( - jobId, rdd, func2, partitions, callSite, listener, SerializationUtils.clone(properties))) + jobId, rdd, func2, rdd.partitions.indices.toArray, callSite, listener, + SerializationUtils.clone(properties))) listener.awaitResult() // Will throw an exception if the job fails } @@ -799,7 +811,9 @@ private[spark] class DAGScheduler( // This makes it easier to avoid race conditions between the user code and the map output // tracker that might result if we told the user the stage had finished, but then they queries // the map output tracker and some node failures had caused the output statistics to be lost. - val waiter = new JobWaiter(this, jobId, 1, (i: Int, r: MapOutputStatistics) => callback(r)) + val waiter = new JobWaiter[MapOutputStatistics]( + this, jobId, 1, + (_: Int, r: MapOutputStatistics) => callback(r)) eventProcessLoop.post(MapStageSubmitted( jobId, dependency, callSite, waiter, SerializationUtils.clone(properties))) waiter @@ -857,7 +871,7 @@ private[spark] class DAGScheduler( * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { - if (failedStages.size > 0) { + if (failedStages.nonEmpty) { // Failed stages may be removed by job cancellation, so failed might be empty even if // the ResubmitFailedStages event has been scheduled. logInfo("Resubmitting failed stages") @@ -919,7 +933,7 @@ private[spark] class DAGScheduler( } private[scheduler] def handleSpeculativeTaskSubmitted(task: Task[_]): Unit = { - listenerBus.post(SparkListenerSpeculativeTaskSubmitted(task.stageId)) + listenerBus.post(SparkListenerSpeculativeTaskSubmitted(task.stageId, task.stageAttemptId)) } private[scheduler] def handleTaskSetFailed( @@ -969,9 +983,7 @@ private[spark] class DAGScheduler( "than the total number of slots in the cluster currently.") // If jobId doesn't exist in the map, Scala coverts its value null to 0: Int automatically. val numCheckFailures = barrierJobIdToNumTasksCheckFailures.compute(jobId, - new BiFunction[Int, Int, Int] { - override def apply(key: Int, value: Int): Int = value + 1 - }) + (_: Int, value: Int) => value + 1) if (numCheckFailures <= maxFailureNumTasksCheck) { messageScheduler.schedule( new Runnable { @@ -1161,6 +1173,10 @@ private[spark] class DAGScheduler( partitions = stage.rdd.partitions } + if (taskBinaryBytes.length > TaskSetManager.TASK_SIZE_TO_WARN_KIB * 1024) { + logWarning(s"Broadcasting large task binary with size " + + s"${Utils.bytesToString(taskBinaryBytes.length)}") + } taskBinary = sc.broadcast(taskBinaryBytes) } catch { // In the case of a failure during serialization, abort the stage. @@ -1170,9 +1186,11 @@ private[spark] class DAGScheduler( // Abort execution return - case NonFatal(e) => + case e: Throwable => abortStage(stage, s"Task serialization failed: $e\n${Utils.exceptionString(e)}", Some(e)) runningStages -= stage + + // Abort execution return } @@ -1208,7 +1226,7 @@ private[spark] class DAGScheduler( return } - if (tasks.size > 0) { + if (tasks.nonEmpty) { logInfo(s"Submitting ${tasks.size} missing tasks from $stage (${stage.rdd}) (first 15 " + s"tasks are for partitions ${tasks.take(15).map(_.partitionId)})") taskScheduler.submitTasks(new TaskSet( @@ -1295,6 +1313,27 @@ private[spark] class DAGScheduler( Utils.getFormattedClassName(event.task), event.reason, event.taskInfo, taskMetrics)) } + /** + * Check [[SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL]] in job properties to see if we should + * interrupt running tasks. Returns `false` if the property value is not a boolean value + */ + private def shouldInterruptTaskThread(job: ActiveJob): Boolean = { + if (job.properties == null) { + false + } else { + val shouldInterruptThread = + job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false") + try { + shouldInterruptThread.toBoolean + } catch { + case e: IllegalArgumentException => + logWarning(s"${SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL} in Job ${job.jobId} " + + s"is invalid: $shouldInterruptThread. Using 'false' instead", e) + false + } + } + } + /** * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. @@ -1350,6 +1389,14 @@ private[spark] class DAGScheduler( event.reason match { case Success => + // An earlier attempt of a stage (which is zombie) may still have running tasks. If these + // tasks complete, they still count and we can mark the corresponding partitions as + // finished. Here we notify the task scheduler to skip running tasks for the same partition, + // to save resource. + if (task.stageAttemptId < stage.latestInfo.attemptNumber()) { + taskScheduler.notifyPartitionCompletion(stageId, task.partitionId) + } + task match { case rt: ResultTask[_, _] => // Cast to ResultStage here because it's part of the ResultTask @@ -1364,6 +1411,21 @@ private[spark] class DAGScheduler( if (job.numFinished == job.numPartitions) { markStageAsFinished(resultStage) cleanupStateForJobAndIndependentStages(job) + try { + // killAllTaskAttempts will fail if a SchedulerBackend does not implement + // killTask. + logInfo(s"Job ${job.jobId} is finished. Cancelling potential speculative " + + "or zombie tasks for this job") + // ResultStage is only used by this job. It's safe to kill speculative or + // zombie tasks in this stage. + taskScheduler.killAllTaskAttempts( + stageId, + shouldInterruptTaskThread(job), + reason = "Stage finished") + } catch { + case e: UnsupportedOperationException => + logWarning(s"Could not cancel tasks for stage $stageId", e) + } listenerBus.post( SparkListenerJobEnd(job.jobId, clock.getTimeMillis(), JobSucceeded)) } @@ -1373,7 +1435,7 @@ private[spark] class DAGScheduler( try { job.listener.taskSucceeded(rt.outputId, event.result) } catch { - case e: Exception => + case e: Throwable if !Utils.isFatalError(e) => // TODO: Perhaps we want to mark the resultStage as failed? job.listener.jobFailed(new SparkDriverExecutionException(e)) } @@ -1887,13 +1949,9 @@ private[spark] class DAGScheduler( job: ActiveJob, failureReason: String, exception: Option[Throwable] = None): Unit = { - val error = new SparkException(failureReason, exception.getOrElse(null)) + val error = new SparkException(failureReason, exception.orNull) var ableToCancelStages = true - val shouldInterruptThread = - if (job.properties == null) false - else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean - // Cancel all independent, running stages. val stages = jobIdToStageIds(job.jobId) if (stages.isEmpty) { @@ -1913,12 +1971,12 @@ private[spark] class DAGScheduler( val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { try { // cancelTasks will fail if a SchedulerBackend does not implement killTask - taskScheduler.cancelTasks(stageId, shouldInterruptThread) + taskScheduler.cancelTasks(stageId, shouldInterruptTaskThread(job)) markStageAsFinished(stage, Some(failureReason)) } catch { case e: UnsupportedOperationException => - logInfo(s"Could not cancel tasks for stage $stageId", e) - ableToCancelStages = false + logWarning(s"Could not cancel tasks for stage $stageId", e) + ableToCancelStages = false } } } @@ -1942,7 +2000,8 @@ private[spark] class DAGScheduler( val visitedRdds = new HashSet[RDD[_]] // We are manually maintaining a stack here to prevent StackOverflowError // caused by recursively visiting - val waitingForVisit = new ArrayStack[RDD[_]] + val waitingForVisit = new ListBuffer[RDD[_]] + waitingForVisit += stage.rdd def visit(rdd: RDD[_]) { if (!visitedRdds(rdd)) { visitedRdds += rdd @@ -1951,17 +2010,16 @@ private[spark] class DAGScheduler( case shufDep: ShuffleDependency[_, _, _] => val mapStage = getOrCreateShuffleMapStage(shufDep, stage.firstJobId) if (!mapStage.isAvailable) { - waitingForVisit.push(mapStage.rdd) + waitingForVisit.prepend(mapStage.rdd) } // Otherwise there's no need to follow the dependency back case narrowDep: NarrowDependency[_] => - waitingForVisit.push(narrowDep.rdd) + waitingForVisit.prepend(narrowDep.rdd) } } } } - waitingForVisit.push(stage.rdd) while (waitingForVisit.nonEmpty) { - visit(waitingForVisit.pop()) + visit(waitingForVisit.remove(0)) } visitedRdds.contains(target.rdd) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 54ab8f8b3e1d8..b514c2e7056f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -19,8 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import scala.language.existentials - import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.util.{AccumulatorV2, CallSite} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 1629e1797977f..20c74b16ce08b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -20,16 +20,12 @@ package org.apache.spark.scheduler import java.io._ import java.net.URI import java.nio.charset.StandardCharsets -import java.util.EnumSet -import java.util.Locale import scala.collection.mutable.{ArrayBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.hdfs.DFSOutputStream -import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag import org.json4s.JsonAST.JValue import org.json4s.jackson.JsonMethods._ @@ -48,6 +44,7 @@ import org.apache.spark.util.{JsonProtocol, Utils} * spark.eventLog.enabled - Whether event logging is enabled. * spark.eventLog.logBlockUpdates.enabled - Whether to log block updates * spark.eventLog.compress - Whether to compress logged events + * spark.eventLog.compression.codec - The codec to compress logged events * spark.eventLog.overwrite - Whether to overwrite any existing files. * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams @@ -70,17 +67,19 @@ private[spark] class EventLoggingListener( private val shouldCompress = sparkConf.get(EVENT_LOG_COMPRESS) private val shouldOverwrite = sparkConf.get(EVENT_LOG_OVERWRITE) private val shouldLogBlockUpdates = sparkConf.get(EVENT_LOG_BLOCK_UPDATES) + private val shouldAllowECLogs = sparkConf.get(EVENT_LOG_ALLOW_EC) private val shouldLogStageExecutorMetrics = sparkConf.get(EVENT_LOG_STAGE_EXECUTOR_METRICS) private val testing = sparkConf.get(EVENT_LOG_TESTING) private val outputBufferSize = sparkConf.get(EVENT_LOG_OUTPUT_BUFFER_SIZE).toInt private val fileSystem = Utils.getHadoopFileSystem(logBaseDir, hadoopConf) private val compressionCodec = if (shouldCompress) { - Some(CompressionCodec.createCodec(sparkConf)) + Some(CompressionCodec.createCodec(sparkConf, sparkConf.get(EVENT_LOG_COMPRESSION_CODEC))) } else { None } - private val compressionCodecName = compressionCodec.map { c => + // Visible for tests only. + private[scheduler] val compressionCodecName = compressionCodec.map { c => CompressionCodec.getShortName(c.getClass.getName) } @@ -122,7 +121,11 @@ private[spark] class EventLoggingListener( if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { new FileOutputStream(uri.getPath) } else { - hadoopDataStream = Some(fileSystem.create(path)) + hadoopDataStream = Some(if (shouldAllowECLogs) { + fileSystem.create(path) + } else { + SparkHadoopUtil.createNonECFile(fileSystem, path) + }) hadoopDataStream.get } @@ -149,10 +152,7 @@ private[spark] class EventLoggingListener( // scalastyle:on println if (flushLogger) { writer.foreach(_.flush()) - hadoopDataStream.foreach(ds => ds.getWrappedStream match { - case wrapped: DFSOutputStream => wrapped.hsync(EnumSet.of(SyncFlag.UPDATE_LENGTH)) - case _ => ds.hflush() - }) + hadoopDataStream.foreach(_.hflush()) } if (testing) { loggedEvents += eventJson @@ -383,19 +383,15 @@ private[spark] object EventLoggingListener extends Logging { appId: String, appAttemptId: Option[String], compressionCodecName: Option[String] = None): String = { - val base = new Path(logBaseDir).toString.stripSuffix("/") + "/" + sanitize(appId) + val base = new Path(logBaseDir).toString.stripSuffix("/") + "/" + Utils.sanitizeDirName(appId) val codec = compressionCodecName.map("." + _).getOrElse("") if (appAttemptId.isDefined) { - base + "_" + sanitize(appAttemptId.get) + codec + base + "_" + Utils.sanitizeDirName(appAttemptId.get) + codec } else { base + codec } } - private def sanitize(str: String): String = { - str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase(Locale.ROOT) - } - /** * Opens an event log file and returns an input stream that contains the event data. * @@ -407,7 +403,7 @@ private[spark] object EventLoggingListener extends Logging { val codec = codecName(log).map { c => codecMap.getOrElseUpdate(c, CompressionCodec.createCodec(new SparkConf, c)) } - codec.map(_.compressedInputStream(in)).getOrElse(in) + codec.map(_.compressedContinuousInputStream(in)).getOrElse(in) } catch { case e: Throwable => in.close() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala new file mode 100644 index 0000000000000..c75931d53b4be --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ExecutorResourceInfo.scala @@ -0,0 +1,101 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable + +import org.apache.spark.SparkException +import org.apache.spark.util.collection.OpenHashMap + +/** + * Class to hold information about a type of Resource on an Executor. This information is managed + * by SchedulerBackend, and TaskScheduler shall schedule tasks on idle Executors based on the + * information. + * Please note that this class is intended to be used in a single thread. + * @param name Resource name + * @param addresses Resource addresses provided by the executor + */ +private[spark] class ExecutorResourceInfo( + val name: String, + addresses: Seq[String]) extends Serializable { + + /** + * Map from an address to its availability, the value `true` means the address is available, + * while value `false` means the address is assigned. + * TODO Use [[OpenHashMap]] instead to gain better performance. + */ + private val addressAvailabilityMap = mutable.HashMap(addresses.map(_ -> true): _*) + + /** + * Sequence of currently available resource addresses. + */ + def availableAddrs: Seq[String] = addressAvailabilityMap.flatMap { case (addr, available) => + if (available) Some(addr) else None + }.toSeq + + /** + * Sequence of currently assigned resource addresses. + * Exposed for testing only. + */ + private[scheduler] def assignedAddrs: Seq[String] = addressAvailabilityMap + .flatMap { case (addr, available) => + if (!available) Some(addr) else None + }.toSeq + + /** + * Acquire a sequence of resource addresses (to a launched task), these addresses must be + * available. When the task finishes, it will return the acquired resource addresses. + * Throw an Exception if an address is not available or doesn't exist. + */ + def acquire(addrs: Seq[String]): Unit = { + addrs.foreach { address => + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to acquire an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) + if (isAvailable) { + addressAvailabilityMap(address) = false + } else { + throw new SparkException(s"Try to acquire an address that is not available. $name " + + s"address $address is not available.") + } + } + } + + /** + * Release a sequence of resource addresses, these addresses must have been assigned. Resource + * addresses are released when a task has finished. + * Throw an Exception if an address is not assigned or doesn't exist. + */ + def release(addrs: Seq[String]): Unit = { + addrs.foreach { address => + if (!addressAvailabilityMap.contains(address)) { + throw new SparkException(s"Try to release an address that doesn't exist. $name address " + + s"$address doesn't exist.") + } + val isAvailable = addressAvailabilityMap(address) + if (!isAvailable) { + addressAvailabilityMap(address) = true + } else { + throw new SparkException(s"Try to release an address that is not assigned. $name " + + s"address $address is not assigned.") + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala index 0e221edf3965a..64f0a060a247c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/MapStatus.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler import java.io.{Externalizable, ObjectInput, ObjectOutput} import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer import org.roaringbitmap.RoaringBitmap @@ -149,7 +148,7 @@ private[spark] class HighlyCompressedMapStatus private ( private[this] var numNonEmptyBlocks: Int, private[this] var emptyBlocks: RoaringBitmap, private[this] var avgSize: Long, - private var hugeBlockSizes: Map[Int, Byte]) + private[this] var hugeBlockSizes: scala.collection.Map[Int, Byte]) extends MapStatus with Externalizable { // loc could be null when the default constructor is called during deserialization @@ -189,13 +188,13 @@ private[spark] class HighlyCompressedMapStatus private ( emptyBlocks.readExternal(in) avgSize = in.readLong() val count = in.readInt() - val hugeBlockSizesArray = mutable.ArrayBuffer[Tuple2[Int, Byte]]() + val hugeBlockSizesImpl = mutable.Map.empty[Int, Byte] (0 until count).foreach { _ => val block = in.readInt() val size = in.readByte() - hugeBlockSizesArray += Tuple2(block, size) + hugeBlockSizesImpl(block) = size } - hugeBlockSizes = hugeBlockSizesArray.toMap + hugeBlockSizes = hugeBlockSizesImpl } } @@ -215,7 +214,7 @@ private[spark] object HighlyCompressedMapStatus { val threshold = Option(SparkEnv.get) .map(_.conf.get(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD)) .getOrElse(config.SHUFFLE_ACCURATE_BLOCK_THRESHOLD.defaultValue.get) - val hugeBlockSizesArray = ArrayBuffer[Tuple2[Int, Byte]]() + val hugeBlockSizes = mutable.Map.empty[Int, Byte] while (i < totalNumBlocks) { val size = uncompressedSizes(i) if (size > 0) { @@ -226,7 +225,7 @@ private[spark] object HighlyCompressedMapStatus { totalSmallBlockSize += size numSmallBlocks += 1 } else { - hugeBlockSizesArray += Tuple2(i, MapStatus.compressSize(uncompressedSizes(i))) + hugeBlockSizes(i) = MapStatus.compressSize(uncompressedSizes(i)) } } else { emptyBlocks.add(i) @@ -241,6 +240,6 @@ private[spark] object HighlyCompressedMapStatus { emptyBlocks.trim() emptyBlocks.runOptimize() new HighlyCompressedMapStatus(loc, numNonEmptyBlocks, emptyBlocks, avgSize, - hugeBlockSizesArray.toMap) + hugeBlockSizes) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index aafeae05b566c..857c89d7a98f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -75,14 +75,14 @@ private[spark] class ResultTask[T, U]( override def runTask(context: TaskContext): U = { // Deserialize the RDD and the func using the broadcast variables. val threadMXBean = ManagementFactory.getThreadMXBean - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTimeNs = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() val (rdd, func) = ser.deserialize[(RDD[T], (TaskContext, Iterator[T]) => U)]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) - _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeTimeNs = System.nanoTime() - deserializeStartTimeNs _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index 5f3c280ec31ed..c85c74f2fb973 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -23,8 +23,9 @@ import java.util.{Locale, NoSuchElementException, Properties} import scala.util.control.NonFatal import scala.xml.{Node, XML} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SCHEDULER_ALLOCATION_FILE import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.util.Utils @@ -56,10 +57,9 @@ private[spark] class FIFOSchedulableBuilder(val rootPool: Pool) private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) extends SchedulableBuilder with Logging { - val SCHEDULER_ALLOCATION_FILE_PROPERTY = "spark.scheduler.allocation.file" - val schedulerAllocFile = conf.getOption(SCHEDULER_ALLOCATION_FILE_PROPERTY) + val schedulerAllocFile = conf.get(SCHEDULER_ALLOCATION_FILE) val DEFAULT_SCHEDULER_FILE = "fairscheduler.xml" - val FAIR_SCHEDULER_PROPERTIES = "spark.scheduler.pool" + val FAIR_SCHEDULER_PROPERTIES = SparkContext.SPARK_SCHEDULER_POOL val DEFAULT_POOL_NAME = "default" val MINIMUM_SHARES_PROPERTY = "minShare" val SCHEDULING_MODE_PROPERTY = "schedulingMode" @@ -85,7 +85,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) } else { logWarning("Fair Scheduler configuration file not found so jobs will be scheduled in " + s"FIFO order. To use fair scheduling, configure pools in $DEFAULT_SCHEDULER_FILE or " + - s"set $SCHEDULER_ALLOCATION_FILE_PROPERTY to a file that contains the configuration.") + s"set ${SCHEDULER_ALLOCATION_FILE.key} to a file that contains the configuration.") None } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index c187ee146301b..9159d2a0158d5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -69,6 +69,13 @@ private[spark] trait SchedulerBackend { */ def getDriverLogUrls: Option[Map[String, String]] = None + /** + * Get the attributes on driver. These attributes are used to replace log URLs when + * custom log url pattern is specified. + * @return Map containing attributes on driver. + */ + def getDriverAttributes: Option[Map[String, String]] = None + /** * Get the max number of tasks that can be concurrent launched currently. * Note that please don't cache the value returned by this method, because the number can change diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index f2cd65fd523ab..710f5eb211dde 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -21,13 +21,10 @@ import java.lang.management.ManagementFactory import java.nio.ByteBuffer import java.util.Properties -import scala.language.existentials - import org.apache.spark._ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD -import org.apache.spark.shuffle.ShuffleWriter /** * A ShuffleMapTask divides the elements of an RDD into multiple buckets (based on a partitioner @@ -80,36 +77,21 @@ private[spark] class ShuffleMapTask( override def runTask(context: TaskContext): MapStatus = { // Deserialize the RDD using the broadcast variable. val threadMXBean = ManagementFactory.getThreadMXBean - val deserializeStartTime = System.currentTimeMillis() + val deserializeStartTimeNs = System.nanoTime() val deserializeStartCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime } else 0L val ser = SparkEnv.get.closureSerializer.newInstance() - val (rdd, dep) = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( + val rddAndDep = ser.deserialize[(RDD[_], ShuffleDependency[_, _, _])]( ByteBuffer.wrap(taskBinary.value), Thread.currentThread.getContextClassLoader) - _executorDeserializeTime = System.currentTimeMillis() - deserializeStartTime + _executorDeserializeTimeNs = System.nanoTime() - deserializeStartTimeNs _executorDeserializeCpuTime = if (threadMXBean.isCurrentThreadCpuTimeSupported) { threadMXBean.getCurrentThreadCpuTime - deserializeStartCpuTime } else 0L - var writer: ShuffleWriter[Any, Any] = null - try { - val manager = SparkEnv.get.shuffleManager - writer = manager.getWriter[Any, Any](dep.shuffleHandle, partitionId, context) - writer.write(rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) - writer.stop(success = true).get - } catch { - case e: Exception => - try { - if (writer != null) { - writer.stop(success = false) - } - } catch { - case e: Exception => - log.debug("Could not stop writer", e) - } - throw e - } + val rdd = rddAndDep._1 + val dep = rddAndDep._2 + dep.shuffleWriterProcessor.write(rdd, dep, partitionId, context, partition) } override def preferredLocations: Seq[TaskLocation] = preferredLocs diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 293e8369677f0..666ce3dc7c16f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -24,12 +24,11 @@ import scala.collection.Map import com.fasterxml.jackson.annotation.JsonTypeInfo -import org.apache.spark.{SparkConf, TaskEndReason} +import org.apache.spark.TaskEndReason import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.storage.{BlockManagerId, BlockUpdatedInfo} -import org.apache.spark.ui.SparkUI @DeveloperApi @JsonTypeInfo(use = JsonTypeInfo.Id.CLASS, include = JsonTypeInfo.As.PROPERTY, property = "Event") @@ -53,7 +52,10 @@ case class SparkListenerTaskStart(stageId: Int, stageAttemptId: Int, taskInfo: T case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent @DeveloperApi -case class SparkListenerSpeculativeTaskSubmitted(stageId: Int) extends SparkListenerEvent +case class SparkListenerSpeculativeTaskSubmitted( + stageId: Int, + stageAttemptId: Int = 0) + extends SparkListenerEvent @DeveloperApi case class SparkListenerTaskEnd( @@ -175,7 +177,7 @@ case class SparkListenerExecutorMetricsUpdate( * @param execId executor id * @param stageId stage id * @param stageAttemptId stage attempt - * @param executorMetrics executor level metrics, indexed by MetricGetter.values + * @param executorMetrics executor level metrics, indexed by ExecutorMetricType.values */ @DeveloperApi case class SparkListenerStageExecutorMetrics( @@ -192,7 +194,8 @@ case class SparkListenerApplicationStart( time: Long, sparkUser: String, appAttemptId: Option[String], - driverLogs: Option[Map[String, String]] = None) extends SparkListenerEvent + driverLogs: Option[Map[String, String]] = None, + driverAttributes: Option[Map[String, String]] = None) extends SparkListenerEvent @DeveloperApi case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 903e25b7986f2..e3216151462bd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -30,14 +30,15 @@ import org.apache.spark.storage.RDDInfo @DeveloperApi class StageInfo( val stageId: Int, - @deprecated("Use attemptNumber instead", "2.3.0") val attemptId: Int, + private val attemptId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo], val parentIds: Seq[Int], val details: String, val taskMetrics: TaskMetrics = null, - private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty) { + private[spark] val taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + private[spark] val shuffleDepId: Option[Int] = None) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -56,6 +57,8 @@ class StageInfo( completionTime = Some(System.currentTimeMillis) } + // This would just be the second constructor arg, except we need to maintain this method + // with parentheses for compatibility def attemptNumber(): Int = attemptId private[spark] def getStatusString: String = { @@ -88,6 +91,10 @@ private[spark] object StageInfo { ): StageInfo = { val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + val shuffleDepId = stage match { + case sms: ShuffleMapStage => Option(sms.shuffleDep).map(_.shuffleId) + case _ => None + } new StageInfo( stage.id, attemptId, @@ -97,6 +104,7 @@ private[spark] object StageInfo { stage.parents.map(_.id), stage.details, taskMetrics, - taskLocalityPreferences) + taskLocalityPreferences, + shuffleDepId) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index eb059f12be6d3..9dfbf862a9c57 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -25,6 +25,8 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.config.APP_CALLER_CONTEXT import org.apache.spark.memory.{MemoryMode, TaskMemoryManager} import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.resource.ResourceInformation import org.apache.spark.util._ /** @@ -73,12 +75,14 @@ private[spark] abstract class Task[T]( * * @param taskAttemptId an identifier for this task attempt that is unique within a SparkContext. * @param attemptNumber how many times this task has been attempted (0 for the first attempt) + * @param resources other host resources (like gpus) that this task attempt can access * @return the result of the task along with updates of Accumulators. */ final def run( taskAttemptId: Long, attemptNumber: Int, - metricsSystem: MetricsSystem): T = { + metricsSystem: MetricsSystem, + resources: Map[String, ResourceInformation]): T = { SparkEnv.get.blockManager.registerTask(taskAttemptId) // TODO SPARK-24874 Allow create BarrierTaskContext based on partitions, instead of whether // the stage is barrier. @@ -91,7 +95,8 @@ private[spark] abstract class Task[T]( taskMemoryManager, localProperties, metricsSystem, - metrics) + metrics, + resources) context = if (isBarrier) { new BarrierTaskContext(taskContext) @@ -154,6 +159,7 @@ private[spark] abstract class Task[T]( // Though we unset the ThreadLocal here, the context member variable itself is still // queried directly in the TaskRunner to check for FetchFailedExceptions. TaskContext.unset() + InputFileBlockHolder.unset() } } } @@ -182,7 +188,7 @@ private[spark] abstract class Task[T]( // context is not yet initialized when kill() is invoked. @volatile @transient private var _reasonIfKilled: String = null - protected var _executorDeserializeTime: Long = 0 + protected var _executorDeserializeTimeNs: Long = 0 protected var _executorDeserializeCpuTime: Long = 0 /** @@ -193,7 +199,7 @@ private[spark] abstract class Task[T]( /** * Returns the amount of time spent deserializing the RDD and function to be run. */ - def executorDeserializeTime: Long = _executorDeserializeTime + def executorDeserializeTimeNs: Long = _executorDeserializeTimeNs def executorDeserializeCpuTime: Long = _executorDeserializeCpuTime /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala index bb4a4442b9433..247cfe721b553 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskDescription.scala @@ -23,8 +23,10 @@ import java.nio.charset.StandardCharsets import java.util.Properties import scala.collection.JavaConverters._ -import scala.collection.mutable.{HashMap, Map} +import scala.collection.immutable +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} +import org.apache.spark.resource.ResourceInformation import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} /** @@ -54,6 +56,7 @@ private[spark] class TaskDescription( val addedFiles: Map[String, Long], val addedJars: Map[String, Long], val properties: Properties, + val resources: immutable.Map[String, ResourceInformation], val serializedTask: ByteBuffer) { override def toString: String = "TaskDescription(TID=%d, index=%d)".format(taskId, index) @@ -62,12 +65,23 @@ private[spark] class TaskDescription( private[spark] object TaskDescription { private def serializeStringLongMap(map: Map[String, Long], dataOut: DataOutputStream): Unit = { dataOut.writeInt(map.size) - for ((key, value) <- map) { + map.foreach { case (key, value) => dataOut.writeUTF(key) dataOut.writeLong(value) } } + private def serializeResources(map: immutable.Map[String, ResourceInformation], + dataOut: DataOutputStream): Unit = { + dataOut.writeInt(map.size) + map.foreach { case (key, value) => + dataOut.writeUTF(key) + dataOut.writeUTF(value.name) + dataOut.writeInt(value.addresses.size) + value.addresses.foreach(dataOut.writeUTF(_)) + } + } + def encode(taskDescription: TaskDescription): ByteBuffer = { val bytesOut = new ByteBufferOutputStream(4096) val dataOut = new DataOutputStream(bytesOut) @@ -95,6 +109,9 @@ private[spark] object TaskDescription { dataOut.write(bytes) } + // Write resources. + serializeResources(taskDescription.resources, dataOut) + // Write the task. The task is already serialized, so write it directly to the byte buffer. Utils.writeByteBuffer(taskDescription.serializedTask, bytesOut) @@ -106,12 +123,35 @@ private[spark] object TaskDescription { private def deserializeStringLongMap(dataIn: DataInputStream): HashMap[String, Long] = { val map = new HashMap[String, Long]() val mapSize = dataIn.readInt() - for (i <- 0 until mapSize) { + var i = 0 + while (i < mapSize) { map(dataIn.readUTF()) = dataIn.readLong() + i += 1 } map } + private def deserializeResources(dataIn: DataInputStream): + immutable.Map[String, ResourceInformation] = { + val map = new HashMap[String, ResourceInformation]() + val mapSize = dataIn.readInt() + var i = 0 + while (i < mapSize) { + val resType = dataIn.readUTF() + val name = dataIn.readUTF() + val numIdentifier = dataIn.readInt() + val identifiers = new ArrayBuffer[String](numIdentifier) + var j = 0 + while (j < numIdentifier) { + identifiers += dataIn.readUTF() + j += 1 + } + map(resType) = new ResourceInformation(name, identifiers.toArray) + i += 1 + } + map.toMap + } + def decode(byteBuffer: ByteBuffer): TaskDescription = { val dataIn = new DataInputStream(new ByteBufferInputStream(byteBuffer)) val taskId = dataIn.readLong() @@ -138,10 +178,13 @@ private[spark] object TaskDescription { properties.setProperty(key, new String(valueBytes, StandardCharsets.UTF_8)) } + // Read resources. + val resources = deserializeResources(dataIn) + // Create a sub-buffer for the serialized task into its own buffer (to be deserialized later). val serializedTask = byteBuffer.slice() new TaskDescription(taskId, attemptNumber, executorId, name, index, partitionId, taskFiles, - taskJars, properties, serializedTask) + taskJars, properties, resources, serializedTask) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index a284f7956cd31..9b7f901c55e00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -80,7 +80,7 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul logDebug("Fetching indirect task result for TID %s".format(tid)) scheduler.handleTaskGettingResult(taskSetManager, tid) val serializedTaskResult = sparkEnv.blockManager.getRemoteBytes(blockId) - if (!serializedTaskResult.isDefined) { + if (serializedTaskResult.isEmpty) { /* We won't be able to get the task result if the machine that ran the task failed * between when the task ended and when we tried to fetch the result, or if the * block manager had to flush the result. */ @@ -128,27 +128,25 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul serializedData: ByteBuffer) { var reason : TaskFailedReason = UnknownReason try { - getTaskResultExecutor.execute(new Runnable { - override def run(): Unit = Utils.logUncaughtExceptions { - val loader = Utils.getContextOrSparkClassLoader - try { - if (serializedData != null && serializedData.limit() > 0) { - reason = serializer.get().deserialize[TaskFailedReason]( - serializedData, loader) - } - } catch { - case cnd: ClassNotFoundException => - // Log an error but keep going here -- the task failed, so not catastrophic - // if we can't deserialize the reason. - logError( - "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) - case ex: Exception => // No-op - } finally { - // If there's an error while deserializing the TaskEndReason, this Runnable - // will die. Still tell the scheduler about the task failure, to avoid a hang - // where the scheduler thinks the task is still running. - scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) + getTaskResultExecutor.execute(() => Utils.logUncaughtExceptions { + val loader = Utils.getContextOrSparkClassLoader + try { + if (serializedData != null && serializedData.limit() > 0) { + reason = serializer.get().deserialize[TaskFailedReason]( + serializedData, loader) } + } catch { + case _: ClassNotFoundException => + // Log an error but keep going here -- the task failed, so not catastrophic + // if we can't deserialize the reason. + logError( + "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) + case _: Exception => // No-op + } finally { + // If there's an error while deserializing the TaskEndReason, this Runnable + // will die. Still tell the scheduler about the task failure, to avoid a hang + // where the scheduler thinks the task is still running. + scheduler.handleFailedTask(taskSetManager, tid, taskState, reason) } }) } catch { @@ -157,6 +155,15 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul } } + // This method calls `TaskSchedulerImpl.handlePartitionCompleted` asynchronously. We do not want + // DAGScheduler to call `TaskSchedulerImpl.handlePartitionCompleted` directly, as it's + // synchronized and may hurt the throughput of the scheduler. + def enqueuePartitionCompletionNotification(stageId: Int, partitionId: Int): Unit = { + getTaskResultExecutor.execute(() => Utils.logUncaughtExceptions { + scheduler.handlePartitionCompleted(stageId, partitionId) + }) + } + def stop() { getTaskResultExecutor.shutdownNow() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 94221eb0d5515..bfdbf0217210a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -68,6 +68,10 @@ private[spark] trait TaskScheduler { // Throw UnsupportedOperationException if the backend doesn't support kill tasks. def killAllTaskAttempts(stageId: Int, interruptThread: Boolean, reason: String): Unit + // Notify the corresponding `TaskSetManager`s of the stage, that a partition has already completed + // and they can skip running tasks for it. + def notifyPartitionCompletion(stageId: Int, partitionId: Int) + // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 4f870e85ad38d..242486cc9bee8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -22,20 +22,20 @@ import java.util.{Locale, Timer, TimerTask} import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import java.util.concurrent.atomic.AtomicLong -import scala.collection.Set -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{ArrayBuffer, Buffer, HashMap, HashSet} import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.ExecutorMetrics -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils import org.apache.spark.rpc.RpcEndpoint import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.scheduler.TaskLocality.TaskLocality import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} +import org.apache.spark.util.{AccumulatorV2, SystemClock, ThreadUtils, Utils} /** * Schedules tasks for multiple types of clusters by acting through a SchedulerBackend. @@ -50,7 +50,12 @@ import org.apache.spark.util.{AccumulatorV2, ThreadUtils, Utils} * threads, so it needs locks in public API methods to maintain its state. In addition, some * [[SchedulerBackend]]s synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while - * we are holding a lock on ourselves. + * we are holding a lock on ourselves. This class is called from many threads, notably: + * * The DAGScheduler Event Loop + * * The RPCHandler threads, responding to status updates from Executors + * * Periodic revival of all offers from the CoarseGrainedSchedulerBackend, to accommodate delay + * scheduling + * * task-result-getter threads */ private[spark] class TaskSchedulerImpl( val sc: SparkContext, @@ -61,7 +66,7 @@ private[spark] class TaskSchedulerImpl( import TaskSchedulerImpl._ def this(sc: SparkContext) = { - this(sc, sc.conf.get(config.MAX_TASK_FAILURES)) + this(sc, sc.conf.get(config.TASK_MAX_FAILURES)) } // Lazily initializing blacklistTrackerOpt to avoid getting empty ExecutorAllocationClient, @@ -71,7 +76,7 @@ private[spark] class TaskSchedulerImpl( val conf = sc.conf // How often to check for speculative tasks - val SPECULATION_INTERVAL_MS = conf.getTimeAsMs("spark.speculation.interval", "100ms") + val SPECULATION_INTERVAL_MS = conf.get(SPECULATION_INTERVAL) // Duplicate copies of a task will only be launched if the original copy has been running for // at least this amount of time. This is to avoid the overhead of launching speculative copies @@ -85,14 +90,18 @@ private[spark] class TaskSchedulerImpl( val STARVATION_TIMEOUT_MS = conf.getTimeAsMs("spark.starvation.timeout", "15s") // CPUs to request per task - val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) + val CPUS_PER_TASK = conf.get(config.CPUS_PER_TASK) + + // Resources to request per task + val resourcesReqsPerTask = ResourceUtils.parseTaskResourceRequirements(sc.conf) // TaskSetManagers are not thread safe, so any access to one should be synchronized - // on this class. + // on this class. Protected by `this` private val taskSetsByStageIdAndAttempt = new HashMap[Int, HashMap[Int, TaskSetManager]] // Protected by `this` private[scheduler] val taskIdToTaskSetManager = new ConcurrentHashMap[Long, TaskSetManager] + // Protected by `this` val taskIdToExecutorId = new HashMap[Long, String] @volatile private var hasReceivedTask = false @@ -117,6 +126,11 @@ private[spark] class TaskSchedulerImpl( protected val executorIdToHost = new HashMap[String, String] + private val abortTimer = new Timer(true) + private val clock = new SystemClock + // Exposed for testing + val unschedulableTaskSetToExpiryTime = new HashMap[TaskSetManager, Long] + // Listener object to pass upcalls into var dagScheduler: DAGScheduler = null @@ -126,7 +140,7 @@ private[spark] class TaskSchedulerImpl( private var schedulableBuilder: SchedulableBuilder = null // default scheduler is FIFO - private val schedulingModeConf = conf.get(SCHEDULER_MODE_PROPERTY, SchedulingMode.FIFO.toString) + private val schedulingModeConf = conf.get(SCHEDULER_MODE) val schedulingMode: SchedulingMode = try { SchedulingMode.withName(schedulingModeConf.toUpperCase(Locale.ROOT)) @@ -144,6 +158,8 @@ private[spark] class TaskSchedulerImpl( private[scheduler] var barrierCoordinator: RpcEndpoint = null + protected val defaultRackValue: Option[String] = None + private def maybeInitBarrierCoordinator(): Unit = { if (barrierCoordinator == null) { barrierCoordinator = new BarrierCoordinator(barrierSyncTimeout, sc.listenerBus, @@ -178,13 +194,11 @@ private[spark] class TaskSchedulerImpl( override def start() { backend.start() - if (!isLocal && conf.getBoolean("spark.speculation", false)) { + if (!isLocal && conf.get(SPECULATION_ENABLED)) { logInfo("Starting speculative execution thread") - speculationScheduler.scheduleWithFixedDelay(new Runnable { - override def run(): Unit = Utils.tryOrStopSparkContext(sc) { - checkSpeculatableTasks() - } - }, SPECULATION_INTERVAL_MS, SPECULATION_INTERVAL_MS, TimeUnit.MILLISECONDS) + speculationScheduler.scheduleWithFixedDelay( + () => Utils.tryOrStopSparkContext(sc) { checkSpeculatableTasks() }, + SPECULATION_INTERVAL_MS, SPECULATION_INTERVAL_MS, TimeUnit.MILLISECONDS) } } @@ -200,14 +214,20 @@ private[spark] class TaskSchedulerImpl( val stage = taskSet.stageId val stageTaskSets = taskSetsByStageIdAndAttempt.getOrElseUpdate(stage, new HashMap[Int, TaskSetManager]) - stageTaskSets(taskSet.stageAttemptId) = manager - val conflictingTaskSet = stageTaskSets.exists { case (_, ts) => - ts.taskSet != taskSet && !ts.isZombie - } - if (conflictingTaskSet) { - throw new IllegalStateException(s"more than one active taskSet for stage $stage:" + - s" ${stageTaskSets.toSeq.map{_._2.taskSet.id}.mkString(",")}") + + // Mark all the existing TaskSetManagers of this stage as zombie, as we are adding a new one. + // This is necessary to handle a corner case. Let's say a stage has 10 partitions and has 2 + // TaskSetManagers: TSM1(zombie) and TSM2(active). TSM1 has a running task for partition 10 + // and it completes. TSM2 finishes tasks for partition 1-9, and thinks he is still active + // because partition 10 is not completed yet. However, DAGScheduler gets task completion + // events for all the 10 partitions and thinks the stage is finished. If it's a shuffle stage + // and somehow it has missing map outputs, then DAGScheduler will resubmit it and create a + // TSM3 for it. As a stage can't have more than one active task set managers, we must mark + // TSM2 as zombie (it actually is). + stageTaskSets.foreach { case (_, ts) => + ts.isZombie = true } + stageTaskSets(taskSet.stageAttemptId) = manager schedulableBuilder.addTaskSetManager(manager, manager.taskSet.properties) if (!isLocal && !hasReceivedTask) { @@ -248,7 +268,10 @@ private[spark] class TaskSchedulerImpl( } } - override def killTaskAttempt(taskId: Long, interruptThread: Boolean, reason: String): Boolean = { + override def killTaskAttempt( + taskId: Long, + interruptThread: Boolean, + reason: String): Boolean = synchronized { logInfo(s"Killing task $taskId: $reason") val execId = taskIdToExecutorId.get(taskId) if (execId.isDefined) { @@ -281,6 +304,10 @@ private[spark] class TaskSchedulerImpl( } } + override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = { + taskResultGetter.enqueuePartitionCompletionNotification(stageId, partitionId) + } + /** * Called to indicate that all task attempts (including speculated tasks) associated with the * given TaskSetManager have completed, so state associated with the TaskSetManager should be @@ -303,6 +330,7 @@ private[spark] class TaskSchedulerImpl( maxLocality: TaskLocality, shuffledOffers: Seq[WorkerOffer], availableCpus: Array[Int], + availableResources: Array[Map[String, Buffer[String]]], tasks: IndexedSeq[ArrayBuffer[TaskDescription]], addressesWithDescs: ArrayBuffer[(String, TaskDescription)]) : Boolean = { var launchedTask = false @@ -311,9 +339,10 @@ private[spark] class TaskSchedulerImpl( for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host - if (availableCpus(i) >= CPUS_PER_TASK) { + if (availableCpus(i) >= CPUS_PER_TASK && + resourcesMeetTaskRequirements(availableResources(i))) { try { - for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { + for (task <- taskSet.resourceOffer(execId, host, maxLocality, availableResources(i))) { tasks(i) += task val tid = task.taskId taskIdToTaskSetManager.put(tid, taskSet) @@ -321,6 +350,15 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(execId).add(tid) availableCpus(i) -= CPUS_PER_TASK assert(availableCpus(i) >= 0) + task.resources.foreach { case (rName, rInfo) => + // Remove the first n elements from availableResources addresses, these removed + // addresses are the same as that we allocated in taskSet.resourceOffer() since it's + // synchronized. We don't remove the exact addresses allocated because the current + // approach produces the identical result with less time complexity. + availableResources(i).getOrElse(rName, + throw new SparkException(s"Try to acquire resource $rName that doesn't exist.")) + .remove(0, rInfo.addresses.size) + } // Only update hosts for a barrier task. if (taskSet.isBarrier) { // The executor address is expected to be non empty. @@ -337,7 +375,16 @@ private[spark] class TaskSchedulerImpl( } } } - return launchedTask + launchedTask + } + + /** + * Check whether the resources from the WorkerOffer are enough to run at least one task. + */ + private def resourcesMeetTaskRequirements(resources: Map[String, Buffer[String]]): Boolean = { + resourcesReqsPerTask.forall { req => + resources.contains(req.resourceName) && resources(req.resourceName).size >= req.amount + } } /** @@ -360,9 +407,10 @@ private[spark] class TaskSchedulerImpl( executorIdToRunningTaskIds(o.executorId) = HashSet[Long]() newExecAvail = true } - for (rack <- getRackForHost(o.host)) { - hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += o.host - } + } + val hosts = offers.map(_.host).toSet.toSeq + for ((host, Some(rack)) <- hosts.zip(getRacksForHosts(hosts))) { + hostsByRack.getOrElseUpdate(rack, new HashSet[String]()) += host } // Before making any offers, remove any nodes from the blacklist whose blacklist has expired. Do @@ -380,6 +428,7 @@ private[spark] class TaskSchedulerImpl( val shuffledOffers = shuffleOffers(filteredOffers) // Build a list of tasks to assign to each worker. val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores / CPUS_PER_TASK)) + val availableResources = shuffledOffers.map(_.resources).toArray val availableCpus = shuffledOffers.map(o => o.cores).toArray val availableSlots = shuffledOffers.map(o => o.cores / CPUS_PER_TASK).sum val sortedTaskSets = rootPool.getSortedTaskSetQueue @@ -411,13 +460,58 @@ private[spark] class TaskSchedulerImpl( var launchedTaskAtCurrentMaxLocality = false do { launchedTaskAtCurrentMaxLocality = resourceOfferSingleTaskSet(taskSet, - currentMaxLocality, shuffledOffers, availableCpus, tasks, addressesWithDescs) + currentMaxLocality, shuffledOffers, availableCpus, + availableResources, tasks, addressesWithDescs) launchedAnyTask |= launchedTaskAtCurrentMaxLocality } while (launchedTaskAtCurrentMaxLocality) } + if (!launchedAnyTask) { - taskSet.abortIfCompletelyBlacklisted(hostToExecutors) + taskSet.getCompletelyBlacklistedTaskIfAny(hostToExecutors).foreach { taskIndex => + // If the taskSet is unschedulable we try to find an existing idle blacklisted + // executor. If we cannot find one, we abort immediately. Else we kill the idle + // executor and kick off an abortTimer which if it doesn't schedule a task within the + // the timeout will abort the taskSet if we were unable to schedule any task from the + // taskSet. + // Note 1: We keep track of schedulability on a per taskSet basis rather than on a per + // task basis. + // Note 2: The taskSet can still be aborted when there are more than one idle + // blacklisted executors and dynamic allocation is on. This can happen when a killed + // idle executor isn't replaced in time by ExecutorAllocationManager as it relies on + // pending tasks and doesn't kill executors on idle timeouts, resulting in the abort + // timer to expire and abort the taskSet. + executorIdToRunningTaskIds.find(x => !isExecutorBusy(x._1)) match { + case Some ((executorId, _)) => + if (!unschedulableTaskSetToExpiryTime.contains(taskSet)) { + blacklistTrackerOpt.foreach(blt => blt.killBlacklistedIdleExecutor(executorId)) + + val timeout = conf.get(config.UNSCHEDULABLE_TASKSET_TIMEOUT) * 1000 + unschedulableTaskSetToExpiryTime(taskSet) = clock.getTimeMillis() + timeout + logInfo(s"Waiting for $timeout ms for completely " + + s"blacklisted task to be schedulable again before aborting $taskSet.") + abortTimer.schedule( + createUnschedulableTaskSetAbortTimer(taskSet, taskIndex), timeout) + } + case None => // Abort Immediately + logInfo("Cannot schedule any task because of complete blacklisting. No idle" + + s" executors can be found to kill. Aborting $taskSet." ) + taskSet.abortSinceCompletelyBlacklisted(taskIndex) + } + } + } else { + // We want to defer killing any taskSets as long as we have a non blacklisted executor + // which can be used to schedule a task from any active taskSets. This ensures that the + // job can make progress. + // Note: It is theoretically possible that a taskSet never gets scheduled on a + // non-blacklisted executor and the abort timer doesn't kick in because of a constant + // submission of new TaskSets. See the PR for more details. + if (unschedulableTaskSetToExpiryTime.nonEmpty) { + logInfo("Clearing the expiry times for all unschedulable taskSets as a task was " + + "recently scheduled.") + unschedulableTaskSetToExpiryTime.clear() + } } + if (launchedAnyTask && taskSet.isBarrier) { // Check whether the barrier tasks are partially launched. // TODO SPARK-24818 handle the assert failure case (that can happen when some locality @@ -447,12 +541,29 @@ private[spark] class TaskSchedulerImpl( // TODO SPARK-24823 Cancel a job that contains barrier stage(s) if the barrier tasks don't get // launched within a configured time. - if (tasks.size > 0) { + if (tasks.nonEmpty) { hasLaunchedTask = true } return tasks } + private def createUnschedulableTaskSetAbortTimer( + taskSet: TaskSetManager, + taskIndex: Int): TimerTask = { + new TimerTask() { + override def run() { + if (unschedulableTaskSetToExpiryTime.contains(taskSet) && + unschedulableTaskSetToExpiryTime(taskSet) <= clock.getTimeMillis()) { + logInfo("Cannot schedule any task because of complete blacklisting. " + + s"Wait time for scheduling expired. Aborting $taskSet.") + taskSet.abortSinceCompletelyBlacklisted(taskIndex) + } else { + this.cancel() + } + } + } + } + /** * Shuffle offers around to avoid always placing tasks on the same workers. Exposed to allow * overriding in tests, so it can be deterministic. @@ -555,6 +666,21 @@ private[spark] class TaskSchedulerImpl( } } + /** + * Marks the task has completed in the active TaskSetManager for the given stage. + * + * After stage failure and retry, there may be multiple TaskSetManagers for the stage. + * If an earlier zombie attempt of a stage completes a task, we can ask the later active attempt + * to skip submitting and running the task for the same partition, to save resource. That also + * means that a task completion from an earlier zombie attempt can lead to the entire stage + * getting marked as successful. + */ + private[scheduler] def handlePartitionCompleted(stageId: Int, partitionId: Int) = synchronized { + taskSetsByStageIdAndAttempt.get(stageId).foreach(_.values.filter(!_.isZombie).foreach { tsm => + tsm.markPartitionCompleted(partitionId) + }) + } + def error(message: String) { synchronized { if (taskSetsByStageIdAndAttempt.nonEmpty) { @@ -590,6 +716,7 @@ private[spark] class TaskSchedulerImpl( barrierCoordinator.stop() } starvationTimer.cancel() + abortTimer.cancel() } override def defaultParallelism(): Int = backend.defaultParallelism() @@ -730,12 +857,29 @@ private[spark] class TaskSchedulerImpl( * Get a snapshot of the currently blacklisted nodes for the entire application. This is * thread-safe -- it can be called without a lock on the TaskScheduler. */ - def nodeBlacklist(): scala.collection.immutable.Set[String] = { - blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(scala.collection.immutable.Set()) + def nodeBlacklist(): Set[String] = { + blacklistTrackerOpt.map(_.nodeBlacklist()).getOrElse(Set.empty) } - // By default, rack is unknown - def getRackForHost(value: String): Option[String] = None + /** + * Get the rack for one host. + * + * Note that [[getRacksForHosts]] should be preferred when possible as that can be much + * more efficient. + */ + def getRackForHost(host: String): Option[String] = { + getRacksForHosts(Seq(host)).head + } + + /** + * Get racks for multiple hosts. + * + * The returned Sequence will be the same length as the hosts argument and can be zipped + * together with the hosts argument. + */ + def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { + hosts.map(_ => defaultRackValue) + } private def waitBackendReady(): Unit = { if (backend.isReady) { @@ -757,9 +901,10 @@ private[spark] class TaskSchedulerImpl( override def applicationAttemptId(): Option[String] = backend.applicationAttemptId() + // exposed for testing private[scheduler] def taskSetManagerForAttempt( stageId: Int, - stageAttemptId: Int): Option[TaskSetManager] = { + stageAttemptId: Int): Option[TaskSetManager] = synchronized { for { attempts <- taskSetsByStageIdAndAttempt.get(stageId) manager <- attempts.get(stageAttemptId) @@ -767,30 +912,12 @@ private[spark] class TaskSchedulerImpl( manager } } - - /** - * Marks the task has completed in all TaskSetManagers for the given stage. - * - * After stage failure and retry, there may be multiple TaskSetManagers for the stage. - * If an earlier attempt of a stage completes a task, we should ensure that the later attempts - * do not also submit those same tasks. That also means that a task completion from an earlier - * attempt can lead to the entire stage getting marked as successful. - */ - private[scheduler] def markPartitionCompletedInAllTaskSets( - stageId: Int, - partitionId: Int, - taskInfo: TaskInfo) = { - taskSetsByStageIdAndAttempt.getOrElse(stageId, Map()).values.foreach { tsm => - tsm.markPartitionCompleted(partitionId, taskInfo) - } - } - } private[spark] object TaskSchedulerImpl { - val SCHEDULER_MODE_PROPERTY = "spark.scheduler.mode" + val SCHEDULER_MODE_PROPERTY = SCHEDULER_MODE.key /** * Used to balance containers across hosts. diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index d5e85a11cb279..e7645fc19b9f6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -21,6 +21,7 @@ import java.io.NotSerializableException import java.nio.ByteBuffer import java.util.concurrent.ConcurrentLinkedQueue +import scala.collection.immutable.Map import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.math.max import scala.util.control.NonFatal @@ -28,6 +29,8 @@ import scala.util.control.NonFatal import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.SchedulingMode._ import org.apache.spark.util.{AccumulatorV2, Clock, LongAccumulator, SystemClock, Utils} import org.apache.spark.util.collection.MedianHeap @@ -37,7 +40,8 @@ import org.apache.spark.util.collection.MedianHeap * each task, retries tasks if they fail (up to a limited number of times), and * handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces * to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, - * and statusUpdate, which tells it that one of its tasks changed state (e.g. finished). + * and handleSuccessfulTask/handleFailedTask, which tells it that one of its tasks changed state + * (e.g. finished/failed). * * THREADING: This class is designed to only be called from code with a lock on the * TaskScheduler (e.g. its event handlers). It should not be called from other threads. @@ -60,14 +64,8 @@ private[spark] class TaskSetManager( private val addedJars = HashMap[String, Long](sched.sc.addedJars.toSeq: _*) private val addedFiles = HashMap[String, Long](sched.sc.addedFiles.toSeq: _*) - // Quantile of tasks at which to start speculation - val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) - val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5) - val maxResultSize = conf.get(config.MAX_RESULT_SIZE) - val speculationEnabled = conf.getBoolean("spark.speculation", false) - // Serializer for closures and tasks. val env = SparkEnv.get val ser = env.closureSerializer.newInstance() @@ -78,6 +76,12 @@ private[spark] class TaskSetManager( val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) + val speculationEnabled = conf.get(SPECULATION_ENABLED) + // Quantile of tasks at which to start speculation + val speculationQuantile = conf.get(SPECULATION_QUANTILE) + val speculationMultiplier = conf.get(SPECULATION_MULTIPLIER) + val minFinishedForSpeculation = math.max((speculationQuantile * numTasks).floor.toInt, 1) + // For each task, tracks whether a copy of the task has succeeded. A task will also be // marked as "succeeded" if it failed with a fetch failure, in which case it should not // be re-run because the missing map data needs to be regenerated first. @@ -184,8 +188,24 @@ private[spark] class TaskSetManager( // Add all our tasks to the pending lists. We do this in reverse order // of task index so that tasks with low indices get launched first. - for (i <- (0 until numTasks).reverse) { - addPendingTask(i) + addPendingTasks() + + private def addPendingTasks(): Unit = { + val (_, duration) = Utils.timeTakenMs { + for (i <- (0 until numTasks).reverse) { + addPendingTask(i, resolveRacks = false) + } + // Resolve the rack for each host. This can be slow, so de-dupe the list of hosts, + // and assign the rack to all relevant task indices. + val (hosts, indicesForHosts) = pendingTasksForHost.toSeq.unzip + val racks = sched.getRacksForHosts(hosts) + racks.zip(indicesForHosts).foreach { + case (Some(rack), indices) => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) ++= indices + case (None, _) => // no rack, nothing to do + } + } + logDebug(s"Adding pending tasks took $duration ms") } /** @@ -212,7 +232,9 @@ private[spark] class TaskSetManager( private[scheduler] var emittedTaskSizeWarning = false /** Add a task to all the pending-task lists that it should be on. */ - private[spark] def addPendingTask(index: Int) { + private[spark] def addPendingTask( + index: Int, + resolveRacks: Boolean = true): Unit = { for (loc <- tasks(index).preferredLocations) { loc match { case e: ExecutorCacheTaskLocation => @@ -232,8 +254,11 @@ private[spark] class TaskSetManager( case _ => } pendingTasksForHost.getOrElseUpdate(loc.host, new ArrayBuffer) += index - for (rack <- sched.getRackForHost(loc.host)) { - pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + + if (resolveRacks) { + sched.getRackForHost(loc.host).foreach { rack => + pendingTasksForRack.getOrElseUpdate(rack, new ArrayBuffer) += index + } } } @@ -329,7 +354,7 @@ private[spark] class TaskSetManager( val executors = prefs.flatMap(_ match { case e: ExecutorCacheTaskLocation => Some(e.executorId) case _ => None - }); + }) if (executors.contains(execId)) { speculatableTasks -= index return Some((index, TaskLocality.PROCESS_LOCAL)) @@ -444,7 +469,8 @@ private[spark] class TaskSetManager( def resourceOffer( execId: String, host: String, - maxLocality: TaskLocality.TaskLocality) + maxLocality: TaskLocality.TaskLocality, + availableResources: Map[String, Seq[String]] = Map.empty) : Option[TaskDescription] = { val offerBlacklisted = taskSetBlacklistHelperOpt.exists { blacklist => @@ -493,12 +519,12 @@ private[spark] class TaskSetManager( abort(s"$msg Exception during serialization: $e") throw new TaskNotSerializableException(e) } - if (serializedTask.limit() > TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024 && + if (serializedTask.limit() > TaskSetManager.TASK_SIZE_TO_WARN_KIB * 1024 && !emittedTaskSizeWarning) { emittedTaskSizeWarning = true logWarning(s"Stage ${task.stageId} contains a task of very large size " + - s"(${serializedTask.limit() / 1024} KB). The maximum recommended task size is " + - s"${TaskSetManager.TASK_SIZE_TO_WARN_KB} KB.") + s"(${serializedTask.limit() / 1024} KiB). The maximum recommended task size is " + + s"${TaskSetManager.TASK_SIZE_TO_WARN_KIB} KiB.") } addRunningTask(taskId) @@ -509,6 +535,17 @@ private[spark] class TaskSetManager( logInfo(s"Starting $taskName (TID $taskId, $host, executor ${info.executorId}, " + s"partition ${task.partitionId}, $taskLocality, ${serializedTask.limit()} bytes)") + val extraResources = sched.resourcesReqsPerTask.map { taskReq => + val rName = taskReq.resourceName + val count = taskReq.amount + val rAddresses = availableResources.getOrElse(rName, Seq.empty) + assert(rAddresses.size >= count, s"Required $count $rName addresses, but only " + + s"${rAddresses.size} available.") + // We'll drop the allocated addresses later inside TaskSchedulerImpl. + val allocatedAddresses = rAddresses.take(count) + (rName, new ResourceInformation(rName, allocatedAddresses.toArray)) + }.toMap + sched.dagScheduler.taskStarted(task, info) new TaskDescription( taskId, @@ -520,6 +557,7 @@ private[spark] class TaskSetManager( addedFiles, addedJars, task.localProperties, + extraResources, serializedTask) } } else { @@ -623,8 +661,8 @@ private[spark] class TaskSetManager( * * It is possible that this taskset has become impossible to schedule *anywhere* due to the * blacklist. The most common scenario would be if there are fewer executors than - * spark.task.maxFailures. We need to detect this so we can fail the task set, otherwise the job - * will hang. + * spark.task.maxFailures. We need to detect this so we can avoid the job from being hung. + * We try to acquire new executor/s by killing an existing idle blacklisted executor. * * There's a tradeoff here: we could make sure all tasks in the task set are schedulable, but that * would add extra time to each iteration of the scheduling loop. Here, we take the approach of @@ -635,9 +673,9 @@ private[spark] class TaskSetManager( * failures (this is because the method picks one unscheduled task, and then iterates through each * executor until it finds one that the task isn't blacklisted on). */ - private[scheduler] def abortIfCompletelyBlacklisted( - hostToExecutors: HashMap[String, HashSet[String]]): Unit = { - taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + private[scheduler] def getCompletelyBlacklistedTaskIfAny( + hostToExecutors: HashMap[String, HashSet[String]]): Option[Int] = { + taskSetBlacklistHelperOpt.flatMap { taskSetBlacklist => val appBlacklist = blacklistTracker.get // Only look for unschedulable tasks when at least one executor has registered. Otherwise, // task sets will be (unnecessarily) aborted in cases when no executors have registered yet. @@ -658,11 +696,11 @@ private[spark] class TaskSetManager( } } - pendingTask.foreach { indexInTaskSet => + pendingTask.find { indexInTaskSet => // try to find some executor this task can run on. Its possible that some *other* // task isn't schedulable anywhere, but we will discover that in some later call, // when that unschedulable task is the last task remaining. - val blacklistedEverywhere = hostToExecutors.forall { case (host, execsOnHost) => + hostToExecutors.forall { case (host, execsOnHost) => // Check if the task can run on the node val nodeBlacklisted = appBlacklist.isNodeBlacklisted(host) || @@ -679,22 +717,27 @@ private[spark] class TaskSetManager( } } } - if (blacklistedEverywhere) { - val partition = tasks(indexInTaskSet).partitionId - abort(s""" - |Aborting $taskSet because task $indexInTaskSet (partition $partition) - |cannot run anywhere due to node and executor blacklist. - |Most recent failure: - |${taskSetBlacklist.getLatestFailureReason} - | - |Blacklisting behavior can be configured via spark.blacklist.*. - |""".stripMargin) - } } + } else { + None } } } + private[scheduler] def abortSinceCompletelyBlacklisted(indexInTaskSet: Int): Unit = { + taskSetBlacklistHelperOpt.foreach { taskSetBlacklist => + val partition = tasks(indexInTaskSet).partitionId + abort(s""" + |Aborting $taskSet because task $indexInTaskSet (partition $partition) + |cannot run anywhere due to node and executor blacklist. + |Most recent failure: + |${taskSetBlacklist.getLatestFailureReason} + | + |Blacklisting behavior can be configured via spark.blacklist.*. + |""".stripMargin) + } + } + /** * Marks the task as getting result and notifies the DAG Scheduler */ @@ -712,7 +755,7 @@ private[spark] class TaskSetManager( calculatedTasks += 1 if (maxResultSize > 0 && totalResultSize > maxResultSize) { val msg = s"Total size of serialized results of ${calculatedTasks} tasks " + - s"(${Utils.bytesToString(totalResultSize)}) is bigger than spark.driver.maxResultSize " + + s"(${Utils.bytesToString(totalResultSize)}) is bigger than ${config.MAX_RESULT_SIZE.key} " + s"(${Utils.bytesToString(maxResultSize)})" logError(msg) abort(msg) @@ -778,9 +821,6 @@ private[spark] class TaskSetManager( logInfo("Ignoring task-finished event for " + info.id + " in stage " + taskSet.id + " because task " + index + " has already completed successfully") } - // There may be multiple tasksets for this stage -- we let all of them know that the partition - // was completed. This may result in some of the tasksets getting completed. - sched.markPartitionCompletedInAllTaskSets(stageId, tasks(index).partitionId, info) // This method is called by "TaskSchedulerImpl.handleSuccessfulTask" which holds the // "TaskSchedulerImpl" lock until exiting. To avoid the SPARK-7655 issue, we should not // "deserialize" the value when holding a lock to avoid blocking other threads. So we call @@ -791,12 +831,9 @@ private[spark] class TaskSetManager( maybeFinishTaskSet() } - private[scheduler] def markPartitionCompleted(partitionId: Int, taskInfo: TaskInfo): Unit = { + private[scheduler] def markPartitionCompleted(partitionId: Int): Unit = { partitionToIndex.get(partitionId).foreach { index => if (!successful(index)) { - if (speculationEnabled && !isZombie) { - successfulTaskDurations.insert(taskInfo.duration) - } tasksSuccessful += 1 successful(index) = true if (tasksSuccessful == numTasks) { @@ -1010,13 +1047,16 @@ private[spark] class TaskSetManager( return false } var foundTasks = false - val minFinishedForSpeculation = (SPECULATION_QUANTILE * numTasks).floor.toInt logDebug("Checking for speculative tasks: minFinished = " + minFinishedForSpeculation) - if (tasksSuccessful >= minFinishedForSpeculation && tasksSuccessful > 0) { + // It's possible that a task is marked as completed by the scheduler, then the size of + // `successfulTaskDurations` may not equal to `tasksSuccessful`. Here we should only count the + // tasks that are submitted by this `TaskSetManager` and are completed successfully. + val numSuccessfulTasks = successfulTaskDurations.size() + if (numSuccessfulTasks >= minFinishedForSpeculation) { val time = clock.getTimeMillis() val medianDuration = successfulTaskDurations.median - val threshold = max(SPECULATION_MULTIPLIER * medianDuration, minTimeToSpeculation) + val threshold = max(speculationMultiplier * medianDuration, minTimeToSpeculation) // TODO: Threshold should also look at standard deviation of task durations and have a lower // bound based on that. logDebug("Task length threshold for speculation: " + threshold) @@ -1038,16 +1078,15 @@ private[spark] class TaskSetManager( } private def getLocalityWait(level: TaskLocality.TaskLocality): Long = { - val defaultWait = conf.get(config.LOCALITY_WAIT) - val localityWaitKey = level match { - case TaskLocality.PROCESS_LOCAL => "spark.locality.wait.process" - case TaskLocality.NODE_LOCAL => "spark.locality.wait.node" - case TaskLocality.RACK_LOCAL => "spark.locality.wait.rack" + val localityWait = level match { + case TaskLocality.PROCESS_LOCAL => config.LOCALITY_WAIT_PROCESS + case TaskLocality.NODE_LOCAL => config.LOCALITY_WAIT_NODE + case TaskLocality.RACK_LOCAL => config.LOCALITY_WAIT_RACK case _ => null } - if (localityWaitKey != null) { - conf.getTimeAsMs(localityWaitKey, defaultWait.toString) + if (localityWait != null) { + conf.get(localityWait) } else { 0L } @@ -1096,5 +1135,5 @@ private[spark] class TaskSetManager( private[spark] object TaskSetManager { // The user will be warned if any stages contain a task that has a serialized size greater than // this. - val TASK_SIZE_TO_WARN_KB = 100 + val TASK_SIZE_TO_WARN_KIB = 1000 } diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index 6ec74913e42f2..522dbfa9457b9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import scala.collection.mutable.Buffer + /** * Represents free resources available on an executor. */ @@ -27,4 +29,5 @@ case class WorkerOffer( cores: Int, // `address` is an optional hostPort string, it provide more useful information than `host` // when multiple executors are launched on the same host. - address: Option[String] = None) + address: Option[String] = None, + resources: Map[String, Buffer[String]] = Map.empty) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index e8b7fc0ef100a..a90fff02ac73d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler.cluster import java.nio.ByteBuffer import org.apache.spark.TaskState.TaskState +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.ExecutorLossReason import org.apache.spark.util.SerializableBuffer @@ -63,17 +64,24 @@ private[spark] object CoarseGrainedClusterMessages { executorRef: RpcEndpointRef, hostname: String, cores: Int, - logUrls: Map[String, String]) + logUrls: Map[String, String], + attributes: Map[String, String], + resources: Map[String, ResourceInformation]) extends CoarseGrainedClusterMessage - case class StatusUpdate(executorId: String, taskId: Long, state: TaskState, - data: SerializableBuffer) extends CoarseGrainedClusterMessage + case class StatusUpdate( + executorId: String, + taskId: Long, + state: TaskState, + data: SerializableBuffer, + resources: Map[String, ResourceInformation] = Map.empty) + extends CoarseGrainedClusterMessage object StatusUpdate { /** Alternate factory method that takes a ByteBuffer directly for the data field */ - def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer) - : StatusUpdate = { - StatusUpdate(executorId, taskId, state, new SerializableBuffer(data)) + def apply(executorId: String, taskId: Long, state: TaskState, data: ByteBuffer, + resources: Map[String, ResourceInformation]): StatusUpdate = { + StatusUpdate(executorId, taskId, state, new SerializableBuffer(data), resources) } } @@ -104,6 +112,9 @@ private[spark] object CoarseGrainedClusterMessages { case class RegisterClusterManager(am: RpcEndpointRef) extends CoarseGrainedClusterMessage + // Used by YARN's client mode AM to retrieve the current set of delegation tokens. + object RetrieveDelegationTokens extends CoarseGrainedClusterMessage + // Request executors by specifying the new total number of executors desired // This includes executors already pending or running case class RequestExecutors( diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index de7c0d813ae65..d81070c362ba6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -18,14 +18,21 @@ package org.apache.spark.scheduler.cluster import java.util.concurrent.TimeUnit -import java.util.concurrent.atomic.AtomicInteger +import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.Future +import org.apache.hadoop.security.UserGroupInformation + import org.apache.spark.{ExecutorAllocationClient, SparkEnv, SparkException, TaskState} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.executor.ExecutorLogUrlHandler import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network._ import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -54,12 +61,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Submit tasks only after (registered resources / total expected resources) // is equal to at least this value, that is double between 0 and 1. private val _minRegisteredRatio = - math.min(1, conf.getDouble("spark.scheduler.minRegisteredResourcesRatio", 0)) + math.min(1, conf.get(SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO).getOrElse(0.0)) // Submit tasks after maxRegisteredWaitingTime milliseconds // if minRegisteredRatio has not yet been reached - private val maxRegisteredWaitingTimeMs = - conf.getTimeAsMs("spark.scheduler.maxRegisteredResourcesWaitingTime", "30s") - private val createTime = System.currentTimeMillis() + private val maxRegisteredWaitingTimeNs = TimeUnit.MILLISECONDS.toNanos( + conf.get(SCHEDULER_MAX_REGISTERED_RESOURCE_WAITING_TIME)) + private val createTimeNs = System.nanoTime() // Accessing `executorDataMap` in `DriverEndpoint.receive/receiveAndReply` doesn't need any // protection. But accessing `executorDataMap` out of `DriverEndpoint.receive/receiveAndReply` @@ -95,35 +102,54 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // The num of current max ExecutorId used to re-register appMaster @volatile protected var currentExecutorIdCounter = 0 + // Current set of delegation tokens to send to executors. + private val delegationTokens = new AtomicReference[Array[Byte]]() + + // The token manager used to create security tokens. + private var delegationTokenManager: Option[HadoopDelegationTokenManager] = None + private val reviveThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") - class DriverEndpoint(override val rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends ThreadSafeRpcEndpoint with Logging { + class DriverEndpoint extends ThreadSafeRpcEndpoint with Logging { + + override val rpcEnv: RpcEnv = CoarseGrainedSchedulerBackend.this.rpcEnv // Executors that have been lost, but for which we don't yet know the real exit reason. protected val executorsPendingLossReason = new HashSet[String] protected val addressToExecutorId = new HashMap[RpcAddress, String] + // Spark configuration sent to executors. This is a lazy val so that subclasses of the + // scheduler can modify the SparkConf object before this view is created. + private lazy val sparkProperties = scheduler.sc.conf.getAll + .filter { case (k, _) => k.startsWith("spark.") } + .toSeq + + private val logUrlHandler: ExecutorLogUrlHandler = new ExecutorLogUrlHandler( + conf.get(UI.CUSTOM_EXECUTOR_LOG_URL)) + override def onStart() { // Periodically revive offers to allow delay scheduling to work - val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "1s") + val reviveIntervalMs = conf.get(SCHEDULER_REVIVE_INTERVAL).getOrElse(1000L) - reviveThread.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - Option(self).foreach(_.send(ReviveOffers)) - } + reviveThread.scheduleAtFixedRate(() => Utils.tryLogNonFatalError { + Option(self).foreach(_.send(ReviveOffers)) }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) } override def receive: PartialFunction[Any, Unit] = { - case StatusUpdate(executorId, taskId, state, data) => + case StatusUpdate(executorId, taskId, state, data, resources) => scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { executorDataMap.get(executorId) match { case Some(executorInfo) => executorInfo.freeCores += scheduler.CPUS_PER_TASK + resources.foreach { case (k, v) => + executorInfo.resourcesInfo.get(k).foreach { r => + r.release(v.addresses) + } + } makeOffers(executorId) case None => // Ignoring the update since we don't know about the executor. @@ -152,9 +178,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } case UpdateDelegationTokens(newDelegationTokens) => - executorDataMap.values.foreach { ed => - ed.executorEndpoint.send(UpdateDelegationTokens(newDelegationTokens)) - } + updateDelegationTokens(newDelegationTokens) case RemoveExecutor(executorId, reason) => // We will remove the executor's state and cannot restore it. However, the connection @@ -166,7 +190,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls) => + case RegisterExecutor(executorId, executorRef, hostname, cores, logUrls, + attributes, resources) => if (executorDataMap.contains(executorId)) { executorRef.send(RegisterExecutorFailed("Duplicate executor ID: " + executorId)) context.reply(true) @@ -189,8 +214,11 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp addressToExecutorId(executorAddress) = executorId totalCoreCount.addAndGet(cores) totalRegisteredExecutors.addAndGet(1) + val resourcesInfo = resources.map{ case (k, v) => + (v.name, new ExecutorResourceInfo(v.name, v.addresses))} val data = new ExecutorData(executorRef, executorAddress, hostname, - cores, cores, logUrls) + cores, cores, logUrlHandler.applyPattern(logUrls, attributes), attributes, + resourcesInfo) // This must be synchronized because variables mutated // in this block are read when requesting executors CoarseGrainedSchedulerBackend.this.synchronized { @@ -230,24 +258,27 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp val reply = SparkAppConfig( sparkProperties, SparkEnv.get.securityManager.getIOEncryptionKey(), - fetchHadoopDelegationTokens()) + Option(delegationTokens.get())) context.reply(reply) } // Make fake resource offers on all executors private def makeOffers() { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing val activeExecutors = executorDataMap.filterKeys(executorIsAlive) val workOffers = activeExecutors.map { case (id, executorData) => new WorkerOffer(id, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort)) + Some(executorData.executorAddress.hostPort), + executorData.resourcesInfo.map { case (rName, rInfo) => + (rName, rInfo.availableAddrs.toBuffer) + }) }.toIndexedSeq scheduler.resourceOffers(workOffers) } - if (!taskDescs.isEmpty) { + if (taskDescs.nonEmpty) { launchTasks(taskDescs) } } @@ -263,19 +294,22 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp // Make fake resource offers on just one executor private def makeOffers(executorId: String) { // Make sure no executor is killed while some task is launching on it - val taskDescs = CoarseGrainedSchedulerBackend.this.synchronized { + val taskDescs = withLock { // Filter out executors under killing if (executorIsAlive(executorId)) { val executorData = executorDataMap(executorId) val workOffers = IndexedSeq( new WorkerOffer(executorId, executorData.executorHost, executorData.freeCores, - Some(executorData.executorAddress.hostPort))) + Some(executorData.executorAddress.hostPort), + executorData.resourcesInfo.map { case (rName, rInfo) => + (rName, rInfo.availableAddrs.toBuffer) + })) scheduler.resourceOffers(workOffers) } else { Seq.empty } } - if (!taskDescs.isEmpty) { + if (taskDescs.nonEmpty) { launchTasks(taskDescs) } } @@ -293,8 +327,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp Option(scheduler.taskIdToTaskSetManager.get(task.taskId)).foreach { taskSetMgr => try { var msg = "Serialized task %s:%d was %d bytes, which exceeds max allowed: " + - "spark.rpc.message.maxSize (%d bytes). Consider increasing " + - "spark.rpc.message.maxSize or using broadcast variables for large values." + s"${RPC_MESSAGE_MAX_SIZE.key} (%d bytes). Consider increasing " + + s"${RPC_MESSAGE_MAX_SIZE.key} or using broadcast variables for large values." msg = msg.format(task.taskId, task.index, serializedTask.limit(), maxRpcMessageSize) taskSetMgr.abort(msg) } catch { @@ -304,7 +338,13 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } else { val executorData = executorDataMap(task.executorId) + // Do resources allocation here. The allocated resources will get released after the task + // finishes. executorData.freeCores -= scheduler.CPUS_PER_TASK + task.resources.foreach { case (rName, rInfo) => + assert(executorData.resourcesInfo.contains(rName)) + executorData.resourcesInfo(rName).acquire(rInfo.addresses) + } logDebug(s"Launching task ${task.taskId} on executor id: ${task.executorId} hostname: " + s"${executorData.executorHost}.") @@ -376,30 +416,32 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } } - var driverEndpoint: RpcEndpointRef = null + val driverEndpoint = rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint()) protected def minRegisteredRatio: Double = _minRegisteredRatio override def start() { - val properties = new ArrayBuffer[(String, String)] - for ((key, value) <- scheduler.sc.conf.getAll) { - if (key.startsWith("spark.")) { - properties += ((key, value)) + if (UserGroupInformation.isSecurityEnabled()) { + delegationTokenManager = createTokenManager() + delegationTokenManager.foreach { dtm => + val ugi = UserGroupInformation.getCurrentUser() + val tokens = if (dtm.renewalEnabled) { + dtm.start() + } else if (ugi.hasKerberosCredentials() || SparkHadoopUtil.get.isProxyUser(ugi)) { + val creds = ugi.getCredentials() + dtm.obtainDelegationTokens(creds) + SparkHadoopUtil.get.serialize(creds) + } else { + null + } + if (tokens != null) { + updateDelegationTokens(tokens) + } } } - - // TODO (prashant) send conf instead of properties - driverEndpoint = createDriverEndpointRef(properties) - } - - protected def createDriverEndpointRef( - properties: ArrayBuffer[(String, String)]): RpcEndpointRef = { - rpcEnv.setupEndpoint(ENDPOINT_NAME, createDriverEndpoint(properties)) } - protected def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new DriverEndpoint(rpcEnv, properties) - } + protected def createDriverEndpoint(): DriverEndpoint = new DriverEndpoint() def stopExecutors() { try { @@ -416,6 +458,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp override def stop() { reviveThread.shutdownNow() stopExecutors() + delegationTokenManager.foreach(_.stop()) try { if (driverEndpoint != null) { driverEndpoint.askSync[Boolean](StopDriver) @@ -479,9 +522,9 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp s"reached minRegisteredResourcesRatio: $minRegisteredRatio") return true } - if ((System.currentTimeMillis() - createTime) >= maxRegisteredWaitingTimeMs) { + if ((System.nanoTime() - createTimeNs) >= maxRegisteredWaitingTimeNs) { logInfo("SchedulerBackend is ready for scheduling beginning after waiting " + - s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeMs(ms)") + s"maxRegisteredResourcesWaitingTime: $maxRegisteredWaitingTimeNs(ns)") return true } false @@ -496,12 +539,21 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp executorDataMap.keySet.toSeq } + override def isExecutorActive(id: String): Boolean = synchronized { + executorDataMap.contains(id) && !executorsPendingToRemove.contains(id) + } + override def maxNumConcurrentTasks(): Int = { executorDataMap.values.map { executor => executor.totalCores / scheduler.CPUS_PER_TASK }.sum } + // this function is for testing only + def getExecutorAvailableResources(executorId: String): Map[String, ExecutorResourceInfo] = { + executorDataMap.get(executorId).map(_.resourcesInfo).getOrElse(Map.empty) + } + /** * Request an additional number of executors from the cluster manager. * @return whether the request is acknowledged. @@ -607,7 +659,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp force: Boolean): Seq[String] = { logInfo(s"Requesting to kill executor(s) ${executorIds.mkString(", ")}") - val response = synchronized { + val response = withLock { val (knownExecutors, unknownExecutors) = executorIds.partition(executorDataMap.contains) unknownExecutors.foreach { id => logWarning(s"Executor to kill $id does not exist!") @@ -645,7 +697,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp } val killExecutors: Boolean => Future[Boolean] = - if (!executorsToKill.isEmpty) { + if (executorsToKill.nonEmpty) { _ => doKillExecutors(executorsToKill) } else { _ => Future.successful(false) @@ -684,7 +736,35 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, val rpcEnv: Rp true } - protected def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { None } + /** + * Create the delegation token manager to be used for the application. This method is called + * once during the start of the scheduler backend (so after the object has already been + * fully constructed), only if security is enabled in the Hadoop configuration. + */ + protected def createTokenManager(): Option[HadoopDelegationTokenManager] = None + + /** + * Called when a new set of delegation tokens is sent to the driver. Child classes can override + * this method but should always call this implementation, which handles token distribution to + * executors. + */ + protected def updateDelegationTokens(tokens: Array[Byte]): Unit = { + SparkHadoopUtil.get.addDelegationTokens(tokens, conf) + delegationTokens.set(tokens) + executorDataMap.values.foreach { ed => + ed.executorEndpoint.send(UpdateDelegationTokens(tokens)) + } + } + + protected def currentDelegationTokens: Array[Byte] = delegationTokens.get() + + // SPARK-27112: We need to ensure that there is ordering of lock acquisition + // between TaskSchedulerImpl and CoarseGrainedSchedulerBackend objects in order to fix + // the deadlock issue exposed in SPARK-27112 + private def withLock[T](fn: => T): T = scheduler.synchronized { + CoarseGrainedSchedulerBackend.this.synchronized { fn } + } + } private[spark] object CoarseGrainedSchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala index b25a4bfb501fb..435365d5b6e00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorData.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} +import org.apache.spark.scheduler.ExecutorResourceInfo /** * Grouping of data for an executor used by CoarseGrainedSchedulerBackend. @@ -27,6 +28,7 @@ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} * @param executorHost The hostname that this executor is running on * @param freeCores The current number of cores available for work on the executor * @param totalCores The total number of cores available to the executor + * @param resourcesInfo The information of the currently available resources on the executor */ private[cluster] class ExecutorData( val executorEndpoint: RpcEndpointRef, @@ -34,5 +36,7 @@ private[cluster] class ExecutorData( override val executorHost: String, var freeCores: Int, override val totalCores: Int, - override val logUrlMap: Map[String, String] -) extends ExecutorInfo(executorHost, totalCores, logUrlMap) + override val logUrlMap: Map[String, String], + override val attributes: Map[String, String], + val resourcesInfo: Map[String, ExecutorResourceInfo] +) extends ExecutorInfo(executorHost, totalCores, logUrlMap, attributes) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala index 7f218566146a1..3197e06fcd13a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/ExecutorInfo.scala @@ -26,7 +26,12 @@ import org.apache.spark.annotation.DeveloperApi class ExecutorInfo( val executorHost: String, val totalCores: Int, - val logUrlMap: Map[String, String]) { + val logUrlMap: Map[String, String], + val attributes: Map[String, String]) { + + def this(executorHost: String, totalCores: Int, logUrlMap: Map[String, String]) = { + this(executorHost, totalCores, logUrlMap, Map.empty) + } def canEqual(other: Any): Boolean = other.isInstanceOf[ExecutorInfo] @@ -35,12 +40,13 @@ class ExecutorInfo( (that canEqual this) && executorHost == that.executorHost && totalCores == that.totalCores && - logUrlMap == that.logUrlMap + logUrlMap == that.logUrlMap && + attributes == that.attributes case _ => false } override def hashCode(): Int = { - val state = Seq(executorHost, totalCores, logUrlMap) + val state = Seq(executorHost, totalCores, logUrlMap, attributes) state.map(_.hashCode()).foldLeft(0)((a, b) => 31 * a + b) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala index f73a58ff5d48c..e0605fee9cbf2 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/StandaloneSchedulerBackend.scala @@ -25,7 +25,8 @@ import scala.concurrent.Future import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{StandaloneAppClient, StandaloneAppClientListener} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler._ @@ -54,7 +55,7 @@ private[spark] class StandaloneSchedulerBackend( private val registrationBarrier = new Semaphore(0) - private val maxCores = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCores = conf.get(config.CORES_MAX) private val totalExpectedCores = maxCores.getOrElse(0) override def start() { @@ -69,8 +70,8 @@ private[spark] class StandaloneSchedulerBackend( // The endpoint for executors to talk to us val driverUrl = RpcEndpointAddress( - sc.conf.get("spark.driver.host"), - sc.conf.get("spark.driver.port").toInt, + sc.conf.get(config.DRIVER_HOST_ADDRESS), + sc.conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString val args = Seq( "--driver-url", driverUrl, @@ -79,18 +80,18 @@ private[spark] class StandaloneSchedulerBackend( "--cores", "{{CORES}}", "--app-id", "{{APP_ID}}", "--worker-url", "{{WORKER_URL}}") - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS) .map(Utils.splitCommandString).getOrElse(Seq.empty) - val classPathEntries = sc.conf.getOption("spark.executor.extraClassPath") + val classPathEntries = sc.conf.get(config.EXECUTOR_CLASS_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) - val libraryPathEntries = sc.conf.getOption("spark.executor.extraLibraryPath") + val libraryPathEntries = sc.conf.get(config.EXECUTOR_LIBRARY_PATH) .map(_.split(java.io.File.pathSeparator).toSeq).getOrElse(Nil) // When testing, expose the parent class path to the child. This is processed by // compute-classpath.{cmd,sh} and makes all needed jars available to child processes // when the assembly is built with the "*-provided" profiles enabled. val testingClassPath = - if (sys.props.contains("spark.testing")) { + if (sys.props.contains(IS_TESTING.key)) { sys.props("java.class.path").split(java.io.File.pathSeparator).toSeq } else { Nil @@ -102,7 +103,7 @@ private[spark] class StandaloneSchedulerBackend( val command = Command("org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs, classPathEntries ++ testingClassPath, libraryPathEntries, javaOpts) val webUrl = sc.ui.map(_.webUrl).getOrElse("") - val coresPerExecutor = conf.getOption("spark.executor.cores").map(_.toInt) + val coresPerExecutor = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) // If we're using dynamic allocation, set our initial executor limit to 0 for now. // ExecutorAllocationManager will send the real initial limit to the Master later. val initialExecutorLimit = @@ -223,8 +224,9 @@ private[spark] class StandaloneSchedulerBackend( if (stopping.compareAndSet(false, true)) { try { super.stop() - client.stop() - + if (client != null) { + client.stop() + } val callback = shutdownCallback if (callback != null) { callback(this) diff --git a/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala new file mode 100644 index 0000000000000..f5beb403555e9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitor.scala @@ -0,0 +1,494 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.dynalloc + +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage.RDDBlockId +import org.apache.spark.util.Clock + +/** + * A monitor for executor activity, used by ExecutorAllocationManager to detect idle executors. + */ +private[spark] class ExecutorMonitor( + conf: SparkConf, + client: ExecutorAllocationClient, + listenerBus: LiveListenerBus, + clock: Clock) extends SparkListener with CleanerListener with Logging { + + private val idleTimeoutMs = TimeUnit.SECONDS.toMillis( + conf.get(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT)) + private val storageTimeoutMs = TimeUnit.SECONDS.toMillis( + conf.get(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT)) + private val shuffleTimeoutMs = conf.get(DYN_ALLOCATION_SHUFFLE_TIMEOUT) + + private val fetchFromShuffleSvcEnabled = conf.get(SHUFFLE_SERVICE_ENABLED) && + conf.get(SHUFFLE_SERVICE_FETCH_RDD_ENABLED) + private val shuffleTrackingEnabled = !conf.get(SHUFFLE_SERVICE_ENABLED) && + conf.get(DYN_ALLOCATION_SHUFFLE_TRACKING) + + private val executors = new ConcurrentHashMap[String, Tracker]() + + // The following fields are an optimization to avoid having to scan all executors on every EAM + // schedule interval to find out which ones are timed out. They keep track of when the next + // executor timeout is expected to happen, and the current list of timed out executors. There's + // also a flag that forces the EAM task to recompute the timed out executors, in case some event + // arrives on the listener bus that may cause the current list of timed out executors to change. + // + // There's also per-executor state used for this purpose, so that recomputations can be triggered + // only when really necessary. + // + // Note that this isn't meant to, and cannot, always make the right decision about which executors + // are indeed timed out. For example, the EAM thread may detect a timed out executor while a new + // "task start" event has just been posted to the listener bus and hasn't yet been delivered to + // this listener. There are safeguards in other parts of the code that would prevent that executor + // from being removed. + private val nextTimeout = new AtomicLong(Long.MaxValue) + private var timedOutExecs = Seq.empty[String] + + // Active job tracking. + // + // The following state is used when an external shuffle service is not in use, and allows Spark + // to scale down based on whether the shuffle data stored in executors is in use. + // + // The algorithm works as following: when jobs start, some state is kept that tracks which stages + // are part of that job, and which shuffle ID is attached to those stages. As tasks finish, the + // executor tracking code is updated to include the list of shuffles for which it's storing + // shuffle data. + // + // If executors hold shuffle data that is related to an active job, then the executor is + // considered to be in "shuffle busy" state; meaning that the executor is not allowed to be + // removed. If the executor has shuffle data but it doesn't relate to any active job, then it + // may be removed when idle, following the shuffle-specific timeout configuration. + // + // The following fields are not thread-safe and should be only used from the event thread. + private val shuffleToActiveJobs = new mutable.HashMap[Int, mutable.ArrayBuffer[Int]]() + private val stageToShuffleID = new mutable.HashMap[Int, Int]() + private val jobToStageIDs = new mutable.HashMap[Int, Seq[Int]]() + + def reset(): Unit = { + executors.clear() + nextTimeout.set(Long.MaxValue) + timedOutExecs = Nil + } + + /** + * Returns the list of executors that are currently considered to be timed out. + * Should only be called from the EAM thread. + */ + def timedOutExecutors(): Seq[String] = { + val now = clock.getTimeMillis() + if (now >= nextTimeout.get()) { + // Temporarily set the next timeout at Long.MaxValue. This ensures that after + // scanning all executors below, we know when the next timeout for non-timed out + // executors is (whether that update came from the scan, or from a new event + // arriving in a different thread). + nextTimeout.set(Long.MaxValue) + + var newNextTimeout = Long.MaxValue + timedOutExecs = executors.asScala + .filter { case (_, exec) => !exec.pendingRemoval && !exec.hasActiveShuffle } + .filter { case (_, exec) => + val deadline = exec.timeoutAt + if (deadline > now) { + newNextTimeout = math.min(newNextTimeout, deadline) + exec.timedOut = false + false + } else { + exec.timedOut = true + true + } + } + .keys + .toSeq + updateNextTimeout(newNextTimeout) + } + timedOutExecs + } + + /** + * Mark the given executors as pending to be removed. Should only be called in the EAM thread. + */ + def executorsKilled(ids: Seq[String]): Unit = { + ids.foreach { id => + val tracker = executors.get(id) + if (tracker != null) { + tracker.pendingRemoval = true + } + } + + // Recompute timed out executors in the next EAM callback, since this call invalidates + // the current list. + nextTimeout.set(Long.MinValue) + } + + def executorCount: Int = executors.size() + + def pendingRemovalCount: Int = executors.asScala.count { case (_, exec) => exec.pendingRemoval } + + override def onJobStart(event: SparkListenerJobStart): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + val shuffleStages = event.stageInfos.flatMap { s => + s.shuffleDepId.toSeq.map { shuffleId => + s.stageId -> shuffleId + } + } + + var updateExecutors = false + shuffleStages.foreach { case (stageId, shuffle) => + val jobIDs = shuffleToActiveJobs.get(shuffle) match { + case Some(jobs) => + // If a shuffle is being re-used, we need to re-scan the executors and update their + // tracker with the information that the shuffle data they're storing is in use. + logDebug(s"Reusing shuffle $shuffle in job ${event.jobId}.") + updateExecutors = true + jobs + + case _ => + logDebug(s"Registered new shuffle $shuffle (from stage $stageId).") + val jobs = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs(shuffle) = jobs + jobs + } + jobIDs += event.jobId + } + + if (updateExecutors) { + val activeShuffleIds = shuffleStages.map(_._2).toSeq + var needTimeoutUpdate = false + val activatedExecs = new mutable.ArrayBuffer[String]() + executors.asScala.foreach { case (id, exec) => + if (!exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffleIds) + if (exec.hasActiveShuffle) { + needTimeoutUpdate = true + activatedExecs += id + } + } + } + + logDebug(s"Activated executors ${activatedExecs.mkString(",")} due to shuffle data " + + s"needed by new job ${event.jobId}.") + + if (needTimeoutUpdate) { + nextTimeout.set(Long.MinValue) + } + } + + stageToShuffleID ++= shuffleStages + jobToStageIDs(event.jobId) = shuffleStages.map(_._1).toSeq + } + + override def onJobEnd(event: SparkListenerJobEnd): Unit = { + if (!shuffleTrackingEnabled) { + return + } + + var updateExecutors = false + val activeShuffles = new mutable.ArrayBuffer[Int]() + shuffleToActiveJobs.foreach { case (shuffleId, jobs) => + jobs -= event.jobId + if (jobs.nonEmpty) { + activeShuffles += shuffleId + } else { + // If a shuffle went idle we need to update all executors to make sure they're correctly + // tracking active shuffles. + updateExecutors = true + } + } + + if (updateExecutors) { + if (log.isDebugEnabled()) { + if (activeShuffles.nonEmpty) { + logDebug( + s"Job ${event.jobId} ended, shuffles ${activeShuffles.mkString(",")} still active.") + } else { + logDebug(s"Job ${event.jobId} ended, no active shuffles remain.") + } + } + + val deactivatedExecs = new mutable.ArrayBuffer[String]() + executors.asScala.foreach { case (id, exec) => + if (exec.hasActiveShuffle) { + exec.updateActiveShuffles(activeShuffles) + if (!exec.hasActiveShuffle) { + deactivatedExecs += id + } + } + } + + logDebug(s"Executors ${deactivatedExecs.mkString(",")} do not have active shuffle data " + + s"after job ${event.jobId} finished.") + } + + jobToStageIDs.remove(event.jobId).foreach { stages => + stages.foreach { id => stageToShuffleID -= id } + } + } + + override def onTaskStart(event: SparkListenerTaskStart): Unit = { + val executorId = event.taskInfo.executorId + // Guard against a late arriving task start event (SPARK-26927). + if (client.isExecutorActive(executorId)) { + val exec = ensureExecutorIsTracked(executorId) + exec.updateRunningTasks(1) + } + } + + override def onTaskEnd(event: SparkListenerTaskEnd): Unit = { + val executorId = event.taskInfo.executorId + val exec = executors.get(executorId) + if (exec != null) { + // If the task succeeded and the stage generates shuffle data, record that this executor + // holds data for the shuffle. This code will track all executors that generate shuffle + // for the stage, even if speculative tasks generate duplicate shuffle data and end up + // being ignored by the map output tracker. + // + // This means that an executor may be marked as having shuffle data, and thus prevented + // from being removed, even though the data may not be used. + if (shuffleTrackingEnabled && event.reason == Success) { + stageToShuffleID.get(event.stageId).foreach { shuffleId => + exec.addShuffle(shuffleId) + } + } + + // Update the number of running tasks after checking for shuffle data, so that the shuffle + // information is up-to-date in case the executor is going idle. + exec.updateRunningTasks(-1) + } + } + + override def onExecutorAdded(event: SparkListenerExecutorAdded): Unit = { + val exec = ensureExecutorIsTracked(event.executorId) + exec.updateRunningTasks(0) + logInfo(s"New executor ${event.executorId} has registered (new total is ${executors.size()})") + } + + override def onExecutorRemoved(event: SparkListenerExecutorRemoved): Unit = { + val removed = executors.remove(event.executorId) + if (removed != null) { + logInfo(s"Executor ${event.executorId} removed (new total is ${executors.size()})") + if (!removed.pendingRemoval) { + nextTimeout.set(Long.MinValue) + } + } + } + + override def onBlockUpdated(event: SparkListenerBlockUpdated): Unit = { + if (!event.blockUpdatedInfo.blockId.isInstanceOf[RDDBlockId]) { + return + } + + val exec = ensureExecutorIsTracked(event.blockUpdatedInfo.blockManagerId.executorId) + val storageLevel = event.blockUpdatedInfo.storageLevel + val blockId = event.blockUpdatedInfo.blockId.asInstanceOf[RDDBlockId] + + // SPARK-27677. When a block can be fetched from the external shuffle service, the executor can + // be removed without hurting the application too much, since the cached data is still + // available. So don't count blocks that can be served by the external service. + if (storageLevel.isValid && (!fetchFromShuffleSvcEnabled || !storageLevel.useDisk)) { + val hadCachedBlocks = exec.cachedBlocks.nonEmpty + val blocks = exec.cachedBlocks.getOrElseUpdate(blockId.rddId, + new mutable.BitSet(blockId.splitIndex)) + blocks += blockId.splitIndex + + if (!hadCachedBlocks) { + exec.updateTimeout() + } + } else { + exec.cachedBlocks.get(blockId.rddId).foreach { blocks => + blocks -= blockId.splitIndex + if (blocks.isEmpty) { + exec.cachedBlocks -= blockId.rddId + if (exec.cachedBlocks.isEmpty) { + exec.updateTimeout() + } + } + } + } + } + + override def onUnpersistRDD(event: SparkListenerUnpersistRDD): Unit = { + executors.values().asScala.foreach { exec => + exec.cachedBlocks -= event.rddId + if (exec.cachedBlocks.isEmpty) { + exec.updateTimeout() + } + } + } + + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case ShuffleCleanedEvent(id) => cleanupShuffle(id) + case _ => + } + + override def rddCleaned(rddId: Int): Unit = { } + + override def shuffleCleaned(shuffleId: Int): Unit = { + // Because this is called in a completely separate thread, we post a custom event to the + // listener bus so that the internal state is safely updated. + listenerBus.post(ShuffleCleanedEvent(shuffleId)) + } + + override def broadcastCleaned(broadcastId: Long): Unit = { } + + override def accumCleaned(accId: Long): Unit = { } + + override def checkpointCleaned(rddId: Long): Unit = { } + + // Visible for testing. + private[dynalloc] def isExecutorIdle(id: String): Boolean = { + Option(executors.get(id)).map(_.isIdle).getOrElse(throw new NoSuchElementException(id)) + } + + // Visible for testing + private[dynalloc] def timedOutExecutors(when: Long): Seq[String] = { + executors.asScala.flatMap { case (id, tracker) => + if (tracker.isIdle && tracker.timeoutAt <= when) Some(id) else None + }.toSeq + } + + // Visible for testing + def executorsPendingToRemove(): Set[String] = { + executors.asScala.filter { case (_, exec) => exec.pendingRemoval }.keys.toSet + } + + /** + * This method should be used when updating executor state. It guards against a race condition in + * which the `SparkListenerTaskStart` event is posted before the `SparkListenerBlockManagerAdded` + * event, which is possible because these events are posted in different threads. (see SPARK-4951) + */ + private def ensureExecutorIsTracked(id: String): Tracker = { + executors.computeIfAbsent(id, _ => new Tracker()) + } + + private def updateNextTimeout(newValue: Long): Unit = { + while (true) { + val current = nextTimeout.get() + if (newValue >= current || nextTimeout.compareAndSet(current, newValue)) { + return + } + } + } + + private def cleanupShuffle(id: Int): Unit = { + logDebug(s"Cleaning up state related to shuffle $id.") + shuffleToActiveJobs -= id + executors.asScala.foreach { case (_, exec) => + exec.removeShuffle(id) + } + } + + private class Tracker { + @volatile var timeoutAt: Long = Long.MaxValue + + // Tracks whether this executor is thought to be timed out. It's used to detect when the list + // of timed out executors needs to be updated due to the executor's state changing. + @volatile var timedOut: Boolean = false + + var pendingRemoval: Boolean = false + var hasActiveShuffle: Boolean = false + + private var idleStart: Long = -1 + private var runningTasks: Int = 0 + + // Maps RDD IDs to the partition IDs stored in the executor. + // This should only be used in the event thread. + val cachedBlocks = new mutable.HashMap[Int, mutable.BitSet]() + + // The set of shuffles for which shuffle data is held by the executor. + // This should only be used in the event thread. + private val shuffleIds = if (shuffleTrackingEnabled) new mutable.HashSet[Int]() else null + + def isIdle: Boolean = idleStart >= 0 && !hasActiveShuffle + + def updateRunningTasks(delta: Int): Unit = { + runningTasks = math.max(0, runningTasks + delta) + idleStart = if (runningTasks == 0) clock.getTimeMillis() else -1L + updateTimeout() + } + + def updateTimeout(): Unit = { + val oldDeadline = timeoutAt + val newDeadline = if (idleStart >= 0) { + val timeout = if (cachedBlocks.nonEmpty || (shuffleIds != null && shuffleIds.nonEmpty)) { + val _cacheTimeout = if (cachedBlocks.nonEmpty) storageTimeoutMs else Long.MaxValue + val _shuffleTimeout = if (shuffleIds != null && shuffleIds.nonEmpty) { + shuffleTimeoutMs + } else { + Long.MaxValue + } + math.min(_cacheTimeout, _shuffleTimeout) + } else { + idleTimeoutMs + } + idleStart + timeout + } else { + Long.MaxValue + } + + timeoutAt = newDeadline + + // If the executor was thought to be timed out, but the new deadline is later than the + // old one, ask the EAM thread to update the list of timed out executors. + if (newDeadline > oldDeadline && timedOut) { + nextTimeout.set(Long.MinValue) + } else { + updateNextTimeout(newDeadline) + } + } + + def addShuffle(id: Int): Unit = { + if (shuffleIds.add(id)) { + hasActiveShuffle = true + } + } + + def removeShuffle(id: Int): Unit = { + if (shuffleIds.remove(id) && shuffleIds.isEmpty) { + hasActiveShuffle = false + if (isIdle) { + updateTimeout() + } + } + } + + def updateActiveShuffles(ids: Iterable[Int]): Unit = { + val hadActiveShuffle = hasActiveShuffle + hasActiveShuffle = ids.exists(shuffleIds.contains) + if (hadActiveShuffle && isIdle) { + updateTimeout() + } + } + } + + private case class ShuffleCleanedEvent(id: Int) extends SparkListenerEvent { + override protected[spark] def logEvent: Boolean = false + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala index 0de57fbd5600c..cbcc5310a59f0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalSchedulerBackend.scala @@ -24,7 +24,7 @@ import java.nio.ByteBuffer import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.{Executor, ExecutorBackend} -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ @@ -81,6 +81,7 @@ private[spark] class LocalEndpoint( } def reviveOffers() { + // local mode doesn't support extra resources like GPUs right now val offers = IndexedSeq(new WorkerOffer(localExecutorId, localExecutorHostname, freeCores, Some(rpcEnv.address.hostPort))) for (task <- scheduler.resourceOffers(offers).flatten) { @@ -116,7 +117,7 @@ private[spark] class LocalSchedulerBackend( * @param conf Spark configuration. */ def getUserClasspath(conf: SparkConf): Seq[URL] = { - val userClassPathStr = conf.getOption("spark.executor.extraClassPath") + val userClassPathStr = conf.get(config.EXECUTOR_CLASS_PATH) userClassPathStr.map(_.split(File.pathSeparator)).toSeq.flatten.map(new File(_).toURI.toURL) } @@ -129,7 +130,8 @@ private[spark] class LocalSchedulerBackend( listenerBus.post(SparkListenerExecutorAdded( System.currentTimeMillis, executorEndpoint.localExecutorId, - new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty))) + new ExecutorInfo(executorEndpoint.localExecutorHostname, totalCores, Map.empty, + Map.empty))) launcherBackend.setAppId(appId) launcherBackend.setState(SparkAppHandle.State.RUNNING) } diff --git a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala index 18b735b8035ab..a4df0d543ecbe 100644 --- a/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala +++ b/core/src/main/scala/org/apache/spark/security/CryptoStreamUtils.scala @@ -20,6 +20,7 @@ import java.io.{Closeable, InputStream, IOException, OutputStream} import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, WritableByteChannel} import java.util.Properties +import java.util.concurrent.TimeUnit import javax.crypto.KeyGenerator import javax.crypto.spec.{IvParameterSpec, SecretKeySpec} @@ -133,10 +134,10 @@ private[spark] object CryptoStreamUtils extends Logging { */ private[this] def createInitializationVector(properties: Properties): Array[Byte] = { val iv = new Array[Byte](IV_LENGTH_IN_BYTES) - val initialIVStart = System.currentTimeMillis() + val initialIVStart = System.nanoTime() CryptoRandomFactory.getCryptoRandom(properties).nextBytes(iv) - val initialIVFinish = System.currentTimeMillis() - val initialIVTime = initialIVFinish - initialIVStart + val initialIVFinish = System.nanoTime() + val initialIVTime = TimeUnit.NANOSECONDS.toMillis(initialIVFinish - initialIVStart) if (initialIVTime > 2000) { logWarning(s"It costs ${initialIVTime} milliseconds to create the Initialization Vector " + s"used by CryptoStream") diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala b/core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala similarity index 92% rename from core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala rename to core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala index ed0905088ab25..cff8d81443efd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HadoopDelegationTokenProvider.scala +++ b/core/src/main/scala/org/apache/spark/security/HadoopDelegationTokenProvider.scala @@ -15,17 +15,20 @@ * limitations under the License. */ -package org.apache.spark.deploy.security +package org.apache.spark.security import org.apache.hadoop.conf.Configuration import org.apache.hadoop.security.Credentials import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** + * ::DeveloperApi:: * Hadoop delegation token provider. */ -private[spark] trait HadoopDelegationTokenProvider { +@DeveloperApi +trait HadoopDelegationTokenProvider { /** * Name of the service to provide delegation tokens. This name should be unique. Spark will diff --git a/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala index ea38ccb289c30..dbcb376905338 100644 --- a/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala +++ b/core/src/main/scala/org/apache/spark/security/SocketAuthHelper.scala @@ -17,7 +17,7 @@ package org.apache.spark.security -import java.io.{DataInputStream, DataOutputStream, InputStream} +import java.io.{DataInputStream, DataOutputStream} import java.net.Socket import java.nio.charset.StandardCharsets.UTF_8 @@ -113,5 +113,4 @@ private[spark] class SocketAuthHelper(conf: SparkConf) { dout.write(bytes, 0, bytes.length) dout.flush() } - } diff --git a/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala b/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala new file mode 100644 index 0000000000000..548fd1b07ddc5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/security/SocketAuthServer.scala @@ -0,0 +1,139 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.security + +import java.io.{BufferedOutputStream, OutputStream} +import java.net.{InetAddress, ServerSocket, Socket} + +import scala.concurrent.Promise +import scala.concurrent.duration.Duration +import scala.util.Try + +import org.apache.spark.SparkEnv +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.util.{ThreadUtils, Utils} + + +/** + * Creates a server in the JVM to communicate with external processes (e.g., Python and R) for + * handling one batch of data, with authentication and error handling. + * + * The socket server can only accept one connection, or close if no connection + * in 15 seconds. + */ +private[spark] abstract class SocketAuthServer[T]( + authHelper: SocketAuthHelper, + threadName: String) { + + def this(env: SparkEnv, threadName: String) = this(new SocketAuthHelper(env.conf), threadName) + def this(threadName: String) = this(SparkEnv.get, threadName) + + private val promise = Promise[T]() + + private def startServer(): (Int, String) = { + val serverSocket = new ServerSocket(0, 1, InetAddress.getByAddress(Array(127, 0, 0, 1))) + // Close the socket if no connection in 15 seconds + serverSocket.setSoTimeout(15000) + + new Thread(threadName) { + setDaemon(true) + override def run(): Unit = { + var sock: Socket = null + try { + sock = serverSocket.accept() + authHelper.authClient(sock) + promise.complete(Try(handleConnection(sock))) + } finally { + JavaUtils.closeQuietly(serverSocket) + JavaUtils.closeQuietly(sock) + } + } + }.start() + (serverSocket.getLocalPort, authHelper.secret) + } + + val (port, secret) = startServer() + + /** + * Handle a connection which has already been authenticated. Any error from this function + * will clean up this connection and the entire server, and get propagated to [[getResult]]. + */ + def handleConnection(sock: Socket): T + + /** + * Blocks indefinitely for [[handleConnection]] to finish, and returns that result. If + * handleConnection throws an exception, this will throw an exception which includes the original + * exception as a cause. + */ + def getResult(): T = { + getResult(Duration.Inf) + } + + def getResult(wait: Duration): T = { + ThreadUtils.awaitResult(promise.future, wait) + } + +} + +/** + * Create a socket server class and run user function on the socket in a background thread + * that can read and write to the socket input/output streams. The function is passed in a + * socket that has been connected and authenticated. + */ +private[spark] class SocketFuncServer( + authHelper: SocketAuthHelper, + threadName: String, + func: Socket => Unit) extends SocketAuthServer[Unit](authHelper, threadName) { + + override def handleConnection(sock: Socket): Unit = { + func(sock) + } +} + +private[spark] object SocketAuthServer { + + /** + * Convenience function to create a socket server and run a user function in a background + * thread to write to an output stream. + * + * The socket server can only accept one connection, or close if no connection + * in 15 seconds. + * + * @param threadName Name for the background serving thread. + * @param authHelper SocketAuthHelper for authentication + * @param writeFunc User function to write to a given OutputStream + * @return 3-tuple (as a Java array) with the port number of a local socket which serves the + * data collected from this job, the secret for authentication, and a socket auth + * server object that can be used to join the JVM serving thread in Python. + */ + def serveToStream( + threadName: String, + authHelper: SocketAuthHelper)(writeFunc: OutputStream => Unit): Array[Any] = { + val handleFunc = (sock: Socket) => { + val out = new BufferedOutputStream(sock.getOutputStream()) + Utils.tryWithSafeFinally { + writeFunc(out) + } { + out.close() + } + } + + val server = new SocketFuncServer(authHelper, threadName, handleFunc) + Array(server.port, server.secret, server) + } +} diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index f60dcfddfdc20..70564eeefda88 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.internal.config._ import org.apache.spark.util.{ByteBufferInputStream, ByteBufferOutputStream, Utils} private[spark] class JavaSerializationStream( @@ -137,8 +138,8 @@ private[spark] class JavaSerializerInstance( */ @DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { - private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 100) - private var extraDebugInfo = conf.getBoolean("spark.serializer.extraDebugInfo", true) + private var counterReset = conf.get(SERIALIZER_OBJECT_STREAM_RESET) + private var extraDebugInfo = conf.get(SERIALIZER_EXTRA_DEBUG_INFO) protected def this() = this(new SparkConf()) // For deserialization only diff --git a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala index 72427dd6ce4d4..20774c8d999c1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -30,6 +30,7 @@ import scala.util.control.NonFatal import com.esotericsoftware.kryo.{Kryo, KryoException, Serializer => KryoClassSerializer} import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import com.esotericsoftware.kryo.io.{UnsafeInput => KryoUnsafeInput, UnsafeOutput => KryoUnsafeOutput} +import com.esotericsoftware.kryo.pool.{KryoCallback, KryoFactory, KryoPool} import com.esotericsoftware.kryo.serializers.{JavaSerializer => KryoJavaSerializer} import com.twitter.chill.{AllScalaRegistrar, EmptyScalaKryoInstantiator} import org.apache.avro.generic.{GenericData, GenericRecord} @@ -38,10 +39,11 @@ import org.roaringbitmap.RoaringBitmap import org.apache.spark._ import org.apache.spark.api.python.PythonBroadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.network.util.ByteUnit import org.apache.spark.scheduler.{CompressedMapStatus, HighlyCompressedMapStatus} import org.apache.spark.storage._ -import org.apache.spark.util.{BoundedPriorityQueue, SerializableConfiguration, SerializableJobConf, Utils} +import org.apache.spark.util.{BoundedPriorityQueue, ByteBufferInputStream, SerializableConfiguration, SerializableJobConf, Utils} import org.apache.spark.util.collection.CompactBuffer /** @@ -57,33 +59,34 @@ class KryoSerializer(conf: SparkConf) with Logging with Serializable { - private val bufferSizeKb = conf.getSizeAsKb("spark.kryoserializer.buffer", "64k") + private val bufferSizeKb = conf.get(KRYO_SERIALIZER_BUFFER_SIZE) if (bufferSizeKb >= ByteUnit.GiB.toKiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer must be less than " + - s"2048 mb, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} mb.") + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_BUFFER_SIZE.key} must be less than " + + s"2048 MiB, got: + ${ByteUnit.KiB.toMiB(bufferSizeKb)} MiB.") } private val bufferSize = ByteUnit.KiB.toBytes(bufferSizeKb).toInt - val maxBufferSizeMb = conf.getSizeAsMb("spark.kryoserializer.buffer.max", "64m").toInt + val maxBufferSizeMb = conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE).toInt if (maxBufferSizeMb >= ByteUnit.GiB.toMiB(2)) { - throw new IllegalArgumentException("spark.kryoserializer.buffer.max must be less than " + - s"2048 mb, got: + $maxBufferSizeMb mb.") + throw new IllegalArgumentException(s"${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} must be less " + + s"than 2048 MiB, got: $maxBufferSizeMb MiB.") } private val maxBufferSize = ByteUnit.MiB.toBytes(maxBufferSizeMb).toInt - private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) - private val registrationRequired = conf.getBoolean("spark.kryo.registrationRequired", false) - private val userRegistrators = conf.get("spark.kryo.registrator", "") - .split(',').map(_.trim) + private val referenceTracking = conf.get(KRYO_REFERENCE_TRACKING) + private val registrationRequired = conf.get(KRYO_REGISTRATION_REQUIRED) + private val userRegistrators = conf.get(KRYO_USER_REGISTRATORS) + .map(_.trim) .filter(!_.isEmpty) - private val classesToRegister = conf.get("spark.kryo.classesToRegister", "") - .split(',').map(_.trim) + private val classesToRegister = conf.get(KRYO_CLASSES_TO_REGISTER) + .map(_.trim) .filter(!_.isEmpty) private val avroSchemas = conf.getAvroSchema // whether to use unsafe based IO for serialization - private val useUnsafe = conf.getBoolean("spark.kryo.unsafe", false) + private val useUnsafe = conf.get(KRYO_USE_UNSAFE) + private val usePool = conf.get(KRYO_USE_POOL) def newKryoOutput(): KryoOutput = if (useUnsafe) { @@ -92,12 +95,41 @@ class KryoSerializer(conf: SparkConf) new KryoOutput(bufferSize, math.max(bufferSize, maxBufferSize)) } + @transient + private lazy val factory: KryoFactory = new KryoFactory() { + override def create: Kryo = { + newKryo() + } + } + + private class PoolWrapper extends KryoPool { + private var pool: KryoPool = getPool + + override def borrow(): Kryo = pool.borrow() + + override def release(kryo: Kryo): Unit = pool.release(kryo) + + override def run[T](kryoCallback: KryoCallback[T]): T = pool.run(kryoCallback) + + def reset(): Unit = { + pool = getPool + } + + private def getPool: KryoPool = { + new KryoPool.Builder(factory).softReferences.build + } + } + + @transient + private lazy val internalPool = new PoolWrapper + + def pool: KryoPool = internalPool + def newKryo(): Kryo = { val instantiator = new EmptyScalaKryoInstantiator val kryo = instantiator.newKryo() kryo.setRegistrationRequired(registrationRequired) - val oldClassLoader = Thread.currentThread.getContextClassLoader val classLoader = defaultClassLoader.getOrElse(Thread.currentThread.getContextClassLoader) // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. @@ -123,23 +155,22 @@ class KryoSerializer(conf: SparkConf) kryo.register(classOf[GenericRecord], new GenericAvroSerializer(avroSchemas)) kryo.register(classOf[GenericData.Record], new GenericAvroSerializer(avroSchemas)) - try { - // scalastyle:off classforname - // Use the default classloader when calling the user registrator. - Thread.currentThread.setContextClassLoader(classLoader) - // Register classes given through spark.kryo.classesToRegister. - classesToRegister - .foreach { className => kryo.register(Class.forName(className, true, classLoader)) } - // Allow the user to register their own classes by setting spark.kryo.registrator. - userRegistrators - .map(Class.forName(_, true, classLoader).newInstance().asInstanceOf[KryoRegistrator]) - .foreach { reg => reg.registerClasses(kryo) } - // scalastyle:on classforname - } catch { - case e: Exception => - throw new SparkException(s"Failed to register classes with Kryo", e) - } finally { - Thread.currentThread.setContextClassLoader(oldClassLoader) + // Use the default classloader when calling the user registrator. + Utils.withContextClassLoader(classLoader) { + try { + // Register classes given through spark.kryo.classesToRegister. + classesToRegister.foreach { className => + kryo.register(Utils.classForName(className, noSparkClassLoader = true)) + } + // Allow the user to register their own classes by setting spark.kryo.registrator. + userRegistrators + .map(Utils.classForName[KryoRegistrator](_, noSparkClassLoader = true). + getConstructor().newInstance()) + .foreach { reg => reg.registerClasses(kryo) } + } catch { + case e: Exception => + throw new SparkException(s"Failed to register classes with Kryo", e) + } } // Register Chill's classes; we do this after our ranges and the user's own classes to let @@ -181,28 +212,8 @@ class KryoSerializer(conf: SparkConf) // We can't load those class directly in order to avoid unnecessary jar dependencies. // We load them safely, ignore it if the class not found. - Seq( - "org.apache.spark.ml.feature.Instance", - "org.apache.spark.ml.feature.LabeledPoint", - "org.apache.spark.ml.feature.OffsetInstance", - "org.apache.spark.ml.linalg.DenseMatrix", - "org.apache.spark.ml.linalg.DenseVector", - "org.apache.spark.ml.linalg.Matrix", - "org.apache.spark.ml.linalg.SparseMatrix", - "org.apache.spark.ml.linalg.SparseVector", - "org.apache.spark.ml.linalg.Vector", - "org.apache.spark.ml.tree.impl.TreePoint", - "org.apache.spark.mllib.clustering.VectorWithNorm", - "org.apache.spark.mllib.linalg.DenseMatrix", - "org.apache.spark.mllib.linalg.DenseVector", - "org.apache.spark.mllib.linalg.Matrix", - "org.apache.spark.mllib.linalg.SparseMatrix", - "org.apache.spark.mllib.linalg.SparseVector", - "org.apache.spark.mllib.linalg.Vector", - "org.apache.spark.mllib.regression.LabeledPoint" - ).foreach { name => + KryoSerializer.loadableSparkClasses.foreach { clazz => try { - val clazz = Utils.classForName(name) kryo.register(clazz) } catch { case NonFatal(_) => // do nothing @@ -214,8 +225,14 @@ class KryoSerializer(conf: SparkConf) kryo } + override def setDefaultClassLoader(classLoader: ClassLoader): Serializer = { + super.setDefaultClassLoader(classLoader) + internalPool.reset() + this + } + override def newInstance(): SerializerInstance = { - new KryoSerializerInstance(this, useUnsafe) + new KryoSerializerInstance(this, useUnsafe, usePool) } private[spark] override lazy val supportsRelocationOfSerializedObjects: Boolean = { @@ -298,7 +315,8 @@ class KryoDeserializationStream( } } -private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boolean) +private[spark] class KryoSerializerInstance( + ks: KryoSerializer, useUnsafe: Boolean, usePool: Boolean) extends SerializerInstance { /** * A re-used [[Kryo]] instance. Methods will borrow this instance by calling `borrowKryo()`, do @@ -306,22 +324,29 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole * pool of size one. SerializerInstances are not thread-safe, hence accesses to this field are * not synchronized. */ - @Nullable private[this] var cachedKryo: Kryo = borrowKryo() + @Nullable private[this] var cachedKryo: Kryo = if (usePool) null else borrowKryo() /** * Borrows a [[Kryo]] instance. If possible, this tries to re-use a cached Kryo instance; * otherwise, it allocates a new instance. */ private[serializer] def borrowKryo(): Kryo = { - if (cachedKryo != null) { - val kryo = cachedKryo - // As a defensive measure, call reset() to clear any Kryo state that might have been modified - // by the last operation to borrow this instance (see SPARK-7766 for discussion of this issue) + if (usePool) { + val kryo = ks.pool.borrow() kryo.reset() - cachedKryo = null kryo } else { - ks.newKryo() + if (cachedKryo != null) { + val kryo = cachedKryo + // As a defensive measure, call reset() to clear any Kryo state that might have + // been modified by the last operation to borrow this instance + // (see SPARK-7766 for discussion of this issue) + kryo.reset() + cachedKryo = null + kryo + } else { + ks.newKryo() + } } } @@ -331,8 +356,12 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole * re-use. */ private[serializer] def releaseKryo(kryo: Kryo): Unit = { - if (cachedKryo == null) { - cachedKryo = kryo + if (usePool) { + ks.pool.release(kryo) + } else { + if (cachedKryo == null) { + cachedKryo = kryo + } } } @@ -348,7 +377,7 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole } catch { case e: KryoException if e.getMessage.startsWith("Buffer overflow") => throw new SparkException(s"Kryo serialization failed: ${e.getMessage}. To avoid this, " + - "increase spark.kryoserializer.buffer.max value.", e) + s"increase ${KRYO_SERIALIZER_MAX_BUFFER_SIZE.key} value.", e) } finally { releaseKryo(kryo) } @@ -358,7 +387,12 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole override def deserialize[T: ClassTag](bytes: ByteBuffer): T = { val kryo = borrowKryo() try { - input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + if (bytes.hasArray) { + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + } else { + input.setBuffer(new Array[Byte](4096)) + input.setInputStream(new ByteBufferInputStream(bytes)) + } kryo.readClassAndObject(input).asInstanceOf[T] } finally { releaseKryo(kryo) @@ -370,7 +404,12 @@ private[spark] class KryoSerializerInstance(ks: KryoSerializer, useUnsafe: Boole val oldClassLoader = kryo.getClassLoader try { kryo.setClassLoader(loader) - input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + if (bytes.hasArray) { + input.setBuffer(bytes.array(), bytes.arrayOffset() + bytes.position(), bytes.remaining()) + } else { + input.setBuffer(new Array[Byte](4096)) + input.setInputStream(new ByteBufferInputStream(bytes)) + } kryo.readClassAndObject(input).asInstanceOf[T] } finally { kryo.setClassLoader(oldClassLoader) @@ -445,6 +484,50 @@ private[serializer] object KryoSerializer { } } ) + + // classForName() is expensive in case the class is not found, so we filter the list of + // SQL / ML / MLlib classes once and then re-use that filtered list in newInstance() calls. + private lazy val loadableSparkClasses: Seq[Class[_]] = { + Seq( + "org.apache.spark.sql.catalyst.expressions.UnsafeRow", + "org.apache.spark.sql.catalyst.expressions.UnsafeArrayData", + "org.apache.spark.sql.catalyst.expressions.UnsafeMapData", + + "org.apache.spark.ml.attribute.Attribute", + "org.apache.spark.ml.attribute.AttributeGroup", + "org.apache.spark.ml.attribute.BinaryAttribute", + "org.apache.spark.ml.attribute.NominalAttribute", + "org.apache.spark.ml.attribute.NumericAttribute", + + "org.apache.spark.ml.feature.Instance", + "org.apache.spark.ml.feature.LabeledPoint", + "org.apache.spark.ml.feature.OffsetInstance", + "org.apache.spark.ml.linalg.DenseMatrix", + "org.apache.spark.ml.linalg.DenseVector", + "org.apache.spark.ml.linalg.Matrix", + "org.apache.spark.ml.linalg.SparseMatrix", + "org.apache.spark.ml.linalg.SparseVector", + "org.apache.spark.ml.linalg.Vector", + "org.apache.spark.ml.stat.distribution.MultivariateGaussian", + "org.apache.spark.ml.tree.impl.TreePoint", + "org.apache.spark.mllib.clustering.VectorWithNorm", + "org.apache.spark.mllib.linalg.DenseMatrix", + "org.apache.spark.mllib.linalg.DenseVector", + "org.apache.spark.mllib.linalg.Matrix", + "org.apache.spark.mllib.linalg.SparseMatrix", + "org.apache.spark.mllib.linalg.SparseVector", + "org.apache.spark.mllib.linalg.Vector", + "org.apache.spark.mllib.regression.LabeledPoint", + "org.apache.spark.mllib.stat.distribution.MultivariateGaussian" + ).flatMap { name => + try { + Some[Class[_]](Utils.classForName(name)) + } catch { + case NonFatal(_) => None // do nothing + case _: NoClassDefFoundError if Utils.isTesting => None // See SPARK-23422. + } + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala index 1d4b05caaa143..3e3c387911d36 100644 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import scala.reflect.ClassTag import org.apache.spark.SparkConf +import org.apache.spark.internal.config import org.apache.spark.io.CompressionCodec import org.apache.spark.security.CryptoStreamUtils import org.apache.spark.storage._ @@ -63,13 +64,13 @@ private[spark] class SerializerManager( } // Whether to compress broadcast variables that are stored - private[this] val compressBroadcast = conf.getBoolean("spark.broadcast.compress", true) + private[this] val compressBroadcast = conf.get(config.BROADCAST_COMPRESS) // Whether to compress shuffle output that are stored - private[this] val compressShuffle = conf.getBoolean("spark.shuffle.compress", true) + private[this] val compressShuffle = conf.get(config.SHUFFLE_COMPRESS) // Whether to compress RDD partitions that are stored serialized - private[this] val compressRdds = conf.getBoolean("spark.rdd.compress", false) + private[this] val compressRdds = conf.get(config.RDD_COMPRESS) // Whether to compress shuffle output temporarily spilled to disk - private[this] val compressShuffleSpill = conf.getBoolean("spark.shuffle.spill.compress", true) + private[this] val compressShuffleSpill = conf.get(config.SHUFFLE_SPILL_COMPRESS) /* The compression codec to use. Note that the "lazy" val is necessary because we want to delay * the initialization of the compression codec until it is first used. The reason is that a Spark diff --git a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala index 74b0e0b3a741a..c7843710413dd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/BlockStoreShuffleReader.scala @@ -33,6 +33,7 @@ private[spark] class BlockStoreShuffleReader[K, C]( startPartition: Int, endPartition: Int, context: TaskContext, + readMetrics: ShuffleReadMetricsReporter, serializerManager: SerializerManager = SparkEnv.get.serializerManager, blockManager: BlockManager = SparkEnv.get.blockManager, mapOutputTracker: MapOutputTracker = SparkEnv.get.mapOutputTracker) @@ -49,11 +50,13 @@ private[spark] class BlockStoreShuffleReader[K, C]( mapOutputTracker.getMapSizesByExecutorId(handle.shuffleId, startPartition, endPartition), serializerManager.wrapStream, // Note: we use getSizeAsMb when no suffix is provided for backwards compatibility - SparkEnv.get.conf.getSizeAsMb("spark.reducer.maxSizeInFlight", "48m") * 1024 * 1024, - SparkEnv.get.conf.getInt("spark.reducer.maxReqsInFlight", Int.MaxValue), + SparkEnv.get.conf.get(config.REDUCER_MAX_SIZE_IN_FLIGHT) * 1024 * 1024, + SparkEnv.get.conf.get(config.REDUCER_MAX_REQS_IN_FLIGHT), SparkEnv.get.conf.get(config.REDUCER_MAX_BLOCKS_IN_FLIGHT_PER_ADDRESS), SparkEnv.get.conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM), - SparkEnv.get.conf.getBoolean("spark.shuffle.detectCorrupt", true)) + SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT), + SparkEnv.get.conf.get(config.SHUFFLE_DETECT_CORRUPT_MEMORY), + readMetrics).toCompletionIterator val serializerInstance = dep.serializer.newInstance() @@ -66,7 +69,6 @@ private[spark] class BlockStoreShuffleReader[K, C]( } // Update the context task metrics for each record read. - val readMetrics = context.taskMetrics.createTempShuffleReadMetrics() val metricIter = CompletionIterator[(Any, Any), Iterator[(Any, Any)]]( recordIter.map { record => readMetrics.incRecordsRead(1) diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala index 4ea8a7120a9cc..18a743fbfa6fc 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleManager.scala @@ -38,7 +38,11 @@ private[spark] trait ShuffleManager { dependency: ShuffleDependency[K, V, C]): ShuffleHandle /** Get a writer for a given partition. Called on executors by map tasks. */ - def getWriter[K, V](handle: ShuffleHandle, mapId: Int, context: TaskContext): ShuffleWriter[K, V] + def getWriter[K, V]( + handle: ShuffleHandle, + mapId: Int, + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] /** * Get a reader for a range of reduce partitions (startPartition to endPartition-1, inclusive). @@ -48,7 +52,8 @@ private[spark] trait ShuffleManager { handle: ShuffleHandle, startPartition: Int, endPartition: Int, - context: TaskContext): ShuffleReader[K, C] + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] /** * Remove a shuffle's metadata from the ShuffleManager. diff --git a/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.scala new file mode 100644 index 0000000000000..5b0c7e9f2b0b4 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/ShuffleWriteProcessor.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.shuffle + +import org.apache.spark.{Partition, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.scheduler.MapStatus + +/** + * The interface for customizing shuffle write process. The driver create a ShuffleWriteProcessor + * and put it into [[ShuffleDependency]], and executors use it in each ShuffleMapTask. + */ +private[spark] class ShuffleWriteProcessor extends Serializable with Logging { + + /** + * Create a [[ShuffleWriteMetricsReporter]] from the task context. As the reporter is a + * per-row operator, here need a careful consideration on performance. + */ + protected def createMetricsReporter(context: TaskContext): ShuffleWriteMetricsReporter = { + context.taskMetrics().shuffleWriteMetrics + } + + /** + * The write process for particular partition, it controls the life circle of [[ShuffleWriter]] + * get from [[ShuffleManager]] and triggers rdd compute, finally return the [[MapStatus]] for + * this task. + */ + def write( + rdd: RDD[_], + dep: ShuffleDependency[_, _, _], + partitionId: Int, + context: TaskContext, + partition: Partition): MapStatus = { + var writer: ShuffleWriter[Any, Any] = null + try { + val manager = SparkEnv.get.shuffleManager + writer = manager.getWriter[Any, Any]( + dep.shuffleHandle, + partitionId, + context, + createMetricsReporter(context)) + writer.write( + rdd.iterator(partition, context).asInstanceOf[Iterator[_ <: Product2[Any, Any]]]) + writer.stop(success = true).get + } catch { + case e: Exception => + try { + if (writer != null) { + writer.stop(success = false) + } + } catch { + case e: Exception => + log.debug("Could not stop writer", e) + } + throw e + } + } +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/metrics.scala b/core/src/main/scala/org/apache/spark/shuffle/metrics.scala new file mode 100644 index 0000000000000..33be677bc90cb --- /dev/null +++ b/core/src/main/scala/org/apache/spark/shuffle/metrics.scala @@ -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. + */ + +package org.apache.spark.shuffle + +/** + * An interface for reporting shuffle read metrics, for each shuffle. This interface assumes + * all the methods are called on a single-threaded, i.e. concrete implementations would not need + * to synchronize. + * + * All methods have additional Spark visibility modifier to allow public, concrete implementations + * that still have these methods marked as private[spark]. + */ +private[spark] trait ShuffleReadMetricsReporter { + private[spark] def incRemoteBlocksFetched(v: Long): Unit + private[spark] def incLocalBlocksFetched(v: Long): Unit + private[spark] def incRemoteBytesRead(v: Long): Unit + private[spark] def incRemoteBytesReadToDisk(v: Long): Unit + private[spark] def incLocalBytesRead(v: Long): Unit + private[spark] def incFetchWaitTime(v: Long): Unit + private[spark] def incRecordsRead(v: Long): Unit +} + + +/** + * An interface for reporting shuffle write metrics. This interface assumes all the methods are + * called on a single-threaded, i.e. concrete implementations would not need to synchronize. + * + * All methods have additional Spark visibility modifier to allow public, concrete implementations + * that still have these methods marked as private[spark]. + */ +private[spark] trait ShuffleWriteMetricsReporter { + private[spark] def incBytesWritten(v: Long): Unit + private[spark] def incRecordsWritten(v: Long): Unit + private[spark] def incWriteTime(v: Long): Unit + private[spark] def decBytesWritten(v: Long): Unit + private[spark] def decRecordsWritten(v: Long): Unit +} diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala index 0caf84c6050a8..b59fa8e8a3ccd 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleManager.scala @@ -33,10 +33,10 @@ import org.apache.spark.shuffle._ * Sort-based shuffle has two different write paths for producing its map output files: * * - Serialized sorting: used when all three of the following conditions hold: - * 1. The shuffle dependency specifies no aggregation or output ordering. + * 1. The shuffle dependency specifies no map-side combine. * 2. The shuffle serializer supports relocation of serialized values (this is currently * supported by KryoSerializer and Spark SQL's custom serializers). - * 3. The shuffle produces fewer than 16777216 output partitions. + * 3. The shuffle produces fewer than or equal to 16777216 output partitions. * - Deserialized sorting: used to handle all other cases. * * ----------------------- @@ -114,16 +114,19 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager handle: ShuffleHandle, startPartition: Int, endPartition: Int, - context: TaskContext): ShuffleReader[K, C] = { + context: TaskContext, + metrics: ShuffleReadMetricsReporter): ShuffleReader[K, C] = { new BlockStoreShuffleReader( - handle.asInstanceOf[BaseShuffleHandle[K, _, C]], startPartition, endPartition, context) + handle.asInstanceOf[BaseShuffleHandle[K, _, C]], + startPartition, endPartition, context, metrics) } /** Get a writer for a given partition. Called on executors by map tasks. */ override def getWriter[K, V]( handle: ShuffleHandle, mapId: Int, - context: TaskContext): ShuffleWriter[K, V] = { + context: TaskContext, + metrics: ShuffleWriteMetricsReporter): ShuffleWriter[K, V] = { numMapsForShuffle.putIfAbsent( handle.shuffleId, handle.asInstanceOf[BaseShuffleHandle[_, _, _]].numMaps) val env = SparkEnv.get @@ -136,15 +139,16 @@ private[spark] class SortShuffleManager(conf: SparkConf) extends ShuffleManager unsafeShuffleHandle, mapId, context, - env.conf) + env.conf, + metrics) case bypassMergeSortHandle: BypassMergeSortShuffleHandle[K @unchecked, V @unchecked] => new BypassMergeSortShuffleWriter( env.blockManager, shuffleBlockResolver.asInstanceOf[IndexShuffleBlockResolver], bypassMergeSortHandle, mapId, - context, - env.conf) + env.conf, + metrics) case other: BaseShuffleHandle[K @unchecked, V @unchecked, _] => new SortShuffleWriter(shuffleBlockResolver, other, mapId, context) } diff --git a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala index 274399b9cc1f3..16058de8bf3ff 100644 --- a/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala +++ b/core/src/main/scala/org/apache/spark/shuffle/sort/SortShuffleWriter.scala @@ -18,7 +18,7 @@ package org.apache.spark.shuffle.sort import org.apache.spark._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.scheduler.MapStatus import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver, ShuffleWriter} import org.apache.spark.storage.ShuffleBlockId @@ -108,7 +108,7 @@ private[spark] object SortShuffleWriter { if (dep.mapSideCombine) { false } else { - val bypassMergeThreshold: Int = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + val bypassMergeThreshold: Int = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD) dep.partitioner.numPartitions <= bypassMergeThreshold } } diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala index 36aaf67b57298..0052fd42d276d 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusListener.scala @@ -19,14 +19,15 @@ package org.apache.spark.status import java.util.Date import java.util.concurrent.ConcurrentHashMap -import java.util.function.Function import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import org.apache.spark._ -import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.CPUS_PER_TASK +import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.status.api.v1 import org.apache.spark.storage._ @@ -44,10 +45,9 @@ private[spark] class AppStatusListener( kvstore: ElementTrackingStore, conf: SparkConf, live: Boolean, + appStatusSource: Option[AppStatusSource] = None, lastUpdateTime: Option[Long] = None) extends SparkListener with Logging { - import config._ - private var sparkVersion = SPARK_VERSION private var appInfo: v1.ApplicationInfo = null private var appSummary = new AppSummary(0, 0) @@ -58,6 +58,12 @@ private[spark] class AppStatusListener( // operations that we can live without when rapidly processing incoming task events. private val liveUpdatePeriodNs = if (live) conf.get(LIVE_ENTITY_UPDATE_PERIOD) else -1L + /** + * Minimum time elapsed before stale UI data is flushed. This avoids UI staleness when incoming + * task events are not fired frequently. + */ + private val liveUpdateMinFlushPeriod = conf.get(LIVE_ENTITY_UPDATE_MIN_FLUSH_PERIOD) + private val maxTasksPerStage = conf.get(MAX_RETAINED_TASKS_PER_STAGE) private val maxGraphRootNodes = conf.get(MAX_RETAINED_ROOT_NODES) @@ -70,10 +76,15 @@ private[spark] class AppStatusListener( private val liveTasks = new HashMap[Long, LiveTask]() private val liveRDDs = new HashMap[Int, LiveRDD]() private val pools = new HashMap[String, SchedulerPool]() + + private val SQL_EXECUTION_ID_KEY = "spark.sql.execution.id" // Keep the active executor count as a separate variable to avoid having to do synchronization // around liveExecutors. @volatile private var activeExecutorCount = 0 + /** The last time when flushing `LiveEntity`s. This is to avoid flushing too frequently. */ + private var lastFlushTimeNs = System.nanoTime() + kvstore.addTrigger(classOf[ExecutorSummaryWrapper], conf.get(MAX_RETAINED_DEAD_EXECUTORS)) { count => cleanupExecutors(count) } @@ -87,7 +98,8 @@ private[spark] class AppStatusListener( kvstore.onFlush { if (!live) { - flush() + val now = System.nanoTime() + flush(update(_, now)) } } @@ -125,9 +137,9 @@ private[spark] class AppStatusListener( // code registers the driver before this event is sent. event.driverLogs.foreach { logs => val driver = liveExecutors.get(SparkContext.DRIVER_IDENTIFIER) - .orElse(liveExecutors.get(SparkContext.LEGACY_DRIVER_IDENTIFIER)) driver.foreach { d => d.executorLogs = logs.toMap + d.attributes = event.driverAttributes.getOrElse(Map.empty).toMap update(d, System.nanoTime()) } } @@ -145,10 +157,11 @@ private[spark] class AppStatusListener( val envInfo = new v1.ApplicationEnvironmentInfo( runtime, details.getOrElse("Spark Properties", Nil), + details.getOrElse("Hadoop Properties", Nil), details.getOrElse("System Properties", Nil), details.getOrElse("Classpath Entries", Nil)) - coresPerTask = envInfo.sparkProperties.toMap.get("spark.task.cpus").map(_.toInt) + coresPerTask = envInfo.sparkProperties.toMap.get(CPUS_PER_TASK.key).map(_.toInt) .getOrElse(coresPerTask) kvstore.write(new ApplicationEnvironmentInfoWrapper(envInfo)) @@ -186,6 +199,7 @@ private[spark] class AppStatusListener( exec.totalCores = event.executorInfo.totalCores exec.maxTasks = event.executorInfo.totalCores / coresPerTask exec.executorLogs = event.executorInfo.logUrlMap + exec.attributes = event.executorInfo.attributes liveUpdate(exec, System.nanoTime()) } @@ -205,6 +219,30 @@ private[spark] class AppStatusListener( update(rdd, now) } } + // Remove all RDD partitions that reference the removed executor + liveRDDs.values.foreach { rdd => + rdd.getPartitions.values + .filter(_.executors.contains(event.executorId)) + .foreach { partition => + if (partition.executors.length == 1) { + rdd.removePartition(partition.blockName) + rdd.memoryUsed = addDeltaToValue(rdd.memoryUsed, partition.memoryUsed * -1) + rdd.diskUsed = addDeltaToValue(rdd.diskUsed, partition.diskUsed * -1) + } else { + rdd.memoryUsed = addDeltaToValue(rdd.memoryUsed, + (partition.memoryUsed / partition.executors.length) * -1) + rdd.diskUsed = addDeltaToValue(rdd.diskUsed, + (partition.diskUsed / partition.executors.length) * -1) + partition.update(partition.executors + .filter(!_.equals(event.executorId)), rdd.storageLevel, + addDeltaToValue(partition.memoryUsed, + (partition.memoryUsed / partition.executors.length) * -1), + addDeltaToValue(partition.diskUsed, + (partition.diskUsed / partition.executors.length) * -1)) + } + } + update(rdd, now) + } if (isExecutorActiveForLiveStages(exec)) { // the executor was running for a currently active stage, so save it for now in // deadExecutors, and remove when there are no active stages overlapping with the @@ -280,6 +318,11 @@ private[spark] class AppStatusListener( private def updateBlackListStatus(execId: String, blacklisted: Boolean): Unit = { liveExecutors.get(execId).foreach { exec => exec.isBlacklisted = blacklisted + if (blacklisted) { + appStatusSource.foreach(_.BLACKLISTED_EXECUTORS.inc()) + } else { + appStatusSource.foreach(_.UNBLACKLISTED_EXECUTORS.inc()) + } liveUpdate(exec, System.nanoTime()) } } @@ -310,17 +353,20 @@ private[spark] class AppStatusListener( } val lastStageInfo = event.stageInfos.sortBy(_.stageId).lastOption - val lastStageName = lastStageInfo.map(_.name).getOrElse("(Unknown Stage Name)") + val jobName = lastStageInfo.map(_.name).getOrElse("") val jobGroup = Option(event.properties) .flatMap { p => Option(p.getProperty(SparkContext.SPARK_JOB_GROUP_ID)) } + val sqlExecutionId = Option(event.properties) + .flatMap(p => Option(p.getProperty(SQL_EXECUTION_ID_KEY)).map(_.toLong)) val job = new LiveJob( event.jobId, - lastStageName, + jobName, if (event.time > 0) Some(new Date(event.time)) else None, event.stageIds, jobGroup, - numTasks) + numTasks, + sqlExecutionId) liveJobs.put(event.jobId, job) liveUpdate(job, now) @@ -382,11 +428,34 @@ private[spark] class AppStatusListener( } job.status = event.jobResult match { - case JobSucceeded => JobExecutionStatus.SUCCEEDED - case JobFailed(_) => JobExecutionStatus.FAILED + case JobSucceeded => + appStatusSource.foreach{_.SUCCEEDED_JOBS.inc()} + JobExecutionStatus.SUCCEEDED + case JobFailed(_) => + appStatusSource.foreach{_.FAILED_JOBS.inc()} + JobExecutionStatus.FAILED } job.completionTime = if (event.time > 0) Some(new Date(event.time)) else None + + for { + source <- appStatusSource + submissionTime <- job.submissionTime + completionTime <- job.completionTime + } { + source.JOB_DURATION.value.set(completionTime.getTime() - submissionTime.getTime()) + } + + // update global app status counters + appStatusSource.foreach { source => + source.COMPLETED_STAGES.inc(job.completedStages.size) + source.FAILED_STAGES.inc(job.failedStages) + source.COMPLETED_TASKS.inc(job.completedTasks) + source.FAILED_TASKS.inc(job.failedTasks) + source.KILLED_TASKS.inc(job.killedTasks) + source.SKIPPED_TASKS.inc(job.skippedTasks) + source.SKIPPED_STAGES.inc(job.skippedStages.size) + } update(job, now, last = true) if (job.status == JobExecutionStatus.SUCCEEDED) { appSummary = new AppSummary(appSummary.numCompletedJobs + 1, appSummary.numCompletedStages) @@ -400,7 +469,7 @@ private[spark] class AppStatusListener( val stage = getOrCreateStage(event.stageInfo) stage.status = v1.StageStatus.ACTIVE stage.schedulingPool = Option(event.properties).flatMap { p => - Option(p.getProperty("spark.scheduler.pool")) + Option(p.getProperty(SparkContext.SPARK_SCHEDULER_POOL)) }.getOrElse(SparkUI.DEFAULT_POOL_NAME) // Look at all active jobs to find the ones that mention this stage. @@ -445,6 +514,7 @@ private[spark] class AppStatusListener( val locality = event.taskInfo.taskLocality.toString() val count = stage.localitySummary.getOrElse(locality, 0L) + 1L stage.localitySummary = stage.localitySummary ++ Map(locality -> count) + stage.activeTasksPerExecutor(event.taskInfo.executorId) += 1 maybeUpdate(stage, now) stage.jobs.foreach { job => @@ -530,6 +600,7 @@ private[spark] class AppStatusListener( if (killedDelta > 0) { stage.killedSummary = killedTasksSummary(event.reason, stage.killedSummary) } + stage.activeTasksPerExecutor(event.taskInfo.executorId) -= 1 // [SPARK-24415] Wait for all tasks to finish before removing stage from live list val removeStage = stage.activeTasks == 0 && @@ -554,7 +625,11 @@ private[spark] class AppStatusListener( if (killedDelta > 0) { job.killedSummary = killedTasksSummary(event.reason, job.killedSummary) } - conditionalLiveUpdate(job, now, removeStage) + if (removeStage) { + update(job, now) + } else { + maybeUpdate(job, now) + } } val esummary = stage.executorSummary(event.taskInfo.executorId) @@ -565,7 +640,16 @@ private[spark] class AppStatusListener( if (metricsDelta != null) { esummary.metrics = LiveEntityHelpers.addMetrics(esummary.metrics, metricsDelta) } - conditionalLiveUpdate(esummary, now, removeStage) + + val isLastTask = stage.activeTasksPerExecutor(event.taskInfo.executorId) == 0 + + // If the last task of the executor finished, then update the esummary + // for both live and history events. + if (isLastTask) { + update(esummary, now) + } else { + maybeUpdate(esummary, now) + } if (!stage.cleaning && stage.savedTasks.get() > maxTasksPerStage) { stage.cleaning = true @@ -598,9 +682,14 @@ private[spark] class AppStatusListener( } } - // Force an update on live applications when the number of active tasks reaches 0. This is - // checked in some tests (e.g. SQLTestUtilsBase) so it needs to be reliably up to date. - conditionalLiveUpdate(exec, now, exec.activeTasks == 0) + // Force an update on both live and history applications when the number of active tasks + // reaches 0. This is checked in some tests (e.g. SQLTestUtilsBase) so it needs to be + // reliably up to date. + if (exec.activeTasks == 0) { + update(exec, now) + } else { + maybeUpdate(exec, now) + } } } @@ -611,6 +700,10 @@ private[spark] class AppStatusListener( val now = System.nanoTime() stage.info = event.stageInfo + // We have to update the stage status AFTER we create all the executorSummaries + // because stage deletion deletes whatever summaries it finds when the status is completed. + stage.executorSummaries.values.foreach(update(_, now)) + // Because of SPARK-20205, old event logs may contain valid stages without a submission time // in their start event. In those cases, we can only detect whether a stage was skipped by // waiting until the completion event, at which point the field would have been set. @@ -639,8 +732,6 @@ private[spark] class AppStatusListener( update(pool, now) } - stage.executorSummaries.values.foreach(update(_, now)) - val executorIdsForStage = stage.blackListedExecutors executorIdsForStage.foreach { executorId => liveExecutors.get(executorId).foreach { exec => @@ -752,6 +843,14 @@ private[spark] class AppStatusListener( } } } + // Flush updates if necessary. Executor heartbeat is an event that happens periodically. Flush + // here to ensure the staleness of Spark UI doesn't last more than + // `max(heartbeat interval, liveUpdateMinFlushPeriod)`. + if (now - lastFlushTimeNs > liveUpdateMinFlushPeriod) { + flush(maybeUpdate(_, now)) + // Re-get the current system time because `flush` may be slow and `now` is stale. + lastFlushTimeNs = System.nanoTime() + } } override def onStageExecutorMetrics(executorMetrics: SparkListenerStageExecutorMetrics): Unit = { @@ -760,11 +859,11 @@ private[spark] class AppStatusListener( // check if there is a new peak value for any of the executor level memory metrics, // while reading from the log. SparkListenerStageExecutorMetrics are only processed // when reading logs. - liveExecutors.get(executorMetrics.execId) - .orElse(deadExecutors.get(executorMetrics.execId)).map { exec => - if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { - update(exec, now) - } + liveExecutors.get(executorMetrics.execId).orElse( + deadExecutors.get(executorMetrics.execId)).foreach { exec => + if (exec.peakExecutorMetrics.compareAndUpdatePeakValues(executorMetrics.executorMetrics)) { + update(exec, now) + } } } @@ -772,22 +871,22 @@ private[spark] class AppStatusListener( event.blockUpdatedInfo.blockId match { case block: RDDBlockId => updateRDDBlock(event, block) case stream: StreamBlockId => updateStreamBlock(event, stream) + case broadcast: BroadcastBlockId => updateBroadcastBlock(event, broadcast) case _ => } } - /** Flush all live entities' data to the underlying store. */ - private def flush(): Unit = { - val now = System.nanoTime() + /** Go through all `LiveEntity`s and use `entityFlushFunc(entity)` to flush them. */ + private def flush(entityFlushFunc: LiveEntity => Unit): Unit = { liveStages.values.asScala.foreach { stage => - update(stage, now) - stage.executorSummaries.values.foreach(update(_, now)) + entityFlushFunc(stage) + stage.executorSummaries.values.foreach(entityFlushFunc) } - liveJobs.values.foreach(update(_, now)) - liveExecutors.values.foreach(update(_, now)) - liveTasks.values.foreach(update(_, now)) - liveRDDs.values.foreach(update(_, now)) - pools.values.foreach(update(_, now)) + liveJobs.values.foreach(entityFlushFunc) + liveExecutors.values.foreach(entityFlushFunc) + liveTasks.values.foreach(entityFlushFunc) + liveRDDs.values.foreach(entityFlushFunc) + pools.values.foreach(entityFlushFunc) } /** @@ -830,15 +929,7 @@ private[spark] class AppStatusListener( // Update the executor stats first, since they are used to calculate the free memory // on tracked RDD distributions. maybeExec.foreach { exec => - if (exec.hasMemoryInfo) { - if (storageLevel.useOffHeap) { - exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta) - } else { - exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta) - } - } - exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta) - exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta) + updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta) } // Update the block entry in the RDD info, keeping track of the deltas above so that we @@ -940,11 +1031,42 @@ private[spark] class AppStatusListener( } } + private def updateBroadcastBlock( + event: SparkListenerBlockUpdated, + broadcast: BroadcastBlockId): Unit = { + val executorId = event.blockUpdatedInfo.blockManagerId.executorId + liveExecutors.get(executorId).foreach { exec => + val now = System.nanoTime() + val storageLevel = event.blockUpdatedInfo.storageLevel + + // Whether values are being added to or removed from the existing accounting. + val diskDelta = event.blockUpdatedInfo.diskSize * (if (storageLevel.useDisk) 1 else -1) + val memoryDelta = event.blockUpdatedInfo.memSize * (if (storageLevel.useMemory) 1 else -1) + + updateExecutorMemoryDiskInfo(exec, storageLevel, memoryDelta, diskDelta) + maybeUpdate(exec, now) + } + } + + private def updateExecutorMemoryDiskInfo( + exec: LiveExecutor, + storageLevel: StorageLevel, + memoryDelta: Long, + diskDelta: Long): Unit = { + if (exec.hasMemoryInfo) { + if (storageLevel.useOffHeap) { + exec.usedOffHeap = addDeltaToValue(exec.usedOffHeap, memoryDelta) + } else { + exec.usedOnHeap = addDeltaToValue(exec.usedOnHeap, memoryDelta) + } + } + exec.memoryUsed = addDeltaToValue(exec.memoryUsed, memoryDelta) + exec.diskUsed = addDeltaToValue(exec.diskUsed, diskDelta) + } + private def getOrCreateStage(info: StageInfo): LiveStage = { val stage = liveStages.computeIfAbsent((info.stageId, info.attemptNumber), - new Function[(Int, Int), LiveStage]() { - override def apply(key: (Int, Int)): LiveStage = new LiveStage() - }) + (_: (Int, Int)) => new LiveStage()) stage.info = info stage } @@ -981,14 +1103,6 @@ private[spark] class AppStatusListener( } } - private def conditionalLiveUpdate(entity: LiveEntity, now: Long, condition: Boolean): Unit = { - if (condition) { - liveUpdate(entity, now) - } else { - maybeUpdate(entity, now) - } - } - private def cleanupExecutors(count: Long): Unit = { // Because the limit is on the number of *dead* executors, we need to calculate whether // there are actually enough dead executors to be deleted. @@ -1030,30 +1144,10 @@ private[spark] class AppStatusListener( s.info.status != v1.StageStatus.ACTIVE && s.info.status != v1.StageStatus.PENDING } - stages.foreach { s => + val stageIds = stages.map { s => val key = Array(s.info.stageId, s.info.attemptId) kvstore.delete(s.getClass(), key) - val execSummaries = kvstore.view(classOf[ExecutorStageSummaryWrapper]) - .index("stage") - .first(key) - .last(key) - .asScala - .toSeq - execSummaries.foreach { e => - kvstore.delete(e.getClass(), e.id) - } - - val tasks = kvstore.view(classOf[TaskDataWrapper]) - .index("stage") - .first(key) - .last(key) - .asScala - - tasks.foreach { t => - kvstore.delete(t.getClass(), t.taskId) - } - // Check whether there are remaining attempts for the same stage. If there aren't, then // also delete the RDD graph data. val remainingAttempts = kvstore.view(classOf[StageDataWrapper]) @@ -1075,7 +1169,14 @@ private[spark] class AppStatusListener( } cleanupCachedQuantiles(key) + key } + + // Delete summaries in one pass, as deleting them for each stage is slow + kvstore.removeAllByIndexValues(classOf[ExecutorStageSummaryWrapper], "stage", stageIds) + + // Delete tasks for all stages in one pass, as deleting them for each stage individually is slow + kvstore.removeAllByIndexValues(classOf[TaskDataWrapper], TaskIndexNames.STAGE, stageIds) } private def cleanupTasks(stage: LiveStage): Unit = { diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala new file mode 100644 index 0000000000000..f6a21578ff499 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/status/AppStatusSource.scala @@ -0,0 +1,78 @@ +/* + * 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.status + +import java.util.concurrent.atomic.AtomicLong + +import AppStatusSource.getCounter +import com.codahale.metrics.{Counter, Gauge, MetricRegistry} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Status.APP_STATUS_METRICS_ENABLED +import org.apache.spark.metrics.source.Source + +private [spark] class JobDuration(val value: AtomicLong) extends Gauge[Long] { + override def getValue: Long = value.get() +} + +private[spark] class AppStatusSource extends Source { + + override implicit val metricRegistry = new MetricRegistry() + + override val sourceName = "appStatus" + + val jobDuration = new JobDuration(new AtomicLong(0L)) + + // Duration of each job in milliseconds + val JOB_DURATION = metricRegistry + .register(MetricRegistry.name("jobDuration"), jobDuration) + + val FAILED_STAGES = getCounter("stages", "failedStages") + + val SKIPPED_STAGES = getCounter("stages", "skippedStages") + + val COMPLETED_STAGES = getCounter("stages", "completedStages") + + val SUCCEEDED_JOBS = getCounter("jobs", "succeededJobs") + + val FAILED_JOBS = getCounter("jobs", "failedJobs") + + val COMPLETED_TASKS = getCounter("tasks", "completedTasks") + + val FAILED_TASKS = getCounter("tasks", "failedTasks") + + val KILLED_TASKS = getCounter("tasks", "killedTasks") + + val SKIPPED_TASKS = getCounter("tasks", "skippedTasks") + + val BLACKLISTED_EXECUTORS = getCounter("tasks", "blackListedExecutors") + + val UNBLACKLISTED_EXECUTORS = getCounter("tasks", "unblackListedExecutors") +} + +private[spark] object AppStatusSource { + + def getCounter(prefix: String, name: String)(implicit metricRegistry: MetricRegistry): Counter = { + metricRegistry.counter(MetricRegistry.name(prefix, name)) + } + + def createSource(conf: SparkConf): Option[AppStatusSource] = { + Option(conf.get(APP_STATUS_METRICS_ENABLED)) + .filter(identity) + .map { _ => new AppStatusSource() } + } +} diff --git a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala index 9839cbb99f862..6a9677834de9b 100644 --- a/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala +++ b/core/src/main/scala/org/apache/spark/status/AppStatusStore.scala @@ -20,11 +20,12 @@ package org.apache.spark.status import java.util.{List => JList} import scala.collection.JavaConverters._ +import scala.collection.mutable.HashMap import org.apache.spark.{JobExecutionStatus, SparkConf} import org.apache.spark.status.api.v1 import org.apache.spark.ui.scope._ -import org.apache.spark.util.{Distribution, Utils} +import org.apache.spark.util.Utils import org.apache.spark.util.kvstore.{InMemoryStore, KVStore} /** @@ -56,6 +57,13 @@ private[spark] class AppStatusStore( store.read(classOf[JobDataWrapper], jobId).info } + // Returns job data and associated SQL execution ID of certain Job ID. + // If there is no related SQL execution, the SQL execution ID part will be None. + def jobWithAssociatedSql(jobId: Int): (v1.JobData, Option[Long]) = { + val data = store.read(classOf[JobDataWrapper], jobId) + (data.info, data.sqlExecutionId) + } + def executorList(activeOnly: Boolean): Seq[v1.ExecutorSummary] = { val base = store.view(classOf[ExecutorSummaryWrapper]) val filtered = if (activeOnly) { @@ -148,11 +156,20 @@ private[spark] class AppStatusStore( // cheaper for disk stores (avoids deserialization). val count = { Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) - .parent(stageKey) - .index(TaskIndexNames.EXEC_RUN_TIME) - .first(0L) - .closeableIterator() + if (store.isInstanceOf[InMemoryStore]) { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.STATUS) + .first("SUCCESS") + .last("SUCCESS") + .closeableIterator() + } else { + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(TaskIndexNames.EXEC_RUN_TIME) + .first(0L) + .closeableIterator() + } ) { it => var _count = 0L while (it.hasNext()) { @@ -221,30 +238,50 @@ private[spark] class AppStatusStore( // stabilize once the stage finishes. It's also slow, especially with disk stores. val indices = quantiles.map { q => math.min((q * count).toLong, count - 1) } + // TODO: Summary metrics needs to display all the successful tasks' metrics (SPARK-26119). + // For InMemory case, it is efficient to find using the following code. But for diskStore case + // we need an efficient solution to avoid deserialization time overhead. For that, we need to + // rework on the way indexing works, so that we can index by specific metrics for successful + // and failed tasks differently (would be tricky). Also would require changing the disk store + // version (to invalidate old stores). def scanTasks(index: String)(fn: TaskDataWrapper => Long): IndexedSeq[Double] = { - Utils.tryWithResource( - store.view(classOf[TaskDataWrapper]) + if (store.isInstanceOf[InMemoryStore]) { + val quantileTasks = store.view(classOf[TaskDataWrapper]) .parent(stageKey) .index(index) .first(0L) - .closeableIterator() - ) { it => - var last = Double.NaN - var currentIdx = -1L - indices.map { idx => - if (idx == currentIdx) { - last - } else { - val diff = idx - currentIdx - currentIdx = idx - if (it.skip(diff - 1)) { - last = fn(it.next()).toDouble + .asScala + .filter { _.status == "SUCCESS"} // Filter "SUCCESS" tasks + .toIndexedSeq + + indices.map { index => + fn(quantileTasks(index.toInt)).toDouble + }.toIndexedSeq + } else { + Utils.tryWithResource( + store.view(classOf[TaskDataWrapper]) + .parent(stageKey) + .index(index) + .first(0L) + .closeableIterator() + ) { it => + var last = Double.NaN + var currentIdx = -1L + indices.map { idx => + if (idx == currentIdx) { last } else { - Double.NaN + val diff = idx - currentIdx + currentIdx = idx + if (it.skip(diff - 1)) { + last = fn(it.next()).toDouble + last + } else { + Double.NaN + } } - } - }.toIndexedSeq + }.toIndexedSeq + } } } @@ -350,8 +387,9 @@ private[spark] class AppStatusStore( def taskList(stageId: Int, stageAttemptId: Int, maxTasks: Int): Seq[v1.TaskData] = { val stageKey = Array(stageId, stageAttemptId) - store.view(classOf[TaskDataWrapper]).index("stage").first(stageKey).last(stageKey).reverse() - .max(maxTasks).asScala.map(_.toApi).toSeq.reverse + val taskDataWrapperIter = store.view(classOf[TaskDataWrapper]).index("stage") + .first(stageKey).last(stageKey).reverse().max(maxTasks).asScala + constructTaskDataList(taskDataWrapperIter).reverse } def taskList( @@ -390,7 +428,8 @@ private[spark] class AppStatusStore( } val ordered = if (ascending) indexed else indexed.reverse() - ordered.skip(offset).max(length).asScala.map(_.toApi).toSeq + val taskDataWrapperIter = ordered.skip(offset).max(length).asScala + constructTaskDataList(taskDataWrapperIter) } def executorSummary(stageId: Int, attemptId: Int): Map[String, v1.ExecutorStageSummary] = { @@ -423,40 +462,53 @@ private[spark] class AppStatusStore( .toMap new v1.StageData( - stage.status, - stage.stageId, - stage.attemptId, - stage.numTasks, - stage.numActiveTasks, - stage.numCompleteTasks, - stage.numFailedTasks, - stage.numKilledTasks, - stage.numCompletedIndices, - stage.executorRunTime, - stage.executorCpuTime, - stage.submissionTime, - stage.firstTaskLaunchedTime, - stage.completionTime, - stage.failureReason, - stage.inputBytes, - stage.inputRecords, - stage.outputBytes, - stage.outputRecords, - stage.shuffleReadBytes, - stage.shuffleReadRecords, - stage.shuffleWriteBytes, - stage.shuffleWriteRecords, - stage.memoryBytesSpilled, - stage.diskBytesSpilled, - stage.name, - stage.description, - stage.details, - stage.schedulingPool, - stage.rddIds, - stage.accumulatorUpdates, - Some(tasks), - Some(executorSummary(stage.stageId, stage.attemptId)), - stage.killedTasksSummary) + status = stage.status, + stageId = stage.stageId, + attemptId = stage.attemptId, + numTasks = stage.numTasks, + numActiveTasks = stage.numActiveTasks, + numCompleteTasks = stage.numCompleteTasks, + numFailedTasks = stage.numFailedTasks, + numKilledTasks = stage.numKilledTasks, + numCompletedIndices = stage.numCompletedIndices, + submissionTime = stage.submissionTime, + firstTaskLaunchedTime = stage.firstTaskLaunchedTime, + completionTime = stage.completionTime, + failureReason = stage.failureReason, + executorDeserializeTime = stage.executorDeserializeTime, + executorDeserializeCpuTime = stage.executorDeserializeCpuTime, + executorRunTime = stage.executorRunTime, + executorCpuTime = stage.executorCpuTime, + resultSize = stage.resultSize, + jvmGcTime = stage.jvmGcTime, + resultSerializationTime = stage.resultSerializationTime, + memoryBytesSpilled = stage.memoryBytesSpilled, + diskBytesSpilled = stage.diskBytesSpilled, + peakExecutionMemory = stage.peakExecutionMemory, + inputBytes = stage.inputBytes, + inputRecords = stage.inputRecords, + outputBytes = stage.outputBytes, + outputRecords = stage.outputRecords, + shuffleRemoteBlocksFetched = stage.shuffleRemoteBlocksFetched, + shuffleLocalBlocksFetched = stage.shuffleLocalBlocksFetched, + shuffleFetchWaitTime = stage.shuffleFetchWaitTime, + shuffleRemoteBytesRead = stage.shuffleRemoteBytesRead, + shuffleRemoteBytesReadToDisk = stage.shuffleRemoteBytesReadToDisk, + shuffleLocalBytesRead = stage.shuffleLocalBytesRead, + shuffleReadBytes = stage.shuffleReadBytes, + shuffleReadRecords = stage.shuffleReadRecords, + shuffleWriteBytes = stage.shuffleWriteBytes, + shuffleWriteTime = stage.shuffleWriteTime, + shuffleWriteRecords = stage.shuffleWriteRecords, + name = stage.name, + description = stage.description, + details = stage.details, + schedulingPool = stage.schedulingPool, + rddIds = stage.rddIds, + accumulatorUpdates = stage.accumulatorUpdates, + tasks = Some(tasks), + executorSummary = Some(executorSummary(stage.stageId, stage.attemptId)), + killedTasksSummary = stage.killedTasksSummary) } def rdd(rddId: Int): v1.RDDStorageInfo = { @@ -496,6 +548,29 @@ private[spark] class AppStatusStore( store.close() } + def constructTaskDataList(taskDataWrapperIter: Iterable[TaskDataWrapper]): Seq[v1.TaskData] = { + val executorIdToLogs = new HashMap[String, Map[String, String]]() + taskDataWrapperIter.map { taskDataWrapper => + val taskDataOld: v1.TaskData = taskDataWrapper.toApi + val executorLogs = executorIdToLogs.getOrElseUpdate(taskDataOld.executorId, { + try { + executorSummary(taskDataOld.executorId).executorLogs + } catch { + case e: NoSuchElementException => + Map.empty + } + }) + + new v1.TaskData(taskDataOld.taskId, taskDataOld.index, + taskDataOld.attempt, taskDataOld.launchTime, taskDataOld.resultFetchStart, + taskDataOld.duration, taskDataOld.executorId, taskDataOld.host, taskDataOld.status, + taskDataOld.taskLocality, taskDataOld.speculative, taskDataOld.accumulatorUpdates, + taskDataOld.errorMessage, taskDataOld.taskMetrics, + executorLogs, + AppStatusUtils.schedulerDelay(taskDataOld), + AppStatusUtils.gettingResultTime(taskDataOld)) + }.toSeq + } } private[spark] object AppStatusStore { @@ -505,10 +580,11 @@ private[spark] object AppStatusStore { /** * Create an in-memory store for a live application. */ - def createLiveStore(conf: SparkConf): AppStatusStore = { + def createLiveStore( + conf: SparkConf, + appStatusSource: Option[AppStatusSource] = None): AppStatusStore = { val store = new ElementTrackingStore(new InMemoryStore(), conf) - val listener = new AppStatusListener(store, conf, true) + val listener = new AppStatusListener(store, conf, true, appStatusSource) new AppStatusStore(store, listener = Some(listener)) } - } diff --git a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala index 863b0967f765e..38cb030297c81 100644 --- a/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala +++ b/core/src/main/scala/org/apache/spark/status/ElementTrackingStore.scala @@ -17,13 +17,18 @@ package org.apache.spark.status +import java.util.Collection import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} import com.google.common.util.concurrent.MoreExecutors import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Status._ +import org.apache.spark.status.ElementTrackingStore._ import org.apache.spark.util.{ThreadUtils, Utils} import org.apache.spark.util.kvstore._ @@ -45,9 +50,27 @@ import org.apache.spark.util.kvstore._ */ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) extends KVStore { - import config._ + private class LatchedTriggers(val triggers: Seq[Trigger[_]]) { + private val pending = new AtomicBoolean(false) - private val triggers = new HashMap[Class[_], Seq[Trigger[_]]]() + def fireOnce(f: Seq[Trigger[_]] => Unit): WriteQueueResult = { + if (pending.compareAndSet(false, true)) { + doAsync { + pending.set(false) + f(triggers) + } + WriteQueued + } else { + WriteSkippedQueue + } + } + + def :+(addlTrigger: Trigger[_]): LatchedTriggers = { + new LatchedTriggers(triggers :+ addlTrigger) + } + } + + private val triggers = new HashMap[Class[_], LatchedTriggers]() private val flushTriggers = new ListBuffer[() => Unit]() private val executor = if (conf.get(ASYNC_TRACKING_ENABLED)) { ThreadUtils.newDaemonSingleThreadExecutor("element-tracking-store-worker") @@ -67,8 +90,13 @@ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) exten * of elements of the registered type currently known to be in the store. */ def addTrigger(klass: Class[_], threshold: Long)(action: Long => Unit): Unit = { - val existing = triggers.getOrElse(klass, Seq()) - triggers(klass) = existing :+ Trigger(threshold, action) + val newTrigger = Trigger(threshold, action) + triggers.get(klass) match { + case None => + triggers(klass) = new LatchedTriggers(Seq(newTrigger)) + case Some(latchedTrigger) => + triggers(klass) = latchedTrigger :+ newTrigger + } } /** @@ -97,23 +125,35 @@ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) exten override def write(value: Any): Unit = store.write(value) /** Write an element to the store, optionally checking for whether to fire triggers. */ - def write(value: Any, checkTriggers: Boolean): Unit = { + def write(value: Any, checkTriggers: Boolean): WriteQueueResult = { write(value) if (checkTriggers && !stopped) { - triggers.get(value.getClass()).foreach { list => - doAsync { - val count = store.count(value.getClass()) + triggers.get(value.getClass).map { latchedList => + latchedList.fireOnce { list => + val count = store.count(value.getClass) list.foreach { t => if (count > t.threshold) { t.action(count) } } } - } + }.getOrElse(WriteSkippedQueue) + } else { + WriteSkippedQueue } } + def removeAllByIndexValues[T](klass: Class[T], index: String, indexValues: Iterable[_]): Boolean = + removeAllByIndexValues(klass, index, indexValues.asJavaCollection) + + override def removeAllByIndexValues[T]( + klass: Class[T], + index: String, + indexValues: Collection[_]): Boolean = { + store.removeAllByIndexValues(klass, index, indexValues) + } + override def delete(klass: Class[_], naturalKey: Any): Unit = store.delete(klass, naturalKey) override def getMetadata[T](klass: Class[T]): T = store.getMetadata(klass) @@ -158,3 +198,14 @@ private[spark] class ElementTrackingStore(store: KVStore, conf: SparkConf) exten action: Long => Unit) } + +private[spark] object ElementTrackingStore { + /** + * This trait is solely to assist testing the correctness of single-fire execution + * The result of write() is otherwise unused. + */ + sealed trait WriteQueueResult + + object WriteQueued extends WriteQueueResult + object WriteSkippedQueue extends WriteQueueResult +} diff --git a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala index 8708e64db3c17..c5a233f14aa6d 100644 --- a/core/src/main/scala/org/apache/spark/status/LiveEntity.scala +++ b/core/src/main/scala/org/apache/spark/status/LiveEntity.scala @@ -61,10 +61,11 @@ private[spark] abstract class LiveEntity { private class LiveJob( val jobId: Int, name: String, - submissionTime: Option[Date], + val submissionTime: Option[Date], val stageIds: Seq[Int], jobGroup: Option[String], - numTasks: Int) extends LiveEntity { + numTasks: Int, + sqlExecutionId: Option[Long]) extends LiveEntity { var activeTasks = 0 var completedTasks = 0 @@ -108,7 +109,7 @@ private class LiveJob( skippedStages.size, failedStages, killedSummary) - new JobDataWrapper(info, skippedStages) + new JobDataWrapper(info, skippedStages, sqlExecutionId) } } @@ -257,6 +258,7 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE var blacklistedInStages: Set[Int] = TreeSet() var executorLogs = Map[String, String]() + var attributes = Map[String, String]() // Memory metrics. They may not be recorded (e.g. old event logs) so if totalOnHeap is not // initialized, the store will not contain this information. @@ -305,7 +307,8 @@ private class LiveExecutor(val executorId: String, _addTime: Long) extends LiveE executorLogs, memoryMetrics, blacklistedInStages, - Some(peakExecutorMetrics).filter(_.isSet)) + Some(peakExecutorMetrics).filter(_.isSet), + attributes) new ExecutorSummaryWrapper(info) } } @@ -376,6 +379,8 @@ private class LiveStage extends LiveEntity { val executorSummaries = new HashMap[String, LiveExecutorStageSummary]() + val activeTasksPerExecutor = new HashMap[String, Int]().withDefaultValue(0) + var blackListedExecutors = new HashSet[String]() // Used for cleanup of tasks after they reach the configured limit. Not written to the store. @@ -389,45 +394,59 @@ private class LiveStage extends LiveEntity { def toApi(): v1.StageData = { new v1.StageData( - status, - info.stageId, - info.attemptNumber, - - info.numTasks, - activeTasks, - completedTasks, - failedTasks, - killedTasks, - completedIndices.size, - - metrics.executorRunTime, - metrics.executorCpuTime, - info.submissionTime.map(new Date(_)), - if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) else None, - info.completionTime.map(new Date(_)), - info.failureReason, - - metrics.inputMetrics.bytesRead, - metrics.inputMetrics.recordsRead, - metrics.outputMetrics.bytesWritten, - metrics.outputMetrics.recordsWritten, - metrics.shuffleReadMetrics.localBytesRead + metrics.shuffleReadMetrics.remoteBytesRead, - metrics.shuffleReadMetrics.recordsRead, - metrics.shuffleWriteMetrics.bytesWritten, - metrics.shuffleWriteMetrics.recordsWritten, - metrics.memoryBytesSpilled, - metrics.diskBytesSpilled, - - info.name, - description, - info.details, - schedulingPool, - - info.rddInfos.map(_.id), - newAccumulatorInfos(info.accumulables.values), - None, - None, - killedSummary) + status = status, + stageId = info.stageId, + attemptId = info.attemptNumber, + numTasks = info.numTasks, + numActiveTasks = activeTasks, + numCompleteTasks = completedTasks, + numFailedTasks = failedTasks, + numKilledTasks = killedTasks, + numCompletedIndices = completedIndices.size, + + submissionTime = info.submissionTime.map(new Date(_)), + firstTaskLaunchedTime = + if (firstLaunchTime < Long.MaxValue) Some(new Date(firstLaunchTime)) else None, + completionTime = info.completionTime.map(new Date(_)), + failureReason = info.failureReason, + + executorDeserializeTime = metrics.executorDeserializeTime, + executorDeserializeCpuTime = metrics.executorDeserializeCpuTime, + executorRunTime = metrics.executorRunTime, + executorCpuTime = metrics.executorCpuTime, + resultSize = metrics.resultSize, + jvmGcTime = metrics.jvmGcTime, + resultSerializationTime = metrics.resultSerializationTime, + memoryBytesSpilled = metrics.memoryBytesSpilled, + diskBytesSpilled = metrics.diskBytesSpilled, + peakExecutionMemory = metrics.peakExecutionMemory, + inputBytes = metrics.inputMetrics.bytesRead, + inputRecords = metrics.inputMetrics.recordsRead, + outputBytes = metrics.outputMetrics.bytesWritten, + outputRecords = metrics.outputMetrics.recordsWritten, + shuffleRemoteBlocksFetched = metrics.shuffleReadMetrics.remoteBlocksFetched, + shuffleLocalBlocksFetched = metrics.shuffleReadMetrics.localBlocksFetched, + shuffleFetchWaitTime = metrics.shuffleReadMetrics.fetchWaitTime, + shuffleRemoteBytesRead = metrics.shuffleReadMetrics.remoteBytesRead, + shuffleRemoteBytesReadToDisk = metrics.shuffleReadMetrics.remoteBytesReadToDisk, + shuffleLocalBytesRead = metrics.shuffleReadMetrics.localBytesRead, + shuffleReadBytes = + metrics.shuffleReadMetrics.localBytesRead + metrics.shuffleReadMetrics.remoteBytesRead, + shuffleReadRecords = metrics.shuffleReadMetrics.recordsRead, + shuffleWriteBytes = metrics.shuffleWriteMetrics.bytesWritten, + shuffleWriteTime = metrics.shuffleWriteMetrics.writeTime, + shuffleWriteRecords = metrics.shuffleWriteMetrics.recordsWritten, + + name = info.name, + description = description, + details = info.details, + schedulingPool = schedulingPool, + + rddIds = info.rddInfos.map(_.id), + accumulatorUpdates = newAccumulatorInfos(info.accumulables.values), + tasks = None, + executorSummary = None, + killedTasksSummary = killedSummary) } override protected def doUpdate(): Any = { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala index 84c2ad48f1f27..83f76db7e89da 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApiRootResource.scala @@ -77,7 +77,7 @@ private[spark] trait UIRoot { /** * Runs some code with the current SparkUI instance for the app / attempt. * - * @throws NoSuchElementException If the app / attempt pair does not exist. + * @throws java.util.NoSuchElementException If the app / attempt pair does not exist. */ def withSparkUI[T](appId: String, attemptId: Option[String])(fn: SparkUI => T): T @@ -85,8 +85,8 @@ private[spark] trait UIRoot { def getApplicationInfo(appId: String): Option[ApplicationInfo] /** - * Write the event logs for the given app to the [[ZipOutputStream]] instance. If attemptId is - * [[None]], event logs for all attempts of this application will be written out. + * Write the event logs for the given app to the `ZipOutputStream` instance. If attemptId is + * `None`, event logs for all attempts of this application will be written out. */ def writeEventLogs(appId: String, attemptId: Option[String], zipStream: ZipOutputStream): Unit = { Response.serverError() diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala index 69054f2b771f1..197cf64ebdc5f 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/ApplicationListResource.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.status.api.v1 -import java.util.{Date, List => JList} +import java.util.{List => JList} import javax.ws.rs.{DefaultValue, GET, Produces, QueryParam} import javax.ws.rs.core.MediaType diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala index 50a286d0d3b0f..846e4f9cc94e5 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/JacksonMessageWriter.scala @@ -19,7 +19,6 @@ package org.apache.spark.status.api.v1 import java.io.OutputStream import java.lang.annotation.Annotation import java.lang.reflect.Type -import java.nio.charset.StandardCharsets import java.text.SimpleDateFormat import java.util.{Calendar, Locale, SimpleTimeZone} import javax.ws.rs.Produces diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala index 1f4082cac8f75..2ee9d3d0815a1 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/OneApplicationResource.scala @@ -25,7 +25,6 @@ import javax.ws.rs.core.{MediaType, Response, StreamingOutput} import scala.util.control.NonFatal import org.apache.spark.{JobExecutionStatus, SparkContext} -import org.apache.spark.ui.UIUtils @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class AbstractApplicationResource extends BaseAppResource { diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala index 30d52b97833e6..44ee322a22a10 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/StagesResource.scala @@ -16,15 +16,13 @@ */ package org.apache.spark.status.api.v1 -import java.util.{List => JList} +import java.util.{HashMap, List => JList, Locale} import javax.ws.rs._ -import javax.ws.rs.core.MediaType +import javax.ws.rs.core.{Context, MediaType, MultivaluedMap, UriInfo} -import org.apache.spark.SparkException -import org.apache.spark.scheduler.StageInfo -import org.apache.spark.status.api.v1.StageStatus._ -import org.apache.spark.status.api.v1.TaskSorting._ -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.jobs.ApiHelper._ +import org.apache.spark.util.Utils @Produces(Array(MediaType.APPLICATION_JSON)) private[v1] class StagesResource extends BaseAppResource { @@ -102,4 +100,129 @@ private[v1] class StagesResource extends BaseAppResource { withUI(_.store.taskList(stageId, stageAttemptId, offset, length, sortBy)) } + // This api needs to stay formatted exactly as it is below, since, it is being used by the + // datatables for the stages page. + @GET + @Path("{stageId: \\d+}/{stageAttemptId: \\d+}/taskTable") + def taskTable( + @PathParam("stageId") stageId: Int, + @PathParam("stageAttemptId") stageAttemptId: Int, + @QueryParam("details") @DefaultValue("true") details: Boolean, + @Context uriInfo: UriInfo): + HashMap[String, Object] = { + withUI { ui => + val uriQueryParameters = uriInfo.getQueryParameters(true) + val totalRecords = uriQueryParameters.getFirst("numTasks") + var isSearch = false + var searchValue: String = null + var filteredRecords = totalRecords + // The datatables client API sends a list of query parameters to the server which contain + // information like the columns to be sorted, search value typed by the user in the search + // box, pagination index etc. For more information on these query parameters, + // refer https://datatables.net/manual/server-side. + if (uriQueryParameters.getFirst("search[value]") != null && + uriQueryParameters.getFirst("search[value]").length > 0) { + isSearch = true + searchValue = uriQueryParameters.getFirst("search[value]") + } + val _tasksToShow: Seq[TaskData] = doPagination(uriQueryParameters, stageId, stageAttemptId, + isSearch, totalRecords.toInt) + val ret = new HashMap[String, Object]() + if (_tasksToShow.nonEmpty) { + // Performs server-side search based on input from user + if (isSearch) { + val filteredTaskList = filterTaskList(_tasksToShow, searchValue) + filteredRecords = filteredTaskList.length.toString + if (filteredTaskList.length > 0) { + val pageStartIndex = uriQueryParameters.getFirst("start").toInt + val pageLength = uriQueryParameters.getFirst("length").toInt + ret.put("aaData", filteredTaskList.slice( + pageStartIndex, pageStartIndex + pageLength)) + } else { + ret.put("aaData", filteredTaskList) + } + } else { + ret.put("aaData", _tasksToShow) + } + } else { + ret.put("aaData", _tasksToShow) + } + ret.put("recordsTotal", totalRecords) + ret.put("recordsFiltered", filteredRecords) + ret + } + } + + // Performs pagination on the server side + def doPagination(queryParameters: MultivaluedMap[String, String], stageId: Int, + stageAttemptId: Int, isSearch: Boolean, totalRecords: Int): Seq[TaskData] = { + var columnNameToSort = queryParameters.getFirst("columnNameToSort") + // Sorting on Logs column will default to Index column sort + if (columnNameToSort.equalsIgnoreCase("Logs")) { + columnNameToSort = "Index" + } + val isAscendingStr = queryParameters.getFirst("order[0][dir]") + var pageStartIndex = 0 + var pageLength = totalRecords + // We fetch only the desired rows upto the specified page length for all cases except when a + // search query is present, in that case, we need to fetch all the rows to perform the search + // on the entire table + if (!isSearch) { + pageStartIndex = queryParameters.getFirst("start").toInt + pageLength = queryParameters.getFirst("length").toInt + } + withUI(_.store.taskList(stageId, stageAttemptId, pageStartIndex, pageLength, + indexName(columnNameToSort), isAscendingStr.equalsIgnoreCase("asc"))) + } + + // Filters task list based on search parameter + def filterTaskList( + taskDataList: Seq[TaskData], + searchValue: String): Seq[TaskData] = { + val defaultOptionString: String = "d" + val searchValueLowerCase = searchValue.toLowerCase(Locale.ROOT) + val containsValue = (taskDataParams: Any) => taskDataParams.toString.toLowerCase( + Locale.ROOT).contains(searchValueLowerCase) + val taskMetricsContainsValue = (task: TaskData) => task.taskMetrics match { + case None => false + case Some(metrics) => + (containsValue(UIUtils.formatDuration(task.taskMetrics.get.executorDeserializeTime)) + || containsValue(UIUtils.formatDuration(task.taskMetrics.get.executorRunTime)) + || containsValue(UIUtils.formatDuration(task.taskMetrics.get.jvmGcTime)) + || containsValue(UIUtils.formatDuration(task.taskMetrics.get.resultSerializationTime)) + || containsValue(Utils.bytesToString(task.taskMetrics.get.memoryBytesSpilled)) + || containsValue(Utils.bytesToString(task.taskMetrics.get.diskBytesSpilled)) + || containsValue(Utils.bytesToString(task.taskMetrics.get.peakExecutionMemory)) + || containsValue(Utils.bytesToString(task.taskMetrics.get.inputMetrics.bytesRead)) + || containsValue(task.taskMetrics.get.inputMetrics.recordsRead) + || containsValue(Utils.bytesToString( + task.taskMetrics.get.outputMetrics.bytesWritten)) + || containsValue(task.taskMetrics.get.outputMetrics.recordsWritten) + || containsValue(UIUtils.formatDuration( + task.taskMetrics.get.shuffleReadMetrics.fetchWaitTime)) + || containsValue(Utils.bytesToString( + task.taskMetrics.get.shuffleReadMetrics.remoteBytesRead)) + || containsValue(Utils.bytesToString( + task.taskMetrics.get.shuffleReadMetrics.localBytesRead + + task.taskMetrics.get.shuffleReadMetrics.remoteBytesRead)) + || containsValue(task.taskMetrics.get.shuffleReadMetrics.recordsRead) + || containsValue(Utils.bytesToString( + task.taskMetrics.get.shuffleWriteMetrics.bytesWritten)) + || containsValue(task.taskMetrics.get.shuffleWriteMetrics.recordsWritten) + || containsValue(UIUtils.formatDuration( + task.taskMetrics.get.shuffleWriteMetrics.writeTime / 1000000))) + } + val filteredTaskDataSequence: Seq[TaskData] = taskDataList.filter(f => + (containsValue(f.taskId) || containsValue(f.index) || containsValue(f.attempt) + || containsValue(UIUtils.formatDate(f.launchTime)) + || containsValue(f.resultFetchStart.getOrElse(defaultOptionString)) + || containsValue(f.executorId) || containsValue(f.host) || containsValue(f.status) + || containsValue(f.taskLocality) || containsValue(f.speculative) + || containsValue(f.errorMessage.getOrElse(defaultOptionString)) + || taskMetricsContainsValue(f) + || containsValue(UIUtils.formatDuration(f.schedulerDelay)) + || containsValue(UIUtils.formatDuration(f.gettingResultTime)))) + filteredTaskDataSequence + } + } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala index 30afd8b769720..7b3636fdc5b47 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/api.scala +++ b/core/src/main/scala/org/apache/spark/status/api/v1/api.scala @@ -106,7 +106,8 @@ class ExecutorSummary private[spark]( val blacklistedInStages: Set[Int], @JsonSerialize(using = classOf[ExecutorMetricsJsonSerializer]) @JsonDeserialize(using = classOf[ExecutorMetricsJsonDeserializer]) - val peakMemoryMetrics: Option[ExecutorMetrics]) + val peakMemoryMetrics: Option[ExecutorMetrics], + val attributes: Map[String, String]) class MemoryMetrics private[spark]( val usedOnHeapStorageMemory: Long, @@ -133,9 +134,9 @@ private[spark] class ExecutorMetricsJsonSerializer jsonGenerator: JsonGenerator, serializerProvider: SerializerProvider): Unit = { metrics.foreach { m: ExecutorMetrics => - val metricsMap = ExecutorMetricType.values.map { metricType => - metricType.name -> m.getMetricValue(metricType) - }.toMap + val metricsMap = ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> m.getMetricValue(metric) + } jsonGenerator.writeObject(metricsMap) } } @@ -209,23 +210,36 @@ class StageData private[spark]( val numKilledTasks: Int, val numCompletedIndices: Int, - val executorRunTime: Long, - val executorCpuTime: Long, val submissionTime: Option[Date], val firstTaskLaunchedTime: Option[Date], val completionTime: Option[Date], val failureReason: Option[String], + val executorDeserializeTime: Long, + val executorDeserializeCpuTime: Long, + val executorRunTime: Long, + val executorCpuTime: Long, + val resultSize: Long, + val jvmGcTime: Long, + val resultSerializationTime: Long, + val memoryBytesSpilled: Long, + val diskBytesSpilled: Long, + val peakExecutionMemory: Long, val inputBytes: Long, val inputRecords: Long, val outputBytes: Long, val outputRecords: Long, + val shuffleRemoteBlocksFetched: Long, + val shuffleLocalBlocksFetched: Long, + val shuffleFetchWaitTime: Long, + val shuffleRemoteBytesRead: Long, + val shuffleRemoteBytesReadToDisk: Long, + val shuffleLocalBytesRead: Long, val shuffleReadBytes: Long, val shuffleReadRecords: Long, val shuffleWriteBytes: Long, + val shuffleWriteTime: Long, val shuffleWriteRecords: Long, - val memoryBytesSpilled: Long, - val diskBytesSpilled: Long, val name: String, val description: Option[String], @@ -253,7 +267,10 @@ class TaskData private[spark]( val speculative: Boolean, val accumulatorUpdates: Seq[AccumulableInfo], val errorMessage: Option[String] = None, - val taskMetrics: Option[TaskMetrics] = None) + val taskMetrics: Option[TaskMetrics] = None, + val executorLogs: Map[String, String], + val schedulerDelay: Long, + val gettingResultTime: Long) class TaskMetrics private[spark]( val executorDeserializeTime: Long, @@ -349,6 +366,7 @@ class VersionInfo private[spark]( class ApplicationEnvironmentInfo private[spark] ( val runtime: RuntimeInfo, val sparkProperties: Seq[(String, String)], + val hadoopProperties: Seq[(String, String)], val systemProperties: Seq[(String, String)], val classpathEntries: Seq[(String, String)]) diff --git a/core/src/main/scala/org/apache/spark/status/storeTypes.scala b/core/src/main/scala/org/apache/spark/status/storeTypes.scala index 646cf25880e37..eea47b3b17098 100644 --- a/core/src/main/scala/org/apache/spark/status/storeTypes.scala +++ b/core/src/main/scala/org/apache/spark/status/storeTypes.scala @@ -68,7 +68,8 @@ private[spark] class ExecutorSummaryWrapper(val info: ExecutorSummary) { */ private[spark] class JobDataWrapper( val info: JobData, - val skippedStages: Set[Int]) { + val skippedStages: Set[Int], + val sqlExecutionId: Option[Long]) { @JsonIgnore @KVIndex private def id: Int = info.jobId @@ -283,7 +284,10 @@ private[spark] class TaskDataWrapper( speculative, accumulatorUpdates, errorMessage, - metrics) + metrics, + executorLogs = null, + schedulerDelay = 0L, + gettingResultTime = 0L) } @JsonIgnore @KVIndex(TaskIndexNames.STAGE) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 0fe82ac0cedc5..c3ec1594faee5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -22,7 +22,7 @@ import java.lang.ref.{ReferenceQueue => JReferenceQueue, WeakReference} import java.nio.ByteBuffer import java.nio.channels.Channels import java.util.Collections -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.mutable import scala.collection.mutable.HashMap @@ -33,15 +33,17 @@ import scala.util.Random import scala.util.control.NonFatal import com.codahale.metrics.{MetricRegistry, MetricSet} -import com.google.common.io.CountingOutputStream +import org.apache.commons.io.IOUtils import org.apache.spark._ -import org.apache.spark.executor.{DataReadMethod, ShuffleWriteMetrics} -import org.apache.spark.internal.{config, Logging} +import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.internal.config.Network import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.metrics.source.Source import org.apache.spark.network._ -import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.client.StreamCallbackWithID import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle._ @@ -50,7 +52,7 @@ import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.serializer.{SerializerInstance, SerializerManager} -import org.apache.spark.shuffle.ShuffleManager +import org.apache.spark.shuffle.{ShuffleManager, ShuffleWriteMetricsReporter} import org.apache.spark.storage.memory._ import org.apache.spark.unsafe.Platform import org.apache.spark.util._ @@ -128,15 +130,16 @@ private[spark] class BlockManager( shuffleManager: ShuffleManager, val blockTransferService: BlockTransferService, securityManager: SecurityManager, - numUsableCores: Int) + externalShuffleClient: Option[ExternalShuffleClient]) extends BlockDataManager with BlockEvictionHandler with Logging { - private[spark] val externalShuffleServiceEnabled = - conf.get(config.SHUFFLE_SERVICE_ENABLED) - private val chunkSize = - conf.getSizeAsBytes("spark.storage.memoryMapLimitForTests", Int.MaxValue.toString).toInt + // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED)` + private[spark] val externalShuffleServiceEnabled: Boolean = externalShuffleClient.isDefined + private val remoteReadNioBufferConversion = - conf.getBoolean("spark.network.remoteReadNioBufferConversion", false) + conf.get(Network.NETWORK_REMOTE_READ_NIO_BUFFER_CONVERSION) + + private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES) val diskBlockManager = { // Only perform cleanup if an external service is not serving our shuffle files. @@ -164,20 +167,7 @@ private[spark] class BlockManager( private val maxOnHeapMemory = memoryManager.maxOnHeapStorageMemory private val maxOffHeapMemory = memoryManager.maxOffHeapStorageMemory - // Port used by the external shuffle service. In Yarn mode, this may be already be - // set through the Hadoop configuration as the server is launched in the Yarn NM. - private val externalShuffleServicePort = { - val tmpPort = Utils.getSparkOrYarnConfig(conf, config.SHUFFLE_SERVICE_PORT.key, - config.SHUFFLE_SERVICE_PORT.defaultValueString).toInt - if (tmpPort == 0) { - // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds - // an open port. But we still need to tell our spark apps the right port to use. So - // only if the yarn config has the port set to 0, we prefer the value in the spark config - conf.get(config.SHUFFLE_SERVICE_PORT.key).toInt - } else { - tmpPort - } - } + private val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf) var blockManagerId: BlockManagerId = _ @@ -187,17 +177,11 @@ private[spark] class BlockManager( // Client to read other executors' shuffle files. This is either an external service, or just the // standard BlockTransferService to directly connect to other Executors. - private[spark] val shuffleClient = if (externalShuffleServiceEnabled) { - val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores) - new ExternalShuffleClient(transConf, securityManager, - securityManager.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT)) - } else { - blockTransferService - } + private[spark] val shuffleClient = externalShuffleClient.getOrElse(blockTransferService) // Max number of failures before this block manager refreshes the block locations from the driver private val maxFailuresBeforeLocationRefresh = - conf.getInt("spark.block.failures.beforeLocationRefresh", 5) + conf.get(config.BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) private val slaveEndpoint = rpcEnv.setupEndpoint( "BlockManagerEndpoint" + BlockManager.ID_GENERATOR.next, @@ -211,7 +195,7 @@ private[spark] class BlockManager( // Field related to peer block managers that are necessary for block replication @volatile private var cachedPeers: Seq[BlockManagerId] = _ private val peerFetchLock = new Object - private var lastPeerFetchTime = 0L + private var lastPeerFetchTimeNs = 0L private var blockReplicationPolicy: BlockReplicationPolicy = _ @@ -222,6 +206,187 @@ private[spark] class BlockManager( new BlockManager.RemoteBlockDownloadFileManager(this) private val maxRemoteBlockToMem = conf.get(config.MAX_REMOTE_BLOCK_SIZE_FETCH_TO_MEM) + /** + * Abstraction for storing blocks from bytes, whether they start in memory or on disk. + * + * @param blockSize the decrypted size of the block + */ + private abstract class BlockStoreUpdater[T]( + blockSize: Long, + blockId: BlockId, + level: StorageLevel, + classTag: ClassTag[T], + tellMaster: Boolean, + keepReadLock: Boolean) { + + /** + * Reads the block content into the memory. If the update of the block store is based on a + * temporary file this could lead to loading the whole file into a ChunkedByteBuffer. + */ + protected def readToByteBuffer(): ChunkedByteBuffer + + protected def blockData(): BlockData + + protected def saveToDiskStore(): Unit + + private def saveDeserializedValuesToMemoryStore(inputStream: InputStream): Boolean = { + try { + val values = serializerManager.dataDeserializeStream(blockId, inputStream)(classTag) + memoryStore.putIteratorAsValues(blockId, values, classTag) match { + case Right(_) => true + case Left(iter) => + // If putting deserialized values in memory failed, we will put the bytes directly + // to disk, so we don't need this iterator and can close it to free resources + // earlier. + iter.close() + false + } + } finally { + IOUtils.closeQuietly(inputStream) + } + } + + private def saveSerializedValuesToMemoryStore(bytes: ChunkedByteBuffer): Boolean = { + val memoryMode = level.memoryMode + memoryStore.putBytes(blockId, blockSize, memoryMode, () => { + if (memoryMode == MemoryMode.OFF_HEAP && bytes.chunks.exists(!_.isDirect)) { + bytes.copy(Platform.allocateDirectBuffer) + } else { + bytes + } + }) + } + + /** + * Put the given data according to the given level in one of the block stores, replicating + * the values if necessary. + * + * If the block already exists, this method will not overwrite it. + * + * If keepReadLock is true, this method will hold the read lock when it returns (even if the + * block already exists). If false, this method will hold no locks when it returns. + * + * @return true if the block was already present or if the put succeeded, false otherwise. + */ + def save(): Boolean = { + doPut(blockId, level, classTag, tellMaster, keepReadLock) { info => + val startTimeNs = System.nanoTime() + + // Since we're storing bytes, initiate the replication before storing them locally. + // This is faster as data is already serialized and ready to send. + val replicationFuture = if (level.replication > 1) { + Future { + // This is a blocking action and should run in futureExecutionContext which is a cached + // thread pool. + replicate(blockId, blockData(), level, classTag) + }(futureExecutionContext) + } else { + null + } + if (level.useMemory) { + // Put it in memory first, even if it also has useDisk set to true; + // We will drop it to disk later if the memory store can't hold it. + val putSucceeded = if (level.deserialized) { + saveDeserializedValuesToMemoryStore(blockData().toInputStream()) + } else { + saveSerializedValuesToMemoryStore(readToByteBuffer()) + } + if (!putSucceeded && level.useDisk) { + logWarning(s"Persisting block $blockId to disk instead.") + saveToDiskStore() + } + } else if (level.useDisk) { + saveToDiskStore() + } + val putBlockStatus = getCurrentBlockStatus(blockId, info) + val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid + if (blockWasSuccessfullyStored) { + // Now that the block is in either the memory or disk store, + // tell the master about it. + info.size = blockSize + if (tellMaster && info.tellMaster) { + reportBlockStatus(blockId, putBlockStatus) + } + addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) + } + logDebug(s"Put block ${blockId} locally took ${Utils.getUsedTimeNs(startTimeNs)}") + if (level.replication > 1) { + // Wait for asynchronous replication to finish + try { + ThreadUtils.awaitReady(replicationFuture, Duration.Inf) + } catch { + case NonFatal(t) => + throw new Exception("Error occurred while waiting for replication to finish", t) + } + } + if (blockWasSuccessfullyStored) { + None + } else { + Some(blockSize) + } + }.isEmpty + } + } + + /** + * Helper for storing a block from bytes already in memory. + * '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing + * so may corrupt or change the data stored by the `BlockManager`. + */ + private case class ByteBufferBlockStoreUpdater[T]( + blockId: BlockId, + level: StorageLevel, + classTag: ClassTag[T], + bytes: ChunkedByteBuffer, + tellMaster: Boolean = true, + keepReadLock: Boolean = false) + extends BlockStoreUpdater[T](bytes.size, blockId, level, classTag, tellMaster, keepReadLock) { + + override def readToByteBuffer(): ChunkedByteBuffer = bytes + + /** + * The ByteBufferBlockData wrapper is not disposed of to avoid releasing buffers that are + * owned by the caller. + */ + override def blockData(): BlockData = new ByteBufferBlockData(bytes, false) + + override def saveToDiskStore(): Unit = diskStore.putBytes(blockId, bytes) + + } + + /** + * Helper for storing a block based from bytes already in a local temp file. + */ + private case class TempFileBasedBlockStoreUpdater[T]( + blockId: BlockId, + level: StorageLevel, + classTag: ClassTag[T], + tmpFile: File, + blockSize: Long, + tellMaster: Boolean = true, + keepReadLock: Boolean = false) + extends BlockStoreUpdater[T](blockSize, blockId, level, classTag, tellMaster, keepReadLock) { + + override def readToByteBuffer(): ChunkedByteBuffer = { + val allocator = level.memoryMode match { + case MemoryMode.ON_HEAP => ByteBuffer.allocate _ + case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ + } + blockData().toChunkedByteBuffer(allocator) + } + + override def blockData(): BlockData = diskStore.getBytes(tmpFile, blockSize) + + override def saveToDiskStore(): Unit = diskStore.moveFileToBlock(tmpFile, blockSize, blockId) + + override def save(): Boolean = { + val res = super.save() + tmpFile.delete() + res + } + + } + /** * Initializes the BlockManager with the given appId. This is not performed in the constructor as * the appId may not be known at BlockManager instantiation time (in particular for the driver, @@ -233,13 +398,13 @@ private[spark] class BlockManager( */ def initialize(appId: String): Unit = { blockTransferService.init(this) - shuffleClient.init(appId) - + externalShuffleClient.foreach { shuffleClient => + shuffleClient.init(appId) + } blockReplicationPolicy = { - val priorityClass = conf.get( - "spark.storage.replication.policy", classOf[RandomBlockReplicationPolicy].getName) + val priorityClass = conf.get(config.STORAGE_REPLICATION_POLICY) val clazz = Utils.classForName(priorityClass) - val ret = clazz.newInstance.asInstanceOf[BlockReplicationPolicy] + val ret = clazz.getConstructor().newInstance().asInstanceOf[BlockReplicationPolicy] logInfo(s"Using $priorityClass for block replication policy") ret } @@ -249,6 +414,7 @@ private[spark] class BlockManager( val idFromMaster = master.registerBlockManager( id, + diskBlockManager.localDirsString, maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint) @@ -283,7 +449,7 @@ private[spark] class BlockManager( private def registerWithExternalShuffleServer() { logInfo("Registering executor with local external shuffle service.") val shuffleConfig = new ExecutorShuffleInfo( - diskBlockManager.localDirs.map(_.toString), + diskBlockManager.localDirsString, diskBlockManager.subDirsPerLocalDir, shuffleManager.getClass.getName) @@ -338,7 +504,8 @@ private[spark] class BlockManager( def reregister(): Unit = { // TODO: We might need to rate limit re-registering. logInfo(s"BlockManager $blockManagerId re-registering with master") - master.registerBlockManager(blockManagerId, maxOnHeapMemory, maxOffHeapMemory, slaveEndpoint) + master.registerBlockManager(blockManagerId, diskBlockManager.localDirsString, maxOnHeapMemory, + maxOffHeapMemory, slaveEndpoint) reportAllBlocks() } @@ -414,10 +581,7 @@ private[spark] class BlockManager( blockId: BlockId, level: StorageLevel, classTag: ClassTag[_]): StreamCallbackWithID = { - // TODO if we're going to only put the data in the disk store, we should just write it directly - // to the final location, but that would require a deeper refactor of this code. So instead - // we just write to a temp file, and call putBytes on the data in that file. - val tmpFile = diskBlockManager.createTempLocalBlock()._2 + val (_, tmpFile) = diskBlockManager.createTempLocalBlock() val channel = new CountingWritableChannel( Channels.newChannel(serializerManager.wrapForEncryption(new FileOutputStream(tmpFile)))) logTrace(s"Streaming block $blockId to tmp file $tmpFile") @@ -433,28 +597,11 @@ private[spark] class BlockManager( override def onComplete(streamId: String): Unit = { logTrace(s"Done receiving block $blockId, now putting into local blockManager") - // Read the contents of the downloaded file as a buffer to put into the blockManager. // Note this is all happening inside the netty thread as soon as it reads the end of the // stream. channel.close() - // TODO SPARK-25035 Even if we're only going to write the data to disk after this, we end up - // using a lot of memory here. We'll read the whole file into a regular - // byte buffer and OOM. We could at least read the tmp file as a stream. - val buffer = securityManager.getIOEncryptionKey() match { - case Some(key) => - // we need to pass in the size of the unencrypted block - val blockSize = channel.getCount - val allocator = level.memoryMode match { - case MemoryMode.ON_HEAP => ByteBuffer.allocate _ - case MemoryMode.OFF_HEAP => Platform.allocateDirectBuffer _ - } - new EncryptedBlockData(tmpFile, blockSize, conf, key).toChunkedByteBuffer(allocator) - - case None => - ChunkedByteBuffer.fromFile(tmpFile, conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS).toInt) - } - putBytes(blockId, buffer, level)(classTag) - tmpFile.delete() + val blockSize = channel.getCount + TempFileBasedBlockStoreUpdater(blockId, level, classTag, tmpFile, blockSize).save() } override def onFailure(streamId: String, cause: Throwable): Unit = { @@ -560,9 +707,9 @@ private[spark] class BlockManager( * Get locations of an array of blocks. */ private def getLocationBlockIds(blockIds: Array[BlockId]): Array[Seq[BlockManagerId]] = { - val startTimeMs = System.currentTimeMillis + val startTimeNs = System.nanoTime() val locations = master.getLocations(blockIds).toArray - logDebug("Got multiple block location in %s".format(Utils.getUsedTimeMs(startTimeMs))) + logDebug(s"Got multiple block location in ${Utils.getUsedTimeNs(startTimeNs)}") locations } @@ -589,7 +736,7 @@ private[spark] class BlockManager( case Some(info) => val level = info.level logDebug(s"Level for block $blockId is $level") - val taskAttemptId = Option(TaskContext.get()).map(_.taskAttemptId()) + val taskContext = Option(TaskContext.get()) if (level.useMemory && memoryStore.contains(blockId)) { val iter: Iterator[Any] = if (level.deserialized) { memoryStore.getValues(blockId).get @@ -601,7 +748,7 @@ private[spark] class BlockManager( // from a different thread which does not have TaskContext set; see SPARK-18406 for // discussion. val ci = CompletionIterator[Any, Iterator[Any]](iter, { - releaseLock(blockId, taskAttemptId) + releaseLock(blockId, taskContext) }) Some(new BlockResult(ci, DataReadMethod.Memory, info.size)) } else if (level.useDisk && diskStore.contains(blockId)) { @@ -620,7 +767,7 @@ private[spark] class BlockManager( } } val ci = CompletionIterator[Any, Iterator[Any]](iterToReturn, { - releaseLockAndDispose(blockId, diskData, taskAttemptId) + releaseLockAndDispose(blockId, diskData, taskContext) }) Some(new BlockResult(ci, DataReadMethod.Disk, info.size)) } else { @@ -634,18 +781,8 @@ private[spark] class BlockManager( */ def getLocalBytes(blockId: BlockId): Option[BlockData] = { logDebug(s"Getting local block $blockId as bytes") - // As an optimization for map output fetches, if the block is for a shuffle, return it - // without acquiring a lock; the disk store never deletes (recent) items so this should work - if (blockId.isShuffle) { - val shuffleBlockResolver = shuffleManager.shuffleBlockResolver - // TODO: This should gracefully handle case where local block is not available. Currently - // downstream code will throw an exception. - val buf = new ChunkedByteBuffer( - shuffleBlockResolver.getBlockData(blockId.asInstanceOf[ShuffleBlockId]).nioByteBuffer()) - Some(new ByteBufferBlockData(buf, true)) - } else { - blockInfoManager.lockForReading(blockId).map { info => doGetLocalBytes(blockId, info) } - } + assert(!blockId.isShuffle, s"Unexpected ShuffleBlockId $blockId") + blockInfoManager.lockForReading(blockId).map { info => doGetLocalBytes(blockId, info) } } /** @@ -693,53 +830,101 @@ private[spark] class BlockManager( * * This does not acquire a lock on this block in this JVM. */ - private def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { + private[spark] def getRemoteValues[T: ClassTag](blockId: BlockId): Option[BlockResult] = { val ct = implicitly[ClassTag[T]] - getRemoteBytes(blockId).map { data => + getRemoteBlock(blockId, (data: ManagedBuffer) => { val values = - serializerManager.dataDeserializeStream(blockId, data.toInputStream(dispose = true))(ct) + serializerManager.dataDeserializeStream(blockId, data.createInputStream())(ct) new BlockResult(values, DataReadMethod.Network, data.size) + }) + } + + /** + * Get the remote block and transform it to the provided data type. + * + * If the block is persisted to the disk and stored at an executor running on the same host then + * first it is tried to be accessed using the local directories of the other executor directly. + * If the file is successfully identified then tried to be transformed by the provided + * transformation function which expected to open the file. If there is any exception during this + * transformation then block access falls back to fetching it from the remote executor via the + * network. + * + * @param blockId identifies the block to get + * @param bufferTransformer this transformer expected to open the file if the block is backed by a + * file by this it is guaranteed the whole content can be loaded + * @tparam T result type + * @return + */ + private[spark] def getRemoteBlock[T]( + blockId: BlockId, + bufferTransformer: ManagedBuffer => T): Option[T] = { + logDebug(s"Getting remote block $blockId") + require(blockId != null, "BlockId is null") + + // Because all the remote blocks are registered in driver, it is not necessary to ask + // all the slave executors to get block status. + val locationsAndStatusOption = master.getLocationsAndStatus(blockId, blockManagerId.host) + if (locationsAndStatusOption.isEmpty) { + logDebug(s"Block $blockId is unknown by block manager master") + None + } else { + val locationsAndStatus = locationsAndStatusOption.get + val blockSize = locationsAndStatus.status.diskSize.max(locationsAndStatus.status.memSize) + + locationsAndStatus.localDirs.flatMap { localDirs => + val blockDataOption = + readDiskBlockFromSameHostExecutor(blockId, localDirs, locationsAndStatus.status.diskSize) + val res = blockDataOption.flatMap { blockData => + try { + Some(bufferTransformer(blockData)) + } catch { + case NonFatal(e) => + logDebug("Block from the same host executor cannot be opened: ", e) + None + } + } + logInfo(s"Read $blockId from the disk of a same host executor is " + + (if (res.isDefined) "successful." else "failed.")) + res + }.orElse { + fetchRemoteManagedBuffer(blockId, blockSize, locationsAndStatus).map(bufferTransformer) + } } } + private def preferExecutors(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { + val (executors, shuffleServers) = locations.partition(_.port != externalShuffleServicePort) + executors ++ shuffleServers + } + /** * Return a list of locations for the given block, prioritizing the local machine since * multiple block managers can share the same host, followed by hosts on the same rack. + * + * Within each of the above listed groups (same host, same rack and others) executors are + * preferred over the external shuffle service. */ - private def sortLocations(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { + private[spark] def sortLocations(locations: Seq[BlockManagerId]): Seq[BlockManagerId] = { val locs = Random.shuffle(locations) - val (preferredLocs, otherLocs) = locs.partition { loc => blockManagerId.host == loc.host } - blockManagerId.topologyInfo match { - case None => preferredLocs ++ otherLocs + val (preferredLocs, otherLocs) = locs.partition(_.host == blockManagerId.host) + val orderedParts = blockManagerId.topologyInfo match { + case None => Seq(preferredLocs, otherLocs) case Some(_) => val (sameRackLocs, differentRackLocs) = otherLocs.partition { loc => blockManagerId.topologyInfo == loc.topologyInfo } - preferredLocs ++ sameRackLocs ++ differentRackLocs + Seq(preferredLocs, sameRackLocs, differentRackLocs) } + orderedParts.map(preferExecutors).reduce(_ ++ _) } /** - * Get block from remote block managers as serialized bytes. + * Fetch the block from remote block managers as a ManagedBuffer. */ - def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { - // TODO if we change this method to return the ManagedBuffer, then getRemoteValues - // could just use the inputStream on the temp file, rather than reading the file into memory. - // Until then, replication can cause the process to use too much memory and get killed - // even though we've read the data to disk. - logDebug(s"Getting remote block $blockId") - require(blockId != null, "BlockId is null") - var runningFailureCount = 0 - var totalFailureCount = 0 - - // Because all the remote blocks are registered in driver, it is not necessary to ask - // all the slave executors to get block status. - val locationsAndStatus = master.getLocationsAndStatus(blockId) - val blockSize = locationsAndStatus.map { b => - b.status.diskSize.max(b.status.memSize) - }.getOrElse(0L) - val blockLocations = locationsAndStatus.map(_.locations).getOrElse(Seq.empty) - + private def fetchRemoteManagedBuffer( + blockId: BlockId, + blockSize: Long, + locationsAndStatus: BlockManagerMessages.BlockLocationsAndStatus): Option[ManagedBuffer] = { // If the block size is above the threshold, we should pass our FileManger to // BlockTransferService, which will leverage it to spill the block; if not, then passed-in // null value means the block will be persisted in memory. @@ -748,16 +933,21 @@ private[spark] class BlockManager( } else { null } - - val locations = sortLocations(blockLocations) + var runningFailureCount = 0 + var totalFailureCount = 0 + val locations = sortLocations(locationsAndStatus.locations) val maxFetchFailures = locations.size var locationIterator = locations.iterator while (locationIterator.hasNext) { val loc = locationIterator.next() logDebug(s"Getting remote block $blockId from $loc") val data = try { - blockTransferService.fetchBlockSync( - loc.host, loc.port, loc.executorId, blockId.toString, tempFileManager) + val buf = blockTransferService.fetchBlockSync(loc.host, loc.port, loc.executorId, + blockId.toString, tempFileManager) + if (blockSize > 0 && buf.size() == 0) { + throw new IllegalStateException("Empty buffer received for non empty block") + } + buf } catch { case NonFatal(e) => runningFailureCount += 1 @@ -791,14 +981,13 @@ private[spark] class BlockManager( } if (data != null) { - // SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to - // ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if - // new path is stable. - if (remoteReadNioBufferConversion) { - return Some(new ChunkedByteBuffer(data.nioByteBuffer())) - } else { - return Some(ChunkedByteBuffer.fromManagedBuffer(data, chunkSize)) - } + // If the ManagedBuffer is a BlockManagerManagedBuffer, the disposal of the + // byte buffers backing it may need to be handled after reading the bytes. + // In this case, since we just fetched the bytes remotely, we do not have + // a BlockManagerManagedBuffer. The assert here is to ensure that this holds + // true (or the disposal is handled). + assert(!data.isInstanceOf[BlockManagerManagedBuffer]) + return Some(data) } logDebug(s"The value of block $blockId is null") } @@ -806,6 +995,48 @@ private[spark] class BlockManager( None } + /** + * Reads the block from the local directories of another executor which runs on the same host. + */ + private[spark] def readDiskBlockFromSameHostExecutor( + blockId: BlockId, + localDirs: Array[String], + blockSize: Long): Option[ManagedBuffer] = { + val file = ExecutorDiskUtils.getFile(localDirs, subDirsPerLocalDir, blockId.name) + if (file.exists()) { + val mangedBuffer = securityManager.getIOEncryptionKey() match { + case Some(key) => + // Encrypted blocks cannot be memory mapped; return a special object that does decryption + // and provides InputStream / FileRegion implementations for reading the data. + new EncryptedManagedBuffer( + new EncryptedBlockData(file, blockSize, conf, key)) + + case _ => + val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle") + new FileSegmentManagedBuffer(transportConf, file, 0, file.length) + } + Some(mangedBuffer) + } else { + None + } + } + + /** + * Get block from remote block managers as serialized bytes. + */ + def getRemoteBytes(blockId: BlockId): Option[ChunkedByteBuffer] = { + getRemoteBlock(blockId, (data: ManagedBuffer) => { + // SPARK-24307 undocumented "escape-hatch" in case there are any issues in converting to + // ChunkedByteBuffer, to go back to old code-path. Can be removed post Spark 2.4 if + // new path is stable. + if (remoteReadNioBufferConversion) { + new ChunkedByteBuffer(data.nioByteBuffer()) + } else { + ChunkedByteBuffer.fromManagedBuffer(data) + } + }) + } + /** * Get a block from the block manager (either local or remote). * @@ -835,13 +1066,20 @@ private[spark] class BlockManager( } /** - * Release a lock on the given block with explicit TID. - * The param `taskAttemptId` should be passed in case we can't get the correct TID from - * TaskContext, for example, the input iterator of a cached RDD iterates to the end in a child + * Release a lock on the given block with explicit TaskContext. + * The param `taskContext` should be passed in case we can't get the correct TaskContext, + * for example, the input iterator of a cached RDD iterates to the end in a child * thread. */ - def releaseLock(blockId: BlockId, taskAttemptId: Option[Long] = None): Unit = { - blockInfoManager.unlock(blockId, taskAttemptId) + def releaseLock(blockId: BlockId, taskContext: Option[TaskContext] = None): Unit = { + val taskAttemptId = taskContext.map(_.taskAttemptId()) + // SPARK-27666. When a task completes, Spark automatically releases all the blocks locked + // by this task. We should not release any locks for a task that is already completed. + if (taskContext.isDefined && taskContext.get.isCompleted) { + logWarning(s"Task ${taskAttemptId.get} already completed, not releasing lock for $blockId") + } else { + blockInfoManager.unlock(blockId, taskAttemptId) + } } /** @@ -934,8 +1172,8 @@ private[spark] class BlockManager( file: File, serializerInstance: SerializerInstance, bufferSize: Int, - writeMetrics: ShuffleWriteMetrics): DiskBlockObjectWriter = { - val syncWrites = conf.getBoolean("spark.shuffle.sync", false) + writeMetrics: ShuffleWriteMetricsReporter): DiskBlockObjectWriter = { + val syncWrites = conf.get(config.SHUFFLE_SYNC) new DiskBlockObjectWriter(file, serializerManager, serializerInstance, bufferSize, syncWrites, writeMetrics, blockId) } @@ -954,111 +1192,14 @@ private[spark] class BlockManager( level: StorageLevel, tellMaster: Boolean = true): Boolean = { require(bytes != null, "Bytes is null") - doPutBytes(blockId, bytes, level, implicitly[ClassTag[T]], tellMaster) - } - - /** - * Put the given bytes according to the given level in one of the block stores, replicating - * the values if necessary. - * - * If the block already exists, this method will not overwrite it. - * - * '''Important!''' Callers must not mutate or release the data buffer underlying `bytes`. Doing - * so may corrupt or change the data stored by the `BlockManager`. - * - * @param keepReadLock if true, this method will hold the read lock when it returns (even if the - * block already exists). If false, this method will hold no locks when it - * returns. - * @return true if the block was already present or if the put succeeded, false otherwise. - */ - private def doPutBytes[T]( - blockId: BlockId, - bytes: ChunkedByteBuffer, - level: StorageLevel, - classTag: ClassTag[T], - tellMaster: Boolean = true, - keepReadLock: Boolean = false): Boolean = { - doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => - val startTimeMs = System.currentTimeMillis - // Since we're storing bytes, initiate the replication before storing them locally. - // This is faster as data is already serialized and ready to send. - val replicationFuture = if (level.replication > 1) { - Future { - // This is a blocking action and should run in futureExecutionContext which is a cached - // thread pool. The ByteBufferBlockData wrapper is not disposed of to avoid releasing - // buffers that are owned by the caller. - replicate(blockId, new ByteBufferBlockData(bytes, false), level, classTag) - }(futureExecutionContext) - } else { - null - } - - val size = bytes.size - - if (level.useMemory) { - // Put it in memory first, even if it also has useDisk set to true; - // We will drop it to disk later if the memory store can't hold it. - val putSucceeded = if (level.deserialized) { - val values = - serializerManager.dataDeserializeStream(blockId, bytes.toInputStream())(classTag) - memoryStore.putIteratorAsValues(blockId, values, classTag) match { - case Right(_) => true - case Left(iter) => - // If putting deserialized values in memory failed, we will put the bytes directly to - // disk, so we don't need this iterator and can close it to free resources earlier. - iter.close() - false - } - } else { - val memoryMode = level.memoryMode - memoryStore.putBytes(blockId, size, memoryMode, () => { - if (memoryMode == MemoryMode.OFF_HEAP && - bytes.chunks.exists(buffer => !buffer.isDirect)) { - bytes.copy(Platform.allocateDirectBuffer) - } else { - bytes - } - }) - } - if (!putSucceeded && level.useDisk) { - logWarning(s"Persisting block $blockId to disk instead.") - diskStore.putBytes(blockId, bytes) - } - } else if (level.useDisk) { - diskStore.putBytes(blockId, bytes) - } - - val putBlockStatus = getCurrentBlockStatus(blockId, info) - val blockWasSuccessfullyStored = putBlockStatus.storageLevel.isValid - if (blockWasSuccessfullyStored) { - // Now that the block is in either the memory or disk store, - // tell the master about it. - info.size = size - if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, putBlockStatus) - } - addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) - } - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) - if (level.replication > 1) { - // Wait for asynchronous replication to finish - try { - ThreadUtils.awaitReady(replicationFuture, Duration.Inf) - } catch { - case NonFatal(t) => - throw new Exception("Error occurred while waiting for replication to finish", t) - } - } - if (blockWasSuccessfullyStored) { - None - } else { - Some(bytes) - } - }.isEmpty + val blockStoreUpdater = + ByteBufferBlockStoreUpdater(blockId, level, implicitly[ClassTag[T]], bytes, tellMaster) + blockStoreUpdater.save() } /** - * Helper method used to abstract common code from [[doPutBytes()]] and [[doPutIterator()]]. + * Helper method used to abstract common code from [[BlockStoreUpdater.save()]] + * and [[doPutIterator()]]. * * @param putBody a function which attempts the actual put() and returns None on success * or Some on failure. @@ -1087,7 +1228,7 @@ private[spark] class BlockManager( } } - val startTimeMs = System.currentTimeMillis + val startTimeNs = System.nanoTime() var exceptionWasThrown: Boolean = true val result: Option[T] = try { val res = putBody(putBlockInfo) @@ -1126,12 +1267,11 @@ private[spark] class BlockManager( addUpdatedBlockStatusToTaskMetrics(blockId, BlockStatus.empty) } } + val usedTimeMs = Utils.getUsedTimeNs(startTimeNs) if (level.replication > 1) { - logDebug("Putting block %s with replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + logDebug(s"Putting block ${blockId} with replication took $usedTimeMs") } else { - logDebug("Putting block %s without replication took %s" - .format(blockId, Utils.getUsedTimeMs(startTimeMs))) + logDebug(s"Putting block ${blockId} without replication took ${usedTimeMs}") } result } @@ -1156,7 +1296,7 @@ private[spark] class BlockManager( tellMaster: Boolean = true, keepReadLock: Boolean = false): Option[PartiallyUnrolledIterator[T]] = { doPut(blockId, level, classTag, tellMaster = tellMaster, keepReadLock = keepReadLock) { info => - val startTimeMs = System.currentTimeMillis + val startTimeNs = System.nanoTime() var iteratorFromFailedMemoryStorePut: Option[PartiallyUnrolledIterator[T]] = None // Size of the block in bytes var size = 0L @@ -1216,9 +1356,9 @@ private[spark] class BlockManager( reportBlockStatus(blockId, putBlockStatus) } addUpdatedBlockStatusToTaskMetrics(blockId, putBlockStatus) - logDebug("Put block %s locally took %s".format(blockId, Utils.getUsedTimeMs(startTimeMs))) + logDebug(s"Put block $blockId locally took ${Utils.getUsedTimeNs(startTimeNs)}") if (level.replication > 1) { - val remoteStartTime = System.currentTimeMillis + val remoteStartTimeNs = System.nanoTime() val bytesToReplicate = doGetLocalBytes(blockId, info) // [SPARK-16550] Erase the typed classTag when using default serialization, since // NettyBlockRpcServer crashes when deserializing repl-defined classes. @@ -1233,8 +1373,7 @@ private[spark] class BlockManager( } finally { bytesToReplicate.dispose() } - logDebug("Put block %s remotely took %s" - .format(blockId, Utils.getUsedTimeMs(remoteStartTime))) + logDebug(s"Put block $blockId remotely took ${Utils.getUsedTimeNs(remoteStartTimeNs)}") } } assert(blockWasSuccessfullyStored == iteratorFromFailedMemoryStorePut.isEmpty) @@ -1331,11 +1470,12 @@ private[spark] class BlockManager( */ private def getPeers(forceFetch: Boolean): Seq[BlockManagerId] = { peerFetchLock.synchronized { - val cachedPeersTtl = conf.getInt("spark.storage.cachedPeersTtl", 60 * 1000) // milliseconds - val timeout = System.currentTimeMillis - lastPeerFetchTime > cachedPeersTtl + val cachedPeersTtl = conf.get(config.STORAGE_CACHED_PEERS_TTL) // milliseconds + val diff = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - lastPeerFetchTimeNs) + val timeout = diff > cachedPeersTtl if (cachedPeers == null || forceFetch || timeout) { cachedPeers = master.getPeers(blockManagerId).sortBy(_.hashCode) - lastPeerFetchTime = System.currentTimeMillis + lastPeerFetchTimeNs = System.nanoTime() logDebug("Fetched peers from master: " + cachedPeers.mkString("[", ",", "]")) } cachedPeers @@ -1385,7 +1525,7 @@ private[spark] class BlockManager( classTag: ClassTag[_], existingReplicas: Set[BlockManagerId] = Set.empty): Unit = { - val maxReplicationFailures = conf.getInt("spark.storage.maxReplicationFailures", 1) + val maxReplicationFailures = conf.get(config.STORAGE_MAX_REPLICATION_FAILURE) val tLevel = StorageLevel( useDisk = level.useDisk, useMemory = level.useMemory, @@ -1608,8 +1748,8 @@ private[spark] class BlockManager( def releaseLockAndDispose( blockId: BlockId, data: BlockData, - taskAttemptId: Option[Long] = None): Unit = { - releaseLock(blockId, taskAttemptId) + taskContext: Option[TaskContext] = None): Unit = { + releaseLock(blockId, taskContext) data.dispose() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index d4a59c33b974c..d188bdd912e5e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -65,8 +65,7 @@ class BlockManagerId private ( def topologyInfo: Option[String] = topologyInfo_ def isDriver: Boolean = { - executorId == SparkContext.DRIVER_IDENTIFIER || - executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER + executorId == SparkContext.DRIVER_IDENTIFIER } override def writeExternal(out: ObjectOutput): Unit = Utils.tryOrIOException { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index d24421b962774..9d13fedfb0c58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -56,13 +56,14 @@ class BlockManagerMaster( * updated BlockManagerId fleshed out with this information. */ def registerBlockManager( - blockManagerId: BlockManagerId, + id: BlockManagerId, + localDirs: Array[String], maxOnHeapMemSize: Long, maxOffHeapMemSize: Long, slaveEndpoint: RpcEndpointRef): BlockManagerId = { - logInfo(s"Registering BlockManager $blockManagerId") + logInfo(s"Registering BlockManager $id") val updatedId = driverEndpoint.askSync[BlockManagerId]( - RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) logInfo(s"Registered BlockManager $updatedId") updatedId } @@ -85,9 +86,11 @@ class BlockManagerMaster( } /** Get locations as well as status of the blockId from the driver */ - def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = { + def getLocationsAndStatus( + blockId: BlockId, + requesterHost: String): Option[BlockLocationsAndStatus] = { driverEndpoint.askSync[Option[BlockLocationsAndStatus]]( - GetLocationsAndStatus(blockId)) + GetLocationsAndStatus(blockId, requesterHost)) } /** Get locations of multiple blockIds from the driver */ @@ -222,14 +225,6 @@ class BlockManagerMaster( timeout.awaitResult(future) } - /** - * Find out if the executor has cached blocks. This method does not consider broadcast blocks, - * since they are not reported the master. - */ - def hasCachedBlocks(executorId: String): Boolean = { - driverEndpoint.askSync[Boolean](HasCachedBlocks(executorId)) - } - /** Stop the driver endpoint, called only on the Spark driver node */ def stop() { if (driverEndpoint != null && isDriver) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala index f984cf76e3463..040fed299ffca 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterEndpoint.scala @@ -19,6 +19,7 @@ package org.apache.spark.storage import java.io.IOException import java.util.{HashMap => JHashMap} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable @@ -27,11 +28,12 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.network.shuffle.ExternalShuffleClient import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ -import org.apache.spark.util.{ThreadUtils, Utils} +import org.apache.spark.util.{RpcUtils, ThreadUtils, Utils} /** * BlockManagerMasterEndpoint is an [[ThreadSafeRpcEndpoint]] on the master node to track statuses @@ -42,12 +44,17 @@ class BlockManagerMasterEndpoint( override val rpcEnv: RpcEnv, val isLocal: Boolean, conf: SparkConf, - listenerBus: LiveListenerBus) + listenerBus: LiveListenerBus, + externalShuffleClient: Option[ExternalShuffleClient]) extends ThreadSafeRpcEndpoint with Logging { // Mapping from block manager id to the block manager's information. private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] + // Mapping from external shuffle service block manager id to the block statuses. + private val blockStatusByShuffleService = + new mutable.HashMap[BlockManagerId, JHashMap[BlockId, BlockStatus]] + // Mapping from executor ID to block manager ID. private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] @@ -60,7 +67,7 @@ class BlockManagerMasterEndpoint( private val topologyMapper = { val topologyMapperClassName = conf.get( - "spark.storage.replication.topologyMapper", classOf[DefaultTopologyMapper].getName) + config.STORAGE_REPLICATION_TOPOLOGY_MAPPER) val clazz = Utils.classForName(topologyMapperClassName) val mapper = clazz.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[TopologyMapper] @@ -68,13 +75,19 @@ class BlockManagerMasterEndpoint( mapper } - val proactivelyReplicate = conf.get("spark.storage.replication.proactive", "false").toBoolean + val proactivelyReplicate = conf.get(config.STORAGE_REPLICATION_PROACTIVE) + + val defaultRpcTimeout = RpcUtils.askRpcTimeout(conf) logInfo("BlockManagerMasterEndpoint up") + // same as `conf.get(config.SHUFFLE_SERVICE_ENABLED) + // && conf.get(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED)` + private val externalShuffleServiceRddFetchEnabled: Boolean = externalShuffleClient.isDefined + private val externalShuffleServicePort: Int = StorageUtils.externalShuffleServicePort(conf) override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { - case RegisterBlockManager(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => - context.reply(register(blockManagerId, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) + case RegisterBlockManager(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) => + context.reply(register(id, localDirs, maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint)) case _updateBlockInfo @ UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => @@ -84,8 +97,8 @@ class BlockManagerMasterEndpoint( case GetLocations(blockId) => context.reply(getLocations(blockId)) - case GetLocationsAndStatus(blockId) => - context.reply(getLocationsAndStatus(blockId)) + case GetLocationsAndStatus(blockId, requesterHost) => + context.reply(getLocationsAndStatus(blockId, requesterHost)) case GetLocationsMultipleBlockIds(blockIds) => context.reply(getLocationsMultipleBlockIds(blockIds)) @@ -105,6 +118,9 @@ class BlockManagerMasterEndpoint( case GetBlockStatus(blockId, askSlaves) => context.reply(blockStatus(blockId, askSlaves)) + case IsExecutorAlive(executorId) => + context.reply(blockManagerIdByExecutor.contains(executorId)) + case GetMatchingBlockIds(filter, askSlaves) => context.reply(getMatchingBlockIds(filter, askSlaves)) @@ -131,47 +147,68 @@ class BlockManagerMasterEndpoint( case BlockManagerHeartbeat(blockManagerId) => context.reply(heartbeatReceived(blockManagerId)) - - case HasCachedBlocks(executorId) => - blockManagerIdByExecutor.get(executorId) match { - case Some(bm) => - if (blockManagerInfo.contains(bm)) { - val bmInfo = blockManagerInfo(bm) - context.reply(bmInfo.cachedBlocks.nonEmpty) - } else { - context.reply(false) - } - case None => context.reply(false) - } } private def removeRdd(rddId: Int): Future[Seq[Int]] = { // First remove the metadata for the given RDD, and then asynchronously remove the blocks // from the slaves. + // The message sent to the slaves to remove the RDD + val removeMsg = RemoveRdd(rddId) + // Find all blocks for the given RDD, remove the block from both blockLocations and - // the blockManagerInfo that is tracking the blocks. + // the blockManagerInfo that is tracking the blocks and create the futures which asynchronously + // remove the blocks from slaves and gives back the number of removed blocks val blocks = blockLocations.asScala.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) + val blocksToDeleteByShuffleService = + new mutable.HashMap[BlockManagerId, mutable.HashSet[RDDBlockId]] + blocks.foreach { blockId => - val bms: mutable.HashSet[BlockManagerId] = blockLocations.get(blockId) - bms.foreach(bm => blockManagerInfo.get(bm).foreach(_.removeBlock(blockId))) - blockLocations.remove(blockId) + val bms: mutable.HashSet[BlockManagerId] = blockLocations.remove(blockId) + + val (bmIdsExtShuffle, bmIdsExecutor) = bms.partition(_.port == externalShuffleServicePort) + val liveExecutorsForBlock = bmIdsExecutor.map(_.executorId).toSet + bmIdsExtShuffle.foreach { bmIdForShuffleService => + // if the original executor is already released then delete this disk block via + // the external shuffle service + if (!liveExecutorsForBlock.contains(bmIdForShuffleService.executorId)) { + val blockIdsToDel = blocksToDeleteByShuffleService.getOrElseUpdate(bmIdForShuffleService, + new mutable.HashSet[RDDBlockId]()) + blockIdsToDel += blockId + blockStatusByShuffleService.get(bmIdForShuffleService).foreach { blockStatus => + blockStatus.remove(blockId) + } + } + } + bmIdsExecutor.foreach { bmId => + blockManagerInfo.get(bmId).foreach { bmInfo => + bmInfo.removeBlock(blockId) + } + } } - - // Ask the slaves to remove the RDD, and put the result in a sequence of Futures. - // The dispatcher is used as an implicit argument into the Future sequence construction. - val removeMsg = RemoveRdd(rddId) - - val futures = blockManagerInfo.values.map { bm => - bm.slaveEndpoint.ask[Int](removeMsg).recover { + val removeRddFromExecutorsFutures = blockManagerInfo.values.map { bmInfo => + bmInfo.slaveEndpoint.ask[Int](removeMsg).recover { case e: IOException => - logWarning(s"Error trying to remove RDD $rddId from block manager ${bm.blockManagerId}", - e) + logWarning(s"Error trying to remove RDD ${removeMsg.rddId} " + + s"from block manager ${bmInfo.blockManagerId}", e) 0 // zero blocks were removed } }.toSeq - Future.sequence(futures) + val removeRddBlockViaExtShuffleServiceFutures = externalShuffleClient.map { shuffleClient => + blocksToDeleteByShuffleService.map { case (bmId, blockIds) => + Future[Int] { + val numRemovedBlocks = shuffleClient.removeBlocks( + bmId.host, + bmId.port, + bmId.executorId, + blockIds.map(_.toString).toArray) + numRemovedBlocks.get(defaultRpcTimeout.duration.toSeconds, TimeUnit.SECONDS) + } + } + }.getOrElse(Seq.empty) + + Future.sequence(removeRddFromExecutorsFutures ++ removeRddBlockViaExtShuffleServiceFutures) } private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { @@ -350,11 +387,18 @@ class BlockManagerMasterEndpoint( ).map(_.flatten.toSeq) } + private def externalShuffleServiceIdOnHost(blockManagerId: BlockManagerId): BlockManagerId = { + // we need to keep the executor ID of the original executor to let the shuffle service know + // which local directories should be used to look for the file + BlockManagerId(blockManagerId.executorId, blockManagerId.host, externalShuffleServicePort) + } + /** * Returns the BlockManagerId with topology information populated, if available. */ private def register( idWithoutTopologyInfo: BlockManagerId, + localDirs: Array[String], maxOnHeapMemSize: Long, maxOffHeapMemSize: Long, slaveEndpoint: RpcEndpointRef): BlockManagerId = { @@ -381,8 +425,17 @@ class BlockManagerMasterEndpoint( blockManagerIdByExecutor(id.executorId) = id - blockManagerInfo(id) = new BlockManagerInfo( - id, System.currentTimeMillis(), maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint) + val externalShuffleServiceBlockStatus = + if (externalShuffleServiceRddFetchEnabled) { + val externalShuffleServiceBlocks = blockStatusByShuffleService + .getOrElseUpdate(externalShuffleServiceIdOnHost(id), new JHashMap[BlockId, BlockStatus]) + Some(externalShuffleServiceBlocks) + } else { + None + } + + blockManagerInfo(id) = new BlockManagerInfo(id, System.currentTimeMillis(), localDirs, + maxOnHeapMemSize, maxOffHeapMemSize, slaveEndpoint, externalShuffleServiceBlockStatus) } listenerBus.post(SparkListenerBlockManagerAdded(time, id, maxOnHeapMemSize + maxOffHeapMemSize, Some(maxOnHeapMemSize), Some(maxOffHeapMemSize))) @@ -427,6 +480,15 @@ class BlockManagerMasterEndpoint( locations.remove(blockManagerId) } + if (blockId.isRDD && storageLevel.useDisk && externalShuffleServiceRddFetchEnabled) { + val externalShuffleServiceId = externalShuffleServiceIdOnHost(blockManagerId) + if (storageLevel.isValid) { + locations.add(externalShuffleServiceId) + } else { + locations.remove(externalShuffleServiceId) + } + } + // Remove the block from master tracking if it has been removed on all slaves. if (locations.size == 0) { blockLocations.remove(blockId) @@ -438,12 +500,30 @@ class BlockManagerMasterEndpoint( if (blockLocations.containsKey(blockId)) blockLocations.get(blockId).toSeq else Seq.empty } - private def getLocationsAndStatus(blockId: BlockId): Option[BlockLocationsAndStatus] = { + private def getLocationsAndStatus( + blockId: BlockId, + requesterHost: String): Option[BlockLocationsAndStatus] = { val locations = Option(blockLocations.get(blockId)).map(_.toSeq).getOrElse(Seq.empty) - val status = locations.headOption.flatMap { bmId => blockManagerInfo(bmId).getStatus(blockId) } + val status = locations.headOption.flatMap { bmId => + if (externalShuffleServiceRddFetchEnabled && bmId.port == externalShuffleServicePort) { + Option(blockStatusByShuffleService(bmId).get(blockId)) + } else { + blockManagerInfo.get(bmId).flatMap(_.getStatus(blockId)) + } + } if (locations.nonEmpty && status.isDefined) { - Some(BlockLocationsAndStatus(locations, status.get)) + val localDirs = locations.find { loc => + if (loc.port != externalShuffleServicePort && loc.host == requesterHost) { + blockManagerInfo + .get(loc) + .flatMap(_.getStatus(blockId).map(_.storageLevel.useDisk)) + .getOrElse(false) + } else { + false + } + }.map(blockManagerInfo(_).localDirs) + Some(BlockLocationsAndStatus(locations, status.get, localDirs)) } else { None } @@ -494,22 +574,23 @@ object BlockStatus { private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, timeMs: Long, + val localDirs: Array[String], val maxOnHeapMem: Long, val maxOffHeapMem: Long, - val slaveEndpoint: RpcEndpointRef) + val slaveEndpoint: RpcEndpointRef, + val externalShuffleServiceBlockStatus: Option[JHashMap[BlockId, BlockStatus]]) extends Logging { val maxMem = maxOnHeapMem + maxOffHeapMem + val externalShuffleServiceEnabled = externalShuffleServiceBlockStatus.isDefined + private var _lastSeenMs: Long = timeMs private var _remainingMem: Long = maxMem // Mapping from block id to its status. private val _blocks = new JHashMap[BlockId, BlockStatus] - // Cached blocks held by this BlockManager. This does not include broadcast blocks. - private val _cachedBlocks = new mutable.HashSet[BlockId] - def getStatus(blockId: BlockId): Option[BlockStatus] = Option(_blocks.get(blockId)) def updateLastSeenMs() { @@ -576,13 +657,18 @@ private[spark] class BlockManagerInfo( s" (size: ${Utils.bytesToString(diskSize)})") } } - if (!blockId.isBroadcast && blockStatus.isCached) { - _cachedBlocks += blockId + + externalShuffleServiceBlockStatus.foreach { shuffleServiceBlocks => + if (!blockId.isBroadcast && blockStatus.diskSize > 0) { + shuffleServiceBlocks.put(blockId, blockStatus) + } } } else if (blockExists) { // If isValid is not true, drop the block. _blocks.remove(blockId) - _cachedBlocks -= blockId + externalShuffleServiceBlockStatus.foreach { blockStatus => + blockStatus.remove(blockId) + } if (originalLevel.useMemory) { logInfo(s"Removed $blockId on ${blockManagerId.hostPort} in memory" + s" (size: ${Utils.bytesToString(originalMemSize)}," + @@ -599,8 +685,10 @@ private[spark] class BlockManagerInfo( if (_blocks.containsKey(blockId)) { _remainingMem += _blocks.get(blockId).memSize _blocks.remove(blockId) + externalShuffleServiceBlockStatus.foreach { blockStatus => + blockStatus.remove(blockId) + } } - _cachedBlocks -= blockId } def remainingMem: Long = _remainingMem @@ -609,9 +697,6 @@ private[spark] class BlockManagerInfo( def blocks: JHashMap[BlockId, BlockStatus] = _blocks - // This does not include broadcast blocks. - def cachedBlocks: collection.Set[BlockId] = _cachedBlocks - override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem def clear() { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 1bbe7a5b39509..895f48d0709fb 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -58,6 +58,7 @@ private[spark] object BlockManagerMessages { case class RegisterBlockManager( blockManagerId: BlockManagerId, + localDirs: Array[String], maxOnHeapMemSize: Long, maxOffHeapMemSize: Long, sender: RpcEndpointRef) @@ -93,10 +94,20 @@ private[spark] object BlockManagerMessages { case class GetLocations(blockId: BlockId) extends ToBlockManagerMaster - case class GetLocationsAndStatus(blockId: BlockId) extends ToBlockManagerMaster + case class GetLocationsAndStatus(blockId: BlockId, requesterHost: String) + extends ToBlockManagerMaster - // The response message of `GetLocationsAndStatus` request. - case class BlockLocationsAndStatus(locations: Seq[BlockManagerId], status: BlockStatus) { + /** + * The response message of `GetLocationsAndStatus` request. + * + * @param localDirs if it is persisted-to-disk on the same host as the requester executor is + * running on then localDirs will be Some and the cached data will be in a file + * in one of those dirs, otherwise it is None. + */ + case class BlockLocationsAndStatus( + locations: Seq[BlockManagerId], + status: BlockStatus, + localDirs: Option[Array[String]]) { assert(locations.nonEmpty) } @@ -122,5 +133,5 @@ private[spark] object BlockManagerMessages { case class BlockManagerHeartbeat(blockManagerId: BlockManagerId) extends ToBlockManagerMaster - case class HasCachedBlocks(executorId: String) extends ToBlockManagerMaster + case class IsExecutorAlive(executorId: String) extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index a69bcc9259995..c3990bf71e604 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -22,7 +22,7 @@ import java.util.UUID import org.apache.spark.SparkConf import org.apache.spark.executor.ExecutorExitCode -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{ShutdownHookManager, Utils} /** @@ -34,7 +34,7 @@ import org.apache.spark.util.{ShutdownHookManager, Utils} */ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolean) extends Logging { - private[spark] val subDirsPerLocalDir = conf.getInt("spark.diskStore.subDirectories", 64) + private[spark] val subDirsPerLocalDir = conf.get(config.DISKSTORE_SUB_DIRECTORIES) /* Create one local directory for each path mentioned in spark.local.dir; then, inside this * directory, create multiple subdirectories that we will hash files into, in order to avoid @@ -44,6 +44,9 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea logError("Failed to create any local dir.") System.exit(ExecutorExitCode.DISK_STORE_FAILED_TO_CREATE_DIR) } + + private[spark] val localDirsString: Array[String] = localDirs.map(_.toString) + // The content of subDirs is immutable but the content of subDirs(i) is mutable. And the content // of subDirs(i) is protected by the lock of subDirs(i) private val subDirs = Array.fill(localDirs.length)(new Array[File](subDirsPerLocalDir)) @@ -52,7 +55,7 @@ private[spark] class DiskBlockManager(conf: SparkConf, deleteFilesOnStop: Boolea /** Looks up a file by hashing it into one of our local subdirectories. */ // This method should be kept in sync with - // org.apache.spark.network.shuffle.ExternalShuffleBlockResolver#getFile(). + // org.apache.spark.network.shuffle.ExecutorDiskUtils#getFile(). def getFile(filename: String): File = { // Figure out which local directory it hashes to, and which subdirectory in that val hash = Utils.nonNegativeHash(filename) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala index a024c83d8d8b7..17390f9c60e79 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockObjectWriter.scala @@ -20,9 +20,9 @@ package org.apache.spark.storage import java.io.{BufferedOutputStream, File, FileOutputStream, OutputStream} import java.nio.channels.FileChannel -import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.internal.Logging import org.apache.spark.serializer.{SerializationStream, SerializerInstance, SerializerManager} +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter import org.apache.spark.util.Utils /** @@ -43,7 +43,7 @@ private[spark] class DiskBlockObjectWriter( syncWrites: Boolean, // These write metrics concurrently shared with other active DiskBlockObjectWriters who // are themselves performing writes. All updates must be relative. - writeMetrics: ShuffleWriteMetrics, + writeMetrics: ShuffleWriteMetricsReporter, val blockId: BlockId = null) extends OutputStream with Logging { diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index d88bd710d1ead..fbda4912e15ad 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -21,18 +21,20 @@ import java.io._ import java.nio.ByteBuffer import java.nio.channels.{Channels, ReadableByteChannel, WritableByteChannel} import java.nio.channels.FileChannel.MapMode -import java.util.concurrent.ConcurrentHashMap +import java.util.concurrent.{ConcurrentHashMap, TimeUnit} import scala.collection.mutable.ListBuffer import com.google.common.io.Closeables import io.netty.channel.DefaultFileRegion +import org.apache.commons.io.FileUtils import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.internal.{config, Logging} import org.apache.spark.network.buffer.ManagedBuffer import org.apache.spark.network.util.{AbstractFileRegion, JavaUtils} import org.apache.spark.security.CryptoStreamUtils +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils import org.apache.spark.util.io.ChunkedByteBuffer @@ -44,7 +46,7 @@ private[spark] class DiskStore( diskManager: DiskBlockManager, securityManager: SecurityManager) extends Logging { - private val minMemoryMapBytes = conf.getSizeAsBytes("spark.storage.memoryMapThreshold", "2m") + private val minMemoryMapBytes = conf.get(config.STORAGE_MEMORY_MAP_THRESHOLD) private val maxMemoryMapBytes = conf.get(config.MEMORY_MAP_LIMIT_FOR_TESTS) private val blockSizes = new ConcurrentHashMap[BlockId, Long]() @@ -60,7 +62,7 @@ private[spark] class DiskStore( throw new IllegalStateException(s"Block $blockId is already present in the disk store") } logDebug(s"Attempting to put block $blockId") - val startTime = System.currentTimeMillis + val startTimeNs = System.nanoTime() val file = diskManager.getFile(blockId) val out = new CountingWritableChannel(openForWrite(file)) var threwException: Boolean = true @@ -83,11 +85,8 @@ private[spark] class DiskStore( } } } - val finishTime = System.currentTimeMillis - logDebug("Block %s stored as %s file on disk in %d ms".format( - file.getName, - Utils.bytesToString(file.length()), - finishTime - startTime)) + logDebug(s"Block ${file.getName} stored as ${Utils.bytesToString(file.length())} file" + + s" on disk in ${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms") } def putBytes(blockId: BlockId, bytes: ChunkedByteBuffer): Unit = { @@ -97,18 +96,17 @@ private[spark] class DiskStore( } def getBytes(blockId: BlockId): BlockData = { - val file = diskManager.getFile(blockId.name) - val blockSize = getSize(blockId) + getBytes(diskManager.getFile(blockId.name), getSize(blockId)) + } - securityManager.getIOEncryptionKey() match { - case Some(key) => - // Encrypted blocks cannot be memory mapped; return a special object that does decryption - // and provides InputStream / FileRegion implementations for reading the data. - new EncryptedBlockData(file, blockSize, conf, key) + def getBytes(f: File, blockSize: Long): BlockData = securityManager.getIOEncryptionKey() match { + case Some(key) => + // Encrypted blocks cannot be memory mapped; return a special object that does decryption + // and provides InputStream / FileRegion implementations for reading the data. + new EncryptedBlockData(f, blockSize, conf, key) - case _ => - new DiskBlockData(minMemoryMapBytes, maxMemoryMapBytes, file, blockSize) - } + case _ => + new DiskBlockData(minMemoryMapBytes, maxMemoryMapBytes, f, blockSize) } def remove(blockId: BlockId): Boolean = { @@ -125,6 +123,16 @@ private[spark] class DiskStore( } } + /** + * @param blockSize if encryption is configured, the file is assumed to already be encrypted and + * blockSize should be the decrypted size + */ + def moveFileToBlock(sourceFile: File, blockSize: Long, targetBlockId: BlockId): Unit = { + blockSizes.put(targetBlockId, blockSize) + val targetFile = diskManager.getFile(targetBlockId.name) + FileUtils.moveFile(sourceFile, targetFile) + } + def contains(blockId: BlockId): Boolean = { val file = diskManager.getFile(blockId.name) file.exists() @@ -201,7 +209,7 @@ private class DiskBlockData( private def open() = new FileInputStream(file).getChannel } -private class EncryptedBlockData( +private[spark] class EncryptedBlockData( file: File, blockSize: Long, conf: SparkConf, @@ -217,7 +225,7 @@ private class EncryptedBlockData( var remaining = blockSize val chunks = new ListBuffer[ByteBuffer]() while (remaining > 0) { - val chunkSize = math.min(remaining, Int.MaxValue) + val chunkSize = math.min(remaining, ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) val chunk = allocator(chunkSize.toInt) remaining -= chunkSize JavaUtils.readFully(source, chunk) @@ -235,7 +243,8 @@ private class EncryptedBlockData( // This is used by the block transfer service to replicate blocks. The upload code reads // all bytes into memory to send the block to the remote executor, so it's ok to do this // as long as the block fits in a Java array. - assert(blockSize <= Int.MaxValue, "Block is too large to be wrapped in a byte buffer.") + assert(blockSize <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "Block is too large to be wrapped in a byte buffer.") val dst = ByteBuffer.allocate(blockSize.toInt) val in = open() try { @@ -263,7 +272,8 @@ private class EncryptedBlockData( } } -private class EncryptedManagedBuffer(val blockData: EncryptedBlockData) extends ManagedBuffer { +private[spark] class EncryptedManagedBuffer( + val blockData: EncryptedBlockData) extends ManagedBuffer { // This is the size of the decrypted data override def size(): Long = blockData.size diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala index aecc2284a9588..a28375737b8ba 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockFetcherIterator.scala @@ -18,21 +18,21 @@ package org.apache.spark.storage import java.io.{InputStream, IOException} -import java.nio.ByteBuffer -import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.{LinkedBlockingQueue, TimeUnit} import javax.annotation.concurrent.GuardedBy import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Queue} +import org.apache.commons.io.IOUtils + import org.apache.spark.{SparkException, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.shuffle._ import org.apache.spark.network.util.TransportConf -import org.apache.spark.shuffle.FetchFailedException -import org.apache.spark.util.Utils -import org.apache.spark.util.io.ChunkedByteBufferOutputStream +import org.apache.spark.shuffle.{FetchFailedException, ShuffleReadMetricsReporter} +import org.apache.spark.util.{CompletionIterator, TaskCompletionListener, Utils} /** * An iterator that fetches multiple blocks. For local blocks, it fetches from the local block @@ -51,7 +51,7 @@ import org.apache.spark.util.io.ChunkedByteBufferOutputStream * For each block we also require the size (in bytes as a long field) in * order to throttle the memory usage. Note that zero-sized blocks are * already excluded, which happened in - * [[MapOutputTracker.convertMapStatuses]]. + * [[org.apache.spark.MapOutputTracker.convertMapStatuses]]. * @param streamWrapper A function to wrap the returned input stream. * @param maxBytesInFlight max size (in bytes) of remote blocks to fetch at any given point. * @param maxReqsInFlight max number of remote requests to fetch blocks at any given point. @@ -59,6 +59,7 @@ import org.apache.spark.util.io.ChunkedByteBufferOutputStream * for a given remote host:port. * @param maxReqSizeShuffleToMem max size (in bytes) of a request that can be shuffled to memory. * @param detectCorrupt whether to detect any corruption in fetched blocks. + * @param shuffleMetrics used to report shuffle metrics. */ private[spark] final class ShuffleBlockFetcherIterator( @@ -71,7 +72,9 @@ final class ShuffleBlockFetcherIterator( maxReqsInFlight: Int, maxBlocksInFlightPerAddress: Int, maxReqSizeShuffleToMem: Long, - detectCorrupt: Boolean) + detectCorrupt: Boolean, + detectCorruptUseExtraMemory: Boolean, + shuffleMetrics: ShuffleReadMetricsReporter) extends Iterator[(BlockId, InputStream)] with DownloadFileManager with Logging { import ShuffleBlockFetcherIterator._ @@ -90,7 +93,7 @@ final class ShuffleBlockFetcherIterator( */ private[this] var numBlocksProcessed = 0 - private[this] val startTime = System.currentTimeMillis + private[this] val startTimeNs = System.nanoTime() /** Local blocks to fetch, excluding zero-sized blocks. */ private[this] val localBlocks = scala.collection.mutable.LinkedHashSet[BlockId]() @@ -137,8 +140,6 @@ final class ShuffleBlockFetcherIterator( */ private[this] val corruptedBlocks = mutable.HashSet[BlockId]() - private[this] val shuffleMetrics = context.taskMetrics().createTempShuffleReadMetrics() - /** * Whether the iterator is still active. If isZombie is true, the callback interface will no * longer place fetched blocks into [[results]]. @@ -153,6 +154,8 @@ final class ShuffleBlockFetcherIterator( @GuardedBy("this") private[this] val shuffleFilesSet = mutable.HashSet[DownloadFile]() + private[this] val onCompleteCallback = new ShuffleFetchCompletionListener(this) + initialize() // Decrements the buffer reference count. @@ -185,7 +188,7 @@ final class ShuffleBlockFetcherIterator( /** * Mark the iterator as zombie, and release all buffers that haven't been deserialized yet. */ - private[this] def cleanup() { + private[storage] def cleanup() { synchronized { isZombie = true } @@ -241,7 +244,7 @@ final class ShuffleBlockFetcherIterator( logDebug("remainingBlocks: " + remainingBlocks) } } - logTrace("Got remote block " + blockId + " after " + Utils.getUsedTimeMs(startTime)) + logTrace(s"Got remote block $blockId after ${Utils.getUsedTimeNs(startTimeNs)}") } override def onBlockFetchFailure(blockId: String, e: Throwable): Unit = { @@ -273,6 +276,8 @@ final class ShuffleBlockFetcherIterator( // Split local and remote blocks. Remote blocks are further split into FetchRequests of size // at most maxBytesInFlight in order to limit the amount of data in flight. val remoteRequests = new ArrayBuffer[FetchRequest] + var localBlockBytes = 0L + var remoteBlockBytes = 0L for ((address, blockInfos) <- blocksByAddress) { if (address.executorId == blockManager.blockManagerId.executorId) { @@ -284,6 +289,7 @@ final class ShuffleBlockFetcherIterator( case None => // do nothing. } localBlocks ++= blockInfos.map(_._1) + localBlockBytes += blockInfos.map(_._2).sum numBlocksToFetch += localBlocks.size } else { val iterator = blockInfos.iterator @@ -291,6 +297,7 @@ final class ShuffleBlockFetcherIterator( var curBlocks = new ArrayBuffer[(BlockId, Long)] while (iterator.hasNext) { val (blockId, size) = iterator.next() + remoteBlockBytes += size if (size < 0) { throw new BlockException(blockId, "Negative block size " + size) } else if (size == 0) { @@ -317,8 +324,10 @@ final class ShuffleBlockFetcherIterator( } } } - logInfo(s"Getting $numBlocksToFetch non-empty blocks including ${localBlocks.size}" + - s" local blocks and ${remoteBlocks.size} remote blocks") + val totalBytes = localBlockBytes + remoteBlockBytes + logInfo(s"Getting $numBlocksToFetch (${Utils.bytesToString(totalBytes)}) non-empty blocks " + + s"including ${localBlocks.size} (${Utils.bytesToString(localBlockBytes)}) local blocks and " + + s"${remoteBlocks.size} (${Utils.bytesToString(remoteBlockBytes)}) remote blocks") remoteRequests } @@ -351,7 +360,7 @@ final class ShuffleBlockFetcherIterator( private[this] def initialize(): Unit = { // Add a task completion callback (called in both success case and failure case) to cleanup. - context.addTaskCompletionListener[Unit](_ => cleanup()) + context.addTaskCompletionListener(onCompleteCallback) // Split local and remote blocks. val remoteRequests = splitLocalRemoteBlocks() @@ -365,11 +374,11 @@ final class ShuffleBlockFetcherIterator( fetchUpToMaxBytes() val numFetches = remoteRequests.size - fetchRequests.size - logInfo("Started " + numFetches + " remote fetches in" + Utils.getUsedTimeMs(startTime)) + logInfo(s"Started $numFetches remote fetches in ${Utils.getUsedTimeNs(startTimeNs)}") // Get Local Blocks fetchLocalBlocks() - logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime)) + logDebug(s"Got local blocks in ${Utils.getUsedTimeNs(startTimeNs)}") } override def hasNext: Boolean = numBlocksProcessed < numBlocksToFetch @@ -384,22 +393,23 @@ final class ShuffleBlockFetcherIterator( */ override def next(): (BlockId, InputStream) = { if (!hasNext) { - throw new NoSuchElementException + throw new NoSuchElementException() } numBlocksProcessed += 1 var result: FetchResult = null var input: InputStream = null + var streamCompressedOrEncrypted: Boolean = false // Take the next fetched result and try to decompress it to detect data corruption, // then fetch it one more time if it's corrupt, throw FailureFetchResult if the second fetch // is also corrupt, so the previous stage could be retried. // For local shuffle block, throw FailureFetchResult for the first IOException. while (result == null) { - val startFetchWait = System.currentTimeMillis() + val startFetchWait = System.nanoTime() result = results.take() - val stopFetchWait = System.currentTimeMillis() - shuffleMetrics.incFetchWaitTime(stopFetchWait - startFetchWait) + val fetchWaitTime = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startFetchWait) + shuffleMetrics.incFetchWaitTime(fetchWaitTime) result match { case r @ SuccessFetchResult(blockId, address, size, buf, isNetworkReqDone) => @@ -448,25 +458,22 @@ final class ShuffleBlockFetcherIterator( buf.release() throwFetchFailedException(blockId, address, e) } - var isStreamCopied: Boolean = false try { input = streamWrapper(blockId, in) - // Only copy the stream if it's wrapped by compression or encryption, also the size of - // block is small (the decompressed block is smaller than maxBytesInFlight) - if (detectCorrupt && !input.eq(in) && size < maxBytesInFlight / 3) { - isStreamCopied = true - val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) - // Decompress the whole block at once to detect any corruption, which could increase - // the memory usage tne potential increase the chance of OOM. + // If the stream is compressed or wrapped, then we optionally decompress/unwrap the + // first maxBytesInFlight/3 bytes into memory, to check for corruption in that portion + // of the data. But even if 'detectCorruptUseExtraMemory' configuration is off, or if + // the corruption is later, we'll still detect the corruption later in the stream. + streamCompressedOrEncrypted = !input.eq(in) + if (streamCompressedOrEncrypted && detectCorruptUseExtraMemory) { // TODO: manage the memory used here, and spill it into disk in case of OOM. - Utils.copyStream(input, out, closeStreams = true) - input = out.toChunkedByteBuffer.toInputStream(dispose = true) + input = Utils.copyStreamUpTo(input, maxBytesInFlight / 3) } } catch { case e: IOException => buf.release() if (buf.isInstanceOf[FileSegmentManagedBuffer] - || corruptedBlocks.contains(blockId)) { + || corruptedBlocks.contains(blockId)) { throwFetchFailedException(blockId, address, e) } else { logWarning(s"got an corrupted block $blockId from $address, fetch again", e) @@ -476,7 +483,9 @@ final class ShuffleBlockFetcherIterator( } } finally { // TODO: release the buf here to free memory earlier - if (isStreamCopied) { + if (input == null) { + // Close the underlying stream if there was an issue in wrapping the stream using + // streamWrapper in.close() } } @@ -490,7 +499,18 @@ final class ShuffleBlockFetcherIterator( } currentResult = result.asInstanceOf[SuccessFetchResult] - (currentResult.blockId, new BufferReleasingInputStream(input, this)) + (currentResult.blockId, + new BufferReleasingInputStream( + input, + this, + currentResult.blockId, + currentResult.address, + detectCorrupt && streamCompressedOrEncrypted)) + } + + def toCompletionIterator: Iterator[(BlockId, InputStream)] = { + CompletionIterator[(BlockId, InputStream), this.type](this, + onCompleteCallback.onComplete(context)) } private def fetchUpToMaxBytes(): Unit = { @@ -548,7 +568,10 @@ final class ShuffleBlockFetcherIterator( } } - private def throwFetchFailedException(blockId: BlockId, address: BlockManagerId, e: Throwable) = { + private[storage] def throwFetchFailedException( + blockId: BlockId, + address: BlockManagerId, + e: Throwable) = { blockId match { case ShuffleBlockId(shufId, mapId, reduceId) => throw new FetchFailedException(address, shufId.toInt, mapId.toInt, reduceId, e) @@ -560,15 +583,28 @@ final class ShuffleBlockFetcherIterator( } /** - * Helper class that ensures a ManagedBuffer is released upon InputStream.close() + * Helper class that ensures a ManagedBuffer is released upon InputStream.close() and + * also detects stream corruption if streamCompressedOrEncrypted is true */ private class BufferReleasingInputStream( - private val delegate: InputStream, - private val iterator: ShuffleBlockFetcherIterator) + // This is visible for testing + private[storage] val delegate: InputStream, + private val iterator: ShuffleBlockFetcherIterator, + private val blockId: BlockId, + private val address: BlockManagerId, + private val detectCorruption: Boolean) extends InputStream { private[this] var closed = false - override def read(): Int = delegate.read() + override def read(): Int = { + try { + delegate.read() + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, address, e) + } + } override def close(): Unit = { if (!closed) { @@ -582,17 +618,62 @@ private class BufferReleasingInputStream( override def mark(readlimit: Int): Unit = delegate.mark(readlimit) - override def skip(n: Long): Long = delegate.skip(n) + override def skip(n: Long): Long = { + try { + delegate.skip(n) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, address, e) + } + } override def markSupported(): Boolean = delegate.markSupported() - override def read(b: Array[Byte]): Int = delegate.read(b) + override def read(b: Array[Byte]): Int = { + try { + delegate.read(b) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, address, e) + } + } - override def read(b: Array[Byte], off: Int, len: Int): Int = delegate.read(b, off, len) + override def read(b: Array[Byte], off: Int, len: Int): Int = { + try { + delegate.read(b, off, len) + } catch { + case e: IOException if detectCorruption => + IOUtils.closeQuietly(this) + iterator.throwFetchFailedException(blockId, address, e) + } + } override def reset(): Unit = delegate.reset() } +/** + * A listener to be called at the completion of the ShuffleBlockFetcherIterator + * @param data the ShuffleBlockFetcherIterator to process + */ +private class ShuffleFetchCompletionListener(var data: ShuffleBlockFetcherIterator) + extends TaskCompletionListener { + + override def onTaskCompletion(context: TaskContext): Unit = { + if (data != null) { + data.cleanup() + // Null out the referent here to make sure we don't keep a reference to this + // ShuffleBlockFetcherIterator, after we're done reading from it, to let it be + // collected during GC. Otherwise we can hold metadata on block locations(blocksByAddress) + data = null + } + } + + // Just an alias for onTaskCompletion to avoid confusing + def onComplete(context: TaskContext): Unit = this.onTaskCompletion(context) +} + private[storage] object ShuffleBlockFetcherIterator { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index adc406bb1c441..fc426eee608c0 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -22,9 +22,13 @@ import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.Map import scala.collection.mutable +import org.apache.commons.lang3.{JavaVersion, SystemUtils} +import sun.misc.Unsafe import sun.nio.ch.DirectBuffer -import org.apache.spark.internal.Logging +import org.apache.spark.SparkConf +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.util.Utils /** * Storage information for each BlockManager. @@ -193,6 +197,31 @@ private[spark] class StorageStatus( /** Helper methods for storage-related objects. */ private[spark] object StorageUtils extends Logging { + + // In Java 8, the type of DirectBuffer.cleaner() was sun.misc.Cleaner, and it was possible + // to access the method sun.misc.Cleaner.clean() to invoke it. The type changed to + // jdk.internal.ref.Cleaner in later JDKs, and the .clean() method is not accessible even with + // reflection. However sun.misc.Unsafe added a invokeCleaner() method in JDK 9+ and this is + // still accessible with reflection. + private val bufferCleaner: DirectBuffer => Unit = + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + val cleanerMethod = + Utils.classForName("sun.misc.Unsafe").getMethod("invokeCleaner", classOf[ByteBuffer]) + val unsafeField = classOf[Unsafe].getDeclaredField("theUnsafe") + unsafeField.setAccessible(true) + val unsafe = unsafeField.get(null).asInstanceOf[Unsafe] + buffer: DirectBuffer => cleanerMethod.invoke(unsafe, buffer) + } else { + val cleanerMethod = Utils.classForName("sun.misc.Cleaner").getMethod("clean") + buffer: DirectBuffer => { + // Careful to avoid the return type of .cleaner(), which changes with JDK + val cleaner: AnyRef = buffer.cleaner() + if (cleaner != null) { + cleanerMethod.invoke(cleaner) + } + } + } + /** * Attempt to clean up a ByteBuffer if it is direct or memory-mapped. This uses an *unsafe* Sun * API that will cause errors if one attempts to read from the disposed buffer. However, neither @@ -204,14 +233,24 @@ private[spark] object StorageUtils extends Logging { def dispose(buffer: ByteBuffer): Unit = { if (buffer != null && buffer.isInstanceOf[MappedByteBuffer]) { logTrace(s"Disposing of $buffer") - cleanDirectBuffer(buffer.asInstanceOf[DirectBuffer]) + bufferCleaner(buffer.asInstanceOf[DirectBuffer]) } } - private def cleanDirectBuffer(buffer: DirectBuffer) = { - val cleaner = buffer.cleaner() - if (cleaner != null) { - cleaner.clean() + /** + * Get the port used by the external shuffle service. In Yarn mode, this may be already be + * set through the Hadoop configuration as the server is launched in the Yarn NM. + */ + def externalShuffleServicePort(conf: SparkConf): Int = { + val tmpPort = Utils.getSparkOrYarnConfig(conf, config.SHUFFLE_SERVICE_PORT.key, + config.SHUFFLE_SERVICE_PORT.defaultValueString).toInt + if (tmpPort == 0) { + // for testing, we set "spark.shuffle.service.port" to 0 in the yarn config, so yarn finds + // an open port. But we still need to tell our spark apps the right port to use. So + // only if the yarn config has the port set to 0, we prefer the value in the spark config + conf.get(config.SHUFFLE_SERVICE_PORT.key).toInt + } else { + tmpPort } } } diff --git a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala index a150a8e3636e4..3c2c4b46dc4ca 100644 --- a/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala +++ b/core/src/main/scala/org/apache/spark/storage/TopologyMapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.storage import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.Utils /** @@ -68,7 +68,7 @@ class DefaultTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with L */ @DeveloperApi class FileBasedTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Logging { - val topologyFile = conf.getOption("spark.storage.replication.topologyFile") + val topologyFile = conf.get(config.STORAGE_REPLICATION_TOPOLOGY_FILE) require(topologyFile.isDefined, "Please specify topology file via " + "spark.storage.replication.topologyFile for FileBasedTopologyMapper.") val topologyMap = Utils.getPropertiesFromFile(topologyFile.get) diff --git a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala index 06fd56e54d9c8..375d05be74bf5 100644 --- a/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/memory/MemoryStore.scala @@ -29,11 +29,12 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkConf, TaskContext} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.{UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} +import org.apache.spark.internal.config.{STORAGE_UNROLL_MEMORY_THRESHOLD, UNROLL_MEMORY_CHECK_PERIOD, UNROLL_MEMORY_GROWTH_FACTOR} import org.apache.spark.memory.{MemoryManager, MemoryMode} import org.apache.spark.serializer.{SerializationStream, SerializerManager} import org.apache.spark.storage._ import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.{SizeEstimator, Utils} import org.apache.spark.util.collection.SizeTrackingVector import org.apache.spark.util.io.{ChunkedByteBuffer, ChunkedByteBufferOutputStream} @@ -99,7 +100,7 @@ private[spark] class MemoryStore( // Initial memory to request before unrolling any block private val unrollMemoryThreshold: Long = - conf.getLong("spark.storage.unrollMemoryThreshold", 1024 * 1024) + conf.get(STORAGE_UNROLL_MEMORY_THRESHOLD) /** Total amount of memory available for storage, in bytes. */ private def maxMemory: Long = { @@ -333,11 +334,11 @@ private[spark] class MemoryStore( // Initial per-task memory to request for unrolling blocks (bytes). val initialMemoryThreshold = unrollMemoryThreshold - val chunkSize = if (initialMemoryThreshold > Int.MaxValue) { + val chunkSize = if (initialMemoryThreshold > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { logWarning(s"Initial memory threshold of ${Utils.bytesToString(initialMemoryThreshold)} " + s"is too large to be set as chunk size. Chunk size has been capped to " + - s"${Utils.bytesToString(Int.MaxValue)}") - Int.MaxValue + s"${Utils.bytesToString(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH)}") + ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH } else { initialMemoryThreshold.toInt } diff --git a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala index 3c4ee4eb6bbb9..f36b31c65a63d 100644 --- a/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala +++ b/core/src/main/scala/org/apache/spark/ui/ConsoleProgressBar.scala @@ -21,6 +21,7 @@ import java.util.{Timer, TimerTask} import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.StageData /** @@ -33,17 +34,12 @@ private[spark] class ConsoleProgressBar(sc: SparkContext) extends Logging { // Carriage return private val CR = '\r' // Update period of progress bar, in milliseconds - private val updatePeriodMSec = - sc.getConf.getTimeAsMs("spark.ui.consoleProgress.update.interval", "200") + private val updatePeriodMSec = sc.getConf.get(UI_CONSOLE_PROGRESS_UPDATE_INTERVAL) // Delay to show up a progress bar, in milliseconds private val firstDelayMSec = 500L // The width of terminal - private val TerminalWidth = if (!sys.env.getOrElse("COLUMNS", "").isEmpty) { - sys.env.get("COLUMNS").get.toInt - } else { - 80 - } + private val TerminalWidth = sys.env.getOrElse("COLUMNS", "80").toInt private var lastFinishTime = 0L private var lastUpdateTime = 0L diff --git a/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala new file mode 100644 index 0000000000000..1c0dd7dee2228 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/HttpSecurityFilter.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.ui + +import java.util.{Enumeration, Map => JMap} +import javax.servlet._ +import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpServletResponse} + +import scala.collection.JavaConverters._ + +import org.apache.commons.lang3.StringEscapeUtils + +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.internal.config.UI._ + +/** + * A servlet filter that implements HTTP security features. The following actions are taken + * for every request: + * + * - perform access control of authenticated requests. + * - check request data for disallowed content (e.g. things that could be used to create XSS + * attacks). + * - set response headers to prevent certain kinds of attacks. + * + * Request parameters are sanitized so that HTML content is escaped, and disallowed content is + * removed. + */ +private class HttpSecurityFilter( + conf: SparkConf, + securityMgr: SecurityManager) extends Filter { + + override def destroy(): Unit = { } + + override def init(config: FilterConfig): Unit = { } + + override def doFilter(req: ServletRequest, res: ServletResponse, chain: FilterChain): Unit = { + val hreq = req.asInstanceOf[HttpServletRequest] + val hres = res.asInstanceOf[HttpServletResponse] + hres.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + + val requestUser = hreq.getRemoteUser() + + // The doAs parameter allows proxy servers (e.g. Knox) to impersonate other users. For + // that to be allowed, the authenticated user needs to be an admin. + val effectiveUser = Option(hreq.getParameter("doAs")) + .map { proxy => + if (requestUser != proxy && !securityMgr.checkAdminPermissions(requestUser)) { + hres.sendError(HttpServletResponse.SC_FORBIDDEN, + s"User $requestUser is not allowed to impersonate others.") + return + } + proxy + } + .getOrElse(requestUser) + + if (!securityMgr.checkUIViewPermissions(effectiveUser)) { + hres.sendError(HttpServletResponse.SC_FORBIDDEN, + s"User $effectiveUser is not authorized to access this page.") + return + } + + // SPARK-10589 avoid frame-related click-jacking vulnerability, using X-Frame-Options + // (see http://tools.ietf.org/html/rfc7034). By default allow framing only from the + // same origin, but allow framing for a specific named URI. + // Example: spark.ui.allowFramingFrom = https://example.com/ + val xFrameOptionsValue = conf.getOption("spark.ui.allowFramingFrom") + .map { uri => s"ALLOW-FROM $uri" } + .getOrElse("SAMEORIGIN") + + hres.setHeader("X-Frame-Options", xFrameOptionsValue) + hres.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION)) + if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) { + hres.setHeader("X-Content-Type-Options", "nosniff") + } + if (hreq.getScheme() == "https") { + conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach( + hres.setHeader("Strict-Transport-Security", _)) + } + + chain.doFilter(new XssSafeRequest(hreq, effectiveUser), res) + } + +} + +private class XssSafeRequest(req: HttpServletRequest, effectiveUser: String) + extends HttpServletRequestWrapper(req) { + + private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r + + private val parameterMap: Map[String, Array[String]] = { + super.getParameterMap().asScala.map { case (name, values) => + stripXSS(name) -> values.map(stripXSS) + }.toMap + } + + override def getRemoteUser(): String = effectiveUser + + override def getParameterMap(): JMap[String, Array[String]] = parameterMap.asJava + + override def getParameterNames(): Enumeration[String] = { + parameterMap.keys.iterator.asJavaEnumeration + } + + override def getParameterValues(name: String): Array[String] = parameterMap.get(name).orNull + + override def getParameter(name: String): String = { + parameterMap.get(name).flatMap(_.headOption).orNull + } + + private def stripXSS(str: String): String = { + if (str != null) { + // Remove new lines and single quotes, followed by escaping HTML version 4.0 + StringEscapeUtils.escapeHtml4(NEWLINE_AND_SINGLE_QUOTE_REGEX.replaceAllIn(str, "")) + } else { + null + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 52a955111231a..ff7baf4d9419b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -18,6 +18,7 @@ package org.apache.spark.ui import java.net.{URI, URL} +import java.util.EnumSet import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} @@ -39,7 +40,7 @@ import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{SecurityManager, SparkConf, SSLOptions} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.Utils /** @@ -68,43 +69,16 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <: AnyRef]( + private def createServlet[T <: AnyRef]( servletParams: ServletParams[T], - securityMgr: SecurityManager, conf: SparkConf): HttpServlet = { - - // SPARK-10589 avoid frame-related click-jacking vulnerability, using X-Frame-Options - // (see http://tools.ietf.org/html/rfc7034). By default allow framing only from the - // same origin, but allow framing for a specific named URI. - // Example: spark.ui.allowFramingFrom = https://example.com/ - val allowFramingFrom = conf.getOption("spark.ui.allowFramingFrom") - val xFrameOptionsValue = - allowFramingFrom.map(uri => s"ALLOW-FROM $uri").getOrElse("SAMEORIGIN") - new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { try { - if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) { - response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) - response.setStatus(HttpServletResponse.SC_OK) - val result = servletParams.responder(request) - response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.setHeader("X-Frame-Options", xFrameOptionsValue) - response.setHeader("X-XSS-Protection", conf.get(UI_X_XSS_PROTECTION)) - if (conf.get(UI_X_CONTENT_TYPE_OPTIONS)) { - response.setHeader("X-Content-Type-Options", "nosniff") - } - if (request.getScheme == "https") { - conf.get(UI_STRICT_TRANSPORT_SECURITY).foreach( - response.setHeader("Strict-Transport-Security", _)) - } - response.getWriter.print(servletParams.extractFn(result)) - } else { - response.setStatus(HttpServletResponse.SC_FORBIDDEN) - response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.sendError(HttpServletResponse.SC_FORBIDDEN, - "User is not authorized to access this page.") - } + response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) + response.setStatus(HttpServletResponse.SC_OK) + val result = servletParams.responder(request) + response.getWriter.print(servletParams.extractFn(result)) } catch { case e: IllegalArgumentException => response.sendError(HttpServletResponse.SC_BAD_REQUEST, e.getMessage) @@ -124,10 +98,9 @@ private[spark] object JettyUtils extends Logging { def createServletHandler[T <: AnyRef]( path: String, servletParams: ServletParams[T], - securityMgr: SecurityManager, conf: SparkConf, basePath: String = ""): ServletContextHandler = { - createServletHandler(path, createServlet(servletParams, securityMgr, conf), basePath) + createServletHandler(path, createServlet(servletParams, conf), basePath) } /** Create a context handler that responds to a request with the given path prefix */ @@ -257,36 +230,6 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add filters, if any, to the given list of ServletContextHandlers */ - def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { - val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { - case filter : String => - if (!filter.isEmpty) { - logInfo(s"Adding filter $filter to ${handlers.map(_.getContextPath).mkString(", ")}.") - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // Get any parameters for each filter - conf.get("spark." + filter + ".params", "").split(',').map(_.trim()).toSet.foreach { - param: String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } - } - - val prefix = s"spark.$filter.param." - conf.getAll - .filter { case (k, v) => k.length() > prefix.length() && k.startsWith(prefix) } - .foreach { case (k, v) => holder.setInitParameter(k.substring(prefix.length()), v) } - - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } - } - } - } - /** * Attempt to start a Jetty server bound to the supplied hostName:port using the given * context handlers. @@ -298,12 +241,9 @@ private[spark] object JettyUtils extends Logging { hostName: String, port: Int, sslOptions: SSLOptions, - handlers: Seq[ServletContextHandler], conf: SparkConf, serverName: String = ""): ServerInfo = { - addFilters(handlers, conf) - // Start the server first, with no connectors. val pool = new QueuedThreadPool if (serverName.nonEmpty) { @@ -356,13 +296,17 @@ private[spark] object JettyUtils extends Logging { (connector, connector.getLocalPort()) } + val httpConfig = new HttpConfiguration() + val requestHeaderSize = conf.get(UI_REQUEST_HEADER_SIZE).toInt + logDebug(s"Using requestHeaderSize: $requestHeaderSize") + httpConfig.setRequestHeaderSize(requestHeaderSize) // If SSL is configured, create the secure connector first. val securePort = sslOptions.createJettySslContextFactory().map { factory => val securePort = sslOptions.port.getOrElse(if (port > 0) Utils.userPort(port, 400) else 0) val secureServerName = if (serverName.nonEmpty) s"$serverName (HTTPS)" else serverName val connectionFactories = AbstractConnectionFactory.getFactories(factory, - new HttpConnectionFactory()) + new HttpConnectionFactory(httpConfig)) def sslConnect(currentPort: Int): (ServerConnector, Int) = { newConnector(connectionFactories, currentPort) @@ -377,7 +321,7 @@ private[spark] object JettyUtils extends Logging { // Bind the HTTP port. def httpConnect(currentPort: Int): (ServerConnector, Int) = { - newConnector(Array(new HttpConnectionFactory()), currentPort) + newConnector(Array(new HttpConnectionFactory(httpConfig)), currentPort) } val (httpConnector, httpPort) = Utils.startServiceOnPort[ServerConnector](port, httpConnect, @@ -396,16 +340,6 @@ private[spark] object JettyUtils extends Logging { } server.addConnector(httpConnector) - - // Add all the known handlers now that connectors are configured. - handlers.foreach { h => - h.setVirtualHosts(toVirtualHosts(SPARK_CONNECTOR_NAME)) - val gzipHandler = new GzipHandler() - gzipHandler.setHandler(h) - collection.addHandler(gzipHandler) - gzipHandler.start() - } - pool.setMaxThreads(math.max(pool.getMaxThreads, minThreads)) ServerInfo(server, httpPort, securePort, conf, collection) } catch { @@ -487,6 +421,16 @@ private[spark] object JettyUtils extends Logging { } } + def addFilter( + handler: ServletContextHandler, + filter: String, + params: Map[String, String]): Unit = { + val holder = new FilterHolder() + holder.setClassName(filter) + params.foreach { case (k, v) => holder.setInitParameter(k, v) } + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + // Create a new URI from the arguments, handling IPv6 host encoding and default ports. private def createRedirectURI( scheme: String, server: String, port: Int, path: String, query: String) = { @@ -507,20 +451,37 @@ private[spark] case class ServerInfo( server: Server, boundPort: Int, securePort: Option[Int], - conf: SparkConf, - private val rootHandler: ContextHandlerCollection) { + private val conf: SparkConf, + private val rootHandler: ContextHandlerCollection) extends Logging { - def addHandler(handler: ServletContextHandler): Unit = { + def addHandler( + handler: ServletContextHandler, + securityMgr: SecurityManager): Unit = synchronized { handler.setVirtualHosts(JettyUtils.toVirtualHosts(JettyUtils.SPARK_CONNECTOR_NAME)) - JettyUtils.addFilters(Seq(handler), conf) - rootHandler.addHandler(handler) + addFilters(handler, securityMgr) + + val gzipHandler = new GzipHandler() + gzipHandler.setHandler(handler) + rootHandler.addHandler(gzipHandler) + if (!handler.isStarted()) { handler.start() } + gzipHandler.start() } - def removeHandler(handler: ContextHandler): Unit = { - rootHandler.removeHandler(handler) + def removeHandler(handler: ServletContextHandler): Unit = synchronized { + // Since addHandler() always adds a wrapping gzip handler, find the container handler + // and remove it. + rootHandler.getHandlers() + .find { h => + h.isInstanceOf[GzipHandler] && h.asInstanceOf[GzipHandler].getHandler() == handler + } + .foreach { h => + rootHandler.removeHandler(h) + h.stop() + } + if (handler.isStarted) { handler.stop() } @@ -535,4 +496,33 @@ private[spark] case class ServerInfo( threadPool.asInstanceOf[LifeCycle].stop } } + + /** + * Add filters, if any, to the given ServletContextHandlers. Always adds a filter at the end + * of the chain to perform security-related functions. + */ + private def addFilters(handler: ServletContextHandler, securityMgr: SecurityManager): Unit = { + conf.get(UI_FILTERS).foreach { filter => + logInfo(s"Adding filter to ${handler.getContextPath()}: $filter") + val oldParams = conf.getOption(s"spark.$filter.params").toSeq + .flatMap(Utils.stringToSeq) + .flatMap { param => + val parts = param.split("=") + if (parts.length == 2) Some(parts(0) -> parts(1)) else None + } + .toMap + + val newParams = conf.getAllWithPrefix(s"spark.$filter.param.").toMap + + JettyUtils.addFilter(handler, filter, oldParams ++ newParams) + } + + // This filter must come after user-installed filters, since that's where authentication + // filters are installed. This means that custom filters will see the request before it's + // been validated by the security filter. + val securityFilter = new HttpSecurityFilter(conf, securityMgr) + val holder = new FilterHolder(securityFilter) + handler.addFilter(holder, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + } diff --git a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala index 65fa38387b9ee..6c2c1f6827948 100644 --- a/core/src/main/scala/org/apache/spark/ui/PagedTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/PagedTable.scala @@ -31,11 +31,7 @@ import org.apache.spark.util.Utils * * @param pageSize the number of rows in a page */ -private[ui] abstract class PagedDataSource[T](val pageSize: Int) { - - if (pageSize <= 0) { - throw new IllegalArgumentException("Page size must be positive") - } +private[spark] abstract class PagedDataSource[T](val pageSize: Int) { /** * Return the size of all data. @@ -51,13 +47,24 @@ private[ui] abstract class PagedDataSource[T](val pageSize: Int) { * Slice the data for this page */ def pageData(page: Int): PageData[T] = { - val totalPages = (dataSize + pageSize - 1) / pageSize - if (page <= 0 || page > totalPages) { - throw new IndexOutOfBoundsException( - s"Page $page is out of range. Please select a page number between 1 and $totalPages.") + // Display all the data in one page, if the pageSize is less than or equal to zero. + val pageTableSize = if (pageSize <= 0) { + dataSize + } else { + pageSize + } + val totalPages = (dataSize + pageTableSize - 1) / pageTableSize + + val pageToShow = if (page <= 0) { + 1 + } else if (page > totalPages) { + totalPages + } else { + page } - val from = (page - 1) * pageSize - val to = dataSize.min(page * pageSize) + + val (from, to) = ((pageToShow - 1) * pageSize, dataSize.min(pageToShow * pageTableSize)) + PageData(totalPages, sliceData(from, to)) } @@ -72,7 +79,7 @@ private[ui] case class PageData[T](totalPage: Int, data: Seq[T]) /** * A paged table that will generate a HTML table for a specified page and also the page navigation. */ -private[ui] trait PagedTable[T] { +private[spark] trait PagedTable[T] { def tableId: String @@ -80,8 +87,6 @@ private[ui] trait PagedTable[T] { def pageSizeFormField: String - def prevPageSizeFormField: String - def pageNumberFormField: String def dataSource: PagedDataSource[T] @@ -94,7 +99,23 @@ private[ui] trait PagedTable[T] { val _dataSource = dataSource try { val PageData(totalPages, data) = _dataSource.pageData(page) - val pageNavi = pageNavigation(page, _dataSource.pageSize, totalPages) + + val pageToShow = if (page <= 0) { + 1 + } else if (page > totalPages) { + totalPages + } else { + page + } + // Display all the data in one page, if the pageSize is less than or equal to zero. + val pageSize = if (_dataSource.pageSize <= 0) { + data.size + } else { + _dataSource.pageSize + } + + val pageNavi = pageNavigation(pageToShow, pageSize, totalPages) +
{pageNavi} @@ -122,13 +143,9 @@ private[ui] trait PagedTable[T] { /** * Return a page navigation. - *
    - *
  • If the totalPages is 1, the page navigation will be empty
  • - *
  • - * If the totalPages is more than 1, it will create a page navigation including a group of - * page numbers and a form to submit the page number. - *
  • - *
+ * + * It will create a page navigation including a group of page numbers and a form + * to submit the page number. * * Here are some examples of the page navigation: * {{{ @@ -154,120 +171,112 @@ private[ui] trait PagedTable[T] { * }}} */ private[ui] def pageNavigation(page: Int, pageSize: Int, totalPages: Int): Seq[Node] = { - if (totalPages == 1) { - Nil - } else { - // A group includes all page numbers will be shown in the page navigation. - // The size of group is 10 means there are 10 page numbers will be shown. - // The first group is 1 to 10, the second is 2 to 20, and so on - val groupSize = 10 - val firstGroup = 0 - val lastGroup = (totalPages - 1) / groupSize - val currentGroup = (page - 1) / groupSize - val startPage = currentGroup * groupSize + 1 - val endPage = totalPages.min(startPage + groupSize - 1) - val pageTags = (startPage to endPage).map { p => - if (p == page) { - // The current page should be disabled so that it cannot be clicked. -
  • {p}
  • - } else { -
  • {p}
  • - } + // A group includes all page numbers will be shown in the page navigation. + // The size of group is 10 means there are 10 page numbers will be shown. + // The first group is 1 to 10, the second is 2 to 20, and so on + val groupSize = 10 + val firstGroup = 0 + val lastGroup = (totalPages - 1) / groupSize + val currentGroup = (page - 1) / groupSize + val startPage = currentGroup * groupSize + 1 + val endPage = totalPages.min(startPage + groupSize - 1) + val pageTags = (startPage to endPage).map { p => + if (p == page) { + // The current page should be disabled so that it cannot be clicked. +
  • {p}
  • + } else { +
  • {p}
  • } + } - val hiddenFormFields = { - if (goButtonFormPath.contains('?')) { - val queryString = goButtonFormPath.split("\\?", 2)(1) - val search = queryString.split("#")(0) - Splitter - .on('&') - .trimResults() - .omitEmptyStrings() - .withKeyValueSeparator("=") - .split(search) - .asScala - .filterKeys(_ != pageSizeFormField) - .filterKeys(_ != prevPageSizeFormField) - .filterKeys(_ != pageNumberFormField) - .mapValues(URLDecoder.decode(_, "UTF-8")) - .map { case (k, v) => - - } - } else { - Seq.empty - } + val hiddenFormFields = { + if (goButtonFormPath.contains('?')) { + val queryString = goButtonFormPath.split("\\?", 2)(1) + val search = queryString.split("#")(0) + Splitter + .on('&') + .trimResults() + .omitEmptyStrings() + .withKeyValueSeparator("=") + .split(search) + .asScala + .filterKeys(_ != pageSizeFormField) + .filterKeys(_ != pageNumberFormField) + .mapValues(URLDecoder.decode(_, "UTF-8")) + .map { case (k, v) => + + } + } else { + Seq.empty } + } +
    -
    -
    - - {hiddenFormFields} - - - - - - - - - -
    - + + {if (page > 1) { +
  • + + + +
  • + }} + {pageTags} + {if (page < totalPages) { +
  • + + + +
  • + }} + {if (currentGroup < lastGroup) { +
  • + + + +
  • + }} +
    - } +
    } /** diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index d315ef66e0dc0..1175bc25de454 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,12 +17,11 @@ package org.apache.spark.ui -import java.util.{Date, List => JList, ServiceLoader} +import java.util.Date -import scala.collection.JavaConverters._ - -import org.apache.spark.{JobExecutionStatus, SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1._ @@ -31,7 +30,6 @@ import org.apache.spark.ui.env.EnvironmentTab import org.apache.spark.ui.exec.ExecutorsTab import org.apache.spark.ui.jobs.{JobsTab, StagesTab} import org.apache.spark.ui.storage.StorageTab -import org.apache.spark.util.Utils /** * Top level user interface for a Spark application. @@ -50,7 +48,7 @@ private[spark] class SparkUI private ( with Logging with UIRoot { - val killEnabled = sc.map(_.conf.getBoolean("spark.ui.killEnabled", true)).getOrElse(false) + val killEnabled = sc.map(_.conf.get(UI_KILL_ENABLED)).getOrElse(false) var appId: String = _ @@ -151,12 +149,11 @@ private[spark] abstract class SparkUITab(parent: SparkUI, prefix: String) } private[spark] object SparkUI { - val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" val DEFAULT_POOL_NAME = "default" def getUIPort(conf: SparkConf): Int = { - conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + conf.get(UI_PORT) } /** diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 732b7528f499e..b59f03a375ee7 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -27,8 +27,6 @@ import scala.util.control.NonFatal import scala.xml._ import scala.xml.transform.{RewriteRule, RuleTransformer} -import org.apache.commons.lang3.StringEscapeUtils - import org.apache.spark.internal.Logging import org.apache.spark.ui.scope.RDDOperationGraph @@ -38,8 +36,6 @@ private[spark] object UIUtils extends Logging { val TABLE_CLASS_STRIPED = TABLE_CLASS_NOT_STRIPED + " table-striped" val TABLE_CLASS_STRIPED_SORTABLE = TABLE_CLASS_STRIPED + " sortable" - private val NEWLINE_AND_SINGLE_QUOTE_REGEX = raw"(?i)(\r\n|\n|\r|%0D%0A|%0A|%0D|'|%27)".r - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. private val dateFormat = new ThreadLocal[SimpleDateFormat]() { override def initialValue(): SimpleDateFormat = @@ -113,12 +109,12 @@ private[spark] object UIUtils extends Logging { } } // if time is more than a year - return s"$yearString $weekString $dayString" + s"$yearString $weekString $dayString" } catch { case e: Exception => logError("Error converting time to string", e) // if there is some error, return blank string - return "" + "" } } @@ -176,7 +172,7 @@ private[spark] object UIUtils extends Logging { - + @@ -199,12 +195,14 @@ private[spark] object UIUtils extends Logging { def dataTablesHeaderNodes(request: HttpServletRequest): Seq[Node] = { + "/static/jquery.dataTables.1.10.18.min.css")} type="text/css"/> - + + @@ -218,7 +216,6 @@ private[spark] object UIUtils extends Logging { title: String, content: => Seq[Node], activeTab: SparkUITab, - refreshInterval: Option[Int] = None, helpText: Option[String] = None, showVisualization: Boolean = false, useDataTables: Boolean = false): Seq[Node] = { @@ -339,7 +336,7 @@ private[spark] object UIUtils extends Logging { def getHeaderContent(header: String): Seq[Node] = { if (newlinesInHeader) {
      - { header.split("\n").map { case t =>
    • {t}
    • } } + { header.split("\n").map(t =>
    • {t}
    • ) }
    } else { Text(header) @@ -366,7 +363,8 @@ private[spark] object UIUtils extends Logging { skipped: Int, reasonToNumKilled: Map[String, Int], total: Int): Seq[Node] = { - val completeWidth = "width: %s%%".format((completed.toDouble/total)*100) + val ratio = if (total == 0) 100.0 else (completed.toDouble/total)*100 + val completeWidth = "width: %s%%".format(ratio) // started + completed can be > total when there are speculative tasks val boundedStarted = math.min(started, total - completed) val startWidth = "width: %s%%".format((boundedStarted.toDouble/total)*100) @@ -448,7 +446,7 @@ private[spark] object UIUtils extends Logging { * the whole string will rendered as a simple escaped text. * * Note: In terms of security, only anchor tags with root relative links are supported. So any - * attempts to embed links outside Spark UI, or other tags like {@code - } - - private def createExecutorTable(stage: StageData) : Seq[Node] = { - val executorSummary = store.executorSummary(stage.stageId, stage.attemptId) - - executorSummary.toSeq.sortBy(_._1).map { case (k, v) => - val executor = store.asOption(store.executorSummary(k)) - - - - - - - - - {if (hasInput(stage)) { - - }} - {if (hasOutput(stage)) { - - }} - {if (hasShuffleRead(stage)) { - - }} - {if (hasShuffleWrite(stage)) { - - }} - {if (hasBytesSpilled(stage)) { - - - }} - { - if (executor.map(_.isBlacklisted).getOrElse(false)) { - - } else if (v.isBlacklistedForStage) { - - } else { - - } - } - - } - } - -} 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 55444a2c0c9ab..02d5365fb3229 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 @@ -26,7 +26,6 @@ import scala.xml.{Node, NodeSeq, Unparsed, Utility} import org.apache.commons.lang3.StringEscapeUtils import org.apache.spark.JobExecutionStatus -import org.apache.spark.scheduler._ import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1 import org.apache.spark.ui._ @@ -62,7 +61,7 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP val stageId = stage.stageId val attemptId = stage.attemptId val name = stage.name - val status = stage.status.toString + val status = stage.status.toString.toLowerCase(Locale.ROOT) val submissionTime = stage.submissionTime.get.getTime() val completionTime = stage.completionTime.map(_.getTime()) .getOrElse(System.currentTimeMillis()) @@ -184,12 +183,11 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP } def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val jobId = parameterId.toInt - val jobData = store.asOption(store.job(jobId)).getOrElse { + val (jobData, sqlExecutionId) = store.asOption(store.jobWithAssociatedSql(jobId)).getOrElse { val content =

    No information to display for job {jobId}

    @@ -197,26 +195,64 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP return UIUtils.headerSparkPage( request, s"Details for Job $jobId", content, parent) } + val isComplete = jobData.status != JobExecutionStatus.RUNNING val stages = jobData.stageIds.map { stageId => // This could be empty if the listener hasn't received information about the // stage or if the stage information has been garbage collected store.asOption(store.lastStageAttempt(stageId)).getOrElse { new v1.StageData( - v1.StageStatus.PENDING, - stageId, - 0, 0, 0, 0, 0, 0, 0, - 0L, 0L, None, None, None, None, - 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, 0L, - "Unknown", - None, - "Unknown", - null, - Nil, - Nil, - None, - None, - Map()) + status = v1.StageStatus.PENDING, + stageId = stageId, + attemptId = 0, + numTasks = 0, + numActiveTasks = 0, + numCompleteTasks = 0, + numFailedTasks = 0, + numKilledTasks = 0, + numCompletedIndices = 0, + + submissionTime = None, + firstTaskLaunchedTime = None, + completionTime = None, + failureReason = None, + + executorDeserializeTime = 0L, + executorDeserializeCpuTime = 0L, + executorRunTime = 0L, + executorCpuTime = 0L, + resultSize = 0L, + jvmGcTime = 0L, + resultSerializationTime = 0L, + memoryBytesSpilled = 0L, + diskBytesSpilled = 0L, + peakExecutionMemory = 0L, + inputBytes = 0L, + inputRecords = 0L, + outputBytes = 0L, + outputRecords = 0L, + shuffleRemoteBlocksFetched = 0L, + shuffleLocalBlocksFetched = 0L, + shuffleFetchWaitTime = 0L, + shuffleRemoteBytesRead = 0L, + shuffleRemoteBytesReadToDisk = 0L, + shuffleLocalBytesRead = 0L, + shuffleReadBytes = 0L, + shuffleReadRecords = 0L, + shuffleWriteBytes = 0L, + shuffleWriteTime = 0L, + shuffleWriteRecords = 0L, + + name = "Unknown", + description = None, + details = "Unknown", + schedulingPool = null, + + rddIds = Nil, + accumulatorUpdates = Nil, + tasks = None, + executorSummary = None, + killedTasksSummary = Map()) } } @@ -278,6 +314,17 @@ private[ui] class JobPage(parent: JobsTab, store: AppStatusStore) extends WebUIP Status: {jobData.status} + { + if (sqlExecutionId.isDefined) { +
  • + Associated SQL Query: + {{sqlExecutionId.get}} +
  • + } + } { if (jobData.jobGroup.isDefined) {
  • diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala index ff1b75e5c5065..c2644a8eea157 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobsTab.scala @@ -19,9 +19,8 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.collection.JavaConverters._ - import org.apache.spark.JobExecutionStatus +import org.apache.spark.internal.config.SCHEDULER_MODE import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.status.AppStatusStore import org.apache.spark.ui._ @@ -37,7 +36,7 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) store .environmentInfo() .sparkProperties - .contains(("spark.scheduler.mode", SchedulingMode.FAIR.toString)) + .contains((SCHEDULER_MODE.key, SchedulingMode.FAIR.toString)) } def getSparkUser: String = parent.getSparkUser @@ -47,9 +46,7 @@ private[ui] class JobsTab(parent: SparkUI, store: AppStatusStore) def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val jobId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) - jobId.foreach { id => + Option(request.getParameter("id")).map(_.toInt).foreach { id => store.asOption(store.job(id)).foreach { job => if (job.status == JobExecutionStatus.RUNNING) { sc.foreach(_.cancelJob(id)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 22a40101e33df..e10385cbca41b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,15 +22,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.status.PoolData -import org.apache.spark.status.api.v1._ import org.apache.spark.ui.{UIUtils, WebUIPage} /** Page showing specific pool details */ private[ui] class PoolPage(parent: StagesTab) extends WebUIPage("pool") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val poolName = Option(UIUtils.stripXSS(request.getParameter("poolname"))).map { poolname => + val poolName = Option(request.getParameter("poolname")).map { poolname => UIUtils.decodeURLParameter(poolname) }.getOrElse { throw new IllegalArgumentException(s"Missing poolname parameter") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0f74b07a6265c..54f0f8e226791 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -27,6 +27,7 @@ import scala.xml.{Node, Unparsed} import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.internal.config.UI._ import org.apache.spark.scheduler.TaskLocality import org.apache.spark.status._ import org.apache.spark.status.api.v1._ @@ -63,7 +64,7 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We // TODO: We should consider increasing the number of this parameter over time // if we find that it's okay. - private val MAX_TIMELINE_TASKS = parent.conf.getInt("spark.ui.timeline.tasks.maximum", 1000) + private val MAX_TIMELINE_TASKS = parent.conf.get(UI_TIMELINE_TASKS_MAXIMUM) private def getLocalitySummaryString(localitySummary: Map[String, Long]): String = { val names = Map( @@ -80,27 +81,27 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We } def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterAttempt = UIUtils.stripXSS(request.getParameter("attempt")) + val parameterAttempt = request.getParameter("attempt") require(parameterAttempt != null && parameterAttempt.nonEmpty, "Missing attempt parameter") - val parameterTaskPage = UIUtils.stripXSS(request.getParameter("task.page")) - val parameterTaskSortColumn = UIUtils.stripXSS(request.getParameter("task.sort")) - val parameterTaskSortDesc = UIUtils.stripXSS(request.getParameter("task.desc")) - val parameterTaskPageSize = UIUtils.stripXSS(request.getParameter("task.pageSize")) - val parameterTaskPrevPageSize = UIUtils.stripXSS(request.getParameter("task.prevPageSize")) + val parameterTaskSortColumn = request.getParameter("task.sort") + val parameterTaskSortDesc = request.getParameter("task.desc") + val parameterTaskPageSize = request.getParameter("task.pageSize") + + val eventTimelineParameterTaskPage = request.getParameter("task.eventTimelinePageNumber") + val eventTimelineParameterTaskPageSize = request.getParameter("task.eventTimelinePageSize") + var eventTimelineTaskPage = Option(eventTimelineParameterTaskPage).map(_.toInt).getOrElse(1) + var eventTimelineTaskPageSize = Option( + eventTimelineParameterTaskPageSize).map(_.toInt).getOrElse(100) - val taskPage = Option(parameterTaskPage).map(_.toInt).getOrElse(1) val taskSortColumn = Option(parameterTaskSortColumn).map { sortColumn => UIUtils.decodeURLParameter(sortColumn) }.getOrElse("Index") val taskSortDesc = Option(parameterTaskSortDesc).map(_.toBoolean).getOrElse(false) val taskPageSize = Option(parameterTaskPageSize).map(_.toInt).getOrElse(100) - val taskPrevPageSize = Option(parameterTaskPrevPageSize).map(_.toInt).getOrElse(taskPageSize) - val stageId = parameterId.toInt val stageAttemptId = parameterAttempt.toInt @@ -128,12 +129,13 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We return UIUtils.headerSparkPage(request, stageHeader, content, parent) } - val storedTasks = store.taskCount(stageData.stageId, stageData.attemptId) - val numCompleted = stageData.numCompleteTasks - val totalTasksNumStr = if (totalTasks == storedTasks) { - s"$totalTasks" - } else { - s"$totalTasks, showing $storedTasks" + if (eventTimelineTaskPageSize < 1 || eventTimelineTaskPageSize > totalTasks) { + eventTimelineTaskPageSize = totalTasks + } + val eventTimelineTotalPages = + (totalTasks + eventTimelineTaskPageSize - 1) / eventTimelineTaskPageSize + if (eventTimelineTaskPage < 1 || eventTimelineTaskPage > eventTimelineTotalPages) { + eventTimelineTaskPage = 1 } val summary = @@ -155,140 +157,52 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We }} {if (hasOutput(stageData)) {
  • - Output: + Output Size / Records: {s"${Utils.bytesToString(stageData.outputBytes)} / ${stageData.outputRecords}"}
  • }} {if (hasShuffleRead(stageData)) {
  • - Shuffle Read: + Shuffle Read Size / Records: {s"${Utils.bytesToString(stageData.shuffleReadBytes)} / " + s"${stageData.shuffleReadRecords}"}
  • }} {if (hasShuffleWrite(stageData)) {
  • - Shuffle Write: + Shuffle Write Size / Records: {s"${Utils.bytesToString(stageData.shuffleWriteBytes)} / " + s"${stageData.shuffleWriteRecords}"}
  • }} {if (hasBytesSpilled(stageData)) {
  • - Shuffle Spill (Memory): + Spill (Memory): {Utils.bytesToString(stageData.memoryBytesSpilled)}
  • - Shuffle Spill (Disk): + Spill (Disk): {Utils.bytesToString(stageData.diskBytesSpilled)}
  • }} {if (!stageJobIds.isEmpty) {
  • Associated Job Ids: - {stageJobIds.map(jobId => {val detailUrl = "%s/jobs/job/?id=%s".format( - UIUtils.prependBaseUri(request, parent.basePath), jobId) - {s"${jobId}"}    - })} + {stageJobIds.sorted.map { jobId => + val jobURL = "%s/jobs/job/?id=%s" + .format(UIUtils.prependBaseUri(request, parent.basePath), jobId) + {jobId.toString}  + }}
  • }}
    - val showAdditionalMetrics = -
    - - - Show Additional Metrics - - -
    - val stageGraph = parent.store.asOption(parent.store.operationGraphForStage(stageId)) val dagViz = UIUtils.showDagVizForStage(stageId, stageGraph) - val accumulableHeaders: Seq[String] = Seq("Accumulable", "Value") - def accumulableRow(acc: AccumulableInfo): Seq[Node] = { - if (acc.name != null && acc.value != null) { - - } else { - Nil - } - } - val accumulableTable = UIUtils.listingTable( - accumulableHeaders, - accumulableRow, - stageData.accumulatorUpdates.toSeq) - - val page: Int = { - // If the user has changed to a larger page size, then go to page 1 in order to avoid - // IndexOutOfBoundsException. - if (taskPageSize <= taskPrevPageSize) { - taskPage - } else { - 1 - } - } val currentTime = System.currentTimeMillis() - val (taskTable, taskTableHTML) = try { + val taskTable = try { val _taskTable = new TaskPagedTable( stageData, UIUtils.prependBaseUri(request, parent.basePath) + @@ -299,218 +213,42 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We desc = taskSortDesc, store = parent.store ) - (_taskTable, _taskTable.table(page)) + _taskTable } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) => - val errorMessage = -
    -

    Error while rendering stage table:

    -
    -              {Utils.exceptionString(e)}
    -            
    -
    - (null, errorMessage) - } - - val jsForScrollingDownToTaskTable = - - - val metricsSummary = store.taskSummary(stageData.stageId, stageData.attemptId, - Array(0, 0.25, 0.5, 0.75, 1.0)) - - val summaryTable = metricsSummary.map { metrics => - def timeQuantiles(data: IndexedSeq[Double]): Seq[Node] = { - data.map { millis => - - } - } - - def sizeQuantiles(data: IndexedSeq[Double]): Seq[Node] = { - data.map { size => - - } - } - - def sizeQuantilesWithRecords( - data: IndexedSeq[Double], - records: IndexedSeq[Double]) : Seq[Node] = { - data.zip(records).map { case (d, r) => - - } - } - - def titleCell(title: String, tooltip: String): Seq[Node] = { - - } - - def simpleTitleCell(title: String): Seq[Node] = - - val deserializationQuantiles = titleCell("Task Deserialization Time", - ToolTips.TASK_DESERIALIZATION_TIME) ++ timeQuantiles(metrics.executorDeserializeTime) - - val serviceQuantiles = simpleTitleCell("Duration") ++ timeQuantiles(metrics.executorRunTime) - - val gcQuantiles = titleCell("GC Time", ToolTips.GC_TIME) ++ timeQuantiles(metrics.jvmGcTime) - - val serializationQuantiles = titleCell("Result Serialization Time", - ToolTips.RESULT_SERIALIZATION_TIME) ++ timeQuantiles(metrics.resultSerializationTime) - - val gettingResultQuantiles = titleCell("Getting Result Time", ToolTips.GETTING_RESULT_TIME) ++ - timeQuantiles(metrics.gettingResultTime) - - val peakExecutionMemoryQuantiles = titleCell("Peak Execution Memory", - ToolTips.PEAK_EXECUTION_MEMORY) ++ sizeQuantiles(metrics.peakExecutionMemory) - - // The scheduler delay includes the network delay to send the task to the worker - // machine and to send back the result (but not the time to fetch the task result, - // if it needed to be fetched from the block manager on the worker). - val schedulerDelayQuantiles = titleCell("Scheduler Delay", ToolTips.SCHEDULER_DELAY) ++ - timeQuantiles(metrics.schedulerDelay) - - def inputQuantiles: Seq[Node] = { - simpleTitleCell("Input Size / Records") ++ - sizeQuantilesWithRecords(metrics.inputMetrics.bytesRead, metrics.inputMetrics.recordsRead) - } - - def outputQuantiles: Seq[Node] = { - simpleTitleCell("Output Size / Records") ++ - sizeQuantilesWithRecords(metrics.outputMetrics.bytesWritten, - metrics.outputMetrics.recordsWritten) - } - - def shuffleReadBlockedQuantiles: Seq[Node] = { - titleCell("Shuffle Read Blocked Time", ToolTips.SHUFFLE_READ_BLOCKED_TIME) ++ - timeQuantiles(metrics.shuffleReadMetrics.fetchWaitTime) - } - - def shuffleReadTotalQuantiles: Seq[Node] = { - titleCell("Shuffle Read Size / Records", ToolTips.SHUFFLE_READ) ++ - sizeQuantilesWithRecords(metrics.shuffleReadMetrics.readBytes, - metrics.shuffleReadMetrics.readRecords) - } - - def shuffleReadRemoteQuantiles: Seq[Node] = { - titleCell("Shuffle Remote Reads", ToolTips.SHUFFLE_READ_REMOTE_SIZE) ++ - sizeQuantiles(metrics.shuffleReadMetrics.remoteBytesRead) - } - - def shuffleWriteQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle Write Size / Records") ++ - sizeQuantilesWithRecords(metrics.shuffleWriteMetrics.writeBytes, - metrics.shuffleWriteMetrics.writeRecords) - } - - def memoryBytesSpilledQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle spill (memory)") ++ sizeQuantiles(metrics.memoryBytesSpilled) - } - - def diskBytesSpilledQuantiles: Seq[Node] = { - simpleTitleCell("Shuffle spill (disk)") ++ sizeQuantiles(metrics.diskBytesSpilled) - } - - val listings: Seq[Seq[Node]] = Seq( - {serviceQuantiles}, - {schedulerDelayQuantiles}, - - {deserializationQuantiles} - - {gcQuantiles}, - - {serializationQuantiles} - , - {gettingResultQuantiles}, - - {peakExecutionMemoryQuantiles} - , - if (hasInput(stageData)) {inputQuantiles} else Nil, - if (hasOutput(stageData)) {outputQuantiles} else Nil, - if (hasShuffleRead(stageData)) { - - {shuffleReadBlockedQuantiles} - - {shuffleReadTotalQuantiles} - - {shuffleReadRemoteQuantiles} - - } else { - Nil - }, - if (hasShuffleWrite(stageData)) {shuffleWriteQuantiles} else Nil, - if (hasBytesSpilled(stageData)) {memoryBytesSpilledQuantiles} else Nil, - if (hasBytesSpilled(stageData)) {diskBytesSpilledQuantiles} else Nil) - - val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", - "Max") - // The summary table does not use CSS to stripe rows, which doesn't work with hidden - // rows (instead, JavaScript in table.js is used to stripe the non-hidden rows). - UIUtils.listingTable( - quantileHeaders, - identity[Seq[Node]], - listings, - fixedWidth = true, - id = Some("task-summary-table"), - stripeRowsWithCss = false) + null } - val executorTable = new ExecutorTable(stageData, parent.store) - - val maybeAccumulableTable: Seq[Node] = - if (hasAccumulators(stageData)) {

    Accumulators

    ++ accumulableTable } else Seq() - - val aggMetrics = - -

    - - Aggregated Metrics by Executor -

    -
    -
    - {executorTable.toNodeSeq} -
    - val content = summary ++ - dagViz ++ - showAdditionalMetrics ++ + dagViz ++
    ++ makeTimeline( // Only show the tasks in the table - Option(taskTable).map(_.dataSource.tasks).getOrElse(Nil), - currentTime) ++ -

    Summary Metrics for {numCompleted} Completed Tasks

    ++ -
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++ - aggMetrics ++ - maybeAccumulableTable ++ - -

    - - Tasks ({totalTasksNumStr}) -

    -
    ++ -
    - {taskTableHTML ++ jsForScrollingDownToTaskTable} -
    - UIUtils.headerSparkPage(request, stageHeader, content, parent, showVisualization = true) + Option(taskTable).map({ taskPagedTable => + val from = (eventTimelineTaskPage - 1) * eventTimelineTaskPageSize + val to = taskPagedTable.dataSource.dataSize.min( + eventTimelineTaskPage * eventTimelineTaskPageSize) + taskPagedTable.dataSource.sliceData(from, to)}).getOrElse(Nil), currentTime, + eventTimelineTaskPage, eventTimelineTaskPageSize, eventTimelineTotalPages, stageId, + stageAttemptId, totalTasks) ++ +
    + + +
    + UIUtils.headerSparkPage(request, stageHeader, content, parent, showVisualization = true, + useDataTables = true) + } - def makeTimeline(tasks: Seq[TaskData], currentTime: Long): Seq[Node] = { + def makeTimeline( + tasks: Seq[TaskData], + currentTime: Long, + page: Int, + pageSize: Int, + totalPages: Int, + stageId: Int, + stageAttemptId: Int, + totalTasks: Int): Seq[Node] = { val executorsSet = new HashSet[(String, String)] var minLaunchTime = Long.MaxValue var maxFinishTime = Long.MinValue @@ -669,6 +407,31 @@ private[ui] class StagePage(parent: StagesTab, store: AppStatusStore) extends We Enable zooming +
    +
    + + + + + + + + + + + +
    {TIMELINE_LEGEND} ++ @@ -732,8 +495,6 @@ private[ui] class TaskPagedTable( override def pageSizeFormField: String = "task.pageSize" - override def prevPageSizeFormField: String = "task.prevPageSize" - override def pageNumberFormField: String = "task.page" override val dataSource: TaskDataSource = new TaskDataSource( @@ -857,7 +618,7 @@ private[ui] class TaskPagedTable( - + @@ -880,18 +641,22 @@ private[ui] class TaskPagedTable( }} {if (hasInput(stage)) { - metricInfo(task) { m => - val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) - val records = m.inputMetrics.recordsRead - - } + }} {if (hasOutput(stage)) { - metricInfo(task) { m => - val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) - val records = m.outputMetrics.recordsWritten - - } + }} {if (hasShuffleRead(stage)) { + + + + + + + + + + + + + + + @@ -190,16 +240,56 @@ of the most common options to set are: and it is up to the application to avoid exceeding the overhead memory space shared with other non-JVM processes. When PySpark is run in YARN or Kubernetes, this memory is added to executor resource requests. +
    + Note: This feature is dependent on Python's `resource` module; therefore, the behaviors and + limitations are inherited. For instance, Windows does not support resource limiting and actual + resource is not limited on MacOS. + + + + + + + + + + + + + + + @@ -221,7 +311,8 @@ of the most common options to set are: stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. - NOTE: In Spark 1.0 and later this will be overridden by SPARK_LOCAL_DIRS (Standalone), MESOS_SANDBOX (Mesos) or +
    + Note: This will be overridden by SPARK_LOCAL_DIRS (Standalone), MESOS_SANDBOX (Mesos) or LOCAL_DIRS (YARN) environment variables set by the cluster manager. @@ -266,6 +357,39 @@ of the most common options to set are: Only has effect in Spark standalone mode or Mesos cluster deploy mode. + + + + + + + + + + + + + + +
    -
    {k}
    -
    - { - executor.map(_.executorLogs).getOrElse(Map.empty).map { - case (logName, logUrl) => - } - } -
    -
    {executor.map { e => e.hostPort }.getOrElse("CANNOT FIND ADDRESS")}{UIUtils.formatDuration(v.taskTime)}{v.failedTasks + v.succeededTasks + v.killedTasks}{v.failedTasks}{v.killedTasks}{v.succeededTasks} - {s"${Utils.bytesToString(v.inputBytes)} / ${v.inputRecords}"} - - {s"${Utils.bytesToString(v.outputBytes)} / ${v.outputRecords}"} - - {s"${Utils.bytesToString(v.shuffleRead)} / ${v.shuffleReadRecords}"} - - {s"${Utils.bytesToString(v.shuffleWrite)} / ${v.shuffleWriteRecords}"} - - {Utils.bytesToString(v.memoryBytesSpilled)} - - {Utils.bytesToString(v.diskBytesSpilled)} - for applicationfor stagefalse
    {acc.name}{acc.value}
    {UIUtils.formatDuration(millis.toLong)}{Utils.bytesToString(size.toLong)}{s"${Utils.bytesToString(d.toLong)} / ${r.toLong}"} - - {title} - - {title}
    {UIUtils.formatDate(task.launchTime)}{formatDuration(task.duration)}{formatDuration(task.taskMetrics.map(_.executorRunTime))} {UIUtils.formatDuration(AppStatusUtils.schedulerDelay(task))} {accumulatorsInfo(task)}{bytesRead} / {records}{ + metricInfo(task) { m => + val bytesRead = Utils.bytesToString(m.inputMetrics.bytesRead) + val records = m.inputMetrics.recordsRead + Unparsed(s"$bytesRead / $records") + } + }{bytesWritten} / {records}{ + metricInfo(task) { m => + val bytesWritten = Utils.bytesToString(m.outputMetrics.bytesWritten) + val records = m.outputMetrics.recordsWritten + Unparsed(s"$bytesWritten / $records") + } + } @@ -972,7 +737,7 @@ private[ui] class TaskPagedTable( } } -private[ui] object ApiHelper { +private[spark] object ApiHelper { val HEADER_ID = "ID" val HEADER_TASK_INDEX = "Index" @@ -997,8 +762,8 @@ private[ui] object ApiHelper { val HEADER_SHUFFLE_REMOTE_READS = "Shuffle Remote Reads" val HEADER_SHUFFLE_WRITE_TIME = "Write Time" val HEADER_SHUFFLE_WRITE_SIZE = "Shuffle Write Size / Records" - val HEADER_MEM_SPILL = "Shuffle Spill (Memory)" - val HEADER_DISK_SPILL = "Shuffle Spill (Disk)" + val HEADER_MEM_SPILL = "Spill (Memory)" + val HEADER_DISK_SPILL = "Spill (Disk)" val HEADER_ERROR = "Errors" private[ui] val COLUMN_TO_INDEX = Map( @@ -1010,7 +775,9 @@ private[ui] object ApiHelper { HEADER_EXECUTOR -> TaskIndexNames.EXECUTOR, HEADER_HOST -> TaskIndexNames.HOST, HEADER_LAUNCH_TIME -> TaskIndexNames.LAUNCH_TIME, - HEADER_DURATION -> TaskIndexNames.DURATION, + // SPARK-26109: Duration of task as executorRunTime to make it consistent with the + // aggregated tasks summary metrics table and the previous versions of Spark. + HEADER_DURATION -> TaskIndexNames.EXEC_RUN_TIME, HEADER_SCHEDULER_DELAY -> TaskIndexNames.SCHEDULER_DELAY, HEADER_DESER_TIME -> TaskIndexNames.DESER_TIME, HEADER_GC_TIME -> TaskIndexNames.GC_TIME, @@ -1057,8 +824,13 @@ private[ui] object ApiHelper { } def lastStageNameAndDescription(store: AppStatusStore, job: JobData): (String, String) = { - val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)._1) - (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name)) + // Some jobs have only 0 partitions. + if (job.stageIds.isEmpty) { + ("", job.name) + } else { + val stage = store.asOption(store.stageAttempt(job.stageIds.max, 0)._1) + (stage.map(_.name).getOrElse(""), stage.flatMap(_.description).getOrElse(job.name)) + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index d01acdae59c9f..72cedb1e60bb6 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -42,19 +42,14 @@ private[ui] class StageTableBase( isFairScheduler: Boolean, killEnabled: Boolean, isFailedStage: Boolean) { - // stripXSS is called to remove suspicious characters used in XSS attacks - val allParameters = request.getParameterMap.asScala.toMap.map { case (k, v) => - UIUtils.stripXSS(k) -> v.map(UIUtils.stripXSS).toSeq - } - val parameterOtherTable = allParameters.filterNot(_._1.startsWith(stageTag)) + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(stageTag)) .map(para => para._1 + "=" + para._2(0)) - val parameterStagePage = UIUtils.stripXSS(request.getParameter(stageTag + ".page")) - val parameterStageSortColumn = UIUtils.stripXSS(request.getParameter(stageTag + ".sort")) - val parameterStageSortDesc = UIUtils.stripXSS(request.getParameter(stageTag + ".desc")) - val parameterStagePageSize = UIUtils.stripXSS(request.getParameter(stageTag + ".pageSize")) - val parameterStagePrevPageSize = - UIUtils.stripXSS(request.getParameter(stageTag + ".prevPageSize")) + val parameterStagePage = request.getParameter(stageTag + ".page") + val parameterStageSortColumn = request.getParameter(stageTag + ".sort") + val parameterStageSortDesc = request.getParameter(stageTag + ".desc") + val parameterStagePageSize = request.getParameter(stageTag + ".pageSize") val stagePage = Option(parameterStagePage).map(_.toInt).getOrElse(1) val stageSortColumn = Option(parameterStageSortColumn).map { sortColumn => @@ -65,18 +60,7 @@ private[ui] class StageTableBase( stageSortColumn == "Stage Id" ) val stagePageSize = Option(parameterStagePageSize).map(_.toInt).getOrElse(100) - val stagePrevPageSize = Option(parameterStagePrevPageSize).map(_.toInt) - .getOrElse(stagePageSize) - - val page: Int = { - // If the user has changed to a larger page size, then go to page 1 in order to avoid - // IndexOutOfBoundsException. - if (stagePageSize <= stagePrevPageSize) { - stagePage - } else { - 1 - } - } + val currentTime = System.currentTimeMillis() val toNodeSeq = try { @@ -96,7 +80,7 @@ private[ui] class StageTableBase( isFailedStage, parameterOtherTable, request - ).table(page) + ).table(stagePage) } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) =>
    @@ -161,8 +145,6 @@ private[ui] class StagePagedTable( override def pageSizeFormField: String = stageTag + ".pageSize" - override def prevPageSizeFormField: String = stageTag + ".prevPageSize" - override def pageNumberFormField: String = stageTag + ".page" val parameterPath = UIUtils.prependBaseUri(request, basePath) + s"/$subPath/?" + @@ -383,7 +365,7 @@ private[ui] class StagePagedTable( {if (cachedRddInfos.nonEmpty) { Text("RDD: ") ++ cachedRddInfos.map { i => - {i.name} + {i.name} } }}
    {s.details}
    @@ -430,8 +412,6 @@ private[ui] class StageDataSource( } private def stageRow(stageData: v1.StageData): StageTableRowData = { - val description = stageData.description.getOrElse("") - val formattedSubmissionTime = stageData.submissionTime match { case Some(t) => UIUtils.formatDate(t) case None => "Unknown" diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala index 10b032084ce4f..2d222b842be55 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagesTab.scala @@ -19,10 +19,11 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest +import org.apache.spark.internal.config.SCHEDULER_MODE import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.StageStatus -import org.apache.spark.ui.{SparkUI, SparkUITab, UIUtils} +import org.apache.spark.ui.{SparkUI, SparkUITab} /** Web UI showing progress status of all stages in the given SparkContext. */ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) @@ -40,14 +41,12 @@ private[ui] class StagesTab(val parent: SparkUI, val store: AppStatusStore) store .environmentInfo() .sparkProperties - .contains(("spark.scheduler.mode", SchedulingMode.FAIR.toString)) + .contains((SCHEDULER_MODE.key, SchedulingMode.FAIR.toString)) } def handleKillRequest(request: HttpServletRequest): Unit = { if (killEnabled && parent.securityManager.checkModifyPermissions(request.getRemoteUser)) { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val stageId = Option(UIUtils.stripXSS(request.getParameter("id"))).map(_.toInt) - stageId.foreach { id => + Option(request.getParameter("id")).map(_.toInt).foreach { id => store.asOption(store.lastStageAttempt(id)).foreach { stage => val status = stage.status if (status == StageStatus.ACTIVE || status == StageStatus.PENDING) { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 238cd31433660..4bd3bbd2e1ac3 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -31,21 +31,18 @@ import org.apache.spark.util.Utils private[ui] class RDDPage(parent: SparkUITab, store: AppStatusStore) extends WebUIPage("rdd") { def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") - val parameterBlockPage = UIUtils.stripXSS(request.getParameter("block.page")) - val parameterBlockSortColumn = UIUtils.stripXSS(request.getParameter("block.sort")) - val parameterBlockSortDesc = UIUtils.stripXSS(request.getParameter("block.desc")) - val parameterBlockPageSize = UIUtils.stripXSS(request.getParameter("block.pageSize")) - val parameterBlockPrevPageSize = UIUtils.stripXSS(request.getParameter("block.prevPageSize")) + val parameterBlockPage = request.getParameter("block.page") + val parameterBlockSortColumn = request.getParameter("block.sort") + val parameterBlockSortDesc = request.getParameter("block.desc") + val parameterBlockPageSize = request.getParameter("block.pageSize") val blockPage = Option(parameterBlockPage).map(_.toInt).getOrElse(1) val blockSortColumn = Option(parameterBlockSortColumn).getOrElse("Block Name") val blockSortDesc = Option(parameterBlockSortDesc).map(_.toBoolean).getOrElse(false) val blockPageSize = Option(parameterBlockPageSize).map(_.toInt).getOrElse(100) - val blockPrevPageSize = Option(parameterBlockPrevPageSize).map(_.toInt).getOrElse(blockPageSize) val rddId = parameterId.toInt val rddStorageInfo = try { @@ -60,16 +57,6 @@ private[ui] class RDDPage(parent: SparkUITab, store: AppStatusStore) extends Web val workerTable = UIUtils.listingTable(workerHeader, workerRow, rddStorageInfo.dataDistribution.get, id = Some("rdd-storage-by-worker-table")) - // Block table - val page: Int = { - // If the user has changed to a larger page size, then go to page 1 in order to avoid - // IndexOutOfBoundsException. - if (blockPageSize <= blockPrevPageSize) { - blockPage - } else { - 1 - } - } val blockTableHTML = try { val _blockTable = new BlockPagedTable( UIUtils.prependBaseUri(request, parent.basePath) + s"/storage/rdd/?id=${rddId}", @@ -78,7 +65,7 @@ private[ui] class RDDPage(parent: SparkUITab, store: AppStatusStore) extends Web blockSortColumn, blockSortDesc, store.executorList(true)) - _blockTable.table(page) + _blockTable.table(blockPage) } catch { case e @ (_ : IllegalArgumentException | _ : IndexOutOfBoundsException) =>
    {e.getMessage}
    @@ -202,7 +189,7 @@ private[ui] class BlockDataSource( rddPartition.memoryUsed, rddPartition.diskUsed, rddPartition.executors - .map { id => executorIdToAddress.get(id).getOrElse(id) } + .map { id => executorIdToAddress.getOrElse(id, id) } .sorted .mkString(" ")) } @@ -242,8 +229,6 @@ private[ui] class BlockPagedTable( override def pageSizeFormField: String = "block.pageSize" - override def prevPageSizeFormField: String = "block.prevPageSize" - override def pageNumberFormField: String = "block.page" override val dataSource: BlockDataSource = new BlockDataSource( diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 3eb546e336e99..2488197814ffd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -78,7 +78,7 @@ private[ui] class StoragePage(parent: SparkUITab, store: AppStatusStore) extends
    {rdd.id} - {rdd.name} diff --git a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala index bf618b4afbce0..d5b3ce36e742a 100644 --- a/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala +++ b/core/src/main/scala/org/apache/spark/util/AccumulatorV2.scala @@ -485,34 +485,3 @@ class CollectionAccumulator[T] extends AccumulatorV2[T, java.util.List[T]] { _list.addAll(newValue) } } - - -class LegacyAccumulatorWrapper[R, T]( - initialValue: R, - param: org.apache.spark.AccumulableParam[R, T]) extends AccumulatorV2[T, R] { - private[spark] var _value = initialValue // Current value on driver - - @transient private lazy val _zero = param.zero(initialValue) - - override def isZero: Boolean = _value.asInstanceOf[AnyRef].eq(_zero.asInstanceOf[AnyRef]) - - override def copy(): LegacyAccumulatorWrapper[R, T] = { - val acc = new LegacyAccumulatorWrapper(initialValue, param) - acc._value = _value - acc - } - - override def reset(): Unit = { - _value = _zero - } - - override def add(v: T): Unit = _value = param.addAccumulator(_value, v) - - override def merge(other: AccumulatorV2[T, R]): Unit = other match { - case o: LegacyAccumulatorWrapper[R, T] => _value = param.addInPlace(_value, o.value) - case _ => throw new UnsupportedOperationException( - s"Cannot merge ${this.getClass.getName} with ${other.getClass.getName}") - } - - override def value: R = _value -} diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 6c4740c002103..6d6ef5a744204 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -21,10 +21,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import java.lang.invoke.SerializedLambda import scala.collection.mutable.{Map, Set, Stack} -import scala.language.existentials -import org.apache.xbean.asm6.{ClassReader, ClassVisitor, MethodVisitor, Type} -import org.apache.xbean.asm6.Opcodes._ +import org.apache.xbean.asm7.{ClassReader, ClassVisitor, MethodVisitor, Type} +import org.apache.xbean.asm7.Opcodes._ import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.internal.Logging @@ -34,8 +33,6 @@ import org.apache.spark.internal.Logging */ private[spark] object ClosureCleaner extends Logging { - private val isScala2_11 = scala.util.Properties.versionString.contains("2.11") - // Get an ASM class reader for a given class from the JAR that loaded it private[util] def getClassReader(cls: Class[_]): ClassReader = { // Copy data over, before delegating to ClassReader - else we can run out of open file handles. @@ -168,9 +165,6 @@ private[spark] object ClosureCleaner extends Logging { * @param closure the closure to check. */ private def getSerializedLambda(closure: AnyRef): Option[SerializedLambda] = { - if (isScala2_11) { - return None - } val isClosureCandidate = closure.getClass.isSynthetic && closure @@ -314,7 +308,9 @@ private[spark] object ClosureCleaner extends Logging { var outerPairs: List[(Class[_], AnyRef)] = outerClasses.zip(outerObjects).reverse var parent: AnyRef = null if (outerPairs.nonEmpty) { - val (outermostClass, outermostObject) = outerPairs.head + val outermostClass = outerPairs.head._1 + val outermostObject = outerPairs.head._2 + if (isClosure(outermostClass)) { logDebug(s" + outermost object is a closure, so we clone it: ${outermostClass}") } else if (outermostClass.getName.startsWith("$line")) { @@ -378,10 +374,8 @@ private[spark] object ClosureCleaner extends Logging { } else { logDebug(s"Cleaning lambda: ${lambdaFunc.get.getImplMethodName}") - // scalastyle:off classforname - val captClass = Class.forName(lambdaFunc.get.getCapturingClass.replace('/', '.'), - false, Thread.currentThread.getContextClassLoader) - // scalastyle:on classforname + val captClass = Utils.classForName(lambdaFunc.get.getCapturingClass.replace('/', '.'), + initialize = false, noSparkClassLoader = true) // Fail fast if we detect return statements in closures getClassReader(captClass) .accept(new ReturnStatementFinder(Some(lambdaFunc.get.getImplMethodName)), 0) @@ -424,7 +418,7 @@ private[spark] class ReturnStatementInClosureException extends SparkException("Return statements aren't allowed in Spark closures") private class ReturnStatementFinder(targetMethodName: Option[String] = None) - extends ClassVisitor(ASM6) { + extends ClassVisitor(ASM7) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { @@ -438,7 +432,7 @@ private class ReturnStatementFinder(targetMethodName: Option[String] = None) val isTargetMethod = targetMethodName.isEmpty || name == targetMethodName.get || name == targetMethodName.get.stripSuffix("$adapted") - new MethodVisitor(ASM6) { + new MethodVisitor(ASM7) { override def visitTypeInsn(op: Int, tp: String) { if (op == NEW && tp.contains("scala/runtime/NonLocalReturnControl") && isTargetMethod) { throw new ReturnStatementInClosureException @@ -446,7 +440,7 @@ private class ReturnStatementFinder(targetMethodName: Option[String] = None) } } } else { - new MethodVisitor(ASM6) {} + new MethodVisitor(ASM7) {} } } } @@ -470,7 +464,7 @@ private[util] class FieldAccessFinder( findTransitively: Boolean, specificMethod: Option[MethodIdentifier[_]] = None, visitedMethods: Set[MethodIdentifier[_]] = Set.empty) - extends ClassVisitor(ASM6) { + extends ClassVisitor(ASM7) { override def visitMethod( access: Int, @@ -485,7 +479,7 @@ private[util] class FieldAccessFinder( return null } - new MethodVisitor(ASM6) { + new MethodVisitor(ASM7) { override def visitFieldInsn(op: Int, owner: String, name: String, desc: String) { if (op == GETFIELD) { for (cl <- fields.keys if cl.getName == owner.replace('/', '.')) { @@ -525,7 +519,7 @@ private[util] class FieldAccessFinder( } } -private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM6) { +private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM7) { var myName: String = null // TODO: Recursively find inner closures that we indirectly reference, e.g. @@ -540,19 +534,15 @@ private class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { - new MethodVisitor(ASM6) { + new MethodVisitor(ASM7) { override def visitMethodInsn( op: Int, owner: String, name: String, desc: String, itf: Boolean) { val argTypes = Type.getArgumentTypes(desc) if (op == INVOKESPECIAL && name == "" && argTypes.length > 0 && argTypes(0).toString.startsWith("L") // is it an object? && argTypes(0).getInternalName == myName) { - // scalastyle:off classforname - output += Class.forName( - owner.replace('/', '.'), - false, - Thread.currentThread.getContextClassLoader) - // scalastyle:on classforname + output += Utils.classForName(owner.replace('/', '.'), + initialize = false, noSparkClassLoader = true) } } } diff --git a/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala index 4b6602b50aa1c..651993b057ab0 100644 --- a/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/CommandLineUtils.scala @@ -19,14 +19,16 @@ package org.apache.spark.util import java.io.PrintStream -import org.apache.spark.SparkException - /** * Contains basic command line parsing functionality and methods to parse some common Spark CLI * options. */ -private[spark] trait CommandLineUtils { +private[spark] trait CommandLineUtils extends CommandLineLoggingUtils { + + def main(args: Array[String]): Unit +} +private[spark] trait CommandLineLoggingUtils { // Exposed for testing private[spark] var exitFn: Int => Unit = (exitCode: Int) => System.exit(exitCode) @@ -41,6 +43,4 @@ private[spark] trait CommandLineUtils { printMessage("Run with --help for usage help or --verbose for debug output") exitFn(1) } - - def main(args: Array[String]): Unit } diff --git a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala index 21acaa95c5645..f4d6c7a28d2e4 100644 --- a/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/CompletionIterator.scala @@ -25,11 +25,14 @@ private[spark] abstract class CompletionIterator[ +A, +I <: Iterator[A]](sub: I) extends Iterator[A] { private[this] var completed = false - def next(): A = sub.next() + private[this] var iter = sub + def next(): A = iter.next() def hasNext: Boolean = { - val r = sub.hasNext + val r = iter.hasNext if (!r && !completed) { completed = true + // reassign to release resources of highly resource consuming iterators early + iter = Iterator.empty.asInstanceOf[I] completion() } r diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 950b69f7db641..240dcfbab60ac 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -31,7 +31,7 @@ import scala.collection.immutable.IndexedSeq */ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) - def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) + def this(data: Iterable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) val length = endIdx - startIdx @@ -42,7 +42,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) + def getQuantiles(probabilities: Iterable[Double] = defaultProbabilities) : IndexedSeq[Double] = { probabilities.toIndexedSeq.map { p: Double => data(closestIndex(p)) } } @@ -75,7 +75,7 @@ private[spark] class Distribution(val data: Array[Double], val startIdx: Int, va private[spark] object Distribution { - def apply(data: Traversable[Double]): Option[Distribution] = { + def apply(data: Iterable[Double]): Option[Distribution] = { if (data.size > 0) { Some(new Distribution(data)) } else { @@ -83,7 +83,7 @@ private[spark] object Distribution { } } - def showQuantiles(out: PrintStream = System.out, quantiles: Traversable[Double]) { + def showQuantiles(out: PrintStream = System.out, quantiles: Iterable[Double]) { // scalastyle:off println out.println("min\t25%\t50%\t75%\tmax") quantiles.foreach{q => out.print(q + "\t")} diff --git a/core/src/main/scala/org/apache/spark/util/EventLoop.scala b/core/src/main/scala/org/apache/spark/util/EventLoop.scala index 651ea4996f6cb..5125adc9f7ca6 100644 --- a/core/src/main/scala/org/apache/spark/util/EventLoop.scala +++ b/core/src/main/scala/org/apache/spark/util/EventLoop.scala @@ -100,7 +100,13 @@ private[spark] abstract class EventLoop[E](name: String) extends Logging { * Put the event into the event queue. The event thread will process it later. */ def post(event: E): Unit = { - eventQueue.put(event) + if (!stopped.get) { + if (eventThread.isAlive) { + eventQueue.put(event) + } else { + onError(new IllegalStateException(s"$name has already been stopped accidentally.")) + } + } } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 0cd8612b8fd1c..bab1d5f0ee7c8 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -171,11 +171,13 @@ private[spark] object JsonProtocol { val environmentDetails = environmentUpdate.environmentDetails val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) + val hadoopProperties = mapToJson(environmentDetails("Hadoop Properties").toMap) val systemProperties = mapToJson(environmentDetails("System Properties").toMap) val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) ("Event" -> SPARK_LISTENER_EVENT_FORMATTED_CLASS_NAMES.environmentUpdate) ~ ("JVM Information" -> jvmInformation) ~ ("Spark Properties" -> sparkProperties) ~ + ("Hadoop Properties" -> hadoopProperties) ~ ("System Properties" -> systemProperties) ~ ("Classpath Entries" -> classpathEntries) } @@ -209,7 +211,8 @@ private[spark] object JsonProtocol { ("Timestamp" -> applicationStart.time) ~ ("User" -> applicationStart.sparkUser) ~ ("App Attempt ID" -> applicationStart.appAttemptId.map(JString(_)).getOrElse(JNothing)) ~ - ("Driver Logs" -> applicationStart.driverLogs.map(mapToJson).getOrElse(JNothing)) + ("Driver Logs" -> applicationStart.driverLogs.map(mapToJson).getOrElse(JNothing)) ~ + ("Driver Attributes" -> applicationStart.driverAttributes.map(mapToJson).getOrElse(JNothing)) } def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { @@ -306,7 +309,7 @@ private[spark] object JsonProtocol { private lazy val accumulableBlacklist = Set("internal.metrics.updatedBlockStatuses") - def accumulablesToJson(accumulables: Traversable[AccumulableInfo]): JArray = { + def accumulablesToJson(accumulables: Iterable[AccumulableInfo]): JArray = { JArray(accumulables .filterNot(_.name.exists(accumulableBlacklist.contains)) .toList.map(accumulableInfoToJson)) @@ -394,10 +397,10 @@ private[spark] object JsonProtocol { /** Convert executor metrics to JSON. */ def executorMetricsToJson(executorMetrics: ExecutorMetrics): JValue = { - val metrics = ExecutorMetricType.values.map{ metricType => - JField(metricType.name, executorMetrics.getMetricValue(metricType)) - } - JObject(metrics: _*) + val metrics = ExecutorMetricType.metricToOffset.map { case (m, _) => + JField(m, executorMetrics.getMetricValue(m)) + } + JObject(metrics.toSeq: _*) } def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { @@ -484,7 +487,8 @@ private[spark] object JsonProtocol { def executorInfoToJson(executorInfo: ExecutorInfo): JValue = { ("Host" -> executorInfo.executorHost) ~ ("Total Cores" -> executorInfo.totalCores) ~ - ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) + ("Log Urls" -> mapToJson(executorInfo.logUrlMap)) ~ + ("Attributes" -> mapToJson(executorInfo.attributes)) } def blockUpdatedInfoToJson(blockUpdatedInfo: BlockUpdatedInfo): JValue = { @@ -611,10 +615,10 @@ private[spark] object JsonProtocol { /** Extract the executor metrics from JSON. */ def executorMetricsFromJson(json: JValue): ExecutorMetrics = { val metrics = - ExecutorMetricType.values.map { metric => - metric.name -> jsonOption(json \ metric.name).map(_.extract[Long]).getOrElse(0L) - }.toMap - new ExecutorMetrics(metrics) + ExecutorMetricType.metricToOffset.map { case (metric, _) => + metric -> jsonOption(json \ metric).map(_.extract[Long]).getOrElse(0L) + } + new ExecutorMetrics(metrics.toMap) } def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { @@ -653,9 +657,13 @@ private[spark] object JsonProtocol { } def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { + // For compatible with previous event logs + val hadoopProperties = jsonOption(json \ "Hadoop Properties").map(mapFromJson(_).toSeq) + .getOrElse(Seq.empty) val environmentDetails = Map[String, Seq[(String, String)]]( "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "Hadoop Properties" -> hadoopProperties, "System Properties" -> mapFromJson(json \ "System Properties").toSeq, "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) SparkListenerEnvironmentUpdate(environmentDetails) @@ -687,7 +695,9 @@ private[spark] object JsonProtocol { val sparkUser = (json \ "User").extract[String] val appAttemptId = jsonOption(json \ "App Attempt ID").map(_.extract[String]) val driverLogs = jsonOption(json \ "Driver Logs").map(mapFromJson) - SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId, driverLogs) + val driverAttributes = jsonOption(json \ "Driver Attributes").map(mapFromJson) + SparkListenerApplicationStart(appName, appId, time, sparkUser, appAttemptId, driverLogs, + driverAttributes) } def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { @@ -1055,7 +1065,11 @@ private[spark] object JsonProtocol { val executorHost = (json \ "Host").extract[String] val totalCores = (json \ "Total Cores").extract[Int] val logUrls = mapFromJson(json \ "Log Urls").toMap - new ExecutorInfo(executorHost, totalCores, logUrls) + val attributes = jsonOption(json \ "Attributes") match { + case Some(attr) => mapFromJson(attr).toMap + case None => Map.empty[String, String] + } + new ExecutorInfo(executorHost, totalCores, logUrls, attributes) } def blockUpdatedInfoFromJson(json: JValue): BlockUpdatedInfo = { diff --git a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala index a8f10684d5a2c..2e517707ff774 100644 --- a/core/src/main/scala/org/apache/spark/util/ListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/util/ListenerBus.scala @@ -60,6 +60,14 @@ private[spark] trait ListenerBus[L <: AnyRef, E] extends Logging { } } + /** + * Remove all listeners and they won't receive any events. This method is thread-safe and can be + * called in any thread. + */ + final def removeAllListeners(): Unit = { + listenersPlusTimers.clear() + } + /** * This can be overridden by subclasses if there is any extra cleanup to do when removing a * listener. In particular AsyncEventQueues can clean up queues in the LiveListenerBus. diff --git a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala b/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala deleted file mode 100644 index 034826c57ef1d..0000000000000 --- a/core/src/main/scala/org/apache/spark/util/MutableURLClassLoader.scala +++ /dev/null @@ -1,75 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.util - -import java.net.{URL, URLClassLoader} -import java.util.Enumeration - -import scala.collection.JavaConverters._ - -/** - * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. - */ -private[spark] class MutableURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) { - - override def addURL(url: URL): Unit = { - super.addURL(url) - } - - override def getURLs(): Array[URL] = { - super.getURLs() - } - -} - -/** - * A mutable class loader that gives preference to its own URLs over the parent class loader - * when loading classes and resources. - */ -private[spark] class ChildFirstURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends MutableURLClassLoader(urls, null) { - - private val parentClassLoader = new ParentClassLoader(parent) - - override def loadClass(name: String, resolve: Boolean): Class[_] = { - try { - super.loadClass(name, resolve) - } catch { - case e: ClassNotFoundException => - parentClassLoader.loadClass(name, resolve) - } - } - - override def getResource(name: String): URL = { - val url = super.findResource(name) - val res = if (url != null) url else parentClassLoader.getResource(name) - res - } - - override def getResources(name: String): Enumeration[URL] = { - val childUrls = super.findResources(name).asScala - val parentUrls = parentClassLoader.getResources(name).asScala - (childUrls ++ parentUrls).asJavaEnumeration - } - - override def addURL(url: URL) { - super.addURL(url) - } - -} diff --git a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala index ce06e18879a49..c105f3229af09 100644 --- a/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala +++ b/core/src/main/scala/org/apache/spark/util/PeriodicCheckpointer.scala @@ -100,7 +100,7 @@ private[spark] abstract class PeriodicCheckpointer[T]( var canDelete = true while (checkpointQueue.size > 1 && canDelete) { // Delete the oldest checkpoint only if the next checkpoint exists. - if (isCheckpointed(checkpointQueue.head)) { + if (isCheckpointed(checkpointQueue(1))) { removeCheckpointFile() } else { canDelete = false diff --git a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala index e5cccf39f9455..7272b375e5388 100644 --- a/core/src/main/scala/org/apache/spark/util/RpcUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/RpcUtils.scala @@ -18,6 +18,8 @@ package org.apache.spark.util import org.apache.spark.SparkConf +import org.apache.spark.internal.config +import org.apache.spark.internal.config.Network._ import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv, RpcTimeout} private[spark] object RpcUtils { @@ -26,40 +28,40 @@ private[spark] object RpcUtils { * Retrieve a `RpcEndpointRef` which is located in the driver via its name. */ def makeDriverRef(name: String, conf: SparkConf, rpcEnv: RpcEnv): RpcEndpointRef = { - val driverHost: String = conf.get("spark.driver.host", "localhost") - val driverPort: Int = conf.getInt("spark.driver.port", 7077) + val driverHost: String = conf.get(config.DRIVER_HOST_ADDRESS.key, "localhost") + val driverPort: Int = conf.getInt(config.DRIVER_PORT.key, 7077) Utils.checkHost(driverHost) rpcEnv.setupEndpointRef(RpcAddress(driverHost, driverPort), name) } /** Returns the configured number of times to retry connecting */ def numRetries(conf: SparkConf): Int = { - conf.getInt("spark.rpc.numRetries", 3) + conf.get(RPC_NUM_RETRIES) } /** Returns the configured number of milliseconds to wait on each retry */ def retryWaitMs(conf: SparkConf): Long = { - conf.getTimeAsMs("spark.rpc.retry.wait", "3s") + conf.get(RPC_RETRY_WAIT) } /** Returns the default Spark timeout to use for RPC ask operations. */ def askRpcTimeout(conf: SparkConf): RpcTimeout = { - RpcTimeout(conf, Seq("spark.rpc.askTimeout", "spark.network.timeout"), "120s") + RpcTimeout(conf, Seq(RPC_ASK_TIMEOUT.key, NETWORK_TIMEOUT.key), "120s") } /** Returns the default Spark timeout to use for RPC remote endpoint lookup. */ def lookupRpcTimeout(conf: SparkConf): RpcTimeout = { - RpcTimeout(conf, Seq("spark.rpc.lookupTimeout", "spark.network.timeout"), "120s") + RpcTimeout(conf, Seq(RPC_LOOKUP_TIMEOUT.key, NETWORK_TIMEOUT.key), "120s") } private val MAX_MESSAGE_SIZE_IN_MB = Int.MaxValue / 1024 / 1024 /** Returns the configured max message size for messages in bytes. */ def maxMessageSizeBytes(conf: SparkConf): Int = { - val maxSizeInMB = conf.getInt("spark.rpc.message.maxSize", 128) + val maxSizeInMB = conf.get(RPC_MESSAGE_MAX_SIZE) if (maxSizeInMB > MAX_MESSAGE_SIZE_IN_MB) { throw new IllegalArgumentException( - s"spark.rpc.message.maxSize should not be greater than $MAX_MESSAGE_SIZE_IN_MB MB") + s"${RPC_MESSAGE_MAX_SIZE.key} should not be greater than $MAX_MESSAGE_SIZE_IN_MB MB") } maxSizeInMB * 1024 * 1024 } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index 3bfdf95db84c6..09c69f5c68b03 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -28,12 +28,13 @@ import com.google.common.collect.MapMaker import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Tests.TEST_USE_COMPRESSED_OOPS_KEY import org.apache.spark.util.collection.OpenHashSet /** * A trait that allows a class to give [[SizeEstimator]] more accurate size estimation. * When a class extends it, [[SizeEstimator]] will query the `estimatedSize` first. - * If `estimatedSize` does not return [[None]], [[SizeEstimator]] will use the returned size + * If `estimatedSize` does not return `None`, [[SizeEstimator]] will use the returned size * as the size of the object. Otherwise, [[SizeEstimator]] will do the estimation work. * The difference between a [[KnownSizeEstimation]] and * [[org.apache.spark.util.collection.SizeTracker]] is that, a @@ -126,12 +127,13 @@ object SizeEstimator extends Logging { private def getIsCompressedOops: Boolean = { // This is only used by tests to override the detection of compressed oops. The test // actually uses a system property instead of a SparkConf, so we'll stick with that. - if (System.getProperty("spark.test.useCompressedOops") != null) { - return System.getProperty("spark.test.useCompressedOops").toBoolean + if (System.getProperty(TEST_USE_COMPRESSED_OOPS_KEY) != null) { + return System.getProperty(TEST_USE_COMPRESSED_OOPS_KEY).toBoolean } - // java.vm.info provides compressed ref info for IBM JDKs - if (System.getProperty("java.vendor").contains("IBM")) { + // java.vm.info provides compressed ref info for IBM and OpenJ9 JDKs + val javaVendor = System.getProperty("java.vendor") + if (javaVendor.contains("IBM") || javaVendor.contains("OpenJ9")) { return System.getProperty("java.vm.info").contains("Compressed Ref") } @@ -333,9 +335,21 @@ object SizeEstimator extends Logging { if (fieldClass.isPrimitive) { sizeCount(primitiveSize(fieldClass)) += 1 } else { - field.setAccessible(true) // Enable future get()'s on this field + // Note: in Java 9+ this would be better with trySetAccessible and canAccess + try { + field.setAccessible(true) // Enable future get()'s on this field + pointerFields = field :: pointerFields + } catch { + // If the field isn't accessible, we can still record the pointer size + // but can't know more about the field, so ignore it + case _: SecurityException => + // do nothing + // Java 9+ can throw InaccessibleObjectException but the class is Java 9+-only + case re: RuntimeException + if re.getClass.getSimpleName == "InaccessibleObjectException" => + // do nothing + } sizeCount(pointerSize) += 1 - pointerFields = field :: pointerFields } } } diff --git a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala index cb0c20541d0d7..04b0b4c37df9e 100644 --- a/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/ThreadUtils.scala @@ -26,7 +26,6 @@ import scala.language.higherKinds import com.google.common.util.concurrent.{MoreExecutors, ThreadFactoryBuilder} import scala.concurrent.{Awaitable, ExecutionContext, ExecutionContextExecutor, Future} import scala.concurrent.duration.{Duration, FiniteDuration} -import scala.concurrent.forkjoin.{ForkJoinPool => SForkJoinPool, ForkJoinWorkerThread => SForkJoinWorkerThread} import scala.util.control.NonFatal import org.apache.spark.SparkException @@ -181,17 +180,17 @@ private[spark] object ThreadUtils { } /** - * Construct a new Scala ForkJoinPool with a specified max parallelism and name prefix. + * Construct a new ForkJoinPool with a specified max parallelism and name prefix. */ - def newForkJoinPool(prefix: String, maxThreadNumber: Int): SForkJoinPool = { + def newForkJoinPool(prefix: String, maxThreadNumber: Int): ForkJoinPool = { // Custom factory to set thread names - val factory = new SForkJoinPool.ForkJoinWorkerThreadFactory { - override def newThread(pool: SForkJoinPool) = - new SForkJoinWorkerThread(pool) { + val factory = new ForkJoinPool.ForkJoinWorkerThreadFactory { + override def newThread(pool: ForkJoinPool) = + new ForkJoinWorkerThread(pool) { setName(prefix + "-" + super.getName) } } - new SForkJoinPool(maxThreadNumber, factory, + new ForkJoinPool(maxThreadNumber, factory, null, // handler false // asyncMode ) 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 93b5826f8a74b..80d70a1d48504 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -31,7 +31,6 @@ import java.security.SecureRandom import java.util.{Locale, Properties, Random, UUID} import java.util.concurrent._ import java.util.concurrent.TimeUnit.NANOSECONDS -import java.util.concurrent.atomic.AtomicBoolean import java.util.zip.GZIPInputStream import scala.annotation.tailrec @@ -61,10 +60,15 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.{config, Logging} import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Streaming._ +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.network.util.JavaUtils import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.status.api.v1.{StackTrace, ThreadStackTrace} +import org.apache.spark.util.io.ChunkedByteBufferOutputStream /** CallSite represents a place in user code. It can have a short and a long form. */ private[spark] case class CallSite(shortForm: String, longForm: String) @@ -93,52 +97,10 @@ private[spark] object Utils extends Logging { private val MAX_DIR_CREATION_ATTEMPTS: Int = 10 @volatile private var localRootDirs: Array[String] = null - /** - * The performance overhead of creating and logging strings for wide schemas can be large. To - * limit the impact, we bound the number of fields to include by default. This can be overridden - * by setting the 'spark.debug.maxToStringFields' conf in SparkEnv. - */ - val DEFAULT_MAX_TO_STRING_FIELDS = 25 - - private[spark] def maxNumToStringFields = { - if (SparkEnv.get != null) { - SparkEnv.get.conf.getInt("spark.debug.maxToStringFields", DEFAULT_MAX_TO_STRING_FIELDS) - } else { - DEFAULT_MAX_TO_STRING_FIELDS - } - } + /** Scheme used for files that are locally available on worker nodes in the cluster. */ + val LOCAL_SCHEME = "local" - /** Whether we have warned about plan string truncation yet. */ - private val truncationWarningPrinted = new AtomicBoolean(false) - - /** - * Format a sequence with semantics similar to calling .mkString(). Any elements beyond - * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. - * - * @return the trimmed and formatted string. - */ - def truncatedString[T]( - seq: Seq[T], - start: String, - sep: String, - end: String, - maxNumFields: Int = maxNumToStringFields): String = { - if (seq.length > maxNumFields) { - if (truncationWarningPrinted.compareAndSet(false, true)) { - logWarning( - "Truncated the string representation of a plan since it was too large. This " + - "behavior can be adjusted by setting 'spark.debug.maxToStringFields' in SparkEnv.conf.") - } - val numFields = math.max(0, maxNumFields - 1) - seq.take(numFields).mkString( - start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) - } else { - seq.mkString(start, sep, end) - } - } - - /** Shorthand for calling truncatedString() without start or end strings. */ - def truncatedString[T](seq: Seq[T], sep: String): String = truncatedString(seq, "", sep, "") + private val PATTERN_FOR_COMMAND_LINE_ARG = "-D(.+?)=(.+)".r /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { @@ -227,15 +189,24 @@ private[spark] object Utils extends Logging { /** Determines whether the provided class is loadable in the current thread. */ def classIsLoadable(clazz: String): Boolean = { - // scalastyle:off classforname - Try { Class.forName(clazz, false, getContextOrSparkClassLoader) }.isSuccess - // scalastyle:on classforname + Try { classForName(clazz, initialize = false) }.isSuccess } // scalastyle:off classforname - /** Preferred alternative to Class.forName(className) */ - def classForName(className: String): Class[_] = { - Class.forName(className, true, getContextOrSparkClassLoader) + /** + * Preferred alternative to Class.forName(className), as well as + * Class.forName(className, initialize, loader) with current thread's ContextClassLoader. + */ + def classForName[C]( + className: String, + initialize: Boolean = true, + noSparkClassLoader: Boolean = false): Class[C] = { + if (!noSparkClassLoader) { + Class.forName(className, initialize, getContextOrSparkClassLoader).asInstanceOf[Class[C]] + } else { + Class.forName(className, initialize, Thread.currentThread().getContextClassLoader). + asInstanceOf[Class[C]] + } // scalastyle:on classforname } @@ -377,6 +348,50 @@ private[spark] object Utils extends Logging { } } + /** + * Copy the first `maxSize` bytes of data from the InputStream to an in-memory + * buffer, primarily to check for corruption. + * + * This returns a new InputStream which contains the same data as the original input stream. + * It may be entirely on in-memory buffer, or it may be a combination of in-memory data, and then + * continue to read from the original stream. The only real use of this is if the original input + * stream will potentially detect corruption while the data is being read (eg. from compression). + * This allows for an eager check of corruption in the first maxSize bytes of data. + * + * @return An InputStream which includes all data from the original stream (combining buffered + * data and remaining data in the original stream) + */ + def copyStreamUpTo(in: InputStream, maxSize: Long): InputStream = { + var count = 0L + val out = new ChunkedByteBufferOutputStream(64 * 1024, ByteBuffer.allocate) + val fullyCopied = tryWithSafeFinally { + val bufSize = Math.min(8192L, maxSize) + val buf = new Array[Byte](bufSize.toInt) + var n = 0 + while (n != -1 && count < maxSize) { + n = in.read(buf, 0, Math.min(maxSize - count, bufSize).toInt) + if (n != -1) { + out.write(buf, 0, n) + count += n + } + } + count < maxSize + } { + try { + if (count < maxSize) { + in.close() + } + } finally { + out.close() + } + } + if (fullyCopied) { + out.toChunkedByteBuffer.toInputStream(dispose = true) + } else { + new SequenceInputStream( out.toChunkedByteBuffer.toInputStream(dispose = true), in) + } + } + def copyFileStreamNIO( input: FileChannel, output: FileChannel, @@ -408,22 +423,6 @@ private[spark] object Utils extends Logging { """.stripMargin) } - /** - * Construct a URI container information used for authentication. - * This also sets the default authenticator to properly negotiation the - * user/password based on the URI. - * - * Note this relies on the Authenticator.setDefault being set properly to decode - * the user name and password. This is currently set in the SecurityManager. - */ - def constructURIForAuthentication(uri: URI, securityMgr: SecurityManager): URI = { - val userCred = securityMgr.getSecretKey() - if (userCred == null) throw new Exception("Secret key is null with authentication on") - val userInfo = securityMgr.getHttpUser() + ":" + userCred - new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), - uri.getQuery(), uri.getFragment()) - } - /** * A file name may contain some invalid URI characters, such as " ". This method will convert the * file name to a raw path accepted by `java.net.URI(String)`. @@ -694,17 +693,7 @@ private[spark] object Utils extends Logging { val is = Channels.newInputStream(source) downloadFile(url, is, targetFile, fileOverwrite) case "http" | "https" | "ftp" => - var uc: URLConnection = null - if (securityMgr.isAuthenticationEnabled()) { - logDebug("fetchFile with security enabled") - val newuri = constructURIForAuthentication(uri, securityMgr) - uc = newuri.toURL().openConnection() - uc.setAllowUserInteraction(false) - } else { - logDebug("fetchFile not using security") - uc = new URL(url).openConnection() - } - + val uc = new URL(url).openConnection() val timeoutMs = conf.getTimeAsSeconds("spark.files.fetchTimeout", "60s").toInt * 1000 uc.setConnectTimeout(timeoutMs) @@ -854,7 +843,7 @@ private[spark] object Utils extends Logging { } else { if (conf.getenv("MESOS_SANDBOX") != null && shuffleServiceEnabled) { logInfo("MESOS_SANDBOX available but not using provided Mesos sandbox because " + - "spark.shuffle.service.enabled is enabled.") + s"${config.SHUFFLE_SERVICE_ENABLED.key} is enabled.") } // In non-Yarn mode (or for the driver in yarn-client mode), we cannot trust the user // configuration to point to a secure directory. So create a subdirectory with restricted @@ -1047,9 +1036,10 @@ private[spark] object Utils extends Logging { /** * Return the string to tell how long has passed in milliseconds. + * @param startTimeNs - a timestamp in nanoseconds returned by `System.nanoTime`. */ - def getUsedTimeMs(startTimeMs: Long): String = { - " " + (System.currentTimeMillis - startTimeMs) + " ms" + def getUsedTimeNs(startTimeNs: Long): String = { + s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms" } /** @@ -1085,7 +1075,7 @@ private[spark] object Utils extends Logging { } /** - * Convert a time parameter such as (50s, 100ms, or 250us) to microseconds for internal use. If + * Convert a time parameter such as (50s, 100ms, or 250us) to milliseconds for internal use. If * no suffix is provided, the passed number is assumed to be in ms. */ def timeStringAsMs(str: String): Long = { @@ -1140,41 +1130,41 @@ private[spark] object Utils extends Logging { * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of mebibytes. */ def memoryStringToMb(str: String): Int = { - // Convert to bytes, rather than directly to MB, because when no units are specified the unit + // Convert to bytes, rather than directly to MiB, because when no units are specified the unit // is assumed to be bytes (JavaUtils.byteStringAsBytes(str) / 1024 / 1024).toInt } /** - * Convert a quantity in bytes to a human-readable string such as "4.0 MB". + * Convert a quantity in bytes to a human-readable string such as "4.0 MiB". */ def bytesToString(size: Long): String = bytesToString(BigInt(size)) def bytesToString(size: BigInt): String = { - val EB = 1L << 60 - val PB = 1L << 50 - val TB = 1L << 40 - val GB = 1L << 30 - val MB = 1L << 20 - val KB = 1L << 10 - - if (size >= BigInt(1L << 11) * EB) { + val EiB = 1L << 60 + val PiB = 1L << 50 + val TiB = 1L << 40 + val GiB = 1L << 30 + val MiB = 1L << 20 + val KiB = 1L << 10 + + if (size >= BigInt(1L << 11) * EiB) { // The number is too large, show it in scientific notation. BigDecimal(size, new MathContext(3, RoundingMode.HALF_UP)).toString() + " B" } else { val (value, unit) = { - if (size >= 2 * EB) { - (BigDecimal(size) / EB, "EB") - } else if (size >= 2 * PB) { - (BigDecimal(size) / PB, "PB") - } else if (size >= 2 * TB) { - (BigDecimal(size) / TB, "TB") - } else if (size >= 2 * GB) { - (BigDecimal(size) / GB, "GB") - } else if (size >= 2 * MB) { - (BigDecimal(size) / MB, "MB") - } else if (size >= 2 * KB) { - (BigDecimal(size) / KB, "KB") + if (size >= 2 * EiB) { + (BigDecimal(size) / EiB, "EiB") + } else if (size >= 2 * PiB) { + (BigDecimal(size) / PiB, "PiB") + } else if (size >= 2 * TiB) { + (BigDecimal(size) / TiB, "TiB") + } else if (size >= 2 * GiB) { + (BigDecimal(size) / GiB, "GiB") + } else if (size >= 2 * MiB) { + (BigDecimal(size) / MiB, "MiB") + } else if (size >= 2 * KiB) { + (BigDecimal(size) / KiB, "KiB") } else { (BigDecimal(size), "B") } @@ -1205,7 +1195,7 @@ private[spark] object Utils extends Logging { } /** - * Convert a quantity in megabytes to a human-readable string such as "4.0 MB". + * Convert a quantity in megabytes to a human-readable string such as "4.0 MiB". */ def megabytesToString(megabytes: Long): String = { bytesToString(megabytes * 1024L * 1024L) @@ -1399,7 +1389,9 @@ private[spark] object Utils extends Logging { originalThrowable = cause try { logError("Aborting task", originalThrowable) - TaskContext.get().markTaskFailed(originalThrowable) + if (TaskContext.get() != null) { + TaskContext.get().markTaskFailed(originalThrowable) + } catchBlock } catch { case t: Throwable => @@ -1500,16 +1492,12 @@ private[spark] object Utils extends Logging { CallSite(shortForm, longForm) } - private val UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF = - "spark.worker.ui.compressedLogFileLengthCacheSize" - private val DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE = 100 private var compressedLogFileLengthCache: LoadingCache[String, java.lang.Long] = null private def getCompressedLogFileLengthCache( sparkConf: SparkConf): LoadingCache[String, java.lang.Long] = this.synchronized { if (compressedLogFileLengthCache == null) { - val compressedLogFileLengthCacheSize = sparkConf.getInt( - UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF, - DEFAULT_UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE) + val compressedLogFileLengthCacheSize = sparkConf.get( + UNCOMPRESSED_LOG_FILE_LENGTH_CACHE_SIZE_CONF) compressedLogFileLengthCache = CacheBuilder.newBuilder() .maximumSize(compressedLogFileLengthCacheSize) .build[String, java.lang.Long](new CacheLoader[String, java.lang.Long]() { @@ -1784,23 +1772,23 @@ private[spark] object Utils extends Logging { * * @param numIters number of iterations * @param f function to be executed. If prepare is not None, the running time of each call to f - * must be an order of magnitude longer than one millisecond for accurate timing. + * must be an order of magnitude longer than one nanosecond for accurate timing. * @param prepare function to be executed before each call to f. Its running time doesn't count. - * @return the total time across all iterations (not counting preparation time) + * @return the total time across all iterations (not counting preparation time) in nanoseconds. */ def timeIt(numIters: Int)(f: => Unit, prepare: Option[() => Unit] = None): Long = { if (prepare.isEmpty) { - val start = System.currentTimeMillis + val startNs = System.nanoTime() times(numIters)(f) - System.currentTimeMillis - start + System.nanoTime() - startNs } else { var i = 0 var sum = 0L while (i < numIters) { prepare.get.apply() - val start = System.currentTimeMillis + val startNs = System.nanoTime() f - sum += System.currentTimeMillis - start + sum += System.nanoTime() - startNs i += 1 } sum @@ -1892,7 +1880,7 @@ private[spark] object Utils extends Logging { * Indicates whether Spark is currently running unit tests. */ def isTesting: Boolean = { - sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + sys.env.contains("SPARK_TESTING") || sys.props.contains(IS_TESTING.key) } /** @@ -2220,7 +2208,7 @@ private[spark] object Utils extends Logging { */ def portMaxRetries(conf: SparkConf): Int = { val maxRetries = conf.getOption("spark.port.maxRetries").map(_.toInt) - if (conf.contains("spark.testing")) { + if (conf.contains(IS_TESTING)) { // Set a higher number of retries for tests... maxRetries.getOrElse(100) } else { @@ -2276,7 +2264,7 @@ private[spark] object Utils extends Logging { s"${e.getMessage}: Service$serviceString failed after " + s"$maxRetries retries (on a random free port)! " + s"Consider explicitly setting the appropriate binding address for " + - s"the service$serviceString (for example spark.driver.bindAddress " + + s"the service$serviceString (for example ${DRIVER_BIND_ADDRESS.key} " + s"for SparkDriver) to the correct binding address." } else { s"${e.getMessage}: Service$serviceString failed after " + @@ -2328,7 +2316,10 @@ private[spark] object Utils extends Logging { * configure a new log4j level */ def setLogLevel(l: org.apache.log4j.Level) { - org.apache.log4j.Logger.getRootLogger().setLevel(l) + val rootLogger = org.apache.log4j.Logger.getRootLogger() + rootLogger.setLevel(l) + // Setting threshold to null as rootLevel will define log level for spark-shell + Logging.sparkShellThresholdLevel = null } /** @@ -2426,11 +2417,11 @@ private[spark] object Utils extends Logging { // Returns the groups to which the current user belongs. def getCurrentUserGroups(sparkConf: SparkConf, username: String): Set[String] = { - val groupProviderClassName = sparkConf.get("spark.user.groups.mapping", - "org.apache.spark.security.ShellBasedGroupsMappingProvider") + val groupProviderClassName = sparkConf.get(USER_GROUPS_MAPPING) if (groupProviderClassName != "") { try { - val groupMappingServiceProvider = classForName(groupProviderClassName).newInstance. + val groupMappingServiceProvider = classForName(groupProviderClassName). + getConstructor().newInstance(). asInstanceOf[org.apache.spark.security.GroupMappingServiceProvider] val currentUserGroups = groupMappingServiceProvider.getGroups(username) return currentUserGroups @@ -2506,9 +2497,15 @@ private[spark] object Utils extends Logging { * Return whether dynamic allocation is enabled in the given conf. */ def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { - val dynamicAllocationEnabled = conf.getBoolean("spark.dynamicAllocation.enabled", false) + val dynamicAllocationEnabled = conf.get(DYN_ALLOCATION_ENABLED) dynamicAllocationEnabled && - (!isLocalMaster(conf) || conf.getBoolean("spark.dynamicAllocation.testing", false)) + (!isLocalMaster(conf) || conf.get(DYN_ALLOCATION_TESTING)) + } + + def isStreamingDynamicAllocationEnabled(conf: SparkConf): Boolean = { + val streamingDynamicAllocationEnabled = conf.get(STREAMING_DYN_ALLOCATION_ENABLED) + streamingDynamicAllocationEnabled && + (!isLocalMaster(conf) || conf.get(STREAMING_DYN_ALLOCATION_TESTING)) } /** @@ -2573,8 +2570,7 @@ private[spark] object Utils extends Logging { * has its own mechanism to distribute jars. */ def getUserJars(conf: SparkConf): Seq[String] = { - val sparkJars = conf.getOption("spark.jars") - sparkJars.map(_.split(",")).map(_.filter(_.nonEmpty)).toSeq.flatten + conf.get(JARS).filter(_.nonEmpty) } /** @@ -2602,7 +2598,7 @@ private[spark] object Utils extends Logging { * Redact the sensitive values in the given map. If a map key matches the redaction pattern then * its value is replaced with a dummy text. */ - def redact(regex: Option[Regex], kvs: Seq[(String, String)]): Seq[(String, String)] = { + def redact[K, V](regex: Option[Regex], kvs: Seq[(K, V)]): Seq[(K, V)] = { regex match { case None => kvs case Some(r) => redact(r, kvs) @@ -2624,7 +2620,7 @@ private[spark] object Utils extends Logging { } } - private def redact(redactionPattern: Regex, kvs: Seq[(String, String)]): Seq[(String, String)] = { + private def redact[K, V](redactionPattern: Regex, kvs: Seq[(K, V)]): Seq[(K, V)] = { // If the sensitive information regex matches with either the key or the value, redact the value // While the original intent was to only redact the value if the key matched with the regex, // we've found that especially in verbose mode, the value of the property may contain sensitive @@ -2638,12 +2634,19 @@ private[spark] object Utils extends Logging { // arbitrary property contained the term 'password', we may redact the value from the UI and // logs. In order to work around it, user would have to make the spark.redaction.regex property // more specific. - kvs.map { case (key, value) => - redactionPattern.findFirstIn(key) - .orElse(redactionPattern.findFirstIn(value)) - .map { _ => (key, REDACTION_REPLACEMENT_TEXT) } - .getOrElse((key, value)) - } + kvs.map { + case (key: String, value: String) => + redactionPattern.findFirstIn(key) + .orElse(redactionPattern.findFirstIn(value)) + .map { _ => (key, REDACTION_REPLACEMENT_TEXT) } + .getOrElse((key, value)) + case (key, value: String) => + redactionPattern.findFirstIn(value) + .map { _ => (key, REDACTION_REPLACEMENT_TEXT) } + .getOrElse((key, value)) + case (key, value) => + (key, value) + }.asInstanceOf[Seq[(K, V)]] } /** @@ -2660,6 +2663,17 @@ private[spark] object Utils extends Logging { redact(redactionPattern, kvs.toArray) } + def redactCommandLineArgs(conf: SparkConf, commands: Seq[String]): Seq[String] = { + val redactionPattern = conf.get(SECRET_REDACTION_PATTERN) + commands.map { + case PATTERN_FOR_COMMAND_LINE_ARG(key, value) => + val (_, newValue) = redact(redactionPattern, Seq((key, value))).head + s"-D$key=$newValue" + + case cmd => cmd + } + } + def stringToSeq(str: String): Seq[String] = { str.split(",").map(_.trim()).filter(_.nonEmpty) } @@ -2676,10 +2690,11 @@ private[spark] object Utils extends Logging { * other state) and decide they do not need to be added. A log message is printed in that case. * Other exceptions are bubbled up. */ - def loadExtensions[T](extClass: Class[T], classes: Seq[String], conf: SparkConf): Seq[T] = { + def loadExtensions[T <: AnyRef]( + extClass: Class[T], classes: Seq[String], conf: SparkConf): Seq[T] = { classes.flatMap { name => try { - val klass = classForName(name) + val klass = classForName[T](name) require(extClass.isAssignableFrom(klass), s"$name is not a subclass of ${extClass.getName()}.") @@ -2779,13 +2794,17 @@ private[spark] object Utils extends Logging { /** * Safer than Class obj's getSimpleName which may throw Malformed class name error in scala. - * This method mimicks scalatest's getSimpleNameOfAnObjectsClass. + * This method mimics scalatest's getSimpleNameOfAnObjectsClass. */ def getSimpleName(cls: Class[_]): String = { try { - return cls.getSimpleName + cls.getSimpleName } catch { - case err: InternalError => return stripDollars(stripPackages(cls.getName)) + // TODO: the value returned here isn't even quite right; it returns simple names + // like UtilsSuite$MalformedClassObject$MalformedClass instead of MalformedClass + // The exact value may not matter much as it's used in log statements + case _: InternalError => + stripDollars(stripPackages(cls.getName)) } } @@ -2863,6 +2882,19 @@ private[spark] object Utils extends Logging { def stringHalfWidth(str: String): Int = { if (str == null) 0 else str.length + fullWidthRegex.findAllIn(str).size } + + def sanitizeDirName(str: String): String = { + str.replaceAll("[ :/]", "-").replaceAll("[.${}'\"]", "_").toLowerCase(Locale.ROOT) + } + + def isClientMode(conf: SparkConf): Boolean = { + "client".equals(conf.get(SparkLauncher.DEPLOY_MODE, "client")) + } + + /** Returns whether the URI is a "local:" URI. */ + def isLocalUri(uri: String): Boolean = { + uri.startsWith(s"$LOCAL_SCHEME:") + } } private[util] object CallerContext extends Logging { diff --git a/core/src/main/scala/org/apache/spark/util/VersionUtils.scala b/core/src/main/scala/org/apache/spark/util/VersionUtils.scala index 828153b868420..c0f8866dd58dc 100644 --- a/core/src/main/scala/org/apache/spark/util/VersionUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/VersionUtils.scala @@ -23,6 +23,7 @@ package org.apache.spark.util private[spark] object VersionUtils { private val majorMinorRegex = """^(\d+)\.(\d+)(\..*)?$""".r + private val shortVersionRegex = """^(\d+\.\d+\.\d+)(.*)?$""".r /** * Given a Spark version string, return the major version number. @@ -36,6 +37,19 @@ private[spark] object VersionUtils { */ def minorVersion(sparkVersion: String): Int = majorMinorVersion(sparkVersion)._2 + /** + * Given a Spark version string, return the short version string. + * E.g., for 3.0.0-SNAPSHOT, return '3.0.0'. + */ + def shortVersion(sparkVersion: String): String = { + shortVersionRegex.findFirstMatchIn(sparkVersion) match { + case Some(m) => m.group(1) + case None => + throw new IllegalArgumentException(s"Spark tried to parse '$sparkVersion' as a Spark" + + s" version string, but it could not find the major/minor/maintenance version numbers.") + } + } + /** * Given a Spark version string, return the (major version number, minor version number). * E.g., for 2.0.1-SNAPSHOT, return (2, 0). diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index 19ff109b673e1..1ba3b7875f8dc 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -29,7 +29,7 @@ import com.google.common.io.ByteStreams import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer.{DeserializationStream, Serializer, SerializerManager} import org.apache.spark.storage.{BlockId, BlockManager} import org.apache.spark.util.CompletionIterator @@ -97,15 +97,14 @@ class ExternalAppendOnlyMap[K, V, C]( * NOTE: Setting this too low can cause excessive copying when serializing, since some serializers * grow internal data structures by growing + copying every time the number of objects doubles. */ - private val serializerBatchSize = sparkConf.getLong("spark.shuffle.spill.batchSize", 10000) + private val serializerBatchSize = sparkConf.get(config.SHUFFLE_SPILL_BATCH_SIZE) // Number of bytes spilled in total private var _diskBytesSpilled = 0L def diskBytesSpilled: Long = _diskBytesSpilled // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - private val fileBufferSize = - sparkConf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 + private val fileBufferSize = sparkConf.get(config.SHUFFLE_FILE_BUFFER_SIZE).toInt * 1024 // Write metrics private val writeMetrics: ShuffleWriteMetrics = new ShuffleWriteMetrics() diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala index b159200d79222..3f3b7d20eb169 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalSorter.scala @@ -27,7 +27,7 @@ import com.google.common.io.ByteStreams import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.serializer._ import org.apache.spark.storage.{BlockId, DiskBlockObjectWriter} @@ -109,7 +109,7 @@ private[spark] class ExternalSorter[K, V, C]( private val serInstance = serializer.newInstance() // Use getSizeAsKb (not bytes) to maintain backwards compatibility if no units are provided - private val fileBufferSize = conf.getSizeAsKb("spark.shuffle.file.buffer", "32k").toInt * 1024 + private val fileBufferSize = conf.get(config.SHUFFLE_FILE_BUFFER_SIZE).toInt * 1024 // Size of object batches when reading/writing from serializers. // @@ -118,7 +118,7 @@ private[spark] class ExternalSorter[K, V, C]( // // NOTE: Setting this too low can cause excessive copying when serializing, since some serializers // grow internal data structures by growing + copying every time the number of objects doubles. - private val serializerBatchSize = conf.getLong("spark.shuffle.spill.batchSize", 10000) + private val serializerBatchSize = conf.get(config.SHUFFLE_SPILL_BATCH_SIZE) // Data structures to store in-memory objects before we spill. Depending on whether we have an // Aggregator set, we either put objects into an AppendOnlyMap where we combine them, or we @@ -143,12 +143,10 @@ private[spark] class ExternalSorter[K, V, C]( // user. (A partial ordering means that equal keys have comparator.compare(k, k) = 0, but some // non-equal keys also have this, so we need to do a later pass to find truly equal keys). // Note that we ignore this if no aggregator and no ordering are given. - private val keyComparator: Comparator[K] = ordering.getOrElse(new Comparator[K] { - override def compare(a: K, b: K): Int = { - val h1 = if (a == null) 0 else a.hashCode() - val h2 = if (b == null) 0 else b.hashCode() - if (h1 < h2) -1 else if (h1 == h2) 0 else 1 - } + private val keyComparator: Comparator[K] = ordering.getOrElse((a: K, b: K) => { + val h1 = if (a == null) 0 else a.hashCode() + val h2 = if (b == null) 0 else b.hashCode() + if (h1 < h2) -1 else if (h1 == h2) 0 else 1 }) private def comparator: Option[Comparator[K]] = { @@ -363,17 +361,15 @@ private[spark] class ExternalSorter[K, V, C]( * Merge-sort a sequence of (K, C) iterators using a given a comparator for the keys. */ private def mergeSort(iterators: Seq[Iterator[Product2[K, C]]], comparator: Comparator[K]) - : Iterator[Product2[K, C]] = - { + : Iterator[Product2[K, C]] = { val bufferedIters = iterators.filter(_.hasNext).map(_.buffered) type Iter = BufferedIterator[Product2[K, C]] - val heap = new mutable.PriorityQueue[Iter]()(new Ordering[Iter] { - // Use the reverse order because PriorityQueue dequeues the max - override def compare(x: Iter, y: Iter): Int = comparator.compare(y.head._1, x.head._1) - }) + // Use the reverse order (compare(y,x)) because PriorityQueue dequeues the max + val heap = new mutable.PriorityQueue[Iter]()( + (x: Iter, y: Iter) => comparator.compare(y.head._1, x.head._1)) heap.enqueue(bufferedIters: _*) // Will contain only the iterators with hasNext = true new Iterator[Product2[K, C]] { - override def hasNext: Boolean = !heap.isEmpty + override def hasNext: Boolean = heap.nonEmpty override def next(): Product2[K, C] = { if (!hasNext) { @@ -400,13 +396,12 @@ private[spark] class ExternalSorter[K, V, C]( mergeCombiners: (C, C) => C, comparator: Comparator[K], totalOrder: Boolean) - : Iterator[Product2[K, C]] = - { + : Iterator[Product2[K, C]] = { if (!totalOrder) { // We only have a partial ordering, e.g. comparing the keys by hash code, which means that // multiple distinct keys might be treated as equal by the ordering. To deal with this, we // need to read all keys considered equal by the ordering at once and compare them. - new Iterator[Iterator[Product2[K, C]]] { + val it = new Iterator[Iterator[Product2[K, C]]] { val sorted = mergeSort(iterators, comparator).buffered // Buffers reused across elements to decrease memory allocation @@ -446,7 +441,8 @@ private[spark] class ExternalSorter[K, V, C]( // equal by the partial order; we flatten this below to get a flat iterator of (K, C). keys.iterator.zip(combiners.iterator) } - }.flatMap(i => i) + } + it.flatten } else { // We have a total ordering, so the objects with the same key are sequential. new Iterator[Product2[K, C]] { @@ -650,7 +646,7 @@ private[spark] class ExternalSorter[K, V, C]( if (spills.isEmpty) { // Special case: if we have only in-memory data, we don't need to merge streams, and perhaps // we don't even need to sort by anything other than partition ID - if (!ordering.isDefined) { + if (ordering.isEmpty) { // The user hasn't requested sorted keys, so only sort by partition ID, not key groupByPartition(destructiveIterator(collection.partitionedDestructiveSortedIterator(None))) } else { @@ -727,9 +723,10 @@ private[spark] class ExternalSorter[K, V, C]( spills.clear() forceSpillFiles.foreach(s => s.file.delete()) forceSpillFiles.clear() - if (map != null || buffer != null) { + if (map != null || buffer != null || readingIterator != null) { map = null // So that the memory can be garbage-collected buffer = null // So that the memory can be garbage-collected + readingIterator = null // So that the memory can be garbage-collected releaseMemory() } } @@ -793,8 +790,8 @@ private[spark] class ExternalSorter[K, V, C]( def nextPartition(): Int = cur._1._1 } - logInfo(s"Task ${context.taskAttemptId} force spilling in-memory map to disk and " + - s" it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory") + logInfo(s"Task ${TaskContext.get().taskAttemptId} force spilling in-memory map to disk " + + s"and it will release ${org.apache.spark.util.Utils.bytesToString(getUsed())} memory") val spillFile = spillMemoryIteratorToDisk(inMemoryIterator) forceSpillFiles += spillFile val spillReader = new SpillReader(spillFile) diff --git a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala index e17a9de97e335..652d8c001e09b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/PartitionedPairBuffer.scala @@ -19,7 +19,6 @@ package org.apache.spark.util.collection import java.util.Comparator -import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.collection.WritablePartitionedPairCollection._ diff --git a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala index 81457b53cd814..bfc0face5d8e5 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/Spillable.scala @@ -51,7 +51,7 @@ private[spark] abstract class Spillable[C](taskMemoryManager: TaskMemoryManager) // Initial threshold for the size of a collection before we start tracking its memory usage // For testing only private[this] val initialMemoryThreshold: Long = - SparkEnv.get.conf.getLong("spark.shuffle.spill.initialMemoryThreshold", 5 * 1024 * 1024) + SparkEnv.get.conf.get(SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD) // Force this collection to spill when there are this many elements in memory // For testing only diff --git a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala index 5232c2bd8d6f6..dd7f68fd038d2 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/WritablePartitionedPairCollection.scala @@ -68,27 +68,20 @@ private[spark] object WritablePartitionedPairCollection { /** * A comparator for (Int, K) pairs that orders them by only their partition ID. */ - def partitionComparator[K]: Comparator[(Int, K)] = new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - a._1 - b._1 - } - } + def partitionComparator[K]: Comparator[(Int, K)] = (a: (Int, K), b: (Int, K)) => a._1 - b._1 /** * A comparator for (Int, K) pairs that orders them both by their partition ID and a key ordering. */ - def partitionKeyComparator[K](keyComparator: Comparator[K]): Comparator[(Int, K)] = { - new Comparator[(Int, K)] { - override def compare(a: (Int, K), b: (Int, K)): Int = { - val partitionDiff = a._1 - b._1 - if (partitionDiff != 0) { - partitionDiff - } else { - keyComparator.compare(a._2, b._2) - } + def partitionKeyComparator[K](keyComparator: Comparator[K]): Comparator[(Int, K)] = + (a: (Int, K), b: (Int, K)) => { + val partitionDiff = a._1 - b._1 + if (partitionDiff != 0) { + partitionDiff + } else { + keyComparator.compare(a._2, b._2) } } - } } /** diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala index 4aa8d45ec7404..2c3730de08b5b 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBuffer.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.config import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer} import org.apache.spark.network.util.{ByteArrayWritableChannel, LimitedInputStream} -import org.apache.spark.storage.StorageUtils +import org.apache.spark.storage.{EncryptedManagedBuffer, StorageUtils} +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils /** @@ -96,7 +97,7 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { * @throws UnsupportedOperationException if this buffer's size exceeds the maximum array size. */ def toArray: Array[Byte] = { - if (size >= Integer.MAX_VALUE) { + if (size >= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { throw new UnsupportedOperationException( s"cannot call toArray because buffer size ($size bytes) exceeds maximum array size") } @@ -169,24 +170,25 @@ private[spark] class ChunkedByteBuffer(var chunks: Array[ByteBuffer]) { } -object ChunkedByteBuffer { - // TODO eliminate this method if we switch BlockManager to getting InputStreams - def fromManagedBuffer(data: ManagedBuffer, maxChunkSize: Int): ChunkedByteBuffer = { +private[spark] object ChunkedByteBuffer { + + def fromManagedBuffer(data: ManagedBuffer): ChunkedByteBuffer = { data match { case f: FileSegmentManagedBuffer => - fromFile(f.getFile, maxChunkSize, f.getOffset, f.getLength) + fromFile(f.getFile, f.getOffset, f.getLength) + case e: EncryptedManagedBuffer => + e.blockData.toChunkedByteBuffer(ByteBuffer.allocate _) case other => new ChunkedByteBuffer(other.nioByteBuffer()) } } - def fromFile(file: File, maxChunkSize: Int): ChunkedByteBuffer = { - fromFile(file, maxChunkSize, 0, file.length()) + def fromFile(file: File): ChunkedByteBuffer = { + fromFile(file, 0, file.length()) } private def fromFile( file: File, - maxChunkSize: Int, offset: Long, length: Long): ChunkedByteBuffer = { // We do *not* memory map the file, because we may end up putting this into the memory store, @@ -195,7 +197,7 @@ object ChunkedByteBuffer { val is = new FileInputStream(file) ByteStreams.skipFully(is, offset) val in = new LimitedInputStream(is, length) - val chunkSize = math.min(maxChunkSize, length).toInt + val chunkSize = math.min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, length).toInt val out = new ChunkedByteBufferOutputStream(chunkSize, ByteBuffer.allocate _) Utils.tryWithSafeFinally { IOUtils.copy(in, out) @@ -218,7 +220,8 @@ private[spark] class ChunkedByteBufferInputStream( dispose: Boolean) extends InputStream { - private[this] var chunks = chunkedByteBuffer.getChunks().iterator + // Filter out empty chunks since `read()` assumes all chunks are non-empty. + private[this] var chunks = chunkedByteBuffer.getChunks().filter(_.hasRemaining).iterator private[this] var currentChunk: ByteBuffer = { if (chunks.hasNext) { chunks.next() diff --git a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala index 9622d0ac05368..23fc0f88f0b93 100644 --- a/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala +++ b/core/src/main/scala/org/apache/spark/util/io/ChunkedByteBufferFileRegion.scala @@ -18,10 +18,6 @@ package org.apache.spark.util.io import java.nio.channels.WritableByteChannel -import io.netty.channel.FileRegion -import io.netty.util.AbstractReferenceCounted - -import org.apache.spark.internal.Logging import org.apache.spark.network.util.AbstractFileRegion diff --git a/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala new file mode 100644 index 0000000000000..c4540433bce97 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/logging/DriverLogger.scala @@ -0,0 +1,203 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util.logging + +import java.io._ +import java.util.concurrent.{ScheduledExecutorService, TimeUnit} + +import org.apache.commons.io.FileUtils +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} +import org.apache.hadoop.fs.permission.FsPermission +import org.apache.log4j.{FileAppender => Log4jFileAppender, _} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.network.util.JavaUtils +import org.apache.spark.util.{ThreadUtils, Utils} + +private[spark] class DriverLogger(conf: SparkConf) extends Logging { + + private val UPLOAD_CHUNK_SIZE = 1024 * 1024 + private val UPLOAD_INTERVAL_IN_SECS = 5 + private val DEFAULT_LAYOUT = "%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n" + private val LOG_FILE_PERMISSIONS = new FsPermission(Integer.parseInt("770", 8).toShort) + + private val localLogFile: String = FileUtils.getFile( + Utils.getLocalDir(conf), + DriverLogger.DRIVER_LOG_DIR, + DriverLogger.DRIVER_LOG_FILE).getAbsolutePath() + private var writer: Option[DfsAsyncWriter] = None + + addLogAppender() + + private def addLogAppender(): Unit = { + val appenders = LogManager.getRootLogger().getAllAppenders() + val layout = if (conf.contains(DRIVER_LOG_LAYOUT)) { + new PatternLayout(conf.get(DRIVER_LOG_LAYOUT).get) + } else if (appenders.hasMoreElements()) { + appenders.nextElement().asInstanceOf[Appender].getLayout() + } else { + new PatternLayout(DEFAULT_LAYOUT) + } + val fa = new Log4jFileAppender(layout, localLogFile) + fa.setName(DriverLogger.APPENDER_NAME) + LogManager.getRootLogger().addAppender(fa) + logInfo(s"Added a local log appender at: ${localLogFile}") + } + + def startSync(hadoopConf: Configuration): Unit = { + try { + // Setup a writer which moves the local file to hdfs continuously + val appId = Utils.sanitizeDirName(conf.getAppId) + writer = Some(new DfsAsyncWriter(appId, hadoopConf)) + } catch { + case e: Exception => + logError(s"Could not persist driver logs to dfs", e) + } + } + + def stop(): Unit = { + try { + val fa = LogManager.getRootLogger.getAppender(DriverLogger.APPENDER_NAME) + LogManager.getRootLogger().removeAppender(DriverLogger.APPENDER_NAME) + Utils.tryLogNonFatalError(fa.close()) + writer.foreach(_.closeWriter()) + } catch { + case e: Exception => + logError(s"Error in persisting driver logs", e) + } finally { + Utils.tryLogNonFatalError { + JavaUtils.deleteRecursively(FileUtils.getFile(localLogFile).getParentFile()) + } + } + } + + // Visible for testing + private[spark] class DfsAsyncWriter(appId: String, hadoopConf: Configuration) extends Runnable + with Logging { + + private var streamClosed = false + private var inStream: InputStream = null + private var outputStream: FSDataOutputStream = null + private val tmpBuffer = new Array[Byte](UPLOAD_CHUNK_SIZE) + private var threadpool: ScheduledExecutorService = _ + init() + + private def init(): Unit = { + val rootDir = conf.get(DRIVER_LOG_DFS_DIR).get + val fileSystem: FileSystem = new Path(rootDir).getFileSystem(hadoopConf) + if (!fileSystem.exists(new Path(rootDir))) { + throw new RuntimeException(s"${rootDir} does not exist." + + s" Please create this dir in order to persist driver logs") + } + val dfsLogFile: String = FileUtils.getFile(rootDir, appId + + DriverLogger.DRIVER_LOG_FILE_SUFFIX).getAbsolutePath() + try { + inStream = new BufferedInputStream(new FileInputStream(localLogFile)) + outputStream = fileSystem.create(new Path(dfsLogFile), true) + fileSystem.setPermission(new Path(dfsLogFile), LOG_FILE_PERMISSIONS) + } catch { + case e: Exception => + JavaUtils.closeQuietly(inStream) + JavaUtils.closeQuietly(outputStream) + throw e + } + threadpool = ThreadUtils.newDaemonSingleThreadScheduledExecutor("dfsSyncThread") + threadpool.scheduleWithFixedDelay(this, UPLOAD_INTERVAL_IN_SECS, UPLOAD_INTERVAL_IN_SECS, + TimeUnit.SECONDS) + logInfo(s"Started driver log file sync to: ${dfsLogFile}") + } + + def run(): Unit = { + if (streamClosed) { + return + } + try { + var remaining = inStream.available() + while (remaining > 0) { + val read = inStream.read(tmpBuffer, 0, math.min(remaining, UPLOAD_CHUNK_SIZE)) + outputStream.write(tmpBuffer, 0, read) + remaining -= read + } + outputStream.hflush() + } catch { + case e: Exception => logError("Failed writing driver logs to dfs", e) + } + } + + private def close(): Unit = { + if (streamClosed) { + return + } + try { + // Write all remaining bytes + run() + } finally { + try { + streamClosed = true + inStream.close() + outputStream.close() + } catch { + case e: Exception => + logError("Error in closing driver log input/output stream", e) + } + } + } + + def closeWriter(): Unit = { + try { + threadpool.execute(() => DfsAsyncWriter.this.close()) + threadpool.shutdown() + threadpool.awaitTermination(1, TimeUnit.MINUTES) + } catch { + case e: Exception => + logError("Error in shutting down threadpool", e) + } + } + } + +} + +private[spark] object DriverLogger extends Logging { + val DRIVER_LOG_DIR = "__driver_logs__" + val DRIVER_LOG_FILE = "driver.log" + val DRIVER_LOG_FILE_SUFFIX = "_" + DRIVER_LOG_FILE + val APPENDER_NAME = "_DriverLogAppender" + + def apply(conf: SparkConf): Option[DriverLogger] = { + if (conf.get(DRIVER_LOG_PERSISTTODFS) && Utils.isClientMode(conf)) { + if (conf.contains(DRIVER_LOG_DFS_DIR)) { + try { + Some(new DriverLogger(conf)) + } catch { + case e: Exception => + logError("Could not add driver logger", e) + None + } + } else { + logWarning(s"Driver logs are not persisted because" + + s" ${DRIVER_LOG_DFS_DIR.key} is not configured") + None + } + } else { + None + } + } +} diff --git a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala index 2f9ad4c8cc3e1..3188e0bd2b70d 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/FileAppender.scala @@ -20,7 +20,7 @@ package org.apache.spark.util.logging import java.io.{File, FileOutputStream, InputStream, IOException} import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{IntParam, Utils} /** @@ -115,11 +115,9 @@ private[spark] object FileAppender extends Logging { /** Create the right appender based on Spark configuration */ def apply(inputStream: InputStream, file: File, conf: SparkConf): FileAppender = { - import RollingFileAppender._ - - val rollingStrategy = conf.get(STRATEGY_PROPERTY, STRATEGY_DEFAULT) - val rollingSizeBytes = conf.get(SIZE_PROPERTY, STRATEGY_DEFAULT) - val rollingInterval = conf.get(INTERVAL_PROPERTY, INTERVAL_DEFAULT) + val rollingStrategy = conf.get(config.EXECUTOR_LOGS_ROLLING_STRATEGY) + val rollingSizeBytes = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_SIZE) + val rollingInterval = conf.get(config.EXECUTOR_LOGS_ROLLING_TIME_INTERVAL) def createTimeBasedAppender(): FileAppender = { val validatedParams: Option[(Long, String)] = rollingInterval match { diff --git a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala index 5d8cec8447b53..59439b68792e5 100644 --- a/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala +++ b/core/src/main/scala/org/apache/spark/util/logging/RollingFileAppender.scala @@ -24,6 +24,7 @@ import com.google.common.io.Files import org.apache.commons.io.IOUtils import org.apache.spark.SparkConf +import org.apache.spark.internal.config /** * Continuously appends data from input stream into the given file, and rolls @@ -44,10 +45,8 @@ private[spark] class RollingFileAppender( bufferSize: Int = RollingFileAppender.DEFAULT_BUFFER_SIZE ) extends FileAppender(inputStream, activeFile, bufferSize) { - import RollingFileAppender._ - - private val maxRetainedFiles = conf.getInt(RETAINED_FILES_PROPERTY, -1) - private val enableCompression = conf.getBoolean(ENABLE_COMPRESSION, false) + private val maxRetainedFiles = conf.get(config.EXECUTOR_LOGS_ROLLING_MAX_RETAINED_FILES) + private val enableCompression = conf.get(config.EXECUTOR_LOGS_ROLLING_ENABLE_COMPRESSION) /** Stop the appender */ override def stop() { @@ -82,7 +81,7 @@ private[spark] class RollingFileAppender( // Roll the log file and compress if enableCompression is true. private def rotateFile(activeFile: File, rolloverFile: File): Unit = { if (enableCompression) { - val gzFile = new File(rolloverFile.getAbsolutePath + GZIP_LOG_SUFFIX) + val gzFile = new File(rolloverFile.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX) var gzOutputStream: GZIPOutputStream = null var inputStream: InputStream = null try { @@ -103,7 +102,7 @@ private[spark] class RollingFileAppender( // Check if the rollover file already exists. private def rolloverFileExist(file: File): Boolean = { - file.exists || new File(file.getAbsolutePath + GZIP_LOG_SUFFIX).exists + file.exists || new File(file.getAbsolutePath + RollingFileAppender.GZIP_LOG_SUFFIX).exists } /** Move the active log file to a new rollover file */ @@ -164,15 +163,7 @@ private[spark] class RollingFileAppender( * names of configurations that configure rolling file appenders. */ private[spark] object RollingFileAppender { - val STRATEGY_PROPERTY = "spark.executor.logs.rolling.strategy" - val STRATEGY_DEFAULT = "" - val INTERVAL_PROPERTY = "spark.executor.logs.rolling.time.interval" - val INTERVAL_DEFAULT = "daily" - val SIZE_PROPERTY = "spark.executor.logs.rolling.maxSize" - val SIZE_DEFAULT = (1024 * 1024).toString - val RETAINED_FILES_PROPERTY = "spark.executor.logs.rolling.maxRetainedFiles" val DEFAULT_BUFFER_SIZE = 8192 - val ENABLE_COMPRESSION = "spark.executor.logs.rolling.enableCompression" val GZIP_LOG_SUFFIX = ".gz" diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index ea99a7e5b4847..70554f1d03067 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -49,7 +49,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable /** return a copy of the RandomSampler object */ override def clone: RandomSampler[T, U] = - throw new NotImplementedError("clone() is not implemented.") + throw new UnsupportedOperationException("clone() is not implemented.") } private[spark] diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index e8cdb6e98bf36..af09e50a157ae 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -22,8 +22,6 @@ import java.util.{Random => JavaRandom} import scala.util.hashing.MurmurHash3 -import org.apache.spark.util.Utils.timeIt - /** * This class implements a XORShift random number generator algorithm * Source: @@ -61,48 +59,9 @@ private[spark] object XORShiftRandom { /** Hash seeds to have 0/1 bits throughout. */ private[random] def hashSeed(seed: Long): Long = { - val bytes = ByteBuffer.allocate(java.lang.Long.SIZE).putLong(seed).array() + val bytes = ByteBuffer.allocate(java.lang.Long.BYTES).putLong(seed).array() val lowBits = MurmurHash3.bytesHash(bytes) val highBits = MurmurHash3.bytesHash(bytes, lowBits) (highBits.toLong << 32) | (lowBits.toLong & 0xFFFFFFFFL) } - - /** - * Main method for running benchmark - * @param args takes one argument - the number of random numbers to generate - */ - def main(args: Array[String]): Unit = { - // scalastyle:off println - if (args.length != 1) { - println("Benchmark of XORShiftRandom vis-a-vis java.util.Random") - println("Usage: XORShiftRandom number_of_random_numbers_to_generate") - System.exit(1) - } - println(benchmark(args(0).toInt)) - // scalastyle:on println - } - - /** - * @param numIters Number of random numbers to generate while running the benchmark - * @return Map of execution times for {@link java.util.Random java.util.Random} - * and XORShift - */ - def benchmark(numIters: Int): Map[String, Long] = { - - val seed = 1L - val million = 1e6.toInt - val javaRand = new JavaRandom(seed) - val xorRand = new XORShiftRandom(seed) - - // this is just to warm up the JIT - we're not timing anything - timeIt(million) { - javaRand.nextInt() - xorRand.nextInt() - } - - /* Return results as a map instead of just printing to screen - in case the user wants to do something with them */ - Map("javaTime" -> timeIt(numIters) { javaRand.nextInt() }, - "xorTime" -> timeIt(numIters) { xorRand.nextInt() }) - } } diff --git a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java index 686eb28010c6a..80cd70282a51d 100644 --- a/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java +++ b/core/src/test/java/org/apache/spark/ExecutorPluginSuite.java @@ -63,10 +63,10 @@ private SparkConf initializeSparkConf(String pluginNames) { @Test public void testPluginClassDoesNotExist() { - SparkConf conf = initializeSparkConf("nonexistant.plugin"); + SparkConf conf = initializeSparkConf("nonexistent.plugin"); try { sc = new JavaSparkContext(conf); - fail("No exception thrown for nonexistant plugin"); + fail("No exception thrown for nonexistent plugin"); } catch (Exception e) { // We cannot catch ClassNotFoundException directly because Java doesn't think it'll be thrown assertTrue(e.toString().startsWith("java.lang.ClassNotFoundException")); diff --git a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java index 6a1a38c1a54f4..773c390175b6d 100644 --- a/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java +++ b/core/src/test/java/org/apache/spark/launcher/SparkLauncherSuite.java @@ -41,6 +41,8 @@ public class SparkLauncherSuite extends BaseSuite { private static final NamedThreadFactory TF = new NamedThreadFactory("SparkLauncherSuite-%d"); + private static final String EXCEPTION_MESSAGE = "dummy-exception"; + private static final RuntimeException DUMMY_EXCEPTION = new RuntimeException(EXCEPTION_MESSAGE); private final SparkLauncher launcher = new SparkLauncher(); @@ -130,17 +132,8 @@ public void testInProcessLauncher() throws Exception { try { inProcessLauncherTestImpl(); } finally { - Properties p = new Properties(); - for (Map.Entry e : properties.entrySet()) { - p.put(e.getKey(), e.getValue()); - } - System.setProperties(p); - // Here DAGScheduler is stopped, while SparkContext.clearActiveContext may not be called yet. - // Wait for a reasonable amount of time to avoid creating two active SparkContext in JVM. - // See SPARK-23019 and SparkContext.stop() for details. - eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> { - assertTrue("SparkContext is still alive.", SparkContext$.MODULE$.getActive().isEmpty()); - }); + restoreSystemProperties(properties); + waitForSparkContextShutdown(); } } @@ -227,6 +220,82 @@ public void testInProcessLauncherDoesNotKillJvm() throws Exception { assertEquals(SparkAppHandle.State.LOST, handle.getState()); } + @Test + public void testInProcessLauncherGetError() throws Exception { + // Because this test runs SparkLauncher in process and in client mode, it pollutes the system + // properties, and that can cause test failures down the test pipeline. So restore the original + // system properties after this test runs. + Map properties = new HashMap<>(System.getProperties()); + + SparkAppHandle handle = null; + try { + handle = new InProcessLauncher() + .setMaster("local") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(ErrorInProcessTestApp.class.getName()) + .addAppArgs("hello") + .startApplication(); + + final SparkAppHandle _handle = handle; + eventually(Duration.ofSeconds(60), Duration.ofMillis(1000), () -> { + assertEquals(SparkAppHandle.State.FAILED, _handle.getState()); + }); + + assertNotNull(handle.getError()); + assertTrue(handle.getError().isPresent()); + assertSame(handle.getError().get(), DUMMY_EXCEPTION); + } finally { + if (handle != null) { + handle.kill(); + } + restoreSystemProperties(properties); + waitForSparkContextShutdown(); + } + } + + @Test + public void testSparkLauncherGetError() throws Exception { + SparkAppHandle handle = null; + try { + handle = new SparkLauncher() + .setMaster("local") + .setAppResource(SparkLauncher.NO_RESOURCE) + .setMainClass(ErrorInProcessTestApp.class.getName()) + .addAppArgs("hello") + .startApplication(); + + final SparkAppHandle _handle = handle; + eventually(Duration.ofSeconds(60), Duration.ofMillis(1000), () -> { + assertEquals(SparkAppHandle.State.FAILED, _handle.getState()); + }); + + assertNotNull(handle.getError()); + assertTrue(handle.getError().isPresent()); + assertTrue(handle.getError().get().getMessage().contains(EXCEPTION_MESSAGE)); + } finally { + if (handle != null) { + handle.kill(); + } + } + } + + private void restoreSystemProperties(Map properties) { + Properties p = new Properties(); + for (Map.Entry e : properties.entrySet()) { + p.put(e.getKey(), e.getValue()); + } + System.setProperties(p); + } + + private void waitForSparkContextShutdown() throws Exception { + // Here DAGScheduler is stopped, while SparkContext.clearActiveContext may not be called yet. + // Wait for a reasonable amount of time to avoid creating two active SparkContext in JVM. + // See SPARK-23019 and SparkContext.stop() for details. + eventually(Duration.ofSeconds(5), Duration.ofMillis(10), () -> { + assertTrue("SparkContext is still alive.", SparkContext$.MODULE$.getActive().isEmpty()); + }); + } + public static class SparkLauncherTestApp { public static void main(String[] args) throws Exception { @@ -264,4 +333,15 @@ public static void main(String[] args) throws Exception { } + /** + * Similar to {@link InProcessTestApp} except it throws an exception + */ + public static class ErrorInProcessTestApp { + + public static void main(String[] args) { + assertNotEquals(0, args.length); + assertEquals(args[0], "hello"); + throw DUMMY_EXCEPTION; + } + } } diff --git a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java index a0664b30d6cc2..9ec303a4eff68 100644 --- a/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java +++ b/core/src/test/java/org/apache/spark/memory/TaskMemoryManagerSuite.java @@ -23,16 +23,17 @@ import org.apache.spark.SparkConf; import org.apache.spark.unsafe.memory.MemoryAllocator; import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.internal.config.package$; public class TaskMemoryManagerSuite { @Test public void leakedPageMemoryIsDetected() { final TaskMemoryManager manager = new TaskMemoryManager( - new StaticMemoryManager( - new SparkConf().set("spark.memory.offHeap.enabled", "false"), - Long.MAX_VALUE, + new UnifiedMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false), Long.MAX_VALUE, + Long.MAX_VALUE / 2, 1), 0); final MemoryConsumer c = new TestMemoryConsumer(manager); @@ -44,8 +45,8 @@ public void leakedPageMemoryIsDetected() { @Test public void encodePageNumberAndOffsetOffHeap() { final SparkConf conf = new SparkConf() - .set("spark.memory.offHeap.enabled", "true") - .set("spark.memory.offHeap.size", "1000"); + .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true) + .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 1000L); final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.OFF_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); @@ -61,7 +62,8 @@ public void encodePageNumberAndOffsetOffHeap() { @Test public void encodePageNumberAndOffsetOnHeap() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); final long encodedAddress = manager.encodePageNumberAndOffset(dataPage, 64); @@ -72,7 +74,8 @@ public void encodePageNumberAndOffsetOnHeap() { @Test public void freeingPageSetsPageNumberToSpecialConstant() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); c.freePage(dataPage); @@ -82,7 +85,8 @@ public void freeingPageSetsPageNumberToSpecialConstant() { @Test(expected = AssertionError.class) public void freeingPageDirectlyInAllocatorTriggersAssertionError() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = manager.allocatePage(256, c); MemoryAllocator.HEAP.free(dataPage); @@ -91,7 +95,8 @@ public void freeingPageDirectlyInAllocatorTriggersAssertionError() { @Test(expected = AssertionError.class) public void callingFreePageOnDirectlyAllocatedPageTriggersAssertionError() { final TaskMemoryManager manager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final MemoryConsumer c = new TestMemoryConsumer(manager, MemoryMode.ON_HEAP); final MemoryBlock dataPage = MemoryAllocator.HEAP.allocate(256); manager.freePage(dataPage, c); @@ -199,7 +204,7 @@ public void offHeapConfigurationBackwardsCompatibility() { // was deprecated in Spark 1.6 and replaced by `spark.memory.offHeap.enabled` (see SPARK-12251). final SparkConf conf = new SparkConf() .set("spark.unsafe.offHeap", "true") - .set("spark.memory.offHeap.size", "1000"); + .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 1000L); final TaskMemoryManager manager = new TaskMemoryManager(new TestMemoryManager(conf), 0); Assert.assertSame(MemoryMode.OFF_HEAP, manager.tungstenMemoryMode); } diff --git a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java index 0bbaea6b834b8..6aa577d1bf797 100644 --- a/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java +++ b/core/src/test/java/org/apache/spark/memory/TestMemoryConsumer.java @@ -38,12 +38,12 @@ public long spill(long size, MemoryConsumer trigger) throws IOException { return used; } - void use(long size) { + public void use(long size) { long got = taskMemoryManager.acquireExecutionMemory(size, this); used += got; } - void free(long size) { + public void free(long size) { used -= size; taskMemoryManager.releaseExecutionMemory(size, this); } diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java index 354efe18dbde7..ccf47a4de558f 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/PackedRecordPointerSuite.java @@ -22,6 +22,7 @@ import org.junit.Test; import org.apache.spark.SparkConf; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.*; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -34,7 +35,7 @@ public class PackedRecordPointerSuite { @Test public void heap() throws IOException { - final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); + final SparkConf conf = new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.ON_HEAP); @@ -55,8 +56,8 @@ public void heap() throws IOException { @Test public void offHeap() throws IOException { final SparkConf conf = new SparkConf() - .set("spark.memory.offHeap.enabled", "true") - .set("spark.memory.offHeap.size", "10000"); + .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), true) + .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 10000L); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); final MemoryConsumer c = new TestMemoryConsumer(memoryManager, MemoryMode.OFF_HEAP); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java index 694352ee2af44..7a17d90d6ed66 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/ShuffleInMemorySorterSuite.java @@ -26,6 +26,7 @@ import org.apache.spark.HashPartitioner; import org.apache.spark.SparkConf; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.MemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryConsumer; @@ -38,7 +39,7 @@ public class ShuffleInMemorySorterSuite { protected boolean shouldUseRadixSort() { return false; } final TestMemoryManager memoryManager = - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")); + new TestMemoryManager(new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)); final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(taskMemoryManager); @@ -69,7 +70,7 @@ public void testBasicSorting() throws Exception { "Lychee", "Mango" }; - final SparkConf conf = new SparkConf().set("spark.memory.offHeap.enabled", "false"); + final SparkConf conf = new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); final TaskMemoryManager memoryManager = new TaskMemoryManager(new TestMemoryManager(conf), 0); final MemoryConsumer c = new TestMemoryConsumer(memoryManager); diff --git a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java index a07d0e84ea854..88125a6b93ade 100644 --- a/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java +++ b/core/src/test/java/org/apache/spark/shuffle/sort/UnsafeShuffleWriterSuite.java @@ -45,6 +45,7 @@ import org.apache.spark.io.LZ4CompressionCodec; import org.apache.spark.io.LZFCompressionCodec; import org.apache.spark.io.SnappyCompressionCodec; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.LimitedInputStream; @@ -100,8 +101,8 @@ public void setUp() throws IOException { partitionSizesInMergedFile = null; spillFilesCreated.clear(); conf = new SparkConf() - .set("spark.buffer.pageSize", "1m") - .set("spark.memory.offHeap.enabled", "false"); + .set(package$.MODULE$.BUFFER_PAGESIZE().key(), "1m") + .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); taskMetrics = new TaskMetrics(); memoryManager = new TestMemoryManager(conf); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); @@ -162,7 +163,8 @@ private UnsafeShuffleWriter createWriter( new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, - conf + conf, + taskContext.taskMetrics().shuffleWriteMetrics() ); } @@ -183,7 +185,7 @@ private List> readRecordsFromFile() throws IOException { fin.getChannel().position(startOffset); InputStream in = new LimitedInputStream(fin, partitionSize); in = blockManager.serializerManager().wrapForEncryption(in); - if (conf.getBoolean("spark.shuffle.compress", true)) { + if ((boolean) conf.get(package$.MODULE$.SHUFFLE_COMPRESS())) { in = CompressionCodec$.MODULE$.createCodec(conf).compressedInputStream(in); } try (DeserializationStream recordsStream = serializer.newInstance().deserializeStream(in)) { @@ -234,6 +236,7 @@ public void writeEmptyIterator() throws Exception { final Option mapStatus = writer.stop(true); assertTrue(mapStatus.isDefined()); assertTrue(mergedOutputFile.exists()); + assertEquals(0, spillFilesCreated.size()); assertArrayEquals(new long[NUM_PARTITITONS], partitionSizesInMergedFile); assertEquals(0, taskMetrics.shuffleWriteMetrics().recordsWritten()); assertEquals(0, taskMetrics.shuffleWriteMetrics().bytesWritten()); @@ -277,12 +280,12 @@ private void testMergingSpills( String compressionCodecName, boolean encrypt) throws Exception { if (compressionCodecName != null) { - conf.set("spark.shuffle.compress", "true"); + conf.set(package$.MODULE$.SHUFFLE_COMPRESS(), true); conf.set("spark.io.compression.codec", compressionCodecName); } else { - conf.set("spark.shuffle.compress", "false"); + conf.set(package$.MODULE$.SHUFFLE_COMPRESS(), false); } - conf.set(org.apache.spark.internal.config.package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); + conf.set(package$.MODULE$.IO_ENCRYPTION_ENABLED(), encrypt); SerializerManager manager; if (encrypt) { @@ -388,7 +391,7 @@ public void mergeSpillsWithFileStreamAndCompressionAndEncryption() throws Except @Test public void mergeSpillsWithCompressionAndEncryptionSlowPath() throws Exception { - conf.set("spark.shuffle.unsafe.fastMergeEnabled", "false"); + conf.set(package$.MODULE$.SHUFFLE_UNDAFE_FAST_MERGE_ENABLE(), false); testMergingSpills(false, LZ4CompressionCodec.class.getName(), true); } @@ -428,14 +431,14 @@ public void writeEnoughDataToTriggerSpill() throws Exception { @Test public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOff() throws Exception { - conf.set("spark.shuffle.sort.useRadixSort", "false"); + conf.set(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT(), false); writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); assertEquals(2, spillFilesCreated.size()); } @Test public void writeEnoughRecordsToTriggerSortBufferExpansionAndSpillRadixOn() throws Exception { - conf.set("spark.shuffle.sort.useRadixSort", "true"); + conf.set(package$.MODULE$.SHUFFLE_SORT_USE_RADIXSORT(), true); writeEnoughRecordsToTriggerSortBufferExpansionAndSpill(); assertEquals(3, spillFilesCreated.size()); } @@ -521,7 +524,8 @@ public void testPeakMemoryUsed() throws Exception { new SerializedShuffleHandle<>(0, 1, shuffleDep), 0, // map id taskContext, - conf); + conf, + taskContext.taskMetrics().shuffleWriteMetrics()); // Peak memory should be monotonically increasing. More specifically, every time // we allocate a new page it should increase by exactly the size of the page. diff --git a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java index 53a233f698c7a..8d03c6778e18b 100644 --- a/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java +++ b/core/src/test/java/org/apache/spark/unsafe/map/AbstractBytesToBytesMapSuite.java @@ -33,6 +33,8 @@ import org.apache.spark.SparkConf; import org.apache.spark.executor.ShuffleWriteMetrics; +import org.apache.spark.memory.MemoryMode; +import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryManager; import org.apache.spark.network.util.JavaUtils; @@ -43,13 +45,14 @@ import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.array.ByteArrayMethods; import org.apache.spark.util.Utils; +import org.apache.spark.internal.config.package$; import static org.hamcrest.Matchers.greaterThan; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.mockito.Answers.RETURNS_SMART_NULLS; -import static org.mockito.Matchers.any; -import static org.mockito.Matchers.anyInt; +import static org.mockito.ArgumentMatchers.any; +import static org.mockito.ArgumentMatchers.anyInt; import static org.mockito.Mockito.when; @@ -61,7 +64,7 @@ public abstract class AbstractBytesToBytesMapSuite { private TaskMemoryManager taskMemoryManager; private SerializerManager serializerManager = new SerializerManager( new JavaSerializer(new SparkConf()), - new SparkConf().set("spark.shuffle.spill.compress", "false")); + new SparkConf().set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)); private static final long PAGE_SIZE_BYTES = 1L << 26; // 64 megabytes final LinkedList spillFilesCreated = new LinkedList<>(); @@ -75,10 +78,10 @@ public void setup() { memoryManager = new TestMemoryManager( new SparkConf() - .set("spark.memory.offHeap.enabled", "" + useOffHeapMemoryAllocator()) - .set("spark.memory.offHeap.size", "256mb") - .set("spark.shuffle.spill.compress", "false") - .set("spark.shuffle.compress", "false")); + .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), useOffHeapMemoryAllocator()) + .set(package$.MODULE$.MEMORY_OFFHEAP_SIZE(), 256 * 1024 * 1024L) + .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false) + .set(package$.MODULE$.SHUFFLE_COMPRESS(), false)); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); tempDir = Utils.createTempDir(System.getProperty("java.io.tmpdir"), "unsafe-test"); @@ -530,7 +533,7 @@ public void failureToGrow() { @Test public void spillInIterator() throws IOException { BytesToBytesMap map = new BytesToBytesMap( - taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024, false); + taskMemoryManager, blockManager, serializerManager, 1, 0.75, 1024); try { int i; for (i = 0; i < 1024; i++) { @@ -569,7 +572,7 @@ public void spillInIterator() throws IOException { @Test public void multipleValuesForSameKey() { BytesToBytesMap map = - new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024, false); + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024); try { int i; for (i = 0; i < 1024; i++) { @@ -622,6 +625,17 @@ public void initialCapacityBoundsChecking() { } catch (IllegalArgumentException e) { // expected exception } + + try { + new BytesToBytesMap( + taskMemoryManager, + 1, + TaskMemoryManager.MAXIMUM_PAGE_SIZE_BYTES + 1); + Assert.fail("Expected IllegalArgumentException to be thrown"); + } catch (IllegalArgumentException e) { + // expected exception + } + } @Test @@ -667,4 +681,49 @@ public void testPeakMemoryUsed() { } } + @Test + public void avoidDeadlock() throws InterruptedException { + memoryManager.limit(PAGE_SIZE_BYTES); + MemoryMode mode = useOffHeapMemoryAllocator() ? MemoryMode.OFF_HEAP: MemoryMode.ON_HEAP; + TestMemoryConsumer c1 = new TestMemoryConsumer(taskMemoryManager, mode); + BytesToBytesMap map = + new BytesToBytesMap(taskMemoryManager, blockManager, serializerManager, 1, 0.5, 1024); + + Thread thread = new Thread(() -> { + int i = 0; + long used = 0; + while (i < 10) { + c1.use(10000000); + used += 10000000; + i++; + } + c1.free(used); + }); + + try { + int i; + for (i = 0; i < 1024; i++) { + final long[] arr = new long[]{i}; + final BytesToBytesMap.Location loc = map.lookup(arr, Platform.LONG_ARRAY_OFFSET, 8); + loc.append(arr, Platform.LONG_ARRAY_OFFSET, 8, arr, Platform.LONG_ARRAY_OFFSET, 8); + } + + // Starts to require memory at another memory consumer. + thread.start(); + + BytesToBytesMap.MapIterator iter = map.destructiveIterator(); + for (i = 0; i < 1024; i++) { + iter.next(); + } + assertFalse(iter.hasNext()); + } finally { + map.free(); + thread.join(); + for (File spillFile : spillFilesCreated) { + assertFalse("Spill file " + spillFile.getPath() + " was not cleaned up", + spillFile.exists()); + } + } + } + } diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java index 411cd5cb57331..c6aa623560d57 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeExternalSorterSuite.java @@ -38,6 +38,7 @@ import org.apache.spark.executor.TaskMetrics; import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TestMemoryManager; +import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.JavaSerializer; import org.apache.spark.serializer.SerializerInstance; @@ -58,11 +59,11 @@ public class UnsafeExternalSorterSuite { final LinkedList spillFilesCreated = new LinkedList<>(); final TestMemoryManager memoryManager = - new TestMemoryManager(conf.clone().set("spark.memory.offHeap.enabled", "false")); + new TestMemoryManager(conf.clone().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)); final TaskMemoryManager taskMemoryManager = new TaskMemoryManager(memoryManager, 0); final SerializerManager serializerManager = new SerializerManager( new JavaSerializer(conf), - conf.clone().set("spark.shuffle.spill.compress", "false")); + conf.clone().set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false)); // Use integer comparison for comparing prefixes (which are partition ids, in this case) final PrefixComparator prefixComparator = PrefixComparators.LONG; // Since the key fits within the 8-byte prefix, we don't need to do any record comparison, so @@ -87,7 +88,8 @@ public int compare( protected boolean shouldUseRadixSort() { return false; } - private final long pageSizeBytes = conf.getSizeAsBytes("spark.buffer.pageSize", "4m"); + private final long pageSizeBytes = conf.getSizeAsBytes( + package$.MODULE$.BUFFER_PAGESIZE().key(), "4m"); private final int spillThreshold = (int) conf.get(package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()); @@ -534,10 +536,10 @@ public void testOOMDuringSpill() throws Exception { insertNumber(sorter, 1024); fail("expected OutOfMmoryError but it seems operation surprisingly succeeded"); } - // we expect an OutOfMemoryError here, anything else (i.e the original NPE is a failure) - catch (OutOfMemoryError oom){ + // we expect an SparkOutOfMemoryError here, anything else (i.e the original NPE is a failure) + catch (SparkOutOfMemoryError oom){ String oomStackTrace = Utils.exceptionString(oom); - assertThat("expected OutOfMemoryError in " + + assertThat("expected SparkOutOfMemoryError in " + "org.apache.spark.util.collection.unsafe.sort.UnsafeInMemorySorter.reset", oomStackTrace, Matchers.containsString( diff --git a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java index 85ffdca436e14..2b8a0602730e1 100644 --- a/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java +++ b/core/src/test/java/org/apache/spark/util/collection/unsafe/sort/UnsafeInMemorySorterSuite.java @@ -27,9 +27,11 @@ import org.apache.spark.SparkConf; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; +import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.unsafe.Platform; import org.apache.spark.unsafe.memory.MemoryBlock; +import org.apache.spark.internal.config.package$; import static org.hamcrest.MatcherAssert.assertThat; import static org.hamcrest.Matchers.greaterThanOrEqualTo; @@ -51,7 +53,8 @@ private static String getStringFromDataPage(Object baseObject, long baseOffset, @Test public void testSortingEmptyInput() { final TaskMemoryManager memoryManager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); final UnsafeInMemorySorter sorter = new UnsafeInMemorySorter(consumer, memoryManager, @@ -77,7 +80,8 @@ public void testSortingOnlyByIntegerPrefix() throws Exception { "Mango" }; final TaskMemoryManager memoryManager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); final MemoryBlock dataPage = memoryManager.allocatePage(2048, consumer); final Object baseObject = dataPage.getBaseObject(); @@ -145,7 +149,7 @@ public int compare( @Test public void freeAfterOOM() { final SparkConf sparkConf = new SparkConf(); - sparkConf.set("spark.memory.offHeap.enabled", "false"); + sparkConf.set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false); final TestMemoryManager testMemoryManager = new TestMemoryManager(sparkConf); @@ -178,8 +182,8 @@ public int compare( testMemoryManager.markExecutionAsOutOfMemoryOnce(); try { sorter.reset(); - fail("expected OutOfMmoryError but it seems operation surprisingly succeeded"); - } catch (OutOfMemoryError oom) { + fail("expected SparkOutOfMemoryError but it seems operation surprisingly succeeded"); + } catch (SparkOutOfMemoryError oom) { // as expected } // [SPARK-21907] this failed on NPE at diff --git a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java index 3992ab7049bdd..a8252e03b5c15 100644 --- a/core/src/test/java/test/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/test/org/apache/spark/JavaAPISuite.java @@ -32,9 +32,9 @@ import java.util.List; import java.util.Map; import java.util.concurrent.*; +import java.util.stream.Collectors; +import java.util.stream.IntStream; -import org.apache.spark.Accumulator; -import org.apache.spark.AccumulatorParam; import org.apache.spark.Partitioner; import org.apache.spark.SparkConf; import org.apache.spark.TaskContext; @@ -108,11 +108,6 @@ public void sparkContextUnion() { // Varargs JavaRDD sUnion = sc.union(s1, s2); assertEquals(4, sUnion.count()); - // List - List> list = new ArrayList<>(); - list.add(s2); - sUnion = sc.union(s1, list); - assertEquals(4, sUnion.count()); // Union of JavaDoubleRDDs List doubles = Arrays.asList(1.0, 2.0); @@ -163,13 +158,16 @@ public void intersection() { @Test public void sample() { - List ints = Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10); + List ints = IntStream.iterate(1, x -> x + 1) + .limit(20) + .boxed() + .collect(Collectors.toList()); JavaRDD rdd = sc.parallelize(ints); // the seeds here are "magic" to make this work out nicely JavaRDD sample20 = rdd.sample(true, 0.2, 8); assertEquals(2, sample20.count()); JavaRDD sample20WithoutReplacement = rdd.sample(false, 0.2, 2); - assertEquals(2, sample20WithoutReplacement.count()); + assertEquals(4, sample20WithoutReplacement.count()); } @Test @@ -186,7 +184,7 @@ public void randomSplit() { long s1 = splits[1].count(); long s2 = splits[2].count(); assertTrue(s0 + " not within expected range", s0 > 150 && s0 < 250); - assertTrue(s1 + " not within expected range", s1 > 250 && s0 < 350); + assertTrue(s1 + " not within expected range", s1 > 250 && s1 < 350); assertTrue(s2 + " not within expected range", s2 > 430 && s2 < 570); } @@ -956,7 +954,7 @@ public void wholeTextFiles() throws Exception { } @Test - public void textFilesCompressed() throws IOException { + public void textFilesCompressed() { String outputDir = new File(tempDir, "output").getAbsolutePath(); JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4)); rdd.saveAsTextFile(outputDir, DefaultCodec.class); @@ -1183,46 +1181,6 @@ public void zipPartitions() { assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); } - @SuppressWarnings("deprecation") - @Test - public void accumulators() { - JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - - Accumulator intAccum = sc.intAccumulator(10); - rdd.foreach(intAccum::add); - assertEquals((Integer) 25, intAccum.value()); - - Accumulator doubleAccum = sc.doubleAccumulator(10.0); - rdd.foreach(x -> doubleAccum.add((double) x)); - assertEquals((Double) 25.0, doubleAccum.value()); - - // Try a custom accumulator type - AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { - @Override - public Float addInPlace(Float r, Float t) { - return r + t; - } - - @Override - public Float addAccumulator(Float r, Float t) { - return r + t; - } - - @Override - public Float zero(Float initialValue) { - return 0.0f; - } - }; - - Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); - rdd.foreach(x -> floatAccum.add((float) x)); - assertEquals((Float) 25.0f, floatAccum.value()); - - // Test the setValue method - floatAccum.setValue(5.0f); - assertEquals((Float) 5.0f, floatAccum.value()); - } - @Test public void keyBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); @@ -1410,13 +1368,13 @@ public void sampleByKeyExact() { JavaPairRDD wrExact = rdd2.sampleByKeyExact(true, fractions, 1L); Map wrExactCounts = wrExact.countByKey(); assertEquals(2, wrExactCounts.size()); - assertTrue(wrExactCounts.get(0) == 2); - assertTrue(wrExactCounts.get(1) == 4); + assertEquals(2, (long) wrExactCounts.get(0)); + assertEquals(4, (long) wrExactCounts.get(1)); JavaPairRDD worExact = rdd2.sampleByKeyExact(false, fractions, 1L); Map worExactCounts = worExact.countByKey(); assertEquals(2, worExactCounts.size()); - assertTrue(worExactCounts.get(0) == 2); - assertTrue(worExactCounts.get(1) == 4); + assertEquals(2, (long) worExactCounts.get(0)); + assertEquals(4, (long) worExactCounts.get(1)); } private static class SomeCustomClass implements Serializable { diff --git a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java index f8e233a05a447..62a0b85915efc 100644 --- a/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java +++ b/core/src/test/java/test/org/apache/spark/JavaTaskContextCompileCheck.java @@ -40,6 +40,10 @@ public static void test() { tc.stageId(); tc.stageAttemptNumber(); tc.taskAttemptId(); + tc.resources(); + tc.taskMetrics(); + tc.taskMemoryManager(); + tc.getLocalProperties(); } /** diff --git a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json index 4ed053899ee6c..a64617256d63a 100644 --- a/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/app_environment_expectation.json @@ -32,6 +32,11 @@ [ "spark.app.id", "app-20161116163331-0000" ], [ "spark.task.maxFailures", "4" ] ], + "hadoopProperties" : [ + [ "mapreduce.jobtracker.address", "local" ], + [ "yarn.resourcemanager.scheduler.monitor.policies", "org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy" ], + [ "mapreduce.jobhistory.client.thread-count", "10" ] + ], "systemProperties" : [ [ "java.io.tmpdir", "/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/" ], [ "line.separator", "\n" ], diff --git a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json index eea6f595efd2a..71ccedced7886 100644 --- a/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/application_list_json_expectation.json @@ -1,4 +1,37 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { + "id" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json index 5e9e8230e2745..b18b19f7eeffb 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_for_stage_expectation.json @@ -1,639 +1,721 @@ { - "status": "COMPLETE", - "stageId": 0, - "attemptId": 0, - "numTasks": 10, - "numActiveTasks": 0, - "numCompleteTasks": 10, - "numFailedTasks": 2, - "numKilledTasks": 0, - "numCompletedIndices": 10, - "executorRunTime": 761, - "executorCpuTime": 269916000, - "submissionTime": "2018-01-09T10:21:18.152GMT", - "firstTaskLaunchedTime": "2018-01-09T10:21:18.347GMT", - "completionTime": "2018-01-09T10:21:19.062GMT", - "inputBytes": 0, - "inputRecords": 0, - "outputBytes": 0, - "outputRecords": 0, - "shuffleReadBytes": 0, - "shuffleReadRecords": 0, - "shuffleWriteBytes": 460, - "shuffleWriteRecords": 10, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "name": "map at :26", - "details": "org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:34)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:36)\n$line17.$read$$iw$$iw$$iw$$iw$$iw.(:38)\n$line17.$read$$iw$$iw$$iw$$iw.(:40)\n$line17.$read$$iw$$iw$$iw.(:42)\n$line17.$read$$iw$$iw.(:44)\n$line17.$read$$iw.(:46)\n$line17.$read.(:48)\n$line17.$read$.(:52)\n$line17.$read$.()\n$line17.$eval$.$print$lzycompute(:7)\n$line17.$eval$.$print(:6)\n$line17.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)", - "schedulingPool": "default", - "rddIds": [ - 1, - 0 - ], - "accumulatorUpdates": [], - "tasks": { - "0": { - "taskId": 0, - "index": 0, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.347GMT", - "duration": 562, - "executorId": "0", - "host": "172.30.65.138", - "status": "FAILED", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "errorMessage": "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n", - "taskMetrics": { - "executorDeserializeTime": 0, - "executorDeserializeCpuTime": 0, - "executorRunTime": 460, - "executorCpuTime": 0, - "resultSize": 0, - "jvmGcTime": 14, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 0, - "writeTime": 3873006, - "recordsWritten": 0 + "status" : "COMPLETE", + "stageId" : 0, + "attemptId" : 0, + "numTasks" : 10, + "numActiveTasks" : 0, + "numCompleteTasks" : 10, + "numFailedTasks" : 2, + "numKilledTasks" : 0, + "numCompletedIndices" : 10, + "submissionTime" : "2018-01-09T10:21:18.152GMT", + "firstTaskLaunchedTime" : "2018-01-09T10:21:18.347GMT", + "completionTime" : "2018-01-09T10:21:19.062GMT", + "executorDeserializeTime" : 327, + "executorDeserializeCpuTime" : 225900000, + "executorRunTime" : 761, + "executorCpuTime" : 269916000, + "resultSize" : 10376, + "jvmGcTime" : 27, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, + "shuffleReadBytes" : 0, + "shuffleReadRecords" : 0, + "shuffleWriteBytes" : 460, + "shuffleWriteTime" : 8711515, + "shuffleWriteRecords" : 10, + "name" : "map at :26", + "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:26)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:34)\n$line17.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:36)\n$line17.$read$$iw$$iw$$iw$$iw$$iw.(:38)\n$line17.$read$$iw$$iw$$iw$$iw.(:40)\n$line17.$read$$iw$$iw$$iw.(:42)\n$line17.$read$$iw$$iw.(:44)\n$line17.$read$$iw.(:46)\n$line17.$read.(:48)\n$line17.$read$.(:52)\n$line17.$read$.()\n$line17.$eval$.$print$lzycompute(:7)\n$line17.$eval$.$print(:6)\n$line17.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)", + "schedulingPool" : "default", + "rddIds" : [ 1, 0 ], + "accumulatorUpdates" : [ ], + "tasks" : { + "0" : { + "taskId" : 0, + "index" : 0, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.347GMT", + "duration" : 562, + "executorId" : "0", + "host" : "172.30.65.138", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 460, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 14, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 3873006, + "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64279/logPage/?appId=app-20180109111548-0000&executorId=0&logType=stdout", + "stderr" : "http://172.30.65.138:64279/logPage/?appId=app-20180109111548-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 102, + "gettingResultTime" : 0 }, - "5": { - "taskId": 5, - "index": 3, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.958GMT", - "duration": 22, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 3, - "executorDeserializeCpuTime": 2586000, - "executorRunTime": 9, - "executorCpuTime": 9635000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 262919, - "recordsWritten": 1 + "5" : { + "taskId" : 5, + "index" : 3, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.958GMT", + "duration" : 22, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 2586000, + "executorRunTime" : 9, + "executorCpuTime" : 9635000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 262919, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 10, + "gettingResultTime" : 0 }, - "10": { - "taskId": 10, - "index": 8, - "attempt": 0, - "launchTime": "2018-01-09T10:21:19.034GMT", - "duration": 12, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 2, - "executorDeserializeCpuTime": 1803000, - "executorRunTime": 6, - "executorCpuTime": 6157000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 243647, - "recordsWritten": 1 + "10" : { + "taskId" : 10, + "index" : 8, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:19.034GMT", + "duration" : 12, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 1803000, + "executorRunTime" : 6, + "executorCpuTime" : 6157000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 243647, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, - "1": { - "taskId": 1, - "index": 1, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.364GMT", - "duration": 565, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 301, - "executorDeserializeCpuTime": 200029000, - "executorRunTime": 212, - "executorCpuTime": 198479000, - "resultSize": 1115, - "jvmGcTime": 13, - "resultSerializationTime": 1, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 2409488, - "recordsWritten": 1 + "1" : { + "taskId" : 1, + "index" : 1, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.364GMT", + "duration" : 565, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 301, + "executorDeserializeCpuTime" : 200029000, + "executorRunTime" : 212, + "executorCpuTime" : 198479000, + "resultSize" : 1115, + "jvmGcTime" : 13, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 2409488, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 51, + "gettingResultTime" : 0 }, - "6": { - "taskId": 6, - "index": 4, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.980GMT", - "duration": 16, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 3, - "executorDeserializeCpuTime": 2610000, - "executorRunTime": 10, - "executorCpuTime": 9622000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 385110, - "recordsWritten": 1 + "6" : { + "taskId" : 6, + "index" : 4, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.980GMT", + "duration" : 16, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 2610000, + "executorRunTime" : 10, + "executorCpuTime" : 9622000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 385110, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, - "9": { - "taskId": 9, - "index": 7, - "attempt": 0, - "launchTime": "2018-01-09T10:21:19.022GMT", - "duration": 12, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 2, - "executorDeserializeCpuTime": 1981000, - "executorRunTime": 7, - "executorCpuTime": 6335000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 259354, - "recordsWritten": 1 + "9" : { + "taskId" : 9, + "index" : 7, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:19.022GMT", + "duration" : 12, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 1981000, + "executorRunTime" : 7, + "executorCpuTime" : 6335000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 259354, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, - "2": { - "taskId": 2, - "index": 2, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.899GMT", - "duration": 27, - "executorId": "0", - "host": "172.30.65.138", - "status": "FAILED", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "errorMessage": "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n", - "taskMetrics": { - "executorDeserializeTime": 0, - "executorDeserializeCpuTime": 0, - "executorRunTime": 16, - "executorCpuTime": 0, - "resultSize": 0, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 0, - "writeTime": 126128, - "recordsWritten": 0 + "2" : { + "taskId" : 2, + "index" : 2, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.899GMT", + "duration" : 27, + "executorId" : "0", + "host" : "172.30.65.138", + "status" : "FAILED", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "errorMessage" : "java.lang.RuntimeException: Bad executor\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:27)\n\tat $line17.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$anonfun$2.apply(:26)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:409)\n\tat org.apache.spark.util.collection.ExternalSorter.insertAll(ExternalSorter.scala:193)\n\tat org.apache.spark.shuffle.sort.SortShuffleWriter.write(SortShuffleWriter.scala:63)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:96)\n\tat org.apache.spark.scheduler.ShuffleMapTask.runTask(ShuffleMapTask.scala:53)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:109)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:345)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)\n\tat java.lang.Thread.run(Thread.java:748)\n", + "taskMetrics" : { + "executorDeserializeTime" : 0, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 16, + "executorCpuTime" : 0, + "resultSize" : 0, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 0, + "writeTime" : 126128, + "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64279/logPage/?appId=app-20180109111548-0000&executorId=0&logType=stdout", + "stderr" : "http://172.30.65.138:64279/logPage/?appId=app-20180109111548-0000&executorId=0&logType=stderr" + }, + "schedulerDelay" : 11, + "gettingResultTime" : 0 }, - "7": { - "taskId": 7, - "index": 5, - "attempt": 0, - "launchTime": "2018-01-09T10:21:18.996GMT", - "duration": 15, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 2, - "executorDeserializeCpuTime": 2231000, - "executorRunTime": 9, - "executorCpuTime": 8407000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 205520, - "recordsWritten": 1 + "7" : { + "taskId" : 7, + "index" : 5, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:18.996GMT", + "duration" : 15, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 2231000, + "executorRunTime" : 9, + "executorCpuTime" : 8407000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 205520, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, - "3": { - "taskId": 3, - "index": 0, - "attempt": 1, - "launchTime": "2018-01-09T10:21:18.919GMT", - "duration": 24, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 8, - "executorDeserializeCpuTime": 8878000, - "executorRunTime": 10, - "executorCpuTime": 9364000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 207014, - "recordsWritten": 1 + "3" : { + "taskId" : 3, + "index" : 0, + "attempt" : 1, + "launchTime" : "2018-01-09T10:21:18.919GMT", + "duration" : 24, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 8, + "executorDeserializeCpuTime" : 8878000, + "executorRunTime" : 10, + "executorCpuTime" : 9364000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 207014, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, - "11": { - "taskId": 11, - "index": 9, - "attempt": 0, - "launchTime": "2018-01-09T10:21:19.045GMT", - "duration": 15, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 3, - "executorDeserializeCpuTime": 2017000, - "executorRunTime": 6, - "executorCpuTime": 6676000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 233652, - "recordsWritten": 1 + "11" : { + "taskId" : 11, + "index" : 9, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:19.045GMT", + "duration" : 15, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 3, + "executorDeserializeCpuTime" : 2017000, + "executorRunTime" : 6, + "executorCpuTime" : 6676000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 233652, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, - "8": { - "taskId": 8, - "index": 6, - "attempt": 0, - "launchTime": "2018-01-09T10:21:19.011GMT", - "duration": 11, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 1, - "executorDeserializeCpuTime": 1554000, - "executorRunTime": 7, - "executorCpuTime": 6034000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 213296, - "recordsWritten": 1 + "8" : { + "taskId" : 8, + "index" : 6, + "attempt" : 0, + "launchTime" : "2018-01-09T10:21:19.011GMT", + "duration" : 11, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 1, + "executorDeserializeCpuTime" : 1554000, + "executorRunTime" : 7, + "executorCpuTime" : 6034000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 213296, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, - "4": { - "taskId": 4, - "index": 2, - "attempt": 1, - "launchTime": "2018-01-09T10:21:18.943GMT", - "duration": 16, - "executorId": "1", - "host": "172.30.65.138", - "status": "SUCCESS", - "taskLocality": "PROCESS_LOCAL", - "speculative": false, - "accumulatorUpdates": [], - "taskMetrics": { - "executorDeserializeTime": 2, - "executorDeserializeCpuTime": 2211000, - "executorRunTime": 9, - "executorCpuTime": 9207000, - "resultSize": 1029, - "jvmGcTime": 0, - "resultSerializationTime": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "peakExecutionMemory": 0, - "inputMetrics": { - "bytesRead": 0, - "recordsRead": 0 - }, - "outputMetrics": { - "bytesWritten": 0, - "recordsWritten": 0 - }, - "shuffleReadMetrics": { - "remoteBlocksFetched": 0, - "localBlocksFetched": 0, - "fetchWaitTime": 0, - "remoteBytesRead": 0, - "remoteBytesReadToDisk": 0, - "localBytesRead": 0, - "recordsRead": 0 - }, - "shuffleWriteMetrics": { - "bytesWritten": 46, - "writeTime": 292381, - "recordsWritten": 1 + "4" : { + "taskId" : 4, + "index" : 2, + "attempt" : 1, + "launchTime" : "2018-01-09T10:21:18.943GMT", + "duration" : 16, + "executorId" : "1", + "host" : "172.30.65.138", + "status" : "SUCCESS", + "taskLocality" : "PROCESS_LOCAL", + "speculative" : false, + "accumulatorUpdates" : [ ], + "taskMetrics" : { + "executorDeserializeTime" : 2, + "executorDeserializeCpuTime" : 2211000, + "executorRunTime" : 9, + "executorCpuTime" : 9207000, + "resultSize" : 1029, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, + "inputMetrics" : { + "bytesRead" : 0, + "recordsRead" : 0 + }, + "outputMetrics" : { + "bytesWritten" : 0, + "recordsWritten" : 0 + }, + "shuffleReadMetrics" : { + "remoteBlocksFetched" : 0, + "localBlocksFetched" : 0, + "fetchWaitTime" : 0, + "remoteBytesRead" : 0, + "remoteBytesReadToDisk" : 0, + "localBytesRead" : 0, + "recordsRead" : 0 + }, + "shuffleWriteMetrics" : { + "bytesWritten" : 46, + "writeTime" : 292381, + "recordsWritten" : 1 } - } + }, + "executorLogs" : { + "stdout" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stdout", + "stderr" : "http://172.30.65.138:64278/logPage/?appId=app-20180109111548-0000&executorId=1&logType=stderr" + }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 } }, - "executorSummary": { - "0": { - "taskTime": 589, - "failedTasks": 2, - "succeededTasks": 0, - "killedTasks": 0, - "inputBytes": 0, - "inputRecords": 0, - "outputBytes": 0, - "outputRecords": 0, - "shuffleRead": 0, - "shuffleReadRecords": 0, - "shuffleWrite": 0, - "shuffleWriteRecords": 0, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "isBlacklistedForStage": true + "executorSummary" : { + "0" : { + "taskTime" : 589, + "failedTasks" : 2, + "succeededTasks" : 0, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 0, + "shuffleWriteRecords" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : true }, - "1": { - "taskTime": 708, - "failedTasks": 0, - "succeededTasks": 10, - "killedTasks": 0, - "inputBytes": 0, - "inputRecords": 0, - "outputBytes": 0, - "outputRecords": 0, - "shuffleRead": 0, - "shuffleReadRecords": 0, - "shuffleWrite": 460, - "shuffleWriteRecords": 10, - "memoryBytesSpilled": 0, - "diskBytesSpilled": 0, - "isBlacklistedForStage": false + "1" : { + "taskTime" : 708, + "failedTasks" : 0, + "succeededTasks" : 10, + "killedTasks" : 0, + "inputBytes" : 0, + "inputRecords" : 0, + "outputBytes" : 0, + "outputRecords" : 0, + "shuffleRead" : 0, + "shuffleReadRecords" : 0, + "shuffleWrite" : 460, + "shuffleWriteRecords" : 10, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "isBlacklistedForStage" : false } }, - "killedTasksSummary": {} + "killedTasksSummary" : { } } diff --git a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json index acd4cc53de6cd..8d11081247913 100644 --- a/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/blacklisting_node_for_stage_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 4, "numKilledTasks" : 0, "numCompletedIndices" : 10, - "executorRunTime" : 5080, - "executorCpuTime" : 1163210819, "submissionTime" : "2018-01-18T18:33:12.658GMT", "firstTaskLaunchedTime" : "2018-01-18T18:33:12.816GMT", "completionTime" : "2018-01-18T18:33:15.279GMT", + "executorDeserializeTime" : 3679, + "executorDeserializeCpuTime" : 1029819716, + "executorRunTime" : 5080, + "executorCpuTime" : 1163210819, + "resultSize" : 10824, + "jvmGcTime" : 370, + "resultSerializationTime" : 5, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 1461, + "shuffleWriteTime" : 33251697, "shuffleWriteRecords" : 30, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "map at :27", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:370)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:27)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw$$iw.(:35)\n$line15.$read$$iw$$iw$$iw$$iw$$iw$$iw.(:37)\n$line15.$read$$iw$$iw$$iw$$iw$$iw.(:39)\n$line15.$read$$iw$$iw$$iw$$iw.(:41)\n$line15.$read$$iw$$iw$$iw.(:43)\n$line15.$read$$iw$$iw.(:45)\n$line15.$read$$iw.(:47)\n$line15.$read.(:49)\n$line15.$read$.(:53)\n$line15.$read$.()\n$line15.$eval$.$print$lzycompute(:7)\n$line15.$eval$.$print(:6)\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\nscala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:786)", "schedulingPool" : "default", @@ -74,7 +87,13 @@ "writeTime" : 3662221, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 68, + "gettingResultTime" : 0 }, "5" : { "taskId" : 5, @@ -122,7 +141,13 @@ "writeTime" : 191901, "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000007/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000007/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 46, + "gettingResultTime" : 0 }, "10" : { "taskId" : 10, @@ -169,7 +194,13 @@ "writeTime" : 301705, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 50, + "gettingResultTime" : 0 }, "1" : { "taskId" : 1, @@ -217,7 +248,13 @@ "writeTime" : 3075188, "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000007/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000007/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 174, + "gettingResultTime" : 0 }, "6" : { "taskId" : 6, @@ -265,7 +302,13 @@ "writeTime" : 183718, "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000005/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000005/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -312,7 +355,13 @@ "writeTime" : 366050, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 42, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -359,7 +408,13 @@ "writeTime" : 369513, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 26, + "gettingResultTime" : 0 }, "2" : { "taskId" : 2, @@ -406,7 +461,13 @@ "writeTime" : 3322956, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000004/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000004/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 74, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -453,7 +514,13 @@ "writeTime" : 319101, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, "7" : { "taskId" : 7, @@ -500,7 +567,13 @@ "writeTime" : 377601, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, "3" : { "taskId" : 3, @@ -547,7 +620,13 @@ "writeTime" : 3587839, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000003/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 63, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -594,7 +673,13 @@ "writeTime" : 323898, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 12, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -641,7 +726,13 @@ "writeTime" : 311940, "recordsWritten" : 3 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-3.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000002/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 84, + "gettingResultTime" : 0 }, "4" : { "taskId" : 4, @@ -689,7 +780,13 @@ "writeTime" : 16858066, "recordsWritten" : 0 } - } + }, + "executorLogs" : { + "stdout" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000005/attilapiros/stdout?start=-4096", + "stderr" : "http://apiros-2.gce.test.com:8042/node/containerlogs/container_1516285256255_0012_01_000005/attilapiros/stderr?start=-4096" + }, + "schedulerDelay" : 338, + "gettingResultTime" : 0 } }, "executorSummary" : { @@ -780,4 +877,4 @@ } }, "killedTasksSummary" : { } -} \ No newline at end of file +} diff --git a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json index 37b7d7269059f..a47cd26ed102b 100644 --- a/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/complete_stage_list_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 162, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT", "completionTime" : "2015-02-03T16:43:07.226GMT", + "executorDeserializeTime" : 36, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 162, + "executorCpuTime" : 0, + "resultSize" : 14496, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 160, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -39,21 +52,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 3476, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", "completionTime" : "2015-02-03T16:43:06.286GMT", + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 3476, + "executorCpuTime" : 0, + "resultSize" : 15216, + "jvmGcTime" : 152, + "resultSerializationTime" : 9, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 13180, + "shuffleWriteTime" : 692000, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -70,21 +96,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 4338, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT", "completionTime" : "2015-02-03T16:43:04.819GMT", + "executorDeserializeTime" : 91, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 4338, + "executorCpuTime" : 0, + "resultSize" : 10144, + "jvmGcTime" : 200, + "resultSerializationTime" : 5, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", diff --git a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json index 7bc7f31be097b..ad5f0ea9699d4 100644 --- a/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/completed_app_list_json_expectation.json @@ -1,4 +1,37 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { + "id" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json index 7bb8fe8fd8f98..8a977a7601abb 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_json_expectation.json @@ -1,5 +1,5 @@ [ { - "id" : "", + "id" : "driver", "hostPort" : "localhost:57971", "isActive" : true, "rddBlocks" : 0, @@ -20,5 +20,6 @@ "maxMemory" : 278302556, "addTime" : "2015-02-03T16:43:00.906GMT", "executorLogs" : { }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json new file mode 100644 index 0000000000000..f0f39e60bdd55 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_garbage_collection_metrics_json_expectation.json @@ -0,0 +1,122 @@ +[ { + "id" : "driver", + "hostPort" : "047.company.com:42509", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 100977868, + "addTime" : "2019-01-08T04:33:44.502GMT", + "executorLogs" : { + "stdout" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096", + "stderr" : "https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 100977868, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "JVMHeapMemory" : 211171816, + "JVMOffHeapMemory" : 90237256, + "OnHeapExecutionMemory" : 0, + "OffHeapExecutionMemory" : 0, + "OnHeapStorageMemory" : 4876, + "OffHeapStorageMemory" : 0, + "OnHeapUnifiedMemory" : 4876, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 806275, + "MappedPoolMemory" : 0, + "ProcessTreeJVMVMemory" : 2646888448, + "ProcessTreeJVMRSSMemory" : 520900608, + "ProcessTreePythonVMemory" : 0, + "ProcessTreePythonRSSMemory" : 0, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount" : 8, + "MinorGCTime" : 374, + "MajorGCCount" : 3, + "MajorGCTime" : 170 + }, + "attributes" : { } +}, { + "id" : "2", + "hostPort" : "028.company.com:46325", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 52, + "totalTasks" : 52, + "totalDuration" : 8879, + "totalGCTime" : 420, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 97832140, + "addTime" : "2019-01-08T04:33:54.270GMT", + "executorLogs" : { + "stdout" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096", + "stderr" : "https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 97832140, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { } +}, { + "id" : "1", + "hostPort" : "036.company.com:35126", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 4, + "maxTasks" : 4, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 48, + "totalTasks" : 48, + "totalDuration" : 8837, + "totalGCTime" : 1192, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 97832140, + "addTime" : "2019-01-08T04:33:55.929GMT", + "executorLogs" : { + "stdout" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096", + "stderr" : "https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 97832140, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "attributes" : { } +} ] \ No newline at end of file diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json index 9bf2086cc8e72..3db537799b292 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_metrics_json_expectation.json @@ -37,8 +37,19 @@ "DirectPoolMemory" : 397602, "MappedPoolMemory" : 0, "JVMHeapMemory" : 629553808, - "OffHeapStorageMemory" : 0 - } + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } }, { "id" : "7", "hostPort" : "node6340.grid.company.com:5933", @@ -70,7 +81,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "6", "hostPort" : "node6644.grid.company.com:8445", @@ -102,7 +114,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "5", "hostPort" : "node2477.grid.company.com:20123", @@ -134,7 +147,8 @@ "totalOnHeapStorageMemory" : 956615884, "totalOffHeapStorageMemory" : 0 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "4", "hostPort" : "node4243.grid.company.com:16084", @@ -177,8 +191,19 @@ "DirectPoolMemory" : 126261, "MappedPoolMemory" : 0, "JVMHeapMemory" : 518613056, - "OffHeapStorageMemory" : 0 - } + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } }, { "id" : "3", "hostPort" : "node0998.grid.company.com:45265", @@ -221,8 +246,19 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 726805712, - "OffHeapStorageMemory" : 0 - } + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } }, { "id" : "2", "hostPort" : "node4045.grid.company.com:29262", @@ -265,8 +301,19 @@ "DirectPoolMemory" : 87796, "MappedPoolMemory" : 0, "JVMHeapMemory" : 595946552, - "OffHeapStorageMemory" : 0 - } + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } }, { "id" : "1", "hostPort" : "node1404.grid.company.com:34043", @@ -309,6 +356,17 @@ "DirectPoolMemory" : 98230, "MappedPoolMemory" : 0, "JVMHeapMemory" : 755008624, - "OffHeapStorageMemory" : 0 - } + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory": 0, + "ProcessTreeJVMRSSMemory": 0, + "ProcessTreePythonVMemory": 0, + "ProcessTreePythonRSSMemory": 0, + "ProcessTreeOtherVMemory": 0, + "ProcessTreeOtherRSSMemory": 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json new file mode 100644 index 0000000000000..2c2efb58341b6 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/executor_list_with_executor_process_tree_metrics_json_expectation.json @@ -0,0 +1,108 @@ +[ { + "id" : "driver", + "hostPort" : "rezamemory-1.gce.something.com:43959", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 0, + "maxTasks" : 0, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 0, + "totalTasks" : 0, + "totalDuration" : 0, + "totalGCTime" : 0, + "totalInputBytes" : 0, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2018-10-02T00:42:47.690GMT", + "executorLogs" : { }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "OnHeapStorageMemory" : 554933, + "JVMOffHeapMemory" : 104976128, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 554933, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 228407, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 350990264, + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory" : 5067235328, + "ProcessTreeJVMRSSMemory" : 710475776, + "ProcessTreePythonVMemory" : 408375296, + "ProcessTreePythonRSSMemory" : 40284160, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } +}, { + "id" : "9", + "hostPort" : "rezamemory-2.gce.something.com:40797", + "isActive" : true, + "rddBlocks" : 0, + "memoryUsed" : 0, + "diskUsed" : 0, + "totalCores" : 1, + "maxTasks" : 1, + "activeTasks" : 0, + "failedTasks" : 0, + "completedTasks" : 2, + "totalTasks" : 2, + "totalDuration" : 6191, + "totalGCTime" : 288, + "totalInputBytes" : 108, + "totalShuffleRead" : 0, + "totalShuffleWrite" : 0, + "isBlacklisted" : false, + "maxMemory" : 384093388, + "addTime" : "2018-10-02T00:43:56.142GMT", + "executorLogs" : { + "stdout" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096", + "stderr" : "http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096" + }, + "memoryMetrics" : { + "usedOnHeapStorageMemory" : 0, + "usedOffHeapStorageMemory" : 0, + "totalOnHeapStorageMemory" : 384093388, + "totalOffHeapStorageMemory" : 0 + }, + "blacklistedInStages" : [ ], + "peakMemoryMetrics" : { + "OnHeapStorageMemory" : 1088805, + "JVMOffHeapMemory" : 59006656, + "OffHeapExecutionMemory" : 0, + "OnHeapUnifiedMemory" : 1088805, + "OnHeapExecutionMemory" : 0, + "OffHeapUnifiedMemory" : 0, + "DirectPoolMemory" : 20181, + "MappedPoolMemory" : 0, + "JVMHeapMemory" : 193766856, + "OffHeapStorageMemory" : 0, + "ProcessTreeJVMVMemory" : 3016261632, + "ProcessTreeJVMRSSMemory" : 405860352, + "ProcessTreePythonVMemory" : 625926144, + "ProcessTreePythonRSSMemory" : 69013504, + "ProcessTreeOtherVMemory" : 0, + "ProcessTreeOtherRSSMemory" : 0, + "MinorGCCount": 0, + "MinorGCTime": 0, + "MajorGCCount": 0, + "MajorGCTime": 0 + }, + "attributes" : { } +} ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json index dd5b1dcb7372b..c7f0a86211ad7 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_memory_usage_expectation.json @@ -26,7 +26,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -58,7 +59,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ,{ "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -90,7 +92,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -122,7 +125,8 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -154,5 +158,6 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json index 3e55d3d9d7eb9..02a0ecd7f4d80 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_expectation.json @@ -26,7 +26,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.167:51485", @@ -58,7 +59,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "2", "hostPort" : "172.22.0.167:51487", @@ -90,7 +92,8 @@ "totalOnHeapStorageMemory" : 384093388, "totalOffHeapStorageMemory" : 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.167:51490", @@ -122,7 +125,8 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.167:51491", @@ -154,5 +158,6 @@ "totalOnHeapStorageMemory": 384093388, "totalOffHeapStorageMemory": 524288000 }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json index e87f3e78f2dc8..4d31a387fbc53 100644 --- a/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/executor_node_blacklisting_unblacklisting_expectation.json @@ -20,7 +20,8 @@ "maxMemory" : 384093388, "addTime" : "2016-11-15T23:20:38.836GMT", "executorLogs" : { }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "3", "hostPort" : "172.22.0.111:64543", @@ -46,7 +47,8 @@ "stdout" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stdout", "stderr" : "http://172.22.0.111:64521/logPage/?appId=app-20161115172038-0000&executorId=3&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "2", "hostPort" : "172.22.0.111:64539", @@ -72,7 +74,8 @@ "stdout" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stdout", "stderr" : "http://172.22.0.111:64519/logPage/?appId=app-20161115172038-0000&executorId=2&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "1", "hostPort" : "172.22.0.111:64541", @@ -98,7 +101,8 @@ "stdout" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stdout", "stderr" : "http://172.22.0.111:64518/logPage/?appId=app-20161115172038-0000&executorId=1&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } }, { "id" : "0", "hostPort" : "172.22.0.111:64540", @@ -124,5 +128,6 @@ "stdout" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stdout", "stderr" : "http://172.22.0.111:64517/logPage/?appId=app-20161115172038-0000&executorId=0&logType=stderr" }, - "blacklistedInStages" : [ ] + "blacklistedInStages" : [ ], + "attributes" : { } } ] diff --git a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json index 2fd55666fa018..da26271e66bc4 100644 --- a/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/failed_stage_list_json_expectation.json @@ -8,22 +8,35 @@ "numFailedTasks" : 1, "numKilledTasks" : 0, "numCompletedIndices" : 7, - "executorRunTime" : 278, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:06.296GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", "completionTime" : "2015-02-03T16:43:06.347GMT", "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:", + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 278, + "executorCpuTime" : 0, + "resultSize" : 6034, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 64, + "shuffleFetchWaitTime" : 1, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :20", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", diff --git a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json index 9e1e65a358815..c303b6c565e38 100644 --- a/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/limit_app_list_json_expectation.json @@ -1,4 +1,37 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { + "id" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { @@ -13,34 +46,4 @@ "startTimeEpoch" : 1524182082734, "endTimeEpoch" : 1524182189134 } ] -}, { - "id" : "application_1516285256255_0012", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-18T18:30:35.119GMT", - "endTime" : "2018-01-18T18:38:27.938GMT", - "lastUpdated" : "", - "duration" : 472819, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1516300235119, - "endTimeEpoch" : 1516300707938 - } ] -}, { - "id" : "app-20180109111548-0000", - "name" : "Spark shell", - "attempts" : [ { - "startTime" : "2018-01-09T10:15:42.372GMT", - "endTime" : "2018-01-09T10:24:37.606GMT", - "lastUpdated" : "", - "duration" : 535234, - "sparkUser" : "attilapiros", - "completed" : true, - "appSparkVersion" : "2.3.0-SNAPSHOT", - "lastUpdatedEpoch" : 0, - "startTimeEpoch" : 1515492942372, - "endTimeEpoch" : 1515493477606 - } ] } ] diff --git a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json index 28c6bf1b3e01e..ba834d5f18d27 100644 --- a/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minDate_app_list_json_expectation.json @@ -1,4 +1,37 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { + "id" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json index f547b79f47e1a..62d3544e0ba11 100644 --- a/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/minEndDate_app_list_json_expectation.json @@ -1,4 +1,37 @@ [ { + "id": "application_1536831636016_59384", + "name": "Spark Pi", + "attempts": [ + { + "attemptId": "1", + "startTime": "2019-01-08T04:33:43.607GMT", + "endTime": "2019-01-08T04:33:58.745GMT", + "lastUpdated": "", + "duration": 15138, + "sparkUser": "lajin", + "completed": true, + "appSparkVersion": "3.0.0-SNAPSHOT", + "lastUpdatedEpoch": 0, + "startTimeEpoch": 1546922023607, + "endTimeEpoch": 1546922038745 + } + ] +}, { + "id" : "application_1538416563558_0014", + "name" : "PythonBisectingKMeansExample", + "attempts" : [ { + "startTime" : "2018-10-02T00:42:39.580GMT", + "endTime" : "2018-10-02T00:44:02.338GMT", + "lastUpdated" : "", + "duration" : 82758, + "sparkUser" : "root", + "completed" : true, + "appSparkVersion" : "2.5.0-SNAPSHOT", + "lastUpdatedEpoch" : 0, + "startTimeEpoch" : 1538440959580, + "endTimeEpoch" : 1538441042338 + } ] +}, { "id" : "application_1506645932520_24630151", "name" : "Spark shell", "attempts" : [ { diff --git a/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json new file mode 100644 index 0000000000000..09afdf5da5b62 --- /dev/null +++ b/core/src/test/resources/HistoryServerExpectations/one_rdd_storage_json_expectation.json @@ -0,0 +1,10 @@ +{ + "id" : 0, + "name" : "0", + "numPartitions" : 8, + "numCachedPartitions" : 0, + "storageLevel" : "Memory Deserialized 1x Replicated", + "memoryUsed" : 0, + "diskUsed" : 0, + "partitions" : [ ] +} diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json index 03f886afa5413..791907045e500 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_attempt_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 3476, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", "completionTime" : "2015-02-03T16:43:06.286GMT", + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 3476, + "executorCpuTime" : 0, + "resultSize" : 15216, + "jvmGcTime" : 152, + "resultSerializationTime" : 9, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 13180, + "shuffleWriteTime" : 692000, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -35,7 +48,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 456, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -74,7 +87,10 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 19, + "gettingResultTime" : 0 }, "14" : { "taskId" : 14, @@ -82,7 +98,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.832GMT", "duration" : 450, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -121,7 +137,10 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -129,7 +148,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -168,7 +187,10 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -176,7 +198,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 452, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -215,7 +237,10 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 14, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -223,7 +248,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -262,7 +287,10 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -270,7 +298,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -309,7 +337,10 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -317,7 +348,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -356,7 +387,10 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, "15" : { "taskId" : 15, @@ -364,7 +398,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", "duration" : 450, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -403,11 +437,14 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 } }, "executorSummary" : { - "" : { + "driver" : { "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, diff --git a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json index 947c89906955d..50d3f74ae775f 100644 --- a/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/one_stage_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 3476, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", "completionTime" : "2015-02-03T16:43:06.286GMT", + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 3476, + "executorCpuTime" : 0, + "resultSize" : 15216, + "jvmGcTime" : 152, + "resultSerializationTime" : 9, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 13180, + "shuffleWriteTime" : 692000, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -35,7 +48,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 456, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -74,7 +87,10 @@ "writeTime" : 76000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 19, + "gettingResultTime" : 0 }, "14" : { "taskId" : 14, @@ -82,7 +98,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.832GMT", "duration" : 450, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -121,7 +137,10 @@ "writeTime" : 88000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, "9" : { "taskId" : 9, @@ -129,7 +148,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -168,7 +187,10 @@ "writeTime" : 98000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "13" : { "taskId" : 13, @@ -176,7 +198,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 452, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -215,7 +237,10 @@ "writeTime" : 73000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 14, + "gettingResultTime" : 0 }, "12" : { "taskId" : 12, @@ -223,7 +248,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.831GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -262,7 +287,10 @@ "writeTime" : 101000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "11" : { "taskId" : 11, @@ -270,7 +298,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.830GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -309,7 +337,10 @@ "writeTime" : 83000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, "8" : { "taskId" : 8, @@ -317,7 +348,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.829GMT", "duration" : 454, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -356,7 +387,10 @@ "writeTime" : 94000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, "15" : { "taskId" : 15, @@ -364,7 +398,7 @@ "attempt" : 0, "launchTime" : "2015-02-03T16:43:05.833GMT", "duration" : 450, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -403,11 +437,14 @@ "writeTime" : 79000, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 } }, "executorSummary" : { - "" : { + "driver" : { "taskTime" : 3624, "failedTasks" : 0, "succeededTasks" : 8, diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json index 1e6fb40d60284..edbac7127039d 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 162, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:07.191GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:07.191GMT", "completionTime" : "2015-02-03T16:43:07.226GMT", + "executorDeserializeTime" : 36, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 162, + "executorCpuTime" : 0, + "resultSize" : 14496, + "jvmGcTime" : 0, + "resultSerializationTime" : 1, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 160, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :17", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -39,22 +52,35 @@ "numFailedTasks" : 1, "numKilledTasks" : 0, "numCompletedIndices" : 7, - "executorRunTime" : 278, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:06.296GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:06.296GMT", "completionTime" : "2015-02-03T16:43:06.347GMT", "failureReason" : "Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:", + "executorDeserializeTime" : 10, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 278, + "executorCpuTime" : 0, + "resultSize" : 6034, + "jvmGcTime" : 0, + "resultSerializationTime" : 0, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 64, + "shuffleFetchWaitTime" : 1, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :20", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -71,21 +97,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 3476, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:05.829GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:05.829GMT", "completionTime" : "2015-02-03T16:43:06.286GMT", + "executorDeserializeTime" : 13, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 3476, + "executorCpuTime" : 0, + "resultSize" : 15216, + "jvmGcTime" : 152, + "resultSerializationTime" : 9, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 28000128, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 13180, + "shuffleWriteTime" : 692000, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "map at :14", "details" : "org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -102,21 +141,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 4338, - "executorCpuTime" : 0, "submissionTime" : "2015-02-03T16:43:04.228GMT", "firstTaskLaunchedTime" : "2015-02-03T16:43:04.234GMT", "completionTime" : "2015-02-03T16:43:04.819GMT", + "executorDeserializeTime" : 91, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 4338, + "executorCpuTime" : 0, + "resultSize" : 10144, + "jvmGcTime" : 200, + "resultSerializationTime" : 5, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "count at :15", "details" : "org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", diff --git a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json index e6284ccf9b73d..836f2cb095097 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_list_with_accumulable_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 120, - "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT", "completionTime" : "2015-03-16T19:25:36.579GMT", + "executorDeserializeTime" : 102, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 120, + "executorCpuTime" : 0, + "resultSize" : 5576, + "jvmGcTime" : 0, + "resultSerializationTime" : 15, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "foreach at :15", "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json index a15ee23523365..f859ab6fff240 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_expectation.json @@ -43,7 +43,10 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 53, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -89,7 +92,10 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 40, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -135,7 +141,10 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 37, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -181,7 +190,10 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 41, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +239,10 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -273,7 +288,10 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 33, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, @@ -319,7 +337,10 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -365,7 +386,10 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 43, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -411,7 +435,10 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -457,7 +484,10 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 10, "index" : 10, @@ -503,7 +533,10 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 18, + "gettingResultTime" : 0 }, { "taskId" : 11, "index" : 11, @@ -549,7 +582,10 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -595,7 +631,10 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 }, { "taskId" : 13, "index" : 13, @@ -641,7 +680,10 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 53, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -687,7 +729,10 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -733,7 +778,10 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -779,7 +827,10 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -825,7 +876,10 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -871,7 +925,10 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -917,5 +974,8 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json index f9182b1658334..5a0d214ff3046 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_1__expectation.json @@ -4,7 +4,7 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", "duration" : 61, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -48,14 +48,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 30, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.521GMT", "duration" : 53, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -99,14 +102,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 48, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -150,14 +156,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 18, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 50, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -201,14 +210,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", "duration" : 52, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -252,14 +264,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 24, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 52, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -303,14 +318,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 23, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.523GMT", "duration" : 51, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -354,14 +372,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.524GMT", "duration" : 51, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -405,5 +426,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json index 76dd2f710b90f..fb9a1699a9cd5 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_from_multi_attempt_app_json_2__expectation.json @@ -4,7 +4,7 @@ "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.515GMT", "duration" : 61, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -48,14 +48,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 30, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.521GMT", "duration" : 53, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -99,14 +102,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 48, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -150,14 +156,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 18, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 50, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -201,14 +210,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.522GMT", "duration" : 52, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -252,14 +264,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 24, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", "duration" : 52, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -303,14 +318,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 23, + "gettingResultTime" : 0 }, { "taskId" : 6, "index" : 6, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.523GMT", "duration" : 51, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -354,14 +372,17 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, "attempt" : 0, "launchTime" : "2015-03-17T23:12:16.524GMT", "duration" : 51, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -405,5 +426,8 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json index 6bdc10465d89e..d83528d84972c 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__offset___length_expectation.json @@ -43,7 +43,10 @@ "writeTime" : 94709, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 18, + "gettingResultTime" : 0 }, { "taskId" : 11, "index" : 11, @@ -89,7 +92,10 @@ "writeTime" : 94507, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -135,7 +141,10 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 }, { "taskId" : 13, "index" : 13, @@ -181,7 +190,10 @@ "writeTime" : 95004, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 53, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -227,7 +239,10 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -273,7 +288,10 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -319,7 +337,10 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -365,7 +386,10 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -411,7 +435,10 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -457,7 +484,10 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -503,7 +533,10 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +582,10 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -595,7 +631,10 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 23, "index" : 23, @@ -641,7 +680,10 @@ "writeTime" : 91844, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, { "taskId" : 24, "index" : 24, @@ -687,7 +729,10 @@ "writeTime" : 157194, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 25, "index" : 25, @@ -733,7 +778,10 @@ "writeTime" : 94134, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 9, + "gettingResultTime" : 0 }, { "taskId" : 26, "index" : 26, @@ -779,7 +827,10 @@ "writeTime" : 108213, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 27, "index" : 27, @@ -825,7 +876,10 @@ "writeTime" : 102019, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 28, "index" : 28, @@ -871,7 +925,10 @@ "writeTime" : 104299, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 17, + "gettingResultTime" : 0 }, { "taskId" : 29, "index" : 29, @@ -917,7 +974,10 @@ "writeTime" : 114938, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 11, + "gettingResultTime" : 0 }, { "taskId" : 30, "index" : 30, @@ -963,7 +1023,10 @@ "writeTime" : 119770, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 24, + "gettingResultTime" : 0 }, { "taskId" : 31, "index" : 31, @@ -1009,7 +1072,10 @@ "writeTime" : 92619, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 14, + "gettingResultTime" : 0 }, { "taskId" : 32, "index" : 32, @@ -1055,7 +1121,10 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 33, "index" : 33, @@ -1101,7 +1170,10 @@ "writeTime" : 118329, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 10, + "gettingResultTime" : 0 }, { "taskId" : 34, "index" : 34, @@ -1147,7 +1219,10 @@ "writeTime" : 127746, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 9, + "gettingResultTime" : 0 }, { "taskId" : 35, "index" : 35, @@ -1193,7 +1268,10 @@ "writeTime" : 160963, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 10, + "gettingResultTime" : 0 }, { "taskId" : 36, "index" : 36, @@ -1239,7 +1317,10 @@ "writeTime" : 123855, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 37, "index" : 37, @@ -1285,7 +1366,10 @@ "writeTime" : 111869, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 38, "index" : 38, @@ -1331,7 +1415,10 @@ "writeTime" : 131158, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 39, "index" : 39, @@ -1377,7 +1464,10 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 40, "index" : 40, @@ -1423,7 +1513,10 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 41, "index" : 41, @@ -1469,7 +1562,10 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 42, "index" : 42, @@ -1515,7 +1611,10 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 15, + "gettingResultTime" : 0 }, { "taskId" : 43, "index" : 43, @@ -1561,7 +1660,10 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 21, + "gettingResultTime" : 0 }, { "taskId" : 44, "index" : 44, @@ -1607,7 +1709,10 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, { "taskId" : 45, "index" : 45, @@ -1653,7 +1758,10 @@ "writeTime" : 92985, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, { "taskId" : 46, "index" : 46, @@ -1699,7 +1807,10 @@ "writeTime" : 113322, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 11, + "gettingResultTime" : 0 }, { "taskId" : 47, "index" : 47, @@ -1745,7 +1856,10 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 48, "index" : 48, @@ -1791,7 +1905,10 @@ "writeTime" : 139844, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 49, "index" : 49, @@ -1837,7 +1954,10 @@ "writeTime" : 94984, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 50, "index" : 50, @@ -1883,7 +2003,10 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 51, "index" : 51, @@ -1929,7 +2052,10 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 }, { "taskId" : 52, "index" : 52, @@ -1975,7 +2101,10 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 53, "index" : 53, @@ -2021,7 +2150,10 @@ "writeTime" : 92835, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 54, "index" : 54, @@ -2067,7 +2199,10 @@ "writeTime" : 90506, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 55, "index" : 55, @@ -2113,7 +2248,10 @@ "writeTime" : 108309, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 56, "index" : 56, @@ -2159,7 +2297,10 @@ "writeTime" : 90329, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 57, "index" : 57, @@ -2205,7 +2346,10 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 }, { "taskId" : 58, "index" : 58, @@ -2251,7 +2395,10 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 59, "index" : 59, @@ -2297,5 +2444,8 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json index bc1cd49909d31..82e339c8f56dd 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_expectation.json @@ -43,7 +43,10 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -89,7 +92,10 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 33, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -135,7 +141,10 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 40, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -181,7 +190,10 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 43, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +239,10 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -273,7 +288,10 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 41, + "gettingResultTime" : 0 }, { "taskId" : 0, "index" : 0, @@ -319,7 +337,10 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 53, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -365,7 +386,10 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 37, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -411,7 +435,10 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -457,7 +484,10 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -503,7 +533,10 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +582,10 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -595,7 +631,10 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -641,7 +680,10 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -687,7 +729,10 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -733,7 +778,10 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -779,7 +827,10 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -825,7 +876,10 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -871,7 +925,10 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -917,5 +974,8 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json index bc1cd49909d31..82e339c8f56dd 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names___runtime_expectation.json @@ -43,7 +43,10 @@ "writeTime" : 4016617, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 5, "index" : 5, @@ -89,7 +92,10 @@ "writeTime" : 3675510, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 33, + "gettingResultTime" : 0 }, { "taskId" : 1, "index" : 1, @@ -135,7 +141,10 @@ "writeTime" : 3934399, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 40, + "gettingResultTime" : 0 }, { "taskId" : 7, "index" : 7, @@ -181,7 +190,10 @@ "writeTime" : 2579051, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 43, + "gettingResultTime" : 0 }, { "taskId" : 4, "index" : 4, @@ -227,7 +239,10 @@ "writeTime" : 83022, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 38, + "gettingResultTime" : 0 }, { "taskId" : 3, "index" : 3, @@ -273,7 +288,10 @@ "writeTime" : 1311694, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 41, + "gettingResultTime" : 0 }, { "taskId" : 0, "index" : 0, @@ -319,7 +337,10 @@ "writeTime" : 3842811, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 53, + "gettingResultTime" : 0 }, { "taskId" : 2, "index" : 2, @@ -365,7 +386,10 @@ "writeTime" : 89885, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 37, + "gettingResultTime" : 0 }, { "taskId" : 22, "index" : 22, @@ -411,7 +435,10 @@ "writeTime" : 143427, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 18, "index" : 18, @@ -457,7 +484,10 @@ "writeTime" : 100836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 17, "index" : 17, @@ -503,7 +533,10 @@ "writeTime" : 99944, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, { "taskId" : 21, "index" : 21, @@ -549,7 +582,10 @@ "writeTime" : 100270, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 19, "index" : 19, @@ -595,7 +631,10 @@ "writeTime" : 95788, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 16, "index" : 16, @@ -641,7 +680,10 @@ "writeTime" : 108320, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 9, "index" : 9, @@ -687,7 +729,10 @@ "writeTime" : 101664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 20, "index" : 20, @@ -733,7 +778,10 @@ "writeTime" : 97716, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 14, "index" : 14, @@ -779,7 +827,10 @@ "writeTime" : 95646, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 8, "index" : 8, @@ -825,7 +876,10 @@ "writeTime" : 121551, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 }, { "taskId" : 12, "index" : 12, @@ -871,7 +925,10 @@ "writeTime" : 102476, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 }, { "taskId" : 15, "index" : 15, @@ -917,5 +974,8 @@ "writeTime" : 602780, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json index 09857cb401acd..01eef1b565bf6 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_task_list_w__sortBy_short_names__runtime_expectation.json @@ -43,7 +43,10 @@ "writeTime" : 94792, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 41, "index" : 41, @@ -89,7 +92,10 @@ "writeTime" : 90765, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 6, + "gettingResultTime" : 0 }, { "taskId" : 43, "index" : 43, @@ -135,7 +141,10 @@ "writeTime" : 171516, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 21, + "gettingResultTime" : 0 }, { "taskId" : 57, "index" : 57, @@ -181,7 +190,10 @@ "writeTime" : 96849, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 }, { "taskId" : 58, "index" : 58, @@ -227,7 +239,10 @@ "writeTime" : 97521, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 68, "index" : 68, @@ -273,7 +288,10 @@ "writeTime" : 101750, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 86, "index" : 86, @@ -319,7 +337,10 @@ "writeTime" : 95848, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 8, + "gettingResultTime" : 0 }, { "taskId" : 32, "index" : 32, @@ -365,7 +386,10 @@ "writeTime" : 89603, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 39, "index" : 39, @@ -411,7 +435,10 @@ "writeTime" : 98748, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 42, "index" : 42, @@ -457,7 +484,10 @@ "writeTime" : 103713, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 15, + "gettingResultTime" : 0 }, { "taskId" : 51, "index" : 51, @@ -503,7 +533,10 @@ "writeTime" : 96013, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 2, + "gettingResultTime" : 0 }, { "taskId" : 59, "index" : 59, @@ -549,7 +582,10 @@ "writeTime" : 100753, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 63, "index" : 63, @@ -595,7 +631,10 @@ "writeTime" : 102779, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 87, "index" : 87, @@ -641,7 +680,10 @@ "writeTime" : 102159, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 7, + "gettingResultTime" : 0 }, { "taskId" : 90, "index" : 90, @@ -687,7 +729,10 @@ "writeTime" : 98472, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 99, "index" : 99, @@ -733,7 +778,10 @@ "writeTime" : 133964, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 3, + "gettingResultTime" : 0 }, { "taskId" : 44, "index" : 44, @@ -779,7 +827,10 @@ "writeTime" : 98293, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 16, + "gettingResultTime" : 0 }, { "taskId" : 47, "index" : 47, @@ -825,7 +876,10 @@ "writeTime" : 103015, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 13, + "gettingResultTime" : 0 }, { "taskId" : 50, "index" : 50, @@ -871,7 +925,10 @@ "writeTime" : 90836, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 4, + "gettingResultTime" : 0 }, { "taskId" : 52, "index" : 52, @@ -917,5 +974,8 @@ "writeTime" : 89664, "recordsWritten" : 10 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 5, + "gettingResultTime" : 0 } ] diff --git a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json index 963f010968b62..735a8257fc343 100644 --- a/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json +++ b/core/src/test/resources/HistoryServerExpectations/stage_with_accumulable_json_expectation.json @@ -8,21 +8,34 @@ "numFailedTasks" : 0, "numKilledTasks" : 0, "numCompletedIndices" : 8, - "executorRunTime" : 120, - "executorCpuTime" : 0, "submissionTime" : "2015-03-16T19:25:36.103GMT", "firstTaskLaunchedTime" : "2015-03-16T19:25:36.515GMT", "completionTime" : "2015-03-16T19:25:36.579GMT", + "executorDeserializeTime" : 102, + "executorDeserializeCpuTime" : 0, + "executorRunTime" : 120, + "executorCpuTime" : 0, + "resultSize" : 5576, + "jvmGcTime" : 0, + "resultSerializationTime" : 15, + "memoryBytesSpilled" : 0, + "diskBytesSpilled" : 0, + "peakExecutionMemory" : 0, "inputBytes" : 0, "inputRecords" : 0, "outputBytes" : 0, "outputRecords" : 0, + "shuffleRemoteBlocksFetched" : 0, + "shuffleLocalBlocksFetched" : 0, + "shuffleFetchWaitTime" : 0, + "shuffleRemoteBytesRead" : 0, + "shuffleRemoteBytesReadToDisk" : 0, + "shuffleLocalBytesRead" : 0, "shuffleReadBytes" : 0, "shuffleReadRecords" : 0, "shuffleWriteBytes" : 0, + "shuffleWriteTime" : 0, "shuffleWriteRecords" : 0, - "memoryBytesSpilled" : 0, - "diskBytesSpilled" : 0, "name" : "foreach at :15", "details" : "org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)", "schedulingPool" : "default", @@ -39,7 +52,7 @@ "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.515GMT", "duration" : 61, - "executorId" : "", + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -83,15 +96,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 30, + "gettingResultTime" : 0 }, - "1" : { - "taskId" : 1, - "index" : 1, + "5" : { + "taskId" : 5, + "index" : 5, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.521GMT", - "duration" : 53, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 52, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -99,11 +115,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "247", - "value" : "2175" + "update" : "897", + "value" : "3750" } ], "taskMetrics" : { - "executorDeserializeTime" : 14, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -135,15 +151,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 23, + "gettingResultTime" : 0 }, - "2" : { - "taskId" : 2, - "index" : 2, + "1" : { + "taskId" : 1, + "index" : 1, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 48, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.521GMT", + "duration" : 53, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -151,11 +170,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "378", - "value" : "378" + "update" : "247", + "value" : "2175" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 14, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -187,15 +206,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, - "3" : { - "taskId" : 3, - "index" : 3, + "6" : { + "taskId" : 6, + "index" : 6, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 50, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.523GMT", + "duration" : 51, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -203,11 +225,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "572", - "value" : "950" + "update" : "978", + "value" : "1928" } ], "taskMetrics" : { - "executorDeserializeTime" : 13, + "executorDeserializeTime" : 12, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -239,15 +261,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, - "4" : { - "taskId" : 4, - "index" : 4, + "2" : { + "taskId" : 2, + "index" : 2, "attempt" : 0, "launchTime" : "2015-03-16T19:25:36.522GMT", - "duration" : 52, - "executorId" : "", + "duration" : 48, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -255,17 +280,17 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "678", - "value" : "2853" + "update" : "378", + "value" : "378" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 1, + "resultSerializationTime" : 2, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "peakExecutionMemory" : 0, @@ -291,15 +316,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 18, + "gettingResultTime" : 0 }, - "5" : { - "taskId" : 5, - "index" : 5, + "7" : { + "taskId" : 7, + "index" : 7, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 52, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.524GMT", + "duration" : 51, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -307,8 +335,8 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "897", - "value" : "3750" + "update" : "1222", + "value" : "4972" } ], "taskMetrics" : { "executorDeserializeTime" : 12, @@ -343,15 +371,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 22, + "gettingResultTime" : 0 }, - "6" : { - "taskId" : 6, - "index" : 6, + "3" : { + "taskId" : 3, + "index" : 3, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.523GMT", - "duration" : 51, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 50, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -359,11 +390,11 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "978", - "value" : "1928" + "update" : "572", + "value" : "950" } ], "taskMetrics" : { - "executorDeserializeTime" : 12, + "executorDeserializeTime" : 13, "executorDeserializeCpuTime" : 0, "executorRunTime" : 15, "executorCpuTime" : 0, @@ -395,15 +426,18 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 20, + "gettingResultTime" : 0 }, - "7" : { - "taskId" : 7, - "index" : 7, + "4" : { + "taskId" : 4, + "index" : 4, "attempt" : 0, - "launchTime" : "2015-03-16T19:25:36.524GMT", - "duration" : 51, - "executorId" : "", + "launchTime" : "2015-03-16T19:25:36.522GMT", + "duration" : 52, + "executorId" : "driver", "host" : "localhost", "status" : "SUCCESS", "taskLocality" : "PROCESS_LOCAL", @@ -411,8 +445,8 @@ "accumulatorUpdates" : [ { "id" : 1, "name" : "my counter", - "update" : "1222", - "value" : "4972" + "update" : "678", + "value" : "2853" } ], "taskMetrics" : { "executorDeserializeTime" : 12, @@ -421,7 +455,7 @@ "executorCpuTime" : 0, "resultSize" : 697, "jvmGcTime" : 0, - "resultSerializationTime" : 2, + "resultSerializationTime" : 1, "memoryBytesSpilled" : 0, "diskBytesSpilled" : 0, "peakExecutionMemory" : 0, @@ -447,11 +481,14 @@ "writeTime" : 0, "recordsWritten" : 0 } - } + }, + "executorLogs" : { }, + "schedulerDelay" : 24, + "gettingResultTime" : 0 } }, "executorSummary" : { - "" : { + "driver" : { "taskTime" : 418, "failedTasks" : 0, "succeededTasks" : 8, diff --git a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager index cf8565c74e95e..60054c8ed5180 100644 --- a/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager +++ b/core/src/test/resources/META-INF/services/org.apache.spark.scheduler.ExternalClusterManager @@ -1,3 +1,3 @@ org.apache.spark.scheduler.DummyExternalClusterManager org.apache.spark.scheduler.MockExternalClusterManager -org.apache.spark.DummyLocalExternalClusterManager +org.apache.spark.scheduler.CSMockExternalClusterManager diff --git a/core/src/test/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider b/core/src/test/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider new file mode 100644 index 0000000000000..f4107befc825b --- /dev/null +++ b/core/src/test/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider @@ -0,0 +1 @@ +org.apache.spark.deploy.security.ExceptionThrowingDelegationTokenProvider diff --git a/core/src/test/resources/ProcfsMetrics/22763/stat b/core/src/test/resources/ProcfsMetrics/22763/stat new file mode 100644 index 0000000000000..cea4b713d0ee4 --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/22763/stat @@ -0,0 +1 @@ +22763 (python2.7) S 22756 22756 7051 0 -1 1077944384 449 0 0 0 4 3 0 0 20 0 3 0 117445 360595456 1912 18446744073709551615 4194304 4196756 140726192435536 140726192432528 140707465485051 0 0 16781312 2 18446744073709551615 0 0 17 1 0 0 0 0 0 6294976 6295604 38744064 140726192440006 140726192440119 140726192440119 140726192443369 0 \ No newline at end of file diff --git a/core/src/test/resources/ProcfsMetrics/26109/stat b/core/src/test/resources/ProcfsMetrics/26109/stat new file mode 100644 index 0000000000000..ae46bfabd047e --- /dev/null +++ b/core/src/test/resources/ProcfsMetrics/26109/stat @@ -0,0 +1 @@ +26109 (java) S 1 26107 5788 0 -1 1077944320 75354 0 0 0 572 52 0 0 20 0 34 0 4355257 4769947648 64114 18446744073709551615 4194304 4196468 140737190381776 140737190364320 139976994791319 0 0 0 16800975 18446744073709551615 0 0 17 2 0 0 0 0 0 6293624 6294260 11276288 140737190385424 140737190414250 140737190414250 140737190416335 0 diff --git a/core/src/test/resources/log4j.properties b/core/src/test/resources/log4j.properties index fb9d9851cb4de..2f46ce1553ee6 100644 --- a/core/src/test/resources/log4j.properties +++ b/core/src/test/resources/log4j.properties @@ -33,4 +33,4 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%t: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/core/src/test/resources/spark-events/app-20161116163331-0000 b/core/src/test/resources/spark-events/app-20161116163331-0000 index 57cfc5b973129..8f77fdd34f431 100755 --- a/core/src/test/resources/spark-events/app-20161116163331-0000 +++ b/core/src/test/resources/spark-events/app-20161116163331-0000 @@ -1,6 +1,6 @@ {"Event":"SparkListenerLogStart","Spark Version":"2.1.0-SNAPSHOT"} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"172.22.0.167","Port":51475},"Maximum Memory":908381388,"Timestamp":1479335611477,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} -{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","Java Version":"1.8.0_92 (Oracle Corporation)","Scala Version":"version 2.11.8"},"Spark Properties":{"spark.blacklist.task.maxTaskAttemptsPerExecutor":"3","spark.blacklist.enabled":"TRUE","spark.driver.host":"172.22.0.167","spark.blacklist.task.maxTaskAttemptsPerNode":"3","spark.eventLog.enabled":"TRUE","spark.driver.port":"51459","spark.repl.class.uri":"spark://172.22.0.167:51459/classes","spark.jars":"","spark.repl.class.outputDir":"/private/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/spark-1cbc97d0-7fe6-4c9f-8c2c-f6fe51ee3cf2/repl-39929169-ac4c-4c6d-b116-f648e4dd62ed","spark.app.name":"Spark shell","spark.blacklist.stage.maxFailedExecutorsPerNode":"3","spark.scheduler.mode":"FIFO","spark.eventLog.overwrite":"TRUE","spark.blacklist.stage.maxFailedTasksPerExecutor":"3","spark.executor.id":"driver","spark.blacklist.application.maxFailedExecutorsPerNode":"2","spark.submit.deployMode":"client","spark.master":"local-cluster[4,4,1024]","spark.home":"/Users/Jose/IdeaProjects/spark","spark.eventLog.dir":"/Users/jose/logs","spark.sql.catalogImplementation":"in-memory","spark.eventLog.compress":"FALSE","spark.blacklist.application.maxFailedTasksPerExecutor":"1","spark.blacklist.timeout":"1000000","spark.app.id":"app-20161116163331-0000","spark.task.maxFailures":"4"},"Hadoop Properties":{"mapreduce.jobtracker.address":"local","yarn.resourcemanager.scheduler.monitor.policies":"org.apache.hadoop.yarn.server.resourcemanager.monitor.capacity.ProportionalCapacityPreemptionPolicy","mapreduce.jobhistory.client.thread-count":"10"},"System Properties":{"java.io.tmpdir":"/var/folders/l4/d46wlzj16593f3d812vk49tw0000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/Jose","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib","user.dir":"/Users/Jose/IdeaProjects/spark","java.library.path":"/Users/Jose/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.92-b14","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_92-b14","java.vm.info":"mixed mode","java.ext.dirs":"/Users/Jose/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","io.netty.maxDirectMemory":"0","java.class.version":"52.0","scala.usejavacp":"true","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.11.6","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"jose","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --master local-cluster[4,4,1024] --conf spark.blacklist.enabled=TRUE --conf spark.blacklist.timeout=1000000 --conf spark.blacklist.application.maxFailedTasksPerExecutor=1 --conf spark.eventLog.overwrite=TRUE --conf spark.blacklist.task.maxTaskAttemptsPerNode=3 --conf spark.blacklist.stage.maxFailedTasksPerExecutor=3 --conf spark.blacklist.task.maxTaskAttemptsPerExecutor=3 --conf spark.eventLog.compress=FALSE --conf spark.blacklist.stage.maxFailedExecutorsPerNode=3 --conf spark.eventLog.enabled=TRUE --conf spark.eventLog.dir=/Users/jose/logs --conf spark.blacklist.application.maxFailedExecutorsPerNode=2 --conf spark.task.maxFailures=4 --class org.apache.spark.repl.Main --name Spark shell spark-shell -i /Users/Jose/dev/jose-utils/blacklist/test-blacklist.scala","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_92.jdk/Contents/Home/jre","java.version":"1.8.0_92","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-mapred-1.7.7-hadoop2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-core-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlet-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-column-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/snappy-java-1.1.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/oro-2.0.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/arpack_combined_all-0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-schema-1.2.15.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-assembly_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javassist-3.18.1-GA.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-tags_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-launcher_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math3-3.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-xml_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/objenesis-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire-macros_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-reflect-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib-local_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-mllib_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-server-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-scala_2.11-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-framework-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-client-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-common/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/zookeeper-3.4.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-auth-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/repl/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-io-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/catalyst/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-unsafe_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-repl_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-continuation-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive-thriftserver/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-annotations-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-graphite-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-api-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/streaming/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-net-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-proxy-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-catalyst_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/lz4-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-crypto-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.annotation-api-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sql_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guava-14.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-collections-3.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/conf/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/unused-1.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-encoding-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/tags/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-jackson_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-cli-1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-server-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/cglib-2.2.1-v20090111.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pyrolite-4.13.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-library-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-parser-combinators_2.11-1.0.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-6.1.26.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/py4j-0.10.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-configuration-1.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/core-1.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/core/target/jars/*":"System Classpath","/Users/Jose/IdeaProjects/spark/common/network-shuffle/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-format-2.3.0-incubating.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/kryo-shaded-3.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/core/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill-java-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-annotations-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-hadoop-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/sql/hive/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xz-1.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-jackson-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-common-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/log4j-1.2.17.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-core-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-util-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalap-2.11.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-1.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compress-1.4.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/yarn/target/scala-2.11/classes":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-plus-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/protobuf-java-2.5.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/unsafe/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-module-paranamer-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/leveldbjni-all-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-core-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-api-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/compress-lzf-1.0.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/stream-2.7.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-shuffle-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-codec-1.10.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-yarn-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/common/sketch/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-core_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-shuffle_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang-2.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/ivy-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-common-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-math-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-hdfs-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scala-compiler-2.11.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-jvm-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-lang3-3.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jsr305-1.3.9.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/minlog-1.3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-3.8.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-webapp-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-ast_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xbean-asm5-shaded-4.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-io-2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/shapeless_2.11-2.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-network-common_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-xml-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-httpclient-3.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/mllib/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/scalatest_2.11-2.2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-client-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-guava-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-jndi-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/graphx/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-app-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/examples/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/xmlenc-0.52.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jets3t-0.7.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/curator-recipes-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/opencsv-2.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jtransforms-2.4.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/antlr4-runtime-4.5.3.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/chill_2.11-0.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-digester-1.8.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/univocity-parsers-2.2.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jline-2.12.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-streaming_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/launcher/target/scala-2.11/classes/":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/breeze-macros_2.11-0.12.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jersey-client-2.22.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jackson-databind-2.6.5.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-servlets-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/paranamer-2.6.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-security-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-ipc-1.7.7-tests.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/avro-1.7.7.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spire_2.11-0.7.4.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-client-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/metrics-json-3.1.2.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/validation-api-1.1.0.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-graphx_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/netty-all-4.0.41.Final.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/janino-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/json4s-core_2.11-3.2.11.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/commons-compiler-3.0.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/guice-3.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-server-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/jetty-http-9.2.16.v20160414.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/parquet-common-1.8.1.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/hadoop-mapreduce-client-jobclient-2.2.0.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/spark-sketch_2.11-2.1.0-SNAPSHOT.jar":"System Classpath","/Users/Jose/IdeaProjects/spark/assembly/target/scala-2.11/jars/pmml-model-1.2.15.jar":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"app-20161116163331-0000","Timestamp":1479335609916,"User":"jose"} {"Event":"SparkListenerExecutorAdded","Timestamp":1479335615320,"Executor ID":"3","Executor Info":{"Host":"172.22.0.167","Total Cores":4,"Log Urls":{"stdout":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stdout","stderr":"http://172.22.0.167:51466/logPage/?appId=app-20161116163331-0000&executorId=3&logType=stderr"}}} {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"172.22.0.167","Port":51485},"Maximum Memory":908381388,"Timestamp":1479335615387,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":524288000} diff --git a/core/src/test/resources/spark-events/application_1536831636016_59384_1 b/core/src/test/resources/spark-events/application_1536831636016_59384_1 new file mode 100755 index 0000000000000..69924afead197 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1536831636016_59384_1 @@ -0,0 +1,214 @@ +{"Event":"SparkListenerLogStart","Spark Version":"3.0.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"047.company.com","Port":42509},"Maximum Memory":100977868,"Timestamp":1546922024502,"Maximum Onheap Memory":100977868,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/apache/releases/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.12.8"},"Spark Properties":{"spark.history.kerberos.keytab":"/etc/security/keytabs/spark.service.keytab","spark.executor.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.driver.host":"047.company.com","spark.history.fs.logDirectory":"hdfs://hercules-sub/spark-logs","spark.eventLog.enabled":"true","spark.ssl.historyServer.trustStore":"/etc/hadoop/truststore.jks","spark.ui.port":"0","spark.driver.port":"36796","spark.shuffle.service.enabled":"true","spark.driver.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.yarn.queue":"default","spark.history.fs.update.interval":"20s","spark.yarn.historyServer.address":"master2.company.com:50070","spark.yarn.app.id":"application_1536831636016_59384","spark.sql.function.eltOutputAsString":"true","spark.yarn.access.namenodes":"hdfs://hercules-sub,hdfs://hercules","spark.app.name":"Spark Pi","spark.scheduler.mode":"FIFO","spark.history.fs.numReplayThreads":"30","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.driver.memory":"512m","spark.executor.instances":"2","spark.history.kerberos.principal":"spark/master2.company.com@COMPANY.COM","spark.yarn.am.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorGCMetrics.enabled":"true","spark.ssl.historyServer.port":"49670","spark.ssl.historyServer.keyStoreType":"JKS","spark.sql.function.concatBinaryAsString":"true","spark.history.fs.cleaner.enabled":"true","spark.executor.id":"driver","spark.yarn.am.memory":"4g","spark.yarn.app.container.log.dir":"/hadoop/12/yarn/log/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","spark.driver.extraJavaOptions":"-XX:MaxMetaspaceSize=512m -XX:ParallelGCThreads=3","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.sql.hive.caseSensitiveInferenceMode":"NEVER_INFER","spark.submit.deployMode":"cluster","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.RM_HA_URLS":"master1.company.com:50030,master2.company.com:50030","spark.master":"yarn","spark.ssl.historyServer.keyStorePassword":"*********(redacted)","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native","spark.executor.memory":"512m","spark.driver.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.eventLog.dir":"hdfs://hercules-sub/spark-logs","spark.ssl.historyServer.enabled":"true","spark.ssl.historyServer.keyStore":"/etc/hadoop/keystore.jks","spark.dynamicAllocation.enabled":"false","spark.executor.extraClassPath":"/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar:/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar","spark.executor.cores":"4","spark.history.ui.port":"49670","spark.ssl.historyServer.trustStorePassword":"*********(redacted)","spark.ssl.historyServer.protocol":"TLSv1.2","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"master1.company.com,master2.company.com","spark.history.kerberos.enabled":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"https://master1.company.com:50030/proxy/application_1536831636016_59384,https://master2.company.com:50030/proxy/application_1536831636016_59384","spark.ssl.historyServer.trustStoreType":"JKS","spark.app.id":"application_1536831636016_59384"},"System Properties":{"java.io.tmpdir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/home/lajin","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/apache/releases/jdk1.8.0_121/jre/lib/amd64","user.dir":"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001","java.library.path":"/apache/hadoop/lib/native/Linux-amd64-64/lib:/apache/hadoop/lib/native::/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"27208684755d94a92186989f695db2d7b21ebc51","java.endorsed.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/apache/releases/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/apache/releases/jdk1.8.0_121/jre/lib/resources.jar:/apache/releases/jdk1.8.0_121/jre/lib/rt.jar:/apache/releases/jdk1.8.0_121/jre/lib/sunrsasign.jar:/apache/releases/jdk1.8.0_121/jre/lib/jsse.jar:/apache/releases/jdk1.8.0_121/jre/lib/jce.jar:/apache/releases/jdk1.8.0_121/jre/lib/charsets.jar:/apache/releases/jdk1.8.0_121/jre/lib/jfr.jar:/apache/releases/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"Etc/GMT+7","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-514.21.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"lajin","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.yarn.ApplicationMaster --class org.apache.spark.examples.SparkPi --jar file:/filer/home/lajin/SPARK-25865_new/spark-3.0.0-SNAPSHOT-bin-SPARK-25865_new/examples/jars/spark-examples_2.12-3.0.0-SNAPSHOT.jar --arg 100 --properties-file /hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__spark_conf__.properties","java.home":"/apache/releases/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-format-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-repl_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-metastore-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-memory-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-all-4.1.30.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/azure-storage-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stream-2.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-jvm-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-ast_2.12-3.5.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-examples-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-core-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-xml_2.12-1.0.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/gson-2.2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-beeline-1.2.1.spark2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-locator-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-guava-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libthrift-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/pyrolite-4.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/activation-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xz-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-core-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib-local_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-launcher_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr4-runtime-4.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-api-1.7.10.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-library-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpcore-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/JavaEWAH-0.3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/eigenbase-properties-1.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/snappy-java-1.0.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-hadoop-bundle-1.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-cli-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-pool-1.5.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/log4j-1.2.17.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/minlog-1.3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/kryo-shaded-4.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-scala_2.12-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xz-1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sql_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/javax.inject-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-log4j12-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill_2.12-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/shapeless_2.12-2.3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-daemon-1.0.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/bonecp-0.8.0.RELEASE.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-auth-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-io-2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jettison-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/opencsv-2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-app-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guava-14.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/netty-all-4.0.23.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/xmlenc-0.52.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-core_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-asn1-api-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-common-2.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-server-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ST4-4.0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/activation-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-collections-3.2.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-configuration-1.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-graphx_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-sharedcachemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/confs/hive/conf":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/java-xmlbuilder-0.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsp-api-2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/osgi-resource-locator-1.0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-core_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-common_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-mapreduce-1.5.3-nohive.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-streaming_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze-macros_2.12-0.13.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-utils-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/paranamer-2.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-applicationhistoryservice-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4-tests.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-resourcemanager-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javolution-5.5.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-cli-1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-parser-combinators_2.12-1.1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/apache/confs/hadoop/conf":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-linq4j-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-network-shuffle_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/compress-lzf-1.0.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/junit-4.11.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-nodemanager-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-shuffle-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.ws.rs-api-2.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hamcrest-core-1.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javassist-3.18.1-GA.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-unsafe_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/httpcore-4.2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/RoaringBitmap-0.5.11.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-databind-2.2.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jetty-6.1.26.hwx.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stringtemplate-3.2.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-logging-1.1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-client-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-annotations-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/derby-10.12.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/objenesis-2.5.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-format-0.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-codec-1.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/aws-java-sdk-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-lang3-3.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-applications-distributedshell-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/gson-2.2.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-client-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.annotation-api-1.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jta-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-core-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apache-log4j-extras-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsr305-3.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/janino-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compress-1.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jsr305-3.0.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-net-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/ivy-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/mockito-all-1.8.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-media-jaxb-2.22.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-registry-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-lang-2.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.inject-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/javax.inject-1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive-thriftserver_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-compress-1.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jtransforms-2.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/chill-java-0.9.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-hs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-common-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-exec-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-client-2.7.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/asm-3.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-web-proxy-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-2.7.7.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zstd-jni-1.3.2-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-json-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-common-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/oro-2.0.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-core-1.2.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/servlet-api-2.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsr305-3.0.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-server-web-proxy-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/univocity-parsers-2.7.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/paranamer-2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-shuffle-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arrow-vector-0.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-io-2.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-math3-3.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-sketch_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-mllib_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jpam-1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-digester-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-hive_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-servlet-3.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jodd-core-3.5.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aircompressor-0.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpclient-4.5.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/snappy-java-1.0.4.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-crypto-1.0.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-collections-3.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-azure-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-client-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-1.8.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jline-2.14.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-jdbc-1.2.1.spark2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/avro-1.7.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-applications-unmanaged-am-launcher-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/avro-ipc-1.8.2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-framework-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-databind-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-scalap_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__/__hadoop_conf__":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/activation-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/py4j-0.10.8.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-timeline-plugins-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xz-1.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/calcite-avatica-1.2.0-incubating.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/macro-compat_2.12-1.1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-hdfs-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/core-1.1.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-beanutils-1.7.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-encoding-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-core-2.9.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/api-asn1-api-1.0.0-M20.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_conf__":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-jackson-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/guava-11.0.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-java-1.1.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xbean-asm7-shaded-4.12.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-api-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hk2-api-2.4.0-b34.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-core-3.2.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/objenesis-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-annotations-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-compiler-3.0.11.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/orc-shims-1.5.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-annotations-2.2.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-1.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/joda-time-2.9.3.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/curator-recipes-2.7.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-yarn_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/hadoop-hdfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jsp-api-2.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-auth-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/spark-2.2.1-yarn-shuffle.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-yarn-client-2.7.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jdo-api-3.0.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jul-to-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-server-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-util-6.1.26.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/super-csv-2.2.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/slf4j-api-1.7.16.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/lz4-java-1.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okio-1.4.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire-macros_2.12-0.13.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jackson-module-paranamer-2.9.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/netty-3.9.9.Final.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/java-xmlbuilder-0.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-beanutils-core-1.8.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-jobclient-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/gson-2.2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/guice-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jsch-0.1.42.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-logging-1.1.3.jar":"System Classpath","/apache/hadoop/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-core-2.2.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-codec-1.4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-lang-2.6.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jackson-jaxrs-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-common-1.10.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/arpack_combined_all-0.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jettison-1.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/httpcore-4.4.10.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/parquet-column-1.10.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/protobuf-java-2.5.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hadoop-mapreduce-client-jobclient-2.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-dbcp-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/stax-api-1.0.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spire_2.12-0.13.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/stax-api-1.0-2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/javax.servlet-api-3.1.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/lib/hadoop-lzo-0.6.0.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jetty-util-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/hadoop-yarn-api-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jsch-0.1.42.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/zookeeper-3.4.6.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/antlr-runtime-3.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-guice-1.9.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-math3-3.1.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/asm-3.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/breeze_2.12-0.13.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-app-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/leveldbjni-all-1.8.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-tags_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-servlet-3.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-core-asl-1.9.13.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-beanutils-core-1.8.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hppc-0.7.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-client-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/validation-api-1.1.0.Final.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jsp-api-2.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-nfs-2.7.1.2.4.2.66-4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jetty-sslengine-6.1.26.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/servlet-api-2.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-json-3.1.5.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jackson-xc-1.9.13.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-compiler-2.12.8.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/guice-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/snappy-0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-logging-1.1.3.jar":"System Classpath","/contrib/capacity-scheduler/*.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/hive-cli-1.2.1.spark2.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/javassist-3.18.1-GA.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-beanutils-1.7.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/jackson-mapper-asl-1.9.13.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/libfb303-0.9.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jaxb-api-2.2.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/datanucleus-rdbms-3.2.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/okhttp-2.4.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/avro-1.7.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/commons-codec-1.4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/zookeeper-3.4.6.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/jetty-6.1.26.hwx.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/slf4j-log4j12-1.7.10.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/fst-2.24.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-tests-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-api-2.2.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/hamcrest-core-1.3.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/jets3t-0.9.0.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-io-2.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/json4s-jackson_2.12-3.5.3.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-catalyst_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/lib/jersey-core-1.9.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/metrics-graphite-3.1.5.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/api-util-1.0.0-M20.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jaxb-impl-2.2.3-1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/mysql-connector-java-5.1.40-bin.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/jersey-core-1.9.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/scala-reflect-2.12.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/curator-framework-2.7.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-httpclient-3.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/htrace-core-3.1.0-incubating.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/xml-apis-1.3.04.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xmlenc-0.52.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/commons-digester-1.8.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jersey-common-2.22.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/xercesImpl-2.9.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-common-2.7.1.2.4.2.66-4-tests.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/guava-11.0.2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/protobuf-java-2.5.0.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/commons-codec-1.4.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/machinist_2.12-0.6.1.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/spark-kvstore_2.12-3.0.0-SNAPSHOT.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-ebay-2.7.1.2.4.2.0-258.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/log4j-1.2.17.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/hadoop-aws-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/hadoop-yarn-server-common-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/netty-3.6.2.Final.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-compress-1.4.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/httpclient-4.2.5.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/guice-servlet-3.0.jar":"System Classpath","/apache/hadoop/share/hadoop/yarn/lib/stax-api-1.0-2.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/common/lib/hadoop-annotations-2.7.1.2.4.2.66-4.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-cli-1.2.jar":"System Classpath","/hadoop/11/yarn/local/usercache/lajin/appcache/application_1536831636016_59384/container_e136_1536831636016_59384_01_000001/__spark_libs__/jcl-over-slf4j-1.7.16.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/hdfs/lib/commons-lang-2.6.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/yarn/lib/commons-net-3.1.jar":"System Classpath","/apache/releases/hadoop-2.7.1.2.4.2.66-4/share/hadoop/mapreduce/hadoop-mapreduce-client-core-2.7.1.2.4.2.66-4.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"Spark Pi","App ID":"application_1536831636016_59384","Timestamp":1546922023607,"User":"lajin","App Attempt ID":"1","Driver Logs":{"stdout":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stdout?start=-4096","stderr":"https://047.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000001/lajin/stderr?start=-4096"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1546922034270,"Executor ID":"2","Executor Info":{"Host":"028.company.com","Total Cores":4,"Log Urls":{"stdout":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stdout?start=-4096","stderr":"https://028.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000003/lajin/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"028.company.com","Port":46325},"Maximum Memory":97832140,"Timestamp":1546922034365,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1546922035929,"Executor ID":"1","Executor Info":{"Host":"036.company.com","Total Cores":4,"Log Urls":{"stdout":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stdout?start=-4096","stderr":"https://036.company.com:50060/node/containerlogs/container_e136_1536831636016_59384_01_000002/lajin/stderr?start=-4096"}}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"036.company.com","Port":35126},"Maximum Memory":97832140,"Timestamp":1546922036120,"Maximum Onheap Memory":97832140,"Maximum Offheap Memory":0} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1546922036359,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"2\",\"name\":\"reduce\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037421,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":86,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":834,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":114738938,"Value":114738938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":134,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":296991078,"Value":296991078,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":725,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":296991078,"Executor Run Time":134,"Executor CPU Time":114738938,"Result Size":834,"JVM GC Time":86,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1546922036517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":86,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":1668,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":116821463,"Value":231560401,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":134,"Value":268,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":250458033,"Value":547449111,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":1450,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":250458033,"Executor Run Time":134,"Executor CPU Time":116821463,"Result Size":834,"JVM GC Time":86,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037432,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":265,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":2502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":139987945,"Value":371548346,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":433,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":443768774,"Value":991217885,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2175,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":443768774,"Executor Run Time":165,"Executor CPU Time":139987945,"Result Size":834,"JVM GC Time":93,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037433,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":93,"Value":358,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":3336,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":143148800,"Value":514697146,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":165,"Value":598,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":257929339,"Value":1249147224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":725,"Value":2900,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":725,"Executor Deserialize CPU Time":257929339,"Executor Run Time":165,"Executor CPU Time":143148800,"Result Size":834,"JVM GC Time":93,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":8,"Attempt":0,"Launch Time":1546922037414,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037500,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":365,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":4170,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":68955072,"Value":583652218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":69,"Value":667,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2463324,"Value":1251610548,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":10,"Value":2910,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":10,"Executor Deserialize CPU Time":2463324,"Executor Run Time":69,"Executor CPU Time":68955072,"Result Size":834,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":9,"Attempt":0,"Launch Time":1546922037416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037501,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":7,"Value":372,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":5004,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":69409062,"Value":653061280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":77,"Value":744,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2197208,"Value":1253807756,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2912,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2197208,"Executor Run Time":77,"Executor CPU Time":69409062,"Result Size":834,"JVM GC Time":7,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":12,"Attempt":0,"Launch Time":1546922037499,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037547,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":5795,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":41068546,"Value":694129826,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":41,"Value":785,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1546605,"Value":1255354361,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":2914,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1546605,"Executor Run Time":41,"Executor CPU Time":41068546,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037584,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":280,"Value":652,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":6629,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":121468700,"Value":815598526,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":136,"Value":921,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":239941709,"Value":1495296070,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":3814,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":239941709,"Executor Run Time":136,"Executor CPU Time":121468700,"Result Size":834,"JVM GC Time":280,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":14,"Attempt":0,"Launch Time":1546922037546,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037588,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":7420,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34831594,"Value":850430120,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":956,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2083011,"Value":1497379081,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":3816,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2083011,"Executor Run Time":35,"Executor CPU Time":34831594,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":16,"Attempt":0,"Launch Time":1546922037587,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037624,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":8211,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31573348,"Value":882003468,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":988,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1326191,"Value":1498705272,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":3817,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1326191,"Executor Run Time":32,"Executor CPU Time":31573348,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1546922036494,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037634,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":938,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9045,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":174111771,"Value":1056115239,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":194,"Value":1182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":414632138,"Value":1913337410,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":4717,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":414632138,"Executor Run Time":194,"Executor CPU Time":174111771,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":15,"Attempt":0,"Launch Time":1546922037583,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037639,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":944,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":9879,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40168837,"Value":1096284076,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":1228,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2240849,"Value":1915578259,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":4720,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2240849,"Executor Run Time":46,"Executor CPU Time":40168837,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1546922036515,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037641,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1230,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":10713,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":189780021,"Value":1286064097,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1431,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":265791948,"Value":2181370207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":5620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":265791948,"Executor Run Time":203,"Executor CPU Time":189780021,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1546922036516,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037642,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":286,"Value":1516,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":11547,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":184630125,"Value":1470694222,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":203,"Value":1634,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":241252253,"Value":2422622460,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":900,"Value":6520,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":900,"Executor Deserialize CPU Time":241252253,"Executor Run Time":203,"Executor CPU Time":184630125,"Result Size":834,"JVM GC Time":286,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":17,"Attempt":0,"Launch Time":1546922037624,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037662,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":12338,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31605508,"Value":1502299730,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":1665,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1157338,"Value":2423779798,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6522,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1157338,"Executor Run Time":31,"Executor CPU Time":31605508,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":10,"Attempt":0,"Launch Time":1546922037431,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037665,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13129,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":227941261,"Value":1730240991,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":228,"Value":1893,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1632598,"Value":2425412396,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6524,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1632598,"Executor Run Time":228,"Executor CPU Time":227941261,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":11,"Attempt":0,"Launch Time":1546922037432,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037672,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":13920,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":234080439,"Value":1964321430,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":234,"Value":2127,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1842114,"Value":2427254510,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6526,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1842114,"Executor Run Time":234,"Executor CPU Time":234080439,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":21,"Attempt":0,"Launch Time":1546922037642,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037685,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":14711,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36307061,"Value":2000628491,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":2164,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2588273,"Value":2429842783,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6528,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2588273,"Executor Run Time":37,"Executor CPU Time":36307061,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":22,"Attempt":0,"Launch Time":1546922037662,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037700,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":15502,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32277689,"Value":2032906180,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":2196,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1148300,"Value":2430991083,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1148300,"Executor Run Time":32,"Executor CPU Time":32277689,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":25,"Attempt":0,"Launch Time":1546922037685,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":16293,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32762038,"Value":2065668218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":2229,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1655404,"Value":2432646487,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1655404,"Executor Run Time":33,"Executor CPU Time":32762038,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":26,"Attempt":0,"Launch Time":1546922037699,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037740,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17084,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30991307,"Value":2096659525,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":2260,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2219464,"Value":2434865951,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6534,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2219464,"Executor Run Time":31,"Executor CPU Time":30991307,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":18,"Attempt":0,"Launch Time":1546922037633,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037745,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":17875,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":104659378,"Value":2201318903,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":104,"Value":2364,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1617759,"Value":2436483710,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6537,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1617759,"Executor Run Time":104,"Executor CPU Time":104659378,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":27,"Attempt":0,"Launch Time":1546922037724,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037759,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":18666,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29044222,"Value":2230363125,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2394,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2017183,"Value":2438500893,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6539,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2017183,"Executor Run Time":30,"Executor CPU Time":29044222,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":28,"Attempt":0,"Launch Time":1546922037740,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037780,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":19457,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34945007,"Value":2265308132,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2429,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998897,"Value":2439499790,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":998897,"Executor Run Time":35,"Executor CPU Time":34945007,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":13,"Attempt":0,"Launch Time":1546922037500,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037781,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":20248,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":275257849,"Value":2540565981,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":275,"Value":2704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1522428,"Value":2441022218,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6542,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1522428,"Executor Run Time":275,"Executor CPU Time":275257849,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":30,"Attempt":0,"Launch Time":1546922037758,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037792,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":20996,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28454753,"Value":2569020734,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":2734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1143042,"Value":2442165260,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1143042,"Executor Run Time":30,"Executor CPU Time":28454753,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":32,"Attempt":0,"Launch Time":1546922037781,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037822,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":21830,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36300767,"Value":2605321501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":2769,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2138947,"Value":2444304207,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6545,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":2138947,"Executor Run Time":35,"Executor CPU Time":36300767,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":33,"Attempt":0,"Launch Time":1546922037791,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037825,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":22621,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28452489,"Value":2633773990,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":2797,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1182965,"Value":2445487172,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6547,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1182965,"Executor Run Time":28,"Executor CPU Time":28452489,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":35,"Index":35,"Attempt":0,"Launch Time":1546922037825,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037858,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":23412,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28114352,"Value":2661888342,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":2826,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1597988,"Value":2447085160,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6548,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1597988,"Executor Run Time":29,"Executor CPU Time":28114352,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":34,"Attempt":0,"Launch Time":1546922037822,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037866,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24203,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":39843327,"Value":2701731669,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":2865,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":979795,"Value":2448064955,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6550,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":979795,"Executor Run Time":39,"Executor CPU Time":39843327,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":36,"Index":36,"Attempt":0,"Launch Time":1546922037857,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037890,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":24994,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27673779,"Value":2729405448,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":27,"Value":2892,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1462274,"Value":2449527229,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6552,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1462274,"Executor Run Time":27,"Executor CPU Time":27673779,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":19,"Attempt":0,"Launch Time":1546922037639,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037894,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":25785,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":250723087,"Value":2980128535,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":250,"Value":3142,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1520638,"Value":2451047867,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6554,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1520638,"Executor Run Time":250,"Executor CPU Time":250723087,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":37,"Index":37,"Attempt":0,"Launch Time":1546922037865,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037917,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1522,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":26619,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":40203057,"Value":3020331592,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":46,"Value":3188,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":987313,"Value":2452035180,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6556,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":987313,"Executor Run Time":46,"Executor CPU Time":40203057,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":23,"Attempt":0,"Launch Time":1546922037665,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037919,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1528,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":27453,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":241618343,"Value":3261949935,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":248,"Value":3436,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1531323,"Value":2453566503,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6558,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1531323,"Executor Run Time":248,"Executor CPU Time":241618343,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":38,"Index":38,"Attempt":0,"Launch Time":1546922037890,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037923,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":28201,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28219664,"Value":3290169599,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":3465,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1047569,"Value":2454614072,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1047569,"Executor Run Time":29,"Executor CPU Time":28219664,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":24,"Attempt":0,"Launch Time":1546922037671,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037952,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1534,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":29035,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":239368713,"Value":3529538312,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":246,"Value":3711,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1606167,"Value":2456220239,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":6561,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":1606167,"Executor Run Time":246,"Executor CPU Time":239368713,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":20,"Attempt":0,"Launch Time":1546922037641,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":29826,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":281427189,"Value":3810965501,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":282,"Value":3993,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1654852,"Value":2457875091,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6563,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1654852,"Executor Run Time":282,"Executor CPU Time":281427189,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":29,"Attempt":0,"Launch Time":1546922037744,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037953,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":30617,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":188231506,"Value":3999197007,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":188,"Value":4181,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1169748,"Value":2459044839,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6565,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1169748,"Executor Run Time":188,"Executor CPU Time":188231506,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":39,"Index":39,"Attempt":0,"Launch Time":1546922037894,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037954,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":31408,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":42202483,"Value":4041399490,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":4223,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1131763,"Value":2460176602,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6567,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1131763,"Executor Run Time":42,"Executor CPU Time":42202483,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":41,"Index":41,"Attempt":0,"Launch Time":1546922037919,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037955,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":32242,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32046930,"Value":4073446420,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4254,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1193077,"Value":2461369679,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6569,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1193077,"Executor Run Time":31,"Executor CPU Time":32046930,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":47,"Index":47,"Attempt":0,"Launch Time":1546922037955,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037990,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":33033,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31475569,"Value":4104921989,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060814,"Value":2462430493,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060814,"Executor Run Time":31,"Executor CPU Time":31475569,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":46,"Index":46,"Attempt":0,"Launch Time":1546922037954,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922037992,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":33781,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33893704,"Value":4138815693,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4320,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":995078,"Value":2463425571,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":995078,"Executor Run Time":35,"Executor CPU Time":33893704,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":49,"Index":49,"Attempt":0,"Launch Time":1546922037991,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038024,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":34529,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":27659093,"Value":4166474786,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":29,"Value":4349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":936644,"Value":2464362215,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":936644,"Executor Run Time":29,"Executor CPU Time":27659093,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":48,"Index":48,"Attempt":0,"Launch Time":1546922037990,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038025,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":35320,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30657164,"Value":4197131950,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":4379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1037555,"Value":2465399770,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6572,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1037555,"Executor Run Time":30,"Executor CPU Time":30657164,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":51,"Index":51,"Attempt":0,"Launch Time":1546922038025,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038061,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":36111,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31510921,"Value":4228642871,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4411,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895198,"Value":2466294968,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6573,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895198,"Executor Run Time":32,"Executor CPU Time":31510921,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":50,"Index":50,"Attempt":0,"Launch Time":1546922038023,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038064,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1540,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":877,"Value":36988,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30458180,"Value":4259101051,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":35,"Value":4446,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":955927,"Value":2467250895,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6575,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":955927,"Executor Run Time":35,"Executor CPU Time":30458180,"Result Size":877,"JVM GC Time":6,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":52,"Index":52,"Attempt":0,"Launch Time":1546922038060,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038095,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":37779,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31390887,"Value":4290491938,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":4477,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":893866,"Value":2468144761,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":893866,"Executor Run Time":31,"Executor CPU Time":31390887,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":53,"Index":53,"Attempt":0,"Launch Time":1546922038063,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038097,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":38570,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":28537105,"Value":4319029043,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":28,"Value":4505,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1137737,"Value":2469282498,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6578,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1137737,"Executor Run Time":28,"Executor CPU Time":28537105,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":31,"Attempt":0,"Launch Time":1546922037779,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038113,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1546,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":39404,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":320279036,"Value":4639308079,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":327,"Value":4832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":2309726,"Value":2471592224,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6580,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":2309726,"Executor Run Time":327,"Executor CPU Time":320279036,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":54,"Index":54,"Attempt":0,"Launch Time":1546922038095,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038131,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":40195,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":32175125,"Value":4671483204,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":4864,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":861437,"Value":2472453661,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6581,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":861437,"Executor Run Time":32,"Executor CPU Time":32175125,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":44,"Index":44,"Attempt":0,"Launch Time":1546922037952,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038137,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1552,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":41029,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":175258771,"Value":4846741975,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":180,"Value":5044,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1272588,"Value":2473726249,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6583,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1272588,"Executor Run Time":180,"Executor CPU Time":175258771,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":55,"Index":55,"Attempt":0,"Launch Time":1546922038097,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038148,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":41820,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":46067636,"Value":4892809611,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":47,"Value":5091,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1539643,"Value":2475265892,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6584,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1539643,"Executor Run Time":47,"Executor CPU Time":46067636,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":57,"Index":57,"Attempt":0,"Launch Time":1546922038131,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038169,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":42611,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33746415,"Value":4926556026,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":5125,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":939846,"Value":2476205738,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6585,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":939846,"Executor Run Time":34,"Executor CPU Time":33746415,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":40,"Index":40,"Attempt":0,"Launch Time":1546922037917,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038175,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":43402,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":253536114,"Value":5180092140,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":254,"Value":5379,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1121119,"Value":2477326857,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6586,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1121119,"Executor Run Time":254,"Executor CPU Time":253536114,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":59,"Index":59,"Attempt":0,"Launch Time":1546922038148,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038185,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":44193,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33229424,"Value":5213321564,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":5412,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":856494,"Value":2478183351,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":856494,"Executor Run Time":33,"Executor CPU Time":33229424,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":60,"Index":60,"Attempt":0,"Launch Time":1546922038168,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038204,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":44941,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31762584,"Value":5245084148,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":32,"Value":5444,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":819419,"Value":2479002770,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":819419,"Executor Run Time":32,"Executor CPU Time":31762584,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":42,"Index":42,"Attempt":0,"Launch Time":1546922037922,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038217,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1558,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":45775,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254396405,"Value":5499480553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":260,"Value":5704,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1523395,"Value":2480526165,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6589,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1523395,"Executor Run Time":260,"Executor CPU Time":254396405,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":62,"Index":62,"Attempt":0,"Launch Time":1546922038185,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038218,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":46523,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":29041950,"Value":5528522503,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":5734,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":920529,"Value":2481446694,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":920529,"Executor Run Time":30,"Executor CPU Time":29041950,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":45,"Index":45,"Attempt":0,"Launch Time":1546922037953,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038221,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1564,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":47357,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":257839847,"Value":5786362350,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":264,"Value":5998,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1031380,"Value":2482478074,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6590,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1031380,"Executor Run Time":264,"Executor CPU Time":257839847,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":63,"Index":63,"Attempt":0,"Launch Time":1546922038204,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038239,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":48191,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31072580,"Value":5817434930,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6028,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":960275,"Value":2483438349,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6592,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":960275,"Executor Run Time":30,"Executor CPU Time":31072580,"Result Size":834,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":64,"Index":64,"Attempt":0,"Launch Time":1546922038217,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038252,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":5,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":48982,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30559817,"Value":5847994747,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":31,"Value":6059,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":860099,"Value":2484298448,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":860099,"Executor Run Time":31,"Executor CPU Time":30559817,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":43,"Index":43,"Attempt":0,"Launch Time":1546922037949,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038253,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":49773,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":296727303,"Value":6144722050,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":297,"Value":6356,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1526695,"Value":2485825143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1526695,"Executor Run Time":297,"Executor CPU Time":296727303,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":67,"Index":67,"Attempt":0,"Launch Time":1546922038239,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038276,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":50521,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33148590,"Value":6177870640,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6390,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":841498,"Value":2486666641,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":841498,"Executor Run Time":34,"Executor CPU Time":33148590,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":68,"Index":68,"Attempt":0,"Launch Time":1546922038251,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038292,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":51269,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35277840,"Value":6213148480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6426,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":981161,"Value":2487647802,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":981161,"Executor Run Time":36,"Executor CPU Time":35277840,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":58,"Index":58,"Attempt":0,"Launch Time":1546922038137,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038295,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":52060,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":152692713,"Value":6365841193,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":153,"Value":6579,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1007896,"Value":2488655698,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6594,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1007896,"Executor Run Time":153,"Executor CPU Time":152692713,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":70,"Index":70,"Attempt":0,"Launch Time":1546922038275,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038313,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":52808,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33306826,"Value":6399148019,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6613,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":956832,"Value":2489612530,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":956832,"Executor Run Time":34,"Executor CPU Time":33306826,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":71,"Index":71,"Attempt":0,"Launch Time":1546922038291,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038330,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":53599,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":33756595,"Value":6432904614,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":33,"Value":6646,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":882246,"Value":2490494776,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6595,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":882246,"Executor Run Time":33,"Executor CPU Time":33756595,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":73,"Index":73,"Attempt":0,"Launch Time":1546922038313,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038348,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":54390,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":30684763,"Value":6463589377,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":30,"Value":6676,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":943134,"Value":2491437910,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6597,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":943134,"Executor Run Time":30,"Executor CPU Time":30684763,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":74,"Index":74,"Attempt":0,"Launch Time":1546922038329,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038368,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":55181,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34465927,"Value":6498055304,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":6710,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":964913,"Value":2492402823,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6598,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":964913,"Executor Run Time":34,"Executor CPU Time":34465927,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":75,"Index":75,"Attempt":0,"Launch Time":1546922038347,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038391,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1570,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":56015,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":31793636,"Value":6529848940,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":6747,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":975320,"Value":2493378143,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6600,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":975320,"Executor Run Time":37,"Executor CPU Time":31793636,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":76,"Index":76,"Attempt":0,"Launch Time":1546922038368,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038409,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":56806,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36274008,"Value":6566122948,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":6783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":972306,"Value":2494350449,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6602,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":972306,"Executor Run Time":36,"Executor CPU Time":36274008,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":61,"Index":61,"Attempt":0,"Launch Time":1546922038175,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038416,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1576,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":57640,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":230475162,"Value":6796598110,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":236,"Value":7019,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989658,"Value":2495340107,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6603,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989658,"Executor Run Time":236,"Executor CPU Time":230475162,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":56,"Index":56,"Attempt":0,"Launch Time":1546922038113,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038424,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1582,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":58474,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":301424170,"Value":7098022280,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":308,"Value":7327,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012899,"Value":2496353006,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6604,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1012899,"Executor Run Time":308,"Executor CPU Time":301424170,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":69,"Index":69,"Attempt":0,"Launch Time":1546922038253,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038436,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1588,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":59308,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":172594019,"Value":7270616299,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":179,"Value":7506,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1197294,"Value":2497550300,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6605,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1197294,"Executor Run Time":179,"Executor CPU Time":172594019,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":78,"Index":78,"Attempt":0,"Launch Time":1546922038409,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038448,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60099,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34408818,"Value":7305025117,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":34,"Value":7540,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":989551,"Value":2498539851,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6606,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":989551,"Executor Run Time":34,"Executor CPU Time":34408818,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":65,"Index":65,"Attempt":0,"Launch Time":1546922038218,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038483,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1594,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":60890,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":254383686,"Value":7559408803,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":261,"Value":7801,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1135212,"Value":2499675063,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1135212,"Executor Run Time":261,"Executor CPU Time":254383686,"Result Size":791,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":82,"Index":82,"Attempt":0,"Launch Time":1546922038448,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038493,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1600,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":61681,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":34883743,"Value":7594292546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":42,"Value":7843,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1012178,"Value":2500687241,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1012178,"Executor Run Time":42,"Executor CPU Time":34883743,"Result Size":791,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":66,"Index":66,"Attempt":0,"Launch Time":1546922038220,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038495,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1606,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":62515,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":264938303,"Value":7859230849,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":270,"Value":8113,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":934827,"Value":2501622068,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6608,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":934827,"Executor Run Time":270,"Executor CPU Time":264938303,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":77,"Index":77,"Attempt":0,"Launch Time":1546922038390,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038507,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":63263,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":112064631,"Value":7971295480,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":8227,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1053290,"Value":2502675358,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1053290,"Executor Run Time":114,"Executor CPU Time":112064631,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":80,"Index":80,"Attempt":0,"Launch Time":1546922038424,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038518,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":64054,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89355066,"Value":8060650546,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":89,"Value":8316,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":998260,"Value":2503673618,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6610,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":998260,"Executor Run Time":89,"Executor CPU Time":89355066,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":84,"Index":84,"Attempt":0,"Launch Time":1546922038492,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038533,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":64802,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":35725959,"Value":8096376505,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":37,"Value":8353,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1128802,"Value":2504802420,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1128802,"Executor Run Time":37,"Executor CPU Time":35725959,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":81,"Index":81,"Attempt":0,"Launch Time":1546922038436,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038537,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":65593,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96824193,"Value":8193200698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":96,"Value":8449,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":857593,"Value":2505660013,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6612,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":857593,"Executor Run Time":96,"Executor CPU Time":96824193,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":72,"Index":72,"Attempt":0,"Launch Time":1546922038294,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038550,"Failed":false,"Killed":false,"Accumulables":[{"ID":5,"Name":"internal.metrics.jvmGCTime","Update":6,"Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":834,"Value":66427,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":245593808,"Value":8438794506,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":251,"Value":8700,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1199629,"Value":2506859642,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6614,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1199629,"Executor Run Time":251,"Executor CPU Time":245593808,"Result Size":834,"JVM GC Time":6,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":79,"Index":79,"Attempt":0,"Launch Time":1546922038416,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038552,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":67175,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":130530302,"Value":8569324808,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":132,"Value":8832,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1409511,"Value":2508269153,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1409511,"Executor Run Time":132,"Executor CPU Time":130530302,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":88,"Index":88,"Attempt":0,"Launch Time":1546922038533,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038574,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":67966,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":36971815,"Value":8606296623,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":36,"Value":8868,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1067084,"Value":2509336237,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6616,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1067084,"Executor Run Time":36,"Executor CPU Time":36971815,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":86,"Index":86,"Attempt":0,"Launch Time":1546922038507,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038612,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":68757,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100349075,"Value":8706645698,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":8968,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1073198,"Value":2510409435,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6617,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1073198,"Executor Run Time":100,"Executor CPU Time":100349075,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":85,"Index":85,"Attempt":0,"Launch Time":1546922038495,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038614,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":69548,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":113541356,"Value":8820187054,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":114,"Value":9082,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":904680,"Value":2511314115,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":904680,"Executor Run Time":114,"Executor CPU Time":113541356,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":87,"Index":87,"Attempt":0,"Launch Time":1546922038517,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038621,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":70339,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":100446517,"Value":8920633571,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":100,"Value":9182,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":895108,"Value":2512209223,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6618,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":895108,"Executor Run Time":100,"Executor CPU Time":100446517,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":89,"Index":89,"Attempt":0,"Launch Time":1546922038536,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038637,"Failed":false,"Killed":false,"Accumulables":[{"ID":6,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":7,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71130,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":96681743,"Value":9017315314,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":97,"Value":9279,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":881317,"Value":2513090540,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":881317,"Executor Run Time":97,"Executor CPU Time":96681743,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":92,"Index":92,"Attempt":0,"Launch Time":1546922038574,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038641,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":71921,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":61426798,"Value":9078742112,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9340,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1009168,"Value":2514099708,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6620,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1009168,"Executor Run Time":61,"Executor CPU Time":61426798,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":91,"Index":91,"Attempt":0,"Launch Time":1546922038551,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038664,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":72669,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":108784588,"Value":9187526700,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":110,"Value":9450,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1001879,"Value":2515101587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":1001879,"Executor Run Time":110,"Executor CPU Time":108784588,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":94,"Index":94,"Attempt":0,"Launch Time":1546922038614,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038681,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":73460,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":60115853,"Value":9247642553,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":61,"Value":9511,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1466213,"Value":2516567800,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6621,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":1466213,"Executor Run Time":61,"Executor CPU Time":60115853,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":83,"Index":83,"Attempt":0,"Launch Time":1546922038483,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":74251,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":214285427,"Value":9461927980,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":214,"Value":9725,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1060594,"Value":2517628394,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6623,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1060594,"Executor Run Time":214,"Executor CPU Time":214285427,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":97,"Index":97,"Attempt":0,"Launch Time":1546922038640,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038705,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75042,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":57994238,"Value":9519922218,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":58,"Value":9783,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1405974,"Value":2519034368,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6625,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1405974,"Executor Run Time":58,"Executor CPU Time":57994238,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":93,"Index":93,"Attempt":0,"Launch Time":1546922038612,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038717,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":75833,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101684514,"Value":9621606732,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":101,"Value":9884,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":968858,"Value":2520003226,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6627,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":968858,"Executor Run Time":101,"Executor CPU Time":101684514,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":99,"Index":99,"Attempt":0,"Launch Time":1546922038680,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038724,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":76624,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":38985492,"Value":9660592224,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":39,"Value":9923,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1411378,"Value":2521414604,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6629,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1411378,"Executor Run Time":39,"Executor CPU Time":38985492,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":90,"Index":90,"Attempt":0,"Launch Time":1546922038550,"Executor ID":"1","Host":"036.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038725,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":77415,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":169783721,"Value":9830375945,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":170,"Value":10093,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805689,"Value":2522220293,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6630,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805689,"Executor Run Time":170,"Executor CPU Time":169783721,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":95,"Index":95,"Attempt":0,"Launch Time":1546922038621,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038726,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":78206,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":101866107,"Value":9932242052,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":102,"Value":10195,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":805277,"Value":2523025570,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":1,"Value":6631,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1,"Executor Deserialize CPU Time":805277,"Executor Run Time":102,"Executor CPU Time":101866107,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":96,"Index":96,"Attempt":0,"Launch Time":1546922038636,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038729,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":748,"Value":78954,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":89123011,"Value":10021365063,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":90,"Value":10285,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":896962,"Value":2523922532,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":0,"Executor Deserialize CPU Time":896962,"Executor Run Time":90,"Executor CPU Time":89123011,"Result Size":748,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":98,"Index":98,"Attempt":0,"Launch Time":1546922038664,"Executor ID":"2","Host":"028.company.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1546922038733,"Failed":false,"Killed":false,"Accumulables":[{"ID":4,"Name":"internal.metrics.resultSize","Update":791,"Value":79745,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Update":64532656,"Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorRunTime","Update":64,"Value":10349,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Update":1115835,"Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Update":2,"Value":6633,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2,"Executor Deserialize CPU Time":1115835,"Executor Run Time":64,"Executor CPU Time":64532656,"Result Size":791,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":211171816,"JVMOffHeapMemory":90237256,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":4876,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":4876,"OffHeapUnifiedMemory":0,"DirectPoolMemory":806275,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2646888448,"ProcessTreeJVMRSSMemory":520900608,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0,"MinorGCCount":8,"MinorGCTime":374,"MajorGCCount":3,"MajorGCTime":170}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"reduce at SparkPi.scala:38","Number of Tasks":100,"RDD Info":[{"RDD ID":1,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"map\"}","Callsite":"map at SparkPi.scala:34","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"ParallelCollectionRDD","Scope":"{\"id\":\"0\",\"name\":\"parallelize\"}","Callsite":"parallelize at SparkPi.scala:34","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":100,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.reduce(RDD.scala:1031)\norg.apache.spark.examples.SparkPi$.main(SparkPi.scala:38)\norg.apache.spark.examples.SparkPi.main(SparkPi.scala)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\norg.apache.spark.deploy.yarn.ApplicationMaster$$anon$2.run(ApplicationMaster.scala:677)","Submission Time":1546922036383,"Completion Time":1546922038734,"Accumulables":[{"ID":2,"Name":"internal.metrics.executorRunTime","Value":10349,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.jvmGCTime","Value":1612,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.resultSize","Value":79745,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeCpuTime","Value":2525038367,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorCpuTime","Value":10085897719,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.resultSerializationTime","Value":7,"Internal":true,"Count Failed Values":true},{"ID":0,"Name":"internal.metrics.executorDeserializeTime","Value":6633,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1546922038738,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1546922038745} diff --git a/core/src/test/resources/spark-events/application_1538416563558_0014 b/core/src/test/resources/spark-events/application_1538416563558_0014 new file mode 100644 index 0000000000000..000288dbc4541 --- /dev/null +++ b/core/src/test/resources/spark-events/application_1538416563558_0014 @@ -0,0 +1,190 @@ +{"Event":"SparkListenerLogStart","Spark Version":"2.5.0-SNAPSHOT"} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"rezamemory-1.gce.something.com","Port":43959},"Maximum Memory":384093388,"Timestamp":1538440967690,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/usr/java/jdk1.8.0_121/jre","Java Version":"1.8.0_121 (Oracle Corporation)","Scala Version":"version 2.11.12"},"Spark Properties":{"spark.serializer":"org.apache.spark.serializer.KryoSerializer","spark.yarn.jars":"local:/opt/some/path/lib/spark2/jars/*","spark.driver.host":"rezamemory-1.gce.something.com","spark.serializer.objectStreamReset":"100","spark.eventLog.enabled":"true","spark.executor.heartbeatInterval":"100ms","spark.hadoop.mapreduce.application.classpath":"","spark.driver.port":"35918","spark.shuffle.service.enabled":"true","spark.rdd.compress":"True","spark.driver.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.executorEnv.PYTHONPATH":"/opt/some/path/lib/spark2/python/lib/py4j-0.10.7-src.zip/opt/some/path/lib/spark2/python/lib/pyspark.zip","spark.yarn.historyServer.address":"http://rezamemory-1.gce.something.com:18089","spark.app.name":"PythonBisectingKMeansExample","spark.ui.killEnabled":"true","spark.sql.hive.metastore.jars":"${env:HADOOP_COMMON_HOME}/../hive/lib/*:${env:HADOOP_COMMON_HOME}/client/*","spark.dynamicAllocation.schedulerBacklogTimeout":"1","spark.yarn.am.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.scheduler.mode":"FIFO","spark.eventLog.logStageExecutorMetrics.enabled":"true","spark.yarn.config.gatewayPath":"/opt/cloudera/parcels","spark.executor.id":"driver","spark.yarn.config.replacementPath":"{{HADOOP_COMMON_HOME}}/../../..","spark.eventLog.logStageExecutorProcessTreeMetrics.enabled":"true","spark.submit.deployMode":"client","spark.shuffle.service.port":"7337","spark.master":"yarn","spark.authenticate":"false","spark.ui.filters":"org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter","spark.executor.extraLibraryPath":"/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native","spark.eventLog.dir":"hdfs://rezamemory-1.gce.something.com:8020/user/spark/spark2ApplicationHistory","spark.dynamicAllocation.enabled":"true","spark.sql.catalogImplementation":"hive","spark.hadoop.yarn.application.classpath":"","spark.driver.appUIAddress":"http://rezamemory-1.gce.something.com:4040","spark.yarn.isPython":"true","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_HOSTS":"rezamemory-1.gce.something.com","spark.dynamicAllocation.minExecutors":"0","spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.param.PROXY_URI_BASES":"http://rezamemory-1.gce.something.com:8088/proxy/application_1538416563558_0014","spark.dynamicAllocation.executorIdleTimeout":"60","spark.app.id":"application_1538416563558_0014","spark.sql.hive.metastore.version":"1.1.0"},"System Properties":{"java.io.tmpdir":"/tmp","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/root","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","sun.arch.data.model":"64","sun.boot.library.path":"/usr/java/jdk1.8.0_121/jre/lib/amd64","user.dir":"/","java.library.path":":/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/lib/native:/usr/java/packages/lib/amd64:/usr/lib64:/lib64:/lib:/usr/lib","sun.cpu.isalist":"","os.arch":"amd64","java.vm.version":"25.121-b13","jetty.git.hash":"unknown","java.endorsed.dirs":"/usr/java/jdk1.8.0_121/jre/lib/endorsed","java.runtime.version":"1.8.0_121-b13","java.vm.info":"mixed mode","java.ext.dirs":"/usr/java/jdk1.8.0_121/jre/lib/ext:/usr/java/packages/lib/ext","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/usr/java/jdk1.8.0_121/jre/lib/resources.jar:/usr/java/jdk1.8.0_121/jre/lib/rt.jar:/usr/java/jdk1.8.0_121/jre/lib/sunrsasign.jar:/usr/java/jdk1.8.0_121/jre/lib/jsse.jar:/usr/java/jdk1.8.0_121/jre/lib/jce.jar:/usr/java/jdk1.8.0_121/jre/lib/charsets.jar:/usr/java/jdk1.8.0_121/jre/lib/jfr.jar:/usr/java/jdk1.8.0_121/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Los_Angeles","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"3.10.0-693.5.2.el7.x86_64","sun.os.patch.level":"unknown","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","user.language":"en","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.print.PSPrinterJob","java.awt.graphicsenv":"sun.awt.X11GraphicsEnvironment","awt.toolkit":"sun.awt.X11.XToolkit","os.name":"Linux","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"root","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --conf spark.executor.heartbeatInterval=100ms --conf spark.eventLog.logStageExecutorProcessTreeMetrics.enabled=true --conf spark.eventLog.logStageExecutorMetrics.enabled=true ./opt/some/path/lib/spark2/examples/src/main/python/mllib/bisecting_k_means_example.py","java.home":"/usr/java/jdk1.8.0_121/jre","java.version":"1.8.0_121","sun.io.unicode.encoding":"UnicodeLittle"},"Classpath Entries":{"/opt/some/path/lib/spark2/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/netty-3.10.5.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/validation-api-1.1.0.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-annotations-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-datalake-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-jaxrs-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-impl-2.2.3-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-compiler-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/logredactor-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-streaming_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-common-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-core-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/RoaringBitmap-0.5.11.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-hadoop-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-jackson-1.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-server-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-api-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jtransforms-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aircompressor-0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-el-1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-mapred-1.8.2-hadoop2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/minlog-1.3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-daemon-1.0.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/kryo-shaded-4.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-crypto-1.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib-local_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-openstack-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang3-3.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/univocity-parsers-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javassist-3.18.1-GA.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-api-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-registry-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/activation-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/objenesis-2.5.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-xc-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-unmanaged-am-launcher-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-repl_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hue-plugins-3.9.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-ast_2.11-3.5.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-resourcemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-math3-3.1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/activation-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-tests-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/snappy-java-1.0.4.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/microsoft-windowsazure-storage-sdk-0.6.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-log4j12-1.7.16.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/metrics-core-2.2.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-json-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-kvstore_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-common-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-guice-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archives-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-archive-logs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-examples-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/snappy-java-1.1.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-web-proxy-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-mapreduce-1.5.2-nohive.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsch-0.1.42.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-jvm-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.annotation-api-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/pyrolite-4.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-jackson-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-jackson_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-client-2.22.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jline-2.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scala_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/spark-streaming-kafka-0-8_2.11-2.2.0.cloudera1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-nodemanager-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-logging-1.1.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-asl-1.9.13.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compiler-3.0.10.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-generator-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-format-2.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-mapper-asl-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-core-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-server-1.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-6.1.26.cloudera.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/mockito-all-1.8.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/aopalliance-repackaged-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/osgi-resource-locator-1.0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsp-api-2.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-unsafe_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/oro-2.0.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-databind-2.6.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-codec-1.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/jars/opencsv-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xbean-asm6-shaded-4.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/javax.inject-1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-encoding-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-library-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-scalap_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/log4j-1.2.17.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jaxb-api-2.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/LICENSE.txt":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-3.9.9.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/json4s-core_2.11-3.5.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-api-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpcore-4.2.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jettison-1.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zookeeper-3.4.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-core-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-auth-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-core-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-shuffle_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-utils-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-beanutils-1.9.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill_2.11-0.9.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-core-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jul-to-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/paranamer-2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-container-servlet-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/janino-3.0.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jetty-util-6.1.26.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-core-1.8.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/py4j-0.10.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/ivy-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/apacheds-kerberos-codec-2.0.0-M15.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-lang-2.6.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-client-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stream-2.7.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xml-apis-1.3.04.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka_2.11-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/metrics-core-3.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-applicationhistoryservice-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-servlet-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-api-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xz-1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-tools-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-format-0.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okio-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/compress-lzf-1.0.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-jobclient-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hppc-0.7.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-yarn_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-applications-distributedshell-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/joda-time-2.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-sls-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jets3t-0.9.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/leveldbjni-all-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guice-3.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-streaming-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/guava-14.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hamcrest-core-1.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aws-java-sdk-bundle-1.11.134.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-client-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-hs-plugins-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-gridmix-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-util-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xz-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-guava-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-compiler-2.11.12.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sql_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-app-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/java-xmlbuilder-0.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/slf4j-api-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-hadoop-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-shuffle-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-pig-bundle-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-digester-1.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-json-3.1.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-codec-1.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-beanutils-1.7.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-catalyst_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-xml_2.11-1.0.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-common-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-parser-combinators_2.11-1.1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jetty-util-6.1.26.cloudera.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpclient-4.5.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/antlr4-runtime-4.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-lang-2.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-mllib_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jersey-media-jaxb-2.22.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-app-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/machinist_2.11-0.6.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-core_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire_2.11-0.13.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-xc-1.9.13.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-thrift-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/htrace-core-3.1.0-incubating.jar":"System Classpath","/opt/some/path/lib/spark2/jars/macro-compat_2.11-1.1.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-annotations-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-io-2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-annotations-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-core-1.5.2-nohive.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-net-3.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-memory-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/shapeless_2.11-2.3.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-graphx_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-core-asl-1.8.8.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/okhttp-2.4.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-sources.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/htrace-core4-4.0.1-incubating.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-datajoin-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-paranamer-2.7.9.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-aws-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/scala-reflect-2.11.12.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-net-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-databind-2.2.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/parquet-column-1.10.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xmlenc-0.52.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/kafka-clients-0.9.0-kafka-2.0.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-io-2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/lz4-java-1.4.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/core-1.1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arrow-vector-0.10.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.ws.rs-api-2.0.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-azure-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-format-2.1.0-cdh5.12.0-javadoc.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-client-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-yarn-server-common-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/httpcore-4.4.10.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.servlet-api-3.1.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-2.4.0-b34.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-core-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/servlet-api-2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-common-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-math3-3.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/javax.inject-1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-jaxrs-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-recipes-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/spark-1.6.0-cdh5.12.0-yarn-shuffle.jar":"System Classpath","/opt/some/path/lib/spark2/jars/breeze-macros_2.11-0.13.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/zookeeper-3.4.5-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/httpclient-4.2.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/metrics-graphite-3.1.5.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jcl-over-slf4j-1.7.16.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-compress-1.4.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-sketch_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-network-common_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/gson-2.2.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-cascading-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-auth-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/orc-shims-1.5.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/log4j-1.2.17.jar":"System Classpath","/opt/some/path/lib/spark2/jars/stax-api-1.0-2.jar":"System Classpath","/opt/some/path/lib/spark2/kafka-0.9/zkclient-0.7.jar":"System Classpath","/opt/some/path/lib/spark2/jars/paranamer-2.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/apacheds-i18n-2.0.0-M15.jar":"System Classpath","/opt/some/path/lib/spark2/jars/gson-2.2.4.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-tags_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-configuration-1.6.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-hdfs-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guice-3.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jsr305-1.3.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/conf/yarn-conf/":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/api-asn1-api-1.0.0-M20.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/azure-data-lake-store-sdk-2.1.4.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-distcp-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/junit-4.11.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-extras-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hk2-locator-2.4.0-b34.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jasper-runtime-5.5.23.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-client-2.7.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-compress-1.8.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jsr305-3.0.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-collections-3.2.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/guava-11.0.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/asm-3.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/avro-1.7.6-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-httpclient-3.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jersey-client-1.9.jar":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-yarn-server-web-proxy-2.7.3.jar":"System Classpath","/opt/some/path/lib/spark2/jars/zstd-jni-1.3.2-2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/commons-cli-1.2.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-scrooge_2.10-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/commons-cli-1.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spire-macros_2.11-0.13.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-ant-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/spark-launcher_2.11-2.5.0-SNAPSHOT.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-nativetask-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/xercesImpl-2.9.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-module-scala_2.11-2.6.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-hdfs-nfs-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-encoding-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-shuffle-2.6.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-avro-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/flatbuffers-1.2.0-3f79e055.jar":"System Classpath","/opt/some/path/lib/spark2/jars/protobuf-java-2.5.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-test-hadoop2-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-mapreduce-client-jobclient-2.6.0-cdh5.12.0-tests.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/aopalliance-1.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-column-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/slf4j-log4j12-1.7.5.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/parquet-protobuf-1.5.0-cdh5.12.0.jar":"System Classpath","/opt/some/path/lib/spark2/jars/avro-ipc-1.8.2.jar":"System Classpath","/opt/some/path/lib/spark2/jars/arpack_combined_all-0.1.jar":"System Classpath","/opt/some/path/lib/spark2/jars/netty-all-4.1.17.Final.jar":"System Classpath","/opt/some/path/lib/spark2/jars/chill-java-0.9.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/lib/hadoop/NOTICE.txt":"System Classpath","/opt/some/path/lib/spark2/jars/hadoop-mapreduce-client-common-2.7.3.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/jackson-mapper-asl-1.8.8.jar":"System Classpath","/opt/some/path/lib/spark2/jars/jackson-annotations-2.6.7.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/curator-framework-2.7.1.jar":"System Classpath","/opt/cloudera/parcels/CDH-5.12.0-1.cdh5.12.0.p0.29/jars/hadoop-rumen-2.6.0-cdh5.12.0.jar":"System Classpath"}} +{"Event":"SparkListenerApplicationStart","App Name":"PythonBisectingKMeansExample","App ID":"application_1538416563558_0014","Timestamp":1538440959580,"User":"root"} +{"Event":"SparkListenerJobStart","Job ID":0,"Submission Time":1538440969009,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[0],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"3\",\"name\":\"first\"}"}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440973727,"Executor ID":"1","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000002/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Maximum Memory":384093388,"Timestamp":1538440973890,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1538440973735,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440977628,"Failed":false,"Killed":false,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Update":208,"Value":208,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Update":1448,"Value":1448,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Update":1105071149,"Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Update":2307,"Value":2307,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Update":651096062,"Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Update":1322,"Value":1322,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1322,"Executor Deserialize CPU Time":651096062,"Executor Run Time":2307,"Executor CPU Time":1105071149,"Result Size":1448,"JVM GC Time":208,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":256071440,"JVMOffHeapMemory":92211424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":333371,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":333371,"OffHeapUnifiedMemory":0,"DirectPoolMemory":134726,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4926242816,"ProcessTreeJVMRSSMemory":525656064,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":0,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":182536928,"JVMOffHeapMemory":58263224,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1086483,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1086483,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20304,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3009855488,"ProcessTreeJVMRSSMemory":404488192,"ProcessTreePythonVMemory":626200576,"ProcessTreePythonRSSMemory":69218304,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"first at BisectingKMeans.scala:163","Number of Tasks":1,"RDD Info":[{"RDD ID":4,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"2\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:163","Parent IDs":[3],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1377)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:163)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440969044,"Completion Time":1538440977644,"Accumulables":[{"ID":23,"Name":"internal.metrics.input.recordsRead","Value":4,"Internal":true,"Count Failed Values":true},{"ID":2,"Name":"internal.metrics.executorDeserializeCpuTime","Value":651096062,"Internal":true,"Count Failed Values":true},{"ID":5,"Name":"internal.metrics.resultSize","Value":1448,"Internal":true,"Count Failed Values":true},{"ID":22,"Name":"internal.metrics.input.bytesRead","Value":72,"Internal":true,"Count Failed Values":true},{"ID":4,"Name":"internal.metrics.executorCpuTime","Value":1105071149,"Internal":true,"Count Failed Values":true},{"ID":7,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":1,"Name":"internal.metrics.executorDeserializeTime","Value":1322,"Internal":true,"Count Failed Values":true},{"ID":3,"Name":"internal.metrics.executorRunTime","Value":2307,"Internal":true,"Count Failed Values":true},{"ID":6,"Name":"internal.metrics.jvmGCTime","Value":208,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":0,"Completion Time":1538440977650,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":1,"Submission Time":1538440977784,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[1,2],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":0,"Attempt":0,"Launch Time":1538440977816,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978683,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":13535058,"Value":13535058,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":202227536,"Value":202227536,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":705,"Value":705,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":65694833,"Value":65694833,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":119,"Value":119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":119,"Executor Deserialize CPU Time":65694833,"Executor Run Time":705,"Executor CPU Time":202227536,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":13535058,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":1,"Attempt":0,"Launch Time":1538440978659,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440978820,"Failed":false,"Killed":false,"Accumulables":[{"ID":48,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Update":289555,"Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2176,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Update":36560031,"Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Update":120,"Value":825,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Update":7042587,"Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Update":8,"Value":127,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":8,"Executor Deserialize CPU Time":7042587,"Executor Run Time":120,"Executor CPU Time":36560031,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":289555,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":292935952,"JVMOffHeapMemory":95141200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":351534,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":351534,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4929392640,"ProcessTreeJVMRSSMemory":539996160,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":1,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":215586960,"JVMOffHeapMemory":60718904,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1492038,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1492038,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3014057984,"ProcessTreeJVMRSSMemory":422723584,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at BisectingKMeans.scala:170","Number of Tasks":2,"RDD Info":[{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:370)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:170)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440977793,"Completion Time":1538440978821,"Accumulables":[{"ID":26,"Name":"internal.metrics.executorDeserializeTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":35,"Name":"internal.metrics.peakExecutionMemory","Value":2176,"Internal":true,"Count Failed Values":true},{"ID":44,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":356,"Internal":true,"Count Failed Values":true},{"ID":29,"Name":"internal.metrics.executorCpuTime","Value":238787567,"Internal":true,"Count Failed Values":true},{"ID":47,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":46,"Name":"internal.metrics.shuffle.write.writeTime","Value":13824613,"Internal":true,"Count Failed Values":true},{"ID":34,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":28,"Name":"internal.metrics.executorRunTime","Value":825,"Internal":true,"Count Failed Values":true},{"ID":45,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":2,"Internal":true,"Count Failed Values":true},{"ID":27,"Name":"internal.metrics.executorDeserializeCpuTime","Value":72737420,"Internal":true,"Count Failed Values":true},{"ID":48,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":30,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":33,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"15\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":1,"Attempt":0,"Launch Time":1538440978844,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979050,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":356,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":88389028,"Value":88389028,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":122,"Value":122,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":27126551,"Value":27126551,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":45,"Value":45,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":45,"Executor Deserialize CPU Time":27126551,"Executor Run Time":122,"Executor CPU Time":88389028,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":356,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":0,"Attempt":0,"Launch Time":1538440979033,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979084,"Failed":false,"Killed":false,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":356,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Update":0,"Value":992,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Update":1706,"Value":3534,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Update":15055355,"Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Update":26,"Value":148,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4722422,"Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":50,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":4722422,"Executor Run Time":26,"Executor CPU Time":15055355,"Result Size":1706,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":303792496,"JVMOffHeapMemory":95545824,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":371127,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":371127,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4931497984,"ProcessTreeJVMRSSMemory":549777408,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":2,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":227393200,"JVMOffHeapMemory":61799392,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":463135,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":463135,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20637,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016163328,"ProcessTreeJVMRSSMemory":436539392,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":10,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"14\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[9],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":9,"Name":"ShuffledRDD","Scope":"{\"id\":\"13\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[1],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:171)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440978830,"Completion Time":1538440979086,"Accumulables":[{"ID":68,"Name":"internal.metrics.shuffle.read.recordsRead","Value":2,"Internal":true,"Count Failed Values":true},{"ID":59,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":53,"Name":"internal.metrics.executorRunTime","Value":148,"Internal":true,"Count Failed Values":true},{"ID":62,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":65,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":55,"Name":"internal.metrics.resultSize","Value":3534,"Internal":true,"Count Failed Values":true},{"ID":64,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":67,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":58,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":52,"Name":"internal.metrics.executorDeserializeCpuTime","Value":31848973,"Internal":true,"Count Failed Values":true},{"ID":60,"Name":"internal.metrics.peakExecutionMemory","Value":992,"Internal":true,"Count Failed Values":true},{"ID":54,"Name":"internal.metrics.executorCpuTime","Value":103444383,"Internal":true,"Count Failed Values":true},{"ID":63,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":2,"Internal":true,"Count Failed Values":true},{"ID":66,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":356,"Internal":true,"Count Failed Values":true},{"ID":51,"Name":"internal.metrics.executorDeserializeTime","Value":50,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":1,"Completion Time":1538440979087,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":2,"Submission Time":1538440979161,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[3,4],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":0,"Attempt":0,"Launch Time":1538440979184,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979348,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":259310,"Value":259310,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":40081727,"Value":40081727,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":98,"Value":98,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":24271689,"Value":24271689,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":39,"Value":39,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":39,"Executor Deserialize CPU Time":24271689,"Executor Run Time":98,"Executor CPU Time":40081727,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":259310,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":1,"Attempt":0,"Launch Time":1538440979344,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979441,"Failed":false,"Killed":false,"Accumulables":[{"ID":98,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Update":221381,"Value":480691,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Update":23089017,"Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Update":74,"Value":172,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3471167,"Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":43,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3471167,"Executor Run Time":74,"Executor CPU Time":23089017,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":221381,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":318926040,"JVMOffHeapMemory":96521592,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":391718,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":391718,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":569753600,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":3,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":236711480,"JVMOffHeapMemory":62683008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":483726,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":483726,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":445640704,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":12,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"25\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[11],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":11,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"24\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979163,"Completion Time":1538440979444,"Accumulables":[{"ID":83,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":95,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":77,"Name":"internal.metrics.executorDeserializeCpuTime","Value":27742856,"Internal":true,"Count Failed Values":true},{"ID":80,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":98,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":85,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":94,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":76,"Name":"internal.metrics.executorDeserializeTime","Value":43,"Internal":true,"Count Failed Values":true},{"ID":79,"Name":"internal.metrics.executorCpuTime","Value":63170744,"Internal":true,"Count Failed Values":true},{"ID":97,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":96,"Name":"internal.metrics.shuffle.write.writeTime","Value":480691,"Internal":true,"Count Failed Values":true},{"ID":78,"Name":"internal.metrics.executorRunTime","Value":172,"Internal":true,"Count Failed Values":true},{"ID":84,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"28\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":0,"Attempt":0,"Launch Time":1538440979462,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979528,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":17714408,"Value":17714408,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":30,"Value":30,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12579502,"Value":12579502,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":12579502,"Executor Run Time":30,"Executor CPU Time":17714408,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":1,"Attempt":0,"Launch Time":1538440979527,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979572,"Failed":false,"Killed":false,"Accumulables":[{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Update":16462125,"Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Update":16,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3190663,"Value":15770165,"Internal":true,"Count Failed Values":true},{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":26,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3190663,"Executor Run Time":16,"Executor CPU Time":16462125,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":1,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":329919832,"JVMOffHeapMemory":96756344,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":413740,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":413740,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935208960,"ProcessTreeJVMRSSMemory":585252864,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":4,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":242876648,"JVMOffHeapMemory":62975784,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":505748,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":505748,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20922,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3019313152,"ProcessTreeJVMRSSMemory":451244032,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":14,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"27\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[13],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":13,"Name":"ShuffledRDD","Scope":"{\"id\":\"26\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[12],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[3],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979446,"Completion Time":1538440979573,"Accumulables":[{"ID":101,"Name":"internal.metrics.executorDeserializeTime","Value":26,"Internal":true,"Count Failed Values":true},{"ID":110,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":104,"Name":"internal.metrics.executorCpuTime","Value":34176533,"Internal":true,"Count Failed Values":true},{"ID":113,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":116,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":115,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":118,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":109,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":103,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":112,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":105,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":114,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":117,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":108,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":102,"Name":"internal.metrics.executorDeserializeCpuTime","Value":15770165,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":2,"Completion Time":1538440979573,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":3,"Submission Time":1538440979609,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[5,6],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":0,"Attempt":0,"Launch Time":1538440979638,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979756,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":272852,"Value":272852,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":23042622,"Value":23042622,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":76,"Value":76,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":13112180,"Value":13112180,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":28,"Executor Deserialize CPU Time":13112180,"Executor Run Time":76,"Executor CPU Time":23042622,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":272852,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":1,"Attempt":0,"Launch Time":1538440979754,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979851,"Failed":false,"Killed":false,"Accumulables":[{"ID":148,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Update":229882,"Value":502734,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Update":22093052,"Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Update":81,"Value":157,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3862579,"Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":32,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":3862579,"Executor Run Time":81,"Executor CPU Time":22093052,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":229882,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341682304,"JVMOffHeapMemory":97514672,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":434309,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":434309,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4935254016,"ProcessTreeJVMRSSMemory":597999616,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":5,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252029672,"JVMOffHeapMemory":63463032,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":526317,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":526317,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21041,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020365824,"ProcessTreeJVMRSSMemory":458960896,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":16,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"35\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[15],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":15,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"34\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979619,"Completion Time":1538440979852,"Accumulables":[{"ID":146,"Name":"internal.metrics.shuffle.write.writeTime","Value":502734,"Internal":true,"Count Failed Values":true},{"ID":128,"Name":"internal.metrics.executorRunTime","Value":157,"Internal":true,"Count Failed Values":true},{"ID":134,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":133,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":127,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16974759,"Internal":true,"Count Failed Values":true},{"ID":145,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":130,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":148,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":129,"Name":"internal.metrics.executorCpuTime","Value":45135674,"Internal":true,"Count Failed Values":true},{"ID":147,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":126,"Name":"internal.metrics.executorDeserializeTime","Value":32,"Internal":true,"Count Failed Values":true},{"ID":135,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":144,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"38\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":6,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":0,"Attempt":0,"Launch Time":1538440979869,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979921,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15546330,"Value":15546330,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":19,"Value":19,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":11263754,"Value":11263754,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":22,"Value":22,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":22,"Executor Deserialize CPU Time":11263754,"Executor Run Time":19,"Executor CPU Time":15546330,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":6,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":1,"Attempt":0,"Launch Time":1538440979920,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440979972,"Failed":false,"Killed":false,"Accumulables":[{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":155,"Name":"internal.metrics.resultSize","Update":1871,"Value":3699,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Update":15089701,"Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Update":27,"Value":46,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3045280,"Value":14309034,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Update":3,"Value":25,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":3,"Executor Deserialize CPU Time":3045280,"Executor Run Time":27,"Executor CPU Time":15089701,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":6,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":350990264,"JVMOffHeapMemory":97710440,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":456312,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":456312,"OffHeapUnifiedMemory":0,"DirectPoolMemory":135031,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4932550656,"ProcessTreeJVMRSSMemory":604299264,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":6,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":18,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"37\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[17],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":17,"Name":"ShuffledRDD","Scope":"{\"id\":\"36\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[16],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[5],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440979854,"Completion Time":1538440979973,"Accumulables":[{"ID":155,"Name":"internal.metrics.resultSize","Value":3699,"Internal":true,"Count Failed Values":true},{"ID":164,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":167,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":158,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":166,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":151,"Name":"internal.metrics.executorDeserializeTime","Value":25,"Internal":true,"Count Failed Values":true},{"ID":160,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":163,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":154,"Name":"internal.metrics.executorCpuTime","Value":30636031,"Internal":true,"Count Failed Values":true},{"ID":157,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":165,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":168,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":159,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":153,"Name":"internal.metrics.executorRunTime","Value":46,"Internal":true,"Count Failed Values":true},{"ID":162,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":152,"Name":"internal.metrics.executorDeserializeCpuTime","Value":14309034,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":3,"Completion Time":1538440979974,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":4,"Submission Time":1538440980008,"Stage Infos":[{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[7,8],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Port":46411},"Timestamp":1538440980522} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440980759,"Executor ID":"1","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"1","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000002 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000002\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":13,"Index":0,"Attempt":0,"Launch Time":1538440980049,"Executor ID":"1","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440980757,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440986317,"Executor ID":"2","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000003/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Maximum Memory":384093388,"Timestamp":1538440986696,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538440988793,"Executor ID":"3","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000004/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Maximum Memory":384093388,"Timestamp":1538440989162,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Port":39119},"Timestamp":1538440993798} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"2","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":14,"Index":0,"Attempt":1,"Launch Time":1538440986317,"Executor ID":"2","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440994010,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440994012,"Executor ID":"2","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000003 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000003\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":7,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":1,"Attempt":0,"Launch Time":1538440988793,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995450,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":4,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":10065137,"Value":10065137,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":1088,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Update":360,"Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":1406669099,"Value":1406669099,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":4128,"Value":4128,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":726605764,"Value":726605764,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":1995,"Value":1995,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1995,"Executor Deserialize CPU Time":726605764,"Executor Run Time":4128,"Executor CPU Time":1406669099,"Result Size":1705,"JVM GC Time":360,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":10065137,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":7,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":2,"Launch Time":1538440995449,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440995696,"Failed":false,"Killed":false,"Accumulables":[{"ID":198,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Update":293846,"Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Update":1662,"Value":3367,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Update":91844758,"Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Update":220,"Value":4348,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8316162,"Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Update":9,"Value":2004,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":9,"Executor Deserialize CPU Time":8316162,"Executor Run Time":220,"Executor CPU Time":91844758,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":293846,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"2","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":201931120,"JVMOffHeapMemory":58230320,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094710,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094710,"OffHeapUnifiedMemory":0,"DirectPoolMemory":45633,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3023769600,"ProcessTreeJVMRSSMemory":410324992,"ProcessTreePythonVMemory":285470720,"ProcessTreePythonRSSMemory":30171136,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":195471784,"JVMOffHeapMemory":100867584,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":476885,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":476885,"OffHeapUnifiedMemory":0,"DirectPoolMemory":171571,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":4971368448,"ProcessTreeJVMRSSMemory":663375872,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"1","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":258718872,"JVMOffHeapMemory":63737056,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":548320,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":548320,"OffHeapUnifiedMemory":0,"DirectPoolMemory":21084,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021418496,"ProcessTreeJVMRSSMemory":466001920,"ProcessTreePythonVMemory":958914560,"ProcessTreePythonRSSMemory":106622976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":7,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":223684056,"JVMOffHeapMemory":60665000,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482102,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482102,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3015626752,"ProcessTreeJVMRSSMemory":404672512,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":7,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":20,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"45\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[19],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":19,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"44\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440980015,"Completion Time":1538440995697,"Accumulables":[{"ID":176,"Name":"internal.metrics.executorDeserializeTime","Value":2004,"Internal":true,"Count Failed Values":true},{"ID":185,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":194,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":184,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":178,"Name":"internal.metrics.executorRunTime","Value":4348,"Internal":true,"Count Failed Values":true},{"ID":196,"Name":"internal.metrics.shuffle.write.writeTime","Value":10358983,"Internal":true,"Count Failed Values":true},{"ID":181,"Name":"internal.metrics.jvmGCTime","Value":360,"Internal":true,"Count Failed Values":true},{"ID":180,"Name":"internal.metrics.resultSize","Value":3367,"Internal":true,"Count Failed Values":true},{"ID":198,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":183,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":177,"Name":"internal.metrics.executorDeserializeCpuTime","Value":734921926,"Internal":true,"Count Failed Values":true},{"ID":195,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":179,"Name":"internal.metrics.executorCpuTime","Value":1498513857,"Internal":true,"Count Failed Values":true},{"ID":197,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"48\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Port":40911},"Timestamp":1538440996257} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"3","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1538440995710,"Executor ID":"3","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538440996467,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538440996468,"Executor ID":"3","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000004 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000004\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441002826,"Executor ID":"4","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000005/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441003031,"Executor ID":"5","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000006/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":8,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Maximum Memory":384093388,"Timestamp":1538441003132,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Maximum Memory":384093388,"Timestamp":1538441003383,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":0,"Attempt":1,"Launch Time":1538441002828,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006147,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":350,"Value":350,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":1914,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":219243972,"Value":219243972,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":893,"Value":893,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":717217987,"Value":717217987,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":1972,"Value":1972,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":1972,"Executor Deserialize CPU Time":717217987,"Executor Run Time":893,"Executor CPU Time":219243972,"Result Size":1914,"JVM GC Time":350,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":1,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":178,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":8,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":1,"Attempt":0,"Launch Time":1538441003032,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441006584,"Failed":false,"Killed":false,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Update":10,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Update":270,"Value":620,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Update":1914,"Value":3828,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Update":210863492,"Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Update":412,"Value":1305,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Update":727356712,"Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Update":2604,"Value":4576,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2604,"Executor Deserialize CPU Time":727356712,"Executor Run Time":412,"Executor CPU Time":210863492,"Result Size":1914,"JVM GC Time":270,"Result Serialization Time":10,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":2,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":355,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":263995432,"JVMOffHeapMemory":101978136,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":498888,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":498888,"OffHeapUnifiedMemory":0,"DirectPoolMemory":191656,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5008089088,"ProcessTreeJVMRSSMemory":663732224,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":150497592,"JVMOffHeapMemory":45958576,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":3446,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2984218624,"ProcessTreeJVMRSSMemory":325042176,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":181352744,"JVMOffHeapMemory":47061200,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":22003,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":22003,"OffHeapUnifiedMemory":0,"DirectPoolMemory":11272,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3013332992,"ProcessTreeJVMRSSMemory":416645120,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"3","Stage ID":8,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":226223752,"JVMOffHeapMemory":60840424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":433558,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":433558,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20318,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016937472,"ProcessTreeJVMRSSMemory":406044672,"ProcessTreePythonVMemory":958963712,"ProcessTreePythonRSSMemory":106639360,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":8,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":22,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"47\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[21],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":21,"Name":"ShuffledRDD","Scope":"{\"id\":\"46\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[20],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[7],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538440995698,"Completion Time":1538441006585,"Accumulables":[{"ID":218,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":209,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":202,"Name":"internal.metrics.executorDeserializeCpuTime","Value":1444574699,"Internal":true,"Count Failed Values":true},{"ID":205,"Name":"internal.metrics.resultSize","Value":3828,"Internal":true,"Count Failed Values":true},{"ID":214,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":217,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":208,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":216,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":207,"Name":"internal.metrics.resultSerializationTime","Value":12,"Internal":true,"Count Failed Values":true},{"ID":210,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":201,"Name":"internal.metrics.executorDeserializeTime","Value":4576,"Internal":true,"Count Failed Values":true},{"ID":213,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":204,"Name":"internal.metrics.executorCpuTime","Value":430107464,"Internal":true,"Count Failed Values":true},{"ID":203,"Name":"internal.metrics.executorRunTime","Value":1305,"Internal":true,"Count Failed Values":true},{"ID":212,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":215,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":206,"Name":"internal.metrics.jvmGCTime","Value":620,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":4,"Completion Time":1538441006585,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":5,"Submission Time":1538441006610,"Stage Infos":[{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[9,10],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Port":39248},"Timestamp":1538441010070} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Port":43165},"Timestamp":1538441010233} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"4","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":20,"Index":0,"Attempt":0,"Launch Time":1538441006622,"Executor ID":"4","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010280,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010281,"Executor ID":"4","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000005 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000005\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"5","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":21,"Index":1,"Attempt":0,"Launch Time":1538441006623,"Executor ID":"5","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441010484,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441010485,"Executor ID":"5","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000006 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000006\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441015443,"Executor ID":"6","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000007/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Maximum Memory":384093388,"Timestamp":1538441015852,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441020314,"Executor ID":"7","Executor Info":{"Host":"rezamemory-3.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stdout?start=-4096","stderr":"http://rezamemory-3.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000008/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Maximum Memory":384093388,"Timestamp":1538441020602,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Port":45593},"Timestamp":1538441022942} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"6","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":22,"Index":1,"Attempt":1,"Launch Time":1538441015444,"Executor ID":"6","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441023152,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441023153,"Executor ID":"6","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000007 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000007\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":9,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":0,"Attempt":1,"Launch Time":1538441020315,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441025900,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":3971129,"Value":3971129,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Update":244,"Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":1705,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":1268816374,"Value":1268816374,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":2978,"Value":2978,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":714859741,"Value":714859741,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":2106,"Value":2106,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2106,"Executor Deserialize CPU Time":714859741,"Executor Run Time":2978,"Executor CPU Time":1268816374,"Result Size":1705,"JVM GC Time":244,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":3971129,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":9,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":1,"Attempt":2,"Launch Time":1538441025899,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026136,"Failed":false,"Killed":false,"Accumulables":[{"ID":248,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":144,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Update":265841,"Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Update":1705,"Value":3410,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Update":88980290,"Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Update":201,"Value":3179,"Internal":true,"Count Failed Values":true},{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Update":8550572,"Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Update":13,"Value":2119,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":13,"Executor Deserialize CPU Time":8550572,"Executor Run Time":201,"Executor CPU Time":88980290,"Result Size":1705,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":265841,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":332727504,"JVMOffHeapMemory":103237664,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519462,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519462,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658915328,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"5","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":184519808,"JVMOffHeapMemory":58341088,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20420,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":2998673408,"ProcessTreeJVMRSSMemory":378527744,"ProcessTreePythonVMemory":0,"ProcessTreePythonRSSMemory":0,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":218694008,"JVMOffHeapMemory":60757008,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1482103,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1482103,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20668,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3020120064,"ProcessTreeJVMRSSMemory":423698432,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"4","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":220189424,"JVMOffHeapMemory":59534504,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1116714,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1116714,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27895,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3024392192,"ProcessTreeJVMRSSMemory":431939584,"ProcessTreePythonVMemory":283738112,"ProcessTreePythonRSSMemory":27226112,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"6","Stage ID":9,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":208356192,"JVMOffHeapMemory":58297728,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1094711,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1094711,"OffHeapUnifiedMemory":0,"DirectPoolMemory":27296,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3027820544,"ProcessTreeJVMRSSMemory":439750656,"ProcessTreePythonVMemory":286220288,"ProcessTreePythonRSSMemory":30846976,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":9,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":24,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"55\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[23],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":23,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"54\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441006612,"Completion Time":1538441026137,"Accumulables":[{"ID":227,"Name":"internal.metrics.executorDeserializeCpuTime","Value":723410313,"Internal":true,"Count Failed Values":true},{"ID":245,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":226,"Name":"internal.metrics.executorDeserializeTime","Value":2119,"Internal":true,"Count Failed Values":true},{"ID":235,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":244,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":229,"Name":"internal.metrics.executorCpuTime","Value":1357796664,"Internal":true,"Count Failed Values":true},{"ID":247,"Name":"internal.metrics.input.bytesRead","Value":144,"Internal":true,"Count Failed Values":true},{"ID":232,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":234,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":228,"Name":"internal.metrics.executorRunTime","Value":3179,"Internal":true,"Count Failed Values":true},{"ID":246,"Name":"internal.metrics.shuffle.write.writeTime","Value":4236970,"Internal":true,"Count Failed Values":true},{"ID":231,"Name":"internal.metrics.jvmGCTime","Value":244,"Internal":true,"Count Failed Values":true},{"ID":230,"Name":"internal.metrics.resultSize","Value":3410,"Internal":true,"Count Failed Values":true},{"ID":248,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":233,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"58\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":10,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1538441026147,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026311,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":1828,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":80311930,"Value":80311930,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":89,"Value":89,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":29610969,"Value":29610969,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":62,"Value":62,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":62,"Executor Deserialize CPU Time":29610969,"Executor Run Time":89,"Executor CPU Time":80311930,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":10,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1538441026309,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026375,"Failed":false,"Killed":false,"Accumulables":[{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Update":1828,"Value":3656,"Internal":true,"Count Failed Values":true},{"ID":254,"Name":"internal.metrics.executorCpuTime","Update":18625831,"Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Update":38,"Value":127,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6238101,"Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":68,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6238101,"Executor Run Time":38,"Executor CPU Time":18625831,"Result Size":1828,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":341644736,"JVMOffHeapMemory":103378144,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":541469,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":541469,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228406,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5011247104,"ProcessTreeJVMRSSMemory":658989056,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":10,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":228132872,"JVMOffHeapMemory":61634808,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20669,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3021172736,"ProcessTreeJVMRSSMemory":436867072,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":10,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":26,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"57\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[25],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":25,"Name":"ShuffledRDD","Scope":"{\"id\":\"56\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[24],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[9],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026138,"Completion Time":1538441026376,"Accumulables":[{"ID":254,"Name":"internal.metrics.executorCpuTime","Value":98937761,"Internal":true,"Count Failed Values":true},{"ID":262,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":253,"Name":"internal.metrics.executorRunTime","Value":127,"Internal":true,"Count Failed Values":true},{"ID":265,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":259,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":268,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":267,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":258,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":252,"Name":"internal.metrics.executorDeserializeCpuTime","Value":35849070,"Internal":true,"Count Failed Values":true},{"ID":255,"Name":"internal.metrics.resultSize","Value":3656,"Internal":true,"Count Failed Values":true},{"ID":264,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":263,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":266,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":260,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":251,"Name":"internal.metrics.executorDeserializeTime","Value":68,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":5,"Completion Time":1538441026376,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerJobStart","Job ID":6,"Submission Time":1538441026404,"Stage Infos":[{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]},{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[12,11],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":11,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":0,"Attempt":0,"Launch Time":1538441026450,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026586,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":8,"Value":8,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":72,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":278446,"Value":278446,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":355,"Value":355,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1264,"Value":1264,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":1662,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23317154,"Value":23317154,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":69,"Value":69,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":17832528,"Value":17832528,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":53,"Value":53,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":53,"Executor Deserialize CPU Time":17832528,"Executor Run Time":69,"Executor CPU Time":23317154,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":355,"Shuffle Write Time":278446,"Shuffle Records Written":2},"Input Metrics":{"Bytes Read":72,"Records Read":8},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":11,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":1,"Attempt":0,"Launch Time":1538441026585,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026700,"Failed":false,"Killed":false,"Accumulables":[{"ID":298,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":12,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Update":215244,"Value":493690,"Internal":true,"Count Failed Values":true},{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Update":1,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Update":178,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Update":1088,"Value":2352,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Update":1662,"Value":3324,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Update":23292541,"Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Update":94,"Value":163,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Update":4400590,"Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Update":4,"Value":57,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":4,"Executor Deserialize CPU Time":4400590,"Executor Run Time":94,"Executor CPU Time":23292541,"Result Size":1662,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":178,"Shuffle Write Time":215244,"Shuffle Records Written":1},"Input Metrics":{"Bytes Read":36,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":198912952,"JVMOffHeapMemory":104016864,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":554933,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":554933,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5040721920,"ProcessTreeJVMRSSMemory":705302528,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":11,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":249428840,"JVMOffHeapMemory":62917480,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":455614,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":455614,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3035901952,"ProcessTreeJVMRSSMemory":447041536,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":11,"Stage Attempt ID":0,"Stage Name":"filter at BisectingKMeans.scala:213","Number of Tasks":2,"RDD Info":[{"RDD ID":28,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"65\",\"name\":\"filter\"}","Callsite":"filter at BisectingKMeans.scala:213","Parent IDs":[27],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":27,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"64\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:372","Parent IDs":[8],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":8,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"12\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:170","Parent IDs":[7],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":7,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"11\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:169","Parent IDs":[6],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":2,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":6,"Name":"ZippedPartitionsRDD2","Scope":"{\"id\":\"10\",\"name\":\"zip\"}","Callsite":"zip at BisectingKMeans.scala:169","Parent IDs":[3,5],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":3,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"1\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[2],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":5,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"9\",\"name\":\"map\"}","Callsite":"map at BisectingKMeans.scala:168","Parent IDs":[3],"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.RDD.filter(RDD.scala:387)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:213)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026408,"Completion Time":1538441026701,"Accumulables":[{"ID":295,"Name":"internal.metrics.shuffle.write.recordsWritten","Value":3,"Internal":true,"Count Failed Values":true},{"ID":298,"Name":"internal.metrics.input.recordsRead","Value":12,"Internal":true,"Count Failed Values":true},{"ID":280,"Name":"internal.metrics.resultSize","Value":3324,"Internal":true,"Count Failed Values":true},{"ID":283,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":277,"Name":"internal.metrics.executorDeserializeCpuTime","Value":22233118,"Internal":true,"Count Failed Values":true},{"ID":294,"Name":"internal.metrics.shuffle.write.bytesWritten","Value":533,"Internal":true,"Count Failed Values":true},{"ID":276,"Name":"internal.metrics.executorDeserializeTime","Value":57,"Internal":true,"Count Failed Values":true},{"ID":285,"Name":"internal.metrics.peakExecutionMemory","Value":2352,"Internal":true,"Count Failed Values":true},{"ID":279,"Name":"internal.metrics.executorCpuTime","Value":46609695,"Internal":true,"Count Failed Values":true},{"ID":297,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":284,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":278,"Name":"internal.metrics.executorRunTime","Value":163,"Internal":true,"Count Failed Values":true},{"ID":296,"Name":"internal.metrics.shuffle.write.writeTime","Value":493690,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"68\",\"name\":\"collect\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":12,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":0,"Attempt":0,"Launch Time":1538441026714,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026795,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":178,"Value":178,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":800,"Value":800,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":1871,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":16951615,"Value":16951615,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":28,"Value":28,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":12613041,"Value":12613041,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":31,"Value":31,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":31,"Executor Deserialize CPU Time":12613041,"Executor Run Time":28,"Executor CPU Time":16951615,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":1,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":178,"Total Records Read":1},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":12,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":1,"Attempt":0,"Launch Time":1538441026794,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441026839,"Failed":false,"Killed":false,"Accumulables":[{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Update":355,"Value":533,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Update":2,"Value":3,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Update":992,"Value":1792,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Update":1871,"Value":3742,"Internal":true,"Count Failed Values":true},{"ID":304,"Name":"internal.metrics.executorCpuTime","Update":17828037,"Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Update":24,"Value":52,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Update":3879530,"Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Update":5,"Value":36,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":5,"Executor Deserialize CPU Time":3879530,"Executor Run Time":24,"Executor CPU Time":17828037,"Result Size":1871,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":2,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":355,"Total Records Read":2},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":0,"Records Read":0},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":204287872,"JVMOffHeapMemory":104055736,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":519458,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":519458,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5047037952,"ProcessTreeJVMRSSMemory":708661248,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"7","Stage ID":12,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":252161344,"JVMOffHeapMemory":63019944,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":441078,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":441078,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20911,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3038007296,"ProcessTreeJVMRSSMemory":451837952,"ProcessTreePythonVMemory":958894080,"ProcessTreePythonRSSMemory":106696704,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":12,"Stage Attempt ID":0,"Stage Name":"collect at BisectingKMeans.scala:304","Number of Tasks":2,"RDD Info":[{"RDD ID":30,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"67\",\"name\":\"mapValues\"}","Callsite":"mapValues at BisectingKMeans.scala:303","Parent IDs":[29],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":29,"Name":"ShuffledRDD","Scope":"{\"id\":\"66\",\"name\":\"aggregateByKey\"}","Callsite":"aggregateByKey at BisectingKMeans.scala:300","Parent IDs":[28],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[11],"Details":"org.apache.spark.rdd.RDD.collect(RDD.scala:944)\norg.apache.spark.mllib.clustering.BisectingKMeans$.org$apache$spark$mllib$clustering$BisectingKMeans$$summarize(BisectingKMeans.scala:304)\norg.apache.spark.mllib.clustering.BisectingKMeans$$anonfun$run$1.apply$mcVI$sp(BisectingKMeans.scala:216)\nscala.collection.immutable.Range.foreach$mVc$sp(Range.scala:160)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:210)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:255)\norg.apache.spark.mllib.clustering.BisectingKMeans.run(BisectingKMeans.scala:261)\norg.apache.spark.mllib.api.python.PythonMLLibAPI.trainBisectingKMeans(PythonMLLibAPI.scala:135)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026702,"Completion Time":1538441026840,"Accumulables":[{"ID":304,"Name":"internal.metrics.executorCpuTime","Value":34779652,"Internal":true,"Count Failed Values":true},{"ID":313,"Name":"internal.metrics.shuffle.read.localBlocksFetched","Value":3,"Internal":true,"Count Failed Values":true},{"ID":307,"Name":"internal.metrics.resultSerializationTime","Value":2,"Internal":true,"Count Failed Values":true},{"ID":316,"Name":"internal.metrics.shuffle.read.localBytesRead","Value":533,"Internal":true,"Count Failed Values":true},{"ID":301,"Name":"internal.metrics.executorDeserializeTime","Value":36,"Internal":true,"Count Failed Values":true},{"ID":310,"Name":"internal.metrics.peakExecutionMemory","Value":1792,"Internal":true,"Count Failed Values":true},{"ID":318,"Name":"internal.metrics.shuffle.read.recordsRead","Value":3,"Internal":true,"Count Failed Values":true},{"ID":309,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":303,"Name":"internal.metrics.executorRunTime","Value":52,"Internal":true,"Count Failed Values":true},{"ID":312,"Name":"internal.metrics.shuffle.read.remoteBlocksFetched","Value":0,"Internal":true,"Count Failed Values":true},{"ID":315,"Name":"internal.metrics.shuffle.read.remoteBytesReadToDisk","Value":0,"Internal":true,"Count Failed Values":true},{"ID":317,"Name":"internal.metrics.shuffle.read.fetchWaitTime","Value":0,"Internal":true,"Count Failed Values":true},{"ID":308,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":302,"Name":"internal.metrics.executorDeserializeCpuTime","Value":16492571,"Internal":true,"Count Failed Values":true},{"ID":314,"Name":"internal.metrics.shuffle.read.remoteBytesRead","Value":0,"Internal":true,"Count Failed Values":true},{"ID":305,"Name":"internal.metrics.resultSize","Value":3742,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":6,"Completion Time":1538441026840,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerUnpersistRDD","RDD ID":32} +{"Event":"SparkListenerUnpersistRDD","RDD ID":5} +{"Event":"SparkListenerJobStart","Job ID":7,"Submission Time":1538441026935,"Stage Infos":[{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Accumulables":[]}],"Stage IDs":[13],"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} +{"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Accumulables":[]},"Properties":{"spark.rdd.scope.noOverride":"true","spark.rdd.scope":"{\"id\":\"79\",\"name\":\"sum\"}"}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Port":40992},"Timestamp":1538441027285} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"7","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":31,"Index":0,"Attempt":0,"Launch Time":1538441026947,"Executor ID":"7","Host":"rezamemory-3.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441027494,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441027495,"Executor ID":"7","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000008 on host: rezamemory-3.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000008\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441032740,"Executor ID":"8","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000009/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Maximum Memory":384093388,"Timestamp":1538441033142,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerExecutorAdded","Timestamp":1538441036142,"Executor ID":"9","Executor Info":{"Host":"rezamemory-2.gce.something.com","Total Cores":1,"Log Urls":{"stdout":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stdout?start=-4096","stderr":"http://rezamemory-2.gce.something.com:8042/node/containerlogs/container_1538416563558_0014_01_000010/root/stderr?start=-4096"}}} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Port":40797},"Maximum Memory":384093388,"Timestamp":1538441036560,"Maximum Onheap Memory":384093388,"Maximum Offheap Memory":0} +{"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Port":41485},"Timestamp":1538441040323} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExecutorLostFailure","Executor ID":"8","Exit Caused By App":true,"Loss Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"},"Task Info":{"Task ID":32,"Index":0,"Attempt":1,"Launch Time":1538441032741,"Executor ID":"8","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441040533,"Failed":true,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerExecutorRemoved","Timestamp":1538441040534,"Executor ID":"8","Removed Reason":"Container marked as failed: container_1538416563558_0014_01_000009 on host: rezamemory-2.gce.something.com. Exit status: 56. Diagnostics: Exception from container-launch.\nContainer id: container_1538416563558_0014_01_000009\nExit code: 56\nStack trace: ExitCodeException exitCode=56: \n\tat org.apache.hadoop.util.Shell.runCommand(Shell.java:601)\n\tat org.apache.hadoop.util.Shell.run(Shell.java:504)\n\tat org.apache.hadoop.util.Shell$ShellCommandExecutor.execute(Shell.java:786)\n\tat org.apache.hadoop.yarn.server.nodemanager.DefaultContainerExecutor.launchContainer(DefaultContainerExecutor.java:213)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:302)\n\tat org.apache.hadoop.yarn.server.nodemanager.containermanager.launcher.ContainerLaunch.call(ContainerLaunch.java:82)\n\tat java.util.concurrent.FutureTask.run(FutureTask.java:266)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)\n\tat java.lang.Thread.run(Thread.java:745)\n\n\nContainer exited with a non-zero exit code 56\n"} +{"Event":"SparkListenerTaskStart","Stage ID":13,"Stage Attempt ID":0,"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Killed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":33,"Index":1,"Attempt":0,"Launch Time":1538441036144,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042185,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":2,"Value":2,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":36,"Value":36,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Update":1,"Value":1,"Internal":true,"Count Failed Values":true},{"ID":331,"Name":"internal.metrics.jvmGCTime","Update":288,"Value":288,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1539,"Value":1539,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":1278640624,"Value":1278640624,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":2796,"Value":2796,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":720112530,"Value":720112530,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":2587,"Value":2587,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":2587,"Executor Deserialize CPU Time":720112530,"Executor Run Time":2796,"Executor CPU Time":1278640624,"Result Size":1539,"JVM GC Time":288,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":36,"Records Read":2},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":13,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":34,"Index":0,"Attempt":2,"Launch Time":1538441042184,"Executor ID":"9","Host":"rezamemory-2.gce.something.com","Locality":"NODE_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1538441042334,"Failed":false,"Killed":false,"Accumulables":[{"ID":348,"Name":"internal.metrics.input.recordsRead","Update":4,"Value":6,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Update":72,"Value":108,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Update":0,"Value":0,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Update":1453,"Value":2992,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Update":69678739,"Value":1348319363,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Update":118,"Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Update":6252896,"Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Update":6,"Value":2593,"Internal":true,"Count Failed Values":true}]},"Task Metrics":{"Executor Deserialize Time":6,"Executor Deserialize CPU Time":6252896,"Executor Run Time":118,"Executor CPU Time":69678739,"Result Size":1453,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":0,"Fetch Wait Time":0,"Remote Bytes Read":0,"Remote Bytes Read To Disk":0,"Local Bytes Read":0,"Total Records Read":0},"Shuffle Write Metrics":{"Shuffle Bytes Written":0,"Shuffle Write Time":0,"Shuffle Records Written":0},"Input Metrics":{"Bytes Read":72,"Records Read":4},"Output Metrics":{"Bytes Written":0,"Records Written":0},"Updated Blocks":[]}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"driver","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":266240264,"JVMOffHeapMemory":104976128,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":534126,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":534126,"OffHeapUnifiedMemory":0,"DirectPoolMemory":228407,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":5067235328,"ProcessTreeJVMRSSMemory":710475776,"ProcessTreePythonVMemory":408375296,"ProcessTreePythonRSSMemory":40284160,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"8","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":197860072,"JVMOffHeapMemory":57762424,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":25453,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3028791296,"ProcessTreeJVMRSSMemory":430297088,"ProcessTreePythonVMemory":286212096,"ProcessTreePythonRSSMemory":30441472,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageExecutorMetrics","Executor ID":"9","Stage ID":13,"Stage Attempt ID":0,"Executor Metrics":{"JVMHeapMemory":193766856,"JVMOffHeapMemory":59006656,"OnHeapExecutionMemory":0,"OffHeapExecutionMemory":0,"OnHeapStorageMemory":1088805,"OffHeapStorageMemory":0,"OnHeapUnifiedMemory":1088805,"OffHeapUnifiedMemory":0,"DirectPoolMemory":20181,"MappedPoolMemory":0,"ProcessTreeJVMVMemory":3016261632,"ProcessTreeJVMRSSMemory":405860352,"ProcessTreePythonVMemory":625926144,"ProcessTreePythonRSSMemory":69013504,"ProcessTreeOtherVMemory":0,"ProcessTreeOtherRSSMemory":0}} +{"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":13,"Stage Attempt ID":0,"Stage Name":"sum at BisectingKMeansModel.scala:101","Number of Tasks":2,"RDD Info":[{"RDD ID":36,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"78\",\"name\":\"map\"}","Callsite":"map at BisectingKMeansModel.scala:101","Parent IDs":[35],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":35,"Name":"MapPartitionsRDD","Scope":"{\"id\":\"77\",\"name\":\"mapPartitions\"}","Callsite":"mapPartitions at PythonMLLibAPI.scala:1346","Parent IDs":[34],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":34,"Name":"PythonRDD","Callsite":"RDD at PythonRDD.scala:53","Parent IDs":[1],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":0,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0},{"RDD ID":1,"Name":"data/mllib/kmeans_data.txt","Scope":"{\"id\":\"0\",\"name\":\"textFile\"}","Callsite":"textFile at NativeMethodAccessorImpl.java:0","Parent IDs":[0],"Storage Level":{"Use Disk":false,"Use Memory":false,"Deserialized":false,"Replication":1},"Number of Partitions":2,"Number of Cached Partitions":0,"Memory Size":0,"Disk Size":0}],"Parent IDs":[],"Details":"org.apache.spark.rdd.DoubleRDDFunctions.sum(DoubleRDDFunctions.scala:34)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:101)\norg.apache.spark.mllib.clustering.BisectingKMeansModel.computeCost(BisectingKMeansModel.scala:108)\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:498)\npy4j.reflection.MethodInvoker.invoke(MethodInvoker.java:244)\npy4j.reflection.ReflectionEngine.invoke(ReflectionEngine.java:357)\npy4j.Gateway.invoke(Gateway.java:282)\npy4j.commands.AbstractCommand.invokeMethod(AbstractCommand.java:132)\npy4j.commands.CallCommand.execute(CallCommand.java:79)\npy4j.GatewayConnection.run(GatewayConnection.java:238)\njava.lang.Thread.run(Thread.java:745)","Submission Time":1538441026936,"Completion Time":1538441042335,"Accumulables":[{"ID":331,"Name":"internal.metrics.jvmGCTime","Value":288,"Internal":true,"Count Failed Values":true},{"ID":334,"Name":"internal.metrics.diskBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":328,"Name":"internal.metrics.executorRunTime","Value":2914,"Internal":true,"Count Failed Values":true},{"ID":327,"Name":"internal.metrics.executorDeserializeCpuTime","Value":726365426,"Internal":true,"Count Failed Values":true},{"ID":348,"Name":"internal.metrics.input.recordsRead","Value":6,"Internal":true,"Count Failed Values":true},{"ID":330,"Name":"internal.metrics.resultSize","Value":2992,"Internal":true,"Count Failed Values":true},{"ID":333,"Name":"internal.metrics.memoryBytesSpilled","Value":0,"Internal":true,"Count Failed Values":true},{"ID":332,"Name":"internal.metrics.resultSerializationTime","Value":1,"Internal":true,"Count Failed Values":true},{"ID":326,"Name":"internal.metrics.executorDeserializeTime","Value":2593,"Internal":true,"Count Failed Values":true},{"ID":347,"Name":"internal.metrics.input.bytesRead","Value":108,"Internal":true,"Count Failed Values":true},{"ID":329,"Name":"internal.metrics.executorCpuTime","Value":1348319363,"Internal":true,"Count Failed Values":true}]}} +{"Event":"SparkListenerJobEnd","Job ID":7,"Completion Time":1538441042335,"Job Result":{"Result":"JobSucceeded"}} +{"Event":"SparkListenerApplicationEnd","Timestamp":1538441042338} diff --git a/core/src/test/resources/spark-events/local-1422981759269 b/core/src/test/resources/spark-events/local-1422981759269 index 4794e56d1107a..8223f3fc926ea 100755 --- a/core/src/test/resources/spark-events/local-1422981759269 +++ b/core/src/test/resources/spark-events/local-1422981759269 @@ -1,88 +1,88 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1422981759407} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1422981759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981759269","Timestamp":1422981758277,"User":"irashid"} {"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981762082,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981762083,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981762075,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981762085,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981762069,"Completion Time":1422981762637,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981763578,"Completion Time":1422981764005,"Accumulables":[]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981764014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764014,"Completion Time":1422981764045,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1422981764399,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1422981764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1422981764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1422981764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981764396,"Completion Time":1422981764648,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1422981765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1422981765029,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1422981765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1422981765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981765026,"Completion Time":1422981765050,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerApplicationEnd","Timestamp":1422981766912} diff --git a/core/src/test/resources/spark-events/local-1422981780767 b/core/src/test/resources/spark-events/local-1422981780767 index f14a000bf2c28..9d00e06fdf0fb 100755 --- a/core/src/test/resources/spark-events/local-1422981780767 +++ b/core/src/test/resources/spark-events/local-1422981780767 @@ -1,82 +1,82 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":57971},"Maximum Memory":278302556,"Timestamp":1422981780906} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57969","spark.repl.class.uri":"http://192.168.1.103:57968","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57970","spark.tachyonStore.folderName":"spark-3f19daee-844c-41d0-a3fc-5e3e508f9731","spark.app.id":"local-1422981780767"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1422981780767","Timestamp":1422981779720,"User":"irashid"} {"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784812,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784814,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1422981784234,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1422981784243,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784816,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":541,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1422981784241,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784817,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1422981784242,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":543,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1422981784240,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981784818,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":542,"Result Size":1268,"JVM GC Time":25,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981784228,"Completion Time":1422981784819,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1422981785832,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786282,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":88000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1422981785829,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":94000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1422981785833,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":435,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":79000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786283,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":73000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":83000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786284,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":436,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":98000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1422981785831,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786285,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":101000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1422981785830,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786286,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":434,"Result Size":1902,"JVM GC Time":19,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":76000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981785829,"Completion Time":1422981786286,"Accumulables":[]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":4,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786337,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":5,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786339,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":1,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":1,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":2,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1422981786296,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786340,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":7,"Attempt":0,"Launch Time":1422981786299,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786341,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":35,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":6,"Attempt":0,"Launch Time":1422981786298,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786342,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":34,"Result Size":862,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"ExceptionFailure","Class Name":"java.lang.RuntimeException","Description":"got a 3, failing","Stack Trace":[{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":18},{"Declaring Class":"$line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1","Method Name":"apply","File Name":"","Line Number":17},{"Declaring Class":"scala.collection.Iterator$$anon$11","Method Name":"next","File Name":"Iterator.scala","Line Number":328},{"Declaring Class":"org.apache.spark.util.Utils$","Method Name":"getIteratorSize","File Name":"Utils.scala","Line Number":1311},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.rdd.RDD$$anonfun$count$1","Method Name":"apply","File Name":"RDD.scala","Line Number":910},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.SparkContext$$anonfun$runJob$4","Method Name":"apply","File Name":"SparkContext.scala","Line Number":1314},{"Declaring Class":"org.apache.spark.scheduler.ResultTask","Method Name":"runTask","File Name":"ResultTask.scala","Line Number":61},{"Declaring Class":"org.apache.spark.scheduler.Task","Method Name":"run","File Name":"Task.scala","Line Number":56},{"Declaring Class":"org.apache.spark.executor.Executor$TaskRunner","Method Name":"run","File Name":"Executor.scala","Line Number":196},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor","Method Name":"runWorker","File Name":"ThreadPoolExecutor.java","Line Number":1145},{"Declaring Class":"java.util.concurrent.ThreadPoolExecutor$Worker","Method Name":"run","File Name":"ThreadPoolExecutor.java","Line Number":615},{"Declaring Class":"java.lang.Thread","Method Name":"run","File Name":"Thread.java","Line Number":745}],"Full Stack Trace":"java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n","Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}},"Task Info":{"Task ID":19,"Index":3,"Attempt":0,"Launch Time":1422981786297,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981786343,"Failed":true,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":36,"Result Size":0,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"count at :20","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:20)\n$line11.$read$$iwC$$iwC$$iwC.(:25)\n$line11.$read$$iwC$$iwC.(:27)\n$line11.$read$$iwC.(:29)\n$line11.$read.(:31)\n$line11.$read$.(:35)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981786296,"Completion Time":1422981786347,"Failure Reason":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobFailed","Exception":{"Message":"Job aborted due to stage failure: Task 3 in stage 2.0 failed 1 times, most recent failure: Lost task 3.0 in stage 2.0 (TID 19, localhost): java.lang.RuntimeException: got a 3, failing\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:18)\n\tat $line11.$read$$iwC$$iwC$$iwC$$iwC$$anonfun$1.apply(:17)\n\tat scala.collection.Iterator$$anon$11.next(Iterator.scala:328)\n\tat org.apache.spark.util.Utils$.getIteratorSize(Utils.scala:1311)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.rdd.RDD$$anonfun$count$1.apply(RDD.scala:910)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.SparkContext$$anonfun$runJob$4.apply(SparkContext.scala:1314)\n\tat org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:61)\n\tat org.apache.spark.scheduler.Task.run(Task.scala:56)\n\tat org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:196)\n\tat java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)\n\tat java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)\n\tat java.lang.Thread.run(Thread.java:745)\n\nDriver stacktrace:","Stack Trace":[{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"org$apache$spark$scheduler$DAGScheduler$$failJobAndIndependentStages","File Name":"DAGScheduler.scala","Line Number":1214},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1203},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$abortStage$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"scala.collection.mutable.ResizableArray$class","Method Name":"foreach","File Name":"ResizableArray.scala","Line Number":59},{"Declaring Class":"scala.collection.mutable.ArrayBuffer","Method Name":"foreach","File Name":"ArrayBuffer.scala","Line Number":47},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"abortStage","File Name":"DAGScheduler.scala","Line Number":1202},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler$$anonfun$handleTaskSetFailed$1","Method Name":"apply","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"scala.Option","Method Name":"foreach","File Name":"Option.scala","Line Number":236},{"Declaring Class":"org.apache.spark.scheduler.DAGScheduler","Method Name":"handleTaskSetFailed","File Name":"DAGScheduler.scala","Line Number":696},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor$$anonfun$receive$2","Method Name":"applyOrElse","File Name":"DAGScheduler.scala","Line Number":1420},{"Declaring Class":"akka.actor.Actor$class","Method Name":"aroundReceive","File Name":"Actor.scala","Line Number":465},{"Declaring Class":"org.apache.spark.scheduler.DAGSchedulerEventProcessActor","Method Name":"aroundReceive","File Name":"DAGScheduler.scala","Line Number":1375},{"Declaring Class":"akka.actor.ActorCell","Method Name":"receiveMessage","File Name":"ActorCell.scala","Line Number":516},{"Declaring Class":"akka.actor.ActorCell","Method Name":"invoke","File Name":"ActorCell.scala","Line Number":487},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"processMailbox","File Name":"Mailbox.scala","Line Number":238},{"Declaring Class":"akka.dispatch.Mailbox","Method Name":"run","File Name":"Mailbox.scala","Line Number":220},{"Declaring Class":"akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask","Method Name":"exec","File Name":"AbstractDispatcher.scala","Line Number":393},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinTask","Method Name":"doExec","File Name":"ForkJoinTask.java","Line Number":260},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool$WorkQueue","Method Name":"runTask","File Name":"ForkJoinPool.java","Line Number":1339},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinPool","Method Name":"runWorker","File Name":"ForkJoinPool.java","Line Number":1979},{"Declaring Class":"scala.concurrent.forkjoin.ForkJoinWorkerThread","Method Name":"run","File Name":"ForkJoinWorkerThread.java","Line Number":107}]}}} {"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":3,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":1,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787222,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":0,"Attempt":0,"Launch Time":1422981787191,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":20,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":2,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":4,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":7,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787223,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":5,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":5,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":3,"Attempt":0,"Launch Time":1422981787192,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787224,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":6,"Executor Run Time":19,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":4,"Attempt":0,"Launch Time":1422981787193,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":22,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":3,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":6,"Attempt":0,"Launch Time":1422981787194,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1422981787225,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":3,"Executor Run Time":21,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":6,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums_2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line19.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line19.$read$$iwC$$iwC$$iwC.(:22)\n$line19.$read$$iwC$$iwC.(:24)\n$line19.$read$$iwC.(:26)\n$line19.$read.(:28)\n$line19.$read$.(:32)\n$line19.$read$.()\n$line19.$eval$.(:7)\n$line19.$eval$.()\n$line19.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1422981787191,"Completion Time":1422981787226,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerApplicationEnd","Timestamp":1422981788731} diff --git a/core/src/test/resources/spark-events/local-1425081759269 b/core/src/test/resources/spark-events/local-1425081759269 index 9745b36b09e44..9f7d0a7a4105a 100755 --- a/core/src/test/resources/spark-events/local-1425081759269 +++ b/core/src/test/resources/spark-events/local-1425081759269 @@ -1,88 +1,88 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":57967},"Maximum Memory":278302556,"Timestamp":1425081759407} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","Java Version":"1.7.0_67 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.103","spark.eventLog.enabled":"true","spark.driver.port":"57965","spark.repl.class.uri":"http://192.168.1.103:57964","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.103:57966","spark.tachyonStore.folderName":"spark-fd6c823a-8a18-4113-8306-1fa7bb623a7f","spark.app.id":"local-1425081759269"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.7","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.7","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"24.65-b04","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.7.0_67-b01","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"51.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.7.0_67.jdk/Contents/Home/jre","java.version":"1.7.0_67","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1425081759269","Timestamp":1425081758277,"User":"irashid"} {"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762632,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_6","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762633,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":520,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_2","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1425081762082,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_3","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1425081762084,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762634,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_5","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1425081762083,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762635,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":10,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_4","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1425081762075,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_0","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1425081762085,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762636,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":9,"Executor Run Time":521,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_7","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1425081762081,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081762637,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":11,"Executor Run Time":522,"Result Size":1268,"JVM GC Time":20,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Updated Blocks":[{"Block ID":"rdd_0_1","Status":{"Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Memory Size":3500016,"Tachyon Size":0,"Disk Size":0}}]}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"count at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":8,"Memory Size":28000128,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081762069,"Completion Time":1425081762637,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":1,"Stage Infos":[{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[1,2]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} -{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":1,"Stage Attempt ID":0,"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":10,"Index":2,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764001,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":138000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":9,"Index":1,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":106000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":12,"Index":4,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764002,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1645,"Shuffle Write Time":99000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":8,"Index":0,"Attempt":0,"Launch Time":1425081763578,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":123000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":15,"Index":7,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764003,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":406,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":108000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":11,"Index":3,"Attempt":0,"Launch Time":1425081763579,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1647,"Shuffle Write Time":97000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":14,"Index":6,"Attempt":0,"Launch Time":1425081763581,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764004,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":132000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} +{"Event":"SparkListenerTaskEnd","Stage ID":1,"Stage Attempt ID":0,"Task Type":"ShuffleMapTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":13,"Index":5,"Attempt":0,"Launch Time":1425081763580,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764005,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":407,"Result Size":1902,"JVM GC Time":18,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Write Metrics":{"Shuffle Bytes Written":1648,"Shuffle Write Time":81000},"Input Metrics":{"Data Read Method":"Memory","Bytes Read":3500016}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":1,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081763578,"Completion Time":1425081764005,"Accumulables":[]}} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskStart","Stage ID":2,"Stage Attempt ID":0,"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":2,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":16,"Index":0,"Attempt":0,"Launch Time":1425081764014,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":0,"Executor Run Time":28,"Result Size":1013,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":2,"Stage Attempt ID":0,"Stage Name":"first at :17","Number of Tasks":1,"RDD Info":[{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.first(RDD.scala:1093)\n$line11.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line11.$read$$iwC$$iwC$$iwC.(:22)\n$line11.$read$$iwC$$iwC.(:24)\n$line11.$read$$iwC.(:26)\n$line11.$read.(:28)\n$line11.$read$.(:32)\n$line11.$read$.()\n$line11.$eval$.(:7)\n$line11.$eval$.()\n$line11.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764014,"Completion Time":1425081764045,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":1,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":2,"Stage Infos":[{"Stage ID":3,"Stage Attempt ID":0,"Stage Name":"map at :14","Number of Tasks":8,"RDD Info":[{"RDD ID":1,"Name":"1","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":true,"Use Tachyon":false,"Deserialized":true,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.map(RDD.scala:271)\n$line10.$read$$iwC$$iwC$$iwC$$iwC.(:14)\n$line10.$read$$iwC$$iwC$$iwC.(:19)\n$line10.$read$$iwC$$iwC.(:21)\n$line10.$read$$iwC.(:23)\n$line10.$read.(:25)\n$line10.$read$.(:29)\n$line10.$read$.()\n$line10.$eval$.(:7)\n$line10.$eval$.()\n$line10.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]},{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[3,4]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} -{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":4,"Stage Attempt ID":0,"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":24,"Index":7,"Attempt":0,"Launch Time":1425081764399,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764642,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":19,"Index":2,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":21,"Index":4,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764643,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":240,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":22,"Index":5,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":23,"Index":6,"Attempt":0,"Launch Time":1425081764398,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764645,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":241,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":17,"Index":0,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":18,"Index":1,"Attempt":0,"Launch Time":1425081764396,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764646,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":243,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} +{"Event":"SparkListenerTaskEnd","Stage ID":4,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":20,"Index":3,"Attempt":0,"Launch Time":1425081764397,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081764648,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":247,"Result Size":824,"JVM GC Time":31,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Remote Blocks Fetched":0,"Local Blocks Fetched":8,"Fetch Wait Time":0,"Remote Bytes Read":0}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":4,"Stage Attempt ID":0,"Stage Name":"saveAsTextFile at :19","Number of Tasks":8,"RDD Info":[{"RDD ID":3,"Name":"3","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":2,"Name":"2","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.saveAsTextFile(RDD.scala:1164)\n$line13.$read$$iwC$$iwC$$iwC$$iwC.(:19)\n$line13.$read$$iwC$$iwC$$iwC.(:24)\n$line13.$read$$iwC$$iwC.(:26)\n$line13.$read$$iwC.(:28)\n$line13.$read.(:30)\n$line13.$read$.(:34)\n$line13.$read$.()\n$line13.$eval$.(:7)\n$line13.$eval$.()\n$line13.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081764396,"Completion Time":1425081764648,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":2,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerJobStart","Job ID":3,"Stage Infos":[{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[5]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} -{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":5,"Stage Attempt ID":0,"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":27,"Index":2,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765045,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":30,"Index":5,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":25,"Index":0,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765046,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":29,"Index":4,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":28,"Index":3,"Attempt":0,"Launch Time":1425081765027,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765047,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":32,"Index":7,"Attempt":0,"Launch Time":1425081765029,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":1,"Executor Run Time":13,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":31,"Index":6,"Attempt":0,"Launch Time":1425081765028,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765048,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":14,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":16}}} +{"Event":"SparkListenerTaskEnd","Stage ID":5,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":26,"Index":1,"Attempt":0,"Launch Time":1425081765026,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1425081765049,"Failed":false,"Accumulables":[]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":2,"Executor Run Time":12,"Result Size":1812,"JVM GC Time":0,"Result Serialization Time":0,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0,"Input Metrics":{"Data Read Method":"Hadoop","Bytes Read":32}}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":5,"Stage Attempt ID":0,"Stage Name":"count at :17","Number of Tasks":8,"RDD Info":[{"RDD ID":5,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0},{"RDD ID":4,"Name":"/Users/irashid/spark-examples/tmp_data/sums","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.count(RDD.scala:910)\n$line15.$read$$iwC$$iwC$$iwC$$iwC.(:17)\n$line15.$read$$iwC$$iwC$$iwC.(:22)\n$line15.$read$$iwC$$iwC.(:24)\n$line15.$read$$iwC.(:26)\n$line15.$read.(:28)\n$line15.$read$.(:32)\n$line15.$read$.()\n$line15.$eval$.(:7)\n$line15.$eval$.()\n$line15.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:606)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1425081765026,"Completion Time":1425081765050,"Accumulables":[]}} {"Event":"SparkListenerJobEnd","Job ID":3,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerApplicationEnd","Timestamp":1425081766912} diff --git a/core/src/test/resources/spark-events/local-1426533911241 b/core/src/test/resources/spark-events/local-1426533911241 index 9ef5bd5d92de5..60ba9e12e8ef3 100755 --- a/core/src/test/resources/spark-events/local-1426533911241 +++ b/core/src/test/resources/spark-events/local-1426533911241 @@ -1,24 +1,24 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426533911361} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426533911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426533910242,"User":"irashid","App Attempt ID":"1"} {"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426533936521,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426533936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426533936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426533936524,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426533936515,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426533936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426533936103,"Completion Time":1426533936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerApplicationEnd","Timestamp":1426533945177} diff --git a/core/src/test/resources/spark-events/local-1426633911242 b/core/src/test/resources/spark-events/local-1426633911242 index e7043282107d3..886f9a9f6cd89 100755 --- a/core/src/test/resources/spark-events/local-1426633911242 +++ b/core/src/test/resources/spark-events/local-1426633911242 @@ -1,24 +1,24 @@ -{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361} +{"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"driver","Host":"localhost","Port":58610},"Maximum Memory":278019440,"Timestamp":1426633911361} {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"Java Home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","Java Version":"1.8.0_25 (Oracle Corporation)","Scala Version":"version 2.10.4"},"Spark Properties":{"spark.driver.host":"192.168.1.105","spark.eventLog.enabled":"true","spark.driver.port":"58608","spark.repl.class.uri":"http://192.168.1.105:58607","spark.jars":"","spark.app.name":"Spark shell","spark.scheduler.mode":"FIFO","spark.executor.id":"driver","spark.master":"local[*]","spark.fileserver.uri":"http://192.168.1.105:58609","spark.tachyonStore.folderName":"spark-5e9b7f26-8e97-4b43-82d6-25c141530da9","spark.app.id":"local-1426633911241"},"System Properties":{"java.io.tmpdir":"/var/folders/36/m29jw1z95qv4ywb1c4n0rz000000gp/T/","line.separator":"\n","path.separator":":","sun.management.compiler":"HotSpot 64-Bit Tiered Compilers","SPARK_SUBMIT":"true","sun.cpu.endian":"little","java.specification.version":"1.8","java.vm.specification.name":"Java Virtual Machine Specification","java.vendor":"Oracle Corporation","java.vm.specification.version":"1.8","user.home":"/Users/irashid","file.encoding.pkg":"sun.io","sun.nio.ch.bugLevel":"","ftp.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","sun.arch.data.model":"64","sun.boot.library.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib","user.dir":"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4","java.library.path":"/Users/irashid/Library/Java/Extensions:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java:.","sun.cpu.isalist":"","os.arch":"x86_64","java.vm.version":"25.25-b02","java.endorsed.dirs":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/endorsed","java.runtime.version":"1.8.0_25-b17","java.vm.info":"mixed mode","java.ext.dirs":"/Users/irashid/Library/Java/Extensions:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/ext:/Library/Java/Extensions:/Network/Library/Java/Extensions:/System/Library/Java/Extensions:/usr/lib/java","java.runtime.name":"Java(TM) SE Runtime Environment","file.separator":"/","java.class.version":"52.0","java.specification.name":"Java Platform API Specification","sun.boot.class.path":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/resources.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/rt.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/sunrsasign.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jsse.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jce.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/charsets.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/lib/jfr.jar:/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre/classes","file.encoding":"UTF-8","user.timezone":"America/Chicago","java.specification.vendor":"Oracle Corporation","sun.java.launcher":"SUN_STANDARD","os.version":"10.9.5","sun.os.patch.level":"unknown","gopherProxySet":"false","java.vm.specification.vendor":"Oracle Corporation","user.country":"US","sun.jnu.encoding":"UTF-8","http.nonProxyHosts":"local|*.local|169.254/16|*.169.254/16","user.language":"en","socksNonProxyHosts":"local|*.local|169.254/16|*.169.254/16","java.vendor.url":"http://java.oracle.com/","java.awt.printerjob":"sun.lwawt.macosx.CPrinterJob","java.awt.graphicsenv":"sun.awt.CGraphicsEnvironment","awt.toolkit":"sun.lwawt.macosx.LWCToolkit","os.name":"Mac OS X","java.vm.vendor":"Oracle Corporation","java.vendor.url.bug":"http://bugreport.sun.com/bugreport/","user.name":"irashid","java.vm.name":"Java HotSpot(TM) 64-Bit Server VM","sun.java.command":"org.apache.spark.deploy.SparkSubmit --class org.apache.spark.repl.Main --conf spark.eventLog.enabled=true spark-shell","java.home":"/Library/Java/JavaVirtualMachines/jdk1.8.0_25.jdk/Contents/Home/jre","java.version":"1.8.0_25","sun.io.unicode.encoding":"UnicodeBig"},"Classpath Entries":{"/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/spark-assembly-1.2.0-hadoop2.4.0.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-api-jdo-3.2.6.jar":"System Classpath","/etc/hadoop":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-rdbms-3.2.9.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/lib/datanucleus-core-3.2.10.jar":"System Classpath","/Users/irashid/spark-examples/releases/spark-1.2.0-bin-hadoop2.4/conf":"System Classpath"}} {"Event":"SparkListenerApplicationStart","App Name":"Spark shell","App ID":"local-1426533911241","Timestamp":1426633910242,"User":"irashid","App Attempt ID":"2"} {"Event":"SparkListenerJobStart","Job ID":0,"Stage Infos":[{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}],"Stage IDs":[0]} {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} -{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskStart","Stage ID":0,"Stage Attempt ID":0,"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":0,"Failed":false,"Accumulables":[]}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":2,"Index":2,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936570,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"378","Value":"378"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":3,"Index":3,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936572,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"572","Value":"950"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":13,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":6,"Index":6,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"978","Value":"1928"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":1,"Index":1,"Attempt":0,"Launch Time":1426633936521,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"247","Value":"2175"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":4,"Index":4,"Attempt":0,"Launch Time":1426633936522,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936574,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"678","Value":"2853"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":1,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":5,"Index":5,"Attempt":0,"Launch Time":1426633936523,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"897","Value":"3750"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":7,"Index":7,"Attempt":0,"Launch Time":1426633936524,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936575,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"1222","Value":"4972"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":12,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} +{"Event":"SparkListenerTaskEnd","Stage ID":0,"Stage Attempt ID":0,"Task Type":"ResultTask","Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":0,"Index":0,"Attempt":0,"Launch Time":1426633936515,"Executor ID":"driver","Host":"localhost","Locality":"PROCESS_LOCAL","Speculative":false,"Getting Result Time":0,"Finish Time":1426633936576,"Failed":false,"Accumulables":[{"ID":1,"Name":"my counter","Update":"78","Value":"5050"}]},"Task Metrics":{"Host Name":"localhost","Executor Deserialize Time":14,"Executor Run Time":15,"Result Size":697,"JVM GC Time":0,"Result Serialization Time":2,"Memory Bytes Spilled":0,"Disk Bytes Spilled":0}} {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":0,"Stage Attempt ID":0,"Stage Name":"foreach at :15","Number of Tasks":8,"RDD Info":[{"RDD ID":0,"Name":"0","Storage Level":{"Use Disk":false,"Use Memory":false,"Use Tachyon":false,"Deserialized":false,"Replication":1},"Number of Partitions":8,"Number of Cached Partitions":0,"Memory Size":0,"Tachyon Size":0,"Disk Size":0}],"Details":"org.apache.spark.rdd.RDD.foreach(RDD.scala:765)\n$line9.$read$$iwC$$iwC$$iwC$$iwC.(:15)\n$line9.$read$$iwC$$iwC$$iwC.(:20)\n$line9.$read$$iwC$$iwC.(:22)\n$line9.$read$$iwC.(:24)\n$line9.$read.(:26)\n$line9.$read$.(:30)\n$line9.$read$.()\n$line9.$eval$.(:7)\n$line9.$eval$.()\n$line9.$eval.$print()\nsun.reflect.NativeMethodAccessorImpl.invoke0(Native Method)\nsun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62)\nsun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43)\njava.lang.reflect.Method.invoke(Method.java:483)\norg.apache.spark.repl.SparkIMain$ReadEvalPrint.call(SparkIMain.scala:852)\norg.apache.spark.repl.SparkIMain$Request.loadAndRun(SparkIMain.scala:1125)\norg.apache.spark.repl.SparkIMain.loadAndRunReq$1(SparkIMain.scala:674)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:705)\norg.apache.spark.repl.SparkIMain.interpret(SparkIMain.scala:669)","Submission Time":1426633936103,"Completion Time":1426633936579,"Accumulables":[{"ID":1,"Name":"my counter","Value":"5050"}]}} {"Event":"SparkListenerJobEnd","Job ID":0,"Job Result":{"Result":"JobSucceeded"}} {"Event":"SparkListenerApplicationEnd","Timestamp":1426633945177} diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 5d0ffd92647bc..435665d8a1ce2 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -28,7 +28,6 @@ import scala.util.control.NonFatal import org.scalatest.Matchers import org.scalatest.exceptions.TestFailedException -import org.apache.spark.AccumulatorParam.StringAccumulatorParam import org.apache.spark.scheduler._ import org.apache.spark.serializer.JavaSerializer import org.apache.spark.util.{AccumulatorContext, AccumulatorMetadata, AccumulatorV2, LongAccumulator} @@ -45,21 +44,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex } } - implicit def setAccum[A]: AccumulableParam[mutable.Set[A], A] = - new AccumulableParam[mutable.Set[A], A] { - def addInPlace(t1: mutable.Set[A], t2: mutable.Set[A]) : mutable.Set[A] = { - t1 ++= t2 - t1 - } - def addAccumulator(t1: mutable.Set[A], t2: A) : mutable.Set[A] = { - t1 += t2 - t1 - } - def zero(t: mutable.Set[A]) : mutable.Set[A] = { - new mutable.HashSet[A]() - } - } - test("accumulator serialization") { val ser = new JavaSerializer(new SparkConf).newInstance() val acc = createLongAccum("x") @@ -81,122 +65,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(acc3.isAtDriverSide) } - test ("basic accumulation") { - sc = new SparkContext("local", "test") - val acc: Accumulator[Int] = sc.accumulator(0) - - val d = sc.parallelize(1 to 20) - d.foreach{x => acc += x} - acc.value should be (210) - - val longAcc = sc.accumulator(0L) - val maxInt = Integer.MAX_VALUE.toLong - d.foreach{x => longAcc += maxInt + x} - longAcc.value should be (210L + maxInt * 20) - } - - test("value not assignable from tasks") { - sc = new SparkContext("local", "test") - val acc: Accumulator[Int] = sc.accumulator(0) - - val d = sc.parallelize(1 to 20) - intercept[SparkException] { - d.foreach(x => acc.value = x) - } - } - - test ("add value to collection accumulators") { - val maxI = 1000 - for (nThreads <- List(1, 10)) { // test single & multi-threaded - sc = new SparkContext("local[" + nThreads + "]", "test") - val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val d = sc.parallelize(1 to maxI) - d.foreach { - x => acc += x - } - val v = acc.value.asInstanceOf[mutable.Set[Int]] - for (i <- 1 to maxI) { - v should contain(i) - } - resetSparkContext() - } - } - - test("value not readable in tasks") { - val maxI = 1000 - for (nThreads <- List(1, 10)) { // test single & multi-threaded - sc = new SparkContext("local[" + nThreads + "]", "test") - val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val d = sc.parallelize(1 to maxI) - an [SparkException] should be thrownBy { - d.foreach { - x => acc.value += x - } - } - resetSparkContext() - } - } - - test ("collection accumulators") { - val maxI = 1000 - for (nThreads <- List(1, 10)) { - // test single & multi-threaded - sc = new SparkContext("local[" + nThreads + "]", "test") - val setAcc = sc.accumulableCollection(mutable.HashSet[Int]()) - val bufferAcc = sc.accumulableCollection(mutable.ArrayBuffer[Int]()) - val mapAcc = sc.accumulableCollection(mutable.HashMap[Int, String]()) - val d = sc.parallelize((1 to maxI) ++ (1 to maxI)) - d.foreach { - x => {setAcc += x; bufferAcc += x; mapAcc += (x -> x.toString)} - } - - // Note that this is typed correctly -- no casts necessary - setAcc.value.size should be (maxI) - bufferAcc.value.size should be (2 * maxI) - mapAcc.value.size should be (maxI) - for (i <- 1 to maxI) { - setAcc.value should contain(i) - bufferAcc.value should contain(i) - mapAcc.value should contain (i -> i.toString) - } - resetSparkContext() - } - } - - test ("localValue readable in tasks") { - val maxI = 1000 - for (nThreads <- List(1, 10)) { // test single & multi-threaded - sc = new SparkContext("local[" + nThreads + "]", "test") - val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val groupedInts = (1 to (maxI/20)).map {x => (20 * (x - 1) to 20 * x).toSet} - val d = sc.parallelize(groupedInts) - d.foreach { - x => acc.localValue ++= x - } - acc.value should be ((0 to maxI).toSet) - resetSparkContext() - } - } - - test ("garbage collection") { - // Create an accumulator and let it go out of scope to test that it's properly garbage collected - sc = new SparkContext("local", "test") - var acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) - val accId = acc.id - val ref = WeakReference(acc) - - // Ensure the accumulator is present - assert(ref.get.isDefined) - - // Remove the explicit reference to it and allow weak reference to get garbage collected - acc = null - System.gc() - assert(ref.get.isEmpty) - - AccumulatorContext.remove(accId) - assert(!AccumulatorContext.get(accId).isDefined) - } - test("get accum") { // Don't register with SparkContext for cleanup var acc = createLongAccum("a") @@ -221,20 +89,6 @@ class AccumulatorSuite extends SparkFunSuite with Matchers with LocalSparkContex assert(AccumulatorContext.get(100000).isEmpty) } - test("string accumulator param") { - val acc = new Accumulator("", StringAccumulatorParam, Some("darkness")) - assert(acc.value === "") - acc.setValue("feeds") - assert(acc.value === "feeds") - acc.add("your") - assert(acc.value === "your") // value is overwritten, not concatenated - acc += "soul" - assert(acc.value === "soul") - acc ++= "with" - assert(acc.value === "with") - acc.merge("kindness") - assert(acc.value === "kindness") - } } private[spark] object AccumulatorSuite { @@ -256,7 +110,7 @@ private[spark] object AccumulatorSuite { } /** - * Make an `AccumulableInfo` out of an [[Accumulable]] with the intent to use the + * Make an `AccumulableInfo` out of an `AccumulatorV2` with the intent to use the * info as an accumulator update. */ def makeInfo(a: AccumulatorV2[_, _]): AccumulableInfo = a.toInfo(Some(a.value), None) diff --git a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala index d49ab4aa7df12..435b927068e60 100644 --- a/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala +++ b/core/src/test/scala/org/apache/spark/BarrierStageOnSubmittedSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark import scala.concurrent.duration._ -import scala.language.postfixOps +import org.apache.spark.internal.config._ import org.apache.spark.rdd.{PartitionPruningRDD, RDD} import org.apache.spark.scheduler.BarrierJobAllocationFailed._ -import org.apache.spark.scheduler.DAGScheduler import org.apache.spark.util.ThreadUtils /** @@ -52,7 +51,7 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext ) val error = intercept[SparkException] { - ThreadUtils.awaitResult(futureAction, 5 seconds) + ThreadUtils.awaitResult(futureAction, 5.seconds) }.getCause.getMessage assert(error.contains(message)) } @@ -157,8 +156,8 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext test("submit a barrier ResultStage with dynamic resource allocation enabled") { val conf = new SparkConf() - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.testing", "true") + .set(DYN_ALLOCATION_ENABLED, true) + .set(DYN_ALLOCATION_TESTING, true) .setMaster("local[4]") .setAppName("test") sc = createSparkContext(Some(conf)) @@ -172,8 +171,8 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext test("submit a barrier ShuffleMapStage with dynamic resource allocation enabled") { val conf = new SparkConf() - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.testing", "true") + .set(DYN_ALLOCATION_ENABLED, true) + .set(DYN_ALLOCATION_TESTING, true) .setMaster("local[4]") .setAppName("test") sc = createSparkContext(Some(conf)) @@ -191,9 +190,9 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext "mode") { val conf = new SparkConf() // Shorten the time interval between two failed checks to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.interval", "1s") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL.key, "1s") // Reduce max check failures allowed to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures", "3") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 3) .setMaster("local[4]") .setAppName("test") sc = createSparkContext(Some(conf)) @@ -208,9 +207,9 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext "local mode") { val conf = new SparkConf() // Shorten the time interval between two failed checks to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.interval", "1s") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL.key, "1s") // Reduce max check failures allowed to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures", "3") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 3) .setMaster("local[4]") .setAppName("test") sc = createSparkContext(Some(conf)) @@ -226,11 +225,11 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext test("submit a barrier ResultStage that requires more slots than current total under " + "local-cluster mode") { val conf = new SparkConf() - .set("spark.task.cpus", "2") + .set(CPUS_PER_TASK, 2) // Shorten the time interval between two failed checks to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.interval", "1s") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL.key, "1s") // Reduce max check failures allowed to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures", "3") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 3) .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = createSparkContext(Some(conf)) @@ -244,11 +243,11 @@ class BarrierStageOnSubmittedSuite extends SparkFunSuite with LocalSparkContext test("submit a barrier ShuffleMapStage that requires more slots than current total under " + "local-cluster mode") { val conf = new SparkConf() - .set("spark.task.cpus", "2") + .set(CPUS_PER_TASK, 2) // Shorten the time interval between two failed checks to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.interval", "1s") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_INTERVAL.key, "1s") // Reduce max check failures allowed to make the test fail faster. - .set("spark.scheduler.barrier.maxConcurrentTasksCheck.maxFailures", "3") + .set(BARRIER_MAX_CONCURRENT_TASKS_CHECK_MAX_FAILURES, 3) .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = createSparkContext(Some(conf)) diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index 48408ccc8f81b..3a43f1a033da1 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -24,6 +24,7 @@ import scala.reflect.ClassTag import com.google.common.io.ByteStreams import org.apache.hadoop.fs.Path +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.rdd._ import org.apache.spark.storage.{BlockId, StorageLevel, TestBlockId} @@ -510,8 +511,8 @@ class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalS assert(rdd.isCheckpointed === false) assert(rdd.isCheckpointedAndMaterialized === false) assert(rdd.count() === 0) - assert(rdd.isCheckpointed === true) - assert(rdd.isCheckpointedAndMaterialized === true) + assert(rdd.isCheckpointed) + assert(rdd.isCheckpointedAndMaterialized) assert(rdd.partitions.size === 0) } @@ -530,7 +531,7 @@ class CheckpointSuite extends SparkFunSuite with RDDCheckpointTester with LocalS checkpoint(rdd2, reliableCheckpoint) rdd2.count() assert(rdd1.isCheckpointed === checkpointAllMarkedAncestors) - assert(rdd2.isCheckpointed === true) + assert(rdd2.isCheckpointed) } finally { sc.setLocalProperty(RDD.CHECKPOINT_ALL_MARKED_ANCESTORS, null) } @@ -586,11 +587,10 @@ object CheckpointSuite { class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { test("checkpoint compression") { - val checkpointDir = Utils.createTempDir() - try { + withTempDir { checkpointDir => val conf = new SparkConf() .set("spark.checkpoint.compress", "true") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext("local", "test", conf) sc.setCheckpointDir(checkpointDir.toString) val rdd = sc.makeRDD(1 to 20, numSlices = 1) @@ -616,8 +616,6 @@ class CheckpointCompressionSuite extends SparkFunSuite with LocalSparkContext { // Verify that the compressed content can be read back assert(rdd.collect().toSeq === (1 to 20)) - } finally { - Utils.deleteRecursively(checkpointDir) } } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index 6724af952505f..6a30a1d32f8c6 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -18,9 +18,9 @@ package org.apache.spark import java.lang.ref.WeakReference +import java.util.concurrent.TimeUnit import scala.collection.mutable.HashSet -import scala.language.existentials import scala.util.Random import org.scalatest.BeforeAndAfter @@ -28,11 +28,11 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration import org.scalatest.time.SpanSugar._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} +import org.apache.spark.internal.config._ import org.apache.spark.rdd.{RDD, ReliableRDDCheckpointData} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.storage._ -import org.apache.spark.util.Utils /** * An abstract base class for context cleaner tests, which sets up a context with a config @@ -42,14 +42,14 @@ import org.apache.spark.util.Utils abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[SortShuffleManager]) extends SparkFunSuite with BeforeAndAfter with LocalSparkContext { - implicit val defaultTimeout = timeout(10000 millis) + implicit val defaultTimeout = timeout(10.seconds) val conf = new SparkConf() .setMaster("local[2]") .setAppName("ContextCleanerSuite") - .set("spark.cleaner.referenceTracking.blocking", "true") - .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") - .set("spark.cleaner.referenceTracking.cleanCheckpoints", "true") - .set("spark.shuffle.manager", shuffleManager.getName) + .set(CLEANER_REFERENCE_TRACKING_BLOCKING, true) + .set(CLEANER_REFERENCE_TRACKING_BLOCKING_SHUFFLE, true) + .set(CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS, true) + .set(config.SHUFFLE_MANAGER, shuffleManager.getName) before { sc = new SparkContext(conf) @@ -69,10 +69,11 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[So protected def newShuffleRDD() = newPairRDD().reduceByKey(_ + _) protected def newBroadcast() = sc.broadcast(1 to 100) - protected def newRDDWithShuffleDependencies(): (RDD[_], Seq[ShuffleDependency[_, _, _]]) = { - def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + protected def newRDDWithShuffleDependencies(): + (RDD[(Int, Int)], Seq[ShuffleDependency[Int, Int, Int]]) = { + def getAllDependencies(rdd: RDD[(Int, Int)]): Seq[Dependency[_]] = { rdd.dependencies ++ rdd.dependencies.flatMap { dep => - getAllDependencies(dep.rdd) + getAllDependencies(dep.rdd.asInstanceOf[RDD[(Int, Int)]]) } } val rdd = newShuffleRDD() @@ -80,7 +81,7 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[So // Get all the shuffle dependencies val shuffleDeps = getAllDependencies(rdd) .filter(_.isInstanceOf[ShuffleDependency[_, _, _]]) - .map(_.asInstanceOf[ShuffleDependency[_, _, _]]) + .map(_.asInstanceOf[ShuffleDependency[Int, Int, Int]]) (rdd, shuffleDeps) } @@ -98,10 +99,10 @@ abstract class ContextCleanerSuiteBase(val shuffleManager: Class[_] = classOf[So /** Run GC and make sure it actually has run */ protected def runGC() { val weakRef = new WeakReference(new Object()) - val startTime = System.currentTimeMillis + val startTimeNs = System.nanoTime() System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. // Wait until a weak reference object has been GCed - while (System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + while (System.nanoTime() - startTimeNs < TimeUnit.SECONDS.toNanos(10) && weakRef.get != null) { System.gc() Thread.sleep(200) } @@ -158,7 +159,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC causes RDD cleanup after dereferencing the RDD @@ -177,7 +178,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } rdd.count() // Defeat early collection by the JVM @@ -195,7 +196,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC causes broadcast cleanup after dereferencing the broadcast variable @@ -207,54 +208,55 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { } test("automatically cleanup normal checkpoint") { - val checkpointDir = Utils.createTempDir() - checkpointDir.delete() - var rdd = newPairRDD() - sc.setCheckpointDir(checkpointDir.toString) - rdd.checkpoint() - rdd.cache() - rdd.collect() - var rddId = rdd.id - - // Confirm the checkpoint directory exists - assert(ReliableRDDCheckpointData.checkpointPath(sc, rddId).isDefined) - val path = ReliableRDDCheckpointData.checkpointPath(sc, rddId).get - val fs = path.getFileSystem(sc.hadoopConfiguration) - assert(fs.exists(path)) - - // the checkpoint is not cleaned by default (without the configuration set) - var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) - rdd = null // Make RDD out of scope, ok if collected earlier - runGC() - postGCTester.assertCleanup() - assert(!fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) - - // Verify that checkpoints are NOT cleaned up if the config is not enabled - sc.stop() - val conf = new SparkConf() - .setMaster("local[2]") - .setAppName("cleanupCheckpoint") - .set("spark.cleaner.referenceTracking.cleanCheckpoints", "false") - sc = new SparkContext(conf) - rdd = newPairRDD() - sc.setCheckpointDir(checkpointDir.toString) - rdd.checkpoint() - rdd.cache() - rdd.collect() - rddId = rdd.id - - // Confirm the checkpoint directory exists - assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) - - // Reference rdd to defeat any early collection by the JVM - rdd.count() - - // Test that GC causes checkpoint data cleanup after dereferencing the RDD - postGCTester = new CleanerTester(sc, Seq(rddId)) - rdd = null // Make RDD out of scope - runGC() - postGCTester.assertCleanup() - assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + withTempDir { checkpointDir => + checkpointDir.delete() + var rdd = newPairRDD() + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + var rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(ReliableRDDCheckpointData.checkpointPath(sc, rddId).isDefined) + val path = ReliableRDDCheckpointData.checkpointPath(sc, rddId).get + val fs = path.getFileSystem(sc.hadoopConfiguration) + assert(fs.exists(path)) + + // the checkpoint is not cleaned by default (without the configuration set) + var postGCTester = new CleanerTester(sc, Seq(rddId), Nil, Nil, Seq(rddId)) + rdd = null // Make RDD out of scope, ok if collected earlier + runGC() + postGCTester.assertCleanup() + assert(!fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + + // Verify that checkpoints are NOT cleaned up if the config is not enabled + sc.stop() + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName("cleanupCheckpoint") + .set(CLEANER_REFERENCE_TRACKING_CLEAN_CHECKPOINTS, false) + sc = new SparkContext(conf) + rdd = newPairRDD() + sc.setCheckpointDir(checkpointDir.toString) + rdd.checkpoint() + rdd.cache() + rdd.collect() + rddId = rdd.id + + // Confirm the checkpoint directory exists + assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + + // Reference rdd to defeat any early collection by the JVM + rdd.count() + + // Test that GC causes checkpoint data cleanup after dereferencing the RDD + postGCTester = new CleanerTester(sc, Seq(rddId)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + assert(fs.exists(ReliableRDDCheckpointData.checkpointPath(sc, rddId).get)) + } } test("automatically clean up local checkpoint") { @@ -270,7 +272,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } // Test that RDD going out of scope does cause the checkpoint blocks to be cleaned up @@ -292,7 +294,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC triggers the cleanup of all variables after the dereferencing them @@ -316,9 +318,9 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val conf2 = new SparkConf() .setMaster("local-cluster[2, 1, 1024]") .setAppName("ContextCleanerSuite") - .set("spark.cleaner.referenceTracking.blocking", "true") - .set("spark.cleaner.referenceTracking.blocking.shuffle", "true") - .set("spark.shuffle.manager", shuffleManager.getName) + .set(CLEANER_REFERENCE_TRACKING_BLOCKING, true) + .set(CLEANER_REFERENCE_TRACKING_BLOCKING_SHUFFLE, true) + .set(config.SHUFFLE_MANAGER, shuffleManager.getName) sc = new SparkContext(conf2) val numRdds = 10 @@ -332,7 +334,7 @@ class ContextCleanerSuite extends ContextCleanerSuiteBase { val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) runGC() intercept[Exception] { - preGCTester.assertCleanup()(timeout(1000 millis)) + preGCTester.assertCleanup()(timeout(1.second)) } // Test that GC triggers the cleanup of all variables after the dereferencing them @@ -406,7 +408,7 @@ class CleanerTester( /** Assert that all the stuff has been cleaned up */ def assertCleanup()(implicit waitTimeout: PatienceConfiguration.Timeout) { try { - eventually(waitTimeout, interval(100 millis)) { + eventually(waitTimeout, interval(100.milliseconds)) { assert(isAllCleanedUp, "The following resources were not cleaned up:\n" + uncleanedResourcesToString) } diff --git a/core/src/test/scala/org/apache/spark/DistributedSuite.scala b/core/src/test/scala/org/apache/spark/DistributedSuite.scala index 629a323042ff2..aad20545bafbe 100644 --- a/core/src/test/scala/org/apache/spark/DistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/DistributedSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.time.{Millis, Span} import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ import org.apache.spark.security.EncryptionFunSuite import org.apache.spark.storage.{RDDBlockId, StorageLevel} import org.apache.spark.util.io.ChunkedByteBuffer @@ -86,7 +87,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex } test("groupByKey where map output sizes exceed maxMbInFlight") { - val conf = new SparkConf().set("spark.reducer.maxSizeInFlight", "1m") + val conf = new SparkConf().set(config.REDUCER_MAX_SIZE_IN_FLIGHT.key, "1m") sc = new SparkContext(clusterUrl, "test", conf) // This data should be around 20 MB, so even with 4 mappers and 2 reducers, each map output // file should be about 2.5 MB @@ -195,7 +196,7 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex new ChunkedByteBuffer(bytes.nioByteBuffer()).toInputStream())(data.elementClassTag).toList assert(deserialized === (1 to 100).toList) } - // This will exercise the getRemoteBytes / getRemoteValues code paths: + // This will exercise the getRemoteValues code path: assert(blockIds.flatMap(id => blockManager.get[Int](id).get.data).toSet === (1 to 1000).toSet) } @@ -216,8 +217,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex test("compute without caching when no partitions fit in memory") { val size = 10000 val conf = new SparkConf() - .set("spark.storage.unrollMemoryThreshold", "1024") - .set("spark.testing.memory", (size / 2).toString) + .set(config.STORAGE_UNROLL_MEMORY_THRESHOLD, 1024L) + .set(TEST_MEMORY, size.toLong / 2) + sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, 2).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) @@ -232,8 +234,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val size = 10000 val numPartitions = 20 val conf = new SparkConf() - .set("spark.storage.unrollMemoryThreshold", "1024") - .set("spark.testing.memory", size.toString) + .set(config.STORAGE_UNROLL_MEMORY_THRESHOLD, 1024L) + .set(TEST_MEMORY, size.toLong) + sc = new SparkContext(clusterUrl, "test", conf) val data = sc.parallelize(1 to size, numPartitions).persist(StorageLevel.MEMORY_ONLY) assert(data.count() === size) @@ -319,9 +322,9 @@ class DistributedSuite extends SparkFunSuite with Matchers with LocalSparkContex val data = sc.parallelize(Seq(true, false, false, false), 4) data.persist(StorageLevel.MEMORY_ONLY_2) data.count - assert(sc.persistentRdds.isEmpty === false) - data.unpersist() - assert(sc.persistentRdds.isEmpty === true) + assert(sc.persistentRdds.nonEmpty) + data.unpersist(blocking = true) + assert(sc.persistentRdds.isEmpty) failAfter(Span(3000, Millis)) { try { diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 896cd2e80aaef..182f28c5cce54 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -33,12 +33,12 @@ class DriverSuite extends SparkFunSuite with TimeLimits { ignore("driver should exit after finishing without cleanup (SPARK-530)") { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val masters = Table("master", "local", "local-cluster[2,1,1024]") - forAll(masters) { (master: String) => + forAll(masters) { master => val process = Utils.executeCommand( Seq(s"$sparkHome/bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) - failAfter(60 seconds) { process.waitFor() } + failAfter(1.minute) { process.waitFor() } // Ensure we still kill the process in case it timed out process.destroy() } diff --git a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala index 5c718cb654ce8..191b516661e49 100644 --- a/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExecutorAllocationManagerSuite.scala @@ -19,92 +19,99 @@ package org.apache.spark import scala.collection.mutable -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, never, verify, when} -import org.scalatest.{BeforeAndAfter, PrivateMethodTester} +import org.scalatest.PrivateMethodTester -import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_SCHEDULE_INTERVAL +import org.apache.spark.metrics.MetricsSystem import org.apache.spark.scheduler._ -import org.apache.spark.scheduler.ExternalClusterManager import org.apache.spark.scheduler.cluster.ExecutorInfo -import org.apache.spark.scheduler.local.LocalSchedulerBackend -import org.apache.spark.storage.BlockManagerMaster -import org.apache.spark.util.ManualClock +import org.apache.spark.util.{Clock, ManualClock, SystemClock} /** * Test add and remove behavior of ExecutorAllocationManager. */ -class ExecutorAllocationManagerSuite - extends SparkFunSuite - with LocalSparkContext - with BeforeAndAfter { +class ExecutorAllocationManagerSuite extends SparkFunSuite { import ExecutorAllocationManager._ import ExecutorAllocationManagerSuite._ - private val contexts = new mutable.ListBuffer[SparkContext]() - - before { - contexts.clear() + private val managers = new mutable.ListBuffer[ExecutorAllocationManager]() + private var listenerBus: LiveListenerBus = _ + private var client: ExecutorAllocationClient = _ + + override def beforeEach(): Unit = { + super.beforeEach() + managers.clear() + listenerBus = new LiveListenerBus(new SparkConf()) + listenerBus.start(null, mock(classOf[MetricsSystem])) + client = mock(classOf[ExecutorAllocationClient]) + when(client.isExecutorActive(any())).thenReturn(true) } - after { - contexts.foreach(_.stop()) + override def afterEach(): Unit = { + try { + listenerBus.stop() + managers.foreach(_.stop()) + } finally { + listenerBus = null + super.afterEach() + } } - private def post(bus: LiveListenerBus, event: SparkListenerEvent): Unit = { - bus.post(event) - bus.waitUntilEmpty(1000) + private def post(event: SparkListenerEvent): Unit = { + listenerBus.post(event) + listenerBus.waitUntilEmpty(1000) } - test("verify min/max executors") { - val conf = new SparkConf() - .setMaster("myDummyLocalExternalClusterManager") - .setAppName("test-executor-allocation-manager") - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.testing", "true") + test("initialize dynamic allocation in SparkContext") { + val conf = createConf(0, 1, 0) + .setMaster("local-cluster[1,1,1024]") + .setAppName(getClass().getName()) + val sc0 = new SparkContext(conf) - contexts += sc0 - assert(sc0.executorAllocationManager.isDefined) - sc0.stop() + try { + assert(sc0.executorAllocationManager.isDefined) + } finally { + sc0.stop() + } + } + test("verify min/max executors") { // Min < 0 - val conf1 = conf.clone().set("spark.dynamicAllocation.minExecutors", "-1") - intercept[SparkException] { contexts += new SparkContext(conf1) } + intercept[SparkException] { + createManager(createConf().set(config.DYN_ALLOCATION_MIN_EXECUTORS, -1)) + } // Max < 0 - val conf2 = conf.clone().set("spark.dynamicAllocation.maxExecutors", "-1") - intercept[SparkException] { contexts += new SparkContext(conf2) } + intercept[SparkException] { + createManager(createConf().set(config.DYN_ALLOCATION_MAX_EXECUTORS, -1)) + } // Both min and max, but min > max - intercept[SparkException] { createSparkContext(2, 1) } + intercept[SparkException] { + createManager(createConf(2, 1)) + } // Both min and max, and min == max - val sc1 = createSparkContext(1, 1) - assert(sc1.executorAllocationManager.isDefined) - sc1.stop() + createManager(createConf(1, 1)) // Both min and max, and min < max - val sc2 = createSparkContext(1, 2) - assert(sc2.executorAllocationManager.isDefined) - sc2.stop() + createManager(createConf(1, 2)) } test("starting state") { - sc = createSparkContext() - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf()) assert(numExecutorsTarget(manager) === 1) assert(executorsPendingToRemove(manager).isEmpty) - assert(executorIds(manager).isEmpty) assert(addTime(manager) === ExecutorAllocationManager.NOT_SET) - assert(removeTimes(manager).isEmpty) } test("add executors") { - sc = createSparkContext(1, 10, 1) - val manager = sc.executorAllocationManager.get - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val manager = createManager(createConf(1, 10, 1)) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Keep adding until the limit is reached assert(numExecutorsTarget(manager) === 1) @@ -147,24 +154,15 @@ class ExecutorAllocationManagerSuite } def testAllocationRatio(cores: Int, divisor: Double, expected: Int): Unit = { - val conf = new SparkConf() - .setMaster("myDummyLocalExternalClusterManager") - .setAppName("test-executor-allocation-manager") - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.testing", "true") - .set("spark.dynamicAllocation.maxExecutors", "15") - .set("spark.dynamicAllocation.minExecutors", "3") - .set("spark.dynamicAllocation.executorAllocationRatio", divisor.toString) - .set("spark.executor.cores", cores.toString) - val sc = new SparkContext(conf) - contexts += sc - var manager = sc.executorAllocationManager.get - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 20))) + val conf = createConf(3, 15) + .set(config.DYN_ALLOCATION_EXECUTOR_ALLOCATION_RATIO, divisor) + .set(config.EXECUTOR_CORES, cores) + val manager = createManager(conf) + post(SparkListenerStageSubmitted(createStageInfo(0, 20))) for (i <- 0 to 5) { addExecutors(manager) } assert(numExecutorsTarget(manager) === expected) - sc.stop() } test("executionAllocationRatio is correctly handled") { @@ -180,9 +178,8 @@ class ExecutorAllocationManagerSuite test("add executors capped by num pending tasks") { - sc = createSparkContext(0, 10, 0) - val manager = sc.executorAllocationManager.get - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 5))) + val manager = createManager(createConf(0, 10, 0)) + post(SparkListenerStageSubmitted(createStageInfo(0, 5))) // Verify that we're capped at number of tasks in the stage assert(numExecutorsTarget(manager) === 0) @@ -198,10 +195,10 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that running a task doesn't affect the target - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 3))) - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + post(SparkListenerStageSubmitted(createStageInfo(1, 3))) + post(SparkListenerExecutorAdded( + 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) + post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 1) assert(numExecutorsTarget(manager) === 6) @@ -214,9 +211,9 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that re-running a task doesn't blow things up - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, 3))) - post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) - post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) + post(SparkListenerStageSubmitted(createStageInfo(2, 3))) + post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 0, "executor-1"))) + post(SparkListenerTaskStart(2, 0, createTaskInfo(1, 0, "executor-1"))) assert(addExecutors(manager) === 1) assert(numExecutorsTarget(manager) === 9) assert(numExecutorsToAdd(manager) === 2) @@ -225,23 +222,22 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that running a task once we're at our limit doesn't blow things up - post(sc.listenerBus, SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) + post(SparkListenerTaskStart(2, 0, createTaskInfo(0, 1, "executor-1"))) assert(addExecutors(manager) === 0) assert(numExecutorsTarget(manager) === 10) } test("add executors when speculative tasks added") { - sc = createSparkContext(0, 10, 0) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(0, 10, 0)) // Verify that we're capped at number of tasks including the speculative ones in the stage - post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) + post(SparkListenerSpeculativeTaskSubmitted(1)) assert(numExecutorsTarget(manager) === 0) assert(numExecutorsToAdd(manager) === 1) assert(addExecutors(manager) === 1) - post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) - post(sc.listenerBus, SparkListenerSpeculativeTaskSubmitted(1)) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 2))) + post(SparkListenerSpeculativeTaskSubmitted(1)) + post(SparkListenerSpeculativeTaskSubmitted(1)) + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 2) assert(addExecutors(manager) === 2) @@ -252,39 +248,59 @@ class ExecutorAllocationManagerSuite assert(numExecutorsToAdd(manager) === 1) // Verify that running a task doesn't affect the target - post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) + post(SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-1"))) assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 0) assert(numExecutorsToAdd(manager) === 1) // Verify that running a speculative task doesn't affect the target - post(sc.listenerBus, SparkListenerTaskStart(1, 0, createTaskInfo(0, 0, "executor-2", true))) + post(SparkListenerTaskStart(1, 0, createTaskInfo(1, 0, "executor-2", true))) assert(numExecutorsTarget(manager) === 5) assert(addExecutors(manager) === 0) assert(numExecutorsToAdd(manager) === 1) } - test("ignore task end events from completed stages") { - sc = createSparkContext(0, 10, 0) - val manager = sc.executorAllocationManager.get + test("properly handle task end events from completed stages") { + val manager = createManager(createConf(0, 10, 0)) + + // We simulate having a stage fail, but with tasks still running. Then another attempt for + // that stage is started, and we get task completions from the first stage attempt. Make sure + // the value of `totalTasksRunning` is consistent as tasks finish from both attempts (we count + // all running tasks, from the zombie & non-zombie attempts) val stage = createStageInfo(0, 5) - post(sc.listenerBus, SparkListenerStageSubmitted(stage)) + post(SparkListenerStageSubmitted(stage)) val taskInfo1 = createTaskInfo(0, 0, "executor-1") val taskInfo2 = createTaskInfo(1, 1, "executor-1") - post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfo1)) - post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfo2)) - - post(sc.listenerBus, SparkListenerStageCompleted(stage)) - - post(sc.listenerBus, SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, null)) - post(sc.listenerBus, SparkListenerTaskEnd(2, 0, null, Success, taskInfo2, null)) + post(SparkListenerTaskStart(0, 0, taskInfo1)) + post(SparkListenerTaskStart(0, 0, taskInfo2)) + + // The tasks in the zombie attempt haven't completed yet, so we still count them + post(SparkListenerStageCompleted(stage)) + + // There are still two tasks that belong to the zombie stage running. + assert(totalRunningTasks(manager) === 2) + + // submit another attempt for the stage. We count completions from the first zombie attempt + val stageAttempt1 = createStageInfo(stage.stageId, 5, attemptId = 1) + post(SparkListenerStageSubmitted(stageAttempt1)) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo1, null)) + assert(totalRunningTasks(manager) === 1) + val attemptTaskInfo1 = createTaskInfo(3, 0, "executor-1") + val attemptTaskInfo2 = createTaskInfo(4, 1, "executor-1") + post(SparkListenerTaskStart(0, 1, attemptTaskInfo1)) + post(SparkListenerTaskStart(0, 1, attemptTaskInfo2)) + assert(totalRunningTasks(manager) === 3) + post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo1, null)) + assert(totalRunningTasks(manager) === 2) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo2, null)) + assert(totalRunningTasks(manager) === 1) + post(SparkListenerTaskEnd(0, 1, null, Success, attemptTaskInfo2, null)) assert(totalRunningTasks(manager) === 0) } - test("cancel pending executors when no longer needed") { - sc = createSparkContext(0, 10, 0) - val manager = sc.executorAllocationManager.get - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, 5))) + testRetry("cancel pending executors when no longer needed") { + val manager = createManager(createConf(0, 10, 0)) + post(SparkListenerStageSubmitted(createStageInfo(2, 5))) assert(numExecutorsTarget(manager) === 0) assert(numExecutorsToAdd(manager) === 1) @@ -295,26 +311,25 @@ class ExecutorAllocationManagerSuite assert(numExecutorsTarget(manager) === 3) val task1Info = createTaskInfo(0, 0, "executor-1") - post(sc.listenerBus, SparkListenerTaskStart(2, 0, task1Info)) + post(SparkListenerTaskStart(2, 0, task1Info)) assert(numExecutorsToAdd(manager) === 4) assert(addExecutors(manager) === 2) val task2Info = createTaskInfo(1, 0, "executor-1") - post(sc.listenerBus, SparkListenerTaskStart(2, 0, task2Info)) + post(SparkListenerTaskStart(2, 0, task2Info)) task1Info.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - post(sc.listenerBus, SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) + post(SparkListenerTaskEnd(2, 0, null, Success, task1Info, null)) task2Info.markFinished(TaskState.FINISHED, System.currentTimeMillis()) - post(sc.listenerBus, SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) + post(SparkListenerTaskEnd(2, 0, null, Success, task2Info, null)) assert(adjustRequestedExecutors(manager) === -1) } test("remove executors") { - sc = createSparkContext(5, 10, 5) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(5, 10, 5)) (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } // Keep removing until the limit is reached @@ -327,8 +342,6 @@ class ExecutorAllocationManagerSuite assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) - assert(!removeExecutor(manager, "100")) // remove non-existent executors - assert(!removeExecutor(manager, "101")) assert(executorsPendingToRemove(manager).size === 3) assert(removeExecutor(manager, "4")) assert(removeExecutor(manager, "5")) @@ -363,8 +376,7 @@ class ExecutorAllocationManagerSuite } test("remove multiple executors") { - sc = createSparkContext(5, 10, 5) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(5, 10, 5)) (1 to 10).map(_.toString).foreach { id => onExecutorAdded(manager, id) } // Keep removing until the limit is reached @@ -376,8 +388,6 @@ class ExecutorAllocationManagerSuite assert(executorsPendingToRemove(manager).size === 3) assert(executorsPendingToRemove(manager).contains("2")) assert(executorsPendingToRemove(manager).contains("3")) - assert(!removeExecutor(manager, "100")) // remove non-existent executors - assert(removeExecutors(manager, Seq("101", "102")) !== Seq("101", "102")) assert(executorsPendingToRemove(manager).size === 3) assert(removeExecutor(manager, "4")) assert(removeExecutors(manager, Seq("5")) === Seq("5")) @@ -412,27 +422,27 @@ class ExecutorAllocationManagerSuite } test ("Removing with various numExecutorsTarget condition") { - sc = createSparkContext(5, 12, 5) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(5, 12, 5)) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 8))) + post(SparkListenerStageSubmitted(createStageInfo(0, 8))) // Remove when numExecutorsTarget is the same as the current number of executors assert(addExecutors(manager) === 1) assert(addExecutors(manager) === 2) + (1 to 8).foreach(execId => onExecutorAdded(manager, execId.toString)) (1 to 8).map { i => createTaskInfo(i, i, s"$i") }.foreach { - info => post(sc.listenerBus, SparkListenerTaskStart(0, 0, info)) } - assert(executorIds(manager).size === 8) + info => post(SparkListenerTaskStart(0, 0, info)) } + assert(manager.executorMonitor.executorCount === 8) assert(numExecutorsTarget(manager) === 8) assert(maxNumExecutorsNeeded(manager) == 8) assert(!removeExecutor(manager, "1")) // won't work since numExecutorsTarget == numExecutors // Remove executors when numExecutorsTarget is lower than current number of executors (1 to 3).map { i => createTaskInfo(i, i, s"$i") }.foreach { info => - post(sc.listenerBus, SparkListenerTaskEnd(0, 0, null, Success, info, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, info, null)) } adjustRequestedExecutors(manager) - assert(executorIds(manager).size === 8) + assert(manager.executorMonitor.executorCount === 8) assert(numExecutorsTarget(manager) === 5) assert(maxNumExecutorsNeeded(manager) == 5) assert(removeExecutor(manager, "1")) @@ -442,9 +452,8 @@ class ExecutorAllocationManagerSuite onExecutorRemoved(manager, "3") // numExecutorsTarget is lower than minNumExecutors - post(sc.listenerBus, - SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), null)) - assert(executorIds(manager).size === 5) + post(SparkListenerTaskEnd(0, 0, null, Success, createTaskInfo(4, 4, "4"), null)) + assert(manager.executorMonitor.executorCount === 5) assert(numExecutorsTarget(manager) === 5) assert(maxNumExecutorsNeeded(manager) == 4) assert(!removeExecutor(manager, "4")) // lower limit @@ -452,9 +461,8 @@ class ExecutorAllocationManagerSuite } test ("interleaving add and remove") { - sc = createSparkContext(5, 12, 5) - val manager = sc.executorAllocationManager.get - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val manager = createManager(createConf(5, 12, 5)) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Add a few executors assert(addExecutors(manager) === 1) @@ -467,7 +475,7 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "6") onExecutorAdded(manager, "7") onExecutorAdded(manager, "8") - assert(executorIds(manager).size === 8) + assert(manager.executorMonitor.executorCount === 8) assert(numExecutorsTarget(manager) === 8) @@ -480,7 +488,7 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "10") onExecutorAdded(manager, "11") onExecutorAdded(manager, "12") - assert(executorIds(manager).size === 12) + assert(manager.executorMonitor.executorCount === 12) assert(numExecutorsTarget(manager) === 8) assert(removeExecutor(manager, "1")) @@ -491,7 +499,7 @@ class ExecutorAllocationManagerSuite onExecutorRemoved(manager, "2") onExecutorRemoved(manager, "3") onExecutorRemoved(manager, "4") - assert(executorIds(manager).size === 8) + assert(manager.executorMonitor.executorCount === 8) // Add until limit assert(!removeExecutor(manager, "7")) // still at lower limit @@ -500,34 +508,32 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "14") onExecutorAdded(manager, "15") onExecutorAdded(manager, "16") - assert(executorIds(manager).size === 12) + assert(manager.executorMonitor.executorCount === 12) // Remove succeeds again, now that we are no longer at the lower limit assert(removeExecutors(manager, Seq("5", "6", "7")) === Seq("5", "6", "7")) assert(removeExecutor(manager, "8")) - assert(executorIds(manager).size === 12) + assert(manager.executorMonitor.executorCount === 12) onExecutorRemoved(manager, "5") onExecutorRemoved(manager, "6") - assert(executorIds(manager).size === 10) + assert(manager.executorMonitor.executorCount === 10) assert(numExecutorsToAdd(manager) === 4) onExecutorRemoved(manager, "9") onExecutorRemoved(manager, "10") assert(addExecutors(manager) === 4) // at upper limit onExecutorAdded(manager, "17") onExecutorAdded(manager, "18") - assert(executorIds(manager).size === 10) + assert(manager.executorMonitor.executorCount === 10) assert(addExecutors(manager) === 0) // still at upper limit onExecutorAdded(manager, "19") onExecutorAdded(manager, "20") - assert(executorIds(manager).size === 12) + assert(manager.executorMonitor.executorCount === 12) assert(numExecutorsTarget(manager) === 12) } test("starting/canceling add timer") { - sc = createSparkContext(2, 10, 2) val clock = new ManualClock(8888L) - val manager = sc.executorAllocationManager.get - manager.setClock(clock) + val manager = createManager(createConf(2, 10, 2), clock = clock) // Starting add timer is idempotent assert(addTime(manager) === NOT_SET) @@ -555,58 +561,9 @@ class ExecutorAllocationManagerSuite assert(firstAddTime !== secondAddTime) } - test("starting/canceling remove timers") { - sc = createSparkContext(2, 10, 2) - val clock = new ManualClock(14444L) - val manager = sc.executorAllocationManager.get - manager.setClock(clock) - - executorIds(manager).asInstanceOf[mutable.Set[String]] ++= List("1", "2", "3") - - // Starting remove timer is idempotent for each executor - assert(removeTimes(manager).isEmpty) - onExecutorIdle(manager, "1") - assert(removeTimes(manager).size === 1) - assert(removeTimes(manager).contains("1")) - val firstRemoveTime = removeTimes(manager)("1") - assert(firstRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.advance(100L) - onExecutorIdle(manager, "1") - assert(removeTimes(manager)("1") === firstRemoveTime) // timer is already started - clock.advance(200L) - onExecutorIdle(manager, "1") - assert(removeTimes(manager)("1") === firstRemoveTime) - clock.advance(300L) - onExecutorIdle(manager, "2") - assert(removeTimes(manager)("2") !== firstRemoveTime) // different executor - assert(removeTimes(manager)("2") === clock.getTimeMillis + executorIdleTimeout * 1000) - clock.advance(400L) - onExecutorIdle(manager, "3") - assert(removeTimes(manager)("3") !== firstRemoveTime) - assert(removeTimes(manager)("3") === clock.getTimeMillis + executorIdleTimeout * 1000) - assert(removeTimes(manager).size === 3) - assert(removeTimes(manager).contains("2")) - assert(removeTimes(manager).contains("3")) - - // Restart remove timer - clock.advance(1000L) - onExecutorBusy(manager, "1") - assert(removeTimes(manager).size === 2) - onExecutorIdle(manager, "1") - assert(removeTimes(manager).size === 3) - assert(removeTimes(manager).contains("1")) - val secondRemoveTime = removeTimes(manager)("1") - assert(secondRemoveTime === clock.getTimeMillis + executorIdleTimeout * 1000) - assert(removeTimes(manager)("1") === secondRemoveTime) // timer is already started - assert(removeTimes(manager)("1") !== firstRemoveTime) - assert(firstRemoveTime !== secondRemoveTime) - } - test("mock polling loop with no events") { - sc = createSparkContext(0, 20, 0) - val manager = sc.executorAllocationManager.get val clock = new ManualClock(2020L) - manager.setClock(clock) + val manager = createManager(createConf(0, 20, 0), clock = clock) // No events - we should not be adding or removing assert(numExecutorsTarget(manager) === 0) @@ -629,11 +586,9 @@ class ExecutorAllocationManagerSuite } test("mock polling loop add behavior") { - sc = createSparkContext(0, 20, 0) val clock = new ManualClock(2020L) - val manager = sc.executorAllocationManager.get - manager.setClock(clock) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1000))) + val manager = createManager(createConf(0, 20, 0), clock = clock) + post(SparkListenerStageSubmitted(createStageInfo(0, 1000))) // Scheduler queue backlogged onSchedulerBacklogged(manager) @@ -679,24 +634,25 @@ class ExecutorAllocationManagerSuite } test("mock polling loop remove behavior") { - sc = createSparkContext(1, 20, 1) val clock = new ManualClock(2020L) - val manager = sc.executorAllocationManager.get - manager.setClock(clock) + val manager = createManager(createConf(1, 20, 1), clock = clock) // Remove idle executors on timeout onExecutorAdded(manager, "executor-1") onExecutorAdded(manager, "executor-2") onExecutorAdded(manager, "executor-3") - assert(removeTimes(manager).size === 3) assert(executorsPendingToRemove(manager).isEmpty) + + // idle threshold not reached yet clock.advance(executorIdleTimeout * 1000 / 2) schedule(manager) - assert(removeTimes(manager).size === 3) // idle threshold not reached yet + assert(manager.executorMonitor.timedOutExecutors().isEmpty) assert(executorsPendingToRemove(manager).isEmpty) + + // idle threshold exceeded clock.advance(executorIdleTimeout * 1000) + assert(manager.executorMonitor.timedOutExecutors().size === 3) schedule(manager) - assert(removeTimes(manager).isEmpty) // idle threshold exceeded assert(executorsPendingToRemove(manager).size === 2) // limit reached (1 executor remaining) // Mark a subset as busy - only idle executors should be removed @@ -704,20 +660,20 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "executor-5") onExecutorAdded(manager, "executor-6") onExecutorAdded(manager, "executor-7") - assert(removeTimes(manager).size === 5) // 5 active executors + assert(manager.executorMonitor.executorCount === 7) assert(executorsPendingToRemove(manager).size === 2) // 2 pending to be removed onExecutorBusy(manager, "executor-4") onExecutorBusy(manager, "executor-5") onExecutorBusy(manager, "executor-6") // 3 busy and 2 idle (of the 5 active ones) + + // after scheduling, the previously timed out executor should be removed, since + // there are new active ones. schedule(manager) - assert(removeTimes(manager).size === 2) // remove only idle executors - assert(!removeTimes(manager).contains("executor-4")) - assert(!removeTimes(manager).contains("executor-5")) - assert(!removeTimes(manager).contains("executor-6")) - assert(executorsPendingToRemove(manager).size === 2) + assert(executorsPendingToRemove(manager).size === 3) + + // advance the clock so that idle executors should time out and move to the pending list clock.advance(executorIdleTimeout * 1000) schedule(manager) - assert(removeTimes(manager).isEmpty) // idle executors are removed assert(executorsPendingToRemove(manager).size === 4) assert(!executorsPendingToRemove(manager).contains("executor-4")) assert(!executorsPendingToRemove(manager).contains("executor-5")) @@ -728,152 +684,45 @@ class ExecutorAllocationManagerSuite onExecutorIdle(manager, "executor-5") onExecutorIdle(manager, "executor-6") schedule(manager) - assert(removeTimes(manager).size === 3) // 0 busy and 3 idle - assert(removeTimes(manager).contains("executor-4")) - assert(removeTimes(manager).contains("executor-5")) - assert(removeTimes(manager).contains("executor-6")) assert(executorsPendingToRemove(manager).size === 4) clock.advance(executorIdleTimeout * 1000) schedule(manager) - assert(removeTimes(manager).isEmpty) assert(executorsPendingToRemove(manager).size === 6) // limit reached (1 executor remaining) } test("listeners trigger add executors correctly") { - sc = createSparkContext(2, 10, 2) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(1, 20, 1)) assert(addTime(manager) === NOT_SET) // Starting a stage should start the add timer val numTasks = 10 - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, numTasks))) + post(SparkListenerStageSubmitted(createStageInfo(0, numTasks))) assert(addTime(manager) !== NOT_SET) // Starting a subset of the tasks should not cancel the add timer val taskInfos = (0 to numTasks - 1).map { i => createTaskInfo(i, i, "executor-1") } - taskInfos.tail.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(0, 0, info)) } + taskInfos.tail.foreach { info => post(SparkListenerTaskStart(0, 0, info)) } assert(addTime(manager) !== NOT_SET) // Starting all remaining tasks should cancel the add timer - post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfos.head)) + post(SparkListenerTaskStart(0, 0, taskInfos.head)) assert(addTime(manager) === NOT_SET) // Start two different stages // The add timer should be canceled only if all tasks in both stages start running - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, numTasks))) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(2, numTasks))) + post(SparkListenerStageSubmitted(createStageInfo(1, numTasks))) + post(SparkListenerStageSubmitted(createStageInfo(2, numTasks))) assert(addTime(manager) !== NOT_SET) - taskInfos.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(1, 0, info)) } + taskInfos.foreach { info => post(SparkListenerTaskStart(1, 0, info)) } assert(addTime(manager) !== NOT_SET) - taskInfos.foreach { info => post(sc.listenerBus, SparkListenerTaskStart(2, 0, info)) } + taskInfos.foreach { info => post(SparkListenerTaskStart(2, 0, info)) } assert(addTime(manager) === NOT_SET) } - test("listeners trigger remove executors correctly") { - sc = createSparkContext(2, 10, 2) - val manager = sc.executorAllocationManager.get - assert(removeTimes(manager).isEmpty) - - // Added executors should start the remove timers for each executor - (1 to 5).map("executor-" + _).foreach { id => onExecutorAdded(manager, id) } - assert(removeTimes(manager).size === 5) - - // Starting a task cancel the remove timer for that executor - post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(1, 1, "executor-1"))) - post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(2, 2, "executor-2"))) - assert(removeTimes(manager).size === 3) - assert(!removeTimes(manager).contains("executor-1")) - assert(!removeTimes(manager).contains("executor-2")) - - // Finishing all tasks running on an executor should start the remove timer for that executor - post(sc.listenerBus, SparkListenerTaskEnd( - 0, 0, "task-type", Success, createTaskInfo(0, 0, "executor-1"), new TaskMetrics)) - post(sc.listenerBus, SparkListenerTaskEnd( - 0, 0, "task-type", Success, createTaskInfo(2, 2, "executor-2"), new TaskMetrics)) - assert(removeTimes(manager).size === 4) - assert(!removeTimes(manager).contains("executor-1")) // executor-1 has not finished yet - assert(removeTimes(manager).contains("executor-2")) - post(sc.listenerBus, SparkListenerTaskEnd( - 0, 0, "task-type", Success, createTaskInfo(1, 1, "executor-1"), new TaskMetrics)) - assert(removeTimes(manager).size === 5) - assert(removeTimes(manager).contains("executor-1")) // executor-1 has now finished - } - - test("listeners trigger add and remove executor callbacks correctly") { - sc = createSparkContext(2, 10, 2) - val manager = sc.executorAllocationManager.get - assert(executorIds(manager).isEmpty) - assert(removeTimes(manager).isEmpty) - - // New executors have registered - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - assert(executorIds(manager).size === 1) - assert(executorIds(manager).contains("executor-1")) - assert(removeTimes(manager).size === 1) - assert(removeTimes(manager).contains("executor-1")) - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host2", 1, Map.empty))) - assert(executorIds(manager).size === 2) - assert(executorIds(manager).contains("executor-2")) - assert(removeTimes(manager).size === 2) - assert(removeTimes(manager).contains("executor-2")) - - // Existing executors have disconnected - post(sc.listenerBus, SparkListenerExecutorRemoved(0L, "executor-1", "")) - assert(executorIds(manager).size === 1) - assert(!executorIds(manager).contains("executor-1")) - assert(removeTimes(manager).size === 1) - assert(!removeTimes(manager).contains("executor-1")) - - // Unknown executor has disconnected - post(sc.listenerBus, SparkListenerExecutorRemoved(0L, "executor-3", "")) - assert(executorIds(manager).size === 1) - assert(removeTimes(manager).size === 1) - } - - test("SPARK-4951: call onTaskStart before onBlockManagerAdded") { - sc = createSparkContext(2, 10, 2) - val manager = sc.executorAllocationManager.get - assert(executorIds(manager).isEmpty) - assert(removeTimes(manager).isEmpty) - - post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - assert(executorIds(manager).size === 1) - assert(executorIds(manager).contains("executor-1")) - assert(removeTimes(manager).size === 0) - } - - test("SPARK-4951: onExecutorAdded should not add a busy executor to removeTimes") { - sc = createSparkContext(2, 10) - val manager = sc.executorAllocationManager.get - assert(executorIds(manager).isEmpty) - assert(removeTimes(manager).isEmpty) - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - post(sc.listenerBus, SparkListenerTaskStart(0, 0, createTaskInfo(0, 0, "executor-1"))) - - assert(executorIds(manager).size === 1) - assert(executorIds(manager).contains("executor-1")) - assert(removeTimes(manager).size === 0) - - post(sc.listenerBus, SparkListenerExecutorAdded( - 0L, "executor-2", new ExecutorInfo("host1", 1, Map.empty))) - assert(executorIds(manager).size === 2) - assert(executorIds(manager).contains("executor-2")) - assert(removeTimes(manager).size === 1) - assert(removeTimes(manager).contains("executor-2")) - assert(!removeTimes(manager).contains("executor-1")) - } - test("avoid ramp up when target < running executors") { - sc = createSparkContext(0, 100000, 0) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(0, 100000, 0)) val stage1 = createStageInfo(0, 1000) - post(sc.listenerBus, SparkListenerStageSubmitted(stage1)) + post(SparkListenerStageSubmitted(stage1)) assert(addExecutors(manager) === 1) assert(addExecutors(manager) === 2) @@ -883,22 +732,20 @@ class ExecutorAllocationManagerSuite (0 until 15).foreach { i => onExecutorAdded(manager, s"executor-$i") } - assert(executorIds(manager).size === 15) - post(sc.listenerBus, SparkListenerStageCompleted(stage1)) + assert(manager.executorMonitor.executorCount === 15) + post(SparkListenerStageCompleted(stage1)) adjustRequestedExecutors(manager) assert(numExecutorsTarget(manager) === 0) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 1000))) + post(SparkListenerStageSubmitted(createStageInfo(1, 1000))) addExecutors(manager) assert(numExecutorsTarget(manager) === 16) } test("avoid ramp down initial executors until first job is submitted") { - sc = createSparkContext(2, 5, 3) - val manager = sc.executorAllocationManager.get val clock = new ManualClock(10000L) - manager.setClock(clock) + val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors assert(numExecutorsTarget(manager) === 3) @@ -906,7 +753,7 @@ class ExecutorAllocationManagerSuite // Verify whether the initial number of executors is kept with no pending tasks assert(numExecutorsTarget(manager) === 3) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(1, 2))) + post(SparkListenerStageSubmitted(createStageInfo(1, 2))) clock.advance(100L) assert(maxNumExecutorsNeeded(manager) === 2) @@ -917,10 +764,8 @@ class ExecutorAllocationManagerSuite } test("avoid ramp down initial executors until idle executor is timeout") { - sc = createSparkContext(2, 5, 3) - val manager = sc.executorAllocationManager.get val clock = new ManualClock(10000L) - manager.setClock(clock) + val manager = createManager(createConf(2, 5, 3), clock = clock) // Verify the initial number of executors assert(numExecutorsTarget(manager) === 3) @@ -935,18 +780,12 @@ class ExecutorAllocationManagerSuite assert(maxNumExecutorsNeeded(manager) === 0) schedule(manager) - // Verify executor is timeout but numExecutorsTarget is not recalculated - assert(numExecutorsTarget(manager) === 3) - - // Schedule again to recalculate the numExecutorsTarget after executor is timeout - schedule(manager) - // Verify that current number of executors should be ramp down when executor is timeout + // Verify executor is timeout,numExecutorsTarget is recalculated assert(numExecutorsTarget(manager) === 2) } test("get pending task number and related locality preference") { - sc = createSparkContext(2, 5, 3) - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf(2, 5, 3)) val localityPreferences1 = Seq( Seq(TaskLocation("host1"), TaskLocation("host2"), TaskLocation("host3")), @@ -956,7 +795,7 @@ class ExecutorAllocationManagerSuite Seq.empty ) val stageInfo1 = createStageInfo(1, 5, localityPreferences1) - post(sc.listenerBus, SparkListenerStageSubmitted(stageInfo1)) + post(SparkListenerStageSubmitted(stageInfo1)) assert(localityAwareTasks(manager) === 3) assert(hostToLocalTaskCount(manager) === @@ -968,45 +807,43 @@ class ExecutorAllocationManagerSuite Seq.empty ) val stageInfo2 = createStageInfo(2, 3, localityPreferences2) - post(sc.listenerBus, SparkListenerStageSubmitted(stageInfo2)) + post(SparkListenerStageSubmitted(stageInfo2)) assert(localityAwareTasks(manager) === 5) assert(hostToLocalTaskCount(manager) === Map("host1" -> 2, "host2" -> 4, "host3" -> 4, "host4" -> 3, "host5" -> 2)) - post(sc.listenerBus, SparkListenerStageCompleted(stageInfo1)) + post(SparkListenerStageCompleted(stageInfo1)) assert(localityAwareTasks(manager) === 2) assert(hostToLocalTaskCount(manager) === Map("host2" -> 1, "host3" -> 2, "host4" -> 1, "host5" -> 2)) } test("SPARK-8366: maxNumExecutorsNeeded should properly handle failed tasks") { - sc = createSparkContext() - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf()) assert(maxNumExecutorsNeeded(manager) === 0) - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 1))) + post(SparkListenerStageSubmitted(createStageInfo(0, 1))) assert(maxNumExecutorsNeeded(manager) === 1) val taskInfo = createTaskInfo(1, 1, "executor-1") - post(sc.listenerBus, SparkListenerTaskStart(0, 0, taskInfo)) + post(SparkListenerTaskStart(0, 0, taskInfo)) assert(maxNumExecutorsNeeded(manager) === 1) // If the task is failed, we expect it to be resubmitted later. val taskEndReason = ExceptionFailure(null, null, null, null, None) - post(sc.listenerBus, SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) + post(SparkListenerTaskEnd(0, 0, null, taskEndReason, taskInfo, null)) assert(maxNumExecutorsNeeded(manager) === 1) } test("reset the state of allocation manager") { - sc = createSparkContext() - val manager = sc.executorAllocationManager.get + val manager = createManager(createConf()) assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 1) // Allocation manager is reset when adding executor requests are sent without reporting back // executor added. - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 10))) + post(SparkListenerStageSubmitted(createStageInfo(0, 10))) assert(addExecutors(manager) === 1) assert(numExecutorsTarget(manager) === 2) @@ -1018,10 +855,10 @@ class ExecutorAllocationManagerSuite manager.reset() assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 1) - assert(executorIds(manager) === Set.empty) + assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are added. - post(sc.listenerBus, SparkListenerStageSubmitted(createStageInfo(0, 10))) + post(SparkListenerStageSubmitted(createStageInfo(0, 10))) addExecutors(manager) addExecutors(manager) @@ -1033,7 +870,7 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "third") onExecutorAdded(manager, "fourth") onExecutorAdded(manager, "fifth") - assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth")) + assert(manager.executorMonitor.executorCount === 5) // Cluster manager lost will make all the live executors lost, so here simulate this behavior onExecutorRemoved(manager, "first") @@ -1045,8 +882,7 @@ class ExecutorAllocationManagerSuite manager.reset() assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 1) - assert(executorIds(manager) === Set.empty) - assert(removeTimes(manager) === Map.empty) + assert(manager.executorMonitor.executorCount === 0) // Allocation manager is reset when executors are pending to remove addExecutors(manager) @@ -1062,14 +898,12 @@ class ExecutorAllocationManagerSuite onExecutorAdded(manager, "sixth") onExecutorAdded(manager, "seventh") onExecutorAdded(manager, "eighth") - assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth", - "sixth", "seventh", "eighth")) + assert(manager.executorMonitor.executorCount === 8) removeExecutor(manager, "first") removeExecutors(manager, Seq("second", "third")) assert(executorsPendingToRemove(manager) === Set("first", "second", "third")) - assert(executorIds(manager) === Set("first", "second", "third", "fourth", "fifth", - "sixth", "seventh", "eighth")) + assert(manager.executorMonitor.executorCount === 8) // Cluster manager lost will make all the live executors lost, so here simulate this behavior @@ -1084,94 +918,129 @@ class ExecutorAllocationManagerSuite assert(numExecutorsTarget(manager) === 1) assert(numExecutorsToAdd(manager) === 1) assert(executorsPendingToRemove(manager) === Set.empty) - assert(removeTimes(manager) === Map.empty) + assert(manager.executorMonitor.executorCount === 0) } test("SPARK-23365 Don't update target num executors when killing idle executors") { - val minExecutors = 1 - val initialExecutors = 1 - val maxExecutors = 2 - val conf = new SparkConf() - .set("spark.dynamicAllocation.enabled", "true") - .set(config.SHUFFLE_SERVICE_ENABLED.key, "true") - .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) - .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.initialExecutors", initialExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", "1000ms") - .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", "1000ms") - .set("spark.dynamicAllocation.executorIdleTimeout", s"3000ms") - val mockAllocationClient = mock(classOf[ExecutorAllocationClient]) - val mockBMM = mock(classOf[BlockManagerMaster]) - val manager = new ExecutorAllocationManager( - mockAllocationClient, mock(classOf[LiveListenerBus]), conf, mockBMM) val clock = new ManualClock() - manager.setClock(clock) + val manager = createManager( + createConf(1, 2, 1).set(config.DYN_ALLOCATION_TESTING, false), + clock = clock) - when(mockAllocationClient.requestTotalExecutors(meq(2), any(), any())).thenReturn(true) + when(client.requestTotalExecutors(meq(2), any(), any())).thenReturn(true) // test setup -- job with 2 tasks, scale up to two executors assert(numExecutorsTarget(manager) === 1) - manager.listener.onExecutorAdded(SparkListenerExecutorAdded( - clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) - manager.listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 2))) + post(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) + post(SparkListenerStageSubmitted(createStageInfo(0, 2))) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) assert(numExecutorsTarget(manager) === 2) val taskInfo0 = createTaskInfo(0, 0, "executor-1") - manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo0)) - manager.listener.onExecutorAdded(SparkListenerExecutorAdded( - clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty))) + post(SparkListenerTaskStart(0, 0, taskInfo0)) + post(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 1, Map.empty, Map.empty))) val taskInfo1 = createTaskInfo(1, 1, "executor-2") - manager.listener.onTaskStart(SparkListenerTaskStart(0, 0, taskInfo1)) + post(SparkListenerTaskStart(0, 0, taskInfo1)) assert(numExecutorsTarget(manager) === 2) // have one task finish -- we should adjust the target number of executors down // but we should *not* kill any executors yet - manager.listener.onTaskEnd(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, null)) + post(SparkListenerTaskEnd(0, 0, null, Success, taskInfo0, null)) assert(maxNumExecutorsNeeded(manager) === 1) assert(numExecutorsTarget(manager) === 2) clock.advance(1000) manager invokePrivate _updateAndSyncNumExecutorsTarget(clock.getTimeMillis()) assert(numExecutorsTarget(manager) === 1) - verify(mockAllocationClient, never).killExecutors(any(), any(), any(), any()) + verify(client, never).killExecutors(any(), any(), any(), any()) // now we cross the idle timeout for executor-1, so we kill it. the really important // thing here is that we do *not* ask the executor allocation client to adjust the target // number of executors down - when(mockAllocationClient.killExecutors(Seq("executor-1"), false, false, false)) + when(client.killExecutors(Seq("executor-1"), false, false, false)) .thenReturn(Seq("executor-1")) clock.advance(3000) schedule(manager) assert(maxNumExecutorsNeeded(manager) === 1) assert(numExecutorsTarget(manager) === 1) // here's the important verify -- we did kill the executors, but did not adjust the target count - verify(mockAllocationClient).killExecutors(Seq("executor-1"), false, false, false) + verify(client).killExecutors(Seq("executor-1"), false, false, false) } - private def createSparkContext( + test("SPARK-26758 check executor target number after idle time out ") { + val clock = new ManualClock(10000L) + val manager = createManager(createConf(1, 5, 3), clock = clock) + assert(numExecutorsTarget(manager) === 3) + post(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-1", new ExecutorInfo("host1", 1, Map.empty))) + post(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-2", new ExecutorInfo("host1", 2, Map.empty))) + post(SparkListenerExecutorAdded( + clock.getTimeMillis(), "executor-3", new ExecutorInfo("host1", 3, Map.empty))) + // make all the executors as idle, so that it will be killed + clock.advance(executorIdleTimeout * 1000) + schedule(manager) + // once the schedule is run target executor number should be 1 + assert(numExecutorsTarget(manager) === 1) + } + + private def createConf( minExecutors: Int = 1, maxExecutors: Int = 5, - initialExecutors: Int = 1): SparkContext = { - val conf = new SparkConf() - .setMaster("myDummyLocalExternalClusterManager") - .setAppName("test-executor-allocation-manager") - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.minExecutors", minExecutors.toString) - .set("spark.dynamicAllocation.maxExecutors", maxExecutors.toString) - .set("spark.dynamicAllocation.initialExecutors", initialExecutors.toString) - .set("spark.dynamicAllocation.schedulerBacklogTimeout", - s"${schedulerBacklogTimeout.toString}s") - .set("spark.dynamicAllocation.sustainedSchedulerBacklogTimeout", + initialExecutors: Int = 1): SparkConf = { + new SparkConf() + .set(config.DYN_ALLOCATION_ENABLED, true) + .set(config.DYN_ALLOCATION_MIN_EXECUTORS, minExecutors) + .set(config.DYN_ALLOCATION_MAX_EXECUTORS, maxExecutors) + .set(config.DYN_ALLOCATION_INITIAL_EXECUTORS, initialExecutors) + .set(config.DYN_ALLOCATION_SCHEDULER_BACKLOG_TIMEOUT.key, + s"${schedulerBacklogTimeout.toString}s") + .set(config.DYN_ALLOCATION_SUSTAINED_SCHEDULER_BACKLOG_TIMEOUT.key, s"${sustainedSchedulerBacklogTimeout.toString}s") - .set("spark.dynamicAllocation.executorIdleTimeout", s"${executorIdleTimeout.toString}s") - .set("spark.dynamicAllocation.testing", "true") + .set(config.DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, s"${executorIdleTimeout.toString}s") + .set(config.SHUFFLE_SERVICE_ENABLED, true) + .set(config.DYN_ALLOCATION_TESTING, true) // SPARK-22864: effectively disable the allocation schedule by setting the period to a // really long value. - .set(TESTING_SCHEDULE_INTERVAL_KEY, "10000") - val sc = new SparkContext(conf) - contexts += sc - sc + .set(TEST_SCHEDULE_INTERVAL, 10000L) } + private def createManager( + conf: SparkConf, + clock: Clock = new SystemClock()): ExecutorAllocationManager = { + val manager = new ExecutorAllocationManager(client, listenerBus, conf, clock = clock) + managers += manager + manager.start() + manager + } + + private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { + post(SparkListenerExecutorAdded(0L, id, null)) + } + + private def onExecutorRemoved(manager: ExecutorAllocationManager, id: String): Unit = { + post(SparkListenerExecutorRemoved(0L, id, null)) + } + + private def onExecutorBusy(manager: ExecutorAllocationManager, id: String): Unit = { + val info = new TaskInfo(1, 1, 1, 0, id, "foo.example.com", TaskLocality.PROCESS_LOCAL, false) + post(SparkListenerTaskStart(1, 1, info)) + } + + private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = { + val info = new TaskInfo(1, 1, 1, 0, id, "foo.example.com", TaskLocality.PROCESS_LOCAL, false) + info.markFinished(TaskState.FINISHED, 1) + post(SparkListenerTaskEnd(1, 1, "foo", Success, info, null)) + } + + private def removeExecutor(manager: ExecutorAllocationManager, executorId: String): Boolean = { + val executorsRemoved = removeExecutors(manager, Seq(executorId)) + executorsRemoved.nonEmpty && executorsRemoved(0) == executorId + } + + private def executorsPendingToRemove(manager: ExecutorAllocationManager): Set[String] = { + manager.executorMonitor.executorsPendingToRemove() + } } /** @@ -1186,9 +1055,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private def createStageInfo( stageId: Int, numTasks: Int, - taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty + taskLocalityPreferences: Seq[Seq[TaskLocation]] = Seq.empty, + attemptId: Int = 0 ): StageInfo = { - new StageInfo(stageId, 0, "name", numTasks, Seq.empty, Seq.empty, "no details", + new StageInfo(stageId, attemptId, "name", numTasks, Seq.empty, Seq.empty, "no details", taskLocalityPreferences = taskLocalityPreferences) } @@ -1200,7 +1070,6 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { new TaskInfo(taskId, taskIndex, 0, 0, executorId, "", TaskLocality.ANY, speculative) } - /* ------------------------------------------------------- * | Helper methods for accessing private methods and fields | * ------------------------------------------------------- */ @@ -1208,23 +1077,14 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { private val _numExecutorsToAdd = PrivateMethod[Int]('numExecutorsToAdd) private val _numExecutorsTarget = PrivateMethod[Int]('numExecutorsTarget) private val _maxNumExecutorsNeeded = PrivateMethod[Int]('maxNumExecutorsNeeded) - private val _executorsPendingToRemove = - PrivateMethod[collection.Set[String]]('executorsPendingToRemove) - private val _executorIds = PrivateMethod[collection.Set[String]]('executorIds) private val _addTime = PrivateMethod[Long]('addTime) - private val _removeTimes = PrivateMethod[collection.Map[String, Long]]('removeTimes) private val _schedule = PrivateMethod[Unit]('schedule) private val _addExecutors = PrivateMethod[Int]('addExecutors) private val _updateAndSyncNumExecutorsTarget = PrivateMethod[Int]('updateAndSyncNumExecutorsTarget) - private val _removeExecutor = PrivateMethod[Boolean]('removeExecutor) private val _removeExecutors = PrivateMethod[Seq[String]]('removeExecutors) - private val _onExecutorAdded = PrivateMethod[Unit]('onExecutorAdded) - private val _onExecutorRemoved = PrivateMethod[Unit]('onExecutorRemoved) private val _onSchedulerBacklogged = PrivateMethod[Unit]('onSchedulerBacklogged) private val _onSchedulerQueueEmpty = PrivateMethod[Unit]('onSchedulerQueueEmpty) - private val _onExecutorIdle = PrivateMethod[Unit]('onExecutorIdle) - private val _onExecutorBusy = PrivateMethod[Unit]('onExecutorBusy) private val _localityAwareTasks = PrivateMethod[Int]('localityAwareTasks) private val _hostToLocalTaskCount = PrivateMethod[Map[String, Int]]('hostToLocalTaskCount) private val _onSpeculativeTaskSubmitted = PrivateMethod[Unit]('onSpeculativeTaskSubmitted) @@ -1238,23 +1098,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _numExecutorsTarget() } - private def executorsPendingToRemove( - manager: ExecutorAllocationManager): collection.Set[String] = { - manager invokePrivate _executorsPendingToRemove() - } - - private def executorIds(manager: ExecutorAllocationManager): collection.Set[String] = { - manager invokePrivate _executorIds() - } - private def addTime(manager: ExecutorAllocationManager): Long = { manager invokePrivate _addTime() } - private def removeTimes(manager: ExecutorAllocationManager): collection.Map[String, Long] = { - manager invokePrivate _removeTimes() - } - private def schedule(manager: ExecutorAllocationManager): Unit = { manager invokePrivate _schedule() } @@ -1272,22 +1119,10 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _updateAndSyncNumExecutorsTarget(0L) } - private def removeExecutor(manager: ExecutorAllocationManager, id: String): Boolean = { - manager invokePrivate _removeExecutor(id) - } - private def removeExecutors(manager: ExecutorAllocationManager, ids: Seq[String]): Seq[String] = { manager invokePrivate _removeExecutors(ids) } - private def onExecutorAdded(manager: ExecutorAllocationManager, id: String): Unit = { - manager invokePrivate _onExecutorAdded(id) - } - - private def onExecutorRemoved(manager: ExecutorAllocationManager, id: String): Unit = { - manager invokePrivate _onExecutorRemoved(id) - } - private def onSchedulerBacklogged(manager: ExecutorAllocationManager): Unit = { manager invokePrivate _onSchedulerBacklogged() } @@ -1296,14 +1131,6 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _onSchedulerQueueEmpty() } - private def onExecutorIdle(manager: ExecutorAllocationManager, id: String): Unit = { - manager invokePrivate _onExecutorIdle(id) - } - - private def onExecutorBusy(manager: ExecutorAllocationManager, id: String): Unit = { - manager invokePrivate _onExecutorBusy(id) - } - private def onSpeculativeTaskSubmitted(manager: ExecutorAllocationManager, id: String) : Unit = { manager invokePrivate _onSpeculativeTaskSubmitted(id) } @@ -1320,66 +1147,3 @@ private object ExecutorAllocationManagerSuite extends PrivateMethodTester { manager invokePrivate _hostToLocalTaskCount() } } - -/** - * A cluster manager which wraps around the scheduler and backend for local mode. It is used for - * testing the dynamic allocation policy. - */ -private class DummyLocalExternalClusterManager extends ExternalClusterManager { - - def canCreate(masterURL: String): Boolean = masterURL == "myDummyLocalExternalClusterManager" - - override def createTaskScheduler( - sc: SparkContext, - masterURL: String): TaskScheduler = new TaskSchedulerImpl(sc, 1, isLocal = true) - - override def createSchedulerBackend( - sc: SparkContext, - masterURL: String, - scheduler: TaskScheduler): SchedulerBackend = { - val sb = new LocalSchedulerBackend(sc.getConf, scheduler.asInstanceOf[TaskSchedulerImpl], 1) - new DummyLocalSchedulerBackend(sc, sb) - } - - override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { - val sc = scheduler.asInstanceOf[TaskSchedulerImpl] - sc.initialize(backend) - } -} - -/** - * A scheduler backend which wraps around local scheduler backend and exposes the executor - * allocation client interface for testing dynamic allocation. - */ -private class DummyLocalSchedulerBackend (sc: SparkContext, sb: SchedulerBackend) - extends SchedulerBackend with ExecutorAllocationClient { - - override private[spark] def getExecutorIds(): Seq[String] = Nil - - override private[spark] def requestTotalExecutors( - numExecutors: Int, - localityAwareTasks: Int, - hostToLocalTaskCount: Map[String, Int]): Boolean = true - - override def requestExecutors(numAdditionalExecutors: Int): Boolean = true - - override def killExecutors( - executorIds: Seq[String], - adjustTargetNumExecutors: Boolean, - countFailures: Boolean, - force: Boolean): Seq[String] = executorIds - - override def start(): Unit = sb.start() - - override def stop(): Unit = sb.stop() - - override def reviveOffers(): Unit = sb.reviveOffers() - - override def defaultParallelism(): Int = sb.defaultParallelism() - - override def maxNumConcurrentTasks(): Int = sb.maxNumConcurrentTasks() - - override def killExecutorsOnHost(host: String): Boolean = { - false - } -} diff --git a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala index 462d5f5604ae3..b2c59087b63dc 100644 --- a/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala +++ b/core/src/test/scala/org/apache/spark/ExternalShuffleServiceSuite.scala @@ -18,12 +18,16 @@ package org.apache.spark import org.scalatest.BeforeAndAfterAll +import org.scalatest.concurrent.Eventually +import org.scalatest.time.SpanSugar._ import org.apache.spark.internal.config import org.apache.spark.network.TransportContext import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.server.TransportServer import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleClient} +import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.util.Utils /** * This suite creates an external shuffle server and routes all shuffle fetches through it. @@ -31,28 +35,34 @@ import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalSh * set up in `ExternalShuffleBlockHandler`, such as changing the format of shuffle files or how * we hash files into folders. */ -class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { +class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll with Eventually { var server: TransportServer = _ + var transportContext: TransportContext = _ var rpcHandler: ExternalShuffleBlockHandler = _ override def beforeAll() { super.beforeAll() val transportConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 2) rpcHandler = new ExternalShuffleBlockHandler(transportConf, null) - val transportContext = new TransportContext(transportConf, rpcHandler) + transportContext = new TransportContext(transportConf, rpcHandler) server = transportContext.createServer() - conf.set("spark.shuffle.manager", "sort") - conf.set(config.SHUFFLE_SERVICE_ENABLED.key, "true") - conf.set(config.SHUFFLE_SERVICE_PORT.key, server.getPort.toString) + conf.set(config.SHUFFLE_MANAGER, "sort") + conf.set(config.SHUFFLE_SERVICE_ENABLED, true) + conf.set(config.SHUFFLE_SERVICE_PORT, server.getPort) } override def afterAll() { - try { + Utils.tryLogNonFatalError{ server.close() - } finally { - super.afterAll() } + Utils.tryLogNonFatalError{ + rpcHandler.close() + } + Utils.tryLogNonFatalError{ + transportContext.close() + } + super.afterAll() } // This test ensures that the external shuffle service is actually in use for the other tests. @@ -85,4 +95,43 @@ class ExternalShuffleServiceSuite extends ShuffleSuite with BeforeAndAfterAll { } e.getMessage should include ("Fetch failure will not retry stage due to testing config") } + + test("SPARK-25888: using external shuffle service fetching disk persisted blocks") { + val confWithRddFetchEnabled = conf.clone.set(config.SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) + sc = new SparkContext("local-cluster[1,1,1024]", "test", confWithRddFetchEnabled) + sc.env.blockManager.externalShuffleServiceEnabled should equal(true) + sc.env.blockManager.shuffleClient.getClass should equal(classOf[ExternalShuffleClient]) + try { + val rdd = sc.parallelize(0 until 100, 2) + .map { i => (i, 1) } + .persist(StorageLevel.DISK_ONLY) + + rdd.count() + + val blockId = RDDBlockId(rdd.id, 0) + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 2) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + sc.killExecutors(sc.getExecutorIds()) + + eventually(timeout(2.seconds), interval(100.milliseconds)) { + val locations = sc.env.blockManager.master.getLocations(blockId) + assert(locations.size === 1) + assert(locations.map(_.port).contains(server.getPort), + "external shuffle service port should be contained") + } + + assert(sc.env.blockManager.getRemoteValues(blockId).isDefined) + + // test unpersist: as executors are killed the blocks will be removed via the shuffle service + rdd.unpersist(true) + assert(sc.env.blockManager.getRemoteValues(blockId).isEmpty) + } finally { + rpcHandler.applicationRemoved(sc.conf.getAppId, true) + } + } } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index d805c67714ff8..5f79b526a419b 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark import java.io.{IOException, NotSerializableException, ObjectInputStream} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.memory.TestMemoryConsumer import org.apache.spark.storage.StorageLevel import org.apache.spark.util.NonSerializable @@ -144,7 +145,7 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { } test("managed memory leak error should not mask other failures (SPARK-9266") { - val conf = new SparkConf().set("spark.unsafe.exceptionOnMemoryLeak", "true") + val conf = new SparkConf().set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) sc = new SparkContext("local[1,1]", "test", conf) // If a task leaks memory but fails due to some other cause, then make sure that the original @@ -257,7 +258,9 @@ class FailureSuite extends SparkFunSuite with LocalSparkContext { sc = new SparkContext("local[1,2]", "test") intercept[SparkException] { sc.parallelize(1 to 2).foreach { i => + // scalastyle:off throwerror throw new LinkageError() + // scalastyle:on throwerror } } } diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 81b18c71f30ee..6651e38f7ed62 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -35,7 +35,7 @@ import org.apache.hadoop.mapreduce.lib.input.{FileSplit => NewFileSplit, TextInp import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} import org.apache.spark.internal.config._ -import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD, RDD} +import org.apache.spark.rdd.{HadoopRDD, NewHadoopRDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -94,6 +94,14 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { assert(compressedFile.length < normalFile.length) } + test("text files do not allow null rows") { + sc = new SparkContext("local", "test") + val outputDir = new File(tempDir, "output").getAbsolutePath + val nums = sc.makeRDD((1 to 100) ++ Seq(null)) + val exception = intercept[SparkException](nums.saveAsTextFile(outputDir)) + assert(Utils.exceptionString(exception).contains("text files do not allow null rows")) + } + test("SequenceFiles") { sc = new SparkContext("local", "test") val outputDir = new File(tempDir, "output").getAbsolutePath @@ -192,30 +200,24 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { } test("object files of classes from a JAR") { - // scalastyle:off classforname - val original = Thread.currentThread().getContextClassLoader val className = "FileSuiteObjectFileTest" val jar = TestUtils.createJarWithClasses(Seq(className)) val loader = new java.net.URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) - Thread.currentThread().setContextClassLoader(loader) - try { + + Utils.withContextClassLoader(loader) { sc = new SparkContext("local", "test") - val objs = sc.makeRDD(1 to 3).map { x => - val loader = Thread.currentThread().getContextClassLoader - Class.forName(className, true, loader).newInstance() + val objs = sc.makeRDD(1 to 3).map { _ => + Utils.classForName[AnyRef](className, noSparkClassLoader = true). + getConstructor().newInstance() } val outputDir = new File(tempDir, "output").getAbsolutePath objs.saveAsObjectFile(outputDir) // Try reading the output back as an object file - val ct = reflect.ClassTag[Any](Class.forName(className, true, loader)) + val ct = reflect.ClassTag[Any](Utils.classForName(className, noSparkClassLoader = true)) val output = sc.objectFile[Any](outputDir) assert(output.collect().size === 3) assert(output.collect().head.getClass.getName === className) } - finally { - Thread.currentThread().setContextClassLoader(original) - } - // scalastyle:on classforname } test("write SequenceFile using new Hadoop API") { @@ -306,20 +308,34 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { .set("spark.files.openCostInBytes", "0") .set("spark.default.parallelism", "1")) - val tempDir = Utils.createTempDir() - val tempDirPath = tempDir.getAbsolutePath + withTempDir { tempDir => + val tempDirPath = tempDir.getAbsolutePath - for (i <- 0 until 8) { - val tempFile = new File(tempDir, s"part-0000$i") - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile, - StandardCharsets.UTF_8) - } + for (i <- 0 until 8) { + val tempFile = new File(tempDir, s"part-0000$i") + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", tempFile, + StandardCharsets.UTF_8) + } - for (p <- Seq(1, 2, 8)) { - assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p) + for (p <- Seq(1, 2, 8)) { + assert(sc.binaryFiles(tempDirPath, minPartitions = p).getNumPartitions === p) + } } } + test("minimum split size per node and per rack should be less than or equal to maxSplitSize") { + sc = new SparkContext("local", "test") + val testOutput = Array[Byte](1, 2, 3, 4, 5) + val outFile = writeBinaryData(testOutput, 1) + sc.hadoopConfiguration.setLong( + "mapreduce.input.fileinputformat.split.minsize.per.node", 5123456) + sc.hadoopConfiguration.setLong( + "mapreduce.input.fileinputformat.split.minsize.per.rack", 5123456) + + val (_, data) = sc.binaryFiles(outFile.getAbsolutePath).collect().head + assert(data.toArray === testOutput) + } + test("fixed record length binary file as byte array") { sc = new SparkContext("local", "test") val testOutput = Array[Byte](1, 2, 3, 4, 5, 6) @@ -366,7 +382,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) randomRDD.saveAsTextFile(tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-00000").exists()) intercept[FileAlreadyExistsException] { randomRDD.saveAsTextFile(tempDir.getPath + "/output") } @@ -378,9 +394,9 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { sc = new SparkContext(conf) val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) randomRDD.saveAsTextFile(tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-00000").exists()) randomRDD.saveAsTextFile(tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-00000").exists()) } test ("prevent user from overwriting the empty directory (new Hadoop API)") { @@ -398,7 +414,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-r-00000").exists()) intercept[FileAlreadyExistsException] { randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempDir.getPath) } @@ -412,10 +428,10 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-r-00000").exists()) randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]]( tempDir.getPath + "/output") - assert(new File(tempDir.getPath + "/output/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/output/part-r-00000").exists()) } test ("save Hadoop Dataset through old Hadoop API") { @@ -428,7 +444,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) job.set("mapreduce.output.fileoutputformat.outputdir", tempDir.getPath + "/outputDataset_old") randomRDD.saveAsHadoopDataset(job) - assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_old/part-00000").exists()) } test ("save Hadoop Dataset through new Hadoop API") { @@ -443,7 +459,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { jobConfig.set("mapreduce.output.fileoutputformat.outputdir", tempDir.getPath + "/outputDataset_new") randomRDD.saveAsNewAPIHadoopDataset(jobConfig) - assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + assert(new File(tempDir.getPath + "/outputDataset_new/part-r-00000").exists()) } test("Get input files via old Hadoop API") { @@ -550,7 +566,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { sc.parallelize(data, actualPartitionNum) .saveAsHadoopFile[TextOutputFormat[String, String]](output.getPath) for (i <- 0 until actualPartitionNum) { - assert(new File(output, s"part-0000$i").exists() === true) + assert(new File(output, s"part-0000$i").exists()) } val hadoopRDD = sc.textFile(new File(output, "part-*").getPath) assert(hadoopRDD.partitions.length === expectedPartitionNum) @@ -591,7 +607,7 @@ class FileSuite extends SparkFunSuite with LocalSparkContext { sc.parallelize(data, actualPartitionNum) .saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](output.getPath) for (i <- 0 until actualPartitionNum) { - assert(new File(output, s"part-r-0000$i").exists() === true) + assert(new File(output, s"part-r-0000$i").exists()) } val hadoopRDD = sc.newAPIHadoopFile(new File(output, "part-r-*").getPath, classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]) diff --git a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala index de479db5fbc0f..dc4f4b4c66d9f 100644 --- a/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala +++ b/core/src/test/scala/org/apache/spark/HeartbeatReceiverSuite.scala @@ -23,12 +23,12 @@ import scala.collection.mutable import scala.concurrent.Future import scala.concurrent.duration._ -import org.mockito.Matchers -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{mock, spy, verify, when} import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.internal.config.DYN_ALLOCATION_TESTING import org.apache.spark.rpc.{RpcCallContext, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -68,7 +68,7 @@ class HeartbeatReceiverSuite val conf = new SparkConf() .setMaster("local[2]") .setAppName("test") - .set("spark.dynamicAllocation.testing", "true") + .set(DYN_ALLOCATION_TESTING, true) sc = spy(new SparkContext(conf)) scheduler = mock(classOf[TaskSchedulerImpl]) when(sc.taskScheduler).thenReturn(scheduler) @@ -151,7 +151,7 @@ class HeartbeatReceiverSuite heartbeatReceiverClock.advance(executorTimeout) heartbeatReceiverRef.askSync[Boolean](ExpireDeadHosts) // Only the second executor should be expired as a dead host - verify(scheduler).executorLost(Matchers.eq(executorId2), any()) + verify(scheduler).executorLost(meq(executorId2), any()) val trackedExecutors = getTrackedExecutors assert(trackedExecutors.size === 1) assert(trackedExecutors.contains(executorId1)) @@ -174,9 +174,11 @@ class HeartbeatReceiverSuite val dummyExecutorEndpointRef1 = rpcEnv.setupEndpoint("fake-executor-1", dummyExecutorEndpoint1) val dummyExecutorEndpointRef2 = rpcEnv.setupEndpoint("fake-executor-2", dummyExecutorEndpoint2) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( - RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty)) + RegisterExecutor(executorId1, dummyExecutorEndpointRef1, "1.2.3.4", 0, Map.empty, Map.empty, + Map.empty)) fakeSchedulerBackend.driverEndpoint.askSync[Boolean]( - RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty)) + RegisterExecutor(executorId2, dummyExecutorEndpointRef2, "1.2.3.5", 0, Map.empty, Map.empty, + Map.empty)) heartbeatReceiverRef.askSync[Boolean](TaskSchedulerIsSet) addExecutorAndVerify(executorId1) addExecutorAndVerify(executorId2) @@ -223,10 +225,10 @@ class HeartbeatReceiverSuite assert(!response.reregisterBlockManager) // Additionally verify that the scheduler callback is called with the correct parameters verify(scheduler).executorHeartbeatReceived( - Matchers.eq(executorId), - Matchers.eq(Array(1L -> metrics.accumulators())), - Matchers.eq(blockManagerId), - Matchers.eq(executorUpdates)) + meq(executorId), + meq(Array(1L -> metrics.accumulators())), + meq(blockManagerId), + meq(executorUpdates)) } } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 61da4138896cd..db90c3113a0bc 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.util.concurrent.Semaphore +import java.util.concurrent.{Semaphore, TimeUnit} import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.ExecutionContext.Implicits.global @@ -27,6 +27,8 @@ import scala.concurrent.duration._ import org.scalatest.BeforeAndAfter import org.scalatest.Matchers +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.{SparkListener, SparkListenerStageCompleted, SparkListenerTaskEnd, SparkListenerTaskStart} import org.apache.spark.util.ThreadUtils @@ -51,7 +53,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("local mode, FIFO scheduler") { - val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + val conf = new SparkConf().set(SCHEDULER_MODE, "FIFO") sc = new SparkContext("local[2]", "test", conf) testCount() testTake() @@ -60,9 +62,9 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("local mode, fair scheduler") { - val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set(SCHEDULER_MODE, "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - conf.set("spark.scheduler.allocation.file", xmlPath) + conf.set(SCHEDULER_ALLOCATION_FILE, xmlPath) sc = new SparkContext("local[2]", "test", conf) testCount() testTake() @@ -71,7 +73,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("cluster mode, FIFO scheduler") { - val conf = new SparkConf().set("spark.scheduler.mode", "FIFO") + val conf = new SparkConf().set(SCHEDULER_MODE, "FIFO") sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) testCount() testTake() @@ -80,9 +82,9 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft } test("cluster mode, fair scheduler") { - val conf = new SparkConf().set("spark.scheduler.mode", "FAIR") + val conf = new SparkConf().set(SCHEDULER_MODE, "FAIR") val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - conf.set("spark.scheduler.allocation.file", xmlPath) + conf.set(SCHEDULER_ALLOCATION_FILE, xmlPath) sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) testCount() testTake() @@ -216,8 +218,8 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft test("task reaper kills JVM if killed tasks keep running for too long") { val conf = new SparkConf() - .set("spark.task.reaper.enabled", "true") - .set("spark.task.reaper.killTimeout", "5s") + .set(TASK_REAPER_ENABLED, true) + .set(TASK_REAPER_KILL_TIMEOUT.key, "5s") sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) // Add a listener to release the semaphore once any tasks are launched. @@ -248,15 +250,15 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. - assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100) + assert(ThreadUtils.awaitResult(jobB, 1.minute) === 100) } test("task reaper will not kill JVM if spark.task.killTimeout == -1") { val conf = new SparkConf() - .set("spark.task.reaper.enabled", "true") - .set("spark.task.reaper.killTimeout", "-1") - .set("spark.task.reaper.PollingInterval", "1s") - .set("spark.deploy.maxExecutorRetries", "1") + .set(TASK_REAPER_ENABLED, true) + .set(TASK_REAPER_KILL_TIMEOUT.key, "-1") + .set(TASK_REAPER_POLLING_INTERVAL.key, "1s") + .set(MAX_EXECUTOR_RETRIES, 1) sc = new SparkContext("local-cluster[2,1,1024]", "test", conf) // Add a listener to release the semaphore once any tasks are launched. @@ -271,8 +273,8 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft val jobA = Future { sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) sc.parallelize(1 to 2, 2).map { i => - val startTime = System.currentTimeMillis() - while (System.currentTimeMillis() < startTime + 10000) { } + val startTimeNs = System.nanoTime() + while (System.nanoTime() < startTimeNs + TimeUnit.SECONDS.toNanos(10)) { } }.count() } @@ -288,7 +290,7 @@ class JobCancellationSuite extends SparkFunSuite with Matchers with BeforeAndAft assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. - assert(ThreadUtils.awaitResult(jobB, 60.seconds) === 100) + assert(ThreadUtils.awaitResult(jobB, 1.minute) === 100) } test("two jobs sharing the same stage") { diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 21f481d477242..d86975964b558 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -19,11 +19,13 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.broadcast.BroadcastManager +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network.{RPC_ASK_TIMEOUT, RPC_MESSAGE_MAX_SIZE} import org.apache.spark.rpc.{RpcAddress, RpcCallContext, RpcEnv} import org.apache.spark.scheduler.{CompressedMapStatus, MapStatus} import org.apache.spark.shuffle.FetchFailedException @@ -169,9 +171,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("remote fetch below max RPC message size") { val newConf = new SparkConf - newConf.set("spark.rpc.message.maxSize", "1") - newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "1048576") + newConf.set(RPC_MESSAGE_MAX_SIZE, 1) + newConf.set(RPC_ASK_TIMEOUT, "1") // Fail fast + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 1048576L) val masterTracker = newTrackerMaster(newConf) val rpcEnv = createRpcEnv("spark") @@ -198,9 +200,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("min broadcast size exceeds max RPC message size") { val newConf = new SparkConf - newConf.set("spark.rpc.message.maxSize", "1") - newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", Int.MaxValue.toString) + newConf.set(RPC_MESSAGE_MAX_SIZE, 1) + newConf.set(RPC_ASK_TIMEOUT, "1") // Fail fast + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, Int.MaxValue.toLong) intercept[IllegalArgumentException] { newTrackerMaster(newConf) } } @@ -242,9 +244,9 @@ class MapOutputTrackerSuite extends SparkFunSuite { test("remote fetch using broadcast") { val newConf = new SparkConf - newConf.set("spark.rpc.message.maxSize", "1") - newConf.set("spark.rpc.askTimeout", "1") // Fail fast - newConf.set("spark.shuffle.mapOutput.minSizeForBroadcast", "10240") // 10 KB << 1MB framesize + newConf.set(RPC_MESSAGE_MAX_SIZE, 1) + newConf.set(RPC_ASK_TIMEOUT, "1") // Fail fast + newConf.set(SHUFFLE_MAPOUTPUT_MIN_SIZE_FOR_BROADCAST, 10240L) // 10 KiB << 1MiB framesize // needs TorrentBroadcast so need a SparkContext withSpark(new SparkContext("local", "MapOutputTrackerSuite", newConf)) { sc => diff --git a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala index 5dbfc5c10a6f8..57d33971a57c6 100644 --- a/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/SSLOptionsSuite.scala @@ -55,11 +55,11 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl") - assert(opts.enabled === true) - assert(opts.trustStore.isDefined === true) + assert(opts.enabled) + assert(opts.trustStore.isDefined) assert(opts.trustStore.get.getName === "truststore") assert(opts.trustStore.get.getAbsolutePath === trustStorePath) - assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.isDefined) assert(opts.keyStore.get.getName === "keystore") assert(opts.keyStore.get.getAbsolutePath === keyStorePath) assert(opts.trustStorePassword === Some("password")) @@ -88,11 +88,11 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None) val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = Some(defaultOpts)) - assert(opts.enabled === true) - assert(opts.trustStore.isDefined === true) + assert(opts.enabled) + assert(opts.trustStore.isDefined) assert(opts.trustStore.get.getName === "truststore") assert(opts.trustStore.get.getAbsolutePath === trustStorePath) - assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.isDefined) assert(opts.keyStore.get.getName === "keystore") assert(opts.keyStore.get.getAbsolutePath === keyStorePath) assert(opts.trustStorePassword === Some("password")) @@ -128,10 +128,10 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { assert(opts.enabled === false) assert(opts.port === Some(4242)) - assert(opts.trustStore.isDefined === true) + assert(opts.trustStore.isDefined) assert(opts.trustStore.get.getName === "truststore") assert(opts.trustStore.get.getAbsolutePath === trustStorePath) - assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.isDefined) assert(opts.keyStore.get.getName === "keystore") assert(opts.keyStore.get.getAbsolutePath === keyStorePath) assert(opts.trustStorePassword === Some("password")) @@ -179,11 +179,11 @@ class SSLOptionsSuite extends SparkFunSuite with BeforeAndAfterAll { val defaultOpts = SSLOptions.parse(conf, hadoopConf, "spark.ssl", defaults = None) val opts = SSLOptions.parse(conf, hadoopConf, "spark.ssl.ui", defaults = Some(defaultOpts)) - assert(opts.enabled === true) - assert(opts.trustStore.isDefined === true) + assert(opts.enabled) + assert(opts.trustStore.isDefined) assert(opts.trustStore.get.getName === "truststore") assert(opts.trustStore.get.getAbsolutePath === trustStorePath) - assert(opts.keyStore.isDefined === true) + assert(opts.keyStore.isDefined) assert(opts.keyStore.get.getName === "keystore") assert(opts.keyStore.get.getAbsolutePath === keyStorePath) assert(opts.trustStorePassword === Some("password")) diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala index e357299770a2e..b31a6b4e2f9a9 100644 --- a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -19,11 +19,14 @@ package org.apache.spark import java.io.File import java.nio.charset.StandardCharsets.UTF_8 +import java.nio.file.Files import java.security.PrivilegedExceptionAction +import java.util.Base64 import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.util.{ResetSystemProperties, SparkConfWithEnv, Utils} @@ -41,24 +44,24 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with conf") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls", "user1,user2") - val securityManager = new SecurityManager(conf); - assert(securityManager.isAuthenticationEnabled() === true) - assert(securityManager.aclsEnabled() === true) - assert(securityManager.checkUIViewPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user2") === true) + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + val securityManager = new SecurityManager(conf) + assert(securityManager.isAuthenticationEnabled()) + assert(securityManager.aclsEnabled()) + assert(securityManager.checkUIViewPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user2")) assert(securityManager.checkUIViewPermissions("user3") === false) } test("set security with conf for groups") { val conf = new SparkConf - conf.set("spark.authenticate", "true") - conf.set("spark.authenticate.secret", "good") - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.ui.view.acls.groups", "group1,group2") + conf.set(NETWORK_AUTH_ENABLED, true) + conf.set(AUTH_SECRET, "good") + conf.set(ACLS_ENABLE, true) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // default ShellBasedGroupsMappingProvider is used to resolve user groups val securityManager = new SecurityManager(conf); // assuming executing user does not belong to group1,group2 @@ -66,27 +69,27 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf2.set("spark.authenticate", "true") - conf2.set("spark.authenticate.secret", "good") - conf2.set("spark.ui.acls.enable", "true") - conf2.set("spark.ui.view.acls.groups", "group1,group2") + conf2.set(NETWORK_AUTH_ENABLED, true) + conf2.set(AUTH_SECRET, "good") + conf2.set(ACLS_ENABLE, true) + conf2.set(UI_VIEW_ACLS_GROUPS, Seq("group1", "group2")) // explicitly specify a custom GroupsMappingServiceProvider - conf2.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf2.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager2 = new SecurityManager(conf2); + val securityManager2 = new SecurityManager(conf2) // group4,group5 do not match - assert(securityManager2.checkUIViewPermissions("user1") === true) - assert(securityManager2.checkUIViewPermissions("user2") === true) + assert(securityManager2.checkUIViewPermissions("user1")) + assert(securityManager2.checkUIViewPermissions("user2")) val conf3 = new SparkConf - conf3.set("spark.authenticate", "true") - conf3.set("spark.authenticate.secret", "good") - conf3.set("spark.ui.acls.enable", "true") - conf3.set("spark.ui.view.acls.groups", "group4,group5") + conf3.set(NETWORK_AUTH_ENABLED, true) + conf3.set(AUTH_SECRET, "good") + conf3.set(ACLS_ENABLE, true) + conf3.set(UI_VIEW_ACLS_GROUPS, Seq("group4", "group5")) // explicitly specify a bogus GroupsMappingServiceProvider - conf3.set("spark.user.groups.mapping", "BogusServiceProvider") + conf3.set(USER_GROUPS_MAPPING, "BogusServiceProvider") - val securityManager3 = new SecurityManager(conf3); + val securityManager3 = new SecurityManager(conf3) // BogusServiceProvider cannot be loaded and an error is logged returning an empty group set assert(securityManager3.checkUIViewPermissions("user1") === false) assert(securityManager3.checkUIViewPermissions("user2") === false) @@ -94,263 +97,268 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { test("set security with api") { val conf = new SparkConf - conf.set("spark.ui.view.acls", "user1,user2") + conf.set(UI_VIEW_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) securityManager.setAcls(false) assert(securityManager.aclsEnabled() === false) // acls are off so doesn't matter what view acls set to - assert(securityManager.checkUIViewPermissions("user4") === true) + assert(securityManager.checkUIViewPermissions("user4")) securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) - securityManager.setViewAcls(Set[String]("user5"), "user6,user7") + assert(securityManager.aclsEnabled()) + securityManager.setViewAcls(Set[String]("user5"), Seq("user6", "user7")) assert(securityManager.checkUIViewPermissions("user1") === false) - assert(securityManager.checkUIViewPermissions("user5") === true) - assert(securityManager.checkUIViewPermissions("user6") === true) - assert(securityManager.checkUIViewPermissions("user7") === true) + assert(securityManager.checkUIViewPermissions("user5")) + assert(securityManager.checkUIViewPermissions("user6")) + assert(securityManager.checkUIViewPermissions("user7")) assert(securityManager.checkUIViewPermissions("user8") === false) - assert(securityManager.checkUIViewPermissions(null) === true) + assert(securityManager.checkUIViewPermissions(null)) } test("set security with api for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setViewAclsGroups("group1,group2") + securityManager.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 match - assert(securityManager.checkUIViewPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user2")) // change groups so they do not match - securityManager.setViewAclsGroups("group4,group5") + securityManager.setViewAclsGroups(Seq("group4", "group5")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) val conf2 = new SparkConf - conf.set("spark.user.groups.mapping", "BogusServiceProvider") + conf.set(USER_GROUPS_MAPPING, "BogusServiceProvider") val securityManager2 = new SecurityManager(conf2) securityManager2.setAcls(true) - securityManager2.setViewAclsGroups("group1,group2") + securityManager2.setViewAclsGroups(Seq("group1", "group2")) // group1,group2 do not match because of BogusServiceProvider assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) // setting viewAclsGroups to empty should still not match because of BogusServiceProvider - securityManager2.setViewAclsGroups("") + securityManager2.setViewAclsGroups(Nil) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) } test("set security modify acls") { val conf = new SparkConf - conf.set("spark.modify.acls", "user1,user2") + conf.set(MODIFY_ACLS, Seq("user1", "user2")) val securityManager = new SecurityManager(conf); securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) securityManager.setAcls(false) assert(securityManager.aclsEnabled() === false) // acls are off so doesn't matter what view acls set to - assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkModifyPermissions("user4")) securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) - securityManager.setModifyAcls(Set("user5"), "user6,user7") + assert(securityManager.aclsEnabled()) + securityManager.setModifyAcls(Set("user5"), Seq("user6", "user7")) assert(securityManager.checkModifyPermissions("user1") === false) - assert(securityManager.checkModifyPermissions("user5") === true) - assert(securityManager.checkModifyPermissions("user6") === true) - assert(securityManager.checkModifyPermissions("user7") === true) + assert(securityManager.checkModifyPermissions("user5")) + assert(securityManager.checkModifyPermissions("user6")) + assert(securityManager.checkModifyPermissions("user7")) assert(securityManager.checkModifyPermissions("user8") === false) - assert(securityManager.checkModifyPermissions(null) === true) + assert(securityManager.checkModifyPermissions(null)) } test("set security modify acls for groups") { val conf = new SparkConf - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - securityManager.setModifyAclsGroups("group1,group2") + securityManager.setModifyAclsGroups(Seq("group1", "group2")) // group1,group2 match - assert(securityManager.checkModifyPermissions("user1") === true) - assert(securityManager.checkModifyPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1")) + assert(securityManager.checkModifyPermissions("user2")) // change groups so they do not match - securityManager.setModifyAclsGroups("group4,group5") + securityManager.setModifyAclsGroups(Seq("group4", "group5")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user2") === false) // change so they match again - securityManager.setModifyAclsGroups("group2,group3") - assert(securityManager.checkModifyPermissions("user1") === true) - assert(securityManager.checkModifyPermissions("user2") === true) + securityManager.setModifyAclsGroups(Seq("group2", "group3")) + + assert(securityManager.checkModifyPermissions("user1")) + assert(securityManager.checkModifyPermissions("user2")) } test("set security admin acls") { val conf = new SparkConf - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "user3") - conf.set("spark.modify.acls", "user4") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("user3")) + conf.set(MODIFY_ACLS, Seq("user4")) - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) - assert(securityManager.checkModifyPermissions("user1") === true) - assert(securityManager.checkModifyPermissions("user2") === true) - assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkModifyPermissions("user1")) + assert(securityManager.checkModifyPermissions("user2")) + assert(securityManager.checkModifyPermissions("user4")) assert(securityManager.checkModifyPermissions("user3") === false) assert(securityManager.checkModifyPermissions("user5") === false) - assert(securityManager.checkModifyPermissions(null) === true) - assert(securityManager.checkUIViewPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user2") === true) - assert(securityManager.checkUIViewPermissions("user3") === true) + assert(securityManager.checkModifyPermissions(null)) + assert(securityManager.checkUIViewPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user2")) + assert(securityManager.checkUIViewPermissions("user3")) assert(securityManager.checkUIViewPermissions("user4") === false) assert(securityManager.checkUIViewPermissions("user5") === false) - assert(securityManager.checkUIViewPermissions(null) === true) - - securityManager.setAdminAcls("user6") - securityManager.setViewAcls(Set[String]("user8"), "user9") - securityManager.setModifyAcls(Set("user11"), "user9") - assert(securityManager.checkModifyPermissions("user6") === true) - assert(securityManager.checkModifyPermissions("user11") === true) - assert(securityManager.checkModifyPermissions("user9") === true) + assert(securityManager.checkUIViewPermissions(null)) + + securityManager.setAdminAcls(Seq("user6")) + securityManager.setViewAcls(Set[String]("user8"), Seq("user9")) + securityManager.setModifyAcls(Set("user11"), Seq("user9")) + assert(securityManager.checkAdminPermissions("user6")) + assert(!securityManager.checkAdminPermissions("user8")) + assert(securityManager.checkModifyPermissions("user6")) + assert(securityManager.checkModifyPermissions("user11")) + assert(securityManager.checkModifyPermissions("user9")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user4") === false) - assert(securityManager.checkModifyPermissions(null) === true) - assert(securityManager.checkUIViewPermissions("user6") === true) - assert(securityManager.checkUIViewPermissions("user8") === true) - assert(securityManager.checkUIViewPermissions("user9") === true) + assert(securityManager.checkModifyPermissions(null)) + assert(securityManager.checkUIViewPermissions("user6")) + assert(securityManager.checkUIViewPermissions("user8")) + assert(securityManager.checkUIViewPermissions("user9")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user3") === false) - assert(securityManager.checkUIViewPermissions(null) === true) + assert(securityManager.checkUIViewPermissions(null)) } test("set security admin acls for groups") { val conf = new SparkConf - conf.set("spark.admin.acls.groups", "group1") - conf.set("spark.ui.view.acls.groups", "group2") - conf.set("spark.modify.acls.groups", "group3") - conf.set("spark.user.groups.mapping", "org.apache.spark.DummyGroupMappingServiceProvider") + conf.set(ADMIN_ACLS_GROUPS, Seq("group1")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("group2")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group3")) + conf.set(USER_GROUPS_MAPPING, "org.apache.spark.DummyGroupMappingServiceProvider") - val securityManager = new SecurityManager(conf); + val securityManager = new SecurityManager(conf) securityManager.setAcls(true) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) // group1,group2,group3 match - assert(securityManager.checkModifyPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkAdminPermissions("user1")) + assert(securityManager.checkModifyPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user1")) // change admin groups so they do not match. view and modify groups are set to admin groups - securityManager.setAdminAclsGroups("group4,group5") + securityManager.setAdminAclsGroups(Seq("group4", "group5")) // invoke the set ui and modify to propagate the changes - securityManager.setViewAclsGroups("") - securityManager.setModifyAclsGroups("") + securityManager.setViewAclsGroups(Nil) + securityManager.setModifyAclsGroups(Nil) - assert(securityManager.checkModifyPermissions("user1") === false) - assert(securityManager.checkUIViewPermissions("user1") === false) + assert(!securityManager.checkAdminPermissions("user1")) + assert(!securityManager.checkModifyPermissions("user1")) + assert(!securityManager.checkUIViewPermissions("user1")) // change modify groups so they match - securityManager.setModifyAclsGroups("group3") - assert(securityManager.checkModifyPermissions("user1") === true) + securityManager.setModifyAclsGroups(Seq("group3")) + assert(securityManager.checkModifyPermissions("user1")) assert(securityManager.checkUIViewPermissions("user1") === false) // change view groups so they match - securityManager.setViewAclsGroups("group2") - securityManager.setModifyAclsGroups("group4") + securityManager.setViewAclsGroups(Seq("group2")) + securityManager.setModifyAclsGroups(Seq("group4")) assert(securityManager.checkModifyPermissions("user1") === false) - assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user1")) // change modify and view groups so they do not match - securityManager.setViewAclsGroups("group7") - securityManager.setModifyAclsGroups("group8") + securityManager.setViewAclsGroups(Seq("group7")) + securityManager.setModifyAclsGroups(Seq("group8")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user1") === false) } test("set security with * in acls") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls", "user1,user2") - conf.set("spark.ui.view.acls", "*") - conf.set("spark.modify.acls", "user4") + conf.set(ACLS_ENABLE.key, "true") + conf.set(ADMIN_ACLS, Seq("user1", "user2")) + conf.set(UI_VIEW_ACLS, Seq("*")) + conf.set(MODIFY_ACLS, Seq("user4")) val securityManager = new SecurityManager(conf) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) // check for viewAcls with * - assert(securityManager.checkUIViewPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user5") === true) - assert(securityManager.checkUIViewPermissions("user6") === true) - assert(securityManager.checkModifyPermissions("user4") === true) + assert(securityManager.checkUIViewPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user5")) + assert(securityManager.checkUIViewPermissions("user6")) + assert(securityManager.checkModifyPermissions("user4")) assert(securityManager.checkModifyPermissions("user7") === false) assert(securityManager.checkModifyPermissions("user8") === false) // check for modifyAcls with * - securityManager.setModifyAcls(Set("user4"), "*") - assert(securityManager.checkModifyPermissions("user7") === true) - assert(securityManager.checkModifyPermissions("user8") === true) + securityManager.setModifyAcls(Set("user4"), Seq("*")) + assert(securityManager.checkModifyPermissions("user7")) + assert(securityManager.checkModifyPermissions("user8")) - securityManager.setAdminAcls("user1,user2") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") + securityManager.setAdminAcls(Seq("user1", "user2")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) assert(securityManager.checkUIViewPermissions("user5") === false) assert(securityManager.checkUIViewPermissions("user6") === false) assert(securityManager.checkModifyPermissions("user7") === false) assert(securityManager.checkModifyPermissions("user8") === false) // check for adminAcls with * - securityManager.setAdminAcls("user1,*") - securityManager.setModifyAcls(Set("user1"), "user2") - securityManager.setViewAcls(Set("user1"), "user2") - assert(securityManager.checkUIViewPermissions("user5") === true) - assert(securityManager.checkUIViewPermissions("user6") === true) - assert(securityManager.checkModifyPermissions("user7") === true) - assert(securityManager.checkModifyPermissions("user8") === true) + securityManager.setAdminAcls(Seq("user1", "*")) + securityManager.setModifyAcls(Set("user1"), Seq("user2")) + securityManager.setViewAcls(Set("user1"), Seq("user2")) + assert(securityManager.checkUIViewPermissions("user5")) + assert(securityManager.checkUIViewPermissions("user6")) + assert(securityManager.checkModifyPermissions("user7")) + assert(securityManager.checkModifyPermissions("user8")) } test("set security with * in acls for groups") { val conf = new SparkConf - conf.set("spark.ui.acls.enable", "true") - conf.set("spark.admin.acls.groups", "group4,group5") - conf.set("spark.ui.view.acls.groups", "*") - conf.set("spark.modify.acls.groups", "group6") + conf.set(ACLS_ENABLE, true) + conf.set(ADMIN_ACLS_GROUPS, Seq("group4", "group5")) + conf.set(UI_VIEW_ACLS_GROUPS, Seq("*")) + conf.set(MODIFY_ACLS_GROUPS, Seq("group6")) val securityManager = new SecurityManager(conf) - assert(securityManager.aclsEnabled() === true) + assert(securityManager.aclsEnabled()) // check for viewAclsGroups with * - assert(securityManager.checkUIViewPermissions("user1") === true) - assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user1")) + assert(securityManager.checkUIViewPermissions("user2")) assert(securityManager.checkModifyPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user2") === false) // check for modifyAcls with * - securityManager.setModifyAclsGroups("*") - securityManager.setViewAclsGroups("group6") + securityManager.setModifyAclsGroups(Seq("*")) + securityManager.setViewAclsGroups(Seq("group6")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkUIViewPermissions("user2") === false) - assert(securityManager.checkModifyPermissions("user1") === true) - assert(securityManager.checkModifyPermissions("user2") === true) + assert(securityManager.checkModifyPermissions("user1")) + assert(securityManager.checkModifyPermissions("user2")) // check for adminAcls with * - securityManager.setAdminAclsGroups("group9,*") - securityManager.setModifyAclsGroups("group4,group5") - securityManager.setViewAclsGroups("group6,group7") - assert(securityManager.checkUIViewPermissions("user5") === true) - assert(securityManager.checkUIViewPermissions("user6") === true) - assert(securityManager.checkModifyPermissions("user7") === true) - assert(securityManager.checkModifyPermissions("user8") === true) + securityManager.setAdminAclsGroups(Seq("group9", "*")) + securityManager.setModifyAclsGroups(Seq("group4", "group5")) + securityManager.setViewAclsGroups(Seq("group6", "group7")) + assert(securityManager.checkUIViewPermissions("user5")) + assert(securityManager.checkUIViewPermissions("user6")) + assert(securityManager.checkModifyPermissions("user7")) + assert(securityManager.checkModifyPermissions("user8")) } test("security for groups default behavior") { @@ -365,13 +373,13 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(securityManager.checkModifyPermissions("user1") === false) // set groups only - securityManager.setAdminAclsGroups("group1,group2") + securityManager.setAdminAclsGroups(Seq("group1", "group2")) assert(securityManager.checkUIViewPermissions("user1") === false) assert(securityManager.checkModifyPermissions("user1") === false) } test("missing secret authentication key") { - val conf = new SparkConf().set("spark.authenticate", "true") + val conf = new SparkConf().set(NETWORK_AUTH_ENABLED, true) val mgr = new SecurityManager(conf) intercept[IllegalArgumentException] { mgr.getSecretKey() @@ -395,15 +403,69 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { assert(keyFromEnv === new SecurityManager(conf2).getSecretKey()) } - test("secret key generation") { - Seq( - ("yarn", true), - ("local", true), - ("local[*]", true), - ("local[1, 2]", true), - ("local-cluster[2, 1, 1024]", false), - ("invalid", false) - ).foreach { case (master, shouldGenerateSecret) => + test("use executor-specific secret file configuration.") { + val secretFileFromDriver = createTempSecretFile("driver-secret") + val secretFileFromExecutor = createTempSecretFile("executor-secret") + val conf = new SparkConf() + .setMaster("k8s://127.0.0.1") + .set(AUTH_SECRET_FILE_DRIVER, Some(secretFileFromDriver.getAbsolutePath)) + .set(AUTH_SECRET_FILE_EXECUTOR, Some(secretFileFromExecutor.getAbsolutePath)) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr = new SecurityManager(conf, authSecretFileConf = AUTH_SECRET_FILE_EXECUTOR) + assert(encodeFileAsBase64(secretFileFromExecutor) === mgr.getSecretKey()) + } + + test("secret file must be defined in both driver and executor") { + val conf1 = new SparkConf() + .set(AUTH_SECRET_FILE_DRIVER, Some("/tmp/driver-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr1 = new SecurityManager(conf1) + intercept[IllegalArgumentException] { + mgr1.initializeAuth() + } + + val conf2 = new SparkConf() + .set(AUTH_SECRET_FILE_EXECUTOR, Some("/tmp/executor-secret.txt")) + .set(SecurityManager.SPARK_AUTH_CONF, "true") + val mgr2 = new SecurityManager(conf2) + intercept[IllegalArgumentException] { + mgr2.initializeAuth() + } + } + + Seq("yarn", "local", "local[*]", "local[1,2]", "mesos://localhost:8080").foreach { master => + test(s"master $master cannot use file mounted secrets") { + val conf = new SparkConf() + .set(AUTH_SECRET_FILE, "/tmp/secret.txt") + .set(SecurityManager.SPARK_AUTH_CONF, "true") + .setMaster(master) + intercept[IllegalArgumentException] { + new SecurityManager(conf).getSecretKey() + } + intercept[IllegalArgumentException] { + new SecurityManager(conf).initializeAuth() + } + } + } + + // How is the secret expected to be generated and stored. + object SecretTestType extends Enumeration { + val MANUAL, AUTO, UGI, FILE = Value + } + + import SecretTestType._ + + Seq( + ("yarn", UGI), + ("local", UGI), + ("local[*]", UGI), + ("local[1, 2]", UGI), + ("k8s://127.0.0.1", AUTO), + ("k8s://127.0.1.1", FILE), + ("local-cluster[2, 1, 1024]", MANUAL), + ("invalid", MANUAL) + ).foreach { case (master, secretType) => + test(s"secret key generation: master '$master'") { val conf = new SparkConf() .set(NETWORK_AUTH_ENABLED, true) .set(SparkLauncher.SPARK_MASTER, master) @@ -412,19 +474,32 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { UserGroupInformation.createUserForTesting("authTest", Array()).doAs( new PrivilegedExceptionAction[Unit]() { override def run(): Unit = { - if (shouldGenerateSecret) { - mgr.initializeAuth() - val creds = UserGroupInformation.getCurrentUser().getCredentials() - val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) - assert(secret != null) - assert(new String(secret, UTF_8) === mgr.getSecretKey()) - } else { - intercept[IllegalArgumentException] { + secretType match { + case UGI => + mgr.initializeAuth() + val creds = UserGroupInformation.getCurrentUser().getCredentials() + val secret = creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) + assert(secret != null) + assert(new String(secret, UTF_8) === mgr.getSecretKey()) + + case AUTO => mgr.initializeAuth() - } - intercept[IllegalArgumentException] { - mgr.getSecretKey() - } + val creds = UserGroupInformation.getCurrentUser().getCredentials() + assert(creds.getSecretKey(SecurityManager.SECRET_LOOKUP_KEY) === null) + + case MANUAL => + intercept[IllegalArgumentException] { + mgr.initializeAuth() + } + intercept[IllegalArgumentException] { + mgr.getSecretKey() + } + + case FILE => + val secretFile = createTempSecretFile() + conf.set(AUTH_SECRET_FILE, secretFile.getAbsolutePath) + mgr.initializeAuth() + assert(encodeFileAsBase64(secretFile) === mgr.getSecretKey()) } } } @@ -432,5 +507,15 @@ class SecurityManagerSuite extends SparkFunSuite with ResetSystemProperties { } } + private def encodeFileAsBase64(secretFile: File) = { + Base64.getEncoder.encodeToString(Files.readAllBytes(secretFile.toPath)) + } + + private def createTempSecretFile(contents: String = "test-secret"): File = { + val secretDir = Utils.createTempDir("temp-secrets") + val secretFile = new File(secretDir, "temp-secret.txt") + Files.write(secretFile.toPath, contents.getBytes(UTF_8)) + secretFile + } } diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index b917469e48747..8b1084a8edc76 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -23,6 +23,8 @@ import java.util.concurrent.{Callable, CyclicBarrier, Executors, ExecutorService import org.scalatest.Matchers import org.apache.spark.ShuffleSuite.NonJavaSerializableClass +import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.{CoGroupedRDD, OrderedRDDFunctions, RDD, ShuffledRDD, SubtractedRDD} import org.apache.spark.scheduler.{MapStatus, MyRDD, SparkListener, SparkListenerTaskEnd} @@ -37,10 +39,10 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC // Ensure that the DAGScheduler doesn't retry stages whose fetches fail, so that we accurately // test that the shuffle works (rather than retrying until all blocks are local to one Executor). - conf.set("spark.test.noStageRetry", "true") + conf.set(TEST_NO_STAGE_RETRY, true) test("groupByKey without compression") { - val myConf = conf.clone().set("spark.shuffle.compress", "false") + val myConf = conf.clone().set(config.SHUFFLE_COMPRESS, false) sc = new SparkContext("local", "test", myConf) val pairs = sc.parallelize(Array((1, 1), (1, 2), (1, 3), (2, 1)), 4) val groups = pairs.groupByKey(4).collect() @@ -214,7 +216,7 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC test("sort with Java non serializable class - Kryo") { // Use a local cluster with 2 processes to make sure there are both local and remote blocks - val myConf = conf.clone().set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + val myConf = conf.clone().set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") sc = new SparkContext("local-cluster[2,1,1024]", "test", myConf) val a = sc.parallelize(1 to 10, 2) val b = a.map { x => @@ -250,8 +252,8 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC val myConf = conf.clone() .setAppName("test") .setMaster("local") - .set("spark.shuffle.spill.compress", shuffleSpillCompress.toString) - .set("spark.shuffle.compress", shuffleCompress.toString) + .set(config.SHUFFLE_SPILL_COMPRESS, shuffleSpillCompress) + .set(config.SHUFFLE_COMPRESS, shuffleCompress) resetSparkContext() sc = new SparkContext(myConf) val diskBlockManager = sc.env.blockManager.diskBlockManager @@ -261,15 +263,15 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC assert(diskBlockManager.getAllFiles().nonEmpty) } catch { case e: Exception => - val errMsg = s"Failed with spark.shuffle.spill.compress=$shuffleSpillCompress," + - s" spark.shuffle.compress=$shuffleCompress" + val errMsg = s"Failed with ${config.SHUFFLE_SPILL_COMPRESS.key}=$shuffleSpillCompress," + + s" ${config.SHUFFLE_COMPRESS.key}=$shuffleCompress" throw new Exception(errMsg, e) } } } test("[SPARK-4085] rerun map stage if reduce stage cannot find its local shuffle file") { - val myConf = conf.clone().set("spark.test.noStageRetry", "false") + val myConf = conf.clone().set(TEST_NO_STAGE_RETRY, false) sc = new SparkContext("local", "test", myConf) val rdd = sc.parallelize(1 to 10, 2).map((_, 1)).reduceByKey(_ + _) rdd.count() @@ -362,15 +364,19 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC mapTrackerMaster.registerShuffle(0, 1) // first attempt -- its successful - val writer1 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem)) + val context1 = + new TaskContextImpl(0, 0, 0, 0L, 0, taskMemoryManager, new Properties, metricsSystem) + val writer1 = manager.getWriter[Int, Int]( + shuffleHandle, 0, context1, context1.taskMetrics.shuffleWriteMetrics) val data1 = (1 to 10).map { x => x -> x} // second attempt -- also successful. We'll write out different data, // just to simulate the fact that the records may get written differently // depending on what gets spilled, what gets combined, etc. - val writer2 = manager.getWriter[Int, Int](shuffleHandle, 0, - new TaskContextImpl(0, 0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem)) + val context2 = + new TaskContextImpl(0, 0, 0, 1L, 0, taskMemoryManager, new Properties, metricsSystem) + val writer2 = manager.getWriter[Int, Int]( + shuffleHandle, 0, context2, context2.taskMetrics.shuffleWriteMetrics) val data2 = (11 to 20).map { x => x -> x} // interleave writes of both attempts -- we want to test that both attempts can occur @@ -397,8 +403,10 @@ abstract class ShuffleSuite extends SparkFunSuite with Matchers with LocalSparkC mapTrackerMaster.registerMapOutput(0, 0, mapStatus) } - val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, - new TaskContextImpl(1, 0, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem)) + val taskContext = new TaskContextImpl( + 1, 0, 0, 2L, 0, taskMemoryManager, new Properties, metricsSystem) + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() + val reader = manager.getReader[Int, Int](shuffleHandle, 0, 1, taskContext, metrics) val readData = reader.read().toIndexedSeq assert(readData === data1.toIndexedSeq || readData === data2.toIndexedSeq) diff --git a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala index c0126e41ff7fa..1aceda498d7c7 100644 --- a/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/SortShuffleSuite.scala @@ -25,6 +25,7 @@ import org.apache.commons.io.FileUtils import org.apache.commons.io.filefilter.TrueFileFilter import org.scalatest.BeforeAndAfterAll +import org.apache.spark.internal.config import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.shuffle.sort.SortShuffleManager @@ -42,7 +43,7 @@ class SortShuffleSuite extends ShuffleSuite with BeforeAndAfterAll { // before/after a test, it could return the same directory even if this property // is configured. Utils.clearLocalRootDirs() - conf.set("spark.shuffle.manager", "sort") + conf.set(config.SHUFFLE_MANAGER, "sort") } override def beforeEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala index 0d06b02e74e34..202b85dcf5695 100644 --- a/core/src/test/scala/org/apache/spark/SparkConfSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkConfSuite.scala @@ -21,16 +21,20 @@ import java.util.concurrent.{Executors, TimeUnit} import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import scala.language.postfixOps import scala.util.{Random, Try} import com.esotericsoftware.kryo.Kryo -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.internal.config.Network._ import org.apache.spark.network.util.ByteUnit +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.serializer.{JavaSerializer, KryoRegistrator, KryoSerializer} -import org.apache.spark.util.{ResetSystemProperties, RpcUtils} +import org.apache.spark.util.{ResetSystemProperties, RpcUtils, Utils} class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties { test("Test byteString conversion") { @@ -78,7 +82,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(conf.get("spark.master") === "local[3]") assert(conf.get("spark.app.name") === "My app") assert(conf.get("spark.home") === "/path") - assert(conf.get("spark.jars") === "a.jar,b.jar") + assert(conf.get(JARS) === Seq("a.jar", "b.jar")) assert(conf.get("spark.executorEnv.VAR1") === "value1") assert(conf.get("spark.executorEnv.VAR2") === "value2") assert(conf.get("spark.executorEnv.VAR3") === "value3") @@ -86,7 +90,7 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // Test the Java-friendly versions of these too conf.setJars(Array("c.jar", "d.jar")) conf.setExecutorEnv(Array(("VAR4", "value4"), ("VAR5", "value5"))) - assert(conf.get("spark.jars") === "c.jar,d.jar") + assert(conf.get(JARS) === Seq("c.jar", "d.jar")) assert(conf.get("spark.executorEnv.VAR4") === "value4") assert(conf.get("spark.executorEnv.VAR5") === "value5") } @@ -109,6 +113,21 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(conf.getOption("k4") === None) } + test("basic getAllWithPrefix") { + val prefix = "spark.prefix." + val conf = new SparkConf(false) + conf.set("spark.prefix.main.suffix", "v1") + assert(conf.getAllWithPrefix(prefix).toSet === + Set(("main.suffix", "v1"))) + + conf.set("spark.prefix.main2.suffix", "v2") + conf.set("spark.prefix.main3.extra1.suffix", "v3") + conf.set("spark.notMatching.main4", "v4") + + assert(conf.getAllWithPrefix(prefix).toSet === + Set(("main.suffix", "v1"), ("main2.suffix", "v2"), ("main3.extra1.suffix", "v3"))) + } + test("creating SparkContext without master and app name") { val conf = new SparkConf(false) intercept[SparkException] { sc = new SparkContext(conf) } @@ -155,16 +174,15 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst test("Thread safeness - SPARK-5425") { val executor = Executors.newSingleThreadScheduledExecutor() - val sf = executor.scheduleAtFixedRate(new Runnable { - override def run(): Unit = - System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString) - }, 0, 1, TimeUnit.MILLISECONDS) + executor.scheduleAtFixedRate( + () => System.setProperty("spark.5425." + Random.nextInt(), Random.nextInt().toString), + 0, 1, TimeUnit.MILLISECONDS) try { - val t0 = System.currentTimeMillis() - while ((System.currentTimeMillis() - t0) < 1000) { + val t0 = System.nanoTime() + while ((System.nanoTime() - t0) < TimeUnit.SECONDS.toNanos(1)) { val conf = Try(new SparkConf(loadDefaults = true)) - assert(conf.isSuccess === true) + assert(conf.isSuccess) } } finally { executor.shutdownNow() @@ -175,19 +193,19 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1], classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName).toSet) conf.registerKryoClasses(Array(classOf[Class3])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) conf.registerKryoClasses(Array(classOf[Class2])) - assert(conf.get("spark.kryo.classesToRegister") === - classOf[Class1].getName + "," + classOf[Class2].getName + "," + classOf[Class3].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === + Seq(classOf[Class1].getName, classOf[Class2].getName, classOf[Class3].getName).toSet) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -198,12 +216,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through registerKryoClasses and custom registrator") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) conf.registerKryoClasses(Array(classOf[Class1])) - assert(conf.get("spark.kryo.classesToRegister") === classOf[Class1].getName) + assert(conf.get(KRYO_CLASSES_TO_REGISTER).toSet === Seq(classOf[Class1].getName).toSet) - conf.set("spark.kryo.registrator", classOf[CustomRegistrator].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[CustomRegistrator].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -213,9 +231,9 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("register kryo classes through conf") { - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") - conf.set("spark.kryo.classesToRegister", "java.lang.StringBuffer") - conf.set("spark.serializer", classOf[KryoSerializer].getName) + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) + conf.set(KRYO_CLASSES_TO_REGISTER, Seq("java.lang.StringBuffer")) + conf.set(SERIALIZER, classOf[KryoSerializer].getName) // Kryo doesn't expose a way to discover registered classes, but at least make sure this doesn't // blow up. @@ -224,8 +242,8 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst } test("deprecated configs") { - val conf = new SparkConf() - val newName = "spark.history.fs.update.interval" + val conf = new SparkConf(false) + val newName = UPDATE_INTERVAL_S.key assert(!conf.contains(newName)) @@ -248,22 +266,28 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(conf.getTimeAsSeconds("spark.yarn.am.waitTime") === 420) conf.set("spark.kryoserializer.buffer.mb", "1.1") - assert(conf.getSizeAsKb("spark.kryoserializer.buffer") === 1100) + assert(conf.getSizeAsKb(KRYO_SERIALIZER_BUFFER_SIZE.key) === 1100) conf.set("spark.history.fs.cleaner.maxAge.seconds", "42") assert(conf.get(MAX_LOG_AGE_S) === 42L) conf.set("spark.scheduler.listenerbus.eventqueue.size", "84") assert(conf.get(LISTENER_BUS_EVENT_QUEUE_CAPACITY) === 84) + + conf.set("spark.yarn.access.namenodes", "testNode") + assert(conf.get(KERBEROS_FILESYSTEMS_TO_ACCESS) === Array("testNode")) + + conf.set("spark.yarn.access.hadoopFileSystems", "testNode") + assert(conf.get(KERBEROS_FILESYSTEMS_TO_ACCESS) === Array("testNode")) } test("akka deprecated configs") { val conf = new SparkConf() - assert(!conf.contains("spark.rpc.numRetries")) - assert(!conf.contains("spark.rpc.retry.wait")) - assert(!conf.contains("spark.rpc.askTimeout")) - assert(!conf.contains("spark.rpc.lookupTimeout")) + assert(!conf.contains(RPC_NUM_RETRIES)) + assert(!conf.contains(RPC_RETRY_WAIT)) + assert(!conf.contains(RPC_ASK_TIMEOUT)) + assert(!conf.contains(RPC_LOOKUP_TIMEOUT)) conf.set("spark.akka.num.retries", "1") assert(RpcUtils.numRetries(conf) === 1) @@ -272,10 +296,10 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(RpcUtils.retryWaitMs(conf) === 2L) conf.set("spark.akka.askTimeout", "3") - assert(RpcUtils.askRpcTimeout(conf).duration === (3 seconds)) + assert(RpcUtils.askRpcTimeout(conf).duration === 3.seconds) conf.set("spark.akka.lookupTimeout", "4") - assert(RpcUtils.lookupRpcTimeout(conf).duration === (4 seconds)) + assert(RpcUtils.lookupRpcTimeout(conf).duration === 4.seconds) } test("SPARK-13727") { @@ -283,12 +307,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst // set the conf in the deprecated way conf.set("spark.io.compression.lz4.block.size", "12345") // get the conf in the recommended way - assert(conf.get("spark.io.compression.lz4.blockSize") === "12345") + assert(conf.get(IO_COMPRESSION_LZ4_BLOCKSIZE.key) === "12345") // we can still get the conf in the deprecated way assert(conf.get("spark.io.compression.lz4.block.size") === "12345") // the contains() also works as expected assert(conf.contains("spark.io.compression.lz4.block.size")) - assert(conf.contains("spark.io.compression.lz4.blockSize")) + assert(conf.contains(IO_COMPRESSION_LZ4_BLOCKSIZE.key)) assert(conf.contains("spark.io.unknown") === false) } @@ -314,12 +338,12 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst val conf = new SparkConf() conf.validateSettings() - conf.set(NETWORK_ENCRYPTION_ENABLED, true) + conf.set(NETWORK_CRYPTO_ENABLED, true) intercept[IllegalArgumentException] { conf.validateSettings() } - conf.set(NETWORK_ENCRYPTION_ENABLED, false) + conf.set(NETWORK_CRYPTO_ENABLED, false) conf.set(SASL_ENCRYPTION_ENABLED, true) intercept[IllegalArgumentException] { conf.validateSettings() @@ -333,12 +357,51 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst val conf = new SparkConf() conf.validateSettings() - conf.set("spark.network.timeout", "5s") + conf.set(NETWORK_TIMEOUT.key, "5s") intercept[IllegalArgumentException] { conf.validateSettings() } } + test("SPARK-26998: SSL configuration not needed on executors") { + val conf = new SparkConf(false) + conf.set("spark.ssl.enabled", "true") + conf.set("spark.ssl.keyPassword", "password") + conf.set("spark.ssl.keyStorePassword", "password") + conf.set("spark.ssl.trustStorePassword", "password") + + val filtered = conf.getAll.filter { case (k, _) => SparkConf.isExecutorStartupConf(k) } + assert(filtered.isEmpty) + } + + test("SPARK-27244 toDebugString redacts sensitive information") { + val conf = new SparkConf(loadDefaults = false) + .set("dummy.password", "dummy-password") + .set("spark.hadoop.hive.server2.keystore.password", "1234") + .set("spark.hadoop.javax.jdo.option.ConnectionPassword", "1234") + .set("spark.regular.property", "regular_value") + assert(conf.toDebugString == + s""" + |dummy.password=${Utils.REDACTION_REPLACEMENT_TEXT} + |spark.hadoop.hive.server2.keystore.password=${Utils.REDACTION_REPLACEMENT_TEXT} + |spark.hadoop.javax.jdo.option.ConnectionPassword=${Utils.REDACTION_REPLACEMENT_TEXT} + |spark.regular.property=regular_value + """.stripMargin.trim) + } + + test("SPARK-28355: Use Spark conf for threshold at which UDFs are compressed by broadcast") { + val conf = new SparkConf() + + // Check the default value + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024 * 1024) + + // Set the conf + conf.set(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD, 1L * 1024) + + // Verify that it has been set properly + assert(conf.get(BROADCAST_FOR_UDF_COMPRESSION_THRESHOLD) === 1L * 1024) + } + val defaultIllegalValue = "SomeIllegalValue" val illegalValueTests : Map[String, (SparkConf, String) => Any] = Map( "getTimeAsSeconds" -> (_.getTimeAsSeconds(_)), @@ -371,6 +434,26 @@ class SparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSyst assert(thrown.getMessage.contains(key)) } } + + test("get task resource requirement from config") { + val conf = new SparkConf() + conf.set(TASK_GPU_ID.amountConf, "2") + conf.set(TASK_FPGA_ID.amountConf, "1") + var taskResourceRequirement = + parseTaskResourceRequirements(conf).map(req => (req.resourceName, req.amount)).toMap + + assert(taskResourceRequirement.size == 2) + assert(taskResourceRequirement(GPU) == 2) + assert(taskResourceRequirement(FPGA) == 1) + + conf.remove(TASK_FPGA_ID.amountConf) + // Ignore invalid prefix + conf.set(ResourceID("spark.invalid.prefix", FPGA).amountConf, "1") + taskResourceRequirement = + parseTaskResourceRequirements(conf).map(req => (req.resourceName, req.amount)).toMap + assert(taskResourceRequirement.size == 1) + assert(taskResourceRequirement.get(FPGA).isEmpty) + } } class Class1 {} diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 8feb3dee050d2..536b4aec75623 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -17,17 +17,20 @@ package org.apache.spark +import scala.concurrent.duration._ + import org.scalatest.Assertions +import org.scalatest.concurrent.Eventually._ import org.apache.spark.storage.StorageLevel class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { sc = new SparkContext("local", "test") - assert(sc.getPersistentRDDs.isEmpty === true) + assert(sc.getPersistentRDDs.isEmpty) val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) - assert(sc.getPersistentRDDs.isEmpty === true) + assert(sc.getPersistentRDDs.isEmpty) rdd.cache() assert(sc.getPersistentRDDs.size === 1) @@ -58,9 +61,12 @@ class SparkContextInfoSuite extends SparkFunSuite with LocalSparkContext { test("getRDDStorageInfo only reports on RDDs that actually persist data") { sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() - assert(sc.getRDDStorageInfo.size === 0) + assert(sc.getRDDStorageInfo.length === 0) rdd.collect() - assert(sc.getRDDStorageInfo.size === 1) + sc.listenerBus.waitUntilEmpty(10000) + eventually(timeout(10.seconds), interval(100.milliseconds)) { + assert(sc.getRDDStorageInfo.length === 1) + } assert(sc.getRDDStorageInfo.head.isCached) assert(sc.getRDDStorageInfo.head.memSize > 0) assert(sc.getRDDStorageInfo.head.storageLevel === StorageLevel.MEMORY_ONLY) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f8938dfedee5b..811b9757232e2 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -23,110 +23,129 @@ import org.apache.spark.internal.Logging import org.apache.spark.scheduler.{SchedulerBackend, TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.StandaloneSchedulerBackend import org.apache.spark.scheduler.local.LocalSchedulerBackend +import org.apache.spark.util.Utils class SparkContextSchedulerCreationSuite extends SparkFunSuite with LocalSparkContext with PrivateMethodTester with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = - createTaskScheduler(master, "client") + def noOp(taskSchedulerImpl: TaskSchedulerImpl): Unit = {} - def createTaskScheduler(master: String, deployMode: String): TaskSchedulerImpl = - createTaskScheduler(master, deployMode, new SparkConf()) + def createTaskScheduler(master: String)(body: TaskSchedulerImpl => Unit = noOp): Unit = + createTaskScheduler(master, "client")(body) + + def createTaskScheduler(master: String, deployMode: String)( + body: TaskSchedulerImpl => Unit): Unit = + createTaskScheduler(master, deployMode, new SparkConf())(body) def createTaskScheduler( master: String, deployMode: String, - conf: SparkConf): TaskSchedulerImpl = { + conf: SparkConf)(body: TaskSchedulerImpl => Unit): Unit = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[Tuple2[SchedulerBackend, TaskScheduler]]('createTaskScheduler) - val (_, sched) = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) - sched.asInstanceOf[TaskSchedulerImpl] + val (_, sched) = + SparkContext invokePrivate createTaskSchedulerMethod(sc, master, deployMode) + try { + body(sched.asInstanceOf[TaskSchedulerImpl]) + } finally { + Utils.tryLogNonFatalError { + sched.stop() + } + } } test("bad-master") { val e = intercept[SparkException] { - createTaskScheduler("localhost:1234") + createTaskScheduler("localhost:1234")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local") { - val sched = createTaskScheduler("local") - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 1) - case _ => fail() + val sched = createTaskScheduler("local") { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 1) + case _ => fail() + } } } test("local-*") { - val sched = createTaskScheduler("local[*]") - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[*]") { sched => + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n") { - val sched = createTaskScheduler("local[5]") - assert(sched.maxTaskFailures === 1) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 5) - case _ => fail() + val sched = createTaskScheduler("local[5]") { sched => + assert(sched.maxTaskFailures === 1) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 5) + case _ => fail() + } } } test("local-*-n-failures") { - val sched = createTaskScheduler("local[* ,2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => - assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() + val sched = createTaskScheduler("local[* ,2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => + assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } } } test("local-n-failures") { - val sched = createTaskScheduler("local[4, 2]") - assert(sched.maxTaskFailures === 2) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.totalCores === 4) - case _ => fail() + val sched = createTaskScheduler("local[4, 2]") { sched => + assert(sched.maxTaskFailures === 2) + sched.backend match { + case s: LocalSchedulerBackend => assert(s.totalCores === 4) + case _ => fail() + } } } test("bad-local-n") { val e = intercept[SparkException] { - createTaskScheduler("local[2*]") + createTaskScheduler("local[2*]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("bad-local-n-failures") { val e = intercept[SparkException] { - createTaskScheduler("local[2*,4]") + createTaskScheduler("local[2*,4]")() } assert(e.getMessage.contains("Could not parse Master URL")) } test("local-default-parallelism") { val conf = new SparkConf().set("spark.default.parallelism", "16") - val sched = createTaskScheduler("local", "client", conf) - sched.backend match { - case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) - case _ => fail() + val sched = createTaskScheduler("local", "client", conf) { sched => + sched.backend match { + case s: LocalSchedulerBackend => assert(s.defaultParallelism() === 16) + case _ => fail() + } } } test("local-cluster") { - createTaskScheduler("local-cluster[3, 14, 1024]").backend match { - case s: StandaloneSchedulerBackend => // OK - case _ => fail() + createTaskScheduler("local-cluster[3, 14, 1024]") { sched => + sched.backend match { + case s: StandaloneSchedulerBackend => // OK + case _ => fail() + } } } } diff --git a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala index e1666a35271d3..fed3ae35ee0e7 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSuite.scala @@ -26,14 +26,22 @@ import scala.concurrent.duration._ import com.google.common.io.Files import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{BytesWritable, LongWritable, Text} import org.apache.hadoop.mapred.TextInputFormat import org.apache.hadoop.mapreduce.lib.input.{TextInputFormat => NewTextInputFormat} +import org.json4s.{DefaultFormats, Extraction} import org.scalatest.Matchers._ import org.scalatest.concurrent.Eventually -import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.TestUtils._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.resource.ResourceAllocation +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.scheduler.{SparkListener, SparkListenerExecutorMetricsUpdate, SparkListenerJobStart, SparkListenerTaskEnd, SparkListenerTaskStart} +import org.apache.spark.shuffle.FetchFailedException import org.apache.spark.util.{ThreadUtils, Utils} @@ -42,7 +50,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Only one SparkContext may be active at a time") { // Regression test for SPARK-4180 val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "false") sc = new SparkContext(conf) val envBefore = SparkEnv.get // A SparkContext is already running, so we shouldn't be able to create a second one @@ -56,7 +63,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("Can still construct a new SparkContext after failing to construct a previous one") { - val conf = new SparkConf().set("spark.driver.allowMultipleContexts", "false") + val conf = new SparkConf() // This is an invalid configuration (no app name or master URL) intercept[SparkException] { new SparkContext(conf) @@ -65,18 +72,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc = new SparkContext(conf.setMaster("local").setAppName("test")) } - test("Check for multiple SparkContexts can be disabled via undocumented debug option") { - var secondSparkContext: SparkContext = null - try { - val conf = new SparkConf().setAppName("test").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true") - sc = new SparkContext(conf) - secondSparkContext = new SparkContext(conf) - } finally { - Option(secondSparkContext).foreach(_.stop()) - } - } - test("Test getOrCreate") { var sc2: SparkContext = null SparkContext.clearActiveContext() @@ -90,10 +85,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc === sc2) assert(sc eq sc2) - // Try creating second context to confirm that it's still possible, if desired - sc2 = new SparkContext(new SparkConf().setAppName("test3").setMaster("local") - .set("spark.driver.allowMultipleContexts", "true")) - sc2.stop() } @@ -114,65 +105,77 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("basic case for addFile and listFiles") { - val dir = Utils.createTempDir() + withTempDir { dir => + val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) + val absolutePath1 = file1.getAbsolutePath + + val file2 = File.createTempFile("someprefix2", "somesuffix2", dir) + val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + + "/" + file2.getName + val absolutePath2 = file2.getAbsolutePath + + try { + Files.write("somewords1", file1, StandardCharsets.UTF_8) + Files.write("somewords2", file2, StandardCharsets.UTF_8) + val length1 = file1.length() + val length2 = file2.length() + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(file1.getAbsolutePath) + sc.addFile(relativePath) + sc.parallelize(Array(1), 1).map(x => { + val gotten1 = new File(SparkFiles.get(file1.getName)) + val gotten2 = new File(SparkFiles.get(file2.getName)) + if (!gotten1.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath1) + } + if (!gotten2.exists()) { + throw new SparkException("file doesn't exist : " + absolutePath2) + } - val file1 = File.createTempFile("someprefix1", "somesuffix1", dir) - val absolutePath1 = file1.getAbsolutePath + if (length1 != gotten1.length()) { + throw new SparkException( + s"file has different length $length1 than added file ${gotten1.length()} : " + + absolutePath1) + } + if (length2 != gotten2.length()) { + throw new SparkException( + s"file has different length $length2 than added file ${gotten2.length()} : " + + absolutePath2) + } - val file2 = File.createTempFile("someprefix2", "somesuffix2", dir) - val relativePath = file2.getParent + "/../" + file2.getParentFile.getName + "/" + file2.getName - val absolutePath2 = file2.getAbsolutePath + if (absolutePath1 == gotten1.getAbsolutePath) { + throw new SparkException("file should have been copied :" + absolutePath1) + } + if (absolutePath2 == gotten2.getAbsolutePath) { + throw new SparkException("file should have been copied : " + absolutePath2) + } + x + }).count() + assert(sc.listFiles().filter(_.contains("somesuffix1")).size == 1) + } finally { + sc.stop() + } + } + } + test("add and list jar files") { + val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") try { - Files.write("somewords1", file1, StandardCharsets.UTF_8) - Files.write("somewords2", file2, StandardCharsets.UTF_8) - val length1 = file1.length() - val length2 = file2.length() - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(file1.getAbsolutePath) - sc.addFile(relativePath) - sc.parallelize(Array(1), 1).map(x => { - val gotten1 = new File(SparkFiles.get(file1.getName)) - val gotten2 = new File(SparkFiles.get(file2.getName)) - if (!gotten1.exists()) { - throw new SparkException("file doesn't exist : " + absolutePath1) - } - if (!gotten2.exists()) { - throw new SparkException("file doesn't exist : " + absolutePath2) - } - - if (length1 != gotten1.length()) { - throw new SparkException( - s"file has different length $length1 than added file ${gotten1.length()} : " + - absolutePath1) - } - if (length2 != gotten2.length()) { - throw new SparkException( - s"file has different length $length2 than added file ${gotten2.length()} : " + - absolutePath2) - } - - if (absolutePath1 == gotten1.getAbsolutePath) { - throw new SparkException("file should have been copied :" + absolutePath1) - } - if (absolutePath2 == gotten2.getAbsolutePath) { - throw new SparkException("file should have been copied : " + absolutePath2) - } - x - }).count() - assert(sc.listFiles().filter(_.contains("somesuffix1")).size == 1) + sc.addJar(jarPath.toString) + assert(sc.listJars().filter(_.contains("TestUDTF.jar")).size == 1) } finally { sc.stop() } } - test("add and list jar files") { - val jarPath = Thread.currentThread().getContextClassLoader.getResource("TestUDTF.jar") + test("add FS jar files not exists") { try { + val jarPath = "hdfs:///no/path/to/TestUDTF.jar" sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addJar(jarPath.toString) - assert(sc.listJars().filter(_.contains("TestUDTF.jar")).size == 1) + sc.addJar(jarPath) + assert(sc.listJars().forall(!_.contains("TestUDTF.jar"))) } finally { sc.stop() } @@ -200,51 +203,51 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("addFile recursive works") { - val pluto = Utils.createTempDir() - val neptune = Utils.createTempDir(pluto.getAbsolutePath) - val saturn = Utils.createTempDir(neptune.getAbsolutePath) - val alien1 = File.createTempFile("alien", "1", neptune) - val alien2 = File.createTempFile("alien", "2", saturn) - - try { - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addFile(neptune.getAbsolutePath, true) - sc.parallelize(Array(1), 1).map(x => { - val sep = File.separator - if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { - throw new SparkException("can't access file under root added directory") - } - if (!new File(SparkFiles.get(neptune.getName + sep + saturn.getName + sep + alien2.getName)) - .exists()) { - throw new SparkException("can't access file in nested directory") - } - if (new File(SparkFiles.get(pluto.getName + sep + neptune.getName + sep + alien1.getName)) - .exists()) { - throw new SparkException("file exists that shouldn't") - } - x - }).count() - } finally { - sc.stop() + withTempDir { pluto => + val neptune = Utils.createTempDir(pluto.getAbsolutePath) + val saturn = Utils.createTempDir(neptune.getAbsolutePath) + val alien1 = File.createTempFile("alien", "1", neptune) + val alien2 = File.createTempFile("alien", "2", saturn) + + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addFile(neptune.getAbsolutePath, true) + sc.parallelize(Array(1), 1).map(x => { + val sep = File.separator + if (!new File(SparkFiles.get(neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("can't access file under root added directory") + } + if (!new File(SparkFiles.get( + neptune.getName + sep + saturn.getName + sep + alien2.getName)).exists()) { + throw new SparkException("can't access file in nested directory") + } + if (new File(SparkFiles.get( + pluto.getName + sep + neptune.getName + sep + alien1.getName)).exists()) { + throw new SparkException("file exists that shouldn't") + } + x + }).count() + } finally { + sc.stop() + } } } test("addFile recursive can't add directories by default") { - val dir = Utils.createTempDir() - - try { - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - intercept[SparkException] { - sc.addFile(dir.getAbsolutePath) + withTempDir { dir => + try { + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + intercept[SparkException] { + sc.addFile(dir.getAbsolutePath) + } + } finally { + sc.stop() } - } finally { - sc.stop() } } test("cannot call addFile with different paths that have the same filename") { - val dir = Utils.createTempDir() - try { + withTempDir { dir => val subdir1 = new File(dir, "subdir1") val subdir2 = new File(dir, "subdir2") assert(subdir1.mkdir()) @@ -265,8 +268,6 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu sc.addFile(file2.getAbsolutePath) } assert(getAddedFileContents() === "old") - } finally { - Utils.deleteRecursively(dir) } } @@ -294,30 +295,33 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu } test("add jar with invalid path") { - val tmpDir = Utils.createTempDir() - val tmpJar = File.createTempFile("test", ".jar", tmpDir) + withTempDir { tmpDir => + val tmpJar = File.createTempFile("test", ".jar", tmpDir) - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - sc.addJar(tmpJar.getAbsolutePath) + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + sc.addJar(tmpJar.getAbsolutePath) - // Invalid jar path will only print the error log, will not add to file server. - sc.addJar("dummy.jar") - sc.addJar("") - sc.addJar(tmpDir.getAbsolutePath) + // Invalid jar path will only print the error log, will not add to file server. + sc.addJar("dummy.jar") + sc.addJar("") + sc.addJar(tmpDir.getAbsolutePath) - assert(sc.listJars().size == 1) - assert(sc.listJars().head.contains(tmpJar.getName)) + assert(sc.listJars().size == 1) + assert(sc.listJars().head.contains(tmpJar.getName)) + } } test("SPARK-22585 addJar argument without scheme is interpreted literally without url decoding") { - val tmpDir = new File(Utils.createTempDir(), "host%3A443") - tmpDir.mkdirs() - val tmpJar = File.createTempFile("t%2F", ".jar", tmpDir) + withTempDir { dir => + val tmpDir = new File(dir, "host%3A443") + tmpDir.mkdirs() + val tmpJar = File.createTempFile("t%2F", ".jar", tmpDir) - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test") - sc.addJar(tmpJar.getAbsolutePath) - assert(sc.listJars().size === 1) + sc.addJar(tmpJar.getAbsolutePath) + assert(sc.listJars().size === 1) + } } test("Cancelling job group should not cause SparkContext to shutdown (SPARK-6414)") { @@ -338,60 +342,61 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("Comma separated paths for newAPIHadoopFile/wholeTextFiles/binaryFiles (SPARK-7155)") { // Regression test for SPARK-7155 // dir1 and dir2 are used for wholeTextFiles and binaryFiles - val dir1 = Utils.createTempDir() - val dir2 = Utils.createTempDir() - - val dirpath1 = dir1.getAbsolutePath - val dirpath2 = dir2.getAbsolutePath - - // file1 and file2 are placed inside dir1, they are also used for - // textFile, hadoopFile, and newAPIHadoopFile - // file3, file4 and file5 are placed inside dir2, they are used for - // textFile, hadoopFile, and newAPIHadoopFile as well - val file1 = new File(dir1, "part-00000") - val file2 = new File(dir1, "part-00001") - val file3 = new File(dir2, "part-00000") - val file4 = new File(dir2, "part-00001") - val file5 = new File(dir2, "part-00002") - - val filepath1 = file1.getAbsolutePath - val filepath2 = file2.getAbsolutePath - val filepath3 = file3.getAbsolutePath - val filepath4 = file4.getAbsolutePath - val filepath5 = file5.getAbsolutePath - - - try { - // Create 5 text files. - Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, - StandardCharsets.UTF_8) - Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) - Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) - Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) - Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) - - sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) - - // Test textFile, hadoopFile, and newAPIHadoopFile for file1 and file2 - assert(sc.textFile(filepath1 + "," + filepath2).count() == 5L) - assert(sc.hadoopFile(filepath1 + "," + filepath2, - classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - assert(sc.newAPIHadoopFile(filepath1 + "," + filepath2, - classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - - // Test textFile, hadoopFile, and newAPIHadoopFile for file3, file4, and file5 - assert(sc.textFile(filepath3 + "," + filepath4 + "," + filepath5).count() == 5L) - assert(sc.hadoopFile(filepath3 + "," + filepath4 + "," + filepath5, - classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - assert(sc.newAPIHadoopFile(filepath3 + "," + filepath4 + "," + filepath5, - classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) - - // Test wholeTextFiles, and binaryFiles for dir1 and dir2 - assert(sc.wholeTextFiles(dirpath1 + "," + dirpath2).count() == 5L) - assert(sc.binaryFiles(dirpath1 + "," + dirpath2).count() == 5L) - - } finally { - sc.stop() + withTempDir { dir1 => + withTempDir { dir2 => + val dirpath1 = dir1.getAbsolutePath + val dirpath2 = dir2.getAbsolutePath + + // file1 and file2 are placed inside dir1, they are also used for + // textFile, hadoopFile, and newAPIHadoopFile + // file3, file4 and file5 are placed inside dir2, they are used for + // textFile, hadoopFile, and newAPIHadoopFile as well + val file1 = new File(dir1, "part-00000") + val file2 = new File(dir1, "part-00001") + val file3 = new File(dir2, "part-00000") + val file4 = new File(dir2, "part-00001") + val file5 = new File(dir2, "part-00002") + + val filepath1 = file1.getAbsolutePath + val filepath2 = file2.getAbsolutePath + val filepath3 = file3.getAbsolutePath + val filepath4 = file4.getAbsolutePath + val filepath5 = file5.getAbsolutePath + + + try { + // Create 5 text files. + Files.write("someline1 in file1\nsomeline2 in file1\nsomeline3 in file1", file1, + StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file2", file2, StandardCharsets.UTF_8) + Files.write("someline1 in file3", file3, StandardCharsets.UTF_8) + Files.write("someline1 in file4\nsomeline2 in file4", file4, StandardCharsets.UTF_8) + Files.write("someline1 in file2\nsomeline2 in file5", file5, StandardCharsets.UTF_8) + + sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local")) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file1 and file2 + assert(sc.textFile(filepath1 + "," + filepath2).count() == 5L) + assert(sc.hadoopFile(filepath1 + "," + filepath2, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath1 + "," + filepath2, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test textFile, hadoopFile, and newAPIHadoopFile for file3, file4, and file5 + assert(sc.textFile(filepath3 + "," + filepath4 + "," + filepath5).count() == 5L) + assert(sc.hadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[TextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + assert(sc.newAPIHadoopFile(filepath3 + "," + filepath4 + "," + filepath5, + classOf[NewTextInputFormat], classOf[LongWritable], classOf[Text]).count() == 5L) + + // Test wholeTextFiles, and binaryFiles for dir1 and dir2 + assert(sc.wholeTextFiles(dirpath1 + "," + dirpath2).count() == 5L) + assert(sc.binaryFiles(dirpath1 + "," + dirpath2).count() == 5L) + + } finally { + sc.stop() + } + } } } @@ -434,7 +439,7 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu test("No exception when both num-executors and dynamic allocation set.") { noException should be thrownBy { sc = new SparkContext(new SparkConf().setAppName("test").setMaster("local") - .set("spark.dynamicAllocation.enabled", "true").set("spark.executor.instances", "6")) + .set(DYN_ALLOCATION_ENABLED, true).set("spark.executor.instances", "6")) assert(sc.executorAllocationManager.isEmpty) assert(sc.getConf.getInt("spark.executor.instances", 0) === 6) } @@ -672,6 +677,205 @@ class SparkContextSuite extends SparkFunSuite with LocalSparkContext with Eventu assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) } } + + test("cancel zombie tasks in a result stage when the job finishes") { + val conf = new SparkConf() + .setMaster("local-cluster[1,2,1024]") + .setAppName("test-cluster") + .set(UI_ENABLED.key, "false") + // Disable this so that if a task is running, we can make sure the executor will always send + // task metrics via heartbeat to driver. + .set(EXECUTOR_HEARTBEAT_DROP_ZERO_ACCUMULATOR_UPDATES.key, "false") + // Set a short heartbeat interval to send SparkListenerExecutorMetricsUpdate fast + .set(EXECUTOR_HEARTBEAT_INTERVAL.key, "1s") + sc = new SparkContext(conf) + sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "true") + @volatile var runningTaskIds: Seq[Long] = null + val listener = new SparkListener { + override def onExecutorMetricsUpdate( + executorMetricsUpdate: SparkListenerExecutorMetricsUpdate): Unit = { + if (executorMetricsUpdate.execId != SparkContext.DRIVER_IDENTIFIER) { + runningTaskIds = executorMetricsUpdate.accumUpdates.map(_._1) + } + } + } + sc.addSparkListener(listener) + sc.range(0, 2).groupBy((x: Long) => x % 2, 2).map { case (x, _) => + val context = org.apache.spark.TaskContext.get() + if (context.stageAttemptNumber == 0) { + if (context.partitionId == 0) { + // Make the first task in the first stage attempt fail. + throw new FetchFailedException(SparkEnv.get.blockManager.blockManagerId, 0, 0, 0, + new java.io.IOException("fake")) + } else { + // Make the second task in the first stage attempt sleep to generate a zombie task + Thread.sleep(60000) + } + } else { + // Make the second stage attempt successful. + } + x + }.collect() + sc.listenerBus.waitUntilEmpty(10000) + // As executors will send the metrics of running tasks via heartbeat, we can use this to check + // whether there is any running task. + eventually(timeout(10.seconds)) { + // Make sure runningTaskIds has been set + assert(runningTaskIds != null) + // Verify there is no running task. + assert(runningTaskIds.isEmpty) + } + } + + test(s"Avoid setting ${CPUS_PER_TASK.key} unreasonably (SPARK-27192)") { + val FAIL_REASON = s"has to be >= the task config: ${CPUS_PER_TASK.key}" + Seq( + ("local", 2, None), + ("local[2]", 3, None), + ("local[2, 1]", 3, None), + ("spark://test-spark-cluster", 2, Option(1)), + ("local-cluster[1, 1, 1000]", 2, Option(1)), + ("yarn", 2, Option(1)) + ).foreach { case (master, cpusPerTask, executorCores) => + val conf = new SparkConf() + conf.set(CPUS_PER_TASK, cpusPerTask) + executorCores.map(executorCores => conf.set(EXECUTOR_CORES, executorCores)) + val ex = intercept[SparkException] { + sc = new SparkContext(master, "test", conf) + } + assert(ex.getMessage.contains(FAIL_REASON)) + resetSparkContext() + } + } + + test("test driver discovery under local-cluster mode") { + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "gpu","addresses":["5", "6"]}""") + + val conf = new SparkConf() + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + conf.set(DRIVER_GPU_ID.amountConf, "1") + conf.set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + sc = new SparkContext(conf) + + // Ensure all executors has started + TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + assert(sc.resources.size === 1) + assert(sc.resources.get(GPU).get.addresses === Array("5", "6")) + assert(sc.resources.get(GPU).get.name === "gpu") + } + } + + test("test gpu driver resource files and discovery under local-cluster mode") { + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "gpu","addresses":["5", "6"]}""") + + implicit val formats = DefaultFormats + val gpusAllocated = + ResourceAllocation(DRIVER_GPU_ID, Seq("0", "1", "8")) + val ja = Extraction.decompose(Seq(gpusAllocated)) + val resourcesFile = createTempJsonFile(dir, "resources", ja) + + val conf = new SparkConf() + .set(DRIVER_RESOURCES_FILE, resourcesFile) + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + conf.set(DRIVER_GPU_ID.amountConf, "1") + conf.set(DRIVER_GPU_ID.discoveryScriptConf, scriptPath) + + sc = new SparkContext(conf) + + // Ensure all executors has started + TestUtils.waitUntilExecutorsUp(sc, 1, 10000) + // driver gpu resources file should take precedence over the script + assert(sc.resources.size === 1) + assert(sc.resources.get(GPU).get.addresses === Array("0", "1", "8")) + assert(sc.resources.get(GPU).get.name === "gpu") + } + } + + test("Test parsing resources task configs with missing executor config") { + val conf = new SparkConf() + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + conf.set(TASK_GPU_ID.amountConf, "1") + + var error = intercept[SparkException] { + sc = new SparkContext(conf) + }.getMessage() + + assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount " + + "needs to be specified since a task requirement config: spark.task.resource.gpu.amount " + + "was specified")) + } + + test("Test parsing resources executor config < task requirements") { + val conf = new SparkConf() + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + conf.set(TASK_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + + var error = intercept[SparkException] { + sc = new SparkContext(conf) + }.getMessage() + + assert(error.contains("The executor resource config: spark.executor.resource.gpu.amount = 1 " + + "has to be >= the requested amount in task resource config: " + + "spark.task.resource.gpu.amount = 2")) + } + + test("Parse resources executor config not the same multiple numbers of the task requirements") { + val conf = new SparkConf() + .setMaster("local-cluster[1, 1, 1024]") + .setAppName("test-cluster") + conf.set(TASK_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_GPU_ID.amountConf, "4") + + var error = intercept[SparkException] { + sc = new SparkContext(conf) + }.getMessage() + + assert(error.contains("The configuration of resource: gpu (exec = 4, task = 2) will result " + + "in wasted resources due to resource CPU limiting the number of runnable tasks per " + + "executor to: 1. Please adjust your configuration.")) + } + + test("test resource scheduling under local-cluster mode") { + import org.apache.spark.TestUtils._ + + assume(!(Utils.isWindows)) + withTempDir { dir => + val discoveryScript = createTempScriptWithExpectedOutput(dir, "resourceDiscoveryScript", + """{"name": "gpu","addresses":["0", "1", "2"]}""") + + val conf = new SparkConf() + .setMaster("local-cluster[3, 3, 1024]") + .setAppName("test-cluster") + conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.amountConf, "3") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, discoveryScript) + + sc = new SparkContext(conf) + + // Ensure all executors has started + TestUtils.waitUntilExecutorsUp(sc, 3, 60000) + + val rdd = sc.makeRDD(1 to 10, 9).mapPartitions { it => + val context = TaskContext.get() + context.resources().get(GPU).get.addresses.iterator + } + val gpus = rdd.collect() + assert(gpus.sorted === Seq("0", "0", "0", "1", "1", "1", "2", "2", "2")) + + eventually(timeout(10.seconds)) { + assert(sc.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + } + } } object SparkContextSuite { diff --git a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala index 31289026b0027..9dd113262653b 100644 --- a/core/src/test/scala/org/apache/spark/SparkFunSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkFunSuite.scala @@ -20,10 +20,14 @@ package org.apache.spark // scalastyle:off import java.io.File -import org.scalatest.{BeforeAndAfterAll, FunSuite, Outcome} +import scala.annotation.tailrec + +import org.apache.log4j.{Appender, Level, Logger} +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, Outcome} import org.apache.spark.internal.Logging -import org.apache.spark.util.AccumulatorContext +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.util.{AccumulatorContext, Utils} /** * Base abstract class for all unit tests in Spark for handling common functionality. @@ -52,6 +56,7 @@ import org.apache.spark.util.AccumulatorContext abstract class SparkFunSuite extends FunSuite with BeforeAndAfterAll + with BeforeAndAfterEach with ThreadAudit with Logging { // scalastyle:on @@ -59,7 +64,7 @@ abstract class SparkFunSuite protected val enableAutoThreadAudit = true protected override def beforeAll(): Unit = { - System.setProperty("spark.testing", "true") + System.setProperty(IS_TESTING.key, "true") if (enableAutoThreadAudit) { doThreadPreAudit() } @@ -87,6 +92,47 @@ abstract class SparkFunSuite getTestResourceFile(file).getCanonicalPath } + /** + * Note: this method doesn't support `BeforeAndAfter`. You must use `BeforeAndAfterEach` to + * set up and tear down resources. + */ + def testRetry(s: String, n: Int = 2)(body: => Unit): Unit = { + test(s) { + retry(n) { + body + } + } + } + + /** + * Note: this method doesn't support `BeforeAndAfter`. You must use `BeforeAndAfterEach` to + * set up and tear down resources. + */ + def retry[T](n: Int)(body: => T): T = { + if (this.isInstanceOf[BeforeAndAfter]) { + throw new UnsupportedOperationException( + s"testRetry/retry cannot be used with ${classOf[BeforeAndAfter]}. " + + s"Please use ${classOf[BeforeAndAfterEach]} instead.") + } + retry0(n, n)(body) + } + + @tailrec private final def retry0[T](n: Int, n0: Int)(body: => T): T = { + try body + catch { case e: Throwable => + if (n > 0) { + logWarning(e.getMessage, e) + logInfo(s"\n\n===== RETRY #${n0 - n + 1} =====\n") + // Reset state before re-attempting in order so that tests which use patterns like + // LocalSparkContext to clean up state can work correctly when retried. + afterEach() + beforeEach() + retry0(n-1, n0)(body) + } + else throw e + } + } + /** * Log the suite name and the test name before and after each test. * @@ -106,4 +152,38 @@ abstract class SparkFunSuite } } + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withTempDir(f: File => Unit): Unit = { + val dir = Utils.createTempDir() + try f(dir) finally { + Utils.deleteRecursively(dir) + } + } + + /** + * Adds a log appender and optionally sets a log level to the root logger or the logger with + * the specified name, then executes the specified function, and in the end removes the log + * appender and restores the log level if necessary. + */ + protected def withLogAppender( + appender: Appender, + loggerName: Option[String] = None, + level: Option[Level] = None)( + f: => Unit): Unit = { + val logger = loggerName.map(Logger.getLogger).getOrElse(Logger.getRootLogger) + val restoreLevel = logger.getLevel + logger.addAppender(appender) + if (level.isDefined) { + logger.setLevel(level.get) + } + try f finally { + logger.removeAppender(appender) + if (level.isDefined) { + logger.setLevel(restoreLevel) + } + } + } } diff --git a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala index a15ae040d43a9..f527bbe718524 100644 --- a/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/StatusTrackerSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark import scala.concurrent.duration._ import scala.language.implicitConversions -import scala.language.postfixOps import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ @@ -28,28 +27,28 @@ import org.apache.spark.JobExecutionStatus._ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkContext { - test("basic status API usage") { + testRetry("basic status API usage") { sc = new SparkContext("local", "test", new SparkConf(false)) val jobFuture = sc.parallelize(1 to 10000, 2).map(identity).groupBy(identity).collectAsync() - val jobId: Int = eventually(timeout(10 seconds)) { + val jobId: Int = eventually(timeout(10.seconds)) { val jobIds = jobFuture.jobIds jobIds.size should be(1) jobIds.head } - val jobInfo = eventually(timeout(10 seconds)) { + val jobInfo = eventually(timeout(10.seconds)) { sc.statusTracker.getJobInfo(jobId).get } jobInfo.status() should not be FAILED val stageIds = jobInfo.stageIds() stageIds.size should be(2) - val firstStageInfo = eventually(timeout(10 seconds)) { + val firstStageInfo = eventually(timeout(10.seconds)) { sc.statusTracker.getStageInfo(stageIds.min).get } firstStageInfo.stageId() should be(stageIds.min) firstStageInfo.currentAttemptId() should be(0) firstStageInfo.numTasks() should be(2) - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { val updatedFirstStageInfo = sc.statusTracker.getStageInfo(stageIds.min).get updatedFirstStageInfo.numCompletedTasks() should be(2) updatedFirstStageInfo.numActiveTasks() should be(0) @@ -61,27 +60,27 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont sc = new SparkContext("local", "test", new SparkConf(false)) // Passing `null` should return jobs that were not run in a job group: val defaultJobGroupFuture = sc.parallelize(1 to 1000).countAsync() - val defaultJobGroupJobId = eventually(timeout(10 seconds)) { + val defaultJobGroupJobId = eventually(timeout(10.seconds)) { defaultJobGroupFuture.jobIds.head } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForGroup(null).toSet should be (Set(defaultJobGroupJobId)) } // Test jobs submitted in job groups: sc.setJobGroup("my-job-group", "description") sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq.empty) val firstJobFuture = sc.parallelize(1 to 1000).countAsync() - val firstJobId = eventually(timeout(10 seconds)) { + val firstJobId = eventually(timeout(10.seconds)) { firstJobFuture.jobIds.head } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForGroup("my-job-group") should be (Seq(firstJobId)) } val secondJobFuture = sc.parallelize(1 to 1000).countAsync() - val secondJobId = eventually(timeout(10 seconds)) { + val secondJobId = eventually(timeout(10.seconds)) { secondJobFuture.jobIds.head } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForGroup("my-job-group").toSet should be ( Set(firstJobId, secondJobId)) } @@ -92,10 +91,10 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont sc.setJobGroup("my-job-group2", "description") sc.statusTracker.getJobIdsForGroup("my-job-group2") shouldBe empty val firstJobFuture = sc.parallelize(1 to 1000, 1).takeAsync(1) - val firstJobId = eventually(timeout(10 seconds)) { + val firstJobId = eventually(timeout(10.seconds)) { firstJobFuture.jobIds.head } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForGroup("my-job-group2") should be (Seq(firstJobId)) } } @@ -105,10 +104,10 @@ class StatusTrackerSuite extends SparkFunSuite with Matchers with LocalSparkCont sc.setJobGroup("my-job-group2", "description") sc.statusTracker.getJobIdsForGroup("my-job-group2") shouldBe empty val firstJobFuture = sc.parallelize(1 to 1000, 2).takeAsync(999) - val firstJobId = eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { firstJobFuture.jobIds.head } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { sc.statusTracker.getJobIdsForGroup("my-job-group2") should have size 2 } } diff --git a/core/src/test/scala/org/apache/spark/ThreadAudit.scala b/core/src/test/scala/org/apache/spark/ThreadAudit.scala index b3cea9de8f304..44d1f220bf6b1 100644 --- a/core/src/test/scala/org/apache/spark/ThreadAudit.scala +++ b/core/src/test/scala/org/apache/spark/ThreadAudit.scala @@ -55,21 +55,47 @@ trait ThreadAudit extends Logging { * creates event loops. One is wrapped inside * [[org.apache.spark.network.server.TransportServer]] * the other one is inside [[org.apache.spark.network.client.TransportClient]]. - * The thread pools behind shut down asynchronously triggered by [[SparkContext#stop]]. - * Manually checked and all of them stopped properly. + * Calling [[SparkContext#stop]] will shut down the thread pool of this event group + * asynchronously. In each case proper stopping is checked manually. */ "rpc-client.*", "rpc-server.*", + /** + * During [[org.apache.spark.network.TransportContext]] construction a separate event loop could + * be created for handling ChunkFetchRequest. + * Calling [[org.apache.spark.network.TransportContext#close]] will shut down the thread pool + * of this event group asynchronously. In each case proper stopping is checked manually. + */ + "shuffle-chunk-fetch-handler.*", + /** * During [[SparkContext]] creation BlockManager creates event loops. One is wrapped inside * [[org.apache.spark.network.server.TransportServer]] * the other one is inside [[org.apache.spark.network.client.TransportClient]]. - * The thread pools behind shut down asynchronously triggered by [[SparkContext#stop]]. - * Manually checked and all of them stopped properly. + * Calling [[SparkContext#stop]] will shut down the thread pool of this event group + * asynchronously. In each case proper stopping is checked manually. */ "shuffle-client.*", - "shuffle-server.*" + "shuffle-server.*", + + /** + * Global cleaner thread that manage statistics data references of Hadoop filesystems. + * This is excluded because their lifecycle is handled by Hadoop and spark has no explicit + * effect on it. + */ + "org.apache.hadoop.fs.FileSystem\\$Statistics\\$StatisticsDataReferenceCleaner", + + /** + * A global thread pool for broadcast exchange executions. + */ + "broadcast-exchange.*", + + /** + * A thread started by JRE to support safe parallel execution of waitFor() and exitStatus() + * methods to forked subprocesses. + */ + "process reaper" ) private var threadNamesSnapshot: Set[String] = Set.empty diff --git a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala index 36273d722f50a..5cf9c087e1dcb 100644 --- a/core/src/test/scala/org/apache/spark/ThreadingSuite.scala +++ b/core/src/test/scala/org/apache/spark/ThreadingSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import java.util.concurrent.Semaphore +import java.util.concurrent.{Semaphore, TimeUnit} import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} import org.apache.spark.internal.Logging @@ -126,8 +126,9 @@ class ThreadingSuite extends SparkFunSuite with LocalSparkContext with Logging { val ans = nums.map(number => { val running = ThreadingSuiteState.runningThreads running.getAndIncrement() - val time = System.currentTimeMillis() - while (running.get() != 4 && System.currentTimeMillis() < time + 1000) { + val timeNs = System.nanoTime() + while (running.get() != 4 && + (System.nanoTime() - timeNs < TimeUnit.SECONDS.toNanos(1))) { Thread.sleep(100) } if (running.get() != 4) { diff --git a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala index b58a3ebe6e4c9..ff6ee7977066a 100644 --- a/core/src/test/scala/org/apache/spark/UnpersistSuite.scala +++ b/core/src/test/scala/org/apache/spark/UnpersistSuite.scala @@ -29,9 +29,9 @@ class UnpersistSuite extends SparkFunSuite with LocalSparkContext with TimeLimit sc = new SparkContext("local", "test") val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2).cache() rdd.count - assert(sc.persistentRdds.isEmpty === false) - rdd.unpersist() - assert(sc.persistentRdds.isEmpty === true) + assert(sc.persistentRdds.nonEmpty) + rdd.unpersist(blocking = true) + assert(sc.persistentRdds.isEmpty) failAfter(Span(3000, Millis)) { try { @@ -44,6 +44,6 @@ class UnpersistSuite extends SparkFunSuite with LocalSparkContext with TimeLimit // is racing this thread to remove entries from the driver. } } - assert(sc.getRDDStorageInfo.isEmpty === true) + assert(sc.getRDDStorageInfo.isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala index b38a3667abee1..dffdd96cd2dcc 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonBroadcastSuite.scala @@ -24,6 +24,7 @@ import scala.io.Source import org.scalatest.Matchers import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils @@ -31,7 +32,6 @@ import org.apache.spark.util.Utils // a PythonBroadcast: class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkContext { test("PythonBroadcast can be serialized with Kryo (SPARK-4882)") { - val tempDir = Utils.createTempDir() val broadcastedString = "Hello, world!" def assertBroadcastIsValid(broadcast: PythonBroadcast): Unit = { val source = Source.fromFile(broadcast.path) @@ -39,22 +39,20 @@ class PythonBroadcastSuite extends SparkFunSuite with Matchers with SharedSparkC source.close() contents should be (broadcastedString) } - try { + withTempDir { tempDir => val broadcastDataFile: File = { val file = new File(tempDir, "broadcastData") - val printWriter = new PrintWriter(file) - printWriter.write(broadcastedString) - printWriter.close() + Utils.tryWithResource(new PrintWriter(file)) { printWriter => + printWriter.write(broadcastedString) + } file } val broadcast = new PythonBroadcast(broadcastDataFile.getAbsolutePath) assertBroadcastIsValid(broadcast) - val conf = new SparkConf().set("spark.kryo.registrationRequired", "true") + val conf = new SparkConf().set(KRYO_REGISTRATION_REQUIRED, true) val deserializedBroadcast = Utils.clone[PythonBroadcast](broadcast, new KryoSerializer(conf).newInstance()) assertBroadcastIsValid(deserializedBroadcast) - } finally { - Utils.deleteRecursively(tempDir) } } } diff --git a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala index 6f9b583898c38..e2ec50fb1f172 100644 --- a/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/api/python/PythonRDDSuite.scala @@ -24,7 +24,7 @@ import java.nio.charset.StandardCharsets import scala.concurrent.duration.Duration import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.security.SocketAuthHelper +import org.apache.spark.security.{SocketAuthHelper, SocketAuthServer} class PythonRDDSuite extends SparkFunSuite { @@ -59,7 +59,7 @@ class PythonRDDSuite extends SparkFunSuite { } class ExceptionPythonServer(authHelper: SocketAuthHelper) - extends PythonServer[Unit](authHelper, "error-server") { + extends SocketAuthServer[Unit](authHelper, "error-server") { override def handleConnection(sock: Socket): Unit = { throw new Exception("exception within handleConnection") diff --git a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala index 7a36b5f02dc4c..73f9d0e2bc0e1 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/Benchmark.scala @@ -111,13 +111,15 @@ private[spark] class Benchmark( // The results are going to be processor specific so it is useful to include that. out.println(Benchmark.getJVMOSInfo()) out.println(Benchmark.getProcessorName()) - out.printf("%-40s %16s %12s %13s %10s\n", name + ":", "Best/Avg Time(ms)", "Rate(M/s)", + out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", name + ":", "Best Time(ms)", "Avg Time(ms)", "Stdev(ms)", "Rate(M/s)", "Per Row(ns)", "Relative") - out.println("-" * 96) + out.println("-" * 120) results.zip(benchmarks).foreach { case (result, benchmark) => - out.printf("%-40s %16s %12s %13s %10s\n", + out.printf("%-40s %14s %14s %11s %12s %13s %10s\n", benchmark.name, - "%5.0f / %4.0f" format (result.bestMs, result.avgMs), + "%5.0f" format result.bestMs, + "%4.0f" format result.avgMs, + "%5.0f" format result.stdevMs, "%10.1f" format result.bestRate, "%6.1f" format (1000 / result.bestRate), "%3.1fX" format (firstBest / result.bestMs)) @@ -148,17 +150,21 @@ private[spark] class Benchmark( if (outputPerIteration) { // scalastyle:off - println(s"Iteration $i took ${runTime / 1000} microseconds") + println(s"Iteration $i took ${NANOSECONDS.toMicros(runTime)} microseconds") // scalastyle:on } i += 1 } // scalastyle:off - println(s" Stopped after $i iterations, ${runTimes.sum / 1000000} ms") + println(s" Stopped after $i iterations, ${NANOSECONDS.toMillis(runTimes.sum)} ms") // scalastyle:on + assert(runTimes.nonEmpty) val best = runTimes.min val avg = runTimes.sum / runTimes.size - Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0) + val stdev = if (runTimes.size > 1) { + math.sqrt(runTimes.map(time => (time - avg) * (time - avg)).sum / (runTimes.size - 1)) + } else 0 + Result(avg / 1000000.0, num / (best / 1000.0), best / 1000000.0, stdev / 1000000.0) } } @@ -191,7 +197,7 @@ private[spark] object Benchmark { } case class Case(name: String, fn: Timer => Unit, numIters: Int) - case class Result(avgMs: Double, bestRate: Double, bestMs: Double) + case class Result(avgMs: Double, bestRate: Double, bestMs: Double, stdevMs: Double) /** * This should return a user helpful processor information. Getting at this depends on the OS. @@ -200,11 +206,12 @@ private[spark] object Benchmark { def getProcessorName(): String = { val cpu = if (SystemUtils.IS_OS_MAC_OSX) { Utils.executeAndGetOutput(Seq("/usr/sbin/sysctl", "-n", "machdep.cpu.brand_string")) + .stripLineEnd } else if (SystemUtils.IS_OS_LINUX) { Try { val grepPath = Utils.executeAndGetOutput(Seq("which", "grep")).stripLineEnd Utils.executeAndGetOutput(Seq(grepPath, "-m", "1", "model name", "/proc/cpuinfo")) - .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") + .stripLineEnd.replaceFirst("model name[\\s*]:[\\s*]", "") }.getOrElse("Unknown processor") } else { System.getenv("PROCESSOR_IDENTIFIER") diff --git a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala index 89e927e5784d2..a6666db4e95c3 100644 --- a/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala +++ b/core/src/test/scala/org/apache/spark/benchmark/BenchmarkBase.scala @@ -30,7 +30,7 @@ abstract class BenchmarkBase { * Implementations of this method are supposed to use the wrapper method `runBenchmark` * for each benchmark scenario. */ - def runBenchmarkSuite(): Unit + def runBenchmarkSuite(mainArgs: Array[String]): Unit final def runBenchmark(benchmarkName: String)(func: => Any): Unit = { val separator = "=" * 96 @@ -51,12 +51,19 @@ abstract class BenchmarkBase { output = Some(new FileOutputStream(file)) } - runBenchmarkSuite() + runBenchmarkSuite(args) output.foreach { o => if (o != null) { o.close() } } + + afterAll() } + + /** + * Any shutdown code to ensure a clean shutdown + */ + def afterAll(): Unit = {} } diff --git a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala index 9ad2e9a5e74ac..66b2f487dc1cb 100644 --- a/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/broadcast/BroadcastSuite.scala @@ -24,6 +24,8 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark._ +import org.apache.spark.internal.config +import org.apache.spark.internal.config.SERIALIZER import org.apache.spark.io.SnappyCompressionCodec import org.apache.spark.rdd.RDD import org.apache.spark.security.EncryptionFunSuite @@ -67,8 +69,8 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio encryptionTest("Accessing TorrentBroadcast variables in a local cluster") { conf => val numSlaves = 4 - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.broadcast.compress", "true") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(config.BROADCAST_COMPRESS, true) sc = new SparkContext("local-cluster[%d, 1, 1024]".format(numSlaves), "test", conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) @@ -145,8 +147,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio encryptionTest("Cache broadcast to disk") { conf => conf.setMaster("local") .setAppName("test") - .set("spark.memory.useLegacyMode", "true") - .set("spark.storage.memoryFraction", "0.0") + .set(config.MEMORY_STORAGE_FRACTION, 0.0) sc = new SparkContext(conf) val list = List[Int](1, 2, 3, 4) val broadcast = sc.broadcast(list) @@ -173,8 +174,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio val conf = new SparkConf() .setMaster("local[4]") .setAppName("test") - .set("spark.memory.useLegacyMode", "true") - .set("spark.storage.memoryFraction", "0.0") + .set(config.MEMORY_STORAGE_FRACTION, 0.0) sc = new SparkContext(conf) val list = List[Int](1, 2, 3, 4) @@ -296,7 +296,7 @@ class BroadcastSuite extends SparkFunSuite with LocalSparkContext with Encryptio // Using this variable on the executors crashes them, which hangs the test. // Instead, crash the driver by directly accessing the broadcast value. intercept[SparkException] { broadcast.value } - intercept[SparkException] { broadcast.unpersist() } + intercept[SparkException] { broadcast.unpersist(blocking = true) } intercept[SparkException] { broadcast.destroy(blocking = true) } } else { val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) @@ -309,7 +309,7 @@ package object testPackage extends Assertions { def runCallSiteTest(sc: SparkContext) { val broadcast = sc.broadcast(Array(1, 2, 3, 4)) - broadcast.destroy() + broadcast.destroy(blocking = true) val thrown = intercept[SparkException] { broadcast.value } assert(thrown.getMessage.contains("BroadcastSuite.scala")) } diff --git a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala index 55a541d60ea3c..784981ef99cd0 100644 --- a/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala +++ b/core/src/test/scala/org/apache/spark/deploy/DeployTestUtils.scala @@ -64,6 +64,7 @@ private[deploy] object DeployTestUtils { 1234, null, "workerId", + "http://", "host", 123, "publicAddress", diff --git a/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala new file mode 100644 index 0000000000000..e38951f7c3f96 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy + +import java.io._ +import java.nio.charset.StandardCharsets + +import com.google.common.io.CharStreams + +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.network.shuffle.{ExternalShuffleBlockHandler, ExternalShuffleBlockResolver} +import org.apache.spark.network.shuffle.TestShuffleDataContext +import org.apache.spark.util.Utils + +/** + * This suite gets BlockData when the ExternalShuffleService is restarted + * with #spark.shuffle.service.db.enabled = true or false + * Note that failures in this suite may arise when#spark.shuffle.service.db.enabled = false + */ +class ExternalShuffleServiceDbSuite extends SparkFunSuite { + val sortBlock0 = "Hello!" + val sortBlock1 = "World!" + val SORT_MANAGER = "org.apache.spark.shuffle.sort.SortShuffleManager" + + var sparkConf: SparkConf = _ + var dataContext: TestShuffleDataContext = _ + + var securityManager: SecurityManager = _ + var externalShuffleService: ExternalShuffleService = _ + var blockHandler: ExternalShuffleBlockHandler = _ + var blockResolver: ExternalShuffleBlockResolver = _ + + override def beforeAll() { + super.beforeAll() + sparkConf = new SparkConf() + sparkConf.set("spark.shuffle.service.enabled", "true") + sparkConf.set("spark.local.dir", System.getProperty("java.io.tmpdir")) + Utils.loadDefaultSparkProperties(sparkConf, null) + securityManager = new SecurityManager(sparkConf) + + dataContext = new TestShuffleDataContext(2, 5) + dataContext.create() + // Write some sort data. + dataContext.insertSortShuffleData(0, 0, + Array[Array[Byte]](sortBlock0.getBytes(StandardCharsets.UTF_8), + sortBlock1.getBytes(StandardCharsets.UTF_8))) + registerExecutor() + } + + override def afterAll() { + try { + dataContext.cleanup() + } finally { + super.afterAll() + } + } + + def shuffleServiceConf: SparkConf = sparkConf.clone().set(SHUFFLE_SERVICE_PORT, 0) + + def registerExecutor(): Unit = { + try { + sparkConf.set("spark.shuffle.service.db.enabled", "true") + externalShuffleService = new ExternalShuffleService(shuffleServiceConf, securityManager) + + // external Shuffle Service start + externalShuffleService.start() + blockHandler = externalShuffleService.getBlockHandler + blockResolver = blockHandler.getBlockResolver + blockResolver.registerExecutor("app0", "exec0", dataContext.createExecutorInfo(SORT_MANAGER)) + } finally { + blockHandler.close() + // external Shuffle Service stop + externalShuffleService.stop() + } + } + + // The beforeAll ensures the shuffle data was already written, and then + // the shuffle service was stopped. Here we restart the shuffle service + // and make we can read the shuffle data + test("Recover shuffle data with spark.shuffle.service.db.enabled=true after " + + "shuffle service restart") { + try { + sparkConf.set("spark.shuffle.service.db.enabled", "true") + externalShuffleService = new ExternalShuffleService(shuffleServiceConf, securityManager) + // externalShuffleService restart + externalShuffleService.start() + blockHandler = externalShuffleService.getBlockHandler + blockResolver = blockHandler.getBlockResolver + + val block0Stream = blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream + val block0 = CharStreams.toString(new InputStreamReader(block0Stream, StandardCharsets.UTF_8)) + block0Stream.close() + assert(sortBlock0 == block0) + // pass + } finally { + blockHandler.close() + // externalShuffleService stop + externalShuffleService.stop() + } + + } + + // The beforeAll ensures the shuffle data was already written, and then + // the shuffle service was stopped. Here we restart the shuffle service , + // but we can't read the shuffle data + test("Can't recover shuffle data with spark.shuffle.service.db.enabled=false after" + + " shuffle service restart") { + try { + sparkConf.set("spark.shuffle.service.db.enabled", "false") + externalShuffleService = new ExternalShuffleService(shuffleServiceConf, securityManager) + // externalShuffleService restart + externalShuffleService.start() + blockHandler = externalShuffleService.getBlockHandler + blockResolver = blockHandler.getBlockResolver + + val error = intercept[RuntimeException] { + blockResolver.getBlockData("app0", "exec0", 0, 0, 0).createInputStream + }.getMessage + + assert(error.contains("not registered")) + } finally { + blockHandler.close() + // externalShuffleService stop + externalShuffleService.stop() + } + } +} 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 652c36ffa6e71..385f549aa1ad9 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -18,11 +18,10 @@ package org.apache.spark.deploy import java.io._ -import java.net.URI +import java.net.{URI, URL} import java.nio.charset.StandardCharsets import java.nio.file.{Files, Paths} -import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.io.Source @@ -42,6 +41,7 @@ import org.apache.spark.deploy.SparkSubmit._ import org.apache.spark.deploy.SparkSubmitUtils.MavenCoordinate import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher.SparkLauncher import org.apache.spark.scheduler.EventLoggingListener import org.apache.spark.util.{CommandLineUtils, ResetSystemProperties, Utils} @@ -72,27 +72,31 @@ trait TestPrematureExit { mainObject.printStream = printStream @volatile var exitedCleanly = false + val original = mainObject.exitFn mainObject.exitFn = (_) => exitedCleanly = true - - @volatile var exception: Exception = null - val thread = new Thread { - override def run() = try { - mainObject.main(input) - } catch { - // Capture the exception to check whether the exception contains searchString or not - case e: Exception => exception = e + try { + @volatile var exception: Exception = null + val thread = new Thread { + override def run() = try { + mainObject.main(input) + } catch { + // Capture the exception to check whether the exception contains searchString or not + case e: Exception => exception = e + } } - } - thread.start() - thread.join() - if (exitedCleanly) { - val joined = printStream.lineBuffer.mkString("\n") - assert(joined.contains(searchString)) - } else { - assert(exception != null) - if (!exception.getMessage.contains(searchString)) { - throw exception + thread.start() + thread.join() + if (exitedCleanly) { + val joined = printStream.lineBuffer.mkString("\n") + assert(joined.contains(searchString)) + } else { + assert(exception != null) + if (!exception.getMessage.contains(searchString)) { + throw exception + } } + } finally { + mainObject.exitFn = original } } } @@ -190,7 +194,7 @@ class SparkSubmitSuite "--name", "myApp", "--class", "Foo", "--num-executors", "0", - "--conf", "spark.dynamicAllocation.enabled=true", + "--conf", s"${DYN_ALLOCATION_ENABLED.key}=true", "thejar.jar") new SparkSubmitArguments(clArgs1) @@ -198,7 +202,7 @@ class SparkSubmitSuite "--name", "myApp", "--class", "Foo", "--num-executors", "0", - "--conf", "spark.dynamicAllocation.enabled=false", + "--conf", s"${DYN_ALLOCATION_ENABLED.key}=false", "thejar.jar") val e = intercept[SparkException](new SparkSubmitArguments(clArgs2)) @@ -216,7 +220,7 @@ class SparkSubmitSuite val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) appArgs.deployMode should be ("client") - conf.get("spark.submit.deployMode") should be ("client") + conf.get(SUBMIT_DEPLOY_MODE) should be ("client") // Both cmd line and configuration are specified, cmdline option takes the priority val clArgs1 = Seq( @@ -230,7 +234,7 @@ class SparkSubmitSuite val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1) appArgs1.deployMode should be ("cluster") - conf1.get("spark.submit.deployMode") should be ("cluster") + conf1.get(SUBMIT_DEPLOY_MODE) should be ("cluster") // Neither cmdline nor configuration are specified, client mode is the default choice val clArgs2 = Seq( @@ -243,7 +247,7 @@ class SparkSubmitSuite val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) appArgs2.deployMode should be ("client") - conf2.get("spark.submit.deployMode") should be ("client") + conf2.get(SUBMIT_DEPLOY_MODE) should be ("client") } test("handles YARN cluster mode") { @@ -285,7 +289,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.files") should include regex (".*file1.txt,.*file2.txt") conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.app.name") should be ("beauty") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -324,7 +328,7 @@ class SparkSubmitSuite conf.get("spark.yarn.dist.archives") should include regex (".*archive1.txt,.*archive2.txt") conf.get("spark.yarn.dist.jars") should include regex (".*one.jar,.*two.jar,.*three.jar,.*thejar.jar") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) sys.props("SPARK_SUBMIT") should be ("true") } @@ -369,13 +373,13 @@ class SparkSubmitSuite val confMap = conf.getAll.toMap confMap.keys should contain ("spark.master") confMap.keys should contain ("spark.app.name") - confMap.keys should contain ("spark.jars") + confMap.keys should contain (JARS.key) confMap.keys should contain ("spark.driver.memory") - confMap.keys should contain ("spark.driver.cores") - confMap.keys should contain ("spark.driver.supervise") - confMap.keys should contain ("spark.ui.enabled") - confMap.keys should contain ("spark.submit.deployMode") - conf.get("spark.ui.enabled") should be ("false") + confMap.keys should contain (DRIVER_CORES.key) + confMap.keys should contain (DRIVER_SUPERVISE.key) + confMap.keys should contain (UI_ENABLED.key) + confMap.keys should contain (SUBMIT_DEPLOY_MODE.key) + conf.get(UI_ENABLED) should be (false) } test("handles standalone client mode") { @@ -397,7 +401,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles mesos client mode") { @@ -419,7 +423,7 @@ class SparkSubmitSuite classpath(0) should endWith ("thejar.jar") conf.get("spark.executor.memory") should be ("5g") conf.get("spark.cores.max") should be ("5") - conf.get("spark.ui.enabled") should be ("false") + conf.get(UI_ENABLED) should be (false) } test("handles k8s cluster mode") { @@ -462,7 +466,7 @@ class SparkSubmitSuite val (_, _, conf, mainClass) = submit.prepareSubmitEnvironment(appArgs) conf.get("spark.executor.memory") should be ("5g") conf.get("spark.master") should be ("yarn") - conf.get("spark.submit.deployMode") should be ("cluster") + conf.get(SUBMIT_DEPLOY_MODE) should be ("cluster") mainClass should be (SparkSubmit.YARN_CLUSTER_SUBMIT_CLASS) } @@ -474,11 +478,29 @@ class SparkSubmitSuite val appArgs1 = new SparkSubmitArguments(clArgs1) val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1) conf1.get(UI_SHOW_CONSOLE_PROGRESS) should be (true) + var sc1: SparkContext = null + try { + sc1 = new SparkContext(conf1) + assert(sc1.progressBar.isDefined) + } finally { + if (sc1 != null) { + sc1.stop() + } + } val clArgs2 = Seq("--class", "org.SomeClass", "thejar.jar") val appArgs2 = new SparkSubmitArguments(clArgs2) val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) assert(!conf2.contains(UI_SHOW_CONSOLE_PROGRESS)) + var sc2: SparkContext = null + try { + sc2 = new SparkContext(conf2) + assert(!sc2.progressBar.isDefined) + } finally { + if (sc2 != null) { + sc2.stop() + } + } } test("launch simple application with spark-submit") { @@ -494,13 +516,11 @@ class SparkSubmitSuite } test("launch simple application with spark-submit with redaction") { - val testDir = Utils.createTempDir() - testDir.deleteOnExit() - val testDirPath = new Path(testDir.getAbsolutePath()) val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val fileSystem = Utils.getHadoopFileSystem("/", SparkHadoopUtil.get.newConfiguration(new SparkConf())) - try { + withTempDir { testDir => + val testDirPath = new Path(testDir.getAbsolutePath()) val args = Seq( "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", @@ -519,8 +539,6 @@ class SparkSubmitSuite Source.fromInputStream(logData).getLines().foreach { line => assert(!line.contains("secret_password")) } - } finally { - Utils.deleteRecursively(testDir) } } @@ -614,108 +632,112 @@ class SparkSubmitSuite assert(new File(rScriptDir).exists) // compile a small jar containing a class that will be called from R code. - val tempDir = Utils.createTempDir() - val srcDir = new File(tempDir, "sparkrtest") - srcDir.mkdirs() - val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath, - """package sparkrtest; + withTempDir { tempDir => + val srcDir = new File(tempDir, "sparkrtest") + srcDir.mkdirs() + val excSource = new JavaSourceFromString(new File(srcDir, "DummyClass").toURI.getPath, + """package sparkrtest; | |public class DummyClass implements java.io.Serializable { | public static String helloWorld(String arg) { return "Hello " + arg; } | public static int addStuff(int arg1, int arg2) { return arg1 + arg2; } |} - """.stripMargin) - val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) - val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) - val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) + """. + stripMargin) + val excFile = TestUtils.createCompiledClass("DummyClass", srcDir, excSource, Seq.empty) + val jarFile = new File(tempDir, "sparkRTestJar-%s.jar".format(System.currentTimeMillis())) + val jarURL = TestUtils.createJar(Seq(excFile), jarFile, directoryPrefix = Some("sparkrtest")) - val args = Seq( - "--name", "testApp", - "--master", "local", - "--jars", jarURL.toString, - "--verbose", - "--conf", "spark.ui.enabled=false", - rScriptDir) - runSparkSubmit(args) + val args = Seq( + "--name", "testApp", + "--master", "local", + "--jars", jarURL.toString, + "--verbose", + "--conf", "spark.ui.enabled=false", + rScriptDir) + runSparkSubmit(args) + } } test("resolves command line argument paths correctly") { - val dir = Utils.createTempDir() - val archive = Paths.get(dir.toPath.toString, "single.zip") - Files.createFile(archive) - val jars = "/jar1,/jar2" - val files = "local:/file1,file2" - val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" - val pyFiles = "py-file1,py-file2" - - // Test jars and files - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--jars", jars, - "--files", files, - "thejar.jar") - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - appArgs.jars should be (Utils.resolveURIs(jars)) - appArgs.files should be (Utils.resolveURIs(files)) - conf.get("spark.jars") should be (Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be (Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--files", files, - "--archives", archives, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - appArgs2.files should be (Utils.resolveURIs(files)) - appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") - conf2.get("spark.yarn.dist.files") should be (Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should fullyMatch regex - ("file:/archive1,file:.*#archive3") - - // Test python files - val clArgs3 = Seq( - "--master", "local", - "--py-files", pyFiles, - "--conf", "spark.pyspark.driver.python=python3.4", - "--conf", "spark.pyspark.python=python3.5", - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - appArgs3.pyFiles should be (Utils.resolveURIs(pyFiles)) - conf3.get("spark.submit.pyFiles") should be ( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - conf3.get(PYSPARK_DRIVER_PYTHON.key) should be ("python3.4") - conf3.get(PYSPARK_PYTHON.key) should be ("python3.5") + withTempDir { dir => + val archive = Paths.get(dir.toPath.toString, "single.zip") + Files.createFile(archive) + val jars = "/jar1,/jar2" + val files = "local:/file1,file2" + val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" + val pyFiles = "py-file1,py-file2" + + // Test jars and files + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--jars", jars, + "--files", files, + "thejar.jar") + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + appArgs.jars should be(Utils.resolveURIs(jars)) + appArgs.files should be(Utils.resolveURIs(files)) + conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) + conf.get(FILES) should be(Utils.resolveURIs(files).split(",").toSeq) + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) + appArgs2.files should be(Utils.resolveURIs(files)) + appArgs2.archives should fullyMatch regex ("file:/archive1,file:.*#archive3") + conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + conf2.get("spark.yarn.dist.archives") should fullyMatch regex + ("file:/archive1,file:.*#archive3") + + // Test python files + val clArgs3 = Seq( + "--master", "local", + "--py-files", pyFiles, + "--conf", "spark.pyspark.driver.python=python3.4", + "--conf", "spark.pyspark.python=python3.5", + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) + appArgs3.pyFiles should be(Utils.resolveURIs(pyFiles)) + conf3.get(SUBMIT_PYTHON_FILES) should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) + conf3.get(PYSPARK_DRIVER_PYTHON.key) should be("python3.4") + conf3.get(PYSPARK_PYTHON.key) should be("python3.5") + } } test("ambiguous archive mapping results in error message") { - val dir = Utils.createTempDir() - val archive1 = Paths.get(dir.toPath.toString, "first.zip") - val archive2 = Paths.get(dir.toPath.toString, "second.zip") - Files.createFile(archive1) - Files.createFile(archive2) - val jars = "/jar1,/jar2" - val files = "local:/file1,file2" - val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" - val pyFiles = "py-file1,py-file2" - - // Test files and archives (Yarn) - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--files", files, - "--archives", archives, - "thejar.jar" - ) + withTempDir { dir => + val archive1 = Paths.get(dir.toPath.toString, "first.zip") + val archive2 = Paths.get(dir.toPath.toString, "second.zip") + Files.createFile(archive1) + Files.createFile(archive2) + val jars = "/jar1,/jar2" + val files = "local:/file1,file2" + val archives = s"file:/archive1,${dir.toPath.toAbsolutePath.toString}/*.zip#archive3" + val pyFiles = "py-file1,py-file2" + + // Test files and archives (Yarn) + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--files", files, + "--archives", archives, + "thejar.jar" + ) - testPrematureExit(clArgs2.toArray, "resolves ambiguously to multiple files") + testPrematureExit(clArgs2.toArray, "resolves ambiguously to multiple files") + } } test("resolves config paths correctly") { @@ -724,77 +746,77 @@ class SparkSubmitSuite val archives = "file:/archive1,archive2" // spark.yarn.dist.archives val pyFiles = "py-file1,py-file2" // spark.submit.pyFiles - val tmpDir = Utils.createTempDir() - - // Test jars and files - val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) - val writer1 = new PrintWriter(f1) - writer1.println("spark.jars " + jars) - writer1.println("spark.files " + files) - writer1.close() - val clArgs = Seq( - "--master", "local", - "--class", "org.SomeClass", - "--properties-file", f1.getPath, - "thejar.jar" - ) - val appArgs = new SparkSubmitArguments(clArgs) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.jars") should be(Utils.resolveURIs(jars + ",thejar.jar")) - conf.get("spark.files") should be(Utils.resolveURIs(files)) - - // Test files and archives (Yarn) - val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) - val writer2 = new PrintWriter(f2) - writer2.println("spark.yarn.dist.files " + files) - writer2.println("spark.yarn.dist.archives " + archives) - writer2.close() - val clArgs2 = Seq( - "--master", "yarn", - "--class", "org.SomeClass", - "--properties-file", f2.getPath, - "thejar.jar" - ) - val appArgs2 = new SparkSubmitArguments(clArgs2) - val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) - conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) - conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) - - // Test python files - val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) - val writer3 = new PrintWriter(f3) - writer3.println("spark.submit.pyFiles " + pyFiles) - writer3.close() - val clArgs3 = Seq( - "--master", "local", - "--properties-file", f3.getPath, - "mister.py" - ) - val appArgs3 = new SparkSubmitArguments(clArgs3) - val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) - conf3.get("spark.submit.pyFiles") should be( - PythonRunner.formatPaths(Utils.resolveURIs(pyFiles)).mkString(",")) - - // Test remote python files - val hadoopConf = new Configuration() - updateConfWithFakeS3Fs(hadoopConf) - val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) - val pyFile1 = File.createTempFile("file1", ".py", tmpDir) - val pyFile2 = File.createTempFile("file2", ".py", tmpDir) - val writer4 = new PrintWriter(f4) - val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" - writer4.println("spark.submit.pyFiles " + remotePyFiles) - writer4.close() - val clArgs4 = Seq( - "--master", "yarn", - "--deploy-mode", "cluster", - "--properties-file", f4.getPath, - "hdfs:///tmp/mister.py" - ) - val appArgs4 = new SparkSubmitArguments(clArgs4) - val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) - // Should not format python path for yarn cluster mode - conf4.get("spark.submit.pyFiles") should be(Utils.resolveURIs(remotePyFiles)) + withTempDir { tmpDir => + // Test jars and files + val f1 = File.createTempFile("test-submit-jars-files", "", tmpDir) + Utils.tryWithResource(new PrintWriter(f1)) { writer => + writer.println("spark.jars " + jars) + writer.println("spark.files " + files) + } + val clArgs = Seq( + "--master", "local", + "--class", "org.SomeClass", + "--properties-file", f1.getPath, + "thejar.jar" + ) + val appArgs = new SparkSubmitArguments(clArgs) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + conf.get(JARS) should be(Utils.resolveURIs(jars + ",thejar.jar").split(",").toSeq) + conf.get(FILES) should be(Utils.resolveURIs(files).split(",").toSeq) + + // Test files and archives (Yarn) + val f2 = File.createTempFile("test-submit-files-archives", "", tmpDir) + Utils.tryWithResource(new PrintWriter(f2)) { writer => + writer.println("spark.yarn.dist.files " + files) + writer.println("spark.yarn.dist.archives " + archives) + } + val clArgs2 = Seq( + "--master", "yarn", + "--class", "org.SomeClass", + "--properties-file", f2.getPath, + "thejar.jar" + ) + val appArgs2 = new SparkSubmitArguments(clArgs2) + val (_, _, conf2, _) = submit.prepareSubmitEnvironment(appArgs2) + conf2.get("spark.yarn.dist.files") should be(Utils.resolveURIs(files)) + conf2.get("spark.yarn.dist.archives") should be(Utils.resolveURIs(archives)) + + // Test python files + val f3 = File.createTempFile("test-submit-python-files", "", tmpDir) + Utils.tryWithResource(new PrintWriter(f3)) { writer => + writer.println("spark.submit.pyFiles " + pyFiles) + } + val clArgs3 = Seq( + "--master", "local", + "--properties-file", f3.getPath, + "mister.py" + ) + val appArgs3 = new SparkSubmitArguments(clArgs3) + val (_, _, conf3, _) = submit.prepareSubmitEnvironment(appArgs3) + conf3.get(SUBMIT_PYTHON_FILES) should be( + PythonRunner.formatPaths(Utils.resolveURIs(pyFiles))) + + // Test remote python files + val hadoopConf = new Configuration() + updateConfWithFakeS3Fs(hadoopConf) + val f4 = File.createTempFile("test-submit-remote-python-files", "", tmpDir) + val pyFile1 = File.createTempFile("file1", ".py", tmpDir) + val pyFile2 = File.createTempFile("file2", ".py", tmpDir) + val remotePyFiles = s"s3a://${pyFile1.getAbsolutePath},s3a://${pyFile2.getAbsolutePath}" + Utils.tryWithResource(new PrintWriter(f4)) { writer => + writer.println("spark.submit.pyFiles " + remotePyFiles) + } + val clArgs4 = Seq( + "--master", "yarn", + "--deploy-mode", "cluster", + "--properties-file", f4.getPath, + "hdfs:///tmp/mister.py" + ) + val appArgs4 = new SparkSubmitArguments(clArgs4) + val (_, _, conf4, _) = submit.prepareSubmitEnvironment(appArgs4, conf = Some(hadoopConf)) + // Should not format python path for yarn cluster mode + conf4.get(SUBMIT_PYTHON_FILES) should be(Utils.resolveURIs(remotePyFiles).split(",")) + } } test("user classpath first in driver") { @@ -828,22 +850,86 @@ class SparkSubmitSuite } test("support glob path") { + withTempDir { tmpJarDir => + withTempDir { tmpFileDir => + withTempDir { tmpPyFileDir => + withTempDir { tmpArchiveDir => + val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) + val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) + + val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) + val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) + + val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) + val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) + + val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) + val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) + + val tempPyFile = File.createTempFile("tmpApp", ".py") + tempPyFile.deleteOnExit() + + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", s"${tmpJarDir.getAbsolutePath}/*.jar", + "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", + "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", + "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", + tempPyFile.toURI().toString()) + + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) + conf.get("spark.yarn.dist.jars").split(",").toSet should be + (Set(jar1.toURI.toString, jar2.toURI.toString)) + conf.get("spark.yarn.dist.files").split(",").toSet should be + (Set(file1.toURI.toString, file2.toURI.toString)) + conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be + (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) + conf.get("spark.yarn.dist.archives").split(",").toSet should be + (Set(archive1.toURI.toString, archive2.toURI.toString)) + } + } + } + } + } + + test("SPARK-27575: yarn confs should merge new value with existing value") { val tmpJarDir = Utils.createTempDir() val jar1 = TestUtils.createJarWithFiles(Map("test.resource" -> "1"), tmpJarDir) val jar2 = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpJarDir) + val tmpJarDirYarnOpt = Utils.createTempDir() + val jar1YarnOpt = TestUtils.createJarWithFiles(Map("test.resource" -> "2"), tmpJarDirYarnOpt) + val jar2YarnOpt = TestUtils.createJarWithFiles(Map("test.resource" -> "USER2"), + tmpJarDirYarnOpt) + val tmpFileDir = Utils.createTempDir() val file1 = File.createTempFile("tmpFile1", "", tmpFileDir) val file2 = File.createTempFile("tmpFile2", "", tmpFileDir) + val tmpFileDirYarnOpt = Utils.createTempDir() + val file1YarnOpt = File.createTempFile("tmpPy1YarnOpt", ".py", tmpFileDirYarnOpt) + val file2YarnOpt = File.createTempFile("tmpPy2YarnOpt", ".egg", tmpFileDirYarnOpt) + val tmpPyFileDir = Utils.createTempDir() val pyFile1 = File.createTempFile("tmpPy1", ".py", tmpPyFileDir) val pyFile2 = File.createTempFile("tmpPy2", ".egg", tmpPyFileDir) + val tmpPyFileDirYarnOpt = Utils.createTempDir() + val pyFile1YarnOpt = File.createTempFile("tmpPy1YarnOpt", ".py", tmpPyFileDirYarnOpt) + val pyFile2YarnOpt = File.createTempFile("tmpPy2YarnOpt", ".egg", tmpPyFileDirYarnOpt) + val tmpArchiveDir = Utils.createTempDir() val archive1 = File.createTempFile("archive1", ".zip", tmpArchiveDir) val archive2 = File.createTempFile("archive2", ".zip", tmpArchiveDir) + val tmpArchiveDirYarnOpt = Utils.createTempDir() + val archive1YarnOpt = File.createTempFile("archive1YarnOpt", ".zip", tmpArchiveDirYarnOpt) + val archive2YarnOpt = File.createTempFile("archive2YarnOpt", ".zip", tmpArchiveDirYarnOpt) + val tempPyFile = File.createTempFile("tmpApp", ".py") tempPyFile.deleteOnExit() @@ -856,18 +942,35 @@ class SparkSubmitSuite "--files", s"${tmpFileDir.getAbsolutePath}/tmpFile*", "--py-files", s"${tmpPyFileDir.getAbsolutePath}/tmpPy*", "--archives", s"${tmpArchiveDir.getAbsolutePath}/*.zip", + "--conf", "spark.yarn.dist.files=" + + s"${Seq(file1YarnOpt, file2YarnOpt).map(_.toURI.toString).mkString(",")}", + "--conf", "spark.yarn.dist.pyFiles=" + + s"${Seq(pyFile1YarnOpt, pyFile2YarnOpt).map(_.toURI.toString).mkString(",")}", + "--conf", "spark.yarn.dist.jars=" + + s"${Seq(jar1YarnOpt, jar2YarnOpt).map(_.toURI.toString).mkString(",")}", + "--conf", "spark.yarn.dist.archives=" + + s"${Seq(archive1YarnOpt, archive2YarnOpt).map(_.toURI.toString).mkString(",")}", tempPyFile.toURI().toString()) + def assertEqualsWithURLs(expected: Set[URL], confValue: String): Unit = { + val confValPaths = confValue.split(",").map(new Path(_)).toSet + assert(expected.map(u => new Path(u.toURI)) === confValPaths) + } + + def assertEqualsWithFiles(expected: Set[File], confValue: String): Unit = { + assertEqualsWithURLs(expected.map(_.toURI.toURL), confValue) + } + val appArgs = new SparkSubmitArguments(args) val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs) - conf.get("spark.yarn.dist.jars").split(",").toSet should be - (Set(jar1.toURI.toString, jar2.toURI.toString)) - conf.get("spark.yarn.dist.files").split(",").toSet should be - (Set(file1.toURI.toString, file2.toURI.toString)) - conf.get("spark.yarn.dist.pyFiles").split(",").toSet should be - (Set(pyFile1.getAbsolutePath, pyFile2.getAbsolutePath)) - conf.get("spark.yarn.dist.archives").split(",").toSet should be - (Set(archive1.toURI.toString, archive2.toURI.toString)) + assertEqualsWithURLs( + Set(jar1, jar2, jar1YarnOpt, jar2YarnOpt), conf.get("spark.yarn.dist.jars")) + assertEqualsWithFiles( + Set(file1, file2, file1YarnOpt, file2YarnOpt), conf.get("spark.yarn.dist.files")) + assertEqualsWithFiles( + Set(pyFile1, pyFile2, pyFile1YarnOpt, pyFile2YarnOpt), conf.get("spark.yarn.dist.pyFiles")) + assertEqualsWithFiles(Set(archive1, archive2, archive1YarnOpt, archive2YarnOpt), + conf.get("spark.yarn.dist.archives")) } // scalastyle:on println @@ -962,41 +1065,61 @@ class SparkSubmitSuite } } + test("remove copies of application jar from classpath") { + val fs = File.separator + val sparkConf = new SparkConf(false) + val hadoopConf = new Configuration() + val secMgr = new SecurityManager(sparkConf) + + val appJarName = "myApp.jar" + val jar1Name = "myJar1.jar" + val jar2Name = "myJar2.jar" + val userJar = s"file:/path${fs}to${fs}app${fs}jar$fs$appJarName" + val jars = s"file:/$jar1Name,file:/$appJarName,file:/$jar2Name" + + val resolvedJars = DependencyUtils + .resolveAndDownloadJars(jars, userJar, sparkConf, hadoopConf, secMgr) + + assert(!resolvedJars.contains(appJarName)) + assert(resolvedJars.contains(jar1Name) && resolvedJars.contains(jar2Name)) + } + test("Avoid re-upload remote resources in yarn client mode") { val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) - val tmpDir = Utils.createTempDir() - val file = File.createTempFile("tmpFile", "", tmpDir) - val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) - val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) - val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) - val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}" + withTempDir { tmpDir => + val file = File.createTempFile("tmpFile", "", tmpDir) + val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) + val mainResource = File.createTempFile("tmpPy", ".py", tmpDir) + val tmpJar = TestUtils.createJarWithFiles(Map("test.resource" -> "USER"), tmpDir) + val tmpJarPath = s"s3a://${new File(tmpJar.toURI).getAbsolutePath}" - val args = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--jars", tmpJarPath, - "--files", s"s3a://${file.getAbsolutePath}", - "--py-files", s"s3a://${pyFile.getAbsolutePath}", - s"s3a://$mainResource" + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--jars", tmpJarPath, + "--files", s"s3a://${file.getAbsolutePath}", + "--py-files", s"s3a://${pyFile.getAbsolutePath}", + s"s3a://$mainResource" ) - val appArgs = new SparkSubmitArguments(args) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) - // All the resources should still be remote paths, so that YARN client will not upload again. - conf.get("spark.yarn.dist.jars") should be (tmpJarPath) - conf.get("spark.yarn.dist.files") should be (s"s3a://${file.getAbsolutePath}") - conf.get("spark.yarn.dist.pyFiles") should be (s"s3a://${pyFile.getAbsolutePath}") + // All the resources should still be remote paths, so that YARN client will not upload again. + conf.get("spark.yarn.dist.jars") should be(tmpJarPath) + conf.get("spark.yarn.dist.files") should be(s"s3a://${file.getAbsolutePath}") + conf.get("spark.yarn.dist.pyFiles") should be(s"s3a://${pyFile.getAbsolutePath}") - // Local repl jars should be a local path. - conf.get("spark.repl.local.jars") should (startWith("file:")) + // Local repl jars should be a local path. + conf.get("spark.repl.local.jars") should (startWith("file:")) - // local py files should not be a URI format. - conf.get("spark.submit.pyFiles") should (startWith("/")) + // local py files should not be a URI format. + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } + } } test("download remote resource if it is not supported by yarn service") { @@ -1076,18 +1199,13 @@ class SparkSubmitSuite } private def forConfDir(defaults: Map[String, String]) (f: String => Unit) = { - val tmpDir = Utils.createTempDir() - - val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") - val writer = new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) - for ((key, value) <- defaults) writer.write(s"$key $value\n") - - writer.close() - - try { + withTempDir { tmpDir => + val defaultsConf = new File(tmpDir.getAbsolutePath, "spark-defaults.conf") + val writer = + new OutputStreamWriter(new FileOutputStream(defaultsConf), StandardCharsets.UTF_8) + for ((key, value) <- defaults) writer.write(s"$key $value\n") + writer.close() f(tmpDir.getAbsolutePath) - } finally { - Utils.deleteRecursively(tmpDir) } } @@ -1115,39 +1233,40 @@ class SparkSubmitSuite val hadoopConf = new Configuration() updateConfWithFakeS3Fs(hadoopConf) - val tmpDir = Utils.createTempDir() - val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) + withTempDir { tmpDir => + val pyFile = File.createTempFile("tmpPy", ".egg", tmpDir) - val args = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--py-files", s"s3a://${pyFile.getAbsolutePath}", - "spark-internal" - ) + val args = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--py-files", s"s3a://${pyFile.getAbsolutePath}", + "spark-internal" + ) - val appArgs = new SparkSubmitArguments(args) - val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) + val appArgs = new SparkSubmitArguments(args) + val (_, _, conf, _) = submit.prepareSubmitEnvironment(appArgs, conf = Some(hadoopConf)) - conf.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf.get("spark.submit.pyFiles") should (startWith("/")) + conf.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } - // Verify "spark.submit.pyFiles" - val args1 = Seq( - "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), - "--name", "testApp", - "--master", "yarn", - "--deploy-mode", "client", - "--conf", s"spark.submit.pyFiles=s3a://${pyFile.getAbsolutePath}", - "spark-internal" - ) + // Verify "spark.submit.pyFiles" + val args1 = Seq( + "--class", UserClasspathFirstTest.getClass.getName.stripPrefix("$"), + "--name", "testApp", + "--master", "yarn", + "--deploy-mode", "client", + "--conf", s"spark.submit.pyFiles=s3a://${pyFile.getAbsolutePath}", + "spark-internal" + ) - val appArgs1 = new SparkSubmitArguments(args1) - val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) + val appArgs1 = new SparkSubmitArguments(args1) + val (_, _, conf1, _) = submit.prepareSubmitEnvironment(appArgs1, conf = Some(hadoopConf)) - conf1.get(PY_FILES.key) should be (s"s3a://${pyFile.getAbsolutePath}") - conf1.get("spark.submit.pyFiles") should (startWith("/")) + conf1.get(PY_FILES.key) should be(s"s3a://${pyFile.getAbsolutePath}") + conf.get(SUBMIT_PYTHON_FILES).foreach { _ should (startWith("/")) } + } } test("handles natural line delimiters in --properties-file and --conf uniformly") { @@ -1196,6 +1315,23 @@ class SparkSubmitSuite conf.get(nonDelimSpaceFromFile._1) should be ("blah") } + + test("get a Spark configuration from arguments") { + val testConf = "spark.test.hello" -> "world" + val masterConf = "spark.master" -> "yarn" + val clArgs = Seq( + "--conf", s"${testConf._1}=${testConf._2}", + "--conf", s"${masterConf._1}=${masterConf._2}", + "--class", "Foo", + "app.jar") + val conf = new SparkSubmitArguments(clArgs).toSparkConf() + Seq( + testConf, + masterConf + ).foreach { case (k, v) => + conf.get(k) should be (v) + } + } } object SparkSubmitSuite extends SparkFunSuite with TimeLimits { @@ -1217,7 +1353,7 @@ object SparkSubmitSuite extends SparkFunSuite with TimeLimits { Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) try { - val exitCode = failAfter(60 seconds) { process.waitFor() } + val exitCode = failAfter(1.minute) { process.waitFor() } if (exitCode != 0) { fail(s"Process returned with exit code $exitCode. See the log4j logs for more detail.") } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala index a0f09891787e0..8e1a519e187ce 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala @@ -94,8 +94,8 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("add dependencies works correctly") { val md = SparkSubmitUtils.getModuleDescriptor - val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.11:0.1," + - "com.databricks:spark-avro_2.11:0.1") + val artifacts = SparkSubmitUtils.extractMavenCoordinates("com.databricks:spark-csv_2.12:0.1," + + "com.databricks:spark-avro_2.12:0.1") SparkSubmitUtils.addDependenciesToIvy(md, artifacts, "default") assert(md.getDependencies.length === 2) @@ -189,7 +189,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { test("neglects Spark and Spark's dependencies") { val coordinates = SparkSubmitUtils.IVY_DEFAULT_EXCLUDES - .map(comp => s"org.apache.spark:spark-${comp}2.11:2.1.1") + .map(comp => s"org.apache.spark:spark-${comp}2.12:2.4.0") .mkString(",") + ",org.apache.spark:spark-core_fake:1.2.0" val path = SparkSubmitUtils.resolveMavenCoordinates( @@ -197,7 +197,7 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with BeforeAndAfterAll { SparkSubmitUtils.buildIvySettings(None, None), isTest = true) assert(path === "", "should return empty path") - val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.11", "1.2.0") + val main = MavenCoordinate("org.apache.spark", "spark-streaming-kafka-assembly_2.12", "1.2.0") IvyTestUtils.withRepository(main, None, None) { repo => val files = SparkSubmitUtils.resolveMavenCoordinates( coordinates + "," + main.toString, 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 a1d2a1283db14..4c6a669a31324 100644 --- a/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/StandaloneDynamicAllocationSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy import scala.collection.mutable import scala.concurrent.duration._ -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, verify, when} import org.scalatest.{BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually._ @@ -67,7 +67,7 @@ class StandaloneDynamicAllocationSuite master = makeMaster() workers = makeWorkers(10, 2048) // Wait until all workers register with master successfully - eventually(timeout(60.seconds), interval(10.millis)) { + eventually(timeout(1.minute), interval(10.milliseconds)) { assert(getMasterState.workers.size === numWorkers) } } @@ -135,7 +135,7 @@ class StandaloneDynamicAllocationSuite } test("dynamic allocation with max cores <= cores per worker") { - sc = new SparkContext(appConf.set("spark.cores.max", "8")) + sc = new SparkContext(appConf.set(config.CORES_MAX, 8)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -190,7 +190,7 @@ class StandaloneDynamicAllocationSuite } test("dynamic allocation with max cores > cores per worker") { - sc = new SparkContext(appConf.set("spark.cores.max", "16")) + sc = new SparkContext(appConf.set(config.CORES_MAX, 16)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -243,7 +243,7 @@ class StandaloneDynamicAllocationSuite } test("dynamic allocation with cores per executor") { - sc = new SparkContext(appConf.set("spark.executor.cores", "2")) + sc = new SparkContext(appConf.set(config.EXECUTOR_CORES, 2)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -296,8 +296,8 @@ class StandaloneDynamicAllocationSuite test("dynamic allocation with cores per executor AND max cores") { sc = new SparkContext(appConf - .set("spark.executor.cores", "2") - .set("spark.cores.max", "8")) + .set(config.EXECUTOR_CORES, 2) + .set(config.CORES_MAX, 8)) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { val apps = getApplications() @@ -457,9 +457,9 @@ class StandaloneDynamicAllocationSuite test("initial executor limit") { val initialExecutorLimit = 1 val myConf = appConf - .set("spark.dynamicAllocation.enabled", "true") - .set(config.SHUFFLE_SERVICE_ENABLED.key, "true") - .set("spark.dynamicAllocation.initialExecutors", initialExecutorLimit.toString) + .set(config.DYN_ALLOCATION_ENABLED, true) + .set(config.SHUFFLE_SERVICE_ENABLED, true) + .set(config.DYN_ALLOCATION_INITIAL_EXECUTORS, initialExecutorLimit) sc = new SparkContext(myConf) val appId = sc.applicationId eventually(timeout(10.seconds), interval(10.millis)) { @@ -497,7 +497,8 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty) + val message = RegisterExecutor("one", endpointRef, "blacklisted-host", 10, Map.empty, Map.empty, + Map.empty) // Get "localhost" on a blacklist. val taskScheduler = mock(classOf[TaskSchedulerImpl]) @@ -526,7 +527,7 @@ class StandaloneDynamicAllocationSuite new SparkConf() .setMaster(masterRpcEnv.address.toSparkURL) .setAppName("test") - .set("spark.executor.memory", "256m") + .set(config.EXECUTOR_MEMORY.key, "256m") } /** Make a master to which our application will send executor requests. */ @@ -621,7 +622,8 @@ class StandaloneDynamicAllocationSuite val endpointRef = mock(classOf[RpcEndpointRef]) val mockAddress = mock(classOf[RpcAddress]) when(endpointRef.address).thenReturn(mockAddress) - val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty) + val message = RegisterExecutor(id, endpointRef, "localhost", 10, Map.empty, Map.empty, + Map.empty) val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] backend.driverEndpoint.askSync[Boolean](message) } diff --git a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala index a1707e6540b39..a1d3077b8fc87 100644 --- a/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/client/AppClientSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.deploy.client +import java.io.Closeable import java.util.concurrent.ConcurrentLinkedQueue import scala.concurrent.duration._ @@ -64,7 +65,7 @@ class AppClientSuite master = makeMaster() workers = makeWorkers(10, 2048) // Wait until all workers register with master successfully - eventually(timeout(60.seconds), interval(10.millis)) { + eventually(timeout(1.minute), interval(10.milliseconds)) { assert(getMasterState.workers.size === numWorkers) } } @@ -85,57 +86,59 @@ class AppClientSuite } test("interface methods of AppClient using local Master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - ci.client.start() + ci.client.start() - // Client should connect with one Master which registers the application - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") - assert(apps.size === 1, "master should have 1 registered app") - } + // Client should connect with one Master which registers the application + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.connectedIdList.size === 1, "client listener should have one connection") + assert(apps.size === 1, "master should have 1 registered app") + } - // Send message to Master to request Executors, verify request by change in executor limit - val numExecutorsRequested = 1 - whenReady( + // Send message to Master to request Executors, verify request by change in executor limit + val numExecutorsRequested = 1 + whenReady( ci.client.requestTotalExecutors(numExecutorsRequested), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") - } + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(apps.head.getExecutorLimit === numExecutorsRequested, s"executor request failed") + } - // Send request to kill executor, verify request was made - val executorId: String = getApplications().head.executors.head._2.fullId - whenReady( + // Send request to kill executor, verify request was made + val executorId: String = getApplications().head.executors.head._2.fullId + whenReady( ci.client.killExecutors(Seq(executorId)), timeout(10.seconds), interval(10.millis)) { acknowledged => - assert(acknowledged) - } + assert(acknowledged) + } - // Issue stop command for Client to disconnect from Master - ci.client.stop() + // Issue stop command for Client to disconnect from Master + ci.client.stop() - // Verify Client is marked dead and unregistered from Master - eventually(timeout(10.seconds), interval(10.millis)) { - val apps = getApplications() - assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") - assert(apps.isEmpty, "master should have 0 registered apps") + // Verify Client is marked dead and unregistered from Master + eventually(timeout(10.seconds), interval(10.millis)) { + val apps = getApplications() + assert(ci.listener.deadReasonList.size === 1, "client should have been marked dead") + assert(apps.isEmpty, "master should have 0 registered apps") + } } } test("request from AppClient before initialized with master") { - val ci = new AppClientInst(masterRpcEnv.address.toSparkURL) + Utils.tryWithResource(new AppClientInst(masterRpcEnv.address.toSparkURL)) { ci => - // requests to master should fail immediately - whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => - assert(success === false) + // requests to master should fail immediately + whenReady(ci.client.requestTotalExecutors(3), timeout(1.seconds)) { success => + assert(success === false) + } } } @@ -219,13 +222,17 @@ class AppClientSuite } /** Create AppClient and supporting objects */ - private class AppClientInst(masterUrl: String) { + private class AppClientInst(masterUrl: String) extends Closeable { val rpcEnv = RpcEnv.create("spark", Utils.localHostName(), 0, conf, securityManager) private val cmd = new Command(TestExecutor.getClass.getCanonicalName.stripSuffix("$"), List(), Map(), Seq(), Seq(), Seq()) private val desc = new ApplicationDescription("AppClientSuite", Some(1), 512, cmd, "ignored") val listener = new AppClientCollector val client = new StandaloneAppClient(rpcEnv, Array(masterUrl), desc, listener, new SparkConf) + + override def close(): Unit = { + rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala index 44f9c566a380d..1148446c9faa1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/ApplicationCacheSuite.scala @@ -24,10 +24,9 @@ import scala.collection.mutable import com.codahale.metrics.Counter import org.eclipse.jetty.servlet.ServletContextHandler -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.Matchers import org.scalatest.mockito.MockitoSugar @@ -374,11 +373,9 @@ class ApplicationCacheSuite extends SparkFunSuite with Logging with MockitoSugar when(request.getRequestURI()).thenReturn("http://localhost:18080/history/local-123/jobs/job/") when(request.getQueryString()).thenReturn("id=2") val resp = mock[HttpServletResponse] - when(resp.encodeRedirectURL(any())).thenAnswer(new Answer[String]() { - override def answer(invocationOnMock: InvocationOnMock): String = { - invocationOnMock.getArguments()(0).asInstanceOf[String] - } - }) + when(resp.encodeRedirectURL(any())).thenAnswer { (invocationOnMock: InvocationOnMock) => + invocationOnMock.getArguments()(0).asInstanceOf[String] + } filter.doFilter(request, resp, null) verify(resp).sendRedirect("http://localhost:18080/history/local-123/jobs/job/?id=2") } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala index 444e8d6e11f88..aaf068e81db0a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/FsHistoryProviderSuite.scala @@ -23,41 +23,49 @@ import java.util.{Date, Locale} import java.util.concurrent.TimeUnit import java.util.zip.{ZipInputStream, ZipOutputStream} +import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import scala.language.postfixOps import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.hadoop.hdfs.DistributedFileSystem +import org.apache.commons.io.FileUtils +import org.apache.hadoop.fs.{FileStatus, FileSystem, FSDataInputStream, Path} +import org.apache.hadoop.hdfs.{DFSInputStream, DistributedFileSystem} import org.apache.hadoop.security.AccessControlException import org.json4s.jackson.JsonMethods._ -import org.mockito.ArgumentMatcher -import org.mockito.Matchers.{any, argThat} +import org.mockito.ArgumentMatchers.{any, argThat} import org.mockito.Mockito.{doThrow, mock, spy, verify, when} -import org.scalatest.BeforeAndAfter import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.history.config._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.DRIVER_LOG_DFS_DIR +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.UI.{ADMIN_ACLS, ADMIN_ACLS_GROUPS, USER_GROUPS_MAPPING} import org.apache.spark.io._ import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.security.GroupMappingServiceProvider import org.apache.spark.status.AppStatusStore import org.apache.spark.status.api.v1.{ApplicationAttemptInfo, ApplicationInfo} import org.apache.spark.util.{Clock, JsonProtocol, ManualClock, Utils} +import org.apache.spark.util.logging.DriverLogger -class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matchers with Logging { +class FsHistoryProviderSuite extends SparkFunSuite with Matchers with Logging { private var testDir: File = null - before { + override def beforeEach(): Unit = { + super.beforeEach() testDir = Utils.createTempDir(namePrefix = s"a b%20c+d") } - after { - Utils.deleteRecursively(testDir) + override def afterEach(): Unit = { + try { + Utils.deleteRecursively(testDir) + } finally { + super.afterEach() + } } /** Create a fake log file using the new log format used in Spark 1.3+ */ @@ -137,7 +145,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc clock.getTimeMillis(), "test", false)) // Make sure the UI can be rendered. - list.foreach { case info => + list.foreach { info => val appUi = provider.getAppUI(info.id, None) appUi should not be null appUi should not be None @@ -276,7 +284,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc list.last.attempts.size should be (3) list.head.attempts.head.attemptId should be (Some("attempt1")) - list.foreach { case app => + list.foreach { app => app.attempts.foreach { attempt => val appUi = provider.getAppUI(app.id, attempt.attemptId) appUi should not be null @@ -286,11 +294,197 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + test("log urls without customization") { + val conf = createTestConf() + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("custom log urls, including FILE_NAME") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) + + // some of available attributes are not used in pattern which should be OK + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrlMap = attr("LOG_FILES").split(",").map { file => + val newLogUrl = getExpectedExecutorLogUrl(attr, Some(file)) + file -> newLogUrl + }.toMap + + execInfo -> newLogUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("custom log urls, excluding FILE_NAME") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = false)) + + // some of available attributes are not used in pattern which should be OK + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrl = getExpectedExecutorLogUrl(attr, None) + + execInfo -> Map("log" -> newLogUrl) + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("custom log urls with invalid attribute") { + // Here we are referring {{NON_EXISTING}} which is not available in attributes, + // which Spark will fail back to provide origin log url with warning log. + + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true) + + "/{{NON_EXISTING}}") + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("custom log urls, LOG_FILES not available while FILE_NAME is specified") { + // For this case Spark will fail back to provide origin log url with warning log. + + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) + + val executorInfos = (1 to 5).map( + createTestExecutorInfo("app1", "user1", _, includingLogFiles = false)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected) + } + + test("custom log urls, app not finished, applyIncompleteApplication: true") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) + .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, true) + + // ensure custom log urls are applied to incomplete application + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + val attr = execInfo.attributes + val newLogUrlMap = attr("LOG_FILES").split(",").map { file => + val newLogUrl = getExpectedExecutorLogUrl(attr, Some(file)) + file -> newLogUrl + }.toMap + + execInfo -> newLogUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected, isCompletedApp = false) + } + + test("custom log urls, app not finished, applyIncompleteApplication: false") { + val conf = createTestConf() + .set(CUSTOM_EXECUTOR_LOG_URL, getCustomExecutorLogUrl(includeFileName = true)) + .set(APPLY_CUSTOM_EXECUTOR_LOG_URL_TO_INCOMPLETE_APP, false) + + // ensure custom log urls are NOT applied to incomplete application + + val executorInfos = (1 to 5).map(createTestExecutorInfo("app1", "user1", _)) + + val expected: Map[ExecutorInfo, Map[String, String]] = executorInfos.map { execInfo => + execInfo -> execInfo.logUrlMap + }.toMap + + testHandlingExecutorLogUrl(conf, expected, isCompletedApp = false) + } + + private def getCustomExecutorLogUrl(includeFileName: Boolean): String = { + val baseUrl = "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}/users/{{USER}}/containers/" + + "{{CONTAINER_ID}}" + if (includeFileName) baseUrl + "/{{FILE_NAME}}" else baseUrl + } + + private def getExpectedExecutorLogUrl( + attributes: Map[String, String], + fileName: Option[String]): String = { + val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" + + fileName match { + case Some(file) => baseUrl + s"/$file" + case None => baseUrl + } + } + + private def testHandlingExecutorLogUrl( + conf: SparkConf, + expectedLogUrlMap: Map[ExecutorInfo, Map[String, String]], + isCompletedApp: Boolean = true): Unit = { + val provider = new FsHistoryProvider(conf) + + val attempt1 = newLogFile("app1", Some("attempt1"), inProgress = true) + + val executorAddedEvents = expectedLogUrlMap.keys.zipWithIndex.map { case (execInfo, idx) => + SparkListenerExecutorAdded(1 + idx, s"exec$idx", execInfo) + }.toList.sortBy(_.time) + val allEvents = List(SparkListenerApplicationStart("app1", Some("app1"), 1L, + "test", Some("attempt1"))) ++ executorAddedEvents ++ + (if (isCompletedApp) List(SparkListenerApplicationEnd(1000L)) else Seq()) + + writeFile(attempt1, true, None, allEvents: _*) + + updateAndCheck(provider) { list => + list.size should be (1) + list.head.attempts.size should be (1) + + list.foreach { app => + app.attempts.foreach { attempt => + val appUi = provider.getAppUI(app.id, attempt.attemptId) + appUi should not be null + val executors = appUi.get.ui.store.executorList(false).iterator + executors should not be null + + val iterForExpectation = expectedLogUrlMap.iterator + + var executorCount = 0 + while (executors.hasNext) { + val executor = executors.next() + val (expectedExecInfo, expectedLogs) = iterForExpectation.next() + + executor.hostPort should startWith(expectedExecInfo.executorHost) + executor.executorLogs should be(expectedLogs) + + executorCount += 1 + } + + executorCount should be (expectedLogUrlMap.size) + } + } + } + } + test("log cleaner") { val maxAge = TimeUnit.SECONDS.toMillis(10) val clock = new ManualClock(maxAge / 2) val provider = new FsHistoryProvider( - createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + createTestConf().set(MAX_LOG_AGE_S.key, s"${maxAge}ms"), clock) val log1 = newLogFile("app1", Some("attempt1"), inProgress = false) writeFile(log1, true, None, @@ -330,13 +524,52 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc assert(!log2.exists()) } + test("should not clean inprogress application with lastUpdated time less than maxTime") { + val firstFileModifiedTime = TimeUnit.DAYS.toMillis(1) + val secondFileModifiedTime = TimeUnit.DAYS.toMillis(6) + val maxAge = TimeUnit.DAYS.toMillis(7) + val clock = new ManualClock(0) + val provider = new FsHistoryProvider( + createTestConf().set(MAX_LOG_AGE_S, maxAge / 1000), clock) + val log = newLogFile("inProgressApp1", None, inProgress = true) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")) + ) + clock.setTime(firstFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null) + ) + + clock.setTime(secondFileModifiedTime) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + clock.setTime(TimeUnit.DAYS.toMillis(10)) + writeFile(log, true, None, + SparkListenerApplicationStart( + "inProgressApp1", Some("inProgressApp1"), 3L, "test", Some("attempt1")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerJobEnd(0, 1L, JobSucceeded) + ) + log.setLastModified(clock.getTimeMillis()) + provider.checkForLogs() + // This should not trigger any cleanup + updateAndCheck(provider) { list => + list.size should be(1) + } + } + test("log cleaner for inProgress files") { val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10) val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) val maxAge = TimeUnit.SECONDS.toMillis(40) val clock = new ManualClock(0) val provider = new FsHistoryProvider( - createTestConf().set("spark.history.fs.cleaner.maxAge", s"${maxAge}ms"), clock) + createTestConf().set(MAX_LOG_AGE_S.key, s"${maxAge}ms"), clock) val log1 = newLogFile("inProgressApp1", None, inProgress = true) writeFile(log1, true, None, @@ -413,6 +646,62 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } + test("driver log cleaner") { + val firstFileModifiedTime = TimeUnit.SECONDS.toMillis(10) + val secondFileModifiedTime = TimeUnit.SECONDS.toMillis(20) + val maxAge = TimeUnit.SECONDS.toSeconds(40) + val clock = new ManualClock(0) + val testConf = new SparkConf() + testConf.set(HISTORY_LOG_DIR, Utils.createTempDir(namePrefix = "eventLog").getAbsolutePath()) + testConf.set(DRIVER_LOG_DFS_DIR, testDir.getAbsolutePath()) + testConf.set(DRIVER_LOG_CLEANER_ENABLED, true) + testConf.set(DRIVER_LOG_CLEANER_INTERVAL, maxAge / 4) + testConf.set(MAX_DRIVER_LOG_AGE_S, maxAge) + val provider = new FsHistoryProvider(testConf, clock) + + val log1 = FileUtils.getFile(testDir, "1" + DriverLogger.DRIVER_LOG_FILE_SUFFIX) + createEmptyFile(log1) + clock.setTime(firstFileModifiedTime) + log1.setLastModified(clock.getTimeMillis()) + provider.cleanDriverLogs() + + val log2 = FileUtils.getFile(testDir, "2" + DriverLogger.DRIVER_LOG_FILE_SUFFIX) + createEmptyFile(log2) + val log3 = FileUtils.getFile(testDir, "3" + DriverLogger.DRIVER_LOG_FILE_SUFFIX) + createEmptyFile(log3) + clock.setTime(secondFileModifiedTime) + log2.setLastModified(clock.getTimeMillis()) + log3.setLastModified(clock.getTimeMillis()) + // This should not trigger any cleanup + provider.cleanDriverLogs() + provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(3) + + // Should trigger cleanup for first file but not second one + clock.setTime(firstFileModifiedTime + TimeUnit.SECONDS.toMillis(maxAge) + 1) + provider.cleanDriverLogs() + provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(2) + assert(!log1.exists()) + assert(log2.exists()) + assert(log3.exists()) + + // Update the third file length while keeping the original modified time + Files.write("Add logs to file".getBytes(), log3) + log3.setLastModified(secondFileModifiedTime) + // Should cleanup the second file but not the third file, as filelength changed. + clock.setTime(secondFileModifiedTime + TimeUnit.SECONDS.toMillis(maxAge) + 1) + provider.cleanDriverLogs() + provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(1) + assert(!log1.exists()) + assert(!log2.exists()) + assert(log3.exists()) + + // Should cleanup the third file as well. + clock.setTime(secondFileModifiedTime + 2 * TimeUnit.SECONDS.toMillis(maxAge) + 2) + provider.cleanDriverLogs() + provider.listing.view(classOf[LogInfo]).iterator().asScala.toSeq.size should be(0) + assert(!log3.exists()) + } + test("SPARK-8372: new logs with no app ID are ignored") { val provider = new FsHistoryProvider(createTestConf()) @@ -448,7 +737,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc provider.inSafeMode = false clock.setTime(10000) - eventually(timeout(1 second), interval(10 millis)) { + eventually(timeout(3.second), interval(10.milliseconds)) { provider.getConfig().keys should not contain ("HDFS State") } } finally { @@ -456,17 +745,17 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } } - test("provider reports error after FS leaves safe mode") { + testRetry("provider reports error after FS leaves safe mode") { testDir.delete() val clock = new ManualClock() val provider = new SafeModeTestProvider(createTestConf(), clock) val errorHandler = mock(classOf[Thread.UncaughtExceptionHandler]) - val initThread = provider.startSafeModeCheckThread(Some(errorHandler)) + provider.startSafeModeCheckThread(Some(errorHandler)) try { provider.inSafeMode = false clock.setTime(10000) - eventually(timeout(1 second), interval(10 millis)) { + eventually(timeout(3.second), interval(10.milliseconds)) { verify(errorHandler).uncaughtException(any(), any()) } } finally { @@ -481,11 +770,11 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // write out one totally bogus hidden file val hiddenGarbageFile = new File(testDir, ".garbage") - val out = new PrintWriter(hiddenGarbageFile) - // scalastyle:off println - out.println("GARBAGE") - // scalastyle:on println - out.close() + Utils.tryWithResource(new PrintWriter(hiddenGarbageFile)) { out => + // scalastyle:off println + out.println("GARBAGE") + // scalastyle:on println + } // also write out one real event log file, but since its a hidden file, we shouldn't read it val tmpNewAppFile = newLogFile("hidden", None, inProgress = false) @@ -524,6 +813,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc "test", Some("attempt1")), SparkListenerEnvironmentUpdate(Map( "Spark Properties" -> properties.toSeq, + "Hadoop Properties" -> Seq.empty, "JVM Information" -> Seq.empty, "System Properties" -> Seq.empty, "Classpath Entries" -> Seq.empty @@ -545,12 +835,12 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test both history ui admin acls and application acls are configured. val conf1 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) - createAndCheck(conf1, ("spark.admin.acls", "user"), ("spark.admin.acls.groups", "group")) { + createAndCheck(conf1, (ADMIN_ACLS.key, "user"), (ADMIN_ACLS_GROUPS.key, "group")) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -567,10 +857,10 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test only history ui admin acls are configured. val conf2 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.history.ui.admin.acls", "user1,user2") - .set("spark.history.ui.admin.acls.groups", "group1") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(HISTORY_SERVER_UI_ADMIN_ACLS, Seq("user1", "user2")) + .set(HISTORY_SERVER_UI_ADMIN_ACLS_GROUPS, Seq("group1")) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf2) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (true) @@ -587,8 +877,8 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc // Test neither history ui admin acls nor application acls are configured. val conf3 = createTestConf() - .set("spark.history.ui.acls.enable", "true") - .set("spark.user.groups.mapping", classOf[TestGroupsMappingProvider].getName) + .set(HISTORY_SERVER_UI_ACLS_ENABLE, true) + .set(USER_GROUPS_MAPPING, classOf[TestGroupsMappingProvider].getName) createAndCheck(conf3) { securityManager => // Test whether user has permission to access UI. securityManager.checkUIViewPermissions("user1") should be (false) @@ -667,53 +957,54 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc } test("clean up stale app information") { - val storeDir = Utils.createTempDir() - val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - val clock = new ManualClock() - val provider = spy(new FsHistoryProvider(conf, clock)) - val appId = "new1" - - // Write logs for two app attempts. - clock.advance(1) - val attempt1 = newLogFile(appId, Some("1"), inProgress = false) - writeFile(attempt1, true, None, - SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("1")), - SparkListenerJobStart(0, 1L, Nil, null), - SparkListenerApplicationEnd(5L) + withTempDir { storeDir => + val conf = createTestConf().set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) + val clock = new ManualClock() + val provider = spy(new FsHistoryProvider(conf, clock)) + val appId = "new1" + + // Write logs for two app attempts. + clock.advance(1) + val attempt1 = newLogFile(appId, Some("1"), inProgress = false) + writeFile(attempt1, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("1")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerApplicationEnd(5L) ) - val attempt2 = newLogFile(appId, Some("2"), inProgress = false) - writeFile(attempt2, true, None, - SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("2")), - SparkListenerJobStart(0, 1L, Nil, null), - SparkListenerApplicationEnd(5L) + val attempt2 = newLogFile(appId, Some("2"), inProgress = false) + writeFile(attempt2, true, None, + SparkListenerApplicationStart(appId, Some(appId), 1L, "test", Some("2")), + SparkListenerJobStart(0, 1L, Nil, null), + SparkListenerApplicationEnd(5L) ) - updateAndCheck(provider) { list => - assert(list.size === 1) - assert(list(0).id === appId) - assert(list(0).attempts.size === 2) - } - - // Load the app's UI. - val ui = provider.getAppUI(appId, Some("1")) - assert(ui.isDefined) - - // Delete the underlying log file for attempt 1 and rescan. The UI should go away, but since - // attempt 2 still exists, listing data should be there. - clock.advance(1) - attempt1.delete() - updateAndCheck(provider) { list => - assert(list.size === 1) - assert(list(0).id === appId) - assert(list(0).attempts.size === 1) - } - assert(!ui.get.valid) - assert(provider.getAppUI(appId, None) === None) + updateAndCheck(provider) { list => + assert(list.size === 1) + assert(list(0).id === appId) + assert(list(0).attempts.size === 2) + } - // Delete the second attempt's log file. Now everything should go away. - clock.advance(1) - attempt2.delete() - updateAndCheck(provider) { list => - assert(list.isEmpty) + // Load the app's UI. + val ui = provider.getAppUI(appId, Some("1")) + assert(ui.isDefined) + + // Delete the underlying log file for attempt 1 and rescan. The UI should go away, but since + // attempt 2 still exists, listing data should be there. + clock.advance(1) + attempt1.delete() + updateAndCheck(provider) { list => + assert(list.size === 1) + assert(list(0).id === appId) + assert(list(0).attempts.size === 1) + } + assert(!ui.get.valid) + assert(provider.getAppUI(appId, None) === None) + + // Delete the second attempt's log file. Now everything should go away. + clock.advance(1) + attempt2.delete() + updateAndCheck(provider) { list => + assert(list.isEmpty) + } } } @@ -781,6 +1072,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc SparkListenerApplicationStart("end-event-test", Some("end-event-test"), 1L, "test", None), SparkListenerEnvironmentUpdate(Map( "Spark Properties" -> Seq.empty, + "Hadoop Properties" -> Seq.empty, "JVM Information" -> Seq.empty, "System Properties" -> Seq.empty, "Classpath Entries" -> Seq.empty @@ -830,21 +1122,16 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc writeFile(accessGranted, true, None, SparkListenerApplicationStart("accessGranted", Some("accessGranted"), 1L, "test", None), SparkListenerApplicationEnd(5L)) + var isReadable = false val mockedFs = spy(provider.fs) doThrow(new AccessControlException("Cannot read accessDenied file")).when(mockedFs).open( - argThat(new ArgumentMatcher[Path]() { - override def matches(path: Any): Boolean = { - path.asInstanceOf[Path].getName.toLowerCase(Locale.ROOT) == "accessdenied" - } - })) + argThat((path: Path) => path.getName.toLowerCase(Locale.ROOT) == "accessdenied" && + !isReadable)) val mockedProvider = spy(provider) when(mockedProvider.fs).thenReturn(mockedFs) updateAndCheck(mockedProvider) { list => list.size should be(1) } - writeFile(accessDenied, true, None, - SparkListenerApplicationStart("accessDenied", Some("accessDenied"), 1L, "test", None), - SparkListenerApplicationEnd(5L)) // Doing 2 times in order to check the blacklist filter too updateAndCheck(mockedProvider) { list => list.size should be(1) @@ -852,8 +1139,100 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc val accessDeniedPath = new Path(accessDenied.getPath) assert(mockedProvider.isBlacklisted(accessDeniedPath)) clock.advance(24 * 60 * 60 * 1000 + 1) // add a bit more than 1d + isReadable = true mockedProvider.cleanLogs() - assert(!mockedProvider.isBlacklisted(accessDeniedPath)) + updateAndCheck(mockedProvider) { list => + assert(!mockedProvider.isBlacklisted(accessDeniedPath)) + assert(list.exists(_.name == "accessDenied")) + assert(list.exists(_.name == "accessGranted")) + list.size should be(2) + } + } + + test("check in-progress event logs absolute length") { + val path = new Path("testapp.inprogress") + val provider = new FsHistoryProvider(createTestConf()) + val mockedProvider = spy(provider) + val mockedFs = mock(classOf[FileSystem]) + val in = mock(classOf[FSDataInputStream]) + val dfsIn = mock(classOf[DFSInputStream]) + when(mockedProvider.fs).thenReturn(mockedFs) + when(mockedFs.open(path)).thenReturn(in) + when(in.getWrappedStream).thenReturn(dfsIn) + when(dfsIn.getFileLength).thenReturn(200) + // FileStatus.getLen is more than logInfo fileSize + var fileStatus = new FileStatus(200, false, 0, 0, 0, path) + var logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), + Some("attemptId"), 100) + assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + + fileStatus = new FileStatus() + fileStatus.setPath(path) + // DFSInputStream.getFileLength is more than logInfo fileSize + logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), + Some("attemptId"), 100) + assert(mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // DFSInputStream.getFileLength is equal to logInfo fileSize + logInfo = new LogInfo(path.toString, 0, LogType.EventLogs, Some("appId"), + Some("attemptId"), 200) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // in.getWrappedStream returns other than DFSInputStream + val bin = mock(classOf[BufferedInputStream]) + when(in.getWrappedStream).thenReturn(bin) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + // fs.open throws exception + when(mockedFs.open(path)).thenThrow(new IOException("Throwing intentionally")) + assert(!mockedProvider.shouldReloadLog(logInfo, fileStatus)) + } + + test("log cleaner with the maximum number of log files") { + val clock = new ManualClock(0) + (5 to 0 by -1).foreach { num => + val log1_1 = newLogFile("app1", Some("attempt1"), inProgress = false) + writeFile(log1_1, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 1L, "test", Some("attempt1")), + SparkListenerApplicationEnd(2L) + ) + log1_1.setLastModified(2L) + + val log2_1 = newLogFile("app2", Some("attempt1"), inProgress = false) + writeFile(log2_1, true, None, + SparkListenerApplicationStart("app2", Some("app2"), 3L, "test", Some("attempt1")), + SparkListenerApplicationEnd(4L) + ) + log2_1.setLastModified(4L) + + val log3_1 = newLogFile("app3", Some("attempt1"), inProgress = false) + writeFile(log3_1, true, None, + SparkListenerApplicationStart("app3", Some("app3"), 5L, "test", Some("attempt1")), + SparkListenerApplicationEnd(6L) + ) + log3_1.setLastModified(6L) + + val log1_2_incomplete = newLogFile("app1", Some("attempt2"), inProgress = false) + writeFile(log1_2_incomplete, true, None, + SparkListenerApplicationStart("app1", Some("app1"), 7L, "test", Some("attempt2")) + ) + log1_2_incomplete.setLastModified(8L) + + val log3_2 = newLogFile("app3", Some("attempt2"), inProgress = false) + writeFile(log3_2, true, None, + SparkListenerApplicationStart("app3", Some("app3"), 9L, "test", Some("attempt2")), + SparkListenerApplicationEnd(10L) + ) + log3_2.setLastModified(10L) + + val provider = new FsHistoryProvider(createTestConf().set(MAX_LOG_NUM.key, s"$num"), clock) + updateAndCheck(provider) { list => + assert(log1_1.exists() == (num > 4)) + assert(log1_2_incomplete.exists()) // Always exists for all configurations + + assert(log2_1.exists() == (num > 3)) + + assert(log3_1.exists() == (num > 2)) + assert(log3_2.exists() == (num > 2)) + } + } } /** @@ -899,7 +1278,7 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc private def createTestConf(inMemory: Boolean = false): SparkConf = { val conf = new SparkConf() - .set(EVENT_LOG_DIR, testDir.getAbsolutePath()) + .set(HISTORY_LOG_DIR, testDir.getAbsolutePath()) .set(FAST_IN_PROGRESS_PARSING, true) if (!inMemory) { @@ -909,6 +1288,26 @@ class FsHistoryProviderSuite extends SparkFunSuite with BeforeAndAfter with Matc conf } + private def createTestExecutorInfo( + appId: String, + user: String, + executorSeqNum: Int, + includingLogFiles: Boolean = true): ExecutorInfo = { + val host = s"host$executorSeqNum" + val container = s"container$executorSeqNum" + val cluster = s"cluster$executorSeqNum" + val logUrlPrefix = s"http://$host:8888/$appId/$container/origin" + + val executorLogUrlMap = Map("stdout" -> s"$logUrlPrefix/stdout", + "stderr" -> s"$logUrlPrefix/stderr") + + val extraAttributes = if (includingLogFiles) Map("LOG_FILES" -> "stdout,stderr") else Map.empty + val executorAttributes = Map("CONTAINER_ID" -> container, "CLUSTER_ID" -> cluster, + "USER" -> user) ++ extraAttributes + + new ExecutorInfo(host, 1, executorLogUrlMap, executorAttributes) + } + private class SafeModeTestProvider(conf: SparkConf, clock: Clock) extends FsHistoryProvider(conf, clock) { diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala index de321db845a66..5903ae71ec66e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerArgumentsSuite.scala @@ -22,49 +22,34 @@ import java.nio.charset.StandardCharsets._ import com.google.common.io.Files import org.apache.spark._ -import org.apache.spark.util.Utils +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Tests._ class HistoryServerArgumentsSuite extends SparkFunSuite { private val logDir = new File("src/test/resources/spark-events") private val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) - .set("spark.history.fs.updateInterval", "1") - .set("spark.testing", "true") + .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) + .set(UPDATE_INTERVAL_S, 1L) + .set(IS_TESTING, true) test("No Arguments Parsing") { val argStrings = Array.empty[String] val hsa = new HistoryServerArguments(conf, argStrings) - assert(conf.get("spark.history.fs.logDirectory") === logDir.getAbsolutePath) - assert(conf.get("spark.history.fs.updateInterval") === "1") - assert(conf.get("spark.testing") === "true") - } - - test("Directory Arguments Parsing --dir or -d") { - val argStrings = Array("--dir", "src/test/resources/spark-events1") - val hsa = new HistoryServerArguments(conf, argStrings) - assert(conf.get("spark.history.fs.logDirectory") === "src/test/resources/spark-events1") - } - - test("Directory Param can also be set directly") { - val argStrings = Array("src/test/resources/spark-events2") - val hsa = new HistoryServerArguments(conf, argStrings) - assert(conf.get("spark.history.fs.logDirectory") === "src/test/resources/spark-events2") + assert(conf.get(HISTORY_LOG_DIR) === logDir.getAbsolutePath) + assert(conf.get(UPDATE_INTERVAL_S) === 1L) + assert(conf.get(IS_TESTING).getOrElse(false)) } test("Properties File Arguments Parsing --properties-file") { - val tmpDir = Utils.createTempDir() - val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) - try { + withTempDir { tmpDir => + val outFile = File.createTempFile("test-load-spark-properties", "test", tmpDir) Files.write("spark.test.CustomPropertyA blah\n" + "spark.test.CustomPropertyB notblah\n", outFile, UTF_8) val argStrings = Array("--properties-file", outFile.getAbsolutePath) val hsa = new HistoryServerArguments(conf, argStrings) assert(conf.get("spark.test.CustomPropertyA") === "blah") assert(conf.get("spark.test.CustomPropertyB") === "notblah") - } finally { - Utils.deleteRecursively(tmpDir) } } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala index 4b1b921582e00..f78469e132490 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerDiskManagerSuite.scala @@ -20,18 +20,19 @@ package org.apache.spark.deploy.history import java.io.File import org.mockito.AdditionalAnswers -import org.mockito.Matchers.{any, anyBoolean, anyLong, eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{anyBoolean, anyLong, eq => meq} +import org.mockito.Mockito.{doAnswer, spy} import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.History._ import org.apache.spark.status.KVUtils import org.apache.spark.util.{ManualClock, Utils} import org.apache.spark.util.kvstore.KVStore class HistoryServerDiskManagerSuite extends SparkFunSuite with BeforeAndAfter { - import config._ + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) private val MAX_USAGE = 3L diff --git a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala index 11a2db81f7c6d..c12b71a518767 100644 --- a/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/history/HistoryServerSuite.scala @@ -25,9 +25,8 @@ import javax.servlet.http.{HttpServletRequest, HttpServletRequestWrapper, HttpSe import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import scala.language.postfixOps -import com.codahale.metrics.Counter +import com.gargoylesoftware.htmlunit.BrowserVersion import com.google.common.io.{ByteStreams, Files} import org.apache.commons.io.{FileUtils, IOUtils} import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} @@ -45,7 +44,10 @@ import org.scalatest.mockito.MockitoSugar import org.scalatest.selenium.WebBrowser import org.apache.spark._ -import org.apache.spark.deploy.history.config._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.History._ +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI._ import org.apache.spark.status.api.v1.ApplicationInfo import org.apache.spark.status.api.v1.JobData import org.apache.spark.ui.SparkUI @@ -78,11 +80,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers Utils.deleteRecursively(storeDir) assert(storeDir.mkdir()) val conf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir) - .set("spark.history.fs.update.interval", "0") - .set("spark.testing", "true") + .set(HISTORY_LOG_DIR, logDir) + .set(UPDATE_INTERVAL_S.key, "0") + .set(IS_TESTING, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - .set("spark.eventLog.logStageExecutorMetrics.enabled", "true") + .set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) + .set(EVENT_LOG_PROCESS_TREE_METRICS, true) conf.setAll(extraConf) provider = new FsHistoryProvider(conf) provider.checkForLogs() @@ -131,6 +134,10 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor list json" -> "applications/local-1422981780767/executors", "executor list with executor metrics json" -> "applications/application_1506645932520_24630151/executors", + "executor list with executor process tree metrics json" -> + "applications/application_1538416563558_0014/executors", + "executor list with executor garbage collection metrics json" -> + "applications/application_1536831636016_59384/1/executors", "stage list json" -> "applications/local-1422981780767/stages", "complete stage list json" -> "applications/local-1422981780767/stages?status=complete", "failed stage list json" -> "applications/local-1422981780767/stages?status=failed", @@ -168,9 +175,11 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers "executor node blacklisting unblacklisting" -> "applications/app-20161115172038-0000/executors", "executor memory usage" -> "applications/app-20161116163331-0000/executors", - "app environment" -> "applications/app-20161116163331-0000/environment" - // Todo: enable this test when logging the even of onBlockUpdated. See: SPARK-13845 - // "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" + "app environment" -> "applications/app-20161116163331-0000/environment", + + // Enable "spark.eventLog.logBlockUpdates.enabled", to get the storage information + // in the history server. + "one rdd storage json" -> "applications/local-1422981780767/storage/rdd/0" ) // run a bunch of characterization tests -- just verify the behavior is the same as what is saved @@ -358,9 +367,8 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers contextHandler.addServlet(holder, "/") server.attachHandler(contextHandler) - implicit val webDriver: WebDriver = new HtmlUnitDriver(true) { - getWebClient.getOptions.setThrowExceptionOnScriptError(false) - } + implicit val webDriver: WebDriver = + new HtmlUnitDriver(BrowserVersion.INTERNET_EXPLORER_11, true) try { val url = s"http://localhost:$port" @@ -396,7 +404,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers */ test("security manager starts with spark.authenticate set") { val conf = new SparkConf() - .set("spark.testing", "true") + .set(IS_TESTING, true) .set(SecurityManager.SPARK_AUTH_CONF, "true") HistoryServer.createSecurityManager(conf) } @@ -413,13 +421,12 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers // allowed refresh rate (1Hz) stop() val myConf = new SparkConf() - .set("spark.history.fs.logDirectory", logDir.getAbsolutePath) - .set("spark.eventLog.dir", logDir.getAbsolutePath) - .set("spark.history.fs.update.interval", "1s") - .set("spark.eventLog.enabled", "true") - .set("spark.history.cache.window", "250ms") + .set(HISTORY_LOG_DIR, logDir.getAbsolutePath) + .set(EVENT_LOG_DIR, logDir.getAbsolutePath) + .set(UPDATE_INTERVAL_S.key, "1s") + .set(EVENT_LOG_ENABLED, true) .set(LOCAL_STORE_DIR, storeDir.getAbsolutePath()) - .remove("spark.testing") + .remove(IS_TESTING) val provider = new FsHistoryProvider(myConf) val securityManager = HistoryServer.createSecurityManager(myConf) @@ -450,16 +457,6 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers val port = server.boundPort val metrics = server.cacheMetrics - // assert that a metric has a value; if not dump the whole metrics instance - def assertMetric(name: String, counter: Counter, expected: Long): Unit = { - val actual = counter.getCount - if (actual != expected) { - // this is here because Scalatest loses stack depth - fail(s"Wrong $name value - expected $expected but got $actual" + - s" in metrics\n$metrics") - } - } - // build a URL for an app or app/attempt plus a page underneath def buildURL(appId: String, suffix: String): URL = { new URL(s"http://localhost:$port/history/$appId$suffix") @@ -470,13 +467,11 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers new URL(s"http://localhost:$port/api/v1/applications/$appId$suffix") } - val historyServerRoot = new URL(s"http://localhost:$port/") - // start initial job val d = sc.parallelize(1 to 10) d.count() - val stdInterval = interval(100 milliseconds) - val appId = eventually(timeout(20 seconds), stdInterval) { + val stdInterval = interval(100.milliseconds) + val appId = eventually(timeout(20.seconds), stdInterval) { val json = getContentAndCode("applications", port)._2.get val apps = parse(json).asInstanceOf[JArray].arr apps should have size 1 @@ -560,7 +555,7 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers d2.count() dumpLogDir("After second job") - val stdTimeout = timeout(10 seconds) + val stdTimeout = timeout(10.seconds) logDebug("waiting for UI to update") eventually(stdTimeout, stdInterval) { assert(2 === getNumJobs(""), @@ -609,9 +604,9 @@ class HistoryServerSuite extends SparkFunSuite with BeforeAndAfter with Matchers stop() init( - "spark.ui.filters" -> classOf[FakeAuthFilter].getName(), - "spark.history.ui.acls.enable" -> "true", - "spark.history.ui.admin.acls" -> admin) + UI_FILTERS.key -> classOf[FakeAuthFilter].getName(), + HISTORY_SERVER_UI_ACLS_ENABLE.key -> "true", + HISTORY_SERVER_UI_ADMIN_ACLS.key -> admin) val tests = Seq( (owner, HttpServletResponse.SC_OK), diff --git a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala index 84b3a29b58bf4..f19e99894644c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/MasterSuite.scala @@ -26,7 +26,6 @@ import scala.collection.mutable import scala.collection.mutable.{HashMap, HashSet} import scala.concurrent.duration._ import scala.io.Source -import scala.language.postfixOps import scala.reflect.ClassTag import org.json4s._ @@ -39,6 +38,10 @@ import other.supplier.{CustomPersistenceEngine, CustomRecoveryModeFactory} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy._ import org.apache.spark.deploy.DeployMessages._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Deploy._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEndpoint, RpcEndpointRef, RpcEnv} import org.apache.spark.serializer @@ -68,7 +71,7 @@ class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extend val appDesc = DeployTestUtils.createAppDesc() val drivers = mutable.HashSet[String]() override def receive: PartialFunction[Any, Unit] = { - case RegisteredWorker(masterRef, _, _) => + case RegisteredWorker(masterRef, _, _, _) => masterRef.send(WorkerLatestState(id, Nil, drivers.toSeq)) case LaunchDriver(driverId, desc) => drivers += driverId @@ -86,6 +89,17 @@ class MockWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) extend } } +class MockExecutorLaunchFailWorker(master: RpcEndpointRef, conf: SparkConf = new SparkConf) + extends MockWorker(master, conf) { + var failedCnt = 0 + override def receive: PartialFunction[Any, Unit] = { + case LaunchExecutor(_, appId, execId, _, _, _) => + failedCnt += 1 + master.send(ExecutorStateChanged(appId, execId, ExecutorState.FAILED, None, None)) + case otherMsg => super.receive(otherMsg) + } +} + class MasterSuite extends SparkFunSuite with Matchers with Eventually with PrivateMethodTester with BeforeAndAfter { @@ -101,10 +115,9 @@ class MasterSuite extends SparkFunSuite test("can use a custom recovery mode factory") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[CustomRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[CustomRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val instantiationAttempts = CustomRecoveryModeFactory.instantiationAttempts @@ -186,10 +199,9 @@ class MasterSuite extends SparkFunSuite test("master correctly recover the application") { val conf = new SparkConf(loadDefaults = false) - conf.set("spark.deploy.recoveryMode", "CUSTOM") - conf.set("spark.deploy.recoveryMode.factory", - classOf[FakeRecoveryModeFactory].getCanonicalName) - conf.set("spark.master.rest.enabled", "false") + conf.set(RECOVERY_MODE, "CUSTOM") + conf.set(RECOVERY_MODE_FACTORY, classOf[FakeRecoveryModeFactory].getCanonicalName) + conf.set(MASTER_REST_SERVER_ENABLED, false) val fakeAppInfo = makeAppInfo(1024) val fakeWorkerInfo = makeWorkerInfo(8192, 16) @@ -214,7 +226,7 @@ class MasterSuite extends SparkFunSuite master = makeMaster(conf) master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) // Wait until Master recover from checkpoint data. - eventually(timeout(5 seconds), interval(100 milliseconds)) { + eventually(timeout(5.seconds), interval(100.milliseconds)) { master.workers.size should be(1) } @@ -232,7 +244,7 @@ class MasterSuite extends SparkFunSuite fakeWorkerInfo.coresUsed should be(0) master.self.send(MasterChangeAcknowledged(fakeAppInfo.id)) - eventually(timeout(1 second), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { // Application state should be WAITING when "MasterChangeAcknowledged" event executed. fakeAppInfo.state should be(ApplicationState.WAITING) } @@ -240,7 +252,7 @@ class MasterSuite extends SparkFunSuite master.self.send( WorkerSchedulerStateResponse(fakeWorkerInfo.id, fakeExecutors, Seq(fakeDriverInfo.id))) - eventually(timeout(5 seconds), interval(100 milliseconds)) { + eventually(timeout(5.seconds), interval(100.milliseconds)) { getState(master) should be(RecoveryState.ALIVE) } @@ -265,7 +277,7 @@ class MasterSuite extends SparkFunSuite val localCluster = new LocalSparkCluster(2, 2, 512, conf) localCluster.start() try { - eventually(timeout(5 seconds), interval(100 milliseconds)) { + eventually(timeout(5.seconds), interval(100.milliseconds)) { val json = Source.fromURL(s"http://localhost:${localCluster.masterWebUIPort}/json") .getLines().mkString("\n") val JArray(workers) = (parse(json) \ "workers") @@ -286,12 +298,12 @@ class MasterSuite extends SparkFunSuite implicit val formats = org.json4s.DefaultFormats val reverseProxyUrl = "http://localhost:8080" val conf = new SparkConf() - conf.set("spark.ui.reverseProxy", "true") - conf.set("spark.ui.reverseProxyUrl", reverseProxyUrl) + conf.set(UI_REVERSE_PROXY, true) + conf.set(UI_REVERSE_PROXY_URL, reverseProxyUrl) val localCluster = new LocalSparkCluster(2, 2, 512, conf) localCluster.start() try { - eventually(timeout(5 seconds), interval(100 milliseconds)) { + eventually(timeout(5.seconds), interval(100.milliseconds)) { val json = Source.fromURL(s"http://localhost:${localCluster.masterWebUIPort}/json") .getLines().mkString("\n") val JArray(workers) = (parse(json) \ "workers") @@ -614,7 +626,7 @@ class MasterSuite extends SparkFunSuite override val rpcEnv: RpcEnv = master.rpcEnv override def receive: PartialFunction[Any, Unit] = { - case RegisteredWorker(_, _, masterAddress) => + case RegisteredWorker(_, _, masterAddress, _) => receivedMasterAddress = masterAddress } }) @@ -634,67 +646,127 @@ class MasterSuite extends SparkFunSuite } } - test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { - val conf = new SparkConf().set("spark.worker.timeout", "1") - val master = makeMaster(conf) + test("SPARK-27510: Master should avoid dead loop while launching executor failed in Worker") { + val master = makeMaster() master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) eventually(timeout(10.seconds)) { val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") } - val worker1 = new MockWorker(master.self) - worker1.rpcEnv.setupEndpoint("worker", worker1) - val worker1Reg = RegisterWorker( - worker1.id, - "localhost", - 9998, - worker1.self, - 10, - 1024, - "http://localhost:8080", - RpcAddress("localhost2", 10000)) - master.self.send(worker1Reg) - val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) - master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) - eventually(timeout(10.seconds)) { - assert(worker1.apps.nonEmpty) + var worker: MockExecutorLaunchFailWorker = null + try { + worker = new MockExecutorLaunchFailWorker(master.self) + worker.rpcEnv.setupEndpoint("worker", worker) + val workerRegMsg = RegisterWorker( + worker.id, + "localhost", + 9999, + worker.self, + 10, + 1234 * 3, + "http://localhost:8080", + master.rpcEnv.address) + master.self.send(workerRegMsg) + val driver = DeployTestUtils.createDriverDesc() + // mimic DriverClient to send RequestSubmitDriver to master + master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) + var appId: String = null + eventually(timeout(10.seconds)) { + // an app would be registered with Master once Driver set up + assert(worker.apps.nonEmpty) + appId = worker.apps.head._1 + assert(master.idToApp.contains(appId)) + } + + eventually(timeout(10.seconds)) { + // Master would continually launch executors until reach MAX_EXECUTOR_RETRIES + assert(worker.failedCnt == master.conf.get(MAX_EXECUTOR_RETRIES)) + // Master would remove the app if no executor could be launched for it + assert(!master.idToApp.contains(appId)) + } + } finally { + if (worker != null) { + worker.rpcEnv.shutdown() + } + if (master != null) { + master.rpcEnv.shutdown() + } } + } + test("SPARK-19900: there should be a corresponding driver for the app after relaunching driver") { + val conf = new SparkConf().set(WORKER_TIMEOUT, 1L) + val master = makeMaster(conf) + master.rpcEnv.setupEndpoint(Master.ENDPOINT_NAME, master) eventually(timeout(10.seconds)) { val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) - assert(masterState.workers(0).state == WorkerState.DEAD) + assert(masterState.status === RecoveryState.ALIVE, "Master is not alive") } + var worker1: MockWorker = null + var worker2: MockWorker = null + try { + worker1 = new MockWorker(master.self) + worker1.rpcEnv.setupEndpoint("worker", worker1) + val worker1Reg = RegisterWorker( + worker1.id, + "localhost", + 9998, + worker1.self, + 10, + 1024, + "http://localhost:8080", + RpcAddress("localhost2", 10000)) + master.self.send(worker1Reg) + val driver = DeployTestUtils.createDriverDesc().copy(supervise = true) + master.self.askSync[SubmitDriverResponse](RequestSubmitDriver(driver)) + + eventually(timeout(10.seconds)) { + assert(worker1.apps.nonEmpty) + } - val worker2 = new MockWorker(master.self) - worker2.rpcEnv.setupEndpoint("worker", worker2) - master.self.send(RegisterWorker( - worker2.id, - "localhost", - 9999, - worker2.self, - 10, - 1024, - "http://localhost:8081", - RpcAddress("localhost", 10001))) - eventually(timeout(10.seconds)) { - assert(worker2.apps.nonEmpty) - } + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + assert(masterState.workers(0).state == WorkerState.DEAD) + } - master.self.send(worker1Reg) - eventually(timeout(10.seconds)) { - val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + worker2 = new MockWorker(master.self) + worker2.rpcEnv.setupEndpoint("worker", worker2) + master.self.send(RegisterWorker( + worker2.id, + "localhost", + 9999, + worker2.self, + 10, + 1024, + "http://localhost:8081", + RpcAddress("localhost", 10001))) + eventually(timeout(10.seconds)) { + assert(worker2.apps.nonEmpty) + } - val worker = masterState.workers.filter(w => w.id == worker1.id) - assert(worker.length == 1) - // make sure the `DriverStateChanged` arrives at Master. - assert(worker(0).drivers.isEmpty) - assert(worker1.apps.isEmpty) - assert(worker1.drivers.isEmpty) - assert(worker2.apps.size == 1) - assert(worker2.drivers.size == 1) - assert(masterState.activeDrivers.length == 1) - assert(masterState.activeApps.length == 1) + master.self.send(worker1Reg) + eventually(timeout(10.seconds)) { + val masterState = master.self.askSync[MasterStateResponse](RequestMasterState) + + val worker = masterState.workers.filter(w => w.id == worker1.id) + assert(worker.length == 1) + // make sure the `DriverStateChanged` arrives at Master. + assert(worker(0).drivers.isEmpty) + assert(worker1.apps.isEmpty) + assert(worker1.drivers.isEmpty) + assert(worker2.apps.size == 1) + assert(worker2.drivers.size == 1) + assert(masterState.activeDrivers.length == 1) + assert(masterState.activeApps.length == 1) + } + } finally { + if (worker1 != null) { + worker1.rpcEnv.shutdown() + } + if (worker2 != null) { + worker2.rpcEnv.shutdown() + } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala index 62fe0eaedfd27..3d8a46bd02e1c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/master/PersistenceEngineSuite.scala @@ -24,6 +24,7 @@ import org.apache.commons.lang3.RandomUtils import org.apache.curator.test.TestingServer import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Deploy.ZOOKEEPER_URL import org.apache.spark.rpc.{RpcEndpoint, RpcEnv} import org.apache.spark.serializer.{JavaSerializer, Serializer} import org.apache.spark.util.Utils @@ -31,14 +32,11 @@ import org.apache.spark.util.Utils class PersistenceEngineSuite extends SparkFunSuite { test("FileSystemPersistenceEngine") { - val dir = Utils.createTempDir() - try { + withTempDir { dir => val conf = new SparkConf() testPersistenceEngine(conf, serializer => new FileSystemPersistenceEngine(dir.getAbsolutePath, serializer) ) - } finally { - Utils.deleteRecursively(dir) } } @@ -51,7 +49,7 @@ class PersistenceEngineSuite extends SparkFunSuite { val zkTestServer = new TestingServer(findFreePort(conf)) try { testPersistenceEngine(conf, serializer => { - conf.set("spark.deploy.zookeeper.url", zkTestServer.getConnectString) + conf.set(ZOOKEEPER_URL, zkTestServer.getConnectString) new ZooKeeperPersistenceEngine(conf, serializer) }) } finally { diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala index 54c168a8218f3..89b8bb4ff7d03 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/StandaloneRestSubmitSuite.scala @@ -83,6 +83,26 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { assert(submitResponse.success) } + test("create submission with multiple masters") { + val submittedDriverId = "your-driver-id" + val submitMessage = "my driver is submitted" + val masterUrl = startDummyServer(submitId = submittedDriverId, submitMessage = submitMessage) + val conf = new SparkConf(loadDefaults = false) + val RANDOM_PORT = 9000 + val allMasters = s"$masterUrl,${Utils.localHostName()}:$RANDOM_PORT" + conf.set("spark.master", allMasters) + conf.set("spark.app.name", "dreamer") + val appArgs = Array("one", "two", "six") + // main method calls this + val response = new RestSubmissionClientApp().run("app-resource", "main-class", appArgs, conf) + val submitResponse = getSubmitResponse(response) + assert(submitResponse.action === Utils.getFormattedClassName(submitResponse)) + assert(submitResponse.serverSparkVersion === SPARK_VERSION) + assert(submitResponse.message === submitMessage) + assert(submitResponse.submissionId === submittedDriverId) + assert(submitResponse.success) + } + test("create submission from main method") { val submittedDriverId = "your-driver-id" val submitMessage = "my driver is submitted" @@ -376,6 +396,18 @@ class StandaloneRestSubmitSuite extends SparkFunSuite with BeforeAndAfterEach { assert(filteredVariables == Map("SPARK_VAR" -> "1")) } + test("client does not send 'SPARK_HOME' env var by default") { + val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_HOME" -> "1") + val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) + assert(filteredVariables == Map("SPARK_VAR" -> "1")) + } + + test("client does not send 'SPARK_CONF_DIR' env var by default") { + val environmentVariables = Map("SPARK_VAR" -> "1", "SPARK_CONF_DIR" -> "1") + val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) + assert(filteredVariables == Map("SPARK_VAR" -> "1")) + } + test("client includes mesos env vars") { val environmentVariables = Map("SPARK_VAR" -> "1", "MESOS_VAR" -> "1", "OTHER_VAR" -> "1") val filteredVariables = RestSubmissionClient.filterSystemEnvironment(environmentVariables) diff --git a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala index 75c50af23c66a..03102fd8c696c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/rest/SubmitRestProtocolSuite.scala @@ -22,6 +22,7 @@ import java.lang.Boolean import org.json4s.jackson.JsonMethods._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ import org.apache.spark.util.Utils /** @@ -93,28 +94,28 @@ class SubmitRestProtocolSuite extends SparkFunSuite { message.sparkProperties = conf.getAll.toMap message.validate() // optional fields - conf.set("spark.jars", "mayonnaise.jar,ketchup.jar") - conf.set("spark.files", "fireball.png") + conf.set(JARS, Seq("mayonnaise.jar", "ketchup.jar")) + conf.set(FILES.key, "fireball.png") conf.set("spark.driver.memory", s"${Utils.DEFAULT_DRIVER_MEM_MB}m") - conf.set("spark.driver.cores", "180") + conf.set(DRIVER_CORES, 180) conf.set("spark.driver.extraJavaOptions", " -Dslices=5 -Dcolor=mostly_red") conf.set("spark.driver.extraClassPath", "food-coloring.jar") conf.set("spark.driver.extraLibraryPath", "pickle.jar") - conf.set("spark.driver.supervise", "false") + conf.set(DRIVER_SUPERVISE, false) conf.set("spark.executor.memory", "256m") - conf.set("spark.cores.max", "10000") + conf.set(CORES_MAX, 10000) message.sparkProperties = conf.getAll.toMap message.appArgs = Array("two slices", "a hint of cinnamon") message.environmentVariables = Map("PATH" -> "/dev/null") message.validate() // bad fields - var badConf = conf.clone().set("spark.driver.cores", "one hundred feet") + var badConf = conf.clone().set(DRIVER_CORES.key, "one hundred feet") message.sparkProperties = badConf.getAll.toMap intercept[SubmitRestProtocolException] { message.validate() } - badConf = conf.clone().set("spark.driver.supervise", "nope, never") + badConf = conf.clone().set(DRIVER_SUPERVISE.key, "nope, never") message.sparkProperties = badConf.getAll.toMap intercept[SubmitRestProtocolException] { message.validate() } - badConf = conf.clone().set("spark.cores.max", "two men") + badConf = conf.clone().set(CORES_MAX.key, "two men") message.sparkProperties = badConf.getAll.toMap intercept[SubmitRestProtocolException] { message.validate() } message.sparkProperties = conf.getAll.toMap @@ -126,17 +127,17 @@ class SubmitRestProtocolSuite extends SparkFunSuite { assert(newMessage.appResource === "honey-walnut-cherry.jar") assert(newMessage.mainClass === "org.apache.spark.examples.SparkPie") assert(newMessage.sparkProperties("spark.app.name") === "SparkPie") - assert(newMessage.sparkProperties("spark.jars") === "mayonnaise.jar,ketchup.jar") - assert(newMessage.sparkProperties("spark.files") === "fireball.png") + assert(newMessage.sparkProperties(JARS.key) === "mayonnaise.jar,ketchup.jar") + assert(newMessage.sparkProperties(FILES.key) === "fireball.png") assert(newMessage.sparkProperties("spark.driver.memory") === s"${Utils.DEFAULT_DRIVER_MEM_MB}m") - assert(newMessage.sparkProperties("spark.driver.cores") === "180") + assert(newMessage.sparkProperties(DRIVER_CORES.key) === "180") assert(newMessage.sparkProperties("spark.driver.extraJavaOptions") === " -Dslices=5 -Dcolor=mostly_red") assert(newMessage.sparkProperties("spark.driver.extraClassPath") === "food-coloring.jar") assert(newMessage.sparkProperties("spark.driver.extraLibraryPath") === "pickle.jar") - assert(newMessage.sparkProperties("spark.driver.supervise") === "false") + assert(newMessage.sparkProperties(DRIVER_SUPERVISE.key) === "false") assert(newMessage.sparkProperties("spark.executor.memory") === "256m") - assert(newMessage.sparkProperties("spark.cores.max") === "10000") + assert(newMessage.sparkProperties(CORES_MAX.key) === "10000") assert(newMessage.appArgs === message.appArgs) assert(newMessage.sparkProperties === message.sparkProperties) assert(newMessage.environmentVariables === message.environmentVariables) diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala index 2849a10a2c81e..70174f7ff939a 100644 --- a/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopDelegationTokenManagerSuite.scala @@ -17,158 +17,56 @@ package org.apache.spark.deploy.security -import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.security.Credentials -import org.scalatest.Matchers import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.security.HadoopDelegationTokenProvider -class HadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { - private var delegationTokenManager: HadoopDelegationTokenManager = null - private var sparkConf: SparkConf = null - private var hadoopConf: Configuration = null +private class ExceptionThrowingDelegationTokenProvider extends HadoopDelegationTokenProvider { + ExceptionThrowingDelegationTokenProvider.constructed = true + throw new IllegalArgumentException - override def beforeAll(): Unit = { - super.beforeAll() + override def serviceName: String = "throw" - sparkConf = new SparkConf() - hadoopConf = new Configuration() - } - - test("Correctly load default credential providers") { - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("bogus") should be (None) - } - - test("disable hive credential provider") { - sparkConf.set("spark.security.credentials.hive.enabled", "false") - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) - } - - test("using deprecated configurations") { - sparkConf.set("spark.yarn.security.tokens.hadoopfs.enabled", "false") - sparkConf.set("spark.yarn.security.credentials.hive.enabled", "false") - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - - delegationTokenManager.getServiceDelegationTokenProvider("hadoopfs") should be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hive") should be (None) - delegationTokenManager.getServiceDelegationTokenProvider("hbase") should not be (None) - } - - test("verify no credentials are obtained") { - delegationTokenManager = new HadoopDelegationTokenManager( - sparkConf, - hadoopConf, - hadoopFSsToAccess) - val creds = new Credentials() - - // Tokens cannot be obtained from HDFS, Hive, HBase in unit tests. - delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) - val tokens = creds.getAllTokens - tokens.size() should be (0) - } + override def delegationTokensRequired( + sparkConf: SparkConf, + hadoopConf: Configuration): Boolean = throw new IllegalArgumentException - test("obtain tokens For HiveMetastore") { - val hadoopConf = new Configuration() - hadoopConf.set("hive.metastore.kerberos.principal", "bob") - // thrift picks up on port 0 and bails out, without trying to talk to endpoint - hadoopConf.set("hive.metastore.uris", "http://localhost:0") - - val hiveCredentialProvider = new HiveDelegationTokenProvider() - val credentials = new Credentials() - hiveCredentialProvider.obtainDelegationTokens(hadoopConf, sparkConf, credentials) - - credentials.getAllTokens.size() should be (0) - } - - test("Obtain tokens For HBase") { - val hadoopConf = new Configuration() - hadoopConf.set("hbase.security.authentication", "kerberos") - - val hbaseTokenProvider = new HBaseDelegationTokenProvider() - val creds = new Credentials() - hbaseTokenProvider.obtainDelegationTokens(hadoopConf, sparkConf, creds) - - creds.getAllTokens.size should be (0) - } - - test("SPARK-23209: obtain tokens when Hive classes are not available") { - // This test needs a custom class loader to hide Hive classes which are in the classpath. - // Because the manager code loads the Hive provider directly instead of using reflection, we - // need to drive the test through the custom class loader so a new copy that cannot find - // Hive classes is loaded. - val currentLoader = Thread.currentThread().getContextClassLoader() - val noHive = new ClassLoader() { - override def loadClass(name: String, resolve: Boolean): Class[_] = { - if (name.startsWith("org.apache.hive") || name.startsWith("org.apache.hadoop.hive")) { - throw new ClassNotFoundException(name) - } + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = throw new IllegalArgumentException +} - if (name.startsWith("java") || name.startsWith("scala")) { - currentLoader.loadClass(name) - } else { - val classFileName = name.replaceAll("\\.", "/") + ".class" - val in = currentLoader.getResourceAsStream(classFileName) - if (in != null) { - val bytes = IOUtils.toByteArray(in) - defineClass(name, bytes, 0, bytes.length) - } else { - throw new ClassNotFoundException(name) - } - } - } - } +private object ExceptionThrowingDelegationTokenProvider { + var constructed = false +} - try { - Thread.currentThread().setContextClassLoader(noHive) - val test = noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$")) - test.getMethod("runTest").invoke(null) - } finally { - Thread.currentThread().setContextClassLoader(currentLoader) - } +class HadoopDelegationTokenManagerSuite extends SparkFunSuite { + private val hadoopConf = new Configuration() + + test("default configuration") { + ExceptionThrowingDelegationTokenProvider.constructed = false + val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf, null) + assert(manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) + // This checks that providers are loaded independently and they have no effect on each other + assert(ExceptionThrowingDelegationTokenProvider.constructed) + assert(!manager.isProviderLoaded("throw")) } - private[spark] def hadoopFSsToAccess(hadoopConf: Configuration): Set[FileSystem] = { - Set(FileSystem.get(hadoopConf)) + test("disable hadoopfs credential provider") { + val sparkConf = new SparkConf(false).set("spark.security.credentials.hadoopfs.enabled", "false") + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) + assert(!manager.isProviderLoaded("hadoopfs")) } -} -/** Test code for SPARK-23209 to avoid using too much reflection above. */ -private object NoHiveTest extends Matchers { - - def runTest(): Unit = { - try { - val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), - _ => Set()) - manager.getServiceDelegationTokenProvider("hive") should be (None) - } catch { - case e: Throwable => - // Throw a better exception in case the test fails, since there may be a lot of nesting. - var cause = e - while (cause.getCause() != null) { - cause = cause.getCause() - } - throw cause - } + test("using deprecated configurations") { + val sparkConf = new SparkConf(false) + .set("spark.yarn.security.tokens.hadoopfs.enabled", "false") + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) + assert(!manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) } - } diff --git a/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala new file mode 100644 index 0000000000000..0f1a9168e2542 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/security/HadoopFSDelegationTokenProviderSuite.scala @@ -0,0 +1,109 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.security + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.scalatest.Matchers + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.STAGING_DIR + +class HadoopFSDelegationTokenProviderSuite extends SparkFunSuite with Matchers { + test("hadoopFSsToAccess should return defaultFS even if not configured") { + val sparkConf = new SparkConf() + val defaultFS = "hdfs://localhost:8020" + val statingDir = "hdfs://localhost:8021" + sparkConf.set("spark.master", "yarn-client") + sparkConf.set(STAGING_DIR, statingDir) + val hadoopConf = new Configuration() + hadoopConf.set("fs.defaultFS", defaultFS) + val expected = Set( + new Path(defaultFS).getFileSystem(hadoopConf), + new Path(statingDir).getFileSystem(hadoopConf) + ) + val result = HadoopFSDelegationTokenProvider.hadoopFSsToAccess(sparkConf, hadoopConf) + result should be (expected) + } + + test("SPARK-24149: retrieve all namenodes from HDFS") { + val sparkConf = new SparkConf() + sparkConf.set("spark.master", "yarn-client") + val basicFederationConf = new Configuration() + basicFederationConf.set("fs.defaultFS", "hdfs://localhost:8020") + basicFederationConf.set("dfs.nameservices", "ns1,ns2") + basicFederationConf.set("dfs.namenode.rpc-address.ns1", "localhost:8020") + basicFederationConf.set("dfs.namenode.rpc-address.ns2", "localhost:8021") + val basicFederationExpected = Set( + new Path("hdfs://localhost:8020").getFileSystem(basicFederationConf), + new Path("hdfs://localhost:8021").getFileSystem(basicFederationConf)) + val basicFederationResult = HadoopFSDelegationTokenProvider.hadoopFSsToAccess( + sparkConf, basicFederationConf) + basicFederationResult should be (basicFederationExpected) + + // when viewfs is enabled, namespaces are handled by it, so we don't need to take care of them + val viewFsConf = new Configuration() + viewFsConf.addResource(basicFederationConf) + viewFsConf.set("fs.defaultFS", "viewfs://clusterX/") + viewFsConf.set("fs.viewfs.mounttable.clusterX.link./home", "hdfs://localhost:8020/") + val viewFsExpected = Set(new Path("viewfs://clusterX/").getFileSystem(viewFsConf)) + HadoopFSDelegationTokenProvider + .hadoopFSsToAccess(sparkConf, viewFsConf) should be (viewFsExpected) + + // invalid config should not throw NullPointerException + val invalidFederationConf = new Configuration() + invalidFederationConf.addResource(basicFederationConf) + invalidFederationConf.unset("dfs.namenode.rpc-address.ns2") + val invalidFederationExpected = Set( + new Path("hdfs://localhost:8020").getFileSystem(invalidFederationConf)) + val invalidFederationResult = HadoopFSDelegationTokenProvider.hadoopFSsToAccess( + sparkConf, invalidFederationConf) + invalidFederationResult should be (invalidFederationExpected) + + // no namespaces defined, ie. old case + val noFederationConf = new Configuration() + noFederationConf.set("fs.defaultFS", "hdfs://localhost:8020") + val noFederationExpected = Set( + new Path("hdfs://localhost:8020").getFileSystem(noFederationConf)) + val noFederationResult = HadoopFSDelegationTokenProvider.hadoopFSsToAccess(sparkConf, + noFederationConf) + noFederationResult should be (noFederationExpected) + + // federation and HA enabled + val federationAndHAConf = new Configuration() + federationAndHAConf.set("fs.defaultFS", "hdfs://clusterXHA") + federationAndHAConf.set("dfs.nameservices", "clusterXHA,clusterYHA") + federationAndHAConf.set("dfs.ha.namenodes.clusterXHA", "x-nn1,x-nn2") + federationAndHAConf.set("dfs.ha.namenodes.clusterYHA", "y-nn1,y-nn2") + federationAndHAConf.set("dfs.namenode.rpc-address.clusterXHA.x-nn1", "localhost:8020") + federationAndHAConf.set("dfs.namenode.rpc-address.clusterXHA.x-nn2", "localhost:8021") + federationAndHAConf.set("dfs.namenode.rpc-address.clusterYHA.y-nn1", "localhost:8022") + federationAndHAConf.set("dfs.namenode.rpc-address.clusterYHA.y-nn2", "localhost:8023") + federationAndHAConf.set("dfs.client.failover.proxy.provider.clusterXHA", + "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider") + federationAndHAConf.set("dfs.client.failover.proxy.provider.clusterYHA", + "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider") + + val federationAndHAExpected = Set( + new Path("hdfs://clusterXHA").getFileSystem(federationAndHAConf), + new Path("hdfs://clusterYHA").getFileSystem(federationAndHAConf)) + val federationAndHAResult = HadoopFSDelegationTokenProvider.hadoopFSsToAccess( + sparkConf, federationAndHAConf) + federationAndHAResult should be (federationAndHAExpected) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index 52956045d5985..c3b580e7ccac4 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -21,10 +21,9 @@ import java.io.File import scala.concurrent.duration._ -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} @@ -57,11 +56,9 @@ class DriverRunnerTest extends SparkFunSuite { superviseRetry: Boolean) = { val runner = createDriverRunner() runner.setSleeper(mock(classOf[Sleeper])) - doAnswer(new Answer[Int] { - def answer(invocation: InvocationOnMock): Int = { - runner.runCommandWithRetry(processBuilder, p => (), supervise = superviseRetry) - } - }).when(runner).prepareAndRunDriver() + doAnswer { (_: InvocationOnMock) => + runner.runCommandWithRetry(processBuilder, p => (), supervise = superviseRetry) + }.when(runner).prepareAndRunDriver() runner } @@ -120,11 +117,9 @@ class DriverRunnerTest extends SparkFunSuite { runner.setSleeper(sleeper) val (processBuilder, process) = createProcessBuilderAndProcess() - when(process.waitFor()).thenAnswer(new Answer[Int] { - def answer(invocation: InvocationOnMock): Int = { - runner.kill() - -1 - } + when(process.waitFor()).thenAnswer((_: InvocationOnMock) => { + runner.kill() + -1 }) runner.runCommandWithRetry(processBuilder, p => (), supervise = true) @@ -169,11 +164,9 @@ class DriverRunnerTest extends SparkFunSuite { val (processBuilder, process) = createProcessBuilderAndProcess() val runner = createTestableDriverRunner(processBuilder, superviseRetry = true) - when(process.waitFor()).thenAnswer(new Answer[Int] { - def answer(invocation: InvocationOnMock): Int = { - runner.kill() - -1 - } + when(process.waitFor()).thenAnswer((_: InvocationOnMock) => { + runner.kill() + -1 }) runner.start() diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 0240bf8aed4cd..988c65fd20fe3 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -29,8 +29,8 @@ class ExecutorRunnerTest extends SparkFunSuite { val sparkHome = sys.props.getOrElse("spark.test.home", fail("spark.test.home is not set!")) val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(appId), Map(), Seq(), Seq(), Seq()), "appUiUrl") - val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", 123, - "publicAddr", new File(sparkHome), new File("ooga"), "blah", conf, Seq("localDir"), + val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "http://", "worker321", + 123, "publicAddr", new File(sparkHome), new File("ooga"), "blah", conf, Seq("localDir"), ExecutorState.RUNNING) val builder = CommandUtils.buildProcessBuilder( appDesc.command, new SecurityManager(conf), 512, sparkHome, er.substituteVariables) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala index e3fe2b696aa1f..37e5fbcca46da 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerSuite.scala @@ -17,22 +17,28 @@ package org.apache.spark.deploy.worker +import java.io.{File, IOException} import java.util.concurrent.atomic.AtomicBoolean import java.util.function.Supplier +import scala.concurrent.duration._ + import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.{BeforeAndAfter, Matchers} +import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.{Command, ExecutorState, ExternalShuffleService} -import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged} +import org.apache.spark.deploy.DeployMessages.{DriverStateChanged, ExecutorStateChanged, WorkDirCleanup} import org.apache.spark.deploy.master.DriverState +import org.apache.spark.internal.config +import org.apache.spark.internal.config.Worker._ import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.util.Utils class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { @@ -54,7 +60,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { val securityMgr = new SecurityManager(conf) val rpcEnv = RpcEnv.create("test", "localhost", 12345, conf, securityMgr) _worker = new Worker(rpcEnv, 50000, 20, 1234 * 5, Array.fill(1)(RpcAddress("1.2.3.4", 1234)), - "Worker", "/tmp", conf, securityMgr, shuffleServiceSupplier) + "Worker", "/tmp", conf, securityMgr, None, shuffleServiceSupplier) _worker } @@ -100,7 +106,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (small number of executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 2.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -124,7 +130,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedExecutors (more executors)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedExecutors", 30.toString) + conf.set(WORKER_UI_RETAINED_EXECUTORS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { @@ -157,7 +163,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (small number of drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 2.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 2) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 5) { @@ -181,7 +187,7 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { test("test clearing of finishedDrivers (more drivers)") { val conf = new SparkConf() - conf.set("spark.worker.ui.retainedDrivers", 30.toString) + conf.set(WORKER_UI_RETAINED_DRIVERS, 30) val worker = makeWorker(conf) // initialize workers for (i <- 0 until 50) { @@ -222,15 +228,12 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { testCleanupFilesWithConfig(false) } - private def testCleanupFilesWithConfig(value: Boolean) = { - val conf = new SparkConf().set("spark.storage.cleanupFilesAfterExecutorExit", value.toString) + private def testCleanupFilesWithConfig(value: Boolean): Unit = { + val conf = new SparkConf().set(config.STORAGE_CLEANUP_FILES_AFTER_EXECUTOR_EXIT, value) val cleanupCalled = new AtomicBoolean(false) - when(shuffleService.executorRemoved(any[String], any[String])).thenAnswer(new Answer[Unit] { - override def answer(invocations: InvocationOnMock): Unit = { - cleanupCalled.set(true) - } - }) + when(shuffleService.executorRemoved(any[String], any[String])).thenAnswer( + (_: InvocationOnMock) => cleanupCalled.set(true)) val externalShuffleServiceSupplier = new Supplier[ExternalShuffleService] { override def get: ExternalShuffleService = shuffleService } @@ -243,4 +246,45 @@ class WorkerSuite extends SparkFunSuite with Matchers with BeforeAndAfter { ExecutorStateChanged("app1", 0, ExecutorState.EXITED, None, None)) assert(cleanupCalled.get() == value) } + + test("WorkDirCleanup cleans app dirs and shuffle metadata when " + + "spark.shuffle.service.db.enabled=true") { + testWorkDirCleanupAndRemoveMetadataWithConfig(true) + } + + test("WorkdDirCleanup cleans only app dirs when" + + "spark.shuffle.service.db.enabled=false") { + testWorkDirCleanupAndRemoveMetadataWithConfig(false) + } + + private def testWorkDirCleanupAndRemoveMetadataWithConfig(dbCleanupEnabled: Boolean): Unit = { + val conf = new SparkConf().set("spark.shuffle.service.db.enabled", dbCleanupEnabled.toString) + conf.set("spark.worker.cleanup.appDataTtl", "60") + conf.set("spark.shuffle.service.enabled", "true") + + val appId = "app1" + val execId = "exec1" + val cleanupCalled = new AtomicBoolean(false) + when(shuffleService.applicationRemoved(any[String])).thenAnswer( + (_: InvocationOnMock) => cleanupCalled.set(true)) + val externalShuffleServiceSupplier = new Supplier[ExternalShuffleService] { + override def get: ExternalShuffleService = shuffleService + } + val worker = makeWorker(conf, externalShuffleServiceSupplier) + val workDir = Utils.createTempDir(namePrefix = "work") + // initialize workers + worker.workDir = workDir + // Create the executor's working directory + val executorDir = new File(worker.workDir, appId + "/" + execId) + + if (!executorDir.exists && !executorDir.mkdirs()) { + throw new IOException("Failed to create directory " + executorDir) + } + executorDir.setLastModified(System.currentTimeMillis - (1000 * 120)) + worker.receive(WorkDirCleanup) + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(!executorDir.exists()) + assert(cleanupCalled.get() == dbCleanupEnabled) + } + } } diff --git a/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala new file mode 100644 index 0000000000000..693b0ee1778b1 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/CoarseGrainedExecutorBackendSuite.scala @@ -0,0 +1,304 @@ +/* + * 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.executor + +import java.io.File +import java.net.URL +import java.nio.ByteBuffer +import java.util.Properties + +import scala.collection.mutable +import scala.concurrent.duration._ + +import org.json4s.{DefaultFormats, Extraction} +import org.json4s.JsonAST.{JArray, JObject} +import org.json4s.JsonDSL._ +import org.mockito.Mockito.when +import org.scalatest.concurrent.Eventually.{eventually, timeout} +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark._ +import org.apache.spark.TestUtils._ +import org.apache.spark.resource.{ResourceAllocation, ResourceInformation} +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.scheduler.TaskDescription +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.LaunchTask +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.util.{SerializableBuffer, Utils} + +class CoarseGrainedExecutorBackendSuite extends SparkFunSuite + with LocalSparkContext with MockitoSugar { + + implicit val formats = DefaultFormats + + test("parsing no resources") { + val conf = new SparkConf + conf.set(TASK_GPU_ID.amountConf, "2") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + withTempDir { tmpDir => + val testResourceArgs: JObject = ("" -> "") + val ja = JArray(List(testResourceArgs)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + var error = intercept[SparkException] { + val parsedResources = backend.parseOrFindResources(Some(f1)) + }.getMessage() + + assert(error.contains("Error parsing resources file"), + s"Calling with no resources didn't error as expected, error: $error") + } + } + + test("parsing one resource") { + val conf = new SparkConf + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(TASK_GPU_ID.amountConf, "2") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + withTempDir { tmpDir => + val ra = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) + val ja = Extraction.decompose(Seq(ra)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + val parsedResources = backend.parseOrFindResources(Some(f1)) + + assert(parsedResources.size === 1) + assert(parsedResources.get(GPU).nonEmpty) + assert(parsedResources.get(GPU).get.name === GPU) + assert(parsedResources.get(GPU).get.addresses.deep === Array("0", "1").deep) + } + } + + test("parsing multiple resources") { + val conf = new SparkConf + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(TASK_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_FPGA_ID.amountConf, "3") + conf.set(TASK_FPGA_ID.amountConf, "3") + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend( env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + withTempDir { tmpDir => + val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) + val fpgaArgs = + ResourceAllocation(EXECUTOR_FPGA_ID, Seq("f1", "f2", "f3")) + val ja = Extraction.decompose(Seq(gpuArgs, fpgaArgs)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + val parsedResources = backend.parseOrFindResources(Some(f1)) + + assert(parsedResources.size === 2) + assert(parsedResources.get(GPU).nonEmpty) + assert(parsedResources.get(GPU).get.name === GPU) + assert(parsedResources.get(GPU).get.addresses.deep === Array("0", "1").deep) + assert(parsedResources.get(FPGA).nonEmpty) + assert(parsedResources.get(FPGA).get.name === FPGA) + assert(parsedResources.get(FPGA).get.addresses.deep === Array("f1", "f2", "f3").deep) + } + } + + test("error checking parsing resources and executor and task configs") { + val conf = new SparkConf + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(TASK_GPU_ID.amountConf, "2") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // not enough gpu's on the executor + withTempDir { tmpDir => + val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + + var error = intercept[IllegalArgumentException] { + val parsedResources = backend.parseOrFindResources(Some(f1)) + }.getMessage() + + assert(error.contains("Resource: gpu, with addresses: 0 is less than what the " + + "user requested: 2")) + } + + // missing resource on the executor + withTempDir { tmpDir => + val fpga = ResourceAllocation(EXECUTOR_FPGA_ID, Seq("0")) + val ja = Extraction.decompose(Seq(fpga)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + + var error = intercept[SparkException] { + val parsedResources = backend.parseOrFindResources(Some(f1)) + }.getMessage() + + assert(error.contains("User is expecting to use resource: gpu but didn't specify a " + + "discovery script!")) + } + } + + test("executor resource found less than required") { + val conf = new SparkConf + conf.set(EXECUTOR_GPU_ID.amountConf, "4") + conf.set(TASK_GPU_ID.amountConf, "1") + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + // executor resources < required + withTempDir { tmpDir => + val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val f1 = createTempJsonFile(tmpDir, "resources", ja) + + var error = intercept[IllegalArgumentException] { + val parsedResources = backend.parseOrFindResources(Some(f1)) + }.getMessage() + + assert(error.contains("Resource: gpu, with addresses: 0,1 is less than what the " + + "user requested: 4")) + } + } + + test("use resource discovery") { + val conf = new SparkConf + conf.set(EXECUTOR_FPGA_ID.amountConf, "3") + conf.set(TASK_FPGA_ID.amountConf, "3") + assume(!(Utils.isWindows)) + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + conf.set(EXECUTOR_FPGA_ID.discoveryScriptConf, scriptPath) + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + + val parsedResources = backend.parseOrFindResources(None) + + assert(parsedResources.size === 1) + assert(parsedResources.get(FPGA).nonEmpty) + assert(parsedResources.get(FPGA).get.name === FPGA) + assert(parsedResources.get(FPGA).get.addresses.deep === Array("f1", "f2", "f3").deep) + } + } + + test("use resource discovery and allocated file option") { + val conf = new SparkConf + conf.set(EXECUTOR_FPGA_ID.amountConf, "3") + conf.set(TASK_FPGA_ID.amountConf, "3") + assume(!(Utils.isWindows)) + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "fpgaDiscoverScript", + """{"name": "fpga","addresses":["f1", "f2", "f3"]}""") + conf.set(EXECUTOR_FPGA_ID.discoveryScriptConf, scriptPath) + + val serializer = new JavaSerializer(conf) + val env = createMockEnv(conf, serializer) + + // we don't really use this, just need it to get at the parser function + val backend = new CoarseGrainedExecutorBackend(env.rpcEnv, "driverurl", "1", "host1", + 4, Seq.empty[URL], env, None) + val gpuArgs = ResourceAllocation(EXECUTOR_GPU_ID, Seq("0", "1")) + val ja = Extraction.decompose(Seq(gpuArgs)) + val f1 = createTempJsonFile(dir, "resources", ja) + val parsedResources = backend.parseOrFindResources(Some(f1)) + + assert(parsedResources.size === 2) + assert(parsedResources.get(GPU).nonEmpty) + assert(parsedResources.get(GPU).get.name === GPU) + assert(parsedResources.get(GPU).get.addresses.deep === Array("0", "1").deep) + assert(parsedResources.get(FPGA).nonEmpty) + assert(parsedResources.get(FPGA).get.name === FPGA) + assert(parsedResources.get(FPGA).get.addresses.deep === Array("f1", "f2", "f3").deep) + } + } + + + test("track allocated resources by taskId") { + val conf = new SparkConf + val securityMgr = new SecurityManager(conf) + val serializer = new JavaSerializer(conf) + var backend: CoarseGrainedExecutorBackend = null + + try { + val rpcEnv = RpcEnv.create("1", "localhost", 0, conf, securityMgr) + val env = createMockEnv(conf, serializer, Some(rpcEnv)) + backend = new CoarseGrainedExecutorBackend(env.rpcEnv, rpcEnv.address.hostPort, "1", + "host1", 4, Seq.empty[URL], env, None) + assert(backend.taskResources.isEmpty) + + 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, + Map(GPU -> new ResourceInformation(GPU, Array("0", "1"))), data) + val serializedTaskDescription = TaskDescription.encode(taskDescription) + backend.executor = mock[Executor] + backend.rpcEnv.setupEndpoint("Executor 1", backend) + + // Launch a new task shall add an entry to `taskResources` map. + backend.self.send(LaunchTask(new SerializableBuffer(serializedTaskDescription))) + eventually(timeout(10.seconds)) { + assert(backend.taskResources.size == 1) + assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + } + + // Update the status of a running task shall not affect `taskResources` map. + backend.statusUpdate(taskId, TaskState.RUNNING, data) + assert(backend.taskResources.size == 1) + assert(backend.taskResources(taskId)(GPU).addresses sameElements Array("0", "1")) + + // Update the status of a finished task shall remove the entry from `taskResources` map. + backend.statusUpdate(taskId, TaskState.FINISHED, data) + assert(backend.taskResources.isEmpty) + } finally { + if (backend != null) { + backend.rpcEnv.shutdown() + } + } + } + + private def createMockEnv(conf: SparkConf, serializer: JavaSerializer, + rpcEnv: Option[RpcEnv] = None): SparkEnv = { + val mockEnv = mock[SparkEnv] + val mockRpcEnv = mock[RpcEnv] + when(mockEnv.conf).thenReturn(conf) + when(mockEnv.serializer).thenReturn(serializer) + when(mockEnv.closureSerializer).thenReturn(serializer) + when(mockEnv.rpcEnv).thenReturn(rpcEnv.getOrElse(mockRpcEnv)) + SparkEnv.set(mockEnv) + mockEnv + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala index 1f8a65707b2f7..8edf95763db4c 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorSuite.scala @@ -24,13 +24,13 @@ import java.util.Properties import java.util.concurrent.{ConcurrentHashMap, CountDownLatch, TimeUnit} import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.immutable import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.Map import scala.concurrent.duration._ -import scala.language.postfixOps import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{inOrder, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -41,9 +41,11 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.TaskState.TaskState import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.memory.TestMemoryManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.rdd.RDD +import org.apache.spark.resource.ResourceInformation import org.apache.spark.rpc.{RpcEndpointRef, RpcEnv, RpcTimeout} import org.apache.spark.scheduler.{FakeTask, ResultTask, Task, TaskDescription} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} @@ -169,7 +171,7 @@ class ExecutorSuite extends SparkFunSuite val conf = new SparkConf() .setMaster("local") .setAppName("executor thread test") - .set("spark.ui.enabled", "false") + .set(UI_ENABLED.key, "false") sc = new SparkContext(conf) val executorThread = sc.parallelize(Seq(1), 1).map { _ => Thread.currentThread.getClass.getName @@ -278,13 +280,10 @@ class ExecutorSuite extends SparkFunSuite val heartbeats = ArrayBuffer[Heartbeat]() val mockReceiver = mock[RpcEndpointRef] when(mockReceiver.askSync(any[Heartbeat], any[RpcTimeout])(any)) - .thenAnswer(new Answer[HeartbeatResponse] { - override def answer(invocation: InvocationOnMock): HeartbeatResponse = { - val args = invocation.getArguments() - val mock = invocation.getMock - heartbeats += args(0).asInstanceOf[Heartbeat] - HeartbeatResponse(false) - } + .thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments() + heartbeats += args(0).asInstanceOf[Heartbeat] + HeartbeatResponse(false) }) val receiverRef = executorClass.getDeclaredField("heartbeatReceiverRef") receiverRef.setAccessible(true) @@ -372,6 +371,7 @@ class ExecutorSuite extends SparkFunSuite addedFiles = Map[String, Long](), addedJars = Map[String, Long](), properties = new Properties, + resources = immutable.Map[String, ResourceInformation](), serializedTask) } @@ -467,7 +467,9 @@ class FetchFailureHidingRDD( } catch { case t: Throwable => if (throwOOM) { + // scalastyle:off throwerror throw new OutOfMemoryError("OOM while handling another exception") + // scalastyle:on throwerror } else if (interrupt) { // make sure our test is setup correctly assert(TaskContext.get().asInstanceOf[TaskContextImpl].fetchFailed.isDefined) diff --git a/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala new file mode 100644 index 0000000000000..9ed1497db5e1d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ProcfsMetricsGetterSuite.scala @@ -0,0 +1,41 @@ +/* + * 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.executor + +import org.apache.spark.SparkFunSuite + + +class ProcfsMetricsGetterSuite extends SparkFunSuite { + + val p = new ProcfsMetricsGetter(getTestResourcePath("ProcfsMetrics")) + + test("testGetProcessInfo") { + var r = ProcfsMetrics(0, 0, 0, 0, 0, 0) + r = p.addProcfsMetricsFromOneProcess(r, 26109) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + assert(r.pythonVmemTotal == 0) + assert(r.pythonRSSTotal == 0) + + r = p.addProcfsMetricsFromOneProcess(r, 22763) + assert(r.pythonVmemTotal == 360595456) + assert(r.pythonRSSTotal == 7831552) + assert(r.jvmVmemTotal == 4769947648L) + assert(r.jvmRSSTotal == 262610944) + } +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala index 817dc082b7d38..576ca1613f75e 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileInputFormatSuite.scala @@ -59,9 +59,7 @@ class WholeTextFileInputFormatSuite extends SparkFunSuite with BeforeAndAfterAll test("for small files minimum split size per node and per rack should be less than or equal to " + "maximum split size.") { - var dir : File = null; - try { - dir = Utils.createTempDir() + withTempDir { dir => logInfo(s"Local disk address is ${dir.toString}.") // Set the minsize per node and rack to be larger than the size of the input file. @@ -75,8 +73,6 @@ class WholeTextFileInputFormatSuite extends SparkFunSuite with BeforeAndAfterAll } // ensure spark job runs successfully without exceptions from the CombineFileInputFormat assert(sc.wholeTextFiles(dir.toString).count == 3) - } finally { - Utils.deleteRecursively(dir) } } } diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index ddf73d6370631..47552916adb22 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -89,52 +89,50 @@ class WholeTextFileRecordReaderSuite extends SparkFunSuite with BeforeAndAfterAl * 3) Does the contents be the same. */ test("Correctness of WholeTextFileRecordReader.") { - val dir = Utils.createTempDir() - logInfo(s"Local disk address is ${dir.toString}.") + withTempDir { dir => + logInfo(s"Local disk address is ${dir.toString}.") - WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => - createNativeFile(dir, filename, contents, false) - } + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents, false) + } - val res = sc.wholeTextFiles(dir.toString, 3).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() - assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, - "Number of files read out does not fit with the actual value.") + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") - for ((filename, contents) <- res) { - val shortName = filename.split('/').last - assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), - s"Missing file name $filename.") - assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, - s"file $filename contents can not match.") + for ((filename, contents) <- res) { + val shortName = filename.split('/').last + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } } - - Utils.deleteRecursively(dir) } test("Correctness of WholeTextFileRecordReader with GzipCodec.") { - val dir = Utils.createTempDir() - logInfo(s"Local disk address is ${dir.toString}.") + withTempDir { dir => + logInfo(s"Local disk address is ${dir.toString}.") - WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => - createNativeFile(dir, filename, contents, true) - } + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents, true) + } - val res = sc.wholeTextFiles(dir.toString, 3).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() - assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, - "Number of files read out does not fit with the actual value.") + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") - for ((filename, contents) <- res) { - val shortName = filename.split('/').last.split('.')(0) + for ((filename, contents) <- res) { + val shortName = filename.split('/').last.split('.')(0) - assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), - s"Missing file name $filename.") - assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, - s"file $filename contents can not match.") + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } } - - Utils.deleteRecursively(dir) } } diff --git a/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala new file mode 100644 index 0000000000000..250ac3dafcabc --- /dev/null +++ b/core/src/test/scala/org/apache/spark/internal/LoggingSuite.scala @@ -0,0 +1,59 @@ +/* + * 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.internal + +import org.apache.log4j.{Level, Logger} +import org.apache.log4j.spi.{Filter, LoggingEvent} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.util.Utils + +class LoggingSuite extends SparkFunSuite { + + test("spark-shell logging filter") { + val ssf = new SparkShellLoggingFilter() + val rootLogger = Logger.getRootLogger() + val originalLevel = rootLogger.getLevel() + rootLogger.setLevel(Level.INFO) + val originalThreshold = Logging.sparkShellThresholdLevel + Logging.sparkShellThresholdLevel = Level.WARN + try { + val logger = Logger.getLogger("a.b.c.D") + val logEvent = new LoggingEvent(logger.getName(), logger, Level.INFO, "Test", null) + assert(ssf.decide(logEvent) === Filter.DENY) + + // log level is less than threshold level but different from root level + val logEvent1 = new LoggingEvent(logger.getName(), logger, Level.DEBUG, "Test", null) + assert(ssf.decide(logEvent1) != Filter.DENY) + + // custom log level configured + val parentLogger = Logger.getLogger("a.b.c") + parentLogger.setLevel(Level.INFO) + assert(ssf.decide(logEvent) != Filter.DENY) + + // log level is greater than or equal to threshold level + val logger2 = Logger.getLogger("a.b.E") + val logEvent2 = new LoggingEvent(logger2.getName(), logger2, Level.INFO, "Test", null) + Utils.setLogLevel(Level.INFO) + assert(ssf.decide(logEvent2) != Filter.DENY) + } finally { + rootLogger.setLevel(originalLevel) + Logging.sparkShellThresholdLevel = originalThreshold + } + } +} diff --git a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala index 02514dc7daef4..c3bfa7ddee5bf 100644 --- a/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala +++ b/core/src/test/scala/org/apache/spark/internal/config/ConfigEntrySuite.scala @@ -70,8 +70,8 @@ class ConfigEntrySuite extends SparkFunSuite { test("conf entry: fallback") { val conf = new SparkConf() - val parentConf = ConfigBuilder(testKey("parent")).intConf.createWithDefault(1) - val confWithFallback = ConfigBuilder(testKey("fallback")).fallbackConf(parentConf) + val parentConf = ConfigBuilder(testKey("parent1")).intConf.createWithDefault(1) + val confWithFallback = ConfigBuilder(testKey("fallback1")).fallbackConf(parentConf) assert(conf.get(confWithFallback) === 1) conf.set(confWithFallback, 2) assert(conf.get(parentConf) === 1) @@ -289,6 +289,92 @@ class ConfigEntrySuite extends SparkFunSuite { assert(conf.get(iConf) === 3) } + test("conf entry: prepend with default separator") { + val conf = new SparkConf() + val prependedKey = testKey("prepended1") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend1")) + .withPrepended(prependedKey) + .stringConf + .createOptional + + conf.set(derivedConf, "1") + assert(conf.get(derivedConf) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(derivedConf) === Some("2 1")) + } + + test("conf entry: prepend with custom separator") { + val conf = new SparkConf() + val prependedKey = testKey("prepended2") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend2")) + .withPrepended(prependedKey, ",") + .stringConf + .createOptional + + conf.set(derivedConf, "1") + assert(conf.get(derivedConf) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(derivedConf) === Some("2,1")) + } + + test("conf entry: prepend with fallback") { + val conf = new SparkConf() + val prependedKey = testKey("prepended3") + val prependedConf = ConfigBuilder(prependedKey).stringConf.createOptional + val derivedConf = ConfigBuilder(testKey("prepend3")) + .withPrepended(prependedKey) + .stringConf + .createOptional + val confWithFallback = ConfigBuilder(testKey("fallback2")).fallbackConf(derivedConf) + + assert(conf.get(confWithFallback) === None) + + conf.set(derivedConf, "1") + assert(conf.get(confWithFallback) === Some("1")) + + conf.set(prependedConf, "2") + assert(conf.get(confWithFallback) === Some("2 1")) + + conf.set(confWithFallback, Some("3")) + assert(conf.get(confWithFallback) === Some("3")) + } + + test("conf entry: prepend should work only with string type") { + var i = 0 + def testPrependFail(createConf: (String, String) => Unit): Unit = { + intercept[IllegalArgumentException] { + createConf(testKey(s"prependedFail$i"), testKey(s"prependFail$i")) + }.getMessage.contains("type must be string if prepend used") + i += 1 + } + + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).intConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).longConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).doubleConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).booleanConf + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).timeConf(TimeUnit.MILLISECONDS) + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).bytesConf(ByteUnit.BYTE) + ) + testPrependFail( (prependedKey, prependKey) => + ConfigBuilder(testKey(prependKey)).withPrepended(prependedKey).regexConf + ) + } + test("onCreate") { var onCreateCalled = false ConfigBuilder(testKey("oc1")).onCreate(_ => onCreateCalled = true).intConf.createWithDefault(1) diff --git a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala index ff117b1c21cb1..083c5e696b753 100644 --- a/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala +++ b/core/src/test/scala/org/apache/spark/io/ChunkedByteBufferSuite.scala @@ -90,7 +90,7 @@ class ChunkedByteBufferSuite extends SparkFunSuite with SharedSparkContext { val empty = ByteBuffer.wrap(Array.empty[Byte]) val bytes1 = ByteBuffer.wrap(Array.tabulate(256)(_.toByte)) val bytes2 = ByteBuffer.wrap(Array.tabulate(128)(_.toByte)) - val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, bytes2)) + val chunkedByteBuffer = new ChunkedByteBuffer(Array(empty, bytes1, empty, bytes2)) assert(chunkedByteBuffer.size === bytes1.limit() + bytes2.limit()) val inputStream = chunkedByteBuffer.toInputStream(dispose = false) diff --git a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala index c88cc13654ce5..edec968d0745a 100644 --- a/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/launcher/LauncherBackendSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.launcher import java.util.concurrent.TimeUnit import scala.concurrent.duration._ -import scala.language.postfixOps import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.util.Utils class LauncherBackendSuite extends SparkFunSuite with Matchers { @@ -48,7 +48,7 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) .setConf(SparkLauncher.DRIVER_EXTRA_CLASSPATH, System.getProperty("java.class.path")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setConf(SparkLauncher.DRIVER_EXTRA_JAVA_OPTIONS, s"-Dtest.appender=console") .setMaster(master) .setAppResource(SparkLauncher.NO_RESOURCE) @@ -56,13 +56,13 @@ class LauncherBackendSuite extends SparkFunSuite with Matchers { .startApplication() try { - eventually(timeout(30 seconds), interval(100 millis)) { + eventually(timeout(30.seconds), interval(100.milliseconds)) { handle.getAppId() should not be (null) } handle.stop() - eventually(timeout(30 seconds), interval(100 millis)) { + eventually(timeout(30.seconds), interval(100.milliseconds)) { handle.getState() should be (SparkAppHandle.State.KILLED) } } finally { diff --git a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala index 85eeb5055ae03..4a8ba0a1e1041 100644 --- a/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/MemoryManagerSuite.scala @@ -23,7 +23,7 @@ import scala.collection.mutable import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration -import org.mockito.Matchers.{any, anyLong} +import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -84,11 +84,8 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft */ protected def makeBadMemoryStore(mm: MemoryManager): MemoryStore = { val ms = mock(classOf[MemoryStore], RETURNS_SMART_NULLS) - when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())).thenAnswer(new Answer[Long] { - override def answer(invocation: InvocationOnMock): Long = { - throw new RuntimeException("bad memory store!") - } - }) + when(ms.evictBlocksToFreeSpace(any(), anyLong(), any())).thenAnswer( + (_: InvocationOnMock) => throw new RuntimeException("bad memory store!")) mm.setMemoryStore(ms) ms } @@ -106,27 +103,24 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft * records the number of bytes this is called with. This variable is expected to be cleared * by the test code later through [[assertEvictBlocksToFreeSpaceCalled]]. */ - private def evictBlocksToFreeSpaceAnswer(mm: MemoryManager): Answer[Long] = { - new Answer[Long] { - override def answer(invocation: InvocationOnMock): Long = { - val args = invocation.getArguments - val numBytesToFree = args(1).asInstanceOf[Long] - assert(numBytesToFree > 0) - require(evictBlocksToFreeSpaceCalled.get() === DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED, - "bad test: evictBlocksToFreeSpace() variable was not reset") - evictBlocksToFreeSpaceCalled.set(numBytesToFree) - if (numBytesToFree <= mm.storageMemoryUsed) { - // We can evict enough blocks to fulfill the request for space - mm.releaseStorageMemory(numBytesToFree, mm.tungstenMemoryMode) - evictedBlocks += Tuple2(null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytesToFree, 0L)) - numBytesToFree - } else { - // No blocks were evicted because eviction would not free enough space. - 0L - } + private def evictBlocksToFreeSpaceAnswer(mm: MemoryManager): Answer[Long] = + (invocation: InvocationOnMock) => { + val args = invocation.getArguments + val numBytesToFree = args(1).asInstanceOf[Long] + assert(numBytesToFree > 0) + require(evictBlocksToFreeSpaceCalled.get() === DEFAULT_EVICT_BLOCKS_TO_FREE_SPACE_CALLED, + "bad test: evictBlocksToFreeSpace() variable was not reset") + evictBlocksToFreeSpaceCalled.set(numBytesToFree) + if (numBytesToFree <= mm.storageMemoryUsed) { + // We can evict enough blocks to fulfill the request for space + mm.releaseStorageMemory(numBytesToFree, mm.tungstenMemoryMode) + evictedBlocks += Tuple2(null, BlockStatus(StorageLevel.MEMORY_ONLY, numBytesToFree, 0L)) + numBytesToFree + } else { + // No blocks were evicted because eviction would not free enough space. + 0L } } - } /** * Assert that [[MemoryStore.evictBlocksToFreeSpace]] is called with the given parameters. @@ -291,7 +285,7 @@ private[memory] trait MemoryManagerSuite extends SparkFunSuite with BeforeAndAft test("off-heap execution allocations cannot exceed limit") { val memoryManager = createMemoryManager( - maxOnHeapExecutionMemory = 0L, + maxOnHeapExecutionMemory = 2L, maxOffHeapExecutionMemory = 1000L) val tMemManager = new TaskMemoryManager(memoryManager, 1) diff --git a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala deleted file mode 100644 index 0f32fe4059fbb..0000000000000 --- a/core/src/test/scala/org/apache/spark/memory/StaticMemoryManagerSuite.scala +++ /dev/null @@ -1,191 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.memory - -import org.mockito.Mockito.when - -import org.apache.spark.SparkConf -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE -import org.apache.spark.storage.TestBlockId -import org.apache.spark.storage.memory.MemoryStore - -class StaticMemoryManagerSuite extends MemoryManagerSuite { - private val conf = new SparkConf().set("spark.storage.unrollFraction", "0.4") - - /** - * Make a [[StaticMemoryManager]] and a [[MemoryStore]] with limited class dependencies. - */ - private def makeThings( - maxExecutionMem: Long, - maxStorageMem: Long): (StaticMemoryManager, MemoryStore) = { - val mm = new StaticMemoryManager( - conf, - maxOnHeapExecutionMemory = maxExecutionMem, - maxOnHeapStorageMemory = maxStorageMem, - numCores = 1) - val ms = makeMemoryStore(mm) - (mm, ms) - } - - override protected def createMemoryManager( - maxOnHeapExecutionMemory: Long, - maxOffHeapExecutionMemory: Long): StaticMemoryManager = { - new StaticMemoryManager( - conf.clone - .set("spark.memory.fraction", "1") - .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) - .set(MEMORY_OFFHEAP_SIZE.key, maxOffHeapExecutionMemory.toString), - maxOnHeapExecutionMemory = maxOnHeapExecutionMemory, - maxOnHeapStorageMemory = 0, - numCores = 1) - } - - test("basic execution memory") { - val maxExecutionMem = 1000L - val taskAttemptId = 0L - val (mm, _) = makeThings(maxExecutionMem, Long.MaxValue) - val memoryMode = MemoryMode.ON_HEAP - assert(mm.executionMemoryUsed === 0L) - assert(mm.acquireExecutionMemory(10L, taskAttemptId, memoryMode) === 10L) - assert(mm.executionMemoryUsed === 10L) - assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) - // Acquire up to the max - assert(mm.acquireExecutionMemory(1000L, taskAttemptId, memoryMode) === 890L) - assert(mm.executionMemoryUsed === maxExecutionMem) - assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 0L) - assert(mm.executionMemoryUsed === maxExecutionMem) - mm.releaseExecutionMemory(800L, taskAttemptId, memoryMode) - assert(mm.executionMemoryUsed === 200L) - // Acquire after release - assert(mm.acquireExecutionMemory(1L, taskAttemptId, memoryMode) === 1L) - assert(mm.executionMemoryUsed === 201L) - // Release beyond what was acquired - mm.releaseExecutionMemory(maxExecutionMem, taskAttemptId, memoryMode) - assert(mm.executionMemoryUsed === 0L) - } - - test("basic storage memory") { - val maxStorageMem = 1000L - val dummyBlock = TestBlockId("you can see the world you brought to live") - val (mm, ms) = makeThings(Long.MaxValue, maxStorageMem) - val memoryMode = MemoryMode.ON_HEAP - assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 10L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 10L) - - assert(mm.acquireStorageMemory(dummyBlock, 100L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 110L) - // Acquire more than the max, not granted - assert(!mm.acquireStorageMemory(dummyBlock, maxStorageMem + 1L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 110L) - // Acquire up to the max, requests after this are still granted due to LRU eviction - assert(mm.acquireStorageMemory(dummyBlock, maxStorageMem, memoryMode)) - assertEvictBlocksToFreeSpaceCalled(ms, 110L) - assert(mm.storageMemoryUsed === 1000L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) - assertEvictBlocksToFreeSpaceCalled(ms, 1L) - assert(evictedBlocks.nonEmpty) - evictedBlocks.clear() - // Note: We evicted 1 byte to put another 1-byte block in, so the storage memory used remains at - // 1000 bytes. This is different from real behavior, where the 1-byte block would have evicted - // the 1000-byte block entirely. This is set up differently so we can write finer-grained tests. - assert(mm.storageMemoryUsed === 1000L) - mm.releaseStorageMemory(800L, memoryMode) - assert(mm.storageMemoryUsed === 200L) - // Acquire after release - assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 201L) - mm.releaseAllStorageMemory() - assert(mm.storageMemoryUsed === 0L) - assert(mm.acquireStorageMemory(dummyBlock, 1L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 1L) - // Release beyond what was acquired - mm.releaseStorageMemory(100L, memoryMode) - assert(mm.storageMemoryUsed === 0L) - } - - test("execution and storage isolation") { - val maxExecutionMem = 200L - val maxStorageMem = 1000L - val taskAttemptId = 0L - val dummyBlock = TestBlockId("ain't nobody love like you do") - val (mm, ms) = makeThings(maxExecutionMem, maxStorageMem) - val memoryMode = MemoryMode.ON_HEAP - // Only execution memory should increase - assert(mm.acquireExecutionMemory(100L, taskAttemptId, memoryMode) === 100L) - assert(mm.storageMemoryUsed === 0L) - assert(mm.executionMemoryUsed === 100L) - assert(mm.acquireExecutionMemory(1000L, taskAttemptId, memoryMode) === 100L) - assert(mm.storageMemoryUsed === 0L) - assert(mm.executionMemoryUsed === 200L) - // Only storage memory should increase - assert(mm.acquireStorageMemory(dummyBlock, 50L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 50L) - assert(mm.executionMemoryUsed === 200L) - // Only execution memory should be released - mm.releaseExecutionMemory(133L, taskAttemptId, memoryMode) - assert(mm.storageMemoryUsed === 50L) - assert(mm.executionMemoryUsed === 67L) - // Only storage memory should be released - mm.releaseAllStorageMemory() - assert(mm.storageMemoryUsed === 0L) - assert(mm.executionMemoryUsed === 67L) - } - - test("unroll memory") { - val maxStorageMem = 1000L - val dummyBlock = TestBlockId("lonely water") - val (mm, ms) = makeThings(Long.MaxValue, maxStorageMem) - val memoryMode = MemoryMode.ON_HEAP - assert(mm.acquireUnrollMemory(dummyBlock, 100L, memoryMode)) - when(ms.currentUnrollMemory).thenReturn(100L) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 100L) - mm.releaseUnrollMemory(40L, memoryMode) - assert(mm.storageMemoryUsed === 60L) - when(ms.currentUnrollMemory).thenReturn(60L) - assert(mm.acquireStorageMemory(dummyBlock, 800L, memoryMode)) - assertEvictBlocksToFreeSpaceNotCalled(ms) - assert(mm.storageMemoryUsed === 860L) - // `spark.storage.unrollFraction` is 0.4, so the max unroll space is 400 bytes. - // As of this point, cache memory is 800 bytes and current unroll memory is 60 bytes. - // Requesting 240 more bytes of unroll memory will leave our total unroll memory at - // 300 bytes, still under the 400-byte limit. Therefore, all 240 bytes are granted. - assert(mm.acquireUnrollMemory(dummyBlock, 240L, memoryMode)) - assertEvictBlocksToFreeSpaceCalled(ms, 100L) // 860 + 240 - 1000 - when(ms.currentUnrollMemory).thenReturn(300L) // 60 + 240 - assert(mm.storageMemoryUsed === 1000L) - evictedBlocks.clear() - // We already have 300 bytes of unroll memory, so requesting 150 more will leave us - // above the 400-byte limit. Since there is not enough free memory, this request will - // fail even after evicting as much as we can (400 - 300 = 100 bytes). - assert(!mm.acquireUnrollMemory(dummyBlock, 150L, memoryMode)) - assertEvictBlocksToFreeSpaceCalled(ms, 100L) - assert(mm.storageMemoryUsed === 900L) - // Release beyond what was acquired - mm.releaseUnrollMemory(maxStorageMem, memoryMode) - assert(mm.storageMemoryUsed === 0L) - } - -} diff --git a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala index d56cfc183d921..0a689f81a5761 100644 --- a/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/memory/UnifiedMemoryManagerSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.PrivateMethodTester import org.apache.spark.SparkConf import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.storage.TestBlockId import org.apache.spark.storage.memory.MemoryStore @@ -42,10 +43,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes maxOnHeapExecutionMemory: Long, maxOffHeapExecutionMemory: Long): UnifiedMemoryManager = { val conf = new SparkConf() - .set("spark.memory.fraction", "1") - .set("spark.testing.memory", maxOnHeapExecutionMemory.toString) - .set(MEMORY_OFFHEAP_SIZE.key, maxOffHeapExecutionMemory.toString) - .set("spark.memory.storageFraction", storageFraction.toString) + .set(MEMORY_FRACTION, 1.0) + .set(TEST_MEMORY, maxOnHeapExecutionMemory) + .set(MEMORY_OFFHEAP_SIZE, maxOffHeapExecutionMemory) + .set(MEMORY_STORAGE_FRACTION, storageFraction) UnifiedMemoryManager(conf, numCores = 1) } @@ -218,19 +219,20 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes } test("small heap") { - val systemMemory = 1024 * 1024 - val reservedMemory = 300 * 1024 + val systemMemory = 1024L * 1024 + val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() - .set("spark.memory.fraction", memoryFraction.toString) - .set("spark.testing.memory", systemMemory.toString) - .set("spark.testing.reservedMemory", reservedMemory.toString) + .set(MEMORY_FRACTION, memoryFraction) + .set(TEST_MEMORY, systemMemory) + .set(TEST_RESERVED_MEMORY, reservedMemory) + val mm = UnifiedMemoryManager(conf, numCores = 1) val expectedMaxMemory = ((systemMemory - reservedMemory) * memoryFraction).toLong assert(mm.maxHeapMemory === expectedMaxMemory) // Try using a system memory that's too small - val conf2 = conf.clone().set("spark.testing.memory", (reservedMemory / 2).toString) + val conf2 = conf.clone().set(TEST_MEMORY, reservedMemory / 2) val exception = intercept[IllegalArgumentException] { UnifiedMemoryManager(conf2, numCores = 1) } @@ -238,17 +240,18 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes } test("insufficient executor memory") { - val systemMemory = 1024 * 1024 - val reservedMemory = 300 * 1024 + val systemMemory = 1024L * 1024 + val reservedMemory = 300L * 1024 val memoryFraction = 0.8 val conf = new SparkConf() - .set("spark.memory.fraction", memoryFraction.toString) - .set("spark.testing.memory", systemMemory.toString) - .set("spark.testing.reservedMemory", reservedMemory.toString) + .set(MEMORY_FRACTION, memoryFraction) + .set(TEST_MEMORY, systemMemory) + .set(TEST_RESERVED_MEMORY, reservedMemory) + val mm = UnifiedMemoryManager(conf, numCores = 1) // Try using an executor memory that's too small - val conf2 = conf.clone().set("spark.executor.memory", (reservedMemory / 2).toString) + val conf2 = conf.clone().set(EXECUTOR_MEMORY.key, (reservedMemory / 2).toString) val exception = intercept[IllegalArgumentException] { UnifiedMemoryManager(conf2, numCores = 1) } @@ -257,9 +260,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("execution can evict cached blocks when there are multiple active tasks (SPARK-12155)") { val conf = new SparkConf() - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "0") - .set("spark.testing.memory", "1000") + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 0.0) + .set(TEST_MEMORY, 1000L) + val mm = UnifiedMemoryManager(conf, numCores = 2) val ms = makeMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP @@ -283,9 +287,10 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("SPARK-15260: atomically resize memory pools") { val conf = new SparkConf() - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "0") - .set("spark.testing.memory", "1000") + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 0.0) + .set(TEST_MEMORY, 1000L) + val mm = UnifiedMemoryManager(conf, numCores = 2) makeBadMemoryStore(mm) val memoryMode = MemoryMode.ON_HEAP @@ -306,9 +311,9 @@ class UnifiedMemoryManagerSuite extends MemoryManagerSuite with PrivateMethodTes test("not enough free memory in the storage pool --OFF_HEAP") { val conf = new SparkConf() - .set(MEMORY_OFFHEAP_SIZE.key, "1000") - .set("spark.testing.memory", "1000") - .set(MEMORY_OFFHEAP_ENABLED.key, "true") + .set(MEMORY_OFFHEAP_SIZE, 1000L) + .set(TEST_MEMORY, 1000L) + .set(MEMORY_OFFHEAP_ENABLED, true) val taskAttemptId = 0L val mm = UnifiedMemoryManager(conf, numCores = 1) val ms = makeMemoryStore(mm) diff --git a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala index 6f4203da1d866..c7bd0c905d027 100644 --- a/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala @@ -51,13 +51,13 @@ class InputOutputMetricsSuite extends SparkFunSuite with SharedSparkContext testTempDir.mkdir() tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt") - val pw = new PrintWriter(new FileWriter(tmpFile)) - for (x <- 1 to numRecords) { - // scalastyle:off println - pw.println(RandomUtils.nextInt(0, numBuckets)) - // scalastyle:on println + Utils.tryWithResource(new PrintWriter(tmpFile)) { pw => + for (x <- 1 to numRecords) { + // scalastyle:off println + pw.println(RandomUtils.nextInt(0, numBuckets)) + // scalastyle:on println + } } - pw.close() // Path to tmpFile tmpFilePath = tmpFile.toURI.toString diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala index a85011b42bbc7..800fc1e4a3f1d 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsConfigSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.METRICS_CONF class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { var filePath: String = _ @@ -31,7 +32,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with default properties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", "dummy-file") + sparkConf.set(METRICS_CONF, "dummy-file") val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -47,7 +48,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with properties set from a file") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -110,7 +111,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { setMetricsProperty(sparkConf, "*.source.jvm.class", "org.apache.spark.SomeOtherSource") setMetricsProperty(sparkConf, "master.sink.console.period", "50") setMetricsProperty(sparkConf, "master.sink.console.unit", "seconds") - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() @@ -135,7 +136,7 @@ class MetricsConfigSuite extends SparkFunSuite with BeforeAndAfter { test("MetricsConfig with subProperties") { val sparkConf = new SparkConf(loadDefaults = false) - sparkConf.set("spark.metrics.conf", filePath) + sparkConf.set(METRICS_CONF, filePath) val conf = new MetricsConfig(sparkConf) conf.initialize() diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index a7a24114f17e2..99c9dde1cf23c 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -35,7 +35,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile - conf = new SparkConf(false).set("spark.metrics.conf", filePath) + conf = new SparkConf(false).set(METRICS_CONF, filePath) securityMgr = new SecurityManager(conf) } @@ -76,7 +76,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM conf.set("spark.app.id", appId) conf.set("spark.executor.id", executorId) - val instanceName = "driver" + val instanceName = MetricsSystemInstances.DRIVER val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) @@ -92,7 +92,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val executorId = "driver" conf.set("spark.executor.id", executorId) - val instanceName = "driver" + val instanceName = MetricsSystemInstances.DRIVER val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) @@ -108,7 +108,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val appId = "testId" conf.set("spark.app.id", appId) - val instanceName = "driver" + val instanceName = MetricsSystemInstances.DRIVER val driverMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = driverMetricsSystem.buildRegistryName(source) @@ -126,7 +126,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM conf.set("spark.app.id", appId) conf.set("spark.executor.id", executorId) - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) @@ -142,7 +142,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val executorId = "1" conf.set("spark.executor.id", executorId) - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) @@ -158,7 +158,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM val appId = "testId" conf.set("spark.app.id", appId) - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) @@ -200,7 +200,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM conf.set("spark.executor.id", executorId) conf.set(METRICS_NAMESPACE, "${spark.app.name}") - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) @@ -218,7 +218,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM conf.set("spark.executor.id", executorId) conf.set(METRICS_NAMESPACE, namespaceToResolve) - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) @@ -238,7 +238,7 @@ class MetricsSystemSuite extends SparkFunSuite with BeforeAndAfter with PrivateM conf.set("spark.app.name", appId) conf.set(METRICS_NAMESPACE, "${spark.app.name}") - val instanceName = "executor" + val instanceName = MetricsSystemInstances.EXECUTOR val executorMetricsSystem = MetricsSystem.createMetricsSystem(instanceName, conf, securityMgr) val metricName = executorMetricsSystem.buildRegistryName(source) diff --git a/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala new file mode 100644 index 0000000000000..45e6e0b4913ed --- /dev/null +++ b/core/src/test/scala/org/apache/spark/metrics/source/AccumulatorSourceSuite.scala @@ -0,0 +1,90 @@ +/* + * 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.metrics.source + +import org.mockito.ArgumentCaptor +import org.mockito.Mockito.{mock, times, verify, when} + +import org.apache.spark.{SparkContext, SparkEnv, SparkFunSuite} +import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.util.{DoubleAccumulator, LongAccumulator} + +class AccumulatorSourceSuite extends SparkFunSuite { + test("that that accumulators register against the metric system's register") { + val acc1 = new LongAccumulator() + val acc2 = new LongAccumulator() + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map("my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + LongAccumulatorSource.register(mockContext, accs) + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert (gauges.size == 2) + assert (gauges.firstKey == "my-accumulator-1") + assert (gauges.lastKey == "my-accumulator-2") + } + + test("the accumulators value property is checked when the gauge's value is requested") { + val acc1 = new LongAccumulator() + acc1.add(123) + val acc2 = new LongAccumulator() + acc2.add(456) + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map("my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + LongAccumulatorSource.register(mockContext, accs) + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert(gauges.get("my-accumulator-1").getValue() == 123) + assert(gauges.get("my-accumulator-2").getValue() == 456) + } + + test("the double accumulators value propety is checked when the gauge's value is requested") { + val acc1 = new DoubleAccumulator() + acc1.add(123.123) + val acc2 = new DoubleAccumulator() + acc2.add(456.456) + val mockContext = mock(classOf[SparkContext]) + val mockEnvironment = mock(classOf[SparkEnv]) + val mockMetricSystem = mock(classOf[MetricsSystem]) + when(mockEnvironment.metricsSystem) thenReturn (mockMetricSystem) + when(mockContext.env) thenReturn (mockEnvironment) + val accs = Map( + "my-accumulator-1" -> acc1, + "my-accumulator-2" -> acc2) + DoubleAccumulatorSource.register(mockContext, accs) + val captor = ArgumentCaptor.forClass(classOf[AccumulatorSource]) + verify(mockMetricSystem, times(1)).registerSource(captor.capture()) + val source = captor.getValue() + val gauges = source.metricRegistry.getGauges() + assert(gauges.get("my-accumulator-1").getValue() == 123.123) + assert(gauges.get("my-accumulator-2").getValue() == 456.456) + } +} diff --git a/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala new file mode 100644 index 0000000000000..d7e4b9166fa04 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/BlockTransferServiceSuite.scala @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network + +import java.io.InputStream +import java.nio.ByteBuffer + +import scala.concurrent.Future +import scala.concurrent.duration._ +import scala.reflect.ClassTag + +import org.scalatest.concurrent._ + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.network.buffer.ManagedBuffer +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.storage.{BlockId, StorageLevel} + +class BlockTransferServiceSuite extends SparkFunSuite with TimeLimits { + + implicit val defaultSignaler: Signaler = ThreadSignaler + + test("fetchBlockSync should not hang when BlockFetchingListener.onBlockFetchSuccess fails") { + // Create a mocked `BlockTransferService` to call `BlockFetchingListener.onBlockFetchSuccess` + // with a bad `ManagedBuffer` which will trigger an exception in `onBlockFetchSuccess`. + val blockTransferService = new BlockTransferService { + override def init(blockDataManager: BlockDataManager): Unit = {} + + override def close(): Unit = {} + + override def port: Int = 0 + + override def hostName: String = "localhost-unused" + + override def fetchBlocks( + host: String, + port: Int, + execId: String, + blockIds: Array[String], + listener: BlockFetchingListener, + tempFileManager: DownloadFileManager): Unit = { + // Notify BlockFetchingListener with a bad ManagedBuffer asynchronously + new Thread() { + override def run(): Unit = { + // This is a bad buffer to trigger `IllegalArgumentException` in + // `BlockFetchingListener.onBlockFetchSuccess`. The real issue we hit is + // `ByteBuffer.allocate` throws `OutOfMemoryError`, but we cannot make it happen in + // a test. Instead, we use a negative size value to make `ByteBuffer.allocate` fail, + // and this should trigger the same code path as `OutOfMemoryError`. + val badBuffer = new ManagedBuffer { + override def size(): Long = -1 + + override def nioByteBuffer(): ByteBuffer = null + + override def createInputStream(): InputStream = null + + override def retain(): ManagedBuffer = this + + override def release(): ManagedBuffer = this + + override def convertToNetty(): AnyRef = null + } + listener.onBlockFetchSuccess("block-id-unused", badBuffer) + } + }.start() + } + + override def uploadBlock( + hostname: String, + port: Int, + execId: String, + blockId: BlockId, + blockData: ManagedBuffer, + level: StorageLevel, + classTag: ClassTag[_]): Future[Unit] = { + // This method is unused in this test + throw new UnsupportedOperationException("uploadBlock") + } + } + + val e = intercept[SparkException] { + failAfter(10.seconds) { + blockTransferService.fetchBlockSync( + "localhost-unused", 0, "exec-id-unused", "block-id-unused", null) + } + } + assert(e.getCause.isInstanceOf[IllegalArgumentException]) + } +} diff --git a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala index 21138bd4a16ba..544d52d48b385 100644 --- a/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala +++ b/core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala @@ -32,6 +32,8 @@ import org.scalatest.Matchers import org.scalatest.mockito.MockitoSugar import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network import org.apache.spark.network.{BlockDataManager, BlockTransferService} import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.shuffle.BlockFetchingListener @@ -50,8 +52,8 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on same password") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") testConnection(conf, conf) match { case Success(_) => // expected @@ -61,10 +63,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security on mismatch password") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate.secret", "bad") + val conf1 = conf0.clone.set(AUTH_SECRET, "bad") testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Mismatched response") @@ -73,10 +75,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on server") { val conf0 = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "false") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, false) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => // any funny error may occur, sever will interpret SASL token as RPC @@ -85,10 +87,10 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security mismatch auth off on client") { val conf0 = new SparkConf() - .set("spark.authenticate", "false") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, false) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - val conf1 = conf0.clone.set("spark.authenticate", "true") + val conf1 = conf0.clone.set(NETWORK_AUTH_ENABLED, true) testConnection(conf0, conf1) match { case Success(_) => fail("Should have failed") case Failure(t) => t.getMessage should include ("Expected SaslMessage") @@ -97,11 +99,11 @@ class NettyBlockTransferSecuritySuite extends SparkFunSuite with MockitoSugar wi test("security with aes encryption") { val conf = new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") .set("spark.app.id", "app-id") - .set("spark.network.crypto.enabled", "true") - .set("spark.network.crypto.saslFallback", "false") + .set(Network.NETWORK_CRYPTO_ENABLED, true) + .set(Network.NETWORK_CRYPTO_SASL_FALLBACK, false) testConnection(conf, conf) match { case Success(_) => // expected case Failure(t) => fail(t) 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 78423ee68a0ec..5d67d3358a9ca 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 @@ -17,13 +17,21 @@ package org.apache.spark.network.netty +import java.io.IOException + +import scala.concurrent.{ExecutionContext, Future} +import scala.reflect.ClassTag import scala.util.Random -import org.mockito.Mockito.mock +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest._ -import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.{ExecutorDeadException, SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.network.BlockDataManager +import org.apache.spark.network.client.{TransportClient, TransportClientFactory} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcTimeout} class NettyBlockTransferServiceSuite extends SparkFunSuite @@ -77,6 +85,48 @@ class NettyBlockTransferServiceSuite verifyServicePort(expectedPort = service0.port + 1, actualPort = service1.port) } + test("SPARK-27637: test fetch block with executor dead") { + implicit val exectionContext = ExecutionContext.global + val port = 17634 + Random.nextInt(10000) + logInfo("random port for test: " + port) + + val driverEndpointRef = new RpcEndpointRef(new SparkConf()) { + override def address: RpcAddress = null + override def name: String = "test" + override def send(message: Any): Unit = {} + // This rpcEndPointRef always return false for unit test to touch ExecutorDeadException. + override def ask[T: ClassTag](message: Any, timeout: RpcTimeout): Future[T] = { + Future{false.asInstanceOf[T]} + } + } + + val clientFactory = mock(classOf[TransportClientFactory]) + val client = mock(classOf[TransportClient]) + // 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(_ => { + createClientCount += 1 + client + }) + + val listener = mock(classOf[BlockFetchingListener]) + var hitExecutorDeadException = false + when(listener.onBlockFetchFailure(any(), any(classOf[ExecutorDeadException]))) + .thenAnswer(_ => {hitExecutorDeadException = true}) + + service0 = createService(port, driverEndpointRef) + val clientFactoryField = service0.getClass.getField( + "org$apache$spark$network$netty$NettyBlockTransferService$$clientFactory") + clientFactoryField.setAccessible(true) + clientFactoryField.set(service0, clientFactory) + + service0.fetchBlocks("localhost", port, "exec1", + Array("block1"), listener, mock(classOf[DownloadFileManager])) + assert(createClientCount === 1) + assert(hitExecutorDeadException) + } + private def verifyServicePort(expectedPort: Int, actualPort: Int): Unit = { actualPort should be >= expectedPort // avoid testing equality in case of simultaneous tests @@ -85,13 +135,15 @@ class NettyBlockTransferServiceSuite actualPort should be <= (expectedPort + 100) } - private def createService(port: Int): NettyBlockTransferService = { + private def createService( + port: Int, + rpcEndpointRef: RpcEndpointRef = null): NettyBlockTransferService = { val conf = new SparkConf() .set("spark.app.id", s"test-${getClass.getName}") val securityManager = new SecurityManager(conf) val blockDataManager = mock(classOf[BlockDataManager]) val service = new NettyBlockTransferService(conf, securityManager, "localhost", "localhost", - port, 1) + port, 1, rpcEndpointRef) service.init(blockDataManager) service } diff --git a/core/src/test/scala/org/apache/spark/network/netty/SparkTransportConfSuite.scala b/core/src/test/scala/org/apache/spark/network/netty/SparkTransportConfSuite.scala new file mode 100644 index 0000000000000..d7265b6c24fe7 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/network/netty/SparkTransportConfSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.network.netty + +import org.scalatest.Matchers +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.network.util.NettyUtils + +class SparkTransportConfSuite extends SparkFunSuite with MockitoSugar { + val module = "rpc" + val serThreads = "serverThreads" + val cliThreads = "clientThreads" + + test("default value is get when neither role nor module is set") { + val numUsableCores = 4 + val conf = new SparkConf() + val sparkTransportConf = SparkTransportConf.fromSparkConf(conf, module, numUsableCores, None) + val expected = NettyUtils.defaultNumThreads(numUsableCores) + val serActual = sparkTransportConf.get(s"spark.$module.io.$serThreads", "") + val cliActual = sparkTransportConf.get(s"spark.$module.io.$cliThreads", "") + assert(serActual == expected.toString) + assert(cliActual == expected.toString) + } + + test("module value is get when role is not set") { + val numUsableCores = 3 + val serExpected = "7" + val cliExpected = "5" + val conf = new SparkConf() + .set(s"spark.$module.io.$serThreads", serExpected) + .set(s"spark.$module.io.$cliThreads", cliExpected) + val sparkTransportConf = SparkTransportConf.fromSparkConf(conf, module, numUsableCores, None) + val serActual = sparkTransportConf.get(s"spark.$module.io.$serThreads", "") + val cliActual = sparkTransportConf.get(s"spark.$module.io.$cliThreads", "") + assert(serActual == serExpected) + assert(cliActual == cliExpected) + } + + test("use correct configuration when both module and role configs are present") { + val role = Some("driver") + val numUsableCores = 10 + val serModule = "7" + val cliModule = "5" + val serExpected = "8" + val cliExpected = "6" + val conf = new SparkConf() + .set(s"spark.$module.io.$serThreads", serModule) + .set(s"spark.$module.io.$cliThreads", cliModule) + .set(s"spark.${role.get}.$module.io.$serThreads", serExpected) + .set(s"spark.${role.get}.$module.io.$cliThreads", cliExpected) + val sparkTransportConf = SparkTransportConf.fromSparkConf(conf, module, numUsableCores, role) + val serActual = sparkTransportConf.get(s"spark.$module.io.$serThreads", "") + val cliActual = sparkTransportConf.get(s"spark.$module.io.$cliThreads", "") + assert(serActual == serExpected) + assert(cliActual == cliExpected) + + val exeRole = Some("executor") + val sparkTransConfExe = SparkTransportConf.fromSparkConf(conf, module, numUsableCores, exeRole) + val serActualExe = sparkTransConfExe.get(s"spark.$module.io.$serThreads", "") + val cliActualExe = sparkTransConfExe.get(s"spark.$module.io.$cliThreads", "") + assert(serActualExe == serModule) + assert(cliActualExe == cliModule) + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 24b0144a38bd2..a7eb0eca72e56 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -142,7 +142,7 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim } assert(f.get() === 10) - failAfter(10 seconds) { + failAfter(10.seconds) { sem.acquire(2) } } @@ -178,7 +178,7 @@ class AsyncRDDActionsSuite extends SparkFunSuite with BeforeAndAfterAll with Tim f.get() } - failAfter(10 seconds) { + failAfter(10.seconds) { sem.acquire(2) } } diff --git a/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala b/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.scala new file mode 100644 index 0000000000000..42b30707f2624 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/rdd/CoalescedRDDBenchmark.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.rdd + +import scala.collection.immutable + +import org.apache.spark.SparkContext +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} + +/** + * Benchmark for CoalescedRDD. + * Measures rdd.coalesce performance under various combinations of + * coalesced partitions and preferred hosts + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/CoalescedRDD-results.txt". + * }}} + * */ +object CoalescedRDDBenchmark extends BenchmarkBase { + val seed = 0x1337 + val sc = new SparkContext(master = "local[4]", appName = "test") + + private def coalescedRDD(numIters: Int): Unit = { + val numBlocks = 100000 + val benchmark = new Benchmark("Coalesced RDD", numBlocks, output = output) + for (numPartitions <- Seq(100, 500, 1000, 5000, 10000)) { + for (numHosts <- Seq(1, 5, 10, 20, 40, 80)) { + + import collection.mutable + val hosts = mutable.ArrayBuffer[String]() + (1 to numHosts).foreach(hosts += "m" + _) + hosts.length + val rnd = scala.util.Random + rnd.setSeed(seed) + val blocks: immutable.Seq[(Int, Seq[String])] = (1 to numBlocks).map { i => + (i, hosts(rnd.nextInt(hosts.size)) :: Nil) + } + + benchmark.addCase( + s"Coalesce Num Partitions: $numPartitions Num Hosts: $numHosts", + numIters) { _ => + performCoalesce(blocks, numPartitions) + } + } + } + + benchmark.run() + } + + private def performCoalesce(blocks: immutable.Seq[(Int, Seq[String])], numPartitions: Int) { + sc.makeRDD(blocks).coalesce(numPartitions).partitions + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val numIters = 3 + runBenchmark("Coalesced RDD , large scale") { + coalescedRDD(numIters) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala index 478f0690f8e45..c942328acc8c1 100644 --- a/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/LocalCheckpointSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.rdd import scala.concurrent.duration._ -import scala.language.postfixOps import org.scalatest.concurrent.Eventually.{eventually, interval, timeout} @@ -174,7 +173,7 @@ class LocalCheckpointSuite extends SparkFunSuite with LocalSparkContext { val blockId = RDDBlockId(rdd.id, numPartitions - 1) bmm.removeBlock(blockId) // Wait until the block has been removed successfully. - eventually(timeout(1 seconds), interval(100 milliseconds)) { + eventually(timeout(1.second), interval(100.milliseconds)) { assert(bmm.getBlockStatus(blockId).isEmpty) } try { diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 47af5c3320dd9..1564435a0bbae 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -470,15 +470,12 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("zero-partition RDD") { - val emptyDir = Utils.createTempDir() - try { + withTempDir { emptyDir => val file = sc.textFile(emptyDir.getAbsolutePath) assert(file.partitions.isEmpty) assert(file.collect().toList === Nil) // Test that a shuffle on the file works, because this used to be a bug assert(file.map(line => (line, 1)).reduceByKey(_ + _).collect().toList === Nil) - } finally { - Utils.deleteRecursively(emptyDir) } } @@ -574,7 +571,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("saveNewAPIHadoopFile should call setConf if format is configurable") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1)))) // No error, non-configurable formats still work pairs.saveAsNewAPIHadoopFile[NewFakeFormat]("ignored") @@ -591,14 +588,14 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { test("The JobId on the driver and executors should be the same during the commit") { // Create more than one rdd to mimic stageId not equal to rddId val pairs = sc.parallelize(Array((1, 2), (2, 3)), 2) - .map { p => (new Integer(p._1 + 1), new Integer(p._2 + 1)) } + .map { p => (Integer.valueOf(p._1 + 1), Integer.valueOf(p._2 + 1)) } .filter { p => p._1 > 0 } pairs.saveAsNewAPIHadoopFile[YetAnotherFakeFormat]("ignored") assert(JobID.jobid != -1) } test("saveAsHadoopFile should respect configured output committers") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(1)))) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(1)))) val conf = new JobConf() conf.setOutputCommitter(classOf[FakeOutputCommitter]) @@ -610,7 +607,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("failure callbacks should be called before calling writer.close() in saveNewAPIHadoopFile") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) FakeWriterWithCallback.calledBy = "" FakeWriterWithCallback.exception = null @@ -625,7 +622,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { } test("failure callbacks should be called before calling writer.close() in saveAsHadoopFile") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val conf = new JobConf() FakeWriterWithCallback.calledBy = "" @@ -643,7 +640,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { test("saveAsNewAPIHadoopDataset should support invalid output paths when " + "there are no files to be committed to an absolute output location") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) def saveRddWithPath(path: String): Unit = { val job = NewJob.getInstance(new Configuration(sc.hadoopConfiguration)) @@ -671,7 +668,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { // for non-null invalid paths. test("saveAsHadoopDataset should respect empty output directory when " + "there are no files to be committed to an absolute output location") { - val pairs = sc.parallelize(Array((new Integer(1), new Integer(2))), 1) + val pairs = sc.parallelize(Array((Integer.valueOf(1), Integer.valueOf(2))), 1) val conf = new JobConf() conf.setOutputKeyClass(classOf[Integer]) @@ -742,7 +739,7 @@ class PairRDDFunctionsSuite extends SparkFunSuite with SharedSparkContext { val dist = new BinomialDistribution(trials, p) val q = dist.cumulativeProbability(actual) withClue(s"p = $p: trials = $trials") { - assert(q >= 0.001 && q <= 0.999) + assert(0.0 < q && q < 1.0) } } } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index 31ce9483cf20a..424d9f825c465 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -215,7 +215,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { } test("exclusive ranges of doubles") { - val data = 1.0 until 100.0 by 1.0 + val data = Range.BigDecimal(1, 100, 1) val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).sum === 99) @@ -223,7 +223,7 @@ class ParallelCollectionSplitSuite extends SparkFunSuite with Checkers { } test("inclusive ranges of doubles") { - val data = 1.0 to 100.0 by 1.0 + val data = Range.BigDecimal.inclusive(1, 100, 1) val slices = ParallelCollectionRDD.slice(data, 3) assert(slices.size === 3) assert(slices.map(_.size).sum === 100) diff --git a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala index 1a0eb250e7cdc..69739a2e58481 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PipedRDDSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.rdd import java.io.File +import scala.collection.JavaConverters._ import scala.collection.Map import scala.io.Codec @@ -83,6 +84,29 @@ class PipedRDDSuite extends SparkFunSuite with SharedSparkContext { } } + test("stdin writer thread should be exited when task is finished") { + assume(TestUtils.testCommandAvailable("cat")) + val nums = sc.makeRDD(Array(1, 2, 3, 4), 1).map { x => + val obj = new Object() + obj.synchronized { + obj.wait() // make the thread waits here. + } + x + } + + val piped = nums.pipe(Seq("cat")) + + val result = piped.mapPartitions(_ => Array.emptyIntArray.iterator) + + assert(result.collect().length === 0) + + // collect stderr writer threads + val stderrWriterThread = Thread.getAllStackTraces.keySet().asScala + .find { _.getName.startsWith(PipedRDD.STDIN_WRITER_THREAD_PREFIX) } + + assert(stderrWriterThread.isEmpty) + } + test("advanced pipe") { assume(TestUtils.testCommandAvailable("cat")) val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala index d57ea4d5501e3..2f6c4d6a42ea3 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDBarrierSuite.scala @@ -26,13 +26,13 @@ class RDDBarrierSuite extends SparkFunSuite with SharedSparkContext { assert(rdd.isBarrier() === false) val rdd2 = rdd.barrier().mapPartitions(iter => iter) - assert(rdd2.isBarrier() === true) + assert(rdd2.isBarrier()) } test("create an RDDBarrier in the middle of a chain of RDDs") { val rdd = sc.parallelize(1 to 10, 4).map(x => x * 2) val rdd2 = rdd.barrier().mapPartitions(iter => iter).map(x => (x, x + 1)) - assert(rdd2.isBarrier() === true) + assert(rdd2.isBarrier()) } test("RDDBarrier with shuffle") { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 2227698cf1ad2..60e63bfd68625 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -18,21 +18,25 @@ package org.apache.spark.rdd import java.io.{File, IOException, ObjectInputStream, ObjectOutputStream} +import java.lang.management.ManagementFactory import scala.collection.JavaConverters._ import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.concurrent.duration._ import scala.reflect.ClassTag import com.esotericsoftware.kryo.KryoException import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapred.{FileSplit, TextInputFormat} +import org.scalatest.concurrent.Eventually import org.apache.spark._ import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.rdd.RDDSuiteUtils._ import org.apache.spark.util.{ThreadUtils, Utils} -class RDDSuite extends SparkFunSuite with SharedSparkContext { +class RDDSuite extends SparkFunSuite with SharedSparkContext with Eventually { var tempDir: File = _ override def beforeAll(): Unit = { @@ -136,12 +140,12 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { assert(serialUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === false) - sc.conf.set("spark.rdd.parallelListingThreshold", "1") + sc.conf.set(RDD_PARALLEL_LISTING_THRESHOLD, 1) val parallelUnion = sc.union(nums1, nums2) val actual = parallelUnion.collect().toList - sc.conf.remove("spark.rdd.parallelListingThreshold") + sc.conf.remove(RDD_PARALLEL_LISTING_THRESHOLD.key) - assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel === true) + assert(parallelUnion.asInstanceOf[UnionRDD[Int]].isPartitionListingParallel) assert(expected === actual) } @@ -444,7 +448,7 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { val splits = coalesced1.glom().collect().map(_.toList).toList assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) - assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + assert(splits.forall(_.length >= 1), "Some partitions were empty") // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. @@ -471,7 +475,7 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { val splits = coalesced1.glom().collect().map(_.toList).toList assert(splits.length === 3, "Supposed to coalesce to 3 but got " + splits.length) - assert(splits.forall(_.length >= 1) === true, "Some partitions were empty") + assert(splits.forall(_.length >= 1), "Some partitions were empty") // If we try to coalesce into more partitions than the original RDD, it should just // keep the original number of partitions. @@ -1175,6 +1179,31 @@ class RDDSuite extends SparkFunSuite with SharedSparkContext { }.collect() } + test("SPARK-27666: Do not release lock while TaskContext already completed") { + val rdd = sc.parallelize(Range(0, 10), 1).cache() + val tid = sc.longAccumulator("threadId") + // validate cache + rdd.collect() + rdd.mapPartitions { iter => + val t = new Thread(() => { + while (iter.hasNext) { + iter.next() + Thread.sleep(100) + } + }) + t.setDaemon(false) + t.start() + tid.add(t.getId) + Iterator(0) + }.collect() + val tmx = ManagementFactory.getThreadMXBean + eventually(timeout(10.seconds)) { + // getThreadInfo() will return null after child thread `t` died + val t = tmx.getThreadInfo(tid.value) + assert(t == null || t.getThreadState == Thread.State.TERMINATED) + } + } + test("SPARK-23496: order of input partitions can result in severe skew in coalesce") { val numInputPartitions = 100 val numCoalescedPartitions = 50 diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceInformationSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceInformationSuite.scala new file mode 100644 index 0000000000000..f5044c22656f2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceInformationSuite.scala @@ -0,0 +1,64 @@ +/* + * 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.resource + +import org.json4s.JsonDSL._ +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{SparkException, SparkFunSuite} + +class ResourceInformationSuite extends SparkFunSuite { + test("ResourceInformation.parseJson for valid JSON") { + val json1 = compact(render(("name" -> "p100") ~ ("addresses" -> Seq("0", "1")))) + val info1 = ResourceInformation.parseJson(json1) + assert(info1.name === "p100") + assert(info1.addresses === Array("0", "1")) + + // Currently we allow empty addresses. + val json2 = compact(render("name" -> "fpga")) + val info2 = ResourceInformation.parseJson(json2) + assert(info2.name === "fpga") + assert(info2.addresses.isEmpty) + + val json3 = compact(render("addresses" -> Seq("0"))) + val json4 = "invalid_json" + for (invalidJson <- Seq(json3, json4)) { + val ex = intercept[SparkException] { + print(ResourceInformation.parseJson(invalidJson)) + } + assert(ex.getMessage.contains("Error parsing JSON into ResourceInformation"), + "Error message should provide context.") + assert(ex.getMessage.contains(invalidJson), "Error message should include input json.") + } + } + + test("ResourceInformation.equals/hashCode") { + val a1 = new ResourceInformation("a", addresses = Array("0")) + val a21 = new ResourceInformation("a", addresses = Array("0", "1")) + val a22 = new ResourceInformation("a", addresses = Array("0", "1")) + val b2 = new ResourceInformation("b", addresses = Array("0", "1")) + object A2 extends ResourceInformation("a", Array("0", "1")) + assert(a1.equals(null) === false) + assert(a1.equals(a1)) + assert(a1.equals(a21) === false) + assert(a21.equals(a22) && a21.hashCode() === a22.hashCode()) + assert(a21.equals(b2) === false) + assert(a21.equals(A2) === false) + } +} diff --git a/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala new file mode 100644 index 0000000000000..51a92e0a50f2f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/ResourceUtilsSuite.scala @@ -0,0 +1,259 @@ +/* + * 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.resource + +import java.io.File +import java.nio.file.{Files => JavaFiles} + +import org.json4s.{DefaultFormats, Extraction} + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.TestUtils._ +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.util.Utils + +class ResourceUtilsSuite extends SparkFunSuite + with LocalSparkContext { + + test("ResourceID") { + val componentName = "spark.test" + val resourceName = "p100" + val id = ResourceID(componentName, resourceName) + val confPrefix = s"$componentName.resource.$resourceName." + assert(id.confPrefix === confPrefix) + assert(id.amountConf === s"${confPrefix}amount") + assert(id.discoveryScriptConf === s"${confPrefix}discoveryScript") + assert(id.vendorConf === s"${confPrefix}vendor") + } + + test("Resource discoverer no addresses errors") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val scriptPath = createTempScriptWithExpectedOutput(dir, "gpuDiscoverScript", + """{"name": "gpu"}""") + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, scriptPath) + + val error = intercept[IllegalArgumentException] { + getOrDiscoverAllResources(conf, SPARK_EXECUTOR_PREFIX, None) + }.getMessage() + assert(error.contains("Resource: gpu, with " + + "addresses: is less than what the user requested: 2")) + } + } + + test("Resource discoverer multiple resource types") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "gpu", "addresses": ["0", "1"]}""") + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, gpuDiscovery) + + val fpgaDiscovery = createTempScriptWithExpectedOutput(dir, "fpgDiscoverScript", + """{"name": "fpga", "addresses": ["f1", "f2", "f3"]}""") + conf.set(EXECUTOR_FPGA_ID.amountConf, "2") + conf.set(EXECUTOR_FPGA_ID.discoveryScriptConf, fpgaDiscovery) + + val resources = getOrDiscoverAllResources(conf, SPARK_EXECUTOR_PREFIX, None) + assert(resources.size === 2) + val gpuValue = resources.get(GPU) + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") + assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") + + val fpgaValue = resources.get(FPGA) + assert(fpgaValue.nonEmpty, "Should have a gpu entry") + assert(fpgaValue.get.name == "fpga", "name should be fpga") + assert(fpgaValue.get.addresses.size == 3, "Should have 3 indexes") + assert(fpgaValue.get.addresses.deep == Array("f1", "f2", "f3").deep, + "should have f1,f2,f3 entries") + } + } + + test("get from resources file and discover the remaining") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + implicit val formats = DefaultFormats + val fpgaAddrs = Seq("f1", "f2", "f3") + val fpgaAllocation = ResourceAllocation(EXECUTOR_FPGA_ID, fpgaAddrs) + val resourcesFile = createTempJsonFile( + dir, "resources", Extraction.decompose(Seq(fpgaAllocation))) + conf.set(EXECUTOR_FPGA_ID.amountConf, "3") + val resourcesFromFileOnly = getOrDiscoverAllResources( + conf, SPARK_EXECUTOR_PREFIX, Some(resourcesFile)) + val expectedFpgaInfo = new ResourceInformation(FPGA, fpgaAddrs.toArray) + assert(resourcesFromFileOnly(FPGA) === expectedFpgaInfo) + + val gpuDiscovery = createTempScriptWithExpectedOutput( + dir, "gpuDiscoveryScript", """{"name": "gpu", "addresses": ["0", "1"]}""") + conf.set(EXECUTOR_GPU_ID.amountConf, "2") + conf.set(EXECUTOR_GPU_ID.discoveryScriptConf, gpuDiscovery) + val resourcesFromBoth = getOrDiscoverAllResources( + conf, SPARK_EXECUTOR_PREFIX, Some(resourcesFile)) + val expectedGpuInfo = new ResourceInformation(GPU, Array("0", "1")) + assert(resourcesFromBoth(FPGA) === expectedFpgaInfo) + assert(resourcesFromBoth(GPU) === expectedGpuInfo) + } + } + + test("list resource ids") { + val conf = new SparkConf + conf.set(DRIVER_GPU_ID.amountConf, "2") + var resources = listResourceIds(conf, SPARK_DRIVER_PREFIX) + assert(resources.size === 1, "should only have GPU for resource") + assert(resources(0).resourceName == GPU, "name should be gpu") + + conf.set(DRIVER_FPGA_ID.amountConf, "2") + val resourcesMap = listResourceIds(conf, SPARK_DRIVER_PREFIX) + .map{ rId => (rId.resourceName, 1)}.toMap + assert(resourcesMap.size === 2, "should only have GPU for resource") + assert(resourcesMap.get(GPU).nonEmpty, "should have GPU") + assert(resourcesMap.get(FPGA).nonEmpty, "should have FPGA") + } + + test("parse resource request") { + val conf = new SparkConf + conf.set(DRIVER_GPU_ID.amountConf, "2") + var request = parseResourceRequest(conf, DRIVER_GPU_ID) + assert(request.id.resourceName === GPU, "should only have GPU for resource") + assert(request.amount === 2, "GPU count should be 2") + assert(request.discoveryScript === None, "discovery script should be empty") + assert(request.vendor === None, "vendor should be empty") + + val vendor = "nvidia.com" + val discoveryScript = "discoveryScriptGPU" + conf.set(DRIVER_GPU_ID.discoveryScriptConf, discoveryScript) + conf.set(DRIVER_GPU_ID.vendorConf, vendor) + request = parseResourceRequest(conf, DRIVER_GPU_ID) + assert(request.id.resourceName === GPU, "should only have GPU for resource") + assert(request.amount === 2, "GPU count should be 2") + assert(request.discoveryScript.get === discoveryScript, "should get discovery script") + assert(request.vendor.get === vendor, "should get vendor") + + conf.remove(DRIVER_GPU_ID.amountConf) + val error = intercept[SparkException] { + request = parseResourceRequest(conf, DRIVER_GPU_ID) + }.getMessage() + + assert(error.contains("You must specify an amount for gpu")) + } + + test("Resource discoverer multiple gpus on driver") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDisocveryScript", + """{"name": "gpu", "addresses": ["0", "1"]}""") + conf.set(DRIVER_GPU_ID.amountConf, "2") + conf.set(DRIVER_GPU_ID.discoveryScriptConf, gpuDiscovery) + + // make sure it reads from correct config, here it should use driver + val resources = getOrDiscoverAllResources(conf, SPARK_DRIVER_PREFIX, None) + val gpuValue = resources.get(GPU) + assert(gpuValue.nonEmpty, "Should have a gpu entry") + assert(gpuValue.get.name == "gpu", "name should be gpu") + assert(gpuValue.get.addresses.size == 2, "Should have 2 indexes") + assert(gpuValue.get.addresses.deep == Array("0", "1").deep, "should have 0,1 entries") + } + } + + test("Resource discoverer script returns mismatched name") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDiscoveryScript", + """{"name": "fpga", "addresses": ["0", "1"]}""") + val request = + ResourceRequest( + DRIVER_GPU_ID, + 2, + Some(gpuDiscovery), + None) + + val error = intercept[SparkException] { + discoverResource(request) + }.getMessage() + + assert(error.contains(s"Error running the resource discovery script $gpuDiscovery: " + + "script returned resource name fpga and we were expecting gpu")) + } + } + + test("Resource discoverer script returns invalid format") { + val conf = new SparkConf + assume(!(Utils.isWindows)) + withTempDir { dir => + val gpuDiscovery = createTempScriptWithExpectedOutput(dir, "gpuDiscoverScript", + """{"addresses": ["0", "1"]}""") + + val request = + ResourceRequest( + EXECUTOR_GPU_ID, + 2, + Some(gpuDiscovery), + None) + + val error = intercept[SparkException] { + discoverResource(request) + }.getMessage() + + assert(error.contains("Error parsing JSON into ResourceInformation")) + } + } + + test("Resource discoverer script doesn't exist") { + val conf = new SparkConf + withTempDir { dir => + val file1 = new File(dir, "bogusfilepath") + try { + val request = + ResourceRequest( + EXECUTOR_GPU_ID, + 2, + Some(file1.getPath()), + None) + + val error = intercept[SparkException] { + discoverResource(request) + }.getMessage() + + assert(error.contains("doesn't exist")) + } finally { + JavaFiles.deleteIfExists(file1.toPath()) + } + } + } + + test("gpu's specified but not a discovery script") { + val request = ResourceRequest(EXECUTOR_GPU_ID, 2, None, None) + + val error = intercept[SparkException] { + discoverResource(request) + }.getMessage() + + assert(error.contains("User is expecting to use resource: gpu but " + + "didn't specify a discovery script!")) + } +} diff --git a/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala new file mode 100644 index 0000000000000..6d2c07d89f5b6 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/resource/TestResourceIDs.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.resource + +import org.apache.spark.internal.config.{SPARK_DRIVER_PREFIX, SPARK_EXECUTOR_PREFIX, SPARK_TASK_PREFIX} +import org.apache.spark.resource.ResourceUtils.{FPGA, GPU} + +object TestResourceIDs { + val DRIVER_GPU_ID = ResourceID(SPARK_DRIVER_PREFIX, GPU) + val EXECUTOR_GPU_ID = ResourceID(SPARK_EXECUTOR_PREFIX, GPU) + val TASK_GPU_ID = ResourceID(SPARK_TASK_PREFIX, GPU) + + val DRIVER_FPGA_ID = ResourceID(SPARK_DRIVER_PREFIX, FPGA) + val EXECUTOR_FPGA_ID = ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) + val TASK_FPGA_ID = ResourceID(SPARK_TASK_PREFIX, FPGA) +} diff --git a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala index a799b1cfb0765..99b4e8fe8280c 100644 --- a/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/RpcEnvSuite.scala @@ -26,16 +26,17 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Await import scala.concurrent.duration._ -import scala.language.postfixOps import com.google.common.io.Files -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{mock, never, verify, when} import org.scalatest.BeforeAndAfterAll import org.scalatest.concurrent.Eventually._ import org.apache.spark.{SecurityManager, SparkConf, SparkEnv, SparkException, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network import org.apache.spark.util.{ThreadUtils, Utils} /** @@ -78,7 +79,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) rpcEndpointRef.send("hello") - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert("hello" === message) } } @@ -99,7 +100,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "send-remotely") try { rpcEndpointRef.send("hello") - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert("hello" === message) } } finally { @@ -171,14 +172,14 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val conf = new SparkConf() val shortProp = "spark.rpc.short.timeout" - conf.set("spark.rpc.retry.wait", "0") - conf.set("spark.rpc.numRetries", "1") + conf.set(Network.RPC_RETRY_WAIT, 0L) + conf.set(Network.RPC_NUM_RETRIES, 1) val anotherEnv = createRpcEnv(conf, "remote", 0, clientMode = true) // Use anotherEnv to find out the RpcEndpointRef val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "ask-timeout") try { val e = intercept[RpcTimeoutException] { - rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1 millis, shortProp)) + rpcEndpointRef.askSync[String]("hello", new RpcTimeout(1.millisecond, shortProp)) } // The SparkException cause should be a RpcTimeoutException with message indicating the // controlling timeout property @@ -234,7 +235,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert(e.getMessage === "Oops!") } } @@ -259,7 +260,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { env.stop(endpointRef) - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert(e.getMessage === "Oops!") } } @@ -280,7 +281,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { endpointRef.send("Foo") - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert(e.getMessage === "Oops!") } } @@ -301,9 +302,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { // Calling `self` in `onStart` is fine - assert(callSelfSuccessfully === true) + assert(callSelfSuccessfully) } } @@ -322,9 +323,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { endpointRef.send("Foo") - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { // Calling `self` in `receive` is fine - assert(callSelfSuccessfully === true) + assert(callSelfSuccessfully) } } @@ -345,9 +346,9 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { env.stop(endpointRef) - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { // Calling `self` in `onStop` will return null, so selfOption will be None - assert(selfOption == None) + assert(selfOption.isEmpty) } } @@ -374,7 +375,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { }.start() } - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { assert(result == 1000) } @@ -399,7 +400,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { env.stop(endpointRef) env.stop(endpointRef) - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { // Calling stop twice should only trigger onStop once. assert(onStopCount == 1) } @@ -415,7 +416,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { }) val f = endpointRef.ask[String]("Hi") - val ack = ThreadUtils.awaitResult(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5.seconds) assert("ack" === ack) env.stop(endpointRef) @@ -435,7 +436,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val rpcEndpointRef = anotherEnv.setupEndpointRef(env.address, "sendWithReply-remotely") try { val f = rpcEndpointRef.ask[String]("hello") - val ack = ThreadUtils.awaitResult(f, 5 seconds) + val ack = ThreadUtils.awaitResult(f, 5.seconds) assert("ack" === ack) } finally { anotherEnv.shutdown() @@ -454,7 +455,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { val f = endpointRef.ask[String]("Hi") val e = intercept[SparkException] { - ThreadUtils.awaitResult(f, 5 seconds) + ThreadUtils.awaitResult(f, 5.seconds) } assert("Oops" === e.getCause.getMessage) @@ -476,7 +477,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { try { val f = rpcEndpointRef.ask[String]("hello") val e = intercept[SparkException] { - ThreadUtils.awaitResult(f, 5 seconds) + ThreadUtils.awaitResult(f, 5.seconds) } assert("Oops" === e.getCause.getMessage) } finally { @@ -528,14 +529,14 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // Send a message to set up the connection serverRefInServer2.send("hello") - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { assert(events.contains(("onConnected", serverEnv2.address))) } serverEnv2.shutdown() serverEnv2.awaitTermination() - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { assert(events.contains(("onConnected", serverEnv2.address))) assert(events.contains(("onDisconnected", serverEnv2.address))) } @@ -556,7 +557,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // Send a message to set up the connection serverRefInClient.send("hello") - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { // We don't know the exact client address but at least we can verify the message type assert(events.asScala.map(_._1).exists(_ == "onConnected")) } @@ -564,7 +565,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { clientEnv.shutdown() clientEnv.awaitTermination() - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { // We don't know the exact client address but at least we can verify the message type assert(events.asScala.map(_._1).exists(_ == "onConnected")) assert(events.asScala.map(_._1).exists(_ == "onDisconnected")) @@ -587,14 +588,14 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // Send a message to set up the connection serverRefInClient.send("hello") - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { assert(events.contains(("onConnected", serverEnv.address))) } serverEnv.shutdown() serverEnv.awaitTermination() - eventually(timeout(5 seconds), interval(5 millis)) { + eventually(timeout(5.seconds), interval(5.milliseconds)) { assert(events.contains(("onConnected", serverEnv.address))) assert(events.contains(("onDisconnected", serverEnv.address))) } @@ -622,7 +623,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { try { val f = rpcEndpointRef.ask[String]("hello") val e = intercept[SparkException] { - ThreadUtils.awaitResult(f, 1 seconds) + ThreadUtils.awaitResult(f, 1.second) } assert(e.getCause.isInstanceOf[NotSerializableException]) } finally { @@ -656,7 +657,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { }) val rpcEndpointRef = remoteEnv.setupEndpointRef(localEnv.address, "send-authentication") rpcEndpointRef.send("hello") - eventually(timeout(5 seconds), interval(10 millis)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { assert("hello" === message) } } finally { @@ -693,44 +694,44 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { test("send with authentication") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("send with SASL encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("send with AES encryption") { testSend(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.network.crypto.enabled", "true") - .set("spark.network.crypto.saslFallback", "false")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(Network.NETWORK_CRYPTO_ENABLED, true) + .set(Network.NETWORK_CRYPTO_SASL_FALLBACK, false)) } test("ask with authentication") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good")) } test("ask with SASL encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.authenticate.enableSaslEncryption", "true")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(SASL_ENCRYPTION_ENABLED, true)) } test("ask with AES encryption") { testAsk(new SparkConf() - .set("spark.authenticate", "true") - .set("spark.authenticate.secret", "good") - .set("spark.network.crypto.enabled", "true") - .set("spark.network.crypto.saslFallback", "false")) + .set(NETWORK_AUTH_ENABLED, true) + .set(AUTH_SECRET, "good") + .set(Network.NETWORK_CRYPTO_ENABLED, true) + .set(Network.NETWORK_CRYPTO_SASL_FALLBACK, false)) } test("construct RpcTimeout with conf property") { @@ -776,8 +777,8 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } }) - val longTimeout = new RpcTimeout(1 second, "spark.rpc.long.timeout") - val shortTimeout = new RpcTimeout(10 millis, "spark.rpc.short.timeout") + val longTimeout = new RpcTimeout(1.second, "spark.rpc.long.timeout") + val shortTimeout = new RpcTimeout(10.milliseconds, "spark.rpc.short.timeout") // Ask with immediate response, should complete successfully val fut1 = rpcEndpointRef.ask[String]("hello", longTimeout) @@ -802,7 +803,7 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { // once the future is complete to verify addMessageIfTimeout was invoked val reply3 = intercept[RpcTimeoutException] { - Await.result(fut3, 2000 millis) + Await.result(fut3, 2.seconds) }.getMessage // scalastyle:on awaitresult @@ -822,63 +823,66 @@ abstract class RpcEnvSuite extends SparkFunSuite with BeforeAndAfterAll { } test("file server") { - val conf = new SparkConf() - val tempDir = Utils.createTempDir() - val file = new File(tempDir, "file") - Files.write(UUID.randomUUID().toString(), file, UTF_8) - val fileWithSpecialChars = new File(tempDir, "file name") - Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8) - val empty = new File(tempDir, "empty") - Files.write("", empty, UTF_8); - val jar = new File(tempDir, "jar") - Files.write(UUID.randomUUID().toString(), jar, UTF_8) - - val dir1 = new File(tempDir, "dir1") - assert(dir1.mkdir()) - val subFile1 = new File(dir1, "file1") - Files.write(UUID.randomUUID().toString(), subFile1, UTF_8) - - val dir2 = new File(tempDir, "dir2") - assert(dir2.mkdir()) - val subFile2 = new File(dir2, "file2") - Files.write(UUID.randomUUID().toString(), subFile2, UTF_8) - - val fileUri = env.fileServer.addFile(file) - val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars) - val emptyUri = env.fileServer.addFile(empty) - val jarUri = env.fileServer.addJar(jar) - val dir1Uri = env.fileServer.addDirectory("/dir1", dir1) - val dir2Uri = env.fileServer.addDirectory("/dir2", dir2) - - // Try registering directories with invalid names. - Seq("/files", "/jars").foreach { uri => - intercept[IllegalArgumentException] { - env.fileServer.addDirectory(uri, dir1) - } - } + withTempDir { tempDir => + withTempDir { destDir => + val conf = new SparkConf() + + val file = new File(tempDir, "file") + Files.write(UUID.randomUUID().toString(), file, UTF_8) + val fileWithSpecialChars = new File(tempDir, "file name") + Files.write(UUID.randomUUID().toString(), fileWithSpecialChars, UTF_8) + val empty = new File(tempDir, "empty") + Files.write("", empty, UTF_8); + val jar = new File(tempDir, "jar") + Files.write(UUID.randomUUID().toString(), jar, UTF_8) + + val dir1 = new File(tempDir, "dir1") + assert(dir1.mkdir()) + val subFile1 = new File(dir1, "file1") + Files.write(UUID.randomUUID().toString(), subFile1, UTF_8) + + val dir2 = new File(tempDir, "dir2") + assert(dir2.mkdir()) + val subFile2 = new File(dir2, "file2") + Files.write(UUID.randomUUID().toString(), subFile2, UTF_8) + + val fileUri = env.fileServer.addFile(file) + val fileWithSpecialCharsUri = env.fileServer.addFile(fileWithSpecialChars) + val emptyUri = env.fileServer.addFile(empty) + val jarUri = env.fileServer.addJar(jar) + val dir1Uri = env.fileServer.addDirectory("/dir1", dir1) + val dir2Uri = env.fileServer.addDirectory("/dir2", dir2) + + // Try registering directories with invalid names. + Seq("/files", "/jars").foreach { uri => + intercept[IllegalArgumentException] { + env.fileServer.addDirectory(uri, dir1) + } + } - val destDir = Utils.createTempDir() - val sm = new SecurityManager(conf) - val hc = SparkHadoopUtil.get.conf - - val files = Seq( - (file, fileUri), - (fileWithSpecialChars, fileWithSpecialCharsUri), - (empty, emptyUri), - (jar, jarUri), - (subFile1, dir1Uri + "/file1"), - (subFile2, dir2Uri + "/file2")) - files.foreach { case (f, uri) => - val destFile = new File(destDir, f.getName()) - Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) - assert(Files.equal(f, destFile)) - } + val sm = new SecurityManager(conf) + val hc = SparkHadoopUtil.get.conf + + val files = Seq( + (file, fileUri), + (fileWithSpecialChars, fileWithSpecialCharsUri), + (empty, emptyUri), + (jar, jarUri), + (subFile1, dir1Uri + "/file1"), + (subFile2, dir2Uri + "/file2")) + files.foreach { case (f, uri) => + val destFile = new File(destDir, f.getName()) + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + assert(Files.equal(f, destFile)) + } - // Try to download files that do not exist. - Seq("files", "jars", "dir1").foreach { root => - intercept[Exception] { - val uri = env.address.toSparkURL + s"/$root/doesNotExist" - Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + // Try to download files that do not exist. + Seq("files", "jars", "dir1").foreach { root => + intercept[Exception] { + val uri = env.address.toSparkURL + s"/$root/doesNotExist" + Utils.fetchFile(uri, destDir, conf, sm, hc, 0L, false) + } + } } } } diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala index f9481f875d439..59b4b706bbcdd 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcEnvSuite.scala @@ -17,13 +17,20 @@ package org.apache.spark.rpc.netty +import java.util.concurrent.ExecutionException + +import scala.concurrent.duration._ + +import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.network.client.TransportClient import org.apache.spark.rpc._ -class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar { +class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar with TimeLimits { + + private implicit val signaler: Signaler = ThreadSignaler override def createRpcEnv( conf: SparkConf, @@ -84,4 +91,48 @@ class NettyRpcEnvSuite extends RpcEnvSuite with MockitoSugar { msg3, RequestMessage(nettyEnv, client, msg3.serialize(nettyEnv))) } + + test("StackOverflowError should be sent back and Dispatcher should survive") { + val numUsableCores = 2 + val conf = new SparkConf + val config = RpcEnvConfig( + conf, + "test", + "localhost", + "localhost", + 0, + new SecurityManager(conf), + numUsableCores, + clientMode = false) + val anotherEnv = new NettyRpcEnvFactory().create(config) + anotherEnv.setupEndpoint("StackOverflowError", new RpcEndpoint { + override val rpcEnv = anotherEnv + + override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { + // scalastyle:off throwerror + case msg: String => throw new StackOverflowError + // scalastyle:on throwerror + case num: Int => context.reply(num) + } + }) + + val rpcEndpointRef = env.setupEndpointRef(anotherEnv.address, "StackOverflowError") + try { + // Send `numUsableCores` messages to trigger `numUsableCores` `StackOverflowError`s + for (_ <- 0 until numUsableCores) { + val e = intercept[SparkException] { + rpcEndpointRef.askSync[String]("hello") + } + // The root cause `e.getCause.getCause` because it is boxed by Scala Promise. + assert(e.getCause.isInstanceOf[ExecutionException]) + assert(e.getCause.getCause.isInstanceOf[StackOverflowError]) + } + failAfter(10.seconds) { + assert(rpcEndpointRef.askSync[Int](100) === 100) + } + } finally { + anotherEnv.shutdown() + anotherEnv.awaitTermination() + } + } } diff --git a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala index a71d8726e7066..4bc001fe8f7c5 100644 --- a/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala +++ b/core/src/test/scala/org/apache/spark/rpc/netty/NettyRpcHandlerSuite.scala @@ -21,7 +21,7 @@ import java.net.InetSocketAddress import java.nio.ByteBuffer import io.netty.channel.Channel -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark.SparkFunSuite 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 36dd620a56853..112fd31a060e6 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BarrierTaskContextSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import scala.util.Random import org.apache.spark._ +import org.apache.spark.internal.config.Tests.TEST_NO_STAGE_RETRY class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { @@ -76,7 +77,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception on barrier() call timeout") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) @@ -101,7 +102,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception if barrier() call doesn't happen on every task") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) @@ -124,7 +125,7 @@ class BarrierTaskContextSuite extends SparkFunSuite with LocalSparkContext { test("throw exception if the number of barrier() calls are not the same on every task") { val conf = new SparkConf() .set("spark.barrier.sync.timeout", "1") - .set("spark.test.noStageRetry", "true") + .set(TEST_NO_STAGE_RETRY, true) .setMaster("local-cluster[4, 1, 1024]") .setAppName("test-cluster") sc = new SparkContext(conf) diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala index fe22d70850c7d..0fe0e5b78233c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistIntegrationSuite.scala @@ -20,6 +20,7 @@ import scala.concurrent.duration._ import org.apache.spark._ import org.apache.spark.internal.config +import org.apache.spark.internal.config.Tests._ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorMockBackend]{ @@ -57,10 +58,10 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM "With default settings, job can succeed despite multiple bad executors on node", extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", - config.MAX_TASK_FAILURES.key -> "4", - "spark.testing.nHosts" -> "2", - "spark.testing.nExecutorsPerHost" -> "5", - "spark.testing.nCoresPerExecutor" -> "10" + config.TASK_MAX_FAILURES.key -> "4", + TEST_N_HOSTS.key -> "2", + TEST_N_EXECUTORS_HOST.key -> "5", + TEST_N_CORES_EXECUTOR.key -> "10" ) ) { // To reliably reproduce the failure that would occur without blacklisting, we have to use 1 @@ -84,7 +85,7 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", // just to avoid this test taking too long - "spark.locality.wait" -> "10ms" + config.LOCALITY_WAIT.key -> "10ms" ) ) { val rdd = new MockRDDWithLocalityPrefs(sc, 10, Nil, badHost) @@ -96,15 +97,16 @@ class BlacklistIntegrationSuite extends SchedulerIntegrationSuite[MultiExecutorM assertDataStructuresEmpty(noFailure = true) } - // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, the job - // doesn't hang + // Make sure that if we've failed on all executors, but haven't hit task.maxFailures yet, we try + // to acquire a new executor and if we aren't able to get one, the job doesn't hang and we abort testScheduler( "SPARK-15865 Progress with fewer executors than maxTaskFailures", extraConfs = Seq( config.BLACKLIST_ENABLED.key -> "true", - "spark.testing.nHosts" -> "2", - "spark.testing.nExecutorsPerHost" -> "1", - "spark.testing.nCoresPerExecutor" -> "1" + TEST_N_HOSTS.key -> "2", + TEST_N_EXECUTORS_HOST.key -> "1", + TEST_N_CORES_EXECUTOR.key -> "1", + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0s" ) ) { def runBackend(): Unit = { @@ -128,9 +130,9 @@ class MultiExecutorMockBackend( conf: SparkConf, taskScheduler: TaskSchedulerImpl) extends MockBackend(conf, taskScheduler) { - val nHosts = conf.getInt("spark.testing.nHosts", 5) - val nExecutorsPerHost = conf.getInt("spark.testing.nExecutorsPerHost", 4) - val nCoresPerExecutor = conf.getInt("spark.testing.nCoresPerExecutor", 2) + val nHosts = conf.get(TEST_N_HOSTS) + val nExecutorsPerHost = conf.get(TEST_N_EXECUTORS_HOST) + val nCoresPerExecutor = conf.get(TEST_N_CORES_EXECUTOR) override val executorIdToExecutor: Map[String, ExecutorTaskStatus] = { (0 until nHosts).flatMap { hostIdx => diff --git a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala index 96c8404327e24..93a88cc30a20c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/BlacklistTrackerSuite.scala @@ -17,10 +17,9 @@ package org.apache.spark.scheduler -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{never, verify, when} import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar @@ -443,20 +442,20 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M (2, 2), (2, 3) ).foreach { case (maxTaskFailures, maxNodeAttempts) => - conf.set(config.MAX_TASK_FAILURES, maxTaskFailures) + conf.set(config.TASK_MAX_FAILURES, maxTaskFailures) conf.set(config.MAX_TASK_ATTEMPTS_PER_NODE.key, maxNodeAttempts.toString) val excMsg = intercept[IllegalArgumentException] { BlacklistTracker.validateBlacklistConfs(conf) }.getMessage() assert(excMsg === s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key} " + - s"( = ${maxNodeAttempts}) was >= ${config.MAX_TASK_FAILURES.key} " + + s"( = ${maxNodeAttempts}) was >= ${config.TASK_MAX_FAILURES.key} " + s"( = ${maxTaskFailures} ). Though blacklisting is enabled, with this configuration, " + s"Spark will not be robust to one bad node. Decrease " + - s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.MAX_TASK_FAILURES.key}, " + + s"${config.MAX_TASK_ATTEMPTS_PER_NODE.key}, increase ${config.TASK_MAX_FAILURES.key}, " + s"or disable blacklisting with ${config.BLACKLIST_ENABLED.key}") } - conf.remove(config.MAX_TASK_FAILURES) + conf.remove(config.TASK_MAX_FAILURES) conf.remove(config.MAX_TASK_ATTEMPTS_PER_NODE) Seq( @@ -480,17 +479,16 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { val allocationClientMock = mock[ExecutorAllocationClient] when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called")) - when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] { + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => // To avoid a race between blacklisting and killing, it is important that the nodeBlacklist // is updated before we ask the executor allocation client to kill all the executors // on a particular host. - override def answer(invocation: InvocationOnMock): Boolean = { - if (blacklist.nodeBlacklist.contains("hostA") == false) { - throw new IllegalStateException("hostA should be on the blacklist") - } + if (blacklist.nodeBlacklist.contains("hostA")) { true + } else { + throw new IllegalStateException("hostA should be on the blacklist") } - }) + } blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) // Disable auto-kill. Blacklist an executor and make sure killExecutors is not called. @@ -552,17 +550,16 @@ class BlacklistTrackerSuite extends SparkFunSuite with BeforeAndAfterEach with M test("fetch failure blacklisting kills executors, configured by BLACKLIST_KILL_ENABLED") { val allocationClientMock = mock[ExecutorAllocationClient] when(allocationClientMock.killExecutors(any(), any(), any(), any())).thenReturn(Seq("called")) - when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer(new Answer[Boolean] { + when(allocationClientMock.killExecutorsOnHost("hostA")).thenAnswer { (_: InvocationOnMock) => // To avoid a race between blacklisting and killing, it is important that the nodeBlacklist // is updated before we ask the executor allocation client to kill all the executors // on a particular host. - override def answer(invocation: InvocationOnMock): Boolean = { - if (blacklist.nodeBlacklist.contains("hostA") == false) { - throw new IllegalStateException("hostA should be on the blacklist") - } + if (blacklist.nodeBlacklist.contains("hostA")) { true + } else { + throw new IllegalStateException("hostA should be on the blacklist") } - }) + } conf.set(config.BLACKLIST_FETCH_FAILURE_ENABLED, true) blacklist = new BlacklistTracker(listenerBusMock, conf, Some(allocationClientMock), clock) 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 80c9c6f0422a8..3edbbeb9c08f1 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CoarseGrainedSchedulerBackendSuite.scala @@ -17,22 +17,40 @@ package org.apache.spark.scheduler +import java.util.Properties import java.util.concurrent.atomic.AtomicBoolean +import scala.collection.immutable +import scala.collection.mutable import scala.concurrent.duration._ +import scala.language.postfixOps +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.when +import org.mockito.invocation.InvocationOnMock import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar._ -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.rdd.RDD -import org.apache.spark.util.{RpcUtils, SerializableBuffer} +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ +import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.{RpcUtils, SerializableBuffer, Utils} class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkContext with Eventually { + private val executorUpTimeout = 1.minute + test("serialized task larger than max RPC message size") { val conf = new SparkConf - conf.set("spark.rpc.message.maxSize", "1") + conf.set(RPC_MESSAGE_MAX_SIZE, 1) conf.set("spark.default.parallelism", "1") sc = new SparkContext("local-cluster[2, 1, 1024]", "test", conf) val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) @@ -51,7 +69,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -60,11 +78,11 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo test("compute max number of concurrent tasks can be launched when spark.task.cpus > 1") { val conf = new SparkConf() - .set("spark.task.cpus", "2") + .set(CPUS_PER_TASK, 2) .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -74,7 +92,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo test("compute max number of concurrent tasks can be launched when some executors are busy") { val conf = new SparkConf() - .set("spark.task.cpus", "2") + .set(CPUS_PER_TASK, 2) .setMaster("local-cluster[4, 3, 1024]") .setAppName("test") sc = new SparkContext(conf) @@ -96,7 +114,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo try { sc.addSparkListener(listener) - eventually(timeout(10.seconds)) { + eventually(timeout(executorUpTimeout)) { // Ensure all executors have been launched. assert(sc.getExecutorIds().length == 4) } @@ -106,7 +124,7 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo eventually(timeout(10.seconds)) { // Ensure some tasks have started and no task finished, so some executors must be busy. - assert(taskStarted.get() == true) + assert(taskStarted.get()) assert(taskEnded.get() == false) // Assert we count in slots on both busy and free executors. assert(sc.maxNumConcurrentTasks() == 4) @@ -116,6 +134,127 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo } } + // Here we just have test for one happy case instead of all cases: other cases are covered in + // FsHistoryProviderSuite. + test("custom log url for Spark UI is applied") { + val customExecutorLogUrl = "http://newhost:9999/logs/clusters/{{CLUSTER_ID}}/users/{{USER}}" + + "/containers/{{CONTAINER_ID}}/{{FILE_NAME}}" + + val conf = new SparkConf() + .set(UI.CUSTOM_EXECUTOR_LOG_URL, customExecutorLogUrl) + .setMaster("local-cluster[0, 3, 1024]") + .setAppName("test") + + sc = new SparkContext(conf) + val backend = sc.schedulerBackend.asInstanceOf[CoarseGrainedSchedulerBackend] + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + + val logUrls = Map( + "stdout" -> "http://oldhost:8888/logs/dummy/stdout", + "stderr" -> "http://oldhost:8888/logs/dummy/stderr") + val attributes = Map( + "CLUSTER_ID" -> "cl1", + "USER" -> "dummy", + "CONTAINER_ID" -> "container1", + "LOG_FILES" -> "stdout,stderr") + val baseUrl = s"http://newhost:9999/logs/clusters/${attributes("CLUSTER_ID")}" + + s"/users/${attributes("USER")}/containers/${attributes("CONTAINER_ID")}" + + var executorAddedCount: Int = 0 + val listener = new SparkListener() { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + executorAddedCount += 1 + assert(executorAdded.executorInfo.logUrlMap === Seq("stdout", "stderr").map { file => + file -> (baseUrl + s"/$file") + }.toMap) + } + } + + sc.addSparkListener(listener) + + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, logUrls, attributes, Map.empty)) + + sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) + assert(executorAddedCount === 3) + } + + test("extra resources from executor") { + import TestUtils._ + + val conf = new SparkConf() + .set(EXECUTOR_CORES, 3) + .set(SCHEDULER_REVIVE_INTERVAL.key, "1m") // don't let it auto revive during test + .setMaster( + "coarseclustermanager[org.apache.spark.scheduler.TestCoarseGrainedSchedulerBackend]") + .setAppName("test") + conf.set(TASK_GPU_ID.amountConf, "1") + conf.set(EXECUTOR_GPU_ID.amountConf, "1") + + sc = new SparkContext(conf) + val backend = sc.schedulerBackend.asInstanceOf[TestCoarseGrainedSchedulerBackend] + val mockEndpointRef = mock[RpcEndpointRef] + val mockAddress = mock[RpcAddress] + when(mockEndpointRef.send(LaunchTask)).thenAnswer((_: InvocationOnMock) => {}) + + val resources = Map(GPU -> new ResourceInformation(GPU, Array("0", "1", "3"))) + + var executorAddedCount: Int = 0 + val listener = new SparkListener() { + override def onExecutorAdded(executorAdded: SparkListenerExecutorAdded): Unit = { + executorAddedCount += 1 + } + } + + sc.addSparkListener(listener) + + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("1", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("2", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + backend.driverEndpoint.askSync[Boolean]( + RegisterExecutor("3", mockEndpointRef, mockAddress.host, 1, Map.empty, Map.empty, resources)) + + val frameSize = RpcUtils.maxMessageSizeBytes(sc.conf) + val bytebuffer = java.nio.ByteBuffer.allocate(frameSize - 100) + val buffer = new SerializableBuffer(bytebuffer) + + var execResources = backend.getExecutorAvailableResources("1") + + assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) + + val taskResources = Map(GPU -> new ResourceInformation(GPU, Array("0"))) + var taskDescs: Seq[Seq[TaskDescription]] = Seq(Seq(new TaskDescription(1, 0, "1", + "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) + + backend.driverEndpoint.send(ReviveOffers) + + eventually(timeout(5 seconds)) { + execResources = backend.getExecutorAvailableResources("1") + assert(execResources(GPU).availableAddrs.sorted === Array("1", "3")) + assert(execResources(GPU).assignedAddrs === Array("0")) + } + + backend.driverEndpoint.send( + StatusUpdate("1", 1, TaskState.FINISHED, buffer, taskResources)) + + eventually(timeout(5 seconds)) { + execResources = backend.getExecutorAvailableResources("1") + assert(execResources(GPU).availableAddrs.sorted === Array("0", "1", "3")) + assert(execResources(GPU).assignedAddrs.isEmpty) + } + sc.listenerBus.waitUntilEmpty(executorUpTimeout.toMillis) + assert(executorAddedCount === 3) + } + private def testSubmitJob(sc: SparkContext, rdd: RDD[Int]): Unit = { sc.submitJob( rdd, @@ -126,3 +265,47 @@ class CoarseGrainedSchedulerBackendSuite extends SparkFunSuite with LocalSparkCo ) } } + +/** Simple cluster manager that wires up our mock backend for the resource tests. */ +private class CSMockExternalClusterManager extends ExternalClusterManager { + + private var ts: TaskSchedulerImpl = _ + + private val MOCK_REGEX = """coarseclustermanager\[(.*)\]""".r + override def canCreate(masterURL: String): Boolean = MOCK_REGEX.findFirstIn(masterURL).isDefined + + override def createTaskScheduler( + sc: SparkContext, + masterURL: String): TaskScheduler = { + ts = mock[TaskSchedulerImpl] + when(ts.sc).thenReturn(sc) + when(ts.applicationId()).thenReturn("appid1") + when(ts.applicationAttemptId()).thenReturn(Some("attempt1")) + when(ts.schedulingMode).thenReturn(SchedulingMode.FIFO) + when(ts.nodeBlacklist()).thenReturn(Set.empty[String]) + ts + } + + override def createSchedulerBackend( + sc: SparkContext, + masterURL: String, + scheduler: TaskScheduler): SchedulerBackend = { + masterURL match { + case MOCK_REGEX(backendClassName) => + val backendClass = Utils.classForName(backendClassName) + val ctor = backendClass.getConstructor(classOf[TaskSchedulerImpl], classOf[RpcEnv]) + ctor.newInstance(scheduler, sc.env.rpcEnv).asInstanceOf[SchedulerBackend] + } + } + + override def initialize(scheduler: TaskScheduler, backend: SchedulerBackend): Unit = { + scheduler.asInstanceOf[TaskSchedulerImpl].initialize(backend) + } +} + +private[spark] +class TestCoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, override val rpcEnv: RpcEnv) + extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + + def getTaskSchedulerImpl(): TaskSchedulerImpl = scheduler +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala b/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala index 838686923767e..1be2e2a067115 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/CustomShuffledRDD.scala @@ -104,8 +104,9 @@ class CustomShuffledRDD[K, V, C]( override def compute(p: Partition, context: TaskContext): Iterator[(K, C)] = { val part = p.asInstanceOf[CustomShuffledRDDPartition] + val metrics = context.taskMetrics().createTempShuffleReadMetrics() SparkEnv.get.shuffleManager.getReader( - dependency.shuffleHandle, part.startIndexInParent, part.endIndexInParent, context) + dependency.shuffleHandle, part.startIndexInParent, part.endIndexInParent, context, metrics) .read() .asInstanceOf[Iterator[(K, C)]] } 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 b41d2acab7152..d58ee4e651e19 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,11 +18,11 @@ package org.apache.spark.scheduler import java.util.Properties +import java.util.concurrent.{CountDownLatch, TimeUnit} import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import scala.annotation.meta.param import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} -import scala.language.reflectiveCalls import scala.util.control.NonFatal import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} @@ -133,6 +133,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi /** Stages for which the DAGScheduler has called TaskScheduler.cancelTasks(). */ val cancelledStages = new HashSet[Int]() + val tasksMarkedAsCompleted = new ArrayBuffer[Task[_]]() + val taskScheduler = new TaskScheduler() { override def schedulingMode: SchedulingMode = SchedulingMode.FIFO override def rootPool: Pool = new Pool("", schedulingMode, 0, 0) @@ -155,6 +157,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskId: Long, interruptThread: Boolean, reason: String): Boolean = false override def killAllTaskAttempts( stageId: Int, interruptThread: Boolean, reason: String): Unit = {} + override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = { + taskSets.filter(_.stageId == stageId).lastOption.foreach { ts => + val tasks = ts.tasks.filter(_.partitionId == partitionId) + assert(tasks.length == 1) + tasksMarkedAsCompleted += tasks.head + } + } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} @@ -164,13 +173,13 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 - val sparkListener = new SparkListener() { - val submittedStageInfos = new HashSet[StageInfo] - val successfulStages = new HashSet[Int] - val failedStages = new ArrayBuffer[Int] - val stageByOrderOfExecution = new ArrayBuffer[Int] - val endedTasks = new HashSet[Long] + val submittedStageInfos = new HashSet[StageInfo] + val successfulStages = new HashSet[Int] + val failedStages = new ArrayBuffer[Int] + val stageByOrderOfExecution = new ArrayBuffer[Int] + val endedTasks = new HashSet[Long] + val sparkListener = new SparkListener() { override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { submittedStageInfos += stageSubmitted.stageInfo } @@ -238,13 +247,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi private def init(testConf: SparkConf): Unit = { sc = new SparkContext("local[2]", "DAGSchedulerSuite", testConf) - sparkListener.submittedStageInfos.clear() - sparkListener.successfulStages.clear() - sparkListener.failedStages.clear() - sparkListener.endedTasks.clear() + submittedStageInfos.clear() + successfulStages.clear() + failedStages.clear() + endedTasks.clear() failure = null sc.addSparkListener(sparkListener) taskSets.clear() + tasksMarkedAsCompleted.clear() cancelledStages.clear() cacheLocations.clear() results.clear() @@ -363,11 +373,11 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("[SPARK-3353] parent stage should have lower stage id") { - sparkListener.stageByOrderOfExecution.clear() + stageByOrderOfExecution.clear() sc.parallelize(1 to 10).map(x => (x, x)).reduceByKey(_ + _, 4).count() sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.stageByOrderOfExecution.length === 2) - assert(sparkListener.stageByOrderOfExecution(0) < sparkListener.stageByOrderOfExecution(1)) + assert(stageByOrderOfExecution.length === 2) + assert(stageByOrderOfExecution(0) < stageByOrderOfExecution(1)) } /** @@ -593,7 +603,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi var rdd: RDD[_] = new MyRDD(sc, 1, Nil) (1 to 30).foreach(_ => rdd = rdd.zip(rdd)) // getPreferredLocs runs quickly, indicating that exponential graph traversal is avoided. - failAfter(10 seconds) { + failAfter(10.seconds) { val preferredLocs = scheduler.getPreferredLocs(rdd, 0) // No preferred locations are returned. assert(preferredLocs.length === 0) @@ -609,8 +619,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assert(failure.getMessage.startsWith( "Job aborted due to stage failure: Task not serializable:")) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.contains(0)) - assert(sparkListener.failedStages.size === 1) + assert(failedStages.contains(0)) + assert(failedStages.size === 1) assertDataStructuresEmpty() } @@ -619,8 +629,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.contains(0)) - assert(sparkListener.failedStages.size === 1) + assert(failedStages.contains(0)) + assert(failedStages.size === 1) assertDataStructuresEmpty() } @@ -630,8 +640,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled ") sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.contains(0)) - assert(sparkListener.failedStages.size === 1) + assert(failedStages.contains(0)) + assert(failedStages.size === 1) assertDataStructuresEmpty() } @@ -657,6 +667,9 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi stageId: Int, interruptThread: Boolean, reason: String): Unit = { throw new UnsupportedOperationException } + override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = { + throw new UnsupportedOperationException + } override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 override def executorHeartbeatReceived( @@ -687,8 +700,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.isEmpty) - assert(sparkListener.successfulStages.contains(0)) + assert(failedStages.isEmpty) + assert(successfulStages.contains(0)) } test("run trivial shuffle") { @@ -890,7 +903,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Confirm job finished successfully sc.listenerBus.waitUntilEmpty(1000) - assert(ended === true) + assert(ended) assert(results === (0 until parts).map { idx => idx -> 42 }.toMap) assertDataStructuresEmpty() } @@ -1049,7 +1062,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi assertDataStructuresEmpty() sc.listenerBus.waitUntilEmpty(1000) - assert(ended === true) + assert(ended) assert(results === Map(0 -> 42)) } @@ -1072,7 +1085,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.contains(1)) + assert(failedStages.contains(1)) // The second ResultTask fails, with a fetch failure for the output from the second mapper. runEvent(makeCompletionEvent( @@ -1081,7 +1094,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi null)) // The SparkListener should not receive redundant failure events. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.size == 1) + assert(failedStages.size == 1) } test("Retry all the tasks on a resubmitted attempt of a barrier stage caused by FetchFailure") { @@ -1128,7 +1141,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(1), TaskKilled("test"), null)) - assert(sparkListener.failedStages === Seq(0)) + assert(failedStages === Seq(0)) assert(mapOutputTracker.findMissingPartitions(shuffleId) === Some(Seq(0, 1))) scheduler.resubmitFailedStages() @@ -1182,7 +1195,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - sparkListener.submittedStageInfos.count(_.stageId == mapStageId) + submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. @@ -1204,7 +1217,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi FetchFailed(makeBlockManagerId("hostA"), shuffleId, 0, 0, "ignored"), null)) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.contains(1)) + assert(failedStages.contains(1)) // Trigger resubmission of the failed map stage. runEvent(ResubmitFailedStages) @@ -1243,10 +1256,10 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi submit(reduceRdd, Array(0, 1)) def countSubmittedReduceStageAttempts(): Int = { - sparkListener.submittedStageInfos.count(_.stageId == 1) + submittedStageInfos.count(_.stageId == 1) } def countSubmittedMapStageAttempts(): Int = { - sparkListener.submittedStageInfos.count(_.stageId == 0) + submittedStageInfos.count(_.stageId == 0) } // The map stage should have been submitted. @@ -1307,7 +1320,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) // verify stage exists assert(scheduler.stageIdToStage.contains(0)) - assert(sparkListener.endedTasks.size == 2) + assert(endedTasks.size == 2) // finish other 2 tasks runEvent(makeCompletionEvent( @@ -1317,7 +1330,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(3), Success, 42, Seq.empty, createFakeTaskInfoWithId(3))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.endedTasks.size == 4) + assert(endedTasks.size == 4) // verify the stage is done assert(!scheduler.stageIdToStage.contains(0)) @@ -1328,14 +1341,14 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(3), Success, 42, Seq.empty, createFakeTaskInfoWithId(5))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.endedTasks.size == 5) + assert(endedTasks.size == 5) // make sure non successful tasks also send out event runEvent(makeCompletionEvent( taskSets(0).tasks(3), UnknownReason, 42, Seq.empty, createFakeTaskInfoWithId(6))) sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.endedTasks.size == 6) + assert(endedTasks.size == 6) } test("ignore late map task completions") { @@ -1409,7 +1422,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Listener bus should get told about the map stage failing, but not the reduce stage // (since the reduce stage hasn't been started yet). sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.failedStages.toSet === Set(0)) + assert(failedStages.toSet === Set(0)) assertDataStructuresEmpty() } @@ -1653,8 +1666,8 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi // Make sure the listeners got told about both failed stages. sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS) - assert(sparkListener.successfulStages.isEmpty) - assert(sparkListener.failedStages.toSet === Set(0, 2)) + assert(successfulStages.isEmpty) + assert(failedStages.toSet === Set(0, 2)) assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") @@ -1901,27 +1914,50 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } /** - * The job will be failed on first task throwing a DAGSchedulerSuiteDummyException. + * The job will be failed on first task throwing an error. * Any subsequent task WILL throw a legitimate java.lang.UnsupportedOperationException. * If multiple tasks, there exists a race condition between the SparkDriverExecutionExceptions * and their differing causes as to which will represent result for job... */ test("misbehaved resultHandler should not crash DAGScheduler and SparkContext") { - val e = intercept[SparkDriverExecutionException] { - // Number of parallelized partitions implies number of tasks of job - val rdd = sc.parallelize(1 to 10, 2) - sc.runJob[Int, Int]( - rdd, - (context: TaskContext, iter: Iterator[Int]) => iter.size, - // For a robust test assertion, limit number of job tasks to 1; that is, - // if multiple RDD partitions, use id of any one partition, say, first partition id=0 - Seq(0), - (part: Int, result: Int) => throw new DAGSchedulerSuiteDummyException) + failAfter(1.minute) { // If DAGScheduler crashes, the following test will hang forever + for (error <- Seq( + new DAGSchedulerSuiteDummyException, + new AssertionError, // E.g., assert(foo == bar) fails + new NotImplementedError // E.g., call a method with `???` implementation. + )) { + val e = intercept[SparkDriverExecutionException] { + // Number of parallelized partitions implies number of tasks of job + val rdd = sc.parallelize(1 to 10, 2) + sc.runJob[Int, Int]( + rdd, + (context: TaskContext, iter: Iterator[Int]) => iter.size, + // For a robust test assertion, limit number of job tasks to 1; that is, + // if multiple RDD partitions, use id of any one partition, say, first partition id=0 + Seq(0), + (part: Int, result: Int) => throw error) + } + assert(e.getCause eq error) + + // Make sure we can still run commands on our SparkContext + assert(sc.parallelize(1 to 10, 2).count() === 10) + } } - assert(e.getCause.isInstanceOf[DAGSchedulerSuiteDummyException]) + } - // Make sure we can still run commands on our SparkContext - assert(sc.parallelize(1 to 10, 2).count() === 10) + test(s"invalid ${SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL} should not crash DAGScheduler") { + sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "invalid") + try { + intercept[SparkException] { + sc.parallelize(1 to 1, 1).foreach { _ => + throw new DAGSchedulerSuiteDummyException + } + } + // Verify the above job didn't crash DAGScheduler by running a simple job + assert(sc.parallelize(1 to 10, 2).count() === 10) + } finally { + sc.setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) + } } test("getPartitions exceptions should not crash DAGScheduler and SparkContext (SPARK-8606)") { @@ -2416,7 +2452,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi * * A <------------s---------, * \ - * B <--s-- C <--s-- D <--n---`-- E + * 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. @@ -2600,13 +2636,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi test("Barrier task failures from the same stage attempt don't trigger multiple stage retries") { val shuffleMapRdd = new MyRDD(sc, 2, Nil).barrier().mapPartitions(iter => iter) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) - val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - sparkListener.submittedStageInfos.count(_.stageId == mapStageId) + submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. @@ -2618,7 +2653,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(0), TaskKilled("test"), null)) - assert(sparkListener.failedStages === Seq(0)) + assert(failedStages === Seq(0)) // The second map task fails with TaskKilled. runEvent(makeCompletionEvent( @@ -2637,13 +2672,12 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi test("Barrier task failures from a previous stage attempt don't trigger stage retry") { val shuffleMapRdd = new MyRDD(sc, 2, Nil).barrier().mapPartitions(iter => iter) val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(2)) - val shuffleId = shuffleDep.shuffleId val reduceRdd = new MyRDD(sc, 2, List(shuffleDep), tracker = mapOutputTracker) submit(reduceRdd, Array(0, 1)) val mapStageId = 0 def countSubmittedMapStageAttempts(): Int = { - sparkListener.submittedStageInfos.count(_.stageId == mapStageId) + submittedStageInfos.count(_.stageId == mapStageId) } // The map stage should have been submitted. @@ -2655,7 +2689,7 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi taskSets(0).tasks(0), TaskKilled("test"), null)) - assert(sparkListener.failedStages === Seq(0)) + assert(failedStages === Seq(0)) // Trigger resubmission of the failed map stage. runEvent(ResubmitFailedStages) @@ -2808,18 +2842,85 @@ class DAGSchedulerSuite extends SparkFunSuite with LocalSparkContext with TimeLi } test("SPARK-23207: reliable checkpoint can avoid rollback (checkpointed before)") { - sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) - shuffleMapRdd.checkpoint() - shuffleMapRdd.doCheckpoint() - assertResultStageNotRollbacked(shuffleMapRdd) + withTempDir { dir => + sc.setCheckpointDir(dir.getCanonicalPath) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + shuffleMapRdd.doCheckpoint() + assertResultStageNotRollbacked(shuffleMapRdd) + } } test("SPARK-23207: reliable checkpoint fail to rollback (checkpointing now)") { - sc.setCheckpointDir(Utils.createTempDir().getCanonicalPath) - val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) - shuffleMapRdd.checkpoint() - assertResultStageFailToRollback(shuffleMapRdd) + withTempDir { dir => + sc.setCheckpointDir(dir.getCanonicalPath) + val shuffleMapRdd = new MyCheckpointRDD(sc, 2, Nil, indeterminate = true) + shuffleMapRdd.checkpoint() + assertResultStageFailToRollback(shuffleMapRdd) + } + } + + test("SPARK-27164: RDD.countApprox on empty RDDs schedules jobs which never complete") { + val latch = new CountDownLatch(1) + val jobListener = new SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + latch.countDown() + } + } + sc.addSparkListener(jobListener) + sc.emptyRDD[Int].countApprox(10000).getFinalValue() + assert(latch.await(10, TimeUnit.SECONDS)) + } + + test("Completions in zombie tasksets update status of non-zombie taskset") { + val parts = 4 + val shuffleMapRdd = new MyRDD(sc, parts, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, new HashPartitioner(parts)) + val reduceRdd = new MyRDD(sc, parts, List(shuffleDep), tracker = mapOutputTracker) + submit(reduceRdd, (0 until parts).toArray) + assert(taskSets.length == 1) + + // Finish the first task of the shuffle map stage. + runEvent(makeCompletionEvent( + taskSets(0).tasks(0), Success, makeMapStatus("hostA", 4), + Seq.empty, createFakeTaskInfoWithId(0))) + + // The second task of the shuffle map stage failed with FetchFailed. + runEvent(makeCompletionEvent( + taskSets(0).tasks(1), + FetchFailed(makeBlockManagerId("hostB"), shuffleDep.shuffleId, 0, 0, "ignored"), + null)) + + scheduler.resubmitFailedStages() + assert(taskSets.length == 2) + // The first partition has completed already, so the new attempt only need to run 3 tasks. + assert(taskSets(1).tasks.length == 3) + + // Finish the first task of the second attempt of the shuffle map stage. + runEvent(makeCompletionEvent( + taskSets(1).tasks(0), Success, makeMapStatus("hostA", 4), + Seq.empty, createFakeTaskInfoWithId(0))) + + // Finish the third task of the first attempt of the shuffle map stage. + runEvent(makeCompletionEvent( + taskSets(0).tasks(2), Success, makeMapStatus("hostA", 4), + Seq.empty, createFakeTaskInfoWithId(0))) + assert(tasksMarkedAsCompleted.length == 1) + assert(tasksMarkedAsCompleted.head.partitionId == 2) + + // Finish the forth task of the first attempt of the shuffle map stage. + runEvent(makeCompletionEvent( + taskSets(0).tasks(3), Success, makeMapStatus("hostA", 4), + Seq.empty, createFakeTaskInfoWithId(0))) + assert(tasksMarkedAsCompleted.length == 2) + assert(tasksMarkedAsCompleted.last.partitionId == 3) + + // Now the shuffle map stage is completed, and the next stage is submitted. + assert(taskSets.length == 3) + + // Finish + complete(taskSets(2), Seq((Success, 42), (Success, 42), (Success, 42), (Success, 42))) + assertDataStructuresEmpty() } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala index cecd6996df7bd..5b8d254e9ea4e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/EventLoggingListenerSuite.scala @@ -33,6 +33,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ import org.apache.spark.io._ import org.apache.spark.metrics.{ExecutorMetricType, MetricsSystem} import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -86,6 +87,20 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit testEventLogging() } + test("spark.eventLog.compression.codec overrides spark.io.compression.codec") { + val conf = new SparkConf + conf.set(EVENT_LOG_COMPRESS, true) + + // The default value is `spark.io.compression.codec`. + val e = new EventLoggingListener("test", None, testDirPath.toUri(), conf) + assert(e.compressionCodecName.contains("lz4")) + + // `spark.eventLog.compression.codec` overrides `spark.io.compression.codec`. + conf.set(EVENT_LOG_COMPRESSION_CODEC, "zstd") + val e2 = new EventLoggingListener("test", None, testDirPath.toUri(), conf) + assert(e2.compressionCodecName.contains("zstd")) + } + test("Basic event logging with compression") { CompressionCodec.ALL_COMPRESSION_CODECS.foreach { codec => testEventLogging(compressionCodec = Some(CompressionCodec.getShortName(codec))) @@ -107,8 +122,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit val secretPassword = "secret_password" val conf = getLoggingConf(testDirPath, None) .set(key, secretPassword) + val hadoopconf = SparkHadoopUtil.get.newConfiguration(new SparkConf()) val eventLogger = new EventLoggingListener("test", None, testDirPath.toUri(), conf) - val envDetails = SparkEnv.environmentDetails(conf, "FIFO", Seq.empty, Seq.empty) + val envDetails = SparkEnv.environmentDetails(conf, hadoopconf, "FIFO", Seq.empty, Seq.empty) val event = SparkListenerEnvironmentUpdate(envDetails) val redactedProps = eventLogger.redactEvent(event).environmentDetails("Spark Properties").toMap assert(redactedProps(key) == "*********(redacted)") @@ -122,7 +138,7 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // Expected IOException, since we haven't enabled log overwrite. intercept[IOException] { testEventLogging() } // Try again, but enable overwriting. - testEventLogging(extraConf = Map("spark.eventLog.overwrite" -> "true")) + testEventLogging(extraConf = Map(EVENT_LOG_OVERWRITE.key -> "true")) } test("Event log name") { @@ -282,53 +298,67 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L, 10L, 90L, 2L, 20L))), createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))), - // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 + new ExecutorMetrics(Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), + // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))), + new ExecutorMetrics(Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))), + new ExecutorMetrics(Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))), + new ExecutorMetrics(Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))), + new ExecutorMetrics(Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L, 10L, 90L, 2L, 20L))), // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks createStageSubmittedEvent(1), // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7; initialize stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))), - // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9; + new ExecutorMetrics(Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, + 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), + // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 3, 6, 7, 9; // initialize stage 1 peaks createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))), + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, + 40L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L))), // complete stage 0, and 3 more updates for each executor with just // stage 1 running createStageCompletedEvent(0), // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))), - // enew ExecutorMetrics(xec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 + new ExecutorMetrics(Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L))), + // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))), + new ExecutorMetrics(Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, + 20L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))), // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))), + new ExecutorMetrics(Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, + 2000L, 1500L, 1000L, 500L, 10L, 90L, 2L, 20L))), // exec 2: new stage 1 peak for metrics at index: 7 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))), + new ExecutorMetrics(Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, + 20L, 7000L, 3000L, 6000L, 2000L, 5000L, 1000L, 10L, 90L, 2L, 20L))), // exec 1: no new stage 1 peaks createExecutorMetricsUpdateEvent(1, - new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))), + new ExecutorMetrics(Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, + 0L, 4000L, 2500L, 3000L, 1500L, 2000L, 500L, 10L, 90L, 2L, 20L))), createExecutorRemovedEvent(1), // exec 2: new stage 1 peak for metrics at index: 6 createExecutorMetricsUpdateEvent(2, - new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))), + new ExecutorMetrics(Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, + 4000L, 6000L, 3000L, 5000L, 2000L, 10L, 90L, 2L, 20L))), createStageCompletedEvent(1), SparkListenerApplicationEnd(1000L)) @@ -342,20 +372,23 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit // expected StageExecutorMetrics, for the given stage id and executor id val expectedMetricsEvents: Map[(Int, String), SparkListenerStageExecutorMetrics] = - Map( - ((0, "1"), - new SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))), - ((0, "2"), - new SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))), - ((1, "1"), - new SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))), - ((1, "2"), - new SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L))))) - + Map( + ((0, "1"), + new SparkListenerStageExecutorMetrics("1", 0, 0, + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L, 10L, 90L, 2L, 20L)))), + ((0, "2"), + new SparkListenerStageExecutorMetrics("2", 0, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, + 80L, 40L, 9000L, 4000L, 8000L, 3000L, 7000L, 2000L, 10L, 90L, 2L, 20L)))), + ((1, "1"), + new SparkListenerStageExecutorMetrics("1", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, + 50L, 0L, 5000L, 3000L, 4000L, 2000L, 3000L, 1000L, 10L, 90L, 2L, 20L)))), + ((1, "2"), + new SparkListenerStageExecutorMetrics("2", 1, 0, + new ExecutorMetrics(Array(7000L, 70L, 50L, 40L, 10L, 30L, 50L, 60L, + 40L, 40L, 8000L, 5000L, 7000L, 4000L, 6000L, 3000L, 10L, 90L, 2L, 20L))))) // Verify the log file contains the expected events. // Posted events should be logged, except for ExecutorMetricsUpdate events -- these // are consolidated, and the peak values for each stage are logged at stage end. @@ -403,7 +436,8 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit } private def createExecutorAddedEvent(executorId: Int) = { - SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + SparkListenerExecutorAdded(0L, executorId.toString, + new ExecutorInfo("host1", 1, Map.empty, Map.empty)) } private def createExecutorRemovedEvent(executorId: Int) = { @@ -456,9 +490,9 @@ class EventLoggingListenerSuite extends SparkFunSuite with LocalSparkContext wit assert(executorMetrics.execId === expectedMetrics.execId) assert(executorMetrics.stageId === expectedMetrics.stageId) assert(executorMetrics.stageAttemptId === expectedMetrics.stageAttemptId) - ExecutorMetricType.values.foreach { metricType => - assert(executorMetrics.executorMetrics.getMetricValue(metricType) === - expectedMetrics.executorMetrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(executorMetrics.executorMetrics.getMetricValue(metric._1) === + expectedMetrics.executorMetrics.getMetricValue(metric._1)) } case None => assert(false) @@ -509,15 +543,15 @@ object EventLoggingListenerSuite { /** Get a SparkConf with event logging enabled. */ def getLoggingConf(logDir: Path, compressionCodec: Option[String] = None): SparkConf = { val conf = new SparkConf - conf.set("spark.eventLog.enabled", "true") - conf.set("spark.eventLog.logBlockUpdates.enabled", "true") - conf.set("spark.eventLog.testing", "true") - conf.set("spark.eventLog.dir", logDir.toString) + conf.set(EVENT_LOG_ENABLED, true) + conf.set(EVENT_LOG_BLOCK_UPDATES, true) + conf.set(EVENT_LOG_TESTING, true) + conf.set(EVENT_LOG_DIR, logDir.toString) compressionCodec.foreach { codec => - conf.set("spark.eventLog.compress", "true") - conf.set("spark.io.compression.codec", codec) + conf.set(EVENT_LOG_COMPRESS, true) + conf.set(EVENT_LOG_COMPRESSION_CODEC, codec) } - conf.set("spark.eventLog.logStageExecutorMetrics.enabled", "true") + conf.set(EVENT_LOG_STAGE_EXECUTOR_METRICS, true) conf } diff --git a/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala new file mode 100644 index 0000000000000..0109d1f82a453 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/ExecutorResourceInfoSuite.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{SparkException, SparkFunSuite} +import org.apache.spark.resource.ResourceUtils.GPU + +class ExecutorResourceInfoSuite extends SparkFunSuite { + + test("Track Executor Resource information") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) + assert(info.assignedAddrs.isEmpty) + + // Acquire addresses + info.acquire(Seq("0", "1")) + assert(info.availableAddrs.sorted sameElements Seq("2", "3")) + assert(info.assignedAddrs.sorted sameElements Seq("0", "1")) + + // release addresses + info.release(Array("0", "1")) + assert(info.availableAddrs.sorted sameElements Seq("0", "1", "2", "3")) + assert(info.assignedAddrs.isEmpty) + } + + test("Don't allow acquire address that is not available") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + // Acquire some addresses. + info.acquire(Seq("0", "1")) + assert(!info.availableAddrs.contains("1")) + // Acquire an address that is not available + val e = intercept[SparkException] { + info.acquire(Array("1")) + } + assert(e.getMessage.contains("Try to acquire an address that is not available.")) + } + + test("Don't allow acquire address that doesn't exist") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(!info.availableAddrs.contains("4")) + // Acquire an address that doesn't exist + val e = intercept[SparkException] { + info.acquire(Array("4")) + } + assert(e.getMessage.contains("Try to acquire an address that doesn't exist.")) + } + + test("Don't allow release address that is not assigned") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + // Acquire addresses + info.acquire(Array("0", "1")) + assert(!info.assignedAddrs.contains("2")) + // Release an address that is not assigned + val e = intercept[SparkException] { + info.release(Array("2")) + } + assert(e.getMessage.contains("Try to release an address that is not assigned.")) + } + + test("Don't allow release address that doesn't exist") { + // Init Executor Resource. + val info = new ExecutorResourceInfo(GPU, ArrayBuffer("0", "1", "2", "3")) + assert(!info.assignedAddrs.contains("4")) + // Release an address that doesn't exist + val e = intercept[SparkException] { + info.release(Array("4")) + } + assert(e.getMessage.contains("Try to release an address that doesn't exist.")) + } +} 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 0621c98d41184..ead34e535723f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ExternalClusterManagerSuite.scala @@ -25,8 +25,7 @@ import org.apache.spark.util.AccumulatorV2 class ExternalClusterManagerSuite extends SparkFunSuite with LocalSparkContext { test("launch of backend and scheduler") { - val conf = new SparkConf().setMaster("myclusterManager"). - setAppName("testcm").set("spark.driver.allowMultipleContexts", "true") + val conf = new SparkConf().setMaster("myclusterManager").setAppName("testcm") sc = new SparkContext(conf) // check if the scheduler components are created and initialized sc.schedulerBackend match { @@ -85,6 +84,7 @@ private class DummyTaskScheduler extends TaskScheduler { taskId: Long, interruptThread: Boolean, reason: String): Boolean = false override def killAllTaskAttempts( stageId: Int, interruptThread: Boolean, reason: String): Unit = {} + override def notifyPartitionCompletion(stageId: Int, partitionId: Int): Unit = {} override def setDAGScheduler(dagScheduler: DAGScheduler): Unit = {} override def defaultParallelism(): Int = 2 override def executorLost(executorId: String, reason: ExecutorLossReason): Unit = {} diff --git a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala index 2155a0f2b6c21..c1e7fb9a1db16 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/MapStatusSuite.scala @@ -21,7 +21,7 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import scala.util.Random -import org.mockito.Mockito._ +import org.mockito.Mockito.mock import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} @@ -31,6 +31,7 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.storage.BlockManagerId class MapStatusSuite extends SparkFunSuite { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) test("compressSize") { assert(MapStatus.compressSize(0L) === 0) @@ -180,7 +181,7 @@ class MapStatusSuite extends SparkFunSuite { test("SPARK-21133 HighlyCompressedMapStatus#writeExternal throws NPE") { val conf = new SparkConf() - .set("spark.serializer", classOf[KryoSerializer].getName) + .set(config.SERIALIZER, classOf[KryoSerializer].getName) .setMaster("local") .setAppName("SPARK-21133") withSpark(new SparkContext(conf)) { sc => diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala index d6ff5bb33055c..848f702935536 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorIntegrationSuite.scala @@ -49,11 +49,8 @@ class OutputCommitCoordinatorIntegrationSuite test("exception thrown in OutputCommitter.commitTask()") { // Regression test for SPARK-10381 failAfter(Span(60, Seconds)) { - val tempDir = Utils.createTempDir() - try { + withTempDir { tempDir => sc.parallelize(1 to 4, 2).map(_.toString).saveAsTextFile(tempDir.getAbsolutePath + "/out") - } finally { - Utils.deleteRecursively(tempDir) } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala index 158c9eb75f2b6..f582ef54dd1f8 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/OutputCommitCoordinatorSuite.scala @@ -22,14 +22,12 @@ import java.util.Date import java.util.concurrent.TimeoutException import scala.concurrent.duration._ -import scala.language.postfixOps import org.apache.hadoop.mapred._ import org.apache.hadoop.mapreduce.TaskType -import org.mockito.Matchers -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{doAnswer, spy, times, verify} import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter import org.apache.spark._ @@ -71,6 +69,8 @@ import org.apache.spark.util.{ThreadUtils, Utils} */ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + var outputCommitCoordinator: OutputCommitCoordinator = null var tempDir: File = null var sc: SparkContext = null @@ -96,34 +96,29 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { // Use Mockito.spy() to maintain the default infrastructure everywhere else val mockTaskScheduler = spy(sc.taskScheduler.asInstanceOf[TaskSchedulerImpl]) - doAnswer(new Answer[Unit]() { - override def answer(invoke: InvocationOnMock): Unit = { - // Submit the tasks, then force the task scheduler to dequeue the - // speculated task - invoke.callRealMethod() - mockTaskScheduler.backend.reviveOffers() - } - }).when(mockTaskScheduler).submitTasks(Matchers.any()) - - doAnswer(new Answer[TaskSetManager]() { - override def answer(invoke: InvocationOnMock): TaskSetManager = { - val taskSet = invoke.getArguments()(0).asInstanceOf[TaskSet] - new TaskSetManager(mockTaskScheduler, taskSet, 4) { - var hasDequeuedSpeculatedTask = false - override def dequeueSpeculativeTask( - execId: String, - host: String, - locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { - if (!hasDequeuedSpeculatedTask) { - hasDequeuedSpeculatedTask = true - Some((0, TaskLocality.PROCESS_LOCAL)) - } else { - None - } + doAnswer { (invoke: InvocationOnMock) => + // Submit the tasks, then force the task scheduler to dequeue the + // speculated task + invoke.callRealMethod() + mockTaskScheduler.backend.reviveOffers() + }.when(mockTaskScheduler).submitTasks(any()) + + doAnswer { (invoke: InvocationOnMock) => + val taskSet = invoke.getArguments()(0).asInstanceOf[TaskSet] + new TaskSetManager(mockTaskScheduler, taskSet, 4) { + private var hasDequeuedSpeculatedTask = false + override def dequeueSpeculativeTask(execId: String, + host: String, + locality: TaskLocality.Value): Option[(Int, TaskLocality.Value)] = { + if (hasDequeuedSpeculatedTask) { + None + } else { + hasDequeuedSpeculatedTask = true + Some((0, TaskLocality.PROCESS_LOCAL)) } } } - }).when(mockTaskScheduler).createTaskSetManager(Matchers.any(), Matchers.any()) + }.when(mockTaskScheduler).createTaskSetManager(any(), any()) sc.taskScheduler = mockTaskScheduler val dagSchedulerWithMockTaskScheduler = new DAGScheduler(sc, mockTaskScheduler) @@ -154,7 +149,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { test("Job should not complete if all commits are denied") { // Create a mock OutputCommitCoordinator that denies all attempts to commit doReturn(false).when(outputCommitCoordinator).handleAskPermissionToCommit( - Matchers.any(), Matchers.any(), Matchers.any(), Matchers.any()) + any(), any(), any(), any()) val rdd: RDD[Int] = sc.parallelize(Seq(1), 1) def resultHandler(x: Int, y: Unit): Unit = {} val futureAction: SimpleFutureAction[Unit] = sc.submitJob[Int, Unit, Unit](rdd, @@ -163,7 +158,7 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { // It's an error if the job completes successfully even though no committer was authorized, // so throw an exception if the job was allowed to complete. intercept[TimeoutException] { - ThreadUtils.awaitResult(futureAction, 5 seconds) + ThreadUtils.awaitResult(futureAction, 5.seconds) } assert(tempDir.list().size === 0) } @@ -268,8 +263,8 @@ class OutputCommitCoordinatorSuite extends SparkFunSuite with BeforeAndAfter { assert(retriedStage.size === 1) assert(sc.dagScheduler.outputCommitCoordinator.isEmpty) verify(sc.env.outputCommitCoordinator, times(2)) - .stageStart(Matchers.eq(retriedStage.head), Matchers.any()) - verify(sc.env.outputCommitCoordinator).stageEnd(Matchers.eq(retriedStage.head)) + .stageStart(meq(retriedStage.head), any()) + verify(sc.env.outputCommitCoordinator).stageEnd(meq(retriedStage.head)) } } 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 5bd3955f5adbb..b953add9d58cb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/PoolSuite.scala @@ -21,6 +21,7 @@ import java.io.FileNotFoundException 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.scheduler.SchedulingMode._ /** @@ -31,7 +32,6 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { val LOCAL = "local" val APP_NAME = "PoolSuite" - val SCHEDULER_ALLOCATION_FILE_PROPERTY = "spark.scheduler.allocation.file" val TEST_POOL = "testPool" def createTaskSetManager(stageId: Int, numTasks: Int, taskScheduler: TaskSchedulerImpl) @@ -80,7 +80,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { */ test("Fair Scheduler Test") { val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() - val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE, xmlPath) sc = new SparkContext(LOCAL, APP_NAME, conf) val taskScheduler = new TaskSchedulerImpl(sc) @@ -182,7 +182,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { test("SPARK-17663: FairSchedulableBuilder sets default values for blank or invalid datas") { val xmlPath = getClass.getClassLoader.getResource("fairscheduler-with-invalid-data.xml") .getFile() - val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE, xmlPath) val rootPool = new Pool("", FAIR, 0, 0) val schedulableBuilder = new FairSchedulableBuilder(rootPool, conf) @@ -218,7 +218,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { val taskSetManager1 = createTaskSetManager(stageId = 1, numTasks = 1, taskScheduler) val properties = new Properties() - properties.setProperty("spark.scheduler.pool", TEST_POOL) + properties.setProperty(SparkContext.SPARK_SCHEDULER_POOL, TEST_POOL) // When FIFO Scheduler is used and task sets are submitted, they should be added to // the root pool, and no additional pools should be created @@ -296,7 +296,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { test("Fair Scheduler should build fair scheduler when " + "valid spark.scheduler.allocation.file property is set") { val xmlPath = getClass.getClassLoader.getResource("fairscheduler-with-valid-data.xml").getFile() - val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, xmlPath) + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE, xmlPath) sc = new SparkContext(LOCAL, APP_NAME, conf) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) @@ -326,7 +326,7 @@ class PoolSuite extends SparkFunSuite with LocalSparkContext { test("Fair Scheduler should throw FileNotFoundException " + "when invalid spark.scheduler.allocation.file property is set") { - val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE_PROPERTY, "INVALID_FILE_PATH") + val conf = new SparkConf().set(SCHEDULER_ALLOCATION_FILE, "INVALID_FILE_PATH") sc = new SparkContext(LOCAL, APP_NAME, conf) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala index d1113c7e0b103..7d0712b353d62 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/ReplayListenerSuite.scala @@ -50,15 +50,15 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Simple replay") { val logFilePath = getFilePath(testDir, "events.txt") val fstream = fileSystem.create(logFilePath) - val writer = new PrintWriter(fstream) val applicationStart = SparkListenerApplicationStart("Greatest App (N)ever", None, 125L, "Mickey", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - // scalastyle:off println - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) - // scalastyle:on println - writer.close() + Utils.tryWithResource(new PrintWriter(fstream)) { writer => + // scalastyle:off println + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + // scalastyle:on println + } val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) @@ -132,16 +132,16 @@ class ReplayListenerSuite extends SparkFunSuite with BeforeAndAfter with LocalSp test("Replay incompatible event log") { val logFilePath = getFilePath(testDir, "incompatible.txt") val fstream = fileSystem.create(logFilePath) - val writer = new PrintWriter(fstream) val applicationStart = SparkListenerApplicationStart("Incompatible App", None, 125L, "UserUsingIncompatibleVersion", None) val applicationEnd = SparkListenerApplicationEnd(1000L) - // scalastyle:off println - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) - writer.println("""{"Event":"UnrecognizedEventOnlyForTest","Timestamp":1477593059313}""") - writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) - // scalastyle:on println - writer.close() + Utils.tryWithResource(new PrintWriter(fstream)) { writer => + // scalastyle:off println + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationStart)))) + writer.println("""{"Event":"UnrecognizedEventOnlyForTest","Timestamp":1477593059313}""") + writer.println(compact(render(JsonProtocol.sparkEventToJson(applicationEnd)))) + // scalastyle:on println + } val conf = EventLoggingListenerSuite.getLoggingConf(logFilePath) val logData = fileSystem.open(logFilePath) 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 ff0f99b5c94d0..577d77e890d78 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SchedulerIntegrationSuite.scala @@ -23,7 +23,6 @@ import java.util.concurrent.atomic.{AtomicBoolean, AtomicReference} import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Future import scala.concurrent.duration.{Duration, SECONDS} -import scala.language.existentials import scala.reflect.ClassTag import org.scalactic.TripleEquals @@ -34,6 +33,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ 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.util.{CallSite, ThreadUtils, Utils} @@ -51,7 +51,7 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa var taskScheduler: TestTaskScheduler = null var scheduler: DAGScheduler = null var backend: T = _ - // Even though the tests aren't doing much, occassionally we see flakiness from pauses over + // Even though the tests aren't doing much, occasionally we see flakiness from pauses over // a second (probably from GC?) so we leave a long timeout in here val duration = Duration(10, SECONDS) @@ -169,7 +169,7 @@ abstract class SchedulerIntegrationSuite[T <: MockBackend: ClassTag] extends Spa // and notifies the job waiter before our original thread in the task scheduler finishes // handling the event and marks the taskset as complete. So its ok if we need to wait a // *little* bit longer for the original taskscheduler thread to finish up to deal w/ the race. - eventually(timeout(1 second), interval(10 millis)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(taskScheduler.runningTaskSets.isEmpty) } assert(!backend.hasTasks) @@ -290,17 +290,17 @@ private[spark] abstract class MockBackend( // Periodically revive offers to allow delay scheduling to work private val reviveThread = ThreadUtils.newDaemonSingleThreadScheduledExecutor("driver-revive-thread") - private val reviveIntervalMs = conf.getTimeAsMs("spark.scheduler.revive.interval", "10ms") + private val reviveIntervalMs = conf.get(SCHEDULER_REVIVE_INTERVAL).getOrElse(10L) /** * Test backends should call this to get a task that has been assigned to them by the scheduler. * Each task should be responded to with either [[taskSuccess]] or [[taskFailed]]. */ - def beginTask(): (TaskDescription, Task[_]) = { + def beginTask[T](): (TaskDescription, Task[T]) = { synchronized { val toRun = assignedTasksWaitingToRun.remove(assignedTasksWaitingToRun.size - 1) runningTasks += toRun._1.taskId - toRun + toRun.asInstanceOf[(TaskDescription, Task[T])] } } @@ -355,13 +355,9 @@ private[spark] abstract class MockBackend( assignedTasksWaitingToRun.nonEmpty } - override def start(): Unit = { - reviveThread.scheduleAtFixedRate(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - reviveOffers() - } - }, 0, reviveIntervalMs, TimeUnit.MILLISECONDS) - } + override def start(): Unit = + reviveThread.scheduleAtFixedRate(() => Utils.tryLogNonFatalError { reviveOffers() }, + 0, reviveIntervalMs, TimeUnit.MILLISECONDS) override def stop(): Unit = { reviveThread.shutdown() @@ -592,7 +588,7 @@ class BasicSchedulerIntegrationSuite extends SchedulerIntegrationSuite[SingleCor backend.taskSuccess(taskDescription, 4321 + partition) } } - withBackend(runBackend _) { + withBackend(() => runBackend()) { val jobFuture = submit(d, (0 until 30).toArray) awaitJobTermination(jobFuture, duration) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 6ffd1e84f7adb..a7869d3251ebc 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.Matchers import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{ResetSystemProperties, RpcUtils} @@ -358,7 +359,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match } test("onTaskGettingResult() called when result fetched remotely") { - val conf = new SparkConf().set("spark.rpc.message.maxSize", "1") + val conf = new SparkConf().set(RPC_MESSAGE_MAX_SIZE, 1) sc = new SparkContext("local", "SparkListenerSuite", conf) val listener = new SaveTaskEvents sc.addSparkListener(listener) @@ -534,7 +535,7 @@ class SparkListenerSuite extends SparkFunSuite with LocalSparkContext with Match /** * Assert that the given list of numbers has an average that is greater than zero. */ - private def checkNonZeroAvg(m: Traversable[Long], msg: String) { + private def checkNonZeroAvg(m: Iterable[Long], msg: String) { assert(m.sum / m.size.toDouble > 0.0, msg) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala index 123f7f49d21b5..a6576e0d1c520 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerWithClusterSuite.scala @@ -19,25 +19,23 @@ package org.apache.spark.scheduler import scala.collection.mutable -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} - import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite, TestUtils} import org.apache.spark.scheduler.cluster.ExecutorInfo /** * Unit tests for SparkListener that require a local cluster. */ -class SparkListenerWithClusterSuite extends SparkFunSuite with LocalSparkContext - with BeforeAndAfter with BeforeAndAfterAll { +class SparkListenerWithClusterSuite extends SparkFunSuite with LocalSparkContext { /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 - before { + override def beforeEach(): Unit = { + super.beforeEach() sc = new SparkContext("local-cluster[2,1,1024]", "SparkListenerSuite") } - test("SparkListener sends executor added message") { + testRetry("SparkListener sends executor added message") { val listener = new SaveExecutorInfo sc.addSparkListener(listener) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala index aa9c36c0aaacb..c16b552d20891 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskContextSuite.scala @@ -19,12 +19,13 @@ package org.apache.spark.scheduler import java.util.Properties -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{Executor, TaskMetrics, TaskMetricsSuite} +import org.apache.spark.internal.config.METRICS_CONF import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.metrics.source.JvmSource import org.apache.spark.network.util.JavaUtils @@ -37,7 +38,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark test("provide metrics sources") { val filePath = getClass.getClassLoader.getResource("test_metrics_config.properties").getFile val conf = new SparkConf(loadDefaults = false) - .set("spark.metrics.conf", filePath) + .set(METRICS_CONF, filePath) sc = new SparkContext("local", "test", conf) val rdd = sc.makeRDD(1 to 1) val result = sc.runJob(rdd, (tc: TaskContext, it: Iterator[Int]) => { @@ -69,9 +70,9 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, closureSerializer.serialize(TaskMetrics.registered).array()) intercept[RuntimeException] { - task.run(0, 0, null) + task.run(0, 0, null, null) } - assert(TaskContextSuite.completed === true) + assert(TaskContextSuite.completed) } test("calls TaskFailureListeners after failure") { @@ -91,7 +92,7 @@ class TaskContextSuite extends SparkFunSuite with BeforeAndAfter with LocalSpark 0, 0, taskBinary, rdd.partitions(0), Seq.empty, 0, new Properties, closureSerializer.serialize(TaskMetrics.registered).array()) intercept[RuntimeException] { - task.run(0, 0, null) + task.run(0, 0, null, null) } assert(TaskContextSuite.lastError.getMessage == "damn error") } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala index ba62eec0522db..5839532f11666 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskDescriptionSuite.scala @@ -24,6 +24,8 @@ import java.util.Properties import scala.collection.mutable.HashMap import org.apache.spark.SparkFunSuite +import org.apache.spark.resource.ResourceInformation +import org.apache.spark.resource.ResourceUtils.GPU class TaskDescriptionSuite extends SparkFunSuite { test("encoding and then decoding a TaskDescription results in the same TaskDescription") { @@ -53,6 +55,9 @@ class TaskDescriptionSuite extends SparkFunSuite { } } + val originalResources = + Map(GPU -> new ResourceInformation(GPU, Array("1", "2", "3"))) + // Create a dummy byte buffer for the task. val taskBuffer = ByteBuffer.wrap(Array[Byte](1, 2, 3, 4)) @@ -66,6 +71,7 @@ class TaskDescriptionSuite extends SparkFunSuite { originalFiles, originalJars, originalProperties, + originalResources, taskBuffer ) @@ -82,6 +88,17 @@ class TaskDescriptionSuite extends SparkFunSuite { assert(decodedTaskDescription.addedFiles.equals(originalFiles)) assert(decodedTaskDescription.addedJars.equals(originalJars)) assert(decodedTaskDescription.properties.equals(originalTaskDescription.properties)) + assert(equalResources(decodedTaskDescription.resources, originalTaskDescription.resources)) assert(decodedTaskDescription.serializedTask.equals(taskBuffer)) + + def equalResources(original: Map[String, ResourceInformation], + target: Map[String, ResourceInformation]): Boolean = { + original.size == target.size && original.forall { case (name, info) => + target.get(name).exists { targetInfo => + info.name.equals(targetInfo.name) && + info.addresses.sameElements(targetInfo.addresses) + } + } + } } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index 1bddba8f6c82b..ae464352da440 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -23,18 +23,18 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ -import scala.language.postfixOps import scala.util.control.NonFatal import com.google.common.util.concurrent.MoreExecutors import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{any, anyLong} +import org.mockito.ArgumentMatchers.{any, anyLong} import org.mockito.Mockito.{spy, times, verify} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.TestUtils.JavaSourceFromString +import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE import org.apache.spark.storage.TaskResultBlockId import org.apache.spark.util.{MutableURLClassLoader, RpcUtils, Utils} @@ -57,18 +57,18 @@ private class ResultDeletingTaskResultGetter(sparkEnv: SparkEnv, scheduler: Task // Only remove the result once, since we'd like to test the case where the task eventually // succeeds. serializer.get().deserialize[TaskResult[_]](serializedData) match { - case IndirectTaskResult(blockId, size) => + case IndirectTaskResult(blockId, _) => sparkEnv.blockManager.master.removeBlock(blockId) // removeBlock is asynchronous. Need to wait it's removed successfully try { - eventually(timeout(3 seconds), interval(200 milliseconds)) { + eventually(timeout(3.seconds), interval(200.milliseconds)) { assert(!sparkEnv.blockManager.master.contains(blockId)) } removeBlockSuccessfully = true } catch { case NonFatal(e) => removeBlockSuccessfully = false } - case directResult: DirectTaskResult[_] => + case _: DirectTaskResult[_] => taskSetManager.abort("Internal error: expect only indirect results") } serializedData.rewind() @@ -110,7 +110,7 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local // Set the RPC message size to be as small as possible (it must be an integer, so 1 is as small // as we can make it) so the tests don't take too long. - def conf: SparkConf = new SparkConf().set("spark.rpc.message.maxSize", "1") + def conf: SparkConf = new SparkConf().set(RPC_MESSAGE_MAX_SIZE, 1) test("handling results smaller than max RPC message size") { sc = new SparkContext("local", "test", conf) @@ -194,7 +194,7 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local // jar. sc = new SparkContext("local", "test", conf) val rdd = sc.parallelize(Seq(1), 1).map { _ => - val exc = excClass.newInstance().asInstanceOf[Exception] + val exc = excClass.getConstructor().newInstance().asInstanceOf[Exception] throw exc } @@ -265,7 +265,9 @@ class TaskResultGetterSuite extends SparkFunSuite with BeforeAndAfter with Local private class UndeserializableException extends Exception { private def readObject(in: ObjectInputStream): Unit = { + // scalastyle:off throwerror throw new NoClassDefFoundError() + // scalastyle:on throwerror } } 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 9e1d13e369ad9..d1b16163a9f0f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -19,16 +19,20 @@ package org.apache.spark.scheduler import java.nio.ByteBuffer -import scala.collection.mutable.HashMap +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.concurrent.duration._ -import org.mockito.Matchers.{anyInt, anyObject, anyString, eq => meq} +import org.mockito.ArgumentMatchers.{any, anyInt, anyString, eq => meq} import org.mockito.Mockito.{atLeast, atMost, never, spy, times, verify, when} import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.util.ManualClock class FakeSchedulerBackend extends SchedulerBackend { @@ -40,7 +44,7 @@ class FakeSchedulerBackend extends SchedulerBackend { } class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with BeforeAndAfterEach - with Logging with MockitoSugar { + with Logging with MockitoSugar with Eventually { var failedTaskSetException: Option[Throwable] = None var failedTaskSetReason: String = null @@ -75,20 +79,30 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } def setupScheduler(confs: (String, String)*): TaskSchedulerImpl = { - val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") + setupSchedulerWithMaster("local", confs: _*) + } + + def setupScheduler(numCores: Int, confs: (String, String)*): TaskSchedulerImpl = { + setupSchedulerWithMaster(s"local[$numCores]", confs: _*) + } + + def setupSchedulerWithMaster(master: String, confs: (String, String)*): TaskSchedulerImpl = { + val conf = new SparkConf().setMaster(master).setAppName("TaskSchedulerImplSuite") confs.foreach { case (k, v) => conf.set(k, v) } sc = new SparkContext(conf) taskScheduler = new TaskSchedulerImpl(sc) setupHelper() } - def setupSchedulerWithMockTaskSetBlacklist(): TaskSchedulerImpl = { + def setupSchedulerWithMockTaskSetBlacklist(confs: (String, String)*): TaskSchedulerImpl = { blacklist = mock[BlacklistTracker] val conf = new SparkConf().setMaster("local").setAppName("TaskSchedulerImplSuite") conf.set(config.BLACKLIST_ENABLED, true) + confs.foreach { case (k, v) => conf.set(k, v) } + sc = new SparkContext(conf) taskScheduler = - new TaskSchedulerImpl(sc, sc.conf.getInt("spark.task.maxFailures", 4)) { + new TaskSchedulerImpl(sc, sc.conf.get(config.TASK_MAX_FAILURES)) { override def createTaskSetManager(taskSet: TaskSet, maxFailures: Int): TaskSetManager = { val tsm = super.createTaskSetManager(taskSet, maxFailures) // we need to create a spied tsm just so we can set the TaskSetBlacklist @@ -151,7 +165,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("Scheduler correctly accounts for multiple CPUs per task") { val taskCpus = 2 - val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + config.CPUS_PER_TASK.key -> taskCpus.toString) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = IndexedSeq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) @@ -181,7 +197,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("Scheduler does not crash when tasks are not serializable") { val taskCpus = 2 - val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + 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) @@ -197,28 +215,39 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B // Even if one of the task sets has not-serializable tasks, the other task set should // still be processed without error taskScheduler.submitTasks(FakeTask.createTaskSet(1)) - taskScheduler.submitTasks(taskSet) + val taskSet2 = new TaskSet( + Array(new NotSerializableFakeTask(1, 0), new NotSerializableFakeTask(0, 1)), 1, 0, 0, null) + taskScheduler.submitTasks(taskSet2) taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten assert(taskDescriptions.map(_.executorId) === Seq("executor0")) } - test("refuse to schedule concurrent attempts for the same stage (SPARK-8103)") { + test("concurrent attempts for the same stage only have one active taskset") { val taskScheduler = setupScheduler() + def isTasksetZombie(taskset: TaskSet): Boolean = { + taskScheduler.taskSetManagerForAttempt(taskset.stageId, taskset.stageAttemptId).get.isZombie + } + val attempt1 = FakeTask.createTaskSet(1, 0) - val attempt2 = FakeTask.createTaskSet(1, 1) taskScheduler.submitTasks(attempt1) - intercept[IllegalStateException] { taskScheduler.submitTasks(attempt2) } + // The first submitted taskset is active + assert(!isTasksetZombie(attempt1)) - // OK to submit multiple if previous attempts are all zombie - taskScheduler.taskSetManagerForAttempt(attempt1.stageId, attempt1.stageAttemptId) - .get.isZombie = true + val attempt2 = FakeTask.createTaskSet(1, 1) taskScheduler.submitTasks(attempt2) + // The first submitted taskset is zombie now + assert(isTasksetZombie(attempt1)) + // The newly submitted taskset is active + assert(!isTasksetZombie(attempt2)) + val attempt3 = FakeTask.createTaskSet(1, 2) - intercept[IllegalStateException] { taskScheduler.submitTasks(attempt3) } - taskScheduler.taskSetManagerForAttempt(attempt2.stageId, attempt2.stageAttemptId) - .get.isZombie = true taskScheduler.submitTasks(attempt3) - assert(!failedTaskSet) + // The first submitted taskset remains zombie + assert(isTasksetZombie(attempt1)) + // The second submitted taskset is zombie now + assert(isTasksetZombie(attempt2)) + // The newly submitted taskset is active + assert(!isTasksetZombie(attempt3)) } test("don't schedule more tasks after a taskset is zombie") { @@ -426,7 +455,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B verify(blacklist, never).updateBlacklistForSuccessfulTaskSet( stageId = meq(2), stageAttemptId = anyInt(), - failuresByExec = anyObject()) + failuresByExec = any()) } test("scheduled tasks obey node and executor blacklists") { @@ -466,7 +495,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } } - test("abort stage when all executors are blacklisted") { + test("abort stage when all executors are blacklisted and we cannot acquire new executor") { taskScheduler = setupSchedulerWithMockTaskSetBlacklist() val taskSet = FakeTask.createTaskSet(numTasks = 10, stageAttemptId = 0) taskScheduler.submitTasks(taskSet) @@ -500,7 +529,186 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B WorkerOffer("executor3", "host1", 2) )).flatten.size === 0) assert(tsm.isZombie) - verify(tsm).abort(anyString(), anyObject()) + verify(tsm).abort(anyString(), any()) + } + + test("SPARK-22148 abort timer should kick in when task is completely blacklisted & no new " + + "executor can be acquired") { + // set the abort timer to fail immediately + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0") + + // We have only 1 task remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = firstTaskAttempts.find(_.executorId == "executor0").get + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite + // Reason being - handleFailedTask is run by an executor service and there is a momentary delay + // before it is launched and this fails the assertion check. + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to kick in immediately + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + // Wait for the abort timer to kick in. Even though we configure the timeout to be 0, there is a + // slight delay as the abort timer is launched in a separate thread. + eventually(timeout(500.milliseconds)) { + assert(tsm.isZombie) + } + } + + test("SPARK-22148 try to acquire a new executor when task is unschedulable with 1 executor") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "10") + + // We have only 1 task remaining with 1 executor + val taskSet = FakeTask.createTaskSet(numTasks = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = firstTaskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + // we explicitly call the handleFailedTask method here to avoid adding a sleep in the test suite + // Reason being - handleFailedTask is run by an executor service and there is a momentary delay + // before it is launched and this fails the assertion check. + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer to expire if no new executors could be acquired. We kill the existing idle blacklisted + // executor and try to acquire a new one. + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm)) + assert(!tsm.isZombie) + + // Offer a new executor which should be accepted + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host0", 1) + )).flatten.size === 1) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + assert(!tsm.isZombie) + } + + // This is to test a scenario where we have two taskSets completely blacklisted and on acquiring + // a new executor we don't want the abort timer for the second taskSet to expire and abort the job + test("SPARK-22148 abort timer should clear unschedulableTaskSetToExpiryTime for all TaskSets") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist() + + // We have 2 taskSets with 1 task remaining in each with 1 executor completely blacklisted + val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet1) + val taskSet2 = FakeTask.createTaskSet(numTasks = 1, stageId = 1, stageAttemptId = 0) + taskScheduler.submitTasks(taskSet2) + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + val firstTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + // Fail the running task + val failedTask = firstTaskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer. We will schedule the task from the second taskSet. Since a task was scheduled + // we do not kick off the abort timer for taskSet1 + val secondTaskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + val tsm2 = stageToMockTaskSetManager(1) + val failedTask2 = secondTaskAttempts.head + taskScheduler.statusUpdate(failedTask2.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm2.handleFailedTask(failedTask2.taskId, TaskState.FAILED, UnknownReason) + when(tsm2.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask2.index)).thenReturn(true) + + // make an offer on the blacklisted executor. We won't schedule anything, and set the abort + // timer for taskSet1 and taskSet2 + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten.size === 0) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm)) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.contains(tsm2)) + assert(taskScheduler.unschedulableTaskSetToExpiryTime.size == 2) + + // Offer a new executor which should be accepted + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host1", 1) + )).flatten.size === 1) + + // Check if all the taskSets are cleared + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + assert(!tsm.isZombie) + } + + // this test is to check that we don't abort a taskSet which is not being scheduled on other + // executors as it is waiting on locality timeout and not being aborted because it is still not + // completely blacklisted. + test("SPARK-22148 Ensure we don't abort the taskSet if we haven't been completely blacklisted") { + taskScheduler = setupSchedulerWithMockTaskSetBlacklist( + config.UNSCHEDULABLE_TASKSET_TIMEOUT.key -> "0", + // This is to avoid any potential flakiness in the test because of large pauses in jenkins + config.LOCALITY_WAIT.key -> "30s" + ) + + val preferredLocation = Seq(ExecutorCacheTaskLocation("host0", "executor0")) + val taskSet1 = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0, + preferredLocation) + taskScheduler.submitTasks(taskSet1) + + val tsm = stageToMockTaskSetManager(0) + + // submit an offer with one executor + var taskAttempts = taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor0", "host0", 1) + )).flatten + + // Fail the running task + val failedTask = taskAttempts.head + taskScheduler.statusUpdate(failedTask.taskId, TaskState.FAILED, ByteBuffer.allocate(0)) + tsm.handleFailedTask(failedTask.taskId, TaskState.FAILED, UnknownReason) + when(tsm.taskSetBlacklistHelperOpt.get.isExecutorBlacklistedForTask( + "executor0", failedTask.index)).thenReturn(true) + + // make an offer but we won't schedule anything yet as scheduler locality is still PROCESS_LOCAL + assert(taskScheduler.resourceOffers(IndexedSeq( + WorkerOffer("executor1", "host0", 1) + )).flatten.isEmpty) + + assert(taskScheduler.unschedulableTaskSetToExpiryTime.isEmpty) + + assert(!tsm.isZombie) } /** @@ -828,7 +1036,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("Locality should be used for bulk offers even with delay scheduling off") { val conf = new SparkConf() - .set("spark.locality.wait", "0") + .set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) // we create a manual clock just so we can be sure the clock doesn't advance at all in this test val clock = new ManualClock() @@ -875,7 +1083,7 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("With delay scheduling off, tasks can be run at any locality level immediately") { val conf = new SparkConf() - .set("spark.locality.wait", "0") + .set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "TaskSchedulerImplSuite", conf) // we create a manual clock just so we can be sure the clock doesn't advance at all in this test @@ -919,113 +1127,11 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B } } - test("Completions in zombie tasksets update status of non-zombie taskset") { - val taskScheduler = setupSchedulerWithMockTaskSetBlacklist() - val valueSer = SparkEnv.get.serializer.newInstance() - - def completeTaskSuccessfully(tsm: TaskSetManager, partition: Int): Unit = { - val indexInTsm = tsm.partitionToIndex(partition) - val matchingTaskInfo = tsm.taskAttempts.flatten.filter(_.index == indexInTsm).head - val result = new DirectTaskResult[Int](valueSer.serialize(1), Seq()) - tsm.handleSuccessfulTask(matchingTaskInfo.taskId, result) - } - - // Submit a task set, have it fail with a fetch failed, and then re-submit the task attempt, - // two times, so we have three active task sets for one stage. (For this to really happen, - // you'd need the previous stage to also get restarted, and then succeed, in between each - // attempt, but that happens outside what we're mocking here.) - val zombieAttempts = (0 until 2).map { stageAttempt => - val attempt = FakeTask.createTaskSet(10, stageAttemptId = stageAttempt) - taskScheduler.submitTasks(attempt) - val tsm = taskScheduler.taskSetManagerForAttempt(0, stageAttempt).get - val offers = (0 until 10).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) } - taskScheduler.resourceOffers(offers) - assert(tsm.runningTasks === 10) - // fail attempt - tsm.handleFailedTask(tsm.taskAttempts.head.head.taskId, TaskState.FAILED, - FetchFailed(null, 0, 0, 0, "fetch failed")) - // the attempt is a zombie, but the tasks are still running (this could be true even if - // we actively killed those tasks, as killing is best-effort) - assert(tsm.isZombie) - assert(tsm.runningTasks === 9) - tsm - } - - // we've now got 2 zombie attempts, each with 9 tasks still active. Submit the 3rd attempt for - // the stage, but this time with insufficient resources so not all tasks are active. - - val finalAttempt = FakeTask.createTaskSet(10, stageAttemptId = 2) - taskScheduler.submitTasks(finalAttempt) - val finalTsm = taskScheduler.taskSetManagerForAttempt(0, 2).get - val offers = (0 until 5).map{ idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) } - val finalAttemptLaunchedPartitions = taskScheduler.resourceOffers(offers).flatten.map { task => - finalAttempt.tasks(task.index).partitionId - }.toSet - assert(finalTsm.runningTasks === 5) - assert(!finalTsm.isZombie) - - // We simulate late completions from our zombie tasksets, corresponding to all the pending - // partitions in our final attempt. This means we're only waiting on the tasks we've already - // launched. - val finalAttemptPendingPartitions = (0 until 10).toSet.diff(finalAttemptLaunchedPartitions) - finalAttemptPendingPartitions.foreach { partition => - completeTaskSuccessfully(zombieAttempts(0), partition) - } - - // If there is another resource offer, we shouldn't run anything. Though our final attempt - // used to have pending tasks, now those tasks have been completed by zombie attempts. The - // remaining tasks to compute are already active in the non-zombie attempt. - assert( - taskScheduler.resourceOffers(IndexedSeq(WorkerOffer("exec-1", "host-1", 1))).flatten.isEmpty) - - val remainingTasks = finalAttemptLaunchedPartitions.toIndexedSeq.sorted - - // finally, if we finish the remaining partitions from a mix of tasksets, all attempts should be - // marked as zombie. - // for each of the remaining tasks, find the tasksets with an active copy of the task, and - // finish the task. - remainingTasks.foreach { partition => - val tsm = if (partition == 0) { - // we failed this task on both zombie attempts, this one is only present in the latest - // taskset - finalTsm - } else { - // should be active in every taskset. We choose a zombie taskset just to make sure that - // we transition the active taskset correctly even if the final completion comes - // from a zombie. - zombieAttempts(partition % 2) - } - completeTaskSuccessfully(tsm, partition) - } - - assert(finalTsm.isZombie) - - // no taskset has completed all of its tasks, so no updates to the blacklist tracker yet - verify(blacklist, never).updateBlacklistForSuccessfulTaskSet(anyInt(), anyInt(), anyObject()) - - // finally, lets complete all the tasks. We simulate failures in attempt 1, but everything - // else succeeds, to make sure we get the right updates to the blacklist in all cases. - (zombieAttempts ++ Seq(finalTsm)).foreach { tsm => - val stageAttempt = tsm.taskSet.stageAttemptId - tsm.runningTasksSet.foreach { index => - if (stageAttempt == 1) { - tsm.handleFailedTask(tsm.taskInfos(index).taskId, TaskState.FAILED, TaskResultLost) - } else { - val result = new DirectTaskResult[Int](valueSer.serialize(1), Seq()) - tsm.handleSuccessfulTask(tsm.taskInfos(index).taskId, result) - } - } - - // we update the blacklist for the stage attempts with all successful tasks. Even though - // some tasksets had failures, we still consider them all successful from a blacklisting - // perspective, as the failures weren't from a problem w/ the tasks themselves. - verify(blacklist).updateBlacklistForSuccessfulTaskSet(meq(0), meq(stageAttempt), anyObject()) - } - } - test("don't schedule for a barrier taskSet if available slots are less than pending tasks") { val taskCpus = 2 - val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + config.CPUS_PER_TASK.key -> taskCpus.toString) val numFreeCores = 3 val workerOffers = IndexedSeq( @@ -1042,7 +1148,9 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B test("schedule tasks for a barrier taskSet if all tasks can be launched together") { val taskCpus = 2 - val taskScheduler = setupScheduler("spark.task.cpus" -> taskCpus.toString) + val taskScheduler = setupSchedulerWithMaster( + s"local[$taskCpus]", + config.CPUS_PER_TASK.key -> taskCpus.toString) val numFreeCores = 3 val workerOffers = IndexedSeq( @@ -1136,4 +1244,36 @@ class TaskSchedulerImplSuite extends SparkFunSuite with LocalSparkContext with B tsm.handleFailedTask(tsm.taskAttempts.head.head.taskId, TaskState.FAILED, TaskKilled("test")) assert(tsm.isZombie) } + + test("Scheduler correctly accounts for GPUs per task") { + 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 taskSet = FakeTask.createTaskSet(3) + + val numFreeCores = 2 + val resources = Map(GPU -> ArrayBuffer("0", "1", "2", "3")) + val singleCoreWorkerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, resources)) + val zeroGpuWorkerOffers = + IndexedSeq(new WorkerOffer("executor0", "host0", numFreeCores, None, Map.empty)) + taskScheduler.submitTasks(taskSet) + // WorkerOffer doesn't contain GPU resource, don't launch any task. + var taskDescriptions = taskScheduler.resourceOffers(zeroGpuWorkerOffers).flatten + assert(0 === taskDescriptions.length) + assert(!failedTaskSet) + // Launch tasks on executor that satisfies resource requirements. + taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten + assert(2 === taskDescriptions.length) + assert(!failedTaskSet) + assert(ArrayBuffer("0") === taskDescriptions(0).resources.get(GPU).get.addresses) + assert(ArrayBuffer("1") === taskDescriptions(1).resources.get(GPU).get.addresses) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala index 6e2709dbe1e8b..b3bc76687ce1b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetBlacklistSuite.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.scheduler -import org.mockito.Matchers.isA +import org.mockito.ArgumentMatchers.isA import org.mockito.Mockito.{never, verify} import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar 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 d264adaef90a5..da566dd82bcec 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -22,17 +22,18 @@ import java.util.{Properties, Random} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import org.mockito.Matchers.{any, anyInt, anyString} -import org.mockito.Mockito.{mock, never, spy, times, verify, when} +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyInt, anyString} +import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.apache.spark._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.serializer.SerializerInstance import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AccumulatorV2, ManualClock, Utils} +import org.apache.spark.util.{AccumulatorV2, ManualClock} class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) extends DAGScheduler(sc) { @@ -69,17 +70,27 @@ class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) // Get the rack for a given host object FakeRackUtil { private val hostToRack = new mutable.HashMap[String, String]() + var numBatchInvocation = 0 + var numSingleHostInvocation = 0 def cleanUp() { hostToRack.clear() + numBatchInvocation = 0 + numSingleHostInvocation = 0 } def assignHostToRack(host: String, rack: String) { hostToRack(host) = rack } - def getRackForHost(host: String): Option[String] = { - hostToRack.get(host) + def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { + assert(hosts.toSet.size == hosts.size) // no dups in hosts + if (hosts.nonEmpty && hosts.length != 1) { + numBatchInvocation += 1 + } else if (hosts.length == 1) { + numSingleHostInvocation += 1 + } + hosts.map(hostToRack.get(_)) } } @@ -99,6 +110,9 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val speculativeTasks = new ArrayBuffer[Int] val executors = new mutable.HashMap[String, String] + + // this must be initialized before addExecutor + override val defaultRackValue: Option[String] = Some("default") for ((execId, host) <- liveExecutors) { addExecutor(execId, host) } @@ -144,8 +158,9 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex } } - - override def getRackForHost(value: String): Option[String] = FakeRackUtil.getRackForHost(value) + override def getRacksForHosts(hosts: Seq[String]): Seq[Option[String]] = { + FakeRackUtil.getRacksForHosts(hosts) + } } /** @@ -153,7 +168,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex */ class LargeTask(stageId: Int) extends Task[Array[Byte]](stageId, 0, 0) { - val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KB * 1024) + val randomBuffer = new Array[Byte](TaskSetManager.TASK_SIZE_TO_WARN_KIB * 1024) val random = new Random(0) random.nextBytes(randomBuffer) @@ -166,7 +181,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg private val conf = new SparkConf - val LOCALITY_WAIT_MS = conf.getTimeAsMs("spark.locality.wait", "3s") + val LOCALITY_WAIT_MS = conf.get(config.LOCALITY_WAIT) val MAX_TASK_FAILURES = 4 var sched: FakeTaskScheduler = null @@ -429,7 +444,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg set(config.BLACKLIST_ENABLED, true). set(config.BLACKLIST_TIMEOUT_CONF, rescheduleDelay). // don't wait to jump locality levels in this test - set("spark.locality.wait", "0") + set(config.LOCALITY_WAIT.key, "0") sc = new SparkContext("local", "test", conf) // two executors on same host, one on different. @@ -655,7 +670,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } test("abort the job if total size of results is too large") { - val conf = new SparkConf().set("spark.driver.maxResultSize", "2m") + val conf = new SparkConf().set(config.MAX_RESULT_SIZE.key, "2m") sc = new SparkContext("local", "test", conf) def genBytes(size: Int): (Int) => Array[Byte] = { (x: Int) => @@ -680,7 +695,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg } test("[SPARK-13931] taskSetManager should not send Resubmitted tasks after being a zombie") { - val conf = new SparkConf().set("spark.speculation", "true") + val conf = new SparkConf().set(config.SPECULATION_ENABLED, true) sc = new SparkContext("local", "test", conf) sched = new FakeTaskScheduler(sc, ("execA", "host1"), ("execB", "host2")) @@ -736,7 +751,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg // Complete one copy of the task, which should result in the task set manager // being marked as a zombie, because at least one copy of its only task has completed. manager.handleSuccessfulTask(task1.taskId, directTaskResult) - assert(manager.isZombie === true) + assert(manager.isZombie) assert(resubmittedTasks === 0) assert(manager.runningTasks === 1) @@ -747,12 +762,12 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg test("[SPARK-22074] Task killed by other attempt task should not be resubmitted") { - val conf = new SparkConf().set("spark.speculation", "true") + val conf = new SparkConf().set(config.SPECULATION_ENABLED, true) sc = new SparkContext("local", "test", conf) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation.quantile", "0.5") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) + sc.conf.set(config.SPECULATION_QUANTILE, 0.5) + sc.conf.set(config.SPECULATION_ENABLED, true) var killTaskCalled = false sched = new FakeTaskScheduler(sc, ("exec1", "host1"), @@ -1013,8 +1028,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(4) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) + sc.conf.set(config.SPECULATION_ENABLED, true) val clock = new ManualClock() val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => @@ -1070,9 +1085,9 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(5) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation.quantile", "0.6") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) + sc.conf.set(config.SPECULATION_QUANTILE, 0.6) + sc.conf.set(config.SPECULATION_ENABLED, true) val clock = new ManualClock() val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => @@ -1190,11 +1205,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskSet = FakeTask.createTaskSet(numTasks = 1, stageId = 0, stageAttemptId = 0) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = new ManualClock(1)) when(mockDAGScheduler.taskEnded(any(), any(), any(), any(), any())).thenAnswer( - new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - assert(manager.isZombie) - } - }) + (invocationOnMock: InvocationOnMock) => assert(manager.isZombie)) val taskOption = manager.resourceOffer("exec1", "host1", NO_PREF) assert(taskOption.isDefined) // this would fail, inside our mock dag scheduler, if it calls dagScheduler.taskEnded() too soon @@ -1316,13 +1327,11 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val taskDesc = taskSetManagerSpy.resourceOffer(exec, host, TaskLocality.ANY) // Assert the task has been black listed on the executor it was last executed on. - when(taskSetManagerSpy.addPendingTask(anyInt())).thenAnswer( - new Answer[Unit] { - override def answer(invocationOnMock: InvocationOnMock): Unit = { - val task = invocationOnMock.getArgumentAt(0, classOf[Int]) - assert(taskSetManager.taskSetBlacklistHelperOpt.get. - isExecutorBlacklistedForTask(exec, task)) - } + when(taskSetManagerSpy.addPendingTask(anyInt(), anyBoolean())).thenAnswer( + (invocationOnMock: InvocationOnMock) => { + val task: Int = invocationOnMock.getArgument(0) + assert(taskSetManager.taskSetBlacklistHelperOpt.get. + isExecutorBlacklistedForTask(exec, task)) } ) @@ -1330,7 +1339,7 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val e = new ExceptionFailure("a", "b", Array(), "c", None) taskSetManagerSpy.handleFailedTask(taskDesc.get.taskId, TaskState.FAILED, e) - verify(taskSetManagerSpy, times(1)).addPendingTask(anyInt()) + verify(taskSetManagerSpy, times(1)).addPendingTask(0, false) } test("SPARK-21563 context's added jars shouldn't change mid-TaskSet") { @@ -1365,13 +1374,13 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg assert(taskOption4.get.addedJars === addedJarsMidTaskSet) } - test("[SPARK-24677] Avoid NoSuchElementException from MedianHeap") { - val conf = new SparkConf().set("spark.speculation", "true") + test("SPARK-24677: Avoid NoSuchElementException from MedianHeap") { + val conf = new SparkConf().set(config.SPECULATION_ENABLED, true) sc = new SparkContext("local", "test", conf) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation.quantile", "0.1") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) + sc.conf.set(config.SPECULATION_QUANTILE, 0.1) + sc.conf.set(config.SPECULATION_ENABLED, true) sched = new FakeTaskScheduler(sc) sched.initialize(new FakeSchedulerBackend()) @@ -1379,50 +1388,28 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg val dagScheduler = new FakeDAGScheduler(sc, sched) sched.setDAGScheduler(dagScheduler) - val taskSet1 = FakeTask.createTaskSet(10) - val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet1.tasks.map { task => - task.metrics.internalAccums - } + val taskSet = FakeTask.createTaskSet(10) - sched.submitTasks(taskSet1) + sched.submitTasks(taskSet) sched.resourceOffers( - (0 until 10).map { idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }) - - val taskSetManager1 = sched.taskSetManagerForAttempt(0, 0).get + (0 until 8).map { idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }) - // fail fetch - taskSetManager1.handleFailedTask( - taskSetManager1.taskAttempts.head.head.taskId, TaskState.FAILED, - FetchFailed(null, 0, 0, 0, "fetch failed")) - - assert(taskSetManager1.isZombie) - assert(taskSetManager1.runningTasks === 9) - - val taskSet2 = FakeTask.createTaskSet(10, stageAttemptId = 1) - sched.submitTasks(taskSet2) - sched.resourceOffers( - (11 until 20).map { idx => WorkerOffer(s"exec-$idx", s"host-$idx", 1) }) - - // Complete the 2 tasks and leave 8 task in running - for (id <- Set(0, 1)) { - taskSetManager1.handleSuccessfulTask(id, createTaskResult(id, accumUpdatesByTask(id))) - assert(sched.endedTasks(id) === Success) - } - - val taskSetManager2 = sched.taskSetManagerForAttempt(0, 1).get - assert(!taskSetManager2.successfulTaskDurations.isEmpty()) - taskSetManager2.checkSpeculatableTasks(0) + val taskSetManager = sched.taskSetManagerForAttempt(0, 0).get + assert(taskSetManager.runningTasks === 8) + taskSetManager.markPartitionCompleted(8) + assert(taskSetManager.successfulTaskDurations.isEmpty()) + taskSetManager.checkSpeculatableTasks(0) } test("SPARK-24755 Executor loss can cause task to not be resubmitted") { - val conf = new SparkConf().set("spark.speculation", "true") + val conf = new SparkConf().set(config.SPECULATION_ENABLED, true) sc = new SparkContext("local", "test", conf) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) - sc.conf.set("spark.speculation.quantile", "0.5") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_QUANTILE, 0.5) + sc.conf.set(config.SPECULATION_ENABLED, true) var killTaskCalled = false sched = new FakeTaskScheduler(sc, ("exec1", "host1"), @@ -1538,8 +1525,8 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = FakeTask.createTaskSet(4) // Set the speculation multiplier to be 0 so speculative tasks are launched immediately - sc.conf.set("spark.speculation.multiplier", "0.0") - sc.conf.set("spark.speculation", "true") + sc.conf.set(config.SPECULATION_MULTIPLIER, 0.0) + sc.conf.set(config.SPECULATION_ENABLED, true) val clock = new ManualClock() val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) val accumUpdatesByTask: Array[Seq[AccumulatorV2[_, _]]] = taskSet.tasks.map { task => @@ -1602,4 +1589,70 @@ class TaskSetManagerSuite extends SparkFunSuite with LocalSparkContext with Logg verify(sched.dagScheduler).taskEnded(manager.tasks(3), Success, result.value(), result.accumUpdates, info3) } + + test("SPARK-13704 Rack Resolution is done with a batch of de-duped hosts") { + val conf = new SparkConf() + .set(config.LOCALITY_WAIT, 0L) + .set(config.LOCALITY_WAIT_RACK, 1L) + sc = new SparkContext("local", "test", conf) + // Create a cluster with 20 racks, with hosts spread out among them + val execAndHost = (0 to 199).map { i => + FakeRackUtil.assignHostToRack("host" + i, "rack" + (i % 20)) + ("exec" + i, "host" + i) + } + sched = new FakeTaskScheduler(sc, execAndHost: _*) + // make a taskset with preferred locations on the first 100 hosts in our cluster + val locations = new ArrayBuffer[Seq[TaskLocation]]() + for (i <- 0 to 99) { + locations += Seq(TaskLocation("host" + i)) + } + val taskSet = FakeTask.createTaskSet(100, locations: _*) + val clock = new ManualClock + // make sure we only do one rack resolution call, for the entire batch of hosts, as this + // can be expensive. The FakeTaskScheduler calls rack resolution more than the real one + // -- that is outside of the scope of this test, we just want to check the task set manager. + FakeRackUtil.numBatchInvocation = 0 + FakeRackUtil.numSingleHostInvocation = 0 + val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock = clock) + 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) + (0 until 20).foreach { rackIdx => + (0 until 5).foreach { offerIdx => + // if we offer hosts which are not in preferred locations, + // we'll reject them at NODE_LOCAL level, + // 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) + assert(manager.resourceOffer("exec" + hostIdx, "host" + hostIdx, TaskLocality.RACK_LOCAL) + .isDefined) + } + } + // check no more expensive calls to the rack resolution. manager.resourceOffer() will call + // the single-host resolution, but the real rack resolution would have cached all hosts + // by that point. + assert(FakeRackUtil.numBatchInvocation === 1) + } + + test("TaskSetManager allocate resource addresses from available resources") { + import TestUtils._ + + sc = new SparkContext("local", "test") + sc.conf.set(TASK_GPU_ID.amountConf, "2") + sched = new FakeTaskScheduler(sc, ("exec1", "host1")) + 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) + 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")) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala new file mode 100644 index 0000000000000..e11ee97469b00 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/scheduler/dynalloc/ExecutorMonitorSuite.scala @@ -0,0 +1,415 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.scheduler.dynalloc + +import java.util.concurrent.TimeUnit + +import scala.collection.mutable + +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{doAnswer, mock, when} + +import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ +import org.apache.spark.util.ManualClock + +class ExecutorMonitorSuite extends SparkFunSuite { + + private val idleTimeoutMs = TimeUnit.SECONDS.toMillis(60L) + private val storageTimeoutMs = TimeUnit.SECONDS.toMillis(120L) + private val shuffleTimeoutMs = TimeUnit.SECONDS.toMillis(240L) + + private val conf = new SparkConf() + .set(DYN_ALLOCATION_EXECUTOR_IDLE_TIMEOUT.key, "60s") + .set(DYN_ALLOCATION_CACHED_EXECUTOR_IDLE_TIMEOUT.key, "120s") + .set(DYN_ALLOCATION_SHUFFLE_TIMEOUT.key, "240s") + .set(SHUFFLE_SERVICE_ENABLED, true) + + private var monitor: ExecutorMonitor = _ + private var client: ExecutorAllocationClient = _ + private var clock: ManualClock = _ + + // List of known executors. Allows easily mocking which executors are alive without + // having to use mockito APIs directly in each test. + private val knownExecs = mutable.HashSet[String]() + + override def beforeEach(): Unit = { + super.beforeEach() + knownExecs.clear() + clock = new ManualClock() + client = mock(classOf[ExecutorAllocationClient]) + when(client.isExecutorActive(any())).thenAnswer { invocation => + knownExecs.contains(invocation.getArguments()(0).asInstanceOf[String]) + } + monitor = new ExecutorMonitor(conf, client, null, clock) + } + + test("basic executor timeout") { + knownExecs += "1" + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.executorCount === 1) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + } + + test("SPARK-4951, SPARK-26927: handle out of order task start events") { + knownExecs ++= Set("1", "2") + + monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("1", 1))) + assert(monitor.executorCount === 1) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.executorCount === 1) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + assert(monitor.executorCount === 2) + + monitor.onExecutorRemoved(SparkListenerExecutorRemoved(clock.getTimeMillis(), "2", null)) + assert(monitor.executorCount === 1) + + knownExecs -= "2" + + monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("2", 2))) + assert(monitor.executorCount === 1) + } + + test("track tasks running on executor") { + knownExecs += "1" + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("1", 1))) + assert(!monitor.isExecutorIdle("1")) + + // Start/end a few tasks and make sure the executor does not go idle. + (2 to 10).foreach { i => + monitor.onTaskStart(SparkListenerTaskStart(i, 1, taskInfo("1", 1))) + assert(!monitor.isExecutorIdle("1")) + + monitor.onTaskEnd(SparkListenerTaskEnd(i, 1, "foo", Success, taskInfo("1", 1), null)) + assert(!monitor.isExecutorIdle("1")) + } + + monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(clock.getTimeMillis()).isEmpty) + assert(monitor.timedOutExecutors(clock.getTimeMillis() + idleTimeoutMs + 1) === Seq("1")) + } + + test("use appropriate time out depending on whether blocks are stored") { + knownExecs += "1" + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 0, "1")) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.NONE)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("1", 1))) + assert(!monitor.isExecutorIdle("1")) + monitor.onBlockUpdated(rddUpdate(1, 0, "1")) + assert(!monitor.isExecutorIdle("1")) + monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.NONE)) + assert(!monitor.isExecutorIdle("1")) + } + + test("keeps track of stored blocks for each rdd and split") { + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + + monitor.onBlockUpdated(rddUpdate(1, 0, "1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 1, "1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(2, 0, "1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 1, "1", level = StorageLevel.NONE)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onUnpersistRDD(SparkListenerUnpersistRDD(1)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + // Make sure that if we get an unpersist event much later, which moves an executor from having + // cached blocks to no longer having cached blocks, it will time out based on the time it + // originally went idle. + clock.setTime(idleDeadline) + monitor.onUnpersistRDD(SparkListenerUnpersistRDD(2)) + assert(monitor.timedOutExecutors(clock.getTimeMillis()) === Seq("1")) + } + + test("handle timeouts correctly with multiple executors") { + knownExecs ++= Set("1", "2", "3") + + // start exec 1 at 0s (should idle time out at 60s) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.isExecutorIdle("1")) + + // start exec 2 at 30s, store a block (should idle time out at 150s) + clock.setTime(TimeUnit.SECONDS.toMillis(30)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + monitor.onBlockUpdated(rddUpdate(1, 0, "2")) + assert(monitor.isExecutorIdle("2")) + assert(!monitor.timedOutExecutors(idleDeadline).contains("2")) + + // start exec 3 at 60s (should idle timeout at 120s, exec 1 should time out) + clock.setTime(TimeUnit.SECONDS.toMillis(60)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", null)) + assert(monitor.timedOutExecutors(clock.getTimeMillis()) === Seq("1")) + + // store block on exec 3 (should now idle time out at 180s) + monitor.onBlockUpdated(rddUpdate(1, 0, "3")) + assert(monitor.isExecutorIdle("3")) + assert(!monitor.timedOutExecutors(idleDeadline).contains("3")) + + // advance to 140s, remove block from exec 3 (time out immediately) + clock.setTime(TimeUnit.SECONDS.toMillis(140)) + monitor.onBlockUpdated(rddUpdate(1, 0, "3", level = StorageLevel.NONE)) + assert(monitor.timedOutExecutors(clock.getTimeMillis()).toSet === Set("1", "3")) + + // advance to 150s, now exec 2 should time out + clock.setTime(TimeUnit.SECONDS.toMillis(150)) + assert(monitor.timedOutExecutors(clock.getTimeMillis()).toSet === Set("1", "2", "3")) + } + + test("SPARK-27677: don't track blocks stored on disk when using shuffle service") { + // First make sure that blocks on disk are counted when no shuffle service is available. + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.DISK_ONLY)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + conf.set(SHUFFLE_SERVICE_ENABLED, true).set(SHUFFLE_SERVICE_FETCH_RDD_ENABLED, true) + monitor = new ExecutorMonitor(conf, client, null, clock) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.MEMORY_ONLY)) + monitor.onBlockUpdated(rddUpdate(1, 1, "1", level = StorageLevel.MEMORY_ONLY)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 0, "1", level = StorageLevel.DISK_ONLY)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline) === Seq("1")) + + monitor.onBlockUpdated(rddUpdate(1, 1, "1", level = StorageLevel.DISK_ONLY)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + // Tag the block as being both in memory and on disk, which may happen after it was + // evicted and then restored into memory. Since it's still on disk the executor should + // still be eligible for removal. + monitor.onBlockUpdated(rddUpdate(1, 1, "1", level = StorageLevel.MEMORY_AND_DISK)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + } + + test("track executors pending for removal") { + knownExecs ++= Set("1", "2", "3") + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "3", null)) + clock.setTime(idleDeadline) + assert(monitor.timedOutExecutors().toSet === Set("1", "2", "3")) + assert(monitor.pendingRemovalCount === 0) + + // Notify that only a subset of executors was killed, to mimic the case where the scheduler + // refuses to kill an executor that is busy for whatever reason the monitor hasn't detected yet. + monitor.executorsKilled(Seq("1")) + assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + assert(monitor.pendingRemovalCount === 1) + + // Check the timed out executors again so that we're sure they're still timed out when no + // events happen. This ensures that the monitor doesn't lose track of them. + assert(monitor.timedOutExecutors().toSet === Set("2", "3")) + + monitor.onTaskStart(SparkListenerTaskStart(1, 1, taskInfo("2", 1))) + assert(monitor.timedOutExecutors().toSet === Set("3")) + + monitor.executorsKilled(Seq("3")) + assert(monitor.pendingRemovalCount === 2) + + monitor.onTaskEnd(SparkListenerTaskEnd(1, 1, "foo", Success, taskInfo("2", 1), null)) + assert(monitor.timedOutExecutors().isEmpty) + clock.advance(idleDeadline) + assert(monitor.timedOutExecutors().toSet === Set("2")) + } + + test("shuffle block tracking") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + // 3 jobs: 2 and 3 share a shuffle, 1 has a separate shuffle. + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + + val stage5 = stageInfo(5, shuffleId = 1) + val stage6 = stageInfo(6) + + // Start jobs 1 and 2. Finish a task on each, but don't finish the jobs. This should prevent the + // executor from going idle since there are active shuffles. + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + // First a failed task, to make sure it does not count. + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", TaskResultLost, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + // Finish the jobs, now the executor should be idle, but with the shuffle timeout, since the + // shuffles are not active. + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(!monitor.isExecutorIdle("1")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(storageDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Start job 3. Since it shares a shuffle with job 2, the executor should not be considered + // idle anymore, even if no tasks are run. + monitor.onJobStart(SparkListenerJobStart(3, clock.getTimeMillis(), Seq(stage5, stage6))) + assert(!monitor.isExecutorIdle("1")) + assert(monitor.timedOutExecutors(shuffleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(3, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + assert(monitor.timedOutExecutors(shuffleDeadline) === Seq("1")) + + // Clean up the shuffles, executor now should now time out at the idle deadline. + monitor.shuffleCleaned(0) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + monitor.shuffleCleaned(1) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("1")) + } + + test("shuffle tracking with multiple executors and concurrent jobs") { + val bus = mockListenerBus() + conf.set(DYN_ALLOCATION_SHUFFLE_TRACKING, true).set(SHUFFLE_SERVICE_ENABLED, false) + monitor = new ExecutorMonitor(conf, client, bus, clock) + + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "1", null)) + monitor.onExecutorAdded(SparkListenerExecutorAdded(clock.getTimeMillis(), "2", null)) + + // Two separate jobs with separate shuffles. The first job will only run tasks on + // executor 1, the second on executor 2. Ensures that jobs finishing don't affect + // executors that are active in other jobs. + + val stage1 = stageInfo(1, shuffleId = 0) + val stage2 = stageInfo(2) + monitor.onJobStart(SparkListenerJobStart(1, clock.getTimeMillis(), Seq(stage1, stage2))) + + val stage3 = stageInfo(3, shuffleId = 1) + val stage4 = stageInfo(4) + monitor.onJobStart(SparkListenerJobStart(2, clock.getTimeMillis(), Seq(stage3, stage4))) + + monitor.onTaskStart(SparkListenerTaskStart(1, 0, taskInfo("1", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(1, 0, "foo", Success, taskInfo("1", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline) === Seq("2")) + + monitor.onTaskStart(SparkListenerTaskStart(3, 0, taskInfo("2", 1))) + monitor.onTaskEnd(SparkListenerTaskEnd(3, 0, "foo", Success, taskInfo("2", 1), null)) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.onJobEnd(SparkListenerJobEnd(1, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("1")) + assert(!monitor.isExecutorIdle("2")) + + monitor.onJobEnd(SparkListenerJobEnd(2, clock.getTimeMillis(), JobSucceeded)) + assert(monitor.isExecutorIdle("2")) + assert(monitor.timedOutExecutors(idleDeadline).isEmpty) + + monitor.shuffleCleaned(0) + monitor.shuffleCleaned(1) + assert(monitor.timedOutExecutors(idleDeadline).toSet === Set("1", "2")) + } + + private def idleDeadline: Long = clock.getTimeMillis() + idleTimeoutMs + 1 + private def storageDeadline: Long = clock.getTimeMillis() + storageTimeoutMs + 1 + private def shuffleDeadline: Long = clock.getTimeMillis() + shuffleTimeoutMs + 1 + + private def stageInfo(id: Int, shuffleId: Int = -1): StageInfo = { + new StageInfo(id, 0, s"stage$id", 1, Nil, Nil, "", + shuffleDepId = if (shuffleId >= 0) Some(shuffleId) else None) + } + + private def taskInfo( + execId: String, + id: Int, + speculative: Boolean = false, + duration: Long = -1L): TaskInfo = { + val start = if (duration > 0) clock.getTimeMillis() - duration else clock.getTimeMillis() + val task = new TaskInfo(id, id, 1, start, execId, "foo.example.com", + TaskLocality.PROCESS_LOCAL, speculative) + if (duration > 0) { + task.markFinished(TaskState.FINISHED, math.max(1, clock.getTimeMillis())) + } + task + } + + private def rddUpdate( + rddId: Int, + splitIndex: Int, + execId: String, + level: StorageLevel = StorageLevel.MEMORY_ONLY): SparkListenerBlockUpdated = { + SparkListenerBlockUpdated( + BlockUpdatedInfo(BlockManagerId(execId, "1.example.com", 42), + RDDBlockId(rddId, splitIndex), level, 1L, 0L)) + } + + /** + * Mock the listener bus *only* for the functionality needed by the shuffle tracking code. + * Any other event sent through the mock bus will fail. + */ + private def mockListenerBus(): LiveListenerBus = { + val bus = mock(classOf[LiveListenerBus]) + doAnswer { invocation => + monitor.onOtherEvent(invocation.getArguments()(0).asInstanceOf[SparkListenerEvent]) + }.when(bus).post(any()) + bus + } + +} diff --git a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala index 0d3611c80b8d0..abccb8e9bbf28 100644 --- a/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/security/CryptoStreamUtilsSuite.scala @@ -24,7 +24,7 @@ import java.nio.file.Files import java.util.{Arrays, Random, UUID} import com.google.common.io.ByteStreams -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.apache.spark._ @@ -75,8 +75,8 @@ class CryptoStreamUtilsSuite extends SparkFunSuite { test("serializer manager integration") { val conf = createConf() - .set("spark.shuffle.compress", "true") - .set("spark.shuffle.spill.compress", "true") + .set(SHUFFLE_COMPRESS, true) + .set(SHUFFLE_SPILL_COMPRESS, true) val plainStr = "hello world" val blockId = new TempShuffleBlockId(UUID.randomUUID()) diff --git a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala index 3734f1cb408fe..8610b18702ec0 100644 --- a/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/GenericAvroSerializerSuite.scala @@ -25,9 +25,10 @@ import org.apache.avro.{Schema, SchemaBuilder} import org.apache.avro.generic.GenericData.Record import org.apache.spark.{SharedSparkContext, SparkFunSuite} +import org.apache.spark.internal.config.SERIALIZER class GenericAvroSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") val schema : Schema = SchemaBuilder .record("testRecord").fields() diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala index f4fc0080f3108..fd228cded783a 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoBenchmark.scala @@ -20,58 +20,50 @@ package org.apache.spark.serializer import scala.reflect.ClassTag import scala.util.Random -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.benchmark.Benchmark +import org.apache.spark.SparkConf +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest._ -class KryoBenchmark extends SparkFunSuite { - val benchmark = new Benchmark("Benchmark Kryo Unsafe vs safe Serialization", 1024 * 1024 * 15, 10) - - ignore(s"Benchmark Kryo Unsafe vs safe Serialization") { - Seq (true, false).foreach (runBenchmark) - benchmark.run() - - // scalastyle:off - /* - Benchmark Kryo Unsafe vs safe Serialization: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - basicTypes: Int with unsafe:true 151 / 170 104.2 9.6 1.0X - basicTypes: Long with unsafe:true 175 / 191 89.8 11.1 0.9X - basicTypes: Float with unsafe:true 177 / 184 88.8 11.3 0.9X - basicTypes: Double with unsafe:true 193 / 216 81.4 12.3 0.8X - Array: Int with unsafe:true 513 / 587 30.7 32.6 0.3X - Array: Long with unsafe:true 1211 / 1358 13.0 77.0 0.1X - Array: Float with unsafe:true 890 / 964 17.7 56.6 0.2X - Array: Double with unsafe:true 1335 / 1428 11.8 84.9 0.1X - Map of string->Double with unsafe:true 931 / 988 16.9 59.2 0.2X - basicTypes: Int with unsafe:false 197 / 217 79.9 12.5 0.8X - basicTypes: Long with unsafe:false 219 / 240 71.8 13.9 0.7X - basicTypes: Float with unsafe:false 208 / 217 75.7 13.2 0.7X - basicTypes: Double with unsafe:false 208 / 225 75.6 13.2 0.7X - Array: Int with unsafe:false 2559 / 2681 6.1 162.7 0.1X - Array: Long with unsafe:false 3425 / 3516 4.6 217.8 0.0X - Array: Float with unsafe:false 2025 / 2134 7.8 128.7 0.1X - Array: Double with unsafe:false 2241 / 2358 7.0 142.5 0.1X - Map of string->Double with unsafe:false 1044 / 1085 15.1 66.4 0.1X - */ - // scalastyle:on +/** + * Benchmark for Kryo Unsafe vs safe Serialization. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/KryoBenchmark-results.txt". + * }}} + */ +object KryoBenchmark extends BenchmarkBase { + + val N = 1000000 + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val name = "Benchmark Kryo Unsafe vs safe Serialization" + runBenchmark(name) { + val benchmark = new Benchmark(name, N, 10, output = output) + Seq(true, false).foreach(useUnsafe => run(useUnsafe, benchmark)) + benchmark.run() + } } - private def runBenchmark(useUnsafe: Boolean): Unit = { + private def run(useUnsafe: Boolean, benchmark: Benchmark): Unit = { def check[T: ClassTag](t: T, ser: SerializerInstance): Int = { - if (ser.deserialize[T](ser.serialize(t)) === t) 1 else 0 + if (ser.deserialize[T](ser.serialize(t)) == t) 1 else 0 } // Benchmark Primitives - val basicTypeCount = 1000000 def basicTypes[T: ClassTag](name: String, gen: () => T): Unit = { lazy val ser = createSerializer(useUnsafe) - val arrayOfBasicType: Array[T] = Array.fill(basicTypeCount)(gen()) + val arrayOfBasicType: Array[T] = Array.fill(N)(gen()) benchmark.addCase(s"basicTypes: $name with unsafe:$useUnsafe") { _ => var sum = 0L var i = 0 - while (i < basicTypeCount) { + while (i < N) { sum += check(arrayOfBasicType(i), ser) i += 1 } @@ -84,11 +76,12 @@ class KryoBenchmark extends SparkFunSuite { basicTypes("Double", () => Random.nextDouble()) // Benchmark Array of Primitives - val arrayCount = 10000 + val arrayCount = 4000 + val arrayLength = N / arrayCount def basicTypeArray[T: ClassTag](name: String, gen: () => T): Unit = { lazy val ser = createSerializer(useUnsafe) val arrayOfArrays: Array[Array[T]] = - Array.fill(arrayCount)(Array.fill[T](Random.nextInt(arrayCount))(gen())) + Array.fill(arrayCount)(Array.fill[T](arrayLength + Random.nextInt(arrayLength / 4))(gen())) benchmark.addCase(s"Array: $name with unsafe:$useUnsafe") { _ => var sum = 0L @@ -107,11 +100,13 @@ class KryoBenchmark extends SparkFunSuite { basicTypeArray("Double", () => Random.nextDouble()) // Benchmark Maps - val mapsCount = 1000 + val mapsCount = 200 + val mapKeyLength = 20 + val mapLength = N / mapsCount / mapKeyLength lazy val ser = createSerializer(useUnsafe) val arrayOfMaps: Array[Map[String, Double]] = Array.fill(mapsCount) { - Array.fill(Random.nextInt(mapsCount)) { - (Random.nextString(mapsCount / 10), Random.nextDouble()) + Array.fill(mapLength + Random.nextInt(mapLength / 4)) { + (Random.nextString(mapKeyLength), Random.nextDouble()) }.toMap } @@ -129,9 +124,9 @@ class KryoBenchmark extends SparkFunSuite { def createSerializer(useUnsafe: Boolean): SerializerInstance = { val conf = new SparkConf() - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", useUnsafe.toString) + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, useUnsafe) new KryoSerializer(conf).newInstance() } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala new file mode 100644 index 0000000000000..2915b99dcfb60 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerBenchmark.scala @@ -0,0 +1,92 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.serializer + +import scala.concurrent._ +import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.duration._ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.serializer.KryoTest._ +import org.apache.spark.util.ThreadUtils + +/** + * Benchmark for KryoPool vs old "pool of 1". + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt "core/test:runMain " + * 3. generate result: + * SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "core/test:runMain " + * Results will be written to "benchmarks/KryoSerializerBenchmark-results.txt". + * }}} + */ +object KryoSerializerBenchmark extends BenchmarkBase { + + var sc: SparkContext = null + val N = 500 + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val name = "Benchmark KryoPool vs old\"pool of 1\" implementation" + runBenchmark(name) { + val benchmark = new Benchmark(name, N, 10, output = output) + Seq(true, false).foreach(usePool => run(usePool, benchmark)) + benchmark.run() + } + } + + private def run(usePool: Boolean, benchmark: Benchmark): Unit = { + lazy val sc = createSparkContext(usePool) + + benchmark.addCase(s"KryoPool:$usePool") { _ => + val futures = for (_ <- 0 until N) yield { + Future { + sc.parallelize(0 until 10).map(i => i + 1).count() + } + } + + val future = Future.sequence(futures) + + ThreadUtils.awaitResult(future, 10.minutes) + } + } + + def createSparkContext(usePool: Boolean): SparkContext = { + val conf = new SparkConf() + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_POOL, usePool) + + if (sc != null) { + sc.stop() + } + + sc = new SparkContext("local-cluster[4,1,1024]", "test", conf) + sc + } + + override def afterAll(): Unit = { + if (sc != null) { + sc.stop() + } + } + +} diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala index 46aa9c37986cc..5d76c096d46ac 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerDistributedSuite.scala @@ -28,9 +28,9 @@ class KryoSerializerDistributedSuite extends SparkFunSuite with LocalSparkContex test("kryo objects are serialised consistently in different processes") { val conf = new SparkConf(false) - .set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - .set("spark.kryo.registrator", classOf[AppJarRegistrator].getName) - .set(config.MAX_TASK_FAILURES, 1) + .set(config.SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + .set(config.Kryo.KRYO_USER_REGISTRATORS, classOf[AppJarRegistrator].getName) + .set(config.TASK_MAX_FAILURES, 1) .set(config.BLACKLIST_ENABLED, false) val jar = TestUtils.createJarWithClasses(List(AppJarRegistrator.customClassName)) @@ -57,10 +57,8 @@ object KryoDistributedTest { class AppJarRegistrator extends KryoRegistrator { override def registerClasses(k: Kryo) { - val classLoader = Thread.currentThread.getContextClassLoader - // scalastyle:off classforname - k.register(Class.forName(AppJarRegistrator.customClassName, true, classLoader)) - // scalastyle:on classforname + k.register(Utils.classForName(AppJarRegistrator.customClassName, + noSparkClassLoader = true)) } } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala index cf01f79f49091..25f0b19c980fb 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerResizableOutputSuite.scala @@ -21,6 +21,8 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.LocalSparkContext._ import org.apache.spark.SparkContext import org.apache.spark.SparkException +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ class KryoSerializerResizableOutputSuite extends SparkFunSuite { @@ -29,9 +31,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo without resizable output buffer should fail on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "1m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "1m") withSpark(new SparkContext("local", "test", conf)) { sc => intercept[SparkException](sc.parallelize(x).collect()) } @@ -39,9 +41,9 @@ class KryoSerializerResizableOutputSuite extends SparkFunSuite { test("kryo with resizable output buffer should succeed on large array") { val conf = new SparkConf(false) - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.kryoserializer.buffer.max", "2m") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "2m") withSpark(new SparkContext("local", "test", conf)) { sc => assert(sc.parallelize(x).collect() === x) } diff --git a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala index ac25bcef54349..2442670b6d3f0 100644 --- a/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/KryoSerializerSuite.scala @@ -17,10 +17,14 @@ package org.apache.spark.serializer -import java.io.{ByteArrayInputStream, ByteArrayOutputStream, FileInputStream, FileOutputStream} +import java.io.{ByteArrayInputStream, ByteArrayOutputStream} +import java.nio.ByteBuffer +import java.util.concurrent.Executors import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.concurrent.{ExecutionContext, Future} +import scala.concurrent.duration._ import scala.reflect.ClassTag import com.esotericsoftware.kryo.{Kryo, KryoException} @@ -28,19 +32,21 @@ import com.esotericsoftware.kryo.io.{Input => KryoInput, Output => KryoOutput} import org.roaringbitmap.RoaringBitmap import org.apache.spark.{SharedSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.scheduler.HighlyCompressedMapStatus import org.apache.spark.serializer.KryoTest._ import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.Utils +import org.apache.spark.util.{ThreadUtils, Utils} class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", "org.apache.spark.serializer.KryoSerializer") - conf.set("spark.kryo.registrator", classOf[MyRegistrator].getName) - conf.set("spark.kryo.unsafe", "false") + conf.set(SERIALIZER, "org.apache.spark.serializer.KryoSerializer") + conf.set(KRYO_USER_REGISTRATORS, classOf[MyRegistrator].getName) + conf.set(KRYO_USE_UNSAFE, false) test("SPARK-7392 configuration limits") { - val kryoBufferProperty = "spark.kryoserializer.buffer" - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferProperty = KRYO_SERIALIZER_BUFFER_SIZE.key + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key def newKryoInstance( conf: SparkConf, @@ -71,13 +77,13 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { val thrown3 = intercept[IllegalArgumentException](newKryoInstance(conf, "2g", "3g")) assert(thrown3.getMessage.contains(kryoBufferProperty)) assert(!thrown3.getMessage.contains(kryoBufferMaxProperty)) - // test configuration with mb is supported properly + // test configuration with MiB is supported properly newKryoInstance(conf, "8m", "9m") } test("basic types") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -110,7 +116,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("pairs") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -137,7 +143,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("Scala data structures") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { @@ -165,7 +171,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { } test("Bug: SPARK-10251") { - val ser = new KryoSerializer(conf.clone.set("spark.kryo.registrationRequired", "true")) + val ser = new KryoSerializer(conf.clone.set(KRYO_REGISTRATION_REQUIRED, true)) .newInstance() def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) @@ -199,7 +205,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { def check[T: ClassTag](t: T) { assert(ser.deserialize[T](ser.serialize(t)) === t) // Check that very long ranges don't get written one element at a time - assert(ser.serialize(t).limit() < 100) + assert(ser.serialize(t).limit() < 200) } check(1 to 1000000) check(1 to 1000000 by 2) @@ -209,10 +215,10 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { check(1L to 1000000L by 2L) check(1L until 1000000L) check(1L until 1000000L by 2L) - check(1.0 to 1000000.0 by 1.0) - check(1.0 to 1000000.0 by 2.0) - check(1.0 until 1000000.0 by 1.0) - check(1.0 until 1000000.0 by 2.0) + check(Range.BigDecimal.inclusive(1, 1000000, 1)) + check(Range.BigDecimal.inclusive(1, 1000000, 2)) + check(Range.BigDecimal(1, 1000000, 1)) + check(Range.BigDecimal(1, 1000000, 2)) } test("asJavaIterable") { @@ -249,7 +255,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { hashMap.put("foo", "bar") check(hashMap) - System.clearProperty("spark.kryo.registrator") + System.clearProperty(KRYO_USER_REGISTRATORS.key) } test("kryo with collect") { @@ -306,9 +312,9 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { import org.apache.spark.SparkException val conf = new SparkConf(false) - conf.set("spark.kryo.registrator", "this.class.does.not.exist") + conf.set(KRYO_USER_REGISTRATORS, "this.class.does.not.exist") - val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance()) + val thrown = intercept[SparkException](new KryoSerializer(conf).newInstance().serialize(1)) assert(thrown.getMessage.contains("Failed to register classes with Kryo")) } @@ -333,7 +339,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("registration of HighlyCompressedMapStatus") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) // these cases require knowing the internals of RoaringBitmap a little. Blocks span 2^16 // values, and they use a bitmap (dense) if they have more than 4096 values, and an @@ -351,7 +357,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("serialization buffer overflow reporting") { import org.apache.spark.SparkException - val kryoBufferMaxProperty = "spark.kryoserializer.buffer.max" + val kryoBufferMaxProperty = KRYO_SERIALIZER_MAX_BUFFER_SIZE.key val largeObject = (1 to 1000000).toArray @@ -364,30 +370,6 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { assert(thrown.getCause.isInstanceOf[KryoException]) } - test("SPARK-12222: deserialize RoaringBitmap throw Buffer underflow exception") { - val dir = Utils.createTempDir() - val tmpfile = dir.toString + "/RoaringBitmap" - val outStream = new FileOutputStream(tmpfile) - val output = new KryoOutput(outStream) - val bitmap = new RoaringBitmap - bitmap.add(1) - bitmap.add(3) - bitmap.add(5) - // Ignore Kryo because it doesn't use writeObject - bitmap.serialize(new KryoOutputObjectOutputBridge(null, output)) - output.flush() - output.close() - - val inStream = new FileInputStream(tmpfile) - val input = new KryoInput(inStream) - val ret = new RoaringBitmap - // Ignore Kryo because it doesn't use readObject - ret.deserialize(new KryoInputObjectInputBridge(null, input)) - input.close() - assert(ret == bitmap) - Utils.deleteRecursively(dir) - } - test("KryoOutputObjectOutputBridge.writeObject and KryoInputObjectInputBridge.readObject") { val kryo = new KryoSerializer(conf).newKryo() @@ -405,7 +387,7 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { test("getAutoReset") { val ser = new KryoSerializer(new SparkConf).newInstance().asInstanceOf[KryoSerializerInstance] assert(ser.getAutoReset) - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser2 = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] assert(!ser2.getAutoReset) @@ -431,11 +413,13 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { ser.deserialize[HashMap[Int, List[String]]](serializedMap) } - private def testSerializerInstanceReuse(autoReset: Boolean, referenceTracking: Boolean): Unit = { + private def testSerializerInstanceReuse( + autoReset: Boolean, referenceTracking: Boolean, usePool: Boolean): Unit = { val conf = new SparkConf(loadDefaults = false) - .set("spark.kryo.referenceTracking", referenceTracking.toString) + .set(KRYO_REFERENCE_TRACKING, referenceTracking) + .set(KRYO_USE_POOL, usePool) if (!autoReset) { - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) } val ser = new KryoSerializer(conf) val serInstance = ser.newInstance().asInstanceOf[KryoSerializerInstance] @@ -456,19 +440,76 @@ class KryoSerializerSuite extends SparkFunSuite with SharedSparkContext { // Regression test for SPARK-7766, an issue where disabling auto-reset and enabling // reference-tracking would lead to corrupted output when serializer instances are re-used - for (referenceTracking <- Set(true, false); autoReset <- Set(true, false)) { - test(s"instance reuse with autoReset = $autoReset, referenceTracking = $referenceTracking") { - testSerializerInstanceReuse(autoReset = autoReset, referenceTracking = referenceTracking) + for { + referenceTracking <- Seq(true, false) + autoReset <- Seq(true, false) + usePool <- Seq(true, false) + } { + test(s"instance reuse with autoReset = $autoReset, referenceTracking = $referenceTracking" + + s", usePool = $usePool") { + testSerializerInstanceReuse( + autoReset, referenceTracking, usePool) + } + } + + test("SPARK-25839 KryoPool implementation works correctly in multi-threaded environment") { + implicit val executionContext: ExecutionContext = ExecutionContext.fromExecutor( + Executors.newFixedThreadPool(4)) + + val ser = new KryoSerializer(conf.clone.set(KRYO_USE_POOL, true)) + + val tests = mutable.ListBuffer[Future[Boolean]]() + + def check[T: ClassTag](t: T) { + tests += Future { + val serializerInstance = ser.newInstance() + serializerInstance.deserialize[T](serializerInstance.serialize(t)) === t + } + } + + check((1, 3)) + check(Array((1, 3))) + check(List((1, 3))) + check(List[Int]()) + check(List[Int](1, 2, 3)) + check(List[String]()) + check(List[String]("x", "y", "z")) + check(None) + check(Some(1)) + check(Some("hi")) + check(1 -> 1) + check(mutable.ArrayBuffer(1, 2, 3)) + check(mutable.ArrayBuffer("1", "2", "3")) + check(mutable.Map()) + check(mutable.Map(1 -> "one", 2 -> "two")) + check(mutable.Map("one" -> 1, "two" -> 2)) + check(mutable.HashMap(1 -> "one", 2 -> "two")) + check(mutable.HashMap("one" -> 1, "two" -> 2)) + check(List(Some(mutable.HashMap(1 -> 1, 2 -> 2)), None, Some(mutable.HashMap(3 -> 4)))) + check(List( + mutable.HashMap("one" -> 1, "two" -> 2), + mutable.HashMap(1 -> "one", 2 -> "two", 3 -> "three"))) + + tests.foreach { f => + assert(ThreadUtils.awaitResult(f, 10.seconds)) } } + + test("SPARK-27216: test RoaringBitmap ser/dser with Kryo") { + val expected = new RoaringBitmap() + expected.add(1787) + val ser = new KryoSerializer(conf).newInstance() + val actual: RoaringBitmap = ser.deserialize(ser.serialize(expected)) + assert(actual === expected) + } } class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSparkContext { - conf.set("spark.serializer", classOf[KryoSerializer].getName) - conf.set("spark.kryo.registrator", classOf[RegistratorWithoutAutoReset].getName) - conf.set("spark.kryo.referenceTracking", "true") - conf.set("spark.shuffle.manager", "sort") - conf.set("spark.shuffle.sort.bypassMergeThreshold", "200") + conf.set(SERIALIZER, classOf[KryoSerializer].getName) + conf.set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) + conf.set(KRYO_REFERENCE_TRACKING, true) + conf.set(SHUFFLE_MANAGER, "sort") + conf.set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 200) test("sort-shuffle with bypassMergeSort (SPARK-7873)") { val myObject = ("Hello", "World") @@ -497,6 +538,17 @@ class KryoSerializerAutoResetDisabledSuite extends SparkFunSuite with SharedSpar deserializationStream.close() assert(serInstance.deserialize[Any](helloHello) === ((hello, hello))) } + + test("SPARK-25786: ByteBuffer.array -- UnsupportedOperationException") { + val serInstance = new KryoSerializer(conf).newInstance().asInstanceOf[KryoSerializerInstance] + val obj = "UnsupportedOperationException" + val serObj = serInstance.serialize(obj) + val byteBuffer = ByteBuffer.allocateDirect(serObj.array().length) + byteBuffer.put(serObj.array()) + byteBuffer.flip() + assert(serInstance.deserialize[Any](serObj) === (obj)) + assert(serInstance.deserialize[Any](byteBuffer) === (obj)) + } } class ClassLoaderTestingObject diff --git a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala index 99882bf76e29d..dad080c5fc161 100644 --- a/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/SerializerPropertiesSuite.scala @@ -24,6 +24,7 @@ import scala.util.Random import org.scalatest.Assertions import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoTest.RegistratorWithoutAutoReset /** @@ -50,7 +51,7 @@ class SerializerPropertiesSuite extends SparkFunSuite { } test("KryoSerializer does not support relocation when auto-reset is disabled") { - val conf = new SparkConf().set("spark.kryo.registrator", + val conf = new SparkConf().set(KRYO_USER_REGISTRATORS, classOf[RegistratorWithoutAutoReset].getName) val ser = new KryoSerializer(conf) assert(!ser.newInstance().asInstanceOf[KryoSerializerInstance].getAutoReset()) diff --git a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala index d63a45ae4a6a9..126ba0e8b1e93 100644 --- a/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala +++ b/core/src/test/scala/org/apache/spark/serializer/UnsafeKryoSerializerSuite.scala @@ -17,17 +17,19 @@ package org.apache.spark.serializer +import org.apache.spark.internal.config.Kryo._ + class UnsafeKryoSerializerSuite extends KryoSerializerSuite { // This test suite should run all tests in KryoSerializerSuite with kryo unsafe. override def beforeAll() { - conf.set("spark.kryo.unsafe", "true") + conf.set(KRYO_USE_UNSAFE, true) super.beforeAll() } override def afterAll() { - conf.set("spark.kryo.unsafe", "false") + conf.set(KRYO_USE_UNSAFE, false) super.afterAll() } } diff --git a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala index 2d8a83c6fabed..6d2ef17a7a790 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/BlockStoreShuffleReaderSuite.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import org.mockito.Mockito.{mock, when} import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} import org.apache.spark.serializer.{JavaSerializer, SerializerManager} import org.apache.spark.storage.{BlockManager, BlockManagerId, ShuffleBlockId} @@ -123,14 +124,17 @@ class BlockStoreShuffleReaderSuite extends SparkFunSuite with LocalSparkContext val serializerManager = new SerializerManager( serializer, new SparkConf() - .set("spark.shuffle.compress", "false") - .set("spark.shuffle.spill.compress", "false")) + .set(config.SHUFFLE_COMPRESS, false) + .set(config.SHUFFLE_SPILL_COMPRESS, false)) + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() val shuffleReader = new BlockStoreShuffleReader( shuffleHandle, reduceId, reduceId + 1, - TaskContext.empty(), + taskContext, + metrics, serializerManager, blockManager, mapOutputTracker) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala index 85ccb33471048..fc1422dfaac75 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/BypassMergeSortShuffleWriterSuite.scala @@ -25,10 +25,9 @@ import scala.collection.mutable.ArrayBuffer import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyInt} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark._ @@ -69,16 +68,14 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte when(dependency.serializer).thenReturn(new JavaSerializer(conf)) when(taskContext.taskMetrics()).thenReturn(taskMetrics) when(blockResolver.getDataFile(0, 0)).thenReturn(outputFile) - doAnswer(new Answer[Void] { - def answer(invocationOnMock: InvocationOnMock): Void = { - val tmp: File = invocationOnMock.getArguments()(3).asInstanceOf[File] - if (tmp != null) { - outputFile.delete - tmp.renameTo(outputFile) - } - null + doAnswer { (invocationOnMock: InvocationOnMock) => + val tmp = invocationOnMock.getArguments()(3).asInstanceOf[File] + if (tmp != null) { + outputFile.delete + tmp.renameTo(outputFile) } - }).when(blockResolver) + null + }.when(blockResolver) .writeIndexFileAndCommit(anyInt, anyInt, any(classOf[Array[Long]]), any(classOf[File])) when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(blockManager.getDiskWriter( @@ -87,37 +84,29 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte any[SerializerInstance], anyInt(), any[ShuffleWriteMetrics] - )).thenAnswer(new Answer[DiskBlockObjectWriter] { - override def answer(invocation: InvocationOnMock): DiskBlockObjectWriter = { - val args = invocation.getArguments - val manager = new SerializerManager(new JavaSerializer(conf), conf) - new DiskBlockObjectWriter( - args(1).asInstanceOf[File], - manager, - args(2).asInstanceOf[SerializerInstance], - args(3).asInstanceOf[Int], - syncWrites = false, - args(4).asInstanceOf[ShuffleWriteMetrics], - blockId = args(0).asInstanceOf[BlockId] - ) - } + )).thenAnswer((invocation: InvocationOnMock) => { + val args = invocation.getArguments + val manager = new SerializerManager(new JavaSerializer(conf), conf) + new DiskBlockObjectWriter( + args(1).asInstanceOf[File], + manager, + args(2).asInstanceOf[SerializerInstance], + args(3).asInstanceOf[Int], + syncWrites = false, + args(4).asInstanceOf[ShuffleWriteMetrics], + blockId = args(0).asInstanceOf[BlockId] + ) }) - when(diskBlockManager.createTempShuffleBlock()).thenAnswer( - new Answer[(TempShuffleBlockId, File)] { - override def answer(invocation: InvocationOnMock): (TempShuffleBlockId, File) = { - val blockId = new TempShuffleBlockId(UUID.randomUUID) - val file = new File(tempDir, blockId.name) - blockIdToFileMap.put(blockId, file) - temporaryFilesCreated += file - (blockId, file) - } - }) - when(diskBlockManager.getFile(any[BlockId])).thenAnswer( - new Answer[File] { - override def answer(invocation: InvocationOnMock): File = { - blockIdToFileMap.get(invocation.getArguments.head.asInstanceOf[BlockId]).get - } + when(diskBlockManager.createTempShuffleBlock()).thenAnswer((_: InvocationOnMock) => { + val blockId = new TempShuffleBlockId(UUID.randomUUID) + val file = new File(tempDir, blockId.name) + blockIdToFileMap.put(blockId, file) + temporaryFilesCreated += file + (blockId, file) }) + when(diskBlockManager.getFile(any[BlockId])).thenAnswer { (invocation: InvocationOnMock) => + blockIdToFileMap(invocation.getArguments.head.asInstanceOf[BlockId]) + } } override def afterEach(): Unit = { @@ -136,8 +125,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId - taskContext, - conf + conf, + taskContext.taskMetrics().shuffleWriteMetrics ) writer.write(Iterator.empty) writer.stop( /* success = */ true) @@ -160,8 +149,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId - taskContext, - conf + conf, + taskContext.taskMetrics().shuffleWriteMetrics ) writer.write(records) writer.stop( /* success = */ true) @@ -195,8 +184,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId - taskContext, - conf + conf, + taskContext.taskMetrics().shuffleWriteMetrics ) intercept[SparkException] { @@ -217,8 +206,8 @@ class BypassMergeSortShuffleWriterSuite extends SparkFunSuite with BeforeAndAfte blockResolver, shuffleHandle, 0, // MapId - taskContext, - conf + conf, + taskContext.taskMetrics().shuffleWriteMetrics ) intercept[SparkException] { writer.write((0 until 100000).iterator.map(i => { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala index 4ce379b76b551..27bb06b4e0636 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/IndexShuffleBlockResolverSuite.scala @@ -21,10 +21,9 @@ import java.io.{DataInputStream, File, FileInputStream, FileOutputStream} import org.mockito.{Mock, MockitoAnnotations} import org.mockito.Answers.RETURNS_SMART_NULLS -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkFunSuite} @@ -48,11 +47,7 @@ class IndexShuffleBlockResolverSuite extends SparkFunSuite with BeforeAndAfterEa when(blockManager.diskBlockManager).thenReturn(diskBlockManager) when(diskBlockManager.getFile(any[BlockId])).thenAnswer( - new Answer[File] { - override def answer(invocation: InvocationOnMock): File = { - new File(tempDir, invocation.getArguments.head.toString) - } - }) + (invocation: InvocationOnMock) => new File(tempDir, invocation.getArguments.head.toString)) } override def afterEach(): Unit = { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala index b9f0e873375b0..8b955c98f7953 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/ShuffleExternalSorterSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.mockito.MockitoSugar import org.apache.spark._ import org.apache.spark.executor.{ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.internal.config.MEMORY_FRACTION +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory._ import org.apache.spark.unsafe.Platform @@ -33,9 +35,10 @@ class ShuffleExternalSorterSuite extends SparkFunSuite with LocalSparkContext wi val conf = new SparkConf() .setMaster("local[1]") .setAppName("ShuffleExternalSorterSuite") - .set("spark.testing", "true") - .set("spark.testing.memory", "1600") - .set("spark.memory.fraction", "1") + .set(IS_TESTING, true) + .set(TEST_MEMORY, 1600L) + .set(MEMORY_FRACTION, 0.9999) + sc = new SparkContext(conf) val memoryManager = UnifiedMemoryManager(conf, 1) diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala index f29dac965c803..e5f3aab6a6a1a 100644 --- a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleManagerSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.shuffle.sort -import org.mockito.Mockito._ +import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.Matchers @@ -31,6 +31,8 @@ import org.apache.spark.serializer.{JavaSerializer, KryoSerializer, Serializer} */ class SortShuffleManagerSuite extends SparkFunSuite with Matchers { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + import SortShuffleManager.canUseSerializedShuffle private class RuntimeExceptionAnswer extends Answer[Object] { diff --git a/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala new file mode 100644 index 0000000000000..690bcd9905257 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/shuffle/sort/SortShuffleWriterSuite.scala @@ -0,0 +1,96 @@ +/* + * 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.shuffle.sort + +import org.mockito.Mockito._ +import org.mockito.MockitoAnnotations +import org.scalatest.Matchers + +import org.apache.spark.{Partitioner, SharedSparkContext, ShuffleDependency, SparkFunSuite} +import org.apache.spark.memory.MemoryTestingUtils +import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.shuffle.{BaseShuffleHandle, IndexShuffleBlockResolver} +import org.apache.spark.util.Utils + + +class SortShuffleWriterSuite extends SparkFunSuite with SharedSparkContext with Matchers { + + private val shuffleId = 0 + private val numMaps = 5 + private var shuffleHandle: BaseShuffleHandle[Int, Int, Int] = _ + private val shuffleBlockResolver = new IndexShuffleBlockResolver(conf) + private val serializer = new JavaSerializer(conf) + + override def beforeEach(): Unit = { + super.beforeEach() + MockitoAnnotations.initMocks(this) + val partitioner = new Partitioner() { + def numPartitions = numMaps + def getPartition(key: Any) = Utils.nonNegativeMod(key.hashCode, numPartitions) + } + shuffleHandle = { + val dependency = mock(classOf[ShuffleDependency[Int, Int, Int]]) + when(dependency.partitioner).thenReturn(partitioner) + when(dependency.serializer).thenReturn(serializer) + when(dependency.aggregator).thenReturn(None) + when(dependency.keyOrdering).thenReturn(None) + new BaseShuffleHandle(shuffleId, numMaps = numMaps, dependency) + } + } + + override def afterAll(): Unit = { + try { + shuffleBlockResolver.stop() + } finally { + super.afterAll() + } + } + + test("write empty iterator") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val writer = new SortShuffleWriter[Int, Int, Int]( + shuffleBlockResolver, + shuffleHandle, + mapId = 1, + context) + writer.write(Iterator.empty) + writer.stop(success = true) + val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 1) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(!dataFile.exists()) + assert(writeMetrics.bytesWritten === 0) + assert(writeMetrics.recordsWritten === 0) + } + + test("write with some records") { + val context = MemoryTestingUtils.fakeTaskContext(sc.env) + val records = List[(Int, Int)]((1, 2), (2, 3), (4, 4), (6, 5)) + val writer = new SortShuffleWriter[Int, Int, Int]( + shuffleBlockResolver, + shuffleHandle, + mapId = 2, + context) + writer.write(records.toIterator) + writer.stop(success = true) + val dataFile = shuffleBlockResolver.getDataFile(shuffleId, 2) + val writeMetrics = context.taskMetrics().shuffleWriteMetrics + assert(dataFile.exists()) + assert(dataFile.length() === writeMetrics.bytesWritten) + assert(records.size === writeMetrics.recordsWritten) + } +} diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala index 0b2bbd2fa8a78..b5800661efa7f 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusListenerSuite.scala @@ -18,8 +18,7 @@ package org.apache.spark.status import java.io.File -import java.lang.{Integer => JInteger, Long => JLong} -import java.util.{Arrays, Date, Properties} +import java.util.{Date, Properties} import scala.collection.JavaConverters._ import scala.collection.immutable.Map @@ -29,6 +28,7 @@ import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.executor.{ExecutorMetrics, TaskMetrics} +import org.apache.spark.internal.config.Status._ import org.apache.spark.metrics.ExecutorMetricType import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster._ @@ -38,8 +38,6 @@ import org.apache.spark.util.Utils class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { - import config._ - private val conf = new SparkConf() .set(LIVE_ENTITY_UPDATE_PERIOD, 0L) .set(ASYNC_TRACKING_ENABLED, false) @@ -137,7 +135,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { execIds.foreach { id => listener.onExecutorAdded(SparkListenerExecutorAdded(time, id, - new ExecutorInfo(s"$id.example.com", 1, Map()))) + new ExecutorInfo(s"$id.example.com", 1, Map.empty, Map.empty))) } execIds.foreach { id => @@ -156,7 +154,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val jobProps = new Properties() jobProps.setProperty(SparkContext.SPARK_JOB_GROUP_ID, "jobGroup") - jobProps.setProperty("spark.scheduler.pool", "schedPool") + jobProps.setProperty(SparkContext.SPARK_SCHEDULER_POOL, "schedPool") listener.onJobStart(SparkListenerJobStart(1, time, stages, jobProps)) @@ -296,7 +294,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(executorStageSummaryWrappersForNode.nonEmpty) executorStageSummaryWrappersForNode.foreach { exec => // both executor is expected to be blacklisted - assert(exec.info.isBlacklistedForStage === true) + assert(exec.info.isBlacklistedForStage) } // Fail one of the tasks, re-start it. @@ -687,7 +685,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { val bm2 = BlockManagerId("2", "2.example.com", 84) Seq(bm1, bm2).foreach { bm => listener.onExecutorAdded(SparkListenerExecutorAdded(1L, bm.executorId, - new ExecutorInfo(bm.host, 1, Map()))) + new ExecutorInfo(bm.host, 1, Map.empty, Map.empty))) listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm, maxMemory)) check[ExecutorSummaryWrapper](bm.executorId) { exec => assert(exec.info.maxMemory === maxMemory) @@ -940,6 +938,24 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { intercept[NoSuchElementException] { check[StreamBlockData](stream1.name) { _ => () } } + + // Update a BroadcastBlock. + val broadcast1 = BroadcastBlockId(1L) + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, broadcast1, level, 1L, 1L))) + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.memoryUsed === 1L) + assert(exec.info.diskUsed === 1L) + } + + // Drop a BroadcastBlock. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, broadcast1, StorageLevel.NONE, 1L, 1L))) + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.memoryUsed === 0) + assert(exec.info.diskUsed === 0) + } } test("eviction of old data") { @@ -1172,12 +1188,12 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Stop task 2 before task 1 time += 1 tasks(1).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd( - SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(1), null)) + listener.onTaskEnd(SparkListenerTaskEnd( + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(1), null)) time += 1 tasks(0).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd( - SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null)) + listener.onTaskEnd(SparkListenerTaskEnd( + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), null)) // Start task 3 and task 2 should be evicted. listener.onTaskStart(SparkListenerTaskStart(stage1.stageId, stage1.attemptNumber, tasks(2))) @@ -1242,8 +1258,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // Task 1 Finished time += 1 tasks(0).markFinished(TaskState.FINISHED, time) - listener.onTaskEnd( - SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", Success, tasks(0), null)) + listener.onTaskEnd(SparkListenerTaskEnd( + stage1.stageId, stage1.attemptNumber, "taskType", Success, tasks(0), null)) // Stage 1 Completed stage1.failureReason = Some("Failed") @@ -1257,7 +1273,7 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { time += 1 tasks(1).markFinished(TaskState.FINISHED, time) listener.onTaskEnd( - SparkListenerTaskEnd(stage1.stageId, stage1.attemptId, "taskType", + SparkListenerTaskEnd(stage1.stageId, stage1.attemptNumber, "taskType", TaskKilled(reason = "Killed"), tasks(1), null)) // Ensure killed task metrics are updated @@ -1275,6 +1291,71 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(allJobs.head.numFailedStages == 1) } + Seq(true, false).foreach { live => + test(s"Total tasks in the executor summary should match total stage tasks (live = $live)") { + + val testConf = if (live) { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, Long.MaxValue) + } else { + conf.clone().set(LIVE_ENTITY_UPDATE_PERIOD, -1L) + } + + val listener = new AppStatusListener(store, testConf, live) + + listener.onExecutorAdded(createExecutorAddedEvent(1)) + listener.onExecutorAdded(createExecutorAddedEvent(2)) + val stage = new StageInfo(1, 0, "stage", 4, Nil, Nil, "details") + listener.onJobStart(SparkListenerJobStart(1, time, Seq(stage), null)) + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + + val tasks = createTasks(4, Array("1", "2")) + tasks.foreach { task => + listener.onTaskStart(SparkListenerTaskStart(stage.stageId, stage.attemptNumber, task)) + } + + time += 1 + tasks(0).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(0), null)) + time += 1 + tasks(1).markFinished(TaskState.FINISHED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + Success, tasks(1), null)) + + stage.failureReason = Some("Failed") + listener.onStageCompleted(SparkListenerStageCompleted(stage)) + time += 1 + listener.onJobEnd(SparkListenerJobEnd(1, time, JobFailed( + new RuntimeException("Bad Executor")))) + + time += 1 + tasks(2).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("1", true, Some("Lost executor")), tasks(2), null)) + time += 1 + tasks(3).markFinished(TaskState.FAILED, time) + listener.onTaskEnd(SparkListenerTaskEnd(stage.stageId, stage.attemptNumber, "taskType", + ExecutorLostFailure("2", true, Some("Lost executor")), tasks(3), null)) + + val esummary = store.view(classOf[ExecutorStageSummaryWrapper]).asScala.map(_.info) + esummary.foreach { execSummary => + assert(execSummary.failedTasks === 1) + assert(execSummary.succeededTasks === 1) + assert(execSummary.killedTasks === 0) + } + + val allExecutorSummary = store.view(classOf[ExecutorSummaryWrapper]).asScala.map(_.info) + assert(allExecutorSummary.size === 2) + allExecutorSummary.foreach { allExecSummary => + assert(allExecSummary.failedTasks === 1) + assert(allExecSummary.activeTasks === 0) + assert(allExecSummary.completedTasks === 1) + } + store.delete(classOf[ExecutorSummaryWrapper], "1") + store.delete(classOf[ExecutorSummaryWrapper], "2") + } + } + test("driver logs") { val listener = new AppStatusListener(store, conf, true) @@ -1304,58 +1385,74 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { // receive 3 metric updates from each executor with just stage 0 running, // with different peak updates for each executor listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L))) + Array(4000L, 50L, 20L, 0L, 40L, 0L, 60L, 0L, 70L, 20L, 7500L, 3500L, + 6500L, 2500L, 5500L, 1500L))) listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L))) + Array(1500L, 50L, 20L, 0L, 0L, 0L, 20L, 0L, 70L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // exec 1: new stage 0 peaks for metrics at indexes: 2, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L))) + Array(4000L, 50L, 50L, 0L, 50L, 0L, 100L, 0L, 70L, 20L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 4, 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L))) + Array(2000L, 50L, 10L, 0L, 10L, 0L, 30L, 0L, 70L, 0L, 9000L, 4000L, + 8000L, 3000L, 7000L, 2000L))) // exec 1: new stage 0 peaks for metrics at indexes: 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L))) + Array(2000L, 40L, 50L, 0L, 40L, 10L, 90L, 10L, 50L, 0L, 8000L, 3500L, + 7000L, 2500L, 6000L, 1500L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 5, 6, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L))) + Array(3500L, 50L, 15L, 0L, 10L, 10L, 35L, 10L, 80L, 0L, 8500L, 3500L, + 7500L, 2500L, 6500L, 1500L))) // now start stage 1, one more metric update for each executor, and new // peaks for some stage 1 metrics (as listed), initialize stage 1 peaks listener.onStageSubmitted(createStageSubmittedEvent(1)) // exec 1: new stage 0 peaks for metrics at indexes: 0, 3, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L))) + Array(5000L, 30L, 50L, 20L, 30L, 10L, 80L, 30L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 0 peaks for metrics at indexes: 0, 1, 2, 3, 6, 7, 9 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L))) + Array(7000L, 80L, 50L, 20L, 0L, 10L, 50L, 30L, 10L, 40L, 8000L, 4000L, + 7000L, 3000L, 6000L, 2000L))) // complete stage 0, and 3 more updates for each executor with just // stage 1 running listener.onStageCompleted(createStageCompletedEvent(0)) // exec 1: new stage 1 peaks for metrics at indexes: 0, 1, 3 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L))) + Array(6000L, 70L, 20L, 30L, 10L, 0L, 30L, 30L, 30L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L))) // exec 2: new stage 1 peaks for metrics at indexes: 3, 4, 7, 8 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L))) + Array(5500L, 30L, 20L, 40L, 10L, 0L, 30L, 40L, 40L, 20L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L))) // exec 1: new stage 1 peaks for metrics at indexes: 0, 4, 5, 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L))) + Array(7000L, 70L, 5L, 25L, 60L, 30L, 65L, 55L, 30L, 0L, 3000L, 2500L, 2000L, + 1500L, 1000L, 500L))) // exec 2: new stage 1 peak for metrics at index: 7 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L))) + Array(5500L, 40L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 20L, 7000L, 3000L, + 6000L, 2000L, 5000L, 1000L))) // exec 1: no new stage 1 peaks listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(1, - Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L))) + Array(5500L, 70L, 15L, 20L, 55L, 20L, 70L, 40L, 20L, 0L, 4000L, 2500L, + 3000L, 1500, 2000L, 500L))) listener.onExecutorRemoved(createExecutorRemovedEvent(1)) // exec 2: new stage 1 peak for metrics at index: 6 listener.onExecutorMetricsUpdate(createExecutorMetricsUpdateEvent(2, - Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L))) + Array(4000L, 20L, 25L, 30L, 10L, 30L, 35L, 60L, 0L, 0L, 7000L, 4000L, 6000L, + 3000L, 5000L, 2000L))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values expectedValues.foreach { case (id, metrics) => @@ -1363,8 +1460,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) @@ -1383,23 +1480,29 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { listener.onStageSubmitted(createStageSubmittedEvent(0)) listener.onStageSubmitted(createStageSubmittedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 0, 0, - new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, 70L, 20L)))) + new ExecutorMetrics(Array(5000L, 50L, 50L, 20L, 50L, 10L, 100L, 30L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 0, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 20L, 10L, 10L, 50L, 30L, 80L, 40L, 9000L, + 4000L, 8000L, 3000L, 7000L, 2000L)))) listener.onStageCompleted(createStageCompletedEvent(0)) // executor 1 is removed before stage 1 has finished, the stage executor metrics // are logged afterwards and should still be used to update the executor metrics. listener.onExecutorRemoved(createExecutorRemovedEvent(1)) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("1", 1, 0, - new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L)))) + new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 80L, 55L, 50L, 0L, 5000L, 3000L, + 4000L, 2000L, 3000L, 1000L)))) listener.onStageExecutorMetrics(SparkListenerStageExecutorMetrics("2", 1, 0, - new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L)))) + new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 40L, 40L, 8000L, 5000L, + 7000L, 4000L, 6000L, 3000L)))) listener.onStageCompleted(createStageCompletedEvent(1)) // expected peak values for each executor val expectedValues = Map( - "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, 70L, 20L)), - "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, 80L, 40L))) + "1" -> new ExecutorMetrics(Array(7000L, 70L, 50L, 30L, 60L, 30L, 100L, 55L, + 70L, 20L, 8000L, 4000L, 7000L, 3000L, 6000L, 2000L)), + "2" -> new ExecutorMetrics(Array(7000L, 80L, 50L, 40L, 10L, 30L, 50L, 60L, + 80L, 40L, 9000L, 5000L, 8000L, 4000L, 7000L, 3000L))) // check that the stored peak values match the expected values for ((id, metrics) <- expectedValues) { @@ -1407,8 +1510,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { assert(exec.info.id === id) exec.info.peakMemoryMetrics match { case Some(actual) => - ExecutorMetricType.values.foreach { metricType => - assert(actual.getMetricValue(metricType) === metrics.getMetricValue(metricType)) + ExecutorMetricType.metricToOffset.foreach { metric => + assert(actual.getMetricValue(metric._1) === metrics.getMetricValue(metric._1)) } case _ => assert(false) @@ -1417,6 +1520,106 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { } } + test("storage information on executor lost/down") { + val listener = new AppStatusListener(store, conf, true) + val maxMemory = 42L + + // Register a couple of block managers. + val bm1 = BlockManagerId("1", "1.example.com", 42) + val bm2 = BlockManagerId("2", "2.example.com", 84) + Seq(bm1, bm2).foreach { bm => + listener.onExecutorAdded(SparkListenerExecutorAdded(1L, bm.executorId, + new ExecutorInfo(bm.host, 1, Map.empty, Map.empty))) + listener.onBlockManagerAdded(SparkListenerBlockManagerAdded(1L, bm, maxMemory)) + } + + val rdd1b1 = RddBlock(1, 1, 1L, 2L) + val rdd1b2 = RddBlock(1, 2, 3L, 4L) + val level = StorageLevel.MEMORY_AND_DISK + + // Submit a stage and make sure the RDDs are recorded. + val rdd1Info = new RDDInfo(rdd1b1.rddId, "rdd1", 2, level, Nil) + val stage = new StageInfo(1, 0, "stage1", 4, Seq(rdd1Info), Nil, "details1") + listener.onStageSubmitted(SparkListenerStageSubmitted(stage, new Properties())) + + // Add partition 1 replicated on two block managers. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, rdd1b1.blockId, level, rdd1b1.memSize, rdd1b1.diskSize))) + + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm2, rdd1b1.blockId, level, rdd1b1.memSize, rdd1b1.diskSize))) + + // Add a second partition only to bm 1. + listener.onBlockUpdated(SparkListenerBlockUpdated( + BlockUpdatedInfo(bm1, rdd1b2.blockId, level, rdd1b2.memSize, rdd1b2.diskSize))) + + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 2L) + assert(wrapper.info.memoryUsed === 2 * rdd1b1.memSize + rdd1b2.memSize) + assert(wrapper.info.diskUsed === 2 * rdd1b1.diskSize + rdd1b2.diskSize) + assert(wrapper.info.dataDistribution.get.size === 2L) + assert(wrapper.info.partitions.get.size === 2L) + + val dist = wrapper.info.dataDistribution.get.find(_.address == bm1.hostPort).get + assert(dist.memoryUsed === rdd1b1.memSize + rdd1b2.memSize) + assert(dist.diskUsed === rdd1b1.diskSize + rdd1b2.diskSize) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + val part1 = wrapper.info.partitions.get.find(_.blockName === rdd1b1.blockId.name).get + assert(part1.storageLevel === level.description) + assert(part1.memoryUsed === 2 * rdd1b1.memSize) + assert(part1.diskUsed === 2 * rdd1b1.diskSize) + assert(part1.executors === Seq(bm1.executorId, bm2.executorId)) + + val part2 = wrapper.info.partitions.get.find(_.blockName === rdd1b2.blockId.name).get + assert(part2.storageLevel === level.description) + assert(part2.memoryUsed === rdd1b2.memSize) + assert(part2.diskUsed === rdd1b2.diskSize) + assert(part2.executors === Seq(bm1.executorId)) + } + + check[ExecutorSummaryWrapper](bm1.executorId) { exec => + assert(exec.info.rddBlocks === 2L) + assert(exec.info.memoryUsed === rdd1b1.memSize + rdd1b2.memSize) + assert(exec.info.diskUsed === rdd1b1.diskSize + rdd1b2.diskSize) + } + + // Remove Executor 1. + listener.onExecutorRemoved(createExecutorRemovedEvent(1)) + + // check that partition info now contains only details about what is remaining in bm2 + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 1L) + assert(wrapper.info.memoryUsed === rdd1b1.memSize) + assert(wrapper.info.diskUsed === rdd1b1.diskSize) + assert(wrapper.info.dataDistribution.get.size === 1L) + assert(wrapper.info.partitions.get.size === 1L) + + val dist = wrapper.info.dataDistribution.get.find(_.address == bm2.hostPort).get + assert(dist.memoryUsed === rdd1b1.memSize) + assert(dist.diskUsed === rdd1b1.diskSize) + assert(dist.memoryRemaining === maxMemory - dist.memoryUsed) + + val part = wrapper.info.partitions.get.find(_.blockName === rdd1b1.blockId.name).get + assert(part.storageLevel === level.description) + assert(part.memoryUsed === rdd1b1.memSize) + assert(part.diskUsed === rdd1b1.diskSize) + assert(part.executors === Seq(bm2.executorId)) + } + + // Remove Executor 2. + listener.onExecutorRemoved(createExecutorRemovedEvent(2)) + // Check that storage cost is zero as both exec are down + check[RDDStorageInfoWrapper](rdd1b1.rddId) { wrapper => + assert(wrapper.info.numCachedPartitions === 0) + assert(wrapper.info.memoryUsed === 0) + assert(wrapper.info.diskUsed === 0) + assert(wrapper.info.dataDistribution.isEmpty) + assert(wrapper.info.partitions.get.isEmpty) + } + } + + private def key(stage: StageInfo): Array[Int] = Array(stage.stageId, stage.attemptNumber) private def check[T: ClassTag](key: Any)(fn: T => Unit): Unit = { @@ -1468,7 +1671,8 @@ class AppStatusListenerSuite extends SparkFunSuite with BeforeAndAfter { /** Create an executor added event for the specified executor Id. */ private def createExecutorAddedEvent(executorId: Int) = { - SparkListenerExecutorAdded(0L, executorId.toString, new ExecutorInfo("host1", 1, Map.empty)) + SparkListenerExecutorAdded(0L, executorId.toString, + new ExecutorInfo("host1", 1, Map.empty, Map.empty)) } /** Create an executor added event for the specified executor Id. */ diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala index 92f90f3d96ddf..75a658161d3ff 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusStoreSuite.scala @@ -77,6 +77,34 @@ class AppStatusStoreSuite extends SparkFunSuite { assert(store.count(classOf[CachedQuantile]) === 2) } + test("only successfull task have taskSummary") { + val store = new InMemoryStore() + (0 until 5).foreach { i => store.write(newTaskData(i, status = "FAILED")) } + val appStore = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles) + assert(appStore.size === 0) + } + + test("summary should contain task metrics of only successfull tasks") { + val store = new InMemoryStore() + + for (i <- 0 to 5) { + if (i % 2 == 1) { + store.write(newTaskData(i, status = "FAILED")) + } else { + store.write(newTaskData(i)) + } + } + + val summary = new AppStatusStore(store).taskSummary(stageId, attemptId, uiQuantiles).get + + val values = Array(0.0, 2.0, 4.0) + + val dist = new Distribution(values, 0, values.length).getQuantiles(uiQuantiles.sorted) + dist.zip(summary.executorRunTime).foreach { case (expected, actual) => + assert(expected === actual) + } + } + private def compareQuantiles(count: Int, quantiles: Array[Double]): Unit = { val store = new InMemoryStore() val values = (0 until count).map { i => @@ -93,12 +121,11 @@ class AppStatusStoreSuite extends SparkFunSuite { } } - private def newTaskData(i: Int): TaskDataWrapper = { + private def newTaskData(i: Int, status: String = "SUCCESS"): TaskDataWrapper = { new TaskDataWrapper( - i, i, i, i, i, i, i.toString, i.toString, i.toString, i.toString, false, Nil, None, + i, i, i, i, i, i, i.toString, i.toString, status, i.toString, false, Nil, None, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, i, stageId, attemptId) } - } diff --git a/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala index 9e74e86ad54b9..a01b24d323d28 100644 --- a/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/AppStatusUtilsSuite.scala @@ -52,7 +52,10 @@ class AppStatusUtilsSuite extends SparkFunSuite { inputMetrics = null, outputMetrics = null, shuffleReadMetrics = null, - shuffleWriteMetrics = null))) + shuffleWriteMetrics = null)), + executorLogs = null, + schedulerDelay = 0L, + gettingResultTime = 0L) assert(AppStatusUtils.schedulerDelay(runningTask) === 0L) val finishedTask = new TaskData( @@ -83,7 +86,10 @@ class AppStatusUtilsSuite extends SparkFunSuite { inputMetrics = null, outputMetrics = null, shuffleReadMetrics = null, - shuffleWriteMetrics = null))) + shuffleWriteMetrics = null)), + executorLogs = null, + schedulerDelay = 0L, + gettingResultTime = 0L) assert(AppStatusUtils.schedulerDelay(finishedTask) === 3L) } } diff --git a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala index 07a7b58404c29..38e88e6a012c4 100644 --- a/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/status/ElementTrackingStoreSuite.scala @@ -17,14 +17,60 @@ package org.apache.spark.status +import java.util.concurrent.atomic.{AtomicBoolean, AtomicInteger} + import org.mockito.Mockito._ +import org.scalatest.Matchers._ +import org.scalatest.concurrent.Eventually import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Status._ +import org.apache.spark.status.ElementTrackingStore._ import org.apache.spark.util.kvstore._ -class ElementTrackingStoreSuite extends SparkFunSuite { +class ElementTrackingStoreSuite extends SparkFunSuite with Eventually { + + test("asynchronous tracking single-fire") { + val store = mock(classOf[KVStore]) + val tracking = new ElementTrackingStore(store, new SparkConf() + .set(ASYNC_TRACKING_ENABLED, true)) + + var done = new AtomicBoolean(false) + var type1 = new AtomicInteger(0) + var queued0: WriteQueueResult = null + var queued1: WriteQueueResult = null + var queued2: WriteQueueResult = null + var queued3: WriteQueueResult = null + + tracking.addTrigger(classOf[Type1], 1) { count => + val count = type1.getAndIncrement() + + count match { + case 0 => + // while in the asynchronous thread, attempt to increment twice. The first should + // succeed, the second should be skipped + queued1 = tracking.write(new Type1, checkTriggers = true) + queued2 = tracking.write(new Type1, checkTriggers = true) + case 1 => + // Verify that once we've started deliver again, that we can enqueue another + queued3 = tracking.write(new Type1, checkTriggers = true) + case 2 => + done.set(true) + } + } - import config._ + when(store.count(classOf[Type1])).thenReturn(2L) + queued0 = tracking.write(new Type1, checkTriggers = true) + eventually { + done.get() shouldEqual true + } + + tracking.close(false) + assert(queued0 == WriteQueued) + assert(queued1 == WriteQueued) + assert(queued2 == WriteSkippedQueue) + assert(queued3 == WriteQueued) + } test("tracking for multiple types") { val store = mock(classOf[KVStore]) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala new file mode 100644 index 0000000000000..49cbd66cccb86 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerInfoSuite.scala @@ -0,0 +1,147 @@ +/* + * 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 java.util.{HashMap => JHashMap} + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite + +class BlockManagerInfoSuite extends SparkFunSuite { + + def testWithShuffleServiceOnOff(testName: String) + (f: (Boolean, BlockManagerInfo) => Unit): Unit = { + Seq(true, false).foreach { svcEnabled => + val bmInfo = new BlockManagerInfo( + BlockManagerId("executor0", "host", 1234, None), + timeMs = 300, + Array(), + maxOnHeapMem = 10000, + maxOffHeapMem = 20000, + slaveEndpoint = null, + if (svcEnabled) Some(new JHashMap[BlockId, BlockStatus]) else None) + test(s"$testName externalShuffleServiceEnabled=$svcEnabled") { + f(svcEnabled, bmInfo) + } + } + } + + testWithShuffleServiceOnOff("broadcast block") { (_, bmInfo) => + val broadcastId: BlockId = BroadcastBlockId(0, "field1") + bmInfo.updateBlockInfo( + broadcastId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 100) + assert(bmInfo.blocks.asScala === + Map(broadcastId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 100))) + assert(bmInfo.remainingMem === 29800) + } + + testWithShuffleServiceOnOff("RDD block with MEMORY_ONLY") { (svcEnabled, bmInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, diskSize = 0) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(bmInfo.remainingMem === 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) + } + } + + testWithShuffleServiceOnOff("RDD block with MEMORY_AND_DISK") { (svcEnabled, bmInfo) => + // This is the effective storage level, not the requested storage level, but MEMORY_AND_DISK + // is still possible if it's first in memory, purged to disk, and later promoted back to memory. + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_AND_DISK, memSize = 200, diskSize = 400) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + assert(bmInfo.remainingMem === 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.MEMORY_AND_DISK, 0, 400))) + } + } + + testWithShuffleServiceOnOff("RDD block with DISK_ONLY") { (svcEnabled, bmInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + val exclusiveCachedBlocksForOneMemoryOnly = if (svcEnabled) Set() else Set(rddId) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + } + + testWithShuffleServiceOnOff("update from MEMORY_ONLY to DISK_ONLY") { (svcEnabled, bmInfo) => + // This happens if MEMORY_AND_DISK is the requested storage level, but the block gets purged + // to disk under memory pressure. + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.MEMORY_ONLY, memSize = 200, 0) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.MEMORY_ONLY, 200, 0))) + assert(bmInfo.remainingMem === 29800) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) + } + + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + } + + testWithShuffleServiceOnOff("using invalid StorageLevel") { (svcEnabled, bmInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + + bmInfo.updateBlockInfo(rddId, StorageLevel.NONE, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.isEmpty) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) + } + } + + testWithShuffleServiceOnOff("remove block") { (svcEnabled, bmInfo) => + val rddId: BlockId = RDDBlockId(0, 0) + bmInfo.updateBlockInfo(rddId, StorageLevel.DISK_ONLY, memSize = 0, diskSize = 200) + assert(bmInfo.blocks.asScala === Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.asScala === + Map(rddId -> BlockStatus(StorageLevel.DISK_ONLY, 0, 200))) + } + + bmInfo.removeBlock(rddId) + assert(bmInfo.blocks.asScala.isEmpty) + assert(bmInfo.remainingMem === 30000) + if (svcEnabled) { + assert(bmInfo.externalShuffleServiceBlockStatus.get.isEmpty) + } + } +} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala index 3962bdc27d22c..05a9ac685e5e7 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerReplicationSuite.scala @@ -22,7 +22,6 @@ import java.util.Locale import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ import scala.language.implicitConversions -import scala.language.postfixOps import org.mockito.Mockito.{mock, when} import org.scalatest.{BeforeAndAfter, Matchers} @@ -31,7 +30,8 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging -import org.apache.spark.internal.config.MEMORY_OFFHEAP_SIZE +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.BlockTransferService import org.apache.spark.network.netty.NettyBlockTransferService @@ -69,13 +69,13 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite protected def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - conf.set("spark.testing.memory", maxMem.toString) - conf.set(MEMORY_OFFHEAP_SIZE.key, maxMem.toString) + conf.set(TEST_MEMORY, maxMem) + conf.set(MEMORY_OFFHEAP_SIZE, maxMem) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val store = new BlockManager(name, rpcEnv, master, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, None) memManager.setMemoryStore(store.memoryStore) store.initialize("app-id") allStores += store @@ -84,24 +84,22 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite before { rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - - conf.set("spark.authenticate", "false") - conf.set("spark.driver.port", rpcEnv.address.port.toString) - conf.set("spark.testing", "true") - conf.set("spark.memory.fraction", "1") - conf.set("spark.memory.storageFraction", "1") - conf.set("spark.storage.unrollFraction", "0.4") - conf.set("spark.storage.unrollMemoryThreshold", "512") + conf.set(NETWORK_AUTH_ENABLED, false) + conf.set(DRIVER_PORT, rpcEnv.address.port) + conf.set(IS_TESTING, true) + conf.set(MEMORY_FRACTION, 1.0) + conf.set(MEMORY_STORAGE_FRACTION, 0.999) + conf.set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) // to make a replication attempt to inactive store fail fast conf.set("spark.core.connection.ack.wait.timeout", "1s") // to make cached peers refresh frequently - conf.set("spark.storage.cachedPeersTtl", "10") + conf.set(STORAGE_CACHED_PEERS_TTL, 10) sc = new SparkContext("local", "test", conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) allStores.clear() } @@ -233,11 +231,11 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val failableTransfer = mock(classOf[BlockTransferService]) // this wont actually work when(failableTransfer.hostName).thenReturn("some-hostname") when(failableTransfer.port).thenReturn(1000) - conf.set("spark.testing.memory", "10000") + conf.set(TEST_MEMORY, 10000L) val memManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(serializer, conf) val failableStore = new BlockManager("failable-store", rpcEnv, master, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, failableTransfer, securityMgr, None) memManager.setMemoryStore(failableStore.memoryStore) failableStore.initialize("app-id") allStores += failableStore // so that this gets stopped after test @@ -248,7 +246,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite // Add another normal block manager and test that 2x replication works makeBlockManager(10000, "anotherStore") - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(replicateAndGetNumCopies("a2") === 2) } } @@ -273,14 +271,14 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite // Add another store, 3x replication should work now, 4x replication should only replicate 3x val newStore1 = makeBlockManager(storeSize, s"newstore1") - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(replicateAndGetNumCopies("a3", 3) === 3) } assert(replicateAndGetNumCopies("a4", 4) === 3) // Add another store, 4x replication should work now val newStore2 = makeBlockManager(storeSize, s"newstore2") - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(replicateAndGetNumCopies("a5", 4) === 4) } @@ -296,7 +294,7 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite val newStores = (3 to 5).map { i => makeBlockManager(storeSize, s"newstore$i") } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(replicateAndGetNumCopies("a7", 3) === 3) } } @@ -418,14 +416,14 @@ trait BlockManagerReplicationBehavior extends SparkFunSuite class BlockManagerReplicationSuite extends BlockManagerReplicationBehavior { val conf = new SparkConf(false).set("spark.app.id", "test") - conf.set("spark.kryoserializer.buffer", "1m") + conf.set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") } class BlockManagerProactiveReplicationSuite extends BlockManagerReplicationBehavior { val conf = new SparkConf(false).set("spark.app.id", "test") - conf.set("spark.kryoserializer.buffer", "1m") - conf.set("spark.storage.replication.proactive", "true") - conf.set("spark.storage.exceptionOnPinLeak", "true") + conf.set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + conf.set(STORAGE_REPLICATION_PROACTIVE, true) + conf.set(STORAGE_EXCEPTION_PIN_LEAK, true) (2 to 5).foreach { i => test(s"proactive block replication - $i replicas - ${i - 1} block manager deletions") { @@ -455,13 +453,13 @@ class BlockManagerProactiveReplicationSuite extends BlockManagerReplicationBehav master.removeExecutor(bm.blockManagerId.executorId) bm.stop() // giving enough time for replication to happen and new block be reported to master - eventually(timeout(5 seconds), interval(100 millis)) { + eventually(timeout(5.seconds), interval(100.milliseconds)) { val newLocations = master.getLocations(blockId).toSet assert(newLocations.size === replicationFactor) } } - val newLocations = eventually(timeout(5 seconds), interval(100 millis)) { + val newLocations = eventually(timeout(5.seconds), interval(100.milliseconds)) { val _newLocations = master.getLocations(blockId).toSet assert(_newLocations.size === replicationFactor) _newLocations @@ -473,7 +471,7 @@ class BlockManagerProactiveReplicationSuite extends BlockManagerReplicationBehav "New locations contain stopped block managers.") // Make sure all locks have been released. - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { initialStores.filter(bm => newLocations.contains(bm.blockManagerId)).foreach { bm => assert(bm.blockInfoManager.getTaskLockCount(BlockInfo.NON_TASK_WRITER) === 0) } @@ -498,12 +496,12 @@ class DummyTopologyMapper(conf: SparkConf) extends TopologyMapper(conf) with Log class BlockManagerBasicStrategyReplicationSuite extends BlockManagerReplicationBehavior { val conf: SparkConf = new SparkConf(false).set("spark.app.id", "test") - conf.set("spark.kryoserializer.buffer", "1m") + conf.set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") conf.set( - "spark.storage.replication.policy", + STORAGE_REPLICATION_POLICY, classOf[BasicBlockReplicationPolicy].getName) conf.set( - "spark.storage.replication.topologyMapper", + STORAGE_REPLICATION_TOPOLOGY_MAPPER, classOf[DummyTopologyMapper].getName) } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 32d6e8b94e1a2..2d6e151f81150 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -17,18 +17,20 @@ package org.apache.spark.storage +import java.io.File import java.nio.ByteBuffer import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.duration._ -import scala.language.{implicitConversions, postfixOps} +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.commons.lang3.RandomUtils -import org.mockito.{Matchers => mc} -import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.{ArgumentMatchers => mc} +import org.mockito.Mockito.{doAnswer, mock, spy, times, verify, when} +import org.mockito.invocation.InvocationOnMock import org.scalatest._ import org.scalatest.concurrent.{Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.Eventually._ @@ -36,16 +38,17 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.executor.DataReadMethod +import org.apache.spark.internal.config import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests._ import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.{BlockDataManager, BlockTransferService, TransportContext} -import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer} +import org.apache.spark.network.buffer.{FileSegmentManagedBuffer, ManagedBuffer, NioManagedBuffer} import org.apache.spark.network.client.{RpcResponseCallback, TransportClient} import org.apache.spark.network.netty.{NettyBlockTransferService, SparkTransportConf} import org.apache.spark.network.server.{NoOpRpcHandler, TransportServer, TransportServerBootstrap} -import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager} +import org.apache.spark.network.shuffle.{BlockFetchingListener, DownloadFileManager, ExecutorDiskUtils, ExternalShuffleClient} import org.apache.spark.network.shuffle.protocol.{BlockTransferMessage, RegisterExecutor} -import org.apache.spark.network.util.TransportConf import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.security.{CryptoStreamUtils, EncryptionFunSuite} @@ -65,9 +68,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE implicit val defaultSignaler: Signaler = ThreadSignaler var conf: SparkConf = null - var store: BlockManager = null - var store2: BlockManager = null - var store3: BlockManager = null + val allStores = ArrayBuffer[BlockManager]() var rpcEnv: RpcEnv = null var master: BlockManagerMaster = null val securityMgr = new SecurityManager(new SparkConf(false)) @@ -76,12 +77,23 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val shuffleManager = new SortShuffleManager(new SparkConf(false)) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) + val serializer = new KryoSerializer( + new SparkConf(false).set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m")) // Implicitly convert strings to BlockIds for test clarity. implicit def StringToBlockId(value: String): BlockId = new TestBlockId(value) def rdd(rddId: Int, splitId: Int): RDDBlockId = RDDBlockId(rddId, splitId) + private def init(sparkConf: SparkConf): Unit = { + sparkConf + .set("spark.app.id", "test") + .set(IS_TESTING, true) + .set(MEMORY_FRACTION, 1.0) + .set(MEMORY_STORAGE_FRACTION, 0.999) + .set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m") + .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) + } + private def makeBlockManager( maxMem: Long, name: String = SparkContext.DRIVER_IDENTIFIER, @@ -89,8 +101,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE transferService: Option[BlockTransferService] = Option.empty, testConf: Option[SparkConf] = None): BlockManager = { val bmConf = testConf.map(_.setAll(conf.getAll)).getOrElse(conf) - bmConf.set("spark.testing.memory", maxMem.toString) - bmConf.set(MEMORY_OFFHEAP_SIZE.key, maxMem.toString) + bmConf.set(TEST_MEMORY, maxMem) + bmConf.set(MEMORY_OFFHEAP_SIZE, maxMem) val serializer = new KryoSerializer(bmConf) val encryptionKey = if (bmConf.get(IO_ENCRYPTION_ENABLED)) { Some(CryptoStreamUtils.createKey(bmConf)) @@ -102,9 +114,17 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE .getOrElse(new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1)) val memManager = UnifiedMemoryManager(bmConf, numCores = 1) val serializerManager = new SerializerManager(serializer, bmConf) + val externalShuffleClient = if (conf.get(config.SHUFFLE_SERVICE_ENABLED)) { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", 0) + Some(new ExternalShuffleClient(transConf, bmSecurityMgr, + bmSecurityMgr.isAuthenticationEnabled(), conf.get(config.SHUFFLE_REGISTRATION_TIMEOUT))) + } else { + None + } val blockManager = new BlockManager(name, rpcEnv, master, serializerManager, bmConf, - memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, bmSecurityMgr, externalShuffleClient) memManager.setMemoryStore(blockManager.memoryStore) + allStores += blockManager blockManager.initialize("app-id") blockManager } @@ -114,17 +134,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case System.setProperty("os.arch", "amd64") conf = new SparkConf(false) - .set("spark.app.id", "test") - .set("spark.testing", "true") - .set("spark.memory.fraction", "1") - .set("spark.memory.storageFraction", "1") - .set("spark.kryoserializer.buffer", "1m") - .set("spark.test.useCompressedOops", "true") - .set("spark.storage.unrollFraction", "0.4") - .set("spark.storage.unrollMemoryThreshold", "512") + init(conf) rpcEnv = RpcEnv.create("test", "localhost", 0, conf, securityMgr) - conf.set("spark.driver.port", rpcEnv.address.port.toString) + conf.set(DRIVER_PORT, rpcEnv.address.port) // Mock SparkContext to reduce the memory usage of tests. It's fine since the only reason we // need to create a SparkContext is to initialize LiveListenerBus. @@ -132,7 +145,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE when(sc.conf).thenReturn(conf) master = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) val initialize = PrivateMethod[Unit]('initialize) SizeEstimator invokePrivate initialize() @@ -141,18 +154,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE override def afterEach(): Unit = { try { conf = null - if (store != null) { - store.stop() - store = null - } - if (store2 != null) { - store2.stop() - store2 = null - } - if (store3 != null) { - store3.stop() - store3 = null - } + allStores.foreach(_.stop()) + allStores.clear() rpcEnv.shutdown() rpcEnv.awaitTermination() rpcEnv = null @@ -162,6 +165,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } } + private def stopBlockManager(blockManager: BlockManager): Unit = { + allStores -= blockManager + blockManager.stop() + } + test("StorageLevel object caching") { val level1 = StorageLevel(false, false, false, 3) // this should return the same object as level1 @@ -198,14 +206,13 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(id2_.eq(id1), "Deserialized id2 is not the same object as original id1") } - test("BlockManagerId.isDriver() backwards-compatibility with legacy driver ids (SPARK-6716)") { + test("BlockManagerId.isDriver() with DRIVER_IDENTIFIER (SPARK-27090)") { assert(BlockManagerId(SparkContext.DRIVER_IDENTIFIER, "XXX", 1).isDriver) - assert(BlockManagerId(SparkContext.LEGACY_DRIVER_IDENTIFIER, "XXX", 1).isDriver) assert(!BlockManagerId("notADriverIdentifier", "XXX", 1).isDriver) } test("master + 1 manager interaction") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -235,8 +242,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("master + 2 managers interaction") { - store = makeBlockManager(2000, "exec1") - store2 = makeBlockManager(2000, "exec2") + val store = makeBlockManager(2000, "exec1") + val store2 = makeBlockManager(2000, "exec2") val peers = master.getPeers(store.blockManagerId) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -251,7 +258,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing block") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -279,19 +286,19 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE master.removeBlock("a2-to-remove") master.removeBlock("a3-to-remove") - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(!store.hasLocalBlock("a1-to-remove")) master.getLocations("a1-to-remove") should have size 0 } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(!store.hasLocalBlock("a2-to-remove")) master.getLocations("a2-to-remove") should have size 0 } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(store.hasLocalBlock("a3-to-remove")) master.getLocations("a3-to-remove") should have size 0 } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { val memStatus = master.getMemoryStatus.head._2 memStatus._1 should equal (40000L) memStatus._2 should equal (40000L) @@ -299,7 +306,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing rdd") { - store = makeBlockManager(20000) + val store = makeBlockManager(20000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -309,15 +316,15 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE store.putSingle("nonrddblock", a3, StorageLevel.MEMORY_ONLY) master.removeRdd(0, blocking = false) - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { store.getSingleAndReleaseLock(rdd(0, 0)) should be (None) master.getLocations(rdd(0, 0)) should have size 0 } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { store.getSingleAndReleaseLock(rdd(0, 1)) should be (None) master.getLocations(rdd(0, 1)) should have size 0 } - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { store.getSingleAndReleaseLock("nonrddblock") should not be (None) master.getLocations("nonrddblock") should have size (1) } @@ -332,7 +339,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("removing broadcast") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val driverStore = store val executorStore = makeBlockManager(2000, "executor") val a1 = new Array[Byte](400) @@ -382,7 +389,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // remove broadcast 1 block from both the stores asynchronously // and verify all broadcast 1 blocks have been removed master.removeBroadcast(1, removeFromMaster = true, blocking = false) - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(!driverStore.hasLocalBlock(broadcast1BlockId)) assert(!executorStore.hasLocalBlock(broadcast1BlockId)) } @@ -390,7 +397,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // remove broadcast 2 from both the stores asynchronously // and verify all broadcast 2 blocks have been removed master.removeBroadcast(2, removeFromMaster = true, blocking = false) - eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(!driverStore.hasLocalBlock(broadcast2BlockId)) assert(!driverStore.hasLocalBlock(broadcast2BlockId2)) assert(!executorStore.hasLocalBlock(broadcast2BlockId)) @@ -398,11 +405,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } executorStore.stop() driverStore.stop() - store = null } test("reregistration on heart beat") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -415,11 +421,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val reregister = !master.driverEndpoint.askSync[Boolean]( BlockManagerHeartbeat(store.blockManagerId)) - assert(reregister == true) + assert(reregister) } test("reregistration on block update") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -437,7 +443,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("reregistration doesn't dead lock") { - store = makeBlockManager(2000) + val store = makeBlockManager(2000) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -475,7 +481,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("correct BlockResult returned from get() calls") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](500), new Array[Byte](1000), new Array[Byte](1500)) val list1SizeEstimate = SizeEstimator.estimate(list1.iterator.toArray) @@ -546,27 +552,133 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-9591: getRemoteBytes from another location when Exception throw") { conf.set("spark.shuffle.io.maxRetries", "0") - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") - store3 = makeBlockManager(8000, "executor3") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") + val store3 = makeBlockManager(8000, "executor3") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) store3.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store2.stop() - store2 = null + stopBlockManager(store2) assert(store.getRemoteBytes("list1").isDefined, "list1Get expected to be fetched") - store3.stop() - store3 = null + stopBlockManager(store3) // Should return None instead of throwing an exception: assert(store.getRemoteBytes("list1").isEmpty) } + Seq( + StorageLevel(useDisk = true, useMemory = false, deserialized = false), + StorageLevel(useDisk = true, useMemory = false, deserialized = true), + StorageLevel(useDisk = true, useMemory = false, deserialized = true, replication = 2) + ).foreach { storageLevel => + test(s"SPARK-27622: avoid the network when block requested from same host, $storageLevel") { + conf.set("spark.shuffle.io.maxRetries", "0") + val sameHostBm = makeBlockManager(8000, "sameHost", master) + + val otherHostTransferSrv = spy(sameHostBm.blockTransferService) + doAnswer { _ => + "otherHost" + }.when(otherHostTransferSrv).hostName + val otherHostBm = makeBlockManager(8000, "otherHost", master, Some(otherHostTransferSrv)) + + // This test always uses the cleanBm to get the block. In case of replication + // the block can be added to the otherHostBm as direct disk read will use + // the local disk of sameHostBm where the block is replicated to. + // When there is no replication then block must be added via sameHostBm directly. + val bmToPutBlock = if (storageLevel.replication > 1) otherHostBm else sameHostBm + val array = Array.fill(16)(Byte.MinValue to Byte.MaxValue).flatten + val blockId = "list" + bmToPutBlock.putIterator(blockId, List(array).iterator, storageLevel, tellMaster = true) + + val sameHostTransferSrv = spy(sameHostBm.blockTransferService) + doAnswer { _ => + fail("Fetching over network is not expected when the block is requested from same host") + }.when(sameHostTransferSrv).fetchBlockSync(mc.any(), mc.any(), mc.any(), mc.any(), mc.any()) + val cleanBm = makeBlockManager(8000, "clean", master, Some(sameHostTransferSrv)) + + // check getRemoteBytes + val bytesViaStore1 = cleanBm.getRemoteBytes(blockId) + assert(bytesViaStore1.isDefined) + val expectedContent = sameHostBm.getBlockData(blockId).nioByteBuffer().array() + assert(bytesViaStore1.get.toArray === expectedContent) + + // check getRemoteValues + val valueViaStore1 = cleanBm.getRemoteValues[List.type](blockId) + assert(valueViaStore1.isDefined) + assert(valueViaStore1.get.data.toList.head === array) + } + } + + private def testWithFileDelAfterLocalDiskRead(level: StorageLevel, getValueOrBytes: Boolean) = { + val testedFunc = if (getValueOrBytes) "getRemoteValue()" else "getRemoteBytes()" + val testNameSuffix = s"$level, $testedFunc" + test(s"SPARK-27622: as file is removed fall back to network fetch, $testNameSuffix") { + conf.set("spark.shuffle.io.maxRetries", "0") + // variable to check the usage of the local disk of the remote executor on the same host + var sameHostExecutorTried: Boolean = false + val store2 = makeBlockManager(8000, "executor2", this.master, + Some(new MockBlockTransferService(0))) + val blockId = "list" + val array = Array.fill(16)(Byte.MinValue to Byte.MaxValue).flatten + store2.putIterator(blockId, List(array).iterator, level, true) + val expectedBlockData = store2.getLocalBytes(blockId) + assert(expectedBlockData.isDefined) + val expectedByteBuffer = expectedBlockData.get.toByteBuffer() + val mockTransferService = new MockBlockTransferService(0) { + override def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String, + tempFileManager: DownloadFileManager): ManagedBuffer = { + assert(sameHostExecutorTried, "before using the network local disk of the remote " + + "executor (running on the same host) is expected to be tried") + new NioManagedBuffer(expectedByteBuffer) + } + } + val store1 = makeBlockManager(8000, "executor1", this.master, Some(mockTransferService)) + val spiedStore1 = spy(store1) + doAnswer { inv => + val blockId = inv.getArguments()(0).asInstanceOf[BlockId] + val localDirs = inv.getArguments()(1).asInstanceOf[Array[String]] + val blockSize = inv.getArguments()(2).asInstanceOf[Long] + val res = store1.readDiskBlockFromSameHostExecutor(blockId, localDirs, blockSize) + assert(res.isDefined) + val file = ExecutorDiskUtils.getFile(localDirs, store1.subDirsPerLocalDir, blockId.name) + // delete the file behind the blockId + assert(file.delete()) + sameHostExecutorTried = true + res + }.when(spiedStore1).readDiskBlockFromSameHostExecutor(mc.any(), mc.any(), mc.any()) + + if (getValueOrBytes) { + val valuesViaStore1 = spiedStore1.getRemoteValues(blockId) + assert(sameHostExecutorTried) + assert(valuesViaStore1.isDefined) + assert(valuesViaStore1.get.data.toList.head === array) + } else { + val bytesViaStore1 = spiedStore1.getRemoteBytes(blockId) + assert(sameHostExecutorTried) + assert(bytesViaStore1.isDefined) + assert(bytesViaStore1.get.toByteBuffer === expectedByteBuffer) + } + } + } + + Seq( + StorageLevel(useDisk = true, useMemory = false, deserialized = false), + StorageLevel(useDisk = true, useMemory = false, deserialized = true) + ).foreach { storageLevel => + Seq(true, false).foreach { valueOrBytes => + testWithFileDelAfterLocalDiskRead(storageLevel, valueOrBytes) + } + } + test("SPARK-14252: getOrElseUpdate should still read from remote storage") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") val list1 = List(new Array[Byte](4000)) store2.putIterator( "list1", list1.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) @@ -574,7 +686,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE "list1", StorageLevel.MEMORY_ONLY, ClassTag.Any, - () => throw new AssertionError("attempted to compute locally")).isLeft) + () => fail("attempted to compute locally")).isLeft) } test("in-memory LRU storage") { @@ -594,7 +706,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } private def testInMemoryLRUStorage(storageLevel: StorageLevel): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -613,7 +725,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of same RDD") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -632,7 +744,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("in-memory LRU for partitions of multiple RDDs") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) @@ -655,7 +767,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("on-disk storage") { _conf => - store = makeBlockManager(1200, testConf = Some(_conf)) + val store = makeBlockManager(1200, testConf = Some(_conf)) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -695,7 +807,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE storageLevel: StorageLevel, getAsBytes: Boolean, testConf: SparkConf): Unit = { - store = makeBlockManager(12000, testConf = Some(testConf)) + val store = makeBlockManager(12000, testConf = Some(testConf)) val accessMethod = if (getAsBytes) store.getLocalBytesAndReleaseLock else store.getSingleAndReleaseLock val a1 = new Array[Byte](4000) @@ -724,7 +836,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val a1 = new Array[Byte](4000) val a2 = new Array[Byte](4000) val a3 = new Array[Byte](4000) @@ -746,7 +858,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("in-memory LRU with streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -774,7 +886,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } encryptionTest("LRU with mixed storage levels and streams") { _conf => - store = makeBlockManager(12000, testConf = Some(_conf)) + val store = makeBlockManager(12000, testConf = Some(_conf)) val list1 = List(new Array[Byte](2000), new Array[Byte](2000)) val list2 = List(new Array[Byte](2000), new Array[Byte](2000)) val list3 = List(new Array[Byte](2000), new Array[Byte](2000)) @@ -827,7 +939,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("overly large block") { - store = makeBlockManager(5000) + val store = makeBlockManager(5000) store.putSingle("a1", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.getSingleAndReleaseLock("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](10000), StorageLevel.MEMORY_AND_DISK) @@ -837,78 +949,71 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("block compression") { try { - conf.set("spark.shuffle.compress", "true") - store = makeBlockManager(20000, "exec1") + conf.set(SHUFFLE_COMPRESS, true) + var store = makeBlockManager(20000, "exec1") store.putSingle( ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.shuffle.compress", "false") + conf.set(SHUFFLE_COMPRESS, false) store = makeBlockManager(20000, "exec2") store.putSingle( ShuffleBlockId(0, 0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 10000, "shuffle_0_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.broadcast.compress", "true") + conf.set(BROADCAST_COMPRESS, true) store = makeBlockManager(20000, "exec3") store.putSingle( BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 1000, "broadcast_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.broadcast.compress", "false") + conf.set(BROADCAST_COMPRESS, false) store = makeBlockManager(20000, "exec4") store.putSingle( BroadcastBlockId(0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 10000, "broadcast_0 was compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.rdd.compress", "true") + conf.set(RDD_COMPRESS, true) store = makeBlockManager(20000, "exec5") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 1000, "rdd_0_0 was not compressed") - store.stop() - store = null + stopBlockManager(store) - conf.set("spark.rdd.compress", "false") + conf.set(RDD_COMPRESS, false) store = makeBlockManager(20000, "exec6") store.putSingle(rdd(0, 0), new Array[Byte](10000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 10000, "rdd_0_0 was compressed") - store.stop() - store = null + stopBlockManager(store) // Check that any other block types are also kept uncompressed store = makeBlockManager(20000, "exec7") store.putSingle("other_block", new Array[Byte](10000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 10000, "other_block was compressed") - store.stop() - store = null + stopBlockManager(store) } finally { - System.clearProperty("spark.shuffle.compress") - System.clearProperty("spark.broadcast.compress") - System.clearProperty("spark.rdd.compress") + System.clearProperty(SHUFFLE_COMPRESS.key) + System.clearProperty(BROADCAST_COMPRESS.key) + System.clearProperty(RDD_COMPRESS.key) } } test("block store put failure") { // Use Java serializer so we can create an unserializable error. - conf.set("spark.testing.memory", "1200") + conf.set(TEST_MEMORY, 1200L) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val memoryManager = UnifiedMemoryManager(conf, numCores = 1) val serializerManager = new SerializerManager(new JavaSerializer(conf), conf) - store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, + val store = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, serializerManager, conf, memoryManager, mapOutputTracker, - shuffleManager, transfer, securityMgr, 0) - memoryManager.setMemoryStore(store.memoryStore) + shuffleManager, transfer, securityMgr, None) + allStores += store store.initialize("app-id") // The put should fail since a1 is not serializable. @@ -919,15 +1024,57 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } // Make sure get a1 doesn't hang and returns None. - failAfter(1 second) { + failAfter(1.second) { assert(store.getSingleAndReleaseLock("a1").isEmpty, "a1 should not be in store") } } + def testPutBlockDataAsStream(blockManager: BlockManager, storageLevel: StorageLevel): Unit = { + val message = "message" + val ser = serializer.newInstance().serialize(message).array() + val blockId = new RDDBlockId(0, 0) + val streamCallbackWithId = + blockManager.putBlockDataAsStream(blockId, storageLevel, ClassTag(message.getClass)) + streamCallbackWithId.onData("0", ByteBuffer.wrap(ser)) + streamCallbackWithId.onComplete("0") + val blockStatusOption = blockManager.getStatus(blockId) + assert(!blockStatusOption.isEmpty) + val blockStatus = blockStatusOption.get + assert((blockStatus.diskSize > 0) === !storageLevel.useMemory) + assert((blockStatus.memSize > 0) === storageLevel.useMemory) + assert(blockManager.getBlockData(blockId).nioByteBuffer().array() === ser) + } + + Seq( + "caching" -> StorageLevel.MEMORY_ONLY, + "caching, serialized" -> StorageLevel.MEMORY_ONLY_SER, + "caching on disk" -> StorageLevel.DISK_ONLY + ).foreach { case (name, storageLevel) => + encryptionTest(s"test putBlockDataAsStream with $name") { conf => + init(conf) + val ioEncryptionKey = + if (conf.get(IO_ENCRYPTION_ENABLED)) Some(CryptoStreamUtils.createKey(conf)) else None + val securityMgr = new SecurityManager(conf, ioEncryptionKey) + val serializerManager = new SerializerManager(serializer, conf, ioEncryptionKey) + val transfer = + new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) + val memoryManager = UnifiedMemoryManager(conf, numCores = 1) + val blockManager = new BlockManager(SparkContext.DRIVER_IDENTIFIER, rpcEnv, master, + serializerManager, conf, memoryManager, mapOutputTracker, + shuffleManager, transfer, securityMgr, None) + try { + blockManager.initialize("app-id") + testPutBlockDataAsStream(blockManager, storageLevel) + } finally { + blockManager.stop() + } + } + } + test("turn off updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, false) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) @@ -955,7 +1102,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("updated block statuses") { val conf = new SparkConf() conf.set(TASK_METRICS_TRACK_UPDATED_BLOCK_STATUSES, true) - store = makeBlockManager(12000, testConf = Some(conf)) + val store = makeBlockManager(12000, testConf = Some(conf)) store.registerTask(0) val list = List.fill(2)(new Array[Byte](2000)) val bigList = List.fill(8)(new Array[Byte](2000)) @@ -1053,7 +1200,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("query block statuses") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](2000)) // Tell master. By LRU, only list2 and list3 remains. @@ -1098,7 +1245,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("get matching blocks") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val list = List.fill(2)(new Array[Byte](100)) // insert some blocks @@ -1142,7 +1289,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) store.putSingle(rdd(0, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](4000), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. @@ -1156,7 +1303,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("safely unroll blocks through putIterator (disk)") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val memoryStore = store.memoryStore val diskStore = store.diskStore val smallList = List.fill(40)(new Array[Byte](100)) @@ -1195,7 +1342,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("read-locked blocks cannot be evicted from memory") { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) val arr = new Array[Byte](4000) // First store a1 and a2, both in memory, and a3, on disk only store.putSingle("a1", arr, StorageLevel.MEMORY_ONLY_SER) @@ -1221,7 +1368,7 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE private def testReadWithLossOfOnDiskFiles( storageLevel: StorageLevel, readMethod: BlockManager => Option[_]): Unit = { - store = makeBlockManager(12000) + val store = makeBlockManager(12000) assert(store.putSingle("blockId", new Array[Byte](4000), storageLevel)) assert(store.getStatus("blockId").isDefined) // Directly delete all files from the disk store, triggering failures when reading blocks: @@ -1260,15 +1407,16 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE test("SPARK-13328: refresh block locations (fetch should fail after hitting a threshold)") { val mockBlockTransferService = - new MockBlockTransferService(conf.getInt("spark.block.failures.beforeLocationRefresh", 5)) - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + new MockBlockTransferService(conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) store.putSingle("item", 999L, StorageLevel.MEMORY_ONLY, tellMaster = true) assert(store.getRemoteBytes("item").isEmpty) } test("SPARK-13328: refresh block locations (fetch should succeed after location refresh)") { val maxFailuresBeforeLocationRefresh = - conf.getInt("spark.block.failures.beforeLocationRefresh", 5) + conf.get(BLOCK_FAILURES_BEFORE_LOCATION_REFRESH) val mockBlockManagerMaster = mock(classOf[BlockManagerMaster]) val mockBlockTransferService = new MockBlockTransferService(maxFailuresBeforeLocationRefresh) @@ -1276,17 +1424,18 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE // so that we have a chance to do location refresh val blockManagerIds = (0 to maxFailuresBeforeLocationRefresh) .map { i => BlockManagerId(s"id-$i", s"host-$i", i + 1) } - when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId])).thenReturn( - Option(BlockLocationsAndStatus(blockManagerIds, BlockStatus.empty))) + when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId], mc.any[String])).thenReturn( + Option(BlockLocationsAndStatus(blockManagerIds, BlockStatus.empty, None))) when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn( blockManagerIds) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] assert(block.isDefined) - verify(mockBlockManagerMaster, times(1)).getLocationsAndStatus("item") + verify(mockBlockManagerMaster, times(1)) + .getLocationsAndStatus("item", "MockBlockTransferServiceHost") verify(mockBlockManagerMaster, times(1)).getLocations("item") } @@ -1302,8 +1451,10 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE throw new InterruptedException("Intentional interrupt") } } - store = makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) - store2 = makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) + val store = + makeBlockManager(8000, "executor1", transferService = Option(mockBlockTransferService)) + val store2 = + makeBlockManager(8000, "executor2", transferService = Option(mockBlockTransferService)) intercept[InterruptedException] { store.putSingle("item", "value", StorageLevel.MEMORY_ONLY_2, tellMaster = true) } @@ -1313,8 +1464,8 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE } test("SPARK-17484: master block locations are updated following an invalid remote block fetch") { - store = makeBlockManager(8000, "executor1") - store2 = makeBlockManager(8000, "executor2") + val store = makeBlockManager(8000, "executor1") + val store2 = makeBlockManager(8000, "executor2") store.putSingle("item", "value", StorageLevel.MEMORY_ONLY, tellMaster = true) assert(master.getLocations("item").nonEmpty) store.removeBlock("item", tellMaster = false) @@ -1323,80 +1474,134 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE assert(master.getLocations("item").isEmpty) } + test("SPARK-25888: serving of removed file not detected by shuffle service") { + // although the existence of the file is checked before serving it but a delete can happen + // somewhere after that check + val store = makeBlockManager(8000, "executor1") + val emptyBlockFetcher = new MockBlockTransferService(0) { + override def fetchBlockSync( + host: String, + port: Int, + execId: String, + blockId: String, + tempFileManager: DownloadFileManager): ManagedBuffer = { + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 1) + // empty ManagedBuffer + new FileSegmentManagedBuffer(transConf, new File("missing.file"), 0, 0) + } + } + val store2 = makeBlockManager(8000, "executor2", this.master, Some(emptyBlockFetcher)) + store.putSingle("item", "value", StorageLevel.DISK_ONLY, tellMaster = true) + assert(master.getLocations("item").nonEmpty) + assert(store2.getRemoteBytes("item").isEmpty) + } + + test("test sorting of block locations") { + val localHost = "localhost" + val otherHost = "otherHost" + val store = makeBlockManager(8000, "executor1") + val externalShuffleServicePort = StorageUtils.externalShuffleServicePort(conf) + val port = store.blockTransferService.port + val rack = Some("rack") + val blockManagerWithTopolgyInfo = BlockManagerId( + store.blockManagerId.executorId, + store.blockManagerId.host, + store.blockManagerId.port, + rack) + store.blockManagerId = blockManagerWithTopolgyInfo + val locations = Seq( + BlockManagerId("executor4", otherHost, externalShuffleServicePort, rack), + BlockManagerId("executor3", otherHost, port, rack), + BlockManagerId("executor6", otherHost, externalShuffleServicePort), + BlockManagerId("executor5", otherHost, port), + BlockManagerId("executor2", localHost, externalShuffleServicePort), + BlockManagerId("executor1", localHost, port)) + val sortedLocations = Seq( + BlockManagerId("executor1", localHost, port), + BlockManagerId("executor2", localHost, externalShuffleServicePort), + BlockManagerId("executor3", otherHost, port, rack), + BlockManagerId("executor4", otherHost, externalShuffleServicePort, rack), + BlockManagerId("executor5", otherHost, port), + BlockManagerId("executor6", otherHost, externalShuffleServicePort)) + assert(store.sortLocations(locations) === sortedLocations) + } + test("SPARK-20640: Shuffle registration timeout and maxAttempts conf are working") { val tryAgainMsg = "test_spark_20640_try_again" val timingoutExecutor = "timingoutExecutor" val tryAgainExecutor = "tryAgainExecutor" val succeedingExecutor = "succeedingExecutor" - // a server which delays response 50ms and must try twice for success. - def newShuffleServer(port: Int): (TransportServer, Int) = { - val failure = new Exception(tryAgainMsg) - val success = ByteBuffer.wrap(new Array[Byte](0)) + val failure = new Exception(tryAgainMsg) + val success = ByteBuffer.wrap(new Array[Byte](0)) - var secondExecutorFailedOnce = false - var thirdExecutorFailedOnce = false + var secondExecutorFailedOnce = false + var thirdExecutorFailedOnce = false - val handler = new NoOpRpcHandler { - override def receive( - client: TransportClient, - message: ByteBuffer, - callback: RpcResponseCallback): Unit = { - val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(message) - msgObj match { + val handler = new NoOpRpcHandler { + override def receive( + client: TransportClient, + message: ByteBuffer, + callback: RpcResponseCallback): Unit = { + val msgObj = BlockTransferMessage.Decoder.fromByteBuffer(message) + msgObj match { - case exec: RegisterExecutor if exec.execId == timingoutExecutor => - () // No reply to generate client-side timeout + case exec: RegisterExecutor if exec.execId == timingoutExecutor => + () // No reply to generate client-side timeout - case exec: RegisterExecutor - if exec.execId == tryAgainExecutor && !secondExecutorFailedOnce => - secondExecutorFailedOnce = true - callback.onFailure(failure) + case exec: RegisterExecutor + if exec.execId == tryAgainExecutor && !secondExecutorFailedOnce => + secondExecutorFailedOnce = true + callback.onFailure(failure) - case exec: RegisterExecutor if exec.execId == tryAgainExecutor => - callback.onSuccess(success) + case exec: RegisterExecutor if exec.execId == tryAgainExecutor => + callback.onSuccess(success) - case exec: RegisterExecutor - if exec.execId == succeedingExecutor && !thirdExecutorFailedOnce => - thirdExecutorFailedOnce = true - callback.onFailure(failure) + case exec: RegisterExecutor + if exec.execId == succeedingExecutor && !thirdExecutorFailedOnce => + thirdExecutorFailedOnce = true + callback.onFailure(failure) - case exec: RegisterExecutor if exec.execId == succeedingExecutor => - callback.onSuccess(success) + case exec: RegisterExecutor if exec.execId == succeedingExecutor => + callback.onSuccess(success) - } } } - - val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 0) - val transCtx = new TransportContext(transConf, handler, true) - (transCtx.createServer(port, Seq.empty[TransportServerBootstrap].asJava), port) } - val candidatePort = RandomUtils.nextInt(1024, 65536) - val (server, shufflePort) = Utils.startServiceOnPort(candidatePort, - newShuffleServer, conf, "ShuffleServer") + val transConf = SparkTransportConf.fromSparkConf(conf, "shuffle", numUsableCores = 0) - conf.set(SHUFFLE_SERVICE_ENABLED.key, "true") - conf.set(SHUFFLE_SERVICE_PORT.key, shufflePort.toString) - conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "40") - conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") - var e = intercept[SparkException] { - makeBlockManager(8000, timingoutExecutor) - }.getMessage - assert(e.contains("TimeoutException")) - - conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") - conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") - e = intercept[SparkException] { - makeBlockManager(8000, tryAgainExecutor) - }.getMessage - assert(e.contains(tryAgainMsg)) + Utils.tryWithResource(new TransportContext(transConf, handler, true)) { transCtx => + // a server which delays response 50ms and must try twice for success. + def newShuffleServer(port: Int): (TransportServer, Int) = { + (transCtx.createServer(port, Seq.empty[TransportServerBootstrap].asJava), port) + } - conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") - conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "2") - makeBlockManager(8000, succeedingExecutor) - server.close() + val candidatePort = RandomUtils.nextInt(1024, 65536) + val (server, shufflePort) = Utils.startServiceOnPort(candidatePort, + newShuffleServer, conf, "ShuffleServer") + + conf.set(SHUFFLE_SERVICE_ENABLED.key, "true") + conf.set(SHUFFLE_SERVICE_PORT.key, shufflePort.toString) + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "40") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") + var e = intercept[SparkException] { + makeBlockManager(8000, timingoutExecutor) + }.getMessage + assert(e.contains("TimeoutException")) + + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "1") + e = intercept[SparkException] { + makeBlockManager(8000, tryAgainExecutor) + }.getMessage + assert(e.contains(tryAgainMsg)) + + conf.set(SHUFFLE_REGISTRATION_TIMEOUT.key, "1000") + conf.set(SHUFFLE_REGISTRATION_MAX_ATTEMPTS.key, "2") + makeBlockManager(8000, succeedingExecutor) + server.close() + } } test("fetch remote block to local disk if block size is larger than threshold") { @@ -1407,11 +1612,11 @@ class BlockManagerSuite extends SparkFunSuite with Matchers with BeforeAndAfterE val blockLocations = Seq(BlockManagerId("id-0", "host-0", 1)) val blockStatus = BlockStatus(StorageLevel.DISK_ONLY, 0L, 2000L) - when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId])).thenReturn( - Option(BlockLocationsAndStatus(blockLocations, blockStatus))) + when(mockBlockManagerMaster.getLocationsAndStatus(mc.any[BlockId], mc.any[String])).thenReturn( + Option(BlockLocationsAndStatus(blockLocations, blockStatus, None))) when(mockBlockManagerMaster.getLocations(mc.any[BlockId])).thenReturn(blockLocations) - store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, + val store = makeBlockManager(8000, "executor1", mockBlockManagerMaster, transferService = Option(mockBlockTransferService)) val block = store.getRemoteBytes("item") .asInstanceOf[Option[ByteBuffer]] diff --git a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala index eec961a491101..97b9c973e97f2 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskStoreSuite.scala @@ -39,7 +39,7 @@ class DiskStoreSuite extends SparkFunSuite { // It will cause error when we tried to re-open the filestore and the // memory-mapped byte buffer tot he file has not been GC on Windows. assume(!Utils.isWindows) - val confKey = "spark.storage.memoryMapThreshold" + val confKey = config.STORAGE_MEMORY_MAP_THRESHOLD.key // Create a non-trivial (not all zeros) byte array val bytes = Array.tabulate[Byte](1000)(_.toByte) @@ -128,11 +128,10 @@ class DiskStoreSuite extends SparkFunSuite { assert(e.getMessage === s"requirement failed: can't create a byte buffer of size ${blockData.size}" + - " since it exceeds 10.0 KB.") + " since it exceeds 10.0 KiB.") } test("block data encryption") { - val testDir = Utils.createTempDir() val testData = new Array[Byte](128 * 1024) new Random().nextBytes(testData) diff --git a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala index 42828506895a7..fc16fe362882c 100644 --- a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage import org.apache.spark._ - +import org.apache.spark.internal.config._ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { /* Tests the ability of Spark to deal with user provided iterators from flatMap @@ -55,7 +55,7 @@ class FlatmapIteratorSuite extends SparkFunSuite with LocalSparkContext { test("Serializer Reset") { val sconf = new SparkConf().setMaster("local").setAppName("serializer_reset_test") - .set("spark.serializer.objectStreamReset", "10") + .set(SERIALIZER_OBJECT_STREAM_RESET, 10) sc = new SparkContext(sconf) val expand_size = 500 val data = sc.parallelize(Seq(1, 2)). diff --git a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala index 7274072e5049a..a7231411e81de 100644 --- a/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/MemoryStoreSuite.scala @@ -20,13 +20,13 @@ package org.apache.spark.storage import java.nio.ByteBuffer import scala.language.implicitConversions -import scala.language.reflectiveCalls import scala.reflect.ClassTag import org.scalatest._ import org.apache.spark._ -import org.apache.spark.memory.{MemoryMode, StaticMemoryManager} +import org.apache.spark.internal.config._ +import org.apache.spark.memory.{MemoryMode, UnifiedMemoryManager} import org.apache.spark.serializer.{KryoSerializer, SerializerManager} import org.apache.spark.storage.memory.{BlockEvictionHandler, MemoryStore, PartiallySerializedBlock, PartiallyUnrolledIterator} import org.apache.spark.util._ @@ -39,12 +39,11 @@ class MemoryStoreSuite with ResetSystemProperties { var conf: SparkConf = new SparkConf(false) - .set("spark.test.useCompressedOops", "true") - .set("spark.storage.unrollFraction", "0.4") - .set("spark.storage.unrollMemoryThreshold", "512") + .set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test - val serializer = new KryoSerializer(new SparkConf(false).set("spark.kryoserializer.buffer", "1m")) + val serializer = new KryoSerializer( + new SparkConf(false).set(Kryo.KRYO_SERIALIZER_BUFFER_SIZE.key, "1m")) val serializerManager = new SerializerManager(serializer, conf) @@ -61,10 +60,10 @@ class MemoryStoreSuite } def makeMemoryStore(maxMem: Long): (MemoryStore, BlockInfoManager) = { - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) + val memManager = new UnifiedMemoryManager(conf, maxMem, maxMem / 2, 1) val blockInfoManager = new BlockInfoManager + var memoryStore: MemoryStore = null val blockEvictionHandler = new BlockEvictionHandler { - var memoryStore: MemoryStore = _ override private[storage] def dropFromMemory[T: ClassTag]( blockId: BlockId, data: () => Either[Array[T], ChunkedByteBuffer]): StorageLevel = { @@ -72,10 +71,9 @@ class MemoryStoreSuite StorageLevel.NONE } } - val memoryStore = + memoryStore = new MemoryStore(conf, blockInfoManager, serializerManager, memManager, blockEvictionHandler) memManager.setMemoryStore(memoryStore) - blockEvictionHandler.memoryStore = memoryStore (memoryStore, blockInfoManager) } @@ -240,7 +238,7 @@ class MemoryStoreSuite } test("safely unroll blocks through putIteratorAsBytes") { - val (memoryStore, blockInfoManager) = makeMemoryStore(12000) + val (memoryStore, blockInfoManager) = makeMemoryStore(8400) val smallList = List.fill(40)(new Array[Byte](100)) val bigList = List.fill(40)(new Array[Byte](1000)) def smallIterator: Iterator[Any] = smallList.iterator.asInstanceOf[Iterator[Any]] @@ -418,12 +416,12 @@ class MemoryStoreSuite val bytesPerSmallBlock = memStoreSize / numInitialBlocks def testFailureOnNthDrop(numValidBlocks: Int, readLockAfterDrop: Boolean): Unit = { val tc = TaskContext.empty() - val memManager = new StaticMemoryManager(conf, Long.MaxValue, memStoreSize, numCores = 1) + val memManager = new UnifiedMemoryManager(conf, memStoreSize, memStoreSize.toInt / 2, 1) val blockInfoManager = new BlockInfoManager blockInfoManager.registerTask(tc.taskAttemptId) var droppedSoFar = 0 + var memoryStore: MemoryStore = null val blockEvictionHandler = new BlockEvictionHandler { - var memoryStore: MemoryStore = _ override private[storage] def dropFromMemory[T: ClassTag]( blockId: BlockId, @@ -443,7 +441,7 @@ class MemoryStoreSuite } } } - val memoryStore = new MemoryStore(conf, blockInfoManager, serializerManager, memManager, + memoryStore = new MemoryStore(conf, blockInfoManager, serializerManager, memManager, blockEvictionHandler) { override def afterDropAction(blockId: BlockId): Unit = { if (readLockAfterDrop) { @@ -455,11 +453,10 @@ class MemoryStoreSuite } } - blockEvictionHandler.memoryStore = memoryStore memManager.setMemoryStore(memoryStore) // Put in some small blocks to fill up the memory store - val initialBlocks = (1 to numInitialBlocks).map { id => + (1 to numInitialBlocks).foreach { id => val blockId = BlockId(s"rdd_1_$id") val blockInfo = new BlockInfo(StorageLevel.MEMORY_ONLY, ct, tellMaster = false) val initialWriteLock = blockInfoManager.lockNewBlockForWriting(blockId, blockInfo) diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala index 535105379963a..3dbc1c4b457a8 100644 --- a/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/PartiallySerializedBlockSuite.scala @@ -24,7 +24,6 @@ import scala.reflect.ClassTag import org.mockito.Mockito import org.mockito.Mockito.atLeastOnce import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.apache.spark.{SparkConf, SparkFunSuite, TaskContext, TaskContextImpl} @@ -59,11 +58,9 @@ class PartiallySerializedBlockSuite val bbos: ChunkedByteBufferOutputStream = { val spy = Mockito.spy(new ChunkedByteBufferOutputStream(128, ByteBuffer.allocate)) - Mockito.doAnswer(new Answer[ChunkedByteBuffer] { - override def answer(invocationOnMock: InvocationOnMock): ChunkedByteBuffer = { - Mockito.spy(invocationOnMock.callRealMethod().asInstanceOf[ChunkedByteBuffer]) - } - }).when(spy).toChunkedByteBuffer + Mockito.doAnswer { (invocationOnMock: InvocationOnMock) => + Mockito.spy(invocationOnMock.callRealMethod().asInstanceOf[ChunkedByteBuffer]) + }.when(spy).toChunkedByteBuffer spy } diff --git a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala index cbc903f17ad75..56860b2e55709 100644 --- a/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/PartiallyUnrolledIteratorSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import org.mockito.Matchers +import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar @@ -45,7 +45,7 @@ class PartiallyUnrolledIteratorSuite extends SparkFunSuite with MockitoSugar { joinIterator.hasNext joinIterator.hasNext verify(memoryStore, times(1)) - .releaseUnrollMemoryForThisTask(Matchers.eq(ON_HEAP), Matchers.eq(unrollSize.toLong)) + .releaseUnrollMemoryForThisTask(meq(ON_HEAP), meq(unrollSize.toLong)) // Secondly, iterate over rest iterator (unrollSize until unrollSize + restSize).foreach { value => diff --git a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala index b268195e09a5b..ed402440e74f1 100644 --- a/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/ShuffleBlockFetcherIteratorSuite.scala @@ -17,17 +17,17 @@ package org.apache.spark.storage -import java.io.{File, InputStream, IOException} +import java.io._ +import java.nio.ByteBuffer import java.util.UUID import java.util.concurrent.Semaphore import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.Future -import org.mockito.Matchers.{any, eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{mock, times, verify, when} import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.PrivateMethodTester import org.apache.spark.{SparkFunSuite, TaskContext} @@ -40,15 +40,17 @@ import org.apache.spark.util.Utils class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodTester { + + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + // Some of the tests are quite tricky because we are testing the cleanup behavior // in the presence of faults. /** Creates a mock [[BlockTransferService]] that returns data from the given map. */ private def createMockTransfer(data: Map[BlockId, ManagedBuffer]): BlockTransferService = { val transfer = mock(classOf[BlockTransferService]) - when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())).thenAnswer( + (invocation: InvocationOnMock) => { val blocks = invocation.getArguments()(3).asInstanceOf[Array[String]] val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] @@ -59,8 +61,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT listener.onBlockFetchFailure(blockId, new BlockNotFoundException(blockId)) } } - } - }) + }) transfer } @@ -102,8 +103,10 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT (remoteBmId, remoteBlocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq) ).toIterator + val taskContext = TaskContext.empty() + val metrics = taskContext.taskMetrics.createTempShuffleReadMetrics() val iterator = new ShuffleBlockFetcherIterator( - TaskContext.empty(), + taskContext, transfer, blockManager, blocksByAddress, @@ -112,7 +115,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) + true, + false, + metrics) // 3 local blocks fetched in initialization verify(blockManager, times(3)).getBlockData(any()) @@ -160,8 +165,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + .thenAnswer((invocation: InvocationOnMock) => { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] Future { // Return the first two blocks, and wait till task completion before returning the 3rd one @@ -173,8 +177,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT listener.onBlockFetchSuccess( ShuffleBlockId(0, 2, 0).toString, blocks(ShuffleBlockId(0, 2, 0))) } - } - }) + }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator @@ -190,7 +193,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) + true, + false, + taskContext.taskMetrics.createTempShuffleReadMetrics()) verify(blocks(ShuffleBlockId(0, 0, 0)), times(0)).release() iterator.next()._2.close() // close() first block's input stream @@ -228,8 +233,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + .thenAnswer((invocation: InvocationOnMock) => { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] Future { // Return the first block, and then fail. @@ -241,8 +245,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 2, 0).toString, new BlockNotFoundException("blah")) sem.release() } - } - }) + }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator @@ -258,7 +261,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) + true, + false, + taskContext.taskMetrics.createTempShuffleReadMetrics()) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -270,15 +275,34 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT intercept[FetchFailedException] { iterator.next() } } - private def mockCorruptBuffer(size: Long = 1L): ManagedBuffer = { - val corruptStream = mock(classOf[InputStream]) - when(corruptStream.read(any(), any(), any())).thenThrow(new IOException("corrupt")) + private def mockCorruptBuffer(size: Long = 1L, corruptAt: Int = 0): ManagedBuffer = { + val corruptStream = new CorruptStream(corruptAt) val corruptBuffer = mock(classOf[ManagedBuffer]) when(corruptBuffer.size()).thenReturn(size) when(corruptBuffer.createInputStream()).thenReturn(corruptStream) corruptBuffer } + private class CorruptStream(corruptAt: Long = 0L) extends InputStream { + var pos = 0 + var closed = false + + override def read(): Int = { + if (pos >= corruptAt) { + throw new IOException("corrupt") + } else { + pos += 1 + pos + } + } + + override def read(dest: Array[Byte], off: Int, len: Int): Int = { + super.read(dest, off, len) + } + + override def close(): Unit = { closed = true } + } + test("retry corrupt blocks") { val blockManager = mock(classOf[BlockManager]) val localBmId = BlockManagerId("test-client", "test-client", 1) @@ -298,8 +322,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + .thenAnswer((invocation: InvocationOnMock) => { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] Future { // Return the first block, and then fail. @@ -311,8 +334,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 2, 0).toString, corruptLocalBuffer) sem.release() } - } - }) + }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator @@ -328,7 +350,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) + true, + true, + taskContext.taskMetrics.createTempShuffleReadMetrics()) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -338,8 +362,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT assert(id1 === ShuffleBlockId(0, 0, 0)) when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + .thenAnswer((invocation: InvocationOnMock) => { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] Future { // Return the first block, and then fail. @@ -347,8 +370,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 1, 0).toString, mockCorruptBuffer()) sem.release() } - } - }) + }) // The next block is corrupt local block (the second one is corrupt and retried) intercept[FetchFailedException] { iterator.next() } @@ -357,28 +379,98 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT intercept[FetchFailedException] { iterator.next() } } - test("big blocks are not checked for corruption") { - val corruptBuffer = mockCorruptBuffer(10000L) - + test("big blocks are also checked for corruption") { + val streamLength = 10000L val blockManager = mock(classOf[BlockManager]) - val localBmId = BlockManagerId("test-client", "test-client", 1) - doReturn(localBmId).when(blockManager).blockManagerId - doReturn(corruptBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0)) - val localBlockLengths = Seq[Tuple2[BlockId, Long]]( - ShuffleBlockId(0, 0, 0) -> corruptBuffer.size() + val localBlockManagerId = BlockManagerId("local-client", "local-client", 1) + doReturn(localBlockManagerId).when(blockManager).blockManagerId + + // This stream will throw IOException when the first byte is read + val corruptBuffer1 = mockCorruptBuffer(streamLength, 0) + val blockManagerId1 = BlockManagerId("remote-client-1", "remote-client-1", 1) + val shuffleBlockId1 = ShuffleBlockId(0, 1, 0) + val blockLengths1 = Seq[Tuple2[BlockId, Long]]( + shuffleBlockId1 -> corruptBuffer1.size() ) - val remoteBmId = BlockManagerId("test-client-1", "test-client-1", 2) - val remoteBlockLengths = Seq[Tuple2[BlockId, Long]]( - ShuffleBlockId(0, 1, 0) -> corruptBuffer.size() + val streamNotCorruptTill = 8 * 1024 + // This stream will throw exception after streamNotCorruptTill bytes are read + val corruptBuffer2 = mockCorruptBuffer(streamLength, streamNotCorruptTill) + val blockManagerId2 = BlockManagerId("remote-client-2", "remote-client-2", 2) + val shuffleBlockId2 = ShuffleBlockId(0, 2, 0) + val blockLengths2 = Seq[Tuple2[BlockId, Long]]( + shuffleBlockId2 -> corruptBuffer2.size() ) val transfer = createMockTransfer( - Map(ShuffleBlockId(0, 0, 0) -> corruptBuffer, ShuffleBlockId(0, 1, 0) -> corruptBuffer)) + Map(shuffleBlockId1 -> corruptBuffer1, shuffleBlockId2 -> corruptBuffer2)) + val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( + (blockManagerId1, blockLengths1), + (blockManagerId2, blockLengths2) + ).toIterator + val taskContext = TaskContext.empty() + val maxBytesInFlight = 3 * 1024 + val iterator = new ShuffleBlockFetcherIterator( + taskContext, + transfer, + blockManager, + blocksByAddress, + (_, in) => new LimitedInputStream(in, streamLength), + maxBytesInFlight, + Int.MaxValue, + Int.MaxValue, + Int.MaxValue, + true, + true, + taskContext.taskMetrics.createTempShuffleReadMetrics()) + + // We'll get back the block which has corruption after maxBytesInFlight/3 because the other + // block will detect corruption on first fetch, and then get added to the queue again for + // a retry + val (id, st) = iterator.next() + assert(id === shuffleBlockId2) + + // The other block will throw a FetchFailedException + intercept[FetchFailedException] { + iterator.next() + } + + // Following will succeed as it reads part of the stream which is not corrupt. This will read + // maxBytesInFlight/3 bytes from the portion copied into memory, and remaining from the + // underlying stream + new DataInputStream(st).readFully( + new Array[Byte](streamNotCorruptTill), 0, streamNotCorruptTill) + + // Following will fail as it reads the remaining part of the stream which is corrupt + intercept[FetchFailedException] { st.read() } + // Buffers are mocked and they return the original input corrupt streams + assert(corruptBuffer1.createInputStream().asInstanceOf[CorruptStream].closed) + assert(corruptBuffer2.createInputStream().asInstanceOf[CorruptStream].closed) + } + + test("ensure big blocks available as a concatenated stream can be read") { + val tmpDir = Utils.createTempDir() + val tmpFile = new File(tmpDir, "someFile.txt") + val os = new FileOutputStream(tmpFile) + val buf = ByteBuffer.allocate(10000) + for (i <- 1 to 2500) { + buf.putInt(i) + } + os.write(buf.array()) + os.close() + val managedBuffer = new FileSegmentManagedBuffer(null, tmpFile, 0, 10000) + + val blockManager = mock(classOf[BlockManager]) + val localBmId = BlockManagerId("test-client", "test-client", 1) + doReturn(localBmId).when(blockManager).blockManagerId + doReturn(managedBuffer).when(blockManager).getBlockData(ShuffleBlockId(0, 0, 0)) + val localBlockLengths = Seq[Tuple2[BlockId, Long]]( + ShuffleBlockId(0, 0, 0) -> 10000 + ) + val transfer = createMockTransfer(Map(ShuffleBlockId(0, 0, 0) -> managedBuffer)) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( - (localBmId, localBlockLengths), - (remoteBmId, remoteBlockLengths) + (localBmId, localBlockLengths) ).toIterator val taskContext = TaskContext.empty() @@ -392,10 +484,19 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) - // Blocks should be returned without exceptions. - assert(Set(iterator.next()._1, iterator.next()._1) === - Set(ShuffleBlockId(0, 0, 0), ShuffleBlockId(0, 1, 0))) + true, + true, + taskContext.taskMetrics.createTempShuffleReadMetrics()) + val (id, st) = iterator.next() + // Check that the test setup is correct -- make sure we have a concatenated stream. + assert (st.asInstanceOf[BufferReleasingInputStream].delegate.isInstanceOf[SequenceInputStream]) + + val dst = new DataInputStream(st) + for (i <- 1 to 2500) { + assert(i === dst.readInt()) + } + assert(dst.read() === -1) + dst.close() } test("retry corrupt blocks (disabled)") { @@ -416,8 +517,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = mock(classOf[BlockTransferService]) when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { + .thenAnswer((invocation: InvocationOnMock) => { val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] Future { // Return the first block, and then fail. @@ -429,8 +529,7 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT ShuffleBlockId(0, 2, 0).toString, mockCorruptBuffer()) sem.release() } - } - }) + }) val blocksByAddress = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( (remoteBmId, blocks.keys.map(blockId => (blockId, 1.asInstanceOf[Long])).toSeq)).toIterator @@ -446,7 +545,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - false) + true, + false, + taskContext.taskMetrics.createTempShuffleReadMetrics()) // Continue only after the mock calls onBlockFetchFailure sem.acquire() @@ -480,14 +581,12 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT val transfer = mock(classOf[BlockTransferService]) var tempFileManager: DownloadFileManager = null when(transfer.fetchBlocks(any(), any(), any(), any(), any(), any())) - .thenAnswer(new Answer[Unit] { - override def answer(invocation: InvocationOnMock): Unit = { - val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] - tempFileManager = invocation.getArguments()(5).asInstanceOf[DownloadFileManager] - Future { - listener.onBlockFetchSuccess( - ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0))) - } + .thenAnswer((invocation: InvocationOnMock) => { + val listener = invocation.getArguments()(4).asInstanceOf[BlockFetchingListener] + tempFileManager = invocation.getArguments()(5).asInstanceOf[DownloadFileManager] + Future { + listener.onBlockFetchSuccess( + ShuffleBlockId(0, 0, 0).toString, remoteBlocks(ShuffleBlockId(0, 0, 0))) } }) @@ -496,8 +595,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT // Set `maxBytesInFlight` and `maxReqsInFlight` to `Int.MaxValue`, so that during the // construction of `ShuffleBlockFetcherIterator`, all requests to fetch remote shuffle blocks // are issued. The `maxReqSizeShuffleToMem` is hard-coded as 200 here. + val taskContext = TaskContext.empty() new ShuffleBlockFetcherIterator( - TaskContext.empty(), + taskContext, transfer, blockManager, blocksByAddress, @@ -506,7 +606,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT maxReqsInFlight = Int.MaxValue, maxBlocksInFlightPerAddress = Int.MaxValue, maxReqSizeShuffleToMem = 200, - detectCorrupt = true) + detectCorrupt = true, + false, + taskContext.taskMetrics.createTempShuffleReadMetrics()) } val blocksByAddress1 = Seq[(BlockManagerId, Seq[(BlockId, Long)])]( @@ -552,7 +654,9 @@ class ShuffleBlockFetcherIteratorSuite extends SparkFunSuite with PrivateMethodT Int.MaxValue, Int.MaxValue, Int.MaxValue, - true) + true, + false, + taskContext.taskMetrics.createTempShuffleReadMetrics()) // All blocks fetched return zero length and should trigger a receive-side error: val e = intercept[FetchFailedException] { iterator.next() } diff --git a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala index bbd252d7be7ea..0bc26adeeb443 100644 --- a/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/TopologyMapperSuite.scala @@ -22,6 +22,7 @@ import java.io.{File, FileOutputStream} import org.scalatest.{BeforeAndAfter, Matchers} import org.apache.spark._ +import org.apache.spark.internal.config.STORAGE_REPLICATION_TOPOLOGY_FILE import org.apache.spark.util.Utils class TopologyMapperSuite extends SparkFunSuite @@ -36,7 +37,7 @@ class TopologyMapperSuite extends SparkFunSuite val propsFile = createPropertiesFile(props) val sparkConf = (new SparkConf(false)) - sparkConf.set("spark.storage.replication.topologyFile", propsFile.getAbsolutePath) + sparkConf.set(STORAGE_REPLICATION_TOPOLOGY_FILE, propsFile.getAbsolutePath) val topologyMapper = new FileBasedTopologyMapper(sparkConf) props.foreach {case (host, topology) => diff --git a/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala new file mode 100644 index 0000000000000..c435852a46707 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ui/HttpSecurityFilterSuite.scala @@ -0,0 +1,184 @@ +/* + * 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 + +import java.util.UUID +import javax.servlet.FilterChain +import javax.servlet.http.{HttpServletRequest, HttpServletResponse} + +import scala.collection.JavaConverters._ + +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{mock, times, verify, when} + +import org.apache.spark._ +import org.apache.spark.internal.config.UI._ + +class HttpSecurityFilterSuite extends SparkFunSuite { + + test("filter bad user input") { + val badValues = Map( + "encoded" -> "Encoding:base64%0d%0a%0d%0aPGh0bWw%2bjcmlwdD48L2h0bWw%2b", + "alert1" -> """>"'> + @@ -184,7 +188,8 @@

    {{ page.title }}

    }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; + 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + + '?config=TeX-AMS-MML_HTMLorMML'; d.getElementsByTagName('head')[0].appendChild(script); }(document)); diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 4d0d043a349bb..2d1a9547e3731 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -37,7 +37,7 @@ # Copy over the unified ScalaDoc for all projects to api/scala. # This directory will be copied over to _site when `jekyll` command is run. - source = "../target/scala-2.11/unidoc" + source = "../target/scala-2.12/unidoc" dest = "api/scala" puts "Making directory " + dest diff --git a/docs/api.md b/docs/api.md index 70484f02de78d..241d552d14329 100644 --- a/docs/api.md +++ b/docs/api.md @@ -1,6 +1,21 @@ --- layout: global title: Spark API Documentation +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. --- Here you can read API docs for Spark and its submodules. diff --git a/docs/building-spark.md b/docs/building-spark.md index 55830d38a9e24..5453a218205e2 100644 --- a/docs/building-spark.md +++ b/docs/building-spark.md @@ -2,6 +2,21 @@ layout: global title: Building Spark redirect_from: "building-with-maven.html" +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 will become a table of contents (this text will be scraped). @@ -12,8 +27,8 @@ redirect_from: "building-with-maven.html" ## Apache Maven The Maven-based build is the build of reference for Apache Spark. -Building Spark using Maven requires Maven 3.3.9 or newer and Java 8+. -Note that support for Java 7 was removed as of Spark 2.2.0. +Building Spark using Maven requires Maven 3.6.1 and Java 8. +Spark requires Scala 2.12; support for Scala 2.11 was removed in Spark 3.0.0. ### Setting up Maven's Memory Usage @@ -36,7 +51,7 @@ You can fix these problems by setting the `MAVEN_OPTS` variable as discussed bef ### build/mvn -Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](http://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: +Spark now comes packaged with a self-contained Maven installation to ease building and deployment of Spark from source located under the `build/` directory. This script will automatically download and setup all necessary build requirements ([Maven](https://maven.apache.org/), [Scala](https://www.scala-lang.org/), and [Zinc](https://github.com/typesafehub/zinc)) locally within the `build/` directory itself. It honors any `mvn` binary if present already, however, will pull down its own copy of Scala and Zinc regardless to ensure proper version requirements are met. `build/mvn` execution acts as a pass through to the `mvn` call allowing easy transition from previous build methods. As an example, one can build a version of Spark as follows: ./build/mvn -DskipTests clean package @@ -68,11 +83,14 @@ Example: To enable Hive integration for Spark SQL along with its JDBC server and CLI, add the `-Phive` and `Phive-thriftserver` profiles to your existing build options. -By default Spark will build with Hive 1.2.1 bindings. +By default, Spark will use Hive 1.2.1 with the `hadoop-2.7` profile, and Hive 2.3.5 with the `hadoop-3.2` profile. # With Hive 1.2.1 support ./build/mvn -Pyarn -Phive -Phive-thriftserver -DskipTests clean package + # With Hive 2.3.5 support + ./build/mvn -Pyarn -Phive -Phive-thriftserver -Phadoop-3.2 -DskipTests clean package + ## Packaging without Hadoop Dependencies for YARN The assembly directory produced by `mvn package` will, by default, include all of Spark's @@ -89,15 +107,6 @@ like ZooKeeper and Hadoop itself. ## Building with Kubernetes support ./build/mvn -Pkubernetes -DskipTests clean package - -## Building with Kafka 0.8 support - -Kafka 0.8 support must be explicitly enabled with the `kafka-0-8` profile. -Note: Kafka 0.8 support is deprecated as of Spark 2.3.0. - - ./build/mvn -Pkafka-0-8 -DskipTests clean package - -Kafka 0.10 support is still automatically built. ## Building submodules individually @@ -105,9 +114,9 @@ It's possible to build Spark submodules using the `mvn -pl` option. For instance, you can build the Spark Streaming module using: - ./build/mvn -pl :spark-streaming_2.11 clean install + ./build/mvn -pl :spark-streaming_{{site.SCALA_BINARY_VERSION}} clean install -where `spark-streaming_2.11` is the `artifactId` as defined in `streaming/pom.xml` file. +where `spark-streaming_{{site.SCALA_BINARY_VERSION}}` is the `artifactId` as defined in `streaming/pom.xml` file. ## Continuous Compilation @@ -119,7 +128,7 @@ should run continuous compilation (i.e. wait for changes). However, this has not extensively. A couple of gotchas to note: * it only scans the paths `src/main` and `src/test` (see -[docs](http://davidb.github.io/scala-maven-plugin/example_cc.html)), so it will only work +[docs](https://davidb.github.io/scala-maven-plugin/example_cc.html)), so it will only work from within certain submodules that have that structure. * you'll typically need to run `mvn install` from the project root for compilation within @@ -147,6 +156,14 @@ To avoid the overhead of launching sbt each time you need to re-compile, you can in interactive mode by running `build/sbt`, and then run all build commands at the command prompt. +### Setting up SBT's Memory Usage +Configure the JVM options for SBT in `.jvmopts` at the project root, for example: + + -Xmx2g + -XX:ReservedCodeCacheSize=512m + +For the meanings of these two options, please carefully read the [Setting up Maven's Memory Usage section](https://spark.apache.org/docs/latest/building-spark.html#setting-up-mavens-memory-usage). + ## Speeding up Compilation Developers who compile Spark frequently may want to speed up compilation; e.g., by using Zinc @@ -224,8 +241,8 @@ The run-tests script also can be limited to a specific Python version or a speci To run the SparkR tests you will need to install the [knitr](https://cran.r-project.org/package=knitr), [rmarkdown](https://cran.r-project.org/package=rmarkdown), [testthat](https://cran.r-project.org/package=testthat), [e1071](https://cran.r-project.org/package=e1071) and [survival](https://cran.r-project.org/package=survival) packages first: - R -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='http://cran.us.r-project.org')" - R -e "devtools::install_version('testthat', version = '1.0.2', repos='http://cran.us.r-project.org')" + R -e "install.packages(c('knitr', 'rmarkdown', 'devtools', 'e1071', 'survival'), repos='https://cloud.r-project.org/')" + R -e "devtools::install_version('testthat', version = '1.0.2', repos='https://cloud.r-project.org/')" You can run just the SparkR tests using the command: @@ -239,7 +256,7 @@ Once installed, the `docker` service needs to be started, if not already running On Linux, this can be done by `sudo service docker start`. ./build/mvn install -DskipTests - ./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_2.11 + ./build/mvn test -Pdocker-integration-tests -pl :spark-docker-integration-tests_{{site.SCALA_BINARY_VERSION}} or @@ -247,16 +264,45 @@ or ## Change Scala Version -To build Spark using another supported Scala version, please change the major Scala version using (e.g. 2.12): +When other versions of Scala like 2.13 are supported, it will be possible to build for that version. +Change the major Scala version using (e.g. 2.13): - ./dev/change-scala-version.sh 2.12 + ./dev/change-scala-version.sh 2.13 -For Maven, please enable the profile (e.g. 2.12): +For Maven, please enable the profile (e.g. 2.13): - ./build/mvn -Pscala-2.12 compile + ./build/mvn -Pscala-2.13 compile -For SBT, specify a complete scala version using (e.g. 2.12.6): +For SBT, specify a complete scala version using (e.g. 2.13.0): - ./build/sbt -Dscala.version=2.12.6 + ./build/sbt -Dscala.version=2.13.0 Otherwise, the sbt-pom-reader plugin will use the `scala.version` specified in the spark-parent pom. + +## Running Jenkins tests with Github Enterprise + +To run tests with Jenkins: + + ./dev/run-tests-jenkins + +If use an individual repository or a repository on GitHub Enterprise, export below environment variables before running above command. + +### Related environment variables + + + + + + + + + + + + + +
    Variable NameDefaultMeaning
    SPARK_PROJECT_URLhttps://github.com/apache/spark + The Spark project URL of GitHub Enterprise. +
    GITHUB_API_BASEhttps://api.github.com/repos/apache/spark + The Spark project API server URL of GitHub Enterprise. +
    diff --git a/docs/cloud-integration.md b/docs/cloud-integration.md index 36753f6373b55..b64ffe55d8359 100644 --- a/docs/cloud-integration.md +++ b/docs/cloud-integration.md @@ -3,20 +3,22 @@ layout: global displayTitle: Integration with Cloud Infrastructures title: Integration with Cloud Infrastructures description: Introduction to cloud storage support in Apache Spark SPARK_VERSION_SHORT ---- - + limitations under the License. +--- * This will become a table of contents (this text will be scraped). {:toc} @@ -85,8 +87,9 @@ is set to the chosen version of Spark: ... org.apache.spark - hadoop-cloud_2.11 + hadoop-cloud_{{site.SCALA_BINARY_VERSION}} ${spark.version} + provided ... diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7277e2fb2731d..cb6d3a5c1f7ce 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -1,6 +1,21 @@ --- layout: global title: Cluster Mode Overview +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 document gives a short overview of how Spark runs on clusters, to make it easier to understand @@ -45,7 +60,7 @@ There are several useful things to note about this architecture: # Cluster Manager Types -The system currently supports three cluster managers: +The system currently supports several cluster managers: * [Standalone](spark-standalone.html) -- a simple cluster manager included with Spark that makes it easy to set up a cluster. diff --git a/docs/configuration.md b/docs/configuration.md index 613e214783d59..108862416f8da 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -2,6 +2,21 @@ layout: global displayTitle: Spark Configuration title: Configuration +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 will become a table of contents (this text will be scraped). {:toc} @@ -166,10 +181,45 @@ of the most common options to set are:
    spark.driver.memoryOverhead driverMemory * 0.10, with minimum of 384 - The amount of off-heap memory to be allocated per driver in cluster mode, in MiB unless - otherwise specified. This is memory that accounts for things like VM overheads, interned strings, + Amount of non-heap memory to be allocated per driver process in cluster mode, in MiB unless + otherwise specified. This is memory that accounts for things like VM overheads, interned strings, other native overheads, etc. This tends to grow with the container size (typically 6-10%). - This option is currently supported on YARN and Kubernetes. + This option is currently supported on YARN, Mesos and Kubernetes. + Note: Non-heap memory includes off-heap memory + (when spark.memory.offHeap.enabled=true) and memory used by other driver processes + (e.g. python process that goes with a PySpark driver) and memory used by other non-driver + processes running in the same container. The maximum memory size of container to running + driver is determined by the sum of spark.driver.memoryOverhead + and spark.driver.memory. +
    spark.driver.resource.{resourceName}.amount0 + Amount of a particular resource type to use on the driver. + If this is used, you must also specify the + spark.driver.resource.{resourceName}.discoveryScript + for the driver to find the resource on startup. +
    spark.driver.resource.{resourceName}.discoveryScriptNone + A script for the driver to run to discover a particular resource type. This should + write to STDOUT a JSON string in the format of the ResourceInformation class. This has a + name and an array of addresses. +
    spark.driver.resource.{resourceName}.vendorNone + Vendor of the resources to use for the driver. This option is currently + only supported on Kubernetes and is actually both the vendor and domain following + the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes + this config would be set to nvidia.com or amd.com)
    spark.executor.memoryOverhead executorMemory * 0.10, with minimum of 384 - The amount of off-heap memory to be allocated per executor, in MiB unless otherwise specified. - This is memory that accounts for things like VM overheads, interned strings, other native - overheads, etc. This tends to grow with the executor size (typically 6-10%). + Amount of non-heap memory to be allocated per executor process in cluster mode, in MiB unless + otherwise specified. This is memory that accounts for things like VM overheads, interned strings, + other native overheads, etc. This tends to grow with the executor size (typically 6-10%). This option is currently supported on YARN and Kubernetes. +
    + Note: Non-heap memory includes off-heap memory + (when spark.memory.offHeap.enabled=true) and memory used by other executor processes + (e.g. python process that goes with a PySpark executor) and memory used by other non-executor + processes running in the same container. The maximum memory size of container to running executor + is determined by the sum of spark.executor.memoryOverhead and + spark.executor.memory. +
    spark.executor.resource.{resourceName}.amount0 + Amount of a particular resource type to use per executor process. + If this is used, you must also specify the + spark.executor.resource.{resourceName}.discoveryScript + for the executor to find the resource on startup. +
    spark.executor.resource.{resourceName}.discoveryScriptNone + A script for the executor to run to discover a particular resource type. This should + write to STDOUT a JSON string in the format of the ResourceInformation class. This has a + name and an array of addresses. +
    spark.executor.resource.{resourceName}.vendorNone + Vendor of the resources to use for the executors. This option is currently + only supported on Kubernetes and is actually both the vendor and domain following + the Kubernetes device plugin naming convention. (e.g. For GPUs on Kubernetes + this config would be set to nvidia.com or amd.com)
    spark.driver.log.dfsDir(none) + Base directory in which Spark driver logs are synced, if spark.driver.log.persistToDfs.enabled + is true. Within this base directory, each application logs the driver logs to an application specific file. + Users may want to set this to a unified location like an HDFS directory so driver log files can be persisted + for later usage. This directory should allow any Spark user to read/write files and the Spark History Server + user to delete files. Additionally, older logs from this directory are cleaned by the + Spark History Server if + spark.history.fs.driverlog.cleaner.enabled is true and, if they are older than max age configured + by setting spark.history.fs.driverlog.cleaner.maxAge. +
    spark.driver.log.persistToDfs.enabledfalse + If true, spark application running in client mode will write driver logs to a persistent storage, configured + in spark.driver.log.dfsDir. If spark.driver.log.dfsDir is not configured, driver logs + will not be persisted. Additionally, enable the cleaner by setting spark.history.fs.driverlog.cleaner.enabled + to true in Spark History Server. +
    spark.driver.log.layout%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n + The layout for the driver logs that are synced to spark.driver.log.dfsDir. If this is not configured, + it uses the layout for the first appender defined in log4j.properties. If that is also not configured, driver logs + use the default layout. +
    Apart from these, the following properties are also available, and may be useful in some situations: @@ -286,11 +410,31 @@ Apart from these, the following properties are also available, and may be useful your default properties file. + + spark.driver.defaultJavaOptions + (none) + + A string of default JVM options to prepend to spark.driver.extraJavaOptions. + This is intended to be set by administrators. + + For instance, GC settings or other logging. + Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap + size settings can be set with spark.driver.memory in the cluster mode and through + the --driver-memory command line option in the client mode. + +
    Note: In client mode, this config must not be set through the SparkConf + directly in your application, because the driver JVM has already started at that point. + Instead, please set this through the --driver-java-options command line option or in + your default properties file. + + spark.driver.extraJavaOptions (none) - A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. + A string of extra JVM options to pass to the driver. This is intended to be set by users. + + For instance, GC settings or other logging. Note that it is illegal to set maximum heap size (-Xmx) settings with this option. Maximum heap size settings can be set with spark.driver.memory in the cluster mode and through the --driver-memory command line option in the client mode. @@ -299,6 +443,8 @@ Apart from these, the following properties are also available, and may be useful directly in your application, because the driver JVM has already started at that point. Instead, please set this through the --driver-java-options command line option or in your default properties file. + + spark.driver.defaultJavaOptions will be prepended to this configuration. @@ -333,11 +479,31 @@ Apart from these, the following properties are also available, and may be useful this option. + + spark.executor.defaultJavaOptions + (none) + + A string of default JVM options to prepend to spark.executor.extraJavaOptions. + This is intended to be set by administrators. + + For instance, GC settings or other logging. + Note that it is illegal to set Spark properties or maximum heap size (-Xmx) settings with this + option. Spark properties should be set using a SparkConf object or the spark-defaults.conf file + used with the spark-submit script. Maximum heap size settings can be set with spark.executor.memory. + + The following symbols, if present will be interpolated: {{APP_ID}} will be replaced by + application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable + 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 + + spark.executor.extraJavaOptions (none) - A string of extra JVM options to pass to executors. For instance, GC settings or other logging. + A string of extra JVM options to pass to executors. This is intended to be set by users. + + For instance, GC settings or other logging. Note that it is illegal to set Spark properties or maximum heap size (-Xmx) settings with this option. Spark properties should be set using a SparkConf object or the spark-defaults.conf file used with the spark-submit script. Maximum heap size settings can be set with spark.executor.memory. @@ -346,6 +512,8 @@ Apart from these, the following properties are also available, and may be useful application ID and {{EXECUTOR_ID}} will be replaced by executor ID. For example, to enable 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 + + spark.executor.defaultJavaOptions will be prepended to this configuration. @@ -419,7 +587,7 @@ Apart from these, the following properties are also available, and may be useful spark.redaction.regex - (?i)secret|password + (?i)secret|password|token Regex to decide which Spark configuration properties and environment variables in driver and executor environments contain sensitive information. When this regex matches a property key or @@ -445,7 +613,7 @@ Apart from these, the following properties are also available, and may be useful The directory which is used to dump the profile result before driver exiting. The results will be dumped as separated file for each RDD. They can be loaded - by ptats.Stats(). If this is specified, the profile result will not be displayed + by pstats.Stats(). If this is specified, the profile result will not be displayed automatically. @@ -465,7 +633,7 @@ Apart from these, the following properties are also available, and may be useful Reuse Python worker or not. If yes, it will use a fixed number of Python workers, does not need to fork() a Python process for every task. It will be very useful - if there is large broadcast, then the broadcast will not be needed to transferred + if there is a large broadcast, then the broadcast will not need to be transferred from JVM to Python worker for every task. @@ -591,19 +759,6 @@ Apart from these, the following properties are also available, and may be useful You can mitigate this issue by setting it to a lower value. - - spark.maxRemoteBlockSizeFetchToMem - Int.MaxValue - 512 - - The remote block will be fetched to disk when size of the block is above this threshold in bytes. - This is to avoid a giant request that takes too much memory. By default, this is only enabled - for blocks > 2GB, as those cannot be fetched directly into memory, no matter what resources are - available. But it can be turned down to a much lower value (eg. 200m) to avoid using too much - memory on smaller blocks as well. Note this configuration will affect both shuffle fetch - and block manager remote block fetch. For users who enabled external shuffle service, - this feature can only be used when external shuffle service is newer than Spark 2.2. - - spark.shuffle.compress true @@ -656,6 +811,17 @@ Apart from these, the following properties are also available, and may be useful is 15 seconds by default, calculated as maxRetries * retryWait. + + spark.shuffle.io.backLog + 64 + + Length of the accept queue for the shuffle service. For large applications, this value may + need to be increased, so that incoming connections are not dropped if the service cannot keep + up with a large number of connections arriving in a short period of time. This needs to + be configured wherever the shuffle service itself is running, which may be outside of the + application (see spark.shuffle.service.enabled option below). + + spark.shuffle.service.enabled false @@ -758,7 +924,25 @@ Apart from these, the following properties are also available, and may be useful false Whether to compress logged events, if spark.eventLog.enabled is true. - Compression will use spark.io.compression.codec. + + + + spark.eventLog.compression.codec + + + The codec to compress logged events. If this is not given, + spark.io.compression.codec will be used. + + + + spark.eventLog.allowErasureCoding + false + + Whether to allow event logs to use erasure coding, or turn erasure coding off, regardless of + filesystem defaults. On HDFS, erasure coded files will not update as quickly as regular + replicated files, so the application updates will take longer to appear in the History Server. + Note that even if this is true, Spark will still not force the file to use erasure coding, it + will simply use filesystem defaults. @@ -823,6 +1007,14 @@ Apart from these, the following properties are also available, and may be useful operations that we can live without when rapidly processing incoming task events. + + spark.ui.liveUpdate.minFlushPeriod + 1s + + Minimum time elapsed before stale UI data is flushed. This avoids UI staleness when incoming + task events are not fired frequently. + + spark.ui.port 4040 @@ -877,6 +1069,21 @@ Apart from these, the following properties are also available, and may be useful progress bars will be displayed on the same line. + + spark.ui.custom.executor.log.url + (none) + + Specifies custom spark executor log URL for supporting external log service instead of using cluster + managers' application log URLs in Spark UI. Spark will support some path variables via patterns + which can vary on cluster manager. Please check the documentation for your cluster manager to + see which patterns are supported, if any.

    + Please note that this configuration also replaces original log urls in event log, + which will be also effective when accessing the application on history server. The new log urls must be + permanent, otherwise you might have dead link for executor log urls. +

    + For now, only YARN mode supports this configuration + + spark.worker.ui.retainedExecutors 1000 @@ -929,6 +1136,14 @@ Apart from these, the following properties are also available, and may be useful
    spark.com.test.filter1.param.name2=bar + + spark.ui.requestHeaderSize + 8k + + The maximum allowed size for a HTTP request header, in bytes unless otherwise specified. + This setting applies for the Spark History Server too. + + ### Compression and Serialization @@ -943,6 +1158,14 @@ Apart from these, the following properties are also available, and may be useful Compression will use spark.io.compression.codec. + + spark.checkpoint.compress + false + + Whether to compress RDD checkpoints. Generally a good idea. + Compression will use spark.io.compression.codec. + + spark.io.compression.codec lz4 @@ -1132,6 +1355,9 @@ Apart from these, the following properties are also available, and may be useful If true, Spark will attempt to use off-heap memory for certain operations. If off-heap memory use is enabled, then spark.memory.offHeap.size must be positive. + Note: If off-heap memory is enabled, may need to raise the non-heap memory size + (e.g. increase spark.driver.memoryOverhead or + spark.executor.memoryOverhead). @@ -1144,51 +1370,6 @@ Apart from these, the following properties are also available, and may be useful This must be set to a positive value when spark.memory.offHeap.enabled=true. - - spark.memory.useLegacyMode - false - - Whether to enable the legacy memory management mode used in Spark 1.5 and before. - The legacy mode rigidly partitions the heap space into fixed-size regions, - potentially leading to excessive spilling if the application was not tuned. - The following deprecated memory fraction configurations are not read unless this is enabled: - spark.shuffle.memoryFraction
    - spark.storage.memoryFraction
    - spark.storage.unrollFraction - - - - spark.shuffle.memoryFraction - 0.2 - - (deprecated) This is read only if spark.memory.useLegacyMode is enabled. - Fraction of Java heap to use for aggregation and cogroups during shuffles. - At any given time, the collective size of - all in-memory maps used for shuffles is bounded by this limit, beyond which the contents will - begin to spill to disk. If spills are often, consider increasing this value at the expense of - spark.storage.memoryFraction. - - - - spark.storage.memoryFraction - 0.6 - - (deprecated) This is read only if spark.memory.useLegacyMode is enabled. - Fraction of Java heap to use for Spark's memory cache. This should not be larger than the "old" - generation of objects in the JVM, which by default is given 0.6 of the heap, but you can - increase it if you configure your own old generation size. - - - - spark.storage.unrollFraction - 0.2 - - (deprecated) This is read only if spark.memory.useLegacyMode is enabled. - Fraction of spark.storage.memoryFraction to use for unrolling blocks in memory. - This is dynamically allocated by dropping existing blocks when there is not enough free - storage space to unroll the new block in its entirety. - - spark.storage.replication.proactive false @@ -1330,14 +1511,14 @@ Apart from these, the following properties are also available, and may be useful spark.files.maxPartitionBytes - 134217728 (128 MB) + 134217728 (128 MiB) The maximum number of bytes to pack into a single partition when reading files. spark.files.openCostInBytes - 4194304 (4 MB) + 4194304 (4 MiB) The estimated cost to open a file, measured by the number of bytes could be scanned at the same time. This is used when putting multiple files into a partition. It is better to overestimate, @@ -1391,7 +1572,7 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.message.maxSize 128 - Maximum message size (in MB) to allow in "control plane" communication; generally only applies to map + Maximum message size (in MiB) to allow in "control plane" communication; generally only applies to map output size information sent between executors and the driver. Increase this if you are running jobs with many thousands of map and reduce tasks and see messages about the RPC message size. @@ -1440,6 +1621,15 @@ Apart from these, the following properties are also available, and may be useful This is used for communicating with the executors and the standalone Master. + + spark.rpc.io.backLog + 64 + + Length of the accept queue for the RPC server. For large applications, this value may + need to be increased, so that incoming connections are not dropped when a large number of + connections arrives in a short period of time. + + spark.network.timeout 120s @@ -1451,6 +1641,16 @@ Apart from these, the following properties are also available, and may be useful spark.rpc.lookupTimeout if they are not configured. + + spark.network.io.preferDirectBufs + true + + If enabled then off-heap buffer allocations are preferred by the shared allocators. + Off-heap buffers are used to reduce garbage collection during shuffle and cache + 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. + + spark.port.maxRetries 16 @@ -1500,6 +1700,17 @@ Apart from these, the following properties are also available, and may be useful you can set larger value. + + spark.maxRemoteBlockSizeFetchToMem + 200m + + Remote block will be fetched to disk when size of the block is above this threshold + in bytes. This is to avoid a giant request takes too much memory. Note this + configuration will affect both shuffle fetch and block manager remote block fetch. + For users who enabled external shuffle service, this feature can only work when + external shuffle service is at least 2.3.0. + + ### Scheduling @@ -1599,6 +1810,14 @@ Apart from these, the following properties are also available, and may be useful driver using more memory. + + spark.scheduler.blacklist.unschedulableTaskSetTimeout + 120s + + 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. + + spark.blacklist.enabled @@ -1727,6 +1946,15 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + spark.task.resource.{resourceName}.amount + 1 + + Amount of a particular resource type to allocate for each task. If this is specified + you must also provide the executor config spark.executor.resource.{resourceName}.amount + and any corresponding discovery configs so that your executors are created with that resource type. + + spark.task.maxFailures 4 @@ -1886,8 +2114,66 @@ Apart from these, the following properties are also available, and may be useful description. + + spark.dynamicAllocation.shuffleTracking.enabled + false + + Experimental. Enables shuffle file tracking for executors, which allows dynamic allocation + without the need for an external shuffle service. This option will try to keep alive executors + that are storing shuffle data for active jobs. + + + + spark.dynamicAllocation.shuffleTimeout + infinity + + When shuffle tracking is enabled, controls the timeout for executors that are holding shuffle + data. The default value means that Spark will rely on the shuffles being garbage collected to be + able to release executors. If for some reason garbage collection is not cleaning up shuffles + quickly enough, this option can be used to control when to time out executors even when they are + storing shuffle data. + + +### Thread Configurations + +Depending on jobs and cluster configurations, we can set number of threads in several places in Spark to utilize +available resources efficiently to get better performance. Prior to Spark 3.0, these thread configurations apply +to all roles of Spark, such as driver, executor, worker and master. From Spark 3.0, we can configure threads in +finer granularity starting from driver and executor. Take RPC module as example in below table. For other modules, +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
    spark.{driver|executor}.rpc.io.serverThreads + Fall back on spark.rpc.io.serverThreads + Number of threads used in the server thread pool
    spark.{driver|executor}.rpc.io.clientThreads + Fall back on spark.rpc.io.clientThreads + Number of threads used in the client thread pool
    spark.{driver|executor}.rpc.netty.dispatcher.numThreads + Fall back on spark.rpc.netty.dispatcher.numThreads + Number of threads used in RPC message dispatcher thread pool
    + +The default value for number of thread-related config keys is the minimum of the number of cores requested for +the driver or executor, or, in the absence of that value, the number of cores available for the JVM (with a hardcoded upper limit of 8). + + ### Security Please refer to the [Security](security.html) page for available options on how to secure different @@ -2017,7 +2303,7 @@ showDF(properties, numRows = 200, truncate = FALSE) Maximum rate (number of records per second) at which data will be read from each Kafka partition when using the new Kafka direct stream API. See the - Kafka Integration guide + Kafka Integration guide for more details. @@ -2029,16 +2315,6 @@ showDF(properties, numRows = 200, truncate = FALSE) partition when using the new Kafka direct stream API. - - spark.streaming.kafka.maxRetries - 1 - - Maximum number of consecutive retries the driver will make in order to find - the latest offsets on the leader of each partition (a default value of 1 - means that the driver will make a maximum of 2 attempts). Only applies to - the new Kafka direct stream API. - - spark.streaming.ui.retainedBatches 1000 @@ -2263,7 +2539,7 @@ In some cases, you may want to avoid hard-coding certain configurations in a `Sp instance, Spark allows you to simply create an empty conf and set spark/spark hadoop properties. {% highlight scala %} -val conf = new SparkConf().set("spark.hadoop.abc.def","xyz") +val conf = new SparkConf().set("spark.hadoop.abc.def", "xyz") val sc = new SparkContext(conf) {% endhighlight %} diff --git a/docs/contributing-to-spark.md b/docs/contributing-to-spark.md index ede5584a0cf99..1e4cf4c0c153f 100644 --- a/docs/contributing-to-spark.md +++ b/docs/contributing-to-spark.md @@ -1,6 +1,21 @@ --- layout: global title: Contributing to Spark +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. --- The Spark team welcomes all forms of contributions, including bug reports, documentation or patches. diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 35293348e3f3d..c96f7aaba88e6 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -3,6 +3,21 @@ layout: global displayTitle: GraphX Programming Guide title: GraphX description: GraphX graph processing library guide for Spark SPARK_VERSION_SHORT +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 will become a table of contents (this text will be scraped). @@ -283,7 +298,7 @@ class Graph[VD, ED] { // Functions for caching graphs ================================================================== def persist(newLevel: StorageLevel = StorageLevel.MEMORY_ONLY): Graph[VD, ED] def cache(): Graph[VD, ED] - def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] + def unpersistVertices(blocking: Boolean = false): Graph[VD, ED] // Change the partitioning heuristic ============================================================ def partitionBy(partitionStrategy: PartitionStrategy): Graph[VD, ED] // Transform vertex and edge attributes ========================================================== @@ -317,7 +332,7 @@ class Graph[VD, ED] { // Iterative graph-parallel computation ========================================================== def pregel[A](initialMsg: A, maxIterations: Int, activeDirection: EdgeDirection)( vprog: (VertexId, VD, A) => VD, - sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId,A)], + sendMsg: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], mergeMsg: (A, A) => A) : Graph[VD, ED] // Basic graph algorithms ======================================================================== @@ -522,7 +537,7 @@ val joinedGraph = graph.joinVertices(uniqueCosts, A key step in many graph analytics tasks is aggregating information about the neighborhood of each vertex. -For example, we might want to know the number of followers each user has or the average age of the +For example, we might want to know the number of followers each user has or the average age of the followers of each user. Many iterative graph algorithms (e.g., PageRank, Shortest Path, and connected components) repeatedly aggregate properties of neighboring vertices (e.g., current PageRank Value, shortest path to the source, and smallest reachable vertex id). @@ -700,7 +715,7 @@ a new value for the vertex property, and then send messages to neighboring verti super step. Unlike Pregel, messages are computed in parallel as a function of the edge triplet and the message computation has access to both the source and destination vertex attributes. Vertices that do not receive a message are skipped within a super -step. The Pregel operators terminates iteration and returns the final graph when there are no +step. The Pregel operator terminates iteration and returns the final graph when there are no messages remaining. > Note, unlike more standard Pregel implementations, vertices in GraphX can only send messages to @@ -726,7 +741,7 @@ class GraphOps[VD, ED] { var g = mapVertices( (vid, vdata) => vprog(vid, vdata, initialMsg) ).cache() // compute the messages - var messages = g.mapReduceTriplets(sendMsg, mergeMsg) + var messages = GraphXUtils.mapReduceTriplets(g, sendMsg, mergeMsg) var activeMessages = messages.count() // Loop until no messages remain or maxIterations is achieved var i = 0 diff --git a/docs/hadoop-provided.md b/docs/hadoop-provided.md index bbd26b343e2e6..37cdaa6150d36 100644 --- a/docs/hadoop-provided.md +++ b/docs/hadoop-provided.md @@ -2,6 +2,21 @@ layout: global displayTitle: Using Spark's "Hadoop Free" Build title: Using Spark's "Hadoop Free" Build +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 uses Hadoop client libraries for HDFS and YARN. Starting in version Spark 1.4, the project packages "Hadoop free" builds that lets you more easily connect a single Spark binary to any Hadoop version. To use these builds, you need to modify `SPARK_DIST_CLASSPATH` to include Hadoop's package jars. The most convenient place to do this is by adding an entry in `conf/spark-env.sh`. diff --git a/docs/hardware-provisioning.md b/docs/hardware-provisioning.md index 896f9302ef300..4e5d681962c8d 100644 --- a/docs/hardware-provisioning.md +++ b/docs/hardware-provisioning.md @@ -1,6 +1,21 @@ --- layout: global title: Hardware Provisioning +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. --- A common question received by Spark developers is how to configure hardware for it. While the right @@ -30,14 +45,14 @@ nodes than the storage system to avoid interference. While Spark can perform a lot of its computation in memory, it still uses local disks to store data that doesn't fit in RAM, as well as to preserve intermediate output between stages. We recommend having **4-8 disks** per node, configured _without_ RAID (just as separate mount points). -In Linux, mount the disks with the [`noatime` option](http://www.centos.org/docs/5/html/Global_File_System/s2-manage-mountnoatime.html) +In Linux, mount the disks with the `noatime` option to reduce unnecessary writes. In Spark, [configure](configuration.html) the `spark.local.dir` variable to be a comma-separated list of the local disks. If you are running HDFS, it's fine to use the same disks as HDFS. # Memory -In general, Spark can run well with anywhere from **8 GB to hundreds of gigabytes** of memory per +In general, Spark can run well with anywhere from **8 GiB to hundreds of gigabytes** of memory per machine. In all cases, we recommend allocating only at most 75% of the memory for Spark; leave the rest for the operating system and buffer cache. @@ -47,7 +62,7 @@ Storage tab of Spark's monitoring UI (`http://:4040`) to see its si Note that memory usage is greatly affected by storage level and serialization format -- see the [tuning guide](tuning.html) for tips on how to reduce it. -Finally, note that the Java VM does not always behave well with more than 200 GB of RAM. If you +Finally, note that the Java VM does not always behave well with more than 200 GiB of RAM. If you purchase machines with more RAM than this, you can run _multiple worker JVMs per node_. In Spark's [standalone mode](spark-standalone.html), you can set the number of workers per node with the `SPARK_WORKER_INSTANCES` variable in `conf/spark-env.sh`, and the number of cores diff --git a/docs/index.md b/docs/index.md index d269f54c73439..ddc792521e7e0 100644 --- a/docs/index.md +++ b/docs/index.md @@ -3,19 +3,39 @@ layout: global displayTitle: Spark Overview title: Overview description: Apache Spark SPARK_VERSION_SHORT documentation homepage +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. --- -Apache Spark is a fast and general-purpose cluster computing system. +Apache Spark is a unified analytics engine for large-scale data processing. It provides high-level APIs in Java, Scala, Python and R, and an optimized engine that supports general execution graphs. -It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Spark Streaming](streaming-programming-guide.html). +It also supports a rich set of higher-level tools including [Spark SQL](sql-programming-guide.html) for SQL and structured data processing, [MLlib](ml-guide.html) for machine learning, [GraphX](graphx-programming-guide.html) for graph processing, and [Structured Streaming](structured-streaming-programming-guide.html) for incremental computation and stream processing. + +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) before downloading and running Spark. # Downloading Get Spark from the [downloads page](https://spark.apache.org/downloads.html) of the project website. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark uses Hadoop's client libraries for HDFS and YARN. Downloads are pre-packaged for a handful of popular Hadoop versions. Users can also download a "Hadoop free" binary and run Spark with any Hadoop version [by augmenting Spark's classpath](hadoop-provided.html). -Scala and Java users can include Spark in their projects using its Maven coordinates and in the future Python users can also install Spark from PyPI. +Scala and Java users can include Spark in their projects using its Maven coordinates and Python users can install Spark from PyPI. If you'd like to build Spark from @@ -26,7 +46,10 @@ Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). It's easy locally on one machine --- all you need is to have `java` installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. -Spark runs on Java 8+, Python 2.7+/3.4+ and R 3.1+. For the Scala API, Spark {{site.SPARK_VERSION}} +Spark runs on Java 8+, Scala 2.12, Python 2.7+/3.4+ and R 3.1+. +Python 2 support is deprecated as of Spark 3.0.0. +R prior to version 3.4 support is deprecated as of Spark 3.0.0. +For the Scala API, Spark {{site.SPARK_VERSION}} uses Scala {{site.SCALA_BINARY_VERSION}}. You will need to use a compatible Scala version ({{site.SCALA_BINARY_VERSION}}.x). @@ -60,8 +83,8 @@ Example applications are also provided in Python. For example, ./bin/spark-submit examples/src/main/python/pi.py 10 -Spark also provides an experimental [R API](sparkr.html) since 1.4 (only DataFrames APIs included). -To run Spark interactively in a R interpreter, use `bin/sparkR`: +Spark also provides an [R API](sparkr.html) since 1.4 (only DataFrames APIs included). +To run Spark interactively in an R interpreter, use `bin/sparkR`: ./bin/sparkR --master local[2] diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index 2316f175676ee..3e70c59d89a37 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -1,6 +1,21 @@ --- layout: global title: Job Scheduling +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 will become a table of contents (this text will be scraped). diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js index 13514e11b9332..287fce61540e9 100644 --- a/docs/js/api-docs.js +++ b/docs/js/api-docs.js @@ -58,6 +58,7 @@ $(document).ready(function() { }); }; script.src = ('https:' == document.location.protocol ? 'https://' : 'http://') + - 'cdn.mathjax.org/mathjax/latest/MathJax.js?config=TeX-AMS-MML_HTMLorMML'; + 'cdnjs.cloudflare.com/ajax/libs/mathjax/2.7.1/MathJax.js' + + '?config=TeX-AMS-MML_HTMLorMML'; document.getElementsByTagName('head')[0].appendChild(script); }); diff --git a/docs/js/main.js b/docs/js/main.js index 2329eb8327dd5..3ee3dbe8eaed2 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -93,5 +93,5 @@ $(function() { // Scroll now too in case we had opened the page on a hash, but wait a bit because some browsers // will try to do *their* initial scroll after running the onReady handler. - $(window).load(function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); + $(window).on('load', function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); }); diff --git a/docs/js/vendor/jquery-1.8.0.min.js b/docs/js/vendor/jquery-1.8.0.min.js deleted file mode 100755 index 066d72c7e3a2e..0000000000000 --- a/docs/js/vendor/jquery-1.8.0.min.js +++ /dev/null @@ -1,2 +0,0 @@ -/*! jQuery v@1.8.0 jquery.com | jquery.org/license */ -(function(a,b){function G(a){var b=F[a]={};return p.each(a.split(s),function(a,c){b[c]=!0}),b}function J(a,c,d){if(d===b&&a.nodeType===1){var e="data-"+c.replace(I,"-$1").toLowerCase();d=a.getAttribute(e);if(typeof d=="string"){try{d=d==="true"?!0:d==="false"?!1:d==="null"?null:+d+""===d?+d:H.test(d)?p.parseJSON(d):d}catch(f){}p.data(a,c,d)}else d=b}return d}function K(a){var b;for(b in a){if(b==="data"&&p.isEmptyObject(a[b]))continue;if(b!=="toJSON")return!1}return!0}function ba(){return!1}function bb(){return!0}function bh(a){return!a||!a.parentNode||a.parentNode.nodeType===11}function bi(a,b){do a=a[b];while(a&&a.nodeType!==1);return a}function bj(a,b,c){b=b||0;if(p.isFunction(b))return p.grep(a,function(a,d){var e=!!b.call(a,d,a);return e===c});if(b.nodeType)return p.grep(a,function(a,d){return a===b===c});if(typeof b=="string"){var d=p.grep(a,function(a){return a.nodeType===1});if(be.test(b))return p.filter(b,d,!c);b=p.filter(b,d)}return p.grep(a,function(a,d){return p.inArray(a,b)>=0===c})}function bk(a){var b=bl.split("|"),c=a.createDocumentFragment();if(c.createElement)while(b.length)c.createElement(b.pop());return c}function bC(a,b){return a.getElementsByTagName(b)[0]||a.appendChild(a.ownerDocument.createElement(b))}function bD(a,b){if(b.nodeType!==1||!p.hasData(a))return;var c,d,e,f=p._data(a),g=p._data(b,f),h=f.events;if(h){delete g.handle,g.events={};for(c in h)for(d=0,e=h[c].length;d").appendTo(e.body),c=b.css("display");b.remove();if(c==="none"||c===""){bI=e.body.appendChild(bI||p.extend(e.createElement("iframe"),{frameBorder:0,width:0,height:0}));if(!bJ||!bI.createElement)bJ=(bI.contentWindow||bI.contentDocument).document,bJ.write(""),bJ.close();b=bJ.body.appendChild(bJ.createElement(a)),c=bH(b,"display"),e.body.removeChild(bI)}return bR[a]=c,c}function ch(a,b,c,d){var e;if(p.isArray(b))p.each(b,function(b,e){c||cd.test(a)?d(a,e):ch(a+"["+(typeof e=="object"?b:"")+"]",e,c,d)});else if(!c&&p.type(b)==="object")for(e in b)ch(a+"["+e+"]",b[e],c,d);else d(a,b)}function cy(a){return function(b,c){typeof b!="string"&&(c=b,b="*");var d,e,f,g=b.toLowerCase().split(s),h=0,i=g.length;if(p.isFunction(c))for(;h)[^>]*$|#([\w\-]*)$)/,v=/^<(\w+)\s*\/?>(?:<\/\1>|)$/,w=/^[\],:{}\s]*$/,x=/(?:^|:|,)(?:\s*\[)+/g,y=/\\(?:["\\\/bfnrt]|u[\da-fA-F]{4})/g,z=/"[^"\\\r\n]*"|true|false|null|-?(?:\d\d*\.|)\d+(?:[eE][\-+]?\d+|)/g,A=/^-ms-/,B=/-([\da-z])/gi,C=function(a,b){return(b+"").toUpperCase()},D=function(){e.addEventListener?(e.removeEventListener("DOMContentLoaded",D,!1),p.ready()):e.readyState==="complete"&&(e.detachEvent("onreadystatechange",D),p.ready())},E={};p.fn=p.prototype={constructor:p,init:function(a,c,d){var f,g,h,i;if(!a)return this;if(a.nodeType)return this.context=this[0]=a,this.length=1,this;if(typeof a=="string"){a.charAt(0)==="<"&&a.charAt(a.length-1)===">"&&a.length>=3?f=[null,a,null]:f=u.exec(a);if(f&&(f[1]||!c)){if(f[1])return c=c instanceof p?c[0]:c,i=c&&c.nodeType?c.ownerDocument||c:e,a=p.parseHTML(f[1],i,!0),v.test(f[1])&&p.isPlainObject(c)&&this.attr.call(a,c,!0),p.merge(this,a);g=e.getElementById(f[2]);if(g&&g.parentNode){if(g.id!==f[2])return d.find(a);this.length=1,this[0]=g}return this.context=e,this.selector=a,this}return!c||c.jquery?(c||d).find(a):this.constructor(c).find(a)}return p.isFunction(a)?d.ready(a):(a.selector!==b&&(this.selector=a.selector,this.context=a.context),p.makeArray(a,this))},selector:"",jquery:"1.8.0",length:0,size:function(){return this.length},toArray:function(){return k.call(this)},get:function(a){return a==null?this.toArray():a<0?this[this.length+a]:this[a]},pushStack:function(a,b,c){var d=p.merge(this.constructor(),a);return d.prevObject=this,d.context=this.context,b==="find"?d.selector=this.selector+(this.selector?" ":"")+c:b&&(d.selector=this.selector+"."+b+"("+c+")"),d},each:function(a,b){return p.each(this,a,b)},ready:function(a){return p.ready.promise().done(a),this},eq:function(a){return a=+a,a===-1?this.slice(a):this.slice(a,a+1)},first:function(){return this.eq(0)},last:function(){return this.eq(-1)},slice:function(){return this.pushStack(k.apply(this,arguments),"slice",k.call(arguments).join(","))},map:function(a){return this.pushStack(p.map(this,function(b,c){return a.call(b,c,b)}))},end:function(){return this.prevObject||this.constructor(null)},push:j,sort:[].sort,splice:[].splice},p.fn.init.prototype=p.fn,p.extend=p.fn.extend=function(){var a,c,d,e,f,g,h=arguments[0]||{},i=1,j=arguments.length,k=!1;typeof h=="boolean"&&(k=h,h=arguments[1]||{},i=2),typeof h!="object"&&!p.isFunction(h)&&(h={}),j===i&&(h=this,--i);for(;i0)return;d.resolveWith(e,[p]),p.fn.trigger&&p(e).trigger("ready").off("ready")},isFunction:function(a){return p.type(a)==="function"},isArray:Array.isArray||function(a){return p.type(a)==="array"},isWindow:function(a){return a!=null&&a==a.window},isNumeric:function(a){return!isNaN(parseFloat(a))&&isFinite(a)},type:function(a){return a==null?String(a):E[m.call(a)]||"object"},isPlainObject:function(a){if(!a||p.type(a)!=="object"||a.nodeType||p.isWindow(a))return!1;try{if(a.constructor&&!n.call(a,"constructor")&&!n.call(a.constructor.prototype,"isPrototypeOf"))return!1}catch(c){return!1}var d;for(d in a);return d===b||n.call(a,d)},isEmptyObject:function(a){var b;for(b in a)return!1;return!0},error:function(a){throw new Error(a)},parseHTML:function(a,b,c){var d;return!a||typeof a!="string"?null:(typeof b=="boolean"&&(c=b,b=0),b=b||e,(d=v.exec(a))?[b.createElement(d[1])]:(d=p.buildFragment([a],b,c?null:[]),p.merge([],(d.cacheable?p.clone(d.fragment):d.fragment).childNodes)))},parseJSON:function(b){if(!b||typeof b!="string")return null;b=p.trim(b);if(a.JSON&&a.JSON.parse)return a.JSON.parse(b);if(w.test(b.replace(y,"@").replace(z,"]").replace(x,"")))return(new Function("return "+b))();p.error("Invalid JSON: "+b)},parseXML:function(c){var d,e;if(!c||typeof c!="string")return null;try{a.DOMParser?(e=new DOMParser,d=e.parseFromString(c,"text/xml")):(d=new ActiveXObject("Microsoft.XMLDOM"),d.async="false",d.loadXML(c))}catch(f){d=b}return(!d||!d.documentElement||d.getElementsByTagName("parsererror").length)&&p.error("Invalid XML: "+c),d},noop:function(){},globalEval:function(b){b&&r.test(b)&&(a.execScript||function(b){a.eval.call(a,b)})(b)},camelCase:function(a){return a.replace(A,"ms-").replace(B,C)},nodeName:function(a,b){return a.nodeName&&a.nodeName.toUpperCase()===b.toUpperCase()},each:function(a,c,d){var e,f=0,g=a.length,h=g===b||p.isFunction(a);if(d){if(h){for(e in a)if(c.apply(a[e],d)===!1)break}else for(;f0&&a[0]&&a[i-1]||i===0||p.isArray(a));if(j)for(;h-1)i.splice(c,1),e&&(c<=g&&g--,c<=h&&h--)}),this},has:function(a){return p.inArray(a,i)>-1},empty:function(){return i=[],this},disable:function(){return i=j=c=b,this},disabled:function(){return!i},lock:function(){return j=b,c||l.disable(),this},locked:function(){return!j},fireWith:function(a,b){return b=b||[],b=[a,b.slice?b.slice():b],i&&(!d||j)&&(e?j.push(b):k(b)),this},fire:function(){return l.fireWith(this,arguments),this},fired:function(){return!!d}};return l},p.extend({Deferred:function(a){var b=[["resolve","done",p.Callbacks("once memory"),"resolved"],["reject","fail",p.Callbacks("once memory"),"rejected"],["notify","progress",p.Callbacks("memory")]],c="pending",d={state:function(){return c},always:function(){return e.done(arguments).fail(arguments),this},then:function(){var a=arguments;return p.Deferred(function(c){p.each(b,function(b,d){var f=d[0],g=a[b];e[d[1]](p.isFunction(g)?function(){var a=g.apply(this,arguments);a&&p.isFunction(a.promise)?a.promise().done(c.resolve).fail(c.reject).progress(c.notify):c[f+"With"](this===e?c:this,[a])}:c[f])}),a=null}).promise()},promise:function(a){return typeof a=="object"?p.extend(a,d):d}},e={};return d.pipe=d.then,p.each(b,function(a,f){var g=f[2],h=f[3];d[f[1]]=g.add,h&&g.add(function(){c=h},b[a^1][2].disable,b[2][2].lock),e[f[0]]=g.fire,e[f[0]+"With"]=g.fireWith}),d.promise(e),a&&a.call(e,e),e},when:function(a){var b=0,c=k.call(arguments),d=c.length,e=d!==1||a&&p.isFunction(a.promise)?d:0,f=e===1?a:p.Deferred(),g=function(a,b,c){return function(d){b[a]=this,c[a]=arguments.length>1?k.call(arguments):d,c===h?f.notifyWith(b,c):--e||f.resolveWith(b,c)}},h,i,j;if(d>1){h=new Array(d),i=new Array(d),j=new Array(d);for(;b
    a",c=n.getElementsByTagName("*"),d=n.getElementsByTagName("a")[0],d.style.cssText="top:1px;float:left;opacity:.5";if(!c||!c.length||!d)return{};f=e.createElement("select"),g=f.appendChild(e.createElement("option")),h=n.getElementsByTagName("input")[0],b={leadingWhitespace:n.firstChild.nodeType===3,tbody:!n.getElementsByTagName("tbody").length,htmlSerialize:!!n.getElementsByTagName("link").length,style:/top/.test(d.getAttribute("style")),hrefNormalized:d.getAttribute("href")==="/a",opacity:/^0.5/.test(d.style.opacity),cssFloat:!!d.style.cssFloat,checkOn:h.value==="on",optSelected:g.selected,getSetAttribute:n.className!=="t",enctype:!!e.createElement("form").enctype,html5Clone:e.createElement("nav").cloneNode(!0).outerHTML!=="<:nav>",boxModel:e.compatMode==="CSS1Compat",submitBubbles:!0,changeBubbles:!0,focusinBubbles:!1,deleteExpando:!0,noCloneEvent:!0,inlineBlockNeedsLayout:!1,shrinkWrapBlocks:!1,reliableMarginRight:!0,boxSizingReliable:!0,pixelPosition:!1},h.checked=!0,b.noCloneChecked=h.cloneNode(!0).checked,f.disabled=!0,b.optDisabled=!g.disabled;try{delete n.test}catch(o){b.deleteExpando=!1}!n.addEventListener&&n.attachEvent&&n.fireEvent&&(n.attachEvent("onclick",m=function(){b.noCloneEvent=!1}),n.cloneNode(!0).fireEvent("onclick"),n.detachEvent("onclick",m)),h=e.createElement("input"),h.value="t",h.setAttribute("type","radio"),b.radioValue=h.value==="t",h.setAttribute("checked","checked"),h.setAttribute("name","t"),n.appendChild(h),i=e.createDocumentFragment(),i.appendChild(n.lastChild),b.checkClone=i.cloneNode(!0).cloneNode(!0).lastChild.checked,b.appendChecked=h.checked,i.removeChild(h),i.appendChild(n);if(n.attachEvent)for(k in{submit:!0,change:!0,focusin:!0})j="on"+k,l=j in n,l||(n.setAttribute(j,"return;"),l=typeof n[j]=="function"),b[k+"Bubbles"]=l;return p(function(){var c,d,f,g,h="padding:0;margin:0;border:0;display:block;overflow:hidden;",i=e.getElementsByTagName("body")[0];if(!i)return;c=e.createElement("div"),c.style.cssText="visibility:hidden;border:0;width:0;height:0;position:static;top:0;margin-top:1px",i.insertBefore(c,i.firstChild),d=e.createElement("div"),c.appendChild(d),d.innerHTML="
    t
    ",f=d.getElementsByTagName("td"),f[0].style.cssText="padding:0;margin:0;border:0;display:none",l=f[0].offsetHeight===0,f[0].style.display="",f[1].style.display="none",b.reliableHiddenOffsets=l&&f[0].offsetHeight===0,d.innerHTML="",d.style.cssText="box-sizing:border-box;-moz-box-sizing:border-box;-webkit-box-sizing:border-box;padding:1px;border:1px;display:block;width:4px;margin-top:1%;position:absolute;top:1%;",b.boxSizing=d.offsetWidth===4,b.doesNotIncludeMarginInBodyOffset=i.offsetTop!==1,a.getComputedStyle&&(b.pixelPosition=(a.getComputedStyle(d,null)||{}).top!=="1%",b.boxSizingReliable=(a.getComputedStyle(d,null)||{width:"4px"}).width==="4px",g=e.createElement("div"),g.style.cssText=d.style.cssText=h,g.style.marginRight=g.style.width="0",d.style.width="1px",d.appendChild(g),b.reliableMarginRight=!parseFloat((a.getComputedStyle(g,null)||{}).marginRight)),typeof d.style.zoom!="undefined"&&(d.innerHTML="",d.style.cssText=h+"width:1px;padding:1px;display:inline;zoom:1",b.inlineBlockNeedsLayout=d.offsetWidth===3,d.style.display="block",d.style.overflow="visible",d.innerHTML="

    ",d.firstChild.style.width="5px",b.shrinkWrapBlocks=d.offsetWidth!==3,c.style.zoom=1),i.removeChild(c),c=d=f=g=null}),i.removeChild(n),c=d=f=g=h=i=n=null,b}();var H=/^(?:\{.*\}|\[.*\])$/,I=/([A-Z])/g;p.extend({cache:{},deletedIds:[],uuid:0,expando:"jQuery"+(p.fn.jquery+Math.random()).replace(/\D/g,""),noData:{embed:!0,object:"clsid:D27CDB6E-AE6D-11cf-96B8-444553540000",applet:!0},hasData:function(a){return a=a.nodeType?p.cache[a[p.expando]]:a[p.expando],!!a&&!K(a)},data:function(a,c,d,e){if(!p.acceptData(a))return;var f,g,h=p.expando,i=typeof c=="string",j=a.nodeType,k=j?p.cache:a,l=j?a[h]:a[h]&&h;if((!l||!k[l]||!e&&!k[l].data)&&i&&d===b)return;l||(j?a[h]=l=p.deletedIds.pop()||++p.uuid:l=h),k[l]||(k[l]={},j||(k[l].toJSON=p.noop));if(typeof c=="object"||typeof c=="function")e?k[l]=p.extend(k[l],c):k[l].data=p.extend(k[l].data,c);return f=k[l],e||(f.data||(f.data={}),f=f.data),d!==b&&(f[p.camelCase(c)]=d),i?(g=f[c],g==null&&(g=f[p.camelCase(c)])):g=f,g},removeData:function(a,b,c){if(!p.acceptData(a))return;var d,e,f,g=a.nodeType,h=g?p.cache:a,i=g?a[p.expando]:p.expando;if(!h[i])return;if(b){d=c?h[i]:h[i].data;if(d){p.isArray(b)||(b in d?b=[b]:(b=p.camelCase(b),b in d?b=[b]:b=b.split(" ")));for(e=0,f=b.length;e1,null,!1))},removeData:function(a){return this.each(function(){p.removeData(this,a)})}}),p.extend({queue:function(a,b,c){var d;if(a)return b=(b||"fx")+"queue",d=p._data(a,b),c&&(!d||p.isArray(c)?d=p._data(a,b,p.makeArray(c)):d.push(c)),d||[]},dequeue:function(a,b){b=b||"fx";var c=p.queue(a,b),d=c.shift(),e=p._queueHooks(a,b),f=function(){p.dequeue(a,b)};d==="inprogress"&&(d=c.shift()),d&&(b==="fx"&&c.unshift("inprogress"),delete e.stop,d.call(a,f,e)),!c.length&&e&&e.empty.fire()},_queueHooks:function(a,b){var c=b+"queueHooks";return p._data(a,c)||p._data(a,c,{empty:p.Callbacks("once memory").add(function(){p.removeData(a,b+"queue",!0),p.removeData(a,c,!0)})})}}),p.fn.extend({queue:function(a,c){var d=2;return typeof a!="string"&&(c=a,a="fx",d--),arguments.length1)},removeAttr:function(a){return this.each(function(){p.removeAttr(this,a)})},prop:function(a,b){return p.access(this,p.prop,a,b,arguments.length>1)},removeProp:function(a){return a=p.propFix[a]||a,this.each(function(){try{this[a]=b,delete this[a]}catch(c){}})},addClass:function(a){var b,c,d,e,f,g,h;if(p.isFunction(a))return this.each(function(b){p(this).addClass(a.call(this,b,this.className))});if(a&&typeof a=="string"){b=a.split(s);for(c=0,d=this.length;c-1)d=d.replace(" "+c[f]+" "," ");e.className=a?p.trim(d):""}}}return this},toggleClass:function(a,b){var c=typeof a,d=typeof b=="boolean";return p.isFunction(a)?this.each(function(c){p(this).toggleClass(a.call(this,c,this.className,b),b)}):this.each(function(){if(c==="string"){var e,f=0,g=p(this),h=b,i=a.split(s);while(e=i[f++])h=d?h:!g.hasClass(e),g[h?"addClass":"removeClass"](e)}else if(c==="undefined"||c==="boolean")this.className&&p._data(this,"__className__",this.className),this.className=this.className||a===!1?"":p._data(this,"__className__")||""})},hasClass:function(a){var b=" "+a+" ",c=0,d=this.length;for(;c-1)return!0;return!1},val:function(a){var c,d,e,f=this[0];if(!arguments.length){if(f)return c=p.valHooks[f.type]||p.valHooks[f.nodeName.toLowerCase()],c&&"get"in c&&(d=c.get(f,"value"))!==b?d:(d=f.value,typeof d=="string"?d.replace(P,""):d==null?"":d);return}return e=p.isFunction(a),this.each(function(d){var f,g=p(this);if(this.nodeType!==1)return;e?f=a.call(this,d,g.val()):f=a,f==null?f="":typeof f=="number"?f+="":p.isArray(f)&&(f=p.map(f,function(a){return a==null?"":a+""})),c=p.valHooks[this.type]||p.valHooks[this.nodeName.toLowerCase()];if(!c||!("set"in c)||c.set(this,f,"value")===b)this.value=f})}}),p.extend({valHooks:{option:{get:function(a){var b=a.attributes.value;return!b||b.specified?a.value:a.text}},select:{get:function(a){var b,c,d,e,f=a.selectedIndex,g=[],h=a.options,i=a.type==="select-one";if(f<0)return null;c=i?f:0,d=i?f+1:h.length;for(;c=0}),c.length||(a.selectedIndex=-1),c}}},attrFn:{},attr:function(a,c,d,e){var f,g,h,i=a.nodeType;if(!a||i===3||i===8||i===2)return;if(e&&p.isFunction(p.fn[c]))return p(a)[c](d);if(typeof a.getAttribute=="undefined")return p.prop(a,c,d);h=i!==1||!p.isXMLDoc(a),h&&(c=c.toLowerCase(),g=p.attrHooks[c]||(T.test(c)?M:L));if(d!==b){if(d===null){p.removeAttr(a,c);return}return g&&"set"in g&&h&&(f=g.set(a,d,c))!==b?f:(a.setAttribute(c,""+d),d)}return g&&"get"in g&&h&&(f=g.get(a,c))!==null?f:(f=a.getAttribute(c),f===null?b:f)},removeAttr:function(a,b){var c,d,e,f,g=0;if(b&&a.nodeType===1){d=b.split(s);for(;g=0}})});var V=/^(?:textarea|input|select)$/i,W=/^([^\.]*|)(?:\.(.+)|)$/,X=/(?:^|\s)hover(\.\S+|)\b/,Y=/^key/,Z=/^(?:mouse|contextmenu)|click/,$=/^(?:focusinfocus|focusoutblur)$/,_=function(a){return p.event.special.hover?a:a.replace(X,"mouseenter$1 mouseleave$1")};p.event={add:function(a,c,d,e,f){var g,h,i,j,k,l,m,n,o,q,r;if(a.nodeType===3||a.nodeType===8||!c||!d||!(g=p._data(a)))return;d.handler&&(o=d,d=o.handler,f=o.selector),d.guid||(d.guid=p.guid++),i=g.events,i||(g.events=i={}),h=g.handle,h||(g.handle=h=function(a){return typeof p!="undefined"&&(!a||p.event.triggered!==a.type)?p.event.dispatch.apply(h.elem,arguments):b},h.elem=a),c=p.trim(_(c)).split(" ");for(j=0;j=0&&(s=s.slice(0,-1),i=!0),s.indexOf(".")>=0&&(t=s.split("."),s=t.shift(),t.sort());if((!f||p.event.customEvent[s])&&!p.event.global[s])return;c=typeof c=="object"?c[p.expando]?c:new p.Event(s,c):new p.Event(s),c.type=s,c.isTrigger=!0,c.exclusive=i,c.namespace=t.join("."),c.namespace_re=c.namespace?new RegExp("(^|\\.)"+t.join("\\.(?:.*\\.|)")+"(\\.|$)"):null,m=s.indexOf(":")<0?"on"+s:"";if(!f){h=p.cache;for(j in h)h[j].events&&h[j].events[s]&&p.event.trigger(c,d,h[j].handle.elem,!0);return}c.result=b,c.target||(c.target=f),d=d!=null?p.makeArray(d):[],d.unshift(c),n=p.event.special[s]||{};if(n.trigger&&n.trigger.apply(f,d)===!1)return;q=[[f,n.bindType||s]];if(!g&&!n.noBubble&&!p.isWindow(f)){r=n.delegateType||s,k=$.test(r+s)?f:f.parentNode;for(l=f;k;k=k.parentNode)q.push([k,r]),l=k;l===(f.ownerDocument||e)&&q.push([l.defaultView||l.parentWindow||a,r])}for(j=0;jq&&u.push({elem:this,matches:o.slice(q)});for(d=0;d0?this.on(b,null,a,c):this.trigger(b)},Y.test(b)&&(p.event.fixHooks[b]=p.event.keyHooks),Z.test(b)&&(p.event.fixHooks[b]=p.event.mouseHooks)}),function(a,b){function bd(a,b,c,d){var e=0,f=b.length;for(;e0?h(g,c,f):[]}function bf(a,c,d,e,f){var g,h,i,j,k,l,m,n,p=0,q=f.length,s=L.POS,t=new RegExp("^"+s.source+"(?!"+r+")","i"),u=function(){var a=1,c=arguments.length-2;for(;ai){m=a.slice(i,g.index),i=n,l=[c],B.test(m)&&(k&&(l=k),k=e);if(h=H.test(m))m=m.slice(0,-5).replace(B,"$&*");g.length>1&&g[0].replace(t,u),k=be(m,g[1],g[2],l,k,h)}}k?(j=j.concat(k),(m=a.slice(i))&&m!==")"?B.test(m)?bd(m,j,d,e):Z(m,c,d,e?e.concat(k):k):o.apply(d,j)):Z(a,c,d,e)}return q===1?d:Z.uniqueSort(d)}function bg(a,b,c){var d,e,f,g=[],i=0,j=D.exec(a),k=!j.pop()&&!j.pop(),l=k&&a.match(C)||[""],m=$.preFilter,n=$.filter,o=!c&&b!==h;for(;(e=l[i])!=null&&k;i++){g.push(d=[]),o&&(e=" "+e);while(e){k=!1;if(j=B.exec(e))e=e.slice(j[0].length),k=d.push({part:j.pop().replace(A," "),captures:j});for(f in n)(j=L[f].exec(e))&&(!m[f]||(j=m[f](j,b,c)))&&(e=e.slice(j.shift().length),k=d.push({part:f,captures:j}));if(!k)break}}return k||Z.error(a),g}function bh(a,b,e){var f=b.dir,g=m++;return a||(a=function(a){return a===e}),b.first?function(b,c){while(b=b[f])if(b.nodeType===1)return a(b,c)&&b}:function(b,e){var h,i=g+"."+d,j=i+"."+c;while(b=b[f])if(b.nodeType===1){if((h=b[q])===j)return b.sizset;if(typeof h=="string"&&h.indexOf(i)===0){if(b.sizset)return b}else{b[q]=j;if(a(b,e))return b.sizset=!0,b;b.sizset=!1}}}}function bi(a,b){return a?function(c,d){var e=b(c,d);return e&&a(e===!0?c:e,d)}:b}function bj(a,b,c){var d,e,f=0;for(;d=a[f];f++)$.relative[d.part]?e=bh(e,$.relative[d.part],b):(d.captures.push(b,c),e=bi(e,$.filter[d.part].apply(null,d.captures)));return e}function bk(a){return function(b,c){var d,e=0;for(;d=a[e];e++)if(d(b,c))return!0;return!1}}var c,d,e,f,g,h=a.document,i=h.documentElement,j="undefined",k=!1,l=!0,m=0,n=[].slice,o=[].push,q=("sizcache"+Math.random()).replace(".",""),r="[\\x20\\t\\r\\n\\f]",s="(?:\\\\.|[-\\w]|[^\\x00-\\xa0])+",t=s.replace("w","w#"),u="([*^$|!~]?=)",v="\\["+r+"*("+s+")"+r+"*(?:"+u+r+"*(?:(['\"])((?:\\\\.|[^\\\\])*?)\\3|("+t+")|)|)"+r+"*\\]",w=":("+s+")(?:\\((?:(['\"])((?:\\\\.|[^\\\\])*?)\\2|((?:[^,]|\\\\,|(?:,(?=[^\\[]*\\]))|(?:,(?=[^\\(]*\\))))*))\\)|)",x=":(nth|eq|gt|lt|first|last|even|odd)(?:\\((\\d*)\\)|)(?=[^-]|$)",y=r+"*([\\x20\\t\\r\\n\\f>+~])"+r+"*",z="(?=[^\\x20\\t\\r\\n\\f])(?:\\\\.|"+v+"|"+w.replace(2,7)+"|[^\\\\(),])+",A=new RegExp("^"+r+"+|((?:^|[^\\\\])(?:\\\\.)*)"+r+"+$","g"),B=new RegExp("^"+y),C=new RegExp(z+"?(?="+r+"*,|$)","g"),D=new RegExp("^(?:(?!,)(?:(?:^|,)"+r+"*"+z+")*?|"+r+"*(.*?))(\\)|$)"),E=new RegExp(z.slice(19,-6)+"\\x20\\t\\r\\n\\f>+~])+|"+y,"g"),F=/^(?:#([\w\-]+)|(\w+)|\.([\w\-]+))$/,G=/[\x20\t\r\n\f]*[+~]/,H=/:not\($/,I=/h\d/i,J=/input|select|textarea|button/i,K=/\\(?!\\)/g,L={ID:new RegExp("^#("+s+")"),CLASS:new RegExp("^\\.("+s+")"),NAME:new RegExp("^\\[name=['\"]?("+s+")['\"]?\\]"),TAG:new RegExp("^("+s.replace("[-","[-\\*")+")"),ATTR:new RegExp("^"+v),PSEUDO:new RegExp("^"+w),CHILD:new RegExp("^:(only|nth|last|first)-child(?:\\("+r+"*(even|odd|(([+-]|)(\\d*)n|)"+r+"*(?:([+-]|)"+r+"*(\\d+)|))"+r+"*\\)|)","i"),POS:new RegExp(x,"ig"),needsContext:new RegExp("^"+r+"*[>+~]|"+x,"i")},M={},N=[],O={},P=[],Q=function(a){return a.sizzleFilter=!0,a},R=function(a){return function(b){return b.nodeName.toLowerCase()==="input"&&b.type===a}},S=function(a){return function(b){var c=b.nodeName.toLowerCase();return(c==="input"||c==="button")&&b.type===a}},T=function(a){var b=!1,c=h.createElement("div");try{b=a(c)}catch(d){}return c=null,b},U=T(function(a){a.innerHTML="";var b=typeof a.lastChild.getAttribute("multiple");return b!=="boolean"&&b!=="string"}),V=T(function(a){a.id=q+0,a.innerHTML="
    ",i.insertBefore(a,i.firstChild);var b=h.getElementsByName&&h.getElementsByName(q).length===2+h.getElementsByName(q+0).length;return g=!h.getElementById(q),i.removeChild(a),b}),W=T(function(a){return a.appendChild(h.createComment("")),a.getElementsByTagName("*").length===0}),X=T(function(a){return a.innerHTML="",a.firstChild&&typeof a.firstChild.getAttribute!==j&&a.firstChild.getAttribute("href")==="#"}),Y=T(function(a){return a.innerHTML="",!a.getElementsByClassName||a.getElementsByClassName("e").length===0?!1:(a.lastChild.className="e",a.getElementsByClassName("e").length!==1)}),Z=function(a,b,c,d){c=c||[],b=b||h;var e,f,g,i,j=b.nodeType;if(j!==1&&j!==9)return[];if(!a||typeof a!="string")return c;g=ba(b);if(!g&&!d)if(e=F.exec(a))if(i=e[1]){if(j===9){f=b.getElementById(i);if(!f||!f.parentNode)return c;if(f.id===i)return c.push(f),c}else if(b.ownerDocument&&(f=b.ownerDocument.getElementById(i))&&bb(b,f)&&f.id===i)return c.push(f),c}else{if(e[2])return o.apply(c,n.call(b.getElementsByTagName(a),0)),c;if((i=e[3])&&Y&&b.getElementsByClassName)return o.apply(c,n.call(b.getElementsByClassName(i),0)),c}return bm(a,b,c,d,g)},$=Z.selectors={cacheLength:50,match:L,order:["ID","TAG"],attrHandle:{},createPseudo:Q,find:{ID:g?function(a,b,c){if(typeof b.getElementById!==j&&!c){var d=b.getElementById(a);return d&&d.parentNode?[d]:[]}}:function(a,c,d){if(typeof c.getElementById!==j&&!d){var e=c.getElementById(a);return e?e.id===a||typeof e.getAttributeNode!==j&&e.getAttributeNode("id").value===a?[e]:b:[]}},TAG:W?function(a,b){if(typeof b.getElementsByTagName!==j)return b.getElementsByTagName(a)}:function(a,b){var c=b.getElementsByTagName(a);if(a==="*"){var d,e=[],f=0;for(;d=c[f];f++)d.nodeType===1&&e.push(d);return e}return c}},relative:{">":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(a){return a[1]=a[1].replace(K,""),a[3]=(a[4]||a[5]||"").replace(K,""),a[2]==="~="&&(a[3]=" "+a[3]+" "),a.slice(0,4)},CHILD:function(a){return a[1]=a[1].toLowerCase(),a[1]==="nth"?(a[2]||Z.error(a[0]),a[3]=+(a[3]?a[4]+(a[5]||1):2*(a[2]==="even"||a[2]==="odd")),a[4]=+(a[6]+a[7]||a[2]==="odd")):a[2]&&Z.error(a[0]),a},PSEUDO:function(a){var b,c=a[4];return L.CHILD.test(a[0])?null:(c&&(b=D.exec(c))&&b.pop()&&(a[0]=a[0].slice(0,b[0].length-c.length-1),c=b[0].slice(0,-1)),a.splice(2,3,c||a[3]),a)}},filter:{ID:g?function(a){return a=a.replace(K,""),function(b){return b.getAttribute("id")===a}}:function(a){return a=a.replace(K,""),function(b){var c=typeof b.getAttributeNode!==j&&b.getAttributeNode("id");return c&&c.value===a}},TAG:function(a){return a==="*"?function(){return!0}:(a=a.replace(K,"").toLowerCase(),function(b){return b.nodeName&&b.nodeName.toLowerCase()===a})},CLASS:function(a){var b=M[a];return b||(b=M[a]=new RegExp("(^|"+r+")"+a+"("+r+"|$)"),N.push(a),N.length>$.cacheLength&&delete M[N.shift()]),function(a){return b.test(a.className||typeof a.getAttribute!==j&&a.getAttribute("class")||"")}},ATTR:function(a,b,c){return b?function(d){var e=Z.attr(d,a),f=e+"";if(e==null)return b==="!=";switch(b){case"=":return f===c;case"!=":return f!==c;case"^=":return c&&f.indexOf(c)===0;case"*=":return c&&f.indexOf(c)>-1;case"$=":return c&&f.substr(f.length-c.length)===c;case"~=":return(" "+f+" ").indexOf(c)>-1;case"|=":return f===c||f.substr(0,c.length+1)===c+"-"}}:function(b){return Z.attr(b,a)!=null}},CHILD:function(a,b,c,d){if(a==="nth"){var e=m++;return function(a){var b,f,g=0,h=a;if(c===1&&d===0)return!0;b=a.parentNode;if(b&&(b[q]!==e||!a.sizset)){for(h=b.firstChild;h;h=h.nextSibling)if(h.nodeType===1){h.sizset=++g;if(h===a)break}b[q]=e}return f=a.sizset-d,c===0?f===0:f%c===0&&f/c>=0}}return function(b){var c=b;switch(a){case"only":case"first":while(c=c.previousSibling)if(c.nodeType===1)return!1;if(a==="first")return!0;c=b;case"last":while(c=c.nextSibling)if(c.nodeType===1)return!1;return!0}}},PSEUDO:function(a,b,c,d){var e=$.pseudos[a]||$.pseudos[a.toLowerCase()];return e||Z.error("unsupported pseudo: "+a),e.sizzleFilter?e(b,c,d):e}},pseudos:{not:Q(function(a,b,c){var d=bl(a.replace(A,"$1"),b,c);return function(a){return!d(a)}}),enabled:function(a){return a.disabled===!1},disabled:function(a){return a.disabled===!0},checked:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&!!a.checked||b==="option"&&!!a.selected},selected:function(a){return a.parentNode&&a.parentNode.selectedIndex,a.selected===!0},parent:function(a){return!$.pseudos.empty(a)},empty:function(a){var b;a=a.firstChild;while(a){if(a.nodeName>"@"||(b=a.nodeType)===3||b===4)return!1;a=a.nextSibling}return!0},contains:Q(function(a){return function(b){return(b.textContent||b.innerText||bc(b)).indexOf(a)>-1}}),has:Q(function(a){return function(b){return Z(a,b).length>0}}),header:function(a){return I.test(a.nodeName)},text:function(a){var b,c;return a.nodeName.toLowerCase()==="input"&&(b=a.type)==="text"&&((c=a.getAttribute("type"))==null||c.toLowerCase()===b)},radio:R("radio"),checkbox:R("checkbox"),file:R("file"),password:R("password"),image:R("image"),submit:S("submit"),reset:S("reset"),button:function(a){var b=a.nodeName.toLowerCase();return b==="input"&&a.type==="button"||b==="button"},input:function(a){return J.test(a.nodeName)},focus:function(a){var b=a.ownerDocument;return a===b.activeElement&&(!b.hasFocus||b.hasFocus())&&(!!a.type||!!a.href)},active:function(a){return a===a.ownerDocument.activeElement}},setFilters:{first:function(a,b,c){return c?a.slice(1):[a[0]]},last:function(a,b,c){var d=a.pop();return c?a:[d]},even:function(a,b,c){var d=[],e=c?1:0,f=a.length;for(;e$.cacheLength&&delete O[P.shift()],g};Z.matches=function(a,b){return Z(a,null,null,b)},Z.matchesSelector=function(a,b){return Z(b,null,null,[a]).length>0};var bm=function(a,b,e,f,g){a=a.replace(A,"$1");var h,i,j,k,l,m,p,q,r,s=a.match(C),t=a.match(E),u=b.nodeType;if(L.POS.test(a))return bf(a,b,e,f,s);if(f)h=n.call(f,0);else if(s&&s.length===1){if(t.length>1&&u===9&&!g&&(s=L.ID.exec(t[0]))){b=$.find.ID(s[1],b,g)[0];if(!b)return e;a=a.slice(t.shift().length)}q=(s=G.exec(t[0]))&&!s.index&&b.parentNode||b,r=t.pop(),m=r.split(":not")[0];for(j=0,k=$.order.length;j",a.querySelectorAll("[selected]").length||e.push("\\["+r+"*(?:checked|disabled|ismap|multiple|readonly|selected|value)"),a.querySelectorAll(":checked").length||e.push(":checked")}),T(function(a){a.innerHTML="

    ",a.querySelectorAll("[test^='']").length&&e.push("[*^$]="+r+"*(?:\"\"|'')"),a.innerHTML="",a.querySelectorAll(":enabled").length||e.push(":enabled",":disabled")}),e=e.length&&new RegExp(e.join("|")),bm=function(a,d,f,g,h){if(!g&&!h&&(!e||!e.test(a)))if(d.nodeType===9)try{return o.apply(f,n.call(d.querySelectorAll(a),0)),f}catch(i){}else if(d.nodeType===1&&d.nodeName.toLowerCase()!=="object"){var j=d.getAttribute("id"),k=j||q,l=G.test(a)&&d.parentNode||d;j?k=k.replace(c,"\\$&"):d.setAttribute("id",k);try{return o.apply(f,n.call(l.querySelectorAll(a.replace(C,"[id='"+k+"'] $&")),0)),f}catch(i){}finally{j||d.removeAttribute("id")}}return b(a,d,f,g,h)},g&&(T(function(b){a=g.call(b,"div");try{g.call(b,"[test!='']:sizzle"),f.push($.match.PSEUDO)}catch(c){}}),f=new RegExp(f.join("|")),Z.matchesSelector=function(b,c){c=c.replace(d,"='$1']");if(!ba(b)&&!f.test(c)&&(!e||!e.test(c)))try{var h=g.call(b,c);if(h||a||b.document&&b.document.nodeType!==11)return h}catch(i){}return Z(c,null,null,[b]).length>0})}(),Z.attr=p.attr,p.find=Z,p.expr=Z.selectors,p.expr[":"]=p.expr.pseudos,p.unique=Z.uniqueSort,p.text=Z.getText,p.isXMLDoc=Z.isXML,p.contains=Z.contains}(a);var bc=/Until$/,bd=/^(?:parents|prev(?:Until|All))/,be=/^.[^:#\[\.,]*$/,bf=p.expr.match.needsContext,bg={children:!0,contents:!0,next:!0,prev:!0};p.fn.extend({find:function(a){var b,c,d,e,f,g,h=this;if(typeof a!="string")return p(a).filter(function(){for(b=0,c=h.length;b0)for(e=d;e=0:p.filter(a,this).length>0:this.filter(a).length>0)},closest:function(a,b){var c,d=0,e=this.length,f=[],g=bf.test(a)||typeof a!="string"?p(a,b||this.context):0;for(;d-1:p.find.matchesSelector(c,a)){f.push(c);break}c=c.parentNode}}return f=f.length>1?p.unique(f):f,this.pushStack(f,"closest",a)},index:function(a){return a?typeof a=="string"?p.inArray(this[0],p(a)):p.inArray(a.jquery?a[0]:a,this):this[0]&&this[0].parentNode?this.prevAll().length:-1},add:function(a,b){var c=typeof a=="string"?p(a,b):p.makeArray(a&&a.nodeType?[a]:a),d=p.merge(this.get(),c);return this.pushStack(bh(c[0])||bh(d[0])?d:p.unique(d))},addBack:function(a){return this.add(a==null?this.prevObject:this.prevObject.filter(a))}}),p.fn.andSelf=p.fn.addBack,p.each({parent:function(a){var b=a.parentNode;return b&&b.nodeType!==11?b:null},parents:function(a){return p.dir(a,"parentNode")},parentsUntil:function(a,b,c){return p.dir(a,"parentNode",c)},next:function(a){return bi(a,"nextSibling")},prev:function(a){return bi(a,"previousSibling")},nextAll:function(a){return p.dir(a,"nextSibling")},prevAll:function(a){return p.dir(a,"previousSibling")},nextUntil:function(a,b,c){return p.dir(a,"nextSibling",c)},prevUntil:function(a,b,c){return p.dir(a,"previousSibling",c)},siblings:function(a){return p.sibling((a.parentNode||{}).firstChild,a)},children:function(a){return p.sibling(a.firstChild)},contents:function(a){return p.nodeName(a,"iframe")?a.contentDocument||a.contentWindow.document:p.merge([],a.childNodes)}},function(a,b){p.fn[a]=function(c,d){var e=p.map(this,b,c);return bc.test(a)||(d=c),d&&typeof d=="string"&&(e=p.filter(d,e)),e=this.length>1&&!bg[a]?p.unique(e):e,this.length>1&&bd.test(a)&&(e=e.reverse()),this.pushStack(e,a,k.call(arguments).join(","))}}),p.extend({filter:function(a,b,c){return c&&(a=":not("+a+")"),b.length===1?p.find.matchesSelector(b[0],a)?[b[0]]:[]:p.find.matches(a,b)},dir:function(a,c,d){var e=[],f=a[c];while(f&&f.nodeType!==9&&(d===b||f.nodeType!==1||!p(f).is(d)))f.nodeType===1&&e.push(f),f=f[c];return e},sibling:function(a,b){var c=[];for(;a;a=a.nextSibling)a.nodeType===1&&a!==b&&c.push(a);return c}});var bl="abbr|article|aside|audio|bdi|canvas|data|datalist|details|figcaption|figure|footer|header|hgroup|mark|meter|nav|output|progress|section|summary|time|video",bm=/ jQuery\d+="(?:null|\d+)"/g,bn=/^\s+/,bo=/<(?!area|br|col|embed|hr|img|input|link|meta|param)(([\w:]+)[^>]*)\/>/gi,bp=/<([\w:]+)/,bq=/]","i"),bv=/^(?:checkbox|radio)$/,bw=/checked\s*(?:[^=]|=\s*.checked.)/i,bx=/\/(java|ecma)script/i,by=/^\s*\s*$/g,bz={option:[1,""],legend:[1,"
    ","
    "],thead:[1,"","
    "],tr:[2,"","
    "],td:[3,"","
    "],col:[2,"","
    "],area:[1,"",""],_default:[0,"",""]},bA=bk(e),bB=bA.appendChild(e.createElement("div"));bz.optgroup=bz.option,bz.tbody=bz.tfoot=bz.colgroup=bz.caption=bz.thead,bz.th=bz.td,p.support.htmlSerialize||(bz._default=[1,"X
    ","
    "]),p.fn.extend({text:function(a){return p.access(this,function(a){return a===b?p.text(this):this.empty().append((this[0]&&this[0].ownerDocument||e).createTextNode(a))},null,a,arguments.length)},wrapAll:function(a){if(p.isFunction(a))return this.each(function(b){p(this).wrapAll(a.call(this,b))});if(this[0]){var b=p(a,this[0].ownerDocument).eq(0).clone(!0);this[0].parentNode&&b.insertBefore(this[0]),b.map(function(){var a=this;while(a.firstChild&&a.firstChild.nodeType===1)a=a.firstChild;return a}).append(this)}return this},wrapInner:function(a){return p.isFunction(a)?this.each(function(b){p(this).wrapInner(a.call(this,b))}):this.each(function(){var b=p(this),c=b.contents();c.length?c.wrapAll(a):b.append(a)})},wrap:function(a){var b=p.isFunction(a);return this.each(function(c){p(this).wrapAll(b?a.call(this,c):a)})},unwrap:function(){return this.parent().each(function(){p.nodeName(this,"body")||p(this).replaceWith(this.childNodes)}).end()},append:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.appendChild(a)})},prepend:function(){return this.domManip(arguments,!0,function(a){(this.nodeType===1||this.nodeType===11)&&this.insertBefore(a,this.firstChild)})},before:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(a,this),"before",this.selector)}},after:function(){if(!bh(this[0]))return this.domManip(arguments,!1,function(a){this.parentNode.insertBefore(a,this.nextSibling)});if(arguments.length){var a=p.clean(arguments);return this.pushStack(p.merge(this,a),"after",this.selector)}},remove:function(a,b){var c,d=0;for(;(c=this[d])!=null;d++)if(!a||p.filter(a,[c]).length)!b&&c.nodeType===1&&(p.cleanData(c.getElementsByTagName("*")),p.cleanData([c])),c.parentNode&&c.parentNode.removeChild(c);return this},empty:function(){var a,b=0;for(;(a=this[b])!=null;b++){a.nodeType===1&&p.cleanData(a.getElementsByTagName("*"));while(a.firstChild)a.removeChild(a.firstChild)}return this},clone:function(a,b){return a=a==null?!1:a,b=b==null?a:b,this.map(function(){return p.clone(this,a,b)})},html:function(a){return p.access(this,function(a){var c=this[0]||{},d=0,e=this.length;if(a===b)return c.nodeType===1?c.innerHTML.replace(bm,""):b;if(typeof a=="string"&&!bs.test(a)&&(p.support.htmlSerialize||!bu.test(a))&&(p.support.leadingWhitespace||!bn.test(a))&&!bz[(bp.exec(a)||["",""])[1].toLowerCase()]){a=a.replace(bo,"<$1>");try{for(;d1&&typeof j=="string"&&bw.test(j))return this.each(function(){p(this).domManip(a,c,d)});if(p.isFunction(j))return this.each(function(e){var f=p(this);a[0]=j.call(this,e,c?f.html():b),f.domManip(a,c,d)});if(this[0]){e=p.buildFragment(a,this,k),g=e.fragment,f=g.firstChild,g.childNodes.length===1&&(g=f);if(f){c=c&&p.nodeName(f,"tr");for(h=e.cacheable||l-1;i0?this.clone(!0):this).get(),p(g[e])[b](d),f=f.concat(d);return this.pushStack(f,a,g.selector)}}),p.extend({clone:function(a,b,c){var d,e,f,g;p.support.html5Clone||p.isXMLDoc(a)||!bu.test("<"+a.nodeName+">")?g=a.cloneNode(!0):(bB.innerHTML=a.outerHTML,bB.removeChild(g=bB.firstChild));if((!p.support.noCloneEvent||!p.support.noCloneChecked)&&(a.nodeType===1||a.nodeType===11)&&!p.isXMLDoc(a)){bE(a,g),d=bF(a),e=bF(g);for(f=0;d[f];++f)e[f]&&bE(d[f],e[f])}if(b){bD(a,g);if(c){d=bF(a),e=bF(g);for(f=0;d[f];++f)bD(d[f],e[f])}}return d=e=null,g},clean:function(a,b,c,d){var f,g,h,i,j,k,l,m,n,o,q,r,s=0,t=[];if(!b||typeof b.createDocumentFragment=="undefined")b=e;for(g=b===e&&bA;(h=a[s])!=null;s++){typeof h=="number"&&(h+="");if(!h)continue;if(typeof h=="string")if(!br.test(h))h=b.createTextNode(h);else{g=g||bk(b),l=l||g.appendChild(b.createElement("div")),h=h.replace(bo,"<$1>"),i=(bp.exec(h)||["",""])[1].toLowerCase(),j=bz[i]||bz._default,k=j[0],l.innerHTML=j[1]+h+j[2];while(k--)l=l.lastChild;if(!p.support.tbody){m=bq.test(h),n=i==="table"&&!m?l.firstChild&&l.firstChild.childNodes:j[1]===""&&!m?l.childNodes:[];for(f=n.length-1;f>=0;--f)p.nodeName(n[f],"tbody")&&!n[f].childNodes.length&&n[f].parentNode.removeChild(n[f])}!p.support.leadingWhitespace&&bn.test(h)&&l.insertBefore(b.createTextNode(bn.exec(h)[0]),l.firstChild),h=l.childNodes,l=g.lastChild}h.nodeType?t.push(h):t=p.merge(t,h)}l&&(g.removeChild(l),h=l=g=null);if(!p.support.appendChecked)for(s=0;(h=t[s])!=null;s++)p.nodeName(h,"input")?bG(h):typeof h.getElementsByTagName!="undefined"&&p.grep(h.getElementsByTagName("input"),bG);if(c){q=function(a){if(!a.type||bx.test(a.type))return d?d.push(a.parentNode?a.parentNode.removeChild(a):a):c.appendChild(a)};for(s=0;(h=t[s])!=null;s++)if(!p.nodeName(h,"script")||!q(h))c.appendChild(h),typeof h.getElementsByTagName!="undefined"&&(r=p.grep(p.merge([],h.getElementsByTagName("script")),q),t.splice.apply(t,[s+1,0].concat(r)),s+=r.length)}return t},cleanData:function(a,b){var c,d,e,f,g=0,h=p.expando,i=p.cache,j=p.support.deleteExpando,k=p.event.special;for(;(e=a[g])!=null;g++)if(b||p.acceptData(e)){d=e[h],c=d&&i[d];if(c){if(c.events)for(f in c.events)k[f]?p.event.remove(e,f):p.removeEvent(e,f,c.handle);i[d]&&(delete i[d],j?delete e[h]:e.removeAttribute?e.removeAttribute(h):e[h]=null,p.deletedIds.push(d))}}}}),function(){var a,b;p.uaMatch=function(a){a=a.toLowerCase();var b=/(chrome)[ \/]([\w.]+)/.exec(a)||/(webkit)[ \/]([\w.]+)/.exec(a)||/(opera)(?:.*version|)[ \/]([\w.]+)/.exec(a)||/(msie) ([\w.]+)/.exec(a)||a.indexOf("compatible")<0&&/(mozilla)(?:.*? rv:([\w.]+)|)/.exec(a)||[];return{browser:b[1]||"",version:b[2]||"0"}},a=p.uaMatch(g.userAgent),b={},a.browser&&(b[a.browser]=!0,b.version=a.version),b.webkit&&(b.safari=!0),p.browser=b,p.sub=function(){function a(b,c){return new a.fn.init(b,c)}p.extend(!0,a,this),a.superclass=this,a.fn=a.prototype=this(),a.fn.constructor=a,a.sub=this.sub,a.fn.init=function c(c,d){return d&&d instanceof p&&!(d instanceof a)&&(d=a(d)),p.fn.init.call(this,c,d,b)},a.fn.init.prototype=a.fn;var b=a(e);return a}}();var bH,bI,bJ,bK=/alpha\([^)]*\)/i,bL=/opacity=([^)]*)/,bM=/^(top|right|bottom|left)$/,bN=/^margin/,bO=new RegExp("^("+q+")(.*)$","i"),bP=new RegExp("^("+q+")(?!px)[a-z%]+$","i"),bQ=new RegExp("^([-+])=("+q+")","i"),bR={},bS={position:"absolute",visibility:"hidden",display:"block"},bT={letterSpacing:0,fontWeight:400,lineHeight:1},bU=["Top","Right","Bottom","Left"],bV=["Webkit","O","Moz","ms"],bW=p.fn.toggle;p.fn.extend({css:function(a,c){return p.access(this,function(a,c,d){return d!==b?p.style(a,c,d):p.css(a,c)},a,c,arguments.length>1)},show:function(){return bZ(this,!0)},hide:function(){return bZ(this)},toggle:function(a,b){var c=typeof a=="boolean";return p.isFunction(a)&&p.isFunction(b)?bW.apply(this,arguments):this.each(function(){(c?a:bY(this))?p(this).show():p(this).hide()})}}),p.extend({cssHooks:{opacity:{get:function(a,b){if(b){var c=bH(a,"opacity");return c===""?"1":c}}}},cssNumber:{fillOpacity:!0,fontWeight:!0,lineHeight:!0,opacity:!0,orphans:!0,widows:!0,zIndex:!0,zoom:!0},cssProps:{"float":p.support.cssFloat?"cssFloat":"styleFloat"},style:function(a,c,d,e){if(!a||a.nodeType===3||a.nodeType===8||!a.style)return;var f,g,h,i=p.camelCase(c),j=a.style;c=p.cssProps[i]||(p.cssProps[i]=bX(j,i)),h=p.cssHooks[c]||p.cssHooks[i];if(d===b)return h&&"get"in h&&(f=h.get(a,!1,e))!==b?f:j[c];g=typeof d,g==="string"&&(f=bQ.exec(d))&&(d=(f[1]+1)*f[2]+parseFloat(p.css(a,c)),g="number");if(d==null||g==="number"&&isNaN(d))return;g==="number"&&!p.cssNumber[i]&&(d+="px");if(!h||!("set"in h)||(d=h.set(a,d,e))!==b)try{j[c]=d}catch(k){}},css:function(a,c,d,e){var f,g,h,i=p.camelCase(c);return c=p.cssProps[i]||(p.cssProps[i]=bX(a.style,i)),h=p.cssHooks[c]||p.cssHooks[i],h&&"get"in h&&(f=h.get(a,!0,e)),f===b&&(f=bH(a,c)),f==="normal"&&c in bT&&(f=bT[c]),d||e!==b?(g=parseFloat(f),d||p.isNumeric(g)?g||0:f):f},swap:function(a,b,c){var d,e,f={};for(e in b)f[e]=a.style[e],a.style[e]=b[e];d=c.call(a);for(e in b)a.style[e]=f[e];return d}}),a.getComputedStyle?bH=function(a,b){var c,d,e,f,g=getComputedStyle(a,null),h=a.style;return g&&(c=g[b],c===""&&!p.contains(a.ownerDocument.documentElement,a)&&(c=p.style(a,b)),bP.test(c)&&bN.test(b)&&(d=h.width,e=h.minWidth,f=h.maxWidth,h.minWidth=h.maxWidth=h.width=c,c=g.width,h.width=d,h.minWidth=e,h.maxWidth=f)),c}:e.documentElement.currentStyle&&(bH=function(a,b){var c,d,e=a.currentStyle&&a.currentStyle[b],f=a.style;return e==null&&f&&f[b]&&(e=f[b]),bP.test(e)&&!bM.test(b)&&(c=f.left,d=a.runtimeStyle&&a.runtimeStyle.left,d&&(a.runtimeStyle.left=a.currentStyle.left),f.left=b==="fontSize"?"1em":e,e=f.pixelLeft+"px",f.left=c,d&&(a.runtimeStyle.left=d)),e===""?"auto":e}),p.each(["height","width"],function(a,b){p.cssHooks[b]={get:function(a,c,d){if(c)return a.offsetWidth!==0||bH(a,"display")!=="none"?ca(a,b,d):p.swap(a,bS,function(){return ca(a,b,d)})},set:function(a,c,d){return b$(a,c,d?b_(a,b,d,p.support.boxSizing&&p.css(a,"boxSizing")==="border-box"):0)}}}),p.support.opacity||(p.cssHooks.opacity={get:function(a,b){return bL.test((b&&a.currentStyle?a.currentStyle.filter:a.style.filter)||"")?.01*parseFloat(RegExp.$1)+"":b?"1":""},set:function(a,b){var c=a.style,d=a.currentStyle,e=p.isNumeric(b)?"alpha(opacity="+b*100+")":"",f=d&&d.filter||c.filter||"";c.zoom=1;if(b>=1&&p.trim(f.replace(bK,""))===""&&c.removeAttribute){c.removeAttribute("filter");if(d&&!d.filter)return}c.filter=bK.test(f)?f.replace(bK,e):f+" "+e}}),p(function(){p.support.reliableMarginRight||(p.cssHooks.marginRight={get:function(a,b){return p.swap(a,{display:"inline-block"},function(){if(b)return bH(a,"marginRight")})}}),!p.support.pixelPosition&&p.fn.position&&p.each(["top","left"],function(a,b){p.cssHooks[b]={get:function(a,c){if(c){var d=bH(a,b);return bP.test(d)?p(a).position()[b]+"px":d}}}})}),p.expr&&p.expr.filters&&(p.expr.filters.hidden=function(a){return a.offsetWidth===0&&a.offsetHeight===0||!p.support.reliableHiddenOffsets&&(a.style&&a.style.display||bH(a,"display"))==="none"},p.expr.filters.visible=function(a){return!p.expr.filters.hidden(a)}),p.each({margin:"",padding:"",border:"Width"},function(a,b){p.cssHooks[a+b]={expand:function(c){var d,e=typeof c=="string"?c.split(" "):[c],f={};for(d=0;d<4;d++)f[a+bU[d]+b]=e[d]||e[d-2]||e[0];return f}},bN.test(a)||(p.cssHooks[a+b].set=b$)});var cc=/%20/g,cd=/\[\]$/,ce=/\r?\n/g,cf=/^(?:color|date|datetime|datetime-local|email|hidden|month|number|password|range|search|tel|text|time|url|week)$/i,cg=/^(?:select|textarea)/i;p.fn.extend({serialize:function(){return p.param(this.serializeArray())},serializeArray:function(){return this.map(function(){return this.elements?p.makeArray(this.elements):this}).filter(function(){return this.name&&!this.disabled&&(this.checked||cg.test(this.nodeName)||cf.test(this.type))}).map(function(a,b){var c=p(this).val();return c==null?null:p.isArray(c)?p.map(c,function(a,c){return{name:b.name,value:a.replace(ce,"\r\n")}}):{name:b.name,value:c.replace(ce,"\r\n")}}).get()}}),p.param=function(a,c){var d,e=[],f=function(a,b){b=p.isFunction(b)?b():b==null?"":b,e[e.length]=encodeURIComponent(a)+"="+encodeURIComponent(b)};c===b&&(c=p.ajaxSettings&&p.ajaxSettings.traditional);if(p.isArray(a)||a.jquery&&!p.isPlainObject(a))p.each(a,function(){f(this.name,this.value)});else for(d in a)ch(d,a[d],c,f);return e.join("&").replace(cc,"+")};var ci,cj,ck=/#.*$/,cl=/^(.*?):[ \t]*([^\r\n]*)\r?$/mg,cm=/^(?:about|app|app\-storage|.+\-extension|file|res|widget):$/,cn=/^(?:GET|HEAD)$/,co=/^\/\//,cp=/\?/,cq=/)<[^<]*)*<\/script>/gi,cr=/([?&])_=[^&]*/,cs=/^([\w\+\.\-]+:)(?:\/\/([^\/?#:]*)(?::(\d+)|)|)/,ct=p.fn.load,cu={},cv={},cw=["*/"]+["*"];try{ci=f.href}catch(cx){ci=e.createElement("a"),ci.href="",ci=ci.href}cj=cs.exec(ci.toLowerCase())||[],p.fn.load=function(a,c,d){if(typeof a!="string"&&ct)return ct.apply(this,arguments);if(!this.length)return this;var e,f,g,h=this,i=a.indexOf(" ");return i>=0&&(e=a.slice(i,a.length),a=a.slice(0,i)),p.isFunction(c)?(d=c,c=b):typeof c=="object"&&(f="POST"),p.ajax({url:a,type:f,dataType:"html",data:c,complete:function(a,b){d&&h.each(d,g||[a.responseText,b,a])}}).done(function(a){g=arguments,h.html(e?p("
    ").append(a.replace(cq,"")).find(e):a)}),this},p.each("ajaxStart ajaxStop ajaxComplete ajaxError ajaxSuccess ajaxSend".split(" "),function(a,b){p.fn[b]=function(a){return this.on(b,a)}}),p.each(["get","post"],function(a,c){p[c]=function(a,d,e,f){return p.isFunction(d)&&(f=f||e,e=d,d=b),p.ajax({type:c,url:a,data:d,success:e,dataType:f})}}),p.extend({getScript:function(a,c){return p.get(a,b,c,"script")},getJSON:function(a,b,c){return p.get(a,b,c,"json")},ajaxSetup:function(a,b){return b?cA(a,p.ajaxSettings):(b=a,a=p.ajaxSettings),cA(a,b),a},ajaxSettings:{url:ci,isLocal:cm.test(cj[1]),global:!0,type:"GET",contentType:"application/x-www-form-urlencoded; charset=UTF-8",processData:!0,async:!0,accepts:{xml:"application/xml, text/xml",html:"text/html",text:"text/plain",json:"application/json, text/javascript","*":cw},contents:{xml:/xml/,html:/html/,json:/json/},responseFields:{xml:"responseXML",text:"responseText"},converters:{"* text":a.String,"text html":!0,"text json":p.parseJSON,"text xml":p.parseXML},flatOptions:{context:!0,url:!0}},ajaxPrefilter:cy(cu),ajaxTransport:cy(cv),ajax:function(a,c){function y(a,c,f,i){var k,s,t,u,w,y=c;if(v===2)return;v=2,h&&clearTimeout(h),g=b,e=i||"",x.readyState=a>0?4:0,f&&(u=cB(l,x,f));if(a>=200&&a<300||a===304)l.ifModified&&(w=x.getResponseHeader("Last-Modified"),w&&(p.lastModified[d]=w),w=x.getResponseHeader("Etag"),w&&(p.etag[d]=w)),a===304?(y="notmodified",k=!0):(k=cC(l,u),y=k.state,s=k.data,t=k.error,k=!t);else{t=y;if(!y||a)y="error",a<0&&(a=0)}x.status=a,x.statusText=""+(c||y),k?o.resolveWith(m,[s,y,x]):o.rejectWith(m,[x,y,t]),x.statusCode(r),r=b,j&&n.trigger("ajax"+(k?"Success":"Error"),[x,l,k?s:t]),q.fireWith(m,[x,y]),j&&(n.trigger("ajaxComplete",[x,l]),--p.active||p.event.trigger("ajaxStop"))}typeof a=="object"&&(c=a,a=b),c=c||{};var d,e,f,g,h,i,j,k,l=p.ajaxSetup({},c),m=l.context||l,n=m!==l&&(m.nodeType||m instanceof p)?p(m):p.event,o=p.Deferred(),q=p.Callbacks("once memory"),r=l.statusCode||{},t={},u={},v=0,w="canceled",x={readyState:0,setRequestHeader:function(a,b){if(!v){var c=a.toLowerCase();a=u[c]=u[c]||a,t[a]=b}return this},getAllResponseHeaders:function(){return v===2?e:null},getResponseHeader:function(a){var c;if(v===2){if(!f){f={};while(c=cl.exec(e))f[c[1].toLowerCase()]=c[2]}c=f[a.toLowerCase()]}return c===b?null:c},overrideMimeType:function(a){return v||(l.mimeType=a),this},abort:function(a){return a=a||w,g&&g.abort(a),y(0,a),this}};o.promise(x),x.success=x.done,x.error=x.fail,x.complete=q.add,x.statusCode=function(a){if(a){var b;if(v<2)for(b in a)r[b]=[r[b],a[b]];else b=a[x.status],x.always(b)}return this},l.url=((a||l.url)+"").replace(ck,"").replace(co,cj[1]+"//"),l.dataTypes=p.trim(l.dataType||"*").toLowerCase().split(s),l.crossDomain==null&&(i=cs.exec(l.url.toLowerCase()),l.crossDomain=!(!i||i[1]==cj[1]&&i[2]==cj[2]&&(i[3]||(i[1]==="http:"?80:443))==(cj[3]||(cj[1]==="http:"?80:443)))),l.data&&l.processData&&typeof l.data!="string"&&(l.data=p.param(l.data,l.traditional)),cz(cu,l,c,x);if(v===2)return x;j=l.global,l.type=l.type.toUpperCase(),l.hasContent=!cn.test(l.type),j&&p.active++===0&&p.event.trigger("ajaxStart");if(!l.hasContent){l.data&&(l.url+=(cp.test(l.url)?"&":"?")+l.data,delete l.data),d=l.url;if(l.cache===!1){var z=p.now(),A=l.url.replace(cr,"$1_="+z);l.url=A+(A===l.url?(cp.test(l.url)?"&":"?")+"_="+z:"")}}(l.data&&l.hasContent&&l.contentType!==!1||c.contentType)&&x.setRequestHeader("Content-Type",l.contentType),l.ifModified&&(d=d||l.url,p.lastModified[d]&&x.setRequestHeader("If-Modified-Since",p.lastModified[d]),p.etag[d]&&x.setRequestHeader("If-None-Match",p.etag[d])),x.setRequestHeader("Accept",l.dataTypes[0]&&l.accepts[l.dataTypes[0]]?l.accepts[l.dataTypes[0]]+(l.dataTypes[0]!=="*"?", "+cw+"; q=0.01":""):l.accepts["*"]);for(k in l.headers)x.setRequestHeader(k,l.headers[k]);if(!l.beforeSend||l.beforeSend.call(m,x,l)!==!1&&v!==2){w="abort";for(k in{success:1,error:1,complete:1})x[k](l[k]);g=cz(cv,l,c,x);if(!g)y(-1,"No Transport");else{x.readyState=1,j&&n.trigger("ajaxSend",[x,l]),l.async&&l.timeout>0&&(h=setTimeout(function(){x.abort("timeout")},l.timeout));try{v=1,g.send(t,y)}catch(B){if(v<2)y(-1,B);else throw B}}return x}return x.abort()},active:0,lastModified:{},etag:{}});var cD=[],cE=/\?/,cF=/(=)\?(?=&|$)|\?\?/,cG=p.now();p.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var a=cD.pop()||p.expando+"_"+cG++;return this[a]=!0,a}}),p.ajaxPrefilter("json jsonp",function(c,d,e){var f,g,h,i=c.data,j=c.url,k=c.jsonp!==!1,l=k&&cF.test(j),m=k&&!l&&typeof i=="string"&&!(c.contentType||"").indexOf("application/x-www-form-urlencoded")&&cF.test(i);if(c.dataTypes[0]==="jsonp"||l||m)return f=c.jsonpCallback=p.isFunction(c.jsonpCallback)?c.jsonpCallback():c.jsonpCallback,g=a[f],l?c.url=j.replace(cF,"$1"+f):m?c.data=i.replace(cF,"$1"+f):k&&(c.url+=(cE.test(j)?"&":"?")+c.jsonp+"="+f),c.converters["script json"]=function(){return h||p.error(f+" was not called"),h[0]},c.dataTypes[0]="json",a[f]=function(){h=arguments},e.always(function(){a[f]=g,c[f]&&(c.jsonpCallback=d.jsonpCallback,cD.push(f)),h&&p.isFunction(g)&&g(h[0]),h=g=b}),"script"}),p.ajaxSetup({accepts:{script:"text/javascript, application/javascript, application/ecmascript, application/x-ecmascript"},contents:{script:/javascript|ecmascript/},converters:{"text script":function(a){return p.globalEval(a),a}}}),p.ajaxPrefilter("script",function(a){a.cache===b&&(a.cache=!1),a.crossDomain&&(a.type="GET",a.global=!1)}),p.ajaxTransport("script",function(a){if(a.crossDomain){var c,d=e.head||e.getElementsByTagName("head")[0]||e.documentElement;return{send:function(f,g){c=e.createElement("script"),c.async="async",a.scriptCharset&&(c.charset=a.scriptCharset),c.src=a.url,c.onload=c.onreadystatechange=function(a,e){if(e||!c.readyState||/loaded|complete/.test(c.readyState))c.onload=c.onreadystatechange=null,d&&c.parentNode&&d.removeChild(c),c=b,e||g(200,"success")},d.insertBefore(c,d.firstChild)},abort:function(){c&&c.onload(0,1)}}}});var cH,cI=a.ActiveXObject?function(){for(var a in cH)cH[a](0,1)}:!1,cJ=0;p.ajaxSettings.xhr=a.ActiveXObject?function(){return!this.isLocal&&cK()||cL()}:cK,function(a){p.extend(p.support,{ajax:!!a,cors:!!a&&"withCredentials"in a})}(p.ajaxSettings.xhr()),p.support.ajax&&p.ajaxTransport(function(c){if(!c.crossDomain||p.support.cors){var d;return{send:function(e,f){var g,h,i=c.xhr();c.username?i.open(c.type,c.url,c.async,c.username,c.password):i.open(c.type,c.url,c.async);if(c.xhrFields)for(h in c.xhrFields)i[h]=c.xhrFields[h];c.mimeType&&i.overrideMimeType&&i.overrideMimeType(c.mimeType),!c.crossDomain&&!e["X-Requested-With"]&&(e["X-Requested-With"]="XMLHttpRequest");try{for(h in e)i.setRequestHeader(h,e[h])}catch(j){}i.send(c.hasContent&&c.data||null),d=function(a,e){var h,j,k,l,m;try{if(d&&(e||i.readyState===4)){d=b,g&&(i.onreadystatechange=p.noop,cI&&delete cH[g]);if(e)i.readyState!==4&&i.abort();else{h=i.status,k=i.getAllResponseHeaders(),l={},m=i.responseXML,m&&m.documentElement&&(l.xml=m);try{l.text=i.responseText}catch(a){}try{j=i.statusText}catch(n){j=""}!h&&c.isLocal&&!c.crossDomain?h=l.text?200:404:h===1223&&(h=204)}}}catch(o){e||f(-1,o)}l&&f(h,j,l,k)},c.async?i.readyState===4?setTimeout(d,0):(g=++cJ,cI&&(cH||(cH={},p(a).unload(cI)),cH[g]=d),i.onreadystatechange=d):d()},abort:function(){d&&d(0,1)}}}});var cM,cN,cO=/^(?:toggle|show|hide)$/,cP=new RegExp("^(?:([-+])=|)("+q+")([a-z%]*)$","i"),cQ=/queueHooks$/,cR=[cX],cS={"*":[function(a,b){var c,d,e,f=this.createTween(a,b),g=cP.exec(b),h=f.cur(),i=+h||0,j=1;if(g){c=+g[2],d=g[3]||(p.cssNumber[a]?"":"px");if(d!=="px"&&i){i=p.css(f.elem,a,!0)||c||1;do e=j=j||".5",i=i/j,p.style(f.elem,a,i+d),j=f.cur()/h;while(j!==1&&j!==e)}f.unit=d,f.start=i,f.end=g[1]?i+(g[1]+1)*c:c}return f}]};p.Animation=p.extend(cV,{tweener:function(a,b){p.isFunction(a)?(b=a,a=["*"]):a=a.split(" ");var c,d=0,e=a.length;for(;d-1,j={},k={},l,m;i?(k=e.position(),l=k.top,m=k.left):(l=parseFloat(g)||0,m=parseFloat(h)||0),p.isFunction(b)&&(b=b.call(a,c,f)),b.top!=null&&(j.top=b.top-f.top+l),b.left!=null&&(j.left=b.left-f.left+m),"using"in b?b.using.call(a,j):e.css(j)}},p.fn.extend({position:function(){if(!this[0])return;var a=this[0],b=this.offsetParent(),c=this.offset(),d=c$.test(b[0].nodeName)?{top:0,left:0}:b.offset();return c.top-=parseFloat(p.css(a,"marginTop"))||0,c.left-=parseFloat(p.css(a,"marginLeft"))||0,d.top+=parseFloat(p.css(b[0],"borderTopWidth"))||0,d.left+=parseFloat(p.css(b[0],"borderLeftWidth"))||0,{top:c.top-d.top,left:c.left-d.left}},offsetParent:function(){return this.map(function(){var a=this.offsetParent||e.body;while(a&&!c$.test(a.nodeName)&&p.css(a,"position")==="static")a=a.offsetParent;return a||e.body})}}),p.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(a,c){var d=/Y/.test(c);p.fn[a]=function(e){return p.access(this,function(a,e,f){var g=c_(a);if(f===b)return g?c in g?g[c]:g.document.documentElement[e]:a[e];g?g.scrollTo(d?p(g).scrollLeft():f,d?f:p(g).scrollTop()):a[e]=f},a,e,arguments.length,null)}}),p.each({Height:"height",Width:"width"},function(a,c){p.each({padding:"inner"+a,content:c,"":"outer"+a},function(d,e){p.fn[e]=function(e,f){var g=arguments.length&&(d||typeof e!="boolean"),h=d||(e===!0||f===!0?"margin":"border");return p.access(this,function(c,d,e){var f;return p.isWindow(c)?c.document.documentElement["client"+a]:c.nodeType===9?(f=c.documentElement,Math.max(c.body["scroll"+a],f["scroll"+a],c.body["offset"+a],f["offset"+a],f["client"+a])):e===b?p.css(c,d,e,h):p.style(c,d,e,h)},c,g?e:b,g)}})}),a.jQuery=a.$=p,typeof define=="function"&&define.amd&&define.amd.jQuery&&define("jquery",[],function(){return p})})(window); \ No newline at end of file diff --git a/docs/js/vendor/jquery-3.4.1.min.js b/docs/js/vendor/jquery-3.4.1.min.js new file mode 100644 index 0000000000000..07c00cd227da0 --- /dev/null +++ b/docs/js/vendor/jquery-3.4.1.min.js @@ -0,0 +1,2 @@ +/*! jQuery v3.4.1 | (c) JS Foundation and other contributors | jquery.org/license */ +!function(e,t){"use strict";"object"==typeof module&&"object"==typeof module.exports?module.exports=e.document?t(e,!0):function(e){if(!e.document)throw new Error("jQuery requires a window with a document");return t(e)}:t(e)}("undefined"!=typeof window?window:this,function(C,e){"use strict";var t=[],E=C.document,r=Object.getPrototypeOf,s=t.slice,g=t.concat,u=t.push,i=t.indexOf,n={},o=n.toString,v=n.hasOwnProperty,a=v.toString,l=a.call(Object),y={},m=function(e){return"function"==typeof e&&"number"!=typeof e.nodeType},x=function(e){return null!=e&&e===e.window},c={type:!0,src:!0,nonce:!0,noModule:!0};function b(e,t,n){var r,i,o=(n=n||E).createElement("script");if(o.text=e,t)for(r in c)(i=t[r]||t.getAttribute&&t.getAttribute(r))&&o.setAttribute(r,i);n.head.appendChild(o).parentNode.removeChild(o)}function w(e){return null==e?e+"":"object"==typeof e||"function"==typeof e?n[o.call(e)]||"object":typeof e}var f="3.4.1",k=function(e,t){return new k.fn.init(e,t)},p=/^[\s\uFEFF\xA0]+|[\s\uFEFF\xA0]+$/g;function d(e){var t=!!e&&"length"in e&&e.length,n=w(e);return!m(e)&&!x(e)&&("array"===n||0===t||"number"==typeof t&&0+~]|"+M+")"+M+"*"),U=new RegExp(M+"|>"),X=new RegExp($),V=new RegExp("^"+I+"$"),G={ID:new RegExp("^#("+I+")"),CLASS:new RegExp("^\\.("+I+")"),TAG:new RegExp("^("+I+"|[*])"),ATTR:new RegExp("^"+W),PSEUDO:new RegExp("^"+$),CHILD:new RegExp("^:(only|first|last|nth|nth-last)-(child|of-type)(?:\\("+M+"*(even|odd|(([+-]|)(\\d*)n|)"+M+"*(?:([+-]|)"+M+"*(\\d+)|))"+M+"*\\)|)","i"),bool:new RegExp("^(?:"+R+")$","i"),needsContext:new RegExp("^"+M+"*[>+~]|:(even|odd|eq|gt|lt|nth|first|last)(?:\\("+M+"*((?:-\\d)?\\d*)"+M+"*\\)|)(?=[^-]|$)","i")},Y=/HTML$/i,Q=/^(?:input|select|textarea|button)$/i,J=/^h\d$/i,K=/^[^{]+\{\s*\[native \w/,Z=/^(?:#([\w-]+)|(\w+)|\.([\w-]+))$/,ee=/[+~]/,te=new RegExp("\\\\([\\da-f]{1,6}"+M+"?|("+M+")|.)","ig"),ne=function(e,t,n){var r="0x"+t-65536;return r!=r||n?t:r<0?String.fromCharCode(r+65536):String.fromCharCode(r>>10|55296,1023&r|56320)},re=/([\0-\x1f\x7f]|^-?\d)|^-$|[^\0-\x1f\x7f-\uFFFF\w-]/g,ie=function(e,t){return t?"\0"===e?"\ufffd":e.slice(0,-1)+"\\"+e.charCodeAt(e.length-1).toString(16)+" ":"\\"+e},oe=function(){T()},ae=be(function(e){return!0===e.disabled&&"fieldset"===e.nodeName.toLowerCase()},{dir:"parentNode",next:"legend"});try{H.apply(t=O.call(m.childNodes),m.childNodes),t[m.childNodes.length].nodeType}catch(e){H={apply:t.length?function(e,t){L.apply(e,O.call(t))}:function(e,t){var n=e.length,r=0;while(e[n++]=t[r++]);e.length=n-1}}}function se(t,e,n,r){var i,o,a,s,u,l,c,f=e&&e.ownerDocument,p=e?e.nodeType:9;if(n=n||[],"string"!=typeof t||!t||1!==p&&9!==p&&11!==p)return n;if(!r&&((e?e.ownerDocument||e:m)!==C&&T(e),e=e||C,E)){if(11!==p&&(u=Z.exec(t)))if(i=u[1]){if(9===p){if(!(a=e.getElementById(i)))return n;if(a.id===i)return n.push(a),n}else if(f&&(a=f.getElementById(i))&&y(e,a)&&a.id===i)return n.push(a),n}else{if(u[2])return H.apply(n,e.getElementsByTagName(t)),n;if((i=u[3])&&d.getElementsByClassName&&e.getElementsByClassName)return H.apply(n,e.getElementsByClassName(i)),n}if(d.qsa&&!A[t+" "]&&(!v||!v.test(t))&&(1!==p||"object"!==e.nodeName.toLowerCase())){if(c=t,f=e,1===p&&U.test(t)){(s=e.getAttribute("id"))?s=s.replace(re,ie):e.setAttribute("id",s=k),o=(l=h(t)).length;while(o--)l[o]="#"+s+" "+xe(l[o]);c=l.join(","),f=ee.test(t)&&ye(e.parentNode)||e}try{return H.apply(n,f.querySelectorAll(c)),n}catch(e){A(t,!0)}finally{s===k&&e.removeAttribute("id")}}}return g(t.replace(B,"$1"),e,n,r)}function ue(){var r=[];return function e(t,n){return r.push(t+" ")>b.cacheLength&&delete e[r.shift()],e[t+" "]=n}}function le(e){return e[k]=!0,e}function ce(e){var t=C.createElement("fieldset");try{return!!e(t)}catch(e){return!1}finally{t.parentNode&&t.parentNode.removeChild(t),t=null}}function fe(e,t){var n=e.split("|"),r=n.length;while(r--)b.attrHandle[n[r]]=t}function pe(e,t){var n=t&&e,r=n&&1===e.nodeType&&1===t.nodeType&&e.sourceIndex-t.sourceIndex;if(r)return r;if(n)while(n=n.nextSibling)if(n===t)return-1;return e?1:-1}function de(t){return function(e){return"input"===e.nodeName.toLowerCase()&&e.type===t}}function he(n){return function(e){var t=e.nodeName.toLowerCase();return("input"===t||"button"===t)&&e.type===n}}function ge(t){return function(e){return"form"in e?e.parentNode&&!1===e.disabled?"label"in e?"label"in e.parentNode?e.parentNode.disabled===t:e.disabled===t:e.isDisabled===t||e.isDisabled!==!t&&ae(e)===t:e.disabled===t:"label"in e&&e.disabled===t}}function ve(a){return le(function(o){return o=+o,le(function(e,t){var n,r=a([],e.length,o),i=r.length;while(i--)e[n=r[i]]&&(e[n]=!(t[n]=e[n]))})})}function ye(e){return e&&"undefined"!=typeof e.getElementsByTagName&&e}for(e in d=se.support={},i=se.isXML=function(e){var t=e.namespaceURI,n=(e.ownerDocument||e).documentElement;return!Y.test(t||n&&n.nodeName||"HTML")},T=se.setDocument=function(e){var t,n,r=e?e.ownerDocument||e:m;return r!==C&&9===r.nodeType&&r.documentElement&&(a=(C=r).documentElement,E=!i(C),m!==C&&(n=C.defaultView)&&n.top!==n&&(n.addEventListener?n.addEventListener("unload",oe,!1):n.attachEvent&&n.attachEvent("onunload",oe)),d.attributes=ce(function(e){return e.className="i",!e.getAttribute("className")}),d.getElementsByTagName=ce(function(e){return e.appendChild(C.createComment("")),!e.getElementsByTagName("*").length}),d.getElementsByClassName=K.test(C.getElementsByClassName),d.getById=ce(function(e){return a.appendChild(e).id=k,!C.getElementsByName||!C.getElementsByName(k).length}),d.getById?(b.filter.ID=function(e){var t=e.replace(te,ne);return function(e){return e.getAttribute("id")===t}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n=t.getElementById(e);return n?[n]:[]}}):(b.filter.ID=function(e){var n=e.replace(te,ne);return function(e){var t="undefined"!=typeof e.getAttributeNode&&e.getAttributeNode("id");return t&&t.value===n}},b.find.ID=function(e,t){if("undefined"!=typeof t.getElementById&&E){var n,r,i,o=t.getElementById(e);if(o){if((n=o.getAttributeNode("id"))&&n.value===e)return[o];i=t.getElementsByName(e),r=0;while(o=i[r++])if((n=o.getAttributeNode("id"))&&n.value===e)return[o]}return[]}}),b.find.TAG=d.getElementsByTagName?function(e,t){return"undefined"!=typeof t.getElementsByTagName?t.getElementsByTagName(e):d.qsa?t.querySelectorAll(e):void 0}:function(e,t){var n,r=[],i=0,o=t.getElementsByTagName(e);if("*"===e){while(n=o[i++])1===n.nodeType&&r.push(n);return r}return o},b.find.CLASS=d.getElementsByClassName&&function(e,t){if("undefined"!=typeof t.getElementsByClassName&&E)return t.getElementsByClassName(e)},s=[],v=[],(d.qsa=K.test(C.querySelectorAll))&&(ce(function(e){a.appendChild(e).innerHTML="",e.querySelectorAll("[msallowcapture^='']").length&&v.push("[*^$]="+M+"*(?:''|\"\")"),e.querySelectorAll("[selected]").length||v.push("\\["+M+"*(?:value|"+R+")"),e.querySelectorAll("[id~="+k+"-]").length||v.push("~="),e.querySelectorAll(":checked").length||v.push(":checked"),e.querySelectorAll("a#"+k+"+*").length||v.push(".#.+[+~]")}),ce(function(e){e.innerHTML="";var t=C.createElement("input");t.setAttribute("type","hidden"),e.appendChild(t).setAttribute("name","D"),e.querySelectorAll("[name=d]").length&&v.push("name"+M+"*[*^$|!~]?="),2!==e.querySelectorAll(":enabled").length&&v.push(":enabled",":disabled"),a.appendChild(e).disabled=!0,2!==e.querySelectorAll(":disabled").length&&v.push(":enabled",":disabled"),e.querySelectorAll("*,:x"),v.push(",.*:")})),(d.matchesSelector=K.test(c=a.matches||a.webkitMatchesSelector||a.mozMatchesSelector||a.oMatchesSelector||a.msMatchesSelector))&&ce(function(e){d.disconnectedMatch=c.call(e,"*"),c.call(e,"[s!='']:x"),s.push("!=",$)}),v=v.length&&new RegExp(v.join("|")),s=s.length&&new RegExp(s.join("|")),t=K.test(a.compareDocumentPosition),y=t||K.test(a.contains)?function(e,t){var n=9===e.nodeType?e.documentElement:e,r=t&&t.parentNode;return e===r||!(!r||1!==r.nodeType||!(n.contains?n.contains(r):e.compareDocumentPosition&&16&e.compareDocumentPosition(r)))}:function(e,t){if(t)while(t=t.parentNode)if(t===e)return!0;return!1},D=t?function(e,t){if(e===t)return l=!0,0;var n=!e.compareDocumentPosition-!t.compareDocumentPosition;return n||(1&(n=(e.ownerDocument||e)===(t.ownerDocument||t)?e.compareDocumentPosition(t):1)||!d.sortDetached&&t.compareDocumentPosition(e)===n?e===C||e.ownerDocument===m&&y(m,e)?-1:t===C||t.ownerDocument===m&&y(m,t)?1:u?P(u,e)-P(u,t):0:4&n?-1:1)}:function(e,t){if(e===t)return l=!0,0;var n,r=0,i=e.parentNode,o=t.parentNode,a=[e],s=[t];if(!i||!o)return e===C?-1:t===C?1:i?-1:o?1:u?P(u,e)-P(u,t):0;if(i===o)return pe(e,t);n=e;while(n=n.parentNode)a.unshift(n);n=t;while(n=n.parentNode)s.unshift(n);while(a[r]===s[r])r++;return r?pe(a[r],s[r]):a[r]===m?-1:s[r]===m?1:0}),C},se.matches=function(e,t){return se(e,null,null,t)},se.matchesSelector=function(e,t){if((e.ownerDocument||e)!==C&&T(e),d.matchesSelector&&E&&!A[t+" "]&&(!s||!s.test(t))&&(!v||!v.test(t)))try{var n=c.call(e,t);if(n||d.disconnectedMatch||e.document&&11!==e.document.nodeType)return n}catch(e){A(t,!0)}return 0":{dir:"parentNode",first:!0}," ":{dir:"parentNode"},"+":{dir:"previousSibling",first:!0},"~":{dir:"previousSibling"}},preFilter:{ATTR:function(e){return e[1]=e[1].replace(te,ne),e[3]=(e[3]||e[4]||e[5]||"").replace(te,ne),"~="===e[2]&&(e[3]=" "+e[3]+" "),e.slice(0,4)},CHILD:function(e){return e[1]=e[1].toLowerCase(),"nth"===e[1].slice(0,3)?(e[3]||se.error(e[0]),e[4]=+(e[4]?e[5]+(e[6]||1):2*("even"===e[3]||"odd"===e[3])),e[5]=+(e[7]+e[8]||"odd"===e[3])):e[3]&&se.error(e[0]),e},PSEUDO:function(e){var t,n=!e[6]&&e[2];return G.CHILD.test(e[0])?null:(e[3]?e[2]=e[4]||e[5]||"":n&&X.test(n)&&(t=h(n,!0))&&(t=n.indexOf(")",n.length-t)-n.length)&&(e[0]=e[0].slice(0,t),e[2]=n.slice(0,t)),e.slice(0,3))}},filter:{TAG:function(e){var t=e.replace(te,ne).toLowerCase();return"*"===e?function(){return!0}:function(e){return e.nodeName&&e.nodeName.toLowerCase()===t}},CLASS:function(e){var t=p[e+" "];return t||(t=new RegExp("(^|"+M+")"+e+"("+M+"|$)"))&&p(e,function(e){return t.test("string"==typeof e.className&&e.className||"undefined"!=typeof e.getAttribute&&e.getAttribute("class")||"")})},ATTR:function(n,r,i){return function(e){var t=se.attr(e,n);return null==t?"!="===r:!r||(t+="","="===r?t===i:"!="===r?t!==i:"^="===r?i&&0===t.indexOf(i):"*="===r?i&&-1:\x20\t\r\n\f]*)[\x20\t\r\n\f]*\/?>(?:<\/\1>|)$/i;function j(e,n,r){return m(n)?k.grep(e,function(e,t){return!!n.call(e,t,e)!==r}):n.nodeType?k.grep(e,function(e){return e===n!==r}):"string"!=typeof n?k.grep(e,function(e){return-1)[^>]*|#([\w-]+))$/;(k.fn.init=function(e,t,n){var r,i;if(!e)return this;if(n=n||q,"string"==typeof e){if(!(r="<"===e[0]&&">"===e[e.length-1]&&3<=e.length?[null,e,null]:L.exec(e))||!r[1]&&t)return!t||t.jquery?(t||n).find(e):this.constructor(t).find(e);if(r[1]){if(t=t instanceof k?t[0]:t,k.merge(this,k.parseHTML(r[1],t&&t.nodeType?t.ownerDocument||t:E,!0)),D.test(r[1])&&k.isPlainObject(t))for(r in t)m(this[r])?this[r](t[r]):this.attr(r,t[r]);return this}return(i=E.getElementById(r[2]))&&(this[0]=i,this.length=1),this}return e.nodeType?(this[0]=e,this.length=1,this):m(e)?void 0!==n.ready?n.ready(e):e(k):k.makeArray(e,this)}).prototype=k.fn,q=k(E);var H=/^(?:parents|prev(?:Until|All))/,O={children:!0,contents:!0,next:!0,prev:!0};function P(e,t){while((e=e[t])&&1!==e.nodeType);return e}k.fn.extend({has:function(e){var t=k(e,this),n=t.length;return this.filter(function(){for(var e=0;e\x20\t\r\n\f]*)/i,he=/^$|^module$|\/(?:java|ecma)script/i,ge={option:[1,""],thead:[1,"
    ","
    "],col:[2,"","
    "],tr:[2,"","
    "],td:[3,"","
    "],_default:[0,"",""]};function ve(e,t){var n;return n="undefined"!=typeof e.getElementsByTagName?e.getElementsByTagName(t||"*"):"undefined"!=typeof e.querySelectorAll?e.querySelectorAll(t||"*"):[],void 0===t||t&&A(e,t)?k.merge([e],n):n}function ye(e,t){for(var n=0,r=e.length;nx",y.noCloneChecked=!!me.cloneNode(!0).lastChild.defaultValue;var Te=/^key/,Ce=/^(?:mouse|pointer|contextmenu|drag|drop)|click/,Ee=/^([^.]*)(?:\.(.+)|)/;function ke(){return!0}function Se(){return!1}function Ne(e,t){return e===function(){try{return E.activeElement}catch(e){}}()==("focus"===t)}function Ae(e,t,n,r,i,o){var a,s;if("object"==typeof t){for(s in"string"!=typeof n&&(r=r||n,n=void 0),t)Ae(e,s,n,r,t[s],o);return e}if(null==r&&null==i?(i=n,r=n=void 0):null==i&&("string"==typeof n?(i=r,r=void 0):(i=r,r=n,n=void 0)),!1===i)i=Se;else if(!i)return e;return 1===o&&(a=i,(i=function(e){return k().off(e),a.apply(this,arguments)}).guid=a.guid||(a.guid=k.guid++)),e.each(function(){k.event.add(this,t,i,r,n)})}function De(e,i,o){o?(Q.set(e,i,!1),k.event.add(e,i,{namespace:!1,handler:function(e){var t,n,r=Q.get(this,i);if(1&e.isTrigger&&this[i]){if(r.length)(k.event.special[i]||{}).delegateType&&e.stopPropagation();else if(r=s.call(arguments),Q.set(this,i,r),t=o(this,i),this[i](),r!==(n=Q.get(this,i))||t?Q.set(this,i,!1):n={},r!==n)return e.stopImmediatePropagation(),e.preventDefault(),n.value}else r.length&&(Q.set(this,i,{value:k.event.trigger(k.extend(r[0],k.Event.prototype),r.slice(1),this)}),e.stopImmediatePropagation())}})):void 0===Q.get(e,i)&&k.event.add(e,i,ke)}k.event={global:{},add:function(t,e,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.get(t);if(v){n.handler&&(n=(o=n).handler,i=o.selector),i&&k.find.matchesSelector(ie,i),n.guid||(n.guid=k.guid++),(u=v.events)||(u=v.events={}),(a=v.handle)||(a=v.handle=function(e){return"undefined"!=typeof k&&k.event.triggered!==e.type?k.event.dispatch.apply(t,arguments):void 0}),l=(e=(e||"").match(R)||[""]).length;while(l--)d=g=(s=Ee.exec(e[l])||[])[1],h=(s[2]||"").split(".").sort(),d&&(f=k.event.special[d]||{},d=(i?f.delegateType:f.bindType)||d,f=k.event.special[d]||{},c=k.extend({type:d,origType:g,data:r,handler:n,guid:n.guid,selector:i,needsContext:i&&k.expr.match.needsContext.test(i),namespace:h.join(".")},o),(p=u[d])||((p=u[d]=[]).delegateCount=0,f.setup&&!1!==f.setup.call(t,r,h,a)||t.addEventListener&&t.addEventListener(d,a)),f.add&&(f.add.call(t,c),c.handler.guid||(c.handler.guid=n.guid)),i?p.splice(p.delegateCount++,0,c):p.push(c),k.event.global[d]=!0)}},remove:function(e,t,n,r,i){var o,a,s,u,l,c,f,p,d,h,g,v=Q.hasData(e)&&Q.get(e);if(v&&(u=v.events)){l=(t=(t||"").match(R)||[""]).length;while(l--)if(d=g=(s=Ee.exec(t[l])||[])[1],h=(s[2]||"").split(".").sort(),d){f=k.event.special[d]||{},p=u[d=(r?f.delegateType:f.bindType)||d]||[],s=s[2]&&new RegExp("(^|\\.)"+h.join("\\.(?:.*\\.|)")+"(\\.|$)"),a=o=p.length;while(o--)c=p[o],!i&&g!==c.origType||n&&n.guid!==c.guid||s&&!s.test(c.namespace)||r&&r!==c.selector&&("**"!==r||!c.selector)||(p.splice(o,1),c.selector&&p.delegateCount--,f.remove&&f.remove.call(e,c));a&&!p.length&&(f.teardown&&!1!==f.teardown.call(e,h,v.handle)||k.removeEvent(e,d,v.handle),delete u[d])}else for(d in u)k.event.remove(e,d+t[l],n,r,!0);k.isEmptyObject(u)&&Q.remove(e,"handle events")}},dispatch:function(e){var t,n,r,i,o,a,s=k.event.fix(e),u=new Array(arguments.length),l=(Q.get(this,"events")||{})[s.type]||[],c=k.event.special[s.type]||{};for(u[0]=s,t=1;t\x20\t\r\n\f]*)[^>]*)\/>/gi,qe=/\s*$/g;function Oe(e,t){return A(e,"table")&&A(11!==t.nodeType?t:t.firstChild,"tr")&&k(e).children("tbody")[0]||e}function Pe(e){return e.type=(null!==e.getAttribute("type"))+"/"+e.type,e}function Re(e){return"true/"===(e.type||"").slice(0,5)?e.type=e.type.slice(5):e.removeAttribute("type"),e}function Me(e,t){var n,r,i,o,a,s,u,l;if(1===t.nodeType){if(Q.hasData(e)&&(o=Q.access(e),a=Q.set(t,o),l=o.events))for(i in delete a.handle,a.events={},l)for(n=0,r=l[i].length;n")},clone:function(e,t,n){var r,i,o,a,s,u,l,c=e.cloneNode(!0),f=oe(e);if(!(y.noCloneChecked||1!==e.nodeType&&11!==e.nodeType||k.isXMLDoc(e)))for(a=ve(c),r=0,i=(o=ve(e)).length;r").attr(n.scriptAttrs||{}).prop({charset:n.scriptCharset,src:n.url}).on("load error",i=function(e){r.remove(),i=null,e&&t("error"===e.type?404:200,e.type)}),E.head.appendChild(r[0])},abort:function(){i&&i()}}});var Vt,Gt=[],Yt=/(=)\?(?=&|$)|\?\?/;k.ajaxSetup({jsonp:"callback",jsonpCallback:function(){var e=Gt.pop()||k.expando+"_"+kt++;return this[e]=!0,e}}),k.ajaxPrefilter("json jsonp",function(e,t,n){var r,i,o,a=!1!==e.jsonp&&(Yt.test(e.url)?"url":"string"==typeof e.data&&0===(e.contentType||"").indexOf("application/x-www-form-urlencoded")&&Yt.test(e.data)&&"data");if(a||"jsonp"===e.dataTypes[0])return r=e.jsonpCallback=m(e.jsonpCallback)?e.jsonpCallback():e.jsonpCallback,a?e[a]=e[a].replace(Yt,"$1"+r):!1!==e.jsonp&&(e.url+=(St.test(e.url)?"&":"?")+e.jsonp+"="+r),e.converters["script json"]=function(){return o||k.error(r+" was not called"),o[0]},e.dataTypes[0]="json",i=C[r],C[r]=function(){o=arguments},n.always(function(){void 0===i?k(C).removeProp(r):C[r]=i,e[r]&&(e.jsonpCallback=t.jsonpCallback,Gt.push(r)),o&&m(i)&&i(o[0]),o=i=void 0}),"script"}),y.createHTMLDocument=((Vt=E.implementation.createHTMLDocument("").body).innerHTML="
    ",2===Vt.childNodes.length),k.parseHTML=function(e,t,n){return"string"!=typeof e?[]:("boolean"==typeof t&&(n=t,t=!1),t||(y.createHTMLDocument?((r=(t=E.implementation.createHTMLDocument("")).createElement("base")).href=E.location.href,t.head.appendChild(r)):t=E),o=!n&&[],(i=D.exec(e))?[t.createElement(i[1])]:(i=we([e],t,o),o&&o.length&&k(o).remove(),k.merge([],i.childNodes)));var r,i,o},k.fn.load=function(e,t,n){var r,i,o,a=this,s=e.indexOf(" ");return-1").append(k.parseHTML(e)).find(r):e)}).always(n&&function(e,t){a.each(function(){n.apply(this,o||[e.responseText,t,e])})}),this},k.each(["ajaxStart","ajaxStop","ajaxComplete","ajaxError","ajaxSuccess","ajaxSend"],function(e,t){k.fn[t]=function(e){return this.on(t,e)}}),k.expr.pseudos.animated=function(t){return k.grep(k.timers,function(e){return t===e.elem}).length},k.offset={setOffset:function(e,t,n){var r,i,o,a,s,u,l=k.css(e,"position"),c=k(e),f={};"static"===l&&(e.style.position="relative"),s=c.offset(),o=k.css(e,"top"),u=k.css(e,"left"),("absolute"===l||"fixed"===l)&&-1<(o+u).indexOf("auto")?(a=(r=c.position()).top,i=r.left):(a=parseFloat(o)||0,i=parseFloat(u)||0),m(t)&&(t=t.call(e,n,k.extend({},s))),null!=t.top&&(f.top=t.top-s.top+a),null!=t.left&&(f.left=t.left-s.left+i),"using"in t?t.using.call(e,f):c.css(f)}},k.fn.extend({offset:function(t){if(arguments.length)return void 0===t?this:this.each(function(e){k.offset.setOffset(this,t,e)});var e,n,r=this[0];return r?r.getClientRects().length?(e=r.getBoundingClientRect(),n=r.ownerDocument.defaultView,{top:e.top+n.pageYOffset,left:e.left+n.pageXOffset}):{top:0,left:0}:void 0},position:function(){if(this[0]){var e,t,n,r=this[0],i={top:0,left:0};if("fixed"===k.css(r,"position"))t=r.getBoundingClientRect();else{t=this.offset(),n=r.ownerDocument,e=r.offsetParent||n.documentElement;while(e&&(e===n.body||e===n.documentElement)&&"static"===k.css(e,"position"))e=e.parentNode;e&&e!==r&&1===e.nodeType&&((i=k(e).offset()).top+=k.css(e,"borderTopWidth",!0),i.left+=k.css(e,"borderLeftWidth",!0))}return{top:t.top-i.top-k.css(r,"marginTop",!0),left:t.left-i.left-k.css(r,"marginLeft",!0)}}},offsetParent:function(){return this.map(function(){var e=this.offsetParent;while(e&&"static"===k.css(e,"position"))e=e.offsetParent;return e||ie})}}),k.each({scrollLeft:"pageXOffset",scrollTop:"pageYOffset"},function(t,i){var o="pageYOffset"===i;k.fn[t]=function(e){return _(this,function(e,t,n){var r;if(x(e)?r=e:9===e.nodeType&&(r=e.defaultView),void 0===n)return r?r[i]:e[t];r?r.scrollTo(o?r.pageXOffset:n,o?n:r.pageYOffset):e[t]=n},t,e,arguments.length)}}),k.each(["top","left"],function(e,n){k.cssHooks[n]=ze(y.pixelPosition,function(e,t){if(t)return t=_e(e,n),$e.test(t)?k(e).position()[n]+"px":t})}),k.each({Height:"height",Width:"width"},function(a,s){k.each({padding:"inner"+a,content:s,"":"outer"+a},function(r,o){k.fn[o]=function(e,t){var n=arguments.length&&(r||"boolean"!=typeof e),i=r||(!0===e||!0===t?"margin":"border");return _(this,function(e,t,n){var r;return x(e)?0===o.indexOf("outer")?e["inner"+a]:e.document.documentElement["client"+a]:9===e.nodeType?(r=e.documentElement,Math.max(e.body["scroll"+a],r["scroll"+a],e.body["offset"+a],r["offset"+a],r["client"+a])):void 0===n?k.css(e,t,i):k.style(e,t,n,i)},s,n?e:void 0,n)}})}),k.each("blur focus focusin focusout resize scroll click dblclick mousedown mouseup mousemove mouseover mouseout mouseenter mouseleave change select submit keydown keypress keyup contextmenu".split(" "),function(e,n){k.fn[n]=function(e,t){return 0 This section has been moved into the diff --git a/docs/ml-classification-regression.md b/docs/ml-classification-regression.md index b3d109039da4d..b83b4ba08a5fd 100644 --- a/docs/ml-classification-regression.md +++ b/docs/ml-classification-regression.md @@ -2,6 +2,21 @@ layout: global title: Classification and regression displayTitle: Classification and regression +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. --- @@ -941,9 +956,9 @@ Essentially isotonic regression is a best fitting the original data points. We implement a -[pool adjacent violators algorithm](http://doi.org/10.1198/TECH.2010.10111) +[pool adjacent violators algorithm](https://doi.org/10.1198/TECH.2010.10111) which uses an approach to -[parallelizing isotonic regression](http://doi.org/10.1007/978-3-642-99789-1_10). +[parallelizing isotonic regression](https://doi.org/10.1007/978-3-642-99789-1_10). The training input is a DataFrame which contains three columns label, features and weight. Additionally, IsotonicRegression algorithm has one optional parameter called $isotonic$ defaulting to true. diff --git a/docs/ml-clustering.md b/docs/ml-clustering.md index 1186fb73d0faf..2775d0421ccca 100644 --- a/docs/ml-clustering.md +++ b/docs/ml-clustering.md @@ -2,6 +2,21 @@ layout: global title: Clustering displayTitle: Clustering +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 describes clustering algorithms in MLlib. @@ -265,3 +280,50 @@ Refer to the [R API docs](api/R/spark.gaussianMixture.html) for more details.
    + +## Power Iteration Clustering (PIC) + +Power Iteration Clustering (PIC) is a scalable graph clustering algorithm +developed by [Lin and Cohen](http://www.cs.cmu.edu/~frank/papers/icml2010-pic-final.pdf). +From the abstract: PIC finds a very low-dimensional embedding of a dataset +using truncated power iteration on a normalized pair-wise similarity matrix of the data. + +`spark.ml`'s PowerIterationClustering implementation takes the following parameters: + +* `k`: the number of clusters to create +* `initMode`: param for the initialization algorithm +* `maxIter`: param for maximum number of iterations +* `srcCol`: param for the name of the input column for source vertex IDs +* `dstCol`: name of the input column for destination vertex IDs +* `weightCol`: Param for weight column name + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.clustering.PowerIterationClustering) for more details. + +{% include_example scala/org/apache/spark/examples/ml/PowerIterationClusteringExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/clustering/PowerIterationClustering.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaPowerIterationClusteringExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.clustering.PowerIterationClustering) for more details. + +{% include_example python/ml/power_iteration_clustering_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.powerIterationClustering.html) for more details. + +{% include_example r/ml/powerIterationClustering.R %} +
    + +
    diff --git a/docs/ml-collaborative-filtering.md b/docs/ml-collaborative-filtering.md index 8b0f287dc39ad..e6e596bed110b 100644 --- a/docs/ml-collaborative-filtering.md +++ b/docs/ml-collaborative-filtering.md @@ -2,6 +2,21 @@ layout: global title: Collaborative Filtering displayTitle: Collaborative Filtering +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. --- * Table of contents @@ -41,7 +56,7 @@ for example, users giving ratings to movies. It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, clicks, purchases, likes, shares etc.). The approach used in `spark.ml` to deal with such data is taken -from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +from [Collaborative Filtering for Implicit Feedback Datasets](https://doi.org/10.1109/ICDM.2008.22). Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data as numbers representing the *strength* in observations of user actions (such as the number of clicks, or the cumulative duration someone spent viewing a movie). Those numbers are then related to the level of @@ -55,7 +70,7 @@ We scale the regularization parameter `regParam` in solving each least squares p the number of ratings the user generated in updating user factors, or the number of ratings the product received in updating product factors. This approach is named "ALS-WR" and discussed in the paper -"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](https://doi.org/10.1007/978-3-540-68880-8_32)". It makes `regParam` less dependent on the scale of the dataset, so we can apply the best parameter learned from a sampled subset to the full dataset and expect similar performance. diff --git a/docs/ml-datasource.md b/docs/ml-datasource.md new file mode 100644 index 0000000000000..71bec9c798ee1 --- /dev/null +++ b/docs/ml-datasource.md @@ -0,0 +1,123 @@ +--- +layout: global +title: Data sources +displayTitle: Data sources +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. +--- + +In this section, we introduce how to use data source in ML to load data. +Besides some general data sources such as Parquet, CSV, JSON and JDBC, we also provide some specific data sources for ML. + +**Table of Contents** + +* This will become a table of contents (this text will be scraped). +{:toc} + +## Image data source + +This image data source is used to load image files from a directory, it can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` in Java library. +The loaded DataFrame has one `StructType` column: "image", containing image data stored as image schema. +The schema of the `image` column is: + - origin: `StringType` (represents the file path of the image) + - height: `IntegerType` (height of the image) + - width: `IntegerType` (width of the image) + - nChannels: `IntegerType` (number of image channels) + - mode: `IntegerType` (OpenCV-compatible type) + - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) + + +
    +
    +[`ImageDataSource`](api/scala/index.html#org.apache.spark.ml.source.image.ImageDataSource) +implements a Spark SQL data source API for loading image data as a DataFrame. + +{% highlight scala %} +scala> val df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") +df: org.apache.spark.sql.DataFrame = [image: struct] + +scala> df.select("image.origin", "image.width", "image.height").show(truncate=false) ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +{% endhighlight %} +
    + +
    +[`ImageDataSource`](api/java/org/apache/spark/ml/source/image/ImageDataSource.html) +implements Spark SQL data source API for loading image data as DataFrame. + +{% highlight java %} +Dataset imagesDF = spark.read().format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens"); +imageDF.select("image.origin", "image.width", "image.height").show(false); +/* +Will output: ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +*/ +{% endhighlight %} +
    + +
    +In PySpark we provide Spark SQL data source API for loading image data as DataFrame. + +{% highlight python %} +>>> df = spark.read.format("image").option("dropInvalid", true).load("data/mllib/images/origin/kittens") +>>> df.select("image.origin", "image.width", "image.height").show(truncate=False) ++-----------------------------------------------------------------------+-----+------+ +|origin |width|height| ++-----------------------------------------------------------------------+-----+------+ +|file:///spark/data/mllib/images/origin/kittens/54893.jpg |300 |311 | +|file:///spark/data/mllib/images/origin/kittens/DP802813.jpg |199 |313 | +|file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg |300 |200 | +|file:///spark/data/mllib/images/origin/kittens/DP153539.jpg |300 |296 | ++-----------------------------------------------------------------------+-----+------+ +{% endhighlight %} +
    + +
    +In SparkR we provide Spark SQL data source API for loading image data as DataFrame. + +{% highlight r %} +> df = read.df("data/mllib/images/origin/kittens", "image") +> head(select(df, df$image.origin, df$image.width, df$image.height)) + +1 file:///spark/data/mllib/images/origin/kittens/54893.jpg +2 file:///spark/data/mllib/images/origin/kittens/DP802813.jpg +3 file:///spark/data/mllib/images/origin/kittens/29.5.a_b_EGDP022204.jpg +4 file:///spark/data/mllib/images/origin/kittens/DP153539.jpg + width height +1 300 311 +2 199 313 +3 300 200 +4 300 296 + +{% endhighlight %} +
    + + +
    diff --git a/docs/ml-decision-tree.md b/docs/ml-decision-tree.md index 5e1eeb95e4724..43c1fcafd262f 100644 --- a/docs/ml-decision-tree.md +++ b/docs/ml-decision-tree.md @@ -2,6 +2,21 @@ layout: global title: Decision trees displayTitle: Decision trees +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 section has been moved into the diff --git a/docs/ml-ensembles.md b/docs/ml-ensembles.md index 97f1bdc803d01..bab04a6780853 100644 --- a/docs/ml-ensembles.md +++ b/docs/ml-ensembles.md @@ -2,6 +2,21 @@ layout: global title: Tree ensemble methods displayTitle: Tree ensemble methods +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 section has been moved into the diff --git a/docs/ml-features.md b/docs/ml-features.md index 882b895a9d154..2da13576c4ef4 100644 --- a/docs/ml-features.md +++ b/docs/ml-features.md @@ -2,6 +2,21 @@ layout: global title: Extracting, transforming and selecting features displayTitle: Extracting, transforming and selecting features +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 section covers algorithms for working with features, roughly divided into these groups: @@ -359,7 +374,7 @@ Assume that we have the following DataFrame with columns `id` and `raw`: ~~~~ id | raw ----|---------- - 0 | [I, saw, the, red, baloon] + 0 | [I, saw, the, red, balloon] 1 | [Mary, had, a, little, lamb] ~~~~ @@ -369,7 +384,7 @@ column, we should get the following: ~~~~ id | raw | filtered ----|-----------------------------|-------------------- - 0 | [I, saw, the, red, baloon] | [saw, red, baloon] + 0 | [I, saw, the, red, balloon] | [saw, red, balloon] 1 | [Mary, had, a, little, lamb]|[Mary, little, lamb] ~~~~ @@ -585,11 +600,13 @@ for more details on the API. ## StringIndexer `StringIndexer` encodes a string column of labels to a column of label indices. -The indices are in `[0, numLabels)`, and four ordering options are supported: +`StringIndexer` can encode multiple columns. The indices are in `[0, numLabels)`, and four ordering options are supported: "frequencyDesc": descending order by label frequency (most frequent label assigned 0), "frequencyAsc": ascending order by label frequency (least frequent label assigned 0), "alphabetDesc": descending alphabetical order, and "alphabetAsc": ascending alphabetical order -(default = "frequencyDesc"). +(default = "frequencyDesc"). Note that in case of equal frequency when under +"frequencyDesc"/"frequencyAsc", the strings are further sorted by alphabet. + The unseen labels will be put at index numLabels if user chooses to keep them. If the input column is numeric, we cast it to string and index the string values. When downstream pipeline components such as `Estimator` or @@ -779,43 +796,37 @@ for more details on the API. -## OneHotEncoder (Deprecated since 2.3.0) - -Because this existing `OneHotEncoder` is a stateless transformer, it is not usable on new data where the number of categories may differ from the training data. In order to fix this, a new `OneHotEncoderEstimator` was created that produces an `OneHotEncoderModel` when fitting. For more detail, please see [SPARK-13030](https://issues.apache.org/jira/browse/SPARK-13030). - -`OneHotEncoder` has been deprecated in 2.3.0 and will be removed in 3.0.0. Please use [OneHotEncoderEstimator](ml-features.html#onehotencoderestimator) instead. - -## OneHotEncoderEstimator +## OneHotEncoder [One-hot encoding](http://en.wikipedia.org/wiki/One-hot) maps a categorical feature, represented as a label index, to a binary vector with at most a single one-value indicating the presence of a specific feature value from among the set of all feature values. This encoding allows algorithms which expect continuous features, such as Logistic Regression, to use categorical features. For string type input data, it is common to encode categorical features using [StringIndexer](ml-features.html#stringindexer) first. -`OneHotEncoderEstimator` can transform multiple columns, returning an one-hot-encoded output vector column for each input column. It is common to merge these vectors into a single feature vector using [VectorAssembler](ml-features.html#vectorassembler). +`OneHotEncoder` can transform multiple columns, returning an one-hot-encoded output vector column for each input column. It is common to merge these vectors into a single feature vector using [VectorAssembler](ml-features.html#vectorassembler). -`OneHotEncoderEstimator` supports the `handleInvalid` parameter to choose how to handle invalid input during transforming data. Available options include 'keep' (any invalid inputs are assigned to an extra categorical index) and 'error' (throw an error). +`OneHotEncoder` supports the `handleInvalid` parameter to choose how to handle invalid input during transforming data. Available options include 'keep' (any invalid inputs are assigned to an extra categorical index) and 'error' (throw an error). **Examples**
    -Refer to the [OneHotEncoderEstimator Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoderEstimator) for more details on the API. +Refer to the [OneHotEncoder Scala docs](api/scala/index.html#org.apache.spark.ml.feature.OneHotEncoder) for more details on the API. -{% include_example scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala %} +{% include_example scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala %}
    -Refer to the [OneHotEncoderEstimator Java docs](api/java/org/apache/spark/ml/feature/OneHotEncoderEstimator.html) +Refer to the [OneHotEncoder Java docs](api/java/org/apache/spark/ml/feature/OneHotEncoder.html) for more details on the API. -{% include_example java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java %} +{% include_example java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java %}
    -Refer to the [OneHotEncoderEstimator Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.OneHotEncoderEstimator) for more details on the API. +Refer to the [OneHotEncoder Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.OneHotEncoder) for more details on the API. -{% include_example python/ml/onehot_encoder_estimator_example.py %} +{% include_example python/ml/onehot_encoder_example.py %}
    @@ -912,6 +923,14 @@ for more details on the API. {% include_example java/org/apache/spark/examples/ml/JavaInteractionExample.java %} + +
    + +Refer to the [Interaction Python docs](api/python/pyspark.ml.html#pyspark.ml.feature.Interaction) +for more details on the API. + +{% include_example python/ml/interaction_example.py %} +
    ## Normalizer @@ -1308,7 +1327,7 @@ need to know vector size, can use that column as an input. To use `VectorSizeHint` a user must set the `inputCol` and `size` parameters. Applying this transformer to a dataframe produces a new dataframe with updated metadata for `inputCol` specifying the vector size. Downstream operations on the resulting dataframe can get this size using the -meatadata. +metadata. `VectorSizeHint` can also take an optional `handleInvalid` parameter which controls its behaviour when the vector column contains nulls or vectors of the wrong size. By default @@ -1316,7 +1335,7 @@ behaviour when the vector column contains nulls or vectors of the wrong size. By also be set to "skip", indicating that rows containing invalid values should be filtered out from the resulting dataframe, or "optimistic", indicating that the column should not be checked for invalid values and all rows should be kept. Note that the use of "optimistic" can cause the -resulting dataframe to be in an inconsistent state, me:aning the metadata for the column +resulting dataframe to be in an inconsistent state, meaning the metadata for the column `VectorSizeHint` was applied to does not match the contents of that column. Users should take care to avoid this kind of inconsistent state. diff --git a/docs/ml-frequent-pattern-mining.md b/docs/ml-frequent-pattern-mining.md index 81634de8aade7..a243188603997 100644 --- a/docs/ml-frequent-pattern-mining.md +++ b/docs/ml-frequent-pattern-mining.md @@ -2,6 +2,21 @@ layout: global title: Frequent Pattern Mining displayTitle: Frequent Pattern Mining +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. --- Mining frequent items, itemsets, subsequences, or other substructures is usually among the @@ -18,7 +33,7 @@ for more information. ## FP-Growth The FP-growth algorithm is described in the paper -[Han et al., Mining frequent patterns without candidate generation](http://dx.doi.org/10.1145/335191.335372), +[Han et al., Mining frequent patterns without candidate generation](https://doi.org/10.1145/335191.335372), where "FP" stands for frequent pattern. Given a dataset of transactions, the first step of FP-growth is to calculate item frequencies and identify frequent items. Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) algorithms designed for the same purpose, @@ -26,7 +41,7 @@ the second step of FP-growth uses a suffix tree (FP-tree) structure to encode tr explicitly, which are usually expensive to generate. After the second step, the frequent itemsets can be extracted from the FP-tree. In `spark.mllib`, we implemented a parallel version of FP-growth called PFP, -as described in [Li et al., PFP: Parallel FP-growth for query recommendation](http://dx.doi.org/10.1145/1454008.1454027). +as described in [Li et al., PFP: Parallel FP-growth for query recommendation](https://doi.org/10.1145/1454008.1454027). PFP distributes the work of growing FP-trees based on the suffixes of transactions, and hence is more scalable than a single-machine implementation. We refer users to the papers for more details. @@ -85,3 +100,56 @@ Refer to the [R API docs](api/R/spark.fpGrowth.html) for more details. + +## PrefixSpan + +PrefixSpan is a sequential pattern mining algorithm described in +[Pei et al., Mining Sequential Patterns by Pattern-Growth: The +PrefixSpan Approach](https://doi.org/10.1109%2FTKDE.2004.77). We refer +the reader to the referenced paper for formalizing the sequential +pattern mining problem. + +`spark.ml`'s PrefixSpan implementation takes the following parameters: + +* `minSupport`: the minimum support required to be considered a frequent + sequential pattern. +* `maxPatternLength`: the maximum length of a frequent sequential + pattern. Any frequent pattern exceeding this length will not be + included in the results. +* `maxLocalProjDBSize`: the maximum number of items allowed in a + prefix-projected database before local iterative processing of the + projected database begins. This parameter should be tuned with respect + to the size of your executors. +* `sequenceCol`: the name of the sequence column in dataset (default "sequence"), rows with + nulls in this column are ignored. + +**Examples** + +
    + +
    +Refer to the [Scala API docs](api/scala/index.html#org.apache.spark.ml.fpm.PrefixSpan) for more details. + +{% include_example scala/org/apache/spark/examples/ml/PrefixSpanExample.scala %} +
    + +
    +Refer to the [Java API docs](api/java/org/apache/spark/ml/fpm/PrefixSpan.html) for more details. + +{% include_example java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java %} +
    + +
    +Refer to the [Python API docs](api/python/pyspark.ml.html#pyspark.ml.fpm.PrefixSpan) for more details. + +{% include_example python/ml/prefixspan_example.py %} +
    + +
    + +Refer to the [R API docs](api/R/spark.prefixSpan.html) for more details. + +{% include_example r/ml/prefixSpan.R %} +
    + +
    diff --git a/docs/ml-guide.md b/docs/ml-guide.md index aea07be34cb86..4661d6cd87c04 100644 --- a/docs/ml-guide.md +++ b/docs/ml-guide.md @@ -2,6 +2,21 @@ layout: global title: "MLlib: Main Guide" displayTitle: "Machine Learning Library (MLlib) Guide" +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. --- MLlib is Spark's machine learning (ML) library. @@ -104,6 +119,21 @@ MLlib is under active development. The APIs marked `Experimental`/`DeveloperApi` may change in future releases, and the migration guide below will explain all changes between releases. +## From 2.4 to 3.0 + +### Breaking changes + +* `OneHotEncoder` which is deprecated in 2.3, is removed in 3.0 and `OneHotEncoderEstimator` is now renamed to `OneHotEncoder`. + +### Changes of behavior + +* [SPARK-11215](https://issues.apache.org/jira/browse/SPARK-11215): + In Spark 2.4 and previous versions, when specifying `frequencyDesc` or `frequencyAsc` as + `stringOrderType` param in `StringIndexer`, in case of equal frequency, the order of + strings is undefined. Since Spark 3.0, the strings with equal frequency are further + sorted by alphabet. And since Spark 3.0, `StringIndexer` supports encoding multiple + columns. + ## From 2.2 to 2.3 ### Breaking changes diff --git a/docs/ml-linear-methods.md b/docs/ml-linear-methods.md index eb39173505aed..047a6c79ff625 100644 --- a/docs/ml-linear-methods.md +++ b/docs/ml-linear-methods.md @@ -2,6 +2,21 @@ layout: global title: Linear methods displayTitle: Linear methods +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 section has been moved into the diff --git a/docs/ml-migration-guides.md b/docs/ml-migration-guides.md index 2047065f71eb8..99edd9bd69efa 100644 --- a/docs/ml-migration-guides.md +++ b/docs/ml-migration-guides.md @@ -3,6 +3,21 @@ layout: global title: Old Migration Guides - MLlib displayTitle: Old Migration Guides - MLlib description: MLlib migration guides from before Spark SPARK_VERSION_SHORT +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. --- The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). diff --git a/docs/ml-pipeline.md b/docs/ml-pipeline.md index e22e9003c30f6..993a428ab5489 100644 --- a/docs/ml-pipeline.md +++ b/docs/ml-pipeline.md @@ -2,6 +2,21 @@ layout: global title: ML Pipelines displayTitle: ML Pipelines +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. --- `\[ @@ -57,12 +72,12 @@ E.g., a learning algorithm is an `Estimator` which trains on a `DataFrame` and p Machine learning can be applied to a wide variety of data types, such as vectors, text, images, and structured data. This API adopts the `DataFrame` from Spark SQL in order to support a variety of data types. -`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-programming-guide.html#data-types) for a list of supported types. +`DataFrame` supports many basic and structured types; see the [Spark SQL datatype reference](sql-reference.html#data-types) for a list of supported types. In addition to the types listed in the Spark SQL guide, `DataFrame` can use ML [`Vector`](mllib-data-types.html#local-vector) types. A `DataFrame` can be created either implicitly or explicitly from a regular `RDD`. See the code examples below and the [Spark SQL programming guide](sql-programming-guide.html) for examples. -Columns in a `DataFrame` are named. The code examples below use names such as "text," "features," and "label." +Columns in a `DataFrame` are named. The code examples below use names such as "text", "features", and "label". ## Pipeline components diff --git a/docs/ml-statistics.md b/docs/ml-statistics.md index 6c82b3bb94b24..c404b628117de 100644 --- a/docs/ml-statistics.md +++ b/docs/ml-statistics.md @@ -2,6 +2,21 @@ layout: global title: Basic Statistics displayTitle: Basic Statistics +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. --- diff --git a/docs/ml-survival-regression.md b/docs/ml-survival-regression.md index efa3c21c7ca1b..e3bb133482d4f 100644 --- a/docs/ml-survival-regression.md +++ b/docs/ml-survival-regression.md @@ -2,6 +2,21 @@ layout: global title: Survival Regression displayTitle: Survival Regression +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 section has been moved into the diff --git a/docs/ml-tuning.md b/docs/ml-tuning.md index 028bfec465bab..0717cce538bf3 100644 --- a/docs/ml-tuning.md +++ b/docs/ml-tuning.md @@ -2,6 +2,21 @@ layout: global title: "ML Tuning" displayTitle: "ML Tuning: model selection and hyperparameter tuning" +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. --- `\[ diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index a7b90de09369c..d99c7ff053147 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -2,6 +2,21 @@ layout: global title: Classification and Regression - RDD-based API displayTitle: Classification and Regression - RDD-based API +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. --- The `spark.mllib` package supports various methods for diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index dc6b095f5d59b..12c33a5e38049 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -2,6 +2,21 @@ layout: global title: Clustering - RDD-based API displayTitle: Clustering - RDD-based API +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. --- [Clustering](https://en.wikipedia.org/wiki/Cluster_analysis) is an unsupervised learning problem whereby we aim to group subsets @@ -222,7 +237,7 @@ LDA supports different inference algorithms via `setOptimizer` function. on the likelihood function and yields comprehensive results, while `OnlineLDAOptimizer` uses iterative mini-batch sampling for [online variational -inference](https://www.cs.princeton.edu/~blei/papers/HoffmanBleiBach2010b.pdf) +inference](https://mimno.infosci.cornell.edu/info6150/readings/HoffmanBleiBach2010b.pdf) and is generally memory friendly. LDA takes in a collection of documents as vectors of word counts and the diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index b2300028e151b..21546a63263f9 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -2,6 +2,21 @@ layout: global title: Collaborative Filtering - RDD-based API displayTitle: Collaborative Filtering - RDD-based API +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. --- * Table of contents @@ -37,7 +52,7 @@ for example, users giving ratings to movies. It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, clicks, purchases, likes, shares etc.). The approach used in `spark.mllib` to deal with such data is taken -from [Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +from [Collaborative Filtering for Implicit Feedback Datasets](https://doi.org/10.1109/ICDM.2008.22). Essentially, instead of trying to model the matrix of ratings directly, this approach treats the data as numbers representing the *strength* in observations of user actions (such as the number of clicks, or the cumulative duration someone spent viewing a movie). Those numbers are then related to the level of @@ -51,7 +66,7 @@ Since v1.1, we scale the regularization parameter `lambda` in solving each least the number of ratings the user generated in updating user factors, or the number of ratings the product received in updating product factors. This approach is named "ALS-WR" and discussed in the paper -"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](http://dx.doi.org/10.1007/978-3-540-68880-8_32)". +"[Large-Scale Parallel Collaborative Filtering for the Netflix Prize](https://doi.org/10.1007/978-3-540-68880-8_32)". It makes `lambda` less dependent on the scale of the dataset, so we can apply the best parameter learned from a sampled subset to the full dataset and expect similar performance. diff --git a/docs/mllib-data-types.md b/docs/mllib-data-types.md index eca101132d2e5..cdac46284b6be 100644 --- a/docs/mllib-data-types.md +++ b/docs/mllib-data-types.md @@ -2,6 +2,21 @@ layout: global title: Data Types - RDD-based API displayTitle: Data Types - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md index ec13b81f85557..045da744239b9 100644 --- a/docs/mllib-decision-tree.md +++ b/docs/mllib-decision-tree.md @@ -2,6 +2,21 @@ layout: global title: Decision Trees - RDD-based API displayTitle: Decision Trees - RDD-based API +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. --- * Table of contents @@ -149,7 +164,7 @@ These parameters may be tuned. Be careful to validate on held-out test data whe * Note that the `maxBins` parameter must be at least the maximum number of categories `$M$` for any categorical feature. * **`maxMemoryInMB`**: Amount of memory to be used for collecting sufficient statistics. - * The default value is conservatively chosen to be 256 MB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. + * The default value is conservatively chosen to be 256 MiB to allow the decision algorithm to work in most scenarios. Increasing `maxMemoryInMB` can lead to faster training (if the memory is available) by allowing fewer passes over the data. However, there may be decreasing returns as `maxMemoryInMB` grows since the amount of communication on each iteration can be proportional to `maxMemoryInMB`. * *Implementation details*: For faster processing, the decision tree algorithm collects statistics about groups of nodes to split (rather than 1 node at a time). The number of nodes which can be handled in one group is determined by the memory requirements (which vary per features). The `maxMemoryInMB` parameter specifies the memory limit in terms of megabytes which each worker can use for these statistics. * **`subsamplingRate`**: Fraction of the training data used for learning the decision tree. This parameter is most relevant for training ensembles of trees (using [`RandomForest`](api/scala/index.html#org.apache.spark.mllib.tree.RandomForest$) and [`GradientBoostedTrees`](api/scala/index.html#org.apache.spark.mllib.tree.GradientBoostedTrees)), where it can be useful to subsample the original data. For training a single decision tree, this parameter is less useful since the number of training instances is generally not the main constraint. diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md index 4e6b4530942f1..5eb36b4228ca3 100644 --- a/docs/mllib-dimensionality-reduction.md +++ b/docs/mllib-dimensionality-reduction.md @@ -2,6 +2,21 @@ layout: global title: Dimensionality Reduction - RDD-based API displayTitle: Dimensionality Reduction - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-ensembles.md b/docs/mllib-ensembles.md index e1984b6c8d5a5..6149f458214e6 100644 --- a/docs/mllib-ensembles.md +++ b/docs/mllib-ensembles.md @@ -2,6 +2,21 @@ layout: global title: Ensembles - RDD-based API displayTitle: Ensembles - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-evaluation-metrics.md b/docs/mllib-evaluation-metrics.md index c65ecdcb67ee4..f931fa32ea541 100644 --- a/docs/mllib-evaluation-metrics.md +++ b/docs/mllib-evaluation-metrics.md @@ -2,6 +2,21 @@ layout: global title: Evaluation Metrics - RDD-based API displayTitle: Evaluation Metrics - RDD-based API +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. --- * Table of contents @@ -413,13 +428,13 @@ A ranking system usually deals with a set of $M$ users $$U = \left\{u_0, u_1, ..., u_{M-1}\right\}$$ -Each user ($u_i$) having a set of $N$ ground truth relevant documents +Each user ($u_i$) having a set of $N_i$ ground truth relevant documents -$$D_i = \left\{d_0, d_1, ..., d_{N-1}\right\}$$ +$$D_i = \left\{d_0, d_1, ..., d_{N_i-1}\right\}$$ -And a list of $Q$ recommended documents, in order of decreasing relevance +And a list of $Q_i$ recommended documents, in order of decreasing relevance -$$R_i = \left[r_0, r_1, ..., r_{Q-1}\right]$$ +$$R_i = \left[r_0, r_1, ..., r_{Q_i-1}\right]$$ The goal of the ranking system is to produce the most relevant set of documents for each user. The relevance of the sets and the effectiveness of the algorithms can be measured using the metrics listed below. @@ -439,10 +454,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Precision at k - $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} rel_{D_i}(R_i(j))}$ + $p(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{k} \sum_{j=0}^{\text{min}(Q_i, k) - 1} rel_{D_i}(R_i(j))}$ - Precision at k is a measure of + Precision at k is a measure of how many of the first k recommended documents are in the set of true relevant documents averaged across all users. In this metric, the order of the recommendations is not taken into account. @@ -450,10 +465,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Mean Average Precision - $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{\left|D_i\right|} \sum_{j=0}^{Q-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ + $MAP=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{N_i} \sum_{j=0}^{Q_i-1} \frac{rel_{D_i}(R_i(j))}{j + 1}}$ - MAP is a measure of how + MAP is a measure of how many of the recommended documents are in the set of true relevant documents, where the order of the recommendations is taken into account (i.e. penalty for highly relevant documents is higher). @@ -462,10 +477,10 @@ $$rel_D(r) = \begin{cases}1 & \text{if $r \in D$}, \\ 0 & \text{otherwise}.\end{ Normalized Discounted Cumulative Gain $NDCG(k)=\frac{1}{M} \sum_{i=0}^{M-1} {\frac{1}{IDCG(D_i, k)}\sum_{j=0}^{n-1} - \frac{rel_{D_i}(R_i(j))}{\text{ln}(j+2)}} \\ + \frac{rel_{D_i}(R_i(j))}{\text{log}(j+2)}} \\ \text{Where} \\ - \hspace{5 mm} n = \text{min}\left(\text{max}\left(|R_i|,|D_i|\right),k\right) \\ - \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{ln}(j+2)}$ + \hspace{5 mm} n = \text{min}\left(\text{max}\left(Q_i, N_i\right),k\right) \\ + \hspace{5 mm} IDCG(D, k) = \sum_{j=0}^{\text{min}(\left|D\right|, k) - 1} \frac{1}{\text{log}(j+2)}$ NDCG at k is a diff --git a/docs/mllib-feature-extraction.md b/docs/mllib-feature-extraction.md index bb29f65c0322f..b7f8ae9d07b0a 100644 --- a/docs/mllib-feature-extraction.md +++ b/docs/mllib-feature-extraction.md @@ -2,6 +2,21 @@ layout: global title: Feature Extraction and Transformation - RDD-based API displayTitle: Feature Extraction and Transformation - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-frequent-pattern-mining.md b/docs/mllib-frequent-pattern-mining.md index 0d3192c6b1d9c..8bc93ac2e8adf 100644 --- a/docs/mllib-frequent-pattern-mining.md +++ b/docs/mllib-frequent-pattern-mining.md @@ -2,6 +2,21 @@ layout: global title: Frequent Pattern Mining - RDD-based API displayTitle: Frequent Pattern Mining - RDD-based API +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. --- Mining frequent items, itemsets, subsequences, or other substructures is usually among the @@ -15,7 +30,7 @@ a popular algorithm to mining frequent itemsets. ## FP-growth The FP-growth algorithm is described in the paper -[Han et al., Mining frequent patterns without candidate generation](http://dx.doi.org/10.1145/335191.335372), +[Han et al., Mining frequent patterns without candidate generation](https://doi.org/10.1145/335191.335372), where "FP" stands for frequent pattern. Given a dataset of transactions, the first step of FP-growth is to calculate item frequencies and identify frequent items. Different from [Apriori-like](http://en.wikipedia.org/wiki/Apriori_algorithm) algorithms designed for the same purpose, @@ -23,7 +38,7 @@ the second step of FP-growth uses a suffix tree (FP-tree) structure to encode tr explicitly, which are usually expensive to generate. After the second step, the frequent itemsets can be extracted from the FP-tree. In `spark.mllib`, we implemented a parallel version of FP-growth called PFP, -as described in [Li et al., PFP: Parallel FP-growth for query recommendation](http://dx.doi.org/10.1145/1454008.1454027). +as described in [Li et al., PFP: Parallel FP-growth for query recommendation](https://doi.org/10.1145/1454008.1454027). PFP distributes the work of growing FP-trees based on the suffixes of transactions, and hence more scalable than a single-machine implementation. We refer users to the papers for more details. @@ -122,7 +137,7 @@ Refer to the [`AssociationRules` Java docs](api/java/org/apache/spark/mllib/fpm/ PrefixSpan is a sequential pattern mining algorithm described in [Pei et al., Mining Sequential Patterns by Pattern-Growth: The -PrefixSpan Approach](http://dx.doi.org/10.1109%2FTKDE.2004.77). We refer +PrefixSpan Approach](https://doi.org/10.1109%2FTKDE.2004.77). We refer the reader to the referenced paper for formalizing the sequential pattern mining problem. diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 30112c72c9c31..dbb74407d030d 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -2,6 +2,21 @@ layout: global title: "MLlib: RDD-based API" displayTitle: "MLlib: RDD-based API" +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 documents sections of the MLlib guide for the RDD-based API (the `spark.mllib` package). diff --git a/docs/mllib-isotonic-regression.md b/docs/mllib-isotonic-regression.md index 99cab98c690c6..d9cc775547bb1 100644 --- a/docs/mllib-isotonic-regression.md +++ b/docs/mllib-isotonic-regression.md @@ -2,6 +2,21 @@ layout: global title: Isotonic regression - RDD-based API displayTitle: Regression - RDD-based API +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. --- ## Isotonic regression @@ -24,9 +39,9 @@ Essentially isotonic regression is a best fitting the original data points. `spark.mllib` supports a -[pool adjacent violators algorithm](http://doi.org/10.1198/TECH.2010.10111) +[pool adjacent violators algorithm](https://doi.org/10.1198/TECH.2010.10111) which uses an approach to -[parallelizing isotonic regression](http://doi.org/10.1007/978-3-642-99789-1_10). +[parallelizing isotonic regression](https://doi.org/10.1007/978-3-642-99789-1_10). The training input is an RDD of tuples of three double values that represent label, feature and weight in this order. Additionally, IsotonicRegression algorithm has one optional parameter called $isotonic$ defaulting to true. diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md index 73f6e206ca543..2d3ec4ca24443 100644 --- a/docs/mllib-linear-methods.md +++ b/docs/mllib-linear-methods.md @@ -2,6 +2,21 @@ layout: global title: Linear Methods - RDD-based API displayTitle: Linear Methods - RDD-based API +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. --- * Table of contents @@ -272,7 +287,7 @@ In `spark.mllib`, the first class $0$ is chosen as the "pivot" class. See Section 4.4 of [The Elements of Statistical Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for references. -Here is an +Here is a [detailed mathematical derivation](http://www.slideshare.net/dbtsai/2014-0620-mlor-36132297). For multiclass classification problems, the algorithm will output a multinomial logistic regression @@ -350,7 +365,7 @@ known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_erro
    -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The following example demonstrates how to load training data, parse it as an RDD of LabeledPoint. The example then uses LinearRegressionWithSGD to build a simple linear model to predict label values. We compute the mean squared error at the end to evaluate [goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). diff --git a/docs/mllib-migration-guides.md b/docs/mllib-migration-guides.md index ea6f93fcf67f3..b746b96e19f07 100644 --- a/docs/mllib-migration-guides.md +++ b/docs/mllib-migration-guides.md @@ -2,6 +2,21 @@ layout: global title: Old Migration Guides - MLlib displayTitle: Old Migration Guides - MLlib +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. --- The migration guide for the current Spark version is kept on the [MLlib Guide main page](ml-guide.html#migration-guide). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md index 7471d18a0dddc..09b15876a3914 100644 --- a/docs/mllib-naive-bayes.md +++ b/docs/mllib-naive-bayes.md @@ -2,6 +2,21 @@ layout: global title: Naive Bayes - RDD-based API displayTitle: Naive Bayes - RDD-based API +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. --- [Naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier) is a simple diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 04758903da89c..f2e128ec215a1 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -2,6 +2,21 @@ layout: global title: Optimization - RDD-based API displayTitle: Optimization - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-pmml-model-export.md b/docs/mllib-pmml-model-export.md index f567565437927..fbc14cefd14ec 100644 --- a/docs/mllib-pmml-model-export.md +++ b/docs/mllib-pmml-model-export.md @@ -2,6 +2,21 @@ layout: global title: PMML model export - RDD-based API displayTitle: PMML model export - RDD-based API +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. --- * Table of contents diff --git a/docs/mllib-statistics.md b/docs/mllib-statistics.md index c29400af85055..4698d3e6347e7 100644 --- a/docs/mllib-statistics.md +++ b/docs/mllib-statistics.md @@ -2,6 +2,21 @@ layout: global title: Basic Statistics - RDD-based API displayTitle: Basic Statistics - RDD-based API +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. --- * Table of contents @@ -239,7 +254,7 @@ Refer to the [`Statistics` Python docs](api/python/pyspark.mllib.html#pyspark.ml ### Streaming Significance Testing `spark.mllib` provides online implementations of some tests to support use cases like A/B testing. These tests may be performed on a Spark Streaming -`DStream[(Boolean,Double)]` where the first element of each tuple +`DStream[(Boolean, Double)]` where the first element of each tuple indicates control group (`false`) or treatment group (`true`) and the second element is the value of an observation. diff --git a/docs/monitoring.md b/docs/monitoring.md index 69bf3082f0f27..0f7210c3b8bb2 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -2,6 +2,21 @@ layout: global title: Monitoring and Instrumentation description: Monitoring, metrics, and instrumentation guide for Spark SPARK_VERSION_SHORT +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. --- There are several ways to monitor Spark applications: web UIs, metrics, and external instrumentation. @@ -175,7 +190,11 @@ Security options for the Spark History Server are covered more detail in the 1d How often the filesystem job history cleaner checks for files to delete. - Files are only deleted if they are older than spark.history.fs.cleaner.maxAge + Files are deleted if at least one of two conditions holds. + First, they're deleted if they're older than spark.history.fs.cleaner.maxAge. + They are also deleted if the number of files is more than + spark.history.fs.cleaner.maxNum, Spark tries to clean up the completed attempts + from the applications based on the order of their oldest attempt time. @@ -185,6 +204,16 @@ Security options for the Spark History Server are covered more detail in the Job history files older than this will be deleted when the filesystem history cleaner runs. + + spark.history.fs.cleaner.maxNum + Int.MaxValue + + The maximum number of files in the event log directory. + Spark tries to clean up the completed attempt logs to maintain the log directory under this limit. + This should be smaller than the underlying file system limit like + `dfs.namenode.fs-limits.max-directory-items` in HDFS. + + spark.history.fs.endEventReparseChunkSize 1m @@ -202,6 +231,28 @@ Security options for the Spark History Server are covered more detail in the applications that fail to rename their event logs listed as in-progress. + + spark.history.fs.driverlog.cleaner.enabled + spark.history.fs.cleaner.enabled + + Specifies whether the History Server should periodically clean up driver logs from storage. + + + + spark.history.fs.driverlog.cleaner.interval + spark.history.fs.cleaner.interval + + How often the filesystem driver log cleaner checks for files to delete. + Files are only deleted if they are older than spark.history.fs.driverlog.cleaner.maxAge + + + + spark.history.fs.driverlog.cleaner.maxAge + spark.history.fs.cleaner.maxAge + + Driver log files older than this will be deleted when the driver log cleaner runs. + + spark.history.fs.numReplayThreads 25% of available cores @@ -226,6 +277,30 @@ Security options for the Spark History Server are covered more detail in the written to disk will be re-used in the event of a history server restart. + + spark.history.custom.executor.log.url + (none) + + Specifies custom spark executor log URL for supporting external log service instead of using cluster + managers' application log URLs in the history server. Spark will support some path variables via patterns + which can vary on cluster manager. Please check the documentation for your cluster manager to + see which patterns are supported, if any. This configuration has no effect on a live application, it only + affects the history server. +

    + For now, only YARN mode supports this configuration + + + + spark.history.custom.executor.log.url.applyIncompleteApplication + false + + Specifies whether to apply custom spark executor log URL to incomplete applications as well. + If executor logs for running applications should be provided as origin log URLs, set this to `false`. + Please note that incomplete applications may include applications which didn't shutdown gracefully. + Even this is set to `true`, this configuration has no effect on a live application, it only affects the history server. + + + Note that in all of these UIs, the tables are sortable by clicking their headers, @@ -563,7 +638,150 @@ 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 infomation. Metrics `peakExecutorMetrics.*` are only enabled if `spark.eventLog.logStageExecutorMetrics.enabled` is true. +A list of the available metrics, with a short description: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Executor Level Metric nameShort description
    totalGCTimeElapsed time the JVM spent in garbage collection summed in this Executor. + The value is expressed in milliseconds.
    totalInputBytesTotal input bytes summed in this Executor.
    totalShuffleReadTotal shuffer read bytes summed in this Executor.
    totalShuffleWriteTotal shuffer write bytes summed in this Executor.
    maxMemoryTotal amount of memory available for storage, in bytes.
    memoryMetrics.*Current value of memory metrics:
        .usedOnHeapStorageMemoryUsed on heap memory currently for storage, in bytes.
        .usedOffHeapStorageMemoryUsed off heap memory currently for storage, in bytes.
        .totalOnHeapStorageMemoryTotal available on heap memory for storage, in bytes. This amount can vary over time, on the MemoryManager implementation.
        .totalOffHeapStorageMemoryTotal available off heap memory for storage, in bytes. This amount can vary over time, depending on the MemoryManager implementation.
    peakMemoryMetrics.*Peak value of memory (and GC) metrics:
        .JVMHeapMemoryPeak memory usage of the heap that is used for object allocation. + The heap consists of one or more memory pools. The used and committed size of the returned memory usage is the sum of those values of all heap memory pools whereas the init and max size of the returned memory usage represents the setting of the heap memory which may not be the sum of those of all heap memory pools. + The amount of used memory in the returned memory usage is the amount of memory occupied by both live objects and garbage objects that have not been collected, if any.
        .JVMOffHeapMemoryPeak memory usage of non-heap memory that is used by the Java virtual machine. The non-heap memory consists of one or more memory pools. The used and committed size of the returned memory usage is the sum of those values of all non-heap memory pools whereas the init and max size of the returned memory usage represents the setting of the non-heap memory which may not be the sum of those of all non-heap memory pools.
        .OnHeapExecutionMemoryPeak on heap execution memory in use, in bytes.
        .OffHeapExecutionMemoryPeak off heap execution memory in use, in bytes.
        .OnHeapStorageMemoryPeak on heap storage memory in use, in bytes.
        .OffHeapStorageMemoryPeak off heap storage memory in use, in bytes.
        .OnHeapUnifiedMemoryPeak on heap memory (execution and storage).
        .OffHeapUnifiedMemoryPeak off heap memory (execution and storage).
        .DirectPoolMemoryPeak memory that the JVM is using for direct buffer pool ([[java.lang.management.BufferPoolMXBean]])
        .MappedPoolMemoryPeak memory that the JVM is using for mapped buffer pool ([[java.lang.management.BufferPoolMXBean]])
        .ProcessTreeJVMVMemoryVirtual memory size in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .ProcessTreeJVMRSSMemoryResident Set Size: number of pages the process has + in real memory. This is just the pages which count + toward text, data, or stack space. This does not + include pages which have not been demand-loaded in, + or which are swapped out. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .ProcessTreePythonVMemoryVirtual memory size for Python in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .ProcessTreePythonRSSMemoryResident Set Size for Python. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .ProcessTreeOtherVMemoryVirtual memory size for other kind of process in bytes. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .ProcessTreeOtherRSSMemoryResident Set Size for other kind of process. Enabled if spark.eventLog.logStageExecutorProcessTreeMetrics.enabled is true.
        .MinorGCCountTotal minor GC count. For example, the garbage collector is one of Copy, PS Scavenge, ParNew, G1 Young Generation and so on.
        .MinorGCTimeElapsed total minor GC time. + The value is expressed in milliseconds.
        .MajorGCCountTotal major GC count. For example, the garbage collector is one of MarkSweepCompact, PS MarkSweep, ConcurrentMarkSweep, G1 Old Generation and so on.
        .MajorGCTimeElapsed total major GC time. + The value is expressed in milliseconds.
    +The computation of RSS and Vmem are based on [proc(5)](http://man7.org/linux/man-pages/man5/proc.5.html) ### API Versioning Policy @@ -587,9 +805,13 @@ keep the paths consistent in both modes. Spark has a configurable metrics system based on the [Dropwizard Metrics Library](http://metrics.dropwizard.io/). This allows users to report Spark metrics to a variety of sinks including HTTP, JMX, and CSV -files. The metrics system is configured via a configuration file that Spark expects to be present +files. The metrics are generated by sources embedded in the Spark code base. They +provide instrumentation for specific activities and Spark components. +The metrics system is configured via a configuration file that Spark expects to be present at `$SPARK_HOME/conf/metrics.properties`. A custom file location can be specified via the `spark.metrics.conf` [configuration property](configuration.html#spark-properties). +Instead of using the configuration file, a set of configuration parameters with prefix +`spark.metrics.conf.` can be used. By default, the root namespace used for driver or executor metrics is the value of `spark.app.id`. However, often times, users want to be able to track the metrics across apps for driver and executors, which is hard to do with application ID @@ -613,6 +835,7 @@ set of sinks to which metrics are reported. The following instances are currentl * `driver`: The Spark driver process (the process in which your SparkContext is created). * `shuffleService`: The Spark shuffle service. * `applicationMaster`: The Spark ApplicationMaster when running on YARN. +* `mesos_cluster`: The Spark cluster scheduler when running on Mesos. Each instance can report to zero or more _sinks_. Sinks are contained in the `org.apache.spark.metrics.sink` package: @@ -637,9 +860,264 @@ code in your Spark package**_. For sbt users, set the the `-Pspark-ganglia-lgpl` profile. In addition to modifying the cluster's Spark build user applications will need to link to the `spark-ganglia-lgpl` artifact. -The syntax of the metrics configuration file is defined in an example configuration file, +The syntax of the metrics configuration file and the parameters available for each sink are defined +in an example configuration file, `$SPARK_HOME/conf/metrics.properties.template`. +When using Spark configuration parameters instead of the metrics configuration file, the relevant +parameter names are composed by the prefix `spark.metrics.conf.` followed by the configuration +details, i.e. the parameters take the following form: +`spark.metrics.conf.[instance|*].sink.[sink_name].[parameter_name]`. +This example shows a list of Spark configuration parameters for a Graphite sink: +``` +"spark.metrics.conf.*.sink.graphite.class"="org.apache.spark.metrics.sink.GraphiteSink" +"spark.metrics.conf.*.sink.graphite.host"="graphiteEndPoint_hostName>" +"spark.metrics.conf.*.sink.graphite.port"= +"spark.metrics.conf.*.sink.graphite.period"=10 +"spark.metrics.conf.*.sink.graphite.unit"=seconds +"spark.metrics.conf.*.sink.graphite.prefix"="optional_prefix" +``` + +Default values of the Spark metrics configuration are as follows: +``` +"*.sink.servlet.class" = "org.apache.spark.metrics.sink.MetricsServlet" +"*.sink.servlet.path" = "/metrics/json" +"master.sink.servlet.path" = "/metrics/master/json" +"applications.sink.servlet.path" = "/metrics/applications/json" +``` + +Additional sources can be configured using the metrics configuration file or the configuration +parameter `spark.metrics.conf.[component_name].source.jvm.class=[source_name]`. At present the +JVM source is the only available optional source. For example the following configuration parameter +activates the JVM source: +`"spark.metrics.conf.*.source.jvm.class"="org.apache.spark.metrics.source.JvmSource"` + +## List of available metrics providers + +Metrics used by Spark are of multiple types: gauge, counter, histogram, meter and timer, +see [Dropwizard library documentation for details](https://metrics.dropwizard.io/3.1.0/getting-started/). +The following list of components and metrics reports the name and some details about the available metrics, +grouped per component instance and source namespace. +The most common time of metrics used in Spark instrumentation are gauges and counters. +Counters can be recognized as they have the `.count` suffix. Timers, meters and histograms are annotated +in the list, the rest of the list elements are metrics of type gauge. +The large majority of metrics are active as soon as their parent component instance is configured, +some metrics require also to be enabled via an additional configuration parameter, the details are +reported in the list. + +### Component instance = Driver +This is the component with the largest amount of instrumented metrics + +- namespace=BlockManager + - disk.diskSpaceUsed_MB + - memory.maxMem_MB + - memory.maxOffHeapMem_MB + - memory.maxOnHeapMem_MB + - memory.memUsed_MB + - memory.offHeapMemUsed_MB + - memory.onHeapMemUsed_MB + - memory.remainingMem_MB + - memory.remainingOffHeapMem_MB + - memory.remainingOnHeapMem_MB + +- namespace=HiveExternalCatalog + - fileCacheHits.count + - filesDiscovered.count + - hiveClientCalls.count + - parallelListingJobCount.count + - partitionsFetched.count + +- namespace=CodeGenerator + - compilationTime (histogram) + - generatedClassSize (histogram) + - generatedMethodSize (histogram) + - hiveClientCalls.count + - sourceCodeSize (histogram) + +- namespace=DAGScheduler + - job.activeJobs + - job.allJobs + - messageProcessingTime (timer) + - stage.failedStages + - stage.runningStages + - stage.waitingStages + +- namespace=LiveListenerBus + - listenerProcessingTime.org.apache.spark.HeartbeatReceiver (timer) + - listenerProcessingTime.org.apache.spark.scheduler.EventLoggingListener (timer) + - listenerProcessingTime.org.apache.spark.status.AppStatusListener (timer) + - numEventsPosted.count + - queue.appStatus.listenerProcessingTime (timer) + - queue.appStatus.numDroppedEvents.count + - queue.appStatus.size + - queue.eventLog.listenerProcessingTime (timer) + - queue.eventLog.numDroppedEvents.count + - queue.eventLog.size + - queue.executorManagement.listenerProcessingTime (timer) + +- namespace=appStatus (all metrics of type=counter) + - **note:** Introduced in Spark 3.0. Conditional to configuration parameter: + `spark.app.status.metrics.enabled=true` (default is false) + - stages.failedStages.count + - stages.skippedStages.count + - stages.completedStages.count + - tasks.blackListedExecutors.count + - tasks.completedTasks.count + - tasks.failedTasks.count + - tasks.killedTasks.count + - tasks.skippedTasks.count + - tasks.unblackListedExecutors.count + - jobs.succeededJobs + - jobs.failedJobs + - jobDuration + +- namespace=AccumulatorSource + - **note:** User-configurable sources to attach accumulators to metric system + - DoubleAccumulatorSource + - LongAccumulatorSource + +- namespace=spark.streaming + - **note:** This applies to Spark Structured Streaming only. Conditional to a configuration + parameter: `spark.sql.streaming.metricsEnabled=true` (default is false) + - eventTime-watermark + - inputRate-total + - latency + - processingRate-total + - states-rowsTotal + - states-usedBytes + +- namespace=JVMCPU + - jvmCpuTime + +### Component instance = Executor +These metrics are exposed by Spark executors. Note, currently they are not available +when running in local mode. + +- namespace=executor (metrics are of type counter or gauge) + - bytesRead.count + - bytesWritten.count + - cpuTime.count + - deserializeCpuTime.count + - deserializeTime.count + - diskBytesSpilled.count + - filesystem.file.largeRead_ops + - filesystem.file.read_bytes + - filesystem.file.read_ops + - filesystem.file.write_bytes + - filesystem.file.write_ops + - filesystem.hdfs.largeRead_ops + - filesystem.hdfs.read_bytes + - filesystem.hdfs.read_ops + - filesystem.hdfs.write_bytes + - filesystem.hdfs.write_ops + - jvmGCTime.count + - memoryBytesSpilled.count + - recordsRead.count + - recordsWritten.count + - resultSerializationTime.count + - resultSize.count + - runTime.count + - shuffleBytesWritten.count + - shuffleFetchWaitTime.count + - shuffleLocalBlocksFetched.count + - shuffleLocalBytesRead.count + - shuffleRecordsRead.count + - shuffleRecordsWritten.count + - shuffleRemoteBlocksFetched.count + - shuffleRemoteBytesRead.count + - shuffleRemoteBytesReadToDisk.count + - shuffleTotalBytesRead.count + - shuffleWriteTime.count + - threadpool.activeTasks + - threadpool.completeTasks + - threadpool.currentPool_size + - threadpool.maxPool_size + +- namespace=JVMCPU + - jvmCpuTime + +- namespace=NettyBlockTransfer + - shuffle-client.usedDirectMemory + - shuffle-client.usedHeapMemory + - shuffle-server.usedDirectMemory + - shuffle-server.usedHeapMemory + +- namespace=HiveExternalCatalog + - fileCacheHits.count + - filesDiscovered.count + - hiveClientCalls.count + - parallelListingJobCount.count + - partitionsFetched.count + +- namespace=CodeGenerator + - compilationTime (histogram) + - generatedClassSize (histogram) + - generatedMethodSize (histogram) + - hiveClientCalls.count + - sourceCodeSize (histogram) + +### Source = JVM Source +Notes: + - Activate this source by setting the relevant `metrics.properties` file entry or the + configuration parameter:`spark.metrics.conf.*.source.jvm.class=org.apache.spark.metrics.source.JvmSource` + - This source is available for driver and executor instances and is also available for other instances. + - This source provides information on JVM metrics using the + [Dropwizard/Codahale Metric Sets for JVM instrumentation](https://metrics.dropwizard.io/3.1.0/manual/jvm/) + and in particular the metric sets BufferPoolMetricSet, GarbageCollectorMetricSet and MemoryUsageGaugeSet. + +### Component instance = applicationMaster +Note: applies when running on YARN + +- numContainersPendingAllocate +- numExecutorsFailed +- numExecutorsRunning +- numLocalityAwareTasks +- numReleasedContainers + +### Component instance = mesos_cluster +Note: applies when running on mesos + +- waitingDrivers +- launchedDrivers +- retryDrivers + +### Component instance = master +Note: applies when running in Spark standalone as master + +- workers +- aliveWorkers +- apps +- waitingApps + +### Component instance = ApplicationSource +Note: applies when running in Spark standalone as master + +- status +- runtime_ms +- cores + +### Component instance = worker +Note: applies when running in Spark standalone as worker + +- executors +- coresUsed +- memUsed_MB +- coresFree +- memFree_MB + +### Component instance = shuffleService +Note: applies to the shuffle service + +- blockTransferRateBytes (meter) +- numActiveConnections.count +- numRegisteredConnections.count +- numCaughtExceptions.count +- openBlockRequestLatencyMillis (histogram) +- registerExecutorRequestLatencyMillis (histogram) +- registeredExecutorsSize +- shuffle-server.usedDirectMemory +- shuffle-server.usedHeapMemory + # Advanced Instrumentation Several external tools can be used to help profile the performance of Spark jobs: diff --git a/docs/programming-guide.md b/docs/programming-guide.md index f8b8f74c53b55..986bea3f7165b 100644 --- a/docs/programming-guide.md +++ b/docs/programming-guide.md @@ -2,6 +2,21 @@ layout: global title: Spark Programming Guide redirect: rdd-programming-guide.html +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 document has moved [here](rdd-programming-guide.html). diff --git a/docs/quick-start.md b/docs/quick-start.md index ef7af6c3f6cec..93abb25f20f47 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -2,6 +2,21 @@ layout: global title: Quick Start description: Quick start tutorial for Spark SPARK_VERSION_SHORT +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 will become a table of contents (this text will be scraped). @@ -17,6 +32,11 @@ you can download a package for any version of Hadoop. Note that, before Spark 2.0, the main programming interface of Spark was the Resilient Distributed Dataset (RDD). After Spark 2.0, RDDs are replaced by Dataset, which is strongly-typed like an RDD, but with richer optimizations under the hood. The RDD interface is still supported, and you can get a more detailed reference at the [RDD programming guide](rdd-programming-guide.html). However, we highly recommend you to switch to use Dataset, which has better performance than RDD. See the [SQL programming guide](sql-programming-guide.html) to get more information about Dataset. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) before running Spark. + # Interactive Analysis with the Spark Shell ## Basics @@ -336,6 +356,7 @@ Note that Spark artifacts are tagged with a Scala version. org.apache.spark spark-sql_{{site.SCALA_BINARY_VERSION}} {{site.SPARK_VERSION}} + provided diff --git a/docs/rdd-programming-guide.md b/docs/rdd-programming-guide.md index 9a07d6ca24b65..5e55c93c4148e 100644 --- a/docs/rdd-programming-guide.md +++ b/docs/rdd-programming-guide.md @@ -2,6 +2,21 @@ layout: global title: RDD Programming Guide description: Spark SPARK_VERSION_SHORT programming guide in Java, Scala and Python +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 will become a table of contents (this text will be scraped). @@ -89,7 +104,7 @@ import org.apache.spark.SparkConf; Spark {{site.SPARK_VERSION}} works with Python 2.7+ or Python 3.4+. It can use the standard CPython interpreter, so C libraries like NumPy can be used. It also works with PyPy 2.3+. -Python 2.6 support was removed in Spark 2.2.0. +Note that Python 2 support is deprecated as of Spark 3.0.0. Spark applications in Python can either be run with the `bin/spark-submit` script which includes Spark at runtime, or by including it in your setup.py as: @@ -138,7 +153,7 @@ The first thing a Spark program must do is to create a [SparkContext](api/scala/ how to access a cluster. To create a `SparkContext` you first need to build a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object that contains information about your application. -Only one SparkContext may be active per JVM. You must `stop()` the active SparkContext before creating a new one. +Only one SparkContext should be active per JVM. You must `stop()` the active SparkContext before creating a new one. {% highlight scala %} val conf = new SparkConf().setAppName(appName).setMaster(master) @@ -330,9 +345,9 @@ One important parameter for parallel collections is the number of *partitions* t

    -Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). +Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight scala %} scala> val distFile = sc.textFile("data.txt") @@ -353,7 +368,7 @@ Apart from text files, Spark's Scala API also supports several other data format * `SparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. Partitioning is determined by data locality which, in some cases, may result in too few partitions. For those cases, `wholeTextFiles` provides an optional second argument for controlling the minimal number of partitions. -* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts. +* For [SequenceFiles](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/IntWritable.html) and [Text](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/Text.html). In addition, Spark allows you to specify native types for a few common Writables; for example, `sequenceFile[Int, String]` will automatically read IntWritables and Texts. * For other Hadoop InputFormats, you can use the `SparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `SparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). @@ -363,9 +378,9 @@ Apart from text files, Spark's Scala API also supports several other data format
    -Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). +Spark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight java %} JavaRDD distFile = sc.textFile("data.txt"); @@ -385,7 +400,7 @@ Apart from text files, Spark's Java API also supports several other data formats * `JavaSparkContext.wholeTextFiles` lets you read a directory containing multiple small text files, and returns each of them as (filename, content) pairs. This is in contrast with `textFile`, which would return one record per line in each file. -* For [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/IntWritable.html) and [Text](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/io/Text.html). +* For [SequenceFiles](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), use SparkContext's `sequenceFile[K, V]` method where `K` and `V` are the types of key and values in the file. These should be subclasses of Hadoop's [Writable](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/Writable.html) interface, like [IntWritable](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/IntWritable.html) and [Text](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/io/Text.html). * For other Hadoop InputFormats, you can use the `JavaSparkContext.hadoopRDD` method, which takes an arbitrary `JobConf` and input format class, key class and value class. Set these the same way you would for a Hadoop job with your input source. You can also use `JavaSparkContext.newAPIHadoopRDD` for InputFormats based on the "new" MapReduce API (`org.apache.hadoop.mapreduce`). @@ -395,9 +410,9 @@ Apart from text files, Spark's Java API also supports several other data formats
    -PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](http://hadoop.apache.org/common/docs/current/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). +PySpark can create distributed datasets from any storage source supported by Hadoop, including your local file system, HDFS, Cassandra, HBase, [Amazon S3](http://wiki.apache.org/hadoop/AmazonS3), etc. Spark supports text files, [SequenceFiles](https://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/SequenceFileInputFormat.html), and any other Hadoop [InputFormat](http://hadoop.apache.org/docs/stable/api/org/apache/hadoop/mapred/InputFormat.html). -Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes an URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: +Text file RDDs can be created using `SparkContext`'s `textFile` method. This method takes a URI for the file (either a local path on the machine, or a `hdfs://`, `s3a://`, etc URI) and reads it as a collection of lines. Here is an example invocation: {% highlight python %} >>> distFile = sc.textFile("data.txt") @@ -1122,7 +1137,7 @@ costly operation. #### Background -To understand what happens during the shuffle we can consider the example of the +To understand what happens during the shuffle, we can consider the example of the [`reduceByKey`](#ReduceByLink) operation. The `reduceByKey` operation generates a new RDD where all values for a single key are combined into a tuple - the key and the result of executing a reduce function against all values associated with that key. The challenge is that not all values for a @@ -1270,7 +1285,8 @@ waiting to recompute a lost partition. Spark automatically monitors cache usage on each node and drops out old data partitions in a least-recently-used (LRU) fashion. If you would like to manually remove an RDD instead of waiting for -it to fall out of the cache, use the `RDD.unpersist()` method. +it to fall out of the cache, use the `RDD.unpersist()` method. Note that this method does not +block by default. To block until resources are freed, specify `blocking=true` when calling this method. # Shared Variables @@ -1342,6 +1358,12 @@ run on the cluster so that `v` is not shipped to the nodes more than once. In ad `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later). +To release the resources that the broadcast variable copied onto executors, call `.unpersist()`. +If the broadcast is used again afterwards, it will be re-broadcast. To permanently release all +resources used by the broadcast variable, call `.destroy()`. The broadcast variable can't be used +after that. Note that these methods do not block by default. To block until resources are freed, +specify `blocking=true` when calling them. + ## Accumulators Accumulators are variables that are only "added" to through an associative and commutative operation and can diff --git a/docs/running-on-kubernetes.md b/docs/running-on-kubernetes.md index b4088d79addff..d4efb52e0fbba 100644 --- a/docs/running-on-kubernetes.md +++ b/docs/running-on-kubernetes.md @@ -1,6 +1,21 @@ --- layout: global title: Running Spark on Kubernetes +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 will become a table of contents (this text will be scraped). {:toc} @@ -12,6 +27,23 @@ Kubernetes scheduler that has been added to Spark. In future versions, there may be behavioral changes around configuration, container images and entrypoints.** +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific advice below before running Spark. + +## User Identity + +Images built from the project provided Dockerfiles contain a default [`USER`](https://docs.docker.com/engine/reference/builder/#user) directive with a default UID of `185`. This means that the resulting images will be running the Spark processes as this UID inside the container. Security conscious deployments should consider providing custom images with `USER` directives specifying their desired unprivileged UID and GID. The resulting UID should include the root group in its supplementary groups in order to be able to run the Spark executables. Users building their own images with the provided `docker-image-tool.sh` script can use the `-u ` option to specify the desired UID. + +Alternatively the [Pod Template](#pod-template) feature can be used to add a [Security Context](https://kubernetes.io/docs/tasks/configure-pod-container/security-context/#volumes-and-file-systems) with a `runAsUser` to the pods that Spark submits. This can be used to override the `USER` directives in the images themselves. Please bear in mind that this requires cooperation from your users and as such may not be a suitable solution for shared environments. Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/#users-and-groups) if they wish to limit the users that pods may run as. + +## Volume Mounts + +As described later in this document under [Using Kubernetes Volumes](#using-kubernetes-volumes) Spark on K8S provides configuration options that allow for mounting certain volume types into the driver and executor pods. In particular it allows for [`hostPath`](https://kubernetes.io/docs/concepts/storage/volumes/#hostpath) volumes which as described in the Kubernetes documentation have known security vulnerabilities. + +Cluster administrators should use [Pod Security Policies](https://kubernetes.io/docs/concepts/policy/pod-security-policy/) to limit the ability to mount `hostPath` volumes appropriately for their environments. + # Prerequisites * A runnable distribution of Spark 2.3 or above. @@ -45,7 +77,8 @@ logs and remains in "completed" state in the Kubernetes API until it's eventuall Note that in the completed state, the driver pod does *not* use any computational or memory resources. -The driver and executor pod scheduling is handled by Kubernetes. It is possible to schedule the +The driver and executor pod scheduling is handled by Kubernetes. Communication to the Kubernetes API is done via fabric8, and we are +currently running kubernetes-client version 4.1.0. Make sure that when you are making infrastructure additions that you are aware of said version. It is possible to schedule the driver and executor pods on a subset of available nodes through a [node selector](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#nodeselector) using the configuration property for it. It will be possible to use more advanced scheduling hints like [node/pod affinities](https://kubernetes.io/docs/concepts/configuration/assign-pod-node/#affinity-and-anti-affinity) in a future release. @@ -69,6 +102,19 @@ Example usage is: $ ./bin/docker-image-tool.sh -r -t my-tag build $ ./bin/docker-image-tool.sh -r -t my-tag push ``` +This will build using the projects provided default `Dockerfiles`. To see more options available for customising the behaviour of this tool, including providing custom `Dockerfiles`, please run with the `-h` flag. + +By default `bin/docker-image-tool.sh` builds docker image for running JVM jobs. You need to opt-in to build additional +language binding docker images. + +Example usage is +```bash +# To build additional PySpark docker image +$ ./bin/docker-image-tool.sh -r -t my-tag -p ./kubernetes/dockerfiles/spark/bindings/python/Dockerfile build + +# To build additional SparkR docker image +$ ./bin/docker-image-tool.sh -r -t my-tag -R ./kubernetes/dockerfiles/spark/bindings/R/Dockerfile build +``` ## Cluster Mode @@ -136,7 +182,7 @@ hostname via `spark.driver.host` and your spark driver's port to `spark.driver.p ### Client Mode Executor Pod Garbage Collection -If you run your Spark driver in a pod, it is highly recommended to set `spark.driver.pod.name` to the name of that pod. +If you run your Spark driver in a pod, it is highly recommended to set `spark.kubernetes.driver.pod.name` to the name of that pod. When this property is set, the Spark scheduler will deploy the executor pods with an [OwnerReference](https://kubernetes.io/docs/concepts/workloads/controllers/garbage-collection/), which in turn will ensure that once the driver pod is deleted from the cluster, all of the application's executor pods will also be deleted. @@ -145,7 +191,7 @@ an OwnerReference pointing to that pod will be added to each executor pod's Owne setting the OwnerReference to a pod that is not actually that driver pod, or else the executors may be terminated prematurely when the wrong pod is deleted. -If your application is not running inside a pod, or if `spark.driver.pod.name` is not set when your application is +If your application is not running inside a pod, or if `spark.kubernetes.driver.pod.name` is not set when your application is actually running in a pod, keep in mind that the executor pods may not be properly deleted from the cluster when the application exits. The Spark scheduler attempts to delete these pods, but if the network request to the API server fails for any reason, these pods will remain in the cluster. The executor processes should exit when they cannot reach the @@ -162,8 +208,31 @@ If your application's dependencies are all hosted in remote locations like HDFS by their appropriate remote URIs. Also, application dependencies can be pre-mounted into custom-built Docker images. Those dependencies can be added to the classpath by referencing them with `local://` URIs and/or setting the `SPARK_EXTRA_CLASSPATH` environment variable in your Dockerfiles. The `local://` scheme is also required when referring to -dependencies in custom-built Docker images in `spark-submit`. Note that using application dependencies from the submission -client's local file system is currently not yet supported. +dependencies in custom-built Docker images in `spark-submit`. We support dependencies from the submission +client's local file system using the `file://` scheme or without a scheme (using a full path), where the destination should be a Hadoop compatible filesystem. +A typical example of this using S3 is via passing the following options: + +``` +... +--packages com.amazonaws:aws-java-sdk:1.7.4,org.apache.hadoop:hadoop-aws:2.7.6 +--conf spark.kubernetes.file.upload.path=s3a:///path +--conf spark.hadoop.fs.s3a.access.key=... +--conf spark.hadoop.fs.s3a.impl=org.apache.hadoop.fs.s3a.S3AFileSystem +--conf spark.hadoop.fs.s3a.fast.upload=true +--conf spark.hadoop.fs.s3a.secret.key=.... +--conf spark.driver.extraJavaOptions=-Divy.cache.dir=/tmp -Divy.home=/tmp +file:///full/path/to/app.jar +``` +The app jar file will be uploaded to the S3 and then when the driver is launched it will be downloaded +to the driver pod and will be added to its classpath. Spark will generate a subdir under the upload path with a random name +to avoid conflicts with spark apps running in parallel. User could manage the subdirs created according to his needs. + +The client scheme is supported for the application jar, and dependencies specified by properties `spark.jars` and `spark.files`. + +Important: all client-side dependencies will be uploaded to the given path with a flat directory structure so +file names must be unique otherwise files will be overwritten. Also make sure in the derived k8s image default ivy dir +has the required access rights or modify the settings as above. The latter is also important if you use `--packages` in +cluster mode. ## Secret Management Kubernetes [Secrets](https://kubernetes.io/docs/concepts/configuration/secret/) can be used to provide credentials for a @@ -185,6 +254,25 @@ To use a secret through an environment variable use the following options to the --conf spark.kubernetes.executor.secretKeyRef.ENV_NAME=name:key ``` +## Pod Template +Kubernetes allows defining pods from [template files](https://kubernetes.io/docs/concepts/workloads/pods/pod-overview/#pod-templates). +Spark users can similarly use template files to define the driver or executor pod configurations that Spark configurations do not support. +To do so, specify the spark properties `spark.kubernetes.driver.podTemplateFile` and `spark.kubernetes.executor.podTemplateFile` +to point to local files accessible to the `spark-submit` process. To allow the driver pod access the executor pod template +file, the file will be automatically mounted onto a volume in the driver pod when it's created. +Spark does not do any validation after unmarshalling these template files and relies on the Kubernetes API server for validation. + +It is important to note that Spark is opinionated about certain pod configurations so there are values in the +pod template that will always be overwritten by Spark. Therefore, users of this feature should note that specifying +the pod template file only lets Spark start with a template pod instead of an empty pod during the pod-building process. +For details, see the [full list](#pod-template-properties) of pod template values that will be overwritten by spark. + +Pod template files can also define multiple containers. In such cases, you can use the spark properties +`spark.kubernetes.driver.podTemplateContainerName` and `spark.kubernetes.executor.podTemplateContainerName` +to indicate which container should be used as a basis for the driver or executor. +If not specified, or if the container name is not valid, Spark will assume that the first container in the list +will be the driver or executor container. + ## Using Kubernetes Volumes Starting with Spark 2.4.0, users can mount the following types of Kubernetes [volumes](https://kubernetes.io/docs/concepts/storage/volumes/) into the driver and executor pods: @@ -192,12 +280,15 @@ Starting with Spark 2.4.0, users can mount the following types of Kubernetes [vo * [emptyDir](https://kubernetes.io/docs/concepts/storage/volumes/#emptydir): an initially empty volume created when a pod is assigned to a node. * [persistentVolumeClaim](https://kubernetes.io/docs/concepts/storage/volumes/#persistentvolumeclaim): used to mount a `PersistentVolume` into a pod. +**NB:** Please see the [Security](#security) section of this document for security issues related to volume mounts. + To mount a volume of any of the types above into the driver pod, use the following configuration property: ``` --conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.path= --conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly= -``` +--conf spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath= +``` Specifically, `VolumeType` can be one of the following values: `hostPath`, `emptyDir`, and `persistentVolumeClaim`. `VolumeName` is the name you want to use for the volume under the `volumes` field in the pod specification. @@ -205,7 +296,7 @@ Each supported type of volumes may have some specific configuration options, whi ``` spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].options.[OptionName]= -``` +``` For example, the claim name of a `persistentVolumeClaim` with volume name `checkpointpvc` can be specified using the following property: @@ -213,7 +304,7 @@ For example, the claim name of a `persistentVolumeClaim` with volume name `check spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.options.claimName=check-point-pvc-claim ``` -The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. +The configuration properties for mounting volumes into the executor pods use prefix `spark.kubernetes.executor.` instead of `spark.kubernetes.driver.`. For a complete list of available options for each supported type of volumes, please refer to the [Spark Properties](#spark-properties) section below. ## Local Storage @@ -281,6 +372,16 @@ the Spark application. ## Kubernetes Features +### Configuration File + +Your Kubernetes config file typically lives under `.kube/config` in your home directory or in a location specified by the `KUBECONFIG` environment variable. Spark on Kubernetes will attempt to use this file to do an initial auto-configuration of the Kubernetes client used to interact with the Kubernetes cluster. A variety of Spark configuration properties are provided that allow further customising the client configuration e.g. using an alternative authentication method. + +### Contexts + +Kubernetes configuration files can contain multiple contexts that allow for switching between different clusters and/or user identities. By default Spark on Kubernetes will use your current context (which can be checked by running `kubectl config current-context`) when doing the initial auto-configuration of the Kubernetes client. + +In order to use an alternative context users can specify the desired context via the Spark configuration property `spark.kubernetes.context` e.g. `spark.kubernetes.context=minikube`. + ### Namespaces Kubernetes has the concept of [namespaces](https://kubernetes.io/docs/concepts/overview/working-with-objects/namespaces/). @@ -340,6 +441,36 @@ RBAC authorization and how to configure Kubernetes service accounts for pods, pl [Using RBAC Authorization](https://kubernetes.io/docs/admin/authorization/rbac/) and [Configure Service Accounts for Pods](https://kubernetes.io/docs/tasks/configure-pod-container/configure-service-account/). +## Spark Application Management + +Kubernetes provides simple application management via the spark-submit CLI tool in cluster mode. +Users can kill a job by providing the submission ID that is printed when submitting their job. +The submission ID follows the format ``namespace:driver-pod-name``. +If user omits the namespace then the namespace set in current k8s context is used. +For example if user has set a specific namespace as follows `kubectl config set-context minikube --namespace=spark` +then the `spark` namespace will be used by default. On the other hand, if there is no namespace added to the specific context +then all namespaces will be considered by default. That means operations will affect all Spark applications matching the given submission ID regardless of namespace. +Moreover, spark-submit for application management uses the same backend code that is used for submitting the driver, so the same properties +like `spark.kubernetes.context` etc., can be re-used. + +For example: +```bash +$ spark-submit --kill spark:spark-pi-1547948636094-driver --master k8s://https://192.168.2.8:8443 +``` +Users also can list the application status by using the `--status` flag: + +```bash +$ spark-submit --status spark:spark-pi-1547948636094-driver --master k8s://https://192.168.2.8:8443 +``` +Both operations support glob patterns. For example user can run: +```bash +$ spark-submit --kill spark:spark-pi* --master k8s://https://192.168.2.8:8443 +``` +The above will kill all application with the specific prefix. + +User can specify the grace period for pod termination via the `spark.kubernetes.appKillPodDeletionGracePeriod` property, +using `--conf` as means to provide it (default value for all K8s pods is 30 secs). + ## Future Work There are several Spark on Kubernetes features that are currently being worked on or planned to be worked on. Those features are expected to eventually make it into future versions of the spark-kubernetes integration. @@ -347,19 +478,27 @@ There are several Spark on Kubernetes features that are currently being worked o Some of these include: * Dynamic Resource Allocation and External Shuffle Service -* Local File Dependency Management -* Spark Application Management * Job Queues and Resource Management # Configuration -See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are -specific to Spark on Kubernetes. +See the [configuration page](configuration.html) for information on Spark configurations. The following configurations are specific to Spark on Kubernetes. #### Spark Properties + + + + + @@ -629,7 +768,7 @@ specific to Spark on Kubernetes. @@ -676,6 +815,15 @@ specific to Spark on Kubernetes. Interval between reports of the current Spark job status in cluster mode. + + + + + @@ -689,7 +837,7 @@ specific to Spark on Kubernetes. @@ -758,6 +906,14 @@ specific to Spark on Kubernetes. spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. + + + + + @@ -782,6 +938,14 @@ specific to Spark on Kubernetes. spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.path=/checkpoint. + + + + + @@ -811,14 +975,286 @@ specific to Spark on Kubernetes. - + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.kubernetes.context(none) + The context from the user Kubernetes configuration file used for the initial + auto-configuration of the Kubernetes client library. When not specified then + the users current context is used. NB: Many of the + auto-configured settings can be overridden by the use of other Spark + configuration properties e.g. spark.kubernetes.namespace. +
    spark.kubernetes.namespace default Add the label specified by LabelName to the executor pods. For example, spark.kubernetes.executor.label.something=true. - Note that Spark also adds its own labels to the driver pod + Note that Spark also adds its own labels to the executor pod for bookkeeping purposes.
    spark.kubernetes.driver.request.cores(none) + Specify the cpu request for the driver pod. Values conform to the Kubernetes convention. + Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. + This takes precedence over spark.driver.cores for specifying the driver pod cpu request if set. +
    spark.kubernetes.driver.limit.cores (none) Specify the cpu request for each executor pod. Values conform to the Kubernetes convention. Example values include 0.1, 500m, 1.5, 5, etc., with the definition of cpu units documented in CPU units. - This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task + This is distinct from spark.executor.cores: it is only used and takes precedence over spark.executor.cores for specifying the executor pod cpu request if set. Task parallelism, e.g., number of tasks an executor can run concurrently is not affected by this.
    spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.subPath(none) + Specifies a subpath to be mounted from the volume into the driver pod. + spark.kubernetes.driver.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. +
    spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.readOnly (none)
    spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.subPath(none) + Specifies a subpath to be mounted from the volume into the executor pod. + spark.kubernetes.executor.volumes.persistentVolumeClaim.checkpointpvc.mount.subPath=checkpoint. +
    spark.kubernetes.executor.volumes.[VolumeType].[VolumeName].mount.readOnly false0.1 This sets the Memory Overhead Factor that will allocate memory to non-JVM memory, which includes off-heap memory allocations, non-JVM tasks, and various systems processes. For JVM-based jobs this value will default to 0.10 and 0.40 for non-JVM jobs. - This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with a higher default. + This is done as non-JVM tasks need more non-JVM heap space and such tasks commonly fail with "Memory Overhead Exceeded" errors. This prempts this error with a higher default.
    spark.kubernetes.pyspark.pythonVersion"2""3" + This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. +
    spark.kubernetes.kerberos.krb5.path(none) + Specify the local location of the krb5.conf file to be mounted on the driver and executors for Kerberos interaction. + It is important to note that the KDC defined needs to be visible from inside the containers. +
    spark.kubernetes.kerberos.krb5.configMapName(none) + Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted on the driver and executors + for Kerberos interaction. The KDC defined needs to be visible from inside the containers. The ConfigMap must also + be in the same namespace of the driver and executor pods. +
    spark.kubernetes.hadoop.configMapName(none) + Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, to be mounted on the driver + and executors for custom Hadoop configuration. +
    spark.kubernetes.kerberos.tokenSecret.name(none) + Specify the name of the secret where your existing delegation tokens are stored. This removes the need for the job user + to provide any kerberos credentials for launching a job. +
    spark.kubernetes.kerberos.tokenSecret.itemKey(none) + Specify the item key of the data where your existing delegation tokens are stored. This removes the need for the job user + to provide any kerberos credentials for launching a job. +
    spark.kubernetes.driver.podTemplateFile(none) + Specify the local file that contains the driver pod template. For example + spark.kubernetes.driver.podTemplateFile=/path/to/driver-pod-template.yaml +
    spark.kubernetes.driver.podTemplateContainerName(none) + Specify the container name to be used as a basis for the driver in the given pod template. + For example spark.kubernetes.driver.podTemplateContainerName=spark-driver +
    spark.kubernetes.executor.podTemplateFile(none) + Specify the local file that contains the executor pod template. For example + spark.kubernetes.executor.podTemplateFile=/path/to/executor-pod-template.yaml +
    spark.kubernetes.executor.podTemplateContainerName(none) + Specify the container name to be used as a basis for the executor in the given pod template. + For example spark.kubernetes.executor.podTemplateContainerName=spark-executor +
    spark.kubernetes.executor.deleteOnTerminationtrue + Specify whether executor pods should be deleted in case of failure or normal termination. +
    spark.kubernetes.submission.connectionTimeout10000 + Connection timeout in milliseconds for the kubernetes client to use for starting the driver. +
    spark.kubernetes.submission.requestTimeout10000 + Request timeout in milliseconds for the kubernetes client to use for starting the driver. +
    spark.kubernetes.driver.connectionTimeout10000 + Connection timeout in milliseconds for the kubernetes client in driver to use when requesting executors. +
    spark.kubernetes.driver.requestTimeout10000 + Request timeout in milliseconds for the kubernetes client in driver to use when requesting executors. +
    spark.kubernetes.appKillPodDeletionGracePeriod(none) + Specify the grace period in seconds when deleting a Spark application using spark-submit. +
    spark.kubernetes.file.upload.path(none) + Path to store files at the spark submit side in cluster mode. For example: + spark.kubernetes.file.upload.path=s3a:///path + File should specified as file://path/to/file or absolute path. +
    + +#### Pod template properties + +See the below table for the full list of pod specifications that will be overwritten by spark. + +### Pod Metadata + + + + + + + + + + + + + + + + + + + + + + + +
    Pod metadata keyModified valueDescription
    nameValue of spark.kubernetes.driver.pod.name + The driver pod name will be overwritten with either the configured or default value of + spark.kubernetes.driver.pod.name. The executor pod names will be unaffected. +
    namespaceValue of spark.kubernetes.namespace + Spark makes strong assumptions about the driver and executor namespaces. Both driver and executor namespaces will + be replaced by either the configured or default spark conf value. +
    labelsAdds the labels from spark.kubernetes.{driver,executor}.label.* + Spark will add additional labels specified by the spark configuration. +
    annotationsAdds the annotations from spark.kubernetes.{driver,executor}.annotation.* + Spark will add additional annotations specified by the spark configuration. +
    + +### Pod Spec + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Pod spec keyModified valueDescription
    imagePullSecretsAdds image pull secrets from spark.kubernetes.container.image.pullSecrets + Additional pull secrets will be added from the spark configuration to both executor pods. +
    nodeSelectorAdds node selectors from spark.kubernetes.node.selector.* + Additional node selectors will be added from the spark configuration to both executor pods. +
    restartPolicy"never" + Spark assumes that both drivers and executors never restart. +
    serviceAccountValue of spark.kubernetes.authenticate.driver.serviceAccountName + Spark will override serviceAccount with the value of the spark configuration for only + driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected. +
    serviceAccountNameValue of spark.kubernetes.authenticate.driver.serviceAccountName + Spark will override serviceAccountName with the value of the spark configuration for only + driver pods, and only if the spark configuration is specified. Executor pods will remain unaffected. +
    volumesAdds volumes from spark.kubernetes.{driver,executor}.volumes.[VolumeType].[VolumeName].mount.path + Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing + spark conf and pod template files. +
    + +### Container spec + +The following affect the driver and executor containers. All other containers in the pod spec will be unaffected. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Container spec keyModified valueDescription
    envAdds env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName] + Spark will add driver env variables from spark.kubernetes.driverEnv.[EnvironmentVariableName], and + executor env variables from spark.executorEnv.[EnvironmentVariableName]. +
    imageValue of spark.kubernetes.{driver,executor}.container.image + The image will be defined by the spark configurations. +
    imagePullPolicyValue of spark.kubernetes.container.image.pullPolicy + Spark will override the pull policy for both driver and executors. +
    nameSee description. + The container name will be assigned by spark ("spark-kubernetes-driver" for the driver container, and + "executor" for each executor container) if not defined by the pod template. If the container is defined by the + template, the template's name will be used. +
    resourcesSee description + The cpu limits are set by spark.kubernetes.{driver,executor}.limit.cores. The cpu is set by + spark.{driver,executor}.cores. The memory request and limit are set by summing the values of + spark.{driver,executor}.memory and spark.{driver,executor}.memoryOverhead. + Other resource limits are set by spark.{driver,executor}.resources.{resourceName}.* configs. +
    volumeMountsAdd volumes from spark.kubernetes.driver.volumes.[VolumeType].[VolumeName].mount.{path,readOnly} - This sets the major Python version of the docker image used to run the driver and executor containers. Can either be 2 or 3. + Spark will add volumes as specified by the spark conf, as well as additional volumes necessary for passing + spark conf and pod template files.
    diff --git a/docs/running-on-mesos.md b/docs/running-on-mesos.md index b473e654563d6..907f414e5dc4c 100644 --- a/docs/running-on-mesos.md +++ b/docs/running-on-mesos.md @@ -1,6 +1,21 @@ --- layout: global title: Running Spark on Mesos +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 will become a table of contents (this text will be scraped). {:toc} @@ -13,6 +28,11 @@ The advantages of deploying Spark with Mesos include: [frameworks](https://mesos.apache.org/documentation/latest/frameworks/) - scalable partitioning between multiple instances of Spark +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # How it Works In a standalone cluster deployment, the cluster manager in the below diagram is a Spark master @@ -103,6 +123,19 @@ Please note that if you specify multiple ways to obtain the credentials then the An equivalent order applies for the secret. Essentially we prefer the configuration to be specified directly rather than indirectly by files, and we prefer that configuration settings are used over environment variables. +### Deploy to a Mesos running on Secure Sockets + +If you want to deploy a Spark Application into a Mesos cluster that is running in a secure mode there are some environment variables that need to be set. + +- `LIBPROCESS_SSL_ENABLED=true` enables SSL communication +- `LIBPROCESS_SSL_VERIFY_CERT=false` verifies the ssl certificate +- `LIBPROCESS_SSL_KEY_FILE=pathToKeyFile.key` path to key +- `LIBPROCESS_SSL_CERT_FILE=pathToCRTFile.crt` the certificate file to be used + +All options can be found at http://mesos.apache.org/documentation/latest/ssl/ + +Then submit happens as described in Client mode or Cluster mode below + ## Uploading Spark Package When Mesos runs a task on a Mesos slave for the first time, that slave must have a Spark binary @@ -432,7 +465,7 @@ See the [configuration page](configuration.html) for information on Spark config spark.mesos.executor.memoryOverhead executor memory * 0.10, with minimum of 384 - The amount of additional memory, specified in MB, to be allocated per executor. By default, + The amount of additional memory, specified in MiB, to be allocated per executor. By default, the overhead will be larger of either 384 or 10% of spark.executor.memory. If set, the final overhead will be this value. @@ -682,7 +715,7 @@ See the [configuration page](configuration.html) for information on Spark config 0 Set the maximum number GPU resources to acquire for this job. Note that executors will still launch when no GPU resources are found - since this configuration is just a upper limit and not a guaranteed amount. + since this configuration is just an upper limit and not a guaranteed amount. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index bdf7b99966e4f..9d9b253a5c8ea 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -1,6 +1,21 @@ --- layout: global title: Running Spark on YARN +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 will become a table of contents (this text will be scraped). {:toc} @@ -9,6 +24,11 @@ Support for running on [YARN (Hadoop NextGen)](http://hadoop.apache.org/docs/stable/hadoop-yarn/hadoop-yarn-site/YARN.html) was added to Spark in version 0.6.0, and improved in subsequent releases. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # Launching Spark on YARN Ensure that `HADOOP_CONF_DIR` or `YARN_CONF_DIR` points to the directory which contains the (client side) configuration files for the Hadoop cluster. @@ -121,6 +141,43 @@ To use a custom metrics.properties for the application master and executors, upd Use lower-case suffixes, e.g. k, m, g, t, and p, for kibi-, mebi-, gibi-, tebi-, and pebibytes, respectively. + + spark.yarn.am.resource.{resource-type}.amount + (none) + + Amount of resource to use for the YARN Application Master in client mode. + In cluster mode, use spark.yarn.driver.resource.<resource-type>.amount instead. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

    + Example: + To request GPU resources from YARN, use: spark.yarn.am.resource.yarn.io/gpu.amount + + + + spark.yarn.driver.resource.{resource-type}.amount + (none) + + Amount of resource to use for the YARN Application Master in cluster mode. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

    + Example: + To request GPU resources from YARN, use: spark.yarn.driver.resource.yarn.io/gpu.amount + + + + spark.yarn.executor.resource.{resource-type}.amount + (none) + + Amount of resource to use per executor process. + Please note that this feature can be used only with YARN 3.0+ + For reference, see YARN Resource Model documentation: https://hadoop.apache.org/docs/r3.0.1/hadoop-yarn/hadoop-yarn-site/ResourceModel.html +

    + Example: + To request GPU resources from YARN, use: spark.yarn.executor.resource.yarn.io/gpu.amount + + spark.yarn.am.cores 1 @@ -420,6 +477,13 @@ To use a custom metrics.properties for the application master and executors, upd spark.blacklist.application.maxFailedExecutorsPerNode. + + spark.yarn.exclude.nodes + (none) + + Comma-separated list of YARN node names which are excluded from resource allocation. + + spark.yarn.metrics.namespace (none) @@ -430,6 +494,54 @@ To use a custom metrics.properties for the application master and executors, upd +#### Available patterns for SHS custom executor log URL + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    PatternMeaning
    {{HTTP_SCHEME}}`http://` or `https://` according to YARN HTTP policy. (Configured via `yarn.http.policy`)
    {{NM_HOST}}The "host" of node where container was run.
    {{NM_PORT}}The "port" of node manager where container was run.
    {{NM_HTTP_PORT}}The "port" of node manager's http server where container was run.
    {{NM_HTTP_ADDRESS}}Http URI of the node on which the container is allocated.
    {{CLUSTER_ID}}The cluster ID of Resource Manager. (Configured via `yarn.resourcemanager.cluster-id`)
    {{CONTAINER_ID}}The ID of container.
    {{USER}}'SPARK_USER' on system environment.
    {{FILE_NAME}}`stdout`, `stderr`.
    + +For example, suppose you would like to point log url link to Job History Server directly instead of let NodeManager http server redirects it, you can configure `spark.history.custom.executor.log.url` as below: + + `{{HTTP_SCHEME}}:/jobhistory/logs/{{NM_HOST}}:{{NM_PORT}}/{{CONTAINER_ID}}/{{CONTAINER_ID}}/{{USER}}/{{FILE_NAME}}?start=-4096` + + NOTE: you need to replace `` and `` with actual value. + # Important notes - Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. @@ -448,17 +560,6 @@ for: filesystem if `spark.yarn.stagingDir` is not set); - if Hadoop federation is enabled, all the federated filesystems in the configuration. -If an application needs to interact with other secure Hadoop filesystems, their URIs need to be -explicitly provided to Spark at launch time. This is done by listing them in the -`spark.yarn.access.hadoopFileSystems` property, described in the configuration section below. - -The YARN integration also supports custom delegation token providers using the Java Services -mechanism (see `java.util.ServiceLoader`). Implementations of -`org.apache.spark.deploy.yarn.security.ServiceCredentialProvider` can be made available to Spark -by listing their names in the corresponding file in the jar's `META-INF/services` directory. These -providers can be disabled individually by setting `spark.security.credentials.{service}.enabled` to -`false`, where `{service}` is the name of the credential provider. - ## YARN-specific Kerberos Configuration @@ -485,18 +586,6 @@ providers can be disabled individually by setting `spark.security.credentials.{s
    (Works also with the "local" master.) - - - - - @@ -602,7 +691,7 @@ spark.security.credentials.hive.enabled false spark.security.credentials.hbase.enabled false ``` -The configuration option `spark.yarn.access.hadoopFileSystems` must be unset. +The configuration option `spark.kerberos.access.hadoopFileSystems` must be unset. # Using the Spark History Server to replace the Spark Web UI @@ -617,4 +706,4 @@ do the following: - On the Spark History Server, add org.apache.spark.deploy.yarn.YarnProxyRedirectFilter to the list of filters in the spark.ui.filters configuration. -Be aware that the history server information may not be up-to-date with the application's state. +Be aware that the history server information may not be up-to-date with the application's state. \ No newline at end of file diff --git a/docs/security.md b/docs/security.md index 7fb3e17de94c9..a4556e242cffc 100644 --- a/docs/security.md +++ b/docs/security.md @@ -2,32 +2,68 @@ layout: global displayTitle: Spark Security title: Security +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 will become a table of contents (this text will be scraped). {:toc} -# Spark RPC +# Spark Security: Things You Need To Know + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Spark supports multiple deployments types and each one supports different levels of security. Not +all deployment types will be secure in all environments and none are secure by default. Be +sure to evaluate your environment, what Spark supports, and take the appropriate measure to secure +your Spark deployment. + +There are many different types of security concerns. Spark does not necessarily protect against +all things. Listed below are some of the things Spark supports. Also check the deployment +documentation for the type of deployment you are using for deployment specific settings. Anything +not documented, Spark does not support. + +# Spark RPC (Communication protocol between Spark processes) ## Authentication Spark currently supports authentication for RPC channels using a shared secret. Authentication can be turned on by setting the `spark.authenticate` configuration parameter. -The exact mechanism used to generate and distribute the shared secret is deployment-specific. +The exact mechanism used to generate and distribute the shared secret is deployment-specific. Unless +specified below, the secret must be defined by setting the `spark.authenticate.secret` config +option. The same secret is shared by all Spark applications and daemons in that case, which limits +the security of these deployments, especially on multi-tenant clusters. + +The REST Submission Server and the MesosClusterDispatcher do not support authentication. You should +ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 +respectively by default) are restricted to hosts that are trusted to submit jobs. + +### YARN -For Spark on [YARN](running-on-yarn.html) and local deployments, Spark will automatically handle -generating and distributing the shared secret. Each application will use a unique shared secret. In +For Spark on [YARN](running-on-yarn.html), Spark will automatically handle generating and +distributing the shared secret. Each application will use a unique shared secret. In the case of YARN, this feature relies on YARN RPC encryption being enabled for the distribution of secrets to be secure. -For other resource managers, `spark.authenticate.secret` must be configured on each of the nodes. -This secret will be shared by all the daemons and applications, so this deployment configuration is -not as secure as the above, especially when considering multi-tenant clusters. In this -configuration, a user with the secret can effectively impersonate any other user. +### Kubernetes -The Rest Submission Server and the MesosClusterDispatcher do not support authentication. You should -ensure that all network access to the REST API & MesosClusterDispatcher (port 6066 and 7077 -respectively by default) are restricted to hosts that are trusted to submit jobs. +On Kubernetes, Spark will also automatically generate an authentication secret unique to each +application. The secret is propagated to executor pods using environment variables. This means +that any user that can list pods in the namespace where the Spark application is running can +also see their authentication secret. Access control rules should be properly set up by the +Kubernetes admin to ensure that Spark authentication is secure.
    spark.yarn.access.hadoopFileSystems(none) - A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For - example, spark.yarn.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032, - webhdfs://nn3.com:50070. The Spark application must have access to the filesystems listed - and Kerberos must be properly configured to be able to access them (either in the same realm - or in a trusted realm). Spark acquires security tokens for each of the filesystems so that - the Spark application can access those remote Hadoop filesystems. -
    spark.yarn.kerberos.relogin.period 1m
    @@ -45,6 +81,50 @@ respectively by default) are restricted to hosts that are trusted to submit jobs
    Property NameDefaultMeaning
    +Alternatively, one can mount authentication secrets using files and Kubernetes secrets that +the user mounts into their pods. + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.authenticate.secret.fileNone + Path pointing to the secret key to use for securing connections. Ensure that the + 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). +
    spark.authenticate.secret.driver.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark driver reads to load the secret. + Useful when in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.executor.file must be specified so that the driver + 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. +
    spark.authenticate.secret.executor.fileThe value of spark.authenticate.secret.file + When specified, overrides the location that the Spark executors read to load the secret. + Useful in client mode, when the location of the secret file may differ in the pod versus + the node the driver is running in. When this is specified, + spark.authenticate.secret.driver.file must be specified so that the driver + 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. +
    + +Note that when using files, Spark will not mount these files into the containers for you. It is up +you to ensure that the secret files are deployed securely into your containers and that the driver's +secret file agrees with the executors' secret file. + ## Encryption Spark supports AES-based encryption for RPC connections. For encryption to be enabled, RPC @@ -123,7 +203,7 @@ The following table describes the different options available for configuring th Spark supports encrypting temporary data written to local disks. This covers shuffle files, shuffle spills and data blocks stored on disk (for both caching and broadcast variables). It does not cover encrypting output data generated by applications with APIs such as `saveAsHadoopFile` or -`saveAsTable`. +`saveAsTable`. It also may not cover temporary files created explicitly by the user. The following settings cover enabling encryption for data written to disk: @@ -324,7 +404,7 @@ Configuration for SSL is organized hierarchically. The user can configure the de which will be used for all the supported communication protocols unless they are overwritten by protocol-specific settings. This way the user can easily provide the common settings for all the protocols without disabling the ability to configure each one individually. The following table -describes the the SSL configuration namespaces: +describes the SSL configuration namespaces: @@ -687,6 +767,15 @@ configuration has Kerberos authentication turned (`hbase.security.authentication Similarly, a Hive token will be obtained if Hive is in the classpath, and the configuration includes URIs for remote metastore services (`hive.metastore.uris` is not empty). +If an application needs to interact with other secure Hadoop filesystems, their URIs need to be +explicitly provided to Spark at launch time. This is done by listing them in the +`spark.kerberos.access.hadoopFileSystems` property, described in the configuration section below. + +Spark also supports custom delegation token providers using the Java Services +mechanism (see `java.util.ServiceLoader`). Implementations of +`org.apache.spark.security.HadoopDelegationTokenProvider` can be made available to Spark +by listing their names in the corresponding file in the jar's `META-INF/services` directory. + Delegation token support is currently only supported in YARN and Mesos modes. Consult the deployment-specific page for more information. @@ -704,6 +793,18 @@ The following options provides finer-grained control for this feature: application being run. + + + + +
    spark.kerberos.access.hadoopFileSystems(none) + A comma-separated list of secure Hadoop filesystems your Spark application is going to access. For + example, spark.kerberos.access.hadoopFileSystems=hdfs://nn1.com:8032,hdfs://nn2.com:8032, + webhdfs://nn3.com:50070. The Spark application must have access to the filesystems listed + and Kerberos must be properly configured to be able to access them (either in the same realm + or in a trusted realm). Spark acquires security tokens for each of the filesystems so that + the Spark application can access those remote Hadoop filesystems. +
    ## Long-Running Applications @@ -711,18 +812,109 @@ The following options provides finer-grained control for this feature: Long-running applications may run into issues if their run time exceeds the maximum delegation token lifetime configured in services it needs to access. -Spark supports automatically creating new tokens for these applications when running in YARN mode. -Kerberos credentials need to be provided to the Spark application via the `spark-submit` command, -using the `--principal` and `--keytab` parameters. +This feature is not available everywhere. In particular, it's only implemented +on YARN and Kubernetes (both client and cluster modes), and on Mesos when using client mode. + +Spark supports automatically creating new tokens for these applications. There are two ways to +enable this functionality. + +### Using a Keytab + +By providing Spark with a principal and keytab (e.g. using `spark-submit` with `--principal` +and `--keytab` parameters), the application will maintain a valid Kerberos login that can be +used to retrieve delegation tokens indefinitely. + +Note that when using a keytab in cluster mode, it will be copied over to the machine running the +Spark driver. In the case of YARN, this means using HDFS as a staging area for the keytab, so it's +strongly recommended that both YARN and HDFS be secured with encryption, at least. + +### Using a ticket cache + +By setting `spark.kerberos.renewal.credentials` to `ccache` in Spark's configuration, the local +Kerberos ticket cache will be used for authentication. Spark will keep the ticket renewed during its +renewable life, but after it expires a new ticket needs to be acquired (e.g. by running `kinit`). + +It's up to the user to maintain an updated ticket cache that Spark can use. + +The location of the ticket cache can be customized by setting the `KRB5CCNAME` environment +variable. -The provided keytab will be copied over to the machine running the Application Master via the Hadoop -Distributed Cache. For this reason, it's strongly recommended that both YARN and HDFS be secured -with encryption, at least. +## Secure Interaction with Kubernetes -The Kerberos login will be periodically renewed using the provided credentials, and new delegation -tokens for supported will be created. +When talking to Hadoop-based services behind Kerberos, it was noted that Spark needs to obtain delegation tokens +so that non-local processes can authenticate. These delegation tokens in Kubernetes are stored in Secrets that are +shared by the Driver and its Executors. As such, there are three ways of submitting a Kerberos job: +In all cases you must define the environment variable: `HADOOP_CONF_DIR` or +`spark.kubernetes.hadoop.configMapName.` +It also important to note that the KDC needs to be visible from inside the containers. + +If a user wishes to use a remote HADOOP_CONF directory, that contains the Hadoop configuration files, this could be +achieved by setting `spark.kubernetes.hadoop.configMapName` to a pre-existing ConfigMap. + +1. Submitting with a $kinit that stores a TGT in the Local Ticket Cache: +```bash +/usr/bin/kinit -kt / +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` +2. Submitting with a local Keytab and Principal +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kerberos.keytab= \ + --conf spark.kerberos.principal= \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` + +3. Submitting with pre-populated secrets, that contain the Delegation Token, already existing within the namespace +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.kerberos.krb5.path=/etc/krb5.conf \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` + +3b. Submitting like in (3) however specifying a pre-created krb5 ConfigMap and pre-created `HADOOP_CONF_DIR` ConfigMap +```bash +/opt/spark/bin/spark-submit \ + --deploy-mode cluster \ + --class org.apache.spark.examples.HdfsTest \ + --master k8s:// \ + --conf spark.executor.instances=1 \ + --conf spark.app.name=spark-hdfs \ + --conf spark.kubernetes.container.image=spark:latest \ + --conf spark.kubernetes.kerberos.tokenSecret.name= \ + --conf spark.kubernetes.kerberos.tokenSecret.itemKey= \ + --conf spark.kubernetes.hadoop.configMapName= \ + --conf spark.kubernetes.kerberos.krb5.configMapName= \ + local:///opt/spark/examples/jars/spark-examples_.jar \ + +``` # Event Logging If your applications are using event logging, the directory where the event logs go @@ -733,3 +925,14 @@ should correspond to the super user who is running the Spark History Server. This will allow all users to write to the directory but will prevent unprivileged users from reading, removing or renaming a file unless they own it. The event log files will be created by Spark with permissions such that only the user and group have read and write access. + +# Persisting driver logs in client mode + +If your applications persist driver logs in client mode by enabling `spark.driver.log.persistToDfs.enabled`, +the directory where the driver logs go (`spark.driver.log.dfsDir`) should be manually created with proper +permissions. To secure the log files, the directory permissions should be set to `drwxrwxrwxt`. The owner +and group of the directory should correspond to the super user who is running the Spark History Server. + +This will allow all users to write to the directory but will prevent unprivileged users from +reading, removing or renaming a file unless they own it. The driver log files will be created by +Spark with permissions such that only the user and group have read and write access. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7975b0c8b11ca..2ca3ee6aa7213 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -1,6 +1,21 @@ --- layout: global title: Spark Standalone Mode +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 will become a table of contents (this text will be scraped). @@ -8,6 +23,11 @@ title: Spark Standalone Mode In addition to running on the Mesos or YARN cluster managers, Spark also provides a simple standalone deploy mode. You can launch a standalone cluster either manually, by starting a master and workers by hand, or use our provided [launch scripts](#cluster-launch-scripts). It is also possible to run these daemons on a single machine for testing. +# Security + +Security in Spark is OFF by default. This could mean you are vulnerable to attack by default. +Please see [Spark Security](security.html) and the specific security sections in this doc before running Spark. + # Installing Spark Standalone to a Cluster To install Spark Standalone mode, you simply place a compiled version of Spark on each node on the cluster. You can obtain pre-built versions of Spark with each release or [build it yourself](building-spark.html). @@ -55,7 +75,7 @@ Finally, the following configuration options can be passed to the master and wor -m MEM, --memory MEM - Total amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GB); only on worker + Total amount of memory to allow Spark applications to use on the machine, in a format like 1000M or 2G (default: your machine's total RAM minus 1 GiB); only on worker -d DIR, --work-dir DIR @@ -80,12 +100,13 @@ If you do not have a password-less setup, you can set the environment variable S Once you've set up this file, you can launch or stop your cluster with the following shell scripts, based on Hadoop's deploy scripts, and available in `SPARK_HOME/sbin`: - `sbin/start-master.sh` - Starts a master instance on the machine the script is executed on. -- `sbin/start-slaves.sh` - Starts a slave instance on each machine specified in the `conf/slaves` file. -- `sbin/start-slave.sh` - Starts a slave instance on the machine the script is executed on. -- `sbin/start-all.sh` - Starts both a master and a number of slaves as described above. +- `sbin/start-slaves.sh` - Starts a worker instance on each machine specified in the `conf/slaves` file. +- `sbin/start-slave.sh` - Starts a worker instance on the machine the script is executed on. +- `sbin/start-all.sh` - Starts both a master and a number of workers as described above. - `sbin/stop-master.sh` - Stops the master that was started via the `sbin/start-master.sh` script. -- `sbin/stop-slaves.sh` - Stops all slave instances on the machines specified in the `conf/slaves` file. -- `sbin/stop-all.sh` - Stops both the master and the slaves as described above. +- `sbin/stop-slave.sh` - Stops all worker instances on the machine the script is executed on. +- `sbin/stop-slaves.sh` - Stops all worker instances on the machines specified in the `conf/slaves` file. +- `sbin/stop-all.sh` - Stops both the master and the workers as described above. Note that these scripts must be executed on the machine you want to run the Spark master on, not your local machine. @@ -123,7 +144,7 @@ You can optionally configure the cluster further by setting environment variable SPARK_WORKER_MEMORY - Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. + Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GiB); note that each application's individual memory is configured using its spark.executor.memory property. SPARK_WORKER_PORT @@ -234,6 +255,7 @@ SPARK_WORKER_OPTS supports the following system properties: Enable periodic cleanup of worker / application directories. Note that this only affects standalone mode, as YARN works differently. Only the directories of stopped applications are cleaned up. + This should be enabled if spark.shuffle.service.db.enabled is "true" @@ -254,6 +276,16 @@ SPARK_WORKER_OPTS supports the following system properties: especially if you run jobs very frequently. + + spark.shuffle.service.db.enabled + true + + Store External Shuffle service state on local disk so that when the external shuffle service is restarted, it will + automatically reload info on current executors. This only affects standalone mode (yarn always has this behavior + 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. + + spark.storage.cleanupFilesAfterExecutorExit true diff --git a/docs/sparkr.md b/docs/sparkr.md index 55e8f15da17ca..7431d025aa629 100644 --- a/docs/sparkr.md +++ b/docs/sparkr.md @@ -2,6 +2,21 @@ layout: global displayTitle: SparkR (R on Spark) title: SparkR (R on Spark) +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 will become a table of contents (this text will be scraped). @@ -104,7 +119,7 @@ The following Spark driver properties can be set in `sparkConfig` with `sparkR.s

    ## Creating SparkDataFrames -With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-programming-guide.html#hive-tables), or from other [data sources](sql-programming-guide.html#data-sources). +With a `SparkSession`, applications can create `SparkDataFrame`s from a local R data frame, from a [Hive table](sql-data-sources-hive-tables.html), or from other [data sources](sql-data-sources.html). ### From local data frames The simplest way to create a data frame is to convert a local R data frame into a SparkDataFrame. Specifically, we can use `as.DataFrame` or `createDataFrame` and pass in the local R data frame to create a SparkDataFrame. As an example, the following creates a `SparkDataFrame` based using the `faithful` dataset from R. @@ -125,7 +140,7 @@ head(df) ### From Data Sources -SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-programming-guide.html#manually-specifying-options) that are available for the built-in data sources. +SparkR supports operating on a variety of data sources through the `SparkDataFrame` interface. This section describes the general methods for loading and saving data using Data Sources. You can check the Spark SQL programming guide for more [specific options](sql-data-sources-load-save-functions.html#manually-specifying-options) that are available for the built-in data sources. The general method for creating SparkDataFrames from data sources is `read.df`. This method takes in the path for the file to load and the type of data source, and the currently active SparkSession will be used automatically. SparkR supports reading JSON, CSV and Parquet files natively, and through packages available from sources like [Third Party Projects](https://spark.apache.org/third-party-projects.html), you can find data source connectors for popular file formats like Avro. These packages can either be added by @@ -133,7 +148,7 @@ specifying `--packages` with `spark-submit` or `sparkR` commands, or if initiali
    {% highlight r %} -sparkR.session(sparkPackages = "com.databricks:spark-avro_2.11:3.0.0") +sparkR.session(sparkPackages = "org.apache.spark:spark-avro_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION}}") {% endhighlight %}
    @@ -180,7 +195,7 @@ write.df(people, path = "people.parquet", source = "parquet", mode = "overwrite" ### From Hive tables -You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-programming-guide.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`). +You can also create SparkDataFrames from Hive tables. To do this we will need to create a SparkSession with Hive support which can access tables in the Hive MetaStore. Note that Spark should have been built with [Hive support](building-spark.html#building-with-hive-and-jdbc-support) and more details can be found in the [SQL programming guide](sql-getting-started.html#starting-point-sparksession). In SparkR, by default it will attempt to create a SparkSession with Hive support enabled (`enableHiveSupport = TRUE`).
    {% highlight r %} @@ -296,7 +311,7 @@ head(agg(rollup(df, "cyl", "disp", "gear"), avg(df$mpg))) ### Operating on Columns -SparkR also provides a number of functions that can directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions. +SparkR also provides a number of functions that can be directly applied to columns for data processing and during aggregation. The example below shows the use of basic arithmetic functions.
    {% highlight r %} @@ -450,6 +465,48 @@ print(model.summaries) {% endhighlight %}
    +### Eager execution + +If eager execution is enabled, the data will be returned to R client immediately when the `SparkDataFrame` is created. By default, eager execution is not enabled and can be enabled by setting the configuration property `spark.sql.repl.eagerEval.enabled` to `true` when the `SparkSession` is started up. + +Maximum number of rows and maximum number of characters per column of data to display can be controlled by `spark.sql.repl.eagerEval.maxNumRows` and `spark.sql.repl.eagerEval.truncate` configuration properties, respectively. These properties are only effective when eager execution is enabled. If these properties are not set explicitly, by default, data up to 20 rows and up to 20 characters per column will be showed. + +
    +{% highlight r %} + +# Start up spark session with eager execution enabled +sparkR.session(master = "local[*]", + sparkConfig = list(spark.sql.repl.eagerEval.enabled = "true", + spark.sql.repl.eagerEval.maxNumRows = as.integer(10))) + +# Create a grouped and sorted SparkDataFrame +df <- createDataFrame(faithful) +df2 <- arrange(summarize(groupBy(df, df$waiting), count = n(df$waiting)), "waiting") + +# Similar to R data.frame, displays the data returned, instead of SparkDataFrame class string +df2 + +##+-------+-----+ +##|waiting|count| +##+-------+-----+ +##| 43.0| 1| +##| 45.0| 3| +##| 46.0| 5| +##| 47.0| 4| +##| 48.0| 3| +##| 49.0| 5| +##| 50.0| 5| +##| 51.0| 6| +##| 52.0| 5| +##| 53.0| 7| +##+-------+-----+ +##only showing top 10 rows + +{% endhighlight %} +
    + +Note that to enable eager execution in `sparkR` shell, add `spark.sql.repl.eagerEval.enabled=true` configuration property to the `--conf` option. + ## Running SQL Queries from SparkR A SparkDataFrame can also be registered as a temporary view in Spark SQL and that allows you to run SQL queries over its data. The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. @@ -502,6 +559,7 @@ SparkR supports the following machine learning algorithms currently: * [`spark.gaussianMixture`](api/R/spark.gaussianMixture.html): [`Gaussian Mixture Model (GMM)`](ml-clustering.html#gaussian-mixture-model-gmm) * [`spark.kmeans`](api/R/spark.kmeans.html): [`K-Means`](ml-clustering.html#k-means) * [`spark.lda`](api/R/spark.lda.html): [`Latent Dirichlet Allocation (LDA)`](ml-clustering.html#latent-dirichlet-allocation-lda) +* [`spark.powerIterationClustering (PIC)`](api/R/spark.powerIterationClustering.html): [`Power Iteration Clustering (PIC)`](ml-clustering.html#power-iteration-clustering-pic) #### Collaborative Filtering @@ -510,6 +568,7 @@ SparkR supports the following machine learning algorithms currently: #### Frequent Pattern Mining * [`spark.fpGrowth`](api/R/spark.fpGrowth.html) : [`FP-growth`](ml-frequent-pattern-mining.html#fp-growth) +* [`spark.prefixSpan`](api/R/spark.prefixSpan.html) : [`PrefixSpan`](ml-frequent-pattern-mining.html#prefixSpan) #### Statistics @@ -598,6 +657,65 @@ The following example shows how to save/load a MLlib model by SparkR. SparkR supports the Structured Streaming API. Structured Streaming is a scalable and fault-tolerant stream processing engine built on the Spark SQL engine. For more information see the R API on the [Structured Streaming Programming Guide](structured-streaming-programming-guide.html) +# Apache Arrow in SparkR + +Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer data between JVM and R processes. See also PySpark optimization done, [PySpark Usage Guide for Pandas with Apache Arrow](sql-pyspark-pandas-with-arrow.html). This guide targets to explain how to use Arrow optimization in SparkR with some key points. + +## Ensure Arrow Installed + +Currently, Arrow R library is not on CRAN yet [ARROW-3204](https://issues.apache.org/jira/browse/ARROW-3204). Therefore, it should be installed directly from Github. You can use `remotes::install_github` as below. + +```bash +Rscript -e 'remotes::install_github("apache/arrow@apache-arrow-0.12.1", subdir = "r")' +``` + +`apache-arrow-0.12.1` is a version tag that can be checked in [Arrow at Github](https://github.com/apache/arrow/releases). You must ensure that Arrow R package is installed and available on all cluster nodes. The current supported version is 0.12.1. + +## Enabling for Conversion to/from R DataFrame, `dapply` and `gapply` + +Arrow optimization is available when converting a Spark DataFrame to an R DataFrame using the call `collect(spark_df)`, +when creating a Spark DataFrame from an R DataFrame with `createDataFrame(r_df)`, when applying an R native function to each partition +via `dapply(...)` and when applying an R native function to grouped data via `gapply(...)`. +To use Arrow when executing these calls, users need to first set the Spark configuration ‘spark.sql.execution.arrow.sparkr.enabled’ +to ‘true’. This is disabled by default. + +In addition, optimizations enabled by ‘spark.sql.execution.arrow.sparkr.enabled’ could fallback automatically to non-Arrow optimization +implementation if an error occurs before the actual computation within Spark during converting a Spark DataFrame to/from an R +DataFrame. + +
    +{% highlight r %} +# Start up spark session with Arrow optimization enabled +sparkR.session(master = "local[*]", + sparkConfig = list(spark.sql.execution.arrow.sparkr.enabled = "true")) + +# Converts Spark DataFrame from an R DataFrame +spark_df <- createDataFrame(mtcars) + +# Converts Spark DataFrame to an R DataFrame +collect(spark_df) + +# Apply an R native function to each partition. +collect(dapply(spark_df, function(rdf) { data.frame(rdf$gear + 1) }, structType("gear double"))) + +# Apply an R native function to grouped data. +collect(gapply(spark_df, + "gear", + function(key, group) { + data.frame(gear = key[[1]], disp = mean(group$disp) > group$disp) + }, + structType("gear double, disp boolean"))) +{% endhighlight %} +
    + +Using the above optimizations with Arrow will produce the same results as when Arrow is not enabled. Note that even with Arrow, +`collect(spark_df)` results in the collection of all records in the DataFrame to the driver program and should be done on a +small subset of the data. + +## Supported SQL Types + +Currently, all Spark SQL data types are supported by Arrow-based conversion except `FloatType`, `BinaryType`, `ArrayType`, `StructType` and `MapType`. + # R Function Name Conflicts When loading and attaching a new package in R, it is possible to have a name [conflict](https://stat.ethz.ch/R-manual/R-devel/library/base/html/library.html), where a @@ -666,8 +784,13 @@ You can inspect the search path in R with [`search()`](https://stat.ethz.ch/R-ma ## Upgrading to SparkR 2.3.1 and above - - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-base. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. + - In SparkR 2.3.0 and earlier, the `start` parameter of `substr` method was wrongly subtracted by one and considered as 0-based. This can lead to inconsistent substring results and also does not match with the behaviour with `substr` in R. In version 2.3.1 and later, it has been fixed so the `start` parameter of `substr` method is now 1-based. As an example, `substr(lit('abcdef'), 2, 4))` would result to `abc` in SparkR 2.3.0, and the result would be `bcd` in SparkR 2.3.1. ## Upgrading to SparkR 2.4.0 - Previously, we don't check the validity of the size of the last layer in `spark.mlp`. For example, if the training data only has two labels, a `layers` param like `c(1, 3)` doesn't cause an error previously, now it does. + +## Upgrading to SparkR 3.0.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. diff --git a/docs/avro-data-source-guide.md b/docs/sql-data-sources-avro.md similarity index 80% rename from docs/avro-data-source-guide.md rename to docs/sql-data-sources-avro.md index d3b81f029d377..726db2ec09ad6 100644 --- a/docs/avro-data-source-guide.md +++ b/docs/sql-data-sources-avro.md @@ -1,6 +1,21 @@ --- layout: global title: Apache Avro Data Source Guide +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 will become a table of contents (this text will be scraped). @@ -66,9 +81,9 @@ write.df(select(df, "name", "favorite_color"), "namesAndFavColors.avro", "avro") ## to_avro() and from_avro() The Avro package provides function `to_avro` to encode a column as binary in Avro format, and `from_avro()` to decode Avro binary data into a column. Both functions transform one column to -another column, and the input/output SQL data type can be complex type or primitive type. +another column, and the input/output SQL data type can be a complex type or a primitive type. -Using Avro record as columns are useful when reading from or writing to a streaming source like Kafka. Each +Using Avro record as columns is useful when reading from or writing to a streaming source like Kafka. Each Kafka key-value record will be augmented with some metadata, such as the ingestion timestamp into Kafka, the offset in Kafka, etc. * If the "value" field that contains your data is in Avro, you could use `from_avro()` to extract your data, enrich it, clean it, and then push it downstream to Kafka again or write it out to a file. * `to_avro()` can be used to turn structs into Avro records. This method is particularly useful when you would like to re-encode multiple columns into a single one when writing data out to Kafka. @@ -78,7 +93,7 @@ Both functions are currently only available in Scala and Java.
    {% highlight scala %} -import org.apache.spark.sql.avro._ +import org.apache.spark.sql.avro.functions._ // `from_avro` requires Avro schema in JSON string format. val jsonFormatSchema = new String(Files.readAllBytes(Paths.get("./examples/src/main/resources/user.avsc"))) @@ -109,7 +124,8 @@ val query = output
    {% highlight java %} -import org.apache.spark.sql.avro.*; +import static org.apache.spark.sql.functions.col; +import static org.apache.spark.sql.avro.functions.*; // `from_avro` requires Avro schema in JSON string format. String jsonFormatSchema = new String(Files.readAllBytes(Paths.get("./examples/src/main/resources/user.avsc"))); @@ -136,20 +152,54 @@ StreamingQuery query = output .option("topic", "topic2") .start(); +{% endhighlight %} +
    +
    +{% highlight python %} +from pyspark.sql.avro.functions import from_avro, to_avro + +# `from_avro` requires Avro schema in JSON string format. +jsonFormatSchema = open("examples/src/main/resources/user.avsc", "r").read() + +df = spark\ + .readStream\ + .format("kafka")\ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2")\ + .option("subscribe", "topic1")\ + .load() + +# 1. Decode the Avro data into a struct; +# 2. Filter by column `favorite_color`; +# 3. Encode the column `name` in Avro format. +output = df\ + .select(from_avro("value", jsonFormatSchema).alias("user"))\ + .where('user.favorite_color == "red"')\ + .select(to_avro("user.name").alias("value")) + +query = output\ + .writeStream\ + .format("kafka")\ + .option("kafka.bootstrap.servers", "host1:port1,host2:port2")\ + .option("topic", "topic2")\ + .start() + {% endhighlight %}
    ## Data Source Option -Data source options of Avro can be set using the `.option` method on `DataFrameReader` or `DataFrameWriter`. +Data source options of Avro can be set via: + * the `.option` method on `DataFrameReader` or `DataFrameWriter`. + * the `options` parameter in function `from_avro`. + - + @@ -177,6 +227,19 @@ Data source options of Avro can be set using the `.option` method on `DataFrameR Currently supported codecs are uncompressed, snappy, deflate, bzip2 and xz.
    If the option is not set, the configuration spark.sql.avro.compression.codec config is taken into account. + + + + + +
    Property NameDefaultMeaningScope
    avroSchema NoneOptional Avro schema provided by an user in JSON format. The date type and naming of record fields - should match the input Avro data or Catalyst data, otherwise the read/write action will fail.Optional Avro schema provided by a user in JSON format. The data type and naming of record fields + should match the Avro data type when reading from Avro or match the Spark's internal data type (e.g., StringType, IntegerType) when writing to Avro files; otherwise, the read/write action will fail. read and write
    write
    modeFAILFASTThe mode option allows to specify parse mode for function from_avro.
    + Currently supported modes are: +
      +
    • FAILFAST: Throws an exception on processing corrupted record.
    • +
    • PERMISSIVE: Corrupt records are processed as null result. Therefore, the + data schema is forced to be fully nullable, which might be different from the one user provided.
    • +
    +
    function from_avro
    ## Configuration diff --git a/docs/sql-data-sources-binaryFile.md b/docs/sql-data-sources-binaryFile.md new file mode 100644 index 0000000000000..0d41c9e441c61 --- /dev/null +++ b/docs/sql-data-sources-binaryFile.md @@ -0,0 +1,68 @@ +--- +layout: global +title: Binary File Data Source +displayTitle: Binary File Data Source +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. +--- + +Since Spark 3.0, Spark supports binary file data source, +which reads binary files and converts each file into a single record that contains the raw content +and metadata of the file. +It produces a DataFrame with the following columns and possibly partition columns: +* `path`: StringType +* `modificationTime`: TimestampType +* `length`: LongType +* `content`: BinaryType + +To read whole binary files, you need to specify the data source `format` as `binaryFile`. +To load files with paths matching a given glob pattern while keeping the behavior of partition discovery, +you can use the general data source option `pathGlobFilter`. +For example, the following code reads all PNG files from the input directory: + +
    +
    +{% highlight scala %} + +spark.read.format("binaryFile").option("pathGlobFilter", "*.png").load("/path/to/data") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +spark.read().format("binaryFile").option("pathGlobFilter", "*.png").load("/path/to/data"); + +{% endhighlight %} +
    +
    +{% highlight python %} + +spark.read.format("binaryFile").option("pathGlobFilter", "*.png").load("/path/to/data") + +{% endhighlight %} +
    +
    +{% highlight r %} + +read.df("/path/to/data", source = "binaryFile", pathGlobFilter = "*.png") + +{% endhighlight %} +
    +
    + +Binary file data source does not support writing a DataFrame back to the original files. diff --git a/docs/sql-data-sources-hive-tables.md b/docs/sql-data-sources-hive-tables.md new file mode 100644 index 0000000000000..5688011514e12 --- /dev/null +++ b/docs/sql-data-sources-hive-tables.md @@ -0,0 +1,181 @@ +--- +layout: global +title: Hive Tables +displayTitle: Hive Tables +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. +--- + +* Table of contents +{:toc} + +Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). +However, since Hive has a large number of dependencies, these dependencies are not included in the +default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them +automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as +they will need access to the Hive serialization and deserialization libraries (SerDes) in order to +access data stored in Hive. + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), +and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. + +When working with Hive, one must instantiate `SparkSession` with Hive support, including +connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. +Users who do not have an existing Hive deployment can still enable Hive support. When not configured +by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and +creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory +`spark-warehouse` in the current directory that the Spark application is started. Note that +the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. +Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. +You may need to grant write privilege to the user who starts the Spark application. + +
    + +
    +{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} +
    + +
    +{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} +
    + +
    +{% include_example spark_hive python/sql/hive.py %} +
    + +
    + +When working with Hive one must instantiate `SparkSession` with Hive support. This +adds support for finding tables in the MetaStore and writing queries using HiveQL. + +{% include_example spark_hive r/RSparkSQLExample.R %} + +
    +
    + +### Specifying storage format for Hive tables + +When you create a Hive table, you need to define how this table should read/write data from/to file system, +i.e. the "input format" and "output format". You also need to define how this table should deserialize the data +to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage +format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. +By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when +creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    fileFormat + A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and + "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. +
    inputFormat, outputFormat + These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, + e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in a pair, and you can not + specify them if you already specified the `fileFormat` option. +
    serde + This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option + if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" + don't include the serde information and you can use this option with these 3 fileFormats. +
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim + These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. +
    + +All other properties defined with `OPTIONS` will be regarded as Hive serde properties. + +### Interacting with Different Versions of Hive Metastore + +One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, +which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary +build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. +Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL +will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). + +The following options can be used to configure the version of Hive that is used to retrieve metadata: + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 + Version of the Hive metastore. Available + options are 0.12.0 through 2.3.5 and 3.0.0 through 3.1.1. +
    spark.sql.hive.metastore.jarsbuiltin + Location of the jars that should be used to instantiate the HiveMetastoreClient. This + property can be one of three options: +
      +
    1. builtin
    2. + Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is + enabled. When this option is chosen, spark.sql.hive.metastore.version must be + either 1.2.1 or not defined. +
    3. maven
    4. + Use Hive jars of specified version downloaded from Maven repositories. This configuration + is not generally recommended for production deployments. +
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive + and its dependencies, including the correct version of Hadoop. These jars only need to be + present on the driver, but if you are running in yarn cluster mode then you must ensure + they are packaged with your application.
    6. +
    +
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    +

    + A comma-separated list of class prefixes that should be loaded using the classloader that is + shared between Spark SQL and a specific version of Hive. An example of classes 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. +

    +
    spark.sql.hive.metastore.barrierPrefixes(empty) +

    + 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.*). +

    +
    diff --git a/docs/sql-data-sources-jdbc.md b/docs/sql-data-sources-jdbc.md new file mode 100644 index 0000000000000..3119ec004b2a1 --- /dev/null +++ b/docs/sql-data-sources-jdbc.md @@ -0,0 +1,238 @@ +--- +layout: global +title: JDBC To Other Databases +displayTitle: JDBC To Other Databases +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. +--- + +* Table of contents +{:toc} + +Spark SQL also includes a data source that can read data from other databases using JDBC. This +functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). +This is because the results are returned +as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. +The JDBC data source is also easier to use from Java or Python as it does not require the user to +provide a ClassTag. +(Note that this is different than the Spark SQL JDBC server, which allows other applications to +run queries using Spark SQL). + +To get started you will need to include the JDBC driver for your particular database on the +spark classpath. For example, to connect to postgres from the Spark Shell you would run the +following command: + +{% highlight bash %} +bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar +{% endhighlight %} + +Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using +the Data Sources API. Users can specify the JDBC connection properties in the data source options. +user and password are normally provided as connection properties for +logging into the data sources. In addition to the connection properties, Spark also supports +the following case-insensitive options: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameMeaning
    url + The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret +
    dbtable + The JDBC table that should be read from or written into. Note that when using it in the read + path anything that is valid in a FROM clause of a SQL query can be used. + For example, instead of a full table you could also use a subquery in parentheses. It is not + allowed to specify `dbtable` and `query` options at the same time. +
    query + A query that will be used to read data into Spark. The specified query will be parenthesized and used + as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. + As an example, spark will issue a query of the following form to the JDBC Source.

    + SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    + Below are a couple of restrictions while using this option.
    +
      +
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. +
    3. It is not allowed to specify `query` and `partitionColumn` options at the same time. When specifying + `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and + partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      + Example:
      + + spark.read.format("jdbc")
      + .option("url", jdbcUrl)
      + .option("query", "select c1, c2 from t1")
      + .load() +
    4. +
    +
    driver + The class name of the JDBC driver to use to connect to this URL. +
    partitionColumn, lowerBound, upperBound + These options must all be specified if any of them is specified. In addition, + numPartitions must be specified. They describe how to partition the table when + reading in parallel from multiple workers. + partitionColumn must be a numeric, date, or timestamp column from the table in question. + Notice that lowerBound and upperBound are just used to decide the + partition stride, not for filtering the rows in table. So all rows in the table will be + partitioned and returned. This option applies only to reading. +
    numPartitions + The maximum number of partitions that can be used for parallelism in table reading and + writing. This also determines the maximum number of concurrent JDBC connections. + If the number of partitions to write exceeds this limit, we decrease it to this limit by + calling coalesce(numPartitions) before writing. +
    queryTimeout + The number of seconds the driver will wait for a Statement object to execute to the given + number of seconds. Zero means there is no limit. In the write path, this option depends on + how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver + checks the timeout of each query instead of an entire JDBC batch. + It defaults to 0. +
    fetchsize + The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. +
    batchsize + The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. +
    isolationLevel + The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. +
    sessionInitStatement + After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") +
    truncate + This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. +
    cascadeTruncate + This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. +
    createTableOptions + This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. +
    createTableColumnTypes + The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. +
    customSchema + The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. +
    pushDownPredicate + The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. +
    + +
    + +
    +{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example jdbc_dataset python/sql/datasource.py %} +
    + +
    +{% include_example jdbc_dataset r/RSparkSQLExample.R %} +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jdbcTable +USING org.apache.spark.sql.jdbc +OPTIONS ( + url "jdbc:postgresql:dbserver", + dbtable "schema.tablename", + user 'username', + password 'password' +) + +INSERT INTO TABLE jdbcTable +SELECT * FROM resultTable +{% endhighlight %} + +
    +
    diff --git a/docs/sql-data-sources-json.md b/docs/sql-data-sources-json.md new file mode 100644 index 0000000000000..4ce4897189846 --- /dev/null +++ b/docs/sql-data-sources-json.md @@ -0,0 +1,96 @@ +--- +layout: global +title: JSON Files +displayTitle: JSON Files +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 can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. +This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. +This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, +or a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` option to `true`. + +{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. +This conversion can be done using `SparkSession.read.json` on a JSON file. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set the `multiLine` parameter to `True`. + +{% include_example json_dataset python/sql/datasource.py %} +
    + +
    +Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using +the `read.json()` function, which loads data from a directory of JSON files where each line of the +files is a JSON object. + +Note that the file that is offered as _a json file_ is not a typical JSON file. Each +line must contain a separate, self-contained valid JSON object. For more information, please see +[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). + +For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. + +{% include_example json_dataset r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW jsonTable +USING org.apache.spark.sql.json +OPTIONS ( + path "examples/src/main/resources/people.json" +) + +SELECT * FROM jsonTable + +{% endhighlight %} + +
    + +
    \ No newline at end of file diff --git a/docs/sql-data-sources-load-save-functions.md b/docs/sql-data-sources-load-save-functions.md new file mode 100644 index 0000000000000..07482137a28a3 --- /dev/null +++ b/docs/sql-data-sources-load-save-functions.md @@ -0,0 +1,368 @@ +--- +layout: global +title: Generic Load/Save Functions +displayTitle: Generic Load/Save 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. +--- + +* Table of contents +{:toc} + + +In the simplest form, the default data source (`parquet` unless otherwise configured by +`spark.sql.sources.default`) will be used for all operations. + + +
    +
    +{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example generic_load_save_functions python/sql/datasource.py %} +
    + +
    + +{% include_example generic_load_save_functions r/RSparkSQLExample.R %} + +
    +
    + +### Manually Specifying Options + +You can also manually specify the data source that will be used along with any extra options +that you would like to pass to the data source. Data sources are specified by their fully qualified +name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short +names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data +source type can be converted into other types using this syntax. + +Please refer the API documentation for available options of built-in sources, for example, +`org.apache.spark.sql.DataFrameReader` and `org.apache.spark.sql.DataFrameWriter`. The +options documented there should be applicable through non-Scala Spark APIs (e.g. PySpark) +as well. For other formats, refer to the API documentation of the particular format. + +To load a JSON file you can use: + +
    +
    +{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options r/RSparkSQLExample.R %} +
    +
    + +To load a CSV file you can use: + +
    +
    +{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_load_options_csv python/sql/datasource.py %} +
    + +
    +{% include_example manual_load_options_csv r/RSparkSQLExample.R %} + +
    +
    + +To load files with paths matching a given glob pattern while keeping the behavior of partition discovery, +you can use: + +
    +
    +{% include_example load_with_path_glob_filter scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example load_with_path_glob_filter java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example load_with_path_glob_filter python/sql/datasource.py %} +
    + +
    +{% include_example load_with_path_glob_filter r/RSparkSQLExample.R %} +
    +
    + +The extra options are also used during write operation. +For example, you can control bloom filters and dictionary encodings for ORC data sources. +The following ORC example will create bloom filter and use dictionary encoding only for `favorite_color`. +For Parquet, there exists `parquet.enable.dictionary`, too. +To find more detailed information about the extra ORC/Parquet options, +visit the official Apache ORC/Parquet websites. + +
    + +
    +{% include_example manual_save_options_orc scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example manual_save_options_orc java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example manual_save_options_orc python/sql/datasource.py %} +
    + +
    +{% include_example manual_save_options_orc r/RSparkSQLExample.R %} +
    + +
    + +{% highlight sql %} +CREATE TABLE users_with_options ( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING ORC +OPTIONS ( + orc.bloom.filter.columns 'favorite_color', + orc.dictionary.key.threshold '1.0', + orc.column.encoding.direct 'name' +) +{% endhighlight %} + +
    + +
    + +### Run SQL on files directly + +Instead of using read API to load a file into DataFrame and query it, you can also query that +file directly with SQL. + +
    +
    +{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example direct_sql python/sql/datasource.py %} +
    + +
    +{% include_example direct_sql r/RSparkSQLExample.R %} + +
    +
    + +### Save Modes + +Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if +present. It is important to realize that these save modes do not utilize any locking and are not +atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the +new data. + + + + + + + + + + + + + + + + + + + + + + + +
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) + When saving a DataFrame to a data source, if data already exists, + an exception is expected to be thrown. +
    SaveMode.Append"append" + When saving a DataFrame to a data source, if data/table already exists, + contents of the DataFrame are expected to be appended to existing data. +
    SaveMode.Overwrite"overwrite" + Overwrite mode means that when saving a DataFrame to a data source, + if data/table already exists, existing data is expected to be overwritten by the contents of + the DataFrame. +
    SaveMode.Ignore"ignore" + Ignore mode means that when saving a DataFrame to a data source, if data already exists, + the save operation is expected not to save the contents of the DataFrame and not to + change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. +
    + +### Saving to Persistent Tables + +`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` +command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a +default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, +`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the +Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as +long as you maintain your connection to the same metastore. A DataFrame for a persistent table can +be created by calling the `table` method on a `SparkSession` with the name of the table. + +For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the +`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, +the custom table path will not be removed and the table data is still there. If no custom table path is +specified, Spark will write data to a default table path under the warehouse directory. When the table is +dropped, the default table path will be removed too. + +Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: + +- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. +- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + +Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. + +### Bucketing, Sorting and Partitioning + +For file-based data source, it is also possible to bucket and sort or partition the output. +Bucketing and sorting are applicable only to persistent tables: + +
    + +
    +{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_sorting_and_bucketing python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_by_name( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +CLUSTERED BY(name) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. + + +
    + +
    +{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partitioning python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_by_favorite_color( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING csv PARTITIONED BY(favorite_color); + +{% endhighlight %} + +
    + +
    + +It is possible to use both partitioning and bucketing for a single table: + +
    + +
    +{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    +{% include_example write_partition_and_bucket python/sql/datasource.py %} +
    + +
    + +{% highlight sql %} + +CREATE TABLE users_bucketed_and_partitioned( + name STRING, + favorite_color STRING, + favorite_numbers array +) USING parquet +PARTITIONED BY (favorite_color) +CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; + +{% endhighlight %} + +
    + +
    + +`partitionBy` creates a directory structure as described in the [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) section. +Thus, it has limited applicability to columns with high cardinality. In contrast + `bucketBy` distributes +data across a fixed number of buckets and can be used when the number of unique values is unbounded. diff --git a/docs/sql-data-sources-orc.md b/docs/sql-data-sources-orc.md new file mode 100644 index 0000000000000..45bff17c6cf2b --- /dev/null +++ b/docs/sql-data-sources-orc.md @@ -0,0 +1,41 @@ +--- +layout: global +title: ORC Files +displayTitle: ORC Files +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. +--- + +Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. +To do that, the following configurations are newly added. The vectorized reader is used for the +native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` +is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC +serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), +the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables 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.
    diff --git a/docs/sql-data-sources-parquet.md b/docs/sql-data-sources-parquet.md new file mode 100644 index 0000000000000..b5309870f485b --- /dev/null +++ b/docs/sql-data-sources-parquet.md @@ -0,0 +1,325 @@ +--- +layout: global +title: Parquet Files +displayTitle: Parquet Files +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. +--- + +* Table of contents +{:toc} + +[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. +Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema +of the original data. When reading Parquet files, all columns are automatically converted to be nullable for +compatibility reasons. + +### Loading Data Programmatically + +Using the data from the above example: + +
    + +
    +{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example basic_parquet_example python/sql/datasource.py %} +
    + +
    + +{% include_example basic_parquet_example r/RSparkSQLExample.R %} + +
    + +
    + +{% highlight sql %} + +CREATE TEMPORARY VIEW parquetTable +USING org.apache.spark.sql.parquet +OPTIONS ( + path "examples/src/main/resources/people.parquet" +) + +SELECT * FROM parquetTable + +{% endhighlight %} + +
    + +
    + +### Partition Discovery + +Table partitioning is a common optimization approach used in systems like Hive. In a partitioned +table, data are usually stored in different directories, with partitioning column values encoded in +the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) +are able to discover and infer partitioning information automatically. +For example, we can store all our previously used +population data into a partitioned table using the following directory structure, with two extra +columns, `gender` and `country` as partitioning columns: + +{% highlight text %} + +path +└── to + └── table + ├── gender=male + │   ├── ... + │   │ + │   ├── country=US + │   │   └── data.parquet + │   ├── country=CN + │   │   └── data.parquet + │   └── ... + └── gender=female +    ├── ... +    │ +    ├── country=US +    │   └── data.parquet +    ├── country=CN +    │   └── data.parquet +    └── ... + +{% endhighlight %} + +By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL +will automatically extract the partitioning information from the paths. +Now the schema of the returned DataFrame becomes: + +{% highlight text %} + +root +|-- name: string (nullable = true) +|-- age: long (nullable = true) +|-- gender: string (nullable = true) +|-- country: string (nullable = true) + +{% endhighlight %} + +Notice that the data types of the partitioning columns are automatically inferred. Currently, +numeric data types, date, timestamp and string type are supported. Sometimes users may not want +to automatically infer the data types of the partitioning columns. For these use cases, the +automatic type inference can be configured by +`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type +inference is disabled, string type will be used for the partitioning columns. + +Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths +by default. For the above example, if users pass `path/to/table/gender=male` to either +`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a +partitioning column. If users need to specify the base path that partition discovery +should start with, they can set `basePath` in the data source options. For example, +when `path/to/table/gender=male` is the path of the data and +users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. + +### Schema Merging + +Like Protocol Buffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with +a simple schema, and gradually add more columns to the schema as needed. In this way, users may end +up with multiple Parquet files with different but mutually compatible schemas. The Parquet data +source is now able to automatically detect this case and merge schemas of all these files. + +Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we +turned it off by default starting from 1.5.0. You may enable it by + +1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the + examples below), or +2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. + +
    + +
    +{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} +
    + +
    +{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} +
    + +
    + +{% include_example schema_merging python/sql/datasource.py %} +
    + +
    + +{% include_example schema_merging r/RSparkSQLExample.R %} + +
    + +
    + +### Hive metastore Parquet table conversion + +When reading from Hive metastore Parquet tables and writing to non-partitioned Hive metastore +Parquet tables, Spark SQL will try to use its own Parquet support instead of Hive SerDe for +better performance. This behavior is controlled by the `spark.sql.hive.convertMetastoreParquet` +configuration, and is turned on by default. + +#### Hive/Parquet Schema Reconciliation + +There are two key differences between Hive and Parquet from the perspective of table schema +processing. + +1. Hive is case insensitive, while Parquet is not +1. Hive considers all columns nullable, while nullability in Parquet is significant + +Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a +Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: + +1. Fields that have the same name in both schema must have the same data type regardless of + nullability. The reconciled field should have the data type of the Parquet side, so that + nullability is respected. + +1. The reconciled schema contains exactly those fields defined in Hive metastore schema. + + - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. + - Any fields that only appear in the Hive metastore schema are added as nullable field in the + reconciled schema. + +#### Metadata Refreshing + +Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table +conversion is enabled, metadata of those converted tables are also cached. If these tables are +updated by Hive or other external tools, you need to refresh them manually to ensure consistent +metadata. + +
    + +
    + +{% highlight scala %} +// spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight java %} +// spark is an existing SparkSession +spark.catalog().refreshTable("my_table"); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +# spark is an existing SparkSession +spark.catalog.refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight r %} +refreshTable("my_table") +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +REFRESH TABLE my_table; +{% endhighlight %} + +
    + +
    + +### Configuration + +Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse + Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do + 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. +
    spark.sql.parquet.int96AsTimestamptrue + 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. +
    spark.sql.parquet.compression.codecsnappy + Sets the compression codec used when writing Parquet files. If either `compression` or + `parquet.compression` is specified in the table-specific options/properties, the precedence would be + `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: + none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. + Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires + `BrotliCodec` to be installed. +
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue + When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in + support. +
    spark.sql.parquet.mergeSchemafalse +

    + When true, the Parquet data source merges schemas collected from all data files, otherwise the + schema is picked from the summary file or a random data file if no summary file is available. +

    +
    spark.sql.parquet.writeLegacyFormatfalse + If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values + will be written in Apache Parquet's fixed-length byte array format, which other systems such as + Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For + 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. +
    diff --git a/docs/sql-data-sources-troubleshooting.md b/docs/sql-data-sources-troubleshooting.md new file mode 100644 index 0000000000000..993d9bf686f36 --- /dev/null +++ b/docs/sql-data-sources-troubleshooting.md @@ -0,0 +1,24 @@ +--- +layout: global +title: Troubleshooting +displayTitle: Troubleshooting +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. +--- + + * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. + * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. + * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. diff --git a/docs/sql-data-sources.md b/docs/sql-data-sources.md new file mode 100644 index 0000000000000..079c54060d15d --- /dev/null +++ b/docs/sql-data-sources.md @@ -0,0 +1,58 @@ +--- +layout: global +title: Data Sources +displayTitle: Data Sources +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 supports operating on a variety of data sources through the DataFrame interface. +A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. +Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section +describes the general methods for loading and saving data using the Spark Data Sources and then +goes into specific options that are available for the built-in data sources. + + +* [Generic Load/Save Functions](sql-data-sources-load-save-functions.html) + * [Manually Specifying Options](sql-data-sources-load-save-functions.html#manually-specifying-options) + * [Run SQL on files directly](sql-data-sources-load-save-functions.html#run-sql-on-files-directly) + * [Save Modes](sql-data-sources-load-save-functions.html#save-modes) + * [Saving to Persistent Tables](sql-data-sources-load-save-functions.html#saving-to-persistent-tables) + * [Bucketing, Sorting and Partitioning](sql-data-sources-load-save-functions.html#bucketing-sorting-and-partitioning) +* [Parquet Files](sql-data-sources-parquet.html) + * [Loading Data Programmatically](sql-data-sources-parquet.html#loading-data-programmatically) + * [Partition Discovery](sql-data-sources-parquet.html#partition-discovery) + * [Schema Merging](sql-data-sources-parquet.html#schema-merging) + * [Hive metastore Parquet table conversion](sql-data-sources-parquet.html#hive-metastore-parquet-table-conversion) + * [Configuration](sql-data-sources-parquet.html#configuration) +* [ORC Files](sql-data-sources-orc.html) +* [JSON Files](sql-data-sources-json.html) +* [Hive Tables](sql-data-sources-hive-tables.html) + * [Specifying storage format for Hive tables](sql-data-sources-hive-tables.html#specifying-storage-format-for-hive-tables) + * [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore) +* [JDBC To Other Databases](sql-data-sources-jdbc.html) +* [Avro Files](sql-data-sources-avro.html) + * [Deploying](sql-data-sources-avro.html#deploying) + * [Load and Save Functions](sql-data-sources-avro.html#load-and-save-functions) + * [to_avro() and from_avro()](sql-data-sources-avro.html#to_avro-and-from_avro) + * [Data Source Option](sql-data-sources-avro.html#data-source-option) + * [Configuration](sql-data-sources-avro.html#configuration) + * [Compatibility with Databricks spark-avro](sql-data-sources-avro.html#compatibility-with-databricks-spark-avro) + * [Supported types for Avro -> Spark SQL conversion](sql-data-sources-avro.html#supported-types-for-avro---spark-sql-conversion) + * [Supported types for Spark SQL -> Avro conversion](sql-data-sources-avro.html#supported-types-for-spark-sql---avro-conversion) +* [Whole Binary Files](sql-data-sources-binaryFile.html) +* [Troubleshooting](sql-data-sources-troubleshooting.html) diff --git a/docs/sql-distributed-sql-engine.md b/docs/sql-distributed-sql-engine.md new file mode 100644 index 0000000000000..fc849d3912b98 --- /dev/null +++ b/docs/sql-distributed-sql-engine.md @@ -0,0 +1,99 @@ +--- +layout: global +title: Distributed SQL Engine +displayTitle: Distributed SQL Engine +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. +--- + +* Table of contents +{:toc} + +Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. +In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, +without the need to write any code. + +## Running the Thrift JDBC/ODBC server + +The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) +in Hive 1.2.1. You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. + +To start the JDBC/ODBC server, run the following in the Spark directory: + + ./sbin/start-thriftserver.sh + +This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to +specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of +all available options. By default, the server listens on localhost:10000. You may override this +behaviour via either environment variables, i.e.: + +{% highlight bash %} +export HIVE_SERVER2_THRIFT_PORT= +export HIVE_SERVER2_THRIFT_BIND_HOST= +./sbin/start-thriftserver.sh \ + --master \ + ... +{% endhighlight %} + +or system properties: + +{% highlight bash %} +./sbin/start-thriftserver.sh \ + --hiveconf hive.server2.thrift.port= \ + --hiveconf hive.server2.thrift.bind.host= \ + --master + ... +{% endhighlight %} + +Now you can use beeline to test the Thrift JDBC/ODBC server: + + ./bin/beeline + +Connect to the JDBC/ODBC server in beeline with: + + beeline> !connect jdbc:hive2://localhost:10000 + +Beeline will ask you for a username and password. In non-secure mode, simply enter the username on +your machine and a blank password. For secure mode, please follow the instructions given in the +[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. + +You may also use the beeline script that comes with Hive. + +Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. +Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: + + hive.server2.transport.mode - Set this to value: http + hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 + hive.server2.http.endpoint - HTTP endpoint; default is cliservice + +To test, use beeline to connect to the JDBC/ODBC server in http mode with: + + beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= + + +## Running the Spark SQL CLI + +The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute +queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. + +To start the Spark SQL CLI, run the following in the Spark directory: + + ./bin/spark-sql + +Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. +You may run `./bin/spark-sql --help` for a complete list of all available options. diff --git a/docs/sql-getting-started.md b/docs/sql-getting-started.md new file mode 100644 index 0000000000000..5d18c48879f93 --- /dev/null +++ b/docs/sql-getting-started.md @@ -0,0 +1,384 @@ +--- +layout: global +title: Getting Started +displayTitle: Getting Started +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. +--- + +* Table of contents +{:toc} + +## Starting Point: SparkSession + +
    +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: + +{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: + +{% include_example init_session python/sql/basic.py %} +
    + +
    + +The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: + +{% include_example init_session r/RSparkSQLExample.R %} + +Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. +
    +
    + +`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to +write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. +To use these features, you do not need to have an existing Hive setup. + +## Creating DataFrames + +
    +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df python/sql/basic.py %} +
    + +
    +With a `SparkSession`, applications can create DataFrames from a local R data.frame, +from a Hive table, or from [Spark data sources](sql-data-sources.html). + +As an example, the following creates a DataFrame based on the content of a JSON file: + +{% include_example create_df r/RSparkSQLExample.R %} + +
    +
    + + +## Untyped Dataset Operations (aka DataFrame Operations) + +DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). + +As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. + +Here we include some basic examples of structured data processing using Datasets: + +
    +
    +{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + +For a complete list of the types of operations that can be performed on a Dataset, refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). +
    + +
    + +{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} + +For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). + +In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). +
    + +
    +In Python, it's possible to access a DataFrame's columns either by attribute +(`df.age`) or by indexing (`df['age']`). While the former is convenient for +interactive data exploration, users are highly encouraged to use the +latter form, which is future proof and won't break with column names that +are also attributes on the DataFrame class. + +{% include_example untyped_ops python/sql/basic.py %} +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). + +
    + +
    + +{% include_example untyped_ops r/RSparkSQLExample.R %} + +For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). + +In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). + +
    + +
    + +## Running SQL Queries Programmatically + +
    +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. + +{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. + +{% include_example run_sql python/sql/basic.py %} +
    + +
    +The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. + +{% include_example run_sql r/RSparkSQLExample.R %} + +
    +
    + + +## Global Temporary View + +Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it +terminates. If you want to have a temporary view that is shared among all sessions and keep alive +until the Spark application terminates, you can create a global temporary view. Global temporary +view is tied to a system preserved database `global_temp`, and we must use the qualified name to +refer it, e.g. `SELECT * FROM global_temp.view1`. + +
    +
    +{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    +{% include_example global_temp_view python/sql/basic.py %} +
    + +
    + +{% highlight sql %} + +CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl + +SELECT * FROM global_temp.temp_view + +{% endhighlight %} + +
    +
    + + +## Creating Datasets + +Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use +a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects +for processing or transmitting over the network. While both encoders and standard serialization are +responsible for turning an object into bytes, encoders are code generated dynamically and use a format +that allows Spark to perform many operations like filtering, sorting and hashing without deserializing +the bytes back into an object. + +
    +
    +{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    +{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    +
    + +## Interoperating with RDDs + +Spark SQL supports two different methods for converting existing RDDs into Datasets. The first +method uses reflection to infer the schema of an RDD that contains specific types of objects. This +reflection-based approach leads to more concise code and works well when you already know the schema +while writing your Spark application. + +The second method for creating Datasets is through a programmatic interface that allows you to +construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows +you to construct Datasets when the columns and their types are not known until runtime. + +### Inferring the Schema Using Reflection +
    + +
    + +The Scala interface for Spark SQL supports automatically converting an RDD containing case classes +to a DataFrame. The case class +defines the schema of the table. The names of the arguments to the case class are read using +reflection and become the names of the columns. Case classes can also be nested or contain complex +types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be +registered as a table. Tables can be used in subsequent SQL statements. + +{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +Spark SQL supports automatically converting an RDD of +[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. +The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL +does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` +fields are supported though. You can create a JavaBean by creating a class that implements +Serializable and has getters and setters for all of its fields. + +{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of +key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, +and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. + +{% include_example schema_inferring python/sql/basic.py %} +
    + +
    + +### Programmatically Specifying the Schema + +
    + +
    + +When case classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed +and fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} +
    + +
    + +When JavaBean classes cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `Dataset` can be created programmatically with three steps. + +1. Create an RDD of `Row`s from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +`Row`s in the RDD created in Step 1. +3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided +by `SparkSession`. + +For example: + +{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} +
    + +
    + +When a dictionary of kwargs cannot be defined ahead of time (for example, +the structure of records is encoded in a string, or a text dataset will be parsed and +fields will be projected differently for different users), +a `DataFrame` can be created programmatically with three steps. + +1. Create an RDD of tuples or lists from the original RDD; +2. Create the schema represented by a `StructType` matching the structure of +tuples or lists in the RDD created in the step 1. +3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. + +For example: + +{% include_example programmatic_schema python/sql/basic.py %} +
    + +
    + +## Aggregations + +The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common +aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. +While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in +[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and +[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. +Moreover, users are not limited to the predefined aggregate functions and can create their own. + +### Untyped User-Defined Aggregate Functions +Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) +abstract class to implement a custom untyped aggregate function. For example, a user-defined average +can look like: + +
    +
    +{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} +
    +
    +{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} +
    +
    + +### Type-Safe User-Defined Aggregate Functions + +User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. +For example, a type-safe user-defined average can look like: + +
    +
    +{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} +
    +
    +{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} +
    +
    diff --git a/docs/sql-keywords.md b/docs/sql-keywords.md new file mode 100644 index 0000000000000..13058cba7564f --- /dev/null +++ b/docs/sql-keywords.md @@ -0,0 +1,294 @@ +--- +layout: global +title: Spark SQL Keywords +displayTitle: Spark SQL Keywords +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. +--- + +When `spark.sql.parser.ansi.enabled` is true, Spark SQL has two kinds of keywords: +* Reserved keywords: Keywords that are reserved and can't be used as identifiers for table, view, column, function, alias, etc. +* Non-reserved keywords: Keywords that have a special meaning only in particular contexts and can be used as identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK can be used as identifiers in other places. + +When `spark.sql.parser.ansi.enabled` is false, Spark SQL has two kinds of keywords: +* Non-reserved keywords: Same definition as the one when `spark.sql.parser.ansi.enabled=true`. +* Strict-non-reserved keywords: A strict version of non-reserved keywords, which can not be used as table alias. + +By default `spark.sql.parser.ansi.enabled` is false. + +Below is a list of all the keywords in Spark SQL. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    KeywordSpark SQLSQL-2011
    ANSI modedefault mode
    ADDnon-reservednon-reservednon-reserved
    AFTERnon-reservednon-reservednon-reserved
    ALLreservednon-reservedreserved
    ALTERnon-reservednon-reservedreserved
    ANALYZEnon-reservednon-reservednon-reserved
    ANDreservednon-reservedreserved
    ANTIreservedstrict-non-reservednon-reserved
    ANYreservednon-reservedreserved
    ARCHIVEnon-reservednon-reservednon-reserved
    ARRAYnon-reservednon-reservedreserved
    ASreservednon-reservedreserved
    ASCnon-reservednon-reservednon-reserved
    ATnon-reservednon-reservedreserved
    AUTHORIZATIONreservednon-reservedreserved
    BETWEENnon-reservednon-reservedreserved
    BOTHreservednon-reservedreserved
    BUCKETnon-reservednon-reservednon-reserved
    BUCKETSnon-reservednon-reservednon-reserved
    BYnon-reservednon-reservedreserved
    CACHEnon-reservednon-reservednon-reserved
    CASCADEnon-reservednon-reservedreserved
    CASEreservednon-reservedreserved
    CASTreservednon-reservedreserved
    CHANGEnon-reservednon-reservednon-reserved
    CHECKreservednon-reservedreserved
    CLEARnon-reservednon-reservednon-reserved
    CLUSTERnon-reservednon-reservednon-reserved
    CLUSTEREDnon-reservednon-reservednon-reserved
    CODEGENnon-reservednon-reservednon-reserved
    COLLATEreservednon-reservedreserved
    COLLECTIONnon-reservednon-reservednon-reserved
    COLUMNreservednon-reservedreserved
    COLUMNSnon-reservednon-reservednon-reserved
    COMMENTnon-reservednon-reservednon-reserved
    COMMITnon-reservednon-reservedreserved
    COMPACTnon-reservednon-reservednon-reserved
    COMPACTIONSnon-reservednon-reservednon-reserved
    COMPUTEnon-reservednon-reservednon-reserved
    CONCATENATEnon-reservednon-reservednon-reserved
    CONSTRAINTreservednon-reservedreserved
    COSTnon-reservednon-reservednon-reserved
    CREATEreservednon-reservedreserved
    CROSSreservedstrict-non-reservedreserved
    CUBEnon-reservednon-reservedreserved
    CURRENTnon-reservednon-reservedreserved
    CURRENT_DATEreservednon-reservedreserved
    CURRENT_TIMEreservednon-reservedreserved
    CURRENT_TIMESTAMPreservednon-reservedreserved
    CURRENT_USERreservednon-reservedreserved
    DATAnon-reservednon-reservednon-reserved
    DATABASEnon-reservednon-reservednon-reserved
    DATABASESnon-reservednon-reservednon-reserved
    DAYreservednon-reservedreserved
    DAYSnon-reservednon-reservednon-reserved
    DBPROPERTIESnon-reservednon-reservednon-reserved
    DEFINEDnon-reservednon-reservednon-reserved
    DELETEnon-reservednon-reservedreserved
    DELIMITEDnon-reservednon-reservednon-reserved
    DESCnon-reservednon-reservednon-reserved
    DESCRIBEnon-reservednon-reservedreserved
    DFSnon-reservednon-reservednon-reserved
    DIRECTORIESnon-reservednon-reservednon-reserved
    DIRECTORYnon-reservednon-reservednon-reserved
    DISTINCTreservednon-reservedreserved
    DISTRIBUTEnon-reservednon-reservednon-reserved
    DIVnon-reservednon-reservednon-reserved
    DROPnon-reservednon-reservedreserved
    ELSEreservednon-reservedreserved
    ENDreservednon-reservedreserved
    ESCAPEDnon-reservednon-reservednon-reserved
    EXCEPTreservedstrict-non-reservedreserved
    EXCHANGEnon-reservednon-reservednon-reserved
    EXISTSnon-reservednon-reservedreserved
    EXPLAINnon-reservednon-reservednon-reserved
    EXPORTnon-reservednon-reservednon-reserved
    EXTENDEDnon-reservednon-reservednon-reserved
    EXTERNALnon-reservednon-reservedreserved
    EXTRACTnon-reservednon-reservedreserved
    FALSEreservednon-reservedreserved
    FETCHreservednon-reservedreserved
    FIELDSnon-reservednon-reservednon-reserved
    FILEFORMATnon-reservednon-reservednon-reserved
    FIRSTnon-reservednon-reservednon-reserved
    FIRST_VALUEreservednon-reservedreserved
    FOLLOWINGnon-reservednon-reservednon-reserved
    FORreservednon-reservedreserved
    FOREIGNreservednon-reservedreserved
    FORMATnon-reservednon-reservednon-reserved
    FORMATTEDnon-reservednon-reservednon-reserved
    FROMreservednon-reservedreserved
    FULLreservedstrict-non-reservedreserved
    FUNCTIONnon-reservednon-reservedreserved
    FUNCTIONSnon-reservednon-reservednon-reserved
    GLOBALnon-reservednon-reservedreserved
    GRANTreservednon-reservedreserved
    GROUPreservednon-reservedreserved
    GROUPINGnon-reservednon-reservedreserved
    HAVINGreservednon-reservedreserved
    HOURreservednon-reservedreserved
    HOURSnon-reservednon-reservednon-reserved
    IFnon-reservednon-reservedreserved
    IGNOREnon-reservednon-reservednon-reserved
    IMPORTnon-reservednon-reservednon-reserved
    INreservednon-reservedreserved
    INDEXnon-reservednon-reservednon-reserved
    INDEXESnon-reservednon-reservednon-reserved
    INNERreservedstrict-non-reservedreserved
    INPATHnon-reservednon-reservednon-reserved
    INPUTFORMATnon-reservednon-reservednon-reserved
    INSERTnon-reservednon-reservedreserved
    INTERSECTreservedstrict-non-reservedreserved
    INTERVALnon-reservednon-reservedreserved
    INTOreservednon-reservedreserved
    ISreservednon-reservedreserved
    ITEMSnon-reservednon-reservednon-reserved
    JOINreservedstrict-non-reservedreserved
    KEYSnon-reservednon-reservednon-reserved
    LASTnon-reservednon-reservednon-reserved
    LAST_VALUEreservednon-reservedreserved
    LATERALnon-reservednon-reservedreserved
    LAZYnon-reservednon-reservednon-reserved
    LEADINGreservednon-reservedreserved
    LEFTreservedstrict-non-reservedreserved
    LIKEnon-reservednon-reservedreserved
    LIMITnon-reservednon-reservednon-reserved
    LINESnon-reservednon-reservednon-reserved
    LISTnon-reservednon-reservednon-reserved
    LOADnon-reservednon-reservednon-reserved
    LOCALnon-reservednon-reservedreserved
    LOCATIONnon-reservednon-reservednon-reserved
    LOCKnon-reservednon-reservednon-reserved
    LOCKSnon-reservednon-reservednon-reserved
    LOGICALnon-reservednon-reservednon-reserved
    MACROnon-reservednon-reservednon-reserved
    MAPnon-reservednon-reservednon-reserved
    MICROSECONDnon-reservednon-reservednon-reserved
    MICROSECONDSnon-reservednon-reservednon-reserved
    MILLISECONDnon-reservednon-reservednon-reserved
    MILLISECONDSnon-reservednon-reservednon-reserved
    MINUSreservedstrict-non-reservednon-reserved
    MINUTEreservednon-reservedreserved
    MINUTESnon-reservednon-reservednon-reserved
    MONTHreservednon-reservedreserved
    MONTHSnon-reservednon-reservednon-reserved
    MSCKnon-reservednon-reservednon-reserved
    NATURALreservedstrict-non-reservedreserved
    NOnon-reservednon-reservedreserved
    NOTreservednon-reservedreserved
    NULLreservednon-reservedreserved
    NULLSnon-reservednon-reservednon-reserved
    OFnon-reservednon-reservedreserved
    ONreservedstrict-non-reservedreserved
    ONLYreservednon-reservedreserved
    OPTIONnon-reservednon-reservednon-reserved
    OPTIONSnon-reservednon-reservednon-reserved
    ORreservednon-reservedreserved
    ORDERreservednon-reservedreserved
    OUTnon-reservednon-reservedreserved
    OUTERreservednon-reservedreserved
    OUTPUTFORMATnon-reservednon-reservednon-reserved
    OVERnon-reservednon-reservednon-reserved
    OVERLAPSreservednon-reservedreserved
    OVERLAYnon-reservednon-reservednon-reserved
    OVERWRITEnon-reservednon-reservednon-reserved
    PARTITIONnon-reservednon-reservedreserved
    PARTITIONEDnon-reservednon-reservednon-reserved
    PARTITIONSnon-reservednon-reservednon-reserved
    PERCENTnon-reservednon-reservednon-reserved
    PIVOTnon-reservednon-reservednon-reserved
    PLACINGnon-reservednon-reservednon-reserved
    POSITIONnon-reservednon-reservedreserved
    PRECEDINGnon-reservednon-reservednon-reserved
    PRIMARYreservednon-reservedreserved
    PRINCIPALSnon-reservednon-reservednon-reserved
    PURGEnon-reservednon-reservednon-reserved
    QUERYnon-reservednon-reservednon-reserved
    RANGEnon-reservednon-reservedreserved
    RECORDREADERnon-reservednon-reservednon-reserved
    RECORDWRITERnon-reservednon-reservednon-reserved
    RECOVERnon-reservednon-reservednon-reserved
    REDUCEnon-reservednon-reservednon-reserved
    REFERENCESreservednon-reservedreserved
    REFRESHnon-reservednon-reservednon-reserved
    RENAMEnon-reservednon-reservednon-reserved
    REPAIRnon-reservednon-reservednon-reserved
    REPLACEnon-reservednon-reservednon-reserved
    RESETnon-reservednon-reservednon-reserved
    RESPECTnon-reservednon-reservednon-reserved
    RESTRICTnon-reservednon-reservednon-reserved
    REVOKEnon-reservednon-reservedreserved
    RIGHTreservedstrict-non-reservedreserved
    RLIKEnon-reservednon-reservednon-reserved
    ROLEnon-reservednon-reservednon-reserved
    ROLESnon-reservednon-reservednon-reserved
    ROLLBACKnon-reservednon-reservedreserved
    ROLLUPnon-reservednon-reservedreserved
    ROWnon-reservednon-reservedreserved
    ROWSnon-reservednon-reservedreserved
    SCHEMAnon-reservednon-reservednon-reserved
    SECONDreservednon-reservedreserved
    SECONDSnon-reservednon-reservednon-reserved
    SELECTreservednon-reservedreserved
    SEMIreservedstrict-non-reservednon-reserved
    SEPARATEDnon-reservednon-reservednon-reserved
    SERDEnon-reservednon-reservednon-reserved
    SERDEPROPERTIESnon-reservednon-reservednon-reserved
    SESSION_USERreservednon-reservedreserved
    SETnon-reservednon-reservedreserved
    SETSnon-reservednon-reservednon-reserved
    SHOWnon-reservednon-reservednon-reserved
    SKEWEDnon-reservednon-reservednon-reserved
    SOMEreservednon-reservedreserved
    SORTnon-reservednon-reservednon-reserved
    SORTEDnon-reservednon-reservednon-reserved
    STARTnon-reservednon-reservedreserved
    STATISTICSnon-reservednon-reservednon-reserved
    STOREDnon-reservednon-reservednon-reserved
    STRATIFYnon-reservednon-reservednon-reserved
    STRUCTnon-reservednon-reservednon-reserved
    SUBSTRnon-reservednon-reservednon-reserved
    SUBSTRINGnon-reservednon-reservednon-reserved
    TABLEreservednon-reservedreserved
    TABLESnon-reservednon-reservednon-reserved
    TABLESAMPLEnon-reservednon-reservedreserved
    TBLPROPERTIESnon-reservednon-reservednon-reserved
    TEMPORARYnon-reservednon-reservednon-reserved
    TERMINATEDnon-reservednon-reservednon-reserved
    THENreservednon-reservedreserved
    TOreservednon-reservedreserved
    TOUCHnon-reservednon-reservednon-reserved
    TRAILINGreservednon-reservedreserved
    TRANSACTIONnon-reservednon-reservednon-reserved
    TRANSACTIONSnon-reservednon-reservednon-reserved
    TRANSFORMnon-reservednon-reservednon-reserved
    TRIMnon-reservednon-reservednon-reserved
    TRUEnon-reservednon-reservedreserved
    TRUNCATEnon-reservednon-reservedreserved
    UNARCHIVEnon-reservednon-reservednon-reserved
    UNBOUNDEDnon-reservednon-reservednon-reserved
    UNCACHEnon-reservednon-reservednon-reserved
    UNIONreservedstrict-non-reservedreserved
    UNIQUEreservednon-reservedreserved
    UNLOCKnon-reservednon-reservednon-reserved
    UNSETnon-reservednon-reservednon-reserved
    USEnon-reservednon-reservednon-reserved
    USERreservednon-reservedreserved
    USINGreservedstrict-non-reservedreserved
    VALUESnon-reservednon-reservedreserved
    VIEWnon-reservednon-reservednon-reserved
    WEEKnon-reservednon-reservednon-reserved
    WEEKSnon-reservednon-reservednon-reserved
    WHENreservednon-reservedreserved
    WHEREreservednon-reservedreserved
    WINDOWnon-reservednon-reservedreserved
    WITHreservednon-reservedreserved
    YEARreservednon-reservedreserved
    YEARSnon-reservednon-reservednon-reserved
    diff --git a/docs/sql-migration-guide-hive-compatibility.md b/docs/sql-migration-guide-hive-compatibility.md new file mode 100644 index 0000000000000..f955e31d49a88 --- /dev/null +++ b/docs/sql-migration-guide-hive-compatibility.md @@ -0,0 +1,167 @@ +--- +layout: global +title: Compatibility with Apache Hive +displayTitle: Compatibility with Apache Hive +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. +--- + +* Table of contents +{:toc} + +Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. +Currently, Hive SerDes and UDFs are based on Hive 1.2.1, +and Spark SQL can be connected to different versions of Hive Metastore +(from 0.12.0 to 2.3.5 and 3.0.0 to 3.1.1. Also see [Interacting with Different Versions of Hive Metastore](sql-data-sources-hive-tables.html#interacting-with-different-versions-of-hive-metastore)). + +#### Deploying in Existing Hive Warehouses + +The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive +installations. You do not need to modify your existing Hive Metastore or change the data placement +or partitioning of your tables. + +### Supported Hive Features + +Spark SQL supports the vast majority of Hive features, such as: + +* Hive query statements, including: + * `SELECT` + * `GROUP BY` + * `ORDER BY` + * `CLUSTER BY` + * `SORT BY` +* All Hive operators, including: + * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) + * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) + * Logical operators (`AND`, `&&`, `OR`, `||`, etc) + * Complex type constructors + * Mathematical functions (`sign`, `ln`, `cos`, etc) + * String functions (`instr`, `length`, `printf`, etc) +* User defined functions (UDF) +* User defined aggregation functions (UDAF) +* User defined serialization formats (SerDes) +* Window functions +* Joins + * `JOIN` + * `{LEFT|RIGHT|FULL} OUTER JOIN` + * `LEFT SEMI JOIN` + * `CROSS JOIN` +* Unions +* Sub-queries + * `SELECT col FROM ( SELECT a + b AS col from t1) t2` +* Sampling +* Explain +* Partitioned tables including dynamic partition insertion +* View + * If column aliases are not specified in view definition queries, both Spark and Hive will + generate alias names, but in different ways. In order for Spark to be able to read views created + by Hive, users should explicitly specify column aliases in view definition queries. As an + example, Spark cannot read `v1` created as below by Hive. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 FROM (SELECT 1 c) t1) t2; + ``` + + Instead, you should create `v1` as below with column aliases explicitly specified. + + ``` + CREATE VIEW v1 AS SELECT * FROM (SELECT c + 1 AS inc_c FROM (SELECT 1 c) t1) t2; + ``` + +* All Hive DDL Functions, including: + * `CREATE TABLE` + * `CREATE TABLE AS SELECT` + * `ALTER TABLE` +* Most Hive Data types, including: + * `TINYINT` + * `SMALLINT` + * `INT` + * `BIGINT` + * `BOOLEAN` + * `FLOAT` + * `DOUBLE` + * `STRING` + * `BINARY` + * `TIMESTAMP` + * `DATE` + * `ARRAY<>` + * `MAP<>` + * `STRUCT<>` + +### Unsupported Hive Functionality + +Below is a list of Hive features that we don't support yet. Most of these features are rarely used +in Hive deployments. + +**Major Hive Features** + +* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL + doesn't support buckets yet. + + +**Esoteric Hive Features** + +* `UNION` type +* Unique join +* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at + the moment and only supports populating the sizeInBytes field of the hive metastore. + +**Hive Input/Output Formats** + +* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. +* Hadoop archive + +**Hive Optimizations** + +A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are +less important due to Spark SQL's in-memory computational model. Others are slotted for future +releases of Spark SQL. + +* Block-level bitmap indexes and virtual columns (used to build indexes) +* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you + need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". +* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still + launches tasks to compute the result. +* Skew data flag: Spark SQL does not follow the skew data flags in Hive. +* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. +* Merge multiple small files for query results: if the result output contains multiple small files, + Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS + metadata. Spark SQL does not support that. + +**Hive UDF/UDTF/UDAF** + +Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: + +* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically + include additional resources required by this UDF. +* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses + a deprecated interface `initialize(ObjectInspector[])` only. +* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize + functions with `MapredContext`, which is inapplicable to Spark. +* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. + Spark SQL does not call this function when tasks finish. +* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. + Spark SQL currently does not support the reuse of aggregation. +* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating + an aggregate over a fixed window. + +### Incompatible Hive UDF + +Below are the scenarios in which Hive and Spark generate different results: + +* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. +* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. +* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. diff --git a/docs/sql-migration-guide-upgrade.md b/docs/sql-migration-guide-upgrade.md new file mode 100644 index 0000000000000..3f31c7e6ee25d --- /dev/null +++ b/docs/sql-migration-guide-upgrade.md @@ -0,0 +1,730 @@ +--- +layout: global +title: Spark SQL Upgrading Guide +displayTitle: Spark SQL Upgrading Guide +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. +--- + +* Table of contents +{:toc} + +## Upgrading From Spark SQL 2.4 to 3.0 + - Since Spark 3.0, we reversed argument order of the trim function from `TRIM(trimStr, str)` to `TRIM(str, trimStr)` to be compatible with other databases. + + - 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 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. + + - Since Spark 3.0, the Dataset and DataFrame API `unionAll` is not deprecated any more. It is an alias for `union`. + + - 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 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` and `DoubleType`, 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`. + + - 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)`. + + - 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, users can create map values with map type key via built-in function like `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 still read map values with map type key from data source or Java/Scala collections, though they are not very useful. + + - 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 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 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, these built-in functions will remove duplicated 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 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 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`. + + - 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 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. + + - 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. + + - 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 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.arrowSafeTypeConversion` to true can enable it. The default setting is false. PySpark's behavior for Arrow versions is illustrated in the table below: +
    + + + + + + + + + + + + + + + + + + + + +
    + 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 +
    + + - In Spark version 2.4 and earlier, if `org.apache.spark.sql.functions.udf(Any, DataType)` gets a Scala closure with primitive-type argument, the returned UDF will return null if the input values is null. 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. + + - 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: + + - CSV/JSON datasources use java.time API for parsing and generating CSV/JSON content. In Spark version 2.4 and earlier, java.text.SimpleDateFormat is used for the same purpose with fallbacks to the parsing mechanisms of Spark 2.0 and 1.x. For example, `2018-12-08 10:39:21.123` with the pattern `yyyy-MM-dd'T'HH:mm:ss.SSS` cannot be parsed since Spark 3.0 because the timestamp does not match to the pattern but it can be parsed by earlier Spark versions due to a fallback to `Timestamp.valueOf`. To parse the same timestamp since Spark 3.0, the pattern should be `yyyy-MM-dd HH:mm:ss.SSS`. + + - The `unix_timestamp`, `date_format`, `to_unix_timestamp`, `from_unixtime`, `to_date`, `to_timestamp` functions. New implementation supports pattern formats as described here https://docs.oracle.com/javase/8/docs/api/java/time/format/DateTimeFormatter.html and 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`. + + - 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 earlier, the conversion is based on the hybrid calendar (Julian + Gregorian) and on default system time zone. + + - Formatting of `TIMESTAMP` and `DATE` literals. + + - 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 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 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`. + + - 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 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`. + + - 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. To set `INT96` to `spark.sql.parquet.outputTimestampType` restores the previous behavior. + + - 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`. + + - 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` to `true`. + + - 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. + + - 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. + + - Since Spark 3.0, we use a new protocol for fetching shuffle blocks, for external shuffle service users, we need to upgrade the server correspondingly. Otherwise, we'll get the error message `UnsupportedOperationException: Unexpected message: FetchShuffleBlocks`. If it is hard to upgrade the shuffle service right now, you can still use the old protocol by setting `spark.shuffle.useOldFetchProtocol` to `true`. + + - 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.arrayExistsFollowsThreeValuedLogic` 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. + + - Since Spark 3.0, substitution order of nested WITH clauses is changed and an inner CTE definition takes precedence over an outer. In version 2.4 and earlier, `WITH t AS (SELECT 1), t2 AS (WITH t AS (SELECT 2) SELECT * FROM t) SELECT * FROM t2` returns `1` while in version 3.0 it returns `2`. The previous behaviour can be restored by setting `spark.sql.legacy.ctePrecedence.enabled` to `true`. + + - 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` resultes in `2019-03-31`. + +## Upgrading from Spark SQL 2.4 to 2.4.1 + + - The value of `spark.executor.heartbeatInterval`, when specified without units like "30" rather than "30s", was + inconsistently interpreted as both seconds and milliseconds in Spark 2.4.0 in different parts of the code. + Unitless values are now consistently interpreted as milliseconds. Applications that set values like "30" + need to specify a value with units like "30s" now, to avoid being interpreted as milliseconds; otherwise, + the extremely short interval that results will likely cause applications to fail. + + - When turning a Dataset to another Dataset, Spark will up cast the fields in the original Dataset to the type of corresponding fields in the target DataSet. In version 2.4 and earlier, this up cast is not very strict, e.g. `Seq("str").toDS.as[Int]` fails, but `Seq("str").toDS.as[Boolean]` works and throw NPE during execution. In Spark 3.0, the up cast is stricter and turning String into something else is not allowed, i.e. `Seq("str").toDS.as[Boolean]` will fail during analysis. + +## Upgrading From Spark SQL 2.3 to 2.4 + + - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. + + + + + + + + + + + + + + + + + + + + + + + + + +
    + Query + + Spark 2.3 or Prior + + Spark 2.4 + + Remarks +
    + SELECT array_contains(array(1), 1.34D); + + true + + false + + In Spark 2.4, left and right parameters are promoted to array type of double type and double type respectively. +
    + SELECT array_contains(array(1), '1'); + + true + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + SELECT array_contains(array(1), 'anystring'); + + null + + AnalysisException is thrown. + + Explicit cast can be used in arguments to avoid the exception. In Spark 2.4, AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. +
    + + - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. + + - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. + + - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. + + - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. + + - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. ORC files created by native ORC writer cannot be read by some old Apache Hive releases. Use `spark.sql.orc.impl=hive` to create the files shared with Hive 2.1.1 and older. + + - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. + + - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. + + - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, a column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. + + - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. + + - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. + + - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. + + - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. + + - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. + + - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. + + - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. + + - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. + + - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. + + - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. + + - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was written as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. + + - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. + + - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. + + - In version 2.3 and earlier, when reading from a Parquet data source table, Spark always returns null for any column whose column names in Hive metastore schema and Parquet schema are in different letter cases, no matter whether `spark.sql.caseSensitive` is set to `true` or `false`. Since 2.4, when `spark.sql.caseSensitive` is set to `false`, Spark does case insensitive column name resolution between Hive metastore schema and Parquet schema, so even column names are in different letter cases, Spark returns corresponding column values. An exception is thrown if there is ambiguity, i.e. more than one Parquet column is matched. This change also applies to Parquet Hive tables when `spark.sql.hive.convertMetastoreParquet` is set to `true`. + +## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above + + - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. + +## Upgrading From Spark SQL 2.2 to 2.3 + + - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. + + - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. + + - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. + + - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    + InputA \ InputB + + NullType + + IntegerType + + LongType + + DecimalType(38,0)* + + DoubleType + + DateType + + TimestampType + + StringType +
    + NullType + NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    + IntegerType + IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    + LongType + LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DecimalType(38,0)* + DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    + DoubleType + DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    + DateType + DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    + TimestampType + TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    + StringType + StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    + + Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. + + - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. + + - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. + + - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. + + - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](sql-performance-tuning.html#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). + + - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. + + - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. + + - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes + + - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). + + - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. + + - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. + + - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. + + - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. + + - 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 2.3, the builder comes to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. + +## Upgrading From Spark SQL 2.1 to 2.2 + + - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. + + - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). + + - Since Spark 2.2, view definitions are stored in a different way from prior versions. This may cause Spark unable to read views created by prior versions. In such cases, you need to recreate the views using `ALTER VIEW AS` or `CREATE OR REPLACE VIEW AS` with newer Spark versions. + +## Upgrading From Spark SQL 2.0 to 2.1 + + - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. + + - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. + + - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. + - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. + + - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. + + - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. + +## Upgrading From Spark SQL 1.6 to 2.0 + + - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and + + `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. + + - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for + `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed + transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., + `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in + Python and R is not a language feature, the concept of Dataset does not apply to these languages’ + APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the + single-node data frame notion in these languages. + + - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` + + - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` + + - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` + + - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. + + - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` + in order to prevent accidental dropping the existing data in the user-provided locations. + That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. + Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. + Note that this is different from the Hive behavior. + + - As a result, `DROP TABLE` statements on those tables will not remove the data. + + - `spark.sql.parquet.cacheMetadata` is no longer used. + See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. + +## Upgrading From Spark SQL 1.5 to 1.6 + + - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC + connection owns a copy of their own SQL configuration and temporary function registry. Cached + tables are still shared though. If you prefer to run the Thrift server in the old single-session + mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add + this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: + + {% highlight bash %} + ./sbin/start-thriftserver.sh \ + --conf spark.sql.hive.thriftServer.singleSession=true \ + ... + {% endhighlight %} + + - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns + of the same name of a DataFrame. + + - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This + change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType + from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for + details. + +## Upgrading From Spark SQL 1.4 to 1.5 + + - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with + code generation for expression evaluation. These features can both be disabled by setting + `spark.sql.tungsten.enabled` to `false`. + + - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting + `spark.sql.parquet.mergeSchema` to `true`. + + - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or + access nested values. For example `df['table.column.nestedField']`. However, this means that if + your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). + + - In-memory columnar storage partition pruning is on by default. It can be disabled by setting + `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. + + - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum + precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now + used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. + + - Timestamps are now stored at a precision of 1us, rather than 1ns + + - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains + unchanged. + + - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). + + - JSON data source will not automatically load new files that are created by other applications + (i.e. files that are not inserted to the dataset through Spark SQL). + For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), + users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method + to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate + the DataFrame and the new DataFrame will include new files. + + - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. + +## Upgrading from Spark SQL 1.3 to 1.4 + +#### DataFrame data reader/writer interface + +Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) +and writing data out (`DataFrame.write`), +and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). + +See the API docs for `SQLContext.read` ( + Scala, + Java, + Python +) and `DataFrame.write` ( + Scala, + Java, + Python +) more information. + + +#### DataFrame.groupBy retains grouping columns + +Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the +grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. + +
    +
    +{% highlight scala %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg($"department", max("age"), sum("expense")) + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")) + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    +{% highlight java %} + +// In 1.3.x, in order for the grouping column "department" to show up, +// it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(col("department"), max("age"), sum("expense")); + +// In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(max("age"), sum("expense")); + +// Revert to 1.3 behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false"); + +{% endhighlight %} +
    + +
    +{% highlight python %} + +import pyspark.sql.functions as func + +# In 1.3.x, in order for the grouping column "department" to show up, +# it must be included explicitly as part of the agg function call. +df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) + +# In 1.4+, grouping column "department" is included automatically. +df.groupBy("department").agg(func.max("age"), func.sum("expense")) + +# Revert to 1.3.x behavior (not retaining grouping column) by: +sqlContext.setConf("spark.sql.retainGroupColumns", "false") + +{% endhighlight %} +
    + +
    + + +#### Behavior change on DataFrame.withColumn + +Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added +as a new column with its specified name in the result DataFrame even if there may be any existing +columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different +name from names of all existing columns or replacing existing columns of the same name. + +Note that this change is only for Scala API, not for PySpark and SparkR. + + +## Upgrading from Spark SQL 1.0-1.2 to 1.3 + +In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the +available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other +releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked +as unstable (i.e., DeveloperAPI or Experimental). + +#### Rename of SchemaRDD to DataFrame + +The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has +been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD +directly, but instead provide most of the functionality that RDDs provide though their own +implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. + +In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for +some use cases. It is still recommended that users update their code to use `DataFrame` instead. +Java and Python users will need to update their code. + +#### Unification of the Java and Scala APIs + +Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) +that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users +of either language should use `SQLContext` and `DataFrame`. In general these classes try to +use types that are usable from both languages (i.e. `Array` instead of language-specific collections). +In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading +is used instead. + +Additionally, the Java specific types API has been removed. Users of both Scala and Java should +use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. + + +#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) + +Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought +all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit +conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. +Users should now write `import sqlContext.implicits._`. + +Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., +case classes or tuples) with a method `toDF`, instead of applying automatically. + +When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import +`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: +`import org.apache.spark.sql.functions._`. + +#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) + +Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users +should instead import the classes in `org.apache.spark.sql.types` + +#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) + +Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been +moved into the udf object in `SQLContext`. + +
    +
    +{% highlight scala %} + +sqlContext.udf.register("strLen", (s: String) => s.length()) + +{% endhighlight %} +
    + +
    +{% highlight java %} + +sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); + +{% endhighlight %} +
    + +
    + +Python UDF registration is unchanged. + +#### Python DataTypes No Longer Singletons + +When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of +referencing a singleton. diff --git a/docs/sql-migration-guide.md b/docs/sql-migration-guide.md new file mode 100644 index 0000000000000..4c23147106b65 --- /dev/null +++ b/docs/sql-migration-guide.md @@ -0,0 +1,38 @@ +--- +layout: global +title: Migration Guide +displayTitle: Migration Guide +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 Upgrading Guide](sql-migration-guide-upgrade.html) + * [Upgrading From Spark SQL 2.4 to 3.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-24-to-30) + * [Upgrading From Spark SQL 2.3 to 2.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-23-to-24) + * [Upgrading From Spark SQL 2.3.0 to 2.3.1 and above](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-230-to-231-and-above) + * [Upgrading From Spark SQL 2.2 to 2.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-22-to-23) + * [Upgrading From Spark SQL 2.1 to 2.2](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-21-to-22) + * [Upgrading From Spark SQL 2.0 to 2.1](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-20-to-21) + * [Upgrading From Spark SQL 1.6 to 2.0](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-16-to-20) + * [Upgrading From Spark SQL 1.5 to 1.6](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-15-to-16) + * [Upgrading From Spark SQL 1.4 to 1.5](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-14-to-15) + * [Upgrading from Spark SQL 1.3 to 1.4](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-13-to-14) + * [Upgrading from Spark SQL 1.0-1.2 to 1.3](sql-migration-guide-upgrade.html#upgrading-from-spark-sql-10-12-to-13) +* [Compatibility with Apache Hive](sql-migration-guide-hive-compatibility.html) + * [Deploying in Existing Hive Warehouses](sql-migration-guide-hive-compatibility.html#deploying-in-existing-hive-warehouses) + * [Supported Hive Features](sql-migration-guide-hive-compatibility.html#supported-hive-features) + * [Unsupported Hive Functionality](sql-migration-guide-hive-compatibility.html#unsupported-hive-functionality) + * [Incompatible Hive UDF](sql-migration-guide-hive-compatibility.html#incompatible-hive-udf) diff --git a/docs/sql-performance-tuning.md b/docs/sql-performance-tuning.md new file mode 100644 index 0000000000000..2a1edda84252c --- /dev/null +++ b/docs/sql-performance-tuning.md @@ -0,0 +1,174 @@ +--- +layout: global +title: Performance Tuning +displayTitle: Performance Tuning +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. +--- + +* Table of contents +{:toc} + +For some workloads, it is possible to improve performance by either caching data in memory, or by +turning on some experimental options. + +## Caching Data In Memory + +Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. +Then Spark SQL will scan only required columns and will automatically tune compression to minimize +memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. + +Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running +`SET key=value` commands using SQL. + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue + When set to true Spark SQL will automatically select a compression codec for each column based + on statistics of the data. +
    spark.sql.inMemoryColumnarStorage.batchSize10000 + Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization + and compression, but risk OOMs when caching data. +
    + +## Other Configuration Options + +The following options can also be used to tune the performance of query execution. It is possible +that these options will be deprecated in future release as more optimizations are performed automatically. + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) + The maximum number of bytes to pack into a single partition when reading files. +
    spark.sql.files.openCostInBytes4194304 (4 MB) + The estimated cost to open a file, measured by the number of bytes could be scanned in the same + time. This is used when putting multiple files into a partition. It is better to over-estimated, + then the partitions with small files will be faster than partitions with bigger files (which is + scheduled first). +
    spark.sql.broadcastTimeout300 +

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

    +
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) + Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when + performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently + statistics are only supported for Hive Metastore tables where the command + ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. +
    spark.sql.shuffle.partitions200 + Configures the number of partitions to use when shuffling data for joins or aggregations. +
    + +## Join Strategy Hints for SQL Queries + +The join strategy hints, namely `BROADCAST`, `MERGE`, `SHUFFLE_HASH` and `SHUFFLE_REPLICATE_NL`, +instruct Spark to use the hinted strategy on each specified relation when joining them with another +relation. For example, when the `BROADCAST` hint is used on table 't1', broadcast join (either +broadcast hash join or broadcast nested loop join depending on whether there is any equi-join key) +with 't1' as the build side will be prioritized by Spark even if the size of table 't1' suggested +by the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. + +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. 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. + +Note that there is no guarantee that Spark will choose the join strategy specified in the hint since +a specific strategy may not support all join types. + +
    + +
    + +{% highlight scala %} +import org.apache.spark.sql.functions.broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight java %} +import static org.apache.spark.sql.functions.broadcast; +broadcast(spark.table("src")).join(spark.table("records"), "key").show(); +{% endhighlight %} + +
    + +
    + +{% highlight python %} +from pyspark.sql.functions import broadcast +broadcast(spark.table("src")).join(spark.table("records"), "key").show() +{% endhighlight %} + +
    + +
    + +{% highlight r %} +src <- sql("SELECT * FROM src") +records <- sql("SELECT * FROM records") +head(join(broadcast(src), records, src$key == records$key)) +{% endhighlight %} + +
    + +
    + +{% highlight sql %} +-- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint +SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key +{% endhighlight %} + +
    +
    diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index fb03ed2e292b3..0a4d07ea37b69 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -2,17 +2,27 @@ layout: global displayTitle: Spark SQL, DataFrames and Datasets Guide title: Spark SQL and DataFrames +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 will become a table of contents (this text will be scraped). -{:toc} - -# Overview - Spark SQL is a Spark module for structured data processing. Unlike the basic Spark RDD API, the interfaces provided by Spark SQL provide Spark with more information about the structure of both the data and the computation being performed. Internally, Spark SQL uses this extra information to perform extra optimizations. There are several ways to -interact with Spark SQL including SQL and the Dataset API. When computing a result +interact with Spark SQL including SQL and the Dataset API. When computing a result, the same execution engine is used, independent of which API/language you are using to express the computation. This unification means that developers can easily switch back and forth between different APIs based on which provides the most natural way to express a given transformation. @@ -24,17 +34,17 @@ the `spark-shell`, `pyspark` shell, or `sparkR` shell. One use of Spark SQL is to execute SQL queries. Spark SQL can also be used to read data from an existing Hive installation. For more on how to -configure this feature, please refer to the [Hive Tables](#hive-tables) section. When running +configure this feature, please refer to the [Hive Tables](sql-data-sources-hive-tables.html) section. When running SQL from within another programming language the results will be returned as a [Dataset/DataFrame](#datasets-and-dataframes). -You can also interact with the SQL interface using the [command-line](#running-the-spark-sql-cli) -or over [JDBC/ODBC](#running-the-thrift-jdbcodbc-server). +You can also interact with the SQL interface using the [command-line](sql-distributed-sql-engine.html#running-the-spark-sql-cli) +or over [JDBC/ODBC](sql-distributed-sql-engine.html#running-the-thrift-jdbcodbc-server). ## Datasets and DataFrames A Dataset is a distributed collection of data. Dataset is a new interface added in Spark 1.6 that provides the benefits of RDDs (strong typing, ability to use powerful lambda functions) with the benefits of Spark SQL's optimized -execution engine. A Dataset can be [constructed](#creating-datasets) from JVM objects and then +execution engine. A Dataset can be [constructed](sql-getting-started.html#creating-datasets) from JVM objects and then manipulated using functional transformations (`map`, `flatMap`, `filter`, etc.). The Dataset API is available in [Scala][scala-datasets] and [Java][java-datasets]. Python does not have the support for the Dataset API. But due to Python's dynamic nature, @@ -43,7 +53,7 @@ many of the benefits of the Dataset API are already available (i.e. you can acce A DataFrame is a *Dataset* organized into named columns. It is conceptually equivalent to a table in a relational database or a data frame in R/Python, but with richer -optimizations under the hood. DataFrames can be constructed from a wide array of [sources](#data-sources) such +optimizations under the hood. DataFrames can be constructed from a wide array of [sources](sql-data-sources.html) such as: structured data files, tables in Hive, external databases, or existing RDDs. The DataFrame API is available in Scala, Java, [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame), and [R](api/R/index.html). @@ -55,3115 +65,3 @@ While, in [Java API][java-datasets], users need to use `Dataset` to represe [java-datasets]: api/java/index.html?org/apache/spark/sql/Dataset.html Throughout this document, we will often refer to Scala/Java Datasets of `Row`s as DataFrames. - -# Getting Started - -## Starting Point: SparkSession - -
    -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/scala/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/java/index.html#org.apache.spark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder()`: - -{% include_example init_session java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/python/pyspark.sql.html#pyspark.sql.SparkSession) class. To create a basic `SparkSession`, just use `SparkSession.builder`: - -{% include_example init_session python/sql/basic.py %} -
    - -
    - -The entry point into all functionality in Spark is the [`SparkSession`](api/R/sparkR.session.html) class. To initialize a basic `SparkSession`, just call `sparkR.session()`: - -{% include_example init_session r/RSparkSQLExample.R %} - -Note that when invoked for the first time, `sparkR.session()` initializes a global `SparkSession` singleton instance, and always returns a reference to this instance for successive invocations. In this way, users only need to initialize the `SparkSession` once, then SparkR functions like `read.df` will be able to access this global instance implicitly, and users don't need to pass the `SparkSession` instance around. -
    -
    - -`SparkSession` in Spark 2.0 provides builtin support for Hive features including the ability to -write queries using HiveQL, access to Hive UDFs, and the ability to read data from Hive tables. -To use these features, you do not need to have an existing Hive setup. - -## Creating DataFrames - -
    -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from an [existing `RDD`](#interoperating-with-rdds), -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df python/sql/basic.py %} -
    - -
    -With a `SparkSession`, applications can create DataFrames from a local R data.frame, -from a Hive table, or from [Spark data sources](#data-sources). - -As an example, the following creates a DataFrame based on the content of a JSON file: - -{% include_example create_df r/RSparkSQLExample.R %} - -
    -
    - - -## Untyped Dataset Operations (aka DataFrame Operations) - -DataFrames provide a domain-specific language for structured data manipulation in [Scala](api/scala/index.html#org.apache.spark.sql.Dataset), [Java](api/java/index.html?org/apache/spark/sql/Dataset.html), [Python](api/python/pyspark.sql.html#pyspark.sql.DataFrame) and [R](api/R/SparkDataFrame.html). - -As mentioned above, in Spark 2.0, DataFrames are just Dataset of `Row`s in Scala and Java API. These operations are also referred as "untyped transformations" in contrast to "typed transformations" come with strongly typed Scala/Java Datasets. - -Here we include some basic examples of structured data processing using Datasets: - -
    -
    -{% include_example untyped_ops scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/scala/index.html#org.apache.spark.sql.Dataset). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/scala/index.html#org.apache.spark.sql.functions$). -
    - -
    - -{% include_example untyped_ops java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} - -For a complete list of the types of operations that can be performed on a Dataset refer to the [API Documentation](api/java/org/apache/spark/sql/Dataset.html). - -In addition to simple column references and expressions, Datasets also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/java/org/apache/spark/sql/functions.html). -
    - -
    -In Python, it's possible to access a DataFrame's columns either by attribute -(`df.age`) or by indexing (`df['age']`). While the former is convenient for -interactive data exploration, users are highly encouraged to use the -latter form, which is future proof and won't break with column names that -are also attributes on the DataFrame class. - -{% include_example untyped_ops python/sql/basic.py %} -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/python/pyspark.sql.html#pyspark.sql.DataFrame). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/python/pyspark.sql.html#module-pyspark.sql.functions). - -
    - -
    - -{% include_example untyped_ops r/RSparkSQLExample.R %} - -For a complete list of the types of operations that can be performed on a DataFrame refer to the [API Documentation](api/R/index.html). - -In addition to simple column references and expressions, DataFrames also have a rich library of functions including string manipulation, date arithmetic, common math operations and more. The complete list is available in the [DataFrame Function Reference](api/R/SparkDataFrame.html). - -
    - -
    - -## Running SQL Queries Programmatically - -
    -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `Dataset`. - -{% include_example run_sql java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -The `sql` function on a `SparkSession` enables applications to run SQL queries programmatically and returns the result as a `DataFrame`. - -{% include_example run_sql python/sql/basic.py %} -
    - -
    -The `sql` function enables applications to run SQL queries programmatically and returns the result as a `SparkDataFrame`. - -{% include_example run_sql r/RSparkSQLExample.R %} - -
    -
    - - -## Global Temporary View - -Temporary views in Spark SQL are session-scoped and will disappear if the session that creates it -terminates. If you want to have a temporary view that is shared among all sessions and keep alive -until the Spark application terminates, you can create a global temporary view. Global temporary -view is tied to a system preserved database `global_temp`, and we must use the qualified name to -refer it, e.g. `SELECT * FROM global_temp.view1`. - -
    -
    -{% include_example global_temp_view scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example global_temp_view java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    -{% include_example global_temp_view python/sql/basic.py %} -
    - -
    - -{% highlight sql %} - -CREATE GLOBAL TEMPORARY VIEW temp_view AS SELECT a + 1, b * 2 FROM tbl - -SELECT * FROM global_temp.temp_view - -{% endhighlight %} - -
    -
    - - -## Creating Datasets - -Datasets are similar to RDDs, however, instead of using Java serialization or Kryo they use -a specialized [Encoder](api/scala/index.html#org.apache.spark.sql.Encoder) to serialize the objects -for processing or transmitting over the network. While both encoders and standard serialization are -responsible for turning an object into bytes, encoders are code generated dynamically and use a format -that allows Spark to perform many operations like filtering, sorting and hashing without deserializing -the bytes back into an object. - -
    -
    -{% include_example create_ds scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    -{% include_example create_ds java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    -
    - -## Interoperating with RDDs - -Spark SQL supports two different methods for converting existing RDDs into Datasets. The first -method uses reflection to infer the schema of an RDD that contains specific types of objects. This -reflection-based approach leads to more concise code and works well when you already know the schema -while writing your Spark application. - -The second method for creating Datasets is through a programmatic interface that allows you to -construct a schema and then apply it to an existing RDD. While this method is more verbose, it allows -you to construct Datasets when the columns and their types are not known until runtime. - -### Inferring the Schema Using Reflection -
    - -
    - -The Scala interface for Spark SQL supports automatically converting an RDD containing case classes -to a DataFrame. The case class -defines the schema of the table. The names of the arguments to the case class are read using -reflection and become the names of the columns. Case classes can also be nested or contain complex -types such as `Seq`s or `Array`s. This RDD can be implicitly converted to a DataFrame and then be -registered as a table. Tables can be used in subsequent SQL statements. - -{% include_example schema_inferring scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -Spark SQL supports automatically converting an RDD of -[JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly) into a DataFrame. -The `BeanInfo`, obtained using reflection, defines the schema of the table. Currently, Spark SQL -does not support JavaBeans that contain `Map` field(s). Nested JavaBeans and `List` or `Array` -fields are supported though. You can create a JavaBean by creating a class that implements -Serializable and has getters and setters for all of its fields. - -{% include_example schema_inferring java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -Spark SQL can convert an RDD of Row objects to a DataFrame, inferring the datatypes. Rows are constructed by passing a list of -key/value pairs as kwargs to the Row class. The keys of this list define the column names of the table, -and the types are inferred by sampling the whole dataset, similar to the inference that is performed on JSON files. - -{% include_example schema_inferring python/sql/basic.py %} -
    - -
    - -### Programmatically Specifying the Schema - -
    - -
    - -When case classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed -and fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} -
    - -
    - -When JavaBean classes cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `Dataset` can be created programmatically with three steps. - -1. Create an RDD of `Row`s from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -`Row`s in the RDD created in Step 1. -3. Apply the schema to the RDD of `Row`s via `createDataFrame` method provided -by `SparkSession`. - -For example: - -{% include_example programmatic_schema java/org/apache/spark/examples/sql/JavaSparkSQLExample.java %} -
    - -
    - -When a dictionary of kwargs cannot be defined ahead of time (for example, -the structure of records is encoded in a string, or a text dataset will be parsed and -fields will be projected differently for different users), -a `DataFrame` can be created programmatically with three steps. - -1. Create an RDD of tuples or lists from the original RDD; -2. Create the schema represented by a `StructType` matching the structure of -tuples or lists in the RDD created in the step 1. -3. Apply the schema to the RDD via `createDataFrame` method provided by `SparkSession`. - -For example: - -{% include_example programmatic_schema python/sql/basic.py %} -
    - -
    - -## Aggregations - -The [built-in DataFrames functions](api/scala/index.html#org.apache.spark.sql.functions$) provide common -aggregations such as `count()`, `countDistinct()`, `avg()`, `max()`, `min()`, etc. -While those functions are designed for DataFrames, Spark SQL also has type-safe versions for some of them in -[Scala](api/scala/index.html#org.apache.spark.sql.expressions.scalalang.typed$) and -[Java](api/java/org/apache/spark/sql/expressions/javalang/typed.html) to work with strongly typed Datasets. -Moreover, users are not limited to the predefined aggregate functions and can create their own. - -### Untyped User-Defined Aggregate Functions -Users have to extend the [UserDefinedAggregateFunction](api/scala/index.html#org.apache.spark.sql.expressions.UserDefinedAggregateFunction) -abstract class to implement a custom untyped aggregate function. For example, a user-defined average -can look like: - -
    -
    -{% include_example untyped_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedUntypedAggregation.scala%} -
    -
    -{% include_example untyped_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedUntypedAggregation.java%} -
    -
    - -### Type-Safe User-Defined Aggregate Functions - -User-defined aggregations for strongly typed Datasets revolve around the [Aggregator](api/scala/index.html#org.apache.spark.sql.expressions.Aggregator) abstract class. -For example, a type-safe user-defined average can look like: - -
    -
    -{% include_example typed_custom_aggregation scala/org/apache/spark/examples/sql/UserDefinedTypedAggregation.scala%} -
    -
    -{% include_example typed_custom_aggregation java/org/apache/spark/examples/sql/JavaUserDefinedTypedAggregation.java%} -
    -
    - -# Data Sources - -Spark SQL supports operating on a variety of data sources through the DataFrame interface. -A DataFrame can be operated on using relational transformations and can also be used to create a temporary view. -Registering a DataFrame as a temporary view allows you to run SQL queries over its data. This section -describes the general methods for loading and saving data using the Spark Data Sources and then -goes into specific options that are available for the built-in data sources. - -## Generic Load/Save Functions - -In the simplest form, the default data source (`parquet` unless otherwise configured by -`spark.sql.sources.default`) will be used for all operations. - -
    -
    -{% include_example generic_load_save_functions scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example generic_load_save_functions java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example generic_load_save_functions python/sql/datasource.py %} -
    - -
    - -{% include_example generic_load_save_functions r/RSparkSQLExample.R %} - -
    -
    - -### Manually Specifying Options - -You can also manually specify the data source that will be used along with any extra options -that you would like to pass to the data source. Data sources are specified by their fully qualified -name (i.e., `org.apache.spark.sql.parquet`), but for built-in sources you can also use their short -names (`json`, `parquet`, `jdbc`, `orc`, `libsvm`, `csv`, `text`). DataFrames loaded from any data -source type can be converted into other types using this syntax. - -To load a JSON file you can use: - -
    -
    -{% include_example manual_load_options scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options r/RSparkSQLExample.R %} -
    -
    - -To load a CSV file you can use: - -
    -
    -{% include_example manual_load_options_csv scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example manual_load_options_csv java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example manual_load_options_csv python/sql/datasource.py %} -
    - -
    -{% include_example manual_load_options_csv r/RSparkSQLExample.R %} - -
    -
    - -### Run SQL on files directly - -Instead of using read API to load a file into DataFrame and query it, you can also query that -file directly with SQL. - -
    -
    -{% include_example direct_sql scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example direct_sql java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example direct_sql python/sql/datasource.py %} -
    - -
    -{% include_example direct_sql r/RSparkSQLExample.R %} - -
    -
    - -### Save Modes - -Save operations can optionally take a `SaveMode`, that specifies how to handle existing data if -present. It is important to realize that these save modes do not utilize any locking and are not -atomic. Additionally, when performing an `Overwrite`, the data will be deleted before writing out the -new data. - - - - - - - - - - - - - - - - - - - - - - - -
    Scala/JavaAny LanguageMeaning
    SaveMode.ErrorIfExists (default)"error" or "errorifexists" (default) - When saving a DataFrame to a data source, if data already exists, - an exception is expected to be thrown. -
    SaveMode.Append"append" - When saving a DataFrame to a data source, if data/table already exists, - contents of the DataFrame are expected to be appended to existing data. -
    SaveMode.Overwrite"overwrite" - Overwrite mode means that when saving a DataFrame to a data source, - if data/table already exists, existing data is expected to be overwritten by the contents of - the DataFrame. -
    SaveMode.Ignore"ignore" - Ignore mode means that when saving a DataFrame to a data source, if data already exists, - the save operation is expected to not save the contents of the DataFrame and to not - change the existing data. This is similar to a CREATE TABLE IF NOT EXISTS in SQL. -
    - -### Saving to Persistent Tables - -`DataFrames` can also be saved as persistent tables into Hive metastore using the `saveAsTable` -command. Notice that an existing Hive deployment is not necessary to use this feature. Spark will create a -default local Hive metastore (using Derby) for you. Unlike the `createOrReplaceTempView` command, -`saveAsTable` will materialize the contents of the DataFrame and create a pointer to the data in the -Hive metastore. Persistent tables will still exist even after your Spark program has restarted, as -long as you maintain your connection to the same metastore. A DataFrame for a persistent table can -be created by calling the `table` method on a `SparkSession` with the name of the table. - -For file-based data source, e.g. text, parquet, json, etc. you can specify a custom table path via the -`path` option, e.g. `df.write.option("path", "/some/path").saveAsTable("t")`. When the table is dropped, -the custom table path will not be removed and the table data is still there. If no custom table path is -specified, Spark will write data to a default table path under the warehouse directory. When the table is -dropped, the default table path will be removed too. - -Starting from Spark 2.1, persistent datasource tables have per-partition metadata stored in the Hive metastore. This brings several benefits: - -- Since the metastore can return only necessary partitions for a query, discovering all the partitions on the first query to the table is no longer needed. -- Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - -Note that partition information is not gathered by default when creating external datasource tables (those with a `path` option). To sync the partition information in the metastore, you can invoke `MSCK REPAIR TABLE`. - -### Bucketing, Sorting and Partitioning - -For file-based data source, it is also possible to bucket and sort or partition the output. -Bucketing and sorting are applicable only to persistent tables: - -
    - -
    -{% include_example write_sorting_and_bucketing scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_sorting_and_bucketing java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_sorting_and_bucketing python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_by_name( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -CLUSTERED BY(name) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -while partitioning can be used with both `save` and `saveAsTable` when using the Dataset APIs. - - -
    - -
    -{% include_example write_partitioning scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partitioning java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partitioning python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_by_favorite_color( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING csv PARTITIONED BY(favorite_color); - -{% endhighlight %} - -
    - -
    - -It is possible to use both partitioning and bucketing for a single table: - -
    - -
    -{% include_example write_partition_and_bucket scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example write_partition_and_bucket java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example write_partition_and_bucket python/sql/datasource.py %} -
    - -
    - -{% highlight sql %} - -CREATE TABLE users_bucketed_and_partitioned( - name STRING, - favorite_color STRING, - favorite_numbers array -) USING parquet -PARTITIONED BY (favorite_color) -CLUSTERED BY(name) SORTED BY (favorite_numbers) INTO 42 BUCKETS; - -{% endhighlight %} - -
    - -
    - -`partitionBy` creates a directory structure as described in the [Partition Discovery](#partition-discovery) section. -Thus, it has limited applicability to columns with high cardinality. In contrast - `bucketBy` distributes -data across a fixed number of buckets and can be used when a number of unique values is unbounded. - -## Parquet Files - -[Parquet](http://parquet.io) is a columnar format that is supported by many other data processing systems. -Spark SQL provides support for both reading and writing Parquet files that automatically preserves the schema -of the original data. When writing Parquet files, all columns are automatically converted to be nullable for -compatibility reasons. - -### Loading Data Programmatically - -Using the data from the above example: - -
    - -
    -{% include_example basic_parquet_example scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example basic_parquet_example java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example basic_parquet_example python/sql/datasource.py %} -
    - -
    - -{% include_example basic_parquet_example r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW parquetTable -USING org.apache.spark.sql.parquet -OPTIONS ( - path "examples/src/main/resources/people.parquet" -) - -SELECT * FROM parquetTable - -{% endhighlight %} - -
    - -
    - -### Partition Discovery - -Table partitioning is a common optimization approach used in systems like Hive. In a partitioned -table, data are usually stored in different directories, with partitioning column values encoded in -the path of each partition directory. All built-in file sources (including Text/CSV/JSON/ORC/Parquet) -are able to discover and infer partitioning information automatically. -For example, we can store all our previously used -population data into a partitioned table using the following directory structure, with two extra -columns, `gender` and `country` as partitioning columns: - -{% highlight text %} - -path -└── to - └── table - ├── gender=male - │   ├── ... - │   │ - │   ├── country=US - │   │   └── data.parquet - │   ├── country=CN - │   │   └── data.parquet - │   └── ... - └── gender=female -    ├── ... -    │ -    ├── country=US -    │   └── data.parquet -    ├── country=CN -    │   └── data.parquet -    └── ... - -{% endhighlight %} - -By passing `path/to/table` to either `SparkSession.read.parquet` or `SparkSession.read.load`, Spark SQL -will automatically extract the partitioning information from the paths. -Now the schema of the returned DataFrame becomes: - -{% highlight text %} - -root -|-- name: string (nullable = true) -|-- age: long (nullable = true) -|-- gender: string (nullable = true) -|-- country: string (nullable = true) - -{% endhighlight %} - -Notice that the data types of the partitioning columns are automatically inferred. Currently, -numeric data types, date, timestamp and string type are supported. Sometimes users may not want -to automatically infer the data types of the partitioning columns. For these use cases, the -automatic type inference can be configured by -`spark.sql.sources.partitionColumnTypeInference.enabled`, which is default to `true`. When type -inference is disabled, string type will be used for the partitioning columns. - -Starting from Spark 1.6.0, partition discovery only finds partitions under the given paths -by default. For the above example, if users pass `path/to/table/gender=male` to either -`SparkSession.read.parquet` or `SparkSession.read.load`, `gender` will not be considered as a -partitioning column. If users need to specify the base path that partition discovery -should start with, they can set `basePath` in the data source options. For example, -when `path/to/table/gender=male` is the path of the data and -users set `basePath` to `path/to/table/`, `gender` will be a partitioning column. - -### Schema Merging - -Like ProtocolBuffer, Avro, and Thrift, Parquet also supports schema evolution. Users can start with -a simple schema, and gradually add more columns to the schema as needed. In this way, users may end -up with multiple Parquet files with different but mutually compatible schemas. The Parquet data -source is now able to automatically detect this case and merge schemas of all these files. - -Since schema merging is a relatively expensive operation, and is not a necessity in most cases, we -turned it off by default starting from 1.5.0. You may enable it by - -1. setting data source option `mergeSchema` to `true` when reading Parquet files (as shown in the - examples below), or -2. setting the global SQL option `spark.sql.parquet.mergeSchema` to `true`. - -
    - -
    -{% include_example schema_merging scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example schema_merging java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    - -{% include_example schema_merging python/sql/datasource.py %} -
    - -
    - -{% include_example schema_merging r/RSparkSQLExample.R %} - -
    - -
    - -### Hive metastore Parquet table conversion - -When reading from and writing to Hive metastore Parquet tables, Spark SQL will try to use its own -Parquet support instead of Hive SerDe for better performance. This behavior is controlled by the -`spark.sql.hive.convertMetastoreParquet` configuration, and is turned on by default. - -#### Hive/Parquet Schema Reconciliation - -There are two key differences between Hive and Parquet from the perspective of table schema -processing. - -1. Hive is case insensitive, while Parquet is not -1. Hive considers all columns nullable, while nullability in Parquet is significant - -Due to this reason, we must reconcile Hive metastore schema with Parquet schema when converting a -Hive metastore Parquet table to a Spark SQL Parquet table. The reconciliation rules are: - -1. Fields that have the same name in both schema must have the same data type regardless of - nullability. The reconciled field should have the data type of the Parquet side, so that - nullability is respected. - -1. The reconciled schema contains exactly those fields defined in Hive metastore schema. - - - Any fields that only appear in the Parquet schema are dropped in the reconciled schema. - - Any fields that only appear in the Hive metastore schema are added as nullable field in the - reconciled schema. - -#### Metadata Refreshing - -Spark SQL caches Parquet metadata for better performance. When Hive metastore Parquet table -conversion is enabled, metadata of those converted tables are also cached. If these tables are -updated by Hive or other external tools, you need to refresh them manually to ensure consistent -metadata. - -
    - -
    - -{% highlight scala %} -// spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight java %} -// spark is an existing SparkSession -spark.catalog().refreshTable("my_table"); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -# spark is an existing SparkSession -spark.catalog.refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight r %} -refreshTable("my_table") -{% endhighlight %} - -
    - -
    - -{% highlight sql %} -REFRESH TABLE my_table; -{% endhighlight %} - -
    - -
    - -### Configuration - -Configuration of Parquet can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.parquet.binaryAsStringfalse - Some other Parquet-producing systems, in particular Impala, Hive, and older versions of Spark SQL, do - 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. -
    spark.sql.parquet.int96AsTimestamptrue - 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. -
    spark.sql.parquet.compression.codecsnappy - Sets the compression codec used when writing Parquet files. If either `compression` or - `parquet.compression` is specified in the table-specific options/properties, the precedence would be - `compression`, `parquet.compression`, `spark.sql.parquet.compression.codec`. Acceptable values include: - none, uncompressed, snappy, gzip, lzo, brotli, lz4, zstd. - Note that `zstd` requires `ZStandardCodec` to be installed before Hadoop 2.9.0, `brotli` requires - `BrotliCodec` to be installed. -
    spark.sql.parquet.filterPushdowntrueEnables Parquet filter push-down optimization when set to true.
    spark.sql.hive.convertMetastoreParquettrue - When set to false, Spark SQL will use the Hive SerDe for parquet tables instead of the built in - support. -
    spark.sql.parquet.mergeSchemafalse -

    - When true, the Parquet data source merges schemas collected from all data files, otherwise the - schema is picked from the summary file or a random data file if no summary file is available. -

    -
    spark.sql.optimizer.metadataOnlytrue -

    - When true, enable the metadata-only query optimization that use the table's metadata to - produce the partition columns instead of table scans. It applies when all the columns scanned - are partition columns and the query has an aggregate operator that satisfies distinct - semantics. -

    -
    spark.sql.parquet.writeLegacyFormatfalse - If true, data will be written in a way of Spark 1.4 and earlier. For example, decimal values - will be written in Apache Parquet's fixed-length byte array format, which other systems such as - Apache Hive and Apache Impala use. If false, the newer format in Parquet will be used. For - 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. -
    - -## ORC Files - -Since Spark 2.3, Spark supports a vectorized ORC reader with a new ORC file format for ORC files. -To do that, the following configurations are newly added. The vectorized reader is used for the -native ORC tables (e.g., the ones created using the clause `USING ORC`) when `spark.sql.orc.impl` -is set to `native` and `spark.sql.orc.enableVectorizedReader` is set to `true`. For the Hive ORC -serde tables (e.g., the ones created using the clause `USING HIVE OPTIONS (fileFormat 'ORC')`), -the vectorized reader is used when `spark.sql.hive.convertMetastoreOrc` is also set to `true`. - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.orc.implnativeThe name of ORC implementation. It can be one of native and hive. native means the native ORC support that is built on Apache ORC 1.4. `hive` means the ORC library in Hive 1.2.1.
    spark.sql.orc.enableVectorizedReadertrueEnables 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.
    - -## JSON Datasets -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset[Row]`. -This conversion can be done using `SparkSession.read.json()` on either a `Dataset[String]`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a `Dataset`. -This conversion can be done using `SparkSession.read().json()` on either a `Dataset`, -or a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` option to `true`. - -{% include_example json_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. -This conversion can be done using `SparkSession.read.json` on a JSON file. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set the `multiLine` parameter to `True`. - -{% include_example json_dataset python/sql/datasource.py %} -
    - -
    -Spark SQL can automatically infer the schema of a JSON dataset and load it as a DataFrame. using -the `read.json()` function, which loads data from a directory of JSON files where each line of the -files is a JSON object. - -Note that the file that is offered as _a json file_ is not a typical JSON file. Each -line must contain a separate, self-contained valid JSON object. For more information, please see -[JSON Lines text format, also called newline-delimited JSON](http://jsonlines.org/). - -For a regular multi-line JSON file, set a named parameter `multiLine` to `TRUE`. - -{% include_example json_dataset r/RSparkSQLExample.R %} - -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jsonTable -USING org.apache.spark.sql.json -OPTIONS ( - path "examples/src/main/resources/people.json" -) - -SELECT * FROM jsonTable - -{% endhighlight %} - -
    - -
    - -## Hive Tables - -Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). -However, since Hive has a large number of dependencies, these dependencies are not included in the -default Spark distribution. If Hive dependencies can be found on the classpath, Spark will load them -automatically. Note that these Hive dependencies must also be present on all of the worker nodes, as -they will need access to the Hive serialization and deserialization libraries (SerDes) in order to -access data stored in Hive. - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` (for security configuration), -and `hdfs-site.xml` (for HDFS configuration) file in `conf/`. - -When working with Hive, one must instantiate `SparkSession` with Hive support, including -connectivity to a persistent Hive metastore, support for Hive serdes, and Hive user-defined functions. -Users who do not have an existing Hive deployment can still enable Hive support. When not configured -by the `hive-site.xml`, the context automatically creates `metastore_db` in the current directory and -creates a directory configured by `spark.sql.warehouse.dir`, which defaults to the directory -`spark-warehouse` in the current directory that the Spark application is started. Note that -the `hive.metastore.warehouse.dir` property in `hive-site.xml` is deprecated since Spark 2.0.0. -Instead, use `spark.sql.warehouse.dir` to specify the default location of database in warehouse. -You may need to grant write privilege to the user who starts the Spark application. - -
    - -
    -{% include_example spark_hive scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala %} -
    - -
    -{% include_example spark_hive java/org/apache/spark/examples/sql/hive/JavaSparkHiveExample.java %} -
    - -
    -{% include_example spark_hive python/sql/hive.py %} -
    - -
    - -When working with Hive one must instantiate `SparkSession` with Hive support. This -adds support for finding tables in the MetaStore and writing queries using HiveQL. - -{% include_example spark_hive r/RSparkSQLExample.R %} - -
    -
    - -### Specifying storage format for Hive tables - -When you create a Hive table, you need to define how this table should read/write data from/to file system, -i.e. the "input format" and "output format". You also need to define how this table should deserialize the data -to rows, or serialize rows to data, i.e. the "serde". The following options can be used to specify the storage -format("serde", "input format", "output format"), e.g. `CREATE TABLE src(id int) USING hive OPTIONS(fileFormat 'parquet')`. -By default, we will read the table files as plain text. Note that, Hive storage handler is not supported yet when -creating table, you can create a table using storage handler at Hive side, and use Spark SQL to read it. - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    fileFormat - A fileFormat is kind of a package of storage format specifications, including "serde", "input format" and - "output format". Currently we support 6 fileFormats: 'sequencefile', 'rcfile', 'orc', 'parquet', 'textfile' and 'avro'. -
    inputFormat, outputFormat - These 2 options specify the name of a corresponding `InputFormat` and `OutputFormat` class as a string literal, - e.g. `org.apache.hadoop.hive.ql.io.orc.OrcInputFormat`. These 2 options must be appeared in pair, and you can not - specify them if you already specified the `fileFormat` option. -
    serde - This option specifies the name of a serde class. When the `fileFormat` option is specified, do not specify this option - if the given `fileFormat` already include the information of serde. Currently "sequencefile", "textfile" and "rcfile" - don't include the serde information and you can use this option with these 3 fileFormats. -
    fieldDelim, escapeDelim, collectionDelim, mapkeyDelim, lineDelim - These options can only be used with "textfile" fileFormat. They define how to read delimited files into rows. -
    - -All other properties defined with `OPTIONS` will be regarded as Hive serde properties. - -### Interacting with Different Versions of Hive Metastore - -One of the most important pieces of Spark SQL's Hive support is interaction with Hive metastore, -which enables Spark SQL to access metadata of Hive tables. Starting from Spark 1.4.0, a single binary -build of Spark SQL can be used to query different versions of Hive metastores, using the configuration described below. -Note that independent of the version of Hive that is being used to talk to the metastore, internally Spark SQL -will compile against Hive 1.2.1 and use those classes for internal execution (serdes, UDFs, UDAFs, etc). - -The following options can be used to configure the version of Hive that is used to retrieve metadata: - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.hive.metastore.version1.2.1 - Version of the Hive metastore. Available - options are 0.12.0 through 2.3.3. -
    spark.sql.hive.metastore.jarsbuiltin - Location of the jars that should be used to instantiate the HiveMetastoreClient. This - property can be one of three options: -
      -
    1. builtin
    2. - Use Hive 1.2.1, which is bundled with the Spark assembly when -Phive is - enabled. When this option is chosen, spark.sql.hive.metastore.version must be - either 1.2.1 or not defined. -
    3. maven
    4. - Use Hive jars of specified version downloaded from Maven repositories. This configuration - is not generally recommended for production deployments. -
    5. A classpath in the standard format for the JVM. This classpath must include all of Hive - and its dependencies, including the correct version of Hadoop. These jars only need to be - present on the driver, but if you are running in yarn cluster mode then you must ensure - they are packaged with your application.
    6. -
    -
    spark.sql.hive.metastore.sharedPrefixescom.mysql.jdbc,
    org.postgresql,
    com.microsoft.sqlserver,
    oracle.jdbc
    -

    - A comma-separated list of class prefixes that should be loaded using the classloader that is - shared between Spark SQL and a specific version of Hive. An example of classes 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. -

    -
    spark.sql.hive.metastore.barrierPrefixes(empty) -

    - 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.*). -

    -
    - - -## JDBC To Other Databases - -Spark SQL also includes a data source that can read data from other databases using JDBC. This -functionality should be preferred over using [JdbcRDD](api/scala/index.html#org.apache.spark.rdd.JdbcRDD). -This is because the results are returned -as a DataFrame and they can easily be processed in Spark SQL or joined with other data sources. -The JDBC data source is also easier to use from Java or Python as it does not require the user to -provide a ClassTag. -(Note that this is different than the Spark SQL JDBC server, which allows other applications to -run queries using Spark SQL). - -To get started you will need to include the JDBC driver for your particular database on the -spark classpath. For example, to connect to postgres from the Spark Shell you would run the -following command: - -{% highlight bash %} -bin/spark-shell --driver-class-path postgresql-9.4.1207.jar --jars postgresql-9.4.1207.jar -{% endhighlight %} - -Tables from the remote database can be loaded as a DataFrame or Spark SQL temporary view using -the Data Sources API. Users can specify the JDBC connection properties in the data source options. -user and password are normally provided as connection properties for -logging into the data sources. In addition to the connection properties, Spark also supports -the following case-insensitive options: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameMeaning
    url - The JDBC URL to connect to. The source-specific connection properties may be specified in the URL. e.g., jdbc:postgresql://localhost/test?user=fred&password=secret -
    dbtable - The JDBC table that should be read from or written into. Note that when using it in the read - path anything that is valid in a FROM clause of a SQL query can be used. - For example, instead of a full table you could also use a subquery in parentheses. It is not - allowed to specify `dbtable` and `query` options at the same time. -
    query - A query that will be used to read data into Spark. The specified query will be parenthesized and used - as a subquery in the FROM clause. Spark will also assign an alias to the subquery clause. - As an example, spark will issue a query of the following form to the JDBC Source.

    - SELECT <columns> FROM (<user_specified_query>) spark_gen_alias

    - Below are couple of restrictions while using this option.
    -
      -
    1. It is not allowed to specify `dbtable` and `query` options at the same time.
    2. -
    3. It is not allowed to spcify `query` and `partitionColumn` options at the same time. When specifying - `partitionColumn` option is required, the subquery can be specified using `dbtable` option instead and - partition columns can be qualified using the subquery alias provided as part of `dbtable`.
      - Example:
      - - spark.read.format("jdbc")
      -    .option("dbtable", "(select c1, c2 from t1) as subq")
      -    .option("partitionColumn", "subq.c1"
      -    .load() -
    4. -
    -
    driver - The class name of the JDBC driver to use to connect to this URL. -
    partitionColumn, lowerBound, upperBound - These options must all be specified if any of them is specified. In addition, - numPartitions must be specified. They describe how to partition the table when - reading in parallel from multiple workers. - partitionColumn must be a numeric, date, or timestamp column from the table in question. - Notice that lowerBound and upperBound are just used to decide the - partition stride, not for filtering the rows in table. So all rows in the table will be - partitioned and returned. This option applies only to reading. -
    numPartitions - The maximum number of partitions that can be used for parallelism in table reading and - writing. This also determines the maximum number of concurrent JDBC connections. - If the number of partitions to write exceeds this limit, we decrease it to this limit by - calling coalesce(numPartitions) before writing. -
    queryTimeout - The number of seconds the driver will wait for a Statement object to execute to the given - number of seconds. Zero means there is no limit. In the write path, this option depends on - how JDBC drivers implement the API setQueryTimeout, e.g., the h2 JDBC driver - checks the timeout of each query instead of an entire JDBC batch. - It defaults to 0. -
    fetchsize - The JDBC fetch size, which determines how many rows to fetch per round trip. This can help performance on JDBC drivers which default to low fetch size (eg. Oracle with 10 rows). This option applies only to reading. -
    batchsize - The JDBC batch size, which determines how many rows to insert per round trip. This can help performance on JDBC drivers. This option applies only to writing. It defaults to 1000. -
    isolationLevel - The transaction isolation level, which applies to current connection. It can be one of NONE, READ_COMMITTED, READ_UNCOMMITTED, REPEATABLE_READ, or SERIALIZABLE, corresponding to standard transaction isolation levels defined by JDBC's Connection object, with default of READ_UNCOMMITTED. This option applies only to writing. Please refer the documentation in java.sql.Connection. -
    sessionInitStatement - After each database session is opened to the remote DB and before starting to read data, this option executes a custom SQL statement (or a PL/SQL block). Use this to implement session initialization code. Example: option("sessionInitStatement", """BEGIN execute immediate 'alter session set "_serial_direct_read"=true'; END;""") -
    truncate - This is a JDBC writer related option. When SaveMode.Overwrite is enabled, this option causes Spark to truncate an existing table instead of dropping and recreating it. This can be more efficient, and prevents the table metadata (e.g., indices) from being removed. However, it will not work in some cases, such as when the new data has a different schema. It defaults to false. This option applies only to writing. -
    cascadeTruncate - This is a JDBC writer related option. If enabled and supported by the JDBC database (PostgreSQL and Oracle at the moment), this options allows execution of a TRUNCATE TABLE t CASCADE (in the case of PostgreSQL a TRUNCATE TABLE ONLY t CASCADE is executed to prevent inadvertently truncating descendant tables). This will affect other tables, and thus should be used with care. This option applies only to writing. It defaults to the default cascading truncate behaviour of the JDBC database in question, specified in the isCascadeTruncate in each JDBCDialect. -
    createTableOptions - This is a JDBC writer related option. If specified, this option allows setting of database-specific table and partition options when creating a table (e.g., CREATE TABLE t (name string) ENGINE=InnoDB.). This option applies only to writing. -
    createTableColumnTypes - The database column data types to use instead of the defaults, when creating the table. Data type information should be specified in the same format as CREATE TABLE columns syntax (e.g: "name CHAR(64), comments VARCHAR(1024)"). The specified types should be valid spark sql data types. This option applies only to writing. -
    customSchema - The custom schema to use for reading data from JDBC connectors. For example, "id DECIMAL(38, 0), name STRING". You can also specify partial fields, and the others use the default type mapping. For example, "id DECIMAL(38, 0)". The column names should be identical to the corresponding column names of JDBC table. Users can specify the corresponding data types of Spark SQL instead of using the defaults. This option applies only to reading. -
    pushDownPredicate - The option to enable or disable predicate push-down into the JDBC data source. The default value is true, in which case Spark will push down filters to the JDBC data source as much as possible. Otherwise, if set to false, no filter will be pushed down to the JDBC data source and thus all filters will be handled by Spark. Predicate push-down is usually turned off when the predicate filtering is performed faster by Spark than by the JDBC data source. -
    - -
    - -
    -{% include_example jdbc_dataset scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala %} -
    - -
    -{% include_example jdbc_dataset java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java %} -
    - -
    -{% include_example jdbc_dataset python/sql/datasource.py %} -
    - -
    -{% include_example jdbc_dataset r/RSparkSQLExample.R %} -
    - -
    - -{% highlight sql %} - -CREATE TEMPORARY VIEW jdbcTable -USING org.apache.spark.sql.jdbc -OPTIONS ( - url "jdbc:postgresql:dbserver", - dbtable "schema.tablename", - user 'username', - password 'password' -) - -INSERT INTO TABLE jdbcTable -SELECT * FROM resultTable -{% endhighlight %} - -
    -
    - -## Avro Files -See the [Apache Avro Data Source Guide](avro-data-source-guide.html). - -## Troubleshooting - - * The JDBC driver class must be visible to the primordial class loader on the client session and on all executors. This is because Java's DriverManager class does a security check that results in it ignoring all drivers not visible to the primordial class loader when one goes to open a connection. One convenient way to do this is to modify compute_classpath.sh on all worker nodes to include your driver JARs. - * Some databases, such as H2, convert all names to upper case. You'll need to use upper case to refer to those names in Spark SQL. - * Users can specify vendor-specific JDBC connection properties in the data source options to do special treatment. For example, `spark.read.format("jdbc").option("url", oracleJdbcUrl).option("oracle.jdbc.mapDateToTimestamp", "false")`. `oracle.jdbc.mapDateToTimestamp` defaults to true, users often need to disable this flag to avoid Oracle date being resolved as timestamp. - -# Performance Tuning - -For some workloads, it is possible to improve performance by either caching data in memory, or by -turning on some experimental options. - -## Caching Data In Memory - -Spark SQL can cache tables using an in-memory columnar format by calling `spark.catalog.cacheTable("tableName")` or `dataFrame.cache()`. -Then Spark SQL will scan only required columns and will automatically tune compression to minimize -memory usage and GC pressure. You can call `spark.catalog.uncacheTable("tableName")` to remove the table from memory. - -Configuration of in-memory caching can be done using the `setConf` method on `SparkSession` or by running -`SET key=value` commands using SQL. - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.inMemoryColumnarStorage.compressedtrue - When set to true Spark SQL will automatically select a compression codec for each column based - on statistics of the data. -
    spark.sql.inMemoryColumnarStorage.batchSize10000 - Controls the size of batches for columnar caching. Larger batch sizes can improve memory utilization - and compression, but risk OOMs when caching data. -
    - -## Other Configuration Options - -The following options can also be used to tune the performance of query execution. It is possible -that these options will be deprecated in future release as more optimizations are performed automatically. - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Property NameDefaultMeaning
    spark.sql.files.maxPartitionBytes134217728 (128 MB) - The maximum number of bytes to pack into a single partition when reading files. -
    spark.sql.files.openCostInBytes4194304 (4 MB) - The estimated cost to open a file, measured by the number of bytes could be scanned in the same - time. This is used when putting multiple files into a partition. It is better to over estimated, - then the partitions with small files will be faster than partitions with bigger files (which is - scheduled first). -
    spark.sql.broadcastTimeout300 -

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

    -
    spark.sql.autoBroadcastJoinThreshold10485760 (10 MB) - Configures the maximum size in bytes for a table that will be broadcast to all worker nodes when - performing a join. By setting this value to -1 broadcasting can be disabled. Note that currently - statistics are only supported for Hive Metastore tables where the command - ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been run. -
    spark.sql.shuffle.partitions200 - Configures the number of partitions to use when shuffling data for joins or aggregations. -
    - -## Broadcast Hint for SQL Queries - -The `BROADCAST` hint guides Spark to broadcast each specified table when joining them with another table or view. -When Spark deciding the join methods, the broadcast hash join (i.e., BHJ) is preferred, -even if the statistics is above the configuration `spark.sql.autoBroadcastJoinThreshold`. -When both sides of a join are specified, Spark broadcasts the one having the lower statistics. -Note Spark does not guarantee BHJ is always chosen, since not all cases (e.g. full outer join) -support BHJ. When the broadcast nested loop join is selected, we still respect the hint. - -
    - -
    - -{% highlight scala %} -import org.apache.spark.sql.functions.broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight java %} -import static org.apache.spark.sql.functions.broadcast; -broadcast(spark.table("src")).join(spark.table("records"), "key").show(); -{% endhighlight %} - -
    - -
    - -{% highlight python %} -from pyspark.sql.functions import broadcast -broadcast(spark.table("src")).join(spark.table("records"), "key").show() -{% endhighlight %} - -
    - -
    - -{% highlight r %} -src <- sql("SELECT * FROM src") -records <- sql("SELECT * FROM records") -head(join(broadcast(src), records, src$key == records$key)) -{% endhighlight %} - -
    - -
    - -{% highlight sql %} --- We accept BROADCAST, BROADCASTJOIN and MAPJOIN for broadcast hint -SELECT /*+ BROADCAST(r) */ * FROM records r JOIN src s ON r.key = s.key -{% endhighlight %} - -
    -
    - -# Distributed SQL Engine - -Spark SQL can also act as a distributed query engine using its JDBC/ODBC or command-line interface. -In this mode, end-users or applications can interact with Spark SQL directly to run SQL queries, -without the need to write any code. - -## Running the Thrift JDBC/ODBC server - -The Thrift JDBC/ODBC server implemented here corresponds to the [`HiveServer2`](https://cwiki.apache.org/confluence/display/Hive/Setting+Up+HiveServer2) -in Hive 1.2.1 You can test the JDBC server with the beeline script that comes with either Spark or Hive 1.2.1. - -To start the JDBC/ODBC server, run the following in the Spark directory: - - ./sbin/start-thriftserver.sh - -This script accepts all `bin/spark-submit` command line options, plus a `--hiveconf` option to -specify Hive properties. You may run `./sbin/start-thriftserver.sh --help` for a complete list of -all available options. By default, the server listens on localhost:10000. You may override this -behaviour via either environment variables, i.e.: - -{% highlight bash %} -export HIVE_SERVER2_THRIFT_PORT= -export HIVE_SERVER2_THRIFT_BIND_HOST= -./sbin/start-thriftserver.sh \ - --master \ - ... -{% endhighlight %} - -or system properties: - -{% highlight bash %} -./sbin/start-thriftserver.sh \ - --hiveconf hive.server2.thrift.port= \ - --hiveconf hive.server2.thrift.bind.host= \ - --master - ... -{% endhighlight %} - -Now you can use beeline to test the Thrift JDBC/ODBC server: - - ./bin/beeline - -Connect to the JDBC/ODBC server in beeline with: - - beeline> !connect jdbc:hive2://localhost:10000 - -Beeline will ask you for a username and password. In non-secure mode, simply enter the username on -your machine and a blank password. For secure mode, please follow the instructions given in the -[beeline documentation](https://cwiki.apache.org/confluence/display/Hive/HiveServer2+Clients). - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. - -You may also use the beeline script that comes with Hive. - -Thrift JDBC server also supports sending thrift RPC messages over HTTP transport. -Use the following setting to enable HTTP mode as system property or in `hive-site.xml` file in `conf/`: - - hive.server2.transport.mode - Set this to value: http - hive.server2.thrift.http.port - HTTP port number to listen on; default is 10001 - hive.server2.http.endpoint - HTTP endpoint; default is cliservice - -To test, use beeline to connect to the JDBC/ODBC server in http mode with: - - beeline> !connect jdbc:hive2://:/?hive.server2.transport.mode=http;hive.server2.thrift.http.path= - - -## Running the Spark SQL CLI - -The Spark SQL CLI is a convenient tool to run the Hive metastore service in local mode and execute -queries input from the command line. Note that the Spark SQL CLI cannot talk to the Thrift JDBC server. - -To start the Spark SQL CLI, run the following in the Spark directory: - - ./bin/spark-sql - -Configuration of Hive is done by placing your `hive-site.xml`, `core-site.xml` and `hdfs-site.xml` files in `conf/`. -You may run `./bin/spark-sql --help` for a complete list of all available -options. - -# PySpark Usage Guide for Pandas with Apache Arrow - -## Apache Arrow in Spark - -Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer -data between JVM and Python processes. This currently is most beneficial to Python users that -work with Pandas/NumPy data. Its usage is not automatic and might require some minor -changes to configuration or code to take full advantage and ensure compatibility. This guide will -give a high-level description of how to use Arrow in Spark and highlight any differences when -working with Arrow-enabled data. - -### Ensure PyArrow Installed - -If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the -SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow -is installed and available on all cluster nodes. The current supported version is 0.8.0. -You can install using pip or conda from the conda-forge channel. See PyArrow -[installation](https://arrow.apache.org/docs/python/install.html) for details. - -## Enabling for Conversion to/from Pandas - -Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame -using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with -`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set -the Spark configuration 'spark.sql.execution.arrow.enabled' to 'true'. This is disabled by default. - -In addition, optimizations enabled by 'spark.sql.execution.arrow.enabled' could fallback automatically -to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. -This can be controlled by 'spark.sql.execution.arrow.fallback.enabled'. - -
    -
    -{% include_example dataframe_with_arrow python/sql/arrow.py %} -
    -
    - -Using the above optimizations with Arrow will produce the same results as when Arrow is not -enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the -DataFrame to the driver program and should be done on a small subset of the data. Not all Spark -data types are currently supported and an error can be raised if a column has an unsupported type, -see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, -Spark will fall back to create the DataFrame without Arrow. - -## Pandas UDFs (a.k.a. Vectorized UDFs) - -Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and -Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator -or to wrap the function, no additional configuration is required. Currently, there are two types of -Pandas UDF: Scalar and Grouped Map. - -### Scalar - -Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such -as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return -a `pandas.Series` of the same length. Internally, Spark 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 the results together. - -The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. - -
    -
    -{% include_example scalar_pandas_udf python/sql/arrow.py %} -
    -
    - -### Grouped Map -Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. -Split-apply-combine consists of three steps: -* Split the data into groups by using `DataFrame.groupBy`. -* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The - input data contains all the rows and columns for each group. -* Combine the results into a new `DataFrame`. - -To use `groupBy().apply()`, the user needs to define the following: -* A Python function that defines the computation for each group. -* A `StructType` object or a string that defines the schema of the output `DataFrame`. - -The column labels of the returned `pandas.DataFrame` must either match the field names in the -defined output schema if specified as strings, or match the field data types by position if not -strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) -on how to label columns when constructing a `pandas.DataFrame`. - -Note that all data for a group will be loaded into memory before the function is applied. This can -lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for -[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user -to ensure that the grouped data will fit into the available memory. - -The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. - -
    -
    -{% include_example grouped_map_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and -[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). - -### Grouped Aggregate - -Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and -[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` -to a scalar value, where each `pandas.Series` represents a column within the group or window. - -Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, -only unbounded window is supported with Grouped aggregate Pandas UDFs currently. - -The following example shows how to use this type of UDF to compute mean with groupBy and window operations: - -
    -
    -{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} -
    -
    - -For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) - -## Usage Notes - -### Supported SQL Types - -Currently, all Spark SQL data types are supported by Arrow-based conversion except `BinaryType`, `MapType`, -`ArrayType` of `TimestampType`, and nested `StructType`. - -### Setting Arrow Batch Size - -Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to -high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow -record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" -to an integer that will determine the maximum number of rows for each batch. The default value is -10,000 records per batch. If the number of columns is large, the value should be adjusted -accordingly. Using this limit, each data partition will be made into 1 or more record batches for -processing. - -### Timestamp with Time Zone Semantics - -Spark internally stores timestamps as UTC values, and timestamp data that is brought in without -a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp -data is exported or displayed in Spark, the session time zone is used to localize the timestamp -values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will -default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond -resolution, `datetime64[ns]`, with optional time zone on a per-column basis. - -When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds -and each column will be converted to the Spark session time zone then localized to that time -zone, which removes the time zone and displays values as local time. This will occur -when calling `toPandas()` or `pandas_udf` with timestamp columns. - -When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This -occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a -`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the -expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond -values will be truncated. - -Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is -different than a Pandas timestamp. It is recommended to use Pandas time series functionality when -working with timestamps in `pandas_udf`s to get the best performance, see -[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. - -# Migration Guide - -## Upgrading From Spark SQL 2.4 to 3.0 - - - 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 come 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`. - -## Upgrading From Spark SQL 2.3 to 2.4 - - - In Spark version 2.3 and earlier, the second parameter to array_contains function is implicitly promoted to the element type of first array type parameter. This type promotion can be lossy and may cause `array_contains` function to return wrong result. This problem has been addressed in 2.4 by employing a safer type promotion mechanism. This can cause some change in behavior and are illustrated in the table below. - - - - - - - - - - - - - - - - - - - - - - - - - -
    - Query - - Result Spark 2.3 or Prior - - Result Spark 2.4 - - Remarks -
    - SELECT
    array_contains(array(1), 1.34D);
    -
    - true - - false - - In Spark 2.4, left and right parameters are promoted to array(double) and double type respectively. -
    - SELECT
    array_contains(array(1), '1');
    -
    - true - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explict cast -
    - SELECT
    array_contains(array(1), 'anystring');
    -
    - null - - AnalysisException is thrown since integer type can not be promoted to string type in a loss-less manner. - - Users can use explict cast -
    - - - Since Spark 2.4, when there is a struct field in front of the IN operator before a subquery, the inner query must contain a struct field as well. In previous versions, instead, the fields of the struct were compared to the output of the inner query. Eg. if `a` is a `struct(a string, b int)`, in Spark 2.4 `a in (select (1 as a, 'a' as b) from range(1))` is a valid query, while `a in (select 1, 'a' from range(1))` is not. In previous version it was the opposite. - - In versions 2.2.1+ and 2.3, if `spark.sql.caseSensitive` is set to true, then the `CURRENT_DATE` and `CURRENT_TIMESTAMP` functions incorrectly became case-sensitive and would resolve to columns (unless typed in lower case). In Spark 2.4 this has been fixed and the functions are no longer case-sensitive. - - Since Spark 2.4, Spark will evaluate the set operations referenced in a query by following a precedence rule as per the SQL standard. If the order is not specified by parentheses, set operations are performed from left to right with the exception that all INTERSECT operations are performed before any UNION, EXCEPT or MINUS operations. The old behaviour of giving equal precedence to all the set operations are preserved under a newly added configuration `spark.sql.legacy.setopsPrecedence.enabled` with a default value of `false`. When this property is set to `true`, spark will evaluate the set operators from left to right as they appear in the query given no explicit ordering is enforced by usage of parenthesis. - - Since Spark 2.4, Spark will display table description column Last Access value as UNKNOWN when the value was Jan 01 1970. - - Since Spark 2.4, Spark maximizes the usage of a vectorized ORC reader for ORC files by default. To do that, `spark.sql.orc.impl` and `spark.sql.orc.filterPushdown` change their default values to `native` and `true` respectively. - - In PySpark, when Arrow optimization is enabled, previously `toPandas` just failed when Arrow optimization is unable to be used whereas `createDataFrame` from Pandas DataFrame allowed the fallback to non-optimization. Now, both `toPandas` and `createDataFrame` from Pandas DataFrame allow the fallback by default, which can be switched off by `spark.sql.execution.arrow.fallback.enabled`. - - Since Spark 2.4, writing an empty dataframe to a directory launches at least one write task, even if physically the dataframe has no partition. This introduces a small behavior change that for self-describing file formats like Parquet and Orc, Spark creates a metadata-only file in the target directory when writing a 0-partition dataframe, so that schema inference can still work if users read that directory later. The new behavior is more reasonable and more consistent regarding writing empty dataframe. - - Since Spark 2.4, expression IDs in UDF arguments do not appear in column names. For example, an column name in Spark 2.4 is not `UDF:f(col0 AS colA#28)` but ``UDF:f(col0 AS `colA`)``. - - Since Spark 2.4, writing a dataframe with an empty or nested empty schema using any file formats (parquet, orc, json, text, csv etc.) is not allowed. An exception is thrown when attempting to write dataframes with empty schema. - - Since Spark 2.4, Spark compares a DATE type with a TIMESTAMP type after promotes both sides to TIMESTAMP. To set `false` to `spark.sql.legacy.compareDateTimestampInTimestamp` restores the previous behavior. This option will be removed in Spark 3.0. - - Since Spark 2.4, creating a managed table with nonempty location is not allowed. An exception is thrown when attempting to create a managed table with nonempty location. To set `true` to `spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation` restores the previous behavior. This option will be removed in Spark 3.0. - - Since Spark 2.4, renaming a managed table to existing location is not allowed. An exception is thrown when attempting to rename a managed table to existing location. - - Since Spark 2.4, the type coercion rules can automatically promote the argument types of the variadic SQL functions (e.g., IN/COALESCE) to the widest common type, no matter how the input arguments order. In prior Spark versions, the promotion could fail in some specific orders (e.g., TimestampType, IntegerType and StringType) and throw an exception. - - Since Spark 2.4, Spark has enabled non-cascading SQL cache invalidation in addition to the traditional cache invalidation mechanism. The non-cascading cache invalidation mechanism allows users to remove a cache without impacting its dependent caches. This new cache invalidation mechanism is used in scenarios where the data of the cache to be removed is still valid, e.g., calling unpersist() on a Dataset, or dropping a temporary view. This allows users to free up memory and keep the desired caches valid at the same time. - - In version 2.3 and earlier, Spark converts Parquet Hive tables by default but ignores table properties like `TBLPROPERTIES (parquet.compression 'NONE')`. This happens for ORC Hive table properties like `TBLPROPERTIES (orc.compress 'NONE')` in case of `spark.sql.hive.convertMetastoreOrc=true`, too. Since Spark 2.4, Spark respects Parquet/ORC specific table properties while converting Parquet/ORC Hive tables. As an example, `CREATE TABLE t(id int) STORED AS PARQUET TBLPROPERTIES (parquet.compression 'NONE')` would generate Snappy parquet files during insertion in Spark 2.3, and in Spark 2.4, the result would be uncompressed parquet files. - - Since Spark 2.0, Spark converts Parquet Hive tables by default for better performance. Since Spark 2.4, Spark converts ORC Hive tables by default, too. It means Spark uses its own ORC support by default instead of Hive SerDe. As an example, `CREATE TABLE t(id int) STORED AS ORC` would be handled with Hive SerDe in Spark 2.3, and in Spark 2.4, it would be converted into Spark's ORC data source table and ORC vectorization would be applied. To set `false` to `spark.sql.hive.convertMetastoreOrc` restores the previous behavior. - - In version 2.3 and earlier, CSV rows are considered as malformed if at least one column value in the row is malformed. CSV parser dropped such rows in the DROPMALFORMED mode or outputs an error in the FAILFAST mode. Since Spark 2.4, CSV row is considered as malformed only when it contains malformed column values requested from CSV datasource, other values can be ignored. As an example, CSV file contains the "id,name" header and one row "1234". In Spark 2.4, selection of the id column consists of a row with one column value 1234 but in Spark 2.3 and earlier it is empty in the DROPMALFORMED mode. To restore the previous behavior, set `spark.sql.csv.parser.columnPruning.enabled` to `false`. - - Since Spark 2.4, File listing for compute statistics is done in parallel by default. This can be disabled by setting `spark.sql.statistics.parallelFileListingInStatsComputation.enabled` to `False`. - - Since Spark 2.4, Metadata files (e.g. Parquet summary files) and temporary files are not counted as data files when calculating table size during Statistics computation. - - Since Spark 2.4, empty strings are saved as quoted empty strings `""`. In version 2.3 and earlier, empty strings are equal to `null` values and do not reflect to any characters in saved CSV files. For example, the row of `"a", null, "", 1` was writted as `a,,,1`. Since Spark 2.4, the same row is saved as `a,,"",1`. To restore the previous behavior, set the CSV option `emptyValue` to empty (not quoted) string. - - Since Spark 2.4, The LOAD DATA command supports wildcard `?` and `*`, which match any one character, and zero or more characters, respectively. Example: `LOAD DATA INPATH '/tmp/folder*/'` or `LOAD DATA INPATH '/tmp/part-?'`. Special Characters like `space` also now work in paths. Example: `LOAD DATA INPATH '/tmp/folder name/'`. - - In Spark version 2.3 and earlier, HAVING without GROUP BY is treated as WHERE. This means, `SELECT 1 FROM range(10) HAVING true` is executed as `SELECT 1 FROM range(10) WHERE true` and returns 10 rows. This violates SQL standard, and has been fixed in Spark 2.4. Since Spark 2.4, HAVING without GROUP BY is treated as a global aggregate, which means `SELECT 1 FROM range(10) HAVING true` will return only one row. To restore the previous behavior, set `spark.sql.legacy.parser.havingWithoutGroupByAsWhere` to `true`. - -## Upgrading From Spark SQL 2.3.0 to 2.3.1 and above - - - As of version 2.3.1 Arrow functionality, including `pandas_udf` and `toPandas()`/`createDataFrame()` with `spark.sql.execution.arrow.enabled` set to `True`, has been marked as experimental. These are still evolving and not currently recommended for use in production. - -## Upgrading From Spark SQL 2.2 to 2.3 - - - Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the referenced columns only include the internal corrupt record column (named `_corrupt_record` by default). For example, `spark.read.schema(schema).json(file).filter($"_corrupt_record".isNotNull).count()` and `spark.read.schema(schema).json(file).select("_corrupt_record").show()`. Instead, you can cache or save the parsed results and then send the same query. For example, `val df = spark.read.schema(schema).json(file).cache()` and then `df.filter($"_corrupt_record".isNotNull).count()`. - - The `percentile_approx` function previously accepted numeric type input and output double type results. Now it supports date type, timestamp type and numeric types as input types. The result type is also changed to be the same as the input type, which is more reasonable for percentiles. - - Since Spark 2.3, the Join/Filter's deterministic predicates that are after the first non-deterministic predicates are also pushed down/through the child operators, if possible. In prior Spark versions, these filters are not eligible for predicate pushdown. - - Partition column inference previously found incorrect common type for different inferred types, for example, previously it ended up with double type as the common type for double type and date type. Now it finds the correct common type for such conflicts. The conflict resolution follows the table below: - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    - InputA \ InputB - - NullType - - IntegerType - - LongType - - DecimalType(38,0)* - - DoubleType - - DateType - - TimestampType - - StringType -
    - NullType - NullTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeDateTypeTimestampTypeStringType
    - IntegerType - IntegerTypeIntegerTypeLongTypeDecimalType(38,0)DoubleTypeStringTypeStringTypeStringType
    - LongType - LongTypeLongTypeLongTypeDecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DecimalType(38,0)* - DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)DecimalType(38,0)StringTypeStringTypeStringTypeStringType
    - DoubleType - DoubleTypeDoubleTypeStringTypeStringTypeDoubleTypeStringTypeStringTypeStringType
    - DateType - DateTypeStringTypeStringTypeStringTypeStringTypeDateTypeTimestampTypeStringType
    - TimestampType - TimestampTypeStringTypeStringTypeStringTypeStringTypeTimestampTypeTimestampTypeStringType
    - StringType - StringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringTypeStringType
    - - Note that, for DecimalType(38,0)*, the table above intentionally does not cover all other combinations of scales and precisions because currently we only infer decimal type like `BigInteger`/`BigInt`. For example, 1.1 is inferred as double type. - - In PySpark, now we need Pandas 0.19.2 or upper if you want to use Pandas related functionalities, such as `toPandas`, `createDataFrame` from Pandas DataFrame, etc. - - In PySpark, the behavior of timestamp values for Pandas related functionalities was changed to respect session timezone. If you want to use the old behavior, you need to set a configuration `spark.sql.execution.pandas.respectSessionTimeZone` to `False`. See [SPARK-22395](https://issues.apache.org/jira/browse/SPARK-22395) for details. - - In PySpark, `na.fill()` or `fillna` also accepts boolean and replaces nulls with booleans. In prior Spark versions, PySpark just ignores it and returns the original Dataset/DataFrame. - - Since Spark 2.3, when either broadcast hash join or broadcast nested loop join is applicable, we prefer to broadcasting the table that is explicitly specified in a broadcast hint. For details, see the section [Broadcast Hint](#broadcast-hint-for-sql-queries) and [SPARK-22489](https://issues.apache.org/jira/browse/SPARK-22489). - - Since Spark 2.3, when all inputs are binary, `functions.concat()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.concatBinaryAsString` to `true`. - - Since Spark 2.3, when all inputs are binary, SQL `elt()` returns an output as binary. Otherwise, it returns as a string. Until Spark 2.3, it always returns as a string despite of input types. To keep the old behavior, set `spark.sql.function.eltOutputAsString` to `true`. - - - Since Spark 2.3, by default arithmetic operations between decimals return a rounded value if an exact representation is not possible (instead of returning NULL). This is compliant with SQL ANSI 2011 specification and Hive's new behavior introduced in Hive 2.2 (HIVE-15331). This involves the following changes - - The rules to determine the result type of an arithmetic operation have been updated. In particular, if the precision / scale needed are out of the range of available values, the scale is reduced up to 6, in order to prevent the truncation of the integer part of the decimals. All the arithmetic operations are affected by the change, ie. addition (`+`), subtraction (`-`), multiplication (`*`), division (`/`), remainder (`%`) and positive module (`pmod`). - - Literal values used in SQL operations are converted to DECIMAL with the exact precision and scale needed by them. - - The configuration `spark.sql.decimalOperations.allowPrecisionLoss` has been introduced. It defaults to `true`, which means the new behavior described here; if set to `false`, Spark uses previous rules, ie. it doesn't adjust the needed scale to represent the values and it returns NULL if an exact representation of the value is not possible. - - In PySpark, `df.replace` does not allow to omit `value` when `to_replace` is not a dictionary. Previously, `value` could be omitted in the other cases and had `None` by default, which is counterintuitive and error-prone. - - Un-aliased subquery's semantic has not been well defined with confusing behaviors. Since Spark 2.3, we invalidate such confusing cases, for example: `SELECT v.i from (SELECT i FROM v)`, Spark will throw an analysis exception in this case because users should not be able to use the qualifier inside a subquery. See [SPARK-20690](https://issues.apache.org/jira/browse/SPARK-20690) and [SPARK-21335](https://issues.apache.org/jira/browse/SPARK-21335) for more details. - - - 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 2.3, the builder come to not update the configurations. If you want to update them, you need to update them prior to creating a `SparkSession`. - -## Upgrading From Spark SQL 2.1 to 2.2 - - - Spark 2.1.1 introduced a new configuration key: `spark.sql.hive.caseSensitiveInferenceMode`. It had a default setting of `NEVER_INFER`, which kept behavior identical to 2.1.0. However, Spark 2.2.0 changes this setting's default value to `INFER_AND_SAVE` to restore compatibility with reading Hive metastore tables whose underlying file schema have mixed-case column names. With the `INFER_AND_SAVE` configuration value, on first access Spark will perform schema inference on any Hive metastore table for which it has not already saved an inferred schema. Note that schema inference can be a very time-consuming operation for tables with thousands of partitions. If compatibility with mixed-case column names is not a concern, you can safely set `spark.sql.hive.caseSensitiveInferenceMode` to `NEVER_INFER` to avoid the initial overhead of schema inference. Note that with the new default `INFER_AND_SAVE` setting, the results of the schema inference are saved as a metastore key for future use. Therefore, the initial schema inference occurs only at a table's first access. - - - Since Spark 2.2.1 and 2.3.0, the schema is always inferred at runtime when the data source tables have the columns that exist in both partition schema and data schema. The inferred schema does not have the partitioned columns. When reading the table, Spark respects the partition values of these overlapping columns instead of the values stored in the data source files. In 2.2.0 and 2.1.x release, the inferred schema is partitioned but the data of the table is invisible to users (i.e., the result set is empty). - -## Upgrading From Spark SQL 2.0 to 2.1 - - - Datasource tables now store partition metadata in the Hive metastore. This means that Hive DDLs such as `ALTER TABLE PARTITION ... SET LOCATION` are now available for tables created with the Datasource API. - - Legacy datasource tables can be migrated to this format via the `MSCK REPAIR TABLE` command. Migrating legacy tables is recommended to take advantage of Hive DDL support and improved planning performance. - - To determine if a table has been migrated, look for the `PartitionProvider: Catalog` attribute when issuing `DESCRIBE FORMATTED` on the table. - - Changes to `INSERT OVERWRITE TABLE ... PARTITION ...` behavior for Datasource tables. - - In prior Spark versions `INSERT OVERWRITE` overwrote the entire Datasource table, even when given a partition specification. Now only partitions matching the specification are overwritten. - - Note that this still differs from the behavior of Hive tables, which is to overwrite only partitions overlapping with newly inserted data. - -## Upgrading From Spark SQL 1.6 to 2.0 - - - `SparkSession` is now the new entry point of Spark that replaces the old `SQLContext` and - `HiveContext`. Note that the old SQLContext and HiveContext are kept for backward compatibility. A new `catalog` interface is accessible from `SparkSession` - existing API on databases and tables access such as `listTables`, `createExternalTable`, `dropTempView`, `cacheTable` are moved here. - - - Dataset API and DataFrame API are unified. In Scala, `DataFrame` becomes a type alias for - `Dataset[Row]`, while Java API users must replace `DataFrame` with `Dataset`. Both the typed - transformations (e.g., `map`, `filter`, and `groupByKey`) and untyped transformations (e.g., - `select` and `groupBy`) are available on the Dataset class. Since compile-time type-safety in - Python and R is not a language feature, the concept of Dataset does not apply to these languages’ - APIs. Instead, `DataFrame` remains the primary programming abstraction, which is analogous to the - single-node data frame notion in these languages. - - - Dataset and DataFrame API `unionAll` has been deprecated and replaced by `union` - - Dataset and DataFrame API `explode` has been deprecated, alternatively, use `functions.explode()` with `select` or `flatMap` - - Dataset and DataFrame API `registerTempTable` has been deprecated and replaced by `createOrReplaceTempView` - - - Changes to `CREATE TABLE ... LOCATION` behavior for Hive tables. - - From Spark 2.0, `CREATE TABLE ... LOCATION` is equivalent to `CREATE EXTERNAL TABLE ... LOCATION` - in order to prevent accidental dropping the existing data in the user-provided locations. - That means, a Hive table created in Spark SQL with the user-specified location is always a Hive external table. - Dropping external tables will not remove the data. Users are not allowed to specify the location for Hive managed tables. - Note that this is different from the Hive behavior. - - As a result, `DROP TABLE` statements on those tables will not remove the data. - - - `spark.sql.parquet.cacheMetadata` is no longer used. - See [SPARK-13664](https://issues.apache.org/jira/browse/SPARK-13664) for details. - -## Upgrading From Spark SQL 1.5 to 1.6 - - - From Spark 1.6, by default, the Thrift server runs in multi-session mode. Which means each JDBC/ODBC - connection owns a copy of their own SQL configuration and temporary function registry. Cached - tables are still shared though. If you prefer to run the Thrift server in the old single-session - mode, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add - this option to `spark-defaults.conf`, or pass it to `start-thriftserver.sh` via `--conf`: - - {% highlight bash %} - ./sbin/start-thriftserver.sh \ - --conf spark.sql.hive.thriftServer.singleSession=true \ - ... - {% endhighlight %} - - Since 1.6.1, withColumn method in sparkR supports adding a new column to or replacing existing columns - of the same name of a DataFrame. - - - From Spark 1.6, LongType casts to TimestampType expect seconds instead of microseconds. This - change was made to match the behavior of Hive 1.2 for more consistent type casting to TimestampType - from numeric types. See [SPARK-11724](https://issues.apache.org/jira/browse/SPARK-11724) for - details. - -## Upgrading From Spark SQL 1.4 to 1.5 - - - Optimized execution using manually managed memory (Tungsten) is now enabled by default, along with - code generation for expression evaluation. These features can both be disabled by setting - `spark.sql.tungsten.enabled` to `false`. - - Parquet schema merging is no longer enabled by default. It can be re-enabled by setting - `spark.sql.parquet.mergeSchema` to `true`. - - Resolution of strings to columns in python now supports using dots (`.`) to qualify the column or - access nested values. For example `df['table.column.nestedField']`. However, this means that if - your column name contains any dots you must now escape them using backticks (e.g., ``table.`column.with.dots`.nested``). - - In-memory columnar storage partition pruning is on by default. It can be disabled by setting - `spark.sql.inMemoryColumnarStorage.partitionPruning` to `false`. - - Unlimited precision decimal columns are no longer supported, instead Spark SQL enforces a maximum - precision of 38. When inferring schema from `BigDecimal` objects, a precision of (38, 18) is now - used. When no precision is specified in DDL then the default remains `Decimal(10, 0)`. - - Timestamps are now stored at a precision of 1us, rather than 1ns - - In the `sql` dialect, floating point numbers are now parsed as decimal. HiveQL parsing remains - unchanged. - - The canonical name of SQL/DataFrame functions are now lower case (e.g., sum vs SUM). - - JSON data source will not automatically load new files that are created by other applications - (i.e. files that are not inserted to the dataset through Spark SQL). - For a JSON persistent table (i.e. the metadata of the table is stored in Hive Metastore), - users can use `REFRESH TABLE` SQL command or `HiveContext`'s `refreshTable` method - to include those new files to the table. For a DataFrame representing a JSON dataset, users need to recreate - the DataFrame and the new DataFrame will include new files. - - DataFrame.withColumn method in pySpark supports adding a new column or replacing existing columns of the same name. - -## Upgrading from Spark SQL 1.3 to 1.4 - -#### DataFrame data reader/writer interface - -Based on user feedback, we created a new, more fluid API for reading data in (`SQLContext.read`) -and writing data out (`DataFrame.write`), -and deprecated the old APIs (e.g., `SQLContext.parquetFile`, `SQLContext.jsonFile`). - -See the API docs for `SQLContext.read` ( - Scala, - Java, - Python -) and `DataFrame.write` ( - Scala, - Java, - Python -) more information. - - -#### DataFrame.groupBy retains grouping columns - -Based on user feedback, we changed the default behavior of `DataFrame.groupBy().agg()` to retain the -grouping columns in the resulting `DataFrame`. To keep the behavior in 1.3, set `spark.sql.retainGroupColumns` to `false`. - -
    -
    -{% highlight scala %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg($"department", max("age"), sum("expense")) - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")) - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    -{% highlight java %} - -// In 1.3.x, in order for the grouping column "department" to show up, -// it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(col("department"), max("age"), sum("expense")); - -// In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(max("age"), sum("expense")); - -// Revert to 1.3 behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false"); - -{% endhighlight %} -
    - -
    -{% highlight python %} - -import pyspark.sql.functions as func - -# In 1.3.x, in order for the grouping column "department" to show up, -# it must be included explicitly as part of the agg function call. -df.groupBy("department").agg(df["department"], func.max("age"), func.sum("expense")) - -# In 1.4+, grouping column "department" is included automatically. -df.groupBy("department").agg(func.max("age"), func.sum("expense")) - -# Revert to 1.3.x behavior (not retaining grouping column) by: -sqlContext.setConf("spark.sql.retainGroupColumns", "false") - -{% endhighlight %} -
    - -
    - - -#### Behavior change on DataFrame.withColumn - -Prior to 1.4, DataFrame.withColumn() supports adding a column only. The column will always be added -as a new column with its specified name in the result DataFrame even if there may be any existing -columns of the same name. Since 1.4, DataFrame.withColumn() supports adding a column of a different -name from names of all existing columns or replacing existing columns of the same name. - -Note that this change is only for Scala API, not for PySpark and SparkR. - - -## Upgrading from Spark SQL 1.0-1.2 to 1.3 - -In Spark 1.3 we removed the "Alpha" label from Spark SQL and as part of this did a cleanup of the -available APIs. From Spark 1.3 onwards, Spark SQL will provide binary compatibility with other -releases in the 1.X series. This compatibility guarantee excludes APIs that are explicitly marked -as unstable (i.e., DeveloperAPI or Experimental). - -#### Rename of SchemaRDD to DataFrame - -The largest change that users will notice when upgrading to Spark SQL 1.3 is that `SchemaRDD` has -been renamed to `DataFrame`. This is primarily because DataFrames no longer inherit from RDD -directly, but instead provide most of the functionality that RDDs provide though their own -implementation. DataFrames can still be converted to RDDs by calling the `.rdd` method. - -In Scala, there is a type alias from `SchemaRDD` to `DataFrame` to provide source compatibility for -some use cases. It is still recommended that users update their code to use `DataFrame` instead. -Java and Python users will need to update their code. - -#### Unification of the Java and Scala APIs - -Prior to Spark 1.3 there were separate Java compatible classes (`JavaSQLContext` and `JavaSchemaRDD`) -that mirrored the Scala API. In Spark 1.3 the Java API and Scala API have been unified. Users -of either language should use `SQLContext` and `DataFrame`. In general these classes try to -use types that are usable from both languages (i.e. `Array` instead of language-specific collections). -In some cases where no common type exists (e.g., for passing in closures or Maps) function overloading -is used instead. - -Additionally, the Java specific types API has been removed. Users of both Scala and Java should -use the classes present in `org.apache.spark.sql.types` to describe schema programmatically. - - -#### Isolation of Implicit Conversions and Removal of dsl Package (Scala-only) - -Many of the code examples prior to Spark 1.3 started with `import sqlContext._`, which brought -all of the functions from sqlContext into scope. In Spark 1.3 we have isolated the implicit -conversions for converting `RDD`s into `DataFrame`s into an object inside of the `SQLContext`. -Users should now write `import sqlContext.implicits._`. - -Additionally, the implicit conversions now only augment RDDs that are composed of `Product`s (i.e., -case classes or tuples) with a method `toDF`, instead of applying automatically. - -When using function inside of the DSL (now replaced with the `DataFrame` API) users used to import -`org.apache.spark.sql.catalyst.dsl`. Instead the public dataframe functions API should be used: -`import org.apache.spark.sql.functions._`. - -#### Removal of the type aliases in org.apache.spark.sql for DataType (Scala-only) - -Spark 1.3 removes the type aliases that were present in the base sql package for `DataType`. Users -should instead import the classes in `org.apache.spark.sql.types` - -#### UDF Registration Moved to `sqlContext.udf` (Java & Scala) - -Functions that are used to register UDFs, either for use in the DataFrame DSL or SQL, have been -moved into the udf object in `SQLContext`. - -
    -
    -{% highlight scala %} - -sqlContext.udf.register("strLen", (s: String) => s.length()) - -{% endhighlight %} -
    - -
    -{% highlight java %} - -sqlContext.udf().register("strLen", (String s) -> s.length(), DataTypes.IntegerType); - -{% endhighlight %} -
    - -
    - -Python UDF registration is unchanged. - -#### Python DataTypes No Longer Singletons - -When using DataTypes in Python you will need to construct them (i.e. `StringType()`) instead of -referencing a singleton. - -## Compatibility with Apache Hive - -Spark SQL is designed to be compatible with the Hive Metastore, SerDes and UDFs. -Currently, Hive SerDes and UDFs are based on Hive 1.2.1, -and Spark SQL can be connected to different versions of Hive Metastore -(from 0.12.0 to 2.3.3. Also see [Interacting with Different Versions of Hive Metastore](#interacting-with-different-versions-of-hive-metastore)). - -#### Deploying in Existing Hive Warehouses - -The Spark SQL Thrift JDBC server is designed to be "out of the box" compatible with existing Hive -installations. You do not need to modify your existing Hive Metastore or change the data placement -or partitioning of your tables. - -### Supported Hive Features - -Spark SQL supports the vast majority of Hive features, such as: - -* Hive query statements, including: - * `SELECT` - * `GROUP BY` - * `ORDER BY` - * `CLUSTER BY` - * `SORT BY` -* All Hive operators, including: - * Relational operators (`=`, `⇔`, `==`, `<>`, `<`, `>`, `>=`, `<=`, etc) - * Arithmetic operators (`+`, `-`, `*`, `/`, `%`, etc) - * Logical operators (`AND`, `&&`, `OR`, `||`, etc) - * Complex type constructors - * Mathematical functions (`sign`, `ln`, `cos`, etc) - * String functions (`instr`, `length`, `printf`, etc) -* User defined functions (UDF) -* User defined aggregation functions (UDAF) -* User defined serialization formats (SerDes) -* Window functions -* Joins - * `JOIN` - * `{LEFT|RIGHT|FULL} OUTER JOIN` - * `LEFT SEMI JOIN` - * `CROSS JOIN` -* Unions -* Sub-queries - * `SELECT col FROM ( SELECT a + b AS col from t1) t2` -* Sampling -* Explain -* Partitioned tables including dynamic partition insertion -* View -* All Hive DDL Functions, including: - * `CREATE TABLE` - * `CREATE TABLE AS SELECT` - * `ALTER TABLE` -* Most Hive Data types, including: - * `TINYINT` - * `SMALLINT` - * `INT` - * `BIGINT` - * `BOOLEAN` - * `FLOAT` - * `DOUBLE` - * `STRING` - * `BINARY` - * `TIMESTAMP` - * `DATE` - * `ARRAY<>` - * `MAP<>` - * `STRUCT<>` - -### Unsupported Hive Functionality - -Below is a list of Hive features that we don't support yet. Most of these features are rarely used -in Hive deployments. - -**Major Hive Features** - -* Tables with buckets: bucket is the hash partitioning within a Hive table partition. Spark SQL - doesn't support buckets yet. - - -**Esoteric Hive Features** - -* `UNION` type -* Unique join -* Column statistics collecting: Spark SQL does not piggyback scans to collect column statistics at - the moment and only supports populating the sizeInBytes field of the hive metastore. - -**Hive Input/Output Formats** - -* File format for CLI: For results showing back to the CLI, Spark SQL only supports TextOutputFormat. -* Hadoop archive - -**Hive Optimizations** - -A handful of Hive optimizations are not yet included in Spark. Some of these (such as indexes) are -less important due to Spark SQL's in-memory computational model. Others are slotted for future -releases of Spark SQL. - -* Block-level bitmap indexes and virtual columns (used to build indexes) -* Automatically determine the number of reducers for joins and groupbys: Currently, in Spark SQL, you - need to control the degree of parallelism post-shuffle using "`SET spark.sql.shuffle.partitions=[num_tasks];`". -* Meta-data only query: For queries that can be answered by using only metadata, Spark SQL still - launches tasks to compute the result. -* Skew data flag: Spark SQL does not follow the skew data flags in Hive. -* `STREAMTABLE` hint in join: Spark SQL does not follow the `STREAMTABLE` hint. -* Merge multiple small files for query results: if the result output contains multiple small files, - Hive can optionally merge the small files into fewer large files to avoid overflowing the HDFS - metadata. Spark SQL does not support that. - -**Hive UDF/UDTF/UDAF** - -Not all the APIs of the Hive UDF/UDTF/UDAF are supported by Spark SQL. Below are the unsupported APIs: - -* `getRequiredJars` and `getRequiredFiles` (`UDF` and `GenericUDF`) are functions to automatically - include additional resources required by this UDF. -* `initialize(StructObjectInspector)` in `GenericUDTF` is not supported yet. Spark SQL currently uses - a deprecated interface `initialize(ObjectInspector[])` only. -* `configure` (`GenericUDF`, `GenericUDTF`, and `GenericUDAFEvaluator`) is a function to initialize - functions with `MapredContext`, which is inapplicable to Spark. -* `close` (`GenericUDF` and `GenericUDAFEvaluator`) is a function to release associated resources. - Spark SQL does not call this function when tasks finish. -* `reset` (`GenericUDAFEvaluator`) is a function to re-initialize aggregation for reusing the same aggregation. - Spark SQL currently does not support the reuse of aggregation. -* `getWindowingEvaluator` (`GenericUDAFEvaluator`) is a function to optimize aggregation by evaluating - an aggregate over a fixed window. - -### Incompatible Hive UDF - -Below are the scenarios in which Hive and Spark generate different results: - -* `SQRT(n)` If n < 0, Hive returns null, Spark SQL returns NaN. -* `ACOS(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. -* `ASIN(n)` If n < -1 or n > 1, Hive returns null, Spark SQL returns NaN. - -# Reference - -## Data Types - -Spark SQL and DataFrames support the following data types: - -* Numeric types - - `ByteType`: Represents 1-byte signed integer numbers. - The range of numbers is from `-128` to `127`. - - `ShortType`: Represents 2-byte signed integer numbers. - The range of numbers is from `-32768` to `32767`. - - `IntegerType`: Represents 4-byte signed integer numbers. - The range of numbers is from `-2147483648` to `2147483647`. - - `LongType`: Represents 8-byte signed integer numbers. - The range of numbers is from `-9223372036854775808` to `9223372036854775807`. - - `FloatType`: Represents 4-byte single-precision floating point numbers. - - `DoubleType`: Represents 8-byte double-precision floating point numbers. - - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. -* String type - - `StringType`: Represents character string values. -* Binary type - - `BinaryType`: Represents byte sequence values. -* Boolean type - - `BooleanType`: Represents boolean values. -* Datetime type - - `TimestampType`: Represents values comprising values of fields year, month, day, - hour, minute, and second. - - `DateType`: Represents values comprising values of fields year, month, day. -* Complex types - - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of - elements with the type of `elementType`. `containsNull` is used to indicate if - elements in a `ArrayType` value can have `null` values. - - `MapType(keyType, valueType, valueContainsNull)`: - Represents values comprising a set of key-value pairs. The data type of keys are - described by `keyType` and the data type of values are described by `valueType`. - For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` - is used to indicate if values of a `MapType` value can have `null` values. - - `StructType(fields)`: Represents values with the structure described by - a sequence of `StructField`s (`fields`). - * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. - The name of a field is indicated by `name`. The data type of a field is indicated - by `dataType`. `nullable` is used to indicate if values of this fields can have - `null` values. - -
    -
    - -All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. -You can access them by doing - -{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte - ByteType -
    ShortType Short - ShortType -
    IntegerType Int - IntegerType -
    LongType Long - LongType -
    FloatType Float - FloatType -
    DoubleType Double - DoubleType -
    DecimalType java.math.BigDecimal - DecimalType -
    StringType String - StringType -
    BinaryType Array[Byte] - BinaryType -
    BooleanType Boolean - BooleanType -
    TimestampType java.sql.Timestamp - TimestampType -
    DateType java.sql.Date - DateType -
    ArrayType scala.collection.Seq - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is true. -
    MapType scala.collection.Map - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is true. -
    StructType org.apache.spark.sql.Row - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Scala of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is true. -
    - -
    - -
    - -All data types of Spark SQL are located in the package of -`org.apache.spark.sql.types`. To access or create a data type, -please use factory methods provided in -`org.apache.spark.sql.types.DataTypes`. - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte - DataTypes.ByteType -
    ShortType short or Short - DataTypes.ShortType -
    IntegerType int or Integer - DataTypes.IntegerType -
    LongType long or Long - DataTypes.LongType -
    FloatType float or Float - DataTypes.FloatType -
    DoubleType double or Double - DataTypes.DoubleType -
    DecimalType java.math.BigDecimal - DataTypes.createDecimalType()
    - DataTypes.createDecimalType(precision, scale). -
    StringType String - DataTypes.StringType -
    BinaryType byte[] - DataTypes.BinaryType -
    BooleanType boolean or Boolean - DataTypes.BooleanType -
    TimestampType java.sql.Timestamp - DataTypes.TimestampType -
    DateType java.sql.Date - DataTypes.DateType -
    ArrayType java.util.List - DataTypes.createArrayType(elementType)
    - Note: The value of containsNull will be true
    - DataTypes.createArrayType(elementType, containsNull). -
    MapType java.util.Map - DataTypes.createMapType(keyType, valueType)
    - Note: The value of valueContainsNull will be true.
    - DataTypes.createMapType(keyType, valueType, valueContainsNull)
    -
    StructType org.apache.spark.sql.Row - DataTypes.createStructType(fields)
    - Note: fields is a List or an array of StructFields. - Also, two fields with the same name are not allowed. -
    StructField The value type in Java of the data type of this field - (For example, int for a StructField with the data type IntegerType) - DataTypes.createStructField(name, dataType, nullable) -
    - -
    - -
    - -All data types of Spark SQL are located in the package of `pyspark.sql.types`. -You can access them by doing -{% highlight python %} -from pyspark.sql.types import * -{% endhighlight %} - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in PythonAPI to access or create a data type
    ByteType - int or long
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - ByteType() -
    ShortType - int or long
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - ShortType() -
    IntegerType int or long - IntegerType() -
    LongType - long
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - LongType() -
    FloatType - float
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - FloatType() -
    DoubleType float - DoubleType() -
    DecimalType decimal.Decimal - DecimalType() -
    StringType string - StringType() -
    BinaryType bytearray - BinaryType() -
    BooleanType bool - BooleanType() -
    TimestampType datetime.datetime - TimestampType() -
    DateType datetime.date - DateType() -
    ArrayType list, tuple, or array - ArrayType(elementType, [containsNull])
    - Note: The default value of containsNull is True. -
    MapType dict - MapType(keyType, valueType, [valueContainsNull])
    - Note: The default value of valueContainsNull is True. -
    StructType list or tuple - StructType(fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in Python of the data type of this field - (For example, Int for a StructField with the data type IntegerType) - StructField(name, dataType, [nullable])
    - Note: The default value of nullable is True. -
    - -
    - -
    - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    Data typeValue type in RAPI to access or create a data type
    ByteType - integer
    - Note: Numbers will be converted to 1-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -128 to 127. -
    - "byte" -
    ShortType - integer
    - Note: Numbers will be converted to 2-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of -32768 to 32767. -
    - "short" -
    IntegerType integer - "integer" -
    LongType - integer
    - Note: Numbers will be converted to 8-byte signed integer numbers at runtime. - Please make sure that numbers are within the range of - -9223372036854775808 to 9223372036854775807. - Otherwise, please convert data to decimal.Decimal and use DecimalType. -
    - "long" -
    FloatType - numeric
    - Note: Numbers will be converted to 4-byte single-precision floating - point numbers at runtime. -
    - "float" -
    DoubleType numeric - "double" -
    DecimalType Not supported - Not supported -
    StringType character - "string" -
    BinaryType raw - "binary" -
    BooleanType logical - "bool" -
    TimestampType POSIXct - "timestamp" -
    DateType Date - "date" -
    ArrayType vector or list - list(type="array", elementType=elementType, containsNull=[containsNull])
    - Note: The default value of containsNull is TRUE. -
    MapType environment - list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    - Note: The default value of valueContainsNull is TRUE. -
    StructType named list - list(type="struct", fields=fields)
    - Note: fields is a Seq of StructFields. Also, two fields with the same - name are not allowed. -
    StructField The value type in R of the data type of this field - (For example, integer for a StructField with the data type IntegerType) - list(name=name, type=dataType, nullable=[nullable])
    - Note: The default value of nullable is TRUE. -
    - -
    - -
    - -## NaN Semantics - -There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that -does not exactly match standard floating point semantics. -Specifically: - - - NaN = NaN returns true. - - In aggregations, all NaN values are grouped together. - - NaN is treated as a normal value in join keys. - - NaN values go last when in ascending order, larger than any other numeric value. - - ## Arithmetic operations - -Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. -This means that in case an operation causes an overflow, the result is the same that the same operation -returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, -the result is a negative number). diff --git a/docs/sql-pyspark-pandas-with-arrow.md b/docs/sql-pyspark-pandas-with-arrow.md new file mode 100644 index 0000000000000..c4d22d520feaf --- /dev/null +++ b/docs/sql-pyspark-pandas-with-arrow.md @@ -0,0 +1,221 @@ +--- +layout: global +title: PySpark Usage Guide for Pandas with Apache Arrow +displayTitle: PySpark Usage Guide for Pandas with Apache Arrow +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. +--- + +* Table of contents +{:toc} + +## Apache Arrow in PySpark + +Apache Arrow is an in-memory columnar data format that is used in Spark to efficiently transfer +data between JVM and Python processes. This currently is most beneficial to Python users that +work with Pandas/NumPy data. Its usage is not automatic and might require some minor +changes to configuration or code to take full advantage and ensure compatibility. This guide will +give a high-level description of how to use Arrow in Spark and highlight any differences when +working with Arrow-enabled data. + +### Ensure PyArrow Installed + +If you install PySpark using pip, then PyArrow can be brought in as an extra dependency of the +SQL module with the command `pip install pyspark[sql]`. Otherwise, you must ensure that PyArrow +is installed and available on all cluster nodes. The current supported version is 0.12.1. +You can install using pip or conda from the conda-forge channel. See PyArrow +[installation](https://arrow.apache.org/docs/python/install.html) for details. + +## Enabling for Conversion to/from Pandas + +Arrow is available as an optimization when converting a Spark DataFrame to a Pandas DataFrame +using the call `toPandas()` and when creating a Spark DataFrame from a Pandas DataFrame with +`createDataFrame(pandas_df)`. To use Arrow when executing these calls, users need to first set +the Spark configuration 'spark.sql.execution.arrow.pyspark.enabled' to 'true'. This is disabled by default. + +In addition, optimizations enabled by 'spark.sql.execution.arrow.pyspark.enabled' could fallback automatically +to non-Arrow optimization implementation if an error occurs before the actual computation within Spark. +This can be controlled by 'spark.sql.execution.arrow.pyspark.fallback.enabled'. + +
    +
    +{% include_example dataframe_with_arrow python/sql/arrow.py %} +
    +
    + +Using the above optimizations with Arrow will produce the same results as when Arrow is not +enabled. Note that even with Arrow, `toPandas()` results in the collection of all records in the +DataFrame to the driver program and should be done on a small subset of the data. Not all Spark +data types are currently supported and an error can be raised if a column has an unsupported type, +see [Supported SQL Types](#supported-sql-types). If an error occurs during `createDataFrame()`, +Spark will fall back to create the DataFrame without Arrow. + +## Pandas UDFs (a.k.a. Vectorized UDFs) + +Pandas UDFs are user defined functions that are executed by Spark using Arrow to transfer data and +Pandas to work with the data. A Pandas UDF is defined using the keyword `pandas_udf` as a decorator +or to wrap the function, no additional configuration is required. Currently, there are two types of +Pandas UDF: Scalar and Grouped Map. + +### Scalar + +Scalar Pandas UDFs are used for vectorizing scalar operations. They can be used with functions such +as `select` and `withColumn`. The Python function should take `pandas.Series` as inputs and return +a `pandas.Series` of the same length. Internally, Spark 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 the results together. + +The following example shows how to create a scalar Pandas UDF that computes the product of 2 columns. + +
    +
    +{% include_example scalar_pandas_udf python/sql/arrow.py %} +
    +
    + +### Scalar Iterator + +Scalar iterator (`SCALAR_ITER`) Pandas UDF is the same as scalar Pandas UDF above except that the +underlying Python function takes an iterator of batches as input instead of a single batch and, +instead of returning a single output batch, it yields output batches or returns an iterator of +output batches. +It is useful when the UDF execution requires initializing some states, e.g., loading an machine +learning model file to apply inference to every input batch. + +The following example shows how to create scalar iterator Pandas UDFs: + +
    +
    +{% include_example scalar_iter_pandas_udf python/sql/arrow.py %} +
    +
    + +### Grouped Map +Grouped map Pandas UDFs are used with `groupBy().apply()` which implements the "split-apply-combine" pattern. +Split-apply-combine consists of three steps: +* Split the data into groups by using `DataFrame.groupBy`. +* Apply a function on each group. The input and output of the function are both `pandas.DataFrame`. The + input data contains all the rows and columns for each group. +* Combine the results into a new `DataFrame`. + +To use `groupBy().apply()`, the user needs to define the following: +* A Python function that defines the computation for each group. +* A `StructType` object or a string that defines the schema of the output `DataFrame`. + +The column labels of the returned `pandas.DataFrame` must either match the field names in the +defined output schema if specified as strings, or match the field data types by position if not +strings, e.g. integer indices. See [pandas.DataFrame](https://pandas.pydata.org/pandas-docs/stable/generated/pandas.DataFrame.html#pandas.DataFrame) +on how to label columns when constructing a `pandas.DataFrame`. + +Note that all data for a group will be loaded into memory before the function is applied. This can +lead to out of memory exceptions, especially if the group sizes are skewed. The configuration for +[maxRecordsPerBatch](#setting-arrow-batch-size) is not applied on groups and it is up to the user +to ensure that the grouped data will fit into the available memory. + +The following example shows how to use `groupby().apply()` to subtract the mean from each value in the group. + +
    +
    +{% include_example grouped_map_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and +[`pyspark.sql.GroupedData.apply`](api/python/pyspark.sql.html#pyspark.sql.GroupedData.apply). + +### Grouped Aggregate + +Grouped aggregate Pandas UDFs are similar to Spark aggregate functions. Grouped aggregate Pandas UDFs are used with `groupBy().agg()` and +[`pyspark.sql.Window`](api/python/pyspark.sql.html#pyspark.sql.Window). It defines an aggregation from one or more `pandas.Series` +to a scalar value, where each `pandas.Series` represents a column within the group or window. + +Note that this type of UDF does not support partial aggregation and all data for a group or window will be loaded into memory. Also, +only unbounded window is supported with Grouped aggregate Pandas UDFs currently. + +The following example shows how to use this type of UDF to compute mean with groupBy and window operations: + +
    +
    +{% include_example grouped_agg_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) + + +### Map Iterator + +Map iterator Pandas UDFs are used to transform data with an iterator of batches. Map iterator +Pandas UDFs can be used with +[`pyspark.sql.DataFrame.mapInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). +It defines a map function that transforms an iterator of `pandas.DataFrame` to another. + +It can return the output of arbitrary length in contrast to the scalar Pandas UDF. It maps an iterator of `pandas.DataFrame`s, +that represents the current `DataFrame`, using the map iterator UDF and returns the result as a `DataFrame`. + +The following example shows how to create map iterator Pandas UDFs: + +
    +
    +{% include_example map_iter_pandas_udf python/sql/arrow.py %} +
    +
    + +For detailed usage, please see [`pyspark.sql.functions.pandas_udf`](api/python/pyspark.sql.html#pyspark.sql.functions.pandas_udf) and +[`pyspark.sql.DataFrame.mapsInPandas`](api/python/pyspark.sql.html#pyspark.sql.DataFrame.mapInPandas). + + +## Usage Notes + +### Supported SQL Types + +Currently, all Spark SQL data types are supported by Arrow-based conversion except `MapType`, +`ArrayType` of `TimestampType`, and nested `StructType`. + +### Setting Arrow Batch Size + +Data partitions in Spark are converted into Arrow record batches, which can temporarily lead to +high memory usage in the JVM. To avoid possible out of memory exceptions, the size of the Arrow +record batches can be adjusted by setting the conf "spark.sql.execution.arrow.maxRecordsPerBatch" +to an integer that will determine the maximum number of rows for each batch. The default value is +10,000 records per batch. If the number of columns is large, the value should be adjusted +accordingly. Using this limit, each data partition will be made into 1 or more record batches for +processing. + +### Timestamp with Time Zone Semantics + +Spark internally stores timestamps as UTC values, and timestamp data that is brought in without +a specified time zone is converted as local time to UTC with microsecond resolution. When timestamp +data is exported or displayed in Spark, the session time zone is used to localize the timestamp +values. The session time zone is set with the configuration 'spark.sql.session.timeZone' and will +default to the JVM system local time zone if not set. Pandas uses a `datetime64` type with nanosecond +resolution, `datetime64[ns]`, with optional time zone on a per-column basis. + +When timestamp data is transferred from Spark to Pandas it will be converted to nanoseconds +and each column will be converted to the Spark session time zone then localized to that time +zone, which removes the time zone and displays values as local time. This will occur +when calling `toPandas()` or `pandas_udf` with timestamp columns. + +When timestamp data is transferred from Pandas to Spark, it will be converted to UTC microseconds. This +occurs when calling `createDataFrame` with a Pandas DataFrame or when returning a timestamp from a +`pandas_udf`. These conversions are done automatically to ensure Spark will have data in the +expected format, so it is not necessary to do any of these conversions yourself. Any nanosecond +values will be truncated. + +Note that a standard UDF (non-Pandas) will load timestamp data as Python datetime objects, which is +different than a Pandas timestamp. It is recommended to use Pandas time series functionality when +working with timestamps in `pandas_udf`s to get the best performance, see +[here](https://pandas.pydata.org/pandas-docs/stable/timeseries.html) for details. diff --git a/docs/sql-reference.md b/docs/sql-reference.md new file mode 100644 index 0000000000000..2ec26ecc2e1e9 --- /dev/null +++ b/docs/sql-reference.md @@ -0,0 +1,658 @@ +--- +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. +--- + +* Table of contents +{:toc} + +## Data Types + +Spark SQL and DataFrames support the following data types: + +* Numeric types + - `ByteType`: Represents 1-byte signed integer numbers. + The range of numbers is from `-128` to `127`. + - `ShortType`: Represents 2-byte signed integer numbers. + The range of numbers is from `-32768` to `32767`. + - `IntegerType`: Represents 4-byte signed integer numbers. + The range of numbers is from `-2147483648` to `2147483647`. + - `LongType`: Represents 8-byte signed integer numbers. + The range of numbers is from `-9223372036854775808` to `9223372036854775807`. + - `FloatType`: Represents 4-byte single-precision floating point numbers. + - `DoubleType`: Represents 8-byte double-precision floating point numbers. + - `DecimalType`: Represents arbitrary-precision signed decimal numbers. Backed internally by `java.math.BigDecimal`. A `BigDecimal` consists of an arbitrary precision integer unscaled value and a 32-bit integer scale. +* String type + - `StringType`: Represents character string values. +* Binary type + - `BinaryType`: Represents byte sequence values. +* Boolean type + - `BooleanType`: Represents boolean values. +* Datetime type + - `TimestampType`: Represents values comprising values of fields year, month, day, + hour, minute, and second, with the session local time-zone. The timestamp value represents an + absolute point in time. + - `DateType`: Represents values comprising values of fields year, month and day, without a + time-zone. +* Complex types + - `ArrayType(elementType, containsNull)`: Represents values comprising a sequence of + elements with the type of `elementType`. `containsNull` is used to indicate if + elements in a `ArrayType` value can have `null` values. + - `MapType(keyType, valueType, valueContainsNull)`: + Represents values comprising a set of key-value pairs. The data type of keys is + described by `keyType` and the data type of values is described by `valueType`. + For a `MapType` value, keys are not allowed to have `null` values. `valueContainsNull` + is used to indicate if values of a `MapType` value can have `null` values. + - `StructType(fields)`: Represents values with the structure described by + a sequence of `StructField`s (`fields`). + * `StructField(name, dataType, nullable)`: Represents a field in a `StructType`. + The name of a field is indicated by `name`. The data type of a field is indicated + by `dataType`. `nullable` is used to indicate if values of these fields can have + `null` values. + +
    +
    + +All data types of Spark SQL are located in the package `org.apache.spark.sql.types`. +You can access them by doing + +{% include_example data_types scala/org/apache/spark/examples/sql/SparkSQLExample.scala %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in ScalaAPI to access or create a data type
    ByteType Byte + ByteType +
    ShortType Short + ShortType +
    IntegerType Int + IntegerType +
    LongType Long + LongType +
    FloatType Float + FloatType +
    DoubleType Double + DoubleType +
    DecimalType java.math.BigDecimal + DecimalType +
    StringType String + StringType +
    BinaryType Array[Byte] + BinaryType +
    BooleanType Boolean + BooleanType +
    TimestampType java.sql.Timestamp + TimestampType +
    DateType java.sql.Date + DateType +
    ArrayType scala.collection.Seq + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is true. +
    MapType scala.collection.Map + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is true. +
    StructType org.apache.spark.sql.Row + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Scala of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is true. +
    + +
    + +
    + +All data types of Spark SQL are located in the package of +`org.apache.spark.sql.types`. To access or create a data type, +please use factory methods provided in +`org.apache.spark.sql.types.DataTypes`. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in JavaAPI to access or create a data type
    ByteType byte or Byte + DataTypes.ByteType +
    ShortType short or Short + DataTypes.ShortType +
    IntegerType int or Integer + DataTypes.IntegerType +
    LongType long or Long + DataTypes.LongType +
    FloatType float or Float + DataTypes.FloatType +
    DoubleType double or Double + DataTypes.DoubleType +
    DecimalType java.math.BigDecimal + DataTypes.createDecimalType()
    + DataTypes.createDecimalType(precision, scale). +
    StringType String + DataTypes.StringType +
    BinaryType byte[] + DataTypes.BinaryType +
    BooleanType boolean or Boolean + DataTypes.BooleanType +
    TimestampType java.sql.Timestamp + DataTypes.TimestampType +
    DateType java.sql.Date + DataTypes.DateType +
    ArrayType java.util.List + DataTypes.createArrayType(elementType)
    + Note: The value of containsNull will be true
    + DataTypes.createArrayType(elementType, containsNull). +
    MapType java.util.Map + DataTypes.createMapType(keyType, valueType)
    + Note: The value of valueContainsNull will be true.
    + DataTypes.createMapType(keyType, valueType, valueContainsNull)
    +
    StructType org.apache.spark.sql.Row + DataTypes.createStructType(fields)
    + Note: fields is a List or an array of StructFields. + Also, two fields with the same name are not allowed. +
    StructField The value type in Java of the data type of this field + (For example, int for a StructField with the data type IntegerType) + DataTypes.createStructField(name, dataType, nullable) +
    + +
    + +
    + +All data types of Spark SQL are located in the package of `pyspark.sql.types`. +You can access them by doing +{% highlight python %} +from pyspark.sql.types import * +{% endhighlight %} + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in PythonAPI to access or create a data type
    ByteType + int or long
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + ByteType() +
    ShortType + int or long
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + ShortType() +
    IntegerType int or long + IntegerType() +
    LongType + long
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + LongType() +
    FloatType + float
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + FloatType() +
    DoubleType float + DoubleType() +
    DecimalType decimal.Decimal + DecimalType() +
    StringType string + StringType() +
    BinaryType bytearray + BinaryType() +
    BooleanType bool + BooleanType() +
    TimestampType datetime.datetime + TimestampType() +
    DateType datetime.date + DateType() +
    ArrayType list, tuple, or array + ArrayType(elementType, [containsNull])
    + Note: The default value of containsNull is True. +
    MapType dict + MapType(keyType, valueType, [valueContainsNull])
    + Note: The default value of valueContainsNull is True. +
    StructType list or tuple + StructType(fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in Python of the data type of this field + (For example, Int for a StructField with the data type IntegerType) + StructField(name, dataType, [nullable])
    + Note: The default value of nullable is True. +
    + +
    + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Data typeValue type in RAPI to access or create a data type
    ByteType + integer
    + Note: Numbers will be converted to 1-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -128 to 127. +
    + "byte" +
    ShortType + integer
    + Note: Numbers will be converted to 2-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of -32768 to 32767. +
    + "short" +
    IntegerType integer + "integer" +
    LongType + integer
    + Note: Numbers will be converted to 8-byte signed integer numbers at runtime. + Please make sure that numbers are within the range of + -9223372036854775808 to 9223372036854775807. + Otherwise, please convert data to decimal.Decimal and use DecimalType. +
    + "long" +
    FloatType + numeric
    + Note: Numbers will be converted to 4-byte single-precision floating + point numbers at runtime. +
    + "float" +
    DoubleType numeric + "double" +
    DecimalType Not supported + Not supported +
    StringType character + "string" +
    BinaryType raw + "binary" +
    BooleanType logical + "bool" +
    TimestampType POSIXct + "timestamp" +
    DateType Date + "date" +
    ArrayType vector or list + list(type="array", elementType=elementType, containsNull=[containsNull])
    + Note: The default value of containsNull is TRUE. +
    MapType environment + list(type="map", keyType=keyType, valueType=valueType, valueContainsNull=[valueContainsNull])
    + Note: The default value of valueContainsNull is TRUE. +
    StructType named list + list(type="struct", fields=fields)
    + Note: fields is a Seq of StructFields. Also, two fields with the same + name are not allowed. +
    StructField The value type in R of the data type of this field + (For example, integer for a StructField with the data type IntegerType) + list(name=name, type=dataType, nullable=[nullable])
    + Note: The default value of nullable is TRUE. +
    + +
    + +
    + +## NaN Semantics + +There is specially handling for not-a-number (NaN) when dealing with `float` or `double` types that +does not exactly match standard floating point semantics. +Specifically: + + - NaN = NaN returns true. + - In aggregations, all NaN values are grouped together. + - NaN is treated as a normal value in join keys. + - NaN values go last when in ascending order, larger than any other numeric value. + +## Arithmetic operations + +Operations performed on numeric types (with the exception of `decimal`) are not checked for overflow. +This means that in case an operation causes an overflow, the result is the same that the same operation +returns in a Java/Scala program (eg. if the sum of 2 integers is higher than the maximum value representable, +the result is a negative number). diff --git a/docs/storage-openstack-swift.md b/docs/storage-openstack-swift.md index dacaa3438d489..6bdcaa7ccbd70 100644 --- a/docs/storage-openstack-swift.md +++ b/docs/storage-openstack-swift.md @@ -1,6 +1,21 @@ --- layout: global title: Accessing OpenStack Swift from Spark +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's support for Hadoop InputFormat allows it to process data in OpenStack Swift using the @@ -29,7 +44,7 @@ For example, for Maven support, add the following to the pom.xml fi ... org.apache.spark - hadoop-cloud_2.11 + hadoop-cloud_2.12 ${spark.version} ... diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index a83ebd9449fa4..a37b961243f47 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -1,6 +1,21 @@ --- layout: global title: Spark Streaming Custom Receivers +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 Streaming can receive streaming data from any arbitrary data source beyond diff --git a/docs/streaming-kafka-0-10-integration.md b/docs/streaming-kafka-0-10-integration.md index 386066a85749f..d8fd6724e91bd 100644 --- a/docs/streaming-kafka-0-10-integration.md +++ b/docs/streaming-kafka-0-10-integration.md @@ -1,9 +1,27 @@ --- layout: global title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +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. --- -The Spark Streaming integration for Kafka 0.10 is similar in design to the 0.8 [Direct Stream approach](streaming-kafka-0-8-integration.html#approach-2-direct-approach-no-receivers). It provides simple parallelism, 1:1 correspondence between Kafka partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses the [new Kafka consumer API](http://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, there are notable differences in usage. This version of the integration is marked as experimental, so the API is potentially subject to change. +The Spark Streaming integration for Kafka 0.10 provides simple parallelism, 1:1 correspondence between Kafka +partitions and Spark partitions, and access to offsets and metadata. However, because the newer integration uses +the [new Kafka consumer API](https://kafka.apache.org/documentation.html#newconsumerapi) instead of the simple API, +there are notable differences in usage. ### Linking For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). @@ -93,7 +111,7 @@ In most cases, you should use `LocationStrategies.PreferConsistent` as shown abo The cache for consumers has a default maximum size of 64. If you expect to be handling more than (64 * number of executors) Kafka partitions, you can change this setting via `spark.streaming.kafka.consumer.cache.maxCapacity`. -If you would like to disable the caching for Kafka consumers, you can set `spark.streaming.kafka.consumer.cache.enabled` to `false`. Disabling the cache may be needed to workaround the problem described in SPARK-19185. This property may be removed in later versions of Spark, once SPARK-19185 is resolved. +If you would like to disable the caching for Kafka consumers, you can set `spark.streaming.kafka.consumer.cache.enabled` to `false`. The cache is keyed by topicpartition and group.id, so use a **separate** `group.id` for each call to `createDirectStream`. @@ -312,3 +330,10 @@ As with any Spark applications, `spark-submit` is used to launch your applicatio For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-10_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). +### Security + +See [Structured Streaming Security](structured-streaming-kafka-integration.html#security). + +##### Additional Caveats + +- Kafka native sink is not available so delegation token used only on consumer side. diff --git a/docs/streaming-kafka-0-8-integration.md b/docs/streaming-kafka-0-8-integration.md deleted file mode 100644 index becf217738d26..0000000000000 --- a/docs/streaming-kafka-0-8-integration.md +++ /dev/null @@ -1,196 +0,0 @@ ---- -layout: global -title: Spark Streaming + Kafka Integration Guide (Kafka broker version 0.8.2.1 or higher) ---- - -**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** - -Here we explain how to configure Spark Streaming to receive data from Kafka. There are two approaches to this - the old approach using Receivers and Kafka's high-level API, and a new approach (introduced in Spark 1.3) without using Receivers. They have different programming models, performance characteristics, and semantics guarantees, so read on for more details. Both approaches are considered stable APIs as of the current version of Spark. - -## Approach 1: Receiver-based Approach -This approach uses a Receiver to receive the data. The Receiver is implemented using the Kafka high-level consumer API. As with all receivers, the data received from Kafka through a Receiver is stored in Spark executors, and then jobs launched by Spark Streaming processes the data. - -However, under default configuration, this approach can lose data under failures (see [receiver reliability](streaming-programming-guide.html#receiver-reliability). To ensure zero-data loss, you have to additionally enable Write-Ahead Logs in Spark Streaming (introduced in Spark 1.2). This synchronously saves all the received Kafka data into write-ahead logs on a distributed file system (e.g HDFS), so that all the data can be recovered on failure. See [Deploying section](streaming-programming-guide.html#deploying-applications) in the streaming programming guide for more details on Write-Ahead Logs. - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** For Scala/Java applications using SBT/Maven project definitions, link your streaming application with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - - For Python applications, you will have to add this above library and its dependencies when deploying your application. See the *Deploying* subsection below. - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val kafkaStream = KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairReceiverInputDStream kafkaStream = - KafkaUtils.createStream(streamingContext, - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]); - - You can also specify the key and value classes and their corresponding decoder classes using variations of `createStream`. See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - - kafkaStream = KafkaUtils.createStream(streamingContext, \ - [ZK quorum], [consumer group id], [per-topic number of Kafka partitions to consume]) - - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). -
    -
    - - **Points to remember:** - - - Topic partitions in Kafka do not correlate to partitions of RDDs generated in Spark Streaming. So increasing the number of topic-specific partitions in the `KafkaUtils.createStream()` only increases the number of threads using which topics that are consumed within a single receiver. It does not increase the parallelism of Spark in processing the data. Refer to the main document for more information on that. - - - Multiple Kafka input DStreams can be created with different groups and topics for parallel receiving of data using multiple receivers. - - - If you have enabled Write-Ahead Logs with a replicated file system like HDFS, the received data is already being replicated in the log. Hence, the storage level in storage level for the input stream to `StorageLevel.MEMORY_AND_DISK_SER` (that is, use -`KafkaUtils.createStream(..., StorageLevel.MEMORY_AND_DISK_SER)`). - -3. **Deploying:** As with any Spark applications, `spark-submit` is used to launch your application. However, the details are slightly different for Scala/Java applications and Python applications. - - For Scala and Java applications, if you are using SBT or Maven for project management, then package `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies into the application JAR. Make sure `spark-core_{{site.SCALA_BINARY_VERSION}}` and `spark-streaming_{{site.SCALA_BINARY_VERSION}}` are marked as `provided` dependencies as those are already present in a Spark installation. Then use `spark-submit` to launch your application (see [Deploying section](streaming-programming-guide.html#deploying-applications) in the main programming guide). - - For Python applications which lack SBT/Maven project management, `spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}` and its dependencies can be directly added to `spark-submit` using `--packages` (see [Application Submission Guide](submitting-applications.html)). That is, - - ./bin/spark-submit --packages org.apache.spark:spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}}:{{site.SPARK_VERSION_SHORT}} ... - - Alternatively, you can also download the JAR of the Maven artifact `spark-streaming-kafka-0-8-assembly` from the - [Maven repository](https://search.maven.org/#search|ga|1|a%3A%22spark-streaming-kafka-0-8-assembly_{{site.SCALA_BINARY_VERSION}}%22%20AND%20v%3A%22{{site.SPARK_VERSION_SHORT}}%22) and add it to `spark-submit` with `--jars`. - -## Approach 2: Direct Approach (No Receivers) -This new receiver-less "direct" approach has been introduced in Spark 1.3 to ensure stronger end-to-end guarantees. Instead of using receivers to receive data, this approach periodically queries Kafka for the latest offsets in each topic+partition, and accordingly defines the offset ranges to process in each batch. When the jobs to process the data are launched, Kafka's simple consumer API is used to read the defined ranges of offsets from Kafka (similar to read files from a file system). Note that this feature was introduced in Spark 1.3 for the Scala and Java API, in Spark 1.4 for the Python API. - -This approach has the following advantages over the receiver-based approach (i.e. Approach 1). - -- *Simplified Parallelism:* No need to create multiple input Kafka streams and union them. With `directStream`, Spark Streaming will create as many RDD partitions as there are Kafka partitions to consume, which will all read data from Kafka in parallel. So there is a one-to-one mapping between Kafka and RDD partitions, which is easier to understand and tune. - -- *Efficiency:* Achieving zero-data loss in the first approach required the data to be stored in a Write-Ahead Log, which further replicated the data. This is actually inefficient as the data effectively gets replicated twice - once by Kafka, and a second time by the Write-Ahead Log. This second approach eliminates the problem as there is no receiver, and hence no need for Write-Ahead Logs. As long as you have sufficient Kafka retention, messages can be recovered from Kafka. - -- *Exactly-once semantics:* The first approach uses Kafka's high-level API to store consumed offsets in Zookeeper. This is traditionally the way to consume data from Kafka. While this approach (in combination with-write-ahead logs) can ensure zero data loss (i.e. at-least once semantics), there is a small chance some records may get consumed twice under some failures. This occurs because of inconsistencies between data reliably received by Spark Streaming and offsets tracked by Zookeeper. Hence, in this second approach, we use simple Kafka API that does not use Zookeeper. Offsets are tracked by Spark Streaming within its checkpoints. This eliminates inconsistencies between Spark Streaming and Zookeeper/Kafka, and so each record is received by Spark Streaming effectively exactly once despite failures. In order to achieve exactly-once semantics for output of your results, your output operation that saves the data to an external data store must be either idempotent, or an atomic transaction that saves results and offsets (see [Semantics of output operations](streaming-programming-guide.html#semantics-of-output-operations) in the main programming guide for further information). - -Note that one disadvantage of this approach is that it does not update offsets in Zookeeper, hence Zookeeper-based Kafka monitoring tools will not show progress. However, you can access the offsets processed by this approach in each batch and update Zookeeper yourself (see below). - -Next, we discuss how to use this approach in your streaming application. - -1. **Linking:** This approach is supported only in Scala/Java application. Link your SBT/Maven project with the following artifact (see [Linking section](streaming-programming-guide.html#linking) in the main programming guide for further information). - - groupId = org.apache.spark - artifactId = spark-streaming-kafka-0-8_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION_SHORT}} - -2. **Programming:** In the streaming application code, import `KafkaUtils` and create an input DStream as follows. - -
    -
    - import org.apache.spark.streaming.kafka._ - - val directKafkaStream = KafkaUtils.createDirectStream[ - [key class], [value class], [key decoder class], [value decoder class] ]( - streamingContext, [map of Kafka parameters], [set of topics to consume]) - - You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - See the [API docs](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$). -
    -
    - import org.apache.spark.streaming.kafka.*; - - JavaPairInputDStream directKafkaStream = - KafkaUtils.createDirectStream(streamingContext, - [key class], [value class], [key decoder class], [value decoder class], - [map of Kafka parameters], [set of topics to consume]); - - You can also pass a `messageHandler` to `createDirectStream` to access `MessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - See the [API docs](api/java/index.html?org/apache/spark/streaming/kafka/KafkaUtils.html). - -
    -
    - from pyspark.streaming.kafka import KafkaUtils - directKafkaStream = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) - - You can also pass a `messageHandler` to `createDirectStream` to access `KafkaMessageAndMetadata` that contains metadata about the current message and transform it to any desired type. - By default, the Python API will decode Kafka data as UTF8 encoded strings. You can specify your custom decoding function to decode the byte arrays in Kafka records to any arbitrary data type. See the [API docs](api/python/pyspark.streaming.html#pyspark.streaming.kafka.KafkaUtils). -
    -
    - - In the Kafka parameters, you must specify either `metadata.broker.list` or `bootstrap.servers`. - By default, it will start consuming from the latest offset of each Kafka partition. If you set configuration `auto.offset.reset` in Kafka parameters to `smallest`, then it will start consuming from the smallest offset. - - You can also start consuming from any arbitrary offset using other variations of `KafkaUtils.createDirectStream`. Furthermore, if you want to access the Kafka offsets consumed in each batch, you can do the following. - -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - var offsetRanges = Array.empty[OffsetRange] - - directKafkaStream.transform { rdd => - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd - }.map { - ... - }.foreachRDD { rdd => - for (o <- offsetRanges) { - println(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - ... - } -
    -
    - // Hold a reference to the current offset ranges, so it can be used downstream - AtomicReference offsetRanges = new AtomicReference<>(); - - directKafkaStream.transformToPair(rdd -> { - OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - offsetRanges.set(offsets); - return rdd; - }).map( - ... - ).foreachRDD(rdd -> { - for (OffsetRange o : offsetRanges.get()) { - System.out.println( - o.topic() + " " + o.partition() + " " + o.fromOffset() + " " + o.untilOffset() - ); - } - ... - }); -
    -
    - offsetRanges = [] - - def storeOffsetRanges(rdd): - global offsetRanges - offsetRanges = rdd.offsetRanges() - return rdd - - def printOffsetRanges(rdd): - for o in offsetRanges: - print "%s %s %s %s" % (o.topic, o.partition, o.fromOffset, o.untilOffset) - - directKafkaStream \ - .transform(storeOffsetRanges) \ - .foreachRDD(printOffsetRanges) -
    -
    - - You can use this to update Zookeeper yourself if you want Zookeeper-based Kafka monitoring tools to show progress of the streaming application. - - Note that the typecast to HasOffsetRanges will only succeed if it is done in the first method called on the directKafkaStream, not later down a chain of methods. You can use transform() instead of foreachRDD() as your first method call in order to access offsets, then call further Spark methods. However, be aware that the one-to-one mapping between RDD partition and Kafka partition does not remain after any methods that shuffle or repartition, e.g. reduceByKey() or window(). - - Another thing to note is that since this approach does not use Receivers, the standard receiver-related (that is, [configurations](configuration.html) of the form `spark.streaming.receiver.*` ) will not apply to the input DStreams created by this approach (will apply to other input DStreams though). Instead, use the [configurations](configuration.html) `spark.streaming.kafka.*`. An important one is `spark.streaming.kafka.maxRatePerPartition` which is the maximum rate (in messages per second) at which each Kafka partition will be read by this direct API. - -3. **Deploying:** This is same as the first approach. diff --git a/docs/streaming-kafka-integration.md b/docs/streaming-kafka-integration.md index 4aca391e4ba1a..6fa363285f385 100644 --- a/docs/streaming-kafka-integration.md +++ b/docs/streaming-kafka-integration.md @@ -1,54 +1,26 @@ --- layout: global title: Spark Streaming + Kafka Integration Guide +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. --- -[Apache Kafka](https://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, replicated commit log service. Please read the [Kafka documentation](https://kafka.apache.org/documentation.html) thoroughly before starting an integration using Spark. +[Apache Kafka](https://kafka.apache.org/) is publish-subscribe messaging rethought as a distributed, partitioned, +replicated commit log service. Please read the [Kafka documentation](https://kafka.apache.org/documentation.html) +thoroughly before starting an integration using Spark. -The Kafka project introduced a new consumer API between versions 0.8 and 0.10, so there are 2 separate corresponding Spark Streaming packages available. Please choose the correct package for your brokers and desired features; note that the 0.8 integration is compatible with later 0.9 and 0.10 brokers, but the 0.10 integration is not compatible with earlier brokers. - -**Note: Kafka 0.8 support is deprecated as of Spark 2.3.0.** - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
    spark-streaming-kafka-0-8spark-streaming-kafka-0-10
    Broker Version0.8.2.1 or higher0.10.0 or higher
    API MaturityDeprecatedStable
    Language SupportScala, Java, PythonScala, Java
    Receiver DStreamYesNo
    Direct DStreamYesYes
    SSL / TLS SupportNoYes
    Offset Commit APINoYes
    Dynamic Topic SubscriptionNoYes
    +At the moment, Spark requires Kafka 0.10 and higher. See +Kafka 0.10 integration documentation for details. \ No newline at end of file diff --git a/docs/streaming-kinesis-integration.md b/docs/streaming-kinesis-integration.md index 6a52e8a7b0ebd..fd6d776045cd7 100644 --- a/docs/streaming-kinesis-integration.md +++ b/docs/streaming-kinesis-integration.md @@ -1,6 +1,21 @@ --- layout: global title: Spark Streaming + Kinesis Integration +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. --- [Amazon Kinesis](http://aws.amazon.com/kinesis/) is a fully managed service for real-time processing of streaming data at massive scale. The Kinesis receiver creates an input DStream using the Kinesis Client Library (KCL) provided by Amazon under the Amazon Software License (ASL). @@ -248,5 +263,5 @@ de-aggregate records during consumption. - `InitialPositionInStream.TRIM_HORIZON` may lead to duplicate processing of records where the impact is dependent on checkpoint frequency and processing idempotency. #### Kinesis retry configuration - - `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms". + - `spark.streaming.kinesis.retry.waitTime` : Wait time between Kinesis retries as a duration string. When reading from Amazon Kinesis, users may hit `ProvisionedThroughputExceededException`'s, when consuming faster than 5 transactions/second or, exceeding the maximum read rate of 2 MiB/second. This configuration can be tweaked to increase the sleep between fetches when a fetch fails to reduce these exceptions. Default is "100ms". - `spark.streaming.kinesis.retry.maxAttempts` : Max number of retries for Kinesis fetches. This config can also be used to tackle the Kinesis `ProvisionedThroughputExceededException`'s in scenarios mentioned above. It can be increased to have more number of retries for Kinesis reads. Default is 3. diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 1103d5c73ff1f..854beb60f1bca 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -3,6 +3,21 @@ layout: global displayTitle: Spark Streaming Programming Guide title: Spark Streaming description: Spark Streaming programming guide and tutorial for Spark SPARK_VERSION_SHORT +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 will become a table of contents (this text will be scraped). @@ -385,11 +400,12 @@ Similar to Spark, Spark Streaming is available through Maven Central. To write y org.apache.spark spark-streaming_{{site.SCALA_BINARY_VERSION}} {{site.SPARK_VERSION}} + provided
    - libraryDependencies += "org.apache.spark" % "spark-streaming_{{site.SCALA_BINARY_VERSION}}" % "{{site.SPARK_VERSION}}" + libraryDependencies += "org.apache.spark" % "spark-streaming_{{site.SCALA_BINARY_VERSION}}" % "{{site.SPARK_VERSION}}" % "provided"
    @@ -733,7 +749,7 @@ for Java, and [StreamingContext](api/python/pyspark.streaming.html#pyspark.strea Python API As of Spark {{site.SPARK_VERSION_SHORT}}, out of these sources, Kafka and Kinesis are available in the Python API. -This category of sources require interfacing with external non-Spark libraries, some of them with +This category of sources requires interfacing with external non-Spark libraries, some of them with complex dependencies (e.g., Kafka). Hence, to minimize issues related to version conflicts of dependencies, the functionality to create DStreams from these sources has been moved to separate libraries that can be [linked](#linking) to explicitly when necessary. @@ -745,7 +761,7 @@ and add it to the classpath. Some of these advanced sources are as follows. -- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.8.2.1 or higher. See the [Kafka Integration Guide](streaming-kafka-integration.html) for more details. +- **Kafka:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kafka broker versions 0.10 or higher. See the [Kafka Integration Guide](streaming-kafka-0-10-integration.html) for more details. - **Kinesis:** Spark Streaming {{site.SPARK_VERSION_SHORT}} is compatible with Kinesis Client Library 1.2.1. See the [Kinesis Integration Guide](streaming-kinesis-integration.html) for more details. @@ -2172,7 +2188,7 @@ the input data stream (using `inputStream.repartition()`). This distributes the received batches of data across the specified number of machines in the cluster before further processing. -For direct stream, please refer to [Spark Streaming + Kafka Integration Guide](streaming-kafka-integration.html) +For direct stream, please refer to [Spark Streaming + Kafka Integration Guide](streaming-kafka-0-10-integration.html) ### Level of Parallelism in Data Processing {:.no_toc} @@ -2433,7 +2449,7 @@ The following table summarizes the semantics under failures: ### With Kafka Direct API {:.no_toc} -In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-integration.html). +In Spark 1.3, we have introduced a new Kafka Direct API, which can ensure that all the Kafka data is received by Spark Streaming exactly once. Along with this, if you implement exactly-once output operation, you can achieve end-to-end exactly-once guarantees. This approach is further discussed in the [Kafka Integration Guide](streaming-kafka-0-10-integration.html). ## Semantics of output operations {:.no_toc} @@ -2463,7 +2479,7 @@ additional effort may be necessary to achieve exactly-once semantics. There are # Where to Go from Here * Additional guides - - [Kafka Integration Guide](streaming-kafka-integration.html) + - [Kafka Integration Guide](streaming-kafka-0-10-integration.html) - [Kinesis Integration Guide](streaming-kinesis-integration.html) - [Custom Receiver Guide](streaming-custom-receivers.html) * Third-party DStream data sources can be found in [Third Party Projects](https://spark.apache.org/third-party-projects.html) diff --git a/docs/structured-streaming-kafka-integration.md b/docs/structured-streaming-kafka-integration.md index 71fd5b10cc407..fe3c60040d0a0 100644 --- a/docs/structured-streaming-kafka-integration.md +++ b/docs/structured-streaming-kafka-integration.md @@ -1,6 +1,21 @@ --- layout: global title: Structured Streaming + Kafka Integration Guide (Kafka broker version 0.10.0 or higher) +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. --- Structured Streaming integration for Kafka 0.10 to read data from and write data to Kafka. @@ -66,8 +81,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribe", "topic1") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); // Subscribe to multiple topics Dataset df = spark @@ -75,8 +90,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribe", "topic1,topic2") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); // Subscribe to a pattern Dataset df = spark @@ -84,8 +99,8 @@ Dataset df = spark .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("subscribePattern", "topic.*") - .load() -df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .load(); +df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)"); {% endhighlight %}
    @@ -123,7 +138,7 @@ df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)")
    -### Creating a Kafka Source for Batch Queries +### Creating a Kafka Source for Batch Queries If you have a use case that is better suited to batch processing, you can create a Dataset/DataFrame for a defined range of offsets. @@ -265,7 +280,7 @@ Each row in the source has the following schema: timestamp - long + timestamp timestampType @@ -340,11 +355,10 @@ The following configurations are optional: failOnDataLoss true or false true - streaming query + streaming and batch Whether to fail the query when it's possible that data is lost (e.g., topics are deleted, or offsets are out of range). This may be a false alarm. You can disable it when it doesn't work - as you expected. Batch queries will always fail if it fails to read any data from the provided - offsets due to lost data. + as you expected. kafkaConsumer.pollTimeoutMs @@ -374,17 +388,60 @@ The following configurations are optional: streaming and batch Rate limit on maximum number of offsets processed per trigger interval. The specified total number of offsets will be proportionally split across topicPartitions of different volume. + + groupIdPrefix + string + spark-kafka-source + streaming and batch + Prefix of consumer group identifiers (`group.id`) that are generated by structured streaming + queries. If "kafka.group.id" is set, this option will be ignored. + + + kafka.group.id + string + none + streaming and batch + The Kafka group id to use in Kafka consumer while reading from Kafka. Use this with caution. + By default, each query generates a unique group id for reading data. This ensures that each Kafka + source has its own consumer group that does not face interference from any other consumer, and + therefore can read all of the partitions of its subscribed topics. In some scenarios (for example, + Kafka group-based authorization), you may want to use a specific authorized group id to read data. + You can optionally set the group id. However, do this with extreme caution as it can cause + unexpected behavior. Concurrently running queries (both, batch and streaming) or sources with the + same group id are likely interfere with each other causing each query to read only part of the + data. This may also occur when queries are started/restarted in quick succession. To minimize such + issues, set the Kafka consumer session timeout (by setting option "kafka.session.timeout.ms") to + be very small. When this is set, option "groupIdPrefix" will be ignored. + +### Consumer Caching + +It's time-consuming to initialize Kafka consumers, especially in streaming scenarios where processing time is a key factor. +Because of this, Spark caches Kafka consumers on executors. The caching key is built up from the following information: +* Topic name +* Topic partition +* Group ID + +The size of the cache is limited by spark.kafka.consumer.cache.capacity (default: 64). +If this threshold is reached, it tries to remove the least-used entry that is currently not in use. +If it cannot be removed, then the cache will keep growing. In the worst case, the cache will grow to +the max number of concurrent tasks that can run in the executor (that is, number of tasks slots), +after which it will never reduce. + +If a task fails for any reason the new task is executed with a newly created Kafka consumer for safety reasons. +At the same time the cached Kafka consumer which was used in the failed execution will be invalidated. Here it has to +be emphasized it will not be closed if any other task is using it. + ## Writing Data to Kafka -Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that +Here, we describe the support for writing Streaming Queries and Batch Queries to Apache Kafka. Take note that Apache Kafka only supports at least once write semantics. Consequently, when writing---either Streaming Queries or Batch Queries---to Kafka, some records may be duplicated; this can happen, for example, if Kafka needs to retry a message that was not acknowledged by a Broker, even though that Broker received and wrote the message record. -Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However, +Structured Streaming cannot prevent such duplicates from occurring due to these Kafka write semantics. However, if writing the query is successful, then you can assume that the query output was written at least once. A possible -solution to remove duplicates when reading the written data could be to introduce a primary (unique) key +solution to remove duplicates when reading the written data could be to introduce a primary (unique) key that can be used to perform de-duplication when reading. The Dataframe being written to Kafka should have the following columns in schema: @@ -405,8 +462,8 @@ The Dataframe being written to Kafka should have the following columns in schema \* The topic column is required if the "topic" configuration option is not specified.
    -The value column is the only required option. If a key column is not specified then -a ```null``` valued key column will be automatically added (see Kafka semantics on +The value column is the only required option. If a key column is not specified then +a ```null``` valued key column will be automatically added (see Kafka semantics on how ```null``` valued key values are handled). If a topic column exists then its value is used as the topic when writing the given row to Kafka, unless the "topic" configuration option is set i.e., the "topic" configuration option overrides the topic column. @@ -472,7 +529,7 @@ StreamingQuery ds = df .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("topic", "topic1") - .start() + .start(); // Write key-value data from a DataFrame to Kafka using a topic specified in the data StreamingQuery ds = df @@ -480,7 +537,7 @@ StreamingQuery ds = df .writeStream() .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .start() + .start(); {% endhighlight %} @@ -540,14 +597,14 @@ df.selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") .option("topic", "topic1") - .save() + .save(); // Write key-value data from a DataFrame to Kafka using a topic specified in the data df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") .write() .format("kafka") .option("kafka.bootstrap.servers", "host1:port1,host2:port2") - .save() + .save(); {% endhighlight %} @@ -568,7 +625,7 @@ df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \ .format("kafka") \ .option("kafka.bootstrap.servers", "host1:port1,host2:port2") \ .save() - + {% endhighlight %} @@ -576,23 +633,26 @@ df.selectExpr("topic", "CAST(key AS STRING)", "CAST(value AS STRING)") \ ## Kafka Specific Configurations -Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, -`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafka parameters, see +Kafka's own configurations can be set via `DataStreamReader.option` with `kafka.` prefix, e.g, +`stream.option("kafka.bootstrap.servers", "host:port")`. For possible kafka parameters, see [Kafka consumer config docs](http://kafka.apache.org/documentation.html#newconsumerconfigs) for parameters related to reading data, and [Kafka producer config docs](http://kafka.apache.org/documentation/#producerconfigs) for parameters related to writing data. Note that the following Kafka params cannot be set and the Kafka source or sink will throw an exception: -- **group.id**: Kafka source will create a unique group id for each query automatically. +- **group.id**: Kafka source will create a unique group id for each query automatically. The user can +set the prefix of the automatically generated group.id's via the optional source option `groupIdPrefix`, +default value is "spark-kafka-source". You can also set "kafka.group.id" to force Spark to use a special +group id, however, please read warnings for this option and use it with caution. - **auto.offset.reset**: Set the source option `startingOffsets` to specify - where to start instead. Structured Streaming manages which offsets are consumed internally, rather - than rely on the kafka Consumer to do it. This will ensure that no data is missed when new + where to start instead. Structured Streaming manages which offsets are consumed internally, rather + than rely on the kafka Consumer to do it. This will ensure that no data is missed when new topics/partitions are dynamically subscribed. Note that `startingOffsets` only applies when a new streaming query is started, and that resuming will always pick up from where the query left off. -- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use +- **key.deserializer**: Keys are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the keys. -- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. +- **value.deserializer**: Values are always deserialized as byte arrays with ByteArrayDeserializer. Use DataFrame operations to explicitly deserialize the values. - **key.serializer**: Keys are always serialized with ByteArraySerializer or StringSerializer. Use DataFrame operations to explicitly serialize the keys into either strings or byte arrays. @@ -615,3 +675,165 @@ For experimenting on `spark-shell`, you can also use `--packages` to add `spark- See [Application Submission Guide](submitting-applications.html) for more details about submitting applications with external dependencies. + +## Security + +Kafka 0.9.0.0 introduced several features that increases security in a cluster. For detailed +description about these possibilities, see [Kafka security docs](http://kafka.apache.org/documentation.html#security). + +It's worth noting that security is optional and turned off by default. + +Spark supports the following ways to authenticate against Kafka cluster: +- **Delegation token (introduced in Kafka broker 1.1.0)** +- **JAAS login configuration** + +### Delegation token + +This way the application can be configured via Spark parameters and may not need JAAS login +configuration (Spark can use Kafka's dynamic JAAS configuration feature). For further information +about delegation tokens, see [Kafka delegation token docs](http://kafka.apache.org/documentation/#security_delegation_token). + +The process is initiated by Spark's Kafka delegation token provider. When `spark.kafka.clusters.${cluster}.auth.bootstrap.servers` is set, +Spark considers the following log in options, in order of preference: +- **JAAS login configuration**, please see example below. +- **Keytab file**, such as, + + ./bin/spark-submit \ + --keytab \ + --principal \ + --conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers= \ + ... + +- **Kerberos credential cache**, such as, + + ./bin/spark-submit \ + --conf spark.kafka.clusters.${cluster}.auth.bootstrap.servers= \ + ... + +The Kafka delegation token provider can be turned off by setting `spark.security.credentials.kafka.enabled` to `false` (default: `true`). + +Spark can be configured to use the following authentication protocols to obtain token (it must match with +Kafka broker configuration): +- **SASL SSL (default)** +- **SSL** +- **SASL PLAINTEXT (for testing)** + +After obtaining delegation token successfully, Spark distributes it across nodes and renews it accordingly. +Delegation token uses `SCRAM` login module for authentication and because of that the appropriate +`spark.kafka.clusters.${cluster}.sasl.token.mechanism` (default: `SCRAM-SHA-512`) has to be configured. Also, this parameter +must match with Kafka broker configuration. + +When delegation token is available on an executor Spark considers the following log in options, in order of preference: +- **JAAS login configuration**, please see example below. +- **Delegation token**, please see spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex parameter for further details. + +When none of the above applies then unsecure connection assumed. + + +#### Configuration + +Delegation tokens can be obtained from multiple clusters and ${cluster} is an arbitrary unique identifier which helps to group different configurations. + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.kafka.clusters.${cluster}.auth.bootstrap.serversNone + A list of coma separated host/port pairs to use for establishing the initial connection + to the Kafka cluster. For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.target.bootstrap.servers.regex.* + Regular expression to match against the bootstrap.servers config for sources and sinks in the application. + If a server address matches this regex, the delegation token obtained from the respective bootstrap servers will be used when connecting. + If multiple clusters match the address, an exception will be thrown and the query won't be started. + Kafka's secure and unsecure listeners are bound to different ports. When both used the secure listener port has to be part of the regular expression. +
    spark.kafka.clusters.${cluster}.security.protocolSASL_SSL + Protocol used to communicate with brokers. For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.sasl.kerberos.service.namekafka + The Kerberos principal name that Kafka runs as. This can be defined either in Kafka's JAAS config or in Kafka's config. + For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.ssl.truststore.locationNone + The location of the trust store file. For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.ssl.truststore.passwordNone + The store password for the trust store file. This is optional and only needed if spark.kafka.clusters.${cluster}.ssl.truststore.location is configured. + For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.ssl.keystore.locationNone + The location of the key store file. This is optional for client and can be used for two-way authentication for client. + For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.ssl.keystore.passwordNone + The store password for the key store file. This is optional and only needed if spark.kafka.clusters.${cluster}.ssl.keystore.location is configured. + For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.ssl.key.passwordNone + The password of the private key in the key store file. This is optional for client. + For further details please see Kafka documentation. Only used to obtain delegation token. +
    spark.kafka.clusters.${cluster}.sasl.token.mechanismSCRAM-SHA-512 + SASL mechanism used for client connections with delegation token. Because SCRAM login module used for authentication a compatible mechanism has to be set here. + For further details please see Kafka documentation (sasl.mechanism). Only used to authenticate against Kafka broker with delegation token. +
    + +#### Kafka Specific Configurations + +Kafka's own configurations can be set with `kafka.` prefix, e.g, `--conf spark.kafka.clusters.${cluster}.kafka.retries=1`. +For possible Kafka parameters, see [Kafka adminclient config docs](http://kafka.apache.org/documentation.html#adminclientconfigs). + +#### Caveats + +- Obtaining delegation token for proxy user is not yet supported ([KAFKA-6945](https://issues.apache.org/jira/browse/KAFKA-6945)). + +### JAAS login configuration + +JAAS login configuration must placed on all nodes where Spark tries to access Kafka cluster. +This provides the possibility to apply any custom authentication logic with a higher cost to maintain. +This can be done several ways. One possibility is to provide additional JVM parameters, such as, + + ./bin/spark-submit \ + --driver-java-options "-Djava.security.auth.login.config=/path/to/custom_jaas.conf" \ + --conf spark.executor.extraJavaOptions=-Djava.security.auth.login.config=/path/to/custom_jaas.conf \ + ... diff --git a/docs/structured-streaming-programming-guide.md b/docs/structured-streaming-programming-guide.md index b6e427735e74b..925e2cfe717c0 100644 --- a/docs/structured-streaming-programming-guide.md +++ b/docs/structured-streaming-programming-guide.md @@ -2,6 +2,21 @@ layout: global displayTitle: Structured Streaming Programming Guide title: Structured Streaming Programming Guide +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 will become a table of contents (this text will be scraped). @@ -495,9 +510,8 @@ returned by `SparkSession.readStream()`. In [R](api/R/read.stream.html), with th #### Input Sources There are a few built-in sources. - - **File source** - Reads files written in a directory as a stream of data. Supported file formats are text, csv, json, orc, parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. - - - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-integration.html) for more details. + - **File source** - Reads files written in a directory as a stream of data. Files will be processed in the order of file modification time. If `latestFirst` is set, order will be reversed. Supported file formats are text, CSV, JSON, ORC, Parquet. See the docs of the DataStreamReader interface for a more up-to-date list, and supported options for each file format. Note that the files must be atomically placed in the given directory, which in most file systems, can be achieved by file move operations. + - **Kafka source** - Reads data from Kafka. It's compatible with Kafka broker versions 0.10.0 or higher. See the [Kafka Integration Guide](structured-streaming-kafka-0-10-integration.html) for more details. - **Socket source (for testing)** - Reads UTF8 text data from a socket connection. The listening server socket is at the driver. Note that this should be used only for testing as this does not provide end-to-end fault-tolerance guarantees. @@ -526,6 +540,8 @@ Here are the details of all the sources in Spark.
    fileNameOnly: whether to check new files based on only the filename instead of on the full path (default: false). With this set to `true`, the following files would be considered as the same file, because their filenames, "dataset.txt", are the same:
    + maxFileAge: Maximum age of a file that can be found in this directory, before it is ignored. For the first batch all files will be considered valid. If latestFirst is set to `true` and maxFilesPerTrigger is set, then this parameter will be ignored, because old files that are valid, and should be processed, may be ignored. The max age is specified with respect to the timestamp of the latest file, and not the timestamp of the current system.(default: 1 week) +
    "file:///dataset.txt"
    "s3://a/dataset.txt"
    "s3n://a/b/dataset.txt"
    @@ -536,7 +552,7 @@ Here are the details of all the sources in Spark. href="api/R/read.stream.html">R). E.g. for "parquet" format options see DataStreamReader.parquet().

    - In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. + In addition, there are session configurations that affect certain file-formats. See the SQL Programming Guide for more details. E.g., for "parquet", see Parquet configuration section. Yes Supports glob paths, but does not support multiple comma-separated paths/globs. @@ -566,7 +582,7 @@ Here are the details of all the sources in Spark. Kafka Source - See the Kafka Integration Guide. + See the Kafka Integration Guide. Yes @@ -922,7 +938,7 @@ late data for that aggregate any more. To enable this, in Spark 2.1, we have int **watermarking**, which lets the engine automatically track the current event time in the data and attempt to clean up old state accordingly. You can define the watermark of a query by specifying the event time column and the threshold on how late the data is expected to be in terms of -event time. For a specific window starting at time `T`, the engine will maintain state and allow late +event time. For a specific window ending at time `T`, the engine will maintain state and allow late data to update the state until `(max event time seen by the engine - late threshold > T)`. In other words, late data within the threshold will be aggregated, but data later than the threshold will start getting dropped @@ -1493,7 +1509,7 @@ Additional details on supported joins: ### Streaming Deduplication You can deduplicate records in data streams using a unique identifier in the events. This is exactly same as deduplication on static using a unique identifier column. The query will store the necessary amount of data from previous records such that it can filter duplicate records. Similar to aggregations, you can use deduplication with or without watermarking. -- *With watermark* - If there is a upper bound on how late a duplicate record may arrive, then you can define a watermark on a event time column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain. +- *With watermark* - If there is an upper bound on how late a duplicate record may arrive, then you can define a watermark on an event time column and deduplicate using both the guid and the event time columns. The query will use the watermark to remove old state data from past records that are not expected to get any duplicates any more. This bounds the amount of the state the query has to maintain. - *Without watermark* - Since there are no bounds on when a duplicate record may arrive, the query stores the data from all the past records as state. @@ -1577,7 +1593,7 @@ event time seen in each input stream, calculates watermarks based on the corresp and chooses a single global watermark with them to be used for stateful operations. By default, the minimum is chosen as the global watermark because it ensures that no data is accidentally dropped as too late if one of the streams falls behind the others -(for example, one of the streams stop receiving data due to upstream failures). In other words, +(for example, one of the streams stops receiving data due to upstream failures). In other words, the global watermark will safely move at the pace of the slowest stream and the query output will be delayed accordingly. @@ -1598,7 +1614,7 @@ Some of them are as follows. - Multiple streaming aggregations (i.e. a chain of aggregations on a streaming DF) are not yet supported on streaming Datasets. -- Limit and take first N rows are not supported on streaming Datasets. +- Limit and take the first N rows are not supported on streaming Datasets. - Distinct operations on streaming Datasets are not supported. @@ -1634,7 +1650,7 @@ returned through `Dataset.writeStream()`. You will have to specify one or more o - *Query name:* Optionally, specify a unique name of the query for identification. -- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has completed. If a trigger time is missed because the previous processing has not completed, then the system will trigger processing immediately. +- *Trigger interval:* Optionally, specify the trigger interval. If it is not specified, the system will check for availability of new data as soon as the previous processing has been completed. If a trigger time is missed because the previous processing has not been completed, then the system will trigger processing immediately. - *Checkpoint location:* For some output sinks where the end-to-end fault-tolerance can be guaranteed, specify the location where the system will write all the checkpoint information. This should be a directory in an HDFS-compatible fault-tolerant file system. The semantics of checkpointing is discussed in more detail in the next section. @@ -1819,7 +1835,7 @@ Here are the details of all the sinks in Spark. Kafka Sink Append, Update, Complete - See the Kafka Integration Guide + See the Kafka Integration Guide Yes (at-least-once) More details in the Kafka Integration Guide @@ -2086,12 +2102,20 @@ With `foreachBatch`, you can do the following. cause the output data to be recomputed (including possible re-reading of the input data). To avoid recomputations, you should cache the output DataFrame/Dataset, write it to multiple locations, and then uncache it. Here is an outline. - streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => - batchDF.persist() - batchDF.write.format(...).save(...) // location 1 - batchDF.write.format(...).save(...) // location 2 - batchDF.unpersist() - } +
    +
    + +{% highlight scala %} +streamingDF.writeStream.foreachBatch { (batchDF: DataFrame, batchId: Long) => + batchDF.persist() + batchDF.write.format(...).save(...) // location 1 + batchDF.write.format(...).save(...) // location 2 + batchDF.unpersist() +} +{% endhighlight %} + +
    +
    - **Apply additional DataFrame operations** - Many DataFrame and Dataset operations are not supported in streaming DataFrames because Spark does not support generating incremental plans in those cases. @@ -2106,7 +2130,7 @@ With `foreachBatch`, you can do the following. ###### Foreach If `foreachBatch` is not an option (for example, corresponding batch data writer does not exist, or -continuous processing mode), then you can express you custom writer logic using `foreach`. +continuous processing mode), then you can express your custom writer logic using `foreach`. Specifically, you can express the data writing logic by dividing it into three methods: `open`, `process`, and `close`. Since Spark 2.4, `foreach` is available in Scala, Java and Python. @@ -2236,8 +2260,8 @@ When the streaming query is started, Spark calls the function or the object’s in the continuous mode, then this guarantee does not hold and therefore should not be used for deduplication. #### Triggers -The trigger settings of a streaming query defines the timing of streaming data processing, whether -the query is going to executed as micro-batch query with a fixed batch interval or as a continuous processing query. +The trigger settings of a streaming query define the timing of streaming data processing, whether +the query is going to be executed as micro-batch query with a fixed batch interval or as a continuous processing query. Here are the different kinds of triggers that are supported. @@ -2539,11 +2563,11 @@ spark.streams().awaitAnyTermination(); // block until any one of them terminat {% highlight python %} spark = ... # spark session -spark.streams().active # get the list of currently active streaming queries +spark.streams.active # get the list of currently active streaming queries -spark.streams().get(id) # get a query object by its unique id +spark.streams.get(id) # get a query object by its unique id -spark.streams().awaitAnyTermination() # block until any one of them terminates +spark.streams.awaitAnyTermination() # block until any one of them terminates {% endhighlight %} @@ -2960,7 +2984,7 @@ the effect of the change is not well-defined. For all of them: - Addition/deletion/modification of rate limits is allowed: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "topic").option("maxOffsetsPerTrigger", ...)` - - Changes to subscribed topics/files is generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` + - Changes to subscribed topics/files are generally not allowed as the results are unpredictable: `spark.readStream.format("kafka").option("subscribe", "topic")` to `spark.readStream.format("kafka").option("subscribe", "newTopic")` - *Changes in the type of output sink*: Changes between a few specific combinations of sinks are allowed. This needs to be verified on a case-by-case basis. Here are a few examples. @@ -2974,17 +2998,17 @@ the effect of the change is not well-defined. For all of them: - *Changes in the parameters of output sink*: Whether this is allowed and whether the semantics of the change are well-defined depends on the sink and the query. Here are a few examples. - - Changes to output directory of a file sink is not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` + - Changes to output directory of a file sink are not allowed: `sdf.writeStream.format("parquet").option("path", "/somePath")` to `sdf.writeStream.format("parquet").option("path", "/anotherPath")` - - Changes to output topic is allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` + - Changes to output topic are allowed: `sdf.writeStream.format("kafka").option("topic", "someTopic")` to `sdf.writeStream.format("kafka").option("topic", "anotherTopic")` - - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) is allowed, but the semantics of the change depends on the code. + - Changes to the user-defined foreach sink (that is, the `ForeachWriter` code) are allowed, but the semantics of the change depends on the code. -- *Changes in projection / filter / map-like operations**: Some cases are allowed. For example: +- *Changes in projection / filter / map-like operations*: Some cases are allowed. For example: - Addition / deletion of filters is allowed: `sdf.selectExpr("a")` to `sdf.where(...).selectExpr("a").filter(...)`. - - Changes in projections with same output schema is allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` + - Changes in projections with same output schema are allowed: `sdf.selectExpr("stringColumn AS json").writeStream` to `sdf.selectExpr("anotherStringColumn AS json").writeStream` - Changes in projections with different output schema are conditionally allowed: `sdf.selectExpr("a").writeStream` to `sdf.selectExpr("b").writeStream` is allowed only if the output sink allows the schema change from `"a"` to `"b"`. @@ -3000,7 +3024,7 @@ the effect of the change is not well-defined. For all of them: - *Streaming deduplication*: For example, `sdf.dropDuplicates("a")`. Any change in number or type of grouping keys or aggregates is not allowed. - *Stream-stream join*: For example, `sdf1.join(sdf2, ...)` (i.e. both inputs are generated with `sparkSession.readStream`). Changes - in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) not allowed. Other changes in the join condition are ill-defined. + in the schema or equi-joining columns are not allowed. Changes in join type (outer or inner) are not allowed. Other changes in the join condition are ill-defined. - *Arbitrary stateful operation*: For example, `sdf.groupByKey(...).mapGroupsWithState(...)` or `sdf.groupByKey(...).flatMapGroupsWithState(...)`. Any change to the schema of the user-defined state and the type of timeout is not allowed. @@ -3083,7 +3107,7 @@ spark \ -A checkpoint interval of 1 second means that the continuous processing engine will records the progress of the query every second. The resulting checkpoints are in a format compatible with the micro-batch engine, hence any query can be restarted with any trigger. For example, a supported query started with the micro-batch mode can be restarted in continuous mode, and vice versa. Note that any time you switch to continuous mode, you will get at-least-once fault-tolerance guarantees. +A checkpoint interval of 1 second means that the continuous processing engine will record the progress of the query every second. The resulting checkpoints are in a format compatible with the micro-batch engine, hence any query can be restarted with any trigger. For example, a supported query started with the micro-batch mode can be restarted in continuous mode, and vice versa. Note that any time you switch to continuous mode, you will get at-least-once fault-tolerance guarantees. ## Supported Queries {:.no_toc} @@ -3113,6 +3137,16 @@ See [Input Sources](#input-sources) and [Output Sinks](#output-sinks) sections f # Additional Information +**Notes** + +- Several configurations are not modifiable after the query has run. To change them, discard the checkpoint and start a new query. These configurations include: + - `spark.sql.shuffle.partitions` + - This is due to the physical partitioning of state: state is partitioned via applying hash function to key, hence the number of partitions for state should be unchanged. + - If you want to run fewer tasks for stateful operations, `coalesce` would help with avoiding unnecessary repartitioning. + - After `coalesce`, the number of (reduced) tasks will be kept unless another shuffle happens. + - `spark.sql.streaming.stateStore.providerClass`: To read the previous state of the query properly, the class of state store provider should be unchanged. + - `spark.sql.streaming.multipleWatermarkPolicy`: Modification of this would lead inconsistent watermark value when query contains multiple watermarks, hence the policy should be unchanged. + **Further Reading** - See and run the diff --git a/docs/submitting-applications.md b/docs/submitting-applications.md index 77aa083c4a584..cbc6f86919d3e 100644 --- a/docs/submitting-applications.md +++ b/docs/submitting-applications.md @@ -1,6 +1,21 @@ --- layout: global title: Submitting Applications +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. --- The `spark-submit` script in Spark's `bin` directory is used to launch applications on a cluster. @@ -44,7 +59,7 @@ Some of the commonly used options are: * `--class`: The entry point for your application (e.g. `org.apache.spark.examples.SparkPi`) * `--master`: The [master URL](#master-urls) for the cluster (e.g. `spark://23.195.26.187:7077`) * `--deploy-mode`: Whether to deploy your driver on the worker nodes (`cluster`) or locally as an external client (`client`) (default: `client`) -* `--conf`: Arbitrary Spark configuration property in key=value format. For values that contain spaces wrap "key=value" in quotes (as shown). +* `--conf`: Arbitrary Spark configuration property in key=value format. For values that contain spaces wrap "key=value" in quotes (as shown). Multiple configurations should be passed as separate arguments. (e.g. `--conf = --conf =`) * `application-jar`: Path to a bundled jar including your application and all dependencies. The URL must be globally visible inside of your cluster, for instance, an `hdfs://` path or a `file://` path that is present on all nodes. * `application-arguments`: Arguments passed to the main method of your main class, if any diff --git a/docs/tuning.md b/docs/tuning.md index cd0f9cd081369..1faf7cfe0d68e 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -3,6 +3,21 @@ layout: global displayTitle: Tuning Spark title: Tuning description: Tuning and performance optimization guide for Spark SPARK_VERSION_SHORT +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 will become a table of contents (this text will be scraped). @@ -115,7 +130,7 @@ variety of workloads without requiring user expertise of how memory is divided i Although there are two relevant configurations, the typical user should not need to adjust them as the default values are applicable to most workloads: -* `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MB) +* `spark.memory.fraction` expresses the size of `M` as a fraction of the (JVM heap space - 300MiB) (default 0.6). The rest of the space (40%) is reserved for user data structures, internal metadata in Spark, and safeguarding against OOM errors in the case of sparse and unusually large records. @@ -147,7 +162,7 @@ pointer-based data structures and wrapper objects. There are several ways to do Java standard library. 2. Avoid nested structures with a lot of small objects and pointers when possible. 3. Consider using numeric IDs or enumeration objects instead of strings for keys. -4. If you have less than 32 GB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be +4. If you have less than 32 GiB of RAM, set the JVM flag `-XX:+UseCompressedOops` to make pointers be four bytes instead of eight. You can add these options in [`spark-env.sh`](configuration.html#environment-variables). @@ -224,8 +239,8 @@ temporary objects created during task execution. Some steps which may be useful * As an example, if your task is reading data from HDFS, the amount of memory used by the task can be estimated using the size of the data block read from HDFS. Note that the size of a decompressed block is often 2 or 3 times the - size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 128 MB, - we can estimate size of Eden to be `4*3*128MB`. + size of the block. So if we wish to have 3 or 4 tasks' worth of working space, and the HDFS block size is 128 MiB, + we can estimate size of Eden to be `4*3*128MiB`. * Monitor how the frequency and time taken by garbage collection changes with the new settings. @@ -233,7 +248,7 @@ Our experience suggests that the effect of GC tuning depends on your application There are [many more tuning options](https://docs.oracle.com/javase/8/docs/technotes/guides/vm/gctuning/index.html) described online, but at a high level, managing how frequently full GC takes place can help in reducing the overhead. -GC tuning flags for executors can be specified by setting `spark.executor.extraJavaOptions` in +GC tuning flags for executors can be specified by setting `spark.executor.defaultJavaOptions` or `spark.executor.extraJavaOptions` in a job's configuration. # Other Considerations @@ -267,7 +282,7 @@ available in `SparkContext` can greatly reduce the size of each serialized task, of launching a job over a cluster. If your tasks use any large object from the driver program inside of them (e.g. a static lookup table), consider turning it into a broadcast variable. Spark prints the serialized size of each task on the master, so you can look at that to -decide whether your tasks are too large; in general tasks larger than about 20 KB are probably +decide whether your tasks are too large; in general tasks larger than about 20 KiB are probably worth optimizing. ## Data Locality diff --git a/examples/pom.xml b/examples/pom.xml index 756c475b4748d..ac148ef4c9c01 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-examples_2.11 + spark-examples_2.12 jar Spark Project Examples http://spark.apache.org/ @@ -110,7 +110,7 @@ 3.7.0 - com.twitter + ${hive.parquet.group} parquet-hadoop-bundle provided diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index c9ca9c9b3a412..7e8df69e7e8da 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -71,7 +71,7 @@ public static void main(String[] args) { JavaSparkContext jsc = new JavaSparkContext(spark.sparkContext()); - Integer slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; + int slices = (args.length > 0) ? Integer.parseInt(args[0]): 2; JavaPairRDD tc = jsc.parallelizePairs(generateGraph(), slices).cache(); // Linear transitive closure: each round grows paths by one edge, diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java index 27052be87b82e..b8d2c9f6a6584 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaALSExample.java @@ -111,7 +111,7 @@ public static void main(String[] args) { .setMetricName("rmse") .setLabelCol("rating") .setPredictionCol("prediction"); - Double rmse = evaluator.evaluate(predictions); + double rmse = evaluator.evaluate(predictions); System.out.println("Root-mean-square error = " + rmse); // Generate top 10 movie recommendations for each user diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java index 8c82aaaacca38..f517dc314b2b7 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaBisectingKMeansExample.java @@ -20,6 +20,7 @@ // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans; import org.apache.spark.ml.clustering.BisectingKMeansModel; +import org.apache.spark.ml.evaluation.ClusteringEvaluator; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -50,9 +51,14 @@ public static void main(String[] args) { BisectingKMeans bkm = new BisectingKMeans().setK(2).setSeed(1); BisectingKMeansModel model = bkm.fit(dataset); - // Evaluate clustering. - double cost = model.computeCost(dataset); - System.out.println("Within Set Sum of Squared Errors = " + cost); + // Make predictions + Dataset predictions = model.transform(dataset); + + // Evaluate clustering by computing Silhouette score + ClusteringEvaluator evaluator = new ClusteringEvaluator(); + + double silhouette = evaluator.evaluate(predictions); + System.out.println("Silhouette with squared euclidean distance = " + silhouette); // Shows the result. System.out.println("Cluster Centers: "); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java similarity index 91% rename from examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java rename to examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java index 6f93cff94b725..4b49bebf7ccfe 100644 --- a/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderEstimatorExample.java +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaOneHotEncoderExample.java @@ -23,7 +23,7 @@ import java.util.Arrays; import java.util.List; -import org.apache.spark.ml.feature.OneHotEncoderEstimator; +import org.apache.spark.ml.feature.OneHotEncoder; import org.apache.spark.ml.feature.OneHotEncoderModel; import org.apache.spark.sql.Dataset; import org.apache.spark.sql.Row; @@ -34,11 +34,11 @@ import org.apache.spark.sql.types.StructType; // $example off$ -public class JavaOneHotEncoderEstimatorExample { +public class JavaOneHotEncoderExample { public static void main(String[] args) { SparkSession spark = SparkSession .builder() - .appName("JavaOneHotEncoderEstimatorExample") + .appName("JavaOneHotEncoderExample") .getOrCreate(); // Note: categorical features are usually first encoded with StringIndexer @@ -59,7 +59,7 @@ public static void main(String[] args) { Dataset df = spark.createDataFrame(data, schema); - OneHotEncoderEstimator encoder = new OneHotEncoderEstimator() + OneHotEncoder encoder = new OneHotEncoder() .setInputCols(new String[] {"categoryIndex1", "categoryIndex2"}) .setOutputCols(new String[] {"categoryVec1", "categoryVec2"}); diff --git a/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java b/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java new file mode 100644 index 0000000000000..891f306cbf9c7 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/ml/JavaPrefixSpanExample.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.ml; + +// $example on$ +import java.util.Arrays; +import java.util.List; + +import org.apache.spark.ml.fpm.PrefixSpan; +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.RowFactory; +import org.apache.spark.sql.SparkSession; +import org.apache.spark.sql.types.*; +// $example off$ + +/** + * An example demonstrating PrefixSpan. + * Run with + *
    + * bin/run-example ml.JavaPrefixSpanExample
    + * 
    + */ +public class JavaPrefixSpanExample { + public static void main(String[] args) { + SparkSession spark = SparkSession + .builder() + .appName("JavaPrefixSpanExample") + .getOrCreate(); + + // $example on$ + List data = Arrays.asList( + RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(3))), + RowFactory.create(Arrays.asList(Arrays.asList(1), Arrays.asList(3, 2), Arrays.asList(1,2))), + RowFactory.create(Arrays.asList(Arrays.asList(1, 2), Arrays.asList(5))), + RowFactory.create(Arrays.asList(Arrays.asList(6))) + ); + StructType schema = new StructType(new StructField[]{ new StructField( + "sequence", new ArrayType(new ArrayType(DataTypes.IntegerType, true), true), + false, Metadata.empty()) + }); + Dataset sequenceDF = spark.createDataFrame(data, schema); + + PrefixSpan prefixSpan = new PrefixSpan().setMinSupport(0.5).setMaxPatternLength(5); + + // Finding frequent sequential patterns + prefixSpan.findFrequentSequentialPatterns(sequenceDF).show(); + // $example off$ + + spark.stop(); + } +} diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java index c0fa0b3cac1e9..9bd858b598905 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaCorrelationsExample.java @@ -46,7 +46,7 @@ public static void main(String[] args) { // compute the correlation using Pearson's method. Enter "spearman" for Spearman's method. // If a method is not specified, Pearson's method will be used by default. - Double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); + double correlation = Statistics.corr(seriesX.srdd(), seriesY.srdd(), "pearson"); System.out.println("Correlation is: " + correlation); // note that each Vector is a row and not a column diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java index 6998ce2156c25..7ee3c0a3e1925 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRandomForestClassificationExample.java @@ -48,14 +48,14 @@ public static void main(String[] args) { // Train a RandomForest model. // Empty categoricalFeaturesInfo indicates all features are continuous. - Integer numClasses = 2; + int numClasses = 2; Map categoricalFeaturesInfo = new HashMap<>(); - Integer numTrees = 3; // Use more in practice. + int numTrees = 3; // Use more in practice. String featureSubsetStrategy = "auto"; // Let the algorithm choose. String impurity = "gini"; - Integer maxDepth = 5; - Integer maxBins = 32; - Integer seed = 12345; + int maxDepth = 5; + int maxBins = 32; + int seed = 12345; RandomForestModel model = RandomForest.trainClassifier(trainingData, numClasses, categoricalFeaturesInfo, numTrees, featureSubsetStrategy, impurity, maxDepth, maxBins, diff --git a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java index dc9970d885274..de398de78055b 100644 --- a/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java +++ b/examples/src/main/java/org/apache/spark/examples/mllib/JavaRankingMetricsExample.java @@ -99,16 +99,20 @@ public static void main(String[] args) { // Instantiate the metrics object RankingMetrics metrics = RankingMetrics.of(relevantDocs); - // Precision and NDCG at k + // Precision, NDCG and Recall at k Integer[] kVector = {1, 3, 5}; for (Integer k : kVector) { System.out.format("Precision at %d = %f\n", k, metrics.precisionAt(k)); System.out.format("NDCG at %d = %f\n", k, metrics.ndcgAt(k)); + System.out.format("Recall at %d = %f\n", k, metrics.recallAt(k)); } // Mean average precision System.out.format("Mean average precision = %f\n", metrics.meanAveragePrecision()); + //Mean average precision at k + System.out.format("Mean average precision at 2 = %f\n", metrics.meanAveragePrecisionAt(2)); + // Evaluate the model using numerical ratings and regression metrics JavaRDD> userProducts = ratings.map(r -> new Tuple2<>(r.user(), r.product())); diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java index ef3c904775697..b2ce0bc08642a 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSQLDataSourceExample.java @@ -123,6 +123,18 @@ private static void runBasicDataSourceExample(SparkSession spark) { .option("header", "true") .load("examples/src/main/resources/people.csv"); // $example off:manual_load_options_csv$ + // $example on:load_with_path_glob_filter$ + Dataset partitionedUsersDF = spark.read().format("orc") + .option("pathGlobFilter", "*.orc") + .load("examples/src/main/resources/partitioned_users.orc"); + // $example off:load_with_path_glob_filter$ + // $example on:manual_save_options_orc$ + usersDF.write().format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .option("orc.column.encoding.direct", "name") + .save("users_with_options.orc"); + // $example off:manual_save_options_orc$ // $example on:direct_sql$ Dataset sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`"); diff --git a/examples/src/main/python/ml/bisecting_k_means_example.py b/examples/src/main/python/ml/bisecting_k_means_example.py index 7842d2009e238..82adb338b5d91 100644 --- a/examples/src/main/python/ml/bisecting_k_means_example.py +++ b/examples/src/main/python/ml/bisecting_k_means_example.py @@ -24,6 +24,7 @@ # $example on$ from pyspark.ml.clustering import BisectingKMeans +from pyspark.ml.evaluation import ClusteringEvaluator # $example off$ from pyspark.sql import SparkSession @@ -41,9 +42,14 @@ bkm = BisectingKMeans().setK(2).setSeed(1) model = bkm.fit(dataset) - # Evaluate clustering. - cost = model.computeCost(dataset) - print("Within Set Sum of Squared Errors = " + str(cost)) + # Make predictions + predictions = model.transform(dataset) + + # Evaluate clustering by computing Silhouette score + evaluator = ClusteringEvaluator() + + silhouette = evaluator.evaluate(predictions) + print("Silhouette with squared euclidean distance = " + str(silhouette)) # Shows the result. print("Cluster Centers: ") diff --git a/examples/src/main/python/ml/interaction_example.py b/examples/src/main/python/ml/interaction_example.py new file mode 100644 index 0000000000000..4b632271916f5 --- /dev/null +++ b/examples/src/main/python/ml/interaction_example.py @@ -0,0 +1,56 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# + +from __future__ import print_function + +# $example on$ +from pyspark.ml.feature import Interaction, VectorAssembler +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("InteractionExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame( + [(1, 1, 2, 3, 8, 4, 5), + (2, 4, 3, 8, 7, 9, 8), + (3, 6, 1, 9, 2, 3, 6), + (4, 10, 8, 6, 9, 4, 5), + (5, 9, 2, 7, 10, 7, 3), + (6, 1, 1, 4, 2, 8, 4)], + ["id1", "id2", "id3", "id4", "id5", "id6", "id7"]) + + assembler1 = VectorAssembler(inputCols=["id2", "id3", "id4"], outputCol="vec1") + + assembled1 = assembler1.transform(df) + + assembler2 = VectorAssembler(inputCols=["id5", "id6", "id7"], outputCol="vec2") + + assembled2 = assembler2.transform(assembled1).select("id1", "vec1", "vec2") + + interaction = Interaction(inputCols=["id1", "vec1", "vec2"], outputCol="interactedCol") + + interacted = interaction.transform(assembled2) + + interacted.show(truncate=False) + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/onehot_encoder_estimator_example.py b/examples/src/main/python/ml/onehot_encoder_example.py similarity index 83% rename from examples/src/main/python/ml/onehot_encoder_estimator_example.py rename to examples/src/main/python/ml/onehot_encoder_example.py index 2723e681cea7c..73775b79e36cb 100644 --- a/examples/src/main/python/ml/onehot_encoder_estimator_example.py +++ b/examples/src/main/python/ml/onehot_encoder_example.py @@ -18,14 +18,14 @@ from __future__ import print_function # $example on$ -from pyspark.ml.feature import OneHotEncoderEstimator +from pyspark.ml.feature import OneHotEncoder # $example off$ from pyspark.sql import SparkSession if __name__ == "__main__": spark = SparkSession\ .builder\ - .appName("OneHotEncoderEstimatorExample")\ + .appName("OneHotEncoderExample")\ .getOrCreate() # Note: categorical features are usually first encoded with StringIndexer @@ -39,8 +39,8 @@ (2.0, 0.0) ], ["categoryIndex1", "categoryIndex2"]) - encoder = OneHotEncoderEstimator(inputCols=["categoryIndex1", "categoryIndex2"], - outputCols=["categoryVec1", "categoryVec2"]) + encoder = OneHotEncoder(inputCols=["categoryIndex1", "categoryIndex2"], + outputCols=["categoryVec1", "categoryVec2"]) model = encoder.fit(df) encoded = model.transform(df) encoded.show() diff --git a/examples/src/main/python/ml/power_iteration_clustering_example.py b/examples/src/main/python/ml/power_iteration_clustering_example.py new file mode 100644 index 0000000000000..c983c4ad2b0d6 --- /dev/null +++ b/examples/src/main/python/ml/power_iteration_clustering_example.py @@ -0,0 +1,49 @@ +# +# 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 demonstrating PowerIterationClustering. +Run with: + bin/spark-submit examples/src/main/python/ml/power_iteration_clustering_example.py +""" +# $example on$ +from pyspark.ml.clustering import PowerIterationClustering +# $example off$ +from pyspark.sql import SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PowerIterationClusteringExample")\ + .getOrCreate() + + # $example on$ + df = spark.createDataFrame([ + (0, 1, 1.0), + (0, 2, 1.0), + (1, 2, 1.0), + (3, 4, 1.0), + (4, 0, 0.1) + ], ["src", "dst", "weight"]) + + pic = PowerIterationClustering(k=2, maxIter=20, initMode="degree", weightCol="weight") + + # Shows the cluster assignment + pic.assignClusters(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/ml/prefixspan_example.py b/examples/src/main/python/ml/prefixspan_example.py new file mode 100644 index 0000000000000..88d1d4197341b --- /dev/null +++ b/examples/src/main/python/ml/prefixspan_example.py @@ -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. +# + +""" +An example demonstrating PrefixSpan. +Run with: + bin/spark-submit examples/src/main/python/ml/prefixspan_example.py +""" +# $example on$ +from pyspark.ml.fpm import PrefixSpan +# $example off$ +from pyspark.sql import Row, SparkSession + +if __name__ == "__main__": + spark = SparkSession\ + .builder\ + .appName("PrefixSpanExample")\ + .getOrCreate() + sc = spark.sparkContext + + # $example on$ + df = sc.parallelize([Row(sequence=[[1, 2], [3]]), + Row(sequence=[[1], [3, 2], [1, 2]]), + Row(sequence=[[1, 2], [5]]), + Row(sequence=[[6]])]).toDF() + + prefixSpan = PrefixSpan(minSupport=0.5, maxPatternLength=5, + maxLocalProjDBSize=32000000) + + # Find frequent sequential patterns. + prefixSpan.findFrequentSequentialPatterns(df).show() + # $example off$ + + spark.stop() diff --git a/examples/src/main/python/mllib/bisecting_k_means_example.py b/examples/src/main/python/mllib/bisecting_k_means_example.py index 31f3e72d7ff1f..36e36fc6897f3 100644 --- a/examples/src/main/python/mllib/bisecting_k_means_example.py +++ b/examples/src/main/python/mllib/bisecting_k_means_example.py @@ -23,7 +23,7 @@ from pyspark import SparkContext # $example on$ -from pyspark.mllib.clustering import BisectingKMeans, BisectingKMeansModel +from pyspark.mllib.clustering import BisectingKMeans # $example off$ if __name__ == "__main__": diff --git a/examples/src/main/python/mllib/isotonic_regression_example.py b/examples/src/main/python/mllib/isotonic_regression_example.py index 33d618ab48ea9..f5322d79c45ba 100644 --- a/examples/src/main/python/mllib/isotonic_regression_example.py +++ b/examples/src/main/python/mllib/isotonic_regression_example.py @@ -23,7 +23,7 @@ from pyspark import SparkContext # $example on$ import math -from pyspark.mllib.regression import LabeledPoint, IsotonicRegression, IsotonicRegressionModel +from pyspark.mllib.regression import IsotonicRegression, IsotonicRegressionModel from pyspark.mllib.util import MLUtils # $example off$ diff --git a/examples/src/main/python/mllib/multi_class_metrics_example.py b/examples/src/main/python/mllib/multi_class_metrics_example.py index 7dc5fb4f9127f..03a564e75be90 100644 --- a/examples/src/main/python/mllib/multi_class_metrics_example.py +++ b/examples/src/main/python/mllib/multi_class_metrics_example.py @@ -45,9 +45,9 @@ metrics = MulticlassMetrics(predictionAndLabels) # Overall statistics - precision = metrics.precision() - recall = metrics.recall() - f1Score = metrics.fMeasure() + precision = metrics.precision(1.0) + recall = metrics.recall(1.0) + f1Score = metrics.fMeasure(1.0) print("Summary Stats") print("Precision = %s" % precision) print("Recall = %s" % recall) diff --git a/examples/src/main/python/mllib/ranking_metrics_example.py b/examples/src/main/python/mllib/ranking_metrics_example.py index 21333deded35d..0913bb34cf9d7 100644 --- a/examples/src/main/python/mllib/ranking_metrics_example.py +++ b/examples/src/main/python/mllib/ranking_metrics_example.py @@ -17,7 +17,7 @@ # $example on$ from pyspark.mllib.recommendation import ALS, Rating -from pyspark.mllib.evaluation import RegressionMetrics, RankingMetrics +from pyspark.mllib.evaluation import RegressionMetrics # $example off$ from pyspark import SparkContext diff --git a/examples/src/main/python/mllib/standard_scaler_example.py b/examples/src/main/python/mllib/standard_scaler_example.py index 442094e1bf366..11ed34427dfe2 100644 --- a/examples/src/main/python/mllib/standard_scaler_example.py +++ b/examples/src/main/python/mllib/standard_scaler_example.py @@ -19,7 +19,7 @@ from pyspark import SparkContext # $example on$ -from pyspark.mllib.feature import StandardScaler, StandardScalerModel +from pyspark.mllib.feature import StandardScaler from pyspark.mllib.linalg import Vectors from pyspark.mllib.util import MLUtils # $example off$ diff --git a/examples/src/main/python/sql/arrow.py b/examples/src/main/python/sql/arrow.py index 5eb164b20ad04..de8d4f755de6f 100644 --- a/examples/src/main/python/sql/arrow.py +++ b/examples/src/main/python/sql/arrow.py @@ -36,7 +36,7 @@ def dataframe_with_arrow_example(spark): import pandas as pd # Enable Arrow-based columnar data transfers - spark.conf.set("spark.sql.execution.arrow.enabled", "true") + spark.conf.set("spark.sql.execution.arrow.pyspark.enabled", "true") # Generate a Pandas DataFrame pdf = pd.DataFrame(np.random.rand(100, 3)) @@ -86,6 +86,92 @@ def multiply_func(a, b): # $example off:scalar_pandas_udf$ +def scalar_iter_pandas_udf_example(spark): + # $example on:scalar_iter_pandas_udf$ + import pandas as pd + + from pyspark.sql.functions import col, pandas_udf, struct, PandasUDFType + + pdf = pd.DataFrame([1, 2, 3], columns=["x"]) + df = spark.createDataFrame(pdf) + + # When the UDF is called with a single column that is not StructType, + # the input to the underlying function is an iterator of pd.Series. + @pandas_udf("long", PandasUDFType.SCALAR_ITER) + def plus_one(batch_iter): + for x in batch_iter: + yield x + 1 + + df.select(plus_one(col("x"))).show() + # +-----------+ + # |plus_one(x)| + # +-----------+ + # | 2| + # | 3| + # | 4| + # +-----------+ + + # When the UDF is called with more than one columns, + # the input to the underlying function is an iterator of pd.Series tuple. + @pandas_udf("long", PandasUDFType.SCALAR_ITER) + def multiply_two_cols(batch_iter): + for a, b in batch_iter: + yield a * b + + df.select(multiply_two_cols(col("x"), col("x"))).show() + # +-----------------------+ + # |multiply_two_cols(x, x)| + # +-----------------------+ + # | 1| + # | 4| + # | 9| + # +-----------------------+ + + # When the UDF is called with a single column that is StructType, + # the input to the underlying function is an iterator of pd.DataFrame. + @pandas_udf("long", PandasUDFType.SCALAR_ITER) + def multiply_two_nested_cols(pdf_iter): + for pdf in pdf_iter: + yield pdf["a"] * pdf["b"] + + df.select( + multiply_two_nested_cols( + struct(col("x").alias("a"), col("x").alias("b")) + ).alias("y") + ).show() + # +---+ + # | y| + # +---+ + # | 1| + # | 4| + # | 9| + # +---+ + + # In the UDF, you can initialize some states before processing batches. + # Wrap your code with try/finally or use context managers to ensure + # the release of resources at the end. + y_bc = spark.sparkContext.broadcast(1) + + @pandas_udf("long", PandasUDFType.SCALAR_ITER) + def plus_y(batch_iter): + y = y_bc.value # initialize states + try: + for x in batch_iter: + yield x + y + finally: + pass # release resources here, if any + + df.select(plus_y(col("x"))).show() + # +---------+ + # |plus_y(x)| + # +---------+ + # | 2| + # | 3| + # | 4| + # +---------+ + # $example off:scalar_iter_pandas_udf$ + + def grouped_map_pandas_udf_example(spark): # $example on:grouped_map_pandas_udf$ from pyspark.sql.functions import pandas_udf, PandasUDFType @@ -150,6 +236,28 @@ def mean_udf(v): # $example off:grouped_agg_pandas_udf$ +def map_iter_pandas_udf_example(spark): + # $example on:map_iter_pandas_udf$ + import pandas as pd + + from pyspark.sql.functions import pandas_udf, PandasUDFType + + df = spark.createDataFrame([(1, 21), (2, 30)], ("id", "age")) + + @pandas_udf(df.schema, PandasUDFType.MAP_ITER) + def filter_func(batch_iter): + for pdf in batch_iter: + yield pdf[pdf.id == 1] + + df.mapInPandas(filter_func).show() + # +---+---+ + # | id|age| + # +---+---+ + # | 1| 21| + # +---+---+ + # $example off:map_iter_pandas_udf$ + + if __name__ == "__main__": spark = SparkSession \ .builder \ @@ -160,7 +268,13 @@ def mean_udf(v): dataframe_with_arrow_example(spark) print("Running pandas_udf scalar example") scalar_pandas_udf_example(spark) + print("Running pandas_udf scalar iterator example") + scalar_iter_pandas_udf_example(spark) print("Running pandas_udf grouped map example") grouped_map_pandas_udf_example(spark) + print("Running pandas_udf grouped agg example") + grouped_agg_pandas_udf_example(spark) + print("Running pandas_udf map iterator example") + map_iter_pandas_udf_example(spark) spark.stop() diff --git a/examples/src/main/python/sql/datasource.py b/examples/src/main/python/sql/datasource.py index d8c879dfe02ed..0d78097ea975e 100644 --- a/examples/src/main/python/sql/datasource.py +++ b/examples/src/main/python/sql/datasource.py @@ -57,6 +57,20 @@ def basic_datasource_example(spark): format="csv", sep=":", inferSchema="true", header="true") # $example off:manual_load_options_csv$ + # $example on:load_with_path_glob_filter$ + df = spark.read.load("examples/src/main/resources/partitioned_users.orc", + format="orc", pathGlobFilter="*.orc") + # $example off:load_with_path_glob_filter$ + + # $example on:manual_save_options_orc$ + df = spark.read.orc("examples/src/main/resources/users.orc") + (df.write.format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .option("orc.column.encoding.direct", "name") + .save("users_with_options.orc")) + # $example off:manual_save_options_orc$ + # $example on:write_sorting_and_bucketing$ df.write.bucketBy(42, "name").sortBy("age").saveAsTable("people_bucketed") # $example off:write_sorting_and_bucketing$ diff --git a/examples/src/main/python/sql/hive.py b/examples/src/main/python/sql/hive.py index 33fc2dfbeefa2..e96a8af71adc3 100644 --- a/examples/src/main/python/sql/hive.py +++ b/examples/src/main/python/sql/hive.py @@ -23,7 +23,7 @@ from __future__ import print_function # $example on:spark_hive$ -from os.path import expanduser, join, abspath +from os.path import join, abspath from pyspark.sql import SparkSession from pyspark.sql import Row diff --git a/examples/src/main/python/streaming/direct_kafka_wordcount.py b/examples/src/main/python/streaming/direct_kafka_wordcount.py deleted file mode 100644 index c5c186c11f79a..0000000000000 --- a/examples/src/main/python/streaming/direct_kafka_wordcount.py +++ /dev/null @@ -1,56 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -r""" - Counts words in UTF8 encoded, '\n' delimited text directly received from Kafka in every 2 seconds. - Usage: direct_kafka_wordcount.py - - To run this on your local machine, you need to setup Kafka and create a producer first, see - http://kafka.apache.org/documentation.html#quickstart - - and then run the example - `$ bin/spark-submit --jars \ - external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ - examples/src/main/python/streaming/direct_kafka_wordcount.py \ - localhost:9092 test` -""" -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.streaming import StreamingContext -from pyspark.streaming.kafka import KafkaUtils - -if __name__ == "__main__": - if len(sys.argv) != 3: - print("Usage: direct_kafka_wordcount.py ", file=sys.stderr) - sys.exit(-1) - - sc = SparkContext(appName="PythonStreamingDirectKafkaWordCount") - ssc = StreamingContext(sc, 2) - - brokers, topic = sys.argv[1:] - kvs = KafkaUtils.createDirectStream(ssc, [topic], {"metadata.broker.list": brokers}) - lines = kvs.map(lambda x: x[1]) - counts = lines.flatMap(lambda line: line.split(" ")) \ - .map(lambda word: (word, 1)) \ - .reduceByKey(lambda a, b: a+b) - counts.pprint() - - ssc.start() - ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/kafka_wordcount.py b/examples/src/main/python/streaming/kafka_wordcount.py deleted file mode 100644 index e9ee08b9fd228..0000000000000 --- a/examples/src/main/python/streaming/kafka_wordcount.py +++ /dev/null @@ -1,56 +0,0 @@ -# -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -# - -r""" - Counts words in UTF8 encoded, '\n' delimited text received from the network every second. - Usage: kafka_wordcount.py - - To run this on your local machine, you need to setup Kafka and create a producer first, see - http://kafka.apache.org/documentation.html#quickstart - - and then run the example - `$ bin/spark-submit --jars \ - external/kafka-assembly/target/scala-*/spark-streaming-kafka-assembly-*.jar \ - examples/src/main/python/streaming/kafka_wordcount.py \ - localhost:2181 test` -""" -from __future__ import print_function - -import sys - -from pyspark import SparkContext -from pyspark.streaming import StreamingContext -from pyspark.streaming.kafka import KafkaUtils - -if __name__ == "__main__": - if len(sys.argv) != 3: - print("Usage: kafka_wordcount.py ", file=sys.stderr) - sys.exit(-1) - - sc = SparkContext(appName="PythonStreamingKafkaWordCount") - ssc = StreamingContext(sc, 1) - - zkQuorum, topic = sys.argv[1:] - kvs = KafkaUtils.createStream(ssc, zkQuorum, "spark-streaming-consumer", {topic: 1}) - lines = kvs.map(lambda x: x[1]) - counts = lines.flatMap(lambda line: line.split(" ")) \ - .map(lambda word: (word, 1)) \ - .reduceByKey(lambda a, b: a+b) - counts.pprint() - - ssc.start() - ssc.awaitTermination() diff --git a/examples/src/main/python/streaming/recoverable_network_wordcount.py b/examples/src/main/python/streaming/recoverable_network_wordcount.py index 60167dc772544..a39c4d0b5b8cd 100644 --- a/examples/src/main/python/streaming/recoverable_network_wordcount.py +++ b/examples/src/main/python/streaming/recoverable_network_wordcount.py @@ -83,9 +83,9 @@ def echo(time, rdd): def filterFunc(wordCount): if wordCount[0] in blacklist.value: droppedWordsCounter.add(wordCount[1]) - False + return False else: - True + return True counts = "Counts at time %s %s" % (time, rdd.filter(filterFunc).collect()) print(counts) diff --git a/examples/src/main/r/RSparkSQLExample.R b/examples/src/main/r/RSparkSQLExample.R index a5ed723da47ca..fa083d5542fae 100644 --- a/examples/src/main/r/RSparkSQLExample.R +++ b/examples/src/main/r/RSparkSQLExample.R @@ -114,10 +114,18 @@ write.df(namesAndAges, "namesAndAges.parquet", "parquet") # $example on:manual_load_options_csv$ -df <- read.df("examples/src/main/resources/people.csv", "csv") +df <- read.df("examples/src/main/resources/people.csv", "csv", sep = ";", inferSchema = TRUE, header = TRUE) namesAndAges <- select(df, "name", "age") # $example off:manual_load_options_csv$ +# $example on:load_with_path_glob_filter$ +df <- read.df("examples/src/main/resources/partitioned_users.orc", "orc", pathGlobFilter = "*.orc") +# $example off:load_with_path_glob_filter$ + +# $example on:manual_save_options_orc$ +df <- read.df("examples/src/main/resources/users.orc", "orc") +write.orc(df, "users_with_options.orc", orc.bloom.filter.columns = "favorite_color", orc.dictionary.key.threshold = 1.0, orc.column.encoding.direct = "name") +# $example off:manual_save_options_orc$ # $example on:direct_sql$ df <- sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") diff --git a/examples/src/main/r/ml/powerIterationClustering.R b/examples/src/main/r/ml/powerIterationClustering.R new file mode 100644 index 0000000000000..3530d88e50509 --- /dev/null +++ b/examples/src/main/r/ml/powerIterationClustering.R @@ -0,0 +1,39 @@ +# +# 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/powerIterationClustering.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-powerIterationCLustering-example") + +# $example on$ +df <- createDataFrame(list(list(0L, 1L, 1.0), list(0L, 2L, 1.0), + list(1L, 2L, 1.0), list(3L, 4L, 1.0), + list(4L, 0L, 0.1)), + schema = c("src", "dst", "weight")) +# assign clusters +clusters <- spark.assignClusters(df, k = 2L, maxIter = 20L, + initMode = "degree", weightCol = "weight") + +showDF(arrange(clusters, clusters$id)) +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/r/ml/prefixSpan.R b/examples/src/main/r/ml/prefixSpan.R new file mode 100644 index 0000000000000..02908aeb02968 --- /dev/null +++ b/examples/src/main/r/ml/prefixSpan.R @@ -0,0 +1,43 @@ +# +# 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/prefixSpan.R + +# Load SparkR library into your R session +library(SparkR) + +# Initialize SparkSession +sparkR.session(appName = "SparkR-ML-prefixSpan-example") + +# $example on$ +# Load training data + +df <- createDataFrame(list(list(list(list(1L, 2L), list(3L))), + list(list(list(1L), list(3L, 2L), list(1L, 2L))), + list(list(list(1L, 2L), list(5L))), + list(list(list(6L)))), + schema = c("sequence")) + +# Finding frequent sequential patterns +frequency <- spark.findFrequentSequentialPatterns(df, minSupport = 0.5, maxPatternLength = 5L, + maxLocalProjDBSize = 32000000L) +showDF(frequency) + +# $example off$ + +sparkR.session.stop() diff --git a/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt b/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt new file mode 100644 index 0000000000000..9c19f2a0449eb --- /dev/null +++ b/examples/src/main/resources/partitioned_users.orc/do_not_read_this.txt @@ -0,0 +1 @@ +do not read this diff --git a/examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc b/examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc new file mode 100644 index 0000000000000..890395a9281ab Binary files /dev/null and b/examples/src/main/resources/partitioned_users.orc/favorite_color=__HIVE_DEFAULT_PARTITION__/users.orc differ diff --git a/examples/src/main/resources/partitioned_users.orc/favorite_color=red/users.orc b/examples/src/main/resources/partitioned_users.orc/favorite_color=red/users.orc new file mode 100644 index 0000000000000..150615a6f3b24 Binary files /dev/null and b/examples/src/main/resources/partitioned_users.orc/favorite_color=red/users.orc differ diff --git a/examples/src/main/resources/users.orc b/examples/src/main/resources/users.orc new file mode 100644 index 0000000000000..12478a5d03c26 Binary files /dev/null and b/examples/src/main/resources/users.orc differ diff --git a/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.scala new file mode 100644 index 0000000000000..5d9a9a73f12ec --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/AccumulatorMetricsTest.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. + */ + +// scalastyle:off println +package org.apache.spark.examples + +import org.apache.spark.metrics.source.{DoubleAccumulatorSource, LongAccumulatorSource} +import org.apache.spark.sql.SparkSession + +/** + * Usage: AccumulatorMetricsTest [numElem] + * + * This example shows how to register accumulators against the accumulator source. + * A simple RDD is created, and during the map, the accumulators are incremented. + * + * The only argument, numElem, sets the number elements in the collection to parallize. + * + * The result is output to stdout in the driver with the values of the accumulators. + * For the long accumulator, it should equal numElem the double accumulator should be + * roughly 1.1 x numElem (within double precision.) This example also sets up a + * ConsoleSink (metrics) instance, and so registered codahale metrics (like the + * accumulator source) are reported to stdout as well. + */ +object AccumulatorMetricsTest { + def main(args: Array[String]) { + + val spark = SparkSession + .builder() + .config("spark.metrics.conf.*.sink.console.class", + "org.apache.spark.metrics.sink.ConsoleSink") + .getOrCreate() + + val sc = spark.sparkContext + + val acc = sc.longAccumulator("my-long-metric") + // register the accumulator, the metric system will report as + // [spark.metrics.namespace].[execId|driver].AccumulatorSource.my-long-metric + LongAccumulatorSource.register(sc, List(("my-long-metric" -> acc)).toMap) + + val acc2 = sc.doubleAccumulator("my-double-metric") + // register the accumulator, the metric system will report as + // [spark.metrics.namespace].[execId|driver].AccumulatorSource.my-double-metric + DoubleAccumulatorSource.register(sc, List(("my-double-metric" -> acc2)).toMap) + + val num = if (args.length > 0) args(0).toInt else 1000000 + + val startTime = System.nanoTime + + val accumulatorTest = sc.parallelize(1 to num).foreach(_ => { + acc.add(1) + acc2.add(1.1) + }) + + // Print a footer with test time and accumulator values + println("Test took %.0f milliseconds".format((System.nanoTime - startTime) / 1E6)) + println("Accumulator values:") + println("*** Long accumulator (my-long-metric): " + acc.value) + println("*** Double accumulator (my-double-metric): " + acc2.value) + + spark.stop() + } +} +// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala index 1a779716ec4c0..a73859891a860 100644 --- a/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/DFSReadWriteTest.scala @@ -22,6 +22,9 @@ import java.io.File import scala.io.Source._ +import org.apache.hadoop.fs.FileSystem +import org.apache.hadoop.fs.Path + import org.apache.spark.sql.SparkSession /** @@ -107,6 +110,13 @@ object DFSReadWriteTest { println("Writing local file to DFS") val dfsFilename = s"$dfsDirPath/dfs_read_write_test" + + // delete file if exists + val fs = FileSystem.get(spark.sessionState.newHadoopConf()) + if (fs.exists(new Path(dfsFilename))) { + fs.delete(new Path(dfsFilename), true) + } + val fileRDD = spark.sparkContext.parallelize(fileContents) fileRDD.saveAsTextFile(dfsFilename) @@ -123,7 +133,6 @@ object DFSReadWriteTest { .sum spark.stop() - if (localWordCount == dfsWordCount) { println(s"Success! Local Word Count $localWordCount and " + s"DFS Word Count $dfsWordCount agree.") @@ -131,7 +140,6 @@ object DFSReadWriteTest { println(s"Failure! Local Word Count $localWordCount " + s"and DFS Word Count $dfsWordCount disagree.") } - } } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index e1f985ece8c06..b327e13533b81 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -18,6 +18,8 @@ // scalastyle:off println package org.apache.spark.examples +import java.util.concurrent.TimeUnit + import org.apache.spark.sql.SparkSession @@ -36,11 +38,13 @@ object HdfsTest { val file = spark.read.text(args(0)).rdd val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { - val start = System.currentTimeMillis() + val startTimeNs = System.nanoTime() for (x <- mapped) { x + 2 } - val end = System.currentTimeMillis() - println(s"Iteration $iter took ${end-start} ms") + val durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + println(s"Iteration $iter took $durationMs ms") } + println(s"File contents: ${file.map(_.toString).take(1).mkString(",").slice(0, 10)}") + println(s"Returned length(s) of: ${file.map(_.length).sum().toString}") spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index c55b68e033964..03187aee044e4 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -32,13 +32,13 @@ object LogQuery { | GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR | 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR | 3.5.30729; Release=ARP)" "UD-1" - "image/jpeg" "whatever" 0.350 "-" - "" 265 923 934 "" - | 62.24.11.25 images.com 1358492167 - Whatup""".stripMargin.lines.mkString, + | 62.24.11.25 images.com 1358492167 - Whatup""".stripMargin.split('\n').mkString, """10.10.10.10 - "FRED" [18/Jan/2013:18:02:37 +1100] "GET http://images.com/2013/Generic.jpg | HTTP/1.1" 304 306 "http:/referall.com" "Mozilla/4.0 (compatible; MSIE 7.0; Windows NT 5.1; | GTB7.4; .NET CLR 2.0.50727; .NET CLR 3.0.04506.30; .NET CLR 3.0.04506.648; .NET CLR | 3.5.21022; .NET CLR 3.0.4506.2152; .NET CLR 1.0.3705; .NET CLR 1.1.4322; .NET CLR | 3.5.30729; Release=ARP)" "UD-1" - "image/jpeg" "whatever" 0.352 "-" - "" 256 977 988 "" - | 0 73.23.2.15 images.com 1358492557 - Whatup""".stripMargin.lines.mkString + | 0 73.23.2.15 images.com 1358492557 - Whatup""".stripMargin.split('\n').mkString ) def main(args: Array[String]) { diff --git a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala index 815404d1218b7..0726fe970fca7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala +++ b/examples/src/main/scala/org/apache/spark/examples/graphx/Analytics.scala @@ -24,14 +24,13 @@ import org.apache.spark._ import org.apache.spark.graphx._ import org.apache.spark.graphx.PartitionStrategy._ import org.apache.spark.graphx.lib._ -import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel /** * Driver program for running graph algorithms. */ -object Analytics extends Logging { +object Analytics { def main(args: Array[String]): Unit = { if (args.length < 2) { @@ -101,7 +100,7 @@ object Analytics extends Logging { println(s"GRAPHX: Total rank: ${pr.map(_._2).reduce(_ + _)}") if (!outFname.isEmpty) { - logWarning(s"Saving pageranks of pages to $outFname") + println(s"Saving pageranks of pages to $outFname") pr.map { case (id, r) => id + "\t" + r }.saveAsTextFile(outFname) } diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala index 5f8f2c99cbaf4..14e13df02733b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/BisectingKMeansExample.scala @@ -21,6 +21,7 @@ package org.apache.spark.examples.ml // $example on$ import org.apache.spark.ml.clustering.BisectingKMeans +import org.apache.spark.ml.evaluation.ClusteringEvaluator // $example off$ import org.apache.spark.sql.SparkSession @@ -48,9 +49,14 @@ object BisectingKMeansExample { val bkm = new BisectingKMeans().setK(2).setSeed(1) val model = bkm.fit(dataset) - // Evaluate clustering. - val cost = model.computeCost(dataset) - println(s"Within Set Sum of Squared Errors = $cost") + // Make predictions + val predictions = model.transform(dataset) + + // Evaluate clustering by computing Silhouette score + val evaluator = new ClusteringEvaluator() + + val silhouette = evaluator.evaluate(predictions) + println(s"Silhouette with squared euclidean distance = $silhouette") // Shows the result. println("Cluster Centers: ") diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala index 59110d70de550..bece0d96c030f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/FPGrowthExample.scala @@ -17,8 +17,6 @@ package org.apache.spark.examples.ml -// scalastyle:off println - // $example on$ import org.apache.spark.ml.fpm.FPGrowth // $example off$ @@ -64,4 +62,3 @@ object FPGrowthExample { spark.stop() } } -// scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala similarity index 89% rename from examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala rename to examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala index 45d816808ed8e..742f3cdeea35c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderEstimatorExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ml/OneHotEncoderExample.scala @@ -19,15 +19,15 @@ package org.apache.spark.examples.ml // $example on$ -import org.apache.spark.ml.feature.OneHotEncoderEstimator +import org.apache.spark.ml.feature.OneHotEncoder // $example off$ import org.apache.spark.sql.SparkSession -object OneHotEncoderEstimatorExample { +object OneHotEncoderExample { def main(args: Array[String]): Unit = { val spark = SparkSession .builder - .appName("OneHotEncoderEstimatorExample") + .appName("OneHotEncoderExample") .getOrCreate() // Note: categorical features are usually first encoded with StringIndexer @@ -41,7 +41,7 @@ object OneHotEncoderEstimatorExample { (2.0, 0.0) )).toDF("categoryIndex1", "categoryIndex2") - val encoder = new OneHotEncoderEstimator() + val encoder = new OneHotEncoder() .setInputCols(Array("categoryIndex1", "categoryIndex2")) .setOutputCols(Array("categoryVec1", "categoryVec2")) val model = encoder.fit(df) diff --git a/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala new file mode 100644 index 0000000000000..b4e0811c506be --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/ml/PrefixSpanExample.scala @@ -0,0 +1,59 @@ +/* + * 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 + +// $example on$ +import org.apache.spark.ml.fpm.PrefixSpan +// $example off$ +import org.apache.spark.sql.SparkSession + +/** + * An example demonstrating PrefixSpan. + * Run with + * {{{ + * bin/run-example ml.PrefixSpanExample + * }}} + */ +object PrefixSpanExample { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .appName(s"${this.getClass.getSimpleName}") + .getOrCreate() + import spark.implicits._ + + // $example on$ + val smallTestData = Seq( + Seq(Seq(1, 2), Seq(3)), + Seq(Seq(1), Seq(3, 2), Seq(1, 2)), + Seq(Seq(1, 2), Seq(5)), + Seq(Seq(6))) + + val df = smallTestData.toDF("sequence") + val result = new PrefixSpan() + .setMinSupport(0.5) + .setMaxPatternLength(5) + .setMaxLocalProjDBSize(32000000) + .findFrequentSequentialPatterns(df) + .show() + // $example off$ + + spark.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala index a1a5b5915264f..e3cc1d9c83361 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/BinaryClassification.scala @@ -117,7 +117,7 @@ object BinaryClassification { val numTest = test.count() println(s"Training: $numTraining, test: $numTest.") - examples.unpersist(blocking = false) + examples.unpersist() val updater = params.regType match { case L1 => new L1Updater() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala index fa47e12857f0c..b5d1b02f92524 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/DecisionTreeRunner.scala @@ -18,17 +18,15 @@ // scalastyle:off println package org.apache.spark.examples.mllib -import scala.language.reflectiveCalls - import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.evaluation.MulticlassMetrics -import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.{impurity, DecisionTree, RandomForest} import org.apache.spark.mllib.tree.configuration.{Algo, Strategy} import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.model.{DecisionTreeModel, RandomForestModel} import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils @@ -247,7 +245,7 @@ object DecisionTreeRunner { val numTest = test.count() println(s"numTraining = $numTraining, numTest = $numTest.") - examples.unpersist(blocking = false) + examples.unpersist() (training, test, numClasses) } @@ -349,19 +347,17 @@ object DecisionTreeRunner { /** * Calculates the mean squared error for regression. - * - * This is just for demo purpose. In general, don't copy this code because it is NOT efficient - * due to the use of structural types, which leads to one reflection call per record. */ - // scalastyle:off structural.type - private[mllib] def meanSquaredError( - model: { def predict(features: Vector): Double }, - data: RDD[LabeledPoint]): Double = { + private[mllib] def meanSquaredError(model: RandomForestModel, data: RDD[LabeledPoint]): Double = + data.map { y => + val err = model.predict(y.features) - y.label + err * err + }.mean() + + private[mllib] def meanSquaredError(model: DecisionTreeModel, data: RDD[LabeledPoint]): Double = data.map { y => val err = model.predict(y.features) - y.label err * err }.mean() - } - // scalastyle:on structural.type } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala index 4020c6b6bca7f..3f264933cd3cc 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/GradientBoostedTreesRunner.scala @@ -22,8 +22,11 @@ import scopt.OptionParser import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.evaluation.MulticlassMetrics +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.GradientBoostedTrees import org.apache.spark.mllib.tree.configuration.{Algo, BoostingStrategy} +import org.apache.spark.mllib.tree.model.GradientBoostedTreesModel +import org.apache.spark.rdd.RDD import org.apache.spark.util.Utils /** @@ -134,13 +137,20 @@ object GradientBoostedTreesRunner { } else { println(model) // Print model summary. } - val trainMSE = DecisionTreeRunner.meanSquaredError(model, training) + val trainMSE = meanSquaredError(model, training) println(s"Train mean squared error = $trainMSE") - val testMSE = DecisionTreeRunner.meanSquaredError(model, test) + val testMSE = meanSquaredError(model, test) println(s"Test mean squared error = $testMSE") } sc.stop() } + + private[mllib] def meanSquaredError( + model: GradientBoostedTreesModel, data: RDD[LabeledPoint]): Double = + data.map { y => + val err = model.predict(y.features) - y.label + err * err + }.mean() } // scalastyle:on println diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala index 86aec363ea421..03222b13ad27d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/LinearRegression.scala @@ -104,7 +104,7 @@ object LinearRegression { val numTest = test.count() println(s"Training: $numTraining, test: $numTest.") - examples.unpersist(blocking = false) + examples.unpersist() val updater = params.regType match { case NONE => new SimpleUpdater() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala index 9bd6927fb7fc0..fd810155d6a88 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/MovieLensALS.scala @@ -155,7 +155,7 @@ object MovieLensALS { val numTest = test.count() println(s"Training: $numTraining, test: $numTest.") - ratings.unpersist(blocking = false) + ratings.unpersist() val model = new ALS() .setRank(params.rank) diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala index d514891da78fc..ea13ec05e2fad 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/RankingMetricsExample.scala @@ -84,11 +84,19 @@ object RankingMetricsExample { // Mean average precision println(s"Mean average precision = ${metrics.meanAveragePrecision}") + // Mean average precision at k + println(s"Mean average precision at 2 = ${metrics.meanAveragePrecisionAt(2)}") + // Normalized discounted cumulative gain Array(1, 3, 5).foreach { k => println(s"NDCG at $k = ${metrics.ndcgAt(k)}") } + // Recall at K + Array(1, 3, 5).foreach { k => + println(s"Recall at $k = ${metrics.recallAt(k)}") + } + // Get predictions for each data point val allPredictions = model.predict(ratings.map(r => (r.user, r.product))).map(r => ((r.user, r.product), r.rating)) diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala index 7d83aacb11548..c7b6a50f0ae7c 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SQLDataSourceExample.scala @@ -56,6 +56,18 @@ object SQLDataSourceExample { .option("header", "true") .load("examples/src/main/resources/people.csv") // $example off:manual_load_options_csv$ + // $example on:load_with_path_glob_filter$ + val partitionedUsersDF = spark.read.format("orc") + .option("pathGlobFilter", "*.orc") + .load("examples/src/main/resources/partitioned_users.orc") + // $example off:load_with_path_glob_filter$ + // $example on:manual_save_options_orc$ + usersDF.write.format("orc") + .option("orc.bloom.filter.columns", "favorite_color") + .option("orc.dictionary.key.threshold", "1.0") + .option("orc.column.encoding.direct", "name") + .save("users_with_options.orc") + // $example off:manual_save_options_orc$ // $example on:direct_sql$ val sqlDF = spark.sql("SELECT * FROM parquet.`examples/src/main/resources/users.parquet`") diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala new file mode 100644 index 0000000000000..5510f0019353b --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SimpleTypedAggregator.scala @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.examples.sql + +import org.apache.spark.sql.{Encoder, Encoders, SparkSession} +import org.apache.spark.sql.expressions.Aggregator + +// scalastyle:off println +object SimpleTypedAggregator { + + def main(args: Array[String]): Unit = { + val spark = SparkSession + .builder + .master("local") + .appName("common typed aggregator implementations") + .getOrCreate() + + import spark.implicits._ + val ds = spark.range(20).select(('id % 3).as("key"), 'id).as[(Long, Long)] + println("input data:") + ds.show() + + println("running typed sum:") + ds.groupByKey(_._1).agg(new TypedSum[(Long, Long)](_._2).toColumn).show() + + println("running typed count:") + ds.groupByKey(_._1).agg(new TypedCount[(Long, Long)](_._2).toColumn).show() + + println("running typed average:") + ds.groupByKey(_._1).agg(new TypedAverage[(Long, Long)](_._2.toDouble).toColumn).show() + + println("running typed minimum:") + ds.groupByKey(_._1).agg(new TypedMin[(Long, Long)](_._2.toDouble).toColumn).show() + + println("running typed maximum:") + ds.groupByKey(_._1).agg(new TypedMax[(Long, Long)](_._2).toColumn).show() + + spark.stop() + } +} +// scalastyle:on println + +class TypedSum[IN](val f: IN => Long) extends Aggregator[IN, Long, Long] { + override def zero: Long = 0L + override def reduce(b: Long, a: IN): Long = b + f(a) + override def merge(b1: Long, b2: Long): Long = b1 + b2 + override def finish(reduction: Long): Long = reduction + + override def bufferEncoder: Encoder[Long] = Encoders.scalaLong + override def outputEncoder: Encoder[Long] = Encoders.scalaLong +} + +class TypedCount[IN](val f: IN => Any) extends Aggregator[IN, Long, Long] { + override def zero: Long = 0 + override def reduce(b: Long, a: IN): Long = { + if (f(a) == null) b else b + 1 + } + override def merge(b1: Long, b2: Long): Long = b1 + b2 + override def finish(reduction: Long): Long = reduction + + override def bufferEncoder: Encoder[Long] = Encoders.scalaLong + override def outputEncoder: Encoder[Long] = Encoders.scalaLong +} + +class TypedAverage[IN](val f: IN => Double) extends Aggregator[IN, (Double, Long), Double] { + override def zero: (Double, Long) = (0.0, 0L) + override def reduce(b: (Double, Long), a: IN): (Double, Long) = (f(a) + b._1, 1 + b._2) + override def finish(reduction: (Double, Long)): Double = reduction._1 / reduction._2 + override def merge(b1: (Double, Long), b2: (Double, Long)): (Double, Long) = { + (b1._1 + b2._1, b1._2 + b2._2) + } + + override def bufferEncoder: Encoder[(Double, Long)] = { + Encoders.tuple(Encoders.scalaDouble, Encoders.scalaLong) + } + override def outputEncoder: Encoder[Double] = Encoders.scalaDouble +} + +class TypedMin[IN](val f: IN => Double) extends Aggregator[IN, MutableDouble, Option[Double]] { + override def zero: MutableDouble = null + override def reduce(b: MutableDouble, a: IN): MutableDouble = { + if (b == null) { + new MutableDouble(f(a)) + } else { + b.value = math.min(b.value, f(a)) + b + } + } + override def merge(b1: MutableDouble, b2: MutableDouble): MutableDouble = { + if (b1 == null) { + b2 + } else if (b2 == null) { + b1 + } else { + b1.value = math.min(b1.value, b2.value) + b1 + } + } + override def finish(reduction: MutableDouble): Option[Double] = { + if (reduction != null) { + Some(reduction.value) + } else { + None + } + } + + override def bufferEncoder: Encoder[MutableDouble] = Encoders.kryo[MutableDouble] + override def outputEncoder: Encoder[Option[Double]] = Encoders.product[Option[Double]] +} + +class TypedMax[IN](val f: IN => Long) extends Aggregator[IN, MutableLong, Option[Long]] { + override def zero: MutableLong = null + override def reduce(b: MutableLong, a: IN): MutableLong = { + if (b == null) { + new MutableLong(f(a)) + } else { + b.value = math.max(b.value, f(a)) + b + } + } + override def merge(b1: MutableLong, b2: MutableLong): MutableLong = { + if (b1 == null) { + b2 + } else if (b2 == null) { + b1 + } else { + b1.value = math.max(b1.value, b2.value) + b1 + } + } + override def finish(reduction: MutableLong): Option[Long] = { + if (reduction != null) { + Some(reduction.value) + } else { + None + } + } + + override def bufferEncoder: Encoder[MutableLong] = Encoders.kryo[MutableLong] + override def outputEncoder: Encoder[Option[Long]] = Encoders.product[Option[Long]] +} + +class MutableLong(var value: Long) extends Serializable + +class MutableDouble(var value: Double) extends Serializable diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala index 958361a6684c5..678cbc64aff1f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/SparkSQLExample.scala @@ -16,7 +16,9 @@ */ package org.apache.spark.examples.sql +// $example on:programmatic_schema$ import org.apache.spark.sql.Row +// $example off:programmatic_schema$ // $example on:init_session$ import org.apache.spark.sql.SparkSession // $example off:init_session$ diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala index 70fb5b27366ec..a832276602b88 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/hive/SparkHiveExample.scala @@ -122,16 +122,16 @@ object SparkHiveExample { val dataDir = "/tmp/parquet_data" spark.range(10).write.parquet(dataDir) // Create a Hive external Parquet table - sql(s"CREATE EXTERNAL TABLE hive_ints(key int) STORED AS PARQUET LOCATION '$dataDir'") + sql(s"CREATE EXTERNAL TABLE hive_bigints(id bigint) STORED AS PARQUET LOCATION '$dataDir'") // The Hive external table should already have data - sql("SELECT * FROM hive_ints").show() + sql("SELECT * FROM hive_bigints").show() // +---+ - // |key| + // | id| // +---+ // | 0| // | 1| // | 2| - // ... + // ... Order may vary, as spark processes the partitions in parallel. // Turn on flag for Hive Dynamic Partitioning spark.sqlContext.setConf("hive.exec.dynamic.partition", "true") diff --git a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala index ed63fb677b9ef..29dbb0d95cc5e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala +++ b/examples/src/main/scala/org/apache/spark/examples/sql/streaming/StructuredSessionization.scala @@ -70,7 +70,7 @@ object StructuredSessionization { line.split(" ").map(word => Event(sessionId = word, timestamp)) } - // Sessionize the events. Track number of events, start and end timestamps of session, and + // Sessionize the events. Track number of events, start and end timestamps of session, // and report session updates. val sessionUpdates = events .groupByKey(event => event.sessionId) diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala index 25c7bf2871972..fc3f8fa53c7ae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/CustomReceiver.scala @@ -23,7 +23,6 @@ import java.net.Socket import java.nio.charset.StandardCharsets import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.receiver.Receiver @@ -63,7 +62,7 @@ object CustomReceiver { class CustomReceiver(host: String, port: Int) - extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) with Logging { + extends Receiver[String](StorageLevel.MEMORY_AND_DISK_2) { def onStart() { // Start the thread that receives data over a connection @@ -82,9 +81,9 @@ class CustomReceiver(host: String, port: Int) var socket: Socket = null var userInput: String = null try { - logInfo(s"Connecting to $host : $port") + println(s"Connecting to $host : $port") socket = new Socket(host, port) - logInfo(s"Connected to $host : $port") + println(s"Connected to $host : $port") val reader = new BufferedReader( new InputStreamReader(socket.getInputStream(), StandardCharsets.UTF_8)) userInput = reader.readLine() @@ -94,7 +93,7 @@ class CustomReceiver(host: String, port: Int) } reader.close() socket.close() - logInfo("Stopped receiving") + println("Stopped receiving") restart("Trying to connect again") } catch { case e: java.net.ConnectException => diff --git a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala index 2082fb71afdf1..3024b59480099 100644 --- a/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/examples/streaming/DirectKafkaWordCount.scala @@ -40,7 +40,7 @@ object DirectKafkaWordCount { def main(args: Array[String]) { if (args.length < 3) { System.err.println(s""" - |Usage: DirectKafkaWordCount + |Usage: DirectKafkaWordCount | is a list of one or more Kafka brokers | is a consumer group name to consume from topics | is a list of one or more kafka topics to consume from diff --git a/examples/src/main/scripts/getGpusResources.sh b/examples/src/main/scripts/getGpusResources.sh new file mode 100755 index 0000000000000..2d488cd612dba --- /dev/null +++ b/examples/src/main/scripts/getGpusResources.sh @@ -0,0 +1,33 @@ +#!/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. +# + +# This script is a basic example script to get resource information about NVIDIA GPUs. +# It assumes the drivers are properly installed and the nvidia-smi command is available. +# It is not guaranteed to work on all setups so please test and customize as needed +# for your environment. It can be passed into SPARK via the config +# spark.{driver/executor}.resource.gpu.discoveryScript to allow the driver or executor to discover +# the GPUs it was allocated. It assumes you are running within an isolated container where the +# GPUs are allocated exclusively to that driver or executor. +# It outputs a JSON formatted string that is expected by the +# spark.{driver/executor}.resource.gpu.discoveryScript config. +# +# Example output: {"name": "gpu", "addresses":["0","1","2","3","4","5","6","7"]} + +ADDRS=`nvidia-smi --query-gpu=index --format=csv,noheader | sed -e ':a' -e 'N' -e'$!ba' -e 's/\n/","/g'` +echo {\"name\": \"gpu\", \"addresses\":[\"$ADDRS\"]} diff --git a/external/avro/benchmarks/AvroReadBenchmark-results.txt b/external/avro/benchmarks/AvroReadBenchmark-results.txt new file mode 100644 index 0000000000000..7900fea453b10 --- /dev/null +++ b/external/avro/benchmarks/AvroReadBenchmark-results.txt @@ -0,0 +1,122 @@ +================================================================================================ +SQL Single Numeric Column Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 2774 / 2815 5.7 176.4 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 2761 / 2777 5.7 175.5 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 2783 / 2870 5.7 176.9 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 3256 / 3266 4.8 207.0 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 2841 / 2867 5.5 180.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum 2981 / 2996 5.3 189.5 1.0X + + +================================================================================================ +Int and String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of columns 4781 / 4783 2.2 456.0 1.0X + + +================================================================================================ +Partitioned Table Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Data column 3372 / 3386 4.7 214.4 1.0X +Partition column 3035 / 3064 5.2 193.0 1.1X +Both columns 3445 / 3461 4.6 219.1 1.0X + + +================================================================================================ +Repeated String Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of string length 3395 / 3401 3.1 323.8 1.0X + + +================================================================================================ +String with Nulls Scan +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of string length 5580 / 5624 1.9 532.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of string length 4622 / 4623 2.3 440.8 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of string length 3238 / 3241 3.2 308.8 1.0X + + +================================================================================================ +Single Column Scan From Wide Columns +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of single column 5472 / 5484 0.2 5218.8 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of single column 10680 / 10701 0.1 10185.1 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Sum of single column 16143 / 16238 0.1 15394.9 1.0X + + diff --git a/external/avro/benchmarks/AvroWriteBenchmark-results.txt b/external/avro/benchmarks/AvroWriteBenchmark-results.txt new file mode 100644 index 0000000000000..fb2a77333eec5 --- /dev/null +++ b/external/avro/benchmarks/AvroWriteBenchmark-results.txt @@ -0,0 +1,10 @@ +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Avro writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Output Single Int Column 3213 / 3373 4.9 204.3 1.0X +Output Single Double Column 3313 / 3345 4.7 210.7 1.0X +Output Int and String Column 7303 / 7316 2.2 464.3 0.4X +Output Partitions 5309 / 5691 3.0 337.5 0.6X +Output Buckets 7031 / 7557 2.2 447.0 0.5X + diff --git a/external/avro/pom.xml b/external/avro/pom.xml index 9d8f319cc9396..ba6f20bfdbf58 100644 --- a/external/avro/pom.xml +++ b/external/avro/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-avro_2.11 + spark-avro_2.12 avro diff --git a/external/avro/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/external/avro/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 95835f0d4ca49..d89f963059642 100644 --- a/external/avro/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/external/avro/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1 +1 @@ -org.apache.spark.sql.avro.AvroFileFormat +org.apache.spark.sql.v2.avro.AvroDataSourceV2 diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala index 915769fa708b0..5656ac7f38e1b 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroDataToCatalyst.scala @@ -17,20 +17,37 @@ package org.apache.spark.sql.avro +import scala.util.control.NonFatal + import org.apache.avro.Schema import org.apache.avro.generic.GenericDatumReader import org.apache.avro.io.{BinaryDecoder, DecoderFactory} -import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, UnaryExpression} +import org.apache.spark.SparkException +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.{ExpectsInputTypes, Expression, SpecificInternalRow, UnaryExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} -import org.apache.spark.sql.types.{AbstractDataType, BinaryType, DataType} +import org.apache.spark.sql.catalyst.util.{FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.types._ -case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) +case class AvroDataToCatalyst( + child: Expression, + jsonFormatSchema: String, + options: Map[String, String]) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(BinaryType) - override lazy val dataType: DataType = SchemaConverters.toSqlType(avroSchema).dataType + override lazy val dataType: DataType = { + val dt = SchemaConverters.toSqlType(avroSchema).dataType + parseMode match { + // With PermissiveMode, the output Catalyst row might contain columns of null values for + // corrupt records, even if some of the columns are not nullable in the user-provided schema. + // Therefore we force the schema to be all nullable here. + case PermissiveMode => dt.asNullable + case _ => dt + } + } override def nullable: Boolean = true @@ -44,24 +61,69 @@ case class AvroDataToCatalyst(child: Expression, jsonFormatSchema: String) @transient private var result: Any = _ - override def nullSafeEval(input: Any): Any = { - val binary = input.asInstanceOf[Array[Byte]] - decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) - result = reader.read(result, decoder) - deserializer.deserialize(result) + @transient private lazy val parseMode: ParseMode = { + val mode = AvroOptions(options).parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw new AnalysisException(unacceptableModeMessage(mode.name)) + } + mode } - override def simpleString: String = { - s"from_avro(${child.sql}, ${dataType.simpleString})" + private def unacceptableModeMessage(name: String): String = { + s"from_avro() doesn't support the $name mode. " + + s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}." } - override def sql: String = { - s"from_avro(${child.sql}, ${dataType.catalogString})" + @transient private lazy val nullResultRow: Any = dataType match { + case st: StructType => + val resultRow = new SpecificInternalRow(st.map(_.dataType)) + for(i <- 0 until st.length) { + resultRow.setNullAt(i) + } + resultRow + + case _ => + null + } + + + override def nullSafeEval(input: Any): Any = { + val binary = input.asInstanceOf[Array[Byte]] + try { + decoder = DecoderFactory.get().binaryDecoder(binary, 0, binary.length, decoder) + result = reader.read(result, decoder) + deserializer.deserialize(result) + } catch { + // There could be multiple possible exceptions here, e.g. java.io.IOException, + // AvroRuntimeException, ArrayIndexOutOfBoundsException, etc. + // To make it simple, catch all the exceptions here. + case NonFatal(e) => parseMode match { + case PermissiveMode => nullResultRow + case FailFastMode => + throw new SparkException("Malformed records are detected in record parsing. " + + s"Current parse Mode: ${FailFastMode.name}. To process malformed records as null " + + "result, try setting the option 'mode' as 'PERMISSIVE'.", e) + case _ => + throw new AnalysisException(unacceptableModeMessage(parseMode.name)) + } + } } + override def prettyName: String = "from_avro" + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) - defineCodeGen(ctx, ev, input => - s"(${CodeGenerator.boxedType(dataType)})$expr.nullSafeEval($input)") + nullSafeCodeGen(ctx, ev, eval => { + val result = ctx.freshName("result") + val dt = CodeGenerator.boxedType(dataType) + s""" + $dt $result = ($dt) $expr.nullSafeEval($eval); + if ($result == null) { + ${ev.isNull} = true; + } else { + ${ev.value} = $result; + } + """ + }) } } 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 272e7d5b388d9..baaccedd2d536 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.avro -import java.math.{BigDecimal} +import java.math.BigDecimal import java.nio.ByteBuffer import scala.collection.JavaConverters._ @@ -218,11 +218,14 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { i += 1 } + // The Avro map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) case (UNION, _) => val allTypes = avroType.getTypes.asScala val nonNullTypes = allTypes.filter(_.getType != NULL) + val nonNullAvroType = Schema.createUnion(nonNullTypes.asJava) if (nonNullTypes.nonEmpty) { if (nonNullTypes.length == 1) { newWriter(nonNullTypes.head, catalystType, path) @@ -251,7 +254,7 @@ class AvroDeserializer(rootAvroType: Schema, rootCatalystType: DataType) { (updater, ordinal, value) => { val row = new SpecificInternalRow(st) val fieldUpdater = new RowUpdater(row) - val i = GenericData.get().resolveUnion(avroType, value) + val i = GenericData.get().resolveUnion(nonNullAvroType, value) fieldWriters(i)(fieldUpdater, i, value) updater.set(ordinal, row) } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala index e60fa88cbeba9..123669ba1376a 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroFileFormat.scala @@ -23,25 +23,23 @@ import java.net.URI import scala.util.control.NonFatal import org.apache.avro.Schema -import org.apache.avro.file.DataFileConstants._ import org.apache.avro.file.DataFileReader import org.apache.avro.generic.{GenericDatumReader, GenericRecord} -import org.apache.avro.mapred.{AvroOutputFormat, FsInput} -import org.apache.avro.mapreduce.AvroJob +import org.apache.avro.mapred.FsInput import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.mapreduce.Job -import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.TaskContext import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} import org.apache.spark.sql.sources.{DataSourceRegister, Filter} -import org.apache.spark.sql.types.StructType -import org.apache.spark.util.{SerializableConfiguration, Utils} +import org.apache.spark.sql.types._ +import org.apache.spark.util.SerializableConfiguration -private[avro] class AvroFileFormat extends FileFormat +private[sql] class AvroFileFormat extends FileFormat with DataSourceRegister with Logging with Serializable { override def equals(other: Any): Boolean = other match { @@ -56,74 +54,13 @@ private[avro] class AvroFileFormat extends FileFormat spark: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { - val conf = spark.sessionState.newHadoopConf() - val parsedOptions = new AvroOptions(options, conf) - - // User can specify an optional avro json schema. - val avroSchema = parsedOptions.schema - .map(new Schema.Parser().parse) - .getOrElse { - inferAvroSchemaFromFiles(files, conf, parsedOptions.ignoreExtension, - spark.sessionState.conf.ignoreCorruptFiles) - } - - SchemaConverters.toSqlType(avroSchema).dataType match { - case t: StructType => Some(t) - case _ => throw new RuntimeException( - s"""Avro schema cannot be converted to a Spark SQL StructType: - | - |${avroSchema.toString(true)} - |""".stripMargin) - } - } - - private def inferAvroSchemaFromFiles( - files: Seq[FileStatus], - conf: Configuration, - ignoreExtension: Boolean, - ignoreCorruptFiles: Boolean): Schema = { - // Schema evolution is not supported yet. Here we only pick first random readable sample file to - // figure out the schema of the whole dataset. - val avroReader = files.iterator.map { f => - val path = f.getPath - if (!ignoreExtension && !path.getName.endsWith(".avro")) { - None - } else { - Utils.tryWithResource { - new FsInput(path, conf) - } { in => - try { - Some(DataFileReader.openReader(in, new GenericDatumReader[GenericRecord]())) - } catch { - case e: IOException => - if (ignoreCorruptFiles) { - logWarning(s"Skipped the footer in the corrupted file: $path", e) - None - } else { - throw new SparkException(s"Could not read file: $path", e) - } - } - } - } - }.collectFirst { - case Some(reader) => reader - } - - avroReader match { - case Some(reader) => - try { - reader.getSchema - } finally { - reader.close() - } - case None => - throw new FileNotFoundException( - "No Avro files found. If files don't have .avro extension, set ignoreExtension to true") - } + AvroUtils.inferSchema(spark, options, files) } override def shortName(): String = "avro" + override def toString(): String = "Avro" + override def isSplitable( sparkSession: SparkSession, options: Map[String, String], @@ -134,32 +71,7 @@ private[avro] class AvroFileFormat extends FileFormat job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - val parsedOptions = new AvroOptions(options, spark.sessionState.newHadoopConf()) - val outputAvroSchema: Schema = parsedOptions.schema - .map(new Schema.Parser().parse) - .getOrElse(SchemaConverters.toAvroType(dataSchema, nullable = false, - parsedOptions.recordName, parsedOptions.recordNamespace)) - - AvroJob.setOutputKeySchema(job, outputAvroSchema) - - if (parsedOptions.compression == "uncompressed") { - job.getConfiguration.setBoolean("mapred.output.compress", false) - } else { - job.getConfiguration.setBoolean("mapred.output.compress", true) - logInfo(s"Compressing Avro output using the ${parsedOptions.compression} codec") - val codec = parsedOptions.compression match { - case DEFLATE_CODEC => - val deflateLevel = spark.sessionState.conf.avroDeflateLevel - logInfo(s"Avro compression level $deflateLevel will be used for $DEFLATE_CODEC codec.") - job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel) - DEFLATE_CODEC - case codec @ (SNAPPY_CODEC | BZIP2_CODEC | XZ_CODEC) => codec - case unknown => throw new IllegalArgumentException(s"Invalid compression codec: $unknown") - } - job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, codec) - } - - new AvroOutputWriterFactory(dataSchema, outputAvroSchema.toString) + AvroUtils.prepareWrite(spark.sessionState.conf, job, options, dataSchema) } override def buildReader( @@ -243,6 +155,8 @@ private[avro] class AvroFileFormat extends FileFormat } } } + + override def supportDataType(dataType: DataType): Boolean = AvroUtils.supportsDataType(dataType) } private[avro] object AvroFileFormat { diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala index 67f56343b4524..338244aa9e53b 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOptions.scala @@ -20,7 +20,8 @@ package org.apache.spark.sql.avro import org.apache.hadoop.conf.Configuration import org.apache.spark.internal.Logging -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, FailFastMode, ParseMode} import org.apache.spark.sql.internal.SQLConf /** @@ -58,6 +59,7 @@ class AvroOptions( * If the option is not set, the Hadoop's config `avro.mapred.ignore.inputs.without.extension` * is taken into account. If the former one is not set too, file extensions are ignored. */ + @deprecated("Use the general data source option pathGlobFilter for filtering file names", "3.0") val ignoreExtension: Boolean = { val ignoreFilesWithoutExtensionByDefault = false val ignoreFilesWithoutExtension = conf.getBoolean( @@ -65,7 +67,7 @@ class AvroOptions( ignoreFilesWithoutExtensionByDefault) parameters - .get("ignoreExtension") + .get(AvroOptions.ignoreExtensionKey) .map(_.toBoolean) .getOrElse(!ignoreFilesWithoutExtension) } @@ -79,4 +81,19 @@ class AvroOptions( val compression: String = { parameters.get("compression").getOrElse(SQLConf.get.avroCompressionCodec) } + + val parseMode: ParseMode = + parameters.get("mode").map(ParseMode.fromString).getOrElse(FailFastMode) +} + +object AvroOptions { + def apply(parameters: Map[String, String]): AvroOptions = { + val hadoopConf = SparkSession + .getActiveSession + .map(_.sessionState.newHadoopConf()) + .getOrElse(new Configuration()) + new AvroOptions(CaseInsensitiveMap(parameters), hadoopConf) + } + + val ignoreExtensionKey = "ignoreExtension" } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala index 116020ed5c433..0074044544c0d 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroOutputWriterFactory.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.types.StructType * @param catalystSchema Catalyst schema of input data. * @param avroSchemaAsJsonString Avro schema of output result, in JSON string format. */ -private[avro] class AvroOutputWriterFactory( +private[sql] class AvroOutputWriterFactory( catalystSchema: StructType, avroSchemaAsJsonString: String) extends OutputWriterFactory { 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 e902b4c77eaad..b7bf7e5543033 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 @@ -21,16 +21,17 @@ import java.nio.ByteBuffer import scala.collection.JavaConverters._ -import org.apache.avro.{LogicalTypes, Schema} import org.apache.avro.Conversions.DecimalConversion +import org.apache.avro.LogicalTypes import org.apache.avro.LogicalTypes.{TimestampMicros, TimestampMillis} import org.apache.avro.Schema import org.apache.avro.Schema.Type import org.apache.avro.Schema.Type._ -import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed, Record} +import org.apache.avro.generic.GenericData.{EnumSymbol, Fixed} import org.apache.avro.generic.GenericData.Record 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.types._ @@ -38,7 +39,8 @@ import org.apache.spark.sql.types._ /** * A serializer to serialize data in catalyst format to data in avro format. */ -class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) { +class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: Boolean) + extends Logging { def serialize(catalystData: Any): Any = { converter.apply(catalystData) @@ -205,18 +207,28 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: throw new IncompatibleSchemaException(s"Cannot convert Catalyst type $catalystStruct to " + s"Avro type $avroStruct.") } - val fieldConverters = catalystStruct.zip(avroStruct.getFields.asScala).map { - case (f1, f2) => newConverter(f1.dataType, resolveNullableType(f2.schema(), f1.nullable)) - } + + val (avroIndices: Array[Int], fieldConverters: Array[Converter]) = + catalystStruct.map { catalystField => + val avroField = avroStruct.getField(catalystField.name) + if (avroField == null) { + throw new IncompatibleSchemaException( + s"Cannot convert Catalyst type $catalystStruct to Avro type $avroStruct.") + } + val converter = newConverter(catalystField.dataType, resolveNullableType( + avroField.schema(), catalystField.nullable)) + (avroField.pos(), converter) + }.toArray.unzip + val numFields = catalystStruct.length - (row: InternalRow) => + row: InternalRow => val result = new Record(avroStruct) var i = 0 while (i < numFields) { if (row.isNullAt(i)) { - result.put(i, null) + result.put(avroIndices(i), null) } else { - result.put(i, fieldConverters(i).apply(row, i)) + result.put(avroIndices(i), fieldConverters(i).apply(row, i)) } i += 1 } @@ -224,7 +236,7 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: } private def resolveNullableType(avroType: Schema, nullable: Boolean): Schema = { - if (nullable && avroType.getType != NULL) { + if (avroType.getType == Type.UNION && nullable) { // avro uses union to represent nullable type. val fields = avroType.getTypes.asScala assert(fields.length == 2) @@ -232,6 +244,10 @@ class AvroSerializer(rootCatalystType: DataType, rootAvroType: Schema, nullable: assert(actualType.length == 1) actualType.head } else { + if (nullable) { + logWarning("Writing avro files with non-nullable avro schema with nullable catalyst " + + "schema will throw runtime exception if there is a record with null value.") + } avroType } } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala new file mode 100644 index 0000000000000..b978b7974b92d --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/AvroUtils.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.avro + +import java.io.{FileNotFoundException, IOException} + +import org.apache.avro.Schema +import org.apache.avro.file.DataFileConstants.{BZIP2_CODEC, DEFLATE_CODEC, SNAPPY_CODEC, XZ_CODEC} +import org.apache.avro.file.DataFileReader +import org.apache.avro.generic.{GenericDatumReader, GenericRecord} +import org.apache.avro.mapred.{AvroOutputFormat, FsInput} +import org.apache.avro.mapreduce.AvroJob +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.FileStatus +import org.apache.hadoop.mapreduce.Job + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.OutputWriterFactory +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +object AvroUtils extends Logging { + def inferSchema( + spark: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + val conf = spark.sessionState.newHadoopConf() + if (options.contains("ignoreExtension")) { + logWarning(s"Option ${AvroOptions.ignoreExtensionKey} is deprecated. Please use the " + + "general data source option pathGlobFilter for filtering file names.") + } + val parsedOptions = new AvroOptions(options, conf) + + // User can specify an optional avro json schema. + val avroSchema = parsedOptions.schema + .map(new Schema.Parser().parse) + .getOrElse { + inferAvroSchemaFromFiles(files, conf, parsedOptions.ignoreExtension, + spark.sessionState.conf.ignoreCorruptFiles) + } + + SchemaConverters.toSqlType(avroSchema).dataType match { + case t: StructType => Some(t) + case _ => throw new RuntimeException( + s"""Avro schema cannot be converted to a Spark SQL StructType: + | + |${avroSchema.toString(true)} + |""".stripMargin) + } + } + + def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall { f => supportsDataType(f.dataType) } + + case ArrayType(elementType, _) => supportsDataType(elementType) + + case MapType(keyType, valueType, _) => + supportsDataType(keyType) && supportsDataType(valueType) + + case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) + + case _: NullType => true + + case _ => false + } + + def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val parsedOptions = new AvroOptions(options, job.getConfiguration) + val outputAvroSchema: Schema = parsedOptions.schema + .map(new Schema.Parser().parse) + .getOrElse(SchemaConverters.toAvroType(dataSchema, nullable = false, + parsedOptions.recordName, parsedOptions.recordNamespace)) + + AvroJob.setOutputKeySchema(job, outputAvroSchema) + + if (parsedOptions.compression == "uncompressed") { + job.getConfiguration.setBoolean("mapred.output.compress", false) + } else { + job.getConfiguration.setBoolean("mapred.output.compress", true) + logInfo(s"Compressing Avro output using the ${parsedOptions.compression} codec") + val codec = parsedOptions.compression match { + case DEFLATE_CODEC => + val deflateLevel = sqlConf.avroDeflateLevel + logInfo(s"Avro compression level $deflateLevel will be used for $DEFLATE_CODEC codec.") + job.getConfiguration.setInt(AvroOutputFormat.DEFLATE_LEVEL_KEY, deflateLevel) + DEFLATE_CODEC + case codec @ (SNAPPY_CODEC | BZIP2_CODEC | XZ_CODEC) => codec + case unknown => throw new IllegalArgumentException(s"Invalid compression codec: $unknown") + } + job.getConfiguration.set(AvroJob.CONF_OUTPUT_CODEC, codec) + } + + new AvroOutputWriterFactory(dataSchema, outputAvroSchema.toString) + } + + private def inferAvroSchemaFromFiles( + files: Seq[FileStatus], + conf: Configuration, + ignoreExtension: Boolean, + ignoreCorruptFiles: Boolean): Schema = { + // Schema evolution is not supported yet. Here we only pick first random readable sample file to + // figure out the schema of the whole dataset. + val avroReader = files.iterator.map { f => + val path = f.getPath + if (!ignoreExtension && !path.getName.endsWith(".avro")) { + None + } else { + Utils.tryWithResource { + new FsInput(path, conf) + } { in => + try { + Some(DataFileReader.openReader(in, new GenericDatumReader[GenericRecord]())) + } catch { + case e: IOException => + if (ignoreCorruptFiles) { + logWarning(s"Skipped the footer in the corrupted file: $path", e) + None + } else { + throw new SparkException(s"Could not read file: $path", e) + } + } + } + } + }.collectFirst { + case Some(reader) => reader + } + + avroReader match { + case Some(reader) => + try { + reader.getSchema + } finally { + reader.close() + } + case None => + throw new FileNotFoundException( + "No Avro files found. If files don't have .avro extension, set ignoreExtension to true") + } + } +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala index 141ff3782adfb..6ed330d92f5e6 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/CatalystDataToAvro.scala @@ -52,13 +52,7 @@ case class CatalystDataToAvro(child: Expression) extends UnaryExpression { out.toByteArray } - override def simpleString: String = { - s"to_avro(${child.sql}, ${child.dataType.simpleString})" - } - - override def sql: String = { - s"to_avro(${child.sql}, ${child.dataType.catalogString})" - } + override def prettyName: String = "to_avro" override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val expr = ctx.addReferenceObj("this", this) diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala index bd1576587d7fa..3947d327dfac6 100644 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/SchemaConverters.scala @@ -43,6 +43,10 @@ object SchemaConverters { * This function takes an avro schema and returns a sql schema. */ def toSqlType(avroSchema: Schema): SchemaType = { + toSqlTypeHelper(avroSchema, Set.empty) + } + + def toSqlTypeHelper(avroSchema: Schema, existingRecordNames: Set[String]): SchemaType = { avroSchema.getType match { case INT => avroSchema.getLogicalType match { case _: Date => SchemaType(DateType, nullable = false) @@ -66,22 +70,31 @@ object SchemaConverters { case ENUM => SchemaType(StringType, nullable = false) + case NULL => SchemaType(NullType, nullable = true) + case RECORD => + if (existingRecordNames.contains(avroSchema.getFullName)) { + throw new IncompatibleSchemaException(s""" + |Found recursive reference in Avro schema, which can not be processed by Spark: + |${avroSchema.toString(true)} + """.stripMargin) + } + val newRecordNames = existingRecordNames + avroSchema.getFullName val fields = avroSchema.getFields.asScala.map { f => - val schemaType = toSqlType(f.schema()) + val schemaType = toSqlTypeHelper(f.schema(), newRecordNames) StructField(f.name, schemaType.dataType, schemaType.nullable) } SchemaType(StructType(fields), nullable = false) case ARRAY => - val schemaType = toSqlType(avroSchema.getElementType) + val schemaType = toSqlTypeHelper(avroSchema.getElementType, existingRecordNames) SchemaType( ArrayType(schemaType.dataType, containsNull = schemaType.nullable), nullable = false) case MAP => - val schemaType = toSqlType(avroSchema.getValueType) + val schemaType = toSqlTypeHelper(avroSchema.getValueType, existingRecordNames) SchemaType( MapType(StringType, schemaType.dataType, valueContainsNull = schemaType.nullable), nullable = false) @@ -91,13 +104,14 @@ object SchemaConverters { // In case of a union with null, eliminate it and make a recursive call val remainingUnionTypes = avroSchema.getTypes.asScala.filterNot(_.getType == NULL) if (remainingUnionTypes.size == 1) { - toSqlType(remainingUnionTypes.head).copy(nullable = true) + toSqlTypeHelper(remainingUnionTypes.head, existingRecordNames).copy(nullable = true) } else { - toSqlType(Schema.createUnion(remainingUnionTypes.asJava)).copy(nullable = true) + toSqlTypeHelper(Schema.createUnion(remainingUnionTypes.asJava), existingRecordNames) + .copy(nullable = true) } } else avroSchema.getTypes.asScala.map(_.getType) match { case Seq(t1) => - toSqlType(avroSchema.getTypes.get(0)) + toSqlTypeHelper(avroSchema.getTypes.get(0), existingRecordNames) case Seq(t1, t2) if Set(t1, t2) == Set(INT, LONG) => SchemaType(LongType, nullable = false) case Seq(t1, t2) if Set(t1, t2) == Set(FLOAT, DOUBLE) => @@ -107,7 +121,7 @@ object SchemaConverters { // This is consistent with the behavior when converting between Avro and Parquet. val fields = avroSchema.getTypes.asScala.zipWithIndex.map { case (s, i) => - val schemaType = toSqlType(s) + val schemaType = toSqlTypeHelper(s, existingRecordNames) // All fields are nullable because only one of them is set at a time StructField(s"member$i", schemaType.dataType, nullable = true) } @@ -139,6 +153,7 @@ object SchemaConverters { case FloatType => builder.floatType() case DoubleType => builder.doubleType() case StringType => builder.stringType() + case NullType => builder.nullType() case d: DecimalType => val avroType = LogicalTypes.decimal(d.precision, d.scale) val fixedSize = minBytesForPrecision(d.precision) @@ -169,7 +184,7 @@ object SchemaConverters { // This should never happen. case other => throw new IncompatibleSchemaException(s"Unexpected type $other.") } - if (nullable) { + if (nullable && catalystType != NullType) { Schema.createUnion(schema, nullSchema) } else { schema diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.scala new file mode 100755 index 0000000000000..5ed7828510d54 --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/functions.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.sql.avro + +import scala.collection.JavaConverters._ + +import org.apache.spark.annotation.Experimental +import org.apache.spark.sql.Column + + +// scalastyle:off: object.name +object functions { +// scalastyle:on: object.name + + /** + * Converts a binary column of avro format into its corresponding catalyst value. The specified + * schema must match the read data, otherwise the behavior is undefined: it may fail or return + * arbitrary result. + * + * @param data the binary column. + * @param jsonFormatSchema the avro schema in JSON string format. + * + * @since 3.0.0 + */ + @Experimental + def from_avro( + data: Column, + jsonFormatSchema: String): Column = { + new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema, Map.empty)) + } + + /** + * Converts a binary column of avro format into its corresponding catalyst value. The specified + * schema must match the read data, otherwise the behavior is undefined: it may fail or return + * arbitrary result. + * + * @param data the binary column. + * @param jsonFormatSchema the avro schema in JSON string format. + * @param options options to control how the Avro record is parsed. + * + * @since 3.0.0 + */ + @Experimental + def from_avro( + data: Column, + jsonFormatSchema: String, + options: java.util.Map[String, String]): Column = { + new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema, options.asScala.toMap)) + } + + /** + * Converts a column into binary of avro format. + * + * @param data the data column. + * + * @since 3.0.0 + */ + @Experimental + def to_avro(data: Column): Column = { + new Column(CatalystDataToAvro(data.expr)) + } +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala b/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala index 97f9427f96c55..af0752e569ea5 100755 --- a/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/avro/package.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.apache.spark.annotation.Experimental package object avro { + /** * Converts a binary column of avro format into its corresponding catalyst value. The specified * schema must match the read data, otherwise the behavior is undefined: it may fail or return @@ -31,9 +32,11 @@ package object avro { * @since 2.4.0 */ @Experimental - def from_avro(data: Column, jsonFormatSchema: String): Column = { - new Column(AvroDataToCatalyst(data.expr, jsonFormatSchema)) - } + @deprecated("Please use 'org.apache.spark.sql.avro.functions.from_avro' instead.", "3.0.0") + def from_avro( + data: Column, + jsonFormatSchema: String): Column = + org.apache.spark.sql.avro.functions.from_avro(data, jsonFormatSchema) /** * Converts a column into binary of avro format. @@ -43,7 +46,6 @@ package object avro { * @since 2.4.0 */ @Experimental - def to_avro(data: Column): Column = { - new Column(CatalystDataToAvro(data.expr)) - } + @deprecated("Please use 'org.apache.spark.sql.avro.functions.to_avro' instead.", "3.0.0") + def to_avro(data: Column): Column = org.apache.spark.sql.avro.functions.to_avro(data) } diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala new file mode 100644 index 0000000000000..3171f1e08b4fc --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroDataSourceV2.scala @@ -0,0 +1,43 @@ +/* + * 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.v2.avro + +import org.apache.spark.sql.avro.AvroFileFormat +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class AvroDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[AvroFileFormat] + + override def shortName(): String = "avro" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + AvroTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + AvroTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala new file mode 100644 index 0000000000000..243af7da47003 --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroPartitionReaderFactory.scala @@ -0,0 +1,127 @@ +/* + * 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.v2.avro + +import java.net.URI + +import scala.util.control.NonFatal + +import org.apache.avro.Schema +import org.apache.avro.file.DataFileReader +import org.apache.avro.generic.{GenericDatumReader, GenericRecord} +import org.apache.avro.mapred.FsInput +import org.apache.hadoop.fs.Path + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.sql.avro.{AvroDeserializer, AvroOptions} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.v2.{EmptyPartitionReader, FilePartitionReaderFactory, PartitionReaderWithPartitionValues} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create AVRO readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of AVRO files. + * @param readDataSchema Required data schema of AVRO files. + * @param partitionSchema Schema of partitions. + * @param options Options for parsing AVRO files. + */ +case class AvroPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType, + options: Map[String, String]) extends FilePartitionReaderFactory with Logging { + + override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { + val conf = broadcastedConf.value.value + val parsedOptions = new AvroOptions(options, conf) + val userProvidedSchema = parsedOptions.schema.map(new Schema.Parser().parse) + + if (parsedOptions.ignoreExtension || partitionedFile.filePath.endsWith(".avro")) { + val reader = { + val in = new FsInput(new Path(new URI(partitionedFile.filePath)), conf) + try { + val datumReader = userProvidedSchema match { + case Some(userSchema) => new GenericDatumReader[GenericRecord](userSchema) + case _ => new GenericDatumReader[GenericRecord]() + } + DataFileReader.openReader(in, datumReader) + } catch { + case NonFatal(e) => + logError("Exception while opening DataFileReader", e) + in.close() + throw e + } + } + + // Ensure that the reader is closed even if the task fails or doesn't consume the entire + // iterator of records. + Option(TaskContext.get()).foreach { taskContext => + taskContext.addTaskCompletionListener[Unit] { _ => + reader.close() + } + } + + reader.sync(partitionedFile.start) + val stop = partitionedFile.start + partitionedFile.length + + val deserializer = + new AvroDeserializer(userProvidedSchema.getOrElse(reader.getSchema), readDataSchema) + + val fileReader = new PartitionReader[InternalRow] { + private[this] var completed = false + + override def next(): Boolean = { + if (completed) { + false + } else { + val r = reader.hasNext && !reader.pastSync(stop) + if (!r) { + reader.close() + completed = true + } + r + } + } + + override def get(): InternalRow = { + if (!next) { + throw new NoSuchElementException("next on empty iterator") + } + val record = reader.next() + deserializer.deserialize(record).asInstanceOf[InternalRow] + } + + override def close(): Unit = reader.close() + } + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, partitionedFile.partitionValues) + } else { + new EmptyPartitionReader[InternalRow] + } + } +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala new file mode 100644 index 0000000000000..6ec351080a118 --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScan.scala @@ -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. + */ +package org.apache.spark.sql.v2.avro + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class AvroScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema) { + override def isSplitable(path: Path): Boolean = true + + override def createReaderFactory(): PartitionReaderFactory = { + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + // The partition values are already truncated in `FileScan.partitions`. + // We should use `readPartitionSchema` as the partition schema here. + AvroPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, readDataSchema, readPartitionSchema, caseSensitiveMap) + } + } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala similarity index 55% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala rename to external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala index 1be071614d92e..815da2bd92d44 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/SimpleStreamingScanConfigBuilder.scala +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroScanBuilder.scala @@ -14,27 +14,23 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.sql.v2.avro -package org.apache.spark.sql.execution.streaming - -import org.apache.spark.sql.sources.v2.reader.{ScanConfig, ScanConfigBuilder} +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.v2.reader.Scan import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap -/** - * A very simple [[ScanConfigBuilder]] implementation that creates a simple [[ScanConfig]] to - * carry schema and offsets for streaming data sources. - */ -class SimpleStreamingScanConfigBuilder( +class AvroScanBuilder ( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, schema: StructType, - start: Offset, - end: Option[Offset] = None) - extends ScanConfigBuilder { - - override def build(): ScanConfig = SimpleStreamingScanConfig(schema, start, end) + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + override def build(): Scan = { + AvroScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + } } - -case class SimpleStreamingScanConfig( - readSchema: StructType, - start: Offset, - end: Option[Offset]) - extends ScanConfig diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala new file mode 100644 index 0000000000000..a781624aa61aa --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroTable.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.v2.avro + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.avro.AvroUtils +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class AvroTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + override def newScanBuilder(options: CaseInsensitiveStringMap): AvroScanBuilder = + new AvroScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + AvroUtils.inferSchema(sparkSession, options.asScala.toMap, files) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new AvroWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = AvroUtils.supportsDataType(dataType) + + override def formatName: String = "AVRO" +} diff --git a/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroWriteBuilder.scala b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroWriteBuilder.scala new file mode 100644 index 0000000000000..c2ddc4b19127d --- /dev/null +++ b/external/avro/src/main/scala/org/apache/spark/sql/v2/avro/AvroWriteBuilder.scala @@ -0,0 +1,41 @@ +/* + * 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.v2.avro + +import org.apache.hadoop.mapreduce.Job + +import org.apache.spark.sql.avro.AvroUtils +import org.apache.spark.sql.execution.datasources.OutputWriterFactory +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class AvroWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) { + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + AvroUtils.prepareWrite(sqlConf, job, options, dataSchema) + } +} diff --git a/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java new file mode 100644 index 0000000000000..a448583dddfb7 --- /dev/null +++ b/external/avro/src/test/java/org/apache/spark/sql/avro/JavaAvroFunctionsSuite.java @@ -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.avro; + +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.sql.Dataset; +import org.apache.spark.sql.QueryTest$; +import org.apache.spark.sql.Row; +import org.apache.spark.sql.test.TestSparkSession; + +import static org.apache.spark.sql.avro.functions.to_avro; +import static org.apache.spark.sql.avro.functions.from_avro; + + +public class JavaAvroFunctionsSuite { + private transient TestSparkSession spark; + + @Before + public void setUp() { + spark = new TestSparkSession(); + } + + @After + public void tearDown() { + spark.stop(); + } + + private static void checkAnswer(Dataset actual, Dataset expected) { + String errorMessage = QueryTest$.MODULE$.checkAnswer(actual, expected.collectAsList()); + if (errorMessage != null) { + Assert.fail(errorMessage); + } + } + + @Test + public void testToAvroFromAvro() { + Dataset rangeDf = spark.range(10); + Dataset df = rangeDf.select( + rangeDf.col("id"), rangeDf.col("id").cast("string").as("str")); + + Dataset avroDF = + df.select( + to_avro(df.col("id")).as("a"), + to_avro(df.col("str")).as("b")); + + String avroTypeLong = "{\"type\": \"int\", \"name\": \"id\"}"; + String avroTypeStr = "{\"type\": \"string\", \"name\": \"str\"}"; + + Dataset actual = avroDF.select( + from_avro(avroDF.col("a"), avroTypeLong), + from_avro(avroDF.col("b"), avroTypeStr)); + + checkAnswer(actual, df); + } +} diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala index 8334cca6cd8f1..80dd4c535ad9c 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroCatalystDataConversionSuite.scala @@ -17,14 +17,19 @@ package org.apache.spark.sql.avro +import org.apache.avro.Schema + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.RandomDataGenerator +import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{ExpressionEvalHelper, GenericInternalRow, Literal} import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} +import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalHelper { +class AvroCatalystDataConversionSuite extends SparkFunSuite + with SharedSQLContext + with ExpressionEvalHelper { private def roundTripTest(data: Literal): Unit = { val avroType = SchemaConverters.toAvroType(data.dataType, data.nullable) @@ -33,14 +38,26 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalH private def checkResult(data: Literal, schema: String, expected: Any): Unit = { checkEvaluation( - AvroDataToCatalyst(CatalystDataToAvro(data), schema), + AvroDataToCatalyst(CatalystDataToAvro(data), schema, Map.empty), prepareExpectedResult(expected)) } - private def assertFail(data: Literal, schema: String): Unit = { - intercept[java.io.EOFException] { - AvroDataToCatalyst(CatalystDataToAvro(data), schema).eval() + protected def checkUnsupportedRead(data: Literal, schema: String): Unit = { + val binary = CatalystDataToAvro(data) + intercept[Exception] { + AvroDataToCatalyst(binary, schema, Map("mode" -> "FAILFAST")).eval() + } + + val expected = { + val avroSchema = new Schema.Parser().parse(schema) + SchemaConverters.toSqlType(avroSchema).dataType match { + case st: StructType => Row.fromSeq((0 until st.length).map(_ => null)) + case _ => null + } } + + checkEvaluation(AvroDataToCatalyst(binary, schema, Map("mode" -> "PERMISSIVE")), + expected) } private val testingTypes = Seq( @@ -121,7 +138,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalH """.stripMargin // When read int as string, avro reader is not able to parse the binary and fail. - assertFail(data, avroTypeJson) + checkUnsupportedRead(data, avroTypeJson) } test("read string as int") { @@ -151,7 +168,7 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalH // When read float data as double, avro reader fails(trying to read 8 bytes while the data have // only 4 bytes). - assertFail(data, avroTypeJson) + checkUnsupportedRead(data, avroTypeJson) } test("read double as float") { @@ -167,4 +184,29 @@ class AvroCatalystDataConversionSuite extends SparkFunSuite with ExpressionEvalH // avro reader reads the first 4 bytes of a double as a float, the result is totally undefined. checkResult(data, avroTypeJson, 5.848603E35f) } + + test("Handle unsupported input of record type") { + val actualSchema = StructType(Seq( + StructField("col_0", StringType, false), + StructField("col_1", ShortType, false), + StructField("col_2", DecimalType(8, 4), false), + StructField("col_3", BooleanType, true), + StructField("col_4", DecimalType(38, 38), false))) + + val expectedSchema = StructType(Seq( + StructField("col_0", BinaryType, false), + StructField("col_1", DoubleType, false), + StructField("col_2", DecimalType(18, 4), false), + StructField("col_3", StringType, true), + StructField("col_4", DecimalType(38, 38), false))) + + val seed = scala.util.Random.nextLong() + withClue(s"create random record with seed $seed") { + val data = RandomDataGenerator.randomRow(new scala.util.Random(seed), actualSchema) + val converter = CatalystTypeConverters.createToCatalystConverter(actualSchema) + val input = Literal.create(converter(data), actualSchema) + val avroSchema = SchemaConverters.toAvroType(expectedSchema).toString + checkUnsupportedRead(input, avroSchema) + } + } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala index 90a4cd6ccf9dd..ffd77c5ff6101 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroFunctionsSuite.scala @@ -17,13 +17,22 @@ package org.apache.spark.sql.avro +import java.io.ByteArrayOutputStream + +import scala.collection.JavaConverters._ + import org.apache.avro.Schema +import org.apache.avro.generic.{GenericDatumWriter, GenericRecord, GenericRecordBuilder} +import org.apache.avro.io.EncoderFactory -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.functions.struct -import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.SparkException +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.execution.LocalTableScanExec +import org.apache.spark.sql.functions.{col, struct} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} -class AvroFunctionsSuite extends QueryTest with SharedSQLContext { +class AvroFunctionsSuite extends QueryTest with SharedSQLContext with SQLTestUtils { import testImplicits._ test("roundtrip in to_avro and from_avro - int and string") { @@ -61,6 +70,36 @@ class AvroFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(avroStructDF.select(from_avro('avro, avroTypeStruct)), df) } + test("handle invalid input in from_avro") { + val count = 10 + val df = spark.range(count).select(struct('id, 'id.as("id2")).as("struct")) + val avroStructDF = df.select(to_avro('struct).as("avro")) + val avroTypeStruct = s""" + |{ + | "type": "record", + | "name": "struct", + | "fields": [ + | {"name": "col1", "type": "long"}, + | {"name": "col2", "type": "double"} + | ] + |} + """.stripMargin + + intercept[SparkException] { + avroStructDF.select( + org.apache.spark.sql.avro.functions.from_avro( + 'avro, avroTypeStruct, Map("mode" -> "FAILFAST").asJava)).collect() + } + + // For PERMISSIVE mode, the result should be row of null columns. + val expected = (0 until count).map(_ => Row(Row(null, null))) + checkAnswer( + avroStructDF.select( + org.apache.spark.sql.avro.functions.from_avro( + 'avro, avroTypeStruct, Map("mode" -> "PERMISSIVE").asJava)), + expected) + } + test("roundtrip in to_avro and from_avro - array with null") { val dfOne = Seq(Tuple1(Tuple1(1) :: Nil), Tuple1(null :: Nil)).toDF("array") val avroTypeArrStruct = s""" @@ -80,4 +119,38 @@ class AvroFunctionsSuite extends QueryTest with SharedSQLContext { .select(from_avro($"avro", avroTypeArrStruct).as("array")) checkAnswer(dfOne, readBackOne) } + + test("SPARK-27798: from_avro produces same value when converted to local relation") { + val simpleSchema = + """ + |{ + | "type": "record", + | "name" : "Payload", + | "fields" : [ {"name" : "message", "type" : "string" } ] + |} + """.stripMargin + + def generateBinary(message: String, avroSchema: String): Array[Byte] = { + val schema = new Schema.Parser().parse(avroSchema) + val out = new ByteArrayOutputStream() + val writer = new GenericDatumWriter[GenericRecord](schema) + val encoder = EncoderFactory.get().binaryEncoder(out, null) + val rootRecord = new GenericRecordBuilder(schema).set("message", message).build() + writer.write(rootRecord, encoder) + encoder.flush() + out.toByteArray + } + + // This bug is hit when the rule `ConvertToLocalRelation` is run. But the rule was excluded + // in `SharedSparkSession`. + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> "") { + val df = Seq("one", "two", "three", "four").map(generateBinary(_, simpleSchema)) + .toDF() + .withColumn("value", + functions.from_avro(col("value"), simpleSchema)) + + assert(df.queryExecution.executedPlan.isInstanceOf[LocalTableScanExec]) + assert(df.collect().map(_.get(0)) === Seq(Row("one"), Row("two"), Row("three"), Row("four"))) + } + } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala index 79ba2871c2264..96382764b053c 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/avro/AvroLogicalTypeSuite.scala @@ -24,14 +24,14 @@ import org.apache.avro.Conversions.DecimalConversion import org.apache.avro.file.DataFileWriter import org.apache.avro.generic.{GenericData, GenericDatumWriter, GenericRecord} -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.{QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types.{StructField, StructType, TimestampType} -class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +abstract class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestUtils { import testImplicits._ val dateSchema = s""" @@ -349,3 +349,19 @@ class AvroLogicalTypeSuite extends QueryTest with SharedSQLContext with SQLTestU } } } + +class AvroV1LogicalTypeSuite extends AvroLogicalTypeSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "avro") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "avro") +} + +class AvroV2LogicalTypeSuite extends AvroLogicalTypeSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "") +} 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 1e08f7b50b115..924bf374c7370 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.{TimeZone, UUID} +import java.util.{Locale, TimeZone, UUID} import scala.collection.JavaConverters._ @@ -33,15 +33,16 @@ 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.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ +import org.apache.spark.sql.TestingUDT.{IntervalData, NullData, NullUDT} import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { +abstract class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { import testImplicits._ val episodesAvro = testFile("episodes.avro") @@ -49,7 +50,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { override protected def beforeAll(): Unit = { super.beforeAll() - spark.conf.set("spark.sql.files.maxPartitionBytes", 1024) + spark.conf.set(SQLConf.FILES_MAX_PARTITION_BYTES.key, 1024) } def checkReloadMatchesSaved(originalFile: String, newFile: String): Unit = { @@ -80,7 +81,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { test("resolve avro data source") { val databricksAvro = "com.databricks.spark.avro" // By default the backward compatibility for com.databricks.spark.avro is enabled. - Seq("avro", "org.apache.spark.sql.avro.AvroFileFormat", databricksAvro).foreach { provider => + Seq("org.apache.spark.sql.avro.AvroFileFormat", databricksAvro).foreach { provider => assert(DataSource.lookupDataSource(provider, spark.sessionState.conf) === classOf[org.apache.spark.sql.avro.AvroFileFormat]) } @@ -135,33 +136,12 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } - test("test NULL avro type") { - withTempPath { dir => - val fields = - Seq(new Field("null", Schema.create(Type.NULL), "doc", null)).asJava - val schema = Schema.createRecord("name", "docs", "namespace", false) - schema.setFields(fields) - val datumWriter = new GenericDatumWriter[GenericRecord](schema) - val dataFileWriter = new DataFileWriter[GenericRecord](datumWriter) - dataFileWriter.create(schema, new File(s"$dir.avro")) - val avroRec = new GenericData.Record(schema) - avroRec.put("null", null) - dataFileWriter.append(avroRec) - dataFileWriter.flush() - dataFileWriter.close() - - intercept[IncompatibleSchemaException] { - spark.read.format("avro").load(s"$dir.avro") - } - } - } - test("union(int, long) is read as long") { withTempPath { dir => val avroSchema: Schema = { val union = Schema.createUnion(List(Schema.create(Type.INT), Schema.create(Type.LONG)).asJava) - val fields = Seq(new Field("field1", union, "doc", null)).asJava + val fields = Seq(new Field("field1", union, "doc", null.asInstanceOf[AnyVal])).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) schema @@ -189,7 +169,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val avroSchema: Schema = { val union = Schema.createUnion(List(Schema.create(Type.FLOAT), Schema.create(Type.DOUBLE)).asJava) - val fields = Seq(new Field("field1", union, "doc", null)).asJava + val fields = Seq(new Field("field1", union, "doc", null.asInstanceOf[AnyVal])).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) schema @@ -221,7 +201,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { Schema.create(Type.NULL) ).asJava ) - val fields = Seq(new Field("field1", union, "doc", null)).asJava + val fields = Seq(new Field("field1", union, "doc", null.asInstanceOf[AnyVal])).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) schema @@ -247,7 +227,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { test("Union of a single type") { withTempPath { dir => val UnionOfOne = Schema.createUnion(List(Schema.create(Type.INT)).asJava) - val fields = Seq(new Field("field1", UnionOfOne, "doc", null)).asJava + val fields = Seq(new Field("field1", UnionOfOne, "doc", null.asInstanceOf[AnyVal])).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) @@ -267,6 +247,32 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } + test("SPARK-27858 Union type: More than one non-null type") { + withTempDir { dir => + val complexNullUnionType = Schema.createUnion( + List(Schema.create(Type.INT), Schema.create(Type.NULL), Schema.create(Type.STRING)).asJava) + val fields = Seq( + new Field("field1", complexNullUnionType, "doc", null.asInstanceOf[AnyVal])).asJava + val schema = Schema.createRecord("name", "docs", "namespace", false) + schema.setFields(fields) + val datumWriter = new GenericDatumWriter[GenericRecord](schema) + val dataFileWriter = new DataFileWriter[GenericRecord](datumWriter) + dataFileWriter.create(schema, new File(s"$dir.avro")) + val avroRec = new GenericData.Record(schema) + avroRec.put("field1", 42) + dataFileWriter.append(avroRec) + val avroRec2 = new GenericData.Record(schema) + avroRec2.put("field1", "Alice") + dataFileWriter.append(avroRec2) + dataFileWriter.flush() + dataFileWriter.close() + + val df = spark.read.format("avro").load(s"$dir.avro") + assert(df.schema === StructType.fromDDL("field1 struct")) + assert(df.collect().toSet == Set(Row(Row(42, null)), Row(Row(null, "Alice")))) + } + } + test("Complex Union Type") { withTempPath { dir => val fixedSchema = Schema.createFixed("fixed_name", "doc", "namespace", 4) @@ -274,10 +280,10 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val complexUnionType = Schema.createUnion( List(Schema.create(Type.INT), Schema.create(Type.STRING), fixedSchema, enumSchema).asJava) val fields = Seq( - new Field("field1", complexUnionType, "doc", null), - new Field("field2", complexUnionType, "doc", null), - new Field("field3", complexUnionType, "doc", null), - new Field("field4", complexUnionType, "doc", null) + new Field("field1", complexUnionType, "doc", null.asInstanceOf[AnyVal]), + new Field("field2", complexUnionType, "doc", null.asInstanceOf[AnyVal]), + new Field("field3", complexUnionType, "doc", null.asInstanceOf[AnyVal]), + new Field("field4", complexUnionType, "doc", null.asInstanceOf[AnyVal]) ).asJava val schema = Schema.createRecord("name", "docs", "namespace", false) schema.setFields(fields) @@ -508,7 +514,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val union2 = spark.read.format("avro").load(testAvro).select("union_float_double").collect() assert( union2 - .map(x => new java.lang.Double(x(0).toString)) + .map(x => java.lang.Double.valueOf(x(0).toString)) .exists(p => Math.abs(p - Math.PI) < 0.001)) val fixed = spark.read.format("avro").load(testAvro).select("fixed3").collect() @@ -712,9 +718,9 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | "type" : "record", | "name" : "test_schema", | "fields" : [{ - | "name": "enum", + | "name": "Suit", | "type": [{ "type": "enum", - | "name": "Suit", + | "name": "SuitEnumType", | "symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"] | }, "null"] | }] @@ -754,9 +760,9 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { | "type" : "record", | "name" : "test_schema", | "fields" : [{ - | "name": "enum", + | "name": "Suit", | "type": { "type": "enum", - | "name": "Suit", + | "name": "SuitEnumType", | "symbols" : ["SPADES", "HEARTS", "DIAMONDS", "CLUBS"] | } | }] @@ -784,7 +790,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { dfWithNull.write.format("avro") .option("avroSchema", avroSchema).save(s"$tempDir/${UUID.randomUUID()}") }.getCause.getMessage - assert(message1.contains("org.apache.avro.AvroRuntimeException: Not a union:")) + assert(message1.contains("org.apache.avro.AvroTypeException: Not an enum: null")) // Writing df containing data not in the enum will throw an exception val message2 = intercept[SparkException] { @@ -903,6 +909,118 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { } } + test("support user provided avro schema for writing / reading fields with different ordering") { + withTempPath { tempDir => + val avroSchema = + """ + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "Age", "type": "int"}, + | {"name": "Name", "type": "string"} + | ] + |} + """.stripMargin + + val avroSchemaReversed = + """ + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "Name", "type": "string"}, + | {"name": "Age", "type": "int"} + | ] + |} + """.stripMargin + + val df = spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row(2, "Aurora"))), + StructType(Seq( + StructField("Age", IntegerType, false), + StructField("Name", StringType, false)))) + + val tempSaveDir = s"$tempDir/save/" + + // Writing avro file with reversed field ordering + df.write.format("avro").option("avroSchema", avroSchemaReversed).save(tempSaveDir) + + // Reading reversed avro file + checkAnswer(df.select("Name", "Age"), spark.read.format("avro").load(tempSaveDir)) + checkAvroSchemaEquals(avroSchemaReversed, getAvroSchemaStringFromFiles(tempSaveDir)) + + // Reading reversed avro file with provided original schema + val avroDf = spark.read.format("avro").option("avroSchema", avroSchema).load(tempSaveDir) + checkAnswer(df, avroDf) + assert(avroDf.schema.fieldNames.sameElements(Array("Age", "Name"))) + } + } + + test("support user provided non-nullable avro schema " + + "for nullable catalyst schema without any null record") { + withTempPath { tempDir => + val catalystSchema = + StructType(Seq( + StructField("Age", IntegerType, true), + StructField("Name", StringType, true))) + + val avroSchema = + """ + |{ + | "type" : "record", + | "name" : "test_schema", + | "fields" : [ + | {"name": "Age", "type": "int"}, + | {"name": "Name", "type": "string"} + | ] + |} + """.stripMargin + + val df = spark.createDataFrame( + spark.sparkContext.parallelize(Seq(Row(2, "Aurora"))), catalystSchema) + + val tempSaveDir = s"$tempDir/save/" + + df.write.format("avro").option("avroSchema", avroSchema).save(tempSaveDir) + checkAvroSchemaEquals(avroSchema, getAvroSchemaStringFromFiles(tempSaveDir)) + + val message = intercept[Exception] { + spark.createDataFrame(spark.sparkContext.parallelize(Seq(Row(2, null))), catalystSchema) + .write.format("avro").option("avroSchema", avroSchema) + .save(s"$tempDir/${UUID.randomUUID()}") + }.getCause.getMessage + assert(message.contains("Caused by: java.lang.NullPointerException: " + + "in test_schema in string null of string in field Name")) + } + } + + test("error handling for unsupported Interval data types") { + withTempDir { dir => + val tempDir = new File(dir, "files").getCanonicalPath + var msg = intercept[AnalysisException] { + sql("select interval 1 days").write.format("avro").mode("overwrite").save(tempDir) + }.getMessage + assert(msg.contains("Cannot save interval data type into external storage.") || + msg.contains("AVRO data source does not support calendarinterval data type.")) + + msg = intercept[AnalysisException] { + spark.udf.register("testType", () => new IntervalData()) + sql("select testType()").write.format("avro").mode("overwrite").save(tempDir) + }.getMessage + assert(msg.toLowerCase(Locale.ROOT) + .contains(s"avro data source does not support calendarinterval data type.")) + } + } + + test("support Null data types") { + withTempDir { dir => + val tempDir = new File(dir, "files").getCanonicalPath + val df = sql("select null") + df.write.format("avro").mode("overwrite").save(tempDir) + checkAnswer(spark.read.format("avro").load(tempDir), df) + } + } + test("throw exception if unable to write with user provided Avro schema") { val input: Seq[(DataType, Schema.Type)] = Seq( (NullType, NULL), @@ -941,7 +1059,7 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { val avroArrayType = resolveNullable(Schema.createArray(avroType), nullable) val avroMapType = resolveNullable(Schema.createMap(avroType), nullable) val name = "foo" - val avroField = new Field(name, avroType, "", null) + val avroField = new Field(name, avroType, "", null.asInstanceOf[AnyVal]) val recordSchema = Schema.createRecord("name", "doc", "space", true, Seq(avroField).asJava) val avroRecordType = resolveNullable(recordSchema, nullable) @@ -1309,4 +1427,85 @@ class AvroSuite extends QueryTest with SharedSQLContext with SQLTestUtils { checkCodec(df, path, "xz") } } + + private def checkSchemaWithRecursiveLoop(avroSchema: String): Unit = { + val message = intercept[IncompatibleSchemaException] { + SchemaConverters.toSqlType(new Schema.Parser().parse(avroSchema)) + }.getMessage + + assert(message.contains("Found recursive reference in Avro schema")) + } + + test("Detect recursive loop") { + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, // each element has a long + | {"name": "next", "type": ["null", "LongList"]} // optional next element + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields": [ + | { + | "name": "value", + | "type": { + | "type": "record", + | "name": "foo", + | "fields": [ + | { + | "name": "parent", + | "type": "LongList" + | } + | ] + | } + | } + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, + | {"name": "array", "type": {"type": "array", "items": "LongList"}} + | ] + |} + """.stripMargin) + + checkSchemaWithRecursiveLoop(""" + |{ + | "type": "record", + | "name": "LongList", + | "fields" : [ + | {"name": "value", "type": "long"}, + | {"name": "map", "type": {"type": "map", "values": "LongList"}} + | ] + |} + """.stripMargin) + } +} + +class AvroV1Suite extends AvroSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "avro") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "avro") +} + +class AvroV2Suite extends AvroSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "") } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala new file mode 100644 index 0000000000000..f2f7d650066fb --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroReadBenchmark.scala @@ -0,0 +1,216 @@ +/* + * 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 scala.util.Random + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.types._ + +/** + * Benchmark to measure Avro read performance. + * {{{ + * To run this benchmark: + * 1. without sbt: bin/spark-submit --class + * --jars ,, + * 2. build/sbt "avro/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "avro/test:runMain " + * Results will be written to "benchmarks/AvroReadBenchmark-results.txt". + * }}} + */ +object AvroReadBenchmark extends SqlBasedBenchmark with SQLHelper { + def withTempTable(tableNames: String*)(f: => Unit): Unit = { + try f finally tableNames.foreach(spark.catalog.dropTempView) + } + + private def prepareTable(dir: File, df: DataFrame, partition: Option[String] = None): Unit = { + val dirAvro = dir.getCanonicalPath + + if (partition.isDefined) { + df.write.partitionBy(partition.get).format("avro").save(dirAvro) + } else { + df.write.format("avro").save(dirAvro) + } + + spark.read.format("avro").load(dirAvro).createOrReplaceTempView("avroTable") + } + + def numericScanBenchmark(values: Int, dataType: DataType): Unit = { + val benchmark = + new Benchmark(s"SQL Single ${dataType.sql} Column Scan", values, output = output) + + withTempPath { dir => + withTempTable("t1", "avroTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql(s"SELECT CAST(value as ${dataType.sql}) id FROM t1")) + + benchmark.addCase("Sum") { _ => + spark.sql("SELECT sum(id) FROM avroTable").collect() + } + + benchmark.run() + } + } + } + + def intStringScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Int and String Scan", values, output = output) + + withTempPath { dir => + withTempTable("t1", "avroTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable( + dir, + spark.sql("SELECT CAST(value AS INT) AS c1, CAST(value as STRING) AS c2 FROM t1")) + + benchmark.addCase("Sum of columns") { _ => + spark.sql("SELECT sum(c1), sum(length(c2)) FROM avroTable").collect() + } + + benchmark.run() + } + } + } + + def partitionTableScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Partitioned Table", values, output = output) + + withTempPath { dir => + withTempTable("t1", "avroTable") { + import spark.implicits._ + spark.range(values).map(_ => Random.nextLong).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT value % 2 AS p, value AS id FROM t1"), Some("p")) + + benchmark.addCase("Data column") { _ => + spark.sql("SELECT sum(id) FROM avroTable").collect() + } + + benchmark.addCase("Partition column") { _ => + spark.sql("SELECT sum(p) FROM avroTable").collect() + } + + benchmark.addCase("Both columns") { _ => + spark.sql("SELECT sum(p), sum(id) FROM avroTable").collect() + } + + benchmark.run() + } + } + } + + def repeatedStringScanBenchmark(values: Int): Unit = { + val benchmark = new Benchmark("Repeated String", values, output = output) + + withTempPath { dir => + withTempTable("t1", "avroTable") { + spark.range(values).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT CAST((id % 200) + 10000 as STRING) AS c1 FROM t1")) + + benchmark.addCase("Sum of string length") { _ => + spark.sql("SELECT sum(length(c1)) FROM avroTable").collect() + } + + benchmark.run() + } + } + } + + def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { + withTempPath { dir => + withTempTable("t1", "avroTable") { + spark.range(values).createOrReplaceTempView("t1") + + prepareTable( + dir, + spark.sql( + s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " + + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1")) + + val percentageOfNulls = fractionOfNulls * 100 + val benchmark = + new Benchmark(s"String with Nulls Scan ($percentageOfNulls%)", values, output = output) + + benchmark.addCase("Sum of string length") { _ => + spark.sql("SELECT SUM(LENGTH(c2)) FROM avroTable " + + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() + } + + benchmark.run() + } + } + } + + def columnsBenchmark(values: Int, width: Int): Unit = { + val benchmark = + new Benchmark(s"Single Column Scan from $width columns", values, output = output) + + withTempPath { dir => + withTempTable("t1", "avroTable") { + import spark.implicits._ + val middle = width / 2 + val selectExpr = (1 to width).map(i => s"value as c$i") + spark.range(values).map(_ => Random.nextLong).toDF() + .selectExpr(selectExpr: _*).createOrReplaceTempView("t1") + + prepareTable(dir, spark.sql("SELECT * FROM t1")) + + benchmark.addCase("Sum of single column") { _ => + spark.sql(s"SELECT sum(c$middle) FROM avroTable").collect() + } + + benchmark.run() + } + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("SQL Single Numeric Column Scan") { + Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => + numericScanBenchmark(1024 * 1024 * 15, dataType) + } + } + runBenchmark("Int and String Scan") { + intStringScanBenchmark(1024 * 1024 * 10) + } + runBenchmark("Partitioned Table Scan") { + partitionTableScanBenchmark(1024 * 1024 * 15) + } + runBenchmark("Repeated String Scan") { + repeatedStringScanBenchmark(1024 * 1024 * 10) + } + runBenchmark("String with Nulls Scan") { + for (fractionOfNulls <- List(0.0, 0.50, 0.95)) { + stringWithNullsScanBenchmark(1024 * 1024 * 10, fractionOfNulls) + } + } + runBenchmark("Single Column Scan From Wide Columns") { + columnsBenchmark(1024 * 1024 * 1, 100) + columnsBenchmark(1024 * 1024 * 1, 200) + columnsBenchmark(1024 * 1024 * 1, 300) + } + } +} diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala index df13b4a1c2d3a..0b11434757c93 100644 --- a/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/benchmark/AvroWriteBenchmark.scala @@ -19,22 +19,19 @@ package org.apache.spark.sql.execution.benchmark /** * Benchmark to measure Avro data sources write performance. - * Usage: - * 1. with spark-submit: bin/spark-submit --class - * 2. with sbt: build/sbt "avro/test:runMain " + * 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 "avro/test:runMain " + * Results will be written to "benchmarks/AvroWriteBenchmark-results.txt". + * }}} */ object AvroWriteBenchmark extends DataSourceWriteBenchmark { - def main(args: Array[String]): Unit = { - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - Avro writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 2481 / 2499 6.3 157.8 1.0X - Output Single Double Column 2705 / 2710 5.8 172.0 0.9X - Output Int and String Column 5539 / 5639 2.8 352.2 0.4X - Output Partitions 4613 / 5004 3.4 293.3 0.5X - Output Buckets 5554 / 5561 2.8 353.1 0.4X - */ - runBenchmark("Avro") + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runDataSourceBenchmark("Avro") } } diff --git a/external/avro/src/test/scala/org/apache/spark/sql/execution/datasources/AvroReadSchemaSuite.scala b/external/avro/src/test/scala/org/apache/spark/sql/execution/datasources/AvroReadSchemaSuite.scala new file mode 100644 index 0000000000000..aa58cbd3e2433 --- /dev/null +++ b/external/avro/src/test/scala/org/apache/spark/sql/execution/datasources/AvroReadSchemaSuite.scala @@ -0,0 +1,29 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +class AvroReadSchemaSuite + extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest + with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest + with ChangePositionTest { + + override val format: String = "avro" +} diff --git a/external/docker-integration-tests/pom.xml b/external/docker-integration-tests/pom.xml index f24254b698080..a4956ff5ee9cc 100644 --- a/external/docker-integration-tests/pom.xml +++ b/external/docker-integration-tests/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-docker-integration-tests_2.11 + spark-docker-integration-tests_2.12 jar Spark Project Docker Integration Tests http://spark.apache.org/ @@ -150,5 +150,11 @@ 10.5.0.5 jar + + com.microsoft.sqlserver + mssql-jdbc + 7.2.1.jre8 + test + diff --git a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala index f5930bc281e8c..32e56f03ee521 100644 --- a/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/DB2IntegrationSuite.scala @@ -158,4 +158,30 @@ class DB2IntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getInt(1) == 20) assert(rows(0).getString(2) == "1") } + + test("query JDBC option") { + val expectedResult = Set( + (42, "fred"), + (17, "dave") + ).map { case (x, y) => + Row(Integer.valueOf(x), String.valueOf(y)) + } + + val query = "SELECT x, y FROM tbl WHERE x > 10" + // query option to pass on the query string. + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("query", query) + .load() + assert(df.collect.toSet === expectedResult) + + // 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') + """.stripMargin.replaceAll("\n", " ")) + assert(sql("select x, y from queryOption").collect.toSet == expectedResult) + } } 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 609696bc8a2c7..e9e547eb9a263 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 @@ -129,7 +129,7 @@ abstract class DockerJDBCIntegrationSuite // Start the container and wait until the database can accept JDBC connections: docker.startContainer(containerId) jdbcUrl = db.getJdbcUrl(dockerIp, externalPort) - eventually(timeout(60.seconds), interval(1.seconds)) { + eventually(timeout(1.minute), interval(1.second)) { val conn = java.sql.DriverManager.getConnection(jdbcUrl) conn.close() } 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 new file mode 100644 index 0000000000000..efd7ca74c796b --- /dev/null +++ b/external/docker-integration-tests/src/test/scala/org/apache/spark/sql/jdbc/MsSqlServerIntegrationSuite.scala @@ -0,0 +1,205 @@ +/* + * 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 java.math.BigDecimal +import java.sql.{Connection, Date, Timestamp} +import java.util.Properties + +import org.apache.spark.tags.DockerTest + +@DockerTest +class MsSqlServerIntegrationSuite extends DockerJDBCIntegrationSuite { + override val db = new DatabaseOnDocker { + override val imageName = "mcr.microsoft.com/mssql/server:2017-GA-ubuntu" + override val env = Map( + "SA_PASSWORD" -> "Sapass123", + "ACCEPT_EULA" -> "Y" + ) + override val usesIpc = false + override val jdbcPort: Int = 1433 + + 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 = { + conn.prepareStatement("CREATE TABLE tbl (x INT, y VARCHAR (50))").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (42,'fred')").executeUpdate() + conn.prepareStatement("INSERT INTO tbl VALUES (17,'dave')").executeUpdate() + + conn.prepareStatement( + """ + |CREATE TABLE numbers ( + |a BIT, + |b TINYINT, c SMALLINT, d INT, e BIGINT, + |f FLOAT, f1 FLOAT(24), + |g REAL, + |h DECIMAL(5,2), i NUMERIC(10,5), + |j MONEY, k SMALLMONEY) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO numbers VALUES ( + |0, + |255, 32767, 2147483647, 9223372036854775807, + |123456789012345.123456789012345, 123456789012345.123456789012345, + |123456789012345.123456789012345, + |123, 12345.12, + |922337203685477.58, 214748.3647) + """.stripMargin).executeUpdate() + + conn.prepareStatement( + """ + |CREATE TABLE dates ( + |a DATE, b DATETIME, c DATETIME2, + |d DATETIMEOFFSET, e SMALLDATETIME, + |f TIME) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO dates VALUES ( + |'1991-11-09', '1999-01-01 13:23:35', '9999-12-31 23:59:59', + |'1901-05-09 23:59:59 +14:00', '1996-01-01 23:23:45', + |'13:31:24') + """.stripMargin).executeUpdate() + + conn.prepareStatement( + """ + |CREATE TABLE strings ( + |a CHAR(10), b VARCHAR(10), + |c NCHAR(10), d NVARCHAR(10), + |e BINARY(4), f VARBINARY(4), + |g TEXT, h NTEXT, + |i IMAGE) + """.stripMargin).executeUpdate() + conn.prepareStatement( + """ + |INSERT INTO strings VALUES ( + |'the', 'quick', + |'brown', 'fox', + |123456, 123456, + |'the', 'lazy', + |'dog') + """.stripMargin).executeUpdate() + } + + test("Basic test") { + val df = spark.read.jdbc(jdbcUrl, "tbl", new Properties) + val rows = df.collect() + assert(rows.length == 2) + val types = rows(0).toSeq.map(x => x.getClass.toString) + assert(types.length == 2) + assert(types(0).equals("class java.lang.Integer")) + assert(types(1).equals("class java.lang.String")) + } + + test("Numeric types") { + val df = spark.read.jdbc(jdbcUrl, "numbers", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val row = rows(0) + val types = row.toSeq.map(x => x.getClass.toString) + assert(types.length == 12) + assert(types(0).equals("class java.lang.Boolean")) + assert(types(1).equals("class java.lang.Integer")) + assert(types(2).equals("class java.lang.Short")) + assert(types(3).equals("class java.lang.Integer")) + assert(types(4).equals("class java.lang.Long")) + assert(types(5).equals("class java.lang.Double")) + assert(types(6).equals("class java.lang.Float")) + assert(types(7).equals("class java.lang.Float")) + assert(types(8).equals("class java.math.BigDecimal")) + assert(types(9).equals("class java.math.BigDecimal")) + assert(types(10).equals("class java.math.BigDecimal")) + assert(types(11).equals("class java.math.BigDecimal")) + assert(row.getBoolean(0) == false) + assert(row.getInt(1) == 255) + assert(row.getShort(2) == 32767) + assert(row.getInt(3) == 2147483647) + assert(row.getLong(4) == 9223372036854775807L) + assert(row.getDouble(5) == 1.2345678901234512E14) // float = float(53) has 15-digits precision + assert(row.getFloat(6) == 1.23456788103168E14) // float(24) has 7-digits precision + assert(row.getFloat(7) == 1.23456788103168E14) // real = float(24) + assert(row.getAs[BigDecimal](8).equals(new BigDecimal("123.00"))) + assert(row.getAs[BigDecimal](9).equals(new BigDecimal("12345.12000"))) + assert(row.getAs[BigDecimal](10).equals(new BigDecimal("922337203685477.5800"))) + assert(row.getAs[BigDecimal](11).equals(new BigDecimal("214748.3647"))) + } + + test("Date types") { + val df = spark.read.jdbc(jdbcUrl, "dates", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val row = rows(0) + val types = row.toSeq.map(x => x.getClass.toString) + assert(types.length == 6) + assert(types(0).equals("class java.sql.Date")) + assert(types(1).equals("class java.sql.Timestamp")) + assert(types(2).equals("class java.sql.Timestamp")) + assert(types(3).equals("class java.lang.String")) + assert(types(4).equals("class java.sql.Timestamp")) + assert(types(5).equals("class java.sql.Timestamp")) + assert(row.getAs[Date](0).equals(Date.valueOf("1991-11-09"))) + assert(row.getAs[Timestamp](1).equals(Timestamp.valueOf("1999-01-01 13:23:35.0"))) + assert(row.getAs[Timestamp](2).equals(Timestamp.valueOf("9999-12-31 23:59:59.0"))) + assert(row.getString(3).equals("1901-05-09 23:59:59.0000000 +14:00")) + assert(row.getAs[Timestamp](4).equals(Timestamp.valueOf("1996-01-01 23:24:00.0"))) + assert(row.getAs[Timestamp](5).equals(Timestamp.valueOf("1900-01-01 13:31:24.0"))) + } + + test("String types") { + val df = spark.read.jdbc(jdbcUrl, "strings", new Properties) + val rows = df.collect() + assert(rows.length == 1) + val row = rows(0) + val types = row.toSeq.map(x => x.getClass.toString) + assert(types.length == 9) + assert(types(0).equals("class java.lang.String")) + assert(types(1).equals("class java.lang.String")) + assert(types(2).equals("class java.lang.String")) + assert(types(3).equals("class java.lang.String")) + assert(types(4).equals("class [B")) + assert(types(5).equals("class [B")) + assert(types(6).equals("class java.lang.String")) + assert(types(7).equals("class java.lang.String")) + assert(types(8).equals("class [B")) + assert(row.getString(0).length == 10) + assert(row.getString(0).trim.equals("the")) + assert(row.getString(1).equals("quick")) + assert(row.getString(2).length == 10) + assert(row.getString(2).trim.equals("brown")) + assert(row.getString(3).equals("fox")) + assert(java.util.Arrays.equals(row.getAs[Array[Byte]](4), Array[Byte](0, 1, -30, 64))) + assert(java.util.Arrays.equals(row.getAs[Array[Byte]](5), Array[Byte](0, 1, -30, 64))) + assert(row.getString(6).equals("the")) + assert(row.getString(7).equals("lazy")) + assert(java.util.Arrays.equals(row.getAs[Array[Byte]](8), Array[Byte](100, 111, 103))) + } + + test("Basic write test") { + val df1 = spark.read.jdbc(jdbcUrl, "numbers", new Properties) + val df2 = spark.read.jdbc(jdbcUrl, "dates", new Properties) + val df3 = spark.read.jdbc(jdbcUrl, "strings", new Properties) + df1.write.jdbc(jdbcUrl, "numberscopy", new Properties) + df2.write.jdbc(jdbcUrl, "datescopy", new Properties) + df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) + } +} 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 a70ed98b52d5d..9cd5c4ec41a52 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 @@ -21,6 +21,7 @@ import java.math.BigDecimal import java.sql.{Connection, Date, Timestamp} import java.util.Properties +import org.apache.spark.sql.{Row, SaveMode} import org.apache.spark.tags.DockerTest @DockerTest @@ -152,4 +153,30 @@ class MySQLIntegrationSuite extends DockerJDBCIntegrationSuite { df2.write.jdbc(jdbcUrl, "datescopy", new Properties) df3.write.jdbc(jdbcUrl, "stringscopy", new Properties) } + + test("query JDBC option") { + val expectedResult = Set( + (42, "fred"), + (17, "dave") + ).map { case (x, y) => + Row(Integer.valueOf(x), String.valueOf(y)) + } + + val query = "SELECT x, y FROM tbl WHERE x > 10" + // query option to pass on the query string. + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("query", query) + .load() + assert(df.collect.toSet === expectedResult) + + // 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') + """.stripMargin.replaceAll("\n", " ")) + assert(sql("select x, y from queryOption").collect.toSet == expectedResult) + } } 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 70d294d0ca650..8cdc4a1806b2b 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 @@ -39,13 +39,16 @@ import org.apache.spark.tags.DockerTest * while Spark QA test run. * * The following would be the steps to test this - * 1. Pull oracle 11g image - docker pull wnameless/oracle-xe-11g - * 2. Start docker - sudo service docker start - * 3. Download oracle 11g driver jar and put it in maven local repo: + * 1. Build Oracle database in Docker, please refer below link about how to. + * https://github.com/oracle/docker-images/blob/master/OracleDatabase/SingleInstance/README.md + * 2. export ORACLE_DOCKER_IMAGE_NAME=$ORACLE_DOCKER_IMAGE_NAME + * Pull oracle $ORACLE_DOCKER_IMAGE_NAME image - docker pull $ORACLE_DOCKER_IMAGE_NAME + * 3. Start docker - sudo service docker start + * 4. Download oracle 11g driver jar and put it in maven local repo: * (com/oracle/ojdbc6/11.2.0.2.0/ojdbc6-11.2.0.2.0.jar) - * 4. The timeout and interval parameter to be increased from 60,1 to a high value for oracle test + * 5. The timeout and interval parameter to be increased from 60,1 to a high value for oracle test * in DockerJDBCIntegrationSuite.scala (Locally tested with 200,200 and executed successfully). - * 5. Run spark test - ./build/sbt "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" + * 6. Run spark test - ./build/sbt "test-only org.apache.spark.sql.jdbc.OracleIntegrationSuite" * * All tests in this suite are ignored because of the dependency with the oracle jar from maven * repository. @@ -55,7 +58,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo import testImplicits._ override val db = new DatabaseOnDocker { - override val imageName = "wnameless/oracle-xe-11g:16.04" + override val imageName = sys.env("ORACLE_DOCKER_IMAGE_NAME") override val env = Map( "ORACLE_ROOT_PASSWORD" -> "oracle" ) @@ -154,7 +157,7 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo // A value with fractions from DECIMAL(3, 2) is correct: assert(row.getDecimal(1).compareTo(BigDecimal.valueOf(1.23)) == 0) // A value > Int.MaxValue from DECIMAL(10) is correct: - assert(row.getDecimal(2).compareTo(BigDecimal.valueOf(9999999999l)) == 0) + assert(row.getDecimal(2).compareTo(BigDecimal.valueOf(9999999999L)) == 0) } @@ -373,8 +376,8 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo val e = intercept[org.apache.spark.SparkException] { spark.read.jdbc(jdbcUrl, "tableWithCustomSchema", new Properties()).collect() } - assert(e.getMessage.contains( - "requirement failed: Decimal precision 39 exceeds max precision 38")) + assert(e.getCause().isInstanceOf[ArithmeticException]) + assert(e.getMessage.contains("Decimal precision 39 exceeds max precision 38")) // custom schema can read data val props = new Properties() @@ -482,4 +485,32 @@ class OracleIntegrationSuite extends DockerJDBCIntegrationSuite with SharedSQLCo } assert(df2.collect.toSet === expectedResult) } + + test("query JDBC option") { + val expectedResult = Set( + (1, "1991-11-09", "1996-01-01 01:23:45") + ).map { case (id, date, timestamp) => + Row(BigDecimal.valueOf(id), Date.valueOf(date), Timestamp.valueOf(timestamp)) + } + + val query = "SELECT id, d, t FROM datetime WHERE id = 1" + // query option to pass on the query string. + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("query", query) + .option("oracle.jdbc.mapDateToTimestamp", "false") + .load() + assert(df.collect.toSet === expectedResult) + + // 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', + | oracle.jdbc.mapDateToTimestamp false) + """.stripMargin.replaceAll("\n", " ")) + assert(sql("select id, d, t from queryOption").collect.toSet == expectedResult) + } } 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 be32cb89f4886..89da9a1de6f74 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 @@ -21,6 +21,7 @@ import java.sql.Connection import java.util.Properties import org.apache.spark.sql.Column +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.types.{ArrayType, DecimalType, FloatType, ShortType} import org.apache.spark.tags.DockerTest @@ -28,7 +29,7 @@ import org.apache.spark.tags.DockerTest @DockerTest class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { override val db = new DatabaseOnDocker { - override val imageName = "postgres:9.4.5" + override val imageName = "postgres:11.4" override val env = Map( "POSTGRES_PASSWORD" -> "rootpass" ) @@ -46,14 +47,15 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { conn.prepareStatement("CREATE TABLE bar (c0 text, c1 integer, c2 double precision, c3 bigint, " + "c4 bit(1), c5 bit(10), c6 bytea, c7 boolean, c8 inet, c9 cidr, " + "c10 integer[], c11 text[], c12 real[], c13 numeric(2,2)[], c14 enum_type, " - + "c15 float4, c16 smallint)").executeUpdate() + + "c15 float4, c16 smallint, c17 numeric[])").executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES ('hello', 42, 1.25, 123456789012345, B'0', " + "B'1000100101', E'\\\\xDEADBEEF', true, '172.16.0.42', '192.168.0.0/16', " - + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1)""" + + """'{1, 2}', '{"a", null, "b"}', '{0.11, 0.22}', '{0.11, 0.22}', 'd1', 1.01, 1, """ + + "'{111.2222, 333.4444}')" ).executeUpdate() conn.prepareStatement("INSERT INTO bar VALUES (null, null, null, null, null, " + "null, null, null, null, null, " - + "null, null, null, null, null, null, null)" + + "null, null, null, null, null, null, null, null)" ).executeUpdate() conn.prepareStatement("CREATE TABLE ts_with_timezone " + @@ -85,7 +87,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows.length == 2) // Test the types, and values using the first row. val types = rows(0).toSeq.map(x => x.getClass) - assert(types.length == 17) + assert(types.length == 18) assert(classOf[String].isAssignableFrom(types(0))) assert(classOf[java.lang.Integer].isAssignableFrom(types(1))) assert(classOf[java.lang.Double].isAssignableFrom(types(2))) @@ -103,6 +105,7 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(classOf[String].isAssignableFrom(types(14))) assert(classOf[java.lang.Float].isAssignableFrom(types(15))) assert(classOf[java.lang.Short].isAssignableFrom(types(16))) + assert(classOf[Seq[BigDecimal]].isAssignableFrom(types(17))) assert(rows(0).getString(0).equals("hello")) assert(rows(0).getInt(1) == 42) assert(rows(0).getDouble(2) == 1.25) @@ -123,6 +126,8 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getString(14) == "d1") assert(rows(0).getFloat(15) == 1.01f) assert(rows(0).getShort(16) == 1) + assert(rows(0).getSeq(17) == + Seq("111.222200000000000000", "333.444400000000000000").map(BigDecimal(_).bigDecimal)) // Test reading null values using the second row. assert(0.until(16).forall(rows(1).isNullAt(_))) @@ -176,4 +181,42 @@ class PostgresIntegrationSuite extends DockerJDBCIntegrationSuite { assert(rows(0).getSeq(8) == Seq("""{"a": "foo", "b": "bar"}""", """{"a": 1, "b": 2}""")) assert(rows(0).getSeq(9) == Seq("""{"a": 1, "b": 2, "c": 3}""")) } + + test("query JDBC option") { + val expectedResult = Set( + (42, 123456789012345L) + ).map { case (c1, c3) => + Row(Integer.valueOf(c1), java.lang.Long.valueOf(c3)) + } + + val query = "SELECT c1, c3 FROM bar WHERE c1 IS NOT NULL" + // query option to pass on the query string. + val df = spark.read.format("jdbc") + .option("url", jdbcUrl) + .option("query", query) + .load() + assert(df.collect.toSet === expectedResult) + + // 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') + """.stripMargin.replaceAll("\n", " ")) + assert(sql("select c1, c3 from queryOption").collect.toSet == expectedResult) + } + + test("write byte as smallint") { + sqlContext.createDataFrame(Seq((1.toByte, 2.toShort))) + .write.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val df = sqlContext.read.jdbc(jdbcUrl, "byte_to_smallint_test", new Properties) + val schema = df.schema + assert(schema.head.dataType == ShortType) + assert(schema(1).dataType == ShortType) + val rows = df.collect() + assert(rows.length === 1) + assert(rows(0).getShort(0) === 1) + assert(rows(0).getShort(1) === 2) + } } diff --git a/external/docker/spark-test/base/Dockerfile b/external/docker/spark-test/base/Dockerfile index c70cd71367679..c1fd630d0b665 100644 --- a/external/docker/spark-test/base/Dockerfile +++ b/external/docker/spark-test/base/Dockerfile @@ -25,13 +25,13 @@ RUN apt-get update && \ apt-get install -y less openjdk-8-jre-headless iproute2 vim-tiny sudo openssh-server && \ rm -rf /var/lib/apt/lists/* -ENV SCALA_VERSION 2.11.8 +ENV SCALA_VERSION 2.12.8 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark ENV PATH $SPARK_HOME:$SCALA_HOME/bin:$PATH # Install Scala -ADD http://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz / +ADD https://www.scala-lang.org/files/archive/scala-$SCALA_VERSION.tgz / RUN (cd / && gunzip < scala-$SCALA_VERSION.tgz)|(cd /opt && tar -xvf -) RUN rm /scala-$SCALA_VERSION.tgz diff --git a/external/kafka-0-10-assembly/pom.xml b/external/kafka-0-10-assembly/pom.xml index 4f9c3163b2408..f2dcf5d217a89 100644 --- a/external/kafka-0-10-assembly/pom.xml +++ b/external/kafka-0-10-assembly/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-streaming-kafka-0-10-assembly_2.11 + spark-streaming-kafka-0-10-assembly_2.12 jar Spark Integration for Kafka 0.10 Assembly http://spark.apache.org/ diff --git a/external/kafka-0-10-sql/pom.xml b/external/kafka-0-10-sql/pom.xml index efd0862fb58ee..827ceb89a0c34 100644 --- a/external/kafka-0-10-sql/pom.xml +++ b/external/kafka-0-10-sql/pom.xml @@ -20,23 +20,26 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml org.apache.spark - spark-sql-kafka-0-10_2.11 + spark-sql-kafka-0-10_2.12 sql-kafka-0-10 - - 2.0.0 jar Kafka 0.10+ Source for Structured Streaming http://spark.apache.org/ + + org.apache.spark + spark-token-provider-kafka-0-10_${scala.binary.version} + ${project.version} + org.apache.spark spark-sql_${scala.binary.version} @@ -89,6 +92,13 @@ + + + org.apache.zookeeper + zookeeper + 3.4.7 + test + net.sf.jopt-simple jopt-simple @@ -101,6 +111,11 @@ ${jetty.version} test + + org.mockito + mockito-core + test + org.scalacheck scalacheck_${scala.binary.version} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala index cd680adf44365..fc177cdc9037e 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/CachedKafkaProducer.scala @@ -28,6 +28,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.kafka010.{KafkaConfigUpdater, KafkaRedactionUtil} private[kafka010] object CachedKafkaProducer extends Logging { @@ -35,15 +36,13 @@ private[kafka010] object CachedKafkaProducer extends Logging { private val defaultCacheExpireTimeout = TimeUnit.MINUTES.toMillis(10) - private lazy val cacheExpireTimeout: Long = - Option(SparkEnv.get).map(_.conf.getTimeAsMs( - "spark.kafka.producer.cache.timeout", - s"${defaultCacheExpireTimeout}ms")).getOrElse(defaultCacheExpireTimeout) + private lazy val cacheExpireTimeout: Long = Option(SparkEnv.get) + .map(_.conf.get(PRODUCER_CACHE_TIMEOUT)) + .getOrElse(defaultCacheExpireTimeout) private val cacheLoader = new CacheLoader[Seq[(String, Object)], Producer] { override def load(config: Seq[(String, Object)]): Producer = { - val configMap = config.map(x => x._1 -> x._2).toMap.asJava - createKafkaProducer(configMap) + createKafkaProducer(config) } } @@ -52,8 +51,11 @@ private[kafka010] object CachedKafkaProducer extends Logging { notification: RemovalNotification[Seq[(String, Object)], Producer]): Unit = { val paramsSeq: Seq[(String, Object)] = notification.getKey val producer: Producer = notification.getValue - logDebug( - s"Evicting kafka producer $producer params: $paramsSeq, due to ${notification.getCause}") + if (log.isDebugEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logDebug(s"Evicting kafka producer $producer params: $redactedParamsSeq, " + + s"due to ${notification.getCause}") + } close(paramsSeq, producer) } } @@ -63,9 +65,12 @@ private[kafka010] object CachedKafkaProducer extends Logging { .removalListener(removalListener) .build[Seq[(String, Object)], Producer](cacheLoader) - private def createKafkaProducer(producerConfiguration: ju.Map[String, Object]): Producer = { - val kafkaProducer: Producer = new Producer(producerConfiguration) - logDebug(s"Created a new instance of KafkaProducer for $producerConfiguration.") + private def createKafkaProducer(paramsSeq: Seq[(String, Object)]): Producer = { + val kafkaProducer: Producer = new Producer(paramsSeq.toMap.asJava) + if (log.isDebugEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logDebug(s"Created a new instance of KafkaProducer for $redactedParamsSeq.") + } kafkaProducer } @@ -75,7 +80,11 @@ private[kafka010] object CachedKafkaProducer extends Logging { * one instance per specified kafkaParams. */ private[kafka010] def getOrCreate(kafkaParams: ju.Map[String, Object]): Producer = { - val paramsSeq: Seq[(String, Object)] = paramsToSeq(kafkaParams) + val updatedKafkaProducerConfiguration = + KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() + val paramsSeq: Seq[(String, Object)] = paramsToSeq(updatedKafkaProducerConfiguration) try { guavaCache.get(paramsSeq) } catch { @@ -99,7 +108,10 @@ private[kafka010] object CachedKafkaProducer extends Logging { /** Auto close on cache evict */ private def close(paramsSeq: Seq[(String, Object)], producer: Producer): Unit = { try { - logInfo(s"Closing the KafkaProducer with params: ${paramsSeq.mkString("\n")}.") + if (log.isInfoEnabled()) { + val redactedParamsSeq = KafkaRedactionUtil.redactParams(paramsSeq) + logInfo(s"Closing the KafkaProducer with params: ${redactedParamsSeq.mkString("\n")}.") + } producer.close() } catch { case NonFatal(e) => logWarning("Error while closing kafka producer.", e) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala index 66511b3065415..7bb829c282eba 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/ConsumerStrategy.scala @@ -25,6 +25,8 @@ import org.apache.kafka.clients.consumer.{Consumer, KafkaConsumer} import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition +import org.apache.spark.kafka010.KafkaConfigUpdater + /** * Subscribe allows you to subscribe to a fixed collection of topics. * SubscribePattern allows you to use a regex to specify topics of interest. @@ -34,18 +36,29 @@ import org.apache.kafka.common.TopicPartition * All three strategies have overloaded constructors that allow you to specify * the starting offset for a particular partition. */ -sealed trait ConsumerStrategy { +private[kafka010] sealed trait ConsumerStrategy { /** Create a [[KafkaConsumer]] and subscribe to topics according to a desired strategy */ def createConsumer(kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] + + /** + * Updates the parameters with security if needed. + * Added a function to hide internals and reduce code duplications because all strategy uses it. + */ + protected def setAuthenticationConfigIfNeeded(kafkaParams: ju.Map[String, Object]) = + KafkaConfigUpdater("source", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() } /** * Specify a fixed collection of partitions. */ -case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStrategy { +private[kafka010] case class AssignStrategy(partitions: Array[TopicPartition]) + extends ConsumerStrategy { override def createConsumer( kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](updatedKafkaParams) consumer.assign(ju.Arrays.asList(partitions: _*)) consumer } @@ -56,10 +69,11 @@ case class AssignStrategy(partitions: Array[TopicPartition]) extends ConsumerStr /** * Subscribe to a fixed collection of topics. */ -case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { +private[kafka010] case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { override def createConsumer( kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](updatedKafkaParams) consumer.subscribe(topics.asJava) consumer } @@ -70,10 +84,12 @@ case class SubscribeStrategy(topics: Seq[String]) extends ConsumerStrategy { /** * Use a regex to specify topics of interest. */ -case class SubscribePatternStrategy(topicPattern: String) extends ConsumerStrategy { +private[kafka010] case class SubscribePatternStrategy(topicPattern: String) + extends ConsumerStrategy { override def createConsumer( kafkaParams: ju.Map[String, Object]): Consumer[Array[Byte], Array[Byte]] = { - val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](updatedKafkaParams) consumer.subscribe( ju.regex.Pattern.compile(topicPattern), new NoOpConsumerRebalanceListener()) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.scala new file mode 100644 index 0000000000000..e3c8536d22070 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatch.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.kafka010 + +import org.apache.kafka.common.TopicPartition + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReaderFactory} + + +private[kafka010] class KafkaBatch( + strategy: ConsumerStrategy, + sourceOptions: Map[String, String], + specifiedKafkaParams: Map[String, String], + failOnDataLoss: Boolean, + startingOffsets: KafkaOffsetRangeLimit, + endingOffsets: KafkaOffsetRangeLimit) + extends Batch with Logging { + assert(startingOffsets != LatestOffsetRangeLimit, + "Starting offset not allowed to be set to latest offsets.") + assert(endingOffsets != EarliestOffsetRangeLimit, + "Ending offset not allowed to be set to earliest offsets.") + + private val pollTimeoutMs = sourceOptions.getOrElse( + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, + (SparkEnv.get.conf.get(NETWORK_TIMEOUT) * 1000L).toString + ).toLong + + override def planInputPartitions(): Array[InputPartition] = { + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = KafkaSourceProvider.batchUniqueGroupId(sourceOptions) + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy, + KafkaSourceProvider.kafkaParamsForDriver(specifiedKafkaParams), + sourceOptions, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + // Leverage the KafkaReader to obtain the relevant partition offsets + val (fromPartitionOffsets, untilPartitionOffsets) = { + try { + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) + } finally { + kafkaOffsetReader.close() + } + } + + // Obtain topicPartitions in both from and until partition offset, ignoring + // topic partitions that were added and/or deleted between the two above calls. + if (fromPartitionOffsets.keySet != untilPartitionOffsets.keySet) { + implicit val topicOrdering: Ordering[TopicPartition] = Ordering.by(t => t.topic()) + val fromTopics = fromPartitionOffsets.keySet.toList.sorted.mkString(",") + val untilTopics = untilPartitionOffsets.keySet.toList.sorted.mkString(",") + throw new IllegalStateException("different topic partitions " + + s"for starting offsets topics[${fromTopics}] and " + + s"ending offsets topics[${untilTopics}]") + } + + // Calculate offset ranges + val offsetRanges = untilPartitionOffsets.keySet.map { tp => + val fromOffset = fromPartitionOffsets.getOrElse(tp, + // This should not happen since topicPartitions contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset")) + val untilOffset = untilPartitionOffsets(tp) + KafkaOffsetRange(tp, fromOffset, untilOffset, None) + }.toArray + + val executorKafkaParams = + KafkaSourceProvider.kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId) + offsetRanges.map { range => + new KafkaBatchInputPartition( + range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, false) + }.toArray + } + + override def createReaderFactory(): PartitionReaderFactory = { + KafkaBatchReaderFactory + } + + override def toString: String = + s"KafkaBatch(strategy=$strategy, start=$startingOffsets, end=$endingOffsets)" +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala new file mode 100644 index 0000000000000..cbc2fbfce3199 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchPartitionReader.scala @@ -0,0 +1,108 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.sources.v2.reader._ + + +/** A [[InputPartition]] for reading Kafka data in a batch based streaming query. */ +private[kafka010] case class KafkaBatchInputPartition( + offsetRange: KafkaOffsetRange, + executorKafkaParams: ju.Map[String, Object], + pollTimeoutMs: Long, + failOnDataLoss: Boolean, + reuseKafkaConsumer: Boolean) extends InputPartition + +private[kafka010] object KafkaBatchReaderFactory extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + val p = partition.asInstanceOf[KafkaBatchInputPartition] + KafkaBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs, + p.failOnDataLoss, p.reuseKafkaConsumer) + } +} + +/** A [[PartitionReader]] for reading Kafka data in a micro-batch streaming query. */ +private case class KafkaBatchPartitionReader( + offsetRange: KafkaOffsetRange, + executorKafkaParams: ju.Map[String, Object], + pollTimeoutMs: Long, + failOnDataLoss: Boolean, + reuseKafkaConsumer: Boolean) extends PartitionReader[InternalRow] with Logging { + + private val consumer = KafkaDataConsumer.acquire( + offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer) + + private val rangeToRead = resolveRange(offsetRange) + private val converter = new KafkaRecordToUnsafeRowConverter + + private var nextOffset = rangeToRead.fromOffset + private var nextRow: UnsafeRow = _ + + override def next(): Boolean = { + if (nextOffset < rangeToRead.untilOffset) { + val record = consumer.get(nextOffset, rangeToRead.untilOffset, pollTimeoutMs, failOnDataLoss) + if (record != null) { + nextRow = converter.toUnsafeRow(record) + nextOffset = record.offset + 1 + true + } else { + false + } + } else { + false + } + } + + override def get(): UnsafeRow = { + assert(nextRow != null) + nextRow + } + + override def close(): Unit = { + consumer.release() + } + + private def resolveRange(range: KafkaOffsetRange): KafkaOffsetRange = { + if (range.fromOffset < 0 || range.untilOffset < 0) { + // Late bind the offset range + val availableOffsetRange = consumer.getAvailableOffsetRange() + val fromOffset = if (range.fromOffset < 0) { + assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST, + s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}") + availableOffsetRange.earliest + } else { + range.fromOffset + } + val untilOffset = if (range.untilOffset < 0) { + assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST, + s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}") + availableOffsetRange.latest + } else { + range.untilOffset + } + KafkaOffsetRange(range.topicPartition, fromOffset, untilOffset, None) + } else { + range + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.scala new file mode 100644 index 0000000000000..47ec07ae128d2 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaBatchWrite.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.kafka010 + +import java.{util => ju} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.types.StructType + +/** + * A [[BatchWrite]] for Kafka writing. Responsible for generating the writer factory. + * + * @param topic The topic this writer is responsible for. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ +private[kafka010] class KafkaBatchWrite( + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) + extends BatchWrite { + + validateQuery(schema.toAttributes, producerParams, topic) + + override def createBatchWriterFactory(): KafkaBatchWriterFactory = + KafkaBatchWriterFactory(topic, producerParams, schema) + + override def commit(messages: Array[WriterCommitMessage]): Unit = {} + override def abort(messages: Array[WriterCommitMessage]): Unit = {} +} + +/** + * A [[DataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to + * generate the per-task data writers. + * @param topic The topic that should be written to. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ +private case class KafkaBatchWriterFactory( + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) + extends DataWriterFactory { + + override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { + new KafkaDataWriter(topic, producerParams, schema.toAttributes) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala similarity index 79% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala index 1753a28fba2fb..8c26bc0d172ba 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaContinuousStream.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} import java.util.concurrent.TimeoutException -import org.apache.kafka.clients.consumer.{ConsumerRecord, OffsetOutOfRangeException} +import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, OffsetOutOfRangeException} import org.apache.kafka.common.TopicPartition import org.apache.spark.TaskContext @@ -30,55 +30,75 @@ import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** - * A [[ContinuousReadSupport]] for data from kafka. + * A [[ContinuousStream]] for data from kafka. * * @param offsetReader a reader used to get kafka offsets. Note that the actual data will be * read by per-task consumers generated later. * @param kafkaParams String params for per-task Kafka consumers. - * @param sourceOptions The [[org.apache.spark.sql.sources.v2.DataSourceOptions]] params which - * are not Kafka consumer params. + * @param options Params which are not Kafka consumer params. * @param metadataPath Path to a directory this reader can use for writing metadata. * @param initialOffsets The Kafka offsets to start reading data at. * @param failOnDataLoss Flag indicating whether reading should fail in data loss * scenarios, where some offsets after the specified initial ones can't be * properly read. */ -class KafkaContinuousReadSupport( +class KafkaContinuousStream( offsetReader: KafkaOffsetReader, kafkaParams: ju.Map[String, Object], - sourceOptions: Map[String, String], + options: CaseInsensitiveStringMap, metadataPath: String, initialOffsets: KafkaOffsetRangeLimit, failOnDataLoss: Boolean) - extends ContinuousReadSupport with Logging { + extends ContinuousStream with Logging { + + private val pollTimeoutMs = + options.getLong(KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, 512) + + // Initialized when creating reader factories. If this diverges from the partitions at the latest + // offsets, we need to reconfigure. + // Exposed outside this object only for unit tests. + @volatile private[sql] var knownPartitions: Set[TopicPartition] = _ - private val pollTimeoutMs = sourceOptions.getOrElse("kafkaConsumer.pollTimeoutMs", "512").toLong override def initialOffset(): Offset = { val offsets = initialOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(offsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => offsetReader.fetchSpecificOffsets(p, reportDataLoss) } logInfo(s"Initial offsets: $offsets") offsets } - override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema - - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new KafkaContinuousScanConfigBuilder(fullSchema(), start, offsetReader, reportDataLoss) - } - override def deserializeOffset(json: String): Offset = { KafkaSourceOffset(JsonUtils.partitionOffsets(json)) } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffsets = config.asInstanceOf[KafkaContinuousScanConfig].startOffsets + override def planInputPartitions(start: Offset): Array[InputPartition] = { + val oldStartPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets + + val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet + val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) + val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) + + val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) + if (deletedPartitions.nonEmpty) { + val message = if ( + offsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) + } + + val startOffsets = newPartitionOffsets ++ + oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) + knownPartitions = startOffsets.keySet + startOffsets.toSeq.map { case (topicPartition, start) => KafkaContinuousInputPartition( @@ -86,8 +106,7 @@ class KafkaContinuousReadSupport( }.toArray } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { KafkaContinuousReaderFactory } @@ -105,9 +124,8 @@ class KafkaContinuousReadSupport( KafkaSourceOffset(mergedMap) } - override def needsReconfiguration(config: ScanConfig): Boolean = { - val knownPartitions = config.asInstanceOf[KafkaContinuousScanConfig].knownPartitions - offsetReader.fetchLatestOffsets().keySet != knownPartitions + override def needsReconfiguration(): Boolean = { + offsetReader.fetchLatestOffsets(None).keySet != knownPartitions } override def toString(): String = s"KafkaSource[$offsetReader]" @@ -151,41 +169,6 @@ object KafkaContinuousReaderFactory extends ContinuousPartitionReaderFactory { } } -class KafkaContinuousScanConfigBuilder( - schema: StructType, - startOffset: Offset, - offsetReader: KafkaOffsetReader, - reportDataLoss: String => Unit) - extends ScanConfigBuilder { - - override def build(): ScanConfig = { - val oldStartPartitionOffsets = KafkaSourceOffset.getPartitionOffsets(startOffset) - - val currentPartitionSet = offsetReader.fetchEarliestOffsets().keySet - val newPartitions = currentPartitionSet.diff(oldStartPartitionOffsets.keySet) - val newPartitionOffsets = offsetReader.fetchEarliestOffsets(newPartitions.toSeq) - - val deletedPartitions = oldStartPartitionOffsets.keySet.diff(currentPartitionSet) - if (deletedPartitions.nonEmpty) { - reportDataLoss(s"Some partitions were deleted: $deletedPartitions") - } - - val startOffsets = newPartitionOffsets ++ - oldStartPartitionOffsets.filterKeys(!deletedPartitions.contains(_)) - KafkaContinuousScanConfig(schema, startOffsets) - } -} - -case class KafkaContinuousScanConfig( - readSchema: StructType, - startOffsets: Map[TopicPartition, Long]) - extends ScanConfig { - - // Created when building the scan config builder. If this diverges from the partitions at the - // latest offsets, we need to reconfigure the kafka read support. - def knownPartitions: Set[TopicPartition] = startOffsets.keySet -} - /** * A per-task data reader for continuous Kafka processing. * diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala index 7b1314bc8c3c0..cbb99fd7118e0 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataConsumer.scala @@ -27,6 +27,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.internal.Logging +import org.apache.spark.kafka010.KafkaConfigUpdater import org.apache.spark.sql.kafka010.KafkaDataConsumer.AvailableOffsetRange import org.apache.spark.sql.kafka010.KafkaSourceProvider._ import org.apache.spark.util.UninterruptibleThread @@ -197,7 +198,10 @@ private[kafka010] case class InternalKafkaConsumer( /** Create a KafkaConsumer to fetch records for `topicPartition` */ private def createConsumer: KafkaConsumer[Array[Byte], Array[Byte]] = { - val c = new KafkaConsumer[Array[Byte], Array[Byte]](kafkaParams) + val updatedKafkaParams = KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() + val c = new KafkaConsumer[Array[Byte], Array[Byte]](updatedKafkaParams) val tps = new ju.ArrayList[TopicPartition]() tps.add(topicPartition) c.assign(tps) @@ -343,7 +347,7 @@ private[kafka010] case class InternalKafkaConsumer( * consumer's `isolation.level` is `read_committed`), it will return a `FetchedRecord` with the * next offset to fetch. * - * This method also will try the best to detect data loss. If `failOnDataLoss` is true`, it will + * This method also will try the best to detect data loss. If `failOnDataLoss` is `true`, it will * throw an exception when we detect an unavailable offset. If `failOnDataLoss` is `false`, this * method will return `null` if the next available record is within [offset, untilOffset). * @@ -519,7 +523,7 @@ private[kafka010] object KafkaDataConsumer extends Logging { // tasks simultaneously using consumers than the capacity. private lazy val cache = { val conf = SparkEnv.get.conf - val capacity = conf.getInt("spark.sql.kafkaConsumerCache.capacity", 64) + val capacity = conf.get(CONSUMER_CACHE_CAPACITY) new ju.LinkedHashMap[CacheKey, InternalKafkaConsumer](capacity, 0.75f, true) { override def removeEldestEntry( entry: ju.Map.Entry[CacheKey, InternalKafkaConsumer]): Boolean = { diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.scala new file mode 100644 index 0000000000000..884773452b2a5 --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaDataWriter.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.kafka010 + +import java.{util => ju} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.sources.v2.writer._ + +/** + * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we + * don't need to really send one. + */ +private case object KafkaDataWriterCommitMessage extends WriterCommitMessage + +/** + * A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to + * process incoming rows. + * + * @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred + * from a `topic` field in the incoming data. + * @param producerParams Parameters to use for the Kafka producer. + * @param inputSchema The attributes in the input data. + */ +private[kafka010] class KafkaDataWriter( + targetTopic: Option[String], + producerParams: ju.Map[String, Object], + inputSchema: Seq[Attribute]) + extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { + + private lazy val producer = CachedKafkaProducer.getOrCreate(producerParams) + + def write(row: InternalRow): Unit = { + checkForErrors() + sendRow(row, producer) + } + + def commit(): WriterCommitMessage = { + // Send is asynchronous, but we can't commit until all rows are actually in Kafka. + // This requires flushing and then checking that no callbacks produced errors. + // We also check for errors before to fail as soon as possible - the check is cheap. + checkForErrors() + producer.flush() + checkForErrors() + KafkaDataWriterCommitMessage + } + + def abort(): Unit = {} + + def close(): Unit = { + checkForErrors() + if (producer != null) { + producer.flush() + checkForErrors() + CachedKafkaProducer.close(producerParams) + } + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala similarity index 69% rename from external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala rename to external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala index bb4de674c3c72..9fb338f0e6688 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchReadSupport.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchStream.scala @@ -22,24 +22,25 @@ import java.io._ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils +import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.UnsafeRow -import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder} -import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchReadSupport +import org.apache.spark.sql.execution.streaming.{HDFSMetadataLog, SerializedOffset} +import org.apache.spark.sql.execution.streaming.sources.RateControlMicroBatchStream import org.apache.spark.sql.kafka010.KafkaSourceProvider.{INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_FALSE, INSTRUCTION_FOR_FAIL_ON_DATA_LOSS_TRUE} -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.UninterruptibleThread /** - * A [[MicroBatchReadSupport]] that reads data from Kafka. + * A [[MicroBatchStream]] that reads data from Kafka. * * The [[KafkaSourceOffset]] is the custom [[Offset]] defined for this source that contains * a map of TopicPartition -> offset. Note that this offset is 1 + (available offset). For @@ -54,20 +55,20 @@ import org.apache.spark.util.UninterruptibleThread * To avoid this issue, you should make sure stopping the query before stopping the Kafka brokers * and not use wrong broker addresses. */ -private[kafka010] class KafkaMicroBatchReadSupport( +private[kafka010] class KafkaMicroBatchStream( kafkaOffsetReader: KafkaOffsetReader, executorKafkaParams: ju.Map[String, Object], - options: DataSourceOptions, + options: CaseInsensitiveStringMap, metadataPath: String, startingOffsets: KafkaOffsetRangeLimit, - failOnDataLoss: Boolean) extends RateControlMicroBatchReadSupport with Logging { + failOnDataLoss: Boolean) extends RateControlMicroBatchStream with Logging { private val pollTimeoutMs = options.getLong( - "kafkaConsumer.pollTimeoutMs", - SparkEnv.get.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L) + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, + SparkEnv.get.conf.get(NETWORK_TIMEOUT) * 1000L) - private val maxOffsetsPerTrigger = - Option(options.get("maxOffsetsPerTrigger").orElse(null)).map(_.toLong) + private val maxOffsetsPerTrigger = Option(options.get(KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER)) + .map(_.toLong) private val rangeCalculator = KafkaOffsetRangeCalculator(options) @@ -84,7 +85,7 @@ private[kafka010] class KafkaMicroBatchReadSupport( override def latestOffset(start: Offset): Offset = { val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets - val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets() + val latestPartitionOffsets = kafkaOffsetReader.fetchLatestOffsets(Some(startPartitionOffsets)) endPartitionOffsets = KafkaSourceOffset(maxOffsetsPerTrigger.map { maxOffsets => rateLimit(maxOffsets, startPartitionOffsets, latestPartitionOffsets) }.getOrElse { @@ -93,16 +94,9 @@ private[kafka010] class KafkaMicroBatchReadSupport( endPartitionOffsets } - override def fullSchema(): StructType = KafkaOffsetReader.kafkaSchema - - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) - } - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startPartitionOffsets = sc.start.asInstanceOf[KafkaSourceOffset].partitionToOffsets - val endPartitionOffsets = sc.end.get.asInstanceOf[KafkaSourceOffset].partitionToOffsets + override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = { + val startPartitionOffsets = start.asInstanceOf[KafkaSourceOffset].partitionToOffsets + val endPartitionOffsets = end.asInstanceOf[KafkaSourceOffset].partitionToOffsets // Find the new partitions, and get their earliest offsets val newPartitions = endPartitionOffsets.keySet.diff(startPartitionOffsets.keySet) @@ -122,7 +116,13 @@ private[kafka010] class KafkaMicroBatchReadSupport( // Find deleted partitions, and report data loss if required val deletedPartitions = startPartitionOffsets.keySet.diff(endPartitionOffsets.keySet) if (deletedPartitions.nonEmpty) { - reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + val message = + if (kafkaOffsetReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) } // Use the end partitions to calculate offset ranges to ignore partitions that have @@ -133,10 +133,21 @@ private[kafka010] class KafkaMicroBatchReadSupport( }.toSeq logDebug("TopicPartitions: " + topicPartitions.mkString(", ")) + val fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets + val untilOffsets = endPartitionOffsets + untilOffsets.foreach { case (tp, untilOffset) => + fromOffsets.get(tp).foreach { fromOffset => + if (untilOffset < fromOffset) { + reportDataLoss(s"Partition $tp's offset was changed from " + + s"$fromOffset to $untilOffset, some data may have been missed") + } + } + } + // Calculate offset ranges val offsetRanges = rangeCalculator.getRanges( - fromOffsets = startPartitionOffsets ++ newPartitionInitialOffsets, - untilOffsets = endPartitionOffsets, + fromOffsets = fromOffsets, + untilOffsets = untilOffsets, executorLocations = getSortedExecutorList()) // Reuse Kafka consumers only when all the offset ranges have distinct TopicPartitions, @@ -145,13 +156,13 @@ private[kafka010] class KafkaMicroBatchReadSupport( // Generate factories based on the offset ranges offsetRanges.map { range => - KafkaMicroBatchInputPartition( + KafkaBatchInputPartition( range, executorKafkaParams, pollTimeoutMs, failOnDataLoss, reuseKafkaConsumer) }.toArray } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - KafkaMicroBatchReaderFactory + override def createReaderFactory(): PartitionReaderFactory = { + KafkaBatchReaderFactory } override def deserializeOffset(json: String): Offset = { @@ -186,7 +197,7 @@ private[kafka010] class KafkaMicroBatchReadSupport( case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaOffsetReader.fetchEarliestOffsets()) case LatestOffsetRangeLimit => - KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets()) + KafkaSourceOffset(kafkaOffsetReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaOffsetReader.fetchSpecificOffsets(p, reportDataLoss) } @@ -218,10 +229,18 @@ private[kafka010] class KafkaMicroBatchReadSupport( until.map { case (tp, end) => tp -> sizes.get(tp).map { size => - val begin = from.get(tp).getOrElse(fromNew(tp)) + val begin = from.getOrElse(tp, fromNew(tp)) val prorate = limit * (size / total) // Don't completely starve small topicpartitions - val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + val prorateLong = (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + // need to be careful of integer overflow + // therefore added canary checks where to see if off variable could be overflowed + // refer to [https://issues.apache.org/jira/browse/SPARK-26718] + val off = if (prorateLong > Long.MaxValue - begin) { + Long.MaxValue + } else { + begin + prorateLong + } // Paranoia, make sure not to return an offset that's past end Math.min(end, off) }.getOrElse(end) @@ -280,7 +299,7 @@ private[kafka010] class KafkaMicroBatchReadSupport( if (content(0) == 'v') { val indexOfNewLine = content.indexOf("\n") if (indexOfNewLine > 0) { - val version = parseVersion(content.substring(0, indexOfNewLine), VERSION) + validateVersion(content.substring(0, indexOfNewLine), VERSION) KafkaSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) } else { throw new IllegalStateException( @@ -293,85 +312,3 @@ private[kafka010] class KafkaMicroBatchReadSupport( } } } - -/** A [[InputPartition]] for reading Kafka data in a micro-batch streaming query. */ -private[kafka010] case class KafkaMicroBatchInputPartition( - offsetRange: KafkaOffsetRange, - executorKafkaParams: ju.Map[String, Object], - pollTimeoutMs: Long, - failOnDataLoss: Boolean, - reuseKafkaConsumer: Boolean) extends InputPartition - -private[kafka010] object KafkaMicroBatchReaderFactory extends PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val p = partition.asInstanceOf[KafkaMicroBatchInputPartition] - KafkaMicroBatchPartitionReader(p.offsetRange, p.executorKafkaParams, p.pollTimeoutMs, - p.failOnDataLoss, p.reuseKafkaConsumer) - } -} - -/** A [[PartitionReader]] for reading Kafka data in a micro-batch streaming query. */ -private[kafka010] case class KafkaMicroBatchPartitionReader( - offsetRange: KafkaOffsetRange, - executorKafkaParams: ju.Map[String, Object], - pollTimeoutMs: Long, - failOnDataLoss: Boolean, - reuseKafkaConsumer: Boolean) extends PartitionReader[InternalRow] with Logging { - - private val consumer = KafkaDataConsumer.acquire( - offsetRange.topicPartition, executorKafkaParams, reuseKafkaConsumer) - - private val rangeToRead = resolveRange(offsetRange) - private val converter = new KafkaRecordToUnsafeRowConverter - - private var nextOffset = rangeToRead.fromOffset - private var nextRow: UnsafeRow = _ - - override def next(): Boolean = { - if (nextOffset < rangeToRead.untilOffset) { - val record = consumer.get(nextOffset, rangeToRead.untilOffset, pollTimeoutMs, failOnDataLoss) - if (record != null) { - nextRow = converter.toUnsafeRow(record) - nextOffset = record.offset + 1 - true - } else { - false - } - } else { - false - } - } - - override def get(): UnsafeRow = { - assert(nextRow != null) - nextRow - } - - override def close(): Unit = { - consumer.release() - } - - private def resolveRange(range: KafkaOffsetRange): KafkaOffsetRange = { - if (range.fromOffset < 0 || range.untilOffset < 0) { - // Late bind the offset range - val availableOffsetRange = consumer.getAvailableOffsetRange() - val fromOffset = if (range.fromOffset < 0) { - assert(range.fromOffset == KafkaOffsetRangeLimit.EARLIEST, - s"earliest offset ${range.fromOffset} does not equal ${KafkaOffsetRangeLimit.EARLIEST}") - availableOffsetRange.earliest - } else { - range.fromOffset - } - val untilOffset = if (range.untilOffset < 0) { - assert(range.untilOffset == KafkaOffsetRangeLimit.LATEST, - s"latest offset ${range.untilOffset} does not equal ${KafkaOffsetRangeLimit.LATEST}") - availableOffsetRange.latest - } else { - range.untilOffset - } - KafkaOffsetRange(range.topicPartition, fromOffset, untilOffset, None) - } else { - range - } - } -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala index fb209c724afba..c188b4c70ffbd 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculator.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition -import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** @@ -37,6 +37,8 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int * the read tasks of the skewed partitions to multiple Spark tasks. * The number of Spark tasks will be *approximately* `numPartitions`. It can be less or more * depending on rounding errors or Kafka partitions that didn't receive any new data. + * + * Empty ranges (`KafkaOffsetRange.size <= 0`) will be dropped. */ def getRanges( fromOffsets: PartitionOffsetMap, @@ -89,8 +91,9 @@ private[kafka010] class KafkaOffsetRangeCalculator(val minPartitions: Option[Int private[kafka010] object KafkaOffsetRangeCalculator { - def apply(options: DataSourceOptions): KafkaOffsetRangeCalculator = { - val optionalValue = Option(options.get("minPartitions").orElse(null)).map(_.toInt) + def apply(options: CaseInsensitiveStringMap): KafkaOffsetRangeCalculator = { + val optionalValue = Option(options.get(KafkaSourceProvider.MIN_PARTITIONS_OPTION_KEY)) + .map(_.toInt) new KafkaOffsetRangeCalculator(optionalValue) } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala index 82066697cb95a..ad608ecafe59f 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaOffsetReader.scala @@ -18,9 +18,10 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.concurrent.{Executors, ThreadFactory} +import java.util.concurrent.Executors import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration import scala.util.control.NonFatal @@ -45,22 +46,20 @@ import org.apache.spark.util.{ThreadUtils, UninterruptibleThread} */ private[kafka010] class KafkaOffsetReader( consumerStrategy: ConsumerStrategy, - driverKafkaParams: ju.Map[String, Object], + val driverKafkaParams: ju.Map[String, Object], readerOptions: Map[String, String], driverGroupIdPrefix: String) extends Logging { /** * Used to ensure execute fetch operations execute in an UninterruptibleThread */ - val kafkaReaderThread = Executors.newSingleThreadExecutor(new ThreadFactory { - override def newThread(r: Runnable): Thread = { - val t = new UninterruptibleThread("Kafka Offset Reader") { - override def run(): Unit = { - r.run() - } + val kafkaReaderThread = Executors.newSingleThreadExecutor((r: Runnable) => { + val t = new UninterruptibleThread("Kafka Offset Reader") { + override def run(): Unit = { + r.run() } - t.setDaemon(true) - t } + t.setDaemon(true) + t }) val execContext = ExecutionContext.fromExecutorService(kafkaReaderThread) @@ -81,17 +80,19 @@ private[kafka010] class KafkaOffsetReader( assert(Thread.currentThread().isInstanceOf[UninterruptibleThread]) if (_consumer == null) { val newKafkaParams = new ju.HashMap[String, Object](driverKafkaParams) - newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + if (driverKafkaParams.get(ConsumerConfig.GROUP_ID_CONFIG) == null) { + newKafkaParams.put(ConsumerConfig.GROUP_ID_CONFIG, nextGroupId()) + } _consumer = consumerStrategy.createConsumer(newKafkaParams) } _consumer } private val maxOffsetFetchAttempts = - readerOptions.getOrElse("fetchOffset.numRetries", "3").toInt + readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_NUM_RETRY, "3").toInt private val offsetFetchAttemptIntervalMs = - readerOptions.getOrElse("fetchOffset.retryIntervalMs", "1000").toLong + readerOptions.getOrElse(KafkaSourceProvider.FETCH_OFFSET_RETRY_INTERVAL_MS, "1000").toLong private def nextGroupId(): String = { groupId = driverGroupIdPrefix + "-" + nextId @@ -121,6 +122,34 @@ private[kafka010] class KafkaOffsetReader( partitions.asScala.toSet } + /** + * Fetch the partition offsets for the topic partitions that are indicated + * in the [[ConsumerStrategy]] and [[KafkaOffsetRangeLimit]]. + */ + def fetchPartitionOffsets(offsetRangeLimit: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { + def validateTopicPartitions(partitions: Set[TopicPartition], + partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { + assert(partitions == partitionOffsets.keySet, + "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + + "Use -1 for latest, -2 for earliest, if you don't care.\n" + + s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") + logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") + partitionOffsets + } + val partitions = fetchTopicPartitions() + // Obtain TopicPartition offsets with late binding support + offsetRangeLimit match { + case EarliestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.EARLIEST + }.toMap + case LatestOffsetRangeLimit => partitions.map { + case tp => tp -> KafkaOffsetRangeLimit.LATEST + }.toMap + case SpecificOffsetRangeLimit(partitionOffsets) => + validateTopicPartitions(partitions, partitionOffsets) + } + } + /** * Resolves the specific offsets based on Kafka seek positions. * This method resolves offset value -1 to the latest and -2 to the @@ -137,6 +166,12 @@ private[kafka010] class KafkaOffsetReader( // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) assert(partitions.asScala == partitionOffsets.keySet, "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + @@ -192,19 +227,82 @@ private[kafka010] class KafkaOffsetReader( /** * Fetch the latest offsets for the topic partitions that are indicated * in the [[ConsumerStrategy]]. + * + * Kafka may return earliest offsets when we are requesting latest offsets if `poll` is called + * right before `seekToEnd` (KAFKA-7703). As a workaround, we will call `position` right after + * `poll` to wait until the potential offset request triggered by `poll(0)` is done. + * + * In addition, to avoid other unknown issues, we also use the given `knownOffsets` to audit the + * latest offsets returned by Kafka. If we find some incorrect offsets (a latest offset is less + * than an offset in `knownOffsets`), we will retry at most `maxOffsetFetchAttempts` times. When + * a topic is recreated, the latest offsets may be less than offsets in `knownOffsets`. We cannot + * distinguish this with KAFKA-7703, so we just return whatever we get from Kafka after retrying. */ - def fetchLatestOffsets(): Map[TopicPartition, Long] = runUninterruptibly { + def fetchLatestOffsets( + knownOffsets: Option[PartitionOffsetMap]): PartitionOffsetMap = runUninterruptibly { withRetriesWithoutInterrupt { // Poll to get the latest assigned partitions consumer.poll(0) val partitions = consumer.assignment() + + // Call `position` to wait until the potential offset request triggered by `poll(0)` is + // done. This is a workaround for KAFKA-7703, which an async `seekToBeginning` triggered by + // `poll(0)` may reset offsets that should have been set by another request. + partitions.asScala.map(p => p -> consumer.position(p)).foreach(_ => {}) + consumer.pause(partitions) logDebug(s"Partitions assigned to consumer: $partitions. Seeking to the end.") - consumer.seekToEnd(partitions) - val partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap - logDebug(s"Got latest offsets for partition : $partitionOffsets") - partitionOffsets + if (knownOffsets.isEmpty) { + consumer.seekToEnd(partitions) + partitions.asScala.map(p => p -> consumer.position(p)).toMap + } else { + var partitionOffsets: PartitionOffsetMap = Map.empty + + /** + * Compare `knownOffsets` and `partitionOffsets`. Returns all partitions that have incorrect + * latest offset (offset in `knownOffsets` is great than the one in `partitionOffsets`). + */ + def findIncorrectOffsets(): Seq[(TopicPartition, Long, Long)] = { + var incorrectOffsets = ArrayBuffer[(TopicPartition, Long, Long)]() + partitionOffsets.foreach { case (tp, offset) => + knownOffsets.foreach(_.get(tp).foreach { knownOffset => + if (knownOffset > offset) { + val incorrectOffset = (tp, knownOffset, offset) + incorrectOffsets += incorrectOffset + } + }) + } + incorrectOffsets + } + + // Retry to fetch latest offsets when detecting incorrect offsets. We don't use + // `withRetriesWithoutInterrupt` to retry because: + // + // - `withRetriesWithoutInterrupt` will reset the consumer for each attempt but a fresh + // consumer has a much bigger chance to hit KAFKA-7703. + // - Avoid calling `consumer.poll(0)` which may cause KAFKA-7703. + var incorrectOffsets: Seq[(TopicPartition, Long, Long)] = Nil + var attempt = 0 + do { + consumer.seekToEnd(partitions) + partitionOffsets = partitions.asScala.map(p => p -> consumer.position(p)).toMap + attempt += 1 + + incorrectOffsets = findIncorrectOffsets() + if (incorrectOffsets.nonEmpty) { + logWarning("Found incorrect offsets in some partitions " + + s"(partition, previous offset, fetched offset): $incorrectOffsets") + if (attempt < maxOffsetFetchAttempts) { + logWarning("Retrying to fetch latest offsets because of incorrect offsets") + Thread.sleep(offsetFetchAttemptIntervalMs) + } + } + } while (incorrectOffsets.nonEmpty && attempt < maxOffsetFetchAttempts) + + logDebug(s"Got latest offsets for partition : $partitionOffsets") + partitionOffsets + } } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala index f35a143e00374..306ef10b775a9 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRecordToUnsafeRowConverter.scala @@ -30,13 +30,18 @@ private[kafka010] class KafkaRecordToUnsafeRowConverter { def toUnsafeRow(record: ConsumerRecord[Array[Byte], Array[Byte]]): UnsafeRow = { rowWriter.reset() + rowWriter.zeroOutNullBytes() if (record.key == null) { rowWriter.setNullAt(0) } else { rowWriter.write(0, record.key) } - rowWriter.write(1, record.value) + if (record.value == null) { + rowWriter.setNullAt(1) + } else { + rowWriter.write(1, record.value) + } rowWriter.write(2, UTF8String.fromString(record.topic)) rowWriter.write(3, record.partition) rowWriter.write(4, record.offset) diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala index e6f9d1259e43e..b2950cff98a0d 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaRelation.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql.kafka010 -import java.util.UUID - import org.apache.kafka.common.TopicPartition import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Row, SQLContext} import org.apache.spark.sql.catalyst.InternalRow @@ -39,17 +38,15 @@ private[kafka010] class KafkaRelation( failOnDataLoss: Boolean, startingOffsets: KafkaOffsetRangeLimit, endingOffsets: KafkaOffsetRangeLimit) - extends BaseRelation with TableScan with Logging { + extends BaseRelation with TableScan with Logging { assert(startingOffsets != LatestOffsetRangeLimit, "Starting offset not allowed to be set to latest offsets.") assert(endingOffsets != EarliestOffsetRangeLimit, "Ending offset not allowed to be set to earliest offsets.") private val pollTimeoutMs = sourceOptions.getOrElse( - "kafkaConsumer.pollTimeoutMs", - (sqlContext.sparkContext.conf.getTimeAsSeconds( - "spark.network.timeout", - "120s") * 1000L).toString + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, + (sqlContext.sparkContext.conf.get(NETWORK_TIMEOUT) * 1000L).toString ).toLong override def schema: StructType = KafkaOffsetReader.kafkaSchema @@ -58,7 +55,7 @@ private[kafka010] class KafkaRelation( // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = s"spark-kafka-relation-${UUID.randomUUID}" + val uniqueGroupId = KafkaSourceProvider.batchUniqueGroupId(sourceOptions) val kafkaOffsetReader = new KafkaOffsetReader( strategy, @@ -69,8 +66,8 @@ private[kafka010] class KafkaRelation( // Leverage the KafkaReader to obtain the relevant partition offsets val (fromPartitionOffsets, untilPartitionOffsets) = { try { - (getPartitionOffsets(kafkaOffsetReader, startingOffsets), - getPartitionOffsets(kafkaOffsetReader, endingOffsets)) + (kafkaOffsetReader.fetchPartitionOffsets(startingOffsets), + kafkaOffsetReader.fetchPartitionOffsets(endingOffsets)) } finally { kafkaOffsetReader.close() } @@ -89,11 +86,10 @@ private[kafka010] class KafkaRelation( // Calculate offset ranges val offsetRanges = untilPartitionOffsets.keySet.map { tp => - val fromOffset = fromPartitionOffsets.get(tp).getOrElse { - // This should not happen since topicPartitions contains all partitions not in - // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a from offset") - } + val fromOffset = fromPartitionOffsets.getOrElse(tp, + // This should not happen since topicPartitions contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset")) val untilOffset = untilPartitionOffsets(tp) KafkaSourceRDDOffsetRange(tp, fromOffset, untilOffset, None) }.toArray @@ -119,32 +115,6 @@ private[kafka010] class KafkaRelation( sqlContext.internalCreateDataFrame(rdd.setName("kafka"), schema).rdd } - private def getPartitionOffsets( - kafkaReader: KafkaOffsetReader, - kafkaOffsets: KafkaOffsetRangeLimit): Map[TopicPartition, Long] = { - def validateTopicPartitions(partitions: Set[TopicPartition], - partitionOffsets: Map[TopicPartition, Long]): Map[TopicPartition, Long] = { - assert(partitions == partitionOffsets.keySet, - "If startingOffsets contains specific offsets, you must specify all TopicPartitions.\n" + - "Use -1 for latest, -2 for earliest, if you don't care.\n" + - s"Specified: ${partitionOffsets.keySet} Assigned: ${partitions}") - logDebug(s"Partitions assigned to consumer: $partitions. Seeking to $partitionOffsets") - partitionOffsets - } - val partitions = kafkaReader.fetchTopicPartitions() - // Obtain TopicPartition offsets with late binding support - kafkaOffsets match { - case EarliestOffsetRangeLimit => partitions.map { - case tp => tp -> KafkaOffsetRangeLimit.EARLIEST - }.toMap - case LatestOffsetRangeLimit => partitions.map { - case tp => tp -> KafkaOffsetRangeLimit.LATEST - }.toMap - case SpecificOffsetRangeLimit(partitionOffsets) => - validateTopicPartitions(partitions, partitionOffsets) - } - } - override def toString: String = s"KafkaRelation(strategy=$strategy, start=$startingOffsets, end=$endingOffsets)" } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala index 66ec7e0cd084a..2858ff1162b58 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSource.scala @@ -22,10 +22,12 @@ import java.io._ import java.nio.charset.StandardCharsets import org.apache.commons.io.IOUtils +import org.apache.kafka.clients.consumer.ConsumerConfig import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Network.NETWORK_TIMEOUT import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -83,12 +85,12 @@ private[kafka010] class KafkaSource( private val sc = sqlContext.sparkContext private val pollTimeoutMs = sourceOptions.getOrElse( - "kafkaConsumer.pollTimeoutMs", - (sc.conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L).toString + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT, + (sc.conf.get(NETWORK_TIMEOUT) * 1000L).toString ).toLong private val maxOffsetsPerTrigger = - sourceOptions.get("maxOffsetsPerTrigger").map(_.toLong) + sourceOptions.get(KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER).map(_.toLong) /** * Lazily initialize `initialPartitionOffsets` to make sure that `KafkaConsumer.poll` is only @@ -114,7 +116,7 @@ private[kafka010] class KafkaSource( if (content(0) == 'v') { val indexOfNewLine = content.indexOf("\n") if (indexOfNewLine > 0) { - val version = parseVersion(content.substring(0, indexOfNewLine), VERSION) + validateVersion(content.substring(0, indexOfNewLine), VERSION) KafkaSourceOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) } else { throw new IllegalStateException( @@ -130,7 +132,7 @@ private[kafka010] class KafkaSource( metadataLog.get(0).getOrElse { val offsets = startingOffsets match { case EarliestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchEarliestOffsets()) - case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets()) + case LatestOffsetRangeLimit => KafkaSourceOffset(kafkaReader.fetchLatestOffsets(None)) case SpecificOffsetRangeLimit(p) => kafkaReader.fetchSpecificOffsets(p, reportDataLoss) } metadataLog.add(0, offsets) @@ -148,7 +150,8 @@ private[kafka010] class KafkaSource( // Make sure initialPartitionOffsets is initialized initialPartitionOffsets - val latest = kafkaReader.fetchLatestOffsets() + val latest = kafkaReader.fetchLatestOffsets( + currentPartitionOffsets.orElse(Some(initialPartitionOffsets))) val offsets = maxOffsetsPerTrigger match { case None => latest @@ -185,11 +188,19 @@ private[kafka010] class KafkaSource( until.map { case (tp, end) => tp -> sizes.get(tp).map { size => - val begin = from.get(tp).getOrElse(fromNew(tp)) + val begin = from.getOrElse(tp, fromNew(tp)) val prorate = limit * (size / total) logDebug(s"rateLimit $tp prorated amount is $prorate") // Don't completely starve small topicpartitions - val off = begin + (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + val prorateLong = (if (prorate < 1) Math.ceil(prorate) else Math.floor(prorate)).toLong + // need to be careful of integer overflow + // therefore added canary checks where to see if off variable could be overflowed + // refer to [https://issues.apache.org/jira/browse/SPARK-26718] + val off = if (prorateLong > Long.MaxValue - begin) { + Long.MaxValue + } else { + begin + prorateLong + } logDebug(s"rateLimit $tp new offset is $off") // Paranoia, make sure not to return an offset that's past end Math.min(end, off) @@ -241,7 +252,12 @@ private[kafka010] class KafkaSource( val deletedPartitions = fromPartitionOffsets.keySet.diff(untilPartitionOffsets.keySet) if (deletedPartitions.nonEmpty) { - reportDataLoss(s"$deletedPartitions are gone. Some data may have been missed") + val message = if (kafkaReader.driverKafkaParams.containsKey(ConsumerConfig.GROUP_ID_CONFIG)) { + s"$deletedPartitions are gone. ${KafkaSourceProvider.CUSTOM_GROUP_ID_ERROR_MESSAGE}" + } else { + s"$deletedPartitions are gone. Some data may have been missed." + } + reportDataLoss(message) } // Use the until partitions to calculate offset ranges to ignore partitions that have @@ -258,13 +274,11 @@ private[kafka010] class KafkaSource( // Calculate offset ranges val offsetRanges = topicPartitions.map { tp => - val fromOffset = fromPartitionOffsets.get(tp).getOrElse { - newPartitionOffsets.getOrElse(tp, { - // This should not happen since newPartitionOffsets contains all partitions not in - // fromPartitionOffsets - throw new IllegalStateException(s"$tp doesn't have a from offset") - }) - } + val fromOffset = fromPartitionOffsets.getOrElse(tp, newPartitionOffsets.getOrElse(tp, { + // This should not happen since newPartitionOffsets contains all partitions not in + // fromPartitionOffsets + throw new IllegalStateException(s"$tp doesn't have a from offset") + })) val untilOffset = untilPartitionOffsets(tp) val preferredLoc = if (numExecutors > 0) { // This allows cached KafkaConsumers in the executors to be re-used to read the same diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala index 8d41c0da2b133..90d70439c5329 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceOffset.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.common.TopicPartition import org.apache.spark.sql.execution.streaming.{Offset, SerializedOffset} -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, PartitionOffset} +import org.apache.spark.sql.sources.v2.reader.streaming.PartitionOffset /** * An [[Offset]] for the [[KafkaSource]]. This one tracks all partitions of subscribed topics and * their offsets. */ private[kafka010] -case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends OffsetV2 { +case class KafkaSourceOffset(partitionToOffsets: Map[TopicPartition, Long]) extends Offset { override val json = JsonUtils.partitionOffsets(partitionToOffsets) } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala index 28c9853bfea9c..a3ea918a8bc10 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaSourceProvider.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.kafka010 import java.{util => ju} -import java.util.{Locale, Optional, UUID} +import java.util.{Locale, UUID} import scala.collection.JavaConverters._ @@ -27,13 +27,20 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.serialization.{ByteArrayDeserializer, ByteArraySerializer} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SparkSession, SQLContext} +import org.apache.spark.kafka010.KafkaConfigUpdater +import org.apache.spark.sql.{AnalysisException, DataFrame, SaveMode, SQLContext} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.streaming.{Sink, Source} import org.apache.spark.sql.sources._ import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.sources.v2.reader.{Batch, Scan, ScanBuilder} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.OutputMode import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * The provider class for all Kafka readers and writers. It is designed such that it throws @@ -45,9 +52,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister with StreamSinkProvider with RelationProvider with CreatableRelationProvider - with StreamingWriteSupportProvider - with ContinuousReadSupportProvider - with MicroBatchReadSupportProvider + with TableProvider with Logging { import KafkaSourceProvider._ @@ -77,15 +82,10 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // Each running query should use its own group id. Otherwise, the query may be only assigned // partial data since Kafka will assign partitions to multiple consumers having the same group // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" + val uniqueGroupId = streamingUniqueGroupId(parameters, metadataPath) val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) @@ -106,85 +106,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister failOnDataLoss(caseInsensitiveParams)) } - /** - * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport]] to read - * batches of Kafka data in a micro-batch streaming query. - */ - override def createMicroBatchReadSupport( - metadataPath: String, - options: DataSourceOptions): KafkaMicroBatchReadSupport = { - - val parameters = options.asMap().asScala.toMap - validateStreamOptions(parameters) - // Each running query should use its own group id. Otherwise, the query may be only assigned - // partial data since Kafka will assign partitions to multiple consumers having the same group - // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap - - val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, - STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) - - val kafkaOffsetReader = new KafkaOffsetReader( - strategy(caseInsensitiveParams), - kafkaParamsForDriver(specifiedKafkaParams), - parameters, - driverGroupIdPrefix = s"$uniqueGroupId-driver") - - new KafkaMicroBatchReadSupport( - kafkaOffsetReader, - kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), - options, - metadataPath, - startingStreamOffsets, - failOnDataLoss(caseInsensitiveParams)) - } - - /** - * Creates a [[org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport]] to read - * Kafka data in a continuous streaming query. - */ - override def createContinuousReadSupport( - metadataPath: String, - options: DataSourceOptions): KafkaContinuousReadSupport = { - val parameters = options.asMap().asScala.toMap - validateStreamOptions(parameters) - // Each running query should use its own group id. Otherwise, the query may be only assigned - // partial data since Kafka will assign partitions to multiple consumers having the same group - // id. Hence, we should generate a unique id for each query. - val uniqueGroupId = s"spark-kafka-source-${UUID.randomUUID}-${metadataPath.hashCode}" - - val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap - - val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit(caseInsensitiveParams, - STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) - - val kafkaOffsetReader = new KafkaOffsetReader( - strategy(caseInsensitiveParams), - kafkaParamsForDriver(specifiedKafkaParams), - parameters, - driverGroupIdPrefix = s"$uniqueGroupId-driver") - - new KafkaContinuousReadSupport( - kafkaOffsetReader, - kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), - parameters, - metadataPath, - startingStreamOffsets, - failOnDataLoss(caseInsensitiveParams)) + override def getTable(options: CaseInsensitiveStringMap): KafkaTable = { + new KafkaTable } /** @@ -198,12 +121,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister parameters: Map[String, String]): BaseRelation = { validateBatchOptions(parameters) val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } - val specifiedKafkaParams = - parameters - .keySet - .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) - .map { k => k.drop(6).toString -> parameters(k) } - .toMap + val specifiedKafkaParams = convertToSpecifiedParams(parameters) val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) @@ -230,8 +148,7 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister outputMode: OutputMode): Sink = { val defaultTopic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) val specifiedKafkaParams = kafkaParamsForProducer(parameters) - new KafkaSink(sqlContext, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), defaultTopic) + new KafkaSink(sqlContext, specifiedKafkaParams, defaultTopic) } override def createRelation( @@ -248,8 +165,8 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } val topic = parameters.get(TOPIC_OPTION_KEY).map(_.trim) val specifiedKafkaParams = kafkaParamsForProducer(parameters) - KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, - new ju.HashMap[String, Object](specifiedKafkaParams.asJava), topic) + KafkaWriter.write(outerSQLContext.sparkSession, data.queryExecution, specifiedKafkaParams, + topic) /* This method is suppose to return a relation that reads the data that was written. * We cannot support this for Kafka. Therefore, in order to make things consistent, @@ -267,31 +184,13 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister } } - override def createStreamingWriteSupport( - queryId: String, - schema: StructType, - mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - import scala.collection.JavaConverters._ - - val spark = SparkSession.getActiveSession.get - val topic = Option(options.get(TOPIC_OPTION_KEY).orElse(null)).map(_.trim) - // We convert the options argument from V2 -> Java map -> scala mutable -> scala immutable. - val producerParams = kafkaParamsForProducer(options.asMap.asScala.toMap) - - KafkaWriter.validateQuery( - schema.toAttributes, new java.util.HashMap[String, Object](producerParams.asJava), topic) - - new KafkaStreamingWriteSupport(topic, producerParams, schema) - } - private def strategy(caseInsensitiveParams: Map[String, String]) = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { - case ("assign", value) => + case (ASSIGN, value) => AssignStrategy(JsonUtils.partitions(value)) - case ("subscribe", value) => + case (SUBSCRIBE, value) => SubscribeStrategy(value.split(",").map(_.trim()).filter(_.nonEmpty)) - case ("subscribepattern", value) => + case (SUBSCRIBE_PATTERN, value) => SubscribePatternStrategy(value.trim()) case _ => // Should never reach here as we are already matching on @@ -318,23 +217,23 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister + STRATEGY_OPTION_KEYS.mkString(", ") + ". See the docs for more details.") } - val strategy = caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { - case ("assign", value) => + caseInsensitiveParams.find(x => STRATEGY_OPTION_KEYS.contains(x._1)).get match { + case (ASSIGN, value) => if (!value.trim.startsWith("{")) { throw new IllegalArgumentException( "No topicpartitions to assign as specified value for option " + s"'assign' is '$value'") } - case ("subscribe", value) => + case (SUBSCRIBE, value) => val topics = value.split(",").map(_.trim).filter(_.nonEmpty) if (topics.isEmpty) { throw new IllegalArgumentException( "No topics to subscribe to as specified value for option " + s"'subscribe' is '$value'") } - case ("subscribepattern", value) => - val pattern = caseInsensitiveParams("subscribepattern").trim() + case (SUBSCRIBE_PATTERN, value) => + val pattern = caseInsensitiveParams(SUBSCRIBE_PATTERN).trim() if (pattern.isEmpty) { throw new IllegalArgumentException( "Pattern to subscribe is empty as specified value for option " + @@ -355,9 +254,11 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister // Validate user-specified Kafka options if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.GROUP_ID_CONFIG}")) { - throw new IllegalArgumentException( - s"Kafka option '${ConsumerConfig.GROUP_ID_CONFIG}' is not supported as " + - s"user-specified consumer groups are not used to track offsets.") + logWarning(CUSTOM_GROUP_ID_ERROR_MESSAGE) + if (caseInsensitiveParams.contains(GROUP_ID_PREFIX)) { + logWarning("Option 'groupIdPrefix' will be ignored as " + + s"option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set.") + } } if (caseInsensitiveParams.contains(s"kafka.${ConsumerConfig.AUTO_OFFSET_RESET_CONFIG}")) { @@ -448,18 +349,155 @@ private[kafka010] class KafkaSourceProvider extends DataSourceRegister validateGeneralOptions(caseInsensitiveParams) // Don't want to throw an error, but at least log a warning. - if (caseInsensitiveParams.get("maxoffsetspertrigger").isDefined) { + if (caseInsensitiveParams.get(MAX_OFFSET_PER_TRIGGER.toLowerCase(Locale.ROOT)).isDefined) { logWarning("maxOffsetsPerTrigger option ignored in batch queries") } } + + class KafkaTable extends Table with SupportsRead with SupportsWrite { + + override def name(): String = "KafkaTable" + + override def schema(): StructType = KafkaOffsetReader.kafkaSchema + + override def capabilities(): ju.Set[TableCapability] = { + // ACCEPT_ANY_SCHEMA is needed because of the following reasons: + // * Kafka writer validates the schema instead of the SQL analyzer (the schema is fixed) + // * Read schema differs from write schema (please see Kafka integration guide) + Set(BATCH_READ, BATCH_WRITE, MICRO_BATCH_READ, CONTINUOUS_READ, STREAMING_WRITE, + ACCEPT_ANY_SCHEMA).asJava + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = + () => new KafkaScan(options) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new WriteBuilder { + private var inputSchema: StructType = _ + private val topic = Option(options.get(TOPIC_OPTION_KEY)).map(_.trim) + private val producerParams = kafkaParamsForProducer(options.asScala.toMap) + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + this.inputSchema = schema + this + } + + override def buildForBatch(): BatchWrite = { + assert(inputSchema != null) + new KafkaBatchWrite(topic, producerParams, inputSchema) + } + + override def buildForStreaming(): StreamingWrite = { + assert(inputSchema != null) + new KafkaStreamingWrite(topic, producerParams, inputSchema) + } + } + } + } + + class KafkaScan(options: CaseInsensitiveStringMap) extends Scan { + + override def readSchema(): StructType = KafkaOffsetReader.kafkaSchema + + override def toBatch(): Batch = { + val caseInsensitiveOptions = CaseInsensitiveMap(options.asScala.toMap) + validateBatchOptions(caseInsensitiveOptions) + val specifiedKafkaParams = convertToSpecifiedParams(caseInsensitiveOptions) + + val startingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveOptions, STARTING_OFFSETS_OPTION_KEY, EarliestOffsetRangeLimit) + + val endingRelationOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveOptions, ENDING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + + new KafkaBatch( + strategy(caseInsensitiveOptions), + caseInsensitiveOptions, + specifiedKafkaParams, + failOnDataLoss(caseInsensitiveOptions), + startingRelationOffsets, + endingRelationOffsets) + } + + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { + val parameters = options.asScala.toMap + validateStreamOptions(parameters) + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = streamingUniqueGroupId(parameters, checkpointLocation) + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedKafkaParams = convertToSpecifiedParams(parameters) + + val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(parameters), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaMicroBatchStream( + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + options, + checkpointLocation, + startingStreamOffsets, + failOnDataLoss(caseInsensitiveParams)) + } + + override def toContinuousStream(checkpointLocation: String): ContinuousStream = { + val parameters = options.asScala.toMap + validateStreamOptions(parameters) + // Each running query should use its own group id. Otherwise, the query may be only assigned + // partial data since Kafka will assign partitions to multiple consumers having the same group + // id. Hence, we should generate a unique id for each query. + val uniqueGroupId = streamingUniqueGroupId(parameters, checkpointLocation) + + val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } + val specifiedKafkaParams = + parameters + .keySet + .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) + .map { k => k.drop(6).toString -> parameters(k) } + .toMap + + val startingStreamOffsets = KafkaSourceProvider.getKafkaOffsetRangeLimit( + caseInsensitiveParams, STARTING_OFFSETS_OPTION_KEY, LatestOffsetRangeLimit) + + val kafkaOffsetReader = new KafkaOffsetReader( + strategy(caseInsensitiveParams), + kafkaParamsForDriver(specifiedKafkaParams), + parameters, + driverGroupIdPrefix = s"$uniqueGroupId-driver") + + new KafkaContinuousStream( + kafkaOffsetReader, + kafkaParamsForExecutors(specifiedKafkaParams, uniqueGroupId), + options, + checkpointLocation, + startingStreamOffsets, + failOnDataLoss(caseInsensitiveParams)) + } + } } private[kafka010] object KafkaSourceProvider extends Logging { - private val STRATEGY_OPTION_KEYS = Set("subscribe", "subscribepattern", "assign") + private val ASSIGN = "assign" + private val SUBSCRIBE_PATTERN = "subscribepattern" + private val SUBSCRIBE = "subscribe" + private val STRATEGY_OPTION_KEYS = Set(SUBSCRIBE, SUBSCRIBE_PATTERN, ASSIGN) private[kafka010] val STARTING_OFFSETS_OPTION_KEY = "startingoffsets" private[kafka010] val ENDING_OFFSETS_OPTION_KEY = "endingoffsets" private val FAIL_ON_DATA_LOSS_OPTION_KEY = "failondataloss" - private val MIN_PARTITIONS_OPTION_KEY = "minpartitions" + private[kafka010] val MIN_PARTITIONS_OPTION_KEY = "minpartitions" + private[kafka010] val MAX_OFFSET_PER_TRIGGER = "maxOffsetsPerTrigger" + private[kafka010] val FETCH_OFFSET_NUM_RETRY = "fetchOffset.numRetries" + private[kafka010] val FETCH_OFFSET_RETRY_INTERVAL_MS = "fetchOffset.retryIntervalMs" + private[kafka010] val CONSUMER_POLL_TIMEOUT = "kafkaConsumer.pollTimeoutMs" + private val GROUP_ID_PREFIX = "groupidprefix" val TOPIC_OPTION_KEY = "topic" @@ -479,8 +517,18 @@ private[kafka010] object KafkaSourceProvider extends Logging { | source option "failOnDataLoss" to "false". """.stripMargin + val CUSTOM_GROUP_ID_ERROR_MESSAGE = + s"""Kafka option 'kafka.${ConsumerConfig.GROUP_ID_CONFIG}' has been set on this query, it is + | not recommended to set this option. This option is unsafe to use since multiple concurrent + | queries or sources using the same group id will interfere with each other as they are part + | of the same consumer group. Restarted queries may also suffer interference from the + | previous run having the same group id. The user should have only one query per group id, + | and/or set the option 'kafka.session.timeout.ms' to be very small so that the Kafka + | consumers from the previous query are marked dead by the Kafka group coordinator before the + | restarted query starts running. + """.stripMargin - + private val serClassName = classOf[ByteArraySerializer].getName private val deserClassName = classOf[ByteArrayDeserializer].getName def getKafkaOffsetRangeLimit( @@ -498,7 +546,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { } def kafkaParamsForDriver(specifiedKafkaParams: Map[String, String]): ju.Map[String, Object] = - ConfigUpdater("source", specifiedKafkaParams) + KafkaConfigUpdater("source", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -510,7 +558,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") // So that the driver does not pull too much data - .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, new java.lang.Integer(1)) + .set(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, java.lang.Integer.valueOf(1)) // If buffer config is not set, set it to reasonable value to work around // buffer issues (see KAFKA-3135) @@ -520,7 +568,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { def kafkaParamsForExecutors( specifiedKafkaParams: Map[String, String], uniqueGroupId: String): ju.Map[String, Object] = - ConfigUpdater("executor", specifiedKafkaParams) + KafkaConfigUpdater("executor", specifiedKafkaParams) .set(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, deserClassName) .set(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, deserClassName) @@ -528,7 +576,7 @@ private[kafka010] object KafkaSourceProvider extends Logging { .set(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none") // So that consumers in executors do not mess with any existing group id - .set(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") + .setIfUnset(ConsumerConfig.GROUP_ID_CONFIG, s"$uniqueGroupId-executor") // So that consumers in executors does not commit offsets unnecessarily .set(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -538,29 +586,30 @@ private[kafka010] object KafkaSourceProvider extends Logging { .setIfUnset(ConsumerConfig.RECEIVE_BUFFER_CONFIG, 65536: java.lang.Integer) .build() - /** Class to conveniently update Kafka config params, while logging the changes */ - private case class ConfigUpdater(module: String, kafkaParams: Map[String, String]) { - private val map = new ju.HashMap[String, Object](kafkaParams.asJava) - - def set(key: String, value: Object): this.type = { - map.put(key, value) - logDebug(s"$module: Set $key to $value, earlier value: ${kafkaParams.getOrElse(key, "")}") - this - } - - def setIfUnset(key: String, value: Object): ConfigUpdater = { - if (!map.containsKey(key)) { - map.put(key, value) - logDebug(s"$module: Set $key to $value") - } - this - } + /** + * Returns a unique batch consumer group (group.id), allowing the user to set the prefix of + * the consumer group + */ + private[kafka010] def batchUniqueGroupId(parameters: Map[String, String]): String = { + val groupIdPrefix = parameters + .getOrElse(GROUP_ID_PREFIX, "spark-kafka-relation") + s"${groupIdPrefix}-${UUID.randomUUID}" + } - def build(): ju.Map[String, Object] = map + /** + * Returns a unique streaming consumer group (group.id), allowing the user to set the prefix of + * the consumer group + */ + private def streamingUniqueGroupId( + parameters: Map[String, String], + metadataPath: String): String = { + val groupIdPrefix = parameters + .getOrElse(GROUP_ID_PREFIX, "spark-kafka-source") + s"${groupIdPrefix}-${UUID.randomUUID}-${metadataPath.hashCode}" } private[kafka010] def kafkaParamsForProducer( - parameters: Map[String, String]): Map[String, String] = { + parameters: Map[String, String]): ju.Map[String, Object] = { val caseInsensitiveParams = parameters.map { case (k, v) => (k.toLowerCase(Locale.ROOT), v) } if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( @@ -568,17 +617,25 @@ private[kafka010] object KafkaSourceProvider extends Logging { + "are serialized with ByteArraySerializer.") } - if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) - { + if (caseInsensitiveParams.contains(s"kafka.${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}")) { throw new IllegalArgumentException( s"Kafka option '${ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG}' is not supported as " + "value are serialized with ByteArraySerializer.") } + + val specifiedKafkaParams = convertToSpecifiedParams(parameters) + + KafkaConfigUpdater("executor", specifiedKafkaParams) + .set(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, serClassName) + .set(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, serClassName) + .build() + } + + private def convertToSpecifiedParams(parameters: Map[String, String]): Map[String, String] = { parameters .keySet .filter(_.toLowerCase(Locale.ROOT).startsWith("kafka.")) .map { k => k.drop(6).toString -> parameters(k) } - .toMap + (ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName, - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> classOf[ByteArraySerializer].getName) + .toMap } } diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala new file mode 100644 index 0000000000000..6dd1d2984a96e --- /dev/null +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWrite.scala @@ -0,0 +1,71 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.types.StructType + +/** + * A [[StreamingWrite]] for Kafka writing. Responsible for generating the writer factory. + * + * @param topic The topic this writer is responsible for. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ +private[kafka010] class KafkaStreamingWrite( + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) + extends StreamingWrite { + + validateQuery(schema.toAttributes, producerParams, topic) + + override def createStreamingWriterFactory(): KafkaStreamWriterFactory = + KafkaStreamWriterFactory(topic, producerParams, schema) + + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} + override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} +} + +/** + * A [[StreamingDataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to + * generate the per-task data writers. + * @param topic The topic that should be written to. If None, topic will be inferred from + * a `topic` field in the incoming data. + * @param producerParams Parameters for Kafka producers in each task. + * @param schema The schema of the input data. + */ +private case class KafkaStreamWriterFactory( + topic: Option[String], + producerParams: ju.Map[String, Object], + schema: StructType) + extends StreamingDataWriterFactory { + + override def createWriter( + partitionId: Int, + taskId: Long, + epochId: Long): DataWriter[InternalRow] = { + new KafkaDataWriter(topic, producerParams, schema.toAttributes) + } +} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala deleted file mode 100644 index 927c56d9ce829..0000000000000 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaStreamingWriteSupport.scala +++ /dev/null @@ -1,118 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.kafka010 - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.kafka010.KafkaWriter.validateQuery -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} -import org.apache.spark.sql.types.StructType - -/** - * Dummy commit message. The DataSourceV2 framework requires a commit message implementation but we - * don't need to really send one. - */ -case object KafkaWriterCommitMessage extends WriterCommitMessage - -/** - * A [[StreamingWriteSupport]] for Kafka writing. Responsible for generating the writer factory. - * - * @param topic The topic this writer is responsible for. If None, topic will be inferred from - * a `topic` field in the incoming data. - * @param producerParams Parameters for Kafka producers in each task. - * @param schema The schema of the input data. - */ -class KafkaStreamingWriteSupport( - topic: Option[String], producerParams: Map[String, String], schema: StructType) - extends StreamingWriteSupport { - - validateQuery(schema.toAttributes, producerParams.toMap[String, Object].asJava, topic) - - override def createStreamingWriterFactory(): KafkaStreamWriterFactory = - KafkaStreamWriterFactory(topic, producerParams, schema) - - override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} -} - -/** - * A [[StreamingDataWriterFactory]] for Kafka writing. Will be serialized and sent to executors to - * generate the per-task data writers. - * @param topic The topic that should be written to. If None, topic will be inferred from - * a `topic` field in the incoming data. - * @param producerParams Parameters for Kafka producers in each task. - * @param schema The schema of the input data. - */ -case class KafkaStreamWriterFactory( - topic: Option[String], producerParams: Map[String, String], schema: StructType) - extends StreamingDataWriterFactory { - - override def createWriter( - partitionId: Int, - taskId: Long, - epochId: Long): DataWriter[InternalRow] = { - new KafkaStreamDataWriter(topic, producerParams, schema.toAttributes) - } -} - -/** - * A [[DataWriter]] for Kafka writing. One data writer will be created in each partition to - * process incoming rows. - * - * @param targetTopic The topic that this data writer is targeting. If None, topic will be inferred - * from a `topic` field in the incoming data. - * @param producerParams Parameters to use for the Kafka producer. - * @param inputSchema The attributes in the input data. - */ -class KafkaStreamDataWriter( - targetTopic: Option[String], producerParams: Map[String, String], inputSchema: Seq[Attribute]) - extends KafkaRowWriter(inputSchema, targetTopic) with DataWriter[InternalRow] { - import scala.collection.JavaConverters._ - - private lazy val producer = CachedKafkaProducer.getOrCreate( - new java.util.HashMap[String, Object](producerParams.asJava)) - - def write(row: InternalRow): Unit = { - checkForErrors() - sendRow(row, producer) - } - - def commit(): WriterCommitMessage = { - // Send is asynchronous, but we can't commit until all rows are actually in Kafka. - // This requires flushing and then checking that no callbacks produced errors. - // We also check for errors before to fail as soon as possible - the check is cheap. - checkForErrors() - producer.flush() - checkForErrors() - KafkaWriterCommitMessage - } - - def abort(): Unit = {} - - def close(): Unit = { - checkForErrors() - if (producer != null) { - producer.flush() - checkForErrors() - CachedKafkaProducer.close(new java.util.HashMap[String, Object](producerParams.asJava)) - } - } -} diff --git a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala index fc09938a43a8c..e1a9191cc5a84 100644 --- a/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala +++ b/external/kafka-0-10-sql/src/main/scala/org/apache/spark/sql/kafka010/KafkaWriter.scala @@ -52,7 +52,7 @@ private[kafka010] object KafkaWriter extends Logging { s"'$TOPIC_ATTRIBUTE_NAME' attribute is present. Use the " + s"${KafkaSourceProvider.TOPIC_OPTION_KEY} option for setting a topic.") } else { - Literal(topic.get, StringType) + Literal.create(topic.get, StringType) } ).dataType match { case StringType => // good 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 43acd6a8d9473..ff19862c20cc6 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 @@ -16,9 +16,26 @@ */ package org.apache.spark.sql +import java.util.concurrent.TimeUnit + import org.apache.kafka.common.TopicPartition +import org.apache.spark.internal.config.ConfigBuilder + package object kafka010 { // scalastyle:ignore // ^^ scalastyle:ignore is for ignoring warnings about digits in package name type PartitionOffsetMap = Map[TopicPartition, Long] + + private[kafka010] val PRODUCER_CACHE_TIMEOUT = + ConfigBuilder("spark.kafka.producer.cache.timeout") + .doc("The expire time to remove the unused producers.") + .timeConf(TimeUnit.MILLISECONDS) + .createWithDefaultString("10m") + + private[kafka010] val CONSUMER_CACHE_CAPACITY = + 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).") + .intConf + .createWithDefault(64) } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala index 3f6fcf6b2e52c..65adbd6b9887c 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSinkSuite.scala @@ -22,9 +22,8 @@ import java.util.Locale import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.time.SpanSugar._ -import scala.collection.JavaConverters._ -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.streaming._ import org.apache.spark.sql.types.{BinaryType, DataType} @@ -227,39 +226,23 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { val topic = newTopic() testUtils.createTopic(topic) - /* No topic field or topic option */ - var writer: StreamingQuery = null - var ex: Exception = null - try { - writer = createKafkaWriter(input.toDF())( + val ex = intercept[AnalysisException] { + /* No topic field or topic option */ + createKafkaWriter(input.toDF())( withSelectExpr = "value as key", "value" ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() } assert(ex.getMessage .toLowerCase(Locale.ROOT) .contains("topic option required when no 'topic' attribute is present")) - try { + val ex2 = intercept[AnalysisException] { /* No value field */ - writer = createKafkaWriter(input.toDF())( + createKafkaWriter(input.toDF())( withSelectExpr = s"'$topic' as topic", "value as key" ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( "required attribute 'value' not found")) } @@ -278,53 +261,30 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { val topic = newTopic() testUtils.createTopic(topic) - var writer: StreamingQuery = null - var ex: Exception = null - try { + val ex = intercept[AnalysisException] { /* topic field wrong type */ - writer = createKafkaWriter(input.toDF())( + createKafkaWriter(input.toDF())( withSelectExpr = s"CAST('1' as INT) as topic", "value" ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() } assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("topic type must be a string")) - try { + val ex2 = intercept[AnalysisException] { /* value field wrong type */ - writer = createKafkaWriter(input.toDF())( + createKafkaWriter(input.toDF())( withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as value" ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( "value attribute type must be a string or binary")) - try { + val ex3 = intercept[AnalysisException] { /* key field wrong type */ - writer = createKafkaWriter(input.toDF())( + createKafkaWriter(input.toDF())( withSelectExpr = s"'$topic' as topic", "CAST(value as INT) as key", "value" ) - testUtils.sendMessages(inputTopic, Array("1", "2", "3", "4", "5")) - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - } finally { - writer.stop() } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + assert(ex3.getMessage.toLowerCase(Locale.ROOT).contains( "key attribute type must be a string or binary")) } @@ -369,35 +329,22 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("subscribe", inputTopic) .load() - var writer: StreamingQuery = null - var ex: Exception = null - try { - writer = createKafkaWriter( + + val ex = intercept[IllegalArgumentException] { + createKafkaWriter( input.toDF(), withOptions = Map("kafka.key.serializer" -> "foo"))() - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'key.serializer' is not supported")) - } finally { - writer.stop() } + assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'key.serializer' is not supported")) - try { - writer = createKafkaWriter( + val ex2 = intercept[IllegalArgumentException] { + createKafkaWriter( input.toDF(), withOptions = Map("kafka.value.serializer" -> "foo"))() - eventually(timeout(streamingTimeout)) { - assert(writer.exception.isDefined) - ex = writer.exception.get - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "kafka option 'value.serializer' is not supported")) - } finally { - writer.stop() } + assert(ex2.getMessage.toLowerCase(Locale.ROOT).contains( + "kafka option 'value.serializer' is not supported")) } test("generic - write big data with small producer buffer") { @@ -409,7 +356,7 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { */ val topic = newTopic() testUtils.createTopic(topic, 1) - val options = new java.util.HashMap[String, String] + val options = new java.util.HashMap[String, Object] options.put("bootstrap.servers", testUtils.brokerAddress) options.put("buffer.memory", "16384") // min buffer size options.put("block.on.buffer.full", "true") @@ -417,7 +364,7 @@ class KafkaContinuousSinkSuite extends KafkaContinuousTest { options.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[ByteArraySerializer].getName) val inputSchema = Seq(AttributeReference("value", BinaryType)()) val data = new Array[Byte](15000) // large value - val writeTask = new KafkaStreamDataWriter(Some(topic), options.asScala.toMap, inputSchema) + val writeTask = new KafkaDataWriter(Some(topic), options, inputSchema) try { val fieldTypes: Array[DataType] = Array(BinaryType) val converter = UnsafeProjection.create(fieldTypes) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala index af510219a6f6f..76c25980fc33f 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousSourceSuite.scala @@ -20,8 +20,8 @@ package org.apache.spark.sql.kafka010 import org.apache.kafka.clients.producer.ProducerRecord import org.apache.spark.sql.Dataset -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec +import org.apache.spark.sql.execution.streaming.ContinuousTrigger import org.apache.spark.sql.streaming.Trigger // Run tests in KafkaSourceSuiteBase in continuous execution mode. @@ -169,6 +169,10 @@ class KafkaContinuousSourceSuite extends KafkaSourceSuiteBase with KafkaContinuo } } } + + test("SPARK-27494: read kafka record containing null key/values.") { + testNullableKeyValue(ContinuousTrigger(100)) + } } class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { @@ -208,12 +212,12 @@ class KafkaContinuousSourceTopicDeletionSuite extends KafkaContinuousTest { eventually(timeout(streamingTimeout)) { assert( query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec - if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] - }.exists { config => + case scan: ContinuousScanExec + if scan.stream.isInstanceOf[KafkaContinuousStream] => + scan.stream.asInstanceOf[KafkaContinuousStream] + }.exists { stream => // Ensure the new topic is present and the old topic is gone. - config.knownPartitions.exists(_.topic == topic2) + stream.knownPartitions.exists(_.topic == topic2) }, s"query never reconfigured to new topic $topic2") } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala index fa6bdc20bd4f9..9ee8cbfa1bef4 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaContinuousTest.scala @@ -21,7 +21,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.SparkContext import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec import org.apache.spark.sql.execution.streaming.StreamExecution import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.streaming.Trigger @@ -30,7 +30,6 @@ import org.apache.spark.sql.test.TestSparkSession // Trait to configure StreamTest for kafka continuous execution tests. trait KafkaContinuousTest extends KafkaSourceTest { override val defaultTrigger = Trigger.Continuous(1000) - override val defaultUseV2Sink = true // We need more than the default local[2] to be able to schedule all partitions simultaneously. override protected def createSparkSession = new TestSparkSession( @@ -47,16 +46,16 @@ trait KafkaContinuousTest extends KafkaSourceTest { eventually(timeout(streamingTimeout)) { assert( query.lastExecution.executedPlan.collectFirst { - case scan: DataSourceV2ScanExec - if scan.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - scan.scanConfig.asInstanceOf[KafkaContinuousScanConfig] + case scan: ContinuousScanExec + if scan.stream.isInstanceOf[KafkaContinuousStream] => + scan.stream.asInstanceOf[KafkaContinuousStream] }.exists(_.knownPartitions.size == newCount), s"query never reconfigured to $newCount partitions") } } // Continuous processing tasks end asynchronously, so test that they actually end. - private val tasksEndedListener = new SparkListener() { + private class TasksEndedListener extends SparkListener { val activeTaskIdCount = new AtomicInteger(0) override def onTaskStart(start: SparkListenerTaskStart): Unit = { @@ -68,6 +67,8 @@ trait KafkaContinuousTest extends KafkaSourceTest { } } + private val tasksEndedListener = new TasksEndedListener() + override def beforeEach(): Unit = { super.beforeEach() spark.sparkContext.addSparkListener(tasksEndedListener) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala index 39c4e3fda1a4b..ba8340ea59c14 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaDontFailOnDataLossSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.kafka010 import java.util.Properties +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable @@ -27,6 +28,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkContext import org.apache.spark.sql.{DataFrame, Dataset, ForeachWriter} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.{SharedSQLContext, TestSparkSession} @@ -98,7 +100,7 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe testUtils.createTopic(topic, partitions = 1) testUtils.sendMessages(topic, (0 until 50).map(_.toString).toArray) - eventually(timeout(60.seconds)) { + eventually(timeout(1.minute)) { assert( testUtils.getEarliestOffsets(Set(topic)).head._2 > 0, "Kafka didn't delete records after 1 minute") @@ -131,9 +133,9 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe } } - test("failOnDataLoss=false should not return duplicated records: v1") { + test("failOnDataLoss=false should not return duplicated records: microbatch v1") { withSQLConf( - "spark.sql.streaming.disabledV2MicroBatchReaders" -> + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key -> classOf[KafkaSourceProvider].getCanonicalName) { verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = true) { (df, table) => val query = df.writeStream.format("memory").queryName(table).start() @@ -146,7 +148,7 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe } } - test("failOnDataLoss=false should not return duplicated records: v2") { + test("failOnDataLoss=false should not return duplicated records: microbatch v2") { verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = true) { (df, table) => val query = df.writeStream.format("memory").queryName(table).start() try { @@ -176,9 +178,19 @@ class KafkaDontFailOnDataLossSuite extends StreamTest with KafkaMissingOffsetsTe } } - test("failOnDataLoss=false should not return duplicated records: batch") { - verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = false) { (df, table) => - df.write.saveAsTable(table) + test("failOnDataLoss=false should not return duplicated records: batch v1") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "kafka") { + verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = false) { (df, table) => + df.write.saveAsTable(table) + } + } + } + + test("failOnDataLoss=false should not return duplicated records: batch v2") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + verifyMissingOffsetsDontCauseDuplicatedRecords(testStreamingQuery = false) { (df, table) => + df.write.saveAsTable(table) + } } } } @@ -221,13 +233,13 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM .as[(String, String)] val query = startStream(kafka.map(kv => kv._2.toInt)) - val testTime = 1.minutes - val startTime = System.currentTimeMillis() + val testTimeNs = TimeUnit.SECONDS.toNanos(20) + val startTimeNs = System.nanoTime() // Track the current existing topics val topics = mutable.ArrayBuffer[String]() // Track topics that have been deleted val deletedTopics = mutable.Set[String]() - while (System.currentTimeMillis() - testTime.toMillis < startTime) { + while (System.nanoTime() - startTimeNs < testTimeNs) { Random.nextInt(10) match { case 0 => // Create a new topic val topic = newTopic() @@ -252,7 +264,7 @@ class KafkaSourceStressForDontFailOnDataLossSuite extends StreamTest with KafkaM testUtils.createTopic(topic, partitions = 1, overwrite = true) logInfo(s"Create topic $topic") case 3 => - Thread.sleep(1000) + Thread.sleep(100) case _ => // Push random messages for (topic <- topics) { val size = Random.nextInt(10) diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala index 5ee76990b54f4..bb9b3696fe8f6 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaMicroBatchSourceSuite.scala @@ -28,7 +28,7 @@ import scala.collection.JavaConverters._ import scala.io.Source import scala.util.Random -import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord, RecordMetadata} +import org.apache.kafka.clients.producer.{ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ @@ -41,10 +41,11 @@ import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.kafka010.KafkaSourceProvider._ -import org.apache.spark.sql.sources.v2.DataSourceOptions -import org.apache.spark.sql.streaming.{ProcessingTime, StreamTest} +import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream +import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.streaming.util.StreamManualClock import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.util.CaseInsensitiveStringMap abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with KafkaTest { @@ -94,7 +95,7 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with Kaf message: String = "", topicAction: (String, Option[Int]) => Unit = (_, _) => {}) extends AddData { - override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { + override def addData(query: Option[StreamExecution]): (SparkDataStream, Offset) = { query match { // Make sure no Spark job is running when deleting a topic case Some(m: MicroBatchExecution) => m.processAllAvailable() @@ -114,18 +115,13 @@ abstract class KafkaSourceTest extends StreamTest with SharedSQLContext with Kaf query.nonEmpty, "Cannot add data when there is no query for finding the active kafka source") - val sources: Seq[BaseStreamingSource] = { + val sources: Seq[SparkDataStream] = { query.get.logicalPlan.collect { case StreamingExecutionRelation(source: KafkaSource, _) => source - case StreamingExecutionRelation(source: KafkaMicroBatchReadSupport, _) => source - } ++ (query.get.lastExecution match { - case null => Seq() - case e => e.logical.collect { - case r: StreamingDataSourceV2Relation - if r.readSupport.isInstanceOf[KafkaContinuousReadSupport] => - r.readSupport.asInstanceOf[KafkaContinuousReadSupport] - } - }) + case r: StreamingDataSourceV2Relation if r.stream.isInstanceOf[KafkaMicroBatchStream] || + r.stream.isInstanceOf[KafkaContinuousStream] => + r.stream + } }.distinct if (sources.isEmpty) { @@ -201,6 +197,41 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { StopStream) } + test("SPARK-26718 Rate limit set to Long.Max should not overflow integer " + + "during end offset calculation") { + val topic = newTopic() + testUtils.createTopic(topic, partitions = 1) + // fill in 5 messages to trigger potential integer overflow + testUtils.sendMessages(topic, (0 until 5).map(_.toString).toArray, Some(0)) + + val partitionOffsets = Map( + new TopicPartition(topic, 0) -> 5L + ) + val startingOffsets = JsonUtils.partitionOffsets(partitionOffsets) + + val kafka = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + // use latest to force begin to be 5 + .option("startingOffsets", startingOffsets) + // use Long.Max to try to trigger overflow + .option("maxOffsetsPerTrigger", Long.MaxValue) + .option("subscribe", topic) + .option("kafka.metadata.max.age.ms", "1") + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + val mapped: org.apache.spark.sql.Dataset[_] = kafka.map(kv => kv._2.toInt) + + testStream(mapped)( + makeSureGetOffsetCalled, + AddKafkaData(Set(topic), 30, 31, 32, 33, 34), + CheckAnswer(30, 31, 32, 33, 34), + StopStream + ) + } + test("maxOffsetsPerTrigger") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) @@ -236,7 +267,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // 1 from smallest, 1 from middle, 8 from biggest CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107), @@ -247,7 +278,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { 11, 108, 109, 110, 111, 112, 113, 114, 115, 116 ), StopStream, - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // smallest now empty, 1 more from middle, 9 more from biggest CheckAnswer(1, 10, 100, 101, 102, 103, 104, 105, 106, 107, @@ -282,7 +313,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { val mapped = kafka.map(kv => kv._2.toInt + 1) testStream(mapped)( - StartStream(trigger = ProcessingTime(1)), + StartStream(trigger = Trigger.ProcessingTime(1)), makeSureGetOffsetCalled, AddKafkaData(Set(topic), 1, 2, 3), CheckAnswer(2, 3, 4), @@ -329,6 +360,54 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } + test("subscribe topic by pattern with topic recreation between batches") { + val topicPrefix = newTopic() + val topic = topicPrefix + "-good" + val topic2 = topicPrefix + "-bad" + testUtils.createTopic(topic, partitions = 1) + testUtils.sendMessages(topic, Array("1", "3")) + testUtils.createTopic(topic2, partitions = 1) + testUtils.sendMessages(topic2, Array("2", "4")) + + val reader = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.metadata.max.age.ms", "1") + .option("kafka.default.api.timeout.ms", "3000") + .option("startingOffsets", "earliest") + .option("subscribePattern", s"$topicPrefix-.*") + + val ds = reader.load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + .map(kv => kv._2.toInt) + + testStream(ds)( + StartStream(), + AssertOnQuery { q => + q.processAllAvailable() + true + }, + CheckAnswer(1, 2, 3, 4), + // Restart the stream in this test to make the test stable. When recreating a topic when a + // consumer is alive, it may not be able to see the recreated topic even if a fresh consumer + // has seen it. + StopStream, + // Recreate `topic2` and wait until it's available + WithOffsetSync(new TopicPartition(topic2, 0), expectedOffset = 1) { () => + testUtils.deleteTopic(topic2) + testUtils.createTopic(topic2) + testUtils.sendMessages(topic2, Array("6")) + }, + StartStream(), + ExpectFailure[IllegalStateException](e => { + // The offset of `topic2` should be changed from 2 to 1 + assert(e.getMessage.contains("was changed from 2 to 1")) + }) + ) + } + test("ensure that initial offset are written with an extra byte in the beginning (SPARK-19517)") { withTempDir { metadataPath => val topic = "kafka-initial-offset-current" @@ -557,7 +636,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { } testStream(kafka)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // 5 from smaller topic, 5 from bigger one CheckLastBatch((0 to 4) ++ (100 to 104): _*), @@ -570,7 +649,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // smaller topic empty, 5 from bigger one CheckLastBatch(110 to 114: _*), StopStream, - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, // smallest now empty, 5 from bigger one CheckLastBatch(115 to 119: _*), @@ -581,6 +660,56 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { ) } + test("allow group.id prefix") { + testGroupId("groupIdPrefix", (expected, actual) => { + assert(actual.exists(_.startsWith(expected)) && !actual.exists(_ === expected), + "Valid consumer groups don't contain the expected group id - " + + s"Valid consumer groups: $actual / expected group id: $expected") + }) + } + + test("allow group.id override") { + testGroupId("kafka.group.id", (expected, actual) => { + assert(actual.exists(_ === expected), "Valid consumer groups don't " + + s"contain the expected group id - Valid consumer groups: $actual / " + + s"expected group id: $expected") + }) + } + + private def testGroupId(groupIdKey: String, validateGroupId: (String, Iterable[String]) => Unit) { + // Tests code path KafkaSourceProvider.{sourceSchema(.), createSource(.)} + // as well as KafkaOffsetReader.createConsumer(.) + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2)) + + val customGroupId = "id-" + Random.nextInt() + val dsKafka = spark + .readStream + .format("kafka") + .option(groupIdKey, customGroupId) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("subscribe", topic) + .option("startingOffsets", "earliest") + .load() + .selectExpr("CAST(value AS STRING)") + .as[String] + .map(_.toInt) + + testStream(dsKafka)( + makeSureGetOffsetCalled, + CheckAnswer(1 to 30: _*), + Execute { _ => + val consumerGroups = testUtils.listConsumerGroups() + val validGroups = consumerGroups.valid().get() + val validGroupsId = validGroups.asScala.map(_.groupId()) + validateGroupId(customGroupId, validGroupsId) + } + ) + } + test("ensure stream-stream self-join generates only one offset in log and correct metrics") { val topic = newTopic() testUtils.createTopic(topic, partitions = 2) @@ -679,7 +808,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // The message values are the same as their offsets to make the test easy to follow testUtils.withTranscationalProducer { producer => testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, CheckAnswer(), WithOffsetSync(topicPartition, expectedOffset = 5) { () => @@ -802,7 +931,7 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { // The message values are the same as their offsets to make the test easy to follow testUtils.withTranscationalProducer { producer => testStream(mapped)( - StartStream(ProcessingTime(100), clock), + StartStream(Trigger.ProcessingTime(100), clock), waitUntilBatchProcessed, CheckNewAnswer(), WithOffsetSync(topicPartition, expectedOffset = 5) { () => @@ -926,6 +1055,10 @@ abstract class KafkaMicroBatchSourceSuiteBase extends KafkaSourceSuiteBase { q.stop() } } + + test("SPARK-27494: read kafka record containing null key/values.") { + testNullableKeyValue(Trigger.ProcessingTime(100)) + } } @@ -933,7 +1066,7 @@ class KafkaMicroBatchV1SourceSuite extends KafkaMicroBatchSourceSuiteBase { override def beforeAll(): Unit = { super.beforeAll() spark.conf.set( - "spark.sql.streaming.disabledV2MicroBatchReaders", + SQLConf.DISABLED_V2_STREAMING_MICROBATCH_READERS.key, classOf[KafkaSourceProvider].getCanonicalName) } @@ -977,9 +1110,10 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { testStream(kafka)( makeSureGetOffsetCalled, AssertOnQuery { query => - query.logicalPlan.collect { - case StreamingExecutionRelation(_: KafkaMicroBatchReadSupport, _) => true - }.nonEmpty + query.logicalPlan.find { + case r: StreamingDataSourceV2Relation => r.stream.isInstanceOf[KafkaMicroBatchStream] + case _ => false + }.isDefined } ) } @@ -1003,13 +1137,12 @@ class KafkaMicroBatchV2SourceSuite extends KafkaMicroBatchSourceSuiteBase { "kafka.bootstrap.servers" -> testUtils.brokerAddress, "subscribe" -> topic ) ++ Option(minPartitions).map { p => "minPartitions" -> p} - val readSupport = provider.createMicroBatchReadSupport( - dir.getAbsolutePath, new DataSourceOptions(options.asJava)) - val config = readSupport.newScanConfigBuilder( + val dsOptions = new CaseInsensitiveStringMap(options.asJava) + val table = provider.getTable(dsOptions) + val stream = table.newScanBuilder(dsOptions).build().toMicroBatchStream(dir.getAbsolutePath) + val inputPartitions = stream.planInputPartitions( KafkaSourceOffset(Map(tp -> 0L)), - KafkaSourceOffset(Map(tp -> 100L))).build() - val inputPartitions = readSupport.planInputPartitions(config) - .map(_.asInstanceOf[KafkaMicroBatchInputPartition]) + KafkaSourceOffset(Map(tp -> 100L))).map(_.asInstanceOf[KafkaBatchInputPartition]) withClue(s"minPartitions = $minPartitions generated factories $inputPartitions\n\t") { assert(inputPartitions.size == numPartitionsGenerated) inputPartitions.foreach { f => assert(f.reuseKafkaConsumer == reusesConsumers) } @@ -1185,7 +1318,6 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { assert(ex.getMessage.toLowerCase(Locale.ROOT).contains("not supported")) } - testUnsupportedConfig("kafka.group.id") testUnsupportedConfig("kafka.auto.offset.reset") testUnsupportedConfig("kafka.enable.auto.commit") testUnsupportedConfig("kafka.interceptor.classes") @@ -1326,7 +1458,7 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { val reader = spark .readStream .format("kafka") - .option("startingOffsets", s"latest") + .option("startingOffsets", "latest") .option("kafka.bootstrap.servers", testUtils.brokerAddress) .option("kafka.metadata.max.age.ms", "1") .option("failOnDataLoss", failOnDataLoss.toString) @@ -1398,6 +1530,60 @@ abstract class KafkaSourceSuiteBase extends KafkaSourceTest { CheckAnswer(2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, 16, 17) ) } + + protected def testNullableKeyValue(trigger: Trigger): Unit = { + val table = "kafka_null_key_value_source_test" + withTable(table) { + val topic = newTopic() + testUtils.createTopic(topic) + testUtils.withTranscationalProducer { producer => + val df = spark + .readStream + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.isolation.level", "read_committed") + .option("startingOffsets", "earliest") + .option("subscribe", topic) + .load() + .selectExpr("CAST(key AS STRING)", "CAST(value AS STRING)") + .as[(String, String)] + + val q = df + .writeStream + .format("memory") + .queryName(table) + .trigger(trigger) + .start() + try { + var idx = 0 + producer.beginTransaction() + val expected1 = Seq.tabulate(5) { _ => + producer.send(new ProducerRecord[String, String](topic, null, null)).get() + (null, null) + }.asInstanceOf[Seq[(String, String)]] + + val expected2 = Seq.tabulate(5) { _ => + idx += 1 + producer.send(new ProducerRecord[String, String](topic, idx.toString, null)).get() + (idx.toString, null) + }.asInstanceOf[Seq[(String, String)]] + + val expected3 = Seq.tabulate(5) { _ => + idx += 1 + producer.send(new ProducerRecord[String, String](topic, null, idx.toString)).get() + (null, idx.toString) + }.asInstanceOf[Seq[(String, String)]] + + producer.commitTransaction() + eventually(timeout(streamingTimeout)) { + checkAnswer(spark.table(table), (expected1 ++ expected2 ++ expected3).toDF()) + } + } finally { + q.stop() + } + } + } + } } object KafkaSourceSuite { diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala index 2ccf3e291bea7..7ffdaab3e74fb 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaOffsetRangeCalculatorSuite.scala @@ -22,13 +22,13 @@ import scala.collection.JavaConverters._ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.util.CaseInsensitiveStringMap class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { def testWithMinPartitions(name: String, minPartition: Int) (f: KafkaOffsetRangeCalculator => Unit): Unit = { - val options = new DataSourceOptions(Map("minPartitions" -> minPartition.toString).asJava) + val options = new CaseInsensitiveStringMap(Map("minPartitions" -> minPartition.toString).asJava) test(s"with minPartition = $minPartition: $name") { f(KafkaOffsetRangeCalculator(options)) } @@ -36,7 +36,7 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { test("with no minPartition: N TopicPartitions to N offset ranges") { - val calc = KafkaOffsetRangeCalculator(DataSourceOptions.empty()) + val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty()) assert( calc.getRanges( fromOffsets = Map(tp1 -> 1), @@ -64,7 +64,7 @@ class KafkaOffsetRangeCalculatorSuite extends SparkFunSuite { } test("with no minPartition: empty ranges ignored") { - val calc = KafkaOffsetRangeCalculator(DataSourceOptions.empty()) + val calc = KafkaOffsetRangeCalculator(CaseInsensitiveStringMap.empty()) assert( calc.getRanges( fromOffsets = Map(tp1 -> 1, tp2 -> 1), diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala index 8cfca56433f5d..84d1ab6f75c3e 100644 --- a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaRelationSuite.scala @@ -20,26 +20,35 @@ package org.apache.spark.sql.kafka010 import java.util.Locale import java.util.concurrent.atomic.AtomicInteger +import scala.collection.JavaConverters._ +import scala.util.Random + import org.apache.kafka.clients.producer.ProducerRecord import org.apache.kafka.common.TopicPartition +import org.apache.spark.SparkConf import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.BaseRelation import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils -class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest { +abstract class KafkaRelationSuiteBase extends QueryTest with SharedSQLContext with KafkaTest { import testImplicits._ private val topicId = new AtomicInteger(0) - private var testUtils: KafkaTestUtils = _ + protected var testUtils: KafkaTestUtils = _ - private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "kafka") - private def assignString(topic: String, partitions: Iterable[Int]): String = { - JsonUtils.partitions(partitions.map(p => new TopicPartition(topic, p))) - } + protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" override def beforeAll(): Unit = { super.beforeAll() @@ -58,7 +67,7 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest } } - private def createDF( + protected def createDF( topic: String, withOptions: Map[String, String] = Map.empty[String, String], brokerAddress: Option[String] = None) = { @@ -74,7 +83,6 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest df.load().selectExpr("CAST(value AS STRING)") } - test("explicit earliest to latest offsets") { val topic = newTopic() testUtils.createTopic(topic, partitions = 3) @@ -196,7 +204,7 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest .read .format("kafka") options.foreach { case (k, v) => reader.option(k, v) } - reader.load() + reader.load().collect() } expectedMsgs.foreach { m => assert(ex.getMessage.toLowerCase(Locale.ROOT).contains(m.toLowerCase(Locale.ROOT))) @@ -239,6 +247,40 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest testBadOptions("subscribePattern" -> "")("pattern to subscribe is empty") } + test("allow group.id prefix") { + testGroupId("groupIdPrefix", (expected, actual) => { + assert(actual.exists(_.startsWith(expected)) && !actual.exists(_ === expected), + "Valid consumer groups don't contain the expected group id - " + + s"Valid consumer groups: $actual / expected group id: $expected") + }) + } + + test("allow group.id override") { + testGroupId("kafka.group.id", (expected, actual) => { + assert(actual.exists(_ === expected), "Valid consumer groups don't " + + s"contain the expected group id - Valid consumer groups: $actual / " + + s"expected group id: $expected") + }) + } + + private def testGroupId(groupIdKey: String, validateGroupId: (String, Iterable[String]) => Unit) { + // Tests code path KafkaSourceProvider.createRelation(.) + val topic = newTopic() + testUtils.createTopic(topic, partitions = 3) + testUtils.sendMessages(topic, (1 to 10).map(_.toString).toArray, Some(0)) + testUtils.sendMessages(topic, (11 to 20).map(_.toString).toArray, Some(1)) + testUtils.sendMessages(topic, (21 to 30).map(_.toString).toArray, Some(2)) + + val customGroupId = "id-" + Random.nextInt() + val df = createDF(topic, withOptions = Map(groupIdKey -> customGroupId)) + checkAnswer(df, (1 to 30).map(_.toString).toDF()) + + val consumerGroups = testUtils.listConsumerGroups() + val validGroups = consumerGroups.valid().get() + val validGroupsId = validGroups.asScala.map(_.groupId()) + validateGroupId(customGroupId, validGroupsId) + } + test("read Kafka transactional messages: read_committed") { val topic = newTopic() testUtils.createTopic(topic) @@ -338,3 +380,33 @@ class KafkaRelationSuite extends QueryTest with SharedSQLContext with KafkaTest } } } + +class KafkaRelationSuiteV1 extends KafkaRelationSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "kafka") + + test("V1 Source is used when set through SQLConf") { + val topic = newTopic() + val df = createDF(topic) + assert(df.logicalPlan.collect { + case LogicalRelation(_, _, _, _) => true + }.nonEmpty) + } +} + +class KafkaRelationSuiteV2 extends KafkaRelationSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + + test("V2 Source is used when set through SQLConf") { + val topic = newTopic() + val df = createDF(topic) + assert(df.logicalPlan.collect { + case DataSourceV2Relation(_, _, _) => true + }.nonEmpty) + } +} 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 d46c4139011da..032adc70466fa 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 @@ -24,22 +24,19 @@ import org.apache.kafka.clients.producer.ProducerConfig import org.apache.kafka.common.serialization.ByteArraySerializer import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException, TestUtils} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, SpecificInternalRow, UnsafeProjection} import org.apache.spark.sql.execution.streaming.MemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{BinaryType, DataType} -class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { - import testImplicits._ - +abstract class KafkaSinkSuiteBase extends QueryTest with SharedSQLContext with KafkaTest { protected var testUtils: KafkaTestUtils = _ - override val streamingTimeout = 30.seconds - override def beforeAll(): Unit = { super.beforeAll() testUtils = new KafkaTestUtils( @@ -58,74 +55,25 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { } } - test("batch - write to kafka") { - val topic = newTopic() - testUtils.createTopic(topic) - val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .save() - checkAnswer( - createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), - Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) - } - - test("batch - null topic field value, and no topic option") { - val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") - val ex = intercept[SparkException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .save() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - "null topic present in the data")) - } - - test("batch - unsupported save modes") { - val topic = newTopic() - testUtils.createTopic(topic) - val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") - - // Test bad save mode Ignore - var ex = intercept[AnalysisException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .mode(SaveMode.Ignore) - .save() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - s"save mode ignore not allowed for kafka")) - - // Test bad save mode Overwrite - ex = intercept[AnalysisException] { - df.write - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .mode(SaveMode.Overwrite) - .save() - } - assert(ex.getMessage.toLowerCase(Locale.ROOT).contains( - s"save mode overwrite not allowed for kafka")) - } + private val topicId = new AtomicInteger(0) - test("SPARK-20496: batch - enforce analyzed plans") { - val inputEvents = - spark.range(1, 1000) - .select(to_json(struct("*")) as 'value) + protected def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - val topic = newTopic() - testUtils.createTopic(topic) - // used to throw UnresolvedException - inputEvents.write + protected def createKafkaReader(topic: String): DataFrame = { + spark.read .format("kafka") .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("topic", topic) - .save() + .option("startingOffsets", "earliest") + .option("endingOffsets", "latest") + .option("subscribe", topic) + .load() } +} + +class KafkaSinkStreamingSuite extends KafkaSinkSuiteBase with StreamTest { + import testImplicits._ + + override val streamingTimeout = 30.seconds test("streaming - write to kafka with topic field") { val input = MemoryStream[String] @@ -232,6 +180,27 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { } } + test("streaming - sink progress is produced") { + /* ensure sink progress is correctly produced. */ + val input = MemoryStream[String] + val topic = newTopic() + testUtils.createTopic(topic) + + val writer = createKafkaWriter( + input.toDF(), + withTopic = Some(topic), + withOutputMode = Some(OutputMode.Update()))() + + try { + input.addData("1", "2", "3") + failAfter(streamingTimeout) { + writer.processAllAvailable() + } + assert(writer.lastProgress.sink.numOutputRows == 3L) + } finally { + writer.stop() + } + } test("streaming - write data with bad schema") { val input = MemoryStream[String] @@ -367,6 +336,114 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { "kafka option 'value.serializer' is not supported")) } + private def createKafkaWriter( + input: DataFrame, + withTopic: Option[String] = None, + withOutputMode: Option[OutputMode] = None, + withOptions: Map[String, String] = Map[String, String]()) + (withSelectExpr: String*): StreamingQuery = { + var stream: DataStreamWriter[Row] = null + withTempDir { checkpointDir => + var df = input.toDF() + if (withSelectExpr.length > 0) { + df = df.selectExpr(withSelectExpr: _*) + } + stream = df.writeStream + .format("kafka") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("kafka.max.block.ms", "5000") + .queryName("kafkaStream") + withTopic.foreach(stream.option("topic", _)) + withOutputMode.foreach(stream.outputMode(_)) + withOptions.foreach(opt => stream.option(opt._1, opt._2)) + } + stream.start() + } +} + +abstract class KafkaSinkBatchSuiteBase extends KafkaSinkSuiteBase { + import testImplicits._ + + test("batch - write to kafka") { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq("1", "2", "3", "4", "5").map(v => (topic, v)).toDF("topic", "value") + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + checkAnswer( + createKafkaReader(topic).selectExpr("CAST(value as STRING) value"), + Row("1") :: Row("2") :: Row("3") :: Row("4") :: Row("5") :: Nil) + } + + test("batch - null topic field value, and no topic option") { + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + val ex = intercept[SparkException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .save() + } + TestUtils.assertExceptionMsg(ex, "null topic present in the data") + } + + protected def testUnsupportedSaveModes(msg: (SaveMode) => String) { + val topic = newTopic() + testUtils.createTopic(topic) + val df = Seq[(String, String)](null.asInstanceOf[String] -> "1").toDF("topic", "value") + + Seq(SaveMode.Ignore, SaveMode.Overwrite).foreach { mode => + val ex = intercept[AnalysisException] { + df.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .mode(mode) + .save() + } + TestUtils.assertExceptionMsg(ex, msg(mode)) + } + } + + test("SPARK-20496: batch - enforce analyzed plans") { + val inputEvents = + spark.range(1, 1000) + .select(to_json(struct("*")) as 'value) + + val topic = newTopic() + testUtils.createTopic(topic) + // used to throw UnresolvedException + inputEvents.write + .format("kafka") + .option("kafka.bootstrap.servers", testUtils.brokerAddress) + .option("topic", topic) + .save() + } +} + +class KafkaSinkBatchSuiteV1 extends KafkaSinkBatchSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "kafka") + + test("batch - unsupported save modes") { + testUnsupportedSaveModes((mode) => s"Save mode ${mode.name} not allowed for Kafka") + } +} + +class KafkaSinkBatchSuiteV2 extends KafkaSinkBatchSuiteBase { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "") + + test("batch - unsupported save modes") { + testUnsupportedSaveModes((mode) => s"cannot be written with ${mode.name} mode") + } + test("generic - write big data with small producer buffer") { /* This test ensures that we understand the semantics of Kafka when * is comes to blocking on a call to send when the send buffer is full. @@ -396,43 +473,4 @@ class KafkaSinkSuite extends StreamTest with SharedSQLContext with KafkaTest { writeTask.close() } } - - private val topicId = new AtomicInteger(0) - - private def newTopic(): String = s"topic-${topicId.getAndIncrement()}" - - private def createKafkaReader(topic: String): DataFrame = { - spark.read - .format("kafka") - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("startingOffsets", "earliest") - .option("endingOffsets", "latest") - .option("subscribe", topic) - .load() - } - - private def createKafkaWriter( - input: DataFrame, - withTopic: Option[String] = None, - withOutputMode: Option[OutputMode] = None, - withOptions: Map[String, String] = Map[String, String]()) - (withSelectExpr: String*): StreamingQuery = { - var stream: DataStreamWriter[Row] = null - withTempDir { checkpointDir => - var df = input.toDF() - if (withSelectExpr.length > 0) { - df = df.selectExpr(withSelectExpr: _*) - } - stream = df.writeStream - .format("kafka") - .option("checkpointLocation", checkpointDir.getCanonicalPath) - .option("kafka.bootstrap.servers", testUtils.brokerAddress) - .option("kafka.max.block.ms", "5000") - .queryName("kafkaStream") - withTopic.foreach(stream.option("topic", _)) - withOutputMode.foreach(stream.outputMode(_)) - withOptions.foreach(opt => stream.option(opt._1, opt._2)) - } - stream.start() - } } diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala new file mode 100644 index 0000000000000..2fcf37a184684 --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSourceProviderSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.kafka010 + +import java.util.Locale + +import scala.collection.JavaConverters._ + +import org.mockito.Mockito.{mock, when} +import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class KafkaSourceProviderSuite extends SparkFunSuite with PrivateMethodTester { + + private val pollTimeoutMsMethod = PrivateMethod[Long]('pollTimeoutMs) + private val maxOffsetsPerTriggerMethod = PrivateMethod[Option[Long]]('maxOffsetsPerTrigger) + + override protected def afterEach(): Unit = { + SparkEnv.set(null) + super.afterEach() + } + + test("micro-batch mode - options should be handled as case-insensitive") { + def verifyFieldsInMicroBatchStream( + options: CaseInsensitiveStringMap, + expectedPollTimeoutMs: Long, + expectedMaxOffsetsPerTrigger: Option[Long]): Unit = { + // KafkaMicroBatchStream reads Spark conf from SparkEnv for default value + // hence we set mock SparkEnv here before creating KafkaMicroBatchStream + val sparkEnv = mock(classOf[SparkEnv]) + when(sparkEnv.conf).thenReturn(new SparkConf()) + SparkEnv.set(sparkEnv) + + val scan = getKafkaDataSourceScan(options) + val stream = scan.toMicroBatchStream("dummy").asInstanceOf[KafkaMicroBatchStream] + + assert(expectedPollTimeoutMs === getField(stream, pollTimeoutMsMethod)) + assert(expectedMaxOffsetsPerTrigger === getField(stream, maxOffsetsPerTriggerMethod)) + } + + val expectedValue = 1000L + buildCaseInsensitiveStringMapForUpperAndLowerKey( + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT -> expectedValue.toString, + KafkaSourceProvider.MAX_OFFSET_PER_TRIGGER -> expectedValue.toString) + .foreach(verifyFieldsInMicroBatchStream(_, expectedValue, Some(expectedValue))) + } + + test("SPARK-28142 - continuous mode - options should be handled as case-insensitive") { + def verifyFieldsInContinuousStream( + options: CaseInsensitiveStringMap, + expectedPollTimeoutMs: Long): Unit = { + val scan = getKafkaDataSourceScan(options) + val stream = scan.toContinuousStream("dummy").asInstanceOf[KafkaContinuousStream] + assert(expectedPollTimeoutMs === getField(stream, pollTimeoutMsMethod)) + } + + val expectedValue = 1000 + buildCaseInsensitiveStringMapForUpperAndLowerKey( + KafkaSourceProvider.CONSUMER_POLL_TIMEOUT -> expectedValue.toString) + .foreach(verifyFieldsInContinuousStream(_, expectedValue)) + } + + private def buildCaseInsensitiveStringMapForUpperAndLowerKey( + options: (String, String)*): Seq[CaseInsensitiveStringMap] = { + Seq(options.map(entry => (entry._1.toUpperCase(Locale.ROOT), entry._2)), + options.map(entry => (entry._1.toLowerCase(Locale.ROOT), entry._2))) + .map(buildKafkaSourceCaseInsensitiveStringMap) + } + + private def buildKafkaSourceCaseInsensitiveStringMap( + options: (String, String)*): CaseInsensitiveStringMap = { + val requiredOptions = Map("kafka.bootstrap.servers" -> "dummy", "subscribe" -> "dummy") + new CaseInsensitiveStringMap((options.toMap ++ requiredOptions).asJava) + } + + private def getKafkaDataSourceScan(options: CaseInsensitiveStringMap): Scan = { + val provider = new KafkaSourceProvider() + provider.getTable(options).newScanBuilder(options).build() + } + + private def getField[T](obj: AnyRef, method: PrivateMethod[T]): T = { + obj.invokePrivate(method()) + } +} diff --git a/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSparkConfSuite.scala b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSparkConfSuite.scala new file mode 100644 index 0000000000000..ca8b8b6e186ef --- /dev/null +++ b/external/kafka-0-10-sql/src/test/scala/org/apache/spark/sql/kafka010/KafkaSparkConfSuite.scala @@ -0,0 +1,30 @@ +/* + * 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.kafka010 + +import org.apache.spark.{LocalSparkContext, SparkConf, SparkFunSuite} +import org.apache.spark.util.ResetSystemProperties + +class KafkaSparkConfSuite extends SparkFunSuite with LocalSparkContext with ResetSystemProperties { + test("deprecated configs") { + val conf = new SparkConf() + + conf.set("spark.sql.kafkaConsumerCache.capacity", "32") + assert(conf.get(CONSUMER_CACHE_CAPACITY) === 32) + } +} 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 bf6934be52705..f2e4ee71450e6 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 @@ -20,20 +20,18 @@ package org.apache.spark.sql.kafka010 import java.io.{File, IOException} import java.lang.{Integer => JInt} import java.net.InetSocketAddress -import java.util.{Map => JMap, Properties, UUID} +import java.util.{Collections, Map => JMap, Properties, UUID} import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ -import scala.language.postfixOps import scala.util.Random -import kafka.admin.AdminUtils import kafka.api.Request import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.checkpoints.OffsetCheckpointFile import kafka.utils.ZkUtils import org.apache.kafka.clients.CommonClientConfigs -import org.apache.kafka.clients.admin.{AdminClient, CreatePartitionsOptions, NewPartitions} +import org.apache.kafka.clients.admin.{AdminClient, CreatePartitionsOptions, ListConsumerGroupsResult, NewPartitions, NewTopic} import org.apache.kafka.clients.consumer.KafkaConsumer import org.apache.kafka.clients.producer._ import org.apache.kafka.common.TopicPartition @@ -139,7 +137,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L setupEmbeddedZookeeper() setupEmbeddedKafkaServer() - eventually(timeout(60.seconds)) { + eventually(timeout(1.minute)) { assert(zkUtils.getAllBrokersInCluster().nonEmpty, "Broker was not up in 60 seconds") } } @@ -195,7 +193,8 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L var created = false while (!created) { try { - AdminUtils.createTopic(zkUtils, topic, partitions, 1) + val newTopic = new NewTopic(topic, partitions, 1) + adminClient.createTopics(Collections.singleton(newTopic)) created = true } catch { // Workaround fact that TopicExistsException is in kafka.common in 0.10.0 and @@ -222,7 +221,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L /** Delete a Kafka topic and wait until it is propagated to the whole cluster */ def deleteTopic(topic: String): Unit = { val partitions = zkUtils.getPartitionsForTopics(Seq(topic))(topic).size - AdminUtils.deleteTopic(zkUtils, topic) + adminClient.deleteTopics(Collections.singleton(topic)) verifyTopicDeletionWithRetries(zkUtils, topic, partitions, List(this.server)) } @@ -311,6 +310,10 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L offsets } + def listConsumerGroups(): ListConsumerGroupsResult = { + adminClient.listConsumerGroups() + } + protected def brokerConfiguration: Properties = { val props = new Properties() props.put("broker.id", "0") @@ -410,7 +413,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L topic: String, numPartitions: Int, servers: Seq[KafkaServer]) { - eventually(timeout(60.seconds), interval(200.millis)) { + eventually(timeout(1.minute), interval(200.milliseconds)) { try { verifyTopicDeletion(topic, numPartitions, servers) } catch { @@ -418,14 +421,15 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L // As pushing messages into Kafka updates Zookeeper asynchronously, there is a small // chance that a topic will be recreated after deletion due to the asynchronous update. // Hence, delete the topic and retry. - AdminUtils.deleteTopic(zkUtils, topic) + adminClient.deleteTopics(Collections.singleton(topic)) throw e } } } private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { - def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + def isPropagated = server.dataPlaneRequestProcessor.metadataCache + .getPartitionInfo(topic, partition) match { case Some(partitionState) => zkUtils.getLeaderForPartition(topic, partition).isDefined && Request.isValidBrokerId(partitionState.basePartitionState.leader) && @@ -434,7 +438,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L case _ => false } - eventually(timeout(60.seconds)) { + eventually(timeout(1.minute)) { assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") } } @@ -443,7 +447,7 @@ class KafkaTestUtils(withBrokerProps: Map[String, Object] = Map.empty) extends L * Wait until the latest offset of the given `TopicPartition` is not less than `offset`. */ def waitUntilOffsetAppears(topicPartition: TopicPartition, offset: Long): Unit = { - eventually(timeout(60.seconds)) { + eventually(timeout(1.minute)) { val currentOffset = getLatestOffsets(Set(topicPartition.topic)).get(topicPartition) assert(currentOffset.nonEmpty && currentOffset.get >= offset) } diff --git a/external/kafka-0-8/pom.xml b/external/kafka-0-10-token-provider/pom.xml similarity index 66% rename from external/kafka-0-8/pom.xml rename to external/kafka-0-10-token-provider/pom.xml index 4545877a9d83f..01ca96b4f8f40 100644 --- a/external/kafka-0-8/pom.xml +++ b/external/kafka-0-10-token-provider/pom.xml @@ -20,23 +20,24 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-streaming-kafka-0-8_2.11 + org.apache.spark + spark-token-provider-kafka-0-10_2.12 - streaming-kafka-0-8 + token-provider-kafka-0-10 jar - Spark Integration for Kafka 0.8 + Kafka 0.10+ Token Provider for Streaming http://spark.apache.org/ org.apache.spark - spark-streaming_${scala.binary.version} + spark-core_${scala.binary.version} ${project.version} provided @@ -49,40 +50,12 @@ org.apache.kafka - kafka_${scala.binary.version} - 0.8.2.1 - - - com.sun.jmx - jmxri - - - com.sun.jdmk - jmxtools - - - net.sf.jopt-simple - jopt-simple - - - org.slf4j - slf4j-simple - - - org.apache.zookeeper - zookeeper - - - - - net.sf.jopt-simple - jopt-simple - 3.2 - test + kafka-clients + ${kafka.version} - org.scalacheck - scalacheck_${scala.binary.version} + org.mockito + mockito-core test @@ -102,8 +75,10 @@ + target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + diff --git a/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider b/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider new file mode 100644 index 0000000000000..34014016584de --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider @@ -0,0 +1 @@ +org.apache.spark.kafka010.KafkaDelegationTokenProvider diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala new file mode 100644 index 0000000000000..0c61045d6d487 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaConfigUpdater.scala @@ -0,0 +1,84 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import scala.collection.JavaConverters._ + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.SaslConfigs + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging + +/** + * Class to conveniently update Kafka config params, while logging the changes + */ +private[spark] case class KafkaConfigUpdater(module: String, kafkaParams: Map[String, Object]) + extends Logging { + private val map = new ju.HashMap[String, Object](kafkaParams.asJava) + + def set(key: String, value: Object): this.type = { + map.put(key, value) + if (log.isDebugEnabled()) { + val redactedValue = KafkaRedactionUtil.redactParams(Seq((key, value))).head._2 + val redactedOldValue = KafkaRedactionUtil + .redactParams(Seq((key, kafkaParams.getOrElse(key, "")))).head._2 + logDebug(s"$module: Set $key to $redactedValue, earlier value: $redactedOldValue") + } + this + } + + def setIfUnset(key: String, value: Object): this.type = { + if (!map.containsKey(key)) { + map.put(key, value) + if (log.isDebugEnabled()) { + val redactedValue = KafkaRedactionUtil.redactParams(Seq((key, value))).head._2 + logDebug(s"$module: Set $key to $redactedValue") + } + } + this + } + + def setAuthenticationConfigIfNeeded(): this.type = { + // There are multiple possibilities to log in and applied in the following order: + // - JVM global security provided -> try to log in with JVM global security configuration + // which can be configured for example with 'java.security.auth.login.config'. + // For this no additional parameter needed. + // - Token is provided -> try to log in with scram module using kafka's dynamic JAAS + // configuration. + if (KafkaTokenUtil.isGlobalJaasConfigurationProvided) { + logDebug("JVM global security configuration detected, using it for login.") + } else { + val clusterConfig = KafkaTokenUtil.findMatchingToken(SparkEnv.get.conf, + map.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG).asInstanceOf[String]) + clusterConfig.foreach { clusterConf => + logDebug("Delegation token detected, using it for login.") + val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) + set(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + require(clusterConf.tokenMechanism.startsWith("SCRAM"), + "Delegation token works only with SCRAM mechanism.") + set(SaslConfigs.SASL_MECHANISM, clusterConf.tokenMechanism) + } + } + this + } + + def build(): ju.Map[String, Object] = map +} diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala new file mode 100644 index 0000000000000..69fcf55d521b0 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaDelegationTokenProvider.scala @@ -0,0 +1,84 @@ +/* + * 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.kafka010 + +import scala.util.control.NonFatal + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.security.Credentials +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.security.HadoopDelegationTokenProvider + +private[spark] class KafkaDelegationTokenProvider + extends HadoopDelegationTokenProvider with Logging { + + override def serviceName: String = "kafka" + + override def obtainDelegationTokens( + hadoopConf: Configuration, + sparkConf: SparkConf, + creds: Credentials): Option[Long] = { + var lowestNextRenewalDate: Option[Long] = None + try { + KafkaTokenSparkConf.getAllClusterConfigs(sparkConf).foreach { clusterConf => + try { + if (delegationTokensRequired(clusterConf)) { + logDebug( + s"Attempting to fetch Kafka security token for cluster ${clusterConf.identifier}.") + val (token, nextRenewalDate) = KafkaTokenUtil.obtainToken(sparkConf, clusterConf) + creds.addToken(token.getService, token) + if (lowestNextRenewalDate.isEmpty || nextRenewalDate < lowestNextRenewalDate.get) { + lowestNextRenewalDate = Some(nextRenewalDate) + } + } else { + logDebug( + s"Cluster ${clusterConf.identifier} does not require delegation token, skipping.") + } + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token from service: $serviceName " + + s"cluster: ${clusterConf.identifier}", e) + } + } + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token cluster configuration", e) + } + lowestNextRenewalDate + } + + override def delegationTokensRequired( + sparkConf: SparkConf, + hadoopConf: Configuration): Boolean = { + try { + KafkaTokenSparkConf.getAllClusterConfigs(sparkConf).exists(delegationTokensRequired(_)) + } catch { + case NonFatal(e) => + logWarning(s"Failed to get token cluster configuration", e) + false + } + } + + private def delegationTokensRequired(clusterConf: KafkaTokenClusterConf): Boolean = + clusterConf.securityProtocol == SASL_SSL.name || + clusterConf.securityProtocol == SSL.name || + clusterConf.securityProtocol == SASL_PLAINTEXT.name +} diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala new file mode 100644 index 0000000000000..daf2d9adff3b5 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaRedactionUtil.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.kafka010 + +import org.apache.kafka.common.config.SaslConfigs + +import org.apache.spark.SparkEnv +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SECRET_REDACTION_PATTERN +import org.apache.spark.util.Utils.{redact, REDACTION_REPLACEMENT_TEXT} + +private[spark] object KafkaRedactionUtil extends Logging { + private[spark] def redactParams(params: Seq[(String, Object)]): Seq[(String, String)] = { + val redactionPattern = Some(SparkEnv.get.conf.get(SECRET_REDACTION_PATTERN)) + params.map { case (key, value) => + if (value != null) { + if (key.equalsIgnoreCase(SaslConfigs.SASL_JAAS_CONFIG)) { + (key, redactJaasParam(value.asInstanceOf[String])) + } else { + val (_, newValue) = redact(redactionPattern, Seq((key, value.toString))).head + (key, newValue) + } + } else { + (key, value.asInstanceOf[String]) + } + } + } + + private[kafka010] def redactJaasParam(param: String): String = { + if (param != null && !param.isEmpty) { + param.replaceAll("password=\".*\"", s"""password="$REDACTION_REPLACEMENT_TEXT"""") + } else { + param + } + } +} diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala new file mode 100644 index 0000000000000..e1f3c800a51f8 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenSparkConf.scala @@ -0,0 +1,102 @@ +/* + * 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.kafka010 + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} +import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL + +import org.apache.spark.SparkConf +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT + +private[spark] case class KafkaTokenClusterConf( + identifier: String, + authBootstrapServers: String, + targetServersRegex: String, + securityProtocol: String, + kerberosServiceName: String, + trustStoreLocation: Option[String], + trustStorePassword: Option[String], + keyStoreLocation: Option[String], + keyStorePassword: Option[String], + keyPassword: Option[String], + tokenMechanism: String, + specifiedKafkaParams: Map[String, String]) { + override def toString: String = s"KafkaTokenClusterConf{" + + s"identifier=$identifier, " + + s"authBootstrapServers=$authBootstrapServers, " + + s"targetServersRegex=$targetServersRegex, " + + s"securityProtocol=$securityProtocol, " + + s"kerberosServiceName=$kerberosServiceName, " + + s"trustStoreLocation=$trustStoreLocation, " + + s"trustStorePassword=${trustStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"keyStoreLocation=$keyStoreLocation, " + + s"keyStorePassword=${keyStorePassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"keyPassword=${keyPassword.map(_ => REDACTION_REPLACEMENT_TEXT)}, " + + s"tokenMechanism=$tokenMechanism, " + + s"specifiedKafkaParams=${KafkaRedactionUtil.redactParams(specifiedKafkaParams.toSeq)}}" +} + +private [kafka010] object KafkaTokenSparkConf extends Logging { + val CLUSTERS_CONFIG_PREFIX = "spark.kafka.clusters." + val DEFAULT_TARGET_SERVERS_REGEX = ".*" + val DEFAULT_SASL_KERBEROS_SERVICE_NAME = "kafka" + val DEFAULT_SASL_TOKEN_MECHANISM = "SCRAM-SHA-512" + + def getClusterConfig(sparkConf: SparkConf, identifier: String): KafkaTokenClusterConf = { + val configPrefix = s"$CLUSTERS_CONFIG_PREFIX$identifier." + val sparkClusterConf = sparkConf.getAllWithPrefix(configPrefix).toMap + val configKafkaPrefix = s"${configPrefix}kafka." + val sparkClusterKafkaConf = sparkConf.getAllWithPrefix(configKafkaPrefix).toMap + val result = KafkaTokenClusterConf( + identifier, + sparkClusterConf + .getOrElse(s"auth.${CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG}", + throw new NoSuchElementException( + s"${configPrefix}auth.${CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG}")), + sparkClusterConf.getOrElse(s"target.${CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG}.regex", + KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX), + sparkClusterConf.getOrElse(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, SASL_SSL.name), + sparkClusterConf.getOrElse(SaslConfigs.SASL_KERBEROS_SERVICE_NAME, + KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME), + sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG), + sparkClusterConf.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG), + sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG), + sparkClusterConf.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG), + sparkClusterConf.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG), + sparkClusterConf.getOrElse("sasl.token.mechanism", + KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM), + sparkClusterKafkaConf + ) + logDebug(s"getClusterConfig($identifier): $result") + result + } + + def getAllClusterConfigs(sparkConf: SparkConf): Set[KafkaTokenClusterConf] = { + sparkConf.getAllWithPrefix(KafkaTokenSparkConf.CLUSTERS_CONFIG_PREFIX).toMap.keySet + .flatMap { k => + val split = k.split('.') + if (split.length > 0 && split(0).nonEmpty) { + Some(split(0)) + } else { + None + } + }.map(getClusterConfig(sparkConf, _)) + } +} diff --git a/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala new file mode 100644 index 0000000000000..950df867e1e8a --- /dev/null +++ b/external/kafka-0-10-token-provider/src/main/scala/org/apache/spark/kafka010/KafkaTokenUtil.scala @@ -0,0 +1,288 @@ +/* + * 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.kafka010 + +import java.{util => ju} +import java.text.SimpleDateFormat +import java.util.regex.Pattern + +import scala.collection.JavaConverters._ +import scala.util.control.NonFatal + +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.security.token.Token +import org.apache.hadoop.security.token.delegation.AbstractDelegationTokenIdentifier +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.clients.admin.{AdminClient, CreateDelegationTokenOptions} +import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} +import org.apache.kafka.common.security.JaasContext +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} +import org.apache.kafka.common.security.scram.ScramLoginModule +import org.apache.kafka.common.security.token.delegation.DelegationToken + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils +import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT + +private[spark] object KafkaTokenUtil extends Logging { + val TOKEN_KIND = new Text("KAFKA_DELEGATION_TOKEN") + private val TOKEN_SERVICE_PREFIX = "kafka.server.delegation.token" + + private[kafka010] def getTokenService(identifier: String): Text = + new Text(s"$TOKEN_SERVICE_PREFIX.$identifier") + + private def getClusterIdentifier(service: Text): String = + service.toString().replace(s"$TOKEN_SERVICE_PREFIX.", "") + + private[spark] class KafkaDelegationTokenIdentifier extends AbstractDelegationTokenIdentifier { + override def getKind: Text = TOKEN_KIND + } + + private[kafka010] def obtainToken( + sparkConf: SparkConf, + clusterConf: KafkaTokenClusterConf): (Token[KafkaDelegationTokenIdentifier], Long) = { + checkProxyUser() + + val adminClient = AdminClient.create(createAdminClientProperties(sparkConf, clusterConf)) + val createDelegationTokenOptions = new CreateDelegationTokenOptions() + val createResult = adminClient.createDelegationToken(createDelegationTokenOptions) + val token = createResult.delegationToken().get() + printToken(token) + + (new Token[KafkaDelegationTokenIdentifier]( + token.tokenInfo.tokenId.getBytes, + token.hmacAsBase64String.getBytes, + TOKEN_KIND, + getTokenService(clusterConf.identifier) + ), token.tokenInfo.expiryTimestamp) + } + + private[kafka010] def checkProxyUser(): Unit = { + val currentUser = UserGroupInformation.getCurrentUser() + // Obtaining delegation token for proxy user is planned but not yet implemented + // See https://issues.apache.org/jira/browse/KAFKA-6945 + require(!SparkHadoopUtil.get.isProxyUser(currentUser), "Obtaining delegation token for proxy " + + "user is not yet supported.") + } + + private[kafka010] def createAdminClientProperties( + sparkConf: SparkConf, + clusterConf: KafkaTokenClusterConf): ju.Properties = { + val adminClientProperties = new ju.Properties + + adminClientProperties.put(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG, + clusterConf.authBootstrapServers) + + adminClientProperties.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, + clusterConf.securityProtocol) + clusterConf.securityProtocol match { + case SASL_SSL.name => + setTrustStoreProperties(clusterConf, adminClientProperties) + + case SSL.name => + setTrustStoreProperties(clusterConf, adminClientProperties) + setKeyStoreProperties(clusterConf, adminClientProperties) + logWarning("Obtaining kafka delegation token with SSL protocol. Please " + + "configure 2-way authentication on the broker side.") + + case SASL_PLAINTEXT.name => + logWarning("Obtaining kafka delegation token through plain communication channel. Please " + + "consider the security impact.") + } + + // There are multiple possibilities to log in and applied in the following order: + // - JVM global security provided -> try to log in with JVM global security configuration + // which can be configured for example with 'java.security.auth.login.config'. + // For this no additional parameter needed. + // - Keytab is provided -> try to log in with kerberos module and keytab using kafka's dynamic + // JAAS configuration. + // - Keytab not provided -> try to log in with kerberos module and ticket cache using kafka's + // dynamic JAAS configuration. + // Kafka client is unable to use subject from JVM which already logged in + // to kdc (see KAFKA-7677) + if (isGlobalJaasConfigurationProvided) { + logDebug("JVM global security configuration detected, using it for login.") + } else { + adminClientProperties.put(SaslConfigs.SASL_MECHANISM, SaslConfigs.GSSAPI_MECHANISM) + if (sparkConf.contains(KEYTAB)) { + logDebug("Keytab detected, using it for login.") + val jaasParams = getKeytabJaasParams(sparkConf, clusterConf) + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + } else { + logDebug("Using ticket cache for login.") + val jaasParams = getTicketCacheJaasParams(clusterConf) + adminClientProperties.put(SaslConfigs.SASL_JAAS_CONFIG, jaasParams) + } + } + + logDebug("AdminClient params before specified params: " + + s"${KafkaRedactionUtil.redactParams(adminClientProperties.asScala.toSeq)}") + + clusterConf.specifiedKafkaParams.foreach { param => + adminClientProperties.setProperty(param._1, param._2) + } + + logDebug("AdminClient params after specified params: " + + s"${KafkaRedactionUtil.redactParams(adminClientProperties.asScala.toSeq)}") + + adminClientProperties + } + + def isGlobalJaasConfigurationProvided: Boolean = { + try { + JaasContext.loadClientContext(ju.Collections.emptyMap[String, Object]()) + true + } catch { + case NonFatal(_) => false + } + } + + private def setTrustStoreProperties( + clusterConf: KafkaTokenClusterConf, + properties: ju.Properties): Unit = { + clusterConf.trustStoreLocation.foreach { truststoreLocation => + properties.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreLocation) + } + clusterConf.trustStorePassword.foreach { truststorePassword => + properties.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststorePassword) + } + } + + private def setKeyStoreProperties( + clusterConf: KafkaTokenClusterConf, + properties: ju.Properties): Unit = { + clusterConf.keyStoreLocation.foreach { keystoreLocation => + properties.put(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG, keystoreLocation) + } + clusterConf.keyStorePassword.foreach { keystorePassword => + properties.put(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG, keystorePassword) + } + clusterConf.keyPassword.foreach { keyPassword => + properties.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, keyPassword) + } + } + + private def getKeytabJaasParams( + sparkConf: SparkConf, + clusterConf: KafkaTokenClusterConf): String = { + val params = + s""" + |${getKrb5LoginModuleName} required + | debug=${isGlobalKrbDebugEnabled()} + | useKeyTab=true + | serviceName="${clusterConf.kerberosServiceName}" + | keyTab="${sparkConf.get(KEYTAB).get}" + | principal="${sparkConf.get(PRINCIPAL).get}"; + """.stripMargin.replace("\n", "") + logDebug(s"Krb keytab JAAS params: $params") + params + } + + private def getTicketCacheJaasParams(clusterConf: KafkaTokenClusterConf): String = { + val params = + s""" + |${getKrb5LoginModuleName} required + | debug=${isGlobalKrbDebugEnabled()} + | useTicketCache=true + | serviceName="${clusterConf.kerberosServiceName}"; + """.stripMargin.replace("\n", "").trim + logDebug(s"Krb ticket cache JAAS params: $params") + params + } + + /** + * Krb5LoginModule package vary in different JVMs. + * Please see Hadoop UserGroupInformation for further details. + */ + private def getKrb5LoginModuleName(): String = { + if (System.getProperty("java.vendor").contains("IBM")) { + "com.ibm.security.auth.module.Krb5LoginModule" + } else { + "com.sun.security.auth.module.Krb5LoginModule" + } + } + + private def isGlobalKrbDebugEnabled(): Boolean = { + if (System.getProperty("java.vendor").contains("IBM")) { + val debug = System.getenv("com.ibm.security.krb5.Krb5Debug") + debug != null && debug.equalsIgnoreCase("all") + } else { + val debug = System.getenv("sun.security.krb5.debug") + debug != null && debug.equalsIgnoreCase("true") + } + } + + private def printToken(token: DelegationToken): Unit = { + if (log.isDebugEnabled) { + val dateFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm") + logDebug("%-15s %-30s %-15s %-25s %-15s %-15s %-15s".format( + "TOKENID", "HMAC", "OWNER", "RENEWERS", "ISSUEDATE", "EXPIRYDATE", "MAXDATE")) + val tokenInfo = token.tokenInfo + logDebug("%-15s %-15s %-15s %-25s %-15s %-15s %-15s".format( + REDACTION_REPLACEMENT_TEXT, + tokenInfo.tokenId, + tokenInfo.owner, + tokenInfo.renewersAsString, + dateFormat.format(tokenInfo.issueTimestamp), + dateFormat.format(tokenInfo.expiryTimestamp), + dateFormat.format(tokenInfo.maxTimestamp))) + } + } + + def findMatchingToken( + sparkConf: SparkConf, + bootStrapServers: String): Option[KafkaTokenClusterConf] = { + val tokens = UserGroupInformation.getCurrentUser().getCredentials.getAllTokens.asScala + val clusterConfigs = tokens + .filter(_.getService().toString().startsWith(TOKEN_SERVICE_PREFIX)) + .map { token => + KafkaTokenSparkConf.getClusterConfig(sparkConf, getClusterIdentifier(token.getService())) + } + .filter { clusterConfig => + val pattern = Pattern.compile(clusterConfig.targetServersRegex) + Utils.stringToSeq(bootStrapServers).exists(pattern.matcher(_).matches()) + } + require(clusterConfigs.size <= 1, "More than one delegation token matches the following " + + s"bootstrap servers: $bootStrapServers.") + clusterConfigs.headOption + } + + def getTokenJaasParams(clusterConf: KafkaTokenClusterConf): String = { + val token = UserGroupInformation.getCurrentUser().getCredentials.getToken( + getTokenService(clusterConf.identifier)) + val username = new String(token.getIdentifier) + val password = new String(token.getPassword) + + val loginModuleName = classOf[ScramLoginModule].getName + val params = + s""" + |$loginModuleName required + | tokenauth=true + | serviceName="${clusterConf.kerberosServiceName}" + | username="$username" + | password="$password"; + """.stripMargin.replace("\n", "").trim + logDebug(s"Scram JAAS params: ${KafkaRedactionUtil.redactJaasParam(params)}") + + params + } +} diff --git a/external/kafka-0-8/src/test/resources/log4j.properties b/external/kafka-0-10-token-provider/src/test/resources/log4j.properties similarity index 96% rename from external/kafka-0-8/src/test/resources/log4j.properties rename to external/kafka-0-10-token-provider/src/test/resources/log4j.properties index fd51f8faf56b9..75e3b53a093f6 100644 --- a/external/kafka-0-8/src/test/resources/log4j.properties +++ b/external/kafka-0-10-token-provider/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.spark-project.jetty=WARN diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala new file mode 100644 index 0000000000000..7a172892e778c --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaConfigUpdaterSuite.scala @@ -0,0 +1,131 @@ +/* + * 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.kafka010 + +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.SaslConfigs + +import org.apache.spark.SparkFunSuite + +class KafkaConfigUpdaterSuite extends SparkFunSuite with KafkaDelegationTokenTest { + private val testModule = "testModule" + private val testKey = "testKey" + private val testValue = "testValue" + private val otherTestValue = "otherTestValue" + + test("set should always set value") { + val params = Map.empty[String, String] + + val updatedParams = KafkaConfigUpdater(testModule, params) + .set(testKey, testValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setIfUnset without existing key should set value") { + val params = Map.empty[String, String] + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setIfUnset(testKey, testValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setIfUnset with existing key should not set value") { + val params = Map[String, String](testKey -> testValue) + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setIfUnset(testKey, otherTestValue) + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(testKey) === testValue) + } + + test("setAuthenticationConfigIfNeeded with global security should not set values") { + val params = Map.empty[String, String] + setGlobalKafkaClientConfig() + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 0) + } + + test("setAuthenticationConfigIfNeeded with token should set values") { + val params = Map( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers + ) + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers + ) + ) + addTokenToUGI(tokenService1) + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 3) + assert(updatedParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) === bootStrapServers) + assert(updatedParams.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) + assert(updatedParams.get(SaslConfigs.SASL_MECHANISM) === + KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + } + + test("setAuthenticationConfigIfNeeded with invalid mechanism should throw exception") { + val params = Map( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers + ) + setSparkEnv( + Map( + s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers" -> bootStrapServers, + s"spark.kafka.clusters.$identifier1.sasl.token.mechanism" -> "intentionally_invalid" + ) + ) + addTokenToUGI(tokenService1) + + val e = intercept[IllegalArgumentException] { + KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + } + + assert(e.getMessage.contains("Delegation token works only with SCRAM mechanism.")) + } + + test("setAuthenticationConfigIfNeeded without security should not set values") { + val params = Map( + CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG -> bootStrapServers + ) + setSparkEnv(Map.empty) + + val updatedParams = KafkaConfigUpdater(testModule, params) + .setAuthenticationConfigIfNeeded() + .build() + + assert(updatedParams.size() === 1) + assert(updatedParams.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) === bootStrapServers) + } +} diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala new file mode 100644 index 0000000000000..eebbf96afa470 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaDelegationTokenTest.scala @@ -0,0 +1,126 @@ +/* + * 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.kafka010 + +import java.{util => ju} +import javax.security.auth.login.{AppConfigurationEntry, Configuration} + +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.token.Token +import org.mockito.Mockito.mock +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkEnv, SparkFunSuite} +import org.apache.spark.kafka010.KafkaTokenUtil.KafkaDelegationTokenIdentifier + +/** + * This is a trait which provides functionalities for Kafka delegation token related test suites. + */ +trait KafkaDelegationTokenTest extends BeforeAndAfterEach { + self: SparkFunSuite => + + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + + protected val tokenId = "tokenId" + ju.UUID.randomUUID().toString + protected val tokenPassword = "tokenPassword" + ju.UUID.randomUUID().toString + + protected val identifier1 = "cluster1" + protected val identifier2 = "cluster2" + protected val tokenService1 = KafkaTokenUtil.getTokenService(identifier1) + protected val tokenService2 = KafkaTokenUtil.getTokenService(identifier2) + protected val bootStrapServers = "127.0.0.1:0" + protected val matchingTargetServersRegex = "127.0.0.*:0" + protected val nonMatchingTargetServersRegex = "127.0.intentionally_non_matching.*:0" + protected val trustStoreLocation = "/path/to/trustStore" + protected val trustStorePassword = "trustStoreSecret" + protected val keyStoreLocation = "/path/to/keyStore" + protected val keyStorePassword = "keyStoreSecret" + protected val keyPassword = "keySecret" + protected val keytab = "/path/to/keytab" + protected val principal = "user@domain.com" + + private class KafkaJaasConfiguration extends Configuration { + val entry = + new AppConfigurationEntry( + "DummyModule", + AppConfigurationEntry.LoginModuleControlFlag.REQUIRED, + ju.Collections.emptyMap[String, Object]() + ) + + override def getAppConfigurationEntry(name: String): Array[AppConfigurationEntry] = { + if (name.equals("KafkaClient")) { + Array(entry) + } else { + null + } + } + } + + override def afterEach(): Unit = { + try { + Configuration.setConfiguration(null) + UserGroupInformation.setLoginUser(null) + SparkEnv.set(null) + } finally { + super.afterEach() + } + } + + protected def setGlobalKafkaClientConfig(): Unit = { + Configuration.setConfiguration(new KafkaJaasConfiguration) + } + + protected def addTokenToUGI(tokenService: Text): Unit = { + val token = new Token[KafkaDelegationTokenIdentifier]( + tokenId.getBytes, + tokenPassword.getBytes, + KafkaTokenUtil.TOKEN_KIND, + tokenService + ) + val creds = new Credentials() + creds.addToken(token.getService, token) + UserGroupInformation.getCurrentUser.addCredentials(creds) + } + + protected def setSparkEnv(settings: Iterable[(String, String)]): Unit = { + val conf = new SparkConf().setAll(settings) + val env = mock(classOf[SparkEnv]) + doReturn(conf).when(env).conf + SparkEnv.set(env) + } + + protected def createClusterConf( + identifier: String, + securityProtocol: String, + specifiedKafkaParams: Map[String, String] = Map.empty): KafkaTokenClusterConf = { + KafkaTokenClusterConf( + identifier, + bootStrapServers, + KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX, + securityProtocol, + KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME, + Some(trustStoreLocation), + Some(trustStorePassword), + Some(keyStoreLocation), + Some(keyStorePassword), + Some(keyPassword), + KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM, + specifiedKafkaParams) + } +} diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala new file mode 100644 index 0000000000000..9aa7618d76da0 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaHadoopDelegationTokenManagerSuite.scala @@ -0,0 +1,32 @@ +/* + * 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.kafka010 + +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.security.HadoopDelegationTokenManager + +class KafkaHadoopDelegationTokenManagerSuite extends SparkFunSuite { + private val hadoopConf = new Configuration() + + test("default configuration") { + val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf, null) + assert(manager.isProviderLoaded("kafka")) + } +} diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaRedactionUtilSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaRedactionUtilSuite.scala new file mode 100644 index 0000000000000..c8b0dac7b213d --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaRedactionUtilSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.kafka010 + +import java.{util => ju} + +import org.apache.kafka.clients.consumer.ConsumerConfig +import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} +import org.apache.kafka.common.security.auth.SecurityProtocol.SASL_SSL +import org.apache.kafka.common.serialization.StringDeserializer + +import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.SECRET_REDACTION_PATTERN +import org.apache.spark.util.Utils.REDACTION_REPLACEMENT_TEXT + +class KafkaRedactionUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { + test("redactParams should give back empty parameters") { + setSparkEnv(Map.empty) + assert(KafkaRedactionUtil.redactParams(Seq()) === Seq()) + } + + test("redactParams should give back null value") { + setSparkEnv(Map.empty) + val kafkaParams = Seq( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> null + ) + + assert(KafkaRedactionUtil.redactParams(kafkaParams) === kafkaParams) + } + + test("redactParams should redact non String parameters") { + setSparkEnv( + Map( + SECRET_REDACTION_PATTERN.key -> ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG + ) + ) + val kafkaParams = Seq( + ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG -> classOf[StringDeserializer] + ) + + val redactedParams = KafkaRedactionUtil.redactParams(kafkaParams).toMap + + assert(redactedParams.size === 1) + assert(redactedParams.get(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG).get + === REDACTION_REPLACEMENT_TEXT) + } + + test("redactParams should redact token password from parameters") { + setSparkEnv(Map.empty) + val groupId = "id-" + ju.UUID.randomUUID().toString + addTokenToUGI(tokenService1) + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) + val kafkaParams = Seq( + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + SaslConfigs.SASL_JAAS_CONFIG -> jaasParams + ) + + val redactedParams = KafkaRedactionUtil.redactParams(kafkaParams).toMap + + assert(redactedParams.size === 2) + assert(redactedParams.get(ConsumerConfig.GROUP_ID_CONFIG).get === groupId) + val redactedJaasParams = redactedParams.get(SaslConfigs.SASL_JAAS_CONFIG).get + assert(redactedJaasParams.contains(tokenId)) + assert(!redactedJaasParams.contains(tokenPassword)) + } + + test("redactParams should redact passwords from parameters") { + setSparkEnv(Map.empty) + val groupId = "id-" + ju.UUID.randomUUID().toString + val kafkaParams = Seq( + ConsumerConfig.GROUP_ID_CONFIG -> groupId, + SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG -> trustStorePassword, + SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG -> keyStorePassword, + SslConfigs.SSL_KEY_PASSWORD_CONFIG -> keyPassword + ) + + val redactedParams = KafkaRedactionUtil.redactParams(kafkaParams).toMap + + assert(redactedParams.size === 4) + assert(redactedParams(ConsumerConfig.GROUP_ID_CONFIG) === groupId) + assert(redactedParams(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) === REDACTION_REPLACEMENT_TEXT) + assert(redactedParams(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) === REDACTION_REPLACEMENT_TEXT) + assert(redactedParams(SslConfigs.SSL_KEY_PASSWORD_CONFIG) === REDACTION_REPLACEMENT_TEXT) + } + + test("redactJaasParam should give back null") { + assert(KafkaRedactionUtil.redactJaasParam(null) === null) + } + + test("redactJaasParam should give back empty string") { + assert(KafkaRedactionUtil.redactJaasParam("") === "") + } + + test("redactJaasParam should redact token password") { + addTokenToUGI(tokenService1) + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) + + val redactedJaasParams = KafkaRedactionUtil.redactJaasParam(jaasParams) + + assert(redactedJaasParams.contains(tokenId)) + assert(!redactedJaasParams.contains(tokenPassword)) + } +} diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala new file mode 100644 index 0000000000000..61184a6fac33d --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenSparkConfSuite.scala @@ -0,0 +1,138 @@ +/* + * 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.kafka010 + +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_SSL, SSL} +import org.scalatest.BeforeAndAfterEach + +import org.apache.spark.{SparkConf, SparkFunSuite} + +class KafkaTokenSparkConfSuite extends SparkFunSuite with BeforeAndAfterEach { + private val identifier1 = "cluster1" + private val identifier2 = "cluster2" + private val authBootStrapServers = "127.0.0.1:0" + private val targetServersRegex = "127.0.0.*:0" + private val securityProtocol = SSL.name + private val kerberosServiceName = "kafka1" + private val trustStoreLocation = "/path/to/trustStore" + private val trustStorePassword = "trustStoreSecret" + private val keyStoreLocation = "/path/to/keyStore" + private val keyStorePassword = "keyStoreSecret" + private val keyPassword = "keySecret" + private val tokenMechanism = "SCRAM-SHA-256" + + private var sparkConf: SparkConf = null + + override def beforeEach(): Unit = { + super.beforeEach() + sparkConf = new SparkConf() + } + + test("getClusterConfig should trow exception when not exists") { + val thrown = intercept[NoSuchElementException] { + KafkaTokenSparkConf.getClusterConfig(sparkConf, "invalid") + } + assert(thrown.getMessage contains "spark.kafka.clusters.invalid.auth.bootstrap.servers") + } + + test("getClusterConfig should return entry with defaults") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", authBootStrapServers) + + val clusterConfig = KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1) + assert(clusterConfig.identifier === identifier1) + assert(clusterConfig.authBootstrapServers === authBootStrapServers) + assert(clusterConfig.targetServersRegex === KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX) + assert(clusterConfig.securityProtocol === SASL_SSL.name) + assert(clusterConfig.kerberosServiceName === + KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME) + assert(clusterConfig.trustStoreLocation === None) + assert(clusterConfig.trustStorePassword === None) + assert(clusterConfig.keyStoreLocation === None) + assert(clusterConfig.keyStorePassword === None) + assert(clusterConfig.keyPassword === None) + assert(clusterConfig.tokenMechanism === KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + } + + test("getClusterConfig should return entry overwrite defaults") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", authBootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.target.bootstrap.servers.regex", + targetServersRegex) + sparkConf.set(s"spark.kafka.clusters.$identifier1.security.protocol", securityProtocol) + sparkConf.set(s"spark.kafka.clusters.$identifier1.sasl.kerberos.service.name", + kerberosServiceName) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.truststore.location", trustStoreLocation) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.truststore.password", trustStorePassword) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.keystore.location", keyStoreLocation) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.keystore.password", keyStorePassword) + sparkConf.set(s"spark.kafka.clusters.$identifier1.ssl.key.password", keyPassword) + sparkConf.set(s"spark.kafka.clusters.$identifier1.sasl.token.mechanism", tokenMechanism) + + val clusterConfig = KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1) + assert(clusterConfig.identifier === identifier1) + assert(clusterConfig.authBootstrapServers === authBootStrapServers) + assert(clusterConfig.targetServersRegex === targetServersRegex) + assert(clusterConfig.securityProtocol === securityProtocol) + assert(clusterConfig.kerberosServiceName === kerberosServiceName) + assert(clusterConfig.trustStoreLocation === Some(trustStoreLocation)) + assert(clusterConfig.trustStorePassword === Some(trustStorePassword)) + assert(clusterConfig.keyStoreLocation === Some(keyStoreLocation)) + assert(clusterConfig.keyStorePassword === Some(keyStorePassword)) + assert(clusterConfig.keyPassword === Some(keyPassword)) + assert(clusterConfig.tokenMechanism === tokenMechanism) + } + + test("getClusterConfig should return specified kafka params") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", authBootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.kafka.customKey", "customValue") + + val clusterConfig = KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1) + assert(clusterConfig.identifier === identifier1) + assert(clusterConfig.authBootstrapServers === authBootStrapServers) + assert(clusterConfig.specifiedKafkaParams === Map("customKey" -> "customValue")) + } + + test("getAllClusterConfigs should return empty list when nothing configured") { + assert(KafkaTokenSparkConf.getAllClusterConfigs(sparkConf).isEmpty) + } + + test("getAllClusterConfigs should return empty list with malformed configuration") { + sparkConf.set(s"spark.kafka.clusters.", authBootStrapServers) + assert(KafkaTokenSparkConf.getAllClusterConfigs(sparkConf).isEmpty) + } + + test("getAllClusterConfigs should return multiple entries") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", authBootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier2.auth.bootstrap.servers", authBootStrapServers) + + val clusterConfigs = KafkaTokenSparkConf.getAllClusterConfigs(sparkConf) + assert(clusterConfigs.size === 2) + clusterConfigs.foreach { clusterConfig => + assert(clusterConfig.authBootstrapServers === authBootStrapServers) + assert(clusterConfig.targetServersRegex === KafkaTokenSparkConf.DEFAULT_TARGET_SERVERS_REGEX) + assert(clusterConfig.securityProtocol === SASL_SSL.name) + assert(clusterConfig.kerberosServiceName === + KafkaTokenSparkConf.DEFAULT_SASL_KERBEROS_SERVICE_NAME) + assert(clusterConfig.trustStoreLocation === None) + assert(clusterConfig.trustStorePassword === None) + assert(clusterConfig.keyStoreLocation === None) + assert(clusterConfig.keyStorePassword === None) + assert(clusterConfig.keyPassword === None) + assert(clusterConfig.tokenMechanism === KafkaTokenSparkConf.DEFAULT_SASL_TOKEN_MECHANISM) + } + } +} diff --git a/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala new file mode 100644 index 0000000000000..5496195b41490 --- /dev/null +++ b/external/kafka-0-10-token-provider/src/test/scala/org/apache/spark/kafka010/KafkaTokenUtilSuite.scala @@ -0,0 +1,231 @@ +/* + * 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.kafka010 + +import java.security.PrivilegedExceptionAction + +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.UserGroupInformation +import org.apache.kafka.clients.CommonClientConfigs +import org.apache.kafka.common.config.{SaslConfigs, SslConfigs} +import org.apache.kafka.common.security.auth.SecurityProtocol.{SASL_PLAINTEXT, SASL_SSL, SSL} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config._ + +class KafkaTokenUtilSuite extends SparkFunSuite with KafkaDelegationTokenTest { + private var sparkConf: SparkConf = null + + override def beforeEach(): Unit = { + super.beforeEach() + sparkConf = new SparkConf() + } + + test("checkProxyUser with proxy current user should throw exception") { + val realUser = UserGroupInformation.createUserForTesting("realUser", Array()) + UserGroupInformation.createProxyUserForTesting("proxyUser", realUser, Array()).doAs( + new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.checkProxyUser() + } + assert(thrown.getMessage contains + "Obtaining delegation token for proxy user is not yet supported.") + } + } + ) + } + + test("createAdminClientProperties with SASL_PLAINTEXT protocol should not include " + + "keystore and truststore config") { + val clusterConf = createClusterConf(identifier1, SASL_PLAINTEXT.name) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_PLAINTEXT.name) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEY_PASSWORD_CONFIG)) + } + + test("createAdminClientProperties with SASL_SSL protocol should include truststore config") { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG) + === trustStoreLocation) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) + === trustStorePassword) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG)) + assert(!adminClientProperties.containsKey(SslConfigs.SSL_KEY_PASSWORD_CONFIG)) + } + + test("createAdminClientProperties with SSL protocol should include keystore and truststore " + + "config") { + val clusterConf = createClusterConf(identifier1, SSL.name) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SSL.name) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG) + === trustStoreLocation) + assert(adminClientProperties.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG) + === trustStorePassword) + assert(adminClientProperties.get(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG) === keyStoreLocation) + assert(adminClientProperties.get(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG) === keyStorePassword) + assert(adminClientProperties.get(SslConfigs.SSL_KEY_PASSWORD_CONFIG) === keyPassword) + } + + test("createAdminClientProperties with global config should not set dynamic jaas config") { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + setGlobalKafkaClientConfig() + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(!adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + assert(!adminClientProperties.containsKey(SaslConfigs.SASL_JAAS_CONFIG)) + } + + test("createAdminClientProperties with keytab should set keytab dynamic jaas config") { + sparkConf.set(KEYTAB, keytab) + sparkConf.set(PRINCIPAL, principal) + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + val saslJaasConfig = adminClientProperties.getProperty(SaslConfigs.SASL_JAAS_CONFIG) + assert(saslJaasConfig.contains("Krb5LoginModule required")) + assert(saslJaasConfig.contains(s"debug=")) + assert(saslJaasConfig.contains("useKeyTab=true")) + assert(saslJaasConfig.contains(s"""keyTab="$keytab"""")) + assert(saslJaasConfig.contains(s"""principal="$principal"""")) + } + + test("createAdminClientProperties without keytab should set ticket cache dynamic jaas config") { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get(CommonClientConfigs.BOOTSTRAP_SERVERS_CONFIG) + === bootStrapServers) + assert(adminClientProperties.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG) + === SASL_SSL.name) + assert(adminClientProperties.containsKey(SaslConfigs.SASL_MECHANISM)) + val saslJaasConfig = adminClientProperties.getProperty(SaslConfigs.SASL_JAAS_CONFIG) + assert(saslJaasConfig.contains("Krb5LoginModule required")) + assert(saslJaasConfig.contains(s"debug=")) + assert(saslJaasConfig.contains("useTicketCache=true")) + } + + test("createAdminClientProperties with specified params should include it") { + val clusterConf = createClusterConf(identifier1, SASL_SSL.name, + Map("customKey" -> "customValue")) + + val adminClientProperties = KafkaTokenUtil.createAdminClientProperties(sparkConf, clusterConf) + + assert(adminClientProperties.get("customKey") === "customValue") + } + + test("isGlobalJaasConfigurationProvided without global config should return false") { + assert(!KafkaTokenUtil.isGlobalJaasConfigurationProvided) + } + + test("isGlobalJaasConfigurationProvided with global config should return false") { + setGlobalKafkaClientConfig() + + assert(KafkaTokenUtil.isGlobalJaasConfigurationProvided) + } + + test("findMatchingToken without token should return None") { + assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === None) + } + + test("findMatchingToken with non-matching tokens should return None") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", bootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.target.bootstrap.servers.regex", + nonMatchingTargetServersRegex) + sparkConf.set(s"spark.kafka.clusters.$identifier2.bootstrap.servers", bootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier2.target.bootstrap.servers.regex", + matchingTargetServersRegex) + addTokenToUGI(tokenService1) + addTokenToUGI(new Text("intentionally_garbage")) + + assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === None) + } + + test("findMatchingToken with one matching token should return cluster configuration") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", bootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.target.bootstrap.servers.regex", + matchingTargetServersRegex) + addTokenToUGI(tokenService1) + + assert(KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) === + Some(KafkaTokenSparkConf.getClusterConfig(sparkConf, identifier1))) + } + + test("findMatchingToken with multiple matching tokens should throw exception") { + sparkConf.set(s"spark.kafka.clusters.$identifier1.auth.bootstrap.servers", bootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier1.target.bootstrap.servers.regex", + matchingTargetServersRegex) + sparkConf.set(s"spark.kafka.clusters.$identifier2.auth.bootstrap.servers", bootStrapServers) + sparkConf.set(s"spark.kafka.clusters.$identifier2.target.bootstrap.servers.regex", + matchingTargetServersRegex) + addTokenToUGI(tokenService1) + addTokenToUGI(tokenService2) + + val thrown = intercept[IllegalArgumentException] { + KafkaTokenUtil.findMatchingToken(sparkConf, bootStrapServers) + } + assert(thrown.getMessage.contains("More than one delegation token matches")) + } + + test("getTokenJaasParams with token should return scram module") { + addTokenToUGI(tokenService1) + val clusterConf = createClusterConf(identifier1, SASL_SSL.name) + + val jaasParams = KafkaTokenUtil.getTokenJaasParams(clusterConf) + + assert(jaasParams.contains("ScramLoginModule required")) + assert(jaasParams.contains("tokenauth=true")) + assert(jaasParams.contains(tokenId)) + assert(jaasParams.contains(tokenPassword)) + } +} diff --git a/external/kafka-0-10/pom.xml b/external/kafka-0-10/pom.xml index f59f07265a0f4..397de87d3cdff 100644 --- a/external/kafka-0-10/pom.xml +++ b/external/kafka-0-10/pom.xml @@ -20,22 +20,25 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-streaming-kafka-0-10_2.11 + spark-streaming-kafka-0-10_2.12 streaming-kafka-0-10 - - 2.0.0 jar Spark Integration for Kafka 0.10 http://spark.apache.org/ + + org.apache.spark + spark-token-provider-kafka-0-10_${scala.binary.version} + ${project.version} + org.apache.spark spark-streaming_${scala.binary.version} @@ -74,6 +77,13 @@ + + + org.apache.zookeeper + zookeeper + 3.4.7 + test + net.sf.jopt-simple jopt-simple @@ -85,6 +95,11 @@ scalacheck_${scala.binary.version} test + + org.mockito + mockito-core + test + org.apache.spark spark-tags_${scala.binary.version} diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala index d2100fc5a4aba..3e32b592b3a3a 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/ConsumerStrategy.scala @@ -26,11 +26,10 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener import org.apache.kafka.common.TopicPartition -import org.apache.spark.annotation.Experimental import org.apache.spark.internal.Logging +import org.apache.spark.kafka010.KafkaConfigUpdater /** - * :: Experimental :: * Choice of how to create and configure underlying Kafka Consumers on driver and executors. * See [[ConsumerStrategies]] to obtain instances. * Kafka 0.10 consumers can require additional, sometimes complex, setup after object @@ -38,7 +37,6 @@ import org.apache.spark.internal.Logging * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ -@Experimental abstract class ConsumerStrategy[K, V] { /** * Kafka @@ -57,6 +55,15 @@ abstract class ConsumerStrategy[K, V] { * checkpoint. */ def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] + + /** + * Updates the parameters with security if needed. + * Added a function to hide internals and reduce code duplications because all strategy uses it. + */ + protected def setAuthenticationConfigIfNeeded(kafkaParams: ju.Map[String, Object]) = + KafkaConfigUpdater("source", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() } /** @@ -81,7 +88,8 @@ private case class Subscribe[K, V]( def executorKafkaParams: ju.Map[String, Object] = kafkaParams def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[K, V](updatedKafkaParams) consumer.subscribe(topics) val toSeek = if (currentOffsets.isEmpty) { offsets @@ -137,7 +145,8 @@ private case class SubscribePattern[K, V]( def executorKafkaParams: ju.Map[String, Object] = kafkaParams def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[K, V](updatedKafkaParams) consumer.subscribe(pattern, new NoOpConsumerRebalanceListener()) val toSeek = if (currentOffsets.isEmpty) { offsets @@ -189,7 +198,8 @@ private case class Assign[K, V]( def executorKafkaParams: ju.Map[String, Object] = kafkaParams def onStart(currentOffsets: ju.Map[TopicPartition, jl.Long]): Consumer[K, V] = { - val consumer = new KafkaConsumer[K, V](kafkaParams) + val updatedKafkaParams = setAuthenticationConfigIfNeeded(kafkaParams) + val consumer = new KafkaConsumer[K, V](updatedKafkaParams) consumer.assign(topicPartitions) val toSeek = if (currentOffsets.isEmpty) { offsets @@ -208,13 +218,10 @@ private case class Assign[K, V]( } /** - * :: Experimental :: - * object for obtaining instances of [[ConsumerStrategy]] + * Object for obtaining instances of [[ConsumerStrategy]] */ -@Experimental object ConsumerStrategies { /** - * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -227,7 +234,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def Subscribe[K, V]( topics: Iterable[jl.String], kafkaParams: collection.Map[String, Object], @@ -235,11 +241,10 @@ object ConsumerStrategies { new Subscribe[K, V]( new ju.ArrayList(topics.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) } /** - * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -249,7 +254,6 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def Subscribe[K, V]( topics: Iterable[jl.String], kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -260,7 +264,6 @@ object ConsumerStrategies { } /** - * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -273,7 +276,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def Subscribe[K, V]( topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object], @@ -282,7 +284,6 @@ object ConsumerStrategies { } /** - * :: Experimental :: * Subscribe to a collection of topics. * @param topics collection of topics to subscribe * @param kafkaParams Kafka @@ -292,14 +293,13 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def Subscribe[K, V]( topics: ju.Collection[jl.String], kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { new Subscribe[K, V](topics, kafkaParams, ju.Collections.emptyMap[TopicPartition, jl.Long]()) } - /** :: Experimental :: + /** * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -313,7 +313,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: collection.Map[String, Object], @@ -321,10 +320,10 @@ object ConsumerStrategies { new SubscribePattern[K, V]( pattern, new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) } - /** :: Experimental :: + /** * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -335,7 +334,6 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -345,7 +343,7 @@ object ConsumerStrategies { ju.Collections.emptyMap[TopicPartition, jl.Long]()) } - /** :: Experimental :: + /** * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -359,7 +357,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: ju.Map[String, Object], @@ -367,7 +364,7 @@ object ConsumerStrategies { new SubscribePattern[K, V](pattern, kafkaParams, offsets) } - /** :: Experimental :: + /** * Subscribe to all topics matching specified pattern to get dynamically assigned partitions. * The pattern matching will be done periodically against topics existing at the time of check. * @param pattern pattern to subscribe to @@ -378,7 +375,6 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def SubscribePattern[K, V]( pattern: ju.regex.Pattern, kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -389,7 +385,6 @@ object ConsumerStrategies { } /** - * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -402,7 +397,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def Assign[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object], @@ -410,11 +404,10 @@ object ConsumerStrategies { new Assign[K, V]( new ju.ArrayList(topicPartitions.asJavaCollection), new ju.HashMap[String, Object](kafkaParams.asJava), - new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(l => new jl.Long(l)).asJava)) + new ju.HashMap[TopicPartition, jl.Long](offsets.mapValues(jl.Long.valueOf).asJava)) } /** - * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -424,7 +417,6 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def Assign[K, V]( topicPartitions: Iterable[TopicPartition], kafkaParams: collection.Map[String, Object]): ConsumerStrategy[K, V] = { @@ -435,7 +427,6 @@ object ConsumerStrategies { } /** - * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -448,7 +439,6 @@ object ConsumerStrategies { * TopicPartition, the committed offset (if applicable) or kafka param * auto.offset.reset will be used. */ - @Experimental def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object], @@ -457,7 +447,6 @@ object ConsumerStrategies { } /** - * :: Experimental :: * Assign a fixed collection of TopicPartitions * @param topicPartitions collection of TopicPartitions to assign * @param kafkaParams Kafka @@ -467,7 +456,6 @@ object ConsumerStrategies { * Requires "bootstrap.servers" to be set * with Kafka broker(s) specified in host1:port1,host2:port2 form. */ - @Experimental def Assign[K, V]( topicPartitions: ju.Collection[TopicPartition], kafkaParams: ju.Map[String, Object]): ConsumerStrategy[K, V] = { diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala index ba4009ef08856..88d6d0eea5367 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/DirectKafkaInputDStream.scala @@ -70,7 +70,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( @transient private var kc: Consumer[K, V] = null def consumer(): Consumer[K, V] = this.synchronized { if (null == kc) { - kc = consumerStrategy.onStart(currentOffsets.mapValues(l => new java.lang.Long(l)).asJava) + kc = consumerStrategy.onStart(currentOffsets.mapValues(l => java.lang.Long.valueOf(l)).asJava) } kc } @@ -229,8 +229,7 @@ private[spark] class DirectKafkaInputDStream[K, V]( val fo = currentOffsets(tp) OffsetRange(tp.topic, tp.partition, fo, uo) } - val useConsumerCache = context.conf.getBoolean("spark.streaming.kafka.consumer.cache.enabled", - true) + val useConsumerCache = context.conf.get(CONSUMER_CACHE_ENABLED) val rdd = new KafkaRDD[K, V](context.sparkContext, executorKafkaParams, offsetRanges.toArray, getPreferredHosts, useConsumerCache) diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala index 68c5fe9ab066a..142e946188ace 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumer.scala @@ -19,11 +19,14 @@ package org.apache.spark.streaming.kafka010 import java.{util => ju} +import scala.collection.JavaConverters._ + import org.apache.kafka.clients.consumer.{ConsumerConfig, ConsumerRecord, KafkaConsumer} import org.apache.kafka.common.{KafkaException, TopicPartition} import org.apache.spark.TaskContext import org.apache.spark.internal.Logging +import org.apache.spark.kafka010.KafkaConfigUpdater private[kafka010] sealed trait KafkaDataConsumer[K, V] { /** @@ -109,7 +112,10 @@ private[kafka010] class InternalKafkaConsumer[K, V]( /** Create a KafkaConsumer to fetch records for `topicPartition` */ private def createConsumer: KafkaConsumer[K, V] = { - val c = new KafkaConsumer[K, V](kafkaParams) + val updatedKafkaParams = KafkaConfigUpdater("executor", kafkaParams.asScala.toMap) + .setAuthenticationConfigIfNeeded() + .build() + val c = new KafkaConsumer[K, V](updatedKafkaParams) val topics = ju.Arrays.asList(topicPartition) c.assign(topics) c diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala index 4513dca44c7c6..bd2e7e11b7383 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaRDD.scala @@ -24,6 +24,7 @@ import org.apache.kafka.common.TopicPartition import org.apache.spark.{Partition, SparkContext, TaskContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Network._ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.ExecutorCacheTaskLocation @@ -64,16 +65,11 @@ private[spark] class KafkaRDD[K, V]( " must be set to false for executor kafka params, else offsets may commit before processing") // TODO is it necessary to have separate configs for initial poll time vs ongoing poll time? - private val pollTimeout = conf.getLong("spark.streaming.kafka.consumer.poll.ms", - conf.getTimeAsSeconds("spark.network.timeout", "120s") * 1000L) - private val cacheInitialCapacity = - conf.getInt("spark.streaming.kafka.consumer.cache.initialCapacity", 16) - private val cacheMaxCapacity = - conf.getInt("spark.streaming.kafka.consumer.cache.maxCapacity", 64) - private val cacheLoadFactor = - conf.getDouble("spark.streaming.kafka.consumer.cache.loadFactor", 0.75).toFloat - private val compacted = - conf.getBoolean("spark.streaming.kafka.allowNonConsecutiveOffsets", false) + private val pollTimeout = conf.get(CONSUMER_POLL_MS).getOrElse(conf.get(NETWORK_TIMEOUT) * 1000L) + private val cacheInitialCapacity = conf.get(CONSUMER_CACHE_INITIAL_CAPACITY) + private val cacheMaxCapacity = conf.get(CONSUMER_CACHE_MAX_CAPACITY) + private val cacheLoadFactor = conf.get(CONSUMER_CACHE_LOAD_FACTOR).toFloat + private val compacted = conf.get(ALLOW_NON_CONSECUTIVE_OFFSETS) override def persist(newLevel: StorageLevel): this.type = { logError("Kafka ConsumerRecord is not serializable. " + diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala index e6bdef04512d5..2516b948f6650 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/KafkaUtils.scala @@ -23,7 +23,6 @@ import org.apache.kafka.clients.consumer._ import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkContext -import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{ JavaRDD, JavaSparkContext } import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD @@ -32,13 +31,10 @@ import org.apache.spark.streaming.api.java.{ JavaInputDStream, JavaStreamingCont import org.apache.spark.streaming.dstream._ /** - * :: Experimental :: * object for constructing Kafka streams and RDDs */ -@Experimental object KafkaUtils extends Logging { /** - * :: Experimental :: * Scala constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -52,7 +48,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createRDD[K, V]( sc: SparkContext, kafkaParams: ju.Map[String, Object], @@ -61,7 +56,7 @@ object KafkaUtils extends Logging { ): RDD[ConsumerRecord[K, V]] = { val preferredHosts = locationStrategy match { case PreferBrokers => - throw new AssertionError( + throw new IllegalArgumentException( "If you want to prefer brokers, you must provide a mapping using PreferFixed " + "A single KafkaRDD does not have a driver consumer and cannot look up brokers for you.") case PreferConsistent => ju.Collections.emptyMap[TopicPartition, String]() @@ -75,7 +70,6 @@ object KafkaUtils extends Logging { } /** - * :: Experimental :: * Java constructor for a batch-oriented interface for consuming from Kafka. * Starting and ending offsets are specified in advance, * so that you can control exactly-once semantics. @@ -89,7 +83,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createRDD[K, V]( jsc: JavaSparkContext, kafkaParams: ju.Map[String, Object], @@ -101,7 +94,6 @@ object KafkaUtils extends Logging { } /** - * :: Experimental :: * Scala constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number @@ -114,7 +106,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createDirectStream[K, V]( ssc: StreamingContext, locationStrategy: LocationStrategy, @@ -125,7 +116,6 @@ object KafkaUtils extends Logging { } /** - * :: Experimental :: * Scala constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -137,7 +127,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createDirectStream[K, V]( ssc: StreamingContext, locationStrategy: LocationStrategy, @@ -148,7 +137,6 @@ object KafkaUtils extends Logging { } /** - * :: Experimental :: * Java constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -158,7 +146,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createDirectStream[K, V]( jssc: JavaStreamingContext, locationStrategy: LocationStrategy, @@ -170,7 +157,6 @@ object KafkaUtils extends Logging { } /** - * :: Experimental :: * Java constructor for a DStream where * each given Kafka topic/partition corresponds to an RDD partition. * @param locationStrategy In most cases, pass in [[LocationStrategies.PreferConsistent]], @@ -182,7 +168,6 @@ object KafkaUtils extends Logging { * @tparam K type of Kafka message key * @tparam V type of Kafka message value */ - @Experimental def createDirectStream[K, V]( jssc: JavaStreamingContext, locationStrategy: LocationStrategy, diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala index c9a8a13f51c32..b4d9669f08387 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/LocationStrategy.scala @@ -23,18 +23,14 @@ import scala.collection.JavaConverters._ import org.apache.kafka.common.TopicPartition -import org.apache.spark.annotation.Experimental - /** - * :: Experimental :: * Choice of how to schedule consumers for a given TopicPartition on an executor. * See [[LocationStrategies]] to obtain instances. * Kafka 0.10 consumers prefetch messages, so it's important for performance * to keep cached consumers on appropriate executors, not recreate them for every partition. * Choice of location is only a preference, not an absolute; partitions may be scheduled elsewhere. */ -@Experimental sealed abstract class LocationStrategy private case object PreferBrokers extends LocationStrategy @@ -44,42 +40,32 @@ private case object PreferConsistent extends LocationStrategy private case class PreferFixed(hostMap: ju.Map[TopicPartition, String]) extends LocationStrategy /** - * :: Experimental :: object to obtain instances of [[LocationStrategy]] - * + * Object to obtain instances of [[LocationStrategy]] */ -@Experimental object LocationStrategies { /** - * :: Experimental :: * Use this only if your executors are on the same nodes as your Kafka brokers. */ - @Experimental def PreferBrokers: LocationStrategy = org.apache.spark.streaming.kafka010.PreferBrokers /** - * :: Experimental :: * Use this in most cases, it will consistently distribute partitions across all executors. */ - @Experimental def PreferConsistent: LocationStrategy = org.apache.spark.streaming.kafka010.PreferConsistent /** - * :: Experimental :: * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ - @Experimental def PreferFixed(hostMap: collection.Map[TopicPartition, String]): LocationStrategy = new PreferFixed(new ju.HashMap[TopicPartition, String](hostMap.asJava)) /** - * :: Experimental :: * Use this to place particular TopicPartitions on particular hosts if your load is uneven. * Any TopicPartition not specified in the map will use a consistent location. */ - @Experimental def PreferFixed(hostMap: ju.Map[TopicPartition, String]): LocationStrategy = new PreferFixed(hostMap) } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala index c66d3c9b8d229..077f02ed28ef7 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/OffsetRange.scala @@ -20,8 +20,6 @@ package org.apache.spark.streaming.kafka010 import org.apache.kafka.clients.consumer.OffsetCommitCallback import org.apache.kafka.common.TopicPartition -import org.apache.spark.annotation.Experimental - /** * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the * offset ranges in RDDs generated by the direct Kafka DStream (see @@ -38,7 +36,6 @@ trait HasOffsetRanges { } /** - * :: Experimental :: * Represents any object that can commit a collection of [[OffsetRange]]s. * The direct Kafka DStream implements this interface (see * [[KafkaUtils.createDirectStream]]). @@ -56,25 +53,20 @@ trait HasOffsetRanges { * }) * }}} */ -@Experimental trait CanCommitOffsets { /** - * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. */ - @Experimental def commitAsync(offsetRanges: Array[OffsetRange]): Unit /** - * :: Experimental :: * Queue up offset ranges for commit to Kafka at a future time. Threadsafe. * This is only needed if you intend to store offsets in Kafka, instead of your own store. * @param offsetRanges The maximum untilOffset for a given partition will be used at commit. * @param callback Only the most recently provided callback will be used at commit. */ - @Experimental def commitAsync(offsetRanges: Array[OffsetRange], callback: OffsetCommitCallback): Unit } diff --git a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala index 4017fdbcaf95e..b261500a454a3 100644 --- a/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala +++ b/external/kafka-0-10/src/main/scala/org/apache/spark/streaming/kafka010/PerPartitionConfig.scala @@ -20,14 +20,11 @@ package org.apache.spark.streaming.kafka010 import org.apache.kafka.common.TopicPartition import org.apache.spark.SparkConf -import org.apache.spark.annotation.Experimental /** - * :: Experimental :: * Interface for user-supplied configurations that can't otherwise be set via Spark properties, * because they need tweaking on a per-partition basis, */ -@Experimental abstract class PerPartitionConfig extends Serializable { /** * Maximum rate (number of records per second) at which data will be read @@ -42,8 +39,8 @@ abstract class PerPartitionConfig extends Serializable { */ private class DefaultPerPartitionConfig(conf: SparkConf) extends PerPartitionConfig { - val maxRate = conf.getLong("spark.streaming.kafka.maxRatePerPartition", 0) - val minRate = conf.getLong("spark.streaming.kafka.minRatePerPartition", 1) + val maxRate = conf.get(MAX_RATE_PER_PARTITION) + val minRate = conf.get(MIN_RATE_PER_PARTITION) def maxRatePerPartition(topicPartition: TopicPartition): Long = maxRate override def minRatePerPartition(topicPartition: TopicPartition): Long = minRate 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 09db6d6062d82..3d2921fa900a3 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 @@ -17,7 +17,52 @@ package org.apache.spark.streaming +import org.apache.spark.internal.config.ConfigBuilder + /** * Spark Integration for Kafka 0.10 */ -package object kafka010 //scalastyle:ignore +package object kafka010 { //scalastyle:ignore + + private[spark] val CONSUMER_CACHE_ENABLED = + ConfigBuilder("spark.streaming.kafka.consumer.cache.enabled") + .booleanConf + .createWithDefault(true) + + private[spark] val CONSUMER_POLL_MS = + ConfigBuilder("spark.streaming.kafka.consumer.poll.ms") + .longConf + .createOptional + + private[spark] val CONSUMER_CACHE_INITIAL_CAPACITY = + ConfigBuilder("spark.streaming.kafka.consumer.cache.initialCapacity") + .intConf + .createWithDefault(16) + + private[spark] val CONSUMER_CACHE_MAX_CAPACITY = + ConfigBuilder("spark.streaming.kafka.consumer.cache.maxCapacity") + .intConf + .createWithDefault(64) + + private[spark] val CONSUMER_CACHE_LOAD_FACTOR = + ConfigBuilder("spark.streaming.kafka.consumer.cache.loadFactor") + .doubleConf + .createWithDefault(0.75) + + private[spark] val MAX_RATE_PER_PARTITION = + ConfigBuilder("spark.streaming.kafka.maxRatePerPartition") + .longConf + .createWithDefault(0) + + private[spark] val MIN_RATE_PER_PARTITION = + ConfigBuilder("spark.streaming.kafka.minRatePerPartition") + .longConf + .createWithDefault(1) + + private[spark] val ALLOW_NON_CONSECUTIVE_OFFSETS = + ConfigBuilder("spark.streaming.kafka.allowNonConsecutiveOffsets") + .booleanConf + .createWithDefault(false) + +} + diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala index 1974bb1e12e15..4d3e476e7cc58 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/DirectKafkaStreamSuite.scala @@ -20,12 +20,12 @@ package org.apache.spark.streaming.kafka010 import java.io.File import java.lang.{ Long => JLong } import java.util.{ Arrays, HashMap => JHashMap, Map => JMap, UUID } +import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.ConcurrentLinkedQueue import java.util.concurrent.atomic.AtomicLong import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import scala.language.postfixOps import scala.util.Random import org.apache.kafka.clients.consumer._ @@ -52,19 +52,22 @@ class DirectKafkaStreamSuite val sparkConf = new SparkConf() .setMaster("local[4]") .setAppName(this.getClass.getSimpleName) + // Set a timeout of 10 seconds that's going to be used to fetch topics/partitions from kafka. + // Otherwise the poll timeout defaults to 2 minutes and causes test cases to run longer. + .set("spark.streaming.kafka.consumer.poll.ms", "10000") private var ssc: StreamingContext = _ private var testDir: File = _ private var kafkaTestUtils: KafkaTestUtils = _ - override def beforeAll { + override def beforeAll() { super.beforeAll() kafkaTestUtils = new KafkaTestUtils kafkaTestUtils.setup() } - override def afterAll { + override def afterAll() { try { if (kafkaTestUtils != null) { kafkaTestUtils.teardown() @@ -150,7 +153,7 @@ class DirectKafkaStreamSuite allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) } ssc.start() - eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { + eventually(timeout(100.seconds), interval(1.second)) { assert(allReceived.size === expectedTotal, "didn't get expected number of messages, messages:\n" + allReceived.asScala.mkString("\n")) @@ -216,7 +219,7 @@ class DirectKafkaStreamSuite allReceived.addAll(Arrays.asList(rdd.map(r => (r.key, r.value)).collect(): _*)) } ssc.start() - eventually(timeout(100000.milliseconds), interval(1000.milliseconds)) { + eventually(timeout(100.seconds), interval(1.second)) { assert(allReceived.size === expectedTotal, "didn't get expected number of messages, messages:\n" + allReceived.asScala.mkString("\n")) @@ -240,7 +243,7 @@ class DirectKafkaStreamSuite // Send some initial messages before starting context kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { + eventually(timeout(10.seconds), interval(20.milliseconds)) { assert(getLatestOffset() > 3) } val offsetBeforeStart = getLatestOffset() @@ -269,7 +272,7 @@ class DirectKafkaStreamSuite ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { collectedData.contains("b") } assert(!collectedData.contains("a")) @@ -292,7 +295,7 @@ class DirectKafkaStreamSuite // Send some initial messages before starting context kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { + eventually(timeout(10.seconds), interval(20.milliseconds)) { assert(getLatestOffset() >= 10) } val offsetBeforeStart = getLatestOffset() @@ -323,7 +326,7 @@ class DirectKafkaStreamSuite ssc.start() val newData = Map("b" -> 10) kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { collectedData.contains("b") } assert(!collectedData.contains("a")) @@ -372,7 +375,7 @@ class DirectKafkaStreamSuite sendData(i) } - eventually(timeout(20 seconds), interval(50 milliseconds)) { + eventually(timeout(20.seconds), interval(50.milliseconds)) { assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) } @@ -411,7 +414,7 @@ class DirectKafkaStreamSuite sendData(i) } - eventually(timeout(20 seconds), interval(50 milliseconds)) { + eventually(timeout(20.seconds), interval(50.milliseconds)) { assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) } ssc.stop() @@ -428,13 +431,13 @@ class DirectKafkaStreamSuite ) val collectedData = new ConcurrentLinkedQueue[String]() - val committed = new JHashMap[TopicPartition, OffsetAndMetadata]() + val committed = new ConcurrentHashMap[TopicPartition, OffsetAndMetadata]() // Send data to Kafka and wait for it to be received def sendDataAndWaitForReceive(data: Seq[Int]) { val strings = data.map { _.toString} kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { assert(strings.forall { collectedData.contains }) } } @@ -451,13 +454,12 @@ class DirectKafkaStreamSuite val data = rdd.map(_.value).collect() collectedData.addAll(Arrays.asList(data: _*)) kafkaStream.asInstanceOf[CanCommitOffsets] - .commitAsync(offsets, new OffsetCommitCallback() { - def onComplete(m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) { - if (null != e) { - logError("commit failed", e) - } else { - committed.putAll(m) - } + .commitAsync(offsets, (m: JMap[TopicPartition, OffsetAndMetadata], e: Exception) => { + if (null != e) { + logError("commit failed", e) + } else { + committed.putAll(m) + logDebug(s"commit succeeded: $m") } }) } @@ -467,8 +469,10 @@ class DirectKafkaStreamSuite for (i <- (1 to 10).grouped(4)) { sendDataAndWaitForReceive(i) } + eventually(timeout(10.seconds), interval(50.milliseconds)) { + assert(!committed.isEmpty) + } ssc.stop() - assert(! committed.isEmpty) val consumer = new KafkaConsumer[String, String](kafkaParams) consumer.subscribe(Arrays.asList(topic)) consumer.poll(0) @@ -611,7 +615,7 @@ class DirectKafkaStreamSuite estimator.updateRate(rate) // Set a new rate. // Expect blocks of data equal to "rate", scaled by the interval length in secs. val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) - eventually(timeout(5.seconds), interval(10 milliseconds)) { + eventually(timeout(5.seconds), interval(10.milliseconds)) { // Assert that rate estimator values are used to determine maxMessagesPerPartition. // Funky "-" in message makes the complete assertion message read better. assert(collectedData.asScala.exists(_.size == expectedSize), diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala index d934c64962adb..d8df5496f612d 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaDataConsumerSuite.scala @@ -25,11 +25,13 @@ import scala.util.Random import org.apache.kafka.clients.consumer.ConsumerConfig._ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.serialization.ByteArrayDeserializer +import org.mockito.Mockito.when import org.scalatest.BeforeAndAfterAll +import org.scalatest.mockito.MockitoSugar import org.apache.spark._ -class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll { +class KafkaDataConsumerSuite extends SparkFunSuite with MockitoSugar with BeforeAndAfterAll { private var testUtils: KafkaTestUtils = _ private val topic = "topic" + Random.nextInt() private val topicPartition = new TopicPartition(topic, 0) @@ -37,6 +39,11 @@ class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll { override def beforeAll(): Unit = { super.beforeAll() + val conf = new SparkConf() + val env = mock[SparkEnv] + SparkEnv.set(env) + when(env.conf).thenReturn(conf) + testUtils = new KafkaTestUtils testUtils.setup() KafkaDataConsumer.init(16, 64, 0.75f) @@ -47,6 +54,7 @@ class KafkaDataConsumerSuite extends SparkFunSuite with BeforeAndAfterAll { testUtils.teardown() testUtils = null } + SparkEnv.set(null) super.afterAll() } diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala index 561bca5f55370..47bc8fec2c80c 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaRDDSuite.scala @@ -41,6 +41,10 @@ class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { private val sparkConf = new SparkConf().setMaster("local[4]") .setAppName(this.getClass.getSimpleName) + // Set a timeout of 10 seconds that's going to be used to fetch topics/partitions from kafka. + // Othewise the poll timeout defaults to 2 minutes and causes test cases to run longer. + .set("spark.streaming.kafka.consumer.poll.ms", "10000") + private var sc: SparkContext = _ override def beforeAll { diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala index efcd5d6a5cdd3..5dec9709011e6 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/KafkaTestUtils.scala @@ -21,7 +21,7 @@ import java.io.{File, IOException} import java.lang.{Integer => JInt} import java.net.InetSocketAddress import java.util.{Map => JMap, Properties} -import java.util.concurrent.TimeoutException +import java.util.concurrent.{TimeoutException, TimeUnit} import scala.annotation.tailrec import scala.collection.JavaConverters._ @@ -265,14 +265,14 @@ private[kafka010] class KafkaTestUtils extends Logging { } } - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() @tailrec def tryAgain(attempt: Int): T = { makeAttempt() match { case Right(result) => result case Left(e) => - val duration = System.currentTimeMillis() - startTime - if (duration < timeout.milliseconds) { + val durationMs = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) + if (durationMs < timeout.milliseconds) { Thread.sleep(interval.milliseconds) } else { throw new TimeoutException(e.getMessage) @@ -286,7 +286,8 @@ private[kafka010] class KafkaTestUtils extends Logging { } private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { - def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { + def isPropagated = server.dataPlaneRequestProcessor.metadataCache + .getPartitionInfo(topic, partition) match { case Some(partitionState) => val leader = partitionState.basePartitionState.leader val isr = partitionState.basePartitionState.isr diff --git a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/mocks/MockTime.scala b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/mocks/MockTime.scala index 8a8646ee4eb94..dedd691cd1b23 100644 --- a/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/mocks/MockTime.scala +++ b/external/kafka-0-10/src/test/scala/org/apache/spark/streaming/kafka010/mocks/MockTime.scala @@ -17,7 +17,9 @@ package org.apache.spark.streaming.kafka010.mocks +import java.lang import java.util.concurrent._ +import java.util.function.Supplier import org.apache.kafka.common.utils.Time @@ -48,6 +50,9 @@ private[kafka010] class MockTime(@volatile private var currentMs: Long) extends scheduler.tick() } + override def waitObject(obj: Any, condition: Supplier[lang.Boolean], timeoutMs: Long): Unit = + throw new UnsupportedOperationException + override def toString(): String = s"MockTime($milliseconds)" } diff --git a/external/kafka-0-8-assembly/pom.xml b/external/kafka-0-8-assembly/pom.xml deleted file mode 100644 index 83edb11f296ab..0000000000000 --- a/external/kafka-0-8-assembly/pom.xml +++ /dev/null @@ -1,170 +0,0 @@ - - - - - 4.0.0 - - org.apache.spark - spark-parent_2.11 - 3.0.0-SNAPSHOT - ../../pom.xml - - - spark-streaming-kafka-0-8-assembly_2.11 - jar - Spark Project External Kafka Assembly - http://spark.apache.org/ - - - streaming-kafka-0-8-assembly - - - - - org.apache.spark - spark-streaming-kafka-0-8_${scala.binary.version} - ${project.version} - - - org.apache.spark - spark-streaming_${scala.binary.version} - ${project.version} - provided - - - - commons-codec - commons-codec - provided - - - commons-lang - commons-lang - provided - - - com.google.protobuf - protobuf-java - provided - - - org.lz4 - lz4-java - provided - - - org.apache.hadoop - hadoop-client - provided - - - org.apache.avro - avro-mapred - ${avro.mapred.classifier} - provided - - - org.apache.curator - curator-recipes - provided - - - org.apache.zookeeper - zookeeper - provided - - - log4j - log4j - provided - - - org.scala-lang - scala-library - provided - - - org.slf4j - slf4j-api - provided - - - org.slf4j - slf4j-log4j12 - provided - - - org.xerial.snappy - snappy-java - provided - - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-shade-plugin - - false - - - *:* - - - - - *:* - - META-INF/*.SF - META-INF/*.DSA - META-INF/*.RSA - - - - - - - package - - shade - - - - - - reference.conf - - - log4j.properties - - - - - - - - - - - - diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala deleted file mode 100644 index 89ccbe219cecd..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/Broker.scala +++ /dev/null @@ -1,68 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import org.apache.spark.annotation.Experimental - -/** - * Represents the host and port info for a Kafka broker. - * Differs from the Kafka project's internal kafka.cluster.Broker, which contains a server ID. - */ -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -final class Broker private( - /** Broker's hostname */ - val host: String, - /** Broker's port */ - val port: Int) extends Serializable { - override def equals(obj: Any): Boolean = obj match { - case that: Broker => - this.host == that.host && - this.port == that.port - case _ => false - } - - override def hashCode: Int = { - 41 * (41 + host.hashCode) + port - } - - override def toString(): String = { - s"Broker($host, $port)" - } -} - -/** - * :: Experimental :: - * Companion object that provides methods to create instances of [[Broker]]. - */ -@Experimental -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -object Broker { - def create(host: String, port: Int): Broker = - new Broker(host, port) - - def apply(host: String, port: Int): Broker = - new Broker(host, port) - - def unapply(broker: Broker): Option[(String, Int)] = { - if (broker == null) { - None - } else { - Some((broker.host, broker.port)) - } - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala deleted file mode 100644 index 2ec771e977147..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/DirectKafkaInputDStream.scala +++ /dev/null @@ -1,233 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.annotation.tailrec -import scala.collection.mutable -import scala.reflect.ClassTag - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.Decoder - -import org.apache.spark.SparkException -import org.apache.spark.internal.Logging -import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.kafka.KafkaCluster.LeaderOffset -import org.apache.spark.streaming.scheduler.{RateController, StreamInputInfo} -import org.apache.spark.streaming.scheduler.rate.RateEstimator - -/** - * A stream of [[KafkaRDD]] where - * each given Kafka topic/partition corresponds to an RDD partition. - * The spark configuration spark.streaming.kafka.maxRatePerPartition gives the maximum number - * of messages - * per second that each '''partition''' will accept. - * Starting offsets are specified in advance, - * and this DStream is not responsible for committing offsets, - * so that you can control exactly-once semantics. - * For an easy interface to Kafka-managed offsets, - * see [[KafkaCluster]] - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler function for translating each message into the desired type - */ -private[streaming] -class DirectKafkaInputDStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[K]: ClassTag, - T <: Decoder[V]: ClassTag, - R: ClassTag]( - _ssc: StreamingContext, - val kafkaParams: Map[String, String], - val fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ) extends InputDStream[R](_ssc) with Logging { - val maxRetries = context.sparkContext.getConf.getInt( - "spark.streaming.kafka.maxRetries", 1) - - private[streaming] override def name: String = s"Kafka direct stream [$id]" - - protected[streaming] override val checkpointData = - new DirectKafkaInputDStreamCheckpointData - - - /** - * Asynchronously maintains & sends new rate limits to the receiver through the receiver tracker. - */ - override protected[streaming] val rateController: Option[RateController] = { - if (RateController.isBackPressureEnabled(ssc.conf)) { - Some(new DirectKafkaRateController(id, - RateEstimator.create(ssc.conf, context.graph.batchDuration))) - } else { - None - } - } - - protected val kc = new KafkaCluster(kafkaParams) - - private val maxRateLimitPerPartition: Long = context.sparkContext.getConf.getLong( - "spark.streaming.kafka.maxRatePerPartition", 0) - - private val initialRate = context.sparkContext.getConf.getLong( - "spark.streaming.backpressure.initialRate", 0) - - protected[streaming] def maxMessagesPerPartition( - offsets: Map[TopicAndPartition, Long]): Option[Map[TopicAndPartition, Long]] = { - - val estimatedRateLimit = rateController.map { x => { - val lr = x.getLatestRate() - if (lr > 0) lr else initialRate - }} - - // calculate a per-partition rate limit based on current lag - val effectiveRateLimitPerPartition = estimatedRateLimit.filter(_ > 0) match { - case Some(rate) => - val lagPerPartition = offsets.map { case (tp, offset) => - tp -> Math.max(offset - currentOffsets(tp), 0) - } - val totalLag = lagPerPartition.values.sum - - lagPerPartition.map { case (tp, lag) => - val backpressureRate = lag / totalLag.toDouble * rate - tp -> (if (maxRateLimitPerPartition > 0) { - Math.min(backpressureRate, maxRateLimitPerPartition)} else backpressureRate) - } - case None => offsets.map { case (tp, offset) => tp -> maxRateLimitPerPartition.toDouble } - } - - if (effectiveRateLimitPerPartition.values.sum > 0) { - val secsPerBatch = context.graph.batchDuration.milliseconds.toDouble / 1000 - Some(effectiveRateLimitPerPartition.map { - case (tp, limit) => tp -> Math.max((secsPerBatch * limit).toLong, 1L) - }) - } else { - None - } - } - - protected var currentOffsets = fromOffsets - - @tailrec - protected final def latestLeaderOffsets(retries: Int): Map[TopicAndPartition, LeaderOffset] = { - val o = kc.getLatestLeaderOffsets(currentOffsets.keySet) - // Either.fold would confuse @tailrec, do it manually - if (o.isLeft) { - val err = o.left.get.toString - if (retries <= 0) { - throw new SparkException(err) - } else { - logError(err) - Thread.sleep(kc.config.refreshLeaderBackoffMs) - latestLeaderOffsets(retries - 1) - } - } else { - o.right.get - } - } - - // limits the maximum number of messages per partition - protected def clamp( - leaderOffsets: Map[TopicAndPartition, LeaderOffset]): Map[TopicAndPartition, LeaderOffset] = { - val offsets = leaderOffsets.mapValues(lo => lo.offset) - - maxMessagesPerPartition(offsets).map { mmp => - mmp.map { case (tp, messages) => - val lo = leaderOffsets(tp) - tp -> lo.copy(offset = Math.min(currentOffsets(tp) + messages, lo.offset)) - } - }.getOrElse(leaderOffsets) - } - - override def compute(validTime: Time): Option[KafkaRDD[K, V, U, T, R]] = { - val untilOffsets = clamp(latestLeaderOffsets(maxRetries)) - val rdd = KafkaRDD[K, V, U, T, R]( - context.sparkContext, kafkaParams, currentOffsets, untilOffsets, messageHandler) - - // Report the record number and metadata of this batch interval to InputInfoTracker. - val offsetRanges = currentOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo.offset) - } - val description = offsetRanges.filter { offsetRange => - // Don't display empty ranges. - offsetRange.fromOffset != offsetRange.untilOffset - }.map { offsetRange => - s"topic: ${offsetRange.topic}\tpartition: ${offsetRange.partition}\t" + - s"offsets: ${offsetRange.fromOffset} to ${offsetRange.untilOffset}" - }.mkString("\n") - // Copy offsetRanges to immutable.List to prevent from being modified by the user - val metadata = Map( - "offsets" -> offsetRanges.toList, - StreamInputInfo.METADATA_KEY_DESCRIPTION -> description) - val inputInfo = StreamInputInfo(id, rdd.count, metadata) - ssc.scheduler.inputInfoTracker.reportInfo(validTime, inputInfo) - - currentOffsets = untilOffsets.map(kv => kv._1 -> kv._2.offset) - Some(rdd) - } - - override def start(): Unit = { - } - - def stop(): Unit = { - } - - private[streaming] - class DirectKafkaInputDStreamCheckpointData extends DStreamCheckpointData(this) { - def batchForTime: mutable.HashMap[Time, Array[(String, Int, Long, Long)]] = { - data.asInstanceOf[mutable.HashMap[Time, Array[OffsetRange.OffsetRangeTuple]]] - } - - override def update(time: Time): Unit = { - batchForTime.clear() - generatedRDDs.foreach { kv => - val a = kv._2.asInstanceOf[KafkaRDD[K, V, U, T, R]].offsetRanges.map(_.toTuple).toArray - batchForTime += kv._1 -> a - } - } - - override def cleanup(time: Time): Unit = { } - - override def restore(): Unit = { - // this is assuming that the topics don't change during execution, which is true currently - val topics = fromOffsets.keySet - val leaders = KafkaCluster.checkErrors(kc.findLeaders(topics)) - - batchForTime.toSeq.sortBy(_._1)(Time.ordering).foreach { case (t, b) => - logInfo(s"Restoring KafkaRDD for time $t ${b.mkString("[", ", ", "]")}") - generatedRDDs += t -> new KafkaRDD[K, V, U, T, R]( - context.sparkContext, kafkaParams, b.map(OffsetRange(_)), leaders, messageHandler) - } - } - } - - /** - * A RateController to retrieve the rate from RateEstimator. - */ - private[streaming] class DirectKafkaRateController(id: Int, estimator: RateEstimator) - extends RateController(id, estimator) { - override def publish(rate: Long): Unit = () - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala deleted file mode 100644 index 570affab11853..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaCluster.scala +++ /dev/null @@ -1,439 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.util.Properties - -import scala.collection.JavaConverters._ -import scala.collection.mutable.ArrayBuffer -import scala.util.Random -import scala.util.control.NonFatal - -import kafka.api._ -import kafka.common.{ErrorMapping, OffsetAndMetadata, OffsetMetadataAndError, TopicAndPartition} -import kafka.consumer.{ConsumerConfig, SimpleConsumer} - -import org.apache.spark.SparkException -import org.apache.spark.annotation.DeveloperApi - -/** - * :: DeveloperApi :: - * Convenience methods for interacting with a Kafka cluster. - * See - * A Guide To The Kafka Protocol for more details on individual api calls. - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form - */ -@DeveloperApi -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -class KafkaCluster(val kafkaParams: Map[String, String]) extends Serializable { - import KafkaCluster.{Err, LeaderOffset, SimpleConsumerConfig} - - // ConsumerConfig isn't serializable - @transient private var _config: SimpleConsumerConfig = null - - def config: SimpleConsumerConfig = this.synchronized { - if (_config == null) { - _config = SimpleConsumerConfig(kafkaParams) - } - _config - } - - def connect(host: String, port: Int): SimpleConsumer = - new SimpleConsumer(host, port, config.socketTimeoutMs, - config.socketReceiveBufferBytes, config.clientId) - - def connectLeader(topic: String, partition: Int): Either[Err, SimpleConsumer] = - findLeader(topic, partition).right.map(hp => connect(hp._1, hp._2)) - - // Metadata api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-MetadataAPI - // scalastyle:on - - def findLeader(topic: String, partition: Int): Either[Err, (String, Int)] = { - val req = TopicMetadataRequest(TopicMetadataRequest.CurrentVersion, - 0, config.clientId, Seq(topic)) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp: TopicMetadataResponse = consumer.send(req) - resp.topicsMetadata.find(_.topic == topic).flatMap { tm: TopicMetadata => - tm.partitionsMetadata.find(_.partitionId == partition) - }.foreach { pm: PartitionMetadata => - pm.leader.foreach { leader => - return Right((leader.host, leader.port)) - } - } - } - Left(errs) - } - - def findLeaders( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, (String, Int)]] = { - val topics = topicAndPartitions.map(_.topic) - val response = getPartitionMetadata(topics).right - val answer = response.flatMap { tms: Set[TopicMetadata] => - val leaderMap = tms.flatMap { tm: TopicMetadata => - tm.partitionsMetadata.flatMap { pm: PartitionMetadata => - val tp = TopicAndPartition(tm.topic, pm.partitionId) - if (topicAndPartitions(tp)) { - pm.leader.map { l => - tp -> (l.host -> l.port) - } - } else { - None - } - } - }.toMap - - if (leaderMap.keys.size == topicAndPartitions.size) { - Right(leaderMap) - } else { - val missing = topicAndPartitions.diff(leaderMap.keySet) - val err = new Err - err += new SparkException(s"Couldn't find leaders for ${missing}") - Left(err) - } - } - answer - } - - def getPartitions(topics: Set[String]): Either[Err, Set[TopicAndPartition]] = { - getPartitionMetadata(topics).right.map { r => - r.flatMap { tm: TopicMetadata => - tm.partitionsMetadata.map { pm: PartitionMetadata => - TopicAndPartition(tm.topic, pm.partitionId) - } - } - } - } - - def getPartitionMetadata(topics: Set[String]): Either[Err, Set[TopicMetadata]] = { - val req = TopicMetadataRequest( - TopicMetadataRequest.CurrentVersion, 0, config.clientId, topics.toSeq) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp: TopicMetadataResponse = consumer.send(req) - val respErrs = resp.topicsMetadata.filter(m => m.errorCode != ErrorMapping.NoError) - - if (respErrs.isEmpty) { - return Right(resp.topicsMetadata.toSet) - } else { - respErrs.foreach { m => - val cause = ErrorMapping.exceptionFor(m.errorCode) - val msg = s"Error getting partition metadata for '${m.topic}'. Does the topic exist?" - errs += new SparkException(msg, cause) - } - } - } - Left(errs) - } - - // Leader offset api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetAPI - // scalastyle:on - - def getLatestLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = - getLeaderOffsets(topicAndPartitions, OffsetRequest.LatestTime) - - def getEarliestLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = - getLeaderOffsets(topicAndPartitions, OffsetRequest.EarliestTime) - - def getLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition], - before: Long - ): Either[Err, Map[TopicAndPartition, LeaderOffset]] = { - getLeaderOffsets(topicAndPartitions, before, 1).right.map { r => - r.map { kv => - // mapValues isn't serializable, see SI-7005 - kv._1 -> kv._2.head - } - } - } - - private def flip[K, V](m: Map[K, V]): Map[V, Seq[K]] = - m.groupBy(_._2).map { kv => - kv._1 -> kv._2.keys.toSeq - } - - def getLeaderOffsets( - topicAndPartitions: Set[TopicAndPartition], - before: Long, - maxNumOffsets: Int - ): Either[Err, Map[TopicAndPartition, Seq[LeaderOffset]]] = { - findLeaders(topicAndPartitions).right.flatMap { tpToLeader => - val leaderToTp: Map[(String, Int), Seq[TopicAndPartition]] = flip(tpToLeader) - val leaders = leaderToTp.keys - var result = Map[TopicAndPartition, Seq[LeaderOffset]]() - val errs = new Err - withBrokers(leaders, errs) { consumer => - val partitionsToGetOffsets: Seq[TopicAndPartition] = - leaderToTp((consumer.host, consumer.port)) - val reqMap = partitionsToGetOffsets.map { tp: TopicAndPartition => - tp -> PartitionOffsetRequestInfo(before, maxNumOffsets) - }.toMap - val req = OffsetRequest(reqMap) - val resp = consumer.getOffsetsBefore(req) - val respMap = resp.partitionErrorAndOffsets - partitionsToGetOffsets.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { por: PartitionOffsetsResponse => - if (por.error == ErrorMapping.NoError) { - if (por.offsets.nonEmpty) { - result += tp -> por.offsets.map { off => - LeaderOffset(consumer.host, consumer.port, off) - } - } else { - errs += new SparkException( - s"Empty offsets for ${tp}, is ${before} before log beginning?") - } - } else { - errs += ErrorMapping.exceptionFor(por.error) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs += new SparkException(s"Couldn't find leader offsets for ${missing}") - Left(errs) - } - } - - // Consumer offset api - // scalastyle:off - // https://cwiki.apache.org/confluence/display/KAFKA/A+Guide+To+The+Kafka+Protocol#AGuideToTheKafkaProtocol-OffsetCommit/FetchAPI - // scalastyle:on - - // this 0 here indicates api version, in this case the original ZK backed api. - private def defaultConsumerApiVersion: Short = 0 - - /** - * Requires Kafka 0.8.1.1 or later. - * Defaults to the original ZooKeeper backed API version. - */ - def getConsumerOffsets( - groupId: String, - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, Long]] = - getConsumerOffsets(groupId, topicAndPartitions, defaultConsumerApiVersion) - - def getConsumerOffsets( - groupId: String, - topicAndPartitions: Set[TopicAndPartition], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Long]] = { - getConsumerOffsetMetadata(groupId, topicAndPartitions, consumerApiVersion).right.map { r => - r.map { kv => - kv._1 -> kv._2.offset - } - } - } - - /** - * Requires Kafka 0.8.1.1 or later. - * Defaults to the original ZooKeeper backed API version. - */ - def getConsumerOffsetMetadata( - groupId: String, - topicAndPartitions: Set[TopicAndPartition] - ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = - getConsumerOffsetMetadata(groupId, topicAndPartitions, defaultConsumerApiVersion) - - def getConsumerOffsetMetadata( - groupId: String, - topicAndPartitions: Set[TopicAndPartition], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, OffsetMetadataAndError]] = { - var result = Map[TopicAndPartition, OffsetMetadataAndError]() - val req = OffsetFetchRequest(groupId, topicAndPartitions.toSeq, consumerApiVersion) - val errs = new Err - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp = consumer.fetchOffsets(req) - val respMap = resp.requestInfo - val needed = topicAndPartitions.diff(result.keySet) - needed.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { ome: OffsetMetadataAndError => - if (ome.error == ErrorMapping.NoError) { - result += tp -> ome - } else { - errs += ErrorMapping.exceptionFor(ome.error) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs += new SparkException(s"Couldn't find consumer offsets for ${missing}") - Left(errs) - } - - /** - * Requires Kafka 0.8.1.1 or later. - * Defaults to the original ZooKeeper backed API version. - */ - def setConsumerOffsets( - groupId: String, - offsets: Map[TopicAndPartition, Long] - ): Either[Err, Map[TopicAndPartition, Short]] = - setConsumerOffsets(groupId, offsets, defaultConsumerApiVersion) - - def setConsumerOffsets( - groupId: String, - offsets: Map[TopicAndPartition, Long], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Short]] = { - val meta = offsets.map { kv => - kv._1 -> OffsetAndMetadata(kv._2) - } - setConsumerOffsetMetadata(groupId, meta, consumerApiVersion) - } - - /** - * Requires Kafka 0.8.1.1 or later. - * Defaults to the original ZooKeeper backed API version. - */ - def setConsumerOffsetMetadata( - groupId: String, - metadata: Map[TopicAndPartition, OffsetAndMetadata] - ): Either[Err, Map[TopicAndPartition, Short]] = - setConsumerOffsetMetadata(groupId, metadata, defaultConsumerApiVersion) - - def setConsumerOffsetMetadata( - groupId: String, - metadata: Map[TopicAndPartition, OffsetAndMetadata], - consumerApiVersion: Short - ): Either[Err, Map[TopicAndPartition, Short]] = { - var result = Map[TopicAndPartition, Short]() - val req = OffsetCommitRequest(groupId, metadata, consumerApiVersion) - val errs = new Err - val topicAndPartitions = metadata.keySet - withBrokers(Random.shuffle(config.seedBrokers), errs) { consumer => - val resp = consumer.commitOffsets(req) - val respMap = resp.commitStatus - val needed = topicAndPartitions.diff(result.keySet) - needed.foreach { tp: TopicAndPartition => - respMap.get(tp).foreach { err: Short => - if (err == ErrorMapping.NoError) { - result += tp -> err - } else { - errs += ErrorMapping.exceptionFor(err) - } - } - } - if (result.keys.size == topicAndPartitions.size) { - return Right(result) - } - } - val missing = topicAndPartitions.diff(result.keySet) - errs += new SparkException(s"Couldn't set offsets for ${missing}") - Left(errs) - } - - // Try a call against potentially multiple brokers, accumulating errors - private def withBrokers(brokers: Iterable[(String, Int)], errs: Err) - (fn: SimpleConsumer => Any): Unit = { - brokers.foreach { hp => - var consumer: SimpleConsumer = null - try { - consumer = connect(hp._1, hp._2) - fn(consumer) - } catch { - case NonFatal(e) => - errs += e - } finally { - if (consumer != null) { - consumer.close() - } - } - } - } -} - -@DeveloperApi -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -object KafkaCluster { - type Err = ArrayBuffer[Throwable] - - /** If the result is right, return it, otherwise throw SparkException */ - def checkErrors[T](result: Either[Err, T]): T = { - result.fold( - errs => throw new SparkException(errs.mkString("\n")), - ok => ok - ) - } - - case class LeaderOffset(host: String, port: Int, offset: Long) - - /** - * High-level kafka consumers connect to ZK. ConsumerConfig assumes this use case. - * Simple consumers connect directly to brokers, but need many of the same configs. - * This subclass won't warn about missing ZK params, or presence of broker params. - */ - class SimpleConsumerConfig private(brokers: String, originalProps: Properties) - extends ConsumerConfig(originalProps) { - val seedBrokers: Array[(String, Int)] = brokers.split(",").map { hp => - val hpa = hp.split(":") - if (hpa.size == 1) { - throw new SparkException(s"Broker not in the correct format of : [$brokers]") - } - (hpa(0), hpa(1).toInt) - } - } - - object SimpleConsumerConfig { - /** - * Make a consumer config without requiring group.id or zookeeper.connect, - * since communicating with brokers also needs common settings such as timeout - */ - def apply(kafkaParams: Map[String, String]): SimpleConsumerConfig = { - // These keys are from other pre-existing kafka configs for specifying brokers, accept either - val brokers = kafkaParams.get("metadata.broker.list") - .orElse(kafkaParams.get("bootstrap.servers")) - .getOrElse(throw new SparkException( - "Must specify metadata.broker.list or bootstrap.servers")) - - val props = new Properties() - kafkaParams.foreach { case (key, value) => - // prevent warnings on parameters ConsumerConfig doesn't know about - if (key != "metadata.broker.list" && key != "bootstrap.servers") { - props.put(key, value) - } - } - - Seq("zookeeper.connect", "group.id").foreach { s => - if (!props.containsKey(s)) { - props.setProperty(s, "") - } - } - - new SimpleConsumerConfig(brokers, props) - } - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala deleted file mode 100644 index 7ff3a98ca52cd..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.util.Properties - -import scala.collection.Map -import scala.reflect.{classTag, ClassTag} - -import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties - -import org.apache.spark.internal.Logging -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receiver.Receiver -import org.apache.spark.util.ThreadUtils - -/** - * Input stream that pulls messages from a Kafka Broker. - * - * @param kafkaParams Map of kafka configuration parameters. - * See: http://kafka.apache.org/configuration.html - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - */ -private[streaming] -class KafkaInputDStream[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag]( - _ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - useReliableReceiver: Boolean, - storageLevel: StorageLevel - ) extends ReceiverInputDStream[(K, V)](_ssc) with Logging { - - def getReceiver(): Receiver[(K, V)] = { - if (!useReliableReceiver) { - new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - } else { - new ReliableKafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - } - } -} - -private[streaming] -class KafkaReceiver[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ) extends Receiver[(K, V)](storageLevel) with Logging { - - // Connection to Kafka - var consumerConnector: ConsumerConnector = null - - def onStop() { - if (consumerConnector != null) { - consumerConnector.shutdown() - consumerConnector = null - } - } - - def onStart() { - - logInfo("Starting Kafka Consumer Stream with group: " + kafkaParams("group.id")) - - // Kafka connection properties - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - - val zkConnect = kafkaParams("zookeeper.connect") - // Create the connection to the cluster - logInfo("Connecting to Zookeeper: " + zkConnect) - val consumerConfig = new ConsumerConfig(props) - consumerConnector = Consumer.create(consumerConfig) - logInfo("Connected to " + zkConnect) - - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[V]] - - // Create threads for each topic/message Stream we are listening - val topicMessageStreams = consumerConnector.createMessageStreams( - topics, keyDecoder, valueDecoder) - - val executorPool = - ThreadUtils.newDaemonFixedThreadPool(topics.values.sum, "KafkaMessageHandler") - try { - // Start the messages handler for each partition - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => executorPool.submit(new MessageHandler(stream)) } - } - } finally { - executorPool.shutdown() // Just causes threads to terminate after work is done - } - } - - // Handles Kafka messages - private class MessageHandler(stream: KafkaStream[K, V]) - extends Runnable { - def run() { - logInfo("Starting MessageHandler.") - try { - val streamIterator = stream.iterator() - while (streamIterator.hasNext()) { - val msgAndMetadata = streamIterator.next() - store((msgAndMetadata.key, msgAndMetadata.message)) - } - } catch { - case e: Throwable => reportError("Error handling message; exiting", e) - } - } - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala deleted file mode 100644 index 791cf0efaf888..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDD.scala +++ /dev/null @@ -1,273 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.collection.mutable.ArrayBuffer -import scala.reflect.{classTag, ClassTag} - -import kafka.api.{FetchRequestBuilder, FetchResponse} -import kafka.common.{ErrorMapping, TopicAndPartition} -import kafka.consumer.SimpleConsumer -import kafka.message.{MessageAndMetadata, MessageAndOffset} -import kafka.serializer.Decoder -import kafka.utils.VerifiableProperties - -import org.apache.spark.{Partition, SparkContext, SparkException, TaskContext} -import org.apache.spark.internal.Logging -import org.apache.spark.partial.{BoundedDouble, PartialResult} -import org.apache.spark.rdd.RDD -import org.apache.spark.util.NextIterator - -/** - * A batch-oriented interface for consuming from Kafka. - * Starting and ending offsets are specified in advance, - * so that you can control exactly-once semantics. - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" to be set - * with Kafka broker(s) specified in host1:port1,host2:port2 form. - * @param offsetRanges offset ranges that define the Kafka data belonging to this RDD - * @param messageHandler function for translating each message into the desired type - */ -private[kafka] -class KafkaRDD[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, - R: ClassTag] private[spark] ( - sc: SparkContext, - kafkaParams: Map[String, String], - val offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, (String, Int)], - messageHandler: MessageAndMetadata[K, V] => R - ) extends RDD[R](sc, Nil) with Logging with HasOffsetRanges { - override def getPartitions: Array[Partition] = { - offsetRanges.zipWithIndex.map { case (o, i) => - val (host, port) = leaders(TopicAndPartition(o.topic, o.partition)) - new KafkaRDDPartition(i, o.topic, o.partition, o.fromOffset, o.untilOffset, host, port) - }.toArray - } - - override def count(): Long = offsetRanges.map(_.count).sum - - override def countApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[BoundedDouble] = { - val c = count - new PartialResult(new BoundedDouble(c, 1.0, c, c), true) - } - - override def isEmpty(): Boolean = count == 0L - - override def take(num: Int): Array[R] = { - val nonEmptyPartitions = this.partitions - .map(_.asInstanceOf[KafkaRDDPartition]) - .filter(_.count > 0) - - if (num < 1 || nonEmptyPartitions.isEmpty) { - return new Array[R](0) - } - - // Determine in advance how many messages need to be taken from each partition - val parts = nonEmptyPartitions.foldLeft(Map[Int, Int]()) { (result, part) => - val remain = num - result.values.sum - if (remain > 0) { - val taken = Math.min(remain, part.count) - result + (part.index -> taken.toInt) - } else { - result - } - } - - val buf = new ArrayBuffer[R] - val res = context.runJob( - this, - (tc: TaskContext, it: Iterator[R]) => it.take(parts(tc.partitionId)).toArray, - parts.keys.toArray) - res.foreach(buf ++= _) - buf.toArray - } - - override def getPreferredLocations(thePart: Partition): Seq[String] = { - val part = thePart.asInstanceOf[KafkaRDDPartition] - // TODO is additional hostname resolution necessary here - Seq(part.host) - } - - private def errBeginAfterEnd(part: KafkaRDDPartition): String = - s"Beginning offset ${part.fromOffset} is after the ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition}. " + - "You either provided an invalid fromOffset, or the Kafka topic has been damaged" - - private def errRanOutBeforeEnd(part: KafkaRDDPartition): String = - s"Ran out of messages before reaching ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + - " This should not happen, and indicates that messages may have been lost" - - private def errOvershotEnd(itemOffset: Long, part: KafkaRDDPartition): String = - s"Got ${itemOffset} > ending offset ${part.untilOffset} " + - s"for topic ${part.topic} partition ${part.partition} start ${part.fromOffset}." + - " This should not happen, and indicates a message may have been skipped" - - override def compute(thePart: Partition, context: TaskContext): Iterator[R] = { - val part = thePart.asInstanceOf[KafkaRDDPartition] - assert(part.fromOffset <= part.untilOffset, errBeginAfterEnd(part)) - if (part.fromOffset == part.untilOffset) { - logInfo(s"Beginning offset ${part.fromOffset} is the same as ending offset " + - s"skipping ${part.topic} ${part.partition}") - Iterator.empty - } else { - new KafkaRDDIterator(part, context) - } - } - - /** - * An iterator that fetches messages directly from Kafka for the offsets in partition. - */ - private class KafkaRDDIterator( - part: KafkaRDDPartition, - context: TaskContext) extends NextIterator[R] { - - context.addTaskCompletionListener{ context => closeIfNeeded() } - - logInfo(s"Computing topic ${part.topic}, partition ${part.partition} " + - s"offsets ${part.fromOffset} -> ${part.untilOffset}") - - val kc = new KafkaCluster(kafkaParams) - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[K]] - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(kc.config.props) - .asInstanceOf[Decoder[V]] - val consumer = connectLeader - var requestOffset = part.fromOffset - var iter: Iterator[MessageAndOffset] = null - - // The idea is to use the provided preferred host, except on task retry attempts, - // to minimize number of kafka metadata requests - private def connectLeader: SimpleConsumer = { - if (context.attemptNumber > 0) { - kc.connectLeader(part.topic, part.partition).fold( - errs => throw new SparkException( - s"Couldn't connect to leader for topic ${part.topic} ${part.partition}: " + - errs.mkString("\n")), - consumer => consumer - ) - } else { - kc.connect(part.host, part.port) - } - } - - private def handleFetchErr(resp: FetchResponse) { - if (resp.hasError) { - val err = resp.errorCode(part.topic, part.partition) - if (err == ErrorMapping.LeaderNotAvailableCode || - err == ErrorMapping.NotLeaderForPartitionCode) { - logError(s"Lost leader for topic ${part.topic} partition ${part.partition}, " + - s" sleeping for ${kc.config.refreshLeaderBackoffMs}ms") - Thread.sleep(kc.config.refreshLeaderBackoffMs) - } - // Let normal rdd retry sort out reconnect attempts - throw ErrorMapping.exceptionFor(err) - } - } - - private def fetchBatch: Iterator[MessageAndOffset] = { - val req = new FetchRequestBuilder() - .clientId(consumer.clientId) - .addFetch(part.topic, part.partition, requestOffset, kc.config.fetchMessageMaxBytes) - .build() - val resp = consumer.fetch(req) - handleFetchErr(resp) - // kafka may return a batch that starts before the requested offset - resp.messageSet(part.topic, part.partition) - .iterator - .dropWhile(_.offset < requestOffset) - } - - override def close(): Unit = { - if (consumer != null) { - consumer.close() - } - } - - override def getNext(): R = { - if (iter == null || !iter.hasNext) { - iter = fetchBatch - } - if (!iter.hasNext) { - assert(requestOffset == part.untilOffset, errRanOutBeforeEnd(part)) - finished = true - null.asInstanceOf[R] - } else { - val item = iter.next() - if (item.offset >= part.untilOffset) { - assert(item.offset == part.untilOffset, errOvershotEnd(item.offset, part)) - finished = true - null.asInstanceOf[R] - } else { - requestOffset = item.nextOffset - messageHandler(new MessageAndMetadata( - part.topic, part.partition, item.message, item.offset, keyDecoder, valueDecoder)) - } - } - } - } -} - -private[kafka] -object KafkaRDD { - import KafkaCluster.LeaderOffset - - /** - * @param kafkaParams Kafka - * configuration parameters. - * Requires "metadata.broker.list" or "bootstrap.servers" to be set with Kafka broker(s), - * NOT zookeeper servers, specified in host1:port1,host2:port2 form. - * @param fromOffsets per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the batch - * @param untilOffsets per-topic/partition Kafka offsets defining the (exclusive) - * ending point of the batch - * @param messageHandler function for translating each message into the desired type - */ - def apply[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - untilOffsets: Map[TopicAndPartition, LeaderOffset], - messageHandler: MessageAndMetadata[K, V] => R - ): KafkaRDD[K, V, U, T, R] = { - val leaders = untilOffsets.map { case (tp, lo) => - tp -> ((lo.host, lo.port)) - } - - val offsetRanges = fromOffsets.map { case (tp, fo) => - val uo = untilOffsets(tp) - OffsetRange(tp.topic, tp.partition, fo, uo.offset) - }.toArray - - new KafkaRDD[K, V, U, T, R](sc, kafkaParams, offsetRanges, leaders, messageHandler) - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala deleted file mode 100644 index ef1968585be60..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaTestUtils.scala +++ /dev/null @@ -1,299 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.{File, IOException} -import java.lang.{Integer => JInt} -import java.net.InetSocketAddress -import java.util.{Map => JMap, Properties} -import java.util.concurrent.TimeoutException - -import scala.annotation.tailrec -import scala.collection.JavaConverters._ -import scala.util.control.NonFatal - -import kafka.admin.AdminUtils -import kafka.api.Request -import kafka.producer.{KeyedMessage, Producer, ProducerConfig} -import kafka.serializer.StringEncoder -import kafka.server.{KafkaConfig, KafkaServer} -import kafka.utils.{ZKStringSerializer, ZkUtils} -import org.I0Itec.zkclient.ZkClient -import org.apache.commons.lang3.RandomUtils -import org.apache.zookeeper.server.{NIOServerCnxnFactory, ZooKeeperServer} - -import org.apache.spark.SparkConf -import org.apache.spark.internal.Logging -import org.apache.spark.streaming.Time -import org.apache.spark.util.Utils - -/** - * This is a helper class for Kafka test suites. This has the functionality to set up - * and tear down local Kafka servers, and to push data using Kafka producers. - * - * The reason to put Kafka test utility class in src is to test Python related Kafka APIs. - */ -private[kafka] class KafkaTestUtils extends Logging { - - // Zookeeper related configurations - private val zkHost = "localhost" - private var zkPort: Int = 0 - private val zkConnectionTimeout = 60000 - private val zkSessionTimeout = 6000 - - private var zookeeper: EmbeddedZookeeper = _ - - private var zkClient: ZkClient = _ - - // Kafka broker related configurations - private val brokerHost = "localhost" - // 0.8.2 server doesn't have a boundPort method, so can't use 0 for a random port - private var brokerPort = RandomUtils.nextInt(1024, 65536) - private var brokerConf: KafkaConfig = _ - - // Kafka broker server - private var server: KafkaServer = _ - - // Kafka producer - private var producer: Producer[String, String] = _ - - // Flag to test whether the system is correctly started - private var zkReady = false - private var brokerReady = false - - def zkAddress: String = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper address") - s"$zkHost:$zkPort" - } - - def brokerAddress: String = { - assert(brokerReady, "Kafka not setup yet or already torn down, cannot get broker address") - s"$brokerHost:$brokerPort" - } - - def zookeeperClient: ZkClient = { - assert(zkReady, "Zookeeper not setup yet or already torn down, cannot get zookeeper client") - Option(zkClient).getOrElse( - throw new IllegalStateException("Zookeeper client is not yet initialized")) - } - - // Set up the Embedded Zookeeper server and get the proper Zookeeper port - private def setupEmbeddedZookeeper(): Unit = { - // Zookeeper server startup - zookeeper = new EmbeddedZookeeper(s"$zkHost:$zkPort") - // Get the actual zookeeper binding port - zkPort = zookeeper.actualPort - zkClient = new ZkClient(s"$zkHost:$zkPort", zkSessionTimeout, zkConnectionTimeout, - ZKStringSerializer) - zkReady = true - } - - // Set up the Embedded Kafka server - private def setupEmbeddedKafkaServer(): Unit = { - assert(zkReady, "Zookeeper should be set up beforehand") - - // Kafka broker startup - Utils.startServiceOnPort(brokerPort, port => { - brokerPort = port - brokerConf = new KafkaConfig(brokerConfiguration) - server = new KafkaServer(brokerConf) - server.startup() - (server, brokerPort) - }, new SparkConf(), "KafkaBroker") - - brokerReady = true - } - - /** setup the whole embedded servers, including Zookeeper and Kafka brokers */ - def setup(): Unit = { - setupEmbeddedZookeeper() - setupEmbeddedKafkaServer() - } - - /** Teardown the whole servers, including Kafka broker and Zookeeper */ - def teardown(): Unit = { - brokerReady = false - zkReady = false - - if (producer != null) { - producer.close() - producer = null - } - - if (server != null) { - server.shutdown() - server.awaitShutdown() - server = null - } - - // On Windows, `logDirs` is left open even after Kafka server above is completely shut down - // in some cases. It leads to test failures on Windows if the directory deletion failure - // throws an exception. - brokerConf.logDirs.foreach { f => - try { - Utils.deleteRecursively(new File(f)) - } catch { - case e: IOException if Utils.isWindows => - logWarning(e.getMessage) - } - } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (zookeeper != null) { - zookeeper.shutdown() - zookeeper = null - } - } - - /** Create a Kafka topic and wait until it is propagated to the whole cluster */ - def createTopic(topic: String, partitions: Int): Unit = { - AdminUtils.createTopic(zkClient, topic, partitions, 1) - // wait until metadata is propagated - (0 until partitions).foreach { p => waitUntilMetadataIsPropagated(topic, p) } - } - - /** Single-argument version for backwards compatibility */ - def createTopic(topic: String): Unit = createTopic(topic, 1) - - /** Java-friendly function for sending messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: JMap[String, JInt]): Unit = { - sendMessages(topic, Map(messageToFreq.asScala.mapValues(_.intValue()).toSeq: _*)) - } - - /** Send the messages to the Kafka broker */ - def sendMessages(topic: String, messageToFreq: Map[String, Int]): Unit = { - val messages = messageToFreq.flatMap { case (s, freq) => Seq.fill(freq)(s) }.toArray - sendMessages(topic, messages) - } - - /** Send the array of messages to the Kafka broker */ - def sendMessages(topic: String, messages: Array[String]): Unit = { - producer = new Producer[String, String](new ProducerConfig(producerConfiguration)) - producer.send(messages.map { new KeyedMessage[String, String](topic, _ ) }: _*) - producer.close() - producer = null - } - - private def brokerConfiguration: Properties = { - val props = new Properties() - props.put("broker.id", "0") - props.put("host.name", "localhost") - props.put("port", brokerPort.toString) - props.put("log.dir", Utils.createTempDir().getAbsolutePath) - props.put("zookeeper.connect", zkAddress) - props.put("log.flush.interval.messages", "1") - props.put("replica.socket.timeout.ms", "1500") - props - } - - private def producerConfiguration: Properties = { - val props = new Properties() - props.put("metadata.broker.list", brokerAddress) - props.put("serializer.class", classOf[StringEncoder].getName) - // wait for all in-sync replicas to ack sends - props.put("request.required.acks", "-1") - props - } - - // A simplified version of scalatest eventually, rewritten here to avoid adding extra test - // dependency - def eventually[T](timeout: Time, interval: Time)(func: => T): T = { - def makeAttempt(): Either[Throwable, T] = { - try { - Right(func) - } catch { - case e if NonFatal(e) => Left(e) - } - } - - val startTime = System.currentTimeMillis() - @tailrec - def tryAgain(attempt: Int): T = { - makeAttempt() match { - case Right(result) => result - case Left(e) => - val duration = System.currentTimeMillis() - startTime - if (duration < timeout.milliseconds) { - Thread.sleep(interval.milliseconds) - } else { - throw new TimeoutException(e.getMessage) - } - - tryAgain(attempt + 1) - } - } - - tryAgain(1) - } - - private def waitUntilMetadataIsPropagated(topic: String, partition: Int): Unit = { - def isPropagated = server.apis.metadataCache.getPartitionInfo(topic, partition) match { - case Some(partitionState) => - val leaderAndInSyncReplicas = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr - - ZkUtils.getLeaderForPartition(zkClient, topic, partition).isDefined && - Request.isValidBrokerId(leaderAndInSyncReplicas.leader) && - leaderAndInSyncReplicas.isr.size >= 1 - - case _ => - false - } - eventually(Time(10000), Time(100)) { - assert(isPropagated, s"Partition [$topic, $partition] metadata not propagated after timeout") - } - } - - private class EmbeddedZookeeper(val zkConnect: String) { - val snapshotDir = Utils.createTempDir() - val logDir = Utils.createTempDir() - - val zookeeper = new ZooKeeperServer(snapshotDir, logDir, 500) - val (ip, port) = { - val splits = zkConnect.split(":") - (splits(0), splits(1).toInt) - } - val factory = new NIOServerCnxnFactory() - factory.configure(new InetSocketAddress(ip, port), 16) - factory.startup(zookeeper) - - val actualPort = factory.getLocalPort - - def shutdown() { - factory.shutdown() - // The directories are not closed even if the ZooKeeper server is shut down. - // Please see ZOOKEEPER-1844, which is fixed in 3.4.6+. It leads to test failures - // on Windows if the directory deletion failure throws an exception. - try { - Utils.deleteRecursively(snapshotDir) - } catch { - case e: IOException if Utils.isWindows => - logWarning(e.getMessage) - } - try { - Utils.deleteRecursively(logDir) - } catch { - case e: IOException if Utils.isWindows => - logWarning(e.getMessage) - } - } - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala deleted file mode 100644 index 36082e93707b8..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ /dev/null @@ -1,806 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.OutputStream -import java.lang.{Integer => JInt, Long => JLong, Number => JNumber} -import java.nio.charset.StandardCharsets -import java.util.{List => JList, Locale, Map => JMap, Set => JSet} - -import scala.collection.JavaConverters._ -import scala.reflect.ClassTag - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.{Decoder, DefaultDecoder, StringDecoder} -import net.razorvine.pickle.{IObjectPickler, Opcodes, Pickler} - -import org.apache.spark.{SparkContext, SparkException} -import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaSparkContext} -import org.apache.spark.api.java.function.{Function => JFunction} -import org.apache.spark.api.python.SerDeUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java._ -import org.apache.spark.streaming.dstream.{DStream, InputDStream, ReceiverInputDStream} -import org.apache.spark.streaming.util.WriteAheadLogUtils - -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -object KafkaUtils { - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel Storage level to use for storing the received objects - * (default: StorageLevel.MEMORY_AND_DISK_SER_2) - * @return DStream of (Kafka message key, Kafka message value) - */ - def createStream( - ssc: StreamingContext, - zkQuorum: String, - groupId: String, - topics: Map[String, Int], - storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): ReceiverInputDStream[(String, String)] = { - val kafkaParams = Map[String, String]( - "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, - "zookeeper.connection.timeout.ms" -> "10000") - createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param ssc StreamingContext object - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel Storage level to use for storing the received objects - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam U type of Kafka message key decoder - * @tparam T type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createStream[K: ClassTag, V: ClassTag, U <: Decoder[_]: ClassTag, T <: Decoder[_]: ClassTag]( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel - ): ReceiverInputDStream[(K, V)] = { - val walEnabled = WriteAheadLogUtils.enableReceiverLog(ssc.conf) - new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, walEnabled, storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..) - * @param groupId The group id for this consumer - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread - * @return DStream of (Kafka message key, Kafka message value) - */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt] - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*)) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param zkQuorum Zookeeper quorum (hostname:port,hostname:port,..). - * @param groupId The group id for this consumer. - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread. - * @param storageLevel RDD storage level. - * @return DStream of (Kafka message key, Kafka message value) - */ - def createStream( - jssc: JavaStreamingContext, - zkQuorum: String, - groupId: String, - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[String, String] = { - createStream(jssc.ssc, zkQuorum, groupId, Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** - * Create an input stream that pulls messages from Kafka Brokers. - * @param jssc JavaStreamingContext object - * @param keyTypeClass Key type of DStream - * @param valueTypeClass value type of Dstream - * @param keyDecoderClass Type of kafka key decoder - * @param valueDecoderClass Type of kafka value decoder - * @param kafkaParams Map of kafka configuration parameters, - * see http://kafka.apache.org/08/configuration.html - * @param topics Map of (topic_name to numPartitions) to consume. Each partition is consumed - * in its own thread - * @param storageLevel RDD storage level. - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam U type of Kafka message key decoder - * @tparam T type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createStream[K, V, U <: Decoder[_], T <: Decoder[_]]( - jssc: JavaStreamingContext, - keyTypeClass: Class[K], - valueTypeClass: Class[V], - keyDecoderClass: Class[U], - valueDecoderClass: Class[T], - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel - ): JavaPairReceiverInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyTypeClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueTypeClass) - - implicit val keyCmd: ClassTag[U] = ClassTag(keyDecoderClass) - implicit val valueCmd: ClassTag[T] = ClassTag(valueDecoderClass) - - createStream[K, V, U, T]( - jssc.ssc, - kafkaParams.asScala.toMap, - Map(topics.asScala.mapValues(_.intValue()).toSeq: _*), - storageLevel) - } - - /** get leaders for the given offset ranges, or throw an exception */ - private def leadersForRanges( - kc: KafkaCluster, - offsetRanges: Array[OffsetRange]): Map[TopicAndPartition, (String, Int)] = { - val topics = offsetRanges.map(o => TopicAndPartition(o.topic, o.partition)).toSet - val leaders = kc.findLeaders(topics) - KafkaCluster.checkErrors(leaders) - } - - /** Make sure offsets are available in kafka, or throw an exception */ - private def checkOffsets( - kc: KafkaCluster, - offsetRanges: Array[OffsetRange]): Unit = { - val topics = offsetRanges.map(_.topicAndPartition).toSet - val result = for { - low <- kc.getEarliestLeaderOffsets(topics).right - high <- kc.getLatestLeaderOffsets(topics).right - } yield { - offsetRanges.filterNot { o => - low(o.topicAndPartition).offset <= o.fromOffset && - o.untilOffset <= high(o.topicAndPartition).offset - } - } - val badRanges = KafkaCluster.checkErrors(result) - if (!badRanges.isEmpty) { - throw new SparkException("Offsets not available on leader: " + badRanges.mkString(",")) - } - } - - private[kafka] def getFromOffsets( - kc: KafkaCluster, - kafkaParams: Map[String, String], - topics: Set[String] - ): Map[TopicAndPartition, Long] = { - val reset = kafkaParams.get("auto.offset.reset").map(_.toLowerCase(Locale.ROOT)) - val result = for { - topicPartitions <- kc.getPartitions(topics).right - leaderOffsets <- (if (reset == Some("smallest")) { - kc.getEarliestLeaderOffsets(topicPartitions) - } else { - kc.getLatestLeaderOffsets(topicPartitions) - }).right - } yield { - leaderOffsets.map { case (tp, lo) => - (tp, lo.offset) - } - } - KafkaCluster.checkErrors(result) - } - - /** - * Create an RDD from Kafka using offset ranges for each topic and partition. - * - * @param sc SparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return RDD of (Kafka message key, Kafka message value) - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange] - ): RDD[(K, V)] = sc.withScope { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val leaders = leadersForRanges(kc, offsetRanges) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, (K, V)](sc, kafkaParams, offsetRanges, leaders, messageHandler) - } - - /** - * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param sc SparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, - * in which case leaders will be looked up on the driver. - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return RDD of R - */ - def createRDD[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, - R: ClassTag]( - sc: SparkContext, - kafkaParams: Map[String, String], - offsetRanges: Array[OffsetRange], - leaders: Map[TopicAndPartition, Broker], - messageHandler: MessageAndMetadata[K, V] => R - ): RDD[R] = sc.withScope { - val kc = new KafkaCluster(kafkaParams) - val leaderMap = if (leaders.isEmpty) { - leadersForRanges(kc, offsetRanges) - } else { - // This could be avoided by refactoring KafkaRDD.leaders and KafkaCluster to use Broker - leaders.map { - case (tp: TopicAndPartition, Broker(host, port)) => (tp, (host, port)) - } - } - val cleanedHandler = sc.clean(messageHandler) - checkOffsets(kc, offsetRanges) - new KafkaRDD[K, V, KD, VD, R](sc, kafkaParams, offsetRanges, leaderMap, cleanedHandler) - } - - /** - * Create an RDD from Kafka using offset ranges for each topic and partition. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param keyClass type of Kafka message key - * @param valueClass type of Kafka message value - * @param keyDecoderClass type of Kafka message key decoder - * @param valueDecoderClass type of Kafka message value decoder - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return RDD of (Kafka message key, Kafka message value) - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange] - ): JavaPairRDD[K, V] = jsc.sc.withScope { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - new JavaPairRDD(createRDD[K, V, KD, VD]( - jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges)) - } - - /** - * Create an RDD from Kafka using offset ranges for each topic and partition. This allows you - * specify the Kafka leader to connect to (to optimize fetching) and access the message as well - * as the metadata. - * - * @param jsc JavaSparkContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param offsetRanges Each OffsetRange in the batch corresponds to a - * range of offsets for a given Kafka topic/partition - * @param leaders Kafka brokers for each TopicAndPartition in offsetRanges. May be an empty map, - * in which case leaders will be looked up on the driver. - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return RDD of R - */ - def createRDD[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jsc: JavaSparkContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - offsetRanges: Array[OffsetRange], - leaders: JMap[TopicAndPartition, Broker], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaRDD[R] = jsc.sc.withScope { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val leaderMap = Map(leaders.asScala.toSeq: _*) - createRDD[K, V, KD, VD, R]( - jsc.sc, Map(kafkaParams.asScala.toSeq: _*), offsetRanges, leaderMap, messageHandler.call(_)) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the `StreamingContext`. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers) specified in - * host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return DStream of R - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag, - R: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - fromOffsets: Map[TopicAndPartition, Long], - messageHandler: MessageAndMetadata[K, V] => R - ): InputDStream[R] = { - val cleanedHandler = ssc.sc.clean(messageHandler) - new DirectKafkaInputDStream[K, V, KD, VD, R]( - ssc, kafkaParams, fromOffsets, cleanedHandler) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the `StreamingContext`. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param ssc StreamingContext object - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createDirectStream[ - K: ClassTag, - V: ClassTag, - KD <: Decoder[K]: ClassTag, - VD <: Decoder[V]: ClassTag] ( - ssc: StreamingContext, - kafkaParams: Map[String, String], - topics: Set[String] - ): InputDStream[(K, V)] = { - val messageHandler = (mmd: MessageAndMetadata[K, V]) => (mmd.key, mmd.message) - val kc = new KafkaCluster(kafkaParams) - val fromOffsets = getFromOffsets(kc, kafkaParams, topics) - new DirectKafkaInputDStream[K, V, KD, VD, (K, V)]( - ssc, kafkaParams, fromOffsets, messageHandler) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the `StreamingContext`. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class of the value decoder - * @param recordClass Class of the records in DStream - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * @param fromOffsets Per-topic/partition Kafka offsets defining the (inclusive) - * starting point of the stream - * @param messageHandler Function for translating each message and metadata into the desired type - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @tparam R type returned by messageHandler - * @return DStream of R - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V], R]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - recordClass: Class[R], - kafkaParams: JMap[String, String], - fromOffsets: JMap[TopicAndPartition, JLong], - messageHandler: JFunction[MessageAndMetadata[K, V], R] - ): JavaInputDStream[R] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - implicit val recordCmt: ClassTag[R] = ClassTag(recordClass) - val cleanedHandler = jssc.sparkContext.clean(messageHandler.call _) - createDirectStream[K, V, KD, VD, R]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Map(fromOffsets.asScala.mapValues(_.longValue()).toSeq: _*), - cleanedHandler - ) - } - - /** - * Create an input stream that directly pulls messages from Kafka Brokers - * without using any receiver. This stream can guarantee that each message - * from Kafka is included in transformations exactly once (see points below). - * - * Points to note: - * - No receivers: This stream does not use any receiver. It directly queries Kafka - * - Offsets: This does not use Zookeeper to store offsets. The consumed offsets are tracked - * by the stream itself. For interoperability with Kafka monitoring tools that depend on - * Zookeeper, you have to update Kafka/Zookeeper yourself from the streaming application. - * You can access the offsets used in each batch from the generated RDDs (see - * [[org.apache.spark.streaming.kafka.HasOffsetRanges]]). - * - Failure Recovery: To recover from driver failures, you have to enable checkpointing - * in the `StreamingContext`. The information on consumed offset can be - * recovered from the checkpoint. See the programming guide for details (constraints, etc.). - * - End-to-end semantics: This stream ensures that every records is effectively received and - * transformed exactly once, but gives no guarantees on whether the transformed data are - * outputted exactly once. For end-to-end exactly-once semantics, you have to either ensure - * that the output operation is idempotent, or use transactions to output records atomically. - * See the programming guide for more details. - * - * @param jssc JavaStreamingContext object - * @param keyClass Class of the keys in the Kafka records - * @param valueClass Class of the values in the Kafka records - * @param keyDecoderClass Class of the key decoder - * @param valueDecoderClass Class type of the value decoder - * @param kafkaParams Kafka - * configuration parameters. Requires "metadata.broker.list" or "bootstrap.servers" - * to be set with Kafka broker(s) (NOT zookeeper servers), specified in - * host1:port1,host2:port2 form. - * If not starting from a checkpoint, "auto.offset.reset" may be set to "largest" or "smallest" - * to determine where the stream starts (defaults to "largest") - * @param topics Names of the topics to consume - * @tparam K type of Kafka message key - * @tparam V type of Kafka message value - * @tparam KD type of Kafka message key decoder - * @tparam VD type of Kafka message value decoder - * @return DStream of (Kafka message key, Kafka message value) - */ - def createDirectStream[K, V, KD <: Decoder[K], VD <: Decoder[V]]( - jssc: JavaStreamingContext, - keyClass: Class[K], - valueClass: Class[V], - keyDecoderClass: Class[KD], - valueDecoderClass: Class[VD], - kafkaParams: JMap[String, String], - topics: JSet[String] - ): JavaPairInputDStream[K, V] = { - implicit val keyCmt: ClassTag[K] = ClassTag(keyClass) - implicit val valueCmt: ClassTag[V] = ClassTag(valueClass) - implicit val keyDecoderCmt: ClassTag[KD] = ClassTag(keyDecoderClass) - implicit val valueDecoderCmt: ClassTag[VD] = ClassTag(valueDecoderClass) - createDirectStream[K, V, KD, VD]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Set(topics.asScala.toSeq: _*) - ) - } -} - -/** - * This is a helper class that wraps the KafkaUtils.createStream() into more - * Python-friendly class and function so that it can be easily - * instantiated and called from Python's KafkaUtils. - * - * The zero-arg constructor helps instantiate this class from the Class object - * classOf[KafkaUtilsPythonHelper].newInstance(), and the createStream() - * takes care of known parameters instead of passing them from Python - */ -private[kafka] class KafkaUtilsPythonHelper { - import KafkaUtilsPythonHelper._ - - def createStream( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JMap[String, JInt], - storageLevel: StorageLevel): JavaPairReceiverInputDStream[Array[Byte], Array[Byte]] = { - KafkaUtils.createStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder]( - jssc, - classOf[Array[Byte]], - classOf[Array[Byte]], - classOf[DefaultDecoder], - classOf[DefaultDecoder], - kafkaParams, - topics, - storageLevel) - } - - def createRDDWithoutMessageHandler( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker]): JavaRDD[(Array[Byte], Array[Byte])] = { - val messageHandler = - (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) - new JavaRDD(createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler)) - } - - def createRDDWithMessageHandler( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker]): JavaRDD[Array[Byte]] = { - val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => - new PythonMessageAndMetadata( - mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) - val rdd = createRDD(jsc, kafkaParams, offsetRanges, leaders, messageHandler). - mapPartitions(picklerIterator) - new JavaRDD(rdd) - } - - private def createRDD[V: ClassTag]( - jsc: JavaSparkContext, - kafkaParams: JMap[String, String], - offsetRanges: JList[OffsetRange], - leaders: JMap[TopicAndPartition, Broker], - messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): RDD[V] = { - KafkaUtils.createRDD[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( - jsc.sc, - kafkaParams.asScala.toMap, - offsetRanges.toArray(new Array[OffsetRange](offsetRanges.size())), - leaders.asScala.toMap, - messageHandler - ) - } - - def createDirectStreamWithoutMessageHandler( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[(Array[Byte], Array[Byte])] = { - val messageHandler = - (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => (mmd.key, mmd.message) - new JavaDStream(createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler)) - } - - def createDirectStreamWithMessageHandler( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JNumber]): JavaDStream[Array[Byte]] = { - val messageHandler = (mmd: MessageAndMetadata[Array[Byte], Array[Byte]]) => - new PythonMessageAndMetadata(mmd.topic, mmd.partition, mmd.offset, mmd.key(), mmd.message()) - val stream = createDirectStream(jssc, kafkaParams, topics, fromOffsets, messageHandler). - mapPartitions(picklerIterator) - new JavaDStream(stream) - } - - private def createDirectStream[V: ClassTag]( - jssc: JavaStreamingContext, - kafkaParams: JMap[String, String], - topics: JSet[String], - fromOffsets: JMap[TopicAndPartition, JNumber], - messageHandler: MessageAndMetadata[Array[Byte], Array[Byte]] => V): DStream[V] = { - - val currentFromOffsets = if (!fromOffsets.isEmpty) { - val topicsFromOffsets = fromOffsets.keySet().asScala.map(_.topic) - if (topicsFromOffsets != topics.asScala.toSet) { - throw new IllegalStateException( - s"The specified topics: ${topics.asScala.toSet.mkString(" ")} " + - s"do not equal to the topic from offsets: ${topicsFromOffsets.mkString(" ")}") - } - Map(fromOffsets.asScala.mapValues { _.longValue() }.toSeq: _*) - } else { - val kc = new KafkaCluster(Map(kafkaParams.asScala.toSeq: _*)) - KafkaUtils.getFromOffsets( - kc, Map(kafkaParams.asScala.toSeq: _*), Set(topics.asScala.toSeq: _*)) - } - - KafkaUtils.createDirectStream[Array[Byte], Array[Byte], DefaultDecoder, DefaultDecoder, V]( - jssc.ssc, - Map(kafkaParams.asScala.toSeq: _*), - Map(currentFromOffsets.toSeq: _*), - messageHandler) - } - - def createOffsetRange(topic: String, partition: JInt, fromOffset: JLong, untilOffset: JLong - ): OffsetRange = OffsetRange.create(topic, partition, fromOffset, untilOffset) - - def createTopicAndPartition(topic: String, partition: JInt): TopicAndPartition = - TopicAndPartition(topic, partition) - - def createBroker(host: String, port: JInt): Broker = Broker(host, port) - - def offsetRangesOfKafkaRDD(rdd: RDD[_]): JList[OffsetRange] = { - val parentRDDs = rdd.getNarrowAncestors - val kafkaRDDs = parentRDDs.filter(rdd => rdd.isInstanceOf[KafkaRDD[_, _, _, _, _]]) - - require( - kafkaRDDs.length == 1, - "Cannot get offset ranges, as there may be multiple Kafka RDDs or no Kafka RDD associated" + - "with this RDD, please call this method only on a Kafka RDD.") - - val kafkaRDD = kafkaRDDs.head.asInstanceOf[KafkaRDD[_, _, _, _, _]] - kafkaRDD.offsetRanges.toSeq.asJava - } -} - -private object KafkaUtilsPythonHelper { - private var initialized = false - - def initialize(): Unit = { - SerDeUtil.initialize() - synchronized { - if (!initialized) { - new PythonMessageAndMetadataPickler().register() - initialized = true - } - } - } - - initialize() - - def picklerIterator(iter: Iterator[Any]): Iterator[Array[Byte]] = { - new SerDeUtil.AutoBatchedPickler(iter) - } - - case class PythonMessageAndMetadata( - topic: String, - partition: JInt, - offset: JLong, - key: Array[Byte], - message: Array[Byte]) - - class PythonMessageAndMetadataPickler extends IObjectPickler { - private val module = "pyspark.streaming.kafka" - - def register(): Unit = { - Pickler.registerCustomPickler(classOf[PythonMessageAndMetadata], this) - Pickler.registerCustomPickler(this.getClass, this) - } - - def pickle(obj: Object, out: OutputStream, pickler: Pickler) { - if (obj == this) { - out.write(Opcodes.GLOBAL) - out.write(s"$module\nKafkaMessageAndMetadata\n".getBytes(StandardCharsets.UTF_8)) - } else { - pickler.save(this) - val msgAndMetaData = obj.asInstanceOf[PythonMessageAndMetadata] - out.write(Opcodes.MARK) - pickler.save(msgAndMetaData.topic) - pickler.save(msgAndMetaData.partition) - pickler.save(msgAndMetaData.offset) - pickler.save(msgAndMetaData.key) - pickler.save(msgAndMetaData.message) - out.write(Opcodes.TUPLE) - out.write(Opcodes.REDUCE) - } - } - } -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala deleted file mode 100644 index 6dab5f950d4cd..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/OffsetRange.scala +++ /dev/null @@ -1,112 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import kafka.common.TopicAndPartition - -/** - * Represents any object that has a collection of [[OffsetRange]]s. This can be used to access the - * offset ranges in RDDs generated by the direct Kafka DStream (see - * `KafkaUtils.createDirectStream()`). - * {{{ - * KafkaUtils.createDirectStream(...).foreachRDD { rdd => - * val offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - * ... - * } - * }}} - */ -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -trait HasOffsetRanges { - def offsetRanges: Array[OffsetRange] -} - -/** - * Represents a range of offsets from a single Kafka TopicAndPartition. Instances of this class - * can be created with `OffsetRange.create()`. - * @param topic Kafka topic name - * @param partition Kafka partition id - * @param fromOffset Inclusive starting offset - * @param untilOffset Exclusive ending offset - */ -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -final class OffsetRange private( - val topic: String, - val partition: Int, - val fromOffset: Long, - val untilOffset: Long) extends Serializable { - import OffsetRange.OffsetRangeTuple - - /** Kafka TopicAndPartition object, for convenience */ - def topicAndPartition(): TopicAndPartition = TopicAndPartition(topic, partition) - - /** Number of messages this OffsetRange refers to */ - def count(): Long = untilOffset - fromOffset - - override def equals(obj: Any): Boolean = obj match { - case that: OffsetRange => - this.topic == that.topic && - this.partition == that.partition && - this.fromOffset == that.fromOffset && - this.untilOffset == that.untilOffset - case _ => false - } - - override def hashCode(): Int = { - toTuple.hashCode() - } - - override def toString(): String = { - s"OffsetRange(topic: '$topic', partition: $partition, range: [$fromOffset -> $untilOffset])" - } - - /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[streaming] - def toTuple: OffsetRangeTuple = (topic, partition, fromOffset, untilOffset) -} - -/** - * Companion object the provides methods to create instances of [[OffsetRange]]. - */ -@deprecated("Update to Kafka 0.10 integration", "2.3.0") -object OffsetRange { - def create(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topic, partition, fromOffset, untilOffset) - - def create( - topicAndPartition: TopicAndPartition, - fromOffset: Long, - untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) - - def apply(topic: String, partition: Int, fromOffset: Long, untilOffset: Long): OffsetRange = - new OffsetRange(topic, partition, fromOffset, untilOffset) - - def apply( - topicAndPartition: TopicAndPartition, - fromOffset: Long, - untilOffset: Long): OffsetRange = - new OffsetRange(topicAndPartition.topic, topicAndPartition.partition, fromOffset, untilOffset) - - /** this is to avoid ClassNotFoundException during checkpoint restore */ - private[kafka] - type OffsetRangeTuple = (String, Int, Long, Long) - - private[kafka] - def apply(t: OffsetRangeTuple) = - new OffsetRange(t._1, t._2, t._3, t._4) -} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala b/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala deleted file mode 100644 index 39abe3c3e29d0..0000000000000 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/ReliableKafkaReceiver.scala +++ /dev/null @@ -1,302 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.util.Properties -import java.util.concurrent.{ConcurrentHashMap, ThreadPoolExecutor} - -import scala.collection.{mutable, Map} -import scala.reflect.{classTag, ClassTag} - -import kafka.common.TopicAndPartition -import kafka.consumer.{Consumer, ConsumerConfig, ConsumerConnector, KafkaStream} -import kafka.message.MessageAndMetadata -import kafka.serializer.Decoder -import kafka.utils.{VerifiableProperties, ZKGroupTopicDirs, ZKStringSerializer, ZkUtils} -import org.I0Itec.zkclient.ZkClient - -import org.apache.spark.SparkEnv -import org.apache.spark.internal.Logging -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver} -import org.apache.spark.util.ThreadUtils - -/** - * ReliableKafkaReceiver offers the ability to reliably store data into BlockManager without loss. - * It is turned off by default and will be enabled when - * spark.streaming.receiver.writeAheadLog.enable is true. The difference compared to KafkaReceiver - * is that this receiver manages topic-partition/offset itself and updates the offset information - * after data is reliably stored as write-ahead log. Offsets will only be updated when data is - * reliably stored, so the potential data loss problem of KafkaReceiver can be eliminated. - * - * Note: ReliableKafkaReceiver will set auto.commit.enable to false to turn off automatic offset - * commit mechanism in Kafka consumer. So setting this configuration manually within kafkaParams - * will not take effect. - */ -private[streaming] -class ReliableKafkaReceiver[ - K: ClassTag, - V: ClassTag, - U <: Decoder[_]: ClassTag, - T <: Decoder[_]: ClassTag]( - kafkaParams: Map[String, String], - topics: Map[String, Int], - storageLevel: StorageLevel) - extends Receiver[(K, V)](storageLevel) with Logging { - - private val groupId = kafkaParams("group.id") - private val AUTO_OFFSET_COMMIT = "auto.commit.enable" - private def conf = SparkEnv.get.conf - - /** High level consumer to connect to Kafka. */ - private var consumerConnector: ConsumerConnector = null - - /** zkClient to connect to Zookeeper to commit the offsets. */ - private var zkClient: ZkClient = null - - /** - * A HashMap to manage the offset for each topic/partition, this HashMap is called in - * synchronized block, so mutable HashMap will not meet concurrency issue. - */ - private var topicPartitionOffsetMap: mutable.HashMap[TopicAndPartition, Long] = null - - /** A concurrent HashMap to store the stream block id and related offset snapshot. */ - private var blockOffsetMap: ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]] = null - - /** - * Manage the BlockGenerator in receiver itself for better managing block store and offset - * commit. - */ - private var blockGenerator: BlockGenerator = null - - /** Thread pool running the handlers for receiving message from multiple topics and partitions. */ - private var messageHandlerThreadPool: ThreadPoolExecutor = null - - override def onStart(): Unit = { - logInfo(s"Starting Kafka Consumer Stream with group: $groupId") - - // Initialize the topic-partition / offset hash map. - topicPartitionOffsetMap = new mutable.HashMap[TopicAndPartition, Long] - - // Initialize the stream block id / offset snapshot hash map. - blockOffsetMap = new ConcurrentHashMap[StreamBlockId, Map[TopicAndPartition, Long]]() - - // Initialize the block generator for storing Kafka message. - blockGenerator = supervisor.createBlockGenerator(new GeneratedBlockHandler) - - if (kafkaParams.contains(AUTO_OFFSET_COMMIT) && kafkaParams(AUTO_OFFSET_COMMIT) == "true") { - logWarning(s"$AUTO_OFFSET_COMMIT should be set to false in ReliableKafkaReceiver, " + - "otherwise we will manually set it to false to turn off auto offset commit in Kafka") - } - - val props = new Properties() - kafkaParams.foreach(param => props.put(param._1, param._2)) - // Manually set "auto.commit.enable" to "false" no matter user explicitly set it to true, - // we have to make sure this property is set to false to turn off auto commit mechanism in - // Kafka. - props.setProperty(AUTO_OFFSET_COMMIT, "false") - - val consumerConfig = new ConsumerConfig(props) - - assert(!consumerConfig.autoCommitEnable) - - logInfo(s"Connecting to Zookeeper: ${consumerConfig.zkConnect}") - consumerConnector = Consumer.create(consumerConfig) - logInfo(s"Connected to Zookeeper: ${consumerConfig.zkConnect}") - - zkClient = new ZkClient(consumerConfig.zkConnect, consumerConfig.zkSessionTimeoutMs, - consumerConfig.zkConnectionTimeoutMs, ZKStringSerializer) - - messageHandlerThreadPool = ThreadUtils.newDaemonFixedThreadPool( - topics.values.sum, "KafkaMessageHandler") - - blockGenerator.start() - - val keyDecoder = classTag[U].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[K]] - - val valueDecoder = classTag[T].runtimeClass.getConstructor(classOf[VerifiableProperties]) - .newInstance(consumerConfig.props) - .asInstanceOf[Decoder[V]] - - val topicMessageStreams = consumerConnector.createMessageStreams( - topics, keyDecoder, valueDecoder) - - topicMessageStreams.values.foreach { streams => - streams.foreach { stream => - messageHandlerThreadPool.submit(new MessageHandler(stream)) - } - } - } - - override def onStop(): Unit = { - if (messageHandlerThreadPool != null) { - messageHandlerThreadPool.shutdown() - messageHandlerThreadPool = null - } - - if (consumerConnector != null) { - consumerConnector.shutdown() - consumerConnector = null - } - - if (zkClient != null) { - zkClient.close() - zkClient = null - } - - if (blockGenerator != null) { - blockGenerator.stop() - blockGenerator = null - } - - if (topicPartitionOffsetMap != null) { - topicPartitionOffsetMap.clear() - topicPartitionOffsetMap = null - } - - if (blockOffsetMap != null) { - blockOffsetMap.clear() - blockOffsetMap = null - } - } - - /** Store a Kafka message and the associated metadata as a tuple. */ - private def storeMessageAndMetadata( - msgAndMetadata: MessageAndMetadata[K, V]): Unit = { - val topicAndPartition = TopicAndPartition(msgAndMetadata.topic, msgAndMetadata.partition) - val data = (msgAndMetadata.key, msgAndMetadata.message) - val metadata = (topicAndPartition, msgAndMetadata.offset) - blockGenerator.addDataWithCallback(data, metadata) - } - - /** Update stored offset */ - private def updateOffset(topicAndPartition: TopicAndPartition, offset: Long): Unit = { - topicPartitionOffsetMap.put(topicAndPartition, offset) - } - - /** - * Remember the current offsets for each topic and partition. This is called when a block is - * generated. - */ - private def rememberBlockOffsets(blockId: StreamBlockId): Unit = { - // Get a snapshot of current offset map and store with related block id. - val offsetSnapshot = topicPartitionOffsetMap.toMap - blockOffsetMap.put(blockId, offsetSnapshot) - topicPartitionOffsetMap.clear() - } - - /** - * Store the ready-to-be-stored block and commit the related offsets to zookeeper. This method - * will try a fixed number of times to push the block. If the push fails, the receiver is stopped. - */ - private def storeBlockAndCommitOffset( - blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - var count = 0 - var pushed = false - var exception: Exception = null - while (!pushed && count <= 3) { - try { - store(arrayBuffer.asInstanceOf[mutable.ArrayBuffer[(K, V)]]) - pushed = true - } catch { - case ex: Exception => - count += 1 - exception = ex - } - } - if (pushed) { - Option(blockOffsetMap.get(blockId)).foreach(commitOffset) - blockOffsetMap.remove(blockId) - } else { - stop("Error while storing block into Spark", exception) - } - } - - /** - * Commit the offset of Kafka's topic/partition, the commit mechanism follow Kafka 0.8.x's - * metadata schema in Zookeeper. - */ - private def commitOffset(offsetMap: Map[TopicAndPartition, Long]): Unit = { - if (zkClient == null) { - val thrown = new IllegalStateException("Zookeeper client is unexpectedly null") - stop("Zookeeper client is not initialized before commit offsets to ZK", thrown) - return - } - - for ((topicAndPart, offset) <- offsetMap) { - try { - val topicDirs = new ZKGroupTopicDirs(groupId, topicAndPart.topic) - val zkPath = s"${topicDirs.consumerOffsetDir}/${topicAndPart.partition}" - - ZkUtils.updatePersistentPath(zkClient, zkPath, offset.toString) - } catch { - case e: Exception => - logWarning(s"Exception during commit offset $offset for topic" + - s"${topicAndPart.topic}, partition ${topicAndPart.partition}", e) - } - - logInfo(s"Committed offset $offset for topic ${topicAndPart.topic}, " + - s"partition ${topicAndPart.partition}") - } - } - - /** Class to handle received Kafka message. */ - private final class MessageHandler(stream: KafkaStream[K, V]) extends Runnable { - override def run(): Unit = { - while (!isStopped) { - try { - val streamIterator = stream.iterator() - while (streamIterator.hasNext) { - storeMessageAndMetadata(streamIterator.next) - } - } catch { - case e: Exception => - reportError("Error handling message", e) - } - } - } - } - - /** Class to handle blocks generated by the block generator. */ - private final class GeneratedBlockHandler extends BlockGeneratorListener { - - def onAddData(data: Any, metadata: Any): Unit = { - // Update the offset of the data that was added to the generator - if (metadata != null) { - val (topicAndPartition, offset) = metadata.asInstanceOf[(TopicAndPartition, Long)] - updateOffset(topicAndPartition, offset) - } - } - - def onGenerateBlock(blockId: StreamBlockId): Unit = { - // Remember the offsets of topics/partitions when a block has been generated - rememberBlockOffsets(blockId) - } - - def onPushBlock(blockId: StreamBlockId, arrayBuffer: mutable.ArrayBuffer[_]): Unit = { - // Store block and commit the blocks offset - storeBlockAndCommitOffset(blockId, arrayBuffer) - } - - def onError(message: String, throwable: Throwable): Unit = { - reportError(message, throwable) - } - } -} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java deleted file mode 100644 index 71404a7331ec4..0000000000000 --- a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaDirectKafkaStreamSuite.java +++ /dev/null @@ -1,170 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka; - -import java.io.Serializable; -import java.util.*; -import java.util.concurrent.atomic.AtomicReference; - -import scala.Tuple2; - -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; -import org.apache.spark.streaming.Durations; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; - -public class JavaDirectKafkaStreamSuite implements Serializable { - private transient JavaStreamingContext ssc = null; - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, Durations.milliseconds(200)); - } - - @After - public void tearDown() { - if (ssc != null) { - ssc.stop(); - ssc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaStream() throws InterruptedException { - final String topic1 = "topic1"; - final String topic2 = "topic2"; - // hold a reference to the current offset ranges, so it can be used downstream - final AtomicReference offsetRanges = new AtomicReference<>(); - - String[] topic1data = createTopicAndSendData(topic1); - String[] topic2data = createTopicAndSendData(topic2); - - Set sent = new HashSet<>(); - sent.addAll(Arrays.asList(topic1data)); - sent.addAll(Arrays.asList(topic2data)); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - kafkaParams.put("auto.offset.reset", "smallest"); - - JavaDStream stream1 = KafkaUtils.createDirectStream( - ssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - topicToSet(topic1) - ).transformToPair( - // Make sure you can get offset ranges from the rdd - new Function, JavaPairRDD>() { - @Override - public JavaPairRDD call(JavaPairRDD rdd) { - OffsetRange[] offsets = ((HasOffsetRanges) rdd.rdd()).offsetRanges(); - offsetRanges.set(offsets); - Assert.assertEquals(topic1, offsets[0].topic()); - return rdd; - } - } - ).map( - new Function, String>() { - @Override - public String call(Tuple2 kv) { - return kv._2(); - } - } - ); - - JavaDStream stream2 = KafkaUtils.createDirectStream( - ssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - String.class, - kafkaParams, - topicOffsetToMap(topic2, 0L), - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); - JavaDStream unifiedStream = stream1.union(stream2); - - final Set result = Collections.synchronizedSet(new HashSet()); - unifiedStream.foreachRDD(new VoidFunction>() { - @Override - public void call(JavaRDD rdd) { - result.addAll(rdd.collect()); - } - } - ); - ssc.start(); - long startTime = System.currentTimeMillis(); - boolean matches = false; - while (!matches && System.currentTimeMillis() - startTime < 20000) { - matches = sent.size() == result.size(); - Thread.sleep(50); - } - Assert.assertEquals(sent, result); - ssc.stop(); - } - - private static Set topicToSet(String topic) { - Set topicSet = new HashSet<>(); - topicSet.add(topic); - return topicSet; - } - - private static Map topicOffsetToMap(String topic, Long offsetToStart) { - Map topicMap = new HashMap<>(); - topicMap.put(new TopicAndPartition(topic, 0), offsetToStart); - return topicMap; - } - - private String[] createTopicAndSendData(String topic) { - String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - kafkaTestUtils.createTopic(topic, 1); - kafkaTestUtils.sendMessages(topic, data); - return data; - } -} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java deleted file mode 100644 index c41b6297b0481..0000000000000 --- a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaRDDSuite.java +++ /dev/null @@ -1,156 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka; - -import java.io.Serializable; -import java.util.HashMap; -import java.util.Map; - -import scala.Tuple2; - -import kafka.common.TopicAndPartition; -import kafka.message.MessageAndMetadata; -import kafka.serializer.StringDecoder; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; - -public class JavaKafkaRDDSuite implements Serializable { - private transient JavaSparkContext sc = null; - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - sc = new JavaSparkContext(sparkConf); - } - - @After - public void tearDown() { - if (sc != null) { - sc.stop(); - sc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaRDD() throws InterruptedException { - String topic1 = "topic1"; - String topic2 = "topic2"; - - createTopicAndSendData(topic1); - createTopicAndSendData(topic2); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("metadata.broker.list", kafkaTestUtils.brokerAddress()); - - OffsetRange[] offsetRanges = { - OffsetRange.create(topic1, 0, 0, 1), - OffsetRange.create(topic2, 0, 0, 1) - }; - - Map emptyLeaders = new HashMap<>(); - Map leaders = new HashMap<>(); - String[] hostAndPort = kafkaTestUtils.brokerAddress().split(":"); - Broker broker = Broker.create(hostAndPort[0], Integer.parseInt(hostAndPort[1])); - leaders.put(new TopicAndPartition(topic1, 0), broker); - leaders.put(new TopicAndPartition(topic2, 0), broker); - - JavaRDD rdd1 = KafkaUtils.createRDD( - sc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - offsetRanges - ).map( - new Function, String>() { - @Override - public String call(Tuple2 kv) { - return kv._2(); - } - } - ); - - JavaRDD rdd2 = KafkaUtils.createRDD( - sc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - String.class, - kafkaParams, - offsetRanges, - emptyLeaders, - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); - - JavaRDD rdd3 = KafkaUtils.createRDD( - sc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - String.class, - kafkaParams, - offsetRanges, - leaders, - new Function, String>() { - @Override - public String call(MessageAndMetadata msgAndMd) { - return msgAndMd.message(); - } - } - ); - - // just making sure the java user apis work; the scala tests handle logic corner cases - long count1 = rdd1.count(); - long count2 = rdd2.count(); - long count3 = rdd3.count(); - Assert.assertTrue(count1 > 0); - Assert.assertEquals(count1, count2); - Assert.assertEquals(count1, count3); - } - - private String[] createTopicAndSendData(String topic) { - String[] data = { topic + "-1", topic + "-2", topic + "-3"}; - kafkaTestUtils.createTopic(topic, 1); - kafkaTestUtils.sendMessages(topic, data); - return data; - } -} diff --git a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java deleted file mode 100644 index 98fe38e826afb..0000000000000 --- a/external/kafka-0-8/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ /dev/null @@ -1,144 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka; - -import java.io.Serializable; -import java.util.*; - -import scala.Tuple2; - -import kafka.serializer.StringDecoder; -import org.junit.After; -import org.junit.Assert; -import org.junit.Before; -import org.junit.Test; - -import org.apache.spark.SparkConf; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.VoidFunction; -import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.Duration; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; - -public class JavaKafkaStreamSuite implements Serializable { - private transient JavaStreamingContext ssc = null; - private transient Random random = new Random(); - private transient KafkaTestUtils kafkaTestUtils = null; - - @Before - public void setUp() { - kafkaTestUtils = new KafkaTestUtils(); - kafkaTestUtils.setup(); - SparkConf sparkConf = new SparkConf() - .setMaster("local[4]").setAppName(this.getClass().getSimpleName()); - ssc = new JavaStreamingContext(sparkConf, new Duration(500)); - } - - @After - public void tearDown() { - if (ssc != null) { - ssc.stop(); - ssc = null; - } - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown(); - kafkaTestUtils = null; - } - } - - @Test - public void testKafkaStream() throws InterruptedException { - String topic = "topic1"; - Map topics = new HashMap<>(); - topics.put(topic, 1); - - Map sent = new HashMap<>(); - sent.put("a", 5); - sent.put("b", 3); - sent.put("c", 10); - - kafkaTestUtils.createTopic(topic, 1); - kafkaTestUtils.sendMessages(topic, sent); - - Map kafkaParams = new HashMap<>(); - kafkaParams.put("zookeeper.connect", kafkaTestUtils.zkAddress()); - kafkaParams.put("group.id", "test-consumer-" + random.nextInt(10000)); - kafkaParams.put("auto.offset.reset", "smallest"); - - JavaPairDStream stream = KafkaUtils.createStream(ssc, - String.class, - String.class, - StringDecoder.class, - StringDecoder.class, - kafkaParams, - topics, - StorageLevel.MEMORY_ONLY_SER()); - - final Map result = Collections.synchronizedMap(new HashMap()); - - JavaDStream words = stream.map( - new Function, String>() { - @Override - public String call(Tuple2 tuple2) { - return tuple2._2(); - } - } - ); - - words.countByValue().foreachRDD(new VoidFunction>() { - @Override - public void call(JavaPairRDD rdd) { - List> ret = rdd.collect(); - for (Tuple2 r : ret) { - if (result.containsKey(r._1())) { - result.put(r._1(), result.get(r._1()) + r._2()); - } else { - result.put(r._1(), r._2()); - } - } - } - } - ); - - ssc.start(); - - long startTime = System.currentTimeMillis(); - AssertionError lastError = null; - while (System.currentTimeMillis() - startTime < 20000) { - try { - Assert.assertEquals(sent.size(), result.size()); - for (Map.Entry e : sent.entrySet()) { - Assert.assertEquals(e.getValue().intValue(), result.get(e.getKey()).intValue()); - } - return; - } catch (AssertionError e) { - lastError = e; - } - Thread.sleep(200); - } - if (lastError != null) { - throw lastError; - } else { - Assert.fail("timeout"); - } - } -} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala deleted file mode 100644 index 3fd37f4c8ac90..0000000000000 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/DirectKafkaStreamSuite.scala +++ /dev/null @@ -1,636 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.File -import java.util.{ Arrays, UUID } -import java.util.concurrent.ConcurrentLinkedQueue -import java.util.concurrent.atomic.AtomicLong - -import scala.collection.JavaConverters._ -import scala.concurrent.duration._ -import scala.language.postfixOps - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.StringDecoder -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.scalatest.concurrent.Eventually - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.streaming.{Milliseconds, StreamingContext, Time} -import org.apache.spark.streaming.dstream.DStream -import org.apache.spark.streaming.scheduler._ -import org.apache.spark.streaming.scheduler.rate.RateEstimator -import org.apache.spark.util.Utils - -class DirectKafkaStreamSuite - extends SparkFunSuite - with BeforeAndAfter - with BeforeAndAfterAll - with Eventually - with Logging { - val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - - private var ssc: StreamingContext = _ - private var testDir: File = _ - - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll { - super.beforeAll() - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll { - try { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } finally { - super.afterAll() - } - } - - after { - if (ssc != null) { - ssc.stop(stopSparkContext = true) - } - if (testDir != null) { - Utils.deleteRecursively(testDir) - } - } - - - test("basic stream receiving with multiple topics and smallest starting offset") { - val topics = Set("basic1", "basic2", "basic3") - val data = Map("a" -> 7, "b" -> 9) - topics.foreach { t => - kafkaTestUtils.createTopic(t) - kafkaTestUtils.sendMessages(t, data) - } - val totalSent = data.values.sum * topics.size - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics) - } - - val allReceived = new ConcurrentLinkedQueue[(String, String)]() - - // hold a reference to the current offset ranges, so it can be used downstream - var offsetRanges = Array[OffsetRange]() - - stream.transform { rdd => - // Get the offset ranges in the RDD - offsetRanges = rdd.asInstanceOf[HasOffsetRanges].offsetRanges - rdd - }.foreachRDD { rdd => - for (o <- offsetRanges) { - logInfo(s"${o.topic} ${o.partition} ${o.fromOffset} ${o.untilOffset}") - } - val collected = rdd.mapPartitionsWithIndex { (i, iter) => - // For each partition, get size of the range in the partition, - // and the number of items in the partition - val off = offsetRanges(i) - val all = iter.toSeq - val partSize = all.size - val rangeSize = off.untilOffset - off.fromOffset - Iterator((partSize, rangeSize)) - }.collect - - // Verify whether number of elements in each partition - // matches with the corresponding offset range - collected.foreach { case (partSize, rangeSize) => - assert(partSize === rangeSize, "offset ranges are wrong") - } - } - stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } - ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(allReceived.size === totalSent, - "didn't get expected number of messages, messages:\n" + - allReceived.asScala.mkString("\n")) - } - ssc.stop() - } - - test("receiving from largest starting offset") { - val topic = "largest" - val topicPartition = TopicAndPartition(topic, 0) - val data = Map("a" -> 10) - kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) - def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset - } - - // Send some initial messages before starting context - kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { - assert(getLatestOffset() > 3) - } - val offsetBeforeStart = getLatestOffset() - - // Setup context and kafka stream with largest offset - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) - - val collectedData = new ConcurrentLinkedQueue[String]() - stream.map { _._2 }.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } - ssc.start() - val newData = Map("b" -> 10) - kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - collectedData.contains("b") - } - assert(!collectedData.contains("a")) - ssc.stop() - } - - - test("creating stream by offset") { - val topic = "offset" - val topicPartition = TopicAndPartition(topic, 0) - val data = Map("a" -> 10) - kafkaTestUtils.createTopic(topic) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "largest" - ) - val kc = new KafkaCluster(kafkaParams) - def getLatestOffset(): Long = { - kc.getLatestLeaderOffsets(Set(topicPartition)).right.get(topicPartition).offset - } - - // Send some initial messages before starting context - kafkaTestUtils.sendMessages(topic, data) - eventually(timeout(10 seconds), interval(20 milliseconds)) { - assert(getLatestOffset() >= 10) - } - val offsetBeforeStart = getLatestOffset() - - // Setup context and kafka stream with largest offset - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder, String]( - ssc, kafkaParams, Map(topicPartition -> 11L), - (m: MessageAndMetadata[String, String]) => m.message()) - } - assert( - stream.asInstanceOf[DirectKafkaInputDStream[_, _, _, _, _]] - .fromOffsets(topicPartition) >= offsetBeforeStart, - "Start offset not from latest" - ) - - val collectedData = new ConcurrentLinkedQueue[String]() - stream.foreachRDD { rdd => collectedData.addAll(Arrays.asList(rdd.collect(): _*)) } - ssc.start() - val newData = Map("b" -> 10) - kafkaTestUtils.sendMessages(topic, newData) - eventually(timeout(10 seconds), interval(50 milliseconds)) { - collectedData.contains("b") - } - assert(!collectedData.contains("a")) - ssc.stop() - } - - // Test to verify the offset ranges can be recovered from the checkpoints - test("offset recovery") { - val topic = "recovery" - kafkaTestUtils.createTopic(topic) - testDir = Utils.createTempDir() - - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - // Send data to Kafka and wait for it to be received - def sendData(data: Seq[Int]) { - val strings = data.map { _.toString} - kafkaTestUtils.sendMessages(topic, strings.map { _ -> 1}.toMap) - } - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(100)) - val kafkaStream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - val keyedStream = kafkaStream.map { v => "key" -> v._2.toInt } - val stateStream = keyedStream.updateStateByKey { (values: Seq[Int], state: Option[Int]) => - Some(values.sum + state.getOrElse(0)) - } - ssc.checkpoint(testDir.getAbsolutePath) - - // This is ensure all the data is eventually receiving only once - stateStream.foreachRDD { (rdd: RDD[(String, Int)]) => - rdd.collect().headOption.foreach { x => - DirectKafkaStreamSuite.total.set(x._2) - } - } - ssc.start() - - // Send some data - for (i <- (1 to 10).grouped(4)) { - sendData(i) - } - - eventually(timeout(20 seconds), interval(50 milliseconds)) { - assert(DirectKafkaStreamSuite.total.get === (1 to 10).sum) - } - - ssc.stop() - - // Verify that offset ranges were generated - // Since "offsetRangesAfterStop" will be used to compare with "recoveredOffsetRanges", we should - // collect offset ranges after stopping. Otherwise, because new RDDs keep being generated before - // stopping, we may not be able to get the latest RDDs, then "recoveredOffsetRanges" will - // contain something not in "offsetRangesAfterStop". - val offsetRangesAfterStop = getOffsetRanges(kafkaStream) - assert(offsetRangesAfterStop.size >= 1, "No offset ranges generated") - assert( - offsetRangesAfterStop.head._2.forall { _.fromOffset === 0 }, - "starting offset not zero" - ) - - logInfo("====== RESTARTING ========") - - // Recover context from checkpoints - ssc = new StreamingContext(testDir.getAbsolutePath) - val recoveredStream = ssc.graph.getInputStreams().head.asInstanceOf[DStream[(String, String)]] - - // Verify offset ranges have been recovered - val recoveredOffsetRanges = getOffsetRanges(recoveredStream).map { x => (x._1, x._2.toSet) } - assert(recoveredOffsetRanges.size > 0, "No offset ranges recovered") - val earlierOffsetRanges = offsetRangesAfterStop.map { x => (x._1, x._2.toSet) } - assert( - recoveredOffsetRanges.forall { or => - earlierOffsetRanges.contains((or._1, or._2)) - }, - "Recovered ranges are not the same as the ones generated\n" + - s"recoveredOffsetRanges: $recoveredOffsetRanges\n" + - s"earlierOffsetRanges: $earlierOffsetRanges" - ) - // Restart context, give more data and verify the total at the end - // If the total is write that means each records has been received only once - ssc.start() - for (i <- (11 to 20).grouped(4)) { - sendData(i) - } - - eventually(timeout(20 seconds), interval(50 milliseconds)) { - assert(DirectKafkaStreamSuite.total.get === (1 to 20).sum) - } - ssc.stop() - } - - test("Direct Kafka stream report input information") { - val topic = "report-test" - val data = Map("a" -> 7, "b" -> 9) - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, data) - - val totalSent = data.values.sum - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - import DirectKafkaStreamSuite._ - ssc = new StreamingContext(sparkConf, Milliseconds(200)) - val collector = new InputInfoCollector - ssc.addStreamingListener(collector) - - val stream = withClue("Error creating direct stream") { - KafkaUtils.createDirectStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Set(topic)) - } - - val allReceived = new ConcurrentLinkedQueue[(String, String)] - - stream.foreachRDD { rdd => allReceived.addAll(Arrays.asList(rdd.collect(): _*)) } - ssc.start() - eventually(timeout(20000.milliseconds), interval(200.milliseconds)) { - assert(allReceived.size === totalSent, - "didn't get expected number of messages, messages:\n" + - allReceived.asScala.mkString("\n")) - - // Calculate all the record number collected in the StreamingListener. - assert(collector.numRecordsSubmitted.get() === totalSent) - assert(collector.numRecordsStarted.get() === totalSent) - assert(collector.numRecordsCompleted.get() === totalSent) - } - ssc.stop() - } - - test("maxMessagesPerPartition with backpressure disabled") { - val topic = "maxMessagesPerPartition" - val kafkaStream = getDirectKafkaStream(topic, None) - - val input = Map(TopicAndPartition(topic, 0) -> 50L, TopicAndPartition(topic, 1) -> 50L) - assert(kafkaStream.maxMessagesPerPartition(input).get == - Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) - } - - test("maxMessagesPerPartition with no lag") { - val topic = "maxMessagesPerPartition" - val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 100)) - val kafkaStream = getDirectKafkaStream(topic, rateController) - - val input = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) - assert(kafkaStream.maxMessagesPerPartition(input).isEmpty) - } - - test("maxMessagesPerPartition respects max rate") { - val topic = "maxMessagesPerPartition" - val rateController = Some(new ConstantRateController(0, new ConstantEstimator(100), 1000)) - val kafkaStream = getDirectKafkaStream(topic, rateController) - - val input = Map(TopicAndPartition(topic, 0) -> 1000L, TopicAndPartition(topic, 1) -> 1000L) - assert(kafkaStream.maxMessagesPerPartition(input).get == - Map(TopicAndPartition(topic, 0) -> 10L, TopicAndPartition(topic, 1) -> 10L)) - } - - test("using rate controller") { - val topic = "backpressure" - val topicPartitions = Set(TopicAndPartition(topic, 0), TopicAndPartition(topic, 1)) - kafkaTestUtils.createTopic(topic, 2) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - val batchIntervalMilliseconds = 100 - val estimator = new ConstantEstimator(100) - val messages = Map("foo" -> 200) - kafkaTestUtils.sendMessages(topic, messages) - - val sparkConf = new SparkConf() - // Safe, even with streaming, because we're using the direct API. - // Using 1 core is useful to make the test more predictable. - .setMaster("local[1]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.kafka.maxRatePerPartition", "100") - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) - - val kafkaStream = withClue("Error creating direct stream") { - val kc = new KafkaCluster(kafkaParams) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val m = kc.getEarliestLeaderOffsets(topicPartitions) - .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) - - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, m, messageHandler) { - override protected[streaming] val rateController = - Some(new DirectKafkaRateController(id, estimator)) - } - } - - val collectedData = new ConcurrentLinkedQueue[Array[String]]() - - // Used for assertion failure messages. - def dataToString: String = - collectedData.asScala.map(_.mkString("[", ",", "]")).mkString("{", ", ", "}") - - // This is to collect the raw data received from Kafka - kafkaStream.foreachRDD { (rdd: RDD[(String, String)], time: Time) => - val data = rdd.map { _._2 }.collect() - collectedData.add(data) - } - - ssc.start() - - // Try different rate limits. - // Wait for arrays of data to appear matching the rate. - Seq(100, 50, 20).foreach { rate => - collectedData.clear() // Empty this buffer on each pass. - estimator.updateRate(rate) // Set a new rate. - // Expect blocks of data equal to "rate", scaled by the interval length in secs. - val expectedSize = Math.round(rate * batchIntervalMilliseconds * 0.001) - eventually(timeout(5.seconds), interval(batchIntervalMilliseconds.milliseconds)) { - // Assert that rate estimator values are used to determine maxMessagesPerPartition. - // Funky "-" in message makes the complete assertion message read better. - assert(collectedData.asScala.exists(_.size == expectedSize), - s" - No arrays of size $expectedSize for rate $rate found in $dataToString") - } - } - - ssc.stop() - } - - test("use backpressure.initialRate with backpressure") { - backpressureTest(maxRatePerPartition = 1000, initialRate = 500, maxMessagesPerPartition = 250) - } - - test("backpressure.initialRate should honor maxRatePerPartition") { - backpressureTest(maxRatePerPartition = 300, initialRate = 1000, maxMessagesPerPartition = 150) - } - - private def backpressureTest( - maxRatePerPartition: Int, - initialRate: Int, - maxMessagesPerPartition: Int) = { - - val topic = UUID.randomUUID().toString - val topicPartitions = Set(TopicAndPartition(topic, 0)) - kafkaTestUtils.createTopic(topic, 1) - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - val sparkConf = new SparkConf() - // Safe, even with streaming, because we're using the direct API. - // Using 1 core is useful to make the test more predictable. - .setMaster("local[1]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.backpressure.enabled", "true") - .set("spark.streaming.backpressure.initialRate", initialRate.toString) - .set("spark.streaming.kafka.maxRatePerPartition", maxRatePerPartition.toString) - - val messages = Map("foo" -> 5000) - kafkaTestUtils.sendMessages(topic, messages) - - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - - val kafkaStream = withClue("Error creating direct stream") { - val kc = new KafkaCluster(kafkaParams) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val m = kc.getEarliestLeaderOffsets(topicPartitions) - .fold(e => Map.empty[TopicAndPartition, Long], m => m.mapValues(lo => lo.offset)) - - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, m, messageHandler) - } - kafkaStream.start() - - val input = Map(new TopicAndPartition(topic, 0) -> 1000L) - - assert(kafkaStream.maxMessagesPerPartition(input).get == - Map(new TopicAndPartition(topic, 0) -> maxMessagesPerPartition)) - - kafkaStream.stop() - } - - test("maxMessagesPerPartition with zero offset and rate equal to one") { - val topic = "backpressure" - val kafkaParams = Map( - "metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "auto.offset.reset" -> "smallest" - ) - - val batchIntervalMilliseconds = 60000 - val sparkConf = new SparkConf() - // Safe, even with streaming, because we're using the direct API. - // Using 1 core is useful to make the test more predictable. - .setMaster("local[1]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.kafka.maxRatePerPartition", "100") - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) - val estimatedRate = 1L - val kafkaStream = withClue("Error creating direct stream") { - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - val fromOffsets = Map( - TopicAndPartition(topic, 0) -> 0L, - TopicAndPartition(topic, 1) -> 0L, - TopicAndPartition(topic, 2) -> 0L, - TopicAndPartition(topic, 3) -> 0L - ) - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, kafkaParams, fromOffsets, messageHandler) { - override protected[streaming] val rateController = - Some(new DirectKafkaRateController(id, null) { - override def getLatestRate() = estimatedRate - }) - } - } - - val offsets = Map( - TopicAndPartition(topic, 0) -> 0L, - TopicAndPartition(topic, 1) -> 100L, - TopicAndPartition(topic, 2) -> 200L, - TopicAndPartition(topic, 3) -> 300L - ) - val result = kafkaStream.maxMessagesPerPartition(offsets) - val expected = Map( - TopicAndPartition(topic, 0) -> 1L, - TopicAndPartition(topic, 1) -> 10L, - TopicAndPartition(topic, 2) -> 20L, - TopicAndPartition(topic, 3) -> 30L - ) - assert(result.contains(expected), s"Number of messages per partition must be at least 1") - } - - /** Get the generated offset ranges from the DirectKafkaStream */ - private def getOffsetRanges[K, V]( - kafkaStream: DStream[(K, V)]): Seq[(Time, Array[OffsetRange])] = { - kafkaStream.generatedRDDs.mapValues { rdd => - rdd.asInstanceOf[KafkaRDD[K, V, _, _, (K, V)]].offsetRanges - }.toSeq.sortBy { _._1 } - } - - private def getDirectKafkaStream(topic: String, mockRateController: Option[RateController]) = { - val batchIntervalMilliseconds = 100 - - val sparkConf = new SparkConf() - .setMaster("local[1]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.kafka.maxRatePerPartition", "100") - - // Setup the streaming context - ssc = new StreamingContext(sparkConf, Milliseconds(batchIntervalMilliseconds)) - - val earliestOffsets = Map(TopicAndPartition(topic, 0) -> 0L, TopicAndPartition(topic, 1) -> 0L) - val messageHandler = (mmd: MessageAndMetadata[String, String]) => (mmd.key, mmd.message) - new DirectKafkaInputDStream[String, String, StringDecoder, StringDecoder, (String, String)]( - ssc, Map[String, String](), earliestOffsets, messageHandler) { - override protected[streaming] val rateController = mockRateController - } - } -} - -object DirectKafkaStreamSuite { - val total = new AtomicLong(-1L) - - class InputInfoCollector extends StreamingListener { - val numRecordsSubmitted = new AtomicLong(0L) - val numRecordsStarted = new AtomicLong(0L) - val numRecordsCompleted = new AtomicLong(0L) - - override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted): Unit = { - numRecordsSubmitted.addAndGet(batchSubmitted.batchInfo.numRecords) - } - - override def onBatchStarted(batchStarted: StreamingListenerBatchStarted): Unit = { - numRecordsStarted.addAndGet(batchStarted.batchInfo.numRecords) - } - - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted): Unit = { - numRecordsCompleted.addAndGet(batchCompleted.batchInfo.numRecords) - } - } -} - -private[streaming] class ConstantEstimator(@volatile private var rate: Long) - extends RateEstimator { - - def updateRate(newRate: Long): Unit = { - rate = newRate - } - - def compute( - time: Long, - elements: Long, - processingDelay: Long, - schedulingDelay: Long): Option[Double] = Some(rate) -} - -private[streaming] class ConstantRateController(id: Int, estimator: RateEstimator, rate: Long) - extends RateController(id, estimator) { - override def publish(rate: Long): Unit = () - override def getLatestRate(): Long = rate -} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala deleted file mode 100644 index 73d528518d486..0000000000000 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaClusterSuite.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.util.Random - -import kafka.common.TopicAndPartition -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark.SparkFunSuite - -class KafkaClusterSuite extends SparkFunSuite with BeforeAndAfterAll { - private val topic = "kcsuitetopic" + Random.nextInt(10000) - private val topicAndPartition = TopicAndPartition(topic, 0) - private var kc: KafkaCluster = null - - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll() { - super.beforeAll() - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, Map("a" -> 1)) - kc = new KafkaCluster(Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress)) - } - - override def afterAll() { - try { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } finally { - super.afterAll() - } - } - - test("metadata apis") { - val leader = kc.findLeaders(Set(topicAndPartition)).right.get(topicAndPartition) - val leaderAddress = s"${leader._1}:${leader._2}" - assert(leaderAddress === kafkaTestUtils.brokerAddress, "didn't get leader") - - val parts = kc.getPartitions(Set(topic)).right.get - assert(parts(topicAndPartition), "didn't get partitions") - - val err = kc.getPartitions(Set(topic + "BAD")) - assert(err.isLeft, "getPartitions for a nonexistant topic should be an error") - } - - test("leader offset apis") { - val earliest = kc.getEarliestLeaderOffsets(Set(topicAndPartition)).right.get - assert(earliest(topicAndPartition).offset === 0, "didn't get earliest") - - val latest = kc.getLatestLeaderOffsets(Set(topicAndPartition)).right.get - assert(latest(topicAndPartition).offset === 1, "didn't get latest") - } - - test("consumer offset apis") { - val group = "kcsuitegroup" + Random.nextInt(10000) - - val offset = Random.nextInt(10000) - - val set = kc.setConsumerOffsets(group, Map(topicAndPartition -> offset)) - assert(set.isRight, "didn't set consumer offsets") - - val get = kc.getConsumerOffsets(group, Set(topicAndPartition)).right.get - assert(get(topicAndPartition) === offset, "didn't get consumer offsets") - } -} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala deleted file mode 100644 index 72f954149fefe..0000000000000 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaRDDSuite.scala +++ /dev/null @@ -1,182 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.util.Random - -import kafka.common.TopicAndPartition -import kafka.message.MessageAndMetadata -import kafka.serializer.StringDecoder -import org.scalatest.BeforeAndAfterAll - -import org.apache.spark._ - -class KafkaRDDSuite extends SparkFunSuite with BeforeAndAfterAll { - - private var kafkaTestUtils: KafkaTestUtils = _ - - private val sparkConf = new SparkConf().setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - private var sc: SparkContext = _ - - override def beforeAll { - super.beforeAll() - sc = new SparkContext(sparkConf) - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll { - try { - try { - if (sc != null) { - sc.stop - sc = null - } - } finally { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - } finally { - super.afterAll() - } - } - - test("basic usage") { - val topic = s"topicbasic-${Random.nextInt}-${System.currentTimeMillis}" - kafkaTestUtils.createTopic(topic) - val messages = Array("the", "quick", "brown", "fox") - kafkaTestUtils.sendMessages(topic, messages) - - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") - - val offsetRanges = Array(OffsetRange(topic, 0, 0, messages.size)) - - val rdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, offsetRanges) - - val received = rdd.map(_._2).collect.toSet - assert(received === messages.toSet) - - // size-related method optimizations return sane results - assert(rdd.count === messages.size) - assert(rdd.countApprox(0).getFinalValue.mean === messages.size) - assert(!rdd.isEmpty) - assert(rdd.take(1).size === 1) - assert(rdd.take(1).head._2 === messages.head) - assert(rdd.take(messages.size + 10).size === messages.size) - - val emptyRdd = KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, Array(OffsetRange(topic, 0, 0, 0))) - - assert(emptyRdd.isEmpty) - - // invalid offset ranges throw exceptions - val badRanges = Array(OffsetRange(topic, 0, 0, messages.size + 1)) - intercept[SparkException] { - KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder]( - sc, kafkaParams, badRanges) - } - } - - test("iterator boundary conditions") { - // the idea is to find e.g. off-by-one errors between what kafka has available and the rdd - val topic = s"topicboundary-${Random.nextInt}-${System.currentTimeMillis}" - val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - kafkaTestUtils.createTopic(topic) - - val kafkaParams = Map("metadata.broker.list" -> kafkaTestUtils.brokerAddress, - "group.id" -> s"test-consumer-${Random.nextInt}-${System.currentTimeMillis}") - - val kc = new KafkaCluster(kafkaParams) - - // this is the "lots of messages" case - kafkaTestUtils.sendMessages(topic, sent) - val sentCount = sent.values.sum - - // rdd defined from leaders after sending messages, should get the number sent - val rdd = getRdd(kc, Set(topic)) - - assert(rdd.isDefined) - - val ranges = rdd.get.asInstanceOf[HasOffsetRanges].offsetRanges - val rangeCount = ranges.map(o => o.untilOffset - o.fromOffset).sum - - assert(rangeCount === sentCount, "offset range didn't include all sent messages") - assert(rdd.get.count === sentCount, "didn't get all sent messages") - - val rangesMap = ranges.map(o => TopicAndPartition(o.topic, o.partition) -> o.untilOffset).toMap - - // make sure consumer offsets are committed before the next getRdd call - kc.setConsumerOffsets(kafkaParams("group.id"), rangesMap).fold( - err => throw new Exception(err.mkString("\n")), - _ => () - ) - - // this is the "0 messages" case - val rdd2 = getRdd(kc, Set(topic)) - // shouldn't get anything, since message is sent after rdd was defined - val sentOnlyOne = Map("d" -> 1) - - kafkaTestUtils.sendMessages(topic, sentOnlyOne) - - assert(rdd2.isDefined) - assert(rdd2.get.count === 0, "got messages when there shouldn't be any") - - // this is the "exactly 1 message" case, namely the single message from sentOnlyOne above - val rdd3 = getRdd(kc, Set(topic)) - // send lots of messages after rdd was defined, they shouldn't show up - kafkaTestUtils.sendMessages(topic, Map("extra" -> 22)) - - assert(rdd3.isDefined) - assert(rdd3.get.count === sentOnlyOne.values.sum, "didn't get exactly one message") - - } - - // get an rdd from the committed consumer offsets until the latest leader offsets, - private def getRdd(kc: KafkaCluster, topics: Set[String]) = { - val groupId = kc.kafkaParams("group.id") - def consumerOffsets(topicPartitions: Set[TopicAndPartition]) = { - kc.getConsumerOffsets(groupId, topicPartitions).right.toOption.orElse( - kc.getEarliestLeaderOffsets(topicPartitions).right.toOption.map { offs => - offs.map(kv => kv._1 -> kv._2.offset) - } - ) - } - kc.getPartitions(topics).right.toOption.flatMap { topicPartitions => - consumerOffsets(topicPartitions).flatMap { from => - kc.getLatestLeaderOffsets(topicPartitions).right.toOption.map { until => - val offsetRanges = from.map { case (tp: TopicAndPartition, fromOffset: Long) => - OffsetRange(tp.topic, tp.partition, fromOffset, until(tp).offset) - }.toArray - - val leaders = until.map { case (tp: TopicAndPartition, lo: KafkaCluster.LeaderOffset) => - tp -> Broker(lo.host, lo.port) - }.toMap - - KafkaUtils.createRDD[String, String, StringDecoder, StringDecoder, String]( - sc, kc.kafkaParams, offsetRanges, leaders, - (mmd: MessageAndMetadata[String, String]) => s"${mmd.offset} ${mmd.message}") - } - } - } - } -} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala deleted file mode 100644 index ed130f5990955..0000000000000 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ /dev/null @@ -1,92 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.util.Random - -import kafka.serializer.StringDecoder -import org.scalatest.BeforeAndAfterAll -import org.scalatest.concurrent.Eventually - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} - -class KafkaStreamSuite extends SparkFunSuite with Eventually with BeforeAndAfterAll { - private var ssc: StreamingContext = _ - private var kafkaTestUtils: KafkaTestUtils = _ - - override def beforeAll(): Unit = { - super.beforeAll() - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - } - - override def afterAll(): Unit = { - try { - try { - if (ssc != null) { - ssc.stop() - ssc = null - } - } finally { - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } - } finally { - super.afterAll() - } - } - - test("Kafka input stream") { - val sparkConf = new SparkConf().setMaster("local[4]").setAppName(this.getClass.getSimpleName) - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - val topic = "topic1" - val sent = Map("a" -> 5, "b" -> 3, "c" -> 10) - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, sent) - - val kafkaParams = Map("zookeeper.connect" -> kafkaTestUtils.zkAddress, - "group.id" -> s"test-consumer-${Random.nextInt(10000)}", - "auto.offset.reset" -> "smallest") - - val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() - stream.map(_._2).countByValue().foreachRDD { r => - r.collect().foreach { kv => - result.synchronized { - val count = result.getOrElseUpdate(kv._1, 0) + kv._2 - result.put(kv._1, count) - } - } - } - - ssc.start() - - eventually(timeout(10000 milliseconds), interval(100 milliseconds)) { - assert(result.synchronized { sent === result }) - } - ssc.stop() - } -} diff --git a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala b/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala deleted file mode 100644 index 5da5ea49d77ed..0000000000000 --- a/external/kafka-0-8/src/test/scala/org/apache/spark/streaming/kafka/ReliableKafkaStreamSuite.scala +++ /dev/null @@ -1,153 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.streaming.kafka - -import java.io.File - -import scala.collection.mutable -import scala.concurrent.duration._ -import scala.language.postfixOps -import scala.util.Random - -import kafka.serializer.StringDecoder -import kafka.utils.{ZKGroupTopicDirs, ZkUtils} -import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} -import org.scalatest.concurrent.Eventually - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.{Milliseconds, StreamingContext} -import org.apache.spark.util.Utils - -class ReliableKafkaStreamSuite extends SparkFunSuite - with BeforeAndAfterAll with BeforeAndAfter with Eventually { - - private val sparkConf = new SparkConf() - .setMaster("local[4]") - .setAppName(this.getClass.getSimpleName) - .set("spark.streaming.receiver.writeAheadLog.enable", "true") - private val data = Map("a" -> 10, "b" -> 10, "c" -> 10) - - private var kafkaTestUtils: KafkaTestUtils = _ - - private var groupId: String = _ - private var kafkaParams: Map[String, String] = _ - private var ssc: StreamingContext = _ - private var tempDirectory: File = null - - override def beforeAll(): Unit = { - super.beforeAll() - kafkaTestUtils = new KafkaTestUtils - kafkaTestUtils.setup() - - groupId = s"test-consumer-${Random.nextInt(10000)}" - kafkaParams = Map( - "zookeeper.connect" -> kafkaTestUtils.zkAddress, - "group.id" -> groupId, - "auto.offset.reset" -> "smallest" - ) - - tempDirectory = Utils.createTempDir() - } - - override def afterAll(): Unit = { - try { - Utils.deleteRecursively(tempDirectory) - - if (kafkaTestUtils != null) { - kafkaTestUtils.teardown() - kafkaTestUtils = null - } - } finally { - super.afterAll() - } - } - - before { - ssc = new StreamingContext(sparkConf, Milliseconds(500)) - ssc.checkpoint(tempDirectory.getAbsolutePath) - } - - after { - if (ssc != null) { - ssc.stop(stopSparkContext = true) - ssc = null - } - } - - test("Reliable Kafka input stream with single topic") { - val topic = "test-topic" - kafkaTestUtils.createTopic(topic) - kafkaTestUtils.sendMessages(topic, data) - - // Verify whether the offset of this group/topic/partition is 0 before starting. - assert(getCommitOffset(groupId, topic, 0) === None) - - val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, Map(topic -> 1), StorageLevel.MEMORY_ONLY) - val result = new mutable.HashMap[String, Long]() - stream.map { case (k, v) => v }.foreachRDD { r => - val ret = r.collect() - ret.foreach { v => - val count = result.getOrElseUpdate(v, 0) + 1 - result.put(v, count) - } - } - ssc.start() - - eventually(timeout(20000 milliseconds), interval(200 milliseconds)) { - // A basic process verification for ReliableKafkaReceiver. - // Verify whether received message number is equal to the sent message number. - assert(data.size === result.size) - // Verify whether each message is the same as the data to be verified. - data.keys.foreach { k => assert(data(k) === result(k).toInt) } - // Verify the offset number whether it is equal to the total message number. - assert(getCommitOffset(groupId, topic, 0) === Some(29L)) - } - } - - test("Reliable Kafka input stream with multiple topics") { - val topics = Map("topic1" -> 1, "topic2" -> 1, "topic3" -> 1) - topics.foreach { case (t, _) => - kafkaTestUtils.createTopic(t) - kafkaTestUtils.sendMessages(t, data) - } - - // Before started, verify all the group/topic/partition offsets are 0. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === None) } - - // Consuming all the data sent to the broker which will potential commit the offsets internally. - val stream = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( - ssc, kafkaParams, topics, StorageLevel.MEMORY_ONLY) - stream.foreachRDD(_ => Unit) - ssc.start() - - eventually(timeout(20000 milliseconds), interval(100 milliseconds)) { - // Verify the offset for each group/topic to see whether they are equal to the expected one. - topics.foreach { case (t, _) => assert(getCommitOffset(groupId, t, 0) === Some(29L)) } - } - } - - - /** Getting partition offset from Zookeeper. */ - private def getCommitOffset(groupId: String, topic: String, partition: Int): Option[Long] = { - val topicDirs = new ZKGroupTopicDirs(groupId, topic) - val zkPath = s"${topicDirs.consumerOffsetDir}/$partition" - ZkUtils.readDataMaybeNull(kafkaTestUtils.zookeeperClient, zkPath)._1.map(_.toLong) - } -} diff --git a/external/kinesis-asl-assembly/pom.xml b/external/kinesis-asl-assembly/pom.xml index 0bf4c265939e7..0ce922349ea66 100644 --- a/external/kinesis-asl-assembly/pom.xml +++ b/external/kinesis-asl-assembly/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-streaming-kinesis-asl-assembly_2.11 + spark-streaming-kinesis-asl-assembly_2.12 jar Spark Project Kinesis Assembly http://spark.apache.org/ diff --git a/external/kinesis-asl/pom.xml b/external/kinesis-asl/pom.xml index 032aca9077e20..7d69764b77de7 100644 --- a/external/kinesis-asl/pom.xml +++ b/external/kinesis-asl/pom.xml @@ -19,13 +19,13 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-streaming-kinesis-asl_2.11 + spark-streaming-kinesis-asl_2.12 jar Spark Kinesis Integration @@ -69,6 +69,13 @@ ${aws.kinesis.producer.version} test + + + com.fasterxml.jackson.dataformat + jackson-dataformat-cbor + ${fasterxml.jackson.version} + org.mockito mockito-core diff --git a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java index 626bde48e1a86..86c42df9e8435 100644 --- a/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java +++ b/external/kinesis-asl/src/main/java/org/apache/spark/examples/streaming/JavaKinesisWordCountASL.java @@ -145,7 +145,7 @@ public static void main(String[] args) throws Exception { // Union all the streams if there is more than 1 stream JavaDStream unionStreams; if (streamsList.size() > 1) { - unionStreams = jssc.union(streamsList.get(0), streamsList.subList(1, streamsList.size())); + unionStreams = jssc.union(streamsList.toArray(new JavaDStream[0])); } else { // Otherwise, just use the 1 stream unionStreams = streamsList.get(0); diff --git a/external/kinesis-asl/src/main/resources/log4j.properties b/external/kinesis-asl/src/main/resources/log4j.properties index 4f5ea7bafe48f..e450494917341 100644 --- a/external/kinesis-asl/src/main/resources/log4j.properties +++ b/external/kinesis-asl/src/main/resources/log4j.properties @@ -31,7 +31,7 @@ log4j.appender.console.layout=org.apache.log4j.PatternLayout log4j.appender.console.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss} %p %c{1}: %m%n # Settings to quiet third party logs that are too verbose -log4j.logger.org.spark_project.jetty=WARN -log4j.logger.org.spark_project.jetty.util.component.AbstractLifeCycle=ERROR +log4j.logger.org.sparkproject.jetty=WARN +log4j.logger.org.sparkproject.jetty.util.component.AbstractLifeCycle=ERROR log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala index 88b294246bb30..5072b3ae21d87 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisBackedBlockRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming.kinesis +import java.util.concurrent.TimeUnit + import scala.collection.JavaConverters._ import scala.reflect.ClassTag import scala.util.control.NonFatal @@ -251,13 +253,16 @@ class KinesisSequenceRangeIterator( /** Helper method to retry Kinesis API request with exponential backoff and timeouts */ private def retryOrTimeout[T](message: String)(body: => T): T = { - val startTimeMs = System.currentTimeMillis() + val startTimeNs = System.nanoTime() var retryCount = 0 var result: Option[T] = None var lastError: Throwable = null var waitTimeInterval = kinesisReadConfigs.retryWaitTimeMs - def isTimedOut = (System.currentTimeMillis() - startTimeMs) >= kinesisReadConfigs.retryTimeoutMs + def isTimedOut = { + val retryTimeoutNs = TimeUnit.MILLISECONDS.toNanos(kinesisReadConfigs.retryTimeoutMs) + (System.nanoTime() - startTimeNs) >= retryTimeoutNs + } def isMaxRetryDone = retryCount >= kinesisReadConfigs.maxRetries while (result.isEmpty && !isTimedOut && !isMaxRetryDone) { diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala index 1ffec01df9f00..d4a428f45c110 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisInputDStream.scala @@ -22,7 +22,7 @@ import scala.reflect.ClassTag import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream import com.amazonaws.services.kinesis.model.Record -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, StorageLevel} import org.apache.spark.streaming.{Duration, StreamingContext, Time} @@ -84,14 +84,14 @@ private[kinesis] class KinesisInputDStream[T: ClassTag]( } } -@InterfaceStability.Evolving +@Evolving object KinesisInputDStream { /** * Builder for [[KinesisInputDStream]] instances. * * @since 2.2.0 */ - @InterfaceStability.Evolving + @Evolving class Builder { // Required params private var streamingContext: Option[StreamingContext] = None diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala index 73ac7a3cd2355..2ac83c8ee6b38 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/KinesisTestUtils.scala @@ -183,9 +183,8 @@ private[kinesis] class KinesisTestUtils(streamShardCount: Int = 2) extends Loggi } private def waitForStreamToBeActive(streamNameToWaitFor: String): Unit = { - val startTime = System.currentTimeMillis() - val endTime = startTime + TimeUnit.SECONDS.toMillis(createStreamTimeoutSeconds) - while (System.currentTimeMillis() < endTime) { + val startTimeNs = System.nanoTime() + while (System.nanoTime() - startTimeNs < TimeUnit.SECONDS.toNanos(createStreamTimeoutSeconds)) { Thread.sleep(TimeUnit.SECONDS.toMillis(describeStreamPollTimeSeconds)) describeStream(streamNameToWaitFor).foreach { description => val streamStatus = description.getStreamStatus() diff --git a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala index 9facfe8ff2b0f..7488971e61634 100644 --- a/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala +++ b/external/kinesis-asl/src/main/scala/org/apache/spark/streaming/kinesis/SparkAWSCredentials.scala @@ -14,13 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.streaming.kinesis -import scala.collection.JavaConverters._ +package org.apache.spark.streaming.kinesis import com.amazonaws.auth._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging /** @@ -84,14 +83,14 @@ private[kinesis] final case class STSCredentials( } } -@InterfaceStability.Evolving +@Evolving object SparkAWSCredentials { /** * Builder for [[SparkAWSCredentials]] instances. * * @since 2.2.0 */ - @InterfaceStability.Evolving + @Evolving class Builder { private var basicCreds: Option[BasicCredentials] = None private var stsCreds: Option[STSCredentials] = None @@ -102,8 +101,8 @@ object SparkAWSCredentials { * * @note The given AWS keypair will be saved in DStream checkpoints if checkpointing is * enabled. Make sure that your checkpoint directory is secure. Prefer using the - * [[http://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default default provider chain]] - * instead if possible. + * default provider chain instead if possible + * (http://docs.aws.amazon.com/sdk-for-java/v1/developer-guide/credentials.html#credentials-default). * * @param accessKeyId AWS access key ID * @param secretKey AWS secret key diff --git a/external/kinesis-asl/src/test/resources/log4j.properties b/external/kinesis-asl/src/test/resources/log4j.properties index 3706a6e361307..a9166df3a7fb4 100644 --- a/external/kinesis-asl/src/test/resources/log4j.properties +++ b/external/kinesis-asl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala index e26f4477d1d7d..ac0e6a8429d06 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisCheckpointerSuite.scala @@ -21,13 +21,11 @@ import java.util.concurrent.TimeoutException import scala.concurrent.{Await, ExecutionContext, Future} import scala.concurrent.duration._ -import scala.language.postfixOps import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers._ import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.scalatest.{BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually import org.scalatest.mockito.MockitoSugar @@ -88,7 +86,7 @@ class KinesisCheckpointerSuite extends TestSuiteBase kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) clock.advance(5 * checkpointInterval.milliseconds) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { verify(checkpointerMock, times(1)).checkpoint(seqNum) verify(checkpointerMock, times(1)).checkpoint(otherSeqNum) } @@ -109,7 +107,7 @@ class KinesisCheckpointerSuite extends TestSuiteBase kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) clock.advance(checkpointInterval.milliseconds * 5) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { verify(checkpointerMock, atMost(1)).checkpoint(anyString()) } } @@ -124,15 +122,13 @@ class KinesisCheckpointerSuite extends TestSuiteBase test("if checkpointing is going on, wait until finished before removing and checkpointing") { when(receiverMock.getLatestSeqNumToCheckpoint(shardId)) .thenReturn(someSeqNum).thenReturn(someOtherSeqNum) - when(checkpointerMock.checkpoint(anyString)).thenAnswer(new Answer[Unit] { - override def answer(invocations: InvocationOnMock): Unit = { - clock.waitTillTime(clock.getTimeMillis() + checkpointInterval.milliseconds / 2) - } - }) + when(checkpointerMock.checkpoint(anyString)).thenAnswer { (_: InvocationOnMock) => + clock.waitTillTime(clock.getTimeMillis() + checkpointInterval.milliseconds / 2) + } kinesisCheckpointer.setCheckpointer(shardId, checkpointerMock) clock.advance(checkpointInterval.milliseconds) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { verify(checkpointerMock, times(1)).checkpoint(anyString()) } // don't block test thread @@ -141,12 +137,12 @@ class KinesisCheckpointerSuite extends TestSuiteBase intercept[TimeoutException] { // scalastyle:off awaitready - Await.ready(f, 50 millis) + Await.ready(f, 50.milliseconds) // scalastyle:on awaitready } clock.advance(checkpointInterval.milliseconds / 2) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { verify(checkpointerMock, times(1)).checkpoint(anyString) verify(checkpointerMock, times(1)).checkpoint() } diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala index 2fadda271ea28..52690847418ef 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisReceiverSuite.scala @@ -24,9 +24,8 @@ import com.amazonaws.services.kinesis.clientlibrary.exceptions._ import com.amazonaws.services.kinesis.clientlibrary.interfaces.IRecordProcessorCheckpointer import com.amazonaws.services.kinesis.clientlibrary.lib.worker.ShutdownReason import com.amazonaws.services.kinesis.model.Record -import org.mockito.Matchers._ -import org.mockito.Matchers.{eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{anyList, anyString, eq => meq} +import org.mockito.Mockito.{never, times, verify, when} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.mockito.MockitoSugar @@ -96,7 +95,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft recordProcessor.processRecords(batch, checkpointerMock) verify(receiverMock, times(1)).isStopped() - verify(receiverMock, never).addRecords(anyString, anyListOf(classOf[Record])) + verify(receiverMock, never).addRecords(anyString, anyList()) verify(receiverMock, never).setCheckpointer(anyString, meq(checkpointerMock)) } @@ -104,7 +103,7 @@ class KinesisReceiverSuite extends TestSuiteBase with Matchers with BeforeAndAft when(receiverMock.isStopped()).thenReturn(false) when(receiverMock.getCurrentLimit).thenReturn(Int.MaxValue) when( - receiverMock.addRecords(anyString, anyListOf(classOf[Record])) + receiverMock.addRecords(anyString, anyList()) ).thenThrow(new RuntimeException()) intercept[RuntimeException] { diff --git a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala index 6d27445c5b606..51ee7fd213de5 100644 --- a/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala +++ b/external/kinesis-asl/src/test/scala/org/apache/spark/streaming/kinesis/KinesisStreamSuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.streaming.kinesis import scala.collection.mutable import scala.concurrent.duration._ -import scala.language.postfixOps import scala.util.Random import com.amazonaws.services.kinesis.clientlibrary.lib.worker.InitialPositionInStream @@ -152,7 +151,7 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun _.asInstanceOf[KinesisBackedBlockRDDPartition] }.toSeq assert(partitions.map { _.seqNumberRanges } === Seq(seqNumRanges1, seqNumRanges2)) assert(partitions.map { _.blockId } === Seq(blockId1, blockId2)) - assert(partitions.forall { _.isBlockIdValid === true }) + assert(partitions.forall { _.isBlockIdValid }) // Verify that KinesisBackedBlockRDD is generated even when there are no blocks val emptyRDD = kinesisStream.createBlockRDD(time, Seq.empty) @@ -198,10 +197,11 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun ssc.start() val testData = 1 to 10 - eventually(timeout(120 seconds), interval(10 second)) { + eventually(timeout(2.minutes), interval(10.seconds)) { testUtils.pushData(testData, aggregateTestData) - assert(collected.synchronized { collected === testData.toSet }, - "\nData received does not match data sent") + collected.synchronized { + assert(collected === testData.toSet, "\nData received does not match data sent") + } } ssc.stop(stopSparkContext = false) } @@ -217,7 +217,7 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun .initialPosition(new Latest()) .checkpointInterval(Seconds(10)) .storageLevel(StorageLevel.MEMORY_ONLY) - .buildWithMessageHandler(addFive(_)) + .buildWithMessageHandler(addFive) stream shouldBe a [ReceiverInputDStream[_]] @@ -231,11 +231,12 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun ssc.start() val testData = 1 to 10 - eventually(timeout(120 seconds), interval(10 second)) { + eventually(timeout(2.minutes), interval(10.seconds)) { testUtils.pushData(testData, aggregateTestData) val modData = testData.map(_ + 5) - assert(collected.synchronized { collected === modData.toSet }, - "\nData received does not match data sent") + collected.synchronized { + assert(collected === modData.toSet, "\nData received does not match data sent") + } } ssc.stop(stopSparkContext = false) } @@ -316,10 +317,11 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun val testData2 = 11 to 20 val testData3 = 21 to 30 - eventually(timeout(60 seconds), interval(10 second)) { + eventually(timeout(1.minute), interval(10.seconds)) { localTestUtils.pushData(testData1, aggregateTestData) - assert(collected.synchronized { collected === testData1.toSet }, - "\nData received does not match data sent") + collected.synchronized { + assert(collected === testData1.toSet, "\nData received does not match data sent") + } } val shardToSplit = localTestUtils.getShards().head @@ -332,10 +334,12 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun assert(splitCloseShards.size == 1) assert(splitOpenShards.size == 2) - eventually(timeout(60 seconds), interval(10 second)) { + eventually(timeout(1.minute), interval(10.seconds)) { localTestUtils.pushData(testData2, aggregateTestData) - assert(collected.synchronized { collected === (testData1 ++ testData2).toSet }, - "\nData received does not match data sent after splitting a shard") + collected.synchronized { + assert(collected === (testData1 ++ testData2).toSet, + "\nData received does not match data sent after splitting a shard") + } } val Seq(shardToMerge, adjShard) = splitOpenShards @@ -348,10 +352,12 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun assert(mergedCloseShards.size == 3) assert(mergedOpenShards.size == 1) - eventually(timeout(60 seconds), interval(10 second)) { + eventually(timeout(1.minute), interval(10.seconds)) { localTestUtils.pushData(testData3, aggregateTestData) - assert(collected.synchronized { collected === (testData1 ++ testData2 ++ testData3).toSet }, - "\nData received does not match data sent after merging shards") + collected.synchronized { + assert(collected === (testData1 ++ testData2 ++ testData3).toSet, + "\nData received does not match data sent after merging shards") + } } } finally { ssc.stop(stopSparkContext = false) @@ -399,7 +405,7 @@ abstract class KinesisStreamTests(aggregateTestData: Boolean) extends KinesisFun // Run until there are at least 10 batches with some data in them // If this times out because numBatchesWithData is empty, then its likely that foreachRDD // function failed with exceptions, and nothing got added to `collectedData` - eventually(timeout(2 minutes), interval(1 seconds)) { + eventually(timeout(2.minutes), interval(1.second)) { testUtils.pushData(1 to 5, aggregateTestData) assert(isCheckpointPresent && numBatchesWithData > 10) } diff --git a/external/spark-ganglia-lgpl/pom.xml b/external/spark-ganglia-lgpl/pom.xml index 35a55b70baf33..a23d255f9187c 100644 --- a/external/spark-ganglia-lgpl/pom.xml +++ b/external/spark-ganglia-lgpl/pom.xml @@ -19,13 +19,13 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-ganglia-lgpl_2.11 + spark-ganglia-lgpl_2.12 jar Spark Ganglia Integration diff --git a/graph/api/pom.xml b/graph/api/pom.xml new file mode 100644 index 0000000000000..a18c5290195fe --- /dev/null +++ b/graph/api/pom.xml @@ -0,0 +1,55 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.12 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-graph-api_2.12 + jar + Spark Project Graph API + + graph-api + + http://spark.apache.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/graph/cypher/pom.xml b/graph/cypher/pom.xml new file mode 100644 index 0000000000000..12b3832d9777a --- /dev/null +++ b/graph/cypher/pom.xml @@ -0,0 +1,100 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.12 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-cypher_2.12 + jar + Spark Project Cypher + + cypher + + http://spark.apache.org/ + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-graph-api_${scala.binary.version} + ${project.version} + + + org.opencypher + okapi-shade + ${okapi.version} + + + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-sql_${scala.binary.version} + ${project.version} + test-jar + test + + + org.apache.spark + spark-catalyst_${scala.binary.version} + ${project.version} + test-jar + test + + + org.opencypher + okapi-tck + ${okapi.version} + test + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/graph/graph/pom.xml b/graph/graph/pom.xml new file mode 100644 index 0000000000000..7438ee5e54fe4 --- /dev/null +++ b/graph/graph/pom.xml @@ -0,0 +1,50 @@ + + + + + 4.0.0 + + org.apache.spark + spark-parent_2.12 + 3.0.0-SNAPSHOT + ../../pom.xml + + + spark-graph_2.12 + jar + Spark Project Graph + + graph + + http://spark.apache.org/ + + + + org.apache.spark + spark-cypher_${scala.binary.version} + ${project.version} + + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + diff --git a/graphx/pom.xml b/graphx/pom.xml index d65a8ceb62b9b..444568a03d6c7 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-graphx_2.11 + spark-graphx_2.12 graphx @@ -53,7 +53,7 @@ org.apache.xbean - xbean-asm6-shaded + xbean-asm7-shaded com.google.guava diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index b3a3420b8494d..de94f5bd11847 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -119,16 +119,20 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab /** * Uncaches both vertices and edges of this graph. This is useful in iterative algorithms that * build a new graph in each iteration. + * + * @param blocking Whether to block until all data is unpersisted (default: false) */ - def unpersist(blocking: Boolean = true): Graph[VD, ED] + def unpersist(blocking: Boolean = false): Graph[VD, ED] /** * Uncaches only the vertices of this graph, leaving the edges alone. This is useful in iterative * algorithms that modify the vertex attributes but reuse the edges. This method can be used to * uncache the vertex attributes of previous iterations once they are no longer needed, improving * GC performance. + * + * @param blocking Whether to block until all data is unpersisted (default: false) */ - def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] + def unpersistVertices(blocking: Boolean = false): Graph[VD, ED] /** * Repartitions the edges in the graph according to `partitionStrategy`. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala index f665727ef90db..6aeea913b4607 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphLoader.scala @@ -17,6 +17,8 @@ package org.apache.spark.graphx +import java.util.concurrent.TimeUnit + import org.apache.spark.SparkContext import org.apache.spark.graphx.impl.{EdgePartitionBuilder, GraphImpl} import org.apache.spark.internal.Logging @@ -63,7 +65,7 @@ object GraphLoader extends Logging { vertexStorageLevel: StorageLevel = StorageLevel.MEMORY_ONLY) : Graph[Int, Int] = { - val startTime = System.currentTimeMillis + val startTimeNs = System.nanoTime() // Parse the edge data table directly into edge partitions val lines = @@ -93,7 +95,8 @@ object GraphLoader extends Logging { }.persist(edgeStorageLevel).setName("GraphLoader.edgeListFile - edges (%s)".format(path)) edges.count() - logInfo("It took %d ms to load the edges".format(System.currentTimeMillis - startTime)) + logInfo(s"It took ${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms" + + " to load the edges") GraphImpl.fromEdgePartitions(edges, defaultVertexAttr = 1, edgeStorageLevel = edgeStorageLevel, vertexStorageLevel = vertexStorageLevel) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala index 755c6febc48e6..75248d515ec78 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Pregel.scala @@ -163,9 +163,9 @@ object Pregel extends Logging { logInfo("Pregel finished iteration " + i) // Unpersist the RDDs hidden by newly-materialized RDDs - oldMessages.unpersist(blocking = false) - prevG.unpersistVertices(blocking = false) - prevG.edges.unpersist(blocking = false) + oldMessages.unpersist() + prevG.unpersistVertices() + prevG.edges.unpersist() // count the iteration i += 1 } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala index da3db3c4dca04..27c08c894a39f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartitionBuilder.scala @@ -130,16 +130,14 @@ private[impl] case class EdgeWithLocalIds[@specialized ED]( private[impl] object EdgeWithLocalIds { implicit def lexicographicOrdering[ED]: Ordering[EdgeWithLocalIds[ED]] = - new Ordering[EdgeWithLocalIds[ED]] { - override def compare(a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]): Int = { - if (a.srcId == b.srcId) { - if (a.dstId == b.dstId) 0 - else if (a.dstId < b.dstId) -1 - else 1 - } else if (a.srcId < b.srcId) -1 + (a: EdgeWithLocalIds[ED], b: EdgeWithLocalIds[ED]) => + if (a.srcId == b.srcId) { + if (a.dstId == b.dstId) 0 + else if (a.dstId < b.dstId) -1 else 1 } - } + else if (a.srcId < b.srcId) -1 + else 1 private[graphx] def edgeArraySortDataFormat[ED] = { new SortDataFormat[EdgeWithLocalIds[ED], Array[EdgeWithLocalIds[ED]]] { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala index 376c7b06f9d2b..2b741c30d1a69 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeRDDImpl.scala @@ -58,7 +58,7 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( this } - override def unpersist(blocking: Boolean = true): this.type = { + override def unpersist(blocking: Boolean = false): this.type = { partitionsRDD.unpersist(blocking) this } @@ -87,7 +87,7 @@ class EdgeRDDImpl[ED: ClassTag, VD: ClassTag] private[graphx] ( /** The number of edges in the RDD. */ override def count(): Long = { - partitionsRDD.map(_._2.size.toLong).reduce(_ + _) + partitionsRDD.map(_._2.size.toLong).fold(0)(_ + _) } override def mapValues[ED2: ClassTag](f: Edge[ED] => ED2): EdgeRDDImpl[ED2, VD] = diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 34e1253ff42ab..0a97ab492600d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -78,13 +78,13 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( } } - override def unpersist(blocking: Boolean = true): Graph[VD, ED] = { + override def unpersist(blocking: Boolean = false): Graph[VD, ED] = { unpersistVertices(blocking) replicatedVertexView.edges.unpersist(blocking) this } - override def unpersistVertices(blocking: Boolean = true): Graph[VD, ED] = { + override def unpersistVertices(blocking: Boolean = false): Graph[VD, ED] = { vertices.unpersist(blocking) // TODO: unpersist the replicated vertices in `replicatedVertexView` but leave the edges alone this diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala index a4e293d74a012..184b96426fa9b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/ShippableVertexPartition.scala @@ -117,13 +117,11 @@ class ShippableVertexPartition[VD: ClassTag]( val initialSize = if (shipSrc && shipDst) routingTable.partitionSize(pid) else 64 val vids = new PrimitiveVector[VertexId](initialSize) val attrs = new PrimitiveVector[VD](initialSize) - var i = 0 routingTable.foreachWithinEdgePartition(pid, shipSrc, shipDst) { vid => if (isDefined(vid)) { vids += vid attrs += this(vid) } - i += 1 } (pid, new VertexAttributeBlock(vids.trim().array, attrs.trim().array)) } @@ -137,12 +135,10 @@ class ShippableVertexPartition[VD: ClassTag]( def shipVertexIds(): Iterator[(PartitionID, Array[VertexId])] = { Iterator.tabulate(routingTable.numEdgePartitions) { pid => val vids = new PrimitiveVector[VertexId](routingTable.partitionSize(pid)) - var i = 0 routingTable.foreachWithinEdgePartition(pid, true, true) { vid => if (isDefined(vid)) { vids += vid } - i += 1 } (pid, vids.trim().array) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala index 3c6f22d97360d..915cf8b2e5865 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/VertexRDDImpl.scala @@ -58,7 +58,7 @@ class VertexRDDImpl[VD] private[graphx] ( this } - override def unpersist(blocking: Boolean = true): this.type = { + override def unpersist(blocking: Boolean = false): this.type = { partitionsRDD.unpersist(blocking) this } @@ -87,7 +87,7 @@ class VertexRDDImpl[VD] private[graphx] ( /** The number of vertices in the RDD. */ override def count(): Long = { - partitionsRDD.map(_.size.toLong).reduce(_ + _) + partitionsRDD.map(_.size.toLong).fold(0)(_ + _) } override private[graphx] def mapVertexPartitions[VD2: ClassTag]( diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala index cb3025f8bef54..7381f59771a42 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/LabelPropagation.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx.lib +import scala.collection.{mutable, Map} import scala.reflect.ClassTag import org.apache.spark.graphx._ @@ -51,11 +52,14 @@ object LabelPropagation { } def mergeMessage(count1: Map[VertexId, Long], count2: Map[VertexId, Long]) : Map[VertexId, Long] = { - (count1.keySet ++ count2.keySet).map { i => + // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 + val map = mutable.Map[VertexId, Long]() + (count1.keySet ++ count2.keySet).foreach { i => val count1Val = count1.getOrElse(i, 0L) val count2Val = count2.getOrElse(i, 0L) - i -> (count1Val + count2Val) - }(collection.breakOut) // more efficient alternative to [[collection.Traversable.toMap]] + map.put(i, count1Val + count2Val) + } + map } def vertexProgram(vid: VertexId, attr: Long, message: Map[VertexId, Long]): VertexId = { if (message.isEmpty) attr else message.maxBy(_._2)._1 diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala index 1305c059b89ce..105ad57d4ed86 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/PageRank.scala @@ -156,8 +156,8 @@ object PageRank extends Logging { rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices logInfo(s"PageRank finished iteration $iteration.") - prevRankGraph.vertices.unpersist(false) - prevRankGraph.edges.unpersist(false) + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() iteration += 1 } @@ -234,9 +234,9 @@ object PageRank extends Logging { popActivations +:+ resetActivations }.cache() - rankGraph.edges.foreachPartition(x => {}) // also materializes rankGraph.vertices - prevRankGraph.vertices.unpersist(false) - prevRankGraph.edges.unpersist(false) + rankGraph.edges.foreachPartition(_ => {}) // also materializes rankGraph.vertices + prevRankGraph.vertices.unpersist() + prevRankGraph.edges.unpersist() logInfo(s"Parallel Personalized PageRank finished iteration $i.") diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala index 59fdd855e6f37..2847a4e172d40 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/SVDPlusPlus.scala @@ -72,7 +72,7 @@ object SVDPlusPlus { // calculate global rating mean edges.cache() - val (rs, rc) = edges.map(e => (e.attr, 1L)).reduce((a, b) => (a._1 + b._1, a._2 + b._2)) + val (rs, rc) = edges.map(e => (e.attr, 1L)).fold((0, 0))((a, b) => (a._1 + b._1, a._2 + b._2)) val u = rs / rc // construct graph diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala index aff0b932e9429..a410473afd3ea 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/ShortestPaths.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx.lib +import scala.collection.{mutable, Map} import scala.reflect.ClassTag import org.apache.spark.graphx._ @@ -34,9 +35,12 @@ object ShortestPaths extends Serializable { private def incrementMap(spmap: SPMap): SPMap = spmap.map { case (v, d) => v -> (d + 1) } private def addMaps(spmap1: SPMap, spmap2: SPMap): SPMap = { - (spmap1.keySet ++ spmap2.keySet).map { - k => k -> math.min(spmap1.getOrElse(k, Int.MaxValue), spmap2.getOrElse(k, Int.MaxValue)) - }(collection.breakOut) // more efficient alternative to [[collection.Traversable.toMap]] + // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 + val map = mutable.Map[VertexId, Int]() + (spmap1.keySet ++ spmap2.keySet).foreach { k => + map.put(k, math.min(spmap1.getOrElse(k, Int.MaxValue), spmap2.getOrElse(k, Int.MaxValue))) + } + map } /** diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala index e4f80ffcb451b..3393ea06ff246 100755 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/StronglyConnectedComponents.scala @@ -72,7 +72,7 @@ object StronglyConnectedComponents { sccGraph.vertices.count() sccGraph.edges.count() // sccGraph materialized so, unpersist can be done on previous - prevSccGraph.unpersist(blocking = false) + prevSccGraph.unpersist() prevSccGraph = sccGraph // only keep vertices that are not final diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 50b03f71379a1..5ece5ae5c359b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -20,10 +20,9 @@ package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet -import scala.language.existentials -import org.apache.xbean.asm6.{ClassReader, ClassVisitor, MethodVisitor} -import org.apache.xbean.asm6.Opcodes._ +import org.apache.xbean.asm7.{ClassReader, ClassVisitor, MethodVisitor} +import org.apache.xbean.asm7.Opcodes._ import org.apache.spark.util.Utils @@ -48,7 +47,7 @@ private[graphx] object BytecodeUtils { return true } } - return false + false } } @@ -59,7 +58,8 @@ private[graphx] object BytecodeUtils { var stack = List[(Class[_], String)]((cls, method)) while (stack.nonEmpty) { - val (c, m) = stack.head + val c = stack.head._1 + val m = stack.head._2 stack = stack.tail seen.add((c, m)) val finder = new MethodInvocationFinder(c.getName, m) @@ -72,7 +72,7 @@ private[graphx] object BytecodeUtils { } } } - return false + false } /** @@ -109,14 +109,14 @@ private[graphx] object BytecodeUtils { * determine the actual method invoked by inspecting the bytecode. */ private class MethodInvocationFinder(className: String, methodName: String) - extends ClassVisitor(ASM6) { + extends ClassVisitor(ASM7) { val methodsInvoked = new HashSet[(Class[_], String)] override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { if (name == methodName) { - new MethodVisitor(ASM6) { + new MethodVisitor(ASM7) { override def visitMethodInsn( op: Int, owner: String, name: String, desc: String, itf: Boolean) { if (op == INVOKEVIRTUAL || op == INVOKESPECIAL || op == INVOKESTATIC) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala index 539b66f747cc9..3fbeeb47660e7 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/PeriodicGraphCheckpointer.scala @@ -98,7 +98,7 @@ private[spark] class PeriodicGraphCheckpointer[VD, ED]( } } - override protected def unpersist(data: Graph[VD, ED]): Unit = data.unpersist(blocking = false) + override protected def unpersist(data: Graph[VD, ED]): Unit = data.unpersist() override protected def getCheckpointFiles(data: Graph[VD, ED]): Iterable[String] = { data.getCheckpointFiles diff --git a/graphx/src/test/resources/log4j.properties b/graphx/src/test/resources/log4j.properties index 3706a6e361307..a9166df3a7fb4 100644 --- a/graphx/src/test/resources/log4j.properties +++ b/graphx/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala index 7a24e320c3e04..8fd3e6f5229cc 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/EdgeRDDSuite.scala @@ -60,4 +60,14 @@ class EdgeRDDSuite extends SparkFunSuite with LocalSparkContext { } } + test("count") { + withSpark { sc => + val empty = EdgeRDD.fromEdges(sc.emptyRDD[Edge[Int]]) + assert(empty.count === 0) + + val edges = List(Edge(0, 1, ()), Edge(1, 2, ()), Edge(2, 0, ())) + val nonempty = EdgeRDD.fromEdges(sc.parallelize(edges)) + assert(nonempty.count === edges.size) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 88b59a343a83c..dede3b5d35ced 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -396,11 +396,11 @@ class GraphSuite extends SparkFunSuite with LocalSparkContext { val g = g0.partitionBy(PartitionStrategy.EdgePartition2D, 2) val cc = g.connectedComponents() assert(sc.getPersistentRDDs.nonEmpty) - cc.unpersist() - g.unpersist() - g0.unpersist() - vert.unpersist() - edges.unpersist() + cc.unpersist(blocking = true) + g.unpersist(blocking = true) + g0.unpersist(blocking = true) + vert.unpersist(blocking = true) + edges.unpersist(blocking = true) assert(sc.getPersistentRDDs.isEmpty) } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala index 8e630435279de..434e6a84edf6d 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/VertexRDDSuite.scala @@ -223,4 +223,15 @@ class VertexRDDSuite extends SparkFunSuite with LocalSparkContext { assert(verts.collect().toSeq === data) // test checkpointed RDD } } + + test("count") { + withSpark { sc => + val empty = VertexRDD(sc.emptyRDD[(Long, Unit)]) + assert(empty.count === 0) + + val n = 100 + val nonempty = vertices(sc, n) + assert(nonempty.count === n + 1) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala index 1e4c6c74bd184..d8f1c497718bb 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/PageRankSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.graphx.util.GraphGenerators object GridPageRank { def apply(nRows: Int, nCols: Int, nIter: Int, resetProb: Double): Seq[(VertexId, Double)] = { - val inNbrs = Array.fill(nRows * nCols)(collection.mutable.MutableList.empty[Int]) + val inNbrs = Array.fill(nRows * nCols)(collection.mutable.ArrayBuffer.empty[Int]) val outDegree = Array.fill(nRows * nCols)(0) // Convert row column address into vertex ids (row major order) def sub2ind(r: Int, c: Int): Int = r * nCols + c diff --git a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala index 2991438f5e57e..da0457c354b51 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/lib/SVDPlusPlusSuite.scala @@ -40,4 +40,13 @@ class SVDPlusPlusSuite extends SparkFunSuite with LocalSparkContext { } } + test("Test SVD++ with no edges") { + withSpark { sc => + val edges = sc.emptyRDD[Edge[Double]] + val conf = new SVDPlusPlus.Conf(10, 2, 0.0, 5.0, 0.007, 0.007, 0.005, 0.015) // 2 iterations + val (graph, _) = SVDPlusPlus.run(edges, conf) + assert(graph.vertices.count == 0) + assert(graph.edges.count == 0) + } + } } diff --git a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala deleted file mode 100644 index 5325978a0a1ec..0000000000000 --- a/graphx/src/test/scala/org/apache/spark/graphx/util/BytecodeUtilsSuite.scala +++ /dev/null @@ -1,119 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.graphx.util - -import org.apache.spark.SparkFunSuite -import org.apache.spark.util.ClosureCleanerSuite2 - - -// scalastyle:off println -class BytecodeUtilsSuite extends SparkFunSuite { - - import BytecodeUtilsSuite.TestClass - - test("closure invokes a method") { - assume(!ClosureCleanerSuite2.supportsLMFs) - val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) - - val c2 = {e: TestClass => println(e.foo); println(e.bar); } - assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) - assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) - assert(!BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) - - val c3 = {e: TestClass => println(e.foo); } - assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) - assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) - assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) - } - - test("closure inside a closure invokes a method") { - assume(!ClosureCleanerSuite2.supportsLMFs) - val c1 = {e: TestClass => println(e.foo); println(e.bar); println(e.baz); } - val c2 = {e: TestClass => c1(e); println(e.foo); } - assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "foo")) - assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c2, classOf[TestClass], "baz")) - } - - test("closure inside a closure inside a closure invokes a method") { - assume(!ClosureCleanerSuite2.supportsLMFs) - val c1 = {e: TestClass => println(e.baz); } - val c2 = {e: TestClass => c1(e); println(e.foo); } - val c3 = {e: TestClass => c2(e) } - assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "foo")) - assert(!BytecodeUtils.invokedMethod(c3, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c3, classOf[TestClass], "baz")) - } - - test("closure calling a function that invokes a method") { - assume(!ClosureCleanerSuite2.supportsLMFs) - def zoo(e: TestClass) { - println(e.baz) - } - val c1 = {e: TestClass => zoo(e)} - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) - } - - test("closure calling a function that invokes a method which uses another closure") { - assume(!ClosureCleanerSuite2.supportsLMFs) - val c2 = {e: TestClass => println(e.baz)} - def zoo(e: TestClass) { - c2(e) - } - val c1 = {e: TestClass => zoo(e)} - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) - } - - test("nested closure") { - assume(!ClosureCleanerSuite2.supportsLMFs) - val c2 = {e: TestClass => println(e.baz)} - def zoo(e: TestClass, c: TestClass => Unit) { - c(e) - } - val c1 = {e: TestClass => zoo(e, c2)} - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) - assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) - assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) - } - - // The following doesn't work yet, because the byte code doesn't contain any information - // about what exactly "c" is. -// test("invoke interface") { -// val c1 = {e: TestClass => c(e)} -// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "foo")) -// assert(!BytecodeUtils.invokedMethod(c1, classOf[TestClass], "bar")) -// assert(BytecodeUtils.invokedMethod(c1, classOf[TestClass], "baz")) -// } - - private val c = {e: TestClass => println(e.baz)} -} - -// scalastyle:on println - -object BytecodeUtilsSuite { - class TestClass(val foo: Int, val bar: Long) { - def baz: Boolean = false - } -} diff --git a/hadoop-cloud/pom.xml b/hadoop-cloud/pom.xml index 3182ab15db5f5..dbf4b98d5fadb 100644 --- a/hadoop-cloud/pom.xml +++ b/hadoop-cloud/pom.xml @@ -21,18 +21,16 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-hadoop-cloud_2.11 + spark-hadoop-cloud_2.12 jar - Spark Project Cloud Integration through Hadoop Libraries + Spark Project Hadoop Cloud Integration - Contains support for cloud infrastructures, specifically the Hadoop JARs and - transitive dependencies needed to interact with the infrastructures, - making everything consistent with Spark's other dependencies. + Contains Hadoop JARs and transitive dependencies needed to interact with cloud infrastructures. hadoop-cloud @@ -199,7 +197,7 @@ enables store-specific committers. --> - hadoop-3.1 + hadoop-3.2 + + org.glassfish.jaxb + jaxb-runtime + org.apache.spark spark-tags_${scala.binary.version} 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 103082b7b9766..69a4dbef138e3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Pipeline.scala @@ -31,6 +31,7 @@ import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.param.{Param, ParamMap, Params} import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.types.StructType @@ -132,7 +133,8 @@ class Pipeline @Since("1.4.0") ( * @return fitted pipeline */ @Since("2.0.0") - override def fit(dataset: Dataset[_]): PipelineModel = { + override def fit(dataset: Dataset[_]): PipelineModel = instrumented( + instr => instr.withFitEvent(this, dataset) { transformSchema(dataset.schema, logging = true) val theStages = $(stages) // Search for the last estimator. @@ -150,7 +152,7 @@ class Pipeline @Since("1.4.0") ( if (index <= indexOfLastEstimator) { val transformer = stage match { case estimator: Estimator[_] => - estimator.fit(curDataset) + instr.withFitEvent(estimator, curDataset)(estimator.fit(curDataset)) case t: Transformer => t case _ => @@ -158,7 +160,8 @@ class Pipeline @Since("1.4.0") ( s"Does not support stage $stage of type ${stage.getClass}") } if (index < indexOfLastEstimator) { - curDataset = transformer.transform(curDataset) + curDataset = instr.withTransformEvent( + transformer, curDataset)(transformer.transform(curDataset)) } transformers += transformer } else { @@ -167,7 +170,7 @@ class Pipeline @Since("1.4.0") ( } new PipelineModel(uid, transformers.toArray).setParent(this) - } + }) @Since("1.4.0") override def copy(extra: ParamMap): Pipeline = { @@ -197,10 +200,12 @@ object Pipeline extends MLReadable[Pipeline] { @Since("1.6.0") override def load(path: String): Pipeline = super.load(path) - private[Pipeline] class PipelineWriter(instance: Pipeline) extends MLWriter { + private[Pipeline] class PipelineWriter(val instance: Pipeline) extends MLWriter { SharedReadWrite.validateStages(instance.getStages) + override def save(path: String): Unit = + instrumented(_.withSaveInstanceEvent(this, path)(super.save(path))) override protected def saveImpl(path: String): Unit = SharedReadWrite.saveImpl(instance, instance.getStages, sc, path) } @@ -210,10 +215,10 @@ object Pipeline extends MLReadable[Pipeline] { /** Checked against metadata when loading model */ private val className = classOf[Pipeline].getName - override def load(path: String): Pipeline = { + override def load(path: String): Pipeline = instrumented(_.withLoadInstanceEvent(this, path) { val (uid: String, stages: Array[PipelineStage]) = SharedReadWrite.load(className, sc, path) new Pipeline(uid).setStages(stages) - } + }) } /** @@ -243,7 +248,7 @@ object Pipeline extends MLReadable[Pipeline] { instance: Params, stages: Array[PipelineStage], sc: SparkContext, - path: String): Unit = { + path: String): Unit = instrumented { instr => val stageUids = stages.map(_.uid) val jsonParams = List("stageUids" -> parse(compact(render(stageUids.toSeq)))) DefaultParamsWriter.saveMetadata(instance, path, sc, paramMap = Some(jsonParams)) @@ -251,8 +256,9 @@ object Pipeline extends MLReadable[Pipeline] { // Save stages val stagesDir = new Path(path, "stages").toString stages.zipWithIndex.foreach { case (stage, idx) => - stage.asInstanceOf[MLWritable].write.save( - getStagePath(stage.uid, idx, stages.length, stagesDir)) + val writer = stage.asInstanceOf[MLWritable].write + val stagePath = getStagePath(stage.uid, idx, stages.length, stagesDir) + instr.withSaveInstanceEvent(writer, stagePath)(writer.save(stagePath)) } } @@ -263,7 +269,7 @@ object Pipeline extends MLReadable[Pipeline] { def load( expectedClassName: String, sc: SparkContext, - path: String): (String, Array[PipelineStage]) = { + path: String): (String, Array[PipelineStage]) = instrumented { instr => val metadata = DefaultParamsReader.loadMetadata(path, sc, expectedClassName) implicit val format = DefaultFormats @@ -271,7 +277,8 @@ object Pipeline extends MLReadable[Pipeline] { val stageUids: Array[String] = (metadata.params \ "stageUids").extract[Seq[String]].toArray val stages: Array[PipelineStage] = stageUids.zipWithIndex.map { case (stageUid, idx) => val stagePath = SharedReadWrite.getStagePath(stageUid, idx, stageUids.length, stagesDir) - DefaultParamsReader.loadParamsInstance[PipelineStage](stagePath, sc) + val reader = DefaultParamsReader.loadParamsInstanceReader[PipelineStage](stagePath, sc) + instr.withLoadInstanceEvent(reader, stagePath)(reader.load(stagePath)) } (metadata.uid, stages) } @@ -301,10 +308,12 @@ class PipelineModel private[ml] ( } @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { + override def transform(dataset: Dataset[_]): DataFrame = instrumented(instr => + instr.withTransformEvent(this, dataset) { transformSchema(dataset.schema, logging = true) - stages.foldLeft(dataset.toDF)((cur, transformer) => transformer.transform(cur)) - } + stages.foldLeft(dataset.toDF)((cur, transformer) => + instr.withTransformEvent(transformer, cur)(transformer.transform(cur))) + }) @Since("1.2.0") override def transformSchema(schema: StructType): StructType = { @@ -331,10 +340,12 @@ object PipelineModel extends MLReadable[PipelineModel] { @Since("1.6.0") override def load(path: String): PipelineModel = super.load(path) - private[PipelineModel] class PipelineModelWriter(instance: PipelineModel) extends MLWriter { + private[PipelineModel] class PipelineModelWriter(val instance: PipelineModel) extends MLWriter { SharedReadWrite.validateStages(instance.stages.asInstanceOf[Array[PipelineStage]]) + override def save(path: String): Unit = + instrumented(_.withSaveInstanceEvent(this, path)(super.save(path))) override protected def saveImpl(path: String): Unit = SharedReadWrite.saveImpl(instance, instance.stages.asInstanceOf[Array[PipelineStage]], sc, path) } @@ -344,7 +355,8 @@ object PipelineModel extends MLReadable[PipelineModel] { /** Checked against metadata when loading model */ private val className = classOf[PipelineModel].getName - override def load(path: String): PipelineModel = { + override def load(path: String): PipelineModel = instrumented(_.withLoadInstanceEvent( + this, path) { val (uid: String, stages: Array[PipelineStage]) = SharedReadWrite.load(className, sc, path) val transformers = stages map { case stage: Transformer => stage @@ -352,6 +364,6 @@ object PipelineModel extends MLReadable[PipelineModel] { s" was not a Transformer. Bad stage ${other.uid} of type ${other.getClass}") } new PipelineModel(uid, transformers) - } + }) } } 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 d8f3dfa874439..58815434cbdaf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/Predictor.scala @@ -204,8 +204,8 @@ abstract class PredictionModel[FeaturesType, M <: PredictionModel[FeaturesType, if ($(predictionCol).nonEmpty) { transformImpl(dataset) } else { - this.logWarning(s"$uid: Predictor.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: Predictor.transform() does nothing" + + " because no output columns were set.") dataset.toDF } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala index 014ff07c21158..2b4b0fc55b955 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/ann/Layer.scala @@ -371,7 +371,7 @@ private[ann] trait TopologyModel extends Serializable { def forward(data: BDM[Double], includeLastLayer: Boolean): Array[BDM[Double]] /** - * Prediction of the model. See {@link ProbabilisticClassificationModel} + * Prediction of the model. See `ProbabilisticClassificationModel`` * * @param features input features * @return prediction @@ -379,7 +379,7 @@ private[ann] trait TopologyModel extends Serializable { def predict(features: Vector): Vector /** - * Raw prediction of the model. See {@link ProbabilisticClassificationModel} + * Raw prediction of the model. See `ProbabilisticClassificationModel` * * @param features input features * @return raw prediction @@ -389,7 +389,7 @@ private[ann] trait TopologyModel extends Serializable { def predictRaw(features: Vector): Vector /** - * Probability of the model. See {@link ProbabilisticClassificationModel} + * Probability of the model. See `ProbabilisticClassificationModel` * * @param rawPrediction raw prediction vector * @return probability 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 1cd2b1ad84092..756dd677cb579 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 @@ -121,7 +121,7 @@ sealed abstract class Attribute extends Serializable { private[attribute] trait AttributeFactory { /** - * Creates an [[Attribute]] from a [[Metadata]] instance. + * Creates an [[Attribute]] from a `Metadata` instance. */ private[attribute] def fromMetadata(metadata: Metadata): Attribute 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 7e5790ab70ee9..568cdd11a12a8 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.DeveloperApi import org.apache.spark.ml.{PredictionModel, Predictor, PredictorParams} import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.{Vector, VectorUDT} @@ -77,17 +77,37 @@ abstract class Classifier[ * @note Throws `SparkException` if any label is a non-integer or is negative */ protected def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = { - require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" + - s" $numClasses, but requires numClasses > 0.") + validateNumClasses(numClasses) dataset.select(col($(labelCol)), col($(featuresCol))).rdd.map { case Row(label: Double, features: Vector) => - require(label % 1 == 0 && label >= 0 && label < numClasses, s"Classifier was given" + - s" dataset with invalid label $label. Labels must be integers in range" + - s" [0, $numClasses).") + validateLabel(label, numClasses) LabeledPoint(label, features) } } + /** + * Validates that number of classes is greater than zero. + * + * @param numClasses Number of classes label can take. + */ + protected def validateNumClasses(numClasses: Int): Unit = { + require(numClasses > 0, s"Classifier (in extractLabeledPoints) found numClasses =" + + s" $numClasses, but requires numClasses > 0.") + } + + /** + * Validates the label on the classifier is a valid integer in the range [0, numClasses). + * + * @param label The label to validate. + * @param numClasses Number of classes label can take. Labels must be integers in the range + * [0, numClasses). + */ + protected def validateLabel(label: Double, numClasses: Int): Unit = { + require(label.toLong == label && label >= 0 && label < numClasses, s"Classifier was given" + + s" dataset with invalid label $label. Labels must be integers in range" + + s" [0, $numClasses).") + } + /** * Get the number of classes. This looks in column metadata first, and if that is missing, * then this assumes classes are indexed 0,1,...,numClasses-1 and computes numClasses @@ -184,8 +204,8 @@ abstract class ClassificationModel[FeaturesType, M <: ClassificationModel[Featur } if (numColsOutput == 0) { - logWarning(s"$uid: ClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + logWarning(s"$uid: ClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala index 6648e78d8eafa..cbb7e4f5f4099 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/DecisionTreeClassifier.scala @@ -22,10 +22,11 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.{DecisionTreeModel, Node, TreeClassifierParams} import org.apache.spark.ml.tree.DecisionTreeModelReadWrite._ import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ @@ -33,8 +34,9 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Dataset - +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.functions.{col, lit} +import org.apache.spark.sql.types.DoubleType /** * Decision tree learning algorithm (http://en.wikipedia.org/wiki/Decision_tree_learning) @@ -55,27 +57,30 @@ class DecisionTreeClassifier @Since("1.4.0") ( /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -87,15 +92,25 @@ class DecisionTreeClassifier @Since("1.4.0") ( * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = set(impurity, value) + def setImpurity(value: String): this.type = set(impurity, value) /** @group setParam */ @Since("1.6.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) + + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) override protected def train( dataset: Dataset[_]): DecisionTreeClassificationModel = instrumented { instr => @@ -104,21 +119,27 @@ class DecisionTreeClassifier @Since("1.4.0") ( val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) val numClasses: Int = getNumClasses(dataset) - instr.logNumClasses(numClasses) if (isDefined(thresholds)) { require($(thresholds).length == numClasses, this.getClass.getSimpleName + ".train() called with non-matching numClasses and thresholds.length." + s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) + validateNumClasses(numClasses) + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances = + dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + validateLabel(label, numClasses) + Instance(label, weight, features) + } val strategy = getOldStrategy(categoricalFeatures, numClasses) - - instr.logParams(this, maxDepth, maxBins, minInstancesPerNode, minInfoGain, maxMemoryInMB, + instr.logNumClasses(numClasses) + instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, + probabilityCol, maxDepth, maxBins, minInstancesPerNode, minInfoGain, maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed) - val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", + val trees = RandomForest.run(instances, strategy, numTrees = 1, featureSubsetStrategy = "all", seed = $(seed), instr = Some(instr), parentUID = Some(uid)) trees.head.asInstanceOf[DecisionTreeClassificationModel] @@ -127,13 +148,13 @@ class DecisionTreeClassifier @Since("1.4.0") ( /** (private[ml]) Train a decision tree on an RDD */ private[ml] def train(data: RDD[LabeledPoint], oldStrategy: OldStrategy): DecisionTreeClassificationModel = instrumented { instr => + val instances = data.map(_.toInstance) instr.logPipelineStage(this) - instr.logDataset(data) + instr.logDataset(instances) instr.logParams(this, maxDepth, maxBins, minInstancesPerNode, minInfoGain, maxMemoryInMB, cacheNodeIds, checkpointInterval, impurity, seed) - - val trees = RandomForest.run(data, oldStrategy, numTrees = 1, featureSubsetStrategy = "all", - seed = 0L, instr = Some(instr), parentUID = Some(uid)) + val trees = RandomForest.run(instances, oldStrategy, numTrees = 1, + featureSubsetStrategy = "all", seed = 0L, instr = Some(instr), parentUID = Some(uid)) trees.head.asInstanceOf[DecisionTreeClassificationModel] } @@ -179,6 +200,7 @@ class DecisionTreeClassificationModel private[ml] ( /** * Construct a decision tree classification model. + * * @param rootNode Root node of tree, with other nodes attached. */ private[ml] def this(rootNode: Node, numFeatures: Int, numClasses: Int) = diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala index 62cfa39746ff0..a5ed4a38a8863 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/GBTClassifier.scala @@ -69,27 +69,27 @@ class GBTClassifier @Since("1.4.0") ( /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -101,7 +101,7 @@ class GBTClassifier @Since("1.4.0") ( * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** * The impurity setting is ignored for GBT models. @@ -110,7 +110,7 @@ class GBTClassifier @Since("1.4.0") ( * @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = { + def setImpurity(value: String): this.type = { logWarning("GBTClassifier.setImpurity should NOT be used") this } @@ -119,25 +119,25 @@ class GBTClassifier @Since("1.4.0") ( /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) // Parameters from GBTParams: /** @group setParam */ @Since("1.4.0") - override def setMaxIter(value: Int): this.type = set(maxIter, value) + def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ @Since("1.4.0") - override def setStepSize(value: Double): this.type = set(stepSize, value) + def setStepSize(value: Double): this.type = set(stepSize, value) /** @group setParam */ @Since("2.3.0") - override def setFeatureSubsetStrategy(value: String): this.type = + def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) // Parameters from GBTClassifierParams: @@ -180,7 +180,6 @@ class GBTClassifier @Since("1.4.0") ( (convert2LabeledPoint(dataset), null) } - val numFeatures = trainDataset.first().features.size val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Classification) val numClasses = 2 @@ -195,8 +194,7 @@ class GBTClassifier @Since("1.4.0") ( instr.logParams(this, labelCol, featuresCol, predictionCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, - validationIndicatorCol) - instr.logNumFeatures(numFeatures) + validationIndicatorCol, validationTol) instr.logNumClasses(numClasses) val (baseLearners, learnerWeights) = if (withValidation) { @@ -206,6 +204,9 @@ class GBTClassifier @Since("1.4.0") ( GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + val numFeatures = baseLearners.head.numFeatures + instr.logNumFeatures(numFeatures) + new GBTClassificationModel(uid, baseLearners, learnerWeights, numFeatures) } @@ -340,11 +341,12 @@ class GBTClassificationModel private[ml]( * The importance vector is normalized to sum to 1. This method is suggested by Hastie et al. * (Hastie, Tibshirani, Friedman. "The Elements of Statistical Learning, 2nd Edition." 2001.) * and follows the implementation from scikit-learn. - + * * See `DecisionTreeClassificationModel.featureImportances` */ @Since("2.0.0") - lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) + lazy val featureImportances: Vector = + TreeEnsembleModel.featureImportances(trees, numFeatures, perTreeNormalization = false) /** Raw prediction for the positive class. */ private def margin(features: Vector): Double = { @@ -427,7 +429,9 @@ object GBTClassificationModel extends MLReadable[GBTClassificationModel] { s" trees based on metadata but found ${trees.length} trees.") val model = new GBTClassificationModel(metadata.uid, trees, treeWeights, numFeatures) - metadata.getAndSetParams(model) + // We ignore the impurity while loading models because in previous models it was wrongly + // set to gini (see SPARK-25959). + metadata.getAndSetParams(model, Some(List("impurity"))) model } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala index 1b5c02fc9a576..f0176052ed437 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LinearSVC.scala @@ -173,8 +173,8 @@ class LinearSVC @Since("2.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, regParam, maxIter, fitIntercept, tol, standardization, threshold, - aggregationDepth) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, + regParam, maxIter, fitIntercept, tol, standardization, threshold, aggregationDepth) val (summarizer, labelSummarizer) = { val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), @@ -248,7 +248,7 @@ class LinearSVC @Since("2.2.0") ( scaledObjectiveHistory += state.adjustedValue } - bcFeaturesStd.destroy(blocking = false) + bcFeaturesStd.destroy() if (state == null) { val msg = s"${optimizer.getClass.getName} failed." instr.logError(msg) diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala index 6f0804f0c8e4a..7790de064e7bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala @@ -503,8 +503,9 @@ class LogisticRegression @Since("1.2.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, regParam, elasticNetParam, standardization, threshold, - maxIter, tol, fitIntercept) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, rawPredictionCol, + probabilityCol, regParam, elasticNetParam, standardization, threshold, maxIter, tol, + fitIntercept) val (summarizer, labelSummarizer) = { val seqOp = (c: (MultivariateOnlineSummarizer, MultiClassSummarizer), @@ -814,7 +815,7 @@ class LogisticRegression @Since("1.2.0") ( state = states.next() arrayBuilder += state.adjustedValue } - bcFeaturesStd.destroy(blocking = false) + bcFeaturesStd.destroy() if (state == null) { val msg = s"${optimizer.getClass.getName} failed." @@ -933,8 +934,8 @@ class LogisticRegressionModel private[spark] ( @Since("2.1.0") val interceptVector: Vector, @Since("1.3.0") override val numClasses: Int, private val isMultinomial: Boolean) - extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] - with LogisticRegressionParams with MLWritable { + extends ProbabilisticClassificationModel[Vector, LogisticRegressionModel] with MLWritable + with LogisticRegressionParams with HasTrainingSummary[LogisticRegressionTrainingSummary] { require(coefficientMatrix.numRows == interceptVector.size, s"Dimension mismatch! Expected " + s"coefficientMatrix.numRows == interceptVector.size, but ${coefficientMatrix.numRows} != " + @@ -1017,20 +1018,16 @@ class LogisticRegressionModel private[spark] ( @Since("1.6.0") override val numFeatures: Int = coefficientMatrix.numCols - private var trainingSummary: Option[LogisticRegressionTrainingSummary] = None - /** * Gets summary of model on training set. An exception is thrown - * if `trainingSummary == None`. + * if `hasSummary` is false. */ @Since("1.5.0") - def summary: LogisticRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException("No training summary available for this LogisticRegressionModel") - } + override def summary: LogisticRegressionTrainingSummary = super.summary /** * Gets summary of model on training set. An exception is thrown - * if `trainingSummary == None` or it is a multiclass model. + * if `hasSummary` is false or it is a multiclass model. */ @Since("2.3.0") def binarySummary: BinaryLogisticRegressionTrainingSummary = summary match { @@ -1061,16 +1058,6 @@ class LogisticRegressionModel private[spark] ( (model, model.getProbabilityCol, model.getPredictionCol) } - private[classification] - def setSummary(summary: Option[LogisticRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** Indicates whether a training summary exists for this model instance. */ - @Since("1.5.0") - def hasSummary: Boolean = trainingSummary.isDefined - /** * Evaluates the model on a test dataset. * @@ -1303,7 +1290,7 @@ private[ml] class MultiClassSummarizer extends Serializable { * @param weight The weight of this instances. * @return This MultilabelSummarizer */ - def add(label: Double, weight: Double = 1.0): this.type = { + def add(label: Double, weight: Double = 1.0): MultiClassSummarizer = { require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") if (weight == 0.0) return this diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala index 4feddce1d9f2d..47b8a8df637b9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/MultilayerPerceptronClassifier.scala @@ -205,8 +205,8 @@ class MultilayerPerceptronClassifier @Since("1.5.0") ( dataset: Dataset[_]): MultilayerPerceptronClassificationModel = instrumented { instr => instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, layers, maxIter, tol, - blockSize, solver, stepSize, seed) + instr.logParams(this, labelCol, featuresCol, predictionCol, rawPredictionCol, layers, maxIter, + tol, blockSize, solver, stepSize, seed) val myLayers = $(layers) val labels = myLayers.last diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala index 51495c1a74e69..1a7a5e7a52344 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/NaiveBayes.scala @@ -146,7 +146,7 @@ class NaiveBayes @Since("1.5.0") ( requireZeroOneBernoulliValues case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } } @@ -196,7 +196,7 @@ class NaiveBayes @Since("1.5.0") ( case Bernoulli => math.log(n + 2.0 * lambda) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } var j = 0 while (j < numFeatures) { @@ -295,7 +295,7 @@ class NaiveBayesModel private[ml] ( (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } @Since("1.6.0") @@ -329,7 +329,7 @@ class NaiveBayesModel private[ml] ( bernoulliCalculation(features) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: ${$(modelType)}.") + throw new IllegalArgumentException(s"Invalid modelType: ${$(modelType)}.") } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala index 1835a91775e0a..675315e3bb070 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/OneVsRest.scala @@ -37,7 +37,7 @@ import org.apache.spark.ml.param.{Param, ParamMap, ParamPair, Params} import org.apache.spark.ml.param.shared.{HasParallelism, HasWeightCol} import org.apache.spark.ml.util._ import org.apache.spark.ml.util.Instrumentation.instrumented -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -169,6 +169,12 @@ final class OneVsRestModel private[ml] ( // Check schema transformSchema(dataset.schema, logging = true) + if (getPredictionCol.isEmpty && getRawPredictionCol.isEmpty) { + logWarning(s"$uid: OneVsRestModel.transform() does nothing" + + " because no output columns were set.") + return dataset.toDF + } + // determine the input columns: these need to be passed through val origCols = dataset.schema.map(f => col(f.name)) @@ -209,7 +215,10 @@ final class OneVsRestModel private[ml] ( newDataset.unpersist() } - if (getRawPredictionCol != "") { + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if (getRawPredictionCol.nonEmpty) { val numClass = models.length // output the RawPrediction as vector @@ -219,24 +228,24 @@ final class OneVsRestModel private[ml] ( Vectors.dense(predArray) } - // output the index of the classifier with highest confidence as prediction - val labelUDF = udf { (rawPredictions: Vector) => rawPredictions.argmax.toDouble } + predictionColNames :+= getRawPredictionCol + predictionColumns :+= rawPredictionUDF(col(accColName)) + } - // output confidence as raw prediction, label and label metadata as prediction - aggregatedDataset - .withColumn(getRawPredictionCol, rawPredictionUDF(col(accColName))) - .withColumn(getPredictionCol, labelUDF(col(getRawPredictionCol)), labelMetadata) - .drop(accColName) - } else { + if (getPredictionCol.nonEmpty) { // output the index of the classifier with highest confidence as prediction val labelUDF = udf { (predictions: Map[Int, Double]) => predictions.maxBy(_._2)._1.toDouble } - // output label and label metadata as prediction - aggregatedDataset - .withColumn(getPredictionCol, labelUDF(col(accColName)), labelMetadata) - .drop(accColName) + + predictionColNames :+= getPredictionCol + predictionColumns :+= labelUDF(col(accColName)) + .as(getPredictionCol, labelMetadata) } + + aggregatedDataset + .withColumns(predictionColNames, predictionColumns) + .drop(accColName) } @Since("1.4.1") @@ -368,7 +377,8 @@ final class OneVsRest @Since("1.4.0") ( instr.logPipelineStage(this) instr.logDataset(dataset) - instr.logParams(this, labelCol, featuresCol, predictionCol, parallelism, rawPredictionCol) + instr.logParams(this, labelCol, weightCol, featuresCol, predictionCol, + rawPredictionCol, parallelism) instr.logNamedValue("classifier", $(classifier).getClass.getCanonicalName) // determine number of classes either from metadata if provided, or via computation. 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 730fcab333e11..5046caa568d5c 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 @@ -147,8 +147,8 @@ abstract class ProbabilisticClassificationModel[ } if (numColsOutput == 0) { - this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() was called as NOOP" + - " since no output columns were set.") + this.logWarning(s"$uid: ProbabilisticClassificationModel.transform() does nothing" + + " because no output columns were set.") } outputData.toDF } diff --git a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala index 57132381b6474..3500f2ad52a56 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/classification/RandomForestClassifier.scala @@ -21,20 +21,21 @@ import org.json4s.{DefaultFormats, JObject} import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ +import org.apache.spark.ml.tree.{RandomForestParams, TreeClassifierParams, TreeEnsembleModel} import org.apache.spark.ml.tree.impl.RandomForest import org.apache.spark.ml.util._ +import org.apache.spark.ml.util.{Identifiable, MetadataUtils} import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions._ - +import org.apache.spark.sql.functions.{col, udf} /** * Random Forest learning algorithm for @@ -57,27 +58,27 @@ class RandomForestClassifier @Since("1.4.0") ( /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -89,31 +90,31 @@ class RandomForestClassifier @Since("1.4.0") ( * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = set(impurity, value) + def setImpurity(value: String): this.type = set(impurity, value) // Parameters from TreeEnsembleParams: /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) // Parameters from RandomForestParams: /** @group setParam */ @Since("1.4.0") - override def setNumTrees(value: Int): this.type = set(numTrees, value) + def setNumTrees(value: Int): this.type = set(numTrees, value) /** @group setParam */ @Since("1.4.0") - override def setFeatureSubsetStrategy(value: String): this.type = + def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) override protected def train( @@ -130,7 +131,7 @@ class RandomForestClassifier @Since("1.4.0") ( s" numClasses=$numClasses, but thresholds has length ${$(thresholds).length}") } - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset, numClasses) + val instances: RDD[Instance] = extractLabeledPoints(dataset, numClasses).map(_.toInstance) val strategy = super.getOldStrategy(categoricalFeatures, numClasses, OldAlgo.Classification, getOldImpurity) @@ -139,10 +140,10 @@ class RandomForestClassifier @Since("1.4.0") ( minInstancesPerNode, seed, subsamplingRate, thresholds, cacheNodeIds, checkpointInterval) val trees = RandomForest - .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) + .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeClassificationModel]) - val numFeatures = oldDataset.first().features.size + val numFeatures = trees.head.numFeatures instr.logNumClasses(numClasses) instr.logNumFeatures(numFeatures) new RandomForestClassificationModel(uid, trees, numFeatures, numClasses) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala index 5cb16cc765887..2247880b52414 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/BisectingKMeans.scala @@ -19,7 +19,6 @@ package org.apache.spark.ml.clustering import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.Vector @@ -30,9 +29,10 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.clustering.{BisectingKMeans => MLlibBisectingKMeans, BisectingKMeansModel => MLlibBisectingKMeansModel} import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.storage.StorageLevel /** @@ -87,8 +87,9 @@ private[clustering] trait BisectingKMeansParams extends Params with HasMaxIter @Since("2.0.0") class BisectingKMeansModel private[ml] ( @Since("2.0.0") override val uid: String, - private val parentModel: MLlibBisectingKMeansModel - ) extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable { + private val parentModel: MLlibBisectingKMeansModel) + extends Model[BisectingKMeansModel] with BisectingKMeansParams with MLWritable + with HasTrainingSummary[BisectingKMeansSummary] { @Since("2.0.0") override def copy(extra: ParamMap): BisectingKMeansModel = { @@ -117,7 +118,8 @@ class BisectingKMeansModel private[ml] ( validateAndTransformSchema(schema) } - private[clustering] def predict(features: Vector): Int = parentModel.predict(features) + @Since("3.0.0") + def predict(features: Vector): Int = parentModel.predict(features) @Since("2.0.0") def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) @@ -125,8 +127,15 @@ class BisectingKMeansModel private[ml] ( /** * Computes the sum of squared distances between the input points and their corresponding cluster * centers. + * + * @deprecated This method is deprecated and will be removed in future versions. Use + * ClusteringEvaluator instead. You can also get the cost on the training dataset in + * the summary. */ @Since("2.0.0") + @deprecated("This method is deprecated and will be removed in future versions. Use " + + "ClusteringEvaluator instead. You can also get the cost on the training dataset in the " + + "summary.", "3.0.0") def computeCost(dataset: Dataset[_]): Double = { SchemaUtils.validateVectorCompatibleColumn(dataset.schema, getFeaturesCol) val data = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) @@ -136,28 +145,12 @@ class BisectingKMeansModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new BisectingKMeansModel.BisectingKMeansModelWriter(this) - private var trainingSummary: Option[BisectingKMeansSummary] = None - - private[clustering] def setSummary(summary: Option[BisectingKMeansSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.1.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.1.0") - def summary: BisectingKMeansSummary = trainingSummary.getOrElse { - throw new SparkException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: BisectingKMeansSummary = super.summary } object BisectingKMeansModel extends MLReadable[BisectingKMeansModel] { @@ -256,7 +249,12 @@ class BisectingKMeans @Since("2.0.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): BisectingKMeansModel = instrumented { instr => transformSchema(dataset.schema, logging = true) + + val handlePersistence = dataset.storageLevel == StorageLevel.NONE val rdd = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) + if (handlePersistence) { + rdd.persist(StorageLevel.MEMORY_AND_DISK) + } instr.logPipelineStage(this) instr.logDataset(dataset) @@ -271,8 +269,17 @@ class BisectingKMeans @Since("2.0.0") ( .setDistanceMeasure($(distanceMeasure)) val parentModel = bkm.run(rdd, Some(instr)) val model = copyValues(new BisectingKMeansModel(uid, parentModel).setParent(this)) + if (handlePersistence) { + rdd.unpersist() + } + val summary = new BisectingKMeansSummary( - model.transform(dataset), $(predictionCol), $(featuresCol), $(k), $(maxIter)) + model.transform(dataset), + $(predictionCol), + $(featuresCol), + $(k), + $(maxIter), + parentModel.trainingCost) instr.logNamedValue("clusterSizes", summary.clusterSizes) instr.logNumFeatures(model.clusterCenters.head.size) model.setSummary(Some(summary)) @@ -302,6 +309,8 @@ object BisectingKMeans extends DefaultParamsReadable[BisectingKMeans] { * @param featuresCol Name for column of features in `predictions`. * @param k Number of clusters. * @param numIter Number of iterations. + * @param trainingCost Sum of the cost to the nearest centroid for all points in the training + * dataset. This is equivalent to sklearn's inertia. */ @Since("2.1.0") @Experimental @@ -310,4 +319,6 @@ class BisectingKMeansSummary private[clustering] ( predictionCol: String, featuresCol: String, k: Int, - numIter: Int) extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) + numIter: Int, + @Since("3.0.0") val trainingCost: Double) + extends ClusteringSummary(predictions, predictionCol, featuresCol, k, numIter) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala index 88abc1605d69f..9a51d2f188460 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/GaussianMixture.scala @@ -33,9 +33,10 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.linalg.{Matrices => OldMatrices, Matrix => OldMatrix, Vector => OldVector, Vectors => OldVectors} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.storage.StorageLevel /** @@ -85,7 +86,8 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override val uid: String, @Since("2.0.0") val weights: Array[Double], @Since("2.0.0") val gaussians: Array[MultivariateGaussian]) - extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable { + extends Model[GaussianMixtureModel] with GaussianMixtureParams with MLWritable + with HasTrainingSummary[GaussianMixtureSummary] { /** @group setParam */ @Since("2.1.0") @@ -108,11 +110,29 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predUDF = udf((vector: Vector) => predict(vector)) - val probUDF = udf((vector: Vector) => predictProbability(vector)) - dataset - .withColumn($(predictionCol), predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) - .withColumn($(probabilityCol), probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predUDF = udf((vector: Vector) => predict(vector)) + predictionColNames :+= $(predictionCol) + predictionColumns :+= predUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if ($(probabilityCol).nonEmpty) { + val probUDF = udf((vector: Vector) => predictProbability(vector)) + predictionColNames :+= $(probabilityCol) + predictionColumns :+= probUDF(DatasetUtils.columnToVector(dataset, getFeaturesCol)) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GaussianMixtureModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() + } } @Since("2.0.0") @@ -120,12 +140,14 @@ class GaussianMixtureModel private[ml] ( validateAndTransformSchema(schema) } - private[clustering] def predict(features: Vector): Int = { + @Since("3.0.0") + def predict(features: Vector): Int = { val r = predictProbability(features) r.argmax } - private[clustering] def predictProbability(features: Vector): Vector = { + @Since("3.0.0") + def predictProbability(features: Vector): Vector = { val probs: Array[Double] = GaussianMixtureModel.computeProbabilities(features.asBreeze.toDenseVector, gaussians, weights) Vectors.dense(probs) @@ -160,28 +182,13 @@ class GaussianMixtureModel private[ml] ( @Since("2.0.0") override def write: MLWriter = new GaussianMixtureModel.GaussianMixtureModelWriter(this) - private var trainingSummary: Option[GaussianMixtureSummary] = None - - private[clustering] def setSummary(summary: Option[GaussianMixtureSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.0.0") - def summary: GaussianMixtureSummary = trainingSummary.getOrElse { - throw new RuntimeException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: GaussianMixtureSummary = super.summary + } @Since("2.0.0") @@ -342,10 +349,15 @@ class GaussianMixture @Since("2.0.0") ( val sc = dataset.sparkSession.sparkContext val numClusters = $(k) + val handlePersistence = dataset.storageLevel == StorageLevel.NONE val instances = dataset .select(DatasetUtils.columnToVector(dataset, getFeaturesCol)).rdd.map { case Row(features: Vector) => features - }.cache() + } + + if (handlePersistence) { + instances.persist(StorageLevel.MEMORY_AND_DISK) + } // Extract the number of features. val numFeatures = instances.first().size @@ -383,8 +395,8 @@ class GaussianMixture @Since("2.0.0") ( case (aggregator1, aggregator2) => aggregator1.merge(aggregator2) }) - bcWeights.destroy(blocking = false) - bcGaussians.destroy(blocking = false) + bcWeights.destroy() + bcGaussians.destroy() if (iter == 0) { val numSamples = sums.count @@ -422,8 +434,10 @@ class GaussianMixture @Since("2.0.0") ( logLikelihood = sums.logLikelihood // this is the freshly computed log-likelihood iter += 1 } + if (handlePersistence) { + instances.unpersist() + } - instances.unpersist(false) val gaussianDists = gaussians.map { case (mean, covVec) => val cov = GaussianMixture.unpackUpperTriangularMatrix(numFeatures, covVec.values) new MultivariateGaussian(mean, cov) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala index 498310d6644e1..b48a9665ec88c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/KMeans.scala @@ -21,7 +21,6 @@ import scala.collection.mutable import org.apache.hadoop.fs.Path -import org.apache.spark.SparkException import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.{Estimator, Model, PipelineStage} import org.apache.spark.ml.linalg.Vector @@ -32,8 +31,7 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.clustering.{DistanceMeasure, KMeans => MLlibKMeans, KMeansModel => MLlibKMeansModel} import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} import org.apache.spark.sql.functions.udf import org.apache.spark.sql.types.{IntegerType, StructType} import org.apache.spark.storage.StorageLevel @@ -107,7 +105,8 @@ private[clustering] trait KMeansParams extends Params with HasMaxIter with HasFe class KMeansModel private[ml] ( @Since("1.5.0") override val uid: String, private[clustering] val parentModel: MLlibKMeansModel) - extends Model[KMeansModel] with KMeansParams with GeneralMLWritable { + extends Model[KMeansModel] with KMeansParams with GeneralMLWritable + with HasTrainingSummary[KMeansSummary] { @Since("1.5.0") override def copy(extra: ParamMap): KMeansModel = { @@ -138,27 +137,12 @@ class KMeansModel private[ml] ( validateAndTransformSchema(schema) } - private[clustering] def predict(features: Vector): Int = parentModel.predict(features) + @Since("3.0.0") + def predict(features: Vector): Int = parentModel.predict(features) @Since("2.0.0") def clusterCenters: Array[Vector] = parentModel.clusterCenters.map(_.asML) - /** - * Return the K-means cost (sum of squared distances of points to their nearest center) for this - * model on the given data. - * - * @deprecated This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator - * instead. You can also get the cost on the training dataset in the summary. - */ - @deprecated("This method is deprecated and will be removed in 3.0.0. Use ClusteringEvaluator " + - "instead. You can also get the cost on the training dataset in the summary.", "2.4.0") - @Since("2.0.0") - def computeCost(dataset: Dataset[_]): Double = { - SchemaUtils.validateVectorCompatibleColumn(dataset.schema, getFeaturesCol) - val data = DatasetUtils.columnToOldVector(dataset, getFeaturesCol) - parentModel.computeCost(data) - } - /** * Returns a [[org.apache.spark.ml.util.GeneralMLWriter]] instance for this ML instance. * @@ -169,28 +153,12 @@ class KMeansModel private[ml] ( @Since("1.6.0") override def write: GeneralMLWriter = new GeneralMLWriter(this) - private var trainingSummary: Option[KMeansSummary] = None - - private[clustering] def setSummary(summary: Option[KMeansSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** - * Return true if there exists summary of model. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - /** * Gets summary of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("2.0.0") - def summary: KMeansSummary = trainingSummary.getOrElse { - throw new SparkException( - s"No training summary available for the ${this.getClass.getSimpleName}") - } + override def summary: KMeansSummary = super.summary } /** Helper class for storing model data */ @@ -279,7 +247,7 @@ object KMeansModel extends MLReadable[KMeansModel] { /** * K-means clustering with support for k-means|| initialization proposed by Bahmani et al. * - * @see Bahmani et al., Scalable k-means++. + * @see Bahmani et al., Scalable k-means++. */ @Since("1.5.0") class KMeans @Since("1.5.0") ( 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 84e73dc19a392..91201e7bd03f9 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 @@ -19,6 +19,8 @@ package org.apache.spark.ml.clustering import java.util.Locale +import breeze.linalg.normalize +import breeze.numerics.exp import org.apache.hadoop.fs.Path import org.json4s.DefaultFormats import org.json4s.JsonAST.JObject @@ -27,7 +29,7 @@ import org.json4s.jackson.JsonMethods._ import org.apache.spark.annotation.{DeveloperApi, Since} import org.apache.spark.internal.Logging import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.linalg.{Matrix, Vector, Vectors, VectorUDT} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasCheckpointInterval, HasFeaturesCol, HasMaxIter, HasSeed} import org.apache.spark.ml.util._ @@ -35,7 +37,7 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.clustering.{DistributedLDAModel => OldDistributedLDAModel, EMLDAOptimizer => OldEMLDAOptimizer, LDA => OldLDA, LDAModel => OldLDAModel, - LDAOptimizer => OldLDAOptimizer, LocalLDAModel => OldLocalLDAModel, + LDAOptimizer => OldLDAOptimizer, LDAUtils => OldLDAUtils, LocalLDAModel => OldLocalLDAModel, OnlineLDAOptimizer => OldOnlineLDAOptimizer} import org.apache.spark.mllib.linalg.{Vector => OldVector, Vectors => OldVectors} import org.apache.spark.mllib.linalg.MatrixImplicits._ @@ -43,8 +45,9 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.functions.{col, monotonically_increasing_id, udf} -import org.apache.spark.sql.types.{ArrayType, DoubleType, FloatType, StructType} +import org.apache.spark.sql.functions.{monotonically_increasing_id, udf} +import org.apache.spark.sql.types.StructType +import org.apache.spark.storage.StorageLevel import org.apache.spark.util.PeriodicCheckpointer import org.apache.spark.util.VersionUtils @@ -456,19 +459,47 @@ abstract class LDAModel private[ml] ( */ @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - if ($(topicDistributionCol).nonEmpty) { + transformSchema(dataset.schema, logging = true) - // TODO: Make the transformer natively in ml framework to avoid extra conversion. - val transformer = oldLocalModel.getTopicDistributionMethod + val func = getTopicDistributionMethod + val transformer = udf(func) + dataset.withColumn($(topicDistributionCol), + transformer(DatasetUtils.columnToVector(dataset, getFeaturesCol))) + } - val t = udf { (v: Vector) => transformer(OldVectors.fromML(v)).asML } - dataset.withColumn($(topicDistributionCol), - t(DatasetUtils.columnToVector(dataset, getFeaturesCol))).toDF() - } else { - logWarning("LDAModel.transform was called without any output columns. Set an output column" + - " such as topicDistributionCol to produce results.") - dataset.toDF() - } + /** + * Get a method usable as a UDF for `topicDistributions()` + */ + private def getTopicDistributionMethod: Vector => Vector = { + val expElogbeta = exp(OldLDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) + val oldModel = oldLocalModel + val docConcentrationBrz = oldModel.docConcentration.asBreeze + val gammaShape = oldModel.gammaShape + val k = oldModel.k + val gammaSeed = oldModel.seed + + vector: Vector => + if (vector.numNonzeros == 0) { + Vectors.zeros(k) + } else { + val (ids: List[Int], cts: Array[Double]) = vector match { + case v: DenseVector => (List.range(0, v.size), v.values) + case v: SparseVector => (v.indices.toList, v.values) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + + val (gamma, _, _) = OldOnlineLDAOptimizer.variationalTopicInference( + ids, + cts, + expElogbeta, + docConcentrationBrz, + gammaShape, + k, + gammaSeed) + Vectors.dense(normalize(gamma, 1.0).toArray) + } } @Since("1.6.0") @@ -904,6 +935,18 @@ class LDA @Since("1.6.0") ( checkpointInterval, keepLastCheckpoint, optimizeDocConcentration, topicConcentration, learningDecay, optimizer, learningOffset, seed) + val oldData = LDA.getOldDataset(dataset, $(featuresCol)) + + // The EM solver will transform this oldData to a graph, and use a internal graphCheckpointer + // to update and cache the graph, so we do not need to cache it. + // The Online solver directly perform sampling on the oldData and update the model. + // However, Online solver will not cache the dataset internally. + val handlePersistence = dataset.storageLevel == StorageLevel.NONE && + getOptimizer.toLowerCase(Locale.ROOT) == "online" + if (handlePersistence) { + oldData.persist(StorageLevel.MEMORY_AND_DISK) + } + val oldLDA = new OldLDA() .setK($(k)) .setDocConcentration(getOldDocConcentration) @@ -912,8 +955,7 @@ class LDA @Since("1.6.0") ( .setSeed($(seed)) .setCheckpointInterval($(checkpointInterval)) .setOptimizer(getOldOptimizer) - // TODO: persist here, or in old LDA? - val oldData = LDA.getOldDataset(dataset, $(featuresCol)) + val oldModel = oldLDA.run(oldData) val newModel = oldModel match { case m: OldLocalLDAModel => @@ -921,6 +963,9 @@ class LDA @Since("1.6.0") ( case m: OldDistributedLDAModel => new DistributedLDAModel(uid, m.vocabSize, m, dataset.sparkSession, None) } + if (handlePersistence) { + oldData.unpersist() + } instr.logNumFeatures(newModel.vocabSize) copyValues(newModel).setParent(this) @@ -940,8 +985,8 @@ object LDA extends MLReadable[LDA] { dataset: Dataset[_], featuresCol: String): RDD[(Long, OldVector)] = { dataset - .withColumn("docId", monotonically_increasing_id()) - .select(col("docId"), DatasetUtils.columnToVector(dataset, featuresCol)) + .select(monotonically_increasing_id(), + DatasetUtils.columnToVector(dataset, featuresCol)) .rdd .map { case Row(docId: Long, features: Vector) => (docId, OldVectors.fromML(features)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala index 1b9a3499947d9..149e99d2f195a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/clustering/PowerIterationClustering.scala @@ -97,8 +97,8 @@ private[clustering] trait PowerIterationClusteringParams extends Params with Has /** * :: Experimental :: * Power Iteration Clustering (PIC), a scalable graph clustering algorithm developed by - * Lin and Cohen. From the abstract: - * PIC finds a very low-dimensional embedding of a dataset using truncated power + * Lin and Cohen. From + * the abstract: PIC finds a very low-dimensional embedding of a dataset using truncated power * iteration on a normalized pair-wise similarity matrix of the data. * * This class is not yet an Estimator/Transformer, use `assignClusters` method to run the @@ -166,6 +166,7 @@ class PowerIterationClustering private[clustering] ( val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) { lit(1.0) } else { + SchemaUtils.checkNumericType(dataset.schema, $(weightCol)) col($(weightCol)).cast(DoubleType) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala index bff72b20e1c3f..c6b04333885ae 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluator.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.types.DoubleType @Since("1.2.0") @Experimental class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Evaluator with HasRawPredictionCol with HasLabelCol with DefaultParamsWritable { + extends Evaluator with HasRawPredictionCol with HasLabelCol + with HasWeightCol with DefaultParamsWritable { @Since("1.2.0") def this() = this(Identifiable.randomUID("binEval")) @@ -68,6 +69,10 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va @Since("1.2.0") def setLabelCol(value: String): this.type = set(labelCol, value) + /** @group setParam */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + setDefault(metricName -> "areaUnderROC") @Since("2.0.0") @@ -75,14 +80,23 @@ class BinaryClassificationEvaluator @Since("1.4.0") (@Since("1.4.0") override va val schema = dataset.schema SchemaUtils.checkColumnTypes(schema, $(rawPredictionCol), Seq(DoubleType, new VectorUDT)) SchemaUtils.checkNumericType(schema, $(labelCol)) + if (isDefined(weightCol)) { + SchemaUtils.checkNumericType(schema, $(weightCol)) + } // TODO: When dataset metadata has been implemented, check rawPredictionCol vector length = 2. - val scoreAndLabels = - dataset.select(col($(rawPredictionCol)), col($(labelCol)).cast(DoubleType)).rdd.map { - case Row(rawPrediction: Vector, label: Double) => (rawPrediction(1), label) - case Row(rawPrediction: Double, label: Double) => (rawPrediction, label) + val scoreAndLabelsWithWeights = + dataset.select( + col($(rawPredictionCol)), + col($(labelCol)).cast(DoubleType), + if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) + else col($(weightCol)).cast(DoubleType)).rdd.map { + case Row(rawPrediction: Vector, label: Double, weight: Double) => + (rawPrediction(1), label, weight) + case Row(rawPrediction: Double, label: Double, weight: Double) => + (rawPrediction, label, weight) } - val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val metrics = new BinaryClassificationMetrics(scoreAndLabelsWithWeights) val metric = $(metricName) match { case "areaUnderROC" => metrics.areaUnderROC() case "areaUnderPR" => metrics.areaUnderPR() diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala index 5c1d1aebdc315..4c915e08d2536 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/ClusteringEvaluator.scala @@ -146,27 +146,27 @@ private[evaluation] abstract class Silhouette { pointClusterId: Double, pointClusterNumOfPoints: Long, averageDistanceToCluster: (Double) => Double): Double = { - // Here we compute the average dissimilarity of the current point to any cluster of which the - // point is not a member. - // The cluster with the lowest average dissimilarity - i.e. the nearest cluster to the current - // point - is said to be the "neighboring cluster". - val otherClusterIds = clusterIds.filter(_ != pointClusterId) - val neighboringClusterDissimilarity = otherClusterIds.map(averageDistanceToCluster).min - - // adjustment for excluding the node itself from the computation of the average dissimilarity - val currentClusterDissimilarity = if (pointClusterNumOfPoints == 1) { + if (pointClusterNumOfPoints == 1) { + // Single-element clusters have silhouette 0 0.0 } else { - averageDistanceToCluster(pointClusterId) * pointClusterNumOfPoints / - (pointClusterNumOfPoints - 1) - } - - if (currentClusterDissimilarity < neighboringClusterDissimilarity) { - 1 - (currentClusterDissimilarity / neighboringClusterDissimilarity) - } else if (currentClusterDissimilarity > neighboringClusterDissimilarity) { - (neighboringClusterDissimilarity / currentClusterDissimilarity) - 1 - } else { - 0.0 + // Here we compute the average dissimilarity of the current point to any cluster of which the + // point is not a member. + // The cluster with the lowest average dissimilarity - i.e. the nearest cluster to the current + // point - is said to be the "neighboring cluster". + val otherClusterIds = clusterIds.filter(_ != pointClusterId) + val neighboringClusterDissimilarity = otherClusterIds.map(averageDistanceToCluster).min + // adjustment for excluding the node itself from the computation of the average dissimilarity + val currentClusterDissimilarity = + averageDistanceToCluster(pointClusterId) * pointClusterNumOfPoints / + (pointClusterNumOfPoints - 1) + if (currentClusterDissimilarity < neighboringClusterDissimilarity) { + 1 - (currentClusterDissimilarity / neighboringClusterDissimilarity) + } else if (currentClusterDissimilarity > neighboringClusterDissimilarity) { + (neighboringClusterDissimilarity / currentClusterDissimilarity) - 1 + } else { + 0.0 + } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala index 794b1e7d9d881..e5cbe55e1d2ef 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluator.scala @@ -18,8 +18,8 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} -import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol} +import org.apache.spark.ml.param.{DoubleParam, Param, ParamMap, ParamValidators} +import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol, HasWeightCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.MulticlassMetrics import org.apache.spark.sql.{Dataset, Row} @@ -33,7 +33,10 @@ import org.apache.spark.sql.types.DoubleType @Since("1.5.0") @Experimental class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasLabelCol + with HasWeightCol with DefaultParamsWritable { + + import MulticlassClassificationEvaluator.supportedMetricNames @Since("1.5.0") def this() = this(Identifiable.randomUID("mcEval")) @@ -44,12 +47,9 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid * @group param */ @Since("1.5.0") - val metricName: Param[String] = { - val allowedParams = ParamValidators.inArray(Array("f1", "weightedPrecision", - "weightedRecall", "accuracy")) - new Param(this, "metricName", "metric name in evaluation " + - "(f1|weightedPrecision|weightedRecall|accuracy)", allowedParams) - } + val metricName: Param[String] = new Param(this, "metricName", + s"metric name in evaluation ${supportedMetricNames.mkString("(", "|", ")")}", + ParamValidators.inArray(supportedMetricNames)) /** @group getParam */ @Since("1.5.0") @@ -59,6 +59,8 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") def setMetricName(value: String): this.type = set(metricName, value) + setDefault(metricName -> "f1") + /** @group setParam */ @Since("1.5.0") def setPredictionCol(value: String): this.type = set(predictionCol, value) @@ -67,7 +69,43 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid @Since("1.5.0") def setLabelCol(value: String): this.type = set(labelCol, value) - setDefault(metricName -> "f1") + /** @group setParam */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + + @Since("3.0.0") + final val metricLabel: DoubleParam = new DoubleParam(this, "metricLabel", + "The class whose metric will be computed in " + + s"${supportedMetricNames.filter(_.endsWith("ByLabel")).mkString("(", "|", ")")}. " + + "Must be >= 0. The default value is 0.", + ParamValidators.gtEq(0.0)) + + /** @group getParam */ + @Since("3.0.0") + def getMetricLabel: Double = $(metricLabel) + + /** @group setParam */ + @Since("3.0.0") + def setMetricLabel(value: Double): this.type = set(metricLabel, value) + + setDefault(metricLabel -> 0.0) + + @Since("3.0.0") + final val beta: DoubleParam = new DoubleParam(this, "beta", + "The beta value, which controls precision vs recall weighting, " + + "used in (weightedFMeasure|fMeasureByLabel). Must be > 0. The default value is 1.", + ParamValidators.gt(0.0)) + + /** @group getParam */ + @Since("3.0.0") + def getBeta: Double = $(beta) + + /** @group setParam */ + @Since("3.0.0") + def setBeta(value: Double): this.type = set(beta, value) + + setDefault(beta -> 1.0) + @Since("2.0.0") override def evaluate(dataset: Dataset[_]): Double = { @@ -75,22 +113,37 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid SchemaUtils.checkColumnType(schema, $(predictionCol), DoubleType) SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabels = - dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType)).rdd.map { - case Row(prediction: Double, label: Double) => (prediction, label) + val predictionAndLabelsWithWeights = + dataset.select(col($(predictionCol)), col($(labelCol)).cast(DoubleType), + if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))) + .rdd.map { + case Row(prediction: Double, label: Double, weight: Double) => (prediction, label, weight) } - val metrics = new MulticlassMetrics(predictionAndLabels) - val metric = $(metricName) match { + val metrics = new MulticlassMetrics(predictionAndLabelsWithWeights) + $(metricName) match { case "f1" => metrics.weightedFMeasure + case "accuracy" => metrics.accuracy case "weightedPrecision" => metrics.weightedPrecision case "weightedRecall" => metrics.weightedRecall - case "accuracy" => metrics.accuracy + case "weightedTruePositiveRate" => metrics.weightedTruePositiveRate + case "weightedFalsePositiveRate" => metrics.weightedFalsePositiveRate + case "weightedFMeasure" => metrics.weightedFMeasure($(beta)) + case "truePositiveRateByLabel" => metrics.truePositiveRate($(metricLabel)) + case "falsePositiveRateByLabel" => metrics.falsePositiveRate($(metricLabel)) + case "precisionByLabel" => metrics.precision($(metricLabel)) + case "recallByLabel" => metrics.recall($(metricLabel)) + case "fMeasureByLabel" => metrics.fMeasure($(metricLabel), $(beta)) } - metric } @Since("1.5.0") - override def isLargerBetter: Boolean = true + override def isLargerBetter: Boolean = { + $(metricName) match { + case "weightedFalsePositiveRate" => false + case "falsePositiveRateByLabel" => false + case _ => true + } + } @Since("1.5.0") override def copy(extra: ParamMap): MulticlassClassificationEvaluator = defaultCopy(extra) @@ -100,6 +153,11 @@ class MulticlassClassificationEvaluator @Since("1.5.0") (@Since("1.5.0") overrid object MulticlassClassificationEvaluator extends DefaultParamsReadable[MulticlassClassificationEvaluator] { + private val supportedMetricNames = Array("f1", "accuracy", "weightedPrecision", "weightedRecall", + "weightedTruePositiveRate", "weightedFalsePositiveRate", "weightedFMeasure", + "truePositiveRateByLabel", "falsePositiveRateByLabel", "precisionByLabel", "recallByLabel", + "fMeasureByLabel") + @Since("1.6.0") override def load(path: String): MulticlassClassificationEvaluator = super.load(path) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.scala new file mode 100644 index 0000000000000..f12c6700be042 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluator.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.evaluation + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.evaluation.MultilabelMetrics +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + + +/** + * :: Experimental :: + * Evaluator for multi-label classification, which expects two input + * columns: prediction and label. + */ +@Since("3.0.0") +@Experimental +class MultilabelClassificationEvaluator (override val uid: String) + extends Evaluator with HasPredictionCol with HasLabelCol + with DefaultParamsWritable { + + import MultilabelClassificationEvaluator.supportedMetricNames + + def this() = this(Identifiable.randomUID("mlcEval")) + + /** + * param for metric name in evaluation (supports `"f1Measure"` (default), `"subsetAccuracy"`, + * `"accuracy"`, `"hammingLoss"`, `"precision"`, `"recall"`, `"precisionByLabel"`, + * `"recallByLabel"`, `"f1MeasureByLabel"`, `"microPrecision"`, `"microRecall"`, + * `"microF1Measure"`) + * @group param + */ + final val metricName: Param[String] = { + val allowedParams = ParamValidators.inArray(supportedMetricNames) + new Param(this, "metricName", "metric name in evaluation " + + s"${supportedMetricNames.mkString("(", "|", ")")}", allowedParams) + } + + /** @group getParam */ + def getMetricName: String = $(metricName) + + /** @group setParam */ + def setMetricName(value: String): this.type = set(metricName, value) + + setDefault(metricName -> "f1Measure") + + final val metricLabel: DoubleParam = new DoubleParam(this, "metricLabel", + "The class whose metric will be computed in " + + s"${supportedMetricNames.filter(_.endsWith("ByLabel")).mkString("(", "|", ")")}. " + + "Must be >= 0. The default value is 0.", + ParamValidators.gtEq(0.0)) + + /** @group getParam */ + def getMetricLabel: Double = $(metricLabel) + + /** @group setParam */ + def setMetricLabel(value: Double): this.type = set(metricLabel, value) + + setDefault(metricLabel -> 0.0) + + /** @group setParam */ + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + def setLabelCol(value: String): this.type = set(labelCol, value) + + + override def evaluate(dataset: Dataset[_]): Double = { + val schema = dataset.schema + SchemaUtils.checkColumnTypes(schema, $(predictionCol), + Seq(ArrayType(DoubleType, false), ArrayType(DoubleType, true))) + SchemaUtils.checkColumnTypes(schema, $(labelCol), + Seq(ArrayType(DoubleType, false), ArrayType(DoubleType, true))) + + val predictionAndLabels = + dataset.select(col($(predictionCol)), col($(labelCol))) + .rdd.map { row => + (row.getSeq[Double](0).toArray, row.getSeq[Double](1).toArray) + } + val metrics = new MultilabelMetrics(predictionAndLabels) + $(metricName) match { + case "subsetAccuracy" => metrics.subsetAccuracy + case "accuracy" => metrics.accuracy + case "hammingLoss" => metrics.hammingLoss + case "precision" => metrics.precision + case "recall" => metrics.recall + case "f1Measure" => metrics.f1Measure + case "precisionByLabel" => metrics.precision($(metricLabel)) + case "recallByLabel" => metrics.recall($(metricLabel)) + case "f1MeasureByLabel" => metrics.f1Measure($(metricLabel)) + case "microPrecision" => metrics.microPrecision + case "microRecall" => metrics.microRecall + case "microF1Measure" => metrics.microF1Measure + } + } + + override def isLargerBetter: Boolean = { + $(metricName) match { + case "hammingLoss" => false + case _ => true + } + } + + override def copy(extra: ParamMap): MultilabelClassificationEvaluator = defaultCopy(extra) +} + + +@Since("3.0.0") +object MultilabelClassificationEvaluator + extends DefaultParamsReadable[MultilabelClassificationEvaluator] { + + private val supportedMetricNames: Array[String] = Array("subsetAccuracy", + "accuracy", "hammingLoss", "precision", "recall", "f1Measure", + "precisionByLabel", "recallByLabel", "f1MeasureByLabel", + "microPrecision", "microRecall", "microF1Measure") + + override def load(path: String): MultilabelClassificationEvaluator = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala new file mode 100644 index 0000000000000..64ab3c3f7fddd --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RankingEvaluator.scala @@ -0,0 +1,118 @@ +/* + * 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.evaluation + +import org.apache.spark.annotation.{Experimental, Since} +import org.apache.spark.ml.param._ +import org.apache.spark.ml.param.shared._ +import org.apache.spark.ml.util._ +import org.apache.spark.mllib.evaluation.RankingMetrics +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * :: Experimental :: + * Evaluator for ranking, which expects two input columns: prediction and label. + */ +@Experimental +@Since("3.0.0") +class RankingEvaluator (override val uid: String) + extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { + + import RankingEvaluator.supportedMetricNames + + def this() = this(Identifiable.randomUID("rankEval")) + + /** + * param for metric name in evaluation (supports `"meanAveragePrecision"` (default), + * `"meanAveragePrecisionAtK"`, `"precisionAtK"`, `"ndcgAtK"`, `"recallAtK"`) + * @group param + */ + final val metricName: Param[String] = { + val allowedParams = ParamValidators.inArray(supportedMetricNames) + new Param(this, "metricName", "metric name in evaluation " + + s"${supportedMetricNames.mkString("(", "|", ")")}", allowedParams) + } + + /** @group getParam */ + def getMetricName: String = $(metricName) + + /** @group setParam */ + def setMetricName(value: String): this.type = set(metricName, value) + + setDefault(metricName -> "meanAveragePrecision") + + final val k = new IntParam(this, "k", + "The ranking position value used in " + + s"${supportedMetricNames.filter(_.endsWith("AtK")).mkString("(", "|", ")")} " + + "Must be > 0. The default value is 10.", + ParamValidators.gt(0)) + + /** @group getParam */ + def getK: Int = $(k) + + /** @group setParam */ + def setK(value: Int): this.type = set(k, value) + + setDefault(k -> 10) + + /** @group setParam */ + def setPredictionCol(value: String): this.type = set(predictionCol, value) + + /** @group setParam */ + def setLabelCol(value: String): this.type = set(labelCol, value) + + + override def evaluate(dataset: Dataset[_]): Double = { + val schema = dataset.schema + SchemaUtils.checkColumnTypes(schema, $(predictionCol), + Seq(ArrayType(DoubleType, false), ArrayType(DoubleType, true))) + SchemaUtils.checkColumnTypes(schema, $(labelCol), + Seq(ArrayType(DoubleType, false), ArrayType(DoubleType, true))) + + val predictionAndLabels = + dataset.select(col($(predictionCol)), col($(labelCol))) + .rdd.map { row => + (row.getSeq[Double](0).toArray, row.getSeq[Double](1).toArray) + } + val metrics = new RankingMetrics[Double](predictionAndLabels) + $(metricName) match { + case "meanAveragePrecision" => metrics.meanAveragePrecision + case "meanAveragePrecisionAtK" => metrics.meanAveragePrecisionAt($(k)) + case "precisionAtK" => metrics.precisionAt($(k)) + case "ndcgAtK" => metrics.ndcgAt($(k)) + case "recallAtK" => metrics.recallAt($(k)) + } + } + + override def isLargerBetter: Boolean = true + + override def copy(extra: ParamMap): RankingEvaluator = defaultCopy(extra) +} + + +@Since("3.0.0") +object RankingEvaluator extends DefaultParamsReadable[RankingEvaluator] { + + private val supportedMetricNames = Array("meanAveragePrecision", + "meanAveragePrecisionAtK", "precisionAtK", "ndcgAtK", "recallAtK") + + override def load(path: String): RankingEvaluator = super.load(path) +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala index 031cd0d635bf4..616569bb55e4c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/evaluation/RegressionEvaluator.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.evaluation import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param.{Param, ParamMap, ParamValidators} -import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol} +import org.apache.spark.ml.param.shared.{HasLabelCol, HasPredictionCol, HasWeightCol} import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable, SchemaUtils} import org.apache.spark.mllib.evaluation.RegressionMetrics import org.apache.spark.sql.{Dataset, Row} @@ -33,7 +33,8 @@ import org.apache.spark.sql.types.{DoubleType, FloatType} @Since("1.4.0") @Experimental final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val uid: String) - extends Evaluator with HasPredictionCol with HasLabelCol with DefaultParamsWritable { + extends Evaluator with HasPredictionCol with HasLabelCol + with HasWeightCol with DefaultParamsWritable { @Since("1.4.0") def this() = this(Identifiable.randomUID("regEval")) @@ -69,6 +70,10 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui @Since("1.4.0") def setLabelCol(value: String): this.type = set(labelCol, value) + /** @group setParam */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + setDefault(metricName -> "rmse") @Since("2.0.0") @@ -77,11 +82,13 @@ final class RegressionEvaluator @Since("1.4.0") (@Since("1.4.0") override val ui SchemaUtils.checkColumnTypes(schema, $(predictionCol), Seq(DoubleType, FloatType)) SchemaUtils.checkNumericType(schema, $(labelCol)) - val predictionAndLabels = dataset - .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType)) + val predictionAndLabelsWithWeights = dataset + .select(col($(predictionCol)).cast(DoubleType), col($(labelCol)).cast(DoubleType), + if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol))) .rdd - .map { case Row(prediction: Double, label: Double) => (prediction, label) } - val metrics = new RegressionMetrics(predictionAndLabels) + .map { case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) } + val metrics = new RegressionMetrics(predictionAndLabelsWithWeights) val metric = $(metricName) match { case "rmse" => metrics.rootMeanSquaredError case "mse" => metrics.meanSquaredError diff --git a/mllib/src/main/scala/org/apache/spark/ml/events.scala b/mllib/src/main/scala/org/apache/spark/ml/events.scala new file mode 100644 index 0000000000000..dc4be4dd9efda --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/events.scala @@ -0,0 +1,182 @@ +/* + * 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 + +import com.fasterxml.jackson.annotation.JsonIgnore + +import org.apache.spark.SparkContext +import org.apache.spark.annotation.Unstable +import org.apache.spark.internal.Logging +import org.apache.spark.ml.util.{MLReader, MLWriter} +import org.apache.spark.scheduler.SparkListenerEvent +import org.apache.spark.sql.{DataFrame, Dataset} + +/** + * Event emitted by ML operations. Events are either fired before and/or + * after each operation (the event should document this). + * + * @note This is supported via [[Pipeline]] and [[PipelineModel]]. + * @note This is experimental and unstable. Do not use this unless you fully + * understand what `Unstable` means. + */ +@Unstable +sealed trait MLEvent extends SparkListenerEvent { + // Do not log ML events in event log. It should be revisited to see + // how it works with history server. + protected[spark] override def logEvent: Boolean = false +} + +/** + * Event fired before `Transformer.transform`. + */ +@Unstable +case class TransformStart() extends MLEvent { + @JsonIgnore var transformer: Transformer = _ + @JsonIgnore var input: Dataset[_] = _ +} + +/** + * Event fired after `Transformer.transform`. + */ +@Unstable +case class TransformEnd() extends MLEvent { + @JsonIgnore var transformer: Transformer = _ + @JsonIgnore var output: Dataset[_] = _ +} + +/** + * Event fired before `Estimator.fit`. + */ +@Unstable +case class FitStart[M <: Model[M]]() extends MLEvent { + @JsonIgnore var estimator: Estimator[M] = _ + @JsonIgnore var dataset: Dataset[_] = _ +} + +/** + * Event fired after `Estimator.fit`. + */ +@Unstable +case class FitEnd[M <: Model[M]]() extends MLEvent { + @JsonIgnore var estimator: Estimator[M] = _ + @JsonIgnore var model: M = _ +} + +/** + * Event fired before `MLReader.load`. + */ +@Unstable +case class LoadInstanceStart[T](path: String) extends MLEvent { + @JsonIgnore var reader: MLReader[T] = _ +} + +/** + * Event fired after `MLReader.load`. + */ +@Unstable +case class LoadInstanceEnd[T]() extends MLEvent { + @JsonIgnore var reader: MLReader[T] = _ + @JsonIgnore var instance: T = _ +} + +/** + * Event fired before `MLWriter.save`. + */ +@Unstable +case class SaveInstanceStart(path: String) extends MLEvent { + @JsonIgnore var writer: MLWriter = _ +} + +/** + * Event fired after `MLWriter.save`. + */ +@Unstable +case class SaveInstanceEnd(path: String) extends MLEvent { + @JsonIgnore var writer: MLWriter = _ +} + +/** + * A small trait that defines some methods to send [[org.apache.spark.ml.MLEvent]]. + */ +private[ml] trait MLEvents extends Logging { + + private def listenerBus = SparkContext.getOrCreate().listenerBus + + /** + * Log [[MLEvent]] to send. By default, it emits a debug-level log. + */ + def logEvent(event: MLEvent): Unit = logDebug(s"Sending an MLEvent: $event") + + def withFitEvent[M <: Model[M]]( + estimator: Estimator[M], dataset: Dataset[_])(func: => M): M = { + val startEvent = FitStart[M]() + startEvent.estimator = estimator + startEvent.dataset = dataset + logEvent(startEvent) + listenerBus.post(startEvent) + val model: M = func + val endEvent = FitEnd[M]() + endEvent.estimator = estimator + endEvent.model = model + logEvent(endEvent) + listenerBus.post(endEvent) + model + } + + def withTransformEvent( + transformer: Transformer, input: Dataset[_])(func: => DataFrame): DataFrame = { + val startEvent = TransformStart() + startEvent.transformer = transformer + startEvent.input = input + logEvent(startEvent) + listenerBus.post(startEvent) + val output: DataFrame = func + val endEvent = TransformEnd() + endEvent.transformer = transformer + endEvent.output = output + logEvent(endEvent) + listenerBus.post(endEvent) + output + } + + def withLoadInstanceEvent[T](reader: MLReader[T], path: String)(func: => T): T = { + val startEvent = LoadInstanceStart[T](path) + startEvent.reader = reader + logEvent(startEvent) + listenerBus.post(startEvent) + val instance: T = func + val endEvent = LoadInstanceEnd[T]() + endEvent.reader = reader + endEvent.instance = instance + logEvent(endEvent) + listenerBus.post(endEvent) + instance + } + + def withSaveInstanceEvent(writer: MLWriter, path: String)(func: => Unit): Unit = { + val startEvent = SaveInstanceStart(path) + startEvent.writer = writer + logEvent(startEvent) + listenerBus.post(startEvent) + func + val endEvent = SaveInstanceEnd(path) + endEvent.writer = writer + logEvent(endEvent) + listenerBus.post(endEvent) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala index f99649f7fa164..16073d5fc1b6b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Bucketizer.scala @@ -19,10 +19,6 @@ package org.apache.spark.ml.feature import java.{util => ju} -import org.json4s.JsonDSL._ -import org.json4s.JValue -import org.json4s.jackson.JsonMethods._ - import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.Model @@ -89,7 +85,8 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String def setOutputCol(value: String): this.type = set(outputCol, value) /** - * Param for how to handle invalid entries. Options are 'skip' (filter out rows with + * Param for how to handle invalid entries containing NaN values. Values outside the splits + * will always be treated as errors. Options are 'skip' (filter out rows with * invalid values), 'error' (throw an error), or 'keep' (keep invalid values in a special * additional bucket). Note that in the multiple column case, the invalid handling is applied * to all columns. That said for 'error' it will throw an error if any invalids are found in @@ -99,7 +96,8 @@ final class Bucketizer @Since("1.4.0") (@Since("1.4.0") override val uid: String */ @Since("2.1.0") override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", - "how to handle invalid entries. Options are skip (filter out rows with invalid values), " + + "how to handle invalid entries containing NaN values. Values outside the splits will always " + + "be treated as errorsOptions are skip (filter out rows with invalid values), " + "error (throw an error), or keep (keep invalid values in a special additional bucket).", ParamValidators.inArray(Bucketizer.supportedHandleInvalids)) 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 dbfb199ccd58f..2a3656c49584e 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 @@ -22,7 +22,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.ml._ import org.apache.spark.ml.attribute.{AttributeGroup, _} -import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ @@ -264,14 +264,25 @@ final class ChiSqSelectorModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - val transformedSchema = transformSchema(dataset.schema, logging = true) - val newField = transformedSchema.last - - // TODO: Make the transformer natively in ml framework to avoid extra conversion. - val transformer: Vector => Vector = v => chiSqSelector.transform(OldVectors.fromML(v)).asML + val outputSchema = transformSchema(dataset.schema, logging = true) + + val newSize = selectedFeatures.length + val func = { vector: Vector => + vector match { + case SparseVector(_, indices, values) => + val (newIndices, newValues) = chiSqSelector.compressSparse(indices, values) + Vectors.sparse(newSize, newIndices, newValues) + case DenseVector(values) => + Vectors.dense(chiSqSelector.compressDense(values)) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } - val selector = udf(transformer) - dataset.withColumn($(outputCol), selector(col($(featuresCol))), newField.metadata) + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(featuresCol))), + outputSchema($(outputCol)).metadata) } @Since("1.6.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala index f860b3a787b4d..ece125ba8ac70 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/ElementwiseProduct.scala @@ -19,11 +19,11 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.Since import org.apache.spark.ml.UnaryTransformer -import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param.Param import org.apache.spark.ml.util.{DefaultParamsReadable, DefaultParamsWritable, Identifiable} -import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.VectorImplicits._ +import org.apache.spark.mllib.feature.{ElementwiseProduct => OldElementwiseProduct} +import org.apache.spark.mllib.linalg.{Vectors => OldVectors} import org.apache.spark.sql.types.DataType /** @@ -55,8 +55,24 @@ class ElementwiseProduct @Since("1.4.0") (@Since("1.4.0") override val uid: Stri override protected def createTransformFunc: Vector => Vector = { require(params.contains(scalingVec), s"transformation requires a weight vector") - val elemScaler = new feature.ElementwiseProduct($(scalingVec)) - v => elemScaler.transform(v) + val elemScaler = new OldElementwiseProduct(OldVectors.fromML($(scalingVec))) + val vectorSize = $(scalingVec).size + + vector: Vector => { + require(vector.size == vectorSize, + s"vector sizes do not match: Expected $vectorSize but found ${vector.size}") + vector match { + case DenseVector(values) => + val newValues = elemScaler.transformDense(values) + Vectors.dense(newValues) + case SparseVector(size, indices, values) => + val (newIndices, newValues) = elemScaler.transformSparse(indices, values) + Vectors.sparse(size, newIndices, newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } } override protected def outputDataType: DataType = new VectorUDT() diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala index dbda5b8d8fd4a..4e4a61d8bec65 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/HashingTF.scala @@ -20,10 +20,11 @@ package org.apache.spark.ml.feature import org.apache.spark.annotation.Since import org.apache.spark.ml.Transformer import org.apache.spark.ml.attribute.AttributeGroup +import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} import org.apache.spark.ml.util._ -import org.apache.spark.mllib.feature +import org.apache.spark.mllib.feature.{HashingTF => OldHashingTF} import org.apache.spark.sql.{DataFrame, Dataset} import org.apache.spark.sql.functions.{col, udf} import org.apache.spark.sql.types.{ArrayType, StructType} @@ -93,11 +94,16 @@ class HashingTF @Since("1.4.0") (@Since("1.4.0") override val uid: String) @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { val outputSchema = transformSchema(dataset.schema) - val hashingTF = new feature.HashingTF($(numFeatures)).setBinary($(binary)) - // TODO: Make the hashingTF.transform natively in ml framework to avoid extra conversion. - val t = udf { terms: Seq[_] => hashingTF.transform(terms).asML } - val metadata = outputSchema($(outputCol)).metadata - dataset.select(col("*"), t(col($(inputCol))).as($(outputCol), metadata)) + + val hashingTF = new OldHashingTF($(numFeatures)).setBinary($(binary)) + val func = (terms: Seq[_]) => { + val seq = hashingTF.transformImpl(terms) + Vectors.sparse(hashingTF.numFeatures, seq) + } + + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(inputCol))), + outputSchema($(outputCol)).metadata) } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala index 58897cca4e5c6..4338421bf8bcf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/IDF.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.ml._ -import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ @@ -32,6 +32,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.StructType +import org.apache.spark.util.VersionUtils.majorVersion /** * Params for [[IDF]] and [[IDFModel]]. @@ -131,9 +132,24 @@ class IDFModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - // TODO: Make the idfModel.transform natively in ml framework to avoid extra conversion. - val idf = udf { vec: Vector => idfModel.transform(OldVectors.fromML(vec)).asML } - dataset.withColumn($(outputCol), idf(col($(inputCol)))) + + val func = { vector: Vector => + vector match { + case SparseVector(size, indices, values) => + val (newIndices, newValues) = feature.IDFModel.transformSparse(idfModel.idf, + indices, values) + Vectors.sparse(size, newIndices, newValues) + case DenseVector(values) => + val newValues = feature.IDFModel.transformDense(idfModel.idf, values) + Vectors.dense(newValues) + 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($(inputCol)))) } @Since("1.4.0") @@ -151,6 +167,15 @@ class IDFModel private[ml] ( @Since("2.0.0") def idf: Vector = idfModel.idf.asML + /** Returns the document frequency */ + @Since("3.0.0") + def docFreq: Array[Long] = idfModel.docFreq + + /** Returns number of documents evaluated to compute idf */ + @Since("3.0.0") + def numDocs: Long = idfModel.numDocs + + @Since("1.6.0") override def write: MLWriter = new IDFModelWriter(this) } @@ -160,11 +185,11 @@ object IDFModel extends MLReadable[IDFModel] { private[IDFModel] class IDFModelWriter(instance: IDFModel) extends MLWriter { - private case class Data(idf: Vector) + private case class Data(idf: Vector, docFreq: Array[Long], numDocs: Long) override protected def saveImpl(path: String): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.idf) + val data = Data(instance.idf, instance.docFreq, instance.numDocs) val dataPath = new Path(path, "data").toString sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } @@ -178,10 +203,19 @@ object IDFModel extends MLReadable[IDFModel] { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString val data = sparkSession.read.parquet(dataPath) - val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf") - .select("idf") - .head() - val model = new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf))) + + val model = if (majorVersion(metadata.sparkVersion) >= 3) { + val Row(idf: Vector, df: Seq[_], numDocs: Long) = data.select("idf", "docFreq", "numDocs") + .head() + new IDFModel(metadata.uid, new feature.IDFModel(OldVectors.fromML(idf), + df.asInstanceOf[Seq[Long]].toArray, numDocs)) + } else { + val Row(idf: Vector) = MLUtils.convertVectorColumnsToML(data, "idf") + .select("idf") + .head() + new IDFModel(metadata.uid, + new feature.IDFModel(OldVectors.fromML(idf), new Array[Long](idf.size), 0L)) + } metadata.getAndSetParams(model) model } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala index c5d0ec1a8d350..f6667b73304a3 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/LabeledPoint.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.feature -import scala.beans.BeanInfo - import org.apache.spark.annotation.Since import org.apache.spark.ml.linalg.Vector @@ -30,9 +28,22 @@ import org.apache.spark.ml.linalg.Vector * @param features List of features for this data point. */ @Since("2.0.0") -@BeanInfo case class LabeledPoint(@Since("2.0.0") label: Double, @Since("2.0.0") features: Vector) { + + def getLabel: Double = label + + def getFeatures: Vector = features + override def toString: String = { s"($label,$features)" } + + private[spark] def toInstance(weight: Double): Instance = { + Instance(label, weight, features) + } + + private[spark] def toInstance: Instance = { + Instance(label, 1.0, features) + } + } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala index 27e4869a020b7..ec9792cbbda8f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoder.scala @@ -17,126 +17,512 @@ package org.apache.spark.ml.feature +import org.apache.hadoop.fs.Path + +import org.apache.spark.SparkException import org.apache.spark.annotation.Since -import org.apache.spark.ml.Transformer +import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.attribute._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCols, HasOutputCols} import org.apache.spark.ml.util._ import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions.{col, udf} -import org.apache.spark.sql.types.{DoubleType, NumericType, StructType} +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions.{col, lit, udf} +import org.apache.spark.sql.types.{DoubleType, StructField, StructType} + +/** Private trait for params and common methods for OneHotEncoder and OneHotEncoderModel */ +private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid + with HasInputCols with HasOutputCols { + + /** + * Param for how to handle invalid data during transform(). + * Options are 'keep' (invalid data presented as an extra categorical feature) or + * 'error' (throw an error). + * Note that this Param is only used during transform; during fitting, invalid data + * will result in an error. + * Default: "error" + * @group param + */ + @Since("2.3.0") + override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", + "How to handle invalid data during transform(). " + + "Options are 'keep' (invalid data presented as an extra categorical feature) " + + "or error (throw an error). Note that this Param is only used during transform; " + + "during fitting, invalid data will result in an error.", + ParamValidators.inArray(OneHotEncoder.supportedHandleInvalids)) + + setDefault(handleInvalid, OneHotEncoder.ERROR_INVALID) + + /** + * Whether to drop the last category in the encoded vector (default: true) + * @group param + */ + @Since("2.3.0") + final val dropLast: BooleanParam = + new BooleanParam(this, "dropLast", "whether to drop the last category") + setDefault(dropLast -> true) + + /** @group getParam */ + @Since("2.3.0") + def getDropLast: Boolean = $(dropLast) + + protected def validateAndTransformSchema( + schema: StructType, + dropLast: Boolean, + keepInvalid: Boolean): StructType = { + val inputColNames = $(inputCols) + val outputColNames = $(outputCols) + + require(inputColNames.length == outputColNames.length, + s"The number of input columns ${inputColNames.length} must be the same as the number of " + + s"output columns ${outputColNames.length}.") + + // Input columns must be NumericType. + inputColNames.foreach(SchemaUtils.checkNumericType(schema, _)) + + // Prepares output columns with proper attributes by examining input columns. + val inputFields = $(inputCols).map(schema(_)) + + val outputFields = inputFields.zip(outputColNames).map { case (inputField, outputColName) => + OneHotEncoderCommon.transformOutputColumnSchema( + inputField, outputColName, dropLast, keepInvalid) + } + outputFields.foldLeft(schema) { case (newSchema, outputField) => + SchemaUtils.appendColumn(newSchema, outputField) + } + } +} /** * A one-hot encoder that maps a column of category indices to a column of binary vectors, with * at most a single one-value per row that indicates the input category index. * For example with 5 categories, an input value of 2.0 would map to an output vector of * `[0.0, 0.0, 1.0, 0.0]`. - * The last category is not included by default (configurable via `OneHotEncoder!.dropLast` + * The last category is not included by default (configurable via `dropLast`), * because it makes the vector entries sum up to one, and hence linearly dependent. * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. * * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories. * The output vectors are sparse. * + * When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is + * added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros + * vector. + * + * @note When encoding multi-column by using `inputCols` and `outputCols` params, input/output cols + * come in pairs, specified by the order in the arrays, and each pair is treated independently. + * * @see `StringIndexer` for converting categorical values into category indices - * @deprecated `OneHotEncoderEstimator` will be renamed `OneHotEncoder` and this `OneHotEncoder` - * will be removed in 3.0.0. */ -@Since("1.4.0") -@deprecated("`OneHotEncoderEstimator` will be renamed `OneHotEncoder` and this `OneHotEncoder`" + - " will be removed in 3.0.0.", "2.3.0") -class OneHotEncoder @Since("1.4.0") (@Since("1.4.0") override val uid: String) extends Transformer - with HasInputCol with HasOutputCol with DefaultParamsWritable { +@Since("3.0.0") +class OneHotEncoder @Since("3.0.0") (@Since("3.0.0") override val uid: String) + extends Estimator[OneHotEncoderModel] with OneHotEncoderBase with DefaultParamsWritable { - @Since("1.4.0") - def this() = this(Identifiable.randomUID("oneHot")) + @Since("3.0.0") + def this() = this(Identifiable.randomUID("oneHotEncoder")) - /** - * Whether to drop the last category in the encoded vector (default: true) - * @group param - */ - @Since("1.4.0") - final val dropLast: BooleanParam = - new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(values: Array[String]): this.type = set(inputCols, values) - /** @group getParam */ - @Since("2.0.0") - def getDropLast: Boolean = $(dropLast) + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(values: Array[String]): this.type = set(outputCols, values) /** @group setParam */ - @Since("1.4.0") + @Since("3.0.0") def setDropLast(value: Boolean): this.type = set(dropLast, value) /** @group setParam */ - @Since("1.4.0") - def setInputCol(value: String): this.type = set(inputCol, value) + @Since("3.0.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + + @Since("3.0.0") + override def transformSchema(schema: StructType): StructType = { + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID + validateAndTransformSchema(schema, dropLast = $(dropLast), + keepInvalid = keepInvalid) + } + + @Since("3.0.0") + override def fit(dataset: Dataset[_]): OneHotEncoderModel = { + transformSchema(dataset.schema) + + // Compute the plain number of categories without `handleInvalid` and + // `dropLast` taken into account. + val transformedSchema = validateAndTransformSchema(dataset.schema, dropLast = false, + keepInvalid = false) + val categorySizes = new Array[Int]($(outputCols).length) + + val columnToScanIndices = $(outputCols).zipWithIndex.flatMap { case (outputColName, idx) => + val numOfAttrs = AttributeGroup.fromStructField( + transformedSchema(outputColName)).size + if (numOfAttrs < 0) { + Some(idx) + } else { + categorySizes(idx) = numOfAttrs + None + } + } + + // Some input columns don't have attributes or their attributes don't have necessary info. + // We need to scan the data to get the number of values for each column. + if (columnToScanIndices.length > 0) { + val inputColNames = columnToScanIndices.map($(inputCols)(_)) + val outputColNames = columnToScanIndices.map($(outputCols)(_)) + + // When fitting data, we want the plain number of categories without `handleInvalid` and + // `dropLast` taken into account. + val attrGroups = OneHotEncoderCommon.getOutputAttrGroupFromData( + dataset, inputColNames, outputColNames, dropLast = false) + attrGroups.zip(columnToScanIndices).foreach { case (attrGroup, idx) => + categorySizes(idx) = attrGroup.size + } + } + + val model = new OneHotEncoderModel(uid, categorySizes).setParent(this) + copyValues(model) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) +} + +@Since("3.0.0") +object OneHotEncoder extends DefaultParamsReadable[OneHotEncoder] { + + private[feature] val KEEP_INVALID: String = "keep" + private[feature] val ERROR_INVALID: String = "error" + private[feature] val supportedHandleInvalids: Array[String] = Array(KEEP_INVALID, ERROR_INVALID) + + @Since("3.0.0") + override def load(path: String): OneHotEncoder = super.load(path) +} + +/** + * @param categorySizes Original number of categories for each feature being encoded. + * The array contains one value for each input column, in order. + */ +@Since("3.0.0") +class OneHotEncoderModel private[ml] ( + @Since("3.0.0") override val uid: String, + @Since("3.0.0") val categorySizes: Array[Int]) + extends Model[OneHotEncoderModel] with OneHotEncoderBase with MLWritable { + + import OneHotEncoderModel._ + + // Returns the category size for each index with `dropLast` and `handleInvalid` + // taken into account. + private def getConfigedCategorySizes: Array[Int] = { + val dropLast = getDropLast + val keepInvalid = getHandleInvalid == OneHotEncoder.KEEP_INVALID + + if (!dropLast && keepInvalid) { + // When `handleInvalid` is "keep", an extra category is added as last category + // for invalid data. + categorySizes.map(_ + 1) + } else if (dropLast && !keepInvalid) { + // When `dropLast` is true, the last category is removed. + categorySizes.map(_ - 1) + } else { + // When `dropLast` is true and `handleInvalid` is "keep", the extra category for invalid + // data is removed. Thus, it is the same as the plain number of categories. + categorySizes + } + } + + private def encoder: UserDefinedFunction = { + val keepInvalid = getHandleInvalid == OneHotEncoder.KEEP_INVALID + val configedSizes = getConfigedCategorySizes + val localCategorySizes = categorySizes + + // The udf performed on input data. The first parameter is the input value. The second + // parameter is the index in inputCols of the column being encoded. + udf { (label: Double, colIdx: Int) => + val origCategorySize = localCategorySizes(colIdx) + // idx: index in vector of the single 1-valued element + val idx = if (label >= 0 && label < origCategorySize) { + label + } else { + if (keepInvalid) { + origCategorySize + } else { + if (label < 0) { + throw new SparkException(s"Negative value: $label. Input can't be negative. " + + s"To handle invalid values, set Param handleInvalid to " + + s"${OneHotEncoder.KEEP_INVALID}") + } else { + throw new SparkException(s"Unseen value: $label. To handle unseen values, " + + s"set Param handleInvalid to ${OneHotEncoder.KEEP_INVALID}.") + } + } + } + + val size = configedSizes(colIdx) + if (idx < size) { + Vectors.sparse(size, Array(idx.toInt), Array(1.0)) + } else { + Vectors.sparse(size, Array.empty[Int], Array.empty[Double]) + } + } + } /** @group setParam */ - @Since("1.4.0") - def setOutputCol(value: String): this.type = set(outputCol, value) + @Since("3.0.0") + def setInputCols(values: Array[String]): this.type = set(inputCols, values) - @Since("1.4.0") + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(values: Array[String]): this.type = set(outputCols, values) + + /** @group setParam */ + @Since("3.0.0") + def setDropLast(value: Boolean): this.type = set(dropLast, value) + + /** @group setParam */ + @Since("3.0.0") + def setHandleInvalid(value: String): this.type = set(handleInvalid, value) + + @Since("3.0.0") override def transformSchema(schema: StructType): StructType = { - val inputColName = $(inputCol) - val outputColName = $(outputCol) - val inputFields = schema.fields + val inputColNames = $(inputCols) + + require(inputColNames.length == categorySizes.length, + s"The number of input columns ${inputColNames.length} must be the same as the number of " + + s"features ${categorySizes.length} during fitting.") + + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID + val transformedSchema = validateAndTransformSchema(schema, dropLast = $(dropLast), + keepInvalid = keepInvalid) + verifyNumOfValues(transformedSchema) + } - require(schema(inputColName).dataType.isInstanceOf[NumericType], - s"Input column must be of type ${NumericType.simpleString} but got " + - schema(inputColName).dataType.catalogString) - require(!inputFields.exists(_.name == outputColName), - s"Output column $outputColName already exists.") + /** + * If the metadata of input columns also specifies the number of categories, we need to + * compare with expected category number with `handleInvalid` and `dropLast` taken into + * account. Mismatched numbers will cause exception. + */ + private def verifyNumOfValues(schema: StructType): StructType = { + val configedSizes = getConfigedCategorySizes + $(outputCols).zipWithIndex.foreach { case (outputColName, idx) => + val inputColName = $(inputCols)(idx) + val attrGroup = AttributeGroup.fromStructField(schema(outputColName)) - val outputField = OneHotEncoderCommon.transformOutputColumnSchema( - schema(inputColName), outputColName, $(dropLast)) - val outputFields = inputFields :+ outputField - StructType(outputFields) + // If the input metadata specifies number of category for output column, + // comparing with expected category number with `handleInvalid` and + // `dropLast` taken into account. + if (attrGroup.attributes.nonEmpty) { + val numCategories = configedSizes(idx) + require(attrGroup.size == numCategories, "OneHotEncoderModel expected " + + s"$numCategories categorical values for input column $inputColName, " + + s"but the input column had metadata specifying ${attrGroup.size} values.") + } + } + schema } - @Since("2.0.0") + @Since("3.0.0") override def transform(dataset: Dataset[_]): DataFrame = { - // schema transformation - val inputColName = $(inputCol) - val outputColName = $(outputCol) + val transformedSchema = transformSchema(dataset.schema, logging = true) + val keepInvalid = $(handleInvalid) == OneHotEncoder.KEEP_INVALID - val outputAttrGroupFromSchema = AttributeGroup.fromStructField( - transformSchema(dataset.schema)(outputColName)) + val encodedColumns = $(inputCols).indices.map { idx => + val inputColName = $(inputCols)(idx) + val outputColName = $(outputCols)(idx) - val outputAttrGroup = if (outputAttrGroupFromSchema.size < 0) { - OneHotEncoderCommon.getOutputAttrGroupFromData( - dataset, Seq(inputColName), Seq(outputColName), $(dropLast))(0) - } else { - outputAttrGroupFromSchema + val outputAttrGroupFromSchema = + AttributeGroup.fromStructField(transformedSchema(outputColName)) + + val metadata = if (outputAttrGroupFromSchema.size < 0) { + OneHotEncoderCommon.createAttrGroupForAttrNames(outputColName, + categorySizes(idx), $(dropLast), keepInvalid).toMetadata() + } else { + outputAttrGroupFromSchema.toMetadata() + } + + encoder(col(inputColName).cast(DoubleType), lit(idx)) + .as(outputColName, metadata) + } + dataset.withColumns($(outputCols), encodedColumns) + } + + @Since("3.0.0") + override def copy(extra: ParamMap): OneHotEncoderModel = { + val copied = new OneHotEncoderModel(uid, categorySizes) + copyValues(copied, extra).setParent(parent) + } + + @Since("3.0.0") + override def write: MLWriter = new OneHotEncoderModelWriter(this) +} + +@Since("3.0.0") +object OneHotEncoderModel extends MLReadable[OneHotEncoderModel] { + + private[OneHotEncoderModel] + class OneHotEncoderModelWriter(instance: OneHotEncoderModel) extends MLWriter { + + private case class Data(categorySizes: Array[Int]) + + override protected def saveImpl(path: String): Unit = { + DefaultParamsWriter.saveMetadata(instance, path, sc) + val data = Data(instance.categorySizes) + val dataPath = new Path(path, "data").toString + sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) + } + } + + private class OneHotEncoderModelReader extends MLReader[OneHotEncoderModel] { + + private val className = classOf[OneHotEncoderModel].getName + + override def load(path: String): OneHotEncoderModel = { + val metadata = DefaultParamsReader.loadMetadata(path, sc, className) + val dataPath = new Path(path, "data").toString + val data = sparkSession.read.parquet(dataPath) + .select("categorySizes") + .head() + val categorySizes = data.getAs[Seq[Int]](0).toArray + val model = new OneHotEncoderModel(metadata.uid, categorySizes) + metadata.getAndSetParams(model) + model + } + } + + @Since("3.0.0") + override def read: MLReader[OneHotEncoderModel] = new OneHotEncoderModelReader + + @Since("3.0.0") + override def load(path: String): OneHotEncoderModel = super.load(path) +} + +/** + * Provides some helper methods used by `OneHotEncoder`. + */ +private[feature] object OneHotEncoderCommon { + + private def genOutputAttrNames(inputCol: StructField): Option[Array[String]] = { + val inputAttr = Attribute.fromStructField(inputCol) + inputAttr match { + case nominal: NominalAttribute => + if (nominal.values.isDefined) { + nominal.values + } else if (nominal.numValues.isDefined) { + nominal.numValues.map(n => Array.tabulate(n)(_.toString)) + } else { + None + } + case binary: BinaryAttribute => + if (binary.values.isDefined) { + binary.values + } else { + Some(Array.tabulate(2)(_.toString)) + } + case _: NumericAttribute => + throw new RuntimeException( + s"The input column ${inputCol.name} cannot be continuous-value.") + case _ => + None // optimistic about unknown attributes } + } - val metadata = outputAttrGroup.toMetadata() + /** Creates an `AttributeGroup` filled by the `BinaryAttribute` named as required. */ + private def genOutputAttrGroup( + outputAttrNames: Option[Array[String]], + outputColName: String): AttributeGroup = { + outputAttrNames.map { attrNames => + val attrs: Array[Attribute] = attrNames.map { name => + BinaryAttribute.defaultAttr.withName(name) + } + new AttributeGroup(outputColName, attrs) + }.getOrElse{ + new AttributeGroup(outputColName) + } + } - // data transformation - val size = outputAttrGroup.size - val oneValue = Array(1.0) - val emptyValues = Array.empty[Double] - val emptyIndices = Array.empty[Int] - val encode = udf { label: Double => - if (label < size) { - Vectors.sparse(size, Array(label.toInt), oneValue) + /** + * Prepares the `StructField` with proper metadata for `OneHotEncoder`'s output column. + */ + def transformOutputColumnSchema( + inputCol: StructField, + outputColName: String, + dropLast: Boolean, + keepInvalid: Boolean = false): StructField = { + val outputAttrNames = genOutputAttrNames(inputCol) + val filteredOutputAttrNames = outputAttrNames.map { names => + if (dropLast && !keepInvalid) { + require(names.length > 1, + s"The input column ${inputCol.name} should have at least two distinct values.") + names.dropRight(1) + } else if (!dropLast && keepInvalid) { + names ++ Seq("invalidValues") } else { - Vectors.sparse(size, emptyIndices, emptyValues) + names } } - dataset.select(col("*"), encode(col(inputColName).cast(DoubleType)).as(outputColName, metadata)) + genOutputAttrGroup(filteredOutputAttrNames, outputColName).toStructField() } - @Since("1.4.1") - override def copy(extra: ParamMap): OneHotEncoder = defaultCopy(extra) -} + /** + * This method is called when we want to generate `AttributeGroup` from actual data for + * one-hot encoder. + */ + def getOutputAttrGroupFromData( + dataset: Dataset[_], + inputColNames: Seq[String], + outputColNames: Seq[String], + dropLast: Boolean): Seq[AttributeGroup] = { + // The RDD approach has advantage of early-stop if any values are invalid. It seems that + // DataFrame ops don't have equivalent functions. + val columns = inputColNames.map { inputColName => + col(inputColName).cast(DoubleType) + } + val numOfColumns = columns.length -@Since("1.6.0") -object OneHotEncoder extends DefaultParamsReadable[OneHotEncoder] { + val numAttrsArray = dataset.select(columns: _*).rdd.map { row => + (0 until numOfColumns).map(idx => row.getDouble(idx)).toArray + }.treeAggregate(new Array[Double](numOfColumns))( + (maxValues, curValues) => { + (0 until numOfColumns).foreach { idx => + val x = curValues(idx) + assert(x <= Int.MaxValue, + s"OneHotEncoder only supports up to ${Int.MaxValue} indices, but got $x.") + assert(x >= 0.0 && x == x.toInt, + s"Values from column ${inputColNames(idx)} must be indices, but got $x.") + maxValues(idx) = math.max(maxValues(idx), x) + } + maxValues + }, + (m0, m1) => { + (0 until numOfColumns).foreach { idx => + m0(idx) = math.max(m0(idx), m1(idx)) + } + m0 + } + ).map(_.toInt + 1) - @Since("1.6.0") - override def load(path: String): OneHotEncoder = super.load(path) + outputColNames.zip(numAttrsArray).map { case (outputColName, numAttrs) => + createAttrGroupForAttrNames(outputColName, numAttrs, dropLast, keepInvalid = false) + } + } + + /** Creates an `AttributeGroup` with the required number of `BinaryAttribute`. */ + def createAttrGroupForAttrNames( + outputColName: String, + numAttrs: Int, + dropLast: Boolean, + keepInvalid: Boolean): AttributeGroup = { + val outputAttrNames = Array.tabulate(numAttrs)(_.toString) + val filtered = if (dropLast && !keepInvalid) { + outputAttrNames.dropRight(1) + } else if (!dropLast && keepInvalid) { + outputAttrNames ++ Seq("invalidValues") + } else { + outputAttrNames + } + genOutputAttrGroup(Some(filtered), outputColName) + } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala deleted file mode 100644 index 4a44f3186538d..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/OneHotEncoderEstimator.scala +++ /dev/null @@ -1,528 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.hadoop.fs.Path - -import org.apache.spark.SparkException -import org.apache.spark.annotation.Since -import org.apache.spark.ml.{Estimator, Model} -import org.apache.spark.ml.attribute._ -import org.apache.spark.ml.linalg.Vectors -import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCols, HasOutputCols} -import org.apache.spark.ml.util._ -import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.expressions.UserDefinedFunction -import org.apache.spark.sql.functions.{col, lit, udf} -import org.apache.spark.sql.types.{DoubleType, StructField, StructType} - -/** Private trait for params and common methods for OneHotEncoderEstimator and OneHotEncoderModel */ -private[ml] trait OneHotEncoderBase extends Params with HasHandleInvalid - with HasInputCols with HasOutputCols { - - /** - * Param for how to handle invalid data during transform(). - * Options are 'keep' (invalid data presented as an extra categorical feature) or - * 'error' (throw an error). - * Note that this Param is only used during transform; during fitting, invalid data - * will result in an error. - * Default: "error" - * @group param - */ - @Since("2.3.0") - override val handleInvalid: Param[String] = new Param[String](this, "handleInvalid", - "How to handle invalid data during transform(). " + - "Options are 'keep' (invalid data presented as an extra categorical feature) " + - "or error (throw an error). Note that this Param is only used during transform; " + - "during fitting, invalid data will result in an error.", - ParamValidators.inArray(OneHotEncoderEstimator.supportedHandleInvalids)) - - setDefault(handleInvalid, OneHotEncoderEstimator.ERROR_INVALID) - - /** - * Whether to drop the last category in the encoded vector (default: true) - * @group param - */ - @Since("2.3.0") - final val dropLast: BooleanParam = - new BooleanParam(this, "dropLast", "whether to drop the last category") - setDefault(dropLast -> true) - - /** @group getParam */ - @Since("2.3.0") - def getDropLast: Boolean = $(dropLast) - - protected def validateAndTransformSchema( - schema: StructType, - dropLast: Boolean, - keepInvalid: Boolean): StructType = { - val inputColNames = $(inputCols) - val outputColNames = $(outputCols) - - require(inputColNames.length == outputColNames.length, - s"The number of input columns ${inputColNames.length} must be the same as the number of " + - s"output columns ${outputColNames.length}.") - - // Input columns must be NumericType. - inputColNames.foreach(SchemaUtils.checkNumericType(schema, _)) - - // Prepares output columns with proper attributes by examining input columns. - val inputFields = $(inputCols).map(schema(_)) - - val outputFields = inputFields.zip(outputColNames).map { case (inputField, outputColName) => - OneHotEncoderCommon.transformOutputColumnSchema( - inputField, outputColName, dropLast, keepInvalid) - } - outputFields.foldLeft(schema) { case (newSchema, outputField) => - SchemaUtils.appendColumn(newSchema, outputField) - } - } -} - -/** - * A one-hot encoder that maps a column of category indices to a column of binary vectors, with - * at most a single one-value per row that indicates the input category index. - * For example with 5 categories, an input value of 2.0 would map to an output vector of - * `[0.0, 0.0, 1.0, 0.0]`. - * The last category is not included by default (configurable via `dropLast`), - * because it makes the vector entries sum up to one, and hence linearly dependent. - * So an input value of 4.0 maps to `[0.0, 0.0, 0.0, 0.0]`. - * - * @note This is different from scikit-learn's OneHotEncoder, which keeps all categories. - * The output vectors are sparse. - * - * When `handleInvalid` is configured to 'keep', an extra "category" indicating invalid values is - * added as last category. So when `dropLast` is true, invalid values are encoded as all-zeros - * vector. - * - * @note When encoding multi-column by using `inputCols` and `outputCols` params, input/output cols - * come in pairs, specified by the order in the arrays, and each pair is treated independently. - * - * @see `StringIndexer` for converting categorical values into category indices - */ -@Since("2.3.0") -class OneHotEncoderEstimator @Since("2.3.0") (@Since("2.3.0") override val uid: String) - extends Estimator[OneHotEncoderModel] with OneHotEncoderBase with DefaultParamsWritable { - - @Since("2.3.0") - def this() = this(Identifiable.randomUID("oneHotEncoder")) - - /** @group setParam */ - @Since("2.3.0") - def setInputCols(values: Array[String]): this.type = set(inputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setOutputCols(values: Array[String]): this.type = set(outputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setDropLast(value: Boolean): this.type = set(dropLast, value) - - /** @group setParam */ - @Since("2.3.0") - def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - - @Since("2.3.0") - override def transformSchema(schema: StructType): StructType = { - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - validateAndTransformSchema(schema, dropLast = $(dropLast), - keepInvalid = keepInvalid) - } - - @Since("2.3.0") - override def fit(dataset: Dataset[_]): OneHotEncoderModel = { - transformSchema(dataset.schema) - - // Compute the plain number of categories without `handleInvalid` and - // `dropLast` taken into account. - val transformedSchema = validateAndTransformSchema(dataset.schema, dropLast = false, - keepInvalid = false) - val categorySizes = new Array[Int]($(outputCols).length) - - val columnToScanIndices = $(outputCols).zipWithIndex.flatMap { case (outputColName, idx) => - val numOfAttrs = AttributeGroup.fromStructField( - transformedSchema(outputColName)).size - if (numOfAttrs < 0) { - Some(idx) - } else { - categorySizes(idx) = numOfAttrs - None - } - } - - // Some input columns don't have attributes or their attributes don't have necessary info. - // We need to scan the data to get the number of values for each column. - if (columnToScanIndices.length > 0) { - val inputColNames = columnToScanIndices.map($(inputCols)(_)) - val outputColNames = columnToScanIndices.map($(outputCols)(_)) - - // When fitting data, we want the plain number of categories without `handleInvalid` and - // `dropLast` taken into account. - val attrGroups = OneHotEncoderCommon.getOutputAttrGroupFromData( - dataset, inputColNames, outputColNames, dropLast = false) - attrGroups.zip(columnToScanIndices).foreach { case (attrGroup, idx) => - categorySizes(idx) = attrGroup.size - } - } - - val model = new OneHotEncoderModel(uid, categorySizes).setParent(this) - copyValues(model) - } - - @Since("2.3.0") - override def copy(extra: ParamMap): OneHotEncoderEstimator = defaultCopy(extra) -} - -@Since("2.3.0") -object OneHotEncoderEstimator extends DefaultParamsReadable[OneHotEncoderEstimator] { - - private[feature] val KEEP_INVALID: String = "keep" - private[feature] val ERROR_INVALID: String = "error" - private[feature] val supportedHandleInvalids: Array[String] = Array(KEEP_INVALID, ERROR_INVALID) - - @Since("2.3.0") - override def load(path: String): OneHotEncoderEstimator = super.load(path) -} - -/** - * @param categorySizes Original number of categories for each feature being encoded. - * The array contains one value for each input column, in order. - */ -@Since("2.3.0") -class OneHotEncoderModel private[ml] ( - @Since("2.3.0") override val uid: String, - @Since("2.3.0") val categorySizes: Array[Int]) - extends Model[OneHotEncoderModel] with OneHotEncoderBase with MLWritable { - - import OneHotEncoderModel._ - - // Returns the category size for each index with `dropLast` and `handleInvalid` - // taken into account. - private def getConfigedCategorySizes: Array[Int] = { - val dropLast = getDropLast - val keepInvalid = getHandleInvalid == OneHotEncoderEstimator.KEEP_INVALID - - if (!dropLast && keepInvalid) { - // When `handleInvalid` is "keep", an extra category is added as last category - // for invalid data. - categorySizes.map(_ + 1) - } else if (dropLast && !keepInvalid) { - // When `dropLast` is true, the last category is removed. - categorySizes.map(_ - 1) - } else { - // When `dropLast` is true and `handleInvalid` is "keep", the extra category for invalid - // data is removed. Thus, it is the same as the plain number of categories. - categorySizes - } - } - - private def encoder: UserDefinedFunction = { - val keepInvalid = getHandleInvalid == OneHotEncoderEstimator.KEEP_INVALID - val configedSizes = getConfigedCategorySizes - val localCategorySizes = categorySizes - - // The udf performed on input data. The first parameter is the input value. The second - // parameter is the index in inputCols of the column being encoded. - udf { (label: Double, colIdx: Int) => - val origCategorySize = localCategorySizes(colIdx) - // idx: index in vector of the single 1-valued element - val idx = if (label >= 0 && label < origCategorySize) { - label - } else { - if (keepInvalid) { - origCategorySize - } else { - if (label < 0) { - throw new SparkException(s"Negative value: $label. Input can't be negative. " + - s"To handle invalid values, set Param handleInvalid to " + - s"${OneHotEncoderEstimator.KEEP_INVALID}") - } else { - throw new SparkException(s"Unseen value: $label. To handle unseen values, " + - s"set Param handleInvalid to ${OneHotEncoderEstimator.KEEP_INVALID}.") - } - } - } - - val size = configedSizes(colIdx) - if (idx < size) { - Vectors.sparse(size, Array(idx.toInt), Array(1.0)) - } else { - Vectors.sparse(size, Array.empty[Int], Array.empty[Double]) - } - } - } - - /** @group setParam */ - @Since("2.3.0") - def setInputCols(values: Array[String]): this.type = set(inputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setOutputCols(values: Array[String]): this.type = set(outputCols, values) - - /** @group setParam */ - @Since("2.3.0") - def setDropLast(value: Boolean): this.type = set(dropLast, value) - - /** @group setParam */ - @Since("2.3.0") - def setHandleInvalid(value: String): this.type = set(handleInvalid, value) - - @Since("2.3.0") - override def transformSchema(schema: StructType): StructType = { - val inputColNames = $(inputCols) - - require(inputColNames.length == categorySizes.length, - s"The number of input columns ${inputColNames.length} must be the same as the number of " + - s"features ${categorySizes.length} during fitting.") - - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - val transformedSchema = validateAndTransformSchema(schema, dropLast = $(dropLast), - keepInvalid = keepInvalid) - verifyNumOfValues(transformedSchema) - } - - /** - * If the metadata of input columns also specifies the number of categories, we need to - * compare with expected category number with `handleInvalid` and `dropLast` taken into - * account. Mismatched numbers will cause exception. - */ - private def verifyNumOfValues(schema: StructType): StructType = { - val configedSizes = getConfigedCategorySizes - $(outputCols).zipWithIndex.foreach { case (outputColName, idx) => - val inputColName = $(inputCols)(idx) - val attrGroup = AttributeGroup.fromStructField(schema(outputColName)) - - // If the input metadata specifies number of category for output column, - // comparing with expected category number with `handleInvalid` and - // `dropLast` taken into account. - if (attrGroup.attributes.nonEmpty) { - val numCategories = configedSizes(idx) - require(attrGroup.size == numCategories, "OneHotEncoderModel expected " + - s"$numCategories categorical values for input column $inputColName, " + - s"but the input column had metadata specifying ${attrGroup.size} values.") - } - } - schema - } - - @Since("2.3.0") - override def transform(dataset: Dataset[_]): DataFrame = { - val transformedSchema = transformSchema(dataset.schema, logging = true) - val keepInvalid = $(handleInvalid) == OneHotEncoderEstimator.KEEP_INVALID - - val encodedColumns = $(inputCols).indices.map { idx => - val inputColName = $(inputCols)(idx) - val outputColName = $(outputCols)(idx) - - val outputAttrGroupFromSchema = - AttributeGroup.fromStructField(transformedSchema(outputColName)) - - val metadata = if (outputAttrGroupFromSchema.size < 0) { - OneHotEncoderCommon.createAttrGroupForAttrNames(outputColName, - categorySizes(idx), $(dropLast), keepInvalid).toMetadata() - } else { - outputAttrGroupFromSchema.toMetadata() - } - - encoder(col(inputColName).cast(DoubleType), lit(idx)) - .as(outputColName, metadata) - } - dataset.withColumns($(outputCols), encodedColumns) - } - - @Since("2.3.0") - override def copy(extra: ParamMap): OneHotEncoderModel = { - val copied = new OneHotEncoderModel(uid, categorySizes) - copyValues(copied, extra).setParent(parent) - } - - @Since("2.3.0") - override def write: MLWriter = new OneHotEncoderModelWriter(this) -} - -@Since("2.3.0") -object OneHotEncoderModel extends MLReadable[OneHotEncoderModel] { - - private[OneHotEncoderModel] - class OneHotEncoderModelWriter(instance: OneHotEncoderModel) extends MLWriter { - - private case class Data(categorySizes: Array[Int]) - - override protected def saveImpl(path: String): Unit = { - DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.categorySizes) - val dataPath = new Path(path, "data").toString - sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) - } - } - - private class OneHotEncoderModelReader extends MLReader[OneHotEncoderModel] { - - private val className = classOf[OneHotEncoderModel].getName - - override def load(path: String): OneHotEncoderModel = { - val metadata = DefaultParamsReader.loadMetadata(path, sc, className) - val dataPath = new Path(path, "data").toString - val data = sparkSession.read.parquet(dataPath) - .select("categorySizes") - .head() - val categorySizes = data.getAs[Seq[Int]](0).toArray - val model = new OneHotEncoderModel(metadata.uid, categorySizes) - metadata.getAndSetParams(model) - model - } - } - - @Since("2.3.0") - override def read: MLReader[OneHotEncoderModel] = new OneHotEncoderModelReader - - @Since("2.3.0") - override def load(path: String): OneHotEncoderModel = super.load(path) -} - -/** - * Provides some helper methods used by both `OneHotEncoder` and `OneHotEncoderEstimator`. - */ -private[feature] object OneHotEncoderCommon { - - private def genOutputAttrNames(inputCol: StructField): Option[Array[String]] = { - val inputAttr = Attribute.fromStructField(inputCol) - inputAttr match { - case nominal: NominalAttribute => - if (nominal.values.isDefined) { - nominal.values - } else if (nominal.numValues.isDefined) { - nominal.numValues.map(n => Array.tabulate(n)(_.toString)) - } else { - None - } - case binary: BinaryAttribute => - if (binary.values.isDefined) { - binary.values - } else { - Some(Array.tabulate(2)(_.toString)) - } - case _: NumericAttribute => - throw new RuntimeException( - s"The input column ${inputCol.name} cannot be continuous-value.") - case _ => - None // optimistic about unknown attributes - } - } - - /** Creates an `AttributeGroup` filled by the `BinaryAttribute` named as required. */ - private def genOutputAttrGroup( - outputAttrNames: Option[Array[String]], - outputColName: String): AttributeGroup = { - outputAttrNames.map { attrNames => - val attrs: Array[Attribute] = attrNames.map { name => - BinaryAttribute.defaultAttr.withName(name) - } - new AttributeGroup(outputColName, attrs) - }.getOrElse{ - new AttributeGroup(outputColName) - } - } - - /** - * Prepares the `StructField` with proper metadata for `OneHotEncoder`'s output column. - */ - def transformOutputColumnSchema( - inputCol: StructField, - outputColName: String, - dropLast: Boolean, - keepInvalid: Boolean = false): StructField = { - val outputAttrNames = genOutputAttrNames(inputCol) - val filteredOutputAttrNames = outputAttrNames.map { names => - if (dropLast && !keepInvalid) { - require(names.length > 1, - s"The input column ${inputCol.name} should have at least two distinct values.") - names.dropRight(1) - } else if (!dropLast && keepInvalid) { - names ++ Seq("invalidValues") - } else { - names - } - } - - genOutputAttrGroup(filteredOutputAttrNames, outputColName).toStructField() - } - - /** - * This method is called when we want to generate `AttributeGroup` from actual data for - * one-hot encoder. - */ - def getOutputAttrGroupFromData( - dataset: Dataset[_], - inputColNames: Seq[String], - outputColNames: Seq[String], - dropLast: Boolean): Seq[AttributeGroup] = { - // The RDD approach has advantage of early-stop if any values are invalid. It seems that - // DataFrame ops don't have equivalent functions. - val columns = inputColNames.map { inputColName => - col(inputColName).cast(DoubleType) - } - val numOfColumns = columns.length - - val numAttrsArray = dataset.select(columns: _*).rdd.map { row => - (0 until numOfColumns).map(idx => row.getDouble(idx)).toArray - }.treeAggregate(new Array[Double](numOfColumns))( - (maxValues, curValues) => { - (0 until numOfColumns).foreach { idx => - val x = curValues(idx) - assert(x <= Int.MaxValue, - s"OneHotEncoder only supports up to ${Int.MaxValue} indices, but got $x.") - assert(x >= 0.0 && x == x.toInt, - s"Values from column ${inputColNames(idx)} must be indices, but got $x.") - maxValues(idx) = math.max(maxValues(idx), x) - } - maxValues - }, - (m0, m1) => { - (0 until numOfColumns).foreach { idx => - m0(idx) = math.max(m0(idx), m1(idx)) - } - m0 - } - ).map(_.toInt + 1) - - outputColNames.zip(numAttrsArray).map { case (outputColName, numAttrs) => - createAttrGroupForAttrNames(outputColName, numAttrs, dropLast, keepInvalid = false) - } - } - - /** Creates an `AttributeGroup` with the required number of `BinaryAttribute`. */ - def createAttrGroupForAttrNames( - outputColName: String, - numAttrs: Int, - dropLast: Boolean, - keepInvalid: Boolean): AttributeGroup = { - val outputAttrNames = Array.tabulate(numAttrs)(_.toString) - val filtered = if (dropLast && !keepInvalid) { - outputAttrNames.dropRight(1) - } else if (!dropLast && keepInvalid) { - outputAttrNames ++ Seq("invalidValues") - } else { - outputAttrNames - } - genOutputAttrGroup(Some(filtered), outputColName) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala index 8172491a517d1..aa5a171d4fec5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/PCA.scala @@ -26,11 +26,7 @@ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ import org.apache.spark.mllib.feature -import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, DenseVector => OldDenseVector, - Matrices => OldMatrices, Vector => OldVector, Vectors => OldVectors} -import org.apache.spark.mllib.linalg.MatrixImplicits._ -import org.apache.spark.mllib.linalg.VectorImplicits._ -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{DenseMatrix => OldDenseMatrix, Vectors => OldVectors} import org.apache.spark.sql._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{StructField, StructType} @@ -92,12 +88,13 @@ class PCA @Since("1.5.0") ( @Since("2.0.0") override def fit(dataset: Dataset[_]): PCAModel = { transformSchema(dataset.schema, logging = true) - val input: RDD[OldVector] = dataset.select($(inputCol)).rdd.map { + val input = dataset.select($(inputCol)).rdd.map { case Row(v: Vector) => OldVectors.fromML(v) } val pca = new feature.PCA(k = $(k)) val pcaModel = pca.fit(input) - copyValues(new PCAModel(uid, pcaModel.pc, pcaModel.explainedVariance).setParent(this)) + copyValues(new PCAModel(uid, pcaModel.pc.asML, pcaModel.explainedVariance.asML) + .setParent(this)) } @Since("1.5.0") @@ -149,15 +146,24 @@ class PCAModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val pcaModel = new feature.PCAModel($(k), - OldMatrices.fromML(pc).asInstanceOf[OldDenseMatrix], - OldVectors.fromML(explainedVariance).asInstanceOf[OldDenseVector]) - // TODO: Make the transformer natively in ml framework to avoid extra conversion. - val transformer: Vector => Vector = v => pcaModel.transform(OldVectors.fromML(v)).asML + val func = { vector: Vector => + vector match { + case dv: DenseVector => + pc.transpose.multiply(dv) + case SparseVector(size, indices, values) => + /* SparseVector -> single row SparseMatrix */ + val sm = Matrices.sparse(size, 1, Array(0, indices.length), indices, values).transpose + val projection = sm.multiply(pc) + Vectors.dense(projection.values) + case _ => + throw new IllegalArgumentException("Unsupported vector format. Expected " + + s"SparseVector or DenseVector. Instead got: ${vector.getClass}") + } + } - val pcaOp = udf(transformer) - dataset.withColumn($(outputCol), pcaOp(col($(inputCol)))) + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(inputCol)))) } @Since("1.5.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala index 56e2c543d100a..5bfaa3b7f3f52 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/QuantileDiscretizer.scala @@ -17,10 +17,6 @@ package org.apache.spark.ml.feature -import org.json4s.JsonDSL._ -import org.json4s.JValue -import org.json4s.jackson.JsonMethods._ - import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.ml._ @@ -209,7 +205,7 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui if (isSet(inputCols)) { val splitsArray = if (isSet(numBucketsArray)) { val probArrayPerCol = $(numBucketsArray).map { numOfBuckets => - (0.0 to 1.0 by 1.0 / numOfBuckets).toArray + (0 to numOfBuckets).map(_.toDouble / numOfBuckets).toArray } val probabilityArray = probArrayPerCol.flatten.sorted.distinct @@ -229,12 +225,12 @@ final class QuantileDiscretizer @Since("1.6.0") (@Since("1.6.0") override val ui } } else { dataset.stat.approxQuantile($(inputCols), - (0.0 to 1.0 by 1.0 / $(numBuckets)).toArray, $(relativeError)) + (0 to $(numBuckets)).map(_.toDouble / $(numBuckets)).toArray, $(relativeError)) } bucketizer.setSplitsArray(splitsArray.map(getDistinctSplits)) } else { val splits = dataset.stat.approxQuantile($(inputCol), - (0.0 to 1.0 by 1.0 / $(numBuckets)).toArray, $(relativeError)) + (0 to $(numBuckets)).map(_.toDouble / $(numBuckets)).toArray, $(relativeError)) bucketizer.setSplits(getDistinctSplits(splits)) } copyValues(bucketizer.setParent(this)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala index 346e1823f00b8..ec8f7031ad3bd 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormula.scala @@ -126,8 +126,9 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with /** * :: Experimental :: * Implements the transforms required for fitting a dataset against an R model formula. Currently - * we support a limited subset of the R operators, including '~', '.', ':', '+', and '-'. Also see - * the R formula docs here: http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html + * we support a limited subset of the R operators, including '~', '.', ':', '+', '-', '*' and '^'. + * Also see the R formula docs here: + * http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html * * The basic operators are: * - `~` separate target and terms @@ -135,6 +136,8 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with * - `-` remove a term, "- 1" means removing intercept * - `:` interaction (multiplication for numeric values, or binarized categorical values) * - `.` all columns except target + * - `*` factor crossing, includes the terms and interactions between them + * - `^` factor crossing to a specified degree * * Suppose `a` and `b` are double columns, we use the following simple examples * to illustrate the effect of `RFormula`: @@ -142,6 +145,10 @@ private[feature] trait RFormulaBase extends HasFeaturesCol with HasLabelCol with * are coefficients. * - `y ~ a + b + a:b - 1` means model `y ~ w1 * a + w2 * b + w3 * a * b` where `w1, w2, w3` * are coefficients. + * - `y ~ a * b` means model `y ~ w0 + w1 * a + w2 * b + w3 * a * b` where `w0` is the + * intercept and `w1, w2, w3` are coefficients + * - `y ~ (a + b)^2` means model `y ~ w0 + w1 * a + w2 * b + w3 * a * b` where `w0` is the + * intercept and `w1, w2, w3` are coefficients * * RFormula produces a vector column of features and a double or string column of label. * Like when formulas are used in R for linear regression, string input columns will be one-hot @@ -246,7 +253,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) // Formula w/o intercept, one of the categories in the first category feature is // being used as reference category, we will not drop any category for that feature. if (!hasIntercept && !keepReferenceCategory) { - encoderStages += new OneHotEncoderEstimator(uid) + encoderStages += new OneHotEncoder(uid) .setInputCols(Array(indexed(term))) .setOutputCols(Array(encodedCol)) .setDropLast(false) @@ -269,7 +276,7 @@ class RFormula @Since("1.5.0") (@Since("1.5.0") override val uid: String) if (oneHotEncodeColumns.nonEmpty) { val (inputCols, outputCols) = oneHotEncodeColumns.toArray.unzip - encoderStages += new OneHotEncoderEstimator(uid) + encoderStages += new OneHotEncoder(uid) .setInputCols(inputCols) .setOutputCols(outputCols) .setDropLast(true) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala index 32835fb3aa6d1..dbbfd8f329431 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/RFormulaParser.scala @@ -34,11 +34,18 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { def resolve(schema: StructType): ResolvedRFormula = { val dotTerms = expandDot(schema) var includedTerms = Seq[Seq[String]]() + val seen = mutable.Set[Set[String]]() terms.foreach { case col: ColumnRef => includedTerms :+= Seq(col.value) case ColumnInteraction(cols) => - includedTerms ++= expandInteraction(schema, cols) + expandInteraction(schema, cols) foreach { t => + // add equivalent interaction terms only once + if (!seen.contains(t.toSet)) { + includedTerms :+= t + seen += t.toSet + } + } case Dot => includedTerms ++= dotTerms.map(Seq(_)) case Deletion(term: Term) => @@ -57,8 +64,12 @@ private[ml] case class ParsedRFormula(label: ColumnRef, terms: Seq[Term]) { case _: Deletion => throw new RuntimeException("Deletion terms cannot be nested") case _: Intercept => + case _: Terms => + case EmptyTerm => } case _: Intercept => + case _: Terms => + case EmptyTerm => } ResolvedRFormula(label.value, includedTerms.distinct, hasIntercept) } @@ -144,10 +155,50 @@ private[ml] case class ResolvedRFormula( * R formula terms. See the R formula docs here for more information: * http://stat.ethz.ch/R-manual/R-patched/library/stats/html/formula.html */ -private[ml] sealed trait Term +private[ml] sealed trait Term { + + /** Default representation of a single Term as a part of summed terms. */ + def asTerms: Terms = Terms(Seq(this)) + + /** Creates a summation term by concatenation of terms. */ + def add(other: Term): Term = Terms(this.asTerms.terms ++ other.asTerms.terms) + + /** + * Fold by adding deletion terms to the left. Double negation + * doesn't cancel deletion in order not to add extra terms, e.g. + * a - (b - c) = a - Deletion(b) - Deletion(c) = a + */ + def subtract(other: Term): Term = { + other.asTerms.terms.foldLeft(this) { + case (left, right) => + right match { + case t: Deletion => left.add(t) + case t: Term => left.add(Deletion(t)) + } + } + } + + /** Default interactions of a Term */ + def interact(other: Term): Term = EmptyTerm +} + +/** Placeholder term for the result of undefined interactions, e.g. '1:1' or 'a:1' */ +private[ml] case object EmptyTerm extends Term /** A term that may be part of an interaction, e.g. 'x' in 'x:y' */ -private[ml] sealed trait InteractableTerm extends Term +private[ml] sealed trait InteractableTerm extends Term { + + /** Convert to ColumnInteraction to wrap all interactions. */ + def asInteraction: ColumnInteraction = ColumnInteraction(Seq(this)) + + /** Interactions of interactable terms. */ + override def interact(other: Term): Term = other match { + case t: InteractableTerm => this.asInteraction.interact(t.asInteraction) + case t: ColumnInteraction => this.asInteraction.interact(t) + case t: Terms => this.asTerms.interact(t) + case t: Term => t.interact(this) // Deletion or non-interactable term + } +} /* R formula reference to all available columns, e.g. "." in a formula */ private[ml] case object Dot extends InteractableTerm @@ -156,19 +207,68 @@ private[ml] case object Dot extends InteractableTerm private[ml] case class ColumnRef(value: String) extends InteractableTerm /* R formula interaction of several columns, e.g. "Sepal_Length:Species" in a formula */ -private[ml] case class ColumnInteraction(terms: Seq[InteractableTerm]) extends Term +private[ml] case class ColumnInteraction(terms: Seq[InteractableTerm]) extends Term { + + // Convert to ColumnInteraction and concat terms + override def interact(other: Term): Term = other match { + case t: InteractableTerm => this.interact(t.asInteraction) + case t: ColumnInteraction => ColumnInteraction(terms ++ t.terms) + case t: Terms => this.asTerms.interact(t) + case t: Term => t.interact(this) + } +} /* R formula intercept toggle, e.g. "+ 0" in a formula */ private[ml] case class Intercept(enabled: Boolean) extends Term /* R formula deletion of a variable, e.g. "- Species" in a formula */ -private[ml] case class Deletion(term: Term) extends Term +private[ml] case class Deletion(term: Term) extends Term { + + // Unnest the deletion and interact + override def interact(other: Term): Term = other match { + case Deletion(t) => Deletion(term.interact(t)) + case t: Term => Deletion(term.interact(t)) + } +} + +/* Wrapper for multiple terms in a formula. */ +private[ml] case class Terms(terms: Seq[Term]) extends Term { + + override def asTerms: Terms = this + + override def interact(other: Term): Term = { + val interactions = for { + left <- terms + right <- other.asTerms.terms + } yield left.interact(right) + Terms(interactions) + } +} /** - * Limited implementation of R formula parsing. Currently supports: '~', '+', '-', '.', ':'. + * Limited implementation of R formula parsing. Currently supports: '~', '+', '-', '.', ':', + * '*', '^'. */ private[ml] object RFormulaParser extends RegexParsers { - private val intercept: Parser[Intercept] = + + private def add(left: Term, right: Term) = left.add(right) + + private def subtract(left: Term, right: Term) = left.subtract(right) + + private def interact(left: Term, right: Term) = left.interact(right) + + private def cross(left: Term, right: Term) = left.add(right).add(left.interact(right)) + + private def power(base: Term, degree: Int): Term = { + val exprs = List.fill(degree)(base) + exprs match { + case Nil => EmptyTerm + case x :: Nil => x + case x :: xs => xs.foldLeft(x)(cross _) + } + } + + private val intercept: Parser[Term] = "([01])".r ^^ { case a => Intercept(a == "1") } private val columnRef: Parser[ColumnRef] = @@ -178,22 +278,27 @@ private[ml] object RFormulaParser extends RegexParsers { private val label: Parser[ColumnRef] = columnRef | empty - private val dot: Parser[InteractableTerm] = "\\.".r ^^ { case _ => Dot } + private val dot: Parser[Term] = "\\.".r ^^ { case _ => Dot } - private val interaction: Parser[List[InteractableTerm]] = rep1sep(columnRef | dot, ":") + private val parens: Parser[Term] = "(" ~> expr <~ ")" - private val term: Parser[Term] = intercept | - interaction ^^ { case terms => ColumnInteraction(terms) } | dot | columnRef + private val term: Parser[Term] = parens | intercept | columnRef | dot - private val terms: Parser[List[Term]] = (term ~ rep("+" ~ term | "-" ~ term)) ^^ { - case op ~ list => list.foldLeft(List(op)) { - case (left, "+" ~ right) => left ++ Seq(right) - case (left, "-" ~ right) => left ++ Seq(Deletion(right)) - } - } + private val pow: Parser[Term] = term ~ "^" ~ "^[1-9]\\d*".r ^^ { + case base ~ "^" ~ degree => power(base, degree.toInt) + } | term + + private val interaction: Parser[Term] = pow * (":" ^^^ { interact _ }) + + private val factor = interaction * ("*" ^^^ { cross _ }) + + private val sum = factor * ("+" ^^^ { add _ } | + "-" ^^^ { subtract _ }) + + private val expr = (sum | term) private val formula: Parser[ParsedRFormula] = - (label ~ "~" ~ terms) ^^ { case r ~ "~" ~ t => ParsedRFormula(r, t) } + (label ~ "~" ~ expr) ^^ { case r ~ "~" ~ t => ParsedRFormula(r, t.asTerms.terms) } def parse(value: String): ParsedRFormula = parseAll(formula, value) match { case Success(result, _) => result diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala index 91b0707dec3f3..81cf2e1a4ff79 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StandardScaler.scala @@ -21,7 +21,7 @@ import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since import org.apache.spark.ml._ -import org.apache.spark.ml.linalg.{Vector, VectorUDT} +import org.apache.spark.ml.linalg._ import org.apache.spark.ml.param._ import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ @@ -162,11 +162,34 @@ class StandardScalerModel private[ml] ( transformSchema(dataset.schema, logging = true) val scaler = new feature.StandardScalerModel(std, mean, $(withStd), $(withMean)) - // TODO: Make the transformer natively in ml framework to avoid extra conversion. - val transformer: Vector => Vector = v => scaler.transform(OldVectors.fromML(v)).asML + val func = if ($(withMean)) { + vector: Vector => + val values = vector match { + // specially handle DenseVector because its toArray does not clone already + case d: DenseVector => d.values.clone() + case v: Vector => v.toArray + } + val newValues = scaler.transformWithMean(values) + Vectors.dense(newValues) + } else if ($(withStd)) { + vector: Vector => + vector match { + case DenseVector(values) => + val newValues = scaler.transformDenseWithStd(values) + Vectors.dense(newValues) + case SparseVector(size, indices, values) => + val (newIndices, newValues) = scaler.transformSparseWithStd(indices, values) + Vectors.sparse(size, newIndices, newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") + } + } else { + vector: Vector => vector + } - val scale = udf(transformer) - dataset.withColumn($(outputCol), scale(col($(inputCol)))) + val transformer = udf(func) + dataset.withColumn($(outputCol), transformer(col($(inputCol)))) } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala index a833d8b270cf1..94f40c3529676 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/StringIndexer.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.feature -import scala.language.existentials - import org.apache.hadoop.fs.Path import org.apache.spark.SparkException @@ -26,18 +24,22 @@ import org.apache.spark.annotation.Since import org.apache.spark.ml.{Estimator, Model, Transformer} import org.apache.spark.ml.attribute.{Attribute, NominalAttribute} import org.apache.spark.ml.param._ -import org.apache.spark.ml.param.shared.{HasHandleInvalid, HasInputCol, HasOutputCol} +import org.apache.spark.ml.param.shared._ import org.apache.spark.ml.util._ -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Encoder, Encoders, Row} +import org.apache.spark.sql.catalyst.expressions.{If, Literal} +import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ +import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.VersionUtils.majorMinorVersion import org.apache.spark.util.collection.OpenHashMap /** * Base trait for [[StringIndexer]] and [[StringIndexerModel]]. */ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid with HasInputCol - with HasOutputCol { + with HasOutputCol with HasInputCols with HasOutputCols { /** * Param for how to handle invalid data (unseen labels or NULL values). @@ -66,6 +68,9 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi * - 'alphabetAsc': ascending alphabetical order * Default is 'frequencyDesc'. * + * Note: In case of equal frequency when under frequencyDesc/Asc, the strings are further sorted + * alphabetically. + * * @group param */ @Since("2.3.0") @@ -79,26 +84,56 @@ private[feature] trait StringIndexerBase extends Params with HasHandleInvalid wi @Since("2.3.0") def getStringOrderType: String = $(stringOrderType) - /** Validates and transforms the input schema. */ - protected def validateAndTransformSchema(schema: StructType): StructType = { - val inputColName = $(inputCol) + /** Returns the input and output column names corresponding in pair. */ + private[feature] def getInOutCols(): (Array[String], Array[String]) = { + ParamValidators.checkSingleVsMultiColumnParams(this, Seq(outputCol), Seq(outputCols)) + + if (isSet(inputCol)) { + (Array($(inputCol)), Array($(outputCol))) + } else { + require($(inputCols).length == $(outputCols).length, + "The number of input columns does not match output columns") + ($(inputCols), $(outputCols)) + } + } + + private def validateAndTransformField( + schema: StructType, + inputColName: String, + outputColName: String): StructField = { val inputDataType = schema(inputColName).dataType require(inputDataType == StringType || inputDataType.isInstanceOf[NumericType], s"The input column $inputColName must be either string type or numeric type, " + s"but got $inputDataType.") - val inputFields = schema.fields - val outputColName = $(outputCol) - require(inputFields.forall(_.name != outputColName), + require(schema.fields.forall(_.name != outputColName), s"Output column $outputColName already exists.") - val attr = NominalAttribute.defaultAttr.withName($(outputCol)) - val outputFields = inputFields :+ attr.toStructField() - StructType(outputFields) + NominalAttribute.defaultAttr.withName($(outputCol)).toStructField() + } + + /** Validates and transforms the input schema. */ + protected def validateAndTransformSchema( + schema: StructType, + skipNonExistsCol: Boolean = false): StructType = { + val (inputColNames, outputColNames) = getInOutCols() + + require(outputColNames.distinct.length == outputColNames.length, + s"Output columns should not be duplicate.") + + val outputFields = inputColNames.zip(outputColNames).flatMap { + case (inputColName, outputColName) => + schema.fieldNames.contains(inputColName) match { + case true => Some(validateAndTransformField(schema, inputColName, outputColName)) + case false if skipNonExistsCol => None + case _ => throw new SparkException(s"Input column $inputColName does not exist.") + } + } + StructType(schema.fields ++ outputFields) } } /** - * A label indexer that maps a string column of labels to an ML column of label indices. - * If the input column is numeric, we cast it to string and index the string values. + * A label indexer that maps string column(s) of labels to ML column(s) of label indices. + * If the input columns are numeric, we cast them to string and index the string values. * The indices are in [0, numLabels). By default, this is ordered by label frequencies * so the most frequent label gets index 0. The ordering behavior is controlled by * setting `stringOrderType`. @@ -130,21 +165,86 @@ class StringIndexer @Since("1.4.0") ( @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) + + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + + private def countByValue( + dataset: Dataset[_], + inputCols: Array[String]): Array[OpenHashMap[String, Long]] = { + + val aggregator = new StringIndexerAggregator(inputCols.length) + implicit val encoder = Encoders.kryo[Array[OpenHashMap[String, Long]]] + + val selectedCols = inputCols.map { colName => + val col = dataset.col(colName) + if (col.expr.dataType == StringType) { + col + } else { + // We don't count for NaN values. Because `StringIndexerAggregator` only processes strings, + // we replace NaNs with null in advance. + new Column(If(col.isNaN.expr, Literal(null), col.expr)).cast(StringType) + } + } + + dataset.select(selectedCols: _*) + .toDF + .groupBy().agg(aggregator.toColumn) + .as[Array[OpenHashMap[String, Long]]] + .collect()(0) + } + @Since("2.0.0") override def fit(dataset: Dataset[_]): StringIndexerModel = { transformSchema(dataset.schema, logging = true) - val values = dataset.na.drop(Array($(inputCol))) - .select(col($(inputCol)).cast(StringType)) - .rdd.map(_.getString(0)) - val labels = $(stringOrderType) match { - case StringIndexer.frequencyDesc => values.countByValue().toSeq.sortBy(-_._2) - .map(_._1).toArray - case StringIndexer.frequencyAsc => values.countByValue().toSeq.sortBy(_._2) - .map(_._1).toArray - case StringIndexer.alphabetDesc => values.distinct.collect.sortWith(_ > _) - case StringIndexer.alphabetAsc => values.distinct.collect.sortWith(_ < _) - } - copyValues(new StringIndexerModel(uid, labels).setParent(this)) + + val (inputCols, _) = getInOutCols() + + // If input dataset is not originally cached, we need to unpersist it + // once we persist it later. + val needUnpersist = dataset.storageLevel == StorageLevel.NONE + + // In case of equal frequency when frequencyDesc/Asc, the strings are further sorted + // alphabetically. + val labelsArray = $(stringOrderType) match { + case StringIndexer.frequencyDesc => + val sortFunc = StringIndexer.getSortFunc(ascending = false) + countByValue(dataset, inputCols).map { counts => + counts.toSeq.sortWith(sortFunc).map(_._1).toArray + } + case StringIndexer.frequencyAsc => + val sortFunc = StringIndexer.getSortFunc(ascending = true) + countByValue(dataset, inputCols).map { counts => + counts.toSeq.sortWith(sortFunc).map(_._1).toArray + } + case StringIndexer.alphabetDesc => + import dataset.sparkSession.implicits._ + dataset.persist() + val labels = inputCols.map { inputCol => + dataset.select(inputCol).na.drop().distinct().sort(dataset(s"$inputCol").desc) + .as[String].collect() + } + if (needUnpersist) { + dataset.unpersist() + } + labels + case StringIndexer.alphabetAsc => + import dataset.sparkSession.implicits._ + dataset.persist() + val labels = inputCols.map { inputCol => + dataset.select(inputCol).na.drop().distinct().sort(dataset(s"$inputCol").asc) + .as[String].collect() + } + if (needUnpersist) { + dataset.unpersist() + } + labels + } + copyValues(new StringIndexerModel(uid, labelsArray).setParent(this)) } @Since("1.4.0") @@ -172,37 +272,76 @@ object StringIndexer extends DefaultParamsReadable[StringIndexer] { @Since("1.6.0") override def load(path: String): StringIndexer = super.load(path) + + // Returns a function used to sort strings by frequency (ascending or descending). + // In case of equal frequency, it sorts strings by alphabet (ascending). + private[feature] def getSortFunc( + ascending: Boolean): ((String, Long), (String, Long)) => Boolean = { + if (ascending) { + case ((strA: String, freqA: Long), (strB: String, freqB: Long)) => + if (freqA == freqB) { + strA < strB + } else { + freqA < freqB + } + } else { + case ((strA: String, freqA: Long), (strB: String, freqB: Long)) => + if (freqA == freqB) { + strA < strB + } else { + freqA > freqB + } + } + } } /** * Model fitted by [[StringIndexer]]. * - * @param labels Ordered list of labels, corresponding to indices to be assigned. + * @param labelsArray Array of ordered list of labels, corresponding to indices to be assigned + * for each input column. * - * @note During transformation, if the input column does not exist, - * `StringIndexerModel.transform` would return the input dataset unmodified. + * @note During transformation, if any input column does not exist, + * `StringIndexerModel.transform` would skip the input column. + * If all input columns do not exist, it returns the input dataset unmodified. * This is a temporary fix for the case when target labels do not exist during prediction. */ @Since("1.4.0") class StringIndexerModel ( @Since("1.4.0") override val uid: String, - @Since("1.5.0") val labels: Array[String]) + @Since("3.0.0") val labelsArray: Array[Array[String]]) extends Model[StringIndexerModel] with StringIndexerBase with MLWritable { import StringIndexerModel._ @Since("1.5.0") - def this(labels: Array[String]) = this(Identifiable.randomUID("strIdx"), labels) - - private val labelToIndex: OpenHashMap[String, Double] = { - val n = labels.length - val map = new OpenHashMap[String, Double](n) - var i = 0 - while (i < n) { - map.update(labels(i), i) - i += 1 + def this(uid: String, labels: Array[String]) = this(uid, Array(labels)) + + @Since("1.5.0") + def this(labels: Array[String]) = this(Identifiable.randomUID("strIdx"), Array(labels)) + + @Since("3.0.0") + def this(labelsArray: Array[Array[String]]) = this(Identifiable.randomUID("strIdx"), labelsArray) + + @deprecated("`labels` is deprecated and will be removed in 3.1.0. Use `labelsArray` " + + "instead.", "3.0.0") + @Since("1.5.0") + def labels: Array[String] = { + require(labelsArray.length == 1, "This StringIndexerModel is fit on multiple columns. " + + "Call `labelsArray` instead.") + labelsArray(0) + } + + // Prepares the maps for string values to corresponding index values. + private val labelsToIndexArray: Array[OpenHashMap[String, Double]] = { + for (labels <- labelsArray) yield { + val n = labels.length + val map = new OpenHashMap[String, Double](n) + labels.zipWithIndex.foreach { case (label, idx) => + map.update(label, idx) + } + map } - map } /** @group setParam */ @@ -217,33 +356,39 @@ class StringIndexerModel ( @Since("1.4.0") def setOutputCol(value: String): this.type = set(outputCol, value) - @Since("2.0.0") - override def transform(dataset: Dataset[_]): DataFrame = { - if (!dataset.schema.fieldNames.contains($(inputCol))) { - logInfo(s"Input column ${$(inputCol)} does not exist during transformation. " + - "Skip StringIndexerModel.") - return dataset.toDF - } - transformSchema(dataset.schema, logging = true) + /** @group setParam */ + @Since("3.0.0") + def setInputCols(value: Array[String]): this.type = set(inputCols, value) - val filteredLabels = getHandleInvalid match { - case StringIndexer.KEEP_INVALID => labels :+ "__unknown" - case _ => labels + /** @group setParam */ + @Since("3.0.0") + def setOutputCols(value: Array[String]): this.type = set(outputCols, value) + + // This filters out any null values and also the input labels which are not in + // the dataset used for fitting. + private def filterInvalidData(dataset: Dataset[_], inputColNames: Seq[String]): Dataset[_] = { + val conditions: Seq[Column] = (0 until inputColNames.length).map { i => + val inputColName = inputColNames(i) + val labelToIndex = labelsToIndexArray(i) + // We have this additional lookup at `labelToIndex` when `handleInvalid` is set to + // `StringIndexer.SKIP_INVALID`. Another idea is to do this lookup natively by SQL + // expression, however, lookup for a key in a map is not efficient in SparkSQL now. + // See `ElementAt` and `GetMapValue` expressions. If SQL's map lookup is improved, + // we can consider to change this. + val filter = udf { label: String => + labelToIndex.contains(label) + } + filter(dataset(inputColName)) } - val metadata = NominalAttribute.defaultAttr - .withName($(outputCol)).withValues(filteredLabels).toMetadata() - // If we are skipping invalid records, filter them out. - val (filteredDataset, keepInvalid) = $(handleInvalid) match { - case StringIndexer.SKIP_INVALID => - val filterer = udf { label: String => - labelToIndex.contains(label) - } - (dataset.na.drop(Array($(inputCol))).where(filterer(dataset($(inputCol)))), false) - case _ => (dataset, getHandleInvalid == StringIndexer.KEEP_INVALID) - } + dataset.na.drop(inputColNames.filter(dataset.schema.fieldNames.contains(_))) + .where(conditions.reduce(_ and _)) + } + + private def getIndexer(labels: Seq[String], labelToIndex: OpenHashMap[String, Double]) = { + val keepInvalid = (getHandleInvalid == StringIndexer.KEEP_INVALID) - val indexer = udf { label: String => + udf { label: String => if (label == null) { if (keepInvalid) { labels.length @@ -257,29 +402,73 @@ class StringIndexerModel ( } else if (keepInvalid) { labels.length } else { - throw new SparkException(s"Unseen label: $label. To handle unseen labels, " + + throw new SparkException(s"Unseen label: $label. To handle unseen labels, " + s"set Param handleInvalid to ${StringIndexer.KEEP_INVALID}.") } } }.asNondeterministic() + } + + @Since("2.0.0") + override def transform(dataset: Dataset[_]): DataFrame = { + transformSchema(dataset.schema, logging = true) + + var (inputColNames, outputColNames) = getInOutCols() + val outputColumns = new Array[Column](outputColNames.length) + + // Skips invalid rows if `handleInvalid` is set to `StringIndexer.SKIP_INVALID`. + val filteredDataset = if (getHandleInvalid == StringIndexer.SKIP_INVALID) { + filterInvalidData(dataset, inputColNames) + } else { + dataset + } + + for (i <- 0 until outputColNames.length) { + val inputColName = inputColNames(i) + val outputColName = outputColNames(i) + val labelToIndex = labelsToIndexArray(i) + val labels = labelsArray(i) + + if (!dataset.schema.fieldNames.contains(inputColName)) { + logWarning(s"Input column ${inputColName} does not exist during transformation. " + + "Skip StringIndexerModel for this column.") + outputColNames(i) = null + } else { + val filteredLabels = getHandleInvalid match { + case StringIndexer.KEEP_INVALID => labels :+ "__unknown" + case _ => labels + } + val metadata = NominalAttribute.defaultAttr + .withName(outputColName) + .withValues(filteredLabels) + .toMetadata() + + val indexer = getIndexer(labels, labelToIndex) - filteredDataset.select(col("*"), - indexer(dataset($(inputCol)).cast(StringType)).as($(outputCol), metadata)) + outputColumns(i) = indexer(dataset(inputColName).cast(StringType)) + .as(outputColName, metadata) + } + } + + val filteredOutputColNames = outputColNames.filter(_ != null) + val filteredOutputColumns = outputColumns.filter(_ != null) + + require(filteredOutputColNames.length == filteredOutputColumns.length) + if (filteredOutputColNames.length > 0) { + filteredDataset.withColumns(filteredOutputColNames, filteredOutputColumns) + } else { + filteredDataset.toDF() + } } @Since("1.4.0") override def transformSchema(schema: StructType): StructType = { - if (schema.fieldNames.contains($(inputCol))) { - validateAndTransformSchema(schema) - } else { - // If the input column does not exist during transformation, we skip StringIndexerModel. - schema - } + validateAndTransformSchema(schema, skipNonExistsCol = true) } @Since("1.4.1") override def copy(extra: ParamMap): StringIndexerModel = { - val copied = new StringIndexerModel(uid, labels) + val copied = new StringIndexerModel(uid, labelsArray) copyValues(copied, extra).setParent(parent) } @@ -293,11 +482,11 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { private[StringIndexerModel] class StringIndexModelWriter(instance: StringIndexerModel) extends MLWriter { - private case class Data(labels: Array[String]) + private case class Data(labelsArray: Array[Array[String]]) override protected def saveImpl(path: String): Unit = { DefaultParamsWriter.saveMetadata(instance, path, sc) - val data = Data(instance.labels) + val data = Data(instance.labelsArray) val dataPath = new Path(path, "data").toString sparkSession.createDataFrame(Seq(data)).repartition(1).write.parquet(dataPath) } @@ -310,11 +499,25 @@ object StringIndexerModel extends MLReadable[StringIndexerModel] { override def load(path: String): StringIndexerModel = { val metadata = DefaultParamsReader.loadMetadata(path, sc, className) val dataPath = new Path(path, "data").toString - val data = sparkSession.read.parquet(dataPath) - .select("labels") - .head() - val labels = data.getAs[Seq[String]](0).toArray - val model = new StringIndexerModel(metadata.uid, labels) + + // We support loading old `StringIndexerModel` saved by previous Spark versions. + // Previous model has `labels`, but new model has `labelsArray`. + val (majorVersion, minorVersion) = majorMinorVersion(metadata.sparkVersion) + val labelsArray = if (majorVersion < 3) { + // Spark 2.4 and before. + val data = sparkSession.read.parquet(dataPath) + .select("labels") + .head() + val labels = data.getAs[Seq[String]](0).toArray + Array(labels) + } else { + // After Spark 3.0. + val data = sparkSession.read.parquet(dataPath) + .select("labelsArray") + .head() + data.getAs[Seq[Seq[String]]](0).map(_.toArray).toArray + } + val model = new StringIndexerModel(metadata.uid, labelsArray) metadata.getAndSetParams(model) model } @@ -421,3 +624,47 @@ object IndexToString extends DefaultParamsReadable[IndexToString] { @Since("1.6.0") override def load(path: String): IndexToString = super.load(path) } + +/** + * A SQL `Aggregator` used by `StringIndexer` to count labels in string columns during fitting. + */ +private class StringIndexerAggregator(numColumns: Int) + extends Aggregator[Row, Array[OpenHashMap[String, Long]], Array[OpenHashMap[String, Long]]] { + + override def zero: Array[OpenHashMap[String, Long]] = + Array.fill(numColumns)(new OpenHashMap[String, Long]()) + + def reduce( + array: Array[OpenHashMap[String, Long]], + row: Row): Array[OpenHashMap[String, Long]] = { + for (i <- 0 until numColumns) { + val stringValue = row.getString(i) + // We don't count for null values. + if (stringValue != null) { + array(i).changeValue(stringValue, 1L, _ + 1) + } + } + array + } + + def merge( + array1: Array[OpenHashMap[String, Long]], + array2: Array[OpenHashMap[String, Long]]): Array[OpenHashMap[String, Long]] = { + for (i <- 0 until numColumns) { + array2(i).foreach { case (key: String, count: Long) => + array1(i).changeValue(key, count, _ + count) + } + } + array1 + } + + def finish(array: Array[OpenHashMap[String, Long]]): Array[OpenHashMap[String, Long]] = array + + override def bufferEncoder: Encoder[Array[OpenHashMap[String, Long]]] = { + Encoders.kryo[Array[OpenHashMap[String, Long]]] + } + + override def outputEncoder: Encoder[Array[OpenHashMap[String, Long]]] = { + Encoders.kryo[Array[OpenHashMap[String, Long]]] + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala index 57e23d5072b88..e6e9bdfd292b1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/VectorAssembler.scala @@ -20,7 +20,6 @@ package org.apache.spark.ml.feature import java.util.NoSuchElementException import scala.collection.mutable -import scala.language.existentials import org.apache.spark.SparkException import org.apache.spark.annotation.Since @@ -131,14 +130,14 @@ class VectorAssembler @Since("1.4.0") (@Since("1.4.0") override val uid: String) throw new SparkException(s"VectorAssembler does not support the $otherType type") } } - val featureAttributes = featureAttributesMap.flatten[Attribute].toArray - val lengths = featureAttributesMap.map(a => a.length).toArray + val featureAttributes = featureAttributesMap.flatten[Attribute] + val lengths = featureAttributesMap.map(a => a.length) val metadata = new AttributeGroup($(outputCol), featureAttributes).toMetadata() - val (filteredDataset, keepInvalid) = $(handleInvalid) match { - case VectorAssembler.SKIP_INVALID => (dataset.na.drop($(inputCols)), false) - case VectorAssembler.KEEP_INVALID => (dataset, true) - case VectorAssembler.ERROR_INVALID => (dataset, false) + val filteredDataset = $(handleInvalid) match { + case VectorAssembler.SKIP_INVALID => dataset.na.drop($(inputCols)) + case VectorAssembler.KEEP_INVALID | VectorAssembler.ERROR_INVALID => dataset } + val keepInvalid = $(handleInvalid) == VectorAssembler.KEEP_INVALID // Data transformation. val assembleFunc = udf { r: Row => VectorAssembler.assemble(lengths, keepInvalid)(r.toSeq: _*) diff --git a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala index fc9996d69ba72..6ae90b8050932 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/feature/Word2Vec.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.feature import org.apache.hadoop.fs.Path import org.apache.spark.annotation.Since +import org.apache.spark.internal.config.Kryo.KRYO_SERIALIZER_MAX_BUFFER_SIZE import org.apache.spark.ml.{Estimator, Model} import org.apache.spark.ml.linalg.{BLAS, Vector, Vectors, VectorUDT} import org.apache.spark.ml.param._ @@ -339,7 +340,7 @@ object Word2VecModel extends MLReadable[Word2VecModel] { val wordVectors = instance.wordVectors.getVectors val dataPath = new Path(path, "data").toString val bufferSizeInBytes = Utils.byteStringAsBytes( - sc.conf.get("spark.kryoserializer.buffer.max", "64m")) + sc.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "64m")) val numPartitions = Word2VecModelWriter.calculateNumberOfPartitions( bufferSizeInBytes, instance.wordVectors.wordIndex.size, instance.getVectorSize) val spark = sparkSession diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala index 840a89b76d26b..7322815c12ab8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/FPGrowth.scala @@ -118,10 +118,10 @@ private[fpm] trait FPGrowthParams extends Params with HasPredictionCol { /** * :: Experimental :: * A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in - * Li et al., PFP: Parallel FP-Growth for Query + * Li et al., PFP: Parallel FP-Growth for Query * Recommendation. PFP distributes computation in such a way that each worker executes an * independent group of mining tasks. The FP-Growth algorithm is described in - * Han et al., Mining frequent patterns without + * Han et al., Mining frequent patterns without * candidate generation. Note null values in the itemsCol column are ignored during fit(). * * @see diff --git a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala index bd1c1a8885201..b0006a8d4a58e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/fpm/PrefixSpan.scala @@ -20,6 +20,7 @@ package org.apache.spark.ml.fpm import org.apache.spark.annotation.{Experimental, Since} import org.apache.spark.ml.param._ import org.apache.spark.ml.util.Identifiable +import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.fpm.{PrefixSpan => mllibPrefixSpan} import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions.col @@ -30,7 +31,7 @@ import org.apache.spark.sql.types.{ArrayType, LongType, StructField, StructType} * A parallel PrefixSpan algorithm to mine frequent sequential patterns. * The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan: Mining Sequential Patterns * Efficiently by Prefix-Projected Pattern Growth - * (see here). + * (see here). * This class is not yet an Estimator/Transformer, use `findFrequentSequentialPatterns` method to * run the PrefixSpan algorithm. * @@ -135,7 +136,10 @@ final class PrefixSpan(@Since("2.4.0") override val uid: String) extends Params * - `freq: Long` */ @Since("2.4.0") - def findFrequentSequentialPatterns(dataset: Dataset[_]): DataFrame = { + def findFrequentSequentialPatterns(dataset: Dataset[_]): DataFrame = instrumented { instr => + instr.logDataset(dataset) + instr.logParams(this, params: _*) + val sequenceColParam = $(sequenceCol) val inputType = dataset.schema(sequenceColParam).dataType require(inputType.isInstanceOf[ArrayType] && diff --git a/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala b/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala index 1fae1dc04ad7b..faa8589cb5322 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/image/HadoopUtils.scala @@ -17,7 +17,6 @@ package org.apache.spark.ml.image -import scala.language.existentials import scala.util.Random import org.apache.commons.io.FilenameUtils @@ -103,7 +102,7 @@ private object SamplePathFilter { // scalastyle:off hadoopconfiguration val hadoopConf = spark.sparkContext.hadoopConfiguration // scalastyle:on hadoopconfiguration - val old = Option(hadoopConf.getClass(flagName, null)) + val old = hadoopConf.getClass(flagName, null) hadoopConf.setDouble(SamplePathFilter.ratioParam, sampleRatio) hadoopConf.setLong(SamplePathFilter.seedParam, seed) hadoopConf.setClass(flagName, classOf[SamplePathFilter], classOf[PathFilter]) @@ -111,8 +110,8 @@ private object SamplePathFilter { hadoopConf.unset(SamplePathFilter.ratioParam) hadoopConf.unset(SamplePathFilter.seedParam) old match { - case Some(v) => hadoopConf.setClass(flagName, v, classOf[PathFilter]) - case None => hadoopConf.unset(flagName) + case null => hadoopConf.unset(flagName) + case v => hadoopConf.setClass(flagName, v, classOf[PathFilter]) } } } else { diff --git a/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala b/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala index 0b13eefdf3f5f..a7ddf2f1f130f 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/image/ImageSchema.scala @@ -133,7 +133,13 @@ object ImageSchema { */ private[spark] def decode(origin: String, bytes: Array[Byte]): Option[Row] = { - val img = ImageIO.read(new ByteArrayInputStream(bytes)) + val img = try { + ImageIO.read(new ByteArrayInputStream(bytes)) + } catch { + // Catch runtime exception because `ImageIO` may throw unexcepted `RuntimeException`. + // But do not catch the declared `IOException` (regarded as FileSystem failure) + case _: RuntimeException => null + } if (img == null) { None diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala index 1b7c15f1f0a8c..9f32603b19a6d 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/WeightedLeastSquares.scala @@ -88,7 +88,7 @@ private[ml] class WeightedLeastSquares( require(regParam >= 0.0, s"regParam cannot be negative: $regParam") require(elasticNetParam >= 0.0 && elasticNetParam <= 1.0, s"elasticNetParam must be in [0, 1]: $elasticNetParam") - require(maxIter >= 0, s"maxIter must be a positive integer: $maxIter") + require(maxIter > 0, s"maxIter must be a positive integer: $maxIter") require(tol >= 0.0, s"tol must be >= 0, but was set to $tol") /** @@ -133,7 +133,7 @@ private[ml] class WeightedLeastSquares( return new WeightedLeastSquaresModel(coefficients, intercept, diagInvAtWA, Array(0D)) } else { require(!(regParam > 0.0 && standardizeLabel), "The standard deviation of the label is " + - "zero. Model cannot be regularized with standardization=true") + "zero. Model cannot be regularized when labels are standardized.") instr.logWarning(s"The standard deviation of the label is zero. Consider setting " + s"fitIntercept=true.") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala index 13f64d2d50424..fc4c423a60b2a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/aggregator/HuberAggregator.scala @@ -81,6 +81,8 @@ private[ml] class HuberAggregator( } else { 0.0 } + // make transient so we do not serialize between aggregation stages + @transient private lazy val coefficients = bcParameters.value.toArray.slice(0, numFeatures) /** * Add a new training instance to this HuberAggregator, and update the loss and gradient @@ -97,7 +99,7 @@ private[ml] class HuberAggregator( if (weight == 0.0) return this val localFeaturesStd = bcFeaturesStd.value - val localCoefficients = bcParameters.value.toArray.slice(0, numFeatures) + val localCoefficients = coefficients val localGradientSumArray = gradientSumArray val margin = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala index 387f7c5b1ff33..84977a6916f07 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/optim/loss/RDDLossFunction.scala @@ -65,7 +65,7 @@ private[ml] class RDDLossFunction[ BLAS.axpy(1.0, regGradient, gradient) regLoss }.getOrElse(0.0) - bcCoefficients.destroy(blocking = false) + bcCoefficients.destroy() (newAgg.loss + regLoss, gradient.asBreeze.toDenseVector) } } 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 e6c347ed17c15..4c50f1e3292bc 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 @@ -97,7 +97,7 @@ class Param[T](val parent: String, val name: String, val doc: String, val isVali case m: Matrix => JsonMatrixConverter.toJson(m) case _ => - throw new NotImplementedError( + throw new UnsupportedOperationException( "The default jsonEncode only supports string, vector and matrix. " + s"${this.getClass.getName} must override jsonEncode for ${value.getClass.getName}.") } @@ -151,7 +151,7 @@ private[ml] object Param { } case _ => - throw new NotImplementedError( + throw new UnsupportedOperationException( "The default jsonDecode only supports string, vector and matrix. " + s"${this.getClass.getName} must override jsonDecode to support its value type.") } 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 7e08675f834da..1afcf1bf7ba4b 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 @@ -22,6 +22,8 @@ import java.io.PrintWriter import scala.reflect.ClassTag import scala.xml.Utility +import org.apache.spark.util.Utils + /** * Code generator for shared params (sharedParams.scala). Run under the Spark folder with * {{{ @@ -103,9 +105,9 @@ private[shared] object SharedParamsCodeGen { val code = genSharedParams(params) val file = "src/main/scala/org/apache/spark/ml/param/shared/sharedParams.scala" - val writer = new PrintWriter(file) - writer.write(code) - writer.close() + Utils.tryWithResource(new PrintWriter(file)) { writer => + writer.write(code) + } } /** Description of a param. */ diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala index 48485e02edda8..1b5f77a9ae897 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/r/AFTSurvivalRegressionWrapper.scala @@ -62,7 +62,7 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg private val FORMULA_REGEXP = """Surv\(([^,]+), ([^,]+)\) ~ (.+)""".r private def formulaRewrite(formula: String): (String, String) = { - var rewritedFormula: String = null + var rewrittenFormula: String = null var censorCol: String = null try { val FORMULA_REGEXP(label, censor, features) = formula @@ -71,14 +71,14 @@ private[r] object AFTSurvivalRegressionWrapper extends MLReadable[AFTSurvivalReg throw new UnsupportedOperationException( "Terms of survreg formula can not support dot operator.") } - rewritedFormula = label.trim + "~" + features.trim + rewrittenFormula = label.trim + "~" + features.trim censorCol = censor.trim } catch { case e: MatchError => throw new SparkException(s"Could not parse formula: $formula") } - (rewritedFormula, censorCol) + (rewrittenFormula, censorCol) } diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala new file mode 100644 index 0000000000000..b5dfad0224ed8 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/PowerIterationClusteringWrapper.scala @@ -0,0 +1,39 @@ +/* + * 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.spark.ml.clustering.PowerIterationClustering + +private[r] object PowerIterationClusteringWrapper { + def getPowerIterationClustering( + k: Int, + initMode: String, + maxIter: Int, + srcCol: String, + dstCol: String, + weightCol: String): PowerIterationClustering = { + val pic = new PowerIterationClustering() + .setK(k) + .setInitMode(initMode) + .setMaxIter(maxIter) + .setSrcCol(srcCol) + .setDstCol(dstCol) + if (weightCol != null) pic.setWeightCol(weightCol) + pic + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala new file mode 100644 index 0000000000000..268d596fafc54 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/r/PrefixSpanWrapper.scala @@ -0,0 +1,34 @@ +/* + * 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.spark.ml.fpm.PrefixSpan + +private[r] object PrefixSpanWrapper { + def getPrefixSpan( + minSupport: Double, + maxPatternLength: Int, + maxLocalProjDBSize: Double, + sequenceCol: String): PrefixSpan = { + new PrefixSpan() + .setMinSupport(minSupport) + .setMaxPatternLength(maxPatternLength) + .setMaxLocalProjDBSize(maxLocalProjDBSize.toLong) + .setSequenceCol(sequenceCol) + } +} 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 ffe592789b3cc..4fd3ed59ca20b 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 @@ -557,7 +557,7 @@ object ALSModel extends MLReadable[ALSModel] { * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * http://dx.doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. + * https://doi.org/10.1109/ICDM.2008.22, adapted for the blocked approach used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, * this finds the approximations for a preference matrix `P` where the elements of `P` are 1 if @@ -847,7 +847,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { } /** Adds an observation. */ - def add(a: Array[Float], b: Double, c: Double = 1.0): this.type = { + def add(a: Array[Float], b: Double, c: Double = 1.0): NormalEquation = { require(c >= 0.0) require(a.length == k) copyToDouble(a) @@ -859,7 +859,7 @@ object ALS extends DefaultParamsReadable[ALS] with Logging { } /** Merges another normal equation object. */ - def merge(other: NormalEquation): this.type = { + def merge(other: NormalEquation): NormalEquation = { require(other.k == k) blas.daxpy(ata.length, 1.0, other.ata, 1, ata, 1) blas.daxpy(atb.length, 1.0, other.atb, 1, atb, 1) diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala index 8d6e36697d2cc..1565782dd6314 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/AFTSurvivalRegression.scala @@ -37,7 +37,7 @@ import org.apache.spark.mllib.linalg.VectorImplicits._ import org.apache.spark.mllib.stat.MultivariateOnlineSummarizer import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset, Row} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ import org.apache.spark.sql.types.{DoubleType, StructType} import org.apache.spark.storage.StorageLevel @@ -275,7 +275,7 @@ class AFTSurvivalRegression @Since("1.6.0") (@Since("1.6.0") override val uid: S state.x.toArray.clone() } - bcFeaturesStd.destroy(blocking = false) + bcFeaturesStd.destroy() if (handlePersistence) instances.unpersist() val rawCoefficients = parameters.slice(2, parameters.length) @@ -355,13 +355,28 @@ class AFTSurvivalRegressionModel private[ml] ( @Since("2.0.0") override def transform(dataset: Dataset[_]): DataFrame = { transformSchema(dataset.schema, logging = true) - val predictUDF = udf { features: Vector => predict(features) } - val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + + if ($(predictionCol).nonEmpty) { + val predictUDF = udf { features: Vector => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) + } + if (hasQuantilesCol) { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) - .withColumn($(quantilesCol), predictQuantilesUDF(col($(featuresCol)))) + val predictQuantilesUDF = udf { features: Vector => predictQuantiles(features)} + predictionColNames :+= $(quantilesCol) + predictionColumns :+= predictQuantilesUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) } else { - dataset.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + this.logWarning(s"$uid: AFTSurvivalRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } } @@ -633,7 +648,7 @@ private class AFTCostFun( case (aggregator1, aggregator2) => aggregator1.merge(aggregator2) }, depth = aggregationDepth) - bcParameters.destroy(blocking = false) + bcParameters.destroy() (aftAggregator.loss, aftAggregator.gradient) } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala index 6fa656275c1fd..6348289de516e 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/DecisionTreeRegressor.scala @@ -23,7 +23,7 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -34,8 +34,9 @@ import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.model.{DecisionTreeModel => OldDecisionTreeModel} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Dataset} +import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.DoubleType /** @@ -54,27 +55,30 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S // Override parameter setters from parent trait for Java API compatibility. /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ + @Since("3.0.0") + def setMinWeightFractionPerNode(value: Double): this.type = set(minWeightFractionPerNode, value) + @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -86,32 +90,47 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = set(impurity, value) + def setImpurity(value: String): this.type = set(impurity, value) /** @group setParam */ @Since("1.6.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) /** @group setParam */ @Since("2.0.0") def setVarianceCol(value: String): this.type = set(varianceCol, value) + /** + * Sets the value of param [[weightCol]]. + * If this is not set or empty, we treat all instance weights as 1.0. + * Default is not set, so all instances have weight one. + * + * @group setParam + */ + @Since("3.0.0") + def setWeightCol(value: String): this.type = set(weightCol, value) + override protected def train( dataset: Dataset[_]): DecisionTreeRegressionModel = instrumented { instr => val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + val w = if (!isDefined(weightCol) || $(weightCol).isEmpty) lit(1.0) else col($(weightCol)) + val instances = + dataset.select(col($(labelCol)).cast(DoubleType), w, col($(featuresCol))).rdd.map { + case Row(label: Double, weight: Double, features: Vector) => + Instance(label, weight, features) + } val strategy = getOldStrategy(categoricalFeatures) instr.logPipelineStage(this) - instr.logDataset(oldDataset) + instr.logDataset(instances) instr.logParams(this, params: _*) - val trees = RandomForest.run(oldDataset, strategy, numTrees = 1, featureSubsetStrategy = "all", + val trees = RandomForest.run(instances, strategy, numTrees = 1, featureSubsetStrategy = "all", seed = $(seed), instr = Some(instr), parentUID = Some(uid)) trees.head.asInstanceOf[DecisionTreeRegressionModel] @@ -126,8 +145,9 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S instr.logDataset(data) instr.logParams(this, params: _*) - val trees = RandomForest.run(data, oldStrategy, numTrees = 1, featureSubsetStrategy, - seed = $(seed), instr = Some(instr), parentUID = Some(uid)) + val instances = data.map(_.toInstance) + val trees = RandomForest.run(instances, oldStrategy, numTrees = 1, + featureSubsetStrategy, seed = $(seed), instr = Some(instr), parentUID = Some(uid)) trees.head.asInstanceOf[DecisionTreeRegressionModel] } @@ -145,7 +165,7 @@ class DecisionTreeRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S @Since("1.4.0") object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor] { /** Accessor for supported impurities: variance */ - final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities + final val supportedImpurities: Array[String] = HasVarianceImpurity.supportedImpurities @Since("2.0.0") override def load(path: String): DecisionTreeRegressor = super.load(path) @@ -155,6 +175,7 @@ object DecisionTreeRegressor extends DefaultParamsReadable[DecisionTreeRegressor * * Decision tree (Wikipedia) model for regression. * It supports both continuous and categorical features. + * * @param rootNode Root of the decision tree */ @Since("1.4.0") @@ -173,6 +194,7 @@ class DecisionTreeRegressionModel private[ml] ( /** * Construct a decision tree regression model. + * * @param rootNode Root node of tree, with other nodes attached. */ private[ml] def this(rootNode: Node, numFeatures: Int) = @@ -194,16 +216,28 @@ class DecisionTreeRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector) => predict(features) } - val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } - var output = dataset.toDF() + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)))) + val predictUDF = udf { (features: Vector) => predict(features) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol))) } + if (isDefined(varianceCol) && $(varianceCol).nonEmpty) { - output = output.withColumn($(varianceCol), predictVarianceUDF(col($(featuresCol)))) + val predictVarianceUDF = udf { (features: Vector) => predictVariance(features) } + predictionColNames :+= $(varianceCol) + predictionColumns :+= predictVarianceUDF(col($(featuresCol))) + } + + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: DecisionTreeRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() } - output } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala index 07f88d8d5f84d..9f0f567a5b53a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GBTRegressor.scala @@ -34,7 +34,6 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{GradientBoostedTreesModel => OldGBTModel} -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset, Row} import org.apache.spark.sql.functions._ @@ -69,27 +68,27 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -101,7 +100,7 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** * The impurity setting is ignored for GBT models. @@ -110,7 +109,7 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) * @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = { + def setImpurity(value: String): this.type = { logWarning("GBTRegressor.setImpurity should NOT be used") this } @@ -119,21 +118,21 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) // Parameters from GBTParams: /** @group setParam */ @Since("1.4.0") - override def setMaxIter(value: Int): this.type = set(maxIter, value) + def setMaxIter(value: Int): this.type = set(maxIter, value) /** @group setParam */ @Since("1.4.0") - override def setStepSize(value: Double): this.type = set(stepSize, value) + def setStepSize(value: Double): this.type = set(stepSize, value) // Parameters from GBTRegressorParams: @@ -143,7 +142,7 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) /** @group setParam */ @Since("2.3.0") - override def setFeatureSubsetStrategy(value: String): this.type = + def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) /** @group setParam */ @@ -166,15 +165,14 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) } else { (extractLabeledPoints(dataset), null) } - val numFeatures = trainDataset.first().features.size val boostingStrategy = super.getOldBoostingStrategy(categoricalFeatures, OldAlgo.Regression) instr.logPipelineStage(this) instr.logDataset(dataset) instr.logParams(this, labelCol, featuresCol, predictionCol, impurity, lossType, maxDepth, maxBins, maxIter, maxMemoryInMB, minInfoGain, minInstancesPerNode, - seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy) - instr.logNumFeatures(numFeatures) + seed, stepSize, subsamplingRate, cacheNodeIds, checkpointInterval, featureSubsetStrategy, + validationIndicatorCol, validationTol) val (baseLearners, learnerWeights) = if (withValidation) { GradientBoostedTrees.runWithValidation(trainDataset, validationDataset, boostingStrategy, @@ -183,6 +181,10 @@ class GBTRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: String) GradientBoostedTrees.run(trainDataset, boostingStrategy, $(seed), $(featureSubsetStrategy)) } + + val numFeatures = baseLearners.head.numFeatures + instr.logNumFeatures(numFeatures) + new GBTRegressionModel(uid, baseLearners, learnerWeights, numFeatures) } @@ -283,7 +285,8 @@ class GBTRegressionModel private[ml]( * @see `DecisionTreeRegressionModel.featureImportances` */ @Since("2.0.0") - lazy val featureImportances: Vector = TreeEnsembleModel.featureImportances(trees, numFeatures) + lazy val featureImportances: Vector = + TreeEnsembleModel.featureImportances(trees, numFeatures, perTreeNormalization = false) /** (private[ml]) Convert to a model in the old API */ private[ml] def toOld: OldGBTModel = { diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala index abb60ea205751..b1a8f95c12618 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala @@ -1001,7 +1001,8 @@ class GeneralizedLinearRegressionModel private[ml] ( @Since("2.0.0") val coefficients: Vector, @Since("2.0.0") val intercept: Double) extends RegressionModel[Vector, GeneralizedLinearRegressionModel] - with GeneralizedLinearRegressionBase with MLWritable { + with GeneralizedLinearRegressionBase with MLWritable + with HasTrainingSummary[GeneralizedLinearRegressionTrainingSummary] { /** * Sets the link prediction (linear predictor) column name. @@ -1040,43 +1041,39 @@ class GeneralizedLinearRegressionModel private[ml] ( } override protected def transformImpl(dataset: Dataset[_]): DataFrame = { - val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } - val predictLinkUDF = udf { (features: Vector, offset: Double) => predictLink(features, offset) } + var predictionColNames = Seq.empty[String] + var predictionColumns = Seq.empty[Column] val offset = if (!hasOffsetCol) lit(0.0) else col($(offsetCol)).cast(DoubleType) - var output = dataset + if ($(predictionCol).nonEmpty) { - output = output.withColumn($(predictionCol), predictUDF(col($(featuresCol)), offset)) + val predictUDF = udf { (features: Vector, offset: Double) => predict(features, offset) } + predictionColNames :+= $(predictionCol) + predictionColumns :+= predictUDF(col($(featuresCol)), offset) } + if (hasLinkPredictionCol) { - output = output.withColumn($(linkPredictionCol), predictLinkUDF(col($(featuresCol)), offset)) + val predictLinkUDF = + udf { (features: Vector, offset: Double) => predictLink(features, offset) } + predictionColNames :+= $(linkPredictionCol) + predictionColumns :+= predictLinkUDF(col($(featuresCol)), offset) } - output.toDF() - } - private var trainingSummary: Option[GeneralizedLinearRegressionTrainingSummary] = None + if (predictionColNames.nonEmpty) { + dataset.withColumns(predictionColNames, predictionColumns) + } else { + this.logWarning(s"$uid: GeneralizedLinearRegressionModel.transform() does nothing" + + " because no output columns were set.") + dataset.toDF() + } + } /** * Gets R-like summary of model on training set. An exception is * thrown if there is no summary available. */ @Since("2.0.0") - def summary: GeneralizedLinearRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException( - "No training summary available for this GeneralizedLinearRegressionModel") - } - - /** - * Indicates if [[summary]] is available. - */ - @Since("2.0.0") - def hasSummary: Boolean = trainingSummary.nonEmpty - - private[regression] - def setSummary(summary: Option[GeneralizedLinearRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } + override def summary: GeneralizedLinearRegressionTrainingSummary = super.summary /** * Evaluate the model on the given dataset, returning a summary of the results. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala index ce6c12cc368dd..09f3f94d346b6 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/LinearRegression.scala @@ -109,12 +109,13 @@ private[regression] trait LinearRegressionParams extends PredictorParams schema: StructType, fitting: Boolean, featuresDataType: DataType): StructType = { - if ($(loss) == Huber) { - require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + - "normal solver, please change solver to auto or l-bfgs.") - require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + - s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") - + if (fitting) { + if ($(loss) == Huber) { + require($(solver)!= Normal, "LinearRegression with huber loss doesn't support " + + "normal solver, please change solver to auto or l-bfgs.") + require($(elasticNetParam) == 0.0, "LinearRegression with huber loss only supports " + + s"L2 regularization, but got elasticNetParam = $getElasticNetParam.") + } } super.validateAndTransformSchema(schema, fitting, featuresDataType) } @@ -531,8 +532,8 @@ class LinearRegression @Since("1.3.0") (@Since("1.3.0") override val uid: String throw new SparkException(msg) } - bcFeaturesMean.destroy(blocking = false) - bcFeaturesStd.destroy(blocking = false) + bcFeaturesMean.destroy() + bcFeaturesStd.destroy() val parameters = state.x.toArray.clone() @@ -647,33 +648,20 @@ class LinearRegressionModel private[ml] ( @Since("1.3.0") val intercept: Double, @Since("2.3.0") val scale: Double) extends RegressionModel[Vector, LinearRegressionModel] - with LinearRegressionParams with GeneralMLWritable { + with LinearRegressionParams with GeneralMLWritable + with HasTrainingSummary[LinearRegressionTrainingSummary] { private[ml] def this(uid: String, coefficients: Vector, intercept: Double) = this(uid, coefficients, intercept, 1.0) - private var trainingSummary: Option[LinearRegressionTrainingSummary] = None - override val numFeatures: Int = coefficients.size /** * Gets summary (e.g. residuals, mse, r-squared ) of model on training set. An exception is - * thrown if `trainingSummary == None`. + * thrown if `hasSummary` is false. */ @Since("1.5.0") - def summary: LinearRegressionTrainingSummary = trainingSummary.getOrElse { - throw new SparkException("No training summary available for this LinearRegressionModel") - } - - private[regression] - def setSummary(summary: Option[LinearRegressionTrainingSummary]): this.type = { - this.trainingSummary = summary - this - } - - /** Indicates whether a training summary exists for this model instance. */ - @Since("1.5.0") - def hasSummary: Boolean = trainingSummary.isDefined + override def summary: LinearRegressionTrainingSummary = super.summary /** * Evaluates the model on a test dataset. diff --git a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala index 82bf66ff66d8a..6f36dfb9ff51c 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/regression/RandomForestRegressor.scala @@ -22,7 +22,6 @@ import org.json4s.JsonDSL._ import org.apache.spark.annotation.Since import org.apache.spark.ml.{PredictionModel, Predictor} -import org.apache.spark.ml.feature.LabeledPoint import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.tree._ @@ -32,10 +31,8 @@ import org.apache.spark.ml.util.DefaultParamsReader.Metadata import org.apache.spark.ml.util.Instrumentation.instrumented import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo} import org.apache.spark.mllib.tree.model.{RandomForestModel => OldRandomForestModel} -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Dataset} -import org.apache.spark.sql.functions._ - +import org.apache.spark.sql.functions.{col, udf} /** * Random Forest @@ -56,27 +53,27 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S /** @group setParam */ @Since("1.4.0") - override def setMaxDepth(value: Int): this.type = set(maxDepth, value) + def setMaxDepth(value: Int): this.type = set(maxDepth, value) /** @group setParam */ @Since("1.4.0") - override def setMaxBins(value: Int): this.type = set(maxBins, value) + def setMaxBins(value: Int): this.type = set(maxBins, value) /** @group setParam */ @Since("1.4.0") - override def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) + def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) /** @group setParam */ @Since("1.4.0") - override def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) /** @group expertSetParam */ @Since("1.4.0") - override def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) + def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) /** @group expertSetParam */ @Since("1.4.0") - override def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) + def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) /** * Specifies how often to checkpoint the cached node IDs. @@ -88,52 +85,53 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S * @group setParam */ @Since("1.4.0") - override def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) + def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) /** @group setParam */ @Since("1.4.0") - override def setImpurity(value: String): this.type = set(impurity, value) + def setImpurity(value: String): this.type = set(impurity, value) // Parameters from TreeEnsembleParams: /** @group setParam */ @Since("1.4.0") - override def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) + def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) /** @group setParam */ @Since("1.4.0") - override def setSeed(value: Long): this.type = set(seed, value) + def setSeed(value: Long): this.type = set(seed, value) // Parameters from RandomForestParams: /** @group setParam */ @Since("1.4.0") - override def setNumTrees(value: Int): this.type = set(numTrees, value) + def setNumTrees(value: Int): this.type = set(numTrees, value) /** @group setParam */ @Since("1.4.0") - override def setFeatureSubsetStrategy(value: String): this.type = + def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) override protected def train( dataset: Dataset[_]): RandomForestRegressionModel = instrumented { instr => val categoricalFeatures: Map[Int, Int] = MetadataUtils.getCategoricalFeatures(dataset.schema($(featuresCol))) - val oldDataset: RDD[LabeledPoint] = extractLabeledPoints(dataset) + + val instances = extractLabeledPoints(dataset).map(_.toInstance) val strategy = super.getOldStrategy(categoricalFeatures, numClasses = 0, OldAlgo.Regression, getOldImpurity) instr.logPipelineStage(this) - instr.logDataset(dataset) + instr.logDataset(instances) instr.logParams(this, labelCol, featuresCol, predictionCol, impurity, numTrees, featureSubsetStrategy, maxDepth, maxBins, maxMemoryInMB, minInfoGain, minInstancesPerNode, seed, subsamplingRate, cacheNodeIds, checkpointInterval) val trees = RandomForest - .run(oldDataset, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) + .run(instances, strategy, getNumTrees, getFeatureSubsetStrategy, getSeed, Some(instr)) .map(_.asInstanceOf[DecisionTreeRegressionModel]) - val numFeatures = oldDataset.first().features.size + val numFeatures = trees.head.numFeatures instr.logNamedValue(Instrumentation.loggerTags.numFeatures, numFeatures) new RandomForestRegressionModel(uid, trees, numFeatures) } @@ -146,7 +144,7 @@ class RandomForestRegressor @Since("1.4.0") (@Since("1.4.0") override val uid: S object RandomForestRegressor extends DefaultParamsReadable[RandomForestRegressor]{ /** Accessor for supported impurity settings: variance */ @Since("1.4.0") - final val supportedImpurities: Array[String] = TreeRegressorParams.supportedImpurities + final val supportedImpurities: Array[String] = HasVarianceImpurity.supportedImpurities /** Accessor for supported featureSubsetStrategy settings: auto, all, onethird, sqrt, log2 */ @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala index a111c95248cf5..d4d74082dc8c5 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/source/image/ImageDataSource.scala @@ -19,14 +19,17 @@ package org.apache.spark.ml.source.image /** * `image` package implements Spark SQL data source API for loading image data as `DataFrame`. - * The loaded `DataFrame` has one `StructType` column: `image`. + * It can load compressed image (jpeg, png, etc.) into raw image representation via `ImageIO` + * in Java library. + * The loaded `DataFrame` has one `StructType` column: `image`, containing image data stored + * as image schema. * The schema of the `image` column is: - * - origin: String (represents the file path of the image) - * - height: Int (height of the image) - * - width: Int (width of the image) - * - nChannels: Int (number of the image channels) - * - mode: Int (OpenCV-compatible type) - * - data: BinaryType (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) + * - origin: `StringType` (represents the file path of the image) + * - height: `IntegerType` (height of the image) + * - width: `IntegerType` (width of the image) + * - nChannels: `IntegerType` (number of image channels) + * - mode: `IntegerType` (OpenCV-compatible type) + * - data: `BinaryType` (Image bytes in OpenCV-compatible order: row-wise BGR in most cases) * * To use image data source, you need to set "image" as the format in `DataFrameReader` and * optionally specify the data source options, for example: diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala index 6e885d7c8aec5..8167ea68a7150 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Correlation.scala @@ -49,7 +49,7 @@ object Correlation { * Supported: `pearson` (default), `spearman` * @return A dataframe that contains the correlation matrix of the column of vectors. This * dataframe contains a single row and a single column of name - * '$METHODNAME($COLUMN)'. + * `$METHODNAME($COLUMN)`. * @throws IllegalArgumentException if the column is not a valid column in the dataset, or if * the content of this column is not of type Vector. * diff --git a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala index d40827edb6d64..ed7d7e0852647 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/stat/Summarizer.scala @@ -96,7 +96,7 @@ object Summarizer extends Logging { * - numNonzeros: a vector with the number of non-zeros for each coefficients * - max: the maximum for each coefficient. * - min: the minimum for each coefficient. - * - normL2: the Euclidian norm for each coefficient. + * - normL2: the Euclidean norm for each coefficient. * - normL1: the L1 norm of each coefficient (sum of the absolute values). * @param metrics metrics that can be provided. * @return a builder. @@ -536,7 +536,7 @@ private[ml] object SummaryBuilderImpl extends Logging { } /** - * L2 (Euclidian) norm of each dimension. + * L2 (Euclidean) norm of each dimension. */ def normL2: Vector = { require(requestedMetrics.contains(NormL2)) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala index 4e372702f0c65..c896b1589a936 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/BaggedPoint.scala @@ -33,13 +33,13 @@ import org.apache.spark.util.random.XORShiftRandom * this datum has 1 copy, 0 copies, and 4 copies in the 3 subsamples, respectively. * * @param datum Data instance - * @param subsampleWeights Weight of this instance in each subsampled dataset. - * - * TODO: This does not currently support (Double) weighted instances. Once MLlib has weighted - * dataset support, update. (We store subsampleWeights as Double for this future extension.) + * @param subsampleCounts Number of samples of this instance in each subsampled dataset. + * @param sampleWeight The weight of this instance. */ -private[spark] class BaggedPoint[Datum](val datum: Datum, val subsampleWeights: Array[Double]) - extends Serializable +private[spark] class BaggedPoint[Datum]( + val datum: Datum, + val subsampleCounts: Array[Int], + val sampleWeight: Double = 1.0) extends Serializable private[spark] object BaggedPoint { @@ -52,6 +52,7 @@ private[spark] object BaggedPoint { * @param subsamplingRate Fraction of the training data used for learning decision tree. * @param numSubsamples Number of subsamples of this RDD to take. * @param withReplacement Sampling with/without replacement. + * @param extractSampleWeight A function to get the sample weight of each datum. * @param seed Random seed. * @return BaggedPoint dataset representation. */ @@ -60,12 +61,14 @@ private[spark] object BaggedPoint { subsamplingRate: Double, numSubsamples: Int, withReplacement: Boolean, + extractSampleWeight: (Datum => Double) = (_: Datum) => 1.0, seed: Long = Utils.random.nextLong()): RDD[BaggedPoint[Datum]] = { + // TODO: implement weighted bootstrapping if (withReplacement) { convertToBaggedRDDSamplingWithReplacement(input, subsamplingRate, numSubsamples, seed) } else { if (numSubsamples == 1 && subsamplingRate == 1.0) { - convertToBaggedRDDWithoutSampling(input) + convertToBaggedRDDWithoutSampling(input, extractSampleWeight) } else { convertToBaggedRDDSamplingWithoutReplacement(input, subsamplingRate, numSubsamples, seed) } @@ -82,16 +85,15 @@ private[spark] object BaggedPoint { val rng = new XORShiftRandom rng.setSeed(seed + partitionIndex + 1) instances.map { instance => - val subsampleWeights = new Array[Double](numSubsamples) + val subsampleCounts = new Array[Int](numSubsamples) var subsampleIndex = 0 while (subsampleIndex < numSubsamples) { - val x = rng.nextDouble() - subsampleWeights(subsampleIndex) = { - if (x < subsamplingRate) 1.0 else 0.0 + if (rng.nextDouble() < subsamplingRate) { + subsampleCounts(subsampleIndex) = 1 } subsampleIndex += 1 } - new BaggedPoint(instance, subsampleWeights) + new BaggedPoint(instance, subsampleCounts) } } } @@ -106,20 +108,20 @@ private[spark] object BaggedPoint { val poisson = new PoissonDistribution(subsample) poisson.reseedRandomGenerator(seed + partitionIndex + 1) instances.map { instance => - val subsampleWeights = new Array[Double](numSubsamples) + val subsampleCounts = new Array[Int](numSubsamples) var subsampleIndex = 0 while (subsampleIndex < numSubsamples) { - subsampleWeights(subsampleIndex) = poisson.sample() + subsampleCounts(subsampleIndex) = poisson.sample() subsampleIndex += 1 } - new BaggedPoint(instance, subsampleWeights) + new BaggedPoint(instance, subsampleCounts) } } } private def convertToBaggedRDDWithoutSampling[Datum] ( - input: RDD[Datum]): RDD[BaggedPoint[Datum]] = { - input.map(datum => new BaggedPoint(datum, Array(1.0))) + input: RDD[Datum], + extractSampleWeight: (Datum => Double)): RDD[BaggedPoint[Datum]] = { + input.map(datum => new BaggedPoint(datum, Array(1), extractSampleWeight(datum))) } - } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala index 5aeea1443d499..17ec161f2f509 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DTStatsAggregator.scala @@ -104,16 +104,21 @@ private[spark] class DTStatsAggregator( /** * Update the stats for a given (feature, bin) for ordered features, using the given label. */ - def update(featureIndex: Int, binIndex: Int, label: Double, instanceWeight: Double): Unit = { + def update( + featureIndex: Int, + binIndex: Int, + label: Double, + numSamples: Int, + sampleWeight: Double): Unit = { val i = featureOffsets(featureIndex) + binIndex * statsSize - impurityAggregator.update(allStats, i, label, instanceWeight) + impurityAggregator.update(allStats, i, label, numSamples, sampleWeight) } /** * Update the parent node stats using the given label. */ - def updateParent(label: Double, instanceWeight: Double): Unit = { - impurityAggregator.update(parentStats, 0, label, instanceWeight) + def updateParent(label: Double, numSamples: Int, sampleWeight: Double): Unit = { + impurityAggregator.update(parentStats, 0, label, numSamples, sampleWeight) } /** @@ -127,9 +132,10 @@ private[spark] class DTStatsAggregator( featureOffset: Int, binIndex: Int, label: Double, - instanceWeight: Double): Unit = { + numSamples: Int, + sampleWeight: Double): Unit = { impurityAggregator.update(allStats, featureOffset + binIndex * statsSize, - label, instanceWeight) + label, numSamples, sampleWeight) } /** diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala index 53189e0797b6a..8f8a17171f980 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/DecisionTreeMetadata.scala @@ -21,7 +21,7 @@ import scala.collection.mutable import scala.util.Try import org.apache.spark.internal.Logging -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.tree.TreeEnsembleParams import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ @@ -32,16 +32,20 @@ import org.apache.spark.rdd.RDD /** * Learning and dataset metadata for DecisionTree. * + * @param weightedNumExamples Weighted count of samples in the tree. * @param numClasses For classification: labels can take values {0, ..., numClasses - 1}. * For regression: fixed at 0 (no meaning). * @param maxBins Maximum number of bins, for all features. * @param featureArity Map: categorical feature index to arity. * I.e., the feature takes values in {0, ..., arity - 1}. * @param numBins Number of bins for each feature. + * @param minWeightFractionPerNode The minimum fraction of the total sample weight that must be + * present in a leaf node in order to be considered a valid split. */ private[spark] class DecisionTreeMetadata( val numFeatures: Int, val numExamples: Long, + val weightedNumExamples: Double, val numClasses: Int, val maxBins: Int, val featureArity: Map[Int, Int], @@ -51,6 +55,7 @@ private[spark] class DecisionTreeMetadata( val quantileStrategy: QuantileStrategy, val maxDepth: Int, val minInstancesPerNode: Int, + val minWeightFractionPerNode: Double, val minInfoGain: Double, val numTrees: Int, val numFeaturesPerNode: Int) extends Serializable { @@ -67,6 +72,8 @@ private[spark] class DecisionTreeMetadata( def isContinuous(featureIndex: Int): Boolean = !featureArity.contains(featureIndex) + def minWeightPerNode: Double = minWeightFractionPerNode * weightedNumExamples + /** * Number of splits for the given feature. * For unordered features, there is 1 bin per split. @@ -104,7 +111,7 @@ private[spark] object DecisionTreeMetadata extends Logging { * as well as the number of splits and bins for each feature. */ def buildMetadata( - input: RDD[LabeledPoint], + input: RDD[Instance], strategy: Strategy, numTrees: Int, featureSubsetStrategy: String): DecisionTreeMetadata = { @@ -115,7 +122,11 @@ private[spark] object DecisionTreeMetadata extends Logging { } require(numFeatures > 0, s"DecisionTree requires number of features > 0, " + s"but was given an empty features vector") - val numExamples = input.count() + val (numExamples, weightSum) = input.aggregate((0L, 0.0))( + seqOp = (cw, instance) => (cw._1 + 1L, cw._2 + instance.weight), + combOp = (cw1, cw2) => (cw1._1 + cw2._1, cw1._2 + cw2._2) + ) + val numClasses = strategy.algo match { case Classification => strategy.numClasses case Regression => 0 @@ -206,17 +217,18 @@ private[spark] object DecisionTreeMetadata extends Logging { } } - new DecisionTreeMetadata(numFeatures, numExamples, numClasses, numBins.max, - strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, + new DecisionTreeMetadata(numFeatures, numExamples, weightSum, numClasses, + numBins.max, strategy.categoricalFeaturesInfo, unorderedFeatures.toSet, numBins, strategy.impurity, strategy.quantileCalculationStrategy, strategy.maxDepth, - strategy.minInstancesPerNode, strategy.minInfoGain, numTrees, numFeaturesPerNode) + strategy.minInstancesPerNode, strategy.minWeightFractionPerNode, strategy.minInfoGain, + numTrees, numFeaturesPerNode) } /** * Version of [[DecisionTreeMetadata#buildMetadata]] for DecisionTree. */ def buildMetadata( - input: RDD[LabeledPoint], + input: RDD[Instance], strategy: Strategy): DecisionTreeMetadata = { buildMetadata(input, strategy, numTrees = 1, featureSubsetStrategy = "all") } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala index bd8c9afb5e209..c31334c92e1c9 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/GradientBoostedTrees.scala @@ -230,7 +230,7 @@ private[spark] object GradientBoostedTrees extends Logging { (a, b) => treesIndices.map(idx => a(idx) + b(idx))) .map(_ / dataCount) - broadcastTrees.destroy(blocking = false) + broadcastTrees.destroy() evaluation.toArray } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala index 5b14a63ada4ef..c270e6c0bd5e8 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/NodeIdCache.scala @@ -95,7 +95,7 @@ private[spark] class NodeIdCache( splits: Array[Array[Split]]): Unit = { if (prevNodeIdsForInstances != null) { // Unpersist the previous one if one exists. - prevNodeIdsForInstances.unpersist(false) + prevNodeIdsForInstances.unpersist() } prevNodeIdsForInstances = nodeIdsForInstances @@ -168,11 +168,11 @@ private[spark] class NodeIdCache( } if (nodeIdsForInstances != null) { // Unpersist current one if one exists. - nodeIdsForInstances.unpersist(false) + nodeIdsForInstances.unpersist() } if (prevNodeIdsForInstances != null) { // Unpersist the previous one if one exists. - prevNodeIdsForInstances.unpersist(false) + prevNodeIdsForInstances.unpersist() } } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala index 822abd2d3522d..7921823374edf 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala @@ -24,10 +24,12 @@ import scala.util.Random import org.apache.spark.internal.Logging import org.apache.spark.ml.classification.DecisionTreeClassificationModel -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance +import org.apache.spark.ml.impl.Utils import org.apache.spark.ml.regression.DecisionTreeRegressionModel import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.Instrumentation +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.configuration.{Algo => OldAlgo, Strategy => OldStrategy} import org.apache.spark.mllib.tree.impurity.ImpurityCalculator import org.apache.spark.mllib.tree.model.ImpurityStats @@ -90,6 +92,24 @@ private[spark] object RandomForest extends Logging with Serializable { strategy: OldStrategy, numTrees: Int, featureSubsetStrategy: String, + seed: Long): Array[DecisionTreeModel] = { + val instances = input.map { case LabeledPoint(label, features) => + Instance(label, 1.0, features.asML) + } + run(instances, strategy, numTrees, featureSubsetStrategy, seed, None) + } + + /** + * Train a random forest. + * + * @param input Training data: RDD of `Instance` + * @return an unweighted set of trees + */ + def run( + input: RDD[Instance], + strategy: OldStrategy, + numTrees: Int, + featureSubsetStrategy: String, seed: Long, instr: Option[Instrumentation], prune: Boolean = true, // exposed for testing only, real trees are always pruned @@ -101,9 +121,10 @@ private[spark] object RandomForest extends Logging with Serializable { timer.start("init") - val retaggedInput = input.retag(classOf[LabeledPoint]) + val retaggedInput = input.retag(classOf[Instance]) val metadata = DecisionTreeMetadata.buildMetadata(retaggedInput, strategy, numTrees, featureSubsetStrategy) + instr match { case Some(instrumentation) => instrumentation.logNumFeatures(metadata.numFeatures) @@ -132,7 +153,8 @@ private[spark] object RandomForest extends Logging with Serializable { val withReplacement = numTrees > 1 val baggedInput = BaggedPoint - .convertToBaggedRDD(treeInput, strategy.subsamplingRate, numTrees, withReplacement, seed) + .convertToBaggedRDD(treeInput, strategy.subsamplingRate, numTrees, withReplacement, + (tp: TreePoint) => tp.weight, seed = seed) .persist(StorageLevel.MEMORY_AND_DISK) // depth of the decision tree @@ -172,14 +194,16 @@ private[spark] object RandomForest extends Logging with Serializable { training the same tree in the next iteration. This focus allows us to send fewer trees to workers on each iteration; see topNodesForGroup below. */ - val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + val nodeStack = new mutable.ListBuffer[(Int, LearningNode)] val rng = new Random() rng.setSeed(seed) // Allocate and queue root nodes. val topNodes = Array.fill[LearningNode](numTrees)(LearningNode.emptyNode(nodeIndex = 1)) - Range(0, numTrees).foreach(treeIndex => nodeStack.push((treeIndex, topNodes(treeIndex)))) + for (treeIndex <- 0 until numTrees) { + nodeStack.prepend((treeIndex, topNodes(treeIndex))) + } timer.stop("init") @@ -254,19 +278,21 @@ private[spark] object RandomForest extends Logging with Serializable { * For unordered features, bins correspond to subsets of categories; either the left or right bin * for each subset is updated. * - * @param agg Array storing aggregate calculation, with a set of sufficient statistics for - * each (feature, bin). - * @param treePoint Data point being aggregated. - * @param splits possible splits indexed (numFeatures)(numSplits) - * @param unorderedFeatures Set of indices of unordered features. - * @param instanceWeight Weight (importance) of instance in dataset. + * @param agg Array storing aggregate calculation, with a set of sufficient statistics for + * each (feature, bin). + * @param treePoint Data point being aggregated. + * @param splits Possible splits indexed (numFeatures)(numSplits) + * @param unorderedFeatures Set of indices of unordered features. + * @param numSamples Number of times this instance occurs in the sample. + * @param sampleWeight Weight (importance) of instance in dataset. */ private def mixedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, splits: Array[Array[Split]], unorderedFeatures: Set[Int], - instanceWeight: Double, + numSamples: Int, + sampleWeight: Double, featuresForNode: Option[Array[Int]]): Unit = { val numFeaturesPerNode = if (featuresForNode.nonEmpty) { // Use subsampled features @@ -293,14 +319,15 @@ private[spark] object RandomForest extends Logging with Serializable { var splitIndex = 0 while (splitIndex < numSplits) { if (featureSplits(splitIndex).shouldGoLeft(featureValue, featureSplits)) { - agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, instanceWeight) + agg.featureUpdate(leftNodeFeatureOffset, splitIndex, treePoint.label, numSamples, + sampleWeight) } splitIndex += 1 } } else { // Ordered feature val binIndex = treePoint.binnedFeatures(featureIndex) - agg.update(featureIndexIdx, binIndex, treePoint.label, instanceWeight) + agg.update(featureIndexIdx, binIndex, treePoint.label, numSamples, sampleWeight) } featureIndexIdx += 1 } @@ -314,12 +341,14 @@ private[spark] object RandomForest extends Logging with Serializable { * @param agg Array storing aggregate calculation, with a set of sufficient statistics for * each (feature, bin). * @param treePoint Data point being aggregated. - * @param instanceWeight Weight (importance) of instance in dataset. + * @param numSamples Number of times this instance occurs in the sample. + * @param sampleWeight Weight (importance) of instance in dataset. */ private def orderedBinSeqOp( agg: DTStatsAggregator, treePoint: TreePoint, - instanceWeight: Double, + numSamples: Int, + sampleWeight: Double, featuresForNode: Option[Array[Int]]): Unit = { val label = treePoint.label @@ -329,7 +358,7 @@ private[spark] object RandomForest extends Logging with Serializable { var featureIndexIdx = 0 while (featureIndexIdx < featuresForNode.get.length) { val binIndex = treePoint.binnedFeatures(featuresForNode.get.apply(featureIndexIdx)) - agg.update(featureIndexIdx, binIndex, label, instanceWeight) + agg.update(featureIndexIdx, binIndex, label, numSamples, sampleWeight) featureIndexIdx += 1 } } else { @@ -338,7 +367,7 @@ private[spark] object RandomForest extends Logging with Serializable { var featureIndex = 0 while (featureIndex < numFeatures) { val binIndex = treePoint.binnedFeatures(featureIndex) - agg.update(featureIndex, binIndex, label, instanceWeight) + agg.update(featureIndex, binIndex, label, numSamples, sampleWeight) featureIndex += 1 } } @@ -371,7 +400,7 @@ private[spark] object RandomForest extends Logging with Serializable { nodesForGroup: Map[Int, Array[LearningNode]], treeToNodeToIndexInfo: Map[Int, Map[Int, NodeIndexInfo]], splits: Array[Array[Split]], - nodeStack: mutable.ArrayStack[(Int, LearningNode)], + nodeStack: mutable.ListBuffer[(Int, LearningNode)], timer: TimeTracker = new TimeTracker, nodeIdCache: Option[NodeIdCache] = None): Unit = { @@ -427,14 +456,16 @@ private[spark] object RandomForest extends Logging with Serializable { if (nodeInfo != null) { val aggNodeIndex = nodeInfo.nodeIndexInGroup val featuresForNode = nodeInfo.featureSubset - val instanceWeight = baggedPoint.subsampleWeights(treeIndex) + val numSamples = baggedPoint.subsampleCounts(treeIndex) + val sampleWeight = baggedPoint.sampleWeight if (metadata.unorderedFeatures.isEmpty) { - orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, instanceWeight, featuresForNode) + orderedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, numSamples, sampleWeight, + featuresForNode) } else { mixedBinSeqOp(agg(aggNodeIndex), baggedPoint.datum, splits, - metadata.unorderedFeatures, instanceWeight, featuresForNode) + metadata.unorderedFeatures, numSamples, sampleWeight, featuresForNode) } - agg(aggNodeIndex).updateParent(baggedPoint.datum.label, instanceWeight) + agg(aggNodeIndex).updateParent(baggedPoint.datum.label, numSamples, sampleWeight) } } @@ -594,8 +625,8 @@ private[spark] object RandomForest extends Logging with Serializable { if (!isLeaf) { node.split = Some(split) val childIsLeaf = (LearningNode.indexToLevel(nodeIndex) + 1) == metadata.maxDepth - val leftChildIsLeaf = childIsLeaf || (stats.leftImpurity == 0.0) - val rightChildIsLeaf = childIsLeaf || (stats.rightImpurity == 0.0) + val leftChildIsLeaf = childIsLeaf || (math.abs(stats.leftImpurity) < Utils.EPSILON) + val rightChildIsLeaf = childIsLeaf || (math.abs(stats.rightImpurity) < Utils.EPSILON) node.leftChild = Some(LearningNode(LearningNode.leftChildIndex(nodeIndex), leftChildIsLeaf, ImpurityStats.getEmptyImpurityStats(stats.leftImpurityCalculator))) node.rightChild = Some(LearningNode(LearningNode.rightChildIndex(nodeIndex), @@ -610,10 +641,10 @@ private[spark] object RandomForest extends Logging with Serializable { // enqueue left child and right child if they are not leaves if (!leftChildIsLeaf) { - nodeStack.push((treeIndex, node.leftChild.get)) + nodeStack.prepend((treeIndex, node.leftChild.get)) } if (!rightChildIsLeaf) { - nodeStack.push((treeIndex, node.rightChild.get)) + nodeStack.prepend((treeIndex, node.rightChild.get)) } logDebug("leftChildIndex = " + node.leftChild.get.id + @@ -659,15 +690,20 @@ private[spark] object RandomForest extends Logging with Serializable { stats.impurity } + val leftRawCount = leftImpurityCalculator.rawCount + val rightRawCount = rightImpurityCalculator.rawCount val leftCount = leftImpurityCalculator.count val rightCount = rightImpurityCalculator.count val totalCount = leftCount + rightCount - // If left child or right child doesn't satisfy minimum instances per node, - // then this split is invalid, return invalid information gain stats. - if ((leftCount < metadata.minInstancesPerNode) || - (rightCount < metadata.minInstancesPerNode)) { + val violatesMinInstancesPerNode = (leftRawCount < metadata.minInstancesPerNode) || + (rightRawCount < metadata.minInstancesPerNode) + val violatesMinWeightPerNode = (leftCount < metadata.minWeightPerNode) || + (rightCount < metadata.minWeightPerNode) + // If left child or right child doesn't satisfy minimum weight per node or minimum + // instances per node, then this split is invalid, return invalid information gain stats. + if (violatesMinInstancesPerNode || violatesMinWeightPerNode) { return ImpurityStats.getInvalidImpurityStats(parentImpurityCalculator) } @@ -734,7 +770,8 @@ private[spark] object RandomForest extends Logging with Serializable { // Find best split. val (bestFeatureSplitIndex, bestFeatureGainStats) = Range(0, numSplits).map { case splitIdx => - val leftChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) + val leftChildStats = + binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx) val rightChildStats = binAggregates.getImpurityCalculator(nodeFeatureOffset, numSplits) rightChildStats.subtract(leftChildStats) @@ -876,14 +913,14 @@ private[spark] object RandomForest extends Logging with Serializable { * and for multiclass classification with a high-arity feature, * there is one bin per category. * - * @param input Training data: RDD of [[LabeledPoint]] + * @param input Training data: RDD of [[Instance]] * @param metadata Learning and dataset metadata * @param seed random seed * @return Splits, an Array of [[Split]] * of size (numFeatures, numSplits) */ protected[tree] def findSplits( - input: RDD[LabeledPoint], + input: RDD[Instance], metadata: DecisionTreeMetadata, seed: Long): Array[Array[Split]] = { @@ -898,14 +935,14 @@ private[spark] object RandomForest extends Logging with Serializable { logDebug("fraction of data used for calculating quantiles = " + fraction) input.sample(withReplacement = false, fraction, new XORShiftRandom(seed).nextInt()) } else { - input.sparkContext.emptyRDD[LabeledPoint] + input.sparkContext.emptyRDD[Instance] } findSplitsBySorting(sampledInput, metadata, continuousFeatures) } private def findSplitsBySorting( - input: RDD[LabeledPoint], + input: RDD[Instance], metadata: DecisionTreeMetadata, continuousFeatures: IndexedSeq[Int]): Array[Array[Split]] = { @@ -917,7 +954,8 @@ private[spark] object RandomForest extends Logging with Serializable { input .flatMap { point => - continuousFeatures.map(idx => (idx, point.features(idx))).filter(_._2 != 0.0) + continuousFeatures.map(idx => (idx, (point.weight, point.features(idx)))) + .filter(_._2._2 != 0.0) }.groupByKey(numPartitions) .map { case (idx, samples) => val thresholds = findSplitsForContinuousFeature(samples, metadata, idx) @@ -982,7 +1020,7 @@ private[spark] object RandomForest extends Logging with Serializable { * could be different from the specified `numSplits`. * The `numSplits` attribute in the `DecisionTreeMetadata` class will be set accordingly. * - * @param featureSamples feature values of each sample + * @param featureSamples feature values and sample weights of each sample * @param metadata decision tree metadata * NOTE: `metadata.numbins` will be changed accordingly * if there are not enough splits to be found @@ -990,7 +1028,7 @@ private[spark] object RandomForest extends Logging with Serializable { * @return array of split thresholds */ private[tree] def findSplitsForContinuousFeature( - featureSamples: Iterable[Double], + featureSamples: Iterable[(Double, Double)], metadata: DecisionTreeMetadata, featureIndex: Int): Array[Double] = { require(metadata.isContinuous(featureIndex), @@ -1002,19 +1040,27 @@ private[spark] object RandomForest extends Logging with Serializable { val numSplits = metadata.numSplits(featureIndex) // get count for each distinct value except zero value - val partNumSamples = featureSamples.size - val partValueCountMap = scala.collection.mutable.Map[Double, Int]() - featureSamples.foreach { x => - partValueCountMap(x) = partValueCountMap.getOrElse(x, 0) + 1 + val partValueCountMap = mutable.Map[Double, Double]() + var partNumSamples = 0.0 + var unweightedNumSamples = 0.0 + featureSamples.foreach { case (sampleWeight, feature) => + partValueCountMap(feature) = partValueCountMap.getOrElse(feature, 0.0) + sampleWeight + partNumSamples += sampleWeight + unweightedNumSamples += 1.0 } // Calculate the expected number of samples for finding splits - val numSamples = (samplesFractionForFindSplits(metadata) * metadata.numExamples).toInt + val weightedNumSamples = samplesFractionForFindSplits(metadata) * + metadata.weightedNumExamples + // scale tolerance by number of samples with constant factor + // Note: constant factor was tuned by running some tests where there were no zero + // feature values and validating we are never within tolerance + val tolerance = Utils.EPSILON * unweightedNumSamples * 100 // add expected zero value count and get complete statistics - val valueCountMap: Map[Double, Int] = if (numSamples - partNumSamples > 0) { - partValueCountMap.toMap + (0.0 -> (numSamples - partNumSamples)) + val valueCountMap = if (weightedNumSamples - partNumSamples > tolerance) { + partValueCountMap + (0.0 -> (weightedNumSamples - partNumSamples)) } else { - partValueCountMap.toMap + partValueCountMap } // sort distinct values @@ -1031,7 +1077,7 @@ private[spark] object RandomForest extends Logging with Serializable { .toArray } else { // stride between splits - val stride: Double = numSamples.toDouble / (numSplits + 1) + val stride: Double = weightedNumSamples / (numSplits + 1) logDebug("stride = " + stride) // iterate `valueCount` to find splits @@ -1087,7 +1133,7 @@ private[spark] object RandomForest extends Logging with Serializable { * The feature indices are None if not subsampling features. */ private[tree] def selectNodesToSplit( - nodeStack: mutable.ArrayStack[(Int, LearningNode)], + nodeStack: mutable.ListBuffer[(Int, LearningNode)], maxMemoryUsage: Long, metadata: DecisionTreeMetadata, rng: Random): (Map[Int, Array[LearningNode]], Map[Int, Map[Int, NodeIndexInfo]]) = { @@ -1102,7 +1148,7 @@ private[spark] object RandomForest extends Logging with Serializable { // so we allow one iteration if memUsage == 0. var groupDone = false while (nodeStack.nonEmpty && !groupDone) { - val (treeIndex, node) = nodeStack.top + val (treeIndex, node) = nodeStack.head // Choose subset of features for node (if subsampling). val featureSubset: Option[Array[Int]] = if (metadata.subsamplingFeatures) { Some(SamplingUtils.reservoirSampleAndCount(Range(0, @@ -1113,7 +1159,7 @@ private[spark] object RandomForest extends Logging with Serializable { // Check if enough memory remains to add this node to the group. val nodeMemUsage = RandomForest.aggregateSizeForNode(metadata, featureSubset) * 8L if (memUsage + nodeMemUsage <= maxMemoryUsage || memUsage == 0) { - nodeStack.pop() + nodeStack.remove(0) mutableNodesForGroup.getOrElseUpdate(treeIndex, new mutable.ArrayBuffer[LearningNode]()) += node mutableTreeToNodeToIndexInfo diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala index a6ac64a0463cc..72440b2c57aa1 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/impl/TreePoint.scala @@ -17,7 +17,7 @@ package org.apache.spark.ml.tree.impl -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.tree.{ContinuousSplit, Split} import org.apache.spark.rdd.RDD @@ -36,10 +36,12 @@ import org.apache.spark.rdd.RDD * @param label Label from LabeledPoint * @param binnedFeatures Binned feature values. * Same length as LabeledPoint.features, but values are bin indices. + * @param weight Sample weight for this TreePoint. */ -private[spark] class TreePoint(val label: Double, val binnedFeatures: Array[Int]) - extends Serializable { -} +private[spark] class TreePoint( + val label: Double, + val binnedFeatures: Array[Int], + val weight: Double) extends Serializable private[spark] object TreePoint { @@ -52,7 +54,7 @@ private[spark] object TreePoint { * @return TreePoint dataset representation */ def convertToTreeRDD( - input: RDD[LabeledPoint], + input: RDD[Instance], splits: Array[Array[Split]], metadata: DecisionTreeMetadata): RDD[TreePoint] = { // Construct arrays for featureArity for efficiency in the inner loop. @@ -82,18 +84,18 @@ private[spark] object TreePoint { * for categorical features. */ private def labeledPointToTreePoint( - labeledPoint: LabeledPoint, + instance: Instance, thresholds: Array[Array[Double]], featureArity: Array[Int]): TreePoint = { - val numFeatures = labeledPoint.features.size + val numFeatures = instance.features.size val arr = new Array[Int](numFeatures) var featureIndex = 0 while (featureIndex < numFeatures) { arr(featureIndex) = - findBin(featureIndex, labeledPoint, featureArity(featureIndex), thresholds(featureIndex)) + findBin(featureIndex, instance, featureArity(featureIndex), thresholds(featureIndex)) featureIndex += 1 } - new TreePoint(labeledPoint.label, arr) + new TreePoint(instance.label, arr, instance.weight) } /** @@ -106,10 +108,10 @@ private[spark] object TreePoint { */ private def findBin( featureIndex: Int, - labeledPoint: LabeledPoint, + instance: Instance, featureArity: Int, thresholds: Array[Double]): Int = { - val featureValue = labeledPoint.features(featureIndex) + val featureValue = instance.features(featureIndex) if (featureArity == 0) { val idx = java.util.Arrays.binarySearch(thresholds, featureValue) @@ -125,7 +127,7 @@ private[spark] object TreePoint { s"DecisionTree given invalid data:" + s" Feature $featureIndex is categorical with values in {0,...,${featureArity - 1}," + s" but a data point gives it value $featureValue.\n" + - " Bad data point: " + labeledPoint.toString) + s" Bad data point: $instance") } featureValue.toInt } diff --git a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala index 4aa4c3617e7fd..e95c55f6048fb 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tree/treeModels.scala @@ -135,7 +135,7 @@ private[ml] object TreeEnsembleModel { * - Average over trees: * - importance(feature j) = sum (over nodes which split on feature j) of the gain, * where gain is scaled by the number of instances passing through node - * - Normalize importances for tree to sum to 1. + * - Normalize importances for tree to sum to 1 (only if `perTreeNormalization` is `true`). * - Normalize feature importance vector to sum to 1. * * References: @@ -145,9 +145,15 @@ private[ml] object TreeEnsembleModel { * @param numFeatures Number of features in model (even if not all are explicitly used by * the model). * If -1, then numFeatures is set based on the max feature index in all trees. + * @param perTreeNormalization By default this is set to `true` and it means that the importances + * of each tree are normalized before being summed. If set to `false`, + * the normalization is skipped. * @return Feature importance values, of length numFeatures. */ - def featureImportances[M <: DecisionTreeModel](trees: Array[M], numFeatures: Int): Vector = { + def featureImportances[M <: DecisionTreeModel]( + trees: Array[M], + numFeatures: Int, + perTreeNormalization: Boolean = true): Vector = { val totalImportances = new OpenHashMap[Int, Double]() trees.foreach { tree => // Aggregate feature importance vector for this tree @@ -155,10 +161,19 @@ private[ml] object TreeEnsembleModel { computeFeatureImportance(tree.rootNode, importances) // Normalize importance vector for this tree, and add it to total. // TODO: In the future, also support normalizing by tree.rootNode.impurityStats.count? - val treeNorm = importances.map(_._2).sum + val treeNorm = if (perTreeNormalization) { + importances.map(_._2).sum + } else { + // We won't use it + Double.NaN + } if (treeNorm != 0) { importances.foreach { case (idx, impt) => - val normImpt = impt / treeNorm + val normImpt = if (perTreeNormalization) { + impt / treeNorm + } else { + impt + } totalImportances.changeValue(idx, normImpt, _ + normImpt) } } @@ -282,6 +297,7 @@ private[ml] object DecisionTreeModelReadWrite { * * @param id Index used for tree reconstruction. Indices follow a pre-order traversal. * @param impurityStats Stats array. Impurity type is stored in metadata. + * @param rawCount The unweighted number of samples falling in this node. * @param gain Gain, or arbitrary value if leaf node. * @param leftChild Left child index, or arbitrary value if leaf node. * @param rightChild Right child index, or arbitrary value if leaf node. @@ -292,6 +308,7 @@ private[ml] object DecisionTreeModelReadWrite { prediction: Double, impurity: Double, impurityStats: Array[Double], + rawCount: Long, gain: Double, leftChild: Int, rightChild: Int, @@ -311,11 +328,12 @@ private[ml] object DecisionTreeModelReadWrite { val (leftNodeData, leftIdx) = build(n.leftChild, id + 1) val (rightNodeData, rightIdx) = build(n.rightChild, leftIdx + 1) val thisNodeData = NodeData(id, n.prediction, n.impurity, n.impurityStats.stats, - n.gain, leftNodeData.head.id, rightNodeData.head.id, SplitData(n.split)) + n.impurityStats.rawCount, n.gain, leftNodeData.head.id, rightNodeData.head.id, + SplitData(n.split)) (thisNodeData +: (leftNodeData ++ rightNodeData), rightIdx) case _: LeafNode => (Seq(NodeData(id, node.prediction, node.impurity, node.impurityStats.stats, - -1.0, -1, -1, SplitData(-1, Array.empty[Double], -1))), + node.impurityStats.rawCount, -1.0, -1, -1, SplitData(-1, Array.empty[Double], -1))), id) } } @@ -360,7 +378,8 @@ private[ml] object DecisionTreeModelReadWrite { // traversal, this guarantees that child nodes will be built before parent nodes. val finalNodes = new Array[Node](nodes.length) nodes.reverseIterator.foreach { case n: NodeData => - val impurityStats = ImpurityCalculator.getCalculator(impurityType, n.impurityStats) + val impurityStats = + ImpurityCalculator.getCalculator(impurityType, n.impurityStats, n.rawCount) val node = if (n.leftChild != -1) { val leftChild = finalNodes(n.leftChild) val rightChild = finalNodes(n.rightChild) 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 00157fe63af41..c1e44e94af477 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 @@ -37,46 +37,61 @@ import org.apache.spark.sql.types.{DataType, DoubleType, StructType} * Note: Marked as private and DeveloperApi since this may be made public in the future. */ private[ml] trait DecisionTreeParams extends PredictorParams - with HasCheckpointInterval with HasSeed { + with HasCheckpointInterval with HasSeed with HasWeightCol { /** - * Maximum depth of the tree (>= 0). + * Maximum depth of the tree (nonnegative). * E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes. * (default = 5) * @group param */ final val maxDepth: IntParam = - new IntParam(this, "maxDepth", "Maximum depth of the tree. (>= 0)" + + new IntParam(this, "maxDepth", "Maximum depth of the tree. (Nonnegative)" + " E.g., depth 0 means 1 leaf node; depth 1 means 1 internal node + 2 leaf nodes.", ParamValidators.gtEq(0)) /** * Maximum number of bins used for discretizing continuous features and for choosing how to split * on features at each node. More bins give higher granularity. - * Must be >= 2 and >= number of categories in any categorical feature. + * Must be at least 2 and at least number of categories in any categorical feature. * (default = 32) * @group param */ final val maxBins: IntParam = new IntParam(this, "maxBins", "Max number of bins for" + - " discretizing continuous features. Must be >=2 and >= number of categories for any" + - " categorical feature.", ParamValidators.gtEq(2)) + " discretizing continuous features. Must be at least 2 and at least number of categories" + + " for any categorical feature.", ParamValidators.gtEq(2)) /** * Minimum number of instances each child must have after split. * If a split causes the left or right child to have fewer than minInstancesPerNode, * the split will be discarded as invalid. - * Should be >= 1. + * Must be at least 1. * (default = 1) * @group param */ final val minInstancesPerNode: IntParam = new IntParam(this, "minInstancesPerNode", "Minimum" + " number of instances each child must have after split. If a split causes the left or right" + " child to have fewer than minInstancesPerNode, the split will be discarded as invalid." + - " Should be >= 1.", ParamValidators.gtEq(1)) + " Must be at least 1.", ParamValidators.gtEq(1)) + + /** + * Minimum fraction of the weighted sample count that each child must have after split. + * If a split causes the fraction of the total weight in the left or right child to be less than + * minWeightFractionPerNode, the split will be discarded as invalid. + * Should be in the interval [0.0, 0.5). + * (default = 0.0) + * @group param + */ + final val minWeightFractionPerNode: DoubleParam = new DoubleParam(this, + "minWeightFractionPerNode", "Minimum fraction of the weighted sample count that each child " + + "must have after split. If a split causes the fraction of the total weight in the left or " + + "right child to be less than minWeightFractionPerNode, the split will be discarded as " + + "invalid. Should be in interval [0.0, 0.5)", + ParamValidators.inRange(0.0, 0.5, lowerInclusive = true, upperInclusive = false)) /** * Minimum information gain for a split to be considered at a tree node. - * Should be >= 0.0. + * Should be at least 0.0. * (default = 0.0) * @group param */ @@ -107,83 +122,31 @@ private[ml] trait DecisionTreeParams extends PredictorParams " algorithm will cache node IDs for each instance. Caching can speed up training of deeper" + " trees.") - setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, minInfoGain -> 0.0, - maxMemoryInMB -> 256, cacheNodeIds -> false, checkpointInterval -> 10) - - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMaxDepth(value: Int): this.type = set(maxDepth, value) + setDefault(maxDepth -> 5, maxBins -> 32, minInstancesPerNode -> 1, + minWeightFractionPerNode -> 0.0, minInfoGain -> 0.0, maxMemoryInMB -> 256, + cacheNodeIds -> false, checkpointInterval -> 10) /** @group getParam */ final def getMaxDepth: Int = $(maxDepth) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMaxBins(value: Int): this.type = set(maxBins, value) - /** @group getParam */ final def getMaxBins: Int = $(maxBins) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMinInstancesPerNode(value: Int): this.type = set(minInstancesPerNode, value) - /** @group getParam */ final def getMinInstancesPerNode: Int = $(minInstancesPerNode) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMinInfoGain(value: Double): this.type = set(minInfoGain, value) + /** @group getParam */ + final def getMinWeightFractionPerNode: Double = $(minWeightFractionPerNode) /** @group getParam */ final def getMinInfoGain: Double = $(minInfoGain) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setSeed(value: Long): this.type = set(seed, value) - - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group expertSetParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMaxMemoryInMB(value: Int): this.type = set(maxMemoryInMB, value) - /** @group expertGetParam */ final def getMaxMemoryInMB: Int = $(maxMemoryInMB) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group expertSetParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setCacheNodeIds(value: Boolean): this.type = set(cacheNodeIds, value) - /** @group expertGetParam */ final def getCacheNodeIds: Boolean = $(cacheNodeIds) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setCheckpointInterval(value: Int): this.type = set(checkpointInterval, value) - /** (private[ml]) Create a Strategy instance to use with the old API. */ private[ml] def getOldStrategy( categoricalFeatures: Map[Int, Int], @@ -199,6 +162,7 @@ private[ml] trait DecisionTreeParams extends PredictorParams strategy.maxMemoryInMB = getMaxMemoryInMB strategy.minInfoGain = getMinInfoGain strategy.minInstancesPerNode = getMinInstancesPerNode + strategy.minWeightFractionPerNode = getMinWeightFractionPerNode strategy.useNodeIdCache = getCacheNodeIds strategy.numClasses = numClasses strategy.categoricalFeaturesInfo = categoricalFeatures @@ -226,13 +190,6 @@ private[ml] trait TreeClassifierParams extends Params { setDefault(impurity -> "gini") - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setImpurity(value: String): this.type = set(impurity, value) - /** @group getParam */ final def getImpurity: String = $(impurity).toLowerCase(Locale.ROOT) @@ -258,11 +215,7 @@ private[ml] object TreeClassifierParams { private[ml] trait DecisionTreeClassifierParams extends DecisionTreeParams with TreeClassifierParams -/** - * Parameters for Decision Tree-based regression algorithms. - */ -private[ml] trait TreeRegressorParams extends Params { - +private[ml] trait HasVarianceImpurity extends Params { /** * Criterion used for information gain calculation (case-insensitive). * Supported: "variance". @@ -271,19 +224,12 @@ private[ml] trait TreeRegressorParams extends Params { */ final val impurity: Param[String] = new Param[String](this, "impurity", "Criterion used for" + " information gain calculation (case-insensitive). Supported options:" + - s" ${TreeRegressorParams.supportedImpurities.mkString(", ")}", + s" ${HasVarianceImpurity.supportedImpurities.mkString(", ")}", (value: String) => - TreeRegressorParams.supportedImpurities.contains(value.toLowerCase(Locale.ROOT))) + HasVarianceImpurity.supportedImpurities.contains(value.toLowerCase(Locale.ROOT))) setDefault(impurity -> "variance") - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setImpurity(value: String): this.type = set(impurity, value) - /** @group getParam */ final def getImpurity: String = $(impurity).toLowerCase(Locale.ROOT) @@ -299,12 +245,17 @@ private[ml] trait TreeRegressorParams extends Params { } } -private[ml] object TreeRegressorParams { +private[ml] object HasVarianceImpurity { // These options should be lowercase. final val supportedImpurities: Array[String] = Array("variance").map(_.toLowerCase(Locale.ROOT)) } +/** + * Parameters for Decision Tree-based regression algorithms. + */ +private[ml] trait TreeRegressorParams extends HasVarianceImpurity + private[ml] trait DecisionTreeRegressorParams extends DecisionTreeParams with TreeRegressorParams with HasVarianceCol { @@ -345,13 +296,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { setDefault(subsamplingRate -> 1.0) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setSubsamplingRate(value: Double): this.type = set(subsamplingRate, value) - /** @group getParam */ final def getSubsamplingRate: Double = $(subsamplingRate) @@ -372,7 +316,7 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { * Supported options: * - "auto": Choose automatically for task: * If numTrees == 1, set to "all." - * If numTrees > 1 (forest), set to "sqrt" for classification and + * If numTrees greater than 1 (forest), set to "sqrt" for classification and * to "onethird" for regression. * - "all": use all features * - "onethird": use 1/3 of the features @@ -405,13 +349,6 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { setDefault(featureSubsetStrategy -> "auto") - /** - * @deprecated This method is deprecated and will be removed in 3.0.0 - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setFeatureSubsetStrategy(value: String): this.type = set(featureSubsetStrategy, value) - /** @group getParam */ final def getFeatureSubsetStrategy: String = $(featureSubsetStrategy).toLowerCase(Locale.ROOT) } @@ -424,8 +361,8 @@ private[ml] trait TreeEnsembleParams extends DecisionTreeParams { private[ml] trait RandomForestParams extends TreeEnsembleParams { /** - * Number of trees to train (>= 1). - * If 1, then no bootstrapping is used. If > 1, then bootstrapping is done. + * Number of trees to train (at least 1). + * If 1, then no bootstrapping is used. If greater than 1, then bootstrapping is done. * TODO: Change to always do bootstrapping (simpler). SPARK-7130 * (default = 20) * @@ -434,18 +371,12 @@ private[ml] trait RandomForestParams extends TreeEnsembleParams { * are a bit different. * @group param */ - final val numTrees: IntParam = new IntParam(this, "numTrees", "Number of trees to train (>= 1)", + final val numTrees: IntParam = + new IntParam(this, "numTrees", "Number of trees to train (at least 1)", ParamValidators.gtEq(1)) setDefault(numTrees -> 20) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setNumTrees(value: Int): this.type = set(numTrees, value) - /** @group getParam */ final def getNumTrees: Int = $(numTrees) } @@ -490,13 +421,6 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS @Since("2.4.0") final def getValidationTol: Double = $(validationTol) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setMaxIter(value: Int): this.type = set(maxIter, value) - /** * Param for Step size (a.k.a. learning rate) in interval (0, 1] for shrinking * the contribution of each estimator. @@ -507,13 +431,6 @@ private[ml] trait GBTParams extends TreeEnsembleParams with HasMaxIter with HasS "(a.k.a. learning rate) in interval (0, 1] for shrinking the contribution of each estimator.", ParamValidators.inRange(0, 1, lowerInclusive = false, upperInclusive = true)) - /** - * @deprecated This method is deprecated and will be removed in 3.0.0. - * @group setParam - */ - @deprecated("This method is deprecated and will be removed in 3.0.0.", "2.1.0") - def setStepSize(value: Double): this.type = set(stepSize, value) - setDefault(maxIter -> 20, stepSize -> 0.1, validationTol -> 0.01) setDefault(featureSubsetStrategy -> "all") @@ -538,7 +455,7 @@ private[ml] object GBTClassifierParams { Array("logistic").map(_.toLowerCase(Locale.ROOT)) } -private[ml] trait GBTClassifierParams extends GBTParams with TreeClassifierParams { +private[ml] trait GBTClassifierParams extends GBTParams with HasVarianceImpurity { /** * Loss function which GBT tries to minimize. (case-insensitive) diff --git a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala index 135828815504a..6d46ea0adcc9a 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/tuning/ValidatorParams.scala @@ -140,8 +140,8 @@ private[ml] object ValidatorParams { "value" -> compact(render(JString(relativePath))), "isJson" -> compact(render(JBool(false)))) case _: MLWritable => - throw new NotImplementedError("ValidatorParams.saveImpl does not handle parameters " + - "of type: MLWritable that are not DefaultParamsWritable") + throw new UnsupportedOperationException("ValidatorParams.saveImpl does not handle" + + " parameters of type: MLWritable that are not DefaultParamsWritable") case _ => Map("parent" -> p.parent, "name" -> p.name, "value" -> p.jsonEncode(v), "isJson" -> compact(render(JBool(true)))) diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala b/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala new file mode 100644 index 0000000000000..edb0208144e10 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/ml/util/HasTrainingSummary.scala @@ -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. + */ + +package org.apache.spark.ml.util + +import org.apache.spark.SparkException +import org.apache.spark.annotation.Since + + +/** + * Trait for models that provides Training summary. + * + * @tparam T Summary instance type + */ +@Since("3.0.0") +private[ml] trait HasTrainingSummary[T] { + + private[ml] final var trainingSummary: Option[T] = None + + /** Indicates whether a training summary exists for this model instance. */ + @Since("3.0.0") + def hasSummary: Boolean = trainingSummary.isDefined + + /** + * Gets summary of model on training set. An exception is + * thrown if if `hasSummary` is false. + */ + @Since("3.0.0") + def summary: T = trainingSummary.getOrElse { + throw new SparkException( + s"No training summary available for this ${this.getClass.getSimpleName}") + } + + private[ml] def setSummary(summary: Option[T]): this.type = { + this.trainingSummary = summary + this + } +} diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala index 49654918bd8f8..8cd4a7ca9493b 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/Instrumentation.scala @@ -17,6 +17,7 @@ package org.apache.spark.ml.util +import java.io.{PrintWriter, StringWriter} import java.util.UUID import scala.util.{Failure, Success, Try} @@ -27,17 +28,18 @@ import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ import org.apache.spark.internal.Logging -import org.apache.spark.ml.PipelineStage +import org.apache.spark.ml.{MLEvents, PipelineStage} import org.apache.spark.ml.param.{Param, Params} import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset import org.apache.spark.util.Utils /** - * A small wrapper that defines a training session for an estimator, and some methods to log - * useful information during this session. + * A small wrapper that defines a training session for an estimator, some methods to log + * useful information during this session, and some methods to send + * [[org.apache.spark.ml.MLEvent]]. */ -private[spark] class Instrumentation private () extends Logging { +private[spark] class Instrumentation private () extends Logging with MLEvents { private val id = UUID.randomUUID() private val shortId = id.toString.take(8) @@ -160,8 +162,9 @@ private[spark] class Instrumentation private () extends Logging { * Logs an exception raised during a training session. */ def logFailure(e: Throwable): Unit = { - val msg = e.getStackTrace.mkString("\n") - super.logError(msg) + val msg = new StringWriter() + e.printStackTrace(new PrintWriter(msg)) + super.logError(msg.toString) } } 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 72a60e04360d6..ce8f3464b0a54 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, InterfaceStability, Since} +import org.apache.spark.annotation.{DeveloperApi, Since, Unstable} import org.apache.spark.internal.Logging import org.apache.spark.ml._ import org.apache.spark.ml.classification.{OneVsRest, OneVsRestModel} @@ -47,18 +47,6 @@ import org.apache.spark.util.{Utils, VersionUtils} private[util] sealed trait BaseReadWrite { private var optionSparkSession: Option[SparkSession] = None - /** - * Sets the Spark SQLContext to use for saving/loading. - * - * @deprecated Use session instead. This method will be removed in 3.0.0. - */ - @Since("1.6.0") - @deprecated("Use session instead. This method will be removed in 3.0.0.", "2.0.0") - def context(sqlContext: SQLContext): this.type = { - optionSparkSession = Option(sqlContext.sparkSession) - this - } - /** * Sets the Spark Session to use for saving/loading. */ @@ -96,7 +84,7 @@ private[util] sealed trait BaseReadWrite { * * @since 2.4.0 */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") trait MLWriterFormat { /** @@ -120,7 +108,7 @@ trait MLWriterFormat { * * @since 2.4.0 */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") trait MLFormatRegister extends MLWriterFormat { /** @@ -215,16 +203,12 @@ abstract class MLWriter extends BaseReadWrite with Logging { // override for Java compatibility @Since("1.6.0") override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) - - // override for Java compatibility - @Since("1.6.0") - override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) } /** * A ML Writer which delegates based on the requested format. */ -@InterfaceStability.Unstable +@Unstable @Since("2.4.0") class GeneralMLWriter(stage: PipelineStage) extends MLWriter with Logging { private var source: String = "internal" @@ -272,7 +256,7 @@ class GeneralMLWriter(stage: PipelineStage) extends MLWriter with Logging { s"Multiple writers found for $source+$stageName, try using the class name of the writer") } if (classOf[MLWriterFormat].isAssignableFrom(writerCls)) { - val writer = writerCls.newInstance().asInstanceOf[MLWriterFormat] + val writer = writerCls.getConstructor().newInstance().asInstanceOf[MLWriterFormat] writer.write(path, sparkSession, optionMap, stage) } else { throw new SparkException(s"ML source $source is not a valid MLWriterFormat") @@ -281,9 +265,6 @@ class GeneralMLWriter(stage: PipelineStage) extends MLWriter with Logging { // override for Java compatibility override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) - - // override for Java compatibility - override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) } /** @@ -310,7 +291,7 @@ trait MLWritable { * Trait for classes that provide `GeneralMLWriter`. */ @Since("2.4.0") -@InterfaceStability.Unstable +@Unstable trait GeneralMLWritable extends MLWritable { /** * Returns an `MLWriter` instance for this ML instance. @@ -352,9 +333,6 @@ abstract class MLReader[T] extends BaseReadWrite { // override for Java compatibility override def session(sparkSession: SparkSession): this.type = super.session(sparkSession) - - // override for Java compatibility - override def context(sqlContext: SQLContext): this.type = super.session(sqlContext.sparkSession) } /** @@ -646,10 +624,17 @@ private[ml] object DefaultParamsReader { * Load a `Params` instance from the given path, and return it. * This assumes the instance implements [[MLReadable]]. */ - def loadParamsInstance[T](path: String, sc: SparkContext): T = { + def loadParamsInstance[T](path: String, sc: SparkContext): T = + loadParamsInstanceReader(path, sc).load(path) + + /** + * Load a `Params` instance reader from the given path, and return it. + * This assumes the instance implements [[MLReadable]]. + */ + def loadParamsInstanceReader[T](path: String, sc: SparkContext): MLReader[T] = { val metadata = DefaultParamsReader.loadMetadata(path, sc) val cls = Utils.classForName(metadata.className) - cls.getMethod("read").invoke(null).asInstanceOf[MLReader[T]].load(path) + cls.getMethod("read").invoke(null).asInstanceOf[MLReader[T]] } } diff --git a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala index e539deca4b036..ca23e6b552400 100644 --- a/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala +++ b/mllib/src/main/scala/org/apache/spark/ml/util/stopwatches.scala @@ -17,6 +17,8 @@ package org.apache.spark.ml.util +import java.util.concurrent.TimeUnit + import scala.collection.mutable import org.apache.spark.SparkContext @@ -73,7 +75,7 @@ private[spark] abstract class Stopwatch extends Serializable { /** * Gets the current time in milliseconds. */ - protected def now: Long = System.currentTimeMillis() + protected def now: Long = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) /** * Adds input duration to total elapsed time. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index db3f074ecfbac..4617073f9decd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -23,7 +23,6 @@ import java.nio.charset.StandardCharsets import java.util.{ArrayList => JArrayList, List => JList, Map => JMap} import scala.collection.JavaConverters._ -import scala.language.existentials import scala.reflect.ClassTag import net.razorvine.pickle._ @@ -54,6 +53,7 @@ import org.apache.spark.mllib.tree.model.{DecisionTreeModel, GradientBoostedTree import org.apache.spark.mllib.util.{LinearDataGenerator, MLUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SparkSession} +import org.apache.spark.sql.types.LongType import org.apache.spark.storage.StorageLevel import org.apache.spark.util.Utils @@ -98,7 +98,7 @@ private[python] class PythonMLLibAPI extends Serializable { List(model.weights, model.intercept).map(_.asInstanceOf[Object]).asJava } } finally { - data.rdd.unpersist(blocking = false) + data.rdd.unpersist() } } @@ -336,7 +336,7 @@ private[python] class PythonMLLibAPI extends Serializable { val model = isotonicRegressionAlg.run(input) List[AnyRef](model.boundaryVector, model.predictionVector).asJava } finally { - data.rdd.unpersist(blocking = false) + data.rdd.unpersist() } } @@ -366,7 +366,7 @@ private[python] class PythonMLLibAPI extends Serializable { try { kMeansAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK)) } finally { - data.rdd.unpersist(blocking = false) + data.rdd.unpersist() } } @@ -411,7 +411,7 @@ private[python] class PythonMLLibAPI extends Serializable { try { new GaussianMixtureModelWrapper(gmmAlg.run(data.rdd.persist(StorageLevel.MEMORY_AND_DISK))) } finally { - data.rdd.unpersist(blocking = false) + data.rdd.unpersist() } } @@ -704,7 +704,7 @@ private[python] class PythonMLLibAPI extends Serializable { val model = word2vec.fit(dataJRDD.rdd.persist(StorageLevel.MEMORY_AND_DISK_SER)) new Word2VecModelWrapper(model) } finally { - dataJRDD.rdd.unpersist(blocking = false) + dataJRDD.rdd.unpersist() } } @@ -742,7 +742,7 @@ private[python] class PythonMLLibAPI extends Serializable { try { DecisionTree.train(data.rdd.persist(StorageLevel.MEMORY_AND_DISK), strategy) } finally { - data.rdd.unpersist(blocking = false) + data.rdd.unpersist() } } @@ -783,7 +783,7 @@ private[python] class PythonMLLibAPI extends Serializable { RandomForest.trainRegressor(cached, strategy, numTrees, featureSubsetStrategy, intSeed) } } finally { - cached.unpersist(blocking = false) + cached.unpersist() } } @@ -814,7 +814,7 @@ private[python] class PythonMLLibAPI extends Serializable { try { GradientBoostedTrees.train(cached, boostingStrategy) } finally { - cached.unpersist(blocking = false) + cached.unpersist() } } @@ -1143,12 +1143,21 @@ private[python] class PythonMLLibAPI extends Serializable { new RowMatrix(rows.rdd, numRows, numCols) } + def createRowMatrix(df: DataFrame, numRows: Long, numCols: Int): RowMatrix = { + require(df.schema.length == 1 && df.schema.head.dataType.getClass == classOf[VectorUDT], + "DataFrame must have a single vector type column") + new RowMatrix(df.rdd.map { case Row(vector: Vector) => vector }, numRows, numCols) + } + /** * Wrapper around IndexedRowMatrix constructor. */ def createIndexedRowMatrix(rows: DataFrame, numRows: Long, numCols: Int): IndexedRowMatrix = { // We use DataFrames for serialization of IndexedRows from Python, // so map each Row in the DataFrame back to an IndexedRow. + require(rows.schema.length == 2 && rows.schema.head.dataType == LongType && + rows.schema(1).dataType.getClass == classOf[VectorUDT], + "DataFrame must consist of a long type index column and a vector type column") val indexedRows = rows.rdd.map { case Row(index: Long, vector: Vector) => IndexedRow(index, vector) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 4b650000736e2..d86aa01c9195a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -163,8 +163,6 @@ class LogisticRegressionModel @Since("1.3.0") ( numFeatures, numClasses, weights, intercept, threshold) } - override protected def formatVersion: String = "1.0" - override def toString: String = { s"${super.toString}, numClasses = ${numClasses}, threshold = ${threshold.getOrElse("None")}" } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 9e8774732efe6..79bb4adac88f5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -83,7 +83,7 @@ class NaiveBayesModel private[spark] ( (Option(thetaMinusNegTheta), Option(negTheta.multiply(ones))) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: $modelType.") + throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } @Since("1.0.0") @@ -170,8 +170,6 @@ class NaiveBayesModel private[spark] ( val data = NaiveBayesModel.SaveLoadV2_0.Data(labels, pi, theta, modelType) NaiveBayesModel.SaveLoadV2_0.save(sc, path, data) } - - override protected def formatVersion: String = "2.0" } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 5fb04ed0ee9a2..087c2c2639831 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -85,8 +85,6 @@ class SVMModel @Since("1.1.0") ( numFeatures = weights.size, numClasses = 2, weights, intercept, threshold) } - override protected def formatVersion: String = "1.0" - override def toString: String = { s"${super.toString}, numClasses = 2, threshold = ${threshold.getOrElse("None")}" } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala index 80ab8eb9bc8b0..b4a31d72f94b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeans.scala @@ -217,7 +217,7 @@ class BisectingKMeans private ( newClusterCenters = newClusters.mapValues(_.center).map(identity) } if (preIndices != null) { - preIndices.unpersist(false) + preIndices.unpersist() } preIndices = indices indices = updateAssignments(assignments, divisibleIndices, newClusterCenters, dMeasure).keys @@ -234,15 +234,16 @@ class BisectingKMeans private ( level += 1 } if (preIndices != null) { - preIndices.unpersist(false) + preIndices.unpersist() } if (indices != null) { - indices.unpersist(false) + indices.unpersist() } - norms.unpersist(false) + norms.unpersist() val clusters = activeClusters ++ inactiveClusters val root = buildTree(clusters, dMeasure) - new BisectingKMeansModel(root, this.distanceMeasure) + val totalCost = root.leafNodes.map(_.cost).sum + new BisectingKMeansModel(root, this.distanceMeasure, totalCost) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala index 9d115afcea75d..c3979118de403 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/BisectingKMeansModel.scala @@ -41,11 +41,12 @@ import org.apache.spark.sql.{Row, SparkSession} @Since("1.6.0") class BisectingKMeansModel private[clustering] ( private[clustering] val root: ClusteringTreeNode, - @Since("2.4.0") val distanceMeasure: String + @Since("2.4.0") val distanceMeasure: String, + @Since("3.0.0") val trainingCost: Double ) extends Serializable with Saveable with Logging { @Since("1.6.0") - def this(root: ClusteringTreeNode) = this(root, DistanceMeasure.EUCLIDEAN) + def this(root: ClusteringTreeNode) = this(root, DistanceMeasure.EUCLIDEAN, 0.0) private val distanceMeasureInstance: DistanceMeasure = DistanceMeasure.decodeFromString(distanceMeasure) @@ -109,10 +110,8 @@ class BisectingKMeansModel private[clustering] ( @Since("2.0.0") override def save(sc: SparkContext, path: String): Unit = { - BisectingKMeansModel.SaveLoadV1_0.save(sc, this, path) + BisectingKMeansModel.SaveLoadV3_0.save(sc, this, path) } - - override protected def formatVersion: String = "1.0" } @Since("2.0.0") @@ -126,13 +125,17 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val model = SaveLoadV1_0.load(sc, path) model case (SaveLoadV2_0.thisClassName, SaveLoadV2_0.thisFormatVersion) => - val model = SaveLoadV1_0.load(sc, path) + val model = SaveLoadV2_0.load(sc, path) + model + case (SaveLoadV3_0.thisClassName, SaveLoadV3_0.thisFormatVersion) => + val model = SaveLoadV3_0.load(sc, path) model case _ => throw new Exception( s"BisectingKMeansModel.load did not recognize model with (className, format version):" + s"($loadedClassName, $formatVersion). Supported:\n" + s" (${SaveLoadV1_0.thisClassName}, ${SaveLoadV1_0.thisClassName}\n" + - s" (${SaveLoadV2_0.thisClassName}, ${SaveLoadV2_0.thisClassName})") + s" (${SaveLoadV2_0.thisClassName}, ${SaveLoadV2_0.thisClassName})\n" + + s" (${SaveLoadV3_0.thisClassName}, ${SaveLoadV3_0.thisClassName})") } } @@ -195,7 +198,8 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap val rootNode = buildTree(rootId, nodes) - new BisectingKMeansModel(rootNode, DistanceMeasure.EUCLIDEAN) + val totalCost = rootNode.leafNodes.map(_.cost).sum + new BisectingKMeansModel(rootNode, DistanceMeasure.EUCLIDEAN, totalCost) } } @@ -231,7 +235,46 @@ object BisectingKMeansModel extends Loader[BisectingKMeansModel] { val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap val rootNode = buildTree(rootId, nodes) - new BisectingKMeansModel(rootNode, distanceMeasure) + val totalCost = rootNode.leafNodes.map(_.cost).sum + new BisectingKMeansModel(rootNode, distanceMeasure, totalCost) + } + } + + private[clustering] object SaveLoadV3_0 { + private[clustering] val thisFormatVersion = "3.0" + + private[clustering] + val thisClassName = "org.apache.spark.mllib.clustering.BisectingKMeansModel" + + def save(sc: SparkContext, model: BisectingKMeansModel, path: String): Unit = { + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val metadata = compact(render( + ("class" -> thisClassName) ~ ("version" -> thisFormatVersion) + ~ ("rootId" -> model.root.index) ~ ("distanceMeasure" -> model.distanceMeasure) + ~ ("trainingCost" -> model.trainingCost))) + sc.parallelize(Seq(metadata), 1).saveAsTextFile(Loader.metadataPath(path)) + + val data = getNodes(model.root).map(node => Data(node.index, node.size, + node.centerWithNorm.vector, node.centerWithNorm.norm, node.cost, node.height, + node.children.map(_.index))) + spark.createDataFrame(data).write.parquet(Loader.dataPath(path)) + } + + def load(sc: SparkContext, path: String): BisectingKMeansModel = { + implicit val formats: DefaultFormats = DefaultFormats + val (className, formatVersion, metadata) = Loader.loadMetadata(sc, path) + assert(className == thisClassName) + assert(formatVersion == thisFormatVersion) + val rootId = (metadata \ "rootId").extract[Int] + val distanceMeasure = (metadata \ "distanceMeasure").extract[String] + val trainingCost = (metadata \ "trainingCost").extract[Double] + val spark = SparkSession.builder().sparkContext(sc).getOrCreate() + val rows = spark.read.parquet(Loader.dataPath(path)) + Loader.checkSchema[Data](rows.schema) + val data = rows.select("index", "size", "center", "norm", "cost", "height", "children") + val nodes = data.rdd.map(Data.apply).collect().map(d => (d.index, d)).toMap + val rootNode = buildTree(rootId, nodes) + new BisectingKMeansModel(rootNode, distanceMeasure, trainingCost) } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala index 4d952ac88c9be..a9ed36e90974e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixture.scala @@ -232,7 +232,7 @@ class GaussianMixture private ( llhp = llh // current becomes previous llh = sums.logLikelihood // this is the freshly computed log-likelihood iter += 1 - compute.destroy(blocking = false) + compute.destroy() } new GaussianMixtureModel(weights, gaussians) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala index 1933d5499c3bf..5d2ecf33129c6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/GaussianMixtureModel.scala @@ -48,8 +48,6 @@ class GaussianMixtureModel @Since("1.3.0") ( require(weights.length == gaussians.length, "Length of weight and Gaussian arrays must match") - override protected def formatVersion = "1.0" - @Since("1.4.0") override def save(sc: SparkContext, path: String): Unit = { GaussianMixtureModel.SaveLoadV1_0.save(sc, path, weights, gaussians) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index d967c672c581f..4bb79bc69eef4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -22,7 +22,6 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.annotation.Since import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging -import org.apache.spark.ml.clustering.{KMeans => NewKMeans} import org.apache.spark.ml.util.Instrumentation import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.linalg.BLAS.axpy @@ -121,26 +120,6 @@ class KMeans private ( this } - /** - * This function has no effect since Spark 2.0.0. - */ - @Since("1.4.0") - @deprecated("This has no effect and always returns 1", "2.1.0") - def getRuns: Int = { - logWarning("Getting number of runs has no effect since Spark 2.0.0.") - 1 - } - - /** - * This function has no effect since Spark 2.0.0. - */ - @Since("0.8.0") - @deprecated("This has no effect", "2.1.0") - def setRuns(runs: Int): this.type = { - logWarning("Setting number of runs has no effect since Spark 2.0.0.") - this - } - /** * Number of steps for the k-means|| initialization mode */ @@ -327,7 +306,7 @@ class KMeans private ( distanceMeasureInstance.centroid(sum, count) } - bcCenters.destroy(blocking = false) + bcCenters.destroy() // Update the cluster centers and costs converged = true @@ -405,8 +384,8 @@ class KMeans private ( }.persist(StorageLevel.MEMORY_AND_DISK) val sumCosts = costs.sum() - bcNewCenters.unpersist(blocking = false) - preCosts.unpersist(blocking = false) + bcNewCenters.unpersist() + preCosts.unpersist() val chosen = data.zip(costs).mapPartitionsWithIndex { (index, pointCosts) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) @@ -417,8 +396,8 @@ class KMeans private ( step += 1 } - costs.unpersist(blocking = false) - bcNewCentersList.foreach(_.destroy(false)) + costs.unpersist() + bcNewCentersList.foreach(_.destroy()) val distinctCenters = centers.map(_.vector).distinct.map(new VectorWithNorm(_)) @@ -433,7 +412,7 @@ class KMeans private ( .map(distanceMeasureInstance.findClosest(bcCenters.value, _)._1) .countByValue() - bcCenters.destroy(blocking = false) + bcCenters.destroy() val myWeights = distinctCenters.indices.map(countMap.getOrElse(_, 0L).toDouble).toArray LocalKMeans.kMeansPlusPlus(0, distinctCenters.toArray, myWeights, k, 30) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index d5c8188144ce2..09c38f72c1ad5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -103,7 +103,7 @@ class KMeansModel (@Since("1.0.0") val clusterCenters: Array[Vector], val cost = data.map(p => distanceMeasureInstance.pointCost(bcCentersWithNorm.value, new VectorWithNorm(p))) .sum() - bcCentersWithNorm.destroy(blocking = false) + bcCentersWithNorm.destroy() cost } @@ -112,8 +112,6 @@ class KMeansModel (@Since("1.0.0") val clusterCenters: Array[Vector], override def save(sc: SparkContext, path: String): Unit = { KMeansModel.SaveLoadV2_0.save(sc, this, path) } - - override protected def formatVersion: String = "1.0" } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala index f915062d77389..85444770fec6b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAModel.scala @@ -194,7 +194,7 @@ class LocalLDAModel private[spark] ( override protected[spark] val gammaShape: Double = 100) extends LDAModel with Serializable { - private var seed: Long = Utils.random.nextLong() + private[spark] var seed: Long = Utils.random.nextLong() @Since("1.3.0") override def k: Int = topics.numCols @@ -216,8 +216,6 @@ class LocalLDAModel private[spark] ( }.toArray } - override protected def formatVersion = "1.0" - /** * Random seed for cluster initialization. */ @@ -338,7 +336,7 @@ class LocalLDAModel private[spark] ( docBound }.sum() - ElogbetaBc.destroy(blocking = false) + ElogbetaBc.destroy() // Bound component for prob(topic-term distributions): // E[log p(beta | eta) - log q(beta | lambda)] @@ -388,31 +386,6 @@ class LocalLDAModel private[spark] ( } } - /** - * Get a method usable as a UDF for `topicDistributions()` - */ - private[spark] def getTopicDistributionMethod: Vector => Vector = { - val expElogbeta = exp(LDAUtils.dirichletExpectation(topicsMatrix.asBreeze.toDenseMatrix.t).t) - val docConcentrationBrz = this.docConcentration.asBreeze - val gammaShape = this.gammaShape - val k = this.k - val gammaSeed = this.seed - - (termCounts: Vector) => - if (termCounts.numNonzeros == 0) { - Vectors.zeros(k) - } else { - val (gamma, _, _) = OnlineLDAOptimizer.variationalTopicInference( - termCounts, - expElogbeta, - docConcentrationBrz, - gammaShape, - k, - gammaSeed) - Vectors.dense(normalize(gamma, 1.0).toArray) - } - } - /** * Predicts the topic mixture distribution for a document (often called "theta" in the * literature). Returns a vector of zeros for an empty document. @@ -835,8 +808,6 @@ class DistributedLDAModel private[clustering] ( // TODO: // override def topicDistributions(documents: RDD[(Long, Vector)]): RDD[(Long, Vector)] = ??? - override protected def formatVersion = "1.0" - @Since("1.5.0") override def save(sc: SparkContext, path: String): Unit = { // Note: This intentionally does not save checkpointFiles. 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 f8e5f3ed76457..5eea69022562b 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 @@ -30,7 +30,7 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.{DenseVector, Matrices, SparseVector, Vector, Vectors} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils + /** * :: DeveloperApi :: @@ -437,6 +437,10 @@ final class OnlineLDAOptimizer extends LDAOptimizer with Logging { this.randomGenerator = new Random(lda.getSeed) this.docs = docs + if (this.docs.getStorageLevel == StorageLevel.NONE) { + logWarning("The input data is not directly cached, which may hurt performance if its" + + " parent RDDs are also uncached.") + } // Initialize the variational distribution q(beta|lambda) this.lambda = getGammaMatrix(k, vocabSize) @@ -505,7 +509,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer with Logging { elementWiseSum, elementWiseSum ) - expElogbetaBc.destroy(false) + expElogbetaBc.destroy() if (nonEmptyDocsN == 0) { logWarning("No non-empty documents were submitted in the batch.") @@ -591,7 +595,7 @@ final class OnlineLDAOptimizer extends LDAOptimizer with Logging { * Serializable companion object containing helper methods and shared code for * [[OnlineLDAOptimizer]] and [[LocalLDAModel]]. */ -private[clustering] object OnlineLDAOptimizer { +private[spark] object OnlineLDAOptimizer { /** * Uses variational inference to infer the topic distribution `gammad` given the term counts * for a document. `termCounts` must contain at least one non-zero entry, otherwise Breeze will @@ -604,27 +608,24 @@ private[clustering] object OnlineLDAOptimizer { * @return Returns a tuple of `gammad` - estimate of gamma, the topic distribution, `sstatsd` - * statistics for updating lambda and `ids` - list of termCounts vector indices. */ - private[clustering] def variationalTopicInference( - termCounts: Vector, + private[spark] def variationalTopicInference( + indices: List[Int], + values: Array[Double], expElogbeta: BDM[Double], alpha: breeze.linalg.Vector[Double], gammaShape: Double, k: Int, seed: Long): (BDV[Double], BDM[Double], List[Int]) = { - val (ids: List[Int], cts: Array[Double]) = termCounts match { - case v: DenseVector => ((0 until v.size).toList, v.values) - case v: SparseVector => (v.indices.toList, v.values) - } // Initialize the variational distribution q(theta|gamma) for the mini-batch val randBasis = new RandBasis(new org.apache.commons.math3.random.MersenneTwister(seed)) val gammad: BDV[Double] = new Gamma(gammaShape, 1.0 / gammaShape)(randBasis).samplesVector(k) // K val expElogthetad: BDV[Double] = exp(LDAUtils.dirichletExpectation(gammad)) // K - val expElogbetad = expElogbeta(ids, ::).toDenseMatrix // ids * K + val expElogbetad = expElogbeta(indices, ::).toDenseMatrix // ids * K val phiNorm: BDV[Double] = expElogbetad * expElogthetad +:+ 1e-100 // ids var meanGammaChange = 1D - val ctsVector = new BDV[Double](cts) // ids + val ctsVector = new BDV[Double](values) // ids // Iterate between gamma and phi until convergence while (meanGammaChange > 1e-3) { @@ -638,6 +639,20 @@ private[clustering] object OnlineLDAOptimizer { } val sstatsd = expElogthetad.asDenseMatrix.t * (ctsVector /:/ phiNorm).asDenseMatrix - (gammad, sstatsd, ids) + (gammad, sstatsd, indices) + } + + private[clustering] def variationalTopicInference( + termCounts: Vector, + expElogbeta: BDM[Double], + alpha: breeze.linalg.Vector[Double], + gammaShape: Double, + k: Int, + seed: Long): (BDV[Double], BDM[Double], List[Int]) = { + val (ids: List[Int], cts: Array[Double]) = termCounts match { + case v: DenseVector => (List.range(0, v.size), v.values) + case v: SparseVector => (v.indices.toList, v.values) + } + variationalTopicInference(ids, cts, expElogbeta, alpha, gammaShape, k, seed) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala index c4bbe51a46c32..b6675907ce0e7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LDAUtils.scala @@ -22,7 +22,7 @@ import breeze.numerics._ /** * Utility methods for LDA. */ -private[clustering] object LDAUtils { +private[spark] object LDAUtils { /** * Log Sum Exp with overflow protection using the identity: * For any a: $\log \sum_{n=1}^N \exp\{x_n\} = a + \log \sum_{n=1}^N \exp\{x_n - a\}$ @@ -44,7 +44,7 @@ private[clustering] object LDAUtils { * Computes [[dirichletExpectation()]] row-wise, assuming each row of alpha are * Dirichlet parameters. */ - private[clustering] def dirichletExpectation(alpha: BDM[Double]): BDM[Double] = { + private[spark] def dirichletExpectation(alpha: BDM[Double]): BDM[Double] = { val rowSum = sum(alpha(breeze.linalg.*, ::)) val digAlpha = digamma(alpha) val digRowSum = digamma(rowSum) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala index 9444f29a91ed8..ba541bbcccd29 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/PowerIterationClustering.scala @@ -48,8 +48,6 @@ class PowerIterationClusteringModel @Since("1.3.0") ( override def save(sc: SparkContext, path: String): Unit = { PowerIterationClusteringModel.SaveLoadV1_0.save(sc, this, path) } - - override protected def formatVersion: String = "1.0" } @Since("1.4.0") @@ -185,6 +183,8 @@ class PowerIterationClustering private[clustering] ( case "random" => randomInit(w) case "degree" => initDegreeVector(w) } + // Materialized the graph w0 in randomInit/initDegreeVector, hence we can unpersist w. + w.unpersist() pic(w0) } @@ -207,6 +207,8 @@ class PowerIterationClustering private[clustering] ( case "random" => randomInit(w) case "degree" => initDegreeVector(w) } + // Materialized the graph w0 in randomInit/initDegreeVector, hence we can unpersist w. + w.unpersist() pic(w0) } @@ -228,11 +230,10 @@ class PowerIterationClustering private[clustering] ( */ private def pic(w: Graph[Double, Double]): PowerIterationClusteringModel = { val v = powerIter(w, maxIterations) - val assignments = kMeans(v, k).mapPartitions({ iter => - iter.map { case (id, cluster) => - Assignment(id, cluster) - } - }, preservesPartitioning = true) + val assignments = kMeans(v, k).map { + case (id, cluster) => Assignment(id, cluster) + } + new PowerIterationClusteringModel(k, assignments) } } @@ -298,12 +299,15 @@ object PowerIterationClustering extends Logging { }, mergeMsg = _ + _, TripletFields.EdgeOnly) - Graph(vD, gA.edges) - .mapTriplets( - e => e.attr / math.max(e.srcAttr, MLUtils.EPSILON), - new TripletFields(/* useSrc */ true, - /* useDst */ false, - /* useEdge */ true)) + val graph = Graph(vD, gA.edges).mapTriplets( + e => e.attr / math.max(e.srcAttr, MLUtils.EPSILON), + new TripletFields(/* useSrc */ true, + /* useDst */ false, + /* useEdge */ true)) + materialize(graph) + gA.unpersist() + + graph } /** @@ -324,7 +328,10 @@ object PowerIterationClustering extends Logging { }, preservesPartitioning = true).cache() val sum = r.values.map(math.abs).sum() val v0 = r.mapValues(x => x / sum) - Graph(VertexRDD(v0), g.edges) + val graph = Graph(VertexRDD(v0), g.edges) + materialize(graph) + r.unpersist() + graph } /** @@ -339,7 +346,9 @@ object PowerIterationClustering extends Logging { def initDegreeVector(g: Graph[Double, Double]): Graph[Double, Double] = { val sum = g.vertices.values.sum() val v0 = g.vertices.mapValues(_ / sum) - Graph(VertexRDD(v0), g.edges) + val graph = Graph(VertexRDD(v0), g.edges) + materialize(graph) + graph } /** @@ -378,10 +387,38 @@ object PowerIterationClustering extends Logging { logInfo(s"$msgPrefix: delta = $delta.") diffDelta = math.abs(delta - prevDelta) logInfo(s"$msgPrefix: diff(delta) = $diffDelta.") + + if (math.abs(diffDelta) < tol) { + /** + * Power Iteration fails to converge if absolute value of top 2 eigen values are equal, + * but with opposite sign. The resultant vector flip-flops between two vectors. + * We should give an exception, if we detect the failure of the convergence of the + * power iteration + */ + + // Rayleigh quotient = x^tAx / x^tx + val xTAx = curG.joinVertices(v) { + case (_, x, y) => x * y + }.vertices.values.sum() + val xTx = curG.vertices.mapValues(x => x * x).values.sum() + val rayleigh = xTAx / xTx + + if (math.abs(norm - math.abs(rayleigh)) > tol) { + logWarning(s"Power Iteration fail to converge. delta = ${delta}," + + s" difference delta = ${diffDelta} and norm = ${norm}") + } + } + curG.vertices.unpersist() + curG.edges.unpersist() // update v curG = Graph(VertexRDD(v1), g.edges) + materialize(curG) + v.unpersist() prevDelta = delta } + + curG.edges.unpersist() + curG.vertices } @@ -393,11 +430,21 @@ object PowerIterationClustering extends Logging { */ private[clustering] def kMeans(v: VertexRDD[Double], k: Int): VertexRDD[Int] = { - val points = v.mapValues(x => Vectors.dense(x)).cache() + val points = v.mapValues(Vectors.dense(_)).cache() val model = new KMeans() .setK(k) .setSeed(0L) .run(points.values) - points.mapValues(p => model.predict(p)).cache() + + val predict = points.mapValues(model.predict(_)) + points.unpersist() + predict + } + + /** + * Forces materialization of a Graph by iterating its RDDs. + */ + private def materialize(g: Graph[_, _]): Unit = { + g.edges.foreachPartition(_ => {}) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala index ed8543da4d4ce..ff4ca0ac40fe2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/StreamingKMeans.scala @@ -42,8 +42,8 @@ import org.apache.spark.util.random.XORShiftRandom *
    * $$ * \begin{align} - * c_t+1 &= [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] \\ - * n_t+1 &= n_t * a + m_t + * c_{t+1} &= [(c_t * n_t * a) + (x_t * m_t)] / [n_t + m_t] \\ + * n_{t+1} &= n_t * a + m_t * \end{align} * $$ *
    diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala index 003d1411a9cf7..cdb84318833f8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.evaluation -import org.apache.spark.mllib.rdd.RDDFunctions._ import org.apache.spark.rdd.RDD /** @@ -42,10 +41,35 @@ private[evaluation] object AreaUnderCurve { * @param curve an RDD of ordered 2D points stored in pairs representing a curve */ def of(curve: RDD[(Double, Double)]): Double = { - curve.sliding(2).aggregate(0.0)( - seqOp = (auc: Double, points: Array[(Double, Double)]) => auc + trapezoid(points), - combOp = _ + _ - ) + val localAreas = curve.mapPartitions { iter => + if (iter.nonEmpty) { + var localArea = 0.0 + var head = true + var firstPoint = (Double.NaN, Double.NaN) + var lastPoint = (Double.NaN, Double.NaN) + + iter.sliding(2).foreach { points => + if (head) { + firstPoint = points.head + head = false + } + lastPoint = points.last + + if (points.length == 2) { + localArea += trapezoid(points) + } + } + Iterator.single((localArea, (firstPoint, lastPoint))) + } else { + Iterator.empty + } + }.collect() + + localAreas.map(_._1).sum + localAreas.iterator.map(_._2) + .sliding(2).withPartial(false) + .map { case Seq((_, last1), (first2, _)) => + trapezoid(Seq(last1, first2)) + }.sum } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala index 2cfcf38eb4ca8..d34a7ca6c9c7f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetrics.scala @@ -21,12 +21,12 @@ import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.mllib.evaluation.binary._ import org.apache.spark.rdd.{RDD, UnionRDD} -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} /** * Evaluator for binary classification. * - * @param scoreAndLabels an RDD of (score, label) pairs. + * @param scoreAndLabels an RDD of (score, label) or (score, label, weight) tuples. * @param numBins if greater than 0, then the curves (ROC curve, PR curve) computed internally * will be down-sampled to this many "bins". If 0, no down-sampling will occur. * This is useful because the curve contains a point for each distinct score @@ -41,9 +41,19 @@ import org.apache.spark.sql.DataFrame * partition boundaries. */ @Since("1.0.0") -class BinaryClassificationMetrics @Since("1.3.0") ( - @Since("1.3.0") val scoreAndLabels: RDD[(Double, Double)], - @Since("1.3.0") val numBins: Int) extends Logging { +class BinaryClassificationMetrics @Since("3.0.0") ( + @Since("1.3.0") val scoreAndLabels: RDD[_ <: Product], + @Since("1.3.0") val numBins: Int = 1000) + extends Logging { + val scoreLabelsWeight: RDD[(Double, (Double, Double))] = scoreAndLabels.map { + case (prediction: Double, label: Double, weight: Double) => + require(weight >= 0.0, s"instance weight, $weight has to be >= 0.0") + (prediction, (label, weight)) + case (prediction: Double, label: Double) => + (prediction, (label, 1.0)) + case other => + throw new IllegalArgumentException(s"Expected tuples, got $other") + } require(numBins >= 0, "numBins must be nonnegative") @@ -58,7 +68,14 @@ class BinaryClassificationMetrics @Since("1.3.0") ( * @param scoreAndLabels a DataFrame with two double columns: score and label */ private[mllib] def this(scoreAndLabels: DataFrame) = - this(scoreAndLabels.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) + this(scoreAndLabels.rdd.map { + case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) + case Row(prediction: Double, label: Double) => + (prediction, label, 1.0) + case other => + throw new IllegalArgumentException(s"Expected Row of tuples, got $other") + }) /** * Unpersist intermediate RDDs used in the computation. @@ -146,11 +163,13 @@ class BinaryClassificationMetrics @Since("1.3.0") ( private lazy val ( cumulativeCounts: RDD[(Double, BinaryLabelCounter)], confusions: RDD[(Double, BinaryConfusionMatrix)]) = { - // Create a bin for each distinct score value, count positives and negatives within each bin, - // and then sort by score values in descending order. - val counts = scoreAndLabels.combineByKey( - createCombiner = (label: Double) => new BinaryLabelCounter(0L, 0L) += label, - mergeValue = (c: BinaryLabelCounter, label: Double) => c += label, + // Create a bin for each distinct score value, count weighted positives and + // negatives within each bin, and then sort by score values in descending order. + val counts = scoreLabelsWeight.combineByKey( + createCombiner = (labelAndWeight: (Double, Double)) => + new BinaryLabelCounter(0.0, 0.0) += (labelAndWeight._1, labelAndWeight._2), + mergeValue = (c: BinaryLabelCounter, labelAndWeight: (Double, Double)) => + c += (labelAndWeight._1, labelAndWeight._2), mergeCombiners = (c1: BinaryLabelCounter, c2: BinaryLabelCounter) => c1 += c2 ).sortByKey(ascending = false) @@ -175,12 +194,15 @@ class BinaryClassificationMetrics @Since("1.3.0") ( grouping = Int.MaxValue } counts.mapPartitions(_.grouped(grouping.toInt).map { pairs => - // The score of the combined point will be just the first one's score - val firstScore = pairs.head._1 - // The point will contain all counts in this chunk + // The score of the combined point will be just the last one's score, which is also + // the minimal in each chunk since all scores are already sorted in descending. + val lastScore = pairs.last._1 + // The combined point will contain all counts in this chunk. Thus, calculated + // metrics (like precision, recall, etc.) on its score (or so-called threshold) are + // the same as those without sampling. val agg = new BinaryLabelCounter() pairs.foreach(pair => agg += pair._2) - (firstScore, agg) + (lastScore, agg) }) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala index 9a6a8dbdccbf3..c8245bf500a2f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MulticlassMetrics.scala @@ -18,45 +18,88 @@ package org.apache.spark.mllib.evaluation import scala.collection.Map +import scala.collection.mutable import org.apache.spark.annotation.Since import org.apache.spark.mllib.linalg.{Matrices, Matrix} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} /** * Evaluator for multiclass classification. * - * @param predictionAndLabels an RDD of (prediction, label) pairs. + * @param predictionAndLabels an RDD of (prediction, label, weight) or + * (prediction, label) tuples. */ @Since("1.1.0") -class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Double)]) { +class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[_ <: Product]) { /** * An auxiliary constructor taking a DataFrame. * @param predictionAndLabels a DataFrame with two double columns: prediction and label */ private[mllib] def this(predictionAndLabels: DataFrame) = - this(predictionAndLabels.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) - - private lazy val labelCountByClass: Map[Double, Long] = predictionAndLabels.values.countByValue() - private lazy val labelCount: Long = labelCountByClass.values.sum - private lazy val tpByClass: Map[Double, Int] = predictionAndLabels - .map { case (prediction, label) => - (label, if (label == prediction) 1 else 0) - }.reduceByKey(_ + _) - .collectAsMap() - private lazy val fpByClass: Map[Double, Int] = predictionAndLabels - .map { case (prediction, label) => - (prediction, if (prediction != label) 1 else 0) - }.reduceByKey(_ + _) - .collectAsMap() - private lazy val confusions = predictionAndLabels - .map { case (prediction, label) => - ((label, prediction), 1) - }.reduceByKey(_ + _) + this(predictionAndLabels.rdd.map { + case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) + case Row(prediction: Double, label: Double) => + (prediction, label, 1.0) + case other => + throw new IllegalArgumentException(s"Expected Row of tuples, got $other") + }) + + + private val confusions = predictionAndLabels.map { + case (prediction: Double, label: Double, weight: Double) => + ((label, prediction), weight) + case (prediction: Double, label: Double) => + ((label, prediction), 1.0) + case other => + throw new IllegalArgumentException(s"Expected tuples, got $other") + }.reduceByKey(_ + _) .collectAsMap() + private lazy val labelCountByClass: Map[Double, Double] = { + val labelCountByClass = mutable.Map.empty[Double, Double] + confusions.iterator.foreach { + case ((label, _), weight) => + val w = labelCountByClass.getOrElse(label, 0.0) + labelCountByClass.update(label, w + weight) + } + labelCountByClass.toMap + } + + private lazy val labelCount: Double = labelCountByClass.values.sum + + private lazy val tpByClass: Map[Double, Double] = { + val tpByClass = mutable.Map.empty[Double, Double] + confusions.iterator.foreach { + case ((label, prediction), weight) => + val w = tpByClass.getOrElse(label, 0.0) + if (label == prediction) { + tpByClass.update(label, w + weight) + } else if (w == 0.0) { + tpByClass.update(label, w) + } + } + tpByClass.toMap + } + + private lazy val fpByClass: Map[Double, Double] = { + val fpByClass = mutable.Map.empty[Double, Double] + confusions.iterator.foreach { + case ((label, prediction), weight) => + val w = fpByClass.getOrElse(prediction, 0.0) + if (label != prediction) { + fpByClass.update(prediction, w + weight) + } else if (w == 0.0) { + fpByClass.update(prediction, w) + } + } + fpByClass.toMap + } + + /** * Returns confusion matrix: * predicted classes are in columns, @@ -71,7 +114,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl while (i < n) { var j = 0 while (j < n) { - values(i + j * n) = confusions.getOrElse((labels(i), labels(j)), 0).toDouble + values(i + j * n) = confusions.getOrElse((labels(i), labels(j)), 0.0) j += 1 } i += 1 @@ -92,8 +135,8 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl */ @Since("1.1.0") def falsePositiveRate(label: Double): Double = { - val fp = fpByClass.getOrElse(label, 0) - fp.toDouble / (labelCount - labelCountByClass(label)) + val fp = fpByClass.getOrElse(label, 0.0) + fp / (labelCount - labelCountByClass(label)) } /** @@ -103,7 +146,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl @Since("1.1.0") def precision(label: Double): Double = { val tp = tpByClass(label) - val fp = fpByClass.getOrElse(label, 0) + val fp = fpByClass.getOrElse(label, 0.0) if (tp + fp == 0) 0 else tp.toDouble / (tp + fp) } @@ -112,7 +155,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * @param label the label. */ @Since("1.1.0") - def recall(label: Double): Double = tpByClass(label).toDouble / labelCountByClass(label) + def recall(label: Double): Double = tpByClass(label) / labelCountByClass(label) /** * Returns f-measure for a given label (category) @@ -134,38 +177,13 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl @Since("1.1.0") def fMeasure(label: Double): Double = fMeasure(label, 1.0) - /** - * Returns precision - */ - @Since("1.1.0") - @deprecated("Use accuracy.", "2.0.0") - lazy val precision: Double = accuracy - - /** - * Returns recall - * (equals to precision for multiclass classifier - * because sum of all false positives is equal to sum - * of all false negatives) - */ - @Since("1.1.0") - @deprecated("Use accuracy.", "2.0.0") - lazy val recall: Double = accuracy - - /** - * Returns f-measure - * (equals to precision and recall because precision equals recall) - */ - @Since("1.1.0") - @deprecated("Use accuracy.", "2.0.0") - lazy val fMeasure: Double = accuracy - /** * Returns accuracy * (equals to the total number of correctly classified instances * out of the total number of instances.) */ @Since("2.0.0") - lazy val accuracy: Double = tpByClass.values.sum.toDouble / labelCount + lazy val accuracy: Double = tpByClass.values.sum / labelCount /** * Returns weighted true positive rate @@ -212,9 +230,7 @@ class MulticlassMetrics @Since("1.1.0") (predictionAndLabels: RDD[(Double, Doubl * Returns weighted averaged f1-measure */ @Since("1.1.0") - lazy val weightedFMeasure: Double = labelCountByClass.map { case (category, count) => - fMeasure(category, 1.0) * count.toDouble / labelCount - }.sum + lazy val weightedFMeasure: Double = weightedFMeasure(1.0) /** * Returns the sequence of labels in ascending order diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala index 77bd0aa30dda1..1565447af7b91 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/MultilabelMetrics.scala @@ -17,6 +17,8 @@ package org.apache.spark.mllib.evaluation +import scala.collection.mutable + import org.apache.spark.annotation.Since import org.apache.spark.rdd.RDD import org.apache.spark.sql.DataFrame @@ -38,76 +40,56 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] (r.getSeq[Double](0).toArray, r.getSeq[Double](1).toArray) }) - private lazy val numDocs: Long = predictionAndLabels.count() + /** + * Use MultilabelSummarizer to calculate all summary statistics of predictions + * and labels on one pass. + */ + private val summary: MultilabelSummarizer = { + predictionAndLabels + .treeAggregate(new MultilabelSummarizer)( + (summary, sample) => summary.add(sample._1, sample._2), + (sum1, sum2) => sum1.merge(sum2) + ) + } - private lazy val numLabels: Long = predictionAndLabels.flatMap { case (_, labels) => - labels}.distinct().count() /** * Returns subset accuracy * (for equal sets of labels) */ @Since("1.2.0") - lazy val subsetAccuracy: Double = predictionAndLabels.filter { case (predictions, labels) => - predictions.deep == labels.deep - }.count().toDouble / numDocs + val subsetAccuracy: Double = summary.subsetAccuracy /** * Returns accuracy */ @Since("1.2.0") - lazy val accuracy: Double = predictionAndLabels.map { case (predictions, labels) => - labels.intersect(predictions).length.toDouble / - (labels.length + predictions.length - labels.intersect(predictions).length)}.sum / numDocs + val accuracy: Double = summary.accuracy /** * Returns Hamming-loss */ @Since("1.2.0") - lazy val hammingLoss: Double = predictionAndLabels.map { case (predictions, labels) => - labels.length + predictions.length - 2 * labels.intersect(predictions).length - }.sum / (numDocs * numLabels) + val hammingLoss: Double = summary.hammingLoss /** * Returns document-based precision averaged by the number of documents */ @Since("1.2.0") - lazy val precision: Double = predictionAndLabels.map { case (predictions, labels) => - if (predictions.length > 0) { - predictions.intersect(labels).length.toDouble / predictions.length - } else { - 0 - } - }.sum / numDocs + val precision: Double = summary.precision /** * Returns document-based recall averaged by the number of documents */ @Since("1.2.0") - lazy val recall: Double = predictionAndLabels.map { case (predictions, labels) => - labels.intersect(predictions).length.toDouble / labels.length - }.sum / numDocs + val recall: Double = summary.recall /** * Returns document-based f1-measure averaged by the number of documents */ @Since("1.2.0") - lazy val f1Measure: Double = predictionAndLabels.map { case (predictions, labels) => - 2.0 * predictions.intersect(labels).length / (predictions.length + labels.length) - }.sum / numDocs - - private lazy val tpPerClass = predictionAndLabels.flatMap { case (predictions, labels) => - predictions.intersect(labels) - }.countByValue() - - private lazy val fpPerClass = predictionAndLabels.flatMap { case (predictions, labels) => - predictions.diff(labels) - }.countByValue() - - private lazy val fnPerClass = predictionAndLabels.flatMap { case(predictions, labels) => - labels.diff(predictions) - }.countByValue() + val f1Measure: Double = summary.f1Measure /** * Returns precision for a given label (category) @@ -115,8 +97,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") def precision(label: Double): Double = { - val tp = tpPerClass(label) - val fp = fpPerClass.getOrElse(label, 0L) + val tp = summary.tpPerClass(label) + val fp = summary.fpPerClass.getOrElse(label, 0L) if (tp + fp == 0) 0.0 else tp.toDouble / (tp + fp) } @@ -126,8 +108,8 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] */ @Since("1.2.0") def recall(label: Double): Double = { - val tp = tpPerClass(label) - val fn = fnPerClass.getOrElse(label, 0L) + val tp = summary.tpPerClass(label) + val fn = summary.fnPerClass.getOrElse(label, 0L) if (tp + fn == 0) 0.0 else tp.toDouble / (tp + fn) } @@ -142,29 +124,23 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] if((p + r) == 0) 0.0 else 2 * p * r / (p + r) } - private lazy val sumTp = tpPerClass.foldLeft(0L) { case (sum, (_, tp)) => sum + tp } - private lazy val sumFpClass = fpPerClass.foldLeft(0L) { case (sum, (_, fp)) => sum + fp } - private lazy val sumFnClass = fnPerClass.foldLeft(0L) { case (sum, (_, fn)) => sum + fn } + private lazy val sumTp = summary.tpPerClass.values.sum + private lazy val sumFpClass = summary.fpPerClass.values.sum + private lazy val sumFnClass = summary.fnPerClass.values.sum /** * Returns micro-averaged label-based precision * (equals to micro-averaged document-based precision) */ @Since("1.2.0") - lazy val microPrecision: Double = { - val sumFp = fpPerClass.foldLeft(0L) { case(cum, (_, fp)) => cum + fp} - sumTp.toDouble / (sumTp + sumFp) - } + lazy val microPrecision: Double = sumTp.toDouble / (sumTp + sumFpClass) /** * Returns micro-averaged label-based recall * (equals to micro-averaged document-based recall) */ @Since("1.2.0") - lazy val microRecall: Double = { - val sumFn = fnPerClass.foldLeft(0.0) { case(cum, (_, fn)) => cum + fn} - sumTp.toDouble / (sumTp + sumFn) - } + lazy val microRecall: Double = sumTp.toDouble / (sumTp + sumFnClass) /** * Returns micro-averaged label-based f1-measure @@ -177,5 +153,129 @@ class MultilabelMetrics @Since("1.2.0") (predictionAndLabels: RDD[(Array[Double] * Returns the sequence of labels in ascending order */ @Since("1.2.0") - lazy val labels: Array[Double] = tpPerClass.keys.toArray.sorted + lazy val labels: Array[Double] = summary.tpPerClass.keys.toArray.sorted +} + + +private[evaluation] class MultilabelSummarizer extends Serializable { + + private var docCnt = 0L + private val labelSet = mutable.Set.empty[Double] + private var subsetAccuracyCnt = 0L + private var accuracySum = 0.0 + private var hammingLossSum = 0L + private var precisionSum = 0.0 + private var recallSum = 0.0 + private var f1MeasureSum = 0.0 + val tpPerClass = mutable.Map.empty[Double, Long] + val fpPerClass = mutable.Map.empty[Double, Long] + val fnPerClass = mutable.Map.empty[Double, Long] + + /** + * Add a new sample (predictions and labels) to this summarizer, and update + * the statistical summary. + * + * @return This MultilabelSummarizer object. + */ + def add(predictions: Array[Double], labels: Array[Double]): this.type = { + val intersection = predictions.intersect(labels) + + docCnt += 1L + + labelSet ++= labels + + if (java.util.Arrays.equals(predictions, labels)) { + subsetAccuracyCnt += 1 + } + + accuracySum += intersection.length.toDouble / + (labels.length + predictions.length - intersection.length) + + hammingLossSum += labels.length + predictions.length - 2 * intersection.length + + if (predictions.length > 0) { + precisionSum += intersection.length.toDouble / predictions.length + } + + recallSum += intersection.length.toDouble / labels.length + + f1MeasureSum += 2.0 * intersection.length / (predictions.length + labels.length) + + intersection.foreach { k => + val v = tpPerClass.getOrElse(k, 0L) + tpPerClass.update(k, v + 1) + } + + predictions.diff(labels).foreach { k => + val v = fpPerClass.getOrElse(k, 0L) + fpPerClass.update(k, v + 1) + } + + labels.diff(predictions).foreach { k => + val v = fnPerClass.getOrElse(k, 0L) + fnPerClass.update(k, v + 1) + } + + this + } + + /** + * Merge another MultilabelSummarizer, and update the statistical summary. + * (Note that it's in place merging; as a result, `this` object will be modified.) + * + * @param other The other MultilabelSummarizer to be merged. + * @return This MultilabelSummarizer object. + */ + def merge(other: MultilabelSummarizer): this.type = { + if (other.docCnt > 0) { + docCnt += other.docCnt + + labelSet ++= other.labelSet + + subsetAccuracyCnt += other.subsetAccuracyCnt + + accuracySum += other.accuracySum + + hammingLossSum += other.hammingLossSum + + precisionSum += other.precisionSum + + recallSum += other.recallSum + + f1MeasureSum += other.f1MeasureSum + + other.tpPerClass.foreach { case (k, v1) => + val v0 = tpPerClass.getOrElse(k, 0L) + tpPerClass.update(k, v0 + v1) + } + + other.fpPerClass.foreach { case (k, v1) => + val v0 = fpPerClass.getOrElse(k, 0L) + fpPerClass.update(k, v0 + v1) + } + + other.fnPerClass.foreach { case (k, v1) => + val v0 = fnPerClass.getOrElse(k, 0L) + fnPerClass.update(k, v0 + v1) + } + } + + this + } + + def numDocs: Long = docCnt + + def numLabels: Long = labelSet.size.toLong + + def subsetAccuracy: Double = subsetAccuracyCnt.toDouble / numDocs + + def accuracy: Double = accuracySum / numDocs + + def hammingLoss: Double = hammingLossSum.toDouble / numDocs / numLabels + + def precision: Double = precisionSum / numDocs + + def recall: Double = recallSum / numDocs + + def f1Measure: Double = f1MeasureSum / numDocs } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala index b98aa0534152b..9e35ee2d60f25 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RankingMetrics.scala @@ -59,23 +59,7 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] def precisionAt(k: Int): Double = { require(k > 0, "ranking position k should be positive") predictionAndLabels.map { case (pred, lab) => - val labSet = lab.toSet - - if (labSet.nonEmpty) { - val n = math.min(pred.length, k) - var i = 0 - var cnt = 0 - while (i < n) { - if (labSet.contains(pred(i))) { - cnt += 1 - } - i += 1 - } - cnt.toDouble / k - } else { - logWarning("Empty ground truth set, check input data") - 0.0 - } + countRelevantItemRatio(pred, lab, k, k) }.mean() } @@ -84,28 +68,58 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] * If a query has an empty ground truth set, the average precision will be zero and a log * warning is generated. */ + @Since("1.2.0") lazy val meanAveragePrecision: Double = { predictionAndLabels.map { case (pred, lab) => val labSet = lab.toSet + val k = math.max(pred.length, labSet.size) + averagePrecision(pred, labSet, k) + }.mean() + } - if (labSet.nonEmpty) { - var i = 0 - var cnt = 0 - var precSum = 0.0 - val n = pred.length - while (i < n) { - if (labSet.contains(pred(i))) { - cnt += 1 - precSum += cnt.toDouble / (i + 1) - } - i += 1 + /** + * Returns the mean average precision (MAP) at ranking position k of all the queries. + * If a query has an empty ground truth set, the average precision will be zero and a log + * warning is generated. + * @param k the position to compute the truncated precision, must be positive + * @return the mean average precision at first k ranking positions + */ + @Since("3.0.0") + def meanAveragePrecisionAt(k: Int): Double = { + require(k > 0, "ranking position k should be positive") + predictionAndLabels.map { case (pred, lab) => + averagePrecision(pred, lab.toSet, k) + }.mean() + } + + /** + * Computes the average precision at first k ranking positions of all the queries. + * If a query has an empty ground truth set, the value will be zero and a log + * warning is generated. + * + * @param pred predicted ranking + * @param lab ground truth + * @param k use the top k predicted ranking, must be positive + * @return average precision at first k ranking positions + */ + private def averagePrecision(pred: Array[T], lab: Set[T], k: Int): Double = { + if (lab.nonEmpty) { + var i = 0 + var cnt = 0 + var precSum = 0.0 + val n = math.min(k, pred.length) + while (i < n) { + if (lab.contains(pred(i))) { + cnt += 1 + precSum += cnt.toDouble / (i + 1) } - precSum / labSet.size - } else { - logWarning("Empty ground truth set, check input data") - 0.0 + i += 1 } - }.mean() + precSum / math.min(lab.size, k) + } else { + logWarning("Empty ground truth set, check input data") + 0.0 + } } /** @@ -138,6 +152,8 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] var dcg = 0.0 var i = 0 while (i < n) { + // Base of the log doesn't matter for calculating NDCG, + // if the relevance value is binary. val gain = 1.0 / math.log(i + 2) if (i < pred.length && labSet.contains(pred(i))) { dcg += gain @@ -155,6 +171,63 @@ class RankingMetrics[T: ClassTag](predictionAndLabels: RDD[(Array[T], Array[T])] }.mean() } + /** + * Compute the average recall of all the queries, truncated at ranking position k. + * + * If for a query, the ranking algorithm returns n results, the recall value will be + * computed as #(relevant items retrieved) / #(ground truth set). This formula + * also applies when the size of the ground truth set is less than k. + * + * If a query has an empty ground truth set, zero will be used as recall together with + * a log warning. + * + * See the following paper for detail: + * + * IR evaluation methods for retrieving highly relevant documents. K. Jarvelin and J. Kekalainen + * + * @param k the position to compute the truncated recall, must be positive + * @return the average recall at the first k ranking positions + */ + @Since("3.0.0") + def recallAt(k: Int): Double = { + require(k > 0, "ranking position k should be positive") + predictionAndLabels.map { case (pred, lab) => + countRelevantItemRatio(pred, lab, k, lab.toSet.size) + }.mean() + } + + /** + * Returns the relevant item ratio computed as #(relevant items retrieved) / denominator. + * If a query has an empty ground truth set, the value will be zero and a log + * warning is generated. + * + * @param pred predicted ranking + * @param lab ground truth + * @param k use the top k predicted ranking, must be positive + * @param denominator the denominator of ratio + * @return relevant item ratio at the first k ranking positions + */ + private def countRelevantItemRatio(pred: Array[T], + lab: Array[T], + k: Int, + denominator: Int): Double = { + val labSet = lab.toSet + if (labSet.nonEmpty) { + val n = math.min(pred.length, k) + var i = 0 + var cnt = 0 + while (i < n) { + if (labSet.contains(pred(i))) { + cnt += 1 + } + i += 1 + } + cnt.toDouble / denominator + } else { + logWarning("Empty ground truth set, check input data") + 0.0 + } + } } object RankingMetrics { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala index 020676cac5a64..e5c2d0d85149c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/RegressionMetrics.scala @@ -22,22 +22,23 @@ import org.apache.spark.internal.Logging import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.{DataFrame, Row} /** * Evaluator for regression. * - * @param predictionAndObservations an RDD of (prediction, observation) pairs + * @param predictionAndObservations an RDD of either (prediction, observation, weight) + * or (prediction, observation) pairs * @param throughOrigin True if the regression is through the origin. For example, in linear * regression, it will be true without fitting intercept. */ @Since("1.2.0") class RegressionMetrics @Since("2.0.0") ( - predictionAndObservations: RDD[(Double, Double)], throughOrigin: Boolean) + predictionAndObservations: RDD[_ <: Product], throughOrigin: Boolean) extends Logging { @Since("1.2.0") - def this(predictionAndObservations: RDD[(Double, Double)]) = + def this(predictionAndObservations: RDD[_ <: Product]) = this(predictionAndObservations, false) /** @@ -46,30 +47,36 @@ class RegressionMetrics @Since("2.0.0") ( * prediction and observation */ private[mllib] def this(predictionAndObservations: DataFrame) = - this(predictionAndObservations.rdd.map(r => (r.getDouble(0), r.getDouble(1)))) + this(predictionAndObservations.rdd.map { + case Row(prediction: Double, label: Double, weight: Double) => + (prediction, label, weight) + case Row(prediction: Double, label: Double) => + (prediction, label, 1.0) + case other => + throw new IllegalArgumentException(s"Expected Row of tuples, got $other") + }) /** * Use MultivariateOnlineSummarizer to calculate summary statistics of observations and errors. */ private lazy val summary: MultivariateStatisticalSummary = { - val summary: MultivariateStatisticalSummary = predictionAndObservations.map { - case (prediction, observation) => Vectors.dense(observation, observation - prediction) + predictionAndObservations.map { + case (prediction: Double, observation: Double, weight: Double) => + (Vectors.dense(observation, observation - prediction, prediction), weight) + case (prediction: Double, observation: Double) => + (Vectors.dense(observation, observation - prediction, prediction), 1.0) }.treeAggregate(new MultivariateOnlineSummarizer())( - (summary, v) => summary.add(v), + (summary, sample) => summary.add(sample._1, sample._2), (sum1, sum2) => sum1.merge(sum2) ) - summary } private lazy val SSy = math.pow(summary.normL2(0), 2) private lazy val SSerr = math.pow(summary.normL2(1), 2) - private lazy val SStot = summary.variance(0) * (summary.count - 1) - private lazy val SSreg = { - val yMean = summary.mean(0) - predictionAndObservations.map { - case (prediction, _) => math.pow(prediction - yMean, 2) - }.sum() - } + private lazy val SStot = summary.variance(0) * (summary.weightSum - 1) + private lazy val SSreg = math.pow(summary.normL2(2), 2) + + math.pow(summary.mean(0), 2) * summary.weightSum - + 2 * summary.mean(0) * summary.mean(2) * summary.weightSum /** * Returns the variance explained by regression. @@ -79,7 +86,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def explainedVariance: Double = { - SSreg / summary.count + SSreg / summary.weightSum } /** @@ -88,7 +95,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def meanAbsoluteError: Double = { - summary.normL1(1) / summary.count + summary.normL1(1) / summary.weightSum } /** @@ -97,7 +104,7 @@ class RegressionMetrics @Since("2.0.0") ( */ @Since("1.2.0") def meanSquaredError: Double = { - SSerr / summary.count + SSerr / summary.weightSum } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala index 5a4c6aef50b7b..d98ca2bdc9ded 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala @@ -27,11 +27,11 @@ private[evaluation] trait BinaryClassificationMetricComputer extends Serializabl /** Precision. Defined as 1.0 when there are no positive examples. */ private[evaluation] object Precision extends BinaryClassificationMetricComputer { override def apply(c: BinaryConfusionMatrix): Double = { - val totalPositives = c.numTruePositives + c.numFalsePositives - if (totalPositives == 0) { + val totalPositives = c.weightedTruePositives + c.weightedFalsePositives + if (totalPositives == 0.0) { 1.0 } else { - c.numTruePositives.toDouble / totalPositives + c.weightedTruePositives / totalPositives } } } @@ -39,10 +39,10 @@ private[evaluation] object Precision extends BinaryClassificationMetricComputer /** False positive rate. Defined as 0.0 when there are no negative examples. */ private[evaluation] object FalsePositiveRate extends BinaryClassificationMetricComputer { override def apply(c: BinaryConfusionMatrix): Double = { - if (c.numNegatives == 0) { + if (c.weightedNegatives == 0.0) { 0.0 } else { - c.numFalsePositives.toDouble / c.numNegatives + c.weightedFalsePositives / c.weightedNegatives } } } @@ -50,10 +50,10 @@ private[evaluation] object FalsePositiveRate extends BinaryClassificationMetricC /** Recall. Defined as 0.0 when there are no positive examples. */ private[evaluation] object Recall extends BinaryClassificationMetricComputer { override def apply(c: BinaryConfusionMatrix): Double = { - if (c.numPositives == 0) { + if (c.weightedPositives == 0.0) { 0.0 } else { - c.numTruePositives.toDouble / c.numPositives + c.weightedTruePositives / c.weightedPositives } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala index 559c6ef7e7251..192c9b1863fe7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -21,23 +21,23 @@ package org.apache.spark.mllib.evaluation.binary * Trait for a binary confusion matrix. */ private[evaluation] trait BinaryConfusionMatrix { - /** number of true positives */ - def numTruePositives: Long + /** weighted number of true positives */ + def weightedTruePositives: Double - /** number of false positives */ - def numFalsePositives: Long + /** weighted number of false positives */ + def weightedFalsePositives: Double - /** number of false negatives */ - def numFalseNegatives: Long + /** weighted number of false negatives */ + def weightedFalseNegatives: Double - /** number of true negatives */ - def numTrueNegatives: Long + /** weighted number of true negatives */ + def weightedTrueNegatives: Double - /** number of positives */ - def numPositives: Long = numTruePositives + numFalseNegatives + /** weighted number of positives */ + def weightedPositives: Double = weightedTruePositives + weightedFalseNegatives - /** number of negatives */ - def numNegatives: Long = numFalsePositives + numTrueNegatives + /** weighted number of negatives */ + def weightedNegatives: Double = weightedFalsePositives + weightedTrueNegatives } /** @@ -51,20 +51,22 @@ private[evaluation] case class BinaryConfusionMatrixImpl( totalCount: BinaryLabelCounter) extends BinaryConfusionMatrix { /** number of true positives */ - override def numTruePositives: Long = count.numPositives + override def weightedTruePositives: Double = count.weightedNumPositives /** number of false positives */ - override def numFalsePositives: Long = count.numNegatives + override def weightedFalsePositives: Double = count.weightedNumNegatives /** number of false negatives */ - override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + override def weightedFalseNegatives: Double = + totalCount.weightedNumPositives - count.weightedNumPositives /** number of true negatives */ - override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + override def weightedTrueNegatives: Double = + totalCount.weightedNumNegatives - count.weightedNumNegatives /** number of positives */ - override def numPositives: Long = totalCount.numPositives + override def weightedPositives: Double = totalCount.weightedNumPositives /** number of negatives */ - override def numNegatives: Long = totalCount.numNegatives + override def weightedNegatives: Double = totalCount.weightedNumNegatives } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala index 1e610c20092a7..1ad91966b2141 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryLabelCounter.scala @@ -20,31 +20,39 @@ package org.apache.spark.mllib.evaluation.binary /** * A counter for positives and negatives. * - * @param numPositives number of positive labels - * @param numNegatives number of negative labels + * @param weightedNumPositives weighted number of positive labels + * @param weightedNumNegatives weighted number of negative labels */ private[evaluation] class BinaryLabelCounter( - var numPositives: Long = 0L, - var numNegatives: Long = 0L) extends Serializable { + var weightedNumPositives: Double = 0.0, + var weightedNumNegatives: Double = 0.0) extends Serializable { /** Processes a label. */ def +=(label: Double): BinaryLabelCounter = { // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle // -1.0 for negative as well. - if (label > 0.5) numPositives += 1L else numNegatives += 1L + if (label > 0.5) weightedNumPositives += 1.0 else weightedNumNegatives += 1.0 + this + } + + /** Processes a label with a weight. */ + def +=(label: Double, weight: Double): BinaryLabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) weightedNumPositives += weight else weightedNumNegatives += weight this } /** Merges another counter. */ def +=(other: BinaryLabelCounter): BinaryLabelCounter = { - numPositives += other.numPositives - numNegatives += other.numNegatives + weightedNumPositives += other.weightedNumPositives + weightedNumNegatives += other.weightedNumNegatives this } override def clone: BinaryLabelCounter = { - new BinaryLabelCounter(numPositives, numNegatives) + new BinaryLabelCounter(weightedNumPositives, weightedNumNegatives) } - override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" + override def toString: String = s"{numPos: $weightedNumPositives, numNeg: $weightedNumNegatives}" } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala index aa78e91b679ac..82f5b279846ba 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ChiSqSelector.scala @@ -75,46 +75,52 @@ class ChiSqSelectorModel @Since("1.3.0") ( private def compress(features: Vector): Vector = { features match { case SparseVector(size, indices, values) => - val newSize = filterIndices.length - val newValues = new ArrayBuilder.ofDouble - val newIndices = new ArrayBuilder.ofInt - var i = 0 - var j = 0 - var indicesIdx = 0 - var filterIndicesIdx = 0 - while (i < indices.length && j < filterIndices.length) { - indicesIdx = indices(i) - filterIndicesIdx = filterIndices(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) - Vectors.sparse(newSize, newIndices.result(), newValues.result()) + val (newIndices, newValues) = compressSparse(indices, values) + Vectors.sparse(filterIndices.length, newIndices, newValues) case DenseVector(values) => - val values = features.toArray - Vectors.dense(filterIndices.map(i => values(i))) + Vectors.dense(compressDense(values)) case other => throw new UnsupportedOperationException( s"Only sparse and dense vectors are supported but got ${other.getClass}.") } } + 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 + var indicesIdx = 0 + var filterIndicesIdx = 0 + while (i < indices.length && j < filterIndices.length) { + indicesIdx = indices(i) + filterIndicesIdx = filterIndices(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()) + } + + private[spark] def compressDense(values: Array[Double]): Array[Double] = { + filterIndices.map(i => values(i)) + } + @Since("1.6.0") override def save(sc: SparkContext, path: String): Unit = { ChiSqSelectorModel.SaveLoadV1_0.save(sc, this, path) } - - override protected def formatVersion: String = "1.0" } object ChiSqSelectorModel extends Loader[ChiSqSelectorModel] { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala index c757fc7f06c58..058598c757843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/ElementwiseProduct.scala @@ -41,25 +41,38 @@ class ElementwiseProduct @Since("1.4.0") ( require(vector.size == scalingVec.size, s"vector sizes do not match: Expected ${scalingVec.size} but found ${vector.size}") vector match { - case dv: DenseVector => - val values: Array[Double] = dv.values.clone() - val dim = scalingVec.size - var i = 0 - while (i < dim) { - values(i) *= scalingVec(i) - i += 1 - } - Vectors.dense(values) - case SparseVector(size, indices, vs) => - val values = vs.clone() - val dim = values.length - var i = 0 - while (i < dim) { - values(i) *= scalingVec(indices(i)) - i += 1 - } - Vectors.sparse(size, indices, values) - case v => throw new IllegalArgumentException("Does not support vector type " + v.getClass) + case DenseVector(values) => + val newValues = transformDense(values) + Vectors.dense(newValues) + case SparseVector(size, indices, values) => + val (newIndices, newValues) = transformSparse(indices, values) + Vectors.sparse(size, newIndices, newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") } } + + private[spark] def transformDense(values: Array[Double]): Array[Double] = { + val newValues = values.clone() + val dim = scalingVec.size + var i = 0 + while (i < dim) { + newValues(i) *= scalingVec(i) + i += 1 + } + newValues + } + + private[spark] def transformSparse(indices: Array[Int], + values: Array[Double]): (Array[Int], Array[Double]) = { + val newValues = values.clone() + val dim = newValues.length + var i = 0 + while (i < dim) { + newValues(i) *= scalingVec(indices(i)) + i += 1 + } + (indices, newValues) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala index 8935c8496cdbb..d3b27e1808e15 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/HashingTF.scala @@ -94,6 +94,11 @@ class HashingTF(val numFeatures: Int) extends Serializable { */ @Since("1.1.0") def transform(document: Iterable[_]): Vector = { + val seq = transformImpl(document) + Vectors.sparse(numFeatures, seq) + } + + private[spark] def transformImpl(document: Iterable[_]): Seq[(Int, Double)] = { val termFrequencies = mutable.HashMap.empty[Int, Double] val setTF = if (binary) (i: Int) => 1.0 else (i: Int) => termFrequencies.getOrElse(i, 0.0) + 1.0 val hashFunc: Any => Int = getHashFunction @@ -101,7 +106,7 @@ class HashingTF(val numFeatures: Int) extends Serializable { val i = Utils.nonNegativeMod(hashFunc(term), numFeatures) termFrequencies.put(i, setTF(i)) } - Vectors.sparse(numFeatures, termFrequencies.toSeq) + termFrequencies.toSeq } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala index bb4b37ef21a84..e868f0f92509a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/IDF.scala @@ -32,6 +32,7 @@ import org.apache.spark.rdd.RDD * This implementation supports filtering out terms which do not appear in a minimum number * of documents (controlled by the variable `minDocFreq`). For terms that are not in * at least `minDocFreq` documents, the IDF is found as 0, resulting in TF-IDFs of 0. + * The document frequency is 0 as well for such terms * * @param minDocFreq minimum of documents in which a term * should appear for filtering @@ -50,12 +51,12 @@ class IDF @Since("1.2.0") (@Since("1.2.0") val minDocFreq: Int) { */ @Since("1.1.0") def fit(dataset: RDD[Vector]): IDFModel = { - val idf = dataset.treeAggregate(new IDF.DocumentFrequencyAggregator( - minDocFreq = minDocFreq))( + val (idf: Vector, docFreq: Array[Long], numDocs: Long) = dataset.treeAggregate( + new IDF.DocumentFrequencyAggregator(minDocFreq = minDocFreq))( seqOp = (df, v) => df.add(v), combOp = (df1, df2) => df1.merge(df2) ).idf() - new IDFModel(idf) + new IDFModel(idf, docFreq, numDocs) } /** @@ -128,13 +129,14 @@ private object IDF { private def isEmpty: Boolean = m == 0L - /** Returns the current IDF vector. */ - def idf(): Vector = { + /** Returns the current IDF vector, docFreq, number of documents */ + def idf(): (Vector, Array[Long], Long) = { if (isEmpty) { throw new IllegalStateException("Haven't seen any document yet.") } val n = df.length val inv = new Array[Double](n) + val dfv = new Array[Long](n) var j = 0 while (j < n) { /* @@ -148,10 +150,11 @@ private object IDF { */ if (df(j) >= minDocFreq) { inv(j) = math.log((m + 1.0) / (df(j) + 1.0)) + dfv(j) = df(j) } j += 1 } - Vectors.dense(inv) + (Vectors.dense(inv), dfv, m) } } } @@ -160,7 +163,9 @@ private object IDF { * Represents an IDF model that can transform term frequency vectors. */ @Since("1.1.0") -class IDFModel private[spark] (@Since("1.1.0") val idf: Vector) extends Serializable { +class IDFModel private[spark](@Since("1.1.0") val idf: Vector, + @Since("3.0.0") val docFreq: Array[Long], + @Since("3.0.0") val numDocs: Long) extends Serializable { /** * Transforms term frequency (TF) vectors to TF-IDF vectors. @@ -198,7 +203,7 @@ class IDFModel private[spark] (@Since("1.1.0") val idf: Vector) extends Serializ } } -private object IDFModel { +private[spark] object IDFModel { /** * Transforms a term frequency (TF) vector to a TF-IDF vector with a IDF vector @@ -208,28 +213,41 @@ private object IDFModel { * @return a TF-IDF vector */ def transform(idf: Vector, v: Vector): Vector = { - val n = v.size v match { case SparseVector(size, indices, values) => - val nnz = indices.length - val newValues = new Array[Double](nnz) - var k = 0 - while (k < nnz) { - newValues(k) = values(k) * idf(indices(k)) - k += 1 - } - Vectors.sparse(n, indices, newValues) + val (newIndices, newValues) = transformSparse(idf, indices, values) + Vectors.sparse(size, newIndices, newValues) case DenseVector(values) => - val newValues = new Array[Double](n) - var j = 0 - while (j < n) { - newValues(j) = values(j) * idf(j) - j += 1 - } + val newValues = transformDense(idf, values) Vectors.dense(newValues) case other => throw new UnsupportedOperationException( s"Only sparse and dense vectors are supported but got ${other.getClass}.") } } + + private[spark] def transformDense(idf: Vector, + values: Array[Double]): Array[Double] = { + val n = values.length + val newValues = new Array[Double](n) + var j = 0 + while (j < n) { + newValues(j) = values(j) * idf(j) + j += 1 + } + newValues + } + + private[spark] def transformSparse(idf: Vector, + indices: Array[Int], + values: Array[Double]): (Array[Int], Array[Double]) = { + val nnz = indices.length + val newValues = new Array[Double](nnz) + var k = 0 + while (k < nnz) { + newValues(k) = values(k) * idf(indices(k)) + k += 1 + } + (indices, newValues) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala index a01503f4b80a6..2fc517cad12db 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/PCA.scala @@ -21,6 +21,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.Statistics import org.apache.spark.rdd.RDD /** @@ -44,12 +45,21 @@ class PCA @Since("1.4.0") (@Since("1.4.0") val k: Int) { require(k <= numFeatures, s"source vector size $numFeatures must be no less than k=$k") - require(PCAUtil.memoryCost(k, numFeatures) < Int.MaxValue, - "The param k and numFeatures is too large for SVD computation. " + - "Try reducing the parameter k for PCA, or reduce the input feature " + - "vector dimension to make this tractable.") + val mat = if (numFeatures > 65535) { + val meanVector = Statistics.colStats(sources).mean.asBreeze + val meanCentredRdd = sources.map { rowVector => + Vectors.fromBreeze(rowVector.asBreeze - meanVector) + } + new RowMatrix(meanCentredRdd) + } else { + require(PCAUtil.memoryCost(k, numFeatures) < Int.MaxValue, + "The param k and numFeatures is too large for SVD computation. " + + "Try reducing the parameter k for PCA, or reduce the input feature " + + "vector dimension to make this tractable.") + + new RowMatrix(sources) + } - val mat = new RowMatrix(sources) val (pc, explainedVariance) = mat.computePrincipalComponentsAndExplainedVariance(k) val densePC = pc match { case dm: DenseMatrix => 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 7667936a3f85f..19e53e7eac844 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 @@ -135,58 +135,76 @@ class StandardScalerModel @Since("1.3.0") ( override def transform(vector: Vector): Vector = { require(mean.size == vector.size) if (withMean) { - // By default, Scala generates Java methods for member variables. So every time when - // the member variables are accessed, `invokespecial` will be called which is expensive. - // This can be avoid by having a local reference of `shift`. - val localShift = shift // Must have a copy of the values since it will be modified in place val values = vector match { // specially handle DenseVector because its toArray does not clone already case d: DenseVector => d.values.clone() case v: Vector => v.toArray } - val size = values.length - if (withStd) { - var i = 0 - while (i < size) { - values(i) = if (std(i) != 0.0) (values(i) - localShift(i)) * (1.0 / std(i)) else 0.0 - i += 1 - } - } else { - var i = 0 - while (i < size) { - values(i) -= localShift(i) - i += 1 - } - } - Vectors.dense(values) + val newValues = transformWithMean(values) + Vectors.dense(newValues) } else if (withStd) { vector match { - case DenseVector(vs) => - val values = vs.clone() - val size = values.length - var i = 0 - while(i < size) { - values(i) *= (if (std(i) != 0.0) 1.0 / std(i) else 0.0) - i += 1 - } - Vectors.dense(values) - case SparseVector(size, indices, vs) => - // For sparse vector, the `index` array inside sparse vector object will not be changed, - // so we can re-use it to save memory. - val values = vs.clone() - val nnz = values.length - var i = 0 - while (i < nnz) { - values(i) *= (if (std(indices(i)) != 0.0) 1.0 / std(indices(i)) else 0.0) - i += 1 - } - Vectors.sparse(size, indices, values) - case v => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + case DenseVector(values) => + val newValues = transformDenseWithStd(values) + Vectors.dense(newValues) + case SparseVector(size, indices, values) => + val (newIndices, newValues) = transformSparseWithStd(indices, values) + Vectors.sparse(size, newIndices, newValues) + case other => + throw new UnsupportedOperationException( + s"Only sparse and dense vectors are supported but got ${other.getClass}.") } } else { // Note that it's safe since we always assume that the data in RDD should be immutable. vector } } + + private[spark] def transformWithMean(values: Array[Double]): Array[Double] = { + // By default, Scala generates Java methods for member variables. So every time when + // the member variables are accessed, `invokespecial` will be called which is expensive. + // This can be avoid by having a local reference of `shift`. + val localShift = shift + val size = values.length + if (withStd) { + var i = 0 + while (i < size) { + values(i) = if (std(i) != 0.0) (values(i) - localShift(i)) * (1.0 / std(i)) else 0.0 + i += 1 + } + } else { + var i = 0 + while (i < size) { + values(i) -= localShift(i) + i += 1 + } + } + values + } + + private[spark] def transformDenseWithStd(values: Array[Double]): Array[Double] = { + val size = values.length + val newValues = values.clone() + var i = 0 + while(i < size) { + newValues(i) *= (if (std(i) != 0.0) 1.0 / std(i) else 0.0) + i += 1 + } + newValues + } + + private[spark] def transformSparseWithStd(indices: Array[Int], + values: Array[Double]): (Array[Int], Array[Double]) = { + // For sparse vector, the `index` array inside sparse vector object will not be changed, + // so we can re-use it to save memory. + val nnz = values.length + val newValues = values.clone() + var i = 0 + while (i < nnz) { + newValues(i) *= (if (std(indices(i)) != 0.0) 1.0 / std(indices(i)) else 0.0) + i += 1 + } + (indices, newValues) + } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala index b8c306d86bace..7888a8001d6b7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/feature/Word2Vec.scala @@ -32,6 +32,7 @@ import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo.KRYO_SERIALIZER_MAX_BUFFER_SIZE import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.{Loader, Saveable} import org.apache.spark.rdd._ @@ -45,7 +46,7 @@ import org.apache.spark.util.random.XORShiftRandom */ private case class VocabWord( var word: String, - var cn: Int, + var cn: Long, var point: Array[Int], var code: Array[Int], var codeLen: Int @@ -194,7 +195,7 @@ class Word2Vec extends Serializable with Logging { new Array[Int](MAX_CODE_LENGTH), 0)) .collect() - .sortWith((a, b) => a.cn > b.cn) + .sortBy(_.cn)(Ordering[Long].reverse) vocabSize = vocab.length require(vocabSize > 0, "The vocabulary size should be > 0. You may need to check " + @@ -232,7 +233,7 @@ class Word2Vec extends Serializable with Logging { a += 1 } while (a < 2 * vocabSize) { - count(a) = 1e9.toInt + count(a) = Long.MaxValue a += 1 } var pos1 = vocabSize - 1 @@ -267,6 +268,8 @@ class Word2Vec extends Serializable with Logging { min2i = pos2 pos2 += 1 } + assert(count(min1i) < Long.MaxValue) + assert(count(min2i) < Long.MaxValue) count(vocabSize + a) = count(min1i) + count(min2i) parentNode(min1i) = vocabSize + a parentNode(min2i) = vocabSize + a @@ -318,9 +321,9 @@ class Word2Vec extends Serializable with Logging { try { doFit(dataset, sc, expTable, bcVocab, bcVocabHash) } finally { - expTable.destroy(blocking = false) - bcVocab.destroy(blocking = false) - bcVocabHash.destroy(blocking = false) + expTable.destroy() + bcVocab.destroy() + bcVocabHash.destroy() } } @@ -450,8 +453,8 @@ class Word2Vec extends Serializable with Logging { } i += 1 } - bcSyn0Global.destroy(false) - bcSyn1Global.destroy(false) + bcSyn0Global.destroy() + bcSyn1Global.destroy() } newSentences.unpersist() @@ -511,8 +514,6 @@ class Word2VecModel private[spark] ( this(Word2VecModel.buildWordIndex(model), Word2VecModel.buildWordVectors(model)) } - override protected def formatVersion = "1.0" - @Since("1.4.0") def save(sc: SparkContext, path: String): Unit = { Word2VecModel.SaveLoadV1_0.save(sc, path, getVectors) @@ -681,7 +682,7 @@ object Word2VecModel extends Loader[Word2VecModel] { // We want to partition the model in partitions smaller than // spark.kryoserializer.buffer.max val bufferSize = Utils.byteStringAsBytes( - spark.conf.get("spark.kryoserializer.buffer.max", "64m")) + spark.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "64m")) // We calculate the approximate size of the model // We only calculate the array size, considering an // average string size of 15 bytes, the formula is: diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala index 3a1bc35186dc3..3531822e77b78 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPGrowth.scala @@ -84,8 +84,6 @@ class FPGrowthModel[Item: ClassTag] @Since("2.4.0") ( override def save(sc: SparkContext, path: String): Unit = { FPGrowthModel.SaveLoadV1_0.save(this, path) } - - override protected val formatVersion: String = "1.0" } @Since("2.0.0") @@ -152,10 +150,10 @@ object FPGrowthModel extends Loader[FPGrowthModel[_]] { /** * A parallel FP-growth algorithm to mine frequent itemsets. The algorithm is described in - * Li et al., PFP: Parallel FP-Growth for Query + * Li et al., PFP: Parallel FP-Growth for Query * Recommendation. PFP distributes computation in such a way that each worker executes an * independent group of mining tasks. The FP-Growth algorithm is described in - * Han et al., Mining frequent patterns without + * Han et al., Mining frequent patterns without * candidate generation. * * @param minSupport the minimal support level of the frequent pattern, any pattern that appears diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala index b0fa287473c3c..6ae8e204bd7cc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/FPTree.scala @@ -33,7 +33,7 @@ private[fpm] class FPTree[T] extends Serializable { private val summaries: mutable.Map[T, Summary[T]] = mutable.Map.empty /** Adds a transaction with count. */ - def add(t: Iterable[T], count: Long = 1L): this.type = { + def add(t: Iterable[T], count: Long = 1L): FPTree[T] = { require(count > 0) var curr = root curr.count += count @@ -53,7 +53,7 @@ private[fpm] class FPTree[T] extends Serializable { } /** Merges another FP-Tree. */ - def merge(other: FPTree[T]): this.type = { + def merge(other: FPTree[T]): FPTree[T] = { other.transactions.foreach { case (t, c) => add(t, c) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala index 7aed2f3bd8a61..69e4b76b2d8c2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/fpm/PrefixSpan.scala @@ -45,7 +45,7 @@ import org.apache.spark.storage.StorageLevel * A parallel PrefixSpan algorithm to mine frequent sequential patterns. * The PrefixSpan algorithm is described in J. Pei, et al., PrefixSpan: Mining Sequential Patterns * Efficiently by Prefix-Projected Pattern Growth - * (see here). + * (see here). * * @param minSupport the minimal support level of the sequential pattern, any pattern that appears * more than (minSupport * size-of-the-dataset) times will be output @@ -174,6 +174,13 @@ class PrefixSpan private ( val freqSequences = results.map { case (seq: Array[Int], count: Long) => new FreqSequence(toPublicRepr(seq), count) } + // Cache the final RDD to the same storage level as input + if (data.getStorageLevel != StorageLevel.NONE) { + freqSequences.persist(data.getStorageLevel) + freqSequences.count() + } + dataInternalRepr.unpersist() + new PrefixSpanModel(freqSequences) } @@ -621,8 +628,6 @@ class PrefixSpanModel[Item] @Since("1.5.0") ( override def save(sc: SparkContext, path: String): Unit = { PrefixSpanModel.SaveLoadV1_0.save(this, path) } - - override protected val formatVersion: String = "1.0" } @Since("2.0.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala index bf9b4cfe15b2c..e474cfa002fad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -30,6 +30,7 @@ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeArrayData} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.array.ByteArrayMethods /** * Trait for a local matrix. @@ -456,7 +457,7 @@ object DenseMatrix { */ @Since("1.3.0") def zeros(numRows: Int, numCols: Int): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, new Array[Double](numRows * numCols)) } @@ -469,7 +470,7 @@ object DenseMatrix { */ @Since("1.3.0") def ones(numRows: Int, numCols: Int): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(1.0)) } @@ -499,7 +500,7 @@ object DenseMatrix { */ @Since("1.3.0") def rand(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextDouble())) } @@ -513,7 +514,7 @@ object DenseMatrix { */ @Since("1.3.0") def randn(numRows: Int, numCols: Int, rng: Random): DenseMatrix = { - require(numRows.toLong * numCols <= Int.MaxValue, + require(numRows.toLong * numCols <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, s"$numRows x $numCols dense matrix is too large to allocate") new DenseMatrix(numRows, numCols, Array.fill(numRows * numCols)(rng.nextGaussian())) } @@ -846,8 +847,8 @@ object SparseMatrix { s"density must be a double in the range 0.0 <= d <= 1.0. Currently, density: $density") val size = numRows.toLong * numCols val expected = size * density - assert(expected < Int.MaxValue, - "The expected number of nonzeros cannot be greater than Int.MaxValue.") + assert(expected < ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, + "The expected number of nonzeros cannot be greater than Int.MaxValue - 15.") val nnz = math.ceil(expected).toInt if (density == 0.0) { new SparseMatrix(numRows, numCols, new Array[Int](numCols + 1), Array.empty, Array.empty) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 6e68d9684a672..9cdf1944329b8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -117,7 +117,7 @@ sealed trait Vector extends Serializable { */ @Since("1.1.0") def copy: Vector = { - throw new NotImplementedError(s"copy is not implemented for ${this.getClass}.") + throw new UnsupportedOperationException(s"copy is not implemented for ${this.getClass}.") } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala index 7caacd13b3459..e32d615af2a47 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/BlockMatrix.scala @@ -17,10 +17,9 @@ package org.apache.spark.mllib.linalg.distributed +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM} import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, Matrix => BM, SparseVector => BSV, Vector => BV} - import org.apache.spark.{Partitioner, SparkException} import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging @@ -28,6 +27,7 @@ import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel + /** * A grid partitioner, which uses a regular grid to partition coordinates. * @@ -273,24 +273,37 @@ class BlockMatrix @Since("1.3.0") ( require(cols < Int.MaxValue, s"The number of columns should be less than Int.MaxValue ($cols).") val rows = blocks.flatMap { case ((blockRowIdx, blockColIdx), mat) => - mat.rowIter.zipWithIndex.map { + mat.rowIter.zipWithIndex.filter(_._1.size > 0).map { case (vector, rowIdx) => - blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector.asBreeze)) + blockRowIdx * rowsPerBlock + rowIdx -> ((blockColIdx, vector)) } }.groupByKey().map { case (rowIdx, vectors) => - val numberNonZeroPerRow = vectors.map(_._2.activeSize).sum.toDouble / cols.toDouble - - val wholeVector = if (numberNonZeroPerRow <= 0.1) { // Sparse at 1/10th nnz - BSV.zeros[Double](cols) - } else { - BDV.zeros[Double](cols) - } + val numberNonZero = vectors.map(_._2.numActives).sum + val numberNonZeroPerRow = numberNonZero.toDouble / cols.toDouble + + val wholeVector = + if (numberNonZeroPerRow <= 0.1) { // Sparse at 1/10th nnz + val arrBufferIndices = new ArrayBuffer[Int](numberNonZero) + val arrBufferValues = new ArrayBuffer[Double](numberNonZero) + + vectors.foreach { case (blockColIdx: Int, vec: Vector) => + val offset = colsPerBlock * blockColIdx + vec.foreachActive { case (colIdx: Int, value: Double) => + arrBufferIndices += offset + colIdx + arrBufferValues += value + } + } + Vectors.sparse(cols, arrBufferIndices.toArray, arrBufferValues.toArray) + } else { + val wholeVectorBuf = BDV.zeros[Double](cols) + vectors.foreach { case (blockColIdx: Int, vec: Vector) => + val offset = colsPerBlock * blockColIdx + wholeVectorBuf(offset until Math.min(cols, offset + colsPerBlock)) := vec.asBreeze + } + Vectors.fromBreeze(wholeVectorBuf) + } - vectors.foreach { case (blockColIdx: Int, vec: BV[_]) => - val offset = colsPerBlock * blockColIdx - wholeVector(offset until Math.min(cols, offset + colsPerBlock)) := vec - } - new IndexedRow(rowIdx, Vectors.fromBreeze(wholeVector)) + IndexedRow(rowIdx, wholeVector) } new IndexedRowMatrix(rows) } @@ -309,7 +322,7 @@ class BlockMatrix @Since("1.3.0") ( val m = numRows().toInt val n = numCols().toInt val mem = m * n / 125000 - if (mem > 500) logWarning(s"Storing this matrix will require $mem MB of memory!") + if (mem > 500) logWarning(s"Storing this matrix will require $mem MiB of memory!") val localBlocks = blocks.collect() val values = new Array[Double](m * n) localBlocks.foreach { case ((blockRowIndex, blockColIndex), submat) => diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 78a8810052aef..43f48befd014f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -27,6 +27,7 @@ import breeze.numerics.{sqrt => brzSqrt} import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.MAX_RESULT_SIZE import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.stat.{MultivariateOnlineSummarizer, MultivariateStatisticalSummary} import org.apache.spark.rdd.RDD @@ -117,17 +118,118 @@ class RowMatrix @Since("1.0.0") ( // Computes n*(n+1)/2, avoiding overflow in the multiplication. // This succeeds when n <= 65535, which is checked above val nt = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) + val gramianSizeInBytes = nt * 8L // Compute the upper triangular part of the gram matrix. - val GU = rows.treeAggregate(new BDV[Double](nt))( - seqOp = (U, v) => { + val GU = rows.treeAggregate(null.asInstanceOf[BDV[Double]])( + seqOp = (maybeU, v) => { + val U = + if (maybeU == null) { + new BDV[Double](nt) + } else { + maybeU + } BLAS.spr(1.0, v, U.data) U - }, combOp = (U1, U2) => U1 += U2) + }, combOp = (U1, U2) => + if (U1 == null) { + U2 + } else if (U2 == null) { + U1 + } else { + U1 += U2 + }, + depth = getTreeAggregateIdealDepth(gramianSizeInBytes) + ) RowMatrix.triuToFull(n, GU.data) } + private def computeDenseVectorCovariance(mean: Vector, n: Int, m: Long): Matrix = { + + val bc = rows.context.broadcast(mean) + + // Computes n*(n+1)/2, avoiding overflow in the multiplication. + // This succeeds when n <= 65535, which is checked above + val nt = if (n % 2 == 0) ((n / 2) * (n + 1)) else (n * ((n + 1) / 2)) + + val MU = rows.treeAggregate(null.asInstanceOf[BDV[Double]])( + seqOp = (maybeU, v) => { + val U = + if (maybeU == null) { + new BDV[Double](nt) + } else { + maybeU + } + + val n = v.size + val na = Array.ofDim[Double](n) + val means = bc.value + + val ta = v.toArray + for (index <- 0 until n) { + na(index) = ta(index) - means(index) + } + + BLAS.spr(1.0, new DenseVector(na), U.data) + U + }, combOp = (U1, U2) => + if (U1 == null) { + U2 + } else if (U2 == null) { + U1 + } else { + U1 += U2 + } + ) + + bc.destroy() + + val M = RowMatrix.triuToFull(n, MU.data).asBreeze + + var i = 0 + var j = 0 + val m1 = m - 1.0 + while (i < n) { + j = i + while (j < n) { + val Mij = M(i, j) / m1 + M(i, j) = Mij + M(j, i) = Mij + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(M) + } + + private def computeSparseVectorCovariance(mean: Vector, n: Int, m: Long): Matrix = { + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + val G = computeGramianMatrix().asBreeze + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = i + while (j < n) { + val Gij = G(i, j) / m1 - alpha * mean(j) + G(i, j) = Gij + G(j, i) = Gij + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + private def checkNumColumns(cols: Int): Unit = { if (cols > 65535) { throw new IllegalArgumentException(s"Argument with more than 65535 cols: $cols") @@ -337,29 +439,11 @@ class RowMatrix @Since("1.0.0") ( " Cannot compute the covariance of a RowMatrix with <= 1 row.") val mean = summary.mean - // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is - // large but Cov(X, Y) is small, but it is good for sparse computation. - // TODO: find a fast and stable way for sparse data. - - val G = computeGramianMatrix().asBreeze - - var i = 0 - var j = 0 - val m1 = m - 1.0 - var alpha = 0.0 - while (i < n) { - alpha = m / m1 * mean(i) - j = i - while (j < n) { - val Gij = G(i, j) / m1 - alpha * mean(j) - G(i, j) = Gij - G(j, i) = Gij - j += 1 - } - i += 1 + if (rows.first().isInstanceOf[DenseVector]) { + computeDenseVectorCovariance(mean, n, m) + } else { + computeSparseVectorCovariance(mean, n, m) } - - Matrices.fromBreeze(G) } /** @@ -370,32 +454,41 @@ class RowMatrix @Since("1.0.0") ( * Each column corresponds for one principal component, * and the columns are in descending order of component variance. * The row data do not need to be "centered" first; it is not necessary for - * the mean of each column to be 0. + * the mean of each column to be 0. But, if the number of columns are more than + * 65535, then the data need to be "centered". * * @param k number of top principal components. * @return a matrix of size n-by-k, whose columns are principal components, and * a vector of values which indicate how much variance each principal component * explains - * - * @note This cannot be computed on matrices with more than 65535 columns. */ @Since("1.6.0") def computePrincipalComponentsAndExplainedVariance(k: Int): (Matrix, Vector) = { val n = numCols().toInt require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") - val Cov = computeCovariance().asBreeze.asInstanceOf[BDM[Double]] + if (n > 65535) { + val svd = computeSVD(k) + val s = svd.s.toArray.map(eigValue => eigValue * eigValue / (n - 1)) + val eigenSum = s.sum + val explainedVariance = s.map(_ / eigenSum) - val brzSvd.SVD(u: BDM[Double], s: BDV[Double], _) = brzSvd(Cov) + (svd.V, Vectors.dense(explainedVariance)) + } else { - val eigenSum = s.data.sum - val explainedVariance = s.data.map(_ / eigenSum) + val Cov = computeCovariance().asBreeze.asInstanceOf[BDM[Double]] - if (k == n) { - (Matrices.dense(n, k, u.data), Vectors.dense(explainedVariance)) - } else { - (Matrices.dense(n, k, Arrays.copyOfRange(u.data, 0, n * k)), - Vectors.dense(Arrays.copyOfRange(explainedVariance, 0, k))) + val brzSvd.SVD(u: BDM[Double], s: BDV[Double], _) = brzSvd(Cov) + + val eigenSum = s.data.sum + val explainedVariance = s.data.map(_ / eigenSum) + + if (k == n) { + (Matrices.dense(n, k, u.data), Vectors.dense(explainedVariance)) + } else { + (Matrices.dense(n, k, Arrays.copyOfRange(u.data, 0, n * k)), + Vectors.dense(Arrays.copyOfRange(explainedVariance, 0, k))) + } } } @@ -531,7 +624,7 @@ class RowMatrix @Since("1.0.0") ( * decomposition (factorization) for the [[RowMatrix]] of a tall and skinny shape. * Reference: * Paul G. Constantine, David F. Gleich. "Tall and skinny QR factorizations in MapReduce - * architectures" (see here) + * architectures" (see here) * * @param computeQ whether to computeQ * @return QRDecomposition(Q, R), Q = null if computeQ = false. @@ -685,6 +778,35 @@ class RowMatrix @Since("1.0.0") ( s"The number of rows $m is different from what specified or previously computed: ${nRows}.") } } + + /** + * Computing desired tree aggregate depth necessary to avoid exceeding + * driver.MaxResultSize during aggregation. + * Based on the formulae: (numPartitions)^(1/depth) * objectSize <= DriverMaxResultSize + * @param aggregatedObjectSizeInBytes the size, in megabytes, of the object being tree aggregated + */ + private[spark] def getTreeAggregateIdealDepth(aggregatedObjectSizeInBytes: Long) = { + require(aggregatedObjectSizeInBytes > 0, + "Cannot compute aggregate depth heuristic based on a zero-size object to aggregate") + + val maxDriverResultSizeInBytes = rows.conf.get[Long](MAX_RESULT_SIZE) + + require(maxDriverResultSizeInBytes > aggregatedObjectSizeInBytes, + s"Cannot aggregate object of size $aggregatedObjectSizeInBytes Bytes, " + + s"as it's bigger than maxResultSize ($maxDriverResultSizeInBytes Bytes)") + + val numerator = math.log(rows.getNumPartitions) + val denominator = math.log(maxDriverResultSizeInBytes) - math.log(aggregatedObjectSizeInBytes) + val desiredTreeDepth = math.ceil(numerator / denominator) + + if (desiredTreeDepth > 4) { + logWarning( + s"Desired tree depth for treeAggregation is big ($desiredTreeDepth)." + + "Consider increasing driver max result size or reducing number of partitions") + } + + math.min(math.max(1, desiredTreeDepth), 10).toInt + } } @Since("1.0.0") 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 593cdd602fafc..ffe3964981a18 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 @@ -246,7 +246,7 @@ object GradientDescent extends Logging { // c: (grad, loss, count) (c1._1 += c2._1, c1._2 + c2._2, c1._3 + c2._3) }) - bcWeights.destroy(blocking = false) + bcWeights.destroy() if (miniBatchSize > 0) { /** 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 21ec287e497d4..65c1159723c80 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 @@ -261,7 +261,7 @@ object LBFGS extends Logging { val (gradientSum, lossSum) = data.treeAggregate((zeroSparseVector, 0.0))(seqOp, combOp) // broadcasted model is not needed anymore - bcW.destroy(blocking = false) + bcW.destroy() /** * regVal is sum of weight squares if it's L2 updater; diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala index a8c32f72bfdeb..27935c6f5291a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExport.scala @@ -19,7 +19,9 @@ package org.apache.spark.mllib.pmml.export import scala.{Array => SArray} -import org.dmg.pmml._ +import org.dmg.pmml.{DataDictionary, DataField, DataType, FieldName, MiningField, + MiningFunction, MiningSchema, OpType} +import org.dmg.pmml.regression.{NumericPredictor, RegressionModel, RegressionTable} import org.apache.spark.mllib.regression.GeneralizedLinearModel @@ -29,7 +31,7 @@ import org.apache.spark.mllib.regression.GeneralizedLinearModel private[mllib] class BinaryClassificationPMMLModelExport( model: GeneralizedLinearModel, description: String, - normalizationMethod: RegressionNormalizationMethodType, + normalizationMethod: RegressionModel.NormalizationMethod, threshold: Double) extends PMMLModelExport { @@ -47,7 +49,7 @@ private[mllib] class BinaryClassificationPMMLModelExport( val miningSchema = new MiningSchema val regressionTableYES = new RegressionTable(model.intercept).setTargetCategory("1") var interceptNO = threshold - if (RegressionNormalizationMethodType.LOGIT == normalizationMethod) { + if (RegressionModel.NormalizationMethod.LOGIT == normalizationMethod) { if (threshold <= 0) { interceptNO = Double.MinValue } else if (threshold >= 1) { @@ -58,7 +60,7 @@ private[mllib] class BinaryClassificationPMMLModelExport( } val regressionTableNO = new RegressionTable(interceptNO).setTargetCategory("0") val regressionModel = new RegressionModel() - .setFunctionName(MiningFunctionType.CLASSIFICATION) + .setMiningFunction(MiningFunction.CLASSIFICATION) .setMiningSchema(miningSchema) .setModelName(description) .setNormalizationMethod(normalizationMethod) @@ -69,7 +71,7 @@ private[mllib] class BinaryClassificationPMMLModelExport( dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema .addMiningFields(new MiningField(fields(i)) - .setUsageType(FieldUsageType.ACTIVE)) + .setUsageType(MiningField.UsageType.ACTIVE)) regressionTableYES.addNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } @@ -79,7 +81,7 @@ private[mllib] class BinaryClassificationPMMLModelExport( .addDataFields(new DataField(targetField, OpType.CATEGORICAL, DataType.STRING)) miningSchema .addMiningFields(new MiningField(targetField) - .setUsageType(FieldUsageType.TARGET)) + .setUsageType(MiningField.UsageType.TARGET)) dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala index 4d951d2973a6f..723224de168e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExport.scala @@ -19,7 +19,9 @@ package org.apache.spark.mllib.pmml.export import scala.{Array => SArray} -import org.dmg.pmml._ +import org.dmg.pmml.{DataDictionary, DataField, DataType, FieldName, MiningField, + MiningFunction, MiningSchema, OpType} +import org.dmg.pmml.regression.{NumericPredictor, RegressionModel, RegressionTable} import org.apache.spark.mllib.regression.GeneralizedLinearModel @@ -45,7 +47,7 @@ private[mllib] class GeneralizedLinearPMMLModelExport( val miningSchema = new MiningSchema val regressionTable = new RegressionTable(model.intercept) val regressionModel = new RegressionModel() - .setFunctionName(MiningFunctionType.REGRESSION) + .setMiningFunction(MiningFunction.REGRESSION) .setMiningSchema(miningSchema) .setModelName(description) .addRegressionTables(regressionTable) @@ -55,7 +57,7 @@ private[mllib] class GeneralizedLinearPMMLModelExport( dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema .addMiningFields(new MiningField(fields(i)) - .setUsageType(FieldUsageType.ACTIVE)) + .setUsageType(MiningField.UsageType.ACTIVE)) regressionTable.addNumericPredictors(new NumericPredictor(fields(i), model.weights(i))) } @@ -64,7 +66,7 @@ private[mllib] class GeneralizedLinearPMMLModelExport( dataDictionary.addDataFields(new DataField(targetField, OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema .addMiningFields(new MiningField(targetField) - .setUsageType(FieldUsageType.TARGET)) + .setUsageType(MiningField.UsageType.TARGET)) dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala index 255c6140e5410..d043c9e58eebf 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExport.scala @@ -19,7 +19,9 @@ package org.apache.spark.mllib.pmml.export import scala.{Array => SArray} -import org.dmg.pmml._ +import org.dmg.pmml.{Array, CompareFunction, ComparisonMeasure, DataDictionary, DataField, DataType, + FieldName, MiningField, MiningFunction, MiningSchema, OpType, SquaredEuclidean} +import org.dmg.pmml.clustering.{Cluster, ClusteringField, ClusteringModel} import org.apache.spark.mllib.clustering.KMeansModel @@ -48,7 +50,7 @@ private[mllib] class KMeansPMMLModelExport(model: KMeansModel) extends PMMLModel .setModelName("k-means") .setMiningSchema(miningSchema) .setComparisonMeasure(comparisonMeasure) - .setFunctionName(MiningFunctionType.CLUSTERING) + .setMiningFunction(MiningFunction.CLUSTERING) .setModelClass(ClusteringModel.ModelClass.CENTER_BASED) .setNumberOfClusters(model.clusterCenters.length) @@ -57,9 +59,9 @@ private[mllib] class KMeansPMMLModelExport(model: KMeansModel) extends PMMLModel dataDictionary.addDataFields(new DataField(fields(i), OpType.CONTINUOUS, DataType.DOUBLE)) miningSchema .addMiningFields(new MiningField(fields(i)) - .setUsageType(FieldUsageType.ACTIVE)) + .setUsageType(MiningField.UsageType.ACTIVE)) clusteringModel.addClusteringFields( - new ClusteringField(fields(i)).setCompareFunction(CompareFunctionType.ABS_DIFF)) + new ClusteringField(fields(i)).setCompareFunction(CompareFunction.ABS_DIFF)) } dataDictionary.setNumberOfFields(dataDictionary.getDataFields.size) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala index 29bd689e1185a..84e63041dc2d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/pmml/export/PMMLModelExportFactory.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.RegressionNormalizationMethodType +import org.dmg.pmml.regression.RegressionModel import org.apache.spark.mllib.classification.LogisticRegressionModel import org.apache.spark.mllib.classification.SVMModel @@ -44,12 +44,12 @@ private[mllib] object PMMLModelExportFactory { new GeneralizedLinearPMMLModelExport(lasso, "lasso regression") case svm: SVMModel => new BinaryClassificationPMMLModelExport( - svm, "linear SVM", RegressionNormalizationMethodType.NONE, + svm, "linear SVM", RegressionModel.NormalizationMethod.NONE, svm.getThreshold.getOrElse(0.0)) case logistic: LogisticRegressionModel => if (logistic.numClasses == 2) { new BinaryClassificationPMMLModelExport( - logistic, "logistic regression", RegressionNormalizationMethodType.LOGIT, + logistic, "logistic regression", RegressionModel.NormalizationMethod.LOGIT, logistic.getThreshold.getOrElse(0.5)) } else { throw new IllegalArgumentException( 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 14288221b6945..12870f819b147 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 @@ -54,7 +54,7 @@ case class Rating @Since("0.8.0") ( * * For implicit preference data, the algorithm used is based on * "Collaborative Filtering for Implicit Feedback Datasets", available at - * here, adapted for the blocked approach + * here, adapted for the blocked approach * used here. * * Essentially instead of finding the low-rank approximations to the rating matrix `R`, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 7b49d4d0812f9..e5e82d19f1cbd 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -196,8 +196,6 @@ class MatrixFactorizationModel @Since("0.8.0") ( .map(t => Rating(t._1, product, t._2)) } - protected override val formatVersion: String = "1.0" - /** * Save this model to the given path. * 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 4d5aaf14111c2..ef3ff2b98a5e6 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 @@ -366,7 +366,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Unpersist cached data if (data.getStorageLevel != StorageLevel.NONE) { - data.unpersist(false) + data.unpersist() } createModel(weights, intercept) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala index 8347ccad6b715..649f9816e6a5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/IsotonicRegression.scala @@ -160,8 +160,6 @@ class IsotonicRegressionModel @Since("1.3.0") ( override def save(sc: SparkContext, path: String): Unit = { IsotonicRegressionModel.SaveLoadV1_0.save(sc, path, boundaries, predictions, isotonic) } - - override protected def formatVersion: String = "1.0" } @Since("1.4.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 4381d6ab20cc0..b320057b25276 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.regression -import scala.beans.BeanInfo - import org.apache.spark.SparkException import org.apache.spark.annotation.Since import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} @@ -32,10 +30,14 @@ import org.apache.spark.mllib.util.NumericParser * @param features List of features for this data point. */ @Since("0.8.0") -@BeanInfo case class LabeledPoint @Since("1.0.0") ( @Since("0.8.0") label: Double, @Since("1.0.0") features: Vector) { + + def getLabel: Double = label + + def getFeatures: Vector = features + override def toString: String = { s"($label,$features)" } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index cef1b4f51b843..ead9f5b300375 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -51,8 +51,6 @@ class LassoModel @Since("1.1.0") ( override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } - - override protected def formatVersion: String = "1.0" } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 60262fdc497a6..cb08216fbf690 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -51,8 +51,6 @@ class LinearRegressionModel @Since("1.1.0") ( override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } - - override protected def formatVersion: String = "1.0" } @Since("1.3.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 52977ac4f062a..43c3154dd053b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -52,8 +52,6 @@ class RidgeRegressionModel @Since("1.1.0") ( override def save(sc: SparkContext, path: String): Unit = { GLMRegressionModel.SaveLoadV1_0.save(sc, path, this.getClass.getName, weights, intercept) } - - override protected def formatVersion: String = "1.0" } @Since("1.3.0") 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 8121880cfb233..6d510e1633d67 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 @@ -52,7 +52,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S private var totalCnt: Long = 0 private var totalWeightSum: Double = 0.0 private var weightSquareSum: Double = 0.0 - private var weightSum: Array[Double] = _ + private var currWeightSum: Array[Double] = _ private var nnz: Array[Long] = _ private var currMax: Array[Double] = _ private var currMin: Array[Double] = _ @@ -78,7 +78,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currM2n = Array.ofDim[Double](n) currM2 = Array.ofDim[Double](n) currL1 = Array.ofDim[Double](n) - weightSum = Array.ofDim[Double](n) + currWeightSum = Array.ofDim[Double](n) nnz = Array.ofDim[Long](n) currMax = Array.fill[Double](n)(Double.MinValue) currMin = Array.fill[Double](n)(Double.MaxValue) @@ -91,7 +91,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val localCurrM2n = currM2n val localCurrM2 = currM2 val localCurrL1 = currL1 - val localWeightSum = weightSum + val localWeightSum = currWeightSum val localNumNonzeros = nnz val localCurrMax = currMax val localCurrMin = currMin @@ -139,8 +139,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S weightSquareSum += other.weightSquareSum var i = 0 while (i < n) { - val thisNnz = weightSum(i) - val otherNnz = other.weightSum(i) + val thisNnz = currWeightSum(i) + val otherNnz = other.currWeightSum(i) val totalNnz = thisNnz + otherNnz val totalCnnz = nnz(i) + other.nnz(i) if (totalNnz != 0.0) { @@ -157,7 +157,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S currMax(i) = math.max(currMax(i), other.currMax(i)) currMin(i) = math.min(currMin(i), other.currMin(i)) } - weightSum(i) = totalNnz + currWeightSum(i) = totalNnz nnz(i) = totalCnnz i += 1 } @@ -170,7 +170,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S this.totalCnt = other.totalCnt this.totalWeightSum = other.totalWeightSum this.weightSquareSum = other.weightSquareSum - this.weightSum = other.weightSum.clone() + this.currWeightSum = other.currWeightSum.clone() this.nnz = other.nnz.clone() this.currMax = other.currMax.clone() this.currMin = other.currMin.clone() @@ -189,7 +189,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val realMean = Array.ofDim[Double](n) var i = 0 while (i < n) { - realMean(i) = currMean(i) * (weightSum(i) / totalWeightSum) + realMean(i) = currMean(i) * (currWeightSum(i) / totalWeightSum) i += 1 } Vectors.dense(realMean) @@ -214,8 +214,8 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S val len = currM2n.length while (i < len) { // We prevent variance from negative value caused by numerical error. - realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * weightSum(i) * - (totalWeightSum - weightSum(i)) / totalWeightSum) / denominator, 0.0) + realVariance(i) = math.max((currM2n(i) + deltaMean(i) * deltaMean(i) * currWeightSum(i) * + (totalWeightSum - currWeightSum(i)) / totalWeightSum) / denominator, 0.0) i += 1 } } @@ -229,6 +229,11 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S @Since("1.1.0") override def count: Long = totalCnt + /** + * Sum of weights. + */ + override def weightSum: Double = totalWeightSum + /** * Number of nonzero elements in each dimension. * @@ -273,7 +278,7 @@ class MultivariateOnlineSummarizer extends MultivariateStatisticalSummary with S } /** - * L2 (Euclidian) norm of each dimension. + * L2 (Euclidean) norm of each dimension. * */ @Since("1.2.0") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala index 39a16fb743d64..a4381032f8c0d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -44,6 +44,12 @@ trait MultivariateStatisticalSummary { @Since("1.0.0") def count: Long + /** + * Sum of weights. + */ + @Since("3.0.0") + def weightSum: Double + /** * Number of nonzero elements (including explicitly presented zero values) in each column. */ 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 4cf662e036346..9a746dcf35556 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 @@ -43,7 +43,7 @@ class MultivariateGaussian @Since("1.3.0") ( require(sigma.numCols == sigma.numRows, "Covariance matrix must be square") require(mu.size == sigma.numCols, "Mean vector length must match covariance matrix size") - private val breezeMu = mu.asBreeze.toDenseVector + @transient private lazy val breezeMu = mu.asBreeze.toDenseVector /** * private[mllib] constructor @@ -60,7 +60,7 @@ class MultivariateGaussian @Since("1.3.0") ( * rootSigmaInv = D^(-1/2)^ * U.t, where sigma = U * D * U.t * u = log((2*pi)^(-k/2)^ * det(sigma)^(-1/2)^) */ - private val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants + @transient private lazy val (rootSigmaInv: DBM[Double], u: Double) = calculateCovarianceConstants /** * Returns density of this multivariate Gaussian at given point, x diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala index 80c6ef0ea1aa1..85ed11d6553d9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTest.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.stat.test -import scala.beans.BeanInfo - import org.apache.spark.annotation.Since import org.apache.spark.internal.Logging import org.apache.spark.streaming.api.java.JavaDStream @@ -32,10 +30,11 @@ import org.apache.spark.util.StatCounter * @param value numeric value of the observation. */ @Since("1.6.0") -@BeanInfo case class BinarySample @Since("1.6.0") ( @Since("1.6.0") isExperiment: Boolean, @Since("1.6.0") value: Double) { + def getIsExperiment: Boolean = isExperiment + def getValue: Double = value override def toString: String = { s"($isExperiment, $value)" } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala index 14ac14d6d61f4..8f3d0f8b3214c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/test/StreamingTestMethod.scala @@ -33,7 +33,7 @@ import org.apache.spark.util.StatCounter /** * Significance testing methods for [[StreamingTest]]. New 2-sample statistical significance tests * should extend [[StreamingTestMethod]] and introduce a new entry in - * [[StreamingTestMethod.TEST_NAME_TO_OBJECT]] + * `StreamingTestMethod.TEST_NAME_TO_OBJECT` */ private[stat] sealed trait StreamingTestMethod extends Serializable { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala index a8c5286f3dc10..94224be80752a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/RandomForest.scala @@ -23,6 +23,7 @@ import scala.util.Try import org.apache.spark.annotation.Since import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging +import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.tree.{DecisionTreeModel => NewDTModel, TreeEnsembleParams => NewRFParams} import org.apache.spark.ml.tree.impl.{RandomForest => NewRandomForest} import org.apache.spark.mllib.regression.LabeledPoint @@ -91,8 +92,8 @@ private class RandomForest ( * @return RandomForestModel that can be used for prediction. */ def run(input: RDD[LabeledPoint]): RandomForestModel = { - val trees: Array[NewDTModel] = NewRandomForest.run(input.map(_.asML), strategy, numTrees, - featureSubsetStrategy, seed.toLong, None) + val trees: Array[NewDTModel] = + NewRandomForest.run(input, strategy, numTrees, featureSubsetStrategy, seed.toLong) new RandomForestModel(strategy.algo, trees.map(_.toOld)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 58e8f5be7b9f0..d9dcb80013400 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -80,7 +80,8 @@ class Strategy @Since("1.3.0") ( @Since("1.0.0") @BeanProperty var maxMemoryInMB: Int = 256, @Since("1.2.0") @BeanProperty var subsamplingRate: Double = 1, @Since("1.2.0") @BeanProperty var useNodeIdCache: Boolean = false, - @Since("1.2.0") @BeanProperty var checkpointInterval: Int = 10) extends Serializable { + @Since("1.2.0") @BeanProperty var checkpointInterval: Int = 10, + @Since("3.0.0") @BeanProperty var minWeightFractionPerNode: Double = 0.0) extends Serializable { /** */ @@ -96,6 +97,31 @@ class Strategy @Since("1.3.0") ( isMulticlassClassification && (categoricalFeaturesInfo.size > 0) } + // scalastyle:off argcount + /** + * Backwards compatible constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] + */ + @Since("1.0.0") + def this( + algo: Algo, + impurity: Impurity, + maxDepth: Int, + numClasses: Int, + maxBins: Int, + quantileCalculationStrategy: QuantileStrategy, + categoricalFeaturesInfo: Map[Int, Int], + minInstancesPerNode: Int, + minInfoGain: Double, + maxMemoryInMB: Int, + subsamplingRate: Double, + useNodeIdCache: Boolean, + checkpointInterval: Int) { + this(algo, impurity, maxDepth, numClasses, maxBins, quantileCalculationStrategy, + categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, maxMemoryInMB, + subsamplingRate, useNodeIdCache, checkpointInterval, 0.0) + } + // scalastyle:on argcount + /** * Java-friendly constructor for [[org.apache.spark.mllib.tree.configuration.Strategy]] */ @@ -108,7 +134,8 @@ class Strategy @Since("1.3.0") ( maxBins: Int, categoricalFeaturesInfo: java.util.Map[java.lang.Integer, java.lang.Integer]) { this(algo, impurity, maxDepth, numClasses, maxBins, Sort, - categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap) + categoricalFeaturesInfo.asInstanceOf[java.util.Map[Int, Int]].asScala.toMap, + minWeightFractionPerNode = 0.0) } /** @@ -171,8 +198,9 @@ class Strategy @Since("1.3.0") ( @Since("1.2.0") def copy: Strategy = { new Strategy(algo, impurity, maxDepth, numClasses, maxBins, - quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, minInfoGain, - maxMemoryInMB, subsamplingRate, useNodeIdCache, checkpointInterval) + quantileCalculationStrategy, categoricalFeaturesInfo, minInstancesPerNode, + minInfoGain, maxMemoryInMB, subsamplingRate, useNodeIdCache, + checkpointInterval, minWeightFractionPerNode) } } 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 d4448da9eef51..f01a98e74886b 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 @@ -83,23 +83,29 @@ object Entropy extends Impurity { * @param numClasses Number of classes for label. */ private[spark] class EntropyAggregator(numClasses: Int) - extends ImpurityAggregator(numClasses) with Serializable { + extends ImpurityAggregator(numClasses + 1) with Serializable { /** * Update stats for one (node, feature, bin) with the given label. * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { - if (label >= statsSize) { + def update( + allStats: Array[Double], + offset: Int, + label: Double, + numSamples: Int, + sampleWeight: Double): Unit = { + if (label >= numClasses) { throw new IllegalArgumentException(s"EntropyAggregator given label $label" + - s" but requires label < numClasses (= $statsSize).") + s" but requires label < numClasses (= ${numClasses}).") } if (label < 0) { throw new IllegalArgumentException(s"EntropyAggregator given label $label" + s"but requires label is non-negative.") } - allStats(offset + label.toInt) += instanceWeight + allStats(offset + label.toInt) += numSamples * sampleWeight + allStats(offset + statsSize - 1) += numSamples } /** @@ -108,7 +114,8 @@ private[spark] class EntropyAggregator(numClasses: Int) * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): EntropyCalculator = { - new EntropyCalculator(allStats.view(offset, offset + statsSize).toArray) + new EntropyCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + allStats(offset + statsSize - 1).toLong) } } @@ -118,12 +125,13 @@ private[spark] class EntropyAggregator(numClasses: Int) * (node, feature, bin). * @param stats Array of sufficient statistics for a (node, feature, bin). */ -private[spark] class EntropyCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { +private[spark] class EntropyCalculator(stats: Array[Double], var rawCount: Long) + extends ImpurityCalculator(stats) { /** * Make a deep copy of this [[ImpurityCalculator]]. */ - def copy: EntropyCalculator = new EntropyCalculator(stats.clone()) + def copy: EntropyCalculator = new EntropyCalculator(stats.clone(), rawCount) /** * Calculate the impurity from the stored sufficient statistics. @@ -131,9 +139,9 @@ private[spark] class EntropyCalculator(stats: Array[Double]) extends ImpurityCal def calculate(): Double = Entropy.calculate(stats, stats.sum) /** - * Number of data points accounted for in the sufficient statistics. + * Weighted number of data points accounted for in the sufficient statistics. */ - def count: Long = stats.sum.toLong + def count: Double = stats.sum /** * Prediction which should be made based on the sufficient statistics. 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 c5e34ffa4f2e5..913ffbbb2457a 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 @@ -80,23 +80,29 @@ object Gini extends Impurity { * @param numClasses Number of classes for label. */ private[spark] class GiniAggregator(numClasses: Int) - extends ImpurityAggregator(numClasses) with Serializable { + extends ImpurityAggregator(numClasses + 1) with Serializable { /** * Update stats for one (node, feature, bin) with the given label. * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { - if (label >= statsSize) { + def update( + allStats: Array[Double], + offset: Int, + label: Double, + numSamples: Int, + sampleWeight: Double): Unit = { + if (label >= numClasses) { throw new IllegalArgumentException(s"GiniAggregator given label $label" + - s" but requires label < numClasses (= $statsSize).") + s" but requires label < numClasses (= ${numClasses}).") } if (label < 0) { throw new IllegalArgumentException(s"GiniAggregator given label $label" + - s"but requires label is non-negative.") + s"but requires label to be non-negative.") } - allStats(offset + label.toInt) += instanceWeight + allStats(offset + label.toInt) += numSamples * sampleWeight + allStats(offset + statsSize - 1) += numSamples } /** @@ -105,7 +111,8 @@ private[spark] class GiniAggregator(numClasses: Int) * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): GiniCalculator = { - new GiniCalculator(allStats.view(offset, offset + statsSize).toArray) + new GiniCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + allStats(offset + statsSize - 1).toLong) } } @@ -115,12 +122,13 @@ private[spark] class GiniAggregator(numClasses: Int) * (node, feature, bin). * @param stats Array of sufficient statistics for a (node, feature, bin). */ -private[spark] class GiniCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { +private[spark] class GiniCalculator(stats: Array[Double], var rawCount: Long) + extends ImpurityCalculator(stats) { /** * Make a deep copy of this [[ImpurityCalculator]]. */ - def copy: GiniCalculator = new GiniCalculator(stats.clone()) + def copy: GiniCalculator = new GiniCalculator(stats.clone(), rawCount) /** * Calculate the impurity from the stored sufficient statistics. @@ -128,9 +136,9 @@ private[spark] class GiniCalculator(stats: Array[Double]) extends ImpurityCalcul def calculate(): Double = Gini.calculate(stats, stats.sum) /** - * Number of data points accounted for in the sufficient statistics. + * Weighted number of data points accounted for in the sufficient statistics. */ - def count: Long = stats.sum.toLong + def count: Double = stats.sum /** * Prediction which should be made based on the sufficient statistics. 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 f151a6a01b658..491473490eba7 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 @@ -81,7 +81,12 @@ private[spark] abstract class ImpurityAggregator(val statsSize: Int) extends Ser * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit + def update( + allStats: Array[Double], + offset: Int, + label: Double, + numSamples: Int, + sampleWeight: Double): Unit /** * Get an [[ImpurityCalculator]] for a (node, feature, bin). @@ -122,6 +127,7 @@ private[spark] abstract class ImpurityCalculator(val stats: Array[Double]) exten stats(i) += other.stats(i) i += 1 } + rawCount += other.rawCount this } @@ -139,13 +145,19 @@ private[spark] abstract class ImpurityCalculator(val stats: Array[Double]) exten stats(i) -= other.stats(i) i += 1 } + rawCount -= other.rawCount this } /** - * Number of data points accounted for in the sufficient statistics. + * Weighted number of data points accounted for in the sufficient statistics. */ - def count: Long + def count: Double + + /** + * Raw number of data points accounted for in the sufficient statistics. + */ + var rawCount: Long /** * Prediction which should be made based on the sufficient statistics. @@ -185,11 +197,14 @@ private[spark] object ImpurityCalculator { * Create an [[ImpurityCalculator]] instance of the given impurity type and with * the given stats. */ - def getCalculator(impurity: String, stats: Array[Double]): ImpurityCalculator = { + def getCalculator( + impurity: String, + stats: Array[Double], + rawCount: Long): ImpurityCalculator = { impurity.toLowerCase(Locale.ROOT) match { - case "gini" => new GiniCalculator(stats) - case "entropy" => new EntropyCalculator(stats) - case "variance" => new VarianceCalculator(stats) + case "gini" => new GiniCalculator(stats, rawCount) + case "entropy" => new EntropyCalculator(stats, rawCount) + case "variance" => new VarianceCalculator(stats, rawCount) case _ => throw new IllegalArgumentException( s"ImpurityCalculator builder did not recognize impurity type: $impurity") 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 c9bf0db4de3c2..a07b919271f71 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 @@ -66,21 +66,32 @@ object Variance extends Impurity { /** * Class for updating views of a vector of sufficient statistics, - * in order to compute impurity from a sample. + * in order to compute impurity from a sample. For variance, we track: + * - sum(w_i) + * - sum(w_i * y_i) + * - sum(w_i * y_i * y_i) + * - count(y_i) * Note: Instances of this class do not hold the data; they operate on views of the data. */ private[spark] class VarianceAggregator() - extends ImpurityAggregator(statsSize = 3) with Serializable { + extends ImpurityAggregator(statsSize = 4) with Serializable { /** * Update stats for one (node, feature, bin) with the given label. * @param allStats Flat stats array, with stats for this (node, feature, bin) contiguous. * @param offset Start index of stats for this (node, feature, bin). */ - def update(allStats: Array[Double], offset: Int, label: Double, instanceWeight: Double): Unit = { + def update( + allStats: Array[Double], + offset: Int, + label: Double, + numSamples: Int, + sampleWeight: Double): Unit = { + val instanceWeight = numSamples * sampleWeight allStats(offset) += instanceWeight allStats(offset + 1) += instanceWeight * label allStats(offset + 2) += instanceWeight * label * label + allStats(offset + 3) += numSamples } /** @@ -89,7 +100,8 @@ private[spark] class VarianceAggregator() * @param offset Start index of stats for this (node, feature, bin). */ def getCalculator(allStats: Array[Double], offset: Int): VarianceCalculator = { - new VarianceCalculator(allStats.view(offset, offset + statsSize).toArray) + new VarianceCalculator(allStats.view(offset, offset + statsSize - 1).toArray, + allStats(offset + statsSize - 1).toLong) } } @@ -99,7 +111,8 @@ private[spark] class VarianceAggregator() * (node, feature, bin). * @param stats Array of sufficient statistics for a (node, feature, bin). */ -private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCalculator(stats) { +private[spark] class VarianceCalculator(stats: Array[Double], var rawCount: Long) + extends ImpurityCalculator(stats) { require(stats.length == 3, s"VarianceCalculator requires sufficient statistics array stats to be of length 3," + @@ -108,7 +121,7 @@ private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCa /** * Make a deep copy of this [[ImpurityCalculator]]. */ - def copy: VarianceCalculator = new VarianceCalculator(stats.clone()) + def copy: VarianceCalculator = new VarianceCalculator(stats.clone(), rawCount) /** * Calculate the impurity from the stored sufficient statistics. @@ -116,9 +129,9 @@ private[spark] class VarianceCalculator(stats: Array[Double]) extends ImpurityCa def calculate(): Double = Variance.calculate(stats(0), stats(1), stats(2)) /** - * Number of data points accounted for in the sufficient statistics. + * Weighted number of data points accounted for in the sufficient statistics. */ - def count: Long = stats(0).toLong + def count: Double = stats(0) /** * Prediction which should be made based on the sufficient statistics. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 27618e122aefd..9983ca7dc5e87 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -126,15 +126,11 @@ class DecisionTreeModel @Since("1.0.0") ( override def save(sc: SparkContext, path: String): Unit = { DecisionTreeModel.SaveLoadV1_0.save(sc, path, this) } - - override protected def formatVersion: String = DecisionTreeModel.formatVersion } @Since("1.3.0") object DecisionTreeModel extends Loader[DecisionTreeModel] with Logging { - private[spark] def formatVersion: String = "1.0" - private[tree] object SaveLoadV1_0 { def thisFormatVersion: String = "1.0" 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 b1e82656a2405..810f528c71906 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 @@ -66,15 +66,11 @@ class RandomForestModel @Since("1.2.0") ( TreeEnsembleModel.SaveLoadV1_0.save(sc, path, this, RandomForestModel.SaveLoadV1_0.thisClassName) } - - override protected def formatVersion: String = RandomForestModel.formatVersion } @Since("1.3.0") object RandomForestModel extends Loader[RandomForestModel] { - private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion - /** * * @param sc Spark context used for loading model files. @@ -167,11 +163,9 @@ class GradientBoostedTreesModel @Since("1.2.0") ( (a, b) => treesIndices.map(idx => a(idx) + b(idx))) .map(_ / dataCount) - broadcastTrees.destroy(blocking = false) + broadcastTrees.destroy() evaluation.toArray } - - override protected def formatVersion: String = GradientBoostedTreesModel.formatVersion } /** @@ -235,8 +229,6 @@ object GradientBoostedTreesModel extends Loader[GradientBoostedTreesModel] { newPredError } - private[mllib] def formatVersion: String = TreeEnsembleModel.SaveLoadV1_0.thisFormatVersion - /** * @param sc Spark context used for loading model files. * @param path Path specifying the directory to which the model was saved. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 14af8b5c73870..6d15a6bb01e4e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -506,8 +506,6 @@ object MLUtils extends Logging { val n = v1.size require(v2.size == n) require(norm1 >= 0.0 && norm2 >= 0.0) - val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 - val normDiff = norm1 - norm2 var sqDist = 0.0 /* * The relative error is @@ -521,19 +519,23 @@ object MLUtils extends Logging { * The bound doesn't need the inner product, so we can use it as a sufficient condition to * check quickly whether the inner product approach is accurate. */ - val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) - if (precisionBound1 < precision) { - sqDist = sumSquaredNorm - 2.0 * dot(v1, v2) - } else if (v1.isInstanceOf[SparseVector] || v2.isInstanceOf[SparseVector]) { - val dotValue = dot(v1, v2) - sqDist = math.max(sumSquaredNorm - 2.0 * dotValue, 0.0) - val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) / - (sqDist + EPSILON) - if (precisionBound2 > precision) { - sqDist = Vectors.sqdist(v1, v2) - } - } else { + if (v1.isInstanceOf[DenseVector] && v2.isInstanceOf[DenseVector]) { sqDist = Vectors.sqdist(v1, v2) + } else { + val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 + val normDiff = norm1 - norm2 + val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) + if (precisionBound1 < precision) { + sqDist = sumSquaredNorm - 2.0 * dot(v1, v2) + } else { + val dotValue = dot(v1, v2) + sqDist = math.max(sumSquaredNorm - 2.0 * dotValue, 0.0) + val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dotValue)) / + (sqDist + EPSILON) + if (precisionBound2 > precision) { + sqDist = Vectors.sqdist(v1, v2) + } + } } sqDist } 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 da0eb04764c57..e8889bfd7c381 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 @@ -54,9 +54,6 @@ trait Saveable { @Since("1.3.0") def save(sc: SparkContext, path: String): Unit - /** Current version of model save/load format. */ - protected def formatVersion: String - } /** diff --git a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java index 683ceffeaed0e..2e177edf2a5c3 100644 --- a/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java +++ b/mllib/src/test/java/org/apache/spark/ml/feature/JavaPCASuite.java @@ -28,7 +28,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.ml.linalg.Vector; import org.apache.spark.ml.linalg.Vectors; -import org.apache.spark.mllib.linalg.DenseVector; import org.apache.spark.mllib.linalg.Matrix; import org.apache.spark.mllib.linalg.distributed.RowMatrix; import org.apache.spark.sql.Dataset; @@ -67,7 +66,7 @@ public void testPCA() { JavaRDD dataRDD = jsc.parallelize(points, 2); RowMatrix mat = new RowMatrix(dataRDD.map( - (Vector vector) -> (org.apache.spark.mllib.linalg.Vector) new DenseVector(vector.toArray()) + (Vector vector) -> org.apache.spark.mllib.linalg.Vectors.fromML(vector) ).rdd()); Matrix pc = mat.computePrincipalComponents(3); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index cb00977412345..fb6c775a49612 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.mllib.regression; +import java.util.ArrayList; import java.util.List; import java.util.Random; @@ -55,7 +56,8 @@ public void runRidgeRegressionUsingConstructor() { int numFeatures = 20; List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize( + new ArrayList(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); @@ -79,7 +81,8 @@ public void runRidgeRegressionUsingStaticMethods() { int numFeatures = 20; List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = jsc.parallelize(data.subList(0, numExamples)); + JavaRDD testRDD = jsc.parallelize( + new ArrayList(data.subList(0, numExamples))); List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); diff --git a/mllib/src/test/resources/log4j.properties b/mllib/src/test/resources/log4j.properties index fd51f8faf56b9..afcd5915b7481 100644 --- a/mllib/src/test/resources/log4j.properties +++ b/mllib/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/mllib/src/test/resources/test-data/strIndexerModel/data/part-00000-cfefeb56-2980-4c42-b8a7-a5a94265c479-c000.snappy.parquet b/mllib/src/test/resources/test-data/strIndexerModel/data/part-00000-cfefeb56-2980-4c42-b8a7-a5a94265c479-c000.snappy.parquet new file mode 100644 index 0000000000000..917984c2608be Binary files /dev/null and b/mllib/src/test/resources/test-data/strIndexerModel/data/part-00000-cfefeb56-2980-4c42-b8a7-a5a94265c479-c000.snappy.parquet differ diff --git a/mllib/src/test/resources/test-data/strIndexerModel/metadata/part-00000 b/mllib/src/test/resources/test-data/strIndexerModel/metadata/part-00000 new file mode 100644 index 0000000000000..5650199c36dca --- /dev/null +++ b/mllib/src/test/resources/test-data/strIndexerModel/metadata/part-00000 @@ -0,0 +1 @@ +{"class":"org.apache.spark.ml.feature.StringIndexerModel","timestamp":1545536052048,"sparkVersion":"2.4.1-SNAPSHOT","uid":"strIdx_056bb5da1bf2","paramMap":{"outputCol":"index","inputCol":"str"},"defaultParamMap":{"outputCol":"strIdx_056bb5da1bf2__output","stringOrderType":"frequencyDesc","handleInvalid":"error"}} diff --git a/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala new file mode 100644 index 0000000000000..e2ee7c05ab399 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/MLEventsSuite.scala @@ -0,0 +1,328 @@ +/* + * 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 + +import scala.collection.mutable +import scala.concurrent.duration._ + +import org.apache.hadoop.fs.Path +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.when +import org.scalatest.BeforeAndAfterEach +import org.scalatest.concurrent.Eventually +import org.scalatest.mockito.MockitoSugar.mock + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamMap +import org.apache.spark.ml.util.{DefaultParamsReader, DefaultParamsWriter, MLWriter} +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql._ +import org.apache.spark.util.JsonProtocol + + +class MLEventsSuite + extends SparkFunSuite with BeforeAndAfterEach with MLlibTestSparkContext with Eventually { + + private val events = mutable.ArrayBuffer.empty[MLEvent] + private val listener: SparkListener = new SparkListener { + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case e: MLEvent => events.append(e) + case _ => + } + } + + override def beforeAll(): Unit = { + super.beforeAll() + spark.sparkContext.addSparkListener(listener) + } + + override def afterEach(): Unit = { + try { + events.clear() + } finally { + super.afterEach() + } + } + + override def afterAll(): Unit = { + try { + if (spark != null) { + spark.sparkContext.removeSparkListener(listener) + } + } finally { + super.afterAll() + } + } + + abstract class MyModel extends Model[MyModel] + + test("pipeline fit events") { + val estimator1 = mock[Estimator[MyModel]] + val model1 = mock[MyModel] + val transformer1 = mock[Transformer] + val estimator2 = mock[Estimator[MyModel]] + val model2 = mock[MyModel] + + when(estimator1.copy(any[ParamMap])).thenReturn(estimator1) + when(model1.copy(any[ParamMap])).thenReturn(model1) + when(transformer1.copy(any[ParamMap])).thenReturn(transformer1) + when(estimator2.copy(any[ParamMap])).thenReturn(estimator2) + when(model2.copy(any[ParamMap])).thenReturn(model2) + + val dataset1 = mock[DataFrame] + val dataset2 = mock[DataFrame] + val dataset3 = mock[DataFrame] + val dataset4 = mock[DataFrame] + val dataset5 = mock[DataFrame] + + when(dataset1.toDF).thenReturn(dataset1) + when(dataset2.toDF).thenReturn(dataset2) + when(dataset3.toDF).thenReturn(dataset3) + when(dataset4.toDF).thenReturn(dataset4) + when(dataset5.toDF).thenReturn(dataset5) + + when(estimator1.fit(meq(dataset1))).thenReturn(model1) + when(model1.transform(meq(dataset1))).thenReturn(dataset2) + when(model1.parent).thenReturn(estimator1) + when(transformer1.transform(meq(dataset2))).thenReturn(dataset3) + when(estimator2.fit(meq(dataset3))).thenReturn(model2) + + val pipeline = new Pipeline() + .setStages(Array(estimator1, transformer1, estimator2)) + assert(events.isEmpty) + val pipelineModel = pipeline.fit(dataset1) + + val event0 = FitStart[PipelineModel]() + event0.estimator = pipeline + event0.dataset = dataset1 + val event1 = FitStart[MyModel]() + event1.estimator = estimator1 + event1.dataset = dataset1 + val event2 = FitEnd[MyModel]() + event2.estimator = estimator1 + event2.model = model1 + val event3 = TransformStart() + event3.transformer = model1 + event3.input = dataset1 + val event4 = TransformEnd() + event4.transformer = model1 + event4.output = dataset2 + val event5 = TransformStart() + event5.transformer = transformer1 + event5.input = dataset2 + val event6 = TransformEnd() + event6.transformer = transformer1 + event6.output = dataset3 + val event7 = FitStart[MyModel]() + event7.estimator = estimator2 + event7.dataset = dataset3 + val event8 = FitEnd[MyModel]() + event8.estimator = estimator2 + event8.model = model2 + val event9 = FitEnd[PipelineModel]() + event9.estimator = pipeline + event9.model = pipelineModel + + val expected = Seq( + event0, event1, event2, event3, event4, event5, event6, event7, event8, event9) + eventually(timeout(10.seconds), interval(1.second)) { + assert(events === expected) + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + } + + test("pipeline model transform events") { + val dataset1 = mock[DataFrame] + val dataset2 = mock[DataFrame] + val dataset3 = mock[DataFrame] + val dataset4 = mock[DataFrame] + when(dataset1.toDF).thenReturn(dataset1) + when(dataset2.toDF).thenReturn(dataset2) + when(dataset3.toDF).thenReturn(dataset3) + when(dataset4.toDF).thenReturn(dataset4) + + val transformer1 = mock[Transformer] + val model = mock[MyModel] + val transformer2 = mock[Transformer] + when(transformer1.transform(meq(dataset1))).thenReturn(dataset2) + when(model.transform(meq(dataset2))).thenReturn(dataset3) + when(transformer2.transform(meq(dataset3))).thenReturn(dataset4) + + val newPipelineModel = new PipelineModel( + "pipeline0", Array(transformer1, model, transformer2)) + assert(events.isEmpty) + val output = newPipelineModel.transform(dataset1) + + val event0 = TransformStart() + event0.transformer = newPipelineModel + event0.input = dataset1 + val event1 = TransformStart() + event1.transformer = transformer1 + event1.input = dataset1 + val event2 = TransformEnd() + event2.transformer = transformer1 + event2.output = dataset2 + val event3 = TransformStart() + event3.transformer = model + event3.input = dataset2 + val event4 = TransformEnd() + event4.transformer = model + event4.output = dataset3 + val event5 = TransformStart() + event5.transformer = transformer2 + event5.input = dataset3 + val event6 = TransformEnd() + event6.transformer = transformer2 + event6.output = dataset4 + val event7 = TransformEnd() + event7.transformer = newPipelineModel + event7.output = output + + val expected = Seq(event0, event1, event2, event3, event4, event5, event6, event7) + eventually(timeout(10.seconds), interval(1.second)) { + assert(events === expected) + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + } + + test("pipeline read/write events") { + def getInstance(w: MLWriter): AnyRef = + w.getClass.getDeclaredMethod("instance").invoke(w) + + withTempDir { dir => + val path = new Path(dir.getCanonicalPath, "pipeline").toUri.toString + val writableStage = new WritableStage("writableStage") + val newPipeline = new Pipeline().setStages(Array(writableStage)) + val pipelineWriter = newPipeline.write + assert(events.isEmpty) + pipelineWriter.save(path) + eventually(timeout(10.seconds), interval(1.second)) { + events.foreach { + case e: SaveInstanceStart if e.writer.isInstanceOf[DefaultParamsWriter] => + assert(e.path.endsWith("writableStage")) + case e: SaveInstanceEnd if e.writer.isInstanceOf[DefaultParamsWriter] => + assert(e.path.endsWith("writableStage")) + case e: SaveInstanceStart if getInstance(e.writer).isInstanceOf[Pipeline] => + assert(getInstance(e.writer).asInstanceOf[Pipeline].uid === newPipeline.uid) + case e: SaveInstanceEnd if getInstance(e.writer).isInstanceOf[Pipeline] => + assert(getInstance(e.writer).asInstanceOf[Pipeline].uid === newPipeline.uid) + case e => fail(s"Unexpected event thrown: $e") + } + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + sc.listenerBus.waitUntilEmpty(timeoutMillis = 10000) + + events.clear() + val pipelineReader = Pipeline.read + assert(events.isEmpty) + pipelineReader.load(path) + eventually(timeout(10.seconds), interval(1.second)) { + events.foreach { + case e: LoadInstanceStart[_] + if e.reader.isInstanceOf[DefaultParamsReader[_]] => + assert(e.path.endsWith("writableStage")) + case e: LoadInstanceEnd[_] + if e.reader.isInstanceOf[DefaultParamsReader[_]] => + assert(e.instance.isInstanceOf[PipelineStage]) + case e: LoadInstanceStart[_] => + assert(e.reader === pipelineReader) + case e: LoadInstanceEnd[_] => + assert(e.instance.asInstanceOf[Pipeline].uid === newPipeline.uid) + case e => fail(s"Unexpected event thrown: $e") + } + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + } + } + + test("pipeline model read/write events") { + def getInstance(w: MLWriter): AnyRef = + w.getClass.getDeclaredMethod("instance").invoke(w) + + withTempDir { dir => + val path = new Path(dir.getCanonicalPath, "pipeline").toUri.toString + val writableStage = new WritableStage("writableStage") + val pipelineModel = + new PipelineModel("pipeline_89329329", Array(writableStage.asInstanceOf[Transformer])) + val pipelineWriter = pipelineModel.write + assert(events.isEmpty) + pipelineWriter.save(path) + eventually(timeout(10.seconds), interval(1.second)) { + events.foreach { + case e: SaveInstanceStart if e.writer.isInstanceOf[DefaultParamsWriter] => + assert(e.path.endsWith("writableStage")) + case e: SaveInstanceEnd if e.writer.isInstanceOf[DefaultParamsWriter] => + assert(e.path.endsWith("writableStage")) + case e: SaveInstanceStart if getInstance(e.writer).isInstanceOf[PipelineModel] => + assert(getInstance(e.writer).asInstanceOf[PipelineModel].uid === pipelineModel.uid) + case e: SaveInstanceEnd if getInstance(e.writer).isInstanceOf[PipelineModel] => + assert(getInstance(e.writer).asInstanceOf[PipelineModel].uid === pipelineModel.uid) + case e => fail(s"Unexpected event thrown: $e") + } + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + sc.listenerBus.waitUntilEmpty(timeoutMillis = 10000) + + events.clear() + val pipelineModelReader = PipelineModel.read + assert(events.isEmpty) + pipelineModelReader.load(path) + eventually(timeout(10.seconds), interval(1.second)) { + events.foreach { + case e: LoadInstanceStart[_] + if e.reader.isInstanceOf[DefaultParamsReader[_]] => + assert(e.path.endsWith("writableStage")) + case e: LoadInstanceEnd[_] + if e.reader.isInstanceOf[DefaultParamsReader[_]] => + assert(e.instance.isInstanceOf[PipelineStage]) + case e: LoadInstanceStart[_] => + assert(e.reader === pipelineModelReader) + case e: LoadInstanceEnd[_] => + assert(e.instance.asInstanceOf[PipelineModel].uid === pipelineModel.uid) + case e => fail(s"Unexpected event thrown: $e") + } + } + // Test if they can be ser/de via JSON protocol. + assert(events.nonEmpty) + events.map(JsonProtocol.sparkEventToJson).foreach { event => + assert(JsonProtocol.sparkEventFromJson(event).isInstanceOf[MLEvent]) + } + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala index 7848eae931a06..1183cb0617610 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PipelineSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ml import scala.collection.JavaConverters._ import org.apache.hadoop.fs.Path -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar.mock diff --git a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala index ec45e32d412a9..dff00eade620f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/PredictorSuite.scala @@ -73,7 +73,7 @@ object PredictorSuite { } override def copy(extra: ParamMap): MockPredictor = - throw new NotImplementedError() + throw new UnsupportedOperationException() } class MockPredictionModel(override val uid: String) @@ -82,9 +82,9 @@ object PredictorSuite { def this() = this(Identifiable.randomUID("mockpredictormodel")) override def predict(features: Vector): Double = - throw new NotImplementedError() + throw new UnsupportedOperationException() override def copy(extra: ParamMap): MockPredictionModel = - throw new NotImplementedError() + throw new UnsupportedOperationException() } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala index 6355e0f179496..7f892fd509e75 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/attribute/AttributeSuite.scala @@ -17,7 +17,9 @@ package org.apache.spark.ml.attribute -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.types._ class AttributeSuite extends SparkFunSuite { @@ -221,4 +223,20 @@ class AttributeSuite extends SparkFunSuite { val decimalFldWithMeta = new StructField("x", DecimalType(38, 18), false, metadata) assert(Attribute.fromStructField(decimalFldWithMeta).isNumeric) } + + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set(KRYO_REGISTRATION_REQUIRED, true) + + val ser = new KryoSerializer(conf).newInstance() + + val numericAttr = new NumericAttribute(Some("numeric"), Some(1), Some(1.0), Some(2.0)) + val nominalAttr = new NominalAttribute(Some("nominal"), Some(2), Some(false)) + val binaryAttr = new BinaryAttribute(Some("binary"), Some(3), Some(Array("i", "j"))) + + Seq(numericAttr, nominalAttr, binaryAttr).foreach { i => + val i2 = ser.deserialize[Attribute](ser.serialize(i)) + assert(i === i2) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala index 87bf2be06c2be..be52d99e54d3b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/ClassifierSuite.scala @@ -117,10 +117,10 @@ object ClassifierSuite { def this() = this(Identifiable.randomUID("mockclassifier")) - override def copy(extra: ParamMap): MockClassifier = throw new NotImplementedError() + override def copy(extra: ParamMap): MockClassifier = throw new UnsupportedOperationException() override def train(dataset: Dataset[_]): MockClassificationModel = - throw new NotImplementedError() + throw new UnsupportedOperationException() // Make methods public override def extractLabeledPoints(dataset: Dataset[_], numClasses: Int): RDD[LabeledPoint] = @@ -133,11 +133,12 @@ object ClassifierSuite { def this() = this(Identifiable.randomUID("mockclassificationmodel")) - protected def predictRaw(features: Vector): Vector = throw new NotImplementedError() + protected def predictRaw(features: Vector): Vector = throw new UnsupportedOperationException() - override def copy(extra: ParamMap): MockClassificationModel = throw new NotImplementedError() + override def copy(extra: ParamMap): MockClassificationModel = + throw new UnsupportedOperationException() - override def numClasses: Int = throw new NotImplementedError() + override def numClasses: Int = throw new UnsupportedOperationException() } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala index 2930f4900d50e..433c782370242 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/DecisionTreeClassifierSuite.scala @@ -42,6 +42,8 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { private var continuousDataPointsForMulticlassRDD: RDD[LabeledPoint] = _ private var categoricalDataPointsForMulticlassForOrderedFeaturesRDD: RDD[LabeledPoint] = _ + private val seed = 42 + override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = @@ -250,7 +252,7 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { MLTestingUtils.checkCopyAndUids(dt, newTree) - testTransformer[(Vector, Double)](newData, newTree, + testTransformer[(Vector, Double, Double)](newData, newTree, "prediction", "rawPrediction", "probability") { case Row(pred: Double, rawPred: Vector, probPred: Vector) => assert(pred === rawPred.argmax, @@ -327,6 +329,49 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { dt.fit(df) } + test("training with sample weights") { + val df = { + val nPoints = 100 + val coefficients = Array( + -0.57997, 0.912083, -0.371077, + -0.16624, -0.84355, -0.048509) + + val xMean = Array(5.843, 3.057) + val xVariance = Array(0.6856, 0.1899) + + val testData = LogisticRegressionSuite.generateMultinomialLogisticInput( + coefficients, xMean, xVariance, addIntercept = true, nPoints, seed) + + sc.parallelize(testData, 4).toDF() + } + val numClasses = 3 + val predEquals = (x: Double, y: Double) => x == y + // (impurity, maxDepth) + val testParams = Seq( + ("gini", 10), + ("entropy", 10), + ("gini", 5) + ) + for ((impurity, maxDepth) <- testParams) { + val estimator = new DecisionTreeClassifier() + .setMaxDepth(maxDepth) + .setSeed(seed) + .setMinWeightFractionPerNode(0.049) + .setImpurity(impurity) + + MLTestingUtils.testArbitrarilyScaledWeights[DecisionTreeClassificationModel, + DecisionTreeClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, predEquals, 0.7)) + MLTestingUtils.testOutliersWithSmallWeights[DecisionTreeClassificationModel, + DecisionTreeClassifier](df.as[LabeledPoint], estimator, + numClasses, MLTestingUtils.modelPredictionEquals(df, predEquals, 0.8), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[DecisionTreeClassificationModel, + DecisionTreeClassifier](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, predEquals, 0.7), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// @@ -350,7 +395,6 @@ class DecisionTreeClassifierSuite extends MLTest with DefaultReadWriteTest { TreeTests.setMetadata(rdd, Map(0 -> 2, 1 -> 3), numClasses = 2) testEstimatorAndModelReadWrite(dt, categoricalData, allParamSettings, allParamSettings, checkModelData) - // Continuous splits with tree depth 2 val continuousData: DataFrame = TreeTests.setMetadata(rdd, Map.empty[Int, Int], numClasses = 2) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala index 304977634189c..379e14fbc0570 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/GBTClassifierSuite.scala @@ -345,7 +345,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { test("Tests of feature subset strategy") { val numClasses = 2 val gbt = new GBTClassifier() - .setSeed(123) + .setSeed(42) .setMaxDepth(3) .setMaxIter(5) .setFeatureSubsetStrategy("all") @@ -363,7 +363,8 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { val gbtWithFeatureSubset = gbt.setFeatureSubsetStrategy("1") val importanceFeatures = gbtWithFeatureSubset.fit(df).featureImportances val mostIF = importanceFeatures.argmax - assert(mostImportantFeature !== mostIF) + assert(mostIF === 1) + assert(importances(mostImportantFeature) !== importanceFeatures(mostIF)) } test("model evaluateEachIteration") { @@ -448,6 +449,7 @@ class GBTClassifierSuite extends MLTest with DefaultReadWriteTest { model2: GBTClassificationModel): Unit = { TreeTests.checkEqual(model, model2) assert(model.numFeatures === model2.numFeatures) + assert(model.featureImportances == model2.featureImportances) } val gbt = new GBTClassifier() diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala index c05c896df5cb1..cb9b8f9b6b472 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LinearSVCSuite.scala @@ -181,8 +181,8 @@ class LinearSVCSuite extends MLTest with DefaultReadWriteTest { } assert(thrown.getMessage.contains("coefficients only supports dense")) - bcCoefficients.destroy(blocking = false) - bcFeaturesStd.destroy(blocking = false) + bcCoefficients.destroy() + bcFeaturesStd.destroy() } test("linearSVC with sample weights") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala index 84c10e2f85c81..6dea4b1903da1 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/LogisticRegressionSuite.scala @@ -18,7 +18,6 @@ package org.apache.spark.ml.classification import scala.collection.JavaConverters._ -import scala.language.existentials import scala.util.Random import scala.util.control.Breaks._ @@ -31,7 +30,7 @@ import org.apache.spark.ml.optim.aggregator.LogisticAggregator import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions.{col, lit, rand} import org.apache.spark.sql.types.LongType @@ -40,11 +39,11 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ private val seed = 42 - @transient var smallBinaryDataset: Dataset[_] = _ - @transient var smallMultinomialDataset: Dataset[_] = _ - @transient var binaryDataset: Dataset[_] = _ - @transient var multinomialDataset: Dataset[_] = _ - @transient var multinomialDatasetWithZeroVar: Dataset[_] = _ + @transient var smallBinaryDataset: DataFrame = _ + @transient var smallMultinomialDataset: DataFrame = _ + @transient var binaryDataset: DataFrame = _ + @transient var multinomialDataset: DataFrame = _ + @transient var multinomialDatasetWithZeroVar: DataFrame = _ private val eps: Double = 1e-5 override def beforeAll(): Unit = { @@ -549,9 +548,9 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { } } assert(thrown.getMessage.contains("coefficients only supports dense")) - bcCoefficientsBinary.destroy(blocking = false) - bcFeaturesStd.destroy(blocking = false) - bcCoefficientsMulti.destroy(blocking = false) + bcCoefficientsBinary.destroy() + bcFeaturesStd.destroy() + bcCoefficientsMulti.destroy() } test("overflow prediction for multiclass") { @@ -664,18 +663,16 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { coefficients = coef(glmnet(features, label, weights=w, family="binomial", alpha = 0, lambda = 0)) coefficients - $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 2.7355261 - data.V3 -0.5734389 - data.V4 0.8911736 - data.V5 -0.3878645 - data.V6 -0.8060570 - + (Intercept) 2.7114519 + data.V3 -0.5667801 + data.V4 0.8818754 + data.V5 -0.3882505 + data.V6 -0.7891183 */ - val coefficientsR = Vectors.dense(-0.5734389, 0.8911736, -0.3878645, -0.8060570) - val interceptR = 2.7355261 + val coefficientsR = Vectors.dense(-0.5667801, 0.8818754, -0.3882505, -0.7891183) + val interceptR = 2.7114519 assert(model1.intercept ~== interceptR relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-3) @@ -707,7 +704,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val model2 = trainer2.fit(binaryDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. - val coefficientsExpected1 = Vectors.dense(0.06079437, 0.0, -0.26351059, -0.59102199) + val coefficientsExpected1 = Vectors.dense( + 0.05997387390575594, 0.0, -0.26536616889454984, -0.5793842425088045) val interceptExpected1 = 1.0 assert(model1.intercept ~== interceptExpected1 relTol 1E-3) @@ -742,8 +740,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val model4 = trainer4.fit(binaryDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. - val coefficientsExpected3 = Vectors.dense(0.0, 0.0, 0.0, -0.71708632) - val interceptExpected3 = 0.58776113 + val coefficientsExpected3 = Vectors.dense(0.0, 0.0, 0.0, -0.7003382019888361) + val interceptExpected3 = 0.5673234605102715 assert(model3.intercept ~== interceptExpected3 relTol 1E-3) assert(model3.coefficients ~= coefficientsExpected3 relTol 1E-3) @@ -775,8 +773,9 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // The solution is generated by https://github.com/yanboliang/bound-optimization. // It should be same as unbound constrained optimization with LBFGS. - val coefficientsExpected5 = Vectors.dense(-0.5734389, 0.8911736, -0.3878645, -0.8060570) - val interceptExpected5 = 2.7355261 + val coefficientsExpected5 = Vectors.dense( + -0.5667990118366208, 0.8819300812352234, -0.38825593561750166, -0.7891233856979563) + val interceptExpected5 = 2.711413425425 assert(model5.intercept ~== interceptExpected5 relTol 1E-3) assert(model5.coefficients ~= coefficientsExpected5 relTol 1E-3) @@ -810,13 +809,13 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { 5 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . - data.V3 -0.3448461 - data.V4 1.2776453 - data.V5 -0.3539178 - data.V6 -0.7469384 + data.V3 -0.3451301 + data.V4 1.2721785 + data.V5 -0.3537743 + data.V6 -0.7315618 */ - val coefficientsR = Vectors.dense(-0.3448461, 1.2776453, -0.3539178, -0.7469384) + val coefficientsR = Vectors.dense(-0.3451301, 1.2721785, -0.3537743, -0.7315618) assert(model1.intercept ~== 0.0 relTol 1E-3) assert(model1.coefficients ~= coefficientsR relTol 1E-2) @@ -844,7 +843,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val model2 = trainer2.fit(binaryDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. - val coefficientsExpected = Vectors.dense(0.20847553, 0.0, -0.24240289, -0.55568071) + val coefficientsExpected = Vectors.dense( + 0.20721074484293306, 0.0, -0.24389739190279183, -0.5446655961212726) assert(model1.intercept ~== 0.0 relTol 1E-3) assert(model1.coefficients ~= coefficientsExpected relTol 1E-3) @@ -877,15 +877,15 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) -0.06775980 + (Intercept) -0.07157076 data.V3 . data.V4 . - data.V5 -0.03933146 - data.V6 -0.03047580 + data.V5 -0.04058143 + data.V6 -0.02322760 */ - val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.03933146, -0.03047580) - val interceptRStd = -0.06775980 + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.04058143, -0.02322760) + val interceptRStd = -0.07157076 assert(model1.intercept ~== interceptRStd relTol 1E-2) assert(model1.coefficients ~= coefficientsRStd absTol 2E-2) @@ -904,15 +904,15 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 0.3544768 + (Intercept) 0.3602029 data.V3 . data.V4 . - data.V5 -0.1626191 + data.V5 -0.1635707 data.V6 . */ - val coefficientsR = Vectors.dense(0.0, 0.0, -0.1626191, 0.0) - val interceptR = 0.3544768 + val coefficientsR = Vectors.dense(0.0, 0.0, -0.1635707, 0.0) + val interceptR = 0.3602029 assert(model2.intercept ~== interceptR relTol 1E-2) assert(model2.coefficients ~== coefficientsR absTol 1E-3) @@ -945,8 +945,8 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { (Intercept) . data.V3 . data.V4 . - data.V5 -0.04967635 - data.V6 -0.04757757 + data.V5 -0.05164150 + data.V6 -0.04079129 coefficients 5 x 1 sparse Matrix of class "dgCMatrix" @@ -954,13 +954,13 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { (Intercept) . data.V3 . data.V4 . - data.V5 -0.08433195 + data.V5 -0.08408014 data.V6 . */ - val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.04967635, -0.04757757) + val coefficientsRStd = Vectors.dense(0.0, 0.0, -0.05164150, -0.04079129) - val coefficientsR = Vectors.dense(0.0, 0.0, -0.08433195, 0.0) + val coefficientsR = Vectors.dense(0.0, 0.0, -0.08408014, 0.0) assert(model1.intercept ~== 0.0 absTol 1E-3) assert(model1.coefficients ~= coefficientsRStd absTol 1E-3) @@ -992,26 +992,26 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { coefficientsStd 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 0.12707703 - data.V3 -0.06980967 - data.V4 0.10803933 - data.V5 -0.04800404 - data.V6 -0.10165096 + (Intercept) 0.12943705 + data.V3 -0.06979418 + data.V4 0.10691465 + data.V5 -0.04835674 + data.V6 -0.09939108 coefficients 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 0.46613016 - data.V3 -0.04944529 - data.V4 0.02326772 - data.V5 -0.11362772 - data.V6 -0.06312848 + (Intercept) 0.47553535 + data.V3 -0.05058465 + data.V4 0.02296823 + data.V5 -0.11368284 + data.V6 -0.06309008 */ - val coefficientsRStd = Vectors.dense(-0.06980967, 0.10803933, -0.04800404, -0.10165096) - val interceptRStd = 0.12707703 - val coefficientsR = Vectors.dense(-0.04944529, 0.02326772, -0.11362772, -0.06312848) - val interceptR = 0.46613016 + val coefficientsRStd = Vectors.dense(-0.06979418, 0.10691465, -0.04835674, -0.09939108) + val interceptRStd = 0.12943705 + val coefficientsR = Vectors.dense(-0.05058465, 0.02296823, -0.11368284, -0.06309008) + val interceptR = 0.47553535 assert(model1.intercept ~== interceptRStd relTol 1E-3) assert(model1.coefficients ~= coefficientsRStd relTol 1E-3) @@ -1042,10 +1042,12 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val model2 = trainer2.fit(binaryDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. - val coefficientsExpectedWithStd = Vectors.dense(-0.06985003, 0.0, -0.04794278, -0.10168595) - val interceptExpectedWithStd = 0.45750141 - val coefficientsExpected = Vectors.dense(-0.0494524, 0.0, -0.11360797, -0.06313577) - val interceptExpected = 0.53722967 + val coefficientsExpectedWithStd = Vectors.dense( + -0.06974410278847253, 0.0, -0.04833486093952599, -0.09941770618793982) + val interceptExpectedWithStd = 0.4564981350661977 + val coefficientsExpected = Vectors.dense( + -0.050579069523730306, 0.0, -0.11367447252893222, -0.06309435539607525) + val interceptExpected = 0.5457873335999178 assert(model1.intercept ~== interceptExpectedWithStd relTol 1E-3) assert(model1.coefficients ~= coefficientsExpectedWithStd relTol 1E-3) @@ -1078,23 +1080,24 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { 5 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . - data.V3 -0.06000152 - data.V4 0.12598737 - data.V5 -0.04669009 - data.V6 -0.09941025 + data.V3 -0.05998915 + data.V4 0.12541885 + data.V5 -0.04697872 + data.V6 -0.09713973 coefficients 5 x 1 sparse Matrix of class "dgCMatrix" s0 (Intercept) . - data.V3 -0.005482255 - data.V4 0.048106338 - data.V5 -0.093411640 - data.V6 -0.054149798 + data.V3 -0.005927466 + data.V4 0.048313659 + data.V5 -0.092956052 + data.V6 -0.053974895 */ - val coefficientsRStd = Vectors.dense(-0.06000152, 0.12598737, -0.04669009, -0.09941025) - val coefficientsR = Vectors.dense(-0.005482255, 0.048106338, -0.093411640, -0.054149798) + val coefficientsRStd = Vectors.dense(-0.05998915, 0.12541885, -0.04697872, -0.09713973) + val coefficientsR = Vectors.dense( + -0.0059320221190687205, 0.04834399477383437, -0.09296353778288495, -0.05398080548228108) assert(model1.intercept ~== 0.0 absTol 1E-3) assert(model1.coefficients ~= coefficientsRStd relTol 1E-2) @@ -1122,8 +1125,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val model2 = trainer2.fit(binaryDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. - val coefficientsExpectedWithStd = Vectors.dense(-0.00796538, 0.0, -0.0394228, -0.0873314) - val coefficientsExpected = Vectors.dense(0.01105972, 0.0, -0.08574949, -0.05079558) + val coefficientsExpectedWithStd = Vectors.dense( + -0.00845365508769699, 0.0, -0.03954848648474558, -0.0851639471468608) + val coefficientsExpected = Vectors.dense( + 0.010675769768102661, 0.0, -0.0852582080623827, -0.050615535080106376) assert(model1.intercept ~== 0.0 relTol 1E-3) assert(model1.coefficients ~= coefficientsExpectedWithStd relTol 1E-3) @@ -1134,8 +1139,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("binary logistic regression with intercept with ElasticNet regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true).setMaxIter(120) .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(true).setWeightCol("weight") - val trainer2 = (new LogisticRegression).setFitIntercept(true).setMaxIter(30) + .setTol(1e-5) + val trainer2 = (new LogisticRegression).setFitIntercept(true).setMaxIter(60) .setElasticNetParam(0.38).setRegParam(0.21).setStandardization(false).setWeightCol("weight") + .setTol(1e-5) val model1 = trainer1.fit(binaryDataset) val model2 = trainer2.fit(binaryDataset) @@ -1155,26 +1162,26 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { coefficientsStd 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 0.49991996 - data.V3 -0.04131110 + (Intercept) 0.51344133 + data.V3 -0.04395595 data.V4 . - data.V5 -0.08585233 - data.V6 -0.15875400 + data.V5 -0.08699271 + data.V6 -0.15249200 coefficients 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) 0.5024256 + (Intercept) 0.50936159 data.V3 . data.V4 . - data.V5 -0.1846038 - data.V6 -0.0559614 + data.V5 -0.18569346 + data.V6 -0.05625862 */ - val coefficientsRStd = Vectors.dense(-0.04131110, 0.0, -0.08585233, -0.15875400) - val interceptRStd = 0.49991996 - val coefficientsR = Vectors.dense(0.0, 0.0, -0.1846038, -0.0559614) - val interceptR = 0.5024256 + val coefficientsRStd = Vectors.dense(-0.04395595, 0.0, -0.08699271, -0.15249200) + val interceptRStd = 0.51344133 + val coefficientsR = Vectors.dense(0.0, 0.0, -0.18569346, -0.05625862) + val interceptR = 0.50936159 assert(model1.intercept ~== interceptRStd relTol 6E-2) assert(model1.coefficients ~== coefficientsRStd absTol 5E-3) @@ -1285,13 +1292,13 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { 5 x 1 sparse Matrix of class "dgCMatrix" s0 - (Intercept) -0.2516986 + (Intercept) -0.2521953 data.V3 0.0000000 data.V4 . data.V5 . data.V6 . */ - val interceptR = -0.2516986 + val interceptR = -0.2521953 val coefficientsR = Vectors.dense(0.0, 0.0, 0.0, 0.0) assert(model1.intercept ~== interceptR relTol 1E-5) @@ -1373,37 +1380,36 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -2.10320093 - data.V3 0.24337896 - data.V4 -0.05916156 - data.V5 0.14446790 - data.V6 0.35976165 + -2.22347257 + data.V3 0.24574397 + data.V4 -0.04054235 + data.V5 0.14963756 + data.V6 0.37504027 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.3394473 - data.V3 -0.3443375 - data.V4 0.9181331 - data.V5 -0.2283959 - data.V6 -0.4388066 + 0.3674309 + data.V3 -0.3266910 + data.V4 0.8939282 + data.V5 -0.2363519 + data.V6 -0.4631336 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 1.76375361 - data.V3 0.10095851 - data.V4 -0.85897154 - data.V5 0.08392798 - data.V6 0.07904499 - + 1.85604170 + data.V3 0.08094703 + data.V4 -0.85338588 + data.V5 0.08671439 + data.V6 0.08809332 */ val coefficientsR = new DenseMatrix(3, 4, Array( - 0.24337896, -0.05916156, 0.14446790, 0.35976165, - -0.3443375, 0.9181331, -0.2283959, -0.4388066, - 0.10095851, -0.85897154, 0.08392798, 0.07904499), isTransposed = true) - val interceptsR = Vectors.dense(-2.10320093, 0.3394473, 1.76375361) + 0.24574397, -0.04054235, 0.14963756, 0.37504027, + -0.3266910, 0.8939282, -0.2363519, -0.4631336, + 0.08094703, -0.85338588, 0.08671439, 0.08809332), isTransposed = true) + val interceptsR = Vectors.dense(-2.22347257, 0.3674309, 1.85604170) model1.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) model2.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) @@ -1484,22 +1490,26 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { .setFitIntercept(true) .setStandardization(true) .setWeightCol("weight") + .setTol(1e-5) val trainer2 = new LogisticRegression() .setLowerBoundsOnCoefficients(lowerBoundsOnCoefficients) .setLowerBoundsOnIntercepts(lowerBoundsOnIntercepts) .setFitIntercept(true) .setStandardization(false) .setWeightCol("weight") + .setTol(1e-5) val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) // The solution is generated by https://github.com/yanboliang/bound-optimization. val coefficientsExpected1 = new DenseMatrix(3, 4, Array( - 2.52076464, 2.73596057, 1.87984904, 2.73264492, - 1.93302281, 3.71363303, 1.50681746, 1.93398782, - 2.37839917, 1.93601818, 1.81924758, 2.45191255), isTransposed = true) - val interceptsExpected1 = Vectors.dense(1.00010477, 3.44237083, 4.86740286) + 2.1156620676212325, 2.7146375863138825, 1.8108730417428125, 2.711975470258063, + 1.54314110882009, 3.648963914233324, 1.4248901324480239, 1.8737908246138315, + 1.950852726788052, 1.9017484391817425, 1.7479497661988832, 2.425055298693075), + isTransposed = true) + val interceptsExpected1 = Vectors.dense( + 1.0000152482448372, 3.591773288423673, 5.079685953744937) checkCoefficientsEquivalent(model1.coefficientMatrix, coefficientsExpected1) assert(model1.interceptVector ~== interceptsExpected1 relTol 0.01) @@ -1532,9 +1542,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // The solution is generated by https://github.com/yanboliang/bound-optimization. val coefficientsExpected3 = new DenseMatrix(3, 4, Array( - 1.61967097, 1.16027835, 1.45131448, 1.97390431, - 1.30529317, 2.0, 1.12985473, 1.26652854, - 1.61647195, 1.0, 1.40642959, 1.72985589), isTransposed = true) + 1.641980508924569, 1.1579023489264648, 1.434651352010351, 1.9541352988127463, + 1.3416273422126057, 2.0, 1.1014102844446283, 1.2076556940852765, + 1.6371808928302913, 1.0, 1.3936094723717016, 1.71022540576362), + isTransposed = true) val interceptsExpected3 = Vectors.dense(1.0, 2.0, 2.0) checkCoefficientsEquivalent(model3.coefficientMatrix, coefficientsExpected3) @@ -1566,10 +1577,12 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // The solution is generated by https://github.com/yanboliang/bound-optimization. // It should be same as unbound constrained optimization with LBFGS. val coefficientsExpected5 = new DenseMatrix(3, 4, Array( - 0.24337896, -0.05916156, 0.14446790, 0.35976165, - -0.3443375, 0.9181331, -0.2283959, -0.4388066, - 0.10095851, -0.85897154, 0.08392798, 0.07904499), isTransposed = true) - val interceptsExpected5 = Vectors.dense(-2.10320093, 0.3394473, 1.76375361) + 0.24573204902629314, -0.040610820463585905, 0.14962716893619094, 0.37502549108817784, + -0.3266914048842952, 0.8940567211111817, -0.23633898260880218, -0.4631024664883818, + 0.08095935585808962, -0.8534459006476851, 0.0867118136726069, 0.0880769754002182), + isTransposed = true) + val interceptsExpected5 = Vectors.dense( + -2.2231282183460723, 0.3669496747012527, 1.856178543644802) checkCoefficientsEquivalent(model5.coefficientMatrix, coefficientsExpected5) assert(model5.interceptVector ~== interceptsExpected5 relTol 0.01) @@ -1602,35 +1615,35 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.07276291 - data.V4 -0.36325496 - data.V5 0.12015088 - data.V6 0.31397340 + data.V3 0.06892068 + data.V4 -0.36546704 + data.V5 0.12274583 + data.V6 0.32616580 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.3180040 - data.V4 0.9679074 - data.V5 -0.2252219 - data.V6 -0.4319914 + data.V3 -0.2987384 + data.V4 0.9483147 + data.V5 -0.2328113 + data.V6 -0.4555157 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.2452411 - data.V4 -0.6046524 - data.V5 0.1050710 - data.V6 0.1180180 + data.V3 0.2298177 + data.V4 -0.5828477 + data.V5 0.1100655 + data.V6 0.1293499 */ val coefficientsR = new DenseMatrix(3, 4, Array( - 0.07276291, -0.36325496, 0.12015088, 0.31397340, - -0.3180040, 0.9679074, -0.2252219, -0.4319914, - 0.2452411, -0.6046524, 0.1050710, 0.1180180), isTransposed = true) + 0.06892068, -0.36546704, 0.12274583, 0.32616580, + -0.2987384, 0.9483147, -0.2328113, -0.4555157, + 0.2298177, -0.5828477, 0.1100655, 0.1293499), isTransposed = true) model1.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) model2.coefficientMatrix.colIter.foreach(v => assert(v.toArray.sum ~== 0.0 absTol eps)) @@ -1664,9 +1677,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // The solution is generated by https://github.com/yanboliang/bound-optimization. val coefficientsExpected = new DenseMatrix(3, 4, Array( - 1.62410051, 1.38219391, 1.34486618, 1.74641729, - 1.23058989, 2.71787825, 1.0, 1.00007073, - 1.79478632, 1.14360459, 1.33011603, 1.55093897), isTransposed = true) + 1.5933935326002155, 1.4427758360562475, 1.356079506266844, 1.7818682794856215, + 1.2224266732592248, 2.762691362720858, 1.0005885171478472, 1.0000022613855966, + 1.7524631428961193, 1.2292565990448736, 1.3433784431904323, 1.5846063017678864), + isTransposed = true) checkCoefficientsEquivalent(model1.coefficientMatrix, coefficientsExpected) assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) @@ -1679,10 +1693,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // use tighter constraints because OWL-QN solver takes longer to converge val trainer1 = (new LogisticRegression).setFitIntercept(true) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true) - .setMaxIter(160).setTol(1e-10).setWeightCol("weight") + .setMaxIter(160).setTol(1e-5).setWeightCol("weight") val trainer2 = (new LogisticRegression).setFitIntercept(true) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false) - .setMaxIter(110).setTol(1e-10).setWeightCol("weight") + .setMaxIter(110).setTol(1e-5).setWeightCol("weight") val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) @@ -1703,27 +1717,27 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.62244703 + -0.69265374 data.V3 . data.V4 . data.V5 . - data.V6 0.08419825 + data.V6 0.09064661 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.2804845 - data.V3 -0.1336960 - data.V4 0.3717091 - data.V5 -0.1530363 - data.V6 -0.2035286 + -0.2260274 + data.V3 -0.1144333 + data.V4 0.3204703 + data.V5 -0.1621061 + data.V6 -0.2308192 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.9029315 + 0.9186811 data.V3 . - data.V4 -0.4629737 + data.V4 -0.4832131 data.V5 . data.V6 . @@ -1732,25 +1746,25 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.44215290 + -0.44707756 data.V3 . data.V4 . - data.V5 0.01767089 - data.V6 0.02542866 + data.V5 0.01641412 + data.V6 0.03570376 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.76308326 - data.V3 -0.06818576 + 0.75180900 + data.V3 -0.05110822 data.V4 . - data.V5 -0.20446351 - data.V6 -0.13017924 + data.V5 -0.21595670 + data.V6 -0.16162836 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.3209304 + -0.3047314 data.V3 . data.V4 . data.V5 . @@ -1759,15 +1773,15 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.08419825, - -0.1336960, 0.3717091, -0.1530363, -0.2035286, - 0.0, -0.4629737, 0.0, 0.0), isTransposed = true) - val interceptsRStd = Vectors.dense(-0.62244703, -0.2804845, 0.9029315) + 0.0, 0.0, 0.0, 0.09064661, + -0.1144333, 0.3204703, -0.1621061, -0.2308192, + 0.0, -0.4832131, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.72638218, -0.01737265, 0.74375484) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.01767089, 0.02542866, - -0.06818576, 0.0, -0.20446351, -0.13017924, + 0.0, 0.0, 0.01641412, 0.03570376, + -0.05110822, 0.0, -0.21595670, -0.16162836, 0.0, 0.0, 0.0, 0.0), isTransposed = true) - val interceptsR = Vectors.dense(-0.44215290, 0.76308326, -0.3209304) + val interceptsR = Vectors.dense(-0.44707756, 0.75180900, -0.3047314) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.05) assert(model1.interceptVector ~== interceptsRStd relTol 0.1) @@ -1780,8 +1794,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("multinomial logistic regression without intercept with L1 regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(true).setWeightCol("weight") + .setTol(1e-5) val trainer2 = (new LogisticRegression).setFitIntercept(false) .setElasticNetParam(1.0).setRegParam(0.05).setStandardization(false).setWeightCol("weight") + .setTol(1e-5) val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) @@ -1800,31 +1816,30 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 + s0 . data.V3 . data.V4 . data.V5 . - data.V6 0.01144225 + data.V6 0.01167 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.1678787 - data.V4 0.5385351 - data.V5 -0.1573039 - data.V6 -0.2471624 + data.V3 -0.1413518 + data.V4 0.5100469 + data.V5 -0.1658025 + data.V6 -0.2755998 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - . - data.V3 . - data.V4 . - data.V5 . - data.V6 . - + s0 + . + data.V3 0.001536337 + data.V4 . + data.V5 . + data.V6 . coefficients $`0` @@ -1841,9 +1856,9 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { s0 . data.V3 . - data.V4 0.1929409 - data.V5 -0.1889121 - data.V6 -0.1010413 + data.V4 0.2094410 + data.V5 -0.1944582 + data.V6 -0.1307681 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" @@ -1857,13 +1872,13 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.01144225, - -0.1678787, 0.5385351, -0.1573039, -0.2471624, - 0.0, 0.0, 0.0, 0.0), isTransposed = true) + 0.0, 0.0, 0.0, 0.01167, + -0.1413518, 0.5100469, -0.1658025, -0.2755998, + 0.001536337, 0.0, 0.0, 0.0), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( 0.0, 0.0, 0.0, 0.0, - 0.0, 0.1929409, -0.1889121, -0.1010413, + 0.0, 0.2094410, -0.1944582, -0.1307681, 0.0, 0.0, 0.0, 0.0), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) @@ -1897,72 +1912,71 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { coefficientsStd $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -1.5898288335 - data.V3 0.1691226336 - data.V4 0.0002983651 - data.V5 0.1001732896 - data.V6 0.2554575585 + s0 + -1.68571384 + data.V3 0.17156077 + data.V4 0.01658014 + data.V5 0.10303296 + data.V6 0.26459585 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.2125746 - data.V3 -0.2304586 - data.V4 0.6153492 - data.V5 -0.1537017 - data.V6 -0.2975443 + 0.2364585 + data.V3 -0.2182805 + data.V4 0.5960025 + data.V5 -0.1587441 + data.V6 -0.3121284 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 1.37725427 - data.V3 0.06133600 - data.V4 -0.61564761 - data.V5 0.05352840 - data.V6 0.04208671 - + 1.44925536 + data.V3 0.04671972 + data.V4 -0.61258267 + data.V5 0.05571116 + data.V6 0.04753251 coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -1.5681088 - data.V3 0.1508182 - data.V4 0.0121955 - data.V5 0.1217930 - data.V6 0.2162850 + s0 + -1.65140201 + data.V3 0.15446206 + data.V4 0.02134769 + data.V5 0.12524946 + data.V6 0.22607972 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 1.1217130 - data.V3 -0.2028984 - data.V4 0.2862431 - data.V5 -0.1843559 - data.V6 -0.2481218 + 1.1367722 + data.V3 -0.1931713 + data.V4 0.2766548 + data.V5 -0.1910455 + data.V6 -0.2629336 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.44639579 - data.V3 0.05208012 - data.V4 -0.29843864 - data.V5 0.06256289 - data.V6 0.03183676 + 0.51462979 + data.V3 0.03870921 + data.V4 -0.29800245 + data.V5 0.06579606 + data.V6 0.03685390 */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.1691226336, 0.0002983651, 0.1001732896, 0.2554575585, - -0.2304586, 0.6153492, -0.1537017, -0.2975443, - 0.06133600, -0.61564761, 0.05352840, 0.04208671), isTransposed = true) - val interceptsRStd = Vectors.dense(-1.5898288335, 0.2125746, 1.37725427) + 0.17156077, 0.01658014, 0.10303296, 0.26459585, + -0.2182805, 0.5960025, -0.1587441, -0.3121284, + 0.04671972, -0.61258267, 0.05571116, 0.04753251), isTransposed = true) + val interceptsRStd = Vectors.dense(-1.68571384, 0.2364585, 1.44925536) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.1508182, 0.0121955, 0.1217930, 0.2162850, - -0.2028984, 0.2862431, -0.1843559, -0.2481218, - 0.05208012, -0.29843864, 0.06256289, 0.03183676), isTransposed = true) - val interceptsR = Vectors.dense(-1.5681088, 1.1217130, 0.44639579) + 0.15446206, 0.02134769, 0.12524946, 0.22607972, + -0.1931713, 0.2766548, -0.1910455, -0.2629336, + 0.03870921, -0.29800245, 0.06579606, 0.03685390), isTransposed = true) + val interceptsR = Vectors.dense(-1.65140201, 1.1367722, 0.51462979) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.001) assert(model1.interceptVector ~== interceptsRStd relTol 0.05) @@ -1996,15 +2010,16 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // The solution is generated by https://github.com/yanboliang/bound-optimization. val coefficientsExpectedWithStd = new DenseMatrix(3, 4, Array( - 1.0, 1.0, 1.0, 1.01647497, - 1.0, 1.44105616, 1.0, 1.0, + 1.0, 1.0, 1.0, 1.025970328910313, + 1.0, 1.4150672323873024, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0), isTransposed = true) - val interceptsExpectedWithStd = Vectors.dense(2.52055893, 1.0, 2.560682) + val interceptsExpectedWithStd = Vectors.dense( + 2.4259954221861473, 1.0000087410832004, 2.490461716522559) val coefficientsExpected = new DenseMatrix(3, 4, Array( - 1.0, 1.0, 1.03189386, 1.0, + 1.0, 1.0, 1.0336746541813002, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0), isTransposed = true) - val interceptsExpected = Vectors.dense(1.06418835, 1.0, 1.20494701) + val interceptsExpected = Vectors.dense(1.0521598454128, 1.0, 1.213158241431565) assert(model1.coefficientMatrix ~== coefficientsExpectedWithStd relTol 0.01) assert(model1.interceptVector ~== interceptsExpectedWithStd relTol 0.01) @@ -2037,69 +2052,68 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.04048126 - data.V4 -0.23075758 - data.V5 0.08228864 - data.V6 0.22277648 + data.V3 0.03804571 + data.V4 -0.23204409 + data.V5 0.08337512 + data.V6 0.23029089 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.2149745 - data.V4 0.6478666 - data.V5 -0.1515158 - data.V6 -0.2930498 + data.V3 -0.2015495 + data.V4 0.6328705 + data.V5 -0.1562475 + data.V6 -0.3071447 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.17449321 - data.V4 -0.41710901 - data.V5 0.06922716 - data.V6 0.07027332 - + data.V3 0.16350376 + data.V4 -0.40082637 + data.V5 0.07287239 + data.V6 0.07685379 coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.003949652 - data.V4 -0.142982415 - data.V5 0.091439598 - data.V6 0.179286241 + data.V3 -0.006493452 + data.V4 -0.143831823 + data.V5 0.092538445 + data.V6 0.187244839 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.09071124 - data.V4 0.39752531 - data.V5 -0.16233832 - data.V6 -0.22206059 + data.V3 -0.08068443 + data.V4 0.39038929 + data.V5 -0.16822390 + data.V6 -0.23667470 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.09466090 - data.V4 -0.25454290 - data.V5 0.07089872 - data.V6 0.04277435 + data.V3 0.08717788 + data.V4 -0.24655746 + data.V5 0.07568546 + data.V6 0.04942986 */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.04048126, -0.23075758, 0.08228864, 0.22277648, - -0.2149745, 0.6478666, -0.1515158, -0.2930498, - 0.17449321, -0.41710901, 0.06922716, 0.07027332), isTransposed = true) + 0.03804571, -0.23204409, 0.08337512, 0.23029089, + -0.2015495, 0.6328705, -0.1562475, -0.3071447, + 0.16350376, -0.40082637, 0.07287239, 0.07685379), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( - -0.003949652, -0.142982415, 0.091439598, 0.179286241, - -0.09071124, 0.39752531, -0.16233832, -0.22206059, - 0.09466090, -0.25454290, 0.07089872, 0.04277435), isTransposed = true) + -0.006493452, -0.143831823, 0.092538445, 0.187244839, + -0.08068443, 0.39038929, -0.16822390, -0.23667470, + 0.08717788, -0.24655746, 0.07568546, 0.04942986), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) assert(model1.interceptVector.toArray === Array.fill(3)(0.0)) @@ -2147,10 +2161,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("multinomial logistic regression with intercept with elasticnet regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) - .setMaxIter(220).setTol(1e-10) + .setMaxIter(180).setTol(1e-5) val trainer2 = (new LogisticRegression).setFitIntercept(true).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) - .setMaxIter(90).setTol(1e-10) + .setMaxIter(150).setTol(1e-5) val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) @@ -2170,54 +2184,53 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.50133383 + -0.55325803 data.V3 . data.V4 . data.V5 . - data.V6 0.08351653 + data.V6 0.09074857 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" - s0 - -0.3151913 - data.V3 -0.1058702 - data.V4 0.3183251 - data.V5 -0.1212969 - data.V6 -0.1629778 + s0 + -0.27291366 + data.V3 -0.09093399 + data.V4 0.28078251 + data.V5 -0.12854559 + data.V6 -0.18382494 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.8165252 + 0.8261717 data.V3 . - data.V4 -0.3943069 + data.V4 -0.4064444 data.V5 . data.V6 . - coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.38857157 + -0.40016908 data.V3 . data.V4 . - data.V5 0.02384198 - data.V6 0.03127749 + data.V5 0.02312769 + data.V6 0.04159224 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - 0.62492165 - data.V3 -0.04949061 + 0.62474768 + data.V3 -0.03776471 data.V4 . - data.V5 -0.18584462 - data.V6 -0.08952455 + data.V5 -0.19588206 + data.V6 -0.11187712 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 - -0.2363501 + -0.2245786 data.V3 . data.V4 . data.V5 . @@ -2226,15 +2239,15 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.08351653, - -0.1058702, 0.3183251, -0.1212969, -0.1629778, - 0.0, -0.3943069, 0.0, 0.0), isTransposed = true) - val interceptsRStd = Vectors.dense(-0.50133383, -0.3151913, 0.8165252) + 0.0, 0.0, 0.0, 0.09074857, + -0.09093399, 0.28078251, -0.12854559, -0.18382494, + 0.0, -0.4064444, 0.0, 0.0), isTransposed = true) + val interceptsRStd = Vectors.dense(-0.55325803, -0.27291366, 0.8261717) val coefficientsR = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.02384198, 0.03127749, - -0.04949061, 0.0, -0.18584462, -0.08952455, + 0.0, 0.0, 0.02312769, 0.04159224, + -0.03776471, 0.0, -0.19588206, -0.11187712, 0.0, 0.0, 0.0, 0.0), isTransposed = true) - val interceptsR = Vectors.dense(-0.38857157, 0.62492165, -0.2363501) + val interceptsR = Vectors.dense(-0.40016908, 0.62474768, -0.2245786) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.05) assert(model1.interceptVector ~== interceptsRStd absTol 0.1) @@ -2247,10 +2260,10 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { test("multinomial logistic regression without intercept with elasticnet regularization") { val trainer1 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(true) - .setMaxIter(75).setTol(1e-10) + .setTol(1e-5) val trainer2 = (new LogisticRegression).setFitIntercept(false).setWeightCol("weight") .setElasticNetParam(0.5).setRegParam(0.1).setStandardization(false) - .setMaxIter(50).setTol(1e-10) + .setTol(1e-5) val model1 = trainer1.fit(multinomialDataset) val model2 = trainer2.fit(multinomialDataset) @@ -2274,27 +2287,26 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { data.V3 . data.V4 . data.V5 . - data.V6 0.03238285 + data.V6 0.03418889 $`1` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 -0.1328284 - data.V4 0.4219321 - data.V5 -0.1247544 - data.V6 -0.1893318 + data.V3 -0.1114779 + data.V4 0.3992145 + data.V5 -0.1315371 + data.V6 -0.2107956 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" s0 . - data.V3 0.004572312 + data.V3 0.006442826 data.V4 . data.V5 . data.V6 . - coefficients $`0` 5 x 1 sparse Matrix of class "dgCMatrix" @@ -2310,9 +2322,9 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { s0 . data.V3 . - data.V4 0.14571623 - data.V5 -0.16456351 - data.V6 -0.05866264 + data.V4 0.15710979 + data.V5 -0.16871602 + data.V6 -0.07928527 $`2` 5 x 1 sparse Matrix of class "dgCMatrix" @@ -2326,13 +2338,13 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { */ val coefficientsRStd = new DenseMatrix(3, 4, Array( - 0.0, 0.0, 0.0, 0.03238285, - -0.1328284, 0.4219321, -0.1247544, -0.1893318, - 0.004572312, 0.0, 0.0, 0.0), isTransposed = true) + 0.0, 0.0, 0.0, 0.03418889, + -0.1114779, 0.3992145, -0.1315371, -0.2107956, + 0.006442826, 0.0, 0.0, 0.0), isTransposed = true) val coefficientsR = new DenseMatrix(3, 4, Array( 0.0, 0.0, 0.0, 0.0, - 0.0, 0.14571623, -0.16456351, -0.05866264, + 0.0, 0.15710979, -0.16871602, -0.07928527, 0.0, 0.0, 0.0, 0.0), isTransposed = true) assert(model1.coefficientMatrix ~== coefficientsRStd absTol 0.01) @@ -2665,6 +2677,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { val trainer1 = new LogisticRegression() .setRegParam(0.1) .setElasticNetParam(1.0) + .setMaxIter(20) // compressed row major is optimal val model1 = trainer1.fit(multinomialDataset.limit(100)) @@ -2680,7 +2693,7 @@ class LogisticRegressionSuite extends MLTest with DefaultReadWriteTest { // coefficients are dense without L1 regularization val trainer2 = new LogisticRegression() - .setElasticNetParam(0.0) + .setElasticNetParam(0.0).setMaxIter(1) val model3 = trainer2.fit(multinomialDataset.limit(100)) assert(model3.coefficientMatrix.isInstanceOf[DenseMatrix]) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala index 5f9ab98a2c3ce..a8c4f091b2aed 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/NaiveBayesSuite.scala @@ -103,7 +103,7 @@ class NaiveBayesSuite extends MLTest with DefaultReadWriteTest { case Bernoulli => expectedBernoulliProbabilities(model, features) case _ => - throw new UnknownError(s"Invalid modelType: $modelType.") + throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } assert(probability ~== expected relTol 1.0e-10) } @@ -378,7 +378,7 @@ object NaiveBayesSuite { counts.toArray.sortBy(_._1).map(_._2) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: $modelType.") + throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } LabeledPoint(y, Vectors.dense(xi)) diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala index 2c3417c7e4028..b6e8c927403ad 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/OneVsRestSuite.scala @@ -134,8 +134,8 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { assert(lrModel1.coefficients ~== lrModel2.coefficients relTol 1E-3) assert(lrModel1.intercept ~== lrModel2.intercept relTol 1E-3) case other => - throw new AssertionError(s"Loaded OneVsRestModel expected model of type" + - s" LogisticRegressionModel but found ${other.getClass.getName}") + fail("Loaded OneVsRestModel expected model of type LogisticRegressionModel " + + s"but found ${other.getClass.getName}") } } @@ -247,8 +247,8 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { assert(lr.getMaxIter === lr2.getMaxIter) assert(lr.getRegParam === lr2.getRegParam) case other => - throw new AssertionError(s"Loaded OneVsRest expected classifier of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded OneVsRest expected classifier of type LogisticRegression" + + s" but found ${other.getClass.getName}") } } @@ -267,8 +267,8 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { assert(classifier.getMaxIter === lr2.getMaxIter) assert(classifier.getRegParam === lr2.getRegParam) case other => - throw new AssertionError(s"Loaded OneVsRestModel expected classifier of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded OneVsRestModel expected classifier of type LogisticRegression" + + s" but found ${other.getClass.getName}") } assert(model.labelMetadata === model2.labelMetadata) @@ -278,8 +278,8 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { assert(lrModel1.coefficients === lrModel2.coefficients) assert(lrModel1.intercept === lrModel2.intercept) case other => - throw new AssertionError(s"Loaded OneVsRestModel expected model of type" + - s" LogisticRegressionModel but found ${other.getClass.getName}") + fail(s"Loaded OneVsRestModel expected model of type LogisticRegressionModel" + + s" but found ${other.getClass.getName}") } } @@ -290,6 +290,32 @@ class OneVsRestSuite extends MLTest with DefaultReadWriteTest { checkModelData(ovaModel, newOvaModel) } + test("should ignore empty output cols") { + val lr = new LogisticRegression().setMaxIter(1) + val ovr = new OneVsRest().setClassifier(lr) + val ovrModel = ovr.fit(dataset) + + val output1 = ovrModel.setPredictionCol("").setRawPredictionCol("") + .transform(dataset) + assert(output1.schema.fieldNames.toSet === + Set("label", "features")) + + val output2 = ovrModel.setPredictionCol("prediction").setRawPredictionCol("") + .transform(dataset) + assert(output2.schema.fieldNames.toSet === + Set("label", "features", "prediction")) + + val output3 = ovrModel.setPredictionCol("").setRawPredictionCol("rawPrediction") + .transform(dataset) + assert(output3.schema.fieldNames.toSet === + Set("label", "features", "rawPrediction")) + + val output4 = ovrModel.setPredictionCol("prediction").setRawPredictionCol("rawPrediction") + .transform(dataset) + assert(output4.schema.fieldNames.toSet === + Set("label", "features", "prediction", "rawPrediction")) + } + test("should support all NumericType labels and not support other types") { val ovr = new OneVsRest().setClassifier(new LogisticRegression().setMaxIter(1)) MLTestingUtils.checkNumericTypes[OneVsRestModel, OneVsRest]( diff --git a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala index ba4a9cf082785..027583ffc60bf 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/classification/RandomForestClassifierSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.classification import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.tree.LeafNode @@ -141,7 +141,7 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { MLTestingUtils.checkCopyAndUids(rf, model) - testTransformer[(Vector, Double)](df, model, "prediction", "rawPrediction", + testTransformer[(Vector, Double, Double)](df, model, "prediction", "rawPrediction", "probability") { case Row(pred: Double, rawPred: Vector, probPred: Vector) => assert(pred === rawPred.argmax, s"Expected prediction $pred but calculated ${rawPred.argmax} from rawPrediction.") @@ -180,7 +180,6 @@ class RandomForestClassifierSuite extends MLTest with DefaultReadWriteTest { ///////////////////////////////////////////////////////////////////////////// // Tests of feature importance ///////////////////////////////////////////////////////////////////////////// - test("Feature importance with toy data") { val numClasses = 2 val rf = new RandomForestClassifier() diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala index 1b7780e171e77..5288595d2e239 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/BisectingKMeansSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.ml.clustering -import scala.language.existentials - import org.apache.spark.SparkException import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.clustering.DistanceMeasure -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.DataFrame class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { @@ -33,9 +31,8 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ final val k = 5 - @transient var dataset: Dataset[_] = _ - - @transient var sparseDataset: Dataset[_] = _ + @transient var dataset: DataFrame = _ + @transient var sparseDataset: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() @@ -74,7 +71,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { rows => val numClusters = rows.distinct.length // Verify we hit the edge case - assert(numClusters < k && numClusters > 1) + assert(numClusters > 1) } } @@ -134,6 +131,8 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { assert(clusterSizes.sum === numRows) assert(clusterSizes.forall(_ >= 0)) assert(summary.numIter == 20) + assert(summary.trainingCost < 0.1) + assert(model.computeCost(dataset) == summary.trainingCost) model.setSummary(None) assert(!model.hasSummary) @@ -189,7 +188,7 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { } test("BisectingKMeans with Array input") { - def trainAndComputeCost(dataset: Dataset[_]): Double = { + def trainAndComputeCost(dataset: DataFrame): Double = { val model = new BisectingKMeans().setK(k).setMaxIter(1).setSeed(1).fit(dataset) model.computeCost(dataset) } @@ -203,6 +202,13 @@ class BisectingKMeansSuite extends MLTest with DefaultReadWriteTest { assert(trueCost ~== doubleArrayCost absTol 1e-6) assert(trueCost ~== floatArrayCost absTol 1e-6) } + + test("prediction on single instance") { + val bikm = new BisectingKMeans().setSeed(123L) + val model = bikm.fit(dataset) + testClusteringModelSinglePrediction(model, model.predict, dataset, + model.getFeaturesCol, model.getPredictionCol) + } } object BisectingKMeansSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala index 13bed9dbe3e89..133536f763f4e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/GaussianMixtureSuite.scala @@ -17,15 +17,13 @@ package org.apache.spark.ml.clustering -import scala.language.existentials - import org.apache.spark.SparkFunSuite import org.apache.spark.ml.linalg.{DenseMatrix, Matrices, Vector, Vectors} import org.apache.spark.ml.param.ParamMap import org.apache.spark.ml.stat.distribution.MultivariateGaussian import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ -import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.{DataFrame, Dataset, Row} class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { @@ -35,11 +33,11 @@ class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { final val k = 5 private val seed = 538009335 - @transient var dataset: Dataset[_] = _ - @transient var denseDataset: Dataset[_] = _ - @transient var sparseDataset: Dataset[_] = _ - @transient var decompositionDataset: Dataset[_] = _ - @transient var rDataset: Dataset[_] = _ + @transient var dataset: DataFrame = _ + @transient var denseDataset: DataFrame = _ + @transient var sparseDataset: DataFrame = _ + @transient var decompositionDataset: DataFrame = _ + @transient var rDataset: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() @@ -268,6 +266,16 @@ class GaussianMixtureSuite extends MLTest with DefaultReadWriteTest { assert(trueLikelihood ~== doubleLikelihood absTol 1e-6) assert(trueLikelihood ~== floatLikelihood absTol 1e-6) } + + test("prediction on single instance") { + val gmm = new GaussianMixture().setSeed(123L) + val model = gmm.fit(dataset) + testClusteringModelSinglePrediction(model, model.predict, dataset, + model.getFeaturesCol, model.getPredictionCol) + + testClusteringModelSingleProbabilisticPrediction(model, model.predictProbability, dataset, + model.getFeaturesCol, model.getProbabilityCol) + } } object GaussianMixtureSuite extends SparkFunSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala index ccbceab53bb66..e3c82fafca218 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/KMeansSuite.scala @@ -17,10 +17,10 @@ package org.apache.spark.ml.clustering -import scala.language.existentials import scala.util.Random -import org.dmg.pmml.{ClusteringModel, PMML} +import org.dmg.pmml.PMML +import org.dmg.pmml.clustering.ClusteringModel import org.apache.spark.SparkException import org.apache.spark.ml.linalg.{Vector, Vectors} @@ -39,7 +39,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes import testImplicits._ final val k = 5 - @transient var dataset: Dataset[_] = _ + @transient var dataset: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() @@ -117,7 +117,6 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes assert(clusters === Set(0, 1, 2, 3, 4)) } - assert(model.computeCost(dataset) < 0.1) assert(model.hasParent) // Check validity of model summary @@ -132,7 +131,6 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes } assert(summary.cluster.columns === Array(predictionColName)) assert(summary.trainingCost < 0.1) - assert(model.computeCost(dataset) == summary.trainingCost) val clusterSizes = summary.clusterSizes assert(clusterSizes.length === k) assert(clusterSizes.sum === numRows) @@ -168,7 +166,7 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes val model = new KMeans() .setK(3) - .setSeed(1) + .setSeed(42) .setInitMode(MLlibKMeans.RANDOM) .setTol(1e-6) .setDistanceMeasure(DistanceMeasure.COSINE) @@ -201,15 +199,15 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes } test("KMean with Array input") { - def trainAndComputeCost(dataset: Dataset[_]): Double = { + def trainAndGetCost(dataset: Dataset[_]): Double = { val model = new KMeans().setK(k).setMaxIter(1).setSeed(1).fit(dataset) - model.computeCost(dataset) + model.summary.trainingCost } val (newDataset, newDatasetD, newDatasetF) = MLTestingUtils.generateArrayFeatureDataset(dataset) - val trueCost = trainAndComputeCost(newDataset) - val doubleArrayCost = trainAndComputeCost(newDatasetD) - val floatArrayCost = trainAndComputeCost(newDatasetF) + val trueCost = trainAndGetCost(newDataset) + val doubleArrayCost = trainAndGetCost(newDatasetD) + val floatArrayCost = trainAndGetCost(newDatasetF) // checking the cost is fine enough as a sanity check assert(trueCost ~== doubleArrayCost absTol 1e-6) @@ -246,6 +244,13 @@ class KMeansSuite extends MLTest with DefaultReadWriteTest with PMMLReadWriteTes } testPMMLWrite(sc, kmeansModel, checkModel) } + + test("prediction on single instance") { + val kmeans = new KMeans().setSeed(123L) + val model = kmeans.fit(dataset) + testClusteringModelSinglePrediction(model, model.predict, dataset, + model.getFeaturesCol, model.getPredictionCol) + } } object KMeansSuite { diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala index bbd5408c9fce3..d0898220b80de 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/LDASuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.clustering -import scala.language.existentials - import org.apache.hadoop.fs.Path import org.apache.spark.ml.linalg.{Vector, Vectors} @@ -64,7 +62,7 @@ class LDASuite extends MLTest with DefaultReadWriteTest { val k: Int = 5 val vocabSize: Int = 30 - @transient var dataset: Dataset[_] = _ + @transient var dataset: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() @@ -329,8 +327,7 @@ class LDASuite extends MLTest with DefaultReadWriteTest { (model.logLikelihood(dataset), model.logPerplexity(dataset)) } - val (newDataset, newDatasetD, newDatasetF) = MLTestingUtils.generateArrayFeatureDataset(dataset) - val (ll, lp) = trainAndLogLikelihoodAndPerplexity(newDataset) + val (_, newDatasetD, newDatasetF) = MLTestingUtils.generateArrayFeatureDataset(dataset) val (llD, lpD) = trainAndLogLikelihoodAndPerplexity(newDatasetD) val (llF, lpF) = trainAndLogLikelihoodAndPerplexity(newDatasetF) // TODO: need to compare the results once we fix the seed issue for LDA (SPARK-22210) diff --git a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala index 55b460f1a4524..d3b8575327a87 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/clustering/PowerIterationClusteringSuite.scala @@ -34,9 +34,9 @@ class PowerIterationClusteringSuite extends SparkFunSuite @transient var data: Dataset[_] = _ final val r1 = 1.0 - final val n1 = 10 + final val n1 = 80 final val r2 = 4.0 - final val n2 = 40 + final val n2 = 80 override def beforeAll(): Unit = { super.beforeAll() @@ -145,6 +145,21 @@ class PowerIterationClusteringSuite extends SparkFunSuite assert(msg.contains("Similarity must be nonnegative")) } + test("check for invalid input types of weight") { + val invalidWeightData = spark.createDataFrame(Seq( + (0L, 1L, "a"), + (2L, 3L, "b") + )).toDF("src", "dst", "weight") + + val msg = intercept[IllegalArgumentException] { + new PowerIterationClustering() + .setWeightCol("weight") + .assignClusters(invalidWeightData) + }.getMessage + assert(msg.contains("requirement failed: Column weight must be of type numeric" + + " but was actually of type string.")) + } + test("test default weight") { val dataWithoutWeight = data.sample(0.5, 1L).select('src, 'dst) @@ -169,6 +184,74 @@ class PowerIterationClusteringSuite extends SparkFunSuite assert(localAssignments === localAssignments2) } + test("power iteration clustering gives incorrect results due to failed to converge") { + /* + Graph: + 1 + / + / + 0 2 -- 3 + */ + val data1 = spark.createDataFrame(Seq( + (0, 1), + (2, 3) + )).toDF("src", "dst") + + val assignments1 = new PowerIterationClustering() + .setInitMode("random") + .setK(2) + .assignClusters(data1) + .select("id", "cluster") + .as[(Long, Int)] + .collect() + val predictions1 = Array.fill(2)(mutable.Set.empty[Long]) + assignments1.foreach { + case (id, cluster) => predictions1(cluster) += id + } + assert(Set(predictions1(0).size, predictions1(1).size) !== Set(2, 2)) + + + /* + Graph: + 1 + / + / + 0 - - 2 3 -- 4 + */ + val data2 = spark.createDataFrame(Seq( + (0, 1), + (0, 2), + (3, 4) + )).toDF("src", "dst").repartition(1) + + var assignments2 = new PowerIterationClustering() + .setInitMode("random") + .setK(2) + .assignClusters(data2) + .select("id", "cluster") + .as[(Long, Int)] + .collect() + val predictions2 = Array.fill(2)(mutable.Set.empty[Long]) + assignments2.foreach { + case (id, cluster) => predictions2(cluster) += id + } + assert(Set(predictions2(0).size, predictions2(1).size) !== Set(2, 3)) + + + var assignments3 = new PowerIterationClustering() + .setInitMode("degree") + .setK(2) + .assignClusters(data2) + .select("id", "cluster") + .as[(Long, Int)] + .collect() + val predictions3 = Array.fill(2)(mutable.Set.empty[Long]) + assignments3.foreach { + case (id, cluster) => predictions3(cluster) += id + } + assert(Set(predictions3(0).size, predictions3(1).size) !== Set(2, 3)) + } + test("read/write") { val t = new PowerIterationClustering() .setK(4) diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala index 2b0909acf69c3..83b213ab51d43 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/BinaryClassificationEvaluatorSuite.scala @@ -45,23 +45,23 @@ class BinaryClassificationEvaluatorSuite .setMetricName("areaUnderPR") val vectorDF = Seq( - (0d, Vectors.dense(12, 2.5)), - (1d, Vectors.dense(1, 3)), - (0d, Vectors.dense(10, 2)) + (0.0, Vectors.dense(12, 2.5)), + (1.0, Vectors.dense(1, 3)), + (0.0, Vectors.dense(10, 2)) ).toDF("label", "rawPrediction") assert(evaluator.evaluate(vectorDF) === 1.0) val doubleDF = Seq( - (0d, 0d), - (1d, 1d), - (0d, 0d) + (0.0, 0.0), + (1.0, 1.0), + (0.0, 0.0) ).toDF("label", "rawPrediction") assert(evaluator.evaluate(doubleDF) === 1.0) val stringDF = Seq( - (0d, "0d"), - (1d, "1d"), - (0d, "0d") + (0.0, "0.0"), + (1.0, "1.0"), + (0.0, "0.0") ).toDF("label", "rawPrediction") val thrown = intercept[IllegalArgumentException] { evaluator.evaluate(stringDF) @@ -71,6 +71,33 @@ class BinaryClassificationEvaluatorSuite assert(thrown.getMessage.replace("\n", "") contains "but was actually of type string.") } + test("should accept weight column") { + val weightCol = "weight" + // get metric with weight column + val evaluator = new BinaryClassificationEvaluator() + .setMetricName("areaUnderROC").setWeightCol(weightCol) + val vectorDF = Seq( + (0.0, Vectors.dense(2.5, 12), 1.0), + (1.0, Vectors.dense(1, 3), 1.0), + (0.0, Vectors.dense(10, 2), 1.0) + ).toDF("label", "rawPrediction", weightCol) + val result = evaluator.evaluate(vectorDF) + // without weight column + val evaluator2 = new BinaryClassificationEvaluator() + .setMetricName("areaUnderROC") + val result2 = evaluator2.evaluate(vectorDF) + assert(result === result2) + // use different weights, validate metrics change + val vectorDF2 = Seq( + (0.0, Vectors.dense(2.5, 12), 2.5), + (1.0, Vectors.dense(1, 3), 0.1), + (0.0, Vectors.dense(10, 2), 2.0) + ).toDF("label", "rawPrediction", weightCol) + val result3 = evaluator.evaluate(vectorDF2) + // Since wrong result weighted more heavily, expect the score to be lower + assert(result3 < result) + } + test("should support all NumericType labels and not support other types") { val evaluator = new BinaryClassificationEvaluator().setRawPredictionCol("prediction") MLTestingUtils.checkNumericTypes(evaluator, spark) diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala index e2d77560293fa..6cf3b1deeac93 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/ClusteringEvaluatorSuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext -import org.apache.spark.sql.Dataset +import org.apache.spark.sql.DataFrame class ClusteringEvaluatorSuite @@ -32,10 +32,10 @@ class ClusteringEvaluatorSuite import testImplicits._ - @transient var irisDataset: Dataset[_] = _ - @transient var newIrisDataset: Dataset[_] = _ - @transient var newIrisDatasetD: Dataset[_] = _ - @transient var newIrisDatasetF: Dataset[_] = _ + @transient var irisDataset: DataFrame = _ + @transient var newIrisDataset: DataFrame = _ + @transient var newIrisDatasetD: DataFrame = _ + @transient var newIrisDatasetF: DataFrame = _ override def beforeAll(): Unit = { super.beforeAll() @@ -134,4 +134,15 @@ class ClusteringEvaluatorSuite // with wrong metadata the evaluator throws an Exception intercept[SparkException](evaluator.evaluate(dfWrong)) } + + test("SPARK-27896: single-element clusters should have silhouette score of 0") { + val twoSingleItemClusters = + irisDataset.where($"label" === 0.0).limit(1).union( + irisDataset.where($"label" === 1.0).limit(1)) + val evaluator = new ClusteringEvaluator() + .setFeaturesCol("features") + .setPredictionCol("label") + assert(evaluator.evaluate(twoSingleItemClusters) === 0.0) + } + } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala index 1a3a8a13a2d09..b41df1b798af6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MulticlassClassificationEvaluatorSuite.scala @@ -21,10 +21,13 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTestingUtils} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class MulticlassClassificationEvaluatorSuite extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + import testImplicits._ + test("params") { ParamsSuite.checkParams(new MulticlassClassificationEvaluator) } @@ -34,10 +37,27 @@ class MulticlassClassificationEvaluatorSuite .setPredictionCol("myPrediction") .setLabelCol("myLabel") .setMetricName("accuracy") + .setMetricLabel(1.0) + .setBeta(2.0) testDefaultReadWrite(evaluator) } test("should support all NumericType labels and not support other types") { MLTestingUtils.checkNumericTypes(new MulticlassClassificationEvaluator, spark) } + + test("evaluation metrics") { + val predictionAndLabels = Seq((0.0, 0.0), (0.0, 1.0), + (0.0, 0.0), (1.0, 0.0), (1.0, 1.0), + (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)).toDF("prediction", "label") + + val evaluator = new MulticlassClassificationEvaluator() + .setMetricName("precisionByLabel") + .setMetricLabel(0.0) + assert(evaluator.evaluate(predictionAndLabels) ~== 2.0 / 3 absTol 1e-5) + + evaluator.setMetricName("truePositiveRateByLabel") + .setMetricLabel(1.0) + assert(evaluator.evaluate(predictionAndLabels) ~== 3.0 / 4 absTol 1e-5) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluatorSuite.scala new file mode 100644 index 0000000000000..f41fc04a5faed --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/MultilabelClassificationEvaluatorSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.ml.evaluation + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class MultilabelClassificationEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("params") { + ParamsSuite.checkParams(new MultilabelClassificationEvaluator) + } + + test("evaluation metrics") { + val scoreAndLabels = Seq((Array(0.0, 1.0), Array(0.0, 2.0)), + (Array(0.0, 2.0), Array(0.0, 1.0)), + (Array.empty[Double], Array(0.0)), + (Array(2.0), Array(2.0)), + (Array(2.0, 0.0), Array(2.0, 0.0)), + (Array(0.0, 1.0, 2.0), Array(0.0, 1.0)), + (Array(1.0), Array(1.0, 2.0))).toDF("prediction", "label") + + val evaluator = new MultilabelClassificationEvaluator() + .setMetricName("subsetAccuracy") + assert(evaluator.evaluate(scoreAndLabels) ~== 2.0 / 7 absTol 1e-5) + + evaluator.setMetricName("recallByLabel") + .setMetricLabel(0.0) + assert(evaluator.evaluate(scoreAndLabels) ~== 0.8 absTol 1e-5) + } + + test("read/write") { + val evaluator = new MultilabelClassificationEvaluator() + .setPredictionCol("myPrediction") + .setLabelCol("myLabel") + .setMetricLabel(1.0) + .setMetricName("precisionByLabel") + testDefaultReadWrite(evaluator) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/evaluation/RankingEvaluatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RankingEvaluatorSuite.scala new file mode 100644 index 0000000000000..02d26d7eb351f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/ml/evaluation/RankingEvaluatorSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.spark.ml.evaluation + +import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.param.ParamsSuite +import org.apache.spark.ml.util.DefaultReadWriteTest +import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ + +class RankingEvaluatorSuite + extends SparkFunSuite with MLlibTestSparkContext with DefaultReadWriteTest { + + import testImplicits._ + + test("params") { + ParamsSuite.checkParams(new RankingEvaluator) + } + + test("read/write") { + val evaluator = new RankingEvaluator() + .setPredictionCol("myPrediction") + .setLabelCol("myLabel") + .setMetricName("precisionAtK") + .setK(10) + testDefaultReadWrite(evaluator) + } + + test("evaluation metrics") { + val scoreAndLabels = Seq( + (Array(1.0, 6.0, 2.0, 7.0, 8.0, 3.0, 9.0, 10.0, 4.0, 5.0), + Array(1.0, 2.0, 3.0, 4.0, 5.0)), + (Array(4.0, 1.0, 5.0, 6.0, 2.0, 7.0, 3.0, 8.0, 9.0, 10.0), + Array(1.0, 2.0, 3.0)), + (Array(1.0, 2.0, 3.0, 4.0, 5.0), Array.empty[Double]) + ).toDF("prediction", "label") + + val evaluator = new RankingEvaluator() + .setMetricName("meanAveragePrecision") + assert(evaluator.evaluate(scoreAndLabels) ~== 0.355026 absTol 1e-5) + + evaluator.setMetricName("precisionAtK") + .setK(2) + assert(evaluator.evaluate(scoreAndLabels) ~== 1.0 / 3 absTol 1e-5) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala index 6734336aac39c..985e396000d05 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/DCTSuite.scala @@ -17,16 +17,16 @@ package org.apache.spark.ml.feature -import scala.beans.BeanInfo - import edu.emory.mathcs.jtransforms.dct.DoubleDCT_1D import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.Row -@BeanInfo -case class DCTTestData(vec: Vector, wantedVec: Vector) +case class DCTTestData(vec: Vector, wantedVec: Vector) { + def getVec: Vector = vec + def getWantedVec: Vector = wantedVec +} class DCTSuite extends MLTest with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala index cdd62be43b54c..73b2b82daaf43 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/IDFSuite.scala @@ -44,7 +44,7 @@ class IDFSuite extends MLTest with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new IDF) - val model = new IDFModel("idf", new OldIDFModel(Vectors.dense(1.0))) + val model = new IDFModel("idf", new OldIDFModel(Vectors.dense(1.0), Array(1L), 1)) ParamsSuite.checkParams(model) } @@ -112,10 +112,13 @@ class IDFSuite extends MLTest with DefaultReadWriteTest { } test("IDFModel read/write") { - val instance = new IDFModel("myIDFModel", new OldIDFModel(Vectors.dense(1.0, 2.0))) + val instance = new IDFModel("myIDFModel", + new OldIDFModel(Vectors.dense(1.0, 2.0), Array(1, 2), 2)) .setInputCol("myInputCol") .setOutputCol("myOutputCol") val newInstance = testDefaultReadWrite(instance) assert(newInstance.idf === instance.idf) + assert(newInstance.docFreq === instance.docFreq) + assert(newInstance.numDocs === instance.numDocs) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala index cca7399b4b9c5..5a74490058398 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/InstanceSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class InstanceSuite extends SparkFunSuite{ test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala index 05c7a58ee5ffd..63c163590af59 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/LabeledPointSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.ml.feature import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.linalg.Vectors import org.apache.spark.serializer.KryoSerializer class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala index 201a335e0d7be..1483d5df4d224 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/NGramSuite.scala @@ -17,14 +17,13 @@ package org.apache.spark.ml.feature -import scala.beans.BeanInfo - import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.{DataFrame, Row} - -@BeanInfo -case class NGramTestData(inputTokens: Array[String], wantedNGrams: Array[String]) +case class NGramTestData(inputTokens: Array[String], wantedNGrams: Array[String]) { + def getInputTokens: Array[String] = inputTokens + def getWantedNGrams: Array[String] = wantedNGrams +} class NGramSuite extends MLTest with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala deleted file mode 100644 index d549e13262273..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderEstimatorSuite.scala +++ /dev/null @@ -1,422 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.ml.feature - -import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} -import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} -import org.apache.spark.ml.param.ParamsSuite -import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} -import org.apache.spark.sql.{Encoder, Row} -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder -import org.apache.spark.sql.functions.col -import org.apache.spark.sql.types._ - -class OneHotEncoderEstimatorSuite extends MLTest with DefaultReadWriteTest { - - import testImplicits._ - - test("params") { - ParamsSuite.checkParams(new OneHotEncoderEstimator) - } - - test("OneHotEncoderEstimator dropLast = false") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - assert(encoder.getDropLast === true) - encoder.setDropLast(false) - assert(encoder.getDropLast === false) - val model = encoder.fit(df) - testTransformer[(Double, Vector)](df, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("OneHotEncoderEstimator dropLast = true") { - val data = Seq( - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(2, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq())), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(df) - testTransformer[(Double, Vector)](df, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("input column with ML attribute") { - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", attr.toMetadata())) - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("size")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) - } - } - - test("input column without ML attribute") { - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("index")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) - } - } - - test("read/write") { - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("index")) - .setOutputCols(Array("encoded")) - testDefaultReadWrite(encoder) - } - - test("OneHotEncoderModel read/write") { - val instance = new OneHotEncoderModel("myOneHotEncoderModel", Array(1, 2, 3)) - val newInstance = testDefaultReadWrite(instance) - assert(newInstance.categorySizes === instance.categorySizes) - } - - test("OneHotEncoderEstimator with varying types") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - class NumericTypeWithEncoder[A](val numericType: NumericType) - (implicit val encoder: Encoder[(A, Vector)]) - - val types = Seq( - new NumericTypeWithEncoder[Short](ShortType), - new NumericTypeWithEncoder[Long](LongType), - new NumericTypeWithEncoder[Int](IntegerType), - new NumericTypeWithEncoder[Float](FloatType), - new NumericTypeWithEncoder[Byte](ByteType), - new NumericTypeWithEncoder[Double](DoubleType), - new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder())) - - for (t <- types) { - val dfWithTypes = df.select(col("input").cast(t.numericType), col("expected")) - val estimator = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setDropLast(false) - - val model = estimator.fit(dfWithTypes) - testTransformer(dfWithTypes, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - }(t.encoder) - } - } - - test("OneHotEncoderEstimator: encoding multiple columns and dropLast = false") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), 3.0, Vectors.sparse(4, Seq((3, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input1", DoubleType), - StructField("expected1", new VectorUDT), - StructField("input2", DoubleType), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("output1", "output2")) - assert(encoder.getDropLast === true) - encoder.setDropLast(false) - assert(encoder.getDropLast === false) - - val model = encoder.fit(df) - testTransformer[(Double, Vector, Double, Vector)]( - df, - model, - "output1", - "output2", - "expected1", - "expected2") { - case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => - assert(output1 === expected1) - assert(output2 === expected2) - } - } - - test("OneHotEncoderEstimator: encoding multiple columns and dropLast = true") { - val data = Seq( - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 2.0, Vectors.sparse(3, Seq((2, 1.0)))), - Row(1.0, Vectors.sparse(2, Seq((1, 1.0))), 3.0, Vectors.sparse(3, Seq())), - Row(2.0, Vectors.sparse(2, Seq()), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(2, Seq()), 2.0, Vectors.sparse(3, Seq((2, 1.0))))) - - val schema = StructType(Array( - StructField("input1", DoubleType), - StructField("expected1", new VectorUDT), - StructField("input2", DoubleType), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input1", "input2")) - .setOutputCols(Array("output1", "output2")) - - val model = encoder.fit(df) - testTransformer[(Double, Vector, Double, Vector)]( - df, - model, - "output1", - "output2", - "expected1", - "expected2") { - case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => - assert(output1 === expected1) - assert(output2 === expected2) - } - } - - test("Throw error on invalid values") { - val trainingData = Seq((0, 0), (1, 1), (2, 2)) - val trainingDF = trainingData.toDF("id", "a") - val testData = Seq((0, 0), (1, 2), (1, 3)) - val testDF = testData.toDF("id", "a") - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("a")) - .setOutputCols(Array("encoded")) - - val model = encoder.fit(trainingDF) - testTransformerByInterceptingException[(Int, Int)]( - testDF, - model, - expectedMessagePart = "Unseen value: 3.0. To handle unseen values", - firstResultCol = "encoded") - - } - - test("Can't transform on negative input") { - val trainingDF = Seq((0, 0), (1, 1), (2, 2)).toDF("a", "b") - val testDF = Seq((0, 0), (-1, 2), (1, 3)).toDF("a", "b") - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("a")) - .setOutputCols(Array("encoded")) - - val model = encoder.fit(trainingDF) - testTransformerByInterceptingException[(Int, Int)]( - testDF, - model, - expectedMessagePart = "Negative value: -1.0. Input can't be negative", - firstResultCol = "encoded") - } - - test("Keep on invalid values: dropLast = false") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setHandleInvalid("keep") - .setDropLast(false) - - val model = encoder.fit(trainingDF) - testTransformer[(Double, Vector)](testDF, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("Keep on invalid values: dropLast = true") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(3, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - .setHandleInvalid("keep") - .setDropLast(true) - - val model = encoder.fit(trainingDF) - testTransformer[(Double, Vector)](testDF, model, "output", "expected") { - case Row(output: Vector, expected: Vector) => - assert(output === expected) - } - } - - test("OneHotEncoderModel changes dropLast") { - val data = Seq( - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), Vectors.sparse(2, Seq((1, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq())), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), - Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq()))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected1", new VectorUDT), - StructField("expected2", new VectorUDT))) - - val df = spark.createDataFrame(sc.parallelize(data), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(df) - - model.setDropLast(false) - testTransformer[(Double, Vector, Vector)](df, model, "output", "expected1") { - case Row(output: Vector, expected1: Vector) => - assert(output === expected1) - } - - model.setDropLast(true) - testTransformer[(Double, Vector, Vector)](df, model, "output", "expected2") { - case Row(output: Vector, expected2: Vector) => - assert(output === expected2) - } - } - - test("OneHotEncoderModel changes handleInvalid") { - val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") - - val testData = Seq( - Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), - Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), - Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) - - val schema = StructType(Array( - StructField("input", DoubleType), - StructField("expected", new VectorUDT))) - - val testDF = spark.createDataFrame(sc.parallelize(testData), schema) - - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("input")) - .setOutputCols(Array("output")) - - val model = encoder.fit(trainingDF) - model.setHandleInvalid("error") - - testTransformerByInterceptingException[(Double, Vector)]( - testDF, - model, - expectedMessagePart = "Unseen value: 3.0. To handle unseen values", - firstResultCol = "output") - - model.setHandleInvalid("keep") - testTransformerByGlobalCheckFunc[(Double, Vector)](testDF, model, "output") { _ => } - } - - test("Transforming on mismatched attributes") { - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", attr.toMetadata())) - val encoder = new OneHotEncoderEstimator() - .setInputCols(Array("size")) - .setOutputCols(Array("encoded")) - val model = encoder.fit(df) - - val testAttr = NominalAttribute.defaultAttr.withValues("tiny", "small", "medium", "large") - val testDF = Seq(0.0, 1.0, 2.0, 3.0).map(Tuple1.apply).toDF("size") - .select(col("size").as("size", testAttr.toMetadata())) - testTransformerByInterceptingException[(Double)]( - testDF, - model, - expectedMessagePart = "OneHotEncoderModel expected 2 categorical values", - firstResultCol = "encoded") - } -} diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala index 41b32b2ffa096..70f8c029a2575 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/OneHotEncoderSuite.scala @@ -18,72 +18,71 @@ package org.apache.spark.ml.feature import org.apache.spark.ml.attribute.{AttributeGroup, BinaryAttribute, NominalAttribute} -import org.apache.spark.ml.linalg.Vector -import org.apache.spark.ml.linalg.Vectors +import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} -import org.apache.spark.sql.{DataFrame, Encoder, Row} +import org.apache.spark.sql.{Encoder, Row} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.functions.col import org.apache.spark.sql.types._ -class OneHotEncoderSuite - extends MLTest with DefaultReadWriteTest { +class OneHotEncoderSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ - def stringIndexed(): DataFrame = { - val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) - val df = data.toDF("id", "label") - val indexer = new StringIndexer() - .setInputCol("label") - .setOutputCol("labelIndex") - .fit(df) - indexer.transform(df) - } - test("params") { ParamsSuite.checkParams(new OneHotEncoder) } test("OneHotEncoder dropLast = false") { - val transformed = stringIndexed() + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") - assert(encoder.getDropLast === true) + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + assert(encoder.getDropLast) encoder.setDropLast(false) assert(encoder.getDropLast === false) - val expected = Seq( - (0, Vectors.sparse(3, Seq((0, 1.0)))), - (1, Vectors.sparse(3, Seq((2, 1.0)))), - (2, Vectors.sparse(3, Seq((1, 1.0)))), - (3, Vectors.sparse(3, Seq((0, 1.0)))), - (4, Vectors.sparse(3, Seq((0, 1.0)))), - (5, Vectors.sparse(3, Seq((1, 1.0))))).toDF("id", "expected") - - val withExpected = transformed.join(expected, "id") - testTransformer[(Int, String, Double, Vector)](withExpected, encoder, "labelVec", "expected") { + val model = encoder.fit(df) + testTransformer[(Double, Vector)](df, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) } } test("OneHotEncoder dropLast = true") { - val transformed = stringIndexed() + val data = Seq( + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(2, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq())), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") - val expected = Seq( - (0, Vectors.sparse(2, Seq((0, 1.0)))), - (1, Vectors.sparse(2, Seq())), - (2, Vectors.sparse(2, Seq((1, 1.0)))), - (3, Vectors.sparse(2, Seq((0, 1.0)))), - (4, Vectors.sparse(2, Seq((0, 1.0)))), - (5, Vectors.sparse(2, Seq((1, 1.0))))).toDF("id", "expected") - - val withExpected = transformed.join(expected, "id") - testTransformer[(Int, String, Double, Vector)](withExpected, encoder, "labelVec", "expected") { + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(df) + testTransformer[(Double, Vector)](df, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) } @@ -94,52 +93,61 @@ class OneHotEncoderSuite val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") .select(col("size").as("size", attr.toMetadata())) val encoder = new OneHotEncoder() - .setInputCol("size") - .setOutputCol("encoded") - testTransformerByGlobalCheckFunc[(Double)](df, encoder, "encoded") { rows => - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) + .setInputCols(Array("size")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => + val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) + assert(group.size === 2) + assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("small").withIndex(0)) + assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("medium").withIndex(1)) } } - test("input column without ML attribute") { val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("index") val encoder = new OneHotEncoder() - .setInputCol("index") - .setOutputCol("encoded") - val rows = encoder.transform(df).select("encoded").collect() - val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) - assert(group.size === 2) - assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) - assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) + .setInputCols(Array("index")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + testTransformerByGlobalCheckFunc[(Double)](df, model, "encoded") { rows => + val group = AttributeGroup.fromStructField(rows.head.schema("encoded")) + assert(group.size === 2) + assert(group.getAttr(0) === BinaryAttribute.defaultAttr.withName("0").withIndex(0)) + assert(group.getAttr(1) === BinaryAttribute.defaultAttr.withName("1").withIndex(1)) + } } test("read/write") { - val t = new OneHotEncoder() - .setInputCol("myInputCol") - .setOutputCol("myOutputCol") - .setDropLast(false) - testDefaultReadWrite(t) + val encoder = new OneHotEncoder() + .setInputCols(Array("index")) + .setOutputCols(Array("encoded")) + testDefaultReadWrite(encoder) + } + + test("OneHotEncoderModel read/write") { + val instance = new OneHotEncoderModel("myOneHotEncoderModel", Array(1, 2, 3)) + val newInstance = testDefaultReadWrite(instance) + assert(newInstance.categorySizes === instance.categorySizes) } test("OneHotEncoder with varying types") { - val df = stringIndexed() - val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") - val expected = Seq( - (0, Vectors.sparse(3, Seq((0, 1.0)))), - (1, Vectors.sparse(3, Seq((2, 1.0)))), - (2, Vectors.sparse(3, Seq((1, 1.0)))), - (3, Vectors.sparse(3, Seq((0, 1.0)))), - (4, Vectors.sparse(3, Seq((0, 1.0)))), - (5, Vectors.sparse(3, Seq((1, 1.0))))).toDF("id", "expected") + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))))) - val withExpected = df.join(expected, "id") + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) class NumericTypeWithEncoder[A](val numericType: NumericType) - (implicit val encoder: Encoder[(A, Vector)]) + (implicit val encoder: Encoder[(A, Vector)]) val types = Seq( new NumericTypeWithEncoder[Short](ShortType), @@ -151,17 +159,264 @@ class OneHotEncoderSuite new NumericTypeWithEncoder[Decimal](DecimalType(10, 0))(ExpressionEncoder())) for (t <- types) { - val dfWithTypes = withExpected.select(col("labelIndex") - .cast(t.numericType).as("labelIndex", attr.toMetadata()), col("expected")) - val encoder = new OneHotEncoder() - .setInputCol("labelIndex") - .setOutputCol("labelVec") + val dfWithTypes = df.select(col("input").cast(t.numericType), col("expected")) + val estimator = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) .setDropLast(false) - testTransformer(dfWithTypes, encoder, "labelVec", "expected") { + val model = estimator.fit(dfWithTypes) + testTransformer(dfWithTypes, model, "output", "expected") { case Row(output: Vector, expected: Vector) => assert(output === expected) }(t.encoder) } } + + test("OneHotEncoder: encoding multiple columns and dropLast = false") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), 3.0, Vectors.sparse(4, Seq((3, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), 0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), 2.0, Vectors.sparse(4, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input1", DoubleType), + StructField("expected1", new VectorUDT), + StructField("input2", DoubleType), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + assert(encoder.getDropLast) + encoder.setDropLast(false) + assert(encoder.getDropLast === false) + + val model = encoder.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model, + "output1", + "output2", + "expected1", + "expected2") { + case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => + assert(output1 === expected1) + assert(output2 === expected2) + } + } + + test("OneHotEncoder: encoding multiple columns and dropLast = true") { + val data = Seq( + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 2.0, Vectors.sparse(3, Seq((2, 1.0)))), + Row(1.0, Vectors.sparse(2, Seq((1, 1.0))), 3.0, Vectors.sparse(3, Seq())), + Row(2.0, Vectors.sparse(2, Seq()), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(0.0, Vectors.sparse(2, Seq((0, 1.0))), 0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(2, Seq()), 2.0, Vectors.sparse(3, Seq((2, 1.0))))) + + val schema = StructType(Array( + StructField("input1", DoubleType), + StructField("expected1", new VectorUDT), + StructField("input2", DoubleType), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input1", "input2")) + .setOutputCols(Array("output1", "output2")) + + val model = encoder.fit(df) + testTransformer[(Double, Vector, Double, Vector)]( + df, + model, + "output1", + "output2", + "expected1", + "expected2") { + case Row(output1: Vector, output2: Vector, expected1: Vector, expected2: Vector) => + assert(output1 === expected1) + assert(output2 === expected2) + } + } + + test("Throw error on invalid values") { + val trainingData = Seq((0, 0), (1, 1), (2, 2)) + val trainingDF = trainingData.toDF("id", "a") + val testData = Seq((0, 0), (1, 2), (1, 3)) + val testDF = testData.toDF("id", "a") + + val encoder = new OneHotEncoder() + .setInputCols(Array("a")) + .setOutputCols(Array("encoded")) + + val model = encoder.fit(trainingDF) + testTransformerByInterceptingException[(Int, Int)]( + testDF, + model, + expectedMessagePart = "Unseen value: 3.0. To handle unseen values", + firstResultCol = "encoded") + + } + + test("Can't transform on negative input") { + val trainingDF = Seq((0, 0), (1, 1), (2, 2)).toDF("a", "b") + val testDF = Seq((0, 0), (-1, 2), (1, 3)).toDF("a", "b") + + val encoder = new OneHotEncoder() + .setInputCols(Array("a")) + .setOutputCols(Array("encoded")) + + val model = encoder.fit(trainingDF) + testTransformerByInterceptingException[(Int, Int)]( + testDF, + model, + expectedMessagePart = "Negative value: -1.0. Input can't be negative", + firstResultCol = "encoded") + } + + test("Keep on invalid values: dropLast = false") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + .setHandleInvalid("keep") + .setDropLast(false) + + val model = encoder.fit(trainingDF) + testTransformer[(Double, Vector)](testDF, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + } + } + + test("Keep on invalid values: dropLast = true") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(3, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + .setHandleInvalid("keep") + .setDropLast(true) + + val model = encoder.fit(trainingDF) + testTransformer[(Double, Vector)](testDF, model, "output", "expected") { + case Row(output: Vector, expected: Vector) => + assert(output === expected) + } + } + + test("OneHotEncoderModel changes dropLast") { + val data = Seq( + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(3, Seq((1, 1.0))), Vectors.sparse(2, Seq((1, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq())), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(0.0, Vectors.sparse(3, Seq((0, 1.0))), Vectors.sparse(2, Seq((0, 1.0)))), + Row(2.0, Vectors.sparse(3, Seq((2, 1.0))), Vectors.sparse(2, Seq()))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected1", new VectorUDT), + StructField("expected2", new VectorUDT))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(df) + + model.setDropLast(false) + testTransformer[(Double, Vector, Vector)](df, model, "output", "expected1") { + case Row(output: Vector, expected1: Vector) => + assert(output === expected1) + } + + model.setDropLast(true) + testTransformer[(Double, Vector, Vector)](df, model, "output", "expected2") { + case Row(output: Vector, expected2: Vector) => + assert(output === expected2) + } + } + + test("OneHotEncoderModel changes handleInvalid") { + val trainingDF = Seq(Tuple1(0), Tuple1(1), Tuple1(2)).toDF("input") + + val testData = Seq( + Row(0.0, Vectors.sparse(4, Seq((0, 1.0)))), + Row(1.0, Vectors.sparse(4, Seq((1, 1.0)))), + Row(3.0, Vectors.sparse(4, Seq((3, 1.0))))) + + val schema = StructType(Array( + StructField("input", DoubleType), + StructField("expected", new VectorUDT))) + + val testDF = spark.createDataFrame(sc.parallelize(testData), schema) + + val encoder = new OneHotEncoder() + .setInputCols(Array("input")) + .setOutputCols(Array("output")) + + val model = encoder.fit(trainingDF) + model.setHandleInvalid("error") + + testTransformerByInterceptingException[(Double, Vector)]( + testDF, + model, + expectedMessagePart = "Unseen value: 3.0. To handle unseen values", + firstResultCol = "output") + + model.setHandleInvalid("keep") + testTransformerByGlobalCheckFunc[(Double, Vector)](testDF, model, "output") { _ => } + } + + test("Transforming on mismatched attributes") { + val attr = NominalAttribute.defaultAttr.withValues("small", "medium", "large") + val df = Seq(0.0, 1.0, 2.0, 1.0).map(Tuple1.apply).toDF("size") + .select(col("size").as("size", attr.toMetadata())) + val encoder = new OneHotEncoder() + .setInputCols(Array("size")) + .setOutputCols(Array("encoded")) + val model = encoder.fit(df) + + val testAttr = NominalAttribute.defaultAttr.withValues("tiny", "small", "medium", "large") + val testDF = Seq(0.0, 1.0, 2.0, 3.0).map(Tuple1.apply).toDF("size") + .select(col("size").as("size", testAttr.toMetadata())) + testTransformerByInterceptingException[(Double)]( + testDF, + model, + expectedMessagePart = "OneHotEncoderModel expected 2 categorical values", + firstResultCol = "encoded") + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala index b009038bbd833..ae086d32d6d0b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/QuantileDiscretizerSuite.scala @@ -29,25 +29,30 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { val spark = this.spark import spark.implicits._ - val datasetSize = 100000 + val datasetSize = 30000 val numBuckets = 5 - val df = sc.parallelize(1.0 to datasetSize by 1.0).map(Tuple1.apply).toDF("input") + val df = sc.parallelize(1 to datasetSize).map(_.toDouble).toDF("input") val discretizer = new QuantileDiscretizer() .setInputCol("input") .setOutputCol("result") .setNumBuckets(numBuckets) val model = discretizer.fit(df) - testTransformerByGlobalCheckFunc[(Double)](df, model, "result") { rows => - val result = rows.map { r => Tuple1(r.getDouble(0)) }.toDF("result") - val observedNumBuckets = result.select("result").distinct.count - assert(observedNumBuckets === numBuckets, - "Observed number of buckets does not equal expected number of buckets.") - val relativeError = discretizer.getRelativeError - val numGoodBuckets = result.groupBy("result").count - .filter(s"abs(count - ${datasetSize / numBuckets}) <= ${relativeError * datasetSize}").count - assert(numGoodBuckets === numBuckets, - "Bucket sizes are not within expected relative error tolerance.") + testTransformerByGlobalCheckFunc[Double](df, model, "result") { rows => + val result = rows.map(_.getDouble(0)).toDF("result").cache() + try { + val observedNumBuckets = result.select("result").distinct().count() + assert(observedNumBuckets === numBuckets, + "Observed number of buckets does not equal expected number of buckets.") + val relativeError = discretizer.getRelativeError + val numGoodBuckets = result.groupBy("result").count() + .filter(s"abs(count - ${datasetSize / numBuckets}) <= ${relativeError * datasetSize}") + .count() + assert(numGoodBuckets === numBuckets, + "Bucket sizes are not within expected relative error tolerance.") + } finally { + result.unpersist() + } } } @@ -114,8 +119,8 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { val spark = this.spark import spark.implicits._ - val trainDF = sc.parallelize(1.0 to 100.0 by 1.0).map(Tuple1.apply).toDF("input") - val testDF = sc.parallelize(-10.0 to 110.0 by 1.0).map(Tuple1.apply).toDF("input") + val trainDF = sc.parallelize((1 to 100).map(_.toDouble)).map(Tuple1.apply).toDF("input") + val testDF = sc.parallelize((-10 to 110).map(_.toDouble)).map(Tuple1.apply).toDF("input") val discretizer = new QuantileDiscretizer() .setInputCol("input") .setOutputCol("result") @@ -162,10 +167,10 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { val spark = this.spark import spark.implicits._ - val datasetSize = 100000 + val datasetSize = 30000 val numBuckets = 5 - val data1 = Array.range(1, 100001, 1).map(_.toDouble) - val data2 = Array.range(1, 200000, 2).map(_.toDouble) + val data1 = Array.range(1, datasetSize + 1, 1).map(_.toDouble) + val data2 = Array.range(1, 2 * datasetSize, 2).map(_.toDouble) val df = data1.zip(data2).toSeq.toDF("input1", "input2") val discretizer = new QuantileDiscretizer() @@ -175,20 +180,24 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { val model = discretizer.fit(df) testTransformerByGlobalCheckFunc[(Double, Double)](df, model, "result1", "result2") { rows => val result = - rows.map { r => Tuple2(r.getDouble(0), r.getDouble(1)) }.toDF("result1", "result2") - val relativeError = discretizer.getRelativeError - for (i <- 1 to 2) { - val observedNumBuckets = result.select("result" + i).distinct.count - assert(observedNumBuckets === numBuckets, - "Observed number of buckets does not equal expected number of buckets.") - - val numGoodBuckets = result - .groupBy("result" + i) - .count - .filter(s"abs(count - ${datasetSize / numBuckets}) <= ${relativeError * datasetSize}") - .count - assert(numGoodBuckets === numBuckets, - "Bucket sizes are not within expected relative error tolerance.") + rows.map(r => (r.getDouble(0), r.getDouble(1))).toDF("result1", "result2").cache() + try { + val relativeError = discretizer.getRelativeError + for (i <- 1 to 2) { + val observedNumBuckets = result.select("result" + i).distinct().count() + assert(observedNumBuckets === numBuckets, + "Observed number of buckets does not equal expected number of buckets.") + + val numGoodBuckets = result + .groupBy("result" + i) + .count() + .filter(s"abs(count - ${datasetSize / numBuckets}) <= ${relativeError * datasetSize}") + .count() + assert(numGoodBuckets === numBuckets, + "Bucket sizes are not within expected relative error tolerance.") + } + } finally { + result.unpersist() } } } @@ -276,10 +285,10 @@ class QuantileDiscretizerSuite extends MLTest with DefaultReadWriteTest { 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0, 1.0) val data2 = Array.range(1, 40, 2).map(_.toDouble) val expected2 = Array (0.0, 0.0, 0.0, 1.0, 1.0, 1.0, 1.0, 2.0, 2.0, 2.0, - 2.0, 2.0, 3.0, 3.0, 3.0, 4.0, 4.0, 4.0, 4.0, 4.0) + 2.0, 3.0, 3.0, 3.0, 3.0, 4.0, 4.0, 4.0, 4.0, 4.0) val data3 = Array.range(1, 60, 3).map(_.toDouble) - val expected3 = Array (0.0, 1.0, 1.0, 2.0, 2.0, 2.0, 3.0, 4.0, 4.0, 5.0, - 5.0, 5.0, 6.0, 6.0, 7.0, 8.0, 8.0, 9.0, 9.0, 9.0) + val expected3 = Array (0.0, 1.0, 1.0, 2.0, 2.0, 3.0, 3.0, 4.0, 4.0, 5.0, + 5.0, 6.0, 6.0, 7.0, 7.0, 8.0, 8.0, 9.0, 9.0, 9.0) val data = (0 until 20).map { idx => (data1(idx), data2(idx), data3(idx), expected1(idx), expected2(idx), expected3(idx)) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaParserSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaParserSuite.scala index 53798c659d4f3..add1cc17ea057 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaParserSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaParserSuite.scala @@ -90,12 +90,48 @@ class RFormulaParserSuite extends SparkFunSuite { test("parse interactions") { checkParse("y ~ a:b", "y", Seq("a:b")) + checkParse("y ~ a:b + b:a", "y", Seq("a:b")) checkParse("y ~ ._a:._x", "y", Seq("._a:._x")) checkParse("y ~ foo:bar", "y", Seq("foo:bar")) checkParse("y ~ a : b : c", "y", Seq("a:b:c")) checkParse("y ~ q + a:b:c + b:c + c:d + z", "y", Seq("q", "a:b:c", "b:c", "c:d", "z")) } + test("parse factor cross") { + checkParse("y ~ a*b", "y", Seq("a", "b", "a:b")) + checkParse("y ~ a*b + b*a", "y", Seq("a", "b", "a:b")) + checkParse("y ~ ._a*._x", "y", Seq("._a", "._x", "._a:._x")) + checkParse("y ~ foo*bar", "y", Seq("foo", "bar", "foo:bar")) + checkParse("y ~ a * b * c", "y", Seq("a", "b", "a:b", "c", "a:c", "b:c", "a:b:c")) + } + + test("interaction distributive") { + checkParse("y ~ (a + b):c", "y", Seq("a:c", "b:c")) + checkParse("y ~ c:(a + b)", "y", Seq("c:a", "c:b")) + } + + test("factor cross distributive") { + checkParse("y ~ (a + b)*c", "y", Seq("a", "b", "c", "a:c", "b:c")) + checkParse("y ~ c*(a + b)", "y", Seq("c", "a", "b", "c:a", "c:b")) + } + + test("parse power") { + val schema = (new StructType) + .add("a", "int", true) + .add("b", "long", false) + .add("c", "string", true) + .add("d", "string", true) + checkParse("a ~ (a + b)^2", "a", Seq("a", "b", "a:b")) + checkParse("a ~ .^2", "a", Seq("b", "c", "d", "b:c", "b:d", "c:d"), schema) + checkParse("a ~ .^3", "a", Seq("b", "c", "d", "b:c", "b:d", "c:d", "b:c:d"), schema) + checkParse("a ~ .^3-.", "a", Seq("b:c", "b:d", "c:d", "b:c:d"), schema) + } + + test("operator precedence") { + checkParse("y ~ a*b:c", "y", Seq("a", "b:c", "a:b:c")) + checkParse("y ~ (a*b):c", "y", Seq("a:c", "b:c", "a:b:c")) + } + test("parse basic interactions with dot") { val schema = (new StructType) .add("a", "int", true) diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala index 0de6528c4cf22..675e7b6311728 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/RFormulaSuite.scala @@ -137,14 +137,17 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { test("encodes string terms") { val formula = new RFormula().setFormula("id ~ a + b") - val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5), + (5, "bar", 6), (6, "foo", 6)) .toDF("id", "a", "b") val model = formula.fit(original) val expected = Seq( (1, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), (2, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 2.0), (3, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 3.0), - (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0) + (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0), + (5, "bar", 6, Vectors.dense(1.0, 0.0, 6.0), 5.0), + (6, "foo", 6, Vectors.dense(0.0, 1.0, 6.0), 6.0) ).toDF("id", "a", "b", "features", "label") testRFormulaTransform[(Int, String, Int)](original, model, expected) } @@ -303,7 +306,8 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { test("index string label") { val formula = new RFormula().setFormula("id ~ a + b") val original = - Seq(("male", "foo", 4), ("female", "bar", 4), ("female", "bar", 5), ("male", "baz", 5)) + Seq(("male", "foo", 4), ("female", "bar", 4), ("female", "bar", 5), ("male", "baz", 5), + ("female", "bar", 6), ("female", "foo", 6)) .toDF("id", "a", "b") val model = formula.fit(original) val attr = NominalAttribute.defaultAttr @@ -311,7 +315,9 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { ("male", "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), ("female", "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), ("female", "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 0.0), - ("male", "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 1.0) + ("male", "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 1.0), + ("female", "bar", 6, Vectors.dense(1.0, 0.0, 6.0), 0.0), + ("female", "foo", 6, Vectors.dense(0.0, 1.0, 6.0), 0.0) ).toDF("id", "a", "b", "features", "label") .select($"id", $"a", $"b", $"features", $"label".as("label", attr.toMetadata())) testRFormulaTransform[(String, String, Int)](original, model, expected) @@ -320,7 +326,8 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { test("force to index label even it is numeric type") { val formula = new RFormula().setFormula("id ~ a + b").setForceIndexLabel(true) val original = spark.createDataFrame( - Seq((1.0, "foo", 4), (1.0, "bar", 4), (0.0, "bar", 5), (1.0, "baz", 5)) + Seq((1.0, "foo", 4), (1.0, "bar", 4), (0.0, "bar", 5), (1.0, "baz", 5), + (1.0, "bar", 6), (0.0, "foo", 6)) ).toDF("id", "a", "b") val model = formula.fit(original) val attr = NominalAttribute.defaultAttr @@ -328,7 +335,9 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { (1.0, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 0.0), (1.0, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 0.0), (0.0, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 1.0), - (1.0, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 0.0)) + (1.0, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 0.0), + (1.0, "bar", 6, Vectors.dense(1.0, 0.0, 6.0), 0.0), + (0.0, "foo", 6, Vectors.dense(0.0, 1.0, 6.0), 1.0)) .toDF("id", "a", "b", "features", "label") .select($"id", $"a", $"b", $"features", $"label".as("label", attr.toMetadata())) testRFormulaTransform[(Double, String, Int)](original, model, expected) @@ -336,14 +345,17 @@ class RFormulaSuite extends MLTest with DefaultReadWriteTest { test("attribute generation") { val formula = new RFormula().setFormula("id ~ a + b") - val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5)) + val original = Seq((1, "foo", 4), (2, "bar", 4), (3, "bar", 5), (4, "baz", 5), + (1, "bar", 6), (0, "foo", 6)) .toDF("id", "a", "b") val model = formula.fit(original) val expected = Seq( (1, "foo", 4, Vectors.dense(0.0, 1.0, 4.0), 1.0), (2, "bar", 4, Vectors.dense(1.0, 0.0, 4.0), 2.0), (3, "bar", 5, Vectors.dense(1.0, 0.0, 5.0), 3.0), - (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0)) + (4, "baz", 5, Vectors.dense(0.0, 0.0, 5.0), 4.0), + (1, "bar", 6, Vectors.dense(1.0, 0.0, 6.0), 1.0), + (0, "foo", 6, Vectors.dense(0.0, 1.0, 6.0), 0.0)) .toDF("id", "a", "b", "features", "label") val expectedAttrs = new AttributeGroup( "features", diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala index df24367177011..f542e342ffaa4 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/StringIndexerSuite.scala @@ -30,12 +30,46 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { test("params") { ParamsSuite.checkParams(new StringIndexer) - val model = new StringIndexerModel("indexer", Array("a", "b")) + val model = new StringIndexerModel("indexer", Array(Array("a", "b"))) val modelWithoutUid = new StringIndexerModel(Array("a", "b")) ParamsSuite.checkParams(model) ParamsSuite.checkParams(modelWithoutUid) } + test("params: input/output columns") { + val stringIndexerSingleCol = new StringIndexer() + .setInputCol("in").setOutputCol("out") + val inOutCols1 = stringIndexerSingleCol.getInOutCols() + assert(inOutCols1._1 === Array("in")) + assert(inOutCols1._2 === Array("out")) + + val stringIndexerMultiCol = new StringIndexer() + .setInputCols(Array("in1", "in2")).setOutputCols(Array("out1", "out2")) + val inOutCols2 = stringIndexerMultiCol.getInOutCols() + assert(inOutCols2._1 === Array("in1", "in2")) + assert(inOutCols2._2 === Array("out1", "out2")) + + + val df = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")).toDF("id", "label") + + intercept[IllegalArgumentException] { + new StringIndexer().setInputCol("in").setOutputCols(Array("out1", "out2")).fit(df) + } + intercept[IllegalArgumentException] { + new StringIndexer().setInputCols(Array("in1", "in2")).setOutputCol("out1").fit(df) + } + intercept[IllegalArgumentException] { + new StringIndexer().setInputCols(Array("in1", "in2")) + .setOutputCols(Array("out1", "out2", "out3")) + .fit(df) + } + intercept[IllegalArgumentException] { + new StringIndexer().setInputCols(Array("in1", "in2")) + .setOutputCols(Array("out1", "out1")) + .fit(df) + } + } + test("StringIndexer") { val data = Seq((0, "a"), (1, "b"), (2, "c"), (3, "a"), (4, "a"), (5, "c")) val df = data.toDF("id", "label") @@ -51,7 +85,7 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { (2, 1.0), (3, 0.0), (4, 0.0), - (5, 1.0) + (5, 1.0) ).toDF("id", "labelIndex") testTransformerByGlobalCheckFunc[(Int, String)](df, indexerModel, "id", "labelIndex") { rows => @@ -167,7 +201,7 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { } test("StringIndexerModel should keep silent if the input column does not exist.") { - val indexerModel = new StringIndexerModel("indexer", Array("a", "b", "c")) + val indexerModel = new StringIndexerModel("indexer", Array(Array("a", "b", "c"))) .setInputCol("label") .setOutputCol("labelIndex") val df = spark.range(0L, 10L).toDF() @@ -207,7 +241,7 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { } test("StringIndexerModel read/write") { - val instance = new StringIndexerModel("myStringIndexerModel", Array("a", "b", "c")) + val instance = new StringIndexerModel("myStringIndexerModel", Array(Array("a", "b", "c"))) .setInputCol("myInputCol") .setOutputCol("myOutputCol") .setHandleInvalid("skip") @@ -323,11 +357,32 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { } } + test("StringIndexer order types: secondary sort by alphabets when frequency equal") { + val data = Seq((0, "a"), (1, "a"), (2, "b"), (3, "b"), (4, "c"), (5, "d")) + val df = data.toDF("id", "label") + val indexer = new StringIndexer() + .setInputCol("label") + .setOutputCol("labelIndex") + + val expected = Seq(Set((0, 0.0), (1, 0.0), (2, 1.0), (3, 1.0), (4, 2.0), (5, 3.0)), + Set((0, 2.0), (1, 2.0), (2, 3.0), (3, 3.0), (4, 0.0), (5, 1.0))) + + var idx = 0 + for (orderType <- Seq("frequencyDesc", "frequencyAsc")) { + val transformed = indexer.setStringOrderType(orderType).fit(df).transform(df) + val output = transformed.select("id", "labelIndex").rdd.map { r => + (r.getInt(0), r.getDouble(1)) + }.collect().toSet + assert(output === expected(idx)) + idx += 1 + } + } + test("SPARK-22446: StringIndexerModel's indexer UDF should not apply on filtered data") { val df = List( - ("A", "London", "StrA"), - ("B", "Bristol", null), - ("C", "New York", "StrC")).toDF("ID", "CITY", "CONTENT") + ("A", "London", "StrA"), + ("B", "Bristol", null), + ("C", "New York", "StrC")).toDF("ID", "CITY", "CONTENT") val dfNoBristol = df.filter($"CONTENT".isNotNull) @@ -343,4 +398,74 @@ class StringIndexerSuite extends MLTest with DefaultReadWriteTest { assert(rows.toList.count(_.getDouble(0) == 1.0) === 1) } } + + test("StringIndexer multiple input columns") { + val data = Seq( + Row("a", 0.0, "e", 1.0), + Row("b", 2.0, "f", 0.0), + Row("c", 1.0, "e", 1.0), + Row("a", 0.0, "f", 0.0), + Row("a", 0.0, "f", 0.0), + Row("c", 1.0, "f", 0.0)) + + val schema = StructType(Array( + StructField("label1", StringType), + StructField("expected1", DoubleType), + StructField("label2", StringType), + StructField("expected2", DoubleType))) + + val df = spark.createDataFrame(sc.parallelize(data), schema) + + val indexer = new StringIndexer() + .setInputCols(Array("label1", "label2")) + .setOutputCols(Array("labelIndex1", "labelIndex2")) + val indexerModel = indexer.fit(df) + + MLTestingUtils.checkCopyAndUids(indexer, indexerModel) + + val transformed = indexerModel.transform(df) + + // Checks output attribute correctness. + val attr1 = Attribute.fromStructField(transformed.schema("labelIndex1")) + .asInstanceOf[NominalAttribute] + assert(attr1.values.get === Array("a", "c", "b")) + val attr2 = Attribute.fromStructField(transformed.schema("labelIndex2")) + .asInstanceOf[NominalAttribute] + assert(attr2.values.get === Array("f", "e")) + + transformed.select("labelIndex1", "expected1").rdd.map { r => + (r.getDouble(0), r.getDouble(1)) + }.collect().foreach { case (index, expected) => + assert(index == expected) + } + + transformed.select("labelIndex2", "expected2").rdd.map { r => + (r.getDouble(0), r.getDouble(1)) + }.collect().foreach { case (index, expected) => + assert(index == expected) + } + } + + test("Correctly skipping NULL and NaN values") { + val df = Seq(("a", Double.NaN), (null, 1.0), ("b", 2.0), (null, 3.0)).toDF("str", "double") + + val indexer = new StringIndexer() + .setInputCols(Array("str", "double")) + .setOutputCols(Array("strIndex", "doubleIndex")) + + val model = indexer.fit(df) + assert(model.labelsArray(0) === Array("a", "b")) + assert(model.labelsArray(1) === Array("1.0", "2.0", "3.0")) + } + + test("Load StringIndexderModel prior to Spark 3.0") { + val modelPath = testFile("test-data/strIndexerModel") + + val loadedModel = StringIndexerModel.load(modelPath) + assert(loadedModel.labelsArray === Array(Array("b", "c", "a"))) + + val metadata = spark.read.json(s"$modelPath/metadata") + val sparkVersionStr = metadata.select("sparkVersion").first().getString(0) + assert(sparkVersionStr == "2.4.1-SNAPSHOT") + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala index be59b0af2c78e..ba8e79f14de95 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/TokenizerSuite.scala @@ -17,14 +17,14 @@ package org.apache.spark.ml.feature -import scala.beans.BeanInfo - import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest} import org.apache.spark.sql.{DataFrame, Row} -@BeanInfo -case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) +case class TokenizerTestData(rawText: String, wantedTokens: Array[String]) { + def getRawText: String = rawText + def getWantedTokens: Array[String] = wantedTokens +} class TokenizerSuite extends MLTest with DefaultReadWriteTest { diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala index e5675e31bbecf..44b0f8f8ae7d8 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/VectorIndexerSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.ml.feature -import scala.beans.{BeanInfo, BeanProperty} - import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.ml.attribute._ @@ -26,7 +24,7 @@ import org.apache.spark.ml.linalg.{SparseVector, Vector, Vectors} import org.apache.spark.ml.param.ParamsSuite import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{DataFrame, Row} +import org.apache.spark.sql.DataFrame class VectorIndexerSuite extends MLTest with DefaultReadWriteTest with Logging { @@ -283,7 +281,9 @@ class VectorIndexerSuite extends MLTest with DefaultReadWriteTest with Logging { points.zip(rows.map(_(0))).foreach { case (orig: SparseVector, indexed: SparseVector) => assert(orig.indices.length == indexed.indices.length) - case _ => throw new UnknownError("Unit test has a bug in it.") // should never happen + case _ => + // should never happen + fail("Unit test has a bug in it.") } } } @@ -337,6 +337,7 @@ class VectorIndexerSuite extends MLTest with DefaultReadWriteTest with Logging { } private[feature] object VectorIndexerSuite { - @BeanInfo - case class FeatureData(@BeanProperty features: Vector) + case class FeatureData(features: Vector) { + def getFeatures: Vector = features + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala index b59c4e7967338..d28f1f4240ad0 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/feature/Word2VecSuite.scala @@ -65,7 +65,7 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { // These expectations are just magic values, characterizing the current // behavior. The test needs to be updated to be more general, see SPARK-11502 - val magicExp = Vectors.dense(0.30153007534417237, -0.6833061711354689, 0.5116530778733167) + val magicExp = Vectors.dense(-0.11654884266582402, 0.3115301721475341, -0.6879349987615239) testTransformer[(Seq[String], Vector)](docDF, model, "result", "expected") { case Row(vector1: Vector, vector2: Vector) => assert(vector1 ~== magicExp absTol 1E-5, "Transformed vector is different with expected.") @@ -98,9 +98,9 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { // These expectations are just magic values, characterizing the current // behavior. The test needs to be updated to be more general, see SPARK-11502 val magicExpected = Seq( - Vectors.dense(0.3326166272163391, -0.5603077411651611, -0.2309209555387497), - Vectors.dense(0.32463887333869934, -0.9306551218032837, 1.393115520477295), - Vectors.dense(-0.27150997519493103, 0.4372006058692932, -0.13465698063373566) + Vectors.dense(0.12662248313426971, 0.6108677387237549, -0.006755620241165161), + Vectors.dense(-0.3870747685432434, 0.023309476673603058, -1.567158818244934), + Vectors.dense(-0.08617416769266129, -0.09897610545158386, 0.6113300323486328) ) realVectors.zip(magicExpected).foreach { @@ -122,7 +122,7 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { .setSeed(42L) .fit(docDF) - val expected = Map(("b", 0.2608488929093532), ("c", -0.8271274846926078)) + val expected = Map(("b", -0.024012837558984756), ("c", -0.19355152547359467)) val findSynonymsResult = model.findSynonyms("a", 2).rdd.map { case Row(w: String, sim: Double) => (w, sim) }.collectAsMap() @@ -130,14 +130,14 @@ class Word2VecSuite extends MLTest with DefaultReadWriteTest { expected.foreach { case (expectedSynonym, expectedSimilarity) => assert(findSynonymsResult.contains(expectedSynonym)) - assert(expectedSimilarity ~== findSynonymsResult.get(expectedSynonym).get absTol 1E-5) + assert(expectedSimilarity ~== findSynonymsResult(expectedSynonym) absTol 1E-5) } val findSynonymsArrayResult = model.findSynonymsArray("a", 2).toMap findSynonymsResult.foreach { case (expectedSynonym, expectedSimilarity) => assert(findSynonymsArrayResult.contains(expectedSynonym)) - assert(expectedSimilarity ~== findSynonymsArrayResult.get(expectedSynonym).get absTol 1E-5) + assert(expectedSimilarity ~== findSynonymsArrayResult(expectedSynonym) absTol 1E-5) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala index bdceba7887cac..8371c33a209dc 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/MatrixUDTSuite.scala @@ -31,7 +31,7 @@ class MatrixUDTSuite extends SparkFunSuite { val sm3 = dm3.toSparse for (m <- Seq(dm1, dm2, dm3, sm1, sm2, sm3)) { - val udt = UDTRegistration.getUDTFor(m.getClass.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(m.getClass.getName).get.getConstructor().newInstance() .asInstanceOf[MatrixUDT] assert(m === udt.deserialize(udt.serialize(m))) assert(udt.typeName == "matrix") diff --git a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala index 6ddb12cb76aac..67c64f762b25e 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/linalg/VectorUDTSuite.scala @@ -31,7 +31,7 @@ class VectorUDTSuite extends SparkFunSuite { val sv2 = Vectors.sparse(2, Array(1), Array(2.0)) for (v <- Seq(dv1, dv2, sv1, sv2)) { - val udt = UDTRegistration.getUDTFor(v.getClass.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(v.getClass.getName).get.getConstructor().newInstance() .asInstanceOf[VectorUDT] assert(v === udt.deserialize(udt.serialize(v))) assert(udt.typeName == "vector") 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 9a59c41740daf..5ba39284f63be 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 @@ -23,7 +23,6 @@ import java.util.Random import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.{ArrayBuffer, WrappedArray} -import scala.language.existentials import com.github.fommil.netlib.BLAS.{getInstance => blas} import org.apache.commons.io.FileUtils @@ -601,7 +600,7 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { val df = maybeDf.get._2 val expected = estimator.fit(df) - val actuals = dfs.filter(_ != baseType).map(t => (t, estimator.fit(t._2))) + val actuals = dfs.map(t => (t, estimator.fit(t._2))) actuals.foreach { case (_, actual) => check(expected, actual) } actuals.foreach { case (t, actual) => check2(expected, actual, t._2, t._1.encoder) } @@ -696,12 +695,14 @@ class ALSSuite extends MLTest with DefaultReadWriteTest with Logging { withClue("transform should fail when ids exceed integer range. ") { val model = als.fit(df) def testTransformIdExceedsIntRange[A : Encoder](dataFrame: DataFrame): Unit = { - assert(intercept[SparkException] { + val e1 = intercept[SparkException] { model.transform(dataFrame).first - }.getMessage.contains(msg)) - assert(intercept[StreamingQueryException] { + } + TestUtils.assertExceptionMsg(e1, msg) + val e2 = intercept[StreamingQueryException] { testTransformer[A](dataFrame, model, "prediction") { _ => } - }.getMessage.contains(msg)) + } + TestUtils.assertExceptionMsg(e2, msg) } testTransformIdExceedsIntRange[(Long, Int)](df.select(df("user_big").as("user"), df("item"))) diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala index 29a438396516b..38bfa6626e3c6 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/DecisionTreeRegressorSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.ml.regression import org.apache.spark.SparkFunSuite -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.Vector import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} @@ -26,6 +26,7 @@ import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.regression.{LabeledPoint => OldLabeledPoint} import org.apache.spark.mllib.tree.{DecisionTree => OldDecisionTree, DecisionTreeSuite => OldDecisionTreeSuite} +import org.apache.spark.mllib.util.LinearDataGenerator import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row} @@ -35,11 +36,17 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { import testImplicits._ private var categoricalDataPointsRDD: RDD[LabeledPoint] = _ + private var linearRegressionData: DataFrame = _ + + private val seed = 42 override def beforeAll() { super.beforeAll() categoricalDataPointsRDD = sc.parallelize(OldDecisionTreeSuite.generateCategoricalDataPoints().map(_.asML)) + linearRegressionData = sc.parallelize(LinearDataGenerator.generateLinearInput( + intercept = 6.3, weights = Array(4.7, 7.2), xMean = Array(0.9, -1.3), + xVariance = Array(0.7, 1.2), nPoints = 1000, seed, eps = 0.5), 2).map(_.asML).toDF() } ///////////////////////////////////////////////////////////////////////////// @@ -88,7 +95,7 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { val df = TreeTests.setMetadata(categoricalDataPointsRDD, categoricalFeatures, numClasses = 0) val model = dt.fit(df) - testTransformer[(Vector, Double)](df, model, "features", "variance") { + testTransformer[(Vector, Double, Double)](df, model, "features", "variance") { case Row(features: Vector, variance: Double) => val expectedVariance = model.rootNode.predictImpl(features).impurityStats.calculate() assert(variance === expectedVariance, @@ -101,7 +108,7 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { .setMaxBins(6) .setSeed(0) - testTransformerByGlobalCheckFunc[(Vector, Double)](varianceDF, dt.fit(varianceDF), + testTransformerByGlobalCheckFunc[(Vector, Double, Double)](varianceDF, dt.fit(varianceDF), "variance") { case rows: Seq[Row] => val calculatedVariances = rows.map(_.getDouble(0)) @@ -159,6 +166,28 @@ class DecisionTreeRegressorSuite extends MLTest with DefaultReadWriteTest { } } + test("training with sample weights") { + val df = linearRegressionData + val numClasses = 0 + val testParams = Seq(5, 10) + for (maxDepth <- testParams) { + val estimator = new DecisionTreeRegressor() + .setMaxDepth(maxDepth) + .setMinWeightFractionPerNode(0.05) + .setSeed(123) + MLTestingUtils.testArbitrarilyScaledWeights[DecisionTreeRegressionModel, + DecisionTreeRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.1, 0.99)) + MLTestingUtils.testOutliersWithSmallWeights[DecisionTreeRegressionModel, + DecisionTreeRegressor](df.as[LabeledPoint], estimator, numClasses, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.1, 0.99), + outlierRatio = 2) + MLTestingUtils.testOversamplingVsWeighting[DecisionTreeRegressionModel, + DecisionTreeRegressor](df.as[LabeledPoint], estimator, + MLTestingUtils.modelPredictionEquals(df, _ ~= _ relTol 0.01, 1.0), seed) + } + } + ///////////////////////////////////////////////////////////////////////////// // Tests of model save/load ///////////////////////////////////////////////////////////////////////////// diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala index b145c7a3dc952..f35c8c64bea65 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GBTRegressorSuite.scala @@ -184,7 +184,7 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { val gbt = new GBTRegressor() .setMaxDepth(3) .setMaxIter(5) - .setSeed(123) + .setSeed(42) .setFeatureSubsetStrategy("all") // In this data, feature 1 is very important. @@ -200,7 +200,8 @@ class GBTRegressorSuite extends MLTest with DefaultReadWriteTest { val gbtWithFeatureSubset = gbt.setFeatureSubsetStrategy("1") val importanceFeatures = gbtWithFeatureSubset.fit(df).featureImportances val mostIF = importanceFeatures.argmax - assert(mostImportantFeature !== mostIF) + assert(mostIF === 1) + assert(importances(mostImportantFeature) !== importanceFeatures(mostIF)) } test("model evaluateEachIteration") { diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala index 600a43242751f..a30c47293c543 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/GeneralizedLinearRegressionSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.ml.regression import scala.util.Random -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkConf import org.apache.spark.ml.classification.LogisticRegressionSuite._ import org.apache.spark.ml.feature.{Instance, OffsetInstance} import org.apache.spark.ml.feature.{LabeledPoint, RFormula} @@ -28,7 +28,6 @@ import org.apache.spark.ml.param.{ParamMap, ParamsSuite} import org.apache.spark.ml.util.{DefaultReadWriteTest, MLTest, MLTestingUtils} import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.random._ -import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.functions._ @@ -232,8 +231,8 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest print(as.vector(coef(model))) } - [1] 2.2960999 0.8087933 - [1] 2.5002642 2.2000403 0.5999485 + [1] 2.2958751 0.8088523 + [1] 2.5009266 2.1997901 0.5999522 data <- read.csv("path", header=FALSE) model1 <- glm(f1, family=gaussian(link=log), data=data, start=c(0,0)) @@ -241,8 +240,8 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest print(as.vector(coef(model1))) print(as.vector(coef(model2))) - [1] 0.23069326 0.07993778 - [1] 0.25001858 0.22002452 0.05998789 + [1] 0.23063118 0.07995495 + [1] 0.25016124 0.21995737 0.05999335 data <- read.csv("path", header=FALSE) for (formula in c(f1, f2)) { @@ -250,17 +249,17 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest print(as.vector(coef(model))) } - [1] 2.3010179 0.8198976 - [1] 2.4108902 2.2130248 0.6086152 + [1] 2.3320341 0.8121904 + [1] 2.2837064 2.2487147 0.6120262 */ val expected = Seq( - Vectors.dense(0.0, 2.2960999, 0.8087933), - Vectors.dense(2.5002642, 2.2000403, 0.5999485), - Vectors.dense(0.0, 0.23069326, 0.07993778), - Vectors.dense(0.25001858, 0.22002452, 0.05998789), - Vectors.dense(0.0, 2.3010179, 0.8198976), - Vectors.dense(2.4108902, 2.2130248, 0.6086152)) + Vectors.dense(0.0, 2.2958751, 0.8088523), + Vectors.dense(2.5009266, 2.1997901, 0.5999522), + Vectors.dense(0.0, 0.23063118, 0.07995495), + Vectors.dense(0.25016124, 0.21995737, 0.05999335), + Vectors.dense(0.0, 2.3320341, 0.8121904), + Vectors.dense(2.2837064, 2.2487147, 0.6120262)) import GeneralizedLinearRegression._ @@ -269,7 +268,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest ("inverse", datasetGaussianInverse))) { for (fitIntercept <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily("gaussian").setLink(link) - .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction").setTol(1e-3) val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gaussian family, " + @@ -308,27 +307,27 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest } } - [1] 0.0000000 2.2961005 0.8087932 - [1] 0.0000000 2.2130368 0.8309556 - [1] 0.0000000 1.7176137 0.9610657 - [1] 2.5002642 2.2000403 0.5999485 - [1] 3.1106389 2.0935142 0.5712711 - [1] 6.7597127 1.4581054 0.3994266 + [1] 0.0000000 2.2958757 0.8088521 + [1] 0.0000000 2.2128149 0.8310136 + [1] 0.0000000 1.7174260 0.9611137 + [1] 2.5009266 2.1997901 0.5999522 + [1] 3.1113269 2.0932659 0.5712717 + [1] 6.7604302 1.4578902 0.3994153 */ val expected = Seq( - Vectors.dense(0.0, 2.2961005, 0.8087932), - Vectors.dense(0.0, 2.2130368, 0.8309556), - Vectors.dense(0.0, 1.7176137, 0.9610657), - Vectors.dense(2.5002642, 2.2000403, 0.5999485), - Vectors.dense(3.1106389, 2.0935142, 0.5712711), - Vectors.dense(6.7597127, 1.4581054, 0.3994266)) + Vectors.dense(0.0, 2.2958757, 0.8088521), + Vectors.dense(0.0, 2.2128149, 0.8310136), + Vectors.dense(0.0, 1.7174260, 0.9611137), + Vectors.dense(2.5009266, 2.1997901, 0.5999522), + Vectors.dense(3.1113269, 2.0932659, 0.5712717), + Vectors.dense(6.7604302, 1.4578902, 0.3994153)) var idx = 0 for (fitIntercept <- Seq(false, true); regParam <- Seq(0.0, 0.1, 1.0)) { val trainer = new GeneralizedLinearRegression().setFamily("gaussian") - .setFitIntercept(fitIntercept).setRegParam(regParam) + .setFitIntercept(fitIntercept).setRegParam(regParam).setTol(1e-3) val model = trainer.fit(datasetGaussianIdentity) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gaussian family, " + @@ -384,7 +383,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest ("cloglog", datasetBinomial))) { for (fitIntercept <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily("binomial").setLink(link) - .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction").setTol(1e-3) val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1), model.coefficients(2), model.coefficients(3)) @@ -457,7 +456,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest ("sqrt", datasetPoissonSqrt))) { for (fitIntercept <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily("poisson").setLink(link) - .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction").setTol(1e-3) val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with poisson family, " + @@ -515,7 +514,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest val dataset = datasetPoissonLogWithZero for (fitIntercept <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily("poisson").setLink(link) - .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction").setTol(1e-3) val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with poisson family, " + @@ -573,7 +572,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest ("identity", datasetGammaIdentity), ("log", datasetGammaLog))) { for (fitIntercept <- Seq(false, true)) { val trainer = new GeneralizedLinearRegression().setFamily("Gamma").setLink(link) - .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") + .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction").setTol(1e-3) val model = trainer.fit(dataset) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with gamma family, " + @@ -659,7 +658,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest variancePower <- Seq(1.6, 2.5)) { val trainer = new GeneralizedLinearRegression().setFamily("tweedie") .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") - .setVariancePower(variancePower).setLinkPower(linkPower) + .setVariancePower(variancePower).setLinkPower(linkPower).setTol(1e-4) val model = trainer.fit(datasetTweedie) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + @@ -736,7 +735,7 @@ class GeneralizedLinearRegressionSuite extends MLTest with DefaultReadWriteTest for (variancePower <- Seq(0.0, 1.0, 2.0, 1.5)) { val trainer = new GeneralizedLinearRegression().setFamily("tweedie") .setFitIntercept(fitIntercept).setLinkPredictionCol("linkPrediction") - .setVariancePower(variancePower) + .setVariancePower(variancePower).setTol(1e-3) val model = trainer.fit(datasetTweedie) val actual = Vectors.dense(model.intercept, model.coefficients(0), model.coefficients(1)) assert(actual ~= expected(idx) absTol 1e-4, "Model mismatch: GLM with tweedie family, " + diff --git a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala index 90ceb7dee38f7..82d984933d815 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/regression/LinearRegressionSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.ml.regression import scala.collection.JavaConverters._ -import scala.collection.mutable import scala.util.Random -import org.dmg.pmml.{OpType, PMML, RegressionModel => PMMLRegressionModel} +import org.dmg.pmml.{OpType, PMML} +import org.dmg.pmml.regression.{RegressionModel => PMMLRegressionModel} import org.apache.spark.ml.feature.Instance import org.apache.spark.ml.feature.LabeledPoint @@ -187,6 +187,18 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe assert(model.numFeatures === numFeatures) } + test("linear regression: can transform data with LinearRegressionModel") { + withClue("training related params like loss are only validated during fitting phase") { + val original = new LinearRegression().fit(datasetWithDenseFeature) + + val deserialized = new LinearRegressionModel(uid = original.uid, + coefficients = original.coefficients, + intercept = original.intercept) + val output = deserialized.transform(datasetWithDenseFeature) + assert(output.collect().size > 0) // simple assertion to ensure no exception thrown + } + } + test("linear regression: illegal params") { withClue("LinearRegression with huber loss only supports L2 regularization") { intercept[IllegalArgumentException] { @@ -891,6 +903,8 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .setStandardization(standardization) .setRegParam(regParam) .setElasticNetParam(elasticNetParam) + .setSolver(solver) + .setMaxIter(1) MLTestingUtils.testArbitrarilyScaledWeights[LinearRegressionModel, LinearRegression]( datasetWithStrongNoise.as[LabeledPoint], estimator, modelEquals) MLTestingUtils.testOutliersWithSmallWeights[LinearRegressionModel, LinearRegression]( @@ -907,6 +921,7 @@ class LinearRegressionSuite extends MLTest with DefaultReadWriteTest with PMMLRe .setFitIntercept(fitIntercept) .setStandardization(standardization) .setRegParam(regParam) + .setMaxIter(1) MLTestingUtils.testArbitrarilyScaledWeights[LinearRegressionModel, LinearRegression]( datasetWithOutlier.as[LabeledPoint], estimator, modelEquals) MLTestingUtils.testOutliersWithSmallWeights[LinearRegressionModel, LinearRegression]( diff --git a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala index 38e25131df867..38bb246d02184 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/source/image/ImageFileFormatSuite.scala @@ -30,6 +30,7 @@ class ImageFileFormatSuite extends SparkFunSuite with MLlibTestSparkContext { // Single column of images named "image" private lazy val imagePath = "../data/mllib/images/partitioned" + private lazy val recursiveImagePath = "../data/mllib/images" test("image datasource count test") { val df1 = spark.read.format("image").load(imagePath) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala index 77ab3d8bb75f7..63985482795bb 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/BaggedPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.SparkFunSuite +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.mllib.tree.EnsembleTestHelper import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -26,12 +27,16 @@ import org.apache.spark.mllib.util.MLlibTestSparkContext */ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { - test("BaggedPoint RDD: without subsampling") { - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + test("BaggedPoint RDD: without subsampling with weights") { + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000).map { lp => + Instance(lp.label, 0.5, lp.features.asML) + } val rdd = sc.parallelize(arr) - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false, 42) + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, 1, false, + (instance: Instance) => instance.weight * 4.0, seed = 42) baggedRDD.collect().foreach { baggedPoint => - assert(baggedPoint.subsampleWeights.size == 1 && baggedPoint.subsampleWeights(0) == 1) + assert(baggedPoint.subsampleCounts.size === 1 && baggedPoint.subsampleCounts(0) === 1) + assert(baggedPoint.sampleWeight === 2.0) } } @@ -40,13 +45,17 @@ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { val (expectedMean, expectedStddev) = (1.0, 1.0) val seeds = Array(123, 5354, 230, 349867, 23987) - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000).map(_.asML) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true, seed) - val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, true, + (_: LabeledPoint) => 2.0, seed) + val subsampleCounts: Array[Array[Double]] = + baggedRDD.map(_.subsampleCounts.map(_.toDouble)).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) + // should ignore weight function for now + assert(baggedRDD.collect().forall(_.sampleWeight === 1.0)) } } @@ -59,8 +68,10 @@ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true, seed) - val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + val baggedRDD = + BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, true, seed = seed) + val subsampleCounts: Array[Array[Double]] = + baggedRDD.map(_.subsampleCounts.map(_.toDouble)).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) } @@ -71,13 +82,17 @@ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { val (expectedMean, expectedStddev) = (1.0, 0) val seeds = Array(123, 5354, 230, 349867, 23987) - val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000).map(_.asML) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false, seed) - val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, 1.0, numSubsamples, false, + (_: LabeledPoint) => 2.0, seed) + val subsampleCounts: Array[Array[Double]] = + baggedRDD.map(_.subsampleCounts.map(_.toDouble)).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) + // should ignore weight function for now + assert(baggedRDD.collect().forall(_.sampleWeight === 1.0)) } } @@ -90,8 +105,10 @@ class BaggedPointSuite extends SparkFunSuite with MLlibTestSparkContext { val arr = EnsembleTestHelper.generateOrderedLabeledPoints(1, 1000) val rdd = sc.parallelize(arr) seeds.foreach { seed => - val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false, seed) - val subsampleCounts: Array[Array[Double]] = baggedRDD.map(_.subsampleWeights).collect() + val baggedRDD = BaggedPoint.convertToBaggedRDD(rdd, subsample, numSubsamples, false, + seed = seed) + val subsampleCounts: Array[Array[Double]] = + baggedRDD.map(_.subsampleCounts.map(_.toDouble)).collect() EnsembleTestHelper.testRandomArrays(subsampleCounts, numSubsamples, expectedMean, expectedStddev, epsilon = 0.01) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala index 743dacf146fe7..a63ab913f2c22 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.ml.tree.impl import scala.annotation.tailrec import scala.collection.mutable +import scala.language.implicitConversions import org.apache.spark.SparkFunSuite import org.apache.spark.ml.classification.DecisionTreeClassificationModel -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, Vectors} import org.apache.spark.ml.tree._ import org.apache.spark.ml.util.TestingUtils._ @@ -39,14 +40,12 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { import RandomForestSuite.mapToVec - private val seed = 42 - ///////////////////////////////////////////////////////////////////////////// // Tests for split calculation ///////////////////////////////////////////////////////////////////////////// test("Binary classification with continuous features: split calculation") { - val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML) + val arr = OldDTSuite.generateOrderedLabeledPointsWithLabel1().map(_.asML.toInstance) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, 100) @@ -58,7 +57,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Binary classification with binary (ordered) categorical features: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML.toInstance) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -75,7 +74,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { test("Binary classification with 3-ary (ordered) categorical features," + " with no samples for one category: split calculation") { - val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML.toInstance) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 2, @@ -93,12 +92,12 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { test("find splits for a continuous feature") { // find splits for normal case { - val fakeMetadata = new DecisionTreeMetadata(1, 200000, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 200000, 200000.0, 0, 0, Map(), Set(), Array(6), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) - val featureSamples = Array.fill(10000)(math.random).filter(_ != 0.0) + val featureSamples = Array.fill(10000)((1.0, math.random)).filter(_._2 != 0.0) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) assert(splits.length === 5) assert(fakeMetadata.numSplits(0) === 5) @@ -109,15 +108,16 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // SPARK-16957: Use midpoints for split values. { - val fakeMetadata = new DecisionTreeMetadata(1, 8, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 8, 8.0, 0, 0, Map(), Set(), Array(3), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) // possibleSplits <= numSplits { - val featureSamples = Array(0, 1, 0, 0, 1, 0, 1, 1).map(_.toDouble).filter(_ != 0.0) + val featureSamples = Array(0, 1, 0, 0, 1, 0, 1, 1) + .map(x => (1.0, x.toDouble)).filter(_._2 != 0.0) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) val expectedSplits = Array((0.0 + 1.0) / 2) assert(splits === expectedSplits) @@ -125,7 +125,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // possibleSplits > numSplits { - val featureSamples = Array(0, 0, 1, 1, 2, 2, 3, 3).map(_.toDouble).filter(_ != 0.0) + val featureSamples = Array(0, 0, 1, 1, 2, 2, 3, 3) + .map(x => (1.0, x.toDouble)).filter(_._2 != 0.0) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) val expectedSplits = Array((0.0 + 1.0) / 2, (2.0 + 3.0) / 2) assert(splits === expectedSplits) @@ -135,12 +136,12 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // find splits should not return identical splits // when there are not enough split candidates, reduce the number of splits in metadata { - val fakeMetadata = new DecisionTreeMetadata(1, 12, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 12, 12.0, 0, 0, Map(), Set(), Array(5), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) - val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3, 3).map(_.toDouble) + val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 3, 3).map(x => (1.0, x.toDouble)) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) val expectedSplits = Array((1.0 + 2.0) / 2, (2.0 + 3.0) / 2) assert(splits === expectedSplits) @@ -150,13 +151,13 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // find splits when most samples close to the minimum { - val fakeMetadata = new DecisionTreeMetadata(1, 18, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 18, 18.0, 0, 0, Map(), Set(), Array(3), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) - val featureSamples = Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5) - .map(_.toDouble) + val featureSamples = + Array(2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 3, 4, 5).map(x => (1.0, x.toDouble)) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) val expectedSplits = Array((2.0 + 3.0) / 2, (3.0 + 4.0) / 2) assert(splits === expectedSplits) @@ -164,37 +165,55 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // find splits when most samples close to the maximum { - val fakeMetadata = new DecisionTreeMetadata(1, 17, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 17, 17.0, 0, 0, Map(), Set(), Array(2), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) - val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2) - .map(_.toDouble).filter(_ != 0.0) + val featureSamples = + Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2).map(x => (1.0, x.toDouble)) val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) val expectedSplits = Array((1.0 + 2.0) / 2) assert(splits === expectedSplits) } - // find splits for constant feature + // find splits for arbitrarily scaled data { - val fakeMetadata = new DecisionTreeMetadata(1, 3, 0, 0, + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0.0, 0, 0, + Map(), Set(), + Array(6), Gini, QuantileStrategy.Sort, + 0, 0, 0.0, 0.0, 0, 0 + ) + val featureSamplesUnitWeight = Array.fill(10)((1.0, math.random)) + val featureSamplesSmallWeight = featureSamplesUnitWeight.map { case (w, x) => (w * 0.001, x)} + val featureSamplesLargeWeight = featureSamplesUnitWeight.map { case (w, x) => (w * 1000, x)} + val splitsUnitWeight = RandomForest + .findSplitsForContinuousFeature(featureSamplesUnitWeight, fakeMetadata, 0) + val splitsSmallWeight = RandomForest + .findSplitsForContinuousFeature(featureSamplesSmallWeight, fakeMetadata, 0) + val splitsLargeWeight = RandomForest + .findSplitsForContinuousFeature(featureSamplesLargeWeight, fakeMetadata, 0) + assert(splitsUnitWeight === splitsSmallWeight) + assert(splitsUnitWeight === splitsLargeWeight) + } + + // find splits when most weight is close to the minimum + { + val fakeMetadata = new DecisionTreeMetadata(1, 0, 0.0, 0, 0, Map(), Set(), Array(3), Gini, QuantileStrategy.Sort, - 0, 0, 0.0, 0, 0 + 0, 0, 0.0, 0.0, 0, 0 ) - val featureSamples = Array(0, 0, 0).map(_.toDouble).filter(_ != 0.0) - val featureSamplesEmpty = Array.empty[Double] + val featureSamples = Array((10, 1), (1, 2), (1, 3), (1, 4), (1, 5), (1, 6)).map { + case (w, x) => (w.toDouble, x.toDouble) + } val splits = RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0) - assert(splits === Array.empty[Double]) - val splitsEmpty = - RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, fakeMetadata, 0) - assert(splitsEmpty === Array.empty[Double]) + assert(splits === Array(1.5, 2.5, 3.5, 4.5, 5.5)) } } test("train with empty arrays") { - val lp = LabeledPoint(1.0, Vectors.dense(Array.empty[Double])) + val lp = LabeledPoint(1.0, Vectors.dense(Array.empty[Double])).toInstance val data = Array.fill(5)(lp) val rdd = sc.parallelize(data) @@ -209,8 +228,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("train with constant features") { - val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)) - val data = Array.fill(5)(lp) + val instance = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0)).toInstance + val data = Array.fill(5)(instance) val rdd = sc.parallelize(data) val strategy = new OldStrategy( OldAlgo.Classification, @@ -222,7 +241,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val Array(tree) = RandomForest.run(rdd, strategy, 1, "all", 42L, instr = None) assert(tree.rootNode.impurity === -1.0) assert(tree.depth === 0) - assert(tree.rootNode.prediction === lp.label) + assert(tree.rootNode.prediction === instance.label) // Test with no categorical features val strategy2 = new OldStrategy( @@ -233,11 +252,11 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val Array(tree2) = RandomForest.run(rdd, strategy2, 1, "all", 42L, instr = None) assert(tree2.rootNode.impurity === -1.0) assert(tree2.depth === 0) - assert(tree2.rootNode.prediction === lp.label) + assert(tree2.rootNode.prediction === instance.label) } test("Multiclass classification with unordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML) + val arr = OldDTSuite.generateCategoricalDataPoints().map(_.asML.toInstance) assert(arr.length === 1000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy( @@ -278,7 +297,8 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { } test("Multiclass classification with ordered categorical features: split calculations") { - val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures().map(_.asML) + val arr = OldDTSuite.generateCategoricalDataPointsForMulticlassForOrderedFeatures() + .map(_.asML.toInstance) assert(arr.length === 3000) val rdd = sc.parallelize(arr) val strategy = new OldStrategy(OldAlgo.Classification, Gini, maxDepth = 2, numClasses = 100, @@ -310,7 +330,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) - val input = sc.parallelize(arr) + val input = sc.parallelize(arr.map(_.toInstance)) val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 1, numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) @@ -328,7 +348,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val treeToNodeToIndexInfo = Map(0 -> Map( topNode.id -> new RandomForest.NodeIndexInfo(0, None) )) - val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + val nodeStack = new mutable.ListBuffer[(Int, LearningNode)] RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode), nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack) @@ -352,7 +372,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { LabeledPoint(1.0, Vectors.dense(0.0, 1.0, 1.0)), LabeledPoint(0.0, Vectors.dense(2.0, 0.0, 0.0)), LabeledPoint(1.0, Vectors.dense(0.0, 2.0, 1.0))) - val input = sc.parallelize(arr) + val input = sc.parallelize(arr.map(_.toInstance)) val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 5, numClasses = 2, categoricalFeaturesInfo = Map(0 -> 3)) @@ -370,7 +390,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val treeToNodeToIndexInfo = Map(0 -> Map( topNode.id -> new RandomForest.NodeIndexInfo(0, None) )) - val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + val nodeStack = new mutable.ListBuffer[(Int, LearningNode)] RandomForest.findBestSplits(baggedInput, metadata, Map(0 -> topNode), nodesForGroup, treeToNodeToIndexInfo, splits, nodeStack) @@ -404,7 +424,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { LabeledPoint(0.0, Vectors.dense(2.0)), LabeledPoint(0.0, Vectors.dense(2.0)), LabeledPoint(1.0, Vectors.dense(2.0))) - val input = sc.parallelize(arr) + val input = sc.parallelize(arr.map(_.toInstance)) // Must set maxBins s.t. the feature will be treated as an ordered categorical feature. val strategy = new OldStrategy(algo = OldAlgo.Classification, impurity = Gini, maxDepth = 1, @@ -417,14 +437,14 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { case n: InternalNode => n.split match { case s: CategoricalSplit => assert(s.leftCategories === Array(1.0)) - case _ => throw new AssertionError("model.rootNode.split was not a CategoricalSplit") + case _ => fail("model.rootNode.split was not a CategoricalSplit") } - case _ => throw new AssertionError("model.rootNode was not an InternalNode") + case _ => fail("model.rootNode was not an InternalNode") } } test("Second level node building with vs. without groups") { - val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML) + val arr = OldDTSuite.generateOrderedLabeledPoints().map(_.asML.toInstance) assert(arr.length === 1000) val rdd = sc.parallelize(arr) // For tree with 1 group @@ -444,7 +464,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(n.leftChild.isInstanceOf[InternalNode]) assert(n.rightChild.isInstanceOf[InternalNode]) Array(n.leftChild.asInstanceOf[InternalNode], n.rightChild.asInstanceOf[InternalNode]) - case _ => throw new AssertionError("rootNode was not an InternalNode") + case _ => fail("rootNode was not an InternalNode") } // Single group second level tree construction. @@ -468,7 +488,7 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { def binaryClassificationTestWithContinuousFeaturesAndSubsampledFeatures(strategy: OldStrategy) { val numFeatures = 50 val arr = EnsembleTestHelper.generateOrderedLabeledPoints(numFeatures, 1000) - val rdd = sc.parallelize(arr).map(_.asML) + val rdd = sc.parallelize(arr).map(_.asML.toInstance) // Select feature subset for top nodes. Return true if OK. def checkFeatureSubsetStrategy( @@ -483,11 +503,11 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { val failString = s"Failed on test with:" + s"numTrees=$numTrees, featureSubsetStrategy=$featureSubsetStrategy," + s" numFeaturesPerNode=$numFeaturesPerNode, seed=$seed" - val nodeStack = new mutable.ArrayStack[(Int, LearningNode)] + val nodeStack = new mutable.ListBuffer[(Int, LearningNode)] val topNodes: Array[LearningNode] = new Array[LearningNode](numTrees) Range(0, numTrees).foreach { treeIndex => topNodes(treeIndex) = LearningNode.emptyNode(nodeIndex = 1) - nodeStack.push((treeIndex, topNodes(treeIndex))) + nodeStack.prepend((treeIndex, topNodes(treeIndex))) } val rng = new scala.util.Random(seed = seed) val (nodesForGroup: Map[Int, Array[LearningNode]], @@ -581,16 +601,16 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { left2 parent left right */ - val leftImp = new GiniCalculator(Array(3.0, 2.0, 1.0)) + val leftImp = new GiniCalculator(Array(3.0, 2.0, 1.0), 6L) val left = new LeafNode(0.0, leftImp.calculate(), leftImp) - val rightImp = new GiniCalculator(Array(1.0, 2.0, 5.0)) + val rightImp = new GiniCalculator(Array(1.0, 2.0, 5.0), 8L) val right = new LeafNode(2.0, rightImp.calculate(), rightImp) val parent = TreeTests.buildParentNode(left, right, new ContinuousSplit(0, 0.5)) val parentImp = parent.impurityStats - val left2Imp = new GiniCalculator(Array(1.0, 6.0, 1.0)) + val left2Imp = new GiniCalculator(Array(1.0, 6.0, 1.0), 8L) val left2 = new LeafNode(0.0, left2Imp.calculate(), left2Imp) val grandParent = TreeTests.buildParentNode(left2, parent, new ContinuousSplit(1, 1.0)) @@ -647,12 +667,12 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // feature_0 = 0 improves the impurity measure, despite the prediction will always be 0 // in both branches. val arr = Array( - LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), - LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(1.0, 0.0)), - LabeledPoint(0.0, Vectors.dense(1.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(1.0, 1.0)) + Instance(0.0, 1.0, Vectors.dense(0.0, 1.0)), + Instance(1.0, 1.0, Vectors.dense(0.0, 1.0)), + Instance(0.0, 1.0, Vectors.dense(0.0, 0.0)), + Instance(1.0, 1.0, Vectors.dense(1.0, 0.0)), + Instance(0.0, 1.0, Vectors.dense(1.0, 0.0)), + Instance(1.0, 1.0, Vectors.dense(1.0, 1.0)) ) val rdd = sc.parallelize(arr) @@ -677,13 +697,13 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { // feature_1 = 1 improves the impurity measure, despite the prediction will always be 0.5 // in both branches. val arr = Array( - LabeledPoint(0.0, Vectors.dense(0.0, 1.0)), - LabeledPoint(1.0, Vectors.dense(0.0, 1.0)), - LabeledPoint(0.0, Vectors.dense(0.0, 0.0)), - LabeledPoint(0.0, Vectors.dense(1.0, 0.0)), - LabeledPoint(1.0, Vectors.dense(1.0, 1.0)), - LabeledPoint(0.0, Vectors.dense(1.0, 1.0)), - LabeledPoint(0.5, Vectors.dense(1.0, 1.0)) + Instance(0.0, 1.0, Vectors.dense(0.0, 1.0)), + Instance(1.0, 1.0, Vectors.dense(0.0, 1.0)), + Instance(0.0, 1.0, Vectors.dense(0.0, 0.0)), + Instance(0.0, 1.0, Vectors.dense(1.0, 0.0)), + Instance(1.0, 1.0, Vectors.dense(1.0, 1.0)), + Instance(0.0, 1.0, Vectors.dense(1.0, 1.0)), + Instance(0.5, 1.0, Vectors.dense(1.0, 1.0)) ) val rdd = sc.parallelize(arr) @@ -700,6 +720,56 @@ class RandomForestSuite extends SparkFunSuite with MLlibTestSparkContext { assert(unprunedTree.numNodes === 5) assert(RandomForestSuite.getSumLeafCounters(List(prunedTree.rootNode)) === arr.size) } + + test("weights at arbitrary scale") { + val arr = EnsembleTestHelper.generateOrderedLabeledPoints(3, 10) + val rddWithUnitWeights = sc.parallelize(arr.map(_.asML.toInstance)) + val rddWithSmallWeights = rddWithUnitWeights.map { inst => + Instance(inst.label, 0.001, inst.features) + } + val rddWithBigWeights = rddWithUnitWeights.map { inst => + Instance(inst.label, 1000, inst.features) + } + val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2) + val unitWeightTrees = RandomForest.run(rddWithUnitWeights, strategy, 3, "all", 42L, None) + + val smallWeightTrees = RandomForest.run(rddWithSmallWeights, strategy, 3, "all", 42L, None) + unitWeightTrees.zip(smallWeightTrees).foreach { case (unitTree, smallWeightTree) => + TreeTests.checkEqual(unitTree, smallWeightTree) + } + + val bigWeightTrees = RandomForest.run(rddWithBigWeights, strategy, 3, "all", 42L, None) + unitWeightTrees.zip(bigWeightTrees).foreach { case (unitTree, bigWeightTree) => + TreeTests.checkEqual(unitTree, bigWeightTree) + } + } + + test("minWeightFraction and minInstancesPerNode") { + val data = Array( + Instance(0.0, 1.0, Vectors.dense(0.0)), + Instance(0.0, 1.0, Vectors.dense(0.0)), + Instance(0.0, 1.0, Vectors.dense(0.0)), + Instance(0.0, 1.0, Vectors.dense(0.0)), + Instance(1.0, 0.1, Vectors.dense(1.0)) + ) + val rdd = sc.parallelize(data) + val strategy = new OldStrategy(OldAlgo.Classification, Gini, 3, 2, + minWeightFractionPerNode = 0.5) + val Array(tree1) = RandomForest.run(rdd, strategy, 1, "all", 42L, None) + assert(tree1.depth === 0) + + strategy.minWeightFractionPerNode = 0.0 + val Array(tree2) = RandomForest.run(rdd, strategy, 1, "all", 42L, None) + assert(tree2.depth === 1) + + strategy.minInstancesPerNode = 2 + val Array(tree3) = RandomForest.run(rdd, strategy, 1, "all", 42L, None) + assert(tree3.depth === 0) + + strategy.minInstancesPerNode = 1 + val Array(tree4) = RandomForest.run(rdd, strategy, 1, "all", 42L, None) + assert(tree4.depth === 1) + } } private object RandomForestSuite { @@ -717,7 +787,7 @@ private object RandomForestSuite { else { nodes.head match { case i: InternalNode => getSumLeafCounters(i.leftChild :: i.rightChild :: nodes.tail, acc) - case l: LeafNode => getSumLeafCounters(nodes.tail, acc + l.impurityStats.count) + case l: LeafNode => getSumLeafCounters(nodes.tail, acc + l.impurityStats.rawCount) } } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala index f41abe48f2c58..f6fe25bc9fe8b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreePointSuite.scala @@ -18,16 +18,17 @@ package org.apache.spark.ml.tree.impl import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoSerializer class TreePointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() - val point = new TreePoint(1.0, Array(1, 2, 3)) + val point = new TreePoint(1.0, Array(1, 2, 3), 1.0) val point2 = ser.deserialize[TreePoint](ser.serialize(point)) assert(point.label === point2.label) assert(point.binnedFeatures === point2.binnedFeatures) diff --git a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala index b6894b30b0c2b..f3096e28d3d4c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tree/impl/TreeTests.scala @@ -18,13 +18,15 @@ package org.apache.spark.ml.tree.impl import scala.collection.JavaConverters._ +import scala.util.Random import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.api.java.JavaRDD import org.apache.spark.ml.attribute.{AttributeGroup, NominalAttribute, NumericAttribute} -import org.apache.spark.ml.feature.LabeledPoint +import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.Vectors import org.apache.spark.ml.tree._ +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SparkSession} @@ -32,6 +34,7 @@ private[ml] object TreeTests extends SparkFunSuite { /** * Convert the given data to a DataFrame, and set the features and label metadata. + * * @param data Dataset. Categorical features and labels must already have 0-based indices. * This must be non-empty. * @param categoricalFeatures Map: categorical feature index to number of distinct values @@ -39,16 +42,22 @@ private[ml] object TreeTests extends SparkFunSuite { * @return DataFrame with metadata */ def setMetadata( - data: RDD[LabeledPoint], + data: RDD[_], categoricalFeatures: Map[Int, Int], numClasses: Int): DataFrame = { + val dataOfInstance: RDD[Instance] = data.map { + _ match { + case instance: Instance => instance + case labeledPoint: LabeledPoint => labeledPoint.toInstance + } + } val spark = SparkSession.builder() .sparkContext(data.sparkContext) .getOrCreate() import spark.implicits._ - val df = data.toDF() - val numFeatures = data.first().features.size + val df = dataOfInstance.toDF() + val numFeatures = dataOfInstance.first().features.size val featuresAttributes = Range(0, numFeatures).map { feature => if (categoricalFeatures.contains(feature)) { NominalAttribute.defaultAttr.withIndex(feature).withNumValues(categoricalFeatures(feature)) @@ -64,7 +73,7 @@ private[ml] object TreeTests extends SparkFunSuite { } val labelMetadata = labelAttribute.toMetadata() df.select(df("features").as("features", featuresMetadata), - df("label").as("label", labelMetadata)) + df("label").as("label", labelMetadata), df("weight")) } /** @@ -80,6 +89,7 @@ private[ml] object TreeTests extends SparkFunSuite { /** * Set label metadata (particularly the number of classes) on a DataFrame. + * * @param data Dataset. Categorical features and labels must already have 0-based indices. * This must be non-empty. * @param numClasses Number of classes label can take. If 0, mark as continuous. @@ -112,7 +122,7 @@ private[ml] object TreeTests extends SparkFunSuite { checkEqual(a.rootNode, b.rootNode) } catch { case ex: Exception => - throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + fail("checkEqual failed since the two trees were not identical.\n" + "TREE A:\n" + a.toDebugString + "\n" + "TREE B:\n" + b.toDebugString + "\n", ex) } @@ -124,8 +134,8 @@ private[ml] object TreeTests extends SparkFunSuite { * make mistakes such as creating loops of Nodes. */ private def checkEqual(a: Node, b: Node): Unit = { - assert(a.prediction === b.prediction) - assert(a.impurity === b.impurity) + assert(a.prediction ~== b.prediction absTol 1e-8) + assert(a.impurity ~== b.impurity absTol 1e-8) (a, b) match { case (aye: InternalNode, bee: InternalNode) => assert(aye.split === bee.split) @@ -133,7 +143,7 @@ private[ml] object TreeTests extends SparkFunSuite { checkEqual(aye.rightChild, bee.rightChild) case (aye: LeafNode, bee: LeafNode) => // do nothing case _ => - throw new AssertionError("Found mismatched nodes") + fail("Found mismatched nodes") } } @@ -148,7 +158,7 @@ private[ml] object TreeTests extends SparkFunSuite { } assert(a.treeWeights === b.treeWeights) } catch { - case ex: Exception => throw new AssertionError( + case ex: Exception => fail( "checkEqual failed since the two tree ensembles were not identical") } } @@ -156,6 +166,7 @@ private[ml] object TreeTests extends SparkFunSuite { /** * Helper method for constructing a tree for testing. * Given left, right children, construct a parent node. + * * @param split Split for parent node * @return Parent node with children attached */ @@ -163,8 +174,8 @@ private[ml] object TreeTests extends SparkFunSuite { val leftImp = left.impurityStats val rightImp = right.impurityStats val parentImp = leftImp.copy.add(rightImp) - val leftWeight = leftImp.count / parentImp.count.toDouble - val rightWeight = rightImp.count / parentImp.count.toDouble + val leftWeight = leftImp.count / parentImp.count + val rightWeight = rightImp.count / parentImp.count val gain = parentImp.calculate() - (leftWeight * leftImp.calculate() + rightWeight * rightImp.calculate()) val pred = parentImp.predict diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index e6ee7220d2279..a30428ec2d283 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -190,8 +190,8 @@ class CrossValidatorSuite assert(lr.uid === lr2.uid) assert(lr.getMaxIter === lr2.getMaxIter) case other => - throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected estimator of type LogisticRegression" + + s" but found ${other.getClass.getName}") } ValidatorParamsSuiteHelpers @@ -281,13 +281,13 @@ class CrossValidatorSuite assert(ova.getClassifier.asInstanceOf[LogisticRegression].getMaxIter === lr.getMaxIter) case other => - throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected estimator of type LogisticRegression" + + s" but found ${other.getClass.getName}") } case other => - throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + - s" OneVsRest but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected estimator of type OneVsRest but " + + s"found ${other.getClass.getName}") } ValidatorParamsSuiteHelpers @@ -364,8 +364,8 @@ class CrossValidatorSuite assert(lr.uid === lr2.uid) assert(lr.getMaxIter === lr2.getMaxIter) case other => - throw new AssertionError(s"Loaded internal CrossValidator expected to be" + - s" LogisticRegression but found type ${other.getClass.getName}") + fail("Loaded internal CrossValidator expected to be LogisticRegression" + + s" but found type ${other.getClass.getName}") } assert(lrcv.uid === lrcv2.uid) assert(lrcv2.getEvaluator.isInstanceOf[BinaryClassificationEvaluator]) @@ -373,12 +373,12 @@ class CrossValidatorSuite ValidatorParamsSuiteHelpers .compareParamMaps(lrParamMaps, lrcv2.getEstimatorParamMaps) case other => - throw new AssertionError("Loaded Pipeline expected stages (HashingTF, CrossValidator)" + - " but found: " + other.map(_.getClass.getName).mkString(", ")) + fail("Loaded Pipeline expected stages (HashingTF, CrossValidator) but found: " + + other.map(_.getClass.getName).mkString(", ")) } case other => - throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + - s" CrossValidator but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected estimator of type CrossValidator but found" + + s" ${other.getClass.getName}") } } @@ -433,8 +433,8 @@ class CrossValidatorSuite assert(lr.uid === lr2.uid) assert(lr.getThreshold === lr2.getThreshold) case other => - throw new AssertionError(s"Loaded CrossValidator expected estimator of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected estimator of type LogisticRegression" + + s" but found ${other.getClass.getName}") } ValidatorParamsSuiteHelpers @@ -447,8 +447,8 @@ class CrossValidatorSuite assert(lrModel.coefficients === lrModel2.coefficients) assert(lrModel.intercept === lrModel2.intercept) case other => - throw new AssertionError(s"Loaded CrossValidator expected bestModel of type" + - s" LogisticRegressionModel but found ${other.getClass.getName}") + fail("Loaded CrossValidator expected bestModel of type LogisticRegressionModel" + + s" but found ${other.getClass.getName}") } assert(cv.avgMetrics === cv2.avgMetrics) } diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala index cd76acf9c67bc..289db336eca5d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/TrainValidationSplitSuite.scala @@ -187,8 +187,8 @@ class TrainValidationSplitSuite assert(lr.uid === lr2.uid) assert(lr.getMaxIter === lr2.getMaxIter) case other => - throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail("Loaded TrainValidationSplit expected estimator of type LogisticRegression" + + s" but found ${other.getClass.getName}") } } @@ -264,13 +264,13 @@ class TrainValidationSplitSuite assert(ova.getClassifier.asInstanceOf[LogisticRegression].getMaxIter === lr.getMaxIter) case other => - throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + - s" LogisticRegression but found ${other.getClass.getName}") + fail(s"Loaded TrainValidationSplit expected estimator of type LogisticRegression" + + s" but found ${other.getClass.getName}") } case other => - throw new AssertionError(s"Loaded TrainValidationSplit expected estimator of type" + - s" OneVsRest but found ${other.getClass.getName}") + fail(s"Loaded TrainValidationSplit expected estimator of type OneVsRest" + + s" but found ${other.getClass.getName}") } ValidatorParamsSuiteHelpers diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala index eae1f5adc8842..cea2f50d3470c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/ValidatorParamsSuiteHelpers.scala @@ -47,8 +47,7 @@ object ValidatorParamsSuiteHelpers extends Assertions { val estimatorParamMap2 = Array(estimator2.extractParamMap()) compareParamMaps(estimatorParamMap, estimatorParamMap2) case other => - throw new AssertionError(s"Expected parameter of type Params but" + - s" found ${otherParam.getClass.getName}") + fail(s"Expected parameter of type Params but found ${otherParam.getClass.getName}") } case _ => assert(otherParam === v) diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala index acac171346a85..8a0a48ff6095b 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTest.scala @@ -21,8 +21,9 @@ import java.io.File import org.scalatest.Suite -import org.apache.spark.{DebugFilesystem, SparkConf, SparkContext} -import org.apache.spark.ml.{PredictionModel, Transformer} +import org.apache.spark.{DebugFilesystem, SparkConf, SparkContext, TestUtils} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK +import org.apache.spark.ml.{Model, PredictionModel, Transformer} import org.apache.spark.ml.linalg.Vector import org.apache.spark.sql.{DataFrame, Dataset, Encoder, Row} import org.apache.spark.sql.execution.streaming.MemoryStream @@ -40,7 +41,7 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => protected override def sparkConf = { new SparkConf() .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) .set(SQLConf.CODEGEN_FALLBACK.key, "false") } @@ -128,21 +129,17 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => expectedMessagePart : String, firstResultCol: String) { - def hasExpectedMessage(exception: Throwable): Boolean = - exception.getMessage.contains(expectedMessagePart) || - (exception.getCause != null && exception.getCause.getMessage.contains(expectedMessagePart)) - withClue(s"""Expected message part "${expectedMessagePart}" is not found in DF test.""") { val exceptionOnDf = intercept[Throwable] { testTransformerOnDF(dataframe, transformer, firstResultCol)(_ => Unit) } - assert(hasExpectedMessage(exceptionOnDf)) + TestUtils.assertExceptionMsg(exceptionOnDf, expectedMessagePart) } withClue(s"""Expected message part "${expectedMessagePart}" is not found in stream test.""") { val exceptionOnStreamData = intercept[Throwable] { testTransformerOnStreamData(dataframe, transformer, firstResultCol)(_ => Unit) } - assert(hasExpectedMessage(exceptionOnStreamData)) + TestUtils.assertExceptionMsg(exceptionOnStreamData, expectedMessagePart) } } @@ -155,4 +152,30 @@ trait MLTest extends StreamTest with TempDirectory { self: Suite => assert(prediction === model.predict(features)) } } + + def testClusteringModelSinglePrediction( + model: Model[_], + transform: Vector => Int, + dataset: Dataset[_], + input: String, + output: String): Unit = { + model.transform(dataset).select(input, output) + .collect().foreach { + case Row(features: Vector, prediction: Int) => + assert(prediction === transform(features)) + } + } + + def testClusteringModelSingleProbabilisticPrediction( + model: Model[_], + transform: Vector => Vector, + dataset: Dataset[_], + input: String, + output: String): Unit = { + model.transform(dataset).select(input, output) + .collect().foreach { + case Row(features: Vector, prediction: Vector) => + assert(prediction === transform(features)) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala index 91a8b14625a86..ad4ac7259165d 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/MLTestingUtils.scala @@ -23,8 +23,7 @@ import org.apache.spark.ml.evaluation.Evaluator import org.apache.spark.ml.feature.{Instance, LabeledPoint} import org.apache.spark.ml.linalg.{Vector, Vectors, VectorUDT} import org.apache.spark.ml.param.ParamMap -import org.apache.spark.ml.param.shared.{HasFeaturesCol, HasLabelCol, HasWeightCol} -import org.apache.spark.ml.recommendation.{ALS, ALSModel} +import org.apache.spark.ml.param.shared.HasWeightCol import org.apache.spark.ml.tree.impl.TreeTests import org.apache.spark.sql.{DataFrame, Dataset, Row, SparkSession} import org.apache.spark.sql.functions._ @@ -205,8 +204,8 @@ object MLTestingUtils extends SparkFunSuite { seed: Long): Unit = { val (overSampledData, weightedData) = genEquivalentOversampledAndWeightedInstances( data, seed) - val weightedModel = estimator.set(estimator.weightCol, "weight").fit(weightedData) val overSampledModel = estimator.set(estimator.weightCol, "").fit(overSampledData) + val weightedModel = estimator.set(estimator.weightCol, "weight").fit(weightedData) modelEquals(weightedModel, overSampledModel) } @@ -228,7 +227,8 @@ object MLTestingUtils extends SparkFunSuite { List.fill(outlierRatio)(Instance(outlierLabel, 0.0001, f)) ++ List(Instance(l, w, f)) } val trueModel = estimator.set(estimator.weightCol, "").fit(data) - val outlierModel = estimator.set(estimator.weightCol, "weight").fit(outlierDS) + val outlierModel = estimator.set(estimator.weightCol, "weight") + .fit(outlierDS) modelEquals(trueModel, outlierModel) } @@ -241,7 +241,7 @@ object MLTestingUtils extends SparkFunSuite { estimator: E with HasWeightCol, modelEquals: (M, M) => Unit): Unit = { estimator.set(estimator.weightCol, "weight") - val models = Seq(0.001, 1.0, 1000.0).map { w => + val models = Seq(0.01, 1.0, 1000.0).map { w => val df = data.withColumn("weight", lit(w)) estimator.fit(df) } @@ -254,8 +254,8 @@ object MLTestingUtils extends SparkFunSuite { * one having double array "features" column with float precision, and one having float array * "features" column. */ - def generateArrayFeatureDataset(dataset: Dataset[_], - featuresColName: String = "features"): (Dataset[_], Dataset[_], Dataset[_]) = { + def generateArrayFeatureDataset(dataset: DataFrame, + featuresColName: String = "features"): (DataFrame, DataFrame, DataFrame) = { val toFloatVectorUDF = udf { (features: Vector) => Vectors.dense(features.toArray.map(_.toFloat.toDouble))} val toDoubleArrayUDF = udf { (features: Vector) => features.toArray} @@ -268,4 +268,20 @@ object MLTestingUtils extends SparkFunSuite { assert(newDatasetF.schema(featuresColName).dataType.equals(new ArrayType(FloatType, false))) (newDataset, newDatasetD, newDatasetF) } + + def modelPredictionEquals[M <: PredictionModel[_, M]]( + data: DataFrame, + compareFunc: (Double, Double) => Boolean, + fractionInTol: Double)( + model1: M, + model2: M): Unit = { + val pred1 = model1.transform(data).select(model1.getPredictionCol).collect() + val pred2 = model2.transform(data).select(model2.getPredictionCol).collect() + val inTol = pred1.zip(pred2).count { case (p1, p2) => + val x = p1.getDouble(0) + val y = p2.getDouble(0) + compareFunc(x, y) + } + assert(inTol / pred1.length.toDouble >= fractionInTol) + } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/PMMLUtils.scala b/mllib/src/test/scala/org/apache/spark/ml/util/PMMLUtils.scala index dbdc69f95d841..620c754a7ba0c 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/PMMLUtils.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/PMMLUtils.scala @@ -16,13 +16,12 @@ */ package org.apache.spark.ml.util -import java.io.StringReader -import javax.xml.bind.Unmarshaller -import javax.xml.transform.Source +import java.io.ByteArrayInputStream +import java.nio.charset.StandardCharsets -import org.dmg.pmml._ -import org.jpmml.model.{ImportFilter, JAXBUtil} -import org.xml.sax.InputSource +import org.dmg.pmml.PMML +import org.jpmml.model.{JAXBUtil, SAXUtil} +import org.jpmml.model.filters.ImportFilter /** * Testing utils for working with PMML. @@ -36,8 +35,9 @@ private[spark] object PMMLUtils { * through external spark-packages. */ def loadFromString(input: String): PMML = { - val is = new StringReader(input) - val transformed = ImportFilter.apply(new InputSource(is)) + val transformed = SAXUtil.createFilteredSource( + new ByteArrayInputStream(input.getBytes(StandardCharsets.UTF_8)), + new ImportFilter()) JAXBUtil.unmarshalPMML(transformed) } } diff --git a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala index 54e363a8b9f2b..de3726facaf39 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/util/StopwatchSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.ml.util import java.util.Random +import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext @@ -121,5 +122,5 @@ private object StopwatchSuite extends SparkFunSuite { } /** The current time in milliseconds. */ - private def now: Long = System.currentTimeMillis() + private def now: Long = TimeUnit.NANOSECONDS.toMillis(System.nanoTime()) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 5ec4c15387e94..8c7d583923b32 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -71,7 +71,7 @@ object NaiveBayesSuite { counts.toArray.sortBy(_._1).map(_._2) case _ => // This should never happen. - throw new UnknownError(s"Invalid modelType: $modelType.") + throw new IllegalArgumentException(s"Invalid modelType: $modelType.") } LabeledPoint(y, Vectors.dense(xi)) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala index 35f7932ae8224..10d5f325d68e9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/BisectingKMeansSuite.scala @@ -187,12 +187,14 @@ class BisectingKMeansSuite extends SparkFunSuite with MLlibTestSparkContext { val points = (1 until 8).map(i => Vectors.dense(i)) val data = sc.parallelize(points, 2) - val model = new BisectingKMeans().run(data) + val model = new BisectingKMeans().setDistanceMeasure(DistanceMeasure.COSINE).run(data) try { model.save(sc, path) val sameModel = BisectingKMeansModel.load(sc, path) assert(model.k === sameModel.k) + assert(model.distanceMeasure === sameModel.distanceMeasure) model.clusterCenters.zip(sameModel.clusterCenters).foreach(c => c._1 === c._2) + assert(model.trainingCost == sameModel.trainingCost) } finally { Utils.deleteRecursively(tempDir) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 1b98250061c7a..c4bf5b27187f6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.clustering import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{DenseVector, SparseVector, Vector, Vectors} import org.apache.spark.mllib.util.{LocalClusterSparkContext, MLlibTestSparkContext} import org.apache.spark.mllib.util.TestingUtils._ @@ -316,7 +317,7 @@ class KMeansSuite extends SparkFunSuite with MLlibTestSparkContext { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() @@ -349,7 +350,7 @@ object KMeansSuite extends SparkFunSuite { case (ca: DenseVector, cb: DenseVector) => assert(ca === cb) case _ => - throw new AssertionError("checkEqual failed since the two clusters were not identical.\n") + fail("checkEqual failed since the two clusters were not identical.\n") } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala index b33b86b39a42f..c25c89b5679a4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/PowerIterationClusteringSuite.scala @@ -47,9 +47,9 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon test("power iteration clustering") { // Generate two circles following the example in the PIC paper. val r1 = 1.0 - val n1 = 10 + val n1 = 80 val r2 = 4.0 - val n2 = 10 + val n2 = 80 val n = n1 + n2 val points = genCircle(r1, n1) ++ genCircle(r2, n2) val similarities = for (i <- 1 until n; j <- 0 until i) yield { @@ -81,9 +81,9 @@ class PowerIterationClusteringSuite extends SparkFunSuite with MLlibTestSparkCon test("power iteration clustering on graph") { // Generate two circles following the example in the PIC paper. val r1 = 1.0 - val n1 = 10 + val n1 = 80 val r2 = 4.0 - val n2 = 10 + val n2 = 80 val n = n1 + n2 val points = genCircle(r1, n1) ++ genCircle(r2, n2) val similarities = for (i <- 1 until n; j <- 0 until i) yield { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala index fdaa098345d13..a1ac10c06c697 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/StreamingKMeansSuite.scala @@ -77,6 +77,7 @@ class StreamingKMeansSuite extends SparkFunSuite with TestSuiteBase { val k = 2 val d = 5 val r = 0.1 + val seed = 987654321 // create model with two clusters val kMeans = new StreamingKMeans() @@ -88,7 +89,7 @@ class StreamingKMeansSuite extends SparkFunSuite with TestSuiteBase { Array(5.0, 5.0)) // generate random data for k-means - val (input, centers) = StreamingKMeansDataGenerator(numPoints, numBatches, k, d, r, 42) + val (input, centers) = StreamingKMeansDataGenerator(numPoints, numBatches, k, d, r, seed) // setup and run the model training ssc = setupStreams(input, (inputDStream: DStream[Vector]) => { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala index a08917ac1ebed..96c1f220f6731 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/BinaryClassificationMetricsSuite.scala @@ -82,6 +82,34 @@ class BinaryClassificationMetricsSuite extends SparkFunSuite with MLlibTestSpark validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls) } + test("binary evaluation metrics with weights") { + val w1 = 1.5 + val w2 = 0.7 + val w3 = 0.4 + val scoreAndLabelsWithWeights = sc.parallelize( + Seq((0.1, 0.0, w1), (0.1, 1.0, w2), (0.4, 0.0, w1), (0.6, 0.0, w3), + (0.6, 1.0, w2), (0.6, 1.0, w2), (0.8, 1.0, w1)), 2) + val metrics = new BinaryClassificationMetrics(scoreAndLabelsWithWeights, 0) + val thresholds = Seq(0.8, 0.6, 0.4, 0.1) + val numTruePositives = + Seq(1 * w1, 1 * w1 + 2 * w2, 1 * w1 + 2 * w2, 3 * w2 + 1 * w1) + val numFalsePositives = Seq(0.0, 1.0 * w3, 1.0 * w1 + 1.0 * w3, 1.0 * w3 + 2.0 * w1) + val numPositives = 3 * w2 + 1 * w1 + val numNegatives = 2 * w1 + w3 + val precisions = numTruePositives.zip(numFalsePositives).map { case (t, f) => + t.toDouble / (t + f) + } + val recalls = numTruePositives.map(_ / numPositives) + val fpr = numFalsePositives.map(_ / numNegatives) + val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recalls) ++ Seq((1.0, 1.0)) + val pr = recalls.zip(precisions) + val prCurve = Seq((0.0, 1.0)) ++ pr + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r)} + val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} + + validateMetrics(metrics, thresholds, rocCurve, prCurve, f1, f2, precisions, recalls) + } + test("binary evaluation metrics for RDD where all examples have positive label") { val scoreAndLabels = sc.parallelize(Seq((0.5, 1.0), (0.5, 1.0)), 2) val metrics = new BinaryClassificationMetrics(scoreAndLabels) @@ -155,6 +183,17 @@ class BinaryClassificationMetricsSuite extends SparkFunSuite with MLlibTestSpark (1.0, 1.0), (1.0, 1.0) ) == downsampledROC) + + val downsampledRecall = downsampled.recallByThreshold().collect().sorted.toList + assert( + // May have to add 1 if the sample factor didn't divide evenly + numBins + (if (scoreAndLabels.size % numBins == 0) 0 else 1) == + downsampledRecall.size) + assert( + List( + (0.1, 1.0), (0.2, 1.0), (0.4, 0.75), (0.6, 0.75), (0.8, 0.25) + ) == + downsampledRecall) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala index 142d1e9812ef1..8779de590a256 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MulticlassMetricsSuite.scala @@ -18,10 +18,14 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.Matrices +import org.apache.spark.ml.linalg.Matrices +import org.apache.spark.ml.util.TestingUtils._ import org.apache.spark.mllib.util.MLlibTestSparkContext class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { + + private val delta = 1e-7 + test("Multiclass evaluation metrics") { /* * Confusion matrix for 3-class classification with total 9 instances: @@ -35,7 +39,6 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { Seq((0.0, 0.0), (0.0, 1.0), (0.0, 0.0), (1.0, 0.0), (1.0, 1.0), (1.0, 1.0), (1.0, 1.0), (2.0, 2.0), (2.0, 0.0)), 2) val metrics = new MulticlassMetrics(predictionAndLabels) - val delta = 0.0000001 val tpRate0 = 2.0 / (2 + 2) val tpRate1 = 3.0 / (3 + 1) val tpRate2 = 1.0 / (1 + 0) @@ -55,44 +58,122 @@ class MulticlassMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val f2measure1 = (1 + 2 * 2) * precision1 * recall1 / (2 * 2 * precision1 + recall1) val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) - assert(metrics.confusionMatrix.toArray.sameElements(confusionMatrix.toArray)) - assert(math.abs(metrics.truePositiveRate(0.0) - tpRate0) < delta) - assert(math.abs(metrics.truePositiveRate(1.0) - tpRate1) < delta) - assert(math.abs(metrics.truePositiveRate(2.0) - tpRate2) < delta) - assert(math.abs(metrics.falsePositiveRate(0.0) - fpRate0) < delta) - assert(math.abs(metrics.falsePositiveRate(1.0) - fpRate1) < delta) - assert(math.abs(metrics.falsePositiveRate(2.0) - fpRate2) < delta) - assert(math.abs(metrics.precision(0.0) - precision0) < delta) - assert(math.abs(metrics.precision(1.0) - precision1) < delta) - assert(math.abs(metrics.precision(2.0) - precision2) < delta) - assert(math.abs(metrics.recall(0.0) - recall0) < delta) - assert(math.abs(metrics.recall(1.0) - recall1) < delta) - assert(math.abs(metrics.recall(2.0) - recall2) < delta) - assert(math.abs(metrics.fMeasure(0.0) - f1measure0) < delta) - assert(math.abs(metrics.fMeasure(1.0) - f1measure1) < delta) - assert(math.abs(metrics.fMeasure(2.0) - f1measure2) < delta) - assert(math.abs(metrics.fMeasure(0.0, 2.0) - f2measure0) < delta) - assert(math.abs(metrics.fMeasure(1.0, 2.0) - f2measure1) < delta) - assert(math.abs(metrics.fMeasure(2.0, 2.0) - f2measure2) < delta) + assert(metrics.confusionMatrix.asML ~== confusionMatrix relTol delta) + assert(metrics.truePositiveRate(0.0) ~== tpRate0 relTol delta) + assert(metrics.truePositiveRate(1.0) ~== tpRate1 relTol delta) + assert(metrics.truePositiveRate(2.0) ~== tpRate2 relTol delta) + assert(metrics.falsePositiveRate(0.0) ~== fpRate0 relTol delta) + assert(metrics.falsePositiveRate(1.0) ~== fpRate1 relTol delta) + assert(metrics.falsePositiveRate(2.0) ~== fpRate2 relTol delta) + assert(metrics.precision(0.0) ~== precision0 relTol delta) + assert(metrics.precision(1.0) ~== precision1 relTol delta) + assert(metrics.precision(2.0) ~== precision2 relTol delta) + assert(metrics.recall(0.0) ~== recall0 relTol delta) + assert(metrics.recall(1.0) ~== recall1 relTol delta) + assert(metrics.recall(2.0) ~== recall2 relTol delta) + assert(metrics.fMeasure(0.0) ~== f1measure0 relTol delta) + assert(metrics.fMeasure(1.0) ~== f1measure1 relTol delta) + assert(metrics.fMeasure(2.0) ~== f1measure2 relTol delta) + assert(metrics.fMeasure(0.0, 2.0) ~== f2measure0 relTol delta) + assert(metrics.fMeasure(1.0, 2.0) ~== f2measure1 relTol delta) + assert(metrics.fMeasure(2.0, 2.0) ~== f2measure2 relTol delta) + + assert(metrics.accuracy ~== + (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1)) relTol delta) + assert(metrics.accuracy ~== metrics.weightedRecall relTol delta) + val weight0 = 4.0 / 9 + val weight1 = 4.0 / 9 + val weight2 = 1.0 / 9 + assert(metrics.weightedTruePositiveRate ~== + (weight0 * tpRate0 + weight1 * tpRate1 + weight2 * tpRate2) relTol delta) + assert(metrics.weightedFalsePositiveRate ~== + (weight0 * fpRate0 + weight1 * fpRate1 + weight2 * fpRate2) relTol delta) + assert(metrics.weightedPrecision ~== + (weight0 * precision0 + weight1 * precision1 + weight2 * precision2) relTol delta) + assert(metrics.weightedRecall ~== + (weight0 * recall0 + weight1 * recall1 + weight2 * recall2) relTol delta) + assert(metrics.weightedFMeasure ~== + (weight0 * f1measure0 + weight1 * f1measure1 + weight2 * f1measure2) relTol delta) + assert(metrics.weightedFMeasure(2.0) ~== + (weight0 * f2measure0 + weight1 * f2measure1 + weight2 * f2measure2) relTol delta) + assert(metrics.labels === labels) + } + + test("Multiclass evaluation metrics with weights") { + /* + * Confusion matrix for 3-class classification with total 9 instances with 2 weights: + * |2 * w1|1 * w2 |1 * w1| true class0 (4 instances) + * |1 * w2|2 * w1 + 1 * w2|0 | true class1 (4 instances) + * |0 |0 |1 * w2| true class2 (1 instance) + */ + val w1 = 2.2 + val w2 = 1.5 + val tw = 2.0 * w1 + 1.0 * w2 + 1.0 * w1 + 1.0 * w2 + 2.0 * w1 + 1.0 * w2 + 1.0 * w2 + val confusionMatrix = Matrices.dense(3, 3, + Array(2 * w1, 1 * w2, 0, 1 * w2, 2 * w1 + 1 * w2, 0, 1 * w1, 0, 1 * w2)) + val labels = Array(0.0, 1.0, 2.0) + val predictionAndLabelsWithWeights = sc.parallelize( + Seq((0.0, 0.0, w1), (0.0, 1.0, w2), (0.0, 0.0, w1), (1.0, 0.0, w2), + (1.0, 1.0, w1), (1.0, 1.0, w2), (1.0, 1.0, w1), (2.0, 2.0, w2), + (2.0, 0.0, w1)), 2) + val metrics = new MulticlassMetrics(predictionAndLabelsWithWeights) + val tpRate0 = (2.0 * w1) / (2.0 * w1 + 1.0 * w2 + 1.0 * w1) + val tpRate1 = (2.0 * w1 + 1.0 * w2) / (2.0 * w1 + 1.0 * w2 + 1.0 * w2) + val tpRate2 = (1.0 * w2) / (1.0 * w2 + 0) + val fpRate0 = (1.0 * w2) / (tw - (2.0 * w1 + 1.0 * w2 + 1.0 * w1)) + val fpRate1 = (1.0 * w2) / (tw - (1.0 * w2 + 2.0 * w1 + 1.0 * w2)) + val fpRate2 = (1.0 * w1) / (tw - (1.0 * w2)) + val precision0 = (2.0 * w1) / (2 * w1 + 1 * w2) + val precision1 = (2.0 * w1 + 1.0 * w2) / (2.0 * w1 + 1.0 * w2 + 1.0 * w2) + val precision2 = (1.0 * w2) / (1 * w1 + 1 * w2) + val recall0 = (2.0 * w1) / (2.0 * w1 + 1.0 * w2 + 1.0 * w1) + val recall1 = (2.0 * w1 + 1.0 * w2) / (2.0 * w1 + 1.0 * w2 + 1.0 * w2) + val recall2 = (1.0 * w2) / (1.0 * w2 + 0) + val f1measure0 = 2 * precision0 * recall0 / (precision0 + recall0) + val f1measure1 = 2 * precision1 * recall1 / (precision1 + recall1) + val f1measure2 = 2 * precision2 * recall2 / (precision2 + recall2) + val f2measure0 = (1 + 2 * 2) * precision0 * recall0 / (2 * 2 * precision0 + recall0) + val f2measure1 = (1 + 2 * 2) * precision1 * recall1 / (2 * 2 * precision1 + recall1) + val f2measure2 = (1 + 2 * 2) * precision2 * recall2 / (2 * 2 * precision2 + recall2) + + assert(metrics.confusionMatrix.asML ~== confusionMatrix relTol delta) + assert(metrics.truePositiveRate(0.0) ~== tpRate0 relTol delta) + assert(metrics.truePositiveRate(1.0) ~== tpRate1 relTol delta) + assert(metrics.truePositiveRate(2.0) ~== tpRate2 relTol delta) + assert(metrics.falsePositiveRate(0.0) ~== fpRate0 relTol delta) + assert(metrics.falsePositiveRate(1.0) ~== fpRate1 relTol delta) + assert(metrics.falsePositiveRate(2.0) ~== fpRate2 relTol delta) + assert(metrics.precision(0.0) ~== precision0 relTol delta) + assert(metrics.precision(1.0) ~== precision1 relTol delta) + assert(metrics.precision(2.0) ~== precision2 relTol delta) + assert(metrics.recall(0.0) ~== recall0 relTol delta) + assert(metrics.recall(1.0) ~== recall1 relTol delta) + assert(metrics.recall(2.0) ~== recall2 relTol delta) + assert(metrics.fMeasure(0.0) ~== f1measure0 relTol delta) + assert(metrics.fMeasure(1.0) ~== f1measure1 relTol delta) + assert(metrics.fMeasure(2.0) ~== f1measure2 relTol delta) + assert(metrics.fMeasure(0.0, 2.0) ~== f2measure0 relTol delta) + assert(metrics.fMeasure(1.0, 2.0) ~== f2measure1 relTol delta) + assert(metrics.fMeasure(2.0, 2.0) ~== f2measure2 relTol delta) - assert(math.abs(metrics.accuracy - - (2.0 + 3.0 + 1.0) / ((2 + 3 + 1) + (1 + 1 + 1))) < delta) - assert(math.abs(metrics.accuracy - metrics.precision) < delta) - assert(math.abs(metrics.accuracy - metrics.recall) < delta) - assert(math.abs(metrics.accuracy - metrics.fMeasure) < delta) - assert(math.abs(metrics.accuracy - metrics.weightedRecall) < delta) - assert(math.abs(metrics.weightedTruePositiveRate - - ((4.0 / 9) * tpRate0 + (4.0 / 9) * tpRate1 + (1.0 / 9) * tpRate2)) < delta) - assert(math.abs(metrics.weightedFalsePositiveRate - - ((4.0 / 9) * fpRate0 + (4.0 / 9) * fpRate1 + (1.0 / 9) * fpRate2)) < delta) - assert(math.abs(metrics.weightedPrecision - - ((4.0 / 9) * precision0 + (4.0 / 9) * precision1 + (1.0 / 9) * precision2)) < delta) - assert(math.abs(metrics.weightedRecall - - ((4.0 / 9) * recall0 + (4.0 / 9) * recall1 + (1.0 / 9) * recall2)) < delta) - assert(math.abs(metrics.weightedFMeasure - - ((4.0 / 9) * f1measure0 + (4.0 / 9) * f1measure1 + (1.0 / 9) * f1measure2)) < delta) - assert(math.abs(metrics.weightedFMeasure(2.0) - - ((4.0 / 9) * f2measure0 + (4.0 / 9) * f2measure1 + (1.0 / 9) * f2measure2)) < delta) - assert(metrics.labels.sameElements(labels)) + assert(metrics.accuracy ~== + (2.0 * w1 + 2.0 * w1 + 1.0 * w2 + 1.0 * w2) / tw relTol delta) + assert(metrics.accuracy ~== metrics.weightedRecall relTol delta) + val weight0 = (2 * w1 + 1 * w2 + 1 * w1) / tw + val weight1 = (1 * w2 + 2 * w1 + 1 * w2) / tw + val weight2 = 1 * w2 / tw + assert(metrics.weightedTruePositiveRate ~== + (weight0 * tpRate0 + weight1 * tpRate1 + weight2 * tpRate2) relTol delta) + assert(metrics.weightedFalsePositiveRate ~== + (weight0 * fpRate0 + weight1 * fpRate1 + weight2 * fpRate2) relTol delta) + assert(metrics.weightedPrecision ~== + (weight0 * precision0 + weight1 * precision1 + weight2 * precision2) relTol delta) + assert(metrics.weightedRecall ~== + (weight0 * recall0 + weight1 * recall1 + weight2 * recall2) relTol delta) + assert(metrics.weightedFMeasure ~== + (weight0 * f1measure0 + weight1 * f1measure1 + weight2 * f1measure2) relTol delta) + assert(metrics.weightedFMeasure(2.0) ~== + (weight0 * f2measure0 + weight1 * f2measure1 + weight2 * f2measure2) relTol delta) + assert(metrics.labels === labels) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala index a660492c7ae59..03afd29e47505 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/MultilabelMetricsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.evaluation import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -79,24 +80,24 @@ class MultilabelMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { val hammingLoss = (1.0 / (7 * 3)) * (2 + 2 + 1 + 0 + 0 + 1 + 1) val strictAccuracy = 2.0 / 7 val accuracy = 1.0 / 7 * (1.0 / 3 + 1.0 /3 + 0 + 1.0 / 1 + 2.0 / 2 + 2.0 / 3 + 1.0 / 2) - assert(math.abs(metrics.precision(0.0) - precision0) < delta) - assert(math.abs(metrics.precision(1.0) - precision1) < delta) - assert(math.abs(metrics.precision(2.0) - precision2) < delta) - assert(math.abs(metrics.recall(0.0) - recall0) < delta) - assert(math.abs(metrics.recall(1.0) - recall1) < delta) - assert(math.abs(metrics.recall(2.0) - recall2) < delta) - assert(math.abs(metrics.f1Measure(0.0) - f1measure0) < delta) - assert(math.abs(metrics.f1Measure(1.0) - f1measure1) < delta) - assert(math.abs(metrics.f1Measure(2.0) - f1measure2) < delta) - assert(math.abs(metrics.microPrecision - microPrecisionClass) < delta) - assert(math.abs(metrics.microRecall - microRecallClass) < delta) - assert(math.abs(metrics.microF1Measure - microF1MeasureClass) < delta) - assert(math.abs(metrics.precision - macroPrecisionDoc) < delta) - assert(math.abs(metrics.recall - macroRecallDoc) < delta) - assert(math.abs(metrics.f1Measure - macroF1MeasureDoc) < delta) - assert(math.abs(metrics.hammingLoss - hammingLoss) < delta) - assert(math.abs(metrics.subsetAccuracy - strictAccuracy) < delta) - assert(math.abs(metrics.accuracy - accuracy) < delta) + assert(metrics.precision(0.0) ~== precision0 absTol delta) + assert(metrics.precision(1.0) ~== precision1 absTol delta) + assert(metrics.precision(2.0) ~== precision2 absTol delta) + assert(metrics.recall(0.0) ~== recall0 absTol delta) + assert(metrics.recall(1.0) ~== recall1 absTol delta) + assert(metrics.recall(2.0) ~== recall2 absTol delta) + assert(metrics.f1Measure(0.0) ~== f1measure0 absTol delta) + assert(metrics.f1Measure(1.0) ~== f1measure1 absTol delta) + assert(metrics.f1Measure(2.0) ~== f1measure2 absTol delta) + assert(metrics.microPrecision ~== microPrecisionClass absTol delta) + assert(metrics.microRecall ~== microRecallClass absTol delta) + assert(metrics.microF1Measure ~== microF1MeasureClass absTol delta) + assert(metrics.precision ~== macroPrecisionDoc absTol delta) + assert(metrics.recall ~== macroRecallDoc absTol delta) + assert(metrics.f1Measure ~== macroF1MeasureDoc absTol delta) + assert(metrics.hammingLoss ~== hammingLoss absTol delta) + assert(metrics.subsetAccuracy ~== strictAccuracy absTol delta) + assert(metrics.accuracy ~== accuracy absTol delta) assert(metrics.labels.sameElements(Array(0.0, 1.0, 2.0))) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala index f334be2c2ba83..489eb15f4dbab 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RankingMetricsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.mllib.util.TestingUtils._ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { - test("Ranking metrics: MAP, NDCG") { + test("Ranking metrics: MAP, NDCG, Recall") { val predictionAndLabels = sc.parallelize( Seq( (Array(1, 6, 2, 7, 8, 3, 9, 10, 4, 5), Array(1, 2, 3, 4, 5)), @@ -45,13 +45,25 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(map ~== 0.355026 absTol eps) + assert(metrics.meanAveragePrecisionAt(1) ~== 0.333334 absTol eps) + assert(metrics.meanAveragePrecisionAt(2) ~== 0.25 absTol eps) + assert(metrics.meanAveragePrecisionAt(3) ~== 0.24074 absTol eps) + assert(metrics.ndcgAt(3) ~== 1.0/3 absTol eps) assert(metrics.ndcgAt(5) ~== 0.328788 absTol eps) assert(metrics.ndcgAt(10) ~== 0.487913 absTol eps) assert(metrics.ndcgAt(15) ~== metrics.ndcgAt(10) absTol eps) + + assert(metrics.recallAt(1) ~== 1.0/15 absTol eps) + assert(metrics.recallAt(2) ~== 8.0/45 absTol eps) + assert(metrics.recallAt(3) ~== 11.0/45 absTol eps) + assert(metrics.recallAt(4) ~== 11.0/45 absTol eps) + assert(metrics.recallAt(5) ~== 16.0/45 absTol eps) + assert(metrics.recallAt(10) ~== 2.0/3 absTol eps) + assert(metrics.recallAt(15) ~== 2.0/3 absTol eps) } - test("MAP, NDCG with few predictions (SPARK-14886)") { + test("MAP, NDCG, Recall with few predictions (SPARK-14886)") { val predictionAndLabels = sc.parallelize( Seq( (Array(1, 6, 2), Array(1, 2, 3, 4, 5)), @@ -64,6 +76,8 @@ class RankingMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { assert(metrics.precisionAt(2) ~== 0.25 absTol eps) assert(metrics.ndcgAt(1) ~== 0.5 absTol eps) assert(metrics.ndcgAt(2) ~== 0.30657 absTol eps) + assert(metrics.recallAt(1) ~== 0.1 absTol eps) + assert(metrics.recallAt(2) ~== 0.1 absTol eps) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala index f1d517383643d..23809777f7d3a 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/RegressionMetricsSuite.scala @@ -133,4 +133,54 @@ class RegressionMetricsSuite extends SparkFunSuite with MLlibTestSparkContext { "root mean squared error mismatch") assert(metrics.r2 ~== 1.0 absTol eps, "r2 score mismatch") } + + test("regression metrics with same (1.0) weight samples") { + val predictionAndObservationWithWeight = sc.parallelize( + Seq((2.25, 3.0, 1.0), (-0.25, -0.5, 1.0), (1.75, 2.0, 1.0), (7.75, 7.0, 1.0)), 2) + val metrics = new RegressionMetrics(predictionAndObservationWithWeight, false) + assert(metrics.explainedVariance ~== 8.79687 absTol eps, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.5 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.3125 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.55901 absTol eps, + "root mean squared error mismatch") + assert(metrics.r2 ~== 0.95717 absTol eps, "r2 score mismatch") + } + + /** + * The following values are hand calculated using the formula: + * [[https://en.wikipedia.org/wiki/Weighted_arithmetic_mean#Reliability_weights]] + * preds = c(2.25, -0.25, 1.75, 7.75) + * obs = c(3.0, -0.5, 2.0, 7.0) + * weights = c(0.1, 0.2, 0.15, 0.05) + * count = 4 + * + * Weighted metrics can be calculated with MultivariateStatisticalSummary. + * (observations, observations - predictions) + * mean (1.7, 0.05) + * variance (7.3, 0.3) + * numNonZeros (0.5, 0.5) + * max (7.0, 0.75) + * min (-0.5, -0.75) + * normL2 (2.0, 0.32596) + * normL1 (1.05, 0.2) + * + * explainedVariance: sum(pow((preds - 1.7),2)*weight) / weightedCount = 5.2425 + * meanAbsoluteError: normL1(1) / weightedCount = 0.4 + * meanSquaredError: pow(normL2(1),2) / weightedCount = 0.2125 + * rootMeanSquaredError: sqrt(meanSquaredError) = 0.46098 + * r2: 1 - pow(normL2(1),2) / (variance(0) * (weightedCount - 1)) = 1.02910 + */ + test("regression metrics with weighted samples") { + val predictionAndObservationWithWeight = sc.parallelize( + Seq((2.25, 3.0, 0.1), (-0.25, -0.5, 0.2), (1.75, 2.0, 0.15), (7.75, 7.0, 0.05)), 2) + val metrics = new RegressionMetrics(predictionAndObservationWithWeight, false) + assert(metrics.explainedVariance ~== 5.2425 absTol eps, + "explained variance regression score mismatch") + assert(metrics.meanAbsoluteError ~== 0.4 absTol eps, "mean absolute error mismatch") + assert(metrics.meanSquaredError ~== 0.2125 absTol eps, "mean squared error mismatch") + assert(metrics.rootMeanSquaredError ~== 0.46098 absTol eps, + "root mean squared error mismatch") + assert(metrics.r2 ~== 1.02910 absTol eps, "r2 score mismatch") + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala index 5c938a61ed990..1049730ffd01d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/IDFSuite.scala @@ -39,9 +39,11 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { math.log((m + 1.0) / (x + 1.0)) }) assert(model.idf ~== expected absTol 1e-12) + assert(model.numDocs === 3) + assert(model.docFreq === Array(0, 3, 1, 2)) val assertHelper = (tfidf: Array[Vector]) => { - assert(tfidf.size === 3) + assert(tfidf.length === 3) val tfidf0 = tfidf(0).asInstanceOf[SparseVector] assert(tfidf0.indices === Array(1, 3)) assert(Vectors.dense(tfidf0.values) ~== @@ -70,19 +72,21 @@ class IDFSuite extends SparkFunSuite with MLlibTestSparkContext { ) val m = localTermFrequencies.size val termFrequencies = sc.parallelize(localTermFrequencies, 2) - val idf = new IDF(minDocFreq = 1) + val idf = new IDF(minDocFreq = 2) val model = idf.fit(termFrequencies) val expected = Vectors.dense(Array(0, 3, 1, 2).map { x => - if (x > 0) { + if (x >= 2) { math.log((m + 1.0) / (x + 1.0)) } else { 0 } }) assert(model.idf ~== expected absTol 1e-12) + assert(model.numDocs === 3) + assert(model.docFreq === Array(0, 3, 0, 2)) val assertHelper = (tfidf: Array[Vector]) => { - assert(tfidf.size === 3) + assert(tfidf.length === 3) val tfidf0 = tfidf(0).asInstanceOf[SparseVector] assert(tfidf0.indices === Array(1, 3)) assert(Vectors.dense(tfidf0.values) ~== diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala index 8eab12416a698..e478f14906ba8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/PCASuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ @@ -54,4 +54,25 @@ class PCASuite extends SparkFunSuite with MLlibTestSparkContext { // check overflowing assert(PCAUtil.memoryCost(40000, 60000) > Int.MaxValue) } + + test("number of features more than 65535") { + val data1 = sc.parallelize(Array( + Vectors.dense(Array.fill(100000)(2.0)), + Vectors.dense(Array.fill(100000)(0.0)) + ), 2) + + val pca = new PCA(2).fit(data1) + // Eigen values should not be negative + assert(pca.explainedVariance.values.forall(_ >= 0)) + // Norm of the principal component should be 1.0 + assert(Math.sqrt(pca.pc.values.slice(0, 100000) + .map(Math.pow(_, 2)).sum) ~== 1.0 relTol 1e-8) + // Leading explainedVariance is 1.0 + assert(pca.explainedVariance(0) ~== 1.0 relTol 1e-12) + + // Leading principal component is '1' vector + val firstValue = pca.pc.values(0) + pca.pc.values.slice(0, 100000).map(values => + assert(values ~== firstValue relTol 1e-12)) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala index f4fa216b8eba0..a679fe43414f2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/feature/Word2VecSuite.scala @@ -18,8 +18,10 @@ package org.apache.spark.mllib.feature import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.sql.internal.SQLConf._ import org.apache.spark.util.Utils class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -109,12 +111,16 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { test("big model load / save") { // backupping old values - val oldBufferConfValue = spark.conf.get("spark.kryoserializer.buffer.max", "64m") - val oldBufferMaxConfValue = spark.conf.get("spark.kryoserializer.buffer", "64k") + val oldBufferConfValue = spark.conf.get(KRYO_SERIALIZER_BUFFER_SIZE.key, "64m") + val oldBufferMaxConfValue = spark.conf.get(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, "64k") + val oldSetCommandRejectsSparkCoreConfs = spark.conf.get( + SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, "true") // setting test values to trigger partitioning - spark.conf.set("spark.kryoserializer.buffer", "50b") - spark.conf.set("spark.kryoserializer.buffer.max", "50b") + + // this is needed to set configurations which are also defined to SparkConf + spark.conf.set(SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, "false") + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, "50b") // create a model bigger than 50 Bytes val word2VecMap = Map((0 to 10).map(i => s"$i" -> Array.fill(10)(0.1f)): _*) @@ -137,8 +143,9 @@ class Word2VecSuite extends SparkFunSuite with MLlibTestSparkContext { "that spans over multiple partitions", t) } finally { Utils.deleteRecursively(tempDir) - spark.conf.set("spark.kryoserializer.buffer", oldBufferConfValue) - spark.conf.set("spark.kryoserializer.buffer.max", oldBufferMaxConfValue) + spark.conf.set(KRYO_SERIALIZER_BUFFER_SIZE.key, oldBufferConfValue) + spark.conf.set(KRYO_SERIALIZER_MAX_BUFFER_SIZE.key, oldBufferMaxConfValue) + spark.conf.set(SET_COMMAND_REJECTS_SPARK_CORE_CONFS.key, oldSetCommandRejectsSparkCoreConfs) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala index dcb1f398b04b8..26a75699248d0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/AssociationRulesSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.fpm import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -63,7 +64,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { [1] 23 */ assert(results1.size === 23) - assert(results1.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(results1.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) val results2 = ar .setMinConfidence(0) @@ -84,7 +85,7 @@ class AssociationRulesSuite extends SparkFunSuite with MLlibTestSparkContext { [1] 23 */ assert(results2.size === 30) - assert(results2.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(results2.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala index dc44c58e97eb4..fa8f03be089ce 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPGrowthSuite.scala @@ -16,10 +16,9 @@ */ package org.apache.spark.mllib.fpm -import scala.language.existentials - import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.Utils class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -174,7 +173,7 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { .collect() assert(rules.size === 23) - assert(rules.count(rule => math.abs(rule.confidence - 1.0D) < 1e-6) == 23) + assert(rules.count(rule => rule.confidence ~= 1.0D absTol 1e-6) == 23) } test("FP-Growth using Int type") { @@ -301,7 +300,7 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val path = tempDir.toURI.toString try { model3.save(sc, path) - val newModel = FPGrowthModel.load(sc, path) + val newModel = FPGrowthModel.load(sc, path).asInstanceOf[FPGrowthModel[String]] val newFreqItemsets = newModel.freqItemsets.collect().map { itemset => (itemset.items.toSet, itemset.freq) } @@ -335,7 +334,7 @@ class FPGrowthSuite extends SparkFunSuite with MLlibTestSparkContext { val path = tempDir.toURI.toString try { model3.save(sc, path) - val newModel = FPGrowthModel.load(sc, path) + val newModel = FPGrowthModel.load(sc, path).asInstanceOf[FPGrowthModel[String]] val newFreqItemsets = newModel.freqItemsets.collect().map { itemset => (itemset.items.toSet, itemset.freq) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala index a56d7b3579213..4e2548d6915a6 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/FPTreeSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.fpm -import scala.language.existentials - import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext diff --git a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala index c2e08d078fc1a..a700706cdc87d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/fpm/PrefixSpanSuite.scala @@ -16,8 +16,6 @@ */ package org.apache.spark.mllib.fpm -import scala.language.existentials - import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.util.Utils @@ -420,7 +418,9 @@ class PrefixSpanSuite extends SparkFunSuite with MLlibTestSparkContext { val path = tempDir.toURI.toString try { model.save(sc, path) - val newModel = PrefixSpanModel.load(sc, path) + // Save/loading this model results in item type "Object" even if in this case + // the objects are Integers -- not Int as in the original saved model. + val newModel = PrefixSpanModel.load(sc, path).asInstanceOf[PrefixSpanModel[AnyRef]] val originalSet = model.freqSequences.collect().map { x => (x.sequence.map(_.toSet).toSeq, x.freq) }.toSet diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index d76edb940b2bd..b4520d42fedf5 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -27,6 +27,7 @@ import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar._ import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -34,7 +35,7 @@ import org.apache.spark.serializer.KryoSerializer class MatricesSuite extends SparkFunSuite { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() @@ -511,10 +512,10 @@ class MatricesSuite extends SparkFunSuite { mat.toString(0, 0) mat.toString(Int.MinValue, Int.MinValue) mat.toString(Int.MaxValue, Int.MaxValue) - var lines = mat.toString(6, 50).lines.toArray + var lines = mat.toString(6, 50).split('\n') assert(lines.size == 5 && lines.forall(_.size <= 50)) - lines = mat.toString(5, 100).lines.toArray + lines = mat.toString(5, 100).split('\n') assert(lines.size == 5 && lines.forall(_.size <= 100)) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala index 6c1d58089867a..5f19e466ecad0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/UDTSerializationBenchmark.scala @@ -32,7 +32,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder */ object UDTSerializationBenchmark extends BenchmarkBase { - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("VectorUDT de/serialization") { val iters = 1e2.toInt diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala index 217b4a35438fd..fee0b02bf8ed8 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala @@ -25,6 +25,7 @@ import org.json4s.jackson.JsonMethods.{parse => parseJson} import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.{linalg => newlinalg} import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.serializer.KryoSerializer @@ -38,7 +39,7 @@ class VectorsSuite extends SparkFunSuite with Logging { test("kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() def check[T: ClassTag](t: T) { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala index 566ce95be084a..cca4eb4e4260e 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -22,6 +22,7 @@ import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { @@ -238,7 +239,7 @@ class IndexedRowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { for (i <- 0 until n; j <- i + 1 until n) { val trueResult = gram(i, j) / scala.math.sqrt(gram(i, i) * gram(j, j)) - assert(math.abs(G(i, j) - trueResult) < 1e-6) + assert(G(i, j) ~== trueResult absTol 1e-6) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 7c9e14f8cee70..a0c4c68243e67 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -101,6 +101,26 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { } } + test("getTreeAggregateIdealDepth") { + val nbPartitions = 100 + val vectors = sc.emptyRDD[Vector] + .repartition(nbPartitions) + val rowMat = new RowMatrix(vectors) + + assert(rowMat.getTreeAggregateIdealDepth(100 * 1024 * 1024) === 2) + assert(rowMat.getTreeAggregateIdealDepth(110 * 1024 * 1024) === 3) + assert(rowMat.getTreeAggregateIdealDepth(700 * 1024 * 1024) === 10) + + val zeroSizeException = intercept[Exception]{ + rowMat.getTreeAggregateIdealDepth(0) + } + assert(zeroSizeException.getMessage.contains("zero-size object to aggregate")) + val objectBiggerThanResultSize = intercept[Exception]{ + rowMat.getTreeAggregateIdealDepth(1100 * 1024 * 1024) + } + assert(objectBiggerThanResultSize.getMessage.contains("it's bigger than maxResultSize")) + } + test("similar columns") { val colMags = Vectors.dense(math.sqrt(126), math.sqrt(66), math.sqrt(94)) val expected = BDM( @@ -266,6 +286,20 @@ class RowMatrixSuite extends SparkFunSuite with MLlibTestSparkContext { } } + test("dense vector covariance accuracy (SPARK-26158)") { + val denseData = Seq( + Vectors.dense(100000.000004, 199999.999999), + Vectors.dense(100000.000012, 200000.000002), + Vectors.dense(99999.9999931, 200000.000003), + Vectors.dense(99999.9999977, 200000.000001) + ) + val denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + + val result = denseMat.computeCovariance() + val expected = breeze.linalg.cov(denseMat.toBreeze()) + assert(closeToZero(abs(expected) - abs(result.asBreeze.asInstanceOf[BDM[Double]]))) + } + test("compute covariance") { for (mat <- Seq(denseMat, sparseMat)) { val result = mat.computeCovariance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 37eb794b0c5c9..6250b0363ee3b 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -190,7 +190,7 @@ class GradientDescentClusterSuite extends SparkFunSuite with LocalClusterSparkCo iter.map(i => (1.0, Vectors.dense(Array.fill(n)(random.nextDouble())))) }.cache() // If we serialize data directly in the task closure, the size of the serialized task would be - // greater than 1MB and hence Spark would throw an error. + // greater than 1MiB and hence Spark would throw an error. val (weights, loss) = GradientDescent.runMiniBatchSGD( points, new LogisticGradient, diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala index 4c6e76e47419b..08c581cd470d9 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/BinaryClassificationPMMLModelExportSuite.scala @@ -17,8 +17,7 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.RegressionModel -import org.dmg.pmml.RegressionNormalizationMethodType +import org.dmg.pmml.regression.RegressionModel import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.classification.LogisticRegressionModel @@ -51,7 +50,8 @@ class BinaryClassificationPMMLModelExportSuite extends SparkFunSuite { assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) // ensure logistic regression has normalization method set to LOGIT - assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.LOGIT) + assert(pmmlRegressionModel.getNormalizationMethod() === + RegressionModel.NormalizationMethod.LOGIT) } test("linear SVM PMML export") { @@ -78,7 +78,8 @@ class BinaryClassificationPMMLModelExportSuite extends SparkFunSuite { assert(pmmlRegressionModel.getRegressionTables.get(1).getTargetCategory === "0") assert(pmmlRegressionModel.getRegressionTables.get(1).getNumericPredictors.size === 0) // ensure linear SVM has normalization method set to NONE - assert(pmmlRegressionModel.getNormalizationMethod() == RegressionNormalizationMethodType.NONE) + assert(pmmlRegressionModel.getNormalizationMethod() === + RegressionModel.NormalizationMethod.NONE) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala index 1d32309481787..bf1a0fd8e6071 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/GeneralizedLinearPMMLModelExportSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.RegressionModel +import org.dmg.pmml.regression.RegressionModel import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.regression.{LassoModel, LinearRegressionModel, RidgeRegressionModel} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala index b3f9750afa730..b61c6225e9012 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/pmml/export/KMeansPMMLModelExportSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.mllib.pmml.export -import org.dmg.pmml.ClusteringModel +import org.dmg.pmml.clustering.ClusteringModel import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.clustering.KMeansModel diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala index e30ad159676ff..8011026e6fa65 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomDataGeneratorSuite.scala @@ -20,9 +20,9 @@ package org.apache.spark.mllib.random import org.apache.commons.math3.special.Gamma import org.apache.spark.SparkFunSuite +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.StatCounter -// TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged class RandomDataGeneratorSuite extends SparkFunSuite { def apiChecks(gen: RandomDataGenerator[Double]) { @@ -61,8 +61,8 @@ class RandomDataGeneratorSuite extends SparkFunSuite { gen.setSeed(seed.toLong) val sample = (0 until 100000).map { _ => gen.nextValue()} val stats = new StatCounter(sample) - assert(math.abs(stats.mean - mean) < epsilon) - assert(math.abs(stats.stdev - stddev) < epsilon) + assert(stats.mean ~== mean absTol epsilon) + assert(stats.stdev ~== stddev absTol epsilon) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala index f464d25c3fbda..9b4dc29d326a1 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/random/RandomRDDsSuite.scala @@ -23,14 +23,13 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.rdd.{RandomRDD, RandomRDDPartition} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.rdd.RDD import org.apache.spark.util.StatCounter /* * Note: avoid including APIs that do not set the seed for the RNG in unit tests * in order to guarantee deterministic behavior. - * - * TODO update tests to use TestingUtils for floating point comparison after PR 1367 is merged */ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Serializable { @@ -43,8 +42,8 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri val stats = rdd.stats() assert(expectedSize === stats.count) assert(expectedNumPartitions === rdd.partitions.size) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } // assume test RDDs are small @@ -63,8 +62,8 @@ class RandomRDDsSuite extends SparkFunSuite with MLlibTestSparkContext with Seri }} assert(expectedRows === values.size / expectedColumns) val stats = new StatCounter(values) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } test("RandomRDD sizes") { diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala index c1449ece740d4..d3366dcde2ede 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LabeledPointSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.ml.feature.{LabeledPoint => NewLabeledPoint} import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.serializer.KryoSerializer @@ -57,7 +58,7 @@ class LabeledPointSuite extends SparkFunSuite { test("Kryo class register") { val conf = new SparkConf(false) - conf.set("spark.kryo.registrationRequired", "true") + conf.set(KRYO_REGISTRATION_REQUIRED, true) val ser = new KryoSerializer(conf).newInstance() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala index e32767edb17a8..4613f7fb6f400 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/CorrelationSuite.scala @@ -26,6 +26,7 @@ import org.apache.spark.mllib.random.RandomRDDs import org.apache.spark.mllib.stat.correlation.{Correlations, PearsonCorrelation, SpearmanCorrelation} import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Logging { @@ -57,15 +58,15 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val expected = 0.6546537 val default = Statistics.corr(x, y) val p1 = Statistics.corr(x, y, "pearson") - assert(approxEqual(expected, default)) - assert(approxEqual(expected, p1)) + assert(expected ~== default absTol 1e-6) + assert(expected ~== p1 absTol 1e-6) // numPartitions >= size for input RDDs for (numParts <- List(xData.size, xData.size * 2)) { val x1 = sc.parallelize(xData, numParts) val y1 = sc.parallelize(yData, numParts) val p2 = Statistics.corr(x1, y1) - assert(approxEqual(expected, p2)) + assert(expected ~== p2 absTol 1e-6) } // RDD of zero variance @@ -78,14 +79,14 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val y = sc.parallelize(yData) val expected = 0.5 val s1 = Statistics.corr(x, y, "spearman") - assert(approxEqual(expected, s1)) + assert(expected ~== s1 absTol 1e-6) // numPartitions >= size for input RDDs for (numParts <- List(xData.size, xData.size * 2)) { val x1 = sc.parallelize(xData, numParts) val y1 = sc.parallelize(yData, numParts) val s2 = Statistics.corr(x1, y1, "spearman") - assert(approxEqual(expected, s2)) + assert(expected ~== s2 absTol 1e-6) } // RDD of zero variance => zero variance in ranks @@ -141,14 +142,14 @@ class CorrelationSuite extends SparkFunSuite with MLlibTestSparkContext with Log val a = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) val b = RandomRDDs.normalRDD(sc, 100000, 10).map(_ + 1000000000.0) val p = Statistics.corr(a, b, method = "pearson") - assert(approxEqual(p, 0.0, 0.01)) + assert(p ~== 0.0 absTol 0.01) } def approxEqual(v1: Double, v2: Double, threshold: Double = 1e-6): Boolean = { if (v1.isNaN) { v2.isNaN } else { - math.abs(v1 - v2) <= threshold + v1 ~== v2 absTol threshold } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala index 5feccdf33681a..9cbb3d0024daa 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/KernelDensitySuite.scala @@ -21,6 +21,7 @@ import org.apache.commons.math3.distribution.NormalDistribution import org.apache.spark.SparkFunSuite import org.apache.spark.mllib.util.MLlibTestSparkContext +import org.apache.spark.mllib.util.TestingUtils._ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { test("kernel density single sample") { @@ -29,8 +30,8 @@ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { val densities = new KernelDensity().setSample(rdd).setBandwidth(3.0).estimate(evaluationPoints) val normal = new NormalDistribution(5.0, 3.0) val acceptableErr = 1e-6 - assert(math.abs(densities(0) - normal.density(5.0)) < acceptableErr) - assert(math.abs(densities(1) - normal.density(6.0)) < acceptableErr) + assert(densities(0) ~== normal.density(5.0) absTol acceptableErr) + assert(densities(1) ~== normal.density(6.0) absTol acceptableErr) } test("kernel density multiple samples") { @@ -40,9 +41,9 @@ class KernelDensitySuite extends SparkFunSuite with MLlibTestSparkContext { val normal1 = new NormalDistribution(5.0, 3.0) val normal2 = new NormalDistribution(10.0, 3.0) val acceptableErr = 1e-6 - assert(math.abs( - densities(0) - (normal1.density(5.0) + normal2.density(5.0)) / 2) < acceptableErr) - assert(math.abs( - densities(1) - (normal1.density(6.0) + normal2.density(6.0)) / 2) < acceptableErr) + assert( + densities(0) ~== ((normal1.density(5.0) + normal2.density(5.0)) / 2) absTol acceptableErr) + assert( + densities(1) ~== ((normal1.density(6.0) + normal2.density(6.0)) / 2) absTol acceptableErr) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala index 669d44223d713..4c88fd3d83a0d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/stat/distribution/MultivariateGaussianSuite.scala @@ -17,10 +17,12 @@ package org.apache.spark.mllib.stat.distribution -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.mllib.linalg.{Matrices, Vectors} import org.apache.spark.mllib.util.MLlibTestSparkContext import org.apache.spark.mllib.util.TestingUtils._ +import org.apache.spark.serializer.KryoSerializer class MultivariateGaussianSuite extends SparkFunSuite with MLlibTestSparkContext { test("univariate") { @@ -80,4 +82,23 @@ class MultivariateGaussianSuite extends SparkFunSuite with MLlibTestSparkContext assert(dist.pdf(x) ~== 7.154782224045512E-5 absTol 1E-9) } + test("Kryo class register") { + val conf = new SparkConf(false) + conf.set(KRYO_REGISTRATION_REQUIRED, true) + + val ser = new KryoSerializer(conf).newInstance() + + val mu = Vectors.dense(0.0, 0.0) + val sigma1 = Matrices.dense(2, 2, Array(1.0, 0.0, 0.0, 1.0)) + val dist1 = new MultivariateGaussian(mu, sigma1) + + val sigma2 = Matrices.dense(2, 2, Array(4.0, -1.0, -1.0, 2.0)) + val dist2 = new MultivariateGaussian(mu, sigma2) + + Seq(dist1, dist2).foreach { i => + val i2 = ser.deserialize[MultivariateGaussian](ser.serialize(i)) + assert(i.sigma === i2.sigma) + assert(i.mu === i2.mu) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index bc59f3f4125fb..8378a599362a2 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -73,7 +73,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -100,7 +100,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { maxDepth = 2, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 2, 1 -> 2)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -116,7 +116,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -133,7 +133,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Gini, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -150,7 +150,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -167,7 +167,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(Classification, Entropy, maxDepth = 3, numClasses = 2, maxBins = 100) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -183,7 +183,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val rdd = sc.parallelize(arr) val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(strategy.isMulticlassClassification) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -240,7 +240,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = maxBins, categoricalFeaturesInfo = Map(0 -> 3, 1 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(metadata.isUnordered(featureIndex = 0)) assert(metadata.isUnordered(featureIndex = 1)) @@ -288,7 +288,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { val strategy = new Strategy(algo = Classification, impurity = Gini, maxDepth = 4, numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 3)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(metadata.isUnordered(featureIndex = 0)) val model = DecisionTree.train(rdd, strategy) @@ -310,7 +310,7 @@ class DecisionTreeSuite extends SparkFunSuite with MLlibTestSparkContext { numClasses = 3, maxBins = 100, categoricalFeaturesInfo = Map(0 -> 10, 1 -> 10)) assert(strategy.isMulticlassClassification) - val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML), strategy) + val metadata = DecisionTreeMetadata.buildMetadata(rdd.map(_.asML.toInstance), strategy) assert(!metadata.isUnordered(featureIndex = 0)) assert(!metadata.isUnordered(featureIndex = 1)) @@ -607,7 +607,7 @@ object DecisionTreeSuite extends SparkFunSuite { checkEqual(a.topNode, b.topNode) } catch { case ex: Exception => - throw new AssertionError("checkEqual failed since the two trees were not identical.\n" + + fail("checkEqual failed since the two trees were not identical.\n" + "TREE A:\n" + a.toDebugString + "\n" + "TREE B:\n" + b.toDebugString + "\n", ex) } @@ -628,20 +628,21 @@ object DecisionTreeSuite extends SparkFunSuite { // TODO: Check other fields besides the information gain. case (Some(aStats), Some(bStats)) => assert(aStats.gain === bStats.gain) case (None, None) => - case _ => throw new AssertionError( - s"Only one instance has stats defined. (a.stats: ${a.stats}, b.stats: ${b.stats})") + case _ => fail(s"Only one instance has stats defined. (a.stats: ${a.stats}, " + + s"b.stats: ${b.stats})") } (a.leftNode, b.leftNode) match { case (Some(aNode), Some(bNode)) => checkEqual(aNode, bNode) case (None, None) => - case _ => throw new AssertionError("Only one instance has leftNode defined. " + - s"(a.leftNode: ${a.leftNode}, b.leftNode: ${b.leftNode})") + case _ => + fail("Only one instance has leftNode defined. (a.leftNode: ${a.leftNode}," + + " b.leftNode: ${b.leftNode})") } (a.rightNode, b.rightNode) match { case (Some(aNode: Node), Some(bNode: Node)) => checkEqual(aNode, bNode) case (None, None) => - case _ => throw new AssertionError("Only one instance has rightNode defined. " + - s"(a.rightNode: ${a.rightNode}, b.rightNode: ${b.rightNode})") + case _ => fail("Only one instance has rightNode defined. (a.rightNode: ${a.rightNode}, " + + "b.rightNode: ${b.rightNode})") } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala index 1cc8f342021a0..d43e62bb65535 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/EnsembleTestHelper.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.tree.model.TreeEnsembleModel +import org.apache.spark.mllib.util.TestingUtils._ import org.apache.spark.util.StatCounter object EnsembleTestHelper { @@ -43,8 +44,8 @@ object EnsembleTestHelper { values ++= row } val stats = new StatCounter(values) - assert(math.abs(stats.mean - expectedMean) < epsilon) - assert(math.abs(stats.stdev - expectedStddev) < epsilon) + assert(stats.mean ~== expectedMean absTol epsilon) + assert(stats.stdev ~== expectedStddev absTol epsilon) } def validateClassifier( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala index d0f02dd966bd5..078c6e6fff9fc 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/ImpuritySuite.scala @@ -18,23 +18,63 @@ package org.apache.spark.mllib.tree import org.apache.spark.SparkFunSuite -import org.apache.spark.mllib.tree.impurity.{EntropyAggregator, GiniAggregator} +import org.apache.spark.ml.util.TestingUtils._ +import org.apache.spark.mllib.tree.impurity._ /** * Test suites for `GiniAggregator` and `EntropyAggregator`. */ class ImpuritySuite extends SparkFunSuite { + + private val seed = 42 + test("Gini impurity does not support negative labels") { val gini = new GiniAggregator(2) intercept[IllegalArgumentException] { - gini.update(Array(0.0, 1.0, 2.0), 0, -1, 0.0) + gini.update(Array(0.0, 1.0, 2.0), 0, -1, 3, 0.0) } } test("Entropy does not support negative labels") { val entropy = new EntropyAggregator(2) intercept[IllegalArgumentException] { - entropy.update(Array(0.0, 1.0, 2.0), 0, -1, 0.0) + entropy.update(Array(0.0, 1.0, 2.0), 0, -1, 3, 0.0) + } + } + + test("Classification impurities are insensitive to scaling") { + val rng = new scala.util.Random(seed) + val weightedCounts = Array.fill(5)(rng.nextDouble()) + val smallWeightedCounts = weightedCounts.map(_ * 0.0001) + val largeWeightedCounts = weightedCounts.map(_ * 10000) + Seq(Gini, Entropy).foreach { impurity => + val impurity1 = impurity.calculate(weightedCounts, weightedCounts.sum) + assert(impurity.calculate(smallWeightedCounts, smallWeightedCounts.sum) + ~== impurity1 relTol 0.005) + assert(impurity.calculate(largeWeightedCounts, largeWeightedCounts.sum) + ~== impurity1 relTol 0.005) } } + + test("Regression impurities are insensitive to scaling") { + def computeStats(samples: Seq[Double], weights: Seq[Double]): (Double, Double, Double) = { + samples.zip(weights).foldLeft((0.0, 0.0, 0.0)) { case ((wn, wy, wyy), (y, w)) => + (wn + w, wy + w * y, wyy + w * y * y) + } + } + val rng = new scala.util.Random(seed) + val samples = Array.fill(10)(rng.nextDouble()) + val _weights = Array.fill(10)(rng.nextDouble()) + val smallWeights = _weights.map(_ * 0.0001) + val largeWeights = _weights.map(_ * 10000) + val (count, sum, sumSquared) = computeStats(samples, _weights) + Seq(Variance).foreach { impurity => + val impurity1 = impurity.calculate(count, sum, sumSquared) + val (smallCount, smallSum, smallSumSquared) = computeStats(samples, smallWeights) + val (largeCount, largeSum, largeSumSquared) = computeStats(samples, largeWeights) + assert(impurity.calculate(smallCount, smallSum, smallSumSquared) ~== impurity1 relTol 0.005) + assert(impurity.calculate(largeCount, largeSum, largeSumSquared) ~== impurity1 relTol 0.005) + } + } + } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala index 95d874b8432eb..2853b752cb85c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalClusterSparkContext.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.util import org.scalatest.{BeforeAndAfterAll, Suite} import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.internal.config.Network.RPC_MESSAGE_MAX_SIZE trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => @transient var sc: SparkContext = _ @@ -29,7 +30,7 @@ trait LocalClusterSparkContext extends BeforeAndAfterAll { self: Suite => val conf = new SparkConf() .setMaster("local-cluster[2, 1, 1024]") .setAppName("test-cluster") - .set("spark.rpc.message.maxSize", "1") // set to 1MB to detect direct serialization of data + .set(RPC_MESSAGE_MAX_SIZE, 1) // set to 1MB to detect direct serialization of data sc = new SparkContext(conf) } diff --git a/pom.xml b/pom.xml index 7ce7c9f0280e5..dae5eb0070155 100644 --- a/pom.xml +++ b/pom.xml @@ -25,7 +25,7 @@ 18 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT pom Spark Project Parent POM @@ -39,7 +39,7 @@ scm:git:git@github.com:apache/spark.git - scm:git:https://git-wip-us.apache.org/repos/asf/spark.git + scm:git:https://gitbox.apache.org/repos/asf/spark.git scm:git:git@github.com:apache/spark.git HEAD @@ -101,10 +101,14 @@ examples repl launcher + external/kafka-0-10-token-provider external/kafka-0-10 external/kafka-0-10-assembly external/kafka-0-10-sql external/avro + graph/api + graph/cypher + graph/graph @@ -114,26 +118,32 @@ 1.8 ${java.version} ${java.version} - 3.5.4 + 3.6.1 spark 1.7.16 1.2.17 - 2.7.3 + 2.7.4 2.5.0 ${hadoop.version} 3.4.6 2.7.1 + 0.4.2 org.spark-project.hive + 1.2.1.spark2 + 2.3.5 1.2.1 + + 2.3.0 10.12.1.1 - 1.10.0 - 1.5.3 + 1.10.1 + 1.5.5 nohive + com.twitter 1.6.0 - 9.3.24.v20180605 + 9.4.18.v20190429 3.1.0 0.9.3 2.4.0 @@ -154,29 +164,32 @@ 3.4.1 3.2.2 - 2.11.12 - 2.11 + 2.12.8 + 2.12 + --diff --test + + true 1.9.13 - 2.9.6 - 1.1.7.1 + 2.9.9 + 1.1.7.3 1.1.2 - 1.2.0-incubating 1.10 2.4 2.6 - 3.5 + 3.8.1 3.2.10 - 3.0.10 + 3.0.13 2.22.2 2.9.3 3.5.2 - 1.3.9 - 0.9.3 - 4.7 + 3.0.0 + 0.12.0 + 4.7.1 1.1 2.52.0 + 2.22 @@ -187,14 +200,14 @@ If you are changing Arrow version specification, please check ./python/pyspark/sql/utils.py, ./python/run-tests.py and ./python/setup.py too. --> - 0.10.0 + 0.12.0 ${java.home} - org.spark_project + org.sparkproject ${project.build.directory}/scala-${scala.binary.version}/jars @@ -213,6 +226,7 @@ --> compile compile + ${hive.deps.scope} compile compile test @@ -311,13 +325,13 @@ chill-java ${chill.version}
    - org.apache.xbean - xbean-asm6-shaded - 4.8 + xbean-asm7-shaded + 4.14 + + + + org.glassfish.jaxb + jaxb-runtime + 2.3.2 + runtime + + + + com.sun.xml.fastinfoset + FastInfoset + + + org.glassfish.jaxb + txw2 + + + org.jvnet.staxex + stax-ex + + + + jakarta.activation + jakarta.activation-api + + + org.apache.commons commons-lang3 ${commons-lang3.version} + + org.apache.commons + commons-text + 1.6 + commons-lang commons-lang @@ -432,6 +482,17 @@ commons-collections ${commons.collections.version} + + commons-beanutils + commons-beanutils + 1.9.3 + + + commons-logging + commons-logging + + 1.1.3 + org.apache.ivy ivy @@ -489,6 +550,19 @@ ${selenium.version} test + + + net.sourceforge.htmlunit + htmlunit + ${htmlunit.version} + test + + + net.sourceforge.htmlunit + htmlunit-core-js + ${htmlunit.version} + test + xml-apis @@ -538,17 +612,17 @@ org.lz4 lz4-java - 1.5.0 + 1.6.0 com.github.luben zstd-jni - 1.3.2-2 + 1.4.0-1 com.clearspring.analytics stream - 2.7.0 + 2.9.6 @@ -571,7 +645,7 @@ org.roaringbitmap RoaringBitmap - 0.5.11 + 0.7.45 commons-net @@ -581,7 +655,7 @@ io.netty netty-all - 4.1.17.Final + 4.1.30.Final io.netty @@ -706,7 +780,7 @@ org.json4s json4s-jackson_${scala.binary.version} - 3.5.3 + 3.6.6 com.fasterxml.jackson.core @@ -714,6 +788,11 @@ + + org.scala-lang.modules + scala-xml_${scala.binary.version} + 1.2.0 + org.scala-lang scala-compiler @@ -739,7 +818,6 @@ scala-parser-combinators_${scala.binary.version} 1.1.0 - jline jline @@ -748,13 +826,13 @@ org.scalatest scalatest_${scala.binary.version} - 3.0.3 + 3.0.5 test org.mockito mockito-core - 1.10.19 + 2.28.2 test @@ -818,7 +896,7 @@ org.postgresql postgresql - 9.4.1207.jre7 + 42.2.6 test @@ -875,6 +953,11 @@ org.jboss.netty netty + + + commons-beanutils + commons-beanutils-core + commons-logging commons-logging @@ -911,19 +994,22 @@ net.java.dev.jets3t jets3t + + + javax.ws.rs + jsr311-api + org.apache.avro avro ${avro.version} - ${hadoop.deps.scope} org.apache.avro avro-ipc ${avro.version} - ${hadoop.deps.scope} io.netty @@ -1153,6 +1239,11 @@ com.sun.jersey.contribs * + + + com.zaxxer + HikariCP-java7 + @@ -1375,15 +1466,44 @@ commons-logging commons-logging + + + + org.apache.orc + orc-core + + + + org.eclipse.jetty.aggregate + jetty-all + + + + org.apache.logging.log4j + * + + + + org.eclipse.jetty.orbit + javax.servlet + + + + org.apache.hive + hive-storage-api + + ${hive.group} hive-exec - + ${hive.classifier} ${hive.version} ${hive.deps.scope} @@ -1433,11 +1553,15 @@ org.apache.avro avro-mapred - + org.apache.calcite calcite-core + + org.apache.calcite + calcite-avatica + org.apache.curator apache-curator @@ -1491,6 +1615,27 @@ org.json json + + + + ${hive.group} + hive-llap-tez + + + + org.apache.calcite + calcite-druid + + + org.apache.calcite.avatica + avatica + + + + org.apache.logging.log4j + * + + @@ -1502,10 +1647,6 @@ ${hive.group} hive-common - - ${hive.group} - hive-common - ${hive.group} hive-metastore @@ -1599,6 +1740,17 @@ org.slf4j slf4j-log4j12 + + + + org.apache.hbase + hbase-client + + + co.cask.tephra + * + + @@ -1656,6 +1808,22 @@ org.codehaus.groovy groovy-all + + + + org.apache.parquet + parquet-hadoop-bundle + + + + tomcat + jasper-compiler + + + tomcat + jasper-runtime + + @@ -1721,8 +1889,145 @@ org.codehaus.groovy groovy-all + + + + org.apache.logging.log4j + log4j-slf4j-impl + + + +
    + + + + org.apache.hive + hive-llap-common + ${hive23.version} + ${hive.deps.scope} + + + org.apache.hive + hive-common + + + org.apache.hive + hive-serde + + + org.slf4j + slf4j-api + + + + + + org.apache.hive + hive-llap-client + ${hive23.version} + test + + + org.apache.hive + hive-common + + + org.apache.hive + hive-serde + + + org.apache.hive + hive-llap-common + + + org.apache.curator + curator-framework + + + org.apache.curator + apache-curator + + + org.apache.zookeeper + zookeeper + + + org.slf4j + slf4j-api + + + + ${hive.group} + hive-contrib + ${hive.version} + test + + + ${hive.group} + hive-exec + + + ${hive.group} + hive-serde + + + ${hive.group} + hive-shims + + + commons-codec + commons-codec + + + org.slf4j + slf4j-api + + + + + ${hive.group}.hcatalog + hive-hcatalog-core + ${hive.version} + test + + + ${hive.group} + hive-exec + + + ${hive.group} + hive-metastore + + + ${hive.group} + hive-cli + + + ${hive.group} + hive-common + + + com.google.guava + guava + + + org.slf4j + slf4j-api + + + org.codehaus.jackson + jackson-mapper-asl + + + org.apache.hadoop + * + + + + org.apache.orc orc-core @@ -1730,6 +2035,10 @@ ${orc.classifier} ${orc.deps.scope} + + javax.xml.bind + jaxb-api + org.apache.hadoop hadoop-common @@ -1798,75 +2107,10 @@ ${parquet.test.deps.scope} - com.twitter + ${hive.parquet.group} parquet-hadoop-bundle ${hive.parquet.version} - compile - - - org.apache.calcite - calcite-core - ${calcite.version} - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - com.google.guava - guava - - - com.google.code.findbugs - jsr305 - - - org.codehaus.janino - janino - - - org.codehaus.janino - commons-compiler - - - - org.hsqldb - hsqldb - - - org.pentaho - pentaho-aggdesigner-algorithm - - - - - org.apache.calcite - calcite-avatica - ${calcite.version} - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - + ${hive.parquet.scope} org.codehaus.janino @@ -1907,7 +2151,7 @@ org.apache.thrift libfb303 - ${libthrift.version} + 0.9.3 org.slf4j @@ -1935,6 +2179,7 @@ com.thoughtworks.paranamer paranamer ${paranamer.version} + runtime org.apache.arrow @@ -1972,7 +2217,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 3.0.0-M1 + 3.0.0-M2 enforce-versions @@ -1989,15 +2234,9 @@ - org.jboss.netty org.codehaus.groovy + *:*_2.11 *:*_2.10 true @@ -2015,8 +2254,7 @@ net.alchim31.maven scala-maven-plugin - - 3.2.2 + 3.4.4 eclipse-add-source @@ -2036,9 +2274,18 @@ testCompile + + attach-scaladocs + verify + + doc-jar + + ${scala.version} + true + true incremental true @@ -2047,6 +2294,7 @@ -feature -explaintypes -Yno-adapted-args + -target:jvm-1.8 -Xms1024m @@ -2065,7 +2313,7 @@ org.apache.maven.plugins maven-compiler-plugin - 3.7.0 + 3.8.0 ${java.version} ${java.version} @@ -2082,7 +2330,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.22.0 + 3.0.0-M3 @@ -2092,7 +2340,7 @@ **/*Suite.java ${project.build.directory}/surefire-reports - -ea -Xmx3g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx3g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + -ea -Xmx4g -Xss4m -XX:ReservedCodeCacheSize=${CodeCacheSize} + ${scalafmt.skip} + dev/.scalafmt.conf + + + + validate + + format + + + + @@ -2629,12 +2915,29 @@ - hadoop-3.1 + hadoop-3.2 - 3.1.0 - 2.12.0 - 3.4.9 + 3.2.0 + 2.13.0 + 3.4.13 + org.apache.hive + core + ${hive23.version} + 2.3.5 + + provided + + 4.1.17 + + + + org.apache.hive + hive-storage-api + 2.6.0 + + @@ -2656,29 +2959,28 @@ kubernetes resource-managers/kubernetes/core - resource-managers/kubernetes/integration-tests + - hive-thriftserver + kubernetes-integration-tests - sql/hive-thriftserver + resource-managers/kubernetes/integration-tests - hadoop-cloud + hive-thriftserver - hadoop-cloud + sql/hive-thriftserver - kafka-0-8 + hadoop-cloud - external/kafka-0-8 - external/kafka-0-8-assembly + hadoop-cloud @@ -2693,42 +2995,8 @@ - - scala-2.11 - - scala-2.12 - - 2.12.7 - 2.12 - - - - - org.apache.maven.plugins - maven-enforcer-plugin - - - enforce-versions - - enforce - - - - - - *:*_2.11 - *:*_2.10 - - - - - - - - - diff --git a/repl/src/main/scala-2.11/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala-2.11/org/apache/spark/repl/SparkExprTyper.scala deleted file mode 100644 index 724ce9af49f77..0000000000000 --- a/repl/src/main/scala-2.11/org/apache/spark/repl/SparkExprTyper.scala +++ /dev/null @@ -1,74 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.repl - -import scala.tools.nsc.interpreter.{ExprTyper, IR} - -trait SparkExprTyper extends ExprTyper { - - import repl._ - import global.{reporter => _, Import => _, _} - import naming.freshInternalVarName - - def doInterpret(code: String): IR.Result = { - // interpret/interpretSynthetic may change the phase, - // which would have unintended effects on types. - val savedPhase = phase - try interpretSynthetic(code) finally phase = savedPhase - } - - override def symbolOfLine(code: String): Symbol = { - def asExpr(): Symbol = { - val name = freshInternalVarName() - // Typing it with a lazy val would give us the right type, but runs - // into compiler bugs with things like existentials, so we compile it - // behind a def and strip the NullaryMethodType which wraps the expr. - val line = "def " + name + " = " + code - - doInterpret(line) match { - case IR.Success => - val sym0 = symbolOfTerm(name) - // drop NullaryMethodType - sym0.cloneSymbol setInfo exitingTyper(sym0.tpe_*.finalResultType) - case _ => NoSymbol - } - } - - def asDefn(): Symbol = { - val old = repl.definedSymbolList.toSet - - doInterpret(code) match { - case IR.Success => - repl.definedSymbolList filterNot old match { - case Nil => NoSymbol - case sym :: Nil => sym - case syms => NoSymbol.newOverloaded(NoPrefix, syms) - } - case _ => NoSymbol - } - } - - def asError(): Symbol = { - doInterpret(code) - NoSymbol - } - - beSilentDuring(asExpr()) orElse beSilentDuring(asDefn()) orElse asError() - } - -} diff --git a/repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala b/repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala deleted file mode 100644 index e736607a9a6b9..0000000000000 --- a/repl/src/main/scala-2.11/org/apache/spark/repl/SparkILoopInterpreter.scala +++ /dev/null @@ -1,239 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.repl - -import scala.collection.mutable -import scala.tools.nsc.Settings -import scala.tools.nsc.interpreter._ - -class SparkILoopInterpreter(settings: Settings, out: JPrintWriter) extends IMain(settings, out) { - self => - - override lazy val memberHandlers = new { - val intp: self.type = self - } with MemberHandlers { - import intp.global._ - - override def chooseHandler(member: intp.global.Tree): MemberHandler = member match { - case member: Import => new SparkImportHandler(member) - case _ => super.chooseHandler(member) - } - - class SparkImportHandler(imp: Import) extends ImportHandler(imp: Import) { - - override def targetType: Type = intp.global.rootMirror.getModuleIfDefined("" + expr) match { - case NoSymbol => intp.typeOfExpression("" + expr) - case sym => sym.tpe - } - - private def safeIndexOf(name: Name, s: String): Int = fixIndexOf(name, pos(name, s)) - private def fixIndexOf(name: Name, idx: Int): Int = if (idx == name.length) -1 else idx - private def pos(name: Name, s: String): Int = { - var i = name.pos(s.charAt(0), 0) - val sLen = s.length() - if (sLen == 1) return i - while (i + sLen <= name.length) { - var j = 1 - while (s.charAt(j) == name.charAt(i + j)) { - j += 1 - if (j == sLen) return i - } - i = name.pos(s.charAt(0), i + 1) - } - name.length - } - - private def isFlattenedSymbol(sym: Symbol): Boolean = - sym.owner.isPackageClass && - sym.name.containsName(nme.NAME_JOIN_STRING) && - sym.owner.info.member(sym.name.take( - safeIndexOf(sym.name, nme.NAME_JOIN_STRING))) != NoSymbol - - private def importableTargetMembers = - importableMembers(exitingTyper(targetType)).filterNot(isFlattenedSymbol).toList - - def isIndividualImport(s: ImportSelector): Boolean = - s.name != nme.WILDCARD && s.rename != nme.WILDCARD - def isWildcardImport(s: ImportSelector): Boolean = - s.name == nme.WILDCARD - - // non-wildcard imports - private def individualSelectors = selectors filter isIndividualImport - - override val importsWildcard: Boolean = selectors exists isWildcardImport - - lazy val importableSymbolsWithRenames: List[(Symbol, Name)] = { - val selectorRenameMap = - individualSelectors.flatMap(x => x.name.bothNames zip x.rename.bothNames).toMap - importableTargetMembers flatMap (m => selectorRenameMap.get(m.name) map (m -> _)) - } - - override lazy val individualSymbols: List[Symbol] = importableSymbolsWithRenames map (_._1) - override lazy val wildcardSymbols: List[Symbol] = - if (importsWildcard) importableTargetMembers else Nil - - } - - } - - object expressionTyper extends { - val repl: SparkILoopInterpreter.this.type = self - } with SparkExprTyper { } - - override def symbolOfLine(code: String): global.Symbol = - expressionTyper.symbolOfLine(code) - - override def typeOfExpression(expr: String, silent: Boolean): global.Type = - expressionTyper.typeOfExpression(expr, silent) - - - import global.Name - override def importsCode(wanted: Set[Name], wrapper: Request#Wrapper, - definesClass: Boolean, generousImports: Boolean): ComputedImports = { - - import global._ - import definitions.{ ObjectClass, ScalaPackage, JavaLangPackage, PredefModule } - import memberHandlers._ - - val header, code, trailingBraces, accessPath = new StringBuilder - val currentImps = mutable.HashSet[Name]() - // only emit predef import header if name not resolved in history, loosely - var predefEscapes = false - - /** - * Narrow down the list of requests from which imports - * should be taken. Removes requests which cannot contribute - * useful imports for the specified set of wanted names. - */ - case class ReqAndHandler(req: Request, handler: MemberHandler) - - def reqsToUse: List[ReqAndHandler] = { - /** - * Loop through a list of MemberHandlers and select which ones to keep. - * 'wanted' is the set of names that need to be imported. - */ - def select(reqs: List[ReqAndHandler], wanted: Set[Name]): List[ReqAndHandler] = { - // Single symbol imports might be implicits! See bug #1752. Rather than - // try to finesse this, we will mimic all imports for now. - def keepHandler(handler: MemberHandler) = handler match { - // While defining classes in class based mode - implicits are not needed. - case h: ImportHandler if isClassBased && definesClass => - h.importedNames.exists(x => wanted.contains(x)) - case _: ImportHandler => true - case x if generousImports => x.definesImplicit || - (x.definedNames exists (d => wanted.exists(w => d.startsWith(w)))) - case x => x.definesImplicit || - (x.definedNames exists wanted) - } - - reqs match { - case Nil => - predefEscapes = wanted contains PredefModule.name ; Nil - case rh :: rest if !keepHandler(rh.handler) => select(rest, wanted) - case rh :: rest => - import rh.handler._ - val augment = rh match { - case ReqAndHandler(_, _: ImportHandler) => referencedNames - case _ => Nil - } - val newWanted = wanted ++ augment -- definedNames -- importedNames - rh :: select(rest, newWanted) - } - } - - /** Flatten the handlers out and pair each with the original request */ - select(allReqAndHandlers reverseMap { case (r, h) => ReqAndHandler(r, h) }, wanted).reverse - } - - // add code for a new object to hold some imports - def addWrapper() { - import nme.{ INTERPRETER_IMPORT_WRAPPER => iw } - code append (wrapper.prewrap format iw) - trailingBraces append wrapper.postwrap - accessPath append s".$iw" - currentImps.clear() - } - - def maybeWrap(names: Name*) = if (names exists currentImps) addWrapper() - - def wrapBeforeAndAfter[T](op: => T): T = { - addWrapper() - try op finally addWrapper() - } - - // imports from Predef are relocated to the template header to allow hiding. - def checkHeader(h: ImportHandler) = h.referencedNames contains PredefModule.name - - // loop through previous requests, adding imports for each one - wrapBeforeAndAfter { - // Reusing a single temporary value when import from a line with multiple definitions. - val tempValLines = mutable.Set[Int]() - for (ReqAndHandler(req, handler) <- reqsToUse) { - val objName = req.lineRep.readPathInstance - handler match { - case h: ImportHandler if checkHeader(h) => - header.clear() - header append f"${h.member}%n" - // If the user entered an import, then just use it; add an import wrapping - // level if the import might conflict with some other import - case x: ImportHandler if x.importsWildcard => - wrapBeforeAndAfter(code append (x.member + "\n")) - case x: ImportHandler => - maybeWrap(x.importedNames: _*) - code append (x.member + "\n") - currentImps ++= x.importedNames - - case x if isClassBased => - for (sym <- x.definedSymbols) { - maybeWrap(sym.name) - x match { - case _: ClassHandler => - code.append(s"import ${objName}${req.accessPath}.`${sym.name}`\n") - case _ => - val valName = s"${req.lineRep.packageName}${req.lineRep.readName}" - if (!tempValLines.contains(req.lineRep.lineId)) { - code.append(s"val $valName: ${objName}.type = $objName\n") - tempValLines += req.lineRep.lineId - } - code.append(s"import ${valName}${req.accessPath}.`${sym.name}`\n") - } - currentImps += sym.name - } - // For other requests, import each defined name. - // import them explicitly instead of with _, so that - // ambiguity errors will not be generated. Also, quote - // the name of the variable, so that we don't need to - // handle quoting keywords separately. - case x => - for (sym <- x.definedSymbols) { - maybeWrap(sym.name) - code append s"import ${x.path}\n" - currentImps += sym.name - } - } - } - } - - val computedHeader = if (predefEscapes) header.toString else "" - ComputedImports(computedHeader, code.toString, trailingBraces.toString, accessPath.toString) - } - - private def allReqAndHandlers = - prevRequestList flatMap (req => req.handlers map (req -> _)) - -} diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 88eb0ad1da3d7..0cfd96193d0fd 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -21,9 +21,11 @@ import java.io.{ByteArrayOutputStream, FileNotFoundException, FilterInputStream, import java.net.{URI, URL, URLEncoder} import java.nio.channels.Channels +import scala.util.control.NonFatal + import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.xbean.asm6._ -import org.apache.xbean.asm6.Opcodes._ +import org.apache.xbean.asm7._ +import org.apache.xbean.asm7.Opcodes._ import org.apache.spark.{SparkConf, SparkEnv} import org.apache.spark.deploy.SparkHadoopUtil @@ -33,8 +35,11 @@ import org.apache.spark.util.ParentClassLoader /** * A ClassLoader that reads classes from a Hadoop FileSystem or Spark RPC endpoint, used to load * classes defined by the interpreter when the REPL is used. Allows the user to specify if user - * class path should be first. This class loader delegates getting/finding resources to parent - * loader, which makes sense until REPL never provide resource dynamically. + * class path should be first. + * This class loader delegates getting/finding resources to parent loader, which makes sense because + * the REPL never produce resources dynamically. One exception is when getting a Class file as + * resource stream, in which case we will try to fetch the Class file in the same way as loading + * the class, so that dynamically generated Classes from the REPL can be picked up. * * Note: [[ClassLoader]] will preferentially load class from parent. Only when parent is null or * the load failed, that it will call the overridden `findClass` function. To avoid the potential @@ -71,6 +76,30 @@ class ExecutorClassLoader( parentLoader.getResources(name) } + override def getResourceAsStream(name: String): InputStream = { + if (userClassPathFirst) { + val res = getClassResourceAsStreamLocally(name) + if (res != null) res else parentLoader.getResourceAsStream(name) + } else { + val res = parentLoader.getResourceAsStream(name) + if (res != null) res else getClassResourceAsStreamLocally(name) + } + } + + private def getClassResourceAsStreamLocally(name: String): InputStream = { + // Class files can be dynamically generated from the REPL. Allow this class loader to + // load such files for purposes other than loading the class. + try { + if (name.endsWith(".class")) fetchFn(name) else null + } catch { + // The helper functions referenced by fetchFn throw CNFE to indicate failure to fetch + // the class. It matches what IOException was supposed to be used for, and + // ClassLoader.getResourceAsStream() catches IOException and returns null in that case. + // So we follow that model and handle CNFE here. + case _: ClassNotFoundException => null + } + } + override def findClass(name: String): Class[_] = { if (userClassPathFirst) { findClassLocally(name).getOrElse(parentLoader.loadClass(name)) @@ -79,7 +108,17 @@ class ExecutorClassLoader( parentLoader.loadClass(name) } catch { case e: ClassNotFoundException => - val classOption = findClassLocally(name) + val classOption = try { + findClassLocally(name) + } catch { + case e: RemoteClassLoaderError => + throw e + case NonFatal(e) => + // Wrap the error to include the class name + // scalastyle:off throwerror + throw new RemoteClassLoaderError(name, e) + // scalastyle:on throwerror + } classOption match { case None => throw new ClassNotFoundException(name, e) case Some(a) => a @@ -88,14 +127,15 @@ class ExecutorClassLoader( } } + // See org.apache.spark.network.server.TransportRequestHandler.processStreamRequest. + private val STREAM_NOT_FOUND_REGEX = s"Stream '.*' was not found.".r.pattern + private def getClassFileInputStreamFromSparkRPC(path: String): InputStream = { - val channel = env.rpcEnv.openChannel(s"$classUri/$path") + val channel = env.rpcEnv.openChannel(s"$classUri/${urlEncode(path)}") new FilterInputStream(Channels.newInputStream(channel)) { override def read(): Int = toClassNotFound(super.read()) - override def read(b: Array[Byte]): Int = toClassNotFound(super.read(b)) - override def read(b: Array[Byte], offset: Int, len: Int) = toClassNotFound(super.read(b, offset, len)) @@ -103,8 +143,15 @@ class ExecutorClassLoader( try { fn } catch { - case e: Exception => + case e: RuntimeException if e.getMessage != null + && STREAM_NOT_FOUND_REGEX.matcher(e.getMessage).matches() => + // Convert a stream not found error to ClassNotFoundException. + // Driver sends this explicit acknowledgment to tell us that the class was missing. throw new ClassNotFoundException(path, e) + case NonFatal(e) => + // scalastyle:off throwerror + throw new RemoteClassLoaderError(path, e) + // scalastyle:on throwerror } } } @@ -136,7 +183,12 @@ class ExecutorClassLoader( case e: Exception => // Something bad happened while checking if the class exists logError(s"Failed to check existence of class $name on REPL class server at $uri", e) - None + if (userClassPathFirst) { + // Allow to try to load from "parentLoader" + None + } else { + throw e + } } finally { if (inputStream != null) { try { @@ -187,7 +239,7 @@ class ExecutorClassLoader( } class ConstructorCleaner(className: String, cv: ClassVisitor) -extends ClassVisitor(ASM6, cv) { +extends ClassVisitor(ASM7, cv) { override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { val mv = cv.visitMethod(access, name, desc, sig, exceptions) @@ -210,3 +262,11 @@ extends ClassVisitor(ASM6, cv) { } } } + +/** + * An error when we cannot load a class due to exceptions. We don't know if this class exists, so + * throw a special one that's neither [[LinkageError]] nor [[ClassNotFoundException]] to make JVM + * retry to load this class later. + */ +private[repl] class RemoteClassLoaderError(className: String, cause: Throwable) + extends Error(className, cause) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index aa9aa2793b8b3..ed3c672e7dafa 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -32,7 +32,7 @@ import scala.tools.nsc.interpreter.{AbstractOrMissingHandler, ILoop, IMain, JPri import scala.tools.nsc.interpreter.{NamedParam, SimpleReader, SplashLoop, SplashReader} import scala.tools.nsc.interpreter.StdReplTags.tagOfIMain import scala.tools.nsc.util.stringFromStream -import scala.util.Properties.{javaVersion, javaVmName, versionNumberString, versionString} +import scala.util.Properties.{javaVersion, javaVmName, versionString} /** * A Spark-specific interactive shell. @@ -42,32 +42,6 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) def this(in0: BufferedReader, out: JPrintWriter) = this(Some(in0), out) def this() = this(None, new JPrintWriter(Console.out, true)) - /** - * TODO: Remove the following `override` when the support of Scala 2.11 is ended - * Scala 2.11 has a bug of finding imported types in class constructors, extends clause - * which is fixed in Scala 2.12 but never be back-ported into Scala 2.11.x. - * As a result, we copied the fixes into `SparkILoopInterpreter`. See SPARK-22393 for detail. - */ - override def createInterpreter(): Unit = { - if (isScala2_11) { - if (addedClasspath != "") { - settings.classpath append addedClasspath - } - // scalastyle:off classforname - // Have to use the default classloader to match the one used in - // `classOf[Settings]` and `classOf[JPrintWriter]`. - intp = Class.forName("org.apache.spark.repl.SparkILoopInterpreter") - .getDeclaredConstructor(Seq(classOf[Settings], classOf[JPrintWriter]): _*) - .newInstance(Seq(settings, out): _*) - .asInstanceOf[IMain] - // scalastyle:on classforname - } else { - super.createInterpreter() - } - } - - private val isScala2_11 = versionNumberString.startsWith("2.11") - val initializationCommands: Seq[String] = Seq( """ @transient val spark = if (org.apache.spark.repl.Main.sparkSession != null) { @@ -145,26 +119,6 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) super.replay() } - /** - * TODO: Remove `runClosure` when the support of Scala 2.11 is ended - */ - private def runClosure(body: => Boolean): Boolean = { - if (isScala2_11) { - // In Scala 2.11, there is a bug that interpret could set the current thread's - // context classloader, but fails to reset it to its previous state when returning - // from that method. This is fixed in SI-8521 https://github.com/scala/scala/pull/5657 - // which is never back-ported into Scala 2.11.x. The following is a workaround fix. - val original = Thread.currentThread().getContextClassLoader - try { - body - } finally { - Thread.currentThread().setContextClassLoader(original) - } - } else { - body - } - } - /** * The following code is mostly a copy of `process` implementation in `ILoop.scala` in Scala * @@ -179,7 +133,7 @@ class SparkILoop(in0: Option[BufferedReader], out: JPrintWriter) * We should remove this duplication once Scala provides a way to load our custom initialization * code, and also customize the ordering of printing welcome message. */ - override def process(settings: Settings): Boolean = runClosure { + override def process(settings: Settings): Boolean = { def newReader = in0.fold(chooseReader(settings))(r => SimpleReader(r, out, interactive = true)) diff --git a/repl/src/test/resources/log4j.properties b/repl/src/test/resources/log4j.properties index 7665bd5e7c070..71580b7d8b1a4 100644 --- a/repl/src/test/resources/log4j.properties +++ b/repl/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala index e5e2094368fb0..dceae13fd8360 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -17,7 +17,8 @@ package org.apache.spark.repl -import java.io.File +import java.io.{File, IOException} +import java.lang.reflect.InvocationTargetException import java.net.{URI, URL, URLClassLoader} import java.nio.channels.{FileChannel, ReadableByteChannel} import java.nio.charset.StandardCharsets @@ -30,7 +31,7 @@ import scala.io.Source import scala.language.implicitConversions import com.google.common.io.Files -import org.mockito.Matchers.anyString +import org.mockito.ArgumentMatchers.{any, anyString} import org.mockito.Mockito._ import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer @@ -38,6 +39,7 @@ import org.scalatest.BeforeAndAfterAll import org.scalatest.mockito.MockitoSugar import org.apache.spark._ +import org.apache.spark.TestUtils.JavaSourceFromString import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.Utils @@ -126,7 +128,7 @@ class ExecutorClassLoaderSuite test("child first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) - val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass2").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") } @@ -134,7 +136,7 @@ class ExecutorClassLoaderSuite test("parent first") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, false) - val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass1").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") } @@ -142,7 +144,7 @@ class ExecutorClassLoaderSuite test("child first can fall back") { val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) - val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass3").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "2") } @@ -151,7 +153,7 @@ class ExecutorClassLoaderSuite val parentLoader = new URLClassLoader(urls2, null) val classLoader = new ExecutorClassLoader(new SparkConf(), null, url1, parentLoader, true) intercept[java.lang.ClassNotFoundException] { - classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + classLoader.loadClass("ReplFakeClassDoesNotExist").getConstructor().newInstance() } } @@ -191,23 +193,166 @@ class ExecutorClassLoaderSuite val env = mock[SparkEnv] val rpcEnv = mock[RpcEnv] when(env.rpcEnv).thenReturn(rpcEnv) - when(rpcEnv.openChannel(anyString())).thenAnswer(new Answer[ReadableByteChannel]() { - override def answer(invocation: InvocationOnMock): ReadableByteChannel = { - val uri = new URI(invocation.getArguments()(0).asInstanceOf[String]) - val path = Paths.get(tempDir1.getAbsolutePath(), uri.getPath().stripPrefix("/")) + when(rpcEnv.openChannel(anyString())).thenAnswer((invocation: InvocationOnMock) => { + val uri = new URI(invocation.getArguments()(0).asInstanceOf[String]) + val path = Paths.get(tempDir1.getAbsolutePath(), uri.getPath().stripPrefix("/")) + if (path.toFile.exists()) { FileChannel.open(path, StandardOpenOption.READ) + } else { + val channel = mock[ReadableByteChannel] + when(channel.read(any())) + .thenThrow(new RuntimeException(s"Stream '${uri.getPath}' was not found.")) + channel } }) val classLoader = new ExecutorClassLoader(new SparkConf(), env, "spark://localhost:1234", getClass().getClassLoader(), false) - val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClass = classLoader.loadClass("ReplFakeClass2").getConstructor().newInstance() val fakeClassVersion = fakeClass.toString assert(fakeClassVersion === "1") intercept[java.lang.ClassNotFoundException] { - classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + classLoader.loadClass("ReplFakeClassDoesNotExist").getConstructor().newInstance() + } + + // classLoader.getResourceAsStream() should also be able to fetch the Class file + val fakeClassInputStream = classLoader.getResourceAsStream("ReplFakeClass2.class") + try { + val magic = new Array[Byte](4) + fakeClassInputStream.read(magic) + // first 4 bytes should match the magic number of Class file + assert(magic === Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte)) + } finally { + if (fakeClassInputStream != null) fakeClassInputStream.close() } } + test("nonexistent class and transient errors should cause different errors") { + val conf = new SparkConf() + .setMaster("local") + .setAppName("executor-class-loader-test") + .set("spark.network.timeout", "11s") + .set("spark.repl.class.outputDir", tempDir1.getAbsolutePath) + val sc = new SparkContext(conf) + try { + val replClassUri = sc.conf.get("spark.repl.class.uri") + + // Create an RpcEnv for executor + val rpcEnv = RpcEnv.create( + SparkEnv.executorSystemName, + "localhost", + "localhost", + 0, + sc.conf, + new SecurityManager(conf), 0, clientMode = true) + + try { + val env = mock[SparkEnv] + when(env.rpcEnv).thenReturn(rpcEnv) + + val classLoader = new ExecutorClassLoader( + conf, + env, + replClassUri, + getClass().getClassLoader(), + false) + + // Test loading a nonexistent class + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("NonexistentClass") + } + + // Stop SparkContext to simulate transient errors in executors + sc.stop() + + val e = intercept[RemoteClassLoaderError] { + classLoader.loadClass("ThisIsAClassName") + } + assert(e.getMessage.contains("ThisIsAClassName")) + // RemoteClassLoaderError must not be LinkageError nor ClassNotFoundException. Otherwise, + // JVM will cache it and doesn't retry to load a class. + assert(!e.isInstanceOf[LinkageError] && !e.isInstanceOf[ClassNotFoundException]) + } finally { + rpcEnv.shutdown() + rpcEnv.awaitTermination() + } + } finally { + sc.stop() + } + } + + test("SPARK-20547 ExecutorClassLoader should not throw ClassNotFoundException without " + + "acknowledgment from driver") { + val tempDir = Utils.createTempDir() + try { + // Create two classes, "TestClassB" calls "TestClassA", so when calling "TestClassB.foo", JVM + // will try to load "TestClassA". + val sourceCodeOfClassA = + """public class TestClassA implements java.io.Serializable { + | @Override public String toString() { return "TestClassA"; } + |}""".stripMargin + val sourceFileA = new JavaSourceFromString("TestClassA", sourceCodeOfClassA) + TestUtils.createCompiledClass( + sourceFileA.name, tempDir, sourceFileA, Seq(tempDir.toURI.toURL)) + + val sourceCodeOfClassB = + """public class TestClassB implements java.io.Serializable { + | public String foo() { return new TestClassA().toString(); } + | @Override public String toString() { return "TestClassB"; } + |}""".stripMargin + val sourceFileB = new JavaSourceFromString("TestClassB", sourceCodeOfClassB) + TestUtils.createCompiledClass( + sourceFileB.name, tempDir, sourceFileB, Seq(tempDir.toURI.toURL)) + + val env = mock[SparkEnv] + val rpcEnv = mock[RpcEnv] + when(env.rpcEnv).thenReturn(rpcEnv) + when(rpcEnv.openChannel(anyString())).thenAnswer(new Answer[ReadableByteChannel]() { + private var count = 0 + + override def answer(invocation: InvocationOnMock): ReadableByteChannel = { + val uri = new URI(invocation.getArguments()(0).asInstanceOf[String]) + val classFileName = uri.getPath().stripPrefix("/") + if (count == 0 && classFileName == "TestClassA.class") { + count += 1 + // Let the first attempt to load TestClassA fail with an IOException + val channel = mock[ReadableByteChannel] + when(channel.read(any())).thenThrow(new IOException("broken pipe")) + channel + } + else { + val path = Paths.get(tempDir.getAbsolutePath(), classFileName) + FileChannel.open(path, StandardOpenOption.READ) + } + } + }) + + val classLoader = new ExecutorClassLoader(new SparkConf(), env, "spark://localhost:1234", + getClass().getClassLoader(), false) + + def callClassBFoo(): String = { + // scalastyle:off classforname + val classB = Class.forName("TestClassB", true, classLoader) + // scalastyle:on classforname + val instanceOfTestClassB = classB.newInstance() + assert(instanceOfTestClassB.toString === "TestClassB") + classB.getMethod("foo").invoke(instanceOfTestClassB).asInstanceOf[String] + } + + // Reflection will wrap the exception with InvocationTargetException + val e = intercept[InvocationTargetException] { + callClassBFoo() + } + // "TestClassA" cannot be loaded because of IOException + assert(e.getCause.isInstanceOf[RemoteClassLoaderError]) + assert(e.getCause.getCause.isInstanceOf[IOException]) + assert(e.getCause.getMessage.contains("TestClassA")) + + // We should be able to re-load TestClassA for IOException + assert(callClassBFoo() === "TestClassA") + } finally { + Utils.deleteRecursively(tempDir) + } + } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4f3df729177fb..38e3fc4f93ae0 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -18,41 +18,45 @@ package org.apache.spark.repl import java.io._ -import java.net.URLClassLoader -import scala.collection.mutable.ArrayBuffer import scala.tools.nsc.interpreter.SimpleReader import org.apache.log4j.{Level, LogManager} +import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkContext, SparkFunSuite} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION -class ReplSuite extends SparkFunSuite { +class ReplSuite extends SparkFunSuite with BeforeAndAfterAll { - def runInterpreter(master: String, input: String): String = { - val CONF_EXECUTOR_CLASSPATH = "spark.executor.extraClassPath" + private var originalClassLoader: ClassLoader = null - val in = new BufferedReader(new StringReader(input + "\n")) - val out = new StringWriter() - val cl = getClass.getClassLoader - var paths = new ArrayBuffer[String] - if (cl.isInstanceOf[URLClassLoader]) { - val urlLoader = cl.asInstanceOf[URLClassLoader] - for (url <- urlLoader.getURLs) { - if (url.getProtocol == "file") { - paths += url.getFile - } - } + override def beforeAll(): Unit = { + originalClassLoader = Thread.currentThread().getContextClassLoader + } + + override def afterAll(): Unit = { + if (originalClassLoader != null) { + // Reset the class loader to not affect other suites. REPL will set its own class loader but + // doesn't reset it. + Thread.currentThread().setContextClassLoader(originalClassLoader) } - val classpath = paths.map(new File(_).getAbsolutePath).mkString(File.pathSeparator) + } + + def runInterpreter(master: String, input: String): String = { + val CONF_EXECUTOR_CLASSPATH = "spark.executor.extraClassPath" val oldExecutorClasspath = System.getProperty(CONF_EXECUTOR_CLASSPATH) + val classpath = System.getProperty("java.class.path") System.setProperty(CONF_EXECUTOR_CLASSPATH, classpath) + Main.sparkContext = null Main.sparkSession = null // causes recreation of SparkContext for each test. Main.conf.set("spark.master", master) + + val in = new BufferedReader(new StringReader(input + "\n")) + val out = new StringWriter() Main.doMain(Array("-classpath", classpath), new SparkILoop(in, new PrintWriter(out))) if (oldExecutorClasspath != null) { @@ -60,7 +64,8 @@ class ReplSuite extends SparkFunSuite { } else { System.clearProperty(CONF_EXECUTOR_CLASSPATH) } - return out.toString + + out.toString } // Simulate the paste mode in Scala REPL. @@ -260,4 +265,36 @@ class ReplSuite extends SparkFunSuite { assertContains("!!2!!", output2) } + test("SPARK-26633: ExecutorClassLoader.getResourceAsStream find REPL classes") { + val output = runInterpreterInPasteMode("local-cluster[1,1,1024]", + """ + |case class TestClass(value: Int) + | + |sc.parallelize(1 to 1).map { _ => + | val clz = classOf[TestClass] + | val name = clz.getName.replace('.', '/') + ".class"; + | val stream = clz.getClassLoader.getResourceAsStream(name) + | if (stream == null) { + | "failed: stream is null" + | } else { + | val magic = new Array[Byte](4) + | try { + | stream.read(magic) + | // the magic number of a Java Class file + | val expected = Array[Byte](0xCA.toByte, 0xFE.toByte, 0xBA.toByte, 0xBE.toByte) + | if (magic sameElements expected) { + | "successful" + | } else { + | "failed: unexpected contents from stream" + | } + | } finally { + | stream.close() + | } + | } + |}.collect() + """.stripMargin) + assertDoesNotContain("failed", output) + assertContains("successful", output) + } + } diff --git a/repl/src/test/scala/org/apache/spark/repl/SingletonReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/SingletonReplSuite.scala index d49e0fd85229f..777de967b6a13 100644 --- a/repl/src/test/scala/org/apache/spark/repl/SingletonReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/SingletonReplSuite.scala @@ -18,9 +18,6 @@ package org.apache.spark.repl import java.io._ -import java.net.URLClassLoader - -import scala.collection.mutable.ArrayBuffer import org.apache.commons.lang3.StringEscapeUtils @@ -42,19 +39,9 @@ class SingletonReplSuite extends SparkFunSuite { override def beforeAll(): Unit = { super.beforeAll() - val cl = getClass.getClassLoader - var paths = new ArrayBuffer[String] - if (cl.isInstanceOf[URLClassLoader]) { - val urlLoader = cl.asInstanceOf[URLClassLoader] - for (url <- urlLoader.getURLs) { - if (url.getProtocol == "file") { - paths += url.getFile - } - } - } - val classpath = paths.map(new File(_).getAbsolutePath).mkString(File.pathSeparator) - + val classpath = System.getProperty("java.class.path") System.setProperty(CONF_EXECUTOR_CLASSPATH, classpath) + Main.conf.set("spark.master", "local-cluster[2,1,1024]") val interp = new SparkILoop( new BufferedReader(new InputStreamReader(new PipedInputStream(in))), @@ -65,9 +52,7 @@ class SingletonReplSuite extends SparkFunSuite { Main.sparkSession = null // Starts a new thread to run the REPL interpreter, so that we won't block. - thread = new Thread(new Runnable { - override def run(): Unit = Main.doMain(Array("-classpath", classpath), interp) - }) + thread = new Thread(() => Main.doMain(Array("-classpath", classpath), interp)) thread.setDaemon(true) thread.start() @@ -405,4 +390,20 @@ class SingletonReplSuite extends SparkFunSuite { assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) } + + test("create encoder in executors") { + val output = runInterpreter( + """ + |case class Foo(s: String) + | + |import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder + | + |val r = + | sc.parallelize(1 to 1).map { i => ExpressionEncoder[Foo](); Foo("bar") }.collect.head + """.stripMargin) + + assertContains("r: Foo = Foo(bar)", output) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + } } diff --git a/resource-managers/kubernetes/core/pom.xml b/resource-managers/kubernetes/core/pom.xml index 90bac19cba019..23106cb7ec68f 100644 --- a/resource-managers/kubernetes/core/pom.xml +++ b/resource-managers/kubernetes/core/pom.xml @@ -19,17 +19,17 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../../pom.xml - spark-kubernetes_2.11 + spark-kubernetes_2.12 jar Spark Project Kubernetes kubernetes - 3.0.0 + 4.1.2 diff --git a/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.SparkSubmitOperation b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.SparkSubmitOperation new file mode 100644 index 0000000000000..d589e6b60f847 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/resources/META-INF/services/org.apache.spark.deploy.SparkSubmitOperation @@ -0,0 +1 @@ +org.apache.spark.deploy.k8s.submit.K8SSparkSubmitOperation \ No newline at end of file diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala index 71e4d321a0e3a..1af8dfbe4385a 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Config.scala @@ -18,11 +18,24 @@ package org.apache.spark.deploy.k8s import java.util.concurrent.TimeUnit +import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigBuilder private[spark] object Config extends Logging { + val KUBERNETES_CONTEXT = + ConfigBuilder("spark.kubernetes.context") + .doc("The desired context from your K8S config file used to configure the K8S " + + "client for interacting with the cluster. Useful if your config file has " + + "multiple clusters or user identities defined. The client library used " + + "locates the config file via the KUBECONFIG environment variable or by defaulting " + + "to .kube/config under your home directory. If not specified then your current " + + "context is used. You can always override specific aspects of the config file " + + "provided configuration using other Spark on K8S configuration options.") + .stringConf + .createOptional + val KUBERNETES_NAMESPACE = ConfigBuilder("spark.kubernetes.namespace") .doc("The namespace that will be used for running the driver and executor pods.") @@ -59,12 +72,12 @@ private[spark] object Config extends Logging { .doc("Comma separated list of the Kubernetes secrets used " + "to access private image registries.") .stringConf - .createOptional + .toSequence + .createWithDefault(Nil) - val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = - "spark.kubernetes.authenticate.driver" - val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = - "spark.kubernetes.authenticate.driver.mounted" + val KUBERNETES_AUTH_DRIVER_CONF_PREFIX = "spark.kubernetes.authenticate.driver" + val KUBERNETES_AUTH_EXECUTOR_CONF_PREFIX = "spark.kubernetes.authenticate.executor" + val KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX = "spark.kubernetes.authenticate.driver.mounted" val KUBERNETES_AUTH_CLIENT_MODE_PREFIX = "spark.kubernetes.authenticate" val OAUTH_TOKEN_CONF_SUFFIX = "oauthToken" val OAUTH_TOKEN_FILE_CONF_SUFFIX = "oauthTokenFile" @@ -72,6 +85,30 @@ private[spark] object Config extends Logging { val CLIENT_CERT_FILE_CONF_SUFFIX = "clientCertFile" val CA_CERT_FILE_CONF_SUFFIX = "caCertFile" + val SUBMISSION_CLIENT_REQUEST_TIMEOUT = + ConfigBuilder("spark.kubernetes.submission.requestTimeout") + .doc("request timeout to be used in milliseconds for starting the driver") + .intConf + .createWithDefault(10000) + + val SUBMISSION_CLIENT_CONNECTION_TIMEOUT = + ConfigBuilder("spark.kubernetes.submission.connectionTimeout") + .doc("connection timeout to be used in milliseconds for starting the driver") + .intConf + .createWithDefault(10000) + + val DRIVER_CLIENT_REQUEST_TIMEOUT = + ConfigBuilder("spark.kubernetes.driver.requestTimeout") + .doc("request timeout to be used in milliseconds for driver to request executors") + .intConf + .createWithDefault(10000) + + val DRIVER_CLIENT_CONNECTION_TIMEOUT = + ConfigBuilder("spark.kubernetes.driver.connectionTimeout") + .doc("connection timeout to be used in milliseconds for driver to request executors") + .intConf + .createWithDefault(10000) + val KUBERNETES_SERVICE_ACCOUNT_NAME = ConfigBuilder(s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.serviceAccountName") .doc("Service account that is used when running the driver pod. The driver pod uses " + @@ -87,6 +124,12 @@ private[spark] object Config extends Logging { .stringConf .createOptional + val KUBERNETES_DRIVER_REQUEST_CORES = + ConfigBuilder("spark.kubernetes.driver.request.cores") + .doc("Specify the cpu request for the driver pod") + .stringConf + .createOptional + val KUBERNETES_DRIVER_SUBMIT_CHECK = ConfigBuilder("spark.kubernetes.submitInDriver") .internal() @@ -111,44 +154,16 @@ private[spark] object Config extends Logging { .stringConf .createOptional - val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = - ConfigBuilder("spark.kubernetes.executor.podNamePrefix") - .doc("Prefix to use in front of the executor pod names.") - .internal() - .stringConf - .createWithDefault("spark") - - val KUBERNETES_PYSPARK_PY_FILES = - ConfigBuilder("spark.kubernetes.python.pyFiles") - .doc("The PyFiles that are distributed via client arguments") - .internal() - .stringConf - .createOptional - - val KUBERNETES_PYSPARK_MAIN_APP_RESOURCE = - ConfigBuilder("spark.kubernetes.python.mainAppResource") - .doc("The main app resource for pyspark jobs") - .internal() - .stringConf - .createOptional - - val KUBERNETES_PYSPARK_APP_ARGS = - ConfigBuilder("spark.kubernetes.python.appArgs") - .doc("The app arguments for PySpark Jobs") + // For testing only. + val KUBERNETES_DRIVER_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.driver.resourceNamePrefix") .internal() .stringConf .createOptional - val KUBERNETES_R_MAIN_APP_RESOURCE = - ConfigBuilder("spark.kubernetes.r.mainAppResource") - .doc("The main app resource for SparkR jobs") - .internal() - .stringConf - .createOptional - - val KUBERNETES_R_APP_ARGS = - ConfigBuilder("spark.kubernetes.r.appArgs") - .doc("The app arguments for SparkR Jobs") + val KUBERNETES_EXECUTOR_POD_NAME_PREFIX = + ConfigBuilder("spark.kubernetes.executor.podNamePrefix") + .doc("Prefix to use in front of the executor pod names.") .internal() .stringConf .createOptional @@ -223,13 +238,51 @@ private[spark] object Config extends Logging { .stringConf .checkValue(pv => List("2", "3").contains(pv), "Ensure that major Python version is either Python2 or Python3") - .createWithDefault("2") + .createWithDefault("3") + + val KUBERNETES_KERBEROS_KRB5_FILE = + ConfigBuilder("spark.kubernetes.kerberos.krb5.path") + .doc("Specify the local location of the krb5.conf file to be mounted on the driver " + + "and executors for Kerberos. Note: The KDC defined needs to be " + + "visible from inside the containers ") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_KRB5_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.kerberos.krb5.configMapName") + .doc("Specify the name of the ConfigMap, containing the krb5.conf file, to be mounted " + + "on the driver and executors for Kerberos. Note: The KDC defined" + + "needs to be visible from inside the containers ") + .stringConf + .createOptional + + val KUBERNETES_HADOOP_CONF_CONFIG_MAP = + ConfigBuilder("spark.kubernetes.hadoop.configMapName") + .doc("Specify the name of the ConfigMap, containing the HADOOP_CONF_DIR files, " + + "to be mounted on the driver and executors for custom Hadoop configuration.") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_DT_SECRET_NAME = + ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.name") + .doc("Specify the name of the secret where your existing delegation tokens are stored. " + + "This removes the need for the job user to provide any keytab for launching a job") + .stringConf + .createOptional + + val KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY = + ConfigBuilder("spark.kubernetes.kerberos.tokenSecret.itemKey") + .doc("Specify the item key of the data where your existing delegation tokens are stored. " + + "This removes the need for the job user to provide any keytab for launching a job") + .stringConf + .createOptional val APP_RESOURCE_TYPE = ConfigBuilder("spark.kubernetes.resource.type") .doc("This sets the resource type internally") .internal() .stringConf + .checkValues(Set(APP_RESOURCE_TYPE_JAVA, APP_RESOURCE_TYPE_PYTHON, APP_RESOURCE_TYPE_R)) .createOptional val KUBERNETES_LOCAL_DIRS_TMPFS = @@ -241,11 +294,56 @@ private[spark] object Config extends Logging { .booleanConf .createWithDefault(false) + val KUBERNETES_DRIVER_PODTEMPLATE_FILE = + ConfigBuilder("spark.kubernetes.driver.podTemplateFile") + .doc("File containing a template pod spec for the driver") + .stringConf + .createOptional + + val KUBERNETES_EXECUTOR_PODTEMPLATE_FILE = + ConfigBuilder("spark.kubernetes.executor.podTemplateFile") + .doc("File containing a template pod spec for executors") + .stringConf + .createOptional + + val KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME = + ConfigBuilder("spark.kubernetes.driver.podTemplateContainerName") + .doc("container name to be used as a basis for the driver in the given pod template") + .stringConf + .createOptional + + val KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME = + ConfigBuilder("spark.kubernetes.executor.podTemplateContainerName") + .doc("container name to be used as a basis for executors in the given pod template") + .stringConf + .createOptional + val KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX = "spark.kubernetes.authenticate.submission" val KUBERNETES_NODE_SELECTOR_PREFIX = "spark.kubernetes.node.selector." + val KUBERNETES_DELETE_EXECUTORS = + ConfigBuilder("spark.kubernetes.executor.deleteOnTermination") + .doc("If set to false then executor pods will not be deleted in case " + + "of failure or normal termination.") + .booleanConf + .createWithDefault(true) + + val KUBERNETES_SUBMIT_GRACE_PERIOD = + ConfigBuilder("spark.kubernetes.appKillPodDeletionGracePeriod") + .doc("Time to wait for graceful deletion of Spark pods when spark-submit" + + " is used for killing an application.") + .timeConf(TimeUnit.SECONDS) + .createOptional + + val KUBERNETES_FILE_UPLOAD_PATH = + ConfigBuilder("spark.kubernetes.file.upload.path") + .doc("Hadoop compatible file system path where files from the local file system " + + "will be uploded to in cluster mode.") + .stringConf + .createOptional + val KUBERNETES_DRIVER_LABEL_PREFIX = "spark.kubernetes.driver.label." val KUBERNETES_DRIVER_ANNOTATION_PREFIX = "spark.kubernetes.driver.annotation." val KUBERNETES_DRIVER_SECRETS_PREFIX = "spark.kubernetes.driver.secrets." @@ -262,6 +360,7 @@ private[spark] object Config extends Logging { val KUBERNETES_VOLUMES_PVC_TYPE = "persistentVolumeClaim" val KUBERNETES_VOLUMES_EMPTYDIR_TYPE = "emptyDir" val KUBERNETES_VOLUMES_MOUNT_PATH_KEY = "mount.path" + val KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY = "mount.subPath" val KUBERNETES_VOLUMES_MOUNT_READONLY_KEY = "mount.readOnly" val KUBERNETES_VOLUMES_OPTIONS_PATH_KEY = "options.path" val KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY = "options.claimName" diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala index 8202d874a4626..a3c74ff7b2885 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/Constants.scala @@ -60,22 +60,55 @@ private[spark] object Constants { val ENV_CLASSPATH = "SPARK_CLASSPATH" val ENV_DRIVER_BIND_ADDRESS = "SPARK_DRIVER_BIND_ADDRESS" val ENV_SPARK_CONF_DIR = "SPARK_CONF_DIR" + val ENV_SPARK_USER = "SPARK_USER" // Spark app configs for containers val SPARK_CONF_VOLUME = "spark-conf-volume" val SPARK_CONF_DIR_INTERNAL = "/opt/spark/conf" val SPARK_CONF_FILE_NAME = "spark.properties" val SPARK_CONF_PATH = s"$SPARK_CONF_DIR_INTERNAL/$SPARK_CONF_FILE_NAME" + val ENV_HADOOP_TOKEN_FILE_LOCATION = "HADOOP_TOKEN_FILE_LOCATION" // BINDINGS - val ENV_PYSPARK_PRIMARY = "PYSPARK_PRIMARY" - val ENV_PYSPARK_FILES = "PYSPARK_FILES" - val ENV_PYSPARK_ARGS = "PYSPARK_APP_ARGS" val ENV_PYSPARK_MAJOR_PYTHON_VERSION = "PYSPARK_MAJOR_PYTHON_VERSION" - val ENV_R_PRIMARY = "R_PRIMARY" - val ENV_R_ARGS = "R_APP_ARGS" + + // Pod spec templates + val EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME = "pod-spec-template.yml" + val EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH = "/opt/spark/pod-template" + val POD_TEMPLATE_VOLUME = "pod-template-volume" + val POD_TEMPLATE_CONFIGMAP = "podspec-configmap" + val POD_TEMPLATE_KEY = "podspec-configmap-key" // Miscellaneous val KUBERNETES_MASTER_INTERNAL_URL = "https://kubernetes.default.svc" - val DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + val DEFAULT_DRIVER_CONTAINER_NAME = "spark-kubernetes-driver" + val DEFAULT_EXECUTOR_CONTAINER_NAME = "spark-kubernetes-executor" val MEMORY_OVERHEAD_MIN_MIB = 384L + val NON_JVM_MEMORY_OVERHEAD_FACTOR = 0.4d + + // Hadoop Configuration + val HADOOP_CONF_VOLUME = "hadoop-properties" + val KRB_FILE_VOLUME = "krb5-file" + val HADOOP_CONF_DIR_PATH = "/opt/hadoop/conf" + val KRB_FILE_DIR_PATH = "/etc" + val ENV_HADOOP_CONF_DIR = "HADOOP_CONF_DIR" + val HADOOP_CONFIG_MAP_NAME = + "spark.kubernetes.executor.hadoopConfigMapName" + + // Kerberos Configuration + val KERBEROS_DT_SECRET_NAME = + "spark.kubernetes.kerberos.dt-secret-name" + val KERBEROS_DT_SECRET_KEY = + "spark.kubernetes.kerberos.dt-secret-key" + val KERBEROS_SECRET_KEY = "hadoop-tokens" + val KERBEROS_KEYTAB_VOLUME = "kerberos-keytab" + val KERBEROS_KEYTAB_MOUNT_POINT = "/mnt/secrets/kerberos-keytab" + + // Hadoop credentials secrets for the Spark app. + val SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR = "/mnt/secrets/hadoop-credentials" + val SPARK_APP_HADOOP_SECRET_VOLUME_NAME = "hadoop-secret" + + // Application resource types. + val APP_RESOURCE_TYPE_JAVA = "java" + val APP_RESOURCE_TYPE_PYTHON = "python" + val APP_RESOURCE_TYPE_R = "r" } 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 cae6e7d5ad518..a2a46614fb8f8 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 @@ -16,7 +16,7 @@ */ package org.apache.spark.deploy.k8s -import scala.collection.mutable +import java.util.Locale import io.fabric8.kubernetes.api.model.{LocalObjectReference, LocalObjectReferenceBuilder, Pod} @@ -24,80 +24,41 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ -import org.apache.spark.deploy.k8s.submit.KubernetesClientApplication._ import org.apache.spark.internal.config.ConfigEntry - - -private[spark] sealed trait KubernetesRoleSpecificConf - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark driver. - */ -private[spark] case class KubernetesDriverSpecificConf( - mainAppResource: Option[MainAppResource], - mainClass: String, - appName: String, - appArgs: Seq[String]) extends KubernetesRoleSpecificConf - -/* - * Structure containing metadata for Kubernetes logic that builds a Spark executor. - */ -private[spark] case class KubernetesExecutorSpecificConf( - executorId: String, - driverPod: Option[Pod]) - extends KubernetesRoleSpecificConf +import org.apache.spark.util.Utils /** * Structure containing metadata for Kubernetes logic to build Spark pods. */ -private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( - sparkConf: SparkConf, - roleSpecificConf: T, - appResourceNamePrefix: String, - appId: String, - roleLabels: Map[String, String], - roleAnnotations: Map[String, String], - roleSecretNamesToMountPaths: Map[String, String], - roleSecretEnvNamesToKeyRefs: Map[String, String], - roleEnvs: Map[String, String], - roleVolumes: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]], - sparkFiles: Seq[String]) { - - def namespace(): String = sparkConf.get(KUBERNETES_NAMESPACE) +private[spark] abstract class KubernetesConf(val sparkConf: SparkConf) { - def sparkJars(): Seq[String] = sparkConf - .getOption("spark.jars") - .map(str => str.split(",").toSeq) - .getOrElse(Seq.empty[String]) + val resourceNamePrefix: String + def labels: Map[String, String] + def environment: Map[String, String] + def annotations: Map[String, String] + def secretEnvNamesToKeyRefs: Map[String, String] + def secretNamesToMountPaths: Map[String, String] + def volumes: Seq[KubernetesVolumeSpec] - def pyFiles(): Option[String] = sparkConf - .get(KUBERNETES_PYSPARK_PY_FILES) + def appName: String = get("spark.app.name", "spark") - def pySparkMainResource(): Option[String] = sparkConf - .get(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE) + def namespace: String = get(KUBERNETES_NAMESPACE) - def pySparkPythonVersion(): String = sparkConf - .get(PYSPARK_MAJOR_PYTHON_VERSION) + def imagePullPolicy: String = get(CONTAINER_IMAGE_PULL_POLICY) - def sparkRMainResource(): Option[String] = sparkConf - .get(KUBERNETES_R_MAIN_APP_RESOURCE) - - def imagePullPolicy(): String = sparkConf.get(CONTAINER_IMAGE_PULL_POLICY) - - def imagePullSecrets(): Seq[LocalObjectReference] = { + def imagePullSecrets: Seq[LocalObjectReference] = { sparkConf .get(IMAGE_PULL_SECRETS) - .map(_.split(",")) - .getOrElse(Array.empty[String]) - .map(_.trim) .map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } } - def nodeSelector(): Map[String, String] = + def nodeSelector: Map[String, String] = KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_NODE_SELECTOR_PREFIX) + def contains(config: ConfigEntry[_]): Boolean = sparkConf.contains(config) + def get[T](config: ConfigEntry[T]): T = sparkConf.get(config) def get(conf: String): String = sparkConf.get(conf) @@ -107,141 +68,144 @@ private[spark] case class KubernetesConf[T <: KubernetesRoleSpecificConf]( def getOption(key: String): Option[String] = sparkConf.getOption(key) } +private[spark] class KubernetesDriverConf( + sparkConf: SparkConf, + val appId: String, + val mainAppResource: MainAppResource, + val mainClass: String, + val appArgs: Array[String]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + val custom = if (Utils.isTesting) get(KUBERNETES_DRIVER_POD_NAME_PREFIX) else None + custom.getOrElse(KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) + val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !driverCustomLabels.contains(key), + s"Label with key $key is not allowed as it is reserved for Spark bookkeeping operations.") + } + + driverCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) + } + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX) + } +} + +private[spark] class KubernetesExecutorConf( + sparkConf: SparkConf, + val appId: String, + val executorId: String, + val driverPod: Option[Pod]) + extends KubernetesConf(sparkConf) { + + override val resourceNamePrefix: String = { + get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX).getOrElse( + KubernetesConf.getResourceNamePrefix(appName)) + } + + override def labels: Map[String, String] = { + val presetLabels = Map( + SPARK_EXECUTOR_ID_LABEL -> executorId, + SPARK_APP_ID_LABEL -> appId, + SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) + + val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( + sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) + + presetLabels.keys.foreach { key => + require( + !executorCustomLabels.contains(key), + s"Custom executor labels cannot contain $key as it is reserved for Spark.") + } + + executorCustomLabels ++ presetLabels + } + + override def environment: Map[String, String] = sparkConf.getExecutorEnv.toMap + + override def annotations: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) + } + + override def secretNamesToMountPaths: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) + } + + override def secretEnvNamesToKeyRefs: Map[String, String] = { + KubernetesUtils.parsePrefixedKeyValuePairs(sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) + } + + override def volumes: Seq[KubernetesVolumeSpec] = { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) + } + +} + private[spark] object KubernetesConf { def createDriverConf( sparkConf: SparkConf, - appName: String, - appResourceNamePrefix: String, appId: String, - mainAppResource: Option[MainAppResource], + mainAppResource: MainAppResource, mainClass: String, - appArgs: Array[String], - maybePyFiles: Option[String]): KubernetesConf[KubernetesDriverSpecificConf] = { - val sparkConfWithMainAppJar = sparkConf.clone() - val additionalFiles = mutable.ArrayBuffer.empty[String] - mainAppResource.foreach { - case JavaMainAppResource(res) => - val previousJars = sparkConf - .getOption("spark.jars") - .map(_.split(",")) - .getOrElse(Array.empty) - if (!previousJars.contains(res)) { - sparkConfWithMainAppJar.setJars(previousJars ++ Seq(res)) - } - // The function of this outer match is to account for multiple nonJVM - // bindings that will all have increased default MEMORY_OVERHEAD_FACTOR to 0.4 - case nonJVM: NonJVMResource => - nonJVM match { - case PythonMainAppResource(res) => - additionalFiles += res - maybePyFiles.foreach{maybePyFiles => - additionalFiles.appendAll(maybePyFiles.split(","))} - sparkConfWithMainAppJar.set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, res) - case RMainAppResource(res) => - additionalFiles += res - sparkConfWithMainAppJar.set(KUBERNETES_R_MAIN_APP_RESOURCE, res) - } - sparkConfWithMainAppJar.setIfMissing(MEMORY_OVERHEAD_FACTOR, 0.4) - } + appArgs: Array[String]): KubernetesDriverConf = { + // Parse executor volumes in order to verify configuration before the driver pod is created. + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX) - val driverCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_LABEL_PREFIX) - require(!driverCustomLabels.contains(SPARK_APP_ID_LABEL), "Label with key " + - s"$SPARK_APP_ID_LABEL is not allowed as it is reserved for Spark bookkeeping " + - "operations.") - require(!driverCustomLabels.contains(SPARK_ROLE_LABEL), "Label with key " + - s"$SPARK_ROLE_LABEL is not allowed as it is reserved for Spark bookkeeping " + - "operations.") - val driverLabels = driverCustomLabels ++ Map( - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) - val driverAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ANNOTATION_PREFIX) - val driverSecretNamesToMountPaths = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRETS_PREFIX) - val driverSecretEnvNamesToKeyRefs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX) - val driverEnvs = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_DRIVER_ENV_PREFIX) - val driverVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_DRIVER_VOLUMES_PREFIX).map(_.get) - // Also parse executor volumes in order to verify configuration - // before the driver pod is created - KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - val sparkFiles = sparkConf - .getOption("spark.files") - .map(str => str.split(",").toSeq) - .getOrElse(Seq.empty[String]) ++ additionalFiles - - KubernetesConf( - sparkConfWithMainAppJar, - KubernetesDriverSpecificConf(mainAppResource, mainClass, appName, appArgs), - appResourceNamePrefix, - appId, - driverLabels, - driverAnnotations, - driverSecretNamesToMountPaths, - driverSecretEnvNamesToKeyRefs, - driverEnvs, - driverVolumes, - sparkFiles) + new KubernetesDriverConf(sparkConf.clone(), appId, mainAppResource, mainClass, appArgs) } def createExecutorConf( sparkConf: SparkConf, executorId: String, appId: String, - driverPod: Option[Pod]): KubernetesConf[KubernetesExecutorSpecificConf] = { - val executorCustomLabels = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_LABEL_PREFIX) - require( - !executorCustomLabels.contains(SPARK_APP_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_APP_ID_LABEL as it is reserved for Spark.") - require( - !executorCustomLabels.contains(SPARK_EXECUTOR_ID_LABEL), - s"Custom executor labels cannot contain $SPARK_EXECUTOR_ID_LABEL as it is reserved for" + - " Spark.") - require( - !executorCustomLabels.contains(SPARK_ROLE_LABEL), - s"Custom executor labels cannot contain $SPARK_ROLE_LABEL as it is reserved for Spark.") - val executorLabels = Map( - SPARK_EXECUTOR_ID_LABEL -> executorId, - SPARK_APP_ID_LABEL -> appId, - SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ - executorCustomLabels - val executorAnnotations = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX) - val executorMountSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRETS_PREFIX) - val executorEnvSecrets = KubernetesUtils.parsePrefixedKeyValuePairs( - sparkConf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX) - val executorEnv = sparkConf.getExecutorEnv.toMap - val executorVolumes = KubernetesVolumeUtils.parseVolumesWithPrefix( - sparkConf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX).map(_.get) - - // If no prefix is defined then we are in pure client mode - // (not the one used by cluster mode inside the container) - val appResourceNamePrefix = { - if (sparkConf.getOption(KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key).isEmpty) { - getResourceNamePrefix(getAppName(sparkConf)) - } else { - sparkConf.get(KUBERNETES_EXECUTOR_POD_NAME_PREFIX) - } - } + driverPod: Option[Pod]): KubernetesExecutorConf = { + new KubernetesExecutorConf(sparkConf.clone(), appId, executorId, driverPod) + } + + def getResourceNamePrefix(appName: String): String = { + val id = KubernetesUtils.uniqueID() + s"$appName-$id" + .trim + .toLowerCase(Locale.ROOT) + .replaceAll("\\s+", "-") + .replaceAll("\\.", "-") + .replaceAll("[^a-z0-9\\-]", "") + .replaceAll("-+", "-") + } - KubernetesConf( - sparkConf.clone(), - KubernetesExecutorSpecificConf(executorId, driverPod), - appResourceNamePrefix, - appId, - executorLabels, - executorAnnotations, - executorMountSecrets, - executorEnvSecrets, - executorEnv, - executorVolumes, - Seq.empty[String]) + /** + * Build a resources name based on the vendor device plugin naming + * convention of: vendor-domain/resource. For example, an NVIDIA GPU is + * advertised as nvidia.com/gpu. + */ + def buildKubernetesResourceName(vendorDomain: String, resourceName: String): String = { + s"${vendorDomain}/${resourceName}" } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesDriverSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesDriverSpec.scala index 0c5ae022f4070..fce8c6a4bf494 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesDriverSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesDriverSpec.scala @@ -22,10 +22,3 @@ private[spark] case class KubernetesDriverSpec( pod: SparkPod, driverKubernetesResources: Seq[HasMetadata], systemProperties: Map[String, String]) - -private[spark] object KubernetesDriverSpec { - def initialSpec(initialProps: Map[String, String]): KubernetesDriverSpec = KubernetesDriverSpec( - SparkPod.initialPod(), - Seq.empty, - initialProps) -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala index f5fae7cc8c470..b1b7751b012e1 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesUtils.scala @@ -16,14 +16,31 @@ */ package org.apache.spark.deploy.k8s +import java.io.{File, IOException} +import java.net.URI +import java.security.SecureRandom +import java.util.UUID + import scala.collection.JavaConverters._ -import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, Time} +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, PodBuilder, Quantity, QuantityBuilder} +import io.fabric8.kubernetes.client.KubernetesClient +import org.apache.commons.codec.binary.Hex +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} -import org.apache.spark.util.Utils +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.Config.KUBERNETES_FILE_UPLOAD_PATH +import org.apache.spark.internal.Logging +import org.apache.spark.launcher.SparkLauncher +import org.apache.spark.resource.ResourceUtils +import org.apache.spark.util.{Clock, SystemClock, Utils} +import org.apache.spark.util.Utils.getHadoopFileSystem + +private[spark] object KubernetesUtils extends Logging { -private[spark] object KubernetesUtils { + private val systemClock = new SystemClock() + private lazy val RNG = new SecureRandom() /** * Extract and parse Spark configuration properties with a given name prefix and @@ -39,28 +56,68 @@ private[spark] object KubernetesUtils { sparkConf.getAllWithPrefix(prefix).toMap } + def requireBothOrNeitherDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenFirstIsMissing: String, + errMessageWhenSecondIsMissing: String): Unit = { + requireSecondIfFirstIsDefined(opt1, opt2, errMessageWhenSecondIsMissing) + requireSecondIfFirstIsDefined(opt2, opt1, errMessageWhenFirstIsMissing) + } + + def requireSecondIfFirstIsDefined( + opt1: Option[_], + opt2: Option[_], + errMessageWhenSecondIsMissing: String): Unit = { + opt1.foreach { _ => + require(opt2.isDefined, errMessageWhenSecondIsMissing) + } + } + def requireNandDefined(opt1: Option[_], opt2: Option[_], errMessage: String): Unit = { opt1.foreach { _ => require(opt2.isEmpty, errMessage) } + opt2.foreach { _ => require(opt1.isEmpty, errMessage) } } - /** - * For the given collection of file URIs, resolves them as follows: - * - File URIs with scheme local:// resolve to just the path of the URI. - * - Otherwise, the URIs are returned as-is. - */ - def resolveFileUrisAndPath(fileUris: Iterable[String]): Iterable[String] = { - fileUris.map { uri => - resolveFileUri(uri) + def loadPodFromTemplate( + kubernetesClient: KubernetesClient, + templateFile: File, + containerName: Option[String]): SparkPod = { + try { + val pod = kubernetesClient.pods().load(templateFile).get() + selectSparkContainer(pod, containerName) + } catch { + case e: Exception => + logError( + s"Encountered exception while attempting to load initial pod spec from file", e) + throw new SparkException("Could not load pod from template file.", e) } } - def resolveFileUri(uri: String): String = { - val fileUri = Utils.resolveURI(uri) - val fileScheme = Option(fileUri.getScheme).getOrElse("file") - fileScheme match { - case "local" => fileUri.getPath - case _ => uri - } + def selectSparkContainer(pod: Pod, containerName: Option[String]): SparkPod = { + def selectNamedContainer( + containers: List[Container], name: String): Option[(Container, List[Container])] = + containers.partition(_.getName == name) match { + case (sparkContainer :: Nil, rest) => Some((sparkContainer, rest)) + case _ => + logWarning( + s"specified container ${name} not found on pod template, " + + s"falling back to taking the first container") + Option.empty + } + val containers = pod.getSpec.getContainers.asScala.toList + containerName + .flatMap(selectNamedContainer(containers, _)) + .orElse(containers.headOption.map((_, containers.tail))) + .map { + case (sparkContainer: Container, rest: List[Container]) => SparkPod( + new PodBuilder(pod) + .editSpec() + .withContainers(rest.asJava) + .endSpec() + .build(), + sparkContainer) + }.getOrElse(SparkPod(pod, new ContainerBuilder().build())) } def parseMasterUrl(url: String): String = url.substring("k8s://".length) @@ -138,7 +195,130 @@ private[spark] object KubernetesUtils { }.getOrElse(Seq(("container state", "N/A"))) } - def formatTime(time: Time): String = { - if (time != null) time.getTime else "N/A" + def formatTime(time: String): String = { + if (time != null) time else "N/A" + } + + /** + * Generates a unique ID to be used as part of identifiers. The returned ID is a hex string + * of a 64-bit value containing the 40 LSBs from the current time + 24 random bits from a + * cryptographically strong RNG. (40 bits gives about 30 years worth of "unique" timestamps.) + * + * This avoids using a UUID for uniqueness (too long), and relying solely on the current time + * (not unique enough). + */ + def uniqueID(clock: Clock = systemClock): String = { + val random = new Array[Byte](3) + synchronized { + RNG.nextBytes(random) + } + + val time = java.lang.Long.toHexString(clock.getTimeMillis() & 0xFFFFFFFFFFL) + Hex.encodeHexString(random) + time + } + + /** + * This function builds the Quantity objects for each resource in the Spark resource + * configs based on the component name(spark.driver.resource or spark.executor.resource). + * It assumes we can use the Kubernetes device plugin format: vendor-domain/resource. + * It returns a set with a tuple of vendor-domain/resource and Quantity for each resource. + */ + def buildResourcesQuantities( + componentName: String, + sparkConf: SparkConf): Map[String, Quantity] = { + val requests = ResourceUtils.parseAllResourceRequests(sparkConf, componentName) + requests.map { request => + val vendorDomain = request.vendor.getOrElse(throw new SparkException("Resource: " + + s"${request.id.resourceName} was requested, but vendor was not specified.")) + val quantity = new QuantityBuilder(false) + .withAmount(request.amount.toString) + .build() + (KubernetesConf.buildKubernetesResourceName(vendorDomain, request.id.resourceName), quantity) + }.toMap + } + + /** + * Upload files and modify their uris + */ + def uploadAndTransformFileUris(fileUris: Iterable[String], conf: Option[SparkConf] = None) + : Iterable[String] = { + fileUris.map { uri => + uploadFileUri(uri, conf) + } + } + + private def isLocalDependency(uri: URI): Boolean = { + uri.getScheme match { + case null | "file" => true + case _ => false + } + } + + def isLocalAndResolvable(resource: String): Boolean = { + resource != SparkLauncher.NO_RESOURCE && + isLocalDependency(Utils.resolveURI(resource)) + } + + def renameMainAppResource(resource: String, conf: SparkConf): String = { + if (isLocalAndResolvable(resource)) { + SparkLauncher.NO_RESOURCE + } else { + resource + } + } + + def uploadFileUri(uri: String, conf: Option[SparkConf] = None): String = { + conf match { + case Some(sConf) => + if (sConf.get(KUBERNETES_FILE_UPLOAD_PATH).isDefined) { + val fileUri = Utils.resolveURI(uri) + try { + val hadoopConf = SparkHadoopUtil.get.newConfiguration(sConf) + val uploadPath = sConf.get(KUBERNETES_FILE_UPLOAD_PATH).get + val fs = getHadoopFileSystem(Utils.resolveURI(uploadPath), hadoopConf) + val randomDirName = s"spark-upload-${UUID.randomUUID()}" + fs.mkdirs(new Path(s"${uploadPath}/${randomDirName}")) + val targetUri = s"${uploadPath}/${randomDirName}/${fileUri.getPath.split("/").last}" + log.info(s"Uploading file: ${fileUri.getPath} to dest: $targetUri...") + uploadFileToHadoopCompatibleFS(new Path(fileUri.getPath), new Path(targetUri), fs) + targetUri + } catch { + case e: Exception => + throw new SparkException(s"Uploading file ${fileUri.getPath} failed...", e) + } + } else { + throw new SparkException("Please specify " + + "spark.kubernetes.file.upload.path property.") + } + case _ => throw new SparkException("Spark configuration is missing...") + } + } + + /** + * Upload a file to a Hadoop-compatible filesystem. + */ + private def uploadFileToHadoopCompatibleFS( + src: Path, + dest: Path, + fs: FileSystem, + delSrc : Boolean = false, + overwrite: Boolean = true): Unit = { + try { + fs.copyFromLocalFile(false, true, src, dest) + } catch { + case e: IOException => + throw new SparkException(s"Error uploading file ${src.getName}", e) + } + } + + def buildPodWithServiceAccount(serviceAccount: Option[String], pod: SparkPod): Option[Pod] = { + serviceAccount.map { account => + new PodBuilder(pod.pod) + .editOrNewSpec() + .withServiceAccount(account) + .withServiceAccountName(account) + .endSpec() + .build() + } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala index b1762d1efe2ea..0ebe8fd26015d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeSpec.scala @@ -18,12 +18,10 @@ package org.apache.spark.deploy.k8s private[spark] sealed trait KubernetesVolumeSpecificConf -private[spark] case class KubernetesHostPathVolumeConf( - hostPath: String) +private[spark] case class KubernetesHostPathVolumeConf(hostPath: String) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesPVCVolumeConf( - claimName: String) +private[spark] case class KubernetesPVCVolumeConf(claimName: String) extends KubernetesVolumeSpecificConf private[spark] case class KubernetesEmptyDirVolumeConf( @@ -31,8 +29,9 @@ private[spark] case class KubernetesEmptyDirVolumeConf( sizeLimit: Option[String]) extends KubernetesVolumeSpecificConf -private[spark] case class KubernetesVolumeSpec[T <: KubernetesVolumeSpecificConf]( +private[spark] case class KubernetesVolumeSpec( volumeName: String, mountPath: String, + mountSubPath: String, mountReadOnly: Boolean, - volumeConf: T) + volumeConf: KubernetesVolumeSpecificConf) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala index 713df5fffc3a2..c0c4f86f1a6a0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtils.scala @@ -16,10 +16,6 @@ */ package org.apache.spark.deploy.k8s -import java.util.NoSuchElementException - -import scala.util.{Failure, Success, Try} - import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ @@ -31,25 +27,21 @@ private[spark] object KubernetesVolumeUtils { * @param prefix the given property name prefix * @return a Map storing with volume name as key and spec as value */ - def parseVolumesWithPrefix( - sparkConf: SparkConf, - prefix: String): Iterable[Try[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]]] = { + def parseVolumesWithPrefix(sparkConf: SparkConf, prefix: String): Seq[KubernetesVolumeSpec] = { val properties = sparkConf.getAllWithPrefix(prefix).toMap getVolumeTypesAndNames(properties).map { case (volumeType, volumeName) => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_PATH_KEY" val readOnlyKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_READONLY_KEY" + val subPathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY" - for { - path <- properties.getTry(pathKey) - volumeConf <- parseVolumeSpecificConf(properties, volumeType, volumeName) - } yield KubernetesVolumeSpec( + KubernetesVolumeSpec( volumeName = volumeName, - mountPath = path, + mountPath = properties(pathKey), + mountSubPath = properties.get(subPathKey).getOrElse(""), mountReadOnly = properties.get(readOnlyKey).exists(_.toBoolean), - volumeConf = volumeConf - ) - } + volumeConf = parseVolumeSpecificConf(properties, volumeType, volumeName)) + }.toSeq } /** @@ -59,9 +51,7 @@ private[spark] object KubernetesVolumeUtils { * @param properties flat mapping of property names to values * @return Set[(volumeType, volumeName)] */ - private def getVolumeTypesAndNames( - properties: Map[String, String] - ): Set[(String, String)] = { + private def getVolumeTypesAndNames(properties: Map[String, String]): Set[(String, String)] = { properties.keys.flatMap { k => k.split('.').toList match { case tpe :: name :: _ => Some((tpe, name)) @@ -71,40 +61,25 @@ private[spark] object KubernetesVolumeUtils { } private def parseVolumeSpecificConf( - options: Map[String, String], - volumeType: String, - volumeName: String): Try[KubernetesVolumeSpecificConf] = { + options: Map[String, String], + volumeType: String, + volumeName: String): KubernetesVolumeSpecificConf = { volumeType match { case KUBERNETES_VOLUMES_HOSTPATH_TYPE => val pathKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_PATH_KEY" - for { - path <- options.getTry(pathKey) - } yield KubernetesHostPathVolumeConf(path) + KubernetesHostPathVolumeConf(options(pathKey)) case KUBERNETES_VOLUMES_PVC_TYPE => val claimNameKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY" - for { - claimName <- options.getTry(claimNameKey) - } yield KubernetesPVCVolumeConf(claimName) + KubernetesPVCVolumeConf(options(claimNameKey)) case KUBERNETES_VOLUMES_EMPTYDIR_TYPE => val mediumKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY" val sizeLimitKey = s"$volumeType.$volumeName.$KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY" - Success(KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey))) + KubernetesEmptyDirVolumeConf(options.get(mediumKey), options.get(sizeLimitKey)) case _ => - Failure(new RuntimeException(s"Kubernetes Volume type `$volumeType` is not supported")) - } - } - - /** - * Convenience wrapper to accumulate key lookup errors - */ - implicit private class MapOps[A, B](m: Map[A, B]) { - def getTry(key: A): Try[B] = { - m - .get(key) - .fold[Try[B]](Failure(new NoSuchElementException(key.toString)))(Success(_)) + throw new IllegalArgumentException(s"Kubernetes Volume type `$volumeType` is not supported") } } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala index c47e78cbf19e3..459259f77796c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkKubernetesClientFactory.scala @@ -21,11 +21,14 @@ import java.io.File import com.google.common.base.Charsets import com.google.common.io.Files import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient, KubernetesClient} +import io.fabric8.kubernetes.client.Config.autoConfigure import io.fabric8.kubernetes.client.utils.HttpClientUtils import okhttp3.Dispatcher import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.util.ThreadUtils /** @@ -33,12 +36,13 @@ import org.apache.spark.util.ThreadUtils * parse configuration keys, similar to the manner in which Spark's SecurityManager parses SSL * options for different components. */ -private[spark] object SparkKubernetesClientFactory { +private[spark] object SparkKubernetesClientFactory extends Logging { def createKubernetesClient( master: String, namespace: Option[String], kubernetesAuthConfPrefix: String, + clientType: ClientType.Value, sparkConf: SparkConf, defaultServiceAccountToken: Option[File], defaultServiceAccountCaCert: Option[File]): KubernetesClient = { @@ -63,10 +67,22 @@ private[spark] object SparkKubernetesClientFactory { .getOption(s"$kubernetesAuthConfPrefix.$CLIENT_CERT_FILE_CONF_SUFFIX") val dispatcher = new Dispatcher( ThreadUtils.newDaemonCachedThreadPool("kubernetes-dispatcher")) - val config = new ConfigBuilder() + + // Allow for specifying a context used to auto-configure from the users K8S config file + val kubeContext = sparkConf.get(KUBERNETES_CONTEXT).filter(_.nonEmpty) + logInfo("Auto-configuring K8S client using " + + kubeContext.map("context " + _).getOrElse("current context") + + " from users K8S config file") + + // Start from an auto-configured config with the desired context + // Fabric 8 uses null to indicate that the users current context should be used so if no + // explicit setting pass null + val config = new ConfigBuilder(autoConfigure(kubeContext.getOrElse(null))) .withApiVersion("v1") .withMasterUrl(master) .withWebsocketPingInterval(0) + .withRequestTimeout(clientType.requestTimeout(sparkConf)) + .withConnectionTimeout(clientType.connectionTimeout(sparkConf)) .withOption(oauthTokenValue) { (token, configBuilder) => configBuilder.withOauthToken(token) }.withOption(oauthTokenFile) { @@ -99,4 +115,20 @@ private[spark] object SparkKubernetesClientFactory { }.getOrElse(configBuilder) } } + + object ClientType extends Enumeration { + import scala.language.implicitConversions + val Driver = Val(DRIVER_CLIENT_REQUEST_TIMEOUT, DRIVER_CLIENT_CONNECTION_TIMEOUT) + val Submission = Val(SUBMISSION_CLIENT_REQUEST_TIMEOUT, SUBMISSION_CLIENT_CONNECTION_TIMEOUT) + + protected case class Val( + requestTimeoutEntry: ConfigEntry[Int], + connectionTimeoutEntry: ConfigEntry[Int]) + extends super.Val { + def requestTimeout(conf: SparkConf): Int = conf.get(requestTimeoutEntry) + def connectionTimeout(conf: SparkConf): Int = conf.get(connectionTimeoutEntry) + } + + implicit def convert(value: Value): Val = value.asInstanceOf[Val] + } } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala index 345dd117fd35f..fd1196368a7ff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/SparkPod.scala @@ -18,7 +18,30 @@ package org.apache.spark.deploy.k8s import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, Pod, PodBuilder} -private[spark] case class SparkPod(pod: Pod, container: Container) +private[spark] case class SparkPod(pod: Pod, container: Container) { + + /** + * Convenience method to apply a series of chained transformations to a pod. + * + * Use it like: + * + * original.modify { case pod => + * // update pod and return new one + * }.modify { case pod => + * // more changes that create a new pod + * }.modify { + * case pod if someCondition => // new pod + * } + * + * This makes it cleaner to apply multiple transformations, avoiding having to create + * a bunch of awkwardly-named local variables. Since the argument is a partial function, + * it can do matching without needing to exhaust all the possibilities. If the function + * is not applied, then the original pod will be kept. + */ + def transform(fn: PartialFunction[SparkPod, SparkPod]): SparkPod = fn.lift(this).getOrElse(this) + +} + private[spark] object SparkPod { def initialPod(): SparkPod = { diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala index 575bc54ffe2bb..1944ba9766cff 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStep.scala @@ -28,34 +28,49 @@ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.submit._ import org.apache.spark.internal.config._ import org.apache.spark.ui.SparkUI +import org.apache.spark.util.Utils -private[spark] class BasicDriverFeatureStep( - conf: KubernetesConf[KubernetesDriverSpecificConf]) +private[spark] class BasicDriverFeatureStep(conf: KubernetesDriverConf) extends KubernetesFeatureConfigStep { private val driverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) - .getOrElse(s"${conf.appResourceNamePrefix}-driver") + .getOrElse(s"${conf.resourceNamePrefix}-driver") private val driverContainerImage = conf .get(DRIVER_CONTAINER_IMAGE) .getOrElse(throw new SparkException("Must specify the driver container image")) // CPU settings - private val driverCpuCores = conf.get("spark.driver.cores", "1") + private val driverCpuCores = conf.get(DRIVER_CORES) + private val driverCoresRequest = conf + .get(KUBERNETES_DRIVER_REQUEST_CORES) + .getOrElse(driverCpuCores.toString) private val driverLimitCores = conf.get(KUBERNETES_DRIVER_LIMIT_CORES) // Memory settings private val driverMemoryMiB = conf.get(DRIVER_MEMORY) + + // The memory overhead factor to use. If the user has not set it, then use a different + // value for non-JVM apps. This value is propagated to executors. + private val overheadFactor = + if (conf.mainAppResource.isInstanceOf[NonJVMResource]) { + if (conf.contains(MEMORY_OVERHEAD_FACTOR)) { + conf.get(MEMORY_OVERHEAD_FACTOR) + } else { + NON_JVM_MEMORY_OVERHEAD_FACTOR + } + } else { + conf.get(MEMORY_OVERHEAD_FACTOR) + } + private val memoryOverheadMiB = conf .get(DRIVER_MEMORY_OVERHEAD) - .getOrElse(math.max((conf.get(MEMORY_OVERHEAD_FACTOR) * driverMemoryMiB).toInt, - MEMORY_OVERHEAD_MIN_MIB)) + .getOrElse(math.max((overheadFactor * driverMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val driverMemoryWithOverheadMiB = driverMemoryMiB + memoryOverheadMiB override def configurePod(pod: SparkPod): SparkPod = { - val driverCustomEnvs = conf.roleEnvs - .toSeq + val driverCustomEnvs = conf.environment.toSeq .map { env => new EnvVarBuilder() .withName(env._1) @@ -64,7 +79,7 @@ private[spark] class BasicDriverFeatureStep( } val driverCpuQuantity = new QuantityBuilder(false) - .withAmount(driverCpuCores) + .withAmount(driverCoresRequest) .build() val driverMemoryQuantity = new QuantityBuilder(false) .withAmount(s"${driverMemoryWithOverheadMiB}Mi") @@ -73,16 +88,19 @@ private[spark] class BasicDriverFeatureStep( ("cpu", new QuantityBuilder(false).withAmount(limitCores).build()) } - val driverPort = conf.sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT) + val driverResourceQuantities = + KubernetesUtils.buildResourcesQuantities(SPARK_DRIVER_PREFIX, conf.sparkConf) + + val driverPort = conf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT) val driverBlockManagerPort = conf.sparkConf.getInt( DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT ) val driverUIPort = SparkUI.getUIPort(conf.sparkConf) val driverContainer = new ContainerBuilder(pod.container) - .withName(DRIVER_CONTAINER_NAME) + .withName(Option(pod.container.getName).getOrElse(DEFAULT_DRIVER_CONTAINER_NAME)) .withImage(driverContainerImage) - .withImagePullPolicy(conf.imagePullPolicy()) + .withImagePullPolicy(conf.imagePullPolicy) .addNewPort() .withName(DRIVER_PORT_NAME) .withContainerPort(driverPort) @@ -98,6 +116,10 @@ private[spark] class BasicDriverFeatureStep( .withContainerPort(driverUIPort) .withProtocol("TCP") .endPort() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(driverCustomEnvs.asJava) .addNewEnv() .withName(ENV_DRIVER_BIND_ADDRESS) @@ -105,24 +127,25 @@ private[spark] class BasicDriverFeatureStep( .withNewFieldRef("v1", "status.podIP") .build()) .endEnv() - .withNewResources() + .editOrNewResources() .addToRequests("cpu", driverCpuQuantity) .addToLimits(maybeCpuLimitQuantity.toMap.asJava) .addToRequests("memory", driverMemoryQuantity) .addToLimits("memory", driverMemoryQuantity) + .addToLimits(driverResourceQuantities.asJava) .endResources() .build() val driverPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(driverPodName) - .addToLabels(conf.roleLabels.asJava) - .addToAnnotations(conf.roleAnnotations.asJava) + .addToLabels(conf.labels.asJava) + .addToAnnotations(conf.annotations.asJava) .endMetadata() - .withNewSpec() + .editOrNewSpec() .withRestartPolicy("Never") - .withNodeSelector(conf.nodeSelector().asJava) - .addToImagePullSecrets(conf.imagePullSecrets(): _*) + .addToNodeSelector(conf.nodeSelector.asJava) + .addToImagePullSecrets(conf.imagePullSecrets: _*) .endSpec() .build() @@ -133,21 +156,18 @@ private[spark] class BasicDriverFeatureStep( val additionalProps = mutable.Map( KUBERNETES_DRIVER_POD_NAME.key -> driverPodName, "spark.app.id" -> conf.appId, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.appResourceNamePrefix, - KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true") - - val resolvedSparkJars = KubernetesUtils.resolveFileUrisAndPath( - conf.sparkJars()) - val resolvedSparkFiles = KubernetesUtils.resolveFileUrisAndPath( - conf.sparkFiles) - if (resolvedSparkJars.nonEmpty) { - additionalProps.put("spark.jars", resolvedSparkJars.mkString(",")) - } - if (resolvedSparkFiles.nonEmpty) { - additionalProps.put("spark.files", resolvedSparkFiles.mkString(",")) + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> conf.resourceNamePrefix, + KUBERNETES_DRIVER_SUBMIT_CHECK.key -> "true", + MEMORY_OVERHEAD_FACTOR.key -> overheadFactor.toString) + // try upload local, resolvable files to a hadoop compatible file system + Seq(JARS, FILES).foreach { key => + val value = conf.get(key).filter(uri => KubernetesUtils.isLocalAndResolvable(uri)) + val resolved = KubernetesUtils.uploadAndTransformFileUris(value, Some(conf.sparkConf)) + if (resolved.nonEmpty) { + additionalProps.put(key.key, resolved.mkString(",")) + } } additionalProps.toMap } - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } + diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala index d89995ba5e4f4..d6487556a371e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStep.scala @@ -20,21 +20,22 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model._ -import org.apache.spark.SparkException +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.config.{EXECUTOR_CLASS_PATH, EXECUTOR_JAVA_OPTIONS, EXECUTOR_MEMORY, EXECUTOR_MEMORY_OVERHEAD, PYSPARK_EXECUTOR_MEMORY} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils private[spark] class BasicExecutorFeatureStep( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]) + kubernetesConf: KubernetesExecutorConf, + secMgr: SecurityManager) extends KubernetesFeatureConfigStep { // Consider moving some of these fields to KubernetesConf or KubernetesExecutorSpecificConf - private val executorExtraClasspath = kubernetesConf.get(EXECUTOR_CLASS_PATH) private val executorContainerImage = kubernetesConf .get(EXECUTOR_CONTAINER_IMAGE) .getOrElse(throw new SparkException("Must specify the executor container image")) @@ -42,11 +43,11 @@ private[spark] class BasicExecutorFeatureStep( .sparkConf .getInt("spark.blockmanager.port", DEFAULT_BLOCKMANAGER_PORT) - private val executorPodNamePrefix = kubernetesConf.appResourceNamePrefix + private val executorPodNamePrefix = kubernetesConf.resourceNamePrefix private val driverUrl = RpcEndpointAddress( - kubernetesConf.get("spark.driver.host"), - kubernetesConf.sparkConf.getInt("spark.driver.port", DEFAULT_DRIVER_PORT), + kubernetesConf.get(DRIVER_HOST_ADDRESS), + kubernetesConf.sparkConf.getInt(DRIVER_PORT.key, DEFAULT_DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString private val executorMemoryMiB = kubernetesConf.get(EXECUTOR_MEMORY) private val executorMemoryString = kubernetesConf.get( @@ -58,18 +59,15 @@ private[spark] class BasicExecutorFeatureStep( (kubernetesConf.get(MEMORY_OVERHEAD_FACTOR) * executorMemoryMiB).toInt, MEMORY_OVERHEAD_MIN_MIB)) private val executorMemoryWithOverhead = executorMemoryMiB + memoryOverheadMiB - private val executorMemoryTotal = kubernetesConf.sparkConf - .getOption(APP_RESOURCE_TYPE.key).map{ res => - val additionalPySparkMemory = res match { - case "python" => - kubernetesConf.sparkConf - .get(PYSPARK_EXECUTOR_MEMORY).map(_.toInt).getOrElse(0) - case _ => 0 - } - executorMemoryWithOverhead + additionalPySparkMemory - }.getOrElse(executorMemoryWithOverhead) + private val executorMemoryTotal = + if (kubernetesConf.get(APP_RESOURCE_TYPE) == Some(APP_RESOURCE_TYPE_PYTHON)) { + executorMemoryWithOverhead + + kubernetesConf.get(PYSPARK_EXECUTOR_MEMORY).map(_.toInt).getOrElse(0) + } else { + executorMemoryWithOverhead + } - private val executorCores = kubernetesConf.sparkConf.getInt("spark.executor.cores", 1) + private val executorCores = kubernetesConf.sparkConf.get(EXECUTOR_CORES) private val executorCoresRequest = if (kubernetesConf.sparkConf.contains(KUBERNETES_EXECUTOR_REQUEST_CORES)) { kubernetesConf.get(KUBERNETES_EXECUTOR_REQUEST_CORES).get @@ -79,56 +77,84 @@ private[spark] class BasicExecutorFeatureStep( private val executorLimitCores = kubernetesConf.get(KUBERNETES_EXECUTOR_LIMIT_CORES) override def configurePod(pod: SparkPod): SparkPod = { - val name = s"$executorPodNamePrefix-exec-${kubernetesConf.roleSpecificConf.executorId}" + val name = s"$executorPodNamePrefix-exec-${kubernetesConf.executorId}" // hostname must be no longer than 63 characters, so take the last 63 characters of the pod // name as the hostname. This preserves uniqueness since the end of name contains // executorId val hostname = name.substring(Math.max(0, name.length - 63)) + // Remove non-word characters from the start of the hostname + .replaceAll("^[^\\w]+", "") + // Replace dangerous characters in the remaining string with a safe alternative. + .replaceAll("[^\\w-]+", "_") + val executorMemoryQuantity = new QuantityBuilder(false) .withAmount(s"${executorMemoryTotal}Mi") .build() val executorCpuQuantity = new QuantityBuilder(false) .withAmount(executorCoresRequest) .build() - val executorExtraClasspathEnv = executorExtraClasspath.map { cp => - new EnvVarBuilder() - .withName(ENV_CLASSPATH) - .withValue(cp) - .build() - } - val executorExtraJavaOptionsEnv = kubernetesConf - .get(EXECUTOR_JAVA_OPTIONS) - .map { opts => - val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, - kubernetesConf.roleSpecificConf.executorId) - val delimitedOpts = Utils.splitCommandString(subsOpts) - delimitedOpts.zipWithIndex.map { - case (opt, index) => - new EnvVarBuilder().withName(s"$ENV_JAVA_OPT_PREFIX$index").withValue(opt).build() + + val executorResourceQuantities = + KubernetesUtils.buildResourcesQuantities(SPARK_EXECUTOR_PREFIX, + kubernetesConf.sparkConf) + + val executorEnv: Seq[EnvVar] = { + (Seq( + (ENV_DRIVER_URL, driverUrl), + (ENV_EXECUTOR_CORES, executorCores.toString), + (ENV_EXECUTOR_MEMORY, executorMemoryString), + (ENV_APPLICATION_ID, kubernetesConf.appId), + // This is to set the SPARK_CONF_DIR to be /opt/spark/conf + (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), + (ENV_EXECUTOR_ID, kubernetesConf.executorId) + ) ++ kubernetesConf.environment).map { case (k, v) => + new EnvVarBuilder() + .withName(k) + .withValue(v) + .build() } - }.getOrElse(Seq.empty[EnvVar]) - val executorEnv = (Seq( - (ENV_DRIVER_URL, driverUrl), - (ENV_EXECUTOR_CORES, executorCores.toString), - (ENV_EXECUTOR_MEMORY, executorMemoryString), - (ENV_APPLICATION_ID, kubernetesConf.appId), - // This is to set the SPARK_CONF_DIR to be /opt/spark/conf - (ENV_SPARK_CONF_DIR, SPARK_CONF_DIR_INTERNAL), - (ENV_EXECUTOR_ID, kubernetesConf.roleSpecificConf.executorId)) ++ - kubernetesConf.roleEnvs) - .map(env => new EnvVarBuilder() - .withName(env._1) - .withValue(env._2) - .build() - ) ++ Seq( - new EnvVarBuilder() - .withName(ENV_EXECUTOR_POD_IP) - .withValueFrom(new EnvVarSourceBuilder() - .withNewFieldRef("v1", "status.podIP") + } ++ { + Seq(new EnvVarBuilder() + .withName(ENV_EXECUTOR_POD_IP) + .withValueFrom(new EnvVarSourceBuilder() + .withNewFieldRef("v1", "status.podIP") + .build()) .build()) - .build() - ) ++ executorExtraJavaOptionsEnv ++ executorExtraClasspathEnv.toSeq + } ++ { + if (kubernetesConf.get(AUTH_SECRET_FILE_EXECUTOR).isEmpty) { + Option(secMgr.getSecretKey()).map { authSecret => + new EnvVarBuilder() + .withName(SecurityManager.ENV_AUTH_SECRET) + .withValue(authSecret) + .build() + } + } else None + } ++ { + kubernetesConf.get(EXECUTOR_CLASS_PATH).map { cp => + new EnvVarBuilder() + .withName(ENV_CLASSPATH) + .withValue(cp) + .build() + } + } ++ { + val userOpts = kubernetesConf.get(EXECUTOR_JAVA_OPTIONS).toSeq.flatMap { opts => + val subsOpts = Utils.substituteAppNExecIds(opts, kubernetesConf.appId, + kubernetesConf.executorId) + Utils.splitCommandString(subsOpts) + } + + val sparkOpts = Utils.sparkJavaOpts(kubernetesConf.sparkConf, + SparkConf.isExecutorStartupConf) + + (userOpts ++ sparkOpts).zipWithIndex.map { case (opt, index) => + new EnvVarBuilder() + .withName(s"$ENV_JAVA_OPT_PREFIX$index") + .withValue(opt) + .build() + } + } + val requiredPorts = Seq( (BLOCK_MANAGER_PORT_NAME, blockManagerPort)) .map { case (name, port) => @@ -139,14 +165,19 @@ private[spark] class BasicExecutorFeatureStep( } val executorContainer = new ContainerBuilder(pod.container) - .withName("executor") + .withName(Option(pod.container.getName).getOrElse(DEFAULT_EXECUTOR_CONTAINER_NAME)) .withImage(executorContainerImage) - .withImagePullPolicy(kubernetesConf.imagePullPolicy()) - .withNewResources() + .withImagePullPolicy(kubernetesConf.imagePullPolicy) + .editOrNewResources() .addToRequests("memory", executorMemoryQuantity) .addToLimits("memory", executorMemoryQuantity) .addToRequests("cpu", executorCpuQuantity) + .addToLimits(executorResourceQuantities.asJava) .endResources() + .addNewEnv() + .withName(ENV_SPARK_USER) + .withValue(Utils.getCurrentUserName()) + .endEnv() .addAllToEnv(executorEnv.asJava) .withPorts(requiredPorts.asJava) .addToArgs("executor") @@ -161,34 +192,30 @@ private[spark] class BasicExecutorFeatureStep( .endResources() .build() }.getOrElse(executorContainer) - val driverPod = kubernetesConf.roleSpecificConf.driverPod - val ownerReference = driverPod.map(pod => + val ownerReference = kubernetesConf.driverPod.map { pod => new OwnerReferenceBuilder() .withController(true) .withApiVersion(pod.getApiVersion) .withKind(pod.getKind) .withName(pod.getMetadata.getName) .withUid(pod.getMetadata.getUid) - .build()) + .build() + } val executorPod = new PodBuilder(pod.pod) .editOrNewMetadata() .withName(name) - .withLabels(kubernetesConf.roleLabels.asJava) - .withAnnotations(kubernetesConf.roleAnnotations.asJava) + .addToLabels(kubernetesConf.labels.asJava) + .addToAnnotations(kubernetesConf.annotations.asJava) .addToOwnerReferences(ownerReference.toSeq: _*) .endMetadata() .editOrNewSpec() .withHostname(hostname) .withRestartPolicy("Never") - .withNodeSelector(kubernetesConf.nodeSelector().asJava) - .addToImagePullSecrets(kubernetesConf.imagePullSecrets(): _*) + .addToNodeSelector(kubernetesConf.nodeSelector.asJava) + .addToImagePullSecrets(kubernetesConf.imagePullSecrets: _*) .endSpec() .build() SparkPod(executorPod, containerWithLimitCores) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } 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 new file mode 100644 index 0000000000000..7faf0d75bdc75 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStep.scala @@ -0,0 +1,102 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder} + +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit._ +import org.apache.spark.launcher.SparkLauncher + +/** + * Creates the driver command for running the user app, and propagates needed configuration so + * executors can also find the app code. + */ +private[spark] class DriverCommandFeatureStep(conf: KubernetesDriverConf) + extends KubernetesFeatureConfigStep { + + override def configurePod(pod: SparkPod): SparkPod = { + conf.mainAppResource match { + case JavaMainAppResource(res) => + configureForJava(pod, res.getOrElse(SparkLauncher.NO_RESOURCE)) + + case PythonMainAppResource(res) => + configureForPython(pod, res) + + case RMainAppResource(res) => + configureForR(pod, res) + } + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = { + val appType = conf.mainAppResource match { + case JavaMainAppResource(_) => + APP_RESOURCE_TYPE_JAVA + + case PythonMainAppResource(_) => + APP_RESOURCE_TYPE_PYTHON + + case RMainAppResource(_) => + APP_RESOURCE_TYPE_R + } + + Map(APP_RESOURCE_TYPE.key -> appType) + } + + private def configureForJava(pod: SparkPod, res: String): SparkPod = { + val driverContainer = baseDriverContainer(pod, res).build() + SparkPod(pod.pod, driverContainer) + } + + private def configureForPython(pod: SparkPod, res: String): SparkPod = { + val pythonEnvs = + Seq(new EnvVarBuilder() + .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) + .withValue(conf.get(PYSPARK_MAJOR_PYTHON_VERSION)) + .build()) + + val pythonContainer = baseDriverContainer(pod, res) + .addAllToEnv(pythonEnvs.asJava) + .build() + + SparkPod(pod.pod, pythonContainer) + } + + private def configureForR(pod: SparkPod, res: String): SparkPod = { + val rContainer = baseDriverContainer(pod, res).build() + SparkPod(pod.pod, rContainer) + } + + private def baseDriverContainer(pod: SparkPod, resource: String): ContainerBuilder = { + // re-write primary resource, app jar is also added to spark.jars by default in SparkSubmit + val resolvedResource = if (conf.mainAppResource.isInstanceOf[JavaMainAppResource]) { + KubernetesUtils.renameMainAppResource(resource, conf.sparkConf) + } else { + resource + } + new ContainerBuilder(pod.container) + .addToArgs("driver") + .addToArgs("--properties-file", SPARK_CONF_PATH) + .addToArgs("--class", conf.mainClass) + .addToArgs(resolvedResource) + .addToArgs(conf.appArgs: _*) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala index ff5ad6673b309..b94fc9d2ffe9c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStep.scala @@ -27,8 +27,9 @@ import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilde import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount -private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf[_]) +private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { // TODO clean up this class, and credentials in general. See also SparkKubernetesClientFactory. // We should use a struct to hold all creds-related fields. A lot of the code is very repetitive. @@ -66,19 +67,11 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube clientCertDataBase64.isDefined private val driverCredentialsSecretName = - s"${kubernetesConf.appResourceNamePrefix}-kubernetes-credentials" + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials" override def configurePod(pod: SparkPod): SparkPod = { if (!shouldMountSecret) { - pod.copy( - pod = driverServiceAccount.map { account => - new PodBuilder(pod.pod) - .editOrNewSpec() - .withServiceAccount(account) - .withServiceAccountName(account) - .endSpec() - .build() - }.getOrElse(pod.pod)) + pod.copy(pod = buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod)) } else { val driverPodWithMountedKubernetesCredentials = new PodBuilder(pod.pod) @@ -122,7 +115,7 @@ private[spark] class DriverKubernetesCredentialsFeatureStep(kubernetesConf: Kube val redactedTokens = kubernetesConf.sparkConf.getAll .filter(_._1.endsWith(OAUTH_TOKEN_CONF_SUFFIX)) .toMap - .mapValues( _ => "") + .map { case (k, v) => (k, "") } redactedTokens ++ resolvedMountedCaCertFile.map { file => Map( diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala index f2d7bbd08f305..925bcdf3e637f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStep.scala @@ -20,14 +20,14 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{HasMetadata, ServiceBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, SparkPod} import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.util.{Clock, SystemClock} private[spark] class DriverServiceFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], - clock: Clock = new SystemClock) + kubernetesConf: KubernetesDriverConf, + clock: Clock = new SystemClock()) extends KubernetesFeatureConfigStep with Logging { import DriverServiceFeatureStep._ @@ -38,11 +38,11 @@ private[spark] class DriverServiceFeatureStep( s"$DRIVER_HOST_KEY is not supported in Kubernetes mode, as the driver's hostname will be " + "managed via a Kubernetes service.") - private val preferredServiceName = s"${kubernetesConf.appResourceNamePrefix}$DRIVER_SVC_POSTFIX" + private val preferredServiceName = s"${kubernetesConf.resourceNamePrefix}$DRIVER_SVC_POSTFIX" private val resolvedServiceName = if (preferredServiceName.length <= MAX_SERVICE_NAME_LENGTH) { preferredServiceName } else { - val randomServiceId = clock.getTimeMillis() + val randomServiceId = KubernetesUtils.uniqueID(clock = clock) val shorterServiceName = s"spark-$randomServiceId$DRIVER_SVC_POSTFIX" logWarning(s"Driver's hostname would preferably be $preferredServiceName, but this is " + s"too long (must be <= $MAX_SERVICE_NAME_LENGTH characters). Falling back to use " + @@ -51,18 +51,18 @@ private[spark] class DriverServiceFeatureStep( } private val driverPort = kubernetesConf.sparkConf.getInt( - "spark.driver.port", DEFAULT_DRIVER_PORT) + config.DRIVER_PORT.key, DEFAULT_DRIVER_PORT) private val driverBlockManagerPort = kubernetesConf.sparkConf.getInt( - org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + config.DRIVER_BLOCK_MANAGER_PORT.key, DEFAULT_BLOCKMANAGER_PORT) + private val driverUIPort = kubernetesConf.get(config.UI.UI_PORT) override def configurePod(pod: SparkPod): SparkPod = pod override def getAdditionalPodSystemProperties(): Map[String, String] = { - val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace()}.svc" + val driverHostname = s"$resolvedServiceName.${kubernetesConf.namespace}.svc" Map(DRIVER_HOST_KEY -> driverHostname, - "spark.driver.port" -> driverPort.toString, - org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key -> - driverBlockManagerPort.toString) + config.DRIVER_PORT.key -> driverPort.toString, + config.DRIVER_BLOCK_MANAGER_PORT.key -> driverBlockManagerPort.toString) } override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { @@ -72,7 +72,7 @@ private[spark] class DriverServiceFeatureStep( .endMetadata() .withNewSpec() .withClusterIP("None") - .withSelector(kubernetesConf.roleLabels.asJava) + .withSelector(kubernetesConf.labels.asJava) .addNewPort() .withName(DRIVER_PORT_NAME) .withPort(driverPort) @@ -83,6 +83,11 @@ private[spark] class DriverServiceFeatureStep( .withPort(driverBlockManagerPort) .withNewTargetPort(driverBlockManagerPort) .endPort() + .addNewPort() + .withName(UI_PORT_NAME) + .withPort(driverUIPort) + .withNewTargetPort(driverUIPort) + .endPort() .endSpec() .build() Seq(driverService) @@ -90,8 +95,8 @@ private[spark] class DriverServiceFeatureStep( } private[spark] object DriverServiceFeatureStep { - val DRIVER_BIND_ADDRESS_KEY = org.apache.spark.internal.config.DRIVER_BIND_ADDRESS.key - val DRIVER_HOST_KEY = org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key + val DRIVER_BIND_ADDRESS_KEY = config.DRIVER_BIND_ADDRESS.key + val DRIVER_HOST_KEY = config.DRIVER_HOST_ADDRESS.key val DRIVER_SVC_POSTFIX = "-driver-svc" val MAX_SERVICE_NAME_LENGTH = 63 } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala index 03ff7d48420ff..d78f04dcc40e6 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStep.scala @@ -20,14 +20,13 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class EnvSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class EnvSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedEnvSecrets = kubernetesConf - .roleSecretEnvNamesToKeyRefs + .secretEnvNamesToKeyRefs .map{ case (envName, keyRef) => // Keyref parts val keyRefParts = keyRef.split(":") @@ -50,8 +49,4 @@ private[spark] class EnvSecretsFeatureStep( .build() SparkPod(pod.pod, containerWithEnvVars) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala new file mode 100644 index 0000000000000..8b84aad553855 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/ExecutorKubernetesCredentialsFeatureStep.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Config.KUBERNETES_SERVICE_ACCOUNT_NAME +import org.apache.spark.deploy.k8s.KubernetesUtils.buildPodWithServiceAccount + +private[spark] class ExecutorKubernetesCredentialsFeatureStep(kubernetesConf: KubernetesConf) + extends KubernetesFeatureConfigStep { + private lazy val driverServiceAccount = kubernetesConf.get(KUBERNETES_SERVICE_ACCOUNT_NAME) + + override def configurePod(pod: SparkPod): SparkPod = { + pod.copy( + // if not setup by the pod template fallback to the driver's sa, + // last option is the default sa. + pod = if (Option(pod.pod.getSpec.getServiceAccount).isEmpty) { + buildPodWithServiceAccount(driverServiceAccount, pod).getOrElse(pod.pod) + } else { + pod.pod + }) + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala new file mode 100644 index 0000000000000..d602ed5481e65 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStep.scala @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ + +import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +/** + * Mounts the Hadoop configuration - either a pre-defined config map, or a local configuration + * directory - on the driver pod. + */ +private[spark] class HadoopConfDriverFeatureStep(conf: KubernetesConf) + extends KubernetesFeatureConfigStep { + + private val confDir = Option(conf.sparkConf.getenv(ENV_HADOOP_CONF_DIR)) + private val existingConfMap = conf.get(KUBERNETES_HADOOP_CONF_CONFIG_MAP) + + KubernetesUtils.requireNandDefined( + confDir, + existingConfMap, + "Do not specify both the `HADOOP_CONF_DIR` in your ENV and the ConfigMap " + + "as the creation of an additional ConfigMap, when one is already specified is extraneous") + + private lazy val confFiles: Seq[File] = { + val dir = new File(confDir.get) + if (dir.isDirectory) { + dir.listFiles.filter(_.isFile).toSeq + } else { + Nil + } + } + + private def newConfigMapName: String = s"${conf.resourceNamePrefix}-hadoop-config" + + private def hasHadoopConf: Boolean = confDir.isDefined || existingConfMap.isDefined + + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if hasHadoopConf => + val confVolume = if (confDir.isDefined) { + val keyPaths = confFiles.map { file => + new KeyToPathBuilder() + .withKey(file.getName()) + .withPath(file.getName()) + .build() + } + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(newConfigMapName) + .withItems(keyPaths.asJava) + .endConfigMap() + .build() + } else { + new VolumeBuilder() + .withName(HADOOP_CONF_VOLUME) + .withNewConfigMap() + .withName(existingConfMap.get) + .endConfigMap() + .build() + } + + val podWithConf = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(confVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(HADOOP_CONF_VOLUME) + .withMountPath(HADOOP_CONF_DIR_PATH) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_CONF_DIR) + .withValue(HADOOP_CONF_DIR_PATH) + .endEnv() + .build() + + SparkPod(podWithConf, containerWithMount) + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (confDir.isDefined) { + val fileMap = confFiles.map { file => + (file.getName(), Files.toString(file, StandardCharsets.UTF_8)) + }.toMap.asJava + + Seq(new ConfigMapBuilder() + .withNewMetadata() + .withName(newConfigMapName) + .endMetadata() + .addToData(fileMap) + .build()) + } else { + Nil + } + } + +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala new file mode 100644 index 0000000000000..a77e8d4dbcff2 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStep.scala @@ -0,0 +1,261 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model._ +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.security.UserGroupInformation + +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s.{KubernetesDriverConf, KubernetesUtils, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +/** + * Provide kerberos / service credentials to the Spark driver. + * + * There are three use cases, in order of precedence: + * + * - keytab: if a kerberos keytab is defined, it is provided to the driver, and the driver will + * manage the kerberos login and the creation of delegation tokens. + * - existing tokens: if a secret containing delegation tokens is provided, it will be mounted + * on the driver pod, and the driver will handle distribution of those tokens to executors. + * - tgt only: if Hadoop security is enabled, the local TGT will be used to create delegation + * tokens which will be provided to the driver. The driver will handle distribution of the + * tokens to executors. + */ +private[spark] class KerberosConfDriverFeatureStep(kubernetesConf: KubernetesDriverConf) + extends KubernetesFeatureConfigStep with Logging { + + private val principal = kubernetesConf.get(org.apache.spark.internal.config.PRINCIPAL) + private val keytab = kubernetesConf.get(org.apache.spark.internal.config.KEYTAB) + private val existingSecretName = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_NAME) + private val existingSecretItemKey = kubernetesConf.get(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY) + private val krb5File = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_FILE) + private val krb5CMap = kubernetesConf.get(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP) + private val hadoopConf = SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf) + + KubernetesUtils.requireNandDefined( + krb5File, + krb5CMap, + "Do not specify both a Krb5 local file and the ConfigMap as the creation " + + "of an additional ConfigMap, when one is already specified, is extraneous") + + KubernetesUtils.requireBothOrNeitherDefined( + keytab, + principal, + "If a Kerberos principal is specified you must also specify a Kerberos keytab", + "If a Kerberos keytab is specified you must also specify a Kerberos principal") + + KubernetesUtils.requireBothOrNeitherDefined( + existingSecretName, + existingSecretItemKey, + "If a secret data item-key where the data of the Kerberos Delegation Token is specified" + + " you must also specify the name of the secret", + "If a secret storing a Kerberos Delegation Token is specified you must also" + + " specify the item-key where the data is stored") + + if (!hasKerberosConf) { + logInfo("You have not specified a krb5.conf file locally or via a ConfigMap. " + + "Make sure that you have the krb5.conf locally on the driver image.") + } + + // Create delegation tokens if needed. This is a lazy val so that it's not populated + // unnecessarily. But it needs to be accessible to different methods in this class, + // since it's not clear based solely on available configuration options that delegation + // tokens are needed when other credentials are not available. + private lazy val delegationTokens: Array[Byte] = { + if (keytab.isEmpty && existingSecretName.isEmpty) { + val tokenManager = new HadoopDelegationTokenManager(kubernetesConf.sparkConf, + SparkHadoopUtil.get.newConfiguration(kubernetesConf.sparkConf), null) + val creds = UserGroupInformation.getCurrentUser().getCredentials() + tokenManager.obtainDelegationTokens(creds) + // If no tokens and no secrets are stored in the credentials, make sure nothing is returned, + // to avoid creating an unnecessary secret. + if (creds.numberOfTokens() > 0 || creds.numberOfSecretKeys() > 0) { + SparkHadoopUtil.get.serialize(creds) + } else { + null + } + } else { + null + } + } + + private def needKeytabUpload: Boolean = keytab.exists(!Utils.isLocalUri(_)) + + private def dtSecretName: String = s"${kubernetesConf.resourceNamePrefix}-delegation-tokens" + + private def ktSecretName: String = s"${kubernetesConf.resourceNamePrefix}-kerberos-keytab" + + private def hasKerberosConf: Boolean = krb5CMap.isDefined | krb5File.isDefined + + private def newConfigMapName: String = s"${kubernetesConf.resourceNamePrefix}-krb5-file" + + override def configurePod(original: SparkPod): SparkPod = { + original.transform { case pod if hasKerberosConf => + val configMapVolume = if (krb5CMap.isDefined) { + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(krb5CMap.get) + .endConfigMap() + .build() + } else { + val krb5Conf = new File(krb5File.get) + new VolumeBuilder() + .withName(KRB_FILE_VOLUME) + .withNewConfigMap() + .withName(newConfigMapName) + .withItems(new KeyToPathBuilder() + .withKey(krb5Conf.getName()) + .withPath(krb5Conf.getName()) + .build()) + .endConfigMap() + .build() + } + + val podWithVolume = new PodBuilder(pod.pod) + .editSpec() + .addNewVolumeLike(configMapVolume) + .endVolume() + .endSpec() + .build() + + val containerWithMount = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(KRB_FILE_VOLUME) + .withMountPath(KRB_FILE_DIR_PATH + "/krb5.conf") + .withSubPath("krb5.conf") + .endVolumeMount() + .build() + + SparkPod(podWithVolume, containerWithMount) + }.transform { + case pod if needKeytabUpload => + // If keytab is defined and is a submission-local file (not local: URI), then create a + // secret for it. The keytab data will be stored in this secret below. + val podWitKeytab = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(KERBEROS_KEYTAB_VOLUME) + .withNewSecret() + .withSecretName(ktSecretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithKeytab = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(KERBEROS_KEYTAB_VOLUME) + .withMountPath(KERBEROS_KEYTAB_MOUNT_POINT) + .endVolumeMount() + .build() + + SparkPod(podWitKeytab, containerWithKeytab) + + case pod if existingSecretName.isDefined | delegationTokens != null => + val secretName = existingSecretName.getOrElse(dtSecretName) + val itemKey = existingSecretItemKey.getOrElse(KERBEROS_SECRET_KEY) + + val podWithTokens = new PodBuilder(pod.pod) + .editOrNewSpec() + .addNewVolume() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withNewSecret() + .withSecretName(secretName) + .endSecret() + .endVolume() + .endSpec() + .build() + + val containerWithTokens = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + .withMountPath(SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR) + .endVolumeMount() + .addNewEnv() + .withName(ENV_HADOOP_TOKEN_FILE_LOCATION) + .withValue(s"$SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR/$itemKey") + .endEnv() + .build() + + SparkPod(podWithTokens, containerWithTokens) + } + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = { + // If a submission-local keytab is provided, update the Spark config so that it knows the + // path of the keytab in the driver container. + if (needKeytabUpload) { + val ktName = new File(keytab.get).getName() + Map(KEYTAB.key -> s"$KERBEROS_KEYTAB_MOUNT_POINT/$ktName") + } else { + Map.empty + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + Seq[HasMetadata]() ++ { + krb5File.map { path => + val file = new File(path) + new ConfigMapBuilder() + .withNewMetadata() + .withName(newConfigMapName) + .endMetadata() + .addToData( + Map(file.getName() -> Files.toString(file, StandardCharsets.UTF_8)).asJava) + .build() + } + } ++ { + // If a submission-local keytab is provided, stash it in a secret. + if (needKeytabUpload) { + val kt = new File(keytab.get) + Seq(new SecretBuilder() + .withNewMetadata() + .withName(ktSecretName) + .endMetadata() + .addToData(kt.getName(), Base64.encodeBase64String(Files.toByteArray(kt))) + .build()) + } else { + Nil + } + } ++ { + if (delegationTokens != null) { + Seq(new SecretBuilder() + .withNewMetadata() + .withName(dtSecretName) + .endMetadata() + .addToData(KERBEROS_SECRET_KEY, Base64.encodeBase64String(delegationTokens)) + .build()) + } else { + Nil + } + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala index 4c1be3bb13293..58cdaa3cadd6b 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/KubernetesFeatureConfigStep.scala @@ -61,11 +61,11 @@ private[spark] trait KubernetesFeatureConfigStep { /** * Return any system properties that should be set on the JVM in accordance to this feature. */ - def getAdditionalPodSystemProperties(): Map[String, String] + def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty /** * Return any additional Kubernetes resources that should be added to support this feature. Only * applicable when creating the driver in cluster mode. */ - def getAdditionalKubernetesResources(): Seq[HasMetadata] + def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala index be386e119d465..19ed2df5551db 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStep.scala @@ -16,16 +16,15 @@ */ package org.apache.spark.deploy.k8s.features -import java.nio.file.Paths import java.util.UUID import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ private[spark] class LocalDirsFeatureStep( - conf: KubernetesConf[_ <: KubernetesRoleSpecificConf], + conf: KubernetesConf, defaultLocalDir: String = s"/var/data/spark-${UUID.randomUUID}") extends KubernetesFeatureConfigStep { @@ -73,8 +72,4 @@ private[spark] class LocalDirsFeatureStep( .build() SparkPod(podWithLocalDirVolumes, containerWithLocalDirVolumeMounts) } - - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala index 97fa9499b2edb..f4e1a3a326729 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStep.scala @@ -18,14 +18,13 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, VolumeBuilder, VolumeMountBuilder} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} -private[spark] class MountSecretsFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountSecretsFeatureStep(kubernetesConf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { val addedVolumes = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .keys .map(secretName => new VolumeBuilder() @@ -40,7 +39,7 @@ private[spark] class MountSecretsFeatureStep( .endSpec() .build() val addedVolumeMounts = kubernetesConf - .roleSecretNamesToMountPaths + .secretNamesToMountPaths .map { case (secretName, mountPath) => new VolumeMountBuilder() @@ -54,9 +53,5 @@ private[spark] class MountSecretsFeatureStep( SparkPod(podWithVolumes, containerWithMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def secretVolumeName(secretName: String): String = s"$secretName-volume" } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala index bb0e2b3128efd..8548e7057cdf0 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStep.scala @@ -20,12 +20,11 @@ import io.fabric8.kubernetes.api.model._ import org.apache.spark.deploy.k8s._ -private[spark] class MountVolumesFeatureStep( - kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf]) +private[spark] class MountVolumesFeatureStep(conf: KubernetesConf) extends KubernetesFeatureConfigStep { override def configurePod(pod: SparkPod): SparkPod = { - val (volumeMounts, volumes) = constructVolumes(kubernetesConf.roleVolumes).unzip + val (volumeMounts, volumes) = constructVolumes(conf.volumes).unzip val podWithVolumes = new PodBuilder(pod.pod) .editSpec() @@ -40,24 +39,22 @@ private[spark] class MountVolumesFeatureStep( SparkPod(podWithVolumes, containerWithVolumeMounts) } - override def getAdditionalPodSystemProperties(): Map[String, String] = Map.empty - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty - private def constructVolumes( - volumeSpecs: Iterable[KubernetesVolumeSpec[_ <: KubernetesVolumeSpecificConf]] + volumeSpecs: Iterable[KubernetesVolumeSpec] ): Iterable[(VolumeMount, Volume)] = { volumeSpecs.map { spec => val volumeMount = new VolumeMountBuilder() .withMountPath(spec.mountPath) .withReadOnly(spec.mountReadOnly) + .withSubPath(spec.mountSubPath) .withName(spec.volumeName) .build() val volumeBuilder = spec.volumeConf match { case KubernetesHostPathVolumeConf(hostPath) => + /* "" means that no checks will be performed before mounting the hostPath volume */ new VolumeBuilder() - .withHostPath(new HostPathVolumeSource(hostPath)) + .withHostPath(new HostPathVolumeSource(hostPath, "")) case KubernetesPVCVolumeConf(claimName) => new VolumeBuilder() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala new file mode 100644 index 0000000000000..7f41ca43589b6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStep.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMapBuilder, ContainerBuilder, HasMetadata, PodBuilder} + +import org.apache.spark.deploy.k8s.{KubernetesConf, SparkPod} +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ + +private[spark] class PodTemplateConfigMapStep(conf: KubernetesConf) + extends KubernetesFeatureConfigStep { + + private val hasTemplate = conf.contains(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + + def configurePod(pod: SparkPod): SparkPod = { + if (hasTemplate) { + val podWithVolume = new PodBuilder(pod.pod) + .editSpec() + .addNewVolume() + .withName(POD_TEMPLATE_VOLUME) + .withNewConfigMap() + .withName(POD_TEMPLATE_CONFIGMAP) + .addNewItem() + .withKey(POD_TEMPLATE_KEY) + .withPath(EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME) + .endItem() + .endConfigMap() + .endVolume() + .endSpec() + .build() + + val containerWithVolume = new ContainerBuilder(pod.container) + .addNewVolumeMount() + .withName(POD_TEMPLATE_VOLUME) + .withMountPath(EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) + .endVolumeMount() + .build() + SparkPod(podWithVolume, containerWithVolume) + } else { + pod + } + } + + override def getAdditionalPodSystemProperties(): Map[String, String] = { + if (hasTemplate) { + Map[String, String]( + KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key -> + (EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) + } else { + Map.empty + } + } + + override def getAdditionalKubernetesResources(): Seq[HasMetadata] = { + if (hasTemplate) { + val podTemplateFile = conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get + val podTemplateString = Files.toString(new File(podTemplateFile), StandardCharsets.UTF_8) + Seq(new ConfigMapBuilder() + .withNewMetadata() + .withName(POD_TEMPLATE_CONFIGMAP) + .endMetadata() + .addToData(POD_TEMPLATE_KEY, podTemplateString) + .build()) + } else { + Nil + } + } +} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala deleted file mode 100644 index 6f063b253cd73..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStep.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata} - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE -import org.apache.spark.deploy.k8s.Constants.SPARK_CONF_PATH -import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep -import org.apache.spark.launcher.SparkLauncher - -private[spark] class JavaDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep { - override def configurePod(pod: SparkPod): SparkPod = { - val withDriverArgs = new ContainerBuilder(pod.container) - .addToArgs("driver") - .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", kubernetesConf.roleSpecificConf.mainClass) - // The user application jar is merged into the spark.jars list and managed through that - // property, so there is no need to reference it explicitly here. - .addToArgs(SparkLauncher.NO_RESOURCE) - .addToArgs(kubernetesConf.roleSpecificConf.appArgs: _*) - .build() - SparkPod(pod.pod, withDriverArgs) - } - override def getAdditionalPodSystemProperties(): Map[String, String] = - Map(APP_RESOURCE_TYPE.key -> "java") - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala deleted file mode 100644 index cf0c03b22bd7e..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStep.scala +++ /dev/null @@ -1,76 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} -import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep - -private[spark] class PythonDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep { - override def configurePod(pod: SparkPod): SparkPod = { - val roleConf = kubernetesConf.roleSpecificConf - require(roleConf.mainAppResource.isDefined, "PySpark Main Resource must be defined") - // Delineation is done by " " because that is input into PythonRunner - val maybePythonArgs = Option(roleConf.appArgs).filter(_.nonEmpty).map( - pyArgs => - new EnvVarBuilder() - .withName(ENV_PYSPARK_ARGS) - .withValue(pyArgs.mkString(" ")) - .build()) - val maybePythonFiles = kubernetesConf.pyFiles().map( - // Dilineation by ":" is to append the PySpark Files to the PYTHONPATH - // of the respective PySpark pod - pyFiles => - new EnvVarBuilder() - .withName(ENV_PYSPARK_FILES) - .withValue(KubernetesUtils.resolveFileUrisAndPath(pyFiles.split(",")) - .mkString(":")) - .build()) - val envSeq = - Seq(new EnvVarBuilder() - .withName(ENV_PYSPARK_PRIMARY) - .withValue(KubernetesUtils.resolveFileUri(kubernetesConf.pySparkMainResource().get)) - .build(), - new EnvVarBuilder() - .withName(ENV_PYSPARK_MAJOR_PYTHON_VERSION) - .withValue(kubernetesConf.pySparkPythonVersion()) - .build()) - val pythonEnvs = envSeq ++ - maybePythonArgs.toSeq ++ - maybePythonFiles.toSeq - - val withPythonPrimaryContainer = new ContainerBuilder(pod.container) - .addAllToEnv(pythonEnvs.asJava) - .addToArgs("driver-py") - .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", roleConf.mainClass) - .build() - - SparkPod(pod.pod, withPythonPrimaryContainer) - } - override def getAdditionalPodSystemProperties(): Map[String, String] = - Map(APP_RESOURCE_TYPE.key -> "python") - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala deleted file mode 100644 index 1a7ef52fefe70..0000000000000 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStep.scala +++ /dev/null @@ -1,62 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{ContainerBuilder, EnvVarBuilder, HasMetadata} - -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkPod} -import org.apache.spark.deploy.k8s.Config.APP_RESOURCE_TYPE -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.features.KubernetesFeatureConfigStep - -private[spark] class RDriverFeatureStep( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]) - extends KubernetesFeatureConfigStep { - override def configurePod(pod: SparkPod): SparkPod = { - val roleConf = kubernetesConf.roleSpecificConf - require(roleConf.mainAppResource.isDefined, "R Main Resource must be defined") - // Delineation is done by " " because that is input into RRunner - val maybeRArgs = Option(roleConf.appArgs).filter(_.nonEmpty).map( - rArgs => - new EnvVarBuilder() - .withName(ENV_R_ARGS) - .withValue(rArgs.mkString(" ")) - .build()) - val envSeq = - Seq(new EnvVarBuilder() - .withName(ENV_R_PRIMARY) - .withValue(KubernetesUtils.resolveFileUri(kubernetesConf.sparkRMainResource().get)) - .build()) - val rEnvs = envSeq ++ - maybeRArgs.toSeq - - val withRPrimaryContainer = new ContainerBuilder(pod.container) - .addAllToEnv(rEnvs.asJava) - .addToArgs("driver-r") - .addToArgs("--properties-file", SPARK_CONF_PATH) - .addToArgs("--class", roleConf.mainClass) - .build() - - SparkPod(pod.pod, withRPrimaryContainer) - } - override def getAdditionalPodSystemProperties(): Map[String, String] = - Map(APP_RESOURCE_TYPE.key -> "r") - - override def getAdditionalKubernetesResources(): Seq[HasMetadata] = Seq.empty -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala new file mode 100644 index 0000000000000..d45c06772c76a --- /dev/null +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOps.scala @@ -0,0 +1,188 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import scala.collection.JavaConverters._ + +import K8SSparkSubmitOperation.getGracePeriod +import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodList} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{NonNamespaceOperation, PodResource} + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.SparkSubmitOperation +import org.apache.spark.deploy.k8s.{KubernetesUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s.Config.{KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, KUBERNETES_SUBMIT_GRACE_PERIOD} +import org.apache.spark.deploy.k8s.Constants.{SPARK_POD_DRIVER_ROLE, SPARK_ROLE_LABEL} +import org.apache.spark.deploy.k8s.KubernetesUtils.formatPodState +import org.apache.spark.util.{CommandLineLoggingUtils, Utils} + +private sealed trait K8sSubmitOp extends CommandLineLoggingUtils { + type NON_NAMESPACED_PODS = + NonNamespaceOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]] + def executeOnPod(pName: String, namespace: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit + def executeOnGlob(pods: List[Pod], ns: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit + def listPodsInNameSpace(namespace: Option[String]) + (implicit client: KubernetesClient): NON_NAMESPACED_PODS = { + namespace match { + case Some(ns) => client.pods.inNamespace(ns) + case None => client.pods + } + } +} + +private class KillApplication extends K8sSubmitOp { + override def executeOnPod(pName: String, namespace: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit = { + val podToDelete = listPodsInNameSpace(namespace).withName(pName) + + if (Option(podToDelete).isDefined) { + getGracePeriod(sparkConf) match { + case Some(period) => podToDelete.withGracePeriod(period).delete() + case _ => podToDelete.delete() + } + } else { + printMessage("Application not found.") + } + } + + override def executeOnGlob(pods: List[Pod], namespace: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit = { + if (pods.nonEmpty) { + pods.foreach { pod => printMessage(s"Deleting driver pod: ${pod.getMetadata.getName}.") } + val listedPods = listPodsInNameSpace(namespace) + + getGracePeriod(sparkConf) match { + case Some(period) => + // this is not using the batch api because no option is provided + // when using the grace period. + pods.foreach { pod => + listedPods + .withName(pod.getMetadata.getName) + .withGracePeriod(period) + .delete() + } + case _ => listedPods.delete(pods.asJava) + } + } else { + printMessage("No applications found.") + } + } +} + +private class ListStatus extends K8sSubmitOp { + override def executeOnPod(pName: String, namespace: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit = { + val pod = listPodsInNameSpace(namespace).withName(pName).get() + if (Option(pod).isDefined) { + printMessage("Application status (driver): " + + Option(pod).map(formatPodState).getOrElse("unknown.")) + } else { + printMessage("Application not found.") + } + } + + override def executeOnGlob(pods: List[Pod], ns: Option[String], sparkConf: SparkConf) + (implicit client: KubernetesClient): Unit = { + if (pods.nonEmpty) { + for (pod <- pods) { + printMessage("Application status (driver): " + + Option(pod).map(formatPodState).getOrElse("unknown.")) + } + } else { + printMessage("No applications found.") + } + } +} + +private[spark] class K8SSparkSubmitOperation extends SparkSubmitOperation + with CommandLineLoggingUtils { + + private def isGlob(name: String): Boolean = { + name.last == '*' + } + + def execute(submissionId: String, sparkConf: SparkConf, op: K8sSubmitOp): Unit = { + val master = KubernetesUtils.parseMasterUrl(sparkConf.get("spark.master")) + submissionId.split(":", 2) match { + case Array(part1, part2@_*) => + val namespace = if (part2.isEmpty) None else Some(part1) + val pName = if (part2.isEmpty) part1 else part2.headOption.get + Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( + master, + namespace, + KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, + SparkKubernetesClientFactory.ClientType.Submission, + sparkConf, + None, + None) + ) { kubernetesClient => + implicit val client: KubernetesClient = kubernetesClient + if (isGlob(pName)) { + val ops = namespace match { + case Some(ns) => + kubernetesClient + .pods + .inNamespace(ns) + case None => + kubernetesClient + .pods + } + val pods = ops + .list() + .getItems + .asScala + .filter { pod => + val meta = pod.getMetadata + meta.getName.startsWith(pName.stripSuffix("*")) && + meta.getLabels.get(SPARK_ROLE_LABEL) == SPARK_POD_DRIVER_ROLE + }.toList + op.executeOnGlob(pods, namespace, sparkConf) + } else { + op.executeOnPod(pName, namespace, sparkConf) + } + } + case _ => + printErrorAndExit(s"Submission ID: {$submissionId} is invalid.") + } + } + + override def kill(submissionId: String, conf: SparkConf): Unit = { + printMessage(s"Submitting a request to kill submission " + + s"${submissionId} in ${conf.get("spark.master")}. " + + s"Grace period in secs: ${getGracePeriod(conf).getOrElse("not set.")}") + execute(submissionId, conf, new KillApplication) + } + + override def printSubmissionStatus(submissionId: String, conf: SparkConf): Unit = { + printMessage(s"Submitting a request for the status of submission" + + s" ${submissionId} in ${conf.get("spark.master")}.") + execute(submissionId, conf, new ListStatus) + } + + override def supports(master: String): Boolean = { + master.startsWith("k8s://") + } +} + +private object K8SSparkSubmitOperation { + def getGracePeriod(sparkConf: SparkConf): Option[Long] = { + sparkConf.get(KUBERNETES_SUBMIT_GRACE_PERIOD) + } +} 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 af3903ac5da56..11bbad9c480a1 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 @@ -17,7 +17,7 @@ package org.apache.spark.deploy.k8s.submit import java.io.StringWriter -import java.util.{Collections, Locale, UUID} +import java.util.{Collections, UUID} import java.util.Properties import io.fabric8.kubernetes.api.model._ @@ -27,7 +27,7 @@ import scala.util.control.NonFatal import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkApplication -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesUtils, SparkKubernetesClientFactory} +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging @@ -39,31 +39,26 @@ import org.apache.spark.util.Utils * @param mainAppResource the main application resource if any * @param mainClass the main class of the application to run * @param driverArgs arguments to the driver - * @param maybePyFiles additional Python files via --py-files */ private[spark] case class ClientArguments( - mainAppResource: Option[MainAppResource], + mainAppResource: MainAppResource, mainClass: String, - driverArgs: Array[String], - maybePyFiles: Option[String]) + driverArgs: Array[String]) private[spark] object ClientArguments { def fromCommandLineArgs(args: Array[String]): ClientArguments = { - var mainAppResource: Option[MainAppResource] = None + var mainAppResource: MainAppResource = JavaMainAppResource(None) var mainClass: Option[String] = None val driverArgs = mutable.ArrayBuffer.empty[String] - var maybePyFiles : Option[String] = None args.sliding(2, 2).toList.foreach { case Array("--primary-java-resource", primaryJavaResource: String) => - mainAppResource = Some(JavaMainAppResource(primaryJavaResource)) + mainAppResource = JavaMainAppResource(Some(primaryJavaResource)) case Array("--primary-py-file", primaryPythonResource: String) => - mainAppResource = Some(PythonMainAppResource(primaryPythonResource)) + mainAppResource = PythonMainAppResource(primaryPythonResource) case Array("--primary-r-file", primaryRFile: String) => - mainAppResource = Some(RMainAppResource(primaryRFile)) - case Array("--other-py-files", pyFiles: String) => - maybePyFiles = Some(pyFiles) + mainAppResource = RMainAppResource(primaryRFile) case Array("--main-class", clazz: String) => mainClass = Some(clazz) case Array("--arg", arg: String) => @@ -78,8 +73,7 @@ private[spark] object ClientArguments { ClientArguments( mainAppResource, mainClass.get, - driverArgs.toArray, - maybePyFiles) + driverArgs.toArray) } } @@ -88,27 +82,24 @@ private[spark] object ClientArguments { * watcher that monitors and logs the application status. Waits for the application to terminate if * spark.kubernetes.submission.waitAppCompletion is true. * + * @param conf The kubernetes driver config. * @param builder Responsible for building the base driver pod based on a composition of * implemented features. - * @param kubernetesConf application configuration * @param kubernetesClient the client to talk to the Kubernetes API server * @param waitForAppCompletion a flag indicating whether the client should wait for the application * to complete - * @param appName the application name * @param watcher a watcher that monitors and logs the application status */ private[spark] class Client( + conf: KubernetesDriverConf, builder: KubernetesDriverBuilder, - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf], kubernetesClient: KubernetesClient, waitForAppCompletion: Boolean, - appName: String, - watcher: LoggingPodStatusWatcher, - kubernetesResourceNamePrefix: String) extends Logging { + watcher: LoggingPodStatusWatcher) extends Logging { def run(): Unit = { - val resolvedDriverSpec = builder.buildFromFeatures(kubernetesConf) - val configMapName = s"$kubernetesResourceNamePrefix-driver-conf-map" + val resolvedDriverSpec = builder.buildFromFeatures(conf, kubernetesClient) + val configMapName = s"${conf.resourceNamePrefix}-driver-conf-map" val configMap = buildConfigMap(configMapName, resolvedDriverSpec.systemProperties) // The include of the ENV_VAR for "SPARK_CONF_DIR" is to allow for the // Spark command builder to pickup on the Java Options present in the ConfigMap @@ -150,12 +141,15 @@ private[spark] class Client( throw e } + val sId = s"${Option(conf.namespace).map(_ + ":").getOrElse("")}" + + s"${resolvedDriverPod.getMetadata.getName}" if (waitForAppCompletion) { - logInfo(s"Waiting for application $appName to finish...") + logInfo(s"Waiting for application ${conf.appName} with submission ID ${sId} to finish...") watcher.awaitCompletion() - logInfo(s"Application $appName finished.") + logInfo(s"Application ${conf.appName} with submission ID ${sId} finished.") } else { - logInfo(s"Deployed Spark application $appName into Kubernetes.") + logInfo(s"Deployed Spark application ${conf.appName} with " + + s"submission ID ${sId} into Kubernetes.") } } } @@ -212,19 +206,12 @@ private[spark] class KubernetesClientApplication extends SparkApplication { // a unique app ID (captured by spark.app.id) in the format below. val kubernetesAppId = s"spark-${UUID.randomUUID().toString.replaceAll("-", "")}" val waitForAppCompletion = sparkConf.get(WAIT_FOR_APP_COMPLETION) - val kubernetesResourceNamePrefix = KubernetesClientApplication.getResourceNamePrefix(appName) - sparkConf.set(KUBERNETES_PYSPARK_PY_FILES, clientArguments.maybePyFiles.getOrElse("")) val kubernetesConf = KubernetesConf.createDriverConf( sparkConf, - appName, - kubernetesResourceNamePrefix, kubernetesAppId, clientArguments.mainAppResource, clientArguments.mainClass, - clientArguments.driverArgs, - clientArguments.maybePyFiles) - val builder = new KubernetesDriverBuilder - val namespace = kubernetesConf.namespace() + clientArguments.driverArgs) // 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")) @@ -234,36 +221,19 @@ private[spark] class KubernetesClientApplication extends SparkApplication { Utils.tryWithResource(SparkKubernetesClientFactory.createKubernetesClient( master, - Some(namespace), + Some(kubernetesConf.namespace), KUBERNETES_AUTH_SUBMISSION_CONF_PREFIX, + SparkKubernetesClientFactory.ClientType.Submission, sparkConf, None, None)) { kubernetesClient => val client = new Client( - builder, kubernetesConf, + new KubernetesDriverBuilder(), kubernetesClient, waitForAppCompletion, - appName, - watcher, - kubernetesResourceNamePrefix) + watcher) client.run() } } } - -private[spark] object KubernetesClientApplication { - - def getAppName(conf: SparkConf): String = conf.getOption("spark.app.name").getOrElse("spark") - - def getResourceNamePrefix(appName: String): String = { - val launchTime = System.currentTimeMillis() - s"$appName-$launchTime" - .trim - .toLowerCase(Locale.ROOT) - .replaceAll("\\s+", "-") - .replaceAll("\\.", "-") - .replaceAll("[^a-z0-9\\-]", "") - .replaceAll("-+", "-") - } -} diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala index 8f3f18ffadc3b..57e4060bc85b9 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilder.scala @@ -16,83 +16,54 @@ */ package org.apache.spark.deploy.k8s.submit -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf} -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep, MountVolumesFeatureStep} -import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} +import java.io.File -private[spark] class KubernetesDriverBuilder( - provideBasicStep: (KubernetesConf[KubernetesDriverSpecificConf]) => BasicDriverFeatureStep = - new BasicDriverFeatureStep(_), - provideCredentialsStep: (KubernetesConf[KubernetesDriverSpecificConf]) - => DriverKubernetesCredentialsFeatureStep = - new DriverKubernetesCredentialsFeatureStep(_), - provideServiceStep: (KubernetesConf[KubernetesDriverSpecificConf]) => DriverServiceFeatureStep = - new DriverServiceFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountSecretsFeatureStep) = - new MountSecretsFeatureStep(_), - provideEnvSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => EnvSecretsFeatureStep) = - new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = - new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = - new MountVolumesFeatureStep(_), - providePythonStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => PythonDriverFeatureStep) = - new PythonDriverFeatureStep(_), - provideRStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => RDriverFeatureStep) = - new RDriverFeatureStep(_), - provideJavaStep: ( - KubernetesConf[KubernetesDriverSpecificConf] - => JavaDriverFeatureStep) = - new JavaDriverFeatureStep(_)) { +import io.fabric8.kubernetes.client.KubernetesClient - def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf]): KubernetesDriverSpec = { - val baseFeatures = Seq( - provideBasicStep(kubernetesConf), - provideCredentialsStep(kubernetesConf), - provideServiceStep(kubernetesConf), - provideLocalDirsStep(kubernetesConf)) +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.features._ + +private[spark] class KubernetesDriverBuilder { - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { - Seq(provideSecretsStep(kubernetesConf)) - } else Nil - val envSecretFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { - Seq(provideEnvSecretsStep(kubernetesConf)) - } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { - Seq(provideVolumesStep(kubernetesConf)) - } else Nil + def buildFromFeatures( + conf: KubernetesDriverConf, + client: KubernetesClient): KubernetesDriverSpec = { + val initialPod = conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE) + .map { file => + KubernetesUtils.loadPodFromTemplate( + client, + new File(file), + conf.get(Config.KUBERNETES_DRIVER_PODTEMPLATE_CONTAINER_NAME)) + } + .getOrElse(SparkPod.initialPod()) - val bindingsStep = kubernetesConf.roleSpecificConf.mainAppResource.map { - case JavaMainAppResource(_) => - provideJavaStep(kubernetesConf) - case PythonMainAppResource(_) => - providePythonStep(kubernetesConf) - case RMainAppResource(_) => - provideRStep(kubernetesConf)} - .getOrElse(provideJavaStep(kubernetesConf)) + val features = Seq( + new BasicDriverFeatureStep(conf), + new DriverKubernetesCredentialsFeatureStep(conf), + new DriverServiceFeatureStep(conf), + new MountSecretsFeatureStep(conf), + new EnvSecretsFeatureStep(conf), + new LocalDirsFeatureStep(conf), + new MountVolumesFeatureStep(conf), + new DriverCommandFeatureStep(conf), + new HadoopConfDriverFeatureStep(conf), + new KerberosConfDriverFeatureStep(conf), + new PodTemplateConfigMapStep(conf)) - val allFeatures = (baseFeatures :+ bindingsStep) ++ - secretFeature ++ envSecretFeature ++ volumesFeature + val spec = KubernetesDriverSpec( + initialPod, + driverKubernetesResources = Seq.empty, + conf.sparkConf.getAll.toMap) - var spec = KubernetesDriverSpec.initialSpec(kubernetesConf.sparkConf.getAll.toMap) - for (feature <- allFeatures) { + features.foldLeft(spec) { case (spec, feature) => val configuredPod = feature.configurePod(spec.pod) val addedSystemProperties = feature.getAdditionalPodSystemProperties() val addedResources = feature.getAdditionalKubernetesResources() - spec = KubernetesDriverSpec( + KubernetesDriverSpec( configuredPod, spec.driverKubernetesResources ++ addedResources, spec.systemProperties ++ addedSystemProperties) } - spec } + } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala index 1889fe5eb3e9b..f16d1f3be7a6c 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/LoggingPodStatusWatcher.scala @@ -18,13 +18,10 @@ package org.apache.spark.deploy.k8s.submit import java.util.concurrent.{CountDownLatch, TimeUnit} -import scala.collection.JavaConverters._ - -import io.fabric8.kubernetes.api.model.{ContainerStateRunning, ContainerStateTerminated, ContainerStateWaiting, ContainerStatus, Pod, Time} +import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action -import org.apache.spark.SparkException import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.internal.Logging import org.apache.spark.util.ThreadUtils @@ -50,9 +47,7 @@ private[k8s] class LoggingPodStatusWatcherImpl( // start timer for periodic logging private val scheduler = ThreadUtils.newDaemonSingleThreadScheduledExecutor("logging-pod-status-watcher") - private val logRunnable: Runnable = new Runnable { - override def run() = logShortStatus() - } + private val logRunnable: Runnable = () => logShortStatus() private var pod = Option.empty[Pod] diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala index dd5a4549743df..a2e01fa2d9a0e 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/deploy/k8s/submit/MainAppResource.scala @@ -20,7 +20,8 @@ private[spark] sealed trait MainAppResource private[spark] sealed trait NonJVMResource -private[spark] case class JavaMainAppResource(primaryResource: String) extends MainAppResource +private[spark] case class JavaMainAppResource(primaryResource: Option[String]) + extends MainAppResource private[spark] case class PythonMainAppResource(primaryResource: String) extends MainAppResource with NonJVMResource diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala index 77bb9c3fcc9f4..da3edfeca9b1f 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocator.scala @@ -22,7 +22,7 @@ import io.fabric8.kubernetes.api.model.PodBuilder import io.fabric8.kubernetes.client.KubernetesClient import scala.collection.mutable -import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.KubernetesConf @@ -31,6 +31,7 @@ import org.apache.spark.util.{Clock, Utils} private[spark] class ExecutorPodsAllocator( conf: SparkConf, + secMgr: SecurityManager, executorBuilder: KubernetesExecutorBuilder, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -51,6 +52,8 @@ private[spark] class ExecutorPodsAllocator( private val kubernetesDriverPodName = conf .get(KUBERNETES_DRIVER_POD_NAME) + private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + private val driverPod = kubernetesDriverPodName .map(name => Option(kubernetesClient.pods() .withName(name) @@ -86,11 +89,16 @@ private[spark] class ExecutorPodsAllocator( s" cluster after $podCreationTimeout milliseconds despite the fact that a" + " previous allocation attempt tried to create it. The executor may have been" + " deleted but the application missed the deletion event.") - Utils.tryLogNonFatalError { - kubernetesClient - .pods() - .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) - .delete() + + if (shouldDeleteExecutors) { + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .withLabel(SPARK_EXECUTOR_ID_LABEL, execId.toString) + .delete() + } } newlyCreatedExecutors -= execId } else { @@ -128,7 +136,8 @@ private[spark] class ExecutorPodsAllocator( newExecutorId.toString, applicationId, driverPod) - val executorPod = executorBuilder.buildFromFeatures(executorConf) + val executorPod = executorBuilder.buildFromFeatures(executorConf, secMgr, + kubernetesClient) val podWithAttachedContainer = new PodBuilder(executorPod.pod) .editOrNewSpec() .addToContainers(executorPod.container) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala index cc254b896249a..95e1ba8362a02 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManager.scala @@ -30,8 +30,7 @@ import org.apache.spark.scheduler.ExecutorExited import org.apache.spark.util.Utils private[spark] class ExecutorPodsLifecycleManager( - conf: SparkConf, - executorBuilder: KubernetesExecutorBuilder, + val conf: SparkConf, kubernetesClient: KubernetesClient, snapshotsStore: ExecutorPodsSnapshotsStore, // Use a best-effort to track which executors have been removed already. It's not generally @@ -44,6 +43,8 @@ private[spark] class ExecutorPodsLifecycleManager( private val eventProcessingInterval = conf.get(KUBERNETES_EXECUTOR_EVENT_PROCESSING_INTERVAL) + private lazy val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + def start(schedulerBackend: KubernetesClusterSchedulerBackend): Unit = { snapshotsStore.addSubscriber(eventProcessingInterval) { onNewSnapshots(schedulerBackend, _) @@ -112,8 +113,10 @@ private[spark] class ExecutorPodsLifecycleManager( execId: Long, schedulerBackend: KubernetesClusterSchedulerBackend, execIdsRemovedInRound: mutable.Set[Long]): Unit = { - removeExecutorFromK8s(podState.pod) removeExecutorFromSpark(schedulerBackend, podState, execId) + if (shouldDeleteExecutors) { + removeExecutorFromK8s(podState.pod) + } execIdsRemovedInRound += execId } diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala index e77e604d00e0f..96a5059df6da3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsPollingSnapshotSource.scala @@ -25,7 +25,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.internal.Logging -import org.apache.spark.util.ThreadUtils +import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class ExecutorPodsPollingSnapshotSource( conf: SparkConf, @@ -53,7 +53,7 @@ private[spark] class ExecutorPodsPollingSnapshotSource( } private class PollRunnable(applicationId: String) extends Runnable { - override def run(): Unit = { + override def run(): Unit = Utils.tryLogNonFatalError { logDebug(s"Resynchronizing full executor pod state from Kubernetes.") snapshotsStore.replaceSnapshot(kubernetesClient .pods() diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala index 5583b4617eeb2..010d93fbf8470 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsSnapshotsStoreImpl.scala @@ -68,7 +68,7 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul } subscribers += newSubscriber pollingTasks += subscribersExecutor.scheduleWithFixedDelay( - toRunnable(() => callSubscriber(newSubscriber)), + () => callSubscriber(newSubscriber), 0L, processBatchIntervalMillis, TimeUnit.MILLISECONDS) @@ -103,10 +103,6 @@ private[spark] class ExecutorPodsSnapshotsStoreImpl(subscribersExecutor: Schedul } } - private def toRunnable[T](runnable: () => Unit): Runnable = new Runnable { - override def run(): Unit = runnable() - } - private case class SnapshotsSubscriber( snapshotsBuffer: BlockingQueue[ExecutorPodsSnapshot], onNewSnapshots: Seq[ExecutorPodsSnapshot] => Unit) diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala index 9999c62c878df..31ca06b721c5d 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterManager.scala @@ -65,10 +65,18 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit apiServerUri, Some(sc.conf.get(KUBERNETES_NAMESPACE)), authConfPrefix, + SparkKubernetesClientFactory.ClientType.Driver, sc.conf, defaultServiceAccountToken, defaultServiceAccountCaCrt) + if (sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).isDefined) { + KubernetesUtils.loadPodFromTemplate( + kubernetesClient, + new File(sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_FILE).get), + sc.conf.get(KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) + } + val requestExecutorsService = ThreadUtils.newDaemonCachedThreadPool( "kubernetes-executor-requests") @@ -81,13 +89,17 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit .build[java.lang.Long, java.lang.Long]() val executorPodsLifecycleEventHandler = new ExecutorPodsLifecycleManager( sc.conf, - new KubernetesExecutorBuilder(), kubernetesClient, snapshotsStore, removedExecutorsCache) val executorPodsAllocator = new ExecutorPodsAllocator( - sc.conf, new KubernetesExecutorBuilder(), kubernetesClient, snapshotsStore, new SystemClock()) + sc.conf, + sc.env.securityManager, + new KubernetesExecutorBuilder(), + kubernetesClient, + snapshotsStore, + new SystemClock()) val podsWatchEventSource = new ExecutorPodsWatchSnapshotSource( snapshotsStore, @@ -100,7 +112,7 @@ private[spark] class KubernetesClusterManager extends ExternalClusterManager wit new KubernetesClusterSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], - sc.env.rpcEnv, + sc, kubernetesClient, requestExecutorsService, snapshotsStore, diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala index fa6dc2c479bbf..4a91a2f5f10b3 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackend.scala @@ -18,18 +18,23 @@ package org.apache.spark.scheduler.cluster.k8s import java.util.concurrent.ExecutorService -import io.fabric8.kubernetes.client.KubernetesClient import scala.concurrent.{ExecutionContext, Future} +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.SparkContext +import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.rpc.{RpcAddress, RpcEnv} +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.internal.config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO +import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef, RpcEnv} import org.apache.spark.scheduler.{ExecutorLossReason, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, SchedulerBackendUtils} import org.apache.spark.util.{ThreadUtils, Utils} private[spark] class KubernetesClusterSchedulerBackend( scheduler: TaskSchedulerImpl, - rpcEnv: RpcEnv, + sc: SparkContext, kubernetesClient: KubernetesClient, requestExecutorsService: ExecutorService, snapshotsStore: ExecutorPodsSnapshotsStore, @@ -37,13 +42,13 @@ private[spark] class KubernetesClusterSchedulerBackend( lifecycleEventHandler: ExecutorPodsLifecycleManager, watchEvents: ExecutorPodsWatchSnapshotSource, pollEvents: ExecutorPodsPollingSnapshotSource) - extends CoarseGrainedSchedulerBackend(scheduler, rpcEnv) { + extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) { - private implicit val requestExecutorContext = ExecutionContext.fromExecutorService( - requestExecutorsService) + private implicit val requestExecutorContext = + ExecutionContext.fromExecutorService(requestExecutorsService) protected override val minRegisteredRatio = - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + if (conf.get(SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO).isEmpty) { 0.8 } else { super.minRegisteredRatio @@ -51,11 +56,24 @@ private[spark] class KubernetesClusterSchedulerBackend( private val initialExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) + private val shouldDeleteExecutors = conf.get(KUBERNETES_DELETE_EXECUTORS) + // Allow removeExecutor to be accessible by ExecutorPodsLifecycleEventHandler private[k8s] def doRemoveExecutor(executorId: String, reason: ExecutorLossReason): Unit = { removeExecutor(executorId, reason) } + /** + * Get an application ID associated with the job. + * This returns the string value of spark.app.id if set, otherwise + * the locally-generated ID from the superclass. + * + * @return The application ID + */ + override def applicationId(): String = { + conf.getOption("spark.app.id").map(_.toString).getOrElse(super.applicationId) + } + override def start(): Unit = { super.start() if (!Utils.isDynamicAllocationEnabled(conf)) { @@ -82,11 +100,14 @@ private[spark] class KubernetesClusterSchedulerBackend( pollEvents.stop() } - Utils.tryLogNonFatalError { - kubernetesClient.pods() - .withLabel(SPARK_APP_ID_LABEL, applicationId()) - .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) - .delete() + if (shouldDeleteExecutors) { + Utils.tryLogNonFatalError { + kubernetesClient + .pods() + .withLabel(SPARK_APP_ID_LABEL, applicationId()) + .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) + .delete() + } } Utils.tryLogNonFatalError { @@ -114,7 +135,8 @@ private[spark] class KubernetesClusterSchedulerBackend( } override def doKillExecutors(executorIds: Seq[String]): Future[Boolean] = Future[Boolean] { - kubernetesClient.pods() + kubernetesClient + .pods() .withLabel(SPARK_APP_ID_LABEL, applicationId()) .withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE) .withLabelIn(SPARK_EXECUTOR_ID_LABEL, executorIds: _*) @@ -122,12 +144,15 @@ private[spark] class KubernetesClusterSchedulerBackend( // Don't do anything else - let event handling from the Kubernetes API do the Spark changes } - override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new KubernetesDriverEndpoint(rpcEnv, properties) + override def createDriverEndpoint(): DriverEndpoint = { + new KubernetesDriverEndpoint() + } + + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint)) } - private class KubernetesDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { + private class KubernetesDriverEndpoint extends DriverEndpoint { override def onDisconnected(rpcAddress: RpcAddress): Unit = { // Don't do anything besides disabling the executor - allow the Kubernetes API events to diff --git a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala index 364b6fb367722..c55488bafab74 100644 --- a/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala +++ b/resource-managers/kubernetes/core/src/main/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilder.scala @@ -16,47 +16,38 @@ */ package org.apache.spark.scheduler.cluster.k8s -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import java.io.File + +import io.fabric8.kubernetes.client.KubernetesClient + +import org.apache.spark.SecurityManager +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.features._ -import org.apache.spark.deploy.k8s.features.{BasicExecutorFeatureStep, EnvSecretsFeatureStep, LocalDirsFeatureStep, MountSecretsFeatureStep} - -private[spark] class KubernetesExecutorBuilder( - provideBasicStep: (KubernetesConf [KubernetesExecutorSpecificConf]) - => BasicExecutorFeatureStep = - new BasicExecutorFeatureStep(_), - provideSecretsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => MountSecretsFeatureStep = - new MountSecretsFeatureStep(_), - provideEnvSecretsStep: - (KubernetesConf[_ <: KubernetesRoleSpecificConf] => EnvSecretsFeatureStep) = - new EnvSecretsFeatureStep(_), - provideLocalDirsStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf]) - => LocalDirsFeatureStep = - new LocalDirsFeatureStep(_), - provideVolumesStep: (KubernetesConf[_ <: KubernetesRoleSpecificConf] - => MountVolumesFeatureStep) = - new MountVolumesFeatureStep(_)) { + +private[spark] class KubernetesExecutorBuilder { def buildFromFeatures( - kubernetesConf: KubernetesConf[KubernetesExecutorSpecificConf]): SparkPod = { - - val baseFeatures = Seq(provideBasicStep(kubernetesConf), provideLocalDirsStep(kubernetesConf)) - val secretFeature = if (kubernetesConf.roleSecretNamesToMountPaths.nonEmpty) { - Seq(provideSecretsStep(kubernetesConf)) - } else Nil - val secretEnvFeature = if (kubernetesConf.roleSecretEnvNamesToKeyRefs.nonEmpty) { - Seq(provideEnvSecretsStep(kubernetesConf)) - } else Nil - val volumesFeature = if (kubernetesConf.roleVolumes.nonEmpty) { - Seq(provideVolumesStep(kubernetesConf)) - } else Nil - - val allFeatures = baseFeatures ++ secretFeature ++ secretEnvFeature ++ volumesFeature - - var executorPod = SparkPod.initialPod() - for (feature <- allFeatures) { - executorPod = feature.configurePod(executorPod) - } - executorPod + conf: KubernetesExecutorConf, + secMgr: SecurityManager, + client: KubernetesClient): SparkPod = { + val initialPod = conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE) + .map { file => + KubernetesUtils.loadPodFromTemplate( + client, + new File(file), + conf.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_CONTAINER_NAME)) + } + .getOrElse(SparkPod.initialPod()) + + val features = Seq( + new BasicExecutorFeatureStep(conf, secMgr), + new ExecutorKubernetesCredentialsFeatureStep(conf), + new MountSecretsFeatureStep(conf), + new EnvSecretsFeatureStep(conf), + new LocalDirsFeatureStep(conf), + new MountVolumesFeatureStep(conf)) + + features.foldLeft(initialPod) { case (pod, feature) => feature.configurePod(pod) } } + } diff --git a/resource-managers/kubernetes/core/src/test/resources/log4j.properties b/resource-managers/kubernetes/core/src/test/resources/log4j.properties index ad95fadb7c0c0..2cf448e4bffbf 100644 --- a/resource-managers/kubernetes/core/src/test/resources/log4j.properties +++ b/resource-managers/kubernetes/core/src/test/resources/log4j.properties @@ -28,4 +28,4 @@ log4j.logger.com.sun.jersey=WARN log4j.logger.org.apache.hadoop=WARN log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.mortbay=WARN -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN 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 e3c19cdb81567..d51b1e661bb1e 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 @@ -26,10 +26,6 @@ import org.apache.spark.deploy.k8s.submit._ class KubernetesConfSuite extends SparkFunSuite { - private val APP_NAME = "test-app" - private val RESOURCE_NAME_PREFIX = "prefix" - private val APP_ID = "test-id" - private val MAIN_CLASS = "test-class" private val APP_ARGS = Array("arg1", "arg2") private val CUSTOM_LABELS = Map( "customLabel1Key" -> "customLabel1Value", @@ -49,119 +45,6 @@ class KubernetesConfSuite extends SparkFunSuite { private val DRIVER_POD = new PodBuilder().build() private val EXECUTOR_ID = "executor-id" - test("Basic driver translated fields.") { - val sparkConf = new SparkConf(false) - val conf = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = None, - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None) - assert(conf.appId === APP_ID) - assert(conf.sparkConf.getAll.toMap === sparkConf.getAll.toMap) - assert(conf.appResourceNamePrefix === RESOURCE_NAME_PREFIX) - assert(conf.roleSpecificConf.appName === APP_NAME) - assert(conf.roleSpecificConf.mainAppResource.isEmpty) - assert(conf.roleSpecificConf.mainClass === MAIN_CLASS) - assert(conf.roleSpecificConf.appArgs === APP_ARGS) - } - - test("Creating driver conf with and without the main app jar influences spark.jars") { - val sparkConf = new SparkConf(false) - .setJars(Seq("local:///opt/spark/jar1.jar")) - val mainAppJar = Some(JavaMainAppResource("local:///opt/spark/main.jar")) - val kubernetesConfWithMainJar = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppJar, - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None) - assert(kubernetesConfWithMainJar.sparkConf.get("spark.jars") - .split(",") - === Array("local:///opt/spark/jar1.jar", "local:///opt/spark/main.jar")) - val kubernetesConfWithoutMainJar = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = None, - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None) - assert(kubernetesConfWithoutMainJar.sparkConf.get("spark.jars").split(",") - === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithoutMainJar.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.1) - } - - test("Creating driver conf with a python primary file") { - val mainResourceFile = "local:///opt/spark/main.py" - val inputPyFiles = Array("local:///opt/spark/example2.py", "local:///example3.py") - val sparkConf = new SparkConf(false) - .setJars(Seq("local:///opt/spark/jar1.jar")) - .set("spark.files", "local:///opt/spark/example4.py") - val mainAppResource = Some(PythonMainAppResource(mainResourceFile)) - val kubernetesConfWithMainResource = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource, - MAIN_CLASS, - APP_ARGS, - Some(inputPyFiles.mkString(","))) - assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") - === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) - assert(kubernetesConfWithMainResource.sparkFiles - === Array("local:///opt/spark/example4.py", mainResourceFile) ++ inputPyFiles) - } - - test("Creating driver conf with a r primary file") { - val mainResourceFile = "local:///opt/spark/main.R" - val sparkConf = new SparkConf(false) - .setJars(Seq("local:///opt/spark/jar1.jar")) - .set("spark.files", "local:///opt/spark/example2.R") - val mainAppResource = Some(RMainAppResource(mainResourceFile)) - val kubernetesConfWithMainResource = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource, - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None) - assert(kubernetesConfWithMainResource.sparkConf.get("spark.jars").split(",") - === Array("local:///opt/spark/jar1.jar")) - assert(kubernetesConfWithMainResource.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.4) - assert(kubernetesConfWithMainResource.sparkFiles - === Array("local:///opt/spark/example2.R", mainResourceFile)) - } - - test("Testing explicit setting of memory overhead on non-JVM tasks") { - val sparkConf = new SparkConf(false) - .set(MEMORY_OVERHEAD_FACTOR, 0.3) - - val mainResourceFile = "local:///opt/spark/main.py" - val mainAppResource = Some(PythonMainAppResource(mainResourceFile)) - val conf = KubernetesConf.createDriverConf( - sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource, - MAIN_CLASS, - APP_ARGS, - None) - assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) - } - test("Resolve driver labels, annotations, secret mount paths, envs, and memory overhead") { val sparkConf = new SparkConf(false) .set(MEMORY_OVERHEAD_FACTOR, 0.3) @@ -183,21 +66,18 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createDriverConf( sparkConf, - APP_NAME, - RESOURCE_NAME_PREFIX, - APP_ID, - mainAppResource = None, - MAIN_CLASS, - APP_ARGS, - maybePyFiles = None) - assert(conf.roleLabels === Map( - SPARK_APP_ID_LABEL -> APP_ID, + KubernetesTestConf.APP_ID, + JavaMainAppResource(None), + KubernetesTestConf.MAIN_CLASS, + APP_ARGS) + assert(conf.labels === Map( + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_DRIVER_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) - assert(conf.roleEnvs === CUSTOM_ENVS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.environment === CUSTOM_ENVS) assert(conf.sparkConf.get(MEMORY_OVERHEAD_FACTOR) === 0.3) } @@ -205,20 +85,20 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( new SparkConf(false), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleSpecificConf.executorId === EXECUTOR_ID) - assert(conf.roleSpecificConf.driverPod.get === DRIVER_POD) + assert(conf.executorId === EXECUTOR_ID) + assert(conf.driverPod.get === DRIVER_POD) } test("Image pull secrets.") { val conf = KubernetesConf.createExecutorConf( new SparkConf(false) - .set(IMAGE_PULL_SECRETS, "my-secret-1,my-secret-2 "), + .set(IMAGE_PULL_SECRETS, Seq("my-secret-1", "my-secret-2 ")), EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.imagePullSecrets() === + assert(conf.imagePullSecrets === Seq( new LocalObjectReferenceBuilder().withName("my-secret-1").build(), new LocalObjectReferenceBuilder().withName("my-secret-2").build())) @@ -242,14 +122,14 @@ class KubernetesConfSuite extends SparkFunSuite { val conf = KubernetesConf.createExecutorConf( sparkConf, EXECUTOR_ID, - APP_ID, + KubernetesTestConf.APP_ID, Some(DRIVER_POD)) - assert(conf.roleLabels === Map( + assert(conf.labels === Map( SPARK_EXECUTOR_ID_LABEL -> EXECUTOR_ID, - SPARK_APP_ID_LABEL -> APP_ID, + SPARK_APP_ID_LABEL -> KubernetesTestConf.APP_ID, SPARK_ROLE_LABEL -> SPARK_POD_EXECUTOR_ROLE) ++ CUSTOM_LABELS) - assert(conf.roleAnnotations === CUSTOM_ANNOTATIONS) - assert(conf.roleSecretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) - assert(conf.roleSecretEnvNamesToKeyRefs === SECRET_ENV_VARS) + assert(conf.annotations === CUSTOM_ANNOTATIONS) + assert(conf.secretNamesToMountPaths === SECRET_NAMES_TO_MOUNT_PATHS) + assert(conf.secretEnvNamesToKeyRefs === SECRET_ENV_VARS) } } 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 new file mode 100644 index 0000000000000..ee830a91f3272 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesTestConf.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.deploy.k8s + +import io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.{JavaMainAppResource, MainAppResource} + +/** + * Builder methods for KubernetesConf that allow easy control over what to return for a few + * properties. For use with tests instead of having to mock specific properties. + */ +object KubernetesTestConf { + + val APP_ID = "appId" + val MAIN_CLASS = "mainClass" + val RESOURCE_PREFIX = "prefix" + val EXECUTOR_ID = "1" + + private val DEFAULT_CONF = new SparkConf(false) + + // scalastyle:off argcount + def createDriverConf( + sparkConf: SparkConf = DEFAULT_CONF, + appId: String = APP_ID, + mainAppResource: MainAppResource = JavaMainAppResource(None), + mainClass: String = MAIN_CLASS, + appArgs: Array[String] = Array.empty, + resourceNamePrefix: Option[String] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesDriverConf = { + val conf = sparkConf.clone() + + resourceNamePrefix.foreach { prefix => + conf.set(KUBERNETES_DRIVER_POD_NAME_PREFIX, prefix) + } + setPrefixedConfigs(conf, KUBERNETES_DRIVER_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ENV_PREFIX, environment) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_DRIVER_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_DRIVER_VOLUMES_PREFIX, volumes) + + new KubernetesDriverConf(conf, appId, mainAppResource, mainClass, appArgs) + } + // scalastyle:on argcount + + def createExecutorConf( + sparkConf: SparkConf = DEFAULT_CONF, + driverPod: Option[Pod] = None, + labels: Map[String, String] = Map.empty, + environment: Map[String, String] = Map.empty, + annotations: Map[String, String] = Map.empty, + secretEnvNamesToKeyRefs: Map[String, String] = Map.empty, + secretNamesToMountPaths: Map[String, String] = Map.empty, + volumes: Seq[KubernetesVolumeSpec] = Seq.empty): KubernetesExecutorConf = { + val conf = sparkConf.clone() + + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_LABEL_PREFIX, labels) + setPrefixedConfigs(conf, "spark.executorEnv.", environment) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_ANNOTATION_PREFIX, annotations) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRETS_PREFIX, secretNamesToMountPaths) + setPrefixedConfigs(conf, KUBERNETES_EXECUTOR_SECRET_KEY_REF_PREFIX, secretEnvNamesToKeyRefs) + setVolumeSpecs(conf, KUBERNETES_EXECUTOR_VOLUMES_PREFIX, volumes) + + new KubernetesExecutorConf(conf, APP_ID, EXECUTOR_ID, driverPod) + } + + private def setPrefixedConfigs( + conf: SparkConf, + prefix: String, + values: Map[String, String]): Unit = { + values.foreach { case (k, v) => + conf.set(s"${prefix}$k", v) + } + } + + private def setVolumeSpecs( + conf: SparkConf, + prefix: String, + volumes: Seq[KubernetesVolumeSpec]): Unit = { + def key(vtype: String, vname: String, subkey: String): String = { + s"${prefix}$vtype.$vname.$subkey" + } + + volumes.foreach { case spec => + val (vtype, configs) = spec.volumeConf match { + case KubernetesHostPathVolumeConf(path) => + (KUBERNETES_VOLUMES_HOSTPATH_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_PATH_KEY -> path)) + + case KubernetesPVCVolumeConf(claimName) => + (KUBERNETES_VOLUMES_PVC_TYPE, + Map(KUBERNETES_VOLUMES_OPTIONS_CLAIM_NAME_KEY -> claimName)) + + case KubernetesEmptyDirVolumeConf(medium, sizeLimit) => + val mconf = medium.map { m => (KUBERNETES_VOLUMES_OPTIONS_MEDIUM_KEY, m) }.toMap + val lconf = sizeLimit.map { l => (KUBERNETES_VOLUMES_OPTIONS_SIZE_LIMIT_KEY, l) }.toMap + (KUBERNETES_VOLUMES_EMPTYDIR_TYPE, mconf ++ lconf) + } + + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_PATH_KEY), spec.mountPath) + if (spec.mountSubPath.nonEmpty) { + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_SUBPATH_KEY), + spec.mountSubPath) + } + conf.set(key(vtype, spec.volumeName, KUBERNETES_VOLUMES_MOUNT_READONLY_KEY), + spec.mountReadOnly.toString) + configs.foreach { case (k, v) => + conf.set(key(vtype, spec.volumeName, k), v) + } + } + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala new file mode 100644 index 0000000000000..7c231586af935 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesUtilsSuite.scala @@ -0,0 +1,68 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.k8s + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model.{Container, ContainerBuilder, PodBuilder} + +import org.apache.spark.SparkFunSuite + +class KubernetesUtilsSuite extends SparkFunSuite { + private val HOST = "test-host" + private val POD = new PodBuilder() + .withNewSpec() + .withHostname(HOST) + .withContainers( + new ContainerBuilder().withName("first").build(), + new ContainerBuilder().withName("second").build()) + .endSpec() + .build() + + test("Selects the given container as spark container.") { + val sparkPod = KubernetesUtils.selectSparkContainer(POD, Some("second")) + assert(sparkPod.pod.getSpec.getHostname == HOST) + assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("first")) + assert(sparkPod.container.getName == "second") + } + + test("Selects the first container if no container name is given.") { + val sparkPod = KubernetesUtils.selectSparkContainer(POD, Option.empty) + assert(sparkPod.pod.getSpec.getHostname == HOST) + assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("second")) + assert(sparkPod.container.getName == "first") + } + + test("Falls back to the first container if given container name does not exist.") { + val sparkPod = KubernetesUtils.selectSparkContainer(POD, Some("does-not-exist")) + assert(sparkPod.pod.getSpec.getHostname == HOST) + assert(sparkPod.pod.getSpec.getContainers.asScala.toList.map(_.getName) == List("second")) + assert(sparkPod.container.getName == "first") + } + + test("constructs spark pod correctly with pod template with no containers") { + val noContainersPod = new PodBuilder(POD).editSpec().withContainers().endSpec().build() + val sparkPod = KubernetesUtils.selectSparkContainer(noContainersPod, Some("does-not-exist")) + assert(sparkPod.pod.getSpec.getHostname == HOST) + assert(sparkPod.container.getName == null) + val sparkPodWithNoContainerName = + KubernetesUtils.selectSparkContainer(noContainersPod, Option.empty) + assert(sparkPodWithNoContainerName.pod.getSpec.getHostname == HOST) + assert(sparkPodWithNoContainerName.container.getName == null) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala index d795d159773a8..16fba354c2afa 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/KubernetesVolumeUtilsSuite.scala @@ -25,24 +25,36 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") - assert(volumeSpec.mountReadOnly === true) + assert(volumeSpec.mountReadOnly) assert(volumeSpec.volumeConf.asInstanceOf[KubernetesHostPathVolumeConf] === KubernetesHostPathVolumeConf("/hostPath")) } + test("Parses subPath correctly") { + val sparkConf = new SparkConf(false) + sparkConf.set("test.emptyDir.volumeName.mount.path", "/path") + sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") + sparkConf.set("test.emptyDir.volumeName.mount.subPath", "subPath") + + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head + assert(volumeSpec.volumeName === "volumeName") + assert(volumeSpec.mountPath === "/path") + assert(volumeSpec.mountSubPath === "subPath") + } + test("Parses persistentVolumeClaim volumes correctly") { val sparkConf = new SparkConf(false) sparkConf.set("test.persistentVolumeClaim.volumeName.mount.path", "/path") sparkConf.set("test.persistentVolumeClaim.volumeName.mount.readOnly", "true") sparkConf.set("test.persistentVolumeClaim.volumeName.options.claimName", "claimeName") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") - assert(volumeSpec.mountReadOnly === true) + assert(volumeSpec.mountReadOnly) assert(volumeSpec.volumeConf.asInstanceOf[KubernetesPVCVolumeConf] === KubernetesPVCVolumeConf("claimeName")) } @@ -54,10 +66,10 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.options.medium", "medium") sparkConf.set("test.emptyDir.volumeName.options.sizeLimit", "5G") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") - assert(volumeSpec.mountReadOnly === true) + assert(volumeSpec.mountReadOnly) assert(volumeSpec.volumeConf.asInstanceOf[KubernetesEmptyDirVolumeConf] === KubernetesEmptyDirVolumeConf(Some("medium"), Some("5G"))) } @@ -67,10 +79,10 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.emptyDir.volumeName.mount.path", "/path") sparkConf.set("test.emptyDir.volumeName.mount.readOnly", "true") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.volumeName === "volumeName") assert(volumeSpec.mountPath === "/path") - assert(volumeSpec.mountReadOnly === true) + assert(volumeSpec.mountReadOnly) assert(volumeSpec.volumeConf.asInstanceOf[KubernetesEmptyDirVolumeConf] === KubernetesEmptyDirVolumeConf(None, None)) } @@ -80,27 +92,29 @@ class KubernetesVolumeUtilsSuite extends SparkFunSuite { sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.options.path", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head.get + val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head assert(volumeSpec.mountReadOnly === false) } - test("Gracefully fails on missing mount key") { + test("Fails on missing mount key") { val sparkConf = new SparkConf(false) sparkConf.set("test.emptyDir.volumeName.mnt.path", "/path") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "emptyDir.volumeName.mount.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("emptyDir.volumeName.mount.path")) } - test("Gracefully fails on missing option key") { + test("Fails on missing option key") { val sparkConf = new SparkConf(false) sparkConf.set("test.hostPath.volumeName.mount.path", "/path") sparkConf.set("test.hostPath.volumeName.mount.readOnly", "true") sparkConf.set("test.hostPath.volumeName.options.pth", "/hostPath") - val volumeSpec = KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.").head - assert(volumeSpec.isFailure === true) - assert(volumeSpec.failed.get.getMessage === "hostPath.volumeName.options.path") + val e = intercept[NoSuchElementException] { + KubernetesVolumeUtils.parseVolumesWithPrefix(sparkConf, "test.") + } + assert(e.getMessage.contains("hostPath.volumeName.options.path")) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala new file mode 100644 index 0000000000000..707c823d69cf0 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/PodBuilderSuite.scala @@ -0,0 +1,177 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s + +import java.io.File + +import io.fabric8.kubernetes.api.model.{Config => _, _} +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.{MixedOperation, PodResource} +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, never, verify, when} +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.internal.config.ConfigEntry + +abstract class PodBuilderSuite extends SparkFunSuite { + + protected def templateFileConf: ConfigEntry[_] + + protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod + + private val baseConf = new SparkConf(false) + .set(Config.CONTAINER_IMAGE, "spark-executor:latest") + + test("use empty initial pod if template is not specified") { + val client = mock(classOf[KubernetesClient]) + buildPod(baseConf.clone(), client) + verify(client, never()).pods() + } + + test("load pod template if specified") { + val client = mockKubernetesClient() + val sparkConf = baseConf.clone().set(templateFileConf.key, "template-file.yaml") + val pod = buildPod(sparkConf, client) + verifyPod(pod) + } + + test("complain about misconfigured pod template") { + val client = mockKubernetesClient( + new PodBuilder() + .withNewMetadata() + .addToLabels("test-label-key", "test-label-value") + .endMetadata() + .build()) + val sparkConf = baseConf.clone().set(templateFileConf.key, "template-file.yaml") + val exception = intercept[SparkException] { + buildPod(sparkConf, client) + } + assert(exception.getMessage.contains("Could not load pod from template file.")) + } + + private def mockKubernetesClient(pod: Pod = podWithSupportedFeatures()): KubernetesClient = { + val kubernetesClient = mock(classOf[KubernetesClient]) + val pods = + mock(classOf[MixedOperation[Pod, PodList, DoneablePod, PodResource[Pod, DoneablePod]]]) + val podResource = mock(classOf[PodResource[Pod, DoneablePod]]) + when(kubernetesClient.pods()).thenReturn(pods) + when(pods.load(any(classOf[File]))).thenReturn(podResource) + when(podResource.get()).thenReturn(pod) + kubernetesClient + } + + private def verifyPod(pod: SparkPod): Unit = { + val metadata = pod.pod.getMetadata + assert(metadata.getLabels.containsKey("test-label-key")) + assert(metadata.getAnnotations.containsKey("test-annotation-key")) + assert(metadata.getNamespace === "namespace") + assert(metadata.getOwnerReferences.asScala.exists(_.getName == "owner-reference")) + val spec = pod.pod.getSpec + assert(!spec.getContainers.asScala.exists(_.getName == "executor-container")) + assert(spec.getDnsPolicy === "dns-policy") + assert(spec.getHostAliases.asScala.exists(_.getHostnames.asScala.exists(_ == "hostname"))) + assert(spec.getImagePullSecrets.asScala.exists(_.getName == "local-reference")) + assert(spec.getInitContainers.asScala.exists(_.getName == "init-container")) + assert(spec.getNodeName == "node-name") + assert(spec.getNodeSelector.get("node-selector-key") === "node-selector-value") + assert(spec.getSchedulerName === "scheduler") + assert(spec.getSecurityContext.getRunAsUser === 1000L) + assert(spec.getServiceAccount === "service-account") + assert(spec.getSubdomain === "subdomain") + assert(spec.getTolerations.asScala.exists(_.getKey == "toleration-key")) + assert(spec.getVolumes.asScala.exists(_.getName == "test-volume")) + val container = pod.container + assert(container.getName === "executor-container") + assert(container.getArgs.contains("arg")) + assert(container.getCommand.equals(List("command").asJava)) + assert(container.getEnv.asScala.exists(_.getName == "env-key")) + assert(container.getResources.getLimits.get("gpu") === + new QuantityBuilder().withAmount("1").build()) + assert(container.getSecurityContext.getRunAsNonRoot) + assert(container.getStdin) + assert(container.getTerminationMessagePath === "termination-message-path") + assert(container.getTerminationMessagePolicy === "termination-message-policy") + assert(pod.container.getVolumeMounts.asScala.exists(_.getName == "test-volume")) + } + + private def podWithSupportedFeatures(): Pod = { + new PodBuilder() + .withNewMetadata() + .addToLabels("test-label-key", "test-label-value") + .addToAnnotations("test-annotation-key", "test-annotation-value") + .withNamespace("namespace") + .addNewOwnerReference() + .withController(true) + .withName("owner-reference") + .endOwnerReference() + .endMetadata() + .withNewSpec() + .withDnsPolicy("dns-policy") + .withHostAliases(new HostAliasBuilder().withHostnames("hostname").build()) + .withImagePullSecrets( + new LocalObjectReferenceBuilder().withName("local-reference").build()) + .withInitContainers(new ContainerBuilder().withName("init-container").build()) + .withNodeName("node-name") + .withNodeSelector(Map("node-selector-key" -> "node-selector-value").asJava) + .withSchedulerName("scheduler") + .withNewSecurityContext() + .withRunAsUser(1000L) + .endSecurityContext() + .withServiceAccount("service-account") + .withSubdomain("subdomain") + .withTolerations(new TolerationBuilder() + .withKey("toleration-key") + .withOperator("Equal") + .withEffect("NoSchedule") + .build()) + .addNewVolume() + .withNewHostPath() + .withPath("/test") + .endHostPath() + .withName("test-volume") + .endVolume() + .addNewContainer() + .withArgs("arg") + .withCommand("command") + .addNewEnv() + .withName("env-key") + .withValue("env-value") + .endEnv() + .withImagePullPolicy("Always") + .withName("executor-container") + .withNewResources() + .withLimits(Map("gpu" -> new QuantityBuilder().withAmount("1").build()).asJava) + .endResources() + .withNewSecurityContext() + .withRunAsNonRoot(true) + .endSecurityContext() + .withStdin(true) + .withTerminationMessagePath("termination-message-path") + .withTerminationMessagePolicy("termination-message-policy") + .addToVolumeMounts( + new VolumeMountBuilder() + .withName("test-volume") + .withMountPath("/test") + .build()) + .endContainer() + .endSpec() + .build() + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala index 0968cce971c31..370672162ef0d 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicDriverFeatureStepSuite.scala @@ -21,88 +21,78 @@ import scala.collection.JavaConverters._ import io.fabric8.kubernetes.api.model.{ContainerPort, ContainerPortBuilder, LocalObjectReferenceBuilder} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.JavaMainAppResource -import org.apache.spark.deploy.k8s.submit.PythonMainAppResource -import org.apache.spark.ui.SparkUI +import org.apache.spark.deploy.k8s.features.KubernetesFeaturesTestUtils.TestResourceInformation +import org.apache.spark.deploy.k8s.submit._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.util.Utils class BasicDriverFeatureStepSuite extends SparkFunSuite { - private val APP_ID = "spark-app-id" - private val RESOURCE_NAME_PREFIX = "spark" private val DRIVER_LABELS = Map("labelkey" -> "labelvalue") private val CONTAINER_IMAGE_PULL_POLICY = "IfNotPresent" - private val APP_NAME = "spark-test" - private val MAIN_CLASS = "org.apache.spark.examples.SparkPi" - private val PY_MAIN_CLASS = "org.apache.spark.deploy.PythonRunner" - private val APP_ARGS = Array("arg1", "arg2", "\"arg 3\"") - private val CUSTOM_ANNOTATION_KEY = "customAnnotation" - private val CUSTOM_ANNOTATION_VALUE = "customAnnotationValue" - private val DRIVER_ANNOTATIONS = Map(CUSTOM_ANNOTATION_KEY -> CUSTOM_ANNOTATION_VALUE) - private val DRIVER_CUSTOM_ENV1 = "customDriverEnv1" - private val DRIVER_CUSTOM_ENV2 = "customDriverEnv2" + private val DRIVER_ANNOTATIONS = Map("customAnnotation" -> "customAnnotationValue") private val DRIVER_ENVS = Map( - DRIVER_CUSTOM_ENV1 -> DRIVER_CUSTOM_ENV1, - DRIVER_CUSTOM_ENV2 -> DRIVER_CUSTOM_ENV2) + "customDriverEnv1" -> "customDriverEnv2", + "customDriverEnv2" -> "customDriverEnv2") private val TEST_IMAGE_PULL_SECRETS = Seq("my-secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => new LocalObjectReferenceBuilder().withName(secret).build() } - private val emptyDriverSpecificConf = KubernetesDriverSpecificConf( - None, - APP_NAME, - MAIN_CLASS, - APP_ARGS) test("Check the pod respects all configurations from the user.") { + val resourceID = ResourceID(SPARK_DRIVER_PREFIX, GPU) + val resources = + Map(("nvidia.com/gpu" -> TestResourceInformation(resourceID, "2", "nvidia.com"))) val sparkConf = new SparkConf() .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") - .set("spark.driver.cores", "2") + .set(DRIVER_CORES, 2) .set(KUBERNETES_DRIVER_LIMIT_CORES, "4") - .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "256M") - .set(org.apache.spark.internal.config.DRIVER_MEMORY_OVERHEAD, 200L) + .set(DRIVER_MEMORY.key, "256M") + .set(DRIVER_MEMORY_OVERHEAD, 200L) .set(CONTAINER_IMAGE, "spark-driver:latest") - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - Seq.empty[String]) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) + resources.foreach { case (_, testRInfo) => + sparkConf.set(testRInfo.rId.amountConf, testRInfo.count) + sparkConf.set(testRInfo.rId.vendorConf, testRInfo.vendor) + } + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS, + environment = DRIVER_ENVS, + annotations = DRIVER_ANNOTATIONS) val featureStep = new BasicDriverFeatureStep(kubernetesConf) val basePod = SparkPod.initialPod() val configuredPod = featureStep.configurePod(basePod) - assert(configuredPod.container.getName === DRIVER_CONTAINER_NAME) + assert(configuredPod.container.getName === DEFAULT_DRIVER_CONTAINER_NAME) assert(configuredPod.container.getImage === "spark-driver:latest") assert(configuredPod.container.getImagePullPolicy === CONTAINER_IMAGE_PULL_POLICY) val expectedPortNames = Set( containerPort(DRIVER_PORT_NAME, DEFAULT_DRIVER_PORT), containerPort(BLOCK_MANAGER_PORT_NAME, DEFAULT_BLOCKMANAGER_PORT), - containerPort(UI_PORT_NAME, SparkUI.DEFAULT_PORT) + containerPort(UI_PORT_NAME, UI_PORT.defaultValue.get) ) val foundPortNames = configuredPod.container.getPorts.asScala.toSet assert(expectedPortNames === foundPortNames) - assert(configuredPod.container.getEnv.size === 3) val envs = configuredPod.container .getEnv .asScala - .map(env => (env.getName, env.getValue)) + .map { env => (env.getName, env.getValue) } .toMap - assert(envs(DRIVER_CUSTOM_ENV1) === DRIVER_ENVS(DRIVER_CUSTOM_ENV1)) - assert(envs(DRIVER_CUSTOM_ENV2) === DRIVER_ENVS(DRIVER_CUSTOM_ENV2)) + DRIVER_ENVS.foreach { case (k, v) => + assert(envs(v) === v) + } + assert(envs(ENV_SPARK_USER) === Utils.getCurrentUserName()) assert(configuredPod.pod.getSpec().getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) @@ -119,97 +109,107 @@ class BasicDriverFeatureStepSuite extends SparkFunSuite { val limits = resourceRequirements.getLimits.asScala assert(limits("memory").getAmount === "456Mi") assert(limits("cpu").getAmount === "4") + resources.foreach { case (k8sName, testRInfo) => + assert(limits(k8sName).getAmount === testRInfo.count) + } val driverPodMetadata = configuredPod.pod.getMetadata assert(driverPodMetadata.getName === "spark-driver-pod") - assert(driverPodMetadata.getLabels.asScala === DRIVER_LABELS) + DRIVER_LABELS.foreach { case (k, v) => + assert(driverPodMetadata.getLabels.get(k) === v) + } assert(driverPodMetadata.getAnnotations.asScala === DRIVER_ANNOTATIONS) assert(configuredPod.pod.getSpec.getRestartPolicy === "Never") val expectedSparkConf = Map( KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, - "spark.kubernetes.submitInDriver" -> "true") + "spark.app.id" -> KubernetesTestConf.APP_ID, + KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> kubernetesConf.resourceNamePrefix, + "spark.kubernetes.submitInDriver" -> "true", + MEMORY_OVERHEAD_FACTOR.key -> MEMORY_OVERHEAD_FACTOR.defaultValue.get.toString) assert(featureStep.getAdditionalPodSystemProperties() === expectedSparkConf) } + test("Check driver pod respects kubernetes driver request cores") { + val sparkConf = new SparkConf() + .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") + .set(CONTAINER_IMAGE, "spark-driver:latest") + + val basePod = SparkPod.initialPod() + // if spark.driver.cores is not set default is 1 + val requests1 = new BasicDriverFeatureStep(KubernetesTestConf.createDriverConf(sparkConf)) + .configurePod(basePod) + .container.getResources + .getRequests.asScala + assert(requests1("cpu").getAmount === "1") + + // if spark.driver.cores is set it should be used + sparkConf.set(DRIVER_CORES, 10) + val requests2 = new BasicDriverFeatureStep(KubernetesTestConf.createDriverConf(sparkConf)) + .configurePod(basePod) + .container.getResources + .getRequests.asScala + assert(requests2("cpu").getAmount === "10") + + // spark.kubernetes.driver.request.cores should be preferred over spark.driver.cores + Seq("0.1", "100m").foreach { value => + sparkConf.set(KUBERNETES_DRIVER_REQUEST_CORES, value) + val requests3 = new BasicDriverFeatureStep(KubernetesTestConf.createDriverConf(sparkConf)) + .configurePod(basePod) + .container.getResources + .getRequests.asScala + assert(requests3("cpu").getAmount === value) + } + } + test("Check appropriate entrypoint rerouting for various bindings") { val javaSparkConf = new SparkConf() - .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") + .set(DRIVER_MEMORY.key, "4g") .set(CONTAINER_IMAGE, "spark-driver:latest") val pythonSparkConf = new SparkConf() - .set(org.apache.spark.internal.config.DRIVER_MEMORY.key, "4g") - .set(CONTAINER_IMAGE, "spark-driver:latest") - val javaKubernetesConf = KubernetesConf( - javaSparkConf, - KubernetesDriverSpecificConf( - Some(JavaMainAppResource("")), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - Seq.empty[String]) - val pythonKubernetesConf = KubernetesConf( - pythonSparkConf, - KubernetesDriverSpecificConf( - Some(PythonMainAppResource("")), - APP_NAME, - PY_MAIN_CLASS, - APP_ARGS), - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - Seq.empty[String]) + .set(DRIVER_MEMORY.key, "4g") + .set(CONTAINER_IMAGE, "spark-driver-py:latest") + val javaKubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = javaSparkConf) + val pythonKubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = pythonSparkConf, + mainAppResource = PythonMainAppResource("")) val javaFeatureStep = new BasicDriverFeatureStep(javaKubernetesConf) val pythonFeatureStep = new BasicDriverFeatureStep(pythonKubernetesConf) val basePod = SparkPod.initialPod() val configuredJavaPod = javaFeatureStep.configurePod(basePod) val configuredPythonPod = pythonFeatureStep.configurePod(basePod) + assert(configuredJavaPod.container.getImage === "spark-driver:latest") + assert(configuredPythonPod.container.getImage === "spark-driver-py:latest") } - test("Additional system properties resolve jars and set cluster-mode confs.") { - val allJars = Seq("local:///opt/spark/jar1.jar", "hdfs:///opt/spark/jar2.jar") - val allFiles = Seq("https://localhost:9000/file1.txt", "local:///opt/spark/file2.txt") - val sparkConf = new SparkConf() - .set(KUBERNETES_DRIVER_POD_NAME, "spark-driver-pod") - .setJars(allJars) - .set("spark.files", allFiles.mkString(",")) - .set(CONTAINER_IMAGE, "spark-driver:latest") - val kubernetesConf = KubernetesConf( - sparkConf, - emptyDriverSpecificConf, - RESOURCE_NAME_PREFIX, - APP_ID, - DRIVER_LABELS, - DRIVER_ANNOTATIONS, - Map.empty, - Map.empty, - DRIVER_ENVS, - Nil, - allFiles) - - val step = new BasicDriverFeatureStep(kubernetesConf) - val additionalProperties = step.getAdditionalPodSystemProperties() - val expectedSparkConf = Map( - KUBERNETES_DRIVER_POD_NAME.key -> "spark-driver-pod", - "spark.app.id" -> APP_ID, - KUBERNETES_EXECUTOR_POD_NAME_PREFIX.key -> RESOURCE_NAME_PREFIX, - "spark.kubernetes.submitInDriver" -> "true", - "spark.jars" -> "/opt/spark/jar1.jar,hdfs:///opt/spark/jar2.jar", - "spark.files" -> "https://localhost:9000/file1.txt,/opt/spark/file2.txt") - assert(additionalProperties === expectedSparkConf) + // Memory overhead tests. Tuples are: + // test name, main resource, overhead factor, expected factor + Seq( + ("java", JavaMainAppResource(None), None, MEMORY_OVERHEAD_FACTOR.defaultValue.get), + ("python default", PythonMainAppResource(null), None, NON_JVM_MEMORY_OVERHEAD_FACTOR), + ("python w/ override", PythonMainAppResource(null), Some(0.9d), 0.9d), + ("r default", RMainAppResource(null), None, NON_JVM_MEMORY_OVERHEAD_FACTOR) + ).foreach { case (name, resource, factor, expectedFactor) => + test(s"memory overhead factor: $name") { + // Choose a driver memory where the default memory overhead is > MEMORY_OVERHEAD_MIN_MIB + val driverMem = MEMORY_OVERHEAD_MIN_MIB / MEMORY_OVERHEAD_FACTOR.defaultValue.get * 2 + + // main app resource, overhead factor + val sparkConf = new SparkConf(false) + .set(CONTAINER_IMAGE, "spark-driver:latest") + .set(DRIVER_MEMORY.key, s"${driverMem.toInt}m") + factor.foreach { value => sparkConf.set(MEMORY_OVERHEAD_FACTOR, value) } + val conf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + mainAppResource = resource) + val step = new BasicDriverFeatureStep(conf) + val pod = step.configurePod(SparkPod.initialPod()) + val mem = pod.container.getResources.getRequests.get("memory").getAmount() + val expected = (driverMem + driverMem * expectedFactor).toInt + assert(mem === s"${expected}Mi") + + val systemProperties = step.getAdditionalPodSystemProperties() + assert(systemProperties(MEMORY_OVERHEAD_FACTOR.key) === expectedFactor.toString) + } } def containerPort(name: String, portNumber: Int): ContainerPort = diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala index 63b237b9dfe46..51067bd889a58 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/BasicExecutorFeatureStepSuite.scala @@ -16,23 +16,33 @@ */ package org.apache.spark.deploy.k8s.features +import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.Files + import scala.collection.JavaConverters._ +import com.google.common.net.InternetDomainName import io.fabric8.kubernetes.api.model._ -import org.mockito.MockitoAnnotations -import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach} +import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.{SecurityManager, SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.features.KubernetesFeaturesTestUtils.TestResourceInformation +import org.apache.spark.internal.config +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils._ +import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointAddress import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend +import org.apache.spark.util.Utils -class BasicExecutorFeatureStepSuite - extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterEach { +class BasicExecutorFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { - private val APP_ID = "app-id" private val DRIVER_HOSTNAME = "localhost" private val DRIVER_PORT = 7098 private val DRIVER_ADDRESS = RpcEndpointAddress( @@ -45,7 +55,6 @@ class BasicExecutorFeatureStepSuite private val RESOURCE_NAME_PREFIX = "base" private val EXECUTOR_IMAGE = "executor-image" private val LABELS = Map("label1key" -> "label1value") - private val ANNOTATIONS = Map("annotation1key" -> "annotation1value") private val TEST_IMAGE_PULL_SECRETS = Seq("my-1secret-1", "my-secret-2") private val TEST_IMAGE_PULL_SECRET_OBJECTS = TEST_IMAGE_PULL_SECRETS.map { secret => @@ -66,37 +75,75 @@ class BasicExecutorFeatureStepSuite private var baseConf: SparkConf = _ before { - MockitoAnnotations.initMocks(this) - baseConf = new SparkConf() + baseConf = new SparkConf(false) .set(KUBERNETES_DRIVER_POD_NAME, DRIVER_POD_NAME) .set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, RESOURCE_NAME_PREFIX) .set(CONTAINER_IMAGE, EXECUTOR_IMAGE) .set(KUBERNETES_DRIVER_SUBMIT_CHECK, true) - .set("spark.driver.host", DRIVER_HOSTNAME) - .set("spark.driver.port", DRIVER_PORT.toString) - .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS.mkString(",")) + .set(config.DRIVER_HOST_ADDRESS, DRIVER_HOSTNAME) + .set(config.DRIVER_PORT, DRIVER_PORT) + .set(IMAGE_PULL_SECRETS, TEST_IMAGE_PULL_SECRETS) .set("spark.kubernetes.resource.type", "java") } + private def newExecutorConf( + environment: Map[String, String] = Map.empty): KubernetesExecutorConf = { + KubernetesTestConf.createExecutorConf( + sparkConf = baseConf, + driverPod = Some(DRIVER_POD), + labels = LABELS, + environment = environment) + } + + test("test spark resource missing vendor") { + baseConf.set(EXECUTOR_GPU_ID.amountConf, "2") + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) + val error = intercept[SparkException] { + val executor = step.configurePod(SparkPod.initialPod()) + }.getMessage() + assert(error.contains("Resource: gpu was requested, but vendor was not specified")) + } + + test("test spark resource missing amount") { + baseConf.set(EXECUTOR_GPU_ID.vendorConf, "nvidia.com") + + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) + val error = intercept[SparkException] { + val executor = step.configurePod(SparkPod.initialPod()) + }.getMessage() + assert(error.contains("You must specify an amount for gpu")) + } + + test("basic executor pod with resources") { + val fpgaResourceID = ResourceID(SPARK_EXECUTOR_PREFIX, FPGA) + val gpuExecutorResourceID = ResourceID(SPARK_EXECUTOR_PREFIX, GPU) + val gpuResources = + Map(("nvidia.com/gpu" -> TestResourceInformation(gpuExecutorResourceID, "2", "nvidia.com")), + ("foo.com/fpga" -> TestResourceInformation(fpgaResourceID, "1", "foo.com"))) + gpuResources.foreach { case (_, testRInfo) => + baseConf.set(testRInfo.rId.amountConf, testRInfo.count) + baseConf.set(testRInfo.rId.vendorConf, testRInfo.vendor) + } + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) + val executor = step.configurePod(SparkPod.initialPod()) + + assert(executor.container.getResources.getLimits.size() === 3) + assert(executor.container.getResources + .getLimits.get("memory").getAmount === "1408Mi") + gpuResources.foreach { case (k8sName, testRInfo) => + assert(executor.container.getResources.getLimits.get(k8sName).getAmount === testRInfo.count) + } + } + test("basic executor pod has reasonable defaults") { - val step = new BasicExecutorFeatureStep( - KubernetesConf( - baseConf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) // The executor pod name and default labels. assert(executor.pod.getMetadata.getName === s"$RESOURCE_NAME_PREFIX-exec-1") - assert(executor.pod.getMetadata.getLabels.asScala === LABELS) + LABELS.foreach { case (k, v) => + assert(executor.pod.getMetadata.getLabels.get(k) === v) + } assert(executor.pod.getSpec.getImagePullSecrets.asScala === TEST_IMAGE_PULL_SECRET_OBJECTS) // There is exactly 1 container with no volume mounts and default memory limits. @@ -111,51 +158,36 @@ class BasicExecutorFeatureStepSuite assert(executor.pod.getSpec.getNodeSelector.isEmpty) assert(executor.pod.getSpec.getVolumes.isEmpty) - checkEnv(executor, Map()) + checkEnv(executor, baseConf, Map()) checkOwnerReferences(executor.pod, DRIVER_POD_UID) } test("executor pod hostnames get truncated to 63 characters") { - val conf = baseConf.clone() val longPodNamePrefix = "loremipsumdolorsitametvimatelitrefficiendisuscipianturvixlegeresple" - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - longPodNamePrefix, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) + baseConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, longPodNamePrefix) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) assert(step.configurePod(SparkPod.initialPod()).pod.getSpec.getHostname.length === 63) } + test("hostname truncation generates valid host names") { + val invalidPrefix = "abcdef-*_/[]{}+==.,;'\"-----------------------------------------------" + + baseConf.set(KUBERNETES_EXECUTOR_POD_NAME_PREFIX, invalidPrefix) + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) + val hostname = step.configurePod(SparkPod.initialPod()).pod.getSpec().getHostname() + assert(hostname.length <= 63) + assert(InternetDomainName.isValid(hostname)) + } + test("classpath and extra java options get translated into environment variables") { - val conf = baseConf.clone() - conf.set(org.apache.spark.internal.config.EXECUTOR_JAVA_OPTIONS, "foo=bar") - conf.set(org.apache.spark.internal.config.EXECUTOR_CLASS_PATH, "bar=baz") - - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map("qux" -> "quux"), - Nil, - Seq.empty[String])) + baseConf.set(config.EXECUTOR_JAVA_OPTIONS, "foo=bar") + baseConf.set(config.EXECUTOR_CLASS_PATH, "bar=baz") + val kconf = newExecutorConf(environment = Map("qux" -> "quux")) + val step = new BasicExecutorFeatureStep(kconf, new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) - checkEnv(executor, + checkEnv(executor, baseConf, Map("SPARK_JAVA_OPT_0" -> "foo=bar", ENV_CLASSPATH -> "bar=baz", "qux" -> "quux")) @@ -163,50 +195,82 @@ class BasicExecutorFeatureStepSuite } test("test executor pyspark memory") { - val conf = baseConf.clone() - conf.set("spark.kubernetes.resource.type", "python") - conf.set(org.apache.spark.internal.config.PYSPARK_EXECUTOR_MEMORY, 42L) - - val step = new BasicExecutorFeatureStep( - KubernetesConf( - conf, - KubernetesExecutorSpecificConf("1", Some(DRIVER_POD)), - RESOURCE_NAME_PREFIX, - APP_ID, - LABELS, - ANNOTATIONS, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) + baseConf.set("spark.kubernetes.resource.type", "python") + baseConf.set(PYSPARK_EXECUTOR_MEMORY, 42L) + + val step = new BasicExecutorFeatureStep(newExecutorConf(), new SecurityManager(baseConf)) val executor = step.configurePod(SparkPod.initialPod()) // This is checking that basic executor + executorMemory = 1408 + 42 = 1450 assert(executor.container.getResources.getRequests.get("memory").getAmount === "1450Mi") } + test("auth secret propagation") { + val conf = baseConf.clone() + .set(config.NETWORK_AUTH_ENABLED, true) + .set("spark.master", "k8s://127.0.0.1") + + val secMgr = new SecurityManager(conf) + secMgr.initializeAuth() + + val step = new BasicExecutorFeatureStep(KubernetesTestConf.createExecutorConf(sparkConf = conf), + secMgr) + + val executor = step.configurePod(SparkPod.initialPod()) + checkEnv(executor, conf, Map(SecurityManager.ENV_AUTH_SECRET -> secMgr.getSecretKey())) + } + + test("Auth secret shouldn't propagate if files are loaded.") { + val secretDir = Utils.createTempDir("temp-secret") + val secretFile = new File(secretDir, "secret-file.txt") + Files.write(secretFile.toPath, "some-secret".getBytes(StandardCharsets.UTF_8)) + val conf = baseConf.clone() + .set(config.NETWORK_AUTH_ENABLED, true) + .set(config.AUTH_SECRET_FILE, secretFile.getAbsolutePath) + .set("spark.master", "k8s://127.0.0.1") + val secMgr = new SecurityManager(conf) + secMgr.initializeAuth() + + val step = new BasicExecutorFeatureStep(KubernetesTestConf.createExecutorConf(sparkConf = conf), + secMgr) + + val executor = step.configurePod(SparkPod.initialPod()) + assert(!KubernetesFeaturesTestUtils.containerHasEnvVar( + executor.container, SecurityManager.ENV_AUTH_SECRET)) + } + // There is always exactly one controller reference, and it points to the driver pod. private def checkOwnerReferences(executor: Pod, driverPodUid: String): Unit = { assert(executor.getMetadata.getOwnerReferences.size() === 1) assert(executor.getMetadata.getOwnerReferences.get(0).getUid === driverPodUid) - assert(executor.getMetadata.getOwnerReferences.get(0).getController === true) + assert(executor.getMetadata.getOwnerReferences.get(0).getController) } // Check that the expected environment variables are present. - private def checkEnv(executorPod: SparkPod, additionalEnvVars: Map[String, String]): Unit = { + private def checkEnv( + executorPod: SparkPod, + conf: SparkConf, + additionalEnvVars: Map[String, String]): Unit = { val defaultEnvs = Map( ENV_EXECUTOR_ID -> "1", ENV_DRIVER_URL -> DRIVER_ADDRESS.toString, ENV_EXECUTOR_CORES -> "1", ENV_EXECUTOR_MEMORY -> "1g", - ENV_APPLICATION_ID -> APP_ID, + ENV_APPLICATION_ID -> KubernetesTestConf.APP_ID, ENV_SPARK_CONF_DIR -> SPARK_CONF_DIR_INTERNAL, - ENV_EXECUTOR_POD_IP -> null) ++ additionalEnvVars + ENV_EXECUTOR_POD_IP -> null, + ENV_SPARK_USER -> Utils.getCurrentUserName()) - assert(executorPod.container.getEnv.size() === defaultEnvs.size) - val mapEnvs = executorPod.container.getEnv.asScala.map { + val extraJavaOptsStart = additionalEnvVars.keys.count(_.startsWith(ENV_JAVA_OPT_PREFIX)) + val extraJavaOpts = Utils.sparkJavaOpts(conf, SparkConf.isExecutorStartupConf) + val extraJavaOptsEnvs = extraJavaOpts.zipWithIndex.map { case (opt, ind) => + s"$ENV_JAVA_OPT_PREFIX${ind + extraJavaOptsStart}" -> opt + }.toMap + + val containerEnvs = executorPod.container.getEnv.asScala.map { x => (x.getName, x.getValue) }.toMap - assert(defaultEnvs === mapEnvs) + + val expectedEnvs = defaultEnvs ++ additionalEnvVars ++ extraJavaOptsEnvs + assert(containerEnvs === expectedEnvs) } } 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 new file mode 100644 index 0000000000000..de80c5614cbfc --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverCommandFeatureStepSuite.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.deploy.k8s.features + +import scala.collection.JavaConverters._ + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit._ +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +class DriverCommandFeatureStepSuite extends SparkFunSuite { + + test("java resource") { + val mainResource = "local:/main.jar" + val spec = applyFeatureStep( + JavaMainAppResource(Some(mainResource)), + appArgs = Array("5", "7")) + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--properties-file", SPARK_CONF_PATH, + "--class", KubernetesTestConf.MAIN_CLASS, + mainResource, "5", "7")) + } + + test("python resource") { + 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")) + + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--properties-file", SPARK_CONF_PATH, + "--class", KubernetesTestConf.MAIN_CLASS, + mainResource, "5", "7", "9")) + + val envs = spec.pod.container.getEnv.asScala + .map { env => (env.getName, env.getValue) } + .toMap + val expected = Map(ENV_PYSPARK_MAJOR_PYTHON_VERSION -> "2") + assert(envs === expected) + } + + test("R resource") { + val mainResource = "local:/main.R" + + val spec = applyFeatureStep( + RMainAppResource(mainResource), + appArgs = Array("5", "7", "9")) + + assert(spec.pod.container.getArgs.asScala === List( + "driver", + "--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 = { + val kubernetesConf = KubernetesTestConf.createDriverConf( + sparkConf = conf, + mainAppResource = resource, + appArgs = appArgs) + val step = new DriverCommandFeatureStep(kubernetesConf) + val pod = step.configurePod(SparkPod.initialPod()) + val props = step.getAdditionalPodSystemProperties() + KubernetesDriverSpec(pod, Nil, props) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala index 7e916b3854404..7d8e9296a6cb5 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverKubernetesCredentialsFeatureStepSuite.scala @@ -18,51 +18,25 @@ package org.apache.spark.deploy.k8s.features import java.io.File +import scala.collection.JavaConverters._ + import com.google.common.base.Charsets import com.google.common.io.{BaseEncoding, Files} -import io.fabric8.kubernetes.api.model.{ContainerBuilder, HasMetadata, PodBuilder, Secret} -import org.mockito.{Mock, MockitoAnnotations} -import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ +import io.fabric8.kubernetes.api.model.Secret import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.util.Utils -class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite { - private val KUBERNETES_RESOURCE_NAME_PREFIX = "spark" - private val APP_ID = "k8s-app" - private var credentialsTempDirectory: File = _ + private val credentialsTempDirectory = Utils.createTempDir() private val BASE_DRIVER_POD = SparkPod.initialPod() - @Mock - private var driverSpecificConf: KubernetesDriverSpecificConf = _ - - before { - MockitoAnnotations.initMocks(this) - credentialsTempDirectory = Utils.createTempDir() - } - - after { - credentialsTempDirectory.delete() - } - test("Don't set any credentials") { - val kubernetesConf = KubernetesConf( - new SparkConf(false), - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) + val kubernetesConf = KubernetesTestConf.createDriverConf() val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalPodSystemProperties().isEmpty) @@ -83,19 +57,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_MOUNTED_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", "/mnt/secrets/my-ca.pem") - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) assert(kubernetesCredentialsStep.configurePod(BASE_DRIVER_POD) === BASE_DRIVER_POD) assert(kubernetesCredentialsStep.getAdditionalKubernetesResources().isEmpty) @@ -122,18 +84,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .set( s"$KUBERNETES_AUTH_DRIVER_CONF_PREFIX.$CA_CERT_FILE_CONF_SUFFIX", caCertFile.getAbsolutePath) - val kubernetesConf = KubernetesConf( - submissionSparkConf, - driverSpecificConf, - KUBERNETES_RESOURCE_NAME_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = submissionSparkConf) val kubernetesCredentialsStep = new DriverKubernetesCredentialsFeatureStep(kubernetesConf) val resolvedProperties = kubernetesCredentialsStep.getAdditionalPodSystemProperties() val expectedSparkConf = Map( @@ -153,7 +104,7 @@ class DriverKubernetesCredentialsFeatureStepSuite extends SparkFunSuite with Bef .head .asInstanceOf[Secret] assert(credentialsSecret.getMetadata.getName === - s"$KUBERNETES_RESOURCE_NAME_PREFIX-kubernetes-credentials") + s"${kubernetesConf.resourceNamePrefix}-kubernetes-credentials") val decodedSecretData = credentialsSecret.getData.asScala.map { data => (data._1, new String(BaseEncoding.base64().decode(data._2), Charsets.UTF_8)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala index 8b91e93eecd8c..9068289bab581 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/DriverServiceFeatureStepSuite.scala @@ -16,23 +16,21 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.Service -import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Mockito.when -import org.scalatest.BeforeAndAfter import scala.collection.JavaConverters._ +import com.google.common.net.InternetDomainName +import io.fabric8.kubernetes.api.model.Service + import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.util.Clock +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ +import org.apache.spark.util.ManualClock -class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { - - private val SHORT_RESOURCE_NAME_PREFIX = - "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX.length) +class DriverServiceFeatureStepSuite extends SparkFunSuite { private val LONG_RESOURCE_NAME_PREFIX = "a" * (DriverServiceFeatureStep.MAX_SERVICE_NAME_LENGTH - @@ -41,34 +39,15 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { "label1key" -> "label1value", "label2key" -> "label2value") - @Mock - private var clock: Clock = _ - - private var sparkConf: SparkConf = _ - - before { - MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - } - - test("Headless service has a port for the driver RPC and the block manager.") { - sparkConf = sparkConf - .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) + test("Headless service has a port for the driver RPC, the block manager and driver ui.") { + val sparkConf = new SparkConf(false) + .set(DRIVER_PORT, 9000) + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(UI_PORT, 4080) + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) assert(configurationStep.configurePod(SparkPod.initialPod()) === SparkPod.initialPod()) assert(configurationStep.getAdditionalKubernetesResources().size === 1) assert(configurationStep.getAdditionalKubernetesResources().head.isInstanceOf[Service]) @@ -79,50 +58,29 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( 9000, 8080, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + 4080, + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", driverService) } test("Hostname and ports are set according to the service name.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf - .set("spark.driver.port", "9000") - .set(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT, 8080) - .set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) - val expectedServiceName = SHORT_RESOURCE_NAME_PREFIX + - DriverServiceFeatureStep.DRIVER_SVC_POSTFIX + val sparkConf = new SparkConf(false) + .set(DRIVER_PORT, 9000) + .set(DRIVER_BLOCK_MANAGER_PORT, 8080) + .set(KUBERNETES_NAMESPACE, "my-namespace") + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) + val expectedServiceName = kconf.resourceNamePrefix + DriverServiceFeatureStep.DRIVER_SVC_POSTFIX val expectedHostName = s"$expectedServiceName.my-namespace.svc" val additionalProps = configurationStep.getAdditionalPodSystemProperties() - verifySparkConfHostNames(additionalProps, expectedHostName) + assert(additionalProps(DRIVER_HOST_ADDRESS.key) === expectedHostName) } test("Ports should resolve to defaults in SparkConf and in the service.") { - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - SHORT_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String])) + val kconf = KubernetesTestConf.createDriverConf(labels = DRIVER_LABELS) + val configurationStep = new DriverServiceFeatureStep(kconf) val resolvedService = configurationStep .getAdditionalKubernetesResources() .head @@ -130,104 +88,83 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { verifyService( DEFAULT_DRIVER_PORT, DEFAULT_BLOCKMANAGER_PORT, - s"$SHORT_RESOURCE_NAME_PREFIX${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", + UI_PORT.defaultValue.get, + s"${kconf.resourceNamePrefix}${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}", resolvedService) val additionalProps = configurationStep.getAdditionalPodSystemProperties() - assert(additionalProps("spark.driver.port") === DEFAULT_DRIVER_PORT.toString) - assert(additionalProps(org.apache.spark.internal.config.DRIVER_BLOCK_MANAGER_PORT.key) - === DEFAULT_BLOCKMANAGER_PORT.toString) + assert(additionalProps(DRIVER_PORT.key) === DEFAULT_DRIVER_PORT.toString) + assert(additionalProps(DRIVER_BLOCK_MANAGER_PORT.key) === DEFAULT_BLOCKMANAGER_PORT.toString) } - test("Long prefixes should switch to using a generated name.") { - when(clock.getTimeMillis()).thenReturn(10000) - val configurationStep = new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(KUBERNETES_NAMESPACE, "my-namespace"), - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]), - clock) - val driverService = configurationStep - .getAdditionalKubernetesResources() - .head - .asInstanceOf[Service] - val expectedServiceName = s"spark-10000${DriverServiceFeatureStep.DRIVER_SVC_POSTFIX}" - assert(driverService.getMetadata.getName === expectedServiceName) - val expectedHostName = s"$expectedServiceName.my-namespace.svc" - val additionalProps = configurationStep.getAdditionalPodSystemProperties() - verifySparkConfHostNames(additionalProps, expectedHostName) + test("Long prefixes should switch to using a generated unique name.") { + val sparkConf = new SparkConf(false) + .set(KUBERNETES_NAMESPACE, "my-namespace") + val kconf = KubernetesTestConf.createDriverConf( + sparkConf = sparkConf, + resourceNamePrefix = Some(LONG_RESOURCE_NAME_PREFIX), + labels = DRIVER_LABELS) + val clock = new ManualClock() + + // Ensure that multiple services created at the same time generate unique names. + val services = (1 to 10).map { _ => + val configurationStep = new DriverServiceFeatureStep(kconf, clock = clock) + val serviceName = configurationStep + .getAdditionalKubernetesResources() + .head + .asInstanceOf[Service] + .getMetadata + .getName + + val hostAddress = configurationStep + .getAdditionalPodSystemProperties()(DRIVER_HOST_ADDRESS.key) + + (serviceName -> hostAddress) + }.toMap + + assert(services.size === 10) + services.foreach { case (name, address) => + assert(!name.startsWith(kconf.resourceNamePrefix)) + assert(!address.startsWith(kconf.resourceNamePrefix)) + assert(InternetDomainName.isValid(address)) + } } test("Disallow bind address and driver host to be set explicitly.") { - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf.set(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS, "host"), - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]), - clock) - fail("The driver bind address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.DRIVER_BIND_ADDRESS_KEY} is" + - " not supported in Kubernetes mode, as the driver's bind address is managed" + - " and set to the driver pod's IP address.") + val sparkConf = new SparkConf(false) + .set(DRIVER_BIND_ADDRESS, "host") + .set("spark.app.name", LONG_RESOURCE_NAME_PREFIX) + val e1 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } - sparkConf.remove(org.apache.spark.internal.config.DRIVER_BIND_ADDRESS) - sparkConf.set(org.apache.spark.internal.config.DRIVER_HOST_ADDRESS, "host") - try { - new DriverServiceFeatureStep( - KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - None, "main", "app", Seq.empty), - LONG_RESOURCE_NAME_PREFIX, - "app-id", - DRIVER_LABELS, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]), - clock) - fail("The driver host address should not be allowed.") - } catch { - case e: Throwable => - assert(e.getMessage === - s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + - " not supported in Kubernetes mode, as the driver's hostname will be managed via" + - " a Kubernetes service.") + assert(e1.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.DRIVER_BIND_ADDRESS_KEY} is" + + " not supported in Kubernetes mode, as the driver's bind address is managed" + + " and set to the driver pod's IP address.") + + sparkConf.remove(DRIVER_BIND_ADDRESS) + sparkConf.set(DRIVER_HOST_ADDRESS, "host") + + val e2 = intercept[IllegalArgumentException] { + new DriverServiceFeatureStep(KubernetesTestConf.createDriverConf(sparkConf = sparkConf)) } + assert(e2.getMessage === + s"requirement failed: ${DriverServiceFeatureStep.DRIVER_HOST_KEY} is" + + " not supported in Kubernetes mode, as the driver's hostname will be managed via" + + " a Kubernetes service.") } private def verifyService( driverPort: Int, blockManagerPort: Int, + drierUIPort: Int, expectedServiceName: String, service: Service): Unit = { assert(service.getMetadata.getName === expectedServiceName) assert(service.getSpec.getClusterIP === "None") - assert(service.getSpec.getSelector.asScala === DRIVER_LABELS) - assert(service.getSpec.getPorts.size() === 2) + DRIVER_LABELS.foreach { case (k, v) => + assert(service.getSpec.getSelector.get(k) === v) + } + assert(service.getSpec.getPorts.size() === 3) val driverServicePorts = service.getSpec.getPorts.asScala assert(driverServicePorts.head.getName === DRIVER_PORT_NAME) assert(driverServicePorts.head.getPort.intValue() === driverPort) @@ -235,11 +172,8 @@ class DriverServiceFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { assert(driverServicePorts(1).getName === BLOCK_MANAGER_PORT_NAME) assert(driverServicePorts(1).getPort.intValue() === blockManagerPort) assert(driverServicePorts(1).getTargetPort.getIntVal === blockManagerPort) - } - - private def verifySparkConfHostNames( - driverSparkConf: Map[String, String], expectedHostName: String): Unit = { - assert(driverSparkConf( - org.apache.spark.internal.config.DRIVER_HOST_ADDRESS.key) === expectedHostName) + assert(driverServicePorts(2).getName === UI_PORT_NAME) + assert(driverServicePorts(2).getPort.intValue() === drierUIPort) + assert(driverServicePorts(2).getTargetPort.getIntVal === drierUIPort) } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala index 85c6cb282d2b0..0455526111067 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/EnvSecretsFeatureStepSuite.scala @@ -16,12 +16,12 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder +import scala.collection.JavaConverters._ -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.SparkFunSuite import org.apache.spark.deploy.k8s._ -class EnvSecretsFeatureStepSuite extends SparkFunSuite{ +class EnvSecretsFeatureStepSuite extends SparkFunSuite { private val KEY_REF_NAME_FOO = "foo" private val KEY_REF_NAME_BAR = "bar" private val KEY_REF_KEY_FOO = "key_foo" @@ -34,28 +34,14 @@ class EnvSecretsFeatureStepSuite extends SparkFunSuite{ val envVarsToKeys = Map( ENV_NAME_BAR -> s"${KEY_REF_NAME_BAR}:${KEY_REF_KEY_BAR}", ENV_NAME_FOO -> s"${KEY_REF_NAME_FOO}:${KEY_REF_KEY_FOO}") - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - Map.empty, - envVarsToKeys, - Map.empty, - Nil, - Seq.empty[String]) + val kubernetesConf = KubernetesTestConf.createDriverConf( + secretEnvNamesToKeyRefs = envVarsToKeys) val step = new EnvSecretsFeatureStep(kubernetesConf) - val driverContainerWithEnvSecrets = step.configurePod(baseDriverPod).container - - val expectedVars = - Seq(s"${ENV_NAME_BAR}", s"${ENV_NAME_FOO}") - - expectedVars.foreach { envName => - assert(KubernetesFeaturesTestUtils.containerHasEnvVar(driverContainerWithEnvSecrets, envName)) + val container = step.configurePod(baseDriverPod).container + val containerEnvKeys = container.getEnv.asScala.map { v => v.getName }.toSet + envVarsToKeys.keys.foreach { envName => + assert(containerEnvKeys.contains(envName)) } } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..e1c01dbdc7358 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/HadoopConfDriverFeatureStepSuite.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.{SparkConfWithEnv, Utils} + +class HadoopConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + test("mount hadoop config map if defined") { + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_HADOOP_CONF_CONFIG_MAP, "testConfigMap") + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create hadoop config map if config dir is defined") { + val confDir = Utils.createTempDir() + val confFiles = Set("core-site.xml", "hdfs-site.xml") + + confFiles.foreach { f => + Files.write("some data", new File(confDir, f), UTF_8) + } + + val sparkConf = new SparkConfWithEnv(Map(ENV_HADOOP_CONF_DIR -> confDir.getAbsolutePath())) + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + + val step = new HadoopConfDriverFeatureStep(conf) + checkPod(step.configurePod(SparkPod.initialPod())) + + val hadoopConfMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(hadoopConfMap.getData().keySet().asScala === confFiles) + } + + private def checkPod(pod: SparkPod): Unit = { + assert(podHasVolume(pod.pod, HADOOP_CONF_VOLUME)) + assert(containerHasVolume(pod.container, HADOOP_CONF_VOLUME, HADOOP_CONF_DIR_PATH)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_CONF_DIR)) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala new file mode 100644 index 0000000000000..41ca3a94ce7a7 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KerberosConfDriverFeatureStepSuite.scala @@ -0,0 +1,171 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.File +import java.nio.charset.StandardCharsets.UTF_8 +import java.security.PrivilegedExceptionAction + +import scala.collection.JavaConverters._ + +import com.google.common.io.Files +import io.fabric8.kubernetes.api.model.{ConfigMap, Secret} +import org.apache.commons.codec.binary.Base64 +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.{Credentials, UserGroupInformation} + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.k8s._ +import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.Constants._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.internal.config._ +import org.apache.spark.util.Utils + +class KerberosConfDriverFeatureStepSuite extends SparkFunSuite { + + import KubernetesFeaturesTestUtils._ + import SecretVolumeUtils._ + + private val tmpDir = Utils.createTempDir() + + test("mount krb5 config map if defined") { + val configMap = "testConfigMap" + val step = createStep( + new SparkConf(false).set(KUBERNETES_KERBEROS_KRB5_CONFIG_MAP, configMap)) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), configMap) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(filter[ConfigMap](step.getAdditionalKubernetesResources()).isEmpty) + } + + test("create krb5.conf config map if local config provided") { + val krbConf = File.createTempFile("krb5", ".conf", tmpDir) + Files.write("some data", krbConf, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_KRB5_FILE, krbConf.getAbsolutePath()) + val step = createStep(sparkConf) + + val confMap = filter[ConfigMap](step.getAdditionalKubernetesResources()).head + assert(confMap.getData().keySet().asScala === Set(krbConf.getName())) + + checkPodForKrbConf(step.configurePod(SparkPod.initialPod()), confMap.getMetadata().getName()) + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("create keytab secret if client keytab file used") { + val keytab = File.createTempFile("keytab", ".bin", tmpDir) + Files.write("some data", keytab, UTF_8) + + val sparkConf = new SparkConf(false) + .set(KEYTAB, keytab.getAbsolutePath()) + .set(PRINCIPAL, "alice") + val step = createStep(sparkConf) + + val pod = step.configurePod(SparkPod.initialPod()) + assert(podHasVolume(pod.pod, KERBEROS_KEYTAB_VOLUME)) + assert(containerHasVolume(pod.container, KERBEROS_KEYTAB_VOLUME, KERBEROS_KEYTAB_MOUNT_POINT)) + + assert(step.getAdditionalPodSystemProperties().keys === Set(KEYTAB.key)) + + val secret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(secret.getData().keySet().asScala === Set(keytab.getName())) + } + + test("do nothing if container-local keytab used") { + val sparkConf = new SparkConf(false) + .set(KEYTAB, "local:/my.keytab") + .set(PRINCIPAL, "alice") + val step = createStep(sparkConf) + + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("mount delegation tokens if provided") { + val dtSecret = "tokenSecret" + val sparkConf = new SparkConf(false) + .set(KUBERNETES_KERBEROS_DT_SECRET_NAME, dtSecret) + .set(KUBERNETES_KERBEROS_DT_SECRET_ITEM_KEY, "dtokens") + val step = createStep(sparkConf) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), dtSecret) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + test("create delegation tokens if needed") { + // Since HadoopDelegationTokenManager does not create any tokens without proper configs and + // services, start with a test user that already has some tokens that will just be piped + // through to the driver. + val testUser = UserGroupInformation.createUserForTesting("k8s", Array()) + testUser.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = { + val creds = testUser.getCredentials() + creds.addSecretKey(new Text("K8S_TEST_KEY"), Array[Byte](0x4, 0x2)) + testUser.addCredentials(creds) + + val tokens = SparkHadoopUtil.get.serialize(creds) + + val step = createStep(new SparkConf(false)) + + val dtSecret = filter[Secret](step.getAdditionalKubernetesResources()).head + assert(dtSecret.getData().get(KERBEROS_SECRET_KEY) === Base64.encodeBase64String(tokens)) + + checkPodForTokens(step.configurePod(SparkPod.initialPod()), + dtSecret.getMetadata().getName()) + + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + }) + } + + test("do nothing if no config and no tokens") { + val step = createStep(new SparkConf(false)) + val initial = SparkPod.initialPod() + assert(step.configurePod(initial) === initial) + assert(step.getAdditionalPodSystemProperties().isEmpty) + assert(step.getAdditionalKubernetesResources().isEmpty) + } + + private def checkPodForKrbConf(pod: SparkPod, confMapName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala.find(_.getName() == KRB_FILE_VOLUME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, KRB_FILE_VOLUME, KRB_FILE_DIR_PATH + "/krb5.conf")) + assert(podVolume.get.getConfigMap().getName() === confMapName) + } + + private def checkPodForTokens(pod: SparkPod, dtSecretName: String): Unit = { + val podVolume = pod.pod.getSpec().getVolumes().asScala + .find(_.getName() == SPARK_APP_HADOOP_SECRET_VOLUME_NAME) + assert(podVolume.isDefined) + assert(containerHasVolume(pod.container, SPARK_APP_HADOOP_SECRET_VOLUME_NAME, + SPARK_APP_HADOOP_CREDENTIALS_BASE_DIR)) + assert(containerHasEnvVar(pod.container, ENV_HADOOP_TOKEN_FILE_LOCATION)) + assert(podVolume.get.getSecret().getSecretName() === dtSecretName) + } + + private def createStep(conf: SparkConf): KerberosConfDriverFeatureStep = { + val kconf = KubernetesTestConf.createDriverConf(sparkConf = conf) + new KerberosConfDriverFeatureStep(kconf) + } + +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala index f90380e30e52a..284887f0bddcb 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/KubernetesFeaturesTestUtils.scala @@ -17,14 +17,15 @@ package org.apache.spark.deploy.k8s.features import scala.collection.JavaConverters._ +import scala.reflect.ClassTag import io.fabric8.kubernetes.api.model.{Container, HasMetadata, PodBuilder, SecretBuilder} -import org.mockito.Matchers -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, when} import org.mockito.invocation.InvocationOnMock -import org.mockito.stubbing.Answer import org.apache.spark.deploy.k8s.SparkPod +import org.apache.spark.resource.ResourceID object KubernetesFeaturesTestUtils { @@ -36,17 +37,15 @@ object KubernetesFeaturesTestUtils { when(mockStep.getAdditionalPodSystemProperties()) .thenReturn(Map(stepType -> stepType)) - when(mockStep.configurePod(Matchers.any(classOf[SparkPod]))) - .thenAnswer(new Answer[SparkPod]() { - override def answer(invocation: InvocationOnMock): SparkPod = { - val originalPod = invocation.getArgumentAt(0, classOf[SparkPod]) - val configuredPod = new PodBuilder(originalPod.pod) - .editOrNewMetadata() - .addToLabels(stepType, stepType) - .endMetadata() - .build() - SparkPod(configuredPod, originalPod.container) - } + when(mockStep.configurePod(any(classOf[SparkPod]))) + .thenAnswer((invocation: InvocationOnMock) => { + val originalPod: SparkPod = invocation.getArgument(0) + val configuredPod = new PodBuilder(originalPod.pod) + .editOrNewMetadata() + .addToLabels(stepType, stepType) + .endMetadata() + .build() + SparkPod(configuredPod, originalPod.container) }) mockStep } @@ -63,4 +62,11 @@ object KubernetesFeaturesTestUtils { def containerHasEnvVar(container: Container, envVarName: String): Boolean = { container.getEnv.asScala.exists(envVar => envVar.getName == envVarName) } + + def filter[T: ClassTag](list: Seq[HasMetadata]): Seq[T] = { + val desired = implicitly[ClassTag[T]].runtimeClass + list.filter(_.getClass() == desired).map(_.asInstanceOf[T]) + } + + case class TestResourceInformation(rId: ResourceID, count: String, vendor: String) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala index acdd07bc594b2..8f34ce5c6b94f 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/LocalDirsFeatureStepSuite.scala @@ -17,44 +17,19 @@ package org.apache.spark.deploy.k8s.features import io.fabric8.kubernetes.api.model.{EnvVarBuilder, VolumeBuilder, VolumeMountBuilder} -import org.mockito.Mockito -import org.scalatest._ -import org.scalatest.BeforeAndAfter import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, KubernetesRoleSpecificConf, SparkPod} +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ +import org.apache.spark.deploy.k8s.submit.JavaMainAppResource +import org.apache.spark.util.SparkConfWithEnv -class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { +class LocalDirsFeatureStepSuite extends SparkFunSuite { private val defaultLocalDir = "/var/data/default-local-dir" - private var sparkConf: SparkConf = _ - private var kubernetesConf: KubernetesConf[_ <: KubernetesRoleSpecificConf] = _ - - before { - val realSparkConf = new SparkConf(false) - sparkConf = Mockito.spy(realSparkConf) - kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - None, - "app-name", - "main", - Seq.empty), - "resource", - "app-id", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - } test("Resolve to default local dir if neither env nor configuration are set") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) + val stepUnderTest = new LocalDirsFeatureStep(KubernetesTestConf.createDriverConf(), + defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) assert(configuredPod.pod.getSpec.getVolumes.get(0) === @@ -78,8 +53,9 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use configured local dirs split on comma if provided.") { - Mockito.doReturn("/var/data/my-local-dir-1,/var/data/my-local-dir-2") - .when(sparkConf).getenv("SPARK_LOCAL_DIRS") + val sparkConf = new SparkConfWithEnv(Map( + "SPARK_LOCAL_DIRS" -> "/var/data/my-local-dir-1,/var/data/my-local-dir-2")) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 2) @@ -115,9 +91,8 @@ class LocalDirsFeatureStepSuite extends SparkFunSuite with BeforeAndAfter { } test("Use tmpfs to back default local dir") { - Mockito.doReturn(null).when(sparkConf).get("spark.local.dir") - Mockito.doReturn(null).when(sparkConf).getenv("SPARK_LOCAL_DIRS") - Mockito.doReturn(true).when(sparkConf).get(KUBERNETES_LOCAL_DIRS_TMPFS) + val sparkConf = new SparkConf(false).set(KUBERNETES_LOCAL_DIRS_TMPFS, true) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) val stepUnderTest = new LocalDirsFeatureStep(kubernetesConf, defaultLocalDir) val configuredPod = stepUnderTest.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size === 1) diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala index dad610c443acc..22f6d26c4d0d3 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountSecretsFeatureStepSuite.scala @@ -16,10 +16,8 @@ */ package org.apache.spark.deploy.k8s.features -import io.fabric8.kubernetes.api.model.PodBuilder - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SecretVolumeUtils, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s.{KubernetesTestConf, SecretVolumeUtils, SparkPod} class MountSecretsFeatureStepSuite extends SparkFunSuite { @@ -32,19 +30,8 @@ class MountSecretsFeatureStepSuite extends SparkFunSuite { val secretNamesToMountPaths = Map( SECRET_FOO -> SECRET_MOUNT_PATH, SECRET_BAR -> SECRET_MOUNT_PATH) - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesExecutorSpecificConf("1", Some(new PodBuilder().build())), - "resource-name-prefix", - "app-id", - Map.empty, - Map.empty, - secretNamesToMountPaths, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) + val kubernetesConf = KubernetesTestConf.createExecutorConf( + secretNamesToMountPaths = secretNamesToMountPaths) val step = new MountSecretsFeatureStep(kubernetesConf) val driverPodWithSecretsMounted = step.configurePod(baseDriverPod).pod diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala index d309aa94ec115..8c430eeb3fa71 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/MountVolumesFeatureStepSuite.scala @@ -16,36 +16,21 @@ */ package org.apache.spark.deploy.k8s.features +import scala.collection.JavaConverters._ + import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.deploy.k8s._ class MountVolumesFeatureStepSuite extends SparkFunSuite { - private val sparkConf = new SparkConf(false) - private val emptyKubernetesConf = KubernetesConf( - sparkConf = sparkConf, - roleSpecificConf = KubernetesDriverSpecificConf( - None, - "app-name", - "main", - Seq.empty), - appResourceNamePrefix = "resource", - appId = "app-id", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - sparkFiles = Nil) - test("Mounts hostPath volumes") { val volumeConf = KubernetesVolumeSpec( "testVolume", "/tmp", + "", false, KubernetesHostPathVolumeConf("/hostPath/tmp") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -61,10 +46,11 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { val volumeConf = KubernetesVolumeSpec( "testVolume", "/tmp", + "", true, KubernetesPVCVolumeConf("pvcClaim") ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -74,7 +60,7 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { assert(configuredPod.container.getVolumeMounts.size() === 1) assert(configuredPod.container.getVolumeMounts.get(0).getMountPath === "/tmp") assert(configuredPod.container.getVolumeMounts.get(0).getName === "testVolume") - assert(configuredPod.container.getVolumeMounts.get(0).getReadOnly === true) + assert(configuredPod.container.getVolumeMounts.get(0).getReadOnly) } @@ -82,10 +68,11 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { val volumeConf = KubernetesVolumeSpec( "testVolume", "/tmp", + "", false, KubernetesEmptyDirVolumeConf(Some("Memory"), Some("6G")) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -103,10 +90,11 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { val volumeConf = KubernetesVolumeSpec( "testVolume", "/tmp", + "", false, KubernetesEmptyDirVolumeConf(None, None) ) - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumeConf :: Nil) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) @@ -124,21 +112,95 @@ class MountVolumesFeatureStepSuite extends SparkFunSuite { val hpVolumeConf = KubernetesVolumeSpec( "hpVolume", "/tmp", + "", false, KubernetesHostPathVolumeConf("/hostPath/tmp") ) val pvcVolumeConf = KubernetesVolumeSpec( "checkpointVolume", "/checkpoints", + "", true, KubernetesPVCVolumeConf("pvcClaim") ) - val volumesConf = hpVolumeConf :: pvcVolumeConf :: Nil - val kubernetesConf = emptyKubernetesConf.copy(roleVolumes = volumesConf) + val kubernetesConf = KubernetesTestConf.createDriverConf( + volumes = Seq(hpVolumeConf, pvcVolumeConf)) val step = new MountVolumesFeatureStep(kubernetesConf) val configuredPod = step.configurePod(SparkPod.initialPod()) assert(configuredPod.pod.getSpec.getVolumes.size() === 2) assert(configuredPod.container.getVolumeMounts.size() === 2) } + + test("Mounts subpath on emptyDir") { + val volumeConf = KubernetesVolumeSpec( + "testVolume", + "/tmp", + "foo", + false, + KubernetesEmptyDirVolumeConf(None, None) + ) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) + val step = new MountVolumesFeatureStep(kubernetesConf) + val configuredPod = step.configurePod(SparkPod.initialPod()) + + assert(configuredPod.pod.getSpec.getVolumes.size() === 1) + val emptyDirMount = configuredPod.container.getVolumeMounts.get(0) + assert(emptyDirMount.getMountPath === "/tmp") + assert(emptyDirMount.getName === "testVolume") + assert(emptyDirMount.getSubPath === "foo") + } + + test("Mounts subpath on persistentVolumeClaims") { + val volumeConf = KubernetesVolumeSpec( + "testVolume", + "/tmp", + "bar", + true, + KubernetesPVCVolumeConf("pvcClaim") + ) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(volumeConf)) + val step = new MountVolumesFeatureStep(kubernetesConf) + val configuredPod = step.configurePod(SparkPod.initialPod()) + + assert(configuredPod.pod.getSpec.getVolumes.size() === 1) + val pvcClaim = configuredPod.pod.getSpec.getVolumes.get(0).getPersistentVolumeClaim + assert(pvcClaim.getClaimName === "pvcClaim") + assert(configuredPod.container.getVolumeMounts.size() === 1) + val pvcMount = configuredPod.container.getVolumeMounts.get(0) + assert(pvcMount.getMountPath === "/tmp") + assert(pvcMount.getName === "testVolume") + assert(pvcMount.getSubPath === "bar") + } + + test("Mounts multiple subpaths") { + val volumeConf = KubernetesEmptyDirVolumeConf(None, None) + val emptyDirSpec = KubernetesVolumeSpec( + "testEmptyDir", + "/tmp/foo", + "foo", + true, + KubernetesEmptyDirVolumeConf(None, None) + ) + val pvcSpec = KubernetesVolumeSpec( + "testPVC", + "/tmp/bar", + "bar", + true, + KubernetesEmptyDirVolumeConf(None, None) + ) + val kubernetesConf = KubernetesTestConf.createDriverConf(volumes = Seq(emptyDirSpec, pvcSpec)) + val step = new MountVolumesFeatureStep(kubernetesConf) + val configuredPod = step.configurePod(SparkPod.initialPod()) + + assert(configuredPod.pod.getSpec.getVolumes.size() === 2) + val mounts = configuredPod.container.getVolumeMounts.asScala.sortBy(_.getName()) + assert(mounts.size === 2) + assert(mounts(0).getName === "testEmptyDir") + assert(mounts(0).getMountPath === "/tmp/foo") + assert(mounts(0).getSubPath === "foo") + assert(mounts(1).getName === "testPVC") + assert(mounts(1).getMountPath === "/tmp/bar") + assert(mounts(1).getSubPath === "bar") + } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala new file mode 100644 index 0000000000000..051320fa44c5e --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/PodTemplateConfigMapStepSuite.scala @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.features + +import java.io.{File, PrintWriter} +import java.nio.file.Files + +import io.fabric8.kubernetes.api.model.ConfigMap + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s._ +import org.apache.spark.util.Utils + +class PodTemplateConfigMapStepSuite extends SparkFunSuite { + + test("Do nothing when executor template is not specified") { + val conf = KubernetesTestConf.createDriverConf() + val step = new PodTemplateConfigMapStep(conf) + + val initialPod = SparkPod.initialPod() + val configuredPod = step.configurePod(initialPod) + assert(configuredPod === initialPod) + + assert(step.getAdditionalKubernetesResources().isEmpty) + assert(step.getAdditionalPodSystemProperties().isEmpty) + } + + test("Mounts executor template volume if config specified") { + val templateFile = Files.createTempFile("pod-template", "yml").toFile + templateFile.deleteOnExit() + + val sparkConf = new SparkConf(false) + .set(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE, templateFile.getAbsolutePath) + val kubernetesConf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + + Utils.tryWithResource(new PrintWriter(templateFile)) { writer => + writer.write("pod-template-contents") + } + + val step = new PodTemplateConfigMapStep(kubernetesConf) + val configuredPod = step.configurePod(SparkPod.initialPod()) + + assert(configuredPod.pod.getSpec.getVolumes.size() === 1) + val volume = configuredPod.pod.getSpec.getVolumes.get(0) + assert(volume.getName === Constants.POD_TEMPLATE_VOLUME) + assert(volume.getConfigMap.getName === Constants.POD_TEMPLATE_CONFIGMAP) + assert(volume.getConfigMap.getItems.size() === 1) + assert(volume.getConfigMap.getItems.get(0).getKey === Constants.POD_TEMPLATE_KEY) + assert(volume.getConfigMap.getItems.get(0).getPath === + Constants.EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME) + + assert(configuredPod.container.getVolumeMounts.size() === 1) + val volumeMount = configuredPod.container.getVolumeMounts.get(0) + assert(volumeMount.getMountPath === Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH) + assert(volumeMount.getName === Constants.POD_TEMPLATE_VOLUME) + + val resources = step.getAdditionalKubernetesResources() + assert(resources.size === 1) + assert(resources.head.getMetadata.getName === Constants.POD_TEMPLATE_CONFIGMAP) + assert(resources.head.isInstanceOf[ConfigMap]) + val configMap = resources.head.asInstanceOf[ConfigMap] + assert(configMap.getData.size() === 1) + assert(configMap.getData.containsKey(Constants.POD_TEMPLATE_KEY)) + assert(configMap.getData.containsValue("pod-template-contents")) + + val systemProperties = step.getAdditionalPodSystemProperties() + assert(systemProperties.size === 1) + assert(systemProperties.contains(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key)) + assert(systemProperties.get(Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE.key).get === + (Constants.EXECUTOR_POD_SPEC_TEMPLATE_MOUNTPATH + "/" + + Constants.EXECUTOR_POD_SPEC_TEMPLATE_FILE_NAME)) + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala deleted file mode 100644 index bf552aeb8b901..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/JavaDriverFeatureStepSuite.scala +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import scala.collection.JavaConverters._ - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.PythonMainAppResource - -class JavaDriverFeatureStepSuite extends SparkFunSuite { - - test("Java Step modifies container correctly") { - val baseDriverPod = SparkPod.initialPod() - val sparkConf = new SparkConf(false) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - Some(PythonMainAppResource("local:///main.jar")), - "test-class", - "java-runner", - Seq("5 7")), - appResourceNamePrefix = "", - appId = "", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - sparkFiles = Seq.empty[String]) - - val step = new JavaDriverFeatureStep(kubernetesConf) - val driverPod = step.configurePod(baseDriverPod).pod - val driverContainerwithJavaStep = step.configurePod(baseDriverPod).container - assert(driverContainerwithJavaStep.getArgs.size === 7) - val args = driverContainerwithJavaStep - .getArgs.asScala - assert(args === List( - "driver", - "--properties-file", SPARK_CONF_PATH, - "--class", "test-class", - "spark-internal", "5 7")) - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala deleted file mode 100644 index c14af1d3b0f01..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/PythonDriverFeatureStepSuite.scala +++ /dev/null @@ -1,110 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import scala.collection.JavaConverters._ - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.PythonMainAppResource - -class PythonDriverFeatureStepSuite extends SparkFunSuite { - - test("Python Step modifies container correctly") { - val expectedMainResource = "/main.py" - val mainResource = "local:///main.py" - val pyFiles = Seq("local:///example2.py", "local:///example3.py") - val expectedPySparkFiles = - "/example2.py:/example3.py" - val baseDriverPod = SparkPod.initialPod() - val sparkConf = new SparkConf(false) - .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) - .set(KUBERNETES_PYSPARK_PY_FILES, pyFiles.mkString(",")) - .set("spark.files", "local:///example.py") - .set(PYSPARK_MAJOR_PYTHON_VERSION, "2") - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - Some(PythonMainAppResource("local:///main.py")), - "test-app", - "python-runner", - Seq("5", "7", "9")), - appResourceNamePrefix = "", - appId = "", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - sparkFiles = Seq.empty[String]) - - val step = new PythonDriverFeatureStep(kubernetesConf) - val driverPod = step.configurePod(baseDriverPod).pod - val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - assert(driverContainerwithPySpark.getEnv.size === 4) - val envs = driverContainerwithPySpark - .getEnv - .asScala - .map(env => (env.getName, env.getValue)) - .toMap - assert(envs(ENV_PYSPARK_PRIMARY) === expectedMainResource) - assert(envs(ENV_PYSPARK_FILES) === expectedPySparkFiles) - assert(envs(ENV_PYSPARK_ARGS) === "5 7 9") - assert(envs(ENV_PYSPARK_MAJOR_PYTHON_VERSION) === "2") - } - test("Python Step testing empty pyfiles") { - val mainResource = "local:///main.py" - val baseDriverPod = SparkPod.initialPod() - val sparkConf = new SparkConf(false) - .set(KUBERNETES_PYSPARK_MAIN_APP_RESOURCE, mainResource) - .set(PYSPARK_MAJOR_PYTHON_VERSION, "3") - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - Some(PythonMainAppResource("local:///main.py")), - "test-class-py", - "python-runner", - Seq.empty[String]), - appResourceNamePrefix = "", - appId = "", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Nil, - sparkFiles = Seq.empty[String]) - val step = new PythonDriverFeatureStep(kubernetesConf) - val driverContainerwithPySpark = step.configurePod(baseDriverPod).container - val args = driverContainerwithPySpark - .getArgs.asScala - assert(driverContainerwithPySpark.getArgs.size === 5) - assert(args === List( - "driver-py", - "--properties-file", SPARK_CONF_PATH, - "--class", "test-class-py")) - val envs = driverContainerwithPySpark - .getEnv - .asScala - .map(env => (env.getName, env.getValue)) - .toMap - assert(envs(ENV_PYSPARK_MAJOR_PYTHON_VERSION) === "3") - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala deleted file mode 100644 index ace0faa8629c3..0000000000000 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/features/bindings/RDriverFeatureStepSuite.scala +++ /dev/null @@ -1,63 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.features.bindings - -import scala.collection.JavaConverters._ - -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpecificConf, SparkPod} -import org.apache.spark.deploy.k8s.Config._ -import org.apache.spark.deploy.k8s.Constants._ -import org.apache.spark.deploy.k8s.submit.RMainAppResource - -class RDriverFeatureStepSuite extends SparkFunSuite { - - test("R Step modifies container correctly") { - val expectedMainResource = "/main.R" - val mainResource = "local:///main.R" - val baseDriverPod = SparkPod.initialPod() - val sparkConf = new SparkConf(false) - .set(KUBERNETES_R_MAIN_APP_RESOURCE, mainResource) - val kubernetesConf = KubernetesConf( - sparkConf, - KubernetesDriverSpecificConf( - Some(RMainAppResource(mainResource)), - "test-app", - "r-runner", - Seq("5", "7", "9")), - appResourceNamePrefix = "", - appId = "", - roleLabels = Map.empty, - roleAnnotations = Map.empty, - roleSecretNamesToMountPaths = Map.empty, - roleSecretEnvNamesToKeyRefs = Map.empty, - roleEnvs = Map.empty, - roleVolumes = Seq.empty, - sparkFiles = Seq.empty[String]) - - val step = new RDriverFeatureStep(kubernetesConf) - val driverContainerwithR = step.configurePod(baseDriverPod).container - assert(driverContainerwithR.getEnv.size === 2) - val envs = driverContainerwithR - .getEnv - .asScala - .map(env => (env.getName, env.getValue)) - .toMap - assert(envs(ENV_R_PRIMARY) === expectedMainResource) - assert(envs(ENV_R_ARGS) === "5 7 9") - } -} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala index 4d8e79189ff32..aa421be6e8412 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/ClientSuite.scala @@ -18,29 +18,27 @@ package org.apache.spark.deploy.k8s.submit import io.fabric8.kubernetes.api.model._ import io.fabric8.kubernetes.client.{KubernetesClient, Watch} -import io.fabric8.kubernetes.client.dsl.{MixedOperation, NamespaceListVisitFromServerGetDeleteRecreateWaitApplicable, PodResource} +import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Mockito.{doReturn, verify, when} +import org.mockito.Mockito.{verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.mockito.MockitoSugar._ -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesDriverSpec, KubernetesDriverSpecificConf, SparkPod} +import org.apache.spark.SparkFunSuite +import org.apache.spark.deploy.k8s._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ class ClientSuite extends SparkFunSuite with BeforeAndAfter { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) + private val DRIVER_POD_UID = "pod-id" private val DRIVER_POD_API_VERSION = "v1" private val DRIVER_POD_KIND = "pod" private val KUBERNETES_RESOURCE_PREFIX = "resource-example" private val POD_NAME = "driver" private val CONTAINER_NAME = "container" - private val APP_ID = "app-id" - private val APP_NAME = "app" - private val MAIN_CLASS = "main" - private val APP_ARGS = Seq("arg1", "arg2") private val RESOLVED_JAVA_OPTIONS = Map( "conf1key" -> "conf1value", "conf2key" -> "conf2value") @@ -122,28 +120,15 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { @Mock private var resourceList: RESOURCE_LIST = _ - private var kubernetesConf: KubernetesConf[KubernetesDriverSpecificConf] = _ - - private var sparkConf: SparkConf = _ + private var kconf: KubernetesDriverConf = _ private var createdPodArgumentCaptor: ArgumentCaptor[Pod] = _ private var createdResourcesArgumentCaptor: ArgumentCaptor[HasMetadata] = _ before { MockitoAnnotations.initMocks(this) - sparkConf = new SparkConf(false) - kubernetesConf = KubernetesConf[KubernetesDriverSpecificConf]( - sparkConf, - KubernetesDriverSpecificConf(None, MAIN_CLASS, APP_NAME, APP_ARGS), - KUBERNETES_RESOURCE_PREFIX, - APP_ID, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - when(driverBuilder.buildFromFeatures(kubernetesConf)).thenReturn(BUILT_KUBERNETES_SPEC) + kconf = KubernetesTestConf.createDriverConf( + resourceNamePrefix = Some(KUBERNETES_RESOURCE_PREFIX)) + when(driverBuilder.buildFromFeatures(kconf, kubernetesClient)).thenReturn(BUILT_KUBERNETES_SPEC) when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(POD_NAME)).thenReturn(namedPods) @@ -158,26 +143,22 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("The client should configure the pod using the builder.") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(podOperations).create(FULL_EXPECTED_POD) } test("The client should create Kubernetes resources") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, false, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() val otherCreatedResources = createdResourcesArgumentCaptor.getAllValues assert(otherCreatedResources.size === 2) @@ -197,13 +178,11 @@ class ClientSuite extends SparkFunSuite with BeforeAndAfter { test("Waiting for app completion should stall on the watcher") { val submissionClient = new Client( + kconf, driverBuilder, - kubernetesConf, kubernetesClient, true, - "spark", - loggingPodStatusWatcher, - KUBERNETES_RESOURCE_PREFIX) + loggingPodStatusWatcher) submissionClient.run() verify(loggingPodStatusWatcher).awaitCompletion() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOpSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOpSuite.scala new file mode 100644 index 0000000000000..d8be13280c2e6 --- /dev/null +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/K8sSubmitOpSuite.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.submit + +import java.io.PrintStream + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.client.KubernetesClient +import io.fabric8.kubernetes.client.dsl.PodResource +import org.mockito.{ArgumentMatchers, Mock, MockitoAnnotations} +import org.mockito.Mockito.{times, verify, when} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config.KUBERNETES_SUBMIT_GRACE_PERIOD +import org.apache.spark.deploy.k8s.Constants.{SPARK_APP_ID_LABEL, SPARK_POD_DRIVER_ROLE, SPARK_ROLE_LABEL} +import org.apache.spark.deploy.k8s.Fabric8Aliases.PODS +import org.apache.spark.scheduler.cluster.k8s.ExecutorLifecycleTestUtils.TEST_SPARK_APP_ID + +class K8sSubmitOpSuite extends SparkFunSuite with BeforeAndAfter { + private val driverPodName1 = "driver1" + private val driverPodName2 = "driver2" + private val driverPod1 = buildDriverPod(driverPodName1, "1") + private val driverPod2 = buildDriverPod(driverPodName2, "2") + private val podList = List(driverPod1, driverPod2) + private val namespace = "test" + + @Mock + private var podOperations: PODS = _ + + @Mock + private var driverPodOperations1: PodResource[Pod, DoneablePod] = _ + + @Mock + private var driverPodOperations2: PodResource[Pod, DoneablePod] = _ + + @Mock + private var kubernetesClient: KubernetesClient = _ + + @Mock + private var err: PrintStream = _ + + before { + MockitoAnnotations.initMocks(this) + when(kubernetesClient.pods()).thenReturn(podOperations) + when(podOperations.inNamespace(namespace)).thenReturn(podOperations) + when(podOperations.delete(podList.asJava)).thenReturn(true) + when(podOperations.withName(driverPodName1)).thenReturn(driverPodOperations1) + when(podOperations.withName(driverPodName2)).thenReturn(driverPodOperations2) + when(driverPodOperations1.get).thenReturn(driverPod1) + when(driverPodOperations1.delete()).thenReturn(true) + when(driverPodOperations2.get).thenReturn(driverPod2) + when(driverPodOperations2.delete()).thenReturn(true) + } + + test("List app status") { + implicit val kubeClient: KubernetesClient = kubernetesClient + val listStatus = new ListStatus + listStatus.printStream = err + listStatus.executeOnPod(driverPodName1, Option(namespace), new SparkConf()) + // scalastyle:off + verify(err).println(ArgumentMatchers.eq(getPodStatus(driverPodName1, "1"))) + // scalastyle:on + } + + test("List status for multiple apps with glob") { + implicit val kubeClient: KubernetesClient = kubernetesClient + val listStatus = new ListStatus + listStatus.printStream = err + listStatus.executeOnGlob(podList, Option(namespace), new SparkConf()) + // scalastyle:off + verify(err).println(ArgumentMatchers.eq(getPodStatus(driverPodName1, "1"))) + verify(err).println(ArgumentMatchers.eq(getPodStatus(driverPodName2, "2"))) + // scalastyle:on + } + + test("Kill app") { + implicit val kubeClient: KubernetesClient = kubernetesClient + val killApp = new KillApplication + killApp.executeOnPod(driverPodName1, Option(namespace), new SparkConf()) + verify(driverPodOperations1, times(1)).delete() + } + + test("Kill app with gracePeriod") { + implicit val kubeClient: KubernetesClient = kubernetesClient + val killApp = new KillApplication + val conf = new SparkConf().set(KUBERNETES_SUBMIT_GRACE_PERIOD, 1L) + when(driverPodOperations1.withGracePeriod(1L)).thenReturn(driverPodOperations1) + killApp.executeOnPod(driverPodName1, Option(namespace), conf) + verify(driverPodOperations1, times(1)).withGracePeriod(1L) + verify(driverPodOperations1, times(1)).delete() + } + + test("Kill multiple apps with glob without gracePeriod") { + implicit val kubeClient: KubernetesClient = kubernetesClient + val killApp = new KillApplication + killApp.printStream = err + killApp.executeOnGlob(podList, Option(namespace), new SparkConf()) + verify(podOperations, times(1)).delete(podList.asJava) + // scalastyle:off + verify(err).println(ArgumentMatchers.eq(s"Deleting driver pod: $driverPodName1.")) + verify(err).println(ArgumentMatchers.eq(s"Deleting driver pod: $driverPodName2.")) + // scalastyle:on + } + + private def buildDriverPod(podName: String, id: String): Pod = { + new PodBuilder() + .withNewMetadata() + .withName(podName) + .withNamespace(namespace) + .addToLabels(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID) + .addToLabels(SPARK_ROLE_LABEL, SPARK_POD_DRIVER_ROLE) + .withUid(s"driver-pod-$id") + .endMetadata() + .withNewSpec() + .withServiceAccountName(s"test$id") + .withVolumes() + .withNodeName(s"testNode$id") + .endSpec() + .withNewStatus() + .withPhase("Running") + .endStatus() + .build() + } + + private def getPodStatus(podName: String, id: String): String = { + "Application status (driver): " + + s"""|${"\n\t"} pod name: $podName + |${"\t"} namespace: N/A + |${"\t"} labels: spark-app-selector -> spark-app-id, spark-role -> driver + |${"\t"} pod uid: driver-pod-$id + |${"\t"} creation time: N/A + |${"\t"} service account name: test$id + |${"\t"} volumes: N/A + |${"\t"} node name: testNode$id + |${"\t"} start time: N/A + |${"\t"} phase: Running + |${"\t"} container status: N/A""".stripMargin + } +} diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala index 4117c5487a41e..6518c91a1a1fd 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/deploy/k8s/submit/KubernetesDriverBuilderSuite.scala @@ -16,240 +16,21 @@ */ package org.apache.spark.deploy.k8s.submit -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.features._ -import org.apache.spark.deploy.k8s.features.{BasicDriverFeatureStep, DriverKubernetesCredentialsFeatureStep, DriverServiceFeatureStep, EnvSecretsFeatureStep, KubernetesFeaturesTestUtils, LocalDirsFeatureStep, MountSecretsFeatureStep} -import org.apache.spark.deploy.k8s.features.bindings.{JavaDriverFeatureStep, PythonDriverFeatureStep, RDriverFeatureStep} - -class KubernetesDriverBuilderSuite extends SparkFunSuite { - - private val BASIC_STEP_TYPE = "basic" - private val CREDENTIALS_STEP_TYPE = "credentials" - private val SERVICE_STEP_TYPE = "service" - private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val SECRETS_STEP_TYPE = "mount-secrets" - private val JAVA_STEP_TYPE = "java-bindings" - private val PYSPARK_STEP_TYPE = "pyspark-bindings" - private val R_STEP_TYPE = "r-bindings" - private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" - - private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - BASIC_STEP_TYPE, classOf[BasicDriverFeatureStep]) - - private val credentialsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - CREDENTIALS_STEP_TYPE, classOf[DriverKubernetesCredentialsFeatureStep]) - - private val serviceStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SERVICE_STEP_TYPE, classOf[DriverServiceFeatureStep]) - - private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - - private val secretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) - - private val javaStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - JAVA_STEP_TYPE, classOf[JavaDriverFeatureStep]) - - private val pythonStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - PYSPARK_STEP_TYPE, classOf[PythonDriverFeatureStep]) - - private val rStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - R_STEP_TYPE, classOf[RDriverFeatureStep]) +import io.fabric8.kubernetes.client.KubernetesClient - private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) - - private val builderUnderTest: KubernetesDriverBuilder = - new KubernetesDriverBuilder( - _ => basicFeatureStep, - _ => credentialsStep, - _ => serviceStep, - _ => secretsStep, - _ => envSecretsStep, - _ => localDirsStep, - _ => mountVolumesStep, - _ => pythonStep, - _ => rStep, - _ => javaStep) - - test("Apply fundamental steps all the time.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - Some(JavaMainAppResource("example.jar")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - JAVA_STEP_TYPE) - } - - test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - None, - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("EnvName" -> "SecretName:secretKey"), - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - SECRETS_STEP_TYPE, - ENV_SECRETS_STEP_TYPE, - JAVA_STEP_TYPE) - } +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s._ +import org.apache.spark.internal.config.ConfigEntry - test("Apply Java step if main resource is none.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - None, - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - JAVA_STEP_TYPE) - } +class KubernetesDriverBuilderSuite extends PodBuilderSuite { - test("Apply Python step if main resource is python.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - Some(PythonMainAppResource("example.py")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - PYSPARK_STEP_TYPE) + override protected def templateFileConf: ConfigEntry[_] = { + Config.KUBERNETES_DRIVER_PODTEMPLATE_FILE } - test("Apply volumes step if mounts are present.") { - val volumeSpec = KubernetesVolumeSpec( - "volume", - "/tmp", - false, - KubernetesHostPathVolumeConf("/path")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - None, - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE, - JAVA_STEP_TYPE) + override protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod = { + val conf = KubernetesTestConf.createDriverConf(sparkConf = sparkConf) + new KubernetesDriverBuilder().buildFromFeatures(conf, client).pod } - test("Apply R step if main resource is R.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesDriverSpecificConf( - Some(RMainAppResource("example.R")), - "test-app", - "main", - Seq.empty), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - CREDENTIALS_STEP_TYPE, - SERVICE_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - R_STEP_TYPE) - } - - private def validateStepTypesApplied(resolvedSpec: KubernetesDriverSpec, stepTypes: String*) - : Unit = { - assert(resolvedSpec.systemProperties.size === stepTypes.size) - stepTypes.foreach { stepType => - assert(resolvedSpec.pod.pod.getMetadata.getLabels.get(stepType) === stepType) - assert(resolvedSpec.driverKubernetesResources.containsSlice( - KubernetesFeaturesTestUtils.getSecretsForStepType(stepType))) - assert(resolvedSpec.systemProperties(stepType) === stepType) - } - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala index c6b667ed85e8c..2e883623a4b1c 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorLifecycleTestUtils.scala @@ -82,7 +82,7 @@ object ExecutorLifecycleTestUtils { def deletedExecutor(executorId: Long): Pod = { new PodBuilder(podWithAttachedContainerForId(executorId)) .editOrNewMetadata() - .withNewDeletionTimestamp("523012521") + .withDeletionTimestamp("523012521") .endMetadata() .build() } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala index 0e617b0021019..5862f647ccb90 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsAllocatorSuite.scala @@ -19,15 +19,15 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.api.model.{DoneablePod, Pod, PodBuilder} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource -import org.mockito.{ArgumentMatcher, Matchers, Mock, MockitoAnnotations} -import org.mockito.Matchers.any +import org.mockito.{Mock, MockitoAnnotations} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito.{never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.k8s.{KubernetesConf, KubernetesExecutorSpecificConf, SparkPod} +import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.{KubernetesExecutorConf, SparkPod} import org.apache.spark.deploy.k8s.Config._ import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ @@ -52,6 +52,7 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { private val podAllocationSize = conf.get(KUBERNETES_ALLOCATION_BATCH_SIZE) private val podAllocationDelay = conf.get(KUBERNETES_ALLOCATION_BATCH_DELAY) private val podCreationTimeout = math.max(podAllocationDelay * 5, 60000L) + private val secMgr = new SecurityManager(conf) private var waitForExecutorPodsClock: ManualClock = _ @@ -79,12 +80,12 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { when(kubernetesClient.pods()).thenReturn(podOperations) when(podOperations.withName(driverPodName)).thenReturn(driverPodOperations) when(driverPodOperations.get).thenReturn(driverPod) - when(executorBuilder.buildFromFeatures(kubernetesConfWithCorrectFields())) - .thenAnswer(executorPodAnswer()) + when(executorBuilder.buildFromFeatures(any(classOf[KubernetesExecutorConf]), meq(secMgr), + meq(kubernetesClient))).thenAnswer(executorPodAnswer()) snapshotsStore = new DeterministicExecutorPodsSnapshotsStore() waitForExecutorPodsClock = new ManualClock(0L) podsAllocatorUnderTest = new ExecutorPodsAllocator( - conf, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) + conf, secMgr, executorBuilder, kubernetesClient, snapshotsStore, waitForExecutorPodsClock) podsAllocatorUnderTest.start(TEST_SPARK_APP_ID) } @@ -138,52 +139,23 @@ class ExecutorPodsAllocatorSuite extends SparkFunSuite with BeforeAndAfter { snapshotsStore.notifySubscribers() snapshotsStore.replaceSnapshot(Seq.empty[Pod]) waitForExecutorPodsClock.setTime(podCreationTimeout + 1) - when(podOperations.withLabel(SPARK_EXECUTOR_ID_LABEL, "1")).thenReturn(labeledPods) + when(podOperations + .withLabel(SPARK_APP_ID_LABEL, TEST_SPARK_APP_ID)) + .thenReturn(podOperations) + when(podOperations + withLabel(SPARK_ROLE_LABEL, SPARK_POD_EXECUTOR_ROLE)) + .thenReturn(podOperations) + when(podOperations + .withLabel(SPARK_EXECUTOR_ID_LABEL, "1")) + .thenReturn(labeledPods) snapshotsStore.notifySubscribers() verify(labeledPods).delete() verify(podOperations).create(podWithAttachedContainerForId(2)) } - private def executorPodAnswer(): Answer[SparkPod] = { - new Answer[SparkPod] { - override def answer(invocation: InvocationOnMock): SparkPod = { - val k8sConf = invocation.getArgumentAt( - 0, classOf[KubernetesConf[KubernetesExecutorSpecificConf]]) - executorPodWithId(k8sConf.roleSpecificConf.executorId.toInt) - } - } + private def executorPodAnswer(): Answer[SparkPod] = + (invocation: InvocationOnMock) => { + val k8sConf: KubernetesExecutorConf = invocation.getArgument(0) + executorPodWithId(k8sConf.executorId.toInt) } - - private def kubernetesConfWithCorrectFields(): KubernetesConf[KubernetesExecutorSpecificConf] = - Matchers.argThat(new ArgumentMatcher[KubernetesConf[KubernetesExecutorSpecificConf]] { - override def matches(argument: scala.Any): Boolean = { - if (!argument.isInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]]) { - false - } else { - val k8sConf = argument.asInstanceOf[KubernetesConf[KubernetesExecutorSpecificConf]] - val executorSpecificConf = k8sConf.roleSpecificConf - val expectedK8sConf = KubernetesConf.createExecutorConf( - conf, - executorSpecificConf.executorId, - TEST_SPARK_APP_ID, - Some(driverPod)) - - // Set prefixes to a common string since KUBERNETES_EXECUTOR_POD_NAME_PREFIX - // has not be set for the tests and thus KubernetesConf will use a random - // string for the prefix, based on the app name, and this comparison here will fail. - val k8sConfCopy = k8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - val expectedK8sConfCopy = expectedK8sConf - .copy(appResourceNamePrefix = "") - .copy(sparkConf = conf) - - k8sConf.sparkConf.getAll.toMap == conf.getAll.toMap && - // Since KubernetesConf.createExecutorConf clones the SparkConf object, force - // deep equality comparison for the SparkConf object and use object equality - // comparison on all other fields. - k8sConfCopy == expectedK8sConfCopy - } - } - }) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala index d8409383b4a1c..9920f4d3ea737 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/ExecutorPodsLifecycleManagerSuite.scala @@ -21,15 +21,15 @@ import io.fabric8.kubernetes.api.model.{DoneablePod, Pod} import io.fabric8.kubernetes.client.KubernetesClient import io.fabric8.kubernetes.client.dsl.PodResource import org.mockito.{Mock, MockitoAnnotations} -import org.mockito.Matchers.any -import org.mockito.Mockito.{mock, times, verify, when} +import org.mockito.ArgumentMatchers.any +import org.mockito.Mockito.{mock, never, times, verify, when} import org.mockito.invocation.InvocationOnMock import org.mockito.stubbing.Answer import org.scalatest.BeforeAndAfter -import scala.collection.JavaConverters._ import scala.collection.mutable import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.k8s.Config import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.deploy.k8s.KubernetesUtils._ import org.apache.spark.scheduler.ExecutorExited @@ -45,9 +45,6 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte @Mock private var podOperations: PODS = _ - @Mock - private var executorBuilder: KubernetesExecutorBuilder = _ - @Mock private var schedulerBackend: KubernetesClusterSchedulerBackend = _ @@ -64,7 +61,6 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte when(podOperations.withName(any(classOf[String]))).thenAnswer(namedPodsAnswer()) eventHandlerUnderTest = new ExecutorPodsLifecycleManager( new SparkConf(), - executorBuilder, kubernetesClient, snapshotsStore, removedExecutorsCache) @@ -104,6 +100,17 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) } + test("Keep executor pods in k8s if configured.") { + val failedPod = failedExecutorWithoutDeletion(1) + eventHandlerUnderTest.conf.set(Config.KUBERNETES_DELETE_EXECUTORS, false) + snapshotsStore.updatePod(failedPod) + snapshotsStore.notifySubscribers() + val msg = exitReasonMessage(1, failedPod) + val expectedLossReason = ExecutorExited(1, exitCausedByApp = true, msg) + verify(schedulerBackend).doRemoveExecutor("1", expectedLossReason) + verify(podOperations, never()).delete() + } + private def exitReasonMessage(failedExecutorId: Int, failedPod: Pod): String = { val reason = Option(failedPod.getStatus.getReason) val message = Option(failedPod.getStatus.getMessage) @@ -117,13 +124,10 @@ class ExecutorPodsLifecycleManagerSuite extends SparkFunSuite with BeforeAndAfte """.stripMargin } - private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = { - new Answer[PodResource[Pod, DoneablePod]] { - override def answer(invocation: InvocationOnMock): PodResource[Pod, DoneablePod] = { - val podName = invocation.getArgumentAt(0, classOf[String]) - namedExecutorPods.getOrElseUpdate( - podName, mock(classOf[PodResource[Pod, DoneablePod]])) - } + private def namedPodsAnswer(): Answer[PodResource[Pod, DoneablePod]] = + (invocation: InvocationOnMock) => { + val podName: String = invocation.getArgument(0) + namedExecutorPods.getOrElseUpdate( + podName, mock(classOf[PodResource[Pod, DoneablePod]])) } - } } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala index 52e7a12dbaf06..8ed934d91dd7e 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesClusterSchedulerBackendSuite.scala @@ -19,11 +19,11 @@ package org.apache.spark.scheduler.cluster.k8s import io.fabric8.kubernetes.client.KubernetesClient import org.jmock.lib.concurrent.DeterministicScheduler import org.mockito.{ArgumentCaptor, Mock, MockitoAnnotations} -import org.mockito.Matchers.{eq => mockitoEq} +import org.mockito.ArgumentMatchers.{eq => mockitoEq} import org.mockito.Mockito.{never, verify, when} import org.scalatest.BeforeAndAfter -import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkFunSuite} import org.apache.spark.deploy.k8s.Constants._ import org.apache.spark.deploy.k8s.Fabric8Aliases._ import org.apache.spark.rpc.{RpcEndpoint, RpcEndpointRef, RpcEnv} @@ -37,10 +37,14 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn private val requestExecutorsService = new DeterministicScheduler() private val sparkConf = new SparkConf(false) .set("spark.executor.instances", "3") + .set("spark.app.id", TEST_SPARK_APP_ID) @Mock private var sc: SparkContext = _ + @Mock + private var env: SparkEnv = _ + @Mock private var rpcEnv: RpcEnv = _ @@ -81,23 +85,25 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn MockitoAnnotations.initMocks(this) when(taskScheduler.sc).thenReturn(sc) when(sc.conf).thenReturn(sparkConf) + when(sc.env).thenReturn(env) + when(env.rpcEnv).thenReturn(rpcEnv) driverEndpoint = ArgumentCaptor.forClass(classOf[RpcEndpoint]) - when(rpcEnv.setupEndpoint( - mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), driverEndpoint.capture())) + when( + rpcEnv.setupEndpoint( + mockitoEq(CoarseGrainedSchedulerBackend.ENDPOINT_NAME), + driverEndpoint.capture())) .thenReturn(driverEndpointRef) when(kubernetesClient.pods()).thenReturn(podOperations) schedulerBackendUnderTest = new KubernetesClusterSchedulerBackend( taskScheduler, - rpcEnv, + sc, kubernetesClient, requestExecutorsService, eventQueue, podAllocator, lifecycleEventHandler, watchEvents, - pollEvents) { - override def applicationId(): String = TEST_SPARK_APP_ID - } + pollEvents) } test("Start all components") { @@ -122,8 +128,7 @@ class KubernetesClusterSchedulerBackendSuite extends SparkFunSuite with BeforeAn test("Remove executor") { schedulerBackendUnderTest.start() - schedulerBackendUnderTest.doRemoveExecutor( - "1", ExecutorKilled) + schedulerBackendUnderTest.doRemoveExecutor("1", ExecutorKilled) verify(driverEndpointRef).send(RemoveExecutor("1", ExecutorKilled)) } diff --git a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala index 44fe4a24e1102..bd716174a8271 100644 --- a/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala +++ b/resource-managers/kubernetes/core/src/test/scala/org/apache/spark/scheduler/cluster/k8s/KubernetesExecutorBuilderSuite.scala @@ -16,107 +16,23 @@ */ package org.apache.spark.scheduler.cluster.k8s -import io.fabric8.kubernetes.api.model.PodBuilder +import io.fabric8.kubernetes.client.KubernetesClient -import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.k8s._ -import org.apache.spark.deploy.k8s.features._ +import org.apache.spark.internal.config.ConfigEntry -class KubernetesExecutorBuilderSuite extends SparkFunSuite { - private val BASIC_STEP_TYPE = "basic" - private val SECRETS_STEP_TYPE = "mount-secrets" - private val ENV_SECRETS_STEP_TYPE = "env-secrets" - private val LOCAL_DIRS_STEP_TYPE = "local-dirs" - private val MOUNT_VOLUMES_STEP_TYPE = "mount-volumes" +class KubernetesExecutorBuilderSuite extends PodBuilderSuite { - private val basicFeatureStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - BASIC_STEP_TYPE, classOf[BasicExecutorFeatureStep]) - private val mountSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - SECRETS_STEP_TYPE, classOf[MountSecretsFeatureStep]) - private val envSecretsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - ENV_SECRETS_STEP_TYPE, classOf[EnvSecretsFeatureStep]) - private val localDirsStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - LOCAL_DIRS_STEP_TYPE, classOf[LocalDirsFeatureStep]) - private val mountVolumesStep = KubernetesFeaturesTestUtils.getMockConfigStepForStepType( - MOUNT_VOLUMES_STEP_TYPE, classOf[MountVolumesFeatureStep]) - - private val builderUnderTest = new KubernetesExecutorBuilder( - _ => basicFeatureStep, - _ => mountSecretsStep, - _ => envSecretsStep, - _ => localDirsStep, - _ => mountVolumesStep) - - test("Basic steps are consistently applied.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), BASIC_STEP_TYPE, LOCAL_DIRS_STEP_TYPE) + override protected def templateFileConf: ConfigEntry[_] = { + Config.KUBERNETES_EXECUTOR_PODTEMPLATE_FILE } - test("Apply secrets step if secrets are present.") { - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map("secret" -> "secretMountPath"), - Map("secret-name" -> "secret-key"), - Map.empty, - Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - SECRETS_STEP_TYPE, - ENV_SECRETS_STEP_TYPE) + override protected def buildPod(sparkConf: SparkConf, client: KubernetesClient): SparkPod = { + sparkConf.set("spark.driver.host", "https://driver.host.com") + val conf = KubernetesTestConf.createExecutorConf(sparkConf = sparkConf) + val secMgr = new SecurityManager(sparkConf) + new KubernetesExecutorBuilder().buildFromFeatures(conf, secMgr, client) } - test("Apply volumes step if mounts are present.") { - val volumeSpec = KubernetesVolumeSpec( - "volume", - "/tmp", - false, - KubernetesHostPathVolumeConf("/checkpoint")) - val conf = KubernetesConf( - new SparkConf(false), - KubernetesExecutorSpecificConf( - "executor-id", Some(new PodBuilder().build())), - "prefix", - "appId", - Map.empty, - Map.empty, - Map.empty, - Map.empty, - Map.empty, - volumeSpec :: Nil, - Seq.empty[String]) - validateStepTypesApplied( - builderUnderTest.buildFromFeatures(conf), - BASIC_STEP_TYPE, - LOCAL_DIRS_STEP_TYPE, - MOUNT_VOLUMES_STEP_TYPE) - } - - private def validateStepTypesApplied(resolvedPod: SparkPod, stepTypes: String*): Unit = { - assert(resolvedPod.pod.getMetadata.getLabels.size === stepTypes.size) - stepTypes.foreach { stepType => - assert(resolvedPod.pod.getMetadata.getLabels.get(stepType) === stepType) - } - } } diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile index 1c4dcd5476872..871d34b11e174 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/Dockerfile @@ -17,9 +17,7 @@ FROM openjdk:8-alpine -ARG spark_jars=jars -ARG img_path=kubernetes/dockerfiles -ARG k8s_tests=kubernetes/tests +ARG spark_uid=185 # Before building the docker image, first build and make a Spark distribution following # the instructions in http://spark.apache.org/docs/latest/building-spark.html. @@ -30,8 +28,10 @@ ARG k8s_tests=kubernetes/tests RUN set -ex && \ apk upgrade --no-cache && \ - apk add --no-cache bash tini libc6-compat linux-pam && \ + ln -s /lib /lib64 && \ + apk add --no-cache bash tini libc6-compat linux-pam krb5 krb5-libs nss && \ mkdir -p /opt/spark && \ + mkdir -p /opt/spark/examples && \ mkdir -p /opt/spark/work-dir && \ touch /opt/spark/RELEASE && \ rm /bin/sh && \ @@ -39,16 +39,20 @@ RUN set -ex && \ echo "auth required pam_wheel.so use_uid" >> /etc/pam.d/su && \ chgrp root /etc/passwd && chmod ug+rw /etc/passwd -COPY ${spark_jars} /opt/spark/jars +COPY jars /opt/spark/jars COPY bin /opt/spark/bin COPY sbin /opt/spark/sbin -COPY ${img_path}/spark/entrypoint.sh /opt/ +COPY kubernetes/dockerfiles/spark/entrypoint.sh /opt/ COPY examples /opt/spark/examples -COPY ${k8s_tests} /opt/spark/tests +COPY kubernetes/tests /opt/spark/tests COPY data /opt/spark/data ENV SPARK_HOME /opt/spark WORKDIR /opt/spark/work-dir +RUN chmod g+w /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +USER ${spark_uid} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile index 9f67422efeb3c..34d449c9f08b9 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/R/Dockerfile @@ -16,8 +16,13 @@ # ARG base_img + FROM $base_img WORKDIR / + +# Reset to root to run installation tasks +USER 0 + RUN mkdir ${SPARK_HOME}/R RUN apk add --no-cache R R-dev @@ -27,3 +32,7 @@ ENV R_HOME /usr/lib/R WORKDIR /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +ARG spark_uid=185 +USER ${spark_uid} diff --git a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile index 69b6efa6149a0..8237c9223223a 100644 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/bindings/python/Dockerfile @@ -16,8 +16,13 @@ # ARG base_img + FROM $base_img WORKDIR / + +# Reset to root to run installation tasks +USER 0 + RUN mkdir ${SPARK_HOME}/python # TODO: Investigate running both pip and pip3 via virtualenvs RUN apk add --no-cache python && \ @@ -32,8 +37,12 @@ RUN apk add --no-cache python && \ # Removed the .cache to save space rm -r /root/.cache +COPY python/pyspark ${SPARK_HOME}/python/pyspark COPY python/lib ${SPARK_HOME}/python/lib -ENV PYTHONPATH ${SPARK_HOME}/python/lib/pyspark.zip:${SPARK_HOME}/python/lib/py4j-*.zip WORKDIR /opt/spark/work-dir ENTRYPOINT [ "/opt/entrypoint.sh" ] + +# Specify the User that the actual main process will run as +ARG spark_uid=185 +USER ${spark_uid} 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 216e8fe31becb..2097fb8865de9 100755 --- a/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh +++ b/resource-managers/kubernetes/docker/src/main/dockerfiles/spark/entrypoint.sh @@ -30,25 +30,12 @@ set -e # If there is no passwd entry for the container UID, attempt to create one if [ -z "$uidentry" ] ; then if [ -w /etc/passwd ] ; then - echo "$myuid:x:$myuid:$mygid:anonymous uid:$SPARK_HOME:/bin/false" >> /etc/passwd + echo "$myuid:x:$myuid:$mygid:${SPARK_USER_NAME:-anonymous uid}:$SPARK_HOME:/bin/false" >> /etc/passwd else echo "Container ENTRYPOINT failed to add passwd entry for anonymous UID" fi fi -SPARK_K8S_CMD="$1" -case "$SPARK_K8S_CMD" in - driver | driver-py | driver-r | executor) - shift 1 - ;; - "") - ;; - *) - echo "Non-spark-on-k8s command provided, proceeding in pass-through mode..." - exec /sbin/tini -s -- "$@" - ;; -esac - SPARK_CLASSPATH="$SPARK_CLASSPATH:${SPARK_HOME}/jars/*" env | grep SPARK_JAVA_OPT_ | sort -t_ -k4 -n | sed 's/[^=]*=\(.*\)/\1/g' > /tmp/java_opts.txt readarray -t SPARK_EXECUTOR_JAVA_OPTS < /tmp/java_opts.txt @@ -57,20 +44,6 @@ if [ -n "$SPARK_EXTRA_CLASSPATH" ]; then SPARK_CLASSPATH="$SPARK_CLASSPATH:$SPARK_EXTRA_CLASSPATH" fi -if [ -n "$PYSPARK_FILES" ]; then - PYTHONPATH="$PYTHONPATH:$PYSPARK_FILES" -fi - -PYSPARK_ARGS="" -if [ -n "$PYSPARK_APP_ARGS" ]; then - PYSPARK_ARGS="$PYSPARK_APP_ARGS" -fi - -R_ARGS="" -if [ -n "$R_APP_ARGS" ]; then - R_ARGS="$R_APP_ARGS" -fi - if [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "2" ]; then pyv="$(python -V 2>&1)" export PYTHON_VERSION="${pyv:7}" @@ -83,8 +56,13 @@ elif [ "$PYSPARK_MAJOR_PYTHON_VERSION" == "3" ]; then export PYSPARK_DRIVER_PYTHON="python3" fi -case "$SPARK_K8S_CMD" in +if ! [ -z ${HADOOP_CONF_DIR+x} ]; then + SPARK_CLASSPATH="$HADOOP_CONF_DIR:$SPARK_CLASSPATH"; +fi + +case "$1" in driver) + shift 1 CMD=( "$SPARK_HOME/bin/spark-submit" --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" @@ -92,23 +70,8 @@ case "$SPARK_K8S_CMD" in "$@" ) ;; - driver-py) - CMD=( - "$SPARK_HOME/bin/spark-submit" - --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" - --deploy-mode client - "$@" $PYSPARK_PRIMARY $PYSPARK_ARGS - ) - ;; - driver-r) - CMD=( - "$SPARK_HOME/bin/spark-submit" - --conf "spark.driver.bindAddress=$SPARK_DRIVER_BIND_ADDRESS" - --deploy-mode client - "$@" $R_PRIMARY $R_ARGS - ) - ;; executor) + shift 1 CMD=( ${JAVA_HOME}/bin/java "${SPARK_EXECUTOR_JAVA_OPTS[@]}" @@ -125,8 +88,9 @@ case "$SPARK_K8S_CMD" in ;; *) - echo "Unknown command: $SPARK_K8S_CMD" 1>&2 - exit 1 + echo "Non-spark-on-k8s command provided, proceeding in pass-through mode..." + CMD=("$@") + ;; esac # Execute the container CMD under tini for better hygiene diff --git a/resource-managers/kubernetes/integration-tests/README.md b/resource-managers/kubernetes/integration-tests/README.md index b3863e6b7d1af..ea8286124a68c 100644 --- a/resource-managers/kubernetes/integration-tests/README.md +++ b/resource-managers/kubernetes/integration-tests/README.md @@ -8,45 +8,235 @@ title: Spark on Kubernetes Integration Tests Note that the integration test framework is currently being heavily revised and is subject to change. Note that currently the integration tests only run with Java 8. -The simplest way to run the integration tests is to install and run Minikube, then run the following: +The simplest way to run the integration tests is to install and run Minikube, then run the following from this +directory: dev/dev-run-integration-tests.sh The minimum tested version of Minikube is 0.23.0. The kube-dns addon must be enabled. Minikube should -run with a minimum of 3 CPUs and 4G of memory: +run with a minimum of 4 CPUs and 6G of memory: - minikube start --cpus 3 --memory 4096 + minikube start --cpus 4 --memory 6144 You can download Minikube [here](https://github.com/kubernetes/minikube/releases). # Integration test customization -Configuration of the integration test runtime is done through passing different arguments to the test script. The main useful options are outlined below. +Configuration of the integration test runtime is done through passing different arguments to the test script. +The main useful options are outlined below. + +## Using a different backend + +The integration test backend i.e. the K8S cluster used for testing is controlled by the `--deploy-mode` option. By +default this is set to `minikube`, the available backends are their prerequisites are as follows. + +### `minikube` + +Uses the local `minikube` cluster, this requires that `minikube` 0.23.0 or greater be installed and that it be allocated +at least 4 CPUs and 6GB memory (some users have reported success with as few as 3 CPUs and 4GB memory). The tests will +check if `minikube` is started and abort early if it isn't currently running. + +### `docker-for-desktop` + +Since July 2018 Docker for Desktop provide an optional Kubernetes cluster that can be enabled as described in this +[blog post](https://blog.docker.com/2018/07/kubernetes-is-now-available-in-docker-desktop-stable-channel/). Assuming +this is enabled using this backend will auto-configure itself from the `docker-for-desktop` context that Docker creates +in your `~/.kube/config` file. If your config file is in a different location you should set the `KUBECONFIG` +environment variable appropriately. + +### `cloud` + +The cloud backend configures the tests to use an arbitrary Kubernetes cluster running in the cloud or otherwise. + +The `cloud` backend auto-configures the cluster to use from your K8S config file, this is assumed to be `~/.kube/config` +unless the `KUBECONFIG` environment variable is set to override this location. By default this will use whatever your +current context is in the config file, to use an alternative context from your config file you can specify the +`--context ` flag with the desired context. + +You can optionally use a different K8S master URL than the one your K8S config file specified, this should be supplied +via the `--spark-master ` flag. ## Re-using Docker Images By default, the test framework will build new Docker images on every test execution. A unique image tag is generated, -and it is written to file at `target/imageTag.txt`. To reuse the images built in a previous run, or to use a Docker image tag -that you have built by other means already, pass the tag to the test script: +and it is written to file at `target/imageTag.txt`. To reuse the images built in a previous run, or to use a Docker +image tag that you have built by other means already, pass the tag to the test script: dev/dev-run-integration-tests.sh --image-tag where if you still want to use images that were built before by the test framework: dev/dev-run-integration-tests.sh --image-tag $(cat target/imageTag.txt) + +### Customising the Image Names + +If your image names do not follow the standard Spark naming convention - `spark`, `spark-py` and `spark-r` - then you can customise the names using several options. + +If you use the same basic pattern but a different prefix for the name e.g. `apache-spark` you can just set `--base-image-name ` e.g. + + dev/dev-run-integration-tests.sh --base-image-name apache-spark + +Alternatively if you use completely custom names then you can set each individually via the `--jvm-image-name `, `--python-image-name ` and `--r-image-name ` arguments e.g. + + dev/dev-run-integration-tests.sh --jvm-image-name jvm-spark --python-image-name pyspark --r-image-name sparkr ## Spark Distribution Under Test -The Spark code to test is handed to the integration test system via a tarball. Here is the option that is used to specify the tarball: +The Spark code to test is handed to the integration test system via a tarball. Here is the option that is used to +specify the tarball: * `--spark-tgz ` - set `` to point to a tarball containing the Spark distribution to test. -TODO: Don't require the packaging of the built Spark artifacts into this tarball, just read them out of the current tree. +This Tarball should be created by first running `dev/make-distribution.sh` passing the `--tgz` flag and `-Pkubernetes` +as one of the options to ensure that Kubernetes support is included in the distribution. For more details on building a +runnable distribution please see the +[Building Spark](https://spark.apache.org/docs/latest/building-spark.html#building-a-runnable-distribution) +documentation. + +**TODO:** Don't require the packaging of the built Spark artifacts into this tarball, just read them out of the current +tree. ## Customizing the Namespace and Service Account -* `--namespace ` - set `` to the namespace in which the tests should be run. -* `--service-account ` - set `` to the name of the Kubernetes service account to -use in the namespace specified by the `--namespace`. The service account is expected to have permissions to get, list, watch, -and create pods. For clusters with RBAC turned on, it's important that the right permissions are granted to the service account -in the namespace through an appropriate role and role binding. A reference RBAC configuration is provided in `dev/spark-rbac.yaml`. +If no namespace is specified then a temporary namespace will be created and deleted during the test run. Similarly if +no service account is specified then the `default` service account for the namespace will be used. + +Using the `--namespace ` flag sets `` to the namespace in which the tests should be run. If this +is supplied then the tests assume this namespace exists in the K8S cluster and will not attempt to create it. +Additionally this namespace must have an appropriately authorized service account which can be customised via the +`--service-account` flag. + +The `--service-account ` flag sets `` to the name of the Kubernetes service +account to use in the namespace specified by the `--namespace` flag. The service account is expected to have permissions +to get, list, watch, and create pods. For clusters with RBAC turned on, it's important that the right permissions are +granted to the service account in the namespace through an appropriate role and role binding. A reference RBAC +configuration is provided in `dev/spark-rbac.yaml`. + +# Running the Test Directly + +If you prefer to run just the integration tests directly, then you can customise the behaviour via passing system +properties to Maven. For example: + + mvn integration-test -am -pl :spark-kubernetes-integration-tests_2.12 \ + -Pkubernetes -Pkubernetes-integration-tests \ + -Phadoop-2.7 -Dhadoop.version=2.7.4 \ + -Dspark.kubernetes.test.sparkTgz=spark-3.0.0-SNAPSHOT-bin-example.tgz \ + -Dspark.kubernetes.test.imageTag=sometag \ + -Dspark.kubernetes.test.imageRepo=docker.io/somerepo \ + -Dspark.kubernetes.test.namespace=spark-int-tests \ + -Dspark.kubernetes.test.deployMode=docker-for-desktop \ + -Dtest.include.tags=k8s + + +## Available Maven Properties + +The following are the available Maven properties that can be passed. For the most part these correspond to flags passed +to the wrapper scripts and using the wrapper scripts will simply set these appropriately behind the scenes. + +
    + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + + +
    PropertyDescriptionDefault
    spark.kubernetes.test.sparkTgz + A runnable Spark distribution to test. +
    spark.kubernetes.test.unpackSparkDir + The directory where the runnable Spark distribution will be unpacked. + ${project.build.directory}/spark-dist-unpacked
    spark.kubernetes.test.deployMode + The integration test backend to use. Acceptable values are minikube, + docker-for-desktop and cloud. + minikube
    spark.kubernetes.test.kubeConfigContext + When using the cloud backend specifies the context from the users K8S config file that should be used + as the target cluster for integration testing. If not set and using the cloud backend then your + current context will be used. +
    spark.kubernetes.test.master + When using the cloud-url backend must be specified to indicate the K8S master URL to communicate + with. +
    spark.kubernetes.test.imageTag + A specific image tag to use, when set assumes images with those tags are already built and available in the + specified image repository. When set to N/A (the default) fresh images will be built. + N/A +
    spark.kubernetes.test.imageTagFile + A file containing the image tag to use, if no specific image tag is set then fresh images will be built with a + generated tag and that tag written to this file. + ${project.build.directory}/imageTag.txt
    spark.kubernetes.test.imageRepo + The Docker image repository that contains the images to be used if a specific image tag is set or to which the + images will be pushed to if fresh images are being built. + docker.io/kubespark
    spark.kubernetes.test.jvmImage + The image name for the JVM based Spark image to test + spark
    spark.kubernetes.test.pythonImage + The image name for the Python based Spark image to test + spark-py
    spark.kubernetes.test.rImage + The image name for the R based Spark image to test + spark-r
    spark.kubernetes.test.namespace + A specific Kubernetes namespace to run the tests in. If specified then the tests assume that this namespace + already exists. When not specified a temporary namespace for the tests will be created and deleted as part of the + test run. +
    spark.kubernetes.test.serviceAccountName + A specific Kubernetes service account to use for running the tests. If not specified then the namespaces default + service account will be used and that must have sufficient permissions or the tests will fail. +
    diff --git a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh index e26c0b3a39c90..ecfe4058f5598 100755 --- a/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh +++ b/resource-managers/kubernetes/integration-tests/dev/dev-run-integration-tests.sh @@ -23,11 +23,22 @@ DEPLOY_MODE="minikube" IMAGE_REPO="docker.io/kubespark" SPARK_TGZ="N/A" IMAGE_TAG="N/A" +BASE_IMAGE_NAME= +JVM_IMAGE_NAME= +PYTHON_IMAGE_NAME= +R_IMAGE_NAME= SPARK_MASTER= NAMESPACE= SERVICE_ACCOUNT= +CONTEXT= INCLUDE_TAGS="k8s" EXCLUDE_TAGS= +MVN="$TEST_ROOT_DIR/build/mvn" + +SCALA_VERSION=$("$MVN" help:evaluate -Dexpression=scala.binary.version 2>/dev/null\ + | grep -v "INFO"\ + | grep -v "WARNING"\ + | tail -n 1) # Parse arguments while (( "$#" )); do @@ -60,6 +71,10 @@ while (( "$#" )); do SERVICE_ACCOUNT="$2" shift ;; + --context) + CONTEXT="$2" + shift + ;; --include-tags) INCLUDE_TAGS="k8s,$2" shift @@ -68,6 +83,22 @@ while (( "$#" )); do EXCLUDE_TAGS="$2" shift ;; + --base-image-name) + BASE_IMAGE_NAME="$2" + shift + ;; + --jvm-image-name) + JVM_IMAGE_NAME="$2" + shift + ;; + --python-image-name) + PYTHON_IMAGE_NAME="$2" + shift + ;; + --r-image-name) + R_IMAGE_NAME="$2" + shift + ;; *) break ;; @@ -93,6 +124,11 @@ then properties=( ${properties[@]} -Dspark.kubernetes.test.serviceAccountName=$SERVICE_ACCOUNT ) fi +if [ -n $CONTEXT ]; +then + properties=( ${properties[@]} -Dspark.kubernetes.test.kubeConfigContext=$CONTEXT ) +fi + if [ -n $SPARK_MASTER ]; then properties=( ${properties[@]} -Dspark.kubernetes.test.master=$SPARK_MASTER ) @@ -103,4 +139,15 @@ then properties=( ${properties[@]} -Dtest.exclude.tags=$EXCLUDE_TAGS ) fi -$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pkubernetes ${properties[@]} +BASE_IMAGE_NAME=${BASE_IMAGE_NAME:-spark} +JVM_IMAGE_NAME=${JVM_IMAGE_NAME:-${BASE_IMAGE_NAME}} +PYTHON_IMAGE_NAME=${PYTHON_IMAGE_NAME:-${BASE_IMAGE_NAME}-py} +R_IMAGE_NAME=${R_IMAGE_NAME:-${BASE_IMAGE_NAME}-r} + +properties+=( + -Dspark.kubernetes.test.jvmImage=$JVM_IMAGE_NAME + -Dspark.kubernetes.test.pythonImage=$PYTHON_IMAGE_NAME + -Dspark.kubernetes.test.rImage=$R_IMAGE_NAME +) + +$TEST_ROOT_DIR/build/mvn integration-test -f $TEST_ROOT_DIR/pom.xml -pl resource-managers/kubernetes/integration-tests -am -Pscala-$SCALA_VERSION -Pkubernetes -Pkubernetes-integration-tests ${properties[@]} diff --git a/resource-managers/kubernetes/integration-tests/pom.xml b/resource-managers/kubernetes/integration-tests/pom.xml index 23453c8957b28..d129ffb3601c0 100644 --- a/resource-managers/kubernetes/integration-tests/pom.xml +++ b/resource-managers/kubernetes/integration-tests/pom.xml @@ -19,25 +19,34 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../../pom.xml - spark-kubernetes-integration-tests_2.11 + spark-kubernetes-integration-tests_2.12 1.3.0 1.4.0 - 3.0.0 + 4.1.2 3.2.2 1.0 kubernetes-integration-tests + + + + ${project.build.directory}/spark-dist-unpacked N/A ${project.build.directory}/imageTag.txt minikube docker.io/kubespark + + + + + @@ -67,6 +76,12 @@ spark-tags_${scala.binary.version} test-jar + + com.amazonaws + aws-java-sdk + 1.7.4 + test + @@ -126,7 +141,7 @@ ${project.build.directory}/surefire-reports . SparkTestSuite.txt - -ea -Xmx3g -XX:ReservedCodeCacheSize=512m ${extraScalaTestArgs} + -ea -Xmx4g -XX:ReservedCodeCacheSize=512m ${extraScalaTestArgs} file:src/test/resources/log4j.properties @@ -135,9 +150,13 @@ ${spark.kubernetes.test.unpackSparkDir} ${spark.kubernetes.test.imageRepo} ${spark.kubernetes.test.deployMode} + ${spark.kubernetes.test.kubeConfigContext} ${spark.kubernetes.test.master} ${spark.kubernetes.test.namespace} ${spark.kubernetes.test.serviceAccountName} + ${spark.kubernetes.test.jvmImage} + ${spark.kubernetes.test.pythonImage} + ${spark.kubernetes.test.rImage} ${test.exclude.tags} ${test.include.tags} @@ -145,14 +164,10 @@ test + none test - - - (?<!Suite) - integration-test diff --git a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh index ccfb8e767c529..84c42cb19ef5c 100755 --- a/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh +++ b/resource-managers/kubernetes/integration-tests/scripts/setup-integration-test-env.sh @@ -16,6 +16,7 @@ # See the License for the specific language governing permissions and # limitations under the License. # +set -ex TEST_ROOT_DIR=$(git rev-parse --show-toplevel) UNPACKED_SPARK_TGZ="$TEST_ROOT_DIR/target/spark-dist-unpacked" IMAGE_TAG_OUTPUT_FILE="$TEST_ROOT_DIR/target/image-tag.txt" @@ -58,32 +59,64 @@ while (( "$#" )); do shift done -if [[ $SPARK_TGZ == "N/A" ]]; +rm -rf "$UNPACKED_SPARK_TGZ" +if [[ $SPARK_TGZ == "N/A" && $IMAGE_TAG == "N/A" ]]; then - echo "Must specify a Spark tarball to build Docker images against with --spark-tgz." && exit 1; + # If there is no spark image tag to test with and no src dir, build from current + SCRIPT_DIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" >/dev/null 2>&1 && pwd )" + SPARK_INPUT_DIR="$(cd "$SCRIPT_DIR/"../../../../ >/dev/null 2>&1 && pwd )" + DOCKER_FILE_BASE_PATH="$SPARK_INPUT_DIR/resource-managers/kubernetes/docker/src/main/dockerfiles/spark" +elif [[ $IMAGE_TAG == "N/A" ]]; +then + # If there is a test src tarball and no image tag we will want to build from that + mkdir -p $UNPACKED_SPARK_TGZ + tar -xzvf $SPARK_TGZ --strip-components=1 -C $UNPACKED_SPARK_TGZ; + SPARK_INPUT_DIR="$UNPACKED_SPARK_TGZ" + DOCKER_FILE_BASE_PATH="$SPARK_INPUT_DIR/kubernetes/dockerfiles/spark" fi -rm -rf $UNPACKED_SPARK_TGZ -mkdir -p $UNPACKED_SPARK_TGZ -tar -xzvf $SPARK_TGZ --strip-components=1 -C $UNPACKED_SPARK_TGZ; +# If there is a specific Spark image skip building and extraction/copy if [[ $IMAGE_TAG == "N/A" ]]; then IMAGE_TAG=$(uuidgen); - cd $UNPACKED_SPARK_TGZ - if [[ $DEPLOY_MODE == cloud ]] ; - then - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG build - if [[ $IMAGE_REPO == gcr.io* ]] ; - then - gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG - else - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push - fi - else - # -m option for minikube. - $UNPACKED_SPARK_TGZ/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG build - fi + cd $SPARK_INPUT_DIR + + # Build PySpark image + LANGUAGE_BINDING_BUILD_ARGS="-p $DOCKER_FILE_BASE_PATH/bindings/python/Dockerfile" + + # Build SparkR image -- disabled since this fails, re-enable as part of SPARK-25152 + # LANGUAGE_BINDING_BUILD_ARGS="$LANGUAGE_BINDING_BUILD_ARGS -R $DOCKER_FILE_BASE_PATH/bindings/R/Dockerfile" + + case $DEPLOY_MODE in + cloud) + # Build images + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + + # Push images appropriately + if [[ $IMAGE_REPO == gcr.io* ]] ; + then + gcloud docker -- push $IMAGE_REPO/spark:$IMAGE_TAG + else + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG push + fi + ;; + + docker-for-desktop) + # Only need to build as this will place it in our local Docker repo which is all + # we need for Docker for Desktop to work so no need to also push + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + ;; + + minikube) + # Only need to build and if we do this with the -m option for minikube we will + # build the images directly using the minikube Docker daemon so no need to push + $SPARK_INPUT_DIR/bin/docker-image-tool.sh -m -r $IMAGE_REPO -t $IMAGE_TAG $LANGUAGE_BINDING_BUILD_ARGS build + ;; + *) + echo "Unrecognized deploy mode $DEPLOY_MODE" && exit 1 + ;; + esac cd - fi diff --git a/external/docker/spark-mesos/Dockerfile b/resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml similarity index 58% rename from external/docker/spark-mesos/Dockerfile rename to resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml index fb3f267fe5c78..0c185be81d59e 100644 --- a/external/docker/spark-mesos/Dockerfile +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/driver-template.yml @@ -1,5 +1,3 @@ -# This is an example Dockerfile for creating a Spark image which can be -# references by the Spark property 'spark.mesos.executor.docker.image' # # Licensed to the Apache Software Foundation (ASF) under one or more # contributor license agreements. See the NOTICE file distributed with @@ -16,15 +14,13 @@ # See the License for the specific language governing permissions and # limitations under the License. # +apiVersion: v1 +Kind: Pod +metadata: + labels: + template-label-key: driver-template-label-value +spec: + containers: + - name: test-driver-container + image: will-be-overwritten -FROM mesosphere/mesos:0.20.1 - -# Update the base ubuntu image with dependencies needed for Spark -RUN apt-get update && \ - apt-get install -y python libnss3 openjdk-7-jre-headless curl - -RUN mkdir /opt/spark && \ - curl http://www.apache.org/dyn/closer.lua/spark/spark-1.4.0/spark-1.4.0-bin-hadoop2.4.tgz \ - | tar -xzC /opt -ENV SPARK_HOME /opt/spark -ENV MESOS_NATIVE_JAVA_LIBRARY /usr/local/lib/libmesos.so diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml new file mode 100644 index 0000000000000..0282e23a39bd2 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/executor-template.yml @@ -0,0 +1,25 @@ +# +# 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. +# +apiVersion: v1 +Kind: Pod +metadata: + labels: + template-label-key: executor-template-label-value +spec: + containers: + - name: test-executor-container + image: will-be-overwritten diff --git a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties index 866126bc3c1c2..ff66ba853ab8e 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties +++ b/resource-managers/kubernetes/integration-tests/src/test/resources/log4j.properties @@ -28,4 +28,4 @@ log4j.logger.com.sun.jersey=WARN log4j.logger.org.apache.hadoop=WARN log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.mortbay=WARN -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala index 4e749c40563dc..6b340f2558cca 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/BasicTestsSuite.scala @@ -52,6 +52,9 @@ private[spark] trait BasicTestsSuite { k8sSuite: KubernetesSuite => } test("Run SparkPi with an argument.", k8sTestTag) { + // This additional configuration with snappy is for SPARK-26995 + sparkAppConf + .set("spark.io.compression.codec", "snappy") runSparkPiAndVerifyCompletion(appArgs = Array("5")) } @@ -81,6 +84,13 @@ private[spark] trait BasicTestsSuite { k8sSuite: KubernetesSuite => }) } + test("All pods have the same service account by default", k8sTestTag) { + runSparkPiAndVerifyCompletion( + executorPodChecker = (executorPod: Pod) => { + doExecutorServiceAccountCheck(executorPod, kubernetesTestComponents.serviceAccountName) + }) + } + test("Run extraJVMOptions check on driver", k8sTestTag) { sparkAppConf .set("spark.driver.extraJavaOptions", "-Dspark.test.foo=spark.test.bar") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala index c8bd584516ea5..2720cdf74ca8f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ClientModeTestsSuite.scala @@ -62,11 +62,12 @@ private[spark] trait ClientModeTestsSuite { k8sSuite: KubernetesSuite => .endMetadata() .withNewSpec() .withServiceAccountName(kubernetesTestComponents.serviceAccountName) + .withRestartPolicy("Never") .addNewContainer() .withName("spark-example") .withImage(image) .withImagePullPolicy("IfNotPresent") - .withCommand("/opt/spark/bin/run-example") + .addToArgs("/opt/spark/bin/run-example") .addToArgs("--master", s"k8s://https://kubernetes.default.svc") .addToArgs("--deploy-mode", "client") .addToArgs("--conf", s"spark.kubernetes.container.image=$image") 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 new file mode 100644 index 0000000000000..b0c4182240da7 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/DepsTestsSuite.scala @@ -0,0 +1,239 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import java.net.URL + +import scala.collection.JavaConverters._ + +import com.amazonaws.auth.BasicAWSCredentials +import com.amazonaws.services.s3.AmazonS3Client +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.api.model.apps.StatefulSetBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Minutes, Span} + +import org.apache.spark.SparkException +import org.apache.spark.deploy.k8s.integrationtest.DepsTestsSuite.{DEPS_TIMEOUT, FILE_CONTENTS, HOST_PATH} +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.{INTERVAL, MinikubeTag, TIMEOUT} +import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.Minikube + +private[spark] trait DepsTestsSuite { k8sSuite: KubernetesSuite => + import KubernetesSuite.k8sTestTag + + val cName = "ceph-nano" + val svcName = s"$cName-s3" + val bucket = "spark" + + private def getCephContainer(): Container = { + val envVars = Map ( "NETWORK_AUTO_DETECT" -> "4", + "RGW_CIVETWEB_PORT" -> "8000", + "SREE_PORT" -> "5001", + "CEPH_DEMO_UID" -> "nano", + "CEPH_DAEMON" -> "demo", + "DEBUG" -> "verbose" + ).map( envV => + new EnvVarBuilder() + .withName(envV._1) + .withValue(envV._2) + .build() + ).toArray + + val resources = Map( + "cpu" -> new QuantityBuilder() + .withAmount("1") + .build(), + "memory" -> new QuantityBuilder() + .withAmount("512M") + .build() + ).asJava + + new ContainerBuilder() + .withImage("ceph/daemon:v4.0.0-stable-4.0-master-centos-7-x86_64") + .withImagePullPolicy("Always") + .withName(cName) + .withPorts(new ContainerPortBuilder() + .withName(svcName) + .withProtocol("TCP") + .withContainerPort(8000) + .build() + ) + .withResources(new ResourceRequirementsBuilder() + .withLimits(resources) + .withRequests(resources) + .build() + ) + .withEnv(envVars: _*) + .build() + } + + // Based on https://github.com/ceph/cn + private def setupCephStorage(): Unit = { + val labels = Map("app" -> "ceph", "daemon" -> "nano").asJava + val cephService = new ServiceBuilder() + .withNewMetadata() + .withName(svcName) + .withLabels(labels) + .endMetadata() + .withNewSpec() + .withPorts(new ServicePortBuilder() + .withName("https") + .withPort(8000) + .withProtocol("TCP") + .withTargetPort(new IntOrString(8000)) + .build() + ) + .withType("NodePort") + .withSelector(labels) + .endSpec() + .build() + + val cephStatefulSet = new StatefulSetBuilder() + .withNewMetadata() + .withName(cName) + .withLabels(labels) + .endMetadata() + .withNewSpec() + .withReplicas(1) + .withNewSelector() + .withMatchLabels(Map("app" -> "ceph").asJava) + .endSelector() + .withServiceName(cName) + .withNewTemplate() + .withNewMetadata() + .withName(cName) + .withLabels(labels) + .endMetadata() + .withNewSpec() + .withContainers(getCephContainer()) + .endSpec() + .endTemplate() + .endSpec() + .build() + + kubernetesTestComponents + .kubernetesClient + .services() + .create(cephService) + + kubernetesTestComponents + .kubernetesClient + .apps() + .statefulSets() + .create(cephStatefulSet) + } + + private def deleteCephStorage(): Unit = { + kubernetesTestComponents + .kubernetesClient + .apps() + .statefulSets() + .withName(cName) + .delete() + + kubernetesTestComponents + .kubernetesClient + .services() + .withName(svcName) + .delete() + } + + 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 + 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.connection.ssl.enabled", "false") + .set("spark.hadoop.fs.s3a.endpoint", s"$cephHost:$cephPort") + .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) + 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")) + } + } + + private def extractS3Key(data: String, key: String): String = { + data.split("\n") + .filter(_.contains(key)) + .head + .split(":") + .last + .trim + .replaceAll("[,|\"]", "") + } + + private def createS3Bucket(accessKey: String, secretKey: String, endPoint: String): Unit = { + Eventually.eventually(TIMEOUT, INTERVAL) { + try { + val credentials = new BasicAWSCredentials(accessKey, secretKey) + val s3client = new AmazonS3Client(credentials) + s3client.setEndpoint(endPoint) + s3client.createBucket(bucket) + } catch { + case e: Exception => + throw new SparkException(s"Failed to create bucket $bucket.", e) + } + } + } + + private def getServiceUrl(serviceName: String): String = { + Eventually.eventually(TIMEOUT, INTERVAL) { + // ns is always available either random or provided by the user + Minikube.minikubeServiceAction(serviceName, "-n", kubernetesTestComponents.namespace, "--url") + } + } +} + +private[spark] object DepsTestsSuite { + val HOST_PATH = "/tmp" + val FILE_CONTENTS = "test deps" + // increase the default because jar resolution takes time in the container + val DEPS_TIMEOUT = PatienceConfiguration.Timeout(Span(4, Minutes)) +} 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 c99a907f98d0a..80d5f239a09cc 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 @@ -19,33 +19,36 @@ package org.apache.spark.deploy.k8s.integrationtest import java.io.File import java.nio.file.{Path, Paths} import java.util.UUID -import java.util.regex.Pattern -import com.google.common.io.PatternFilenameFilter +import scala.collection.JavaConverters._ + +import com.google.common.base.Charsets +import com.google.common.io.Files import io.fabric8.kubernetes.api.model.Pod import io.fabric8.kubernetes.client.{KubernetesClientException, Watcher} import io.fabric8.kubernetes.client.Watcher.Action import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, Tag} import org.scalatest.Matchers import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.concurrent.PatienceConfiguration.{Interval, Timeout} import org.scalatest.time.{Minutes, Seconds, Span} -import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -import org.apache.spark.deploy.k8s.integrationtest.TestConfig._ +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.deploy.k8s.integrationtest.backend.{IntegrationTestBackend, IntegrationTestBackendFactory} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ -private[spark] class KubernetesSuite extends SparkFunSuite +class KubernetesSuite extends SparkFunSuite with BeforeAndAfterAll with BeforeAndAfter with BasicTestsSuite with SecretsTestsSuite - with PythonTestsSuite with ClientModeTestsSuite - with Logging with Eventually with Matchers { + with PythonTestsSuite with ClientModeTestsSuite with PodTemplateSuite with PVTestsSuite + with DepsTestsSuite with Logging with Eventually with Matchers { import KubernetesSuite._ - private var sparkHomeDir: Path = _ - private var pyImage: String = _ - private var rImage: String = _ + protected var sparkHomeDir: Path = _ + protected var pyImage: String = _ + protected var rImage: String = _ protected var image: String = _ protected var testBackend: IntegrationTestBackend = _ @@ -66,6 +69,30 @@ private[spark] class KubernetesSuite extends SparkFunSuite private val extraExecTotalMemory = s"${(1024 + memOverheadConstant*1024 + additionalMemory).toInt}Mi" + /** + * Build the image ref for the given image name, taking the repo and tag from the + * test configuration. + */ + private def testImageRef(name: String): String = { + val tag = sys.props.get(CONFIG_KEY_IMAGE_TAG_FILE) + .map { path => + val tagFile = new File(path) + require(tagFile.isFile, + s"No file found for image tag at ${tagFile.getAbsolutePath}.") + Files.toString(tagFile, Charsets.UTF_8).trim + } + .orElse(sys.props.get(CONFIG_KEY_IMAGE_TAG)) + .getOrElse { + throw new IllegalArgumentException( + s"One of $CONFIG_KEY_IMAGE_TAG_FILE or $CONFIG_KEY_IMAGE_TAG is required.") + } + val repo = sys.props.get(CONFIG_KEY_IMAGE_REPO) + .map { _ + "/" } + .getOrElse("") + + s"$repo$name:$tag" + } + override def beforeAll(): Unit = { super.beforeAll() // The scalatest-maven-plugin gives system properties that are referenced but not set null @@ -77,22 +104,25 @@ private[spark] class KubernetesSuite extends SparkFunSuite System.clearProperty(key) } - val sparkDirProp = System.getProperty("spark.kubernetes.test.unpackSparkDir") - require(sparkDirProp != null, "Spark home directory must be provided in system properties.") + val possible_spark_dirs = List( + // If someone specified the tgz for the tests look at the extraction dir + System.getProperty(CONFIG_KEY_UNPACK_DIR), + // Try the spark test home + sys.props("spark.test.home") + ) + val sparkDirProp = possible_spark_dirs.filter(x => + new File(Paths.get(x).toFile, "bin/spark-submit").exists).headOption.getOrElse(null) + require(sparkDirProp != null, + s"Spark home directory must be provided in system properties tested $possible_spark_dirs") sparkHomeDir = Paths.get(sparkDirProp) require(sparkHomeDir.toFile.isDirectory, s"No directory found for spark home specified at $sparkHomeDir.") - val imageTag = getTestImageTag - val imageRepo = getTestImageRepo - image = s"$imageRepo/spark:$imageTag" - pyImage = s"$imageRepo/spark-py:$imageTag" - rImage = s"$imageRepo/spark-r:$imageTag" - - val sparkDistroExamplesJarFile: File = sparkHomeDir.resolve(Paths.get("examples", "jars")) - .toFile - .listFiles(new PatternFilenameFilter(Pattern.compile("^spark-examples_.*\\.jar$")))(0) - containerLocalSparkDistroExamplesJar = s"local:///opt/spark/examples/jars/" + - s"${sparkDistroExamplesJarFile.getName}" + image = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_JVM, "spark")) + pyImage = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_PYTHON, "spark-py")) + rImage = testImageRef(sys.props.getOrElse(CONFIG_KEY_IMAGE_R, "spark-r")) + + containerLocalSparkDistroExamplesJar = + s"local:///opt/spark/examples/jars/${Utils.getExamplesJarName()}" testBackend = IntegrationTestBackendFactory.getTestBackend testBackend.initialize() kubernetesTestComponents = new KubernetesTestComponents(testBackend.getKubernetesClient) @@ -114,6 +144,7 @@ private[spark] class KubernetesSuite extends SparkFunSuite .set("spark.kubernetes.driver.pod.name", driverPodName) .set("spark.kubernetes.driver.label.spark-app-locator", appLocator) .set("spark.kubernetes.executor.label.spark-app-locator", appLocator) + .set(NETWORK_AUTH_ENABLED.key, "true") if (!kubernetesTestComponents.hasUserSpecifiedNamespace) { kubernetesTestComponents.createNamespace() } @@ -144,12 +175,36 @@ private[spark] class KubernetesSuite extends SparkFunSuite isJVM) } + protected def runDFSReadWriteAndVerifyCompletion( + wordCount: Int, + appResource: String = containerLocalSparkDistroExamplesJar, + driverPodChecker: Pod => Unit = doBasicDriverPodCheck, + executorPodChecker: Pod => Unit = doBasicExecutorPodCheck, + appArgs: Array[String] = Array.empty[String], + appLocator: String = appLocator, + isJVM: Boolean = true, + interval: Option[PatienceConfiguration.Interval] = None): Unit = { + runSparkApplicationAndVerifyCompletion( + appResource, + SPARK_DFS_READ_WRITE_TEST, + Seq(s"Success! Local Word Count $wordCount and " + + s"DFS Word Count $wordCount agree."), + appArgs, + driverPodChecker, + executorPodChecker, + appLocator, + isJVM, + None, + Option((interval, None))) + } + protected def runSparkRemoteCheckAndVerifyCompletion( appResource: String = containerLocalSparkDistroExamplesJar, driverPodChecker: Pod => Unit = doBasicDriverPodCheck, executorPodChecker: Pod => Unit = doBasicExecutorPodCheck, appArgs: Array[String], - appLocator: String = appLocator): Unit = { + appLocator: String = appLocator, + timeout: Option[PatienceConfiguration.Timeout] = None): Unit = { runSparkApplicationAndVerifyCompletion( appResource, SPARK_REMOTE_MAIN_CLASS, @@ -158,7 +213,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite driverPodChecker, executorPodChecker, appLocator, - true) + true, + executorPatience = Option((None, timeout))) } protected def runSparkJVMCheckAndVerifyCompletion( @@ -207,7 +263,8 @@ private[spark] class KubernetesSuite extends SparkFunSuite executorPodChecker: Pod => Unit, appLocator: String, isJVM: Boolean, - pyFiles: Option[String] = None): Unit = { + pyFiles: Option[String] = None, + executorPatience: Option[(Option[Interval], Option[Timeout])] = None): Unit = { val appArguments = SparkAppArguments( mainAppResource = appResource, mainClass = mainClass, @@ -247,10 +304,20 @@ private[spark] class KubernetesSuite extends SparkFunSuite } } }) - Eventually.eventually(TIMEOUT, INTERVAL) { execPods.values.nonEmpty should be (true) } + + val (patienceInterval, patienceTimeout) = { + executorPatience match { + case Some(patience) => (patience._1.getOrElse(INTERVAL), patience._2.getOrElse(TIMEOUT)) + case _ => (INTERVAL, TIMEOUT) + } + } + + Eventually.eventually(patienceTimeout, patienceInterval) { + execPods.values.nonEmpty should be (true) + } execWatcher.close() execPods.values.foreach(executorPodChecker(_)) - Eventually.eventually(TIMEOUT, INTERVAL) { + Eventually.eventually(TIMEOUT, patienceInterval) { expectedLogOnCompletion.foreach { e => assert(kubernetesTestComponents.kubernetesClient .pods() @@ -268,6 +335,10 @@ private[spark] class KubernetesSuite extends SparkFunSuite === baseMemory) } + protected def doExecutorServiceAccountCheck(executorPod: Pod, account: String): Unit = { + doBasicExecutorPodCheck(executorPod) + assert(executorPod.getSpec.getServiceAccount == kubernetesTestComponents.serviceAccountName) + } protected def doBasicDriverPyPodCheck(driverPod: Pod): Unit = { assert(driverPod.getMetadata.getName === driverPodName) @@ -288,21 +359,21 @@ private[spark] class KubernetesSuite extends SparkFunSuite protected def doBasicExecutorPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === image) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === baseMemory) } protected def doBasicExecutorPyPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === pyImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } protected def doBasicExecutorRPodCheck(executorPod: Pod): Unit = { assert(executorPod.getSpec.getContainers.get(0).getImage === rImage) - assert(executorPod.getSpec.getContainers.get(0).getName === "executor") + assert(executorPod.getSpec.getContainers.get(0).getName === "spark-kubernetes-executor") assert(executorPod.getSpec.getContainers.get(0).getResources.getRequests.get("memory").getAmount === standardNonJVMMemory) } @@ -348,7 +419,9 @@ private[spark] class KubernetesSuite extends SparkFunSuite private[spark] object KubernetesSuite { val k8sTestTag = Tag("k8s") + val MinikubeTag = Tag("minikube") val SPARK_PI_MAIN_CLASS: String = "org.apache.spark.examples.SparkPi" + val SPARK_DFS_READ_WRITE_TEST = "org.apache.spark.examples.DFSReadWriteTest" val SPARK_REMOTE_MAIN_CLASS: String = "org.apache.spark.examples.SparkRemoteFileTest" val SPARK_DRIVER_MAIN_CLASS: String = "org.apache.spark.examples.DriverSubmissionTest" val TIMEOUT = PatienceConfiguration.Timeout(Span(2, Minutes)) diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala index 5615d6173eebd..4cfda8ab9ba77 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/KubernetesTestComponents.scala @@ -21,19 +21,25 @@ import java.util.UUID import scala.collection.JavaConverters._ import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer import io.fabric8.kubernetes.client.DefaultKubernetesClient import org.scalatest.concurrent.Eventually +import org.apache.spark.SparkConf +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.JARS +import org.apache.spark.internal.config.Tests.IS_TESTING +import org.apache.spark.internal.config.UI.UI_ENABLED private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesClient) { - val namespaceOption = Option(System.getProperty("spark.kubernetes.test.namespace")) + val namespaceOption = Option(System.getProperty(CONFIG_KEY_KUBE_NAMESPACE)) val hasUserSpecifiedNamespace = namespaceOption.isDefined val namespace = namespaceOption.getOrElse(UUID.randomUUID().toString.replaceAll("-", "")) val serviceAccountName = - Option(System.getProperty("spark.kubernetes.test.serviceAccountName")) + Option(System.getProperty(CONFIG_KEY_KUBE_SVC_ACCOUNT)) .getOrElse("default") val kubernetesClient = defaultClient.inNamespace(namespace) val clientConfig = kubernetesClient.getConfiguration @@ -65,8 +71,8 @@ private[spark] class KubernetesTestComponents(defaultClient: DefaultKubernetesCl .set("spark.executor.cores", "1") .set("spark.executors.instances", "1") .set("spark.app.name", "spark-test-app") - .set("spark.ui.enabled", "true") - .set("spark.testing", "false") + .set(IS_TESTING.key, "false") + .set(UI_ENABLED.key, "true") .set("spark.kubernetes.submission.waitAppCompletion", "false") .set("spark.kubernetes.authenticate.driver.serviceAccountName", serviceAccountName) } @@ -83,11 +89,13 @@ private[spark] class SparkAppConf { def get(key: String): String = map.getOrElse(key, "") - def setJars(jars: Seq[String]): Unit = set("spark.jars", jars.mkString(",")) + def setJars(jars: Seq[String]): Unit = set(JARS.key, jars.mkString(",")) override def toString: String = map.toString def toStringArray: Iterable[String] = map.toList.flatMap(t => List("--conf", s"${t._1}=${t._2}")) + + def toSparkConf: SparkConf = new SparkConf().setAll(map) } private[spark] case class SparkAppArguments( @@ -120,7 +128,7 @@ private[spark] object SparkAppLauncher extends Logging { appConf.toStringArray :+ appArguments.mainAppResource if (appArguments.appArgs.nonEmpty) { - commandLine += appArguments.appArgs.mkString(" ") + commandLine ++= appArguments.appArgs } logInfo(s"Launching a spark app with command line: ${commandLine.mkString(" ")}") ProcessUtils.executeProcess(commandLine.toArray, timeoutSecs) 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 new file mode 100644 index 0000000000000..7776bc679de24 --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PVTestsSuite.scala @@ -0,0 +1,167 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import scala.collection.JavaConverters._ + +import io.fabric8.kubernetes.api.model._ +import io.fabric8.kubernetes.api.model.storage.StorageClassBuilder +import org.scalatest.concurrent.{Eventually, PatienceConfiguration} +import org.scalatest.time.{Milliseconds, Span} + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite._ + +private[spark] trait PVTestsSuite { k8sSuite: KubernetesSuite => + import PVTestsSuite._ + + private def setupLocalStorage(): Unit = { + val scBuilder = new StorageClassBuilder() + .withKind("StorageClass") + .withApiVersion("storage.k8s.io/v1") + .withNewMetadata() + .withName(STORAGE_NAME) + .endMetadata() + .withProvisioner("kubernetes.io/no-provisioner") + .withVolumeBindingMode("WaitForFirstConsumer") + + val pvBuilder = new PersistentVolumeBuilder() + .withKind("PersistentVolume") + .withApiVersion("v1") + .withNewMetadata() + .withName("test-local-pv") + .endMetadata() + .withNewSpec() + .withCapacity(Map("storage" -> new QuantityBuilder().withAmount("1Gi").build()).asJava) + .withAccessModes("ReadWriteOnce") + .withPersistentVolumeReclaimPolicy("Retain") + .withStorageClassName("test-local-storage") + .withLocal(new LocalVolumeSourceBuilder().withPath(VM_PATH).build()) + .withNewNodeAffinity() + .withNewRequired() + .withNodeSelectorTerms(new NodeSelectorTermBuilder() + .withMatchExpressions(new NodeSelectorRequirementBuilder() + .withKey("kubernetes.io/hostname") + .withOperator("In") + .withValues("minikube").build()).build()) + .endRequired() + .endNodeAffinity() + .endSpec() + + val pvcBuilder = new PersistentVolumeClaimBuilder() + .withKind("PersistentVolumeClaim") + .withApiVersion("v1") + .withNewMetadata() + .withName(PVC_NAME) + .endMetadata() + .withNewSpec() + .withAccessModes("ReadWriteOnce") + .withStorageClassName("test-local-storage") + .withResources(new ResourceRequirementsBuilder() + .withRequests(Map("storage" -> new QuantityBuilder() + .withAmount("1Gi").build()).asJava).build()) + .endSpec() + + kubernetesTestComponents + .kubernetesClient + .storage() + .storageClasses() + .create(scBuilder.build()) + + kubernetesTestComponents + .kubernetesClient + .persistentVolumes() + .create(pvBuilder.build()) + + kubernetesTestComponents + .kubernetesClient + .persistentVolumeClaims() + .create(pvcBuilder.build()) + } + + private def deleteLocalStorage(): Unit = { + kubernetesTestComponents + .kubernetesClient + .persistentVolumeClaims() + .withName(PVC_NAME) + .delete() + + kubernetesTestComponents + .kubernetesClient + .persistentVolumes() + .withName(PV_NAME) + .delete() + + kubernetesTestComponents + .kubernetesClient + .storage() + .storageClasses() + .withName(STORAGE_NAME) + .delete() + } + + private def checkPVs(pod: Pod, file: String) = { + Eventually.eventually(TIMEOUT, INTERVAL) { + implicit val podName: String = pod.getMetadata.getName + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val contents = Utils.executeCommand("cat", s"$CONTAINER_MOUNT_PATH/$file") + assert(contents.toString.trim.equals(FILE_CONTENTS)) + } + } + + test("PVs with local storage", k8sTestTag, MinikubeTag) { + sparkAppConf + .set(s"spark.kubernetes.driver.volumes.persistentVolumeClaim.data.mount.path", + CONTAINER_MOUNT_PATH) + .set(s"spark.kubernetes.driver.volumes.persistentVolumeClaim.data.options.claimName", + PVC_NAME) + .set(s"spark.kubernetes.executor.volumes.persistentVolumeClaim.data.mount.path", + CONTAINER_MOUNT_PATH) + .set(s"spark.kubernetes.executor.volumes.persistentVolumeClaim.data.options.claimName", + PVC_NAME) + val file = Utils.createTempFile(FILE_CONTENTS, HOST_PATH) + try { + setupLocalStorage() + runDFSReadWriteAndVerifyCompletion( + FILE_CONTENTS.split(" ").length, + driverPodChecker = (driverPod: Pod) => { + doBasicDriverPodCheck(driverPod) + checkPVs(driverPod, file) + }, + executorPodChecker = (executorPod: Pod) => { + doBasicExecutorPodCheck(executorPod) + checkPVs(executorPod, file) + }, + appArgs = Array(s"$CONTAINER_MOUNT_PATH/$file", s"$CONTAINER_MOUNT_PATH"), + interval = Some(PV_TESTS_INTERVAL) + ) + } finally { + // make sure this always runs + deleteLocalStorage() + } + } +} + +private[spark] object PVTestsSuite { + val STORAGE_NAME = "test-local-storage" + val PV_NAME = "test-local-pv" + val PVC_NAME = "test-local-pvc" + val CONTAINER_MOUNT_PATH = "/opt/spark/pv-tests" + val HOST_PATH = sys.env.getOrElse("PVC_TESTS_HOST_PATH", "/tmp") + val VM_PATH = sys.env.getOrElse("PVC_TESTS_VM_PATH", "/tmp") + val FILE_CONTENTS = "test PVs" + val PV_TESTS_INTERVAL = PatienceConfiguration.Interval(Span(10, Milliseconds)) +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala new file mode 100644 index 0000000000000..e5a847e7210cb --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PodTemplateSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest + +import java.io.File + +import io.fabric8.kubernetes.api.model.Pod + +import org.apache.spark.deploy.k8s.integrationtest.KubernetesSuite.k8sTestTag + +private[spark] trait PodTemplateSuite { k8sSuite: KubernetesSuite => + + import PodTemplateSuite._ + + test("Start pod creation from template", k8sTestTag) { + sparkAppConf + .set("spark.kubernetes.driver.podTemplateFile", DRIVER_TEMPLATE_FILE.getAbsolutePath) + .set("spark.kubernetes.executor.podTemplateFile", EXECUTOR_TEMPLATE_FILE.getAbsolutePath) + runSparkPiAndVerifyCompletion( + driverPodChecker = (driverPod: Pod) => { + assert(driverPod.getMetadata.getName === driverPodName) + assert(driverPod.getSpec.getContainers.get(0).getImage === image) + assert(driverPod.getSpec.getContainers.get(0).getName === "test-driver-container") + assert(driverPod.getMetadata.getLabels.containsKey(LABEL_KEY)) + assert(driverPod.getMetadata.getLabels.get(LABEL_KEY) === "driver-template-label-value") + }, + executorPodChecker = (executorPod: Pod) => { + assert(executorPod.getSpec.getContainers.get(0).getImage === image) + assert(executorPod.getSpec.getContainers.get(0).getName === "test-executor-container") + assert(executorPod.getMetadata.getLabels.containsKey(LABEL_KEY)) + assert(executorPod.getMetadata.getLabels.get(LABEL_KEY) === "executor-template-label-value") + } + ) + } +} + +private[spark] object PodTemplateSuite { + val LABEL_KEY = "template-label-key" + val DRIVER_TEMPLATE_FILE = new File(getClass.getResource("/driver-template.yml").getFile) + val EXECUTOR_TEMPLATE_FILE = new File(getClass.getResource("/executor-template.yml").getFile) +} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala index d8f3a6cec05c3..9ead70f670891 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/ProcessUtils.scala @@ -28,7 +28,10 @@ object ProcessUtils extends Logging { * executeProcess is used to run a command and return the output if it * completes within timeout seconds. */ - def executeProcess(fullCommand: Array[String], timeout: Long): Seq[String] = { + def executeProcess( + fullCommand: Array[String], + timeout: Long, + dumpErrors: Boolean = false): Seq[String] = { val pb = new ProcessBuilder().command(fullCommand: _*) pb.redirectErrorStream(true) val proc = pb.start() @@ -40,7 +43,9 @@ object ProcessUtils extends Logging { }) assert(proc.waitFor(timeout, TimeUnit.SECONDS), s"Timed out while executing ${fullCommand.mkString(" ")}") - assert(proc.exitValue == 0, s"Failed to execute ${fullCommand.mkString(" ")}") + assert(proc.exitValue == 0, + s"Failed to execute ${fullCommand.mkString(" ")}" + + s"${if (dumpErrors) "\n" + outputLines.mkString("\n")}") outputLines } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala index 06b73107ec236..904279923334f 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/PythonTestsSuite.scala @@ -16,18 +16,14 @@ */ package org.apache.spark.deploy.k8s.integrationtest -import org.apache.spark.deploy.k8s.integrationtest.TestConfig.{getTestImageRepo, getTestImageTag} - private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => import PythonTestsSuite._ import KubernetesSuite.k8sTestTag - private val pySparkDockerImage = - s"${getTestImageRepo}/spark-py:${getTestImageTag}" test("Run PySpark on simple pi.py example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_PI, mainClass = "", @@ -41,7 +37,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python2 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "2") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -59,7 +55,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with Python3 to test a pyfiles example", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "3") runSparkApplicationAndVerifyCompletion( appResource = PYSPARK_FILES, @@ -77,7 +73,7 @@ private[spark] trait PythonTestsSuite { k8sSuite: KubernetesSuite => test("Run PySpark with memory customization", k8sTestTag) { sparkAppConf - .set("spark.kubernetes.container.image", pySparkDockerImage) + .set("spark.kubernetes.container.image", pyImage) .set("spark.kubernetes.pyspark.pythonVersion", "3") .set("spark.kubernetes.memoryOverheadFactor", s"$memOverheadConstant") .set("spark.executor.pyspark.memory", s"${additionalMemory}m") diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala index 885a23cfb4864..e81562a923228 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/RTestsSuite.scala @@ -16,16 +16,13 @@ */ package org.apache.spark.deploy.k8s.integrationtest -import org.apache.spark.deploy.k8s.integrationtest.TestConfig.{getTestImageRepo, getTestImageTag} - private[spark] trait RTestsSuite { k8sSuite: KubernetesSuite => import RTestsSuite._ import KubernetesSuite.k8sTestTag test("Run SparkR on simple dataframe.R example", k8sTestTag) { - sparkAppConf - .set("spark.kubernetes.container.image", s"${getTestImageRepo}/spark-r:${getTestImageTag}") + sparkAppConf.set("spark.kubernetes.container.image", rImage) runSparkApplicationAndVerifyCompletion( appResource = SPARK_R_DATAFRAME_TEST, mainClass = "", diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala index b18a6aebda497..cd61ea1040f35 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/SecretsTestsSuite.scala @@ -83,33 +83,18 @@ private[spark] trait SecretsTestsSuite { k8sSuite: KubernetesSuite => private def checkSecrets(pod: Pod): Unit = { Eventually.eventually(TIMEOUT, INTERVAL) { implicit val podName: String = pod.getMetadata.getName - val env = executeCommand("env") + implicit val components: KubernetesTestComponents = kubernetesTestComponents + val env = Utils.executeCommand("env") assert(env.toString.contains(ENV_SECRET_VALUE_1)) assert(env.toString.contains(ENV_SECRET_VALUE_2)) - val fileUsernameContents = executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") - val filePasswordContents = executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") + val fileUsernameContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_1") + val filePasswordContents = Utils + .executeCommand("cat", s"$SECRET_MOUNT_PATH/$ENV_SECRET_KEY_2") assert(fileUsernameContents.toString.trim.equals(ENV_SECRET_VALUE_1)) assert(filePasswordContents.toString.trim.equals(ENV_SECRET_VALUE_2)) } } - - private def executeCommand(cmd: String*)(implicit podName: String): String = { - val out = new ByteArrayOutputStream() - val watch = kubernetesTestComponents - .kubernetesClient - .pods() - .withName(podName) - .readingInput(System.in) - .writingOutput(out) - .writingError(System.err) - .withTTY() - .exec(cmd.toArray: _*) - // wait to get some result back - Thread.sleep(1000) - watch.close() - out.flush() - out.toString() - } } private[spark] object SecretsTestsSuite { diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala deleted file mode 100644 index 5a49e0779160c..0000000000000 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConfig.scala +++ /dev/null @@ -1,38 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.k8s.integrationtest - -import java.io.File - -import com.google.common.base.Charsets -import com.google.common.io.Files - -object TestConfig { - def getTestImageTag: String = { - val imageTagFileProp = System.getProperty("spark.kubernetes.test.imageTagFile") - require(imageTagFileProp != null, "Image tag file must be provided in system properties.") - val imageTagFile = new File(imageTagFileProp) - require(imageTagFile.isFile, s"No file found for image tag at ${imageTagFile.getAbsolutePath}.") - Files.toString(imageTagFile, Charsets.UTF_8).trim - } - - def getTestImageRepo: String = { - val imageRepo = System.getProperty("spark.kubernetes.test.imageRepo") - require(imageRepo != null, "Image repo must be provided in system properties.") - imageRepo - } -} diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala index 8595d0eab1126..2b1fd08164616 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/TestConstants.scala @@ -17,6 +17,20 @@ package org.apache.spark.deploy.k8s.integrationtest object TestConstants { - val MINIKUBE_TEST_BACKEND = "minikube" - val GCE_TEST_BACKEND = "gce" + val BACKEND_MINIKUBE = "minikube" + val BACKEND_DOCKER_FOR_DESKTOP = "docker-for-desktop" + val BACKEND_CLOUD = "cloud" + + val CONFIG_KEY_DEPLOY_MODE = "spark.kubernetes.test.deployMode" + val CONFIG_KEY_KUBE_CONFIG_CONTEXT = "spark.kubernetes.test.kubeConfigContext" + val CONFIG_KEY_KUBE_MASTER_URL = "spark.kubernetes.test.master" + val CONFIG_KEY_KUBE_NAMESPACE = "spark.kubernetes.test.namespace" + val CONFIG_KEY_KUBE_SVC_ACCOUNT = "spark.kubernetes.test.serviceAccountName" + val CONFIG_KEY_IMAGE_JVM = "spark.kubernetes.test.jvmImage" + val CONFIG_KEY_IMAGE_PYTHON = "spark.kubernetes.test.pythonImage" + val CONFIG_KEY_IMAGE_R = "spark.kubernetes.test.rImage" + val CONFIG_KEY_IMAGE_TAG = "spark.kubernetes.test.imageTag" + val CONFIG_KEY_IMAGE_TAG_FILE = "spark.kubernetes.test.imageTagFile" + val CONFIG_KEY_IMAGE_REPO = "spark.kubernetes.test.imageRepo" + val CONFIG_KEY_UNPACK_DIR = "spark.kubernetes.test.unpackSparkDir" } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala index 663f8b6523ac8..a687a1bca1800 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/Utils.scala @@ -16,15 +16,87 @@ */ package org.apache.spark.deploy.k8s.integrationtest -import java.io.Closeable -import java.net.URI +import java.io.{Closeable, File, PrintWriter} +import java.nio.file.{Files, Path} +import scala.collection.JavaConverters._ + +import org.apache.commons.io.output.ByteArrayOutputStream + +import org.apache.spark.{SPARK_VERSION, SparkException} import org.apache.spark.internal.Logging object Utils extends Logging { + def getExamplesJarName(): String = { + val scalaVersion = scala.util.Properties.versionNumberString + .split("\\.") + .take(2) + .mkString(".") + s"spark-examples_$scalaVersion-${SPARK_VERSION}.jar" + } + def tryWithResource[R <: Closeable, T](createResource: => R)(f: R => T): T = { val resource = createResource try f.apply(resource) finally resource.close() } + + def executeCommand(cmd: String*)( + implicit podName: String, + kubernetesTestComponents: KubernetesTestComponents): String = { + val out = new ByteArrayOutputStream() + val watch = kubernetesTestComponents + .kubernetesClient + .pods() + .withName(podName) + .readingInput(System.in) + .writingOutput(out) + .writingError(System.err) + .withTTY() + .exec(cmd.toArray: _*) + // wait to get some result back + Thread.sleep(1000) + watch.close() + out.flush() + out.toString() + } + + def createTempFile(contents: String, hostPath: String): String = { + val filename = try { + val f = File.createTempFile("tmp", ".txt", new File(hostPath)) + f.deleteOnExit() + new PrintWriter(f) { + try { + write(contents) + } finally { + close() + } + } + f.getName + } catch { + case e: Exception => e.printStackTrace(); throw e; + } + filename + } + + def getExamplesJarAbsolutePath(sparkHomeDir: Path): String = { + val jarName = getExamplesJarName() + val jarPathsFound = Files + .walk(sparkHomeDir) + .filter(Files.isRegularFile(_)) + .filter((f: Path) => {f.toFile.getName == jarName}) + // we should not have more than one here under current test build dir + // we only need one though + val jarPath = jarPathsFound + .iterator() + .asScala + .map(_.toAbsolutePath.toString) + .toArray + .headOption + jarPath match { + case Some(jar) => jar + case _ => throw new SparkException(s"No valid $jarName file was found " + + s"under spark home test dir ${sparkHomeDir.toAbsolutePath}!") + } + } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala index 284712c6d250e..56ddae0c9c57c 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/IntegrationTestBackend.scala @@ -19,6 +19,9 @@ package org.apache.spark.deploy.k8s.integrationtest.backend import io.fabric8.kubernetes.client.DefaultKubernetesClient +import org.apache.spark.deploy.k8s.integrationtest.TestConstants._ +import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend +import org.apache.spark.deploy.k8s.integrationtest.backend.docker.DockerForDesktopBackend import org.apache.spark.deploy.k8s.integrationtest.backend.minikube.MinikubeTestBackend private[spark] trait IntegrationTestBackend { @@ -28,16 +31,16 @@ private[spark] trait IntegrationTestBackend { } private[spark] object IntegrationTestBackendFactory { - val deployModeConfigKey = "spark.kubernetes.test.deployMode" - def getTestBackend: IntegrationTestBackend = { - val deployMode = Option(System.getProperty(deployModeConfigKey)) - .getOrElse("minikube") - if (deployMode == "minikube") { - MinikubeTestBackend - } else { - throw new IllegalArgumentException( - "Invalid " + deployModeConfigKey + ": " + deployMode) + val deployMode = Option(System.getProperty(CONFIG_KEY_DEPLOY_MODE)) + .getOrElse(BACKEND_MINIKUBE) + deployMode match { + case BACKEND_MINIKUBE => MinikubeTestBackend + case BACKEND_CLOUD => + new KubeConfigBackend(System.getProperty(CONFIG_KEY_KUBE_CONFIG_CONTEXT)) + case BACKEND_DOCKER_FOR_DESKTOP => DockerForDesktopBackend + case _ => throw new IllegalArgumentException("Invalid " + + CONFIG_KEY_DEPLOY_MODE + ": " + deployMode) } } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala new file mode 100644 index 0000000000000..be1834c0b5dea --- /dev/null +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/cloud/KubeConfigBackend.scala @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.deploy.k8s.integrationtest.backend.cloud + +import java.nio.file.Paths + +import io.fabric8.kubernetes.client.{Config, DefaultKubernetesClient} +import io.fabric8.kubernetes.client.utils.Utils +import org.apache.commons.lang3.StringUtils + +import org.apache.spark.deploy.k8s.integrationtest.TestConstants +import org.apache.spark.deploy.k8s.integrationtest.backend.IntegrationTestBackend +import org.apache.spark.internal.Logging +import org.apache.spark.util.Utils.checkAndGetK8sMasterUrl + +private[spark] class KubeConfigBackend(var context: String) + extends IntegrationTestBackend with Logging { + logInfo(s"K8S Integration tests will run against " + + s"${if (context != null) s"context ${context}" else "default context"}" + + s" from users K8S config file") + + private var defaultClient: DefaultKubernetesClient = _ + + override def initialize(): Unit = { + // Auto-configure K8S client from K8S config file + if (Utils.getSystemPropertyOrEnvVar(Config.KUBERNETES_KUBECONFIG_FILE, null: String) == null) { + // Fabric 8 client will automatically assume a default location in this case + logWarning("No explicit KUBECONFIG specified, will assume $HOME/.kube/config") + } + val config = Config.autoConfigure(context) + + // If an explicit master URL was specified then override that detected from the + // K8S config if it is different + var masterUrl = Option(System.getProperty(TestConstants.CONFIG_KEY_KUBE_MASTER_URL)) + .getOrElse(null) + if (StringUtils.isNotBlank(masterUrl)) { + // Clean up master URL which would have been specified in Spark format into a normal + // K8S master URL + masterUrl = checkAndGetK8sMasterUrl(masterUrl).replaceFirst("k8s://", "") + if (!StringUtils.equals(config.getMasterUrl, masterUrl)) { + logInfo(s"Overriding K8S master URL ${config.getMasterUrl} from K8S config file " + + s"with user specified master URL ${masterUrl}") + config.setMasterUrl(masterUrl) + } + } + + defaultClient = new DefaultKubernetesClient(config) + } + + override def cleanUp(): Unit = { + super.cleanUp() + } + + override def getKubernetesClient: DefaultKubernetesClient = { + defaultClient + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeKeyValueSorter.java b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala similarity index 69% rename from sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeKeyValueSorter.java rename to resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala index 59c774da74acf..81a11ae9dcdc6 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeKeyValueSorter.java +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/docker/DockerForDesktopBackend.scala @@ -14,17 +14,12 @@ * See the License for the specific language governing permissions and * limitations under the License. */ +package org.apache.spark.deploy.k8s.integrationtest.backend.docker -package org.apache.spark.sql.execution; +import org.apache.spark.deploy.k8s.integrationtest.TestConstants +import org.apache.spark.deploy.k8s.integrationtest.backend.cloud.KubeConfigBackend -import java.io.IOException; +private[spark] object DockerForDesktopBackend + extends KubeConfigBackend(TestConstants.BACKEND_DOCKER_FOR_DESKTOP) { -import org.apache.spark.sql.catalyst.expressions.UnsafeRow; -import org.apache.spark.unsafe.KVIterator; - -public abstract class UnsafeKeyValueSorter { - - public abstract void insert(UnsafeRow key, UnsafeRow value); - - public abstract KVIterator sort() throws IOException; } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala index 6494cbc18f33e..ce2ce1c61c194 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/Minikube.scala @@ -16,7 +16,6 @@ */ package org.apache.spark.deploy.k8s.integrationtest.backend.minikube -import java.io.File import java.nio.file.Paths import io.fabric8.kubernetes.client.{ConfigBuilder, DefaultKubernetesClient} @@ -26,8 +25,18 @@ import org.apache.spark.internal.Logging // TODO support windows private[spark] object Minikube extends Logging { - private val MINIKUBE_STARTUP_TIMEOUT_SECONDS = 60 + private val HOST_PREFIX = "host:" + private val KUBELET_PREFIX = "kubelet:" + private val APISERVER_PREFIX = "apiserver:" + private val KUBECTL_PREFIX = "kubectl:" + private val MINIKUBE_VM_PREFIX = "minikubeVM: " + private val MINIKUBE_PREFIX = "minikube: " + private val MINIKUBE_PATH = ".minikube" + + def logVersion(): Unit = { + logInfo(executeMinikube("version").mkString("\n")) + } def getMinikubeIp: String = { val outputs = executeMinikube("ip") @@ -38,12 +47,21 @@ private[spark] object Minikube extends Logging { def getMinikubeStatus: MinikubeStatus.Value = { val statusString = executeMinikube("status") - .filter(line => line.contains("minikubeVM: ") || line.contains("minikube:")) - .head - .replaceFirst("minikubeVM: ", "") - .replaceFirst("minikube: ", "") - MinikubeStatus.unapply(statusString) + logInfo(s"Minikube status command output:\n$statusString") + // up to minikube version v0.30.0 use this to check for minikube status + val oldMinikube = statusString + .filter(line => line.contains(MINIKUBE_VM_PREFIX) || line.contains(MINIKUBE_PREFIX)) + + if (oldMinikube.isEmpty) { + getIfNewMinikubeStatus(statusString) + } else { + val finalStatusString = oldMinikube + .head + .replaceFirst(MINIKUBE_VM_PREFIX, "") + .replaceFirst(MINIKUBE_PREFIX, "") + MinikubeStatus.unapply(finalStatusString) .getOrElse(throw new IllegalStateException(s"Unknown status $statusString")) + } } def getKubernetesClient: DefaultKubernetesClient = { @@ -52,16 +70,54 @@ private[spark] object Minikube extends Logging { val kubernetesConf = new ConfigBuilder() .withApiVersion("v1") .withMasterUrl(kubernetesMaster) - .withCaCertFile(Paths.get(userHome, ".minikube", "ca.crt").toFile.getAbsolutePath) - .withClientCertFile(Paths.get(userHome, ".minikube", "apiserver.crt").toFile.getAbsolutePath) - .withClientKeyFile(Paths.get(userHome, ".minikube", "apiserver.key").toFile.getAbsolutePath) + .withCaCertFile( + Paths.get(userHome, MINIKUBE_PATH, "ca.crt").toFile.getAbsolutePath) + .withClientCertFile( + Paths.get(userHome, MINIKUBE_PATH, "apiserver.crt").toFile.getAbsolutePath) + .withClientKeyFile( + Paths.get(userHome, MINIKUBE_PATH, "apiserver.key").toFile.getAbsolutePath) .build() new DefaultKubernetesClient(kubernetesConf) } + // Covers minikube status output after Minikube V0.30. + private def getIfNewMinikubeStatus(statusString: Seq[String]): MinikubeStatus.Value = { + val hostString = statusString.find(_.contains(s"$HOST_PREFIX ")) + val kubeletString = statusString.find(_.contains(s"$KUBELET_PREFIX ")) + val apiserverString = statusString.find(_.contains(s"$APISERVER_PREFIX ")) + val kubectlString = statusString.find(_.contains(s"$KUBECTL_PREFIX ")) + + if (hostString.isEmpty || kubeletString.isEmpty + || apiserverString.isEmpty || kubectlString.isEmpty) { + MinikubeStatus.NONE + } else { + val status1 = hostString.get.replaceFirst(s"$HOST_PREFIX ", "") + val status2 = kubeletString.get.replaceFirst(s"$KUBELET_PREFIX ", "") + val status3 = apiserverString.get.replaceFirst(s"$APISERVER_PREFIX ", "") + val status4 = kubectlString.get.replaceFirst(s"$KUBECTL_PREFIX ", "") + if (!status4.contains("Correctly Configured:")) { + MinikubeStatus.NONE + } else { + val stats = List(status1, status2, status3) + .map(MinikubeStatus.unapply) + .map(_.getOrElse(throw new IllegalStateException(s"Unknown status $statusString"))) + if (stats.exists(_ != MinikubeStatus.RUNNING)) { + MinikubeStatus.NONE + } else { + MinikubeStatus.RUNNING + } + } + } + } + private def executeMinikube(action: String, args: String*): Seq[String] = { ProcessUtils.executeProcess( - Array("bash", "-c", s"minikube $action") ++ args, MINIKUBE_STARTUP_TIMEOUT_SECONDS) + Array("bash", "-c", s"minikube $action ${args.mkString(" ")}"), + MINIKUBE_STARTUP_TIMEOUT_SECONDS) + } + + def minikubeServiceAction(args: String*): String = { + executeMinikube("service", args: _*).head } } diff --git a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala index cb9324179d70e..f92977ddacdf5 100644 --- a/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala +++ b/resource-managers/kubernetes/integration-tests/src/test/scala/org/apache/spark/deploy/k8s/integrationtest/backend/minikube/MinikubeTestBackend.scala @@ -25,6 +25,7 @@ private[spark] object MinikubeTestBackend extends IntegrationTestBackend { private var defaultClient: DefaultKubernetesClient = _ override def initialize(): Unit = { + Minikube.logVersion() val minikubeStatus = Minikube.getMinikubeStatus require(minikubeStatus == MinikubeStatus.RUNNING, s"Minikube must be running to use the Minikube backend for integration tests." + diff --git a/resource-managers/mesos/pom.xml b/resource-managers/mesos/pom.xml index 9585bdfafdcf4..107ba365aa775 100644 --- a/resource-managers/mesos/pom.xml +++ b/resource-managers/mesos/pom.xml @@ -19,12 +19,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-mesos_2.11 + spark-mesos_2.12 jar Spark Project Mesos @@ -77,6 +77,7 @@ ${hive.group} hive-exec + ${hive.classifier} provided diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala index 32ac4f37c5f99..bc1247ad78936 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/MesosClusterDispatcher.scala @@ -25,6 +25,7 @@ import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.ui.MesosClusterUI import org.apache.spark.deploy.rest.mesos.MesosRestServer import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.scheduler.cluster.mesos._ import org.apache.spark.util.{CommandLineUtils, ShutdownHookManager, SparkUncaughtExceptionHandler, Utils} diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala index d134847dc74d2..79a113792de91 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/config.scala @@ -56,6 +56,31 @@ package object config { .createOptional } + private[spark] val CREDENTIAL_PRINCIPAL = + ConfigBuilder("spark.mesos.principal") + .doc("Name of the Kerberos principal to authenticate Spark to Mesos.") + .stringConf + .createOptional + + private[spark] val CREDENTIAL_PRINCIPAL_FILE = + ConfigBuilder("spark.mesos.principal.file") + .doc("The path of file which contains the name of the Kerberos principal " + + "to authenticate Spark to Mesos.") + .stringConf + .createOptional + + private[spark] val CREDENTIAL_SECRET = + ConfigBuilder("spark.mesos.secret") + .doc("The secret value to authenticate Spark to Mesos.") + .stringConf + .createOptional + + private[spark] val CREDENTIAL_SECRET_FILE = + ConfigBuilder("spark.mesos.secret.file") + .doc("The path of file which contains the secret value to authenticate Spark to Mesos.") + .stringConf + .createOptional + /* Common app configuration. */ private[spark] val SHUFFLE_CLEANER_INTERVAL_S = @@ -63,11 +88,6 @@ package object config { .timeConf(TimeUnit.SECONDS) .createWithDefaultString("30s") - private[spark] val RECOVERY_MODE = - ConfigBuilder("spark.deploy.recoveryMode") - .stringConf - .createWithDefault("NONE") - private[spark] val DISPATCHER_WEBUI_URL = ConfigBuilder("spark.mesos.dispatcher.webui.url") .doc("Set the Spark Mesos dispatcher webui_url for interacting with the " + @@ -75,13 +95,6 @@ package object config { .stringConf .createOptional - private[spark] val ZOOKEEPER_URL = - ConfigBuilder("spark.deploy.zookeeper.url") - .doc("When `spark.deploy.recoveryMode` is set to ZOOKEEPER, this " + - "configuration is used to set the zookeeper URL to connect to.") - .stringConf - .createOptional - private[spark] val HISTORY_SERVER_URL = ConfigBuilder("spark.mesos.dispatcher.historyServer.url") .doc("Set the URL of the history server. The dispatcher will then " + @@ -97,6 +110,13 @@ package object config { .stringConf .createOptional + private[spark] val DRIVER_WEBUI_URL = + ConfigBuilder("spark.mesos.driver.webui.url") + .doc("Set the Spark Mesos driver webui_url for interacting with the framework. " + + "If unset it will point to Spark's internal web UI.") + .stringConf + .createOptional + private[spark] val driverSecretConfig = new MesosSecretConfig("driver") private[spark] val executorSecretConfig = new MesosSecretConfig("executor") @@ -129,4 +149,212 @@ package object config { "when launching drivers. Default is to accept all offers with sufficient resources.") .stringConf .createWithDefault("") + + private[spark] val DRIVER_FRAMEWORK_ID = + ConfigBuilder("spark.mesos.driver.frameworkId") + .stringConf + .createOptional + + private[spark] val EXECUTOR_URI = + ConfigBuilder("spark.executor.uri").stringConf.createOptional + + private[spark] val PROXY_BASE_URL = + ConfigBuilder("spark.mesos.proxy.baseURL").stringConf.createOptional + + private[spark] val COARSE_MODE = + ConfigBuilder("spark.mesos.coarse") + .doc("If set to true, runs over Mesos clusters in \"coarse-grained\" sharing mode, where " + + "Spark acquires one long-lived Mesos task on each machine. If set to false, runs over " + + "Mesos cluster in \"fine-grained\" sharing mode, where one Mesos task is created per " + + "Spark task.") + .booleanConf.createWithDefault(true) + + private[spark] val COARSE_SHUTDOWN_TIMEOUT = + ConfigBuilder("spark.mesos.coarse.shutdownTimeout") + .timeConf(TimeUnit.MILLISECONDS) + .checkValue(_ >= 0, s"spark.mesos.coarse.shutdownTimeout must be >= 0") + .createWithDefaultString("10s") + + private[spark] val MAX_DRIVERS = + ConfigBuilder("spark.mesos.maxDrivers").intConf.createWithDefault(200) + + private[spark] val RETAINED_DRIVERS = + ConfigBuilder("spark.mesos.retainedDrivers").intConf.createWithDefault(200) + + private[spark] val CLUSTER_RETRY_WAIT_MAX_SECONDS = + ConfigBuilder("spark.mesos.cluster.retry.wait.max") + .intConf + .createWithDefault(60) // 1 minute + + private[spark] val ENABLE_FETCHER_CACHE = + ConfigBuilder("spark.mesos.fetcherCache.enable") + .doc("If set to true, all URIs (example: `spark.executor.uri`, `spark.mesos.uris`) will be " + + "cached by the Mesos Fetcher Cache.") + .booleanConf + .createWithDefault(false) + + private[spark] val APP_JAR_LOCAL_RESOLUTION_MODE = + ConfigBuilder("spark.mesos.appJar.local.resolution.mode") + .doc("Provides support for the `local:///` scheme to reference the app jar resource in " + + "cluster mode. If user uses a local resource (`local:///path/to/jar`) and the config " + + "option is not used it defaults to `host` eg. the mesos fetcher tries to get the " + + "resource from the host's file system. If the value is unknown it prints a warning msg " + + "in the dispatcher logs and defaults to `host`. If the value is `container` then spark " + + "submit in the container will use the jar in the container's path: `/path/to/jar`.") + .stringConf + .checkValues(Set("host", "container")) + .createWithDefault("host") + + private[spark] val REJECT_OFFER_DURATION = + ConfigBuilder("spark.mesos.rejectOfferDuration") + .doc("Time to consider unused resources refused, serves as a fallback of " + + "`spark.mesos.rejectOfferDurationForUnmetConstraints`, " + + "`spark.mesos.rejectOfferDurationForReachedMaxCores`.") + .timeConf(TimeUnit.SECONDS) + .createWithDefaultString("120s") + + private[spark] val REJECT_OFFER_DURATION_FOR_UNMET_CONSTRAINTS = + ConfigBuilder("spark.mesos.rejectOfferDurationForUnmetConstraints") + .doc("Time to consider unused resources refused with unmet constraints.") + .timeConf(TimeUnit.SECONDS) + .createOptional + + private[spark] val REJECT_OFFER_DURATION_FOR_REACHED_MAX_CORES = + ConfigBuilder("spark.mesos.rejectOfferDurationForReachedMaxCores") + .doc("Time to consider unused resources refused when maximum number of cores " + + "`spark.cores.max` is reached.") + .timeConf(TimeUnit.SECONDS) + .createOptional + + private[spark] val URIS_TO_DOWNLOAD = + ConfigBuilder("spark.mesos.uris") + .doc("A comma-separated list of URIs to be downloaded to the sandbox when driver or " + + "executor is launched by Mesos. This applies to both coarse-grained and fine-grained " + + "mode.") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[spark] val EXECUTOR_HOME = + ConfigBuilder("spark.mesos.executor.home") + .doc("Set the directory in which Spark is installed on the executors in Mesos. " + + "By default, the executors will simply use the driver's Spark home directory, which may " + + "not be visible to them. Note that this is only relevant if a Spark binary package is " + + "not specified through `spark.executor.uri`.") + .stringConf + .createOptional + + private[spark] val EXECUTOR_CORES = + ConfigBuilder("spark.mesos.mesosExecutor.cores") + .doc("(Fine-grained mode only) Number of cores to give each Mesos executor. This does not " + + "include the cores used to run the Spark tasks. In other words, even if no Spark task " + + "is being run, each Mesos executor will occupy the number of cores configured here. " + + "The value can be a floating point number.") + .doubleConf + .createWithDefault(1.0) + + private[spark] val EXTRA_CORES_PER_EXECUTOR = + ConfigBuilder("spark.mesos.extra.cores") + .doc("Set the extra number of cores for an executor to advertise. This does not result in " + + "more cores allocated. It instead means that an executor will \"pretend\" it has more " + + "cores, so that the driver will send it more tasks. Use this to increase parallelism. " + + "This setting is only used for Mesos coarse-grained mode.") + .intConf + .createWithDefault(0) + + private[spark] val EXECUTOR_MEMORY_OVERHEAD = + ConfigBuilder("spark.mesos.executor.memoryOverhead") + .doc("The amount of additional memory, specified in MiB, to be allocated per executor. " + + "By default, the overhead will be larger of either 384 or 10% of " + + "`spark.executor.memory`. If set, the final overhead will be this value.") + .intConf + .createOptional + + private[spark] val EXECUTOR_DOCKER_IMAGE = + ConfigBuilder("spark.mesos.executor.docker.image") + .doc("Set the name of the docker image that the Spark executors will run in. The selected " + + "image must have Spark installed, as well as a compatible version of the Mesos library. " + + "The installed path of Spark in the image can be specified with " + + "`spark.mesos.executor.home`; the installed path of the Mesos library can be specified " + + "with `spark.executorEnv.MESOS_NATIVE_JAVA_LIBRARY`.") + .stringConf + .createOptional + + private[spark] val EXECUTOR_DOCKER_FORCE_PULL_IMAGE = + ConfigBuilder("spark.mesos.executor.docker.forcePullImage") + .doc("Force Mesos agents to pull the image specified in " + + "`spark.mesos.executor.docker.image`. By default Mesos agents will not pull images they " + + "already have cached.") + .booleanConf + .createOptional + + private[spark] val EXECUTOR_DOCKER_PORT_MAPS = + ConfigBuilder("spark.mesos.executor.docker.portmaps") + .stringConf + .toSequence + .createOptional + + private[spark] val EXECUTOR_DOCKER_PARAMETERS = + ConfigBuilder("spark.mesos.executor.docker.parameters") + .doc("Set the list of custom parameters which will be passed into the `docker run` " + + "command when launching the Spark executor on Mesos using the docker containerizer. " + + "The format of this property is a list of key/value pairs which pair looks key1=value1.") + .stringConf + .toSequence + .createOptional + + private[spark] val EXECUTOR_DOCKER_VOLUMES = + ConfigBuilder("spark.mesos.executor.docker.volumes") + .doc("Set the list of volumes which will be mounted into the Docker image, which was set " + + "using `spark.mesos.executor.docker.image`. The format of this property is a list of " + + "mappings following the form passed to `docker run -v`. That is they take the form: " + + "`[host_path:]container_path[:ro|:rw]`") + .stringConf + .toSequence + .createOptional + + private[spark] val MAX_GPUS = + ConfigBuilder("spark.mesos.gpus.max") + .doc("Set the maximum number GPU resources to acquire for this job. Note that executors " + + "will still launch when no GPU resources are found since this configuration is just an " + + "upper limit and not a guaranteed amount.") + .intConf + .createWithDefault(0) + + private[spark] val TASK_LABELS = + ConfigBuilder("spark.mesos.task.labels") + .doc("Set the Mesos labels to add to each task. Labels are free-form key-value pairs. " + + "Key-value pairs should be separated by a colon, and commas used to list more than one. " + + "If your label includes a colon or comma, you can escape it with a backslash. " + + "Ex. key:value,key2:a\\:b.") + .stringConf + .createWithDefault("") + + private[spark] val CONSTRAINTS = + ConfigBuilder("spark.mesos.constraints") + .doc("Attribute-based constraints on mesos resource offers. By default, all resource " + + "offers will be accepted. This setting applies only to executors. Refer to Mesos " + + "Attributes & Resources doc for more information on attributes.") + .stringConf + .createWithDefault("") + + private[spark] val CONTAINERIZER = + ConfigBuilder("spark.mesos.containerizer") + .doc("This only affects docker containers, and must be one of \"docker\" or \"mesos\". " + + "Mesos supports two types of containerizers for docker: the \"docker\" containerizer, " + + "and the preferred \"mesos\" containerizer. " + + "Read more here: http://mesos.apache.org/documentation/latest/container-image/") + .stringConf + .checkValues(Set("docker", "mesos")) + .createWithDefault("docker") + + private[spark] val ROLE = + ConfigBuilder("spark.mesos.role") + .doc("Set the role of this Spark framework for Mesos. Roles are used in Mesos for " + + "reservations and resource weight sharing.") + .stringConf + .createOptional + + private[spark] val DRIVER_ENV_PREFIX = "spark.mesos.driverEnv." + private[spark] val DISPATCHER_DRIVER_DEFAULT_PREFIX = "spark.mesos.dispatcher.driverDefault." } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala index 91f64141e5318..6e4571eba0361 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/DriverPage.scala @@ -29,8 +29,7 @@ import org.apache.spark.ui.{UIUtils, WebUIPage} private[ui] class DriverPage(parent: MesosClusterUI) extends WebUIPage("driver") { override def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val driverId = UIUtils.stripXSS(request.getParameter("id")) + val driverId = request.getParameter("id") require(driverId != null && driverId.nonEmpty, "Missing id parameter") val state = parent.scheduler.getDriverState(driverId) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala index c53285331ea68..1755cb7f66117 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/mesos/ui/MesosClusterPage.scala @@ -77,7 +77,7 @@ private[mesos] class MesosClusterPage(parent: MesosClusterUI) extends WebUIPage( private def driverRow(state: MesosClusterSubmissionState): Seq[Node] = { val id = state.driverDescription.submissionId - val proxy = parent.conf.getOption("spark.mesos.proxy.baseURL") + val proxy = parent.conf.get(PROXY_BASE_URL) val sandboxCol = if (proxy.isDefined) { val clusterSchedulerId = parent.scheduler.getSchedulerState().frameworkId diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala index 68f6921153d89..2fd13a5903243 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/deploy/rest/mesos/MesosRestServer.scala @@ -27,6 +27,8 @@ import org.apache.spark.{SPARK_VERSION => sparkVersion, SparkConf} import org.apache.spark.deploy.Command import org.apache.spark.deploy.mesos.MesosDriverDescription import org.apache.spark.deploy.rest._ +import org.apache.spark.internal.config +import org.apache.spark.launcher.SparkLauncher import org.apache.spark.scheduler.cluster.mesos.MesosClusterScheduler import org.apache.spark.util.Utils @@ -67,6 +69,10 @@ private[mesos] class MesosSubmitRequestServlet( private def newDriverId(submitDate: Date): String = f"driver-${createDateFormat.format(submitDate)}-${nextDriverNumber.incrementAndGet()}%04d" + // These defaults copied from YARN + private val MEMORY_OVERHEAD_FACTOR = 0.10 + private val MEMORY_OVERHEAD_MIN = 384 + /** * Build a driver description from the fields specified in the submit request. * @@ -92,32 +98,38 @@ private[mesos] class MesosSubmitRequestServlet( // Optional fields val sparkProperties = request.sparkProperties - val driverExtraJavaOptions = sparkProperties.get("spark.driver.extraJavaOptions") - val driverExtraClassPath = sparkProperties.get("spark.driver.extraClassPath") - val driverExtraLibraryPath = sparkProperties.get("spark.driver.extraLibraryPath") - val superviseDriver = sparkProperties.get("spark.driver.supervise") - val driverMemory = sparkProperties.get("spark.driver.memory") - val driverCores = sparkProperties.get("spark.driver.cores") + val driverDefaultJavaOptions = sparkProperties.get(SparkLauncher.DRIVER_DEFAULT_JAVA_OPTIONS) + val driverExtraJavaOptions = sparkProperties.get(config.DRIVER_JAVA_OPTIONS.key) + val driverExtraClassPath = sparkProperties.get(config.DRIVER_CLASS_PATH.key) + val driverExtraLibraryPath = sparkProperties.get(config.DRIVER_LIBRARY_PATH.key) + val superviseDriver = sparkProperties.get(config.DRIVER_SUPERVISE.key) + val driverMemory = sparkProperties.get(config.DRIVER_MEMORY.key) + val driverMemoryOverhead = sparkProperties.get(config.DRIVER_MEMORY_OVERHEAD.key) + val driverCores = sparkProperties.get(config.DRIVER_CORES.key) val name = request.sparkProperties.getOrElse("spark.app.name", mainClass) // Construct driver description val conf = new SparkConf(false).setAll(sparkProperties) val extraClassPath = driverExtraClassPath.toSeq.flatMap(_.split(File.pathSeparator)) val extraLibraryPath = driverExtraLibraryPath.toSeq.flatMap(_.split(File.pathSeparator)) + val defaultJavaOpts = driverDefaultJavaOptions.map(Utils.splitCommandString) + .getOrElse(Seq.empty) val extraJavaOpts = driverExtraJavaOptions.map(Utils.splitCommandString).getOrElse(Seq.empty) val sparkJavaOpts = Utils.sparkJavaOpts(conf) - val javaOpts = sparkJavaOpts ++ extraJavaOpts + val javaOpts = sparkJavaOpts ++ defaultJavaOpts ++ extraJavaOpts val command = new Command( mainClass, appArgs, environmentVariables, extraClassPath, extraLibraryPath, javaOpts) val actualSuperviseDriver = superviseDriver.map(_.toBoolean).getOrElse(DEFAULT_SUPERVISE) val actualDriverMemory = driverMemory.map(Utils.memoryStringToMb).getOrElse(DEFAULT_MEMORY) + val actualDriverMemoryOverhead = driverMemoryOverhead.map(_.toInt).getOrElse( + math.max((MEMORY_OVERHEAD_FACTOR * actualDriverMemory).toInt, MEMORY_OVERHEAD_MIN)) val actualDriverCores = driverCores.map(_.toDouble).getOrElse(DEFAULT_CORES) val submitDate = new Date() val submissionId = newDriverId(submitDate) new MesosDriverDescription( - name, appResource, actualDriverMemory, actualDriverCores, actualSuperviseDriver, - command, request.sparkProperties, submissionId, submitDate) + name, appResource, actualDriverMemory + actualDriverMemoryOverhead, actualDriverCores, + actualSuperviseDriver, command, request.sparkProperties, submissionId, submitDate) } protected override def handleSubmit( diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 61bfa27a84fd8..bfa88d68d06c2 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -29,6 +29,7 @@ import org.apache.spark.{SparkConf, SparkEnv, TaskState} import org.apache.spark.TaskState import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.EXECUTOR_ID import org.apache.spark.scheduler.TaskDescription import org.apache.spark.scheduler.cluster.mesos.MesosSchedulerUtils import org.apache.spark.util.Utils @@ -74,6 +75,7 @@ private[spark] class MesosExecutorBackend val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) ++ Seq[(String, String)](("spark.app.id", frameworkInfo.getId.getValue)) val conf = new SparkConf(loadDefaults = true).setAll(properties) + conf.set(EXECUTOR_ID, executorId) val env = SparkEnv.createExecutorEnv( conf, executorId, slaveInfo.getHostname, cpusPerTask, None, isLocal = false) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala index da71f8f9e407c..ed1b3d7a16739 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.scheduler.cluster.mesos import org.apache.spark.SparkContext +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.internal.config._ import org.apache.spark.scheduler.{ExternalClusterManager, SchedulerBackend, TaskScheduler, TaskSchedulerImpl} @@ -42,7 +43,7 @@ private[spark] class MesosClusterManager extends ExternalClusterManager { "I/O encryption is currently not supported in Mesos.") val mesosUrl = MESOS_REGEX.findFirstMatchIn(masterURL).get.group(1) - val coarse = sc.conf.getBoolean("spark.mesos.coarse", defaultValue = true) + val coarse = sc.conf.get(COARSE_MODE) if (coarse) { new MesosCoarseGrainedSchedulerBackend( scheduler.asInstanceOf[TaskSchedulerImpl], diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala index 61ab3e87c5711..123412f21e2a2 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterPersistenceEngine.scala @@ -26,6 +26,7 @@ import org.apache.zookeeper.KeeperException.NoNodeException import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkCuratorUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Deploy._ import org.apache.spark.util.Utils /** @@ -94,13 +95,13 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( zk: CuratorFramework, conf: SparkConf) extends MesosClusterPersistenceEngine with Logging { - private val WORKING_DIR = - conf.get("spark.deploy.zookeeper.dir", "/spark_mesos_dispatcher") + "/" + baseDir + private val workingDir = + conf.get(ZOOKEEPER_DIRECTORY).getOrElse("/spark_mesos_dispatcher") + "/" + baseDir - SparkCuratorUtil.mkdir(zk, WORKING_DIR) + SparkCuratorUtil.mkdir(zk, workingDir) def path(name: String): String = { - WORKING_DIR + "/" + name + workingDir + "/" + name } override def expunge(name: String): Unit = { @@ -129,6 +130,6 @@ private[spark] class ZookeeperMesosClusterPersistenceEngine( } override def fetchAll[T](): Iterable[T] = { - zk.getChildren.forPath(WORKING_DIR).asScala.flatMap(fetch[T]) + zk.getChildren.forPath(workingDir).asScala.flatMap(fetch[T]) } } diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala index cb1bcba651be6..289b109a42747 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterScheduler.scala @@ -32,7 +32,8 @@ import org.apache.mesos.Protos.TaskStatus.Reason import org.apache.spark.{SecurityManager, SparkConf, SparkException, TaskState} import org.apache.spark.deploy.mesos.{config, MesosDriverDescription} import org.apache.spark.deploy.rest.{CreateSubmissionResponse, KillSubmissionResponse, SubmissionStatusResponse} -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.internal.config._ +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.util.Utils /** @@ -123,13 +124,13 @@ private[spark] class MesosClusterScheduler( extends Scheduler with MesosSchedulerUtils { var frameworkUrl: String = _ private val metricsSystem = - MetricsSystem.createMetricsSystem("mesos_cluster", conf, new SecurityManager(conf)) + MetricsSystem.createMetricsSystem(MetricsSystemInstances.MESOS_CLUSTER, conf, + new SecurityManager(conf)) private val master = conf.get("spark.master") private val appName = conf.get("spark.app.name") - private val queuedCapacity = conf.getInt("spark.mesos.maxDrivers", 200) - private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200) - private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute - private val useFetchCache = conf.getBoolean("spark.mesos.fetchCache.enable", false) + private val queuedCapacity = conf.get(config.MAX_DRIVERS) + private val retainedDrivers = conf.get(config.RETAINED_DRIVERS) + private val maxRetryWaitTime = conf.get(config.CLUSTER_RETRY_WAIT_MAX_SECONDS) private val schedulerState = engineFactory.createEngine("scheduler") private val stateLock = new Object() // Keyed by submission id @@ -365,8 +366,7 @@ private[spark] class MesosClusterScheduler( } private def getDriverExecutorURI(desc: MesosDriverDescription): Option[String] = { - desc.conf.getOption("spark.executor.uri") - .orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) + desc.conf.get(config.EXECUTOR_URI).orElse(desc.command.environment.get("SPARK_EXECUTOR_URI")) } private def getDriverFrameworkID(desc: MesosDriverDescription): String = { @@ -390,10 +390,10 @@ private[spark] class MesosClusterScheduler( private def getDriverEnvironment(desc: MesosDriverDescription): Environment = { // TODO(mgummelt): Don't do this here. This should be passed as a --conf val commandEnv = adjust(desc.command.environment, "SPARK_SUBMIT_OPTS", "")( - v => s"$v -Dspark.mesos.driver.frameworkId=${getDriverFrameworkID(desc)}" + v => s"$v -D${config.DRIVER_FRAMEWORK_ID.key}=${getDriverFrameworkID(desc)}" ) - val env = desc.conf.getAllWithPrefix("spark.mesos.driverEnv.") ++ commandEnv + val env = desc.conf.getAllWithPrefix(config.DRIVER_ENV_PREFIX) ++ commandEnv val envBuilder = Environment.newBuilder() @@ -419,22 +419,19 @@ private[spark] class MesosClusterScheduler( private def isContainerLocalAppJar(desc: MesosDriverDescription): Boolean = { val isLocalJar = desc.jarUrl.startsWith("local://") - val isContainerLocal = desc.conf.getOption("spark.mesos.appJar.local.resolution.mode").exists { + val isContainerLocal = desc.conf.get(config.APP_JAR_LOCAL_RESOLUTION_MODE) match { case "container" => true case "host" => false - case other => - logWarning(s"Unknown spark.mesos.appJar.local.resolution.mode $other, using host.") - false - } + } isLocalJar && isContainerLocal } private def getDriverUris(desc: MesosDriverDescription): List[CommandInfo.URI] = { - val confUris = List(conf.getOption("spark.mesos.uris"), - desc.conf.getOption("spark.mesos.uris"), - desc.conf.getOption("spark.submit.pyFiles")).flatMap( - _.map(_.split(",").map(_.trim)) - ).flatten + val useFetchCache = desc.conf.get(config.ENABLE_FETCHER_CACHE) || + conf.get(config.ENABLE_FETCHER_CACHE) + val confUris = (conf.get(config.URIS_TO_DOWNLOAD) ++ + desc.conf.get(config.URIS_TO_DOWNLOAD) ++ + desc.conf.get(SUBMIT_PYTHON_FILES)).toList if (isContainerLocalAppJar(desc)) { (confUris ++ getDriverExecutorURI(desc).toList).map(uri => @@ -463,8 +460,8 @@ private[spark] class MesosClusterScheduler( containerInfo } - private def getDriverCommandValue(desc: MesosDriverDescription): String = { - val dockerDefined = desc.conf.contains("spark.mesos.executor.docker.image") + private[mesos] def getDriverCommandValue(desc: MesosDriverDescription): String = { + val dockerDefined = desc.conf.contains(config.EXECUTOR_DOCKER_IMAGE) val executorUri = getDriverExecutorURI(desc) // Gets the path to run spark-submit, and the path to the Mesos sandbox. val (executable, sandboxPath) = if (dockerDefined) { @@ -474,7 +471,7 @@ private[spark] class MesosClusterScheduler( } else if (executorUri.isDefined) { val folderBasename = executorUri.get.split('/').last.split('.').head - val entries = conf.getOption("spark.executor.extraLibraryPath") + val entries = conf.get(EXECUTOR_LIBRARY_PATH) .map(path => Seq(path) ++ desc.command.libraryPathEntries) .getOrElse(desc.command.libraryPathEntries) @@ -484,11 +481,11 @@ private[spark] class MesosClusterScheduler( // Sandbox path points to the parent folder as we chdir into the folderBasename. (cmdExecutable, "..") } else { - val executorSparkHome = desc.conf.getOption("spark.mesos.executor.home") + val executorSparkHome = desc.conf.get(config.EXECUTOR_HOME) .orElse(conf.getOption("spark.home")) .orElse(Option(System.getenv("SPARK_HOME"))) .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + throw new SparkException(s"Executor Spark home `${config.EXECUTOR_HOME}` is not set!") } val cmdExecutable = new File(executorSparkHome, "./bin/spark-submit").getPath // Sandbox points to the current directory by default with Mesos. @@ -503,7 +500,7 @@ private[spark] class MesosClusterScheduler( } } - val appArguments = desc.command.arguments.mkString(" ") + val appArguments = desc.command.arguments.map(shellEscape).mkString(" ") s"$executable $cmdOptions $primaryResource $appArguments" } @@ -528,26 +525,26 @@ private[spark] class MesosClusterScheduler( options ++= Seq("--class", desc.command.mainClass) } - desc.conf.getOption("spark.executor.memory").foreach { v => + desc.conf.getOption(EXECUTOR_MEMORY.key).foreach { v => options ++= Seq("--executor-memory", v) } - desc.conf.getOption("spark.cores.max").foreach { v => + desc.conf.getOption(CORES_MAX.key).foreach { v => options ++= Seq("--total-executor-cores", v) } - desc.conf.getOption("spark.submit.pyFiles").foreach { pyFiles => - val formattedFiles = pyFiles.split(",") - .map { path => new File(sandboxPath, path.split("/").last).toString() } - .mkString(",") - options ++= Seq("--py-files", formattedFiles) - } + + val pyFiles = desc.conf.get(SUBMIT_PYTHON_FILES) + val formattedFiles = pyFiles.map { path => + new File(sandboxPath, path.split("/").last).toString() + }.mkString(",") + options ++= Seq("--py-files", formattedFiles) // --conf val replicatedOptionsBlacklist = Set( - "spark.jars", // Avoids duplicate classes in classpath - "spark.submit.deployMode", // this would be set to `cluster`, but we need client + JARS.key, // Avoids duplicate classes in classpath + SUBMIT_DEPLOY_MODE.key, // this would be set to `cluster`, but we need client "spark.master" // this contains the address of the dispatcher, not master ) - val defaultConf = conf.getAllWithPrefix("spark.mesos.dispatcher.driverDefault.").toMap + val defaultConf = conf.getAllWithPrefix(config.DISPATCHER_DRIVER_DEFAULT_PREFIX).toMap val driverConf = desc.conf.getAll .filter { case (key, _) => !replicatedOptionsBlacklist.contains(key) } .toMap @@ -769,6 +766,10 @@ private[spark] class MesosClusterScheduler( val state = launchedDrivers(subId) // Check if the driver is supervise enabled and can be relaunched. if (state.driverDescription.supervise && shouldRelaunch(status.getState)) { + if (isTaskOutdated(taskId, state)) { + // Prevent outdated task from overwriting a more recent status + return + } removeFromLaunchedDrivers(subId) state.finishDate = Some(new Date()) val retryState: Option[MesosClusterRetryState] = state.driverDescription.retryState @@ -789,6 +790,16 @@ private[spark] class MesosClusterScheduler( } } + /** + * Check if the task is outdated i.e. has already been launched or is pending + * If neither, the taskId is outdated and should be ignored + * This is to avoid scenarios where an outdated status update arrives + * after a supervised driver has already been relaunched + */ + private def isTaskOutdated(taskId: String, state: MesosClusterSubmissionState): Boolean = + taskId != state.taskId.getValue && + !pendingRetryDrivers.exists(_.submissionId == state.driverDescription.submissionId) + private def retireDriver( submissionId: String, state: MesosClusterSubmissionState) = { diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala index bac0246b7ddc5..6681ca3778f4d 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackend.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.File import java.util.{Collections, List => JList} +import java.util.concurrent.TimeUnit import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong} import java.util.concurrent.locks.ReentrantLock @@ -26,18 +27,18 @@ import scala.collection.JavaConverters._ import scala.collection.mutable import scala.concurrent.Future -import org.apache.hadoop.security.UserGroupInformation import org.apache.mesos.Protos.{TaskInfo => MesosTaskInfo, _} import org.apache.mesos.SchedulerDriver import org.apache.spark.{SecurityManager, SparkConf, SparkContext, SparkException, TaskState} import org.apache.spark.deploy.mesos.config._ +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.config -import org.apache.spark.internal.config.EXECUTOR_HEARTBEAT_INTERVAL +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle} import org.apache.spark.network.netty.SparkTransportConf import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient -import org.apache.spark.rpc.RpcEndpointAddress +import org.apache.spark.rpc.{RpcEndpointAddress, RpcEndpointRef} import org.apache.spark.scheduler.{SlaveLost, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.util.Utils @@ -60,15 +61,12 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( extends CoarseGrainedSchedulerBackend(scheduler, sc.env.rpcEnv) with org.apache.mesos.Scheduler with MesosSchedulerUtils { - private lazy val hadoopDelegationTokenManager: MesosHadoopDelegationTokenManager = - new MesosHadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint) - // Blacklist a slave after this many failures private val MAX_SLAVE_FAILURES = 2 - private val maxCoresOption = conf.getOption("spark.cores.max").map(_.toInt) + private val maxCoresOption = conf.get(config.CORES_MAX) - private val executorCoresOption = conf.getOption("spark.executor.cores").map(_.toInt) + private val executorCoresOption = conf.getOption(config.EXECUTOR_CORES.key).map(_.toInt) private val minCoresPerExecutor = executorCoresOption.getOrElse(1) @@ -79,15 +77,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( cores - (cores % minCoresPerExecutor) } - private val useFetcherCache = conf.getBoolean("spark.mesos.fetcherCache.enable", false) + private val useFetcherCache = conf.get(ENABLE_FETCHER_CACHE) - private val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + private val maxGpus = conf.get(MAX_GPUS) - private val taskLabels = conf.get("spark.mesos.task.labels", "") + private val taskLabels = conf.get(TASK_LABELS) - private[this] val shutdownTimeoutMS = - conf.getTimeAsMs("spark.mesos.coarse.shutdownTimeout", "10s") - .ensuring(_ >= 0, "spark.mesos.coarse.shutdownTimeout must be >= 0") + private[this] val shutdownTimeoutMS = conf.get(COARSE_SHUTDOWN_TIMEOUT) // Synchronization protected by stateLock private[this] var stopCalled: Boolean = false @@ -112,9 +108,9 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( private var totalGpusAcquired = 0 // The amount of time to wait for locality scheduling - private val localityWait = conf.get(config.LOCALITY_WAIT) + private val localityWaitNs = TimeUnit.MILLISECONDS.toNanos(conf.get(config.LOCALITY_WAIT)) // The start of the waiting, for data local scheduling - private var localityWaitStartTime = System.currentTimeMillis() + private var localityWaitStartTimeNs = System.nanoTime() // If true, the scheduler is in the process of launching executors to reach the requested // executor limit private var launchingExecutors = false @@ -147,11 +143,11 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // may lead to deadlocks since the superclass might also try to lock private val stateLock = new ReentrantLock - private val extraCoresPerExecutor = conf.getInt("spark.mesos.extra.cores", 0) + private val extraCoresPerExecutor = conf.get(EXTRA_CORES_PER_EXECUTOR) // Offer constraints private val slaveOfferConstraints = - parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + parseConstraintString(sc.conf.get(CONSTRAINTS)) // Reject offers with mismatched constraints in seconds private val rejectOfferDurationForUnmetConstraints = @@ -211,10 +207,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( sc.sparkUser, sc.appName, sc.conf, - sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), + sc.conf.get(DRIVER_WEBUI_URL).orElse(sc.ui.map(_.webUrl)), None, Some(sc.conf.get(DRIVER_FAILOVER_TIMEOUT)), - sc.conf.getOption("spark.mesos.driver.frameworkId").map(_ + suffix) + sc.conf.get(DRIVER_FRAMEWORK_ID).map(_ + suffix) ) launcherBackend.setState(SparkAppHandle.State.SUBMITTED) @@ -223,18 +219,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( def createCommand(offer: Offer, numCores: Int, taskId: String): CommandInfo = { val environment = Environment.newBuilder() - val extraClassPath = conf.getOption("spark.executor.extraClassPath") + val extraClassPath = conf.get(config.EXECUTOR_CLASS_PATH) extraClassPath.foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, taskId) }.getOrElse("") // Set the environment variable through a command prefix // to append to the existing value of the variable - val prefixEnv = conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -264,14 +260,13 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = conf.getOption("spark.executor.uri") - .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) + val uri = conf.get(EXECUTOR_URI).orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) if (uri.isEmpty) { - val executorSparkHome = conf.getOption("spark.mesos.executor.home") + val executorSparkHome = conf.get(EXECUTOR_HOME) .orElse(sc.getSparkHome()) .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + throw new SparkException(s"Executor Spark home `$EXECUTOR_HOME` is not set!") } val runScript = new File(executorSparkHome, "./bin/spark-class").getPath command.setValue( @@ -297,18 +292,18 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( command.addUris(CommandInfo.URI.newBuilder().setValue(uri.get).setCache(useFetcherCache)) } - conf.getOption("spark.mesos.uris").foreach(setupUris(_, command, useFetcherCache)) + setupUris(conf.get(URIS_TO_DOWNLOAD), command, useFetcherCache) command.build() } protected def driverURL: String = { - if (conf.contains("spark.testing")) { + if (conf.contains(IS_TESTING)) { "driverURL" } else { RpcEndpointAddress( - conf.get("spark.driver.host"), - conf.get("spark.driver.port").toInt, + conf.get(config.DRIVER_HOST_ADDRESS), + conf.get(config.DRIVER_PORT), CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString } } @@ -363,7 +358,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } else { if (!launchingExecutors) { launchingExecutors = true - localityWaitStartTime = System.currentTimeMillis() + localityWaitStartTimeNs = System.nanoTime() } } @@ -597,7 +592,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( // increase coverage. val remainingHosts = allDesiredHosts -- currentHosts if (!remainingHosts.contains(offerHostname) && - (System.currentTimeMillis() - localityWaitStartTime <= localityWait)) { + (System.nanoTime() - localityWaitStartTimeNs <= localityWaitNs)) { logDebug("Skipping host and waiting for locality. host: " + offerHostname) return false } @@ -634,9 +629,8 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( .registerDriverWithShuffleService( slave.hostname, externalShufflePort, - sc.conf.getTimeAsMs("spark.storage.blockManagerSlaveTimeoutMs", - s"${sc.conf.getTimeAsSeconds("spark.network.timeout", "120s")}s"), - sc.conf.get(EXECUTOR_HEARTBEAT_INTERVAL)) + sc.conf.get(config.STORAGE_BLOCKMANAGER_SLAVE_TIMEOUT), + sc.conf.get(config.EXECUTOR_HEARTBEAT_INTERVAL)) slave.shuffleRegistered = true } @@ -678,7 +672,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( launcherBackend.close() } - private def stopSchedulerBackend() { + private def stopSchedulerBackend(): Unit = { // Make sure we're not launching tasks during shutdown stateLock.synchronized { if (stopCalled) { @@ -757,7 +751,7 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( logInfo("Capping the total amount of executors to " + requestedTotal) executorLimitOption = Some(requestedTotal) // Update the locality wait start time to continue trying for locality. - localityWaitStartTime = System.currentTimeMillis() + localityWaitStartTimeNs = System.nanoTime() true } @@ -777,6 +771,10 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( } } + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(conf, sc.hadoopConfiguration, driverEndpoint)) + } + private def numExecutors(): Int = { slaves.values.map(_.taskIDs.size).sum } @@ -789,14 +787,6 @@ private[spark] class MesosCoarseGrainedSchedulerBackend( offer.getHostname } } - - override def fetchHadoopDelegationTokens(): Option[Array[Byte]] = { - if (UserGroupInformation.isSecurityEnabled) { - Some(hadoopDelegationTokenManager.getTokens()) - } else { - None - } - } } private class Slave(val hostname: String) { 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 0bb6fe0fa4bdf..a03fecdb2abee 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 @@ -28,8 +28,9 @@ import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString import org.apache.spark.{SparkContext, SparkException, TaskState} -import org.apache.spark.deploy.mesos.config +import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.internal.config import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.util.Utils @@ -59,11 +60,11 @@ private[spark] class MesosFineGrainedSchedulerBackend( // The listener bus to publish executor added/removed events. val listenerBus = sc.listenerBus - private[mesos] val mesosExecutorCores = sc.conf.getDouble("spark.mesos.mesosExecutor.cores", 1) + private[mesos] val mesosExecutorCores = sc.conf.get(mesosConfig.EXECUTOR_CORES) // Offer constraints private[this] val slaveOfferConstraints = - parseConstraintString(sc.conf.get("spark.mesos.constraints", "")) + parseConstraintString(sc.conf.get(mesosConfig.CONSTRAINTS)) // reject offers with mismatched constraints in seconds private val rejectOfferDurationForUnmetConstraints = @@ -81,10 +82,10 @@ private[spark] class MesosFineGrainedSchedulerBackend( sc.sparkUser, sc.appName, sc.conf, - sc.conf.getOption("spark.mesos.driver.webui.url").orElse(sc.ui.map(_.webUrl)), + sc.conf.get(mesosConfig.DRIVER_WEBUI_URL).orElse(sc.ui.map(_.webUrl)), Option.empty, Option.empty, - sc.conf.getOption("spark.mesos.driver.frameworkId") + sc.conf.get(mesosConfig.DRIVER_FRAMEWORK_ID) ) unsetFrameworkID(sc) @@ -101,21 +102,21 @@ private[spark] class MesosFineGrainedSchedulerBackend( def createExecutorInfo( availableResources: JList[Resource], execId: String): (MesosExecutorInfo, JList[Resource]) = { - val executorSparkHome = sc.conf.getOption("spark.mesos.executor.home") + val executorSparkHome = sc.conf.get(mesosConfig.EXECUTOR_HOME) .orElse(sc.getSparkHome()) // Fall back to driver Spark home for backward compatibility .getOrElse { - throw new SparkException("Executor Spark home `spark.mesos.executor.home` is not set!") + throw new SparkException(s"Executor Spark home `${mesosConfig.EXECUTOR_HOME}` is not set!") } val environment = Environment.newBuilder() - sc.conf.getOption("spark.executor.extraClassPath").foreach { cp => + sc.conf.get(config.EXECUTOR_CLASS_PATH).foreach { cp => environment.addVariables( Environment.Variable.newBuilder().setName("SPARK_EXECUTOR_CLASSPATH").setValue(cp).build()) } - val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions").map { + val extraJavaOpts = sc.conf.get(config.EXECUTOR_JAVA_OPTIONS).map { Utils.substituteAppNExecIds(_, appId, execId) }.getOrElse("") - val prefixEnv = sc.conf.getOption("spark.executor.extraLibraryPath").map { p => + val prefixEnv = sc.conf.get(config.EXECUTOR_LIBRARY_PATH).map { p => Utils.libraryPathEnvPrefix(Seq(p)) }.getOrElse("") @@ -132,7 +133,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( } val command = CommandInfo.newBuilder() .setEnvironment(environment) - val uri = sc.conf.getOption("spark.executor.uri") + val uri = sc.conf.get(mesosConfig.EXECUTOR_URI) .orElse(Option(System.getenv("SPARK_EXECUTOR_URI"))) val executorBackendName = classOf[MesosExecutorBackend].getName @@ -155,7 +156,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( builder.addAllResources(usedCpuResources.asJava) builder.addAllResources(usedMemResources.asJava) - sc.conf.getOption("spark.mesos.uris").foreach(setupUris(_, command)) + setupUris(sc.conf.get(mesosConfig.URIS_TO_DOWNLOAD), command) val executorInfo = builder .setExecutorId(ExecutorID.newBuilder().setValue(execId).build()) @@ -328,7 +329,7 @@ private[spark] class MesosFineGrainedSchedulerBackend( slaveIdToWorkerOffer.get(slaveId).foreach(o => listenerBus.post(SparkListenerExecutorAdded(System.currentTimeMillis(), slaveId, // TODO: Add support for log urls for Mesos - new ExecutorInfo(o.host, o.cores, Map.empty))) + new ExecutorInfo(o.host, o.cores, Map.empty, Map.empty))) ) logTrace(s"Launching Mesos tasks on slave '$slaveId', tasks:\n${getTasksSummary(tasks)}") d.launchTasks(Collections.singleton(slaveIdToOffer(slaveId).getId), tasks, filters) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala deleted file mode 100644 index a1bf4f0c048fe..0000000000000 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosHadoopDelegationTokenManager.scala +++ /dev/null @@ -1,160 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster.mesos - -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.UserGroupInformation - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.internal.{config, Logging} -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.ThreadUtils - - -/** - * The MesosHadoopDelegationTokenManager fetches and updates Hadoop delegation tokens on the behalf - * of the MesosCoarseGrainedSchedulerBackend. It is modeled after the YARN AMCredentialRenewer, - * and similarly will renew the Credentials when 75% of the renewal interval has passed. - * The principal difference is that instead of writing the new credentials to HDFS and - * incrementing the timestamp of the file, the new credentials (called Tokens when they are - * serialized) are broadcast to all running executors. On the executor side, when new Tokens are - * received they overwrite the current credentials. - */ -private[spark] class MesosHadoopDelegationTokenManager( - conf: SparkConf, - hadoopConfig: Configuration, - driverEndpoint: RpcEndpointRef) - extends Logging { - - require(driverEndpoint != null, "DriverEndpoint is not initialized") - - private val credentialRenewerThread: ScheduledExecutorService = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Renewal Thread") - - private val tokenManager: HadoopDelegationTokenManager = - new HadoopDelegationTokenManager(conf, hadoopConfig) - - private val principal: String = conf.get(config.PRINCIPAL).orNull - - private var (tokens: Array[Byte], timeOfNextRenewal: Long) = { - try { - val creds = UserGroupInformation.getCurrentUser.getCredentials - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val rt = tokenManager.obtainDelegationTokens(hadoopConf, creds) - logInfo(s"Initialized tokens: ${SparkHadoopUtil.get.dumpTokens(creds)}") - (SparkHadoopUtil.get.serialize(creds), SparkHadoopUtil.nextCredentialRenewalTime(rt, conf)) - } catch { - case e: Exception => - logError(s"Failed to fetch Hadoop delegation tokens $e") - throw e - } - } - - private val keytabFile: Option[String] = conf.get(config.KEYTAB) - - scheduleTokenRenewal() - - private def scheduleTokenRenewal(): Unit = { - if (keytabFile.isDefined) { - require(principal != null, "Principal is required for Keytab-based authentication") - logInfo(s"Using keytab: ${keytabFile.get} and principal $principal") - } else { - logInfo("Using ticket cache for Kerberos authentication, no token renewal.") - return - } - - def scheduleRenewal(runnable: Runnable): Unit = { - val remainingTime = timeOfNextRenewal - System.currentTimeMillis() - if (remainingTime <= 0) { - logInfo("Credentials have expired, creating new ones now.") - runnable.run() - } else { - logInfo(s"Scheduling login from keytab in $remainingTime millis.") - credentialRenewerThread.schedule(runnable, remainingTime, TimeUnit.MILLISECONDS) - } - } - - val credentialRenewerRunnable = - new Runnable { - override def run(): Unit = { - try { - getNewDelegationTokens() - broadcastDelegationTokens(tokens) - } catch { - case e: Exception => - // Log the error and try to write new tokens back in an hour - val delay = TimeUnit.SECONDS.toMillis(conf.get(config.CREDENTIALS_RENEWAL_RETRY_WAIT)) - logWarning( - s"Couldn't broadcast tokens, trying again in ${UIUtils.formatDuration(delay)}", e) - credentialRenewerThread.schedule(this, delay, TimeUnit.MILLISECONDS) - return - } - scheduleRenewal(this) - } - } - scheduleRenewal(credentialRenewerRunnable) - } - - private def getNewDelegationTokens(): Unit = { - logInfo(s"Attempting to login to KDC with principal ${principal}") - // Get new delegation tokens by logging in with a new UGI inspired by AMCredentialRenewer.scala - // Don't protect against keytabFile being empty because it's guarded above. - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytabFile.get) - logInfo("Successfully logged into KDC") - val tempCreds = ugi.getCredentials - val hadoopConf = SparkHadoopUtil.get.newConfiguration(conf) - val nextRenewalTime = ugi.doAs(new PrivilegedExceptionAction[Long] { - override def run(): Long = { - tokenManager.obtainDelegationTokens(hadoopConf, tempCreds) - } - }) - - val currTime = System.currentTimeMillis() - timeOfNextRenewal = if (nextRenewalTime <= currTime) { - logWarning(s"Next credential renewal time ($nextRenewalTime) is earlier than " + - s"current time ($currTime), which is unexpected, please check your credential renewal " + - "related configurations in the target services.") - currTime - } else { - SparkHadoopUtil.nextCredentialRenewalTime(nextRenewalTime, conf) - } - logInfo(s"Time of next renewal is in ${timeOfNextRenewal - System.currentTimeMillis()} ms") - - // Add the temp credentials back to the original ones. - UserGroupInformation.getCurrentUser.addCredentials(tempCreds) - // update tokens for late or dynamically added executors - tokens = SparkHadoopUtil.get.serialize(tempCreds) - } - - private def broadcastDelegationTokens(tokens: Array[Byte]) = { - logInfo("Sending new tokens to all executors") - driverEndpoint.send(UpdateDelegationTokens(tokens)) - } - - def getTokens(): Array[Byte] = { - tokens - } -} - diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala index b4364a5e2eb3a..a217deb8b49d7 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtil.scala @@ -24,7 +24,7 @@ import org.apache.mesos.protobuf.ByteString import org.apache.spark.SparkConf import org.apache.spark.SparkException -import org.apache.spark.deploy.mesos.config.{NETWORK_LABELS, NETWORK_NAME} +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.deploy.mesos.config.MesosSecretConfig import org.apache.spark.internal.Logging @@ -34,11 +34,11 @@ import org.apache.spark.internal.Logging */ private[mesos] object MesosSchedulerBackendUtil extends Logging { /** - * Parse a comma-delimited list of volume specs, each of which + * Parse a list of volume specs, each of which * takes the form [host-dir:]container-dir[:rw|:ro]. */ - def parseVolumesSpec(volumes: String): List[Volume] = { - volumes.split(",").map(_.split(":")).flatMap { spec => + def parseVolumesSpec(volumes: Seq[String]): List[Volume] = { + volumes.map(_.split(":")).flatMap { spec => val vol: Volume.Builder = Volume .newBuilder() .setMode(Volume.Mode.RW) @@ -71,7 +71,7 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { } /** - * Parse a comma-delimited list of port mapping specs, each of which + * Parse a list of port mapping specs, each of which * takes the form host_port:container_port[:udp|:tcp] * * Note: @@ -81,8 +81,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { * anticipates the expansion of the docker form to allow for a protocol * and leaves open the chance for mesos to begin to accept an 'ip' field */ - def parsePortMappingsSpec(portmaps: String): List[DockerInfo.PortMapping] = { - portmaps.split(",").map(_.split(":")).flatMap { spec: Array[String] => + def parsePortMappingsSpec(portmaps: Seq[String]): List[DockerInfo.PortMapping] = { + portmaps.map(_.split(":")).flatMap { spec: Array[String] => val portmap: DockerInfo.PortMapping.Builder = DockerInfo.PortMapping .newBuilder() .setProtocol("tcp") @@ -108,10 +108,10 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { * Parse a list of docker parameters, each of which * takes the form key=value */ - private def parseParamsSpec(params: String): List[Parameter] = { + private def parseParamsSpec(params: Seq[String]): List[Parameter] = { // split with limit of 2 to avoid parsing error when '=' // exists in the parameter value - params.split(",").map(_.split("=", 2)).flatMap { spec: Array[String] => + params.map(_.split("=", 2)).flatMap { spec: Array[String] => val param: Parameter.Builder = Parameter.newBuilder() spec match { case Array(key, value) => @@ -127,8 +127,8 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { } def buildContainerInfo(conf: SparkConf): ContainerInfo.Builder = { - val containerType = if (conf.contains("spark.mesos.executor.docker.image") && - conf.get("spark.mesos.containerizer", "docker") == "docker") { + val containerType = if (conf.contains(EXECUTOR_DOCKER_IMAGE) && + conf.get(CONTAINERIZER) == "docker") { ContainerInfo.Type.DOCKER } else { ContainerInfo.Type.MESOS @@ -137,18 +137,17 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { val containerInfo = ContainerInfo.newBuilder() .setType(containerType) - conf.getOption("spark.mesos.executor.docker.image").map { image => + conf.get(EXECUTOR_DOCKER_IMAGE).map { image => val forcePullImage = conf - .getOption("spark.mesos.executor.docker.forcePullImage") - .exists(_.equals("true")) + .get(EXECUTOR_DOCKER_FORCE_PULL_IMAGE).contains(true) val portMaps = conf - .getOption("spark.mesos.executor.docker.portmaps") + .get(EXECUTOR_DOCKER_PORT_MAPS) .map(parsePortMappingsSpec) .getOrElse(List.empty) val params = conf - .getOption("spark.mesos.executor.docker.parameters") + .get(EXECUTOR_DOCKER_PARAMETERS) .map(parseParamsSpec) .getOrElse(List.empty) @@ -159,7 +158,7 @@ private[mesos] object MesosSchedulerBackendUtil extends Logging { } val volumes = conf - .getOption("spark.mesos.executor.docker.volumes") + .get(EXECUTOR_DOCKER_VOLUMES) .map(parseVolumesSpec) volumes.foreach(_.foreach(containerInfo.addVolumes(_))) diff --git a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala index 8ef1e18f83de3..06993712035ff 100644 --- a/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala +++ b/resource-managers/mesos/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtils.scala @@ -36,8 +36,9 @@ import org.apache.mesos.protobuf.{ByteString, GeneratedMessageV3} import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.TaskState +import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.{Status => _, _} import org.apache.spark.util.Utils /** @@ -83,10 +84,10 @@ trait MesosSchedulerUtils extends Logging { fwInfoBuilder.setId(FrameworkID.newBuilder().setValue(id).build()) } - conf.getOption("spark.mesos.role").foreach { role => + conf.get(mesosConfig.ROLE).foreach { role => fwInfoBuilder.setRole(role) } - val maxGpus = conf.getInt("spark.mesos.gpus.max", 0) + val maxGpus = conf.get(mesosConfig.MAX_GPUS) if (maxGpus > 0) { fwInfoBuilder.addCapabilities(Capability.newBuilder().setType(Capability.Type.GPU_RESOURCES)) } @@ -103,10 +104,10 @@ trait MesosSchedulerUtils extends Logging { conf: SparkConf, fwInfoBuilder: Protos.FrameworkInfo.Builder): Protos.Credential.Builder = { val credBuilder = Credential.newBuilder() - conf.getOption("spark.mesos.principal") + conf.get(mesosConfig.CREDENTIAL_PRINCIPAL) .orElse(Option(conf.getenv("SPARK_MESOS_PRINCIPAL"))) .orElse( - conf.getOption("spark.mesos.principal.file") + conf.get(mesosConfig.CREDENTIAL_PRINCIPAL_FILE) .orElse(Option(conf.getenv("SPARK_MESOS_PRINCIPAL_FILE"))) .map { principalFile => Files.toString(new File(principalFile), StandardCharsets.UTF_8) @@ -115,10 +116,10 @@ trait MesosSchedulerUtils extends Logging { fwInfoBuilder.setPrincipal(principal) credBuilder.setPrincipal(principal) } - conf.getOption("spark.mesos.secret") + conf.get(mesosConfig.CREDENTIAL_SECRET) .orElse(Option(conf.getenv("SPARK_MESOS_SECRET"))) .orElse( - conf.getOption("spark.mesos.secret.file") + conf.get(mesosConfig.CREDENTIAL_SECRET_FILE) .orElse(Option(conf.getenv("SPARK_MESOS_SECRET_FILE"))) .map { secretFile => Files.toString(new File(secretFile), StandardCharsets.UTF_8) @@ -128,7 +129,8 @@ trait MesosSchedulerUtils extends Logging { } if (credBuilder.hasSecret && !fwInfoBuilder.hasPrincipal) { throw new SparkException( - "spark.mesos.principal must be configured when spark.mesos.secret is set") + s"${mesosConfig.CREDENTIAL_PRINCIPAL} must be configured when " + + s"${mesosConfig.CREDENTIAL_SECRET} is set") } credBuilder } @@ -399,37 +401,31 @@ trait MesosSchedulerUtils extends Logging { * (whichever is larger) */ def executorMemory(sc: SparkContext): Int = { - sc.conf.getInt("spark.mesos.executor.memoryOverhead", + sc.conf.get(mesosConfig.EXECUTOR_MEMORY_OVERHEAD).getOrElse( math.max(MEMORY_OVERHEAD_FRACTION * sc.executorMemory, MEMORY_OVERHEAD_MINIMUM).toInt) + sc.executorMemory } - def setupUris(uris: String, + def setupUris(uris: Seq[String], builder: CommandInfo.Builder, useFetcherCache: Boolean = false): Unit = { - uris.split(",").foreach { uri => + uris.foreach { uri => builder.addUris(CommandInfo.URI.newBuilder().setValue(uri.trim()).setCache(useFetcherCache)) } } - private def getRejectOfferDurationStr(conf: SparkConf): String = { - conf.get("spark.mesos.rejectOfferDuration", "120s") - } - protected def getRejectOfferDuration(conf: SparkConf): Long = { - Utils.timeStringAsSeconds(getRejectOfferDurationStr(conf)) + conf.get(mesosConfig.REJECT_OFFER_DURATION) } protected def getRejectOfferDurationForUnmetConstraints(conf: SparkConf): Long = { - conf.getTimeAsSeconds( - "spark.mesos.rejectOfferDurationForUnmetConstraints", - getRejectOfferDurationStr(conf)) + conf.get(mesosConfig.REJECT_OFFER_DURATION_FOR_UNMET_CONSTRAINTS) + .getOrElse(getRejectOfferDuration(conf)) } protected def getRejectOfferDurationForReachedMaxCores(conf: SparkConf): Long = { - conf.getTimeAsSeconds( - "spark.mesos.rejectOfferDurationForReachedMaxCores", - getRejectOfferDurationStr(conf)) + conf.get(mesosConfig.REJECT_OFFER_DURATION_FOR_REACHED_MAX_CORES) + .getOrElse(getRejectOfferDuration(conf)) } /** @@ -558,8 +554,8 @@ trait MesosSchedulerUtils extends Logging { * framework ID, the driver calls this method after the first registration. */ def unsetFrameworkID(sc: SparkContext) { - sc.conf.remove("spark.mesos.driver.frameworkId") - System.clearProperty("spark.mesos.driver.frameworkId") + sc.conf.remove(mesosConfig.DRIVER_FRAMEWORK_ID) + System.clearProperty(mesosConfig.DRIVER_FRAMEWORK_ID.key) } def mesosToTaskState(state: MesosTaskState): TaskState.TaskState = state match { diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.scala new file mode 100644 index 0000000000000..1f83149a05652 --- /dev/null +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/deploy/rest/mesos/MesosRestServerSuite.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.deploy.rest.mesos + +import javax.servlet.http.HttpServletResponse + +import org.scalatest.mockito.MockitoSugar + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.TestPrematureExit +import org.apache.spark.deploy.mesos.MesosDriverDescription +import org.apache.spark.deploy.rest.{CreateSubmissionRequest, CreateSubmissionResponse, SubmitRestProtocolMessage, SubmitRestProtocolResponse} +import org.apache.spark.internal.config +import org.apache.spark.scheduler.cluster.mesos.{MesosClusterPersistenceEngineFactory, MesosClusterScheduler} + +class MesosRestServerSuite extends SparkFunSuite + with TestPrematureExit with MockitoSugar { + + test("test default driver overhead memory") { + testOverheadMemory(new SparkConf(), "2000M", 2384) + } + + test("test driver overhead memory with overhead factor") { + testOverheadMemory(new SparkConf(), "5000M", 5500) + } + + test("test configured driver overhead memory") { + val conf = new SparkConf() + conf.set(config.DRIVER_MEMORY_OVERHEAD.key, "1000") + testOverheadMemory(conf, "2000M", 3000) + } + + def testOverheadMemory(conf: SparkConf, driverMemory: String, expectedResult: Int) { + conf.set("spark.master", "testmaster") + conf.set("spark.app.name", "testapp") + conf.set(config.DRIVER_MEMORY.key, driverMemory) + var actualMem = 0 + class TestMesosClusterScheduler extends MesosClusterScheduler( + mock[MesosClusterPersistenceEngineFactory], conf) { + override def submitDriver(desc: MesosDriverDescription): CreateSubmissionResponse = { + actualMem = desc.mem + mock[CreateSubmissionResponse] + } + } + + class TestServlet extends MesosSubmitRequestServlet(new TestMesosClusterScheduler, conf) { + override def handleSubmit( + requestMessageJson: String, + requestMessage: SubmitRestProtocolMessage, + responseServlet: HttpServletResponse): SubmitRestProtocolResponse = { + super.handleSubmit(requestMessageJson, requestMessage, responseServlet) + } + + override def findUnknownFields( + requestJson: String, + requestMessage: SubmitRestProtocolMessage): Array[String] = { + Array() + } + } + val servlet = new TestServlet() + val request = new CreateSubmissionRequest() + request.appResource = "testresource" + request.mainClass = "mainClass" + request.appArgs = Array("appArgs") + request.environmentVariables = Map("envVar" -> "envVal") + request.sparkProperties = conf.getAll.toMap + servlet.handleSubmit("null", request, null) + assert(actualMem == expectedResult) + } +} diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala index a55855428b471..e8520061ac38d 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterManagerSuite.scala @@ -18,11 +18,12 @@ package org.apache.spark.scheduler.cluster.mesos import org.apache.spark._ +import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.internal.config._ class MesosClusterManagerSuite extends SparkFunSuite with LocalSparkContext { def testURL(masterURL: String, expectedClass: Class[_], coarse: Boolean) { - val conf = new SparkConf().set("spark.mesos.coarse", coarse.toString) + val conf = new SparkConf().set(mesosConfig.COARSE_MODE, coarse) sc = new SparkContext("local", "test", conf) val clusterManager = new MesosClusterManager() diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala index 082d4bcfdf83a..924a991c50f23 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosClusterSchedulerSuite.scala @@ -24,7 +24,8 @@ import scala.collection.JavaConverters._ import org.apache.mesos.Protos.{TaskState => MesosTaskState, _} import org.apache.mesos.Protos.Value.{Scalar, Type} import org.apache.mesos.SchedulerDriver -import org.mockito.{ArgumentCaptor, Matchers} +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar @@ -104,7 +105,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 1200, 1.5, true, command, - Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")), + Map((config.EXECUTOR_HOME.key, "test"), ("spark.app.name", "test")), "s1", new Date())) assert(response.success) @@ -133,7 +134,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi when( driver.launchTasks( - Matchers.eq(Collections.singleton(offer.getId)), + meq(Collections.singleton(offer.getId)), capture.capture()) ).thenReturn(Status.valueOf(1)) @@ -156,7 +157,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(mem.exists(_.getRole() == "*")) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(offer.getId)), + meq(Collections.singleton(offer.getId)), capture.capture() ) } @@ -208,9 +209,9 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", mem, cpu, true, command, - Map("spark.mesos.executor.home" -> "test", + Map(config.EXECUTOR_HOME.key -> "test", "spark.app.name" -> "test", - "spark.mesos.driverEnv.TEST_ENV" -> "TEST_VAL"), + config.DRIVER_ENV_PREFIX + "TEST_ENV" -> "TEST_VAL"), "s1", new Date())) assert(response.success) @@ -232,10 +233,10 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", mem, cpu, true, command, - Map("spark.mesos.executor.home" -> "test", + Map(config.EXECUTOR_HOME.key -> "test", "spark.app.name" -> "test", - "spark.mesos.network.name" -> "test-network-name", - "spark.mesos.network.labels" -> "key1:val1,key2:val2"), + config.NETWORK_NAME.key -> "test-network-name", + config.NETWORK_LABELS.key -> "key1:val1,key2:val2"), "s1", new Date())) @@ -254,6 +255,80 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(networkInfos.get(0).getLabels.getLabels(1).getValue == "val2") } + test("supports setting fetcher cache") { + setScheduler() + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map(config.EXECUTOR_HOME.key -> "test", + config.ENABLE_FETCHER_CACHE.key -> "true", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.asScala.forall(_.getCache)) + } + + test("supports setting fetcher cache on the dispatcher") { + setScheduler(Map(config.ENABLE_FETCHER_CACHE.key -> "true")) + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map(config.EXECUTOR_HOME.key -> "test", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.asScala.forall(_.getCache)) + } + + test("supports disabling fetcher cache") { + setScheduler() + + val mem = 1000 + val cpu = 1 + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", mem, cpu, true, + command, + Map(config.EXECUTOR_HOME.key -> "test", + config.ENABLE_FETCHER_CACHE.key -> "false", + "spark.app.name" -> "test"), + "s1", + new Date())) + + assert(response.success) + + val offer = Utils.createOffer("o1", "s1", mem, cpu) + scheduler.resourceOffers(driver, List(offer).asJava) + + val launchedTasks = Utils.verifyTaskLaunched(driver, "o1") + val uris = launchedTasks.head.getCommand.getUrisList + assert(uris.asScala.forall(!_.getCache)) + } + test("accept/decline offers with driver constraints") { setScheduler() @@ -272,7 +347,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", mem, cpu, true, command, - Map("spark.mesos.executor.home" -> "test", + Map(config.EXECUTOR_HOME.key -> "test", "spark.app.name" -> "test", config.DRIVER_CONSTRAINTS.key -> driverConstraints), "s1", @@ -310,9 +385,9 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", mem, cpu, true, command, - Map("spark.mesos.executor.home" -> "test", + Map(config.EXECUTOR_HOME.key -> "test", "spark.app.name" -> "test", - "spark.mesos.driver.labels" -> "key:value"), + config.DRIVER_LABELS.key -> "key:value"), "s1", new Date())) @@ -336,7 +411,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi val response = scheduler.submitDriver( new MesosDriverDescription("d1", "jar", 100, 1, true, command, - Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")), "s1", new Date())) + Map((config.EXECUTOR_HOME.key, "test"), ("spark.app.name", "test")), "s1", new Date())) assert(response.success) val slaveId = SlaveID.newBuilder().setValue("s1").build() val offer = Offer.newBuilder() @@ -375,6 +450,86 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi assert(state.finishedDrivers.size == 1) } + test("SPARK-27347: do not restart outdated supervised drivers") { + // Covers scenario where: + // - agent goes down + // - supervised job is relaunched on another agent + // - first agent re-registers and sends status update: TASK_FAILED + // - job should NOT be relaunched again + val conf = new SparkConf() + conf.setMaster("mesos://localhost:5050") + conf.setAppName("SparkMesosDriverRetries") + setScheduler(conf.getAll.toMap) + + val mem = 1000 + val cpu = 1 + val offers = List( + Utils.createOffer("o1", "s1", mem, cpu, None), + Utils.createOffer("o2", "s2", mem, cpu, None), + Utils.createOffer("o3", "s1", mem, cpu, None)) + + val response = scheduler.submitDriver( + new MesosDriverDescription("d1", "jar", 100, 1, true, command, + Map(("spark.mesos.executor.home", "test"), ("spark.app.name", "test")), "sub1", new Date())) + assert(response.success) + + // Offer a resource to launch the submitted driver + scheduler.resourceOffers(driver, Collections.singletonList(offers.head)) + var state = scheduler.getSchedulerState() + assert(state.launchedDrivers.size == 1) + + // Signal agent lost with status with TASK_LOST + val agent1 = SlaveID.newBuilder().setValue("s1").build() + var taskStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) + .setSlaveId(agent1) + .setReason(TaskStatus.Reason.REASON_SLAVE_REMOVED) + .setState(MesosTaskState.TASK_LOST) + .build() + + scheduler.statusUpdate(driver, taskStatus) + state = scheduler.getSchedulerState() + assert(state.pendingRetryDrivers.size == 1) + assert(state.pendingRetryDrivers.head.submissionId == taskStatus.getTaskId.getValue) + assert(state.launchedDrivers.isEmpty) + + // Offer new resource to retry driver on a new agent + Thread.sleep(1500) // sleep to cover nextRetry's default wait time of 1s + scheduler.resourceOffers(driver, Collections.singletonList(offers(1))) + + val agent2 = SlaveID.newBuilder().setValue("s2").build() + taskStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) + .setSlaveId(agent2) + .setState(MesosTaskState.TASK_RUNNING) + .build() + + scheduler.statusUpdate(driver, taskStatus) + state = scheduler.getSchedulerState() + assert(state.pendingRetryDrivers.isEmpty) + assert(state.launchedDrivers.size == 1) + assert(state.launchedDrivers.head.taskId.getValue.endsWith("-retry-1")) + assert(state.launchedDrivers.head.taskId.getValue != taskStatus.getTaskId.getValue) + + // Agent1 comes back online and sends status update: TASK_FAILED + taskStatus = TaskStatus.newBuilder() + .setTaskId(TaskID.newBuilder().setValue(response.submissionId).build()) + .setSlaveId(agent1) + .setState(MesosTaskState.TASK_FAILED) + .setMessage("Abnormal executor termination") + .setReason(TaskStatus.Reason.REASON_EXECUTOR_TERMINATED) + .build() + + scheduler.statusUpdate(driver, taskStatus) + scheduler.resourceOffers(driver, Collections.singletonList(offers.last)) + + // Assert driver does not restart 2nd time + state = scheduler.getSchedulerState() + assert(state.pendingRetryDrivers.isEmpty) + assert(state.launchedDrivers.size == 1) + assert(state.launchedDrivers.head.taskId.getValue.endsWith("-retry-1")) + } + test("Declines offer with refuse seconds = 120.") { setScheduler() @@ -411,6 +566,43 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi Utils.verifyFileBasedValueSecrets(launchedTasks) } + test("assembles a valid driver command, escaping all confs and args") { + setScheduler() + + val mem = 1000 + val cpu = 1 + val driverDesc = new MesosDriverDescription( + "d1", + "jar", + mem, + cpu, + true, + new Command( + "Main", + Seq("--a=$2", "--b", "x y z"), + Map(), + Seq(), + Seq(), + Seq()), + Map("spark.app.name" -> "app name", + config.EXECUTOR_URI.key -> "s3a://bucket/spark-version.tgz", + "another.conf" -> "\\value"), + "s1", + new Date()) + + val expectedCmd = "cd spark-version*; " + + "bin/spark-submit --name \"app name\" --master mesos://mesos://localhost:5050 " + + "--driver-cores 1.0 --driver-memory 1000M --class Main --py-files " + + "--conf spark.executor.uri=s3a://bucket/spark-version.tgz " + + "--conf \"another.conf=\\\\value\" " + + "--conf \"spark.app.name=app name\" " + + "../jar " + + "\"--a=\\$2\" " + + "--b \"x y z\"" + + assert(scheduler.getDriverCommandValue(driverDesc) == expectedCmd) + } + private def launchDriverTask(addlSparkConfVars: Map[String, String]): List[TaskInfo] = { setScheduler() val mem = 1000 @@ -422,7 +614,7 @@ class MesosClusterSchedulerSuite extends SparkFunSuite with LocalSparkContext wi cpu, true, command, - Map("spark.mesos.executor.home" -> "test", + Map(config.EXECUTOR_HOME.key -> "test", "spark.app.name" -> "test") ++ addlSparkConfVars, "s1", diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala index da33d85d8fb2e..34b5f6cd19bb1 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosCoarseGrainedSchedulerBackendSuite.scala @@ -24,15 +24,14 @@ import scala.concurrent.duration._ import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos._ -import org.mockito.Matchers -import org.mockito.Matchers._ -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyInt, anyLong, anyString, eq => meq} +import org.mockito.Mockito.{times, verify, when} import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.ScalaFutures import org.scalatest.mockito.MockitoSugar import org.apache.spark.{LocalSparkContext, SecurityManager, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.mesos.config._ +import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.internal.config._ import org.apache.spark.network.shuffle.mesos.MesosExternalShuffleClient import org.apache.spark.rpc.{RpcAddress, RpcEndpointRef} @@ -60,8 +59,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports killing and limiting executors") { setBackend() - sparkConf.set("spark.driver.host", "driverHost") - sparkConf.set("spark.driver.port", "1234") + sparkConf.set(DRIVER_HOST_ADDRESS, "driverHost") + sparkConf.set(DRIVER_PORT, 1234) val minMem = backend.executorMemory(sc) val minCpu = 4 @@ -110,7 +109,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports spark.executor.cores") { val executorCores = 4 - setBackend(Map("spark.executor.cores" -> executorCores.toString)) + setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString)) val executorMemory = backend.executorMemory(sc) val offers = List(Resources(executorMemory * 2, executorCores + 1)) @@ -139,7 +138,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos does not acquire more than spark.cores.max") { val maxCores = 10 - setBackend(Map("spark.cores.max" -> maxCores.toString)) + setBackend(Map(CORES_MAX.key -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List(Resources(executorMemory, maxCores + 1))) @@ -167,7 +166,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos does not acquire more than spark.mesos.gpus.max") { val maxGpus = 5 - setBackend(Map("spark.mesos.gpus.max" -> maxGpus.toString)) + setBackend(Map(mesosConfig.MAX_GPUS.key -> maxGpus.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List(Resources(executorMemory, 1, maxGpus + 1))) @@ -181,14 +180,14 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos declines offers that violate attribute constraints") { - setBackend(Map("spark.mesos.constraints" -> "x:true")) + setBackend(Map(mesosConfig.CONSTRAINTS.key -> "x:true")) offerResources(List(Resources(backend.executorMemory(sc), 4))) verifyDeclinedOffer(driver, createOfferId("o1"), true) } test("mesos declines offers with a filter when reached spark.cores.max") { val maxCores = 3 - setBackend(Map("spark.cores.max" -> maxCores.toString)) + setBackend(Map(CORES_MAX.key -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List( @@ -203,8 +202,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val maxCores = 4 val executorCores = 3 setBackend(Map( - "spark.cores.max" -> maxCores.toString, - "spark.executor.cores" -> executorCores.toString + CORES_MAX.key -> maxCores.toString, + EXECUTOR_CORES.key -> executorCores.toString )) val executorMemory = backend.executorMemory(sc) offerResources(List( @@ -219,8 +218,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite val maxCores = 4 val executorCores = 2 setBackend(Map( - "spark.cores.max" -> maxCores.toString, - "spark.executor.cores" -> executorCores.toString + CORES_MAX.key -> maxCores.toString, + EXECUTOR_CORES.key -> executorCores.toString )) val executorMemory = backend.executorMemory(sc) offerResources(List( @@ -236,8 +235,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos assigns tasks round-robin on offers") { val executorCores = 4 val maxCores = executorCores * 2 - setBackend(Map("spark.executor.cores" -> executorCores.toString, - "spark.cores.max" -> maxCores.toString)) + setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString, + CORES_MAX.key -> maxCores.toString)) val executorMemory = backend.executorMemory(sc) offerResources(List( @@ -250,7 +249,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos creates multiple executors on a single slave") { val executorCores = 4 - setBackend(Map("spark.executor.cores" -> executorCores.toString)) + setBackend(Map(EXECUTOR_CORES.key -> executorCores.toString)) // offer with room for two executors val executorMemory = backend.executorMemory(sc) @@ -371,7 +370,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("failover timeout is set in created scheduler driver") { val failoverTimeoutIn = 3600.0 - initializeSparkConf(Map(DRIVER_FAILOVER_TIMEOUT.key -> failoverTimeoutIn.toString)) + initializeSparkConf(Map(mesosConfig.DRIVER_FAILOVER_TIMEOUT.key -> failoverTimeoutIn.toString)) sc = new SparkContext(sparkConf) val taskScheduler = mock[TaskSchedulerImpl] @@ -405,7 +404,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("honors unset spark.mesos.containerizer") { - setBackend(Map("spark.mesos.executor.docker.image" -> "test")) + setBackend(Map(mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "test")) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -418,8 +417,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("honors spark.mesos.containerizer=\"mesos\"") { setBackend(Map( - "spark.mesos.executor.docker.image" -> "test", - "spark.mesos.containerizer" -> "mesos")) + mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "test", + mesosConfig.CONTAINERIZER.key -> "mesos")) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -432,10 +431,10 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("docker settings are reflected in created tasks") { setBackend(Map( - "spark.mesos.executor.docker.image" -> "some_image", - "spark.mesos.executor.docker.forcePullImage" -> "true", - "spark.mesos.executor.docker.volumes" -> "/host_vol:/container_vol:ro", - "spark.mesos.executor.docker.portmaps" -> "8080:80:tcp" + mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "some_image", + mesosConfig.EXECUTOR_DOCKER_FORCE_PULL_IMAGE.key -> "true", + mesosConfig.EXECUTOR_DOCKER_VOLUMES.key -> "/host_vol:/container_vol:ro", + mesosConfig.EXECUTOR_DOCKER_PORT_MAPS.key -> "8080:80:tcp" )) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -470,7 +469,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("force-pull-image option is disabled by default") { setBackend(Map( - "spark.mesos.executor.docker.image" -> "some_image" + mesosConfig.EXECUTOR_DOCKER_IMAGE.key -> "some_image" )) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -493,7 +492,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports spark.executor.uri") { val url = "spark.spark.spark.com" setBackend(Map( - "spark.executor.uri" -> url + mesosConfig.EXECUTOR_URI.key -> url ), null) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -508,8 +507,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports setting fetcher cache") { val url = "spark.spark.spark.com" setBackend(Map( - "spark.mesos.fetcherCache.enable" -> "true", - "spark.executor.uri" -> url + mesosConfig.ENABLE_FETCHER_CACHE.key -> "true", + mesosConfig.EXECUTOR_URI.key -> url ), null) val offers = List(Resources(backend.executorMemory(sc), 1)) offerResources(offers) @@ -522,8 +521,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports disabling fetcher cache") { val url = "spark.spark.spark.com" setBackend(Map( - "spark.mesos.fetcherCache.enable" -> "false", - "spark.executor.uri" -> url + mesosConfig.ENABLE_FETCHER_CACHE.key -> "false", + mesosConfig.EXECUTOR_URI.key -> url ), null) val offers = List(Resources(backend.executorMemory(sc), 1)) offerResources(offers) @@ -547,7 +546,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos sets configurable labels on tasks") { val taskLabelsString = "mesos:test,label:test" setBackend(Map( - "spark.mesos.task.labels" -> taskLabelsString + mesosConfig.TASK_LABELS.key -> taskLabelsString )) // Build up the labels @@ -569,8 +568,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("mesos supports spark.mesos.network.name and spark.mesos.network.labels") { setBackend(Map( - "spark.mesos.network.name" -> "test-network-name", - "spark.mesos.network.labels" -> "key1:val1,key2:val2" + mesosConfig.NETWORK_NAME.key -> "test-network-name", + mesosConfig.NETWORK_LABELS.key -> "key1:val1,key2:val2" )) val (mem, cpu) = (backend.executorMemory(sc), 4) @@ -591,8 +590,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("supports spark.scheduler.minRegisteredResourcesRatio") { val expectedCores = 1 setBackend(Map( - "spark.cores.max" -> expectedCores.toString, - "spark.scheduler.minRegisteredResourcesRatio" -> "1.0")) + CORES_MAX.key -> expectedCores.toString, + SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO.key -> "1.0")) val offers = List(Resources(backend.executorMemory(sc), expectedCores)) offerResources(offers) @@ -605,9 +604,9 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite test("supports data locality with dynamic allocation") { setBackend(Map( - "spark.dynamicAllocation.enabled" -> "true", - "spark.dynamicAllocation.testing" -> "true", - "spark.locality.wait" -> "1s")) + DYN_ALLOCATION_ENABLED.key -> "true", + DYN_ALLOCATION_TESTING.key -> "true", + LOCALITY_WAIT.key -> "1s")) assert(backend.getExecutorIds().isEmpty) @@ -653,22 +652,26 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite } test("Creates an env-based reference secrets.") { - val launchedTasks = launchExecutorTasks(configEnvBasedRefSecrets(executorSecretConfig)) + val launchedTasks = launchExecutorTasks( + configEnvBasedRefSecrets(mesosConfig.executorSecretConfig)) verifyEnvBasedRefSecrets(launchedTasks) } test("Creates an env-based value secrets.") { - val launchedTasks = launchExecutorTasks(configEnvBasedValueSecrets(executorSecretConfig)) + val launchedTasks = launchExecutorTasks( + configEnvBasedValueSecrets(mesosConfig.executorSecretConfig)) verifyEnvBasedValueSecrets(launchedTasks) } test("Creates file-based reference secrets.") { - val launchedTasks = launchExecutorTasks(configFileBasedRefSecrets(executorSecretConfig)) + val launchedTasks = launchExecutorTasks( + configFileBasedRefSecrets(mesosConfig.executorSecretConfig)) verifyFileBasedRefSecrets(launchedTasks) } test("Creates a file-based value secrets.") { - val launchedTasks = launchExecutorTasks(configFileBasedValueSecrets(executorSecretConfig)) + val launchedTasks = launchExecutorTasks( + configFileBasedValueSecrets(mesosConfig.executorSecretConfig)) verifyFileBasedValueSecrets(launchedTasks) } @@ -688,7 +691,8 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite private def registerMockExecutor(executorId: String, slaveId: String, cores: Integer) = { val mockEndpointRef = mock[RpcEndpointRef] val mockAddress = mock[RpcAddress] - val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty) + val message = RegisterExecutor(executorId, mockEndpointRef, slaveId, cores, Map.empty, + Map.empty, Map.empty) backend.driverEndpoint.askSync[Boolean](message) } @@ -697,9 +701,9 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite offerId: OfferID, filter: Boolean = false): Unit = { if (filter) { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId), anyObject[Filters]) + verify(driver, times(1)).declineOffer(meq(offerId), any[Filters]()) } else { - verify(driver, times(1)).declineOffer(Matchers.eq(offerId)) + verify(driver, times(1)).declineOffer(meq(offerId)) } } @@ -770,7 +774,7 @@ class MesosCoarseGrainedSchedulerBackendSuite extends SparkFunSuite sparkConf = (new SparkConf) .setMaster("local[*]") .setAppName("test-mesos-dynamic-alloc") - .set("spark.mesos.driver.webui.url", "http://webui") + .set(mesosConfig.DRIVER_WEBUI_URL, "http://webui") if (home != null) { sparkConf.setSparkHome(home) 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 1ead4b1ed7c7e..79a57ad031b71 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 @@ -24,19 +24,23 @@ import java.util.Collections import java.util.Properties import scala.collection.JavaConverters._ +import scala.collection.immutable import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import org.apache.mesos.{Protos, Scheduler, SchedulerDriver} import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.Scalar -import org.mockito.{ArgumentCaptor, Matchers} -import org.mockito.Matchers._ +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} import org.mockito.Mockito._ import org.scalatest.mockito.MockitoSugar -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, + SparkFunSuite} +import org.apache.spark.deploy.mesos.config._ import org.apache.spark.executor.MesosExecutorBackend +import org.apache.spark.resource.ResourceInformation import org.apache.spark.scheduler.{LiveListenerBus, SparkListenerExecutorAdded, TaskDescription, TaskSchedulerImpl, WorkerOffer} import org.apache.spark.scheduler.cluster.ExecutorInfo @@ -46,7 +50,7 @@ class MesosFineGrainedSchedulerBackendSuite test("weburi is set in created scheduler driver") { val conf = new SparkConf - conf.set("spark.mesos.driver.webui.url", "http://webui") + conf.set(DRIVER_WEBUI_URL, "http://webui") conf.set("spark.app.name", "name1") val sc = mock[SparkContext] @@ -80,13 +84,14 @@ class MesosFineGrainedSchedulerBackendSuite } test("Use configured mesosExecutor.cores for ExecutorInfo") { - val mesosExecutorCores = 3 + val mesosExecutorCores = 3.0 val conf = new SparkConf - conf.set("spark.mesos.mesosExecutor.cores", mesosExecutorCores.toString) + conf.set(EXECUTOR_CORES, mesosExecutorCores) val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.getSparkHome()).thenReturn(Option("/spark-home")) @@ -113,11 +118,12 @@ class MesosFineGrainedSchedulerBackendSuite test("check spark-class location correctly") { val conf = new SparkConf - conf.set("spark.mesos.executor.home", "/mesos-home") + conf.set(EXECUTOR_HOME, "/mesos-home") val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.getSparkHome()).thenReturn(Option("/spark-home")) @@ -140,7 +146,7 @@ class MesosFineGrainedSchedulerBackendSuite s" /mesos-home/bin/spark-class ${classOf[MesosExecutorBackend].getName}") // uri exists. - conf.set("spark.executor.uri", "hdfs:///test-app-1.0.0.tgz") + conf.set(EXECUTOR_URI, "hdfs:///test-app-1.0.0.tgz") val (executorInfo1, _) = mesosSchedulerBackend.createExecutorInfo(resources, "test-id") assert(executorInfo1.getCommand.getValue === s"cd test-app-1*; ./bin/spark-class ${classOf[MesosExecutorBackend].getName}") @@ -150,14 +156,15 @@ class MesosFineGrainedSchedulerBackendSuite val taskScheduler = mock[TaskSchedulerImpl] val conf = new SparkConf() - .set("spark.mesos.executor.docker.image", "spark/mock") - .set("spark.mesos.executor.docker.forcePullImage", "true") - .set("spark.mesos.executor.docker.volumes", "/a,/b:/b,/c:/c:rw,/d:ro,/e:/e:ro") - .set("spark.mesos.executor.docker.portmaps", "80:8080,53:53:tcp") + .set(EXECUTOR_DOCKER_IMAGE, "spark/mock") + .set(EXECUTOR_DOCKER_FORCE_PULL_IMAGE, true) + .set(EXECUTOR_DOCKER_VOLUMES, Seq("/a", "/b:/b", "/c:/c:rw", "/d:ro", "/e:/e:ro")) + .set(EXECUTOR_DOCKER_PORT_MAPS, Seq("80:8080", "53:53:tcp")) val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -217,7 +224,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -257,6 +265,7 @@ class MesosFineGrainedSchedulerBackendSuite addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), + resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(2) @@ -264,7 +273,7 @@ class MesosFineGrainedSchedulerBackendSuite val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -275,7 +284,7 @@ class MesosFineGrainedSchedulerBackendSuite backend.resourceOffers(driver, mesosOffers) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -308,7 +317,8 @@ class MesosFineGrainedSchedulerBackendSuite val listenerBus = mock[LiveListenerBus] listenerBus.post( - SparkListenerExecutorAdded(anyLong, "s1", new ExecutorInfo("host1", 2, Map.empty))) + SparkListenerExecutorAdded(anyLong, "s1", + new ExecutorInfo("host1", 2, Map.empty, Map.empty))) val sc = mock[SparkContext] when(sc.executorMemory).thenReturn(100) @@ -366,6 +376,7 @@ class MesosFineGrainedSchedulerBackendSuite addedFiles = mutable.Map.empty[String, Long], addedJars = mutable.Map.empty[String, Long], properties = new Properties(), + resources = immutable.Map.empty[String, ResourceInformation], ByteBuffer.wrap(new Array[Byte](0))) when(taskScheduler.resourceOffers(expectedWorkerOffers)).thenReturn(Seq(Seq(taskDesc))) when(taskScheduler.CPUS_PER_TASK).thenReturn(1) @@ -373,7 +384,7 @@ class MesosFineGrainedSchedulerBackendSuite val capture = ArgumentCaptor.forClass(classOf[Collection[TaskInfo]]) when( driver.launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) @@ -382,7 +393,7 @@ class MesosFineGrainedSchedulerBackendSuite backend.resourceOffers(driver, mesosOffers) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(mesosOffers.get(0).getId)), + meq(Collections.singleton(mesosOffers.get(0).getId)), capture.capture(), any(classOf[Filters]) ) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala index 442c43960ec1f..6b7ae900768ef 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackendUtilSuite.scala @@ -24,8 +24,8 @@ class MesosSchedulerBackendUtilSuite extends SparkFunSuite { test("ContainerInfo fails to parse invalid docker parameters") { val conf = new SparkConf() - conf.set("spark.mesos.executor.docker.parameters", "a,b") - conf.set("spark.mesos.executor.docker.image", "test") + conf.set(config.EXECUTOR_DOCKER_PARAMETERS, Seq("a", "b")) + conf.set(config.EXECUTOR_DOCKER_IMAGE, "test") val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo( conf) @@ -36,8 +36,8 @@ class MesosSchedulerBackendUtilSuite extends SparkFunSuite { test("ContainerInfo parses docker parameters") { val conf = new SparkConf() - conf.set("spark.mesos.executor.docker.parameters", "a=1,b=2,c=3") - conf.set("spark.mesos.executor.docker.image", "test") + conf.set(config.EXECUTOR_DOCKER_PARAMETERS, Seq("a=1", "b=2", "c=3")) + conf.set(config.EXECUTOR_DOCKER_IMAGE, "test") val containerInfo = MesosSchedulerBackendUtil.buildContainerInfo( conf) diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala index 8d90e1a8591ad..6a9ebfb8a4fae 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerUtilsSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.scheduler.cluster.mesos import java.io.{File, FileNotFoundException} import scala.collection.JavaConverters._ -import scala.language.reflectiveCalls import com.google.common.io.Files import org.apache.mesos.Protos.{FrameworkInfo, Resource, Value} @@ -29,19 +28,20 @@ import org.scalatest._ import org.scalatest.mockito.MockitoSugar import org.apache.spark.{SparkConf, SparkContext, SparkException, SparkFunSuite} +import org.apache.spark.deploy.mesos.{config => mesosConfig} import org.apache.spark.internal.config._ import org.apache.spark.util.SparkConfWithEnv class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoSugar { - // scalastyle:off structural.type - // this is the documented way of generating fixtures in scalatest - def fixture: Object {val sc: SparkContext; val sparkConf: SparkConf} = new { + class SparkConfFixture { val sparkConf = new SparkConf - val sc = mock[SparkContext] + val sc: SparkContext = mock[SparkContext] when(sc.conf).thenReturn(sparkConf) } + def fixture: SparkConfFixture = new SparkConfFixture() + private def createTestPortResource(range: (Long, Long), role: Option[String] = None): Resource = { val rangeValue = Value.Range.newBuilder() rangeValue.setBegin(range._1) @@ -77,7 +77,6 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } val utils = new MesosSchedulerUtils { } - // scalastyle:on structural.type test("use at-least minimum overhead") { val f = fixture @@ -94,7 +93,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("use spark.mesos.executor.memoryOverhead (if set)") { val f = fixture when(f.sc.executorMemory).thenReturn(1024) - f.sparkConf.set("spark.mesos.executor.memoryOverhead", "512") + f.sparkConf.set(mesosConfig.EXECUTOR_MEMORY_OVERHEAD, 512) utils.executorMemory(f.sc) shouldBe 1536 } @@ -215,7 +214,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Port reservation is done correctly with user specified ports only - multiple ranges") { val conf = new SparkConf() - conf.set("spark.blockManager.port", "4000") + conf.set(BLOCK_MANAGER_PORT, 4000) val portResourceList = List(createTestPortResource((3000, 5000), Some("my_role")), createTestPortResource((2000, 2500), Some("other_role"))) val (resourcesLeft, resourcesToBeUsed) = utils @@ -244,7 +243,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Principal specified via spark.mesos.principal") { val conf = new SparkConf() - conf.set("spark.mesos.principal", "test-principal") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -252,11 +251,11 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } test("Principal specified via spark.mesos.principal.file") { - val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt"); + val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") pFile.deleteOnExit() - Files.write("test-principal".getBytes("UTF-8"), pFile); + Files.write("test-principal".getBytes("UTF-8"), pFile) val conf = new SparkConf() - conf.set("spark.mesos.principal.file", pFile.getAbsolutePath()) + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL_FILE, pFile.getAbsolutePath()) val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -265,7 +264,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Principal specified via spark.mesos.principal.file that does not exist") { val conf = new SparkConf() - conf.set("spark.mesos.principal.file", "/tmp/does-not-exist") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL_FILE, "/tmp/does-not-exist") intercept[FileNotFoundException] { utils.buildCredentials(conf, FrameworkInfo.newBuilder()) @@ -281,9 +280,9 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } test("Principal specified via SPARK_MESOS_PRINCIPAL_FILE") { - val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt"); + val pFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") pFile.deleteOnExit() - Files.write("test-principal".getBytes("UTF-8"), pFile); + Files.write("test-principal".getBytes("UTF-8"), pFile) val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL_FILE" -> pFile.getAbsolutePath())) val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) @@ -301,8 +300,8 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Secret specified via spark.mesos.secret") { val conf = new SparkConf() - conf.set("spark.mesos.principal", "test-principal") - conf.set("spark.mesos.secret", "my-secret") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") + conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -312,12 +311,12 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } test("Principal specified via spark.mesos.secret.file") { - val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt"); + val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") sFile.deleteOnExit() - Files.write("my-secret".getBytes("UTF-8"), sFile); + Files.write("my-secret".getBytes("UTF-8"), sFile) val conf = new SparkConf() - conf.set("spark.mesos.principal", "test-principal") - conf.set("spark.mesos.secret.file", sFile.getAbsolutePath()) + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") + conf.set(mesosConfig.CREDENTIAL_SECRET_FILE, sFile.getAbsolutePath()) val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -328,8 +327,8 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Principal specified via spark.mesos.secret.file that does not exist") { val conf = new SparkConf() - conf.set("spark.mesos.principal", "test-principal") - conf.set("spark.mesos.secret.file", "/tmp/does-not-exist") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") + conf.set(mesosConfig.CREDENTIAL_SECRET_FILE, "/tmp/does-not-exist") intercept[FileNotFoundException] { utils.buildCredentials(conf, FrameworkInfo.newBuilder()) @@ -339,7 +338,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Principal specified via SPARK_MESOS_SECRET") { val env = Map("SPARK_MESOS_SECRET" -> "my-secret") val conf = new SparkConfWithEnv(env) - conf.set("spark.mesos.principal", "test-principal") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -349,14 +348,14 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS } test("Principal specified via SPARK_MESOS_SECRET_FILE") { - val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt"); + val sFile = File.createTempFile("MesosSchedulerUtilsSuite", ".txt") sFile.deleteOnExit() - Files.write("my-secret".getBytes("UTF-8"), sFile); + Files.write("my-secret".getBytes("UTF-8"), sFile) val sFilePath = sFile.getAbsolutePath() val env = Map("SPARK_MESOS_SECRET_FILE" -> sFilePath) val conf = new SparkConfWithEnv(env) - conf.set("spark.mesos.principal", "test-principal") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -367,7 +366,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Secret specified with no principal") { val conf = new SparkConf() - conf.set("spark.mesos.secret", "my-secret") + conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") intercept[SparkException] { utils.buildCredentials(conf, FrameworkInfo.newBuilder()) @@ -376,7 +375,7 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Principal specification preference") { val conf = new SparkConfWithEnv(Map("SPARK_MESOS_PRINCIPAL" -> "other-principal")) - conf.set("spark.mesos.principal", "test-principal") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true @@ -385,8 +384,8 @@ class MesosSchedulerUtilsSuite extends SparkFunSuite with Matchers with MockitoS test("Secret specification preference") { val conf = new SparkConfWithEnv(Map("SPARK_MESOS_SECRET" -> "other-secret")) - conf.set("spark.mesos.principal", "test-principal") - conf.set("spark.mesos.secret", "my-secret") + conf.set(mesosConfig.CREDENTIAL_PRINCIPAL, "test-principal") + conf.set(mesosConfig.CREDENTIAL_SECRET, "my-secret") val credBuilder = utils.buildCredentials(conf, FrameworkInfo.newBuilder()) credBuilder.hasPrincipal shouldBe true diff --git a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala index c9f47471cd75e..65e595e3cf2bf 100644 --- a/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala +++ b/resource-managers/mesos/src/test/scala/org/apache/spark/scheduler/cluster/mesos/Utils.scala @@ -25,8 +25,9 @@ import org.apache.mesos.Protos._ import org.apache.mesos.Protos.Value.{Range => MesosRange, Ranges, Scalar} import org.apache.mesos.SchedulerDriver import org.apache.mesos.protobuf.ByteString -import org.mockito.{ArgumentCaptor, Matchers} -import org.mockito.Mockito._ +import org.mockito.ArgumentCaptor +import org.mockito.ArgumentMatchers.{any, eq => meq} +import org.mockito.Mockito.{times, verify} import org.apache.spark.deploy.mesos.config.MesosSecretConfig @@ -84,15 +85,15 @@ object Utils { def verifyTaskLaunched(driver: SchedulerDriver, offerId: String): List[TaskInfo] = { val captor = ArgumentCaptor.forClass(classOf[java.util.Collection[TaskInfo]]) verify(driver, times(1)).launchTasks( - Matchers.eq(Collections.singleton(createOfferId(offerId))), + meq(Collections.singleton(createOfferId(offerId))), captor.capture()) captor.getValue.asScala.toList } def verifyTaskNotLaunched(driver: SchedulerDriver, offerId: String): Unit = { verify(driver, times(0)).launchTasks( - Matchers.eq(Collections.singleton(createOfferId(offerId))), - Matchers.any(classOf[java.util.Collection[TaskInfo]])) + meq(Collections.singleton(createOfferId(offerId))), + any(classOf[java.util.Collection[TaskInfo]])) } def createOfferId(offerId: String): OfferID = { diff --git a/resource-managers/yarn/pom.xml b/resource-managers/yarn/pom.xml index e55b814be8465..0e5df14e060da 100644 --- a/resource-managers/yarn/pom.xml +++ b/resource-managers/yarn/pom.xml @@ -19,17 +19,17 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-yarn_2.11 + spark-yarn_2.12 jar Spark Project YARN yarn - 1.9 + 1.19 @@ -166,12 +166,19 @@ test ${jersey-1.version} + + com.sun.jersey + jersey-servlet + test + ${jersey-1.version} + ${hive.group} hive-exec + ${hive.classifier} provided diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 8f94e3f731007..5a67caf8e2a60 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -30,21 +30,24 @@ import scala.util.control.NonFatal import org.apache.commons.lang3.{StringUtils => ComStrUtils} import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationAttemptNotFoundException +import org.apache.hadoop.yarn.server.webproxy.ProxyUriUtils import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.history.HistoryServer import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.AMCredentialRenewer import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.internal.config.Streaming.STREAMING_DYN_ALLOCATION_MAX_EXECUTORS +import org.apache.spark.internal.config.UI._ +import org.apache.spark.metrics.{MetricsSystem, MetricsSystemInstances} import org.apache.spark.rpc._ import org.apache.spark.scheduler.cluster.{CoarseGrainedSchedulerBackend, YarnSchedulerBackend} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -53,35 +56,27 @@ import org.apache.spark.util._ /** * Common application master functionality for Spark on Yarn. */ -private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends Logging { +private[spark] class ApplicationMaster( + args: ApplicationMasterArguments, + sparkConf: SparkConf, + yarnConf: YarnConfiguration) extends Logging { // TODO: Currently, task to container is computed once (TaskSetManager) - which need not be // optimal as more containers are available. Might need to handle this better. - private val isClusterMode = args.userClass != null - - private val sparkConf = new SparkConf() - if (args.propertiesFile != null) { - Utils.getPropertiesFromFile(args.propertiesFile).foreach { case (k, v) => - sparkConf.set(k, v) + private val appAttemptId = + if (System.getenv(ApplicationConstants.Environment.CONTAINER_ID.name()) != null) { + YarnSparkHadoopUtil.getContainerId.getApplicationAttemptId() + } else { + null } - } + + private val isClusterMode = args.userClass != null private val securityMgr = new SecurityManager(sparkConf) private var metricsSystem: Option[MetricsSystem] = None - // Set system properties for each config entry. This covers two use cases: - // - The default configuration stored by the SparkHadoopUtil class - // - The user application creating a new SparkConf in cluster mode - // - // Both cases create a new SparkConf object which reads these configs from system properties. - sparkConf.getAll.foreach { case (k, v) => - sys.props(k) = v - } - - private val yarnConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) - private val userClassLoader = { val classpath = Client.getUserClasspath(sparkConf) val urls = classpath.map { entry => @@ -99,34 +94,16 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } } - private val credentialRenewer: Option[AMCredentialRenewer] = sparkConf.get(KEYTAB).map { _ => - new AMCredentialRenewer(sparkConf, yarnConf) - } - - private val ugi = credentialRenewer match { - case Some(cr) => - // Set the context class loader so that the token renewer has access to jars distributed - // by the user. - val currentLoader = Thread.currentThread().getContextClassLoader() - Thread.currentThread().setContextClassLoader(userClassLoader) - try { - cr.start() - } finally { - Thread.currentThread().setContextClassLoader(currentLoader) - } - - case _ => - SparkHadoopUtil.get.createSparkUser() - } - - private val client = doAsUser { new YarnRMClient() } + private val client = new YarnRMClient() // Default to twice the number of executors (twice the maximum number of executors if dynamic // allocation is enabled), with a minimum of 3. private val maxNumExecutorFailures = { val effectiveNumExecutors = - if (Utils.isDynamicAllocationEnabled(sparkConf)) { + if (Utils.isStreamingDynamicAllocationEnabled(sparkConf)) { + sparkConf.get(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS) + } else if (Utils.isDynamicAllocationEnabled(sparkConf)) { sparkConf.get(DYN_ALLOCATION_MAX_EXECUTORS) } else { sparkConf.get(EXECUTOR_INSTANCES).getOrElse(0) @@ -171,15 +148,14 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // Next wait interval before allocator poll. private var nextAllocationInterval = initialAllocationInterval - private var rpcEnv: RpcEnv = null - // In cluster mode, used to tell the AM when the user's SparkContext has been initialized. private val sparkContextPromise = Promise[SparkContext]() - // Load the list of localized files set by the client. This is used when launching executors, - // and is loaded here so that these configs don't pollute the Web UI's environment page in - // cluster mode. - private val localResources = doAsUser { + /** + * Load the list of localized files set by the client, used when launching executors. This should + * be called in a context where the needed credentials to access HDFS are available. + */ + private def prepareLocalResources(distCacheConf: SparkConf): Map[String, LocalResource] = { logInfo("Preparing Local resources") val resources = HashMap[String, LocalResource]() @@ -201,11 +177,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends resources(fileName) = amJarRsrc } - val distFiles = sparkConf.get(CACHED_FILES) - val fileSizes = sparkConf.get(CACHED_FILES_SIZES) - val timeStamps = sparkConf.get(CACHED_FILES_TIMESTAMPS) - val visibilities = sparkConf.get(CACHED_FILES_VISIBILITIES) - val resTypes = sparkConf.get(CACHED_FILES_TYPES) + val distFiles = distCacheConf.get(CACHED_FILES) + val fileSizes = distCacheConf.get(CACHED_FILES_SIZES) + val timeStamps = distCacheConf.get(CACHED_FILES_TIMESTAMPS) + val visibilities = distCacheConf.get(CACHED_FILES_VISIBILITIES) + val resTypes = distCacheConf.get(CACHED_FILES_TYPES) for (i <- 0 to distFiles.size - 1) { val resType = LocalResourceType.valueOf(resTypes(i)) @@ -214,7 +190,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } // Distribute the conf archive to executors. - sparkConf.get(CACHED_CONF_ARCHIVE).foreach { path => + distCacheConf.get(CACHED_CONF_ARCHIVE).foreach { path => val uri = new URI(path) val fs = FileSystem.get(uri, yarnConf) val status = fs.getFileStatus(new Path(uri)) @@ -227,46 +203,27 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends LocalResourceVisibility.PRIVATE.name()) } - // Clean up the configuration so it doesn't show up in the Web UI (since it's really noisy). - CACHE_CONFIGS.foreach { e => - sparkConf.remove(e) - sys.props.remove(e.key) - } - resources.toMap } - def getAttemptId(): ApplicationAttemptId = { - client.getAttemptId() - } - final def run(): Int = { - doAsUser { - runImpl() - } - exitCode - } - - private def runImpl(): Unit = { try { - val appAttemptId = client.getAttemptId() - - var attemptID: Option[String] = None - - if (isClusterMode) { + val attemptID = if (isClusterMode) { // Set the web ui port to be ephemeral for yarn so we don't conflict with // other spark processes running on the same box - System.setProperty("spark.ui.port", "0") + System.setProperty(UI_PORT.key, "0") // Set the master and deploy mode property to match the requested mode. System.setProperty("spark.master", "yarn") - System.setProperty("spark.submit.deployMode", "cluster") + System.setProperty(SUBMIT_DEPLOY_MODE.key, "cluster") // Set this internal configuration if it is running on cluster mode, this // configuration will be checked in SparkContext to avoid misuse of yarn cluster mode. System.setProperty("spark.yarn.app.id", appAttemptId.getApplicationId().toString()) - attemptID = Option(appAttemptId.getAttemptId.toString) + Option(appAttemptId.getAttemptId.toString) + } else { + None } new CallerContext( @@ -279,7 +236,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val priority = ShutdownHookManager.SPARK_CONTEXT_SHUTDOWN_PRIORITY - 1 ShutdownHookManager.addShutdownHook(priority) { () => val maxAppAttempts = client.getMaxRegAttempts(sparkConf, yarnConf) - val isLastAttempt = client.getAttemptId().getAttemptId() >= maxAppAttempts + val isLastAttempt = appAttemptId.getAttemptId() >= maxAppAttempts if (!finished) { // The default state of ApplicationMaster is failed if it is invoked by shut down hook. @@ -296,7 +253,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends // we only want to unregister if we don't want the RM to retry if (finalStatus == FinalApplicationStatus.SUCCEEDED || isLastAttempt) { unregister(finalStatus, finalMsg) - cleanupStagingDir() + cleanupStagingDir(new Path(System.getenv("SPARK_YARN_STAGING_DIR"))) } } } @@ -324,6 +281,62 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends logWarning("Exception during stopping of the metric system: ", e) } } + + exitCode + } + + def runUnmanaged( + clientRpcEnv: RpcEnv, + appAttemptId: ApplicationAttemptId, + stagingDir: Path, + cachedResourcesConf: SparkConf): Unit = { + try { + new CallerContext( + "APPMASTER", sparkConf.get(APP_CALLER_CONTEXT), + Option(appAttemptId.getApplicationId.toString), None).setCurrentContext() + + val driverRef = clientRpcEnv.setupEndpointRef( + RpcAddress(sparkConf.get(DRIVER_HOST_ADDRESS), + sparkConf.get(DRIVER_PORT)), + YarnSchedulerBackend.ENDPOINT_NAME) + // The client-mode AM doesn't listen for incoming connections, so report an invalid port. + registerAM(Utils.localHostName, -1, sparkConf, + sparkConf.getOption("spark.driver.appUIAddress"), appAttemptId) + addAmIpFilter(Some(driverRef), ProxyUriUtils.getPath(appAttemptId.getApplicationId)) + createAllocator(driverRef, sparkConf, clientRpcEnv, appAttemptId, cachedResourcesConf) + reporterThread.join() + } catch { + case e: Exception => + // catch everything else if not specifically handled + logError("Uncaught exception: ", e) + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_UNCAUGHT_EXCEPTION, + "Uncaught exception: " + StringUtils.stringifyException(e)) + if (!unregistered) { + unregister(finalStatus, finalMsg) + cleanupStagingDir(stagingDir) + } + } finally { + try { + metricsSystem.foreach { ms => + ms.report() + ms.stop() + } + } catch { + case e: Exception => + logWarning("Exception during stopping of the metric system: ", e) + } + } + } + + def stopUnmanaged(stagingDir: Path): Unit = { + if (!finished) { + finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) + } + if (!unregistered) { + unregister(finalStatus, finalMsg) + cleanupStagingDir(stagingDir) + } } /** @@ -379,9 +392,6 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends logDebug("shutting down user thread") userClassThread.interrupt() } - if (!inShutdown) { - credentialRenewer.foreach(_.stop()) - } } } } @@ -406,9 +416,10 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends host: String, port: Int, _sparkConf: SparkConf, - uiAddress: Option[String]): Unit = { - val appId = client.getAttemptId().getApplicationId().toString() - val attemptId = client.getAttemptId().getAttemptId().toString() + uiAddress: Option[String], + appAttempt: ApplicationAttemptId): Unit = { + val appId = appAttempt.getApplicationId().toString() + val attemptId = appAttempt.getAttemptId().toString() val historyAddress = ApplicationMaster .getHistoryServerAddress(_sparkConf, yarnConf, appId, attemptId) @@ -416,10 +427,26 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends registered = true } - private def createAllocator(driverRef: RpcEndpointRef, _sparkConf: SparkConf): Unit = { - val appId = client.getAttemptId().getApplicationId().toString() + private def createAllocator( + driverRef: RpcEndpointRef, + _sparkConf: SparkConf, + rpcEnv: RpcEnv, + appAttemptId: ApplicationAttemptId, + distCacheConf: SparkConf): Unit = { + // In client mode, the AM may be restarting after delegation tokens have reached their TTL. So + // always contact the driver to get the current set of valid tokens, so that local resources can + // be initialized below. + if (!isClusterMode) { + val tokens = driverRef.askSync[Array[Byte]](RetrieveDelegationTokens) + if (tokens != null) { + SparkHadoopUtil.get.addDelegationTokens(tokens, _sparkConf) + } + } + + val appId = appAttemptId.getApplicationId().toString() val driverUrl = RpcEndpointAddress(driverRef.address.host, driverRef.address.port, CoarseGrainedSchedulerBackend.ENDPOINT_NAME).toString + val localResources = prepareLocalResources(distCacheConf) // Before we initialize the allocator, let's log the information about how executors will // be run up front, to avoid printing this out for every single executor being launched. @@ -435,29 +462,30 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends allocator = client.createAllocator( yarnConf, _sparkConf, + appAttemptId, driverUrl, driverRef, securityMgr, localResources) - credentialRenewer.foreach(_.setDriverRef(driverRef)) - // Initialize the AM endpoint *after* the allocator has been initialized. This ensures // that when the driver sends an initial executor request (e.g. after an AM restart), // the allocator is ready to service requests. rpcEnv.setupEndpoint("YarnAM", new AMEndpoint(rpcEnv, driverRef)) allocator.allocateResources() - val ms = MetricsSystem.createMetricsSystem("applicationMaster", sparkConf, securityMgr) + val ms = MetricsSystem.createMetricsSystem(MetricsSystemInstances.APPLICATION_MASTER, + sparkConf, securityMgr) val prefix = _sparkConf.get(YARN_METRICS_NAMESPACE).getOrElse(appId) ms.registerSource(new ApplicationMasterSource(prefix, allocator)) - ms.start() + // do not register static sources in this case as per SPARK-25277 + ms.start(false) metricsSystem = Some(ms) reporterThread = launchReporterThread() } private def runDriver(): Unit = { - addAmIpFilter(None) + addAmIpFilter(None, System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)) userClassThread = startUserApplication() // This a bit hacky, but we need to wait until the spark.driver.port property has @@ -468,17 +496,17 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val sc = ThreadUtils.awaitResult(sparkContextPromise.future, Duration(totalWaitTime, TimeUnit.MILLISECONDS)) if (sc != null) { - rpcEnv = sc.env.rpcEnv + val rpcEnv = sc.env.rpcEnv val userConf = sc.getConf - val host = userConf.get("spark.driver.host") - val port = userConf.get("spark.driver.port").toInt - registerAM(host, port, userConf, sc.ui.map(_.webUrl)) + val host = userConf.get(DRIVER_HOST_ADDRESS) + val port = userConf.get(DRIVER_PORT) + registerAM(host, port, userConf, sc.ui.map(_.webUrl), appAttemptId) val driverRef = rpcEnv.setupEndpointRef( RpcAddress(host, port), YarnSchedulerBackend.ENDPOINT_NAME) - createAllocator(driverRef, userConf) + createAllocator(driverRef, userConf, rpcEnv, appAttemptId, distCacheConf) } else { // Sanity check; should never happen in normal operation, since sc should only be null // if the user app did not create a SparkContext. @@ -502,11 +530,11 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends private def runExecutorLauncher(): Unit = { val hostname = Utils.localHostName val amCores = sparkConf.get(AM_CORES) - rpcEnv = RpcEnv.create("sparkYarnAM", hostname, hostname, -1, sparkConf, securityMgr, + val rpcEnv = RpcEnv.create("sparkYarnAM", hostname, hostname, -1, sparkConf, securityMgr, amCores, true) // The client-mode AM doesn't listen for incoming connections, so report an invalid port. - registerAM(hostname, -1, sparkConf, sparkConf.getOption("spark.driver.appUIAddress")) + registerAM(hostname, -1, sparkConf, sparkConf.get(DRIVER_APP_UI_ADDRESS), appAttemptId) // The driver should be up and listening, so unlike cluster mode, just try to connect to it // with no waiting or retrying. @@ -514,95 +542,102 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends val driverRef = rpcEnv.setupEndpointRef( RpcAddress(driverHost, driverPort), YarnSchedulerBackend.ENDPOINT_NAME) - addAmIpFilter(Some(driverRef)) - createAllocator(driverRef, sparkConf) + addAmIpFilter(Some(driverRef), + System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV)) + createAllocator(driverRef, sparkConf, rpcEnv, appAttemptId, distCacheConf) // In client mode the actor will stop the reporter thread. reporterThread.join() } - private def launchReporterThread(): Thread = { - // The number of failures in a row until Reporter thread give up + private def allocationThreadImpl(): Unit = { + // The number of failures in a row until the allocation thread gives up. val reporterMaxFailures = sparkConf.get(MAX_REPORTER_THREAD_FAILURES) - - val t = new Thread { - override def run() { - var failureCount = 0 - while (!finished) { - try { - if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, - s"Max number of executor failures ($maxNumExecutorFailures) reached") - } else if (allocator.isAllNodeBlacklisted) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, - "Due to executor failures all available nodes are blacklisted") - } else { - logDebug("Sending progress") - allocator.allocateResources() - } - failureCount = 0 - } catch { - case i: InterruptedException => // do nothing - case e: ApplicationAttemptNotFoundException => - failureCount += 1 - logError("Exception from Reporter thread.", e) - finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE, - e.getMessage) - case e: Throwable => - failureCount += 1 - if (!NonFatal(e) || failureCount >= reporterMaxFailures) { - finish(FinalApplicationStatus.FAILED, - ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + - s"$failureCount time(s) from Reporter thread.") - } else { - logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e) - } + var failureCount = 0 + while (!finished) { + try { + if (allocator.getNumExecutorsFailed >= maxNumExecutorFailures) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, + s"Max number of executor failures ($maxNumExecutorFailures) reached") + } else if (allocator.isAllNodeBlacklisted) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_MAX_EXECUTOR_FAILURES, + "Due to executor failures all available nodes are blacklisted") + } else { + logDebug("Sending progress") + allocator.allocateResources() + } + failureCount = 0 + } catch { + case i: InterruptedException => // do nothing + case e: ApplicationAttemptNotFoundException => + failureCount += 1 + logError("Exception from Reporter thread.", e) + finish(FinalApplicationStatus.FAILED, ApplicationMaster.EXIT_REPORTER_FAILURE, + e.getMessage) + case e: Throwable => + failureCount += 1 + if (!NonFatal(e) || failureCount >= reporterMaxFailures) { + finish(FinalApplicationStatus.FAILED, + ApplicationMaster.EXIT_REPORTER_FAILURE, "Exception was thrown " + + s"$failureCount time(s) from Reporter thread.") + } else { + logWarning(s"Reporter thread fails $failureCount time(s) in a row.", e) } - try { - val numPendingAllocate = allocator.getPendingAllocate.size - var sleepStart = 0L - var sleepInterval = 200L // ms - allocatorLock.synchronized { - sleepInterval = - if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) { - val currentAllocationInterval = - math.min(heartbeatInterval, nextAllocationInterval) - nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow - currentAllocationInterval - } else { - nextAllocationInterval = initialAllocationInterval - heartbeatInterval - } - sleepStart = System.currentTimeMillis() - allocatorLock.wait(sleepInterval) - } - val sleepDuration = System.currentTimeMillis() - sleepStart - if (sleepDuration < sleepInterval) { - // log when sleep is interrupted - logDebug(s"Number of pending allocations is $numPendingAllocate. " + - s"Slept for $sleepDuration/$sleepInterval ms.") - // if sleep was less than the minimum interval, sleep for the rest of it - val toSleep = math.max(0, initialAllocationInterval - sleepDuration) - if (toSleep > 0) { - logDebug(s"Going back to sleep for $toSleep ms") - // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up - // by the methods that signal allocatorLock because this is just finishing the min - // sleep interval, which should happen even if this is signalled again. - Thread.sleep(toSleep) - } + } + try { + val numPendingAllocate = allocator.getPendingAllocate.size + var sleepStartNs = 0L + var sleepInterval = 200L // ms + allocatorLock.synchronized { + sleepInterval = + if (numPendingAllocate > 0 || allocator.getNumPendingLossReasonRequests > 0) { + val currentAllocationInterval = + math.min(heartbeatInterval, nextAllocationInterval) + nextAllocationInterval = currentAllocationInterval * 2 // avoid overflow + currentAllocationInterval } else { - logDebug(s"Number of pending allocations is $numPendingAllocate. " + - s"Slept for $sleepDuration/$sleepInterval.") + nextAllocationInterval = initialAllocationInterval + heartbeatInterval } - } catch { - case e: InterruptedException => + sleepStartNs = System.nanoTime() + allocatorLock.wait(sleepInterval) + } + val sleepDuration = System.nanoTime() - sleepStartNs + if (sleepDuration < TimeUnit.MILLISECONDS.toNanos(sleepInterval)) { + // log when sleep is interrupted + logDebug(s"Number of pending allocations is $numPendingAllocate. " + + s"Slept for $sleepDuration/$sleepInterval ms.") + // if sleep was less than the minimum interval, sleep for the rest of it + val toSleep = math.max(0, initialAllocationInterval - sleepDuration) + if (toSleep > 0) { + logDebug(s"Going back to sleep for $toSleep ms") + // use Thread.sleep instead of allocatorLock.wait. there is no need to be woken up + // by the methods that signal allocatorLock because this is just finishing the min + // sleep interval, which should happen even if this is signalled again. + Thread.sleep(toSleep) } + } else { + logDebug(s"Number of pending allocations is $numPendingAllocate. " + + s"Slept for $sleepDuration/$sleepInterval.") + } + } catch { + case e: InterruptedException => + } + } + } + + private def launchReporterThread(): Thread = { + val t = new Thread { + override def run(): Unit = { + try { + allocationThreadImpl() + } finally { + allocator.stop() } } } - // setting to daemon status, though this is usually not a good idea. t.setDaemon(true) t.setName("Reporter") t.start() @@ -611,15 +646,23 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends t } + private def distCacheConf(): SparkConf = { + val distCacheConf = new SparkConf(false) + if (args.distCacheConf != null) { + Utils.getPropertiesFromFile(args.distCacheConf).foreach { case (k, v) => + distCacheConf.set(k, v) + } + } + distCacheConf + } + /** * Clean up the staging directory. */ - private def cleanupStagingDir(): Unit = { - var stagingDirPath: Path = null + private def cleanupStagingDir(stagingDirPath: Path): Unit = { try { val preserveFiles = sparkConf.get(PRESERVE_STAGING_FILES) if (!preserveFiles) { - stagingDirPath = new Path(System.getenv("SPARK_YARN_STAGING_DIR")) logInfo("Deleting staging directory " + stagingDirPath) val fs = stagingDirPath.getFileSystem(yarnConf) fs.delete(stagingDirPath, true) @@ -631,8 +674,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends } /** Add the Yarn IP filter that is required for properly securing the UI. */ - private def addAmIpFilter(driver: Option[RpcEndpointRef]) = { - val proxyBase = System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + private def addAmIpFilter(driver: Option[RpcEndpointRef], proxyBase: String) = { val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" val params = client.getAmIpFilterParams(yarnConf, proxyBase) driver match { @@ -640,7 +682,7 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends d.send(AddWebUIFilter(amFilter, params.toMap, proxyBase)) case None => - System.setProperty("spark.ui.filters", amFilter) + System.setProperty(UI_FILTERS.key, amFilter) params.foreach { case (k, v) => System.setProperty(s"spark.$amFilter.param.$k", v) } } } @@ -756,24 +798,21 @@ private[spark] class ApplicationMaster(args: ApplicationMasterArguments) extends case None => logWarning("Container allocator is not ready to find executor loss reasons yet.") } + + case UpdateDelegationTokens(tokens) => + SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { - // In cluster mode, do not rely on the disassociated event to exit + // In cluster mode or unmanaged am case, do not rely on the disassociated event to exit // This avoids potentially reporting incorrect exit codes if the driver fails - if (!isClusterMode) { + if (!(isClusterMode || sparkConf.get(YARN_UNMANAGED_AM))) { logInfo(s"Driver terminated or disconnected! Shutting down. $remoteAddress") finish(FinalApplicationStatus.SUCCEEDED, ApplicationMaster.EXIT_SUCCESS) } } } - private def doAsUser[T](fn: => T): T = { - ugi.doAs(new PrivilegedExceptionAction[T]() { - override def run: T = fn - }) - } - } object ApplicationMaster extends Logging { @@ -793,8 +832,41 @@ object ApplicationMaster extends Logging { def main(args: Array[String]): Unit = { SignalUtils.registerLogger(log) val amArgs = new ApplicationMasterArguments(args) - master = new ApplicationMaster(amArgs) - System.exit(master.run()) + val sparkConf = new SparkConf() + if (amArgs.propertiesFile != null) { + Utils.getPropertiesFromFile(amArgs.propertiesFile).foreach { case (k, v) => + sparkConf.set(k, v) + } + } + // Set system properties for each config entry. This covers two use cases: + // - The default configuration stored by the SparkHadoopUtil class + // - The user application creating a new SparkConf in cluster mode + // + // Both cases create a new SparkConf object which reads these configs from system properties. + sparkConf.getAll.foreach { case (k, v) => + sys.props(k) = v + } + + val yarnConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) + master = new ApplicationMaster(amArgs, sparkConf, yarnConf) + + val ugi = sparkConf.get(PRINCIPAL) match { + case Some(principal) => + val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() + SparkHadoopUtil.get.loginUserFromKeytab(principal, sparkConf.get(KEYTAB).orNull) + val newUGI = UserGroupInformation.getCurrentUser() + // Transfer the original user's tokens to the new user, since it may contain needed tokens + // (such as those user to connect to YARN). + newUGI.addCredentials(originalCreds) + newUGI + + case _ => + SparkHadoopUtil.get.createSparkUser() + } + + ugi.doAs(new PrivilegedExceptionAction[Unit]() { + override def run(): Unit = System.exit(master.run()) + }) } private[spark] def sparkContextInitialized(sc: SparkContext): Unit = { @@ -802,7 +874,7 @@ object ApplicationMaster extends Logging { } private[spark] def getAttemptId(): ApplicationAttemptId = { - master.getAttemptId + master.appAttemptId } private[spark] def getHistoryServerAddress( diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index cc76a7c8f13f5..c10206c847271 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -26,6 +26,7 @@ class ApplicationMasterArguments(val args: Array[String]) { var primaryRFile: String = null var userArgs: Seq[String] = Nil var propertiesFile: String = null + var distCacheConf: String = null parseArgs(args.toList) @@ -62,6 +63,10 @@ class ApplicationMasterArguments(val args: Array[String]) { propertiesFile = value args = tail + case ("--dist-cache-conf") :: value :: tail => + distCacheConf = value + args = tail + case _ => printUsageAndExit(1, args) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 01bdebc000b9f..651e706021fcb 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -21,7 +21,6 @@ import java.io.{FileSystem => _, _} import java.net.{InetAddress, UnknownHostException, URI} import java.nio.ByteBuffer import java.nio.charset.StandardCharsets -import java.security.PrivilegedExceptionAction import java.util.{Locale, Properties, UUID} import java.util.zip.{ZipEntry, ZipOutputStream} @@ -34,9 +33,9 @@ import com.google.common.io.Files import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission -import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.io.Text import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.security.{Credentials, UserGroupInformation} +import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment @@ -45,20 +44,26 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.{YarnClient, YarnClientApplication} import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.exceptions.ApplicationNotFoundException +import org.apache.hadoop.yarn.security.AMRMTokenIdentifier import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} +import org.apache.spark.api.python.PythonUtils import org.apache.spark.deploy.{SparkApplication, SparkHadoopUtil} +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.deploy.yarn.security.YARNHadoopDelegationTokenManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.launcher.{LauncherBackend, SparkAppHandle, YarnCommandBuilderUtils} +import org.apache.spark.rpc.RpcEnv import org.apache.spark.util.{CallerContext, Utils} private[spark] class Client( val args: ClientArguments, - val sparkConf: SparkConf) + val sparkConf: SparkConf, + val rpcEnv: RpcEnv) extends Logging { import Client._ @@ -67,7 +72,11 @@ private[spark] class Client( private val yarnClient = YarnClient.createYarnClient private val hadoopConf = new YarnConfiguration(SparkHadoopUtil.newConfiguration(sparkConf)) - private val isClusterMode = sparkConf.get("spark.submit.deployMode", "client") == "cluster" + private val isClusterMode = sparkConf.get(SUBMIT_DEPLOY_MODE) == "cluster" + + private val isClientUnmanagedAMEnabled = sparkConf.get(YARN_UNMANAGED_AM) && !isClusterMode + private var appMaster: ApplicationMaster = _ + private var stagingDirPath: Path = _ // AM related configurations private val amMemory = if (isClusterMode) { @@ -99,23 +108,23 @@ private[spark] class Client( } private val distCacheMgr = new ClientDistributedCacheManager() + private val cachedResourcesConf = new SparkConf(false) - private val principal = sparkConf.get(PRINCIPAL).orNull private val keytab = sparkConf.get(KEYTAB).orNull - private val loginFromKeytab = principal != null - private val amKeytabFileName: String = { + private val amKeytabFileName: Option[String] = if (keytab != null && isClusterMode) { + val principal = sparkConf.get(PRINCIPAL).orNull require((principal == null) == (keytab == null), "Both principal and keytab must be defined, or neither.") - if (loginFromKeytab) { - logInfo(s"Kerberos credentials: principal = $principal, keytab = $keytab") - // Generate a file name that can be used for the keytab file, that does not conflict - // with any user file. - new File(keytab).getName() + "-" + UUID.randomUUID().toString - } else { - null - } + logInfo(s"Kerberos credentials: principal = $principal, keytab = $keytab") + // Generate a file name that can be used for the keytab file, that does not conflict + // with any user file. + Some(new File(keytab).getName() + "-" + UUID.randomUUID().toString) + } else { + None } + require(keytab == null || !Utils.isLocalUri(keytab), "Keytab should reference a local file.") + private val launcherBackend = new LauncherBackend() { override protected def conf: SparkConf = sparkConf @@ -132,16 +141,14 @@ private[spark] class Client( private var appId: ApplicationId = null - // The app staging dir based on the STAGING_DIR configuration if configured - // otherwise based on the users home directory. - private val appStagingBaseDir = sparkConf.get(STAGING_DIR).map { new Path(_) } - .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) - def reportLauncherState(state: SparkAppHandle.State): Unit = { launcherBackend.setState(state) } def stop(): Unit = { + if (appMaster != null) { + appMaster.stopUnmanaged(stagingDirPath) + } launcherBackend.close() yarnClient.stop() } @@ -154,6 +161,8 @@ private[spark] class Client( * available in the alpha API. */ def submitApplication(): ApplicationId = { + ResourceRequestHelper.validateResources(sparkConf) + var appId: ApplicationId = null try { launcherBackend.connect() @@ -168,6 +177,13 @@ private[spark] class Client( val newAppResponse = newApp.getNewApplicationResponse() appId = newAppResponse.getApplicationId() + // The app staging dir based on the STAGING_DIR configuration if configured + // otherwise based on the users home directory. + val appStagingBaseDir = sparkConf.get(STAGING_DIR) + .map { new Path(_, UserGroupInformation.getCurrentUser.getShortUserName) } + .getOrElse(FileSystem.get(hadoopConf).getHomeDirectory()) + stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) + new CallerContext("CLIENT", sparkConf.get(APP_CALLER_CONTEXT), Option(appId.toString)).setCurrentContext() @@ -187,8 +203,8 @@ private[spark] class Client( appId } catch { case e: Throwable => - if (appId != null) { - cleanupStagingDir(appId) + if (stagingDirPath != null) { + cleanupStagingDir() } throw e } @@ -197,13 +213,12 @@ private[spark] class Client( /** * Cleanup application staging directory. */ - private def cleanupStagingDir(appId: ApplicationId): Unit = { + private def cleanupStagingDir(): Unit = { if (sparkConf.get(PRESERVE_STAGING_FILES)) { return } def cleanupStagingDirInternal(): Unit = { - val stagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) try { val fs = stagingDirPath.getFileSystem(hadoopConf) if (fs.delete(stagingDirPath, true)) { @@ -215,16 +230,7 @@ private[spark] class Client( } } - if (isClusterMode && principal != null && keytab != null) { - val newUgi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - newUgi.doAs(new PrivilegedExceptionAction[Unit] { - override def run(): Unit = { - cleanupStagingDirInternal() - } - }) - } else { - cleanupStagingDirInternal() - } + cleanupStagingDirInternal() } /** @@ -234,6 +240,16 @@ private[spark] class Client( def createApplicationSubmissionContext( newApp: YarnClientApplication, containerContext: ContainerLaunchContext): ApplicationSubmissionContext = { + + val componentName = if (isClusterMode) { + config.YARN_DRIVER_RESOURCE_TYPES_PREFIX + } else { + config.YARN_AM_RESOURCE_TYPES_PREFIX + } + val yarnAMResources = getYarnResourcesAndAmounts(sparkConf, componentName) + val amResources = yarnAMResources ++ + getYarnResourcesFromSparkResources(SPARK_DRIVER_PREFIX, sparkConf) + logDebug(s"AM resources: $amResources") val appContext = newApp.getApplicationSubmissionContext appContext.setApplicationName(sparkConf.get("spark.app.name", "Spark")) appContext.setQueue(sparkConf.get(QUEUE_NAME)) @@ -256,6 +272,10 @@ private[spark] class Client( val capability = Records.newRecord(classOf[Resource]) capability.setMemory(amMemory + amMemoryOverhead) capability.setVirtualCores(amCores) + if (amResources.nonEmpty) { + ResourceRequestHelper.setResourceRequests(amResources, capability) + } + logDebug(s"Created resource capability for AM request: $capability") sparkConf.get(AM_NODE_LABEL_EXPRESSION) match { case Some(expr) => @@ -284,33 +304,28 @@ private[spark] class Client( "does not support it", e) } } - + appContext.setUnmanagedAM(isClientUnmanagedAMEnabled) appContext } /** * Set up security tokens for launching our ApplicationMaster container. * - * This method will obtain delegation tokens from all the registered providers, and set them in - * the AM's launch context. + * In client mode, a set of credentials has been obtained by the scheduler, so they are copied + * and sent to the AM. In cluster mode, new credentials are obtained and then sent to the AM, + * along with whatever credentials the current user already has. */ private def setupSecurityToken(amContainer: ContainerLaunchContext): Unit = { - val credentials = UserGroupInformation.getCurrentUser().getCredentials() - val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - credentialManager.obtainDelegationTokens(hadoopConf, credentials) - - // When using a proxy user, copy the delegation tokens to the user's credentials. Avoid - // that for regular users, since in those case the user already has access to the TGT, - // and adding delegation tokens could lead to expired or cancelled tokens being used - // later, as reported in SPARK-15754. val currentUser = UserGroupInformation.getCurrentUser() - if (SparkHadoopUtil.get.isProxyUser(currentUser)) { - currentUser.addCredentials(credentials) + val credentials = currentUser.getCredentials() + + if (isClusterMode) { + val credentialManager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) + credentialManager.obtainDelegationTokens(credentials) } - val dob = new DataOutputBuffer - credentials.writeTokenStorageToStream(dob) - amContainer.setTokens(ByteBuffer.wrap(dob.getData)) + val serializedCreds = SparkHadoopUtil.get.serialize(credentials) + amContainer.setTokens(ByteBuffer.wrap(serializedCreds)) } /** Get the application report from the ResourceManager for an application we have submitted. */ @@ -459,7 +474,7 @@ private[spark] class Client( appMasterOnly: Boolean = false): (Boolean, String) = { val trimmedPath = path.trim() val localURI = Utils.resolveURI(trimmedPath) - if (localURI.getScheme != LOCAL_SCHEME) { + if (localURI.getScheme != Utils.LOCAL_SCHEME) { if (addDistributedUri(localURI)) { val localPath = getQualifiedLocalPath(localURI, hadoopConf) val linkname = targetDir.map(_ + "/").getOrElse("") + @@ -480,11 +495,11 @@ private[spark] class Client( // If we passed in a keytab, make sure we copy the keytab to the staging directory on // HDFS, and setup the relevant environment vars, so the AM can login again. - if (loginFromKeytab) { + amKeytabFileName.foreach { kt => logInfo("To enable the AM to login from keytab, credentials are being copied over to the AM" + " via the YARN Secure Distributed Cache.") val (_, localizedPath) = distribute(keytab, - destName = Some(amKeytabFileName), + destName = Some(kt), appMasterOnly = true) require(localizedPath != null, "Keytab file already distributed.") } @@ -502,7 +517,7 @@ private[spark] class Client( val sparkArchive = sparkConf.get(SPARK_ARCHIVE) if (sparkArchive.isDefined) { val archive = sparkArchive.get - require(!isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") + require(!Utils.isLocalUri(archive), s"${SPARK_ARCHIVE.key} cannot be a local URI.") distribute(Utils.resolveURI(archive).toString, resType = LocalResourceType.ARCHIVE, destName = Some(LOCALIZED_LIB_DIR)) @@ -512,7 +527,7 @@ private[spark] class Client( // Break the list of jars to upload, and resolve globs. val localJars = new ArrayBuffer[String]() jars.foreach { jar => - if (!isLocalUri(jar)) { + if (!Utils.isLocalUri(jar)) { val path = getQualifiedLocalPath(Utils.resolveURI(jar), hadoopConf) val pathFs = FileSystem.get(path.toUri(), hadoopConf) pathFs.globStatus(path).filter(_.isFile()).foreach { entry => @@ -620,7 +635,7 @@ private[spark] class Client( // Update the configuration with all the distributed files, minus the conf archive. The // conf archive will be handled by the AM differently so that we avoid having to send // this configuration by other means. See SPARK-14602 for one reason of why this is needed. - distCacheMgr.updateConfiguration(sparkConf) + distCacheMgr.updateConfiguration(cachedResourcesConf) // Upload the conf archive to HDFS manually, and record its location in the configuration. // This will allow the AM to know where the conf archive is in HDFS, so that it can be @@ -632,7 +647,7 @@ private[spark] class Client( // system. val remoteConfArchivePath = new Path(destDir, LOCALIZED_CONF_ARCHIVE) val remoteFs = FileSystem.get(remoteConfArchivePath.toUri(), hadoopConf) - sparkConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString()) + cachedResourcesConf.set(CACHED_CONF_ARCHIVE, remoteConfArchivePath.toString()) val localConfArchive = new Path(createConfArchive().toURI()) copyFileToRemote(destDir, localConfArchive, replication, symlinkCache, force = true, @@ -644,11 +659,6 @@ private[spark] class Client( remoteFs, hadoopConf, remoteConfArchivePath, localResources, LocalResourceType.ARCHIVE, LOCALIZED_CONF_DIR, statCache, appMasterOnly = false) - // Clear the cache-related entries from the configuration to avoid them polluting the - // UI's environment page. This works for client mode; for cluster mode, this is handled - // by the AM. - CACHE_CONFIGS.foreach(sparkConf.remove) - localResources } @@ -713,6 +723,7 @@ private[spark] class Client( new File(Utils.getLocalDir(sparkConf))) val confStream = new ZipOutputStream(new FileOutputStream(confArchive)) + logDebug(s"Creating an archive with the config files for distribution at $confArchive.") try { confStream.setLevel(0) @@ -752,19 +763,25 @@ private[spark] class Client( hadoopConf.writeXml(confStream) confStream.closeEntry() - // Save Spark configuration to a file in the archive, but filter out the app's secret. - val props = new Properties() - sparkConf.getAll.foreach { case (k, v) => - props.setProperty(k, v) + // Save Spark configuration to a file in the archive. + val props = confToProperties(sparkConf) + + // If propagating the keytab to the AM, override the keytab name with the name of the + // distributed file. Otherwise remove princpal/keytab from the conf, so they're not seen + // by the AM at all. + amKeytabFileName match { + case Some(kt) => + props.setProperty(KEYTAB.key, kt) + case None => + props.remove(PRINCIPAL.key) + props.remove(KEYTAB.key) } - // Override spark.yarn.key to point to the location in distributed cache which will be used - // by AM. - Option(amKeytabFileName).foreach { k => props.setProperty(KEYTAB.key, k) } - confStream.putNextEntry(new ZipEntry(SPARK_CONF_FILE)) - val writer = new OutputStreamWriter(confStream, StandardCharsets.UTF_8) - props.store(writer, "Spark configuration.") - writer.flush() - confStream.closeEntry() + + writePropertiesToArchive(props, SPARK_CONF_FILE, confStream) + + // Write the distributed cache config to the archive. + writePropertiesToArchive(confToProperties(cachedResourcesConf), DIST_CACHE_CONF_FILE, + confStream) } finally { confStream.close() } @@ -801,7 +818,7 @@ private[spark] class Client( } (pySparkArchives ++ pyArchives).foreach { path => val uri = Utils.resolveURI(path) - if (uri.getScheme != LOCAL_SCHEME) { + if (uri.getScheme != Utils.LOCAL_SCHEME) { pythonPath += buildPath(Environment.PWD.$$(), new Path(uri).getName()) } else { pythonPath += uri.getPath() @@ -843,7 +860,6 @@ private[spark] class Client( : ContainerLaunchContext = { logInfo("Setting up container launch context for our AM") val appId = newAppResponse.getApplicationId - val appStagingDirPath = new Path(appStagingBaseDir, getAppStagingDir(appId)) val pySparkArchives = if (sparkConf.get(IS_PYTHON_APP)) { findPySparkArchives() @@ -851,8 +867,8 @@ private[spark] class Client( Nil } - val launchEnv = setupLaunchEnv(appStagingDirPath, pySparkArchives) - val localResources = prepareLocalResources(appStagingDirPath, pySparkArchives) + val launchEnv = setupLaunchEnv(stagingDirPath, pySparkArchives) + val localResources = prepareLocalResources(stagingDirPath, pySparkArchives) val amContainer = Records.newRecord(classOf[ContainerLaunchContext]) amContainer.setLocalResources(localResources.asJava) @@ -968,7 +984,10 @@ private[spark] class Client( } val amArgs = Seq(amClass) ++ userClass ++ userJar ++ primaryPyFile ++ primaryRFile ++ userArgs ++ - Seq("--properties-file", buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, SPARK_CONF_FILE)) + Seq("--properties-file", + buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, SPARK_CONF_FILE)) ++ + Seq("--dist-cache-conf", + buildPath(Environment.PWD.$$(), LOCALIZED_CONF_DIR, DIST_CACHE_CONF_FILE)) // Command for the ApplicationMaster val commands = prefixEnv ++ @@ -1031,7 +1050,7 @@ private[spark] class Client( } catch { case e: ApplicationNotFoundException => logError(s"Application $appId not found.") - cleanupStagingDir(appId) + cleanupStagingDir() return YarnAppReport(YarnApplicationState.KILLED, FinalApplicationStatus.KILLED, None) case NonFatal(e) => val msg = s"Failed to contact YARN for application $appId." @@ -1078,14 +1097,17 @@ private[spark] class Client( if (state == YarnApplicationState.FINISHED || state == YarnApplicationState.FAILED || state == YarnApplicationState.KILLED) { - cleanupStagingDir(appId) + cleanupStagingDir() return createAppReport(report) } if (returnOnRunning && state == YarnApplicationState.RUNNING) { return createAppReport(report) } - + if (state == YarnApplicationState.ACCEPTED && isClientUnmanagedAMEnabled && + appMaster == null && report.getAMRMToken != null) { + appMaster = startApplicationMasterService(report) + } lastState = state } @@ -1093,6 +1115,30 @@ private[spark] class Client( throw new SparkException("While loop is depleted! This should never happen...") } + private def startApplicationMasterService(report: ApplicationReport): ApplicationMaster = { + // Add AMRMToken to establish connection between RM and AM + val token = report.getAMRMToken + val amRMToken: org.apache.hadoop.security.token.Token[AMRMTokenIdentifier] = + new org.apache.hadoop.security.token.Token[AMRMTokenIdentifier]( + token.getIdentifier().array(), token.getPassword().array, + new Text(token.getKind()), new Text(token.getService())) + val currentUGI = UserGroupInformation.getCurrentUser + currentUGI.addToken(amRMToken) + + // Start Application Service in a separate thread and continue with application monitoring + val appMaster = new ApplicationMaster( + new ApplicationMasterArguments(Array.empty), sparkConf, hadoopConf) + val amService = new Thread("Unmanaged Application Master Service") { + override def run(): Unit = { + appMaster.runUnmanaged(rpcEnv, report.getCurrentApplicationAttemptId, + stagingDirPath, cachedResourcesConf) + } + } + amService.setDaemon(true) + amService.start() + appMaster + } + private def formatReportDetails(report: ApplicationReport): String = { val details = Seq[(String, String)]( ("client token", getClientToken(report)), @@ -1156,7 +1202,7 @@ private[spark] class Client( val pyArchivesFile = new File(pyLibPath, "pyspark.zip") require(pyArchivesFile.exists(), s"$pyArchivesFile not found; cannot run pyspark application in YARN mode.") - val py4jFile = new File(pyLibPath, "py4j-0.10.7-src.zip") + val py4jFile = new File(pyLibPath, PythonUtils.PY4J_ZIP_NAME) require(py4jFile.exists(), s"$py4jFile not found; cannot run pyspark application in YARN mode.") Seq(pyArchivesFile.getAbsolutePath(), py4jFile.getAbsolutePath()) @@ -1170,9 +1216,6 @@ private object Client extends Logging { // Alias for the user jar val APP_JAR_NAME: String = "__app__.jar" - // URI scheme that identifies local resources - val LOCAL_SCHEME = "local" - // Staging directory for any temporary jars or files val SPARK_STAGING: String = ".sparkStaging" @@ -1200,6 +1243,9 @@ private object Client extends Logging { // Name of the file in the conf archive containing Spark configuration. val SPARK_CONF_FILE = "__spark_conf__.properties" + // Name of the file in the conf archive containing the distributed cache info. + val DIST_CACHE_CONF_FILE = "__spark_dist_cache__.properties" + // Subdirectory where the user's python files (not archives) will be placed. val LOCALIZED_PYTHON_DIR = "__pyfiles__" @@ -1294,7 +1340,7 @@ private object Client extends Logging { addClasspathEntry(buildPath(Environment.PWD.$$(), LOCALIZED_LIB_DIR, "*"), env) if (sparkConf.get(SPARK_ARCHIVE).isEmpty) { sparkConf.get(SPARK_JARS).foreach { jars => - jars.filter(isLocalUri).foreach { jar => + jars.filter(Utils.isLocalUri).foreach { jar => val uri = new URI(jar) addClasspathEntry(getClusterPath(sparkConf, uri.getPath()), env) } @@ -1327,7 +1373,7 @@ private object Client extends Logging { private def getMainJarUri(mainJar: Option[String]): Option[URI] = { mainJar.flatMap { path => val uri = Utils.resolveURI(path) - if (uri.getScheme == LOCAL_SCHEME) Some(uri) else None + if (uri.getScheme == Utils.LOCAL_SCHEME) Some(uri) else None }.orElse(Some(new URI(APP_JAR_NAME))) } @@ -1355,7 +1401,7 @@ private object Client extends Logging { uri: URI, fileName: String, env: HashMap[String, String]): Unit = { - if (uri != null && uri.getScheme == LOCAL_SCHEME) { + if (uri != null && uri.getScheme == Utils.LOCAL_SCHEME) { addClasspathEntry(getClusterPath(conf, uri.getPath), env) } else if (fileName != null) { addClasspathEntry(buildPath(Environment.PWD.$$(), fileName), env) @@ -1476,11 +1522,6 @@ private object Client extends Logging { components.mkString(Path.SEPARATOR) } - /** Returns whether the URI is a "local:" URI. */ - def isLocalUri(uri: String): Boolean = { - uri.startsWith(s"$LOCAL_SCHEME:") - } - def createAppReport(report: ApplicationReport): YarnAppReport = { val diags = report.getDiagnostics() val diagsOpt = if (diags != null && diags.nonEmpty) Some(diags) else None @@ -1504,6 +1545,22 @@ private object Client extends Logging { } getClusterPath(conf, cmdPrefix) } + + def confToProperties(conf: SparkConf): Properties = { + val props = new Properties() + conf.getAll.foreach { case (k, v) => + props.setProperty(k, v) + } + props + } + + def writePropertiesToArchive(props: Properties, name: String, out: ZipOutputStream): Unit = { + out.putNextEntry(new ZipEntry(name)) + val writer = new OutputStreamWriter(out, StandardCharsets.UTF_8) + props.store(writer, "Spark configuration.") + writer.flush() + out.closeEntry() + } } private[spark] class YarnClusterApplication extends SparkApplication { @@ -1511,10 +1568,10 @@ private[spark] class YarnClusterApplication extends SparkApplication { override def start(args: Array[String], conf: SparkConf): Unit = { // SparkSubmit would use yarn cache to distribute files & jars in yarn mode, // so remove them from sparkConf here for yarn mode. - conf.remove("spark.jars") - conf.remove("spark.files") + conf.remove(JARS) + conf.remove(FILES) - new Client(new ClientArguments(args), conf).run() + new Client(new ClientArguments(args), conf, null).run() } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 49a0b93aa5c40..7046ad74056fc 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -19,11 +19,12 @@ package org.apache.spark.deploy.yarn import java.io.File import java.nio.ByteBuffer -import java.util.Collections +import java.util.{Collections, Locale} import scala.collection.JavaConverters._ import scala.collection.mutable.{HashMap, ListBuffer} +import org.apache.hadoop.HadoopIllegalArgumentException import org.apache.hadoop.fs.Path import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.security.UserGroupInformation @@ -33,13 +34,13 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.NMClient import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import org.apache.hadoop.yarn.util.Records import org.apache.spark.{SecurityManager, SparkConf, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.JavaUtils -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, YarnContainerInfoHelper} private[yarn] class ExecutorRunnable( container: Option[Container], @@ -75,7 +76,7 @@ private[yarn] class ExecutorRunnable( | env: |${Utils.redact(sparkConf, env.toSeq).map { case (k, v) => s" $k -> $v\n" }.mkString} | command: - | ${commands.mkString(" \\ \n ")} + | ${Utils.redactCommandLineArgs(sparkConf, commands).mkString(" \\ \n ")} | | resources: |${localResources.map { case (k, v) => s" $k -> $v\n" }.mkString} @@ -201,7 +202,7 @@ private[yarn] class ExecutorRunnable( val commands = prefixEnv ++ Seq(Environment.JAVA_HOME.$$() + "/bin/java", "-server") ++ javaOpts ++ - Seq("org.apache.spark.executor.CoarseGrainedExecutorBackend", + Seq("org.apache.spark.executor.YarnCoarseGrainedExecutorBackend", "--driver-url", masterAddress, "--executor-id", executorId, "--hostname", hostname, @@ -220,13 +221,6 @@ private[yarn] class ExecutorRunnable( val env = new HashMap[String, String]() Client.populateClasspath(null, conf, sparkConf, env, sparkConf.get(EXECUTOR_CLASS_PATH)) - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = conf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - System.getenv().asScala.filterKeys(_.startsWith("SPARK")) .foreach { case (k, v) => env(k) = v } @@ -241,18 +235,6 @@ private[yarn] class ExecutorRunnable( } } - // Add log urls - container.foreach { c => - sys.env.get("SPARK_USER").foreach { user => - val containerId = ConverterUtils.toString(c.getId) - val address = c.getNodeHttpAddress - val baseUrl = s"$httpScheme$address/node/containerlogs/$containerId/$user" - - env("SPARK_LOG_URL_STDERR") = s"$baseUrl/stderr?start=-4096" - env("SPARK_LOG_URL_STDOUT") = s"$baseUrl/stdout?start=-4096" - } - } - env } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala index 0a7a16f468fbd..2288bb55d8b47 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala @@ -138,9 +138,8 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy( // Only filter out the ratio which is larger than 0, which means the current host can // still be allocated with new container request. val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray - val racks = hosts.map { h => - resolver.resolve(yarnConf, h) - }.toSet + val racks = resolver.resolve(hosts).map(_.getNetworkLocation) + .filter(_ != null).toSet containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray) // Minus 1 each time when the host is used. When the current ratio is 0, diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala new file mode 100644 index 0000000000000..522c16b3a1082 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/ResourceRequestHelper.scala @@ -0,0 +1,208 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import java.lang.{Long => JLong} +import java.lang.reflect.InvocationTargetException + +import scala.collection.mutable +import scala.util.Try + +import org.apache.hadoop.yarn.api.records.Resource + +import org.apache.spark.{SparkConf, SparkException} +import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.{AMOUNT, FPGA, GPU} +import org.apache.spark.util.{CausedBy, Utils} + +/** + * This helper class uses some of Hadoop 3 methods from the YARN API, + * so we need to use reflection to avoid compile error when building against Hadoop 2.x + */ +private object ResourceRequestHelper extends Logging { + private val AMOUNT_AND_UNIT_REGEX = "([0-9]+)([A-Za-z]*)".r + private val RESOURCE_INFO_CLASS = "org.apache.hadoop.yarn.api.records.ResourceInformation" + val YARN_GPU_RESOURCE_CONFIG = "yarn.io/gpu" + val YARN_FPGA_RESOURCE_CONFIG = "yarn.io/fpga" + + private[yarn] def getYarnResourcesAndAmounts( + sparkConf: SparkConf, + componentName: String): Map[String, String] = { + sparkConf.getAllWithPrefix(s"$componentName").map { case (key, value) => + val splitIndex = key.lastIndexOf('.') + if (splitIndex == -1) { + val errorMessage = s"Missing suffix for ${componentName}${key}, you must specify" + + s" a suffix - $AMOUNT is currently the only supported suffix." + throw new IllegalArgumentException(errorMessage.toString()) + } + val resourceName = key.substring(0, splitIndex) + val resourceSuffix = key.substring(splitIndex + 1) + if (!AMOUNT.equals(resourceSuffix)) { + val errorMessage = s"Unsupported suffix: $resourceSuffix in: ${componentName}${key}, " + + s"only .$AMOUNT is supported." + throw new IllegalArgumentException(errorMessage.toString()) + } + (resourceName, value) + }.toMap + } + + /** + * Convert Spark resources into YARN resources. + * The only resources we know how to map from spark configs to yarn configs are + * gpus and fpgas, everything else the user has to specify them in both the + * spark.yarn.*.resource and the spark.*.resource configs. + */ + private[yarn] def getYarnResourcesFromSparkResources( + confPrefix: String, + sparkConf: SparkConf + ): Map[String, String] = { + Map(GPU -> YARN_GPU_RESOURCE_CONFIG, FPGA -> YARN_FPGA_RESOURCE_CONFIG).map { + case (rName, yarnName) => + (yarnName -> sparkConf.get(ResourceID(confPrefix, rName).amountConf, "0")) + }.filter { case (_, count) => count.toLong > 0 } + } + + /** + * Validates sparkConf and throws a SparkException if any of standard resources (memory or cores) + * is defined with the property spark.yarn.x.resource.y + * Need to reject all combinations of AM / Driver / Executor and memory / CPU cores resources, as + * Spark has its own names for them (memory, cores), + * but YARN have its names too: (memory, memory-mb, mb) and (cores, vcores, cpu-vcores). + * We need to disable every possible way YARN could receive the resource definitions above. + */ + def validateResources(sparkConf: SparkConf): Unit = { + val resourceDefinitions = Seq[(String, String)]( + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory"), + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "mb"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "mb"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "mb"), + (AM_MEMORY.key, YARN_AM_RESOURCE_TYPES_PREFIX + "memory-mb"), + (DRIVER_MEMORY.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "memory-mb"), + (EXECUTOR_MEMORY.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "memory-mb"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cores"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "vcores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "vcores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "vcores"), + (AM_CORES.key, YARN_AM_RESOURCE_TYPES_PREFIX + "cpu-vcores"), + (DRIVER_CORES.key, YARN_DRIVER_RESOURCE_TYPES_PREFIX + "cpu-vcores"), + (EXECUTOR_CORES.key, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX + "cpu-vcores"), + (ResourceID(SPARK_EXECUTOR_PREFIX, "fpga").amountConf, + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${YARN_FPGA_RESOURCE_CONFIG}"), + (ResourceID(SPARK_DRIVER_PREFIX, "fpga").amountConf, + s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${YARN_FPGA_RESOURCE_CONFIG}"), + (ResourceID(SPARK_EXECUTOR_PREFIX, "gpu").amountConf, + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${YARN_GPU_RESOURCE_CONFIG}"), + (ResourceID(SPARK_DRIVER_PREFIX, "gpu").amountConf, + s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${YARN_GPU_RESOURCE_CONFIG}")) + + val errorMessage = new mutable.StringBuilder() + + resourceDefinitions.foreach { case (sparkName, resourceRequest) => + val resourceRequestAmount = s"${resourceRequest}.${AMOUNT}" + if (sparkConf.contains(resourceRequestAmount)) { + errorMessage.append(s"Error: Do not use $resourceRequestAmount, " + + s"please use $sparkName instead!\n") + } + } + + if (errorMessage.nonEmpty) { + throw new SparkException(errorMessage.toString()) + } + } + + /** + * Sets resource amount with the corresponding unit to the passed resource object. + * @param resources resource values to set + * @param resource resource object to update + */ + def setResourceRequests( + resources: Map[String, String], + resource: Resource): Unit = { + require(resource != null, "Resource parameter should not be null!") + + logDebug(s"Custom resources requested: $resources") + if (!isYarnResourceTypesAvailable()) { + if (resources.nonEmpty) { + logWarning("Ignoring custom resource requests because " + + "the version of YARN does not support it!") + } + return + } + + val resInfoClass = Utils.classForName(RESOURCE_INFO_CLASS) + val setResourceInformationMethod = + resource.getClass.getMethod("setResourceInformation", classOf[String], resInfoClass) + resources.foreach { case (name, rawAmount) => + try { + val AMOUNT_AND_UNIT_REGEX(amountPart, unitPart) = rawAmount + val amount = amountPart.toLong + val unit = unitPart match { + case "g" => "G" + case "t" => "T" + case "p" => "P" + case _ => unitPart + } + logDebug(s"Registering resource with name: $name, amount: $amount, unit: $unit") + val resourceInformation = createResourceInformation(name, amount, unit, resInfoClass) + setResourceInformationMethod.invoke( + resource, name, resourceInformation.asInstanceOf[AnyRef]) + } catch { + case _: MatchError => + throw new IllegalArgumentException(s"Resource request for '$name' ('$rawAmount') " + + s"does not match pattern $AMOUNT_AND_UNIT_REGEX.") + case CausedBy(e: IllegalArgumentException) => + throw new IllegalArgumentException(s"Invalid request for $name: ${e.getMessage}") + case e: InvocationTargetException if e.getCause != null => throw e.getCause + } + } + } + + private def createResourceInformation( + resourceName: String, + amount: Long, + unit: String, + resInfoClass: Class[_]): Any = { + val resourceInformation = + if (unit.nonEmpty) { + val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", + classOf[String], classOf[String], JLong.TYPE) + resInfoNewInstanceMethod.invoke(null, resourceName, unit, amount.asInstanceOf[JLong]) + } else { + val resInfoNewInstanceMethod = resInfoClass.getMethod("newInstance", + classOf[String], JLong.TYPE) + resInfoNewInstanceMethod.invoke(null, resourceName, amount.asInstanceOf[JLong]) + } + resourceInformation + } + + /** + * Checks whether Hadoop 2.x or 3 is used as a dependency. + * In case of Hadoop 3 and later, the ResourceInformation class + * should be available on the classpath. + */ + def isYarnResourceTypesAvailable(): Boolean = { + Try(Utils.classForName(RESOURCE_INFO_CLASS)).isSuccess + } +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala index c711d088f2116..51be93261683a 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala @@ -17,24 +17,103 @@ package org.apache.spark.deploy.yarn +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import com.google.common.base.Strings import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.CommonConfigurationKeysPublic +import org.apache.hadoop.net._ +import org.apache.hadoop.util.ReflectionUtils import org.apache.hadoop.yarn.util.RackResolver import org.apache.log4j.{Level, Logger} +import org.apache.spark.internal.Logging + /** - * Wrapper around YARN's [[RackResolver]]. This allows Spark tests to easily override the - * default behavior, since YARN's class self-initializes the first time it's called, and - * future calls all use the initial configuration. + * Re-implement YARN's [[RackResolver]] for hadoop releases without YARN-9332. + * This also allows Spark tests to easily override the default behavior, since YARN's class + * self-initializes the first time it's called, and future calls all use the initial configuration. */ -private[yarn] class SparkRackResolver { +private[spark] class SparkRackResolver(conf: Configuration) extends Logging { // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) } - def resolve(conf: Configuration, hostName: String): String = { - RackResolver.resolve(conf, hostName).getNetworkLocation() + private val dnsToSwitchMapping: DNSToSwitchMapping = { + val dnsToSwitchMappingClass = + conf.getClass(CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY, + classOf[ScriptBasedMapping], classOf[DNSToSwitchMapping]) + ReflectionUtils.newInstance(dnsToSwitchMappingClass, conf) + .asInstanceOf[DNSToSwitchMapping] match { + case c: CachedDNSToSwitchMapping => c + case o => new CachedDNSToSwitchMapping(o) + } + } + + def resolve(hostName: String): String = { + coreResolve(Seq(hostName)).head.getNetworkLocation + } + + /** + * Added in SPARK-13704. + * This should be changed to `RackResolver.resolve(conf, hostNames)` + * in hadoop releases with YARN-9332. + */ + def resolve(hostNames: Seq[String]): Seq[Node] = { + coreResolve(hostNames) + } + + private def coreResolve(hostNames: Seq[String]): Seq[Node] = { + if (hostNames.isEmpty) { + return Seq.empty + } + val nodes = new ArrayBuffer[Node] + // dnsToSwitchMapping is thread-safe + val rNameList = dnsToSwitchMapping.resolve(hostNames.toList.asJava).asScala + if (rNameList == null || rNameList.isEmpty) { + hostNames.foreach(nodes += new NodeBase(_, NetworkTopology.DEFAULT_RACK)) + logInfo(s"Got an error when resolving hostNames. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK} for all") + } else { + for ((hostName, rName) <- hostNames.zip(rNameList)) { + if (Strings.isNullOrEmpty(rName)) { + nodes += new NodeBase(hostName, NetworkTopology.DEFAULT_RACK) + logDebug(s"Could not resolve $hostName. " + + s"Falling back to ${NetworkTopology.DEFAULT_RACK}") + } else { + nodes += new NodeBase(hostName, rName) + } + } + } + nodes.toList + } +} + +/** + * Utility to resolve the rack for hosts in an efficient manner. + * It will cache the rack for individual hosts to avoid + * repeatedly performing the same expensive lookup. + */ +object SparkRackResolver extends Logging { + @volatile private var instance: SparkRackResolver = _ + + /** + * It will return the static resolver instance. If there is already an instance, the passed + * conf is entirely ignored. If there is not a shared instance, it will create one with the + * given conf. + */ + def get(conf: Configuration): SparkRackResolver = { + if (instance == null) { + synchronized { + if (instance == null) { + instance = new SparkRackResolver(conf) + } + } + } + instance } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala index 8a7551de7c088..8ec7bd66b2507 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala @@ -20,7 +20,6 @@ package org.apache.spark.deploy.yarn import java.util.Collections import java.util.concurrent._ import java.util.concurrent.atomic.AtomicInteger -import java.util.regex.Pattern import scala.collection.JavaConverters._ import scala.collection.mutable @@ -33,10 +32,12 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{SecurityManager, SparkConf, SparkException} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Python._ import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason} import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.RemoveExecutor @@ -128,7 +129,7 @@ private[yarn] class YarnAllocator( private var numUnexpectedContainerRelease = 0L private val containerIdToExecutorId = new HashMap[ContainerId, String] - // Executor memory in MB. + // Executor memory in MiB. protected val executorMemory = sparkConf.get(EXECUTOR_MEMORY).toInt // Additional memory overhead. protected val memoryOverhead: Int = sparkConf.get(EXECUTOR_MEMORY_OVERHEAD).getOrElse( @@ -140,10 +141,19 @@ private[yarn] class YarnAllocator( } // Number of cores per executor. protected val executorCores = sparkConf.get(EXECUTOR_CORES) - // Resource capability requested for each executors - private[yarn] val resource = Resource.newInstance( - executorMemory + memoryOverhead + pysparkWorkerMemory, - executorCores) + + private val executorResourceRequests = + getYarnResourcesAndAmounts(sparkConf, config.YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) ++ + getYarnResourcesFromSparkResources(SPARK_EXECUTOR_PREFIX, sparkConf) + + // Resource capability requested for each executor + private[yarn] val resource: Resource = { + val resource = Resource.newInstance( + executorMemory + memoryOverhead + pysparkWorkerMemory, executorCores) + ResourceRequestHelper.setResourceRequests(executorResourceRequests, resource) + logDebug(s"Created resource capability: $resource") + resource + } private val launcherPool = ThreadUtils.newDaemonCachedThreadPool( "ContainerLauncher", sparkConf.get(CONTAINER_LAUNCH_MAX_THREADS)) @@ -184,11 +194,9 @@ private[yarn] class YarnAllocator( * A sequence of pending container requests at the given location that have not yet been * fulfilled. */ - private def getPendingAtLocation(location: String): Seq[ContainerRequest] = { + private def getPendingAtLocation(location: String): Seq[ContainerRequest] = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, location, resource).asScala .flatMap(_.asScala) - .toSeq - } /** * Request as many executors from the ResourceManager as needed to reach the desired total. If @@ -287,19 +295,26 @@ private[yarn] class YarnAllocator( s"pending: $numPendingAllocate, running: ${runningExecutors.size}, " + s"executorsStarting: ${numExecutorsStarting.get}") + // Split the pending container request into three groups: locality matched list, locality + // unmatched list and non-locality list. Take the locality matched container request into + // consideration of container placement, treat as allocated containers. + // For locality unmatched and locality free container requests, cancel these container + // requests, since required locality preference has been changed, recalculating using + // container placement strategy. + val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( + hostToLocalTaskCounts, pendingAllocate) + if (missing > 0) { - logInfo(s"Will request $missing executor container(s), each with " + - s"${resource.getVirtualCores} core(s) and " + - s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)") - - // Split the pending container request into three groups: locality matched list, locality - // unmatched list and non-locality list. Take the locality matched container request into - // consideration of container placement, treat as allocated containers. - // For locality unmatched and locality free container requests, cancel these container - // requests, since required locality preference has been changed, recalculating using - // container placement strategy. - val (localRequests, staleRequests, anyHostRequests) = splitPendingAllocationsByLocality( - hostToLocalTaskCounts, pendingAllocate) + if (log.isInfoEnabled()) { + var requestContainerMessage = s"Will request $missing executor container(s), each with " + + s"${resource.getVirtualCores} core(s) and " + + s"${resource.getMemory} MB memory (including $memoryOverhead MB of overhead)" + if (ResourceRequestHelper.isYarnResourceTypesAvailable() && + executorResourceRequests.nonEmpty) { + requestContainerMessage ++= s" with custom resources: " + resource.toString + } + logInfo(requestContainerMessage) + } // cancel "stale" requests for locations that are no longer needed staleRequests.foreach { stale => @@ -360,17 +375,19 @@ private[yarn] class YarnAllocator( val numToCancel = math.min(numPendingAllocate, -missing) logInfo(s"Canceling requests for $numToCancel executor container(s) to have a new desired " + s"total $targetNumExecutors executors.") - - val matchingRequests = amClient.getMatchingRequests(RM_REQUEST_PRIORITY, ANY_HOST, resource) - if (!matchingRequests.isEmpty) { - matchingRequests.iterator().next().asScala - .take(numToCancel).foreach(amClient.removeContainerRequest) - } else { - logWarning("Expected to find pending requests, but found none.") - } + // cancel pending allocate requests by taking locality preference into account + val cancelRequests = (staleRequests ++ anyHostRequests ++ localRequests).take(numToCancel) + cancelRequests.foreach(amClient.removeContainerRequest) } } + def stop(): Unit = { + // Forcefully shut down the launcher pool, in case this is being called in the middle of + // container allocation. This will prevent queued executors from being started - and + // potentially interrupt active ExecutorRunnable instances too. + launcherPool.shutdownNow() + } + private def hostStr(request: ContainerRequest): String = { Option(request.getNodes) match { case Some(nodes) => nodes.asScala.mkString(",") @@ -407,12 +424,40 @@ private[yarn] class YarnAllocator( containersToUse, remainingAfterHostMatches) } - // Match remaining by rack + // Match remaining by rack. Because YARN's RackResolver swallows thread interrupts + // (see SPARK-27094), which can cause this code to miss interrupts from the AM, use + // a separate thread to perform the operation. val remainingAfterRackMatches = new ArrayBuffer[Container] - for (allocatedContainer <- remainingAfterHostMatches) { - val rack = resolver.resolve(conf, allocatedContainer.getNodeId.getHost) - matchContainerToRequest(allocatedContainer, rack, containersToUse, - remainingAfterRackMatches) + if (remainingAfterHostMatches.nonEmpty) { + var exception: Option[Throwable] = None + val thread = new Thread("spark-rack-resolver") { + override def run(): Unit = { + try { + for (allocatedContainer <- remainingAfterHostMatches) { + val rack = resolver.resolve(allocatedContainer.getNodeId.getHost) + matchContainerToRequest(allocatedContainer, rack, containersToUse, + remainingAfterRackMatches) + } + } catch { + case e: Throwable => + exception = Some(e) + } + } + } + thread.setDaemon(true) + thread.start() + + try { + thread.join() + } catch { + case e: InterruptedException => + thread.interrupt() + throw e + } + + if (exception.isDefined) { + throw exception.get + } } // Assign remaining that are neither node-local nor rack-local @@ -422,7 +467,7 @@ private[yarn] class YarnAllocator( remainingAfterOffRackMatches) } - if (!remainingAfterOffRackMatches.isEmpty) { + if (remainingAfterOffRackMatches.nonEmpty) { logDebug(s"Releasing ${remainingAfterOffRackMatches.size} unneeded containers that were " + s"allocated to us") for (container <- remainingAfterOffRackMatches) { @@ -456,13 +501,20 @@ private[yarn] class YarnAllocator( // memory, but use the asked vcore count for matching, effectively disabling matching on vcore // count. val matchingResource = Resource.newInstance(allocatedContainer.getResource.getMemory, - resource.getVirtualCores) + resource.getVirtualCores) + + ResourceRequestHelper.setResourceRequests(executorResourceRequests, matchingResource) + + logDebug(s"Calling amClient.getMatchingRequests with parameters: " + + s"priority: ${allocatedContainer.getPriority}, " + + s"location: $location, resource: $matchingResource") val matchingRequests = amClient.getMatchingRequests(allocatedContainer.getPriority, location, matchingResource) // Match the allocation to a request if (!matchingRequests.isEmpty) { val containerRequest = matchingRequests.get(0).iterator.next + logDebug(s"Removing container request via AM client: $containerRequest") amClient.removeContainerRequest(containerRequest) containersToUse += allocatedContainer } else { @@ -498,35 +550,33 @@ private[yarn] class YarnAllocator( if (runningExecutors.size() < targetNumExecutors) { numExecutorsStarting.incrementAndGet() if (launchContainers) { - launcherPool.execute(new Runnable { - override def run(): Unit = { - try { - new ExecutorRunnable( - Some(container), - conf, - sparkConf, - driverUrl, - executorId, - executorHostname, - executorMemory, - executorCores, - appAttemptId.getApplicationId.toString, - securityMgr, - localResources - ).run() - updateInternalState() - } catch { - case e: Throwable => - numExecutorsStarting.decrementAndGet() - if (NonFatal(e)) { - logError(s"Failed to launch executor $executorId on container $containerId", e) - // Assigned container should be released immediately - // to avoid unnecessary resource occupation. - amClient.releaseAssignedContainer(containerId) - } else { - throw e - } - } + launcherPool.execute(() => { + try { + new ExecutorRunnable( + Some(container), + conf, + sparkConf, + driverUrl, + executorId, + executorHostname, + executorMemory, + executorCores, + appAttemptId.getApplicationId.toString, + securityMgr, + localResources + ).run() + updateInternalState() + } catch { + case e: Throwable => + numExecutorsStarting.decrementAndGet() + if (NonFatal(e)) { + logError(s"Failed to launch executor $executorId on container $containerId", e) + // Assigned container should be released immediately + // to avoid unnecessary resource occupation. + amClient.releaseAssignedContainer(containerId) + } else { + throw e + } } }) } else { @@ -576,20 +626,38 @@ private[yarn] class YarnAllocator( (false, s"Container ${containerId}${onHostStr} was preempted.") // Should probably still count memory exceeded exit codes towards task failures case VMEM_EXCEEDED_EXIT_CODE => - (true, memLimitExceededLogMessage( - completedContainer.getDiagnostics, - VMEM_EXCEEDED_PATTERN)) + val vmemExceededPattern = raw"$MEM_REGEX of $MEM_REGEX virtual memory used".r + val diag = vmemExceededPattern.findFirstIn(completedContainer.getDiagnostics) + .map(_.concat(".")).getOrElse("") + val message = "Container killed by YARN for exceeding virtual memory limits. " + + s"$diag Consider boosting ${EXECUTOR_MEMORY_OVERHEAD.key} or boosting " + + s"${YarnConfiguration.NM_VMEM_PMEM_RATIO} or disabling " + + s"${YarnConfiguration.NM_VMEM_CHECK_ENABLED} because of YARN-4714." + (true, message) case PMEM_EXCEEDED_EXIT_CODE => - (true, memLimitExceededLogMessage( - completedContainer.getDiagnostics, - PMEM_EXCEEDED_PATTERN)) - case _ => - // all the failures which not covered above, like: - // disk failure, kill by app master or resource manager, ... - allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) - (true, "Container marked as failed: " + containerId + onHostStr + - ". Exit status: " + completedContainer.getExitStatus + - ". Diagnostics: " + completedContainer.getDiagnostics) + val pmemExceededPattern = raw"$MEM_REGEX of $MEM_REGEX physical memory used".r + val diag = pmemExceededPattern.findFirstIn(completedContainer.getDiagnostics) + .map(_.concat(".")).getOrElse("") + val message = "Container killed by YARN for exceeding physical memory limits. " + + s"$diag Consider boosting ${EXECUTOR_MEMORY_OVERHEAD.key}." + (true, message) + case other_exit_status => + // SPARK-26269: follow YARN's blacklisting behaviour(see https://github + // .com/apache/hadoop/blob/228156cfd1b474988bc4fedfbf7edddc87db41e3/had + // oop-yarn-project/hadoop-yarn/hadoop-yarn-common/src/main/java/org/ap + // ache/hadoop/yarn/util/Apps.java#L273 for details) + if (NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS.contains(other_exit_status)) { + (false, s"Container marked as failed: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } else { + // completed container from a bad node + allocatorBlacklistTracker.handleResourceAllocationFailure(hostOpt) + (true, s"Container from a bad node: $containerId$onHostStr" + + s". Exit status: ${completedContainer.getExitStatus}" + + s". Diagnostics: ${completedContainer.getDiagnostics}.") + } + } if (exitCausedByApp) { @@ -706,25 +774,21 @@ private[yarn] class YarnAllocator( } } - (localityMatched.toSeq, localityUnMatched.toSeq, localityFree.toSeq) + (localityMatched, localityUnMatched, localityFree) } } private object YarnAllocator { val MEM_REGEX = "[0-9.]+ [KMG]B" - val PMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX physical memory used") - val VMEM_EXCEEDED_PATTERN = - Pattern.compile(s"$MEM_REGEX of $MEM_REGEX virtual memory used") val VMEM_EXCEEDED_EXIT_CODE = -103 val PMEM_EXCEEDED_EXIT_CODE = -104 - def memLimitExceededLogMessage(diagnostics: String, pattern: Pattern): String = { - val matcher = pattern.matcher(diagnostics) - val diag = if (matcher.find()) " " + matcher.group() + "." else "" - s"Container killed by YARN for exceeding memory limits. $diag " + - "Consider boosting spark.yarn.executor.memoryOverhead or " + - "disabling yarn.nodemanager.vmem-check-enabled because of YARN-4714." - } + val NOT_APP_AND_SYSTEM_FAULT_EXIT_STATUS = Set( + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED + ) } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala index ceac7cda5f8be..fa8c9610220c8 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTracker.scala @@ -56,6 +56,8 @@ private[spark] class YarnAllocatorBlacklistTracker( private val maxFailuresPerHost = sparkConf.get(MAX_FAILED_EXEC_PER_NODE) + private val excludeNodes = sparkConf.get(YARN_EXCLUDE_NODES).toSet + private val allocatorBlacklist = new HashMap[String, Long]() private var currentBlacklistedYarnNodes = Set.empty[String] @@ -105,7 +107,7 @@ private[spark] class YarnAllocatorBlacklistTracker( private def refreshBlacklistedNodes(): Unit = { removeExpiredYarnBlacklistedNodes() - val allBlacklistedNodes = schedulerBlacklist ++ allocatorBlacklist.keySet + val allBlacklistedNodes = excludeNodes ++ schedulerBlacklist ++ allocatorBlacklist.keySet synchronizeBlacklistedNodeWithYarn(allBlacklistedNodes) } @@ -120,7 +122,9 @@ private[spark] class YarnAllocatorBlacklistTracker( if (removals.nonEmpty) { logInfo(s"removing nodes from YARN application master's blacklist: $removals") } - amClient.updateBlacklist(additions.asJava, removals.asJava) + if (additions.nonEmpty || removals.nonEmpty) { + amClient.updateBlacklist(additions.asJava, removals.asJava) + } currentBlacklistedYarnNodes = nodesToBlacklist } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala index 05a7b1e1310c4..7c67493c33160 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala @@ -76,13 +76,14 @@ private[spark] class YarnRMClient extends Logging { def createAllocator( conf: YarnConfiguration, sparkConf: SparkConf, + appAttemptId: ApplicationAttemptId, driverUrl: String, driverRef: RpcEndpointRef, securityMgr: SecurityManager, localResources: Map[String, LocalResource]): YarnAllocator = { require(registered, "Must register AM before creating allocator.") - new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr, - localResources, new SparkRackResolver()) + new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, appAttemptId, securityMgr, + localResources, SparkRackResolver.get(conf)) } /** @@ -100,11 +101,6 @@ private[spark] class YarnRMClient extends Logging { } } - /** Returns the attempt ID. */ - def getAttemptId(): ApplicationAttemptId = { - YarnSparkHadoopUtil.getContainerId.getApplicationAttemptId() - } - /** Returns the configuration for the AmIpFilter to add to the Spark UI. */ def getAmIpFilterParams(conf: YarnConfiguration, proxyBase: String): Map[String, String] = { // Figure out which scheme Yarn is using. Note the method seems to have been added after 2.2, diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 3a3272216294f..11035520ae185 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -21,15 +21,14 @@ import java.util.regex.{Matcher, Pattern} import scala.collection.mutable.{HashMap, ListBuffer} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.yarn.api.records.{ApplicationAccessType, ContainerId, Priority} import org.apache.hadoop.yarn.util.ConverterUtils import org.apache.spark.{SecurityManager, SparkConf} -import org.apache.spark.deploy.yarn.config._ import org.apache.spark.launcher.YarnCommandBuilderUtils +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils._ import org.apache.spark.util.Utils object YarnSparkHadoopUtil { @@ -185,40 +184,4 @@ object YarnSparkHadoopUtil { ConverterUtils.toContainerId(containerIdString) } - /** The filesystems for which YARN should fetch delegation tokens. */ - def hadoopFSsToAccess( - sparkConf: SparkConf, - hadoopConf: Configuration): Set[FileSystem] = { - val filesystemsToAccess = sparkConf.get(FILESYSTEMS_TO_ACCESS) - val requestAllDelegationTokens = filesystemsToAccess.isEmpty - - val stagingFS = sparkConf.get(STAGING_DIR) - .map(new Path(_).getFileSystem(hadoopConf)) - .getOrElse(FileSystem.get(hadoopConf)) - - // Add the list of available namenodes for all namespaces in HDFS federation. - // If ViewFS is enabled, this is skipped as ViewFS already handles delegation tokens for its - // namespaces. - val hadoopFilesystems = if (!requestAllDelegationTokens || stagingFS.getScheme == "viewfs") { - filesystemsToAccess.map(new Path(_).getFileSystem(hadoopConf)).toSet - } else { - val nameservices = hadoopConf.getTrimmedStrings("dfs.nameservices") - // Retrieving the filesystem for the nameservices where HA is not enabled - val filesystemsWithoutHA = nameservices.flatMap { ns => - Option(hadoopConf.get(s"dfs.namenode.rpc-address.$ns")).map { nameNode => - new Path(s"hdfs://$nameNode").getFileSystem(hadoopConf) - } - } - // Retrieving the filesystem for the nameservices where HA is enabled - val filesystemsWithHA = nameservices.flatMap { ns => - Option(hadoopConf.get(s"dfs.ha.namenodes.$ns")).map { _ => - new Path(s"hdfs://$ns").getFileSystem(hadoopConf) - } - } - (filesystemsWithoutHA ++ filesystemsWithHA).toSet - } - - hadoopFilesystems + stagingFS - } - } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala index ab8273bd6321d..4c187b2cc68e7 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/config.scala @@ -122,11 +122,6 @@ package object config { .intConf .createOptional - private[spark] val STAGING_DIR = ConfigBuilder("spark.yarn.stagingDir") - .doc("Staging directory used while submitting applications.") - .stringConf - .createOptional - /* Launcher configuration. */ private[spark] val WAIT_FOR_APP_COMPLETION = ConfigBuilder("spark.yarn.submit.waitAppCompletion") @@ -186,12 +181,6 @@ package object config { .timeConf(TimeUnit.MILLISECONDS) .createWithDefaultString("200ms") - private[spark] val SCHEDULER_SERVICES = ConfigBuilder("spark.yarn.services") - .doc("A comma-separated list of class names of services to add to the scheduler.") - .stringConf - .toSequence - .createWithDefault(Nil) - private[spark] val AM_FINAL_MSG_LIMIT = ConfigBuilder("spark.yarn.am.finalMessageLimit") .doc("The limit size of final diagnostic message for our ApplicationMaster to unregister from" + " the ResourceManager.") @@ -224,35 +213,25 @@ package object config { /* Driver configuration. */ - private[spark] val DRIVER_CORES = ConfigBuilder("spark.driver.cores") - .intConf - .createWithDefault(1) + private[spark] val DRIVER_APP_UI_ADDRESS = ConfigBuilder("spark.driver.appUIAddress") + .stringConf + .createOptional /* Executor configuration. */ - private[spark] val EXECUTOR_CORES = ConfigBuilder("spark.executor.cores") - .intConf - .createWithDefault(1) - private[spark] val EXECUTOR_NODE_LABEL_EXPRESSION = ConfigBuilder("spark.yarn.executor.nodeLabelExpression") .doc("Node label expression for executors.") .stringConf .createOptional - /* Security configuration. */ + /* Unmanaged AM configuration. */ - private[spark] val NAMENODES_TO_ACCESS = ConfigBuilder("spark.yarn.access.namenodes") - .doc("Extra NameNode URLs for which to request delegation tokens. The NameNode that hosts " + - "fs.defaultFS does not need to be listed here.") - .stringConf - .toSequence - .createWithDefault(Nil) - - private[spark] val FILESYSTEMS_TO_ACCESS = ConfigBuilder("spark.yarn.access.hadoopFileSystems") - .doc("Extra Hadoop filesystem URLs for which to request delegation tokens. The filesystem " + - "that hosts fs.defaultFS does not need to be listed here.") - .fallbackConf(NAMENODES_TO_ACCESS) + private[spark] val YARN_UNMANAGED_AM = ConfigBuilder("spark.yarn.unmanagedAM.enabled") + .doc("In client mode, whether to launch the Application Master service as part of the client " + + "using unmanaged am.") + .booleanConf + .createWithDefault(false) /* Rolled log aggregation configuration. */ @@ -325,24 +304,20 @@ package object config { .stringConf .createOptional - private[spark] val KERBEROS_RELOGIN_PERIOD = ConfigBuilder("spark.yarn.kerberos.relogin.period") - .timeConf(TimeUnit.SECONDS) - .createWithDefaultString("1m") - - // The list of cache-related config entries. This is used by Client and the AM to clean - // up the environment so that these settings do not appear on the web UI. - private[yarn] val CACHE_CONFIGS = Seq( - CACHED_FILES, - CACHED_FILES_SIZES, - CACHED_FILES_TIMESTAMPS, - CACHED_FILES_VISIBILITIES, - CACHED_FILES_TYPES, - CACHED_CONF_ARCHIVE) - /* YARN allocator-level blacklisting related config entries. */ private[spark] val YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED = ConfigBuilder("spark.yarn.blacklist.executor.launch.blacklisting.enabled") .booleanConf .createWithDefault(false) + /* Initially blacklisted YARN nodes. */ + private[spark] val YARN_EXCLUDE_NODES = ConfigBuilder("spark.yarn.exclude.nodes") + .stringConf + .toSequence + .createWithDefault(Nil) + + private[yarn] val YARN_EXECUTOR_RESOURCE_TYPES_PREFIX = "spark.yarn.executor.resource." + private[yarn] val YARN_DRIVER_RESOURCE_TYPES_PREFIX = "spark.yarn.driver.resource." + private[yarn] val YARN_AM_RESOURCE_TYPES_PREFIX = "spark.yarn.am.resource." + } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala deleted file mode 100644 index bc8d47dbd54c6..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/AMCredentialRenewer.scala +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.deploy.yarn.security - -import java.security.PrivilegedExceptionAction -import java.util.concurrent.{ScheduledExecutorService, TimeUnit} -import java.util.concurrent.atomic.AtomicReference - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.internal.config._ -import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages.UpdateDelegationTokens -import org.apache.spark.ui.UIUtils -import org.apache.spark.util.ThreadUtils - -/** - * A manager tasked with periodically updating delegation tokens needed by the application. - * - * This manager is meant to make sure long-running apps (such as Spark Streaming apps) can run - * without interruption while accessing secured services. It periodically logs in to the KDC with - * user-provided credentials, and contacts all the configured secure services to obtain delegation - * tokens to be distributed to the rest of the application. - * - * This class will manage the kerberos login, by renewing the TGT when needed. Because the UGI API - * does not expose the TTL of the TGT, a configuration controls how often to check that a relogin is - * necessary. This is done reasonably often since the check is a no-op when the relogin is not yet - * needed. The check period can be overridden in the configuration. - * - * New delegation tokens are created once 75% of the renewal interval of the original tokens has - * elapsed. The new tokens are sent to the Spark driver endpoint once it's registered with the AM. - * The driver is tasked with distributing the tokens to other processes that might need them. - */ -private[yarn] class AMCredentialRenewer( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { - - private val principal = sparkConf.get(PRINCIPAL).get - private val keytab = sparkConf.get(KEYTAB).get - private val credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - - private val renewalExecutor: ScheduledExecutorService = - ThreadUtils.newDaemonSingleThreadScheduledExecutor("Credential Refresh Thread") - - private val driverRef = new AtomicReference[RpcEndpointRef]() - - private val renewalTask = new Runnable() { - override def run(): Unit = { - updateTokensTask() - } - } - - def setDriverRef(ref: RpcEndpointRef): Unit = { - driverRef.set(ref) - } - - /** - * Start the token renewer. Upon start, the renewer will: - * - * - log in the configured user, and set up a task to keep that user's ticket renewed - * - obtain delegation tokens from all available providers - * - schedule a periodic task to update the tokens when needed. - * - * @return The newly logged in user. - */ - def start(): UserGroupInformation = { - val originalCreds = UserGroupInformation.getCurrentUser().getCredentials() - val ugi = doLogin() - - val tgtRenewalTask = new Runnable() { - override def run(): Unit = { - ugi.checkTGTAndReloginFromKeytab() - } - } - val tgtRenewalPeriod = sparkConf.get(KERBEROS_RELOGIN_PERIOD) - renewalExecutor.scheduleAtFixedRate(tgtRenewalTask, tgtRenewalPeriod, tgtRenewalPeriod, - TimeUnit.SECONDS) - - val creds = obtainTokensAndScheduleRenewal(ugi) - ugi.addCredentials(creds) - - // Transfer the original user's tokens to the new user, since that's needed to connect to - // YARN. Explicitly avoid overwriting tokens that already exist in the current user's - // credentials, since those were freshly obtained above (see SPARK-23361). - val existing = ugi.getCredentials() - existing.mergeAll(originalCreds) - ugi.addCredentials(existing) - - ugi - } - - def stop(): Unit = { - renewalExecutor.shutdown() - } - - private def scheduleRenewal(delay: Long): Unit = { - val _delay = math.max(0, delay) - logInfo(s"Scheduling login from keytab in ${UIUtils.formatDuration(delay)}.") - renewalExecutor.schedule(renewalTask, _delay, TimeUnit.MILLISECONDS) - } - - /** - * Periodic task to login to the KDC and create new delegation tokens. Re-schedules itself - * to fetch the next set of tokens when needed. - */ - private def updateTokensTask(): Unit = { - try { - val freshUGI = doLogin() - val creds = obtainTokensAndScheduleRenewal(freshUGI) - val tokens = SparkHadoopUtil.get.serialize(creds) - - val driver = driverRef.get() - if (driver != null) { - logInfo("Updating delegation tokens.") - driver.send(UpdateDelegationTokens(tokens)) - } else { - // This shouldn't really happen, since the driver should register way before tokens expire - // (or the AM should time out the application). - logWarning("Delegation tokens close to expiration but no driver has registered yet.") - SparkHadoopUtil.get.addDelegationTokens(tokens, sparkConf) - } - } catch { - case e: Exception => - val delay = TimeUnit.SECONDS.toMillis(sparkConf.get(CREDENTIALS_RENEWAL_RETRY_WAIT)) - logWarning(s"Failed to update tokens, will try again in ${UIUtils.formatDuration(delay)}!" + - " If this happens too often tasks will fail.", e) - scheduleRenewal(delay) - } - } - - /** - * Obtain new delegation tokens from the available providers. Schedules a new task to fetch - * new tokens before the new set expires. - * - * @return Credentials containing the new tokens. - */ - private def obtainTokensAndScheduleRenewal(ugi: UserGroupInformation): Credentials = { - ugi.doAs(new PrivilegedExceptionAction[Credentials]() { - override def run(): Credentials = { - val creds = new Credentials() - val nextRenewal = credentialManager.obtainDelegationTokens(hadoopConf, creds) - - val timeToWait = SparkHadoopUtil.nextCredentialRenewalTime(nextRenewal, sparkConf) - - System.currentTimeMillis() - scheduleRenewal(timeToWait) - creds - } - }) - } - - private def doLogin(): UserGroupInformation = { - logInfo(s"Attempting to login to KDC using principal: $principal") - val ugi = UserGroupInformation.loginUserFromKeytabAndReturnUGI(principal, keytab) - logInfo("Successfully logged into KDC.") - ugi - } - -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala deleted file mode 100644 index cc24ac4d9bcf6..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/ServiceCredentialProvider.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.{Credentials, UserGroupInformation} - -import org.apache.spark.SparkConf - -/** - * A credential provider for a service. User must implement this if they need to access a - * secure service from Spark. - */ -trait ServiceCredentialProvider { - - /** - * Name of the service to provide credentials. This name should unique, Spark internally will - * use this name to differentiate credential provider. - */ - def serviceName: String - - /** - * Returns true if credentials are required by this service. By default, it is based on whether - * Hadoop security is enabled. - */ - def credentialsRequired(hadoopConf: Configuration): Boolean = { - UserGroupInformation.isSecurityEnabled - } - - /** - * Obtain credentials for this service and get the time of the next renewal. - * - * @param hadoopConf Configuration of current Hadoop Compatible system. - * @param sparkConf Spark configuration. - * @param creds Credentials to add tokens and security keys to. - * @return If this Credential is renewable and can be renewed, return the time of the next - * renewal, otherwise None should be returned. - */ - def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala deleted file mode 100644 index 26a2e5d730218..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManager.scala +++ /dev/null @@ -1,86 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn.security - -import java.util.ServiceLoader - -import scala.collection.JavaConverters._ - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials - -import org.apache.spark.SparkConf -import org.apache.spark.deploy.security.HadoopDelegationTokenManager -import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils - -/** - * This class loads delegation token providers registered under the YARN-specific - * [[ServiceCredentialProvider]] interface, as well as the builtin providers defined - * in [[HadoopDelegationTokenManager]]. - */ -private[yarn] class YARNHadoopDelegationTokenManager( - sparkConf: SparkConf, - hadoopConf: Configuration) extends Logging { - - private val delegationTokenManager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, - conf => YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, conf)) - - // public for testing - val credentialProviders = getCredentialProviders - if (credentialProviders.nonEmpty) { - logDebug("Using the following YARN-specific credential providers: " + - s"${credentialProviders.keys.mkString(", ")}.") - } - - /** - * Writes delegation tokens to creds. Delegation tokens are fetched from all registered - * providers. - * - * @return Time after which the fetched delegation tokens should be renewed. - */ - def obtainDelegationTokens(hadoopConf: Configuration, creds: Credentials): Long = { - val superInterval = delegationTokenManager.obtainDelegationTokens(hadoopConf, creds) - - credentialProviders.values.flatMap { provider => - if (provider.credentialsRequired(hadoopConf)) { - provider.obtainCredentials(hadoopConf, sparkConf, creds) - } else { - logDebug(s"Service ${provider.serviceName} does not require a token." + - s" Check your configuration to see if security is disabled or not.") - None - } - }.foldLeft(superInterval)(math.min) - } - - private def getCredentialProviders: Map[String, ServiceCredentialProvider] = { - val providers = loadCredentialProviders - - providers. - filter { p => delegationTokenManager.isServiceEnabled(p.serviceName) } - .map { p => (p.serviceName, p) } - .toMap - } - - private def loadCredentialProviders: List[ServiceCredentialProvider] = { - ServiceLoader.load(classOf[ServiceCredentialProvider], Utils.getContextOrSparkClassLoader) - .asScala - .toList - } -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.scala new file mode 100644 index 0000000000000..2e5748b6144f9 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/executor/YarnCoarseGrainedExecutorBackend.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.executor + +import java.net.URL + +import org.apache.spark.SparkEnv +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.internal.Logging +import org.apache.spark.rpc.RpcEnv +import org.apache.spark.util.YarnContainerInfoHelper + +/** + * Custom implementation of CoarseGrainedExecutorBackend for YARN resource manager. + * This class extracts executor log URLs and executor attributes from system environment which + * properties are available for container being set via YARN. + */ +private[spark] class YarnCoarseGrainedExecutorBackend( + rpcEnv: RpcEnv, + driverUrl: String, + executorId: String, + hostname: String, + cores: Int, + userClassPath: Seq[URL], + env: SparkEnv, + resourcesFile: Option[String]) + extends CoarseGrainedExecutorBackend( + rpcEnv, + driverUrl, + executorId, + hostname, + cores, + userClassPath, + env, + resourcesFile) with Logging { + + private lazy val hadoopConfiguration = SparkHadoopUtil.get.newConfiguration(env.conf) + + override def extractLogUrls: Map[String, String] = { + YarnContainerInfoHelper.getLogUrls(hadoopConfiguration, container = None) + .getOrElse(Map()) + } + + override def extractAttributes: Map[String, String] = { + YarnContainerInfoHelper.getAttributes(hadoopConfiguration, container = None) + .getOrElse(Map()) + } +} + +private[spark] object YarnCoarseGrainedExecutorBackend extends Logging { + + def main(args: Array[String]): Unit = { + val createFn: (RpcEnv, CoarseGrainedExecutorBackend.Arguments, SparkEnv) => + CoarseGrainedExecutorBackend = { case (rpcEnv, arguments, env) => + new YarnCoarseGrainedExecutorBackend(rpcEnv, arguments.driverUrl, arguments.executorId, + arguments.hostname, arguments.cores, arguments.userClassPath, env, + arguments.resourcesFileOpt) + } + val backendArgs = CoarseGrainedExecutorBackend.parseArguments(args, + this.getClass.getCanonicalName.stripSuffix("$")) + CoarseGrainedExecutorBackend.run(backendArgs, createFn) + System.exit(0) + } + +} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala deleted file mode 100644 index 4ed285230ff81..0000000000000 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/SchedulerExtensionService.scala +++ /dev/null @@ -1,143 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster - -import java.util.concurrent.atomic.AtomicBoolean - -import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} - -import org.apache.spark.SparkContext -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging -import org.apache.spark.util.Utils - -/** - * An extension service that can be loaded into a Spark YARN scheduler. - * A Service that can be started and stopped. - * - * 1. For implementations to be loadable by `SchedulerExtensionServices`, - * they must provide an empty constructor. - * 2. The `stop()` operation MUST be idempotent, and succeed even if `start()` was - * never invoked. - */ -trait SchedulerExtensionService { - - /** - * Start the extension service. This should be a no-op if - * called more than once. - * @param binding binding to the spark application and YARN - */ - def start(binding: SchedulerExtensionServiceBinding): Unit - - /** - * Stop the service - * The `stop()` operation MUST be idempotent, and succeed even if `start()` was - * never invoked. - */ - def stop(): Unit -} - -/** - * Binding information for a [[SchedulerExtensionService]]. - * - * The attempt ID will be set if the service is started within a YARN application master; - * there is then a different attempt ID for every time that AM is restarted. - * When the service binding is instantiated in client mode, there's no attempt ID, as it lacks - * this information. - * @param sparkContext current spark context - * @param applicationId YARN application ID - * @param attemptId YARN attemptID. This will always be unset in client mode, and always set in - * cluster mode. - */ -case class SchedulerExtensionServiceBinding( - sparkContext: SparkContext, - applicationId: ApplicationId, - attemptId: Option[ApplicationAttemptId] = None) - -/** - * Container for [[SchedulerExtensionService]] instances. - * - * Loads Extension Services from the configuration property - * `"spark.yarn.services"`, instantiates and starts them. - * When stopped, it stops all child entries. - * - * The order in which child extension services are started and stopped - * is undefined. - */ -private[spark] class SchedulerExtensionServices extends SchedulerExtensionService - with Logging { - private var serviceOption: Option[String] = None - private var services: List[SchedulerExtensionService] = Nil - private val started = new AtomicBoolean(false) - private var binding: SchedulerExtensionServiceBinding = _ - - /** - * Binding operation will load the named services and call bind on them too; the - * entire set of services are then ready for `init()` and `start()` calls. - * - * @param binding binding to the spark application and YARN - */ - def start(binding: SchedulerExtensionServiceBinding): Unit = { - if (started.getAndSet(true)) { - logWarning("Ignoring re-entrant start operation") - return - } - require(binding.sparkContext != null, "Null context parameter") - require(binding.applicationId != null, "Null appId parameter") - this.binding = binding - val sparkContext = binding.sparkContext - val appId = binding.applicationId - val attemptId = binding.attemptId - logInfo(s"Starting Yarn extension services with app $appId and attemptId $attemptId") - - services = sparkContext.conf.get(SCHEDULER_SERVICES).map { sClass => - val instance = Utils.classForName(sClass) - .newInstance() - .asInstanceOf[SchedulerExtensionService] - // bind this service - instance.start(binding) - logInfo(s"Service $sClass started") - instance - }.toList - } - - /** - * Get the list of services. - * - * @return a list of services; Nil until the service is started - */ - def getServices: List[SchedulerExtensionService] = services - - /** - * Stop the services; idempotent. - * - */ - override def stop(): Unit = { - if (started.getAndSet(false)) { - logInfo(s"Stopping $this") - services.foreach { s => - Utils.tryLogNonFatalError(s.stop()) - } - } - } - - override def toString(): String = s"""SchedulerExtensionServices - |(serviceOption=$serviceOption, - | services=$services, - | started=$started)""".stripMargin -} diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 9397a1e3de9ac..c7c495fef853f 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -24,9 +24,10 @@ import org.apache.hadoop.yarn.api.records.YarnApplicationState import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.deploy.yarn.{Client, ClientArguments, YarnAppReport} import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging +import org.apache.spark.internal.{config, Logging} import org.apache.spark.launcher.SparkAppHandle import org.apache.spark.scheduler.TaskSchedulerImpl +import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ private[spark] class YarnClientSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -42,10 +43,12 @@ private[spark] class YarnClientSchedulerBackend( * This waits until the application is running. */ override def start() { - val driverHost = conf.get("spark.driver.host") - val driverPort = conf.get("spark.driver.port") + super.start() + + val driverHost = conf.get(config.DRIVER_HOST_ADDRESS) + val driverPort = conf.get(config.DRIVER_PORT) val hostport = driverHost + ":" + driverPort - sc.ui.foreach { ui => conf.set("spark.driver.appUIAddress", ui.webUrl) } + sc.ui.foreach { ui => conf.set(DRIVER_APP_UI_ADDRESS, ui.webUrl) } val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ("--arg", hostport) @@ -53,17 +56,15 @@ private[spark] class YarnClientSchedulerBackend( logDebug("ClientArguments called with: " + argsArrayBuf.mkString(" ")) val args = new ClientArguments(argsArrayBuf.toArray) totalExpectedExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(conf) - client = new Client(args, conf) + client = new Client(args, conf, sc.env.rpcEnv) bindToYarn(client.submitApplication(), None) - // SPARK-8687: Ensure all necessary properties have already been set before - // we initialize our driver scheduler backend, which serves these properties - // to the executors - super.start() waitForApplication() monitorThread = asyncMonitorApplication() monitorThread.start() + + startBindings() } /** @@ -166,4 +167,9 @@ private[spark] class YarnClientSchedulerBackend( logInfo("Stopped") } + override protected def updateDelegationTokens(tokens: Array[Byte]): Unit = { + super.updateDelegationTokens(tokens) + amEndpoint.foreach(_.send(UpdateDelegationTokens(tokens))) + } + } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala index 96c9151fc351d..1f622a02a62ae 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterScheduler.scala @@ -21,7 +21,7 @@ import org.apache.spark._ import org.apache.spark.deploy.yarn.ApplicationMaster /** - * This is a simple extension to ClusterScheduler - to ensure that appropriate initialization of + * This is a simple extension to YarnScheduler - to ensure that appropriate initialization of * ApplicationMaster, etc is done */ private[spark] class YarnClusterScheduler(sc: SparkContext) extends YarnScheduler(sc) { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala index 62bf9818ee248..b5575a10a05a0 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnClusterSchedulerBackend.scala @@ -17,13 +17,10 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment -import org.apache.hadoop.yarn.conf.YarnConfiguration - import org.apache.spark.SparkContext -import org.apache.spark.deploy.yarn.{ApplicationMaster, YarnSparkHadoopUtil} +import org.apache.spark.deploy.yarn.ApplicationMaster import org.apache.spark.scheduler.TaskSchedulerImpl -import org.apache.spark.util.Utils +import org.apache.spark.util.YarnContainerInfoHelper private[spark] class YarnClusterSchedulerBackend( scheduler: TaskSchedulerImpl, @@ -35,33 +32,14 @@ private[spark] class YarnClusterSchedulerBackend( bindToYarn(attemptId.getApplicationId(), Some(attemptId)) super.start() totalExpectedExecutors = SchedulerBackendUtils.getInitialTargetExecutorNumber(sc.conf) + startBindings() } override def getDriverLogUrls: Option[Map[String, String]] = { - var driverLogs: Option[Map[String, String]] = None - try { - val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) - val containerId = YarnSparkHadoopUtil.getContainerId + YarnContainerInfoHelper.getLogUrls(sc.hadoopConfiguration, container = None) + } - val httpAddress = System.getenv(Environment.NM_HOST.name()) + - ":" + System.getenv(Environment.NM_HTTP_PORT.name()) - // lookup appropriate http scheme for container log urls - val yarnHttpPolicy = yarnConf.get( - YarnConfiguration.YARN_HTTP_POLICY_KEY, - YarnConfiguration.YARN_HTTP_POLICY_DEFAULT - ) - val user = Utils.getCurrentUserName() - val httpScheme = if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" - val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" - logDebug(s"Base URL for logs: $baseUrl") - driverLogs = Some(Map( - "stdout" -> s"$baseUrl/stdout?start=-4096", - "stderr" -> s"$baseUrl/stderr?start=-4096")) - } catch { - case e: Exception => - logInfo("Error while building AM log links, so AM" + - " logs link will not appear in application UI", e) - } - driverLogs + override def getDriverAttributes: Option[Map[String, String]] = { + YarnContainerInfoHelper.getAttributes(sc.hadoopConfiguration, container = None) } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala index 029382133ddf2..d466ed77a929e 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnScheduler.scala @@ -17,23 +17,23 @@ package org.apache.spark.scheduler.cluster -import org.apache.hadoop.yarn.util.RackResolver -import org.apache.log4j.{Level, Logger} +import org.apache.hadoop.net.NetworkTopology import org.apache.spark._ +import org.apache.spark.deploy.yarn.SparkRackResolver import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.util.Utils private[spark] class YarnScheduler(sc: SparkContext) extends TaskSchedulerImpl(sc) { - // RackResolver logs an INFO message whenever it resolves a rack, which is way too often. - if (Logger.getLogger(classOf[RackResolver]).getLevel == null) { - Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN) - } + override val defaultRackValue: Option[String] = Some(NetworkTopology.DEFAULT_RACK) + + private[spark] val resolver = SparkRackResolver.get(sc.hadoopConfiguration) - // By default, rack is unknown - override def getRackForHost(hostPort: String): Option[String] = { - val host = Utils.parseHostPort(hostPort)._1 - Option(RackResolver.resolve(sc.hadoopConfiguration, host).getNetworkLocation) + override def getRacksForHosts(hostPorts: Seq[String]): Seq[Option[String]] = { + val hosts = hostPorts.map(Utils.parseHostPort(_)._1) + resolver.resolve(hosts).map { node => + Option(node.getNetworkLocation) + } } } diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala index 63bea3e7a5003..dda8172fb6369 100644 --- a/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackend.scala @@ -17,23 +17,26 @@ package org.apache.spark.scheduler.cluster +import java.util.EnumSet import java.util.concurrent.atomic.{AtomicBoolean} +import javax.servlet.DispatcherType -import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.ExecutionContext.Implicits.global +import scala.concurrent.Future import scala.util.{Failure, Success} import scala.util.control.NonFatal -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} +import org.eclipse.jetty.servlet.{FilterHolder, FilterMapping} import org.apache.spark.SparkContext -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.security.HadoopDelegationTokenManager import org.apache.spark.internal.Logging +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI._ import org.apache.spark.rpc._ import org.apache.spark.scheduler._ import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ -import org.apache.spark.ui.JettyUtils import org.apache.spark.util.{RpcUtils, ThreadUtils} /** @@ -48,7 +51,7 @@ private[spark] abstract class YarnSchedulerBackend( private val stopped = new AtomicBoolean(false) override val minRegisteredRatio = - if (conf.getOption("spark.scheduler.minRegisteredResourcesRatio").isEmpty) { + if (conf.get(config.SCHEDULER_MIN_REGISTERED_RESOURCES_RATIO).isEmpty) { 0.8 } else { super.minRegisteredRatio @@ -57,6 +60,7 @@ private[spark] abstract class YarnSchedulerBackend( protected var totalExpectedExecutors = 0 private val yarnSchedulerEndpoint = new YarnSchedulerEndpoint(rpcEnv) + protected var amEndpoint: Option[RpcEndpointRef] = None private val yarnSchedulerEndpointRef = rpcEnv.setupEndpoint( YarnSchedulerBackend.ENDPOINT_NAME, yarnSchedulerEndpoint) @@ -69,10 +73,6 @@ private[spark] abstract class YarnSchedulerBackend( /** Attempt ID. This is unset for client-mode schedulers */ private var attemptId: Option[ApplicationAttemptId] = None - /** Scheduler extension services. */ - private val services: SchedulerExtensionServices = new SchedulerExtensionServices() - - /** * Bind to YARN. This *must* be done before calling [[start()]]. * @@ -84,11 +84,8 @@ private[spark] abstract class YarnSchedulerBackend( this.attemptId = attemptId } - override def start() { + protected def startBindings(): Unit = { require(appId.isDefined, "application ID unset") - val binding = SchedulerExtensionServiceBinding(sc, appId.get, attemptId) - services.start(binding) - super.start() } override def stop(): Unit = { @@ -99,7 +96,6 @@ private[spark] abstract class YarnSchedulerBackend( super.stop() } finally { stopped.set(true) - services.stop() } } @@ -158,7 +154,7 @@ private[spark] abstract class YarnSchedulerBackend( /** * Add filters to the SparkUI. */ - private def addWebUIFilter( + private[cluster] def addWebUIFilter( filterName: String, filterParams: Map[String, String], proxyBase: String): Unit = { @@ -170,15 +166,31 @@ private[spark] abstract class YarnSchedulerBackend( filterName != null && filterName.nonEmpty && filterParams != null && filterParams.nonEmpty if (hasFilter) { + // SPARK-26255: Append user provided filters(spark.ui.filters) with yarn filter. + val allFilters = Seq(filterName) ++ conf.get(UI_FILTERS) logInfo(s"Add WebUI Filter. $filterName, $filterParams, $proxyBase") - conf.set("spark.ui.filters", filterName) - filterParams.foreach { case (k, v) => conf.set(s"spark.$filterName.param.$k", v) } - scheduler.sc.ui.foreach { ui => JettyUtils.addFilters(ui.getHandlers, conf) } + + // For already installed handlers, prepend the filter. + scheduler.sc.ui.foreach { ui => + // Lock the UI so that new handlers are not added while this is running. Set the updated + // filter config inside the lock so that we're sure all handlers will properly get it. + ui.synchronized { + filterParams.foreach { case (k, v) => + conf.set(s"spark.$filterName.param.$k", v) + } + conf.set(UI_FILTERS, allFilters) + + ui.getDelegatingHandlers.foreach { h => + h.addFilter(filterName, filterName, filterParams) + h.prependFilterMapping(filterName, "/*", EnumSet.allOf(classOf[DispatcherType])) + } + } + } } } - override def createDriverEndpoint(properties: Seq[(String, String)]): DriverEndpoint = { - new YarnDriverEndpoint(rpcEnv, properties) + override def createDriverEndpoint(): DriverEndpoint = { + new YarnDriverEndpoint() } /** @@ -191,13 +203,16 @@ private[spark] abstract class YarnSchedulerBackend( sc.executorAllocationManager.foreach(_.reset()) } + override protected def createTokenManager(): Option[HadoopDelegationTokenManager] = { + Some(new HadoopDelegationTokenManager(sc.conf, sc.hadoopConfiguration, driverEndpoint)) + } + /** * Override the DriverEndpoint to add extra logic for the case when an executor is disconnected. * This endpoint communicates with the executors and queries the AM for an executor's exit * status when the executor is disconnected. */ - private class YarnDriverEndpoint(rpcEnv: RpcEnv, sparkProperties: Seq[(String, String)]) - extends DriverEndpoint(rpcEnv, sparkProperties) { + private class YarnDriverEndpoint extends DriverEndpoint { /** * When onDisconnected is received at the driver endpoint, the superclass DriverEndpoint @@ -226,7 +241,6 @@ private[spark] abstract class YarnSchedulerBackend( */ private class YarnSchedulerEndpoint(override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { - private var amEndpoint: Option[RpcEndpointRef] = None private[YarnSchedulerBackend] def handleExecutorDisconnectedFromDriver( executorId: String, @@ -266,12 +280,6 @@ private[spark] abstract class YarnSchedulerBackend( logWarning(s"Requesting driver to remove executor $executorId for reason $reason") driverEndpoint.send(r) } - - case u @ UpdateDelegationTokens(tokens) => - // Add the tokens to the current user and send a message to the scheduler so that it - // notifies all registered executors of the new tokens. - SparkHadoopUtil.get.addDelegationTokens(tokens, sc.conf) - driverEndpoint.send(u) } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { @@ -305,6 +313,9 @@ private[spark] abstract class YarnSchedulerBackend( case RetrieveLastAllocatedExecutorId => context.reply(currentExecutorIdCounter) + + case RetrieveDelegationTokens => + context.reply(currentDelegationTokens) } override def onDisconnected(remoteAddress: RpcAddress): Unit = { diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala new file mode 100644 index 0000000000000..5e39422e868b7 --- /dev/null +++ b/resource-managers/yarn/src/main/scala/org/apache/spark/util/YarnContainerInfoHelper.scala @@ -0,0 +1,121 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.util + +import org.apache.hadoop.HadoopIllegalArgumentException +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.yarn.api.ApplicationConstants.Environment +import org.apache.hadoop.yarn.api.records.{Container, ContainerId} +import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils + +import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil +import org.apache.spark.internal.Logging + +private[spark] object YarnContainerInfoHelper extends Logging { + def getLogUrls( + conf: Configuration, + container: Option[Container]): Option[Map[String, String]] = { + try { + val yarnConf = new YarnConfiguration(conf) + + val containerId = getContainerId(container) + val user = Utils.getCurrentUserName() + val httpScheme = getYarnHttpScheme(yarnConf) + val httpAddress = getNodeManagerHttpAddress(container) + + val baseUrl = s"$httpScheme$httpAddress/node/containerlogs/$containerId/$user" + logDebug(s"Base URL for logs: $baseUrl") + + Some(Map( + "stdout" -> s"$baseUrl/stdout?start=-4096", + "stderr" -> s"$baseUrl/stderr?start=-4096")) + } catch { + case e: Exception => + logInfo("Error while building executor logs - executor logs will not be available", e) + None + } + } + + def getAttributes( + conf: Configuration, + container: Option[Container]): Option[Map[String, String]] = { + try { + val yarnConf = new YarnConfiguration(conf) + Some(Map( + "HTTP_SCHEME" -> getYarnHttpScheme(yarnConf), + "NM_HOST" -> getNodeManagerHost(container), + "NM_PORT" -> getNodeManagerPort(container), + "NM_HTTP_PORT" -> getNodeManagerHttpPort(container), + "NM_HTTP_ADDRESS" -> getNodeManagerHttpAddress(container), + "CLUSTER_ID" -> getClusterId(yarnConf).getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(getContainerId(container)), + "USER" -> Utils.getCurrentUserName(), + "LOG_FILES" -> "stderr,stdout" + )) + } catch { + case e: Exception => + logInfo("Error while retrieving executor attributes - executor logs will not be replaced " + + "with custom log pattern", e) + None + } + } + + def getContainerId(container: Option[Container]): ContainerId = container match { + case Some(c) => c.getId + case None => YarnSparkHadoopUtil.getContainerId + } + + def getClusterId(yarnConf: YarnConfiguration): Option[String] = { + try { + Some(YarnConfiguration.getClusterId(yarnConf)) + } catch { + case _: HadoopIllegalArgumentException => None + } + } + + def getYarnHttpScheme(yarnConf: YarnConfiguration): String = { + // lookup appropriate http scheme for container log urls + val yarnHttpPolicy = yarnConf.get( + YarnConfiguration.YARN_HTTP_POLICY_KEY, + YarnConfiguration.YARN_HTTP_POLICY_DEFAULT + ) + if (yarnHttpPolicy == "HTTPS_ONLY") "https://" else "http://" + } + + def getNodeManagerHttpAddress(container: Option[Container]): String = container match { + case Some(c) => c.getNodeHttpAddress + case None => getNodeManagerHost(None) + ":" + getNodeManagerHttpPort(None) + } + + def getNodeManagerHost(container: Option[Container]): String = container match { + case Some(c) => c.getNodeHttpAddress.split(":")(0) + case None => System.getenv(Environment.NM_HOST.name()) + } + + def getNodeManagerHttpPort(container: Option[Container]): String = container match { + case Some(c) => c.getNodeHttpAddress.split(":")(1) + case None => System.getenv(Environment.NM_HTTP_PORT.name()) + } + + def getNodeManagerPort(container: Option[Container]): String = container match { + case Some(_) => "-1" // Just return invalid port given we cannot retrieve the value + case None => System.getenv(Environment.NM_PORT.name()) + } + +} diff --git a/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java b/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java new file mode 100644 index 0000000000000..df0ebcc9871ac --- /dev/null +++ b/resource-managers/yarn/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.net; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.ServerSocket; +import java.util.Random; + +/** + * Copied from + * hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/net/ServerSocketUtil.java + * for Hadoop-3.x testing + */ +public class ServerSocketUtil { + + private static final Logger LOG = LoggerFactory.getLogger(ServerSocketUtil.class); + private static Random rand = new Random(); + + /** + * Port scan & allocate is how most other apps find ports + * + * @param port given port + * @param retries number of retries + * @return + * @throws IOException + */ + public static int getPort(int port, int retries) throws IOException { + int tryPort = port; + int tries = 0; + while (true) { + if (tries > 0 || tryPort == 0) { + tryPort = port + rand.nextInt(65535 - port); + } + if (tryPort == 0) { + continue; + } + try (ServerSocket s = new ServerSocket(tryPort)) { + LOG.info("Using port " + tryPort); + return tryPort; + } catch (IOException e) { + tries++; + if (tries >= retries) { + LOG.info("Port is already in use; giving up"); + throw e; + } else { + LOG.info("Port is already in use; trying again"); + } + } + } + } + + /** + * Check whether port is available or not. + * + * @param port given port + * @return + */ + private static boolean isPortAvailable(int port) { + try (ServerSocket s = new ServerSocket(port)) { + return true; + } catch (IOException e) { + return false; + } + } + + /** + * Wait till the port available. + * + * @param port given port + * @param retries number of retries for given port + * @return + * @throws InterruptedException + * @throws IOException + */ + public static int waitForPort(int port, int retries) + throws InterruptedException, IOException { + int tries = 0; + while (true) { + if (isPortAvailable(port)) { + return port; + } else { + tries++; + if (tries >= retries) { + throw new IOException( + "Port is already in use; giving up after " + tries + " times."); + } + Thread.sleep(1000); + } + } + } + + /** + * Find the specified number of unique ports available. + * The ports are all closed afterwards, + * so other network services started may grab those same ports. + * + * @param numPorts number of required port nubmers + * @return array of available port numbers + * @throws IOException + */ + public static int[] getPorts(int numPorts) throws IOException { + ServerSocket[] sockets = new ServerSocket[numPorts]; + int[] ports = new int[numPorts]; + for (int i = 0; i < numPorts; i++) { + ServerSocket sock = new ServerSocket(0); + sockets[i] = sock; + ports[i] = sock.getLocalPort(); + } + for (ServerSocket sock : sockets) { + sock.close(); + } + return ports; + } +} diff --git a/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/SessionManager.java b/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/SessionManager.java new file mode 100644 index 0000000000000..bf89f8d7c804a --- /dev/null +++ b/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/SessionManager.java @@ -0,0 +1,290 @@ +// +// ======================================================================== +// Copyright (c) 1995-2018 Mort Bay Consulting Pty. Ltd. +// ------------------------------------------------------------------------ +// All rights reserved. This program and the accompanying materials +// are made available under the terms of the Eclipse Public License v1.0 +// and Apache License v2.0 which accompanies this distribution. +// +// The Eclipse Public License is available at +// http://www.eclipse.org/legal/epl-v10.html +// +// The Apache License v2.0 is available at +// http://www.opensource.org/licenses/apache2.0.php +// +// You may elect to redistribute this code under either of these licenses. +// ======================================================================== +// + +package org.eclipse.jetty.server; + +import javax.servlet.SessionCookieConfig; +import javax.servlet.SessionTrackingMode; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpSession; +import java.util.EventListener; +import java.util.Set; + +import org.eclipse.jetty.http.HttpCookie; +import org.eclipse.jetty.server.session.SessionHandler; +import org.eclipse.jetty.util.component.LifeCycle; + +/** + * Adapted from https://github.com/eclipse/jetty.project/blob/jetty-9.3.25.v20180904/ + * jetty-server/src/main/java/org/eclipse/jetty/server/SessionManager.java + */ +public interface SessionManager extends LifeCycle { + /** + * Session cookie name. + * Defaults to JSESSIONID, but can be set with the + * org.eclipse.jetty.servlet.SessionCookie context init parameter. + */ + String __SessionCookieProperty = "org.eclipse.jetty.servlet.SessionCookie"; + String __DefaultSessionCookie = "JSESSIONID"; + + /** + * Session id path parameter name. + * Defaults to jsessionid, but can be set with the + * org.eclipse.jetty.servlet.SessionIdPathParameterName context init parameter. + * If set to null or "none" no URL rewriting will be done. + */ + String __SessionIdPathParameterNameProperty = + "org.eclipse.jetty.servlet.SessionIdPathParameterName"; + String __DefaultSessionIdPathParameterName = "jsessionid"; + String __CheckRemoteSessionEncoding = "org.eclipse.jetty.servlet.CheckingRemoteSessionIdEncoding"; + + /** + * Session Domain. + * If this property is set as a ServletContext InitParam, then it is + * used as the domain for session cookies. If it is not set, then + * no domain is specified for the session cookie. + */ + String __SessionDomainProperty = "org.eclipse.jetty.servlet.SessionDomain"; + String __DefaultSessionDomain = null; + + /** + * Session Path. + * If this property is set as a ServletContext InitParam, then it is + * used as the path for the session cookie. If it is not set, then + * the context path is used as the path for the cookie. + */ + String __SessionPathProperty = "org.eclipse.jetty.servlet.SessionPath"; + + /** + * Session Max Age. + * If this property is set as a ServletContext InitParam, then it is + * used as the max age for the session cookie. If it is not set, then + * a max age of -1 is used. + */ + String __MaxAgeProperty = "org.eclipse.jetty.servlet.MaxAge"; + + /** + * Returns the HttpSession with the given session id + * + * @param id the session id + * @return the HttpSession with the corresponding id + * or null if no session with the given id exists + */ + HttpSession getHttpSession(String id); + + /** + * Creates a new HttpSession. + * + * @param request the HttpServletRequest containing the requested session id + * @return the new HttpSession + */ + HttpSession newHttpSession(HttpServletRequest request); + + /** + * @return true if session cookies should be HTTP-only (Microsoft extension) + * @see HttpCookie#isHttpOnly() + */ + boolean getHttpOnly(); + + /** + * @return the max period of inactivity, after which the session is invalidated, in seconds. + * @see #setMaxInactiveInterval(int) + */ + int getMaxInactiveInterval(); + + /** + * Sets the max period of inactivity, after which the session is invalidated, in seconds. + * + * @param seconds the max inactivity period, in seconds. + * @see #getMaxInactiveInterval() + */ + void setMaxInactiveInterval(int seconds); + + /** + * Sets the {@link SessionHandler}. + * + * @param handler the SessionHandler object + */ + void setSessionHandler(SessionHandler handler); + + /** + * Adds an event listener for session-related events. + * + * @param listener the session event listener to add + * Individual SessionManagers implementations may accept arbitrary listener types, + * but they are expected to at least handle HttpSessionActivationListener, + * HttpSessionAttributeListener, + * HttpSessionBindingListener and HttpSessionListener. + * @see #removeEventListener(EventListener) + */ + void addEventListener(EventListener listener); + + /** + * Removes an event listener for for session-related events. + * + * @param listener the session event listener to remove + * @see #addEventListener(EventListener) + */ + void removeEventListener(EventListener listener); + + /** + * Removes all event listeners for session-related events. + * + * @see #removeEventListener(EventListener) + */ + void clearEventListeners(); + + /** + * Gets a Cookie for a session. + * + * @param session the session to which the cookie should refer. + * @param contextPath the context to which the cookie should be linked. + * The client will only send the cookie value when + * requesting resources under this path. + * @param requestIsSecure whether the client is accessing the server over + * a secure protocol (i.e. HTTPS). + * @return if this SessionManager uses cookies, then this method will return a new + * {@link Cookie cookie object} that should be set on the client + * in order to link future HTTP requests + * with the session. If cookies are not in use, + * this method returns null. + */ + HttpCookie getSessionCookie(HttpSession session, String contextPath, boolean requestIsSecure); + + /** + * @return the cross context session id manager. + * @see #setSessionIdManager(SessionIdManager) + */ + SessionIdManager getSessionIdManager(); + + /** + * @return the cross context session id manager. + * @deprecated use {@link #getSessionIdManager()} + */ + @Deprecated + SessionIdManager getMetaManager(); + + /** + * Sets the cross context session id manager + * + * @param idManager the cross context session id manager. + * @see #getSessionIdManager() + */ + void setSessionIdManager(SessionIdManager idManager); + + /** + * @param session the session to test for validity + * @return whether the given session is valid, that is, it has not been invalidated. + */ + boolean isValid(HttpSession session); + + /** + * @param session the session object + * @return the unique id of the session within the cluster, extended with an optional node id. + * @see #getClusterId(HttpSession) + */ + String getNodeId(HttpSession session); + + /** + * @param session the session object + * @return the unique id of the session within the cluster (without a node id extension) + * @see #getNodeId(HttpSession) + */ + String getClusterId(HttpSession session); + + /** + * Called by the {@link SessionHandler} when a session is first accessed by a request. + * + * @param session the session object + * @param secure whether the request is secure or not + * @return the session cookie. If not null, + * this cookie should be set on the response to either migrate + * the session or to refresh a session cookie that may expire. + * @see #complete(HttpSession) + */ + HttpCookie access(HttpSession session, boolean secure); + + /** + * Called by the {@link SessionHandler} when a session is last accessed by a request. + * + * @param session the session object + * @see #access(HttpSession, boolean) + */ + void complete(HttpSession session); + + /** + * Sets the session id URL path parameter name. + * + * @param parameterName the URL path parameter name + * for session id URL rewriting (null or "none" for no rewriting). + * @see #getSessionIdPathParameterName() + * @see #getSessionIdPathParameterNamePrefix() + */ + void setSessionIdPathParameterName(String parameterName); + + /** + * @return the URL path parameter name for session id URL rewriting, by default "jsessionid". + * @see #setSessionIdPathParameterName(String) + */ + String getSessionIdPathParameterName(); + + /** + * @return a formatted version of {@link #getSessionIdPathParameterName()}, by default + * ";" + sessionIdParameterName + "=", for easier lookup in URL strings. + * @see #getSessionIdPathParameterName() + */ + String getSessionIdPathParameterNamePrefix(); + + /** + * @return whether the session management is handled via cookies. + */ + boolean isUsingCookies(); + + /** + * @return whether the session management is handled via URLs. + */ + boolean isUsingURLs(); + + Set getDefaultSessionTrackingModes(); + + Set getEffectiveSessionTrackingModes(); + + void setSessionTrackingModes(Set sessionTrackingModes); + + SessionCookieConfig getSessionCookieConfig(); + + /** + * @return True if absolute URLs are check for remoteness before being session encoded. + */ + boolean isCheckingRemoteSessionIdEncoding(); + + /** + * @param remote True if absolute URLs are check for remoteness before being session encoded. + */ + void setCheckingRemoteSessionIdEncoding(boolean remote); + + /** Change the existing session id. + * + * @param oldClusterId the old cluster id + * @param oldNodeId the old node id + * @param newClusterId the new cluster id + * @param newNodeId the new node id + */ + void renewSessionId(String oldClusterId, String oldNodeId, String newClusterId, String newNodeId); +} diff --git a/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/session/SessionHandler.java b/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/session/SessionHandler.java new file mode 100644 index 0000000000000..f8bcf8de82f65 --- /dev/null +++ b/resource-managers/yarn/src/test/java/org/eclipse/jetty/server/session/SessionHandler.java @@ -0,0 +1,90 @@ +// +// ======================================================================== +// Copyright (c) 1995-2018 Mort Bay Consulting Pty. Ltd. +// ------------------------------------------------------------------------ +// All rights reserved. This program and the accompanying materials +// are made available under the terms of the Eclipse Public License v1.0 +// and Apache License v2.0 which accompanies this distribution. +// +// The Eclipse Public License is available at +// http://www.eclipse.org/legal/epl-v10.html +// +// The Apache License v2.0 is available at +// http://www.opensource.org/licenses/apache2.0.php +// +// You may elect to redistribute this code under either of these licenses. +// ======================================================================== +// + +package org.eclipse.jetty.server.session; + +import javax.servlet.ServletException; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import java.io.IOException; + +import org.eclipse.jetty.server.Request; +import org.eclipse.jetty.server.SessionManager; +import org.eclipse.jetty.server.handler.ScopedHandler; + +/** + * Adapted from https://github.com/eclipse/jetty.project/blob/jetty-9.3.25.v20180904/ + * jetty-server/src/main/java/org/eclipse/jetty/server/session/SessionHandler.java + */ +public class SessionHandler extends ScopedHandler { + private SessionManager _sessionManager; + + public SessionHandler() { + } + + /** + * @param manager + * The session manager + */ + public SessionHandler(SessionManager manager) { + setSessionManager(manager); + } + + /** + * @return Returns the sessionManager. + */ + public SessionManager getSessionManager() { + return _sessionManager; + } + + /** + * @param sessionManager + * The sessionManager to set. + */ + public void setSessionManager(SessionManager sessionManager) { + if (isStarted()) { + throw new IllegalStateException(); + } + if (sessionManager != null) { + updateBean(_sessionManager,sessionManager); + _sessionManager=sessionManager; + } + } + + /* + * @see org.eclipse.jetty.server.Handler#handle(javax.servlet.http.HttpServletRequest, + * javax.servlet.http.HttpServletResponse, int) + */ + @Override + public void doHandle(String target, Request baseRequest, HttpServletRequest request, + HttpServletResponse response) throws IOException, ServletException { + // start manual inline of nextHandle(target,baseRequest,request,response); + if (_nextScope != null && _nextScope == _handler) { + _nextScope.doHandle(target,baseRequest,request,response); + } else if (_handler != null) { + _handler.handle(target,baseRequest,request,response); + // end manual inline + } + } + + public void clearEventListeners() { + if (_sessionManager != null) { + _sessionManager.clearEventListeners(); + } + } +} diff --git a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider b/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider deleted file mode 100644 index f31c232693133..0000000000000 --- a/resource-managers/yarn/src/test/resources/META-INF/services/org.apache.spark.deploy.yarn.security.ServiceCredentialProvider +++ /dev/null @@ -1 +0,0 @@ -org.apache.spark.deploy.yarn.security.YARNTestCredentialProvider diff --git a/resource-managers/yarn/src/test/resources/log4j.properties b/resource-managers/yarn/src/test/resources/log4j.properties index d13454d5ae5d5..f2f3d17efa121 100644 --- a/resource-managers/yarn/src/test/resources/log4j.properties +++ b/resource-managers/yarn/src/test/resources/log4j.properties @@ -28,4 +28,4 @@ log4j.logger.com.sun.jersey=WARN log4j.logger.org.apache.hadoop=WARN log4j.logger.org.eclipse.jetty=WARN log4j.logger.org.mortbay=WARN -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ApplicationMasterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ApplicationMasterSuite.scala index 695a82f3583e6..d9bdace6caa69 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ApplicationMasterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ApplicationMasterSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.yarn import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.yarn.config._ class ApplicationMasterSuite extends SparkFunSuite { @@ -28,7 +29,7 @@ class ApplicationMasterSuite extends SparkFunSuite { val port = 18080 val sparkConf = new SparkConf() - sparkConf.set("spark.yarn.historyServer.address", + sparkConf.set(HISTORY_SERVER_ADDRESS, "http://${hadoopconf-yarn.resourcemanager.hostname}:${spark.history.ui.port}") val yarnConf = new YarnConfiguration() yarnConf.set("yarn.resourcemanager.hostname", host) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala index 3a7913122dd83..e16857e84887c 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/BaseYarnClusterSuite.scala @@ -24,10 +24,8 @@ import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.concurrent.duration._ -import scala.language.postfixOps import com.google.common.io.Files -import org.apache.commons.lang3.SerializationUtils import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.server.MiniYARNCluster import org.scalatest.{BeforeAndAfterAll, Matchers} @@ -54,7 +52,7 @@ abstract class BaseYarnClusterSuite |log4j.logger.org.apache.hadoop=WARN |log4j.logger.org.eclipse.jetty=WARN |log4j.logger.org.mortbay=WARN - |log4j.logger.org.spark_project.jetty=WARN + |log4j.logger.org.sparkproject.jetty=WARN """.stripMargin private var yarnCluster: MiniYARNCluster = _ @@ -100,9 +98,9 @@ abstract class BaseYarnClusterSuite // This hack loops for a bit waiting for the port to change, and fails the test if it hasn't // done so in a timely manner (defined to be 10 seconds). val config = yarnCluster.getConfig() - val deadline = System.currentTimeMillis() + TimeUnit.SECONDS.toMillis(10) + val startTimeNs = System.nanoTime() while (config.get(YarnConfiguration.RM_ADDRESS).split(":")(1) == "0") { - if (System.currentTimeMillis() > deadline) { + if (System.nanoTime() - startTimeNs > TimeUnit.SECONDS.toNanos(10)) { throw new IllegalStateException("Timed out waiting for RM to come up.") } logDebug("RM address still not set in configuration, waiting...") @@ -149,7 +147,7 @@ abstract class BaseYarnClusterSuite launcher.setSparkHome(sys.props("spark.test.home")) .setMaster("yarn") .setDeployMode(deployMode) - .setConf("spark.executor.instances", "1") + .setConf(EXECUTOR_INSTANCES.key, "1") .setPropertiesFile(propsFile) .addAppArgs(appArgs.toArray: _*) @@ -169,7 +167,7 @@ abstract class BaseYarnClusterSuite val handle = launcher.startApplication() try { - eventually(timeout(2 minutes), interval(1 second)) { + eventually(timeout(3.minutes), interval(1.second)) { assert(handle.getState().isFinal()) } } finally { diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala index 26013a109c42b..847fc3773de59 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ClientSuite.scala @@ -33,15 +33,20 @@ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.YarnClientApplication import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.util.Records -import org.mockito.Matchers.{eq => meq, _} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyBoolean, anyShort, eq => meq} +import org.mockito.Mockito.{spy, verify} import org.scalatest.Matchers -import org.apache.spark.{SparkConf, SparkFunSuite, TestUtils} +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite, TestUtils} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceID +import org.apache.spark.resource.ResourceUtils.AMOUNT import org.apache.spark.util.{SparkConfWithEnv, Utils} class ClientSuite extends SparkFunSuite with Matchers { + private def doReturn(value: Any) = org.mockito.Mockito.doReturn(value, Seq.empty: _*) import Client._ @@ -99,7 +104,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val cp = env("CLASSPATH").split(":|;|") s"$SPARK,$USER,$ADDED".split(",").foreach({ entry => val uri = new URI(entry) - if (LOCAL_SCHEME.equals(uri.getScheme())) { + if (Utils.LOCAL_SCHEME.equals(uri.getScheme())) { cp should contain (uri.getPath()) } else { cp should not contain (uri.getPath()) @@ -135,7 +140,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val expected = ADDED.split(",") .map(p => { val uri = new URI(p) - if (LOCAL_SCHEME == uri.getScheme()) { + if (Utils.LOCAL_SCHEME == uri.getScheme()) { p } else { Option(uri.getFragment()).getOrElse(new File(p).getName()) @@ -182,7 +187,7 @@ class ClientSuite extends SparkFunSuite with Matchers { val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) - val client = new Client(args, sparkConf) + val client = new Client(args, sparkConf, null) client.createApplicationSubmissionContext( new YarnClientApplication(getNewApplicationResponse, appContext), containerLaunchContext) @@ -248,7 +253,7 @@ class ClientSuite extends SparkFunSuite with Matchers { any(classOf[MutableHashMap[URI, Path]]), anyBoolean(), any()) classpath(client) should contain (buildPath(PWD, LOCALIZED_LIB_DIR, "*")) - sparkConf.set(SPARK_ARCHIVE, LOCAL_SCHEME + ":" + archive.getPath()) + sparkConf.set(SPARK_ARCHIVE, Utils.LOCAL_SCHEME + ":" + archive.getPath()) intercept[IllegalArgumentException] { client.prepareLocalResources(new Path(temp.getAbsolutePath()), Nil) } @@ -357,6 +362,115 @@ class ClientSuite extends SparkFunSuite with Matchers { sparkConf.get(SECONDARY_JARS) should be (Some(Seq(new File(jar2.toURI).getName))) } + Seq( + "client" -> YARN_AM_RESOURCE_TYPES_PREFIX, + "cluster" -> YARN_DRIVER_RESOURCE_TYPES_PREFIX + ).foreach { case (deployMode, prefix) => + test(s"custom resource request ($deployMode mode)") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val resources = Map("fpga" -> 2, "gpu" -> 3) + ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) + + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, deployMode) + resources.foreach { case (name, v) => + conf.set(s"${prefix}${name}.${AMOUNT}", v.toString) + } + + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) + val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) + + val client = new Client(new ClientArguments(Array()), conf, null) + client.createApplicationSubmissionContext( + new YarnClientApplication(getNewApplicationResponse, appContext), + containerLaunchContext) + + resources.foreach { case (name, value) => + ResourceRequestTestHelper.getRequestedValue(appContext.getResource, name) should be (value) + } + } + } + + test("custom driver resource request yarn config and spark config fails") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "gpu", YARN_FPGA_RESOURCE_CONFIG -> "fpga") + ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) + + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") + resources.keys.foreach { yarnName => + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") + } + resources.values.foreach { rName => + conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") + } + + val error = intercept[SparkException] { + ResourceRequestHelper.validateResources(conf) + }.getMessage() + + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/fpga.amount," + + " please use spark.driver.resource.fpga.amount")) + assert(error.contains("Do not use spark.yarn.driver.resource.yarn.io/gpu.amount," + + " please use spark.driver.resource.gpu.amount")) + } + + test("custom executor resource request yarn config and spark config fails") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "gpu", YARN_FPGA_RESOURCE_CONFIG -> "fpga") + ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) + + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") + resources.keys.foreach { yarnName => + conf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnName}.${AMOUNT}", "2") + } + resources.values.foreach { rName => + conf.set(ResourceID(SPARK_EXECUTOR_PREFIX, rName).amountConf, "3") + } + + val error = intercept[SparkException] { + ResourceRequestHelper.validateResources(conf) + }.getMessage() + + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/fpga.amount," + + " please use spark.executor.resource.fpga.amount")) + assert(error.contains("Do not use spark.yarn.executor.resource.yarn.io/gpu.amount," + + " please use spark.executor.resource.gpu.amount")) + } + + + test("custom resources spark config mapped to yarn config") { + assume(ResourceRequestHelper.isYarnResourceTypesAvailable()) + val yarnMadeupResource = "yarn.io/madeup" + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "gpu", + YARN_FPGA_RESOURCE_CONFIG -> "fpga", + yarnMadeupResource -> "madeup") + ResourceRequestTestHelper.initializeResourceTypes(resources.keys.toSeq) + + val conf = new SparkConf().set(SUBMIT_DEPLOY_MODE, "cluster") + resources.values.foreach { rName => + conf.set(ResourceID(SPARK_DRIVER_PREFIX, rName).amountConf, "3") + } + // also just set yarn one that we don't convert + conf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}", "5") + val appContext = Records.newRecord(classOf[ApplicationSubmissionContext]) + val getNewApplicationResponse = Records.newRecord(classOf[GetNewApplicationResponse]) + val containerLaunchContext = Records.newRecord(classOf[ContainerLaunchContext]) + + val client = new Client(new ClientArguments(Array()), conf, null) + val newContext = client.createApplicationSubmissionContext( + new YarnClientApplication(getNewApplicationResponse, appContext), + containerLaunchContext) + + val yarnRInfo = ResourceRequestTestHelper.getResources(newContext.getResource) + val allResourceInfo = yarnRInfo.map(rInfo => (rInfo.name -> rInfo.value)).toMap + assert(allResourceInfo.get(YARN_GPU_RESOURCE_CONFIG).nonEmpty) + assert(allResourceInfo.get(YARN_GPU_RESOURCE_CONFIG).get === 3) + assert(allResourceInfo.get(YARN_FPGA_RESOURCE_CONFIG).nonEmpty) + assert(allResourceInfo.get(YARN_FPGA_RESOURCE_CONFIG).get === 3) + assert(allResourceInfo.get(yarnMadeupResource).nonEmpty) + assert(allResourceInfo.get(yarnMadeupResource).get === 5) + } + private val matching = Seq( ("files URI match test1", "file:///file1", "file:///file2"), ("files URI match test2", "file:///c:file1", "file://c:file2"), @@ -424,7 +538,7 @@ class ClientSuite extends SparkFunSuite with Matchers { sparkConf: SparkConf, args: Array[String] = Array()): Client = { val clientArgs = new ClientArguments(args) - spy(new Client(clientArgs, sparkConf)) + spy(new Client(clientArgs, sparkConf, null)) } private def classpath(client: Client): Array[String] = { @@ -432,5 +546,4 @@ class ClientSuite extends SparkFunSuite with Matchers { populateClasspath(null, new Configuration(), client.sparkConf, env) classpath(env) } - } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala new file mode 100644 index 0000000000000..f5ec531e26e0c --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestHelperSuite.scala @@ -0,0 +1,184 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import org.apache.hadoop.yarn.api.records.Resource +import org.apache.hadoop.yarn.util.Records +import org.scalatest.Matchers + +import org.apache.spark.{SparkConf, SparkException, SparkFunSuite} +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ +import org.apache.spark.deploy.yarn.ResourceRequestTestHelper.ResourceInformation +import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config.{DRIVER_CORES, DRIVER_MEMORY, EXECUTOR_CORES, EXECUTOR_MEMORY} +import org.apache.spark.resource.ResourceUtils.AMOUNT + +class ResourceRequestHelperSuite extends SparkFunSuite with Matchers { + + private val CUSTOM_RES_1 = "custom-resource-type-1" + private val CUSTOM_RES_2 = "custom-resource-type-2" + private val MEMORY = "memory" + private val CORES = "cores" + private val NEW_CONFIG_EXECUTOR_MEMORY = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_EXECUTOR_CORES = + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_AM_MEMORY = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_AM_CORES = s"${YARN_AM_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_MEMORY = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${MEMORY}.${AMOUNT}" + private val NEW_CONFIG_DRIVER_CORES = s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${CORES}.${AMOUNT}" + + test("empty SparkConf should be valid") { + val sparkConf = new SparkConf() + validateResources(sparkConf) + } + + test("just normal resources are defined") { + val sparkConf = new SparkConf() + sparkConf.set(DRIVER_MEMORY.key, "3G") + sparkConf.set(DRIVER_CORES.key, "4") + sparkConf.set(EXECUTOR_MEMORY.key, "4G") + sparkConf.set(EXECUTOR_CORES.key, "2") + validateResources(sparkConf) + } + + test("get yarn resources from configs") { + val sparkConf = new SparkConf() + val resources = Map(YARN_GPU_RESOURCE_CONFIG -> "2G", + YARN_FPGA_RESOURCE_CONFIG -> "3G", "custom" -> "4") + resources.foreach { case (name, value) => + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_DRIVER_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + sparkConf.set(s"${YARN_AM_RESOURCE_TYPES_PREFIX}${name}.${AMOUNT}", value) + } + var parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_DRIVER_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + parsedResources = getYarnResourcesAndAmounts(sparkConf, YARN_AM_RESOURCE_TYPES_PREFIX) + assert(parsedResources === resources) + } + + test("get invalid yarn resources from configs") { + val sparkConf = new SparkConf() + + val missingAmountConfig = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}missingAmount" + // missing .amount + sparkConf.set(missingAmountConfig, "2g") + var thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Missing suffix for") + + sparkConf.remove(missingAmountConfig) + sparkConf.set(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}customResource.invalid", "2g") + + thrown = intercept[IllegalArgumentException] { + getYarnResourcesAndAmounts(sparkConf, YARN_EXECUTOR_RESOURCE_TYPES_PREFIX) + } + thrown.getMessage should include("Unsupported suffix") + } + + Seq( + "value with unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 2, "G")), + "value without unit" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "")), + "multiple resources" -> Seq(ResourceInformation(CUSTOM_RES_1, 123, "m"), + ResourceInformation(CUSTOM_RES_2, 10, "G")) + ).foreach { case (name, resources) => + test(s"valid request: $name") { + assume(isYarnResourceTypesAvailable()) + val resourceDefs = resources.map { r => r.name } + val requests = resources.map { r => (r.name, r.value.toString + r.unit) }.toMap + + ResourceRequestTestHelper.initializeResourceTypes(resourceDefs) + + val resource = createResource() + setResourceRequests(requests, resource) + + resources.foreach { r => + val requested = ResourceRequestTestHelper.getResourceInformationByName(resource, r.name) + assert(requested === r) + } + } + } + + Seq( + ("value does not match pattern", CUSTOM_RES_1, "**@#"), + ("only unit defined", CUSTOM_RES_1, "m"), + ("invalid unit", CUSTOM_RES_1, "123ppp") + ).foreach { case (name, key, value) => + test(s"invalid request: $name") { + assume(isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(Seq(key)) + + val resource = createResource() + val thrown = intercept[IllegalArgumentException] { + setResourceRequests(Map(key -> value), resource) + } + thrown.getMessage should include (key) + } + } + + Seq( + NEW_CONFIG_EXECUTOR_MEMORY -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}memory-mb.$AMOUNT" -> "30G", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}mb.$AMOUNT" -> "30G", + NEW_CONFIG_EXECUTOR_CORES -> "5", + s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}vcores.$AMOUNT" -> "5", + NEW_CONFIG_AM_MEMORY -> "1G", + NEW_CONFIG_DRIVER_MEMORY -> "1G", + NEW_CONFIG_AM_CORES -> "3", + NEW_CONFIG_DRIVER_CORES -> "1G" + ).foreach { case (key, value) => + test(s"disallowed resource request: $key") { + assume(isYarnResourceTypesAvailable()) + val conf = new SparkConf(false).set(key, value) + val thrown = intercept[SparkException] { + validateResources(conf) + } + thrown.getMessage should include (key) + } + } + + test("multiple disallowed resources in config") { + val sparkConf = new SparkConf() + sparkConf.set(DRIVER_MEMORY.key, "2G") + sparkConf.set(DRIVER_CORES.key, "2") + sparkConf.set(EXECUTOR_MEMORY.key, "2G") + sparkConf.set(EXECUTOR_CORES.key, "4") + sparkConf.set(AM_MEMORY.key, "3G") + sparkConf.set(NEW_CONFIG_EXECUTOR_MEMORY, "3G") + sparkConf.set(NEW_CONFIG_AM_MEMORY, "2G") + sparkConf.set(NEW_CONFIG_DRIVER_MEMORY, "2G") + + val thrown = intercept[SparkException] { + validateResources(sparkConf) + } + thrown.getMessage should ( + include(NEW_CONFIG_EXECUTOR_MEMORY) and + include(NEW_CONFIG_AM_MEMORY) and + include(NEW_CONFIG_DRIVER_MEMORY)) + } + + private def createResource(): Resource = { + val resource = Records.newRecord(classOf[Resource]) + resource.setMemory(512) + resource.setVirtualCores(2) + resource + } +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala new file mode 100644 index 0000000000000..796e099032347 --- /dev/null +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/ResourceRequestTestHelper.scala @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.deploy.yarn + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.yarn.api.records.Resource + +import org.apache.spark.util.Utils + +object ResourceRequestTestHelper { + def initializeResourceTypes(resourceTypes: Seq[String]): Unit = { + if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { + throw new IllegalStateException("This method should not be invoked " + + "since YARN resource types is not available because of old Hadoop version!" ) + } + + // ResourceUtils.reinitializeResources() is the YARN-way + // to specify resources for the execution of the tests. + // This method should receive standard resources with names of memory-mb and vcores. + // Without specifying the standard resources or specifying them + // with different names e.g. memory, YARN would throw various exceptions + // because it relies on that standard resources are always specified. + val defaultResourceTypes = List( + createResourceTypeInfo("memory-mb"), + createResourceTypeInfo("vcores")) + val customResourceTypes = resourceTypes.map(createResourceTypeInfo) + val allResourceTypes = defaultResourceTypes ++ customResourceTypes + + val resourceUtilsClass = + Utils.classForName("org.apache.hadoop.yarn.util.resource.ResourceUtils") + val reinitializeResourcesMethod = resourceUtilsClass.getMethod("reinitializeResources", + classOf[java.util.List[AnyRef]]) + reinitializeResourcesMethod.invoke(null, allResourceTypes.asJava) + } + + private def createResourceTypeInfo(resourceName: String): AnyRef = { + val resTypeInfoClass = Utils.classForName("org.apache.hadoop.yarn.api.records.ResourceTypeInfo") + val resTypeInfoNewInstanceMethod = resTypeInfoClass.getMethod("newInstance", classOf[String]) + resTypeInfoNewInstanceMethod.invoke(null, resourceName) + } + + def getRequestedValue(res: Resource, rtype: String): AnyRef = { + val resourceInformation = getResourceInformation(res, rtype) + invokeMethod(resourceInformation, "getValue") + } + + def getResourceInformationByName(res: Resource, nameParam: String): ResourceInformation = { + val resourceInformation: AnyRef = getResourceInformation(res, nameParam) + val name = invokeMethod(resourceInformation, "getName").asInstanceOf[String] + val value = invokeMethod(resourceInformation, "getValue").asInstanceOf[Long] + val units = invokeMethod(resourceInformation, "getUnits").asInstanceOf[String] + ResourceInformation(name, value, units) + } + + private def getResourceInformation(res: Resource, name: String): AnyRef = { + if (!ResourceRequestHelper.isYarnResourceTypesAvailable()) { + throw new IllegalStateException("assertResourceTypeValue() should not be invoked " + + "since yarn resource types is not available because of old Hadoop version!") + } + + val getResourceInformationMethod = res.getClass.getMethod("getResourceInformation", + classOf[String]) + val resourceInformation = getResourceInformationMethod.invoke(res, name) + resourceInformation + } + + private def invokeMethod(resourceInformation: AnyRef, methodName: String): AnyRef = { + val getValueMethod = resourceInformation.getClass.getMethod(methodName) + getValueMethod.invoke(resourceInformation) + } + + def getResources(res: Resource): Array[ResourceInformation] = { + val getResourceInformationMethod = res.getClass.getMethod("getResources") + val rInfoArray = getResourceInformationMethod.invoke(res).asInstanceOf[Array[AnyRef]] + rInfoArray.map { rInfo => + val name = invokeMethod(rInfo, "getName").asInstanceOf[String] + val value = invokeMethod(rInfo, "getValue").asInstanceOf[Long] + val units = invokeMethod(rInfo, "getUnits").asInstanceOf[String] + ResourceInformation(name, value, units) + } + } + + case class ResourceInformation(name: String, value: Long, unit: String) +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala index aeac68e6ed330..c07a4ac76b989 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorBlacklistTrackerSuite.scala @@ -25,7 +25,7 @@ import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.deploy.yarn.config.YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED +import org.apache.spark.deploy.yarn.config.{YARN_EXCLUDE_NODES, YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED} import org.apache.spark.internal.config.{BLACKLIST_TIMEOUT_CONF, MAX_FAILED_EXEC_PER_NODE} import org.apache.spark.util.ManualClock @@ -35,27 +35,31 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers val BLACKLIST_TIMEOUT = 100L val MAX_FAILED_EXEC_PER_NODE_VALUE = 2 + var sparkConf: SparkConf = _ var amClientMock: AMRMClient[ContainerRequest] = _ - var yarnBlacklistTracker: YarnAllocatorBlacklistTracker = _ - var failureTracker: FailureTracker = _ var clock: ManualClock = _ override def beforeEach(): Unit = { - val sparkConf = new SparkConf() + sparkConf = new SparkConf() sparkConf.set(BLACKLIST_TIMEOUT_CONF, BLACKLIST_TIMEOUT) sparkConf.set(YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED, true) sparkConf.set(MAX_FAILED_EXEC_PER_NODE, MAX_FAILED_EXEC_PER_NODE_VALUE) clock = new ManualClock() - amClientMock = mock(classOf[AMRMClient[ContainerRequest]]) - failureTracker = new FailureTracker(sparkConf, clock) - yarnBlacklistTracker = + super.beforeEach() + } + + private def createYarnAllocatorBlacklistTracker( + sparkConf: SparkConf = sparkConf): YarnAllocatorBlacklistTracker = { + val failureTracker = new FailureTracker(sparkConf, clock) + val yarnBlacklistTracker = new YarnAllocatorBlacklistTracker(sparkConf, amClientMock, failureTracker) yarnBlacklistTracker.setNumClusterNodes(4) - super.beforeEach() + yarnBlacklistTracker } test("expiring its own blacklisted nodes") { + val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() (1 to MAX_FAILED_EXEC_PER_NODE_VALUE).foreach { _ => { yarnBlacklistTracker.handleResourceAllocationFailure(Some("host")) @@ -77,6 +81,8 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers } test("not handling the expiry of scheduler blacklisted nodes") { + val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() + yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) verify(amClientMock) .updateBlacklist(Arrays.asList("host1", "host2"), Collections.emptyList()) @@ -87,10 +93,19 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers // expired blacklisted nodes (simulating a resource request) yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2")) // no change is communicated to YARN regarding the blacklisting - verify(amClientMock).updateBlacklist(Collections.emptyList(), Collections.emptyList()) + verify(amClientMock, times(0)).updateBlacklist(Collections.emptyList(), Collections.emptyList()) } test("combining scheduler and allocation blacklist") { + sparkConf.set(YARN_EXCLUDE_NODES, Seq("initial1", "initial2")) + val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker(sparkConf) + yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set()) + + // initial1 and initial2 is added as blacklisted nodes at the very first updateBlacklist call + // and they are never removed + verify(amClientMock) + .updateBlacklist(Arrays.asList("initial1", "initial2"), Collections.emptyList()) + (1 to MAX_FAILED_EXEC_PER_NODE_VALUE).foreach { _ => { yarnBlacklistTracker.handleResourceAllocationFailure(Some("host1")) @@ -117,6 +132,7 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers } test("blacklist all available nodes") { + val yarnBlacklistTracker = createYarnAllocatorBlacklistTracker() yarnBlacklistTracker.setSchedulerBlacklistedNodes(Set("host1", "host2", "host3")) verify(amClientMock) .updateBlacklist(Arrays.asList("host1", "host2", "host3"), Collections.emptyList()) @@ -135,6 +151,7 @@ class YarnAllocatorBlacklistTrackerSuite extends SparkFunSuite with Matchers yarnBlacklistTracker.handleResourceAllocationFailure(Some("host4")) verify(amClientMock).updateBlacklist(Arrays.asList("host4"), Collections.emptyList()) - assert(yarnBlacklistTracker.isAllNodeBlacklisted === true) + assert(yarnBlacklistTracker.isAllNodeBlacklisted) } + } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala index 3f783baed110d..4ac27ede64831 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala @@ -17,27 +17,33 @@ package org.apache.spark.deploy.yarn +import java.util.Collections + import scala.collection.JavaConverters._ -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.mockito.ArgumentCaptor import org.mockito.Mockito._ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} -import org.apache.spark.deploy.yarn.YarnAllocator._ +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.deploy.yarn.ResourceRequestHelper._ import org.apache.spark.deploy.yarn.YarnSparkHadoopUtil._ import org.apache.spark.deploy.yarn.config._ +import org.apache.spark.internal.config._ +import org.apache.spark.resource.ResourceUtils.{AMOUNT, GPU} +import org.apache.spark.resource.TestResourceIDs._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.scheduler.SplitInfo import org.apache.spark.util.ManualClock -class MockResolver extends SparkRackResolver { +class MockResolver extends SparkRackResolver(SparkHadoopUtil.get.conf) { - override def resolve(conf: Configuration, hostName: String): String = { + override def resolve(hostName: String): String = { if (hostName == "host3") "/rack2" else "/rack1" } @@ -46,8 +52,8 @@ class MockResolver extends SparkRackResolver { class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach { val conf = new YarnConfiguration() val sparkConf = new SparkConf() - sparkConf.set("spark.driver.host", "localhost") - sparkConf.set("spark.driver.port", "4040") + sparkConf.set(DRIVER_HOST_ADDRESS, "localhost") + sparkConf.set(DRIVER_PORT, 4040) sparkConf.set(SPARK_JARS, Seq("notarealjar.jar")) sparkConf.set("spark.yarn.launchContainers", "false") @@ -86,15 +92,21 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter def createAllocator( maxExecutors: Int = 5, - rmClient: AMRMClient[ContainerRequest] = rmClient): YarnAllocator = { + rmClient: AMRMClient[ContainerRequest] = rmClient, + additionalConfigs: Map[String, String] = Map()): YarnAllocator = { val args = Array( "--jar", "somejar.jar", "--class", "SomeClass") val sparkConfClone = sparkConf.clone() sparkConfClone - .set("spark.executor.instances", maxExecutors.toString) - .set("spark.executor.cores", "5") - .set("spark.executor.memory", "2048") + .set(EXECUTOR_INSTANCES, maxExecutors) + .set(EXECUTOR_CORES, 5) + .set(EXECUTOR_MEMORY, 2048L) + + for ((name, value) <- additionalConfigs) { + sparkConfClone.set(name, value) + } + new YarnAllocator( "not used", mock(classOf[RpcEndpointRef]), @@ -108,14 +120,29 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock) } - def createContainer(host: String): Container = { - // When YARN 2.6+ is required, avoid deprecation by using version with long second arg - val containerId = ContainerId.newInstance(appAttemptId, containerNum) + def createContainer( + host: String, + containerNumber: Int = containerNum, + resource: Resource = containerResource): Container = { + val containerId: ContainerId = ContainerId.newContainerId(appAttemptId, containerNum) containerNum += 1 val nodeId = NodeId.newInstance(host, 1000) - Container.newInstance(containerId, nodeId, "", containerResource, RM_REQUEST_PRIORITY, null) + Container.newInstance(containerId, nodeId, "", resource, RM_REQUEST_PRIORITY, null) } + def createContainers(hosts: Seq[String], containerIds: Seq[Int]): Seq[Container] = { + hosts.zip(containerIds).map{case (host, id) => createContainer(host, id)} + } + + def createContainerStatus( + containerId: ContainerId, + exitStatus: Int, + containerState: ContainerState = ContainerState.COMPLETE, + diagnostics: String = "diagnostics"): ContainerStatus = { + ContainerStatus.newInstance(containerId, containerState, diagnostics, exitStatus) + } + + test("single container allocated") { // request a single container and receive it val handler = createAllocator(1) @@ -134,6 +161,53 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter size should be (0) } + test("custom resource requested from yarn") { + assume(isYarnResourceTypesAvailable()) + ResourceRequestTestHelper.initializeResourceTypes(List("gpu")) + + val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) + val handler = createAllocator(1, mockAmClient, + Map(s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${GPU}.${AMOUNT}" -> "2G")) + + handler.updateResourceRequests() + val container = createContainer("host1", resource = handler.resource) + handler.handleAllocatedContainers(Array(container)) + + // get amount of memory and vcores from resource, so effectively skipping their validation + val expectedResources = Resource.newInstance(handler.resource.getMemory(), + handler.resource.getVirtualCores) + setResourceRequests(Map("gpu" -> "2G"), expectedResources) + val captor = ArgumentCaptor.forClass(classOf[ContainerRequest]) + + verify(mockAmClient).addContainerRequest(captor.capture()) + val containerRequest: ContainerRequest = captor.getValue + assert(containerRequest.getCapability === expectedResources) + } + + test("custom spark resource mapped to yarn resource configs") { + assume(isYarnResourceTypesAvailable()) + val yarnMadeupResource = "yarn.io/madeup" + val yarnResources = Seq(YARN_GPU_RESOURCE_CONFIG, YARN_FPGA_RESOURCE_CONFIG, yarnMadeupResource) + ResourceRequestTestHelper.initializeResourceTypes(yarnResources) + val mockAmClient = mock(classOf[AMRMClient[ContainerRequest]]) + val madeupConfigName = s"${YARN_EXECUTOR_RESOURCE_TYPES_PREFIX}${yarnMadeupResource}.${AMOUNT}" + val sparkResources = + Map(EXECUTOR_GPU_ID.amountConf -> "3", + EXECUTOR_FPGA_ID.amountConf -> "2", + madeupConfigName -> "5") + val handler = createAllocator(1, mockAmClient, sparkResources) + + handler.updateResourceRequests() + val yarnRInfo = ResourceRequestTestHelper.getResources(handler.resource) + val allResourceInfo = yarnRInfo.map( rInfo => (rInfo.name -> rInfo.value) ).toMap + assert(allResourceInfo.get(YARN_GPU_RESOURCE_CONFIG).nonEmpty) + assert(allResourceInfo.get(YARN_GPU_RESOURCE_CONFIG).get === 3) + assert(allResourceInfo.get(YARN_FPGA_RESOURCE_CONFIG).nonEmpty) + assert(allResourceInfo.get(YARN_FPGA_RESOURCE_CONFIG).get === 2) + assert(allResourceInfo.get(yarnMadeupResource).nonEmpty) + assert(allResourceInfo.get(yarnMadeupResource).get === 5) + } + test("container should not be created if requested number if met") { // request a single container and receive it val handler = createAllocator(1) @@ -347,19 +421,8 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter verify(mockAmClient).updateBlacklist(Seq[String]().asJava, Seq("hostA", "hostB").asJava) } - test("memory exceeded diagnostic regexes") { - val diagnostics = - "Container [pid=12465,containerID=container_1412887393566_0003_01_000002] is running " + - "beyond physical memory limits. Current usage: 2.1 MB of 2 GB physical memory used; " + - "5.8 GB of 4.2 GB virtual memory used. Killing container." - val vmemMsg = memLimitExceededLogMessage(diagnostics, VMEM_EXCEEDED_PATTERN) - val pmemMsg = memLimitExceededLogMessage(diagnostics, PMEM_EXCEEDED_PATTERN) - assert(vmemMsg.contains("5.8 GB of 4.2 GB virtual memory used.")) - assert(pmemMsg.contains("2.1 MB of 2 GB physical memory used.")) - } - test("window based failure executor counting") { - sparkConf.set("spark.yarn.executor.failuresValidityInterval", "100s") + sparkConf.set(EXECUTOR_ATTEMPT_FAILURE_VALIDITY_INTERVAL_MS, 100 * 1000L) val handler = createAllocator(4) handler.updateResourceRequests() @@ -400,4 +463,55 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter clock.advance(50 * 1000L) handler.getNumExecutorsFailed should be (0) } + + test("SPARK-26269: YarnAllocator should have same blacklist behaviour with YARN") { + val rmClientSpy = spy(rmClient) + val maxExecutors = 11 + + val handler = createAllocator( + maxExecutors, + rmClientSpy, + Map( + YARN_EXECUTOR_LAUNCH_BLACKLIST_ENABLED.key -> "true", + MAX_FAILED_EXEC_PER_NODE.key -> "0")) + handler.updateResourceRequests() + + val hosts = (0 until maxExecutors).map(i => s"host$i") + val ids = 0 to maxExecutors + val containers = createContainers(hosts, ids) + + val nonBlacklistedStatuses = Seq( + ContainerExitStatus.SUCCESS, + ContainerExitStatus.PREEMPTED, + ContainerExitStatus.KILLED_EXCEEDED_VMEM, + ContainerExitStatus.KILLED_EXCEEDED_PMEM, + ContainerExitStatus.KILLED_BY_RESOURCEMANAGER, + ContainerExitStatus.KILLED_BY_APPMASTER, + ContainerExitStatus.KILLED_AFTER_APP_COMPLETION, + ContainerExitStatus.ABORTED, + ContainerExitStatus.DISKS_FAILED) + + val nonBlacklistedContainerStatuses = nonBlacklistedStatuses.zipWithIndex.map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + val BLACKLISTED_EXIT_CODE = 1 + val blacklistedStatuses = Seq(ContainerExitStatus.INVALID, BLACKLISTED_EXIT_CODE) + + val blacklistedContainerStatuses = blacklistedStatuses.zip(9 until maxExecutors).map { + case (exitStatus, idx) => createContainerStatus(containers(idx).getId, exitStatus) + } + + handler.handleAllocatedContainers(containers.slice(0, 9)) + handler.processCompletedContainers(nonBlacklistedContainerStatuses) + verify(rmClientSpy, never()) + .updateBlacklist(hosts.slice(0, 9).asJava, Collections.emptyList()) + + handler.handleAllocatedContainers(containers.slice(9, 11)) + handler.processCompletedContainers(blacklistedContainerStatuses) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(9, 10).asJava, Collections.emptyList()) + verify(rmClientSpy) + .updateBlacklist(hosts.slice(10, 11).asJava, Collections.emptyList()) + } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala index 58d11e96942e1..7264e2e51ee45 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala @@ -18,30 +18,31 @@ package org.apache.spark.deploy.yarn import java.io.File -import java.net.URL import java.nio.charset.StandardCharsets import java.util.{HashMap => JHashMap} import scala.collection.mutable import scala.concurrent.duration._ import scala.io.Source -import scala.language.postfixOps import com.google.common.io.{ByteStreams, Files} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.ConverterUtils import org.scalatest.Matchers import org.scalatest.concurrent.Eventually._ import org.apache.spark._ +import org.apache.spark.api.python.PythonUtils import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.UI._ import org.apache.spark.launcher._ -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, - SparkListenerExecutorAdded} +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationStart, SparkListenerExecutorAdded} import org.apache.spark.scheduler.cluster.ExecutorInfo import org.apache.spark.tags.ExtendedYarnTest -import org.apache.spark.util.Utils +import org.apache.spark.util.{Utils, YarnContainerInfoHelper} /** * Integration tests for YARN; these tests use a mini Yarn cluster to run Spark-on-YARN @@ -87,13 +88,17 @@ class YarnClusterSuite extends BaseYarnClusterSuite { testBasicYarnApp(false) } + test("run Spark in yarn-client mode with unmanaged am") { + testBasicYarnApp(true, Map(YARN_UNMANAGED_AM.key -> "true")) + } + test("run Spark in yarn-client mode with different configurations, ensuring redaction") { testBasicYarnApp(true, Map( - "spark.driver.memory" -> "512m", - "spark.executor.cores" -> "1", - "spark.executor.memory" -> "512m", - "spark.executor.instances" -> "2", + DRIVER_MEMORY.key -> "512m", + EXECUTOR_CORES.key -> "1", + EXECUTOR_MEMORY.key -> "512m", + EXECUTOR_INSTANCES.key -> "2", // Sending some sensitive information, which we'll make sure gets redacted "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD" -> YarnClusterDriver.SECRET_PASSWORD, "spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD" -> YarnClusterDriver.SECRET_PASSWORD @@ -103,11 +108,11 @@ class YarnClusterSuite extends BaseYarnClusterSuite { test("run Spark in yarn-cluster mode with different configurations, ensuring redaction") { testBasicYarnApp(false, Map( - "spark.driver.memory" -> "512m", - "spark.driver.cores" -> "1", - "spark.executor.cores" -> "1", - "spark.executor.memory" -> "512m", - "spark.executor.instances" -> "2", + DRIVER_MEMORY.key -> "512m", + DRIVER_CORES.key -> "1", + EXECUTOR_CORES.key -> "1", + EXECUTOR_MEMORY.key -> "512m", + EXECUTOR_INSTANCES.key -> "2", // Sending some sensitive information, which we'll make sure gets redacted "spark.executorEnv.HADOOP_CREDSTORE_PASSWORD" -> YarnClusterDriver.SECRET_PASSWORD, "spark.yarn.appMasterEnv.HADOOP_CREDSTORE_PASSWORD" -> YarnClusterDriver.SECRET_PASSWORD @@ -192,7 +197,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { val propsFile = createConfFile() val handle = new SparkLauncher(env) .setSparkHome(sys.props("spark.test.home")) - .setConf("spark.ui.enabled", "false") + .setConf(UI_ENABLED.key, "false") .setPropertiesFile(propsFile) .setMaster("yarn") .setDeployMode("client") @@ -201,7 +206,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { .startApplication() try { - eventually(timeout(30 seconds), interval(100 millis)) { + eventually(timeout(3.minutes), interval(100.milliseconds)) { handle.getState() should be (SparkAppHandle.State.RUNNING) } @@ -209,7 +214,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { handle.getAppId() should startWith ("application_") handle.stop() - eventually(timeout(30 seconds), interval(100 millis)) { + eventually(timeout(3.minutes), interval(100.milliseconds)) { handle.getState() should be (SparkAppHandle.State.KILLED) } } finally { @@ -265,7 +270,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { // needed locations. val sparkHome = sys.props("spark.test.home") val pythonPath = Seq( - s"$sparkHome/python/lib/py4j-0.10.7-src.zip", + s"$sparkHome/python/lib/${PythonUtils.PY4J_ZIP_NAME}", s"$sparkHome/python") val extraEnvVars = Map( "PYSPARK_ARCHIVES_PATH" -> pythonPath.map("local:" + _).mkString(File.pathSeparator), @@ -326,6 +331,7 @@ class YarnClusterSuite extends BaseYarnClusterSuite { private[spark] class SaveExecutorInfo extends SparkListener { val addedExecutorInfos = mutable.Map[String, ExecutorInfo]() var driverLogs: Option[collection.Map[String, String]] = None + var driverAttributes: Option[collection.Map[String, String]] = None override def onExecutorAdded(executor: SparkListenerExecutorAdded) { addedExecutorInfos(executor.executorId) = executor.executorInfo @@ -333,6 +339,7 @@ private[spark] class SaveExecutorInfo extends SparkListener { override def onApplicationStart(appStart: SparkListenerApplicationStart): Unit = { driverLogs = appStart.driverLogs + driverAttributes = appStart.driverAttributes } } @@ -437,11 +444,12 @@ private object YarnClusterDriver extends Logging with Matchers { s"Executor logs contain sensitive info (${SECRET_PASSWORD}): \n${log} " ) } + assert(info.attributes.nonEmpty) } // If we are running in yarn-cluster mode, verify that driver logs links and present and are // in the expected format. - if (conf.get("spark.submit.deployMode") == "cluster") { + if (conf.get(SUBMIT_DEPLOY_MODE) == "cluster") { assert(listener.driverLogs.nonEmpty) val driverLogs = listener.driverLogs.get assert(driverLogs.size === 2) @@ -455,9 +463,27 @@ private object YarnClusterDriver extends Logging with Matchers { s"Driver logs contain sensitive info (${SECRET_PASSWORD}): \n${log} " ) } - val containerId = YarnSparkHadoopUtil.getContainerId + + val yarnConf = new YarnConfiguration(sc.hadoopConfiguration) + val containerId = YarnContainerInfoHelper.getContainerId(container = None) val user = Utils.getCurrentUserName() + assert(urlStr.endsWith(s"/node/containerlogs/$containerId/$user/stderr?start=-4096")) + + assert(listener.driverAttributes.nonEmpty) + val driverAttributes = listener.driverAttributes.get + val expectationAttributes = Map( + "HTTP_SCHEME" -> YarnContainerInfoHelper.getYarnHttpScheme(yarnConf), + "NM_HOST" -> YarnContainerInfoHelper.getNodeManagerHost(container = None), + "NM_PORT" -> YarnContainerInfoHelper.getNodeManagerPort(container = None), + "NM_HTTP_PORT" -> YarnContainerInfoHelper.getNodeManagerHttpPort(container = None), + "NM_HTTP_ADDRESS" -> YarnContainerInfoHelper.getNodeManagerHttpAddress(container = None), + "CLUSTER_ID" -> YarnContainerInfoHelper.getClusterId(yarnConf).getOrElse(""), + "CONTAINER_ID" -> ConverterUtils.toString(containerId), + "USER" -> user, + "LOG_FILES" -> "stderr,stdout") + + assert(driverAttributes === expectationAttributes) } } finally { Files.write(result, status, StandardCharsets.UTF_8) diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala index 37bccaf0439b4..8c62069a8dd67 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnShuffleIntegrationSuite.scala @@ -29,6 +29,7 @@ import org.apache.spark._ import org.apache.spark.deploy.yarn.config._ import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Network._ import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.yarn.{YarnShuffleService, YarnTestAccessor} import org.apache.spark.tags.ExtendedYarnTest @@ -94,14 +95,14 @@ class YarnShuffleAuthSuite extends YarnShuffleIntegrationSuite { override def newYarnConfig(): YarnConfiguration = { val yarnConfig = super.newYarnConfig() yarnConfig.set(NETWORK_AUTH_ENABLED.key, "true") - yarnConfig.set(NETWORK_ENCRYPTION_ENABLED.key, "true") + yarnConfig.set(NETWORK_CRYPTO_ENABLED.key, "true") yarnConfig } override protected def extraSparkConf(): Map[String, String] = { super.extraSparkConf() ++ Map( NETWORK_AUTH_ENABLED.key -> "true", - NETWORK_ENCRYPTION_ENABLED.key -> "true" + NETWORK_CRYPTO_ENABLED.key -> "true" ) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala index 61c0c43f7c04f..e7cde03a01b46 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtilSuite.scala @@ -21,8 +21,6 @@ import java.io.{File, IOException} import java.nio.charset.StandardCharsets import com.google.common.io.{ByteStreams, Files} -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path import org.apache.hadoop.yarn.api.records.ApplicationAccessType import org.apache.hadoop.yarn.conf.YarnConfiguration import org.scalatest.Matchers @@ -30,6 +28,7 @@ import org.scalatest.Matchers import org.apache.spark.{SecurityManager, SparkConf, SparkFunSuite} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.util.{ResetSystemProperties, Utils} class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging @@ -83,7 +82,7 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // spark acls on, just pick up default user val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") + sparkConf.set(ACLS_ENABLE, true) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) @@ -111,9 +110,9 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging // default spark acls are on and specify acls val sparkConf = new SparkConf() - sparkConf.set("spark.acls.enable", "true") - sparkConf.set("spark.ui.view.acls", "user1,user2") - sparkConf.set("spark.modify.acls", "user3,user4") + sparkConf.set(ACLS_ENABLE, true) + sparkConf.set(UI_VIEW_ACLS, Seq("user1", "user2")) + sparkConf.set(MODIFY_ACLS, Seq("user3", "user4")) val securityMgr = new SecurityManager(sparkConf) val acls = YarnSparkHadoopUtil.getApplicationAclsForYarn(securityMgr) @@ -141,67 +140,4 @@ class YarnSparkHadoopUtilSuite extends SparkFunSuite with Matchers with Logging } } - - test("SPARK-24149: retrieve all namenodes from HDFS") { - val sparkConf = new SparkConf() - val basicFederationConf = new Configuration() - basicFederationConf.set("fs.defaultFS", "hdfs://localhost:8020") - basicFederationConf.set("dfs.nameservices", "ns1,ns2") - basicFederationConf.set("dfs.namenode.rpc-address.ns1", "localhost:8020") - basicFederationConf.set("dfs.namenode.rpc-address.ns2", "localhost:8021") - val basicFederationExpected = Set( - new Path("hdfs://localhost:8020").getFileSystem(basicFederationConf), - new Path("hdfs://localhost:8021").getFileSystem(basicFederationConf)) - val basicFederationResult = YarnSparkHadoopUtil.hadoopFSsToAccess( - sparkConf, basicFederationConf) - basicFederationResult should be (basicFederationExpected) - - // when viewfs is enabled, namespaces are handled by it, so we don't need to take care of them - val viewFsConf = new Configuration() - viewFsConf.addResource(basicFederationConf) - viewFsConf.set("fs.defaultFS", "viewfs://clusterX/") - viewFsConf.set("fs.viewfs.mounttable.clusterX.link./home", "hdfs://localhost:8020/") - val viewFsExpected = Set(new Path("viewfs://clusterX/").getFileSystem(viewFsConf)) - YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, viewFsConf) should be (viewFsExpected) - - // invalid config should not throw NullPointerException - val invalidFederationConf = new Configuration() - invalidFederationConf.addResource(basicFederationConf) - invalidFederationConf.unset("dfs.namenode.rpc-address.ns2") - val invalidFederationExpected = Set( - new Path("hdfs://localhost:8020").getFileSystem(invalidFederationConf)) - val invalidFederationResult = YarnSparkHadoopUtil.hadoopFSsToAccess( - sparkConf, invalidFederationConf) - invalidFederationResult should be (invalidFederationExpected) - - // no namespaces defined, ie. old case - val noFederationConf = new Configuration() - noFederationConf.set("fs.defaultFS", "hdfs://localhost:8020") - val noFederationExpected = Set( - new Path("hdfs://localhost:8020").getFileSystem(noFederationConf)) - val noFederationResult = YarnSparkHadoopUtil.hadoopFSsToAccess(sparkConf, noFederationConf) - noFederationResult should be (noFederationExpected) - - // federation and HA enabled - val federationAndHAConf = new Configuration() - federationAndHAConf.set("fs.defaultFS", "hdfs://clusterXHA") - federationAndHAConf.set("dfs.nameservices", "clusterXHA,clusterYHA") - federationAndHAConf.set("dfs.ha.namenodes.clusterXHA", "x-nn1,x-nn2") - federationAndHAConf.set("dfs.ha.namenodes.clusterYHA", "y-nn1,y-nn2") - federationAndHAConf.set("dfs.namenode.rpc-address.clusterXHA.x-nn1", "localhost:8020") - federationAndHAConf.set("dfs.namenode.rpc-address.clusterXHA.x-nn2", "localhost:8021") - federationAndHAConf.set("dfs.namenode.rpc-address.clusterYHA.y-nn1", "localhost:8022") - federationAndHAConf.set("dfs.namenode.rpc-address.clusterYHA.y-nn2", "localhost:8023") - federationAndHAConf.set("dfs.client.failover.proxy.provider.clusterXHA", - "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider") - federationAndHAConf.set("dfs.client.failover.proxy.provider.clusterYHA", - "org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider") - - val federationAndHAExpected = Set( - new Path("hdfs://clusterXHA").getFileSystem(federationAndHAConf), - new Path("hdfs://clusterYHA").getFileSystem(federationAndHAConf)) - val federationAndHAResult = YarnSparkHadoopUtil.hadoopFSsToAccess( - sparkConf, federationAndHAConf) - federationAndHAResult should be (federationAndHAExpected) - } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala deleted file mode 100644 index 9fa749b14c98c..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/security/YARNHadoopDelegationTokenManagerSuite.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.deploy.yarn.security - -import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.security.Credentials -import org.scalatest.Matchers - -import org.apache.spark.{SparkConf, SparkFunSuite} - -class YARNHadoopDelegationTokenManagerSuite extends SparkFunSuite with Matchers { - private var credentialManager: YARNHadoopDelegationTokenManager = null - private var sparkConf: SparkConf = null - private var hadoopConf: Configuration = null - - override def beforeAll(): Unit = { - super.beforeAll() - sparkConf = new SparkConf() - hadoopConf = new Configuration() - } - - test("Correctly loads credential providers") { - credentialManager = new YARNHadoopDelegationTokenManager(sparkConf, hadoopConf) - credentialManager.credentialProviders.get("yarn-test") should not be (None) - } -} - -class YARNTestCredentialProvider extends ServiceCredentialProvider { - override def serviceName: String = "yarn-test" - - override def credentialsRequired(conf: Configuration): Boolean = true - - override def obtainCredentials( - hadoopConf: Configuration, - sparkConf: SparkConf, - creds: Credentials): Option[Long] = None -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala index 40b92282a3b8f..3736681f4e7f0 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceMetricsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.network.yarn import scala.collection.JavaConverters._ import org.apache.hadoop.metrics2.MetricsRecordBuilder -import org.mockito.Matchers._ +import org.mockito.ArgumentMatchers.{any, anyDouble, anyInt, anyLong} import org.mockito.Mockito.{mock, times, verify, when} import org.scalatest.Matchers @@ -38,7 +38,8 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { test("metrics named as expected") { val allMetrics = Set( "openBlockRequestLatencyMillis", "registerExecutorRequestLatencyMillis", - "blockTransferRateBytes", "registeredExecutorsSize") + "blockTransferRateBytes", "registeredExecutorsSize", "numActiveConnections", + "numRegisteredConnections", "numCaughtExceptions") metrics.getMetrics.keySet().asScala should be (allMetrics) } @@ -55,8 +56,8 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { YarnShuffleServiceMetrics.collectMetric(builder, testname, metrics.getMetrics.get(testname)) - verify(builder).addCounter(anyObject(), anyLong()) - verify(builder, times(4)).addGauge(anyObject(), anyDouble()) + verify(builder).addCounter(any(), anyLong()) + verify(builder, times(4)).addGauge(any(), anyDouble()) } } @@ -68,6 +69,6 @@ class YarnShuffleServiceMetricsSuite extends SparkFunSuite with Matchers { metrics.getMetrics.get("registeredExecutorsSize")) // only one - verify(builder).addGauge(anyObject(), anyInt()) + verify(builder).addGauge(any(), anyInt()) } } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala index 268f4bd13f6c3..381a93580f961 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/network/yarn/YarnShuffleServiceSuite.scala @@ -24,7 +24,6 @@ import java.util.EnumSet import scala.annotation.tailrec import scala.concurrent.duration._ -import scala.language.postfixOps import org.apache.hadoop.fs.Path import org.apache.hadoop.service.ServiceStateException @@ -36,6 +35,7 @@ import org.scalatest.concurrent.Eventually._ import org.apache.spark.SecurityManager import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config._ import org.apache.spark.network.shuffle.ShuffleTestAccessor import org.apache.spark.network.shuffle.protocol.ExecutorShuffleInfo import org.apache.spark.util.Utils @@ -52,7 +52,7 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd yarnConfig.set(YarnConfiguration.NM_AUX_SERVICES, "spark_shuffle") yarnConfig.set(YarnConfiguration.NM_AUX_SERVICE_FMT.format("spark_shuffle"), classOf[YarnShuffleService].getCanonicalName) - yarnConfig.setInt("spark.shuffle.service.port", 0) + yarnConfig.setInt(SHUFFLE_SERVICE_PORT.key, 0) yarnConfig.setBoolean(YarnShuffleService.STOP_ON_FAILURE_KEY, true) val localDir = Utils.createTempDir() yarnConfig.set(YarnConfiguration.NM_LOCAL_DIRS, localDir.getAbsolutePath) @@ -326,10 +326,10 @@ class YarnShuffleServiceSuite extends SparkFunSuite with Matchers with BeforeAnd recoveryPath.toString should be (new Path(execStateFile2.getParentFile.toURI).toString) recoveryPath.toString should be (new Path(secretsFile2.getParentFile.toURI).toString) - eventually(timeout(10 seconds), interval(5 millis)) { + eventually(timeout(10.seconds), interval(5.milliseconds)) { assert(!execStateFile.exists()) } - eventually(timeout(10 seconds), interval(5 millis)) { + eventually(timeout(10.seconds), interval(5.milliseconds)) { assert(!secretsFile.exists()) } diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala deleted file mode 100644 index 6ea7984c64514..0000000000000 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/ExtensionServiceIntegrationSuite.scala +++ /dev/null @@ -1,72 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.scheduler.cluster - -import org.scalatest.BeforeAndAfter - -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, SparkFunSuite} -import org.apache.spark.deploy.yarn.config._ -import org.apache.spark.internal.Logging - -/** - * Test the integration with [[SchedulerExtensionServices]] - */ -class ExtensionServiceIntegrationSuite extends SparkFunSuite - with LocalSparkContext with BeforeAndAfter - with Logging { - - val applicationId = new StubApplicationId(0, 1111L) - val attemptId = new StubApplicationAttemptId(applicationId, 1) - - /* - * Setup phase creates the spark context - */ - before { - val sparkConf = new SparkConf() - sparkConf.set(SCHEDULER_SERVICES, Seq(classOf[SimpleExtensionService].getName())) - sparkConf.setMaster("local").setAppName("ExtensionServiceIntegrationSuite") - sc = new SparkContext(sparkConf) - } - - test("Instantiate") { - val services = new SchedulerExtensionServices() - assertResult(Nil, "non-nil service list") { - services.getServices - } - services.start(SchedulerExtensionServiceBinding(sc, applicationId)) - services.stop() - } - - test("Contains SimpleExtensionService Service") { - val services = new SchedulerExtensionServices() - try { - services.start(SchedulerExtensionServiceBinding(sc, applicationId)) - val serviceList = services.getServices - assert(serviceList.nonEmpty, "empty service list") - val (service :: Nil) = serviceList - val simpleService = service.asInstanceOf[SimpleExtensionService] - assert(simpleService.started.get, "service not started") - services.stop() - assert(!simpleService.started.get, "service not stopped") - } finally { - services.stop() - } - } -} - - diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala index 7fac57ff68abc..835f0736c5a1e 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala +++ b/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/YarnSchedulerBackendSuite.scala @@ -16,26 +16,51 @@ */ package org.apache.spark.scheduler.cluster -import scala.language.reflectiveCalls +import java.net.URL +import java.util.concurrent.atomic.AtomicReference +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import org.mockito.Mockito.when import org.scalatest.mockito.MockitoSugar -import org.apache.spark.{LocalSparkContext, SparkContext, SparkFunSuite} +import org.apache.spark._ import org.apache.spark.scheduler.TaskSchedulerImpl import org.apache.spark.serializer.JavaSerializer +import org.apache.spark.ui.TestFilter class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with LocalSparkContext { - test("RequestExecutors reflects node blacklist and is serializable") { - sc = new SparkContext("local", "YarnSchedulerBackendSuite") - val sched = mock[TaskSchedulerImpl] - when(sched.sc).thenReturn(sc) - val yarnSchedulerBackend = new YarnSchedulerBackend(sched, sc) { - def setHostToLocalTaskCount(hostToLocalTaskCount: Map[String, Int]): Unit = { - this.hostToLocalTaskCount = hostToLocalTaskCount + private var yarnSchedulerBackend: YarnSchedulerBackend = _ + + override def afterEach(): Unit = { + try { + if (yarnSchedulerBackend != null) { + yarnSchedulerBackend.stop() } + } finally { + super.afterEach() + } + } + + private class TestTaskSchedulerImpl(sc: SparkContext) extends TaskSchedulerImpl(sc) { + val blacklistedNodes = new AtomicReference[Set[String]]() + def setNodeBlacklist(nodeBlacklist: Set[String]): Unit = blacklistedNodes.set(nodeBlacklist) + override def nodeBlacklist(): Set[String] = blacklistedNodes.get() + } + + private class TestYarnSchedulerBackend(scheduler: TaskSchedulerImpl, sc: SparkContext) + extends YarnSchedulerBackend(scheduler, sc) { + def setHostToLocalTaskCount(hostToLocalTaskCount: Map[String, Int]): Unit = { + this.hostToLocalTaskCount = hostToLocalTaskCount } + } + + test("RequestExecutors reflects node blacklist and is serializable") { + sc = new SparkContext("local", "YarnSchedulerBackendSuite") + // Subclassing the TaskSchedulerImpl here instead of using Mockito. For details see SPARK-26891. + val sched = new TestTaskSchedulerImpl(sc) + val yarnSchedulerBackendExtended = new TestYarnSchedulerBackend(sched, sc) + yarnSchedulerBackend = yarnSchedulerBackendExtended val ser = new JavaSerializer(sc.conf).newInstance() for { blacklist <- IndexedSeq(Set[String](), Set("a", "b", "c")) @@ -45,16 +70,62 @@ class YarnSchedulerBackendSuite extends SparkFunSuite with MockitoSugar with Loc Map("a" -> 1, "b" -> 2) ) } { - yarnSchedulerBackend.setHostToLocalTaskCount(hostToLocalCount) - when(sched.nodeBlacklist()).thenReturn(blacklist) - val req = yarnSchedulerBackend.prepareRequestExecutors(numRequested) + yarnSchedulerBackendExtended.setHostToLocalTaskCount(hostToLocalCount) + sched.setNodeBlacklist(blacklist) + val req = yarnSchedulerBackendExtended.prepareRequestExecutors(numRequested) assert(req.requestedTotal === numRequested) assert(req.nodeBlacklist === blacklist) assert(req.hostToLocalTaskCount.keySet.intersect(blacklist).isEmpty) // Serialize to make sure serialization doesn't throw an error ser.serialize(req) } - sc.stop() + } + + test("Respect user filters when adding AM IP filter") { + val conf = new SparkConf(false) + .set("spark.ui.filters", classOf[TestFilter].getName()) + .set(s"spark.${classOf[TestFilter].getName()}.param.responseCode", + HttpServletResponse.SC_BAD_GATEWAY.toString) + + sc = new SparkContext("local", "YarnSchedulerBackendSuite", conf) + val sched = mock[TaskSchedulerImpl] + when(sched.sc).thenReturn(sc) + + val url = new URL(sc.uiWebUrl.get) + // Before adding the "YARN" filter, should get the code from the filter in SparkConf. + assert(TestUtils.httpResponseCode(url) === HttpServletResponse.SC_BAD_GATEWAY) + + yarnSchedulerBackend = new YarnSchedulerBackend(sched, sc) { } + + yarnSchedulerBackend.addWebUIFilter(classOf[TestFilter2].getName(), + Map("responseCode" -> HttpServletResponse.SC_NOT_ACCEPTABLE.toString), "") + + sc.ui.get.getDelegatingHandlers.foreach { h => + // Two filters above + security filter. + assert(h.filterCount() === 3) + } + + // The filter should have been added first in the chain, so we should get SC_NOT_ACCEPTABLE + // instead of SC_OK. + assert(TestUtils.httpResponseCode(url) === HttpServletResponse.SC_NOT_ACCEPTABLE) + + // Add a new handler and make sure the added filter is properly registered. + val servlet = new HttpServlet() { + override def doGet(req: HttpServletRequest, res: HttpServletResponse): Unit = { + res.sendError(HttpServletResponse.SC_CONFLICT) + } + } + + sc.ui.get.attachHandler("/new-handler", servlet, "/") + + val newUrl = new URL(sc.uiWebUrl.get + "/new-handler/") + assert(TestUtils.httpResponseCode(newUrl) === HttpServletResponse.SC_NOT_ACCEPTABLE) + + val bypassUrl = new URL(sc.uiWebUrl.get + "/new-handler/?bypass") + assert(TestUtils.httpResponseCode(bypassUrl) === HttpServletResponse.SC_CONFLICT) } } + +// Just extend the test filter so we can configure two of them. +class TestFilter2 extends TestFilter diff --git a/sbin/spark-config.sh b/sbin/spark-config.sh index bf3da18c3706e..0771e2a044757 100755 --- a/sbin/spark-config.sh +++ b/sbin/spark-config.sh @@ -28,6 +28,6 @@ export SPARK_CONF_DIR="${SPARK_CONF_DIR:-"${SPARK_HOME}/conf"}" # Add the PySpark classes to the PYTHONPATH: if [ -z "${PYSPARK_PYTHONPATH_SET}" ]; then export PYTHONPATH="${SPARK_HOME}/python:${PYTHONPATH}" - export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.7-src.zip:${PYTHONPATH}" + export PYTHONPATH="${SPARK_HOME}/python/lib/py4j-0.10.8.1-src.zip:${PYTHONPATH}" export PYSPARK_PYTHONPATH_SET=1 fi diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh index 38a43b98c3992..71dace47767cb 100755 --- a/sbin/start-history-server.sh +++ b/sbin/start-history-server.sh @@ -28,7 +28,22 @@ if [ -z "${SPARK_HOME}" ]; then export SPARK_HOME="$(cd "`dirname "$0"`"/..; pwd)" fi +# NOTE: This exact class name is matched downstream by SparkSubmit. +# Any changes need to be reflected there. +CLASS="org.apache.spark.deploy.history.HistoryServer" + +if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then + echo "Usage: ./sbin/start-history-server.sh [options]" + pattern="Usage:" + pattern+="\|Using Spark's default log4j profile:" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" + + "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 + exit 1 +fi + . "${SPARK_HOME}/sbin/spark-config.sh" . "${SPARK_HOME}/bin/load-spark-env.sh" -exec "${SPARK_HOME}/sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 "$@" +exec "${SPARK_HOME}/sbin"/spark-daemon.sh start $CLASS 1 "$@" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 97ee32159b6de..b6a566e4daf4b 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -31,7 +31,8 @@ if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then echo "Usage: ./sbin/start-master.sh [options]" pattern="Usage:" pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Registered signal handlers for" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 exit 1 diff --git a/sbin/start-mesos-shuffle-service.sh b/sbin/start-mesos-shuffle-service.sh index 1845845676029..7b7dc9cf5d866 100755 --- a/sbin/start-mesos-shuffle-service.sh +++ b/sbin/start-mesos-shuffle-service.sh @@ -17,11 +17,11 @@ # limitations under the License. # -# Starts the Mesos external shuffle server on the machine this script is executed on. +# Starts the Mesos external shuffle service on the machine this script is executed on. # The Mesos external shuffle service detects when an application exits and automatically # cleans up its shuffle files. # -# Usage: start-mesos-shuffle-server.sh +# Usage: start-mesos-shuffle-service.sh # # Use the SPARK_SHUFFLE_OPTS environment variable to set shuffle service configuration. # diff --git a/sbin/start-slave.sh b/sbin/start-slave.sh index 8c268b8859155..2cb17a04f67ef 100755 --- a/sbin/start-slave.sh +++ b/sbin/start-slave.sh @@ -40,10 +40,11 @@ fi CLASS="org.apache.spark.deploy.worker.Worker" if [[ $# -lt 1 ]] || [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then - echo "Usage: ./sbin/start-slave.sh [options] " + echo "Usage: ./sbin/start-slave.sh [options]" pattern="Usage:" pattern+="\|Using Spark's default log4j profile:" - pattern+="\|Registered signal handlers for" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-class $CLASS --help 2>&1 | grep -v "$pattern" 1>&2 exit 1 diff --git a/sbin/start-thriftserver.sh b/sbin/start-thriftserver.sh index f02f31793e346..b1d38713218b7 100755 --- a/sbin/start-thriftserver.sh +++ b/sbin/start-thriftserver.sh @@ -39,6 +39,10 @@ function usage { pattern+="\|Spark Command: " pattern+="\|=======" pattern+="\|--help" + pattern+="\|Using Spark's default log4j profile:" + pattern+="\|^log4j:" + pattern+="\|Started daemon with process name" + pattern+="\|Registered signal handler for" "${SPARK_HOME}"/bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 echo @@ -48,7 +52,7 @@ function usage { if [[ "$@" = *--help ]] || [[ "$@" = *-h ]]; then usage - exit 0 + exit 1 fi export SUBMIT_USAGE_FUNCTION=usage diff --git a/scalastyle-config.xml b/scalastyle-config.xml index 36a73e3362218..4892819ae9973 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -240,6 +240,17 @@ This file is divided into 3 sections: ]]> + + throw new \w+Error\( + + + JavaConversions diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 2e7df4fd14042..79d9b4951b151 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-catalyst_2.11 + spark-catalyst_2.12 jar Spark Project Catalyst http://spark.apache.org/ @@ -71,6 +71,11 @@ test-jar test + + org.mockito + mockito-core + test + org.apache.spark @@ -103,6 +108,16 @@ commons-codec commons-codec + + com.univocity + univocity-parsers + 2.7.3 + jar + + + org.apache.arrow + arrow-vector + target/scala-${scala.binary.version}/classes @@ -150,6 +165,7 @@ true ../catalyst/src/main/antlr4 + true 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 3a9d1cb21d115..f70c44ca15ba1 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 @@ -44,6 +44,11 @@ grammar SqlBase; return true; } } + + /** + * When true, ANSI SQL parsing mode is enabled. + */ + public boolean ansi = false; } singleStatement @@ -58,6 +63,10 @@ singleTableIdentifier : tableIdentifier EOF ; +singleMultipartIdentifier + : multipartIdentifier EOF + ; + singleFunctionIdentifier : functionIdentifier EOF ; @@ -72,15 +81,20 @@ singleTableSchema statement : query #statementDefault - | USE db=identifier #use - | CREATE DATABASE (IF NOT EXISTS)? identifier - (COMMENT comment=STRING)? locationSpec? - (WITH DBPROPERTIES tablePropertyList)? #createDatabase - | ALTER DATABASE identifier SET DBPROPERTIES tablePropertyList #setDatabaseProperties - | DROP DATABASE (IF EXISTS)? identifier (RESTRICT | CASCADE)? #dropDatabase + | ctes? dmlStatementNoWith #dmlStatement + | USE db=errorCapturingIdentifier #use + | CREATE database (IF NOT EXISTS)? db=errorCapturingIdentifier + ((COMMENT comment=STRING) | + locationSpec | + (WITH DBPROPERTIES tablePropertyList))* #createDatabase + | ALTER database db=errorCapturingIdentifier + SET DBPROPERTIES tablePropertyList #setDatabaseProperties + | DROP database (IF EXISTS)? db=errorCapturingIdentifier + (RESTRICT | CASCADE)? #dropDatabase + | SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases | createTableHeader ('(' colTypeList ')')? tableProvider ((OPTIONS options=tablePropertyList) | - (PARTITIONED BY partitionColumnNames=identifierList) | + (PARTITIONED BY partitioning=transformList) | bucketSpec | locationSpec | (COMMENT comment=STRING) | @@ -88,7 +102,8 @@ statement (AS? query)? #createTable | createTableHeader ('(' columns=colTypeList ')')? ((COMMENT comment=STRING) | - (PARTITIONED BY '(' partitionColumns=colTypeList ')') | + (PARTITIONED BY '(' partitionColumns=colTypeList ')' | + PARTITIONED BY partitionColumnNames=identifierList) | bucketSpec | skewSpec | rowFormat | @@ -100,16 +115,30 @@ statement LIKE source=tableIdentifier locationSpec? #createTableLike | ANALYZE TABLE tableIdentifier partitionSpec? COMPUTE STATISTICS (identifier | FOR COLUMNS identifierSeq | FOR ALL COLUMNS)? #analyze - | ALTER TABLE tableIdentifier - ADD COLUMNS '(' columns=colTypeList ')' #addTableColumns + | ALTER TABLE multipartIdentifier + ADD (COLUMN | COLUMNS) + columns=qualifiedColTypeWithPositionList #addTableColumns + | ALTER TABLE multipartIdentifier + ADD (COLUMN | COLUMNS) + '(' columns=qualifiedColTypeWithPositionList ')' #addTableColumns + | ALTER TABLE multipartIdentifier + RENAME COLUMN from=qualifiedName TO to=identifier #renameTableColumn + | ALTER TABLE multipartIdentifier + DROP (COLUMN | COLUMNS) '(' columns=qualifiedNameList ')' #dropTableColumns + | ALTER TABLE multipartIdentifier + DROP (COLUMN | COLUMNS) columns=qualifiedNameList #dropTableColumns | ALTER (TABLE | VIEW) from=tableIdentifier RENAME TO to=tableIdentifier #renameTable - | ALTER (TABLE | VIEW) tableIdentifier + | ALTER (TABLE | VIEW) multipartIdentifier SET TBLPROPERTIES tablePropertyList #setTableProperties - | ALTER (TABLE | VIEW) tableIdentifier + | ALTER (TABLE | VIEW) multipartIdentifier UNSET TBLPROPERTIES (IF EXISTS)? tablePropertyList #unsetTableProperties + | ALTER TABLE multipartIdentifier + (ALTER | CHANGE) COLUMN? qualifiedName + (TYPE dataType)? (COMMENT comment=STRING)? colPosition? #alterTableColumn | ALTER TABLE tableIdentifier partitionSpec? - CHANGE COLUMN? identifier colType colPosition? #changeColumn + CHANGE COLUMN? + colName=errorCapturingIdentifier colType colPosition? #changeColumn | ALTER TABLE tableIdentifier (partitionSpec)? SET SERDE STRING (WITH SERDEPROPERTIES tablePropertyList)? #setTableSerDe | ALTER TABLE tableIdentifier (partitionSpec)? @@ -124,15 +153,18 @@ statement dropPartitionSpec (',' dropPartitionSpec)* PURGE? #dropTablePartitions | ALTER VIEW tableIdentifier DROP (IF EXISTS)? dropPartitionSpec (',' dropPartitionSpec)* #dropTablePartitions - | ALTER TABLE tableIdentifier partitionSpec? SET locationSpec #setTableLocation + | ALTER TABLE multipartIdentifier SET locationSpec #setTableLocation + | ALTER TABLE tableIdentifier partitionSpec SET locationSpec #setPartitionLocation | ALTER TABLE tableIdentifier RECOVER PARTITIONS #recoverPartitions - | DROP TABLE (IF EXISTS)? tableIdentifier PURGE? #dropTable - | DROP VIEW (IF EXISTS)? tableIdentifier #dropTable + | DROP TABLE (IF EXISTS)? multipartIdentifier PURGE? #dropTable + | DROP VIEW (IF EXISTS)? multipartIdentifier #dropView | CREATE (OR REPLACE)? (GLOBAL? TEMPORARY)? VIEW (IF NOT EXISTS)? tableIdentifier - identifierCommentList? (COMMENT STRING)? - (PARTITIONED ON identifierList)? - (TBLPROPERTIES tablePropertyList)? AS query #createView + identifierCommentList? + ((COMMENT STRING) | + (PARTITIONED ON identifierList) | + (TBLPROPERTIES tablePropertyList))* + AS query #createView | CREATE (OR REPLACE)? GLOBAL? TEMPORARY VIEW tableIdentifier ('(' colTypeList ')')? tableProvider (OPTIONS tablePropertyList)? #createTempViewUsing @@ -143,26 +175,27 @@ statement | DROP TEMPORARY? FUNCTION (IF EXISTS)? qualifiedName #dropFunction | EXPLAIN (LOGICAL | FORMATTED | EXTENDED | CODEGEN | COST)? statement #explain - | SHOW TABLES ((FROM | IN) db=identifier)? + | SHOW TABLES ((FROM | IN) db=errorCapturingIdentifier)? (LIKE? pattern=STRING)? #showTables - | SHOW TABLE EXTENDED ((FROM | IN) db=identifier)? + | SHOW TABLE EXTENDED ((FROM | IN) db=errorCapturingIdentifier)? LIKE pattern=STRING partitionSpec? #showTable - | SHOW DATABASES (LIKE? pattern=STRING)? #showDatabases | SHOW TBLPROPERTIES table=tableIdentifier ('(' key=tablePropertyKey ')')? #showTblProperties | SHOW COLUMNS (FROM | IN) tableIdentifier - ((FROM | IN) db=identifier)? #showColumns + ((FROM | IN) db=errorCapturingIdentifier)? #showColumns | SHOW PARTITIONS tableIdentifier partitionSpec? #showPartitions | SHOW identifier? FUNCTIONS (LIKE? (qualifiedName | pattern=STRING))? #showFunctions | SHOW CREATE TABLE tableIdentifier #showCreateTable | (DESC | DESCRIBE) FUNCTION EXTENDED? describeFuncName #describeFunction - | (DESC | DESCRIBE) DATABASE EXTENDED? identifier #describeDatabase + | (DESC | DESCRIBE) database EXTENDED? db=errorCapturingIdentifier #describeDatabase | (DESC | DESCRIBE) TABLE? option=(EXTENDED | FORMATTED)? tableIdentifier partitionSpec? describeColName? #describeTable + | (DESC | DESCRIBE) QUERY? query #describeQuery | REFRESH TABLE tableIdentifier #refreshTable | REFRESH (STRING | .*?) #refreshResource - | CACHE LAZY? TABLE tableIdentifier (AS? query)? #cacheTable + | CACHE LAZY? TABLE tableIdentifier + (OPTIONS options=tablePropertyList)? (AS? query)? #cacheTable | UNCACHE TABLE (IF EXISTS)? tableIdentifier #uncacheTable | CLEAR CACHE #clearCache | LOAD DATA LOCAL? INPATH path=STRING OVERWRITE? INTO TABLE @@ -225,7 +258,7 @@ unsupportedHiveNativeCommands ; createTableHeader - : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? tableIdentifier + : CREATE TEMPORARY? EXTERNAL? TABLE (IF NOT EXISTS)? multipartIdentifier ; bucketSpec @@ -245,7 +278,7 @@ locationSpec ; query - : ctes? queryNoWith + : ctes? queryTerm queryOrganization ; insertInto @@ -275,6 +308,11 @@ dropPartitionVal : identifier (comparisonOperator constant)? ; +database + : DATABASE + | SCHEMA + ; + describeFuncName : qualifiedName | STRING @@ -292,7 +330,7 @@ ctes ; namedQuery - : name=identifier AS? '(' query ')' + : name=errorCapturingIdentifier (columnAliases=identifierList)? AS? '(' query ')' ; tableProvider @@ -345,9 +383,9 @@ resource : identifier STRING ; -queryNoWith - : insertInto? queryTerm queryOrganization #singleInsertQuery - | fromClause multiInsertQueryBody+ #multiInsertQuery +dmlStatementNoWith + : insertInto queryTerm queryOrganization #singleInsertQuery + | fromClause multiInsertQueryBody+ #multiInsertQuery ; queryOrganization @@ -355,14 +393,12 @@ queryOrganization (CLUSTER BY clusterBy+=expression (',' clusterBy+=expression)*)? (DISTRIBUTE BY distributeBy+=expression (',' distributeBy+=expression)*)? (SORT BY sort+=sortItem (',' sort+=sortItem)*)? - windows? + windowClause? (LIMIT (ALL | limit=expression))? ; multiInsertQueryBody - : insertInto? - querySpecification - queryOrganization + : insertInto fromStatementBody ; queryTerm @@ -377,34 +413,68 @@ queryTerm queryPrimary : querySpecification #queryPrimaryDefault - | TABLE tableIdentifier #table + | fromStatement #fromStmt + | TABLE multipartIdentifier #table | inlineTable #inlineTableDefault1 - | '(' queryNoWith ')' #subquery + | '(' query ')' #subquery ; sortItem : expression ordering=(ASC | DESC)? (NULLS nullOrder=(LAST | FIRST))? ; +fromStatement + : fromClause fromStatementBody+ + ; + +fromStatementBody + : transformClause + whereClause? + queryOrganization + | selectClause + lateralView* + whereClause? + aggregationClause? + havingClause? + windowClause? + queryOrganization + ; + querySpecification - : (((SELECT kind=TRANSFORM '(' namedExpressionSeq ')' - | kind=MAP namedExpressionSeq - | kind=REDUCE namedExpressionSeq)) - inRowFormat=rowFormat? - (RECORDWRITER recordWriter=STRING)? - USING script=STRING - (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))? - outRowFormat=rowFormat? - (RECORDREADER recordReader=STRING)? - fromClause? - (WHERE where=booleanExpression)?) - | ((kind=SELECT (hints+=hint)* setQuantifier? namedExpressionSeq fromClause? - | fromClause (kind=SELECT setQuantifier? namedExpressionSeq)?) - lateralView* - (WHERE where=booleanExpression)? - aggregation? - (HAVING having=booleanExpression)? - windows?) + : transformClause + fromClause? + whereClause? #transformQuerySpecification + | selectClause + fromClause? + lateralView* + whereClause? + aggregationClause? + havingClause? + windowClause? #regularQuerySpecification + ; + +transformClause + : (SELECT kind=TRANSFORM '(' namedExpressionSeq ')' + | kind=MAP namedExpressionSeq + | kind=REDUCE namedExpressionSeq) + inRowFormat=rowFormat? + (RECORDWRITER recordWriter=STRING)? + USING script=STRING + (AS (identifierSeq | colTypeList | ('(' (identifierSeq | colTypeList) ')')))? + outRowFormat=rowFormat? + (RECORDREADER recordReader=STRING)? + ; + +selectClause + : SELECT (hints+=hint)* setQuantifier? namedExpressionSeq + ; + +whereClause + : WHERE booleanExpression + ; + +havingClause + : HAVING booleanExpression ; hint @@ -420,7 +490,7 @@ fromClause : FROM relation (',' relation)* lateralView* pivotClause? ; -aggregation +aggregationClause : GROUP BY groupingExpressions+=expression (',' groupingExpressions+=expression)* ( WITH kind=ROLLUP | WITH kind=CUBE @@ -468,7 +538,7 @@ joinType : INNER? | CROSS | LEFT OUTER? - | LEFT SEMI + | LEFT? SEMI | RIGHT OUTER? | FULL OUTER? | LEFT? ANTI @@ -496,7 +566,7 @@ identifierList ; identifierSeq - : identifier (',' identifier)* + : ident+=errorCapturingIdentifier (',' ident+=errorCapturingIdentifier)* ; orderedIdentifierList @@ -504,7 +574,7 @@ orderedIdentifierList ; orderedIdentifier - : identifier ordering=(ASC | DESC)? + : ident=errorCapturingIdentifier ordering=(ASC | DESC)? ; identifierCommentList @@ -516,8 +586,8 @@ identifierComment ; relationPrimary - : tableIdentifier sample? tableAlias #tableName - | '(' queryNoWith ')' sample? tableAlias #aliasedQuery + : multipartIdentifier sample? tableAlias #tableName + | '(' query ')' sample? tableAlias #aliasedQuery | '(' relation ')' sample? tableAlias #aliasedRelation | inlineTable #inlineTableDefault2 | functionTable #tableValuedFunction @@ -528,7 +598,7 @@ inlineTable ; functionTable - : identifier '(' (expression (',' expression)*)? ')' tableAlias + : funcName=errorCapturingIdentifier '(' (expression (',' expression)*)? ')' tableAlias ; tableAlias @@ -545,22 +615,41 @@ rowFormat (NULL DEFINED AS nullDefinedAs=STRING)? #rowFormatDelimited ; +multipartIdentifier + : parts+=errorCapturingIdentifier ('.' parts+=errorCapturingIdentifier)* + ; + tableIdentifier - : (db=identifier '.')? table=identifier + : (db=errorCapturingIdentifier '.')? table=errorCapturingIdentifier ; functionIdentifier - : (db=identifier '.')? function=identifier + : (db=errorCapturingIdentifier '.')? function=errorCapturingIdentifier ; namedExpression - : expression (AS? (identifier | identifierList))? + : expression (AS? (name=errorCapturingIdentifier | identifierList))? ; namedExpressionSeq : namedExpression (',' namedExpression)* ; +transformList + : '(' transforms+=transform (',' transforms+=transform)* ')' + ; + +transform + : qualifiedName #identityTransform + | transformName=identifier + '(' argument+=transformArgument (',' argument+=transformArgument)* ')' #applyTransform + ; + +transformArgument + : qualifiedName + | constant + ; + expression : booleanExpression ; @@ -594,12 +683,13 @@ valueExpression ; primaryExpression - : CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase + : name=(CURRENT_DATE | CURRENT_TIMESTAMP) #currentDatetime + | CASE whenClause+ (ELSE elseExpression=expression)? END #searchedCase | CASE value=expression whenClause+ (ELSE elseExpression=expression)? END #simpleCase | CAST '(' expression AS dataType ')' #cast | STRUCT '(' (argument+=namedExpression (',' argument+=namedExpression)*)? ')' #struct - | FIRST '(' expression (IGNORE NULLS)? ')' #first - | LAST '(' expression (IGNORE NULLS)? ')' #last + | (FIRST | FIRST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #first + | (LAST | LAST_VALUE) '(' expression ((IGNORE | RESPECT) NULLS)? ')' #last | POSITION '(' substr=valueExpression IN str=valueExpression ')' #position | constant #constantDefault | ASTERISK #star @@ -608,8 +698,6 @@ primaryExpression | '(' query ')' #subqueryExpression | qualifiedName '(' (setQuantifier? argument+=expression (',' argument+=expression)*)? ')' (OVER windowSpec)? #functionCall - | qualifiedName '(' trimOption=(BOTH | LEADING | TRAILING) argument+=expression - FROM argument+=expression ')' #functionCall | IDENTIFIER '->' expression #lambda | '(' IDENTIFIER (',' IDENTIFIER)+ ')' '->' expression #lambda | value=primaryExpression '[' index=valueExpression ']' #subscript @@ -617,6 +705,12 @@ primaryExpression | base=primaryExpression '.' fieldName=identifier #dereference | '(' expression ')' #parenthesizedExpression | EXTRACT '(' field=identifier FROM source=valueExpression ')' #extract + | (SUBSTR | SUBSTRING) '(' str=valueExpression (FROM | ',') pos=valueExpression + ((FOR | ',') len=valueExpression)? ')' #substring + | TRIM '(' trimOption=(BOTH | LEADING | TRAILING)? (trimStr=valueExpression)? + FROM srcStr=valueExpression ')' #trim + | OVERLAY '(' input=valueExpression PLACING replace=valueExpression + FROM position=valueExpression (FOR length=valueExpression)? ')' #overlay ; constant @@ -645,11 +739,12 @@ booleanValue ; interval - : INTERVAL intervalField* + : {ansi}? INTERVAL? intervalField+ + | {!ansi}? INTERVAL intervalField* ; intervalField - : value=intervalValue unit=identifier (TO to=identifier)? + : value=intervalValue unit=intervalUnit (TO to=intervalUnit)? ; intervalValue @@ -657,8 +752,29 @@ intervalValue | STRING ; +intervalUnit + : DAY + | DAYS + | HOUR + | HOURS + | MICROSECOND + | MICROSECONDS + | MILLISECOND + | MILLISECONDS + | MINUTE + | MINUTES + | MONTH + | MONTHS + | SECOND + | SECONDS + | WEEK + | WEEKS + | YEAR + | YEARS + ; + colPosition - : FIRST | AFTER identifier + : FIRST | AFTER qualifiedName ; dataType @@ -668,12 +784,20 @@ dataType | identifier ('(' INTEGER_VALUE (',' INTEGER_VALUE)* ')')? #primitiveDataType ; +qualifiedColTypeWithPositionList + : qualifiedColTypeWithPosition (',' qualifiedColTypeWithPosition)* + ; + +qualifiedColTypeWithPosition + : name=qualifiedName dataType (COMMENT comment=STRING)? colPosition? + ; + colTypeList : colType (',' colType)* ; colType - : identifier dataType (COMMENT STRING)? + : colName=errorCapturingIdentifier dataType (COMMENT STRING)? ; complexColTypeList @@ -688,22 +812,23 @@ whenClause : WHEN condition=expression THEN result=expression ; -windows +windowClause : WINDOW namedWindow (',' namedWindow)* ; namedWindow - : identifier AS windowSpec + : name=errorCapturingIdentifier AS windowSpec ; windowSpec - : name=identifier #windowRef + : name=errorCapturingIdentifier #windowRef + | '('name=errorCapturingIdentifier')' #windowRef | '(' ( CLUSTER BY partition+=expression (',' partition+=expression)* | ((PARTITION | DISTRIBUTE) BY partition+=expression (',' partition+=expression)*)? ((ORDER | SORT) BY sortItem (',' sortItem)*)?) windowFrame? - ')' #windowDef + ')' #windowDef ; windowFrame @@ -719,20 +844,37 @@ frameBound | expression boundType=(PRECEDING | FOLLOWING) ; +qualifiedNameList + : qualifiedName (',' qualifiedName)* + ; + qualifiedName : identifier ('.' identifier)* ; +// this rule is used for explicitly capturing wrong identifiers such as test-table, which should actually be `test-table` +// replace identifier with errorCapturingIdentifier where the immediate follow symbol is not an expression, otherwise +// valid expressions such as "a-b" can be recognized as an identifier +errorCapturingIdentifier + : identifier errorCapturingIdentifierExtra + ; + +// extra left-factoring grammar +errorCapturingIdentifierExtra + : (MINUS identifier)+ #errorIdent + | #realIdent + ; + identifier : strictIdentifier - | ANTI | FULL | INNER | LEFT | SEMI | RIGHT | NATURAL | JOIN | CROSS | ON - | UNION | INTERSECT | EXCEPT | SETMINUS + | {!ansi}? strictNonReserved ; strictIdentifier - : IDENTIFIER #unquotedIdentifier - | quotedIdentifier #quotedIdentifierAlternative - | nonReserved #unquotedIdentifier + : IDENTIFIER #unquotedIdentifier + | quotedIdentifier #quotedIdentifierAlternative + | {ansi}? ansiNonReserved #unquotedIdentifier + | {!ansi}? nonReserved #unquotedIdentifier ; quotedIdentifier @@ -749,162 +891,740 @@ number | MINUS? BIGDECIMAL_LITERAL #bigDecimalLiteral ; -nonReserved - : SHOW | TABLES | COLUMNS | COLUMN | PARTITIONS | FUNCTIONS | DATABASES - | ADD - | OVER | PARTITION | RANGE | ROWS | PRECEDING | FOLLOWING | CURRENT | ROW | LAST | FIRST | AFTER - | MAP | ARRAY | STRUCT - | PIVOT | LATERAL | WINDOW | REDUCE | TRANSFORM | SERDE | SERDEPROPERTIES | RECORDREADER - | DELIMITED | FIELDS | TERMINATED | COLLECTION | ITEMS | KEYS | ESCAPED | LINES | SEPARATED - | EXTENDED | REFRESH | CLEAR | CACHE | UNCACHE | LAZY | GLOBAL | TEMPORARY | OPTIONS - | GROUPING | CUBE | ROLLUP - | EXPLAIN | FORMAT | LOGICAL | FORMATTED | CODEGEN | COST - | TABLESAMPLE | USE | TO | BUCKET | PERCENTLIT | OUT | OF - | SET | RESET - | VIEW | REPLACE +// When `spark.sql.parser.ansi.enabled=true`, there are 2 kinds of keywords in Spark SQL. +// - Reserved keywords: +// Keywords that are reserved and can't be used as identifiers for table, view, column, +// function, alias, etc. +// - Non-reserved keywords: +// Keywords that have a special meaning only in particular contexts and can be used as +// identifiers in other contexts. For example, `SELECT 1 WEEK` is an interval literal, but WEEK +// can be used as identifiers in other places. +// You can find the full keywords list by searching "Start of the keywords list" in this file. +// The non-reserved keywords are listed below. Keywords not in this list are reserved keywords. +ansiNonReserved + : ADD + | AFTER + | ALTER + | ANALYZE + | ARCHIVE + | ARRAY + | ASC + | AT + | BETWEEN + | BUCKET + | BUCKETS + | BY + | CACHE + | CASCADE + | CHANGE + | CLEAR + | CLUSTER + | CLUSTERED + | CODEGEN + | COLLECTION + | COLUMNS + | COMMENT + | COMMIT + | COMPACT + | COMPACTIONS + | COMPUTE + | CONCATENATE + | COST + | CUBE + | CURRENT + | DATA + | DATABASE + | DATABASES + | DAYS + | DBPROPERTIES + | DEFINED + | DELETE + | DELIMITED + | DESC + | DESCRIBE + | DFS + | DIRECTORIES + | DIRECTORY + | DISTRIBUTE + | DIV + | DROP + | ESCAPED + | EXCHANGE + | EXISTS + | EXPLAIN + | EXPORT + | EXTENDED + | EXTERNAL + | EXTRACT + | FIELDS + | FILEFORMAT + | FIRST + | FOLLOWING + | FORMAT + | FORMATTED + | FUNCTION + | FUNCTIONS + | GLOBAL + | GROUPING + | HOURS | IF + | IGNORE + | IMPORT + | INDEX + | INDEXES + | INPATH + | INPUTFORMAT + | INSERT + | INTERVAL + | ITEMS + | KEYS + | LAST + | LATERAL + | LAZY + | LIKE + | LIMIT + | LINES + | LIST + | LOAD + | LOCAL + | LOCATION + | LOCK + | LOCKS + | LOGICAL + | MACRO + | MAP + | MICROSECOND + | MICROSECONDS + | MILLISECOND + | MILLISECONDS + | MINUTES + | MONTHS + | MSCK + | NO + | NULLS + | OF + | OPTION + | OPTIONS + | OUT + | OUTPUTFORMAT + | OVER + | OVERLAY + | OVERWRITE + | PARTITION + | PARTITIONED + | PARTITIONS + | PERCENTLIT + | PIVOT + | PLACING | POSITION - | EXTRACT - | NO | DATA - | START | TRANSACTION | COMMIT | ROLLBACK | IGNORE - | SORT | CLUSTER | DISTRIBUTE | UNSET | TBLPROPERTIES | SKEWED | STORED | DIRECTORIES | LOCATION - | EXCHANGE | ARCHIVE | UNARCHIVE | FILEFORMAT | TOUCH | COMPACT | CONCATENATE | CHANGE - | CASCADE | RESTRICT | BUCKETS | CLUSTERED | SORTED | PURGE | INPUTFORMAT | OUTPUTFORMAT - | DBPROPERTIES | DFS | TRUNCATE | COMPUTE | LIST - | STATISTICS | ANALYZE | PARTITIONED | EXTERNAL | DEFINED | RECORDWRITER - | REVOKE | GRANT | LOCK | UNLOCK | MSCK | REPAIR | RECOVER | EXPORT | IMPORT | LOAD | VALUES | COMMENT | ROLE - | ROLES | COMPACTIONS | PRINCIPALS | TRANSACTIONS | INDEX | INDEXES | LOCKS | OPTION | LOCAL | INPATH - | ASC | DESC | LIMIT | RENAME | SETS - | AT | NULLS | OVERWRITE | ALL | ANY | ALTER | AS | BETWEEN | BY | CREATE | DELETE - | DESCRIBE | DROP | EXISTS | FALSE | FOR | GROUP | IN | INSERT | INTO | IS |LIKE - | NULL | ORDER | OUTER | TABLE | TRUE | WITH | RLIKE - | AND | CASE | CAST | DISTINCT | DIV | ELSE | END | FUNCTION | INTERVAL | MACRO | OR | STRATIFY | THEN - | UNBOUNDED | WHEN - | DATABASE | SELECT | FROM | WHERE | HAVING | TO | TABLE | WITH | NOT - | DIRECTORY - | BOTH | LEADING | TRAILING + | PRECEDING + | PRINCIPALS + | PURGE + | QUERY + | RANGE + | RECORDREADER + | RECORDWRITER + | RECOVER + | REDUCE + | REFRESH + | RENAME + | REPAIR + | REPLACE + | RESET + | RESPECT + | RESTRICT + | REVOKE + | RLIKE + | ROLE + | ROLES + | ROLLBACK + | ROLLUP + | ROW + | ROWS + | SCHEMA + | SECONDS + | SEPARATED + | SERDE + | SERDEPROPERTIES + | SET + | SETS + | SHOW + | SKEWED + | SORT + | SORTED + | START + | STATISTICS + | STORED + | STRATIFY + | STRUCT + | SUBSTR + | SUBSTRING + | TABLES + | TABLESAMPLE + | TBLPROPERTIES + | TEMPORARY + | TERMINATED + | TOUCH + | TRANSACTION + | TRANSACTIONS + | TRANSFORM + | TRIM + | TRUE + | TRUNCATE + | UNARCHIVE + | UNBOUNDED + | UNCACHE + | UNLOCK + | UNSET + | USE + | VALUES + | VIEW + | WEEK + | WEEKS + | WINDOW + | YEARS + ; + +// When `spark.sql.parser.ansi.enabled=false`, there are 2 kinds of keywords in Spark SQL. +// - Non-reserved keywords: +// Same definition as the one when `spark.sql.parser.ansi.enabled=true`. +// - Strict-non-reserved keywords: +// A strict version of non-reserved keywords, which can not be used as table alias. +// You can find the full keywords list by searching "Start of the keywords list" in this file. +// The strict-non-reserved keywords are listed in `strictNonReserved`. +// The non-reserved keywords are listed in `nonReserved`. +// These 2 together contain all the keywords. +strictNonReserved + : ANTI + | CROSS + | EXCEPT + | FULL + | INNER + | INTERSECT + | JOIN + | LEFT + | NATURAL + | ON + | RIGHT + | SEMI + | SETMINUS + | UNION + | USING ; -SELECT: 'SELECT'; -FROM: 'FROM'; +nonReserved + : ADD + | AFTER + | ALL + | ALTER + | ANALYZE + | AND + | ANY + | ARCHIVE + | ARRAY + | AS + | ASC + | AT + | AUTHORIZATION + | BETWEEN + | BOTH + | BUCKET + | BUCKETS + | BY + | CACHE + | CASCADE + | CASE + | CAST + | CHANGE + | CHECK + | CLEAR + | CLUSTER + | CLUSTERED + | CODEGEN + | COLLATE + | COLLECTION + | COLUMN + | COLUMNS + | COMMENT + | COMMIT + | COMPACT + | COMPACTIONS + | COMPUTE + | CONCATENATE + | CONSTRAINT + | COST + | CREATE + | CUBE + | CURRENT + | CURRENT_DATE + | CURRENT_TIME + | CURRENT_TIMESTAMP + | CURRENT_USER + | DATA + | DATABASE + | DATABASES + | DAY + | DAYS + | DBPROPERTIES + | DEFINED + | DELETE + | DELIMITED + | DESC + | DESCRIBE + | DFS + | DIRECTORIES + | DIRECTORY + | DISTINCT + | DISTRIBUTE + | DIV + | DROP + | ELSE + | END + | ESCAPED + | EXCHANGE + | EXISTS + | EXPLAIN + | EXPORT + | EXTENDED + | EXTERNAL + | EXTRACT + | FALSE + | FETCH + | FIELDS + | FILEFORMAT + | FIRST + | FIRST_VALUE + | FOLLOWING + | FOR + | FOREIGN + | FORMAT + | FORMATTED + | FROM + | FUNCTION + | FUNCTIONS + | GLOBAL + | GRANT + | GROUP + | GROUPING + | HAVING + | HOUR + | HOURS + | IF + | IGNORE + | IMPORT + | IN + | INDEX + | INDEXES + | INPATH + | INPUTFORMAT + | INSERT + | INTERVAL + | INTO + | IS + | ITEMS + | KEYS + | LAST + | LAST_VALUE + | LATERAL + | LAZY + | LEADING + | LIKE + | LIMIT + | LINES + | LIST + | LOAD + | LOCAL + | LOCATION + | LOCK + | LOCKS + | LOGICAL + | MACRO + | MAP + | MICROSECOND + | MICROSECONDS + | MILLISECOND + | MILLISECONDS + | MINUTE + | MINUTES + | MONTH + | MONTHS + | MSCK + | NO + | NOT + | NULL + | NULLS + | OF + | ONLY + | OPTION + | OPTIONS + | OR + | ORDER + | OUT + | OUTER + | OUTPUTFORMAT + | OVER + | OVERLAPS + | OVERLAY + | OVERWRITE + | PARTITION + | PARTITIONED + | PARTITIONS + | PERCENTLIT + | PIVOT + | PLACING + | POSITION + | PRECEDING + | PRIMARY + | PRINCIPALS + | PURGE + | QUERY + | RANGE + | RECORDREADER + | RECORDWRITER + | RECOVER + | REDUCE + | REFERENCES + | REFRESH + | RENAME + | REPAIR + | REPLACE + | RESET + | RESPECT + | RESTRICT + | REVOKE + | RLIKE + | ROLE + | ROLES + | ROLLBACK + | ROLLUP + | ROW + | ROWS + | SCHEMA + | SECOND + | SECONDS + | SELECT + | SEPARATED + | SERDE + | SERDEPROPERTIES + | SESSION_USER + | SET + | SETS + | SHOW + | SKEWED + | SOME + | SORT + | SORTED + | START + | STATISTICS + | STORED + | STRATIFY + | STRUCT + | SUBSTR + | SUBSTRING + | TABLE + | TABLES + | TABLESAMPLE + | TBLPROPERTIES + | TEMPORARY + | TERMINATED + | THEN + | TO + | TOUCH + | TRAILING + | TRANSACTION + | TRANSACTIONS + | TRANSFORM + | TRIM + | TRUE + | TRUNCATE + | TYPE + | UNARCHIVE + | UNBOUNDED + | UNCACHE + | UNIQUE + | UNLOCK + | UNSET + | USE + | USER + | VALUES + | VIEW + | WEEK + | WEEKS + | WHEN + | WHERE + | WINDOW + | WITH + | YEAR + | YEARS + ; + +// NOTE: If you add a new token in the list below, you should update the list of keywords +// in `docs/sql-keywords.md`. If the token is a non-reserved keyword, +// please update `ansiNonReserved` and `nonReserved` as well. + +//============================ +// Start of the keywords list +//============================ ADD: 'ADD'; -AS: 'AS'; +AFTER: 'AFTER'; ALL: 'ALL'; +ALTER: 'ALTER'; +ANALYZE: 'ANALYZE'; +AND: 'AND'; +ANTI: 'ANTI'; ANY: 'ANY'; -DISTINCT: 'DISTINCT'; -WHERE: 'WHERE'; -GROUP: 'GROUP'; +ARCHIVE: 'ARCHIVE'; +ARRAY: 'ARRAY'; +AS: 'AS'; +ASC: 'ASC'; +AT: 'AT'; +AUTHORIZATION: 'AUTHORIZATION'; +BETWEEN: 'BETWEEN'; +BOTH: 'BOTH'; +BUCKET: 'BUCKET'; +BUCKETS: 'BUCKETS'; BY: 'BY'; -GROUPING: 'GROUPING'; -SETS: 'SETS'; +CACHE: 'CACHE'; +CASCADE: 'CASCADE'; +CASE: 'CASE'; +CAST: 'CAST'; +CHANGE: 'CHANGE'; +CHECK: 'CHECK'; +CLEAR: 'CLEAR'; +CLUSTER: 'CLUSTER'; +CLUSTERED: 'CLUSTERED'; +CODEGEN: 'CODEGEN'; +COLLATE: 'COLLATE'; +COLLECTION: 'COLLECTION'; +COLUMN: 'COLUMN'; +COLUMNS: 'COLUMNS'; +COMMENT: 'COMMENT'; +COMMIT: 'COMMIT'; +COMPACT: 'COMPACT'; +COMPACTIONS: 'COMPACTIONS'; +COMPUTE: 'COMPUTE'; +CONCATENATE: 'CONCATENATE'; +CONSTRAINT: 'CONSTRAINT'; +COST: 'COST'; +CREATE: 'CREATE'; +CROSS: 'CROSS'; CUBE: 'CUBE'; -ROLLUP: 'ROLLUP'; -ORDER: 'ORDER'; -HAVING: 'HAVING'; -LIMIT: 'LIMIT'; -AT: 'AT'; -OR: 'OR'; -AND: 'AND'; -IN: 'IN'; -NOT: 'NOT' | '!'; -NO: 'NO'; +CURRENT: 'CURRENT'; +CURRENT_DATE: 'CURRENT_DATE'; +CURRENT_TIME: 'CURRENT_TIME'; +CURRENT_TIMESTAMP: 'CURRENT_TIMESTAMP'; +CURRENT_USER: 'CURRENT_USER'; +DATA: 'DATA'; +DATABASE: 'DATABASE'; +DATABASES: 'DATABASES' | 'SCHEMAS'; +DAY: 'DAY'; +DAYS: 'DAYS'; +DBPROPERTIES: 'DBPROPERTIES'; +DEFINED: 'DEFINED'; +DELETE: 'DELETE'; +DELIMITED: 'DELIMITED'; +DESC: 'DESC'; +DESCRIBE: 'DESCRIBE'; +DFS: 'DFS'; +DIRECTORIES: 'DIRECTORIES'; +DIRECTORY: 'DIRECTORY'; +DISTINCT: 'DISTINCT'; +DISTRIBUTE: 'DISTRIBUTE'; +DROP: 'DROP'; +ELSE: 'ELSE'; +END: 'END'; +ESCAPED: 'ESCAPED'; +EXCEPT: 'EXCEPT'; +EXCHANGE: 'EXCHANGE'; EXISTS: 'EXISTS'; -BETWEEN: 'BETWEEN'; -LIKE: 'LIKE'; -RLIKE: 'RLIKE' | 'REGEXP'; -IS: 'IS'; -NULL: 'NULL'; -TRUE: 'TRUE'; +EXPLAIN: 'EXPLAIN'; +EXPORT: 'EXPORT'; +EXTENDED: 'EXTENDED'; +EXTERNAL: 'EXTERNAL'; +EXTRACT: 'EXTRACT'; FALSE: 'FALSE'; -NULLS: 'NULLS'; -ASC: 'ASC'; -DESC: 'DESC'; +FETCH: 'FETCH'; +FIELDS: 'FIELDS'; +FILEFORMAT: 'FILEFORMAT'; +FIRST: 'FIRST'; +FIRST_VALUE: 'FIRST_VALUE'; +FOLLOWING: 'FOLLOWING'; FOR: 'FOR'; +FOREIGN: 'FOREIGN'; +FORMAT: 'FORMAT'; +FORMATTED: 'FORMATTED'; +FROM: 'FROM'; +FULL: 'FULL'; +FUNCTION: 'FUNCTION'; +FUNCTIONS: 'FUNCTIONS'; +GLOBAL: 'GLOBAL'; +GRANT: 'GRANT'; +GROUP: 'GROUP'; +GROUPING: 'GROUPING'; +HAVING: 'HAVING'; +HOUR: 'HOUR'; +HOURS: 'HOURS'; +IF: 'IF'; +IGNORE: 'IGNORE'; +IMPORT: 'IMPORT'; +IN: 'IN'; +INDEX: 'INDEX'; +INDEXES: 'INDEXES'; +INNER: 'INNER'; +INPATH: 'INPATH'; +INPUTFORMAT: 'INPUTFORMAT'; +INSERT: 'INSERT'; +INTERSECT: 'INTERSECT'; INTERVAL: 'INTERVAL'; -CASE: 'CASE'; -WHEN: 'WHEN'; -THEN: 'THEN'; -ELSE: 'ELSE'; -END: 'END'; +INTO: 'INTO'; +IS: 'IS'; +ITEMS: 'ITEMS'; JOIN: 'JOIN'; -CROSS: 'CROSS'; -OUTER: 'OUTER'; -INNER: 'INNER'; +KEYS: 'KEYS'; +LAST: 'LAST'; +LAST_VALUE: 'LAST_VALUE'; +LATERAL: 'LATERAL'; +LAZY: 'LAZY'; +LEADING: 'LEADING'; LEFT: 'LEFT'; -SEMI: 'SEMI'; -RIGHT: 'RIGHT'; -FULL: 'FULL'; +LIKE: 'LIKE'; +LIMIT: 'LIMIT'; +LINES: 'LINES'; +LIST: 'LIST'; +LOAD: 'LOAD'; +LOCAL: 'LOCAL'; +LOCATION: 'LOCATION'; +LOCK: 'LOCK'; +LOCKS: 'LOCKS'; +LOGICAL: 'LOGICAL'; +MACRO: 'MACRO'; +MAP: 'MAP'; +MICROSECOND: 'MICROSECOND'; +MICROSECONDS: 'MICROSECONDS'; +MILLISECOND: 'MILLISECOND'; +MILLISECONDS: 'MILLISECONDS'; +MINUTE: 'MINUTE'; +MINUTES: 'MINUTES'; +MONTH: 'MONTH'; +MONTHS: 'MONTHS'; +MSCK: 'MSCK'; NATURAL: 'NATURAL'; +NO: 'NO'; +NOT: 'NOT' | '!'; +NULL: 'NULL'; +NULLS: 'NULLS'; +OF: 'OF'; ON: 'ON'; -PIVOT: 'PIVOT'; -LATERAL: 'LATERAL'; -WINDOW: 'WINDOW'; +ONLY: 'ONLY'; +OPTION: 'OPTION'; +OPTIONS: 'OPTIONS'; +OR: 'OR'; +ORDER: 'ORDER'; +OUT: 'OUT'; +OUTER: 'OUTER'; +OUTPUTFORMAT: 'OUTPUTFORMAT'; OVER: 'OVER'; +OVERLAPS: 'OVERLAPS'; +OVERLAY: 'OVERLAY'; +OVERWRITE: 'OVERWRITE'; PARTITION: 'PARTITION'; -RANGE: 'RANGE'; -ROWS: 'ROWS'; -UNBOUNDED: 'UNBOUNDED'; +PARTITIONED: 'PARTITIONED'; +PARTITIONS: 'PARTITIONS'; +PERCENTLIT: 'PERCENT'; +PIVOT: 'PIVOT'; +PLACING: 'PLACING'; +POSITION: 'POSITION'; PRECEDING: 'PRECEDING'; -FOLLOWING: 'FOLLOWING'; -CURRENT: 'CURRENT'; -FIRST: 'FIRST'; -AFTER: 'AFTER'; -LAST: 'LAST'; -ROW: 'ROW'; -WITH: 'WITH'; -VALUES: 'VALUES'; -CREATE: 'CREATE'; -TABLE: 'TABLE'; -DIRECTORY: 'DIRECTORY'; -VIEW: 'VIEW'; +PRIMARY: 'PRIMARY'; +PRINCIPALS: 'PRINCIPALS'; +PURGE: 'PURGE'; +QUERY: 'QUERY'; +RANGE: 'RANGE'; +RECORDREADER: 'RECORDREADER'; +RECORDWRITER: 'RECORDWRITER'; +RECOVER: 'RECOVER'; +REDUCE: 'REDUCE'; +REFERENCES: 'REFERENCES'; +REFRESH: 'REFRESH'; +RENAME: 'RENAME'; +REPAIR: 'REPAIR'; REPLACE: 'REPLACE'; -INSERT: 'INSERT'; -DELETE: 'DELETE'; -INTO: 'INTO'; -DESCRIBE: 'DESCRIBE'; -EXPLAIN: 'EXPLAIN'; -FORMAT: 'FORMAT'; -LOGICAL: 'LOGICAL'; -CODEGEN: 'CODEGEN'; -COST: 'COST'; -CAST: 'CAST'; -SHOW: 'SHOW'; -TABLES: 'TABLES'; -COLUMNS: 'COLUMNS'; -COLUMN: 'COLUMN'; -USE: 'USE'; -PARTITIONS: 'PARTITIONS'; -FUNCTIONS: 'FUNCTIONS'; -DROP: 'DROP'; -UNION: 'UNION'; -EXCEPT: 'EXCEPT'; +RESET: 'RESET'; +RESPECT: 'RESPECT'; +RESTRICT: 'RESTRICT'; +REVOKE: 'REVOKE'; +RIGHT: 'RIGHT'; +RLIKE: 'RLIKE' | 'REGEXP'; +ROLE: 'ROLE'; +ROLES: 'ROLES'; +ROLLBACK: 'ROLLBACK'; +ROLLUP: 'ROLLUP'; +ROW: 'ROW'; +ROWS: 'ROWS'; +SCHEMA: 'SCHEMA'; +SECOND: 'SECOND'; +SECONDS: 'SECONDS'; +SELECT: 'SELECT'; +SEMI: 'SEMI'; +SEPARATED: 'SEPARATED'; +SERDE: 'SERDE'; +SERDEPROPERTIES: 'SERDEPROPERTIES'; +SESSION_USER: 'SESSION_USER'; +SET: 'SET'; SETMINUS: 'MINUS'; -INTERSECT: 'INTERSECT'; -TO: 'TO'; -TABLESAMPLE: 'TABLESAMPLE'; +SETS: 'SETS'; +SHOW: 'SHOW'; +SKEWED: 'SKEWED'; +SOME: 'SOME'; +SORT: 'SORT'; +SORTED: 'SORTED'; +START: 'START'; +STATISTICS: 'STATISTICS'; +STORED: 'STORED'; STRATIFY: 'STRATIFY'; -ALTER: 'ALTER'; -RENAME: 'RENAME'; -ARRAY: 'ARRAY'; -MAP: 'MAP'; STRUCT: 'STRUCT'; -COMMENT: 'COMMENT'; -SET: 'SET'; -RESET: 'RESET'; -DATA: 'DATA'; -START: 'START'; -TRANSACTION: 'TRANSACTION'; -COMMIT: 'COMMIT'; -ROLLBACK: 'ROLLBACK'; -MACRO: 'MACRO'; -IGNORE: 'IGNORE'; -BOTH: 'BOTH'; -LEADING: 'LEADING'; +SUBSTR: 'SUBSTR'; +SUBSTRING: 'SUBSTRING'; +TABLE: 'TABLE'; +TABLES: 'TABLES'; +TABLESAMPLE: 'TABLESAMPLE'; +TBLPROPERTIES: 'TBLPROPERTIES'; +TEMPORARY: 'TEMPORARY' | 'TEMP'; +TERMINATED: 'TERMINATED'; +THEN: 'THEN'; +TO: 'TO'; +TOUCH: 'TOUCH'; TRAILING: 'TRAILING'; - -IF: 'IF'; -POSITION: 'POSITION'; -EXTRACT: 'EXTRACT'; +TRANSACTION: 'TRANSACTION'; +TRANSACTIONS: 'TRANSACTIONS'; +TRANSFORM: 'TRANSFORM'; +TRIM: 'TRIM'; +TRUE: 'TRUE'; +TRUNCATE: 'TRUNCATE'; +TYPE: 'TYPE'; +UNARCHIVE: 'UNARCHIVE'; +UNBOUNDED: 'UNBOUNDED'; +UNCACHE: 'UNCACHE'; +UNION: 'UNION'; +UNIQUE: 'UNIQUE'; +UNLOCK: 'UNLOCK'; +UNSET: 'UNSET'; +USE: 'USE'; +USER: 'USER'; +USING: 'USING'; +VALUES: 'VALUES'; +VIEW: 'VIEW'; +WEEK: 'WEEK'; +WEEKS: 'WEEKS'; +WHEN: 'WHEN'; +WHERE: 'WHERE'; +WINDOW: 'WINDOW'; +WITH: 'WITH'; +YEAR: 'YEAR'; +YEARS: 'YEARS'; +//============================ +// End of the keywords list +//============================ EQ : '=' | '=='; NSEQ: '<=>'; @@ -927,99 +1647,6 @@ PIPE: '|'; CONCAT_PIPE: '||'; HAT: '^'; -PERCENTLIT: 'PERCENT'; -BUCKET: 'BUCKET'; -OUT: 'OUT'; -OF: 'OF'; - -SORT: 'SORT'; -CLUSTER: 'CLUSTER'; -DISTRIBUTE: 'DISTRIBUTE'; -OVERWRITE: 'OVERWRITE'; -TRANSFORM: 'TRANSFORM'; -REDUCE: 'REDUCE'; -USING: 'USING'; -SERDE: 'SERDE'; -SERDEPROPERTIES: 'SERDEPROPERTIES'; -RECORDREADER: 'RECORDREADER'; -RECORDWRITER: 'RECORDWRITER'; -DELIMITED: 'DELIMITED'; -FIELDS: 'FIELDS'; -TERMINATED: 'TERMINATED'; -COLLECTION: 'COLLECTION'; -ITEMS: 'ITEMS'; -KEYS: 'KEYS'; -ESCAPED: 'ESCAPED'; -LINES: 'LINES'; -SEPARATED: 'SEPARATED'; -FUNCTION: 'FUNCTION'; -EXTENDED: 'EXTENDED'; -REFRESH: 'REFRESH'; -CLEAR: 'CLEAR'; -CACHE: 'CACHE'; -UNCACHE: 'UNCACHE'; -LAZY: 'LAZY'; -FORMATTED: 'FORMATTED'; -GLOBAL: 'GLOBAL'; -TEMPORARY: 'TEMPORARY' | 'TEMP'; -OPTIONS: 'OPTIONS'; -UNSET: 'UNSET'; -TBLPROPERTIES: 'TBLPROPERTIES'; -DBPROPERTIES: 'DBPROPERTIES'; -BUCKETS: 'BUCKETS'; -SKEWED: 'SKEWED'; -STORED: 'STORED'; -DIRECTORIES: 'DIRECTORIES'; -LOCATION: 'LOCATION'; -EXCHANGE: 'EXCHANGE'; -ARCHIVE: 'ARCHIVE'; -UNARCHIVE: 'UNARCHIVE'; -FILEFORMAT: 'FILEFORMAT'; -TOUCH: 'TOUCH'; -COMPACT: 'COMPACT'; -CONCATENATE: 'CONCATENATE'; -CHANGE: 'CHANGE'; -CASCADE: 'CASCADE'; -RESTRICT: 'RESTRICT'; -CLUSTERED: 'CLUSTERED'; -SORTED: 'SORTED'; -PURGE: 'PURGE'; -INPUTFORMAT: 'INPUTFORMAT'; -OUTPUTFORMAT: 'OUTPUTFORMAT'; -DATABASE: 'DATABASE' | 'SCHEMA'; -DATABASES: 'DATABASES' | 'SCHEMAS'; -DFS: 'DFS'; -TRUNCATE: 'TRUNCATE'; -ANALYZE: 'ANALYZE'; -COMPUTE: 'COMPUTE'; -LIST: 'LIST'; -STATISTICS: 'STATISTICS'; -PARTITIONED: 'PARTITIONED'; -EXTERNAL: 'EXTERNAL'; -DEFINED: 'DEFINED'; -REVOKE: 'REVOKE'; -GRANT: 'GRANT'; -LOCK: 'LOCK'; -UNLOCK: 'UNLOCK'; -MSCK: 'MSCK'; -REPAIR: 'REPAIR'; -RECOVER: 'RECOVER'; -EXPORT: 'EXPORT'; -IMPORT: 'IMPORT'; -LOAD: 'LOAD'; -ROLE: 'ROLE'; -ROLES: 'ROLES'; -COMPACTIONS: 'COMPACTIONS'; -PRINCIPALS: 'PRINCIPALS'; -TRANSACTIONS: 'TRANSACTIONS'; -INDEX: 'INDEX'; -INDEXES: 'INDEXES'; -LOCKS: 'LOCKS'; -OPTION: 'OPTION'; -ANTI: 'ANTI'; -LOCAL: 'LOCAL'; -INPATH: 'INPATH'; - STRING : '\'' ( ~('\''|'\\') | ('\\' .) )* '\'' | '"' ( ~('"'|'\\') | ('\\' .) )* '"' diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java index 2ce1fdcbf56ae..0258e66ffb6e5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/RowFactory.java @@ -17,7 +17,7 @@ package org.apache.spark.sql; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; import org.apache.spark.sql.catalyst.expressions.GenericRow; /** @@ -25,7 +25,7 @@ * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public class RowFactory { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java new file mode 100644 index 0000000000000..5d4995a05d233 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/CatalogPlugin.java @@ -0,0 +1,61 @@ +/* + * 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.catalog.v2; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A marker interface to provide a catalog implementation for Spark. + *

    + * Implementations can provide catalog functions by implementing additional interfaces for tables, + * views, and functions. + *

    + * Catalog implementations must implement this marker interface to be loaded by + * {@link Catalogs#load(String, SQLConf)}. The loader will instantiate catalog classes using the + * required public no-arg constructor. After creating an instance, it will be configured by calling + * {@link #initialize(String, CaseInsensitiveStringMap)}. + *

    + * Catalog implementations are registered to a name by adding a configuration option to Spark: + * {@code spark.sql.catalog.catalog-name=com.example.YourCatalogClass}. All configuration properties + * in the Spark configuration that share the catalog name prefix, + * {@code spark.sql.catalog.catalog-name.(key)=(value)} will be passed in the case insensitive + * string map of options in initialization with the prefix removed. + * {@code name}, is also passed and is the catalog's name; in this case, "catalog-name". + */ +@Experimental +public interface CatalogPlugin { + /** + * Called to initialize configuration. + *

    + * This method is called once, just after the provider is instantiated. + * + * @param name the name used to identify and load this catalog + * @param options a case-insensitive string map of configuration + */ + void initialize(String name, CaseInsensitiveStringMap options); + + /** + * Called to get this catalog's name. + *

    + * This method is only called after {@link #initialize(String, CaseInsensitiveStringMap)} is + * called to pass the catalog's name. + */ + String name(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java new file mode 100644 index 0000000000000..7511d947615a6 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Catalogs.java @@ -0,0 +1,124 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2; + +import org.apache.spark.SparkException; +import org.apache.spark.annotation.Private; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.apache.spark.util.Utils; + +import java.lang.reflect.InvocationTargetException; +import java.util.HashMap; +import java.util.Map; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static scala.collection.JavaConverters.mapAsJavaMapConverter; + +@Private +public class Catalogs { + private Catalogs() { + } + + /** + * Load and configure a catalog by name. + *

    + * This loads, instantiates, and initializes the catalog plugin for each call; it does not cache + * or reuse instances. + * + * @param name a String catalog name + * @param conf a SQLConf + * @return an initialized CatalogPlugin + * @throws CatalogNotFoundException if the plugin class cannot be found + * @throws SparkException if the plugin class cannot be instantiated + */ + public static CatalogPlugin load(String name, SQLConf conf) + throws CatalogNotFoundException, SparkException { + String pluginClassName = conf.getConfString("spark.sql.catalog." + name, null); + if (pluginClassName == null) { + throw new CatalogNotFoundException(String.format( + "Catalog '%s' plugin class not found: spark.sql.catalog.%s is not defined", name, name)); + } + + ClassLoader loader = Utils.getContextOrSparkClassLoader(); + + try { + Class pluginClass = loader.loadClass(pluginClassName); + + if (!CatalogPlugin.class.isAssignableFrom(pluginClass)) { + throw new SparkException(String.format( + "Plugin class for catalog '%s' does not implement CatalogPlugin: %s", + name, pluginClassName)); + } + + CatalogPlugin plugin = + CatalogPlugin.class.cast(pluginClass.getDeclaredConstructor().newInstance()); + + plugin.initialize(name, catalogOptions(name, conf)); + + return plugin; + + } catch (ClassNotFoundException e) { + throw new SparkException(String.format( + "Cannot find catalog plugin class for catalog '%s': %s", name, pluginClassName)); + + } catch (NoSuchMethodException e) { + throw new SparkException(String.format( + "Failed to find public no-arg constructor for catalog '%s': %s", name, pluginClassName), + e); + + } catch (IllegalAccessException e) { + throw new SparkException(String.format( + "Failed to call public no-arg constructor for catalog '%s': %s", name, pluginClassName), + e); + + } catch (InstantiationException e) { + throw new SparkException(String.format( + "Cannot instantiate abstract catalog plugin class for catalog '%s': %s", name, + pluginClassName), e.getCause()); + + } catch (InvocationTargetException e) { + throw new SparkException(String.format( + "Failed during instantiating constructor for catalog '%s': %s", name, pluginClassName), + e.getCause()); + } + } + + /** + * Extracts a named catalog's configuration from a SQLConf. + * + * @param name a catalog name + * @param conf a SQLConf + * @return a case insensitive string map of options starting with spark.sql.catalog.(name). + */ + private static CaseInsensitiveStringMap catalogOptions(String name, SQLConf conf) { + Map allConfs = mapAsJavaMapConverter(conf.getAllConfs()).asJava(); + Pattern prefix = Pattern.compile("^spark\\.sql\\.catalog\\." + name + "\\.(.+)"); + + HashMap options = new HashMap<>(); + for (Map.Entry entry : allConfs.entrySet()) { + Matcher matcher = prefix.matcher(entry.getKey()); + if (matcher.matches() && matcher.groupCount() > 0) { + options.put(matcher.group(1), entry.getValue()); + } + } + + return new CaseInsensitiveStringMap(options); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java new file mode 100644 index 0000000000000..3e697c1945bfc --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/Identifier.java @@ -0,0 +1,41 @@ +/* + * 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.catalog.v2; + +import org.apache.spark.annotation.Experimental; + +/** + * Identifies an object in a catalog. + */ +@Experimental +public interface Identifier { + + static Identifier of(String[] namespace, String name) { + return new IdentifierImpl(namespace, name); + } + + /** + * @return the namespace in the catalog + */ + String[] namespace(); + + /** + * @return the object name + */ + String name(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java new file mode 100644 index 0000000000000..34f3882c9c412 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/IdentifierImpl.java @@ -0,0 +1,87 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2; + +import com.google.common.base.Preconditions; +import org.apache.spark.annotation.Experimental; + +import java.util.Arrays; +import java.util.Objects; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * An {@link Identifier} implementation. + */ +@Experimental +class IdentifierImpl implements Identifier { + + private String[] namespace; + private String name; + + IdentifierImpl(String[] namespace, String name) { + Preconditions.checkNotNull(namespace, "Identifier namespace cannot be null"); + Preconditions.checkNotNull(name, "Identifier name cannot be null"); + this.namespace = namespace; + this.name = name; + } + + @Override + public String[] namespace() { + return namespace; + } + + @Override + public String name() { + return name; + } + + private String escapeQuote(String part) { + if (part.contains("`")) { + return part.replace("`", "``"); + } else { + return part; + } + } + + @Override + public String toString() { + return Stream.concat(Stream.of(namespace), Stream.of(name)) + .map(part -> '`' + escapeQuote(part) + '`') + .collect(Collectors.joining(".")); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + + if (o == null || getClass() != o.getClass()) { + return false; + } + + IdentifierImpl that = (IdentifierImpl) o; + return Arrays.equals(namespace, that.namespace) && name.equals(that.name); + } + + @Override + public int hashCode() { + return Objects.hash(Arrays.hashCode(namespace), name); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java new file mode 100644 index 0000000000000..681629d2d5405 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableCatalog.java @@ -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.sql.catalog.v2; + +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.catalyst.analysis.NoSuchNamespaceException; +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException; +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.types.StructType; + +import java.util.Map; + +/** + * Catalog methods for working with Tables. + *

    + * TableCatalog implementations may be case sensitive or case insensitive. Spark will pass + * {@link Identifier table identifiers} without modification. Field names passed to + * {@link #alterTable(Identifier, TableChange...)} will be normalized to match the case used in the + * table schema when updating, renaming, or dropping existing columns when catalyst analysis is case + * insensitive. + */ +public interface TableCatalog extends CatalogPlugin { + /** + * List the tables in a namespace from the catalog. + *

    + * If the catalog supports views, this must return identifiers for only tables and not views. + * + * @param namespace a multi-part namespace + * @return an array of Identifiers for tables + * @throws NoSuchNamespaceException If the namespace does not exist (optional). + */ + Identifier[] listTables(String[] namespace) throws NoSuchNamespaceException; + + /** + * Load table metadata by {@link Identifier identifier} from the catalog. + *

    + * If the catalog supports views and contains a view for the identifier and not a table, this + * must throw {@link NoSuchTableException}. + * + * @param ident a table identifier + * @return the table's metadata + * @throws NoSuchTableException If the table doesn't exist or is a view + */ + Table loadTable(Identifier ident) throws NoSuchTableException; + + /** + * Invalidate cached table metadata for an {@link Identifier identifier}. + *

    + * If the table is already loaded or cached, drop cached data. If the table does not exist or is + * not cached, do nothing. Calling this method should not query remote services. + * + * @param ident a table identifier + */ + default void invalidateTable(Identifier ident) { + } + + /** + * Test whether a table exists using an {@link Identifier identifier} from the catalog. + *

    + * If the catalog supports views and contains a view for the identifier and not a table, this + * must return false. + * + * @param ident a table identifier + * @return true if the table exists, false otherwise + */ + default boolean tableExists(Identifier ident) { + try { + return loadTable(ident) != null; + } catch (NoSuchTableException e) { + return false; + } + } + + /** + * Create a table in the catalog. + * + * @param ident a table identifier + * @param schema the schema of the new table, as a struct type + * @param partitions transforms to use for partitioning data in the table + * @param properties a string map of table properties + * @return metadata for the new table + * @throws TableAlreadyExistsException If a table or view already exists for the identifier + * @throws UnsupportedOperationException If a requested partition transform is not supported + * @throws NoSuchNamespaceException If the identifier namespace does not exist (optional) + */ + Table createTable( + Identifier ident, + StructType schema, + Transform[] partitions, + Map properties) throws TableAlreadyExistsException, NoSuchNamespaceException; + + /** + * Apply a set of {@link TableChange changes} to a table in the catalog. + *

    + * Implementations may reject the requested changes. If any change is rejected, none of the + * changes should be applied to the table. + *

    + * If the catalog supports views and contains a view for the identifier and not a table, this + * must throw {@link NoSuchTableException}. + * + * @param ident a table identifier + * @param changes changes to apply to the table + * @return updated metadata for the table + * @throws NoSuchTableException If the table doesn't exist or is a view + * @throws IllegalArgumentException If any change is rejected by the implementation. + */ + Table alterTable( + Identifier ident, + TableChange... changes) throws NoSuchTableException; + + /** + * Drop a table in the catalog. + *

    + * If the catalog supports views and contains a view for the identifier and not a table, this + * must not drop the view and must return false. + * + * @param ident a table identifier + * @return true if a table was deleted, false if no table exists for the identifier + */ + boolean dropTable(Identifier ident); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java new file mode 100644 index 0000000000000..7eef6aea88120 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/TableChange.java @@ -0,0 +1,375 @@ +/* + * 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.catalog.v2; + +import org.apache.spark.sql.types.DataType; + +/** + * TableChange subclasses represent requested changes to a table. These are passed to + * {@link TableCatalog#alterTable}. For example, + *

    + *   import TableChange._
    + *   val catalog = Catalogs.load(name)
    + *   catalog.asTableCatalog.alterTable(ident,
    + *       addColumn("x", IntegerType),
    + *       renameColumn("a", "b"),
    + *       deleteColumn("c")
    + *     )
    + * 
    + */ +public interface TableChange { + + /** + * Create a TableChange for setting a table property. + *

    + * If the property already exists, it will be replaced with the new value. + * + * @param property the property name + * @param value the new property value + * @return a TableChange for the addition + */ + static TableChange setProperty(String property, String value) { + return new SetProperty(property, value); + } + + /** + * Create a TableChange for removing a table property. + *

    + * If the property does not exist, the change will succeed. + * + * @param property the property name + * @return a TableChange for the addition + */ + static TableChange removeProperty(String property) { + return new RemoveProperty(property); + } + + /** + * Create a TableChange for adding an optional column. + *

    + * If the field already exists, the change will result in an {@link IllegalArgumentException}. + * If the new field is nested and its parent does not exist or is not a struct, the change will + * result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the new column + * @param dataType the new column's data type + * @return a TableChange for the addition + */ + static TableChange addColumn(String[] fieldNames, DataType dataType) { + return new AddColumn(fieldNames, dataType, true, null); + } + + /** + * Create a TableChange for adding a column. + *

    + * If the field already exists, the change will result in an {@link IllegalArgumentException}. + * If the new field is nested and its parent does not exist or is not a struct, the change will + * result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the new column + * @param dataType the new column's data type + * @param isNullable whether the new column can contain null + * @return a TableChange for the addition + */ + static TableChange addColumn(String[] fieldNames, DataType dataType, boolean isNullable) { + return new AddColumn(fieldNames, dataType, isNullable, null); + } + + /** + * Create a TableChange for adding a column. + *

    + * If the field already exists, the change will result in an {@link IllegalArgumentException}. + * If the new field is nested and its parent does not exist or is not a struct, the change will + * result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the new column + * @param dataType the new column's data type + * @param isNullable whether the new column can contain null + * @param comment the new field's comment string + * @return a TableChange for the addition + */ + static TableChange addColumn( + String[] fieldNames, + DataType dataType, + boolean isNullable, + String comment) { + return new AddColumn(fieldNames, dataType, isNullable, comment); + } + + /** + * Create a TableChange for renaming a field. + *

    + * The name is used to find the field to rename. The new name will replace the leaf field name. + * For example, renameColumn(["a", "b", "c"], "x") should produce column a.b.x. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames the current field names + * @param newName the new name + * @return a TableChange for the rename + */ + static TableChange renameColumn(String[] fieldNames, String newName) { + return new RenameColumn(fieldNames, newName); + } + + /** + * Create a TableChange for updating the type of a field that is nullable. + *

    + * The field names are used to find the field to update. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the column to update + * @param newDataType the new data type + * @return a TableChange for the update + */ + static TableChange updateColumnType(String[] fieldNames, DataType newDataType) { + return new UpdateColumnType(fieldNames, newDataType, true); + } + + /** + * Create a TableChange for updating the type of a field. + *

    + * The field names are used to find the field to update. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the column to update + * @param newDataType the new data type + * @return a TableChange for the update + */ + static TableChange updateColumnType( + String[] fieldNames, + DataType newDataType, + boolean isNullable) { + return new UpdateColumnType(fieldNames, newDataType, isNullable); + } + + /** + * Create a TableChange for updating the comment of a field. + *

    + * The name is used to find the field to update. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the column to update + * @param newComment the new comment + * @return a TableChange for the update + */ + static TableChange updateColumnComment(String[] fieldNames, String newComment) { + return new UpdateColumnComment(fieldNames, newComment); + } + + /** + * Create a TableChange for deleting a field. + *

    + * If the field does not exist, the change will result in an {@link IllegalArgumentException}. + * + * @param fieldNames field names of the column to delete + * @return a TableChange for the delete + */ + static TableChange deleteColumn(String[] fieldNames) { + return new DeleteColumn(fieldNames); + } + + /** + * A TableChange to set a table property. + *

    + * If the property already exists, it must be replaced with the new value. + */ + final class SetProperty implements TableChange { + private final String property; + private final String value; + + private SetProperty(String property, String value) { + this.property = property; + this.value = value; + } + + public String property() { + return property; + } + + public String value() { + return value; + } + } + + /** + * A TableChange to remove a table property. + *

    + * If the property does not exist, the change should succeed. + */ + final class RemoveProperty implements TableChange { + private final String property; + + private RemoveProperty(String property) { + this.property = property; + } + + public String property() { + return property; + } + } + + interface ColumnChange extends TableChange { + String[] fieldNames(); + } + + /** + * A TableChange to add a field. + *

    + * If the field already exists, the change must result in an {@link IllegalArgumentException}. + * If the new field is nested and its parent does not exist or is not a struct, the change must + * result in an {@link IllegalArgumentException}. + */ + final class AddColumn implements ColumnChange { + private final String[] fieldNames; + private final DataType dataType; + private final boolean isNullable; + private final String comment; + + private AddColumn(String[] fieldNames, DataType dataType, boolean isNullable, String comment) { + this.fieldNames = fieldNames; + this.dataType = dataType; + this.isNullable = isNullable; + this.comment = comment; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + + public DataType dataType() { + return dataType; + } + + public boolean isNullable() { + return isNullable; + } + + public String comment() { + return comment; + } + } + + /** + * A TableChange to rename a field. + *

    + * The name is used to find the field to rename. The new name will replace the leaf field name. + * For example, renameColumn("a.b.c", "x") should produce column a.b.x. + *

    + * If the field does not exist, the change must result in an {@link IllegalArgumentException}. + */ + final class RenameColumn implements ColumnChange { + private final String[] fieldNames; + private final String newName; + + private RenameColumn(String[] fieldNames, String newName) { + this.fieldNames = fieldNames; + this.newName = newName; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + + public String newName() { + return newName; + } + } + + /** + * A TableChange to update the type of a field. + *

    + * The field names are used to find the field to update. + *

    + * If the field does not exist, the change must result in an {@link IllegalArgumentException}. + */ + final class UpdateColumnType implements ColumnChange { + private final String[] fieldNames; + private final DataType newDataType; + private final boolean isNullable; + + private UpdateColumnType(String[] fieldNames, DataType newDataType, boolean isNullable) { + this.fieldNames = fieldNames; + this.newDataType = newDataType; + this.isNullable = isNullable; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + + public DataType newDataType() { + return newDataType; + } + + public boolean isNullable() { + return isNullable; + } + } + + /** + * A TableChange to update the comment of a field. + *

    + * The field names are used to find the field to update. + *

    + * If the field does not exist, the change must result in an {@link IllegalArgumentException}. + */ + final class UpdateColumnComment implements ColumnChange { + private final String[] fieldNames; + private final String newComment; + + private UpdateColumnComment(String[] fieldNames, String newComment) { + this.fieldNames = fieldNames; + this.newComment = newComment; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + + public String newComment() { + return newComment; + } + } + + /** + * A TableChange to delete a field. + *

    + * If the field does not exist, the change must result in an {@link IllegalArgumentException}. + */ + final class DeleteColumn implements ColumnChange { + private final String[] fieldNames; + + private DeleteColumn(String[] fieldNames) { + this.fieldNames = fieldNames; + } + + @Override + public String[] fieldNames() { + return fieldNames; + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java similarity index 73% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java index ac96c2765368f..1e2aca9556df4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSink.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expression.java @@ -15,13 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.streaming; +package org.apache.spark.sql.catalog.v2.expressions; + +import org.apache.spark.annotation.Experimental; /** - * The shared interface between V1 and V2 streaming sinks. - * - * This is a temporary interface for compatibility during migration. It should not be implemented - * directly, and will be removed in future versions. + * Base class of the public logical expression API. */ -public interface BaseStreamingSink { +@Experimental +public interface Expression { + /** + * Format the expression as a human readable SQL-like string. + */ + String describe(); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java new file mode 100644 index 0000000000000..7b264e7480e17 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Expressions.java @@ -0,0 +1,162 @@ +/* + * 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.catalog.v2.expressions; + +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.types.DataType; +import scala.collection.JavaConverters; + +import java.util.Arrays; + +/** + * Helper methods to create logical transforms to pass into Spark. + */ +@Experimental +public class Expressions { + private Expressions() { + } + + /** + * Create a logical transform for applying a named transform. + *

    + * This transform can represent applying any named transform. + * + * @param name the transform name + * @param args expression arguments to the transform + * @return a logical transform + */ + public static Transform apply(String name, Expression... args) { + return LogicalExpressions.apply(name, + JavaConverters.asScalaBuffer(Arrays.asList(args)).toSeq()); + } + + /** + * Create a named reference expression for a column. + * + * @param name a column name + * @return a named reference for the column + */ + public static NamedReference column(String name) { + return LogicalExpressions.reference(name); + } + + /** + * Create a literal from a value. + *

    + * The JVM type of the value held by a literal must be the type used by Spark's InternalRow API + * for the literal's {@link DataType SQL data type}. + * + * @param value a value + * @param the JVM type of the value + * @return a literal expression for the value + */ + public static Literal literal(T value) { + return LogicalExpressions.literal(value); + } + + /** + * Create a bucket transform for one or more columns. + *

    + * This transform represents a logical mapping from a value to a bucket id in [0, numBuckets) + * based on a hash of the value. + *

    + * The name reported by transforms created with this method is "bucket". + * + * @param numBuckets the number of output buckets + * @param columns input columns for the bucket transform + * @return a logical bucket transform with name "bucket" + */ + public static Transform bucket(int numBuckets, String... columns) { + return LogicalExpressions.bucket(numBuckets, + JavaConverters.asScalaBuffer(Arrays.asList(columns)).toSeq()); + } + + /** + * Create an identity transform for a column. + *

    + * This transform represents a logical mapping from a value to itself. + *

    + * The name reported by transforms created with this method is "identity". + * + * @param column an input column + * @return a logical identity transform with name "identity" + */ + public static Transform identity(String column) { + return LogicalExpressions.identity(column); + } + + /** + * Create a yearly transform for a timestamp or date column. + *

    + * This transform represents a logical mapping from a timestamp or date to a year, such as 2018. + *

    + * The name reported by transforms created with this method is "years". + * + * @param column an input timestamp or date column + * @return a logical yearly transform with name "years" + */ + public static Transform years(String column) { + return LogicalExpressions.years(column); + } + + /** + * Create a monthly transform for a timestamp or date column. + *

    + * This transform represents a logical mapping from a timestamp or date to a month, such as + * 2018-05. + *

    + * The name reported by transforms created with this method is "months". + * + * @param column an input timestamp or date column + * @return a logical monthly transform with name "months" + */ + public static Transform months(String column) { + return LogicalExpressions.months(column); + } + + /** + * Create a daily transform for a timestamp or date column. + *

    + * This transform represents a logical mapping from a timestamp or date to a date, such as + * 2018-05-13. + *

    + * The name reported by transforms created with this method is "days". + * + * @param column an input timestamp or date column + * @return a logical daily transform with name "days" + */ + public static Transform days(String column) { + return LogicalExpressions.days(column); + } + + /** + * Create an hourly transform for a timestamp column. + *

    + * This transform represents a logical mapping from a timestamp to a date and hour, such as + * 2018-05-13, hour 19. + *

    + * The name reported by transforms created with this method is "hours". + * + * @param column an input timestamp column + * @return a logical hourly transform with name "hours" + */ + public static Transform hours(String column) { + return LogicalExpressions.hours(column); + } + +} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java similarity index 56% rename from resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala rename to sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java index bffa0e09befd2..e41bcf9000c52 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationId.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Literal.java @@ -15,28 +15,28 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package org.apache.spark.sql.catalog.v2.expressions; -import org.apache.hadoop.yarn.api.records.ApplicationId +import org.apache.spark.annotation.Experimental; +import org.apache.spark.sql.types.DataType; /** - * Simple Testing Application Id; ID and cluster timestamp are set in constructor - * and cannot be updated. - * @param id app id - * @param clusterTimestamp timestamp + * Represents a constant literal value in the public expression API. + *

    + * The JVM type of the value held by a literal must be the type used by Spark's InternalRow API for + * the literal's {@link DataType SQL data type}. + * + * @param the JVM type of a value held by the literal */ -private[spark] class StubApplicationId(id: Int, clusterTimestamp: Long) extends ApplicationId { - override def getId: Int = { - id - } - - override def getClusterTimestamp: Long = { - clusterTimestamp - } - - override def setId(id: Int): Unit = {} - - override def setClusterTimestamp(clusterTimestamp: Long): Unit = {} +@Experimental +public interface Literal extends Expression { + /** + * Returns the literal value. + */ + T value(); - override def build(): Unit = {} + /** + * Returns the SQL data type of the literal. + */ + DataType dataType(); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java new file mode 100644 index 0000000000000..c71ffbe70651f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/NamedReference.java @@ -0,0 +1,33 @@ +/* + * 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.catalog.v2.expressions; + +import org.apache.spark.annotation.Experimental; + +/** + * Represents a field or column reference in the public logical expression API. + */ +@Experimental +public interface NamedReference extends Expression { + /** + * Returns the referenced field name as an array of String parts. + *

    + * Each string in the returned array represents a field name. + */ + String[] fieldNames(); +} diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java similarity index 53% rename from external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala rename to sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java index 02917becf0ff9..c85e0c412f1ab 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/KafkaRDDPartition.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/expressions/Transform.java @@ -15,28 +15,30 @@ * limitations under the License. */ -package org.apache.spark.streaming.kafka +package org.apache.spark.sql.catalog.v2.expressions; -import org.apache.spark.Partition +import org.apache.spark.annotation.Experimental; /** - * @param topic kafka topic name - * @param partition kafka partition id - * @param fromOffset inclusive starting offset - * @param untilOffset exclusive ending offset - * @param host preferred kafka host, i.e. the leader at the time the rdd was created - * @param port preferred kafka host's port + * Represents a transform function in the public logical expression API. + *

    + * For example, the transform date(ts) is used to derive a date value from a timestamp column. The + * transform name is "date" and its argument is a reference to the "ts" column. */ -private[kafka] -class KafkaRDDPartition( - val index: Int, - val topic: String, - val partition: Int, - val fromOffset: Long, - val untilOffset: Long, - val host: String, - val port: Int -) extends Partition { - /** Number of messages this partition refers to */ - def count(): Long = untilOffset - fromOffset +@Experimental +public interface Transform extends Expression { + /** + * Returns the transform function name. + */ + String name(); + + /** + * Returns all field references in the transform arguments. + */ + NamedReference[] references(); + + /** + * Returns the arguments passed to the transform function. + */ + Expression[] arguments(); } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala new file mode 100644 index 0000000000000..7cc80c41a9013 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalog/v2/utils/CatalogV2Util.scala @@ -0,0 +1,192 @@ +/* + * 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.catalog.v2.utils + +import java.util +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, TableChange} +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RemoveProperty, RenameColumn, SetProperty, UpdateColumnComment, UpdateColumnType} +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.{ArrayType, MapType, StructField, StructType} + +object CatalogV2Util { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + /** + * Apply properties changes to a map and return the result. + */ + def applyPropertiesChanges( + properties: Map[String, String], + changes: Seq[TableChange]): Map[String, String] = { + applyPropertiesChanges(properties.asJava, changes).asScala.toMap + } + + /** + * Apply properties changes to a Java map and return the result. + */ + def applyPropertiesChanges( + properties: util.Map[String, String], + changes: Seq[TableChange]): util.Map[String, String] = { + val newProperties = new util.HashMap[String, String](properties) + + changes.foreach { + case set: SetProperty => + newProperties.put(set.property, set.value) + + case unset: RemoveProperty => + newProperties.remove(unset.property) + + case _ => + // ignore non-property changes + } + + Collections.unmodifiableMap(newProperties) + } + + /** + * Apply schema changes to a schema and return the result. + */ + def applySchemaChanges(schema: StructType, changes: Seq[TableChange]): StructType = { + changes.foldLeft(schema) { (schema, change) => + change match { + case add: AddColumn => + add.fieldNames match { + case Array(name) => + val newField = StructField(name, add.dataType, nullable = add.isNullable) + Option(add.comment) match { + case Some(comment) => + schema.add(newField.withComment(comment)) + case _ => + schema.add(newField) + } + + case names => + replace(schema, names.init, parent => parent.dataType match { + case parentType: StructType => + val field = StructField(names.last, add.dataType, nullable = add.isNullable) + val newParentType = Option(add.comment) match { + case Some(comment) => + parentType.add(field.withComment(comment)) + case None => + parentType.add(field) + } + + Some(StructField(parent.name, newParentType, parent.nullable, parent.metadata)) + + case _ => + throw new IllegalArgumentException(s"Not a struct: ${names.init.last}") + }) + } + + case rename: RenameColumn => + replace(schema, rename.fieldNames, field => + Some(StructField(rename.newName, field.dataType, field.nullable, field.metadata))) + + case update: UpdateColumnType => + replace(schema, update.fieldNames, field => { + if (!update.isNullable && field.nullable) { + throw new IllegalArgumentException( + s"Cannot change optional column to required: $field.name") + } + Some(StructField(field.name, update.newDataType, update.isNullable, field.metadata)) + }) + + case update: UpdateColumnComment => + replace(schema, update.fieldNames, field => + Some(field.withComment(update.newComment))) + + case delete: DeleteColumn => + replace(schema, delete.fieldNames, _ => None) + + case _ => + // ignore non-schema changes + schema + } + } + } + + private def replace( + struct: StructType, + fieldNames: Seq[String], + update: StructField => Option[StructField]): StructType = { + + val pos = struct.getFieldIndex(fieldNames.head) + .getOrElse(throw new IllegalArgumentException(s"Cannot find field: ${fieldNames.head}")) + val field = struct.fields(pos) + val replacement: Option[StructField] = (fieldNames.tail, field.dataType) match { + case (Seq(), _) => + update(field) + + case (names, struct: StructType) => + val updatedType: StructType = replace(struct, names, update) + Some(StructField(field.name, updatedType, field.nullable, field.metadata)) + + case (Seq("key"), map @ MapType(keyType, _, _)) => + val updated = update(StructField("key", keyType, nullable = false)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map key")) + Some(field.copy(dataType = map.copy(keyType = updated.dataType))) + + case (Seq("key", names @ _*), map @ MapType(keyStruct: StructType, _, _)) => + Some(field.copy(dataType = map.copy(keyType = replace(keyStruct, names, update)))) + + case (Seq("value"), map @ MapType(_, mapValueType, isNullable)) => + val updated = update(StructField("value", mapValueType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete map value")) + Some(field.copy(dataType = map.copy( + valueType = updated.dataType, + valueContainsNull = updated.nullable))) + + case (Seq("value", names @ _*), map @ MapType(_, valueStruct: StructType, _)) => + Some(field.copy(dataType = map.copy(valueType = replace(valueStruct, names, update)))) + + case (Seq("element"), array @ ArrayType(elementType, isNullable)) => + val updated = update(StructField("element", elementType, nullable = isNullable)) + .getOrElse(throw new IllegalArgumentException(s"Cannot delete array element")) + Some(field.copy(dataType = array.copy( + elementType = updated.dataType, + containsNull = updated.nullable))) + + case (Seq("element", names @ _*), array @ ArrayType(elementStruct: StructType, _)) => + Some(field.copy(dataType = array.copy(elementType = replace(elementStruct, names, update)))) + + case (names, dataType) => + throw new IllegalArgumentException( + s"Cannot find field: ${names.head} in ${dataType.simpleString}") + } + + val newFields = struct.fields.zipWithIndex.flatMap { + case (_, index) if pos == index => + replacement + case (other, _) => + Some(other) + } + + new StructType(newFields) + } + + def loadTable(catalog: CatalogPlugin, ident: Identifier): Option[Table] = + try { + Option(catalog.asTableCatalog.loadTable(ident)) + } catch { + case _: NoSuchTableException => None + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java index ea6fffaebc9a1..acdf6afe10ce5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionDescription.java @@ -31,43 +31,73 @@ * `usage()` will be used for the function usage in brief way. * * These below are concatenated and used for the function usage in verbose way, suppose arguments, - * examples, note and since will be provided. + * examples, note, since and deprecated will be provided. * - * `arguments()` describes arguments for the expression. This should follow the format as below: + * `arguments()` describes arguments for the expression. * - * Arguments: - * * arg0 - ... - * .... - * * arg1 - ... - * .... - * - * `examples()` describes examples for the expression. This should follow the format as below: - * - * Examples: - * > SELECT ...; - * ... - * > SELECT ...; - * ... + * `examples()` describes examples for the expression. * * `note()` contains some notes for the expression optionally. * * `since()` contains version information for the expression. Version is specified by, * for example, "2.2.0". * - * We can refer the function name by `_FUNC_`, in `usage`, `arguments` and `examples`, as it's - * registered in `FunctionRegistry`. + * `deprecated()` contains deprecation information for the expression optionally, for example, + * "Deprecated since 2.2.0. Use something else instead". + * + * The format, in particular for `arguments()`, `examples()`,`note()`, `since()` and + * `deprecated()`, should strictly be as follows. + * + *

    + * @ExpressionDescription(
    + *   ...
    + *   arguments = """
    + *     Arguments:
    + *       * arg0 - ...
    + *           ....
    + *       * arg1 - ...
    + *           ....
    + *   """,
    + *   examples = """
    + *     Examples:
    + *       > SELECT ...;
    + *        ...
    + *       > SELECT ...;
    + *        ...
    + *   """,
    + *   note = """
    + *     ...
    + *   """,
    + *   since = "3.0.0",
    + *   deprecated = """
    + *     ...
    + *   """)
    + * 
    + * 
    + * + * We can refer the function name by `_FUNC_`, in `usage()`, `arguments()` and `examples()` as + * it is registered in `FunctionRegistry`. + * + * Note that, if `extended()` is defined, `arguments()`, `examples()`, `note()`, `since()` and + * `deprecated()` should be not defined together. `extended()` exists for backward compatibility. * - * Note that, if `extended()` is defined, `arguments()`, `examples()`, `note()` and `since()` will - * be ignored and `extended()` will be used for the extended description for backward - * compatibility. + * Note this contents are used in the SparkSQL documentation for built-in functions. The contents + * here are considered as a Markdown text and then rendered. */ @DeveloperApi @Retention(RetentionPolicy.RUNTIME) public @interface ExpressionDescription { String usage() default ""; + /** + * @deprecated This field is deprecated as of Spark 3.0. Use {@link #arguments}, + * {@link #examples}, {@link #note}, {@link #since} and {@link #deprecated} instead + * to document the extended usage. + */ + @Deprecated String extended() default ""; String arguments() default ""; String examples() default ""; String note() default ""; String since() default ""; + String deprecated() default ""; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java index ab13ac9cc5483..0d1f6c2b4d5b3 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/ExpressionInfo.java @@ -30,6 +30,7 @@ public class ExpressionInfo { private String examples; private String note; private String since; + private String deprecated; public String getClassName() { return className; @@ -63,6 +64,10 @@ public String getNote() { return note; } + public String getDeprecated() { + return deprecated; + } + public String getDb() { return db; } @@ -75,13 +80,15 @@ public ExpressionInfo( String arguments, String examples, String note, - String since) { + String since, + String deprecated) { assert name != null; assert arguments != null; assert examples != null; - assert examples.isEmpty() || examples.startsWith("\n Examples:"); + assert examples.isEmpty() || examples.contains(" Examples:"); assert note != null; assert since != null; + assert deprecated != null; this.className = className; this.db = db; @@ -91,6 +98,7 @@ public ExpressionInfo( this.examples = examples; this.note = note; this.since = since; + this.deprecated = deprecated; // Make the extended description. this.extended = arguments + examples; @@ -98,25 +106,48 @@ public ExpressionInfo( this.extended = "\n No example/argument for _FUNC_.\n"; } if (!note.isEmpty()) { + if (!note.contains(" ") || !note.endsWith(" ")) { + throw new IllegalArgumentException("'note' is malformed in the expression [" + + this.name + "]. It should start with a newline and 4 leading spaces; end " + + "with a newline and two spaces; however, got [" + note + "]."); + } this.extended += "\n Note:\n " + note.trim() + "\n"; } if (!since.isEmpty()) { + if (Integer.parseInt(since.split("\\.")[0]) < 0) { + throw new IllegalArgumentException("'since' is malformed in the expression [" + + this.name + "]. It should not start with a negative number; however, " + + "got [" + since + "]."); + } this.extended += "\n Since: " + since + "\n"; } + if (!deprecated.isEmpty()) { + if (!deprecated.contains(" ") || !deprecated.endsWith(" ")) { + throw new IllegalArgumentException("'deprecated' is malformed in the " + + "expression [" + this.name + "]. It should start with a newline and 4 " + + "leading spaces; end with a newline and two spaces; however, got [" + + deprecated + "]."); + } + this.extended += "\n Deprecated:\n " + deprecated.trim() + "\n"; + } } public ExpressionInfo(String className, String name) { - this(className, null, name, null, "", "", "", ""); + this(className, null, name, null, "", "", "", "", ""); } public ExpressionInfo(String className, String db, String name) { - this(className, db, name, null, "", "", "", ""); + this(className, db, name, null, "", "", "", "", ""); } - // This is to keep the original constructor just in case. + /** + * @deprecated This constructor is deprecated as of Spark 3.0. Use other constructors to fully + * specify each argument for extended usage. + */ + @Deprecated public ExpressionInfo(String className, String db, String name, String usage, String extended) { // `arguments` and `examples` are concatenated for the extended description. So, here // simply pass the `extended` as `arguments` and an empty string for `examples`. - this(className, db, name, usage, extended, "", "", ""); + this(className, db, name, usage, extended, "", "", "", ""); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java index 460513816dfd9..6344cf18c11b8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatch.java @@ -20,6 +20,7 @@ import java.io.IOException; import org.apache.spark.memory.MemoryConsumer; +import org.apache.spark.memory.SparkOutOfMemoryError; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.memory.MemoryBlock; @@ -126,7 +127,7 @@ public final void close() { private boolean acquirePage(long requiredSize) { try { page = allocatePage(requiredSize); - } catch (OutOfMemoryError e) { + } catch (SparkOutOfMemoryError e) { logger.warn("Failed to allocate page ({} bytes).", requiredSize); return false; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java new file mode 100644 index 0000000000000..ea0648a6cb909 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/SpecializedGettersReader.java @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions; + +import org.apache.spark.sql.types.*; + +public final class SpecializedGettersReader { + private SpecializedGettersReader() {} + + public static Object read( + SpecializedGetters obj, + int ordinal, + DataType dataType, + boolean handleNull, + boolean handleUserDefinedType) { + if (handleNull && (obj.isNullAt(ordinal) || dataType instanceof NullType)) { + return null; + } + if (dataType instanceof BooleanType) { + return obj.getBoolean(ordinal); + } + if (dataType instanceof ByteType) { + return obj.getByte(ordinal); + } + if (dataType instanceof ShortType) { + return obj.getShort(ordinal); + } + if (dataType instanceof IntegerType) { + return obj.getInt(ordinal); + } + if (dataType instanceof LongType) { + return obj.getLong(ordinal); + } + if (dataType instanceof FloatType) { + return obj.getFloat(ordinal); + } + if (dataType instanceof DoubleType) { + return obj.getDouble(ordinal); + } + if (dataType instanceof StringType) { + return obj.getUTF8String(ordinal); + } + if (dataType instanceof DecimalType) { + DecimalType dt = (DecimalType) dataType; + return obj.getDecimal(ordinal, dt.precision(), dt.scale()); + } + if (dataType instanceof DateType) { + return obj.getInt(ordinal); + } + if (dataType instanceof TimestampType) { + return obj.getLong(ordinal); + } + if (dataType instanceof CalendarIntervalType) { + return obj.getInterval(ordinal); + } + if (dataType instanceof BinaryType) { + return obj.getBinary(ordinal); + } + if (dataType instanceof StructType) { + return obj.getStruct(ordinal, ((StructType) dataType).size()); + } + if (dataType instanceof ArrayType) { + return obj.getArray(ordinal); + } + if (dataType instanceof MapType) { + return obj.getMap(ordinal); + } + if (handleUserDefinedType && dataType instanceof UserDefinedType) { + return obj.get(ordinal, ((UserDefinedType)dataType).sqlType()); + } + + throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString()); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java index 9002abdcfd474..db6401b18c0e4 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeArrayData.java @@ -17,10 +17,19 @@ package org.apache.spark.sql.catalyst.expressions; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + import org.apache.spark.sql.catalyst.util.ArrayData; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.Platform; @@ -30,6 +39,8 @@ import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; +import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET; + /** * An Unsafe implementation of Array which is backed by raw memory instead of Java objects. * @@ -52,8 +63,7 @@ * Instances of `UnsafeArrayData` act as pointers to row data stored in this format. */ -public final class UnsafeArrayData extends ArrayData { - +public final class UnsafeArrayData extends ArrayData implements Externalizable, KryoSerializable { public static int calculateHeaderPortionInBytes(int numFields) { return (int)calculateHeaderPortionInBytes((long)numFields); } @@ -137,46 +147,7 @@ public boolean isNullAt(int ordinal) { @Override public Object get(int ordinal, DataType dataType) { - if (isNullAt(ordinal) || dataType instanceof NullType) { - return null; - } else if (dataType instanceof BooleanType) { - return getBoolean(ordinal); - } else if (dataType instanceof ByteType) { - return getByte(ordinal); - } else if (dataType instanceof ShortType) { - return getShort(ordinal); - } else if (dataType instanceof IntegerType) { - return getInt(ordinal); - } else if (dataType instanceof LongType) { - return getLong(ordinal); - } else if (dataType instanceof FloatType) { - return getFloat(ordinal); - } else if (dataType instanceof DoubleType) { - return getDouble(ordinal); - } else if (dataType instanceof DecimalType) { - DecimalType dt = (DecimalType) dataType; - return getDecimal(ordinal, dt.precision(), dt.scale()); - } else if (dataType instanceof DateType) { - return getInt(ordinal); - } else if (dataType instanceof TimestampType) { - return getLong(ordinal); - } else if (dataType instanceof BinaryType) { - return getBinary(ordinal); - } else if (dataType instanceof StringType) { - return getUTF8String(ordinal); - } else if (dataType instanceof CalendarIntervalType) { - return getInterval(ordinal); - } else if (dataType instanceof StructType) { - return getStruct(ordinal, ((StructType) dataType).size()); - } else if (dataType instanceof ArrayType) { - return getArray(ordinal); - } else if (dataType instanceof MapType) { - return getMap(ordinal); - } else if (dataType instanceof UserDefinedType) { - return get(ordinal, ((UserDefinedType)dataType).sqlType()); - } else { - throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString()); - } + return SpecializedGettersReader.read(this, ordinal, dataType, true, true); } @Override @@ -300,6 +271,7 @@ public UnsafeMapData getMap(int ordinal) { @Override public void update(int ordinal, Object value) { throw new UnsupportedOperationException(); } + @Override public void setNullAt(int ordinal) { assertIndexIsValid(ordinal); BitSetMethods.set(baseObject, baseOffset + 8, ordinal); @@ -308,43 +280,44 @@ public void setNullAt(int ordinal) { will be set to 0 later by the caller side */ } + @Override public void setBoolean(int ordinal, boolean value) { assertIndexIsValid(ordinal); Platform.putBoolean(baseObject, getElementOffset(ordinal, 1), value); } + @Override public void setByte(int ordinal, byte value) { assertIndexIsValid(ordinal); Platform.putByte(baseObject, getElementOffset(ordinal, 1), value); } + @Override public void setShort(int ordinal, short value) { assertIndexIsValid(ordinal); Platform.putShort(baseObject, getElementOffset(ordinal, 2), value); } + @Override public void setInt(int ordinal, int value) { assertIndexIsValid(ordinal); Platform.putInt(baseObject, getElementOffset(ordinal, 4), value); } + @Override public void setLong(int ordinal, long value) { assertIndexIsValid(ordinal); Platform.putLong(baseObject, getElementOffset(ordinal, 8), value); } + @Override public void setFloat(int ordinal, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } assertIndexIsValid(ordinal); Platform.putFloat(baseObject, getElementOffset(ordinal, 4), value); } + @Override public void setDouble(int ordinal, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } assertIndexIsValid(ordinal); Platform.putDouble(baseObject, getElementOffset(ordinal, 8), value); } @@ -523,4 +496,40 @@ public static UnsafeArrayData fromPrimitiveArray(float[] arr) { public static UnsafeArrayData fromPrimitiveArray(double[] arr) { return fromPrimitiveArray(arr, Platform.DOUBLE_ARRAY_OFFSET, arr.length, 8); } + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + byte[] bytes = UnsafeDataUtils.getBytes(baseObject, baseOffset, sizeInBytes); + out.writeInt(bytes.length); + out.writeInt(this.numElements); + out.write(bytes); + } + + @Override + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + this.baseOffset = BYTE_ARRAY_OFFSET; + this.sizeInBytes = in.readInt(); + this.numElements = in.readInt(); + this.elementOffset = baseOffset + calculateHeaderPortionInBytes(this.numElements); + this.baseObject = new byte[sizeInBytes]; + in.readFully((byte[]) baseObject); + } + + @Override + public void write(Kryo kryo, Output output) { + byte[] bytes = UnsafeDataUtils.getBytes(baseObject, baseOffset, sizeInBytes); + output.writeInt(bytes.length); + output.writeInt(this.numElements); + output.write(bytes); + } + + @Override + public void read(Kryo kryo, Input input) { + this.baseOffset = BYTE_ARRAY_OFFSET; + this.sizeInBytes = input.readInt(); + this.numElements = input.readInt(); + this.elementOffset = baseOffset + calculateHeaderPortionInBytes(this.numElements); + this.baseObject = new byte[sizeInBytes]; + input.read((byte[]) baseObject); + } } diff --git a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeDataUtils.java similarity index 57% rename from core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala rename to sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeDataUtils.java index 1cd37185d6601..9b600192ac250 100644 --- a/core/src/main/scala/org/apache/spark/status/api/v1/SecurityFilter.scala +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeDataUtils.java @@ -14,23 +14,27 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.status.api.v1 +package org.apache.spark.sql.catalyst.expressions; -import javax.ws.rs.container.{ContainerRequestContext, ContainerRequestFilter} -import javax.ws.rs.core.Response -import javax.ws.rs.ext.Provider +import org.apache.spark.unsafe.Platform; -@Provider -private[v1] class SecurityFilter extends ContainerRequestFilter with ApiRequestContext { - override def filter(req: ContainerRequestContext): Unit = { - val user = httpRequest.getRemoteUser() - if (!uiRoot.securityManager.checkUIViewPermissions(user)) { - req.abortWith( - Response - .status(Response.Status.FORBIDDEN) - .entity(raw"""user "$user" is not authorized""") - .build() - ) +/** + * General utilities available for unsafe data + */ +final class UnsafeDataUtils { + + private UnsafeDataUtils() { + } + + public static byte[] getBytes(Object baseObject, long baseOffset, int sizeInBytes) { + if (baseObject instanceof byte[] + && baseOffset == Platform.BYTE_ARRAY_OFFSET + && (((byte[]) baseObject).length == sizeInBytes)) { + return (byte[]) baseObject; } + byte[] bytes = new byte[sizeInBytes]; + Platform.copyMemory(baseObject, baseOffset, bytes, Platform.BYTE_ARRAY_OFFSET, + sizeInBytes); + return bytes; } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java index f17441dfccb6d..60ac69b05994f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeMapData.java @@ -17,20 +17,34 @@ package org.apache.spark.sql.catalyst.expressions; +import java.io.Externalizable; +import java.io.IOException; +import java.io.ObjectInput; +import java.io.ObjectOutput; import java.nio.ByteBuffer; +import com.esotericsoftware.kryo.Kryo; +import com.esotericsoftware.kryo.KryoSerializable; +import com.esotericsoftware.kryo.io.Input; +import com.esotericsoftware.kryo.io.Output; + import org.apache.spark.sql.catalyst.util.MapData; import org.apache.spark.unsafe.Platform; +import static org.apache.spark.unsafe.Platform.BYTE_ARRAY_OFFSET; + /** * An Unsafe implementation of Map which is backed by raw memory instead of Java objects. * * Currently we just use 2 UnsafeArrayData to represent UnsafeMapData, with extra 8 bytes at head * to indicate the number of bytes of the unsafe key array. * [unsafe key array numBytes] [unsafe key array] [unsafe value array] + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. */ // TODO: Use a more efficient format which doesn't depend on unsafe array. -public final class UnsafeMapData extends MapData { +public final class UnsafeMapData extends MapData implements Externalizable, KryoSerializable { private Object baseObject; private long baseOffset; @@ -120,4 +134,36 @@ public UnsafeMapData copy() { mapCopy.pointTo(mapDataCopy, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); return mapCopy; } + + @Override + public void writeExternal(ObjectOutput out) throws IOException { + byte[] bytes = UnsafeDataUtils.getBytes(baseObject, baseOffset, sizeInBytes); + out.writeInt(bytes.length); + out.write(bytes); + } + + @Override + public void readExternal(ObjectInput in) throws IOException, ClassNotFoundException { + this.baseOffset = BYTE_ARRAY_OFFSET; + this.sizeInBytes = in.readInt(); + this.baseObject = new byte[sizeInBytes]; + in.readFully((byte[]) baseObject); + pointTo(baseObject, baseOffset, sizeInBytes); + } + + @Override + public void write(Kryo kryo, Output output) { + byte[] bytes = UnsafeDataUtils.getBytes(baseObject, baseOffset, sizeInBytes); + output.writeInt(bytes.length); + output.write(bytes); + } + + @Override + public void read(Kryo kryo, Input input) { + this.baseOffset = BYTE_ARRAY_OFFSET; + this.sizeInBytes = input.readInt(); + this.baseObject = new byte[sizeInBytes]; + input.read((byte[]) baseObject); + pointTo(baseObject, baseOffset, sizeInBytes); + } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java index a76e6ef8c91c1..8fd6029e976ee 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java @@ -224,9 +224,6 @@ public void setLong(int ordinal, long value) { public void setDouble(int ordinal, double value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); - if (Double.isNaN(value)) { - value = Double.NaN; - } Platform.putDouble(baseObject, getFieldOffset(ordinal), value); } @@ -255,9 +252,6 @@ public void setByte(int ordinal, byte value) { public void setFloat(int ordinal, float value) { assertIndexIsValid(ordinal); setNotNullAt(ordinal); - if (Float.isNaN(value)) { - value = Float.NaN; - } Platform.putFloat(baseObject, getFieldOffset(ordinal), value); } @@ -305,46 +299,7 @@ public void setDecimal(int ordinal, Decimal value, int precision) { @Override public Object get(int ordinal, DataType dataType) { - if (isNullAt(ordinal) || dataType instanceof NullType) { - return null; - } else if (dataType instanceof BooleanType) { - return getBoolean(ordinal); - } else if (dataType instanceof ByteType) { - return getByte(ordinal); - } else if (dataType instanceof ShortType) { - return getShort(ordinal); - } else if (dataType instanceof IntegerType) { - return getInt(ordinal); - } else if (dataType instanceof LongType) { - return getLong(ordinal); - } else if (dataType instanceof FloatType) { - return getFloat(ordinal); - } else if (dataType instanceof DoubleType) { - return getDouble(ordinal); - } else if (dataType instanceof DecimalType) { - DecimalType dt = (DecimalType) dataType; - return getDecimal(ordinal, dt.precision(), dt.scale()); - } else if (dataType instanceof DateType) { - return getInt(ordinal); - } else if (dataType instanceof TimestampType) { - return getLong(ordinal); - } else if (dataType instanceof BinaryType) { - return getBinary(ordinal); - } else if (dataType instanceof StringType) { - return getUTF8String(ordinal); - } else if (dataType instanceof CalendarIntervalType) { - return getInterval(ordinal); - } else if (dataType instanceof StructType) { - return getStruct(ordinal, ((StructType) dataType).size()); - } else if (dataType instanceof ArrayType) { - return getArray(ordinal); - } else if (dataType instanceof MapType) { - return getMap(ordinal); - } else if (dataType instanceof UserDefinedType) { - return get(ordinal, ((UserDefinedType)dataType).sqlType()); - } else { - throw new UnsupportedOperationException("Unsupported data type " + dataType.simpleString()); - } + return SpecializedGettersReader.read(this, ordinal, dataType, true, true); } @Override @@ -586,14 +541,7 @@ public boolean equals(Object other) { * Returns the underlying bytes for this UnsafeRow. */ public byte[] getBytes() { - if (baseObject instanceof byte[] && baseOffset == Platform.BYTE_ARRAY_OFFSET - && (((byte[]) baseObject).length == sizeInBytes)) { - return (byte[]) baseObject; - } else { - byte[] bytes = new byte[sizeInBytes]; - Platform.copyMemory(baseObject, baseOffset, bytes, Platform.BYTE_ARRAY_OFFSET, sizeInBytes); - return bytes; - } + return UnsafeDataUtils.getBytes(baseObject, baseOffset, sizeInBytes); } // This is for debugging diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java index a78dd970d23e4..bf6792313ae2d 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeArrayWriter.java @@ -74,7 +74,7 @@ public void initialize(int numElements) { } private long getElementOffset(int ordinal) { - return startingOffset + headerInBytes + ordinal * elementSize; + return startingOffset + headerInBytes + ordinal * (long) elementSize; } private void setNullBit(int ordinal) { @@ -82,24 +82,28 @@ private void setNullBit(int ordinal) { BitSetMethods.set(getBuffer(), startingOffset + 8, ordinal); } + @Override public void setNull1Bytes(int ordinal) { setNullBit(ordinal); // put zero into the corresponding field when set null writeByte(getElementOffset(ordinal), (byte)0); } + @Override public void setNull2Bytes(int ordinal) { setNullBit(ordinal); // put zero into the corresponding field when set null writeShort(getElementOffset(ordinal), (short)0); } + @Override public void setNull4Bytes(int ordinal) { setNullBit(ordinal); // put zero into the corresponding field when set null writeInt(getElementOffset(ordinal), 0); } + @Override public void setNull8Bytes(int ordinal) { setNullBit(ordinal); // put zero into the corresponding field when set null @@ -108,41 +112,49 @@ public void setNull8Bytes(int ordinal) { public void setNull(int ordinal) { setNull8Bytes(ordinal); } + @Override public void write(int ordinal, boolean value) { assertIndexIsValid(ordinal); writeBoolean(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, byte value) { assertIndexIsValid(ordinal); writeByte(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, short value) { assertIndexIsValid(ordinal); writeShort(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, int value) { assertIndexIsValid(ordinal); writeInt(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, long value) { assertIndexIsValid(ordinal); writeLong(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, float value) { assertIndexIsValid(ordinal); writeFloat(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, double value) { assertIndexIsValid(ordinal); writeDouble(getElementOffset(ordinal), value); } + @Override public void write(int ordinal, Decimal input, int precision, int scale) { // make sure Decimal object has the same scale as DecimalType assertIndexIsValid(ordinal); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java index 3960d6d520476..582882374e183 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriter.java @@ -132,47 +132,55 @@ public void setNull8Bytes(int ordinal) { } public long getFieldOffset(int ordinal) { - return startingOffset + nullBitsSize + 8 * ordinal; + return startingOffset + nullBitsSize + 8L * ordinal; } + @Override public void write(int ordinal, boolean value) { final long offset = getFieldOffset(ordinal); writeLong(offset, 0L); writeBoolean(offset, value); } + @Override public void write(int ordinal, byte value) { final long offset = getFieldOffset(ordinal); writeLong(offset, 0L); writeByte(offset, value); } + @Override public void write(int ordinal, short value) { final long offset = getFieldOffset(ordinal); writeLong(offset, 0L); writeShort(offset, value); } + @Override public void write(int ordinal, int value) { final long offset = getFieldOffset(ordinal); writeLong(offset, 0L); writeInt(offset, value); } + @Override public void write(int ordinal, long value) { writeLong(getFieldOffset(ordinal), value); } + @Override public void write(int ordinal, float value) { final long offset = getFieldOffset(ordinal); writeLong(offset, 0); writeFloat(offset, value); } + @Override public void write(int ordinal, double value) { writeDouble(getFieldOffset(ordinal), value); } + @Override public void write(int ordinal, Decimal input, int precision, int scale) { if (precision <= Decimal.MAX_LONG_DIGITS()) { // make sure Decimal object has the same scale as DecimalType diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java index 2781655002000..95263a0da95a8 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeWriter.java @@ -199,16 +199,10 @@ protected final void writeLong(long offset, long value) { } protected final void writeFloat(long offset, float value) { - if (Float.isNaN(value)) { - value = Float.NaN; - } Platform.putFloat(getBuffer(), offset, value); } protected final void writeDouble(long offset, double value) { - if (Double.isNaN(value)) { - value = Double.NaN; - } Platform.putDouble(getBuffer(), offset, value); } } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java index 023ec139652c5..e9f18229b54c2 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtil.java @@ -180,7 +180,7 @@ public long skip(long ns) throws IOException { return 0; } // Bound skip by beginning and end of the source - long n = Math.min(length - next, ns); + int n = (int) Math.min(length - next, ns); n = Math.max(-next, n); next += n; return n; diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java similarity index 84% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java index bbe430e299261..d27fbfdd14617 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SessionConfigSupport.java @@ -17,15 +17,15 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to + * A mix-in interface for {@link TableProvider}. Data sources can implement this interface to * propagate session configs with the specified key-prefix to all data source operations in this * session. */ -@InterfaceStability.Evolving -public interface SessionConfigSupport extends DataSourceV2 { +@Evolving +public interface SessionConfigSupport extends TableProvider { /** * Key prefix of the session configs to propagate, which is usually the data source name. Spark diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java new file mode 100644 index 0000000000000..826fa2f8a0720 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsRead.java @@ -0,0 +1,39 @@ +/* + * 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.sources.v2; + +import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * A mix-in interface of {@link Table}, to indicate that it's readable. This adds + * {@link #newScanBuilder(CaseInsensitiveStringMap)} that is used to create a scan for batch, + * micro-batch, or continuous processing. + */ +public interface SupportsRead extends Table { + + /** + * Returns a {@link ScanBuilder} which can be used to build a {@link Scan}. Spark will call this + * method to configure each data source scan. + * + * @param options The options for reading, which is an immutable case-insensitive + * string-to-string map. + */ + ScanBuilder newScanBuilder(CaseInsensitiveStringMap options); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java similarity index 54% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java index 6e31e84bf6c72..c52e54569dc0c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceV2.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/SupportsWrite.java @@ -17,18 +17,20 @@ package org.apache.spark.sql.sources.v2; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.sql.sources.v2.writer.BatchWrite; +import org.apache.spark.sql.sources.v2.writer.WriteBuilder; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; /** - * The base interface for data source v2. Implementations must have a public, 0-arg constructor. - * - * Note that this is an empty interface. Data source implementations must mix in interfaces such as - * {@link BatchReadSupportProvider} or {@link BatchWriteSupportProvider}, which can provide - * batch or streaming read/write support instances. Otherwise it's just a dummy data source which - * is un-readable/writable. - * - * If Spark fails to execute any methods in the implementations of this interface (by throwing an - * exception), the read action will fail and no Spark job will be submitted. + * A mix-in interface of {@link Table}, to indicate that it's writable. This adds + * {@link #newWriteBuilder(CaseInsensitiveStringMap)} that is used to create a write + * for batch or streaming. */ -@InterfaceStability.Evolving -public interface DataSourceV2 {} +public interface SupportsWrite extends Table { + + /** + * Returns a {@link WriteBuilder} which can be used to create {@link BatchWrite}. Spark will call + * this method to configure each data source write. + */ + WriteBuilder newWriteBuilder(CaseInsensitiveStringMap options); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java new file mode 100644 index 0000000000000..21e1e8c533f3b --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/Table.java @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.catalog.v2.expressions.Transform; +import org.apache.spark.sql.types.StructType; + +import java.util.Collections; +import java.util.Map; +import java.util.Set; + +/** + * An interface representing a logical structured data set of a data source. For example, the + * implementation can be a directory on the file system, a topic of Kafka, or a table in the + * catalog, etc. + *

    + * This interface can mixin {@code SupportsRead} and {@code SupportsWrite} to provide data reading + * and writing ability. + *

    + * The default implementation of {@link #partitioning()} returns an empty array of partitions, and + * the default implementation of {@link #properties()} returns an empty map. These should be + * overridden by implementations that support partitioning and table properties. + */ +@Evolving +public interface Table { + + /** + * A name to identify this table. Implementations should provide a meaningful name, like the + * database and table name from catalog, or the location of files for this table. + */ + String name(); + + /** + * Returns the schema of this table. If the table is not readable and doesn't have a schema, an + * empty schema can be returned here. + */ + StructType schema(); + + /** + * Returns the physical partitioning of this table. + */ + default Transform[] partitioning() { + return new Transform[0]; + } + + /** + * Returns the string map of table properties. + */ + default Map properties() { + return Collections.emptyMap(); + } + + /** + * Returns the set of capabilities for this table. + */ + Set capabilities(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java new file mode 100644 index 0000000000000..c44a12b174f4c --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableCapability.java @@ -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.sources.v2; + +import org.apache.spark.annotation.Experimental; + +/** + * Capabilities that can be provided by a {@link Table} implementation. + *

    + * Tables use {@link Table#capabilities()} to return a set of capabilities. Each capability signals + * to Spark that the table supports a feature identified by the capability. For example, returning + * {@code BATCH_READ} allows Spark to read from the table using a batch scan. + */ +@Experimental +public enum TableCapability { + /** + * Signals that the table supports reads in batch execution mode. + */ + BATCH_READ, + + /** + * Signals that the table supports reads in micro-batch streaming execution mode. + */ + MICRO_BATCH_READ, + + /** + * Signals that the table supports reads in continuous streaming execution mode. + */ + CONTINUOUS_READ, + + /** + * Signals that the table supports append writes in batch execution mode. + *

    + * Tables that return this capability must support appending data and may also support additional + * write modes, like {@link #TRUNCATE}, {@link #OVERWRITE_BY_FILTER}, and + * {@link #OVERWRITE_DYNAMIC}. + */ + BATCH_WRITE, + + /** + * Signals that the table supports append writes in streaming execution mode. + *

    + * Tables that return this capability must support appending data and may also support additional + * write modes, like {@link #TRUNCATE}, {@link #OVERWRITE_BY_FILTER}, and + * {@link #OVERWRITE_DYNAMIC}. + */ + STREAMING_WRITE, + + /** + * Signals that the table can be truncated in a write operation. + *

    + * Truncating a table removes all existing rows. + *

    + * See {@code org.apache.spark.sql.sources.v2.writer.SupportsTruncate}. + */ + TRUNCATE, + + /** + * Signals that the table can replace existing data that matches a filter with appended data in + * a write operation. + *

    + * See {@code org.apache.spark.sql.sources.v2.writer.SupportsOverwrite}. + */ + OVERWRITE_BY_FILTER, + + /** + * Signals that the table can dynamically replace existing data partitions with appended data in + * a write operation. + *

    + * See {@code org.apache.spark.sql.sources.v2.writer.SupportsDynamicOverwrite}. + */ + OVERWRITE_DYNAMIC, + + /** + * Signals that the table accepts input of any schema in a write operation. + */ + ACCEPT_ANY_SCHEMA +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java new file mode 100644 index 0000000000000..1d37ff042bd33 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/TableProvider.java @@ -0,0 +1,61 @@ +/* + * 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.sources.v2; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +/** + * The base interface for v2 data sources which don't have a real catalog. Implementations must + * have a public, 0-arg constructor. + *

    + * Note that, TableProvider can only apply data operations to existing tables, like read, append, + * delete, and overwrite. It does not support the operations that require metadata changes, like + * create/drop tables. + *

    + * The major responsibility of this interface is to return a {@link Table} for read/write. + *

    + */ +@Evolving +public interface TableProvider { + + /** + * Return a {@link Table} instance to do read/write with user-specified options. + * + * @param options the user-specified options that can identify a table, e.g. file path, Kafka + * topic name, etc. It's an immutable case-insensitive string-to-string map. + */ + Table getTable(CaseInsensitiveStringMap options); + + /** + * Return a {@link Table} instance to do read/write with user-specified schema and options. + *

    + * By default this method throws {@link UnsupportedOperationException}, implementations should + * override this method to handle user-specified schema. + *

    + * @param options the user-specified options that can identify a table, e.g. file path, Kafka + * topic name, etc. It's an immutable case-insensitive string-to-string map. + * @param schema the user-specified schema. + * @throws UnsupportedOperationException + */ + default Table getTable(CaseInsensitiveStringMap options, StructType schema) { + throw new UnsupportedOperationException( + this.getClass().getSimpleName() + " source does not support user-specified schema"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java similarity index 52% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java index a58ddb288f1ed..28d80b7a5bc3f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ReadSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Batch.java @@ -17,34 +17,32 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.types.StructType; +import org.apache.spark.annotation.Evolving; /** - * The base interface for all the batch and streaming read supports. Data sources should implement - * concrete read support interfaces like {@link BatchReadSupport}. - * - * If Spark fails to execute any methods in the implementations of this interface (by throwing an - * exception), the read action will fail and no Spark job will be submitted. + * A physical representation of a data source scan for batch queries. This interface is used to + * provide physical information, like how many partitions the scanned data has, and how to read + * records from the partitions. */ -@InterfaceStability.Evolving -public interface ReadSupport { - - /** - * Returns the full schema of this data source, which is usually the physical schema of the - * underlying storage. This full schema should not be affected by column pruning or other - * optimizations. - */ - StructType fullSchema(); +@Evolving +public interface Batch { /** * Returns a list of {@link InputPartition input partitions}. Each {@link InputPartition} * represents a data split that can be processed by one Spark task. The number of input * partitions returned here is the same as the number of RDD partitions this scan outputs. - * - * Note that, this may not be a full scan if the data source supports optimization like filter - * push-down. Implementations should check the input {@link ScanConfig} and adjust the resulting - * {@link InputPartition input partitions}. + *

    + * If the {@link Scan} supports filter pushdown, this Batch is likely configured with a filter + * and is responsible for creating splits for that filter, which is not a full scan. + *

    + *

    + * This method will be called only once during a data source scan, to launch one Spark job. + *

    + */ + InputPartition[] planInputPartitions(); + + /** + * Returns a factory to create a {@link PartitionReader} for each {@link InputPartition}. */ - InputPartition[] planInputPartitions(ScanConfig config); + PartitionReaderFactory createReaderFactory(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java similarity index 94% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java index 95c30de907e44..413349782efa2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/InputPartition.java @@ -19,11 +19,11 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A serializable representation of an input partition returned by - * {@link ReadSupport#planInputPartitions(ScanConfig)}. + * {@link Batch#planInputPartitions()} and the corresponding ones in streaming . * * Note that {@link InputPartition} will be serialized and sent to executors, then * {@link PartitionReader} will be created by @@ -32,7 +32,7 @@ * the actual reading. So {@link InputPartition} must be serializable while {@link PartitionReader} * doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface InputPartition extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java index 04ff8d0a19fc3..2945925959538 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReader.java @@ -20,7 +20,7 @@ import java.io.Closeable; import java.io.IOException; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A partition reader returned by {@link PartitionReaderFactory#createReader(InputPartition)} or @@ -32,7 +32,7 @@ * data sources(whose {@link PartitionReaderFactory#supportColumnarReads(InputPartition)} * returns true). */ -@InterfaceStability.Evolving +@Evolving public interface PartitionReader extends Closeable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java similarity index 96% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java index f35de9310eee3..97f4a473953fc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/PartitionReaderFactory.java @@ -19,7 +19,7 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -30,7 +30,7 @@ * {@link PartitionReader} (by throwing an exception), corresponding Spark task would fail and * get retried until hitting the maximum retry times. */ -@InterfaceStability.Evolving +@Evolving public interface PartitionReaderFactory extends Serializable { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java new file mode 100644 index 0000000000000..c3964e2176d4f --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Scan.java @@ -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.sources.v2.reader; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousStream; +import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchStream; +import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableCapability; + +/** + * A logical representation of a data source scan. This interface is used to provide logical + * information, like what the actual read schema is. + *

    + * This logical representation is shared between batch scan, micro-batch streaming scan and + * continuous streaming scan. Data sources must implement the corresponding methods in this + * interface, to match what the table promises to support. For example, {@link #toBatch()} must be + * implemented, if the {@link Table} that creates this {@link Scan} returns + * {@link TableCapability#BATCH_READ} support in its {@link Table#capabilities()}. + *

    + */ +@Evolving +public interface Scan { + + /** + * Returns the actual schema of this data source scan, which may be different from the physical + * schema of the underlying storage, as column pruning or other optimizations may happen. + */ + StructType readSchema(); + + /** + * A description string of this scan, which may includes information like: what filters are + * configured for this scan, what's the value of some important options like path, etc. The + * description doesn't need to include {@link #readSchema()}, as Spark already knows it. + *

    + * By default this returns the class name of the implementation. Please override it to provide a + * meaningful description. + *

    + */ + default String description() { + return this.getClass().toString(); + } + + /** + * Returns the physical representation of this scan for batch query. By default this method throws + * exception, data sources must overwrite this method to provide an implementation, if the + * {@link Table} that creates this scan returns {@link TableCapability#BATCH_READ} support in its + * {@link Table#capabilities()}. + * + * @throws UnsupportedOperationException + */ + default Batch toBatch() { + throw new UnsupportedOperationException(description() + ": Batch scan are not supported"); + } + + /** + * Returns the physical representation of this scan for streaming query with micro-batch mode. By + * default this method throws exception, data sources must overwrite this method to provide an + * implementation, if the {@link Table} that creates this scan returns + * {@link TableCapability#MICRO_BATCH_READ} support in its {@link Table#capabilities()}. + * + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Data streams for the same logical source in the same query + * will be given the same checkpointLocation. + * + * @throws UnsupportedOperationException + */ + default MicroBatchStream toMicroBatchStream(String checkpointLocation) { + throw new UnsupportedOperationException(description() + ": Micro-batch scan are not supported"); + } + + /** + * Returns the physical representation of this scan for streaming query with continuous mode. By + * default this method throws exception, data sources must overwrite this method to provide an + * implementation, if the {@link Table} that creates this scan returns + * {@link TableCapability#CONTINUOUS_READ} support in its {@link Table#capabilities()}. + * + * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure + * recovery. Data streams for the same logical source in the same query + * will be given the same checkpointLocation. + * + * @throws UnsupportedOperationException + */ + default ContinuousStream toContinuousStream(String checkpointLocation) { + throw new UnsupportedOperationException(description() + ": Continuous scan are not supported"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java similarity index 70% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java index 4c0eedfddfe22..d4bc1ff977132 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfigBuilder.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanBuilder.java @@ -17,14 +17,14 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** - * An interface for building the {@link ScanConfig}. Implementations can mixin those - * SupportsPushDownXYZ interfaces to do operator pushdown, and keep the operator pushdown result in - * the returned {@link ScanConfig}. + * An interface for building the {@link Scan}. Implementations can mixin SupportsPushDownXYZ + * interfaces to do operator pushdown, and keep the operator pushdown result in the returned + * {@link Scan}. */ -@InterfaceStability.Evolving -public interface ScanConfigBuilder { - ScanConfig build(); +@Evolving +public interface ScanBuilder { + Scan build(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java similarity index 87% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java index 44799c7d49137..a0b194a41f585 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/Statistics.java @@ -19,13 +19,13 @@ import java.util.OptionalLong; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface to represent statistics for a data source, which is returned by - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}. + * {@link SupportsReportStatistics#estimateStatistics()}. */ -@InterfaceStability.Evolving +@Evolving public interface Statistics { OptionalLong sizeInBytes(); OptionalLong numRows(); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java similarity index 82% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java index 5e7985f645a06..f10fd884daabe 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownFilters.java @@ -17,18 +17,21 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.Filter; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this interface to + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this interface to * push down filters to the data source and reduce the size of the data to be read. */ -@InterfaceStability.Evolving -public interface SupportsPushDownFilters extends ScanConfigBuilder { +@Evolving +public interface SupportsPushDownFilters extends ScanBuilder { /** * Pushes down filters, and returns filters that need to be evaluated after scanning. + *

    + * Rows should be returned from the data source if and only if all of the filters match. That is, + * filters must be interpreted as ANDed together. */ Filter[] pushFilters(Filter[] filters); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java similarity index 81% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java index edb164937d6ef..862bd14bffede 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsPushDownRequiredColumns.java @@ -17,16 +17,16 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.StructType; /** - * A mix-in interface for {@link ScanConfigBuilder}. Data sources can implement this + * A mix-in interface for {@link ScanBuilder}. Data sources can implement this * interface to push down required columns to the data source and only read these columns during * scan to reduce the size of the data to be read. */ -@InterfaceStability.Evolving -public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { +@Evolving +public interface SupportsPushDownRequiredColumns extends ScanBuilder { /** * Applies column pruning w.r.t. the given requiredSchema. @@ -35,7 +35,7 @@ public interface SupportsPushDownRequiredColumns extends ScanConfigBuilder { * also OK to do the pruning partially, e.g., a data source may not be able to prune nested * fields, and only prune top-level columns. * - * Note that, {@link ScanConfig#readSchema()} implementation should take care of the column + * Note that, {@link Scan#readSchema()} implementation should take care of the column * pruning applied here. */ void pruneColumns(StructType requiredSchema); diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java similarity index 74% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java index db62cd4515362..4ce97bc5e76b5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportPartitioning.java @@ -17,21 +17,21 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to + * A mix in interface for {@link Scan}. Data sources can implement this interface to * report data partitioning and try to avoid shuffle at Spark side. * - * Note that, when a {@link ReadSupport} implementation creates exactly one {@link InputPartition}, + * Note that, when a {@link Scan} implementation creates exactly one {@link InputPartition}, * Spark may avoid adding a shuffle even if the reader does not implement this interface. */ -@InterfaceStability.Evolving -public interface SupportsReportPartitioning extends ReadSupport { +@Evolving +public interface SupportsReportPartitioning extends Scan { /** * Returns the output data partitioning that this reader guarantees. */ - Partitioning outputPartitioning(ScanConfig config); + Partitioning outputPartitioning(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java similarity index 80% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java index 1831488ba096f..d7364af69e89c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/SupportsReportStatistics.java @@ -17,21 +17,21 @@ package org.apache.spark.sql.sources.v2.reader; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** - * A mix in interface for {@link BatchReadSupport}. Data sources can implement this interface to + * A mix in interface for {@link Scan}. Data sources can implement this interface to * report statistics to Spark. * * As of Spark 2.4, statistics are reported to the optimizer before any operator is pushed to the * data source. Implementations that return more accurate statistics based on pushed operators will * not improve query performance until the planner can push operators before getting stats. */ -@InterfaceStability.Evolving -public interface SupportsReportStatistics extends ReadSupport { +@Evolving +public interface SupportsReportStatistics extends Scan { /** * Returns the estimated statistics of this data source scan. */ - Statistics estimateStatistics(ScanConfig config); + Statistics estimateStatistics(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java similarity index 94% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java index 6764d4b7665c7..1cdc02f5736b1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/ClusteredDistribution.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** @@ -25,7 +25,7 @@ * share the same values for the {@link #clusteredColumns} will be produced by the same * {@link PartitionReader}. */ -@InterfaceStability.Evolving +@Evolving public class ClusteredDistribution implements Distribution { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java index 364a3f553923c..02b0e68974919 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Distribution.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** @@ -37,5 +37,5 @@ *

  • {@link ClusteredDistribution}
  • * */ -@InterfaceStability.Evolving +@Evolving public interface Distribution {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java similarity index 89% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java index fb0b6f1df43bb..c7370eb3d38af 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/partitioning/Partitioning.java @@ -17,18 +17,17 @@ package org.apache.spark.sql.sources.v2.reader.partitioning; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanConfig; import org.apache.spark.sql.sources.v2.reader.SupportsReportPartitioning; /** * An interface to represent the output data partitioning for a data source, which is returned by - * {@link SupportsReportPartitioning#outputPartitioning(ScanConfig)}. Note that this should work + * {@link SupportsReportPartitioning#outputPartitioning()}. Note that this should work * like a snapshot. Once created, it should be deterministic and always report the same number of * partitions and the same "satisfy" result for a certain distribution. */ -@InterfaceStability.Evolving +@Evolving public interface Partitioning { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java similarity index 94% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java index 9101c8a44d34e..c7f6fce6e81af 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReader.java @@ -17,13 +17,13 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.reader.PartitionReader; /** * A variation on {@link PartitionReader} for use with continuous streaming processing. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousPartitionReader extends PartitionReader { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java index 2d9f1ca1686a1..41195befe5e57 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousPartitionReaderFactory.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.reader.InputPartition; import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; @@ -28,7 +28,7 @@ * instead of {@link org.apache.spark.sql.sources.v2.reader.PartitionReader}. It's used for * continuous streaming processing. */ -@InterfaceStability.Evolving +@Evolving public interface ContinuousPartitionReaderFactory extends PartitionReaderFactory { @Override ContinuousPartitionReader createReader(InputPartition partition); diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java new file mode 100644 index 0000000000000..fff5b95a4de14 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousStream.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader.streaming; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.sources.v2.reader.Scan; + +/** + * A {@link SparkDataStream} for streaming queries with continuous mode. + */ +@Evolving +public interface ContinuousStream extends SparkDataStream { + + /** + * Returns a list of {@link InputPartition input partitions} given the start offset. Each + * {@link InputPartition} represents a data split that can be processed by one Spark task. The + * number of input partitions returned here is the same as the number of RDD partitions this scan + * outputs. + *

    + * If the {@link Scan} supports filter pushdown, this stream is likely configured with a filter + * and is responsible for creating splits for that filter, which is not a full scan. + *

    + *

    + * This method will be called to launch one Spark job for reading the data stream. It will be + * called more than once, if {@link #needsReconfiguration()} returns true and Spark needs to + * launch a new job. + *

    + */ + InputPartition[] planInputPartitions(Offset start); + + /** + * Returns a factory to create a {@link ContinuousPartitionReader} for each + * {@link InputPartition}. + */ + ContinuousPartitionReaderFactory createContinuousReaderFactory(); + + /** + * Merge partitioned offsets coming from {@link ContinuousPartitionReader} instances + * for each partition to a single global offset. + */ + Offset mergeOffsets(PartitionOffset[] offsets); + + /** + * The execution engine will call this method in every epoch to determine if new input + * partitions need to be generated, which may be required if for example the underlying + * source system has had partitions added or removed. + * + * If true, the Spark job to scan this continuous data stream will be interrupted and Spark will + * launch it again with a new list of {@link InputPartition input partitions}. + */ + default boolean needsReconfiguration() { + return false; + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java new file mode 100644 index 0000000000000..330f07ba4f2f8 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchStream.java @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.reader.streaming; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; +import org.apache.spark.sql.sources.v2.reader.Scan; + +/** + * A {@link SparkDataStream} for streaming queries with micro-batch mode. + */ +@Evolving +public interface MicroBatchStream extends SparkDataStream { + + /** + * Returns the most recent offset available. + */ + Offset latestOffset(); + + /** + * Returns a list of {@link InputPartition input partitions} given the start and end offsets. Each + * {@link InputPartition} represents a data split that can be processed by one Spark task. The + * number of input partitions returned here is the same as the number of RDD partitions this scan + * outputs. + *

    + * If the {@link Scan} supports filter pushdown, this stream is likely configured with a filter + * and is responsible for creating splits for that filter, which is not a full scan. + *

    + *

    + * This method will be called multiple times, to launch one Spark job for each micro-batch in this + * data stream. + *

    + */ + InputPartition[] planInputPartitions(Offset start, Offset end); + + /** + * Returns a factory to create a {@link PartitionReader} for each {@link InputPartition}. + */ + PartitionReaderFactory createReaderFactory(); +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java similarity index 75% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java index 6cf27734867cb..1d34fdd1c28ab 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/Offset.java @@ -17,21 +17,17 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** - * An abstract representation of progress through a {@link MicroBatchReadSupport} or - * {@link ContinuousReadSupport}. + * An abstract representation of progress through a {@link MicroBatchStream} or + * {@link ContinuousStream}. * During execution, offsets provided by the data source implementation will be logged and used as * restart checkpoints. Each source should provide an offset implementation which the source can use * to reconstruct a position in the stream up to which data has been seen/processed. - * - * Note: This class currently extends {@link org.apache.spark.sql.execution.streaming.Offset} to - * maintain compatibility with DataSource V1 APIs. This extension will be removed once we - * get rid of V1 completely. */ -@InterfaceStability.Evolving -public abstract class Offset extends org.apache.spark.sql.execution.streaming.Offset { +@Evolving +public abstract class Offset { /** * A JSON-serialized representation of an Offset that is * used for saving offsets to the offset log. @@ -49,9 +45,8 @@ public abstract class Offset extends org.apache.spark.sql.execution.streaming.Of */ @Override public boolean equals(Object obj) { - if (obj instanceof org.apache.spark.sql.execution.streaming.Offset) { - return this.json() - .equals(((org.apache.spark.sql.execution.streaming.Offset) obj).json()); + if (obj instanceof Offset) { + return this.json().equals(((Offset) obj).json()); } else { return false; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java similarity index 93% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java index 383e73db6762b..2c97d924a0629 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/PartitionOffset.java @@ -19,7 +19,7 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * Used for per-partition offsets in continuous processing. ContinuousReader implementations will @@ -27,6 +27,6 @@ * * These offsets must be serializable. */ -@InterfaceStability.Evolving +@Evolving public interface PartitionOffset extends Serializable { } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java similarity index 77% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java index 84872d1ebc26e..2068a84fc6bb1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/StreamingReadSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/SparkDataStream.java @@ -17,14 +17,17 @@ package org.apache.spark.sql.sources.v2.reader.streaming; -import org.apache.spark.sql.sources.v2.reader.ReadSupport; +import org.apache.spark.annotation.Evolving; /** - * A base interface for streaming read support. This is package private and is invisible to data - * sources. Data sources should implement concrete streaming read support interfaces: - * {@link MicroBatchReadSupport} or {@link ContinuousReadSupport}. + * The base interface representing a readable data stream in a Spark streaming query. It's + * responsible to manage the offsets of the streaming source in the streaming query. + * + * Data sources should implement concrete data stream interfaces: + * {@link MicroBatchStream} and {@link ContinuousStream}. */ -interface StreamingReadSupport extends ReadSupport { +@Evolving +public interface SparkDataStream { /** * Returns the initial offset for a streaming query to start reading from. Note that the @@ -46,4 +49,9 @@ interface StreamingReadSupport extends ReadSupport { * equal to `end` and will only request offsets greater than `end` in the future. */ void commit(Offset end); + + /** + * Stop this source and free any resources it has allocated. + */ + void stop(); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java similarity index 97% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java index 0ec9e05d6a02b..91297759971b5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWriteSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/BatchWrite.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources.v2.writer; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * An interface that defines how to write the data to data source for batch processing. @@ -37,8 +37,8 @@ * * Please refer to the documentation of commit/abort methods for detailed specifications. */ -@InterfaceStability.Evolving -public interface BatchWriteSupport { +@Evolving +public interface BatchWrite { /** * Creates a writer factory which will be serialized and sent to executors. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java similarity index 89% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java index 5fb067966ee67..11228ad1ea672 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriter.java @@ -19,7 +19,7 @@ import java.io.IOException; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * A data writer returned by {@link DataWriterFactory#createWriter(int, long)} and is @@ -36,11 +36,11 @@ * * If this data writer succeeds(all records are successfully written and {@link #commit()} * succeeds), a {@link WriterCommitMessage} will be sent to the driver side and pass to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} with commit messages from other data + * {@link BatchWrite#commit(WriterCommitMessage[])} with commit messages from other data * writers. If this data writer fails(one record fails to write or {@link #commit()} fails), an * exception will be sent to the driver side, and Spark may retry this writing task a few times. * In each retry, {@link DataWriterFactory#createWriter(int, long)} will receive a - * different `taskId`. Spark will call {@link BatchWriteSupport#abort(WriterCommitMessage[])} + * different `taskId`. Spark will call {@link BatchWrite#abort(WriterCommitMessage[])} * when the configured number of retries is exhausted. * * Besides the retry mechanism, Spark may launch speculative tasks if the existing writing task @@ -55,7 +55,7 @@ * * Note that, Currently the type `T` can only be {@link org.apache.spark.sql.catalyst.InternalRow}. */ -@InterfaceStability.Evolving +@Evolving public interface DataWriter { /** @@ -71,11 +71,11 @@ public interface DataWriter { /** * Commits this writer after all records are written successfully, returns a commit message which * will be sent back to driver side and passed to - * {@link BatchWriteSupport#commit(WriterCommitMessage[])}. + * {@link BatchWrite#commit(WriterCommitMessage[])}. * * The written data should only be visible to data source readers after - * {@link BatchWriteSupport#commit(WriterCommitMessage[])} succeeds, which means this method - * should still "hide" the written data and ask the {@link BatchWriteSupport} at driver side to + * {@link BatchWrite#commit(WriterCommitMessage[])} succeeds, which means this method + * should still "hide" the written data and ask the {@link BatchWrite} at driver side to * do the final commit via {@link WriterCommitMessage}. * * If this method fails (by throwing an exception), {@link #abort()} will be called and this @@ -93,7 +93,7 @@ public interface DataWriter { * failed. * * If this method fails(by throwing an exception), the underlying data source may have garbage - * that need to be cleaned by {@link BatchWriteSupport#abort(WriterCommitMessage[])} or manually, + * that need to be cleaned by {@link BatchWrite#abort(WriterCommitMessage[])} or manually, * but these garbage should not be visible to data source readers. * * @throws IOException if failure happens during disk/network IO like writing files. diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java similarity index 95% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java index 19a36dd232456..bf2db9059b088 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/DataWriterFactory.java @@ -20,18 +20,18 @@ import java.io.Serializable; import org.apache.spark.TaskContext; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; /** - * A factory of {@link DataWriter} returned by {@link BatchWriteSupport#createBatchWriterFactory()}, + * A factory of {@link DataWriter} returned by {@link BatchWrite#createBatchWriterFactory()}, * which is responsible for creating and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be * serializable and {@link DataWriter} doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface DataWriterFactory extends Serializable { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java new file mode 100644 index 0000000000000..8058964b662bd --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsDynamicOverwrite.java @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.sources.v2.writer; + +/** + * Write builder trait for tables that support dynamic partition overwrite. + *

    + * A write that dynamically overwrites partitions removes all existing data in each logical + * partition for which the write will commit new data. Any existing logical partition for which the + * write does not contain data will remain unchanged. + *

    + * This is provided to implement SQL compatible with Hive table operations but is not recommended. + * Instead, use the {@link SupportsOverwrite overwrite by filter API} to explicitly replace data. + */ +public interface SupportsDynamicOverwrite extends WriteBuilder { + /** + * Configures a write to dynamically replace partitions with data committed in the write. + * + * @return this write builder for method chaining + */ + WriteBuilder overwriteDynamicPartitions(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java new file mode 100644 index 0000000000000..b443b3c3aeb4a --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsOverwrite.java @@ -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. + */ + +package org.apache.spark.sql.sources.v2.writer; + +import org.apache.spark.sql.sources.AlwaysTrue$; +import org.apache.spark.sql.sources.Filter; + +/** + * Write builder trait for tables that support overwrite by filter. + *

    + * Overwriting data by filter will delete any data that matches the filter and replace it with data + * that is committed in the write. + */ +public interface SupportsOverwrite extends WriteBuilder, SupportsTruncate { + /** + * Configures a write to replace data matching the filters with data committed in the write. + *

    + * Rows must be deleted from the data source if and only if all of the filters match. That is, + * filters must be interpreted as ANDed together. + * + * @param filters filters used to match data to overwrite + * @return this write builder for method chaining + */ + WriteBuilder overwrite(Filter[] filters); + + @Override + default WriteBuilder truncate() { + return overwrite(new Filter[] { AlwaysTrue$.MODULE$ }); + } +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java new file mode 100644 index 0000000000000..69c2ba5e01a49 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/SupportsTruncate.java @@ -0,0 +1,32 @@ +/* + * 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.sources.v2.writer; + +/** + * Write builder trait for tables that support truncation. + *

    + * Truncation removes all data in a table and replaces it with data that is committed in the write. + */ +public interface SupportsTruncate extends WriteBuilder { + /** + * Configures a write to replace all existing data with data committed in the write. + * + * @return this write builder for method chaining + */ + WriteBuilder truncate(); +} diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java new file mode 100644 index 0000000000000..158066d402922 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriteBuilder.java @@ -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.sources.v2.writer; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; +import org.apache.spark.sql.types.StructType; + +/** + * An interface for building the {@link BatchWrite}. Implementations can mix in some interfaces to + * support different ways to write data to data sources. + * + * Unless modified by a mixin interface, the {@link BatchWrite} configured by this builder is to + * append data without affecting existing data. + */ +@Evolving +public interface WriteBuilder { + + /** + * Passes the `queryId` from Spark to data source. `queryId` is a unique string of the query. It's + * possible that there are many queries running at the same time, or a query is restarted and + * resumed. {@link BatchWrite} can use this id to identify the query. + * + * @return a new builder with the `queryId`. By default it returns `this`, which means the given + * `queryId` is ignored. Please override this method to take the `queryId`. + */ + default WriteBuilder withQueryId(String queryId) { + return this; + } + + /** + * Passes the schema of the input data from Spark to data source. + * + * @return a new builder with the `schema`. By default it returns `this`, which means the given + * `schema` is ignored. Please override this method to take the `schema`. + */ + default WriteBuilder withInputDataSchema(StructType schema) { + return this; + } + + /** + * Returns a {@link BatchWrite} to write data to batch source. By default this method throws + * exception, data sources must overwrite this method to provide an implementation, if the + * {@link Table} that creates this write returns {@link TableCapability#BATCH_WRITE} support in + * its {@link Table#capabilities()}. + */ + default BatchWrite buildForBatch() { + throw new UnsupportedOperationException(getClass().getName() + + " does not support batch write"); + } + + /** + * Returns a {@link StreamingWrite} to write data to streaming source. By default this method + * throws exception, data sources must overwrite this method to provide an implementation, if the + * {@link Table} that creates this write returns {@link TableCapability#STREAMING_WRITE} support + * in its {@link Table#capabilities()}. + */ + default StreamingWrite buildForStreaming() { + throw new UnsupportedOperationException(getClass().getName() + + " does not support streaming write"); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java similarity index 83% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java index 123335c414e9f..23e8580c404d4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/WriterCommitMessage.java @@ -19,16 +19,16 @@ import java.io.Serializable; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite; /** * A commit message returned by {@link DataWriter#commit()} and will be sent back to the driver side - * as the input parameter of {@link BatchWriteSupport#commit(WriterCommitMessage[])} or - * {@link StreamingWriteSupport#commit(long, WriterCommitMessage[])}. + * as the input parameter of {@link BatchWrite#commit(WriterCommitMessage[])} or + * {@link StreamingWrite#commit(long, WriterCommitMessage[])}. * * This is an empty interface, data sources should define their own message class and use it when * generating messages at executor side and handling the messages at driver side. */ -@InterfaceStability.Evolving +@Evolving public interface WriterCommitMessage extends Serializable {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java similarity index 93% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java index a4da24fc5ae68..af2f03c9d4192 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingDataWriterFactory.java @@ -20,20 +20,20 @@ import java.io.Serializable; import org.apache.spark.TaskContext; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.sources.v2.writer.DataWriter; /** * A factory of {@link DataWriter} returned by - * {@link StreamingWriteSupport#createStreamingWriterFactory()}, which is responsible for creating + * {@link StreamingWrite#createStreamingWriterFactory()}, which is responsible for creating * and initializing the actual data writer at executor side. * * Note that, the writer factory will be serialized and sent to executors, then the data writer * will be created on executors and do the actual writing. So this interface must be * serializable and {@link DataWriter} doesn't need to be. */ -@InterfaceStability.Evolving +@Evolving public interface StreamingDataWriterFactory extends Serializable { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java similarity index 71% rename from sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java index 3fdfac5e1c84a..5617f1cdc0efc 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWriteSupport.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/sources/v2/writer/streaming/StreamingWrite.java @@ -17,18 +17,31 @@ package org.apache.spark.sql.sources.v2.writer.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.sources.v2.writer.DataWriter; import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage; /** - * An interface that defines how to write the data to data source for streaming processing. + * An interface that defines how to write the data to data source in streaming queries. * - * Streaming queries are divided into intervals of data called epochs, with a monotonically - * increasing numeric ID. This writer handles commits and aborts for each successive epoch. + * The writing procedure is: + * 1. Create a writer factory by {@link #createStreamingWriterFactory()}, serialize and send it to + * all the partitions of the input data(RDD). + * 2. For each epoch in each partition, create the data writer, and write the data of the epoch in + * the partition with this writer. If all the data are written successfully, call + * {@link DataWriter#commit()}. If exception happens during the writing, call + * {@link DataWriter#abort()}. + * 3. If writers in all partitions of one epoch are successfully committed, call + * {@link #commit(long, WriterCommitMessage[])}. If some writers are aborted, or the job failed + * with an unknown reason, call {@link #abort(long, WriterCommitMessage[])}. + * + * While Spark will retry failed writing tasks, Spark won't retry failed writing jobs. Users should + * do it manually in their Spark applications if they want to retry. + * + * Please refer to the documentation of commit/abort methods for detailed specifications. */ -@InterfaceStability.Evolving -public interface StreamingWriteSupport { +@Evolving +public interface StreamingWrite { /** * Creates a writer factory which will be serialized and sent to executors. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java index 5f1032d1229da..5f6a46f2b8e89 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/GroupStateTimeout.java @@ -17,8 +17,8 @@ package org.apache.spark.sql.streaming; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.catalyst.plans.logical.*; /** @@ -29,7 +29,7 @@ * @since 2.2.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving public class GroupStateTimeout { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java index 470c128ee6c3d..a3d72a1f5d49f 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/streaming/OutputMode.java @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.streaming.InternalOutputModes; /** @@ -26,7 +26,7 @@ * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving public class OutputMode { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java index 0f8570fe470bd..d786374f69e20 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/DataTypes.java @@ -19,7 +19,7 @@ import java.util.*; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * To get/create specific data type, users should use singleton objects and factory methods @@ -27,7 +27,7 @@ * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public class DataTypes { /** * Gets the StringType object. diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java index 1290614a3207d..a54398324fc66 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/types/SQLUserDefinedType.java @@ -20,7 +20,7 @@ import java.lang.annotation.*; import org.apache.spark.annotation.DeveloperApi; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; /** * ::DeveloperApi:: @@ -31,7 +31,7 @@ @DeveloperApi @Retention(RetentionPolicy.RUNTIME) @Target(ElementType.TYPE) -@InterfaceStability.Evolving +@Evolving public @interface SQLUserDefinedType { /** diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java new file mode 100644 index 0000000000000..c344a62be40c1 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/util/CaseInsensitiveStringMap.java @@ -0,0 +1,199 @@ +/* + * 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.util; + +import org.apache.spark.annotation.Experimental; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Set; + +/** + * Case-insensitive map of string keys to string values. + *

    + * This is used to pass options to v2 implementations to ensure consistent case insensitivity. + *

    + * Methods that return keys in this map, like {@link #entrySet()} and {@link #keySet()}, return + * keys converted to lower case. This map doesn't allow null key. + */ +@Experimental +public class CaseInsensitiveStringMap implements Map { + private final Logger logger = LoggerFactory.getLogger(CaseInsensitiveStringMap.class); + + private String unsupportedOperationMsg = "CaseInsensitiveStringMap is read-only."; + + public static CaseInsensitiveStringMap empty() { + return new CaseInsensitiveStringMap(new HashMap<>(0)); + } + + private final Map original; + + private final Map delegate; + + public CaseInsensitiveStringMap(Map originalMap) { + original = new HashMap<>(originalMap); + delegate = new HashMap<>(originalMap.size()); + for (Map.Entry entry : originalMap.entrySet()) { + String key = toLowerCase(entry.getKey()); + if (delegate.containsKey(key)) { + logger.warn("Converting duplicated key " + entry.getKey() + + " into CaseInsensitiveStringMap."); + } + delegate.put(key, entry.getValue()); + } + } + + @Override + public int size() { + return delegate.size(); + } + + @Override + public boolean isEmpty() { + return delegate.isEmpty(); + } + + private String toLowerCase(Object key) { + return key.toString().toLowerCase(Locale.ROOT); + } + + @Override + public boolean containsKey(Object key) { + return delegate.containsKey(toLowerCase(key)); + } + + @Override + public boolean containsValue(Object value) { + return delegate.containsValue(value); + } + + @Override + public String get(Object key) { + return delegate.get(toLowerCase(key)); + } + + @Override + public String put(String key, String value) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public String remove(Object key) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public void putAll(Map m) { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public void clear() { + throw new UnsupportedOperationException(unsupportedOperationMsg); + } + + @Override + public Set keySet() { + return delegate.keySet(); + } + + @Override + public Collection values() { + return delegate.values(); + } + + @Override + public Set> entrySet() { + return delegate.entrySet(); + } + + /** + * Returns the boolean value to which the specified key is mapped, + * or defaultValue if there is no mapping for the key. The key match is case-insensitive. + */ + public boolean getBoolean(String key, boolean defaultValue) { + String value = get(key); + // We can't use `Boolean.parseBoolean` here, as it returns false for invalid strings. + if (value == null) { + return defaultValue; + } else if (value.equalsIgnoreCase("true")) { + return true; + } else if (value.equalsIgnoreCase("false")) { + return false; + } else { + throw new IllegalArgumentException(value + " is not a boolean string."); + } + } + + /** + * Returns the integer value to which the specified key is mapped, + * or defaultValue if there is no mapping for the key. The key match is case-insensitive. + */ + public int getInt(String key, int defaultValue) { + String value = get(key); + return value == null ? defaultValue : Integer.parseInt(value); + } + + /** + * Returns the long value to which the specified key is mapped, + * or defaultValue if there is no mapping for the key. The key match is case-insensitive. + */ + public long getLong(String key, long defaultValue) { + String value = get(key); + return value == null ? defaultValue : Long.parseLong(value); + } + + /** + * Returns the double value to which the specified key is mapped, + * or defaultValue if there is no mapping for the key. The key match is case-insensitive. + */ + public double getDouble(String key, double defaultValue) { + String value = get(key); + return value == null ? defaultValue : Double.parseDouble(value); + } + + /** + * Returns the original case-sensitive map. + */ + public Map asCaseSensitiveMap() { + return Collections.unmodifiableMap(original); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + CaseInsensitiveStringMap that = (CaseInsensitiveStringMap) o; + return delegate.equals(that.delegate); + } + + @Override + public int hashCode() { + return Objects.hash(delegate); + } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java similarity index 98% rename from sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java index 5f58b031f6aef..07d17ee14ce23 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ArrowColumnVector.java @@ -22,8 +22,8 @@ import org.apache.arrow.vector.complex.*; import org.apache.arrow.vector.holders.NullableVarCharHolder; -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.arrow.ArrowUtils; +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.util.ArrowUtils; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.UTF8String; @@ -31,7 +31,7 @@ * A column vector backed by Apache Arrow. Currently calendar interval type and map type are not * supported. */ -@InterfaceStability.Evolving +@Evolving public final class ArrowColumnVector extends ColumnVector { private final ArrowVectorAccessor accessor; diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java similarity index 98% rename from sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java index ad99b450a4809..f18d00359c90c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnVector.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.types.DataType; import org.apache.spark.sql.types.Decimal; import org.apache.spark.unsafe.types.CalendarInterval; @@ -47,7 +47,7 @@ * format. Since it is expected to reuse the ColumnVector instance while loading data, the storage * footprint is negligible. */ -@InterfaceStability.Evolving +@Evolving public abstract class ColumnVector implements AutoCloseable { /** @@ -287,7 +287,7 @@ public final CalendarInterval getInterval(int rowId) { /** * @return child [[ColumnVector]] at the given ordinal. */ - protected abstract ColumnVector getChild(int ordinal); + public abstract ColumnVector getChild(int ordinal); /** * Data type for this column. diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java similarity index 73% rename from sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java index 72a192d089b9f..147dd24e16ce0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarArray.java @@ -16,8 +16,11 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.catalyst.expressions.SpecializedGettersReader; +import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData; import org.apache.spark.sql.catalyst.util.ArrayData; +import org.apache.spark.sql.catalyst.util.GenericArrayData; import org.apache.spark.sql.types.*; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -25,7 +28,7 @@ /** * Array abstraction in {@link ColumnVector}. */ -@InterfaceStability.Evolving +@Evolving public final class ColumnarArray extends ArrayData { // The data for this array. This array contains elements from // data[offset] to data[offset + length). @@ -46,7 +49,25 @@ public int numElements() { @Override public ArrayData copy() { - throw new UnsupportedOperationException(); + DataType dt = data.dataType(); + + if (dt instanceof BooleanType) { + return UnsafeArrayData.fromPrimitiveArray(toBooleanArray()); + } else if (dt instanceof ByteType) { + return UnsafeArrayData.fromPrimitiveArray(toByteArray()); + } else if (dt instanceof ShortType) { + return UnsafeArrayData.fromPrimitiveArray(toShortArray()); + } else if (dt instanceof IntegerType || dt instanceof DateType) { + return UnsafeArrayData.fromPrimitiveArray(toIntArray()); + } else if (dt instanceof LongType || dt instanceof TimestampType) { + return UnsafeArrayData.fromPrimitiveArray(toLongArray()); + } else if (dt instanceof FloatType) { + return UnsafeArrayData.fromPrimitiveArray(toFloatArray()); + } else if (dt instanceof DoubleType) { + return UnsafeArrayData.fromPrimitiveArray(toDoubleArray()); + } else { + return new GenericArrayData(toObjectArray(dt)); + } } @Override @@ -154,42 +175,7 @@ public ColumnarMap getMap(int ordinal) { @Override public Object get(int ordinal, DataType dataType) { - if (dataType instanceof BooleanType) { - return getBoolean(ordinal); - } else if (dataType instanceof ByteType) { - return getByte(ordinal); - } else if (dataType instanceof ShortType) { - return getShort(ordinal); - } else if (dataType instanceof IntegerType) { - return getInt(ordinal); - } else if (dataType instanceof LongType) { - return getLong(ordinal); - } else if (dataType instanceof FloatType) { - return getFloat(ordinal); - } else if (dataType instanceof DoubleType) { - return getDouble(ordinal); - } else if (dataType instanceof StringType) { - return getUTF8String(ordinal); - } else if (dataType instanceof BinaryType) { - return getBinary(ordinal); - } else if (dataType instanceof DecimalType) { - DecimalType t = (DecimalType) dataType; - return getDecimal(ordinal, t.precision(), t.scale()); - } else if (dataType instanceof DateType) { - return getInt(ordinal); - } else if (dataType instanceof TimestampType) { - return getLong(ordinal); - } else if (dataType instanceof ArrayType) { - return getArray(ordinal); - } else if (dataType instanceof StructType) { - return getStruct(ordinal, ((StructType)dataType).fields().length); - } else if (dataType instanceof MapType) { - return getMap(ordinal); - } else if (dataType instanceof CalendarIntervalType) { - return getInterval(ordinal); - } else { - throw new UnsupportedOperationException("Datatype not supported " + dataType); - } + return SpecializedGettersReader.read(this, ordinal, dataType, false, false); } @Override diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java new file mode 100644 index 0000000000000..a2feac869ece6 --- /dev/null +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java @@ -0,0 +1,291 @@ +/* + * 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.vectorized; + +import java.util.*; + +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; +import org.apache.spark.sql.types.*; +import org.apache.spark.unsafe.types.CalendarInterval; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * This class wraps multiple ColumnVectors as a row-wise table. It provides a row view of this + * batch so that Spark can access the data row by row. Instance of it is meant to be reused during + * the entire data loading process. + */ +@Evolving +public final class ColumnarBatch implements AutoCloseable { + private int numRows; + private final ColumnVector[] columns; + + // Staging row returned from `getRow`. + private final ColumnarBatchRow row; + + /** + * Called to close all the columns in this batch. It is not valid to access the data after + * calling this. This must be called at the end to clean up memory allocations. + */ + @Override + public void close() { + for (ColumnVector c: columns) { + c.close(); + } + } + + /** + * Returns an iterator over the rows in this batch. + */ + public Iterator rowIterator() { + final int maxRows = numRows; + final ColumnarBatchRow row = new ColumnarBatchRow(columns); + return new Iterator() { + int rowId = 0; + + @Override + public boolean hasNext() { + return rowId < maxRows; + } + + @Override + public InternalRow next() { + if (rowId >= maxRows) { + throw new NoSuchElementException(); + } + row.rowId = rowId++; + return row; + } + + @Override + public void remove() { + throw new UnsupportedOperationException(); + } + }; + } + + /** + * Sets the number of rows in this batch. + */ + public void setNumRows(int numRows) { + this.numRows = numRows; + } + + /** + * Returns the number of columns that make up this batch. + */ + public int numCols() { return columns.length; } + + /** + * Returns the number of rows for read, including filtered rows. + */ + public int numRows() { return numRows; } + + /** + * Returns the column at `ordinal`. + */ + public ColumnVector column(int ordinal) { return columns[ordinal]; } + + /** + * Returns the row in this batch at `rowId`. Returned row is reused across calls. + */ + public InternalRow getRow(int rowId) { + assert(rowId >= 0 && rowId < numRows); + row.rowId = rowId; + return row; + } + + public ColumnarBatch(ColumnVector[] columns) { + this(columns, 0); + } + + /** + * Create a new batch from existing column vectors. + * @param columns The columns of this batch + * @param numRows The number of rows in this batch + */ + public ColumnarBatch(ColumnVector[] columns, int numRows) { + this.columns = columns; + this.numRows = numRows; + this.row = new ColumnarBatchRow(columns); + } +} + +/** + * An internal class, which wraps an array of {@link ColumnVector} and provides a row view. + */ +class ColumnarBatchRow extends InternalRow { + public int rowId; + private final ColumnVector[] columns; + + ColumnarBatchRow(ColumnVector[] columns) { + this.columns = columns; + } + + @Override + public int numFields() { return columns.length; } + + @Override + public InternalRow copy() { + GenericInternalRow row = new GenericInternalRow(columns.length); + for (int i = 0; i < numFields(); i++) { + if (isNullAt(i)) { + row.setNullAt(i); + } else { + DataType dt = columns[i].dataType(); + if (dt instanceof BooleanType) { + row.setBoolean(i, getBoolean(i)); + } else if (dt instanceof ByteType) { + row.setByte(i, getByte(i)); + } else if (dt instanceof ShortType) { + row.setShort(i, getShort(i)); + } else if (dt instanceof IntegerType) { + row.setInt(i, getInt(i)); + } else if (dt instanceof LongType) { + row.setLong(i, getLong(i)); + } else if (dt instanceof FloatType) { + row.setFloat(i, getFloat(i)); + } else if (dt instanceof DoubleType) { + row.setDouble(i, getDouble(i)); + } else if (dt instanceof StringType) { + row.update(i, getUTF8String(i).copy()); + } else if (dt instanceof BinaryType) { + row.update(i, getBinary(i)); + } else if (dt instanceof DecimalType) { + DecimalType t = (DecimalType)dt; + row.setDecimal(i, getDecimal(i, t.precision(), t.scale()), t.precision()); + } else if (dt instanceof DateType) { + row.setInt(i, getInt(i)); + } else if (dt instanceof TimestampType) { + row.setLong(i, getLong(i)); + } else { + throw new RuntimeException("Not implemented. " + dt); + } + } + } + return row; + } + + @Override + public boolean anyNull() { + throw new UnsupportedOperationException(); + } + + @Override + public boolean isNullAt(int ordinal) { return columns[ordinal].isNullAt(rowId); } + + @Override + public boolean getBoolean(int ordinal) { return columns[ordinal].getBoolean(rowId); } + + @Override + public byte getByte(int ordinal) { return columns[ordinal].getByte(rowId); } + + @Override + public short getShort(int ordinal) { return columns[ordinal].getShort(rowId); } + + @Override + public int getInt(int ordinal) { return columns[ordinal].getInt(rowId); } + + @Override + public long getLong(int ordinal) { return columns[ordinal].getLong(rowId); } + + @Override + public float getFloat(int ordinal) { return columns[ordinal].getFloat(rowId); } + + @Override + public double getDouble(int ordinal) { return columns[ordinal].getDouble(rowId); } + + @Override + public Decimal getDecimal(int ordinal, int precision, int scale) { + return columns[ordinal].getDecimal(rowId, precision, scale); + } + + @Override + public UTF8String getUTF8String(int ordinal) { + return columns[ordinal].getUTF8String(rowId); + } + + @Override + public byte[] getBinary(int ordinal) { + return columns[ordinal].getBinary(rowId); + } + + @Override + public CalendarInterval getInterval(int ordinal) { + return columns[ordinal].getInterval(rowId); + } + + @Override + public ColumnarRow getStruct(int ordinal, int numFields) { + return columns[ordinal].getStruct(rowId); + } + + @Override + public ColumnarArray getArray(int ordinal) { + return columns[ordinal].getArray(rowId); + } + + @Override + public ColumnarMap getMap(int ordinal) { + return columns[ordinal].getMap(rowId); + } + + @Override + public Object get(int ordinal, DataType dataType) { + if (dataType instanceof BooleanType) { + return getBoolean(ordinal); + } else if (dataType instanceof ByteType) { + return getByte(ordinal); + } else if (dataType instanceof ShortType) { + return getShort(ordinal); + } else if (dataType instanceof IntegerType) { + return getInt(ordinal); + } else if (dataType instanceof LongType) { + return getLong(ordinal); + } else if (dataType instanceof FloatType) { + return getFloat(ordinal); + } else if (dataType instanceof DoubleType) { + return getDouble(ordinal); + } else if (dataType instanceof StringType) { + return getUTF8String(ordinal); + } else if (dataType instanceof BinaryType) { + return getBinary(ordinal); + } else if (dataType instanceof DecimalType) { + DecimalType t = (DecimalType) dataType; + return getDecimal(ordinal, t.precision(), t.scale()); + } else if (dataType instanceof DateType) { + return getInt(ordinal); + } else if (dataType instanceof TimestampType) { + return getLong(ordinal); + } else if (dataType instanceof ArrayType) { + return getArray(ordinal); + } else if (dataType instanceof StructType) { + return getStruct(ordinal, ((StructType)dataType).fields().length); + } else if (dataType instanceof MapType) { + return getMap(ordinal); + } else { + throw new UnsupportedOperationException("Datatype not supported " + dataType); + } + } + + @Override + public void update(int ordinal, Object value) { throw new UnsupportedOperationException(); } + + @Override + public void setNullAt(int ordinal) { throw new UnsupportedOperationException(); } +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarMap.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarMap.java similarity index 100% rename from sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarMap.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarMap.java diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java similarity index 98% rename from sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java rename to sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java index f2f2279590023..4b9d3c5f59915 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java +++ b/sql/catalyst/src/main/java/org/apache/spark/sql/vectorized/ColumnarRow.java @@ -16,7 +16,7 @@ */ package org.apache.spark.sql.vectorized; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Evolving; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.types.*; @@ -26,7 +26,7 @@ /** * Row abstraction in {@link ColumnVector}. */ -@InterfaceStability.Evolving +@Evolving public final class ColumnarRow extends InternalRow { // The data for this row. // E.g. the value of 3rd int field is `data.getChild(3).getInt(rowId)`. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala index 50ee6cd4085ea..f5c87677ab9eb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/AnalysisException.scala @@ -17,16 +17,15 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan - /** * Thrown when a query fails to analyze, usually because the query itself is invalid. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class AnalysisException protected[sql] ( val message: String, val line: Option[Int] = None, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala index 7b02317b8538f..9853a4fcc2f9d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoder.scala @@ -20,10 +20,9 @@ package org.apache.spark.sql import scala.annotation.implicitNotFound import scala.reflect.ClassTag -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.types._ - /** * :: Experimental :: * Used to convert a JVM object of type `T` to and from the internal Spark SQL representation. @@ -67,7 +66,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving @implicitNotFound("Unable to find encoder for type ${T}. An implicit Encoder[${T}] is needed to " + "store ${T} instances in a Dataset. Primitive types (Int, String, etc) and Product types (case " + "classes) are supported by importing spark.implicits._ Support for serializing other types " + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala index b47ec0b72c638..055fbc49bdcd7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Encoders.scala @@ -22,7 +22,7 @@ import java.lang.reflect.Modifier import scala.reflect.{classTag, ClassTag} import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{BoundReference, Cast} @@ -36,7 +36,7 @@ import org.apache.spark.sql.types._ * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving object Encoders { /** @@ -109,6 +109,14 @@ object Encoders { */ def DATE: Encoder[java.sql.Date] = ExpressionEncoder() + /** + * Creates an encoder that serializes instances of the `java.time.LocalDate` class + * to the internal representation of nullable Catalyst's DateType. + * + * @since 3.0.0 + */ + def LOCALDATE: Encoder[java.time.LocalDate] = ExpressionEncoder() + /** * An encoder for nullable timestamp type. * @@ -116,6 +124,14 @@ object Encoders { */ def TIMESTAMP: Encoder[java.sql.Timestamp] = ExpressionEncoder() + /** + * Creates an encoder that serializes instances of the `java.time.Instant` class + * to the internal representation of nullable Catalyst's TimestampType. + * + * @since 3.0.0 + */ + def INSTANT: Encoder[java.time.Instant] = ExpressionEncoder() + /** * An encoder for arrays of bytes. * @@ -133,7 +149,7 @@ object Encoders { * - boxed types: Boolean, Integer, Double, etc. * - String * - java.math.BigDecimal, java.math.BigInteger - * - time related: java.sql.Date, java.sql.Timestamp + * - time related: java.sql.Date, java.sql.Timestamp, java.time.LocalDate, java.time.Instant * - collection types: only array and java.util.List currently, map support is in progress * - nested java bean. * @@ -203,12 +219,10 @@ object Encoders { validatePublicClass[T]() ExpressionEncoder[T]( - schema = new StructType().add("value", BinaryType), - flat = true, - serializer = Seq( + objSerializer = EncodeUsingSerializer( - BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo)), - deserializer = + BoundReference(0, ObjectType(classOf[AnyRef]), nullable = true), kryo = useKryo), + objDeserializer = DecodeUsingSerializer[T]( Cast(GetColumnByOrdinal(0, BinaryType), BinaryType), classTag[T], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala index 180c2d130074e..494387ecfe0c1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/Row.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.util.hashing.MurmurHash3 -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.types.StructType /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object Row { /** * This method can be used to extract fields from a [[Row]] object in a pattern match. Example: @@ -57,6 +57,7 @@ object Row { /** * Merge multiple rows into a single row, one after another. */ + @deprecated("This method is deprecated and will be removed in future versions.", "3.0.0") def merge(rows: Row*): Row = { // TODO: Improve the performance of this if used in performance critical part. new GenericRow(rows.flatMap(_.toSeq).toArray) @@ -124,7 +125,7 @@ object Row { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait Row extends Serializable { /** Number of elements in the Row. */ def size: Int = length @@ -268,6 +269,13 @@ trait Row extends Serializable { */ def getDate(i: Int): java.sql.Date = getAs[java.sql.Date](i) + /** + * Returns the value at position i of date type as java.time.LocalDate. + * + * @throws ClassCastException when data type does not match. + */ + def getLocalDate(i: Int): java.time.LocalDate = getAs[java.time.LocalDate](i) + /** * Returns the value at position i of date type as java.sql.Timestamp. * @@ -275,6 +283,13 @@ trait Row extends Serializable { */ def getTimestamp(i: Int): java.sql.Timestamp = getAs[java.sql.Timestamp](i) + /** + * Returns the value at position i of date type as java.time.Instant. + * + * @throws ClassCastException when data type does not match. + */ + def getInstant(i: Int): java.time.Instant = getAs[java.time.Instant](i) + /** * Returns the value at position i of array type as a Scala Seq. * @@ -357,7 +372,7 @@ trait Row extends Serializable { }.toMap } - override def toString: String = s"[${this.mkString(",")}]" + override def toString: String = this.mkString("[", ",", "]") /** * Make a copy of the current [[Row]] object. @@ -450,16 +465,31 @@ trait Row extends Serializable { } /** Displays all elements of this sequence in a string (without a separator). */ - def mkString: String = toSeq.mkString + def mkString: String = mkString("") /** Displays all elements of this sequence in a string using a separator string. */ - def mkString(sep: String): String = toSeq.mkString(sep) + def mkString(sep: String): String = mkString("", sep, "") /** * Displays all elements of this traversable or iterator in a string using * start, end, and separator strings. */ - def mkString(start: String, sep: String, end: String): String = toSeq.mkString(start, sep, end) + def mkString(start: String, sep: String, end: String): String = { + val n = length + val builder = new StringBuilder + builder.append(start) + if (n > 0) { + builder.append(get(0)) + var i = 1 + while (i < n) { + builder.append(sep) + builder.append(get(i)) + i += 1 + } + } + builder.append(end) + builder.toString() + } /** * Returns the value at position i. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala new file mode 100644 index 0000000000000..86de1c9285b73 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogNotFoundException.scala @@ -0,0 +1,28 @@ +/* + * 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.catalog.v2 + +import org.apache.spark.SparkException +import org.apache.spark.annotation.Experimental + +@Experimental +class CatalogNotFoundException(message: String, cause: Throwable) + extends SparkException(message, cause) { + + def this(message: String) = this(message, null) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala new file mode 100644 index 0000000000000..f512cd5e23c6b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/CatalogV2Implicits.scala @@ -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.sql.catalog.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, IdentityTransform, LogicalExpressions, Transform} +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.types.StructType + +/** + * Conversion helpers for working with v2 [[CatalogPlugin]]. + */ +object CatalogV2Implicits { + implicit class PartitionTypeHelper(partitionType: StructType) { + def asTransforms: Array[Transform] = partitionType.names.map(LogicalExpressions.identity) + } + + implicit class BucketSpecHelper(spec: BucketSpec) { + def asTransform: BucketTransform = { + if (spec.sortColumnNames.nonEmpty) { + throw new AnalysisException( + s"Cannot convert bucketing with sort columns to a transform: $spec") + } + + LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*) + } + } + + implicit class TransformHelper(transforms: Seq[Transform]) { + def asPartitionColumns: Seq[String] = { + val (idTransforms, nonIdTransforms) = transforms.partition(_.isInstanceOf[IdentityTransform]) + + if (nonIdTransforms.nonEmpty) { + throw new AnalysisException("Transforms cannot be converted to partition columns: " + + nonIdTransforms.map(_.describe).mkString(", ")) + } + + idTransforms.map(_.asInstanceOf[IdentityTransform]).map(_.reference).map { ref => + val parts = ref.fieldNames + if (parts.size > 1) { + throw new AnalysisException(s"Cannot partition by nested column: $ref") + } else { + parts(0) + } + } + } + } + + implicit class CatalogHelper(plugin: CatalogPlugin) { + def asTableCatalog: TableCatalog = plugin match { + case tableCatalog: TableCatalog => + tableCatalog + case _ => + throw new AnalysisException(s"Cannot use catalog ${plugin.name}: not a TableCatalog") + } + } + + implicit class NamespaceHelper(namespace: Array[String]) { + def quoted: String = namespace.map(quote).mkString(".") + } + + implicit class IdentifierHelper(ident: Identifier) { + def quoted: String = { + if (ident.namespace.nonEmpty) { + ident.namespace.map(quote).mkString(".") + "." + quote(ident.name) + } else { + quote(ident.name) + } + } + } + + implicit class MultipartIdentifierHelper(namespace: Seq[String]) { + def quoted: String = namespace.map(quote).mkString(".") + } + + private def quote(part: String): String = { + if (part.contains(".") || part.contains("`")) { + s"`${part.replace("`", "``")}`" + } else { + part + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala new file mode 100644 index 0000000000000..5f7ee30cdab79 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/LookupCatalog.scala @@ -0,0 +1,143 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2 + +import scala.util.control.NonFatal + +import org.apache.spark.annotation.Experimental +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.TableIdentifier + +/** + * A trait to encapsulate catalog lookup function and helpful extractors. + */ +@Experimental +trait LookupCatalog extends Logging { + + import LookupCatalog._ + + protected def defaultCatalogName: Option[String] = None + protected def lookupCatalog(name: String): CatalogPlugin + + /** + * Returns the default catalog. When set, this catalog is used for all identifiers that do not + * set a specific catalog. When this is None, the session catalog is responsible for the + * identifier. + * + * If this is None and a table's provider (source) is a v2 provider, the v2 session catalog will + * be used. + */ + def defaultCatalog: Option[CatalogPlugin] = { + try { + defaultCatalogName.map(lookupCatalog) + } catch { + case NonFatal(e) => + logError(s"Cannot load default v2 catalog: ${defaultCatalogName.get}", e) + None + } + } + + /** + * This catalog is a v2 catalog that delegates to the v1 session catalog. it is used when the + * session catalog is responsible for an identifier, but the source requires the v2 catalog API. + * This happens when the source implementation extends the v2 TableProvider API and is not listed + * in the fallback configuration, spark.sql.sources.write.useV1SourceList + */ + def sessionCatalog: Option[CatalogPlugin] = { + try { + Some(lookupCatalog(SESSION_CATALOG_NAME)) + } catch { + case NonFatal(e) => + logError("Cannot load v2 session catalog", e) + None + } + } + + /** + * Extract catalog plugin and remaining identifier names. + * + * This does not substitute the default catalog if no catalog is set in the identifier. + */ + private object CatalogAndIdentifier { + def unapply(parts: Seq[String]): Some[(Option[CatalogPlugin], Seq[String])] = parts match { + case Seq(_) => + Some((None, parts)) + case Seq(catalogName, tail @ _*) => + try { + Some((Some(lookupCatalog(catalogName)), tail)) + } catch { + case _: CatalogNotFoundException => + Some((None, parts)) + } + } + } + + type CatalogObjectIdentifier = (Option[CatalogPlugin], Identifier) + + /** + * Extract catalog and identifier from a multi-part identifier with the default catalog if needed. + */ + object CatalogObjectIdentifier { + def unapply(parts: Seq[String]): Some[CatalogObjectIdentifier] = parts match { + case CatalogAndIdentifier(maybeCatalog, nameParts) => + Some(( + maybeCatalog.orElse(defaultCatalog), + Identifier.of(nameParts.init.toArray, nameParts.last) + )) + } + } + + /** + * Extract legacy table identifier from a multi-part identifier. + * + * For legacy support only. Please use [[CatalogObjectIdentifier]] instead on DSv2 code paths. + */ + object AsTableIdentifier { + def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { + case CatalogAndIdentifier(None, names) if defaultCatalog.isEmpty => + names match { + case Seq(name) => + Some(TableIdentifier(name)) + case Seq(database, name) => + Some(TableIdentifier(name, Some(database))) + case _ => + None + } + case _ => + None + } + } + + /** + * For temp views, extract a table identifier from a multi-part identifier if it has no catalog. + */ + object AsTemporaryViewIdentifier { + def unapply(parts: Seq[String]): Option[TableIdentifier] = parts match { + case CatalogAndIdentifier(None, Seq(table)) => + Some(TableIdentifier(table)) + case CatalogAndIdentifier(None, Seq(database, table)) => + Some(TableIdentifier(table, Some(database))) + case _ => + None + } + } +} + +object LookupCatalog { + val SESSION_CATALOG_NAME: String = "session" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala new file mode 100644 index 0000000000000..ea5fc05dd5ff3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalog/v2/expressions/expressions.scala @@ -0,0 +1,245 @@ +/* + * 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.catalog.v2.expressions + +import org.apache.spark.sql.catalyst +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, IntegerType, StringType} + +/** + * Helper methods for working with the logical expressions API. + * + * Factory methods can be used when referencing the logical expression nodes is ambiguous because + * logical and internal expressions are used. + */ +private[sql] object LogicalExpressions { + // a generic parser that is only used for parsing multi-part field names. + // because this is only used for field names, the SQL conf passed in does not matter. + private lazy val parser = new CatalystSqlParser(SQLConf.get) + + def literal[T](value: T): LiteralValue[T] = { + val internalLit = catalyst.expressions.Literal(value) + literal(value, internalLit.dataType) + } + + def literal[T](value: T, dataType: DataType): LiteralValue[T] = LiteralValue(value, dataType) + + def reference(name: String): NamedReference = + FieldReference(parser.parseMultipartIdentifier(name)) + + def apply(name: String, arguments: Expression*): Transform = ApplyTransform(name, arguments) + + def bucket(numBuckets: Int, columns: String*): BucketTransform = + BucketTransform(literal(numBuckets, IntegerType), columns.map(reference)) + + def identity(column: String): IdentityTransform = IdentityTransform(reference(column)) + + def years(column: String): YearsTransform = YearsTransform(reference(column)) + + def months(column: String): MonthsTransform = MonthsTransform(reference(column)) + + def days(column: String): DaysTransform = DaysTransform(reference(column)) + + def hours(column: String): HoursTransform = HoursTransform(reference(column)) +} + +/** + * Base class for simple transforms of a single column. + */ +private[sql] abstract class SingleColumnTransform(ref: NamedReference) extends Transform { + + def reference: NamedReference = ref + + override def references: Array[NamedReference] = Array(ref) + + override def arguments: Array[Expression] = Array(ref) + + override def describe: String = name + "(" + reference.describe + ")" + + override def toString: String = describe +} + +private[sql] final case class BucketTransform( + numBuckets: Literal[Int], + columns: Seq[NamedReference]) extends Transform { + + override val name: String = "bucket" + + override def references: Array[NamedReference] = { + arguments + .filter(_.isInstanceOf[NamedReference]) + .map(_.asInstanceOf[NamedReference]) + } + + override def arguments: Array[Expression] = numBuckets +: columns.toArray + + override def describe: String = s"bucket(${arguments.map(_.describe).mkString(", ")})" + + override def toString: String = describe +} + +private[sql] object BucketTransform { + def unapply(transform: Transform): Option[(Int, NamedReference)] = transform match { + case NamedTransform("bucket", Seq( + Lit(value: Int, IntegerType), + Ref(seq: Seq[String]))) => + Some((value, FieldReference(seq))) + case _ => + None + } +} + +private[sql] final case class ApplyTransform( + name: String, + args: Seq[Expression]) extends Transform { + + override def arguments: Array[Expression] = args.toArray + + override def references: Array[NamedReference] = { + arguments + .filter(_.isInstanceOf[NamedReference]) + .map(_.asInstanceOf[NamedReference]) + } + + override def describe: String = s"$name(${arguments.map(_.describe).mkString(", ")})" + + override def toString: String = describe +} + +/** + * Convenience extractor for any Literal. + */ +private object Lit { + def unapply[T](literal: Literal[T]): Some[(T, DataType)] = { + Some((literal.value, literal.dataType)) + } +} + +/** + * Convenience extractor for any NamedReference. + */ +private object Ref { + def unapply(named: NamedReference): Some[Seq[String]] = { + Some(named.fieldNames) + } +} + +/** + * Convenience extractor for any Transform. + */ +private object NamedTransform { + def unapply(transform: Transform): Some[(String, Seq[Expression])] = { + Some((transform.name, transform.arguments)) + } +} + +private[sql] final case class IdentityTransform( + ref: NamedReference) extends SingleColumnTransform(ref) { + override val name: String = "identity" + override def describe: String = ref.describe +} + +private[sql] object IdentityTransform { + def unapply(transform: Transform): Option[FieldReference] = transform match { + case NamedTransform("identity", Seq(Ref(parts))) => + Some(FieldReference(parts)) + case _ => + None + } +} + +private[sql] final case class YearsTransform( + ref: NamedReference) extends SingleColumnTransform(ref) { + override val name: String = "years" +} + +private[sql] object YearsTransform { + def unapply(transform: Transform): Option[FieldReference] = transform match { + case NamedTransform("years", Seq(Ref(parts))) => + Some(FieldReference(parts)) + case _ => + None + } +} + +private[sql] final case class MonthsTransform( + ref: NamedReference) extends SingleColumnTransform(ref) { + override val name: String = "months" +} + +private[sql] object MonthsTransform { + def unapply(transform: Transform): Option[FieldReference] = transform match { + case NamedTransform("months", Seq(Ref(parts))) => + Some(FieldReference(parts)) + case _ => + None + } +} + +private[sql] final case class DaysTransform( + ref: NamedReference) extends SingleColumnTransform(ref) { + override val name: String = "days" +} + +private[sql] object DaysTransform { + def unapply(transform: Transform): Option[FieldReference] = transform match { + case NamedTransform("days", Seq(Ref(parts))) => + Some(FieldReference(parts)) + case _ => + None + } +} + +private[sql] final case class HoursTransform( + ref: NamedReference) extends SingleColumnTransform(ref) { + override val name: String = "hours" +} + +private[sql] object HoursTransform { + def unapply(transform: Transform): Option[FieldReference] = transform match { + case NamedTransform("hours", Seq(Ref(parts))) => + Some(FieldReference(parts)) + case _ => + None + } +} + +private[sql] final case class LiteralValue[T](value: T, dataType: DataType) extends Literal[T] { + override def describe: String = { + if (dataType.isInstanceOf[StringType]) { + s"'$value'" + } else { + s"$value" + } + } + override def toString: String = describe +} + +private[sql] final case class FieldReference(parts: Seq[String]) extends NamedReference { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.MultipartIdentifierHelper + override def fieldNames: Array[String] = parts.toArray + override def describe: String = parts.quoted + override def toString: String = describe +} + +private[sql] object FieldReference { + def apply(column: String): NamedReference = { + LogicalExpressions.reference(column) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala index 6f5fbdd79e668..6020b068155fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystTypeConverters.scala @@ -21,6 +21,7 @@ import java.lang.{Iterable => JavaIterable} import java.math.{BigDecimal => JavaBigDecimal} import java.math.{BigInteger => JavaBigInteger} import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} import java.util.{Map => JavaMap} import javax.annotation.Nullable @@ -29,6 +30,7 @@ import scala.language.existentials import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -40,7 +42,7 @@ object CatalystTypeConverters { // Since the map values can be mutable, we explicitly import scala.collection.Map at here. import scala.collection.Map - private def isPrimitive(dataType: DataType): Boolean = { + private[sql] def isPrimitive(dataType: DataType): Boolean = { dataType match { case BooleanType => true case ByteType => true @@ -60,7 +62,9 @@ object CatalystTypeConverters { case mapType: MapType => MapConverter(mapType.keyType, mapType.valueType) case structType: StructType => StructConverter(structType) case StringType => StringConverter + case DateType if SQLConf.get.datetimeJava8ApiEnabled => LocalDateConverter case DateType => DateConverter + case TimestampType if SQLConf.get.datetimeJava8ApiEnabled => InstantConverter case TimestampType => TimestampConverter case dt: DecimalType => new DecimalConverter(dt) case BooleanType => BooleanConverter @@ -305,6 +309,18 @@ object CatalystTypeConverters { DateTimeUtils.toJavaDate(row.getInt(column)) } + private object LocalDateConverter extends CatalystTypeConverter[LocalDate, LocalDate, Any] { + override def toCatalystImpl(scalaValue: LocalDate): Int = { + DateTimeUtils.localDateToDays(scalaValue) + } + override def toScala(catalystValue: Any): LocalDate = { + if (catalystValue == null) null + else DateTimeUtils.daysToLocalDate(catalystValue.asInstanceOf[Int]) + } + override def toScalaImpl(row: InternalRow, column: Int): LocalDate = + DateTimeUtils.daysToLocalDate(row.getInt(column)) + } + private object TimestampConverter extends CatalystTypeConverter[Timestamp, Timestamp, Any] { override def toCatalystImpl(scalaValue: Timestamp): Long = DateTimeUtils.fromJavaTimestamp(scalaValue) @@ -315,6 +331,16 @@ object CatalystTypeConverters { DateTimeUtils.toJavaTimestamp(row.getLong(column)) } + private object InstantConverter extends CatalystTypeConverter[Instant, Instant, Any] { + override def toCatalystImpl(scalaValue: Instant): Long = + DateTimeUtils.instantToMicros(scalaValue) + override def toScala(catalystValue: Any): Instant = + if (catalystValue == null) null + else DateTimeUtils.microsToInstant(catalystValue.asInstanceOf[Long]) + override def toScalaImpl(row: InternalRow, column: Int): Instant = + DateTimeUtils.microsToInstant(row.getLong(column)) + } + private class DecimalConverter(dataType: DecimalType) extends CatalystTypeConverter[Any, JavaBigDecimal, Decimal] { override def toCatalystImpl(scalaValue: Any): Decimal = { @@ -420,7 +446,9 @@ object CatalystTypeConverters { def convertToCatalyst(a: Any): Any = a match { case s: String => StringConverter.toCatalyst(s) case d: Date => DateConverter.toCatalyst(d) + case ld: LocalDate => LocalDateConverter.toCatalyst(ld) case t: Timestamp => TimestampConverter.toCatalyst(t) + case i: Instant => InstantConverter.toCatalyst(i) case d: BigDecimal => new DecimalConverter(DecimalType(d.precision, d.scale)).toCatalyst(d) case d: JavaBigDecimal => new DecimalConverter(DecimalType(d.precision, d.scale)).toCatalyst(d) case seq: Seq[Any] => new GenericArrayData(seq.map(convertToCatalyst).toArray) @@ -431,12 +459,6 @@ object CatalystTypeConverters { map, (key: Any) => convertToCatalyst(key), (value: Any) => convertToCatalyst(value)) - case (keys: Array[_], values: Array[_]) => - // case for mapdata with duplicate keys - new ArrayBasedMapData( - new GenericArrayData(keys.map(convertToCatalyst)), - new GenericArrayData(values.map(convertToCatalyst)) - ) case other => other } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala new file mode 100644 index 0000000000000..e55c25c4b0c54 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/DeserializerBuildHelper.scala @@ -0,0 +1,166 @@ +/* + * 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 + +import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField, UpCast} +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, StaticInvoke} +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ + +object DeserializerBuildHelper { + /** Returns the current path with a sub-field extracted. */ + def addToPath( + path: Expression, + part: String, + dataType: DataType, + walkedTypePath: WalkedTypePath): Expression = { + val newPath = UnresolvedExtractValue(path, expressions.Literal(part)) + upCastToExpectedType(newPath, dataType, walkedTypePath) + } + + /** Returns the current path with a field at ordinal extracted. */ + def addToPathOrdinal( + path: Expression, + ordinal: Int, + dataType: DataType, + walkedTypePath: WalkedTypePath): Expression = { + val newPath = GetStructField(path, ordinal) + upCastToExpectedType(newPath, dataType, walkedTypePath) + } + + def deserializerForWithNullSafetyAndUpcast( + expr: Expression, + dataType: DataType, + nullable: Boolean, + walkedTypePath: WalkedTypePath, + funcForCreatingDeserializer: (Expression, WalkedTypePath) => Expression): Expression = { + val casted = upCastToExpectedType(expr, dataType, walkedTypePath) + expressionWithNullSafety(funcForCreatingDeserializer(casted, walkedTypePath), + nullable, walkedTypePath) + } + + def expressionWithNullSafety( + expr: Expression, + nullable: Boolean, + walkedTypePath: WalkedTypePath): Expression = { + if (nullable) { + expr + } else { + AssertNotNull(expr, walkedTypePath.getPaths) + } + } + + def createDeserializerForTypesSupportValueOf( + path: Expression, + clazz: Class[_]): Expression = { + StaticInvoke( + clazz, + ObjectType(clazz), + "valueOf", + path :: Nil, + returnNullable = false) + } + + def createDeserializerForString(path: Expression, returnNullable: Boolean): Expression = { + Invoke(path, "toString", ObjectType(classOf[java.lang.String]), + returnNullable = returnNullable) + } + + def createDeserializerForSqlDate(path: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + ObjectType(classOf[java.sql.Date]), + "toJavaDate", + path :: Nil, + returnNullable = false) + } + + def createDeserializerForLocalDate(path: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + ObjectType(classOf[java.time.LocalDate]), + "daysToLocalDate", + path :: Nil, + returnNullable = false) + } + + def createDeserializerForInstant(path: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + ObjectType(classOf[java.time.Instant]), + "microsToInstant", + path :: Nil, + returnNullable = false) + } + + def createDeserializerForSqlTimestamp(path: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + ObjectType(classOf[java.sql.Timestamp]), + "toJavaTimestamp", + path :: Nil, + returnNullable = false) + } + + def createDeserializerForJavaBigDecimal( + path: Expression, + returnNullable: Boolean): Expression = { + Invoke(path, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), + returnNullable = returnNullable) + } + + def createDeserializerForScalaBigDecimal( + path: Expression, + returnNullable: Boolean): Expression = { + Invoke(path, "toBigDecimal", ObjectType(classOf[BigDecimal]), returnNullable = returnNullable) + } + + def createDeserializerForJavaBigInteger( + path: Expression, + returnNullable: Boolean): Expression = { + Invoke(path, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger]), + returnNullable = returnNullable) + } + + def createDeserializerForScalaBigInt(path: Expression): Expression = { + Invoke(path, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt]), + returnNullable = false) + } + + /** + * When we build the `deserializer` for an encoder, we set up a lot of "unresolved" stuff + * and lost the required data type, which may lead to runtime error if the real type doesn't + * match the encoder's schema. + * For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type + * is [a: int, b: long], then we will hit runtime error and say that we can't construct class + * `Data` with int and long, because we lost the information that `b` should be a string. + * + * This method help us "remember" the required data type by adding a `UpCast`. Note that we + * only need to do this for leaf nodes. + */ + private def upCastToExpectedType( + expr: Expression, + expected: DataType, + walkedTypePath: WalkedTypePath): Expression = expected match { + case _: StructType => expr + case _: ArrayType => expr + case _: MapType => expr + case _ => UpCast(expr, expected, walkedTypePath.getPaths) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala index 274d75e680f03..bdab407688a65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/InternalRow.scala @@ -125,22 +125,58 @@ object InternalRow { * actually takes a `SpecializedGetters` input because it can be generalized to other classes * that implements `SpecializedGetters` (e.g., `ArrayData`) too. */ - def getAccessor(dataType: DataType): (SpecializedGetters, Int) => Any = dataType match { - case BooleanType => (input, ordinal) => input.getBoolean(ordinal) - case ByteType => (input, ordinal) => input.getByte(ordinal) - case ShortType => (input, ordinal) => input.getShort(ordinal) - case IntegerType | DateType => (input, ordinal) => input.getInt(ordinal) - case LongType | TimestampType => (input, ordinal) => input.getLong(ordinal) - case FloatType => (input, ordinal) => input.getFloat(ordinal) - case DoubleType => (input, ordinal) => input.getDouble(ordinal) - case StringType => (input, ordinal) => input.getUTF8String(ordinal) - case BinaryType => (input, ordinal) => input.getBinary(ordinal) - case CalendarIntervalType => (input, ordinal) => input.getInterval(ordinal) - case t: DecimalType => (input, ordinal) => input.getDecimal(ordinal, t.precision, t.scale) - case t: StructType => (input, ordinal) => input.getStruct(ordinal, t.size) - case _: ArrayType => (input, ordinal) => input.getArray(ordinal) - case _: MapType => (input, ordinal) => input.getMap(ordinal) - case u: UserDefinedType[_] => getAccessor(u.sqlType) - case _ => (input, ordinal) => input.get(ordinal, dataType) + def getAccessor(dt: DataType, nullable: Boolean = true): (SpecializedGetters, Int) => Any = { + val getValueNullSafe: (SpecializedGetters, Int) => Any = dt match { + case BooleanType => (input, ordinal) => input.getBoolean(ordinal) + case ByteType => (input, ordinal) => input.getByte(ordinal) + case ShortType => (input, ordinal) => input.getShort(ordinal) + case IntegerType | DateType => (input, ordinal) => input.getInt(ordinal) + case LongType | TimestampType => (input, ordinal) => input.getLong(ordinal) + case FloatType => (input, ordinal) => input.getFloat(ordinal) + case DoubleType => (input, ordinal) => input.getDouble(ordinal) + case StringType => (input, ordinal) => input.getUTF8String(ordinal) + case BinaryType => (input, ordinal) => input.getBinary(ordinal) + case CalendarIntervalType => (input, ordinal) => input.getInterval(ordinal) + case t: DecimalType => (input, ordinal) => input.getDecimal(ordinal, t.precision, t.scale) + case t: StructType => (input, ordinal) => input.getStruct(ordinal, t.size) + case _: ArrayType => (input, ordinal) => input.getArray(ordinal) + case _: MapType => (input, ordinal) => input.getMap(ordinal) + case u: UserDefinedType[_] => getAccessor(u.sqlType, nullable) + case _ => (input, ordinal) => input.get(ordinal, dt) + } + + if (nullable) { + (getter, index) => { + if (getter.isNullAt(index)) { + null + } else { + getValueNullSafe(getter, index) + } + } + } else { + getValueNullSafe + } + } + + /** + * Returns a writer for an `InternalRow` with given data type. + */ + def getWriter(ordinal: Int, dt: DataType): (InternalRow, Any) => Unit = dt match { + case BooleanType => (input, v) => input.setBoolean(ordinal, v.asInstanceOf[Boolean]) + case ByteType => (input, v) => input.setByte(ordinal, v.asInstanceOf[Byte]) + case ShortType => (input, v) => input.setShort(ordinal, v.asInstanceOf[Short]) + case IntegerType | DateType => (input, v) => input.setInt(ordinal, v.asInstanceOf[Int]) + case LongType | TimestampType => (input, v) => input.setLong(ordinal, v.asInstanceOf[Long]) + case FloatType => (input, v) => input.setFloat(ordinal, v.asInstanceOf[Float]) + case DoubleType => (input, v) => input.setDouble(ordinal, v.asInstanceOf[Double]) + case DecimalType.Fixed(precision, _) => + (input, v) => input.setDecimal(ordinal, v.asInstanceOf[Decimal], precision) + case udt: UserDefinedType[_] => getWriter(ordinal, udt.sqlType) + case NullType => (input, _) => input.setNullAt(ordinal) + case StringType => (input, v) => input.update(ordinal, v.asInstanceOf[UTF8String].copy()) + case _: StructType => (input, v) => input.update(ordinal, v.asInstanceOf[InternalRow].copy()) + case _: ArrayType => (input, v) => input.update(ordinal, v.asInstanceOf[ArrayData].copy()) + case _: MapType => (input, v) => input.update(ordinal, v.asInstanceOf[MapData].copy()) + case _ => (input, v) => input.update(ordinal, v) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala index 3ecc137c8cd7f..c5be3efc6371e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/JavaTypeInference.scala @@ -26,12 +26,13 @@ import scala.language.existentials import com.google.common.reflect.TypeToken -import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} +import org.apache.spark.sql.catalyst.DeserializerBuildHelper._ +import org.apache.spark.sql.catalyst.SerializerBuildHelper._ +import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String /** * Type-inference utilities for POJOs and Java collections. @@ -73,10 +74,10 @@ object JavaTypeInference { : (DataType, Boolean) = { typeToken.getRawType match { case c: Class[_] if c.isAnnotationPresent(classOf[SQLUserDefinedType]) => - (c.getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance(), true) + (c.getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance(), true) case c: Class[_] if UDTRegistration.exists(c.getName) => - val udt = UDTRegistration.getUDTFor(c.getName).get.newInstance() + val udt = UDTRegistration.getUDTFor(c.getName).get.getConstructor().newInstance() .asInstanceOf[UserDefinedType[_ >: Null]] (udt, true) @@ -101,7 +102,9 @@ object JavaTypeInference { case c: Class[_] if c == classOf[java.math.BigDecimal] => (DecimalType.SYSTEM_DEFAULT, true) case c: Class[_] if c == classOf[java.math.BigInteger] => (DecimalType.BigIntDecimal, true) + case c: Class[_] if c == classOf[java.time.LocalDate] => (DateType, true) case c: Class[_] if c == classOf[java.sql.Date] => (DateType, true) + case c: Class[_] if c == classOf[java.time.Instant] => (TimestampType, true) case c: Class[_] if c == classOf[java.sql.Timestamp] => (TimestampType, true) case _ if typeToken.isArray => @@ -187,26 +190,29 @@ object JavaTypeInference { } /** - * Returns an expression that can be used to deserialize an internal row to an object of java bean - * `T` with a compatible schema. Fields of the row will be extracted using UnresolvedAttributes - * of the same name as the constructor arguments. Nested classes will have their fields accessed - * using UnresolvedExtractValue. + * Returns an expression that can be used to deserialize a Spark SQL representation to an object + * of java bean `T` with a compatible schema. The Spark SQL representation is located at ordinal + * 0 of a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed + * using `UnresolvedExtractValue`. */ def deserializerFor(beanClass: Class[_]): Expression = { - deserializerFor(TypeToken.of(beanClass), None) + val typeToken = TypeToken.of(beanClass) + val walkedTypePath = new WalkedTypePath().recordRoot(beanClass.getCanonicalName) + val (dataType, nullable) = inferDataType(typeToken) + + // Assumes we are deserializing the first column of a row. + deserializerForWithNullSafetyAndUpcast(GetColumnByOrdinal(0, dataType), dataType, + nullable = nullable, walkedTypePath, (casted, walkedTypePath) => { + deserializerFor(typeToken, casted, walkedTypePath) + }) } - private def deserializerFor(typeToken: TypeToken[_], path: Option[Expression]): Expression = { - /** Returns the current path with a sub-field extracted. */ - def addToPath(part: String): Expression = path - .map(p => UnresolvedExtractValue(p, expressions.Literal(part))) - .getOrElse(UnresolvedAttribute(part)) - - /** Returns the current path or `GetColumnByOrdinal`. */ - def getPath: Expression = path.getOrElse(GetColumnByOrdinal(0, inferDataType(typeToken)._1)) - + private def deserializerFor( + typeToken: TypeToken[_], + path: Expression, + walkedTypePath: WalkedTypePath): Expression = { typeToken.getRawType match { - case c if !inferExternalType(c).isInstanceOf[ObjectType] => getPath + case c if !inferExternalType(c).isInstanceOf[ObjectType] => path case c if c == classOf[java.lang.Short] || c == classOf[java.lang.Integer] || @@ -215,88 +221,92 @@ object JavaTypeInference { c == classOf[java.lang.Float] || c == classOf[java.lang.Byte] || c == classOf[java.lang.Boolean] => - StaticInvoke( - c, - ObjectType(c), - "valueOf", - getPath :: Nil, - returnNullable = false) + createDeserializerForTypesSupportValueOf(path, c) + + case c if c == classOf[java.time.LocalDate] => + createDeserializerForLocalDate(path) case c if c == classOf[java.sql.Date] => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(c), - "toJavaDate", - getPath :: Nil, - returnNullable = false) + createDeserializerForSqlDate(path) + + case c if c == classOf[java.time.Instant] => + createDeserializerForInstant(path) case c if c == classOf[java.sql.Timestamp] => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(c), - "toJavaTimestamp", - getPath :: Nil, - returnNullable = false) + createDeserializerForSqlTimestamp(path) case c if c == classOf[java.lang.String] => - Invoke(getPath, "toString", ObjectType(classOf[String])) + createDeserializerForString(path, returnNullable = true) case c if c == classOf[java.math.BigDecimal] => - Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal])) + createDeserializerForJavaBigDecimal(path, returnNullable = true) + + case c if c == classOf[java.math.BigInteger] => + createDeserializerForJavaBigInteger(path, returnNullable = true) case c if c.isArray => val elementType = c.getComponentType - val primitiveMethod = elementType match { - case c if c == java.lang.Boolean.TYPE => Some("toBooleanArray") - case c if c == java.lang.Byte.TYPE => Some("toByteArray") - case c if c == java.lang.Short.TYPE => Some("toShortArray") - case c if c == java.lang.Integer.TYPE => Some("toIntArray") - case c if c == java.lang.Long.TYPE => Some("toLongArray") - case c if c == java.lang.Float.TYPE => Some("toFloatArray") - case c if c == java.lang.Double.TYPE => Some("toDoubleArray") - case _ => None + val newTypePath = walkedTypePath.recordArray(elementType.getCanonicalName) + val (dataType, elementNullable) = inferDataType(elementType) + val mapFunction: Expression => Expression = element => { + // upcast the array element to the data type the encoder expected. + deserializerForWithNullSafetyAndUpcast( + element, + dataType, + nullable = elementNullable, + newTypePath, + (casted, typePath) => deserializerFor(typeToken.getComponentType, casted, typePath)) } - primitiveMethod.map { method => - Invoke(getPath, method, ObjectType(c)) - }.getOrElse { - Invoke( - MapObjects( - p => deserializerFor(typeToken.getComponentType, Some(p)), - getPath, - inferDataType(elementType)._1), - "array", - ObjectType(c)) + val arrayData = UnresolvedMapObjects(mapFunction, path) + + val methodName = elementType match { + case c if c == java.lang.Integer.TYPE => "toIntArray" + case c if c == java.lang.Long.TYPE => "toLongArray" + case c if c == java.lang.Double.TYPE => "toDoubleArray" + case c if c == java.lang.Float.TYPE => "toFloatArray" + case c if c == java.lang.Short.TYPE => "toShortArray" + case c if c == java.lang.Byte.TYPE => "toByteArray" + case c if c == java.lang.Boolean.TYPE => "toBooleanArray" + // non-primitive + case _ => "array" } + Invoke(arrayData, methodName, ObjectType(c)) case c if listType.isAssignableFrom(typeToken) => val et = elementType(typeToken) - MapObjects( - p => deserializerFor(et, Some(p)), - getPath, - inferDataType(et)._1, - customCollectionCls = Some(c)) + val newTypePath = walkedTypePath.recordArray(et.getType.getTypeName) + val (dataType, elementNullable) = inferDataType(et) + val mapFunction: Expression => Expression = element => { + // upcast the array element to the data type the encoder expected. + deserializerForWithNullSafetyAndUpcast( + element, + dataType, + nullable = elementNullable, + newTypePath, + (casted, typePath) => deserializerFor(et, casted, typePath)) + } + + UnresolvedMapObjects(mapFunction, path, customCollectionCls = Some(c)) case _ if mapType.isAssignableFrom(typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) - val keyDataType = inferDataType(keyType)._1 - val valueDataType = inferDataType(valueType)._1 + val newTypePath = walkedTypePath.recordMap(keyType.getType.getTypeName, + valueType.getType.getTypeName) val keyData = Invoke( - MapObjects( - p => deserializerFor(keyType, Some(p)), - Invoke(getPath, "keyArray", ArrayType(keyDataType)), - keyDataType), + UnresolvedMapObjects( + p => deserializerFor(keyType, p, newTypePath), + MapKeys(path)), "array", ObjectType(classOf[Array[Any]])) val valueData = Invoke( - MapObjects( - p => deserializerFor(valueType, Some(p)), - Invoke(getPath, "valueArray", ArrayType(valueDataType)), - valueDataType), + UnresolvedMapObjects( + p => deserializerFor(valueType, p, newTypePath), + MapValues(path)), "array", ObjectType(classOf[Array[Any]])) @@ -308,53 +318,45 @@ object JavaTypeInference { returnNullable = false) case other if other.isEnum => - StaticInvoke( - other, - ObjectType(other), - "valueOf", - Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) :: Nil, - returnNullable = false) + createDeserializerForTypesSupportValueOf( + createDeserializerForString(path, returnNullable = false), + other) case other => val properties = getJavaBeanReadableAndWritableProperties(other) val setters = properties.map { p => val fieldName = p.getName val fieldType = typeToken.method(p.getReadMethod).getReturnType - val (_, nullable) = inferDataType(fieldType) - val constructor = deserializerFor(fieldType, Some(addToPath(fieldName))) - val setter = if (nullable) { - constructor - } else { - AssertNotNull(constructor, Seq("currently no type path record in java")) - } + val (dataType, nullable) = inferDataType(fieldType) + val newTypePath = walkedTypePath.recordField(fieldType.getType.getTypeName, fieldName) + val setter = expressionWithNullSafety( + deserializerFor(fieldType, addToPath(path, fieldName, dataType, newTypePath), + newTypePath), + nullable = nullable, + newTypePath) p.getWriteMethod.getName -> setter }.toMap val newInstance = NewInstance(other, Nil, ObjectType(other), propagateNull = false) val result = InitializeJavaBean(newInstance, setters) - if (path.nonEmpty) { - expressions.If( - IsNull(getPath), - expressions.Literal.create(null, ObjectType(other)), - result - ) - } else { + expressions.If( + IsNull(path), + expressions.Literal.create(null, ObjectType(other)), result - } + ) } } /** - * Returns an expression for serializing an object of the given type to an internal row. + * Returns an expression for serializing an object of the given type to a Spark SQL + * representation. The input object is located at ordinal 0 of a row, i.e., + * `BoundReference(0, _)`. */ - def serializerFor(beanClass: Class[_]): CreateNamedStruct = { + def serializerFor(beanClass: Class[_]): Expression = { val inputObject = BoundReference(0, ObjectType(beanClass), nullable = true) val nullSafeInput = AssertNotNull(inputObject, Seq("top level input bean")) - serializerFor(nullSafeInput, TypeToken.of(beanClass)) match { - case expressions.If(_, _, s: CreateNamedStruct) => s - case other => CreateNamedStruct(expressions.Literal("value") :: other :: Nil) - } + serializerFor(nullSafeInput, TypeToken.of(beanClass)) } private def serializerFor(inputObject: Expression, typeToken: TypeToken[_]): Expression = { @@ -362,12 +364,15 @@ object JavaTypeInference { def toCatalystArray(input: Expression, elementType: TypeToken[_]): Expression = { val (dataType, nullable) = inferDataType(elementType) if (ScalaReflection.isNativeType(dataType)) { - NewInstance( - classOf[GenericArrayData], - input :: Nil, - dataType = ArrayType(dataType, nullable)) + val cls = input.dataType.asInstanceOf[ObjectType].cls + if (cls.isArray && cls.getComponentType.isPrimitive) { + createSerializerForPrimitiveArray(input, dataType) + } else { + createSerializerForGenericArray(input, dataType, nullable = nullable) + } } else { - MapObjects(serializerFor(_, elementType), input, ObjectType(elementType.getRawType)) + createSerializerForMapObjects(input, ObjectType(elementType.getRawType), + serializerFor(_, elementType)) } } @@ -375,52 +380,26 @@ object JavaTypeInference { inputObject } else { typeToken.getRawType match { - case c if c == classOf[String] => - StaticInvoke( - classOf[UTF8String], - StringType, - "fromString", - inputObject :: Nil, - returnNullable = false) - - case c if c == classOf[java.sql.Timestamp] => - StaticInvoke( - DateTimeUtils.getClass, - TimestampType, - "fromJavaTimestamp", - inputObject :: Nil, - returnNullable = false) - - case c if c == classOf[java.sql.Date] => - StaticInvoke( - DateTimeUtils.getClass, - DateType, - "fromJavaDate", - inputObject :: Nil, - returnNullable = false) + case c if c == classOf[String] => createSerializerForString(inputObject) + + case c if c == classOf[java.time.Instant] => createSerializerForJavaInstant(inputObject) + + case c if c == classOf[java.sql.Timestamp] => createSerializerForSqlTimestamp(inputObject) + + case c if c == classOf[java.time.LocalDate] => createSerializerForJavaLocalDate(inputObject) + + case c if c == classOf[java.sql.Date] => createSerializerForSqlDate(inputObject) case c if c == classOf[java.math.BigDecimal] => - StaticInvoke( - Decimal.getClass, - DecimalType.SYSTEM_DEFAULT, - "apply", - inputObject :: Nil, - returnNullable = false) - - case c if c == classOf[java.lang.Boolean] => - Invoke(inputObject, "booleanValue", BooleanType) - case c if c == classOf[java.lang.Byte] => - Invoke(inputObject, "byteValue", ByteType) - case c if c == classOf[java.lang.Short] => - Invoke(inputObject, "shortValue", ShortType) - case c if c == classOf[java.lang.Integer] => - Invoke(inputObject, "intValue", IntegerType) - case c if c == classOf[java.lang.Long] => - Invoke(inputObject, "longValue", LongType) - case c if c == classOf[java.lang.Float] => - Invoke(inputObject, "floatValue", FloatType) - case c if c == classOf[java.lang.Double] => - Invoke(inputObject, "doubleValue", DoubleType) + createSerializerForJavaBigDecimal(inputObject) + + case c if c == classOf[java.lang.Boolean] => createSerializerForBoolean(inputObject) + case c if c == classOf[java.lang.Byte] => createSerializerForByte(inputObject) + case c if c == classOf[java.lang.Short] => createSerializerForShort(inputObject) + case c if c == classOf[java.lang.Integer] => createSerializerForInteger(inputObject) + case c if c == classOf[java.lang.Long] => createSerializerForLong(inputObject) + case c if c == classOf[java.lang.Float] => createSerializerForFloat(inputObject) + case c if c == classOf[java.lang.Double] => createSerializerForDouble(inputObject) case _ if typeToken.isArray => toCatalystArray(inputObject, typeToken.getComponentType) @@ -431,38 +410,34 @@ object JavaTypeInference { case _ if mapType.isAssignableFrom(typeToken) => val (keyType, valueType) = mapKeyValueType(typeToken) - ExternalMapToCatalyst( + createSerializerForMap( inputObject, - ObjectType(keyType.getRawType), - serializerFor(_, keyType), - keyNullable = true, - ObjectType(valueType.getRawType), - serializerFor(_, valueType), - valueNullable = true + MapElementInformation( + ObjectType(keyType.getRawType), + nullable = true, + serializerFor(_, keyType)), + MapElementInformation( + ObjectType(valueType.getRawType), + nullable = true, + serializerFor(_, valueType)) ) case other if other.isEnum => - StaticInvoke( - classOf[UTF8String], - StringType, - "fromString", - Invoke(inputObject, "name", ObjectType(classOf[String]), returnNullable = false) :: Nil, - returnNullable = false) + createSerializerForString( + Invoke(inputObject, "name", ObjectType(classOf[String]), returnNullable = false)) case other => val properties = getJavaBeanReadableAndWritableProperties(other) - val nonNullOutput = CreateNamedStruct(properties.flatMap { p => + val fields = properties.map { p => val fieldName = p.getName val fieldType = typeToken.method(p.getReadMethod).getReturnType val fieldValue = Invoke( inputObject, p.getReadMethod.getName, inferExternalType(fieldType.getRawType)) - expressions.Literal(fieldName) :: serializerFor(fieldValue, fieldType) :: Nil - }) - - val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) - expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + (fieldName, serializerFor(fieldValue, fieldType)) + } + createSerializerForObject(inputObject, fields) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala new file mode 100644 index 0000000000000..cd75407c7ee7a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/QueryPlanningTracker.scala @@ -0,0 +1,163 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import scala.collection.JavaConverters._ + +import org.apache.spark.util.BoundedPriorityQueue + + +/** + * A simple utility for tracking runtime and associated stats in query planning. + * + * There are two separate concepts we track: + * + * 1. Phases: These are broad scope phases in query planning, as listed below, i.e. analysis, + * optimizationm and physical planning (just planning). + * + * 2. Rules: These are the individual Catalyst rules that we track. In addition to time, we also + * track the number of invocations and effective invocations. + */ +object QueryPlanningTracker { + + // Define a list of common phases here. + val PARSING = "parsing" + val ANALYSIS = "analysis" + val OPTIMIZATION = "optimization" + val PLANNING = "planning" + + /** + * Summary for a rule. + * @param totalTimeNs total amount of time, in nanosecs, spent in this rule. + * @param numInvocations number of times the rule has been invoked. + * @param numEffectiveInvocations number of times the rule has been invoked and + * resulted in a plan change. + */ + class RuleSummary( + var totalTimeNs: Long, var numInvocations: Long, var numEffectiveInvocations: Long) { + + def this() = this(totalTimeNs = 0, numInvocations = 0, numEffectiveInvocations = 0) + + override def toString: String = { + s"RuleSummary($totalTimeNs, $numInvocations, $numEffectiveInvocations)" + } + } + + /** + * Summary of a phase, with start time and end time so we can construct a timeline. + */ + class PhaseSummary(val startTimeMs: Long, val endTimeMs: Long) { + + def durationMs: Long = endTimeMs - startTimeMs + + override def toString: String = { + s"PhaseSummary($startTimeMs, $endTimeMs)" + } + } + + /** + * A thread local variable to implicitly pass the tracker around. This assumes the query planner + * is single-threaded, and avoids passing the same tracker context in every function call. + */ + private val localTracker = new ThreadLocal[QueryPlanningTracker]() { + override def initialValue: QueryPlanningTracker = null + } + + /** Returns the current tracker in scope, based on the thread local variable. */ + def get: Option[QueryPlanningTracker] = Option(localTracker.get()) + + /** Sets the current tracker for the execution of function f. We assume f is single-threaded. */ + def withTracker[T](tracker: QueryPlanningTracker)(f: => T): T = { + val originalTracker = localTracker.get() + localTracker.set(tracker) + try f finally { localTracker.set(originalTracker) } + } +} + + +class QueryPlanningTracker { + + import QueryPlanningTracker._ + + // Mapping from the name of a rule to a rule's summary. + // Use a Java HashMap for less overhead. + private val rulesMap = new java.util.HashMap[String, RuleSummary] + + // From a phase to its start time and end time, in ms. + private val phasesMap = new java.util.HashMap[String, PhaseSummary] + + /** + * Measure the start and end time of a phase. Note that if this function is called multiple + * times for the same phase, the recorded start time will be the start time of the first call, + * and the recorded end time will be the end time of the last call. + */ + def measurePhase[T](phase: String)(f: => T): T = { + val startTime = System.currentTimeMillis() + val ret = f + val endTime = System.currentTimeMillis + + if (phasesMap.containsKey(phase)) { + val oldSummary = phasesMap.get(phase) + phasesMap.put(phase, new PhaseSummary(oldSummary.startTimeMs, endTime)) + } else { + phasesMap.put(phase, new PhaseSummary(startTime, endTime)) + } + ret + } + + /** + * Record a specific invocation of a rule. + * + * @param rule name of the rule + * @param timeNs time taken to run this invocation + * @param effective whether the invocation has resulted in a plan change + */ + def recordRuleInvocation(rule: String, timeNs: Long, effective: Boolean): Unit = { + var s = rulesMap.get(rule) + if (s eq null) { + s = new RuleSummary + rulesMap.put(rule, s) + } + + s.totalTimeNs += timeNs + s.numInvocations += 1 + s.numEffectiveInvocations += (if (effective) 1 else 0) + } + + // ------------ reporting functions below ------------ + + def rules: Map[String, RuleSummary] = rulesMap.asScala.toMap + + def phases: Map[String, PhaseSummary] = phasesMap.asScala.toMap + + /** + * Returns the top k most expensive rules (as measured by time). If k is larger than the rules + * seen so far, return all the rules. If there is no rule seen so far or k <= 0, return empty seq. + */ + def topRulesByTime(k: Int): Seq[(String, RuleSummary)] = { + if (k <= 0) { + Seq.empty + } else { + val orderingByTime: Ordering[(String, RuleSummary)] = Ordering.by(e => e._2.totalTimeNs) + val q = new BoundedPriorityQueue(k)(orderingByTime) + rulesMap.asScala.foreach(q.+=) + q.toSeq.sortBy(r => -r._2.totalTimeNs) + } + } + +} 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 0238d57de2446..2a4e84241a941 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,14 +17,15 @@ package org.apache.spark.sql.catalyst -import java.lang.reflect.Constructor - import org.apache.commons.lang3.reflect.ConstructorUtils -import org.apache.spark.sql.catalyst.analysis.{GetColumnByOrdinal, UnresolvedAttribute, UnresolvedExtractValue} -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.DeserializerBuildHelper._ +import org.apache.spark.sql.catalyst.SerializerBuildHelper._ +import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal +import org.apache.spark.sql.catalyst.expressions.{Expression, _} import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData, MapData} +import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -37,6 +38,9 @@ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} trait DefinedByConstructorParams +private[catalyst] object ScalaSubtypeLock + + /** * A default version of ScalaReflection that uses the runtime universe. */ @@ -65,19 +69,32 @@ object ScalaReflection extends ScalaReflection { */ def dataTypeFor[T : TypeTag]: DataType = dataTypeFor(localTypeOf[T]) + /** + * Synchronize to prevent concurrent usage of `<:<` operator. + * This operator is not thread safe in any current version of scala; i.e. + * (2.11.12, 2.12.8, 2.13.0-M5). + * + * See https://github.com/scala/bug/issues/10766 + */ + private[catalyst] def isSubtype(tpe1: `Type`, tpe2: `Type`): Boolean = { + ScalaSubtypeLock.synchronized { + tpe1 <:< tpe2 + } + } + private def dataTypeFor(tpe: `Type`): DataType = cleanUpReflectionObjects { tpe.dealias match { - case t if t <:< definitions.NullTpe => NullType - case t if t <:< definitions.IntTpe => IntegerType - case t if t <:< definitions.LongTpe => LongType - case t if t <:< definitions.DoubleTpe => DoubleType - case t if t <:< definitions.FloatTpe => FloatType - case t if t <:< definitions.ShortTpe => ShortType - case t if t <:< definitions.ByteTpe => ByteType - case t if t <:< definitions.BooleanTpe => BooleanType - case t if t <:< localTypeOf[Array[Byte]] => BinaryType - case t if t <:< localTypeOf[CalendarInterval] => CalendarIntervalType - case t if t <:< localTypeOf[Decimal] => DecimalType.SYSTEM_DEFAULT + case t if isSubtype(t, definitions.NullTpe) => NullType + case t if isSubtype(t, definitions.IntTpe) => IntegerType + case t if isSubtype(t, definitions.LongTpe) => LongType + case t if isSubtype(t, definitions.DoubleTpe) => DoubleType + case t if isSubtype(t, definitions.FloatTpe) => FloatType + case t if isSubtype(t, definitions.ShortTpe) => ShortType + case t if isSubtype(t, definitions.ByteTpe) => ByteType + case t if isSubtype(t, definitions.BooleanTpe) => BooleanType + case t if isSubtype(t, localTypeOf[Array[Byte]]) => BinaryType + case t if isSubtype(t, localTypeOf[CalendarInterval]) => CalendarIntervalType + case t if isSubtype(t, localTypeOf[Decimal]) => DecimalType.SYSTEM_DEFAULT case _ => val className = getClassNameFromType(tpe) className match { @@ -100,13 +117,13 @@ object ScalaReflection extends ScalaReflection { */ private def arrayClassFor(tpe: `Type`): ObjectType = cleanUpReflectionObjects { val cls = tpe.dealias match { - case t if t <:< definitions.IntTpe => classOf[Array[Int]] - case t if t <:< definitions.LongTpe => classOf[Array[Long]] - case t if t <:< definitions.DoubleTpe => classOf[Array[Double]] - case t if t <:< definitions.FloatTpe => classOf[Array[Float]] - case t if t <:< definitions.ShortTpe => classOf[Array[Short]] - case t if t <:< definitions.ByteTpe => classOf[Array[Byte]] - case t if t <:< definitions.BooleanTpe => classOf[Array[Boolean]] + case t if isSubtype(t, definitions.IntTpe) => classOf[Array[Int]] + case t if isSubtype(t, definitions.LongTpe) => classOf[Array[Long]] + case t if isSubtype(t, definitions.DoubleTpe) => classOf[Array[Double]] + case t if isSubtype(t, definitions.FloatTpe) => classOf[Array[Float]] + case t if isSubtype(t, definitions.ShortTpe) => classOf[Array[Short]] + case t if isSubtype(t, definitions.ByteTpe) => classOf[Array[Byte]] + case t if isSubtype(t, definitions.BooleanTpe) => classOf[Array[Boolean]] case other => // There is probably a better way to do this, but I couldn't find it... val elementType = dataTypeFor(other).asInstanceOf[ObjectType].cls @@ -125,256 +142,200 @@ object ScalaReflection extends ScalaReflection { case _ => false } + private def baseType(tpe: `Type`): `Type` = { + tpe.dealias match { + case annotatedType: AnnotatedType => annotatedType.underlying + case other => other + } + } + /** - * Returns an expression that can be used to deserialize an input row to an object of type `T` - * with a compatible schema. Fields of the row will be extracted using UnresolvedAttributes - * of the same name as the constructor arguments. Nested classes will have their fields accessed - * using UnresolvedExtractValue. + * Returns an expression that can be used to deserialize a Spark SQL representation to an object + * of type `T` with a compatible schema. The Spark SQL representation is located at ordinal 0 of + * a row, i.e., `GetColumnByOrdinal(0, _)`. Nested classes will have their fields accessed using + * `UnresolvedExtractValue`. * - * When used on a primitive type, the constructor will instead default to extracting the value - * from ordinal 0 (since there are no names to map to). The actual location can be moved by - * calling resolve/bind with a new schema. + * The returned expression is used by `ExpressionEncoder`. The encoder will resolve and bind this + * deserializer expression when using it. */ - def deserializerFor[T : TypeTag]: Expression = { - val tpe = localTypeOf[T] + def deserializerForType(tpe: `Type`): Expression = { val clsName = getClassNameFromType(tpe) - val walkedTypePath = s"""- root class: "$clsName"""" :: Nil - val expr = deserializerFor(tpe, None, walkedTypePath) - val Schema(_, nullable) = schemaFor(tpe) - if (nullable) { - expr - } else { - AssertNotNull(expr, walkedTypePath) - } + val walkedTypePath = new WalkedTypePath().recordRoot(clsName) + val Schema(dataType, nullable) = schemaFor(tpe) + + // Assumes we are deserializing the first column of a row. + deserializerForWithNullSafetyAndUpcast(GetColumnByOrdinal(0, dataType), dataType, + nullable = nullable, walkedTypePath, + (casted, typePath) => deserializerFor(tpe, casted, typePath)) } + /** + * Returns an expression that can be used to deserialize an input expression to an object of type + * `T` with a compatible schema. + * + * @param tpe The `Type` of deserialized object. + * @param path The expression which can be used to extract serialized value. + * @param walkedTypePath The paths from top to bottom to access current field when deserializing. + */ private def deserializerFor( tpe: `Type`, - path: Option[Expression], - walkedTypePath: Seq[String]): Expression = cleanUpReflectionObjects { - - /** Returns the current path with a sub-field extracted. */ - def addToPath(part: String, dataType: DataType, walkedTypePath: Seq[String]): Expression = { - val newPath = path - .map(p => UnresolvedExtractValue(p, expressions.Literal(part))) - .getOrElse(UnresolvedAttribute.quoted(part)) - upCastToExpectedType(newPath, dataType, walkedTypePath) - } + path: Expression, + walkedTypePath: WalkedTypePath): Expression = cleanUpReflectionObjects { + baseType(tpe) match { + case t if !dataTypeFor(t).isInstanceOf[ObjectType] => path - /** Returns the current path with a field at ordinal extracted. */ - def addToPathOrdinal( - ordinal: Int, - dataType: DataType, - walkedTypePath: Seq[String]): Expression = { - val newPath = path - .map(p => GetStructField(p, ordinal)) - .getOrElse(GetColumnByOrdinal(ordinal, dataType)) - upCastToExpectedType(newPath, dataType, walkedTypePath) - } + case t if isSubtype(t, localTypeOf[Option[_]]) => + val TypeRef(_, _, Seq(optType)) = t + val className = getClassNameFromType(optType) + val newTypePath = walkedTypePath.recordOption(className) + WrapOption(deserializerFor(optType, path, newTypePath), dataTypeFor(optType)) - /** Returns the current path or `GetColumnByOrdinal`. */ - def getPath: Expression = { - val dataType = schemaFor(tpe).dataType - if (path.isDefined) { - path.get - } else { - upCastToExpectedType(GetColumnByOrdinal(0, dataType), dataType, walkedTypePath) - } - } + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Integer]) - /** - * When we build the `deserializer` for an encoder, we set up a lot of "unresolved" stuff - * and lost the required data type, which may lead to runtime error if the real type doesn't - * match the encoder's schema. - * For example, we build an encoder for `case class Data(a: Int, b: String)` and the real type - * is [a: int, b: long], then we will hit runtime error and say that we can't construct class - * `Data` with int and long, because we lost the information that `b` should be a string. - * - * This method help us "remember" the required data type by adding a `UpCast`. Note that we - * only need to do this for leaf nodes. - */ - def upCastToExpectedType( - expr: Expression, - expected: DataType, - walkedTypePath: Seq[String]): Expression = expected match { - case _: StructType => expr - case _: ArrayType => expr - // TODO: ideally we should also skip MapType, but nested StructType inside MapType is rare and - // it's not trivial to support by-name resolution for StructType inside MapType. - case _ => UpCast(expr, expected, walkedTypePath) - } + case t if isSubtype(t, localTypeOf[java.lang.Long]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Long]) - tpe.dealias match { - case t if !dataTypeFor(t).isInstanceOf[ObjectType] => getPath + case t if isSubtype(t, localTypeOf[java.lang.Double]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Double]) - case t if t <:< localTypeOf[Option[_]] => - val TypeRef(_, _, Seq(optType)) = t - val className = getClassNameFromType(optType) - val newTypePath = s"""- option value class: "$className"""" +: walkedTypePath - WrapOption(deserializerFor(optType, path, newTypePath), dataTypeFor(optType)) + case t if isSubtype(t, localTypeOf[java.lang.Float]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Float]) + + case t if isSubtype(t, localTypeOf[java.lang.Short]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Short]) + + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Byte]) + + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => + createDeserializerForTypesSupportValueOf(path, + classOf[java.lang.Boolean]) + + case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => + createDeserializerForLocalDate(path) + + case t if isSubtype(t, localTypeOf[java.sql.Date]) => + createDeserializerForSqlDate(path) + + case t if isSubtype(t, localTypeOf[java.time.Instant]) => + createDeserializerForInstant(path) - case t if t <:< localTypeOf[java.lang.Integer] => - val boxedType = classOf[java.lang.Integer] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Long] => - val boxedType = classOf[java.lang.Long] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Double] => - val boxedType = classOf[java.lang.Double] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Float] => - val boxedType = classOf[java.lang.Float] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Short] => - val boxedType = classOf[java.lang.Short] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Byte] => - val boxedType = classOf[java.lang.Byte] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Boolean] => - val boxedType = classOf[java.lang.Boolean] - val objectType = ObjectType(boxedType) - StaticInvoke(boxedType, objectType, "valueOf", getPath :: Nil, returnNullable = false) - - case t if t <:< localTypeOf[java.sql.Date] => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(classOf[java.sql.Date]), - "toJavaDate", - getPath :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.sql.Timestamp] => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(classOf[java.sql.Timestamp]), - "toJavaTimestamp", - getPath :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.lang.String] => - Invoke(getPath, "toString", ObjectType(classOf[String]), returnNullable = false) - - case t if t <:< localTypeOf[java.math.BigDecimal] => - Invoke(getPath, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), - returnNullable = false) - - case t if t <:< localTypeOf[BigDecimal] => - Invoke(getPath, "toBigDecimal", ObjectType(classOf[BigDecimal]), returnNullable = false) - - case t if t <:< localTypeOf[java.math.BigInteger] => - Invoke(getPath, "toJavaBigInteger", ObjectType(classOf[java.math.BigInteger]), - returnNullable = false) - - case t if t <:< localTypeOf[scala.math.BigInt] => - Invoke(getPath, "toScalaBigInt", ObjectType(classOf[scala.math.BigInt]), - returnNullable = false) - - case t if t <:< localTypeOf[Array[_]] => + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => + createDeserializerForSqlTimestamp(path) + + case t if isSubtype(t, localTypeOf[java.lang.String]) => + createDeserializerForString(path, returnNullable = false) + + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => + createDeserializerForJavaBigDecimal(path, returnNullable = false) + + case t if isSubtype(t, localTypeOf[BigDecimal]) => + createDeserializerForScalaBigDecimal(path, returnNullable = false) + + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => + createDeserializerForJavaBigInteger(path, returnNullable = false) + + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => + createDeserializerForScalaBigInt(path) + + case t if isSubtype(t, localTypeOf[Array[_]]) => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, elementNullable) = schemaFor(elementType) val className = getClassNameFromType(elementType) - val newTypePath = s"""- array element class: "$className"""" +: walkedTypePath + val newTypePath = walkedTypePath.recordArray(className) val mapFunction: Expression => Expression = element => { // upcast the array element to the data type the encoder expected. - val casted = upCastToExpectedType(element, dataType, newTypePath) - val converter = deserializerFor(elementType, Some(casted), newTypePath) - if (elementNullable) { - converter - } else { - AssertNotNull(converter, newTypePath) - } + deserializerForWithNullSafetyAndUpcast( + element, + dataType, + nullable = elementNullable, + newTypePath, + (casted, typePath) => deserializerFor(elementType, casted, typePath)) } - val arrayData = UnresolvedMapObjects(mapFunction, getPath) + val arrayData = UnresolvedMapObjects(mapFunction, path) val arrayCls = arrayClassFor(elementType) - if (elementNullable) { - Invoke(arrayData, "array", arrayCls, returnNullable = false) - } else { - val primitiveMethod = elementType match { - case t if t <:< definitions.IntTpe => "toIntArray" - case t if t <:< definitions.LongTpe => "toLongArray" - case t if t <:< definitions.DoubleTpe => "toDoubleArray" - case t if t <:< definitions.FloatTpe => "toFloatArray" - case t if t <:< definitions.ShortTpe => "toShortArray" - case t if t <:< definitions.ByteTpe => "toByteArray" - case t if t <:< definitions.BooleanTpe => "toBooleanArray" - case other => throw new IllegalStateException("expect primitive array element type " + - "but got " + other) - } - Invoke(arrayData, primitiveMethod, arrayCls, returnNullable = false) + val methodName = elementType match { + case t if isSubtype(t, definitions.IntTpe) => "toIntArray" + case t if isSubtype(t, definitions.LongTpe) => "toLongArray" + case t if isSubtype(t, definitions.DoubleTpe) => "toDoubleArray" + case t if isSubtype(t, definitions.FloatTpe) => "toFloatArray" + case t if isSubtype(t, definitions.ShortTpe) => "toShortArray" + case t if isSubtype(t, definitions.ByteTpe) => "toByteArray" + case t if isSubtype(t, definitions.BooleanTpe) => "toBooleanArray" + // non-primitive + case _ => "array" } + Invoke(arrayData, methodName, arrayCls, returnNullable = false) // We serialize a `Set` to Catalyst array. When we deserialize a Catalyst array // to a `Set`, if there are duplicated elements, the elements will be de-duplicated. - case t if t <:< localTypeOf[Seq[_]] || - t <:< localTypeOf[scala.collection.Set[_]] => + case t if isSubtype(t, localTypeOf[Seq[_]]) || + isSubtype(t, localTypeOf[scala.collection.Set[_]]) => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, elementNullable) = schemaFor(elementType) val className = getClassNameFromType(elementType) - val newTypePath = s"""- array element class: "$className"""" +: walkedTypePath + val newTypePath = walkedTypePath.recordArray(className) val mapFunction: Expression => Expression = element => { - // upcast the array element to the data type the encoder expected. - val casted = upCastToExpectedType(element, dataType, newTypePath) - val converter = deserializerFor(elementType, Some(casted), newTypePath) - if (elementNullable) { - converter - } else { - AssertNotNull(converter, newTypePath) - } + deserializerForWithNullSafetyAndUpcast( + element, + dataType, + nullable = elementNullable, + newTypePath, + (casted, typePath) => deserializerFor(elementType, casted, typePath)) } val companion = t.dealias.typeSymbol.companion.typeSignature val cls = companion.member(TermName("newBuilder")) match { - case NoSymbol if t <:< localTypeOf[Seq[_]] => classOf[Seq[_]] - case NoSymbol if t <:< localTypeOf[scala.collection.Set[_]] => + case NoSymbol if isSubtype(t, localTypeOf[Seq[_]]) => classOf[Seq[_]] + case NoSymbol if isSubtype(t, localTypeOf[scala.collection.Set[_]]) => classOf[scala.collection.Set[_]] case _ => mirror.runtimeClass(t.typeSymbol.asClass) } - UnresolvedMapObjects(mapFunction, getPath, Some(cls)) + UnresolvedMapObjects(mapFunction, path, Some(cls)) - case t if t <:< localTypeOf[Map[_, _]] => - // TODO: add walked type path for map + case t if isSubtype(t, localTypeOf[Map[_, _]]) => val TypeRef(_, _, Seq(keyType, valueType)) = t - CatalystToExternalMap( - p => deserializerFor(keyType, Some(p), walkedTypePath), - p => deserializerFor(valueType, Some(p), walkedTypePath), - getPath, + val classNameForKey = getClassNameFromType(keyType) + val classNameForValue = getClassNameFromType(valueType) + + val newTypePath = walkedTypePath.recordMap(classNameForKey, classNameForValue) + + UnresolvedCatalystToExternalMap( + path, + p => deserializerFor(keyType, p, newTypePath), + p => deserializerFor(valueType, p, newTypePath), mirror.runtimeClass(t.typeSymbol.asClass) ) case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() val obj = NewInstance( udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), Nil, dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] val obj = NewInstance( udt.getClass, Nil, dataType = ObjectType(udt.getClass)) - Invoke(obj, "deserialize", ObjectType(udt.userClass), getPath :: Nil) + Invoke(obj, "deserialize", ObjectType(udt.userClass), path :: Nil) case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) @@ -384,43 +345,39 @@ object ScalaReflection extends ScalaReflection { val arguments = params.zipWithIndex.map { case ((fieldName, fieldType), i) => val Schema(dataType, nullable) = schemaFor(fieldType) val clsName = getClassNameFromType(fieldType) - val newTypePath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath + val newTypePath = walkedTypePath.recordField(clsName, fieldName) + // For tuples, we based grab the inner fields by ordinal instead of name. - val constructor = if (cls.getName startsWith "scala.Tuple") { + val newPath = if (cls.getName startsWith "scala.Tuple") { deserializerFor( fieldType, - Some(addToPathOrdinal(i, dataType, newTypePath)), + addToPathOrdinal(path, i, dataType, newTypePath), newTypePath) } else { deserializerFor( fieldType, - Some(addToPath(fieldName, dataType, newTypePath)), + addToPath(path, fieldName, dataType, newTypePath), newTypePath) } - - if (!nullable) { - AssertNotNull(constructor, newTypePath) - } else { - constructor - } + expressionWithNullSafety( + newPath, + nullable = nullable, + newTypePath) } val newInstance = NewInstance(cls, arguments, ObjectType(cls), propagateNull = false) - if (path.nonEmpty) { - expressions.If( - IsNull(getPath), - expressions.Literal.create(null, ObjectType(cls)), - newInstance - ) - } else { + expressions.If( + IsNull(path), + expressions.Literal.create(null, ObjectType(cls)), newInstance - } + ) } } /** - * Returns an expression for serializing an object of type T to an internal row. + * Returns an expression for serializing an object of type T to Spark SQL representation. The + * input object is located at ordinal 0 of a row, i.e., `BoundReference(0, _)`. * * If the given type is not supported, i.e. there is no encoder can be built for this type, * an [[UnsupportedOperationException]] will be thrown with detailed error message to explain @@ -431,93 +388,89 @@ object ScalaReflection extends ScalaReflection { * * the element type of [[Array]] or [[Seq]]: `array element class: "abc.xyz.MyClass"` * * the field of [[Product]]: `field (class: "abc.xyz.MyClass", name: "myField")` */ - def serializerFor[T : TypeTag](inputObject: Expression): CreateNamedStruct = { - val tpe = localTypeOf[T] + def serializerForType(tpe: `Type`): Expression = ScalaReflection.cleanUpReflectionObjects { val clsName = getClassNameFromType(tpe) - val walkedTypePath = s"""- root class: "$clsName"""" :: Nil - serializerFor(inputObject, tpe, walkedTypePath) match { - case expressions.If(_, _, s: CreateNamedStruct) if definedByConstructorParams(tpe) => s - case other => CreateNamedStruct(expressions.Literal("value") :: other :: Nil) - } + val walkedTypePath = new WalkedTypePath().recordRoot(clsName) + + // The input object to `ExpressionEncoder` is located at first column of an row. + val isPrimitive = tpe.typeSymbol.asClass.isPrimitive + val inputObject = BoundReference(0, dataTypeFor(tpe), nullable = !isPrimitive) + + serializerFor(inputObject, tpe, walkedTypePath) } - /** Helper for extracting internal fields from a case class. */ + /** + * Returns an expression for serializing the value of an input expression into Spark SQL + * internal representation. + */ private def serializerFor( inputObject: Expression, tpe: `Type`, - walkedTypePath: Seq[String], + walkedTypePath: WalkedTypePath, seenTypeSet: Set[`Type`] = Set.empty): Expression = cleanUpReflectionObjects { def toCatalystArray(input: Expression, elementType: `Type`): Expression = { dataTypeFor(elementType) match { case dt: ObjectType => val clsName = getClassNameFromType(elementType) - val newPath = s"""- array element class: "$clsName"""" +: walkedTypePath - MapObjects(serializerFor(_, elementType, newPath, seenTypeSet), input, dt) + val newPath = walkedTypePath.recordArray(clsName) + createSerializerForMapObjects(input, dt, + serializerFor(_, elementType, newPath, seenTypeSet)) - case dt @ (BooleanType | ByteType | ShortType | IntegerType | LongType | - FloatType | DoubleType) => + case dt @ (BooleanType | ByteType | ShortType | IntegerType | LongType | + FloatType | DoubleType) => val cls = input.dataType.asInstanceOf[ObjectType].cls if (cls.isArray && cls.getComponentType.isPrimitive) { - StaticInvoke( - classOf[UnsafeArrayData], - ArrayType(dt, false), - "fromPrimitiveArray", - input :: Nil, - returnNullable = false) + createSerializerForPrimitiveArray(input, dt) } else { - NewInstance( - classOf[GenericArrayData], - input :: Nil, - dataType = ArrayType(dt, schemaFor(elementType).nullable)) + createSerializerForGenericArray(input, dt, nullable = schemaFor(elementType).nullable) } case dt => - NewInstance( - classOf[GenericArrayData], - input :: Nil, - dataType = ArrayType(dt, schemaFor(elementType).nullable)) + createSerializerForGenericArray(input, dt, nullable = schemaFor(elementType).nullable) } } - - tpe.dealias match { + baseType(tpe) match { case _ if !inputObject.dataType.isInstanceOf[ObjectType] => inputObject - case t if t <:< localTypeOf[Option[_]] => + case t if isSubtype(t, localTypeOf[Option[_]]) => val TypeRef(_, _, Seq(optType)) = t val className = getClassNameFromType(optType) - val newPath = s"""- option value class: "$className"""" +: walkedTypePath + val newPath = walkedTypePath.recordOption(className) val unwrapped = UnwrapOption(dataTypeFor(optType), inputObject) serializerFor(unwrapped, optType, newPath, seenTypeSet) // Since List[_] also belongs to localTypeOf[Product], we put this case before // "case t if definedByConstructorParams(t)" to make sure it will match to the // case "localTypeOf[Seq[_]]" - case t if t <:< localTypeOf[Seq[_]] => + case t if isSubtype(t, localTypeOf[Seq[_]]) => val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Array[_]] => + case t if isSubtype(t, localTypeOf[Array[_]]) => val TypeRef(_, _, Seq(elementType)) = t toCatalystArray(inputObject, elementType) - case t if t <:< localTypeOf[Map[_, _]] => + case t if isSubtype(t, localTypeOf[Map[_, _]]) => val TypeRef(_, _, Seq(keyType, valueType)) = t val keyClsName = getClassNameFromType(keyType) val valueClsName = getClassNameFromType(valueType) - val keyPath = s"""- map key class: "$keyClsName"""" +: walkedTypePath - val valuePath = s"""- map value class: "$valueClsName"""" +: walkedTypePath + val keyPath = walkedTypePath.recordKeyForMap(keyClsName) + val valuePath = walkedTypePath.recordValueForMap(valueClsName) - ExternalMapToCatalyst( + createSerializerForMap( inputObject, - dataTypeFor(keyType), - serializerFor(_, keyType, keyPath, seenTypeSet), - keyNullable = !keyType.typeSymbol.asClass.isPrimitive, - dataTypeFor(valueType), - serializerFor(_, valueType, valuePath, seenTypeSet), - valueNullable = !valueType.typeSymbol.asClass.isPrimitive) - - case t if t <:< localTypeOf[scala.collection.Set[_]] => + MapElementInformation( + dataTypeFor(keyType), + nullable = !keyType.typeSymbol.asClass.isPrimitive, + serializerFor(_, keyType, keyPath, seenTypeSet)), + MapElementInformation( + dataTypeFor(valueType), + nullable = !valueType.typeSymbol.asClass.isPrimitive, + serializerFor(_, valueType, valuePath, seenTypeSet)) + ) + + case t if isSubtype(t, localTypeOf[scala.collection.Set[_]]) => val TypeRef(_, _, Seq(elementType)) = t // There's no corresponding Catalyst type for `Set`, we serialize a `Set` to Catalyst array. @@ -530,94 +483,53 @@ object ScalaReflection extends ScalaReflection { toCatalystArray(newInput, elementType) - case t if t <:< localTypeOf[String] => - StaticInvoke( - classOf[UTF8String], - StringType, - "fromString", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.sql.Timestamp] => - StaticInvoke( - DateTimeUtils.getClass, - TimestampType, - "fromJavaTimestamp", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.sql.Date] => - StaticInvoke( - DateTimeUtils.getClass, - DateType, - "fromJavaDate", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[BigDecimal] => - StaticInvoke( - Decimal.getClass, - DecimalType.SYSTEM_DEFAULT, - "apply", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.math.BigDecimal] => - StaticInvoke( - Decimal.getClass, - DecimalType.SYSTEM_DEFAULT, - "apply", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.math.BigInteger] => - StaticInvoke( - Decimal.getClass, - DecimalType.BigIntDecimal, - "apply", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[scala.math.BigInt] => - StaticInvoke( - Decimal.getClass, - DecimalType.BigIntDecimal, - "apply", - inputObject :: Nil, - returnNullable = false) - - case t if t <:< localTypeOf[java.lang.Integer] => - Invoke(inputObject, "intValue", IntegerType) - case t if t <:< localTypeOf[java.lang.Long] => - Invoke(inputObject, "longValue", LongType) - case t if t <:< localTypeOf[java.lang.Double] => - Invoke(inputObject, "doubleValue", DoubleType) - case t if t <:< localTypeOf[java.lang.Float] => - Invoke(inputObject, "floatValue", FloatType) - case t if t <:< localTypeOf[java.lang.Short] => - Invoke(inputObject, "shortValue", ShortType) - case t if t <:< localTypeOf[java.lang.Byte] => - Invoke(inputObject, "byteValue", ByteType) - case t if t <:< localTypeOf[java.lang.Boolean] => - Invoke(inputObject, "booleanValue", BooleanType) + case t if isSubtype(t, localTypeOf[String]) => createSerializerForString(inputObject) + + case t if isSubtype(t, localTypeOf[java.time.Instant]) => + createSerializerForJavaInstant(inputObject) + + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => + createSerializerForSqlTimestamp(inputObject) + + case t if isSubtype(t, localTypeOf[java.time.LocalDate]) => + createSerializerForJavaLocalDate(inputObject) + + case t if isSubtype(t, localTypeOf[java.sql.Date]) => createSerializerForSqlDate(inputObject) + + case t if isSubtype(t, localTypeOf[BigDecimal]) => + createSerializerForScalaBigDecimal(inputObject) + + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => + createSerializerForJavaBigDecimal(inputObject) + + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => + createSerializerForJavaBigInteger(inputObject) + + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => + createSerializerForScalaBigInt(inputObject) + + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => + createSerializerForInteger(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Long]) => createSerializerForLong(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Double]) => + createSerializerForDouble(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Float]) => createSerializerForFloat(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Short]) => createSerializerForShort(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => createSerializerForByte(inputObject) + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => + createSerializerForBoolean(inputObject) case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => val udt = getClassFromType(t) - .getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() - val obj = NewInstance( - udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt(), - Nil, - dataType = ObjectType(udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt())) - Invoke(obj, "serialize", udt, inputObject :: Nil) + .getAnnotation(classOf[SQLUserDefinedType]).udt().getConstructor().newInstance() + val udtClass = udt.userClass.getAnnotation(classOf[SQLUserDefinedType]).udt() + createSerializerForUserDefinedType(inputObject, udt, udtClass) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] - val obj = NewInstance( - udt.getClass, - Nil, - dataType = ObjectType(udt.getClass)) - Invoke(obj, "serialize", udt, inputObject :: Nil) + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] + val udtClass = udt.getClass + createSerializerForUserDefinedType(inputObject, udt, udtClass) case t if definedByConstructorParams(t) => if (seenTypeSet.contains(t)) { @@ -626,26 +538,27 @@ object ScalaReflection extends ScalaReflection { } val params = getConstructorParameters(t) - val nonNullOutput = CreateNamedStruct(params.flatMap { case (fieldName, fieldType) => + 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.mkString("\n")) + "cannot be used as field name\n" + walkedTypePath) } - val fieldValue = Invoke( - AssertNotNull(inputObject, walkedTypePath), fieldName, dataTypeFor(fieldType), + // SPARK-26730 inputObject won't be null with If's guard below. And KnownNotNul + // is necessary here. Because for a nullable nested inputObject with struct data + // type, e.g. StructType(IntegerType, StringType), it will return nullable=true + // for IntegerType without KnownNotNull. And that's what we do not expect to. + val fieldValue = Invoke(KnownNotNull(inputObject), fieldName, dataTypeFor(fieldType), returnNullable = !fieldType.typeSymbol.asClass.isPrimitive) val clsName = getClassNameFromType(fieldType) - val newPath = s"""- field (class: "$clsName", name: "$fieldName")""" +: walkedTypePath - expressions.Literal(fieldName) :: - serializerFor(fieldValue, fieldType, newPath, seenTypeSet + t) :: Nil - }) - val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) - expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + val newPath = walkedTypePath.recordField(clsName, fieldName) + (fieldName, serializerFor(fieldValue, fieldType, newPath, seenTypeSet + t)) + } + createSerializerForObject(inputObject, fields) - case other => + case _ => throw new UnsupportedOperationException( - s"No Encoder found for $tpe\n" + walkedTypePath.mkString("\n")) + s"No Encoder found for $tpe\n" + walkedTypePath) } } @@ -655,7 +568,7 @@ object ScalaReflection extends ScalaReflection { */ def optionOfProductType(tpe: `Type`): Boolean = cleanUpReflectionObjects { tpe.dealias match { - case t if t <:< localTypeOf[Option[_]] => + case t if isSubtype(t, localTypeOf[Option[_]]) => val TypeRef(_, _, Seq(optType)) = t definedByConstructorParams(optType) case _ => false @@ -718,63 +631,70 @@ object ScalaReflection extends ScalaReflection { /** Returns a catalyst DataType and its nullability for the given Scala Type using reflection. */ def schemaFor(tpe: `Type`): Schema = cleanUpReflectionObjects { - tpe.dealias match { + baseType(tpe) match { // this must be the first case, since all objects in scala are instances of Null, therefore // Null type would wrongly match the first of them, which is Option as of now - case t if t <:< definitions.NullTpe => Schema(NullType, nullable = true) + case t if isSubtype(t, definitions.NullTpe) => Schema(NullType, nullable = true) case t if t.typeSymbol.annotations.exists(_.tree.tpe =:= typeOf[SQLUserDefinedType]) => - val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt().newInstance() + val udt = getClassFromType(t).getAnnotation(classOf[SQLUserDefinedType]).udt(). + getConstructor().newInstance() Schema(udt, nullable = true) case t if UDTRegistration.exists(getClassNameFromType(t)) => - val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.newInstance() - .asInstanceOf[UserDefinedType[_]] + val udt = UDTRegistration.getUDTFor(getClassNameFromType(t)).get.getConstructor(). + newInstance().asInstanceOf[UserDefinedType[_]] Schema(udt, nullable = true) - case t if t <:< localTypeOf[Option[_]] => + case t if isSubtype(t, localTypeOf[Option[_]]) => val TypeRef(_, _, Seq(optType)) = t Schema(schemaFor(optType).dataType, nullable = true) - case t if t <:< localTypeOf[Array[Byte]] => Schema(BinaryType, nullable = true) - case t if t <:< localTypeOf[Array[_]] => + case t if isSubtype(t, localTypeOf[Array[Byte]]) => Schema(BinaryType, nullable = true) + case t if isSubtype(t, localTypeOf[Array[_]]) => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< localTypeOf[Seq[_]] => + case t if isSubtype(t, localTypeOf[Seq[_]]) => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< localTypeOf[Map[_, _]] => + case t if isSubtype(t, localTypeOf[Map[_, _]]) => val TypeRef(_, _, Seq(keyType, valueType)) = t val Schema(valueDataType, valueNullable) = schemaFor(valueType) Schema(MapType(schemaFor(keyType).dataType, valueDataType, valueContainsNull = valueNullable), nullable = true) - case t if t <:< localTypeOf[Set[_]] => + case t if isSubtype(t, localTypeOf[Set[_]]) => val TypeRef(_, _, Seq(elementType)) = t val Schema(dataType, nullable) = schemaFor(elementType) Schema(ArrayType(dataType, containsNull = nullable), nullable = true) - case t if t <:< localTypeOf[String] => Schema(StringType, nullable = true) - case t if t <:< localTypeOf[java.sql.Timestamp] => Schema(TimestampType, nullable = true) - case t if t <:< localTypeOf[java.sql.Date] => Schema(DateType, nullable = true) - case t if t <:< localTypeOf[BigDecimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if t <:< localTypeOf[java.math.BigDecimal] => + case t if isSubtype(t, localTypeOf[String]) => Schema(StringType, nullable = true) + case t if isSubtype(t, localTypeOf[java.time.Instant]) => + Schema(TimestampType, nullable = true) + case t if isSubtype(t, localTypeOf[java.sql.Timestamp]) => + 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[BigDecimal]) => + Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + case t if isSubtype(t, localTypeOf[java.math.BigDecimal]) => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if t <:< localTypeOf[java.math.BigInteger] => + case t if isSubtype(t, localTypeOf[java.math.BigInteger]) => Schema(DecimalType.BigIntDecimal, nullable = true) - case t if t <:< localTypeOf[scala.math.BigInt] => + case t if isSubtype(t, localTypeOf[scala.math.BigInt]) => Schema(DecimalType.BigIntDecimal, nullable = true) - case t if t <:< localTypeOf[Decimal] => Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) - case t if t <:< localTypeOf[java.lang.Integer] => Schema(IntegerType, nullable = true) - case t if t <:< localTypeOf[java.lang.Long] => Schema(LongType, nullable = true) - case t if t <:< localTypeOf[java.lang.Double] => Schema(DoubleType, nullable = true) - case t if t <:< localTypeOf[java.lang.Float] => Schema(FloatType, nullable = true) - case t if t <:< localTypeOf[java.lang.Short] => Schema(ShortType, nullable = true) - case t if t <:< localTypeOf[java.lang.Byte] => Schema(ByteType, nullable = true) - case t if t <:< localTypeOf[java.lang.Boolean] => Schema(BooleanType, nullable = true) - case t if t <:< definitions.IntTpe => Schema(IntegerType, nullable = false) - case t if t <:< definitions.LongTpe => Schema(LongType, nullable = false) - case t if t <:< definitions.DoubleTpe => Schema(DoubleType, nullable = false) - case t if t <:< definitions.FloatTpe => Schema(FloatType, nullable = false) - case t if t <:< definitions.ShortTpe => Schema(ShortType, nullable = false) - case t if t <:< definitions.ByteTpe => Schema(ByteType, nullable = false) - case t if t <:< definitions.BooleanTpe => Schema(BooleanType, nullable = false) + case t if isSubtype(t, localTypeOf[Decimal]) => + Schema(DecimalType.SYSTEM_DEFAULT, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Integer]) => Schema(IntegerType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Long]) => Schema(LongType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Double]) => Schema(DoubleType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Float]) => Schema(FloatType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Short]) => Schema(ShortType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Byte]) => Schema(ByteType, nullable = true) + case t if isSubtype(t, localTypeOf[java.lang.Boolean]) => Schema(BooleanType, nullable = true) + case t if isSubtype(t, definitions.IntTpe) => Schema(IntegerType, nullable = false) + case t if isSubtype(t, definitions.LongTpe) => Schema(LongType, nullable = false) + case t if isSubtype(t, definitions.DoubleTpe) => Schema(DoubleType, nullable = false) + case t if isSubtype(t, definitions.FloatTpe) => Schema(FloatType, nullable = false) + case t if isSubtype(t, definitions.ShortTpe) => Schema(ShortType, nullable = false) + case t if isSubtype(t, definitions.ByteTpe) => Schema(ByteType, nullable = false) + case t if isSubtype(t, definitions.BooleanTpe) => Schema(BooleanType, nullable = false) case t if definedByConstructorParams(t) => val params = getConstructorParameters(t) Schema(StructType( @@ -788,12 +708,37 @@ object ScalaReflection extends ScalaReflection { } /** - * Finds an accessible constructor with compatible parameters. This is a more flexible search - * than the exact matching algorithm in `Class.getConstructor`. The first assignment-compatible - * matching constructor is returned. Otherwise, it returns `None`. + * Finds an accessible constructor with compatible parameters. This is a more flexible search than + * the exact matching algorithm in `Class.getConstructor`. The first assignment-compatible + * matching constructor is returned if it exists. Otherwise, we check for additional compatible + * constructors defined in the companion object as `apply` methods. Otherwise, it returns `None`. */ - def findConstructor(cls: Class[_], paramTypes: Seq[Class[_]]): Option[Constructor[_]] = { - Option(ConstructorUtils.getMatchingAccessibleConstructor(cls, paramTypes: _*)) + def findConstructor[T](cls: Class[T], paramTypes: Seq[Class[_]]): Option[Seq[AnyRef] => T] = { + Option(ConstructorUtils.getMatchingAccessibleConstructor(cls, paramTypes: _*)) match { + case Some(c) => Some(x => c.newInstance(x: _*).asInstanceOf[T]) + case None => + val companion = mirror.staticClass(cls.getName).companion + val moduleMirror = mirror.reflectModule(companion.asModule) + val applyMethods = companion.asTerm.typeSignature + .member(universe.TermName("apply")).asTerm.alternatives + applyMethods.find { method => + val params = method.typeSignature.paramLists.head + // Check that the needed params are the same length and of matching types + params.size == paramTypes.tail.size && + params.zip(paramTypes.tail).forall { case(ps, pc) => + ps.typeSignature.typeSymbol == mirror.classSymbol(pc) + } + }.map { applyMethodSymbol => + val expectedArgsCount = applyMethodSymbol.typeSignature.paramLists.head.size + val instanceMirror = mirror.reflect(moduleMirror.instance) + val method = instanceMirror.reflectMethod(applyMethodSymbol.asMethod) + (_args: Seq[AnyRef]) => { + // Drop the "outer" argument if it is provided + val args = if (_args.size == expectedArgsCount) _args else _args.tail + method.apply(args: _*).asInstanceOf[T] + } + } + } } /** @@ -802,9 +747,9 @@ object ScalaReflection extends ScalaReflection { def definedByConstructorParams(tpe: Type): Boolean = cleanUpReflectionObjects { tpe.dealias match { // `Option` is a `Product`, but we don't wanna treat `Option[Int]` as a struct type. - case t if t <:< localTypeOf[Option[_]] => definedByConstructorParams(t.typeArgs.head) - case _ => tpe.dealias <:< localTypeOf[Product] || - tpe.dealias <:< localTypeOf[DefinedByConstructorParams] + case t if isSubtype(t, localTypeOf[Option[_]]) => definedByConstructorParams(t.typeArgs.head) + case _ => isSubtype(tpe.dealias, localTypeOf[Product]) || + isSubtype(tpe.dealias, localTypeOf[DefinedByConstructorParams]) } } @@ -879,7 +824,7 @@ object ScalaReflection extends ScalaReflection { * Support for generating catalyst schemas for scala objects. Note that unlike its companion * object, this trait able to work in both the runtime and the compile time (macro) universe. */ -trait ScalaReflection { +trait ScalaReflection extends Logging { /** The universe we work in (runtime or macro) */ val universe: scala.reflect.api.Universe @@ -956,8 +901,19 @@ trait ScalaReflection { } } + /** + * If our type is a Scala trait it may have a companion object that + * only defines a constructor via `apply` method. + */ + private def getCompanionConstructor(tpe: Type): Symbol = { + tpe.typeSymbol.asClass.companion.asTerm.typeSignature.member(universe.TermName("apply")) + } + protected def constructParams(tpe: Type): Seq[Symbol] = { - val constructorSymbol = tpe.dealias.member(termNames.CONSTRUCTOR) + val constructorSymbol = tpe.member(termNames.CONSTRUCTOR) match { + case NoSymbol => getCompanionConstructor(tpe) + case sym => sym + } val params = if (constructorSymbol.isMethod) { constructorSymbol.asMethod.paramLists } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.scala new file mode 100644 index 0000000000000..75c278e781140 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SerializerBuildHelper.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.sql.catalyst + +import org.apache.spark.sql.catalyst.expressions.{CheckOverflow, CreateNamedStruct, Expression, IsNull, UnsafeArrayData} +import org.apache.spark.sql.catalyst.expressions.objects._ +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +object SerializerBuildHelper { + + private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + + def createSerializerForBoolean(inputObject: Expression): Expression = { + Invoke(inputObject, "booleanValue", BooleanType) + } + + def createSerializerForByte(inputObject: Expression): Expression = { + Invoke(inputObject, "byteValue", ByteType) + } + + def createSerializerForShort(inputObject: Expression): Expression = { + Invoke(inputObject, "shortValue", ShortType) + } + + def createSerializerForInteger(inputObject: Expression): Expression = { + Invoke(inputObject, "intValue", IntegerType) + } + + def createSerializerForLong(inputObject: Expression): Expression = { + Invoke(inputObject, "longValue", LongType) + } + + def createSerializerForFloat(inputObject: Expression): Expression = { + Invoke(inputObject, "floatValue", FloatType) + } + + def createSerializerForDouble(inputObject: Expression): Expression = { + Invoke(inputObject, "doubleValue", DoubleType) + } + + def createSerializerForString(inputObject: Expression): Expression = { + StaticInvoke( + classOf[UTF8String], + StringType, + "fromString", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForJavaInstant(inputObject: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + TimestampType, + "instantToMicros", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForSqlTimestamp(inputObject: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + TimestampType, + "fromJavaTimestamp", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForJavaLocalDate(inputObject: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + DateType, + "localDateToDays", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForSqlDate(inputObject: Expression): Expression = { + StaticInvoke( + DateTimeUtils.getClass, + DateType, + "fromJavaDate", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForJavaBigDecimal(inputObject: Expression): Expression = { + CheckOverflow(StaticInvoke( + Decimal.getClass, + DecimalType.SYSTEM_DEFAULT, + "apply", + inputObject :: Nil, + returnNullable = false), DecimalType.SYSTEM_DEFAULT, nullOnOverflow) + } + + def createSerializerForScalaBigDecimal(inputObject: Expression): Expression = { + createSerializerForJavaBigDecimal(inputObject) + } + + def createSerializerForJavaBigInteger(inputObject: Expression): Expression = { + CheckOverflow(StaticInvoke( + Decimal.getClass, + DecimalType.BigIntDecimal, + "apply", + inputObject :: Nil, + returnNullable = false), DecimalType.BigIntDecimal, nullOnOverflow) + } + + def createSerializerForScalaBigInt(inputObject: Expression): Expression = { + createSerializerForJavaBigInteger(inputObject) + } + + def createSerializerForPrimitiveArray( + inputObject: Expression, + dataType: DataType): Expression = { + StaticInvoke( + classOf[UnsafeArrayData], + ArrayType(dataType, false), + "fromPrimitiveArray", + inputObject :: Nil, + returnNullable = false) + } + + def createSerializerForGenericArray( + inputObject: Expression, + dataType: DataType, + nullable: Boolean): Expression = { + NewInstance( + classOf[GenericArrayData], + inputObject :: Nil, + dataType = ArrayType(dataType, nullable)) + } + + def createSerializerForMapObjects( + inputObject: Expression, + dataType: ObjectType, + funcForNewExpr: Expression => Expression): Expression = { + MapObjects(funcForNewExpr, inputObject, dataType) + } + + case class MapElementInformation( + dataType: DataType, + nullable: Boolean, + funcForNewExpr: Expression => Expression) + + def createSerializerForMap( + inputObject: Expression, + keyInformation: MapElementInformation, + valueInformation: MapElementInformation): Expression = { + ExternalMapToCatalyst( + inputObject, + keyInformation.dataType, + keyInformation.funcForNewExpr, + keyNullable = keyInformation.nullable, + valueInformation.dataType, + valueInformation.funcForNewExpr, + valueNullable = valueInformation.nullable + ) + } + + private def argumentsForFieldSerializer( + fieldName: String, + serializerForFieldValue: Expression): Seq[Expression] = { + expressions.Literal(fieldName) :: serializerForFieldValue :: Nil + } + + def createSerializerForObject( + inputObject: Expression, + fields: Seq[(String, Expression)]): Expression = { + val nonNullOutput = CreateNamedStruct(fields.flatMap { case(fieldName, fieldExpr) => + argumentsForFieldSerializer(fieldName, fieldExpr) + }) + val nullOutput = expressions.Literal.create(null, nonNullOutput.dataType) + expressions.If(IsNull(inputObject), nullOutput, nonNullOutput) + } + + def createSerializerForUserDefinedType( + inputObject: Expression, + udt: UserDefinedType[_], + udtClass: Class[_]): Expression = { + val obj = NewInstance(udtClass, Nil, dataType = ObjectType(udtClass)) + Invoke(obj, "serialize", udt, inputObject :: Nil) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala new file mode 100644 index 0000000000000..cbf1f01344c92 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/WalkedTypePath.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +/** + * This class records the paths the serializer and deserializer walk through to reach current path. + * Note that this class adds new path in prior to recorded paths so it maintains + * the paths as reverse order. + */ +case class WalkedTypePath(private val walkedPaths: Seq[String] = Nil) { + def recordRoot(className: String): WalkedTypePath = + newInstance(s"""- root class: "$className"""") + + def recordOption(className: String): WalkedTypePath = + newInstance(s"""- option value class: "$className"""") + + def recordArray(elementClassName: String): WalkedTypePath = + newInstance(s"""- array element class: "$elementClassName"""") + + def recordMap(keyClassName: String, valueClassName: String): WalkedTypePath = { + newInstance(s"""- map key class: "$keyClassName"""" + + s""", value class: "$valueClassName"""") + } + + def recordKeyForMap(keyClassName: String): WalkedTypePath = + newInstance(s"""- map key class: "$keyClassName"""") + + def recordValueForMap(valueClassName: String): WalkedTypePath = + newInstance(s"""- map value class: "$valueClassName"""") + + def recordField(className: String, fieldName: String): WalkedTypePath = + newInstance(s"""- field (class: "$className", name: "$fieldName")""") + + override def toString: String = { + walkedPaths.mkString("\n") + } + + def getPaths: Seq[String] = walkedPaths + + private def newInstance(newRecord: String): WalkedTypePath = + WalkedTypePath(newRecord +: walkedPaths) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala index 6d587abd8fd4d..f5e9a146bf359 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/AlreadyExistException.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ +import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec /** @@ -25,13 +27,26 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ class DatabaseAlreadyExistsException(db: String) - extends AnalysisException(s"Database '$db' already exists") + extends NamespaceAlreadyExistsException(s"Database '$db' already exists") -class TableAlreadyExistsException(db: String, table: String) - extends AnalysisException(s"Table or view '$table' already exists in database '$db'") +class NamespaceAlreadyExistsException(message: String) extends AnalysisException(message) { + def this(namespace: Array[String]) = { + this(s"Namespace '${namespace.quoted}' already exists") + } +} + +class TableAlreadyExistsException(message: String) extends AnalysisException(message) { + def this(db: String, table: String) = { + this(s"Table or view '$table' already exists in database '$db'") + } + + def this(tableIdent: Identifier) = { + this(s"Table ${tableIdent.quoted} already exists") + } +} class TempTableAlreadyExistsException(table: String) - extends AnalysisException(s"Temporary view '$table' already exists") + extends TableAlreadyExistsException(s"Temporary view '$table' already exists") class PartitionAlreadyExistsException(db: String, table: String, spec: TablePartitionSpec) extends AnalysisException( 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 7f641ace46298..e55cdfedd3234 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 @@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer import scala.util.Random import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, LookupCatalog, TableChange} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.encoders.OuterScopes @@ -33,9 +34,11 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.trees.TreeNodeRef import org.apache.spark.sql.catalyst.util.toPrettySQL +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -96,22 +99,29 @@ class Analyzer( catalog: SessionCatalog, conf: SQLConf, maxIterations: Int) - extends RuleExecutor[LogicalPlan] with CheckAnalysis { + extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog { def this(catalog: SessionCatalog, conf: SQLConf) = { this(catalog, conf, conf.optimizerMaxIterations) } - def executeAndCheck(plan: LogicalPlan): LogicalPlan = AnalysisHelper.markInAnalyzer { - val analyzed = execute(plan) - try { - checkAnalysis(analyzed) - analyzed - } catch { - case e: AnalysisException => - val ae = new AnalysisException(e.message, e.line, e.startPosition, Option(analyzed)) - ae.setStackTrace(e.getStackTrace) - throw ae + override protected def defaultCatalogName: Option[String] = conf.defaultV2Catalog + + override protected def lookupCatalog(name: String): CatalogPlugin = + throw new CatalogNotFoundException("No catalog lookup function") + + def executeAndCheck(plan: LogicalPlan, tracker: QueryPlanningTracker): LogicalPlan = { + AnalysisHelper.markInAnalyzer { + val analyzed = executeAndTrack(plan, tracker) + try { + checkAnalysis(analyzed) + analyzed + } catch { + case e: AnalysisException => + val ae = new AnalysisException(e.message, e.line, e.startPosition, Option(analyzed)) + ae.setStackTrace(e.getStackTrace) + throw ae + } } } @@ -144,9 +154,9 @@ class Analyzer( lazy val batches: Seq[Batch] = Seq( Batch("Hints", fixedPoint, - new ResolveHints.ResolveBroadcastHints(conf), + new ResolveHints.ResolveJoinStrategyHints(conf), ResolveHints.ResolveCoalesceHints, - ResolveHints.RemoveAllHints), + new ResolveHints.RemoveAllHints(conf)), Batch("Simple Sanity Check", Once, LookupFunctions), Batch("Substitution", fixedPoint, @@ -156,6 +166,8 @@ class Analyzer( new SubstituteUnresolvedOrdinals(conf)), Batch("Resolution", fixedPoint, ResolveTableValuedFunctions :: + ResolveAlterTable :: + ResolveTables :: ResolveRelations :: ResolveReferences :: ResolveCreateNamedStruct :: @@ -189,48 +201,18 @@ class Analyzer( TypeCoercion.typeCoercionRules(conf) ++ extendedResolutionRules : _*), Batch("Post-Hoc Resolution", Once, postHocResolutionRules: _*), - Batch("View", Once, - AliasViewChild(conf)), Batch("Nondeterministic", Once, PullOutNondeterministic), Batch("UDF", Once, HandleNullInputsForUDF), - Batch("FixNullability", Once, - FixNullability), + Batch("UpdateNullability", Once, + UpdateAttributeNullability), Batch("Subquery", Once, UpdateOuterReferences), Batch("Cleanup", fixedPoint, CleanupAliases) ) - /** - * Analyze cte definitions and substitute child plan with analyzed cte definitions. - */ - object CTESubstitution extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case With(child, relations) => - substituteCTE(child, relations.foldLeft(Seq.empty[(String, LogicalPlan)]) { - case (resolved, (name, relation)) => - resolved :+ name -> executeSameContext(substituteCTE(relation, resolved)) - }) - case other => other - } - - def substituteCTE(plan: LogicalPlan, cteRelations: Seq[(String, LogicalPlan)]): LogicalPlan = { - plan resolveOperatorsDown { - case u: UnresolvedRelation => - cteRelations.find(x => resolver(x._1, u.tableIdentifier.table)) - .map(_._2).getOrElse(u) - case other => - // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. - other transformExpressions { - case e: SubqueryExpression => - e.withNewPlan(substituteCTE(e.plan, cteRelations)) - } - } - } - } - /** * Substitute child plan with WindowSpecDefinitions. */ @@ -648,6 +630,24 @@ class Analyzer( } } + /** + * Resolve table relations with concrete relations from v2 catalog. + * + * [[ResolveRelations]] still resolves v1 tables. + */ + object ResolveTables extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util._ + + def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { + case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) + if catalog.isTemporaryTable(ident) => + u // temporary views take precedence over catalog table names + + case u @ UnresolvedRelation(CatalogObjectIdentifier(Some(catalogPlugin), ident)) => + loadTable(catalogPlugin, ident).map(DataSourceV2Relation.create).getOrElse(u) + } + } + /** * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. */ @@ -680,10 +680,19 @@ class Analyzer( // Note this is compatible with the views defined by older versions of Spark(before 2.2), which // have empty defaultDatabase and all the relations in viewText have database part defined. def resolveRelation(plan: LogicalPlan): LogicalPlan = plan match { - case u: UnresolvedRelation if !isRunningDirectlyOnFiles(u.tableIdentifier) => + case u @ UnresolvedRelation(AsTemporaryViewIdentifier(ident)) + if catalog.isTemporaryTable(ident) => + resolveRelation(lookupTableFromCatalog(ident, u, AnalysisContext.get.defaultDatabase)) + + case u @ UnresolvedRelation(AsTableIdentifier(ident)) if !isRunningDirectlyOnFiles(ident) => val defaultDatabase = AnalysisContext.get.defaultDatabase - val foundRelation = lookupTableFromCatalog(u, defaultDatabase) - resolveRelation(foundRelation) + val foundRelation = lookupTableFromCatalog(ident, u, defaultDatabase) + if (foundRelation != u) { + resolveRelation(foundRelation) + } else { + u + } + // The view's child should be a logical plan parsed from the `desc.viewText`, the variable // `viewText` should be defined, or else we throw an error on the generation of the View // operator. @@ -706,8 +715,9 @@ class Analyzer( } def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case i @ InsertIntoTable(u: UnresolvedRelation, parts, child, _, _) if child.resolved => - EliminateSubqueryAliases(lookupTableFromCatalog(u)) match { + case i @ InsertIntoTable(u @ UnresolvedRelation(AsTableIdentifier(ident)), _, child, _, _) + if child.resolved => + EliminateSubqueryAliases(lookupTableFromCatalog(ident, u)) match { case v: View => u.failAnalysis(s"Inserting into a view is not allowed. View: ${v.desc.identifier}.") case other => i.copy(table = other) @@ -722,20 +732,16 @@ class Analyzer( // and the default database is only used to look up a view); // 3. Use the currentDb of the SessionCatalog. private def lookupTableFromCatalog( + tableIdentifier: TableIdentifier, u: UnresolvedRelation, defaultDatabase: Option[String] = None): LogicalPlan = { - val tableIdentWithDb = u.tableIdentifier.copy( - database = u.tableIdentifier.database.orElse(defaultDatabase)) + val tableIdentWithDb = tableIdentifier.copy( + database = tableIdentifier.database.orElse(defaultDatabase)) try { catalog.lookupRelation(tableIdentWithDb) } catch { - case e: NoSuchTableException => - u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}", e) - // If the database is defined and that database is not found, throw an AnalysisException. - // Note that if the database is not defined, it is possible we are looking up a temp view. - case e: NoSuchDatabaseException => - u.failAnalysis(s"Table or view not found: ${tableIdentWithDb.unquotedString}, the " + - s"database ${e.db} doesn't exist.", e) + case _: NoSuchTableException | _: NoSuchDatabaseException => + u } } @@ -751,6 +757,86 @@ class Analyzer( } } + /** + * Resolve ALTER TABLE statements that use a DSv2 catalog. + * + * This rule converts unresolved ALTER TABLE statements to v2 when a v2 catalog is responsible + * for the table identifier. A v2 catalog is responsible for an identifier when the identifier + * has a catalog specified, like prod_catalog.db.table, or when a default v2 catalog is set and + * the table identifier does not include a catalog. + */ + object ResolveAlterTable extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case alter @ AlterTableAddColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map { col => + TableChange.addColumn(col.name.toArray, col.dataType, true, col.comment.orNull) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableAlterColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), colName, dataType, comment) => + val typeChange = dataType.map { newDataType => + TableChange.updateColumnType(colName.toArray, newDataType, true) + } + + val commentChange = comment.map { newComment => + TableChange.updateColumnComment(colName.toArray, newComment) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + typeChange.toSeq ++ commentChange.toSeq) + + case alter @ AlterTableRenameColumnStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), col, newName) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.renameColumn(col.toArray, newName))) + + case alter @ AlterTableDropColumnsStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), cols) => + val changes = cols.map(col => TableChange.deleteColumn(col.toArray)) + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes) + + case alter @ AlterTableSetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), props) => + val changes = props.map { + case (key, value) => + TableChange.setProperty(key, value) + } + + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + changes.toSeq) + + case alter @ AlterTableUnsetPropertiesStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), keys, _) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + keys.map(key => TableChange.removeProperty(key))) + + case alter @ AlterTableSetLocationStatement( + CatalogObjectIdentifier(Some(v2Catalog), ident), newLoc) => + AlterTable( + v2Catalog.asTableCatalog, ident, + UnresolvedRelation(alter.tableName), + Seq(TableChange.setProperty("location", newLoc))) + } + } + /** * Replaces [[UnresolvedAttribute]]s with concrete [[AttributeReference]]s from * a logical plan node's children. @@ -824,7 +910,8 @@ class Analyzer( } private def dedupAttr(attr: Attribute, attrMap: AttributeMap[Attribute]): Attribute = { - attrMap.get(attr).getOrElse(attr).withQualifier(attr.qualifier) + val exprId = attrMap.getOrElse(attr, attr).exprId + attr.withExprId(exprId) } /** @@ -870,7 +957,7 @@ class Analyzer( private def dedupOuterReferencesInSubquery( plan: LogicalPlan, attrMap: AttributeMap[Attribute]): LogicalPlan = { - plan resolveOperatorsDown { case currentFragment => + plan transformDown { case currentFragment => currentFragment transformExpressions { case OuterReference(a: Attribute) => OuterReference(dedupAttr(a, attrMap)) @@ -880,21 +967,38 @@ class Analyzer( } } - private def resolve(e: Expression, q: LogicalPlan): Expression = e match { - case f: LambdaFunction if !f.bound => f - case u @ UnresolvedAttribute(nameParts) => - // Leave unchanged if resolution fails. Hopefully will be resolved next round. - val result = - withPosition(u) { - q.resolveChildren(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, q)) - .getOrElse(u) - } - logDebug(s"Resolving $u to $result") - result - case UnresolvedExtractValue(child, fieldExpr) if child.resolved => - ExtractValue(child, fieldExpr, resolver) - case _ => e.mapChildren(resolve(_, q)) + /** + * Resolves the attribute and extract value expressions(s) by traversing the + * input expression in top down manner. The traversal is done in top-down manner as + * we need to skip over unbound lamda function expression. The lamda expressions are + * resolved in a different rule [[ResolveLambdaVariables]] + * + * Example : + * SELECT transform(array(1, 2, 3), (x, i) -> x + i)" + * + * In the case above, x and i are resolved as lamda variables in [[ResolveLambdaVariables]] + * + * Note : In this routine, the unresolved attributes are resolved from the input plan's + * children attributes. + */ + private def resolveExpressionTopDown(e: Expression, q: LogicalPlan): Expression = { + if (e.resolved) return e + e match { + case f: LambdaFunction if !f.bound => f + case u @ UnresolvedAttribute(nameParts) => + // Leave unchanged if resolution fails. Hopefully will be resolved next round. + val result = + withPosition(u) { + q.resolveChildren(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, q)) + .getOrElse(u) + } + logDebug(s"Resolving $u to $result") + result + case UnresolvedExtractValue(child, fieldExpr) if child.resolved => + ExtractValue(child, fieldExpr, resolver) + case _ => e.mapChildren(resolveExpressionTopDown(_, q)) + } } def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { @@ -923,17 +1027,33 @@ class Analyzer( failAnalysis("Invalid usage of '*' in explode/json_tuple/UDTF") // To resolve duplicate expression IDs for Join and Intersect - case j @ Join(left, right, _, _) if !j.duplicateResolved => + case j @ Join(left, right, _, _, _) if !j.duplicateResolved => j.copy(right = dedupRight(left, right)) case i @ Intersect(left, right, _) if !i.duplicateResolved => i.copy(right = dedupRight(left, right)) case e @ Except(left, right, _) if !e.duplicateResolved => e.copy(right = dedupRight(left, right)) + case u @ Union(children) if !u.duplicateResolved => + // Use projection-based de-duplication for Union to avoid breaking the checkpoint sharing + // feature in streaming. + val newChildren = children.foldRight(Seq.empty[LogicalPlan]) { (head, tail) => + head +: tail.map { + case child if head.outputSet.intersect(child.outputSet).isEmpty => + child + case child => + val projectList = child.output.map { attr => + Alias(attr, attr.name)() + } + Project(projectList, child) + } + } + u.copy(children = newChildren) + // When resolve `SortOrder`s in Sort based on child, don't report errors as // we still have chance to resolve it based on its descendants case s @ Sort(ordering, global, child) if child.resolved && !s.resolved => val newOrdering = - ordering.map(order => resolveExpression(order, child).asInstanceOf[SortOrder]) + ordering.map(order => resolveExpressionBottomUp(order, child).asInstanceOf[SortOrder]) Sort(newOrdering, global, child) // A special case for Generate, because the output of Generate should not be resolved by @@ -941,7 +1061,7 @@ class Analyzer( case g @ Generate(generator, _, _, _, _, _) if generator.resolved => g case g @ Generate(generator, join, outer, qualifier, output, child) => - val newG = resolveExpression(generator, child, throws = true) + val newG = resolveExpressionBottomUp(generator, child, throws = true) if (newG.fastEquals(generator)) { g } else { @@ -952,9 +1072,20 @@ class Analyzer( // rule: ResolveDeserializer. case plan if containsDeserializer(plan.expressions) => plan + // SPARK-25942: Resolves aggregate expressions with `AppendColumns`'s children, instead of + // `AppendColumns`, because `AppendColumns`'s serializer might produce conflict attribute + // names leading to ambiguous references exception. + case a @ Aggregate(groupingExprs, aggExprs, appendColumns: AppendColumns) => + a.mapExpressions(resolveExpressionTopDown(_, appendColumns)) + + case o: OverwriteByExpression if !o.outputResolved => + // do not resolve expression attributes until the query attributes are resolved against the + // table by ResolveOutputRelation. that rule will alias the attributes to the table's names. + o + case q: LogicalPlan => - logTrace(s"Attempting to resolve ${q.simpleString}") - q.mapExpressions(resolve(_, q)) + logTrace(s"Attempting to resolve ${q.simpleString(SQLConf.get.maxToStringFields)}") + q.mapExpressions(resolveExpressionTopDown(_, q)) } def newAliases(expressions: Seq[NamedExpression]): Seq[NamedExpression] = { @@ -1016,6 +1147,11 @@ class Analyzer( case s: Star => s.expand(child, resolver) case o => o :: Nil }) + case p: XxHash64 if containsStar(p.children) => + p.copy(children = p.children.flatMap { + case s: Star => s.expand(child, resolver) + case o => o :: Nil + }) // count(*) has been replaced by count(1) case o if containsStar(o.children) => failAnalysis(s"Invalid usage of '*' in expression '${o.prettyName}'") @@ -1051,7 +1187,22 @@ class Analyzer( func.map(wrapper) } - protected[sql] def resolveExpression( + /** + * Resolves the attribute, column value and extract value expressions(s) by traversing the + * input expression in bottom-up manner. In order to resolve the nested complex type fields + * correctly, this function makes use of `throws` parameter to control when to raise an + * AnalysisException. + * + * Example : + * SELECT a.b FROM t ORDER BY b[0].d + * + * In the above example, in b needs to be resolved before d can be resolved. Given we are + * doing a bottom up traversal, it will first attempt to resolve d and fail as b has not + * been resolved yet. If `throws` is false, this function will handle the exception by + * returning the original attribute. In this case `d` will be resolved in subsequent passes + * after `b` is resolved. + */ + protected[sql] def resolveExpressionBottomUp( expr: Expression, plan: LogicalPlan, throws: Boolean = false): Expression = { @@ -1064,11 +1215,14 @@ class Analyzer( expr transformUp { case GetColumnByOrdinal(ordinal, _) => plan.output(ordinal) case u @ UnresolvedAttribute(nameParts) => - withPosition(u) { - plan.resolve(nameParts, resolver) - .orElse(resolveLiteralFunction(nameParts, u, plan)) - .getOrElse(u) - } + val result = + withPosition(u) { + plan.resolve(nameParts, resolver) + .orElse(resolveLiteralFunction(nameParts, u, plan)) + .getOrElse(u) + } + logDebug(s"Resolving $u to $result") + result case UnresolvedExtractValue(child, fieldName) if child.resolved => ExtractValue(child, fieldName, resolver) } @@ -1214,7 +1368,7 @@ class Analyzer( plan match { case p: Project => // Resolving expressions against current plan. - val maybeResolvedExprs = exprs.map(resolveExpression(_, p)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, p)) // Recursively resolving expressions on the child of current plan. val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, p.child) // If some attributes used by expressions are resolvable only on the rewritten child @@ -1223,7 +1377,7 @@ class Analyzer( (newExprs, Project(p.projectList ++ missingAttrs, newChild)) case a @ Aggregate(groupExprs, aggExprs, child) => - val maybeResolvedExprs = exprs.map(resolveExpression(_, a)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, a)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, child) val missingAttrs = (AttributeSet(newExprs) -- a.outputSet).intersect(newChild.outputSet) if (missingAttrs.forall(attr => groupExprs.exists(_.semanticEquals(attr)))) { @@ -1235,20 +1389,20 @@ class Analyzer( } case g: Generate => - val maybeResolvedExprs = exprs.map(resolveExpression(_, g)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, g)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, g.child) (newExprs, g.copy(unrequiredChildIndex = Nil, child = newChild)) // For `Distinct` and `SubqueryAlias`, we can't recursively resolve and add attributes // via its children. case u: UnaryNode if !u.isInstanceOf[Distinct] && !u.isInstanceOf[SubqueryAlias] => - val maybeResolvedExprs = exprs.map(resolveExpression(_, u)) + val maybeResolvedExprs = exprs.map(resolveExpressionBottomUp(_, u)) val (newExprs, newChild) = resolveExprsAndAddMissingAttrs(maybeResolvedExprs, u.child) (newExprs, u.withNewChildren(Seq(newChild))) // For other operators, we can't recursively resolve and add attributes via its children. case other => - (exprs.map(resolveExpression(_, other)), other) + (exprs.map(resolveExpressionBottomUp(_, other)), other) } } } @@ -1733,7 +1887,7 @@ class Analyzer( case p if p.expressions.exists(hasGenerator) => throw new AnalysisException("Generators are not supported outside the SELECT clause, but " + - "got: " + p.simpleString) + "got: " + p.simpleString(SQLConf.get.maxToStringFields)) } } @@ -1777,40 +1931,6 @@ class Analyzer( } } - /** - * Fixes nullability of Attributes in a resolved LogicalPlan by using the nullability of - * corresponding Attributes of its children output Attributes. This step is needed because - * users can use a resolved AttributeReference in the Dataset API and outer joins - * can change the nullability of an AttribtueReference. Without the fix, a nullable column's - * nullable field can be actually set as non-nullable, which cause illegal optimization - * (e.g., NULL propagation) and wrong answers. - * See SPARK-13484 and SPARK-13801 for the concrete queries of this case. - */ - object FixNullability extends Rule[LogicalPlan] { - - def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case p if !p.resolved => p // Skip unresolved nodes. - case p: LogicalPlan if p.resolved => - val childrenOutput = p.children.flatMap(c => c.output).groupBy(_.exprId).flatMap { - case (exprId, attributes) => - // If there are multiple Attributes having the same ExprId, we need to resolve - // the conflict of nullable field. We do not really expect this happen. - val nullable = attributes.exists(_.nullable) - attributes.map(attr => attr.withNullability(nullable)) - }.toSeq - // At here, we create an AttributeMap that only compare the exprId for the lookup - // operation. So, we can find the corresponding input attribute's nullability. - val attributeMap = AttributeMap[Attribute](childrenOutput.map(attr => attr -> attr)) - // For an Attribute used by the current LogicalPlan, if it is from its children, - // we fix the nullable field by using the nullability setting of the corresponding - // output Attribute from the children. - p.transformExpressions { - case attr: Attribute if attributeMap.contains(attr) => - attr.withNullability(attributeMap(attr).nullable) - } - } - } - /** * Extracts [[WindowExpression]]s from the projectList of a [[Project]] operator and * aggregateExpressions of an [[Aggregate]] operator and creates individual [[Window]] @@ -2137,35 +2257,35 @@ class Analyzer( case p => p transformExpressionsUp { - case udf@ScalaUDF(func, _, inputs, _, _, _, _, nullableTypes) => - if (nullableTypes.isEmpty) { - // If no nullability info is available, do nothing. No fields will be specially - // checked for null in the plan. If nullability info is incorrect, the results - // of the UDF could be wrong. - udf - } else { - // 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(nullableTypes.length == inputs.length) - - // TODO: skip null handling for not-nullable primitive inputs after we can completely - // trust the `nullable` information. - val needsNullCheck = (nullable: Boolean, expr: Expression) => - nullable && !expr.isInstanceOf[KnownNotNull] - val inputsNullCheck = nullableTypes.zip(inputs) - .filter { case (nullableType, expr) => needsNullCheck(!nullableType, expr) } - .map { case (_, expr) => IsNull(expr) } - .reduceLeftOption[Expression]((e1, e2) => Or(e1, e2)) + case udf @ ScalaUDF(_, _, inputs, inputPrimitives, _, _, _, _) + if 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) + + val inputPrimitivesPair = inputPrimitives.zip(inputs) + val inputNullCheck = inputPrimitivesPair.collect { + case (isPrimitive, input) if isPrimitive && input.nullable => + IsNull(input) + }.reduceLeftOption[Expression](Or) + + if (inputNullCheck.isDefined) { // Once we add an `If` check above the udf, it is safe to mark those checked inputs - // as not nullable (i.e., wrap them with `KnownNotNull`), because the null-returning + // as null-safe (i.e., wrap with `KnownNotNull`), because the null-returning // branch of `If` will be called if any of these checked inputs is null. Thus we can // prevent this rule from being applied repeatedly. - val newInputs = nullableTypes.zip(inputs).map { case (nullable, expr) => - if (nullable) expr else KnownNotNull(expr) + val newInputs = inputPrimitivesPair.map { + case (isPrimitive, input) => + if (isPrimitive && input.nullable) { + KnownNotNull(input) + } else { + input + } } - inputsNullCheck - .map(If(_, Literal.create(null, udf.dataType), udf.copy(children = newInputs))) - .getOrElse(udf) + val newUDF = udf.copy(children = newInputs) + If(inputNullCheck.get, Literal.create(null, udf.dataType), newUDF) + } else { + udf } } } @@ -2214,13 +2334,14 @@ class Analyzer( */ object ResolveNaturalAndUsingJoin extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperatorsUp { - case j @ Join(left, right, UsingJoin(joinType, usingCols), _) + case j @ Join(left, right, UsingJoin(joinType, usingCols), _, hint) if left.resolved && right.resolved && j.duplicateResolved => - commonNaturalJoinProcessing(left, right, joinType, usingCols, None) - case j @ Join(left, right, NaturalJoin(joinType), condition) if j.resolvedExceptNatural => + commonNaturalJoinProcessing(left, right, joinType, usingCols, None, hint) + case j @ Join(left, right, NaturalJoin(joinType), condition, hint) + if j.resolvedExceptNatural => // find common column names from both sides val joinNames = left.output.map(_.name).intersect(right.output.map(_.name)) - commonNaturalJoinProcessing(left, right, joinType, joinNames, condition) + commonNaturalJoinProcessing(left, right, joinType, joinNames, condition, hint) } } @@ -2235,7 +2356,7 @@ class Analyzer( object ResolveOutputRelation extends Rule[LogicalPlan] { override def apply(plan: LogicalPlan): LogicalPlan = plan.resolveOperators { case append @ AppendData(table, query, isByName) - if table.resolved && query.resolved && !append.resolved => + if table.resolved && query.resolved && !append.outputResolved => val projection = resolveOutputColumns(table.name, table.output, query, isByName) if (projection != query) { @@ -2243,6 +2364,26 @@ class Analyzer( } else { append } + + case overwrite @ OverwriteByExpression(table, _, query, isByName) + if table.resolved && query.resolved && !overwrite.outputResolved => + val projection = resolveOutputColumns(table.name, table.output, query, isByName) + + if (projection != query) { + overwrite.copy(query = projection) + } else { + overwrite + } + + case overwrite @ OverwritePartitionsDynamic(table, query, isByName) + if table.resolved && query.resolved && !overwrite.outputResolved => + val projection = resolveOutputColumns(table.name, table.output, query, isByName) + + if (projection != query) { + overwrite.copy(query = projection) + } else { + overwrite + } } def resolveOutputColumns( @@ -2263,7 +2404,7 @@ class Analyzer( expected.flatMap { tableAttr => query.resolveQuoted(tableAttr.name, resolver) match { case Some(queryExpr) => - checkField(tableAttr, queryExpr, err => errors += err) + checkField(tableAttr, queryExpr, byName, err => errors += err) case None => errors += s"Cannot find data for output column '${tableAttr.name}'" None @@ -2281,7 +2422,7 @@ class Analyzer( query.output.zip(expected).flatMap { case (queryExpr, tableAttr) => - checkField(tableAttr, queryExpr, err => errors += err) + checkField(tableAttr, queryExpr, byName, err => errors += err) } } @@ -2296,11 +2437,12 @@ class Analyzer( private def checkField( tableAttr: Attribute, queryExpr: NamedExpression, + byName: Boolean, addError: String => Unit): Option[NamedExpression] = { // run the type check first to ensure type errors are present val canWrite = DataType.canWrite( - queryExpr.dataType, tableAttr.dataType, resolver, tableAttr.name, addError) + queryExpr.dataType, tableAttr.dataType, byName, resolver, tableAttr.name, addError) if (queryExpr.nullable && !tableAttr.nullable) { addError(s"Cannot write nullable values to non-null column '${tableAttr.name}'") @@ -2312,7 +2454,7 @@ class Analyzer( } else { // always add an UpCast. it will be removed in the optimizer if it is unnecessary. Some(Alias( - UpCast(queryExpr, tableAttr.dataType, Seq()), tableAttr.name + UpCast(queryExpr, tableAttr.dataType), tableAttr.name )( explicitMetadata = Option(tableAttr.metadata) )) @@ -2325,7 +2467,8 @@ class Analyzer( right: LogicalPlan, joinType: JoinType, joinNames: Seq[String], - condition: Option[Expression]) = { + condition: Option[Expression], + hint: JoinHint) = { val leftKeys = joinNames.map { keyName => left.output.find(attr => resolver(attr.name, keyName)).getOrElse { throw new AnalysisException(s"USING column `$keyName` cannot be resolved on the left " + @@ -2366,7 +2509,7 @@ class Analyzer( sys.error("Unsupported natural join type " + joinType) } // use Project to trim unnecessary fields - Project(projectList, Join(left, right, joinType, newCondition)) + Project(projectList, Join(left, right, joinType, newCondition, hint)) } /** @@ -2387,20 +2530,29 @@ class Analyzer( } validateTopLevelTupleFields(deserializer, inputs) - val resolved = resolveExpression( + val resolved = resolveExpressionBottomUp( deserializer, LocalRelation(inputs), throws = true) val result = resolved transformDown { case UnresolvedMapObjects(func, inputData, cls) if inputData.resolved => inputData.dataType match { case ArrayType(et, cn) => - val expr = MapObjects(func, inputData, et, cn, cls) transformUp { + MapObjects(func, inputData, et, cn, cls) transformUp { case UnresolvedExtractValue(child, fieldName) if child.resolved => ExtractValue(child, fieldName, resolver) } - expr case other => throw new AnalysisException("need an array field but got " + other.catalogString) } + case u: UnresolvedCatalystToExternalMap if u.child.resolved => + u.child.dataType match { + case _: MapType => + CatalystToExternalMap(u) transformUp { + case UnresolvedExtractValue(child, fieldName) if child.resolved => + ExtractValue(child, fieldName, resolver) + } + case other => + throw new AnalysisException("need a map field but got " + other.catalogString) + } } validateNestedTupleFields(result) result @@ -2488,7 +2640,7 @@ class Analyzer( case e => e.sql } throw new AnalysisException(s"Cannot up cast $fromStr from " + - s"${from.dataType.catalogString} to ${to.catalogString} as it may truncate\n" + + s"${from.dataType.catalogString} to ${to.catalogString}.\n" + "The type path of the target object is:\n" + walkedTypePath.mkString("", "\n", "\n") + "You can either add an explicit cast to the input data or choose a higher precision " + "type of the field in the target object") @@ -2501,11 +2653,15 @@ class Analyzer( case p => p transformExpressions { case u @ UpCast(child, _, _) if !child.resolved => u - case UpCast(child, dataType, walkedTypePath) - if Cast.mayTruncate(child.dataType, dataType) => + case UpCast(child, dt: AtomicType, _) + if SQLConf.get.getConf(SQLConf.LEGACY_LOOSE_UPCAST) && + child.dataType == StringType => + Cast(child, dt.asNullable) + + case UpCast(child, dataType, walkedTypePath) if !Cast.canUpCast(child.dataType, dataType) => fail(child, dataType, walkedTypePath) - case UpCast(child, dataType, walkedTypePath) => Cast(child, dataType.asNullable) + case UpCast(child, dataType, _) => Cast(child, dataType.asNullable) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala new file mode 100644 index 0000000000000..60e6bf8db06d7 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/CTESubstitution.scala @@ -0,0 +1,136 @@ +/* + * 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.analysis + +import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, With} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf.LEGACY_CTE_PRECEDENCE_ENABLED + +/** + * Analyze WITH nodes and substitute child plan with CTE definitions. + */ +object CTESubstitution extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + if (SQLConf.get.getConf(LEGACY_CTE_PRECEDENCE_ENABLED)) { + legacyTraverseAndSubstituteCTE(plan) + } else { + traverseAndSubstituteCTE(plan, false) + } + } + + private def legacyTraverseAndSubstituteCTE(plan: LogicalPlan): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child, relations) => + // substitute CTE expressions right-to-left to resolve references to previous CTEs: + // with a as (select * from t), b as (select * from a) select * from b + relations.foldRight(child) { + case ((cteName, ctePlan), currentPlan) => substituteCTE(currentPlan, cteName, ctePlan) + } + } + } + + /** + * Traverse the plan and expression nodes as a tree and replace matching references to CTE + * definitions. + * - If the rule encounters a WITH node then it substitutes the child of the node with CTE + * definitions of the node right-to-left order as a definition can reference to a previous + * one. + * For example the following query is valid: + * WITH + * t AS (SELECT 1), + * t2 AS (SELECT * FROM t) + * SELECT * FROM t2 + * - If a CTE definition contains an inner WITH node then substitution of inner should take + * precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH + * t AS (SELECT 1), + * t2 AS ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * SELECT * FROM t2 + * - If a CTE definition contains a subquery that contains an inner WITH node then substitution + * of inner should take precedence because it can shadow an outer CTE definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1 AS c) + * SELECT max(c) FROM ( + * WITH t AS (SELECT 2 AS c) + * SELECT * FROM t + * ) + * - If a CTE definition contains a subquery expression that contains an inner WITH node then + * substitution of inner should take precedence because it can shadow an outer CTE + * definition. + * For example the following query should return 2: + * WITH t AS (SELECT 1) + * SELECT ( + * WITH t AS (SELECT 2) + * SELECT * FROM t + * ) + * @param plan the plan to be traversed + * @param inTraverse whether the current traverse is called from another traverse, only in this + * case name collision can occur + * @return the plan where CTE substitution is applied + */ + private def traverseAndSubstituteCTE(plan: LogicalPlan, inTraverse: Boolean): LogicalPlan = { + plan.resolveOperatorsUp { + case With(child: LogicalPlan, relations) => + // child might contain an inner CTE that has priority so traverse and substitute inner CTEs + // in child first + val traversedChild: LogicalPlan = child transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + + // Substitute CTE definitions from last to first as a CTE definition can reference a + // previous one + relations.foldRight(traversedChild) { + case ((cteName, ctePlan), currentPlan) => + // A CTE definition might contain an inner CTE that has priority, so traverse and + // substitute CTE defined in ctePlan. + // A CTE definition might not be used at all or might be used multiple times. To avoid + // computation if it is not used and to avoid multiple recomputation if it is used + // multiple times we use a lazy construct with call-by-name parameter passing. + lazy val substitutedCTEPlan = traverseAndSubstituteCTE(ctePlan, true) + substituteCTE(currentPlan, cteName, substitutedCTEPlan) + } + + // CTE name collision can occur only when inTraverse is true, it helps to avoid eager CTE + // substitution in a subquery expression. + case other if inTraverse => + other.transformExpressions { + case e: SubqueryExpression => e.withNewPlan(traverseAndSubstituteCTE(e.plan, true)) + } + } + } + + private def substituteCTE( + plan: LogicalPlan, + cteName: String, + ctePlan: => LogicalPlan): LogicalPlan = + plan resolveOperatorsUp { + case UnresolvedRelation(Seq(table)) if plan.conf.resolver(cteName, table) => ctePlan + + case other => + // This cannot be done in ResolveSubquery because ResolveSubquery does not know the CTE. + other transformExpressions { + case e: SubqueryExpression => e.withNewPlan(substituteCTE(e.plan, cteName, ctePlan)) + } + } +} 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 6a91d556b2f3e..ae19d02e4475e 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 @@ -19,12 +19,14 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, DeleteColumn, RenameColumn, UpdateColumnComment, UpdateColumnType} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -32,6 +34,8 @@ import org.apache.spark.sql.types._ */ trait CheckAnalysis extends PredicateHelper { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + /** * Override to provide additional checks for correct analysis. * These rules will be evaluated after our built-in check rules. @@ -87,7 +91,7 @@ trait CheckAnalysis extends PredicateHelper { case p if p.analyzed => // Skip already analyzed sub-plans case u: UnresolvedRelation => - u.failAnalysis(s"Table or view not found: ${u.tableIdentifier}") + u.failAnalysis(s"Table or view not found: ${u.multipartIdentifier.quoted}") case operator: LogicalPlan => // Check argument data types of higher-order functions downwards first. @@ -106,7 +110,7 @@ trait CheckAnalysis extends PredicateHelper { operator transformExpressionsUp { case a: Attribute if !a.resolved => - val from = operator.inputSet.map(_.qualifiedName).mkString(", ") + val from = operator.inputSet.toSeq.map(_.qualifiedName).mkString(", ") a.failAnalysis(s"cannot resolve '${a.sql}' given input columns: [$from]") case e: Expression if e.checkInputDataTypes().isFailure => @@ -134,11 +138,6 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis("An offset window function can only be evaluated in an ordered " + s"row-based window frame with a single offset: $w") - case _ @ WindowExpression(_: PythonUDF, - WindowSpecDefinition(_, _, frame: SpecifiedWindowFrame)) - if !frame.isUnbounded => - failAnalysis("Only unbounded window frame is supported with Pandas UDFs.") - case w @ WindowExpression(e, s) => // Only allow window functions with an aggregate expression or an offset window // function or a Pandas window UDF. @@ -176,13 +175,13 @@ trait CheckAnalysis extends PredicateHelper { failAnalysis("Null-aware predicate sub-queries cannot be used in nested " + s"conditions: $condition") - case j @ Join(_, _, _, Some(condition)) if condition.dataType != BooleanType => + case j @ Join(_, _, _, Some(condition), _) if condition.dataType != BooleanType => failAnalysis( s"join condition '${condition.sql}' " + s"of type ${condition.dataType.catalogString} is not a boolean.") case Aggregate(groupingExprs, aggregateExprs, child) => - def isAggregateExpression(expr: Expression) = { + def isAggregateExpression(expr: Expression): Boolean = { expr.isInstanceOf[AggregateExpression] || PythonUDF.isGroupedAggPandasUDF(expr) } @@ -300,6 +299,114 @@ trait CheckAnalysis extends PredicateHelper { } } + case CreateTableAsSelect(_, _, partitioning, query, _, _, _) => + val references = partitioning.flatMap(_.references).toSet + val badReferences = references.map(_.fieldNames).flatMap { column => + query.schema.findNestedField(column) match { + case Some(_) => + None + case _ => + Some(s"${column.quoted} is missing or is in a map or array") + } + } + + if (badReferences.nonEmpty) { + failAnalysis(s"Invalid partitioning: ${badReferences.mkString(", ")}") + } + + // If the view output doesn't have the same number of columns neither with the child + // output, nor with the query column names, throw an AnalysisException. + // If the view's child output can't up cast to the view output, + // throw an AnalysisException, too. + case v @ View(desc, output, child) if child.resolved && !v.sameOutput(child) => + val queryColumnNames = desc.viewQueryColumnNames + val queryOutput = if (queryColumnNames.nonEmpty) { + if (output.length != queryColumnNames.length) { + // If the view output doesn't have the same number of columns with the query column + // names, throw an AnalysisException. + throw new AnalysisException( + s"The view output ${output.mkString("[", ",", "]")} doesn't have the same" + + "number of columns with the query column names " + + s"${queryColumnNames.mkString("[", ",", "]")}") + } + val resolver = SQLConf.get.resolver + queryColumnNames.map { colName => + child.output.find { attr => + resolver(attr.name, colName) + }.getOrElse(throw new AnalysisException( + s"Attribute with name '$colName' is not found in " + + s"'${child.output.map(_.name).mkString("(", ",", ")")}'")) + } + } else { + child.output + } + + output.zip(queryOutput).foreach { + case (attr, originAttr) if !attr.dataType.sameType(originAttr.dataType) => + // The dataType of the output attributes may be not the same with that of the view + // output, so we should cast the attribute to the dataType of the view output + // attribute. Will throw an AnalysisException if the cast is not a up-cast. + if (!Cast.canUpCast(originAttr.dataType, attr.dataType)) { + throw new AnalysisException(s"Cannot up cast ${originAttr.sql} from " + + s"${originAttr.dataType.catalogString} to ${attr.dataType.catalogString} " + + "as it may truncate\n") + } + case _ => + } + + case alter: AlterTable if alter.childrenResolved => + val table = alter.table + def findField(operation: String, fieldName: Array[String]): StructField = { + // include collections because structs nested in maps and arrays may be altered + val field = table.schema.findNestedField(fieldName, includeCollections = true) + if (field.isEmpty) { + throw new AnalysisException( + s"Cannot $operation missing field in ${table.name} schema: ${fieldName.quoted}") + } + field.get + } + + alter.changes.foreach { + case add: AddColumn => + val parent = add.fieldNames.init + if (parent.nonEmpty) { + findField("add to", parent) + } + case update: UpdateColumnType => + val field = findField("update", update.fieldNames) + val fieldName = update.fieldNames.quoted + update.newDataType match { + case _: StructType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a struct by adding, deleting, or updating its fields") + case _: MapType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update a map by updating $fieldName.key or $fieldName.value") + case _: ArrayType => + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName type: " + + s"update the element by updating $fieldName.element") + case _: AtomicType => + // update is okay + } + if (!Cast.canUpCast(field.dataType, update.newDataType)) { + throw new AnalysisException( + s"Cannot update ${table.name} field $fieldName: " + + s"${field.dataType.simpleString} cannot be cast to " + + s"${update.newDataType.simpleString}") + } + case rename: RenameColumn => + findField("rename", rename.fieldNames) + case update: UpdateColumnComment => + findField("update", update.fieldNames) + case delete: DeleteColumn => + findField("delete", delete.fieldNames) + case _ => + // no validation needed for set and remove property + } + case _ => // Fallbacks to the following checks } @@ -308,7 +415,7 @@ trait CheckAnalysis extends PredicateHelper { val missingAttributes = o.missingInput.mkString(",") val input = o.inputSet.mkString(",") val msgForMissingAttributes = s"Resolved attribute(s) $missingAttributes missing " + - s"from $input in operator ${operator.simpleString}." + s"from $input in operator ${operator.simpleString(SQLConf.get.maxToStringFields)}." val resolver = plan.conf.resolver val attrsWithSameName = o.missingInput.filter { missing => @@ -373,19 +480,42 @@ trait CheckAnalysis extends PredicateHelper { s"""nondeterministic expressions are only allowed in |Project, Filter, Aggregate or Window, found: | ${o.expressions.map(_.sql).mkString(",")} - |in operator ${operator.simpleString} + |in operator ${operator.simpleString(SQLConf.get.maxToStringFields)} """.stripMargin) case _: UnresolvedHint => throw new IllegalStateException( "Internal error: logical hint operator should have been removed during analysis") + case InsertIntoTable(u: UnresolvedRelation, _, _, _, _) => + failAnalysis(s"Table not found: ${u.multipartIdentifier.quoted}") + + case f @ Filter(condition, _) + if PlanHelper.specialExpressionsInUnsupportedOperator(f).nonEmpty => + val invalidExprSqls = PlanHelper.specialExpressionsInUnsupportedOperator(f).map(_.sql) + failAnalysis( + s""" + |Aggregate/Window/Generate expressions are not valid in where clause of the query. + |Expression in where clause: [${condition.sql}] + |Invalid expressions: [${invalidExprSqls.mkString(", ")}]""".stripMargin) + + case other if PlanHelper.specialExpressionsInUnsupportedOperator(other).nonEmpty => + val invalidExprSqls = + PlanHelper.specialExpressionsInUnsupportedOperator(other).map(_.sql) + failAnalysis( + s""" + |The query operator `${other.nodeName}` contains one or more unsupported + |expression types Aggregate, Window or Generate. + |Invalid expressions: [${invalidExprSqls.mkString(", ")}]""".stripMargin + ) + case _ => // Analysis successful! } } extendedCheckRules.foreach(_(plan)) plan.foreachUp { - case o if !o.resolved => failAnalysis(s"unresolved operator ${o.simpleString}") + case o if !o.resolved => + failAnalysis(s"unresolved operator ${o.simpleString(SQLConf.get.maxToStringFields)}") case _ => } @@ -612,7 +742,7 @@ trait CheckAnalysis extends PredicateHelper { failOnNonEqualCorrelatedPredicate(foundNonEqualCorrelatedPred, a) // Join can host correlated expressions. - case j @ Join(left, right, joinType, _) => + case j @ Join(left, right, joinType, _, _) => joinType match { // Inner join, like Filter, can be anywhere. case _: InnerLike => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala index 82692334544e2..24cb6b301dfe0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecision.scala @@ -82,6 +82,8 @@ object DecimalPrecision extends TypeCoercionRule { PromotePrecision(Cast(e, dataType)) } + private def nullOnOverflow: Boolean = SQLConf.get.decimalOperationsNullOnOverflow + override protected def coerceTypes(plan: LogicalPlan): LogicalPlan = plan resolveOperators { // fix decimal precision for expressions case q => q.transformExpressionsUp( @@ -105,7 +107,7 @@ object DecimalPrecision extends TypeCoercionRule { DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) } CheckOverflow(Add(promotePrecision(e1, resultType), promotePrecision(e2, resultType)), - resultType) + resultType, nullOnOverflow) case Subtract(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultScale = max(s1, s2) @@ -116,7 +118,7 @@ object DecimalPrecision extends TypeCoercionRule { DecimalType.bounded(max(p1 - s1, p2 - s2) + resultScale + 1, resultScale) } CheckOverflow(Subtract(promotePrecision(e1, resultType), promotePrecision(e2, resultType)), - resultType) + resultType, nullOnOverflow) case Multiply(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { @@ -126,7 +128,7 @@ object DecimalPrecision extends TypeCoercionRule { } val widerType = widerDecimalType(p1, s1, p2, s2) CheckOverflow(Multiply(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), - resultType) + resultType, nullOnOverflow) case Divide(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { @@ -148,7 +150,7 @@ object DecimalPrecision extends TypeCoercionRule { } val widerType = widerDecimalType(p1, s1, p2, s2) CheckOverflow(Divide(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), - resultType) + resultType, nullOnOverflow) case Remainder(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { @@ -159,7 +161,7 @@ object DecimalPrecision extends TypeCoercionRule { // resultType may have lower precision, so we cast them into wider type first. val widerType = widerDecimalType(p1, s1, p2, s2) CheckOverflow(Remainder(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), - resultType) + resultType, nullOnOverflow) case Pmod(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) => val resultType = if (SQLConf.get.decimalOperationsAllowPrecisionLoss) { @@ -170,7 +172,7 @@ object DecimalPrecision extends TypeCoercionRule { // resultType may have lower precision, so we cast them into wider type first. val widerType = widerDecimalType(p1, s1, p2, s2) CheckOverflow(Pmod(promotePrecision(e1, widerType), promotePrecision(e2, widerType)), - resultType) + resultType, nullOnOverflow) case b @ BinaryComparison(e1 @ DecimalType.Expression(p1, s1), e2 @ DecimalType.Expression(p2, s2)) if p1 != p2 || s1 != s2 => 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 7dafebff79874..c72400a8b72c2 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 @@ -21,10 +21,10 @@ import java.util.Locale import javax.annotation.concurrent.GuardedBy import scala.collection.mutable -import scala.language.existentials import scala.reflect.ClassTag import scala.util.{Failure, Success, Try} +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder @@ -87,7 +87,7 @@ trait FunctionRegistry { override def clone(): FunctionRegistry = throw new CloneNotSupportedException() } -class SimpleFunctionRegistry extends FunctionRegistry { +class SimpleFunctionRegistry extends FunctionRegistry with Logging { @GuardedBy("this") private val functionBuilders = @@ -103,7 +103,13 @@ class SimpleFunctionRegistry extends FunctionRegistry { name: FunctionIdentifier, info: ExpressionInfo, builder: FunctionBuilder): Unit = synchronized { - functionBuilders.put(normalizeFuncName(name), (info, builder)) + val normalizedName = normalizeFuncName(name) + val newFunction = (info, builder) + functionBuilders.put(normalizedName, newFunction) match { + case Some(previousFunction) if previousFunction != newFunction => + logWarning(s"The function $normalizedName replaced a previously registered function.") + case _ => + } } override def lookupFunction(name: FunctionIdentifier, children: Seq[Expression]): Expression = { @@ -216,9 +222,12 @@ object FunctionRegistry { // math functions expression[Acos]("acos"), + expression[Acosh]("acosh"), expression[Asin]("asin"), + expression[Asinh]("asinh"), expression[Atan]("atan"), expression[Atan2]("atan2"), + expression[Atanh]("atanh"), expression[Bin]("bin"), expression[BRound]("bround"), expression[Cbrt]("cbrt"), @@ -275,6 +284,7 @@ object FunctionRegistry { expression[Average]("avg"), expression[Corr]("corr"), expression[Count]("count"), + expression[CountIf]("count_if"), expression[CovPopulation]("covar_pop"), expression[CovSample]("covar_samp"), expression[First]("first"), @@ -283,8 +293,10 @@ object FunctionRegistry { expression[Last]("last"), expression[Last]("last_value"), expression[Max]("max"), + expression[MaxBy]("max_by"), expression[Average]("mean"), expression[Min]("min"), + expression[MinBy]("min_by"), expression[Percentile]("percentile"), expression[Skewness]("skewness"), expression[ApproximatePercentile]("percentile_approx"), @@ -300,6 +312,9 @@ object FunctionRegistry { expression[CollectList]("collect_list"), expression[CollectSet]("collect_set"), expression[CountMinSketchAgg]("count_min_sketch"), + expression[EveryAgg]("every"), + expression[AnyAgg]("any"), + expression[SomeAgg]("some"), // string functions expression[Ascii]("ascii"), @@ -336,6 +351,7 @@ object FunctionRegistry { expression[RegExpReplace]("regexp_replace"), expression[StringRepeat]("repeat"), expression[StringReplace]("replace"), + expression[Overlay]("overlay"), expression[RLike]("rlike"), expression[StringRPad]("rpad"), expression[StringTrimRight]("rtrim"), @@ -451,6 +467,7 @@ object FunctionRegistry { expression[Md5]("md5"), expression[Uuid]("uuid"), expression[Murmur3Hash]("hash"), + expression[XxHash64]("xxhash64"), expression[Sha1]("sha"), expression[Sha1]("sha1"), expression[Sha2]("sha2"), @@ -520,7 +537,12 @@ object FunctionRegistry { castAlias("date", DateType), castAlias("timestamp", TimestampType), castAlias("binary", BinaryType), - castAlias("string", StringType) + castAlias("string", StringType), + + // csv + expression[CsvToStructs]("from_csv"), + expression[SchemaOfCsv]("schema_of_csv"), + expression[StructsToCsv]("to_csv") ) val builtin: SimpleFunctionRegistry = { @@ -605,7 +627,7 @@ object FunctionRegistry { val clazz = scala.reflect.classTag[Cast].runtimeClass val usage = "_FUNC_(expr) - Casts the value `expr` to the target data type `_FUNC_`." val expressionInfo = - new ExpressionInfo(clazz.getCanonicalName, null, name, usage, "", "", "", "") + new ExpressionInfo(clazz.getCanonicalName, null, name, usage, "", "", "", "", "") (name, (expressionInfo, builder)) } @@ -625,7 +647,8 @@ object FunctionRegistry { df.arguments(), df.examples(), df.note(), - df.since()) + df.since(), + df.deprecated()) } else { // This exists for the backward compatibility with old `ExpressionDescription`s defining // the extended description in `extended()`. 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 new file mode 100644 index 0000000000000..c6e0c74527b8f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HintErrorLogger.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.analysis + +import org.apache.spark.internal.Logging +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 { + + 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 " + + s"'${hintToPrettyString(name, parameters)}'.") + } + } + + override def joinNotFoundForJoinHint(hint: HintInfo): Unit = { + logWarning(s"A join hint $hint is specified but it is not part of a join relation.") + } + + override def hintOverridden(hint: HintInfo): Unit = { + logWarning(s"Hint $hint is overridden by another hint and will not take effect.") + } + + private def hintToPrettyString(name: String, parameters: Seq[Any]): String = { + val prettyParameters = parameters.map { + case a: UnresolvedAttribute => a.nameParts.mkString(".") + case e: Any => e.toString + } + s"$name${prettyParameters.mkString("(", ", ", ")")}" + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NamedRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NamedRelation.scala index ad201f947b671..56b8d84441c95 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NamedRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NamedRelation.scala @@ -21,4 +21,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan trait NamedRelation extends LogicalPlan { def name: String + + // When false, the schema of input data must match the schema of this relation, during write. + def skipSchemaResolution: Boolean = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala index f5aae60431c15..7ac8ae61ed537 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/NoSuchItemException.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ +import org.apache.spark.sql.catalog.v2.Identifier import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec @@ -25,10 +27,24 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec * Thrown by a catalog when an item cannot be found. The analyzer will rethrow the exception * as an [[org.apache.spark.sql.AnalysisException]] with the correct position information. */ -class NoSuchDatabaseException(val db: String) extends AnalysisException(s"Database '$db' not found") +class NoSuchDatabaseException( + val db: String) extends NoSuchNamespaceException(s"Database '$db' not found") -class NoSuchTableException(db: String, table: String) - extends AnalysisException(s"Table or view '$table' not found in database '$db'") +class NoSuchNamespaceException(message: String) extends AnalysisException(message) { + def this(namespace: Array[String]) = { + this(s"Namespace '${namespace.quoted}' not found") + } +} + +class NoSuchTableException(message: String) extends AnalysisException(message) { + def this(db: String, table: String) = { + this(s"Table or view '$table' not found in database '$db'") + } + + def this(tableIdent: Identifier) = { + this(s"Table ${tableIdent.quoted} not found") + } +} class NoSuchPartitionException( db: String, @@ -40,10 +56,10 @@ class NoSuchPartitionException( class NoSuchPermanentFunctionException(db: String, func: String) extends AnalysisException(s"Function '$func' not found in database '$db'") -class NoSuchFunctionException(db: String, func: String) +class NoSuchFunctionException(db: String, func: String, cause: Option[Throwable] = None) extends AnalysisException( s"Undefined function: '$func'. This function is neither a registered temporary function nor " + - s"a permanent function registered in the database '$db'.") + s"a permanent function registered in the database '$db'.", cause = cause) class NoSuchPartitionsException(db: String, table: String, specs: Seq[TablePartitionSpec]) extends AnalysisException( 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 dbd4ed845e329..3a9c4b7392e38 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale +import scala.collection.mutable + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.IntegerLiteral import org.apache.spark.sql.catalyst.plans.logical._ @@ -28,45 +30,70 @@ import org.apache.spark.sql.internal.SQLConf /** - * Collection of rules related to hints. The only hint currently available is broadcast join hint. + * Collection of rules related to hints. The only hint currently available is join strategy hint. * * Note that this is separately into two rules because in the future we might introduce new hint - * rules that have different ordering requirements from broadcast. + * rules that have different ordering requirements from join strategies. */ object ResolveHints { /** - * For broadcast hint, we accept "BROADCAST", "BROADCASTJOIN", and "MAPJOIN", and a sequence of - * relation aliases can be specified in the hint. A broadcast hint plan node will be inserted - * on top of any relation (that is not aliased differently), subquery, or common table expression - * that match the specified name. + * The list of allowed join strategy hints is defined in [[JoinStrategyHint.strategies]], and a + * sequence of relation aliases can be specified with a join strategy hint, e.g., "MERGE(a, c)", + * "BROADCAST(a)". A join strategy hint plan node will be inserted on top of any relation (that + * is not aliased differently), subquery, or common table expression that match the specified + * name. * * The hint resolution works by recursively traversing down the query plan to find a relation or - * subquery that matches one of the specified broadcast aliases. The traversal does not go past - * beyond any existing broadcast hints, subquery aliases. + * subquery that matches one of the specified relation aliases. The traversal does not go past + * beyond any view reference, with clause or subquery alias. * * This rule must happen before common table expressions. */ - class ResolveBroadcastHints(conf: SQLConf) extends Rule[LogicalPlan] { - private val BROADCAST_HINT_NAMES = Set("BROADCAST", "BROADCASTJOIN", "MAPJOIN") + class ResolveJoinStrategyHints(conf: SQLConf) extends Rule[LogicalPlan] { + private val STRATEGY_HINT_NAMES = JoinStrategyHint.strategies.flatMap(_.hintAliases) + + private val hintErrorHandler = conf.hintErrorHandler def resolver: Resolver = conf.resolver - private def applyBroadcastHint(plan: LogicalPlan, toBroadcast: Set[String]): LogicalPlan = { + private def createHintInfo(hintName: String): HintInfo = { + HintInfo(strategy = + JoinStrategyHint.strategies.find( + _.hintAliases.map( + _.toUpperCase(Locale.ROOT)).contains(hintName.toUpperCase(Locale.ROOT)))) + } + + private def applyJoinStrategyHint( + plan: LogicalPlan, + relations: mutable.HashSet[String], + hintName: String): LogicalPlan = { // Whether to continue recursing down the tree var recurse = true val newNode = CurrentOrigin.withOrigin(plan.origin) { plan match { - case u: UnresolvedRelation if toBroadcast.exists(resolver(_, u.tableIdentifier.table)) => - ResolvedHint(plan, HintInfo(broadcast = true)) - case r: SubqueryAlias if toBroadcast.exists(resolver(_, r.alias)) => - ResolvedHint(plan, HintInfo(broadcast = true)) + case ResolvedHint(u @ UnresolvedRelation(ident), hint) + if relations.exists(resolver(_, ident.last)) => + relations.remove(ident.last) + ResolvedHint(u, createHintInfo(hintName).merge(hint, hintErrorHandler)) + + case ResolvedHint(r: SubqueryAlias, hint) + if relations.exists(resolver(_, r.alias)) => + relations.remove(r.alias) + ResolvedHint(r, createHintInfo(hintName).merge(hint, hintErrorHandler)) + + case u @ UnresolvedRelation(ident) if relations.exists(resolver(_, ident.last)) => + relations.remove(ident.last) + ResolvedHint(plan, createHintInfo(hintName)) + + case r: SubqueryAlias if relations.exists(resolver(_, r.alias)) => + relations.remove(r.alias) + ResolvedHint(plan, createHintInfo(hintName)) case _: ResolvedHint | _: View | _: With | _: SubqueryAlias => // Don't traverse down these nodes. - // For an existing broadcast hint, there is no point going down (if we do, we either - // won't change the structure, or will introduce another broadcast hint that is useless. + // For an existing strategy hint, there is no chance for a match from this point down. // The rest (view, with, subquery) indicates different scopes that we shouldn't traverse // down. Note that technically when this rule is executed, we haven't completed view // resolution yet and as a result the view part should be deadcode. I'm leaving it here @@ -80,25 +107,31 @@ object ResolveHints { } if ((plan fastEquals newNode) && recurse) { - newNode.mapChildren(child => applyBroadcastHint(child, toBroadcast)) + newNode.mapChildren(child => applyJoinStrategyHint(child, relations, hintName)) } else { newNode } } def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case h: UnresolvedHint if BROADCAST_HINT_NAMES.contains(h.name.toUpperCase(Locale.ROOT)) => + case h: UnresolvedHint if STRATEGY_HINT_NAMES.contains(h.name.toUpperCase(Locale.ROOT)) => if (h.parameters.isEmpty) { - // If there is no table alias specified, turn the entire subtree into a BroadcastHint. - ResolvedHint(h.child, HintInfo(broadcast = true)) + // If there is no table alias specified, apply the hint on the entire subtree. + ResolvedHint(h.child, createHintInfo(h.name)) } else { - // Otherwise, find within the subtree query plans that should be broadcasted. - applyBroadcastHint(h.child, h.parameters.map { + // 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 - case unsupported => throw new AnalysisException("Broadcast hint parameter should be " + - s"an identifier or string but was $unsupported (${unsupported.getClass}") - }.toSet) + 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) + applied } } } @@ -133,9 +166,14 @@ object ResolveHints { * Removes all the hints, used to remove invalid hints provided by the user. * This must be executed after all the other hint rules are executed. */ - object RemoveAllHints extends Rule[LogicalPlan] { + class RemoveAllHints(conf: SQLConf) extends Rule[LogicalPlan] { + + private val hintErrorHandler = conf.hintErrorHandler + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case h: UnresolvedHint => h.child + case h: UnresolvedHint => + hintErrorHandler.hintNotRecognized(h.name, h.parameters) + h.child } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala index 7a0aa08289efa..c1d72f9b58a4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/StreamingJoinHelper.scala @@ -41,7 +41,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging { */ def isWatermarkInJoinKeys(plan: LogicalPlan): Boolean = { plan match { - case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _) => + case ExtractEquiJoinKeys(_, leftKeys, rightKeys, _, _, _, _) => (leftKeys ++ rightKeys).exists { case a: AttributeReference => a.metadata.contains(EventTimeWatermark.delayKey) case _ => false @@ -236,7 +236,7 @@ object StreamingJoinHelper extends PredicateHelper with Logging { collect(left, negate) ++ collect(right, !negate) case UnaryMinus(child) => collect(child, !negate) - case CheckOverflow(child, _) => + case CheckOverflow(child, _, _) => collect(child, negate) case PromotePrecision(child) => collect(child, negate) 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 72ac80e0a0a18..3125f8cb732db 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 @@ -59,7 +59,7 @@ object TypeCoercion { CaseWhenCoercion :: IfCoercion :: StackCoercion :: - Division :: + Division(conf) :: ImplicitTypeCasts :: DateTimeOperations :: WindowFrameCoercion :: @@ -120,13 +120,14 @@ object TypeCoercion { */ private def findCommonTypeForBinaryComparison( dt1: DataType, dt2: DataType, conf: SQLConf): Option[DataType] = (dt1, dt2) match { - // We should cast all relative timestamp/date/string comparison into string comparisons - // This behaves as a user would expect because timestamp strings sort lexicographically. - // i.e. TimeStamp(2013-01-01 00:00 ...) < "2014" = true - case (StringType, DateType) => Some(StringType) - case (DateType, StringType) => Some(StringType) - case (StringType, TimestampType) => Some(StringType) - case (TimestampType, StringType) => Some(StringType) + case (StringType, DateType) + => if (conf.castDatetimeToString) Some(StringType) else Some(DateType) + case (DateType, StringType) + => if (conf.castDatetimeToString) Some(StringType) else Some(DateType) + case (StringType, TimestampType) + => if (conf.castDatetimeToString) Some(StringType) else Some(TimestampType) + case (TimestampType, StringType) + => if (conf.castDatetimeToString) Some(StringType) else Some(TimestampType) case (StringType, NullType) => Some(StringType) case (NullType, StringType) => Some(StringType) @@ -181,8 +182,9 @@ object TypeCoercion { } /** - * The method finds a common type for data types that differ only in nullable, containsNull - * and valueContainsNull flags. If the input types are too different, None is returned. + * The method finds a common type for data types that differ only in nullable flags, including + * `nullable`, `containsNull` of [[ArrayType]] and `valueContainsNull` of [[MapType]]. + * If the input types are different besides nullable flags, None is returned. */ def findCommonTypeDifferentOnlyInNullFlags(t1: DataType, t2: DataType): Option[DataType] = { if (t1 == t2) { @@ -664,7 +666,7 @@ object TypeCoercion { * Hive only performs integral division with the DIV operator. The arguments to / are always * converted to fractional types. */ - object Division extends TypeCoercionRule { + case class Division(conf: SQLConf) extends TypeCoercionRule { override protected def coerceTypes( plan: LogicalPlan): LogicalPlan = plan resolveExpressions { // Skip nodes who has not been resolved yet, @@ -675,7 +677,12 @@ object TypeCoercion { case d: Divide if d.dataType == DoubleType => d case d: Divide if d.dataType.isInstanceOf[DecimalType] => d case Divide(left, right) if isNumericOrNull(left) && isNumericOrNull(right) => - Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + (left.dataType, right.dataType) match { + case (_: IntegralType, _: IntegralType) if conf.preferIntegralDivision => + IntegralDivide(left, right) + case _ => + Divide(Cast(left, DoubleType), Cast(right, DoubleType)) + } } private def isNumericOrNull(ex: Expression): Boolean = { @@ -818,8 +825,10 @@ object TypeCoercion { } /** - * Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType - * to TimeAdd/TimeSub + * 1. Turns Add/Subtract of DateType/TimestampType/StringType and CalendarIntervalType + * to TimeAdd/TimeSub. + * 2. Turns Add/Subtract of DateType/IntegerType and IntegerType/DateType + * to DateAdd/DateSub/DateDiff. */ object DateTimeOperations extends Rule[LogicalPlan] { @@ -835,6 +844,11 @@ object TypeCoercion { Cast(TimeAdd(l, r), l.dataType) case Subtract(l, r @ CalendarIntervalType()) if acceptedTypes.contains(l.dataType) => Cast(TimeSub(l, r), l.dataType) + + case Add(l @ DateType(), r @ IntegerType()) => DateAdd(l, r) + case Add(l @ IntegerType(), r @ DateType()) => DateAdd(r, l) + case Subtract(l @ DateType(), r @ IntegerType()) => DateSub(l, r) + case Subtract(l @ DateType(), r @ DateType()) => DateDiff(l, r) } } @@ -878,6 +892,48 @@ object TypeCoercion { } } e.withNewChildren(children) + + case udf: ScalaUDF if udf.inputTypes.nonEmpty => + val children = udf.children.zip(udf.inputTypes).map { case (in, expected) => + // Currently Scala UDF will only expect `AnyDataType` at top level, so this trick works. + // In the future we should create types like `AbstractArrayType`, so that Scala UDF can + // accept inputs of array type of arbitrary element type. + if (expected == AnyDataType) { + in + } else { + implicitCast( + in, + udfInputToCastType(in.dataType, expected.asInstanceOf[DataType]) + ).getOrElse(in) + } + + } + udf.withNewChildren(children) + } + + private def udfInputToCastType(input: DataType, expectedType: DataType): DataType = { + (input, expectedType) match { + // SPARK-26308: avoid casting to an arbitrary precision and scale for decimals. Please note + // that precision and scale cannot be inferred properly for a ScalaUDF because, when it is + // created, it is not bound to any column. So here the precision and scale of the input + // column is used. + case (in: DecimalType, _: DecimalType) => in + case (ArrayType(dtIn, _), ArrayType(dtExp, nullableExp)) => + ArrayType(udfInputToCastType(dtIn, dtExp), nullableExp) + case (MapType(keyDtIn, valueDtIn, _), MapType(keyDtExp, valueDtExp, nullableExp)) => + MapType(udfInputToCastType(keyDtIn, keyDtExp), + udfInputToCastType(valueDtIn, valueDtExp), + nullableExp) + case (StructType(fieldsIn), StructType(fieldsExp)) => + val fieldTypes = + fieldsIn.map(_.dataType).zip(fieldsExp.map(_.dataType)).map { case (dtIn, dtExp) => + udfInputToCastType(dtIn, dtExp) + } + StructType(fieldsExp.zip(fieldTypes).map { case (field, newDt) => + field.copy(dataType = newDt) + }) + case (_, other) => other + } } /** @@ -1037,8 +1093,8 @@ trait TypeCoercionRule extends Rule[LogicalPlan] with Logging { // Leave the same if the dataTypes match. case Some(newType) if a.dataType == newType.dataType => a case Some(newType) => - logDebug( - s"Promoting $a from ${a.dataType} to ${newType.dataType} in ${q.simpleString}") + logDebug(s"Promoting $a from ${a.dataType} to ${newType.dataType} in " + + s" ${q.simpleString(SQLConf.get.maxToStringFields)}") newType } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala index cff4cee09427f..288ff1a04737e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationChecker.scala @@ -229,16 +229,15 @@ object UnsupportedOperationChecker { throwError("dropDuplicates is not supported after aggregation on a " + "streaming DataFrame/Dataset") - case Join(left, right, joinType, condition) => + case Join(left, right, joinType, condition, _) => + if (left.isStreaming && right.isStreaming && outputMode != InternalOutputModes.Append) { + throwError("Join between two streaming DataFrames/Datasets is not supported" + + s" in ${outputMode} output mode, only in Append output mode") + } joinType match { - case _: InnerLike => - if (left.isStreaming && right.isStreaming && - outputMode != InternalOutputModes.Append) { - throwError("Inner join between two streaming DataFrames/Datasets is not supported" + - s" in ${outputMode} output mode, only in Append output mode") - } + // no further validations needed case FullOuter => if (left.isStreaming || right.isStreaming) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.scala new file mode 100644 index 0000000000000..3eae34da7e502 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/UpdateAttributeNullability.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.catalyst.analysis + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * Updates nullability of Attributes in a resolved LogicalPlan by using the nullability of + * corresponding Attributes of its children output Attributes. This step is needed because + * users can use a resolved AttributeReference in the Dataset API and outer joins + * can change the nullability of an AttribtueReference. Without this rule, a nullable column's + * nullable field can be actually set as non-nullable, which cause illegal optimization + * (e.g., NULL propagation) and wrong answers. + * See SPARK-13484 and SPARK-13801 for the concrete queries of this case. + */ +object UpdateAttributeNullability extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { + // Skip unresolved nodes. + case p if !p.resolved => p + // Skip leaf node, as it has no child and no need to update nullability. + case p: LeafNode => p + case p: LogicalPlan if p.childrenResolved => + val nullabilities = p.children.flatMap(c => c.output).groupBy(_.exprId).map { + // If there are multiple Attributes having the same ExprId, we need to resolve + // the conflict of nullable field. We do not really expect this to happen. + case (exprId, attributes) => exprId -> attributes.exists(_.nullable) + } + // For an Attribute used by the current LogicalPlan, if it is from its children, + // we fix the nullable field by using the nullability setting of the corresponding + // output Attribute from the children. + p.transformExpressions { + case attr: Attribute if nullabilities.contains(attr.exprId) => + attr.withNullability(nullabilities(attr.exprId)) + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala index a8a7bbd9f9cd0..1cd7f412bb678 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/higherOrderFunctions.scala @@ -150,13 +150,14 @@ case class ResolveLambdaVariables(conf: SQLConf) extends Rule[LogicalPlan] { val lambdaMap = l.arguments.map(v => canonicalizer(v.name) -> v).toMap l.mapChildren(resolve(_, parentLambdaMap ++ lambdaMap)) - case u @ UnresolvedAttribute(name +: nestedFields) => + case u @ UnresolvedNamedLambdaVariable(name +: nestedFields) => parentLambdaMap.get(canonicalizer(name)) match { case Some(lambda) => nestedFields.foldLeft(lambda: Expression) { (expr, fieldName) => ExtractValue(expr, Literal(fieldName), conf.resolver) } - case None => u + case None => + UnresolvedAttribute(u.nameParts) } case _ => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala index c1ec736c32ed4..9e0e0d528a968 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -38,19 +38,27 @@ class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: Str /** * Holds the name of a relation that has yet to be looked up in a catalog. * - * @param tableIdentifier table name + * @param multipartIdentifier table name */ -case class UnresolvedRelation(tableIdentifier: TableIdentifier) - extends LeafNode { +case class UnresolvedRelation( + multipartIdentifier: Seq[String]) extends LeafNode with NamedRelation { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ /** Returns a `.` separated name for this relation. */ - def tableName: String = tableIdentifier.unquotedString + def tableName: String = multipartIdentifier.quoted + + override def name: String = tableName override def output: Seq[Attribute] = Nil override lazy val resolved = false } +object UnresolvedRelation { + def apply(tableIdentifier: TableIdentifier): UnresolvedRelation = + UnresolvedRelation(tableIdentifier.database.toSeq :+ tableIdentifier.table) +} + /** * An inline table that has not been resolved yet. Once resolved, it is turned by the analyzer into * a [[org.apache.spark.sql.catalyst.plans.logical.LocalRelation]]. @@ -112,6 +120,7 @@ case class UnresolvedAttribute(nameParts: Seq[String]) extends Attribute with Un override def withQualifier(newQualifier: Seq[String]): UnresolvedAttribute = this override def withName(newName: String): UnresolvedAttribute = UnresolvedAttribute.quoted(newName) override def withMetadata(newMetadata: Metadata): Attribute = this + override def withExprId(newExprId: ExprId): UnresolvedAttribute = this override def toString: String = s"'$name" @@ -203,10 +212,10 @@ case class UnresolvedGenerator(name: FunctionIdentifier, children: Seq[Expressio throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") override def terminate(): TraversableOnce[InternalRow] = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot terminate expression: $this") } case class UnresolvedFunction( @@ -407,7 +416,10 @@ case class ResolvedStar(expressions: Seq[NamedExpression]) extends Star with Une * can be key of Map, index of Array, field name of Struct. */ case class UnresolvedExtractValue(child: Expression, extraction: Expression) - extends UnaryExpression with Unevaluable { + extends BinaryExpression with Unevaluable { + + override def left: Expression = child + override def right: Expression = extraction override def dataType: DataType = throw new UnresolvedException(this, "dataType") override def foldable: Boolean = throw new UnresolvedException(this, "foldable") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala index af74693000c44..76bf3740ed0fc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast} +import org.apache.spark.sql.catalyst.expressions.Alias import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -28,7 +27,12 @@ import org.apache.spark.sql.internal.SQLConf */ /** - * Make sure that a view's child plan produces the view's output attributes. We try to wrap the + * This rule has two goals: + * + * 1. Removes [[View]] operators from the plan. The operator is respected till the end of analysis + * stage because we want to see which part of an analyzed logical plan is generated from a view. + * + * 2. Make sure that a view's child plan produces the view's output attributes. We try to wrap the * child by: * 1. Generate the `queryOutput` by: * 1.1. If the query column names are defined, map the column names to attributes in the child @@ -41,27 +45,29 @@ import org.apache.spark.sql.internal.SQLConf * 2. Map the `queryOutput` to view output by index, if the corresponding attributes don't match, * try to up cast and alias the attribute in `queryOutput` to the attribute in the view output. * 3. Add a Project over the child, with the new output generated by the previous steps. - * If the view output doesn't have the same number of columns neither with the child output, nor - * with the query column names, throw an AnalysisException. + * + * Once reaches this rule, it means `CheckAnalysis` did necessary checks on number of columns + * between the view output and the child output or the query column names. `CheckAnalysis` also + * checked the cast from the view's child to the Project is up-cast. * * This should be only done after the batch of Resolution, because the view attributes are not * completely resolved during the batch of Resolution. */ -case class AliasViewChild(conf: SQLConf) extends Rule[LogicalPlan] with CastSupport { - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperatorsUp { - case v @ View(desc, output, child) if child.resolved && output != child.output => +object EliminateView extends Rule[LogicalPlan] with CastSupport { + override def conf: SQLConf = SQLConf.get + + override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { + // The child has the different output attributes with the View operator. Adds a Project over + // the child of the view. + case v @ View(desc, output, child) if child.resolved && !v.sameOutput(child) => val resolver = conf.resolver val queryColumnNames = desc.viewQueryColumnNames val queryOutput = if (queryColumnNames.nonEmpty) { - // If the view output doesn't have the same number of columns with the query column names, - // throw an AnalysisException. - if (output.length != queryColumnNames.length) { - throw new AnalysisException( - s"The view output ${output.mkString("[", ",", "]")} doesn't have the same number of " + - s"columns with the query column names ${queryColumnNames.mkString("[", ",", "]")}") - } + // Find the attribute that has the expected attribute name from an attribute list, the names + // are compared using conf.resolver. + // `CheckAnalysis` already guarantees the expected attribute can be found for sure. desc.viewQueryColumnNames.map { colName => - findAttributeByName(colName, child.output, resolver) + child.output.find(attr => resolver(attr.name, colName)).get } } else { // For view created before Spark 2.2.0, the view text is already fully qualified, the plan @@ -70,52 +76,17 @@ case class AliasViewChild(conf: SQLConf) extends Rule[LogicalPlan] with CastSupp } // Map the attributes in the query output to the attributes in the view output by index. val newOutput = output.zip(queryOutput).map { - case (attr, originAttr) if attr != originAttr => - // The dataType of the output attributes may be not the same with that of the view - // output, so we should cast the attribute to the dataType of the view output attribute. - // Will throw an AnalysisException if the cast can't perform or might truncate. - if (Cast.mayTruncate(originAttr.dataType, attr.dataType)) { - throw new AnalysisException(s"Cannot up cast ${originAttr.sql} from " + - s"${originAttr.dataType.catalogString} to ${attr.dataType.catalogString} as it " + - s"may truncate\n") - } else { - Alias(cast(originAttr, attr.dataType), attr.name)(exprId = attr.exprId, - qualifier = attr.qualifier, explicitMetadata = Some(attr.metadata)) - } + case (attr, originAttr) if !attr.semanticEquals(originAttr) => + // `CheckAnalysis` already guarantees that the cast is a up-cast for sure. + Alias(cast(originAttr, attr.dataType), attr.name)(exprId = attr.exprId, + qualifier = attr.qualifier, explicitMetadata = Some(attr.metadata)) case (_, originAttr) => originAttr } - v.copy(child = Project(newOutput, child)) - } + Project(newOutput, child) - /** - * Find the attribute that has the expected attribute name from an attribute list, the names - * are compared using conf.resolver. - * If the expected attribute is not found, throw an AnalysisException. - */ - private def findAttributeByName( - name: String, - attrs: Seq[Attribute], - resolver: Resolver): Attribute = { - attrs.find { attr => - resolver(attr.name, name) - }.getOrElse(throw new AnalysisException( - s"Attribute with name '$name' is not found in " + - s"'${attrs.map(_.name).mkString("(", ",", ")")}'")) - } -} - -/** - * Removes [[View]] operators from the plan. The operator is respected till the end of analysis - * stage because we want to see which part of an analyzed logical plan is generated from a view. - */ -object EliminateView extends Rule[LogicalPlan] { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { // The child should have the same output attributes with the View operator, so we simply // remove the View operator. - case View(_, output, child) => - assert(output == child.output, - s"The output of the child ${child.output.mkString("[", ",", "]")} is different from the " + - s"view output ${output.mkString("[", ",", "]")}") + case View(_, _, child) => child } } 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 1a145c24d78cc..dcc143982a4aa 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 @@ -128,6 +128,8 @@ trait ExternalCatalog { def getTable(db: String, table: String): CatalogTable + def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] + def tableExists(db: String, table: String): Boolean def listTables(db: String): Seq[String] 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 2f009be5816fa..86113d3ec3ead 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 @@ -138,6 +138,10 @@ class ExternalCatalogWithListener(delegate: ExternalCatalog) delegate.getTable(db, table) } + override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] = { + delegate.getTablesByName(db, tables) + } + override def tableExists(db: String, table: String): Boolean = { delegate.tableExists(db, table) } 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 741dc46b07382..abf69939dea13 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 @@ -327,6 +327,11 @@ class InMemoryCatalog( catalog(db).tables(table).table } + override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] = { + requireDbExists(db) + tables.flatMap(catalog(db).tables.get).map(_.table) + } + override def tableExists(db: String, table: String): Boolean = synchronized { requireDbExists(db) catalog(db).tables.contains(table) 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 c11b444212946..74559f5d88796 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 @@ -39,6 +39,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View} import org.apache.spark.sql.catalyst.util.StringUtils import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -71,7 +72,7 @@ class SessionCatalog( conf: SQLConf) { this( () => externalCatalog, - () => new GlobalTempViewManager("global_temp"), + () => new GlobalTempViewManager(conf.getConf(GLOBAL_TEMP_DATABASE)), functionRegistry, conf, new Configuration(), @@ -218,6 +219,11 @@ class SessionCatalog( if (dbName == DEFAULT_DATABASE) { throw new AnalysisException(s"Can not drop default database") } + if (cascade && databaseExists(dbName)) { + listTables(dbName).foreach { t => + invalidateCachedTable(QualifiedTableName(dbName, t.table)) + } + } externalCatalog.dropDatabase(dbName, ignoreIfNotExists, cascade) } @@ -429,6 +435,34 @@ class SessionCatalog( externalCatalog.getTable(db, table) } + /** + * Retrieve all metadata of existing permanent tables/views. If no database is specified, + * assume the table/view is in the current database. + * Only the tables/views belong to the same database that can be retrieved are returned. + * For example, if none of the requested tables could be retrieved, an empty list is returned. + * There is no guarantee of ordering of the returned tables. + */ + @throws[NoSuchDatabaseException] + def getTablesByName(names: Seq[TableIdentifier]): Seq[CatalogTable] = { + if (names.nonEmpty) { + val dbs = names.map(_.database.getOrElse(getCurrentDatabase)) + if (dbs.distinct.size != 1) { + val tables = names.map(name => formatTableName(name.table)) + val qualifiedTableNames = dbs.zip(tables).map { case (d, t) => QualifiedTableName(d, t)} + throw new AnalysisException( + s"Only the tables/views belong to the same database can be retrieved. Querying " + + s"tables/views are $qualifiedTableNames" + ) + } + val db = formatDatabaseName(dbs.head) + requireDbExists(db) + val tables = names.map(name => formatTableName(name.table)) + externalCatalog.getTablesByName(db, tables) + } else { + Seq.empty + } + } + /** * Load files stored in given path into an existing metastore table. * If no database is specified, assume the table is in the current database. @@ -750,7 +784,19 @@ class SessionCatalog( * Note that, if the specified database is global temporary view database, we will list global * temporary views. */ - def listTables(db: String, pattern: String): Seq[TableIdentifier] = { + def listTables(db: String, pattern: String): Seq[TableIdentifier] = listTables(db, pattern, true) + + /** + * List all matching tables in the specified database, including local temporary views + * if includeLocalTempViews is enabled. + * + * Note that, if the specified database is global temporary view database, we will list global + * temporary views. + */ + def listTables( + db: String, + pattern: String, + includeLocalTempViews: Boolean): Seq[TableIdentifier] = { val dbName = formatDatabaseName(db) val dbTables = if (dbName == globalTempViewManager.database) { globalTempViewManager.listViewNames(pattern).map { name => @@ -762,12 +808,23 @@ class SessionCatalog( TableIdentifier(name, Some(dbName)) } } - val localTempViews = synchronized { + + if (includeLocalTempViews) { + dbTables ++ listLocalTempViews(pattern) + } else { + dbTables + } + } + + /** + * List all matching local temporary views. + */ + def listLocalTempViews(pattern: String): Seq[TableIdentifier] = { + synchronized { StringUtils.filterPattern(tempViews.keys.toSeq, pattern).map { name => TableIdentifier(name) } } - dbTables ++ localTempViews } /** @@ -1100,10 +1157,11 @@ class SessionCatalog( * Check if the function with the specified name exists */ def functionExists(name: FunctionIdentifier): Boolean = { - val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) - requireDbExists(db) - functionRegistry.functionExists(name) || + functionRegistry.functionExists(name) || { + val db = formatDatabaseName(name.database.getOrElse(getCurrentDatabase)) + requireDbExists(db) externalCatalog.functionExists(db, name.funcName) + } } // ---------------------------------------------------------------- @@ -1134,7 +1192,8 @@ class SessionCatalog( if (clsForUDAF.isAssignableFrom(clazz)) { val cls = Utils.classForName("org.apache.spark.sql.execution.aggregate.ScalaUDAF") val e = cls.getConstructor(classOf[Seq[Expression]], clsForUDAF, classOf[Int], classOf[Int]) - .newInstance(input, clazz.newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) + .newInstance(input, + clazz.getConstructor().newInstance().asInstanceOf[Object], Int.box(1), Int.box(1)) .asInstanceOf[ImplicitCastInputTypes] // Check input argument size @@ -1221,9 +1280,10 @@ class SessionCatalog( databaseExists(db) && externalCatalog.functionExists(db, name.funcName) } - protected def failFunctionLookup(name: FunctionIdentifier): Nothing = { + protected[sql] def failFunctionLookup( + name: FunctionIdentifier, cause: Option[Throwable] = None): Nothing = { throw new NoSuchFunctionException( - db = name.database.getOrElse(getCurrentDatabase), func = name.funcName) + db = name.database.getOrElse(getCurrentDatabase), func = name.funcName, cause) } /** 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 817abebd72ac0..2d646721f87a2 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.catalog import java.net.URI +import java.time.ZoneOffset import java.util.Date import scala.collection.mutable @@ -30,7 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Cast, ExprId, Literal} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils -import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.quoteIdentifier import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -373,7 +374,7 @@ object CatalogTable { /** * This class of statistics is used in [[CatalogTable]] to interact with metastore. * We define this new class instead of directly using [[Statistics]] here because there are no - * concepts of attributes or broadcast hint in catalog. + * concepts of attributes in catalog. */ case class CatalogStatistics( sizeInBytes: BigInt, @@ -415,7 +416,8 @@ case class CatalogColumnStat( nullCount: Option[BigInt] = None, avgLen: Option[Long] = None, maxLen: Option[Long] = None, - histogram: Option[Histogram] = None) { + histogram: Option[Histogram] = None, + version: Int = CatalogColumnStat.VERSION) { /** * Returns a map from string to string that can be used to serialize the column stats. @@ -429,7 +431,7 @@ case class CatalogColumnStat( */ def toMap(colName: String): Map[String, String] = { val map = new scala.collection.mutable.HashMap[String, String] - map.put(s"${colName}.${CatalogColumnStat.KEY_VERSION}", "1") + map.put(s"${colName}.${CatalogColumnStat.KEY_VERSION}", CatalogColumnStat.VERSION.toString) distinctCount.foreach { v => map.put(s"${colName}.${CatalogColumnStat.KEY_DISTINCT_COUNT}", v.toString) } @@ -452,12 +454,13 @@ case class CatalogColumnStat( dataType: DataType): ColumnStat = ColumnStat( distinctCount = distinctCount, - min = min.map(CatalogColumnStat.fromExternalString(_, colName, dataType)), - max = max.map(CatalogColumnStat.fromExternalString(_, colName, dataType)), + min = min.map(CatalogColumnStat.fromExternalString(_, colName, dataType, version)), + max = max.map(CatalogColumnStat.fromExternalString(_, colName, dataType, version)), nullCount = nullCount, avgLen = avgLen, maxLen = maxLen, - histogram = histogram) + histogram = histogram, + version = version) } object CatalogColumnStat extends Logging { @@ -472,14 +475,23 @@ object CatalogColumnStat extends Logging { private val KEY_MAX_LEN = "maxLen" private val KEY_HISTOGRAM = "histogram" + val VERSION = 2 + + private def getTimestampFormatter(): TimestampFormatter = { + TimestampFormatter(format = "yyyy-MM-dd HH:mm:ss.SSSSSS", zoneId = ZoneOffset.UTC) + } + /** * Converts from string representation of data type to the corresponding Catalyst data type. */ - def fromExternalString(s: String, name: String, dataType: DataType): Any = { + def fromExternalString(s: String, name: String, dataType: DataType, version: Int): Any = { dataType match { case BooleanType => s.toBoolean - case DateType => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) - case TimestampType => DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) + case DateType if version == 1 => DateTimeUtils.fromJavaDate(java.sql.Date.valueOf(s)) + case DateType => DateFormatter().parse(s) + case TimestampType if version == 1 => + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf(s)) + case TimestampType => getTimestampFormatter().parse(s) case ByteType => s.toByte case ShortType => s.toShort case IntegerType => s.toInt @@ -501,8 +513,8 @@ object CatalogColumnStat extends Logging { */ def toExternalString(v: Any, colName: String, dataType: DataType): String = { val externalValue = dataType match { - case DateType => DateTimeUtils.toJavaDate(v.asInstanceOf[Int]) - case TimestampType => DateTimeUtils.toJavaTimestamp(v.asInstanceOf[Long]) + case DateType => DateFormatter().format(v.asInstanceOf[Int]) + case TimestampType => getTimestampFormatter().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. @@ -532,7 +544,8 @@ object CatalogColumnStat extends Logging { nullCount = map.get(s"${colName}.${KEY_NULL_COUNT}").map(v => BigInt(v.toLong)), avgLen = map.get(s"${colName}.${KEY_AVG_LEN}").map(_.toLong), maxLen = map.get(s"${colName}.${KEY_MAX_LEN}").map(_.toLong), - histogram = map.get(s"${colName}.${KEY_HISTOGRAM}").map(HistogramSerializer.deserialize) + histogram = map.get(s"${colName}.${KEY_HISTOGRAM}").map(HistogramSerializer.deserialize), + version = map(s"${colName}.${KEY_VERSION}").toInt )) } catch { case NonFatal(e) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala new file mode 100644 index 0000000000000..bbe27831f01df --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtils.scala @@ -0,0 +1,82 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.csv + +object CSVExprUtils { + /** + * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). + * This is currently being used in CSV reading path and CSV schema inference. + */ + def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = { + iter.filter { line => + line.trim.nonEmpty && !line.startsWith(options.comment.toString) + } + } + + def skipComments(iter: Iterator[String], options: CSVOptions): Iterator[String] = { + if (options.isCommentSet) { + val commentPrefix = options.comment.toString + iter.dropWhile { line => + line.trim.isEmpty || line.trim.startsWith(commentPrefix) + } + } else { + iter.dropWhile(_.trim.isEmpty) + } + } + + /** + * Extracts header and moves iterator forward so that only data remains in it + */ + def extractHeader(iter: Iterator[String], options: CSVOptions): Option[String] = { + val nonEmptyLines = skipComments(iter, options) + if (nonEmptyLines.hasNext) { + Some(nonEmptyLines.next()) + } else { + None + } + } + + /** + * Helper method that converts string representation of a character to actual character. + * It handles some Java escaped strings and throws exception if given string is longer than one + * character. + */ + @throws[IllegalArgumentException] + def toChar(str: String): Char = { + (str: Seq[Char]) match { + case Seq() => throw new IllegalArgumentException("Delimiter cannot be empty string") + case Seq('\\') => throw new IllegalArgumentException("Single backslash is prohibited." + + " It has special meaning as beginning of an escape sequence." + + " To get the backslash character, pass a string with two backslashes as the delimiter.") + case Seq(c) => c + case Seq('\\', 't') => '\t' + case Seq('\\', 'r') => '\r' + case Seq('\\', 'b') => '\b' + case Seq('\\', 'f') => '\f' + // In case user changes quote char and uses \" as delimiter in options + case Seq('\\', '\"') => '\"' + case Seq('\\', '\'') => '\'' + case Seq('\\', '\\') => '\\' + case _ if str == """\u0000""" => '\u0000' + case Seq('\\', _) => + throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") + case _ => + throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala similarity index 97% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala index 558ee91c419b9..c39f77e891ae1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVHeaderChecker.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVHeaderChecker.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv import com.univocity.parsers.csv.CsvParser @@ -123,7 +123,7 @@ class CSVHeaderChecker( // Note: if there are only comments in the first block, the header would probably // be not extracted. if (options.headerFlag && isStartOfFile) { - CSVUtils.extractHeader(lines, options).foreach { header => + CSVExprUtils.extractHeader(lines, options).foreach { header => checkHeaderColumnNames(tokenizer.parseLine(header)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala similarity index 67% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala index 3596ff105fd7f..03cc3cbdf790a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchema.scala @@ -15,18 +15,31 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv -import java.math.BigDecimal +import java.util.Locale -import scala.util.control.Exception._ +import scala.util.control.Exception.allCatch import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.util.TimestampFormatter import org.apache.spark.sql.types._ -private[csv] object CSVInferSchema { +class CSVInferSchema(val options: CSVOptions) extends Serializable { + + private val timestampParser = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) + + private val decimalParser = if (options.locale == Locale.US) { + // Special handling the default locale for backward compatibility + s: String => new java.math.BigDecimal(s) + } else { + ExprUtils.getDecimalParser(options.locale) + } /** * Similar to the JSON schema inference @@ -36,20 +49,13 @@ private[csv] object CSVInferSchema { */ def infer( tokenRDD: RDD[Array[String]], - header: Array[String], - options: CSVOptions): StructType = { + header: Array[String]): StructType = { val fields = if (options.inferSchemaFlag) { val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) val rootTypes: Array[DataType] = - tokenRDD.aggregate(startType)(inferRowType(options), mergeRowTypes) - - header.zip(rootTypes).map { case (thisHeader, rootType) => - val dType = rootType match { - case _: NullType => StringType - case other => other - } - StructField(thisHeader, dType, nullable = true) - } + tokenRDD.aggregate(startType)(inferRowType, mergeRowTypes) + + toStructFields(rootTypes, header) } else { // By default fields are assumed to be StringType header.map(fieldName => StructField(fieldName, StringType, nullable = true)) @@ -58,11 +64,22 @@ private[csv] object CSVInferSchema { StructType(fields) } - private def inferRowType(options: CSVOptions) - (rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { + def toStructFields( + fieldTypes: Array[DataType], + header: Array[String]): Array[StructField] = { + header.zip(fieldTypes).map { case (thisHeader, rootType) => + val dType = rootType match { + case _: NullType => StringType + case other => other + } + StructField(thisHeader, dType, nullable = true) + } + } + + def inferRowType(rowSoFar: Array[DataType], next: Array[String]): Array[DataType] = { var i = 0 while (i < math.min(rowSoFar.length, next.length)) { // May have columns on right missing. - rowSoFar(i) = inferField(rowSoFar(i), next(i), options) + rowSoFar(i) = inferField(rowSoFar(i), next(i)) i+=1 } rowSoFar @@ -78,20 +95,20 @@ private[csv] object CSVInferSchema { * Infer type of string field. Given known type Double, and a string "1", there is no * point checking if it is an Int, as the final type must be Double or higher. */ - def inferField(typeSoFar: DataType, field: String, options: CSVOptions): DataType = { + def inferField(typeSoFar: DataType, field: String): DataType = { if (field == null || field.isEmpty || field == options.nullValue) { typeSoFar } else { typeSoFar match { - case NullType => tryParseInteger(field, options) - case IntegerType => tryParseInteger(field, options) - case LongType => tryParseLong(field, options) + case NullType => tryParseInteger(field) + case IntegerType => tryParseInteger(field) + case LongType => tryParseLong(field) case _: DecimalType => // DecimalTypes have different precisions and scales, so we try to find the common type. - compatibleType(typeSoFar, tryParseDecimal(field, options)).getOrElse(StringType) - case DoubleType => tryParseDouble(field, options) - case TimestampType => tryParseTimestamp(field, options) - case BooleanType => tryParseBoolean(field, options) + compatibleType(typeSoFar, tryParseDecimal(field)).getOrElse(StringType) + case DoubleType => tryParseDouble(field) + case TimestampType => tryParseTimestamp(field) + case BooleanType => tryParseBoolean(field) case StringType => StringType case other: DataType => throw new UnsupportedOperationException(s"Unexpected data type $other") @@ -99,30 +116,30 @@ private[csv] object CSVInferSchema { } } - private def isInfOrNan(field: String, options: CSVOptions): Boolean = { + private def isInfOrNan(field: String): Boolean = { field == options.nanValue || field == options.negativeInf || field == options.positiveInf } - private def tryParseInteger(field: String, options: CSVOptions): DataType = { + private def tryParseInteger(field: String): DataType = { if ((allCatch opt field.toInt).isDefined) { IntegerType } else { - tryParseLong(field, options) + tryParseLong(field) } } - private def tryParseLong(field: String, options: CSVOptions): DataType = { + private def tryParseLong(field: String): DataType = { if ((allCatch opt field.toLong).isDefined) { LongType } else { - tryParseDecimal(field, options) + tryParseDecimal(field) } } - private def tryParseDecimal(field: String, options: CSVOptions): DataType = { + private def tryParseDecimal(field: String): DataType = { val decimalTry = allCatch opt { - // `BigDecimal` conversion can fail when the `field` is not a form of number. - val bigDecimal = new BigDecimal(field) + // The conversion can fail when the `field` is not a form of number. + val bigDecimal = decimalParser(field) // Because many other formats do not support decimal, it reduces the cases for // decimals by disallowing values having scale (eg. `1.1`). if (bigDecimal.scale <= 0) { @@ -131,33 +148,30 @@ private[csv] object CSVInferSchema { // 2. scale is bigger than precision. DecimalType(bigDecimal.precision, bigDecimal.scale) } else { - tryParseDouble(field, options) + tryParseDouble(field) } } - decimalTry.getOrElse(tryParseDouble(field, options)) + decimalTry.getOrElse(tryParseDouble(field)) } - private def tryParseDouble(field: String, options: CSVOptions): DataType = { - if ((allCatch opt field.toDouble).isDefined || isInfOrNan(field, options)) { + private def tryParseDouble(field: String): DataType = { + if ((allCatch opt field.toDouble).isDefined || isInfOrNan(field)) { DoubleType } else { - tryParseTimestamp(field, options) + tryParseTimestamp(field) } } - private def tryParseTimestamp(field: String, options: CSVOptions): DataType = { + private def tryParseTimestamp(field: String): DataType = { // This case infers a custom `dataFormat` is set. - if ((allCatch opt options.timestampFormat.parse(field)).isDefined) { - TimestampType - } else if ((allCatch opt DateTimeUtils.stringToTime(field)).isDefined) { - // We keep this for backwards compatibility. + if ((allCatch opt timestampParser.parse(field)).isDefined) { TimestampType } else { - tryParseBoolean(field, options) + tryParseBoolean(field) } } - private def tryParseBoolean(field: String, options: CSVOptions): DataType = { + private def tryParseBoolean(field: String): DataType = { if ((allCatch opt field.toBoolean).isDefined) { BooleanType } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala similarity index 82% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala index 492a21be6df3b..1268fcffcfcd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVOptions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/CSVOptions.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv import java.nio.charset.StandardCharsets -import java.util.{Locale, TimeZone} +import java.time.ZoneId +import java.util.Locale import com.univocity.parsers.csv.{CsvParserSettings, CsvWriterSettings, UnescapedQuoteHandling} -import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf class CSVOptions( @transient val parameters: CaseInsensitiveMap[String], @@ -33,11 +34,22 @@ class CSVOptions( defaultColumnNameOfCorruptRecord: String) extends Logging with Serializable { + def this( + parameters: Map[String, String], + columnPruning: Boolean, + defaultTimeZoneId: String) = { + this( + CaseInsensitiveMap(parameters), + columnPruning, + defaultTimeZoneId, + SQLConf.get.columnNameOfCorruptRecord) + } + def this( parameters: Map[String, String], columnPruning: Boolean, defaultTimeZoneId: String, - defaultColumnNameOfCorruptRecord: String = "") = { + defaultColumnNameOfCorruptRecord: String) = { this( CaseInsensitiveMap(parameters), columnPruning, @@ -83,7 +95,7 @@ class CSVOptions( } } - val delimiter = CSVUtils.toChar( + val delimiter = CSVExprUtils.toChar( parameters.getOrElse("sep", parameters.getOrElse("delimiter", ","))) val parseMode: ParseMode = parameters.get("mode").map(ParseMode.fromString).getOrElse(PermissiveMode) @@ -128,16 +140,16 @@ class CSVOptions( name.map(CompressionCodecs.getCodecClassName) } - val timeZone: TimeZone = DateTimeUtils.getTimeZone( + val zoneId: ZoneId = DateTimeUtils.getZoneId( parameters.getOrElse(DateTimeUtils.TIMEZONE_OPTION, defaultTimeZoneId)) - // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. - val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), Locale.US) + // A language tag in IETF BCP 47 format + val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) + + val dateFormat: String = parameters.getOrElse("dateFormat", "yyyy-MM-dd") - val timestampFormat: FastDateFormat = - FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, Locale.US) + val timestampFormat: String = + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) @@ -177,6 +189,20 @@ class CSVOptions( */ val emptyValueInWrite = emptyValue.getOrElse("\"\"") + /** + * A string between two consecutive JSON records. + */ + val lineSeparator: Option[String] = parameters.get("lineSep").map { sep => + require(sep.nonEmpty, "'lineSep' cannot be an empty string.") + require(sep.length == 1, "'lineSep' can contain only 1 character.") + sep + } + + val lineSeparatorInRead: Option[Array[Byte]] = lineSeparator.map { lineSep => + lineSep.getBytes(charset) + } + val lineSeparatorInWrite: Option[String] = lineSeparator + def asWriterSettings: CsvWriterSettings = { val writerSettings = new CsvWriterSettings() val format = writerSettings.getFormat @@ -185,6 +211,8 @@ class CSVOptions( format.setQuoteEscape(escape) charToEscapeQuoteEscaping.foreach(format.setCharToEscapeQuoteEscaping) format.setComment(comment) + lineSeparatorInWrite.foreach(format.setLineSeparator) + writerSettings.setIgnoreLeadingWhitespaces(ignoreLeadingWhiteSpaceFlagInWrite) writerSettings.setIgnoreTrailingWhitespaces(ignoreTrailingWhiteSpaceFlagInWrite) writerSettings.setNullValue(nullValue) @@ -201,8 +229,10 @@ class CSVOptions( format.setDelimiter(delimiter) format.setQuote(quote) format.setQuoteEscape(escape) + lineSeparator.foreach(format.setLineSeparator) charToEscapeQuoteEscaping.foreach(format.setCharToEscapeQuoteEscaping) format.setComment(comment) + settings.setIgnoreLeadingWhitespaces(ignoreLeadingWhiteSpaceInRead) settings.setIgnoreTrailingWhitespaces(ignoreTrailingWhiteSpaceInRead) settings.setReadInputOnSeparateThread(false) @@ -212,6 +242,11 @@ class CSVOptions( settings.setEmptyValue(emptyValueInRead) settings.setMaxCharsPerColumn(maxCharsPerColumn) settings.setUnescapedQuoteHandling(UnescapedQuoteHandling.STOP_AT_DELIMITER) + settings.setLineSeparatorDetectionEnabled(lineSeparatorInRead.isEmpty && multiLine) + lineSeparatorInRead.foreach { _ => + settings.setNormalizeLineEndingsWithinQuotes(!multiLine) + } + settings } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala similarity index 78% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala index 4082a0df8ba75..9ca94501f5c58 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityGenerator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityGenerator.scala @@ -15,24 +15,23 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv import java.io.Writer import com.univocity.parsers.csv.CsvWriter import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, TimestampFormatter} import org.apache.spark.sql.types._ -private[csv] class UnivocityGenerator( +class UnivocityGenerator( schema: StructType, writer: Writer, options: CSVOptions) { private val writerSettings = options.asWriterSettings writerSettings.setHeaders(schema.fieldNames: _*) private val gen = new CsvWriter(writer, writerSettings) - private var printHeader = options.headerFlag // A `ValueConverter` is responsible for converting a value of an `InternalRow` to `String`. // When the value is null, this converter should not be called. @@ -42,14 +41,18 @@ private[csv] class UnivocityGenerator( private val valueConverters: Array[ValueConverter] = schema.map(_.dataType).map(makeConverter).toArray + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private def makeConverter(dataType: DataType): ValueConverter = dataType match { case DateType => - (row: InternalRow, ordinal: Int) => - options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + (row: InternalRow, ordinal: Int) => dateFormatter.format(row.getInt(ordinal)) case TimestampType => - (row: InternalRow, ordinal: Int) => - options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + (row: InternalRow, ordinal: Int) => timestampFormatter.format(row.getLong(ordinal)) case udt: UserDefinedType[_] => makeConverter(udt.sqlType) @@ -72,15 +75,19 @@ private[csv] class UnivocityGenerator( values } + def writeHeaders(): Unit = { + gen.writeHeaders() + } + /** * Writes a single InternalRow to CSV using Univocity. */ def write(row: InternalRow): Unit = { - if (printHeader) { - gen.writeHeaders() - } gen.writeRow(convertRow(row): _*) - printHeader = false + } + + def writeToString(row: InternalRow): String = { + gen.writeRowToString(convertRow(row): _*) } def close(): Unit = gen.close() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala similarity index 77% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala index fbd19c6e677e5..8456b7d218ead 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/csv/UnivocityParser.scala @@ -15,21 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv import java.io.InputStream -import java.math.BigDecimal -import scala.util.Try import scala.util.control.NonFatal import com.univocity.parsers.csv.CsvParser import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.util.{BadRecordException, DateTimeUtils} -import org.apache.spark.sql.execution.datasources.FailureSafeParser +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericInternalRow} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -77,6 +74,12 @@ class UnivocityParser( private val row = new GenericInternalRow(requiredSchema.length) + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + // Retrieve the raw record string. private def getCurrentInput: UTF8String = { UTF8String.fromString(tokenizer.getContext.currentParsedContent().stripLineEnd) @@ -102,9 +105,11 @@ class UnivocityParser( // // output row - ["A", 2] private val valueConverters: Array[ValueConverter] = { - requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable, options)).toArray + requiredSchema.map(f => makeConverter(f.name, f.dataType, f.nullable)).toArray } + private val decimalParser = ExprUtils.getDecimalParser(options.locale) + /** * Create a converter which converts the string value to a value according to a desired type. * Currently, we do not support complex types (`ArrayType`, `MapType`, `StructType`). @@ -115,8 +120,7 @@ class UnivocityParser( def makeConverter( name: String, dataType: DataType, - nullable: Boolean = true, - options: CSVOptions): ValueConverter = dataType match { + nullable: Boolean = true): ValueConverter = dataType match { case _: ByteType => (d: String) => nullSafeDatum(d, name, nullable, options)(_.toByte) @@ -150,39 +154,20 @@ class UnivocityParser( case dt: DecimalType => (d: String) => nullSafeDatum(d, name, nullable, options) { datum => - val value = new BigDecimal(datum.replaceAll(",", "")) - Decimal(value, dt.precision, dt.scale) + Decimal(decimalParser(datum), dt.precision, dt.scale) } case _: TimestampType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Try(options.timestampFormat.parse(datum).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(datum).getTime * 1000L - } - } + nullSafeDatum(d, name, nullable, options)(timestampFormatter.parse) case _: DateType => (d: String) => - nullSafeDatum(d, name, nullable, options) { datum => - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(datum).getTime)) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(datum).getTime) - } - } + nullSafeDatum(d, name, nullable, options)(dateFormatter.parse) case _: StringType => (d: String) => nullSafeDatum(d, name, nullable, options)(UTF8String.fromString) - case udt: UserDefinedType[_] => (datum: String) => - makeConverter(name, udt.sqlType, nullable, options) + case udt: UserDefinedType[_] => + makeConverter(name, udt.sqlType, nullable) // We don't actually hit this exception though, we keep it for understandability case _ => throw new RuntimeException(s"Unsupported type: ${dataType.typeName}") @@ -203,11 +188,19 @@ class UnivocityParser( } } + private val doParse = if (requiredSchema.nonEmpty) { + (input: String) => convert(tokenizer.parseLine(input)) + } else { + // If `columnPruning` enabled and partition attributes scanned only, + // `schema` gets empty. + (_: String) => InternalRow.empty + } + /** * Parses a single CSV string and turns it into either one resulting row or no row (if the * the record is malformed). */ - def parse(input: String): InternalRow = convert(tokenizer.parseLine(input)) + def parse(input: String): InternalRow = doParse(input) private val getToken = if (options.columnPruning) { (tokens: Array[String], index: Int) => tokens(index) @@ -244,27 +237,31 @@ class UnivocityParser( () => getPartialResult(), new RuntimeException("Malformed CSV record")) } else { - try { - // When the length of the returned tokens is identical to the length of the parsed schema, - // we just need to convert the tokens that correspond to the required columns. - var i = 0 - while (i < requiredSchema.length) { + // When the length of the returned tokens is identical to the length of the parsed schema, + // we just need to convert the tokens that correspond to the required columns. + var badRecordException: Option[Throwable] = None + var i = 0 + while (i < requiredSchema.length) { + try { row(i) = valueConverters(i).apply(getToken(tokens, i)) - i += 1 + } catch { + case NonFatal(e) => + badRecordException = badRecordException.orElse(Some(e)) + row.setNullAt(i) } + i += 1 + } + + if (badRecordException.isEmpty) { row - } catch { - case NonFatal(e) => - // For corrupted records with the number of tokens same as the schema, - // CSV reader doesn't support partial results. All fields other than the field - // configured by `columnNameOfCorruptRecord` are set to `null`. - throw BadRecordException(() => getCurrentInput, () => None, e) + } else { + throw BadRecordException(() => getCurrentInput, () => Some(row), badRecordException.get) } } } } -private[csv] object UnivocityParser { +private[sql] object UnivocityParser { /** * Parses a stream that contains CSV strings and turns it into an iterator of tokens. @@ -272,11 +269,12 @@ private[csv] object UnivocityParser { def tokenizeStream( inputStream: InputStream, shouldDropHeader: Boolean, - tokenizer: CsvParser): Iterator[Array[String]] = { + tokenizer: CsvParser, + encoding: String): Iterator[Array[String]] = { val handleHeader: () => Unit = () => if (shouldDropHeader) tokenizer.parseNext - convertStream(inputStream, tokenizer, handleHeader)(tokens => tokens) + convertStream(inputStream, tokenizer, handleHeader, encoding)(tokens => tokens) } /** @@ -292,13 +290,12 @@ private[csv] object UnivocityParser { input => Seq(parser.convert(input)), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) val handleHeader: () => Unit = () => headerChecker.checkHeaderColumnNames(tokenizer) - convertStream(inputStream, tokenizer, handleHeader) { tokens => + convertStream(inputStream, tokenizer, handleHeader, parser.options.charset) { tokens => safeParser.parse(tokens) }.flatten } @@ -306,9 +303,10 @@ private[csv] object UnivocityParser { private def convertStream[T]( inputStream: InputStream, tokenizer: CsvParser, - handleHeader: () => Unit)( + handleHeader: () => Unit, + encoding: String)( convert: Array[String] => T) = new Iterator[T] { - tokenizer.beginParsing(inputStream) + tokenizer.beginParsing(inputStream, encoding) // We can handle header here since here the stream is open. handleHeader() @@ -339,14 +337,13 @@ private[csv] object UnivocityParser { val options = parser.options - val filteredLines: Iterator[String] = CSVUtils.filterCommentAndEmpty(lines, options) + val filteredLines: Iterator[String] = CSVExprUtils.filterCommentAndEmpty(lines, options) val safeParser = new FailureSafeParser[String]( input => Seq(parser.parse(input)), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) filteredLines.flatMap(safeParser.parse) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala index 176ea823b1fcd..54fc1f9abb086 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -136,7 +136,7 @@ package object dsl { implicit def longToLiteral(l: Long): Literal = Literal(l) implicit def floatToLiteral(f: Float): Literal = Literal(f) implicit def doubleToLiteral(d: Double): Literal = Literal(d) - implicit def stringToLiteral(s: String): Literal = Literal(s) + implicit def stringToLiteral(s: String): Literal = Literal.create(s, StringType) implicit def dateToLiteral(d: Date): Literal = Literal(d) implicit def bigDecimalToLiteral(d: BigDecimal): Literal = Literal(d.underlying()) implicit def bigDecimalToLiteral(d: java.math.BigDecimal): Literal = Literal(d) @@ -295,10 +295,7 @@ package object dsl { object expressions extends ExpressionConversions // scalastyle:ignore object plans { // scalastyle:ignore - def table(ref: String): LogicalPlan = UnresolvedRelation(TableIdentifier(ref)) - - def table(db: String, ref: String): LogicalPlan = - UnresolvedRelation(TableIdentifier(ref, Option(db))) + def table(parts: String*): LogicalPlan = UnresolvedRelation(parts) implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) { def select(exprs: Expression*): LogicalPlan = { @@ -325,7 +322,7 @@ package object dsl { otherPlan: LogicalPlan, joinType: JoinType = Inner, condition: Option[Expression] = None): LogicalPlan = - Join(logicalPlan, otherPlan, joinType, condition) + Join(logicalPlan, otherPlan, joinType, condition, JoinHint.NONE) def cogroup[Key: Encoder, Left: Encoder, Right: Encoder, Result: Encoder]( otherPlan: LogicalPlan, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala index cbea3c017a265..c97303be1d27c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/ExpressionEncoder.scala @@ -24,11 +24,13 @@ import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.{InternalRow, JavaTypeInference, ScalaReflection} import org.apache.spark.sql.catalyst.analysis.{Analyzer, GetColumnByOrdinal, SimpleAnalyzer, UnresolvedAttribute, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateSafeProjection, GenerateUnsafeProjection} -import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, Invoke, NewInstance} -import org.apache.spark.sql.catalyst.optimizer.SimplifyCasts -import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LocalRelation} -import org.apache.spark.sql.types.{BooleanType, ObjectType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.catalyst.expressions.objects.{AssertNotNull, InitializeJavaBean, Invoke, NewInstance} +import org.apache.spark.sql.catalyst.optimizer.{ReassignLambdaVariableID, SimplifyCasts} +import org.apache.spark.sql.catalyst.plans.logical.{CatalystSerde, DeserializeToObject, LeafNode, LocalRelation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ObjectType, StringType, StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils /** @@ -43,40 +45,17 @@ import org.apache.spark.util.Utils * to the name `value`. */ object ExpressionEncoder { + def apply[T : TypeTag](): ExpressionEncoder[T] = { - // We convert the not-serializable TypeTag into StructType and ClassTag. val mirror = ScalaReflection.mirror val tpe = typeTag[T].in(mirror).tpe - if (ScalaReflection.optionOfProductType(tpe)) { - throw new UnsupportedOperationException( - "Cannot create encoder for Option of Product type, because Product type is represented " + - "as a row, and the entire row can not be null in Spark SQL like normal databases. " + - "You can wrap your type with Tuple1 if you do want top level null Product objects, " + - "e.g. instead of creating `Dataset[Option[MyClass]]`, you can do something like " + - "`val ds: Dataset[Tuple1[MyClass]] = Seq(Tuple1(MyClass(...)), Tuple1(null)).toDS`") - } - val cls = mirror.runtimeClass(tpe) - val flat = !ScalaReflection.definedByConstructorParams(tpe) - - val inputObject = BoundReference(0, ScalaReflection.dataTypeFor[T], nullable = !cls.isPrimitive) - val nullSafeInput = if (flat) { - inputObject - } else { - // For input object of Product type, we can't encode it to row if it's null, as Spark SQL - // doesn't allow top-level row to be null, only its columns can be null. - AssertNotNull(inputObject, Seq("top level Product input object")) - } - val serializer = ScalaReflection.serializerFor[T](nullSafeInput) - val deserializer = ScalaReflection.deserializerFor[T] - - val schema = serializer.dataType + val serializer = ScalaReflection.serializerForType(tpe) + val deserializer = ScalaReflection.deserializerForType(tpe) new ExpressionEncoder[T]( - schema, - flat, - serializer.flatten, + serializer, deserializer, ClassTag[T](cls)) } @@ -86,14 +65,12 @@ object ExpressionEncoder { val schema = JavaTypeInference.inferDataType(beanClass)._1 assert(schema.isInstanceOf[StructType]) - val serializer = JavaTypeInference.serializerFor(beanClass) - val deserializer = JavaTypeInference.deserializerFor(beanClass) + val objSerializer = JavaTypeInference.serializerFor(beanClass) + val objDeserializer = JavaTypeInference.deserializerFor(beanClass) new ExpressionEncoder[T]( - schema.asInstanceOf[StructType], - flat = false, - serializer.flatten, - deserializer, + objSerializer, + objDeserializer, ClassTag[T](beanClass)) } @@ -103,77 +80,56 @@ object ExpressionEncoder { * name/positional binding is preserved. */ def tuple(encoders: Seq[ExpressionEncoder[_]]): ExpressionEncoder[_] = { - encoders.foreach(_.assertUnresolved()) + if (encoders.length > 22) { + throw new UnsupportedOperationException("Due to Scala's limited support of tuple, " + + "tuple with more than 22 elements are not supported.") + } - val schema = StructType(encoders.zipWithIndex.map { - case (e, i) => - val (dataType, nullable) = if (e.flat) { - e.schema.head.dataType -> e.schema.head.nullable - } else { - e.schema -> true - } - StructField(s"_${i + 1}", dataType, nullable) - }) + encoders.foreach(_.assertUnresolved()) val cls = Utils.getContextOrSparkClassLoader.loadClass(s"scala.Tuple${encoders.size}") - val serializer = encoders.zipWithIndex.map { case (enc, index) => - val originalInputObject = enc.serializer.head.collect { case b: BoundReference => b }.head + val newSerializerInput = BoundReference(0, ObjectType(cls), nullable = true) + val serializers = encoders.zipWithIndex.map { case (enc, index) => + val boundRefs = enc.objSerializer.collect { case b: BoundReference => b }.distinct + assert(boundRefs.size == 1, "object serializer should have only one bound reference but " + + s"there are ${boundRefs.size}") + + val originalInputObject = boundRefs.head val newInputObject = Invoke( - BoundReference(0, ObjectType(cls), nullable = true), + newSerializerInput, s"_${index + 1}", - originalInputObject.dataType) - - val newSerializer = enc.serializer.map(_.transformUp { - case b: BoundReference if b == originalInputObject => newInputObject - }) - - val serializerExpr = if (enc.flat) { - newSerializer.head - } else { - // For non-flat encoder, the input object is not top level anymore after being combined to - // a tuple encoder, thus it can be null and we should wrap the `CreateStruct` with `If` and - // null check to handle null case correctly. - // e.g. for Encoder[(Int, String)], the serializer expressions will create 2 columns, and is - // not able to handle the case when the input tuple is null. This is not a problem as there - // is a check to make sure the input object won't be null. However, if this encoder is used - // to create a bigger tuple encoder, the original input object becomes a filed of the new - // input tuple and can be null. So instead of creating a struct directly here, we should add - // a null/None check and return a null struct if the null/None check fails. - val struct = CreateStruct(newSerializer) - val nullCheck = Or( - IsNull(newInputObject), - Invoke(Literal.fromObject(None), "equals", BooleanType, newInputObject :: Nil)) - If(nullCheck, Literal.create(null, struct.dataType), struct) + originalInputObject.dataType, + returnNullable = originalInputObject.nullable) + + val newSerializer = enc.objSerializer.transformUp { + case BoundReference(0, _, _) => newInputObject } - Alias(serializerExpr, s"_${index + 1}")() + + Alias(newSerializer, s"_${index + 1}")() } + val newSerializer = CreateStruct(serializers) + + val newDeserializerInput = GetColumnByOrdinal(0, newSerializer.dataType) + val deserializers = encoders.zipWithIndex.map { case (enc, index) => + val getColExprs = enc.objDeserializer.collect { case c: GetColumnByOrdinal => c }.distinct + assert(getColExprs.size == 1, "object deserializer should have only one " + + s"`GetColumnByOrdinal`, but there are ${getColExprs.size}") - val childrenDeserializers = encoders.zipWithIndex.map { case (enc, index) => - if (enc.flat) { - enc.deserializer.transform { - case g: GetColumnByOrdinal => g.copy(ordinal = index) - } - } else { - val input = GetColumnByOrdinal(index, enc.schema) - val deserialized = enc.deserializer.transformUp { - case UnresolvedAttribute(nameParts) => - assert(nameParts.length == 1) - UnresolvedExtractValue(input, Literal(nameParts.head)) - case GetColumnByOrdinal(ordinal, _) => GetStructField(input, ordinal) - } - If(IsNull(input), Literal.create(null, deserialized.dataType), deserialized) + val input = GetStructField(newDeserializerInput, index) + enc.objDeserializer.transformUp { + case GetColumnByOrdinal(0, _) => input } } + val newDeserializer = NewInstance(cls, deserializers, ObjectType(cls), propagateNull = false) - val deserializer = - NewInstance(cls, childrenDeserializers, ObjectType(cls), propagateNull = false) + def nullSafe(input: Expression, result: Expression): Expression = { + If(IsNull(input), Literal.create(null, result.dataType), result) + } new ExpressionEncoder[Any]( - schema, - flat = false, - serializer, - deserializer, + nullSafe(newSerializerInput, newSerializer), + nullSafe(newDeserializerInput, newDeserializer), ClassTag(cls)) } @@ -212,21 +168,104 @@ object ExpressionEncoder { * A generic encoder for JVM objects that uses Catalyst Expressions for a `serializer` * and a `deserializer`. * - * @param schema The schema after converting `T` to a Spark SQL row. - * @param serializer A set of expressions, one for each top-level field that can be used to - * extract the values from a raw object into an [[InternalRow]]. - * @param deserializer An expression that will construct an object given an [[InternalRow]]. + * @param objSerializer An expression that can be used to encode a raw object to corresponding + * Spark SQL representation that can be a primitive column, array, map or a + * struct. This represents how Spark SQL generally serializes an object of + * type `T`. + * @param objDeserializer An expression that will construct an object given a Spark SQL + * representation. This represents how Spark SQL generally deserializes + * a serialized value in Spark SQL representation back to an object of + * type `T`. * @param clsTag A classtag for `T`. */ case class ExpressionEncoder[T]( - schema: StructType, - flat: Boolean, - serializer: Seq[Expression], - deserializer: Expression, + objSerializer: Expression, + objDeserializer: Expression, clsTag: ClassTag[T]) extends Encoder[T] { - if (flat) require(serializer.size == 1) + /** + * A sequence of expressions, one for each top-level field that can be used to + * extract the values from a raw object into an [[InternalRow]]: + * 1. If `serializer` encodes a raw object to a struct, strip the outer If-IsNull and get + * the `CreateNamedStruct`. + * 2. For other cases, wrap the single serializer with `CreateNamedStruct`. + */ + val serializer: Seq[NamedExpression] = { + val clsName = Utils.getSimpleName(clsTag.runtimeClass) + + if (isSerializedAsStructForTopLevel) { + val nullSafeSerializer = objSerializer.transformUp { + case r: BoundReference => + // For input object of Product type, we can't encode it to row if it's null, as Spark SQL + // doesn't allow top-level row to be null, only its columns can be null. + AssertNotNull(r, Seq("top level Product or row object")) + } + nullSafeSerializer match { + case If(_: IsNull, _, s: CreateNamedStruct) => s + case _ => + throw new RuntimeException(s"class $clsName has unexpected serializer: $objSerializer") + } + } else { + // For other input objects like primitive, array, map, etc., we construct a struct to wrap + // the serializer which is a column of an row. + // + // Note: Because Spark SQL doesn't allow top-level row to be null, to encode + // top-level Option[Product] type, we make it as a top-level struct column. + CreateNamedStruct(Literal("value") :: objSerializer :: Nil) + } + }.flatten + + /** + * Returns an expression that can be used to deserialize an input row to an object of type `T` + * with a compatible schema. Fields of the row will be extracted using `UnresolvedAttribute`. + * of the same name as the constructor arguments. + * + * For complex objects that are encoded to structs, Fields of the struct will be extracted using + * `GetColumnByOrdinal` with corresponding ordinal. + */ + val deserializer: Expression = { + if (isSerializedAsStructForTopLevel) { + // We serialized this kind of objects to root-level row. The input of general deserializer + // is a `GetColumnByOrdinal(0)` expression to extract first column of a row. We need to + // transform attributes accessors. + objDeserializer.transform { + case UnresolvedExtractValue(GetColumnByOrdinal(0, _), + Literal(part: UTF8String, StringType)) => + UnresolvedAttribute.quoted(part.toString) + case GetStructField(GetColumnByOrdinal(0, dt), ordinal, _) => + GetColumnByOrdinal(ordinal, dt) + case If(IsNull(GetColumnByOrdinal(0, _)), _, n: NewInstance) => n + case If(IsNull(GetColumnByOrdinal(0, _)), _, i: InitializeJavaBean) => i + } + } else { + // For other input objects like primitive, array, map, etc., we deserialize the first column + // of a row to the object. + objDeserializer + } + } + + // The schema after converting `T` to a Spark SQL row. This schema is dependent on the given + // serializer. + val schema: StructType = StructType(serializer.map { s => + StructField(s.name, s.dataType, s.nullable) + }) + + /** + * Returns true if the type `T` is serialized as a struct by `objSerializer`. + */ + def isSerializedAsStruct: Boolean = objSerializer.dataType.isInstanceOf[StructType] + + /** + * If the type `T` is serialized as a struct, when it is encoded to a Spark SQL row, fields in + * the struct are naturally mapped to top-level columns in a row. In other words, the serialized + * struct is flattened to row. But in case of the `T` is also an `Option` type, it can't be + * flattened to top-level row, because in Spark SQL top-level row can't be null. This method + * returns true if `T` is serialized as struct and is not `Option` type. + */ + def isSerializedAsStructForTopLevel: Boolean = { + isSerializedAsStruct && !classOf[Option[_]].isAssignableFrom(clsTag.runtimeClass) + } // serializer expressions are used to encode an object to a row, while the object is usually an // intermediate value produced inside an operator, not from the output of the child operator. This @@ -258,17 +297,29 @@ case class ExpressionEncoder[T]( analyzer.checkAnalysis(analyzedPlan) val resolved = SimplifyCasts(analyzedPlan).asInstanceOf[DeserializeToObject].deserializer val bound = BindReferences.bindReference(resolved, attrs) - copy(deserializer = bound) + copy(objDeserializer = bound) } @transient - private lazy val extractProjection = GenerateUnsafeProjection.generate(serializer) + private lazy val extractProjection = GenerateUnsafeProjection.generate({ + // When using `ExpressionEncoder` directly, we will skip the normal query processing steps + // (analyzer, optimizer, etc.). Here we apply the ReassignLambdaVariableID rule, as it's + // important to codegen performance. + val optimizedPlan = ReassignLambdaVariableID.apply(DummyExpressionHolder(serializer)) + optimizedPlan.asInstanceOf[DummyExpressionHolder].exprs + }) @transient private lazy val inputRow = new GenericInternalRow(1) @transient - private lazy val constructProjection = GenerateSafeProjection.generate(deserializer :: Nil) + private lazy val constructProjection = SafeProjection.create({ + // When using `ExpressionEncoder` directly, we will skip the normal query processing steps + // (analyzer, optimizer, etc.). Here we apply the ReassignLambdaVariableID rule, as it's + // important to codegen performance. + val optimizedPlan = ReassignLambdaVariableID.apply(DummyExpressionHolder(Seq(deserializer))) + optimizedPlan.asInstanceOf[DummyExpressionHolder].exprs + }) /** * Returns a new set (with unique ids) of [[NamedExpression]] that represent the serialized form @@ -289,8 +340,8 @@ case class ExpressionEncoder[T]( extractProjection(inputRow) } catch { case e: Exception => - throw new RuntimeException( - s"Error while encoding: $e\n${serializer.map(_.simpleString).mkString("\n")}", e) + throw new RuntimeException(s"Error while encoding: $e\n" + + s"${serializer.map(_.simpleString(SQLConf.get.maxToStringFields)).mkString("\n")}", e) } /** @@ -302,7 +353,8 @@ case class ExpressionEncoder[T]( constructProjection(row).get(0, ObjectType(clsTag.runtimeClass)).asInstanceOf[T] } catch { case e: Exception => - throw new RuntimeException(s"Error while decoding: $e\n${deserializer.simpleString}", e) + throw new RuntimeException(s"Error while decoding: $e\n" + + s"${deserializer.simpleString(SQLConf.get.maxToStringFields)}", e) } /** @@ -331,3 +383,9 @@ case class ExpressionEncoder[T]( override def toString: String = s"class[$schemaString]" } + +// A dummy logical plan that can hold expressions and go through optimizer rules. +case class DummyExpressionHolder(exprs: Seq[Expression]) extends LeafNode { + override lazy val resolved = true + override def output: Seq[Attribute] = Nil +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala index a1f0312bd853c..665b2cd1274fd 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/OuterScopes.scala @@ -53,6 +53,12 @@ object OuterScopes { val outer = outerScopes.get(outerClassName) if (outer == null) { outerClassName match { + case AmmoniteREPLClass(cellClassName) => + () => { + val objClass = Utils.classForName(cellClassName) + val objInstance = objClass.getField("MODULE$").get(null) + objClass.getMethod("instance").invoke(objInstance) + } // If the outer class is generated by REPL, users don't need to register it as it has // only one instance and there is a way to retrieve it: get the `$read` object, call the // `INSTANCE()` method to get the single instance of class `$read`. Then call `$iw()` @@ -95,4 +101,8 @@ object OuterScopes { // The format of REPL generated wrapper class's name, e.g. `$line12.$read$$iw$$iw` private[this] val REPLClass = """^(\$line(?:\d+)\.\$read)(?:\$\$iw)+$""".r + + // The format of ammonite REPL generated wrapper class's name, + // e.g. `ammonite.$sess.cmd8$Helper$Foo` -> `ammonite.$sess.cmd8.instance.Foo` + private[this] val AmmoniteREPLClass = """^(ammonite\.\$sess\.cmd(?:\d+)\$).*""".r } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala index 3340789398f9c..afe8a23f8f150 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/encoders/RowEncoder.scala @@ -22,13 +22,15 @@ import scala.reflect.ClassTag import org.apache.spark.SparkException import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.{ScalaReflection, WalkedTypePath} +import org.apache.spark.sql.catalyst.DeserializerBuildHelper._ +import org.apache.spark.sql.catalyst.SerializerBuildHelper._ import org.apache.spark.sql.catalyst.analysis.GetColumnByOrdinal import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, DateTimeUtils} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String /** * A factory for constructing encoders that convert external row to/from the Spark SQL @@ -45,8 +47,11 @@ import org.apache.spark.unsafe.types.UTF8String * StringType -> String * DecimalType -> java.math.BigDecimal or scala.math.BigDecimal or Decimal * - * DateType -> java.sql.Date - * TimestampType -> java.sql.Timestamp + * DateType -> java.sql.Date if spark.sql.datetime.java8API.enabled is false + * DateType -> java.time.LocalDate if spark.sql.datetime.java8API.enabled is true + * + * TimestampType -> java.sql.Timestamp if spark.sql.datetime.java8API.enabled is false + * TimestampType -> java.time.Instant if spark.sql.datetime.java8API.enabled is true * * BinaryType -> byte array * ArrayType -> scala.collection.Seq or Array @@ -58,12 +63,10 @@ object RowEncoder { def apply(schema: StructType): ExpressionEncoder[Row] = { val cls = classOf[Row] val inputObject = BoundReference(0, ObjectType(cls), nullable = true) - val serializer = serializerFor(AssertNotNull(inputObject, Seq("top level row object")), schema) - val deserializer = deserializerFor(schema) + val serializer = serializerFor(inputObject, schema) + val deserializer = deserializerFor(GetColumnByOrdinal(0, serializer.dataType), schema) new ExpressionEncoder[Row]( - schema, - flat = false, - serializer.asInstanceOf[CreateNamedStruct].flatten, + serializer, deserializer, ClassTag(cls)) } @@ -92,36 +95,28 @@ object RowEncoder { Invoke(obj, "serialize", udt, inputObject :: Nil, returnNullable = false) case TimestampType => - StaticInvoke( - DateTimeUtils.getClass, - TimestampType, - "fromJavaTimestamp", - inputObject :: Nil, - returnNullable = false) + if (SQLConf.get.datetimeJava8ApiEnabled) { + createSerializerForJavaInstant(inputObject) + } else { + createSerializerForSqlTimestamp(inputObject) + } case DateType => - StaticInvoke( - DateTimeUtils.getClass, - DateType, - "fromJavaDate", - inputObject :: Nil, - returnNullable = false) + if (SQLConf.get.datetimeJava8ApiEnabled) { + createSerializerForJavaLocalDate(inputObject) + } else { + createSerializerForSqlDate(inputObject) + } case d: DecimalType => - StaticInvoke( + CheckOverflow(StaticInvoke( Decimal.getClass, d, "fromDecimal", inputObject :: Nil, - returnNullable = false) + returnNullable = false), d, SQLConf.get.decimalOperationsNullOnOverflow) - case StringType => - StaticInvoke( - classOf[UTF8String], - StringType, - "fromString", - inputObject :: Nil, - returnNullable = false) + case StringType => createSerializerForString(inputObject) case t @ ArrayType(et, containsNull) => et match { @@ -133,17 +128,14 @@ object RowEncoder { inputObject :: Nil, returnNullable = false) - case _ => MapObjects( - element => { - val value = serializerFor(ValidateExternalType(element, et), et) - if (!containsNull) { - AssertNotNull(value, Seq.empty) - } else { - value - } - }, - inputObject, - ObjectType(classOf[Object])) + case _ => + createSerializerForMapObjects( + inputObject, + ObjectType(classOf[Object]), + element => { + val value = serializerFor(ValidateExternalType(element, et), et) + expressionWithNullSafety(value, containsNull, WalkedTypePath()) + }) } case t @ MapType(kt, vt, valueNullable) => @@ -170,9 +162,7 @@ object RowEncoder { propagateNull = false) if (inputObject.nullable) { - If(IsNull(inputObject), - Literal.create(null, inputType), - nonNullOutput) + expressionForNullableExpr(inputObject, nonNullOutput) } else { nonNullOutput } @@ -187,7 +177,9 @@ object RowEncoder { val convertedField = if (field.nullable) { If( Invoke(inputObject, "isNullAt", BooleanType, Literal(index) :: Nil), - Literal.create(null, field.dataType), + // Because we strip UDTs, `field.dataType` can be different from `fieldValue.dataType`. + // We should use `fieldValue.dataType` here. + Literal.create(null, fieldValue.dataType), fieldValue ) } else { @@ -197,9 +189,7 @@ object RowEncoder { }) if (inputObject.nullable) { - If(IsNull(inputObject), - Literal.create(null, inputType), - nonNullOutput) + expressionForNullableExpr(inputObject, nonNullOutput) } else { nonNullOutput } @@ -224,8 +214,18 @@ object RowEncoder { def externalDataTypeFor(dt: DataType): DataType = dt match { case _ if ScalaReflection.isNativeType(dt) => dt - case TimestampType => ObjectType(classOf[java.sql.Timestamp]) - case DateType => ObjectType(classOf[java.sql.Date]) + case TimestampType => + if (SQLConf.get.datetimeJava8ApiEnabled) { + ObjectType(classOf[java.time.Instant]) + } else { + ObjectType(classOf[java.sql.Timestamp]) + } + case DateType => + if (SQLConf.get.datetimeJava8ApiEnabled) { + ObjectType(classOf[java.time.LocalDate]) + } else { + ObjectType(classOf[java.sql.Date]) + } case _: DecimalType => ObjectType(classOf[java.math.BigDecimal]) case StringType => ObjectType(classOf[java.lang.String]) case _: ArrayType => ObjectType(classOf[scala.collection.Seq[_]]) @@ -235,13 +235,9 @@ object RowEncoder { case udt: UserDefinedType[_] => ObjectType(udt.userClass) } - private def deserializerFor(schema: StructType): Expression = { + private def deserializerFor(input: Expression, schema: StructType): Expression = { val fields = schema.zipWithIndex.map { case (f, i) => - val dt = f.dataType match { - case p: PythonUserDefinedType => p.sqlType - case other => other - } - deserializerFor(GetColumnByOrdinal(i, dt)) + deserializerFor(GetStructField(input, i)) } CreateExternalRow(fields, schema) } @@ -272,27 +268,22 @@ object RowEncoder { Invoke(obj, "deserialize", ObjectType(udt.userClass), input :: Nil) case TimestampType => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(classOf[java.sql.Timestamp]), - "toJavaTimestamp", - input :: Nil, - returnNullable = false) + if (SQLConf.get.datetimeJava8ApiEnabled) { + createDeserializerForInstant(input) + } else { + createDeserializerForSqlTimestamp(input) + } case DateType => - StaticInvoke( - DateTimeUtils.getClass, - ObjectType(classOf[java.sql.Date]), - "toJavaDate", - input :: Nil, - returnNullable = false) + if (SQLConf.get.datetimeJava8ApiEnabled) { + createDeserializerForLocalDate(input) + } else { + createDeserializerForSqlDate(input) + } - case _: DecimalType => - Invoke(input, "toJavaBigDecimal", ObjectType(classOf[java.math.BigDecimal]), - returnNullable = false) + case _: DecimalType => createDeserializerForJavaBigDecimal(input, returnNullable = false) - case StringType => - Invoke(input, "toString", ObjectType(classOf[String]), returnNullable = false) + case StringType => createDeserializerForString(input, returnNullable = false) case ArrayType(et, nullable) => val arrayData = @@ -332,4 +323,10 @@ object RowEncoder { Literal.create(null, externalDataTypeFor(input.dataType)), CreateExternalRow(convertedFields, schema)) } + + private def expressionForNullableExpr( + expr: Expression, + newExprWhenNotNull: Expression): Expression = { + If(IsNull(expr), Literal.create(null, newExprWhenNotNull.dataType), newExprWhenNotNull) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala index a7e09eee617e9..038ebb2037a27 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/AttributeSet.scala @@ -63,7 +63,7 @@ object AttributeSet { * when the transformation was a no-op). */ class AttributeSet private (val baseSet: Set[AttributeEquals]) - extends Traversable[Attribute] with Serializable { + extends Iterable[Attribute] with Serializable { override def hashCode: Int = baseSet.hashCode() @@ -99,7 +99,7 @@ class AttributeSet private (val baseSet: Set[AttributeEquals]) * Returns a new [[AttributeSet]] that does not contain any of the [[Attribute Attributes]] found * in `other`. */ - def --(other: Traversable[NamedExpression]): AttributeSet = { + def --(other: Iterable[NamedExpression]): AttributeSet = { other match { case otherSet: AttributeSet => new AttributeSet(baseSet -- otherSet.baseSet) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala index 77582e10f9ff2..7ae5924b20faf 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -34,15 +34,11 @@ case class BoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) override def toString: String = s"input[$ordinal, ${dataType.simpleString}, $nullable]" - private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType) + private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType, nullable) // Use special getter for primitive types (for UnsafeRow) override def eval(input: InternalRow): Any = { - if (nullable && input.isNullAt(ordinal)) { - null - } else { - accessor(input, ordinal) - } + accessor(input, ordinal) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -90,4 +86,13 @@ object BindReferences extends Logging { } }.asInstanceOf[A] // Kind of a hack, but safe. TODO: Tighten return type when possible. } + + /** + * A helper function to bind given expressions to an input schema. + */ + def bindReferences[A <: Expression]( + expressions: Seq[A], + input: AttributeSeq): Seq[A] = { + expressions.map(BindReferences.bindReference(_, input)) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala index fe6db8b344d3d..4d218b936b3a2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Canonicalize.scala @@ -26,6 +26,7 @@ package org.apache.spark.sql.catalyst.expressions * * The following rules are applied: * - Names and nullability hints for [[org.apache.spark.sql.types.DataType]]s are stripped. + * - Names for [[GetStructField]] are stripped. * - Commutative and associative operations ([[Add]] and [[Multiply]]) have their children ordered * by `hashCode`. * - [[EqualTo]] and [[EqualNullSafe]] are reordered by `hashCode`. @@ -37,10 +38,11 @@ object Canonicalize { expressionReorder(ignoreNamesTypes(e)) } - /** Remove names and nullability from types. */ + /** Remove names and nullability from types, and names from `GetStructField`. */ private[expressions] def ignoreNamesTypes(e: Expression): Expression = e match { case a: AttributeReference => AttributeReference("none", a.dataType.asNullable)(exprId = a.exprId) + case GetStructField(child, ordinal, Some(_)) => GetStructField(child, ordinal, None) case _ => e } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala index ee463bf5eb6ac..969128838eba4 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 @@ -18,14 +18,18 @@ package org.apache.spark.sql.catalyst.expressions import java.math.{BigDecimal => JavaBigDecimal} +import java.time.ZoneId +import java.util.concurrent.TimeUnit._ import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{InternalRow, WalkedTypePath} import org.apache.spark.sql.catalyst.analysis.{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._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.unsafe.types.UTF8String.{IntWrapper, LongWrapper} @@ -42,6 +46,7 @@ object Cast { case (_, StringType) => true case (StringType, BinaryType) => true + case (_: IntegralType, BinaryType) => true case (StringType, BooleanType) => true case (DateType, BooleanType) => true @@ -116,35 +121,36 @@ object Cast { } /** - * Return true iff we may truncate during casting `from` type to `to` type. e.g. long -> int, - * timestamp -> date. + * Returns true iff we can safely up-cast the `from` type to `to` type without any truncating or + * precision lose or possible runtime failures. For example, long -> int, string -> int are not + * up-cast. */ - def mayTruncate(from: DataType, to: DataType): Boolean = (from, to) match { - case (from: NumericType, to: DecimalType) if !to.isWiderThan(from) => true - case (from: DecimalType, to: NumericType) if !from.isTighterThan(to) => true - case (from, to) if illegalNumericPrecedence(from, to) => true - case (TimestampType, DateType) => true - case (StringType, to: NumericType) => true - case _ => false - } - - private def illegalNumericPrecedence(from: DataType, to: DataType): Boolean = { - val fromPrecedence = TypeCoercion.numericPrecedence.indexOf(from) - val toPrecedence = TypeCoercion.numericPrecedence.indexOf(to) - toPrecedence > 0 && fromPrecedence > toPrecedence - } - - /** - * Returns true iff we can safely cast the `from` type to `to` type without any truncating or - * precision lose, e.g. int -> long, date -> timestamp. - */ - def canSafeCast(from: AtomicType, to: AtomicType): Boolean = (from, to) match { + def canUpCast(from: DataType, to: DataType): Boolean = (from, to) match { case _ if from == to => true case (from: NumericType, to: DecimalType) if to.isWiderThan(from) => true case (from: DecimalType, to: NumericType) if from.isTighterThan(to) => true - case (from, to) if legalNumericPrecedence(from, to) => true + case (f, t) if legalNumericPrecedence(f, t) => true case (DateType, TimestampType) => true case (_, StringType) => true + + // Spark supports casting between long and timestamp, please see `longToTimestamp` and + // `timestampToLong` for details. + case (TimestampType, LongType) => true + case (LongType, TimestampType) => true + + case (ArrayType(fromType, fn), ArrayType(toType, tn)) => + resolvableNullability(fn, tn) && canUpCast(fromType, toType) + + case (MapType(fromKey, fromValue, fn), MapType(toKey, toValue, tn)) => + resolvableNullability(fn, tn) && canUpCast(fromKey, toKey) && canUpCast(fromValue, toValue) + + case (StructType(fromFields), StructType(toFields)) => + fromFields.length == toFields.length && + fromFields.zip(toFields).forall { + case (f1, f2) => + resolvableNullability(f1.nullable, f2.nullable) && canUpCast(f1.dataType, f2.dataType) + } + case _ => false } @@ -230,12 +236,15 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // [[func]] assumes the input is no longer null because eval already does the null check. @inline private[this] def buildCast[T](a: Any, func: T => Any): Any = func(a.asInstanceOf[T]) + private lazy val dateFormatter = DateFormatter() + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter(zoneId) + // UDFToString private[this] def castToString(from: DataType): Any => Any = from match { case BinaryType => buildCast[Array[Byte]](_, UTF8String.fromBytes) - case DateType => buildCast[Int](_, d => UTF8String.fromString(DateTimeUtils.dateToString(d))) + case DateType => buildCast[Int](_, d => UTF8String.fromString(dateFormatter.format(d))) case TimestampType => buildCast[Long](_, - t => UTF8String.fromString(DateTimeUtils.timestampToString(t, timeZone))) + t => UTF8String.fromString(DateTimeUtils.timestampToString(timestampFormatter, t))) case ArrayType(et, _) => buildCast[ArrayData](_, array => { val builder = new UTF8StringBuilder @@ -321,6 +330,10 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // BinaryConverter private[this] def castToBinary(from: DataType): Any => Any = from match { case StringType => buildCast[UTF8String](_, _.getBytes) + case ByteType => buildCast[Byte](_, NumberConverter.toBinary) + case ShortType => buildCast[Short](_, NumberConverter.toBinary) + case IntegerType => buildCast[Int](_, NumberConverter.toBinary) + case LongType => buildCast[Long](_, NumberConverter.toBinary) } // UDFToBoolean @@ -359,7 +372,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // TimestampConverter private[this] def castToTimestamp(from: DataType): Any => Any = from match { case StringType => - buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, timeZone).orNull) + buildCast[UTF8String](_, utfs => DateTimeUtils.stringToTimestamp(utfs, zoneId).orNull) case BooleanType => buildCast[Boolean](_, b => if (b) 1L else 0) case LongType => @@ -371,7 +384,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case ByteType => buildCast[Byte](_, b => longToTimestamp(b.toLong)) case DateType => - buildCast[Int](_, d => DateTimeUtils.daysToMillis(d, timeZone) * 1000) + buildCast[Int](_, d => epochDaysToMicros(d, zoneId)) // TimestampWritable.decimalToTimestamp case DecimalType() => buildCast[Decimal](_, d => decimalToTimestamp(d)) @@ -384,19 +397,21 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } private[this] def decimalToTimestamp(d: Decimal): Long = { - (d.toBigDecimal * 1000000L).longValue() + (d.toBigDecimal * MICROS_PER_SECOND).longValue() } private[this] def doubleToTimestamp(d: Double): Any = { - if (d.isNaN || d.isInfinite) null else (d * 1000000L).toLong + if (d.isNaN || d.isInfinite) null else (d * MICROS_PER_SECOND).toLong } // converting seconds to us - private[this] def longToTimestamp(t: Long): Long = t * 1000000L + private[this] def longToTimestamp(t: Long): Long = SECONDS.toMicros(t) // converting us to seconds - private[this] def timestampToLong(ts: Long): Long = math.floor(ts.toDouble / 1000000L).toLong + private[this] def timestampToLong(ts: Long): Long = { + Math.floorDiv(ts, MICROS_PER_SECOND) + } // converting us to seconds in double private[this] def timestampToDouble(ts: Long): Double = { - ts / 1000000.0 + ts / MICROS_PER_SECOND.toDouble } // DateConverter @@ -406,7 +421,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case TimestampType => // throw valid precision more than seconds, according to Hive. // Timestamp.nanos is in 0 to 999,999,999, no more than a second. - buildCast[Long](_, t => DateTimeUtils.millisToDays(t / 1000L, timeZone)) + buildCast[Long](_, t => microsToEpochDays(t, zoneId)) } // IntervalConverter @@ -607,6 +622,12 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String // We can return what the children return. Same thing should happen in the codegen path. if (DataType.equalsStructurally(from, to)) { identity + } else if (from == NullType) { + // According to `canCast`, NullType can be casted to any type. + // For primitive types, we don't reach here because the guard of `nullSafeEval`. + // But for nested types like struct, we might reach here for nested null type field. + // We won't call the returned function actually, but returns a placeholder. + _ => throw new SparkException(s"should not directly cast from NullType to $to.") } else { to match { case dt if dt == from => identity[Any] @@ -843,12 +864,16 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case BinaryType => (c, evPrim, evNull) => code"$evPrim = UTF8String.fromBytes($c);" case DateType => - (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.dateToString($c));""" + val df = JavaCode.global( + ctx.addReferenceObj("dateFormatter", dateFormatter), + dateFormatter.getClass) + (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString(${df}.format($c));""" case TimestampType => - val tz = JavaCode.global(ctx.addReferenceObj("timeZone", timeZone), timeZone.getClass) + val tf = JavaCode.global( + ctx.addReferenceObj("timestampFormatter", timestampFormatter), + timestampFormatter.getClass) (c, evPrim, evNull) => code"""$evPrim = UTF8String.fromString( - org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($c, $tz));""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampToString($tf, $c));""" case ArrayType(et, _) => (c, evPrim, evNull) => { val buffer = ctx.freshVariable("buffer", classOf[UTF8StringBuilder]) @@ -897,7 +922,11 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String private[this] def castToBinaryCode(from: DataType): CastFunction = from match { case StringType => - (c, evPrim, evNull) => code"$evPrim = $c.getBytes();" + (c, evPrim, evNull) => + code"$evPrim = $c.getBytes();" + case _: IntegralType => + (c, evPrim, evNull) => + code"$evPrim = ${NumberConverter.getClass.getName.stripSuffix("$")}.toBinary($c);" } private[this] def castToDateCode( @@ -915,10 +944,13 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } """ case TimestampType => - val tz = JavaCode.global(ctx.addReferenceObj("timeZone", timeZone), timeZone.getClass) + val zoneIdClass = classOf[ZoneId] + val zid = JavaCode.global( + ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName), + zoneIdClass) (c, evPrim, evNull) => code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.millisToDays($c / 1000L, $tz);""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.microsToEpochDays($c, $zid);""" case _ => (c, evPrim, evNull) => code"$evNull = true;" } @@ -1005,12 +1037,15 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String from: DataType, ctx: CodegenContext): CastFunction = from match { case StringType => - val tz = JavaCode.global(ctx.addReferenceObj("timeZone", timeZone), timeZone.getClass) + val zoneIdClass = classOf[ZoneId] + val zid = JavaCode.global( + ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName), + zoneIdClass) val longOpt = ctx.freshVariable("longOpt", classOf[Option[Long]]) (c, evPrim, evNull) => code""" scala.Option $longOpt = - org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToTimestamp($c, $tz); + org.apache.spark.sql.catalyst.util.DateTimeUtils.stringToTimestamp($c, $zid); if ($longOpt.isDefined()) { $evPrim = ((Long) $longOpt.get()).longValue(); } else { @@ -1022,10 +1057,13 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String case _: IntegralType => (c, evPrim, evNull) => code"$evPrim = ${longToTimeStampCode(c)};" case DateType => - val tz = JavaCode.global(ctx.addReferenceObj("timeZone", timeZone), timeZone.getClass) + val zoneIdClass = classOf[ZoneId] + val zid = JavaCode.global( + ctx.addReferenceObj("zoneId", zoneId, zoneIdClass.getName), + zoneIdClass) (c, evPrim, evNull) => code"""$evPrim = - org.apache.spark.sql.catalyst.util.DateTimeUtils.daysToMillis($c, $tz) * 1000;""" + org.apache.spark.sql.catalyst.util.DateTimeUtils.epochDaysToMicros($c, $zid);""" case DecimalType() => (c, evPrim, evNull) => code"$evPrim = ${decimalToTimestampCode(c)};" case DoubleType => @@ -1034,7 +1072,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (Double.isNaN($c) || Double.isInfinite($c)) { $evNull = true; } else { - $evPrim = (long)($c * 1000000L); + $evPrim = (long)($c * $MICROS_PER_SECOND); } """ case FloatType => @@ -1043,7 +1081,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String if (Float.isNaN($c) || Float.isInfinite($c)) { $evNull = true; } else { - $evPrim = (long)($c * 1000000L); + $evPrim = (long)($c * $MICROS_PER_SECOND); } """ } @@ -1060,14 +1098,14 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String } private[this] def decimalToTimestampCode(d: ExprValue): Block = { - val block = inline"new java.math.BigDecimal(1000000L)" + val block = inline"new java.math.BigDecimal($MICROS_PER_SECOND)" code"($d.toBigDecimal().bigDecimal().multiply($block)).longValue()" } - private[this] def longToTimeStampCode(l: ExprValue): Block = code"$l * 1000000L" + private[this] def longToTimeStampCode(l: ExprValue): Block = code"$l * (long)$MICROS_PER_SECOND" private[this] def timestampToIntegerCode(ts: ExprValue): Block = - code"java.lang.Math.floor((double) $ts / 1000000L)" + code"java.lang.Math.floorDiv($ts, $MICROS_PER_SECOND)" private[this] def timestampToDoubleCode(ts: ExprValue): Block = - code"$ts / 1000000.0" + code"$ts / (double)$MICROS_PER_SECOND" private[this] def castToBooleanCode(from: DataType): CastFunction = from match { case StringType => @@ -1365,7 +1403,7 @@ case class Cast(child: Expression, dataType: DataType, timeZoneId: Option[String * Cast the child expression to the target data type, but will throw error if the cast might * truncate, e.g. long -> int, timestamp -> data. */ -case class UpCast(child: Expression, dataType: DataType, walkedTypePath: Seq[String]) +case class UpCast(child: Expression, dataType: DataType, walkedTypePath: Seq[String] = Nil) extends UnaryExpression with Unevaluable { override lazy val resolved = false } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala index 8d06804ce1e90..72ff9361d8f75 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/EquivalentExpressions.scala @@ -41,7 +41,7 @@ class EquivalentExpressions { } // For each expression, the set of equivalent expressions. - private val equivalenceMap = mutable.HashMap.empty[Expr, mutable.MutableList[Expression]] + private val equivalenceMap = mutable.HashMap.empty[Expr, mutable.ArrayBuffer[Expression]] /** * Adds each expression to this data structure, grouping them with existing equivalent @@ -56,7 +56,7 @@ class EquivalentExpressions { f.get += expr true } else { - equivalenceMap.put(e, mutable.MutableList(expr)) + equivalenceMap.put(e, mutable.ArrayBuffer(expr)) false } } else { @@ -102,7 +102,7 @@ class EquivalentExpressions { * an empty collection if there are none. */ def getEquivalentExprs(e: Expression): Seq[Expression] = { - equivalenceMap.getOrElse(Expr(e), mutable.MutableList()) + equivalenceMap.getOrElse(Expr(e), Seq.empty) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala new file mode 100644 index 0000000000000..3f3d6b2b63a06 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ExprUtils.scala @@ -0,0 +1,107 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.text.{DecimalFormat, DecimalFormatSymbols, ParsePosition} +import java.util.Locale + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.ArrayBasedMapData +import org.apache.spark.sql.types.{DataType, MapType, StringType, StructType} +import org.apache.spark.unsafe.types.UTF8String + +object ExprUtils { + + def evalSchemaExpr(exp: Expression): StructType = { + // Use `DataType.fromDDL` since the type string can be struct<...>. + val dataType = exp match { + case Literal(s, StringType) => + DataType.fromDDL(s.toString) + case e @ SchemaOfCsv(_: Literal, _) => + val ddlSchema = e.eval(EmptyRow).asInstanceOf[UTF8String] + DataType.fromDDL(ddlSchema.toString) + case e => throw new AnalysisException( + "Schema should be specified in DDL format as a string literal or output of " + + s"the schema_of_csv function instead of ${e.sql}") + } + + if (!dataType.isInstanceOf[StructType]) { + throw new AnalysisException( + s"Schema should be struct type but got ${dataType.sql}.") + } + dataType.asInstanceOf[StructType] + } + + def evalTypeExpr(exp: Expression): DataType = exp match { + case Literal(s, StringType) => DataType.fromDDL(s.toString) + case e @ SchemaOfJson(_: Literal, _) => + val ddlSchema = e.eval(EmptyRow).asInstanceOf[UTF8String] + DataType.fromDDL(ddlSchema.toString) + case e => throw new AnalysisException( + "Schema should be specified in DDL format as a string literal or output of " + + s"the schema_of_json function instead of ${e.sql}") + } + + def convertToMapData(exp: Expression): Map[String, String] = exp match { + case m: CreateMap + if m.dataType.acceptsType(MapType(StringType, StringType, valueContainsNull = false)) => + val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] + ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => + key.toString -> value.toString + } + case m: CreateMap => + throw new AnalysisException( + s"A type of keys and values in map() must be string, but got ${m.dataType.catalogString}") + case _ => + throw new AnalysisException("Must use a map() function for options") + } + + /** + * A convenient function for schema validation in datasources supporting + * `columnNameOfCorruptRecord` as an option. + */ + def verifyColumnNameOfCorruptRecord( + schema: StructType, + columnNameOfCorruptRecord: String): Unit = { + schema.getFieldIndex(columnNameOfCorruptRecord).foreach { corruptFieldIndex => + val f = schema(corruptFieldIndex) + if (f.dataType != StringType || !f.nullable) { + throw new AnalysisException( + "The field for corrupt records must be string type and nullable") + } + } + } + + def getDecimalParser(locale: Locale): String => java.math.BigDecimal = { + if (locale == Locale.US) { // Special handling the default locale for backward compatibility + (s: String) => new java.math.BigDecimal(s.replaceAll(",", "")) + } else { + val decimalFormat = new DecimalFormat("", new DecimalFormatSymbols(locale)) + decimalFormat.setParseBigDecimal(true) + (s: String) => { + val pos = new ParsePosition(0) + val result = decimalFormat.parse(s, pos).asInstanceOf[java.math.BigDecimal] + if (pos.getIndex() != s.length() || pos.getErrorIndex() != -1) { + throw new IllegalArgumentException("Cannot parse any decimal"); + } else { + result + } + } + } + } +} 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 c215735ab1c98..b793d81f61788 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 @@ -21,11 +21,14 @@ 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.expressions.aggregate.DeclarativeAggregate import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines the basic expression abstract classes in Catalyst. @@ -38,26 +41,51 @@ import org.apache.spark.util.Utils * "name(arguments...)", the concrete implementation must be a case class whose constructor * arguments are all Expressions types. See [[Substring]] for an example. * - * There are a few important traits: + * There are a few important traits or abstract classes: * * - [[Nondeterministic]]: an expression that is not deterministic. + * - [[Stateful]]: an expression that contains mutable state. For example, MonotonicallyIncreasingID + * and Rand. A stateful expression is always non-deterministic. * - [[Unevaluable]]: an expression that is not supposed to be evaluated. * - [[CodegenFallback]]: an expression that does not have code gen implemented and falls back to * interpreted mode. + * - [[NullIntolerant]]: an expression that is null intolerant (i.e. any null input will result in + * null output). + * - [[NonSQLExpression]]: a common base trait for the expressions that do not have SQL + * expressions like representation. For example, `ScalaUDF`, `ScalaUDAF`, + * and object `MapObjects` and `Invoke`. + * - [[UserDefinedExpression]]: a common base trait for user-defined functions, including + * UDF/UDAF/UDTF. + * - [[HigherOrderFunction]]: a common base trait for higher order functions that take one or more + * (lambda) functions and applies these to some objects. The function + * produces a number of variables which can be consumed by some lambda + * functions. + * - [[NamedExpression]]: An [[Expression]] that is named. + * - [[TimeZoneAwareExpression]]: A common base trait for time zone aware expressions. + * - [[SubqueryExpression]]: A base interface for expressions that contain a [[LogicalPlan]]. * * - [[LeafExpression]]: an expression that has no child. * - [[UnaryExpression]]: an expression that has one child. * - [[BinaryExpression]]: an expression that has two children. * - [[TernaryExpression]]: an expression that has three children. + * - [[QuaternaryExpression]]: an expression that has four children. * - [[BinaryOperator]]: a special case of [[BinaryExpression]] that requires two children to have * the same output data type. * + * A few important traits used for type coercion rules: + * - [[ExpectsInputTypes]]: an expression that has the expected input types. This trait is typically + * used by operator expressions (e.g. [[Add]], [[Subtract]]) to define + * expected input types without any implicit casting. + * - [[ImplicitCastInputTypes]]: an expression that has the expected input types, which can be + * implicitly castable using [[TypeCoercion.ImplicitTypeCasts]]. + * - [[ComplexTypeMergingExpression]]: to resolve output types of the complex expressions + * (e.g., [[CaseWhen]]). */ abstract class Expression extends TreeNode[Expression] { /** * Returns true when an expression is a candidate for static evaluation before the query is - * executed. + * executed. A typical use case: [[org.apache.spark.sql.catalyst.optimizer.ConstantFolding]] * * The following conditions are used to determine suitability for constant folding: * - A [[Coalesce]] is foldable if all of its children are foldable @@ -70,7 +98,8 @@ abstract class Expression extends TreeNode[Expression] { /** * Returns true when the current expression always return the same result for fixed inputs from - * children. + * children. The non-deterministic expressions should not change in number and order. They should + * not be evaluated during the query planning. * * Note that this means that an expression should be considered as non-deterministic if: * - it relies on some mutable internal state, or @@ -85,7 +114,14 @@ abstract class Expression extends TreeNode[Expression] { def nullable: Boolean - def references: AttributeSet = AttributeSet.fromAttributeSets(children.map(_.references)) + /** + * Workaround scala compiler so that we can call super on lazy vals + */ + @transient + private lazy val _references: AttributeSet = + AttributeSet.fromAttributeSets(children.map(_.references)) + + def references: AttributeSet = _references /** Returns the result of evaluating this expression on a given input Row */ def eval(input: InternalRow = null): Any @@ -120,7 +156,8 @@ abstract class Expression extends TreeNode[Expression] { private def reduceCodeSize(ctx: CodegenContext, eval: ExprCode): Unit = { // TODO: support whole stage codegen too - if (eval.code.length > 1024 && ctx.INPUT_ROW != null && ctx.currentVars == null) { + val splitThreshold = SQLConf.get.methodSplitThreshold + if (eval.code.length > splitThreshold && ctx.INPUT_ROW != null && ctx.currentVars == null) { val setIsNull = if (!eval.isNull.isInstanceOf[LiteralValue]) { val globalIsNull = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "globalIsNull") val localIsNull = eval.isNull @@ -224,18 +261,18 @@ abstract class Expression extends TreeNode[Expression] { def prettyName: String = nodeName.toLowerCase(Locale.ROOT) protected def flatArguments: Iterator[Any] = productIterator.flatMap { - case t: Traversable[_] => t + case t: Iterable[_] => t case single => single :: Nil } // Marks this as final, Expression.verboseString should never be called, and thus shouldn't be // overridden by concrete classes. - final override def verboseString: String = simpleString + final override def verboseString(maxFields: Int): String = simpleString(maxFields) - override def simpleString: String = toString + override def simpleString(maxFields: Int): String = toString - override def toString: String = prettyName + Utils.truncatedString( - flatArguments.toSeq, "(", ", ", ")") + override def toString: String = prettyName + truncatedString( + flatArguments.toSeq, "(", ", ", ")", SQLConf.get.maxToStringFields) /** * Returns SQL representation of this expression. For expressions extending [[NonSQLExpression]], @@ -249,8 +286,9 @@ abstract class Expression extends TreeNode[Expression] { /** - * An expression that cannot be evaluated. Some expressions don't live past analysis or optimization - * time (e.g. Star). This trait is used by those expressions. + * An expression that cannot be evaluated. These expressions don't live past analysis or + * optimization time (e.g. Star) and should not be evaluated during query planning and + * execution. */ trait Unevaluable extends Expression { @@ -258,7 +296,7 @@ trait Unevaluable extends Expression { throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") } @@ -282,6 +320,31 @@ trait RuntimeReplaceable extends UnaryExpression with Unevaluable { override lazy val canonicalized: Expression = child.canonicalized } +/** + * An aggregate expression that gets rewritten (currently by the optimizer) into a + * different aggregate expression for evaluation. This is mainly used to provide compatibility + * with other databases. For example, we use this to support every, any/some aggregates by rewriting + * them with Min and Max respectively. + */ +trait UnevaluableAggregate extends DeclarativeAggregate { + + override def nullable: Boolean = true + + override lazy val aggBufferAttributes = + throw new UnsupportedOperationException(s"Cannot evaluate aggBufferAttributes: $this") + + override lazy val initialValues: Seq[Expression] = + throw new UnsupportedOperationException(s"Cannot evaluate initialValues: $this") + + override lazy val updateExpressions: Seq[Expression] = + throw new UnsupportedOperationException(s"Cannot evaluate updateExpressions: $this") + + override lazy val mergeExpressions: Seq[Expression] = + throw new UnsupportedOperationException(s"Cannot evaluate mergeExpressions: $this") + + override lazy val evaluateExpression: Expression = + throw new UnsupportedOperationException(s"Cannot evaluate evaluateExpression: $this") +} /** * Expressions that don't have SQL representation should extend this trait. Examples are @@ -572,7 +635,7 @@ abstract class BinaryOperator extends BinaryExpression with ExpectsInputTypes { def sqlOperator: String = symbol - override def toString: String = s"($left $symbol $right)" + override def toString: String = s"($left $sqlOperator $right)" override def inputTypes: Seq[AbstractDataType] = Seq(inputType, inputType) @@ -696,9 +759,115 @@ abstract class TernaryExpression extends Expression { } /** - * A trait resolving nullable, containsNull, valueContainsNull flags of the output date type. - * This logic is usually utilized by expressions combining data from multiple child expressions - * of non-primitive types (e.g. [[CaseWhen]]). + * An expression with four inputs and one output. The output is by default evaluated to null + * if any input is evaluated to null. + */ +abstract class QuaternaryExpression extends Expression { + + override def foldable: Boolean = children.forall(_.foldable) + + override def nullable: Boolean = children.exists(_.nullable) + + /** + * Default behavior of evaluation according to the default nullability of QuaternaryExpression. + * If subclass of QuaternaryExpression override nullable, probably should also override this. + */ + override def eval(input: InternalRow): Any = { + val exprs = children + val value1 = exprs(0).eval(input) + if (value1 != null) { + val value2 = exprs(1).eval(input) + if (value2 != null) { + val value3 = exprs(2).eval(input) + if (value3 != null) { + val value4 = exprs(3).eval(input) + if (value4 != null) { + return nullSafeEval(value1, value2, value3, value4) + } + } + } + } + null + } + + /** + * Called by default [[eval]] implementation. If subclass of QuaternaryExpression keep the + * default nullability, they can override this method to save null-check code. If we need + * full control of evaluation process, we should override [[eval]]. + */ + protected def nullSafeEval(input1: Any, input2: Any, input3: Any, input4: Any): Any = + sys.error(s"QuaternaryExpressions must override either eval or nullSafeEval") + + /** + * Short hand for generating quaternary evaluation code. + * If either of the sub-expressions is null, the result of this computation + * is assumed to be null. + * + * @param f accepts four variable names and returns Java code to compute the output. + */ + protected def defineCodeGen( + ctx: CodegenContext, + ev: ExprCode, + f: (String, String, String, String) => String): ExprCode = { + nullSafeCodeGen(ctx, ev, (eval1, eval2, eval3, eval4) => { + s"${ev.value} = ${f(eval1, eval2, eval3, eval4)};" + }) + } + + /** + * Short hand for generating quaternary evaluation code. + * If either of the sub-expressions is null, the result of this computation + * is assumed to be null. + * + * @param f function that accepts the 4 non-null evaluation result names of children + * and returns Java code to compute the output. + */ + protected def nullSafeCodeGen( + ctx: CodegenContext, + ev: ExprCode, + f: (String, String, String, String) => String): ExprCode = { + val firstGen = children(0).genCode(ctx) + val secondGen = children(1).genCode(ctx) + val thridGen = children(2).genCode(ctx) + val fourthGen = children(3).genCode(ctx) + val resultCode = f(firstGen.value, secondGen.value, thridGen.value, fourthGen.value) + + if (nullable) { + val nullSafeEval = + firstGen.code + ctx.nullSafeExec(children(0).nullable, firstGen.isNull) { + secondGen.code + ctx.nullSafeExec(children(1).nullable, secondGen.isNull) { + thridGen.code + ctx.nullSafeExec(children(2).nullable, thridGen.isNull) { + fourthGen.code + ctx.nullSafeExec(children(3).nullable, fourthGen.isNull) { + s""" + ${ev.isNull} = false; // resultCode could change nullability. + $resultCode + """ + } + } + } + } + + ev.copy(code = code""" + boolean ${ev.isNull} = true; + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + $nullSafeEval""") + } else { + ev.copy(code = code""" + ${firstGen.code} + ${secondGen.code} + ${thridGen.code} + ${fourthGen.code} + ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; + $resultCode""", isNull = FalseLiteral) + } + } +} + +/** + * A trait used for resolving nullable flags, including `nullable`, `containsNull` of [[ArrayType]] + * and `valueContainsNull` of [[MapType]], containsNull, valueContainsNull flags of the output date + * type. This is usually utilized by the expressions (e.g. [[CaseWhen]]) that combine data from + * multiple child expressions of non-primitive types. */ trait ComplexTypeMergingExpression extends Expression { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GetStructFieldObject.scala similarity index 88% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GetStructFieldObject.scala index c88b2f8c034fc..0bea0cbfa2324 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GetStructFieldObject.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/GetStructFieldObject.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions.{Expression, GetStructField} import org.apache.spark.sql.types.StructField /** @@ -25,7 +24,7 @@ import org.apache.spark.sql.types.StructField * This is in contrast to the [[GetStructField]] case class extractor which returns the field * ordinal instead of the field itself. */ -private[execution] object GetStructFieldObject { +object GetStructFieldObject { def unapply(getStructField: GetStructField): Option[(Expression, StructField)] = Some(( getStructField.child, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala index 0654108cea281..5c8aa4e2e9d83 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedMutableProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp @@ -30,7 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp */ class InterpretedMutableProjection(expressions: Seq[Expression]) extends MutableProjection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = - this(toBoundExprs(expressions, inputSchema)) + this(bindReferences(expressions, inputSchema)) private[this] val buffer = new Array[Any](expressions.size) @@ -49,10 +50,31 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable def currentValue: InternalRow = mutableRow override def target(row: InternalRow): MutableProjection = { + // If `mutableRow` is `UnsafeRow`, `MutableProjection` accepts fixed-length types only + require(!row.isInstanceOf[UnsafeRow] || + validExprs.forall { case (e, _) => UnsafeRow.isFixedLength(e.dataType) }, + "MutableProjection cannot use UnsafeRow for output data types: " + + validExprs.map(_._1.dataType).filterNot(UnsafeRow.isFixedLength) + .map(_.catalogString).mkString(", ")) mutableRow = row this } + private[this] val fieldWriters: Array[Any => Unit] = validExprs.map { case (e, i) => + val writer = InternalRow.getWriter(i, e.dataType) + if (!e.nullable) { + (v: Any) => writer(mutableRow, v) + } else { + (v: Any) => { + if (v == null) { + mutableRow.setNullAt(i) + } else { + writer(mutableRow, v) + } + } + } + }.toArray + override def apply(input: InternalRow): InternalRow = { var i = 0 while (i < validExprs.length) { @@ -64,7 +86,7 @@ class InterpretedMutableProjection(expressions: Seq[Expression]) extends Mutable i = 0 while (i < validExprs.length) { val (_, ordinal) = validExprs(i) - mutableRow(ordinal) = buffer(ordinal) + fieldWriters(i)(buffer(ordinal)) i += 1 } mutableRow diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala new file mode 100644 index 0000000000000..70789dac1d87a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/InterpretedSafeProjection.scala @@ -0,0 +1,125 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.types._ + + +/** + * An interpreted version of a safe projection. + * + * @param expressions that produces the resulting fields. These expressions must be bound + * to a schema. + */ +class InterpretedSafeProjection(expressions: Seq[Expression]) extends Projection { + + private[this] val mutableRow = new SpecificInternalRow(expressions.map(_.dataType)) + + private[this] val exprsWithWriters = expressions.zipWithIndex.filter { + case (NoOp, _) => false + case _ => true + }.map { case (e, i) => + val converter = generateSafeValueConverter(e.dataType) + val writer = InternalRow.getWriter(i, e.dataType) + val f = if (!e.nullable) { + (v: Any) => writer(mutableRow, converter(v)) + } else { + (v: Any) => { + if (v == null) { + mutableRow.setNullAt(i) + } else { + writer(mutableRow, converter(v)) + } + } + } + (e, f) + } + + private def generateSafeValueConverter(dt: DataType): Any => Any = dt match { + case ArrayType(elemType, _) => + val elementConverter = generateSafeValueConverter(elemType) + v => { + val arrayValue = v.asInstanceOf[ArrayData] + val result = new Array[Any](arrayValue.numElements()) + arrayValue.foreach(elemType, (i, e) => { + result(i) = elementConverter(e) + }) + new GenericArrayData(result) + } + + case st: StructType => + val fieldTypes = st.fields.map(_.dataType) + val fieldConverters = fieldTypes.map(generateSafeValueConverter) + v => { + val row = v.asInstanceOf[InternalRow] + val ar = new Array[Any](row.numFields) + var idx = 0 + while (idx < row.numFields) { + ar(idx) = fieldConverters(idx)(row.get(idx, fieldTypes(idx))) + idx += 1 + } + new GenericInternalRow(ar) + } + + case MapType(keyType, valueType, _) => + lazy val keyConverter = generateSafeValueConverter(keyType) + lazy val valueConverter = generateSafeValueConverter(valueType) + v => { + val mapValue = v.asInstanceOf[MapData] + val keys = mapValue.keyArray().toArray[Any](keyType) + val values = mapValue.valueArray().toArray[Any](valueType) + val convertedKeys = keys.map(keyConverter) + val convertedValues = values.map(valueConverter) + ArrayBasedMapData(convertedKeys, convertedValues) + } + + case udt: UserDefinedType[_] => + generateSafeValueConverter(udt.sqlType) + + case _ => identity + } + + override def apply(row: InternalRow): InternalRow = { + var i = 0 + while (i < exprsWithWriters.length) { + val (expr, writer) = exprsWithWriters(i) + writer(expr.eval(row)) + i += 1 + } + mutableRow + } +} + +/** + * Helper functions for creating an [[InterpretedSafeProjection]]. + */ +object InterpretedSafeProjection { + + /** + * Returns an [[SafeProjection]] for given sequence of bound Expressions. + */ + def createProjection(exprs: Seq[Expression]): Projection = { + // We need to make sure that we do not reuse stateful expressions. + val cleanedExpressions = exprs.map(_.transform { + case s: Stateful => s.freshCopy() + }) + new InterpretedSafeProjection(cleanedExpressions) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala index 792646cf9f10c..eaaf94baac216 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen.{GenerateMutableProjection, GenerateSafeProjection, GenerateUnsafeProjection} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DataType, StructType} @@ -30,7 +31,7 @@ import org.apache.spark.sql.types.{DataType, StructType} */ class InterpretedProjection(expressions: Seq[Expression]) extends Projection { def this(expressions: Seq[Expression], inputSchema: Seq[Attribute]) = - this(expressions.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(expressions, inputSchema)) override def initialize(partitionIndex: Int): Unit = { expressions.foreach(_.foreach { @@ -99,7 +100,7 @@ object MutableProjection * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): MutableProjection = { - create(toBoundExprs(exprs, inputSchema)) + create(bindReferences(exprs, inputSchema)) } } @@ -162,33 +163,47 @@ object UnsafeProjection * `inputSchema`. */ def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): UnsafeProjection = { - create(toBoundExprs(exprs, inputSchema)) + create(bindReferences(exprs, inputSchema)) } } /** * A projection that could turn UnsafeRow into GenericInternalRow */ -object FromUnsafeProjection { +object SafeProjection extends CodeGeneratorWithInterpretedFallback[Seq[Expression], Projection] { + + override protected def createCodeGeneratedObject(in: Seq[Expression]): Projection = { + GenerateSafeProjection.generate(in) + } + + override protected def createInterpretedObject(in: Seq[Expression]): Projection = { + InterpretedSafeProjection.createProjection(in) + } + + /** + * Returns a SafeProjection for given StructType. + */ + def create(schema: StructType): Projection = create(schema.fields.map(_.dataType)) /** - * Returns a Projection for given StructType. + * Returns a SafeProjection for given Array of DataTypes. */ - def apply(schema: StructType): Projection = { - apply(schema.fields.map(_.dataType)) + def create(fields: Array[DataType]): Projection = { + createObject(fields.zipWithIndex.map(x => new BoundReference(x._2, x._1, true))) } /** - * Returns an UnsafeProjection for given Array of DataTypes. + * Returns a SafeProjection for given sequence of Expressions (bounded). */ - def apply(fields: Seq[DataType]): Projection = { - create(fields.zipWithIndex.map(x => new BoundReference(x._2, x._1, true))) + def create(exprs: Seq[Expression]): Projection = { + createObject(exprs) } /** - * Returns a Projection for given sequence of Expressions (bounded). + * Returns a SafeProjection for given sequence of Expressions, which will be bound to + * `inputSchema`. */ - private def create(exprs: Seq[Expression]): Projection = { - GenerateSafeProjection.generate(exprs) + def create(exprs: Seq[Expression], inputSchema: Seq[Attribute]): Projection = { + create(bindReferences(exprs, inputSchema)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala similarity index 90% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala index 612a7b87b9832..13c6f8db7c129 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ProjectionOverSchema.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ProjectionOverSchema.scala @@ -15,9 +15,8 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ /** @@ -26,7 +25,7 @@ import org.apache.spark.sql.types._ * are adjusted to fit the schema. All other expressions are left as-is. This * class is motivated by columnar nested schema pruning. */ -private[execution] case class ProjectionOverSchema(schema: StructType) { +case class ProjectionOverSchema(schema: StructType) { private val fieldNames = schema.fieldNames.toSet def unapply(expr: Expression): Option[Expression] = getProjection(expr) @@ -50,6 +49,10 @@ private[execution] case class ProjectionOverSchema(schema: StructType) { s"unmatched child schema for GetArrayStructFields: ${projSchema.toString}" ) } + case MapKeys(child) => + getProjection(child).map { projection => MapKeys(projection) } + case MapValues(child) => + getProjection(child).map { projection => MapValues(projection) } case GetMapValue(child, key) => getProjection(child).map { projection => GetMapValue(projection, key) } case GetStructFieldObject(child, field: StructField) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala index 6530b176968f2..690969e1d359d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/PythonUDF.scala @@ -27,7 +27,8 @@ import org.apache.spark.sql.types.DataType object PythonUDF { private[this] val SCALAR_TYPES = Set( PythonEvalType.SQL_BATCHED_UDF, - PythonEvalType.SQL_SCALAR_PANDAS_UDF + PythonEvalType.SQL_SCALAR_PANDAS_UDF, + PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF ) def isScalarPythonUDF(e: Expression): Boolean = { @@ -45,7 +46,8 @@ object PythonUDF { } /** - * A serialized version of a Python lambda function. + * A serialized version of a Python lambda function. This is a special expression, which needs a + * dedicated physical operator to execute it, and thus can't be pushed down to data sources. */ case class PythonUDF( name: String, 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 8954fe8a58e6e..10f8ec9617d1b 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 @@ -18,10 +18,10 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkException -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.types.{AbstractDataType, DataType} /** * User-defined function. @@ -31,6 +31,10 @@ import org.apache.spark.sql.types.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 @@ -39,35 +43,21 @@ import org.apache.spark.sql.types.DataType * @param nullable True if the UDF can return null value. * @param udfDeterministic True if the UDF is deterministic. Deterministic UDF returns same result * each time it is invoked with a particular input. - * @param nullableTypes which of the inputTypes are nullable (i.e. not primitive) */ case class ScalaUDF( function: AnyRef, dataType: DataType, children: Seq[Expression], - inputTypes: Seq[DataType] = Nil, + inputPrimitives: Seq[Boolean], + inputTypes: Seq[AbstractDataType] = Nil, udfName: Option[String] = None, nullable: Boolean = true, - udfDeterministic: Boolean = true, - nullableTypes: Seq[Boolean] = Nil) - extends Expression with ImplicitCastInputTypes with NonSQLExpression with UserDefinedExpression { - - // The constructor for SPARK 2.1 and 2.2 - def this( - function: AnyRef, - dataType: DataType, - children: Seq[Expression], - inputTypes: Seq[DataType], - udfName: Option[String]) = { - this( - function, dataType, children, inputTypes, udfName, nullable = true, - udfDeterministic = true, nullableTypes = Nil) - } + udfDeterministic: Boolean = true) + extends Expression with NonSQLExpression with UserDefinedExpression { override lazy val deterministic: Boolean = udfDeterministic && children.forall(_.deterministic) - override def toString: String = - s"${udfName.map(name => s"UDF:$name").getOrElse("UDF")}(${children.mkString(", ")})" + override def toString: String = s"${udfName.getOrElse("UDF")}(${children.mkString(", ")})" // scalastyle:off line.size.limit @@ -1012,22 +1002,38 @@ case class ScalaUDF( // such as IntegerType, its javaType is `int` and the returned type of user-defined // function is Object. Trying to convert an Object to `int` will cause casting exception. val evalCode = evals.map(_.code).mkString("\n") - val (funcArgs, initArgs) = evals.zipWithIndex.map { case (eval, i) => - val argTerm = ctx.freshName("arg") - val convert = s"$convertersTerm[$i].apply(${eval.value})" - val initArg = s"Object $argTerm = ${eval.isNull} ? null : $convert;" - (argTerm, initArg) + val (funcArgs, initArgs) = evals.zipWithIndex.zip(children.map(_.dataType)).map { + case ((eval, i), dt) => + val argTerm = ctx.freshName("arg") + val initArg = if (CatalystTypeConverters.isPrimitive(dt)) { + val convertedTerm = ctx.freshName("conv") + s""" + |${CodeGenerator.boxedType(dt)} $convertedTerm = ${eval.value}; + |Object $argTerm = ${eval.isNull} ? null : $convertedTerm; + """.stripMargin + } else { + s"Object $argTerm = ${eval.isNull} ? null : $convertersTerm[$i].apply(${eval.value});" + } + (argTerm, initArg) }.unzip val udf = ctx.addReferenceObj("udf", function, s"scala.Function${children.length}") val getFuncResult = s"$udf.apply(${funcArgs.mkString(", ")})" val resultConverter = s"$convertersTerm[${children.length}]" val boxedType = CodeGenerator.boxedType(dataType) + + val funcInvokation = if (CatalystTypeConverters.isPrimitive(dataType) + // If the output is nullable, the returned value must be unwrapped from the Option + && !nullable) { + s"$resultTerm = ($boxedType)$getFuncResult" + } else { + s"$resultTerm = ($boxedType)$resultConverter.apply($getFuncResult)" + } val callFunc = s""" |$boxedType $resultTerm = null; |try { - | $resultTerm = ($boxedType)$resultConverter.apply($getFuncResult); + | $funcInvokation; |} catch (Exception e) { | throw new org.apache.spark.SparkException($errorMsgTerm, e); |} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala new file mode 100644 index 0000000000000..6213267c41c64 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruning.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.types._ + +object SchemaPruning { + /** + * Filters the schema by the requested fields. For example, if the schema is struct, + * and given requested field are "a", the field "b" is pruned in the returned schema. + * Note that schema field ordering at original schema is still preserved in pruned schema. + */ + def pruneDataSchema( + dataSchema: StructType, + requestedRootFields: Seq[RootField]): StructType = { + // Merge the requested root fields into a single schema. Note the ordering of the fields + // in the resulting schema may differ from their ordering in the logical relation's + // original schema + val mergedSchema = requestedRootFields + .map { case root: RootField => StructType(Array(root.field)) } + .reduceLeft(_ merge _) + val dataSchemaFieldNames = dataSchema.fieldNames.toSet + val mergedDataSchema = + StructType(mergedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) + // Sort the fields of mergedDataSchema according to their order in dataSchema, + // recursively. This makes mergedDataSchema a pruned schema of dataSchema + sortLeftFieldsByRight(mergedDataSchema, dataSchema).asInstanceOf[StructType] + } + + /** + * Sorts the fields and descendant fields of structs in left according to their order in + * right. This function assumes that the fields of left are a subset of the fields of + * right, recursively. That is, left is a "subschema" of right, ignoring order of + * fields. + */ + private def sortLeftFieldsByRight(left: DataType, right: DataType): DataType = + (left, right) match { + case (ArrayType(leftElementType, containsNull), ArrayType(rightElementType, _)) => + ArrayType( + sortLeftFieldsByRight(leftElementType, rightElementType), + containsNull) + case (MapType(leftKeyType, leftValueType, containsNull), + MapType(rightKeyType, rightValueType, _)) => + MapType( + sortLeftFieldsByRight(leftKeyType, rightKeyType), + sortLeftFieldsByRight(leftValueType, rightValueType), + containsNull) + case (leftStruct: StructType, rightStruct: StructType) => + val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains) + val sortedLeftFields = filteredRightFieldNames.map { fieldName => + val leftFieldType = leftStruct(fieldName).dataType + val rightFieldType = rightStruct(fieldName).dataType + val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType) + StructField(fieldName, sortedLeftFieldType, nullable = leftStruct(fieldName).nullable) + } + StructType(sortedLeftFields) + case _ => left + } + + /** + * Returns the set of fields from projection and filtering predicates that the query plan needs. + */ + def identifyRootFields( + projects: Seq[NamedExpression], + filters: Seq[Expression]): Seq[RootField] = { + val projectionRootFields = projects.flatMap(getRootFields) + val filterRootFields = filters.flatMap(getRootFields) + + // Kind of expressions don't need to access any fields of a root fields, e.g., `IsNotNull`. + // For them, if there are any nested fields accessed in the query, we don't need to add root + // field access of above expressions. + // For example, for a query `SELECT name.first FROM contacts WHERE name IS NOT NULL`, + // we don't need to read nested fields of `name` struct other than `first` field. + val (rootFields, optRootFields) = (projectionRootFields ++ filterRootFields) + .distinct.partition(!_.prunedIfAnyChildAccessed) + + optRootFields.filter { opt => + !rootFields.exists { root => + root.field.name == opt.field.name && { + // Checking if current optional root field can be pruned. + // For each required root field, we merge it with the optional root field: + // 1. If this optional root field has nested fields and any nested field of it is used + // in the query, the merged field type must equal to the optional root field type. + // We can prune this optional root field. For example, for optional root field + // `struct>`, if its field + // `struct>` is used, we don't need to add this optional + // root field. + // 2. If this optional root field has no nested fields, the merged field type equals + // to the optional root field only if they are the same. If they are, we can prune + // this optional root field too. + val rootFieldType = StructType(Array(root.field)) + val optFieldType = StructType(Array(opt.field)) + val merged = optFieldType.merge(rootFieldType) + merged.sameType(optFieldType) + } + } + } ++ rootFields + } + + /** + * Gets the root (aka top-level, no-parent) [[StructField]]s for the given [[Expression]]. + * When expr is an [[Attribute]], construct a field around it and indicate that that + * field was derived from an attribute. + */ + private def getRootFields(expr: Expression): Seq[RootField] = { + expr match { + case att: Attribute => + RootField(StructField(att.name, att.dataType, att.nullable), derivedFromAtt = true) :: Nil + case SelectedField(field) => RootField(field, derivedFromAtt = false) :: Nil + // Root field accesses by `IsNotNull` and `IsNull` are special cases as the expressions + // don't actually use any nested fields. These root field accesses might be excluded later + // if there are any nested fields accesses in the query plan. + case IsNotNull(SelectedField(field)) => + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil + case IsNull(SelectedField(field)) => + RootField(field, derivedFromAtt = false, prunedIfAnyChildAccessed = true) :: Nil + case IsNotNull(_: Attribute) | IsNull(_: Attribute) => + expr.children.flatMap(getRootFields).map(_.copy(prunedIfAnyChildAccessed = true)) + case _ => + expr.children.flatMap(getRootFields) + } + } + + /** + * This represents a "root" schema field (aka top-level, no-parent). `field` is the + * `StructField` for field name and datatype. `derivedFromAtt` indicates whether it + * was derived from an attribute or had a proper child. `prunedIfAnyChildAccessed` means + * whether this root field can be pruned if any of child field is used in the query. + */ + case class RootField(field: StructField, derivedFromAtt: Boolean, + prunedIfAnyChildAccessed: Boolean = false) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala new file mode 100644 index 0000000000000..7ba3d302d553b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SelectedField.scala @@ -0,0 +1,134 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.types._ + +/** + * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst + * complex type extractor. For example, consider a relation with the following schema: + * + * {{{ + * root + * |-- name: struct (nullable = true) + * | |-- first: string (nullable = true) + * | |-- last: string (nullable = true) + * }}} + * + * Further, suppose we take the select expression `name.first`. This will parse into an + * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern: + * + * {{{ + * GetStructFieldObject( + * AttributeReference("name", StructType(_), _, _), + * StructField("first", StringType, _, _)) + * }}} + * + * [[SelectedField]] converts that expression into + * + * {{{ + * StructField("name", StructType(Array(StructField("first", StringType)))) + * }}} + * + * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the + * same name as its child (or "parent" going right to left in the select expression) and a data + * type appropriate to the complex type extractor. In our example, the name of the child expression + * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string + * field named "first". + */ +object SelectedField { + def unapply(expr: Expression): Option[StructField] = { + // If this expression is an alias, work on its child instead + val unaliased = expr match { + case Alias(child, _) => child + case expr => expr + } + selectField(unaliased, None) + } + + /** + * Convert an expression into the parts of the schema (the field) it accesses. + */ + private def selectField(expr: Expression, dataTypeOpt: Option[DataType]): Option[StructField] = { + expr match { + case a: Attribute => + dataTypeOpt.map { dt => + StructField(a.name, dt, a.nullable) + } + case c: GetStructField => + val field = c.childSchema(c.ordinal) + val newField = field.copy(dataType = dataTypeOpt.getOrElse(field.dataType)) + selectField(c.child, Option(struct(newField))) + case GetArrayStructFields(child, field, _, _, containsNull) => + val newFieldDataType = dataTypeOpt match { + case None => + // GetArrayStructFields is the top level extractor. This means its result is + // not pruned and we need to use the element type of the array its producing. + field.dataType + case Some(ArrayType(dataType, _)) => + // GetArrayStructFields is part of a chain of extractors and its result is pruned + // by a parent expression. In this case need to use the parent element type. + dataType + case Some(x) => + // This should not happen. + throw new AnalysisException(s"DataType '$x' is not supported by GetArrayStructFields.") + } + val newField = StructField(field.name, newFieldDataType, field.nullable) + selectField(child, Option(ArrayType(struct(newField), containsNull))) + case GetMapValue(child, _) => + // GetMapValue does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val MapType(keyType, _, valueContainsNull) = child.dataType + val opt = dataTypeOpt.map(dt => MapType(keyType, dt, valueContainsNull)) + selectField(child, opt) + case MapValues(child) => + val MapType(keyType, _, valueContainsNull) = child.dataType + // MapValues does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val opt = dataTypeOpt.map { + case ArrayType(dataType, _) => MapType(keyType, dataType, valueContainsNull) + case x => + // This should not happen. + throw new AnalysisException(s"DataType '$x' is not supported by MapValues.") + } + selectField(child, opt) + case MapKeys(child) => + val MapType(_, valueType, valueContainsNull) = child.dataType + // MapKeys does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val opt = dataTypeOpt.map { + case ArrayType(dataType, _) => MapType(dataType, valueType, valueContainsNull) + case x => + // This should not happen. + throw new AnalysisException(s"DataType '$x' is not supported by MapKeys.") + } + selectField(child, opt) + case GetArrayItem(child, _) => + // GetArrayItem does not select a field from a struct (i.e. prune the struct) so it can't be + // the top-level extractor. However it can be part of an extractor chain. + val ArrayType(_, containsNull) = child.dataType + val opt = dataTypeOpt.map(dt => ArrayType(dt, containsNull)) + selectField(child, opt) + case _ => + None + } + } + + private def struct(field: StructField): StructType = StructType(Array(field)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala index 8e48856d4607c..9aae678deb4bc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/TimeWindow.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.commons.lang3.StringUtils - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.analysis.TypeCheckResult.TypeCheckFailure @@ -104,20 +102,7 @@ object TimeWindow { * precision. */ private def getIntervalInMicroSeconds(interval: String): Long = { - if (StringUtils.isBlank(interval)) { - throw new IllegalArgumentException( - "The window duration, slide duration and start time cannot be null or blank.") - } - val intervalString = if (interval.startsWith("interval")) { - interval - } else { - "interval " + interval - } - val cal = CalendarInterval.fromString(intervalString) - if (cal == null) { - throw new IllegalArgumentException( - s"The provided interval ($interval) did not correspond to a valid interval string.") - } + val cal = CalendarInterval.fromCaseInsensitiveString(interval) if (cal.months > 0) { throw new IllegalArgumentException( s"Intervals greater than a month is not supported ($interval).") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala index c790d87492c73..ea0ed2e8fa11b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/ApproximatePercentile.scala @@ -64,7 +64,8 @@ import org.apache.spark.sql.types._ [10.0,10.0,10.0] > SELECT _FUNC_(10.0, 0.5, 100); 10.0 - """) + """, + since = "2.1.0") case class ApproximatePercentile( child: Expression, percentageExpression: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala index 5ecb77be5965e..66ac73087b4d9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Average.scala @@ -23,9 +23,29 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ -abstract class AverageLike(child: Expression) extends DeclarativeAggregate { +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns the mean calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col); + 2.0 + > SELECT _FUNC_(col) FROM VALUES (1), (2), (NULL) AS tab(col); + 1.5 + """, + since = "1.0.0") +case class Average(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes { + + override def prettyName: String = "avg" + + override def children: Seq[Expression] = child :: Nil + + override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) + + override def checkInputDataTypes(): TypeCheckResult = + TypeUtils.checkForNumericExpr(child.dataType, "function average") override def nullable: Boolean = true + // Return data type. override def dataType: DataType = resultType @@ -63,28 +83,11 @@ abstract class AverageLike(child: Expression) extends DeclarativeAggregate { sum.cast(resultType) / count.cast(resultType) } - protected def updateExpressionsDef: Seq[Expression] = Seq( + override lazy val updateExpressions: Seq[Expression] = Seq( /* sum = */ Add( sum, coalesce(child.cast(sumDataType), Literal(0).cast(sumDataType))), /* count = */ If(child.isNull, count, count + 1L) ) - - override lazy val updateExpressions = updateExpressionsDef -} - -@ExpressionDescription( - usage = "_FUNC_(expr) - Returns the mean calculated from values of a group.") -case class Average(child: Expression) - extends AverageLike(child) with ImplicitCastInputTypes { - - override def prettyName: String = "avg" - - override def children: Seq[Expression] = child :: Nil - - override def inputTypes: Seq[AbstractDataType] = Seq(NumericType) - - override def checkInputDataTypes(): TypeCheckResult = - TypeUtils.checkForNumericExpr(child.dataType, "function average") } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala index e2ff0efba07ca..1870c58c548c9 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CentralMomentAgg.scala @@ -135,7 +135,13 @@ abstract class CentralMomentAgg(child: Expression) // Compute the population standard deviation of a column // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the population standard deviation calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the population standard deviation calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col); + 0.816496580927726 + """, + since = "1.6.0") // scalastyle:on line.size.limit case class StddevPop(child: Expression) extends CentralMomentAgg(child) { @@ -151,7 +157,13 @@ case class StddevPop(child: Expression) extends CentralMomentAgg(child) { // Compute the sample standard deviation of a column // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the sample standard deviation calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sample standard deviation calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col); + 1.0 + """, + since = "1.6.0") // scalastyle:on line.size.limit case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { @@ -167,7 +179,13 @@ case class StddevSamp(child: Expression) extends CentralMomentAgg(child) { // Compute the population variance of a column @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the population variance calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the population variance calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col); + 0.6666666666666666 + """, + since = "1.6.0") case class VariancePop(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 2 @@ -181,7 +199,13 @@ case class VariancePop(child: Expression) extends CentralMomentAgg(child) { // Compute the sample variance of a column @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the sample variance calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sample variance calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (3) AS tab(col); + 1.0 + """, + since = "1.6.0") case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 2 @@ -195,7 +219,15 @@ case class VarianceSamp(child: Expression) extends CentralMomentAgg(child) { } @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the skewness value calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the skewness value calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); + 1.1135657469022013 + > SELECT _FUNC_(col) FROM VALUES (-1000), (-100), (10), (20) AS tab(col); + -1.1135657469022011 + """, + since = "1.6.0") case class Skewness(child: Expression) extends CentralMomentAgg(child) { override def prettyName: String = "skewness" @@ -209,7 +241,15 @@ case class Skewness(child: Expression) extends CentralMomentAgg(child) { } @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the kurtosis value calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the kurtosis value calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (-10), (-20), (100), (1000) AS tab(col); + -0.7014368047529618 + > SELECT _FUNC_(col) FROM VALUES (1), (10), (100), (10), (1) as tab(col); + 0.19432323191698986 + """, + since = "1.6.0") case class Kurtosis(child: Expression) extends CentralMomentAgg(child) { override protected def momentOrder = 4 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala index e14cc716ea223..91446e05d853d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Corr.scala @@ -93,7 +93,13 @@ abstract class PearsonCorrelation(x: Expression, y: Expression) // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr1, expr2) - Returns Pearson coefficient of correlation between a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns Pearson coefficient of correlation between a set of number pairs.", + examples = """ + Examples: + > SELECT _FUNC_(c1, c2) FROM VALUES (3, 2), (3, 3), (6, 4) as tab(c1, c2); + 0.8660254037844387 + """, + since = "1.6.0") // scalastyle:on line.size.limit case class Corr(x: Expression, y: Expression) extends PearsonCorrelation(x, y) { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala index 40582d0abd762..2a8edac502c0f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Count.scala @@ -21,10 +21,27 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.types._ -/** - * Base class for all counting aggregators. - */ -abstract class CountLike extends DeclarativeAggregate { +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. + + _FUNC_(expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are all non-null. + + _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. + """, + examples = """ + Examples: + > SELECT _FUNC_(*) FROM VALUES (NULL), (5), (5), (20) AS tab(col); + 4 + > SELECT _FUNC_(col) FROM VALUES (NULL), (5), (5), (20) AS tab(col); + 3 + > SELECT _FUNC_(DISTINCT col) FROM VALUES (NULL), (5), (5), (10) AS tab(col); + 2 + """, + since = "1.0.0") +// scalastyle:on line.size.limit +case class Count(children: Seq[Expression]) extends DeclarativeAggregate { override def nullable: Boolean = false // Return data type. @@ -45,19 +62,6 @@ abstract class CountLike extends DeclarativeAggregate { override lazy val evaluateExpression = count override def defaultResult: Option[Literal] = Option(Literal(0L)) -} - -// scalastyle:off line.size.limit -@ExpressionDescription( - usage = """ - _FUNC_(*) - Returns the total number of retrieved rows, including rows containing null. - - _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-null. - - _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-null. - """) -// scalastyle:on line.size.limit -case class Count(children: Seq[Expression]) extends CountLike { override lazy val updateExpressions = { val nullableChildren = children.filter(_.nullable) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala new file mode 100644 index 0000000000000..d31355cd022fa --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountIf.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionDescription, ImplicitCastInputTypes, UnevaluableAggregate} +import org.apache.spark.sql.types.{AbstractDataType, BooleanType, DataType, LongType} + +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns the number of `TRUE` values for the expression. + """, + examples = """ + Examples: + > SELECT _FUNC_(col % 2 = 0) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col); + 2 + > SELECT _FUNC_(col IS NULL) FROM VALUES (NULL), (0), (1), (2), (3) AS tab(col); + 1 + """, + since = "3.0.0") +case class CountIf(predicate: Expression) extends UnevaluableAggregate with ImplicitCastInputTypes { + override def prettyName: String = "count_if" + + override def children: Seq[Expression] = Seq(predicate) + + override def nullable: Boolean = false + + override def dataType: DataType = LongType + + override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType) + + override def checkInputDataTypes(): TypeCheckResult = predicate.dataType match { + case BooleanType => + TypeCheckResult.TypeCheckSuccess + case _ => + TypeCheckResult.TypeCheckFailure( + s"function $prettyName requires boolean type, not ${predicate.dataType.catalogString}" + ) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala index dae88c7b1861c..4bd13cf284935 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/CountMinSketchAgg.scala @@ -43,7 +43,8 @@ import org.apache.spark.util.sketch.CountMinSketch confidence and seed. The result is an array of bytes, which can be deserialized to a `CountMinSketch` before usage. Count-min sketch is a probabilistic data structure used for cardinality estimation using sub-linear space. - """) + """, + since = "2.2.0") case class CountMinSketchAgg( child: Expression, epsExpression: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala index ee28eb591882f..ac99fa8049f93 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Covariance.scala @@ -80,7 +80,13 @@ abstract class Covariance(x: Expression, y: Expression) } @ExpressionDescription( - usage = "_FUNC_(expr1, expr2) - Returns the population covariance of a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns the population covariance of a set of number pairs.", + examples = """ + Examples: + > SELECT _FUNC_(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2); + 0.6666666666666666 + """, + since = "2.0.0") case class CovPopulation(left: Expression, right: Expression) extends Covariance(left, right) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), ck / n) @@ -90,7 +96,13 @@ case class CovPopulation(left: Expression, right: Expression) extends Covariance @ExpressionDescription( - usage = "_FUNC_(expr1, expr2) - Returns the sample covariance of a set of number pairs.") + usage = "_FUNC_(expr1, expr2) - Returns the sample covariance of a set of number pairs.", + examples = """ + Examples: + > SELECT _FUNC_(c1, c2) FROM VALUES (1,1), (2,2), (3,3) AS tab(c1, c2); + 1.0 + """, + since = "2.0.0") case class CovSample(left: Expression, right: Expression) extends Covariance(left, right) { override val evaluateExpression: Expression = { If(n === 0.0, Literal.create(null, DoubleType), 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 f51bfd591204a..9f351395846e4 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 @@ -33,8 +33,17 @@ import org.apache.spark.sql.types._ @ExpressionDescription( usage = """ _FUNC_(expr[, isIgnoreNull]) - Returns the first value of `expr` for a group of rows. - If `isIgnoreNull` is true, returns only non-null values. - """) + If `isIgnoreNull` is true, returns only non-null values.""", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (10), (5), (20) AS tab(col); + 10 + > SELECT _FUNC_(col) FROM VALUES (NULL), (5), (20) AS tab(col); + NULL + > SELECT _FUNC_(col, true) FROM VALUES (NULL), (5), (20) AS tab(col); + 5 + """, + since = "2.0.0") case class First(child: Expression, ignoreNullsExpr: Expression) extends DeclarativeAggregate with ExpectsInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala index 0b737885f4b17..e3c628e70d11b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/HyperLogLogPlusPlus.scala @@ -47,8 +47,13 @@ import org.apache.spark.sql.types._ @ExpressionDescription( usage = """ _FUNC_(expr[, relativeSD]) - Returns the estimated cardinality by HyperLogLog++. - `relativeSD` defines the maximum estimation error allowed. - """) + `relativeSD` defines the maximum estimation error allowed.""", + examples = """ + Examples: + > SELECT _FUNC_(col1) FROM VALUES (1), (1), (2), (2), (3) tab(col1); + 3 + """, + since = "1.6.0") case class HyperLogLogPlusPlus( child: Expression, relativeSD: Double = 0.05, 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 2650d7b5908fd..405719faaeb58 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 @@ -33,8 +33,17 @@ import org.apache.spark.sql.types._ @ExpressionDescription( usage = """ _FUNC_(expr[, isIgnoreNull]) - Returns the last value of `expr` for a group of rows. - If `isIgnoreNull` is true, returns only non-null values. - """) + If `isIgnoreNull` is true, returns only non-null values""", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (10), (5), (20) AS tab(col); + 20 + > SELECT _FUNC_(col) FROM VALUES (10), (5), (NULL) AS tab(col); + NULL + > SELECT _FUNC_(col, true) FROM VALUES (10), (5), (NULL) AS tab(col); + 5 + """, + since = "2.0.0") case class Last(child: Expression, ignoreNullsExpr: Expression) extends DeclarativeAggregate with ExpectsInputTypes { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala index 71099eba0fc75..7520db146ba6a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Max.scala @@ -24,7 +24,13 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the maximum value of `expr`.") + usage = "_FUNC_(expr) - Returns the maximum value of `expr`.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (10), (50), (20) AS tab(col); + 50 + """, + since = "1.0.0") case class Max(child: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala new file mode 100644 index 0000000000000..c7fdb15130c4f --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/MaxByAndMinBy.scala @@ -0,0 +1,128 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.types._ + +/** + * The shared abstract superclass for `MaxBy` and `MinBy` SQL aggregate functions. + */ +abstract class MaxMinBy extends DeclarativeAggregate { + + def valueExpr: Expression + def orderingExpr: Expression + + protected def funcName: String + // The predicate compares two ordering values. + protected def predicate(oldExpr: Expression, newExpr: Expression): Expression + // The arithmetic expression returns greatest/least value of all parameters. + // Used to pick up updated ordering value. + protected def orderingUpdater(oldExpr: Expression, newExpr: Expression): Expression + + override def children: Seq[Expression] = valueExpr :: orderingExpr :: Nil + + override def nullable: Boolean = true + + // Return data type. + override def dataType: DataType = valueExpr.dataType + + override def checkInputDataTypes(): TypeCheckResult = + TypeUtils.checkForOrderingExpr(orderingExpr.dataType, s"function $funcName") + + // The attributes used to keep extremum (max or min) and associated aggregated values. + private lazy val extremumOrdering = + AttributeReference("extremumOrdering", orderingExpr.dataType)() + private lazy val valueWithExtremumOrdering = + AttributeReference("valueWithExtremumOrdering", valueExpr.dataType)() + + override lazy val aggBufferAttributes: Seq[AttributeReference] = + valueWithExtremumOrdering :: extremumOrdering :: Nil + + private lazy val nullValue = Literal.create(null, valueExpr.dataType) + private lazy val nullOrdering = Literal.create(null, orderingExpr.dataType) + + override lazy val initialValues: Seq[Literal] = Seq( + /* valueWithExtremumOrdering = */ nullValue, + /* extremumOrdering = */ nullOrdering + ) + + override lazy val updateExpressions: Seq[Expression] = Seq( + /* valueWithExtremumOrdering = */ + CaseWhen( + (extremumOrdering.isNull && orderingExpr.isNull, nullValue) :: + (extremumOrdering.isNull, valueExpr) :: + (orderingExpr.isNull, valueWithExtremumOrdering) :: Nil, + If(predicate(extremumOrdering, orderingExpr), valueWithExtremumOrdering, valueExpr) + ), + /* extremumOrdering = */ orderingUpdater(extremumOrdering, orderingExpr) + ) + + override lazy val mergeExpressions: Seq[Expression] = Seq( + /* valueWithExtremumOrdering = */ + CaseWhen( + (extremumOrdering.left.isNull && extremumOrdering.right.isNull, nullValue) :: + (extremumOrdering.left.isNull, valueWithExtremumOrdering.right) :: + (extremumOrdering.right.isNull, valueWithExtremumOrdering.left) :: Nil, + If(predicate(extremumOrdering.left, extremumOrdering.right), + valueWithExtremumOrdering.left, valueWithExtremumOrdering.right) + ), + /* extremumOrdering = */ orderingUpdater(extremumOrdering.left, extremumOrdering.right) + ) + + override lazy val evaluateExpression: AttributeReference = valueWithExtremumOrdering +} + +@ExpressionDescription( + usage = "_FUNC_(x, y) - Returns the value of `x` associated with the maximum value of `y`.", + examples = """ + Examples: + > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); + b + """, + since = "3.0") +case class MaxBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { + override protected def funcName: String = "max_by" + + override protected def predicate(oldExpr: Expression, newExpr: Expression): Expression = + oldExpr > newExpr + + override protected def orderingUpdater(oldExpr: Expression, newExpr: Expression): Expression = + greatest(oldExpr, newExpr) +} + +@ExpressionDescription( + usage = "_FUNC_(x, y) - Returns the value of `x` associated with the minimum value of `y`.", + examples = """ + Examples: + > SELECT _FUNC_(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y); + a + """, + since = "3.0") +case class MinBy(valueExpr: Expression, orderingExpr: Expression) extends MaxMinBy { + override protected def funcName: String = "min_by" + + override protected def predicate(oldExpr: Expression, newExpr: Expression): Expression = + oldExpr < newExpr + + override protected def orderingUpdater(oldExpr: Expression, newExpr: Expression): Expression = + least(oldExpr, newExpr) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala index 8c4ba93231cbe..106eb968e3917 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Min.scala @@ -24,7 +24,13 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the minimum value of `expr`.") + usage = "_FUNC_(expr) - Returns the minimum value of `expr`.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (10), (-1), (20) AS tab(col); + -1 + """, + since = "1.0.0") case class Min(child: Expression) extends DeclarativeAggregate { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala index 4894036e27463..a91a6d7d166ea 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala @@ -54,7 +54,15 @@ import org.apache.spark.util.collection.OpenHashMap of the percentage array must be between 0.0 and 1.0. The value of frequency should be positive integral - """) + """, + examples = """ + Examples: + > SELECT _FUNC_(col, 0.3) FROM VALUES (0), (10) AS tab(col); + 3.0 + > SELECT _FUNC_(col, array(0.25, 0.75)) FROM VALUES (0), (10) AS tab(col); + [2.5,7.5] + """, + since = "2.1.0") case class Percentile( child: Expression, percentageExpression: Expression, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala index 761dba111c074..ef204ec82c527 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Sum.scala @@ -24,7 +24,17 @@ import org.apache.spark.sql.catalyst.util.TypeUtils import org.apache.spark.sql.types._ @ExpressionDescription( - usage = "_FUNC_(expr) - Returns the sum calculated from values of a group.") + usage = "_FUNC_(expr) - Returns the sum calculated from values of a group.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (5), (10), (15) AS tab(col); + 30 + > SELECT _FUNC_(col) FROM VALUES (NULL), (10), (15) AS tab(col); + 25 + > SELECT _FUNC_(col) FROM VALUES (NULL), (NULL) AS tab(col); + NULL + """, + since = "1.0.0") case class Sum(child: Expression) extends DeclarativeAggregate with ImplicitCastInputTypes { override def children: Seq[Expression] = child :: Nil diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala new file mode 100644 index 0000000000000..4562fbcff5f3d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/UnevaluableAggs.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.types._ + +abstract class UnevaluableBooleanAggBase(arg: Expression) + extends UnevaluableAggregate with ImplicitCastInputTypes { + + override def children: Seq[Expression] = arg :: Nil + + override def dataType: DataType = BooleanType + + override def inputTypes: Seq[AbstractDataType] = Seq(BooleanType) + + override def checkInputDataTypes(): TypeCheckResult = { + arg.dataType match { + case dt if dt != BooleanType => + TypeCheckResult.TypeCheckFailure(s"Input to function '$prettyName' should have been " + + s"${BooleanType.simpleString}, but it's [${arg.dataType.catalogString}].") + case _ => TypeCheckResult.TypeCheckSuccess + } + } +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns true if all values of `expr` are true.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (true), (true), (true) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (NULL), (true), (true) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (true), (false), (true) AS tab(col); + false + """, + since = "3.0.0") +case class EveryAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = "Every" +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (true), (false), (false) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (NULL), (true), (false) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (false), (false), (NULL) AS tab(col); + false + """, + since = "3.0.0") +case class AnyAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = "Any" +} + +@ExpressionDescription( + usage = "_FUNC_(expr) - Returns true if at least one value of `expr` is true.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (true), (false), (false) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (NULL), (true), (false) AS tab(col); + true + > SELECT _FUNC_(col) FROM VALUES (false), (false), (NULL) AS tab(col); + false + """, + since = "3.0.0") +case class SomeAgg(arg: Expression) extends UnevaluableBooleanAggBase(arg) { + override def nodeName: String = "Some" +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala index be972f006352e..7cc43dfdfac33 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/collect.scala @@ -86,7 +86,13 @@ abstract class Collect[T <: Growable[Any] with Iterable[Any]] extends TypedImper * Collect a list of elements. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Collects and returns a list of non-unique elements.") + usage = "_FUNC_(expr) - Collects and returns a list of non-unique elements.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (1) AS tab(col); + [1,2,1] + """, + since = "2.0.0") case class CollectList( child: Expression, mutableAggBufferOffset: Int = 0, @@ -109,7 +115,13 @@ case class CollectList( * Collect a set of unique elements. */ @ExpressionDescription( - usage = "_FUNC_(expr) - Collects and returns a set of unique elements.") + usage = "_FUNC_(expr) - Collects and returns a set of unique elements.", + examples = """ + Examples: + > SELECT _FUNC_(col) FROM VALUES (1), (2), (1) AS tab(col); + [1,2] + """, + since = "2.0.0") case class CollectSet( child: Expression, mutableAggBufferOffset: Int = 0, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index e1d16a2cd38b0..d863f83686020 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -127,13 +127,12 @@ case class AggregateExpression( override def foldable: Boolean = false override def nullable: Boolean = aggregateFunction.nullable - override def references: AttributeSet = { - val childReferences = mode match { - case Partial | Complete => aggregateFunction.references.toSeq - case PartialMerge | Final => aggregateFunction.aggBufferAttributes + @transient + override lazy val references: AttributeSet = { + mode match { + case Partial | Complete => aggregateFunction.references + case PartialMerge | Final => AttributeSet(aggregateFunction.aggBufferAttributes) } - - AttributeSet(childReferences) } override def toString: String = { @@ -160,7 +159,7 @@ case class AggregateExpression( * ([[aggBufferAttributes]]) of an aggregation buffer which is used to hold partial aggregate * results. At runtime, multiple aggregate functions are evaluated by the same operator using a * combined aggregation buffer which concatenates the aggregation buffers of the individual - * aggregate functions. + * aggregate functions. Please note that aggregate functions should be stateless. * * Code which accepts [[AggregateFunction]] instances should be prepared to handle both types of * aggregate functions. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala index cc24e397cc14a..c766bd8e56bb6 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/bitwiseExpressions.scala @@ -95,7 +95,7 @@ case class BitwiseOr(left: Expression, right: Expression) extends BinaryArithmet examples = """ Examples: > SELECT 3 _FUNC_ 5; - 2 + 6 """) case class BitwiseXor(left: Expression, right: Expression) extends BinaryArithmetic { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala index ea1bb87d415c9..2ec31458270f3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeFormatter.scala @@ -41,7 +41,8 @@ object CodeFormatter { val commentReplaced = commentHolder.replaceAllIn( line.trim, m => code.comment.get(m.group(1)).map(Matcher.quoteReplacement).getOrElse(m.group(0))) - formatter.addLine(commentReplaced) + val comments = commentReplaced.split("\n") + comments.foreach(formatter.addLine) } if (needToTruncate) { formatter.addLine(s"[truncated to $maxLines lines (total lines is ${lines.length})]") 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 d5857e060a2c4..95fad412002e2 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 @@ -23,7 +23,6 @@ import java.util.{Map => JavaMap} import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer -import scala.language.existentials import scala.util.control.NonFatal import com.google.common.cache.{CacheBuilder, CacheLoader} @@ -32,7 +31,7 @@ import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.{ByteArrayClassLoader, ClassBodyEvaluator, InternalCompilerException, SimpleCompiler} import org.codehaus.janino.util.ClassFile -import org.apache.spark.{SparkEnv, TaskContext, TaskKilledException} +import org.apache.spark.{TaskContext, TaskKilledException} import org.apache.spark.executor.InputMetrics import org.apache.spark.internal.Logging import org.apache.spark.metrics.source.CodegenMetrics @@ -40,10 +39,10 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, MapData} +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types._ import org.apache.spark.util.{ParentClassLoader, Utils} @@ -910,12 +909,13 @@ class CodegenContext { val blocks = new ArrayBuffer[String]() val blockBuilder = new StringBuilder() var length = 0 + val splitThreshold = SQLConf.get.methodSplitThreshold for (code <- expressions) { // We can't know how many bytecode will be generated, so use the length of source code // as metric. A method should not go beyond 8K, otherwise it will not be JITted, should // also not be too small, or it will have many function calls (for wide table), see the // results in BenchmarkWideTable. - if (length > 1024) { + if (length > splitThreshold) { blocks += blockBuilder.toString() blockBuilder.clear() length = 0 @@ -1304,7 +1304,7 @@ object CodeGenerator extends Logging { throw new CompileException(msg, e.getLocation) } - (evaluator.getClazz().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) + (evaluator.getClazz().getConstructor().newInstance().asInstanceOf[GeneratedClass], maxCodeSize) } /** @@ -1351,7 +1351,11 @@ object CodeGenerator extends Logging { } }.flatten - codeSizes.max + if (codeSizes.nonEmpty) { + codeSizes.max + } else { + 0 + } } /** @@ -1371,7 +1375,7 @@ object CodeGenerator extends Logging { val startTime = System.nanoTime() val result = doCompile(code) val endTime = System.nanoTime() - def timeMs: Double = (endTime - startTime).toDouble / 1000000 + def timeMs: Double = (endTime - startTime).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") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala index d588e7f081303..838bd1c679e4d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateMutableProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp // MutableProjection is not accessible in Java @@ -35,7 +36,7 @@ object GenerateMutableProjection extends CodeGenerator[Seq[Expression], MutableP in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) def generate( expressions: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala index 9a51be6ed5aeb..b66b80ad31dc2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateOrdering.scala @@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.io.{Input, Output} 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.expressions.BindReferences.bindReferences import org.apache.spark.sql.types.StructType import org.apache.spark.util.Utils @@ -46,7 +47,7 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR in.map(ExpressionCanonicalizer.execute(_).asInstanceOf[SortOrder]) protected def bind(in: Seq[SortOrder], inputSchema: Seq[Attribute]): Seq[SortOrder] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) /** * Creates a code gen ordering for sorting this schema, in ascending order. @@ -68,62 +69,55 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR genComparisons(ctx, ordering) } + /** + * Creates the variables for ordering based on the given order. + */ + private def createOrderKeys( + ctx: CodegenContext, + row: String, + ordering: Seq[SortOrder]): Seq[ExprCode] = { + ctx.INPUT_ROW = row + // to use INPUT_ROW we must make sure currentVars is null + ctx.currentVars = null + ordering.map(_.child.genCode(ctx)) + } + /** * Generates the code for ordering based on the given order. */ def genComparisons(ctx: CodegenContext, ordering: Seq[SortOrder]): String = { val oldInputRow = ctx.INPUT_ROW val oldCurrentVars = ctx.currentVars - val inputRow = "i" - ctx.INPUT_ROW = inputRow - // to use INPUT_ROW we must make sure currentVars is null - ctx.currentVars = null - - val comparisons = ordering.map { order => - val eval = order.child.genCode(ctx) - val asc = order.isAscending - val isNullA = ctx.freshName("isNullA") - val primitiveA = ctx.freshName("primitiveA") - val isNullB = ctx.freshName("isNullB") - val primitiveB = ctx.freshName("primitiveB") + val rowAKeys = createOrderKeys(ctx, "a", ordering) + val rowBKeys = createOrderKeys(ctx, "b", ordering) + val comparisons = rowAKeys.zip(rowBKeys).zipWithIndex.map { case ((l, r), i) => + val dt = ordering(i).child.dataType + val asc = ordering(i).isAscending + val nullOrdering = ordering(i).nullOrdering + val lRetValue = nullOrdering match { + case NullsFirst => "-1" + case NullsLast => "1" + } + val rRetValue = nullOrdering match { + case NullsFirst => "1" + case NullsLast => "-1" + } s""" - ${ctx.INPUT_ROW} = a; - boolean $isNullA; - ${CodeGenerator.javaType(order.child.dataType)} $primitiveA; - { - ${eval.code} - $isNullA = ${eval.isNull}; - $primitiveA = ${eval.value}; - } - ${ctx.INPUT_ROW} = b; - boolean $isNullB; - ${CodeGenerator.javaType(order.child.dataType)} $primitiveB; - { - ${eval.code} - $isNullB = ${eval.isNull}; - $primitiveB = ${eval.value}; - } - if ($isNullA && $isNullB) { - // Nothing - } else if ($isNullA) { - return ${ - order.nullOrdering match { - case NullsFirst => "-1" - case NullsLast => "1" - }}; - } else if ($isNullB) { - return ${ - order.nullOrdering match { - case NullsFirst => "1" - case NullsLast => "-1" - }}; - } else { - int comp = ${ctx.genComp(order.child.dataType, primitiveA, primitiveB)}; - if (comp != 0) { - return ${if (asc) "comp" else "-comp"}; - } - } - """ + |${l.code} + |${r.code} + |if (${l.isNull} && ${r.isNull}) { + | // Nothing + |} else if (${l.isNull}) { + | return $lRetValue; + |} else if (${r.isNull}) { + | return $rRetValue; + |} else { + | int comp = ${ctx.genComp(dt, l.value, r.value)}; + | if (comp != 0) { + | return ${if (asc) "comp" else "-comp"}; + | } + |} + """.stripMargin } val code = ctx.splitExpressions( @@ -133,30 +127,24 @@ object GenerateOrdering extends CodeGenerator[Seq[SortOrder], Ordering[InternalR returnType = "int", makeSplitFunction = { body => s""" - InternalRow ${ctx.INPUT_ROW} = null; // Holds current row being evaluated. - $body - return 0; - """ + |$body + |return 0; + """.stripMargin }, foldFunctions = { funCalls => funCalls.zipWithIndex.map { case (funCall, i) => val comp = ctx.freshName("comp") s""" - int $comp = $funCall; - if ($comp != 0) { - return $comp; - } - """ + |int $comp = $funCall; + |if ($comp != 0) { + | return $comp; + |} + """.stripMargin }.mkString }) ctx.currentVars = oldCurrentVars ctx.INPUT_ROW = oldInputRow - // make sure INPUT_ROW is declared even if splitExpressions - // returns an inlined block - s""" - |InternalRow $inputRow = null; - |$code - """.stripMargin + code } protected def create(ordering: Seq[SortOrder]): BaseOrdering = { @@ -201,7 +189,7 @@ class LazilyGeneratedOrdering(val ordering: Seq[SortOrder]) extends Ordering[InternalRow] with KryoSerializable { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = - this(ordering.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(ordering, inputSchema)) @transient private[this] var generatedOrdering = GenerateOrdering.generate(ordering) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala index 39778661d1c48..e285398ba1958 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateSafeProjection.scala @@ -21,6 +21,7 @@ import scala.annotation.tailrec import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} @@ -41,7 +42,7 @@ object GenerateSafeProjection extends CodeGenerator[Seq[Expression], Projection] in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) private def createCodeForStruct( ctx: CodegenContext, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala index 0ecd0de8d8203..fb1d8a3c8e739 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.types._ @@ -317,7 +318,7 @@ object GenerateUnsafeProjection extends CodeGenerator[Seq[Expression], UnsafePro in.map(ExpressionCanonicalizer.execute) protected def bind(in: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = - in.map(BindReferences.bindReference(_, inputSchema)) + bindReferences(in, inputSchema) def generate( expressions: Seq[Expression], diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala index febf7b0c96c2a..070570d8f20b2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeRowJoiner.scala @@ -55,7 +55,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U def create(schema1: StructType, schema2: StructType): UnsafeRowJoiner = { val ctx = new CodegenContext - val offset = Platform.BYTE_ARRAY_OFFSET + val offset = "Platform.BYTE_ARRAY_OFFSET" val getLong = "Platform.getLong" val putLong = "Platform.putLong" @@ -92,7 +92,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U s"$getLong(obj2, offset2 + ${(i - bitset1Words) * 8})" } } - s"$putLong(buf, ${offset + i * 8}, $bits);\n" + s"$putLong(buf, $offset + ${i * 8}, $bits);\n" } val copyBitsets = ctx.splitExpressions( @@ -102,12 +102,12 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U ("java.lang.Object", "obj2") :: ("long", "offset2") :: Nil) // --------------------- copy fixed length portion from row 1 ----------------------- // - var cursor = offset + outputBitsetWords * 8 + var cursor = outputBitsetWords * 8 val copyFixedLengthRow1 = s""" |// Copy fixed length data for row1 |Platform.copyMemory( | obj1, offset1 + ${bitset1Words * 8}, - | buf, $cursor, + | buf, $offset + $cursor, | ${schema1.size * 8}); """.stripMargin cursor += schema1.size * 8 @@ -117,7 +117,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U |// Copy fixed length data for row2 |Platform.copyMemory( | obj2, offset2 + ${bitset2Words * 8}, - | buf, $cursor, + | buf, $offset + $cursor, | ${schema2.size * 8}); """.stripMargin cursor += schema2.size * 8 @@ -129,7 +129,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U |long numBytesVariableRow1 = row1.getSizeInBytes() - $numBytesBitsetAndFixedRow1; |Platform.copyMemory( | obj1, offset1 + ${(bitset1Words + schema1.size) * 8}, - | buf, $cursor, + | buf, $offset + $cursor, | numBytesVariableRow1); """.stripMargin @@ -140,7 +140,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U |long numBytesVariableRow2 = row2.getSizeInBytes() - $numBytesBitsetAndFixedRow2; |Platform.copyMemory( | obj2, offset2 + ${(bitset2Words + schema2.size) * 8}, - | buf, $cursor + numBytesVariableRow1, + | buf, $offset + $cursor + numBytesVariableRow1, | numBytesVariableRow2); """.stripMargin @@ -161,7 +161,7 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U } else { s"(${(outputBitsetWords - bitset2Words + schema1.size) * 8}L + numBytesVariableRow1)" } - val cursor = offset + outputBitsetWords * 8 + i * 8 + val cursor = outputBitsetWords * 8 + i * 8 // UnsafeRow is a little underspecified, so in what follows we'll treat UnsafeRowWriter's // output as a de-facto specification for the internal layout of data. // @@ -198,9 +198,9 @@ object GenerateUnsafeRowJoiner extends CodeGenerator[(StructType, StructType), U // Thus it is safe to perform `existingOffset != 0` checks here in the place of // more expensive null-bit checks. s""" - |existingOffset = $getLong(buf, $cursor); + |existingOffset = $getLong(buf, $offset + $cursor); |if (existingOffset != 0) { - | $putLong(buf, $cursor, existingOffset + ($shift << 32)); + | $putLong(buf, $offset + $cursor, existingOffset + ($shift << 32)); |} """.stripMargin } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala index 17d4a0dc4e884..c3b79900d308d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/javaCode.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions.codegen import java.lang.{Boolean => JBool} import scala.collection.mutable.ArrayBuffer -import scala.language.{existentials, implicitConversions} +import scala.language.implicitConversions import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.types.{BooleanType, DataType} @@ -183,7 +183,7 @@ trait Block extends TreeNode[Block] with JavaCode { def doTransform(arg: Any): AnyRef = arg match { case e: ExprValue => transform(e) case Some(value) => Some(doTransform(value)) - case seq: Traversable[_] => seq.map(doTransform) + case seq: Iterable[_] => seq.map(doTransform) case other: AnyRef => other } @@ -197,7 +197,7 @@ trait Block extends TreeNode[Block] with JavaCode { case _ => code"$this\n$other" } - override def verboseString: String = toString + override def verboseString(maxFields: Int): String = toString } object Block { @@ -224,7 +224,7 @@ object Block { } else { args.foreach { case _: ExprValue | _: Inline | _: Block => - case _: Int | _: Long | _: Float | _: Double | _: String => + case _: Boolean | _: Int | _: Long | _: Float | _: Double | _: String => case other => throw new IllegalArgumentException( s"Can not interpolate ${other.getClass.getName} into code block.") } 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 b24d7486f3454..5314821ea3a59 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 @@ -16,7 +16,8 @@ */ package org.apache.spark.sql.catalyst.expressions -import java.util.{Comparator, TimeZone} +import java.time.ZoneId +import java.util.Comparator import scala.collection.mutable import scala.reflect.ClassTag @@ -30,7 +31,7 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.Platform +import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.array.ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH import org.apache.spark.unsafe.types.{ByteArray, UTF8String} @@ -273,7 +274,7 @@ case class ArraysZip(children: Seq[Expression]) extends Expression with ExpectsI } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - if (children.length == 0) { + if (children.isEmpty) { emptyInputGenCode(ev) } else { nonEmptyInputGenCode(ctx, ev) @@ -350,7 +351,7 @@ case class MapValues(child: Expression) > SELECT _FUNC_(map(1, 'a', 2, 'b')); [{"key":1,"value":"a"},{"key":2,"value":"b"}] """, - since = "2.4.0") + since = "3.0.0") case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(MapType) @@ -452,7 +453,7 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp val z = ctx.freshName("z") val calculateHeader = "UnsafeArrayData.calculateHeaderPortionInBytes" - val baseOffset = Platform.BYTE_ARRAY_OFFSET + val baseOffset = "Platform.BYTE_ARRAY_OFFSET" val wordSize = UnsafeRow.WORD_SIZE val structSizeAsLong = s"${structSize}L" @@ -521,13 +522,18 @@ case class MapEntries(child: Expression) extends UnaryExpression with ExpectsInp case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpression { override def checkInputDataTypes(): TypeCheckResult = { - var funcName = s"function $prettyName" + val funcName = s"function $prettyName" if (children.exists(!_.dataType.isInstanceOf[MapType])) { TypeCheckResult.TypeCheckFailure( s"input to $funcName should all be of type map, but it's " + children.map(_.dataType.catalogString).mkString("[", ", ", "]")) } else { - TypeUtils.checkForSameTypeInputExpr(children.map(_.dataType), funcName) + val sameTypeCheck = TypeUtils.checkForSameTypeInputExpr(children.map(_.dataType), funcName) + if (sameTypeCheck.isFailure) { + sameTypeCheck + } else { + TypeUtils.checkForMapKeyType(dataType.keyType) + } } } @@ -541,51 +547,25 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres override def nullable: Boolean = children.exists(_.nullable) + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def eval(input: InternalRow): Any = { - val maps = children.map(_.eval(input)) + val maps = children.map(_.eval(input).asInstanceOf[MapData]) if (maps.contains(null)) { return null } - val keyArrayDatas = maps.map(_.asInstanceOf[MapData].keyArray()) - val valueArrayDatas = maps.map(_.asInstanceOf[MapData].valueArray()) - - val numElements = keyArrayDatas.foldLeft(0L)((sum, ad) => sum + ad.numElements()) - if (numElements > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { - throw new RuntimeException(s"Unsuccessful attempt to concat maps with $numElements " + - s"elements due to exceeding the map size limit " + - s"${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") - } - val finalKeyArray = new Array[AnyRef](numElements.toInt) - val finalValueArray = new Array[AnyRef](numElements.toInt) - var position = 0 - for (i <- keyArrayDatas.indices) { - val keyArray = keyArrayDatas(i).toObjectArray(dataType.keyType) - val valueArray = valueArrayDatas(i).toObjectArray(dataType.valueType) - Array.copy(keyArray, 0, finalKeyArray, position, keyArray.length) - Array.copy(valueArray, 0, finalValueArray, position, valueArray.length) - position += keyArray.length - } - new ArrayBasedMapData(new GenericArrayData(finalKeyArray), - new GenericArrayData(finalValueArray)) + for (map <- maps) { + mapBuilder.putAll(map.keyArray(), map.valueArray()) + } + mapBuilder.build() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val mapCodes = children.map(_.genCode(ctx)) - val keyType = dataType.keyType - val valueType = dataType.valueType val argsName = ctx.freshName("args") val hasNullName = ctx.freshName("hasNull") - val mapDataClass = classOf[MapData].getName - val arrayBasedMapDataClass = classOf[ArrayBasedMapData].getName - val arrayDataClass = classOf[ArrayData].getName - - val init = - s""" - |$mapDataClass[] $argsName = new $mapDataClass[${mapCodes.size}]; - |boolean ${ev.isNull}, $hasNullName = false; - |$mapDataClass ${ev.value} = null; - """.stripMargin + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) val assignments = mapCodes.zip(children.map(_.nullable)).zipWithIndex.map { case ((m, true), i) => @@ -608,10 +588,10 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres """.stripMargin } - val codes = ctx.splitExpressionsWithCurrentInputs( + val prepareMaps = ctx.splitExpressionsWithCurrentInputs( expressions = assignments, funcName = "getMapConcatInputs", - extraArguments = (s"$mapDataClass[]", argsName) :: ("boolean", hasNullName) :: Nil, + extraArguments = (s"MapData[]", argsName) :: ("boolean", hasNullName) :: Nil, returnType = "boolean", makeSplitFunction = body => s""" @@ -622,91 +602,27 @@ case class MapConcat(children: Seq[Expression]) extends ComplexTypeMergingExpres ) val idxName = ctx.freshName("idx") - val numElementsName = ctx.freshName("numElems") - val finKeysName = ctx.freshName("finalKeys") - val finValsName = ctx.freshName("finalValues") - - val keyConcat = genCodeForArrays(ctx, keyType, false) - - val valueConcat = - if (valueType.sameType(keyType) && - !(CodeGenerator.isPrimitiveType(valueType) && dataType.valueContainsNull)) { - keyConcat - } else { - genCodeForArrays(ctx, valueType, dataType.valueContainsNull) - } - - val keyArgsName = ctx.freshName("keyArgs") - val valArgsName = ctx.freshName("valArgs") - val mapMerge = s""" - |${ev.isNull} = $hasNullName; - |if (!${ev.isNull}) { - | $arrayDataClass[] $keyArgsName = new $arrayDataClass[${mapCodes.size}]; - | $arrayDataClass[] $valArgsName = new $arrayDataClass[${mapCodes.size}]; - | long $numElementsName = 0; - | for (int $idxName = 0; $idxName < $argsName.length; $idxName++) { - | $keyArgsName[$idxName] = $argsName[$idxName].keyArray(); - | $valArgsName[$idxName] = $argsName[$idxName].valueArray(); - | $numElementsName += $argsName[$idxName].numElements(); - | } - | if ($numElementsName > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | throw new RuntimeException("Unsuccessful attempt to concat maps with " + - | $numElementsName + " elements due to exceeding the map size limit " + - | "${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}."); - | } - | $arrayDataClass $finKeysName = $keyConcat($keyArgsName, - | (int) $numElementsName); - | $arrayDataClass $finValsName = $valueConcat($valArgsName, - | (int) $numElementsName); - | ${ev.value} = new $arrayBasedMapDataClass($finKeysName, $finValsName); + |for (int $idxName = 0; $idxName < $argsName.length; $idxName++) { + | $builderTerm.putAll($argsName[$idxName].keyArray(), $argsName[$idxName].valueArray()); |} + |${ev.value} = $builderTerm.build(); """.stripMargin ev.copy( code = code""" - |$init - |$codes - |$mapMerge + |MapData[] $argsName = new MapData[${mapCodes.size}]; + |boolean $hasNullName = false; + |$prepareMaps + |boolean ${ev.isNull} = $hasNullName; + |MapData ${ev.value} = null; + |if (!$hasNullName) { + | $mapMerge + |} """.stripMargin) } - private def genCodeForArrays( - ctx: CodegenContext, - elementType: DataType, - checkForNull: Boolean): String = { - val counter = ctx.freshName("counter") - val arrayData = ctx.freshName("arrayData") - val argsName = ctx.freshName("args") - val numElemName = ctx.freshName("numElements") - val y = ctx.freshName("y") - val z = ctx.freshName("z") - - val allocation = CodeGenerator.createArrayData( - arrayData, elementType, numElemName, s" $prettyName failed.") - val assignment = CodeGenerator.createArrayAssignment( - arrayData, elementType, s"$argsName[$y]", counter, z, checkForNull) - - val concat = ctx.freshName("concat") - val concatDef = - s""" - |private ArrayData $concat(ArrayData[] $argsName, int $numElemName) { - | $allocation - | int $counter = 0; - | for (int $y = 0; $y < ${children.length}; $y++) { - | for (int $z = 0; $z < $argsName[$y].numElements(); $z++) { - | $assignment - | $counter++; - | } - | } - | return $arrayData; - |} - """.stripMargin - - ctx.addNewFunction(concat, concatDef) - } - override def prettyName: String = "map_concat" } @@ -740,176 +656,50 @@ case class MapFromEntries(child: Expression) extends UnaryExpression { @transient override lazy val dataType: MapType = dataTypeDetails.get._1 override def checkInputDataTypes(): TypeCheckResult = dataTypeDetails match { - case Some(_) => TypeCheckResult.TypeCheckSuccess + case Some((mapType, _, _)) => + TypeUtils.checkForMapKeyType(mapType.keyType) case None => TypeCheckResult.TypeCheckFailure(s"'${child.sql}' is of " + s"${child.dataType.catalogString} type. $prettyName accepts only arrays of pair structs.") } + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override protected def nullSafeEval(input: Any): Any = { - val arrayData = input.asInstanceOf[ArrayData] - val numEntries = arrayData.numElements() + val entries = input.asInstanceOf[ArrayData] + val numEntries = entries.numElements() var i = 0 - if(nullEntries) { + if (nullEntries) { while (i < numEntries) { - if (arrayData.isNullAt(i)) return null + if (entries.isNullAt(i)) return null i += 1 } } - val keyArray = new Array[AnyRef](numEntries) - val valueArray = new Array[AnyRef](numEntries) + i = 0 while (i < numEntries) { - val entry = arrayData.getStruct(i, 2) - val key = entry.get(0, dataType.keyType) - if (key == null) { - throw new RuntimeException("The first field from a struct (key) can't be null.") - } - keyArray.update(i, key) - val value = entry.get(1, dataType.valueType) - valueArray.update(i, value) + mapBuilder.put(entries.getStruct(i, 2)) i += 1 } - ArrayBasedMapData(keyArray, valueArray) + mapBuilder.build() } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, c => { val numEntries = ctx.freshName("numEntries") - val isKeyPrimitive = CodeGenerator.isPrimitiveType(dataType.keyType) - val isValuePrimitive = CodeGenerator.isPrimitiveType(dataType.valueType) - val code = if (isKeyPrimitive && isValuePrimitive) { - genCodeForPrimitiveElements(ctx, c, ev.value, numEntries) - } else { - genCodeForAnyElements(ctx, c, ev.value, numEntries) - } + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) + val i = ctx.freshName("idx") ctx.nullArrayElementsSaveExec(nullEntries, ev.isNull, c) { s""" |final int $numEntries = $c.numElements(); - |$code + |for (int $i = 0; $i < $numEntries; $i++) { + | $builderTerm.put($c.getStruct($i, 2)); + |} + |${ev.value} = $builderTerm.build(); """.stripMargin } }) } - private def genCodeForAssignmentLoop( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String, - keyAssignment: (String, String) => String, - valueAssignment: (String, String) => String): String = { - val entry = ctx.freshName("entry") - val i = ctx.freshName("idx") - - val nullKeyCheck = if (dataTypeDetails.get._2) { - s""" - |if ($entry.isNullAt(0)) { - | throw new RuntimeException("The first field from a struct (key) can't be null."); - |} - """.stripMargin - } else { - "" - } - - s""" - |for (int $i = 0; $i < $numEntries; $i++) { - | InternalRow $entry = $childVariable.getStruct($i, 2); - | $nullKeyCheck - | ${keyAssignment(CodeGenerator.getValue(entry, dataType.keyType, "0"), i)} - | ${valueAssignment(entry, i)} - |} - """.stripMargin - } - - private def genCodeForPrimitiveElements( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String): String = { - val byteArraySize = ctx.freshName("byteArraySize") - val keySectionSize = ctx.freshName("keySectionSize") - val valueSectionSize = ctx.freshName("valueSectionSize") - val data = ctx.freshName("byteArray") - val unsafeMapData = ctx.freshName("unsafeMapData") - val keyArrayData = ctx.freshName("keyArrayData") - val valueArrayData = ctx.freshName("valueArrayData") - - val baseOffset = Platform.BYTE_ARRAY_OFFSET - val keySize = dataType.keyType.defaultSize - val valueSize = dataType.valueType.defaultSize - val kByteSize = s"UnsafeArrayData.calculateSizeOfUnderlyingByteArray($numEntries, $keySize)" - val vByteSize = s"UnsafeArrayData.calculateSizeOfUnderlyingByteArray($numEntries, $valueSize)" - - val keyAssignment = (key: String, idx: String) => - CodeGenerator.setArrayElement(keyArrayData, dataType.keyType, idx, key) - val valueAssignment = (entry: String, idx: String) => - CodeGenerator.createArrayAssignment( - valueArrayData, dataType.valueType, entry, idx, "1", dataType.valueContainsNull) - val assignmentLoop = genCodeForAssignmentLoop( - ctx, - childVariable, - mapData, - numEntries, - keyAssignment, - valueAssignment - ) - - s""" - |final long $keySectionSize = $kByteSize; - |final long $valueSectionSize = $vByteSize; - |final long $byteArraySize = 8 + $keySectionSize + $valueSectionSize; - |if ($byteArraySize > ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}) { - | ${genCodeForAnyElements(ctx, childVariable, mapData, numEntries)} - |} else { - | final byte[] $data = new byte[(int)$byteArraySize]; - | UnsafeMapData $unsafeMapData = new UnsafeMapData(); - | Platform.putLong($data, $baseOffset, $keySectionSize); - | Platform.putLong($data, ${baseOffset + 8}, $numEntries); - | Platform.putLong($data, ${baseOffset + 8} + $keySectionSize, $numEntries); - | $unsafeMapData.pointTo($data, $baseOffset, (int)$byteArraySize); - | ArrayData $keyArrayData = $unsafeMapData.keyArray(); - | ArrayData $valueArrayData = $unsafeMapData.valueArray(); - | $assignmentLoop - | $mapData = $unsafeMapData; - |} - """.stripMargin - } - - private def genCodeForAnyElements( - ctx: CodegenContext, - childVariable: String, - mapData: String, - numEntries: String): String = { - val keys = ctx.freshName("keys") - val values = ctx.freshName("values") - val mapDataClass = classOf[ArrayBasedMapData].getName() - - val isValuePrimitive = CodeGenerator.isPrimitiveType(dataType.valueType) - val valueAssignment = (entry: String, idx: String) => { - val value = CodeGenerator.getValue(entry, dataType.valueType, "1") - if (dataType.valueContainsNull && isValuePrimitive) { - s"$values[$idx] = $entry.isNullAt(1) ? null : (Object)$value;" - } else { - s"$values[$idx] = $value;" - } - } - val keyAssignment = (key: String, idx: String) => s"$keys[$idx] = $key;" - val assignmentLoop = genCodeForAssignmentLoop( - ctx, - childVariable, - mapData, - numEntries, - keyAssignment, - valueAssignment) - - s""" - |final Object[] $keys = new Object[$numEntries]; - |final Object[] $values = new Object[$numEntries]; - |$assignmentLoop - |$mapData = $mapDataClass.apply($keys, $values); - """.stripMargin - } - override def prettyName: String = "map_from_entries" } @@ -929,17 +719,15 @@ trait ArraySortLike extends ExpectsInputTypes { case _ @ ArrayType(s: StructType, _) => s.interpretedOrdering.asInstanceOf[Ordering[Any]] } - new Comparator[Any]() { - override def compare(o1: Any, o2: Any): Int = { - if (o1 == null && o2 == null) { - 0 - } else if (o1 == null) { - nullOrder - } else if (o2 == null) { - -nullOrder - } else { - ordering.compare(o1, o2) - } + (o1: Any, o2: Any) => { + if (o1 == null && o2 == null) { + 0 + } else if (o1 == null) { + nullOrder + } else if (o2 == null) { + -nullOrder + } else { + ordering.compare(o1, o2) } } } @@ -951,17 +739,15 @@ trait ArraySortLike extends ExpectsInputTypes { case _ @ ArrayType(s: StructType, _) => s.interpretedOrdering.asInstanceOf[Ordering[Any]] } - new Comparator[Any]() { - override def compare(o1: Any, o2: Any): Int = { - if (o1 == null && o2 == null) { - 0 - } else if (o1 == null) { - -nullOrder - } else if (o2 == null) { - nullOrder - } else { - ordering.compare(o2, o1) - } + (o1: Any, o2: Any) => { + if (o1 == null && o2 == null) { + 0 + } else if (o1 == null) { + -nullOrder + } else if (o2 == null) { + nullOrder + } else { + ordering.compare(o2, o1) } } } @@ -980,7 +766,6 @@ trait ArraySortLike extends ExpectsInputTypes { } def sortCodegen(ctx: CodegenContext, ev: ExprCode, base: String, order: String): String = { - val arrayData = classOf[ArrayData].getName val genericArrayData = classOf[GenericArrayData].getName val unsafeArrayData = classOf[UnsafeArrayData].getName val array = ctx.freshName("array") @@ -1175,7 +960,9 @@ case class ArraySort(child: Expression) extends UnaryExpression with ArraySortLi > SELECT _FUNC_(array(1, 20, null, 3)); [20,null,3,1] """, - note = "The function is non-deterministic.", + note = """ + The function is non-deterministic. + """, since = "2.4.0") case class Shuffle(child: Expression, randomSeed: Option[Long] = None) extends UnaryExpression with ExpectsInputTypes with Stateful with ExpressionWithRandomSeed { @@ -1259,7 +1046,9 @@ case class Shuffle(child: Expression, randomSeed: Option[Long] = None) [3,4,1,2] """, since = "1.5.0", - note = "Reverse logic for arrays is available since 2.4.0." + note = """ + Reverse logic for arrays is available since 2.4.0. + """ ) case class Reverse(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { @@ -2140,7 +1929,8 @@ case class ArrayPosition(left: Expression, right: Expression) b """, since = "2.4.0") -case class ElementAt(left: Expression, right: Expression) extends GetMapValueUtil { +case class ElementAt(left: Expression, right: Expression) + extends GetMapValueUtil with GetArrayItemUtil { @transient private lazy val mapKeyType = left.dataType.asInstanceOf[MapType].keyType @@ -2185,7 +1975,10 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti } } - override def nullable: Boolean = true + override def nullable: Boolean = left.dataType match { + case _: ArrayType => computeNullabilityFromArray(left, right) + case _: MapType => true + } override def nullSafeEval(value: Any, ordinal: Any): Any = doElementAt(value, ordinal) @@ -2269,7 +2062,9 @@ case class ElementAt(left: Expression, right: Expression) extends GetMapValueUti > SELECT _FUNC_(array(1, 2, 3), array(4, 5), array(6)); [1,2,3,4,5,6] """, - note = "Concat logic for arrays is available since 2.4.0.") + note = """ + Concat logic for arrays is available since 2.4.0. + """) case class Concat(children: Seq[Expression]) extends ComplexTypeMergingExpression { private def allowedTypes: Seq[AbstractDataType] = Seq(StringType, BinaryType, ArrayType) @@ -2665,10 +2460,10 @@ case class Sequence( new IntegralSequenceImpl(iType)(ct, iType.integral) case TimestampType => - new TemporalSequenceImpl[Long](LongType, 1, identity, timeZone) + new TemporalSequenceImpl[Long](LongType, 1, identity, zoneId) case DateType => - new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, timeZone) + new TemporalSequenceImpl[Int](IntegerType, MICROS_PER_DAY, _.toInt, zoneId) } override def eval(input: InternalRow): Any = { @@ -2809,7 +2604,7 @@ object Sequence { } private class TemporalSequenceImpl[T: ClassTag] - (dt: IntegralType, scale: Long, fromLong: Long => T, timeZone: TimeZone) + (dt: IntegralType, scale: Long, fromLong: Long => T, zoneId: ZoneId) (implicit num: Integral[T]) extends SequenceImpl { override val defaultStep: DefaultStep = new DefaultStep( @@ -2847,8 +2642,8 @@ object Sequence { while (t < exclusiveItem ^ stepSign < 0) { arr(i) = fromLong(t / scale) - t = timestampAddInterval(t, stepMonths, stepMicros, timeZone) i += 1 + t = timestampAddInterval(startMicros, i * stepMonths, i * stepMicros, zoneId) } // truncate array to the correct length @@ -2874,7 +2669,7 @@ object Sequence { val exclusiveItem = ctx.freshName("exclusiveItem") val t = ctx.freshName("t") val i = ctx.freshName("i") - val genTimeZone = ctx.addReferenceObj("timeZone", timeZone, classOf[TimeZone].getName) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val sequenceLengthCode = s""" @@ -2882,12 +2677,6 @@ object Sequence { |${genSequenceLengthCode(ctx, startMicros, stopMicros, intervalInMicros, arrLength)} """.stripMargin - val timestampAddIntervalCode = - s""" - |$t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( - | $t, $stepMonths, $stepMicros, $genTimeZone); - """.stripMargin - s""" |final int $stepMonths = $step.months; |final long $stepMicros = $step.microseconds; @@ -2911,8 +2700,9 @@ object Sequence { | | while ($t < $exclusiveItem ^ $stepSign < 0) { | $arr[$i] = ($elemType) ($t / ${scale}L); - | $timestampAddIntervalCode | $i += 1; + | $t = org.apache.spark.sql.catalyst.util.DateTimeUtils.timestampAddInterval( + | $startMicros, $i * $stepMonths, $i * $stepMicros, $zid); | } | | if ($arr.length > $i) { @@ -2991,7 +2781,7 @@ case class ArrayRepeat(left: Expression, right: Expression) } else { if (count.asInstanceOf[Int] > ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { throw new RuntimeException(s"Unsuccessful try to create array with $count elements " + - s"due to exceeding the array size limit ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}."); + s"due to exceeding the array size limit ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") } val element = left.eval(input) new GenericArrayData(Array.fill(count.asInstanceOf[Int])(element)) @@ -3319,29 +3109,29 @@ case class ArrayDistinct(child: Expression) (data: Array[AnyRef]) => new GenericArrayData(data.distinct.asInstanceOf[Array[Any]]) } else { (data: Array[AnyRef]) => { - var foundNullElement = false - var pos = 0 + val arrayBuffer = new scala.collection.mutable.ArrayBuffer[AnyRef] + var alreadyStoredNull = false for (i <- 0 until data.length) { - if (data(i) == null) { - if (!foundNullElement) { - foundNullElement = true - pos = pos + 1 - } - } else { + if (data(i) != null) { + var found = false var j = 0 - var done = false - while (j <= i && !done) { - if (data(j) != null && ordering.equiv(data(j), data(i))) { - done = true - } - j = j + 1 + while (!found && j < arrayBuffer.size) { + val va = arrayBuffer(j) + found = (va != null) && ordering.equiv(va, data(i)) + j += 1 } - if (i == j - 1) { - pos = pos + 1 + if (!found) { + arrayBuffer += data(i) + } + } else { + // De-duplicate the null values. + if (!alreadyStoredNull) { + arrayBuffer += data(i) + alreadyStoredNull = true } } } - new GenericArrayData(data.slice(0, pos)) + new GenericArrayData(arrayBuffer) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala index 0361372b6b732..319a7fc87e59a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeCreator.scala @@ -24,8 +24,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String /** @@ -62,7 +60,7 @@ case class CreateArray(children: Seq[Expression]) extends Expression { override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val et = dataType.elementType val (allocation, assigns, arrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, et, children, false, "createArray") + GenArrayData.genCodeToCreateArrayData(ctx, et, children, "createArray") ev.copy( code = code"${allocation}${assigns}", value = JavaCode.variable(arrayData, dataType), @@ -79,7 +77,6 @@ private [sql] object GenArrayData { * @param ctx a [[CodegenContext]] * @param elementType data type of underlying array elements * @param elementsExpr concatenated set of [[Expression]] for each element of an underlying array - * @param isMapKey if true, throw an exception when the element is null * @param functionName string to include in the error message * @return (array allocation, concatenated assignments to each array elements, arrayData name) */ @@ -87,7 +84,6 @@ private [sql] object GenArrayData { ctx: CodegenContext, elementType: DataType, elementsExpr: Seq[Expression], - isMapKey: Boolean, functionName: String): (String, String, String) = { val arrayDataName = ctx.freshName("arrayData") val numElements = s"${elementsExpr.length}L" @@ -103,15 +99,9 @@ private [sql] object GenArrayData { val assignment = if (!expr.nullable) { setArrayElement } else { - val isNullAssignment = if (!isMapKey) { - s"$arrayDataName.setNullAt($i);" - } else { - "throw new RuntimeException(\"Cannot use null as map key!\");" - } - s""" |if (${eval.isNull}) { - | $isNullAssignment + | $arrayDataName.setNullAt($i); |} else { | $setArrayElement |} @@ -161,11 +151,11 @@ case class CreateMap(children: Seq[Expression]) extends Expression { "The given values of function map should all be the same type, but they are " + values.map(_.dataType.catalogString).mkString("[", ", ", "]")) } else { - TypeCheckResult.TypeCheckSuccess + TypeUtils.checkForMapKeyType(dataType.keyType) } } - override def dataType: DataType = { + override lazy val dataType: MapType = { MapType( keyType = TypeCoercion.findCommonTypeDifferentOnlyInNullFlags(keys.map(_.dataType)) .getOrElse(StringType), @@ -176,32 +166,33 @@ case class CreateMap(children: Seq[Expression]) extends Expression { override def nullable: Boolean = false + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def eval(input: InternalRow): Any = { - val keyArray = keys.map(_.eval(input)).toArray - if (keyArray.contains(null)) { - throw new RuntimeException("Cannot use null as map key!") + var i = 0 + while (i < keys.length) { + mapBuilder.put(keys(i).eval(input), values(i).eval(input)) + i += 1 } - val valueArray = values.map(_.eval(input)).toArray - new ArrayBasedMapData(new GenericArrayData(keyArray), new GenericArrayData(valueArray)) + mapBuilder.build() } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val mapClass = classOf[ArrayBasedMapData].getName val MapType(keyDt, valueDt, _) = dataType val (allocationKeyData, assignKeys, keyArrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, keyDt, keys, true, "createMap") + GenArrayData.genCodeToCreateArrayData(ctx, keyDt, keys, "createMap") val (allocationValueData, assignValues, valueArrayData) = - GenArrayData.genCodeToCreateArrayData(ctx, valueDt, values, false, "createMap") + GenArrayData.genCodeToCreateArrayData(ctx, valueDt, values, "createMap") + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) val code = code""" - final boolean ${ev.isNull} = false; $allocationKeyData $assignKeys $allocationValueData $assignValues - final MapData ${ev.value} = new $mapClass($keyArrayData, $valueArrayData); + final MapData ${ev.value} = $builderTerm.from($keyArrayData, $valueArrayData); """ - ev.copy(code = code) + ev.copy(code = code, isNull = FalseLiteral) } override def prettyName: String = "map" @@ -224,53 +215,35 @@ case class MapFromArrays(left: Expression, right: Expression) override def inputTypes: Seq[AbstractDataType] = Seq(ArrayType, ArrayType) - override def dataType: DataType = { + override def checkInputDataTypes(): TypeCheckResult = { + val defaultCheck = super.checkInputDataTypes() + if (defaultCheck.isFailure) { + defaultCheck + } else { + val keyType = left.dataType.asInstanceOf[ArrayType].elementType + TypeUtils.checkForMapKeyType(keyType) + } + } + + override def dataType: MapType = { MapType( keyType = left.dataType.asInstanceOf[ArrayType].elementType, valueType = right.dataType.asInstanceOf[ArrayType].elementType, valueContainsNull = right.dataType.asInstanceOf[ArrayType].containsNull) } + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) + override def nullSafeEval(keyArray: Any, valueArray: Any): Any = { val keyArrayData = keyArray.asInstanceOf[ArrayData] val valueArrayData = valueArray.asInstanceOf[ArrayData] - if (keyArrayData.numElements != valueArrayData.numElements) { - throw new RuntimeException("The given two arrays should have the same length") - } - val leftArrayType = left.dataType.asInstanceOf[ArrayType] - if (leftArrayType.containsNull) { - var i = 0 - while (i < keyArrayData.numElements) { - if (keyArrayData.isNullAt(i)) { - throw new RuntimeException("Cannot use null as map key!") - } - i += 1 - } - } - new ArrayBasedMapData(keyArrayData.copy(), valueArrayData.copy()) + mapBuilder.from(keyArrayData.copy(), valueArrayData.copy()) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (keyArrayData, valueArrayData) => { - val arrayBasedMapData = classOf[ArrayBasedMapData].getName - val leftArrayType = left.dataType.asInstanceOf[ArrayType] - val keyArrayElemNullCheck = if (!leftArrayType.containsNull) "" else { - val i = ctx.freshName("i") - s""" - |for (int $i = 0; $i < $keyArrayData.numElements(); $i++) { - | if ($keyArrayData.isNullAt($i)) { - | throw new RuntimeException("Cannot use null as map key!"); - | } - |} - """.stripMargin - } - s""" - |if ($keyArrayData.numElements() != $valueArrayData.numElements()) { - | throw new RuntimeException("The given two arrays should have the same length"); - |} - |$keyArrayElemNullCheck - |${ev.value} = new $arrayBasedMapData($keyArrayData.copy(), $valueArrayData.copy()); - """.stripMargin + val builderTerm = ctx.addReferenceObj("mapBuilder", mapBuilder) + s"${ev.value} = $builderTerm.from($keyArrayData.copy(), $valueArrayData.copy());" }) } @@ -315,6 +288,7 @@ object CreateStruct extends FunctionBuilder { "", "", "", + "", "") ("struct", (info, this)) } @@ -444,7 +418,7 @@ case class CreateNamedStructUnsafe(children: Seq[Expression]) extends CreateName */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(text[, pairDelim[, keyValueDelim]]) - Creates a map after splitting the text into key/value pairs using delimiters. Default delimiters are ',' for `pairDelim` and ':' for `keyValueDelim`.", + usage = "_FUNC_(text[, pairDelim[, keyValueDelim]]) - Creates a map after splitting the text into key/value pairs using delimiters. Default delimiters are ',' for `pairDelim` and ':' for `keyValueDelim`. Both `pairDelim` and `keyValueDelim` are treated as regular expressions.", examples = """ Examples: > SELECT _FUNC_('a:1,b:2,c:3', ',', ':'); @@ -478,28 +452,25 @@ case class StringToMap(text: Expression, pairDelim: Expression, keyValueDelim: E } } + private lazy val mapBuilder = new ArrayBasedMapBuilder(StringType, StringType) + override def nullSafeEval( inputString: Any, stringDelimiter: Any, keyValueDelimiter: Any): Any = { val keyValues = inputString.asInstanceOf[UTF8String].split(stringDelimiter.asInstanceOf[UTF8String], -1) - - val iterator = new Iterator[(UTF8String, UTF8String)] { - var index = 0 - val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] - - override def hasNext: Boolean = { - keyValues.length > index - } - - override def next(): (UTF8String, UTF8String) = { - val keyValueArray = keyValues(index).split(keyValueDelimiterUTF8String, 2) - index += 1 - (keyValueArray(0), if (keyValueArray.length < 2) null else keyValueArray(1)) - } + val keyValueDelimiterUTF8String = keyValueDelimiter.asInstanceOf[UTF8String] + + var i = 0 + while (i < keyValues.length) { + val keyValueArray = keyValues(i).split(keyValueDelimiterUTF8String, 2) + val key = keyValueArray(0) + val value = if (keyValueArray.length < 2) null else keyValueArray(1) + mapBuilder.put(key, value) + i += 1 } - ArrayBasedMapData(iterator, keyValues.size, identity, identity) + mapBuilder.build() } override def prettyName: String = "str_to_map" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala index 8994eeff92c7f..e9d60ed3a481f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala @@ -221,7 +221,8 @@ case class GetArrayStructFields( * We need to do type checking here as `ordinal` expression maybe unresolved. */ case class GetArrayItem(child: Expression, ordinal: Expression) - extends BinaryExpression with ExpectsInputTypes with ExtractValue with NullIntolerant { + extends BinaryExpression with GetArrayItemUtil with ExpectsInputTypes with ExtractValue + with NullIntolerant { // We have done type checking for child in `ExtractValue`, so only need to check the `ordinal`. override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType, IntegralType) @@ -231,10 +232,7 @@ case class GetArrayItem(child: Expression, ordinal: Expression) override def left: Expression = child override def right: Expression = ordinal - - /** `Null` is returned for invalid ordinals. */ - override def nullable: Boolean = true - + override def nullable: Boolean = computeNullabilityFromArray(left, right) override def dataType: DataType = child.dataType.asInstanceOf[ArrayType].elementType protected override def nullSafeEval(value: Any, ordinal: Any): Any = { @@ -268,10 +266,34 @@ case class GetArrayItem(child: Expression, ordinal: Expression) } /** - * Common base class for [[GetMapValue]] and [[ElementAt]]. + * Common trait for [[GetArrayItem]] and [[ElementAt]]. */ +trait GetArrayItemUtil { + + /** `Null` is returned for invalid ordinals. */ + protected def computeNullabilityFromArray(child: Expression, ordinal: Expression): Boolean = { + if (ordinal.foldable && !ordinal.nullable) { + val intOrdinal = ordinal.eval().asInstanceOf[Number].intValue() + child match { + case CreateArray(ar) if intOrdinal < ar.length => + ar(intOrdinal).nullable + case GetArrayStructFields(CreateArray(elements), field, _, _, _) + if intOrdinal < elements.length => + elements(intOrdinal).nullable || field.nullable + case _ => + true + } + } else { + true + } + } +} + +/** + * Common trait for [[GetMapValue]] and [[ElementAt]]. + */ +trait GetMapValueUtil extends BinaryExpression with ImplicitCastInputTypes { -abstract class GetMapValueUtil extends BinaryExpression with ImplicitCastInputTypes { // todo: current search is O(n), improve it. def getValueEval(value: Any, ordinal: Any, keyType: DataType, ordering: Ordering[Any]): Any = { val map = value.asInstanceOf[MapData] @@ -367,9 +389,14 @@ case class GetMapValue(child: Expression, key: Expression) override def left: Expression = child override def right: Expression = key - /** `Null` is returned for invalid ordinals. */ + /** + * `Null` is returned for invalid ordinals. + * + * TODO: We could make nullability more precise in foldable cases (e.g., literal input). + * But, since the key search is O(n), it takes much time to compute nullability. + * If we find efficient key searches, revisit this. + */ override def nullable: Boolean = true - override def dataType: DataType = child.dataType.asInstanceOf[MapType].valueType // todo: current search is O(n), improve it. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala index bed581a61b2dc..f92c196250f30 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/conditionalExpressions.scala @@ -148,8 +148,12 @@ case class CaseWhen( s"but the ${index + 1}th when expression's type is ${branches(index)._1}") } } else { + val branchesStr = branches.map(_._2.dataType).map(dt => s"WHEN ... THEN ${dt.catalogString}") + .mkString(" ") + val elseStr = elseValue.map(expr => s" ELSE ${expr.dataType.catalogString}").getOrElse("") TypeCheckResult.TypeCheckFailure( - "THEN and ELSE expressions should all be same type or coercible to a common type") + "THEN and ELSE expressions should all be same type or coercible to a common type," + + s" got CASE $branchesStr$elseStr END") } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala index 2917b0b8c9c53..5bfae7b77e096 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/constraintExpressions.scala @@ -21,15 +21,21 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode, FalseLiteral} import org.apache.spark.sql.types.DataType -case class KnownNotNull(child: Expression) extends UnaryExpression { - override def nullable: Boolean = false +trait TaggingExpression extends UnaryExpression { + override def nullable: Boolean = child.nullable override def dataType: DataType = child.dataType + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = child.genCode(ctx) + + override def eval(input: InternalRow): Any = child.eval(input) +} + +case class KnownNotNull(child: Expression) extends TaggingExpression { + override def nullable: Boolean = false + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { child.genCode(ctx).copy(isNull = FalseLiteral) } - - override def eval(input: InternalRow): Any = { - child.eval(input) - } } + +case class KnownFloatingPointNormalized(child: Expression) extends TaggingExpression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala new file mode 100644 index 0000000000000..65b10f36373d1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/csvExpressions.scala @@ -0,0 +1,254 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.io.CharArrayWriter + +import com.univocity.parsers.csv.CsvParser + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.catalyst.csv._ +import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +/** + * Converts a CSV input string to a [[StructType]] with the specified schema. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(csvStr, schema[, options]) - Returns a struct value with the given `csvStr` and `schema`.", + examples = """ + Examples: + > SELECT _FUNC_('1, 0.8', 'a INT, b DOUBLE'); + {"a":1, "b":0.8} + > SELECT _FUNC_('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) + {"time":2015-08-26 00:00:00.0} + """, + since = "3.0.0") +// scalastyle:on line.size.limit +case class CsvToStructs( + schema: StructType, + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends UnaryExpression + with TimeZoneAwareExpression + with CodegenFallback + with ExpectsInputTypes + with NullIntolerant { + + override def nullable: Boolean = child.nullable + + // The CSV input data might be missing certain fields. We force the nullability + // of the user-provided schema to avoid data corruptions. + val nullableSchema: StructType = schema.asNullable + + // Used in `FunctionRegistry` + def this(child: Expression, schema: Expression, options: Map[String, String]) = + this( + schema = ExprUtils.evalSchemaExpr(schema), + options = options, + child = child, + timeZoneId = None) + + def this(child: Expression, schema: Expression) = this(child, schema, Map.empty[String, String]) + + def this(child: Expression, schema: Expression, options: Expression) = + this( + schema = ExprUtils.evalSchemaExpr(schema), + options = ExprUtils.convertToMapData(options), + child = child, + timeZoneId = None) + + // This converts parsed rows to the desired output by the given schema. + @transient + lazy val converter = (rows: Iterator[InternalRow]) => { + if (rows.hasNext) { + val result = rows.next() + // CSV's parser produces one record only. + assert(!rows.hasNext) + result + } else { + throw new IllegalArgumentException("Expected one row from CSV parser.") + } + } + + val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + + @transient lazy val parser = { + val parsedOptions = new CSVOptions( + options, + columnPruning = true, + defaultTimeZoneId = timeZoneId.get, + defaultColumnNameOfCorruptRecord = nameOfCorruptRecord) + val mode = parsedOptions.parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw new AnalysisException(s"from_csv() doesn't support the ${mode.name} mode. " + + s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}.") + } + ExprUtils.verifyColumnNameOfCorruptRecord( + nullableSchema, + parsedOptions.columnNameOfCorruptRecord) + + val actualSchema = + StructType(nullableSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val rawParser = new UnivocityParser(actualSchema, actualSchema, parsedOptions) + new FailureSafeParser[String]( + input => Seq(rawParser.parse(input)), + mode, + nullableSchema, + parsedOptions.columnNameOfCorruptRecord) + } + + override def dataType: DataType = nullableSchema + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = { + copy(timeZoneId = Option(timeZoneId)) + } + + override def nullSafeEval(input: Any): Any = { + val csv = input.asInstanceOf[UTF8String].toString + converter(parser.parse(csv)) + } + + override def inputTypes: Seq[AbstractDataType] = StringType :: Nil + + override def prettyName: String = "from_csv" +} + +/** + * A function infers schema of CSV string. + */ +@ExpressionDescription( + usage = "_FUNC_(csv[, options]) - Returns schema in the DDL format of CSV string.", + examples = """ + Examples: + > SELECT _FUNC_('1,abc'); + struct<_c0:int,_c1:string> + """, + since = "3.0.0") +case class SchemaOfCsv( + child: Expression, + options: Map[String, String]) + extends UnaryExpression with CodegenFallback { + + def this(child: Expression) = this(child, Map.empty[String, String]) + + def this(child: Expression, options: Expression) = this( + child = child, + options = ExprUtils.convertToMapData(options)) + + override def dataType: DataType = StringType + + override def nullable: Boolean = false + + @transient + private lazy val csv = child.eval().asInstanceOf[UTF8String] + + override def checkInputDataTypes(): TypeCheckResult = child match { + case Literal(s, StringType) if s != null => super.checkInputDataTypes() + case _ => TypeCheckResult.TypeCheckFailure( + s"The input csv should be a string literal and not null; however, got ${child.sql}.") + } + + override def eval(v: InternalRow): Any = { + val parsedOptions = new CSVOptions(options, true, "UTC") + val parser = new CsvParser(parsedOptions.asParserSettings) + val row = parser.parseLine(csv.toString) + assert(row != null, "Parsed CSV record should not be null.") + + val header = row.zipWithIndex.map { case (_, index) => s"_c$index" } + val startType: Array[DataType] = Array.fill[DataType](header.length)(NullType) + val inferSchema = new CSVInferSchema(parsedOptions) + val fieldTypes = inferSchema.inferRowType(startType, row) + val st = StructType(inferSchema.toStructFields(fieldTypes, header)) + UTF8String.fromString(st.catalogString) + } + + override def prettyName: String = "schema_of_csv" +} + +/** + * Converts a [[StructType]] to a CSV output string. + */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(expr[, options]) - Returns a CSV string with a given struct value", + examples = """ + Examples: + > SELECT _FUNC_(named_struct('a', 1, 'b', 2)); + 1,2 + > SELECT _FUNC_(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); + "26/08/2015" + """, + since = "3.0.0") +// scalastyle:on line.size.limit +case class StructsToCsv( + options: Map[String, String], + child: Expression, + timeZoneId: Option[String] = None) + extends UnaryExpression with TimeZoneAwareExpression with CodegenFallback with ExpectsInputTypes { + override def nullable: Boolean = true + + def this(options: Map[String, String], child: Expression) = this(options, child, None) + + // Used in `FunctionRegistry` + def this(child: Expression) = this(Map.empty, child, None) + + def this(child: Expression, options: Expression) = + this( + options = ExprUtils.convertToMapData(options), + child = child, + timeZoneId = None) + + @transient + lazy val writer = new CharArrayWriter() + + @transient + lazy val inputSchema: StructType = child.dataType match { + case st: StructType => st + case other => + throw new IllegalArgumentException(s"Unsupported input type ${other.catalogString}") + } + + @transient + lazy val gen = new UnivocityGenerator( + inputSchema, writer, new CSVOptions(options, columnPruning = true, timeZoneId.get)) + + // This converts rows to the CSV output according to the given schema. + @transient + lazy val converter: Any => UTF8String = { + (row: Any) => UTF8String.fromString(gen.writeToString(row.asInstanceOf[InternalRow])) + } + + override def dataType: DataType = StringType + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) + + override def nullSafeEval(value: Any): Any = converter(value) + + override def inputTypes: Seq[AbstractDataType] = StructType :: Nil + + override def prettyName: String = "to_csv" +} 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 45e17ae235a94..53329fd10a9c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/datetimeExpressions.scala @@ -18,17 +18,21 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import java.text.DateFormat -import java.util.{Calendar, TimeZone} +import java.time.{Instant, LocalDate, ZoneId} +import java.time.temporal.IsoFields +import java.util.{Locale, TimeZone} import scala.util.control.NonFatal import org.apache.commons.lang3.StringEscapeUtils +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow 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 +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} @@ -47,6 +51,7 @@ trait TimeZoneAwareExpression extends Expression { def withTimeZone(timeZoneId: String): TimeZoneAwareExpression @transient lazy val timeZone: TimeZone = DateTimeUtils.getTimeZone(timeZoneId.get) + @transient lazy val zoneId: ZoneId = DateTimeUtils.getZoneId(timeZoneId.get) } /** @@ -72,7 +77,7 @@ case class CurrentDate(timeZoneId: Option[String] = None) copy(timeZoneId = Option(timeZoneId)) override def eval(input: InternalRow): Any = { - DateTimeUtils.millisToDays(System.currentTimeMillis(), timeZone) + localDateToDays(LocalDate.now(zoneId)) } override def prettyName: String = "current_date" @@ -94,7 +99,7 @@ case class CurrentTimestamp() extends LeafExpression with CodegenFallback { override def dataType: DataType = TimestampType override def eval(input: InternalRow): Any = { - System.currentTimeMillis() * 1000L + instantToMicros(Instant.now()) } override def prettyName: String = "current_timestamp" @@ -430,20 +435,14 @@ case class DayOfMonth(child: Expression) extends UnaryExpression with ImplicitCa case class DayOfWeek(child: Expression) extends DayWeek { override protected def nullSafeEval(date: Any): Any = { - cal.setTimeInMillis(date.asInstanceOf[Int] * 1000L * 3600L * 24L) - cal.get(Calendar.DAY_OF_WEEK) + val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) + localDate.getDayOfWeek.plus(1).getValue } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, time => { - val cal = classOf[Calendar].getName - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val c = "calDayOfWeek" - ctx.addImmutableStateIfNotExists(cal, c, - v => s"""$v = $cal.getInstance($dtu.getTimeZone("UTC"));""") + nullSafeCodeGen(ctx, ev, days => { s""" - $c.setTimeInMillis($time * 1000L * 3600L * 24L); - ${ev.value} = $c.get($cal.DAY_OF_WEEK); + ${ev.value} = java.time.LocalDate.ofEpochDay($days).getDayOfWeek().plus(1).getValue(); """ }) } @@ -462,20 +461,14 @@ case class DayOfWeek(child: Expression) extends DayWeek { case class WeekDay(child: Expression) extends DayWeek { override protected def nullSafeEval(date: Any): Any = { - cal.setTimeInMillis(date.asInstanceOf[Int] * 1000L * 3600L * 24L) - (cal.get(Calendar.DAY_OF_WEEK) + 5 ) % 7 + val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) + localDate.getDayOfWeek.ordinal() } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, time => { - val cal = classOf[Calendar].getName - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val c = "calWeekDay" - ctx.addImmutableStateIfNotExists(cal, c, - v => s"""$v = $cal.getInstance($dtu.getTimeZone("UTC"));""") + nullSafeCodeGen(ctx, ev, days => { s""" - $c.setTimeInMillis($time * 1000L * 3600L * 24L); - ${ev.value} = ($c.get($cal.DAY_OF_WEEK) + 5) % 7; + ${ev.value} = java.time.LocalDate.ofEpochDay($days).getDayOfWeek().ordinal(); """ }) } @@ -486,10 +479,6 @@ abstract class DayWeek extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[AbstractDataType] = Seq(DateType) override def dataType: DataType = IntegerType - - @transient protected lazy val cal: Calendar = { - Calendar.getInstance(DateTimeUtils.getTimeZone("UTC")) - } } // scalastyle:off line.size.limit @@ -508,32 +497,16 @@ case class WeekOfYear(child: Expression) extends UnaryExpression with ImplicitCa override def dataType: DataType = IntegerType - @transient private lazy val c = { - val c = Calendar.getInstance(DateTimeUtils.getTimeZone("UTC")) - c.setFirstDayOfWeek(Calendar.MONDAY) - c.setMinimalDaysInFirstWeek(4) - c - } - override protected def nullSafeEval(date: Any): Any = { - c.setTimeInMillis(date.asInstanceOf[Int] * 1000L * 3600L * 24L) - c.get(Calendar.WEEK_OF_YEAR) + val localDate = LocalDate.ofEpochDay(date.asInstanceOf[Int]) + localDate.get(IsoFields.WEEK_OF_WEEK_BASED_YEAR) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - nullSafeCodeGen(ctx, ev, time => { - val cal = classOf[Calendar].getName - val c = "calWeekOfYear" - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - ctx.addImmutableStateIfNotExists(cal, c, v => - s""" - |$v = $cal.getInstance($dtu.getTimeZone("UTC")); - |$v.setFirstDayOfWeek($cal.MONDAY); - |$v.setMinimalDaysInFirstWeek(4); - """.stripMargin) + nullSafeCodeGen(ctx, ev, days => { s""" - |$c.setTimeInMillis($time * 1000L * 3600L * 24L); - |${ev.value} = $c.get($cal.WEEK_OF_YEAR); + |${ev.value} = java.time.LocalDate.ofEpochDay($days).get( + | java.time.temporal.IsoFields.WEEK_OF_WEEK_BASED_YEAR); """.stripMargin }) } @@ -562,16 +535,17 @@ case class DateFormatClass(left: Expression, right: Expression, timeZoneId: Opti copy(timeZoneId = Option(timeZoneId)) override protected def nullSafeEval(timestamp: Any, format: Any): Any = { - val df = DateTimeUtils.newDateFormat(format.toString, timeZone) - UTF8String.fromString(df.format(new java.util.Date(timestamp.asInstanceOf[Long] / 1000))) + val df = TimestampFormatter(format.toString, zoneId) + UTF8String.fromString(df.format(timestamp.asInstanceOf[Long])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") - val tz = ctx.addReferenceObj("timeZone", timeZone) + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val locale = ctx.addReferenceObj("locale", Locale.US) defineCodeGen(ctx, ev, (timestamp, format) => { - s"""UTF8String.fromString($dtu.newDateFormat($format.toString(), $tz) - .format(new java.util.Date($timestamp / 1000)))""" + s"""UTF8String.fromString($tf$$.MODULE$$.apply($format.toString(), $zid, $locale) + .format($timestamp))""" }) } @@ -612,9 +586,8 @@ case class ToUnixTimestamp( } /** - * Converts time string with given pattern. - * (see [http://docs.oracle.com/javase/tutorial/i18n/format/simpleDateFormat.html]) - * to Unix time stamp (in seconds), returns null if fail. + * 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]. * 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. @@ -653,9 +626,13 @@ case class UnixTimestamp(timeExp: Expression, format: Expression, timeZoneId: Op override def prettyName: String = "unix_timestamp" } -abstract class UnixTime +abstract class ToTimestamp extends BinaryExpression with TimeZoneAwareExpression with ExpectsInputTypes { + // The result of the conversion to timestamp is microseconds divided by this factor. + // For example if the factor is 1000000, the result of the expression is in seconds. + protected def downScaleFactor: Long + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, DateType, TimestampType), StringType) @@ -663,9 +640,9 @@ abstract class UnixTime override def nullable: Boolean = true private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: DateFormat = + private lazy val formatter: TimestampFormatter = try { - DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + TimestampFormatter(constFormat.toString, zoneId) } catch { case NonFatal(_) => null } @@ -677,16 +654,16 @@ abstract class UnixTime } else { left.dataType match { case DateType => - DateTimeUtils.daysToMillis(t.asInstanceOf[Int], timeZone) / 1000L + epochDaysToMicros(t.asInstanceOf[Int], zoneId) / downScaleFactor case TimestampType => - t.asInstanceOf[Long] / 1000000L + t.asInstanceOf[Long] / downScaleFactor case StringType if right.foldable => if (constFormat == null || formatter == null) { null } else { try { formatter.parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L + t.asInstanceOf[UTF8String].toString) / downScaleFactor } catch { case NonFatal(_) => null } @@ -698,8 +675,8 @@ abstract class UnixTime } else { val formatString = f.asInstanceOf[UTF8String].toString try { - DateTimeUtils.newDateFormat(formatString, timeZone).parse( - t.asInstanceOf[UTF8String].toString).getTime / 1000L + TimestampFormatter(formatString, zoneId).parse( + t.asInstanceOf[UTF8String].toString) / downScaleFactor } catch { case NonFatal(_) => null } @@ -712,7 +689,7 @@ abstract class UnixTime val javaType = CodeGenerator.javaType(dataType) left.dataType match { case StringType if right.foldable => - val df = classOf[DateFormat].getName + val df = classOf[TimestampFormatter].getName if (formatter == null) { ExprCode.forNullValue(dataType) } else { @@ -724,24 +701,35 @@ abstract class UnixTime $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { try { - ${ev.value} = $formatterName.parse(${eval1.value}.toString()).getTime() / 1000L; + ${ev.value} = $formatterName.parse(${eval1.value}.toString()) / $downScaleFactor; + } catch (java.lang.IllegalArgumentException e) { + ${ev.isNull} = true; } catch (java.text.ParseException e) { ${ev.isNull} = true; + } catch (java.time.format.DateTimeParseException e) { + ${ev.isNull} = true; + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; } }""") } case StringType => - val tz = ctx.addReferenceObj("timeZone", timeZone) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val locale = ctx.addReferenceObj("locale", Locale.US) + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (string, format) => { s""" try { - ${ev.value} = $dtu.newDateFormat($format.toString(), $tz) - .parse($string.toString()).getTime() / 1000L; + ${ev.value} = $tf$$.MODULE$$.apply($format.toString(), $zid, $locale) + .parse($string.toString()) / $downScaleFactor; } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } catch (java.text.ParseException e) { ${ev.isNull} = true; + } catch (java.time.format.DateTimeParseException e) { + ${ev.isNull} = true; + } catch (java.time.DateTimeException e) { + ${ev.isNull} = true; } """ }) @@ -752,10 +740,10 @@ abstract class UnixTime boolean ${ev.isNull} = ${eval1.isNull}; $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = ${eval1.value} / 1000000L; + ${ev.value} = ${eval1.value} / $downScaleFactor; }""") case DateType => - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") val eval1 = left.genCode(ctx) ev.copy(code = code""" @@ -763,12 +751,16 @@ abstract class UnixTime boolean ${ev.isNull} = ${eval1.isNull}; $javaType ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { - ${ev.value} = $dtu.daysToMillis(${eval1.value}, $tz) / 1000L; + ${ev.value} = $dtu.epochDaysToMicros(${eval1.value}, $zid) / $downScaleFactor; }""") } } } +abstract class UnixTime extends ToTimestamp { + override val downScaleFactor: Long = MICROS_PER_SECOND +} + /** * 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 given @@ -806,9 +798,9 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ copy(timeZoneId = Option(timeZoneId)) private lazy val constFormat: UTF8String = right.eval().asInstanceOf[UTF8String] - private lazy val formatter: DateFormat = + private lazy val formatter: TimestampFormatter = try { - DateTimeUtils.newDateFormat(constFormat.toString, timeZone) + TimestampFormatter(constFormat.toString, zoneId) } catch { case NonFatal(_) => null } @@ -823,8 +815,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(formatter.format( - new java.util.Date(time.asInstanceOf[Long] * 1000L))) + UTF8String.fromString(formatter.format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null } @@ -835,8 +826,8 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ null } else { try { - UTF8String.fromString(DateTimeUtils.newDateFormat(f.toString, timeZone) - .format(new java.util.Date(time.asInstanceOf[Long] * 1000L))) + UTF8String.fromString(TimestampFormatter(f.toString, zoneId) + .format(time.asInstanceOf[Long] * MICROS_PER_SECOND)) } catch { case NonFatal(_) => null } @@ -846,7 +837,7 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val df = classOf[DateFormat].getName + val df = classOf[TimestampFormatter].getName if (format.foldable) { if (formatter == null) { ExprCode.forNullValue(StringType) @@ -859,21 +850,21 @@ case class FromUnixTime(sec: Expression, format: Expression, timeZoneId: Option[ ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${ev.isNull}) { try { - ${ev.value} = UTF8String.fromString($formatterName.format( - new java.util.Date(${t.value} * 1000L))); + ${ev.value} = UTF8String.fromString($formatterName.format(${t.value} * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; } }""") } } else { - val tz = ctx.addReferenceObj("timeZone", timeZone) - val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) + val locale = ctx.addReferenceObj("locale", Locale.US) + val tf = TimestampFormatter.getClass.getName.stripSuffix("$") nullSafeCodeGen(ctx, ev, (seconds, f) => { s""" try { - ${ev.value} = UTF8String.fromString($dtu.newDateFormat($f.toString(), $tz).format( - new java.util.Date($seconds * 1000L))); + ${ev.value} = UTF8String.fromString($tf$$.MODULE$$.apply($f.toString(), $zid, $locale). + format($seconds * 1000000L)); } catch (java.lang.IllegalArgumentException e) { ${ev.isNull} = true; }""" @@ -1005,14 +996,14 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], itvl.months, itvl.microseconds, timeZone) + start.asInstanceOf[Long], itvl.months, itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, $i.months, $i.microseconds, $zid)""" }) } } @@ -1039,11 +1030,19 @@ case class TimeAdd(start: Expression, interval: Expression, timeZoneId: Option[S > SELECT from_utc_timestamp('2016-08-31', 'Asia/Seoul'); 2016-08-31 09:00:00 """, - since = "1.5.0") + since = "1.5.0", + deprecated = """ + Deprecated since 3.0.0. See SPARK-25496. + """) // scalastyle:on line.size.limit case class FromUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { + if (!SQLConf.get.utcTimestampFuncEnabled) { + throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0." + + s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.") + } + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override def dataType: DataType = TimestampType override def prettyName: String = "from_utc_timestamp" @@ -1112,14 +1111,14 @@ case class TimeSub(start: Expression, interval: Expression, timeZoneId: Option[S override def nullSafeEval(start: Any, interval: Any): Any = { val itvl = interval.asInstanceOf[CalendarInterval] DateTimeUtils.timestampAddInterval( - start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, timeZone) + start.asInstanceOf[Long], 0 - itvl.months, 0 - itvl.microseconds, zoneId) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val tz = ctx.addReferenceObj("timeZone", timeZone) + val zid = ctx.addReferenceObj("zoneId", zoneId, classOf[ZoneId].getName) val dtu = DateTimeUtils.getClass.getName.stripSuffix("$") defineCodeGen(ctx, ev, (sd, i) => { - s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $tz)""" + s"""$dtu.timestampAddInterval($sd, 0 - $i.months, 0 - $i.microseconds, $zid)""" }) } } @@ -1245,11 +1244,19 @@ case class MonthsBetween( > SELECT _FUNC_('2016-08-31', 'Asia/Seoul'); 2016-08-30 15:00:00 """, - since = "1.5.0") + since = "1.5.0", + deprecated = """ + Deprecated since 3.0.0. See SPARK-25496. + """) // scalastyle:on line.size.limit case class ToUTCTimestamp(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { + if (!SQLConf.get.utcTimestampFuncEnabled) { + throw new AnalysisException(s"The $prettyName function has been disabled since Spark 3.0. " + + s"Set ${SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key} to true to enable this function.") + } + override def inputTypes: Seq[AbstractDataType] = Seq(TimestampType, StringType) override def dataType: DataType = TimestampType override def prettyName: String = "to_utc_timestamp" @@ -1358,7 +1365,7 @@ case class ParseToTimestamp(left: Expression, format: Option[Expression], child: extends RuntimeReplaceable { def this(left: Expression, format: Expression) = { - this(left, Option(format), Cast(UnixTimestamp(left, format), TimestampType)) + this(left, Option(format), GetTimestamp(left, format)) } def this(left: Expression) = this(left, None, Cast(left, TimestampType)) @@ -1582,3 +1589,19 @@ case class DateDiff(endDate: Expression, startDate: Expression) defineCodeGen(ctx, ev, (end, start) => s"$end - $start") } } + +/** + * Gets timestamps from strings using given pattern. + */ +private case class GetTimestamp( + left: Expression, + right: Expression, + timeZoneId: Option[String] = None) + extends ToTimestamp { + + override val downScaleFactor = 1 + override def dataType: DataType = TimestampType + + override def withTimeZone(timeZoneId: String): TimeZoneAwareExpression = + copy(timeZoneId = Option(timeZoneId)) +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala index 04de83343be71..b5b712cda8ea3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/decimalExpressions.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, EmptyBlock, ExprCode} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -46,19 +47,38 @@ case class UnscaledValue(child: Expression) extends UnaryExpression { */ case class MakeDecimal(child: Expression, precision: Int, scale: Int) extends UnaryExpression { + private val nullOnOverflow = SQLConf.get.decimalOperationsNullOnOverflow + override def dataType: DataType = DecimalType(precision, scale) - override def nullable: Boolean = true + override def nullable: Boolean = child.nullable || nullOnOverflow override def toString: String = s"MakeDecimal($child,$precision,$scale)" - protected override def nullSafeEval(input: Any): Any = - Decimal(input.asInstanceOf[Long], precision, scale) + protected override def nullSafeEval(input: Any): Any = { + val longInput = input.asInstanceOf[Long] + val result = new Decimal() + if (nullOnOverflow) { + result.setOrNull(longInput, precision, scale) + } else { + result.set(longInput, precision, scale) + } + } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, eval => { + val setMethod = if (nullOnOverflow) { + "setOrNull" + } else { + "set" + } + val setNull = if (nullable) { + s"${ev.isNull} = ${ev.value} == null;" + } else { + "" + } s""" - ${ev.value} = (new Decimal()).setOrNull($eval, $precision, $scale); - ${ev.isNull} = ${ev.value} == null; - """ + |${ev.value} = (new Decimal()).$setMethod($eval, $precision, $scale); + |$setNull + |""".stripMargin }) } } @@ -81,30 +101,34 @@ case class PromotePrecision(child: Expression) extends UnaryExpression { /** * Rounds the decimal to given scale and check whether the decimal can fit in provided precision - * or not, returns null if not. + * or not. If not, if `nullOnOverflow` is `true`, it returns `null`; otherwise an + * `ArithmeticException` is thrown. */ -case class CheckOverflow(child: Expression, dataType: DecimalType) extends UnaryExpression { +case class CheckOverflow( + child: Expression, + dataType: DecimalType, + nullOnOverflow: Boolean) extends UnaryExpression { override def nullable: Boolean = true override def nullSafeEval(input: Any): Any = - input.asInstanceOf[Decimal].toPrecision(dataType.precision, dataType.scale) + input.asInstanceOf[Decimal].toPrecision( + dataType.precision, + dataType.scale, + Decimal.ROUND_HALF_UP, + nullOnOverflow) override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, eval => { - val tmp = ctx.freshName("tmp") s""" - | Decimal $tmp = $eval.clone(); - | if ($tmp.changePrecision(${dataType.precision}, ${dataType.scale})) { - | ${ev.value} = $tmp; - | } else { - | ${ev.isNull} = true; - | } + |${ev.value} = $eval.toPrecision( + | ${dataType.precision}, ${dataType.scale}, Decimal.ROUND_HALF_UP(), $nullOnOverflow); + |${ev.isNull} = ${ev.value} == null; """.stripMargin }) } - override def toString: String = s"CheckOverflow($child, $dataType)" + override def toString: String = s"CheckOverflow($child, $dataType, $nullOnOverflow)" override def sql: String = child.sql } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala index d6e67b9ac3d10..82a7d9825e30a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.analysis.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, MapData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ /** @@ -101,7 +102,7 @@ case class UserDefinedGenerator( inputRow = new InterpretedProjection(children) convertToScala = { val inputSchema = StructType(children.map { e => - StructField(e.simpleString, e.dataType, nullable = true) + StructField(e.simpleString(SQLConf.get.maxToStringFields), e.dataType, nullable = true) }) CatalystTypeConverters.createToScalaConverter(inputSchema) }.asInstanceOf[InternalRow => Row] @@ -126,14 +127,16 @@ case class UserDefinedGenerator( * 3 NULL * }}} */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(n, expr1, ..., exprk) - Separates `expr1`, ..., `exprk` into `n` rows.", + usage = "_FUNC_(n, expr1, ..., exprk) - Separates `expr1`, ..., `exprk` into `n` rows. Uses column names col0, col1, etc. by default unless specified otherwise.", examples = """ Examples: > SELECT _FUNC_(2, 1, 2, 3); 1 2 3 NULL """) +// scalastyle:on line.size.limit case class Stack(children: Seq[Expression]) extends Generator { private lazy val numRows = children.head.eval().asInstanceOf[Int] @@ -258,7 +261,7 @@ case class GeneratorOuter(child: Generator) extends UnaryExpression with Generat throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") final override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = - throw new UnsupportedOperationException(s"Cannot evaluate expression: $this") + throw new UnsupportedOperationException(s"Cannot generate code for expression: $this") override def elementSchema: StructType = child.elementSchema @@ -351,7 +354,7 @@ abstract class ExplodeBase extends UnaryExpression with CollectionGenerator with */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows, or the elements of map `expr` into multiple rows and columns.", + usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows, or the elements of map `expr` into multiple rows and columns. Unless specified otherwise, uses the default column name `col` for elements of the array or `key` and `value` for the elements of the map.", examples = """ Examples: > SELECT _FUNC_(array(10, 20)); @@ -374,7 +377,7 @@ case class Explode(child: Expression) extends ExplodeBase { */ // scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows with positions, or the elements of map `expr` into multiple rows and columns with positions.", + usage = "_FUNC_(expr) - Separates the elements of array `expr` into multiple rows with positions, or the elements of map `expr` into multiple rows and columns with positions. Unless specified otherwise, uses the column name `pos` for position, `col` for elements of the array or `key` and `value` for elements of the map.", examples = """ Examples: > SELECT _FUNC_(array(10,20)); @@ -389,14 +392,16 @@ case class PosExplode(child: Expression) extends ExplodeBase { /** * Explodes an array of structs into a table. */ +// scalastyle:off line.size.limit @ExpressionDescription( - usage = "_FUNC_(expr) - Explodes an array of structs into a table.", + usage = "_FUNC_(expr) - Explodes an array of structs into a table. Uses column names col1, col2, etc. by default unless specified otherwise.", examples = """ Examples: > SELECT _FUNC_(array(struct(1, 'a'), struct(2, 'b'))); 1 a 2 b """) +// scalastyle:on line.size.limit case class Inline(child: Expression) extends UnaryExpression with CollectionGenerator { override val inline: Boolean = true override val position: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala index 3be761c8676c9..221b97bdc7856 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/grouping.scala @@ -38,16 +38,69 @@ trait GroupingSet extends Expression with CodegenFallback { override def eval(input: InternalRow): Any = throw new UnsupportedOperationException } +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_([col1[, col2 ..]]) - create a multi-dimensional cube using the specified columns + so that we can run aggregation on them. + """, + examples = """ + Examples: + > SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY _FUNC_(name, age); + NULL 2 1 + NULL NULL 2 + Alice 2 1 + Bob 5 1 + NULL 5 1 + Bob NULL 1 + Alice NULL 1 + """, + since = "2.0.0") +// scalastyle:on line.size.limit case class Cube(groupByExprs: Seq[Expression]) extends GroupingSet {} +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_([col1[, col2 ..]]) - create a multi-dimensional rollup using the specified columns + so that we can run aggregation on them. + """, + examples = """ + Examples: + > SELECT name, age, count(*) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY _FUNC_(name, age); + NULL NULL 2 + Alice 2 1 + Bob 5 1 + Bob NULL 1 + Alice NULL 1 + """, + since = "2.0.0") +// scalastyle:on line.size.limit case class Rollup(groupByExprs: Seq[Expression]) extends GroupingSet {} /** * Indicates whether a specified column expression in a GROUP BY list is aggregated or not. * GROUPING returns 1 for aggregated or 0 for not aggregated in the result set. */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_(col) - indicates whether a specified column in a GROUP BY is aggregated or + not, returns 1 for aggregated or 0 for not aggregated in the result set.", + """, + examples = """ + Examples: + > SELECT name, _FUNC_(name), sum(age) FROM VALUES (2, 'Alice'), (5, 'Bob') people(age, name) GROUP BY cube(name); + Alice 0 2 + NULL 1 7 + Bob 0 5 + """, + since = "2.0.0") +// scalastyle:on line.size.limit case class Grouping(child: Expression) extends Expression with Unevaluable { - override def references: AttributeSet = AttributeSet(VirtualColumn.groupingIdAttribute :: Nil) + @transient + override lazy val references: AttributeSet = + AttributeSet(VirtualColumn.groupingIdAttribute :: Nil) override def children: Seq[Expression] = child :: Nil override def dataType: DataType = ByteType override def nullable: Boolean = false @@ -58,8 +111,33 @@ case class Grouping(child: Expression) extends Expression with Unevaluable { * * If groupByExprs is empty, it means all grouping expressions in GroupingSets. */ +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = """ + _FUNC_([col1[, col2 ..]]) - returns the level of grouping, equals to + `(grouping(c1) << (n-1)) + (grouping(c2) << (n-2)) + ... + grouping(cn)` + """, + examples = """ + Examples: + > SELECT name, _FUNC_(), sum(age), avg(height) FROM VALUES (2, 'Alice', 165), (5, 'Bob', 180) people(age, name, height) GROUP BY cube(name, height); + NULL 2 2 165.0 + Alice 0 2 165.0 + NULL 2 5 180.0 + NULL 3 7 172.5 + Bob 0 5 180.0 + Bob 1 5 180.0 + Alice 1 2 165.0 + """, + note = """ + Input columns should match with grouping columns exactly, or empty (means all the grouping + columns). + """, + since = "2.0.0") +// scalastyle:on line.size.limit case class GroupingID(groupByExprs: Seq[Expression]) extends Expression with Unevaluable { - override def references: AttributeSet = AttributeSet(VirtualColumn.groupingIdAttribute :: Nil) + @transient + override lazy val references: AttributeSet = + AttributeSet(VirtualColumn.groupingIdAttribute :: Nil) override def children: Seq[Expression] = groupByExprs override def dataType: DataType = IntegerType override def nullable: Boolean = false diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala index 742a4f87a9c04..2aa1e6cc518cb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/hash.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.math.{BigDecimal, RoundingMode} import java.security.{MessageDigest, NoSuchAlgorithmException} +import java.util.concurrent.TimeUnit._ import java.util.zip.CRC32 import scala.annotation.tailrec @@ -30,6 +31,7 @@ import org.apache.spark.sql.catalyst.analysis.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, MapData} +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.hash.Murmur3_x86_32 @@ -61,7 +63,7 @@ case class Md5(child: Expression) extends UnaryExpression with ImplicitCastInput override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, c => - s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.md5Hex($c))") + s"UTF8String.fromString(${classOf[DigestUtils].getName}.md5Hex($c))") } } @@ -118,7 +120,7 @@ case class Sha2(left: Expression, right: Expression) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val digestUtils = "org.apache.commons.codec.digest.DigestUtils" + val digestUtils = classOf[DigestUtils].getName nullSafeCodeGen(ctx, ev, (eval1, eval2) => { s""" if ($eval2 == 224) { @@ -168,7 +170,7 @@ case class Sha1(child: Expression) extends UnaryExpression with ImplicitCastInpu override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { defineCodeGen(ctx, ev, c => - s"UTF8String.fromString(org.apache.commons.codec.digest.DigestUtils.sha1Hex($c))" + s"UTF8String.fromString(${classOf[DigestUtils].getName}.sha1Hex($c))" ) } } @@ -592,12 +594,19 @@ object Murmur3HashFunction extends InterpretedHashFunction { /** * A xxHash64 64-bit hash expression. */ +@ExpressionDescription( + usage = "_FUNC_(expr1, expr2, ...) - Returns a 64-bit hash value of the arguments.", + examples = """ + Examples: + > SELECT _FUNC_('Spark', array(123), 2); + 5602566077635097486 + """) case class XxHash64(children: Seq[Expression], seed: Long) extends HashExpression[Long] { def this(arguments: Seq[Expression]) = this(arguments, 42L) override def dataType: DataType = LongType - override def prettyName: String = "xxHash" + override def prettyName: String = "xxhash64" override protected def hasherClassName: String = classOf[XXH64].getName @@ -863,8 +872,8 @@ object HiveHashFunction extends InterpretedHashFunction { * Mimics TimestampWritable.hashCode() in Hive */ def hashTimestamp(timestamp: Long): Long = { - val timestampInSeconds = timestamp / 1000000 - val nanoSecondsPortion = (timestamp % 1000000) * 1000 + val timestampInSeconds = MICROSECONDS.toSeconds(timestamp) + val nanoSecondsPortion = (timestamp % MICROS_PER_SECOND) * NANOS_PER_MICROS var result = timestampInSeconds result <<= 30 // the nanosecond part fits in 30 bits diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala index b07d9466ba0d1..b326e1c4c6af4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/higherOrderFunctions.scala @@ -22,12 +22,35 @@ import java.util.concurrent.atomic.AtomicReference import scala.collection.mutable import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, TypeCoercion, UnresolvedAttribute, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.array.ByteArrayMethods +/** + * A placeholder of lambda variables to prevent unexpected resolution of [[LambdaFunction]]. + */ +case class UnresolvedNamedLambdaVariable(nameParts: Seq[String]) + extends LeafExpression with NamedExpression with Unevaluable { + + override def name: String = + nameParts.map(n => if (n.contains(".")) s"`$n`" else n).mkString(".") + + override def exprId: ExprId = throw new UnresolvedException(this, "exprId") + override def dataType: DataType = throw new UnresolvedException(this, "dataType") + override def nullable: Boolean = throw new UnresolvedException(this, "nullable") + override def qualifier: Seq[String] = throw new UnresolvedException(this, "qualifier") + override def toAttribute: Attribute = throw new UnresolvedException(this, "toAttribute") + override def newInstance(): NamedExpression = throw new UnresolvedException(this, "newInstance") + override lazy val resolved = false + + override def toString: String = s"lambda '$name" + + override def sql: String = name +} + /** * A named lambda variable. */ @@ -54,7 +77,9 @@ case class NamedLambdaVariable( override def toString: String = s"lambda $name#${exprId.id}$typeSuffix" - override def simpleString: String = s"lambda $name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Int): String = { + s"lambda $name#${exprId.id}: ${dataType.simpleString(maxFields)}" + } } /** @@ -79,7 +104,7 @@ case class LambdaFunction( object LambdaFunction { val identity: LambdaFunction = { - val id = UnresolvedAttribute.quoted("id") + val id = UnresolvedNamedLambdaVariable(Seq("id")) LambdaFunction(id, Seq(id)) } } @@ -264,13 +289,13 @@ case class ArrayTransform( * Filters entries in a map using the provided function. */ @ExpressionDescription( -usage = "_FUNC_(expr, func) - Filters entries in a map using the function.", -examples = """ + usage = "_FUNC_(expr, func) - Filters entries in a map using the function.", + examples = """ Examples: > SELECT _FUNC_(map(1, 0, 2, 2, 3, -1), (k, v) -> k > v); {1:0,3:-1} """, -since = "2.4.0") + since = "3.0.0") case class MapFilter( argument: Expression, function: Expression) @@ -364,6 +389,10 @@ case class ArrayFilter( Examples: > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 0); true + > SELECT _FUNC_(array(1, 2, 3), x -> x % 2 == 10); + false + > SELECT _FUNC_(array(1, null, 3), x -> x % 2 == 0); + NULL """, since = "2.4.0") case class ArrayExists( @@ -371,6 +400,16 @@ case class ArrayExists( function: Expression) extends ArrayBasedSimpleHigherOrderFunction with CodegenFallback { + private val followThreeValuedLogic = + SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC) + + override def nullable: Boolean = + if (followThreeValuedLogic) { + super.nullable || function.nullable + } else { + super.nullable + } + override def dataType: DataType = BooleanType override def functionType: AbstractDataType = BooleanType @@ -386,15 +425,25 @@ case class ArrayExists( val arr = argumentValue.asInstanceOf[ArrayData] val f = functionForEval var exists = false + var foundNull = false var i = 0 while (i < arr.numElements && !exists) { elementVar.value.set(arr.get(i, elementVar.dataType)) - if (f.eval(inputRow).asInstanceOf[Boolean]) { + val ret = f.eval(inputRow) + if (ret == null) { + foundNull = true + } else if (ret.asInstanceOf[Boolean]) { exists = true } i += 1 } - exists + if (exists) { + true + } else if (followThreeValuedLogic && foundNull) { + null + } else { + false + } } override def prettyName: String = "exists" @@ -504,7 +553,7 @@ case class ArrayAggregate( > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); {2:1,4:2,6:3} """, - since = "2.4.0") + since = "3.0.0") case class TransformKeys( argument: Expression, function: Expression) @@ -512,7 +561,11 @@ case class TransformKeys( @transient lazy val MapType(keyType, valueType, valueContainsNull) = argument.dataType - override def dataType: DataType = MapType(function.dataType, valueType, valueContainsNull) + override def dataType: MapType = MapType(function.dataType, valueType, valueContainsNull) + + override def checkInputDataTypes(): TypeCheckResult = { + TypeUtils.checkForMapKeyType(function.dataType) + } override def bind(f: (Expression, Seq[(DataType, Boolean)]) => LambdaFunction): TransformKeys = { copy(function = f(function, (keyType, false) :: (valueType, valueContainsNull) :: Nil)) @@ -521,6 +574,7 @@ case class TransformKeys( @transient lazy val LambdaFunction( _, (keyVar: NamedLambdaVariable) :: (valueVar: NamedLambdaVariable) :: Nil, _) = function + private lazy val mapBuilder = new ArrayBasedMapBuilder(dataType.keyType, dataType.valueType) override def nullSafeEval(inputRow: InternalRow, argumentValue: Any): Any = { val map = argumentValue.asInstanceOf[MapData] @@ -530,13 +584,10 @@ case class TransformKeys( keyVar.value.set(map.keyArray().get(i, keyVar.dataType)) valueVar.value.set(map.valueArray().get(i, valueVar.dataType)) val result = functionForEval.eval(inputRow) - if (result == null) { - throw new RuntimeException("Cannot use null as map key!") - } resultKeys.update(i, result) i += 1 } - new ArrayBasedMapData(resultKeys, map.valueArray()) + mapBuilder.from(resultKeys, map.valueArray()) } override def prettyName: String = "transform_keys" @@ -554,7 +605,7 @@ case class TransformKeys( > SELECT _FUNC_(map_from_arrays(array(1, 2, 3), array(1, 2, 3)), (k, v) -> k + v); {1:2,2:4,3:6} """, - since = "2.4.0") + since = "3.0.0") case class TransformValues( argument: Expression, function: Expression) @@ -605,7 +656,7 @@ case class TransformValues( > SELECT _FUNC_(map(1, 'a', 2, 'b'), map(1, 'x', 2, 'y'), (k, v1, v2) -> concat(v1, v2)); {1:"ax",2:"by"} """, - since = "2.4.0") + since = "3.0.0") case class MapZipWith(left: Expression, right: Expression, function: Expression) extends HigherOrderFunction with CodegenFallback { 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 f5297dde10ed6..655e44e4e4919 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/jsonExpressions.scala @@ -23,12 +23,10 @@ import scala.util.parsing.combinator.RegexParsers import com.fasterxml.jackson.core._ -import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.catalyst.json._ -import org.apache.spark.sql.catalyst.json.JsonInferSchema.inferField import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -529,7 +527,7 @@ case class JsonToStructs( // Used in `FunctionRegistry` def this(child: Expression, schema: Expression, options: Map[String, String]) = this( - schema = JsonExprUtils.evalSchemaExpr(schema), + schema = ExprUtils.evalTypeExpr(schema), options = options, child = child, timeZoneId = None) @@ -538,8 +536,8 @@ case class JsonToStructs( def this(child: Expression, schema: Expression, options: Expression) = this( - schema = JsonExprUtils.evalSchemaExpr(schema), - options = JsonExprUtils.convertToMapData(options), + schema = ExprUtils.evalTypeExpr(schema), + options = ExprUtils.convertToMapData(options), child = child, timeZoneId = None) @@ -554,18 +552,38 @@ case class JsonToStructs( @transient lazy val converter = nullableSchema match { case _: StructType => - (rows: Seq[InternalRow]) => if (rows.length == 1) rows.head else null + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next() else null case _: ArrayType => - (rows: Seq[InternalRow]) => rows.head.getArray(0) + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getArray(0) else null case _: MapType => - (rows: Seq[InternalRow]) => rows.head.getMap(0) + (rows: Iterator[InternalRow]) => if (rows.hasNext) rows.next().getMap(0) else null } - @transient - lazy val parser = - new JacksonParser( - nullableSchema, - new JSONOptions(options + ("mode" -> FailFastMode.name), timeZoneId.get)) + val nameOfCorruptRecord = SQLConf.get.getConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD) + @transient lazy val parser = { + val parsedOptions = new JSONOptions(options, timeZoneId.get, nameOfCorruptRecord) + val mode = parsedOptions.parseMode + if (mode != PermissiveMode && mode != FailFastMode) { + throw new IllegalArgumentException(s"from_json() doesn't support the ${mode.name} mode. " + + s"Acceptable modes are ${PermissiveMode.name} and ${FailFastMode.name}.") + } + val (parserSchema, actualSchema) = nullableSchema match { + case s: StructType => + ExprUtils.verifyColumnNameOfCorruptRecord(s, parsedOptions.columnNameOfCorruptRecord) + (s, StructType(s.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord))) + case other => + (StructType(StructField("value", other) :: Nil), other) + } + + val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = false) + val createParser = CreateJacksonParser.utf8String _ + + new FailureSafeParser[UTF8String]( + input => rawParser.parse(input, createParser, identity[UTF8String]), + mode, + parserSchema, + parsedOptions.columnNameOfCorruptRecord) + } override def dataType: DataType = nullableSchema @@ -573,35 +591,7 @@ case class JsonToStructs( copy(timeZoneId = Option(timeZoneId)) override def nullSafeEval(json: Any): Any = { - // When input is, - // - `null`: `null`. - // - invalid json: `null`. - // - empty string: `null`. - // - // When the schema is array, - // - json array: `Array(Row(...), ...)` - // - json object: `Array(Row(...))` - // - empty json array: `Array()`. - // - empty json object: `Array(Row(null))`. - // - // When the schema is a struct, - // - json object/array with single element: `Row(...)` - // - json array with multiple elements: `null` - // - empty json array: `null`. - // - empty json object: `Row(null)`. - - // We need `null` if the input string is an empty string. `JacksonParser` can - // deal with this but produces `Nil`. - if (json.toString.trim.isEmpty) return null - - try { - converter(parser.parse( - json.asInstanceOf[UTF8String], - CreateJacksonParser.utf8String, - identity[UTF8String])) - } catch { - case _: BadRecordException => null - } + converter(parser.parse(json.asInstanceOf[UTF8String])) } override def inputTypes: Seq[AbstractDataType] = StringType :: Nil @@ -610,6 +600,8 @@ case class JsonToStructs( case _: MapType => "entries" case _ => super.sql } + + override def prettyName: String = "from_json" } /** @@ -650,7 +642,7 @@ case class StructsToJson( def this(child: Expression) = this(Map.empty, child, None) def this(child: Expression, options: Expression) = this( - options = JsonExprUtils.convertToMapData(options), + options = ExprUtils.convertToMapData(options), child = child, timeZoneId = None) @@ -730,6 +722,8 @@ case class StructsToJson( override def nullSafeEval(value: Any): Any = converter(value) override def inputTypes: Seq[AbstractDataType] = TypeCollection(ArrayType, StructType) :: Nil + + override def prettyName: String = "to_json" } /** @@ -748,13 +742,17 @@ case class StructsToJson( case class SchemaOfJson( child: Expression, options: Map[String, String]) - extends UnaryExpression with String2StringExpression with CodegenFallback { + extends UnaryExpression with CodegenFallback { def this(child: Expression) = this(child, Map.empty[String, String]) def this(child: Expression, options: Expression) = this( child = child, - options = JsonExprUtils.convertToMapData(options)) + options = ExprUtils.convertToMapData(options)) + + override def dataType: DataType = StringType + + override def nullable: Boolean = false @transient private lazy val jsonOptions = new JSONOptions(options, "UTC") @@ -766,39 +764,26 @@ case class SchemaOfJson( factory } - override def convert(v: UTF8String): UTF8String = { - val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, v)) { parser => - parser.nextToken() - inferField(parser, jsonOptions) - } + @transient + private lazy val jsonInferSchema = new JsonInferSchema(jsonOptions) - UTF8String.fromString(dt.catalogString) + @transient + private lazy val json = child.eval().asInstanceOf[UTF8String] + + override def checkInputDataTypes(): TypeCheckResult = child match { + case Literal(s, StringType) if s != null => super.checkInputDataTypes() + case _ => TypeCheckResult.TypeCheckFailure( + s"The input json should be a string literal and not null; however, got ${child.sql}.") } -} -object JsonExprUtils { + override def eval(v: InternalRow): Any = { + val dt = Utils.tryWithResource(CreateJacksonParser.utf8String(jsonFactory, json)) { parser => + parser.nextToken() + jsonInferSchema.inferField(parser) + } - def evalSchemaExpr(exp: Expression): DataType = exp match { - case Literal(s, StringType) => DataType.fromDDL(s.toString) - case e @ SchemaOfJson(_: Literal, _) => - val ddlSchema = e.eval().asInstanceOf[UTF8String] - DataType.fromDDL(ddlSchema.toString) - case e => throw new AnalysisException( - "Schema should be specified in DDL format as a string literal" + - s" or output of the schema_of_json function instead of ${e.sql}") + UTF8String.fromString(dt.catalogString) } - def convertToMapData(exp: Expression): Map[String, String] = exp match { - case m: CreateMap - if m.dataType.acceptsType(MapType(StringType, StringType, valueContainsNull = false)) => - val arrayMap = m.eval().asInstanceOf[ArrayBasedMapData] - ArrayBasedMapData.toScalaMap(arrayMap).map { case (key, value) => - key.toString -> value.toString - } - case m: CreateMap => - throw new AnalysisException( - s"A type of keys and values in map() must be string, but got ${m.dataType.catalogString}") - case _ => - throw new AnalysisException("Must use a map() function for options") - } + override def prettyName: String = "schema_of_json" } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala index 2bcbb92f1a469..9cef3ecadc543 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -27,6 +27,7 @@ import java.lang.{Short => JavaShort} import java.math.{BigDecimal => JavaBigDecimal} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} +import java.time.{Instant, LocalDate} import java.util import java.util.Objects import javax.xml.bind.DatatypeConverter @@ -40,9 +41,12 @@ import org.json4s.JsonAST._ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils.instantToMicros +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types._ +import org.apache.spark.util.Utils object Literal { val TrueLiteral: Literal = Literal(true, BooleanType) @@ -63,9 +67,12 @@ object Literal { case d: JavaBigDecimal => Literal(Decimal(d), DecimalType(Math.max(d.precision, d.scale), d.scale())) case d: Decimal => Literal(d, DecimalType(Math.max(d.precision, d.scale), d.scale)) + case i: Instant => Literal(instantToMicros(i), TimestampType) case t: Timestamp => Literal(DateTimeUtils.fromJavaTimestamp(t), TimestampType) + case ld: LocalDate => Literal(ld.toEpochDay.toInt, DateType) case d: Date => Literal(DateTimeUtils.fromJavaDate(d), DateType) case a: Array[Byte] => Literal(a, BinaryType) + case a: collection.mutable.WrappedArray[_] => apply(a.array) case a: Array[_] => val elementType = componentTypeToDataType(a.getClass.getComponentType()) val dataType = ArrayType(elementType) @@ -94,7 +101,9 @@ object Literal { case JavaBoolean.TYPE => BooleanType // java classes + case _ if clz == classOf[LocalDate] => DateType case _ if clz == classOf[Date] => DateType + case _ if clz == classOf[Instant] => TimestampType case _ if clz == classOf[Timestamp] => TimestampType case _ if clz == classOf[JavaBigDecimal] => DecimalType.SYSTEM_DEFAULT case _ if clz == classOf[Array[Byte]] => BinaryType @@ -124,40 +133,6 @@ object Literal { def fromObject(obj: Any, objType: DataType): Literal = new Literal(obj, objType) def fromObject(obj: Any): Literal = new Literal(obj, ObjectType(obj.getClass)) - def fromJSON(json: JValue): Literal = { - val dataType = DataType.parseDataType(json \ "dataType") - json \ "value" match { - case JNull => Literal.create(null, dataType) - case JString(str) => fromString(str, dataType) - case other => sys.error(s"$other is not a valid Literal json value") - } - } - - /** - * Constructs a Literal from a String - */ - def fromString(str: String, dataType: DataType): Literal = { - val value = dataType match { - case BooleanType => str.toBoolean - case ByteType => str.toByte - case ShortType => str.toShort - case IntegerType => str.toInt - case LongType => str.toLong - case FloatType => str.toFloat - case DoubleType => str.toDouble - case StringType => UTF8String.fromString(str) - case DateType => java.sql.Date.valueOf(str) - case TimestampType => java.sql.Timestamp.valueOf(str) - case CalendarIntervalType => CalendarInterval.fromString(str) - case t: DecimalType => - val d = Decimal(str) - assert(d.changePrecision(t.precision, t.scale)) - d - case _ => null - } - Literal.create(value, dataType) - } - def create(v: Any, dataType: DataType): Literal = { Literal(CatalystTypeConverters.convertToCatalyst(v), dataType) } @@ -196,6 +171,47 @@ object Literal { case other => throw new RuntimeException(s"no default for type $dataType") } + + private[expressions] def validateLiteralValue(value: Any, dataType: DataType): Unit = { + def doValidate(v: Any, dataType: DataType): Boolean = dataType match { + case _ if v == null => true + case BooleanType => v.isInstanceOf[Boolean] + case ByteType => v.isInstanceOf[Byte] + case ShortType => v.isInstanceOf[Short] + case IntegerType | DateType => v.isInstanceOf[Int] + case LongType | TimestampType => v.isInstanceOf[Long] + case FloatType => v.isInstanceOf[Float] + case DoubleType => v.isInstanceOf[Double] + case _: DecimalType => v.isInstanceOf[Decimal] + case CalendarIntervalType => v.isInstanceOf[CalendarInterval] + case BinaryType => v.isInstanceOf[Array[Byte]] + case StringType => v.isInstanceOf[UTF8String] + case st: StructType => + v.isInstanceOf[InternalRow] && { + val row = v.asInstanceOf[InternalRow] + st.fields.map(_.dataType).zipWithIndex.forall { + case (dt, i) => doValidate(row.get(i, dt), dt) + } + } + case at: ArrayType => + v.isInstanceOf[ArrayData] && { + val ar = v.asInstanceOf[ArrayData] + ar.numElements() == 0 || doValidate(ar.get(0, at.elementType), at.elementType) + } + case mt: MapType => + v.isInstanceOf[MapData] && { + val map = v.asInstanceOf[MapData] + doValidate(map.keyArray(), ArrayType(mt.keyType)) && + doValidate(map.valueArray(), ArrayType(mt.valueType)) + } + case ObjectType(cls) => cls.isInstance(v) + case udt: UserDefinedType[_] => doValidate(v, udt.sqlType) + case _ => false + } + require(doValidate(value, dataType), + s"Literal must have a corresponding value to ${dataType.catalogString}, " + + s"but class ${Utils.getSimpleName(value.getClass)} found.") + } } /** @@ -240,6 +256,8 @@ object DecimalLiteral { */ case class Literal (value: Any, dataType: DataType) extends LeafExpression { + Literal.validateLiteralValue(value, dataType) + override def foldable: Boolean = true override def nullable: Boolean = value == null @@ -353,8 +371,11 @@ case class Literal (value: Any, dataType: DataType) extends LeafExpression { case _ => v + "D" } case (v: Decimal, t: DecimalType) => v + "BD" - case (v: Int, DateType) => s"DATE '${DateTimeUtils.toJavaDate(v)}'" - case (v: Long, TimestampType) => s"TIMESTAMP('${DateTimeUtils.toJavaTimestamp(v)}')" + case (v: Int, DateType) => s"DATE '${DateFormatter().format(v)}'" + case (v: Long, TimestampType) => + val formatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + s"TIMESTAMP('${formatter.format(v)}')" case (v: Array[Byte], BinaryType) => s"X'${DatatypeConverter.printHexBinary(v)}'" case _ => value.toString } 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 c2e1720259b53..e873f8ed1a21c 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 @@ -287,6 +287,29 @@ case class Cos(child: Expression) extends UnaryMathExpression(math.cos, "COS") """) case class Cosh(child: Expression) extends UnaryMathExpression(math.cosh, "COSH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic cosine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(1); + 0.0 + > SELECT _FUNC_(0); + NaN + """, + since = "3.0.0") +case class Acosh(child: Expression) + extends UnaryMathExpression((x: Double) => math.log(x + math.sqrt(x * x - 1.0)), "ACOSH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c - 1.0))") + } +} + /** * Convert a num from one base to another * @@ -557,6 +580,31 @@ case class Sin(child: Expression) extends UnaryMathExpression(math.sin, "SIN") """) case class Sinh(child: Expression) extends UnaryMathExpression(math.sinh, "SINH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic sine of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + """, + since = "3.0.0") +case class Asinh(child: Expression) + extends UnaryMathExpression((x: Double) => x match { + case Double.NegativeInfinity => Double.NegativeInfinity + case _ => math.log(x + math.sqrt(x * x + 1.0)) }, "ASINH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => + s"$c == Double.NEGATIVE_INFINITY ? Double.NEGATIVE_INFINITY : " + + s"java.lang.Math.log($c + java.lang.Math.sqrt($c * $c + 1.0))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Returns the square root of `expr`.", examples = """ @@ -617,6 +665,29 @@ case class Cot(child: Expression) """) case class Tanh(child: Expression) extends UnaryMathExpression(math.tanh, "TANH") +@ExpressionDescription( + usage = """ + _FUNC_(expr) - Returns inverse hyperbolic tangent of `expr`. + """, + arguments = """ + Arguments: + * expr - hyperbolic angle + """, + examples = """ + Examples: + > SELECT _FUNC_(0); + 0.0 + > SELECT _FUNC_(2); + NaN + """, + since = "3.0.0") +case class Atanh(child: Expression) + extends UnaryMathExpression((x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x)), "ATANH") { + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, c => s"0.5 * java.lang.Math.log((1.0 + $c)/(1.0 - $c))") + } +} + @ExpressionDescription( usage = "_FUNC_(expr) - Converts radians to degrees.", arguments = """ @@ -1138,8 +1209,10 @@ abstract class RoundBase(child: Expression, scale: Expression, val evaluationCode = dataType match { case DecimalType.Fixed(_, s) => s""" - ${ev.value} = ${ce.value}.toPrecision(${ce.value}.precision(), $s, Decimal.$modeStr()); - ${ev.isNull} = ${ev.value} == null;""" + |${ev.value} = ${ce.value}.toPrecision(${ce.value}.precision(), $s, + | Decimal.$modeStr(), true); + |${ev.isNull} = ${ev.value} == null; + """.stripMargin case ByteType => if (_scale < 0) { s""" 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 0cdeda9b10516..2af2b13ad77f5 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.RandomUUIDGenerator +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -40,7 +41,7 @@ case class PrintToStderr(child: Expression) extends UnaryExpression { input } - private val outputPrefix = s"Result of ${child.simpleString} is " + private val outputPrefix = s"Result of ${child.simpleString(SQLConf.get.maxToStringFields)} is " override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { val outputPrefixField = ctx.addReferenceObj("outputPrefix", outputPrefix) @@ -72,7 +73,7 @@ case class AssertTrue(child: Expression) extends UnaryExpression with ImplicitCa override def prettyName: String = "assert_true" - private val errMsg = s"'${child.simpleString}' is not true!" + private val errMsg = s"'${child.simpleString(SQLConf.get.maxToStringFields)}' is not true!" override def eval(input: InternalRow) : Any = { val v = child.eval(input) @@ -124,7 +125,9 @@ case class CurrentDatabase() extends LeafExpression with Unevaluable { > SELECT _FUNC_(); 46707d92-02f4-4817-8116-a4c3b23e6266 """, - note = "The function is non-deterministic.") + note = """ + The function is non-deterministic. + """) // scalastyle:on line.size.limit case class Uuid(randomSeed: Option[Long] = None) extends LeafExpression with Stateful with ExpressionWithRandomSeed { 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 a80764c5a8d19..8456f4daa2c49 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 @@ -109,12 +109,14 @@ trait NamedExpression extends Expression { abstract class Attribute extends LeafExpression with NamedExpression with NullIntolerant { - override def references: AttributeSet = AttributeSet(this) + @transient + override lazy val references: AttributeSet = AttributeSet(this) def withNullability(newNullability: Boolean): Attribute def withQualifier(newQualifier: Seq[String]): Attribute def withName(newName: String): Attribute def withMetadata(newMetadata: Metadata): Attribute + def withExprId(newExprId: ExprId): Attribute override def toAttribute: Attribute = this def newInstance(): Attribute @@ -129,6 +131,9 @@ abstract class Attribute extends LeafExpression with NamedExpression with NullIn * Note that exprId and qualifiers are in a separate parameter list because * we only pattern match on child and name. * + * Note that when creating a new Alias, all the [[AttributeReference]] that refer to + * the original alias should be updated to the new one. + * * @param child The computation being performed * @param name The name to be associated with the result of computing [[child]]. * @param exprId A globally unique id used to check if an [[AttributeReference]] refers to this @@ -299,7 +304,7 @@ case class AttributeReference( } } - def withExprId(newExprId: ExprId): AttributeReference = { + override def withExprId(newExprId: ExprId): AttributeReference = { if (exprId == newExprId) { this } else { @@ -307,7 +312,7 @@ case class AttributeReference( } } - override def withMetadata(newMetadata: Metadata): Attribute = { + override def withMetadata(newMetadata: Metadata): AttributeReference = { AttributeReference(name, dataType, nullable, newMetadata)(exprId, qualifier) } @@ -326,7 +331,9 @@ case class AttributeReference( // Since the expression id is not in the first constructor it is missing from the default // tree string. - override def simpleString: String = s"$name#${exprId.id}: ${dataType.simpleString}" + override def simpleString(maxFields: Int): String = { + s"$name#${exprId.id}: ${dataType.simpleString(maxFields)}" + } override def sql: String = { val qualifierPrefix = if (qualifier.nonEmpty) qualifier.mkString(".") + "." else "" @@ -362,6 +369,8 @@ case class PrettyAttribute( throw new UnsupportedOperationException override def qualifier: Seq[String] = throw new UnsupportedOperationException override def exprId: ExprId = throw new UnsupportedOperationException + override def withExprId(newExprId: ExprId): Attribute = + throw new UnsupportedOperationException override def nullable: Boolean = true } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala index b683d2a7e9ef3..293d28e93039a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullExpressions.scala @@ -42,7 +42,8 @@ import org.apache.spark.sql.types._ Examples: > SELECT _FUNC_(NULL, 1, NULL); 1 - """) + """, + since = "1.0.0") // scalastyle:on line.size.limit case class Coalesce(children: Seq[Expression]) extends ComplexTypeMergingExpression { @@ -127,7 +128,8 @@ case class Coalesce(children: Seq[Expression]) extends ComplexTypeMergingExpress Examples: > SELECT _FUNC_(NULL, array('2')); ["2"] - """) + """, + since = "2.0.0") case class IfNull(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { @@ -146,7 +148,8 @@ case class IfNull(left: Expression, right: Expression, child: Expression) Examples: > SELECT _FUNC_(2, 2); NULL - """) + """, + since = "2.0.0") case class NullIf(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { @@ -165,7 +168,8 @@ case class NullIf(left: Expression, right: Expression, child: Expression) Examples: > SELECT _FUNC_(NULL, array('2')); ["2"] - """) + """, + since = "2.0.0") case class Nvl(left: Expression, right: Expression, child: Expression) extends RuntimeReplaceable { def this(left: Expression, right: Expression) = { @@ -184,7 +188,8 @@ case class Nvl(left: Expression, right: Expression, child: Expression) extends R Examples: > SELECT _FUNC_(NULL, 2, 1); 1 - """) + """, + since = "2.0.0") // scalastyle:on line.size.limit case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression, child: Expression) extends RuntimeReplaceable { @@ -207,7 +212,8 @@ case class Nvl2(expr1: Expression, expr2: Expression, expr3: Expression, child: Examples: > SELECT _FUNC_(cast('NaN' as double)); true - """) + """, + since = "1.5.0") case class IsNaN(child: Expression) extends UnaryExpression with Predicate with ImplicitCastInputTypes { @@ -249,7 +255,8 @@ case class IsNaN(child: Expression) extends UnaryExpression Examples: > SELECT _FUNC_(cast('NaN' as double), 123); 123.0 - """) + """, + since = "1.5.0") case class NaNvl(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -309,7 +316,8 @@ case class NaNvl(left: Expression, right: Expression) Examples: > SELECT _FUNC_(1); false - """) + """, + since = "1.0.0") case class IsNull(child: Expression) extends UnaryExpression with Predicate { override def nullable: Boolean = false @@ -335,7 +343,8 @@ case class IsNull(child: Expression) extends UnaryExpression with Predicate { Examples: > SELECT _FUNC_(1); true - """) + """, + since = "1.0.0") case class IsNotNull(child: Expression) extends UnaryExpression with Predicate { override def nullable: Boolean = false 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 3189e6841a525..871aba67cf132 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 @@ -21,7 +21,6 @@ import java.lang.reflect.{Method, Modifier} import scala.collection.JavaConverters._ import scala.collection.mutable.Builder -import scala.language.existentials import scala.reflect.ClassTag import scala.util.Try @@ -36,7 +35,6 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, GenericArrayData, MapData} import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.Utils /** @@ -462,12 +460,12 @@ case class NewInstance( val d = outerObj.getClass +: paramTypes val c = getConstructor(outerObj.getClass +: paramTypes) (args: Seq[AnyRef]) => { - c.newInstance(outerObj +: args: _*) + c(outerObj +: args) } }.getOrElse { val c = getConstructor(paramTypes) (args: Seq[AnyRef]) => { - c.newInstance(args: _*) + c(args) } } } @@ -486,10 +484,16 @@ case class NewInstance( ev.isNull = resultIsNull - val constructorCall = outer.map { gen => - s"${gen.value}.new ${cls.getSimpleName}($argString)" - }.getOrElse { - s"new $className($argString)" + val constructorCall = cls.getConstructors.size match { + // If there are no constructors, the `new` method will fail. In + // this case we can try to call the apply method constructor + // that might be defined on the companion object. + case 0 => s"$className$$.MODULE$$.apply($argString)" + case _ => outer.map { gen => + s"${gen.value}.new ${cls.getSimpleName}($argString)" + }.getOrElse { + s"new $className($argString)" + } } val code = code""" @@ -571,36 +575,64 @@ case class WrapOption(child: Expression, optType: DataType) } } +object LambdaVariable { + private val curId = new java.util.concurrent.atomic.AtomicLong() + + // Returns the codegen-ed `LambdaVariable` and add it to mutable states, so that it can be + // accessed anywhere in the generated code. + def prepareLambdaVariable(ctx: CodegenContext, variable: LambdaVariable): ExprCode = { + val variableCode = variable.genCode(ctx) + assert(variableCode.code.isEmpty) + + ctx.addMutableState( + CodeGenerator.javaType(variable.dataType), + variableCode.value, + forceInline = true, + useFreshName = false) + + if (variable.nullable) { + ctx.addMutableState( + CodeGenerator.JAVA_BOOLEAN, + variableCode.isNull, + forceInline = true, + useFreshName = false) + } + + variableCode + } +} + /** - * A placeholder for the loop variable used in [[MapObjects]]. This should never be constructed + * A placeholder for the loop variable used in [[MapObjects]]. This should never be constructed * manually, but will instead be passed into the provided lambda function. */ +// TODO: Merge this and `NamedLambdaVariable`. case class LambdaVariable( - value: String, - isNull: String, + name: String, dataType: DataType, - nullable: Boolean = true) extends LeafExpression with NonSQLExpression { + nullable: Boolean, + id: Long = LambdaVariable.curId.incrementAndGet) extends LeafExpression with NonSQLExpression { - private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType) + private val accessor: (InternalRow, Int) => Any = InternalRow.getAccessor(dataType, nullable) // Interpreted execution of `LambdaVariable` always get the 0-index element from input row. override def eval(input: InternalRow): Any = { assert(input.numFields == 1, "The input row of interpreted LambdaVariable should have only 1 field.") - if (nullable && input.isNullAt(0)) { - null - } else { - accessor(input, 0) - } + accessor(input, 0) } override def genCode(ctx: CodegenContext): ExprCode = { - val isNullValue = if (nullable) { - JavaCode.isNullVariable(isNull) + // If `LambdaVariable` IDs are reassigned by the `ReassignLambdaVariableID` rule, the IDs will + // all be negative. + val suffix = "lambda_variable_" + math.abs(id) + val isNull = if (nullable) { + JavaCode.isNullVariable(s"isNull_${name}_$suffix") } else { FalseLiteral } - ExprCode(value = JavaCode.variable(value, dataType), isNull = isNullValue) + val value = JavaCode.variable(s"value_${name}_$suffix", dataType) + ExprCode(isNull, value) } // This won't be called as `genCode` is overrided, just overriding it to make @@ -630,8 +662,6 @@ case class UnresolvedMapObjects( } object MapObjects { - private val curId = new java.util.concurrent.atomic.AtomicInteger() - /** * Construct an instance of MapObjects case class. * @@ -649,16 +679,8 @@ object MapObjects { elementType: DataType, elementNullable: Boolean = true, customCollectionCls: Option[Class[_]] = None): MapObjects = { - val id = curId.getAndIncrement() - val loopValue = s"MapObjects_loopValue$id" - val loopIsNull = if (elementNullable) { - s"MapObjects_loopIsNull$id" - } else { - "false" - } - val loopVar = LambdaVariable(loopValue, loopIsNull, elementType, elementNullable) - MapObjects( - loopValue, loopIsNull, elementType, function(loopVar), inputData, customCollectionCls) + val loopVar = LambdaVariable("MapObject", elementType, elementNullable) + MapObjects(loopVar, function(loopVar), inputData, customCollectionCls) } } @@ -674,12 +696,8 @@ object MapObjects { * The following collection ObjectTypes are currently supported on input: * Seq, Array, ArrayData, java.util.List * - * @param loopValue the name of the loop variable that used when iterate the collection, and used - * as input for the `lambdaFunction` - * @param loopIsNull the nullity of the loop variable that used when iterate the collection, and - * used as input for the `lambdaFunction` - * @param loopVarDataType the data type of the loop variable that used when iterate the collection, - * and used as input for the `lambdaFunction` + * @param loopVar the [[LambdaVariable]] expression representing the loop variable that used to + * iterate the collection, and used as input for the `lambdaFunction`. * @param lambdaFunction A function that take the `loopVar` as input, and used as lambda function * to handle collection elements. * @param inputData An expression that when evaluated returns a collection object. @@ -687,16 +705,14 @@ object MapObjects { * or None (returning ArrayType) */ case class MapObjects private( - loopValue: String, - loopIsNull: String, - loopVarDataType: DataType, + loopVar: LambdaVariable, lambdaFunction: Expression, inputData: Expression, customCollectionCls: Option[Class[_]]) extends Expression with NonSQLExpression { override def nullable: Boolean = inputData.nullable - override def children: Seq[Expression] = lambdaFunction :: inputData :: Nil + override def children: Seq[Expression] = Seq(loopVar, lambdaFunction, inputData) // The data with UserDefinedType are actually stored with the data type of its sqlType. // When we want to apply MapObjects on it, we have to use it. @@ -790,8 +806,8 @@ case class MapObjects private( ArrayType(lambdaFunction.dataType, containsNull = lambdaFunction.nullable)) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val elementJavaType = CodeGenerator.javaType(loopVarDataType) - ctx.addMutableState(elementJavaType, loopValue, forceInline = true, useFreshName = false) + val elementJavaType = CodeGenerator.javaType(loopVar.dataType) + val loopVarCode = LambdaVariable.prepareLambdaVariable(ctx, loopVar) val genInputData = inputData.genCode(ctx) val genFunction = lambdaFunction.genCode(ctx) val dataLength = ctx.freshName("dataLength") @@ -879,12 +895,10 @@ case class MapObjects private( case _ => genFunction.value } - val loopNullCheck = if (loopIsNull != "false") { - ctx.addMutableState( - CodeGenerator.JAVA_BOOLEAN, loopIsNull, forceInline = true, useFreshName = false) + val loopNullCheck = if (loopVar.nullable) { inputDataType match { - case _: ArrayType => s"$loopIsNull = ${genInputData.value}.isNullAt($loopIndex);" - case _ => s"$loopIsNull = $loopValue == null;" + case _: ArrayType => s"${loopVarCode.isNull} = ${genInputData.value}.isNullAt($loopIndex);" + case _ => s"${loopVarCode.isNull} = ${loopVarCode.value} == null;" } } else { "" @@ -942,7 +956,7 @@ case class MapObjects private( int $loopIndex = 0; $prepareLoop while ($loopIndex < $dataLength) { - $loopValue = ($elementJavaType) ($getLoopVar); + ${loopVarCode.value} = ($elementJavaType) ($getLoopVar); $loopNullCheck ${genFunction.code} @@ -962,37 +976,36 @@ case class MapObjects private( } } -object CatalystToExternalMap { - private val curId = new java.util.concurrent.atomic.AtomicInteger() +/** + * Similar to [[UnresolvedMapObjects]], this is a placeholder of [[CatalystToExternalMap]]. + * + * @param child An expression that when evaluated returns a map object. + * @param keyFunction The function applied on the key collection elements. + * @param valueFunction The function applied on the value collection elements. + * @param collClass The type of the resulting collection. + */ +case class UnresolvedCatalystToExternalMap( + child: Expression, + @transient keyFunction: Expression => Expression, + @transient valueFunction: Expression => Expression, + collClass: Class[_]) extends UnaryExpression with Unevaluable { - /** - * Construct an instance of CatalystToExternalMap case class. - * - * @param keyFunction The function applied on the key collection elements. - * @param valueFunction The function applied on the value collection elements. - * @param inputData An expression that when evaluated returns a map object. - * @param collClass The type of the resulting collection. - */ - def apply( - keyFunction: Expression => Expression, - valueFunction: Expression => Expression, - inputData: Expression, - collClass: Class[_]): CatalystToExternalMap = { - val id = curId.getAndIncrement() - val keyLoopValue = s"CatalystToExternalMap_keyLoopValue$id" - val mapType = inputData.dataType.asInstanceOf[MapType] - val keyLoopVar = LambdaVariable(keyLoopValue, "", mapType.keyType, nullable = false) - val valueLoopValue = s"CatalystToExternalMap_valueLoopValue$id" - val valueLoopIsNull = if (mapType.valueContainsNull) { - s"CatalystToExternalMap_valueLoopIsNull$id" - } else { - "false" - } - val valueLoopVar = LambdaVariable(valueLoopValue, valueLoopIsNull, mapType.valueType) + override lazy val resolved = false + + override def dataType: DataType = ObjectType(collClass) +} + +object CatalystToExternalMap { + def apply(u: UnresolvedCatalystToExternalMap): CatalystToExternalMap = { + val mapType = u.child.dataType.asInstanceOf[MapType] + val keyLoopVar = LambdaVariable( + "CatalystToExternalMap_key", mapType.keyType, nullable = false) + val valueLoopVar = LambdaVariable( + "CatalystToExternalMap_value", mapType.valueType, mapType.valueContainsNull) CatalystToExternalMap( - keyLoopValue, keyFunction(keyLoopVar), - valueLoopValue, valueLoopIsNull, valueFunction(valueLoopVar), - inputData, collClass) + keyLoopVar, u.keyFunction(keyLoopVar), + valueLoopVar, u.valueFunction(valueLoopVar), + u.child, u.collClass) } } @@ -1001,33 +1014,31 @@ object CatalystToExternalMap { * The collection is constructed using the associated builder, obtained by calling `newBuilder` * on the collection's companion object. * - * @param keyLoopValue the name of the loop variable that is used when iterating over the key - * collection, and which is used as input for the `keyLambdaFunction` + * @param keyLoopVar the [[LambdaVariable]] expression representing the loop variable that is used + * when iterating over the key collection, and which is used as input for the + * `keyLambdaFunction`. * @param keyLambdaFunction A function that takes the `keyLoopVar` as input, and is used as * a lambda function to handle collection elements. - * @param valueLoopValue the name of the loop variable that is used when iterating over the value - * collection, and which is used as input for the `valueLambdaFunction` - * @param valueLoopIsNull the nullability of the loop variable that is used when iterating over - * the value collection, and which is used as input for the - * `valueLambdaFunction` + * @param valueLoopVar the [[LambdaVariable]] expression representing the loop variable that is used + * when iterating over the value collection, and which is used as input for the + * `valueLambdaFunction`. * @param valueLambdaFunction A function that takes the `valueLoopVar` as input, and is used as * a lambda function to handle collection elements. * @param inputData An expression that when evaluated returns a map object. * @param collClass The type of the resulting collection. */ case class CatalystToExternalMap private( - keyLoopValue: String, + keyLoopVar: LambdaVariable, keyLambdaFunction: Expression, - valueLoopValue: String, - valueLoopIsNull: String, + valueLoopVar: LambdaVariable, valueLambdaFunction: Expression, inputData: Expression, collClass: Class[_]) extends Expression with NonSQLExpression { override def nullable: Boolean = inputData.nullable - override def children: Seq[Expression] = - keyLambdaFunction :: valueLambdaFunction :: inputData :: Nil + override def children: Seq[Expression] = Seq( + keyLoopVar, keyLambdaFunction, valueLoopVar, valueLambdaFunction, inputData) private lazy val inputMapType = inputData.dataType.asInstanceOf[MapType] @@ -1068,20 +1079,11 @@ case class CatalystToExternalMap private( override def dataType: DataType = ObjectType(collClass) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - // The data with PythonUserDefinedType are actually stored with the data type of its sqlType. - // When we want to apply MapObjects on it, we have to use it. - def inputDataType(dataType: DataType) = dataType match { - case p: PythonUserDefinedType => p.sqlType - case _ => dataType - } - - val mapType = inputDataType(inputData.dataType).asInstanceOf[MapType] - val keyElementJavaType = CodeGenerator.javaType(mapType.keyType) - ctx.addMutableState(keyElementJavaType, keyLoopValue, forceInline = true, useFreshName = false) + val keyCode = LambdaVariable.prepareLambdaVariable(ctx, keyLoopVar) + val valueCode = LambdaVariable.prepareLambdaVariable(ctx, valueLoopVar) + val keyElementJavaType = CodeGenerator.javaType(keyLoopVar.dataType) val genKeyFunction = keyLambdaFunction.genCode(ctx) - val valueElementJavaType = CodeGenerator.javaType(mapType.valueType) - ctx.addMutableState(valueElementJavaType, valueLoopValue, forceInline = true, - useFreshName = false) + val valueElementJavaType = CodeGenerator.javaType(valueLoopVar.dataType) val genValueFunction = valueLambdaFunction.genCode(ctx) val genInputData = inputData.genCode(ctx) val dataLength = ctx.freshName("dataLength") @@ -1089,17 +1091,10 @@ case class CatalystToExternalMap private( val tupleLoopValue = ctx.freshName("tupleLoopValue") val builderValue = ctx.freshName("builderValue") - val getLength = s"${genInputData.value}.numElements()" - val keyArray = ctx.freshName("keyArray") val valueArray = ctx.freshName("valueArray") - val getKeyArray = - s"${classOf[ArrayData].getName} $keyArray = ${genInputData.value}.keyArray();" - val getKeyLoopVar = CodeGenerator.getValue(keyArray, inputDataType(mapType.keyType), loopIndex) - val getValueArray = - s"${classOf[ArrayData].getName} $valueArray = ${genInputData.value}.valueArray();" - val getValueLoopVar = CodeGenerator.getValue( - valueArray, inputDataType(mapType.valueType), loopIndex) + val getKeyLoopVar = CodeGenerator.getValue(keyArray, keyLoopVar.dataType, loopIndex) + val getValueLoopVar = CodeGenerator.getValue(valueArray, valueLoopVar.dataType, loopIndex) // Make a copy of the data if it's unsafe-backed def makeCopyIfInstanceOf(clazz: Class[_ <: Any], value: String) = @@ -1114,10 +1109,8 @@ case class CatalystToExternalMap private( val genKeyFunctionValue = genFunctionValue(keyLambdaFunction, genKeyFunction) val genValueFunctionValue = genFunctionValue(valueLambdaFunction, genValueFunction) - val valueLoopNullCheck = if (valueLoopIsNull != "false") { - ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, valueLoopIsNull, forceInline = true, - useFreshName = false) - s"$valueLoopIsNull = $valueArray.isNullAt($loopIndex);" + val valueLoopNullCheck = if (valueLoopVar.nullable) { + s"${valueCode.isNull} = $valueArray.isNullAt($loopIndex);" } else { "" } @@ -1146,15 +1139,15 @@ case class CatalystToExternalMap private( ${CodeGenerator.javaType(dataType)} ${ev.value} = ${CodeGenerator.defaultValue(dataType)}; if (!${genInputData.isNull}) { - int $dataLength = $getLength; + int $dataLength = ${genInputData.value}.numElements(); $constructBuilder - $getKeyArray - $getValueArray + ArrayData $keyArray = ${genInputData.value}.keyArray(); + ArrayData $valueArray = ${genInputData.value}.valueArray(); int $loopIndex = 0; while ($loopIndex < $dataLength) { - $keyLoopValue = ($keyElementJavaType) ($getKeyLoopVar); - $valueLoopValue = ($valueElementJavaType) ($getValueLoopVar); + ${keyCode.value} = ($keyElementJavaType) ($getKeyLoopVar); + ${valueCode.value} = ($valueElementJavaType) ($getValueLoopVar); $valueLoopNullCheck ${genKeyFunction.code} @@ -1173,8 +1166,6 @@ case class CatalystToExternalMap private( } object ExternalMapToCatalyst { - private val curId = new java.util.concurrent.atomic.AtomicInteger() - def apply( inputMap: Expression, keyType: DataType, @@ -1183,31 +1174,14 @@ object ExternalMapToCatalyst { valueType: DataType, valueConverter: Expression => Expression, valueNullable: Boolean): ExternalMapToCatalyst = { - val id = curId.getAndIncrement() - val keyName = "ExternalMapToCatalyst_key" + id - val keyIsNull = if (keyNullable) { - "ExternalMapToCatalyst_key_isNull" + id - } else { - "false" - } - val valueName = "ExternalMapToCatalyst_value" + id - val valueIsNull = if (valueNullable) { - "ExternalMapToCatalyst_value_isNull" + id - } else { - "false" - } - + val keyLoopVar = LambdaVariable("ExternalMapToCatalyst_key", keyType, keyNullable) + val valueLoopVar = LambdaVariable("ExternalMapToCatalyst_value", valueType, valueNullable) ExternalMapToCatalyst( - keyName, - keyIsNull, - keyType, - keyConverter(LambdaVariable(keyName, keyIsNull, keyType, keyNullable)), - valueName, - valueIsNull, - valueType, - valueConverter(LambdaVariable(valueName, valueIsNull, valueType, valueNullable)), - inputMap - ) + keyLoopVar, + keyConverter(keyLoopVar), + valueLoopVar, + valueConverter(valueLoopVar), + inputMap) } } @@ -1215,37 +1189,32 @@ object ExternalMapToCatalyst { * Converts a Scala/Java map object into catalyst format, by applying the key/value converter when * iterate the map. * - * @param key the name of the map key variable that used when iterate the map, and used as input for - * the `keyConverter` - * @param keyIsNull the nullability of the map key variable that used when iterate the map, and - * used as input for the `keyConverter` - * @param keyType the data type of the map key variable that used when iterate the map, and used as - * input for the `keyConverter` + * @param keyLoopVar the [[LambdaVariable]] expression representing the loop variable that is used + * when iterating over the key collection, and which is used as input for the + * `keyConverter`. * @param keyConverter A function that take the `key` as input, and converts it to catalyst format. - * @param value the name of the map value variable that used when iterate the map, and used as input - * for the `valueConverter` - * @param valueIsNull the nullability of the map value variable that used when iterate the map, and - * used as input for the `valueConverter` - * @param valueType the data type of the map value variable that used when iterate the map, and - * used as input for the `valueConverter` + * @param valueLoopVar the [[LambdaVariable]] expression representing the loop variable that is used + * when iterating over the value collection, and which is used as input for the + * `valueConverter`. * @param valueConverter A function that take the `value` as input, and converts it to catalyst * format. - * @param child An expression that when evaluated returns the input map object. + * @param inputData An expression that when evaluated returns the input map object. */ case class ExternalMapToCatalyst private( - key: String, - keyIsNull: String, - keyType: DataType, + keyLoopVar: LambdaVariable, keyConverter: Expression, - value: String, - valueIsNull: String, - valueType: DataType, + valueLoopVar: LambdaVariable, valueConverter: Expression, - child: Expression) - extends UnaryExpression with NonSQLExpression { + inputData: Expression) + extends Expression with NonSQLExpression { override def foldable: Boolean = false + override def nullable: Boolean = inputData.nullable + + override def children: Seq[Expression] = Seq( + keyLoopVar, keyConverter, valueLoopVar, valueConverter, inputData) + override def dataType: MapType = MapType( keyConverter.dataType, valueConverter.dataType, valueContainsNull = valueConverter.nullable) @@ -1256,7 +1225,7 @@ case class ExternalMapToCatalyst private( rowBuffer } - child.dataType match { + inputData.dataType match { case ObjectType(cls) if classOf[java.util.Map[_, _]].isAssignableFrom(cls) => (input: Any) => { val data = input.asInstanceOf[java.util.Map[Any, Any]] @@ -1307,7 +1276,7 @@ case class ExternalMapToCatalyst private( } override def eval(input: InternalRow): Any = { - val result = child.eval(input) + val result = inputData.eval(input) if (result != null) { val (keys, values) = mapCatalystConverter(result) new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values)) @@ -1317,7 +1286,7 @@ case class ExternalMapToCatalyst private( } override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val inputMap = child.genCode(ctx) + val inputMap = inputData.genCode(ctx) val genKeyConverter = keyConverter.genCode(ctx) val genValueConverter = valueConverter.genCode(ctx) val length = ctx.freshName("length") @@ -1327,12 +1296,12 @@ case class ExternalMapToCatalyst private( val entry = ctx.freshName("entry") val entries = ctx.freshName("entries") - val keyElementJavaType = CodeGenerator.javaType(keyType) - val valueElementJavaType = CodeGenerator.javaType(valueType) - ctx.addMutableState(keyElementJavaType, key, forceInline = true, useFreshName = false) - ctx.addMutableState(valueElementJavaType, value, forceInline = true, useFreshName = false) + val keyJavaType = CodeGenerator.javaType(keyLoopVar.dataType) + val valueJavaType = CodeGenerator.javaType(valueLoopVar.dataType) + val keyCode = LambdaVariable.prepareLambdaVariable(ctx, keyLoopVar) + val valueCode = LambdaVariable.prepareLambdaVariable(ctx, valueLoopVar) - val (defineEntries, defineKeyValue) = child.dataType match { + val (defineEntries, defineKeyValue) = inputData.dataType match { case ObjectType(cls) if classOf[java.util.Map[_, _]].isAssignableFrom(cls) => val javaIteratorCls = classOf[java.util.Iterator[_]].getName val javaMapEntryCls = classOf[java.util.Map.Entry[_, _]].getName @@ -1343,8 +1312,8 @@ case class ExternalMapToCatalyst private( val defineKeyValue = s""" final $javaMapEntryCls $entry = ($javaMapEntryCls) $entries.next(); - $key = (${CodeGenerator.boxedType(keyType)}) $entry.getKey(); - $value = (${CodeGenerator.boxedType(valueType)}) $entry.getValue(); + ${keyCode.value} = (${CodeGenerator.boxedType(keyJavaType)}) $entry.getKey(); + ${valueCode.value} = (${CodeGenerator.boxedType(valueJavaType)}) $entry.getValue(); """ defineEntries -> defineKeyValue @@ -1358,25 +1327,21 @@ case class ExternalMapToCatalyst private( val defineKeyValue = s""" final $scalaMapEntryCls $entry = ($scalaMapEntryCls) $entries.next(); - $key = (${CodeGenerator.boxedType(keyType)}) $entry._1(); - $value = (${CodeGenerator.boxedType(valueType)}) $entry._2(); + ${keyCode.value} = (${CodeGenerator.boxedType(keyJavaType)}) $entry._1(); + ${valueCode.value} = (${CodeGenerator.boxedType(valueJavaType)}) $entry._2(); """ defineEntries -> defineKeyValue } - val keyNullCheck = if (keyIsNull != "false") { - ctx.addMutableState( - CodeGenerator.JAVA_BOOLEAN, keyIsNull, forceInline = true, useFreshName = false) - s"$keyIsNull = $key == null;" + val keyNullCheck = if (keyLoopVar.nullable) { + s"${keyCode.isNull} = ${keyCode.value} == null;" } else { "" } - val valueNullCheck = if (valueIsNull != "false") { - ctx.addMutableState( - CodeGenerator.JAVA_BOOLEAN, valueIsNull, forceInline = true, useFreshName = false) - s"$valueIsNull = $value == null;" + val valueNullCheck = if (valueLoopVar.nullable) { + s"${valueCode.isNull} = ${valueCode.value} == null;" } else { "" } 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 e24a3de3cfdbe..c8d667143f452 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.types._ @@ -27,7 +28,7 @@ import org.apache.spark.sql.types._ class InterpretedOrdering(ordering: Seq[SortOrder]) extends Ordering[InternalRow] { def this(ordering: Seq[SortOrder], inputSchema: Seq[Attribute]) = - this(ordering.map(BindReferences.bindReference(_, inputSchema))) + this(bindReferences(ordering, inputSchema)) def compare(a: InternalRow, b: InternalRow): Int = { var i = 0 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 0083ee64653e9..932c364737249 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 @@ -85,13 +85,6 @@ package object expressions { override def apply(row: InternalRow): InternalRow = row } - /** - * A helper function to bind given expressions to an input schema. - */ - def toBoundExprs(exprs: Seq[Expression], inputSchema: Seq[Attribute]): Seq[Expression] = { - exprs.map(BindReferences.bindReference(_, inputSchema)) - } - /** * Helper functions for working with `Seq[Attribute]`. */ @@ -101,7 +94,7 @@ package object expressions { StructType(attrs.map(a => StructField(a.name, a.dataType, a.nullable, a.metadata))) } - // It's possible that `attrs` is a linked list, which can lead to bad O(n^2) loops when + // It's possible that `attrs` is a linked list, which can lead to bad O(n) loops when // accessing attributes by their ordinals. To avoid this performance penalty, convert the input // to an array. @transient private lazy val attrsArray = attrs.toArray diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala index 2125340f38ee8..02d5a1f27aa7f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGe import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.TypeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -115,6 +116,10 @@ trait PredicateHelper { // non-correlated subquery will be replaced as literal e.children.isEmpty case a: AttributeReference => true + // PythonUDF will be executed by dedicated physical operator later. + // For PythonUDFs that can't be evaluated in join condition, `PullOutPythonUDFInJoinCondition` + // will pull them out later. + case _: PythonUDF => true case e: Unevaluable => false case e => e.children.forall(canEvaluateWithinJoin) } @@ -129,6 +134,13 @@ case class Not(child: Expression) override def inputTypes: Seq[DataType] = Seq(BooleanType) + // +---------+-----------+ + // | CHILD | NOT CHILD | + // +---------+-----------+ + // | TRUE | FALSE | + // | FALSE | TRUE | + // | UNKNOWN | UNKNOWN | + // +---------+-----------+ protected override def nullSafeEval(input: Any): Any = !input.asInstanceOf[Boolean] override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -360,31 +372,67 @@ case class InSet(child: Expression, hset: Set[Any]) extends UnaryExpression with } @transient lazy val set: Set[Any] = child.dataType match { - case _: AtomicType => hset + case t: AtomicType if !t.isInstanceOf[BinaryType] => hset case _: NullType => hset case _ => // for structs use interpreted ordering to be able to compare UnsafeRows with non-UnsafeRows - TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ hset + TreeSet.empty(TypeUtils.getInterpretedOrdering(child.dataType)) ++ (hset - null) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { - val setTerm = ctx.addReferenceObj("set", set) - val childGen = child.genCode(ctx) - val setIsNull = if (hasNull) { - s"${ev.isNull} = !${ev.value};" + if (canBeComputedUsingSwitch && hset.size <= SQLConf.get.optimizerInSetSwitchThreshold) { + genCodeWithSwitch(ctx, ev) } else { - "" + genCodeWithSet(ctx, ev) } + } + + private def canBeComputedUsingSwitch: Boolean = child.dataType match { + case ByteType | ShortType | IntegerType | DateType => true + case _ => false + } + + private def genCodeWithSet(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, c => { + val setTerm = ctx.addReferenceObj("set", set) + val setIsNull = if (hasNull) { + s"${ev.isNull} = !${ev.value};" + } else { + "" + } + s""" + |${ev.value} = $setTerm.contains($c); + |$setIsNull + """.stripMargin + }) + } + + // spark.sql.optimizer.inSetSwitchThreshold has an appropriate upper limit, + // so the code size should not exceed 64KB + private def genCodeWithSwitch(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val caseValuesGen = hset.filter(_ != null).map(Literal(_).genCode(ctx)) + val valueGen = child.genCode(ctx) + + val caseBranches = caseValuesGen.map(literal => + code""" + case ${literal.value}: + ${ev.value} = true; + break; + """) + ev.copy(code = code""" - |${childGen.code} - |${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${childGen.isNull}; - |${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; - |if (!${ev.isNull}) { - | ${ev.value} = $setTerm.contains(${childGen.value}); - | $setIsNull - |} - """.stripMargin) + ${valueGen.code} + ${CodeGenerator.JAVA_BOOLEAN} ${ev.isNull} = ${valueGen.isNull}; + ${CodeGenerator.JAVA_BOOLEAN} ${ev.value} = false; + if (!${valueGen.isNull}) { + switch (${valueGen.value}) { + ${caseBranches.mkString("\n")} + default: + ${ev.isNull} = $hasNull; + } + } + """) } override def sql: String = { @@ -404,6 +452,13 @@ case class And(left: Expression, right: Expression) extends BinaryOperator with override def sqlOperator: String = "AND" + // +---------+---------+---------+---------+ + // | AND | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | UNKNOWN | + // | FALSE | FALSE | FALSE | FALSE | + // | UNKNOWN | UNKNOWN | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == false) { @@ -467,6 +522,13 @@ case class Or(left: Expression, right: Expression) extends BinaryOperator with P override def sqlOperator: String = "OR" + // +---------+---------+---------+---------+ + // | OR | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | TRUE | TRUE | + // | FALSE | TRUE | FALSE | UNKNOWN | + // | UNKNOWN | TRUE | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) if (input1 == true) { @@ -590,6 +652,13 @@ case class EqualTo(left: Expression, right: Expression) override def symbol: String = "=" + // +---------+---------+---------+---------+ + // | = | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | UNKNOWN | + // | FALSE | FALSE | TRUE | UNKNOWN | + // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ protected override def nullSafeEval(left: Any, right: Any): Any = ordering.equiv(left, right) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { @@ -627,6 +696,13 @@ case class EqualNullSafe(left: Expression, right: Expression) extends BinaryComp override def nullable: Boolean = false + // +---------+---------+---------+---------+ + // | <=> | TRUE | FALSE | UNKNOWN | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | FALSE | FALSE | + // | FALSE | FALSE | TRUE | FALSE | + // | UNKNOWN | FALSE | FALSE | TRUE | + // +---------+---------+---------+---------+ override def eval(input: InternalRow): Any = { val input1 = left.eval(input) val input2 = right.eval(input) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala index b70c34141b97d..cc05828cfcccb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/randomExpressions.scala @@ -78,7 +78,10 @@ trait ExpressionWithRandomSeed { > SELECT _FUNC_(null); 0.8446490682263027 """, - note = "The function is non-deterministic in general case.") + note = """ + The function is non-deterministic in general case. + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Rand(child: Expression) extends RDG with ExpressionWithRandomSeed { @@ -118,7 +121,10 @@ object Rand { > SELECT _FUNC_(null); 1.1164209726833079 """, - note = "The function is non-deterministic in general case.") + note = """ + The function is non-deterministic in general case. + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Randn(child: Expression) extends RDG with ExpressionWithRandomSeed { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala index 4f5ea1e95f833..65c485845daf4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/regexpExpressions.scala @@ -101,7 +101,8 @@ abstract class StringRegexExpression extends BinaryExpression """, note = """ Use RLIKE to match with standard regular expressions. - """) + """, + since = "1.0.0") case class Like(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = StringUtils.escapeLikeRegex(v) @@ -179,7 +180,8 @@ case class Like(left: Expression, right: Expression) extends StringRegexExpressi """, note = """ Use LIKE to match with simple string pattern. - """) + """, + since = "1.0.0") case class RLike(left: Expression, right: Expression) extends StringRegexExpression { override def escape(v: String): String = v @@ -254,7 +256,8 @@ case class RLike(left: Expression, right: Expression) extends StringRegexExpress ["one","two","three",""] > SELECT _FUNC_('oneAtwoBthreeC', '[ABC]', 2); ["one","twoBthreeC"] - """) + """, + since = "1.5.0") case class StringSplit(str: Expression, regex: Expression, limit: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -294,7 +297,8 @@ case class StringSplit(str: Expression, regex: Expression, limit: Expression) Examples: > SELECT _FUNC_('100-200', '(\\d+)', 'num'); num-num - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -393,7 +397,8 @@ case class RegExpReplace(subject: Expression, regexp: Expression, rep: Expressio Examples: > SELECT _FUNC_('100-200', '(\\d+)-(\\d+)', 1); 100 - """) + """, + since = "1.5.0") case class RegExpExtract(subject: Expression, regexp: Expression, idx: Expression) extends TernaryExpression with ImplicitCastInputTypes { def this(s: Expression, r: Expression) = this(s, r, Literal(1)) 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 cd824ee87ca53..d7a5fb27a3d56 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 @@ -24,12 +24,15 @@ import java.util.regex.Pattern 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.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData, TypeUtils} import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.UTF8StringBuilder import org.apache.spark.unsafe.types.{ByteArray, UTF8String} //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -50,7 +53,8 @@ import org.apache.spark.unsafe.types.{ByteArray, UTF8String} Examples: > SELECT _FUNC_(' ', 'Spark', 'SQL'); Spark SQL - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class ConcatWs(children: Seq[Expression]) extends Expression with ImplicitCastInputTypes { @@ -203,7 +207,8 @@ case class ConcatWs(children: Seq[Expression]) Examples: > SELECT _FUNC_(1, 'scala', 'java'); scala - """) + """, + since = "2.0.0") // scalastyle:on line.size.limit case class Elt(children: Seq[Expression]) extends Expression { @@ -326,7 +331,8 @@ trait String2StringExpression extends ImplicitCastInputTypes { Examples: > SELECT _FUNC_('SparkSql'); SPARKSQL - """) + """, + since = "1.0.1") case class Upper(child: Expression) extends UnaryExpression with String2StringExpression { @@ -348,7 +354,8 @@ case class Upper(child: Expression) Examples: > SELECT _FUNC_('SparkSql'); sparksql - """) + """, + since = "1.0.1") case class Lower(child: Expression) extends UnaryExpression with String2StringExpression { // scalastyle:off caselocale @@ -421,7 +428,8 @@ case class EndsWith(left: Expression, right: Expression) extends StringPredicate Examples: > SELECT _FUNC_('ABCabc', 'abc', 'DEF'); ABCDEF - """) + """, + since = "2.3.0") // scalastyle:on line.size.limit case class StringReplace(srcExpr: Expression, searchExpr: Expression, replaceExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -447,6 +455,69 @@ case class StringReplace(srcExpr: Expression, searchExpr: Expression, replaceExp override def prettyName: String = "replace" } +object Overlay { + + def calculate(input: UTF8String, replace: UTF8String, pos: Int, len: Int): UTF8String = { + val builder = new UTF8StringBuilder + builder.append(input.substringSQL(1, pos - 1)) + builder.append(replace) + // If you specify length, it must be a positive whole number or zero. + // Otherwise it will be ignored. + // The default value for length is the length of replace. + val length = if (len >= 0) { + len + } else { + replace.numChars + } + builder.append(input.substringSQL(pos + length, Int.MaxValue)) + builder.build() + } +} + +// scalastyle:off line.size.limit +@ExpressionDescription( + usage = "_FUNC_(input, replace, pos[, len]) - Replace `input` with `replace` that starts at `pos` and is of length `len`.", + examples = """ + Examples: + > SELECT _FUNC_('Spark SQL' PLACING '_' FROM 6); + Spark_SQL + > SELECT _FUNC_('Spark SQL' PLACING 'CORE' FROM 7); + Spark CORE + > SELECT _FUNC_('Spark SQL' PLACING 'ANSI ' FROM 7 FOR 0); + Spark ANSI SQL + > SELECT _FUNC_('Spark SQL' PLACING 'tructured' FROM 2 FOR 4); + Structured SQL + """) +// scalastyle:on line.size.limit +case class Overlay(input: Expression, replace: Expression, pos: Expression, len: Expression) + extends QuaternaryExpression with ImplicitCastInputTypes with NullIntolerant { + + def this(str: Expression, replace: Expression, pos: Expression) = { + this(str, replace, pos, Literal.create(-1, IntegerType)) + } + + override def dataType: DataType = StringType + + override def inputTypes: Seq[AbstractDataType] = + Seq(StringType, StringType, IntegerType, IntegerType) + + override def children: Seq[Expression] = input :: replace :: pos :: len :: Nil + + override def nullSafeEval(inputEval: Any, replaceEval: Any, posEval: Any, lenEval: Any): Any = { + val inputStr = inputEval.asInstanceOf[UTF8String] + val replaceStr = replaceEval.asInstanceOf[UTF8String] + val position = posEval.asInstanceOf[Int] + val length = lenEval.asInstanceOf[Int] + Overlay.calculate(inputStr, replaceStr, position, length) + } + + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + defineCodeGen(ctx, ev, (input, replace, pos, len) => + "org.apache.spark.sql.catalyst.expressions.Overlay" + + s".calculate($input, $replace, $pos, $len);") + } +} + object StringTranslate { def buildDict(matchingString: UTF8String, replaceString: UTF8String) @@ -479,7 +550,8 @@ object StringTranslate { Examples: > SELECT _FUNC_('AaBbCc', 'abc', '123'); A1B2C3 - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replaceExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -543,7 +615,8 @@ case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, replac Examples: > SELECT _FUNC_('ab','abc,b,ab,c,def'); 3 - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class FindInSet(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -596,11 +669,21 @@ object StringTrim { usage = """ _FUNC_(str) - Removes the leading and trailing space characters from `str`. - _FUNC_(BOTH trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str` + _FUNC_(BOTH FROM str) - Removes the leading and trailing space characters from `str`. + + _FUNC_(LEADING FROM str) - Removes the leading space characters from `str`. + + _FUNC_(TRAILING FROM str) - Removes the trailing space characters from `str`. - _FUNC_(LEADING trimStr FROM str) - Remove the leading `trimStr` characters from `str` + _FUNC_(str, trimStr) - Remove the leading and trailing `trimStr` characters from `str`. - _FUNC_(TRAILING trimStr FROM str) - Remove the trailing `trimStr` characters from `str` + _FUNC_(trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str`. + + _FUNC_(BOTH trimStr FROM str) - Remove the leading and trailing `trimStr` characters from `str`. + + _FUNC_(LEADING trimStr FROM str) - Remove the leading `trimStr` characters from `str`. + + _FUNC_(TRAILING trimStr FROM str) - Remove the trailing `trimStr` characters from `str`. """, arguments = """ Arguments: @@ -617,7 +700,15 @@ object StringTrim { Examples: > SELECT _FUNC_(' SparkSQL '); SparkSQL - > SELECT _FUNC_('SL', 'SSparkSQLS'); + > SELECT _FUNC_(BOTH FROM ' SparkSQL '); + SparkSQL + > SELECT _FUNC_(LEADING FROM ' SparkSQL '); + SparkSQL + > SELECT _FUNC_(TRAILING FROM ' SparkSQL '); + SparkSQL + > SELECT _FUNC_('SSparkSQLS', 'SL'); + parkSQ + > SELECT _FUNC_('SL' FROM 'SSparkSQLS'); parkSQ > SELECT _FUNC_(BOTH 'SL' FROM 'SSparkSQLS'); parkSQ @@ -625,13 +716,14 @@ object StringTrim { parkSQLS > SELECT _FUNC_(TRAILING 'SL' FROM 'SSparkSQLS'); SSparkSQ - """) + """, + since = "1.5.0") case class StringTrim( srcStr: Expression, trimStr: Option[Expression] = None) extends String2TrimExpression { - def this(trimStr: Expression, srcStr: Expression) = this(srcStr, Option(trimStr)) + def this(srcStr: Expression, trimStr: Expression) = this(srcStr, Option(trimStr)) def this(srcStr: Expression) = this(srcStr, None) @@ -724,13 +816,14 @@ object StringTrimLeft { SparkSQL > SELECT _FUNC_('Sp', 'SSparkSQLS'); arkSQLS - """) + """, + since = "1.5.0") case class StringTrimLeft( srcStr: Expression, trimStr: Option[Expression] = None) extends String2TrimExpression { - def this(trimStr: Expression, srcStr: Expression) = this(srcStr, Option(trimStr)) + def this(srcStr: Expression, trimStr: Expression) = this(srcStr, Option(trimStr)) def this(srcStr: Expression) = this(srcStr, None) @@ -825,14 +918,15 @@ object StringTrimRight { SparkSQL > SELECT _FUNC_('LQSa', 'SSparkSQLS'); SSpark - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class StringTrimRight( srcStr: Expression, trimStr: Option[Expression] = None) extends String2TrimExpression { - def this(trimStr: Expression, srcStr: Expression) = this(srcStr, Option(trimStr)) + def this(srcStr: Expression, trimStr: Expression) = this(srcStr, Option(trimStr)) def this(srcStr: Expression) = this(srcStr, None) @@ -905,7 +999,8 @@ case class StringTrimRight( Examples: > SELECT _FUNC_('SparkSQL', 'SQL'); 6 - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class StringInstr(str: Expression, substr: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -946,7 +1041,8 @@ case class StringInstr(str: Expression, substr: Expression) Examples: > SELECT _FUNC_('www.apache.org', '.', 2); www.apache - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -985,7 +1081,8 @@ case class SubstringIndex(strExpr: Expression, delimExpr: Expression, countExpr: 7 > SELECT POSITION('bar' IN 'foobarbar'); 4 - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class StringLocate(substr: Expression, str: Expression, start: Expression) extends TernaryExpression with ImplicitCastInputTypes { @@ -1061,8 +1158,9 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) */ @ExpressionDescription( usage = """ - _FUNC_(str, len, pad) - Returns `str`, left-padded with `pad` to a length of `len`. + _FUNC_(str, len[, pad]) - Returns `str`, left-padded with `pad` to a length of `len`. If `str` is longer than `len`, the return value is shortened to `len` characters. + If `pad` is not specified, `str` will be padded to the left with space characters. """, examples = """ Examples: @@ -1070,10 +1168,17 @@ case class StringLocate(substr: Expression, str: Expression, start: Expression) ???hi > SELECT _FUNC_('hi', 1, '??'); h - """) -case class StringLPad(str: Expression, len: Expression, pad: Expression) + > SELECT _FUNC_('hi', 5); + hi + """, + since = "1.5.0") +case class StringLPad(str: Expression, len: Expression, pad: Expression = Literal(" ")) extends TernaryExpression with ImplicitCastInputTypes { + def this(str: Expression, len: Expression) = { + this(str, len, Literal(" ")) + } + override def children: Seq[Expression] = str :: len :: pad :: Nil override def dataType: DataType = StringType override def inputTypes: Seq[DataType] = Seq(StringType, IntegerType, StringType) @@ -1094,8 +1199,9 @@ case class StringLPad(str: Expression, len: Expression, pad: Expression) */ @ExpressionDescription( usage = """ - _FUNC_(str, len, pad) - Returns `str`, right-padded with `pad` to a length of `len`. + _FUNC_(str, len[, pad]) - Returns `str`, right-padded with `pad` to a length of `len`. If `str` is longer than `len`, the return value is shortened to `len` characters. + If `pad` is not specified, `str` will be padded to the right with space characters. """, examples = """ Examples: @@ -1103,10 +1209,17 @@ case class StringLPad(str: Expression, len: Expression, pad: Expression) hi??? > SELECT _FUNC_('hi', 1, '??'); h - """) -case class StringRPad(str: Expression, len: Expression, pad: Expression) + > SELECT _FUNC_('hi', 5); + hi + """, + since = "1.5.0") +case class StringRPad(str: Expression, len: Expression, pad: Expression = Literal(" ")) extends TernaryExpression with ImplicitCastInputTypes { + def this(str: Expression, len: Expression) = { + this(str, len, Literal(" ")) + } + override def children: Seq[Expression] = str :: len :: pad :: Nil override def dataType: DataType = StringType override def inputTypes: Seq[DataType] = Seq(StringType, IntegerType, StringType) @@ -1148,7 +1261,8 @@ object ParseUrl { query=1 > SELECT _FUNC_('http://spark.apache.org/path?query=1', 'QUERY', 'query') 1 - """) + """, + since = "2.0.0") case class ParseUrl(children: Seq[Expression]) extends Expression with ExpectsInputTypes with CodegenFallback { @@ -1299,7 +1413,8 @@ case class ParseUrl(children: Seq[Expression]) Examples: > SELECT _FUNC_("Hello World %d %s", 100, "days"); Hello World 100 days - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class FormatString(children: Expression*) extends Expression with ImplicitCastInputTypes { @@ -1386,7 +1501,8 @@ case class FormatString(children: Expression*) extends Expression with ImplicitC Examples: > SELECT _FUNC_('sPark sql'); Spark Sql - """) + """, + since = "1.5.0") case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def inputTypes: Seq[DataType] = Seq(StringType) @@ -1411,7 +1527,8 @@ case class InitCap(child: Expression) extends UnaryExpression with ImplicitCastI Examples: > SELECT _FUNC_('123', 2); 123123 - """) + """, + since = "1.5.0") case class StringRepeat(str: Expression, times: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1440,7 +1557,8 @@ case class StringRepeat(str: Expression, times: Expression) Examples: > SELECT concat(_FUNC_(2), '1'); 1 - """) + """, + since = "1.5.0") case class StringSpace(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { @@ -1477,7 +1595,8 @@ case class StringSpace(child: Expression) SQL > SELECT _FUNC_('Spark SQL', 5, 1); k - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Substring(str: Expression, pos: Expression, len: Expression) extends TernaryExpression with ImplicitCastInputTypes with NullIntolerant { @@ -1523,7 +1642,8 @@ case class Substring(str: Expression, pos: Expression, len: Expression) Examples: > SELECT _FUNC_('Spark SQL', 3); SQL - """) + """, + since = "2.3.0") // scalastyle:on line.size.limit case class Right(str: Expression, len: Expression, child: Expression) extends RuntimeReplaceable { def this(str: Expression, len: Expression) = { @@ -1545,7 +1665,8 @@ case class Right(str: Expression, len: Expression, child: Expression) extends Ru Examples: > SELECT _FUNC_('Spark SQL', 3); Spa - """) + """, + since = "2.3.0") // scalastyle:on line.size.limit case class Left(str: Expression, len: Expression, child: Expression) extends RuntimeReplaceable { def this(str: Expression, len: Expression) = { @@ -1571,7 +1692,8 @@ case class Left(str: Expression, len: Expression, child: Expression) extends Run 10 > SELECT CHARACTER_LENGTH('Spark SQL '); 10 - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Length(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType @@ -1599,7 +1721,8 @@ case class Length(child: Expression) extends UnaryExpression with ImplicitCastIn Examples: > SELECT _FUNC_('Spark SQL'); 72 - """) + """, + since = "2.3.0") case class BitLength(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, BinaryType)) @@ -1629,7 +1752,8 @@ case class BitLength(child: Expression) extends UnaryExpression with ImplicitCas Examples: > SELECT _FUNC_('Spark SQL'); 9 - """) + """, + since = "2.3.0") case class OctetLength(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(StringType, BinaryType)) @@ -1658,7 +1782,8 @@ case class OctetLength(child: Expression) extends UnaryExpression with ImplicitC Examples: > SELECT _FUNC_('kitten', 'sitting'); 3 - """) + """, + since = "1.5.0") case class Levenshtein(left: Expression, right: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1683,7 +1808,8 @@ case class Levenshtein(left: Expression, right: Expression) extends BinaryExpres Examples: > SELECT _FUNC_('Miller'); M460 - """) + """, + since = "1.5.0") case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputTypes { override def dataType: DataType = StringType @@ -1708,7 +1834,8 @@ case class SoundEx(child: Expression) extends UnaryExpression with ExpectsInputT 50 > SELECT _FUNC_(2); 50 - """) + """, + since = "1.5.0") case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = IntegerType @@ -1748,7 +1875,8 @@ case class Ascii(child: Expression) extends UnaryExpression with ImplicitCastInp Examples: > SELECT _FUNC_(65); A - """) + """, + since = "2.3.0") // scalastyle:on line.size.limit case class Chr(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { @@ -1791,22 +1919,21 @@ case class Chr(child: Expression) extends UnaryExpression with ImplicitCastInput Examples: > SELECT _FUNC_('Spark SQL'); U3BhcmsgU1FM - """) + """, + since = "1.5.0") case class Base64(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = StringType override def inputTypes: Seq[DataType] = Seq(BinaryType) protected override def nullSafeEval(bytes: Any): Any = { - UTF8String.fromBytes( - org.apache.commons.codec.binary.Base64.encodeBase64( - bytes.asInstanceOf[Array[Byte]])) + UTF8String.fromBytes(CommonsBase64.encodeBase64(bytes.asInstanceOf[Array[Byte]])) } override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (child) => { s"""${ev.value} = UTF8String.fromBytes( - org.apache.commons.codec.binary.Base64.encodeBase64($child)); + ${classOf[CommonsBase64].getName}.encodeBase64($child)); """}) } } @@ -1820,19 +1947,20 @@ case class Base64(child: Expression) extends UnaryExpression with ImplicitCastIn Examples: > SELECT _FUNC_('U3BhcmsgU1FM'); Spark SQL - """) + """, + since = "1.5.0") case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCastInputTypes { override def dataType: DataType = BinaryType override def inputTypes: Seq[DataType] = Seq(StringType) protected override def nullSafeEval(string: Any): Any = - org.apache.commons.codec.binary.Base64.decodeBase64(string.asInstanceOf[UTF8String].toString) + CommonsBase64.decodeBase64(string.asInstanceOf[UTF8String].toString) override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { nullSafeCodeGen(ctx, ev, (child) => { s""" - ${ev.value} = org.apache.commons.codec.binary.Base64.decodeBase64($child.toString()); + ${ev.value} = ${classOf[CommonsBase64].getName}.decodeBase64($child.toString()); """}) } } @@ -1849,7 +1977,8 @@ case class UnBase64(child: Expression) extends UnaryExpression with ImplicitCast Examples: > SELECT _FUNC_(encode('abc', 'utf-8'), 'utf-8'); abc - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Decode(bin: Expression, charset: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1888,7 +2017,8 @@ case class Decode(bin: Expression, charset: Expression) Examples: > SELECT _FUNC_('abc', 'utf-8'); abc - """) + """, + since = "1.5.0") // scalastyle:on line.size.limit case class Encode(value: Expression, charset: Expression) extends BinaryExpression with ImplicitCastInputTypes { @@ -1932,7 +2062,8 @@ case class Encode(value: Expression, charset: Expression) 12,332.1235 > SELECT _FUNC_(12332.123456, '##################.###'); 12332.123 - """) + """, + since = "1.5.0") case class FormatNumber(x: Expression, d: Expression) extends BinaryExpression with ExpectsInputTypes { @@ -2102,7 +2233,8 @@ case class FormatNumber(x: Expression, d: Expression) Examples: > SELECT _FUNC_('Hi there! Good morning.'); [["Hi","there"],["Good","morning"]] - """) + """, + since = "2.0.0") case class Sentences( str: Expression, language: Expression = Literal(""), 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 fc1caed84e272..bf9f2970ded16 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 @@ -37,6 +37,9 @@ abstract class PlanExpression[T <: QueryPlan[_]] extends Expression { /** Updates the expression with a new plan. */ def withNewPlan(plan: T): PlanExpression[T] + /** Defines how the canonicalization should work for this expression. */ + def canonicalize(attrs: AttributeSeq): PlanExpression[T] + protected def conditionString: String = children.mkString("[", " && ", "]") } @@ -58,7 +61,7 @@ abstract class SubqueryExpression( children.zip(p.children).forall(p => p._1.semanticEquals(p._2)) case _ => false } - def canonicalize(attrs: AttributeSeq): SubqueryExpression = { + override def canonicalize(attrs: AttributeSeq): SubqueryExpression = { // Normalize the outer references in the subquery plan. val normalizedPlan = plan.transformAllExpressions { case OuterReference(r) => OuterReference(QueryPlan.normalizeExprId(r, attrs)) @@ -248,7 +251,10 @@ case class ScalarSubquery( children: Seq[Expression] = Seq.empty, exprId: ExprId = NamedExpression.newExprId) extends SubqueryExpression(plan, children, exprId) with Unevaluable { - override def dataType: DataType = plan.schema.fields.head.dataType + override def dataType: DataType = { + assert(plan.schema.fields.nonEmpty, "Scalar subquery should have only one column") + plan.schema.fields.head.dataType + } override def nullable: Boolean = true override def withNewPlan(plan: LogicalPlan): ScalarSubquery = copy(plan = plan) override def toString: String = s"scalar-subquery#${exprId.id} $conditionString" diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala index 707f312499734..0b674d025d1ac 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/windowExpressions.scala @@ -206,7 +206,7 @@ case class SpecifiedWindowFrame( // Check combination (of expressions). (lower, upper) match { case (l: Expression, u: Expression) if !isValidFrameBoundary(l, u) => - TypeCheckFailure(s"Window frame upper bound '$upper' does not followes the lower bound " + + TypeCheckFailure(s"Window frame upper bound '$upper' does not follow the lower bound " + s"'$lower'.") case (l: SpecialFrameBoundary, _) => TypeCheckSuccess case (_, u: SpecialFrameBoundary) => TypeCheckSuccess @@ -242,8 +242,12 @@ case class SpecifiedWindowFrame( case e: Expression => e.sql + " FOLLOWING" } - private def isGreaterThan(l: Expression, r: Expression): Boolean = { - GreaterThan(l, r).eval().asInstanceOf[Boolean] + // Check whether the left boundary value is greater than the right boundary value. It's required + // that the both expressions have the same data type. + // Since CalendarIntervalType is not comparable, we only compare expressions that are AtomicType. + private def isGreaterThan(l: Expression, r: Expression): Boolean = l.dataType match { + case _: AtomicType => GreaterThan(l, r).eval().asInstanceOf[Boolean] + case _ => false } private def checkBoundary(b: Expression, location: String): TypeCheckResult = b match { 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 64152e04928d2..788eb00ba0160 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 @@ -18,10 +18,10 @@ package org.apache.spark.sql.catalyst.json import java.nio.charset.{Charset, StandardCharsets} -import java.util.{Locale, TimeZone} +import java.time.ZoneId +import java.util.Locale import com.fasterxml.jackson.core.{JsonFactory, JsonParser} -import org.apache.commons.lang3.time.FastDateFormat import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.util._ @@ -76,16 +76,16 @@ private[sql] class JSONOptions( // Whether to ignore column of all null values or empty array/struct during schema inference val dropFieldIfAllNull = parameters.get("dropFieldIfAllNull").map(_.toBoolean).getOrElse(false) - val timeZone: TimeZone = DateTimeUtils.getTimeZone( + // A language tag in IETF BCP 47 format + val locale: Locale = parameters.get("locale").map(Locale.forLanguageTag).getOrElse(Locale.US) + + val zoneId: ZoneId = DateTimeUtils.getZoneId( parameters.getOrElse(DateTimeUtils.TIMEZONE_OPTION, defaultTimeZoneId)) - // Uses `FastDateFormat` which can be direct replacement for `SimpleDateFormat` and thread-safe. - val dateFormat: FastDateFormat = - FastDateFormat.getInstance(parameters.getOrElse("dateFormat", "yyyy-MM-dd"), Locale.US) + val dateFormat: String = parameters.getOrElse("dateFormat", "yyyy-MM-dd") - val timestampFormat: FastDateFormat = - FastDateFormat.getInstance( - parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX"), timeZone, Locale.US) + val timestampFormat: String = + parameters.getOrElse("timestampFormat", "yyyy-MM-dd'T'HH:mm:ss.SSSXXX") val multiLine = parameters.get("multiLine").map(_.toBoolean).getOrElse(false) @@ -118,6 +118,12 @@ private[sql] class JSONOptions( */ val pretty: Boolean = parameters.get("pretty").map(_.toBoolean).getOrElse(false) + /** + * Enables inferring of TimestampType from strings matched to the timestamp pattern + * defined by the timestampFormat option. + */ + val inferTimestamp: Boolean = parameters.get("inferTimestamp").map(_.toBoolean).getOrElse(true) + /** Sets config options on a Jackson [[JsonFactory]]. */ def setJacksonOptions(factory: JsonFactory): Unit = { factory.configure(JsonParser.Feature.ALLOW_COMMENTS, allowComments) 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 d02a2be8ddad6..3378040d1b640 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 @@ -23,7 +23,7 @@ import com.fasterxml.jackson.core._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters -import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, MapData} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.types._ /** @@ -77,6 +77,12 @@ private[sql] class JacksonGenerator( private val lineSeparator: String = options.lineSeparatorInWrite + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + private def makeWriter(dataType: DataType): ValueWriter = dataType match { case NullType => (row: SpecializedGetters, ordinal: Int) => @@ -116,14 +122,12 @@ private[sql] class JacksonGenerator( case TimestampType => (row: SpecializedGetters, ordinal: Int) => - val timestampString = - options.timestampFormat.format(DateTimeUtils.toJavaTimestamp(row.getLong(ordinal))) + val timestampString = timestampFormatter.format(row.getLong(ordinal)) gen.writeString(timestampString) case DateType => (row: SpecializedGetters, ordinal: Int) => - val dateString = - options.dateFormat.format(DateTimeUtils.toJavaDate(row.getInt(ordinal))) + val dateString = dateFormatter.format(row.getInt(ordinal)) gen.writeString(dateString) case BinaryType => 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 984979ac5e9b4..19bc5bf3b29e3 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 @@ -22,6 +22,7 @@ import java.nio.charset.MalformedInputException import scala.collection.mutable.ArrayBuffer import scala.util.Try +import scala.util.control.NonFatal import com.fasterxml.jackson.core._ @@ -38,7 +39,8 @@ import org.apache.spark.util.Utils */ class JacksonParser( schema: DataType, - val options: JSONOptions) extends Logging { + val options: JSONOptions, + allowArrayAsStructs: Boolean) extends Logging { import JacksonUtils._ import com.fasterxml.jackson.core.JsonToken._ @@ -53,6 +55,12 @@ class JacksonParser( private val factory = new JsonFactory() options.setJacksonOptions(factory) + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) + private val dateFormatter = DateFormatter(options.dateFormat, options.locale) + /** * Create a converter which converts the JSON documents held by the `JsonParser` * to a value according to a desired schema. This is a wrapper for the method @@ -84,7 +92,7 @@ class JacksonParser( // List([str_a_1,null]) // List([str_a_2,null], [null,str_b_3]) // - case START_ARRAY => + case START_ARRAY if allowArrayAsStructs => val array = convertArray(parser, elementConverter) // Here, as we support reading top level JSON arrays and take every element // in such an array as a row, this case is possible. @@ -93,6 +101,8 @@ class JacksonParser( } else { array.toArray[InternalRow](schema).toSeq } + case START_ARRAY => + throw new RuntimeException("Parsing JSON arrays as structs is forbidden.") } } @@ -132,6 +142,8 @@ class JacksonParser( } } + private val decimalParser = ExprUtils.getDecimalParser(options.locale) + /** * Create a converter which converts the JSON documents held by the `JsonParser` * to a value according to a desired schema. @@ -168,7 +180,7 @@ class JacksonParser( case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => parser.getFloatValue - case VALUE_STRING => + case VALUE_STRING if parser.getTextLength >= 1 => // Special case handling for NaN and Infinity. parser.getText match { case "NaN" => Float.NaN @@ -184,7 +196,7 @@ class JacksonParser( case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => parser.getDoubleValue - case VALUE_STRING => + case VALUE_STRING if parser.getTextLength >= 1 => // Special case handling for NaN and Infinity. parser.getText match { case "NaN" => Double.NaN @@ -211,18 +223,8 @@ class JacksonParser( case TimestampType => (parser: JsonParser) => parseJsonToken[java.lang.Long](parser, dataType) { - case VALUE_STRING => - val stringValue = parser.getText - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681. - Long.box { - Try(options.timestampFormat.parse(stringValue).getTime * 1000L) - .getOrElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - DateTimeUtils.stringToTime(stringValue).getTime * 1000L - } - } + case VALUE_STRING if parser.getTextLength >= 1 => + timestampFormatter.parse(parser.getText) case VALUE_NUMBER_INT => parser.getLongValue * 1000000L @@ -230,23 +232,8 @@ class JacksonParser( case DateType => (parser: JsonParser) => parseJsonToken[java.lang.Integer](parser, dataType) { - case VALUE_STRING => - val stringValue = parser.getText - // This one will lose microseconds parts. - // See https://issues.apache.org/jira/browse/SPARK-10681.x - Int.box { - Try(DateTimeUtils.millisToDays(options.dateFormat.parse(stringValue).getTime)) - .orElse { - // If it fails to parse, then tries the way used in 2.0 and 1.x for backwards - // compatibility. - Try(DateTimeUtils.millisToDays(DateTimeUtils.stringToTime(stringValue).getTime)) - } - .getOrElse { - // In Spark 1.5.0, we store the data as number of days since epoch in string. - // So, we just convert it to Int. - stringValue.toInt - } - } + case VALUE_STRING if parser.getTextLength >= 1 => + dateFormatter.parse(parser.getText) } case BinaryType => @@ -258,6 +245,9 @@ class JacksonParser( (parser: JsonParser) => parseJsonToken[Decimal](parser, dataType) { case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) => Decimal(parser.getDecimalValue, dt.precision, dt.scale) + case VALUE_STRING if parser.getTextLength >= 1 => + val bigDecimal = decimalParser(parser.getText) + Decimal(bigDecimal, dt.precision, dt.scale) } case st: StructType => @@ -310,16 +300,17 @@ class JacksonParser( } /** - * This function throws an exception for failed conversion, but returns null for empty string, - * to guard the non string types. + * This function throws an exception for failed conversion. For empty string on data types + * except for string and binary types, this also throws an exception. */ private def failedConversion[R >: Null]( parser: JsonParser, dataType: DataType): PartialFunction[JsonToken, R] = { + + // SPARK-25040: Disallow empty strings for data types except for string and binary types. case VALUE_STRING if parser.getTextLength < 1 => - // If conversion is failed, this produces `null` rather than throwing exception. - // This will protect the mismatch of types. - null + throw new RuntimeException( + s"Failed to parse an empty string for data type ${dataType.catalogString}") case token => // We cannot parse this token based on the given data type. So, we throw a @@ -337,17 +328,28 @@ class JacksonParser( schema: StructType, fieldConverters: Array[ValueConverter]): InternalRow = { val row = new GenericInternalRow(schema.length) + var badRecordException: Option[Throwable] = None + while (nextUntil(parser, JsonToken.END_OBJECT)) { schema.getFieldIndex(parser.getCurrentName) match { case Some(index) => - row.update(index, fieldConverters(index).apply(parser)) - + try { + row.update(index, fieldConverters(index).apply(parser)) + } catch { + case NonFatal(e) => + badRecordException = badRecordException.orElse(Some(e)) + parser.skipChildren() + } case None => parser.skipChildren() } } - row + if (badRecordException.isEmpty) { + row + } else { + throw PartialResultException(row, badRecordException.get) + } } /** @@ -363,6 +365,8 @@ class JacksonParser( values += fieldConverter.apply(parser) } + // The JSON map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. ArrayBasedMapData(keys.toArray, values.toArray) } @@ -416,6 +420,11 @@ class JacksonParser( val wrappedCharException = new CharConversionException(msg) wrappedCharException.initCause(e) throw BadRecordException(() => recordLiteral(record), () => None, wrappedCharException) + case PartialResultException(row, cause) => + throw BadRecordException( + record = () => recordLiteral(record), + partialResult = () => Some(row), + cause) } } } 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 9999a005106f9..c5a97c7b8835a 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 @@ -19,18 +19,28 @@ package org.apache.spark.sql.catalyst.json import java.util.Comparator +import scala.util.control.Exception.allCatch + import com.fasterxml.jackson.core._ import org.apache.spark.SparkException import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.TypeCoercion +import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.JacksonUtils.nextUntil -import org.apache.spark.sql.catalyst.util.{DropMalformedMode, FailFastMode, ParseMode, PermissiveMode} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils -private[sql] object JsonInferSchema { +private[sql] class JsonInferSchema(options: JSONOptions) extends Serializable { + + private val decimalParser = ExprUtils.getDecimalParser(options.locale) + + private val timestampFormatter = TimestampFormatter( + options.timestampFormat, + options.zoneId, + options.locale) /** * Infer the type of a collection of json records in three stages: @@ -40,21 +50,20 @@ private[sql] object JsonInferSchema { */ def infer[T]( json: RDD[T], - configOptions: JSONOptions, createParser: (JsonFactory, T) => JsonParser): StructType = { - val parseMode = configOptions.parseMode - val columnNameOfCorruptRecord = configOptions.columnNameOfCorruptRecord + val parseMode = options.parseMode + val columnNameOfCorruptRecord = options.columnNameOfCorruptRecord // In each RDD partition, perform schema inference on each row and merge afterwards. - val typeMerger = compatibleRootType(columnNameOfCorruptRecord, parseMode) + val typeMerger = JsonInferSchema.compatibleRootType(columnNameOfCorruptRecord, parseMode) val mergedTypesFromPartitions = json.mapPartitions { iter => val factory = new JsonFactory() - configOptions.setJacksonOptions(factory) + options.setJacksonOptions(factory) iter.flatMap { row => try { Utils.tryWithResource(createParser(factory, row)) { parser => parser.nextToken() - Some(inferField(parser, configOptions)) + Some(inferField(parser)) } } catch { case e @ (_: RuntimeException | _: JsonProcessingException) => parseMode match { @@ -82,7 +91,7 @@ private[sql] object JsonInferSchema { } json.sparkContext.runJob(mergedTypesFromPartitions, foldPartition, mergeResult) - canonicalizeType(rootType, configOptions) match { + canonicalizeType(rootType, options) match { case Some(st: StructType) => st case _ => // canonicalizeType erases all empty structs, including the only one we want to keep @@ -90,34 +99,17 @@ private[sql] object JsonInferSchema { } } - private[this] val structFieldComparator = new Comparator[StructField] { - override def compare(o1: StructField, o2: StructField): Int = { - o1.name.compareTo(o2.name) - } - } - - private def isSorted(arr: Array[StructField]): Boolean = { - var i: Int = 0 - while (i < arr.length - 1) { - if (structFieldComparator.compare(arr(i), arr(i + 1)) > 0) { - return false - } - i += 1 - } - true - } - /** * Infer the type of a json document from the parser's token stream */ - def inferField(parser: JsonParser, configOptions: JSONOptions): DataType = { + def inferField(parser: JsonParser): DataType = { import com.fasterxml.jackson.core.JsonToken._ parser.getCurrentToken match { case null | VALUE_NULL => NullType case FIELD_NAME => parser.nextToken() - inferField(parser, configOptions) + inferField(parser) case VALUE_STRING if parser.getTextLength < 1 => // Zero length strings and nulls have special handling to deal @@ -128,18 +120,32 @@ private[sql] object JsonInferSchema { // record fields' types have been combined. NullType - case VALUE_STRING => StringType + case VALUE_STRING => + val field = parser.getText + lazy val decimalTry = allCatch opt { + val bigDecimal = decimalParser(field) + DecimalType(bigDecimal.precision, bigDecimal.scale) + } + if (options.prefersDecimal && decimalTry.isDefined) { + decimalTry.get + } else if (options.inferTimestamp && + (allCatch opt timestampFormatter.parse(field)).isDefined) { + TimestampType + } else { + StringType + } + case START_OBJECT => val builder = Array.newBuilder[StructField] while (nextUntil(parser, END_OBJECT)) { builder += StructField( parser.getCurrentName, - inferField(parser, configOptions), + inferField(parser), nullable = true) } val fields: Array[StructField] = builder.result() // Note: other code relies on this sorting for correctness, so don't remove it! - java.util.Arrays.sort(fields, structFieldComparator) + java.util.Arrays.sort(fields, JsonInferSchema.structFieldComparator) StructType(fields) case START_ARRAY => @@ -148,15 +154,15 @@ private[sql] object JsonInferSchema { // the type as we pass through all JSON objects. var elementType: DataType = NullType while (nextUntil(parser, END_ARRAY)) { - elementType = compatibleType( - elementType, inferField(parser, configOptions)) + elementType = JsonInferSchema.compatibleType( + elementType, inferField(parser)) } ArrayType(elementType) - case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) if configOptions.primitivesAsString => StringType + case (VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT) if options.primitivesAsString => StringType - case (VALUE_TRUE | VALUE_FALSE) if configOptions.primitivesAsString => StringType + case (VALUE_TRUE | VALUE_FALSE) if options.primitivesAsString => StringType case VALUE_NUMBER_INT | VALUE_NUMBER_FLOAT => import JsonParser.NumberType._ @@ -172,7 +178,7 @@ private[sql] object JsonInferSchema { } else { DoubleType } - case FLOAT | DOUBLE if configOptions.prefersDecimal => + case FLOAT | DOUBLE if options.prefersDecimal => val v = parser.getDecimalValue if (Math.max(v.precision(), v.scale()) <= DecimalType.MAX_PRECISION) { DecimalType(Math.max(v.precision(), v.scale()), v.scale()) @@ -217,12 +223,31 @@ private[sql] object JsonInferSchema { case other => Some(other) } +} + +object JsonInferSchema { + val structFieldComparator = new Comparator[StructField] { + override def compare(o1: StructField, o2: StructField): Int = { + o1.name.compareTo(o2.name) + } + } + + def isSorted(arr: Array[StructField]): Boolean = { + var i: Int = 0 + while (i < arr.length - 1) { + if (structFieldComparator.compare(arr(i), arr(i + 1)) > 0) { + return false + } + i += 1 + } + true + } - private def withCorruptField( + def withCorruptField( struct: StructType, other: DataType, columnNameOfCorruptRecords: String, - parseMode: ParseMode) = parseMode match { + parseMode: ParseMode): StructType = parseMode match { case PermissiveMode => // If we see any other data type at the root level, we get records that cannot be // parsed. So, we use the struct as the data type and add the corrupt field to the schema. @@ -230,7 +255,7 @@ private[sql] object JsonInferSchema { // If this given struct does not have a column used for corrupt records, // add this field. val newFields: Array[StructField] = - StructField(columnNameOfCorruptRecords, StringType, nullable = true) +: struct.fields + StructField(columnNameOfCorruptRecords, StringType, nullable = true) +: struct.fields // Note: other code relies on this sorting for correctness, so don't remove it! java.util.Arrays.sort(newFields, structFieldComparator) StructType(newFields) @@ -253,7 +278,7 @@ private[sql] object JsonInferSchema { /** * Remove top-level ArrayType wrappers and merge the remaining schemas */ - private def compatibleRootType( + def compatibleRootType( columnNameOfCorruptRecords: String, parseMode: ParseMode): (DataType, DataType) => DataType = { // Since we support array of json objects at the top level, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala index 064ca68b7a628..f92d8f5b8e534 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions.{And, Attribute, AttributeSet, Expression, PredicateHelper} import org.apache.spark.sql.catalyst.plans.{Inner, InnerLike, JoinType} -import org.apache.spark.sql.catalyst.plans.logical.{BinaryNode, Join, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -42,15 +42,16 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { } else { val result = plan transformDown { // Start reordering with a joinable item, which is an InnerLike join with conditions. - case j @ Join(_, _, _: InnerLike, Some(cond)) => + // Avoid reordering if a join hint is present. + case j @ Join(_, _, _: InnerLike, Some(cond), JoinHint.NONE) => reorder(j, j.output) - case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond))) + case p @ Project(projectList, Join(_, _, _: InnerLike, Some(cond), JoinHint.NONE)) if projectList.forall(_.isInstanceOf[Attribute]) => reorder(p, p.output) } - // After reordering is finished, convert OrderedJoin back to Join - result transformDown { - case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond) + // After reordering is finished, convert OrderedJoin back to Join. + result transform { + case OrderedJoin(left, right, jt, cond) => Join(left, right, jt, cond, JoinHint.NONE) } } } @@ -76,12 +77,12 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { */ private def extractInnerJoins(plan: LogicalPlan): (Seq[LogicalPlan], Set[Expression]) = { plan match { - case Join(left, right, _: InnerLike, Some(cond)) => + case Join(left, right, _: InnerLike, Some(cond), JoinHint.NONE) => val (leftPlans, leftConditions) = extractInnerJoins(left) val (rightPlans, rightConditions) = extractInnerJoins(right) (leftPlans ++ rightPlans, splitConjunctivePredicates(cond).toSet ++ leftConditions ++ rightConditions) - case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond))) + case Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), JoinHint.NONE)) if projectList.forall(_.isInstanceOf[Attribute]) => extractInnerJoins(j) case _ => @@ -90,11 +91,11 @@ object CostBasedJoinReorder extends Rule[LogicalPlan] with PredicateHelper { } private def replaceWithOrderedJoin(plan: LogicalPlan): LogicalPlan = plan match { - case j @ Join(left, right, jt: InnerLike, Some(cond)) => + case j @ Join(left, right, jt: InnerLike, Some(cond), JoinHint.NONE) => val replacedLeft = replaceWithOrderedJoin(left) val replacedRight = replaceWithOrderedJoin(right) OrderedJoin(replacedLeft, replacedRight, jt, Some(cond)) - case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond))) => + case p @ Project(projectList, j @ Join(_, _, _: InnerLike, Some(cond), JoinHint.NONE)) => p.copy(child = replaceWithOrderedJoin(j)) case _ => plan @@ -175,11 +176,20 @@ object JoinReorderDP extends PredicateHelper with Logging { assert(topOutputSet == p.outputSet) // Keep the same order of final output attributes. p.copy(projectList = output) + case finalPlan if !sameOutput(finalPlan, output) => + Project(output, finalPlan) case finalPlan => finalPlan } } + private def sameOutput(plan: LogicalPlan, expectedOutput: Seq[Attribute]): Boolean = { + val thisOutput = plan.output + thisOutput.length == expectedOutput.length && thisOutput.zip(expectedOutput).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } + /** Find all possible plans at the next level, based on existing levels. */ private def searchLevel( existingLevels: Seq[JoinPlanMap], @@ -285,7 +295,7 @@ object JoinReorderDP extends PredicateHelper with Logging { } else { (otherPlan, onePlan) } - val newJoin = Join(left, right, Inner, joinConds.reduceOption(And)) + val newJoin = Join(left, right, Inner, joinConds.reduceOption(And), JoinHint.NONE) val collectedJoinConds = joinConds ++ oneJoinPlan.joinConds ++ otherJoinPlan.joinConds val remainingConds = conditions -- collectedJoinConds val neededAttr = AttributeSet(remainingConds.flatMap(_.references)) ++ topOutput diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala new file mode 100644 index 0000000000000..aa0ce5d4ed31a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf + +/** + * Replaces [[ResolvedHint]] operators from the plan. Move the [[HintInfo]] to associated [[Join]] + * operators, otherwise remove it if no [[Join]] operator is matched. + */ +object EliminateResolvedHint extends Rule[LogicalPlan] { + + private val hintErrorHandler = SQLConf.get.hintErrorHandler + + // This is also called in the beginning of the optimization phase, and as a result + // is using transformUp rather than resolveOperators. + def apply(plan: LogicalPlan): LogicalPlan = { + val pulledUp = plan transformUp { + case j: Join if j.hint == JoinHint.NONE => + val (newLeft, leftHints) = extractHintsFromPlan(j.left) + val (newRight, rightHints) = extractHintsFromPlan(j.right) + val newJoinHint = JoinHint(mergeHints(leftHints), mergeHints(rightHints)) + j.copy(left = newLeft, right = newRight, hint = newJoinHint) + } + pulledUp.transformUp { + case h: ResolvedHint => + hintErrorHandler.joinNotFoundForJoinHint(h.hints) + h.child + } + } + + /** + * Combine a list of [[HintInfo]]s into one [[HintInfo]]. + */ + private def mergeHints(hints: Seq[HintInfo]): Option[HintInfo] = { + hints.reduceOption((h1, h2) => h1.merge(h2, hintErrorHandler)) + } + + /** + * Extract all hints from the plan, returning a list of extracted hints and the transformed plan + * with [[ResolvedHint]] nodes removed. The returned hint list comes in top-down order. + * Note that hints can only be extracted from under certain nodes. Those that cannot be extracted + * in this method will be cleaned up later by this rule, and may emit warnings depending on the + * configurations. + */ + private[sql] def extractHintsFromPlan(plan: LogicalPlan): (LogicalPlan, Seq[HintInfo]) = { + plan match { + case h: ResolvedHint => + val (plan, hints) = extractHintsFromPlan(h.child) + (plan, h.hints +: hints) + case u: UnaryNode => + val (plan, hints) = extractHintsFromPlan(u.child) + (u.withNewChildren(Seq(plan)), hints) + // TODO revisit this logic: + // except and intersect are semi/anti-joins which won't return more data then + // their left argument, so the broadcast hint should be propagated here + case i: Intersect => + val (plan, hints) = extractHintsFromPlan(i.left) + (i.copy(left = plan), hints) + case e: Except => + val (plan, hints) = extractHintsFromPlan(e.left) + (e.copy(left = plan), hints) + case p: LogicalPlan => (p, Seq.empty) + } + } +} 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 new file mode 100644 index 0000000000000..43a6006f9b5c0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasing.scala @@ -0,0 +1,169 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * This aims to handle a nested column aliasing pattern inside the `ColumnPruning` optimizer rule. + * If a project or its child references to nested fields, and not all the fields + * in a nested attribute are used, we can substitute them by alias attributes; then a project + * of the nested fields as aliases on the children of the child will be created. + */ +object NestedColumnAliasing { + + def unapply(plan: LogicalPlan) + : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = plan match { + case Project(projectList, child) + if SQLConf.get.nestedSchemaPruningEnabled && canProjectPushThrough(child) => + getAliasSubMap(projectList) + case _ => None + } + + /** + * Replace nested columns to prune unused nested columns later. + */ + def replaceToAliases( + plan: LogicalPlan, + nestedFieldToAlias: Map[ExtractValue, Alias], + attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = plan match { + case Project(projectList, child) => + Project( + getNewProjectList(projectList, nestedFieldToAlias), + replaceChildrenWithAliases(child, attrToAliases)) + } + + /** + * Return a replaced project list. + */ + def getNewProjectList( + projectList: Seq[NamedExpression], + nestedFieldToAlias: Map[ExtractValue, Alias]): Seq[NamedExpression] = { + projectList.map(_.transform { + case f: ExtractValue if nestedFieldToAlias.contains(f) => + nestedFieldToAlias(f).toAttribute + }.asInstanceOf[NamedExpression]) + } + + /** + * Return a plan with new children replaced with aliases. + */ + def replaceChildrenWithAliases( + plan: LogicalPlan, + attrToAliases: Map[ExprId, Seq[Alias]]): LogicalPlan = { + plan.withNewChildren(plan.children.map { plan => + Project(plan.output.flatMap(a => attrToAliases.getOrElse(a.exprId, Seq(a))), plan) + }) + } + + /** + * Returns true for those operators that project can be pushed through. + */ + private def canProjectPushThrough(plan: LogicalPlan) = plan match { + case _: GlobalLimit => true + case _: LocalLimit => true + case _: Repartition => true + case _: Sample => true + case _ => false + } + + /** + * Return root references that are individually accessed as a whole, and `GetStructField`s + * or `GetArrayStructField`s which on top of other `ExtractValue`s or special expressions. + * Check `SelectedField` to see which expressions should be listed here. + */ + private def collectRootReferenceAndExtractValue(e: Expression): Seq[Expression] = e match { + case _: AttributeReference => Seq(e) + case GetStructField(_: ExtractValue | _: AttributeReference, _, _) => Seq(e) + case GetArrayStructFields(_: MapValues | + _: MapKeys | + _: ExtractValue | + _: AttributeReference, _, _, _, _) => Seq(e) + case es if es.children.nonEmpty => es.children.flatMap(collectRootReferenceAndExtractValue) + case _ => Seq.empty + } + + /** + * Return two maps in order to replace nested fields to aliases. + * + * 1. ExtractValue -> Alias: A new alias is created for each nested field. + * 2. ExprId -> Seq[Alias]: A reference attribute has multiple aliases pointing it. + */ + def getAliasSubMap(exprList: Seq[Expression]) + : Option[(Map[ExtractValue, Alias], Map[ExprId, Seq[Alias]])] = { + val (nestedFieldReferences, otherRootReferences) = + exprList.flatMap(collectRootReferenceAndExtractValue).partition { + case _: ExtractValue => true + case _ => false + } + + val aliasSub = nestedFieldReferences.asInstanceOf[Seq[ExtractValue]] + .filter(!_.references.subsetOf(AttributeSet(otherRootReferences))) + .groupBy(_.references.head) + .flatMap { case (attr, nestedFields: Seq[ExtractValue]) => + // Each expression can contain multiple nested fields. + // Note that we keep the original names to deliver to parquet in a case-sensitive way. + val nestedFieldToAlias = nestedFields.distinct.map { f => + val exprId = NamedExpression.newExprId + (f, Alias(f, s"_gen_alias_${exprId.id}")(exprId, Seq.empty, None)) + } + + // If all nested fields of `attr` are used, we don't need to introduce new aliases. + // By default, ColumnPruning rule uses `attr` already. + if (nestedFieldToAlias.nonEmpty && + nestedFieldToAlias.length < totalFieldNum(attr.dataType)) { + Some(attr.exprId -> nestedFieldToAlias) + } else { + None + } + } + + if (aliasSub.isEmpty) { + None + } else { + Some((aliasSub.values.flatten.toMap, aliasSub.map(x => (x._1, x._2.map(_._2))))) + } + } + + /** + * Return total number of fields of this type. This is used as a threshold to use nested column + * pruning. It's okay to underestimate. If the number of reference is bigger than this, the parent + * reference is used instead of nested field references. + */ + private def totalFieldNum(dataType: DataType): Int = dataType match { + case _: AtomicType => 1 + case StructType(fields) => fields.map(f => totalFieldNum(f.dataType)).sum + case ArrayType(elementType, _) => totalFieldNum(elementType) + case MapType(keyType, valueType, _) => totalFieldNum(keyType) + totalFieldNum(valueType) + case _ => 1 // UDT and others + } + + /** + * This is a while-list for pruning nested fields at `Generator`. + */ + def canPruneGenerator(g: Generator): Boolean = g match { + case _: Explode => true + case _: Stack => true + case _: PosExplode => true + case _: Inline => true + case _ => false + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala new file mode 100644 index 0000000000000..b036092cf1fcc --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingNumbers.scala @@ -0,0 +1,208 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.expressions.{Alias, And, ArrayTransform, CreateArray, CreateMap, CreateNamedStruct, CreateNamedStructUnsafe, CreateStruct, EqualTo, ExpectsInputTypes, Expression, GetStructField, KnownFloatingPointNormalized, LambdaFunction, NamedLambdaVariable, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery, Window} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.types._ + +/** + * We need to take care of special floating numbers (NaN and -0.0) in several places: + * 1. When compare values, different NaNs should be treated as same, `-0.0` and `0.0` should be + * treated as same. + * 2. In aggregate grouping keys, different NaNs should belong to the same group, -0.0 and 0.0 + * should belong to the same group. + * 3. In join keys, different NaNs should be treated as same, `-0.0` and `0.0` should be + * treated as same. + * 4. In window partition keys, different NaNs should belong to the same partition, -0.0 and 0.0 + * should belong to the same partition. + * + * Case 1 is fine, as we handle NaN and -0.0 well during comparison. For complex types, we + * recursively compare the fields/elements, so it's also fine. + * + * Case 2, 3 and 4 are problematic, as Spark SQL turns grouping/join/window partition keys into + * binary `UnsafeRow` and compare the binary data directly. Different NaNs have different binary + * representation, and the same thing happens for -0.0 and 0.0. + * + * This rule normalizes NaN and -0.0 in window partition keys, join keys and aggregate grouping + * keys. + * + * Ideally we should do the normalization in the physical operators that compare the + * binary `UnsafeRow` directly. We don't need this normalization if the Spark SQL execution engine + * is not optimized to run on binary data. This rule is created to simplify the implementation, so + * that we have a single place to do normalization, which is more maintainable. + * + * Note that, this rule must be executed at the end of optimizer, because the optimizer may create + * new joins(the subquery rewrite) and new join conditions(the join reorder). + */ +object NormalizeFloatingNumbers extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = plan match { + // A subquery will be rewritten into join later, and will go through this rule + // eventually. Here we skip subquery, as we only need to run this rule once. + case _: Subquery => plan + + case _ => plan transform { + case w: Window if w.partitionSpec.exists(p => needNormalize(p)) => + // Although the `windowExpressions` may refer to `partitionSpec` expressions, we don't need + // to normalize the `windowExpressions`, as they are executed per input row and should take + // the input row as it is. + w.copy(partitionSpec = w.partitionSpec.map(normalize)) + + // Only hash join and sort merge join need the normalization. Here we catch all Joins with + // join keys, assuming Joins with join keys are always planned as hash join or sort merge + // join. It's very unlikely that we will break this assumption in the near future. + case j @ ExtractEquiJoinKeys(_, leftKeys, rightKeys, condition, _, _, _) + // The analyzer guarantees left and right joins keys are of the same data type. Here we + // only need to check join keys of one side. + if leftKeys.exists(k => needNormalize(k)) => + val newLeftJoinKeys = leftKeys.map(normalize) + val newRightJoinKeys = rightKeys.map(normalize) + val newConditions = newLeftJoinKeys.zip(newRightJoinKeys).map { + case (l, r) => EqualTo(l, r) + } ++ condition + j.copy(condition = Some(newConditions.reduce(And))) + + // TODO: ideally Aggregate should also be handled here, but its grouping expressions are + // mixed in its aggregate expressions. It's unreliable to change the grouping expressions + // here. For now we normalize grouping expressions in `AggUtils` during planning. + } + } + + /** + * Short circuit if the underlying expression is already normalized + */ + private def needNormalize(expr: Expression): Boolean = expr match { + case KnownFloatingPointNormalized(_) => false + case _ => needNormalize(expr.dataType) + } + + private def needNormalize(dt: DataType): Boolean = dt match { + case FloatType | DoubleType => true + case StructType(fields) => fields.exists(f => needNormalize(f.dataType)) + case ArrayType(et, _) => needNormalize(et) + // Currently MapType is not comparable and analyzer should fail earlier if this case happens. + case _: MapType => + throw new IllegalStateException("grouping/join/window partition keys cannot be map type.") + case _ => false + } + + private[sql] def normalize(expr: Expression): Expression = expr match { + case _ if !needNormalize(expr) => expr + + case a: Alias => + a.withNewChildren(Seq(normalize(a.child))) + + case CreateNamedStruct(children) => + CreateNamedStruct(children.map(normalize)) + + case CreateNamedStructUnsafe(children) => + CreateNamedStructUnsafe(children.map(normalize)) + + case CreateArray(children) => + CreateArray(children.map(normalize)) + + case CreateMap(children) => + CreateMap(children.map(normalize)) + + case _ if expr.dataType == FloatType || expr.dataType == DoubleType => + KnownFloatingPointNormalized(NormalizeNaNAndZero(expr)) + + case _ if expr.dataType.isInstanceOf[StructType] => + val fields = expr.dataType.asInstanceOf[StructType].fields.indices.map { i => + normalize(GetStructField(expr, i)) + } + CreateStruct(fields) + + case _ if expr.dataType.isInstanceOf[ArrayType] => + val ArrayType(et, containsNull) = expr.dataType + val lv = NamedLambdaVariable("arg", et, containsNull) + val function = normalize(lv) + KnownFloatingPointNormalized(ArrayTransform(expr, LambdaFunction(function, Seq(lv)))) + + case _ => throw new IllegalStateException(s"fail to normalize $expr") + } +} + +case class NormalizeNaNAndZero(child: Expression) extends UnaryExpression with ExpectsInputTypes { + + override def dataType: DataType = child.dataType + + override def inputTypes: Seq[AbstractDataType] = Seq(TypeCollection(FloatType, DoubleType)) + + private lazy val normalizer: Any => Any = child.dataType match { + case FloatType => (input: Any) => { + val f = input.asInstanceOf[Float] + if (f.isNaN) { + Float.NaN + } else if (f == -0.0f) { + 0.0f + } else { + f + } + } + + case DoubleType => (input: Any) => { + val d = input.asInstanceOf[Double] + if (d.isNaN) { + Double.NaN + } else if (d == -0.0d) { + 0.0d + } else { + d + } + } + } + + override def nullSafeEval(input: Any): Any = { + normalizer(input) + } + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + val codeToNormalize = child.dataType match { + case FloatType => (f: String) => { + s""" + |if (Float.isNaN($f)) { + | ${ev.value} = Float.NaN; + |} else if ($f == -0.0f) { + | ${ev.value} = 0.0f; + |} else { + | ${ev.value} = $f; + |} + """.stripMargin + } + + case DoubleType => (d: String) => { + s""" + |if (Double.isNaN($d)) { + | ${ev.value} = Double.NaN; + |} else if ($d == -0.0d) { + | ${ev.value} = 0.0d; + |} else { + | ${ev.value} = $d; + |} + """.stripMargin + } + } + + nullSafeCodeGen(ctx, ev, codeToNormalize) + } +} 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 da8009d50b5ec..206d09a6f79e9 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 @@ -38,10 +38,13 @@ import org.apache.spark.util.Utils abstract class Optimizer(sessionCatalog: SessionCatalog) extends RuleExecutor[LogicalPlan] { - // Check for structural integrity of the plan in test mode. Currently we only check if a plan is - // still resolved after the execution of each rule. + // Check for structural integrity of the plan in test mode. + // Currently we check after the execution of each rule if a plan: + // - is still resolved + // - only host special expressions in supported operators override protected def isPlanIntegral(plan: LogicalPlan): Boolean = { - !Utils.isTesting || plan.resolved + !Utils.isTesting || (plan.resolved && + plan.find(PlanHelper.specialExpressionsInUnsupportedOperator(_).nonEmpty).isEmpty) } protected def fixedPoint = FixedPoint(SQLConf.get.optimizerMaxIterations) @@ -60,8 +63,9 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) PushProjectionThroughUnion, ReorderJoin, EliminateOuterJoin, - PushPredicateThroughJoin, - PushDownPredicate, + PushDownPredicates, + PushDownLeftSemiAntiJoin, + PushLeftSemiLeftAntiThroughJoin, LimitPushDown, ColumnPruning, InferFiltersFromConstraints, @@ -73,6 +77,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) CombineLimits, CombineUnions, // Constant folding and strength reduction + TransposeWindow, NullPropagation, ConstantPropagation, FoldablePropagation, @@ -84,6 +89,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) SimplifyConditionals, RemoveDispensableExpressions, SimplifyBinaryComparison, + ReplaceNullWithFalseInPredicate, PruneFilters, EliminateSorts, SimplifyCasts, @@ -91,7 +97,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) RewriteCorrelatedScalarSubquery, EliminateSerialization, RemoveRedundantAliases, - RemoveRedundantProject, + RemoveNoopOperators, SimplifyExtractValueOps, CombineConcats) ++ extendedOperatorOptimizationRules @@ -113,6 +119,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) // However, because we also use the analyzer to canonicalized queries (for view definition), // we do not eliminate subqueries or compute current time in the analyzer. Batch("Finish Analysis", Once, + EliminateResolvedHint, EliminateSubqueryAliases, EliminateView, ReplaceExpressions, @@ -130,11 +137,13 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) // since the other rules might make two separate Unions operators adjacent. Batch("Union", Once, CombineUnions) :: - // run this once earlier. this might simplify the plan and reduce cost of optimizer. - // for example, a query such as Filter(LocalRelation) would go through all the heavy + Batch("OptimizeLimitZero", Once, + OptimizeLimitZero) :: + // Run this once earlier. This might simplify the plan and reduce cost of optimizer. + // For example, a query such as Filter(LocalRelation) would go through all the heavy // optimizer rules that are triggered when there is a filter - // (e.g. InferFiltersFromConstraints). if we run this batch earlier, the query becomes just - // LocalRelation and does not trigger many rules + // (e.g. InferFiltersFromConstraints). If we run this batch earlier, the query becomes just + // LocalRelation and does not trigger many rules. Batch("LocalRelation early", fixedPoint, ConvertToLocalRelation, PropagateEmptyRelation) :: @@ -161,7 +170,9 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) DecimalAggregates) :+ Batch("Object Expressions Optimization", fixedPoint, EliminateMapObjects, - CombineTypedFilters) :+ + CombineTypedFilters, + ObjectSerializerPruning, + ReassignLambdaVariableID) :+ Batch("LocalRelation", fixedPoint, ConvertToLocalRelation, PropagateEmptyRelation) :+ @@ -175,9 +186,9 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) RewritePredicateSubquery, ColumnPruning, CollapseProject, - RemoveRedundantProject) :+ - Batch("UpdateAttributeReferences", Once, - UpdateNullabilityInAttributeReferences) + RemoveNoopOperators) :+ + // This batch must be executed after the `RewriteSubquery` batch, which creates joins. + Batch("NormalizeFloatingNumbers", Once, NormalizeFloatingNumbers) } /** @@ -190,6 +201,7 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) */ def nonExcludableRules: Seq[String] = EliminateDistinct.ruleName :: + EliminateResolvedHint.ruleName :: EliminateSubqueryAliases.ruleName :: EliminateView.ruleName :: ReplaceExpressions.ruleName :: @@ -206,7 +218,8 @@ abstract class Optimizer(sessionCatalog: SessionCatalog) PullupCorrelatedPredicates.ruleName :: RewriteCorrelatedScalarSubquery.ruleName :: RewritePredicateSubquery.ruleName :: - PullOutPythonUDFInJoinCondition.ruleName :: Nil + PullOutPythonUDFInJoinCondition.ruleName :: + NormalizeFloatingNumbers.ruleName :: Nil /** * Optimize all the subqueries inside expression. @@ -354,7 +367,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { // not allowed to use the same attributes. We use a blacklist to prevent us from creating a // situation in which this happens; the rule will only remove an alias if its child // attribute is not on the black list. - case Join(left, right, joinType, condition) => + case Join(left, right, joinType, condition, hint) => val newLeft = removeRedundantAliases(left, blacklist ++ right.outputSet) val newRight = removeRedundantAliases(right, blacklist ++ newLeft.outputSet) val mapping = AttributeMap( @@ -363,7 +376,7 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { val newCondition = condition.map(_.transform { case a: Attribute => mapping.getOrElse(a, a) }) - Join(newLeft, newRight, joinType, newCondition) + Join(newLeft, newRight, joinType, newCondition, hint) case _ => // Remove redundant aliases in the subtree(s). @@ -401,11 +414,15 @@ object RemoveRedundantAliases extends Rule[LogicalPlan] { } /** - * Remove projections from the query plan that do not make any modifications. + * Remove no-op operators from the query plan that do not make any modifications. */ -object RemoveRedundantProject extends Rule[LogicalPlan] { +object RemoveNoopOperators extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case p @ Project(_, child) if p.output == child.output => child + // Eliminate no-op Projects + case p @ Project(_, child) if child.sameOutput(p) => child + + // Eliminate no-op Window + case w: Window if w.windowExpressions.isEmpty => w.child } } @@ -454,7 +471,7 @@ object LimitPushDown extends Rule[LogicalPlan] { // on both sides if it is applied multiple times. Therefore: // - If one side is already limited, stack another limit on top if the new limit is smaller. // The redundant limit will be collapsed by the CombineLimits rule. - case LocalLimit(exp, join @ Join(left, right, joinType, _)) => + case LocalLimit(exp, join @ Join(left, right, joinType, _, _)) => val newJoin = joinType match { case RightOuter => join.copy(right = maybePushLocalLimit(exp, right)) case LeftOuter => join.copy(left = maybePushLocalLimit(exp, left)) @@ -470,7 +487,7 @@ object LimitPushDown extends Rule[LogicalPlan] { * Union: * Right now, Union means UNION ALL, which does not de-duplicate rows. So, it is * safe to pushdown Filters and Projections through it. Filter pushdown is handled by another - * rule PushDownPredicate. Once we add UNION DISTINCT, we will not be able to pushdown Projections. + * rule PushDownPredicates. Once we add UNION DISTINCT, we will not be able to pushdown Projections. */ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper { @@ -530,9 +547,6 @@ object PushProjectionThroughUnion extends Rule[LogicalPlan] with PredicateHelper * p2 is usually inserted by this rule and useless, p1 could prune the columns anyway. */ object ColumnPruning extends Rule[LogicalPlan] { - private def sameOutput(output1: Seq[Attribute], output2: Seq[Attribute]): Boolean = - output1.size == output2.size && - output1.zip(output2).forall(pair => pair._1.semanticEquals(pair._2)) def apply(plan: LogicalPlan): LogicalPlan = removeProjectBeforeFilter(plan transform { // Prunes the unused columns from project list of Project/Aggregate/Expand @@ -574,8 +588,26 @@ 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) => + NestedColumnAliasing.getAliasSubMap(projectList ++ g.generator.children).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) + // Eliminate unneeded attributes from right side of a Left Existence Join. - case j @ Join(_, right, LeftExistence(_), _) => + case j @ Join(_, right, LeftExistence(_), _, _) => j.copy(right = prunedChild(right, j.references)) // all the columns will be used to compare, so we can't prune them @@ -603,17 +635,15 @@ object ColumnPruning extends Rule[LogicalPlan] { p.copy(child = w.copy( windowExpressions = w.windowExpressions.filter(p.references.contains))) - // Eliminate no-op Window - case w: Window if w.windowExpressions.isEmpty => w.child - - // Eliminate no-op Projects - case p @ Project(_, child) if sameOutput(child.output, p.output) => child - // Can't prune the columns on LeafNode case p @ Project(_, _: LeafNode) => p + case p @ NestedColumnAliasing(nestedFieldToAlias, attrToAliases) => + NestedColumnAliasing.replaceToAliases(p, nestedFieldToAlias, attrToAliases) + // for all other logical plans that inherits the output from it's children - case p @ Project(_, child) => + // Project over project is handled by the first case, skip it here. + case p @ Project(_, child) if !child.isInstanceOf[Project] => val required = child.references ++ p.references if (!child.inputSet.subsetOf(required)) { val newChildren = child.children.map(c => prunedChild(c, required)) @@ -644,8 +674,12 @@ object ColumnPruning extends Rule[LogicalPlan] { } /** - * Combines two adjacent [[Project]] operators into one and perform alias substitution, - * merging the expressions into one single expression. + * Combines two [[Project]] operators into one and perform alias substitution, + * merging the expressions into one single expression for the following cases. + * 1. When two [[Project]] operators are adjacent. + * 2. When two [[Project]] operators have LocalLimit/Sample/Repartition operator between them + * and the upper project consists of the same number of columns which is equal or aliasing. + * `GlobalLimit(LocalLimit)` pattern is also considered. */ object CollapseProject extends Rule[LogicalPlan] { @@ -663,6 +697,17 @@ object CollapseProject extends Rule[LogicalPlan] { agg.copy(aggregateExpressions = buildCleanedProjectList( p.projectList, agg.aggregateExpressions)) } + case Project(l1, g @ GlobalLimit(_, limit @ LocalLimit(_, p2 @ Project(l2, _)))) + if isRenaming(l1, l2) => + val newProjectList = buildCleanedProjectList(l1, l2) + g.copy(child = limit.copy(child = p2.copy(projectList = newProjectList))) + case Project(l1, limit @ LocalLimit(_, p2 @ Project(l2, _))) if isRenaming(l1, l2) => + val newProjectList = buildCleanedProjectList(l1, l2) + limit.copy(child = p2.copy(projectList = newProjectList)) + case Project(l1, r @ Repartition(_, _, p @ Project(l2, _))) if isRenaming(l1, l2) => + r.copy(child = p.copy(projectList = buildCleanedProjectList(l1, p.projectList))) + case Project(l1, s @ Sample(_, _, _, _, p2 @ Project(l2, _))) if isRenaming(l1, l2) => + s.copy(child = p2.copy(projectList = buildCleanedProjectList(l1, p2.projectList))) } private def collectAliases(projectList: Seq[NamedExpression]): AttributeMap[Alias] = { @@ -703,6 +748,14 @@ object CollapseProject extends Rule[LogicalPlan] { CleanupAliases.trimNonTopLevelAliases(p).asInstanceOf[NamedExpression] } } + + private def isRenaming(list1: Seq[NamedExpression], list2: Seq[NamedExpression]): Boolean = { + list1.length == list2.length && list1.zip(list2).forall { + case (e1, e2) if e1.semanticEquals(e2) => true + case (Alias(a: Attribute, _), b) if a.metadata == Metadata.empty && a.name == b.name => true + case _ => false + } + } } /** @@ -735,6 +788,7 @@ object CollapseWindow extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { case w1 @ Window(we1, ps1, os1, w2 @ Window(we2, ps2, os2, grandChild)) if ps1 == ps2 && os1 == os2 && w1.references.intersect(w2.windowOutputSet).isEmpty && + we1.nonEmpty && we2.nonEmpty && // This assumes Window contains the same type of window expressions. This is ensured // by ExtractWindowFunctions. WindowFunctionType.functionType(we1.head) == WindowFunctionType.functionType(we2.head) => @@ -794,7 +848,7 @@ object InferFiltersFromConstraints extends Rule[LogicalPlan] filter } - case join @ Join(left, right, joinType, conditionOpt) => + case join @ Join(left, right, joinType, conditionOpt, _) => joinType match { // For inner join, we can infer additional filters for both sides. LeftSemi is kind of an // inner join, it just drops the right side in the final output. @@ -874,7 +928,9 @@ object CombineUnions extends Rule[LogicalPlan] { * one conjunctive predicate. */ object CombineFilters extends Rule[LogicalPlan] with PredicateHelper { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally + + val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { // The query execution/optimization does not guarantee the expressions are evaluated in order. // We only can combine them if and only if both are deterministic. case Filter(fc, nf @ Filter(nc, grandChild)) if fc.deterministic && nc.deterministic => @@ -921,7 +977,6 @@ object RemoveRedundantSorts extends Rule[LogicalPlan] { def canEliminateSort(plan: LogicalPlan): Boolean = plan match { case p: Project => p.projectList.forall(_.deterministic) case f: Filter => f.condition.deterministic - case _: ResolvedHint => true case _ => false } } @@ -960,6 +1015,19 @@ object PruneFilters extends Rule[LogicalPlan] with PredicateHelper { } } +/** + * The unified version for predicate pushdown of normal operators and joins. + * This rule improves performance of predicate pushdown for cascading joins such as: + * Filter-Join-Join-Join. Most predicates can be pushed down in a single pass. + */ +object PushDownPredicates extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + CombineFilters.applyLocally + .orElse(PushPredicateThroughNonJoin.applyLocally) + .orElse(PushPredicateThroughJoin.applyLocally) + } +} + /** * Pushes [[Filter]] operators through many operators iff: * 1) the operator is deterministic @@ -967,8 +1035,10 @@ object PruneFilters extends Rule[LogicalPlan] with PredicateHelper { * * This heuristic is valid assuming the expression evaluation cost is minimal. */ -object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { - def apply(plan: LogicalPlan): LogicalPlan = plan transform { +object PushPredicateThroughNonJoin extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally + + val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { // SPARK-13473: We can't push the predicate down when the underlying projection output non- // deterministic field(s). Non-deterministic expressions are essentially stateful. This // implies that, for a given input row, the output are determined by the expression's initial @@ -977,24 +1047,13 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { // This also applies to Aggregate. case Filter(condition, project @ Project(fields, grandChild)) if fields.forall(_.deterministic) && canPushThroughCondition(grandChild, condition) => - - // Create a map of Aliases to their values from the child projection. - // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). - val aliasMap = AttributeMap(fields.collect { - case a: Alias => (a.toAttribute, a.child) - }) - + val aliasMap = getAliasMap(project) project.copy(child = Filter(replaceAlias(condition, aliasMap), grandChild)) case filter @ Filter(condition, aggregate: Aggregate) if aggregate.aggregateExpressions.forall(_.deterministic) && aggregate.groupingExpressions.nonEmpty => - // 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 - val aliasMap = AttributeMap(aggregate.aggregateExpressions.collect { - case a: Alias if a.child.find(_.isInstanceOf[AggregateExpression]).isEmpty => - (a.toAttribute, a.child) - }) + val aliasMap = getAliasMap(aggregate) // For each filter, expand the alias and check if the filter can be evaluated using // attributes produced by the aggregate operator's child operator. @@ -1092,11 +1151,26 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { } } - private def canPushThrough(p: UnaryNode): Boolean = p match { + def getAliasMap(plan: Project): AttributeMap[Expression] = { + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT a + b AS c, d ...' produces Map(c -> a + b). + AttributeMap(plan.projectList.collect { case a: Alias => (a.toAttribute, a.child) }) + } + + 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 + val aliasMap = plan.aggregateExpressions.collect { + case a: Alias if a.child.find(_.isInstanceOf[AggregateExpression]).isEmpty => + (a.toAttribute, a.child) + } + AttributeMap(aliasMap) + } + + def canPushThrough(p: UnaryNode): Boolean = p match { // Note that some operators (e.g. project, aggregate, union) are being handled separately // (earlier in this rule). case _: AppendColumns => true - case _: ResolvedHint => true case _: Distinct => true case _: Generate => true case _: Pivot => true @@ -1104,6 +1178,8 @@ object PushDownPredicate extends Rule[LogicalPlan] with PredicateHelper { case _: Repartition => true case _: ScriptTransformation => true case _: Sort => true + case _: BatchEvalPython => true + case _: ArrowEvalPython => true case _ => false } @@ -1179,9 +1255,11 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { (leftEvaluateCondition, rightEvaluateCondition, commonCondition ++ nonDeterministic) } - def apply(plan: LogicalPlan): LogicalPlan = plan transform { + def apply(plan: LogicalPlan): LogicalPlan = plan transform applyLocally + + val applyLocally: PartialFunction[LogicalPlan, LogicalPlan] = { // push the where condition down into join filter - case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition)) => + case f @ Filter(filterCondition, Join(left, right, joinType, joinCondition, hint)) => val (leftFilterConditions, rightFilterConditions, commonFilterCondition) = split(splitConjunctivePredicates(filterCondition), left, right) joinType match { @@ -1195,7 +1273,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { commonFilterCondition.partition(canEvaluateWithinJoin) val newJoinCond = (newJoinConditions ++ joinCondition).reduceLeftOption(And) - val join = Join(newLeft, newRight, joinType, newJoinCond) + val join = Join(newLeft, newRight, joinType, newJoinCond, hint) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) } else { @@ -1207,7 +1285,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = rightFilterConditions. reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = joinCondition - val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond) + val newJoin = Join(newLeft, newRight, RightOuter, newJoinCond, hint) (leftFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) @@ -1217,7 +1295,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, left)).getOrElse(left) val newRight = right val newJoinCond = joinCondition - val newJoin = Join(newLeft, newRight, joinType, newJoinCond) + val newJoin = Join(newLeft, newRight, joinType, newJoinCond, hint) (rightFilterConditions ++ commonFilterCondition). reduceLeftOption(And).map(Filter(_, newJoin)).getOrElse(newJoin) @@ -1227,7 +1305,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { } // push down the join filter into sub query scanning if applicable - case j @ Join(left, right, joinType, joinCondition) => + case j @ Join(left, right, joinType, joinCondition, hint) => val (leftJoinConditions, rightJoinConditions, commonJoinCondition) = split(joinCondition.map(splitConjunctivePredicates).getOrElse(Nil), left, right) @@ -1240,7 +1318,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = commonJoinCondition.reduceLeftOption(And) - Join(newLeft, newRight, joinType, newJoinCond) + Join(newLeft, newRight, joinType, newJoinCond, hint) case RightOuter => // push down the left side only join filter for left side sub query val newLeft = leftJoinConditions. @@ -1248,7 +1326,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { val newRight = right val newJoinCond = (rightJoinConditions ++ commonJoinCondition).reduceLeftOption(And) - Join(newLeft, newRight, RightOuter, newJoinCond) + Join(newLeft, newRight, RightOuter, newJoinCond, hint) case LeftOuter | LeftAnti | ExistenceJoin(_) => // push down the right side only join filter for right sub query val newLeft = left @@ -1256,7 +1334,7 @@ object PushPredicateThroughJoin extends Rule[LogicalPlan] with PredicateHelper { reduceLeftOption(And).map(Filter(_, right)).getOrElse(right) val newJoinCond = (leftJoinConditions ++ commonJoinCondition).reduceLeftOption(And) - Join(newLeft, newRight, joinType, newJoinCond) + Join(newLeft, newRight, joinType, newJoinCond, hint) case FullOuter => j case NaturalJoin(_) => sys.error("Untransformed NaturalJoin node") case UsingJoin(_, _) => sys.error("Untransformed Using join node") @@ -1312,7 +1390,7 @@ object CheckCartesianProducts extends Rule[LogicalPlan] with PredicateHelper { if (SQLConf.get.crossJoinEnabled) { plan } else plan transform { - case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _) + case j @ Join(left, right, Inner | LeftOuter | RightOuter | FullOuter, _, _) if isCartesianProduct(j) => throw new AnalysisException( s"""Detected implicit cartesian product for ${j.joinType.sql} join between logical plans @@ -1372,18 +1450,16 @@ object DecimalAggregates extends Rule[LogicalPlan] { } /** - * Converts local operations (i.e. ones that don't require data exchange) on LocalRelation to - * another LocalRelation. - * - * This is relatively simple as it currently handles only 2 single case: Project and Limit. + * Converts local operations (i.e. ones that don't require data exchange) on `LocalRelation` to + * another `LocalRelation`. */ object ConvertToLocalRelation extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case Project(projectList, LocalRelation(output, data, isStreaming)) if !projectList.exists(hasUnevaluableExpr) => - val projection = new InterpretedProjection(projectList, output) + val projection = new InterpretedMutableProjection(projectList, output) projection.initialize(0) - LocalRelation(projectList.map(_.toAttribute), data.map(projection), isStreaming) + LocalRelation(projectList.map(_.toAttribute), data.map(projection(_).copy()), isStreaming) case Limit(IntegerLiteral(limit), LocalRelation(output, data, isStreaming)) => LocalRelation(output, data.take(limit), isStreaming) @@ -1453,7 +1529,7 @@ object ReplaceIntersectWithSemiJoin extends Rule[LogicalPlan] { case Intersect(left, right, false) => assert(left.output.size == right.output.size) val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) } - Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And))) + Distinct(Join(left, right, LeftSemi, joinCond.reduceLeftOption(And), JoinHint.NONE)) } } @@ -1474,7 +1550,7 @@ object ReplaceExceptWithAntiJoin extends Rule[LogicalPlan] { case Except(left, right, false) => assert(left.output.size == right.output.size) val joinCond = left.output.zip(right.output).map { case (l, r) => EqualNullSafe(l, r) } - Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And))) + Distinct(Join(left, right, LeftAnti, joinCond.reduceLeftOption(And), JoinHint.NONE)) } } @@ -1650,16 +1726,35 @@ object RemoveRepetitionFromGroupExpressions extends Rule[LogicalPlan] { } /** - * Updates nullability in [[AttributeReference]]s if nullability is different between - * non-leaf plan's expressions and the children output. + * Replaces GlobalLimit 0 and LocalLimit 0 nodes (subtree) with empty Local Relation, as they don't + * return any rows. */ -object UpdateNullabilityInAttributeReferences extends Rule[LogicalPlan] { +object OptimizeLimitZero extends Rule[LogicalPlan] { + // returns empty Local Relation corresponding to given plan + private def empty(plan: LogicalPlan) = + LocalRelation(plan.output, data = Seq.empty, isStreaming = plan.isStreaming) + def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case p if !p.isInstanceOf[LeafNode] => - val nullabilityMap = AttributeMap(p.children.flatMap(_.output).map { x => x -> x.nullable }) - p transformExpressions { - case ar: AttributeReference if nullabilityMap.contains(ar) => - ar.withNullability(nullabilityMap(ar)) - } + // Nodes below GlobalLimit or LocalLimit can be pruned if the limit value is zero (0). + // Any subtree in the logical plan that has GlobalLimit 0 or LocalLimit 0 as its root is + // semantically equivalent to an empty relation. + // + // In such cases, the effects of Limit 0 can be propagated through the Logical Plan by replacing + // the (Global/Local) Limit subtree with an empty LocalRelation, thereby pruning the subtree + // below and triggering other optimization rules of PropagateEmptyRelation to propagate the + // changes up the Logical Plan. + // + // Replace Global Limit 0 nodes with empty Local Relation + case gl @ GlobalLimit(IntegerLiteral(0), _) => + empty(gl) + + // Note: For all SQL queries, if a LocalLimit 0 node exists in the Logical Plan, then a + // GlobalLimit 0 node would also exist. Thus, the above case would be sufficient to handle + // almost all cases. However, if a user explicitly creates a Logical Plan with LocalLimit 0 node + // then the following rule will handle that case as well. + // + // Replace Local Limit 0 nodes with empty Local Relation + case ll @ LocalLimit(IntegerLiteral(0), _) => + empty(ll) } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala index c3fdb924243df..b19e13870aa65 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelation.scala @@ -56,7 +56,7 @@ object PropagateEmptyRelation extends Rule[LogicalPlan] with PredicateHelper wit // Joins on empty LocalRelations generated from streaming sources are not eliminated // as stateful streaming joins need to perform other state management operations other than // just processing the input data. - case p @ Join(_, _, joinType, _) + case p @ Join(_, _, joinType, _, _) if !p.children.exists(_.isStreaming) => val isLeftEmpty = isEmptyLocalRelation(p.left) val isRightEmpty = isEmptyLocalRelation(p.right) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala new file mode 100644 index 0000000000000..606db85fcdea6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/PushDownLeftSemiAntiJoin.scala @@ -0,0 +1,265 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.Rule + +/** + * This rule is a variant of [[PushPredicateThroughNonJoin]] which can handle + * pushing down Left semi and Left Anti joins below the following operators. + * 1) Project + * 2) Window + * 3) Union + * 4) Aggregate + * 5) Other permissible unary operators. please see [[PushPredicateThroughNonJoin.canPushThrough]]. + */ +object PushDownLeftSemiAntiJoin extends Rule[LogicalPlan] with PredicateHelper { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // LeftSemi/LeftAnti over Project + case Join(p @ Project(pList, gChild), rightOp, LeftSemiOrAnti(joinType), joinCond, hint) + if pList.forall(_.deterministic) && + !pList.exists(ScalarSubquery.hasCorrelatedScalarSubquery) && + canPushThroughCondition(Seq(gChild), joinCond, rightOp) => + if (joinCond.isEmpty) { + // No join condition, just push down the Join below Project + p.copy(child = Join(gChild, rightOp, joinType, joinCond, hint)) + } else { + val aliasMap = PushPredicateThroughNonJoin.getAliasMap(p) + val newJoinCond = if (aliasMap.nonEmpty) { + Option(replaceAlias(joinCond.get, aliasMap)) + } else { + joinCond + } + p.copy(child = Join(gChild, rightOp, joinType, newJoinCond, hint)) + } + + // LeftSemi/LeftAnti over Aggregate + case join @ Join(agg: Aggregate, rightOp, LeftSemiOrAnti(_), _, _) + if agg.aggregateExpressions.forall(_.deterministic) && agg.groupingExpressions.nonEmpty && + !agg.aggregateExpressions.exists(ScalarSubquery.hasCorrelatedScalarSubquery) => + val aliasMap = PushPredicateThroughNonJoin.getAliasMap(agg) + val canPushDownPredicate = (predicate: Expression) => { + val replaced = replaceAlias(predicate, aliasMap) + predicate.references.nonEmpty && + replaced.references.subsetOf(agg.child.outputSet ++ rightOp.outputSet) + } + val makeJoinCondition = (predicates: Seq[Expression]) => { + replaceAlias(predicates.reduce(And), aliasMap) + } + pushDownJoin(join, canPushDownPredicate, makeJoinCondition) + + // LeftSemi/LeftAnti over Window + case join @ Join(w: Window, rightOp, LeftSemiOrAnti(_), _, _) + if w.partitionSpec.forall(_.isInstanceOf[AttributeReference]) => + val partitionAttrs = AttributeSet(w.partitionSpec.flatMap(_.references)) ++ rightOp.outputSet + pushDownJoin(join, _.references.subsetOf(partitionAttrs), _.reduce(And)) + + // LeftSemi/LeftAnti over Union + case Join(union: Union, rightOp, LeftSemiOrAnti(joinType), joinCond, hint) + if canPushThroughCondition(union.children, joinCond, rightOp) => + if (joinCond.isEmpty) { + // Push down the Join below Union + val newGrandChildren = union.children.map { Join(_, rightOp, joinType, joinCond, hint) } + union.withNewChildren(newGrandChildren) + } else { + val output = union.output + val newGrandChildren = union.children.map { grandchild => + val newCond = joinCond.get transform { + case e if output.exists(_.semanticEquals(e)) => + grandchild.output(output.indexWhere(_.semanticEquals(e))) + } + assert(newCond.references.subsetOf(grandchild.outputSet ++ rightOp.outputSet)) + Join(grandchild, rightOp, joinType, Option(newCond), hint) + } + union.withNewChildren(newGrandChildren) + } + + // LeftSemi/LeftAnti over UnaryNode + case join @ Join(u: UnaryNode, rightOp, LeftSemiOrAnti(_), _, _) + if PushPredicateThroughNonJoin.canPushThrough(u) && u.expressions.forall(_.deterministic) => + val validAttrs = u.child.outputSet ++ rightOp.outputSet + pushDownJoin(join, _.references.subsetOf(validAttrs), _.reduce(And)) + } + + /** + * Check if we can safely push a join through a project or union by making sure that attributes + * referred in join condition do not contain the same attributes as the plan they are moved + * into. This can happen when both sides of join refers to the same source (self join). This + * function makes sure that the join condition refers to attributes that are not ambiguous (i.e + * present in both the legs of the join) or else the resultant plan will be invalid. + */ + private def canPushThroughCondition( + plans: Seq[LogicalPlan], + condition: Option[Expression], + rightOp: LogicalPlan): Boolean = { + val attributes = AttributeSet(plans.flatMap(_.output)) + if (condition.isDefined) { + val matched = condition.get.references.intersect(rightOp.outputSet).intersect(attributes) + matched.isEmpty + } else { + true + } + } + + private def pushDownJoin( + join: Join, + canPushDownPredicate: Expression => Boolean, + makeJoinCondition: Seq[Expression] => Expression): LogicalPlan = { + assert(join.left.children.length == 1) + + if (join.condition.isEmpty) { + join.left.withNewChildren(Seq(join.copy(left = join.left.children.head))) + } else { + val (pushDown, stayUp) = splitConjunctivePredicates(join.condition.get) + .partition(canPushDownPredicate) + + // Check if the remaining predicates do not contain columns from the right hand side of the + // join. Since the remaining predicates will be kept as a filter over the operator under join, + // this check is necessary after the left-semi/anti join is pushed down. The reason is, for + // this kind of join, we only output from the left leg of the join. + val referRightSideCols = AttributeSet(stayUp.toSet).intersect(join.right.outputSet).nonEmpty + + if (pushDown.isEmpty || referRightSideCols) { + join + } else { + val newPlan = join.left.withNewChildren(Seq(join.copy( + left = join.left.children.head, condition = Some(makeJoinCondition(pushDown))))) + // If there is no more filter to stay up, return the new plan that has join pushed down. + if (stayUp.isEmpty) { + newPlan + } else { + join.joinType match { + // In case of Left semi join, the part of the join condition which does not refer to + // to attributes of the grandchild are kept as a Filter above. + case LeftSemi => Filter(stayUp.reduce(And), newPlan) + // In case of left-anti join, the join is pushed down only when the entire join + // condition is eligible to be pushed down to preserve the semantics of left-anti join. + case _ => join + } + } + } + } + } +} + +/** + * This rule is a variant of [[PushPredicateThroughJoin]] which can handle + * pushing down Left semi and Left Anti joins below a join operator. The + * allowable join types are: + * 1) Inner + * 2) Cross + * 3) LeftOuter + * 4) RightOuter + * + * TODO: + * Currently this rule can push down the left semi or left anti joins to either + * left or right leg of the child join. This matches the behaviour of `PushPredicateThroughJoin` + * when the lefi semi or left anti join is in expression form. We need to explore the possibility + * to push the left semi/anti joins to both legs of join if the join condition refers to + * both left and right legs of the child join. + */ +object PushLeftSemiLeftAntiThroughJoin extends Rule[LogicalPlan] with PredicateHelper { + /** + * Define an enumeration to identify whether a LeftSemi/LeftAnti join can be pushed down to + * the left leg or the right leg of the join. + */ + object PushdownDirection extends Enumeration { + val TO_LEFT_BRANCH, TO_RIGHT_BRANCH, NONE = Value + } + + object AllowedJoin { + def unapply(join: Join): Option[Join] = join.joinType match { + case Inner | Cross | LeftOuter | RightOuter => Some(join) + case _ => None + } + } + + /** + * Determine which side of the join a LeftSemi/LeftAnti join can be pushed to. + */ + private def pushTo(leftChild: Join, rightChild: LogicalPlan, joinCond: Option[Expression]) = { + val left = leftChild.left + val right = leftChild.right + val joinType = leftChild.joinType + val rightOutput = rightChild.outputSet + + if (joinCond.nonEmpty) { + val conditions = splitConjunctivePredicates(joinCond.get) + val (leftConditions, rest) = + conditions.partition(_.references.subsetOf(left.outputSet ++ rightOutput)) + val (rightConditions, commonConditions) = + rest.partition(_.references.subsetOf(right.outputSet ++ rightOutput)) + + if (rest.isEmpty && leftConditions.nonEmpty) { + // When the join conditions can be computed based on the left leg of + // leftsemi/anti join then push the leftsemi/anti join to the left side. + PushdownDirection.TO_LEFT_BRANCH + } else if (leftConditions.isEmpty && rightConditions.nonEmpty && commonConditions.isEmpty) { + // When the join conditions can be computed based on the attributes from right leg of + // leftsemi/anti join then push the leftsemi/anti join to the right side. + PushdownDirection.TO_RIGHT_BRANCH + } else { + PushdownDirection.NONE + } + } else { + /** + * When the join condition is empty, + * 1) if this is a left outer join or inner join, push leftsemi/anti join down + * to the left leg of join. + * 2) if a right outer join, to the right leg of join, + */ + joinType match { + case _: InnerLike | LeftOuter => + PushdownDirection.TO_LEFT_BRANCH + case RightOuter => + PushdownDirection.TO_RIGHT_BRANCH + case _ => + PushdownDirection.NONE + } + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // push LeftSemi/LeftAnti down into the join below + case j @ Join(AllowedJoin(left), right, LeftSemiOrAnti(joinType), joinCond, parentHint) => + val (childJoinType, childLeft, childRight, childCondition, childHint) = + (left.joinType, left.left, left.right, left.condition, left.hint) + val action = pushTo(left, right, joinCond) + + action match { + case PushdownDirection.TO_LEFT_BRANCH + if (childJoinType == LeftOuter || childJoinType.isInstanceOf[InnerLike]) => + // push down leftsemi/anti join to the left table + val newLeft = Join(childLeft, right, joinType, joinCond, parentHint) + Join(newLeft, childRight, childJoinType, childCondition, childHint) + case PushdownDirection.TO_RIGHT_BRANCH + if (childJoinType == RightOuter || childJoinType.isInstanceOf[InnerLike]) => + // push down leftsemi/anti join to the right table + val newRight = Join(childRight, right, joinType, joinCond, parentHint) + Join(childLeft, newRight, childJoinType, childCondition, childHint) + case _ => + // Do nothing + j + } + } +} + + diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala index efd3944eba7f5..4996d24dfd298 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceExceptWithFilter.scala @@ -36,7 +36,8 @@ import org.apache.spark.sql.catalyst.rules.Rule * Note: * Before flipping the filter condition of the right node, we should: * 1. Combine all it's [[Filter]]. - * 2. Apply InferFiltersFromConstraints rule (to take into account of NULL values in the condition). + * 2. Update the attribute references to the left node; + * 3. Add a Coalesce(condition, False) (to take into account of NULL values in the condition). */ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { @@ -47,23 +48,28 @@ object ReplaceExceptWithFilter extends Rule[LogicalPlan] { plan.transform { case e @ Except(left, right, false) if isEligible(left, right) => - val newCondition = transformCondition(left, skipProject(right)) - newCondition.map { c => - Distinct(Filter(Not(c), left)) - }.getOrElse { + val filterCondition = combineFilters(skipProject(right)).asInstanceOf[Filter].condition + if (filterCondition.deterministic) { + transformCondition(left, filterCondition).map { c => + Distinct(Filter(Not(c), left)) + }.getOrElse { + e + } + } else { e } } } - private def transformCondition(left: LogicalPlan, right: LogicalPlan): Option[Expression] = { - val filterCondition = - InferFiltersFromConstraints(combineFilters(right)).asInstanceOf[Filter].condition - - val attributeNameMap: Map[String, Attribute] = left.output.map(x => (x.name, x)).toMap - - if (filterCondition.references.forall(r => attributeNameMap.contains(r.name))) { - Some(filterCondition.transform { case a: AttributeReference => attributeNameMap(a.name) }) + private def transformCondition(plan: LogicalPlan, condition: Expression): Option[Expression] = { + val attributeNameMap: Map[String, Attribute] = plan.output.map(x => (x.name, x)).toMap + if (condition.references.forall(r => attributeNameMap.contains(r.name))) { + val rewrittenCondition = condition.transform { + case a: AttributeReference => attributeNameMap(a.name) + } + // We need to consider as False when the condition is NULL, otherwise we do not return those + // rows containing NULL which are instead filtered in the Except right plan + Some(Coalesce(Seq(rewrittenCondition, Literal.FalseLiteral))) } else { None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala new file mode 100644 index 0000000000000..b8edf985dbda3 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicate.scala @@ -0,0 +1,112 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, CaseWhen, Expression, If} +import org.apache.spark.sql.catalyst.expressions.{LambdaFunction, Literal, MapFilter, Or} +import org.apache.spark.sql.catalyst.expressions.Literal.FalseLiteral +import org.apache.spark.sql.catalyst.plans.logical.{Filter, Join, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.BooleanType +import org.apache.spark.util.Utils + + +/** + * A rule that replaces `Literal(null, BooleanType)` with `FalseLiteral`, if possible, in the search + * condition of the WHERE/HAVING/ON(JOIN) clauses, which contain an implicit Boolean operator + * "(search condition) = TRUE". The replacement is only valid when `Literal(null, BooleanType)` is + * semantically equivalent to `FalseLiteral` when evaluating the whole search condition. + * + * Please note that FALSE and NULL are not exchangeable in most cases, when the search condition + * contains NOT and NULL-tolerant expressions. Thus, the rule is very conservative and applicable + * in very limited cases. + * + * For example, `Filter(Literal(null, BooleanType))` is equal to `Filter(FalseLiteral)`. + * + * Another example containing branches is `Filter(If(cond, FalseLiteral, Literal(null, _)))`; + * this can be optimized to `Filter(If(cond, FalseLiteral, FalseLiteral))`, and eventually + * `Filter(FalseLiteral)`. + * + * Moreover, this rule also transforms predicates in all [[If]] expressions as well as branch + * conditions in all [[CaseWhen]] expressions, even if they are not part of the search conditions. + * + * For example, `Project(If(And(cond, Literal(null)), Literal(1), Literal(2)))` can be simplified + * into `Project(Literal(2))`. + */ +object ReplaceNullWithFalseInPredicate extends Rule[LogicalPlan] { + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case f @ Filter(cond, _) => f.copy(condition = replaceNullWithFalse(cond)) + case j @ Join(_, _, _, Some(cond), _) => j.copy(condition = Some(replaceNullWithFalse(cond))) + case p: LogicalPlan => p transformExpressions { + case i @ If(pred, _, _) => i.copy(predicate = replaceNullWithFalse(pred)) + case cw @ CaseWhen(branches, _) => + val newBranches = branches.map { case (cond, value) => + replaceNullWithFalse(cond) -> value + } + cw.copy(branches = newBranches) + case af @ ArrayFilter(_, lf @ LambdaFunction(func, _, _)) => + val newLambda = lf.copy(function = replaceNullWithFalse(func)) + af.copy(function = newLambda) + case ae @ ArrayExists(_, lf @ LambdaFunction(func, _, _)) + if !SQLConf.get.getConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC) => + val newLambda = lf.copy(function = replaceNullWithFalse(func)) + ae.copy(function = newLambda) + case mf @ MapFilter(_, lf @ LambdaFunction(func, _, _)) => + val newLambda = lf.copy(function = replaceNullWithFalse(func)) + mf.copy(function = newLambda) + } + } + + /** + * Recursively traverse the Boolean-type expression to replace + * `Literal(null, BooleanType)` with `FalseLiteral`, if possible. + * + * Note that `transformExpressionsDown` can not be used here as we must stop as soon as we hit + * an expression that is not [[CaseWhen]], [[If]], [[And]], [[Or]] or + * `Literal(null, BooleanType)`. + */ + private def replaceNullWithFalse(e: Expression): Expression = e match { + case Literal(null, BooleanType) => + FalseLiteral + case And(left, right) => + And(replaceNullWithFalse(left), replaceNullWithFalse(right)) + case Or(left, right) => + Or(replaceNullWithFalse(left), replaceNullWithFalse(right)) + case cw: CaseWhen if cw.dataType == BooleanType => + val newBranches = cw.branches.map { case (cond, value) => + replaceNullWithFalse(cond) -> replaceNullWithFalse(value) + } + val newElseValue = cw.elseValue.map(replaceNullWithFalse) + CaseWhen(newBranches, newElseValue) + case i @ If(pred, trueVal, falseVal) if i.dataType == BooleanType => + If(replaceNullWithFalse(pred), replaceNullWithFalse(trueVal), replaceNullWithFalse(falseVal)) + case e if e.dataType == BooleanType => + e + case e => + val message = "Expected a Boolean type expression in replaceNullWithFalse, " + + s"but got the type `${e.dataType.catalogString}` in `${e.sql}`." + if (Utils.isTesting) { + throw new IllegalArgumentException(message) + } else { + logWarning(message) + e + } + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala index 4448ace7105a4..b9468007cac61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregates.scala @@ -95,7 +95,7 @@ import org.apache.spark.sql.types.IntegerType * * This rule duplicates the input data by two or more times (# distinct groups + an optional * non-distinct group). This will put quite a bit of memory pressure of the used aggregate and - * exchange operators. Keeping the number of distinct groups as low a possible should be priority, + * exchange operators. Keeping the number of distinct groups as low as possible should be priority, * we could improve this in the current rule by applying more advanced expression canonicalization * techniques. */ @@ -241,7 +241,7 @@ object RewriteDistinctAggregates extends Rule[LogicalPlan] { groupByAttrs ++ distinctAggChildAttrs ++ Seq(gid) ++ regularAggChildAttrMap.map(_._2), a.child) - // Construct the first aggregate operator. This de-duplicates the all the children of + // Construct the first aggregate operator. This de-duplicates all the children of // distinct operators, and applies the regular aggregate operators. val firstAggregateGroupBy = groupByAttrs ++ distinctAggChildAttrs :+ gid val firstAggregate = Aggregate( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala index f8037588fa71e..39709529c00d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala @@ -276,15 +276,37 @@ object BooleanSimplification extends Rule[LogicalPlan] with PredicateHelper { case a And b if a.semanticEquals(b) => a case a Or b if a.semanticEquals(b) => a - case a And (b Or c) if Not(a).semanticEquals(b) => And(a, c) - case a And (b Or c) if Not(a).semanticEquals(c) => And(a, b) - case (a Or b) And c if a.semanticEquals(Not(c)) => And(b, c) - case (a Or b) And c if b.semanticEquals(Not(c)) => And(a, c) - - case a Or (b And c) if Not(a).semanticEquals(b) => Or(a, c) - case a Or (b And c) if Not(a).semanticEquals(c) => Or(a, b) - case (a And b) Or c if a.semanticEquals(Not(c)) => Or(b, c) - case (a And b) Or c if b.semanticEquals(Not(c)) => Or(a, c) + // The following optimizations are applicable only when the operands are not nullable, + // since the three-value logic of AND and OR are different in NULL handling. + // See the chart: + // +---------+---------+---------+---------+ + // | operand | operand | OR | AND | + // +---------+---------+---------+---------+ + // | TRUE | TRUE | TRUE | TRUE | + // | TRUE | FALSE | TRUE | FALSE | + // | FALSE | FALSE | FALSE | FALSE | + // | UNKNOWN | TRUE | TRUE | UNKNOWN | + // | UNKNOWN | FALSE | UNKNOWN | FALSE | + // | UNKNOWN | UNKNOWN | UNKNOWN | UNKNOWN | + // +---------+---------+---------+---------+ + + // (NULL And (NULL Or FALSE)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. + case a And (b Or c) if !a.nullable && Not(a).semanticEquals(b) => And(a, c) + // (NULL And (FALSE Or NULL)) = NULL, but (NULL And FALSE) = FALSE. Thus, a can't be nullable. + case a And (b Or c) if !a.nullable && Not(a).semanticEquals(c) => And(a, b) + // ((NULL Or FALSE) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. + case (a Or b) And c if !c.nullable && a.semanticEquals(Not(c)) => And(b, c) + // ((FALSE Or NULL) And NULL) = NULL, but (FALSE And NULL) = FALSE. Thus, c can't be nullable. + case (a Or b) And c if !c.nullable && b.semanticEquals(Not(c)) => And(a, c) + + // (NULL Or (NULL And TRUE)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. + case a Or (b And c) if !a.nullable && Not(a).semanticEquals(b) => Or(a, c) + // (NULL Or (TRUE And NULL)) = NULL, but (NULL Or TRUE) = TRUE. Thus, a can't be nullable. + case a Or (b And c) if !a.nullable && Not(a).semanticEquals(c) => Or(a, b) + // ((NULL And TRUE) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. + case (a And b) Or c if !c.nullable && a.semanticEquals(Not(c)) => Or(b, c) + // ((TRUE And NULL) Or NULL) = NULL, but (TRUE Or NULL) = TRUE. Thus, c can't be nullable. + case (a And b) Or c if !c.nullable && b.semanticEquals(Not(c)) => Or(a, c) // Common factor elimination for conjunction case and @ (left And right) => @@ -578,7 +600,7 @@ object FoldablePropagation extends Rule[LogicalPlan] { // propagating the foldable expressions. // TODO(cloud-fan): It seems more reasonable to use new attributes as the output attributes // of outer join. - case j @ Join(left, right, joinType, _) if foldableMap.nonEmpty => + case j @ Join(left, right, joinType, _, _) if foldableMap.nonEmpty => val newJoin = j.transformExpressions(replaceFoldable) val missDerivedAttrsSet: AttributeSet = AttributeSet(joinType match { case _: InnerLike | LeftExistence(_) => Nil @@ -626,7 +648,6 @@ object FoldablePropagation extends Rule[LogicalPlan] { case _: Distinct => true case _: AppendColumns => true case _: AppendColumnsWithObject => true - case _: ResolvedHint => true case _: RepartitionByExpression => true case _: Repartition => true case _: Sort => true diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala index af0837e36e8ad..69ba76827c781 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/finishAnalysis.scala @@ -17,10 +17,13 @@ package org.apache.spark.sql.catalyst.optimizer +import java.time.LocalDate + import scala.collection.mutable import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.util.DateTimeUtils @@ -28,13 +31,25 @@ import org.apache.spark.sql.types._ /** - * Finds all [[RuntimeReplaceable]] expressions and replace them with the expressions that can - * be evaluated. This is mainly used to provide compatibility with other databases. - * For example, we use this to support "nvl" by replacing it with "coalesce". + * Finds all the expressions that are unevaluable and replace/rewrite them with semantically + * equivalent expressions that can be evaluated. Currently we replace two kinds of expressions: + * 1) [[RuntimeReplaceable]] expressions + * 2) [[UnevaluableAggregate]] expressions such as Every, Some, Any, CountIf + * This is mainly used to provide compatibility with other databases. + * Few examples are: + * we use this to support "nvl" by replacing it with "coalesce". + * we use this to replace Every and Any with Min and Max respectively. + * + * TODO: In future, explore an option to replace aggregate functions similar to + * how RuntimeReplaceable does. */ object ReplaceExpressions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { case e: RuntimeReplaceable => e.child + case CountIf(predicate) => Count(new NullIf(predicate, Literal.FalseLiteral)) + case SomeAgg(arg) => Max(arg) + case AnyAgg(arg) => Max(arg) + case EveryAgg(arg) => Min(arg) } } @@ -53,7 +68,7 @@ object ComputeCurrentTime extends Rule[LogicalPlan] { case CurrentDate(Some(timeZoneId)) => currentDates.getOrElseUpdate(timeZoneId, { Literal.create( - DateTimeUtils.millisToDays(timestamp / 1000L, DateTimeUtils.getTimeZone(timeZoneId)), + LocalDate.now(DateTimeUtils.getZoneId(timeZoneId)), DateType) }) case CurrentTimestamp() => currentTime diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala index 7149edee0173e..22704b2d3cff8 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala @@ -45,8 +45,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { * @param conditions a list of condition for join. */ @tailrec - final def createOrderedJoin(input: Seq[(LogicalPlan, InnerLike)], conditions: Seq[Expression]) - : LogicalPlan = { + final def createOrderedJoin( + input: Seq[(LogicalPlan, InnerLike)], + conditions: Seq[Expression]): LogicalPlan = { assert(input.size >= 2) if (input.size == 2) { val (joinConditions, others) = conditions.partition(canEvaluateWithinJoin) @@ -55,7 +56,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { case (Inner, Inner) => Inner case (_, _) => Cross } - val join = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And)) + val join = Join(left, right, innerJoinType, + joinConditions.reduceLeftOption(And), JoinHint.NONE) if (others.nonEmpty) { Filter(others.reduceLeft(And), join) } else { @@ -78,7 +80,8 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { val joinedRefs = left.outputSet ++ right.outputSet val (joinConditions, others) = conditions.partition( e => e.references.subsetOf(joinedRefs) && canEvaluateWithinJoin(e)) - val joined = Join(left, right, innerJoinType, joinConditions.reduceLeftOption(And)) + val joined = Join(left, right, innerJoinType, + joinConditions.reduceLeftOption(And), JoinHint.NONE) // should not have reference to same logical plan createOrderedJoin(Seq((joined, Inner)) ++ rest.filterNot(_._1 eq right), others) @@ -86,9 +89,9 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case ExtractFiltersAndInnerJoins(input, conditions) + case p @ ExtractFiltersAndInnerJoins(input, conditions) if input.size > 2 && conditions.nonEmpty => - if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { + val reordered = if (SQLConf.get.starSchemaDetection && !SQLConf.get.cboEnabled) { val starJoinPlan = StarSchemaDetection.reorderStarJoins(input, conditions) if (starJoinPlan.nonEmpty) { val rest = input.filterNot(starJoinPlan.contains(_)) @@ -99,6 +102,14 @@ object ReorderJoin extends Rule[LogicalPlan] with PredicateHelper { } else { createOrderedJoin(input, conditions) } + + if (p.sameOutput(reordered)) { + reordered + } else { + // Reordering the joins have changed the order of the columns. + // Inject a projection to make sure we restore to the expected ordering. + Project(p.output, reordered) + } } } @@ -148,28 +159,29 @@ object EliminateOuterJoin extends Rule[LogicalPlan] with PredicateHelper { } def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _)) => + case f @ Filter(condition, j @ Join(_, _, RightOuter | LeftOuter | FullOuter, _, _)) => val newJoinType = buildNewJoinType(f, j) if (j.joinType == newJoinType) f else Filter(condition, j.copy(joinType = newJoinType)) } } /** - * PythonUDF in join condition can not be evaluated, this rule will detect the PythonUDF - * and pull them out from join condition. For python udf accessing attributes from only one side, - * they are pushed down by operation push down rules. If not (e.g. user disables filter push - * down rules), we need to pull them out in this rule too. + * PythonUDF in join condition can't be evaluated if it refers to attributes from both join sides. + * See `ExtractPythonUDFs` for details. This rule will detect un-evaluable PythonUDF and pull them + * out from join condition. */ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateHelper { - def hasPythonUDF(expression: Expression): Boolean = { - expression.collectFirst { case udf: PythonUDF => udf }.isDefined + + private def hasUnevaluablePythonUDF(expr: Expression, j: Join): Boolean = { + expr.find { e => + PythonUDF.isScalarPythonUDF(e) && !canEvaluate(e, j.left) && !canEvaluate(e, j.right) + }.isDefined } override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case j @ Join(_, _, joinType, condition) - if condition.isDefined && hasPythonUDF(condition.get) => - if (!joinType.isInstanceOf[InnerLike] && joinType != LeftSemi) { - // The current strategy only support InnerLike and LeftSemi join because for other type, + case j @ Join(_, _, joinType, Some(cond), _) if hasUnevaluablePythonUDF(cond, j) => + if (!joinType.isInstanceOf[InnerLike]) { + // The current strategy supports only InnerLike join because for other types, // it breaks SQL semantic if we run the join condition as a filter after join. If we pass // the plan here, it'll still get a an invalid PythonUDF RuntimeException with message // `requires attributes from more than one child`, we throw firstly here for better @@ -179,10 +191,9 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH } // If condition expression contains python udf, it will be moved out from // the new join conditions. - val (udf, rest) = - splitConjunctivePredicates(condition.get).partition(hasPythonUDF) + val (udf, rest) = splitConjunctivePredicates(cond).partition(hasUnevaluablePythonUDF(_, j)) val newCondition = if (rest.isEmpty) { - logWarning(s"The join condition:$condition of the join plan contains PythonUDF only," + + logWarning(s"The join condition:$cond of the join plan contains PythonUDF only," + s" it will be moved out and the join plan will be turned to cross join.") None } else { @@ -191,10 +202,6 @@ object PullOutPythonUDFInJoinCondition extends Rule[LogicalPlan] with PredicateH val newJoin = j.copy(condition = newCondition) joinType match { case _: InnerLike => Filter(udf.reduceLeft(And), newJoin) - case LeftSemi => - Project( - j.left.output.map(_.toAttribute), - Filter(udf.reduceLeft(And), newJoin.copy(joinType = Inner))) case _ => throw new AnalysisException("Using PythonUDF in join condition of join type" + s" $joinType is not supported.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala index 8cdc6425bcad8..ad93ef347a974 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/objects.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst.optimizer +import scala.collection.mutable.ArrayBuffer + import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructType, UserDefinedType} /* * This file defines optimization rules related to object manipulation (for the Dataset API). @@ -106,6 +110,172 @@ object CombineTypedFilters extends Rule[LogicalPlan] { */ object EliminateMapObjects extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { - case MapObjects(_, _, _, LambdaVariable(_, _, _, false), inputData, None) => inputData + case MapObjects(_, LambdaVariable(_, _, false, _), inputData, None) => inputData + } +} + +/** + * Prunes unnecessary object serializers from query plan. This rule prunes both individual + * serializer and nested fields in serializers. + */ +object ObjectSerializerPruning extends Rule[LogicalPlan] { + + /** + * Visible for testing. + * Collects all struct types from given data type object, recursively. + */ + def collectStructType(dt: DataType, structs: ArrayBuffer[StructType]): ArrayBuffer[StructType] = { + dt match { + case s @ StructType(fields) => + structs += s + fields.map(f => collectStructType(f.dataType, structs)) + case ArrayType(elementType, _) => + collectStructType(elementType, structs) + case MapType(keyType, valueType, _) => + collectStructType(keyType, structs) + collectStructType(valueType, structs) + // We don't use UserDefinedType in those serializers. + case _: UserDefinedType[_] => + case _ => + } + structs + } + + /** + * This method returns pruned `CreateNamedStruct` expression given an original `CreateNamedStruct` + * and a pruned `StructType`. + */ + private def pruneNamedStruct(struct: CreateNamedStruct, prunedType: StructType) = { + // Filters out the pruned fields. + val resolver = SQLConf.get.resolver + val prunedFields = struct.nameExprs.zip(struct.valExprs).filter { case (nameExpr, _) => + val name = nameExpr.eval(EmptyRow).toString + prunedType.fieldNames.exists(resolver(_, name)) + }.flatMap(pair => Seq(pair._1, pair._2)) + + CreateNamedStruct(prunedFields) + } + + /** + * When we change nested serializer data type, `If` expression will be unresolved because + * literal null's data type doesn't match now. We need to align it with new data type. + * Note: we should do `transformUp` explicitly to change data types. + */ + private def alignNullTypeInIf(expr: Expression) = expr.transformUp { + case i @ If(_: IsNull, Literal(null, dt), ser) if !dt.sameType(ser.dataType) => + i.copy(trueValue = Literal(null, ser.dataType)) + } + + /** + * This method prunes given serializer expression by given pruned data type. For example, + * given a serializer creating struct(a int, b int) and pruned data type struct(a int), + * this method returns pruned serializer creating struct(a int). + */ + def pruneSerializer( + serializer: NamedExpression, + prunedDataType: DataType): NamedExpression = { + val prunedStructTypes = collectStructType(prunedDataType, ArrayBuffer.empty[StructType]) + .toIterator + + def transformer: PartialFunction[Expression, Expression] = { + case m: ExternalMapToCatalyst => + val prunedKeyConverter = m.keyConverter.transformDown(transformer) + val prunedValueConverter = m.valueConverter.transformDown(transformer) + + m.copy(keyConverter = alignNullTypeInIf(prunedKeyConverter), + valueConverter = alignNullTypeInIf(prunedValueConverter)) + + case s: CreateNamedStruct if prunedStructTypes.hasNext => + val prunedType = prunedStructTypes.next() + pruneNamedStruct(s, prunedType) + } + + val transformedSerializer = serializer.transformDown(transformer) + val prunedSerializer = alignNullTypeInIf(transformedSerializer).asInstanceOf[NamedExpression] + + if (prunedSerializer.dataType.sameType(prunedDataType)) { + prunedSerializer + } else { + serializer + } + } + + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case p @ Project(_, s: SerializeFromObject) => + // Prunes individual serializer if it is not used at all by above projection. + val usedRefs = p.references + val prunedSerializer = s.serializer.filter(usedRefs.contains) + + val rootFields = SchemaPruning.identifyRootFields(p.projectList, Seq.empty) + + if (SQLConf.get.serializerNestedSchemaPruningEnabled && rootFields.nonEmpty) { + // Prunes nested fields in serializers. + val prunedSchema = SchemaPruning.pruneDataSchema( + StructType.fromAttributes(prunedSerializer.map(_.toAttribute)), rootFields) + val nestedPrunedSerializer = prunedSerializer.zipWithIndex.map { case (serializer, idx) => + pruneSerializer(serializer, prunedSchema(idx).dataType) + } + + // Builds new projection. + val projectionOverSchema = ProjectionOverSchema(prunedSchema) + val newProjects = p.projectList.map(_.transformDown { + case projectionOverSchema(expr) => expr + }).map { case expr: NamedExpression => expr } + p.copy(projectList = newProjects, + child = SerializeFromObject(nestedPrunedSerializer, s.child)) + } else { + p.copy(child = SerializeFromObject(prunedSerializer, s.child)) + } + } +} + +/** + * Reassigns per-query unique IDs to `LambdaVariable`s, whose original IDs are globally unique. This + * can help Spark to hit codegen cache more often and improve performance. + */ +object ReassignLambdaVariableID extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = { + if (!SQLConf.get.getConf(SQLConf.OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID)) return plan + + // The original LambdaVariable IDs are all positive. To avoid conflicts, the new IDs are all + // negative and starts from -1. + var newId = 0L + val oldIdToNewId = scala.collection.mutable.Map.empty[Long, Long] + + // The `LambdaVariable` IDs in a query should be all positive or negative. Otherwise it's a bug + // and we should fail earlier. + var hasNegativeIds = false + var hasPositiveIds = false + + plan.transformAllExpressions { + case lr: LambdaVariable if lr.id == 0 => + throw new IllegalStateException("LambdaVariable should never has 0 as its ID.") + + case lr: LambdaVariable if lr.id < 0 => + hasNegativeIds = true + if (hasPositiveIds) { + throw new IllegalStateException( + "LambdaVariable IDs in a query should be all positive or negative.") + + } + lr + + case lr: LambdaVariable if lr.id > 0 => + hasPositiveIds = true + if (hasNegativeIds) { + throw new IllegalStateException( + "LambdaVariable IDs in a query should be all positive or negative.") + } + + if (oldIdToNewId.contains(lr.id)) { + // This `LambdaVariable` has appeared before, reuse the newly generated ID. + lr.copy(id = oldIdToNewId(lr.id)) + } else { + // This is the first appearance of this `LambdaVariable`, generate a new ID. + newId -= 1 + oldIdToNewId(lr.id) = newId + lr.copy(id = newId) + } + } } } 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 e9b7a8b76e683..e78ed1c3c5d94 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.collection.mutable.ArrayBuffer -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.SubExprUtils._ import org.apache.spark.sql.catalyst.expressions.aggregate._ @@ -43,31 +43,53 @@ import org.apache.spark.sql.types._ * condition. */ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { - private def dedupJoin(joinPlan: LogicalPlan): LogicalPlan = joinPlan match { + + private def buildJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + joinType: JoinType, + condition: Option[Expression]): Join = { + // Deduplicate conflicting attributes if any. + val dedupSubplan = dedupSubqueryOnSelfJoin(outerPlan, subplan, None, condition) + Join(outerPlan, dedupSubplan, joinType, condition, JoinHint.NONE) + } + + private def dedupSubqueryOnSelfJoin( + outerPlan: LogicalPlan, + subplan: LogicalPlan, + valuesOpt: Option[Seq[Expression]], + condition: Option[Expression] = None): LogicalPlan = { // SPARK-21835: It is possibly that the two sides of the join have conflicting attributes, // the produced join then becomes unresolved and break structural integrity. We should - // de-duplicate conflicting attributes. We don't use transformation here because we only - // care about the most top join converted from correlated predicate subquery. - case j @ Join(left, right, joinType @ (LeftSemi | LeftAnti | ExistenceJoin(_)), joinCond) => - val duplicates = right.outputSet.intersect(left.outputSet) - if (duplicates.nonEmpty) { - val aliasMap = AttributeMap(duplicates.map { dup => - dup -> Alias(dup, dup.toString)() - }.toSeq) - val aliasedExpressions = right.output.map { ref => - aliasMap.getOrElse(ref, ref) - } - val newRight = Project(aliasedExpressions, right) - val newJoinCond = joinCond.map { condExpr => - condExpr transform { - case a: Attribute => aliasMap.getOrElse(a, a).toAttribute + // de-duplicate conflicting attributes. + // SPARK-26078: it may also happen that the subquery has conflicting attributes with the outer + // values. In this case, the resulting join would contain trivially true conditions (eg. + // id#3 = id#3) which cannot be de-duplicated after. In this method, if there are conflicting + // attributes in the join condition, the subquery's conflicting attributes are changed using + // a projection which aliases them and resolves the problem. + val outerReferences = valuesOpt.map(values => + AttributeSet.fromAttributeSets(values.map(_.references))).getOrElse(AttributeSet.empty) + val outerRefs = outerPlan.outputSet ++ outerReferences + val duplicates = outerRefs.intersect(subplan.outputSet) + if (duplicates.nonEmpty) { + condition.foreach { e => + val conflictingAttrs = e.references.intersect(duplicates) + if (conflictingAttrs.nonEmpty) { + throw new AnalysisException("Found conflicting attributes " + + s"${conflictingAttrs.mkString(",")} in the condition joining outer plan:\n " + + s"$outerPlan\nand subplan:\n $subplan") } - } - Join(left, newRight, joinType, newJoinCond) - } else { - j } - case _ => joinPlan + val rewrites = AttributeMap(duplicates.map { dup => + dup -> Alias(dup, dup.toString)() + }.toSeq) + val aliasedExpressions = subplan.output.map { ref => + rewrites.getOrElse(ref, ref) + } + Project(aliasedExpressions, subplan) + } else { + subplan + } } def apply(plan: LogicalPlan): LogicalPlan = plan transform { @@ -85,17 +107,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { withSubquery.foldLeft(newFilter) { case (p, Exists(sub, conditions, _)) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + buildJoin(outerPlan, sub, LeftSemi, joinCond) case (p, Not(Exists(sub, conditions, _))) => val (joinCond, outerPlan) = rewriteExistentialExpr(conditions, p) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, joinCond)) + buildJoin(outerPlan, sub, LeftAnti, joinCond) case (p, InSubquery(values, ListQuery(sub, conditions, _, _))) => - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftSemi, joinCond)) + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions ++ conditions, p) + Join(outerPlan, newSub, LeftSemi, joinCond, JoinHint.NONE) case (p, Not(InSubquery(values, ListQuery(sub, conditions, _, _)))) => // This is a NULL-aware (left) anti join (NAAJ) e.g. col NOT IN expr // Construct the condition. A NULL in one of the conditions is regarded as a positive @@ -103,7 +124,10 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // Note that will almost certainly be planned as a Broadcast Nested Loop join. // Use EXISTS if performance matters to you. - val inConditions = values.zip(sub.output).map(EqualTo.tupled) + + // Deduplicate conflicting attributes if any. + val newSub = dedupSubqueryOnSelfJoin(p, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) val (joinCond, outerPlan) = rewriteExistentialExpr(inConditions, p) // Expand the NOT IN expression with the NULL-aware semantic // to its full form. That is from: @@ -118,8 +142,7 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { // will have the final conditions in the LEFT ANTI as // (A.A1 = B.B1 OR ISNULL(A.A1 = B.B1)) AND (B.B2 = A.A2) AND B.B3 > 1 val finalJoinCond = (nullAwareJoinConds ++ conditions).reduceLeft(And) - // Deduplicate conflicting attributes if any. - dedupJoin(Join(outerPlan, sub, LeftAnti, Option(finalJoinCond))) + Join(outerPlan, newSub, LeftAnti, Option(finalJoinCond), JoinHint.NONE) case (p, predicate) => val (newCond, inputPlan) = rewriteExistentialExpr(Seq(predicate), p) Project(p.output, Filter(newCond.get, inputPlan)) @@ -140,16 +163,16 @@ object RewritePredicateSubquery extends Rule[LogicalPlan] with PredicateHelper { e transformUp { case Exists(sub, conditions, _) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - // Deduplicate conflicting attributes if any. - newPlan = dedupJoin( - Join(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And))) + newPlan = + buildJoin(newPlan, sub, ExistenceJoin(exists), conditions.reduceLeftOption(And)) exists case InSubquery(values, ListQuery(sub, conditions, _, _)) => val exists = AttributeReference("exists", BooleanType, nullable = false)() - val inConditions = values.zip(sub.output).map(EqualTo.tupled) - val newConditions = (inConditions ++ conditions).reduceLeftOption(And) // Deduplicate conflicting attributes if any. - newPlan = dedupJoin(Join(newPlan, sub, ExistenceJoin(exists), newConditions)) + val newSub = dedupSubqueryOnSelfJoin(newPlan, sub, Some(values)) + val inConditions = values.zip(newSub.output).map(EqualTo.tupled) + val newConditions = (inConditions ++ conditions).reduceLeftOption(And) + newPlan = Join(newPlan, newSub, ExistenceJoin(exists), newConditions, JoinHint.NONE) exists } } @@ -427,7 +450,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { // CASE 1: Subquery guaranteed not to have the COUNT bug Project( currentChild.output :+ origOutput, - Join(currentChild, query, LeftOuter, conditions.reduceOption(And))) + Join(currentChild, query, LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // Subquery might have the COUNT bug. Add appropriate corrections. val (topPart, havingNode, aggNode) = splitSubquery(query) @@ -454,7 +477,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { aggValRef), origOutput.name)(exprId = origOutput.exprId), Join(currentChild, Project(query.output :+ alwaysTrueExpr, query), - LeftOuter, conditions.reduceOption(And))) + LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } else { // CASE 3: Subquery with HAVING clause. Pull the HAVING clause above the join. @@ -484,7 +507,7 @@ object RewriteCorrelatedScalarSubquery extends Rule[LogicalPlan] { currentChild.output :+ caseExpr, Join(currentChild, Project(subqueryRoot.output :+ alwaysTrueExpr, subqueryRoot), - LeftOuter, conditions.reduceOption(And))) + LeftOuter, conditions.reduceOption(And), JoinHint.NONE)) } } 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 cb128b85c41c4..5df70a7ba200d 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 @@ -17,7 +17,6 @@ package org.apache.spark.sql.catalyst.parser -import java.sql.{Date, Timestamp} import java.util.Locale import javax.xml.bind.DatatypeConverter @@ -29,17 +28,21 @@ import org.antlr.v4.runtime.tree.{ParseTree, RuleNode, TerminalNode} import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalog.v2 +import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, Transform, YearsTransform} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} import org.apache.spark.util.random.RandomSampler /** @@ -86,6 +89,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging visitFunctionIdentifier(ctx.functionIdentifier) } + override def visitSingleMultipartIdentifier( + ctx: SingleMultipartIdentifierContext): Seq[String] = withOrigin(ctx) { + visitMultipartIdentifier(ctx.multipartIdentifier) + } + override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) { visitSparkDataType(ctx.dataType) } @@ -103,17 +111,72 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a top-level plan with Common Table Expressions. */ override def visitQuery(ctx: QueryContext): LogicalPlan = withOrigin(ctx) { - val query = plan(ctx.queryNoWith) + val query = plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses) // Apply CTEs - query.optional(ctx.ctes) { - val ctes = ctx.ctes.namedQuery.asScala.map { nCtx => - val namedQuery = visitNamedQuery(nCtx) - (namedQuery.alias, namedQuery) - } - // Check for duplicate names. - checkDuplicateKeys(ctes, ctx) - With(query, ctes) + query.optionalMap(ctx.ctes)(withCTE) + } + + override def visitDmlStatement(ctx: DmlStatementContext): AnyRef = withOrigin(ctx) { + val dmlStmt = plan(ctx.dmlStatementNoWith) + // Apply CTEs + dmlStmt.optionalMap(ctx.ctes)(withCTE) + } + + private def withCTE(ctx: CtesContext, plan: LogicalPlan): LogicalPlan = { + val ctes = ctx.namedQuery.asScala.map { nCtx => + val namedQuery = visitNamedQuery(nCtx) + (namedQuery.alias, namedQuery) + } + // Check for duplicate names. + val duplicates = ctes.groupBy(_._1).filter(_._2.size > 1).keys + if (duplicates.nonEmpty) { + throw new ParseException( + s"CTE definition can't have duplicate names: ${duplicates.mkString("'", "', '", "'")}.", + ctx) + } + With(plan, ctes) + } + + /** + * Create a logical query plan for a hive-style FROM statement body. + */ + private def withFromStatementBody( + ctx: FromStatementBodyContext, plan: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // two cases for transforms and selects + if (ctx.transformClause != null) { + withTransformQuerySpecification( + ctx, + ctx.transformClause, + ctx.whereClause, + plan + ) + } else { + withSelectQuerySpecification( + ctx, + ctx.selectClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + plan + ) + } + } + + override def visitFromStatement(ctx: FromStatementContext): LogicalPlan = withOrigin(ctx) { + val from = visitFromClause(ctx.fromClause) + val selects = ctx.fromStatementBody.asScala.map { body => + withFromStatementBody(body, from). + // Add organization statements. + optionalMap(body.queryOrganization)(withQueryResultClauses) + } + // If there are multiple SELECT just UNION them together into one query. + if (selects.length == 1) { + selects.head + } else { + Union(selects) } } @@ -123,7 +186,11 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * This is only used for Common Table Expressions. */ override def visitNamedQuery(ctx: NamedQueryContext): SubqueryAlias = withOrigin(ctx) { - SubqueryAlias(ctx.name.getText, plan(ctx.query)) + val subQuery: LogicalPlan = plan(ctx.query).optionalMap(ctx.columnAliases)( + (columnAliases, plan) => + UnresolvedSubqueryColumnAliases(visitIdentifierList(columnAliases), plan) + ) + SubqueryAlias(ctx.name.getText, subQuery) } /** @@ -146,23 +213,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val from = visitFromClause(ctx.fromClause) // Build the insert clauses. - val inserts = ctx.multiInsertQueryBody.asScala.map { - body => - validate(body.querySpecification.fromClause == null, - "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements", - body) - - withQuerySpecification(body.querySpecification, from). - // Add organization statements. - optionalMap(body.queryOrganization)(withQueryResultClauses). - // Add insert. - optionalMap(body.insertInto())(withInsertInto) + val inserts = ctx.multiInsertQueryBody.asScala.map { body => + withInsertInto(body.insertInto, + withFromStatementBody(body.fromStatementBody, from). + optionalMap(body.fromStatementBody.queryOrganization)(withQueryResultClauses)) } // If there are multiple INSERTS just UNION them together into one query. - inserts match { - case Seq(query) => query - case queries => Union(queries) + if (inserts.length == 1) { + inserts.head + } else { + Union(inserts) } } @@ -171,11 +232,9 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitSingleInsertQuery( ctx: SingleInsertQueryContext): LogicalPlan = withOrigin(ctx) { - plan(ctx.queryTerm). - // Add organization statements. - optionalMap(ctx.queryOrganization)(withQueryResultClauses). - // Add insert. - optionalMap(ctx.insertInto())(withInsertInto) + withInsertInto( + ctx.insertInto(), + plan(ctx.queryTerm).optionalMap(ctx.queryOrganization)(withQueryResultClauses)) } /** @@ -370,7 +429,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } // WINDOWS - val withWindow = withOrder.optionalMap(windows)(withWindows) + val withWindow = withOrder.optionalMap(windowClause)(withWindowClause) // LIMIT // - LIMIT ALL is the same as omitting the LIMIT clause @@ -389,140 +448,177 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging throw new ParseException("DISTRIBUTE BY is not supported", ctx) } - /** - * Create a logical plan using a query specification. - */ - override def visitQuerySpecification( - ctx: QuerySpecificationContext): LogicalPlan = withOrigin(ctx) { + override def visitTransformQuerySpecification( + ctx: TransformQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { val from = OneRowRelation().optional(ctx.fromClause) { visitFromClause(ctx.fromClause) } - withQuerySpecification(ctx, from) + withTransformQuerySpecification(ctx, ctx.transformClause, ctx.whereClause, from) } - /** - * Add a query specification to a logical plan. The query specification is the core of the logical - * plan, this is where sourcing (FROM clause), transforming (SELECT TRANSFORM/MAP/REDUCE), - * projection (SELECT), aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place. - * - * Note that query hints are ignored (both by the parser and the builder). - */ - private def withQuerySpecification( - ctx: QuerySpecificationContext, - relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { - import ctx._ - - // WHERE - def filter(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = { - Filter(expression(ctx), plan) - } - - def withHaving(ctx: BooleanExpressionContext, plan: LogicalPlan): LogicalPlan = { - // Note that we add a cast to non-predicate expressions. If the expression itself is - // already boolean, the optimizer will get rid of the unnecessary cast. - val predicate = expression(ctx) match { - case p: Predicate => p - case e => Cast(e, BooleanType) - } - Filter(predicate, plan) + override def visitRegularQuerySpecification( + ctx: RegularQuerySpecificationContext): LogicalPlan = withOrigin(ctx) { + val from = OneRowRelation().optional(ctx.fromClause) { + visitFromClause(ctx.fromClause) } - - - // Expressions. - val expressions = Option(namedExpressionSeq).toSeq + withSelectQuerySpecification( + ctx, + ctx.selectClause, + ctx.lateralView, + ctx.whereClause, + ctx.aggregationClause, + ctx.havingClause, + ctx.windowClause, + from + ) + } + + override def visitNamedExpressionSeq( + ctx: NamedExpressionSeqContext): Seq[Expression] = { + Option(ctx).toSeq .flatMap(_.namedExpression.asScala) .map(typedVisit[Expression]) + } - // Create either a transform or a regular query. - val specType = Option(kind).map(_.getType).getOrElse(SqlBaseParser.SELECT) - specType match { - case SqlBaseParser.MAP | SqlBaseParser.REDUCE | SqlBaseParser.TRANSFORM => - // Transform - - // Add where. - val withFilter = relation.optionalMap(where)(filter) - - // Create the attributes. - val (attributes, schemaLess) = if (colTypeList != null) { - // Typed return columns. - (createSchema(colTypeList).toAttributes, false) - } else if (identifierSeq != null) { - // Untyped return columns. - val attrs = visitIdentifierSeq(identifierSeq).map { name => - AttributeReference(name, StringType, nullable = true)() - } - (attrs, false) - } else { - (Seq(AttributeReference("key", StringType)(), - AttributeReference("value", StringType)()), true) - } - - // Create the transform. - ScriptTransformation( - expressions, - string(script), - attributes, - withFilter, - withScriptIOSchema( - ctx, inRowFormat, recordWriter, outRowFormat, recordReader, schemaLess)) + /** + * Create a logical plan using a having clause. + */ + private def withHavingClause( + ctx: HavingClauseContext, plan: LogicalPlan): LogicalPlan = { + // Note that we add a cast to non-predicate expressions. If the expression itself is + // already boolean, the optimizer will get rid of the unnecessary cast. + val predicate = expression(ctx.booleanExpression) match { + case p: Predicate => p + case e => Cast(e, BooleanType) + } + Filter(predicate, plan) + } - case SqlBaseParser.SELECT => - // Regular select + /** + * Create a logical plan using a where clause. + */ + private def withWhereClause(ctx: WhereClauseContext, plan: LogicalPlan): LogicalPlan = { + Filter(expression(ctx.booleanExpression), plan) + } - // Add lateral views. - val withLateralView = ctx.lateralView.asScala.foldLeft(relation)(withGenerate) + /** + * Add a hive-style transform (SELECT TRANSFORM/MAP/REDUCE) query specification to a logical plan. + */ + private def withTransformQuerySpecification( + ctx: ParserRuleContext, + transformClause: TransformClauseContext, + whereClause: WhereClauseContext, + relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // Add where. + val withFilter = relation.optionalMap(whereClause)(withWhereClause) - // Add where. - val withFilter = withLateralView.optionalMap(where)(filter) + // Create the transform. + val expressions = visitNamedExpressionSeq(transformClause.namedExpressionSeq) - // Add aggregation or a project. - val namedExpressions = expressions.map { - case e: NamedExpression => e - case e: Expression => UnresolvedAlias(e) - } + // Create the attributes. + val (attributes, schemaLess) = if (transformClause.colTypeList != null) { + // Typed return columns. + (createSchema(transformClause.colTypeList).toAttributes, false) + } else if (transformClause.identifierSeq != null) { + // Untyped return columns. + val attrs = visitIdentifierSeq(transformClause.identifierSeq).map { name => + AttributeReference(name, StringType, nullable = true)() + } + (attrs, false) + } else { + (Seq(AttributeReference("key", StringType)(), + AttributeReference("value", StringType)()), true) + } + + // Create the transform. + ScriptTransformation( + expressions, + string(transformClause.script), + attributes, + withFilter, + withScriptIOSchema( + ctx, + transformClause.inRowFormat, + transformClause.recordWriter, + transformClause.outRowFormat, + transformClause.recordReader, + schemaLess + ) + ) + } + + /** + * Add a regular (SELECT) query specification to a logical plan. The query specification + * is the core of the logical plan, this is where sourcing (FROM clause), projection (SELECT), + * aggregation (GROUP BY ... HAVING ...) and filtering (WHERE) takes place. + * + * Note that query hints are ignored (both by the parser and the builder). + */ + private def withSelectQuerySpecification( + ctx: ParserRuleContext, + selectClause: SelectClauseContext, + lateralView: java.util.List[LateralViewContext], + whereClause: WhereClauseContext, + aggregationClause: AggregationClauseContext, + havingClause: HavingClauseContext, + windowClause: WindowClauseContext, + relation: LogicalPlan): LogicalPlan = withOrigin(ctx) { + // Add lateral views. + val withLateralView = lateralView.asScala.foldLeft(relation)(withGenerate) - def createProject() = if (namedExpressions.nonEmpty) { - Project(namedExpressions, withFilter) - } else { - withFilter - } + // Add where. + val withFilter = withLateralView.optionalMap(whereClause)(withWhereClause) - val withProject = if (aggregation == null && having != null) { - if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { - // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. - withHaving(having, createProject()) - } else { - // According to SQL standard, HAVING without GROUP BY means global aggregate. - withHaving(having, Aggregate(Nil, namedExpressions, withFilter)) - } - } else if (aggregation != null) { - val aggregate = withAggregation(aggregation, namedExpressions, withFilter) - aggregate.optionalMap(having)(withHaving) - } else { - // When hitting this branch, `having` must be null. - createProject() - } + val expressions = visitNamedExpressionSeq(selectClause.namedExpressionSeq) + // Add aggregation or a project. + val namedExpressions = expressions.map { + case e: NamedExpression => e + case e: Expression => UnresolvedAlias(e) + } - // Distinct - val withDistinct = if (setQuantifier() != null && setQuantifier().DISTINCT() != null) { - Distinct(withProject) - } else { - withProject - } + def createProject() = if (namedExpressions.nonEmpty) { + Project(namedExpressions, withFilter) + } else { + withFilter + } - // Window - val withWindow = withDistinct.optionalMap(windows)(withWindows) + val withProject = if (aggregationClause == null && havingClause != null) { + if (conf.getConf(SQLConf.LEGACY_HAVING_WITHOUT_GROUP_BY_AS_WHERE)) { + // If the legacy conf is set, treat HAVING without GROUP BY as WHERE. + withHavingClause(havingClause, createProject()) + } else { + // According to SQL standard, HAVING without GROUP BY means global aggregate. + withHavingClause(havingClause, Aggregate(Nil, namedExpressions, withFilter)) + } + } else if (aggregationClause != null) { + val aggregate = withAggregationClause(aggregationClause, namedExpressions, withFilter) + aggregate.optionalMap(havingClause)(withHavingClause) + } else { + // When hitting this branch, `having` must be null. + createProject() + } - // Hint - hints.asScala.foldRight(withWindow)(withHints) + // Distinct + val withDistinct = if ( + selectClause.setQuantifier() != null && + selectClause.setQuantifier().DISTINCT() != null) { + Distinct(withProject) + } else { + withProject } + + // Window + val withWindow = withDistinct.optionalMap(windowClause)(withWindowClause) + + // Hint + selectClause.hints.asScala.foldRight(withWindow)(withHints) } /** * Create a (Hive based) [[ScriptInputOutputSchema]]. */ protected def withScriptIOSchema( - ctx: QuerySpecificationContext, + ctx: ParserRuleContext, inRowFormat: RowFormatContext, recordWriter: Token, outRowFormat: RowFormatContext, @@ -538,7 +634,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitFromClause(ctx: FromClauseContext): LogicalPlan = withOrigin(ctx) { val from = ctx.relation.asScala.foldLeft(null: LogicalPlan) { (left, relation) => val right = plan(relation.relationPrimary) - val join = right.optionalMap(left)(Join(_, _, Inner, None)) + val join = right.optionalMap(left)(Join(_, _, Inner, None, JoinHint.NONE)) withJoinRelations(join, relation) } if (ctx.pivotClause() != null) { @@ -587,13 +683,13 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Add a [[WithWindowDefinition]] operator to a logical plan. */ - private def withWindows( - ctx: WindowsContext, + private def withWindowClause( + ctx: WindowClauseContext, query: LogicalPlan): LogicalPlan = withOrigin(ctx) { // Collect all window specifications defined in the WINDOW clause. val baseWindowMap = ctx.namedWindow.asScala.map { wCtx => - (wCtx.identifier.getText, typedVisit[WindowSpec](wCtx.windowSpec)) + (wCtx.name.getText, typedVisit[WindowSpec](wCtx.windowSpec)) }.toMap // Handle cases like @@ -621,8 +717,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging /** * Add an [[Aggregate]] or [[GroupingSets]] to a logical plan. */ - private def withAggregation( - ctx: AggregationContext, + private def withAggregationClause( + ctx: AggregationClauseContext, selectExpressions: Seq[NamedExpression], query: LogicalPlan): LogicalPlan = withOrigin(ctx) { val groupByExpressions = expressionList(ctx.groupingExpressions) @@ -750,7 +846,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case None => (baseJoinType, None) } - Join(left, plan(join.right), joinType, condition) + Join(left, plan(join.right), joinType, condition, JoinHint.NONE) } } } @@ -818,7 +914,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a logical plan for a sub-query. */ override def visitSubquery(ctx: SubqueryContext): LogicalPlan = withOrigin(ctx) { - plan(ctx.queryNoWith) + plan(ctx.query) } /** @@ -830,14 +926,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitTable(ctx: TableContext): LogicalPlan = withOrigin(ctx) { - UnresolvedRelation(visitTableIdentifier(ctx.tableIdentifier)) + UnresolvedRelation(visitMultipartIdentifier(ctx.multipartIdentifier)) } /** * Create an aliased table reference. This is typically used in FROM clauses. */ override def visitTableName(ctx: TableNameContext): LogicalPlan = withOrigin(ctx) { - val tableId = visitTableIdentifier(ctx.tableIdentifier) + val tableId = visitMultipartIdentifier(ctx.multipartIdentifier) val table = mayApplyAliasPlan(ctx.tableAlias, UnresolvedRelation(tableId)) table.optionalMap(ctx.sample)(withSample) } @@ -855,7 +951,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } val tvf = UnresolvedTableValuedFunction( - func.identifier.getText, func.expression.asScala.map(expression), aliases) + func.funcName.getText, func.expression.asScala.map(expression), aliases) tvf.optionalMap(func.tableAlias.strictIdentifier)(aliasPlan) } @@ -906,7 +1002,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * }}} */ override def visitAliasedQuery(ctx: AliasedQueryContext): LogicalPlan = withOrigin(ctx) { - val relation = plan(ctx.queryNoWith).optionalMap(ctx.sample)(withSample) + val relation = plan(ctx.query).optionalMap(ctx.sample)(withSample) if (ctx.tableAlias.strictIdentifier == null) { // For un-aliased subqueries, use a default alias name that is not likely to conflict with // normal subquery names, so that parent operators can only access the columns in subquery by @@ -954,7 +1050,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a Sequence of Strings for an identifier list. */ override def visitIdentifierSeq(ctx: IdentifierSeqContext): Seq[String] = withOrigin(ctx) { - ctx.identifier.asScala.map(_.getText) + ctx.ident.asScala.map(_.getText) } /* ******************************************************************************************** @@ -976,6 +1072,14 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText)) } + /** + * Create a multi-part identifier. + */ + override def visitMultipartIdentifier( + ctx: MultipartIdentifierContext): Seq[String] = withOrigin(ctx) { + ctx.parts.asScala.map(_.getText) + } + /* ******************************************************************************************** * Expression parsing * ******************************************************************************************** */ @@ -1006,8 +1110,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitNamedExpression(ctx: NamedExpressionContext): Expression = withOrigin(ctx) { val e = expression(ctx.expression) - if (ctx.identifier != null) { - Alias(e, ctx.identifier.getText)() + if (ctx.name != null) { + Alias(e, ctx.name.getText)() } else if (ctx.identifierList != null) { MultiAlias(e, visitIdentifierList(ctx.identifierList)) } else { @@ -1248,6 +1352,21 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + override def visitCurrentDatetime(ctx: CurrentDatetimeContext): Expression = withOrigin(ctx) { + if (conf.ansiParserEnabled) { + ctx.name.getType match { + case SqlBaseParser.CURRENT_DATE => + CurrentDate() + case SqlBaseParser.CURRENT_TIMESTAMP => + CurrentTimestamp() + } + } else { + // If the parser is not in ansi mode, we should return `UnresolvedAttribute`, in case there + // are columns named `CURRENT_DATE` or `CURRENT_TIMESTAMP`. + UnresolvedAttribute.quoted(ctx.name.getText) + } + } + /** * Create a [[Cast]] expression. */ @@ -1313,30 +1432,54 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } } + /** + * Create a Substring/Substr expression. + */ + override def visitSubstring(ctx: SubstringContext): Expression = withOrigin(ctx) { + if (ctx.len != null) { + Substring(expression(ctx.str), expression(ctx.pos), expression(ctx.len)) + } else { + new Substring(expression(ctx.str), expression(ctx.pos)) + } + } + + /** + * Create a Trim expression. + */ + override def visitTrim(ctx: TrimContext): Expression = withOrigin(ctx) { + val srcStr = expression(ctx.srcStr) + val trimStr = Option(ctx.trimStr).map(expression) + Option(ctx.trimOption).map(_.getType).getOrElse(SqlBaseParser.BOTH) match { + case SqlBaseParser.BOTH => + StringTrim(srcStr, trimStr) + case SqlBaseParser.LEADING => + StringTrimLeft(srcStr, trimStr) + case SqlBaseParser.TRAILING => + StringTrimRight(srcStr, trimStr) + case other => + throw new ParseException("Function trim doesn't support with " + + s"type $other. Please use BOTH, LEADING or TRAILING as trim type", ctx) + } + } + + /** + * Create a Overlay expression. + */ + override def visitOverlay(ctx: OverlayContext): Expression = withOrigin(ctx) { + val input = expression(ctx.input) + val replace = expression(ctx.replace) + val position = expression(ctx.position) + val lengthOpt = Option(ctx.length).map(expression) + lengthOpt match { + case Some(length) => Overlay(input, replace, position, length) + case None => new Overlay(input, replace, position) + } + } + /** * Create a (windowed) Function expression. */ override def visitFunctionCall(ctx: FunctionCallContext): Expression = withOrigin(ctx) { - def replaceFunctions( - funcID: FunctionIdentifier, - ctx: FunctionCallContext): FunctionIdentifier = { - val opt = ctx.trimOption - if (opt != null) { - if (ctx.qualifiedName.getText.toLowerCase(Locale.ROOT) != "trim") { - throw new ParseException(s"The specified function ${ctx.qualifiedName.getText} " + - s"doesn't support with option ${opt.getText}.", ctx) - } - opt.getType match { - case SqlBaseParser.BOTH => funcID - case SqlBaseParser.LEADING => funcID.copy(funcName = "ltrim") - case SqlBaseParser.TRAILING => funcID.copy(funcName = "rtrim") - case _ => throw new ParseException("Function trim doesn't support with " + - s"type ${opt.getType}. Please use BOTH, LEADING or Trailing as trim type", ctx) - } - } else { - funcID - } - } // Create the function call. val name = ctx.qualifiedName.getText val isDistinct = Option(ctx.setQuantifier()).exists(_.DISTINCT != null) @@ -1348,9 +1491,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging case expressions => expressions } - val funcId = replaceFunctions(visitFunctionName(ctx.qualifiedName), ctx) - val function = UnresolvedFunction(funcId, arguments, isDistinct) - + val function = UnresolvedFunction(visitFunctionName(ctx.qualifiedName), arguments, isDistinct) // Check if the function is evaluated in a windowed context. ctx.windowSpec match { @@ -1378,16 +1519,19 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging */ override def visitLambda(ctx: LambdaContext): Expression = withOrigin(ctx) { val arguments = ctx.IDENTIFIER().asScala.map { name => - UnresolvedAttribute.quoted(name.getText) + UnresolvedNamedLambdaVariable(UnresolvedAttribute.quoted(name.getText).nameParts) } - LambdaFunction(expression(ctx.expression), arguments) + val function = expression(ctx.expression).transformUp { + case a: UnresolvedAttribute => UnresolvedNamedLambdaVariable(a.nameParts) + } + LambdaFunction(function, arguments) } /** * Create a reference to a window frame, i.e. [[WindowSpecReference]]. */ override def visitWindowRef(ctx: WindowRefContext): WindowSpecReference = withOrigin(ctx) { - WindowSpecReference(ctx.identifier.getText) + WindowSpecReference(ctx.name.getText) } /** @@ -1589,12 +1733,17 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging override def visitTypeConstructor(ctx: TypeConstructorContext): Literal = withOrigin(ctx) { val value = string(ctx.STRING) val valueType = ctx.identifier.getText.toUpperCase(Locale.ROOT) + def toLiteral[T](f: UTF8String => Option[T], t: DataType): Literal = { + f(UTF8String.fromString(value)).map(Literal(_, t)).getOrElse { + throw new ParseException(s"Cannot parse the $valueType value: $value", ctx) + } + } try { valueType match { - case "DATE" => - Literal(Date.valueOf(value)) + case "DATE" => toLiteral(stringToDate, DateType) case "TIMESTAMP" => - Literal(Timestamp.valueOf(value)) + val zoneId = getZoneId(SQLConf.get.sessionLocalTimeZone) + toLiteral(stringToTimestamp(_, zoneId), TimestampType) case "X" => val padding = if (value.length % 2 != 0) "0" else "" Literal(DatatypeConverter.parseHexBinary(padding + value)) @@ -1742,7 +1891,8 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging * Create a [[CalendarInterval]] for a unit value pair. Two unit configuration types are * supported: * - Single unit. - * - From-To unit (only 'YEAR TO MONTH' and 'DAY TO SECOND' are supported). + * - From-To unit ('YEAR TO MONTH', 'DAY TO HOUR', 'DAY TO MINUTE', 'DAY TO SECOND', + * 'HOUR TO MINUTE', 'HOUR TO SECOND' and 'MINUTE TO SECOND' are supported). */ override def visitIntervalField(ctx: IntervalFieldContext): CalendarInterval = withOrigin(ctx) { import ctx._ @@ -1757,8 +1907,18 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging CalendarInterval.fromSingleUnitString(u, s) case ("year", Some("month")) => CalendarInterval.fromYearMonthString(s) + case ("day", Some("hour")) => + CalendarInterval.fromDayTimeString(s, "day", "hour") + case ("day", Some("minute")) => + CalendarInterval.fromDayTimeString(s, "day", "minute") case ("day", Some("second")) => - CalendarInterval.fromDayTimeString(s) + CalendarInterval.fromDayTimeString(s, "day", "second") + case ("hour", Some("minute")) => + CalendarInterval.fromDayTimeString(s, "hour", "minute") + case ("hour", Some("second")) => + CalendarInterval.fromDayTimeString(s, "hour", "second") + case ("minute", Some("second")) => + CalendarInterval.fromDayTimeString(s, "minute", "second") case (from, Some(t)) => throw new ParseException(s"Intervals FROM $from TO $t are not supported.", ctx) } @@ -1859,7 +2019,7 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging } StructField( - identifier.getText, + colName.getText, cleanedDataType, nullable = true, builder.build()) @@ -1888,4 +2048,455 @@ class AstBuilder(conf: SQLConf) extends SqlBaseBaseVisitor[AnyRef] with Logging val structField = StructField(identifier.getText, typedVisit(dataType), nullable = true) if (STRING == null) structField else structField.withComment(string(STRING)) } + + /** + * Create location string. + */ + override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { + string(ctx.STRING) + } + + /** + * Create a [[BucketSpec]]. + */ + override def visitBucketSpec(ctx: BucketSpecContext): BucketSpec = withOrigin(ctx) { + BucketSpec( + ctx.INTEGER_VALUE.getText.toInt, + visitIdentifierList(ctx.identifierList), + Option(ctx.orderedIdentifierList) + .toSeq + .flatMap(_.orderedIdentifier.asScala) + .map { orderedIdCtx => + Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => + if (dir.toLowerCase(Locale.ROOT) != "asc") { + operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) + } + } + + orderedIdCtx.ident.getText + }) + } + + /** + * Convert a table property list into a key-value map. + * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. + */ + override def visitTablePropertyList( + ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { + val properties = ctx.tableProperty.asScala.map { property => + val key = visitTablePropertyKey(property.key) + val value = visitTablePropertyValue(property.value) + key -> value + } + // Check for duplicate property names. + checkDuplicateKeys(properties, ctx) + properties.toMap + } + + /** + * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. + */ + def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.collect { case (key, null) => key } + if (badKeys.nonEmpty) { + operationNotAllowed( + s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props + } + + /** + * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. + */ + def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { + val props = visitTablePropertyList(ctx) + val badKeys = props.filter { case (_, v) => v != null }.keys + if (badKeys.nonEmpty) { + operationNotAllowed( + s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) + } + props.keys.toSeq + } + + /** + * A table property key can either be String or a collection of dot separated elements. This + * function extracts the property key based on whether its a string literal or a table property + * identifier. + */ + override def visitTablePropertyKey(key: TablePropertyKeyContext): String = { + if (key.STRING != null) { + string(key.STRING) + } else { + key.getText + } + } + + /** + * A table property value can be String, Integer, Boolean or Decimal. This function extracts + * the property value based on whether its a string, integer, boolean or decimal literal. + */ + override def visitTablePropertyValue(value: TablePropertyValueContext): String = { + if (value == null) { + null + } else if (value.STRING != null) { + string(value.STRING) + } else if (value.booleanValue != null) { + value.getText.toLowerCase(Locale.ROOT) + } else { + value.getText + } + } + + /** + * Type to keep track of a table header: (identifier, isTemporary, ifNotExists, isExternal). + */ + type TableHeader = (Seq[String], Boolean, Boolean, Boolean) + + /** + * Validate a create table statement and return the [[TableIdentifier]]. + */ + override def visitCreateTableHeader( + ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { + val temporary = ctx.TEMPORARY != null + val ifNotExists = ctx.EXISTS != null + if (temporary && ifNotExists) { + operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + } + val multipartIdentifier = ctx.multipartIdentifier.parts.asScala.map(_.getText) + (multipartIdentifier, temporary, ifNotExists, ctx.EXTERNAL != null) + } + + /** + * Parse a qualified name to a multipart name. + */ + override def visitQualifiedName(ctx: QualifiedNameContext): Seq[String] = withOrigin(ctx) { + ctx.identifier.asScala.map(_.getText) + } + + /** + * Parse a list of transforms. + */ + override def visitTransformList(ctx: TransformListContext): Seq[Transform] = withOrigin(ctx) { + def getFieldReference( + ctx: ApplyTransformContext, + arg: v2.expressions.Expression): FieldReference = { + lazy val name: String = ctx.identifier.getText + arg match { + case ref: FieldReference => + ref + case nonRef => + throw new ParseException( + s"Expected a column reference for transform $name: ${nonRef.describe}", ctx) + } + } + + def getSingleFieldReference( + ctx: ApplyTransformContext, + arguments: Seq[v2.expressions.Expression]): FieldReference = { + lazy val name: String = ctx.identifier.getText + if (arguments.size > 1) { + throw new ParseException(s"Too many arguments for transform $name", ctx) + } else if (arguments.isEmpty) { + throw new ParseException(s"Not enough arguments for transform $name", ctx) + } else { + getFieldReference(ctx, arguments.head) + } + } + + ctx.transforms.asScala.map { + case identityCtx: IdentityTransformContext => + IdentityTransform(FieldReference(typedVisit[Seq[String]](identityCtx.qualifiedName))) + + case applyCtx: ApplyTransformContext => + val arguments = applyCtx.argument.asScala.map(visitTransformArgument) + + applyCtx.identifier.getText match { + case "bucket" => + val numBuckets: Int = arguments.head match { + case LiteralValue(shortValue, ShortType) => + shortValue.asInstanceOf[Short].toInt + case LiteralValue(intValue, IntegerType) => + intValue.asInstanceOf[Int] + case LiteralValue(longValue, LongType) => + longValue.asInstanceOf[Long].toInt + case lit => + throw new ParseException(s"Invalid number of buckets: ${lit.describe}", applyCtx) + } + + val fields = arguments.tail.map(arg => getFieldReference(applyCtx, arg)) + + BucketTransform(LiteralValue(numBuckets, IntegerType), fields) + + case "years" => + YearsTransform(getSingleFieldReference(applyCtx, arguments)) + + case "months" => + MonthsTransform(getSingleFieldReference(applyCtx, arguments)) + + case "days" => + DaysTransform(getSingleFieldReference(applyCtx, arguments)) + + case "hours" => + HoursTransform(getSingleFieldReference(applyCtx, arguments)) + + case name => + ApplyTransform(name, arguments) + } + } + } + + /** + * Parse an argument to a transform. An argument may be a field reference (qualified name) or + * a value literal. + */ + override def visitTransformArgument(ctx: TransformArgumentContext): v2.expressions.Expression = { + withOrigin(ctx) { + val reference = Option(ctx.qualifiedName) + .map(typedVisit[Seq[String]]) + .map(FieldReference(_)) + val literal = Option(ctx.constant) + .map(typedVisit[Literal]) + .map(lit => LiteralValue(lit.value, lit.dataType)) + reference.orElse(literal) + .getOrElse(throw new ParseException(s"Invalid transform argument", ctx)) + } + } + + /** + * Create a table, returning a [[CreateTableStatement]] logical plan. + * + * Expected format: + * {{{ + * CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name + * USING table_provider + * create_table_clauses + * [[AS] select_statement]; + * + * create_table_clauses (order insensitive): + * [OPTIONS table_property_list] + * [PARTITIONED BY (col_name, transform(col_name), transform(constant, col_name), ...)] + * [CLUSTERED BY (col_name, col_name, ...) + * [SORTED BY (col_name [ASC|DESC], ...)] + * INTO num_buckets BUCKETS + * ] + * [LOCATION path] + * [COMMENT table_comment] + * [TBLPROPERTIES (property_name=property_value, ...)] + * }}} + */ + override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = withOrigin(ctx) { + val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) + if (external) { + operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) + } + + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + + val schema = Option(ctx.colTypeList()).map(createSchema) + val partitioning: Seq[Transform] = + Option(ctx.partitioning).map(visitTransformList).getOrElse(Nil) + val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) + val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + + val provider = ctx.tableProvider.qualifiedName.getText + val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) + val comment = Option(ctx.comment).map(string) + + Option(ctx.query).map(plan) match { + case Some(_) if temp => + operationNotAllowed("CREATE TEMPORARY TABLE ... USING ... AS query", ctx) + + case Some(_) if schema.isDefined => + operationNotAllowed( + "Schema may not be specified in a Create Table As Select (CTAS) statement", + ctx) + + case Some(query) => + CreateTableAsSelectStatement( + table, query, partitioning, bucketSpec, properties, provider, options, location, comment, + ifNotExists = ifNotExists) + + case None if temp => + // CREATE TEMPORARY TABLE ... USING ... is not supported by the catalyst parser. + // Use CREATE TEMPORARY VIEW ... USING ... instead. + operationNotAllowed("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) + + case _ => + CreateTableStatement(table, schema.getOrElse(new StructType), partitioning, bucketSpec, + properties, provider, options, location, comment, ifNotExists = ifNotExists) + } + } + + /** + * Create a [[DropTableStatement]] command. + */ + override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { + DropTableStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.EXISTS != null, + ctx.PURGE != null) + } + + /** + * Create a [[DropViewStatement]] command. + */ + override def visitDropView(ctx: DropViewContext): AnyRef = withOrigin(ctx) { + DropViewStatement( + visitMultipartIdentifier(ctx.multipartIdentifier()), + ctx.EXISTS != null) + } + + /** + * Parse new column info from ADD COLUMN into a QualifiedColType. + */ + override def visitQualifiedColTypeWithPosition( + ctx: QualifiedColTypeWithPositionContext): QualifiedColType = withOrigin(ctx) { + if (ctx.colPosition != null) { + operationNotAllowed("ALTER TABLE table ADD COLUMN ... FIRST | AFTER otherCol", ctx) + } + + QualifiedColType( + typedVisit[Seq[String]](ctx.name), + typedVisit[DataType](ctx.dataType), + Option(ctx.comment).map(string)) + } + + /** + * Parse a [[AlterTableAddColumnsStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 + * ADD COLUMNS (col_name data_type [COMMENT col_comment], ...); + * }}} + */ + override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { + AlterTableAddColumnsStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.columns.qualifiedColTypeWithPosition.asScala.map(typedVisit[QualifiedColType]) + ) + } + + /** + * Parse a [[AlterTableRenameColumnStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 RENAME COLUMN a.b.c TO x + * }}} + */ + override def visitRenameTableColumn( + ctx: RenameTableColumnContext): LogicalPlan = withOrigin(ctx) { + AlterTableRenameColumnStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + ctx.from.identifier.asScala.map(_.getText), + ctx.to.getText) + } + + /** + * Parse a [[AlterTableAlterColumnStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint + * ALTER TABLE table1 ALTER COLUMN a.b.c TYPE bigint COMMENT 'new comment' + * ALTER TABLE table1 ALTER COLUMN a.b.c COMMENT 'new comment' + * }}} + */ + override def visitAlterTableColumn( + ctx: AlterTableColumnContext): LogicalPlan = withOrigin(ctx) { + val verb = if (ctx.CHANGE != null) "CHANGE" else "ALTER" + if (ctx.colPosition != null) { + operationNotAllowed(s"ALTER TABLE table $verb COLUMN ... FIRST | AFTER otherCol", ctx) + } + + if (ctx.dataType == null && ctx.comment == null) { + operationNotAllowed(s"ALTER TABLE table $verb COLUMN requires a TYPE or a COMMENT", ctx) + } + + AlterTableAlterColumnStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + typedVisit[Seq[String]](ctx.qualifiedName), + Option(ctx.dataType).map(typedVisit[DataType]), + Option(ctx.comment).map(string)) + } + + /** + * Parse a [[AlterTableDropColumnsStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE table1 DROP COLUMN a.b.c + * ALTER TABLE table1 DROP COLUMNS a.b.c, x, y + * }}} + */ + override def visitDropTableColumns( + ctx: DropTableColumnsContext): LogicalPlan = withOrigin(ctx) { + val columnsToDrop = ctx.columns.qualifiedName.asScala.map(typedVisit[Seq[String]]) + AlterTableDropColumnsStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + columnsToDrop) + } + + /** + * Parse [[AlterViewSetPropertiesStatement]] or [[AlterTableSetPropertiesStatement]] commands. + * + * For example: + * {{{ + * ALTER TABLE table SET TBLPROPERTIES ('comment' = new_comment); + * ALTER VIEW view SET TBLPROPERTIES ('comment' = new_comment); + * }}} + */ + override def visitSetTableProperties( + ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { + val identifier = visitMultipartIdentifier(ctx.multipartIdentifier) + val properties = visitPropertyKeyValues(ctx.tablePropertyList) + if (ctx.VIEW != null) { + AlterViewSetPropertiesStatement(identifier, properties) + } else { + AlterTableSetPropertiesStatement(identifier, properties) + } + } + + /** + * Parse [[AlterViewUnsetPropertiesStatement]] or [[AlterTableUnsetPropertiesStatement]] commands. + * + * For example: + * {{{ + * ALTER TABLE table UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + * ALTER VIEW view UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + * }}} + */ + override def visitUnsetTableProperties( + ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { + val identifier = visitMultipartIdentifier(ctx.multipartIdentifier) + val properties = visitPropertyKeys(ctx.tablePropertyList) + val ifExists = ctx.EXISTS != null + if (ctx.VIEW != null) { + AlterViewUnsetPropertiesStatement(identifier, properties, ifExists) + } else { + AlterTableUnsetPropertiesStatement(identifier, properties, ifExists) + } + } + + /** + * Create an [[AlterTableSetLocationStatement]] command. + * + * For example: + * {{{ + * ALTER TABLE table SET LOCATION "loc"; + * }}} + */ + override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { + AlterTableSetLocationStatement( + visitMultipartIdentifier(ctx.multipartIdentifier), + visitLocationSpec(ctx.locationSpec)) + } } 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 7d8cb1f18b4b5..932e795f1d0bc 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 @@ -57,6 +57,13 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { } } + /** Creates a multi-part identifier for a given SQL string */ + override def parseMultipartIdentifier(sqlText: String): Seq[String] = { + parse(sqlText) { parser => + astBuilder.visitSingleMultipartIdentifier(parser.singleMultipartIdentifier()) + } + } + /** * Creates StructType for a given SQL string, which is a comma separated list of field * definitions which will preserve the correct Hive metadata. @@ -85,6 +92,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { lexer.removeErrorListeners() lexer.addErrorListener(ParseErrorListener) lexer.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced + lexer.ansi = SQLConf.get.ansiParserEnabled val tokenStream = new CommonTokenStream(lexer) val parser = new SqlBaseParser(tokenStream) @@ -92,6 +100,7 @@ abstract class AbstractSqlParser extends ParserInterface with Logging { parser.removeErrorListeners() parser.addErrorListener(ParseErrorListener) parser.legacy_setops_precedence_enbled = SQLConf.get.setOpsPrecedenceEnforced + parser.ansi = SQLConf.get.ansiParserEnabled try { try { @@ -131,7 +140,7 @@ class CatalystSqlParser(conf: SQLConf) extends AbstractSqlParser { /** For test-only. */ object CatalystSqlParser extends AbstractSqlParser { - val astBuilder = new AstBuilder(new SQLConf()) + val astBuilder = new AstBuilder(SQLConf.get) } /** @@ -194,8 +203,17 @@ case object ParseErrorListener extends BaseErrorListener { charPositionInLine: Int, msg: String, e: RecognitionException): Unit = { - val position = Origin(Some(line), Some(charPositionInLine)) - throw new ParseException(None, msg, position, position) + val (start, stop) = offendingSymbol match { + case token: CommonToken => + val start = Origin(Some(line), Some(token.getCharPositionInLine)) + val length = token.getStopIndex - token.getStartIndex + 1 + val stop = Origin(Some(line), Some(token.getCharPositionInLine + length)) + (start, stop) + case _ => + val start = Origin(Some(line), Some(charPositionInLine)) + (start, start) + } + throw new ParseException(None, msg, start, stop) } } @@ -247,6 +265,14 @@ class ParseException( */ case object PostProcessor extends SqlBaseBaseListener { + /** Throws error message when exiting a explicitly captured wrong identifier rule */ + override def exitErrorIdent(ctx: SqlBaseParser.ErrorIdentContext): Unit = { + val ident = ctx.getParent.getText + + throw new ParseException(s"Possibly unquoted identifier $ident detected. " + + s"Please consider quoting it with back-quotes as `$ident`", ctx) + } + /** Remove the back ticks from an Identifier. */ override def exitQuotedIdentifier(ctx: SqlBaseParser.QuotedIdentifierContext): Unit = { replaceTokenByIdentifier(ctx, 1) { token => 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 75240d2196222..77e357ad073da 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 @@ -52,6 +52,12 @@ trait ParserInterface { @throws[ParseException]("Text cannot be parsed to a FunctionIdentifier") def parseFunctionIdentifier(sqlText: String): FunctionIdentifier + /** + * Parse a string to a multi-part identifier. + */ + @throws[ParseException]("Text cannot be parsed to a multi-part identifier") + def parseMultipartIdentifier(sqlText: String): Seq[String] + /** * Parse a string to a [[StructType]]. The passed SQL string should be a comma separated list * of field definitions which will preserve the correct Hive metadata. diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala index 84be677e438a6..51d2a73ea97b7 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.planning +import scala.collection.mutable + import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ @@ -98,12 +100,13 @@ object PhysicalOperation extends PredicateHelper { * value). */ object ExtractEquiJoinKeys extends Logging with PredicateHelper { - /** (joinType, leftKeys, rightKeys, condition, leftChild, rightChild) */ + /** (joinType, leftKeys, rightKeys, condition, leftChild, rightChild, joinHint) */ type ReturnType = - (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) + (JoinType, Seq[Expression], Seq[Expression], + Option[Expression], LogicalPlan, LogicalPlan, JoinHint) - def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { - case join @ Join(left, right, joinType, condition) => + def unapply(join: Join): Option[ReturnType] = join match { + case Join(left, right, joinType, condition, hint) => logDebug(s"Considering join on: $condition") // Find equi-join predicates that can be evaluated before the join, and thus can be used // as join keys. @@ -115,29 +118,32 @@ object ExtractEquiJoinKeys extends Logging with PredicateHelper { // Replace null with default value for joining key, then those rows with null in it could // be joined together case EqualNullSafe(l, r) if canEvaluate(l, left) && canEvaluate(r, right) => - Some((Coalesce(Seq(l, Literal.default(l.dataType))), - Coalesce(Seq(r, Literal.default(r.dataType))))) + Seq((Coalesce(Seq(l, Literal.default(l.dataType))), + Coalesce(Seq(r, Literal.default(r.dataType)))), + (IsNull(l), IsNull(r)) + ) case EqualNullSafe(l, r) if canEvaluate(l, right) && canEvaluate(r, left) => - Some((Coalesce(Seq(r, Literal.default(r.dataType))), - Coalesce(Seq(l, Literal.default(l.dataType))))) + Seq((Coalesce(Seq(r, Literal.default(r.dataType))), + Coalesce(Seq(l, Literal.default(l.dataType)))), + (IsNull(r), IsNull(l)) + ) case other => None } val otherPredicates = predicates.filterNot { case EqualTo(l, r) if l.references.isEmpty || r.references.isEmpty => false - case EqualTo(l, r) => + case Equality(l, r) => canEvaluate(l, left) && canEvaluate(r, right) || canEvaluate(l, right) && canEvaluate(r, left) - case other => false + case _ => false } if (joinKeys.nonEmpty) { val (leftKeys, rightKeys) = joinKeys.unzip logDebug(s"leftKeys:$leftKeys | rightKeys:$rightKeys") - Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right, hint)) } else { None } - case _ => None } } @@ -166,22 +172,24 @@ object ExtractFiltersAndInnerJoins extends PredicateHelper { */ def flattenJoin(plan: LogicalPlan, parentJoinType: InnerLike = Inner) : (Seq[(LogicalPlan, InnerLike)], Seq[Expression]) = plan match { - case Join(left, right, joinType: InnerLike, cond) => + case Join(left, right, joinType: InnerLike, cond, hint) if hint == JoinHint.NONE => val (plans, conditions) = flattenJoin(left, joinType) (plans ++ Seq((right, joinType)), conditions ++ cond.toSeq.flatMap(splitConjunctivePredicates)) - case Filter(filterCondition, j @ Join(left, right, _: InnerLike, joinCondition)) => + case Filter(filterCondition, j @ Join(_, _, _: InnerLike, _, hint)) if hint == JoinHint.NONE => val (plans, conditions) = flattenJoin(j) (plans, conditions ++ splitConjunctivePredicates(filterCondition)) case _ => (Seq((plan, parentJoinType)), Seq.empty) } - def unapply(plan: LogicalPlan): Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])] + def unapply(plan: LogicalPlan) + : Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])] = plan match { - case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _)) => + case f @ Filter(filterCondition, j @ Join(_, _, joinType: InnerLike, _, hint)) + if hint == JoinHint.NONE => Some(flattenJoin(f)) - case j @ Join(_, _, joinType, _) => + case j @ Join(_, _, joinType, _, hint) if hint == JoinHint.NONE => Some(flattenJoin(j)) case _ => None } 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 ca0cea6ba7de3..4bb15d6b01c86 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.plans +import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode} import org.apache.spark.sql.internal.SQLConf @@ -36,13 +37,8 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT /** * Returns the set of attributes that are output by this node. */ - def outputSet: AttributeSet = AttributeSet(output) - - /** - * All Attributes that appear in expressions from this operator. Note that this set does not - * include attributes that are implicitly referenced by being passed through to the output tuple. - */ - def references: AttributeSet = AttributeSet.fromAttributeSets(expressions.map(_.references)) + @transient + lazy val outputSet: AttributeSet = AttributeSet(output) /** * The set of all attributes that are input to this operator by its children. @@ -55,12 +51,19 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT */ def producedAttributes: AttributeSet = AttributeSet.empty + /** + * All Attributes that appear in expressions from this operator. Note that this set does not + * include attributes that are implicitly referenced by being passed through to the output tuple. + */ + @transient + lazy val references: AttributeSet = { + AttributeSet.fromAttributeSets(expressions.map(_.references)) -- producedAttributes + } + /** * Attributes that are referenced by expressions but not provided by this node's children. - * Subclasses should override this method if they produce attributes internally as it is used by - * assertions designed to prevent the construction of invalid plans. */ - def missingInput: AttributeSet = references -- inputSet -- producedAttributes + final def missingInput: AttributeSet = references -- inputSet /** * Runs [[transformExpressionsDown]] with `rule` on all expressions present @@ -118,7 +121,7 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT case m: Map[_, _] => m case d: DataType => d // Avoid unpacking Structs case stream: Stream[_] => stream.map(recursiveTransform).force - case seq: Traversable[_] => seq.map(recursiveTransform) + case seq: Iterable[_] => seq.map(recursiveTransform) case other: AnyRef => other case null => null } @@ -141,16 +144,16 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT /** Returns all of the expressions present in this query plan operator. */ final def expressions: Seq[Expression] = { // Recursively find all expressions from a traversable. - def seqToExpressions(seq: Traversable[Any]): Traversable[Expression] = seq.flatMap { + def seqToExpressions(seq: Iterable[Any]): Iterable[Expression] = seq.flatMap { case e: Expression => e :: Nil - case s: Traversable[_] => seqToExpressions(s) + case s: Iterable[_] => seqToExpressions(s) case other => Nil } productIterator.flatMap { case e: Expression => e :: Nil case s: Some[_] => seqToExpressions(s.toSeq) - case seq: Traversable[_] => seqToExpressions(seq) + case seq: Iterable[_] => seqToExpressions(seq) case other => Nil }.toSeq } @@ -172,9 +175,9 @@ abstract class QueryPlan[PlanType <: QueryPlan[PlanType]] extends TreeNode[PlanT */ protected def statePrefix = if (missingInput.nonEmpty && children.nonEmpty) "!" else "" - override def simpleString: String = statePrefix + super.simpleString + override def simpleString(maxFields: Int): String = statePrefix + super.simpleString(maxFields) - override def verboseString: String = simpleString + override def verboseString(maxFields: Int): String = simpleString(maxFields) /** * All the subqueries of current plan. @@ -278,7 +281,7 @@ object QueryPlan extends PredicateHelper { */ def normalizeExprId[T <: Expression](e: T, input: AttributeSeq): T = { e.transformUp { - case s: SubqueryExpression => s.canonicalize(input) + case s: PlanExpression[_] => s.canonicalize(input) case ar: AttributeReference => val ordinal = input.indexOf(ar.exprId) if (ordinal == -1) { @@ -301,4 +304,20 @@ object QueryPlan extends PredicateHelper { Nil } } + + /** + * Converts the query plan to string and appends it via provided function. + */ + def append[T <: QueryPlan[T]]( + plan: => QueryPlan[T], + append: String => Unit, + verbose: Boolean, + addSuffix: Boolean, + maxFields: Int = SQLConf.get.maxToStringFields): Unit = { + try { + plan.treeString(append, verbose, addSuffix, maxFields) + } catch { + case e: AnalysisException => append(e.toString) + } + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index c77849035a975..feea1d2177ef0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -27,8 +27,8 @@ object JoinType { case "outer" | "full" | "fullouter" => FullOuter case "leftouter" | "left" => LeftOuter case "rightouter" | "right" => RightOuter - case "leftsemi" => LeftSemi - case "leftanti" => LeftAnti + case "leftsemi" | "semi" => LeftSemi + case "leftanti" | "anti" => LeftAnti case "cross" => Cross case _ => val supported = Seq( @@ -36,8 +36,8 @@ object JoinType { "outer", "full", "fullouter", "full_outer", "leftouter", "left", "left_outer", "rightouter", "right", "right_outer", - "leftsemi", "left_semi", - "leftanti", "left_anti", + "leftsemi", "left_semi", "semi", + "leftanti", "left_anti", "anti", "cross") throw new IllegalArgumentException(s"Unsupported join type '$typ'. " + @@ -114,3 +114,10 @@ object LeftExistence { case _ => None } } + +object LeftSemiOrAnti { + def unapply(joinType: JoinType): Option[JoinType] = joinType match { + case LeftSemi | LeftAnti => Some(joinType) + case _ => None + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala index 7a927e1e083b5..8441c2c481ec5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/EventTimeWatermark.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.plans.logical +import java.util.concurrent.TimeUnit + import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.unsafe.types.CalendarInterval @@ -27,7 +29,7 @@ object EventTimeWatermark { def getDelayMs(delay: CalendarInterval): Long = { // We define month as `31 days` to simplify calculation. - val millisPerMonth = CalendarInterval.MICROS_PER_DAY / 1000 * 31 + val millisPerMonth = TimeUnit.MICROSECONDS.toMillis(CalendarInterval.MICROS_PER_DAY) * 31 delay.milliseconds + delay.months * millisPerMonth } } diff --git a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala similarity index 81% rename from external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala index 2e5ab0fb3bef9..85958cb43d4f8 100644 --- a/external/kafka-0-8/src/main/scala/org/apache/spark/streaming/kafka/package-info.java +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/IgnoreCachedData.scala @@ -15,7 +15,9 @@ * limitations under the License. */ +package org.apache.spark.sql.catalyst.plans.logical + /** - * Kafka receiver for spark streaming. + * A [[LogicalPlan]] operator that does not use the cached results stored in CacheManager */ -package org.apache.spark.streaming.kafka; +trait IgnoreCachedData extends LogicalPlan {} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala index 5f136629eb15b..ec9bf90247f88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -34,10 +34,10 @@ abstract class LogicalPlan with Logging { /** Returns true if this subtree has data from a streaming data source. */ - def isStreaming: Boolean = children.exists(_.isStreaming == true) + def isStreaming: Boolean = children.exists(_.isStreaming) - override def verboseStringWithSuffix: String = { - super.verboseString + statsCache.map(", " + _.toString).getOrElse("") + override def verboseStringWithSuffix(maxFields: Int): String = { + super.verboseString(maxFields) + statsCache.map(", " + _.toString).getOrElse("") } /** @@ -93,7 +93,7 @@ abstract class LogicalPlan /** * Optionally resolves the given strings to a [[NamedExpression]] using the input from all child * nodes of this LogicalPlan. The attribute is expressed as - * as string in the following form: `[scope].AttributeName.[nested].[fields]...`. + * string in the following form: `[scope].AttributeName.[nested].[fields]...`. */ def resolveChildren( nameParts: Seq[String], @@ -130,6 +130,20 @@ abstract class LogicalPlan * Returns the output ordering that this plan generates. */ def outputOrdering: Seq[SortOrder] = Nil + + /** + * Returns true iff `other`'s output is semantically the same, ie.: + * - it contains the same number of `Attribute`s; + * - references are the same; + * - the order is equal too. + */ + def sameOutput(other: LogicalPlan): Boolean = { + val thisOutput = this.output + val otherOutput = other.output + thisOutput.length == otherOutput.length && thisOutput.zip(otherOutput).forall { + case (a1, a2) => a1.semanticEquals(a2) + } + } } /** @@ -152,10 +166,10 @@ abstract class UnaryNode extends LogicalPlan { override final def children: Seq[LogicalPlan] = child :: Nil /** - * Generates an additional set of aliased constraints by replacing the original constraint - * expressions with the corresponding alias + * Generates all valid constraints including an set of aliased constraints by replacing the + * original constraint expressions with the corresponding alias */ - protected def getAliasedConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { + protected def getAllValidConstraints(projectList: Seq[NamedExpression]): Set[Expression] = { var allConstraints = child.constraints.asInstanceOf[Set[Expression]] projectList.foreach { case a @ Alias(l: Literal, _) => @@ -170,10 +184,10 @@ abstract class UnaryNode extends LogicalPlan { case _ => // Don't change. } - allConstraints -- child.constraints + allConstraints } - override protected def validConstraints: Set[Expression] = child.constraints + override protected lazy val validConstraints: Set[Expression] = child.constraints } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala index 2c248d74869ce..18baced8f3d61 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlanVisitor.scala @@ -37,7 +37,6 @@ trait LogicalPlanVisitor[T] { case p: Project => visitProject(p) case p: Repartition => visitRepartition(p) case p: RepartitionByExpression => visitRepartitionByExpr(p) - case p: ResolvedHint => visitHint(p) case p: Sample => visitSample(p) case p: ScriptTransformation => visitScriptTransform(p) case p: Union => visitUnion(p) @@ -61,8 +60,6 @@ trait LogicalPlanVisitor[T] { def visitGlobalLimit(p: GlobalLimit): T - def visitHint(p: ResolvedHint): T - def visitIntersect(p: Intersect): T def visitJoin(p: Join): T diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala new file mode 100644 index 0000000000000..4a28d879d1145 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/PlanHelper.scala @@ -0,0 +1,53 @@ +/* + * 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.plans.logical + +import org.apache.spark.sql.catalyst.expressions.{Expression, Generator, WindowExpression} +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression + +/** + * [[PlanHelper]] contains utility methods that can be used by Analyzer and Optimizer. + * It can also be container of methods that are common across multiple rules in Analyzer + * and Optimizer. + */ +object PlanHelper { + /** + * Check if there's any expression in this query plan operator that is + * - A WindowExpression but the plan is not Window + * - An AggregateExpresion but the plan is not Aggregate or Window + * - A Generator but the plan is not Generate + * Returns the list of invalid expressions that this operator hosts. This can happen when + * 1. The input query from users contain invalid expressions. + * Example : SELECT * FROM tab WHERE max(c1) > 0 + * 2. Query rewrites inadvertently produce plans that are invalid. + */ + def specialExpressionsInUnsupportedOperator(plan: LogicalPlan): Seq[Expression] = { + val exprs = plan.expressions + val invalidExpressions = exprs.flatMap { root => + root.collect { + case e: WindowExpression + if !plan.isInstanceOf[Window] => e + case e: AggregateExpression + if !(plan.isInstanceOf[Aggregate] || plan.isInstanceOf[Window]) => e + case e: Generator + if !plan.isInstanceOf[Generate] => e + } + } + invalidExpressions + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala index cc352c59dff80..1355003358b9f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/QueryPlanConstraints.scala @@ -50,7 +50,7 @@ trait QueryPlanConstraints extends ConstraintHelper { self: LogicalPlan => * * See [[Canonicalize]] for more details. */ - protected def validConstraints: Set[Expression] = Set.empty + protected lazy val validConstraints: Set[Expression] = Set.empty } trait ConstraintHelper { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index e176e9b82bf33..30bff884b2249 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -33,7 +33,8 @@ case class ScriptTransformation( output: Seq[Attribute], child: LogicalPlan, ioschema: ScriptInputOutputSchema) extends UnaryNode { - override def references: AttributeSet = AttributeSet(input.flatMap(_.references)) + @transient + override lazy val references: AttributeSet = AttributeSet(input.flatMap(_.references)) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala index b3a48860aa63b..c008d776e4794 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala @@ -52,13 +52,11 @@ import org.apache.spark.util.Utils * defaults to the product of children's `sizeInBytes`. * @param rowCount Estimated number of rows. * @param attributeStats Statistics for Attributes. - * @param hints Query hints. */ case class Statistics( sizeInBytes: BigInt, rowCount: Option[BigInt] = None, - attributeStats: AttributeMap[ColumnStat] = AttributeMap(Nil), - hints: HintInfo = HintInfo()) { + attributeStats: AttributeMap[ColumnStat] = AttributeMap(Nil)) { override def toString: String = "Statistics(" + simpleString + ")" @@ -70,8 +68,7 @@ case class Statistics( s"rowCount=${BigDecimal(rowCount.get, new MathContext(3, RoundingMode.HALF_UP)).toString()}" } else { "" - }, - s"hints=$hints" + } ).filter(_.nonEmpty).mkString(", ") } } @@ -93,6 +90,7 @@ case class Statistics( * @param avgLen average length of the values. For fixed-length types, this should be a constant. * @param maxLen maximum length of the values. For fixed-length types, this should be a constant. * @param histogram histogram of the values + * @param version version of statistics saved to or retrieved from the catalog */ case class ColumnStat( distinctCount: Option[BigInt] = None, @@ -101,7 +99,8 @@ case class ColumnStat( nullCount: Option[BigInt] = None, avgLen: Option[Long] = None, maxLen: Option[Long] = None, - histogram: Option[Histogram] = None) { + histogram: Option[Histogram] = None, + version: Int = CatalogColumnStat.VERSION) { // Are distinctCount and nullCount statistics defined? val hasCountStats = distinctCount.isDefined && nullCount.isDefined @@ -120,7 +119,8 @@ case class ColumnStat( nullCount = nullCount, avgLen = avgLen, maxLen = maxLen, - histogram = histogram) + histogram = histogram, + version = version) } /** diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala index 7ff83a9be3622..2cb04c9ec70c5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala @@ -17,15 +17,18 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.{AliasIdentifier} +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.TableChange.{AddColumn, ColumnChange} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.AliasIdentifier import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils import org.apache.spark.util.random.RandomSampler /** @@ -63,8 +66,8 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) !expressions.exists(!_.resolved) && childrenResolved && !hasSpecialExpressions } - override def validConstraints: Set[Expression] = - child.constraints.union(getAliasedConstraints(projectList)) + override lazy val validConstraints: Set[Expression] = + getAllValidConstraints(projectList) } /** @@ -131,7 +134,7 @@ case class Filter(condition: Expression, child: LogicalPlan) override def maxRows: Option[Long] = child.maxRows - override protected def validConstraints: Set[Expression] = { + override protected lazy val validConstraints: Set[Expression] = { val predicates = splitConjunctivePredicates(condition) .filterNot(SubqueryExpression.hasCorrelatedSubquery) child.constraints.union(predicates.toSet) @@ -176,7 +179,7 @@ case class Intersect( leftAttr.withNullability(leftAttr.nullable && rightAttr.nullable) } - override protected def validConstraints: Set[Expression] = + override protected lazy val validConstraints: Set[Expression] = leftConstraints.union(rightConstraints) override def maxRows: Option[Long] = { @@ -196,7 +199,7 @@ case class Except( /** We don't use right.output because those rows get excluded from the set. */ override def output: Seq[Attribute] = left.output - override protected def validConstraints: Set[Expression] = leftConstraints + override protected lazy val validConstraints: Set[Expression] = leftConstraints } /** Factory for constructing new `Union` nodes. */ @@ -229,10 +232,25 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { } } + def duplicateResolved: Boolean = { + children.map(_.outputSet.size).sum == + AttributeSet.fromAttributeSets(children.map(_.outputSet)).size + } + // updating nullability to make all the children consistent - override def output: Seq[Attribute] = - children.map(_.output).transpose.map(attrs => - attrs.head.withNullability(attrs.exists(_.nullable))) + override def output: Seq[Attribute] = { + children.map(_.output).transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.merge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } override lazy val resolved: Boolean = { // allChildrenCompatible needs to be evaluated after childrenResolved @@ -277,7 +295,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan { common ++ others } - override protected def validConstraints: Set[Expression] = { + override protected lazy val validConstraints: Set[Expression] = { children .map(child => rewriteConstraints(children.head.output, child.output, child.constraints)) .reduce(merge(_, _)) @@ -288,7 +306,8 @@ case class Join( left: LogicalPlan, right: LogicalPlan, joinType: JoinType, - condition: Option[Expression]) + condition: Option[Expression], + hint: JoinHint) extends BinaryNode with PredicateHelper { override def output: Seq[Attribute] = { @@ -308,7 +327,7 @@ case class Join( } } - override protected def validConstraints: Set[Expression] = { + override protected lazy val validConstraints: Set[Expression] = { joinType match { case _: InnerLike if condition.isDefined => left.constraints @@ -350,40 +369,180 @@ case class Join( case UsingJoin(_, _) => false case _ => resolvedExceptNatural } + + // Ignore hint for canonicalization + protected override def doCanonicalize(): LogicalPlan = + super.doCanonicalize().asInstanceOf[Join].copy(hint = JoinHint.NONE) + + // Do not include an empty join hint in string description + protected override def stringArgs: Iterator[Any] = super.stringArgs.filter { e => + (!e.isInstanceOf[JoinHint] + || e.asInstanceOf[JoinHint].leftHint.isDefined + || e.asInstanceOf[JoinHint].rightHint.isDefined) + } } /** - * Append data to an existing table. + * Base trait for DataSourceV2 write commands */ -case class AppendData( - table: NamedRelation, - query: LogicalPlan, - isByName: Boolean) extends LogicalPlan { +trait V2WriteCommand extends Command { + def table: NamedRelation + def query: LogicalPlan + override def children: Seq[LogicalPlan] = Seq(query) - override def output: Seq[Attribute] = Seq.empty - override lazy val resolved: Boolean = { - table.resolved && query.resolved && query.output.size == table.output.size && + override lazy val resolved: Boolean = outputResolved + + def outputResolved: Boolean = { + // If the table doesn't require schema match, we don't need to resolve the output columns. + table.skipSchemaResolution || { + table.resolved && query.resolved && query.output.size == table.output.size && query.output.zip(table.output).forall { case (inAttr, outAttr) => // names and types must match, nullability must be compatible inAttr.name == outAttr.name && - DataType.equalsIgnoreCompatibleNullability(outAttr.dataType, inAttr.dataType) && - (outAttr.nullable || !inAttr.nullable) + DataType.equalsIgnoreCompatibleNullability(outAttr.dataType, inAttr.dataType) && + (outAttr.nullable || !inAttr.nullable) } + } + } +} + +/** + * Create a new table with a v2 catalog. + */ +case class CreateV2Table( + catalog: TableCatalog, + tableName: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + properties: Map[String, String], + ignoreIfExists: Boolean) extends Command + +/** + * Create a new table from a select query with a v2 catalog. + */ +case class CreateTableAsSelect( + catalog: TableCatalog, + tableName: Identifier, + partitioning: Seq[Transform], + query: LogicalPlan, + properties: Map[String, String], + writeOptions: Map[String, String], + ignoreIfExists: Boolean) extends Command { + + override def children: Seq[LogicalPlan] = Seq(query) + + override lazy val resolved: Boolean = childrenResolved && { + // the table schema is created from the query schema, so the only resolution needed is to check + // that the columns referenced by the table's partitioning exist in the query schema + val references = partitioning.flatMap(_.references).toSet + references.map(_.fieldNames).forall(query.schema.findNestedField(_).isDefined) } } +/** + * Append data to an existing table. + */ +case class AppendData( + table: NamedRelation, + query: LogicalPlan, + isByName: Boolean) extends V2WriteCommand + object AppendData { def byName(table: NamedRelation, df: LogicalPlan): AppendData = { - new AppendData(table, df, true) + new AppendData(table, df, isByName = true) } def byPosition(table: NamedRelation, query: LogicalPlan): AppendData = { - new AppendData(table, query, false) + new AppendData(table, query, isByName = false) } } +/** + * Overwrite data matching a filter in an existing table. + */ +case class OverwriteByExpression( + table: NamedRelation, + deleteExpr: Expression, + query: LogicalPlan, + isByName: Boolean) extends V2WriteCommand { + override lazy val resolved: Boolean = outputResolved && deleteExpr.resolved +} + +object OverwriteByExpression { + def byName( + table: NamedRelation, df: LogicalPlan, deleteExpr: Expression): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, df, isByName = true) + } + + def byPosition( + table: NamedRelation, query: LogicalPlan, deleteExpr: Expression): OverwriteByExpression = { + OverwriteByExpression(table, deleteExpr, query, isByName = false) + } +} + +/** + * Dynamically overwrite partitions in an existing table. + */ +case class OverwritePartitionsDynamic( + table: NamedRelation, + query: LogicalPlan, + isByName: Boolean) extends V2WriteCommand + +object OverwritePartitionsDynamic { + def byName(table: NamedRelation, df: LogicalPlan): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, df, isByName = true) + } + + def byPosition(table: NamedRelation, query: LogicalPlan): OverwritePartitionsDynamic = { + OverwritePartitionsDynamic(table, query, isByName = false) + } +} + +/** + * Drop a table. + */ +case class DropTable( + catalog: TableCatalog, + ident: Identifier, + ifExists: Boolean) extends Command + +/** + * Alter a table. + */ +case class AlterTable( + catalog: TableCatalog, + ident: Identifier, + table: NamedRelation, + changes: Seq[TableChange]) extends Command { + + override def children: Seq[LogicalPlan] = Seq(table) + + override lazy val resolved: Boolean = childrenResolved && { + changes.forall { + case add: AddColumn => + add.fieldNames match { + case Array(_) => + // a top-level field can always be added + true + case _ => + // the parent field must exist + table.schema.findNestedField(add.fieldNames.init, includeCollections = true).isDefined + } + + case colChange: ColumnChange => + // the column that will be changed must exist + table.schema.findNestedField(colChange.fieldNames, includeCollections = true).isDefined + + case _ => + // property changes require no resolution checks + true + } + } +} + + /** * Insert some data into a table. Note that this plan is unresolved and has to be replaced by the * concrete implementations during analysis. @@ -462,13 +621,15 @@ case class View( output: Seq[Attribute], child: LogicalPlan) extends LogicalPlan with MultiInstanceRelation { + override def producedAttributes: AttributeSet = outputSet + override lazy val resolved: Boolean = child.resolved override def children: Seq[LogicalPlan] = child :: Nil override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"View (${desc.identifier}, ${output.mkString("[", ",", "]")})" } } @@ -484,8 +645,8 @@ case class View( case class With(child: LogicalPlan, cteRelations: Seq[(String, SubqueryAlias)]) extends UnaryNode { override def output: Seq[Attribute] = child.output - override def simpleString: String = { - val cteAliases = Utils.truncatedString(cteRelations.map(_._1), "[", ", ", "]") + override def simpleString(maxFields: Int): String = { + val cteAliases = truncatedString(cteRelations.map(_._1), "[", ", ", "]", maxFields) s"CTE $cteAliases" } @@ -557,7 +718,7 @@ case class Range( override def newInstance(): Range = copy(output = output.map(_.newInstance())) - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"Range ($start, $end, step=$step, splits=$numSlices)" } @@ -575,6 +736,18 @@ case class Range( } } +/** + * This is a Group by operator with the aggregate functions and projections. + * + * @param groupingExpressions expressions for grouping keys + * @param aggregateExpressions expressions for a project list, which could contain + * [[AggregateFunction]]s. + * + * Note: Currently, aggregateExpressions is the project list of this Group by operator. Before + * separating projection from grouping and aggregate, we should avoid expression-level optimization + * on aggregateExpressions, which could reference an expression in groupingExpressions. + * For example, see the rule [[org.apache.spark.sql.catalyst.optimizer.SimplifyExtractValueOps]] + */ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], @@ -593,9 +766,9 @@ case class Aggregate( override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute) override def maxRows: Option[Long] = child.maxRows - override def validConstraints: Set[Expression] = { + override lazy val validConstraints: Set[Expression] = { val nonAgg = aggregateExpressions.filter(_.find(_.isInstanceOf[AggregateExpression]).isEmpty) - child.constraints.union(getAliasedConstraints(nonAgg)) + getAllValidConstraints(nonAgg) } } @@ -692,12 +865,13 @@ case class Expand( projections: Seq[Seq[Expression]], output: Seq[Attribute], child: LogicalPlan) extends UnaryNode { - override def references: AttributeSet = + @transient + override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) // This operator can reuse attributes (for example making them null when doing a roll up) so // the constraints of the child may no longer be valid. - override protected def validConstraints: Set[Expression] = Set.empty[Expression] + override protected lazy val validConstraints: Set[Expression] = Set.empty[Expression] } /** @@ -964,7 +1138,11 @@ case class OneRowRelation() extends LeafNode { override def computeStats(): Statistics = Statistics(sizeInBytes = 1) /** [[org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy()]] does not support 0-arg ctor. */ - override def makeCopy(newArgs: Array[AnyRef]): OneRowRelation = OneRowRelation() + override def makeCopy(newArgs: Array[AnyRef]): OneRowRelation = { + val newCopy = OneRowRelation() + newCopy.copyTagsFrom(this) + newCopy + } } /** A logical plan for `dropDuplicates`. */ 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 cbb626590d1d7..20c15947d16cc 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.util.Utils /** * A general hint for the child that is not yet resolved. This node is generated by the parser and @@ -35,6 +36,7 @@ case class UnresolvedHint(name: String, parameters: Seq[Any], child: LogicalPlan /** * A resolved hint node. The analyzer should convert all [[UnresolvedHint]] into [[ResolvedHint]]. + * This node will be eliminated before optimization starts. */ case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo()) extends UnaryNode { @@ -44,18 +46,149 @@ case class ResolvedHint(child: LogicalPlan, hints: HintInfo = HintInfo()) override def doCanonicalize(): LogicalPlan = child.canonicalized } +/** + * Hint that is associated with a [[Join]] node, with [[HintInfo]] on its left child and on its + * right child respectively. + */ +case class JoinHint(leftHint: Option[HintInfo], rightHint: Option[HintInfo]) { + + override def toString: String = { + Seq( + leftHint.map("leftHint=" + _), + rightHint.map("rightHint=" + _)) + .filter(_.isDefined).map(_.get).mkString(", ") + } +} -case class HintInfo(broadcast: Boolean = false) { +object JoinHint { + val NONE = JoinHint(None, None) +} - /** Must be called when computing stats for a join operator to reset hints. */ - def resetForJoin(): HintInfo = copy(broadcast = false) +/** + * The hint attributes to be applied on a specific node. + * + * @param strategy The preferred join strategy. + */ +case class HintInfo(strategy: Option[JoinStrategyHint] = None) { - override def toString: String = { - val hints = scala.collection.mutable.ArrayBuffer.empty[String] - if (broadcast) { - hints += "broadcast" + /** + * Combine this [[HintInfo]] with another [[HintInfo]] and return the new [[HintInfo]]. + * @param other the other [[HintInfo]] + * @param hintErrorHandler the error handler to notify if any [[HintInfo]] has been overridden + * in this merge. + * + * Currently, for join strategy hints, the new [[HintInfo]] will contain the strategy in this + * [[HintInfo]] if defined, otherwise the strategy in the other [[HintInfo]]. The + * `hintOverriddenCallback` will be called if this [[HintInfo]] and the other [[HintInfo]] + * both have a strategy defined but the join strategies are different. + */ + def merge(other: HintInfo, hintErrorHandler: HintErrorHandler): HintInfo = { + if (this.strategy.isDefined && + other.strategy.isDefined && + this.strategy.get != other.strategy.get) { + hintErrorHandler.hintOverridden(other) } - - if (hints.isEmpty) "none" else hints.mkString("(", ", ", ")") + HintInfo(strategy = this.strategy.orElse(other.strategy)) } + + override def toString: String = strategy.map(s => s"(strategy=$s)").getOrElse("none") +} + +sealed abstract class JoinStrategyHint { + + def displayName: String + def hintAliases: Set[String] + + override def toString: String = displayName +} + +/** + * The enumeration of join strategy hints. + * + * The hinted strategy will be used for the join with which it is associated if doable. In case + * of contradicting strategy hints specified for each side of the join, hints are prioritized as + * BROADCAST over SHUFFLE_MERGE over SHUFFLE_HASH over SHUFFLE_REPLICATE_NL. + */ +object JoinStrategyHint { + + val strategies: Set[JoinStrategyHint] = Set( + BROADCAST, + SHUFFLE_MERGE, + SHUFFLE_HASH, + SHUFFLE_REPLICATE_NL) +} + +/** + * The hint for broadcast hash join or broadcast nested loop join, depending on the availability of + * equi-join keys. + */ +case object BROADCAST extends JoinStrategyHint { + override def displayName: String = "broadcast" + override def hintAliases: Set[String] = Set( + "BROADCAST", + "BROADCASTJOIN", + "MAPJOIN") +} + +/** + * The hint for shuffle sort merge join. + */ +case object SHUFFLE_MERGE extends JoinStrategyHint { + override def displayName: String = "merge" + override def hintAliases: Set[String] = Set( + "SHUFFLE_MERGE", + "MERGE", + "MERGEJOIN") +} + +/** + * The hint for shuffle hash join. + */ +case object SHUFFLE_HASH extends JoinStrategyHint { + override def displayName: String = "shuffle_hash" + override def hintAliases: Set[String] = Set( + "SHUFFLE_HASH") +} + +/** + * The hint for shuffle-and-replicate nested loop join, a.k.a. cartesian product join. + */ +case object SHUFFLE_REPLICATE_NL extends JoinStrategyHint { + override def displayName: String = "shuffle_replicate_nl" + override def hintAliases: Set[String] = Set( + "SHUFFLE_REPLICATE_NL") +} + +/** + * The callback for implementing customized strategies of handling hint errors. + */ +trait HintErrorHandler { + + /** + * Callback for an unknown hint. + * @param name the unrecognized hint name + * @param parameters the hint parameters + */ + def hintNotRecognized(name: String, parameters: Seq[Any]): Unit + + /** + * Callback for relation names specified in a hint that cannot be associated with any relation + * in the current scope. + * @param name the hint name + * @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 + + /** + * Callback for a join hint specified on a relation that is not part of a join. + * @param hint the [[HintInfo]] + */ + def joinNotFoundForJoinHint(hint: HintInfo): Unit + + /** + * Callback for a hint being overridden by another conflicting hint of the same kind. + * @param hint the [[HintInfo]] being overridden + */ + def hintOverridden(hint: HintInfo): Unit } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala index bfb70c2ef4c89..d383532cbd3d3 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/object.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.plans.logical -import scala.language.existentials - import org.apache.spark.api.java.function.FilterFunction import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.{Encoder, Row} @@ -26,9 +24,9 @@ import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke -import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode } +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode} import org.apache.spark.sql.types._ -import org.apache.spark.util.Utils object CatalystSerde { def deserialize[T : Encoder](child: LogicalPlan): DeserializeToObject = { @@ -69,7 +67,8 @@ trait ObjectConsumer extends UnaryNode { assert(child.output.length == 1) // This operator always need all columns of its child, even it doesn't reference to. - override def references: AttributeSet = child.outputSet + @transient + override lazy val references: AttributeSet = child.outputSet def inputObjAttr: Attribute = child.output.head } @@ -122,16 +121,25 @@ object MapPartitionsInR { schema: StructType, encoder: ExpressionEncoder[Row], child: LogicalPlan): LogicalPlan = { - val deserialized = CatalystSerde.deserialize(child)(encoder) - val mapped = MapPartitionsInR( - func, - packageNames, - broadcastVars, - encoder.schema, - schema, - CatalystSerde.generateObjAttr(RowEncoder(schema)), - deserialized) - CatalystSerde.serialize(mapped)(RowEncoder(schema)) + if (SQLConf.get.arrowSparkREnabled) { + MapPartitionsInRWithArrow( + func, + packageNames, + broadcastVars, + encoder.schema, + schema.toAttributes, + child) + } else { + val deserialized = CatalystSerde.deserialize(child)(encoder) + CatalystSerde.serialize(MapPartitionsInR( + func, + packageNames, + broadcastVars, + encoder.schema, + schema, + CatalystSerde.generateObjAttr(RowEncoder(schema)), + deserialized))(RowEncoder(schema)) + } } } @@ -153,6 +161,29 @@ case class MapPartitionsInR( outputObjAttr, child) } +/** + * Similar with `MapPartitionsInR` but serializes and deserializes input/output in + * Arrow format. + * + * This is somewhat similar with `org.apache.spark.sql.execution.python.ArrowEvalPython` + */ +case class MapPartitionsInRWithArrow( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + // This operator always need all columns of its child, even it doesn't reference to. + @transient + override lazy val references: AttributeSet = child.outputSet + + override protected def stringArgs: Iterator[Any] = Iterator( + inputSchema, StructType.fromAttributes(output), child) + + override val producedAttributes = AttributeSet(output) +} + object MapElements { def apply[T : Encoder, U : Encoder]( func: AnyRef, @@ -213,12 +244,12 @@ case class TypedFilter( } def typedCondition(input: Expression): Expression = { - val (funcClass, methodName) = func match { - case m: FilterFunction[_] => classOf[FilterFunction[_]] -> "call" + val funcMethod = func match { + case _: FilterFunction[_] => (classOf[FilterFunction[_]], "call") case _ => FunctionUtils.getFunctionOneName(BooleanType, input.dataType) } - val funcObj = Literal.create(func, ObjectType(funcClass)) - Invoke(funcObj, methodName, BooleanType, input :: Nil) + val funcObj = Literal.create(func, ObjectType(funcMethod._1)) + Invoke(funcObj, funcMethod._2, BooleanType, input :: Nil) } } @@ -234,9 +265,9 @@ object FunctionUtils { } } - def getFunctionOneName(outputDT: DataType, inputDT: DataType): (Class[_], String) = { - // load "scala.Function1" using Java API to avoid requirements of type parameters - Utils.classForName("scala.Function1") -> { + def getFunctionOneName(outputDT: DataType, inputDT: DataType): + (Class[scala.Function1[_, _]], String) = { + classOf[scala.Function1[_, _]] -> { // if a pair of an argument and return types is one of specific types // whose specialized method (apply$mc..$sp) is generated by scalac, // Catalyst generated a direct method call to the specialized method. @@ -437,19 +468,30 @@ object FlatMapGroupsInR { groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], child: LogicalPlan): LogicalPlan = { - val mapped = FlatMapGroupsInR( - func, - packageNames, - broadcastVars, - inputSchema, - schema, - UnresolvedDeserializer(keyDeserializer, groupingAttributes), - UnresolvedDeserializer(valueDeserializer, dataAttributes), - groupingAttributes, - dataAttributes, - CatalystSerde.generateObjAttr(RowEncoder(schema)), - child) - CatalystSerde.serialize(mapped)(RowEncoder(schema)) + if (SQLConf.get.arrowSparkREnabled) { + FlatMapGroupsInRWithArrow( + func, + packageNames, + broadcastVars, + inputSchema, + schema.toAttributes, + UnresolvedDeserializer(keyDeserializer, groupingAttributes), + groupingAttributes, + child) + } else { + CatalystSerde.serialize(FlatMapGroupsInR( + func, + packageNames, + broadcastVars, + inputSchema, + schema, + UnresolvedDeserializer(keyDeserializer, groupingAttributes), + UnresolvedDeserializer(valueDeserializer, dataAttributes), + groupingAttributes, + dataAttributes, + CatalystSerde.generateObjAttr(RowEncoder(schema)), + child))(RowEncoder(schema)) + } } } @@ -464,7 +506,7 @@ case class FlatMapGroupsInR( groupingAttributes: Seq[Attribute], dataAttributes: Seq[Attribute], outputObjAttr: Attribute, - child: LogicalPlan) extends UnaryNode with ObjectProducer{ + child: LogicalPlan) extends UnaryNode with ObjectProducer { override lazy val schema = outputSchema @@ -473,6 +515,30 @@ case class FlatMapGroupsInR( child) } +/** + * Similar with `FlatMapGroupsInR` but serializes and deserializes input/output in + * Arrow format. + * This is also somewhat similar with [[FlatMapGroupsInPandas]]. + */ +case class FlatMapGroupsInRWithArrow( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + output: Seq[Attribute], + keyDeserializer: Expression, + groupingAttributes: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + // This operator always need all columns of its child, even it doesn't reference to. + @transient + override lazy val references: AttributeSet = child.outputSet + + override protected def stringArgs: Iterator[Any] = Iterator( + inputSchema, StructType.fromAttributes(output), keyDeserializer, groupingAttributes, child) + + override val producedAttributes = AttributeSet(output) +} + /** Factory for constructing new `CoGroup` nodes. */ object CoGroup { def apply[K : Encoder, L : Encoder, R : Encoder, OUT : Encoder]( diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala index 254687ec00880..dc2185194d84e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/pythonLogicalOperators.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSet, Expression, PythonUDF} /** * FlatMap groups using an udf: pandas.Dataframe -> pandas.DataFrame. @@ -38,3 +38,43 @@ case class FlatMapGroupsInPandas( */ override val producedAttributes = AttributeSet(output) } + +/** + * Map partitions using an udf: iter(pandas.Dataframe) -> iter(pandas.DataFrame). + * This is used by DataFrame.mapInPandas() + */ +case class MapInPandas( + functionExpr: Expression, + output: Seq[Attribute], + child: LogicalPlan) extends UnaryNode { + + override val producedAttributes = AttributeSet(output) +} + +trait BaseEvalPython extends UnaryNode { + + def udfs: Seq[PythonUDF] + + def resultAttrs: Seq[Attribute] + + override def output: Seq[Attribute] = child.output ++ resultAttrs + + override def producedAttributes: AttributeSet = AttributeSet(resultAttrs) +} + +/** + * A logical plan that evaluates a [[PythonUDF]] + */ +case class BatchEvalPython( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: LogicalPlan) extends BaseEvalPython + +/** + * A logical plan that evaluates a [[PythonUDF]] with Apache Arrow. + */ +case class ArrowEvalPython( + udfs: Seq[PythonUDF], + resultAttrs: Seq[Attribute], + child: LogicalPlan, + evalType: Int) extends BaseEvalPython diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala new file mode 100644 index 0000000000000..9d7dec9ae0ce0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterTableStatements.scala @@ -0,0 +1,78 @@ +/* + * 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.plans.logical.sql + +import org.apache.spark.sql.types.DataType + +/** + * Column data as parsed by ALTER TABLE ... ADD COLUMNS. + */ +case class QualifiedColType(name: Seq[String], dataType: DataType, comment: Option[String]) + +/** + * ALTER TABLE ... ADD COLUMNS command, as parsed from SQL. + */ +case class AlterTableAddColumnsStatement( + tableName: Seq[String], + columnsToAdd: Seq[QualifiedColType]) extends ParsedStatement + +/** + * ALTER TABLE ... CHANGE COLUMN command, as parsed from SQL. + */ +case class AlterTableAlterColumnStatement( + tableName: Seq[String], + column: Seq[String], + dataType: Option[DataType], + comment: Option[String]) extends ParsedStatement + +/** + * ALTER TABLE ... RENAME COLUMN command, as parsed from SQL. + */ +case class AlterTableRenameColumnStatement( + tableName: Seq[String], + column: Seq[String], + newName: String) extends ParsedStatement + +/** + * ALTER TABLE ... DROP COLUMNS command, as parsed from SQL. + */ +case class AlterTableDropColumnsStatement( + tableName: Seq[String], + columnsToDrop: Seq[Seq[String]]) extends ParsedStatement + +/** + * ALTER TABLE ... SET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableSetPropertiesStatement( + tableName: Seq[String], + properties: Map[String, String]) extends ParsedStatement + +/** + * ALTER TABLE ... UNSET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterTableUnsetPropertiesStatement( + tableName: Seq[String], + propertyKeys: Seq[String], + ifExists: Boolean) extends ParsedStatement + +/** + * ALTER TABLE ... SET LOCATION command, as parsed from SQL. + */ +case class AlterTableSetLocationStatement( + tableName: Seq[String], + location: String) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala new file mode 100644 index 0000000000000..bba7f12c94e50 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/AlterViewStatements.scala @@ -0,0 +1,33 @@ +/* + * 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.plans.logical.sql + +/** + * ALTER VIEW ... SET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterViewSetPropertiesStatement( + viewName: Seq[String], + properties: Map[String, String]) extends ParsedStatement + +/** + * ALTER VIEW ... UNSET TBLPROPERTIES command, as parsed from SQL. + */ +case class AlterViewUnsetPropertiesStatement( + viewName: Seq[String], + propertyKeys: Seq[String], + ifExists: Boolean) extends ParsedStatement diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala new file mode 100644 index 0000000000000..190711303e32d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/CreateTableStatement.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.plans.logical.sql + +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.types.StructType + +/** + * A CREATE TABLE command, as parsed from SQL. + * + * This is a metadata-only command and is not used to write data to the created table. + */ +case class CreateTableStatement( + tableName: Seq[String], + tableSchema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + ifNotExists: Boolean) extends ParsedStatement + +/** + * A CREATE TABLE AS SELECT command, as parsed from SQL. + */ +case class CreateTableAsSelectStatement( + tableName: Seq[String], + asSelect: LogicalPlan, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + ifNotExists: 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/sql/DropTableStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala new file mode 100644 index 0000000000000..d41e8a5010257 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropTableStatement.scala @@ -0,0 +1,34 @@ +/* + * 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.plans.logical.sql + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * A DROP TABLE statement, as parsed from SQL. + */ +case class DropTableStatement( + tableName: Seq[String], + ifExists: Boolean, + purge: Boolean) extends ParsedStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala new file mode 100644 index 0000000000000..523158788e834 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/DropViewStatement.scala @@ -0,0 +1,33 @@ +/* + * 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.plans.logical.sql + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * A DROP VIEW statement, as parsed from SQL. + */ +case class DropViewStatement( + viewName: Seq[String], + ifExists: Boolean) extends ParsedStatement { + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala new file mode 100644 index 0000000000000..23fc009fecdc2 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/sql/ParsedStatement.scala @@ -0,0 +1,49 @@ +/* + * 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.plans.logical.sql + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan + +/** + * A logical plan node that contains exactly what was parsed from SQL. + * + * This is used to hold information parsed from SQL when there are multiple implementations of a + * query or command. For example, CREATE TABLE may be implemented by different nodes for v1 and v2. + * Instead of parsing directly to a v1 CreateTable that keeps metadata in CatalogTable, and then + * converting that v1 metadata to the v2 equivalent, the sql [[CreateTableStatement]] plan is + * produced by the parser and converted once into both implementations. + * + * Parsed logical plans are not resolved because they must be converted to concrete logical plans. + * + * Parsed logical plans are located in Catalyst so that as much SQL parsing logic as possible is be + * kept in a [[org.apache.spark.sql.catalyst.parser.AbstractSqlParser]]. + */ +private[sql] abstract class ParsedStatement extends LogicalPlan { + // Redact properties and options when parsed nodes are used by generic methods like toString + override def productIterator: Iterator[Any] = super.productIterator.map { + case mapArg: Map[_, _] => conf.redactOptions(mapArg) + case other => other + } + + override def output: Seq[Attribute] = Seq.empty + + override def children: Seq[LogicalPlan] = Seq.empty + + final override lazy val resolved = false +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala index 111c594a53e52..ffe071ef25b07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.plans.logical.statsEstimation -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Statistics} @@ -39,8 +39,16 @@ object AggregateEstimation { // Multiply distinct counts of group-by columns. This is an upper bound, which assumes // the data contains all combinations of distinct values of group-by columns. var outputRows: BigInt = agg.groupingExpressions.foldLeft(BigInt(1))( - (res, expr) => res * - childStats.attributeStats(expr.asInstanceOf[Attribute]).distinctCount.get) + (res, expr) => { + val columnStat = childStats.attributeStats(expr.asInstanceOf[Attribute]) + val distinctCount = columnStat.distinctCount.get + val distinctValue: BigInt = if (columnStat.nullCount.get > 0) { + distinctCount + 1 + } else { + distinctCount + } + res * distinctValue + }) outputRows = if (agg.groupingExpressions.isEmpty) { // If there's no group-by columns, the output is a single row containing values of aggregate @@ -52,12 +60,14 @@ object AggregateEstimation { outputRows.min(childStats.rowCount.get) } - val outputAttrStats = getOutputMap(childStats.attributeStats, agg.output) + val aliasStats = EstimationUtils.getAliasStats(agg.expressions, childStats.attributeStats) + + val outputAttrStats = getOutputMap( + AttributeMap(childStats.attributeStats.toSeq ++ aliasStats), agg.output) Some(Statistics( sizeInBytes = getOutputSize(agg.output, outputRows, outputAttrStats), rowCount = Some(outputRows), - attributeStats = outputAttrStats, - hints = childStats.hints)) + attributeStats = outputAttrStats)) } else { None } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala index b6c16079d1984..b8c652dc8f12e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/BasicStatsPlanVisitor.scala @@ -47,8 +47,6 @@ object BasicStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { override def visitGlobalLimit(p: GlobalLimit): Statistics = fallback(p) - override def visitHint(p: ResolvedHint): Statistics = fallback(p) - override def visitIntersect(p: Intersect): Statistics = fallback(p) override def visitJoin(p: Join): Statistics = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala index 211a2a0717371..11d2f024c13a0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.plans.logical.statsEstimation import scala.collection.mutable.ArrayBuffer import scala.math.BigDecimal.RoundingMode -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, Expression} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.types.{DecimalType, _} @@ -71,6 +71,18 @@ object EstimationUtils { AttributeMap(output.flatMap(a => inputMap.get(a).map(a -> _))) } + /** + * Returns the stats for aliases of child's attributes + */ + def getAliasStats( + expressions: Seq[Expression], + attributeStats: AttributeMap[ColumnStat]): Seq[(Attribute, ColumnStat)] = { + expressions.collect { + case alias @ Alias(attr: Attribute, _) if attributeStats.contains(attr) => + alias.toAttribute -> attributeStats(attr) + } + } + def getSizePerRow( attributes: Seq[Attribute], attrStats: AttributeMap[ColumnStat] = AttributeMap(Nil)): BigInt = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala index 5a3eeefaedb18..2c5beef43f52a 100755 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala @@ -670,6 +670,14 @@ case class FilterEstimation(plan: Filter) extends Logging { logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft) return None case _ => + if (!colStatsMap.hasMinMaxStats(attrLeft)) { + logDebug("[CBO] No min/max statistics for " + attrLeft) + return None + } + if (!colStatsMap.hasMinMaxStats(attrRight)) { + logDebug("[CBO] No min/max statistics for " + attrRight) + return None + } } val colStatLeft = colStatsMap(attrLeft) @@ -879,13 +887,13 @@ case class ColumnStatsMap(originalMap: AttributeMap[ColumnStat]) { } def hasCountStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasCountStats) def hasDistinctCount(a: Attribute): Boolean = - get(a).map(_.distinctCount.isDefined).getOrElse(false) + get(a).exists(_.distinctCount.isDefined) def hasMinMaxStats(a: Attribute): Boolean = - get(a).map(_.hasCountStats).getOrElse(false) + get(a).exists(_.hasMinMaxStats) /** * Gets column stat for the given attribute. Prefer the column stat in updatedMap than that in diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala index 2543e38a92c0a..19a0d1279cc32 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala @@ -56,7 +56,7 @@ case class JoinEstimation(join: Join) extends Logging { case _ if !rowCountsExist(join.left, join.right) => None - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _) => + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, _, _, _, _) => // 1. Compute join selectivity val joinKeyPairs = extractJoinKeysWithColStats(leftKeys, rightKeys) val (numInnerJoinedRows, keyStatsAfterJoin) = computeCardinalityAndStats(joinKeyPairs) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala index 489eb904ffd05..6925423f003ba 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala @@ -26,14 +26,10 @@ object ProjectEstimation { def estimate(project: Project): Option[Statistics] = { if (rowCountsExist(project.child)) { val childStats = project.child.stats - val inputAttrStats = childStats.attributeStats - // Match alias with its child's column stat - val aliasStats = project.expressions.collect { - case alias @ Alias(attr: Attribute, _) if inputAttrStats.contains(attr) => - alias.toAttribute -> inputAttrStats(attr) - } + val aliasStats = EstimationUtils.getAliasStats(project.expressions, childStats.attributeStats) + val outputAttrStats = - getOutputMap(AttributeMap(inputAttrStats.toSeq ++ aliasStats), project.output) + getOutputMap(AttributeMap(childStats.attributeStats.toSeq ++ aliasStats), project.output) Some(childStats.copy( sizeInBytes = getOutputSize(project.output, childStats.rowCount.get, outputAttrStats), attributeStats = outputAttrStats)) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala index ee43f9126386b..da36db7ae1f5f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/SizeInBytesOnlyStatsPlanVisitor.scala @@ -44,7 +44,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { } // Don't propagate rowCount and attributeStats, since they are not estimated here. - Statistics(sizeInBytes = sizeInBytes, hints = p.child.stats.hints) + Statistics(sizeInBytes = sizeInBytes) } /** @@ -60,8 +60,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { if (p.groupingExpressions.isEmpty) { Statistics( sizeInBytes = EstimationUtils.getOutputSize(p.output, outputRowCount = 1), - rowCount = Some(1), - hints = p.child.stats.hints) + rowCount = Some(1)) } else { visitUnaryNode(p) } @@ -87,19 +86,15 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { // Don't propagate column stats, because we don't know the distribution after limit Statistics( sizeInBytes = EstimationUtils.getOutputSize(p.output, rowCount, childStats.attributeStats), - rowCount = Some(rowCount), - hints = childStats.hints) + rowCount = Some(rowCount)) } - override def visitHint(p: ResolvedHint): Statistics = p.child.stats.copy(hints = p.hints) - override def visitIntersect(p: Intersect): Statistics = { val leftSize = p.left.stats.sizeInBytes val rightSize = p.right.stats.sizeInBytes val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize Statistics( - sizeInBytes = sizeInBytes, - hints = p.left.stats.hints.resetForJoin()) + sizeInBytes = sizeInBytes) } override def visitJoin(p: Join): Statistics = { @@ -108,10 +103,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { // LeftSemi and LeftAnti won't ever be bigger than left p.left.stats case _ => - // Make sure we don't propagate isBroadcastable in other joins, because - // they could explode the size. - val stats = default(p) - stats.copy(hints = stats.hints.resetForJoin()) + default(p) } } @@ -121,7 +113,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { if (limit == 0) { // sizeInBytes can't be zero, or sizeInBytes of BinaryNode will also be zero // (product of children). - Statistics(sizeInBytes = 1, rowCount = Some(0), hints = childStats.hints) + Statistics(sizeInBytes = 1, rowCount = Some(0)) } else { // The output row count of LocalLimit should be the sum of row counts from each partition. // However, since the number of partitions is not available here, we just use statistics of @@ -147,7 +139,7 @@ object SizeInBytesOnlyStatsPlanVisitor extends LogicalPlanVisitor[Statistics] { } val sampleRows = p.child.stats.rowCount.map(c => EstimationUtils.ceil(BigDecimal(c) * ratio)) // Don't propagate column stats, because we don't know the distribution after a sample operation - Statistics(sizeInBytes, sampleRows, hints = p.child.stats.hints) + Statistics(sizeInBytes, sampleRows) } override def visitScriptTransform(p: ScriptTransformation): Statistics = default(p) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala index cc1a5e835d9cd..17e1cb416fc8a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -22,13 +22,11 @@ import org.apache.spark.sql.types.{DataType, IntegerType} /** * Specifies how tuples that share common expressions will be distributed when a query is executed - * in parallel on many machines. Distribution can be used to refer to two distinct physical - * properties: - * - Inter-node partitioning of data: In this case the distribution describes how tuples are - * partitioned across physical machines in a cluster. Knowing this property allows some - * operators (e.g., Aggregate) to perform partition local operations instead of global ones. - * - Intra-partition ordering of data: In this case the distribution describes guarantees made - * about how tuples are distributed within a single partition. + * in parallel on many machines. + * + * Distribution here refers to inter-node partitioning of data. That is, it describes how tuples + * are partitioned across physical machines in a cluster. Knowing this property allows some + * operators (e.g., Aggregate) to perform partition local operations instead of global ones. */ sealed trait Distribution { /** @@ -70,9 +68,7 @@ case object AllTuples extends Distribution { /** * Represents data where tuples that share the same values for the `clustering` - * [[Expression Expressions]] will be co-located. Based on the context, this - * can mean such tuples are either co-located in the same partition or they will be contiguous - * within a single partition. + * [[Expression Expressions]] will be co-located in the same partition. */ case class ClusteredDistribution( clustering: Seq[Expression], @@ -118,10 +114,12 @@ case class HashClusteredDistribution( /** * Represents data where tuples have been ordered according to the `ordering` - * [[Expression Expressions]]. This is a strictly stronger guarantee than - * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the - * same value for the ordering expressions are contiguous and will never be split across - * partitions. + * [[Expression Expressions]]. Its requirement is defined as the following: + * - Given any 2 adjacent partitions, all the rows of the second partition must be larger than or + * equal to any row in the first partition, according to the `ordering` expressions. + * + * In other words, this distribution requires the rows to be ordered across partitions, but not + * necessarily within a partition. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { require( @@ -241,12 +239,12 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) /** * Represents a partitioning where rows are split across partitions based on some total ordering of - * the expressions specified in `ordering`. When data is partitioned in this manner the following - * two conditions are guaranteed to hold: - * - All row where the expressions in `ordering` evaluate to the same values will be in the same - * partition. - * - Each partition will have a `min` and `max` row, relative to the given ordering. All rows - * that are in between `min` and `max` in this `ordering` will reside in this partition. + * the expressions specified in `ordering`. When data is partitioned in this manner, it guarantees: + * Given any 2 adjacent partitions, all the rows of the second partition must be larger than any row + * in the first partition, according to the `ordering` expressions. + * + * This is a strictly stronger guarantee than what `OrderedDistribution(ordering)` requires, as + * there is no overlap between partitions. * * This class extends expression primarily so that transformations over expression will descend * into its child. @@ -262,6 +260,22 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) super.satisfies0(required) || { required match { case OrderedDistribution(requiredOrdering) => + // If `ordering` is a prefix of `requiredOrdering`: + // Let's say `ordering` is [a, b] and `requiredOrdering` is [a, b, c]. According to the + // RangePartitioning definition, any [a, b] in a previous partition must be smaller + // than any [a, b] in the following partition. This also means any [a, b, c] in a + // previous partition must be smaller than any [a, b, c] in the following partition. + // Thus `RangePartitioning(a, b)` satisfies `OrderedDistribution(a, b, c)`. + // + // If `requiredOrdering` is a prefix of `ordering`: + // Let's say `ordering` is [a, b, c] and `requiredOrdering` is [a, b]. According to the + // RangePartitioning definition, any [a, b, c] in a previous partition must be smaller + // than any [a, b, c] in the following partition. If there is a [a1, b1] from a previous + // partition which is larger than a [a2, b2] from the following partition, then there + // must be a [a1, b1 c1] larger than [a2, b2, c2], which violates RangePartitioning + // definition. So it's guaranteed that, any [a, b] in a previous partition must not be + // greater(i.e. smaller or equal to) than any [a, b] in the following partition. Thus + // `RangePartitioning(a, b, c)` satisfies `OrderedDistribution(a, b)`. val minSize = Seq(requiredOrdering.size, ordering.size).min requiredOrdering.take(minSize) == ordering.take(minSize) case ClusteredDistribution(requiredClustering, _) => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/QueryExecutionMetering.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/QueryExecutionMetering.scala index 62f7541150a6e..7a86433d56c03 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/QueryExecutionMetering.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/QueryExecutionMetering.scala @@ -21,6 +21,8 @@ import scala.collection.JavaConverters._ import com.google.common.util.concurrent.AtomicLongMap +import org.apache.spark.sql.catalyst.util.DateTimeUtils.NANOS_PER_SECOND + case class QueryExecutionMetering() { private val timeMap = AtomicLongMap.create[String]() private val numRunsMap = AtomicLongMap.create[String]() @@ -62,7 +64,11 @@ case class QueryExecutionMetering() { /** Dump statistics about time spent running specific rules. */ def dumpTimeSpent(): String = { val map = timeMap.asMap().asScala - val maxLengthRuleNames = map.keys.map(_.toString.length).max + val maxLengthRuleNames = if (map.isEmpty) { + 0 + } else { + map.keys.map(_.toString.length).max + } val colRuleName = "Rule".padTo(maxLengthRuleNames, " ").mkString val colRunTime = "Effective Time / Total Time".padTo(len = 47, " ").mkString @@ -82,7 +88,7 @@ case class QueryExecutionMetering() { s""" |=== Metrics of Analyzer/Optimizer Rules === |Total number of runs: $totalNumRuns - |Total time: ${totalTime / 1000000000D} seconds + |Total time: ${totalTime / NANOS_PER_SECOND.toDouble} seconds | |$colRuleName $colRunTime $colNumRuns |$ruleMetrics diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala index e991a2dc7462f..3e8a6e0b0b757 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.rules import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.util.sideBySide @@ -66,6 +67,17 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { */ protected def isPlanIntegral(plan: TreeType): Boolean = true + /** + * Executes the batches of rules defined by the subclass, and also tracks timing info for each + * rule using the provided tracker. + * @see [[execute]] + */ + def executeAndTrack(plan: TreeType, tracker: QueryPlanningTracker): TreeType = { + QueryPlanningTracker.withTracker(tracker) { + execute(plan) + } + } + /** * Executes the batches of rules defined by the subclass. The batches are executed serially * using the defined execution strategy. Within each batch, rules are also executed serially. @@ -74,6 +86,14 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { var curPlan = plan val queryExecutionMetrics = RuleExecutor.queryExecutionMeter val planChangeLogger = new PlanChangeLogger() + val tracker: Option[QueryPlanningTracker] = QueryPlanningTracker.get + + // Run the structural integrity checker against the initial input + if (!isPlanIntegral(plan)) { + val message = "The structural integrity of the input plan is broken in " + + s"${this.getClass.getName.stripSuffix("$")}." + throw new TreeNodeException(plan, message, null) + } batches.foreach { batch => val batchStartPlan = curPlan @@ -88,15 +108,19 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { val startTime = System.nanoTime() val result = rule(plan) val runTime = System.nanoTime() - startTime + val effective = !result.fastEquals(plan) - if (!result.fastEquals(plan)) { + if (effective) { queryExecutionMetrics.incNumEffectiveExecution(rule.ruleName) queryExecutionMetrics.incTimeEffectiveExecutionBy(rule.ruleName, runTime) - planChangeLogger.log(rule.ruleName, plan, result) + planChangeLogger.logRule(rule.ruleName, plan, result) } queryExecutionMetrics.incExecutionTimeBy(rule.ruleName, runTime) queryExecutionMetrics.incNumExecution(rule.ruleName) + // Record timing information using QueryPlanningTracker + tracker.foreach(_.recordRuleInvocation(rule.ruleName, runTime, effective)) + // Run the structural integrity checker against the plan after each rule. if (!isPlanIntegral(result)) { val message = s"After applying rule ${rule.ruleName} in batch ${batch.name}, " + @@ -128,15 +152,7 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { lastPlan = curPlan } - if (!batchStartPlan.fastEquals(curPlan)) { - logDebug( - s""" - |=== Result of Batch ${batch.name} === - |${sideBySide(batchStartPlan.treeString, curPlan.treeString).mkString("\n")} - """.stripMargin) - } else { - logTrace(s"Batch ${batch.name} has no effect.") - } + planChangeLogger.logBatch(batch.name, batchStartPlan, curPlan) } curPlan @@ -148,21 +164,46 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { private val logRules = SQLConf.get.optimizerPlanChangeRules.map(Utils.stringToSeq) - def log(ruleName: String, oldPlan: TreeType, newPlan: TreeType): Unit = { + private val logBatches = SQLConf.get.optimizerPlanChangeBatches.map(Utils.stringToSeq) + + def logRule(ruleName: String, oldPlan: TreeType, newPlan: TreeType): Unit = { if (logRules.isEmpty || logRules.get.contains(ruleName)) { - lazy val message = + def message(): String = { s""" |=== Applying Rule ${ruleName} === |${sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n")} """.stripMargin - logLevel match { - case "TRACE" => logTrace(message) - case "DEBUG" => logDebug(message) - case "INFO" => logInfo(message) - case "WARN" => logWarning(message) - case "ERROR" => logError(message) - case _ => logTrace(message) } + + logBasedOnLevel(message) + } + } + + def logBatch(batchName: String, oldPlan: TreeType, newPlan: TreeType): Unit = { + if (logBatches.isEmpty || logBatches.get.contains(batchName)) { + def message(): String = { + if (!oldPlan.fastEquals(newPlan)) { + s""" + |=== Result of Batch ${batchName} === + |${sideBySide(oldPlan.treeString, newPlan.treeString).mkString("\n")} + """.stripMargin + } else { + s"Batch ${batchName} has no effect." + } + } + + logBasedOnLevel(message) + } + } + + private def logBasedOnLevel(f: => String): Unit = { + logLevel match { + case "TRACE" => logTrace(f) + case "DEBUG" => logDebug(f) + case "INFO" => logInfo(f) + case "WARN" => logWarning(f) + case "ERROR" => logError(f) + case _ => logTrace(f) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala index becfa8d982213..0596dc00985a1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.trees import java.util.UUID -import scala.collection.Map +import scala.collection.{mutable, Map} import scala.reflect.ClassTag import org.apache.commons.lang3.ClassUtils @@ -35,9 +35,11 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, Partitioning} +import org.apache.spark.sql.catalyst.util.StringUtils.PlanStringConcat +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.Utils /** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ private class MutableInt(var i: Int) @@ -72,6 +74,9 @@ object CurrentOrigin { } } +// A tag of a `TreeNode`, which defines name and type +case class TreeNodeTag[T](name: String) + // scalastyle:off abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { // scalastyle:on @@ -79,6 +84,24 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { val origin: Origin = CurrentOrigin.get + /** + * A mutable map for holding auxiliary information of this tree node. It will be carried over + * when this node is copied via `makeCopy`, or transformed via `transformUp`/`transformDown`. + */ + private val tags: mutable.Map[TreeNodeTag[_], Any] = mutable.Map.empty + + protected def copyTagsFrom(other: BaseType): Unit = { + tags ++= other.tags + } + + def setTagValue[T](tag: TreeNodeTag[T], value: T): Unit = { + tags(tag) = value + } + + def getTagValue[T](tag: TreeNodeTag[T]): Option[T] = { + tags.get(tag).map(_.asInstanceOf[T]) + } + /** * Returns a Seq of the children of this node. * Children should not change. Immutability required for containsChild optimization @@ -211,6 +234,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } def mapChild(child: Any): Any = child match { case arg: TreeNode[_] if containsChild(arg) => mapTreeNode(arg) + // CaseWhen Case or any tuple type + case (left, right) => (mapChild(left), mapChild(right)) case nonChild: AnyRef => nonChild case null => null } @@ -226,6 +251,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { // `mapValues` is lazy and we need to force it to materialize m.mapValues(mapChild).view.force case arg: TreeNode[_] if containsChild(arg) => mapTreeNode(arg) + case Some(child) => Some(mapChild(child)) case nonChild: AnyRef => nonChild case null => null } @@ -260,6 +286,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { if (this fastEquals afterRule) { mapChildren(_.transformDown(rule)) } else { + // If the transform function replaces this node with a new one, carry over the tags. + afterRule.tags ++= this.tags afterRule.mapChildren(_.transformDown(rule)) } } @@ -273,7 +301,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { */ def transformUp(rule: PartialFunction[BaseType, BaseType]): BaseType = { val afterRuleOnChildren = mapChildren(_.transformUp(rule)) - if (this fastEquals afterRuleOnChildren) { + val newNode = if (this fastEquals afterRuleOnChildren) { CurrentOrigin.withOrigin(origin) { rule.applyOrElse(this, identity[BaseType]) } @@ -282,83 +310,98 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) } } + // If the transform function replaces this node with a new one, carry over the tags. + newNode.tags ++= this.tags + newNode } /** - * Returns a copy of this node where `f` has been applied to all the nodes children. + * Returns a copy of this node where `f` has been applied to all the nodes in `children`. */ def mapChildren(f: BaseType => BaseType): BaseType = { - if (children.nonEmpty) { - var changed = false - def mapChild(child: Any): Any = child match { - case arg: TreeNode[_] if containsChild(arg) => - val newChild = f(arg.asInstanceOf[BaseType]) - if (!(newChild fastEquals arg)) { - changed = true - newChild - } else { - arg - } - case tuple@(arg1: TreeNode[_], arg2: TreeNode[_]) => - val newChild1 = if (containsChild(arg1)) { - f(arg1.asInstanceOf[BaseType]) - } else { - arg1.asInstanceOf[BaseType] - } + if (containsChild.nonEmpty) { + mapChildren(f, forceCopy = false) + } else { + this + } + } - val newChild2 = if (containsChild(arg2)) { - f(arg2.asInstanceOf[BaseType]) - } else { - arg2.asInstanceOf[BaseType] - } + /** + * Returns a copy of this node where `f` has been applied to all the nodes in `children`. + * @param f The transform function to be applied on applicable `TreeNode` elements. + * @param forceCopy Whether to force making a copy of the nodes even if no child has been changed. + */ + private def mapChildren( + f: BaseType => BaseType, + forceCopy: Boolean): BaseType = { + var changed = false - if (!(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { - changed = true - (newChild1, newChild2) - } else { - tuple - } - case other => other - } + def mapChild(child: Any): Any = child match { + case arg: TreeNode[_] if containsChild(arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if (forceCopy || !(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case tuple @ (arg1: TreeNode[_], arg2: TreeNode[_]) => + val newChild1 = if (containsChild(arg1)) { + f(arg1.asInstanceOf[BaseType]) + } else { + arg1.asInstanceOf[BaseType] + } + + val newChild2 = if (containsChild(arg2)) { + f(arg2.asInstanceOf[BaseType]) + } else { + arg2.asInstanceOf[BaseType] + } + + if (forceCopy || !(newChild1 fastEquals arg1) || !(newChild2 fastEquals arg2)) { + changed = true + (newChild1, newChild2) + } else { + tuple + } + case other => other + } - val newArgs = mapProductIterator { + val newArgs = mapProductIterator { + case arg: TreeNode[_] if containsChild(arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if (forceCopy || !(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + case Some(arg: TreeNode[_]) if containsChild(arg) => + val newChild = f(arg.asInstanceOf[BaseType]) + if (forceCopy || !(newChild fastEquals arg)) { + changed = true + Some(newChild) + } else { + Some(arg) + } + case m: Map[_, _] => m.mapValues { case arg: TreeNode[_] if containsChild(arg) => val newChild = f(arg.asInstanceOf[BaseType]) - if (!(newChild fastEquals arg)) { + if (forceCopy || !(newChild fastEquals arg)) { changed = true newChild } else { arg } - case Some(arg: TreeNode[_]) if containsChild(arg) => - val newChild = f(arg.asInstanceOf[BaseType]) - if (!(newChild fastEquals arg)) { - changed = true - Some(newChild) - } else { - Some(arg) - } - case m: Map[_, _] => m.mapValues { - case arg: TreeNode[_] if containsChild(arg) => - val newChild = f(arg.asInstanceOf[BaseType]) - if (!(newChild fastEquals arg)) { - changed = true - newChild - } else { - arg - } - case other => other - }.view.force // `mapValues` is lazy and we need to force it to materialize - case d: DataType => d // Avoid unpacking Structs - case args: Stream[_] => args.map(mapChild).force // Force materialization on stream - case args: Traversable[_] => args.map(mapChild) - case nonChild: AnyRef => nonChild - case null => null - } - if (changed) makeCopy(newArgs) else this - } else { - this + case other => other + }.view.force // `mapValues` is lazy and we need to force it to materialize + case d: DataType => d // Avoid unpacking Structs + case args: Stream[_] => args.map(mapChild).force // Force materialization on stream + case args: Iterable[_] => args.map(mapChild) + case nonChild: AnyRef => nonChild + case null => null } + if (forceCopy || changed) makeCopy(newArgs, forceCopy) else this } /** @@ -374,9 +417,20 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { * that are not present in the productIterator. * @param newArgs the new product arguments. */ - def makeCopy(newArgs: Array[AnyRef]): BaseType = attachTree(this, "makeCopy") { + def makeCopy(newArgs: Array[AnyRef]): BaseType = makeCopy(newArgs, allowEmptyArgs = false) + + /** + * Creates a copy of this type of tree node after a transformation. + * Must be overridden by child classes that have constructor arguments + * that are not present in the productIterator. + * @param newArgs the new product arguments. + * @param allowEmptyArgs whether to allow argument list to be empty. + */ + private def makeCopy( + newArgs: Array[AnyRef], + allowEmptyArgs: Boolean): BaseType = attachTree(this, "makeCopy") { // Skip no-arg constructors that are just there for kryo. - val ctors = getClass.getConstructors.filter(_.getParameterTypes.size != 0) + val ctors = getClass.getConstructors.filter(allowEmptyArgs || _.getParameterTypes.size != 0) if (ctors.isEmpty) { sys.error(s"No valid constructor for $nodeName") } @@ -400,7 +454,9 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { try { CurrentOrigin.withOrigin(origin) { - defaultCtor.newInstance(allArgs.toArray: _*).asInstanceOf[BaseType] + val res = defaultCtor.newInstance(allArgs.toArray: _*).asInstanceOf[BaseType] + res.copyTagsFrom(this) + res } } catch { case e: java.lang.IllegalArgumentException => @@ -417,6 +473,10 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { } } + override def clone(): BaseType = { + mapChildren(_.clone(), forceCopy = true) + } + /** * Returns the name of this type of TreeNode. Defaults to the class name. * Note that we remove the "Exec" suffix for physical operators here. @@ -431,17 +491,17 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { private lazy val allChildren: Set[TreeNode[_]] = (children ++ innerChildren).toSet[TreeNode[_]] /** Returns a string representing the arguments to this node, minus any children */ - def argString: String = stringArgs.flatMap { + def argString(maxFields: Int): String = stringArgs.flatMap { case tn: TreeNode[_] if allChildren.contains(tn) => Nil case Some(tn: TreeNode[_]) if allChildren.contains(tn) => Nil - case Some(tn: TreeNode[_]) => tn.simpleString :: Nil - case tn: TreeNode[_] => tn.simpleString :: Nil + case Some(tn: TreeNode[_]) => tn.simpleString(maxFields) :: Nil + case tn: TreeNode[_] => tn.simpleString(maxFields) :: Nil case seq: Seq[Any] if seq.toSet.subsetOf(allChildren.asInstanceOf[Set[Any]]) => Nil case iter: Iterable[_] if iter.isEmpty => Nil - case seq: Seq[_] => Utils.truncatedString(seq, "[", ", ", "]") :: Nil - case set: Set[_] => Utils.truncatedString(set.toSeq, "{", ", ", "}") :: Nil + case seq: Seq[_] => truncatedString(seq, "[", ", ", "]", maxFields) :: Nil + case set: Set[_] => truncatedString(set.toSeq, "{", ", ", "}", maxFields) :: Nil case array: Array[_] if array.isEmpty => Nil - case array: Array[_] => Utils.truncatedString(array, "[", ", ", "]") :: Nil + case array: Array[_] => truncatedString(array, "[", ", ", "]", maxFields) :: Nil case null => Nil case None => Nil case Some(null) => Nil @@ -454,22 +514,42 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { case other => other :: Nil }.mkString(", ") - /** ONE line description of this node. */ - def simpleString: String = s"$nodeName $argString".trim + /** + * ONE line description of this node. + * @param maxFields Maximum number of fields that will be converted to strings. + * Any elements beyond the limit will be dropped. + */ + def simpleString(maxFields: Int): String = { + s"$nodeName ${argString(maxFields)}".trim + } /** ONE line description of this node with more information */ - def verboseString: String + def verboseString(maxFields: Int): String /** ONE line description of this node with some suffix information */ - def verboseStringWithSuffix: String = verboseString + def verboseStringWithSuffix(maxFields: Int): String = verboseString(maxFields) override def toString: String = treeString /** Returns a string representation of the nodes in this tree */ - def treeString: String = treeString(verbose = true) + final def treeString: String = treeString(verbose = true) + + final def treeString( + verbose: Boolean, + addSuffix: Boolean = false, + maxFields: Int = SQLConf.get.maxToStringFields): String = { + val concat = new PlanStringConcat() - def treeString(verbose: Boolean, addSuffix: Boolean = false): String = { - generateTreeString(0, Nil, new StringBuilder, verbose = verbose, addSuffix = addSuffix).toString + treeString(concat.append, verbose, addSuffix, maxFields) + concat.toString + } + + def treeString( + append: String => Unit, + verbose: Boolean, + addSuffix: Boolean, + maxFields: Int): Unit = { + generateTreeString(0, Nil, append, verbose, "", addSuffix, maxFields) } /** @@ -521,7 +601,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { protected def innerChildren: Seq[TreeNode[_]] = Seq.empty /** - * Appends the string representation of this node and its children to the given StringBuilder. + * Appends the string representation of this node and its children to the given Writer. * * The `i`-th element in `lastChildren` indicates whether the ancestor of the current node at * depth `i + 1` is the last child of its own parent node. The depth of the root node is 0, and @@ -532,44 +612,43 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + append: String => Unit, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { + addSuffix: Boolean = false, + maxFields: Int): Unit = { if (depth > 0) { lastChildren.init.foreach { isLast => - builder.append(if (isLast) " " else ": ") + append(if (isLast) " " else ": ") } - builder.append(if (lastChildren.last) "+- " else ":- ") + append(if (lastChildren.last) "+- " else ":- ") } val str = if (verbose) { - if (addSuffix) verboseStringWithSuffix else verboseString + if (addSuffix) verboseStringWithSuffix(maxFields) else verboseString(maxFields) } else { - simpleString + simpleString(maxFields) } - builder.append(prefix) - builder.append(str) - builder.append("\n") + append(prefix) + append(str) + append("\n") if (innerChildren.nonEmpty) { innerChildren.init.foreach(_.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ false, builder, verbose, - addSuffix = addSuffix)) + depth + 2, lastChildren :+ children.isEmpty :+ false, append, verbose, + addSuffix = addSuffix, maxFields = maxFields)) innerChildren.last.generateTreeString( - depth + 2, lastChildren :+ children.isEmpty :+ true, builder, verbose, - addSuffix = addSuffix) + depth + 2, lastChildren :+ children.isEmpty :+ true, append, verbose, + addSuffix = addSuffix, maxFields = maxFields) } if (children.nonEmpty) { children.init.foreach(_.generateTreeString( - depth + 1, lastChildren :+ false, builder, verbose, prefix, addSuffix)) + depth + 1, lastChildren :+ false, append, verbose, prefix, addSuffix, maxFields)) children.last.generateTreeString( - depth + 1, lastChildren :+ true, builder, verbose, prefix, addSuffix) + depth + 1, lastChildren :+ true, append, verbose, prefix, addSuffix, maxFields) } - - builder } /** @@ -609,7 +688,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { val fieldNames = getConstructorParameterNames(getClass) val fieldValues = productIterator.toSeq ++ otherCopyArgs assert(fieldNames.length == fieldValues.length, s"${getClass.getSimpleName} fields: " + - fieldNames.mkString(", ") + s", values: " + fieldValues.map(_.toString).mkString(", ")) + fieldNames.mkString(", ") + s", values: " + fieldValues.mkString(", ")) fieldNames.zip(fieldValues).map { // If the field value is a child, then use an int to encode it, represents the index of @@ -651,7 +730,7 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { t.forall(_.isInstanceOf[Partitioning]) || t.forall(_.isInstanceOf[DataType]) => JArray(t.map(parseToJson).toList) case t: Seq[_] if t.length > 0 && t.head.isInstanceOf[String] => - JString(Utils.truncatedString(t, "[", ", ", "]")) + JString(truncatedString(t, "[", ", ", "]", SQLConf.get.maxToStringFields)) case t: Seq[_] => JNull case m: Map[_, _] => JNull // if it's a scala object, we can simply keep the full class path. @@ -662,7 +741,8 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] extends Product { try { val fieldNames = getConstructorParameterNames(p.getClass) val fieldValues = p.productIterator.toSeq - assert(fieldNames.length == fieldValues.length) + assert(fieldNames.length == fieldValues.length, s"${getClass.getSimpleName} fields: " + + fieldNames.mkString(", ") + s", values: " + fieldValues.mkString(", ")) ("product-class" -> JString(p.getClass.getName)) :: fieldNames.zip(fieldValues).map { case (name, value) => name -> parseToJson(value) }.toList diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala new file mode 100644 index 0000000000000..98934368205ec --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilder.scala @@ -0,0 +1,130 @@ +/* + * 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 scala.collection.mutable + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.array.ByteArrayMethods + +/** + * A builder of [[ArrayBasedMapData]], which fails if a null map key is detected, and removes + * duplicated map keys w.r.t. the last wins policy. + */ +class ArrayBasedMapBuilder(keyType: DataType, valueType: DataType) extends Serializable { + assert(!keyType.existsRecursively(_.isInstanceOf[MapType]), "key of map cannot be/contain map") + assert(keyType != NullType, "map key cannot be null type.") + + private lazy val keyToIndex = keyType match { + // Binary type data is `byte[]`, which can't use `==` to check equality. + case _: AtomicType | _: CalendarIntervalType if !keyType.isInstanceOf[BinaryType] => + new java.util.HashMap[Any, Int]() + case _ => + // for complex types, use interpreted ordering to be able to compare unsafe data with safe + // data, e.g. UnsafeRow vs GenericInternalRow. + new java.util.TreeMap[Any, Int](TypeUtils.getInterpretedOrdering(keyType)) + } + + // TODO: specialize it + private lazy val keys = mutable.ArrayBuffer.empty[Any] + private lazy val values = mutable.ArrayBuffer.empty[Any] + + private lazy val keyGetter = InternalRow.getAccessor(keyType) + private lazy val valueGetter = InternalRow.getAccessor(valueType) + + def put(key: Any, value: Any): Unit = { + if (key == null) { + throw new RuntimeException("Cannot use null as map key.") + } + + val index = keyToIndex.getOrDefault(key, -1) + if (index == -1) { + if (size >= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + throw new RuntimeException(s"Unsuccessful attempt to build maps with $size elements " + + s"due to exceeding the map size limit ${ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH}.") + } + keyToIndex.put(key, values.length) + keys.append(key) + values.append(value) + } else { + // Overwrite the previous value, as the policy is last wins. + values(index) = value + } + } + + // write a 2-field row, the first field is key and the second field is value. + def put(entry: InternalRow): Unit = { + if (entry.isNullAt(0)) { + throw new RuntimeException("Cannot use null as map key.") + } + put(keyGetter(entry, 0), valueGetter(entry, 1)) + } + + def putAll(keyArray: ArrayData, valueArray: ArrayData): Unit = { + if (keyArray.numElements() != valueArray.numElements()) { + throw new RuntimeException( + "The key array and value array of MapData must have the same length.") + } + + var i = 0 + while (i < keyArray.numElements()) { + put(keyGetter(keyArray, i), valueGetter(valueArray, i)) + i += 1 + } + } + + private def reset(): Unit = { + keyToIndex.clear() + keys.clear() + values.clear() + } + + /** + * Builds the result [[ArrayBasedMapData]] and reset this builder to free up the resources. The + * builder becomes fresh afterward and is ready to take input and build another map. + */ + def build(): ArrayBasedMapData = { + val map = new ArrayBasedMapData( + new GenericArrayData(keys.toArray), new GenericArrayData(values.toArray)) + reset() + map + } + + /** + * Builds a [[ArrayBasedMapData]] from the given key and value array and reset this builder. The + * builder becomes fresh afterward and is ready to take input and build another map. + */ + def from(keyArray: ArrayData, valueArray: ArrayData): ArrayBasedMapData = { + assert(keyToIndex.isEmpty, "'from' can only be called with a fresh ArrayBasedMapBuilder.") + putAll(keyArray, valueArray) + if (keyToIndex.size == keyArray.numElements()) { + // If there is no duplicated map keys, creates the MapData with the input key and value array, + // as they might already in unsafe format and are more efficient. + reset() + new ArrayBasedMapData(keyArray, valueArray) + } else { + build() + } + } + + /** + * Returns the current size of the map which is going to be produced by the current builder. + */ + def size: Int = keys.size +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala index 91b3139443696..5df2af93b94e0 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapData.scala @@ -19,6 +19,12 @@ package org.apache.spark.sql.catalyst.util import java.util.{Map => JavaMap} +/** + * A simple `MapData` implementation which is backed by 2 arrays. + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + */ class ArrayBasedMapData(val keyArray: ArrayData, val valueArray: ArrayData) extends MapData { require(keyArray.numElements() == valueArray.numElements()) @@ -42,11 +48,10 @@ object ArrayBasedMapData { * @param valueConverter This function is applied over all the values of the input map to * obtain the output map's values */ - def apply( - javaMap: JavaMap[_, _], + def apply[K, V]( + javaMap: JavaMap[K, V], keyConverter: (Any) => Any, valueConverter: (Any) => Any): ArrayBasedMapData = { - import scala.language.existentials val keys: Array[Any] = new Array[Any](javaMap.size()) val values: Array[Any] = new Array[Any](javaMap.size()) @@ -83,6 +88,9 @@ object ArrayBasedMapData { * Creates a [[ArrayBasedMapData]] by applying the given converters over * each (key -> value) pair from the given iterator * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + * * @param iterator Input iterator * @param size Number of elements * @param keyConverter This function is applied over all the keys extracted from the @@ -108,6 +116,12 @@ object ArrayBasedMapData { ArrayBasedMapData(keys, values) } + /** + * Creates a [[ArrayBasedMapData]] from a key and value array. + * + * Note that, user is responsible to guarantee that the key array does not have duplicated + * elements, otherwise the behavior is undefined. + */ def apply(keys: Array[_], values: Array[_]): ArrayBasedMapData = { new ArrayBasedMapData(new GenericArrayData(keys), new GenericArrayData(values)) } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala index 4da8ce05fe8a3..ebbf241088f80 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/ArrayData.scala @@ -172,11 +172,7 @@ abstract class ArrayData extends SpecializedGetters with Serializable { val values = new Array[T](size) var i = 0 while (i < size) { - if (isNullAt(i)) { - values(i) = null.asInstanceOf[T] - } else { - values(i) = accessor(this, i).asInstanceOf[T] - } + values(i) = accessor(this, i).asInstanceOf[T] i += 1 } values @@ -187,11 +183,7 @@ abstract class ArrayData extends SpecializedGetters with Serializable { val accessor = InternalRow.getAccessor(elementType) var i = 0 while (i < size) { - if (isNullAt(i)) { - f(i, null) - } else { - f(i, accessor(this, i)) - } + f(i, accessor(this, i)) i += 1 } } @@ -208,11 +200,7 @@ class ArrayDataIndexedSeq[T](arrayData: ArrayData, dataType: DataType) extends I override def apply(idx: Int): T = if (0 <= idx && idx < arrayData.numElements()) { - if (arrayData.isNullAt(idx)) { - null.asInstanceOf[T] - } else { - accessor(arrayData, idx).asInstanceOf[T] - } + accessor(arrayData, idx).asInstanceOf[T] } else { throw new IndexOutOfBoundsException( s"Index $idx must be between 0 and the length of the ArrayData.") diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala index 985f0dc1cd60e..d719a33929fcc 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/BadRecordException.scala @@ -20,6 +20,16 @@ package org.apache.spark.sql.catalyst.util import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.unsafe.types.UTF8String +/** + * Exception thrown when the underlying parser returns a partial result of parsing. + * @param partialResult the partial result of parsing a bad record. + * @param cause the actual exception about why the parser cannot return full result. + */ +case class PartialResultException( + partialResult: InternalRow, + cause: Throwable) + extends Exception(cause) + /** * Exception thrown when the underlying parser meet a bad record and can't parse it. * @param record a function to return the record that cause the parser to fail diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala new file mode 100644 index 0000000000000..984329797c461 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateFormatter.scala @@ -0,0 +1,56 @@ +/* + * 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 +import java.util.Locale + +sealed trait DateFormatter extends Serializable { + def parse(s: String): Int // returns days since epoch + def format(days: Int): String +} + +class Iso8601DateFormatter( + pattern: String, + locale: Locale) extends DateFormatter with DateTimeFormatterHelper { + + @transient + private lazy val formatter = getOrCreateFormatter(pattern, locale) + + override def parse(s: String): Int = { + val localDate = LocalDate.parse(s, formatter) + DateTimeUtils.localDateToDays(localDate) + } + + override def format(days: Int): String = { + LocalDate.ofEpochDay(days).format(formatter) + } +} + +object DateFormatter { + val defaultPattern: String = "yyyy-MM-dd" + val defaultLocale: Locale = Locale.US + + def apply(format: String, locale: Locale): DateFormatter = { + new Iso8601DateFormatter(format, locale) + } + + def apply(format: String): DateFormatter = apply(format, defaultLocale) + + def apply(): DateFormatter = apply(defaultPattern) +} 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 new file mode 100644 index 0000000000000..a7b6309baf614 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeFormatterHelper.scala @@ -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.sql.catalyst.util + +import java.time._ +import java.time.chrono.IsoChronology +import java.time.format.{DateTimeFormatter, DateTimeFormatterBuilder, ResolverStyle} +import java.time.temporal.{ChronoField, TemporalAccessor, TemporalQueries} +import java.util.Locale + +import com.google.common.cache.CacheBuilder + +import org.apache.spark.sql.catalyst.util.DateTimeFormatterHelper._ + +trait DateTimeFormatterHelper { + // Converts the parsed temporal object to ZonedDateTime. It sets time components to zeros + // if they does not exist in the parsed object. + protected def toZonedDateTime( + temporalAccessor: TemporalAccessor, + zoneId: ZoneId): ZonedDateTime = { + // Parsed input might not have time related part. In that case, time component is set to zeros. + val parsedLocalTime = temporalAccessor.query(TemporalQueries.localTime) + val localTime = if (parsedLocalTime == null) LocalTime.MIDNIGHT else parsedLocalTime + // Parsed input must have date component. At least, year must present in temporalAccessor. + val localDate = temporalAccessor.query(TemporalQueries.localDate) + + ZonedDateTime.of(localDate, localTime, zoneId) + } + + // Gets a formatter from the cache or creates new one. The buildFormatter method can be called + // a few times with the same parameters in parallel if the cache does not contain values + // associated to those parameters. Since the formatter is immutable, it does not matter. + // 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 key = (pattern, locale) + var formatter = cache.getIfPresent(key) + if (formatter == null) { + formatter = buildFormatter(pattern, locale) + cache.put(key, formatter) + } + formatter + } +} + +private object DateTimeFormatterHelper { + val cache = CacheBuilder.newBuilder() + .maximumSize(128) + .build[(String, Locale), DateTimeFormatter]() + + def createBuilder(): DateTimeFormatterBuilder = { + new DateTimeFormatterBuilder().parseCaseInsensitive() + } + + 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) + .parseDefaulting(ChronoField.SECOND_OF_MINUTE, 0) + .toFormatter(locale) + .withChronology(IsoChronology.INSTANCE) + .withResolverStyle(ResolverStyle.STRICT) + } + + def buildFormatter(pattern: String, locale: Locale): DateTimeFormatter = { + val builder = createBuilder().appendPattern(pattern) + toFormatter(builder, locale) + } + + lazy val fractionFormatter: DateTimeFormatter = { + val builder = createBuilder() + .append(DateTimeFormatter.ISO_LOCAL_DATE) + .appendLiteral(' ') + .appendValue(ChronoField.HOUR_OF_DAY, 2).appendLiteral(':') + .appendValue(ChronoField.MINUTE_OF_HOUR, 2).appendLiteral(':') + .appendValue(ChronoField.SECOND_OF_MINUTE, 2) + .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true) + toFormatter(builder, TimestampFormatter.defaultLocale) + } +} 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 81d7274607ac8..10a7f9bd550e2 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 @@ -18,13 +18,12 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} -import java.text.{DateFormat, SimpleDateFormat} -import java.util.{Calendar, Locale, TimeZone} -import java.util.concurrent.ConcurrentHashMap -import java.util.function.{Function => JFunction} -import javax.xml.bind.DatatypeConverter +import java.time._ +import java.time.temporal.{ChronoUnit, IsoFields} +import java.util.{Locale, TimeZone} +import java.util.concurrent.TimeUnit._ -import scala.annotation.tailrec +import scala.util.control.NonFatal import org.apache.spark.unsafe.types.UTF8String @@ -44,17 +43,21 @@ object DateTimeUtils { // see http://stackoverflow.com/questions/466321/convert-unix-timestamp-to-julian // it's 2440587.5, rounding up to compatible with Hive final val JULIAN_DAY_OF_EPOCH = 2440588 - final val SECONDS_PER_DAY = 60 * 60 * 24L - final val MICROS_PER_MILLIS = 1000L - final val MICROS_PER_SECOND = MICROS_PER_MILLIS * MILLIS_PER_SECOND - final val MILLIS_PER_SECOND = 1000L - final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L - final val MICROS_PER_DAY = MICROS_PER_SECOND * SECONDS_PER_DAY - final val MILLIS_PER_DAY = SECONDS_PER_DAY * 1000L + // Pre-calculated values can provide an opportunity of additional optimizations + // to the compiler like constants propagation and folding. + final val NANOS_PER_MICROS: Long = 1000 + final val MICROS_PER_MILLIS: Long = 1000 + final val MILLIS_PER_SECOND: Long = 1000 + final val SECONDS_PER_DAY: Long = 24 * 60 * 60 + final val MICROS_PER_SECOND: Long = MILLIS_PER_SECOND * MICROS_PER_MILLIS + final val NANOS_PER_MILLIS: Long = NANOS_PER_MICROS * MICROS_PER_MILLIS + final val NANOS_PER_SECOND: Long = NANOS_PER_MICROS * MICROS_PER_SECOND + final val MICROS_PER_DAY: Long = SECONDS_PER_DAY * MICROS_PER_SECOND + final val MILLIS_PER_MINUTE: Long = 60 * MILLIS_PER_SECOND + final val MILLIS_PER_HOUR: Long = 60 * MILLIS_PER_MINUTE + final val MILLIS_PER_DAY: Long = SECONDS_PER_DAY * MILLIS_PER_SECOND - // number of days in 400 years - final val daysIn400Years: Int = 146097 // number of days between 1.1.1970 and 1.1.2001 final val to2001 = -11323 @@ -63,62 +66,14 @@ object DateTimeUtils { final val toYearZero = to2001 + 7304850 final val TimeZoneGMT = TimeZone.getTimeZone("GMT") final val TimeZoneUTC = TimeZone.getTimeZone("UTC") - final val MonthOf31Days = Set(1, 3, 5, 7, 8, 10, 12) val TIMEZONE_OPTION = "timeZone" def defaultTimeZone(): TimeZone = TimeZone.getDefault() - // Reuse the Calendar object in each thread as it is expensive to create in each method call. - private val threadLocalGmtCalendar = new ThreadLocal[Calendar] { - override protected def initialValue: Calendar = { - Calendar.getInstance(TimeZoneGMT) - } - } - - // `SimpleDateFormat` is not thread-safe. - private val threadLocalTimestampFormat = new ThreadLocal[DateFormat] { - override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - } - } - - def getThreadLocalTimestampFormat(timeZone: TimeZone): DateFormat = { - val sdf = threadLocalTimestampFormat.get() - sdf.setTimeZone(timeZone) - sdf - } - - // `SimpleDateFormat` is not thread-safe. - private val threadLocalDateFormat = new ThreadLocal[DateFormat] { - override def initialValue(): SimpleDateFormat = { - new SimpleDateFormat("yyyy-MM-dd", Locale.US) - } - } - - def getThreadLocalDateFormat(timeZone: TimeZone): DateFormat = { - val sdf = threadLocalDateFormat.get() - sdf.setTimeZone(timeZone) - sdf - } - - private val computedTimeZones = new ConcurrentHashMap[String, TimeZone] - private val computeTimeZone = new JFunction[String, TimeZone] { - override def apply(timeZoneId: String): TimeZone = TimeZone.getTimeZone(timeZoneId) - } - + def getZoneId(timeZoneId: String): ZoneId = ZoneId.of(timeZoneId, ZoneId.SHORT_IDS) def getTimeZone(timeZoneId: String): TimeZone = { - computedTimeZones.computeIfAbsent(timeZoneId, computeTimeZone) - } - - def newDateFormat(formatString: String, timeZone: TimeZone): DateFormat = { - val sdf = new SimpleDateFormat(formatString, Locale.US) - sdf.setTimeZone(timeZone) - // Enable strict parsing, if the input date/format is invalid, it will throw an exception. - // e.g. to parse invalid date '2016-13-12', or '2016-01-12' with invalid format 'yyyy-aa-dd', - // an exception will be throwed. - sdf.setLenient(false) - sdf + TimeZone.getTimeZone(getZoneId(timeZoneId)) } // we should use the exact day as Int, for example, (year, month, day) -> day @@ -127,10 +82,10 @@ object DateTimeUtils { } def millisToDays(millisUtc: Long, timeZone: TimeZone): SQLDate = { - // SPARK-6785: use Math.floor so negative number of days (dates before 1970) + // SPARK-6785: use Math.floorDiv so negative number of days (dates before 1970) // will correctly work as input for function toJavaDate(Int) val millisLocal = millisUtc + timeZone.getOffset(millisUtc) - Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt + Math.floorDiv(millisLocal, MILLIS_PER_DAY).toInt } // reverse of millisToDays @@ -143,51 +98,9 @@ object DateTimeUtils { millisLocal - getOffsetFromLocalMillis(millisLocal, timeZone) } - def dateToString(days: SQLDate): String = - getThreadLocalDateFormat(defaultTimeZone()).format(toJavaDate(days)) - - def dateToString(days: SQLDate, timeZone: TimeZone): String = { - getThreadLocalDateFormat(timeZone).format(toJavaDate(days)) - } - // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(us: SQLTimestamp): String = { - timestampToString(us, defaultTimeZone()) - } - - // Converts Timestamp to string according to Hive TimestampWritable convention. - def timestampToString(us: SQLTimestamp, timeZone: TimeZone): String = { - val ts = toJavaTimestamp(us) - val timestampString = ts.toString - val timestampFormat = getThreadLocalTimestampFormat(timeZone) - val formatted = timestampFormat.format(ts) - - if (timestampString.length > 19 && timestampString.substring(19) != ".0") { - formatted + timestampString.substring(19) - } else { - formatted - } - } - - @tailrec - def stringToTime(s: String): java.util.Date = { - val indexOfGMT = s.indexOf("GMT") - if (indexOfGMT != -1) { - // ISO8601 with a weird time zone specifier (2000-01-01T00:00GMT+01:00) - val s0 = s.substring(0, indexOfGMT) - val s1 = s.substring(indexOfGMT + 3) - // Mapped to 2000-01-01T00:00+01:00 - stringToTime(s0 + s1) - } else if (!s.contains('T')) { - // JDBC escape string - if (s.contains(' ')) { - Timestamp.valueOf(s) - } else { - Date.valueOf(s) - } - } else { - DatatypeConverter.parseDateTime(s).getTime() - } + def timestampToString(tf: TimestampFormatter, us: SQLTimestamp): String = { + tf.format(us) } /** @@ -208,29 +121,14 @@ object DateTimeUtils { * Returns a java.sql.Timestamp from number of micros since epoch. */ def toJavaTimestamp(us: SQLTimestamp): Timestamp = { - // setNanos() will overwrite the millisecond part, so the milliseconds should be - // cut off at seconds - var seconds = us / MICROS_PER_SECOND - var micros = us % MICROS_PER_SECOND - // setNanos() can not accept negative value - if (micros < 0) { - micros += MICROS_PER_SECOND - seconds -= 1 - } - val t = new Timestamp(seconds * 1000) - t.setNanos(micros.toInt * 1000) - t + Timestamp.from(microsToInstant(us)) } /** * Returns the number of micros since epoch from java.sql.Timestamp. */ def fromJavaTimestamp(t: Timestamp): SQLTimestamp = { - if (t != null) { - t.getTime() * 1000L + (t.getNanos().toLong / 1000) % 1000L - } else { - 0L - } + instantToMicros(t.toInstant) } /** @@ -240,7 +138,7 @@ 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 * MICROS_PER_SECOND + nanoseconds / 1000L + SECONDS.toMicros(seconds) + NANOSECONDS.toMicros(nanoseconds) } /** @@ -252,7 +150,7 @@ object DateTimeUtils { val julian_us = us + JULIAN_DAY_OF_EPOCH * MICROS_PER_DAY val day = julian_us / MICROS_PER_DAY val micros = julian_us % MICROS_PER_DAY - (day.toInt, micros * 1000L) + (day.toInt, MICROSECONDS.toNanos(micros)) } /* @@ -263,18 +161,29 @@ object DateTimeUtils { // When the timestamp is negative i.e before 1970, we need to adjust the millseconds portion. // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision. // In millis precision the above needs to be represented as (-157700927877). - Math.floor(us.toDouble / MILLIS_PER_SECOND).toLong + Math.floorDiv(us, MICROS_PER_MILLIS) } /* - * Converts millseconds since epoch to SQLTimestamp. + * Converts milliseconds since epoch to SQLTimestamp. */ def fromMillis(millis: Long): SQLTimestamp = { - millis * 1000L + MILLISECONDS.toMicros(millis) + } + + def microsToEpochDays(epochMicros: SQLTimestamp, zoneId: ZoneId): SQLDate = { + localDateToDays(microsToInstant(epochMicros).atZone(zoneId).toLocalDate) + } + + def epochDaysToMicros(epochDays: SQLDate, zoneId: ZoneId): SQLTimestamp = { + val localDate = LocalDate.ofEpochDay(epochDays) + val zeroLocalTime = LocalTime.MIDNIGHT + val localDateTime = LocalDateTime.of(localDate, zeroLocalTime) + instantToMicros(localDateTime.atZone(zoneId).toInstant) } /** - * Parses a given UTF8 date string to the corresponding a corresponding [[Long]] value. + * Trim and parse a given UTF8 date string to the corresponding a corresponding [[Long]] value. * The return type is [[Option]] in order to distinguish between 0L and null. The following * formats are allowed: * @@ -299,11 +208,7 @@ object DateTimeUtils { * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us]-[h]h:[m]m` * `T[h]h:[m]m:[s]s.[ms][ms][ms][us][us][us]+[h]h:[m]m` */ - def stringToTimestamp(s: UTF8String): Option[SQLTimestamp] = { - stringToTimestamp(s, defaultTimeZone()) - } - - def stringToTimestamp(s: UTF8String, timeZone: TimeZone): Option[SQLTimestamp] = { + def stringToTimestamp(s: UTF8String, timeZoneId: ZoneId): Option[SQLTimestamp] = { if (s == null) { return None } @@ -311,7 +216,7 @@ object DateTimeUtils { val segments: Array[Int] = Array[Int](1, 1, 1, 0, 0, 0, 0, 0, 0) var i = 0 var currentSegmentValue = 0 - val bytes = s.getBytes + val bytes = s.trim.getBytes var j = 0 var digitsMilli = 0 var justTime = false @@ -410,38 +315,55 @@ object DateTimeUtils { segments(6) /= 10 digitsMilli -= 1 } - - if (!justTime && isInvalidDate(segments(0), segments(1), segments(2))) { - return None + try { + val zoneId = if (tz.isEmpty) { + timeZoneId + } else { + val sign = if (tz.get.toChar == '-') -1 else 1 + ZoneOffset.ofHoursMinutes(sign * segments(7), sign * segments(8)) + } + val nanoseconds = MICROSECONDS.toNanos(segments(6)) + val localTime = LocalTime.of(segments(3), segments(4), segments(5), nanoseconds.toInt) + val localDate = if (justTime) { + LocalDate.now(zoneId) + } else { + LocalDate.of(segments(0), segments(1), segments(2)) + } + val localDateTime = LocalDateTime.of(localDate, localTime) + val zonedDateTime = ZonedDateTime.of(localDateTime, zoneId) + val instant = Instant.from(zonedDateTime) + Some(instantToMicros(instant)) + } catch { + case NonFatal(_) => None } + } - if (segments(3) < 0 || segments(3) > 23 || segments(4) < 0 || segments(4) > 59 || - segments(5) < 0 || segments(5) > 59 || segments(6) < 0 || segments(6) > 999999 || - segments(7) < 0 || segments(7) > 23 || segments(8) < 0 || segments(8) > 59) { - return None - } + def instantToMicros(instant: Instant): Long = { + val us = Math.multiplyExact(instant.getEpochSecond, MICROS_PER_SECOND) + val result = Math.addExact(us, NANOSECONDS.toMicros(instant.getNano)) + result + } - val c = if (tz.isEmpty) { - Calendar.getInstance(timeZone) - } else { - Calendar.getInstance( - getTimeZone(f"GMT${tz.get.toChar}${segments(7)}%02d:${segments(8)}%02d")) - } - c.set(Calendar.MILLISECOND, 0) + def microsToInstant(us: Long): Instant = { + val secs = Math.floorDiv(us, MICROS_PER_SECOND) + val mos = Math.floorMod(us, MICROS_PER_SECOND) + Instant.ofEpochSecond(secs, mos * NANOS_PER_MICROS) + } - if (justTime) { - c.set(Calendar.HOUR_OF_DAY, segments(3)) - c.set(Calendar.MINUTE, segments(4)) - c.set(Calendar.SECOND, segments(5)) - } else { - c.set(segments(0), segments(1) - 1, segments(2), segments(3), segments(4), segments(5)) - } + def instantToDays(instant: Instant): Int = { + val seconds = instant.getEpochSecond + val days = Math.floorDiv(seconds, SECONDS_PER_DAY) + days.toInt + } - Some(c.getTimeInMillis * 1000 + segments(6)) + def localDateToDays(localDate: LocalDate): Int = { + Math.toIntExact(localDate.toEpochDay) } + def daysToLocalDate(days: Int): LocalDate = LocalDate.ofEpochDay(days) + /** - * Parses a given UTF8 date string to a corresponding [[Int]] value. + * Trim and parse a given UTF8 date string to a corresponding [[Int]] value. * The return type is [[Option]] in order to distinguish between 0 and null. The following * formats are allowed: * @@ -459,7 +381,7 @@ object DateTimeUtils { val segments: Array[Int] = Array[Int](1, 1, 1) var i = 0 var currentSegmentValue = 0 - val bytes = s.getBytes + val bytes = s.trim.getBytes var j = 0 while (j < bytes.length && (i < 3 && !(bytes(j) == ' ' || bytes(j) == 'T'))) { val b = bytes(j) @@ -485,35 +407,17 @@ object DateTimeUtils { // year should have exact four digits return None } - segments(i) = currentSegmentValue - if (isInvalidDate(segments(0), segments(1), segments(2))) { + if (i < 2 && j < bytes.length) { + // For the `yyyy` and `yyyy-[m]m` formats, entire input must be consumed. return None } - - val c = threadLocalGmtCalendar.get() - c.clear() - c.set(segments(0), segments(1) - 1, segments(2), 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - Some((c.getTimeInMillis / MILLIS_PER_DAY).toInt) - } - - /** - * Return true if the date is invalid. - */ - private def isInvalidDate(year: Int, month: Int, day: Int): Boolean = { - if (year < 0 || year > 9999 || month < 1 || month > 12 || day < 1 || day > 31) { - return true - } - if (month == 2) { - if (isLeapYear(year) && day > 29) { - return true - } else if (!isLeapYear(year) && day > 28) { - return true - } - } else if (!MonthOf31Days.contains(month) && day > 30) { - return true + segments(i) = currentSegmentValue + try { + val localDate = LocalDate.of(segments(0), segments(1), segments(2)) + Some(localDateToDays(localDate)) + } catch { + case NonFatal(_) => None } - false } /** @@ -523,34 +427,16 @@ object DateTimeUtils { microsec + toYearZero * MICROS_PER_DAY } - private def localTimestamp(microsec: SQLTimestamp): SQLTimestamp = { - localTimestamp(microsec, defaultTimeZone()) - } - private def localTimestamp(microsec: SQLTimestamp, timeZone: TimeZone): SQLTimestamp = { - absoluteMicroSecond(microsec) + timeZone.getOffset(microsec / 1000) * 1000L - } - - /** - * Returns the hour value of a given timestamp value. The timestamp is expressed in microseconds. - */ - def getHours(microsec: SQLTimestamp): Int = { - ((localTimestamp(microsec) / MICROS_PER_SECOND / 3600) % 24).toInt + val zoneOffsetUs = MILLISECONDS.toMicros(timeZone.getOffset(MICROSECONDS.toMillis(microsec))) + absoluteMicroSecond(microsec) + zoneOffsetUs } /** * Returns the hour value of a given timestamp value. The timestamp is expressed in microseconds. */ def getHours(microsec: SQLTimestamp, timeZone: TimeZone): Int = { - ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND / 3600) % 24).toInt - } - - /** - * Returns the minute value of a given timestamp value. The timestamp is expressed in - * microseconds. - */ - def getMinutes(microsec: SQLTimestamp): Int = { - ((localTimestamp(microsec) / MICROS_PER_SECOND / 60) % 60).toInt + (MICROSECONDS.toHours(localTimestamp(microsec, timeZone)) % 24).toInt } /** @@ -558,15 +444,7 @@ object DateTimeUtils { * microseconds. */ def getMinutes(microsec: SQLTimestamp, timeZone: TimeZone): Int = { - ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND / 60) % 60).toInt - } - - /** - * Returns the second value of a given timestamp value. The timestamp is expressed in - * microseconds. - */ - def getSeconds(microsec: SQLTimestamp): Int = { - ((localTimestamp(microsec) / MICROS_PER_SECOND) % 60).toInt + (MICROSECONDS.toMinutes(localTimestamp(microsec, timeZone)) % 60).toInt } /** @@ -574,55 +452,7 @@ object DateTimeUtils { * microseconds. */ def getSeconds(microsec: SQLTimestamp, timeZone: TimeZone): Int = { - ((localTimestamp(microsec, timeZone) / MICROS_PER_SECOND) % 60).toInt - } - - private[this] def isLeapYear(year: Int): Boolean = { - (year % 4) == 0 && ((year % 100) != 0 || (year % 400) == 0) - } - - /** - * Return the number of days since the start of 400 year period. - * The second year of a 400 year period (year 1) starts on day 365. - */ - private[this] def yearBoundary(year: Int): Int = { - year * 365 + ((year / 4 ) - (year / 100) + (year / 400)) - } - - /** - * Calculates the number of years for the given number of days. This depends - * on a 400 year period. - * @param days days since the beginning of the 400 year period - * @return (number of year, days in year) - */ - private[this] def numYears(days: Int): (Int, Int) = { - val year = days / 365 - val boundary = yearBoundary(year) - if (days > boundary) (year, days - boundary) else (year - 1, days - yearBoundary(year - 1)) - } - - /** - * Calculates the year and the number of the day in the year for the given - * number of days. The given days is the number of days since 1.1.1970. - * - * The calculation uses the fact that the period 1.1.2001 until 31.12.2400 is - * equals to the period 1.1.1601 until 31.12.2000. - */ - private[this] def getYearAndDayInYear(daysSince1970: SQLDate): (Int, Int) = { - // add the difference (in days) between 1.1.1970 and the artificial year 0 (-17999) - var daysSince1970Tmp = daysSince1970 - // Since Julian calendar was replaced with the Gregorian calendar, - // the 10 days after Oct. 4 were skipped. - // (1582-10-04) -141428 days since 1970-01-01 - if (daysSince1970 <= -141428) { - daysSince1970Tmp -= 10 - } - val daysNormalized = daysSince1970Tmp + toYearZero - val numOfQuarterCenturies = daysNormalized / daysIn400Years - val daysInThis400 = daysNormalized % daysIn400Years + 1 - val (years, dayInYear) = numYears(daysInThis400) - val year: Int = (2001 - 20000) + 400 * numOfQuarterCenturies + years - (year, dayInYear) + (MICROSECONDS.toSeconds(localTimestamp(microsec, timeZone)) % 60).toInt } /** @@ -630,7 +460,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getDayInYear(date: SQLDate): Int = { - getYearAndDayInYear(date)._2 + LocalDate.ofEpochDay(date).getDayOfYear } /** @@ -638,7 +468,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getYear(date: SQLDate): Int = { - getYearAndDayInYear(date)._1 + LocalDate.ofEpochDay(date).getYear } /** @@ -646,19 +476,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getQuarter(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - dayInYear = dayInYear - 1 - } - if (dayInYear <= 90) { - 1 - } else if (dayInYear <= 181) { - 2 - } else if (dayInYear <= 273) { - 3 - } else { - 4 - } + LocalDate.ofEpochDay(date).get(IsoFields.QUARTER_OF_YEAR) } /** @@ -666,43 +484,8 @@ object DateTimeUtils { * year, month (Jan is Month 1), dayInMonth, daysToMonthEnd (0 if it's last day of month). */ def splitDate(date: SQLDate): (Int, Int, Int, Int) = { - var (year, dayInYear) = getYearAndDayInYear(date) - val isLeap = isLeapYear(year) - if (isLeap && dayInYear == 60) { - (year, 2, 29, 0) - } else { - if (isLeap && dayInYear > 60) dayInYear -= 1 - - if (dayInYear <= 181) { - if (dayInYear <= 31) { - (year, 1, dayInYear, 31 - dayInYear) - } else if (dayInYear <= 59) { - (year, 2, dayInYear - 31, if (isLeap) 60 - dayInYear else 59 - dayInYear) - } else if (dayInYear <= 90) { - (year, 3, dayInYear - 59, 90 - dayInYear) - } else if (dayInYear <= 120) { - (year, 4, dayInYear - 90, 120 - dayInYear) - } else if (dayInYear <= 151) { - (year, 5, dayInYear - 120, 151 - dayInYear) - } else { - (year, 6, dayInYear - 151, 181 - dayInYear) - } - } else { - if (dayInYear <= 212) { - (year, 7, dayInYear - 181, 212 - dayInYear) - } else if (dayInYear <= 243) { - (year, 8, dayInYear - 212, 243 - dayInYear) - } else if (dayInYear <= 273) { - (year, 9, dayInYear - 243, 273 - dayInYear) - } else if (dayInYear <= 304) { - (year, 10, dayInYear - 273, 304 - dayInYear) - } else if (dayInYear <= 334) { - (year, 11, dayInYear - 304, 334 - dayInYear) - } else { - (year, 12, dayInYear - 334, 365 - dayInYear) - } - } - } + val ld = LocalDate.ofEpochDay(date) + (ld.getYear, ld.getMonthValue, ld.getDayOfMonth, ld.lengthOfMonth() - ld.getDayOfMonth) } /** @@ -710,40 +493,7 @@ object DateTimeUtils { * since 1.1.1970. January is month 1. */ def getMonth(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - if (dayInYear == 60) { - return 2 - } else if (dayInYear > 60) { - dayInYear = dayInYear - 1 - } - } - - if (dayInYear <= 31) { - 1 - } else if (dayInYear <= 59) { - 2 - } else if (dayInYear <= 90) { - 3 - } else if (dayInYear <= 120) { - 4 - } else if (dayInYear <= 151) { - 5 - } else if (dayInYear <= 181) { - 6 - } else if (dayInYear <= 212) { - 7 - } else if (dayInYear <= 243) { - 8 - } else if (dayInYear <= 273) { - 9 - } else if (dayInYear <= 304) { - 10 - } else if (dayInYear <= 334) { - 11 - } else { - 12 - } + LocalDate.ofEpochDay(date).getMonthValue } /** @@ -751,73 +501,7 @@ object DateTimeUtils { * since 1.1.1970. */ def getDayOfMonth(date: SQLDate): Int = { - var (year, dayInYear) = getYearAndDayInYear(date) - if (isLeapYear(year)) { - if (dayInYear == 60) { - return 29 - } else if (dayInYear > 60) { - dayInYear = dayInYear - 1 - } - } - - if (dayInYear <= 31) { - dayInYear - } else if (dayInYear <= 59) { - dayInYear - 31 - } else if (dayInYear <= 90) { - dayInYear - 59 - } else if (dayInYear <= 120) { - dayInYear - 90 - } else if (dayInYear <= 151) { - dayInYear - 120 - } else if (dayInYear <= 181) { - dayInYear - 151 - } else if (dayInYear <= 212) { - dayInYear - 181 - } else if (dayInYear <= 243) { - dayInYear - 212 - } else if (dayInYear <= 273) { - dayInYear - 243 - } else if (dayInYear <= 304) { - dayInYear - 273 - } else if (dayInYear <= 334) { - dayInYear - 304 - } else { - dayInYear - 334 - } - } - - /** - * The number of days for each month (not leap year) - */ - private val monthDays = Array(31, 28, 31, 30, 31, 30, 31, 31, 30, 31, 30, 31) - - /** - * Returns the date value for the first day of the given month. - * The month is expressed in months since year zero (17999 BC), starting from 0. - */ - private def firstDayOfMonth(absoluteMonth: Int): SQLDate = { - val absoluteYear = absoluteMonth / 12 - var monthInYear = absoluteMonth - absoluteYear * 12 - var date = getDateFromYear(absoluteYear) - if (monthInYear >= 2 && isLeapYear(absoluteYear + YearZero)) { - date += 1 - } - while (monthInYear > 0) { - date += monthDays(monthInYear - 1) - monthInYear -= 1 - } - date - } - - /** - * Returns the date value for January 1 of the given year. - * The year is expressed in years since year zero (17999 BC), starting from 0. - */ - private def getDateFromYear(absoluteYear: Int): SQLDate = { - val absoluteDays = (absoluteYear * 365 + absoluteYear / 400 - absoluteYear / 100 - + absoluteYear / 4) - absoluteDays - toYearZero + LocalDate.ofEpochDay(date).getDayOfMonth } /** @@ -825,30 +509,7 @@ object DateTimeUtils { * Returns a date value, expressed in days since 1.1.1970. */ def dateAddMonths(days: SQLDate, months: Int): SQLDate = { - val (year, monthInYear, dayOfMonth, daysToMonthEnd) = splitDate(days) - val absoluteMonth = (year - YearZero) * 12 + monthInYear - 1 + months - val nonNegativeMonth = if (absoluteMonth >= 0) absoluteMonth else 0 - val currentMonthInYear = nonNegativeMonth % 12 - val currentYear = nonNegativeMonth / 12 - - val leapDay = if (currentMonthInYear == 1 && isLeapYear(currentYear + YearZero)) 1 else 0 - val lastDayOfMonth = monthDays(currentMonthInYear) + leapDay - - val currentDayInMonth = if (daysToMonthEnd == 0 || dayOfMonth >= lastDayOfMonth) { - // last day of the month - lastDayOfMonth - } else { - dayOfMonth - } - firstDayOfMonth(nonNegativeMonth) + currentDayInMonth - 1 - } - - /** - * Add timestamp and full interval. - * Returns a timestamp value, expressed in microseconds since 1.1.1970 00:00:00. - */ - def timestampAddInterval(start: SQLTimestamp, months: Int, microseconds: Long): SQLTimestamp = { - timestampAddInterval(start, months, microseconds, defaultTimeZone()) + LocalDate.ofEpochDay(days).plusMonths(months).toEpochDay.toInt } /** @@ -859,12 +520,12 @@ object DateTimeUtils { start: SQLTimestamp, months: Int, microseconds: Long, - timeZone: TimeZone): SQLTimestamp = { - val days = millisToDays(start / 1000L, timeZone) - val newDays = dateAddMonths(days, months) - start + - daysToMillis(newDays, timeZone) * 1000L - daysToMillis(days, timeZone) * 1000L + - microseconds + zoneId: ZoneId): SQLTimestamp = { + val resultTimestamp = microsToInstant(start) + .atZone(zoneId) + .plusMonths(months) + .plus(microseconds, ChronoUnit.MICROS) + instantToMicros(resultTimestamp.toInstant) } /** @@ -882,8 +543,8 @@ object DateTimeUtils { time2: SQLTimestamp, roundOff: Boolean, timeZone: TimeZone): Double = { - val millis1 = time1 / 1000L - val millis2 = time2 / 1000L + val millis1 = MICROSECONDS.toMillis(time1) + val millis2 = MICROSECONDS.toMillis(time2) val date1 = millisToDays(millis1, timeZone) val date2 = millisToDays(millis2, timeZone) val (year1, monthInYear1, dayInMonth1, daysToMonthEnd1) = splitDate(date1) @@ -899,12 +560,11 @@ object DateTimeUtils { } // using milliseconds can cause precision loss with more than 8 digits // we follow Hive's implementation which uses seconds - val secondsInDay1 = (millis1 - daysToMillis(date1, timeZone)) / 1000L - val secondsInDay2 = (millis2 - daysToMillis(date2, timeZone)) / 1000L + val secondsInDay1 = MILLISECONDS.toSeconds(millis1 - daysToMillis(date1, timeZone)) + val secondsInDay2 = MILLISECONDS.toSeconds(millis2 - daysToMillis(date2, timeZone)) val secondsDiff = (dayInMonth1 - dayInMonth2) * SECONDS_PER_DAY + secondsInDay1 - secondsInDay2 - // 2678400D is the number of seconds in 31 days - // every month is considered to be 31 days long in this function - val diff = monthDiff + secondsDiff / 2678400D + val secondsInMonth = DAYS.toSeconds(31) + val diff = monthDiff + secondsDiff / secondsInMonth.toDouble if (roundOff) { // rounding to 8 digits math.round(diff * 1e8) / 1e8 @@ -953,8 +613,8 @@ object DateTimeUtils { * since 1.1.1970. */ def getLastDayOfMonth(date: SQLDate): SQLDate = { - val (_, _, _, daysToMonthEnd) = splitDate(date) - date + daysToMonthEnd + val localDate = LocalDate.ofEpochDay(date) + (date - localDate.getDayOfMonth) + localDate.lengthOfMonth() } // Visible for testing. @@ -988,7 +648,7 @@ object DateTimeUtils { * Trunc level should be generated using `parseTruncLevel()`, should be between 1 and 8 */ def truncTimestamp(t: SQLTimestamp, level: Int, timeZone: TimeZone): SQLTimestamp = { - var millis = t / MICROS_PER_MILLIS + var millis = MICROSECONDS.toMillis(t) val truncated = level match { case TRUNC_TO_YEAR => val dDays = millisToDays(millis, timeZone) @@ -999,13 +659,13 @@ object DateTimeUtils { case TRUNC_TO_DAY => val offset = timeZone.getOffset(millis) millis += offset - millis - millis % (MILLIS_PER_SECOND * SECONDS_PER_DAY) - offset + millis - millis % MILLIS_PER_DAY - offset case TRUNC_TO_HOUR => val offset = timeZone.getOffset(millis) millis += offset - millis - millis % (60 * 60 * MILLIS_PER_SECOND) - offset + millis - millis % MILLIS_PER_HOUR - offset case TRUNC_TO_MINUTE => - millis - millis % (60 * MILLIS_PER_SECOND) + millis - millis % MILLIS_PER_MINUTE case TRUNC_TO_SECOND => millis - millis % MILLIS_PER_SECOND case TRUNC_TO_WEEK => @@ -1014,18 +674,9 @@ object DateTimeUtils { daysToMillis(prevMonday, timeZone) case TRUNC_TO_QUARTER => val dDays = millisToDays(millis, timeZone) - millis = daysToMillis(truncDate(dDays, TRUNC_TO_MONTH), timeZone) - val cal = Calendar.getInstance() - cal.setTimeInMillis(millis) - val quarter = getQuarter(dDays) - val month = quarter match { - case 1 => Calendar.JANUARY - case 2 => Calendar.APRIL - case 3 => Calendar.JULY - case 4 => Calendar.OCTOBER - } - cal.set(Calendar.MONTH, month) - cal.getTimeInMillis() + val daysOfQuarter = LocalDate.ofEpochDay(dDays) + .`with`(IsoFields.DAY_OF_QUARTER, 1L).toEpochDay.toInt + daysToMillis(daysOfQuarter, timeZone) case _ => // caller make sure that this should never be reached sys.error(s"Invalid trunc level: $level") @@ -1033,10 +684,6 @@ object DateTimeUtils { truncated * MICROS_PER_MILLIS } - def truncTimestamp(d: SQLTimestamp, level: Int): SQLTimestamp = { - truncTimestamp(d, level, defaultTimeZone()) - } - /** * Returns the truncate level, could be TRUNC_YEAR, TRUNC_MONTH, TRUNC_TO_DAY, TRUNC_TO_HOUR, * TRUNC_TO_MINUTE, TRUNC_TO_SECOND, TRUNC_TO_WEEK, TRUNC_TO_QUARTER or TRUNC_INVALID, @@ -1072,26 +719,15 @@ object DateTimeUtils { if (offset != guess) { guess = tz.getOffset(millisLocal - offset) if (guess != offset) { - // fallback to do the reverse lookup using java.sql.Timestamp + // fallback to do the reverse lookup using java.time.LocalDateTime // this should only happen near the start or end of DST - val days = Math.floor(millisLocal.toDouble / MILLIS_PER_DAY).toInt - val year = getYear(days) - val month = getMonth(days) - val day = getDayOfMonth(days) - - var millisOfDay = (millisLocal % MILLIS_PER_DAY).toInt - if (millisOfDay < 0) { - millisOfDay += MILLIS_PER_DAY.toInt - } - val seconds = (millisOfDay / 1000L).toInt - val hh = seconds / 3600 - val mm = seconds / 60 % 60 - val ss = seconds % 60 - val ms = millisOfDay % 1000 - val calendar = Calendar.getInstance(tz) - calendar.set(year, month - 1, day, hh, mm, ss) - calendar.set(Calendar.MILLISECOND, ms) - guess = (millisLocal - calendar.getTimeInMillis()).toInt + val localDate = LocalDate.ofEpochDay(MILLISECONDS.toDays(millisLocal)) + val localTime = LocalTime.ofNanoOfDay(MILLISECONDS.toNanos( + Math.floorMod(millisLocal, MILLIS_PER_DAY))) + val localDateTime = LocalDateTime.of(localDate, localTime) + val millisEpoch = localDateTime.atZone(tz.toZoneId).toInstant.toEpochMilli + + guess = (millisLocal - millisEpoch).toInt } } guess @@ -1111,15 +747,19 @@ object DateTimeUtils { ts } else { // get the human time using local time zone, that actually is in fromZone. - val localTs = ts + localZone.getOffset(ts / 1000L) * 1000L // in fromZone - localTs - getOffsetFromLocalMillis(localTs / 1000L, fromZone) * 1000L + val localZoneOffsetMs = localZone.getOffset(MICROSECONDS.toMillis(ts)) + val localTsUs = ts + MILLISECONDS.toMicros(localZoneOffsetMs) // in fromZone + val offsetFromLocalMs = getOffsetFromLocalMillis(MICROSECONDS.toMillis(localTsUs), fromZone) + localTsUs - MILLISECONDS.toMicros(offsetFromLocalMs) } if (toZone.getID == localZone.getID) { utcTs } else { - val localTs = utcTs + toZone.getOffset(utcTs / 1000L) * 1000L // in toZone + val toZoneOffsetMs = toZone.getOffset(MICROSECONDS.toMillis(utcTs)) + val localTsUs = utcTs + MILLISECONDS.toMicros(toZoneOffsetMs) // in toZone // treat it as local timezone, convert to UTC (we could get the expected human time back) - localTs - getOffsetFromLocalMillis(localTs / 1000L, localZone) * 1000L + val offsetFromLocalMs = getOffsetFromLocalMillis(MICROSECONDS.toMillis(localTsUs), localZone) + localTsUs - MILLISECONDS.toMicros(offsetFromLocalMs) } } @@ -1138,13 +778,4 @@ object DateTimeUtils { def toUTCTime(time: SQLTimestamp, timeZone: String): SQLTimestamp = { convertTz(time, getTimeZone(timeZone), TimeZoneGMT) } - - /** - * Re-initialize the current thread's thread locals. Exposed for testing. - */ - private[util] def resetThreadLocals(): Unit = { - threadLocalGmtCalendar.remove() - threadLocalTimestampFormat.remove() - threadLocalDateFormat.remove() - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala similarity index 84% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala index 90e81661bae7a..361c8b29db33d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FailureSafeParser.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/FailureSafeParser.scala @@ -15,13 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources +package org.apache.spark.sql.catalyst.util import org.apache.spark.SparkException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow -import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.unsafe.types.UTF8String @@ -29,8 +27,7 @@ class FailureSafeParser[IN]( rawParser: IN => Seq[InternalRow], mode: ParseMode, schema: StructType, - columnNameOfCorruptRecord: String, - isMultiLine: Boolean) { + columnNameOfCorruptRecord: String) { private val corruptFieldIndex = schema.getFieldIndex(columnNameOfCorruptRecord) private val actualSchema = StructType(schema.filterNot(_.name == columnNameOfCorruptRecord)) @@ -58,15 +55,9 @@ class FailureSafeParser[IN]( } } - private val skipParsing = !isMultiLine && mode == PermissiveMode && schema.isEmpty - def parse(input: IN): Iterator[InternalRow] = { try { - if (skipParsing) { - Iterator.single(InternalRow.empty) - } else { - rawParser.apply(input).toIterator.map(row => toResultRow(Some(row), () => null)) - } + rawParser.apply(input).toIterator.map(row => toResultRow(Some(row), () => null)) } catch { case e: BadRecordException => mode match { case PermissiveMode => @@ -75,7 +66,8 @@ class FailureSafeParser[IN]( Iterator.empty case FailFastMode => throw new SparkException("Malformed records are detected in record parsing. " + - s"Parse Mode: ${FailFastMode.name}.", e.cause) + s"Parse Mode: ${FailFastMode.name}. To process malformed records as null " + + "result, try setting the option 'mode' as 'PERMISSIVE'.", e) } } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/HyperLogLogPlusPlusHelper.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/HyperLogLogPlusPlusHelper.scala index 9bacd3b925be3..ea619c6a7666c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/HyperLogLogPlusPlusHelper.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/HyperLogLogPlusPlusHelper.scala @@ -199,7 +199,7 @@ class HyperLogLogPlusPlusHelper(relativeSD: Double) extends Serializable { var shift = 0 while (idx < m && i < REGISTERS_PER_WORD) { val Midx = (word >>> shift) & REGISTER_WORD_MASK - zInverse += 1.0 / (1 << Midx) + zInverse += 1.0 / (1L << Midx) if (Midx == 0) { V += 1.0d } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala index 9c3f6b7c5d245..7dbdd1ef1cdc5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/NumberConverter.scala @@ -169,4 +169,39 @@ object NumberConverter { } UTF8String.fromBytes(java.util.Arrays.copyOfRange(temp, resultStartPos, temp.length)) } + + def toBinary(l: Long): Array[Byte] = { + val result = new Array[Byte](8) + result(0) = (l >>> 56 & 0xFF).toByte + result(1) = (l >>> 48 & 0xFF).toByte + result(2) = (l >>> 40 & 0xFF).toByte + result(3) = (l >>> 32 & 0xFF).toByte + result(4) = (l >>> 24 & 0xFF).toByte + result(5) = (l >>> 16 & 0xFF).toByte + result(6) = (l >>> 8 & 0xFF).toByte + result(7) = (l & 0xFF).toByte + result + } + + def toBinary(i: Int): Array[Byte] = { + val result = new Array[Byte](4) + result(0) = (i >>> 24 & 0xFF).toByte + result(1) = (i >>> 16 & 0xFF).toByte + result(2) = (i >>> 8 & 0xFF).toByte + result(3) = (i & 0xFF).toByte + result + } + + def toBinary(s: Short): Array[Byte] = { + val result = new Array[Byte](2) + result(0) = (s >>> 8 & 0xFF).toByte + result(1) = (s & 0xFF).toByte + result + } + + def toBinary(s: Byte): Array[Byte] = { + val result = new Array[Byte](1) + result(0) = s + result + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala index 3190e511e2cb5..2a03f85ab594b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/QuantileSummaries.scala @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.util.QuantileSummaries.Stats * Helper class to compute approximate quantile summary. * This implementation is based on the algorithm proposed in the paper: * "Space-efficient Online Computation of Quantile Summaries" by Greenwald, Michael - * and Khanna, Sanjeev. (http://dx.doi.org/10.1145/375663.375670) + * and Khanna, Sanjeev. (https://doi.org/10.1145/375663.375670) * * In order to optimize for speed, it maintains an internal buffer of the last seen samples, * and only inserts them after crossing a certain size threshold. This guarantees a near-constant diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala index bc861a805ce61..6510bacf55899 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/StringUtils.scala @@ -17,12 +17,18 @@ package org.apache.spark.sql.catalyst.util +import java.util.concurrent.atomic.AtomicBoolean import java.util.regex.{Pattern, PatternSyntaxException} +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.UTF8String -object StringUtils { +object StringUtils extends Logging { /** * Validate and convert SQL 'like' pattern to a Java regular expression. @@ -87,4 +93,70 @@ object StringUtils { } funcNames.toSeq } + + /** + * Concatenation of sequence of strings to final string with cheap append method + * and one memory allocation for the final string. Can also bound the final size of + * the string. + */ + class StringConcat(val maxLength: Int = ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) { + protected val strings = new ArrayBuffer[String] + protected var length: Int = 0 + + def atLimit: Boolean = length >= maxLength + + /** + * Appends a string and accumulates its length to allocate a string buffer for all + * appended strings once in the toString method. Returns true if the string still + * has room for further appends before it hits its max limit. + */ + def append(s: String): Unit = { + if (s != null) { + val sLen = s.length + if (!atLimit) { + val available = maxLength - length + val stringToAppend = if (available >= sLen) s else s.substring(0, available) + strings.append(stringToAppend) + } + length += sLen + } + } + + /** + * The method allocates memory for all appended strings, writes them to the memory and + * returns concatenated string. + */ + override def toString: String = { + val finalLength = if (atLimit) maxLength else length + val result = new java.lang.StringBuilder(finalLength) + strings.foreach(result.append) + result.toString + } + } + + /** + * A string concatenator for plan strings. Uses length from a configured value, and + * prints a warning the first time a plan is truncated. + */ + class PlanStringConcat extends StringConcat(Math.max(0, SQLConf.get.maxPlanStringLength - 30)) { + override def toString: String = { + if (atLimit) { + logWarning( + "Truncated the string representation of a plan since it was too long. The " + + s"plan had length ${length} and the maximum is ${maxLength}. This behavior " + + s"can be adjusted by setting '${SQLConf.MAX_PLAN_STRING_LENGTH.key}'.") + val truncateMsg = if (maxLength == 0) { + s"Truncated plan of $length characters" + } else { + s"... ${length - maxLength} more characters" + } + val result = new java.lang.StringBuilder(maxLength + truncateMsg.length) + strings.foreach(result.append) + result.append(truncateMsg) + result.toString + } else { + super.toString + } + } + } } 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 new file mode 100644 index 0000000000000..f2a1a95d1dc58 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TimestampFormatter.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.catalyst.util + +import java.text.ParseException +import java.time._ +import java.time.format.DateTimeParseException +import java.time.temporal.ChronoField.MICRO_OF_SECOND +import java.time.temporal.TemporalQueries +import java.util.{Locale, TimeZone} +import java.util.concurrent.TimeUnit.SECONDS + +sealed trait TimestampFormatter extends Serializable { + /** + * Parses a timestamp in a string and converts it to microseconds. + * + * @param s - string with timestamp to parse + * @return microseconds since epoch. + * @throws ParseException can be thrown by legacy parser + * @throws DateTimeParseException can be thrown by new parser + * @throws DateTimeException unable to obtain local date or time + */ + @throws(classOf[ParseException]) + @throws(classOf[DateTimeParseException]) + @throws(classOf[DateTimeException]) + def parse(s: String): Long + def format(us: Long): String +} + +class Iso8601TimestampFormatter( + pattern: String, + zoneId: ZoneId, + locale: Locale) extends TimestampFormatter with DateTimeFormatterHelper { + @transient + protected lazy val formatter = getOrCreateFormatter(pattern, locale) + + override def parse(s: String): Long = { + val parsed = formatter.parse(s) + val parsedZoneId = parsed.query(TemporalQueries.zone()) + val timeZoneId = if (parsedZoneId == null) zoneId else parsedZoneId + val zonedDateTime = toZonedDateTime(parsed, timeZoneId) + val epochSeconds = zonedDateTime.toEpochSecond + val microsOfSecond = zonedDateTime.get(MICRO_OF_SECOND) + + Math.addExact(SECONDS.toMicros(epochSeconds), microsOfSecond) + } + + override def format(us: Long): String = { + val instant = DateTimeUtils.microsToInstant(us) + formatter.withZone(zoneId).format(instant) + } +} + +/** + * The formatter parses/formats timestamps according to the pattern `yyyy-MM-dd HH:mm:ss.[..fff..]` + * where `[..fff..]` is a fraction of second up to microsecond resolution. The formatter does not + * output trailing zeros in the fraction. For example, the timestamp `2019-03-05 15:00:01.123400` is + * formatted as the string `2019-03-05 15:00:01.1234`. + * + * @param zoneId the time zone identifier in which the formatter parses or format timestamps + */ +class FractionTimestampFormatter(zoneId: ZoneId) + extends Iso8601TimestampFormatter("", zoneId, TimestampFormatter.defaultLocale) { + + @transient + override protected lazy val formatter = DateTimeFormatterHelper.fractionFormatter +} + +object TimestampFormatter { + val defaultPattern: String = "yyyy-MM-dd HH:mm:ss" + val defaultLocale: Locale = Locale.US + + def apply(format: String, zoneId: ZoneId, locale: Locale): TimestampFormatter = { + new Iso8601TimestampFormatter(format, zoneId, locale) + } + + def apply(format: String, zoneId: ZoneId): TimestampFormatter = { + apply(format, zoneId, defaultLocale) + } + + def apply(zoneId: ZoneId): TimestampFormatter = { + apply(defaultPattern, zoneId, defaultLocale) + } + + def getFractionFormatter(zoneId: ZoneId): TimestampFormatter = { + new FractionTimestampFormatter(zoneId) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala index 76218b459ef0d..fed2a1ac4b8b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/TypeUtils.scala @@ -46,12 +46,20 @@ object TypeUtils { if (TypeCoercion.haveSameType(types)) { TypeCheckResult.TypeCheckSuccess } else { - return TypeCheckResult.TypeCheckFailure( + TypeCheckResult.TypeCheckFailure( s"input to $caller should all be the same type, but it's " + types.map(_.catalogString).mkString("[", ", ", "]")) } } + def checkForMapKeyType(keyType: DataType): TypeCheckResult = { + if (keyType.existsRecursively(_.isInstanceOf[MapType])) { + TypeCheckResult.TypeCheckFailure("The key of map cannot be/contain map.") + } else { + TypeCheckResult.TypeCheckSuccess + } + } + def getNumeric(t: DataType): Numeric[Any] = t.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]] @@ -65,11 +73,12 @@ object TypeUtils { } def compareBinary(x: Array[Byte], y: Array[Byte]): Int = { - for (i <- 0 until x.length; if i < y.length) { - val v1 = x(i) & 0xff - val v2 = y(i) & 0xff - val res = v1 - v2 + val limit = if (x.length <= y.length) x.length else y.length + var i = 0 + while (i < limit) { + val res = (x(i) & 0xff) - (y(i) & 0xff) if (res != 0) return res + i += 1 } x.length - y.length } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala index 0978e92dd4f72..eefabbffff62c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -19,25 +19,24 @@ package org.apache.spark.sql.catalyst import java.io._ import java.nio.charset.StandardCharsets +import java.util.concurrent.atomic.AtomicBoolean +import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{NumericType, StringType} import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.Utils -package object util { +package object util extends Logging { /** Silences output to stderr or stdout for the duration of f */ def quietly[A](f: => A): A = { val origErr = System.err val origOut = System.out try { - System.setErr(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) - System.setOut(new PrintStream(new OutputStream { - def write(b: Int) = {} - })) + System.setErr(new PrintStream((_: Int) => {})) + System.setOut(new PrintStream((_: Int) => {})) f } finally { @@ -94,9 +93,9 @@ package object util { } def stringToFile(file: File, str: String): File = { - val out = new PrintWriter(file) - out.write(str) - out.close() + Utils.tryWithResource(new PrintWriter(file)) { out => + out.write(str) + } file } @@ -116,24 +115,13 @@ package object util { def stackTraceToString(t: Throwable): String = { val out = new java.io.ByteArrayOutputStream - val writer = new PrintWriter(out) - t.printStackTrace(writer) - writer.flush() + Utils.tryWithResource(new PrintWriter(out)) { writer => + t.printStackTrace(writer) + writer.flush() + } new String(out.toByteArray, StandardCharsets.UTF_8) } - def stringOrNull(a: AnyRef): String = if (a == null) null else a.toString - - def benchmark[A](f: => A): A = { - val startTime = System.nanoTime() - val ret = f - val endTime = System.nanoTime() - // scalastyle:off println - println(s"${(endTime - startTime).toDouble / 1000000}ms") - // scalastyle:on println - ret - } - // Replaces attributes, string literals, complex type extractors with their pretty form so that // generated column names don't contain back-ticks or double-quotes. def usePrettyExpression(e: Expression): Expression = e transform { @@ -155,7 +143,6 @@ package object util { def toPrettySQL(e: Expression): String = usePrettyExpression(e).sql - def escapeSingleQuotedString(str: String): String = { val builder = StringBuilder.newBuilder @@ -167,10 +154,37 @@ package object util { builder.toString() } - /* FIX ME - implicit class debugLogging(a: Any) { - def debugLogging() { - org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) + /** Whether we have warned about plan string truncation yet. */ + private val truncationWarningPrinted = new AtomicBoolean(false) + + /** + * Format a sequence with semantics similar to calling .mkString(). Any elements beyond + * maxNumToStringFields will be dropped and replaced by a "... N more fields" placeholder. + * + * @return the trimmed and formatted string. + */ + def truncatedString[T]( + seq: Seq[T], + start: String, + sep: String, + end: String, + maxFields: Int): String = { + if (seq.length > maxFields) { + if (truncationWarningPrinted.compareAndSet(false, true)) { + logWarning( + "Truncated the string representation of a plan since it was too large. This " + + s"behavior can be adjusted by setting '${SQLConf.MAX_TO_STRING_FIELDS.key}'.") + } + val numFields = math.max(0, maxFields - 1) + seq.take(numFields).mkString( + start, sep, sep + "... " + (seq.length - numFields) + " more fields" + end) + } else { + seq.mkString(start, sep, end) } - } */ + } + + /** Shorthand for calling truncatedString() without start or end strings. */ + def truncatedString[T](seq: Seq[T], sep: String, maxFields: Int): String = { + truncatedString(seq, "", sep, "", maxFields) + } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala new file mode 100644 index 0000000000000..eed69cdc8cac6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Implicits.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} + +object DataSourceV2Implicits { + implicit class TableHelper(table: Table) { + def asReadable: SupportsRead = { + table match { + case support: SupportsRead => + support + case _ => + throw new AnalysisException(s"Table does not support reads: ${table.name}") + } + } + + def asWritable: SupportsWrite = { + table match { + case support: SupportsWrite => + support + case _ => + throw new AnalysisException(s"Table does not support writes: ${table.name}") + } + } + + def supports(capability: TableCapability): Boolean = table.capabilities.contains(capability) + + def supportsAny(capabilities: TableCapability*): Boolean = capabilities.exists(supports) + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala new file mode 100644 index 0000000000000..9ae3dbbc45502 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala @@ -0,0 +1,125 @@ +/* + * 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.v2 + +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.reader.{Statistics => V2Statistics, _} +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * A logical plan representing a data source v2 table. + * + * @param table The table that this relation represents. + * @param options The options for this table operation. It's used to create fresh [[ScanBuilder]] + * and [[WriteBuilder]]. + */ +case class DataSourceV2Relation( + table: Table, + output: Seq[AttributeReference], + options: CaseInsensitiveStringMap) + extends LeafNode with MultiInstanceRelation with NamedRelation { + + import DataSourceV2Implicits._ + + override def name: String = table.name() + + override def skipSchemaResolution: Boolean = table.supports(TableCapability.ACCEPT_ANY_SCHEMA) + + override def simpleString(maxFields: Int): String = { + s"RelationV2${truncatedString(output, "[", ", ", "]", maxFields)} $name" + } + + def newScanBuilder(): ScanBuilder = { + table.asReadable.newScanBuilder(options) + } + + override def computeStats(): Statistics = { + val scan = newScanBuilder().build() + scan match { + case r: SupportsReportStatistics => + val statistics = r.estimateStatistics() + DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes) + case _ => + Statistics(sizeInBytes = conf.defaultSizeInBytes) + } + } + + override def newInstance(): DataSourceV2Relation = { + copy(output = output.map(_.newInstance())) + } +} + +/** + * A specialization of [[DataSourceV2Relation]] with the streaming bit set to true. + * + * Note that, this plan has a mutable reader, so Spark won't apply operator push-down for this plan, + * to avoid making the plan mutable. We should consolidate this plan and [[DataSourceV2Relation]] + * after we figure out how to apply operator push-down for streaming data sources. + */ +case class StreamingDataSourceV2Relation( + output: Seq[Attribute], + scan: Scan, + stream: SparkDataStream, + startOffset: Option[Offset] = None, + endOffset: Option[Offset] = None) + extends LeafNode with MultiInstanceRelation { + + override def isStreaming: Boolean = true + + override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) + + override def computeStats(): Statistics = scan match { + case r: SupportsReportStatistics => + val statistics = r.estimateStatistics() + DataSourceV2Relation.transformV2Stats(statistics, None, conf.defaultSizeInBytes) + case _ => + Statistics(sizeInBytes = conf.defaultSizeInBytes) + } +} + +object DataSourceV2Relation { + def create(table: Table, options: CaseInsensitiveStringMap): DataSourceV2Relation = { + val output = table.schema().toAttributes + DataSourceV2Relation(table, output, options) + } + + def create(table: Table): DataSourceV2Relation = create(table, CaseInsensitiveStringMap.empty) + + /** + * This is used to transform data source v2 statistics to logical.Statistics. + */ + def transformV2Stats( + v2Statistics: V2Statistics, + defaultRowCount: Option[BigInt], + defaultSizeInBytes: Long): Statistics = { + val numRows: Option[BigInt] = if (v2Statistics.numRows().isPresent) { + Some(v2Statistics.numRows().getAsLong) + } else { + defaultRowCount + } + Statistics( + sizeInBytes = v2Statistics.sizeInBytes().orElse(defaultSizeInBytes), + rowCount = numRows) + } +} 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 da70d7da7351b..fbdb1c5f957d9 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 @@ -27,15 +27,16 @@ import scala.collection.immutable import scala.util.matching.Regex import org.apache.hadoop.fs.Path -import org.tukaani.xz.LZMA2Options import org.apache.spark.{SparkContext, TaskContext} import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ import org.apache.spark.network.util.ByteUnit -import org.apache.spark.sql.catalyst.analysis.Resolver +import org.apache.spark.sql.catalyst.analysis.{HintErrorLogger, Resolver} import org.apache.spark.sql.catalyst.expressions.CodegenObjectFactoryMode import org.apache.spark.sql.catalyst.expressions.codegen.CodeGenerator +import org.apache.spark.sql.catalyst.plans.logical.HintErrorHandler +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.util.Utils //////////////////////////////////////////////////////////////////////////////////////////////////// @@ -45,7 +46,7 @@ import org.apache.spark.util.Utils object SQLConf { - private val sqlConfEntries = java.util.Collections.synchronizedMap( + private[sql] val sqlConfEntries = java.util.Collections.synchronizedMap( new java.util.HashMap[String, ConfigEntry[_]]()) val staticConfKeys: java.util.Set[String] = @@ -171,11 +172,21 @@ object SQLConf { .intConf .createWithDefault(10) + val OPTIMIZER_INSET_SWITCH_THRESHOLD = + buildConf("spark.sql.optimizer.inSetSwitchThreshold") + .internal() + .doc("Configures the max set size in InSet for which Spark will generate code with " + + "switch statements. This is applicable only to bytes, shorts, ints, dates.") + .intConf + .checkValue(threshold => threshold >= 0 && threshold <= 600, "The max set size " + + "for using switch statements in InSet must be non-negative and less than or equal to 600") + .createWithDefault(400) + val OPTIMIZER_PLAN_CHANGE_LOG_LEVEL = buildConf("spark.sql.optimizer.planChangeLog.level") .internal() .doc("Configures the log level for logging the change from the original plan to the new " + - "plan after a rule is applied. The value can be 'trace', 'debug', 'info', 'warn', or " + - "'error'. The default log level is 'trace'.") + "plan after a rule or batch is applied. The value can be 'trace', 'debug', 'info', " + + "'warn', or 'error'. The default log level is 'trace'.") .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValue(logLevel => Set("TRACE", "DEBUG", "INFO", "WARN", "ERROR").contains(logLevel), @@ -185,12 +196,25 @@ object SQLConf { val OPTIMIZER_PLAN_CHANGE_LOG_RULES = buildConf("spark.sql.optimizer.planChangeLog.rules") .internal() - .doc("If this configuration is set, the optimizer will only log plan changes caused by " + - "applying the rules specified in this configuration. The value can be a list of rule " + - "names separated by comma.") + .doc("Configures a list of rules to be logged in the optimizer, in which the rules are " + + "specified by their rule names and separated by comma.") + .stringConf + .createOptional + + val OPTIMIZER_PLAN_CHANGE_LOG_BATCHES = buildConf("spark.sql.optimizer.planChangeLog.batches") + .internal() + .doc("Configures a list of batches to be logged in the optimizer, in which the batches " + + "are specified by their batch names and separated by comma.") .stringConf .createOptional + val OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID = + buildConf("spark.sql.optimizer.reassignLambdaVariableID") + .doc("When true, Spark optimizer reassigns per-query unique IDs to LambdaVariable, so that " + + "it's more likely to hit codegen cache.") + .booleanConf + .createWithDefault(true) + val COMPRESS_CACHED = buildConf("spark.sql.inMemoryColumnarStorage.compressed") .doc("When set to true Spark SQL will automatically select a compression codec for each " + "column based on statistics of the data.") @@ -210,6 +234,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED = + buildConf("spark.sql.inMemoryTableScanStatistics.enable") + .internal() + .doc("When true, enable in-memory table scan accumulators.") + .booleanConf + .createWithDefault(false) + val CACHE_VECTORIZED_READER_ENABLED = buildConf("spark.sql.inMemoryColumnarStorage.enableVectorizedReader") .doc("Enables vectorized reader for columnar caching.") @@ -244,7 +275,7 @@ object SQLConf { "command ANALYZE TABLE <tableName> COMPUTE STATISTICS noscan has been " + "run, and file-based data source tables where the statistics are computed directly on " + "the files of data.") - .longConf + .bytesConf(ByteUnit.BYTE) .createWithDefault(10L * 1024 * 1024) val LIMIT_SCALE_UP_FACTOR = buildConf("spark.sql.limit.scaleUpFactor") @@ -263,8 +294,11 @@ object SQLConf { .createWithDefault(true) val SHUFFLE_PARTITIONS = buildConf("spark.sql.shuffle.partitions") - .doc("The default number of partitions to use when shuffling data for joins or aggregations.") + .doc("The default number of partitions to use when shuffling data for joins or aggregations. " + + "Note: For structured streaming, this configuration cannot be changed between query " + + "restarts from the same checkpoint location.") .intConf + .checkValue(_ > 0, "The value of spark.sql.shuffle.partitions must be positive") .createWithDefault(200) val SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE = @@ -278,16 +312,30 @@ object SQLConf { .booleanConf .createWithDefault(false) + val REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED = + buildConf("spark.sql.adaptive.reducePostShufflePartitions.enabled") + .doc("When true and adaptive execution is enabled, this enables reducing the number of " + + "post-shuffle partitions based on map output statistics.") + .booleanConf + .createWithDefault(true) + val SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS = buildConf("spark.sql.adaptive.minNumPostShufflePartitions") - .internal() - .doc("The advisory minimal number of post-shuffle partitions provided to " + - "ExchangeCoordinator. This setting is used in our test to make sure we " + - "have enough parallelism to expose issues that will not be exposed with a " + - "single partition. When the value is a non-positive value, this setting will " + - "not be provided to ExchangeCoordinator.") + .doc("The advisory minimum number of post-shuffle partitions used in adaptive execution.") + .intConf + .checkValue(_ > 0, "The minimum shuffle partition number " + + "must be a positive integer.") + .createWithDefault(1) + + val SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS = + buildConf("spark.sql.adaptive.maxNumPostShufflePartitions") + .doc("The advisory maximum number of post-shuffle partitions used in adaptive execution. " + + "This is used as the initial number of pre-shuffle partitions. By default it equals to " + + "spark.sql.shuffle.partitions") .intConf - .createWithDefault(-1) + .checkValue(_ > 0, "The maximum shuffle partition number " + + "must be a positive integer.") + .createOptional val SUBEXPRESSION_ELIMINATION_ENABLED = buildConf("spark.sql.subexpressionElimination.enabled") @@ -312,6 +360,12 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ANSI_SQL_PARSER = + buildConf("spark.sql.parser.ansi.enabled") + .doc("When true, tries to conform to ANSI SQL syntax.") + .booleanConf + .createWithDefault(false) + val ESCAPED_STRING_LITERALS = buildConf("spark.sql.parser.escapedStringLiterals") .internal() .doc("When true, string literals (including regex patterns) remain escaped in our SQL " + @@ -326,7 +380,7 @@ object SQLConf { "factor as the estimated data size, in case the data is compressed in the file and lead to" + " a heavily underestimated result.") .doubleConf - .checkValue(_ > 0, "the value of fileDataSizeFactor must be larger than 0") + .checkValue(_ > 0, "the value of fileDataSizeFactor must be greater than 0") .createWithDefault(1.0) val PARQUET_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.parquet.mergeSchema") @@ -380,7 +434,7 @@ object SQLConf { .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValues(ParquetOutputTimestampType.values.map(_.toString)) - .createWithDefault(ParquetOutputTimestampType.INT96.toString) + .createWithDefault(ParquetOutputTimestampType.TIMESTAMP_MICROS.toString) val PARQUET_INT64_AS_TIMESTAMP_MILLIS = buildConf("spark.sql.parquet.int64AsTimestampMillis") .doc(s"(Deprecated since Spark 2.3, please set ${PARQUET_OUTPUT_TIMESTAMP_TYPE.key}.) " + @@ -408,7 +462,8 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_DATE_ENABLED = buildConf("spark.sql.parquet.filterPushdown.date") .doc("If true, enables Parquet filter push-down optimization for Date. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + + "enabled.") .internal() .booleanConf .createWithDefault(true) @@ -416,7 +471,7 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED = buildConf("spark.sql.parquet.filterPushdown.timestamp") .doc("If true, enables Parquet filter push-down optimization for Timestamp. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is " + + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + "enabled and Timestamp stored as TIMESTAMP_MICROS or TIMESTAMP_MILLIS type.") .internal() .booleanConf @@ -425,7 +480,8 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED = buildConf("spark.sql.parquet.filterPushdown.decimal") .doc("If true, enables Parquet filter push-down optimization for Decimal. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + + "enabled.") .internal() .booleanConf .createWithDefault(true) @@ -433,7 +489,8 @@ object SQLConf { val PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED = buildConf("spark.sql.parquet.filterPushdown.string.startsWith") .doc("If true, enables Parquet filter push-down optimization for string startsWith function. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + + "enabled.") .internal() .booleanConf .createWithDefault(true) @@ -444,7 +501,8 @@ object SQLConf { "Large threshold won't necessarily provide much better performance. " + "The experiment argued that 300 is the limit threshold. " + "By setting this value to 0 this feature can be disabled. " + - "This configuration only has an effect when 'spark.sql.parquet.filterPushdown' is enabled.") + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' is " + + "enabled.") .internal() .intConf .checkValue(threshold => threshold >= 0, "The threshold must not be negative.") @@ -459,13 +517,6 @@ object SQLConf { .booleanConf .createWithDefault(false) - val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") - .doc("If true, enables Parquet's native record-level filtering using the pushed down " + - "filters. This configuration only has an effect when 'spark.sql.parquet.filterPushdown' " + - "is enabled.") - .booleanConf - .createWithDefault(false) - val PARQUET_OUTPUT_COMMITTER_CLASS = buildConf("spark.sql.parquet.output.committer.class") .doc("The output committer class used by Parquet. The specified class needs to be a " + "subclass of org.apache.hadoop.mapreduce.OutputCommitter. Typically, it's also a subclass " + @@ -481,6 +532,15 @@ object SQLConf { .booleanConf .createWithDefault(true) + val PARQUET_RECORD_FILTER_ENABLED = buildConf("spark.sql.parquet.recordLevelFilter.enabled") + .doc("If true, enables Parquet's native record-level filtering using the pushed down " + + "filters. " + + s"This configuration only has an effect when '${PARQUET_FILTER_PUSHDOWN_ENABLED.key}' " + + "is enabled and the vectorized reader is not used. You can ensure the vectorized reader " + + s"is not used by setting '${PARQUET_VECTORIZED_READER_ENABLED.key}' to false.") + .booleanConf + .createWithDefault(false) + val PARQUET_VECTORIZED_READER_BATCH_SIZE = buildConf("spark.sql.parquet.columnarReaderBatchSize") .doc("The number of rows to include in a parquet vectorized reader batch. The number should " + "be carefully chosen to minimize overhead and avoid OOMs in reading data.") @@ -516,18 +576,17 @@ object SQLConf { .intConf .createWithDefault(4096) - val ORC_COPY_BATCH_TO_SPARK = buildConf("spark.sql.orc.copyBatchToSpark") - .doc("Whether or not to copy the ORC columnar batch to Spark columnar batch in the " + - "vectorized ORC reader.") - .internal() - .booleanConf - .createWithDefault(false) - val ORC_FILTER_PUSHDOWN_ENABLED = buildConf("spark.sql.orc.filterPushdown") .doc("When true, enable filter pushdown for ORC files.") .booleanConf .createWithDefault(true) + val ORC_SCHEMA_MERGING_ENABLED = buildConf("spark.sql.orc.mergeSchema") + .doc("When true, the Orc data source merges schemas collected from all data files, " + + "otherwise the schema is picked from a random data file.") + .booleanConf + .createWithDefault(false) + val HIVE_VERIFY_PARTITION_PATH = buildConf("spark.sql.hive.verifyPartitionPath") .doc("When true, check all the partition paths under the table\'s root directory " + "when reading data stored in HDFS. This configuration will be deprecated in the future " + @@ -566,27 +625,27 @@ object SQLConf { } val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode") - .doc("Sets the action to take when a case-sensitive schema cannot be read from a Hive " + - "table's properties. Although Spark SQL itself is not case-sensitive, Hive compatible file " + - "formats such as Parquet are. Spark SQL must use a case-preserving schema when querying " + - "any table backed by files containing case-sensitive field names or queries may not return " + - "accurate results. Valid options include INFER_AND_SAVE (the default mode-- infer the " + - "case-sensitive schema from the underlying data files and write it back to the table " + - "properties), INFER_ONLY (infer the schema but don't attempt to write it to the table " + - "properties) and NEVER_INFER (fallback to using the case-insensitive metastore schema " + - "instead of inferring).") + .internal() + .doc("Sets the action to take when a case-sensitive schema cannot be read from a Hive Serde " + + "table's properties when reading the table with Spark native data sources. Valid options " + + "include INFER_AND_SAVE (infer the case-sensitive schema from the underlying data files " + + "and write it back to the table properties), INFER_ONLY (infer the schema but don't " + + "attempt to write it to the table properties) and NEVER_INFER (the default mode-- fallback " + + "to using the case-insensitive metastore schema instead of inferring).") .stringConf .transform(_.toUpperCase(Locale.ROOT)) .checkValues(HiveCaseSensitiveInferenceMode.values.map(_.toString)) - .createWithDefault(HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) + .createWithDefault(HiveCaseSensitiveInferenceMode.NEVER_INFER.toString) val OPTIMIZER_METADATA_ONLY = buildConf("spark.sql.optimizer.metadataOnly") + .internal() .doc("When true, enable the metadata-only query optimization that use the table's metadata " + "to produce the partition columns instead of table scans. It applies when all the columns " + "scanned are partition columns and the query has an aggregate operator that satisfies " + - "distinct semantics.") + "distinct semantics. By default the optimization is disabled, since it may return " + + "incorrect results when the files are empty.") .booleanConf - .createWithDefault(true) + .createWithDefault(false) val COLUMN_NAME_OF_CORRUPT_RECORD = buildConf("spark.sql.columnNameOfCorruptRecord") .doc("The name of internal column for storing raw/un-parsed JSON and CSV records that fail " + @@ -641,7 +700,7 @@ object SQLConf { .internal() .doc("When true, a table created by a Hive CTAS statement (no USING clause) " + "without specifying any storage property will be converted to a data source table, " + - "using the data source set by spark.sql.sources.default.") + s"using the data source set by ${DEFAULT_DATA_SOURCE_NAME.key}.") .booleanConf .createWithDefault(false) @@ -667,7 +726,7 @@ object SQLConf { val BUCKETING_MAX_BUCKETS = buildConf("spark.sql.sources.bucketing.maxBuckets") .doc("The maximum number of buckets allowed. Defaults to 100000") .intConf - .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be larger than 0") + .checkValue(_ > 0, "the value of spark.sql.sources.bucketing.maxBuckets must be greater than 0") .createWithDefault(100000) val CROSS_JOINS_ENABLED = buildConf("spark.sql.crossJoin.enabled") @@ -812,6 +871,18 @@ object SQLConf { .intConf .createWithDefault(65535) + val CODEGEN_METHOD_SPLIT_THRESHOLD = buildConf("spark.sql.codegen.methodSplitThreshold") + .internal() + .doc("The threshold of source-code splitting in the codegen. When the number of characters " + + "in a single Java function (without comment) exceeds the threshold, the function will be " + + "automatically split to multiple smaller ones. We cannot know how many bytecode will be " + + "generated, so use the code length as metric. When running on HotSpot, a function's " + + "bytecode should not go beyond 8KB, otherwise it will not be JITted; it also should not " + + "be too small, otherwise there will be many function calls.") + .intConf + .checkValue(threshold => threshold > 0, "The threshold must be a positive integer.") + .createWithDefault(1024) + val WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR = buildConf("spark.sql.codegen.splitConsumeFuncByOperator") .internal() @@ -823,7 +894,7 @@ object SQLConf { val FILES_MAX_PARTITION_BYTES = buildConf("spark.sql.files.maxPartitionBytes") .doc("The maximum number of bytes to pack into a single partition when reading files.") - .longConf + .bytesConf(ByteUnit.BYTE) .createWithDefault(128 * 1024 * 1024) // parquet.block.size val FILES_OPEN_COST_IN_BYTES = buildConf("spark.sql.files.openCostInBytes") @@ -859,12 +930,20 @@ object SQLConf { .booleanConf .createWithDefault(true) + val SUBQUERY_REUSE_ENABLED = buildConf("spark.sql.subquery.reuse") + .internal() + .doc("When true, the planner will try to find out duplicated subqueries and re-use them.") + .booleanConf + .createWithDefault(true) + val STATE_STORE_PROVIDER_CLASS = buildConf("spark.sql.streaming.stateStore.providerClass") .internal() .doc( "The class used to manage state data in stateful streaming queries. This class must " + - "be a subclass of StateStoreProvider, and must have a zero-arg constructor.") + "be a subclass of StateStoreProvider, and must have a zero-arg constructor. " + + "Note: For structured streaming, this configuration cannot be changed between query " + + "restarts from the same checkpoint location.") .stringConf .createWithDefault( "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider") @@ -890,6 +969,12 @@ object SQLConf { .stringConf .createOptional + val FORCE_DELETE_TEMP_CHECKPOINT_LOCATION = + buildConf("spark.sql.streaming.forceDeleteTempCheckpointLocation") + .doc("When true, enable temporary checkpoint locations force delete.") + .booleanConf + .createWithDefault(false) + val MIN_BATCHES_TO_RETAIN = buildConf("spark.sql.streaming.minBatchesToRetain") .internal() .doc("The minimum number of batches that must be retained and made recoverable.") @@ -946,6 +1031,13 @@ object SQLConf { .booleanConf .createWithDefault(true) + val ENABLE_VECTORIZED_HASH_MAP = + buildConf("spark.sql.codegen.aggregate.map.vectorized.enable") + .internal() + .doc("Enable vectorized aggregate hash map. This is for testing/benchmarking only.") + .booleanConf + .createWithDefault(false) + val MAX_NESTED_VIEW_DEPTH = buildConf("spark.sql.view.maxNestedViewDepth") .internal() @@ -1089,6 +1181,14 @@ object SQLConf { .internal() .stringConf + val STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED = + buildConf("spark.sql.streaming.checkpoint.escapedPathCheck.enabled") + .doc("Whether to detect a streaming query may pick up an incorrect checkpoint path due " + + "to SPARK-26824.") + .internal() + .booleanConf + .createWithDefault(true) + val PARALLEL_FILE_LISTING_IN_STATS_COMPUTATION = buildConf("spark.sql.statistics.parallelFileListingInStatsComputation.enabled") .internal() @@ -1107,10 +1207,10 @@ object SQLConf { val DEFAULT_SIZE_IN_BYTES = buildConf("spark.sql.defaultSizeInBytes") .internal() .doc("The default table size used in query planning. By default, it is set to Long.MaxValue " + - "which is larger than `spark.sql.autoBroadcastJoinThreshold` to be more conservative. " + + s"which is larger than `${AUTO_BROADCASTJOIN_THRESHOLD.key}` to be more conservative. " + "That is to say by default the optimizer will not choose to broadcast a table unless it " + "knows for sure its size is small enough.") - .longConf + .bytesConf(ByteUnit.BYTE) .createWithDefault(Long.MaxValue) val NDV_MAX_ERROR = @@ -1136,7 +1236,7 @@ object SQLConf { .internal() .doc("The number of bins when generating histograms.") .intConf - .checkValue(num => num > 1, "The number of bins must be larger than 1.") + .checkValue(num => num > 1, "The number of bins must be greater than 1.") .createWithDefault(254) val PERCENTILE_ACCURACY = @@ -1228,7 +1328,7 @@ object SQLConf { .doc("Threshold for number of rows guaranteed to be held in memory by the sort merge " + "join operator") .intConf - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) val SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD = buildConf("spark.sql.sortMergeJoinExec.buffer.spill.threshold") @@ -1268,21 +1368,46 @@ object SQLConf { val ARROW_EXECUTION_ENABLED = buildConf("spark.sql.execution.arrow.enabled") - .doc("When true, make use of Apache Arrow for columnar data transfers. Currently available " + - "for use with pyspark.sql.DataFrame.toPandas, and " + - "pyspark.sql.SparkSession.createDataFrame when its input is a Pandas DataFrame. " + + .doc("(Deprecated since Spark 3.0, please set 'spark.sql.execution.arrow.pyspark.enabled'.)") + .booleanConf + .createWithDefault(false) + + val ARROW_PYSPARK_EXECUTION_ENABLED = + buildConf("spark.sql.execution.arrow.pyspark.enabled") + .doc("When true, make use of Apache Arrow for columnar data transfers in PySpark. " + + "This optimization applies to: " + + "1. pyspark.sql.DataFrame.toPandas " + + "2. pyspark.sql.SparkSession.createDataFrame when its input is a Pandas DataFrame " + "The following data types are unsupported: " + "BinaryType, MapType, ArrayType of TimestampType, and nested StructType.") + .fallbackConf(ARROW_EXECUTION_ENABLED) + + val ARROW_SPARKR_EXECUTION_ENABLED = + buildConf("spark.sql.execution.arrow.sparkr.enabled") + .doc("When true, make use of Apache Arrow for columnar data transfers in SparkR. " + + "This optimization applies to: " + + "1. createDataFrame when its input is an R DataFrame " + + "2. collect " + + "3. dapply " + + "4. gapply " + + "The following data types are unsupported: " + + "FloatType, BinaryType, ArrayType, StructType and MapType.") .booleanConf .createWithDefault(false) val ARROW_FALLBACK_ENABLED = buildConf("spark.sql.execution.arrow.fallback.enabled") - .doc("When true, optimizations enabled by 'spark.sql.execution.arrow.enabled' will " + - "fallback automatically to non-optimized implementations if an error occurs.") + .doc("(Deprecated since Spark 3.0, please set " + + "'spark.sql.execution.arrow.pyspark.fallback.enabled'.)") .booleanConf .createWithDefault(true) + val ARROW_PYSPARK_FALLBACK_ENABLED = + buildConf("spark.sql.execution.arrow.pyspark.fallback.enabled") + .doc(s"When true, optimizations enabled by '${ARROW_PYSPARK_EXECUTION_ENABLED.key}' will " + + "fallback automatically to non-optimized implementations if an error occurs.") + .fallbackConf(ARROW_FALLBACK_ENABLED) + val ARROW_EXECUTION_MAX_RECORDS_PER_BATCH = buildConf("spark.sql.execution.arrow.maxRecordsPerBatch") .doc("When using Apache Arrow, limit the maximum number of records that can be written " + @@ -1290,6 +1415,15 @@ object SQLConf { .intConf .createWithDefault(10000) + val PANDAS_UDF_BUFFER_SIZE = + buildConf("spark.sql.pandas.udf.buffer.size") + .doc( + s"Same as ${BUFFER_SIZE} but only applies to Pandas UDF executions. If it is not set, " + + s"the fallback is ${BUFFER_SIZE}. 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.") + .fallbackConf(BUFFER_SIZE) + val PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE = buildConf("spark.sql.execution.pandas.respectSessionTimeZone") .internal() @@ -1309,6 +1443,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val PANDAS_ARROW_SAFE_TYPE_CONVERSION = + buildConf("spark.sql.execution.pandas.arrowSafeTypeConversion") + .internal() + .doc("When true, Arrow will perform safe type conversion when converting " + + "Pandas.Series to Arrow array during serialization. Arrow will raise errors " + + "when detecting unsafe type conversion like overflow. When false, disabling Arrow's type " + + "check and do type conversions anyway. This config only works for Arrow 0.11.0+.") + .booleanConf + .createWithDefault(false) + val REPLACE_EXCEPT_WITH_FILTER = buildConf("spark.sql.optimizer.replaceExceptWithFilter") .internal() .doc("When true, the apply function of the rule verifies whether the right node of the" + @@ -1332,6 +1476,16 @@ object SQLConf { .booleanConf .createWithDefault(true) + val DECIMAL_OPERATIONS_NULL_ON_OVERFLOW = + buildConf("spark.sql.decimalOperations.nullOnOverflow") + .internal() + .doc("When true (default), if an overflow on a decimal occurs, then NULL is returned. " + + "Spark's older versions and Hive behave in this way. If turned to false, SQL ANSI 2011 " + + "specification will be followed instead: an arithmetic exception is thrown, as most " + + "of the SQL databases do.") + .booleanConf + .createWithDefault(true) + val LITERAL_PICK_MINIMUM_PRECISION = buildConf("spark.sql.legacy.literal.pickMinimumPrecision") .internal() @@ -1370,6 +1524,12 @@ object SQLConf { .booleanConf .createWithDefault(false) + val PREFER_INTEGRAL_DIVISION = buildConf("spark.sql.function.preferIntegralDivision") + .doc("When true, will perform integral division with the / operator " + + "if both sides are integral types.") + .booleanConf + .createWithDefault(false) + val ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION = buildConf("spark.sql.legacy.allowCreatingManagedTableUsingNonemptyLocation") .internal() @@ -1378,6 +1538,23 @@ object SQLConf { .booleanConf .createWithDefault(false) + val VALIDATE_PARTITION_COLUMNS = + buildConf("spark.sql.sources.validatePartitionColumns") + .internal() + .doc("When this option is set to true, partition column values will be validated with " + + "user-specified schema. If the validation fails, a runtime exception is thrown." + + "When this option is set to false, the partition column value will be converted to null " + + "if it can not be casted to corresponding user-specified schema.") + .booleanConf + .createWithDefault(true) + + val CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE = + buildConf("spark.sql.streaming.continuous.epochBacklogQueueSize") + .doc("The max number of entries to be stored in queue to wait for late epochs. " + + "If this parameter is exceeded by the size of the queue, stream will stop with an error.") + .intConf + .createWithDefault(10000) + val CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE = buildConf("spark.sql.streaming.continuous.executorQueueSize") .internal() @@ -1394,8 +1571,23 @@ object SQLConf { .timeConf(TimeUnit.MILLISECONDS) .createWithDefault(100) - val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") + val USE_V1_SOURCE_READER_LIST = buildConf("spark.sql.sources.read.useV1SourceList") .internal() + .doc("A comma-separated list of data source short names or fully qualified data source" + + " register class names for which data source V2 read paths are disabled. Reads from these" + + " sources will fall back to the V1 sources.") + .stringConf + .createWithDefault("") + + val USE_V1_SOURCE_WRITER_LIST = buildConf("spark.sql.sources.write.useV1SourceList") + .internal() + .doc("A comma-separated list of data source short names or fully qualified data source" + + " register class names for which data source V2 write paths are disabled. Writes from these" + + " sources will fall back to the V1 sources.") + .stringConf + .createWithDefault("csv,json,orc,text,parquet") + + val DISABLED_V2_STREAMING_WRITERS = buildConf("spark.sql.streaming.disabledV2Writers") .doc("A comma-separated list of fully qualified data source register class names for which" + " StreamWriteSupport is disabled. Writes to these sources will fall back to the V1 Sinks.") .stringConf @@ -1450,8 +1642,27 @@ object SQLConf { .internal() .doc("Prune nested fields from a logical relation's output which are unnecessary in " + "satisfying a query. This optimization allows columnar file format readers to avoid " + - "reading unnecessary nested column data. Currently Parquet is the only data source that " + - "implements this optimization.") + "reading unnecessary nested column data. Currently Parquet and ORC are the " + + "data sources that implement this optimization.") + .booleanConf + .createWithDefault(false) + + val SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED = + buildConf("spark.sql.optimizer.serializer.nestedSchemaPruning.enabled") + .internal() + .doc("Prune nested fields from object serialization operator which are unnecessary in " + + "satisfying a query. This optimization allows object serializers to avoid " + + "executing unnecessary nested expressions.") + .booleanConf + .createWithDefault(false) + + val NESTED_PRUNING_ON_EXPRESSIONS = + buildConf("spark.sql.optimizer.expression.nestedPruning.enabled") + .internal() + .doc("Prune nested fields from expressions in an operator which are unnecessary in " + + "satisfying a query. Note that this optimization doesn't prune nested fields from " + + "physical data source scanning. For pruning nested fields from scanning, please use " + + "`spark.sql.optimizer.nestedSchemaPruning.enabled` config.") .booleanConf .createWithDefault(false) @@ -1462,7 +1673,7 @@ object SQLConf { "'SELECT x FROM t ORDER BY y LIMIT m', if m is under this threshold, do a top-K sort" + " in memory, otherwise do a global sort which spills to disk if necessary.") .intConf - .createWithDefault(Int.MaxValue) + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) object Deprecated { val MAPRED_REDUCE_TASKS = "mapred.reduce.tasks" @@ -1482,15 +1693,16 @@ object SQLConf { val REPL_EAGER_EVAL_ENABLED = buildConf("spark.sql.repl.eagerEval.enabled") .doc("Enables eager evaluation or not. When true, the top K rows of Dataset will be " + "displayed if and only if the REPL supports the eager evaluation. Currently, the " + - "eager evaluation is only supported in PySpark. For the notebooks like Jupyter, " + - "the HTML table (generated by _repr_html_) will be returned. For plain Python REPL, " + - "the returned outputs are formatted like dataframe.show().") + "eager evaluation is supported in PySpark and SparkR. In PySpark, for the notebooks like " + + "Jupyter, the HTML table (generated by _repr_html_) will be returned. For plain Python " + + "REPL, the returned outputs are formatted like dataframe.show(). In SparkR, the returned " + + "outputs are showed similar to R data.frame would.") .booleanConf .createWithDefault(false) val REPL_EAGER_EVAL_MAX_NUM_ROWS = buildConf("spark.sql.repl.eagerEval.maxNumRows") .doc("The max number of rows that are returned by eager evaluation. This only takes " + - "effect when spark.sql.repl.eagerEval.enabled is set to true. The valid range of this " + + s"effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true. The valid range of this " + "config is from 0 to (Int.MaxValue - 1), so the invalid config like negative and " + "greater than (Int.MaxValue - 1) will be normalized to 0 and (Int.MaxValue - 1).") .intConf @@ -1498,7 +1710,7 @@ object SQLConf { val REPL_EAGER_EVAL_TRUNCATE = buildConf("spark.sql.repl.eagerEval.truncate") .doc("The max number of characters for each cell that is returned by eager evaluation. " + - "This only takes effect when spark.sql.repl.eagerEval.enabled is set to true.") + s"This only takes effect when ${REPL_EAGER_EVAL_ENABLED.key} is set to true.") .intConf .createWithDefault(20) @@ -1575,6 +1787,97 @@ object SQLConf { "WHERE, which does not follow SQL standard.") .booleanConf .createWithDefault(false) + + val NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE = + buildConf("spark.sql.legacy.dataset.nameNonStructGroupingKeyAsValue") + .internal() + .doc("When set to true, the key attribute resulted from running `Dataset.groupByKey` " + + "for non-struct key type, will be named as `value`, following the behavior of Spark " + + "version 2.4 and earlier.") + .booleanConf + .createWithDefault(false) + + val MAX_TO_STRING_FIELDS = buildConf("spark.sql.debug.maxToStringFields") + .doc("Maximum number of fields of sequence-like entries can be converted to strings " + + "in debug output. Any elements beyond the limit will be dropped and replaced by a" + + """ "... N more fields" placeholder.""") + .intConf + .createWithDefault(25) + + val MAX_PLAN_STRING_LENGTH = buildConf("spark.sql.maxPlanStringLength") + .doc("Maximum number of characters to output for a plan string. If the plan is " + + "longer, further output will be truncated. The default setting always generates a full " + + "plan. Set this to a lower value such as 8k if plan strings are taking up too much " + + "memory or are causing OutOfMemory errors in the driver or UI processes.") + .bytesConf(ByteUnit.BYTE) + .checkValue(i => i >= 0 && i <= ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH, "Invalid " + + "value for 'spark.sql.maxPlanStringLength'. Length must be a valid string length " + + "(nonnegative and shorter than the maximum size).") + .createWithDefault(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH) + + val SET_COMMAND_REJECTS_SPARK_CORE_CONFS = + buildConf("spark.sql.legacy.setCommandRejectsSparkCoreConfs") + .internal() + .doc("If it is set to true, SET command will fail when the key is registered as " + + "a SparkConf entry.") + .booleanConf + .createWithDefault(true) + + val DATETIME_JAVA8API_ENABLED = buildConf("spark.sql.datetime.java8API.enabled") + .doc("If the configuration property is set to true, java.time.Instant and " + + "java.time.LocalDate classes of Java 8 API are used as external types for " + + "Catalyst's TimestampType and DateType. If it is set to false, java.sql.Timestamp " + + "and java.sql.Date are used for the same purpose.") + .booleanConf + .createWithDefault(false) + + val UTC_TIMESTAMP_FUNC_ENABLED = buildConf("spark.sql.legacy.utcTimestampFunc.enabled") + .doc("The configuration property enables the to_utc_timestamp() " + + "and from_utc_timestamp() functions.") + .booleanConf + .createWithDefault(false) + + 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. " + + "The theoretical max is Int.MaxValue, though VMs might implement a smaller max.") + .internal() + .intConf + .createWithDefault(Int.MaxValue) + + val LEGACY_CAST_DATETIME_TO_STRING = + buildConf("spark.sql.legacy.typeCoercion.datetimeToString") + .doc("If it is set to true, date/timestamp will cast to string in binary comparisons " + + "with String") + .booleanConf + .createWithDefault(false) + + val DEFAULT_V2_CATALOG = buildConf("spark.sql.default.catalog") + .doc("Name of the default v2 catalog, used when a catalog is not identified in queries") + .stringConf + .createOptional + + val V2_SESSION_CATALOG = buildConf("spark.sql.catalog.session") + .doc("Name of the default v2 catalog, used when a catalog is not identified in queries") + .stringConf + .createWithDefault("org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog") + + val LEGACY_LOOSE_UPCAST = buildConf("spark.sql.legacy.looseUpcast") + .doc("When true, the upcast will be loose and allows string to atomic types.") + .booleanConf + .createWithDefault(false) + + val LEGACY_CTE_PRECEDENCE_ENABLED = buildConf("spark.sql.legacy.ctePrecedence.enabled") + .internal() + .doc("When true, outer CTE definitions takes precedence over inner definitions.") + .booleanConf + .createWithDefault(false) + + val LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC = + buildConf("spark.sql.legacy.arrayExistsFollowsThreeValuedLogic") + .doc("When true, the ArrayExists will follow the three-valued boolean logic.") + .booleanConf + .createWithDefault(true) } /** @@ -1603,10 +1906,14 @@ class SQLConf extends Serializable with Logging { def optimizerInSetConversionThreshold: Int = getConf(OPTIMIZER_INSET_CONVERSION_THRESHOLD) + def optimizerInSetSwitchThreshold: Int = getConf(OPTIMIZER_INSET_SWITCH_THRESHOLD) + def optimizerPlanChangeLogLevel: String = getConf(OPTIMIZER_PLAN_CHANGE_LOG_LEVEL) def optimizerPlanChangeRules: Option[String] = getConf(OPTIMIZER_PLAN_CHANGE_LOG_RULES) + def optimizerPlanChangeBatches: Option[String] = getConf(OPTIMIZER_PLAN_CHANGE_LOG_BATCHES) + def stateStoreProviderClass: String = getConf(STATE_STORE_PROVIDER_CLASS) def stateStoreMinDeltasForSnapshot: Int = getConf(STATE_STORE_MIN_DELTAS_FOR_SNAPSHOT) @@ -1678,9 +1985,14 @@ class SQLConf extends Serializable with Logging { def adaptiveExecutionEnabled: Boolean = getConf(ADAPTIVE_EXECUTION_ENABLED) + def reducePostShufflePartitionsEnabled: Boolean = getConf(REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED) + def minNumPostShufflePartitions: Int = getConf(SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS) + def maxNumPostShufflePartitions: Int = + getConf(SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS).getOrElse(numShufflePartitions) + def minBatchesToRetain: Int = getConf(MIN_BATCHES_TO_RETAIN) def maxBatchesToRetainInMemory: Int = getConf(MAX_BATCHES_TO_RETAIN_IN_MEMORY) @@ -1701,6 +2013,8 @@ class SQLConf extends Serializable with Logging { def orcFilterPushDown: Boolean = getConf(ORC_FILTER_PUSHDOWN_ENABLED) + def isOrcSchemaMergingEnabled: Boolean = getConf(ORC_SCHEMA_MERGING_ENABLED) + def verifyPartitionPath: Boolean = getConf(HIVE_VERIFY_PARTITION_PATH) def metastorePartitionPruning: Boolean = getConf(HIVE_METASTORE_PARTITION_PRUNING) @@ -1732,6 +2046,8 @@ class SQLConf extends Serializable with Logging { def hugeMethodLimit: Int = getConf(WHOLESTAGE_HUGE_METHOD_LIMIT) + def methodSplitThreshold: Int = getConf(CODEGEN_METHOD_SPLIT_THRESHOLD) + def wholeStageSplitConsumeFuncByOperator: Boolean = getConf(WHOLESTAGE_SPLIT_CONSUME_FUNC_BY_OPERATOR) @@ -1742,10 +2058,14 @@ class SQLConf extends Serializable with Logging { def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED) + def subqueryReuseEnabled: Boolean = getConf(SUBQUERY_REUSE_ENABLED) + def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE) def constraintPropagationEnabled: Boolean = getConf(CONSTRAINT_PROPAGATION_ENABLED) + def ansiParserEnabled: Boolean = getConf(ANSI_SQL_PARSER) + def escapedStringLiterals: Boolean = getConf(ESCAPED_STRING_LITERALS) def fileCompressionFactor: Double = getConf(FILE_COMRESSION_FACTOR) @@ -1758,6 +2078,10 @@ class SQLConf extends Serializable with Logging { def fastHashAggregateRowMaxCapacityBit: Int = getConf(FAST_HASH_AGGREGATE_MAX_ROWS_CAPACITY_BIT) + def datetimeJava8ApiEnabled: Boolean = getConf(DATETIME_JAVA8API_ENABLED) + + def utcTimestampFuncEnabled: Boolean = getConf(UTC_TIMESTAMP_FUNC_ENABLED) + /** * Returns the [[Resolver]] for the current configuration, which can be used to determine if two * identifiers are equal. @@ -1770,6 +2094,11 @@ class SQLConf extends Serializable with Logging { } } + /** + * Returns the error handler for handling hint errors. + */ + def hintErrorHandler: HintErrorHandler = HintErrorLogger + def subexpressionEliminationEnabled: Boolean = getConf(SUBEXPRESSION_ELIMINATION_ENABLED) @@ -1816,11 +2145,16 @@ class SQLConf extends Serializable with Logging { def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING) + def inMemoryTableScanStatisticsEnabled: Boolean = getConf(IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED) + def offHeapColumnVectorEnabled: Boolean = getConf(COLUMN_VECTOR_OFFHEAP_ENABLED) def columnNameOfCorruptRecord: String = getConf(COLUMN_NAME_OF_CORRUPT_RECORD) - def broadcastTimeout: Long = getConf(BROADCAST_TIMEOUT) + def broadcastTimeout: Long = { + val timeoutValue = getConf(BROADCAST_TIMEOUT) + if (timeoutValue < 0) Long.MaxValue else timeoutValue + } def defaultDataSourceName: String = getConf(DEFAULT_DATA_SOURCE_NAME) @@ -1852,6 +2186,8 @@ class SQLConf extends Serializable with Logging { def enableTwoLevelAggMap: Boolean = getConf(ENABLE_TWOLEVEL_AGG_MAP) + def enableVectorizedHashMap: Boolean = getConf(ENABLE_VECTORIZED_HASH_MAP) + def useObjectHashAggregation: Boolean = getConf(USE_OBJECT_HASH_AGG) def objectAggSortBasedFallbackThreshold: Int = getConf(OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD) @@ -1928,28 +2264,43 @@ class SQLConf extends Serializable with Logging { def rangeExchangeSampleSizePerPartition: Int = getConf(RANGE_EXCHANGE_SAMPLE_SIZE_PER_PARTITION) - def arrowEnabled: Boolean = getConf(ARROW_EXECUTION_ENABLED) + def arrowPySparkEnabled: Boolean = getConf(ARROW_PYSPARK_EXECUTION_ENABLED) + + def arrowSparkREnabled: Boolean = getConf(ARROW_SPARKR_EXECUTION_ENABLED) - def arrowFallbackEnabled: Boolean = getConf(ARROW_FALLBACK_ENABLED) + def arrowPySparkFallbackEnabled: Boolean = getConf(ARROW_PYSPARK_FALLBACK_ENABLED) def arrowMaxRecordsPerBatch: Int = getConf(ARROW_EXECUTION_MAX_RECORDS_PER_BATCH) + def pandasUDFBufferSize: Int = getConf(PANDAS_UDF_BUFFER_SIZE) + def pandasRespectSessionTimeZone: Boolean = getConf(PANDAS_RESPECT_SESSION_LOCAL_TIMEZONE) def pandasGroupedMapAssignColumnsByName: Boolean = getConf(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME) + def arrowSafeTypeConversion: Boolean = getConf(SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION) + def replaceExceptWithFilter: Boolean = getConf(REPLACE_EXCEPT_WITH_FILTER) def decimalOperationsAllowPrecisionLoss: Boolean = getConf(DECIMAL_OPERATIONS_ALLOW_PREC_LOSS) + def decimalOperationsNullOnOverflow: Boolean = getConf(DECIMAL_OPERATIONS_NULL_ON_OVERFLOW) + def literalPickMinimumPrecision: Boolean = getConf(LITERAL_PICK_MINIMUM_PRECISION) + def continuousStreamingEpochBacklogQueueSize: Int = + getConf(CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE) + def continuousStreamingExecutorQueueSize: Int = getConf(CONTINUOUS_STREAMING_EXECUTOR_QUEUE_SIZE) def continuousStreamingExecutorPollIntervalMs: Long = getConf(CONTINUOUS_STREAMING_EXECUTOR_POLL_INTERVAL_MS) + def useV1SourceReaderList: String = getConf(USE_V1_SOURCE_READER_LIST) + + def useV1SourceWriterList: String = getConf(USE_V1_SOURCE_WRITER_LIST) + def disabledV2StreamingWriters: String = getConf(DISABLED_V2_STREAMING_WRITERS) def disabledV2StreamingMicroBatchReaders: String = @@ -1959,14 +2310,23 @@ class SQLConf extends Serializable with Logging { def eltOutputAsString: Boolean = getConf(ELT_OUTPUT_AS_STRING) + def preferIntegralDivision: Boolean = getConf(PREFER_INTEGRAL_DIVISION) + def allowCreatingManagedTableUsingNonemptyLocation: Boolean = getConf(ALLOW_CREATING_MANAGED_TABLE_USING_NONEMPTY_LOCATION) + def validatePartitionColumns: Boolean = getConf(VALIDATE_PARTITION_COLUMNS) + def partitionOverwriteMode: PartitionOverwriteMode.Value = PartitionOverwriteMode.withName(getConf(PARTITION_OVERWRITE_MODE)) def nestedSchemaPruningEnabled: Boolean = getConf(NESTED_SCHEMA_PRUNING_ENABLED) + def serializerNestedSchemaPruningEnabled: Boolean = + getConf(SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED) + + def nestedPruningOnExpressions: Boolean = getConf(NESTED_PRUNING_ON_EXPRESSIONS) + def csvColumnPruning: Boolean = getConf(SQLConf.CSV_PARSER_COLUMN_PRUNING) def legacySizeOfNull: Boolean = getConf(SQLConf.LEGACY_SIZE_OF_NULL) @@ -1988,6 +2348,20 @@ class SQLConf extends Serializable with Logging { def integralDivideReturnLong: Boolean = getConf(SQLConf.LEGACY_INTEGRALDIVIDE_RETURN_LONG) + def nameNonStructGroupingKeyAsValue: Boolean = + getConf(SQLConf.NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE) + + def maxToStringFields: Int = getConf(SQLConf.MAX_TO_STRING_FIELDS) + + def maxPlanStringLength: Int = getConf(SQLConf.MAX_PLAN_STRING_LENGTH).toInt + + def setCommandRejectsSparkCoreConfs: Boolean = + getConf(SQLConf.SET_COMMAND_REJECTS_SPARK_CORE_CONFS) + + def castDatetimeToString: Boolean = getConf(SQLConf.LEGACY_CAST_DATETIME_TO_STRING) + + def defaultV2Catalog: Option[String] = getConf(DEFAULT_V2_CATALOG) + /** ********************** SQLConf functionality methods ************ */ /** Set Spark SQL configuration properties. */ @@ -2097,7 +2471,7 @@ class SQLConf extends Serializable with Logging { /** * Redacts the given option map according to the description of SQL_OPTIONS_REDACTION_PATTERN. */ - def redactOptions(options: Map[String, String]): Map[String, String] = { + def redactOptions[K, V](options: Map[K, V]): Map[K, V] = { val regexes = Seq( getConf(SQL_OPTIONS_REDACTION_PATTERN), SECRET_REDACTION_PATTERN.readFrom(reader)) 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 d9c354b165e52..d665d16ae4195 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.internal +import java.util.Locale + import org.apache.spark.util.Utils @@ -42,6 +44,7 @@ object StaticSQLConf { val GLOBAL_TEMP_DATABASE = buildStaticConf("spark.sql.globalTempDatabase") .internal() .stringConf + .transform(_.toLowerCase(Locale.ROOT)) .createWithDefault("global_temp") // This is used to control when we will split a schema's JSON string to multiple pieces @@ -99,9 +102,15 @@ object StaticSQLConf { .createWithDefault(false) val SPARK_SESSION_EXTENSIONS = buildStaticConf("spark.sql.extensions") - .doc("Name of the class used to configure Spark Session extensions. The class should " + - "implement Function1[SparkSessionExtension, Unit], and must have a no-args constructor.") + .doc("A comma-separated list of classes that implement " + + "Function1[SparkSessionExtensions, Unit] used to configure Spark Session extensions. The " + + "classes must have a no-args constructor. If multiple extensions are specified, they are " + + "applied in the specified order. For the case of rules and planner strategies, they are " + + "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.") .stringConf + .toSequence .createOptional val QUERY_EXECUTION_LISTENERS = buildStaticConf("spark.sql.queryExecutionListeners") @@ -126,4 +135,30 @@ object StaticSQLConf { .intConf .createWithDefault(1000) + val BROADCAST_EXCHANGE_MAX_THREAD_THRESHOLD = + buildStaticConf("spark.sql.broadcastExchange.maxThreadThreshold") + .internal() + .doc("The maximum degree of parallelism to fetch and broadcast the table. " + + "If we encounter memory issue like frequently full GC or OOM when broadcast table " + + "we can decrease this number in order to reduce memory usage. " + + "Notice the number should be carefully chosen since decreasing parallelism might " + + "cause longer waiting for other broadcasting. Also, increasing parallelism may " + + "cause memory problem.") + .intConf + .checkValue(thres => thres > 0 && thres <= 128, "The threshold must be in [0,128].") + .createWithDefault(128) + + 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.") + .intConf + .checkValue(_ >= 0, "Must be set greater or equal to zero") + .createWithDefault(Int.MaxValue) + + val SQL_LEGACY_SESSION_INIT_WITH_DEFAULTS = + 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.") + .booleanConf + .createWithDefault(false) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala similarity index 90% rename from sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala index bdd8c4da6bd30..a1ab55a7185ce 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/filters.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/sources/filters.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.{Evolving, Stable} //////////////////////////////////////////////////////////////////////////////////////////////////// // This file defines all the filters that we can push down to the data sources. @@ -28,7 +28,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class Filter { /** * List of columns that are referenced by this filter. @@ -48,7 +48,7 @@ abstract class Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class EqualTo(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -60,7 +60,7 @@ case class EqualTo(attribute: String, value: Any) extends Filter { * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable case class EqualNullSafe(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -71,7 +71,7 @@ case class EqualNullSafe(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class GreaterThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -82,7 +82,7 @@ case class GreaterThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -93,7 +93,7 @@ case class GreaterThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class LessThan(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -104,7 +104,7 @@ case class LessThan(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class LessThanOrEqual(attribute: String, value: Any) extends Filter { override def references: Array[String] = Array(attribute) ++ findReferences(value) } @@ -114,7 +114,7 @@ case class LessThanOrEqual(attribute: String, value: Any) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class In(attribute: String, values: Array[Any]) extends Filter { override def hashCode(): Int = { var h = attribute.hashCode @@ -141,7 +141,7 @@ case class In(attribute: String, values: Array[Any]) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class IsNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -151,7 +151,7 @@ case class IsNull(attribute: String) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class IsNotNull(attribute: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -161,7 +161,7 @@ case class IsNotNull(attribute: String) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class And(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -171,7 +171,7 @@ case class And(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class Or(left: Filter, right: Filter) extends Filter { override def references: Array[String] = left.references ++ right.references } @@ -181,7 +181,7 @@ case class Or(left: Filter, right: Filter) extends Filter { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class Not(child: Filter) extends Filter { override def references: Array[String] = child.references } @@ -192,7 +192,7 @@ case class Not(child: Filter) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringStartsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -203,7 +203,7 @@ case class StringStartsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringEndsWith(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } @@ -214,7 +214,31 @@ case class StringEndsWith(attribute: String, value: String) extends Filter { * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable case class StringContains(attribute: String, value: String) extends Filter { override def references: Array[String] = Array(attribute) } + +/** + * A filter that always evaluates to `true`. + */ +@Evolving +case class AlwaysTrue() extends Filter { + override def references: Array[String] = Array.empty +} + +@Evolving +object AlwaysTrue extends AlwaysTrue { +} + +/** + * A filter that always evaluates to `false`. + */ +@Evolving +case class AlwaysFalse() extends Filter { + override def references: Array[String] = Array.empty +} + +@Evolving +object AlwaysFalse extends AlwaysFalse { +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala index c43cc748655e8..d2ef08873187e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/AbstractDataType.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.types import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.Expression /** @@ -96,7 +96,7 @@ private[sql] object TypeCollection { /** * An `AbstractDataType` that matches any concrete data types. */ -protected[sql] object AnyDataType extends AbstractDataType { +protected[sql] object AnyDataType extends AbstractDataType with Serializable { // Note that since AnyDataType matches any concrete types, defaultConcreteType should never // be invoked. @@ -134,7 +134,7 @@ object AtomicType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class NumericType extends AtomicType { // Unfortunately we can't get this implicitly as that breaks Spark Serialization. In order for // implicitly[Numeric[JvmType]] to be valid, we have to change JvmType from a type variable to a diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala index 58c75b5dc7a35..7465569868f07 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ArrayType.scala @@ -21,7 +21,7 @@ import scala.math.Ordering import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.ArrayData /** @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.util.ArrayData * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object ArrayType extends AbstractDataType { /** * Construct a [[ArrayType]] object with the given element type. The `containsNull` is true. @@ -60,7 +60,7 @@ object ArrayType extends AbstractDataType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class ArrayType(elementType: DataType, containsNull: Boolean) extends DataType { /** No-arg constructor for kryo. */ diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala index 032d6b54aeb79..dddf874b9c6c2 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BinaryType.scala @@ -17,18 +17,16 @@ package org.apache.spark.sql.types -import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.TypeUtils - /** * The data type representing `Array[Byte]` values. * Please use the singleton `DataTypes.BinaryType`. */ -@InterfaceStability.Stable +@Stable class BinaryType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BinaryType$" in byte code. @@ -38,11 +36,8 @@ class BinaryType private() extends AtomicType { @transient private[sql] lazy val tag = typeTag[InternalType] - private[sql] val ordering = new Ordering[InternalType] { - def compare(x: Array[Byte], y: Array[Byte]): Int = { - TypeUtils.compareBinary(x, y) - } - } + private[sql] val ordering = + (x: Array[Byte], y: Array[Byte]) => TypeUtils.compareBinary(x, y) /** * The default size of a value of the BinaryType is 100 bytes. @@ -55,5 +50,5 @@ class BinaryType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object BinaryType extends BinaryType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala index 63f354d2243cf..5e3de71caa37e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/BooleanType.scala @@ -20,15 +20,14 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability - +import org.apache.spark.annotation.Stable /** * The data type representing `Boolean` values. Please use the singleton `DataTypes.BooleanType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class BooleanType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "BooleanType$" in byte code. @@ -48,5 +47,5 @@ class BooleanType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object BooleanType extends BooleanType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala index 5854c3f5ba116..9d400eefc0f8d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ByteType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Byte` values. Please use the singleton `DataTypes.ByteType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ByteType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ByteType$" in byte code. @@ -52,5 +52,5 @@ class ByteType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object ByteType extends ByteType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala index 2342036a57460..8e297874a0d62 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/CalendarIntervalType.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing calendar time intervals. The calendar time interval is stored @@ -29,7 +29,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable class CalendarIntervalType private() extends DataType { override def defaultSize: Int = 16 @@ -40,5 +40,5 @@ class CalendarIntervalType private() extends DataType { /** * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable case object CalendarIntervalType extends CalendarIntervalType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala index e53628d11ccf3..a35e971d08823 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DataType.scala @@ -26,7 +26,7 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.analysis.Resolver import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser @@ -38,7 +38,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class DataType extends AbstractDataType { /** * Enables matching against DataType for expressions: @@ -111,7 +111,7 @@ abstract class DataType extends AbstractDataType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object DataType { private val FIXED_DECIMAL = """decimal\(\s*(\d+)\s*,\s*(\-?\d+)\s*\)""".r @@ -180,7 +180,7 @@ object DataType { ("pyClass", _), ("sqlType", _), ("type", JString("udt"))) => - Utils.classForName(udtClass).newInstance().asInstanceOf[UserDefinedType[_]] + Utils.classForName[UserDefinedType[_]](udtClass).getConstructor().newInstance() // Python UDT case JSortedObject( @@ -353,10 +353,9 @@ object DataType { * compatible (read allows nulls or write does not contain nulls). * - Both types are maps and the map key and value types are compatible, and value nullability * is compatible (read allows nulls or write does not contain nulls). - * - Both types are structs and each field in the read struct is present in the write struct and - * compatible (including nullability), or is nullable if the write struct does not contain the - * field. Write-side structs are not compatible if they contain fields that are not present in - * the read-side struct. + * - Both types are structs and have the same number of fields. The type and nullability of each + * field from read/write is compatible. If byName is true, the name of each field from + * read/write needs to be the same. * - Both types are atomic and the write type can be safely cast to the read type. * * Extra fields in write-side structs are not allowed to avoid accidentally writing data that @@ -369,14 +368,15 @@ object DataType { def canWrite( write: DataType, read: DataType, + byName: Boolean, resolver: Resolver, context: String, - addError: String => Unit = (_: String) => {}): Boolean = { + addError: String => Unit): Boolean = { (write, read) match { case (wArr: ArrayType, rArr: ArrayType) => // run compatibility check first to produce all error messages - val typesCompatible = - canWrite(wArr.elementType, rArr.elementType, resolver, context + ".element", addError) + val typesCompatible = canWrite( + wArr.elementType, rArr.elementType, byName, resolver, context + ".element", addError) if (wArr.containsNull && !rArr.containsNull) { addError(s"Cannot write nullable elements to array of non-nulls: '$context'") @@ -390,31 +390,30 @@ object DataType { // read. map keys can be missing fields as long as they are nullable in the read schema. // run compatibility check first to produce all error messages - val keyCompatible = - canWrite(wMap.keyType, rMap.keyType, resolver, context + ".key", addError) - val valueCompatible = - canWrite(wMap.valueType, rMap.valueType, resolver, context + ".value", addError) - val typesCompatible = keyCompatible && valueCompatible + val keyCompatible = canWrite( + wMap.keyType, rMap.keyType, byName, resolver, context + ".key", addError) + val valueCompatible = canWrite( + wMap.valueType, rMap.valueType, byName, resolver, context + ".value", addError) if (wMap.valueContainsNull && !rMap.valueContainsNull) { addError(s"Cannot write nullable values to map of non-nulls: '$context'") false } else { - typesCompatible + keyCompatible && valueCompatible } case (StructType(writeFields), StructType(readFields)) => var fieldCompatible = true - readFields.zip(writeFields).foreach { - case (rField, wField) => - val namesMatch = resolver(wField.name, rField.name) || isSparkGeneratedName(wField.name) + readFields.zip(writeFields).zipWithIndex.foreach { + case ((rField, wField), i) => + val nameMatch = resolver(wField.name, rField.name) || isSparkGeneratedName(wField.name) val fieldContext = s"$context.${rField.name}" - val typesCompatible = - canWrite(wField.dataType, rField.dataType, resolver, fieldContext, addError) + val typesCompatible = canWrite( + wField.dataType, rField.dataType, byName, resolver, fieldContext, addError) - if (!namesMatch) { - addError(s"Struct '$context' field name does not match (may be out of order): " + - s"expected '${rField.name}', found '${wField.name}'") + if (byName && !nameMatch) { + addError(s"Struct '$context' $i-th field name does not match " + + s"(may be out of order): expected '${rField.name}', found '${wField.name}'") fieldCompatible = false } else if (!rField.nullable && wField.nullable) { addError(s"Cannot write nullable values to non-null field: '$fieldContext'") @@ -427,7 +426,7 @@ object DataType { if (readFields.size > writeFields.size) { val missingFieldsStr = readFields.takeRight(readFields.size - writeFields.size) - .map(f => s"'${f.name}'").mkString(", ") + .map(f => s"'${f.name}'").mkString(", ") if (missingFieldsStr.nonEmpty) { addError(s"Struct '$context' missing fields: $missingFieldsStr") fieldCompatible = false @@ -435,7 +434,7 @@ object DataType { } else if (writeFields.size > readFields.size) { val extraFieldsStr = writeFields.takeRight(writeFields.size - readFields.size) - .map(f => s"'${f.name}'").mkString(", ") + .map(f => s"'${f.name}'").mkString(", ") addError(s"Cannot write extra fields to struct '$context': $extraFieldsStr") fieldCompatible = false } @@ -443,7 +442,7 @@ object DataType { fieldCompatible case (w: AtomicType, r: AtomicType) => - if (!Cast.canSafeCast(w, r)) { + if (!Cast.canUpCast(w, r)) { addError(s"Cannot safely cast '$context': $w to $r") false } else { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala index 9e70dd486a125..ba322fa80b62d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DateType.scala @@ -20,22 +20,21 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** - * A date type, supporting "0001-01-01" through "9999-12-31". - * - * Please use the singleton `DataTypes.DateType`. - * - * Internally, this is represented as the number of days from 1970-01-01. + * The date type represents a valid date in the proleptic Gregorian calendar. + * Valid range is [0001-01-01, 9999-12-31]. * + * Please use the singleton `DataTypes.DateType` to refer the type. * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class DateType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "DateType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. + /** + * Internally, a date is stored as a simple incrementing count of days + * where day 0 is 1970-01-01. Negative numbers represent earlier days. + */ private[sql] type InternalType = Int @transient private[sql] lazy val tag = typeTag[InternalType] @@ -51,7 +50,12 @@ class DateType private() extends AtomicType { } /** + * The companion case object and the DateType class is separated so the companion object + * also subclasses the class. Otherwise, the companion object would be of type "DateType$" + * in byte code. The DateType class is defined with a private constructor so its companion + * object is the only possible instantiation. + * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object DateType extends DateType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala index 9eed2eb202045..1bf322af21799 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Decimal.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import java.lang.{Long => JLong} import java.math.{BigInteger, MathContext, RoundingMode} -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Unstable import org.apache.spark.sql.AnalysisException /** @@ -31,7 +31,7 @@ import org.apache.spark.sql.AnalysisException * - If decimalVal is set, it represents the whole decimal value * - Otherwise, the decimal value is longVal / (10 ** _scale) */ -@InterfaceStability.Unstable +@Unstable final class Decimal extends Ordered[Decimal] with Serializable { import org.apache.spark.sql.types.Decimal._ @@ -76,7 +76,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { */ def set(unscaled: Long, precision: Int, scale: Int): Decimal = { if (setOrNull(unscaled, precision, scale) == null) { - throw new IllegalArgumentException("Unscaled value too large for precision") + throw new ArithmeticException("Unscaled value too large for precision") } this } @@ -111,9 +111,10 @@ final class Decimal extends Ordered[Decimal] with Serializable { */ def set(decimal: BigDecimal, precision: Int, scale: Int): Decimal = { this.decimalVal = decimal.setScale(scale, ROUND_HALF_UP) - require( - decimalVal.precision <= precision, - s"Decimal precision ${decimalVal.precision} exceeds max precision $precision") + if (decimalVal.precision > precision) { + throw new ArithmeticException( + s"Decimal precision ${decimalVal.precision} exceeds max precision $precision") + } this.longVal = 0L this._precision = precision this._scale = scale @@ -185,9 +186,21 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } - def toScalaBigInt: BigInt = BigInt(toLong) + def toScalaBigInt: BigInt = { + if (decimalVal.ne(null)) { + decimalVal.toBigInt() + } else { + BigInt(toLong) + } + } - def toJavaBigInteger: java.math.BigInteger = java.math.BigInteger.valueOf(toLong) + def toJavaBigInteger: java.math.BigInteger = { + if (decimalVal.ne(null)) { + decimalVal.underlying().toBigInteger() + } else { + java.math.BigInteger.valueOf(toLong) + } + } def toUnscaledLong: Long = { if (decimalVal.ne(null)) { @@ -237,14 +250,25 @@ final class Decimal extends Ordered[Decimal] with Serializable { /** * Create new `Decimal` with given precision and scale. * - * @return a non-null `Decimal` value if successful or `null` if overflow would occur. + * @return a non-null `Decimal` value if successful. Otherwise, if `nullOnOverflow` is true, null + * is returned; if `nullOnOverflow` is false, an `ArithmeticException` is thrown. */ private[sql] def toPrecision( precision: Int, scale: Int, - roundMode: BigDecimal.RoundingMode.Value = ROUND_HALF_UP): Decimal = { + roundMode: BigDecimal.RoundingMode.Value = ROUND_HALF_UP, + nullOnOverflow: Boolean = true): Decimal = { val copy = clone() - if (copy.changePrecision(precision, scale, roundMode)) copy else null + if (copy.changePrecision(precision, scale, roundMode)) { + copy + } else { + if (nullOnOverflow) { + null + } else { + throw new ArithmeticException( + s"$toDebugString cannot be represented as Decimal($precision, $scale).") + } + } } /** @@ -407,7 +431,7 @@ final class Decimal extends Ordered[Decimal] with Serializable { } } -@InterfaceStability.Unstable +@Unstable object Decimal { val ROUND_HALF_UP = BigDecimal.RoundingMode.HALF_UP val ROUND_HALF_EVEN = BigDecimal.RoundingMode.HALF_EVEN diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala index 15004e4b9667d..25eddaf06a780 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DecimalType.scala @@ -21,11 +21,10 @@ import java.util.Locale import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} - /** * The data type representing `java.math.BigDecimal` values. * A Decimal that must have fixed precision (the maximum number of digits) and scale (the number @@ -39,7 +38,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, Literal} * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class DecimalType(precision: Int, scale: Int) extends FractionalType { if (scale > precision) { @@ -110,7 +109,7 @@ case class DecimalType(precision: Int, scale: Int) extends FractionalType { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object DecimalType extends AbstractDataType { import scala.math.min diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala index a5c79ff01ca06..7a48202ff7fdb 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/DoubleType.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.types -import scala.math.{Fractional, Numeric, Ordering} +import scala.math.{Fractional, Numeric} import scala.math.Numeric.DoubleAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.util.Utils /** @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class DoubleType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "DoubleType$" in byte code. @@ -38,9 +38,8 @@ class DoubleType private() extends FractionalType { @transient private[sql] lazy val tag = typeTag[InternalType] private[sql] val numeric = implicitly[Numeric[Double]] private[sql] val fractional = implicitly[Fractional[Double]] - private[sql] val ordering = new Ordering[Double] { - override def compare(x: Double, y: Double): Int = Utils.nanSafeCompareDoubles(x, y) - } + private[sql] val ordering = + (x: Double, y: Double) => Utils.nanSafeCompareDoubles(x, y) private[sql] val asIntegral = DoubleAsIfIntegral /** @@ -54,5 +53,5 @@ class DoubleType private() extends FractionalType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object DoubleType extends DoubleType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala index 352147ec936c9..652edb9b0f7e1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/FloatType.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.types -import scala.math.{Fractional, Numeric, Ordering} +import scala.math.{Fractional, Numeric} import scala.math.Numeric.FloatAsIfIntegral import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.util.Utils /** @@ -29,7 +29,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class FloatType private() extends FractionalType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "FloatType$" in byte code. @@ -38,9 +38,8 @@ class FloatType private() extends FractionalType { @transient private[sql] lazy val tag = typeTag[InternalType] private[sql] val numeric = implicitly[Numeric[Float]] private[sql] val fractional = implicitly[Fractional[Float]] - private[sql] val ordering = new Ordering[Float] { - override def compare(x: Float, y: Float): Int = Utils.nanSafeCompareFloats(x, y) - } + private[sql] val ordering = + (x: Float, y: Float) => Utils.nanSafeCompareFloats(x, y) private[sql] val asIntegral = FloatAsIfIntegral /** @@ -55,5 +54,5 @@ class FloatType private() extends FractionalType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object FloatType extends FloatType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala index a85e3729188d9..0755202d20df1 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/IntegerType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Int` values. Please use the singleton `DataTypes.IntegerType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class IntegerType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "IntegerType$" in byte code. @@ -51,5 +51,5 @@ class IntegerType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object IntegerType extends IntegerType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala index 0997028fc1057..3c49c721fdc88 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/LongType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Long` values. Please use the singleton `DataTypes.LongType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class LongType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "LongType$" in byte code. @@ -51,5 +51,5 @@ class LongType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object LongType extends LongType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala index 594e155268bf6..29b9ffc0c3549 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/MapType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type for Maps. Keys in a map are not allowed to have `null` values. @@ -31,7 +31,7 @@ import org.apache.spark.annotation.InterfaceStability * @param valueType The data type of map values. * @param valueContainsNull Indicates if map values have `null` values. */ -@InterfaceStability.Stable +@Stable case class MapType( keyType: DataType, valueType: DataType, @@ -78,7 +78,7 @@ case class MapType( /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object MapType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = apply(NullType, NullType) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala index 7c15dc0de4b6b..982f6244f8a0d 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/Metadata.scala @@ -22,7 +22,7 @@ import scala.collection.mutable import org.json4s._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** @@ -37,7 +37,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable sealed class Metadata private[types] (private[types] val map: Map[String, Any]) extends Serializable { @@ -88,7 +88,7 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) map.keysIterator.forall { key => that.map.get(key) match { case Some(otherValue) => - val ourValue = map.get(key).get + val ourValue = map(key) (ourValue, otherValue) match { case (v0: Array[Long], v1: Array[Long]) => java.util.Arrays.equals(v0, v1) case (v0: Array[Double], v1: Array[Double]) => java.util.Arrays.equals(v0, v1) @@ -117,7 +117,7 @@ sealed class Metadata private[types] (private[types] val map: Map[String, Any]) /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object Metadata { private[this] val _empty = new Metadata(Map.empty) @@ -190,6 +190,8 @@ object Metadata { JBool(x) case x: String => JString(x) + case null => + JNull case x: Metadata => toJsonValue(x.map) case other => @@ -228,7 +230,7 @@ object Metadata { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class MetadataBuilder { private val map: mutable.Map[String, Any] = mutable.Map.empty diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala index 494225b47a270..14097a5280d50 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/NullType.scala @@ -17,15 +17,14 @@ package org.apache.spark.sql.types -import org.apache.spark.annotation.InterfaceStability - +import org.apache.spark.annotation.Stable /** * The data type representing `NULL` values. Please use the singleton `DataTypes.NullType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class NullType private() extends DataType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "NullType$" in byte code. @@ -38,5 +37,5 @@ class NullType private() extends DataType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object NullType extends NullType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala index 203e85e1c99bd..e79c0a4b62c4b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ObjectType.scala @@ -17,11 +17,9 @@ package org.apache.spark.sql.types -import scala.language.existentials +import org.apache.spark.annotation.Evolving -import org.apache.spark.annotation.InterfaceStability - -@InterfaceStability.Evolving +@Evolving object ObjectType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = throw new UnsupportedOperationException( @@ -38,7 +36,7 @@ object ObjectType extends AbstractDataType { /** * Represents a JVM object that is passing through Spark SQL expression evaluation. */ -@InterfaceStability.Evolving +@Evolving case class ObjectType(cls: Class[_]) extends DataType { override def defaultSize: Int = 4096 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala index ee655c338b59f..9b5ddfef1ccf5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/ShortType.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.types import scala.math.{Integral, Numeric, Ordering} import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * The data type representing `Short` values. Please use the singleton `DataTypes.ShortType`. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ShortType private() extends IntegralType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "ShortType$" in byte code. @@ -51,5 +51,5 @@ class ShortType private() extends IntegralType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object ShortType extends ShortType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala index 59b124cda7d14..8ce1cd078e312 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StringType.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.unsafe.types.UTF8String /** @@ -28,7 +28,7 @@ import org.apache.spark.unsafe.types.UTF8String * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class StringType private() extends AtomicType { // The companion object and this class is separated so the companion object also subclasses // this type. Otherwise, the companion object would be of type "StringType$" in byte code. @@ -48,6 +48,6 @@ class StringType private() extends AtomicType { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object StringType extends StringType diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala index 35f9970a0aaec..6f6b561d67d49 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructField.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.types import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} /** @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdenti * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class StructField( name: String, dataType: DataType, diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala index 06289b1483203..236f73ba3832c 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/StructType.scala @@ -17,18 +17,18 @@ package org.apache.spark.sql.types -import scala.collection.mutable.ArrayBuffer +import scala.collection.{mutable, Map} import scala.util.Try import scala.util.control.NonFatal import org.json4s.JsonDSL._ import org.apache.spark.SparkException -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, InterpretedOrdering} import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, LegacyTypeStringParser} -import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} -import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.util.{quoteIdentifier, truncatedString} +import org.apache.spark.sql.internal.SQLConf /** * A [[StructType]] object can be constructed by @@ -57,7 +57,7 @@ import org.apache.spark.util.Utils * * // If this struct does not have a field called "d", it throws an exception. * struct("d") - * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // java.lang.IllegalArgumentException: d does not exist. * // ... * * // Extract multiple StructFields. Field names are provided in a set. @@ -69,7 +69,7 @@ import org.apache.spark.util.Utils * // Any names without matching fields will throw an exception. * // For the case shown below, an exception is thrown due to "d". * struct(Set("b", "c", "d")) - * // java.lang.IllegalArgumentException: Field "d" does not exist. + * // java.lang.IllegalArgumentException: d does not exist. * // ... * }}} * @@ -95,7 +95,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case class StructType(fields: Array[StructField]) extends DataType with Seq[StructField] { /** No-arg constructor for kryo. */ @@ -272,22 +272,21 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru def apply(name: String): StructField = { nameToField.getOrElse(name, throw new IllegalArgumentException( - s"""Field "$name" does not exist. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin)) + s"$name does not exist. Available: ${fieldNames.mkString(", ")}")) } /** * Returns a [[StructType]] containing [[StructField]]s of the given names, preserving the * original order of fields. * - * @throws IllegalArgumentException if a field cannot be found for any of the given names + * @throws IllegalArgumentException if at least one given field name does not exist */ def apply(names: Set[String]): StructType = { val nonExistFields = names -- fieldNamesSet if (nonExistFields.nonEmpty) { throw new IllegalArgumentException( - s"""Nonexistent field(s): ${nonExistFields.mkString(", ")}. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin) + s"${nonExistFields.mkString(", ")} do(es) not exist. " + + s"Available: ${fieldNames.mkString(", ")}") } // Preserve the original order of fields. StructType(fields.filter(f => names.contains(f.name))) @@ -301,24 +300,78 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru def fieldIndex(name: String): Int = { nameToIndex.getOrElse(name, throw new IllegalArgumentException( - s"""Field "$name" does not exist. - |Available fields: ${fieldNames.mkString(", ")}""".stripMargin)) + s"$name does not exist. Available: ${fieldNames.mkString(", ")}")) } private[sql] def getFieldIndex(name: String): Option[Int] = { nameToIndex.get(name) } + /** + * Returns a field in this struct and its child structs. + * + * If includeCollections is true, this will return fields that are nested in maps and arrays. + */ + private[sql] def findNestedField( + fieldNames: Seq[String], + includeCollections: Boolean = false): Option[StructField] = { + fieldNames.headOption.flatMap(nameToField.get) match { + case Some(field) => + (fieldNames.tail, field.dataType, includeCollections) match { + case (Seq(), _, _) => + Some(field) + + case (names, struct: StructType, _) => + struct.findNestedField(names, includeCollections) + + case (_, _, false) => + None // types nested in maps and arrays are not used + + case (Seq("key"), MapType(keyType, _, _), true) => + // return the key type as a struct field to include nullability + Some(StructField("key", keyType, nullable = false)) + + case (Seq("key", names @ _*), MapType(struct: StructType, _, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("value"), MapType(_, valueType, isNullable), true) => + // return the value type as a struct field to include nullability + Some(StructField("value", valueType, nullable = isNullable)) + + case (Seq("value", names @ _*), MapType(_, struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case (Seq("element"), ArrayType(elementType, isNullable), true) => + // return the element type as a struct field to include nullability + Some(StructField("element", elementType, nullable = isNullable)) + + case (Seq("element", names @ _*), ArrayType(struct: StructType, _), true) => + struct.findNestedField(names, includeCollections) + + case _ => + None + } + case _ => + None + } + } + protected[sql] def toAttributes: Seq[AttributeReference] = map(f => AttributeReference(f.name, f.dataType, f.nullable, f.metadata)()) - def treeString: String = { + def treeString: String = treeString(Int.MaxValue) + + def treeString(level: Int): String = { val builder = new StringBuilder builder.append("root\n") val prefix = " |" fields.foreach(field => field.buildFormattedString(prefix, builder)) - builder.toString() + if (level <= 0 || level == Int.MaxValue) { + builder.toString() + } else { + builder.toString().split("\n").filter(_.lastIndexOf("|--") < level * 5 + 1).mkString("\n") + } } // scalastyle:off println @@ -346,7 +399,10 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru override def simpleString: String = { val fieldTypes = fields.view.map(field => s"${field.name}:${field.dataType.simpleString}") - Utils.truncatedString(fieldTypes, "struct<", ",", ">") + truncatedString( + fieldTypes, + "struct<", ",", ">", + SQLConf.get.maxToStringFields) } override def catalogString: String = { @@ -422,7 +478,7 @@ case class StructType(fields: Array[StructField]) extends DataType with Seq[Stru /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable object StructType extends AbstractDataType { override private[sql] def defaultConcreteType: DataType = new StructType @@ -486,7 +542,7 @@ object StructType extends AbstractDataType { leftContainsNull || rightContainsNull) case (StructType(leftFields), StructType(rightFields)) => - val newFields = ArrayBuffer.empty[StructField] + val newFields = mutable.ArrayBuffer.empty[StructField] val rightMapped = fieldsMap(rightFields) leftFields.foreach { @@ -545,7 +601,10 @@ object StructType extends AbstractDataType { } private[sql] def fieldsMap(fields: Array[StructField]): Map[String, StructField] = { - import scala.collection.breakOut - fields.map(s => (s.name, s))(breakOut) + // Mimics the optimization of breakOut, not present in Scala 2.13, while working in 2.12 + val map = mutable.Map[String, StructField]() + map.sizeHint(fields.length) + fields.foreach(s => map.put(s.name, s)) + map } } diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala index fdb91e0499920..8dbe4dd15178b 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/types/TimestampType.scala @@ -20,19 +20,23 @@ package org.apache.spark.sql.types import scala.math.Ordering import scala.reflect.runtime.universe.typeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** - * The data type representing `java.sql.Timestamp` values. - * Please use the singleton `DataTypes.TimestampType`. + * The timestamp type represents a time instant in microsecond precision. + * Valid range is [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] where + * the left/right-bound is a date and time of the proleptic Gregorian + * calendar in UTC+00:00. * + * Please use the singleton `DataTypes.TimestampType` to refer the type. * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class TimestampType private() extends AtomicType { - // The companion object and this class is separated so the companion object also subclasses - // this type. Otherwise, the companion object would be of type "TimestampType$" in byte code. - // Defined with a private constructor so the companion object is the only possible instantiation. + /** + * Internally, a timestamp is stored as the number of microseconds from + * the epoch of 1970-01-01T00:00:00.000000Z (UTC+00:00) + */ private[sql] type InternalType = Long @transient private[sql] lazy val tag = typeTag[InternalType] @@ -48,7 +52,12 @@ class TimestampType private() extends AtomicType { } /** + * The companion case object and its class is separated so the companion object also subclasses + * the TimestampType class. Otherwise, the companion object would be of type "TimestampType$" + * in byte code. Defined with a private constructor so the companion object is the only possible + * instantiation. + * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable case object TimestampType extends TimestampType diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala similarity index 96% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala rename to sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala index b1e8fb39ac9de..62546a322d3c9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowUtils.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/util/ArrowUtils.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.arrow +package org.apache.spark.sql.util import scala.collection.JavaConverters._ @@ -133,6 +133,8 @@ object ArrowUtils { } val pandasColsByName = Seq(SQLConf.PANDAS_GROUPED_MAP_ASSIGN_COLUMNS_BY_NAME.key -> conf.pandasGroupedMapAssignColumnsByName.toString) - Map(timeZoneConf ++ pandasColsByName: _*) + val arrowSafeTypeCheck = Seq(SQLConf.PANDAS_ARROW_SAFE_TYPE_CONVERSION.key -> + conf.arrowSafeTypeConversion.toString) + Map(timeZoneConf ++ pandasColsByName ++ arrowSafeTypeCheck: _*) } } diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java new file mode 100644 index 0000000000000..6bd9192816992 --- /dev/null +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalog/v2/CatalogLoadingSuite.java @@ -0,0 +1,211 @@ +/* + * 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.catalog.v2; + +import org.apache.spark.SparkException; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; +import org.junit.Assert; +import org.junit.Test; + +import java.util.concurrent.Callable; + +public class CatalogLoadingSuite { + @Test + public void testLoad() throws SparkException { + SQLConf conf = new SQLConf(); + conf.setConfString("spark.sql.catalog.test-name", TestCatalogPlugin.class.getCanonicalName()); + + CatalogPlugin plugin = Catalogs.load("test-name", conf); + Assert.assertNotNull("Should instantiate a non-null plugin", plugin); + Assert.assertEquals("Plugin should have correct implementation", + TestCatalogPlugin.class, plugin.getClass()); + + TestCatalogPlugin testPlugin = (TestCatalogPlugin) plugin; + Assert.assertEquals("Options should contain no keys", 0, testPlugin.options.size()); + Assert.assertEquals("Catalog should have correct name", "test-name", testPlugin.name()); + } + + @Test + public void testInitializationOptions() throws SparkException { + SQLConf conf = new SQLConf(); + conf.setConfString("spark.sql.catalog.test-name", TestCatalogPlugin.class.getCanonicalName()); + conf.setConfString("spark.sql.catalog.test-name.name", "not-catalog-name"); + conf.setConfString("spark.sql.catalog.test-name.kEy", "valUE"); + + CatalogPlugin plugin = Catalogs.load("test-name", conf); + Assert.assertNotNull("Should instantiate a non-null plugin", plugin); + Assert.assertEquals("Plugin should have correct implementation", + TestCatalogPlugin.class, plugin.getClass()); + + TestCatalogPlugin testPlugin = (TestCatalogPlugin) plugin; + + Assert.assertEquals("Options should contain only two keys", 2, testPlugin.options.size()); + Assert.assertEquals("Options should contain correct value for name (not overwritten)", + "not-catalog-name", testPlugin.options.get("name")); + Assert.assertEquals("Options should contain correct value for key", + "valUE", testPlugin.options.get("key")); + } + + @Test + public void testLoadWithoutConfig() { + SQLConf conf = new SQLConf(); + + SparkException exc = intercept(CatalogNotFoundException.class, + () -> Catalogs.load("missing", conf)); + + Assert.assertTrue("Should complain that implementation is not configured", + exc.getMessage() + .contains("plugin class not found: spark.sql.catalog.missing is not defined")); + Assert.assertTrue("Should identify the catalog by name", + exc.getMessage().contains("missing")); + } + + @Test + public void testLoadMissingClass() { + SQLConf conf = new SQLConf(); + conf.setConfString("spark.sql.catalog.missing", "com.example.NoSuchCatalogPlugin"); + + SparkException exc = intercept(SparkException.class, () -> Catalogs.load("missing", conf)); + + Assert.assertTrue("Should complain that the class is not found", + exc.getMessage().contains("Cannot find catalog plugin class")); + Assert.assertTrue("Should identify the catalog by name", + exc.getMessage().contains("missing")); + Assert.assertTrue("Should identify the missing class", + exc.getMessage().contains("com.example.NoSuchCatalogPlugin")); + } + + @Test + public void testLoadNonCatalogPlugin() { + SQLConf conf = new SQLConf(); + String invalidClassName = InvalidCatalogPlugin.class.getCanonicalName(); + conf.setConfString("spark.sql.catalog.invalid", invalidClassName); + + SparkException exc = intercept(SparkException.class, () -> Catalogs.load("invalid", conf)); + + Assert.assertTrue("Should complain that class does not implement CatalogPlugin", + exc.getMessage().contains("does not implement CatalogPlugin")); + Assert.assertTrue("Should identify the catalog by name", + exc.getMessage().contains("invalid")); + Assert.assertTrue("Should identify the class", + exc.getMessage().contains(invalidClassName)); + } + + @Test + public void testLoadConstructorFailureCatalogPlugin() { + SQLConf conf = new SQLConf(); + String invalidClassName = ConstructorFailureCatalogPlugin.class.getCanonicalName(); + conf.setConfString("spark.sql.catalog.invalid", invalidClassName); + + SparkException exc = intercept(SparkException.class, () -> Catalogs.load("invalid", conf)); + + Assert.assertTrue("Should identify the constructor error", + exc.getMessage().contains("Failed during instantiating constructor for catalog")); + Assert.assertTrue("Should have expected error message", + exc.getCause().getMessage().contains("Expected failure")); + } + + @Test + public void testLoadAccessErrorCatalogPlugin() { + SQLConf conf = new SQLConf(); + String invalidClassName = AccessErrorCatalogPlugin.class.getCanonicalName(); + conf.setConfString("spark.sql.catalog.invalid", invalidClassName); + + SparkException exc = intercept(SparkException.class, () -> Catalogs.load("invalid", conf)); + + Assert.assertTrue("Should complain that no public constructor is provided", + exc.getMessage().contains("Failed to call public no-arg constructor for catalog")); + Assert.assertTrue("Should identify the catalog by name", + exc.getMessage().contains("invalid")); + Assert.assertTrue("Should identify the class", + exc.getMessage().contains(invalidClassName)); + } + + @SuppressWarnings("unchecked") + public static E intercept(Class expected, Callable callable) { + try { + callable.call(); + Assert.fail("No exception was thrown, expected: " + + expected.getName()); + } catch (Exception actual) { + try { + Assert.assertEquals(expected, actual.getClass()); + return (E) actual; + } catch (AssertionError e) { + e.addSuppressed(actual); + throw e; + } + } + // Compiler doesn't catch that Assert.fail will always throw an exception. + throw new UnsupportedOperationException("[BUG] Should not reach this statement"); + } +} + +class TestCatalogPlugin implements CatalogPlugin { + String name = null; + CaseInsensitiveStringMap options = null; + + TestCatalogPlugin() { + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + this.name = name; + this.options = options; + } + + @Override + public String name() { + return name; + } +} + +class ConstructorFailureCatalogPlugin implements CatalogPlugin { // fails in its constructor + ConstructorFailureCatalogPlugin() { + throw new RuntimeException("Expected failure."); + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + } + + @Override + public String name() { + return null; + } +} + +class AccessErrorCatalogPlugin implements CatalogPlugin { // no public constructor + private AccessErrorCatalogPlugin() { + } + + @Override + public void initialize(String name, CaseInsensitiveStringMap options) { + } + + @Override + public String name() { + return null; + } +} + +class InvalidCatalogPlugin { // doesn't implement CatalogPlugin + public void initialize(CaseInsensitiveStringMap options) { + } +} diff --git a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java index 8da778800bb9f..f3d82b485e54e 100644 --- a/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java +++ b/sql/catalyst/src/test/java/org/apache/spark/sql/catalyst/expressions/RowBasedKeyValueBatchSuite.java @@ -29,6 +29,7 @@ import org.apache.spark.sql.types.DataTypes; import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter; import org.apache.spark.unsafe.types.UTF8String; +import org.apache.spark.internal.config.package$; import java.util.Random; @@ -104,9 +105,9 @@ private boolean checkValue(UnsafeRow row, long v1, long v2) { @Before public void setup() { memoryManager = new TestMemoryManager(new SparkConf() - .set("spark.memory.offHeap.enabled", "false") - .set("spark.shuffle.spill.compress", "false") - .set("spark.shuffle.compress", "false")); + .set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false) + .set(package$.MODULE$.SHUFFLE_SPILL_COMPRESS(), false) + .set(package$.MODULE$.SHUFFLE_COMPRESS(), false)); taskMemoryManager = new TaskMemoryManager(memoryManager, 0); } diff --git a/sql/catalyst/src/test/resources/log4j.properties b/sql/catalyst/src/test/resources/log4j.properties index 3706a6e361307..a9166df3a7fb4 100644 --- a/sql/catalyst/src/test/resources/log4j.properties +++ b/sql/catalyst/src/test/resources/log4j.properties @@ -24,4 +24,4 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala index 4226ab3773fe7..3b4b80daf0843 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashBenchmark.scala @@ -102,7 +102,7 @@ object HashBenchmark extends BenchmarkBase { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val singleInt = new StructType().add("i", IntegerType) test("single ints", singleInt, 1 << 15, 1 << 14) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala index 7dc865d85af04..dbfa7bb18aa65 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/HashByteArrayBenchmark.scala @@ -83,7 +83,7 @@ object HashByteArrayBenchmark extends BenchmarkBase { benchmark.run() } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Benchmark for MurMurHash 3 and xxHash64") { test(8, 42L, 1 << 10, 1 << 11) test(16, 42L, 1 << 10, 1 << 11) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala index 8ae3ff5043e68..d361e6248e2f5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGenerator.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql -import java.lang.Double.longBitsToDouble -import java.lang.Float.intBitsToFloat import java.math.MathContext import scala.collection.mutable @@ -69,6 +67,28 @@ object RandomDataGenerator { Some(f) } + /** + * A wrapper of Float.intBitsToFloat to use a unique NaN value for all NaN values. + * This prevents `checkEvaluationWithUnsafeProjection` from failing due to + * the difference between `UnsafeRow` binary presentation for NaN. + * This is visible for testing. + */ + def intBitsToFloat(bits: Int): Float = { + val value = java.lang.Float.intBitsToFloat(bits) + if (value.isNaN) Float.NaN else value + } + + /** + * A wrapper of Double.longBitsToDouble to use a unique NaN value for all NaN values. + * This prevents `checkEvaluationWithUnsafeProjection` from failing due to + * the difference between `UnsafeRow` binary presentation for NaN. + * This is visible for testing. + */ + def longBitsToDouble(bits: Long): Double = { + val value = java.lang.Double.longBitsToDouble(bits) + if (value.isNaN) Double.NaN else value + } + /** * Returns a randomly generated schema, based on the given accepted types. * diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala index 3c2f8a28875f7..3e62ca069e9ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/RandomDataGeneratorSuite.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql +import java.nio.ByteBuffer +import java.util.Arrays + import scala.util.Random import org.apache.spark.SparkFunSuite @@ -106,4 +109,32 @@ class RandomDataGeneratorSuite extends SparkFunSuite { assert(deviation.toDouble / expectedTotalElements < 2e-1) } } + + test("Use Float.NaN for all NaN values") { + val bits = -6966608 + val nan1 = java.lang.Float.intBitsToFloat(bits) + val nan2 = RandomDataGenerator.intBitsToFloat(bits) + assert(nan1.isNaN) + assert(nan2.isNaN) + + val arrayExpected = ByteBuffer.allocate(4).putFloat(Float.NaN).array + val array1 = ByteBuffer.allocate(4).putFloat(nan1).array + val array2 = ByteBuffer.allocate(4).putFloat(nan2).array + assert(!Arrays.equals(array1, arrayExpected)) + assert(Arrays.equals(array2, arrayExpected)) + } + + test("Use Double.NaN for all NaN values") { + val bits = -6966608 + val nan1 = java.lang.Double.longBitsToDouble(bits) + val nan2 = RandomDataGenerator.longBitsToDouble(bits) + assert(nan1.isNaN) + assert(nan2.isNaN) + + val arrayExpected = ByteBuffer.allocate(8).putDouble(Double.NaN).array + val array1 = ByteBuffer.allocate(8).putDouble(nan1).array + val array2 = ByteBuffer.allocate(8).putDouble(nan2).array + assert(!Arrays.equals(array1, arrayExpected)) + assert(Arrays.equals(array2, arrayExpected)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala index e7a99485cdf04..42a4cfc91f826 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/UnsafeProjectionBenchmark.scala @@ -41,7 +41,7 @@ object UnsafeProjectionBenchmark extends BenchmarkBase { (1 to numRows).map(_ => encoder.toRow(generator().asInstanceOf[Row]).copy()).toArray } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("unsafe projection") { val iters = 1024 * 16 val numRows = 1024 * 16 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala new file mode 100644 index 0000000000000..9c1b9a3e53de2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TableCatalogSuite.scala @@ -0,0 +1,657 @@ +/* + * 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.catalog.v2 + +import java.util +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class TableCatalogSuite extends SparkFunSuite { + import CatalogV2Implicits._ + + private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] + private val schema: StructType = new StructType() + .add("id", IntegerType) + .add("data", StringType) + + private def newCatalog(): TableCatalog = { + val newCatalog = new TestTableCatalog + newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) + newCatalog + } + + private val testIdent = Identifier.of(Array("`", "."), "test_table") + + test("Catalogs can load the catalog") { + val catalog = newCatalog() + + val conf = new SQLConf + conf.setConfString("spark.sql.catalog.test", catalog.getClass.getName) + + val loaded = Catalogs.load("test", conf) + assert(loaded.getClass == catalog.getClass) + } + + test("listTables") { + val catalog = newCatalog() + val ident1 = Identifier.of(Array("ns"), "test_table_1") + val ident2 = Identifier.of(Array("ns"), "test_table_2") + val ident3 = Identifier.of(Array("ns2"), "test_table_1") + + assert(catalog.listTables(Array("ns")).isEmpty) + + catalog.createTable(ident1, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) + assert(catalog.listTables(Array("ns2")).isEmpty) + + catalog.createTable(ident3, schema, Array.empty, emptyProps) + catalog.createTable(ident2, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident1) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident2)) + + catalog.dropTable(ident2) + + assert(catalog.listTables(Array("ns")).isEmpty) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + } + + test("createTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("`", ".", "test_table")) + assert(table.schema == schema) + assert(table.properties.asScala == Map()) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: with properties") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("property", "value") + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("`", ".", "test_table")) + assert(table.schema == schema) + assert(table.properties == properties) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: table already exists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val exc = intercept[TableAlreadyExistsException] { + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + } + + assert(exc.message.contains(table.name())) + assert(exc.message.contains("already exists")) + + assert(catalog.tableExists(testIdent)) + } + + test("tableExists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + catalog.dropTable(testIdent) + + assert(!catalog.tableExists(testIdent)) + } + + test("loadTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("loadTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.loadTable(testIdent) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("invalidateTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.invalidateTable(testIdent) + + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("invalidateTable: table does not exist") { + val catalog = newCatalog() + + assert(catalog.tableExists(testIdent) === false) + + catalog.invalidateTable(testIdent) + } + + test("alterTable: add property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-1", "1")) + assert(updated.properties.asScala == Map("prop-1" -> "1")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1")) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add property to existing") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-2", "2")) + assert(updated.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove existing property") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove missing property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType)) + + assert(updated.schema == schema.add("ts", TimestampType)) + } + + test("alterTable: add required column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false)) + + assert(updated.schema == schema.add("ts", TimestampType, nullable = false)) + } + + test("alterTable: add column with comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false, "comment text")) + + val field = StructField("ts", TimestampType, nullable = false).withComment("comment text") + assert(updated.schema == schema.add(field)) + } + + test("alterTable: add nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("point", "z"), DoubleType)) + + val expectedSchema = schema.add("point", pointStruct.add("z", DoubleType)) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add column to primitive field fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) + } + + assert(exc.getMessage.contains("Not a struct")) + assert(exc.getMessage.contains("data")) + + // the table has not changed + assert(catalog.loadTable(testIdent).schema == schema) + } + + test("alterTable: add field to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.addColumn(Array("missing_col", "new_field"), StringType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: update column data type") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update column data type and nullability") { + val catalog = newCatalog() + + val originalSchema = new StructType() + .add("id", IntegerType, nullable = false) + .add("data", StringType) + val table = catalog.createTable(testIdent, originalSchema, Array.empty, emptyProps) + + assert(table.schema == originalSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("id"), LongType, true)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update optional column to required fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType, false)) + } + + assert(exc.getMessage.contains("Cannot change optional column to required")) + assert(exc.getMessage.contains("id")) + } + + test("alterTable: update missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("missing_col"), LongType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: add comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "comment text") + .add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: replace comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "replacement comment") + .add("data", StringType) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "replacement comment")) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add comment to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("missing_col"), "comment")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: rename top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("id"), "some_id")) + + val expectedSchema = new StructType().add("some_id", IntegerType).add("data", StringType) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first")) + + val newPointStruct = new StructType().add("first", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename struct column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point"), "p")) + + val newPointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("p", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.renameColumn(Array("missing_col"), "new_name")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: multiple changes") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first"), + TableChange.renameColumn(Array("point", "y"), "second")) + + val newPointStruct = new StructType().add("first", DoubleType).add("second", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("id"))) + + val expectedSchema = new StructType().add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("point", "y"))) + + val newPointStruct = new StructType().add("x", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"))) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: delete missing nested column fails") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"))) + } + + assert(exc.getMessage.contains("z")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("dropTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(wasDropped) + assert(!catalog.tableExists(testIdent)) + } + + test("dropTable: table does not exist") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(!wasDropped) + assert(!catalog.tableExists(testIdent)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.scala new file mode 100644 index 0000000000000..6ba140fa5ddec --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/TestTableCatalog.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.catalog.v2 + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class TestTableCatalog extends TableCatalog { + import CatalogV2Implicits._ + + private val tables: util.Map[Identifier, Table] = new ConcurrentHashMap[Identifier, Table]() + private var _name: Option[String] = None + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + _name = Some(name) + } + + override def name: String = _name.get + + override def listTables(namespace: Array[String]): Array[Identifier] = { + tables.keySet.asScala.filter(_.namespace.sameElements(namespace)).toArray + } + + override def loadTable(ident: Identifier): Table = { + Option(tables.get(ident)) match { + case Some(table) => + table + case _ => + throw new NoSuchTableException(ident) + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + if (tables.containsKey(ident)) { + throw new TableAlreadyExistsException(ident) + } + + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + val table = InMemoryTable(ident.quoted, schema, properties) + + tables.put(ident, table) + + table + } + + override def alterTable(ident: Identifier, changes: TableChange*): Table = { + val table = loadTable(ident) + val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) + val newTable = InMemoryTable(table.name, schema, properties) + + tables.put(ident, newTable) + + newTable + } + + override def dropTable(ident: Identifier): Boolean = Option(tables.remove(ident)).isDefined +} + +case class InMemoryTable( + name: String, + schema: StructType, + override val properties: util.Map[String, String]) extends Table { + override def partitioning: Array[Transform] = Array.empty + override def capabilities: util.Set[TableCapability] = InMemoryTable.CAPABILITIES +} + +object InMemoryTable { + val CAPABILITIES: util.Set[TableCapability] = Set.empty[TableCapability].asJava +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala new file mode 100644 index 0000000000000..c0a5dada19dba --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalog/v2/expressions/TransformExtractorSuite.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalog.v2.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst +import org.apache.spark.sql.types.DataType + +class TransformExtractorSuite extends SparkFunSuite { + /** + * Creates a Literal using an anonymous class. + */ + private def lit[T](literal: T): Literal[T] = new Literal[T] { + override def value: T = literal + override def dataType: DataType = catalyst.expressions.Literal(literal).dataType + override def describe: String = literal.toString + } + + /** + * Creates a NamedReference using an anonymous class. + */ + private def ref(names: String*): NamedReference = new NamedReference { + override def fieldNames: Array[String] = names.toArray + override def describe: String = names.mkString(".") + } + + /** + * Creates a Transform using an anonymous class. + */ + private def transform(func: String, ref: NamedReference): Transform = new Transform { + override def name: String = func + override def references: Array[NamedReference] = Array(ref) + override def arguments: Array[Expression] = Array(ref) + override def describe: String = ref.describe + } + + test("Identity extractor") { + transform("identity", ref("a", "b")) match { + case IdentityTransform(FieldReference(seq)) => + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match IdentityTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case IdentityTransform(FieldReference(_)) => + fail("Matched unknown transform") + case _ => + // expected + } + } + + test("Years extractor") { + transform("years", ref("a", "b")) match { + case YearsTransform(FieldReference(seq)) => + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match YearsTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case YearsTransform(FieldReference(_)) => + fail("Matched unknown transform") + case _ => + // expected + } + } + + test("Months extractor") { + transform("months", ref("a", "b")) match { + case MonthsTransform(FieldReference(seq)) => + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match MonthsTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case MonthsTransform(FieldReference(_)) => + fail("Matched unknown transform") + case _ => + // expected + } + } + + test("Days extractor") { + transform("days", ref("a", "b")) match { + case DaysTransform(FieldReference(seq)) => + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match DaysTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case DaysTransform(FieldReference(_)) => + fail("Matched unknown transform") + case _ => + // expected + } + } + + test("Hours extractor") { + transform("hours", ref("a", "b")) match { + case HoursTransform(FieldReference(seq)) => + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match HoursTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case HoursTransform(FieldReference(_)) => + fail("Matched unknown transform") + case _ => + // expected + } + } + + test("Bucket extractor") { + val col = ref("a", "b") + val bucketTransform = new Transform { + override def name: String = "bucket" + override def references: Array[NamedReference] = Array(col) + override def arguments: Array[Expression] = Array(lit(16), col) + override def describe: String = s"bucket(16, ${col.describe})" + } + + bucketTransform match { + case BucketTransform(numBuckets, FieldReference(seq)) => + assert(numBuckets === 16) + assert(seq === Seq("a", "b")) + case _ => + fail("Did not match BucketTransform extractor") + } + + transform("unknown", ref("a", "b")) match { + case BucketTransform(_, _) => + fail("Matched unknown transform") + case _ => + // expected + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala index 89452ee05cff3..b9e7cf3049896 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/CatalystTypeConvertersSuite.scala @@ -17,14 +17,18 @@ package org.apache.spark.sql.catalyst +import java.time.{Instant, LocalDate} + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData -import org.apache.spark.sql.catalyst.util.GenericArrayData +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.{DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String -class CatalystTypeConvertersSuite extends SparkFunSuite { +class CatalystTypeConvertersSuite extends SparkFunSuite with SQLHelper { private val simpleTypes: Seq[DataType] = Seq( StringType, @@ -147,4 +151,69 @@ class CatalystTypeConvertersSuite extends SparkFunSuite { val expected = UTF8String.fromString("X") assert(converter(chr) === expected) } + + test("converting java.time.Instant to TimestampType") { + Seq( + "0101-02-16T10:11:32Z", + "1582-10-02T01:02:03.04Z", + "1582-12-31T23:59:59.999999Z", + "1970-01-01T00:00:01.123Z", + "1972-12-31T23:59:59.123456Z", + "2019-02-16T18:12:30Z", + "2119-03-16T19:13:31Z").foreach { timestamp => + val input = Instant.parse(timestamp) + val result = CatalystTypeConverters.convertToCatalyst(input) + val expected = DateTimeUtils.instantToMicros(input) + assert(result === expected) + } + } + + test("converting TimestampType to java.time.Instant") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + Seq( + -9463427405253013L, + -244000001L, + 0L, + 99628200102030L, + 1543749753123456L).foreach { us => + val instant = DateTimeUtils.microsToInstant(us) + assert(CatalystTypeConverters.createToScalaConverter(TimestampType)(us) === instant) + } + } + } + + test("converting java.time.LocalDate to DateType") { + Seq( + "0101-02-16", + "1582-10-02", + "1582-12-31", + "1970-01-01", + "1972-12-31", + "2019-02-16", + "2119-03-16").foreach { timestamp => + val input = LocalDate.parse(timestamp) + val result = CatalystTypeConverters.convertToCatalyst(input) + val expected = DateTimeUtils.localDateToDays(input) + assert(result === expected) + } + } + + test("converting DateType to java.time.LocalDate") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + Seq( + -701265, + -371419, + -199722, + -1, + 0, + 967, + 2094, + 17877, + 24837, + 1110657).foreach { days => + val localDate = DateTimeUtils.daysToLocalDate(days) + assert(CatalystTypeConverters.createToScalaConverter(DateType)(days) === localDate) + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala new file mode 100644 index 0000000000000..9593a720e4248 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/QueryPlanningTrackerSuite.scala @@ -0,0 +1,89 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst + +import org.apache.spark.SparkFunSuite + +class QueryPlanningTrackerSuite extends SparkFunSuite { + + test("phases") { + val t = new QueryPlanningTracker + t.measurePhase("p1") { + Thread.sleep(1) + } + + assert(t.phases("p1").durationMs > 0) + assert(!t.phases.contains("p2")) + } + + test("multiple measurePhase call") { + val t = new QueryPlanningTracker + t.measurePhase("p1") { Thread.sleep(1) } + val s1 = t.phases("p1") + assert(s1.durationMs > 0) + + t.measurePhase("p1") { Thread.sleep(1) } + val s2 = t.phases("p1") + assert(s2.durationMs > s1.durationMs) + } + + test("rules") { + val t = new QueryPlanningTracker + t.recordRuleInvocation("r1", 1, effective = false) + t.recordRuleInvocation("r2", 2, effective = true) + t.recordRuleInvocation("r3", 1, effective = false) + t.recordRuleInvocation("r3", 2, effective = true) + + val rules = t.rules + + assert(rules("r1").totalTimeNs == 1) + assert(rules("r1").numInvocations == 1) + assert(rules("r1").numEffectiveInvocations == 0) + + assert(rules("r2").totalTimeNs == 2) + assert(rules("r2").numInvocations == 1) + assert(rules("r2").numEffectiveInvocations == 1) + + assert(rules("r3").totalTimeNs == 3) + assert(rules("r3").numInvocations == 2) + assert(rules("r3").numEffectiveInvocations == 1) + } + + test("topRulesByTime") { + val t = new QueryPlanningTracker + + // Return empty seq when k = 0 + assert(t.topRulesByTime(0) == Seq.empty) + assert(t.topRulesByTime(1) == Seq.empty) + + t.recordRuleInvocation("r2", 2, effective = true) + t.recordRuleInvocation("r4", 4, effective = true) + t.recordRuleInvocation("r1", 1, effective = false) + t.recordRuleInvocation("r3", 3, effective = false) + + // k <= total size + assert(t.topRulesByTime(0) == Seq.empty) + val top = t.topRulesByTime(2) + assert(top.size == 2) + assert(top(0)._1 == "r4") + assert(top(1)._1 == "r3") + + // k > total size + assert(t.topRulesByTime(10).size == 4) + } +} 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 f9ee948b97e0a..a2e58c3eaa0bd 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 @@ -19,12 +19,13 @@ package org.apache.spark.sql.catalyst import java.sql.{Date, Timestamp} +import scala.reflect.runtime.universe.TypeTag + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute -import org.apache.spark.sql.catalyst.expressions.{BoundReference, Expression, Literal, SpecificInternalRow, UpCast} +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.UTF8String case class PrimitiveData( intField: Int, @@ -80,6 +81,10 @@ case class MultipleConstructorsData(a: Int, b: String, c: Double) { def this(b: String, a: Int) = this(a, b, c = 1.0) } +class FooAnnotation extends scala.annotation.StaticAnnotation + +case class FooWithAnnotation(f1: String @FooAnnotation, f2: Option[String] @FooAnnotation) + case class SpecialCharAsFieldData(`field.1`: String, `field 2`: String) object TestingUDT { @@ -108,10 +113,48 @@ object TestingUDT { } } +/** An example derived from Twitter/Scrooge codegen for thrift */ +object ScroogeLikeExample { + def apply(x: Int): ScroogeLikeExample = new Immutable(x) + + def unapply(_item: ScroogeLikeExample): Option[Int] = Some(_item.x) + + class Immutable(val x: Int) extends ScroogeLikeExample +} + +trait ScroogeLikeExample extends Product1[Int] with Serializable { + import ScroogeLikeExample._ + + def x: Int + + def _1: Int = x + + override def canEqual(other: Any): Boolean = other.isInstanceOf[ScroogeLikeExample] + + override def equals(other: Any): Boolean = + canEqual(other) && + this.x == other.asInstanceOf[ScroogeLikeExample].x + + override def hashCode: Int = x +} class ScalaReflectionSuite extends SparkFunSuite { import org.apache.spark.sql.catalyst.ScalaReflection._ + // A helper method used to test `ScalaReflection.serializerForType`. + private def serializerFor[T: TypeTag]: Expression = + serializerForType(ScalaReflection.localTypeOf[T]) + + // A helper method used to test `ScalaReflection.deserializerForType`. + private def deserializerFor[T: TypeTag]: Expression = + deserializerForType(ScalaReflection.localTypeOf[T]) + + test("isSubtype") { + assert(isSubtype(localTypeOf[Option[Int]], localTypeOf[Option[_]])) + assert(isSubtype(localTypeOf[Option[Int]], localTypeOf[Option[Int]])) + assert(!isSubtype(localTypeOf[Option[_]], localTypeOf[Option[Int]])) + } + test("SQLUserDefinedType annotation on Scala structure") { val schema = schemaFor[TestingUDT.NestedStruct] assert(schema === Schema( @@ -263,13 +306,9 @@ class ScalaReflectionSuite extends SparkFunSuite { test("SPARK-15062: Get correct serializer for List[_]") { val list = List(1, 2, 3) - val serializer = serializerFor[List[Int]](BoundReference( - 0, ObjectType(list.getClass), nullable = false)) - assert(serializer.children.size == 2) - assert(serializer.children.head.isInstanceOf[Literal]) - assert(serializer.children.head.asInstanceOf[Literal].value === UTF8String.fromString("value")) - assert(serializer.children.last.isInstanceOf[NewInstance]) - assert(serializer.children.last.asInstanceOf[NewInstance] + val serializer = serializerFor[List[Int]] + assert(serializer.isInstanceOf[NewInstance]) + assert(serializer.asInstanceOf[NewInstance] .cls.isAssignableFrom(classOf[org.apache.spark.sql.catalyst.util.GenericArrayData])) } @@ -280,59 +319,58 @@ class ScalaReflectionSuite extends SparkFunSuite { test("serialize and deserialize arbitrary sequence types") { import scala.collection.immutable.Queue - val queueSerializer = serializerFor[Queue[Int]](BoundReference( - 0, ObjectType(classOf[Queue[Int]]), nullable = false)) - assert(queueSerializer.dataType.head.dataType == + val queueSerializer = serializerFor[Queue[Int]] + assert(queueSerializer.dataType == ArrayType(IntegerType, containsNull = false)) val queueDeserializer = deserializerFor[Queue[Int]] assert(queueDeserializer.dataType == ObjectType(classOf[Queue[_]])) import scala.collection.mutable.ArrayBuffer - val arrayBufferSerializer = serializerFor[ArrayBuffer[Int]](BoundReference( - 0, ObjectType(classOf[ArrayBuffer[Int]]), nullable = false)) - assert(arrayBufferSerializer.dataType.head.dataType == + val arrayBufferSerializer = serializerFor[ArrayBuffer[Int]] + assert(arrayBufferSerializer.dataType == ArrayType(IntegerType, containsNull = false)) val arrayBufferDeserializer = deserializerFor[ArrayBuffer[Int]] assert(arrayBufferDeserializer.dataType == ObjectType(classOf[ArrayBuffer[_]])) } test("serialize and deserialize arbitrary map types") { - val mapSerializer = serializerFor[Map[Int, Int]](BoundReference( - 0, ObjectType(classOf[Map[Int, Int]]), nullable = false)) - assert(mapSerializer.dataType.head.dataType == + val mapSerializer = serializerFor[Map[Int, Int]] + assert(mapSerializer.dataType == MapType(IntegerType, IntegerType, valueContainsNull = false)) val mapDeserializer = deserializerFor[Map[Int, Int]] assert(mapDeserializer.dataType == ObjectType(classOf[Map[_, _]])) import scala.collection.immutable.HashMap - val hashMapSerializer = serializerFor[HashMap[Int, Int]](BoundReference( - 0, ObjectType(classOf[HashMap[Int, Int]]), nullable = false)) - assert(hashMapSerializer.dataType.head.dataType == + val hashMapSerializer = serializerFor[HashMap[Int, Int]] + assert(hashMapSerializer.dataType == MapType(IntegerType, IntegerType, valueContainsNull = false)) val hashMapDeserializer = deserializerFor[HashMap[Int, Int]] assert(hashMapDeserializer.dataType == ObjectType(classOf[HashMap[_, _]])) import scala.collection.mutable.{LinkedHashMap => LHMap} - val linkedHashMapSerializer = serializerFor[LHMap[Long, String]](BoundReference( - 0, ObjectType(classOf[LHMap[Long, String]]), nullable = false)) - assert(linkedHashMapSerializer.dataType.head.dataType == + val linkedHashMapSerializer = serializerFor[LHMap[Long, String]] + assert(linkedHashMapSerializer.dataType == MapType(LongType, StringType, valueContainsNull = true)) val linkedHashMapDeserializer = deserializerFor[LHMap[Long, String]] assert(linkedHashMapDeserializer.dataType == ObjectType(classOf[LHMap[_, _]])) } test("SPARK-22442: Generate correct field names for special characters") { - val serializer = serializerFor[SpecialCharAsFieldData](BoundReference( - 0, ObjectType(classOf[SpecialCharAsFieldData]), nullable = false)) + val serializer = serializerFor[SpecialCharAsFieldData] + .collect { + case If(_, _, s: CreateNamedStruct) => s + }.head val deserializer = deserializerFor[SpecialCharAsFieldData] assert(serializer.dataType(0).name == "field.1") assert(serializer.dataType(1).name == "field 2") - val argumentsFields = deserializer.asInstanceOf[NewInstance].arguments.flatMap { _.collect { - case UpCast(u: UnresolvedAttribute, _, _) => u.nameParts + val newInstance = deserializer.collect { case n: NewInstance => n }.head + + val argumentsFields = newInstance.arguments.flatMap { _.collect { + case UpCast(u: UnresolvedExtractValue, _, _) => u.extraction.toString }} - assert(argumentsFields(0) == Seq("field.1")) - assert(argumentsFields(1) == Seq("field 2")) + assert(argumentsFields(0) == "field.1") + assert(argumentsFields(1) == "field 2") } test("SPARK-22472: add null check for top-level primitive values") { @@ -351,11 +389,25 @@ class ScalaReflectionSuite extends SparkFunSuite { test("SPARK-23835: add null check to non-nullable types in Tuples") { def numberOfCheckedArguments(deserializer: Expression): Int = { - assert(deserializer.isInstanceOf[NewInstance]) - deserializer.asInstanceOf[NewInstance].arguments.count(_.isInstanceOf[AssertNotNull]) + val newInstance = deserializer.collect { case n: NewInstance => n}.head + newInstance.arguments.count(_.isInstanceOf[AssertNotNull]) } assert(numberOfCheckedArguments(deserializerFor[(Double, Double)]) == 2) assert(numberOfCheckedArguments(deserializerFor[(java.lang.Double, Int)]) == 1) assert(numberOfCheckedArguments(deserializerFor[(java.lang.Integer, java.lang.Integer)]) == 0) } + + test("SPARK-8288: schemaFor works for a class with only a companion object constructor") { + val schema = schemaFor[ScroogeLikeExample] + assert(schema === Schema( + StructType(Seq( + StructField("x", IntegerType, nullable = false))), nullable = true)) + } + + test("SPARK-27625: annotated data types") { + assert(serializerFor[FooWithAnnotation].dataType == StructType(Seq( + StructField("f1", StringType), + StructField("f2", StringType)))) + assert(deserializerFor[FooWithAnnotation].dataType == ObjectType(classOf[FooWithAnnotation])) + } } 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 94778840d706b..4d42f5fb73362 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 @@ -17,8 +17,6 @@ package org.apache.spark.sql.catalyst.analysis -import scala.beans.{BeanInfo, BeanProperty} - import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -30,8 +28,9 @@ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, GenericArrayData, MapData} import org.apache.spark.sql.types._ -@BeanInfo -private[sql] case class GroupableData(@BeanProperty data: Int) +private[sql] case class GroupableData(data: Int) { + def getData: Int = data +} private[sql] class GroupableUDT extends UserDefinedType[GroupableData] { @@ -50,8 +49,9 @@ private[sql] class GroupableUDT extends UserDefinedType[GroupableData] { private[spark] override def asNullable: GroupableUDT = this } -@BeanInfo -private[sql] case class UngroupableData(@BeanProperty data: Map[Int, Int]) +private[sql] case class UngroupableData(data: Map[Int, Int]) { + def getData: Map[Int, Int] = data +} private[sql] class UngroupableUDT extends UserDefinedType[UngroupableData] { @@ -215,11 +215,6 @@ class AnalysisErrorSuite extends AnalysisTest { testRelation2.groupBy('a)(sum(UnresolvedStar(None))), "Invalid usage of '*'" :: "in expression 'sum'" :: Nil) - errorTest( - "bad casts", - testRelation.select(Literal(1).cast(BinaryType).as('badCast)), - "cannot cast" :: Literal(1).dataType.simpleString :: BinaryType.simpleString :: Nil) - errorTest( "sorting by unsupported column types", mapRelation.orderBy('map.asc), @@ -228,7 +223,7 @@ class AnalysisErrorSuite extends AnalysisTest { errorTest( "sorting by attributes are not from grouping expressions", testRelation2.groupBy('a, 'c)('a, 'c, count('a).as("a3")).orderBy('b.asc), - "cannot resolve" :: "'`b`'" :: "given input columns" :: "[a, c, a3]" :: Nil) + "cannot resolve" :: "'`b`'" :: "given input columns" :: "[a, a3, c]" :: Nil) errorTest( "non-boolean filters", @@ -443,7 +438,7 @@ class AnalysisErrorSuite extends AnalysisTest { } test("error test for self-join") { - val join = Join(testRelation, testRelation, Cross, None) + val join = Join(testRelation, testRelation, Cross, None, JoinHint.NONE) val error = intercept[AnalysisException] { SimpleAnalyzer.checkAnalysis(join) } @@ -565,7 +560,8 @@ class AnalysisErrorSuite extends AnalysisTest { LocalRelation(b), Filter(EqualTo(UnresolvedAttribute("a"), c), LocalRelation(c)), LeftOuter, - Option(EqualTo(b, c)))), + Option(EqualTo(b, c)), + JoinHint.NONE)), LocalRelation(a)) assertAnalysisError(plan1, "Accessing outer query column is not allowed in" :: Nil) @@ -575,7 +571,8 @@ class AnalysisErrorSuite extends AnalysisTest { Filter(EqualTo(UnresolvedAttribute("a"), c), LocalRelation(c)), LocalRelation(b), RightOuter, - Option(EqualTo(b, c)))), + Option(EqualTo(b, c)), + JoinHint.NONE)), LocalRelation(a)) assertAnalysisError(plan2, "Accessing outer query column is not allowed in" :: Nil) @@ -602,4 +599,12 @@ class AnalysisErrorSuite extends AnalysisTest { assertAnalysisError(plan5, "Accessing outer query column is not allowed in" :: Nil) } + + test("Error on filter condition containing aggregate expressions") { + val a = AttributeReference("a", IntegerType)() + val b = AttributeReference("b", IntegerType)() + val plan = Filter('a === UnresolvedFunction("max", Seq(b), true), LocalRelation(a, b)) + assertAnalysisError(plan, + "Aggregate/Window/Generate expressions are not valid in where clause of the query" :: Nil) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.scala new file mode 100644 index 0000000000000..1464d5c645b0d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisExternalCatalogSuite.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.catalyst.analysis + +import java.net.URI + +import org.mockito.Mockito._ +import org.scalatest.Matchers + +import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, CatalogStorageFormat, CatalogTable, CatalogTableType, ExternalCatalog, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +class AnalysisExternalCatalogSuite extends AnalysisTest with Matchers { + private def getAnalyzer(externCatalog: ExternalCatalog): Analyzer = { + val conf = new SQLConf() + val catalog = new SessionCatalog(externCatalog, FunctionRegistry.builtin, conf) + catalog.createDatabase( + CatalogDatabase("default", "", new URI("loc"), Map.empty), + ignoreIfExists = false) + catalog.createTable( + CatalogTable( + TableIdentifier("t1", Some("default")), + CatalogTableType.MANAGED, + CatalogStorageFormat.empty, + StructType(Seq(StructField("a", IntegerType, nullable = true)))), + ignoreIfExists = false) + new Analyzer(catalog, conf) + } + + test("query builtin functions don't call the external catalog") { + val inMemoryCatalog = new InMemoryCatalog + val catalog = spy(inMemoryCatalog) + val analyzer = getAnalyzer(catalog) + reset(catalog) + val testRelation = LocalRelation(AttributeReference("a", IntegerType, nullable = true)()) + val func = + Alias(UnresolvedFunction("sum", Seq(UnresolvedAttribute("a")), isDistinct = false), "s")() + val plan = Project(Seq(func), testRelation) + analyzer.execute(plan) + verifyZeroInteractions(catalog) + } + + test("check the existence of builtin functions don't call the external catalog") { + val inMemoryCatalog = new InMemoryCatalog + val externCatalog = spy(inMemoryCatalog) + val catalog = new SessionCatalog(externCatalog, FunctionRegistry.builtin, conf) + catalog.createDatabase( + CatalogDatabase("default", "", new URI("loc"), Map.empty), + ignoreIfExists = false) + reset(externCatalog) + catalog.functionExists(FunctionIdentifier("sum")) + verifyZeroInteractions(externCatalog) + } + +} 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 cf76c92b093b7..74445a111e4d7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.analysis -import java.util.TimeZone +import java.util.{Locale, TimeZone} import scala.reflect.ClassTag @@ -25,13 +25,16 @@ import org.scalatest.Matchers import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser.parsePlan import org.apache.spark.sql.catalyst.plans.{Cross, Inner} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, RangePartitioning, RoundRobinPartitioning} +import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -301,33 +304,40 @@ class AnalysisSuite extends AnalysisTest with Matchers { } test("SPARK-11725: correctly handle null inputs for ScalaUDF") { - val string = testRelation2.output(0) - val double = testRelation2.output(2) - val short = testRelation2.output(4) + val testRelation = LocalRelation( + AttributeReference("a", StringType)(), + AttributeReference("b", DoubleType)(), + AttributeReference("c", ShortType)(), + AttributeReference("d", DoubleType, nullable = false)()) + + val string = testRelation.output(0) + val double = testRelation.output(1) + val short = testRelation.output(2) + val nonNullableDouble = testRelation.output(3) val nullResult = Literal.create(null, StringType) def checkUDF(udf: Expression, transformed: Expression): Unit = { checkAnalysis( - Project(Alias(udf, "")() :: Nil, testRelation2), - Project(Alias(transformed, "")() :: Nil, testRelation2) + Project(Alias(udf, "")() :: Nil, testRelation), + Project(Alias(transformed, "")() :: Nil, testRelation) ) } // non-primitive parameters do not need special null handling - val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil) + val udf1 = ScalaUDF((s: String) => "x", StringType, string :: Nil, false :: 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, - nullableTypes = true :: false :: Nil) + false :: true :: 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, - nullableTypes = false :: false :: Nil) + true :: true :: Nil) val expected3 = If( IsNull(short) || IsNull(double), nullResult, @@ -335,24 +345,23 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkUDF(udf3, expected3) // we can skip special null handling for primitive parameters that are not nullable - // TODO: this is disabled for now as we can not completely trust `nullable`. val udf4 = ScalaUDF( (s: Short, d: Double) => "x", StringType, - short :: double.withNullability(false) :: Nil, - nullableTypes = false :: false :: Nil) + short :: nonNullableDouble :: Nil, + true :: true :: Nil) val expected4 = If( IsNull(short), nullResult, - udf4.copy(children = KnownNotNull(short) :: double.withNullability(false) :: Nil)) - // checkUDF(udf4, expected4) + udf4.copy(children = KnownNotNull(short) :: nonNullableDouble :: Nil)) + checkUDF(udf4, expected4) } 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, nullableTypes = false :: false :: Nil) - val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, nullableTypes = false :: false :: Nil) + val udf1 = ScalaUDF(func, IntegerType, a :: a :: Nil, false :: false :: Nil) + val udf2 = ScalaUDF(func, IntegerType, a :: udf1 :: Nil, false :: false :: Nil) val plan = Project(Alias(udf2, "")() :: Nil, testRelation) comparePlans(plan.analyze, plan.analyze.analyze) } @@ -395,7 +404,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { Join( Project(Seq($"x.key"), SubqueryAlias("x", input)), Project(Seq($"y.key"), SubqueryAlias("y", input)), - Cross, None)) + Cross, None, JoinHint.NONE)) assertAnalysisSuccess(query) } @@ -576,7 +585,7 @@ class AnalysisSuite extends AnalysisTest with Matchers { Seq(UnresolvedAttribute("a")), pythonUdf, output, project) val left = SubqueryAlias("temp0", flatMapGroupsInPandas) val right = SubqueryAlias("temp1", flatMapGroupsInPandas) - val join = Join(left, right, Inner, None) + val join = Join(left, right, Inner, None, JoinHint.NONE) assertAnalysisSuccess( Project(Seq(UnresolvedAttribute("temp0.a"), UnresolvedAttribute("temp1.a")), join)) } @@ -604,4 +613,41 @@ class AnalysisSuite extends AnalysisTest with Matchers { checkAnalysis(input, expected) } } + + test("SPARK-25691: AliasViewChild with different nullabilities") { + object ViewAnalyzer extends RuleExecutor[LogicalPlan] { + val batches = Batch("View", Once, EliminateView) :: Nil + } + val relation = LocalRelation('a.int.notNull, 'b.string) + val view = View(CatalogTable( + identifier = TableIdentifier("v1"), + tableType = CatalogTableType.VIEW, + storage = CatalogStorageFormat.empty, + schema = StructType(Seq(StructField("a", IntegerType), StructField("b", StringType)))), + output = Seq('a.int, 'b.string), + child = relation) + val tz = Option(conf.sessionLocalTimeZone) + val expected = Project(Seq( + Alias(Cast('a.int.notNull, IntegerType, tz), "a")(), + Alias(Cast('b.string, StringType, tz), "b")()), + relation) + val res = ViewAnalyzer.execute(view) + comparePlans(res, expected) + } + + test("CTE with non-existing column alias") { + assertAnalysisError(parsePlan("WITH t(x) AS (SELECT 1) SELECT * FROM t WHERE y = 1"), + Seq("cannot resolve '`y`' given input columns: [x]")) + } + + test("CTE with non-matching column alias") { + assertAnalysisError(parsePlan("WITH t(x, y) AS (SELECT 1) SELECT * FROM t WHERE x = 1"), + Seq("Number of column aliases does not match number of columns. Number of column aliases: " + + "2; number of columns: 1.")) + } + + test("SPARK-28251: Insert into non-existing table error message is user friendly") { + assertAnalysisError(parsePlan("INSERT INTO test VALUES (1)"), + Seq("Table not found: test")) + } } 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 3d7c91870133b..7d196f8b8edd2 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 @@ -21,7 +21,9 @@ import java.net.URI import java.util.Locale import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.catalog.{CatalogDatabase, InMemoryCatalog, SessionCatalog} +import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.internal.SQLConf @@ -54,7 +56,7 @@ trait AnalysisTest extends PlanTest { expectedPlan: LogicalPlan, caseSensitive: Boolean = true): Unit = { val analyzer = getAnalyzer(caseSensitive) - val actualPlan = analyzer.executeAndCheck(inputPlan) + val actualPlan = analyzer.executeAndCheck(inputPlan, new QueryPlanningTracker) comparePlans(actualPlan, expectedPlan) } @@ -106,4 +108,12 @@ trait AnalysisTest extends PlanTest { """.stripMargin) } } + + protected def interceptParseException( + parser: String => Any)(sqlCommand: String, messages: String*): Unit = { + val e = intercept[ParseException](parser(sqlCommand)) + messages.foreach { message => + assert(e.message.contains(message)) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala new file mode 100644 index 0000000000000..1ce8852f71bc8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/CreateTablePartitioningValidationSuite.scala @@ -0,0 +1,153 @@ +/* + * 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.analysis + +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TestTableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.LogicalExpressions +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, LeafNode} +import org.apache.spark.sql.types.{DoubleType, LongType, StringType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class CreateTablePartitioningValidationSuite extends AnalysisTest { + import CreateTablePartitioningValidationSuite._ + + test("CreateTableAsSelect: fail missing top-level column") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "does_not_exist") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assert(!plan.resolved) + assertAnalysisError(plan, Seq( + "Invalid partitioning", + "does_not_exist is missing or is in a map or array")) + } + + test("CreateTableAsSelect: fail missing top-level column nested reference") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "does_not_exist.z") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assert(!plan.resolved) + assertAnalysisError(plan, Seq( + "Invalid partitioning", + "does_not_exist.z is missing or is in a map or array")) + } + + test("CreateTableAsSelect: fail missing nested column") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "point.z") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assert(!plan.resolved) + assertAnalysisError(plan, Seq( + "Invalid partitioning", + "point.z is missing or is in a map or array")) + } + + test("CreateTableAsSelect: fail with multiple errors") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "does_not_exist", "point.z") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assert(!plan.resolved) + assertAnalysisError(plan, Seq( + "Invalid partitioning", + "point.z is missing or is in a map or array", + "does_not_exist is missing or is in a map or array")) + } + + test("CreateTableAsSelect: success with top-level column") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "id") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assertAnalysisSuccess(plan) + } + + test("CreateTableAsSelect: success using nested column") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "point.x") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assertAnalysisSuccess(plan) + } + + test("CreateTableAsSelect: success using complex column") { + val plan = CreateTableAsSelect( + catalog, + Identifier.of(Array(), "table_name"), + LogicalExpressions.bucket(4, "point") :: Nil, + TestRelation2, + Map.empty, + Map.empty, + ignoreIfExists = false) + + assertAnalysisSuccess(plan) + } +} + +private object CreateTablePartitioningValidationSuite { + val catalog: TableCatalog = { + val cat = new TestTableCatalog() + cat.initialize("test", CaseInsensitiveStringMap.empty()) + cat + } + + val schema: StructType = new StructType() + .add("id", LongType) + .add("data", StringType) + .add("point", new StructType().add("x", DoubleType).add("y", DoubleType)) +} + +private case object TestRelation2 extends LeafNode with NamedRelation { + override def name: String = "source_relation" + override def output: Seq[AttributeReference] = + CreateTablePartitioningValidationSuite.schema.toAttributes +} + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala index 48a3ca2ccfb0b..388eb238db0c0 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DSLHintSuite.scala @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.catalyst.analysis -import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ class DSLHintSuite extends AnalysisTest { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DataSourceV2AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DataSourceV2AnalysisSuite.scala index 6c899b610ac5b..58923f26c1ead 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DataSourceV2AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DataSourceV2AnalysisSuite.scala @@ -19,15 +19,98 @@ package org.apache.spark.sql.catalyst.analysis import java.util.Locale -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, UpCast} -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LeafNode, LogicalPlan, Project} -import org.apache.spark.sql.types.{DoubleType, FloatType, StructField, StructType} +import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Cast, Expression, LessThanOrEqual, Literal} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.types._ + +class V2AppendDataAnalysisSuite extends DataSourceV2AnalysisSuite { + override def byName(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + AppendData.byName(table, query) + } + + override def byPosition(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + AppendData.byPosition(table, query) + } +} + +class V2OverwritePartitionsDynamicAnalysisSuite extends DataSourceV2AnalysisSuite { + override def byName(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwritePartitionsDynamic.byName(table, query) + } + + override def byPosition(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwritePartitionsDynamic.byPosition(table, query) + } +} + +class V2OverwriteByExpressionAnalysisSuite extends DataSourceV2AnalysisSuite { + override def byName(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwriteByExpression.byName(table, query, Literal(true)) + } + + override def byPosition(table: NamedRelation, query: LogicalPlan): LogicalPlan = { + OverwriteByExpression.byPosition(table, query, Literal(true)) + } + + test("delete expression is resolved using table fields") { + val table = TestRelation(StructType(Seq( + StructField("x", DoubleType, nullable = false), + StructField("y", DoubleType))).toAttributes) + + val query = TestRelation(StructType(Seq( + StructField("a", DoubleType, nullable = false), + StructField("b", DoubleType))).toAttributes) + + val a = query.output.head + val b = query.output.last + val x = table.output.head + + val parsedPlan = OverwriteByExpression.byPosition(table, query, + LessThanOrEqual(UnresolvedAttribute(Seq("x")), Literal(15.0d))) + + val expectedPlan = OverwriteByExpression.byPosition(table, + Project(Seq( + Alias(Cast(a, DoubleType, Some(conf.sessionLocalTimeZone)), "x")(), + Alias(Cast(b, DoubleType, Some(conf.sessionLocalTimeZone)), "y")()), + query), + LessThanOrEqual( + AttributeReference("x", DoubleType, nullable = false)(x.exprId), + Literal(15.0d))) + + assertNotResolved(parsedPlan) + checkAnalysis(parsedPlan, expectedPlan) + assertResolved(expectedPlan) + } + + test("delete expression is not resolved using query fields") { + val xRequiredTable = TestRelation(StructType(Seq( + StructField("x", DoubleType, nullable = false), + StructField("y", DoubleType))).toAttributes) + + val query = TestRelation(StructType(Seq( + StructField("a", DoubleType, nullable = false), + StructField("b", DoubleType))).toAttributes) + + // the write is resolved (checked above). this test plan is not because of the expression. + val parsedPlan = OverwriteByExpression.byPosition(xRequiredTable, query, + LessThanOrEqual(UnresolvedAttribute(Seq("a")), Literal(15.0d))) + + assertNotResolved(parsedPlan) + assertAnalysisError(parsedPlan, Seq("cannot resolve", "`a`", "given input columns", "x, y")) + } +} case class TestRelation(output: Seq[AttributeReference]) extends LeafNode with NamedRelation { override def name: String = "table-name" } -class DataSourceV2AnalysisSuite extends AnalysisTest { +case class TestRelationAcceptAnySchema(output: Seq[AttributeReference]) + extends LeafNode with NamedRelation { + override def name: String = "test-name" + override def skipSchemaResolution: Boolean = true +} + +abstract class DataSourceV2AnalysisSuite extends AnalysisTest { val table = TestRelation(StructType(Seq( StructField("x", FloatType), StructField("y", FloatType))).toAttributes) @@ -40,21 +123,25 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { StructField("x", DoubleType), StructField("y", DoubleType))).toAttributes) - test("Append.byName: basic behavior") { + def byName(table: NamedRelation, query: LogicalPlan): LogicalPlan + + def byPosition(table: NamedRelation, query: LogicalPlan): LogicalPlan + + test("byName: basic behavior") { val query = TestRelation(table.schema.toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) checkAnalysis(parsedPlan, parsedPlan) assertResolved(parsedPlan) } - test("Append.byName: does not match by position") { + test("byName: does not match by position") { val query = TestRelation(StructType(Seq( StructField("a", FloatType), StructField("b", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -62,12 +149,12 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot find data for output column", "'x'", "'y'")) } - test("Append.byName: case sensitive column resolution") { + test("byName: case sensitive column resolution") { val query = TestRelation(StructType(Seq( StructField("X", FloatType), // doesn't match case! StructField("y", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -76,7 +163,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { caseSensitive = true) } - test("Append.byName: case insensitive column resolution") { + test("byName: case insensitive column resolution") { val query = TestRelation(StructType(Seq( StructField("X", FloatType), // doesn't match case! StructField("y", FloatType))).toAttributes) @@ -84,8 +171,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { val X = query.output.head val y = query.output.last - val parsedPlan = AppendData.byName(table, query) - val expectedPlan = AppendData.byName(table, + val parsedPlan = byName(table, query) + val expectedPlan = byName(table, Project(Seq( Alias(Cast(toLower(X), FloatType, Some(conf.sessionLocalTimeZone)), "x")(), Alias(Cast(y, FloatType, Some(conf.sessionLocalTimeZone)), "y")()), @@ -96,7 +183,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byName: data columns are reordered by name") { + test("byName: data columns are reordered by name") { // out of order val query = TestRelation(StructType(Seq( StructField("y", FloatType), @@ -105,8 +192,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { val y = query.output.head val x = query.output.last - val parsedPlan = AppendData.byName(table, query) - val expectedPlan = AppendData.byName(table, + val parsedPlan = byName(table, query) + val expectedPlan = byName(table, Project(Seq( Alias(Cast(x, FloatType, Some(conf.sessionLocalTimeZone)), "x")(), Alias(Cast(y, FloatType, Some(conf.sessionLocalTimeZone)), "y")()), @@ -117,26 +204,26 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byName: fail nullable data written to required columns") { - val parsedPlan = AppendData.byName(requiredTable, table) + test("byName: fail nullable data written to required columns") { + val parsedPlan = byName(requiredTable, table) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( "Cannot write incompatible data to table", "'table-name'", "Cannot write nullable values to non-null column", "'x'", "'y'")) } - test("Append.byName: allow required data written to nullable columns") { - val parsedPlan = AppendData.byName(table, requiredTable) + test("byName: allow required data written to nullable columns") { + val parsedPlan = byName(table, requiredTable) assertResolved(parsedPlan) checkAnalysis(parsedPlan, parsedPlan) } - test("Append.byName: missing required columns cause failure and are identified by name") { + test("byName: missing required columns cause failure and are identified by name") { // missing required field x val query = TestRelation(StructType(Seq( StructField("y", FloatType, nullable = false))).toAttributes) - val parsedPlan = AppendData.byName(requiredTable, query) + val parsedPlan = byName(requiredTable, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -144,12 +231,12 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot find data for output column", "'x'")) } - test("Append.byName: missing optional columns cause failure and are identified by name") { + test("byName: missing optional columns cause failure and are identified by name") { // missing optional field x val query = TestRelation(StructType(Seq( StructField("y", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -157,8 +244,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot find data for output column", "'x'")) } - test("Append.byName: fail canWrite check") { - val parsedPlan = AppendData.byName(table, widerTable) + test("byName: fail canWrite check") { + val parsedPlan = byName(table, widerTable) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -166,12 +253,12 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot safely cast", "'x'", "'y'", "DoubleType to FloatType")) } - test("Append.byName: insert safe cast") { + test("byName: insert safe cast") { val x = table.output.head val y = table.output.last - val parsedPlan = AppendData.byName(widerTable, table) - val expectedPlan = AppendData.byName(widerTable, + val parsedPlan = byName(widerTable, table) + val expectedPlan = byName(widerTable, Project(Seq( Alias(Cast(x, DoubleType, Some(conf.sessionLocalTimeZone)), "x")(), Alias(Cast(y, DoubleType, Some(conf.sessionLocalTimeZone)), "y")()), @@ -182,13 +269,13 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byName: fail extra data fields") { + test("byName: fail extra data fields") { val query = TestRelation(StructType(Seq( StructField("x", FloatType), StructField("y", FloatType), StructField("z", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -197,7 +284,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Data columns: 'x', 'y', 'z'")) } - test("Append.byName: multiple field errors are reported") { + test("byName: multiple field errors are reported") { val xRequiredTable = TestRelation(StructType(Seq( StructField("x", FloatType, nullable = false), StructField("y", DoubleType))).toAttributes) @@ -206,7 +293,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { StructField("x", DoubleType), StructField("b", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(xRequiredTable, query) + val parsedPlan = byName(xRequiredTable, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -216,7 +303,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot find data for output column", "'y'")) } - test("Append.byPosition: basic behavior") { + test("byPosition: basic behavior") { val query = TestRelation(StructType(Seq( StructField("a", FloatType), StructField("b", FloatType))).toAttributes) @@ -224,8 +311,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { val a = query.output.head val b = query.output.last - val parsedPlan = AppendData.byPosition(table, query) - val expectedPlan = AppendData.byPosition(table, + val parsedPlan = byPosition(table, query) + val expectedPlan = byPosition(table, Project(Seq( Alias(Cast(a, FloatType, Some(conf.sessionLocalTimeZone)), "x")(), Alias(Cast(b, FloatType, Some(conf.sessionLocalTimeZone)), "y")()), @@ -236,7 +323,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byPosition: data columns are not reordered") { + test("byPosition: data columns are not reordered") { // out of order val query = TestRelation(StructType(Seq( StructField("y", FloatType), @@ -245,8 +332,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { val y = query.output.head val x = query.output.last - val parsedPlan = AppendData.byPosition(table, query) - val expectedPlan = AppendData.byPosition(table, + val parsedPlan = byPosition(table, query) + val expectedPlan = byPosition(table, Project(Seq( Alias(Cast(y, FloatType, Some(conf.sessionLocalTimeZone)), "x")(), Alias(Cast(x, FloatType, Some(conf.sessionLocalTimeZone)), "y")()), @@ -257,26 +344,26 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byPosition: fail nullable data written to required columns") { - val parsedPlan = AppendData.byPosition(requiredTable, table) + test("byPosition: fail nullable data written to required columns") { + val parsedPlan = byPosition(requiredTable, table) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( "Cannot write incompatible data to table", "'table-name'", "Cannot write nullable values to non-null column", "'x'", "'y'")) } - test("Append.byPosition: allow required data written to nullable columns") { - val parsedPlan = AppendData.byPosition(table, requiredTable) + test("byPosition: allow required data written to nullable columns") { + val parsedPlan = byPosition(table, requiredTable) assertResolved(parsedPlan) checkAnalysis(parsedPlan, parsedPlan) } - test("Append.byPosition: missing required columns cause failure") { + test("byPosition: missing required columns cause failure") { // missing optional field x val query = TestRelation(StructType(Seq( StructField("y", FloatType, nullable = false))).toAttributes) - val parsedPlan = AppendData.byPosition(requiredTable, query) + val parsedPlan = byPosition(requiredTable, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -285,12 +372,12 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Data columns: 'y'")) } - test("Append.byPosition: missing optional columns cause failure") { + test("byPosition: missing optional columns cause failure") { // missing optional field x val query = TestRelation(StructType(Seq( StructField("y", FloatType))).toAttributes) - val parsedPlan = AppendData.byPosition(table, query) + val parsedPlan = byPosition(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -299,12 +386,12 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Data columns: 'y'")) } - test("Append.byPosition: fail canWrite check") { + test("byPosition: fail canWrite check") { val widerTable = TestRelation(StructType(Seq( StructField("a", DoubleType), StructField("b", DoubleType))).toAttributes) - val parsedPlan = AppendData.byPosition(table, widerTable) + val parsedPlan = byPosition(table, widerTable) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -312,7 +399,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot safely cast", "'x'", "'y'", "DoubleType to FloatType")) } - test("Append.byPosition: insert safe cast") { + test("byPosition: insert safe cast") { val widerTable = TestRelation(StructType(Seq( StructField("a", DoubleType), StructField("b", DoubleType))).toAttributes) @@ -320,8 +407,8 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { val x = table.output.head val y = table.output.last - val parsedPlan = AppendData.byPosition(widerTable, table) - val expectedPlan = AppendData.byPosition(widerTable, + val parsedPlan = byPosition(widerTable, table) + val expectedPlan = byPosition(widerTable, Project(Seq( Alias(Cast(x, DoubleType, Some(conf.sessionLocalTimeZone)), "a")(), Alias(Cast(y, DoubleType, Some(conf.sessionLocalTimeZone)), "b")()), @@ -332,13 +419,13 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { assertResolved(expectedPlan) } - test("Append.byPosition: fail extra data fields") { + test("byPosition: fail extra data fields") { val query = TestRelation(StructType(Seq( StructField("a", FloatType), StructField("b", FloatType), StructField("c", FloatType))).toAttributes) - val parsedPlan = AppendData.byName(table, query) + val parsedPlan = byName(table, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -347,7 +434,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Data columns: 'a', 'b', 'c'")) } - test("Append.byPosition: multiple field errors are reported") { + test("byPosition: multiple field errors are reported") { val xRequiredTable = TestRelation(StructType(Seq( StructField("x", FloatType, nullable = false), StructField("y", DoubleType))).toAttributes) @@ -356,7 +443,7 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { StructField("x", DoubleType), StructField("b", FloatType))).toAttributes) - val parsedPlan = AppendData.byPosition(xRequiredTable, query) + val parsedPlan = byPosition(xRequiredTable, query) assertNotResolved(parsedPlan) assertAnalysisError(parsedPlan, Seq( @@ -365,6 +452,64 @@ class DataSourceV2AnalysisSuite extends AnalysisTest { "Cannot safely cast", "'x'", "DoubleType to FloatType")) } + test("bypass output column resolution") { + val table = TestRelationAcceptAnySchema(StructType(Seq( + StructField("a", FloatType, nullable = false), + StructField("b", DoubleType))).toAttributes) + + val query = TestRelation(StructType(Seq( + StructField("s", StringType))).toAttributes) + + withClue("byName") { + val parsedPlan = byName(table, query) + assertResolved(parsedPlan) + checkAnalysis(parsedPlan, parsedPlan) + } + + withClue("byPosition") { + val parsedPlan = byPosition(table, query) + assertResolved(parsedPlan) + checkAnalysis(parsedPlan, parsedPlan) + } + } + + test("check fields of struct type column") { + val tableWithStructCol = TestRelation( + new StructType().add( + "col", new StructType().add("a", IntegerType).add("b", IntegerType) + ).toAttributes + ) + + val query = TestRelation( + new StructType().add( + "col", new StructType().add("x", IntegerType).add("y", IntegerType) + ).toAttributes + ) + + withClue("byName") { + val parsedPlan = byName(tableWithStructCol, query) + assertNotResolved(parsedPlan) + assertAnalysisError(parsedPlan, Seq( + "Cannot write incompatible data to table", "'table-name'", + "Struct 'col' 0-th field name does not match", "expected 'a', found 'x'", + "Struct 'col' 1-th field name does not match", "expected 'b', found 'y'")) + } + + withClue("byPosition") { + val parsedPlan = byPosition(tableWithStructCol, query) + assertNotResolved(parsedPlan) + + val expectedQuery = Project(Seq(Alias( + Cast( + query.output.head, + new StructType().add("a", IntegerType).add("b", IntegerType), + Some(conf.sessionLocalTimeZone)), + "col")()), + query) + checkAnalysis(parsedPlan, byPosition(tableWithStructCol, expectedQuery)) + } + } + def assertNotResolved(logicalPlan: LogicalPlan): Unit = { assert(!logicalPlan.resolved, s"Plan should not be resolved: $logicalPlan") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala index bd87ca6017e99..79fc38c4d30ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala @@ -174,18 +174,18 @@ class DecimalPrecisionSuite extends AnalysisTest with BeforeAndAfter { assert(d0.isWiderThan(d1) === false) assert(d1.isWiderThan(d0) === false) assert(d1.isWiderThan(d2) === false) - assert(d2.isWiderThan(d1) === true) + assert(d2.isWiderThan(d1)) assert(d2.isWiderThan(d3) === false) - assert(d3.isWiderThan(d2) === true) - assert(d4.isWiderThan(d3) === true) + assert(d3.isWiderThan(d2)) + assert(d4.isWiderThan(d3)) assert(d1.isWiderThan(ByteType) === false) - assert(d2.isWiderThan(ByteType) === true) + assert(d2.isWiderThan(ByteType)) assert(d2.isWiderThan(ShortType) === false) - assert(d3.isWiderThan(ShortType) === true) - assert(d3.isWiderThan(IntegerType) === true) + assert(d3.isWiderThan(ShortType)) + assert(d3.isWiderThan(IntegerType)) assert(d3.isWiderThan(LongType) === false) - assert(d4.isWiderThan(LongType) === true) + assert(d4.isWiderThan(LongType)) assert(d4.isWiderThan(FloatType) === false) assert(d4.isWiderThan(DoubleType) === false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala index 8eec14842c7e7..725764755c626 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ExpressionTypeCheckingSuite.scala @@ -144,6 +144,9 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { assertSuccess(Sum('stringField)) assertSuccess(Average('stringField)) assertSuccess(Min('arrayField)) + assertSuccess(new EveryAgg('booleanField)) + assertSuccess(new AnyAgg('booleanField)) + assertSuccess(new SomeAgg('booleanField)) assertError(Min('mapField), "min does not support ordering on type") assertError(Max('mapField), "max does not support ordering on type") @@ -158,6 +161,7 @@ class ExpressionTypeCheckingSuite extends SparkFunSuite { "input to function coalesce should all be the same type") assertError(Coalesce(Nil), "function coalesce requires at least one argument") assertError(new Murmur3Hash(Nil), "function hash requires at least one argument") + assertError(new XxHash64(Nil), "function xxhash64 requires at least one argument") assertError(Explode('intField), "input to function explode should be array or map type") assertError(PosExplode('intField), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala index 8da4d7e3aa372..aa5eda8e5ba87 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveGroupingAnalyticsSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.analysis import java.util.TimeZone +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -109,7 +110,7 @@ class ResolveGroupingAnalyticsSuite extends AnalysisTest { Seq(UnresolvedAlias(Multiply(unresolved_a, Literal(2))), unresolved_b, UnresolvedAlias(count(unresolved_c)))) - val resultPlan = getAnalyzer(true).executeAndCheck(originalPlan2) + val resultPlan = getAnalyzer(true).executeAndCheck(originalPlan2, new QueryPlanningTracker) val gExpressions = resultPlan.asInstanceOf[Aggregate].groupingExpressions assert(gExpressions.size == 3) val firstGroupingExprAttrName = 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 bd66ee5355f45..474e58a335e7c 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 @@ -17,6 +17,11 @@ package org.apache.spark.sql.catalyst.analysis +import scala.collection.mutable.ArrayBuffer + +import org.apache.log4j.{AppenderSkeleton, Level} +import org.apache.log4j.spi.LoggingEvent + import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.Literal @@ -27,6 +32,14 @@ import org.apache.spark.sql.catalyst.plans.logical._ class ResolveHintsSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.analysis.TestRelations._ + class MockAppender extends AppenderSkeleton { + val loggingEvents = new ArrayBuffer[LoggingEvent]() + + override def append(loggingEvent: LoggingEvent): Unit = loggingEvents.append(loggingEvent) + override def close(): Unit = {} + override def requiresLayout(): Boolean = false + } + test("invalid hints should be ignored") { checkAnalysis( UnresolvedHint("some_random_hint_that_does_not_exist", Seq("TaBlE"), table("TaBlE")), @@ -37,17 +50,17 @@ class ResolveHintsSuite extends AnalysisTest { test("case-sensitive or insensitive parameters") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(broadcast = true)), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(broadcast = true)), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("TaBlE"), table("TaBlE")), - ResolvedHint(testRelation, HintInfo(broadcast = true)), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = true) checkAnalysis( @@ -59,28 +72,29 @@ class ResolveHintsSuite extends AnalysisTest { test("multiple broadcast hint aliases") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table", "table2"), table("table").join(table("table2"))), - Join(ResolvedHint(testRelation, HintInfo(broadcast = true)), - ResolvedHint(testRelation2, HintInfo(broadcast = true)), Inner, None), + Join(ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), + ResolvedHint(testRelation2, HintInfo(strategy = Some(BROADCAST))), + Inner, None, JoinHint.NONE), caseSensitive = false) } test("do not traverse past existing broadcast hints") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("table"), - ResolvedHint(table("table").where('a > 1), HintInfo(broadcast = true))), - ResolvedHint(testRelation.where('a > 1), HintInfo(broadcast = true)).analyze, + ResolvedHint(table("table").where('a > 1), HintInfo(strategy = Some(BROADCAST)))), + ResolvedHint(testRelation.where('a > 1), HintInfo(strategy = Some(BROADCAST))).analyze, caseSensitive = false) } test("should work for subqueries") { checkAnalysis( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").as("tableAlias")), - ResolvedHint(testRelation, HintInfo(broadcast = true)), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) checkAnalysis( UnresolvedHint("MAPJOIN", Seq("tableAlias"), table("table").subquery('tableAlias)), - ResolvedHint(testRelation, HintInfo(broadcast = true)), + ResolvedHint(testRelation, HintInfo(strategy = Some(BROADCAST))), caseSensitive = false) // Negative case: if the alias doesn't match, don't match the original table name. @@ -105,7 +119,7 @@ class ResolveHintsSuite extends AnalysisTest { |SELECT /*+ BROADCAST(ctetable) */ * FROM ctetable """.stripMargin ), - ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(broadcast = true)) + ResolvedHint(testRelation.where('a > 1).select('a), HintInfo(strategy = Some(BROADCAST))) .select('a).analyze, caseSensitive = false) } @@ -155,4 +169,17 @@ class ResolveHintsSuite extends AnalysisTest { UnresolvedHint("REPARTITION", Seq(Literal(true)), table("TaBlE")), Seq(errMsgRepa)) } + + test("log warnings for invalid hints") { + val logAppender = new MockAppender() + withLogAppender(logAppender) { + checkAnalysis( + UnresolvedHint("unknown_hint", Seq("TaBlE"), table("TaBlE")), + testRelation, + caseSensitive = false) + } + assert(logAppender.loggingEvents.exists( + e => e.getLevel == Level.WARN && + e.getRenderedMessage.contains("Unrecognized hint: unknown_hint"))) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala index c4171c75ecd03..a5847ba7c522d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolveLambdaVariablesSuite.scala @@ -49,19 +49,21 @@ class ResolveLambdaVariablesSuite extends PlanTest { comparePlans(Analyzer.execute(plan(e1)), plan(e2)) } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("resolution - no op") { checkExpression(key, key) } test("resolution - simple") { - val in = ArrayTransform(values1, LambdaFunction('x.attr + 1, 'x.attr :: Nil)) + val in = ArrayTransform(values1, LambdaFunction(lv('x) + 1, lv('x) :: Nil)) val out = ArrayTransform(values1, LambdaFunction(lvInt + 1, lvInt :: Nil)) checkExpression(in, out) } test("resolution - nested") { val in = ArrayTransform(values2, LambdaFunction( - ArrayTransform('x.attr, LambdaFunction('x.attr + 1, 'x.attr :: Nil)), 'x.attr :: Nil)) + ArrayTransform(lv('x), LambdaFunction(lv('x) + 1, lv('x) :: Nil)), lv('x) :: Nil)) val out = ArrayTransform(values2, LambdaFunction( ArrayTransform(lvArray, LambdaFunction(lvInt + 1, lvInt :: Nil)), lvArray :: Nil)) checkExpression(in, out) @@ -75,14 +77,14 @@ class ResolveLambdaVariablesSuite extends PlanTest { test("fail - name collisions") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'X.attr, 'x.attr :: 'X.attr :: Nil))) + LambdaFunction(lv('x) + lv('X), lv('x) :: lv('X) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("arguments should not have names that are semantically the same")) } test("fail - lambda arguments") { val p = plan(ArrayTransform(values1, - LambdaFunction('x.attr + 'y.attr + 'z.attr, 'x.attr :: 'y.attr :: 'z.attr :: Nil))) + LambdaFunction(lv('x) + lv('y) + lv('z), lv('x) :: lv('y) :: lv('z) :: Nil))) val msg = intercept[AnalysisException](Analyzer.execute(p)).getMessage assert(msg.contains("does not match the number of arguments expected")) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala index fe57c199b8744..64bd07534b19b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/ResolvedUuidExpressionsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions._ @@ -34,6 +35,7 @@ class ResolvedUuidExpressionsSuite extends AnalysisTest { private lazy val uuid3 = Uuid().as('_uuid3) private lazy val uuid1Ref = uuid1.toAttribute + private val tracker = new QueryPlanningTracker private val analyzer = getAnalyzer(caseSensitive = true) private def getUuidExpressions(plan: LogicalPlan): Seq[Uuid] = { @@ -47,7 +49,7 @@ class ResolvedUuidExpressionsSuite extends AnalysisTest { test("analyzed plan sets random seed for Uuid expression") { val plan = r.select(a, uuid1) - val resolvedPlan = analyzer.executeAndCheck(plan) + val resolvedPlan = analyzer.executeAndCheck(plan, tracker) getUuidExpressions(resolvedPlan).foreach { u => assert(u.resolved) assert(u.randomSeed.isDefined) @@ -56,14 +58,14 @@ class ResolvedUuidExpressionsSuite extends AnalysisTest { test("Uuid expressions should have different random seeds") { val plan = r.select(a, uuid1).groupBy(uuid1Ref)(uuid2, uuid3) - val resolvedPlan = analyzer.executeAndCheck(plan) + val resolvedPlan = analyzer.executeAndCheck(plan, tracker) assert(getUuidExpressions(resolvedPlan).map(_.randomSeed.get).distinct.length == 3) } test("Different analyzed plans should have different random seeds in Uuids") { val plan = r.select(a, uuid1).groupBy(uuid1Ref)(uuid2, uuid3) - val resolvedPlan1 = analyzer.executeAndCheck(plan) - val resolvedPlan2 = analyzer.executeAndCheck(plan) + val resolvedPlan1 = analyzer.executeAndCheck(plan, tracker) + val resolvedPlan2 = analyzer.executeAndCheck(plan, tracker) val uuids1 = getUuidExpressions(resolvedPlan1) val uuids2 = getUuidExpressions(resolvedPlan2) assert(uuids1.distinct.length == 3) 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 0eba1c537d67d..949bb30d15503 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 @@ -742,7 +742,7 @@ class TypeCoercionSuite extends AnalysisTest { val nullLit = Literal.create(null, NullType) val floatNullLit = Literal.create(null, FloatType) val floatLit = Literal.create(1.0f, FloatType) - val timestampLit = Literal.create("2017-04-12", TimestampType) + val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) val decimalLit = Literal(new java.math.BigDecimal("1000000000000000000000")) val tsArrayLit = Literal(Array(new Timestamp(System.currentTimeMillis()))) val strArrayLit = Literal(Array("c")) @@ -793,11 +793,11 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, CreateArray(Literal(1.0) :: Literal(1) - :: Literal.create(1.0, FloatType) + :: Literal.create(1.0f, FloatType) :: Nil), CreateArray(Literal(1.0) :: Cast(Literal(1), DoubleType) - :: Cast(Literal.create(1.0, FloatType), DoubleType) + :: Cast(Literal.create(1.0f, FloatType), DoubleType) :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, @@ -834,23 +834,23 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal(1) :: Literal("a") - :: Literal.create(2.0, FloatType) + :: Literal.create(2.0f, FloatType) :: Literal("b") :: Nil), CreateMap(Cast(Literal(1), FloatType) :: Literal("a") - :: Literal.create(2.0, FloatType) + :: Literal.create(2.0f, FloatType) :: Literal("b") :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, CreateMap(Literal.create(null, DecimalType(5, 3)) :: Literal("a") - :: Literal.create(2.0, FloatType) + :: Literal.create(2.0f, FloatType) :: Literal("b") :: Nil), CreateMap(Literal.create(null, DecimalType(5, 3)).cast(DoubleType) :: Literal("a") - :: Literal.create(2.0, FloatType).cast(DoubleType) + :: Literal.create(2.0f, FloatType).cast(DoubleType) :: Literal("b") :: Nil)) // type coercion for map values @@ -895,11 +895,11 @@ class TypeCoercionSuite extends AnalysisTest { ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1.0) :: Literal(1) - :: Literal.create(1.0, FloatType) + :: Literal.create(1.0f, FloatType) :: Nil), operator(Literal(1.0) :: Cast(Literal(1), DoubleType) - :: Cast(Literal.create(1.0, FloatType), DoubleType) + :: Cast(Literal.create(1.0f, FloatType), DoubleType) :: Nil)) ruleTest(TypeCoercion.FunctionArgumentConversion, operator(Literal(1L) @@ -966,7 +966,7 @@ class TypeCoercionSuite extends AnalysisTest { val falseLit = Literal.create(false, BooleanType) val stringLit = Literal.create("c", StringType) val floatLit = Literal.create(1.0f, FloatType) - val timestampLit = Literal.create("2017-04-12", TimestampType) + val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) val decimalLit = Literal(new java.math.BigDecimal("1000000000000000000000")) ruleTest(rule, @@ -1016,14 +1016,16 @@ class TypeCoercionSuite extends AnalysisTest { CaseKeyWhen(Literal(true), Seq(Literal(1), Literal("a"))) ) ruleTest(TypeCoercion.CaseWhenCoercion, - CaseWhen(Seq((Literal(true), Literal(1.2))), Literal.create(1, DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Literal(1.2))), - Cast(Literal.create(1, DecimalType(7, 2)), DoubleType)) + Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Literal(1.2))), + Cast(Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2)), DoubleType)) ) ruleTest(TypeCoercion.CaseWhenCoercion, - CaseWhen(Seq((Literal(true), Literal(100L))), Literal.create(1, DecimalType(7, 2))), + CaseWhen(Seq((Literal(true), Literal(100L))), + Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2))), CaseWhen(Seq((Literal(true), Cast(Literal(100L), DecimalType(22, 2)))), - Cast(Literal.create(1, DecimalType(7, 2)), DecimalType(22, 2))) + Cast(Literal.create(BigDecimal.valueOf(1), DecimalType(7, 2)), DecimalType(22, 2))) ) } @@ -1124,14 +1126,14 @@ class TypeCoercionSuite extends AnalysisTest { Concat(Seq(Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.concatBinaryAsString" -> "true") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "true") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.concatBinaryAsString" -> "false") { + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "false") { ruleTest(rule, Concat(Seq(Literal("123".getBytes), Literal("456".getBytes))), Concat(Seq(Literal("123".getBytes), Literal("456".getBytes)))) @@ -1178,14 +1180,14 @@ class TypeCoercionSuite extends AnalysisTest { Elt(Seq(Literal(2), Cast(Literal(new java.sql.Date(0)), StringType), Cast(Literal(new Timestamp(0)), StringType)))) - withSQLConf("spark.sql.function.eltOutputAsString" -> "true") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "true") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Cast(Literal("123".getBytes), StringType), Cast(Literal("456".getBytes), StringType)))) } - withSQLConf("spark.sql.function.eltOutputAsString" -> "false") { + withSQLConf(SQLConf.ELT_OUTPUT_AS_STRING.key -> "false") { ruleTest(rule, Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes))), Elt(Seq(Literal(1), Literal("123".getBytes), Literal("456".getBytes)))) @@ -1404,6 +1406,7 @@ class TypeCoercionSuite extends AnalysisTest { val timestamp = Literal(new Timestamp(0L)) val interval = Literal(new CalendarInterval(0, 0)) val str = Literal("2015-01-01") + val intValue = Literal(0, IntegerType) ruleTest(dateTimeOperations, Add(date, interval), Cast(TimeAdd(date, interval), DateType)) ruleTest(dateTimeOperations, Add(interval, date), Cast(TimeAdd(date, interval), DateType)) @@ -1422,6 +1425,11 @@ class TypeCoercionSuite extends AnalysisTest { // interval operations should not be effected ruleTest(dateTimeOperations, Add(interval, interval), Add(interval, interval)) ruleTest(dateTimeOperations, Subtract(interval, interval), Subtract(interval, interval)) + + ruleTest(dateTimeOperations, Add(date, intValue), DateAdd(date, intValue)) + ruleTest(dateTimeOperations, Add(intValue, date), DateAdd(date, intValue)) + ruleTest(dateTimeOperations, Subtract(date, intValue), DateSub(date, intValue)) + ruleTest(dateTimeOperations, Subtract(date, date), DateDiff(date, date)) } /** @@ -1448,7 +1456,7 @@ class TypeCoercionSuite extends AnalysisTest { test("SPARK-15776 Divide expression's dataType should be casted to Double or Decimal " + "in aggregation function like sum") { - val rules = Seq(FunctionArgumentConversion, Division) + val rules = Seq(FunctionArgumentConversion, Division(conf)) // Casts Integer to Double ruleTest(rules, sum(Divide(4, 3)), sum(Divide(Cast(4, DoubleType), Cast(3, DoubleType)))) // Left expression is Double, right expression is Int. Another rule ImplicitTypeCasts will @@ -1467,12 +1475,35 @@ class TypeCoercionSuite extends AnalysisTest { } test("SPARK-17117 null type coercion in divide") { - val rules = Seq(FunctionArgumentConversion, Division, ImplicitTypeCasts) + val rules = Seq(FunctionArgumentConversion, Division(conf), ImplicitTypeCasts) val nullLit = Literal.create(null, NullType) ruleTest(rules, Divide(1L, nullLit), Divide(Cast(1L, DoubleType), Cast(nullLit, DoubleType))) ruleTest(rules, Divide(nullLit, 1L), Divide(Cast(nullLit, DoubleType), Cast(1L, DoubleType))) } + test("SPARK-28395 Division operator support integral division") { + val rules = Seq(FunctionArgumentConversion, Division(conf)) + Seq(true, false).foreach { preferIntegralDivision => + withSQLConf(SQLConf.PREFER_INTEGRAL_DIVISION.key -> s"$preferIntegralDivision") { + val result1 = if (preferIntegralDivision) { + IntegralDivide(1L, 1L) + } else { + Divide(Cast(1L, DoubleType), Cast(1L, DoubleType)) + } + ruleTest(rules, Divide(1L, 1L), result1) + val result2 = if (preferIntegralDivision) { + IntegralDivide(1, Cast(1, ShortType)) + } else { + Divide(Cast(1, DoubleType), Cast(Cast(1, ShortType), DoubleType)) + } + ruleTest(rules, Divide(1, Cast(1, ShortType)), result2) + + ruleTest(rules, Divide(1L, 1D), Divide(Cast(1L, DoubleType), Cast(1D, DoubleType))) + ruleTest(rules, Divide(Decimal(1.1), 1L), Divide(Decimal(1.1), 1L)) + } + } + } + test("binary comparison with string promotion") { val rule = TypeCoercion.PromoteStrings(conf) ruleTest(rule, @@ -1490,7 +1521,7 @@ class TypeCoercionSuite extends AnalysisTest { DoubleType))) Seq(true, false).foreach { convertToTS => withSQLConf( - "spark.sql.legacy.compareDateTimestampInTimestamp" -> convertToTS.toString) { + SQLConf.COMPARE_DATE_TIMESTAMP_IN_TIMESTAMP.key -> convertToTS.toString) { val date0301 = Literal(java.sql.Date.valueOf("2017-03-01")) val timestamp0301000000 = Literal(Timestamp.valueOf("2017-03-01 00:00:00")) val timestamp0301000001 = Literal(Timestamp.valueOf("2017-03-01 00:00:01")) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala index 28a164b5d0cad..0fe646edb340e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/UnsupportedOperationsSuite.scala @@ -404,7 +404,7 @@ class UnsupportedOperationsSuite extends SparkFunSuite { _.join(_, joinType = Inner), outputMode = Update, streamStreamSupported = false, - expectedMsg = "inner join") + expectedMsg = "is not supported in Update output mode") // Full outer joins: only batch-batch is allowed testBinaryOperationInStreamingPlan( @@ -422,6 +422,20 @@ class UnsupportedOperationsSuite extends SparkFunSuite { streamStreamSupported = false, expectedMsg = "outer join") + // Left outer joins: update and complete mode not allowed + assertNotSupportedInStreamingPlan( + s"left outer join with stream-stream relations and update mode", + streamRelation.join(streamRelation, joinType = LeftOuter, + condition = Some(attribute === attribute)), + OutputMode.Update(), + Seq("is not supported in Update output mode")) + assertNotSupportedInStreamingPlan( + s"left outer join with stream-stream relations and complete mode", + Aggregate(Nil, aggExprs("d"), streamRelation.join(streamRelation, joinType = LeftOuter, + condition = Some(attribute === attribute))), + OutputMode.Complete(), + Seq("is not supported in Complete output mode")) + // Left outer joins: stream-stream allowed with join on watermark attribute // Note that the attribute need not be watermarked on both sides. assertSupportedInStreamingPlan( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala index 2fcaeca34db3f..366188c3327be 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogEventSuite.scala @@ -38,11 +38,7 @@ class ExternalCatalogEventSuite extends SparkFunSuite { f: (ExternalCatalog, Seq[ExternalCatalogEvent] => Unit) => Unit): Unit = test(name) { val catalog = new ExternalCatalogWithListener(newCatalog) val recorder = mutable.Buffer.empty[ExternalCatalogEvent] - catalog.addListener(new ExternalCatalogEventListener { - override def onEvent(event: ExternalCatalogEvent): Unit = { - recorder += event - } - }) + catalog.addListener((event: ExternalCatalogEvent) => recorder += event) f(catalog, (expected: Seq[ExternalCatalogEvent]) => { val actual = recorder.clone() recorder.clear() @@ -174,9 +170,6 @@ class ExternalCatalogEventSuite extends SparkFunSuite { className = "", resources = Seq.empty) - val newIdentifier = functionDefinition.identifier.copy(funcName = "fn4") - val renamedFunctionDefinition = functionDefinition.copy(identifier = newIdentifier) - catalog.createDatabase(dbDefinition, ignoreIfExists = false) checkEvents(CreateDatabasePreEvent("db5") :: CreateDatabaseEvent("db5") :: Nil) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala index b376108399c1c..6b1c35094e4a4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala @@ -277,6 +277,28 @@ abstract class ExternalCatalogSuite extends SparkFunSuite with BeforeAndAfterEac } } + test("get tables by name") { + assert(newBasicCatalog().getTablesByName("db2", Seq("tbl1", "tbl2")) + .map(_.identifier.table) == Seq("tbl1", "tbl2")) + } + + test("get tables by name when some tables do not exists") { + assert(newBasicCatalog().getTablesByName("db2", Seq("tbl1", "tblnotexist")) + .map(_.identifier.table) == Seq("tbl1")) + } + + test("get tables by name when contains invalid name") { + // scalastyle:off + val name = "砖" + // scalastyle:on + assert(newBasicCatalog().getTablesByName("db2", Seq("tbl1", name)) + .map(_.identifier.table) == Seq("tbl1")) + } + + test("get tables by name when empty table list") { + assert(newBasicCatalog().getTablesByName("db2", Seq.empty).isEmpty) + } + test("list tables without pattern") { val catalog = newBasicCatalog() intercept[AnalysisException] { catalog.listTables("unknown_db") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala index 19e8c0334689c..bce85534ce7e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala @@ -509,6 +509,96 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } + test("get tables by name") { + withBasicCatalog { catalog => + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1", Some("db2")), + TableIdentifier("tbl2", Some("db2")) + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1", "tbl2"))) + // Get table without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1"), + TableIdentifier("tbl2") + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1", "tbl2"))) + } + } + + test("get tables by name when some tables do not exist") { + withBasicCatalog { catalog => + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1", Some("db2")), + TableIdentifier("tblnotexit", Some("db2")) + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1"))) + // Get table without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1"), + TableIdentifier("tblnotexit") + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1"))) + } + } + + test("get tables by name when contains invalid name") { + // scalastyle:off + val name = "砖" + // scalastyle:on + withBasicCatalog { catalog => + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1", Some("db2")), + TableIdentifier(name, Some("db2")) + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1"))) + // Get table without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.getTablesByName( + Seq( + TableIdentifier("tbl1"), + TableIdentifier(name) + ) + ) == catalog.externalCatalog.getTablesByName("db2", Seq("tbl1"))) + } + } + + test("get tables by name when empty") { + withBasicCatalog { catalog => + assert(catalog.getTablesByName(Seq.empty) + == catalog.externalCatalog.getTablesByName("db2", Seq.empty)) + // Get table without explicitly specifying database + catalog.setCurrentDatabase("db2") + assert(catalog.getTablesByName(Seq.empty) + == catalog.externalCatalog.getTablesByName("db2", Seq.empty)) + } + } + + test("get tables by name when tables belong to different databases") { + withBasicCatalog { catalog => + intercept[AnalysisException](catalog.getTablesByName( + Seq( + TableIdentifier("tbl1", Some("db1")), + TableIdentifier("tbl2", Some("db2")) + ) + )) + // Get table without explicitly specifying database + catalog.setCurrentDatabase("db2") + intercept[AnalysisException](catalog.getTablesByName( + Seq( + TableIdentifier("tbl1", Some("db1")), + TableIdentifier("tbl2") + ) + )) + } + } + test("lookup table relation") { withBasicCatalog { catalog => val tempTable1 = Range(1, 10, 1, 10) @@ -627,6 +717,71 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } + test("list tables with pattern and includeLocalTempViews") { + withEmptyCatalog { catalog => + catalog.createDatabase(newDb("mydb"), ignoreIfExists = false) + catalog.createTable(newTable("tbl1", "mydb"), ignoreIfExists = false) + catalog.createTable(newTable("tbl2", "mydb"), ignoreIfExists = false) + val tempTable = Range(1, 10, 2, 10) + catalog.createTempView("temp_view1", tempTable, overrideIfExists = false) + catalog.createTempView("temp_view4", tempTable, overrideIfExists = false) + + assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "*").toSet) + assert(catalog.listTables("mydb").toSet == catalog.listTables("mydb", "*", true).toSet) + assert(catalog.listTables("mydb").toSet == + catalog.listTables("mydb", "*", false).toSet ++ catalog.listLocalTempViews("*")) + assert(catalog.listTables("mydb", "*", true).toSet == + Set(TableIdentifier("tbl1", Some("mydb")), + TableIdentifier("tbl2", Some("mydb")), + TableIdentifier("temp_view1"), + TableIdentifier("temp_view4"))) + assert(catalog.listTables("mydb", "*", false).toSet == + Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb")))) + assert(catalog.listTables("mydb", "tbl*", true).toSet == + Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb")))) + assert(catalog.listTables("mydb", "tbl*", false).toSet == + Set(TableIdentifier("tbl1", Some("mydb")), TableIdentifier("tbl2", Some("mydb")))) + assert(catalog.listTables("mydb", "temp_view*", true).toSet == + Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4"))) + assert(catalog.listTables("mydb", "temp_view*", false).toSet == Set.empty) + } + } + + test("list temporary view with pattern") { + withBasicCatalog { catalog => + val tempTable = Range(1, 10, 2, 10) + catalog.createTempView("temp_view1", tempTable, overrideIfExists = false) + catalog.createTempView("temp_view4", tempTable, overrideIfExists = false) + assert(catalog.listLocalTempViews("*").toSet == + Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4"))) + assert(catalog.listLocalTempViews("temp_view*").toSet == + Set(TableIdentifier("temp_view1"), TableIdentifier("temp_view4"))) + assert(catalog.listLocalTempViews("*1").toSet == Set(TableIdentifier("temp_view1"))) + assert(catalog.listLocalTempViews("does_not_exist").toSet == Set.empty) + } + } + + test("list global temporary view and local temporary view with pattern") { + withBasicCatalog { catalog => + val tempTable = Range(1, 10, 2, 10) + catalog.createTempView("temp_view1", tempTable, overrideIfExists = false) + catalog.createTempView("temp_view4", tempTable, overrideIfExists = false) + catalog.globalTempViewManager.create("global_temp_view1", tempTable, overrideIfExists = false) + catalog.globalTempViewManager.create("global_temp_view2", tempTable, overrideIfExists = false) + assert(catalog.listTables(catalog.globalTempViewManager.database, "*").toSet == + Set(TableIdentifier("temp_view1"), + TableIdentifier("temp_view4"), + TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database)), + TableIdentifier("global_temp_view2", Some(catalog.globalTempViewManager.database)))) + assert(catalog.listTables(catalog.globalTempViewManager.database, "*temp_view1").toSet == + Set(TableIdentifier("temp_view1"), + TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database)))) + assert(catalog.listTables(catalog.globalTempViewManager.database, "global*").toSet == + Set(TableIdentifier("global_temp_view1", Some(catalog.globalTempViewManager.database)), + TableIdentifier("global_temp_view2", Some(catalog.globalTempViewManager.database)))) + } + } + // -------------------------------------------------------------------------- // Partitions // -------------------------------------------------------------------------- @@ -1448,4 +1603,18 @@ abstract class SessionCatalogSuite extends AnalysisTest { } } } + + test("SPARK-24544: test print actual failure cause when look up function failed") { + withBasicCatalog { catalog => + val cause = intercept[NoSuchFunctionException] { + catalog.failFunctionLookup(FunctionIdentifier("failureFunc"), + Some(new Exception("Actual error"))) + } + + // fullStackTrace will be printed, but `cause.getMessage` has been + // override in `AnalysisException`,so here we get the root cause + // exception message for check. + assert(cause.cause.get.getMessage.contains("Actual error")) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala new file mode 100644 index 0000000000000..52543d16d4815 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/v2/LookupCatalogSuite.scala @@ -0,0 +1,195 @@ +/* + * 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.catalog.v2 + +import org.scalatest.Inside +import org.scalatest.Matchers._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +private case class TestCatalogPlugin(override val name: String) extends CatalogPlugin { + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = Unit +} + +class LookupCatalogSuite extends SparkFunSuite with LookupCatalog with Inside { + import CatalystSqlParser._ + + private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap + + override def lookupCatalog(name: String): CatalogPlugin = + catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + + test("catalog object identifier") { + Seq( + ("tbl", None, Seq.empty, "tbl"), + ("db.tbl", None, Seq("db"), "tbl"), + ("prod.func", catalogs.get("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", None, Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs.get("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs.get("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", None, Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", None, Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", None, + Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { + case (sql, expectedCatalog, namespace, name) => + inside(parseMultipartIdentifier(sql)) { + case CatalogObjectIdentifier(catalog, ident) => + catalog shouldEqual expectedCatalog + ident shouldEqual Identifier.of(namespace.toArray, name) + } + } + } + + test("table identifier") { + Seq( + ("tbl", "tbl", None), + ("db.tbl", "tbl", Some("db")), + ("`db.tbl`", "db.tbl", None), + ("parquet.`file:/tmp/db.tbl`", "file:/tmp/db.tbl", Some("parquet")), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", "s3://buck/tmp/abc.json", + Some("org.apache.spark.sql.json"))).foreach { + case (sql, table, db) => + inside (parseMultipartIdentifier(sql)) { + case AsTableIdentifier(ident) => + ident shouldEqual TableIdentifier(table, db) + } + } + Seq( + "prod.func", + "prod.db.tbl", + "ns1.ns2.tbl").foreach { sql => + parseMultipartIdentifier(sql) match { + case AsTableIdentifier(_) => + fail(s"$sql should not be resolved as TableIdentifier") + case _ => + } + } + } + + test("temporary table identifier") { + Seq( + ("tbl", TableIdentifier("tbl")), + ("db.tbl", TableIdentifier("tbl", Some("db"))), + ("`db.tbl`", TableIdentifier("db.tbl")), + ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { + case (sqlIdent: String, expectedTableIdent: TableIdentifier) => + // when there is no catalog and the namespace has one part, the rule should match + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(ident) => + ident shouldEqual expectedTableIdent + } + } + + Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") + .foreach { sqlIdent => + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(_) => + fail("AsTemporaryViewIdentifier should not match when " + + "the catalog is set or the namespace has multiple parts") + case _ => + // expected + } + } + } +} + +class LookupCatalogWithDefaultSuite extends SparkFunSuite with LookupCatalog with Inside { + import CatalystSqlParser._ + + private val catalogs = Seq("prod", "test").map(x => x -> new TestCatalogPlugin(x)).toMap + + override def defaultCatalogName: Option[String] = Some("prod") + + override def lookupCatalog(name: String): CatalogPlugin = + catalogs.getOrElse(name, throw new CatalogNotFoundException(s"$name not found")) + + test("catalog object identifier") { + Seq( + ("tbl", catalogs.get("prod"), Seq.empty, "tbl"), + ("db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), + ("prod.func", catalogs.get("prod"), Seq.empty, "func"), + ("ns1.ns2.tbl", catalogs.get("prod"), Seq("ns1", "ns2"), "tbl"), + ("prod.db.tbl", catalogs.get("prod"), Seq("db"), "tbl"), + ("test.db.tbl", catalogs.get("test"), Seq("db"), "tbl"), + ("test.ns1.ns2.ns3.tbl", catalogs.get("test"), Seq("ns1", "ns2", "ns3"), "tbl"), + ("`db.tbl`", catalogs.get("prod"), Seq.empty, "db.tbl"), + ("parquet.`file:/tmp/db.tbl`", catalogs.get("prod"), Seq("parquet"), "file:/tmp/db.tbl"), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", catalogs.get("prod"), + Seq("org.apache.spark.sql.json"), "s3://buck/tmp/abc.json")).foreach { + case (sql, expectedCatalog, namespace, name) => + inside(parseMultipartIdentifier(sql)) { + case CatalogObjectIdentifier(catalog, ident) => + catalog shouldEqual expectedCatalog + ident shouldEqual Identifier.of(namespace.toArray, name) + } + } + } + + test("table identifier") { + Seq( + "tbl", + "db.tbl", + "`db.tbl`", + "parquet.`file:/tmp/db.tbl`", + "`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + "prod.func", + "prod.db.tbl", + "ns1.ns2.tbl").foreach { sql => + parseMultipartIdentifier(sql) match { + case AsTableIdentifier(_) => + fail(s"$sql should not be resolved as TableIdentifier") + case _ => + } + } + } + + test("temporary table identifier") { + Seq( + ("tbl", TableIdentifier("tbl")), + ("db.tbl", TableIdentifier("tbl", Some("db"))), + ("`db.tbl`", TableIdentifier("db.tbl")), + ("parquet.`file:/tmp/db.tbl`", TableIdentifier("file:/tmp/db.tbl", Some("parquet"))), + ("`org.apache.spark.sql.json`.`s3://buck/tmp/abc.json`", + TableIdentifier("s3://buck/tmp/abc.json", Some("org.apache.spark.sql.json")))).foreach { + case (sqlIdent: String, expectedTableIdent: TableIdentifier) => + // when there is no catalog and the namespace has one part, the rule should match + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(ident) => + ident shouldEqual expectedTableIdent + } + } + + Seq("prod.func", "prod.db.tbl", "test.db.tbl", "ns1.ns2.tbl", "test.ns1.ns2.ns3.tbl") + .foreach { sqlIdent => + inside(parseMultipartIdentifier(sqlIdent)) { + case AsTemporaryViewIdentifier(_) => + fail("AsTemporaryViewIdentifier should not match when " + + "the catalog is set or the namespace has multiple parts") + case _ => + // expected + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala similarity index 55% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala index 221e44ce2cff6..838ac42184fa5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVExprUtilsSuite.scala @@ -15,33 +15,47 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.datasources.csv +package org.apache.spark.sql.catalyst.csv import org.apache.spark.SparkFunSuite -class CSVUtilsSuite extends SparkFunSuite { +class CSVExprUtilsSuite extends SparkFunSuite { test("Can parse escaped characters") { - assert(CSVUtils.toChar("""\t""") === '\t') - assert(CSVUtils.toChar("""\r""") === '\r') - assert(CSVUtils.toChar("""\b""") === '\b') - assert(CSVUtils.toChar("""\f""") === '\f') - assert(CSVUtils.toChar("""\"""") === '\"') - assert(CSVUtils.toChar("""\'""") === '\'') - assert(CSVUtils.toChar("""\u0000""") === '\u0000') + assert(CSVExprUtils.toChar("""\t""") === '\t') + assert(CSVExprUtils.toChar("""\r""") === '\r') + assert(CSVExprUtils.toChar("""\b""") === '\b') + assert(CSVExprUtils.toChar("""\f""") === '\f') + assert(CSVExprUtils.toChar("""\"""") === '\"') + assert(CSVExprUtils.toChar("""\'""") === '\'') + assert(CSVExprUtils.toChar("""\u0000""") === '\u0000') + assert(CSVExprUtils.toChar("""\\""") === '\\') } test("Does not accept delimiter larger than one character") { val exception = intercept[IllegalArgumentException]{ - CSVUtils.toChar("ab") + CSVExprUtils.toChar("ab") } assert(exception.getMessage.contains("cannot be more than one character")) } test("Throws exception for unsupported escaped characters") { val exception = intercept[IllegalArgumentException]{ - CSVUtils.toChar("""\1""") + CSVExprUtils.toChar("""\1""") } assert(exception.getMessage.contains("Unsupported special character for delimiter")) } + test("string with one backward slash is prohibited") { + val exception = intercept[IllegalArgumentException]{ + CSVExprUtils.toChar("""\""") + } + assert(exception.getMessage.contains("Single backslash is prohibited")) + } + + test("output proper error message for empty string") { + val exception = intercept[IllegalArgumentException]{ + CSVExprUtils.toChar("") + } + assert(exception.getMessage.contains("Delimiter cannot be empty string")) + } } 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 new file mode 100644 index 0000000000000..24d909ed99b93 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/CSVInferSchemaSuite.scala @@ -0,0 +1,192 @@ +/* + * 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.csv + +import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.util.Locale + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +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 inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(NullType, "") == NullType) + assert(inferSchema.inferField(NullType, null) == NullType) + assert(inferSchema.inferField(NullType, "100000000000") == LongType) + assert(inferSchema.inferField(NullType, "60") == IntegerType) + assert(inferSchema.inferField(NullType, "3.5") == DoubleType) + assert(inferSchema.inferField(NullType, "test") == StringType) + assert(inferSchema.inferField(NullType, "2015-08-20 15:57:00") == TimestampType) + assert(inferSchema.inferField(NullType, "True") == BooleanType) + assert(inferSchema.inferField(NullType, "FAlSE") == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(inferSchema.inferField(NullType, textValueOne) == expectedTypeOne) + } + + 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 inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(LongType, "1.0") == DoubleType) + assert(inferSchema.inferField(LongType, "test") == StringType) + assert(inferSchema.inferField(IntegerType, "1.0") == DoubleType) + assert(inferSchema.inferField(DoubleType, null) == DoubleType) + assert(inferSchema.inferField(DoubleType, "test") == StringType) + assert(inferSchema.inferField(LongType, "2015-08-20 14:57:00") == TimestampType) + assert(inferSchema.inferField(DoubleType, "2015-08-20 15:57:00") == TimestampType) + assert(inferSchema.inferField(LongType, "True") == BooleanType) + assert(inferSchema.inferField(IntegerType, "FALSE") == BooleanType) + assert(inferSchema.inferField(TimestampType, "FALSE") == BooleanType) + + val textValueOne = Long.MaxValue.toString + "0" + val decimalValueOne = new java.math.BigDecimal(textValueOne) + val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) + assert(inferSchema.inferField(IntegerType, textValueOne) == expectedTypeOne) + } + + test("Timestamp field types are inferred correctly via custom data format") { + var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "GMT") + var inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(TimestampType, "2015-08") == TimestampType) + + options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "GMT") + 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 inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(IntegerType, "2015-08-20 14") == StringType) + assert(inferSchema.inferField(DoubleType, "2015-08-20 14:10") == StringType) + assert(inferSchema.inferField(LongType, "2015-08 14:49:00") == StringType) + } + + test("Boolean fields types are inferred correctly from other types") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(LongType, "Fale") == StringType) + assert(inferSchema.inferField(DoubleType, "TRUEe") == StringType) + } + + test("Type arrays are merged to highest common type") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) + + assert( + inferSchema.mergeRowTypes(Array(StringType), + Array(DoubleType)).deep == Array(StringType).deep) + assert( + inferSchema.mergeRowTypes(Array(IntegerType), + Array(LongType)).deep == Array(LongType).deep) + assert( + inferSchema.mergeRowTypes(Array(DoubleType), + Array(LongType)).deep == Array(DoubleType).deep) + } + + test("Null fields are handled properly when a nullValue is specified") { + var options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") + 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") + inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(IntegerType, "\\N") == IntegerType) + assert(inferSchema.inferField(DoubleType, "\\N") == DoubleType) + assert(inferSchema.inferField(TimestampType, "\\N") == TimestampType) + assert(inferSchema.inferField(BooleanType, "\\N") == BooleanType) + assert(inferSchema.inferField(DecimalType(1, 1), "\\N") == DecimalType(1, 1)) + } + + test("Merging Nulltypes should yield Nulltype.") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val inferSchema = new CSVInferSchema(options) + + val mergedNullTypes = inferSchema.mergeRowTypes(Array(NullType), Array(NullType)) + assert(mergedNullTypes.deep == Array(NullType).deep) + } + + test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { + val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "GMT") + 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 inferSchema = new CSVInferSchema(options) + + // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). + assert(inferSchema.inferField(DecimalType(3, -10), "1.19E11") == + DecimalType(4, -9)) + + // BigDecimal("12345678901234567890.01234567890123456789") is precision 40 and scale 20. + val value = "12345678901234567890.01234567890123456789" + assert(inferSchema.inferField(DecimalType(3, -10), value) == DoubleType) + + // Seq(s"${Long.MaxValue}1", "2015-12-01 00:00:00") should be StringType + assert(inferSchema.inferField(NullType, s"${Long.MaxValue}1") == DecimalType(20, 0)) + assert(inferSchema.inferField(DecimalType(20, 0), "2015-12-01 00:00:00") + == StringType) + } + + 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") + val inferSchema = new CSVInferSchema(options) + + assert(inferSchema.inferField(NullType, "nan") == DoubleType) + assert(inferSchema.inferField(NullType, "inf") == DoubleType) + assert(inferSchema.inferField(NullType, "-inf") == DoubleType) + } + + test("inferring the decimal type using locale") { + def checkDecimalInfer(langTag: String, expectedType: DataType): Unit = { + val options = new CSVOptions( + parameters = Map("locale" -> langTag, "inferSchema" -> "true", "sep" -> "|"), + columnPruning = false, + defaultTimeZoneId = "GMT") + val inferSchema = new CSVInferSchema(options) + + val df = new DecimalFormat("", new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) + val input = df.format(Decimal(1000001).toBigDecimal) + + assert(inferSchema.inferField(NullType, input) == expectedType) + } + + // input like '1,0' is inferred as strings for backward compatibility. + Seq("en-US").foreach(checkDecimalInfer(_, StringType)) + Seq("ko-KR", "ru-RU", "de-DE").foreach(checkDecimalInfer(_, DecimalType(7, 0))) + } +} 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 new file mode 100644 index 0000000000000..933c5764d77d7 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/csv/UnivocityParserSuite.scala @@ -0,0 +1,269 @@ +/* + * 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.csv + +import java.math.BigDecimal +import java.text.{DecimalFormat, DecimalFormatSymbols} +import java.util.{Locale, TimeZone} + +import org.apache.commons.lang3.time.FastDateFormat + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String + +class UnivocityParserSuite extends SparkFunSuite with SQLHelper { + private def assertNull(v: Any) = assert(v == null) + + test("Can parse decimal type values") { + val stringValues = Seq("10.05", "1,000.01", "158,058,049.001") + val decimalValues = Seq(10.05, 1000.01, 158058049.001) + val decimalType = new DecimalType() + + stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => + val decimalValue = new BigDecimal(decimalVal.toString) + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + assert(parser.makeConverter("_1", decimalType).apply(strVal) === + Decimal(decimalValue, decimalType.precision, decimalType.scale)) + } + } + + test("Nullable types are handled") { + val types = Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + BooleanType, DecimalType.DoubleDecimal, TimestampType, DateType, StringType) + + // Nullable field with nullValue option. + types.foreach { t => + // Tests that a custom nullValue. + val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") + 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") + parser = new UnivocityParser(StructType(Seq.empty), options) + assertNull(parser.makeConverter("_1", t, nullable = true).apply("")) + } + + // Not nullable field with nullValue option. + types.foreach { t => + // Casts a null to not nullable field should throw an exception. + val options = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + val converter = parser.makeConverter("_1", t, nullable = false) + var message = intercept[RuntimeException] { + converter.apply("-") + }.getMessage + assert(message.contains("null value found but field _1 is not nullable.")) + message = intercept[RuntimeException] { + converter.apply(null) + }.getMessage + assert(message.contains("null value found but field _1 is not nullable.")) + } + + // 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 parser = new UnivocityParser(StructType(Seq.empty), options) + val converter = parser.makeConverter("_1", StringType, nullable = b) + assert(converter.apply("") == UTF8String.fromString("")) + } + } + + test("Throws exception for empty string with non null type") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + val exception = intercept[RuntimeException]{ + parser.makeConverter("_1", IntegerType, nullable = false).apply("") + } + assert(exception.getMessage.contains("null value found but field _1 is not nullable.")) + } + + test("Types are cast correctly") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + var parser = new UnivocityParser(StructType(Seq.empty), options) + assert(parser.makeConverter("_1", ByteType).apply("10") == 10) + assert(parser.makeConverter("_1", ShortType).apply("10") == 10) + assert(parser.makeConverter("_1", IntegerType).apply("10") == 10) + assert(parser.makeConverter("_1", LongType).apply("10") == 10) + assert(parser.makeConverter("_1", FloatType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", DoubleType).apply("1.00") == 1.0) + assert(parser.makeConverter("_1", BooleanType).apply("true") == true) + + var timestampsOptions = + new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy HH:mm"), false, "GMT") + parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) + val customTimestamp = "31/01/2015 00:00" + var format = FastDateFormat.getInstance( + timestampsOptions.timestampFormat, + TimeZone.getTimeZone(timestampsOptions.zoneId), + timestampsOptions.locale) + val expectedTime = format.parse(customTimestamp).getTime + val castedTimestamp = parser.makeConverter("_1", TimestampType, nullable = true) + .apply(customTimestamp) + assert(castedTimestamp == expectedTime * 1000L) + + val customDate = "31/01/2015" + val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "GMT") + parser = new UnivocityParser(StructType(Seq.empty), dateOptions) + format = FastDateFormat.getInstance( + dateOptions.dateFormat, + TimeZone.getTimeZone(dateOptions.zoneId), + dateOptions.locale) + val expectedDate = format.parse(customDate).getTime + val castedDate = parser.makeConverter("_1", DateType, nullable = true) + .apply(customDate) + assert(castedDate == DateTimeUtils.millisToDays(expectedDate, TimeZone.getTimeZone("GMT"))) + + val timestamp = "2015-01-01 00:00:00" + timestampsOptions = new CSVOptions(Map( + "timestampFormat" -> "yyyy-MM-dd HH:mm:ss", + "dateFormat" -> "yyyy-MM-dd"), false, "UTC") + parser = new UnivocityParser(StructType(Seq.empty), timestampsOptions) + val expected = 1420070400 * DateTimeUtils.MICROS_PER_SECOND + assert(parser.makeConverter("_1", TimestampType).apply(timestamp) == + expected) + assert(parser.makeConverter("_1", DateType).apply("2015-01-01") == + expected / DateTimeUtils.MICROS_PER_DAY) + } + + test("Throws exception for casting an invalid string to Float and Double Types") { + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + val types = Seq(DoubleType, FloatType) + val input = Seq("10u000", "abc", "1 2/3") + types.foreach { dt => + input.foreach { v => + val message = intercept[NumberFormatException] { + parser.makeConverter("_1", dt).apply(v) + }.getMessage + assert(message.contains(v)) + } + } + } + + test("Float NaN values are parsed correctly") { + val options = new CSVOptions(Map("nanValue" -> "nn"), false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + val floatVal: Float = parser.makeConverter( + "_1", FloatType, nullable = true).apply("nn").asInstanceOf[Float] + + // Java implements the IEEE-754 floating point standard which guarantees that any comparison + // against NaN will return false (except != which returns true) + assert(floatVal != floatVal) + } + + test("Double NaN values are parsed correctly") { + val options = new CSVOptions(Map("nanValue" -> "-"), false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + val doubleVal: Double = parser.makeConverter( + "_1", DoubleType, nullable = true).apply("-").asInstanceOf[Double] + + assert(doubleVal.isNaN) + } + + test("Float infinite values can be parsed") { + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + 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") + parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) + val floatVal2 = parser.makeConverter( + "_1", FloatType, nullable = true).apply("max").asInstanceOf[Float] + + assert(floatVal2 == Float.PositiveInfinity) + } + + test("Double infinite values can be parsed") { + val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") + 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") + parser = new UnivocityParser(StructType(Seq.empty), positiveInfOptions) + val doubleVal2 = parser.makeConverter( + "_1", DoubleType, nullable = true).apply("max").asInstanceOf[Double] + + assert(doubleVal2 == Double.PositiveInfinity) + } + + test("parse decimals using locale") { + def checkDecimalParsing(langTag: String): Unit = { + val decimalVal = new BigDecimal("1000.001") + val decimalType = new DecimalType(10, 5) + val expected = Decimal(decimalVal, decimalType.precision, decimalType.scale) + 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 parser = new UnivocityParser(new StructType().add("d", decimalType), options) + + assert(parser.makeConverter("_1", decimalType).apply(input) === expected) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalParsing) + } + + test("SPARK-27591 UserDefinedType can be read") { + + @SQLUserDefinedType(udt = classOf[StringBasedUDT]) + case class NameId(name: String, id: Int) + + class StringBasedUDT extends UserDefinedType[NameId] { + override def sqlType: DataType = StringType + + override def serialize(obj: NameId): Any = s"${obj.name}\t${obj.id}" + + override def deserialize(datum: Any): NameId = datum match { + case s: String => + val split = s.split("\t") + if (split.length != 2) throw new RuntimeException(s"Can't parse $s into NameId"); + NameId(split(0), Integer.parseInt(split(1))) + case _ => throw new RuntimeException(s"Can't parse $datum into NameId"); + } + + override def userClass: Class[NameId] = classOf[NameId] + } + + object StringBasedUDT extends StringBasedUDT + + val input = "name\t42" + val expected = UTF8String.fromString(input) + + val options = new CSVOptions(Map.empty[String, String], false, "GMT") + val parser = new UnivocityParser(StructType(Seq.empty), options) + + val convertedValue = parser.makeConverter("_1", StringBasedUDT, nullable = false).apply(input) + + assert(convertedValue.isInstanceOf[UTF8String]) + assert(convertedValue == expected) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala index dd20e6497fbb4..da1b695919dec 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala @@ -80,7 +80,7 @@ class EncoderResolutionSuite extends PlanTest { val attrs = Seq('arr.array(StringType)) assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message == s""" - |Cannot up cast array element from string to bigint as it may truncate + |Cannot up cast array element from string to bigint. |The type path of the target object is: |- array element class: "scala.Long" |- field (class: "scala.Array", name: "arr") @@ -202,7 +202,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg1 == s""" - |Cannot up cast `b` from bigint to int as it may truncate + |Cannot up cast `b` from bigint to int. |The type path of the target object is: |- field (class: "scala.Int", name: "b") |- root class: "org.apache.spark.sql.catalyst.encoders.StringIntClass" @@ -215,7 +215,7 @@ class EncoderResolutionSuite extends PlanTest { }.message assert(msg2 == s""" - |Cannot up cast `b`.`b` from decimal(38,18) to bigint as it may truncate + |Cannot up cast `b`.`b` from decimal(38,18) to bigint. |The type path of the target object is: |- field (class: "scala.Long", name: "b") |- field (class: "org.apache.spark.sql.catalyst.encoders.StringLongClass", name: "b") 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 f0d61de97ffcd..f4feeca1d05ad 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 @@ -28,10 +28,11 @@ import org.apache.spark.sql.{Encoder, Encoders} import org.apache.spark.sql.catalyst.{OptionalData, PrimitiveData} import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference} +import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +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.util.ClosureCleaner @@ -128,13 +129,13 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes encodeDecodeTest(-3.7f, "primitive float") encodeDecodeTest(-3.7, "primitive double") - encodeDecodeTest(new java.lang.Boolean(false), "boxed boolean") - encodeDecodeTest(new java.lang.Byte(-3.toByte), "boxed byte") - encodeDecodeTest(new java.lang.Short(-3.toShort), "boxed short") - encodeDecodeTest(new java.lang.Integer(-3), "boxed int") - encodeDecodeTest(new java.lang.Long(-3L), "boxed long") - encodeDecodeTest(new java.lang.Float(-3.7f), "boxed float") - encodeDecodeTest(new java.lang.Double(-3.7), "boxed double") + encodeDecodeTest(java.lang.Boolean.FALSE, "boxed boolean") + encodeDecodeTest(java.lang.Byte.valueOf(-3: Byte), "boxed byte") + encodeDecodeTest(java.lang.Short.valueOf(-3: Short), "boxed short") + encodeDecodeTest(java.lang.Integer.valueOf(-3), "boxed int") + encodeDecodeTest(java.lang.Long.valueOf(-3L), "boxed long") + encodeDecodeTest(java.lang.Float.valueOf(-3.7f), "boxed float") + encodeDecodeTest(java.lang.Double.valueOf(-3.7), "boxed double") encodeDecodeTest(BigDecimal("32131413.211321313"), "scala decimal") encodeDecodeTest(new java.math.BigDecimal("231341.23123"), "java decimal") @@ -224,7 +225,7 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes productTest( RepeatedData( Seq(1, 2), - Seq(new Integer(1), null, new Integer(2)), + Seq(Integer.valueOf(1), null, Integer.valueOf(2)), Map(1 -> 2L), Map(1 -> null), PrimitiveData(1, 1, 1, 1, 1, 1, true))) @@ -329,8 +330,8 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes { val schema = ExpressionEncoder[(Int, (String, Int))].schema assert(schema(0).nullable === false) - assert(schema(1).nullable === true) - assert(schema(1).dataType.asInstanceOf[StructType](0).nullable === true) + assert(schema(1).nullable) + assert(schema(1).dataType.asInstanceOf[StructType](0).nullable) assert(schema(1).dataType.asInstanceOf[StructType](1).nullable === false) } @@ -340,15 +341,15 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes ExpressionEncoder[Int], ExpressionEncoder[(String, Int)]).schema assert(schema(0).nullable === false) - assert(schema(1).nullable === true) - assert(schema(1).dataType.asInstanceOf[StructType](0).nullable === true) + assert(schema(1).nullable) + assert(schema(1).dataType.asInstanceOf[StructType](0).nullable) assert(schema(1).dataType.asInstanceOf[StructType](1).nullable === false) } } test("nullable of encoder serializer") { def checkNullable[T: Encoder](nullable: Boolean): Unit = { - assert(encoderFor[T].serializer.forall(_.nullable === nullable)) + assert(encoderFor[T].objSerializer.nullable === nullable) } // test for flat encoders @@ -370,6 +371,87 @@ class ExpressionEncoderSuite extends CodegenInterpretedPlanTest with AnalysisTes assert(e.getMessage.contains("Cannot use null as map key")) } + test("throw exception for tuples with more than 22 elements") { + val encoders = (0 to 22).map(_ => Encoders.scalaInt.asInstanceOf[ExpressionEncoder[_]]) + + val e = intercept[UnsupportedOperationException] { + ExpressionEncoder.tuple(encoders) + } + assert(e.getMessage.contains("tuple with more than 22 elements are not supported")) + } + + // Scala / Java big decimals ---------------------------------------------------------- + + encodeDecodeTest(BigDecimal(("9" * 20) + "." + "9" * 18), + "scala decimal within precision/scale limit") + encodeDecodeTest(new java.math.BigDecimal(("9" * 20) + "." + "9" * 18), + "java decimal within precision/scale limit") + + encodeDecodeTest(-BigDecimal(("9" * 20) + "." + "9" * 18), + "negative scala decimal within precision/scale limit") + encodeDecodeTest(new java.math.BigDecimal(("9" * 20) + "." + "9" * 18).negate, + "negative java decimal within precision/scale limit") + + testOverflowingBigNumeric(BigDecimal("1" * 21), "scala big decimal") + testOverflowingBigNumeric(new java.math.BigDecimal("1" * 21), "java big decimal") + + testOverflowingBigNumeric(-BigDecimal("1" * 21), "negative scala big decimal") + testOverflowingBigNumeric(new java.math.BigDecimal("1" * 21).negate, "negative java big decimal") + + testOverflowingBigNumeric(BigDecimal(("1" * 21) + ".123"), + "scala big decimal with fractional part") + testOverflowingBigNumeric(new java.math.BigDecimal(("1" * 21) + ".123"), + "java big decimal with fractional part") + + testOverflowingBigNumeric(BigDecimal(("1" * 21) + "." + "9999" * 100), + "scala big decimal with long fractional part") + testOverflowingBigNumeric(new java.math.BigDecimal(("1" * 21) + "." + "9999" * 100), + "java big decimal with long fractional part") + + // Scala / Java big integers ---------------------------------------------------------- + + encodeDecodeTest(BigInt("9" * 38), "scala big integer within precision limit") + encodeDecodeTest(new BigInteger("9" * 38), "java big integer within precision limit") + + encodeDecodeTest(-BigInt("9" * 38), + "negative scala big integer within precision limit") + encodeDecodeTest(new BigInteger("9" * 38).negate(), + "negative java big integer within precision limit") + + testOverflowingBigNumeric(BigInt("1" * 39), "scala big int") + testOverflowingBigNumeric(new BigInteger("1" * 39), "java big integer") + + testOverflowingBigNumeric(-BigInt("1" * 39), "negative scala big int") + testOverflowingBigNumeric(new BigInteger("1" * 39).negate, "negative java big integer") + + testOverflowingBigNumeric(BigInt("9" * 100), "scala very large big int") + testOverflowingBigNumeric(new BigInteger("9" * 100), "java very big int") + + private def testOverflowingBigNumeric[T: TypeTag](bigNumeric: T, testName: String): Unit = { + Seq(true, false).foreach { allowNullOnOverflow => + testAndVerifyNotLeakingReflectionObjects( + s"overflowing $testName, allowNullOnOverflow=$allowNullOnOverflow") { + withSQLConf( + SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> allowNullOnOverflow.toString + ) { + // Need to construct Encoder here rather than implicitly resolving it + // so that SQLConf changes are respected. + val encoder = ExpressionEncoder[T]() + if (allowNullOnOverflow) { + val convertedBack = encoder.resolveAndBind().fromRow(encoder.toRow(bigNumeric)) + assert(convertedBack === null) + } else { + val e = intercept[RuntimeException] { + encoder.toRow(bigNumeric) + } + assert(e.getMessage.contains("Error while encoding")) + assert(e.getCause.getClass === classOf[ArithmeticException]) + } + } + } + } + } + private def encodeDecodeTest[T : ExpressionEncoder]( input: T, testName: String): Unit = { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala index 8d89f9c6c41d4..5d21e4a2a83ca 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/RowEncoderSuite.scala @@ -21,7 +21,8 @@ import scala.util.Random import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest -import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} +import org.apache.spark.sql.catalyst.util.{ArrayData, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @SQLUserDefinedType(udt = classOf[ExamplePointUDT]) @@ -161,6 +162,32 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { assert(row.toSeq(schema).head == decimal) } + test("SPARK-23179: RowEncoder should respect nullOnOverflow for decimals") { + val schema = new StructType().add("decimal", DecimalType.SYSTEM_DEFAULT) + testDecimalOverflow(schema, Row(BigDecimal("9" * 100))) + testDecimalOverflow(schema, Row(new java.math.BigDecimal("9" * 100))) + } + + private def testDecimalOverflow(schema: StructType, row: Row): Unit = { + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + val encoder = RowEncoder(schema).resolveAndBind() + intercept[Exception] { + encoder.toRow(row) + } match { + case e: ArithmeticException => + assert(e.getMessage.contains("cannot be represented as Decimal")) + case e: RuntimeException => + assert(e.getCause.isInstanceOf[ArithmeticException]) + assert(e.getCause.getMessage.contains("cannot be represented as Decimal")) + } + } + + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + val encoder = RowEncoder(schema).resolveAndBind() + assert(encoder.fromRow(encoder.toRow(row)).get(0) == null) + } + } + test("RowEncoder should preserve schema nullability") { val schema = new StructType().add("int", IntegerType, nullable = false) val encoder = RowEncoder(schema).resolveAndBind() @@ -239,7 +266,7 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { val encoder = RowEncoder(schema) val e = intercept[RuntimeException](encoder.toRow(null)) assert(e.getMessage.contains("Null value appeared in non-nullable field")) - assert(e.getMessage.contains("top level row object")) + assert(e.getMessage.contains("top level Product or row object")) } test("RowEncoder should validate external type") { @@ -273,6 +300,38 @@ class RowEncoderSuite extends CodegenInterpretedPlanTest { assert(e4.getMessage.contains("java.lang.String is not a valid external type")) } + test("SPARK-25791: Datatype of serializers should be accessible") { + val udtSQLType = new StructType().add("a", IntegerType) + val pythonUDT = new PythonUserDefinedType(udtSQLType, "pyUDT", "serializedPyClass") + val schema = new StructType().add("pythonUDT", pythonUDT, true) + val encoder = RowEncoder(schema) + assert(encoder.serializer(0).dataType == pythonUDT.sqlType) + } + + test("encoding/decoding TimestampType to/from java.time.Instant") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val schema = new StructType().add("t", TimestampType) + val encoder = RowEncoder(schema).resolveAndBind() + val instant = java.time.Instant.parse("2019-02-26T16:56:00Z") + val row = encoder.toRow(Row(instant)) + assert(row.getLong(0) === DateTimeUtils.instantToMicros(instant)) + val readback = encoder.fromRow(row) + assert(readback.get(0) === instant) + } + } + + test("encoding/decoding DateType to/from java.time.LocalDate") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val schema = new StructType().add("d", DateType) + val encoder = RowEncoder(schema).resolveAndBind() + val localDate = java.time.LocalDate.parse("2019-02-27") + val row = encoder.toRow(Row(localDate)) + assert(row.getLong(0) === DateTimeUtils.localDateToDays(localDate)) + val readback = encoder.fromRow(row) + assert(readback.get(0).equals(localDate)) + } + } + for { elementType <- Seq(IntegerType, StringType) containsNull <- Seq(true, false) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala index b6e8b667a2400..ca855afd53e12 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/AttributeSetSuite.scala @@ -39,8 +39,8 @@ class AttributeSetSuite extends SparkFunSuite { } test("checks by id not name") { - assert(aSet.contains(aUpper) === true) - assert(aSet.contains(aLower) === true) + assert(aSet.contains(aUpper)) + assert(aSet.contains(aLower)) assert(aSet.contains(fakeA) === false) assert(aSet.contains(bUpper) === false) @@ -48,8 +48,8 @@ class AttributeSetSuite extends SparkFunSuite { } test("++ preserves AttributeSet") { - assert((aSet ++ bSet).contains(aUpper) === true) - assert((aSet ++ bSet).contains(aLower) === true) + assert((aSet ++ bSet).contains(aUpper)) + assert((aSet ++ bSet).contains(aLower)) } test("extracts all references ") { @@ -65,7 +65,7 @@ class AttributeSetSuite extends SparkFunSuite { } test("subset") { - assert(aSet.subsetOf(aAndBSet) === true) + assert(aSet.subsetOf(aAndBSet)) assert(aAndBSet.subsetOf(aSet) === false) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala index 28e6940f3cca3..9802a6e5891b8 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CanonicalizeSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.plans.logical.Range +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class CanonicalizeSuite extends SparkFunSuite { @@ -50,4 +51,32 @@ class CanonicalizeSuite extends SparkFunSuite { assert(range.where(arrays1).sameResult(range.where(arrays2))) assert(!range.where(arrays1).sameResult(range.where(arrays3))) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + // GetStructField with different names are semantically equal + val fieldA1 = GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")) + val fieldA2 = GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")) + assert(fieldA1.semanticEquals(fieldA2)) + + val fieldB1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldB2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + assert(fieldB1.semanticEquals(fieldB2)) + } } 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 94dee7ea048c3..4d667fd61ae01 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CastSuite.scala @@ -18,17 +18,17 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} -import java.util.{Calendar, Locale, TimeZone} - -import scala.util.Random +import java.util.{Calendar, TimeZone} +import java.util.concurrent.TimeUnit._ 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.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -293,16 +293,16 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { test("cast from string") { assert(cast("abcdef", StringType).nullable === false) assert(cast("abcdef", BinaryType).nullable === false) - assert(cast("abcdef", BooleanType).nullable === true) - assert(cast("abcdef", TimestampType).nullable === true) - assert(cast("abcdef", LongType).nullable === true) - assert(cast("abcdef", IntegerType).nullable === true) - assert(cast("abcdef", ShortType).nullable === true) - assert(cast("abcdef", ByteType).nullable === true) - assert(cast("abcdef", DecimalType.USER_DEFAULT).nullable === true) - assert(cast("abcdef", DecimalType(4, 2)).nullable === true) - assert(cast("abcdef", DoubleType).nullable === true) - assert(cast("abcdef", FloatType).nullable === true) + assert(cast("abcdef", BooleanType).nullable) + assert(cast("abcdef", TimestampType).nullable) + assert(cast("abcdef", LongType).nullable) + assert(cast("abcdef", IntegerType).nullable) + assert(cast("abcdef", ShortType).nullable) + assert(cast("abcdef", ByteType).nullable) + assert(cast("abcdef", DecimalType.USER_DEFAULT).nullable) + assert(cast("abcdef", DecimalType(4, 2)).nullable) + assert(cast("abcdef", DoubleType).nullable) + assert(cast("abcdef", FloatType).nullable) } test("data type casting") { @@ -320,13 +320,13 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation( cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), TimestampType, timeZoneId), - c.getTimeInMillis * 1000) + MILLISECONDS.toMicros(c.getTimeInMillis)) c = Calendar.getInstance(TimeZoneGMT) c.set(2015, 10, 1, 2, 30, 0) checkEvaluation( cast(cast(new Timestamp(c.getTimeInMillis), StringType, timeZoneId), TimestampType, timeZoneId), - c.getTimeInMillis * 1000) + MILLISECONDS.toMicros(c.getTimeInMillis)) } val gmtId = Option("GMT") @@ -402,33 +402,33 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { test("casting to fixed-precision decimals") { assert(cast(123, DecimalType.USER_DEFAULT).nullable === false) - assert(cast(10.03f, DecimalType.SYSTEM_DEFAULT).nullable === true) - assert(cast(10.03, DecimalType.SYSTEM_DEFAULT).nullable === true) + assert(cast(10.03f, DecimalType.SYSTEM_DEFAULT).nullable) + assert(cast(10.03, DecimalType.SYSTEM_DEFAULT).nullable) assert(cast(Decimal(10.03), DecimalType.SYSTEM_DEFAULT).nullable === false) - assert(cast(123, DecimalType(2, 1)).nullable === true) - assert(cast(10.03f, DecimalType(2, 1)).nullable === true) - assert(cast(10.03, DecimalType(2, 1)).nullable === true) - assert(cast(Decimal(10.03), DecimalType(2, 1)).nullable === true) + assert(cast(123, DecimalType(2, 1)).nullable) + assert(cast(10.03f, DecimalType(2, 1)).nullable) + assert(cast(10.03, DecimalType(2, 1)).nullable) + assert(cast(Decimal(10.03), DecimalType(2, 1)).nullable) assert(cast(123, DecimalType.IntDecimal).nullable === false) - assert(cast(10.03f, DecimalType.FloatDecimal).nullable === true) - assert(cast(10.03, DecimalType.DoubleDecimal).nullable === true) + assert(cast(10.03f, DecimalType.FloatDecimal).nullable) + assert(cast(10.03, DecimalType.DoubleDecimal).nullable) assert(cast(Decimal(10.03), DecimalType(4, 2)).nullable === false) assert(cast(Decimal(10.03), DecimalType(5, 3)).nullable === false) - assert(cast(Decimal(10.03), DecimalType(3, 1)).nullable === true) + assert(cast(Decimal(10.03), DecimalType(3, 1)).nullable) assert(cast(Decimal(10.03), DecimalType(4, 1)).nullable === false) - assert(cast(Decimal(9.95), DecimalType(2, 1)).nullable === true) + assert(cast(Decimal(9.95), DecimalType(2, 1)).nullable) assert(cast(Decimal(9.95), DecimalType(3, 1)).nullable === false) - assert(cast(Decimal("1003"), DecimalType(3, -1)).nullable === true) + assert(cast(Decimal("1003"), DecimalType(3, -1)).nullable) assert(cast(Decimal("1003"), DecimalType(4, -1)).nullable === false) - assert(cast(Decimal("995"), DecimalType(2, -1)).nullable === true) + assert(cast(Decimal("995"), DecimalType(2, -1)).nullable) assert(cast(Decimal("995"), DecimalType(3, -1)).nullable === false) assert(cast(true, DecimalType.SYSTEM_DEFAULT).nullable === false) - assert(cast(true, DecimalType(1, 1)).nullable === true) + assert(cast(true, DecimalType(1, 1)).nullable) checkEvaluation(cast(10.03, DecimalType.SYSTEM_DEFAULT), Decimal(10.03)) @@ -521,17 +521,17 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(cast(ts, FloatType), 15.003f) checkEvaluation(cast(ts, DoubleType), 15.003) checkEvaluation(cast(cast(tss, ShortType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * 1000) + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation(cast(cast(tss, IntegerType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * 1000) + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation(cast(cast(tss, LongType), TimestampType), - DateTimeUtils.fromJavaTimestamp(ts) * 1000) + DateTimeUtils.fromJavaTimestamp(ts) * MILLIS_PER_SECOND) checkEvaluation( - cast(cast(millis.toFloat / 1000, TimestampType), FloatType), - millis.toFloat / 1000) + cast(cast(millis.toFloat / MILLIS_PER_SECOND, TimestampType), FloatType), + millis.toFloat / MILLIS_PER_SECOND) checkEvaluation( - cast(cast(millis.toDouble / 1000, TimestampType), DoubleType), - millis.toDouble / 1000) + cast(cast(millis.toDouble / MILLIS_PER_SECOND, TimestampType), DoubleType), + millis.toDouble / MILLIS_PER_SECOND) checkEvaluation( cast(cast(Decimal(1), TimestampType), DecimalType.SYSTEM_DEFAULT), Decimal(1)) @@ -555,7 +555,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(array, ArrayType(IntegerType, containsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Seq(123, null, null, null)) } { @@ -564,7 +564,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } { val ret = cast(array, ArrayType(BooleanType, containsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Seq(null, true, false, null)) } { @@ -574,7 +574,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(array_notNull, ArrayType(IntegerType, containsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Seq(123, null, null)) } { @@ -583,7 +583,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } { val ret = cast(array_notNull, ArrayType(BooleanType, containsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Seq(null, true, false)) } { @@ -609,7 +609,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(map, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null, "d" -> null)) } { @@ -618,7 +618,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } { val ret = cast(map, MapType(StringType, BooleanType, valueContainsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Map("a" -> null, "b" -> true, "c" -> false, "d" -> null)) } { @@ -632,7 +632,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { { val ret = cast(map_notNull, MapType(StringType, IntegerType, valueContainsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Map("a" -> 123, "b" -> null, "c" -> null)) } { @@ -641,7 +641,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { } { val ret = cast(map_notNull, MapType(StringType, BooleanType, valueContainsNull = true)) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, Map("a" -> null, "b" -> true, "c" -> false)) } { @@ -695,7 +695,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("b", IntegerType, nullable = true), StructField("c", IntegerType, nullable = true), StructField("d", IntegerType, nullable = true)))) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, InternalRow(123, null, null, null)) } { @@ -712,7 +712,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("b", BooleanType, nullable = true), StructField("c", BooleanType, nullable = true), StructField("d", BooleanType, nullable = true)))) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, InternalRow(null, true, false, null)) } { @@ -729,7 +729,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("a", IntegerType, nullable = true), StructField("b", IntegerType, nullable = true), StructField("c", IntegerType, nullable = true)))) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, InternalRow(123, null, null)) } { @@ -744,7 +744,7 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { StructField("a", BooleanType, nullable = true), StructField("b", BooleanType, nullable = true), StructField("c", BooleanType, nullable = true)))) - assert(ret.resolved === true) + assert(ret.resolved) checkEvaluation(ret, InternalRow(null, true, false)) } { @@ -955,4 +955,67 @@ class CastSuite extends SparkFunSuite with ExpressionEvalHelper { val ret6 = cast(Literal.create((1, Map(1 -> "a", 2 -> "b", 3 -> "c"))), StringType) checkEvaluation(ret6, "[1, [1 -> a, 2 -> b, 3 -> c]]") } + + test("up-cast") { + def isCastSafe(from: NumericType, to: NumericType): Boolean = (from, to) match { + case (_, dt: DecimalType) => dt.isWiderThan(from) + case (dt: DecimalType, _) => dt.isTighterThan(to) + case _ => numericPrecedence.indexOf(from) <= numericPrecedence.indexOf(to) + } + + def makeComplexTypes(dt: NumericType, nullable: Boolean): Seq[DataType] = { + Seq( + new StructType().add("a", dt, nullable).add("b", dt, nullable), + ArrayType(dt, nullable), + MapType(dt, dt, nullable), + ArrayType(new StructType().add("a", dt, nullable), nullable), + new StructType().add("a", ArrayType(dt, nullable), nullable) + ) + } + + import DataTypeTestUtils.numericTypes + numericTypes.foreach { from => + val (safeTargetTypes, unsafeTargetTypes) = numericTypes.partition(to => isCastSafe(from, to)) + + safeTargetTypes.foreach { to => + assert(Cast.canUpCast(from, to), s"It should be possible to up-cast $from to $to") + + // If the nullability is compatible, we can up-cast complex types too. + Seq(true -> true, false -> false, false -> true).foreach { case (fn, tn) => + makeComplexTypes(from, fn).zip(makeComplexTypes(to, tn)).foreach { + case (complexFromType, complexToType) => + assert(Cast.canUpCast(complexFromType, complexToType)) + } + } + + makeComplexTypes(from, true).zip(makeComplexTypes(to, false)).foreach { + case (complexFromType, complexToType) => + assert(!Cast.canUpCast(complexFromType, complexToType)) + } + } + + unsafeTargetTypes.foreach { to => + assert(!Cast.canUpCast(from, to), s"It shouldn't be possible to up-cast $from to $to") + makeComplexTypes(from, true).zip(makeComplexTypes(to, true)).foreach { + case (complexFromType, complexToType) => + assert(!Cast.canUpCast(complexFromType, complexToType)) + } + } + } + } + + test("SPARK-27671: cast from nested null type in struct") { + import DataTypeTestUtils._ + + atomicTypes.foreach { atomicType => + val struct = Literal.create( + InternalRow(null), + StructType(Seq(StructField("a", NullType, nullable = true)))) + + val ret = cast(struct, StructType(Seq( + StructField("a", atomicType, nullable = true)))) + assert(ret.resolved) + checkEvaluation(ret, InternalRow(null)) + } + } } 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 7843003a4aac3..4e64313da136b 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.Timestamp -import org.apache.log4j.{Appender, AppenderSkeleton, Logger} +import org.apache.log4j.AppenderSkeleton import org.apache.log4j.spi.LoggingEvent import org.apache.spark.SparkFunSuite @@ -31,6 +31,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.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.ThreadUtils @@ -189,36 +190,42 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-17702: split wide constructor into blocks due to JVM code size limit") { - val length = 5000 - val expressions = Seq.fill(length) { - ToUTCTimestamp( - Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType), - Literal.create("PST", StringType)) - } - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - val expected = Seq.fill(length)( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00"))) - - if (actual != expected) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + val length = 5000 + val expressions = Seq.fill(length) { + ToUTCTimestamp( + Literal.create(Timestamp.valueOf("2015-07-24 00:00:00"), TimestampType), + Literal.create("PST", StringType)) + } + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq.fill(length)( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2015-07-24 07:00:00"))) + + if (actual != expected) { + fail( + s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + } } } test("SPARK-22226: group splitted expressions into one method per nested class") { - val length = 10000 - val expressions = Seq.fill(length) { - ToUTCTimestamp( - Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType), - Literal.create("PST", StringType)) - } - val plan = GenerateMutableProjection.generate(expressions) - val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) - val expected = Seq.fill(length)( - DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00"))) - - if (actual != expected) { - fail(s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + val length = 10000 + val expressions = Seq.fill(length) { + ToUTCTimestamp( + Literal.create(Timestamp.valueOf("2017-10-10 00:00:00"), TimestampType), + Literal.create("PST", StringType)) + } + val plan = GenerateMutableProjection.generate(expressions) + val actual = plan(new GenericInternalRow(length)).toSeq(expressions.map(_.dataType)) + val expected = Seq.fill(length)( + DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf("2017-10-10 07:00:00"))) + + if (actual != expected) { + fail( + s"Incorrect Evaluation: expressions: $expressions, actual: $actual, expected: $expected") + } } } @@ -251,7 +258,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { UTF8String.fromString("c")) assert(unsafeRow.getStruct(3, 1).getStruct(0, 2).getInt(1) === 3) - val fromUnsafe = FromUnsafeProjection(schema) + val fromUnsafe = SafeProjection.create(schema) val internalRow2 = fromUnsafe(unsafeRow) assert(internalRow === internalRow2) @@ -338,7 +345,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { test("should not apply common subexpression elimination on conditional expressions") { val row = InternalRow(null) val bound = BoundReference(0, IntegerType, true) - val assertNotNull = AssertNotNull(bound, Nil) + val assertNotNull = AssertNotNull(bound) val expr = If(IsNull(bound), Literal(1), Add(assertNotNull, assertNotNull)) val projection = GenerateUnsafeProjection.generate( Seq(expr), subexpressionEliminationEnabled = true) @@ -424,7 +431,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx1.inlinedMutableStates.size == CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD) // When the number of primitive type mutable states is over the threshold, others are // allocated into an array - assert(ctx1.arrayCompactedMutableStates.get(CodeGenerator.JAVA_INT).get.arrayNames.size == 1) + assert(ctx1.arrayCompactedMutableStates(CodeGenerator.JAVA_INT).arrayNames.size == 1) assert(ctx1.mutableStateInitCode.size == CodeGenerator.OUTER_CLASS_VARIABLES_THRESHOLD + 10) val ctx2 = new CodegenContext @@ -434,7 +441,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { // When the number of non-primitive type mutable states is over the threshold, others are // allocated into a new array assert(ctx2.inlinedMutableStates.isEmpty) - assert(ctx2.arrayCompactedMutableStates.get("InternalRow[]").get.arrayNames.size == 2) + assert(ctx2.arrayCompactedMutableStates("InternalRow[]").arrayNames.size == 2) assert(ctx2.arrayCompactedMutableStates("InternalRow[]").getCurrentIndex == 10) assert(ctx2.mutableStateInitCode.size == CodeGenerator.MUTABLESTATEARRAY_SIZE_LIMIT + 10) } @@ -529,7 +536,7 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } val appender = new MockAppender() - withLogAppender(appender) { + withLogAppender(appender, loggerName = Some(classOf[CodeGenerator[_, _]].getName)) { val x = 42 val expr = HugeCodeIntExpression(x) val proj = GenerateUnsafeProjection.generate(Seq(expr)) @@ -538,15 +545,6 @@ class CodeGenerationSuite extends SparkFunSuite with ExpressionEvalHelper { } assert(appender.seenMessage) } - - private def withLogAppender(appender: Appender)(f: => Unit): Unit = { - val logger = - Logger.getLogger(classOf[CodeGenerator[_, _]].getName) - logger.addAppender(appender) - try f finally { - logger.removeAppender(appender) - } - } } case class HugeCodeIntExpression(value: Int) extends Expression { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala index 6ea3b05ff9c1e..da5bddb0c09fd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CodeGeneratorWithInterpretedFallbackSuite.scala @@ -106,4 +106,19 @@ class CodeGeneratorWithInterpretedFallbackSuite extends SparkFunSuite with PlanT assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) } } + + test("SPARK-25374 Correctly handles NoOp in SafeProjection") { + val exprs = Seq(Add(BoundReference(0, IntegerType, nullable = true), Literal.create(1)), NoOp) + val input = InternalRow.fromSeq(1 :: 1 :: Nil) + val expected = 2 :: null :: Nil + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> codegenOnly) { + val proj = SafeProjection.createObject(exprs) + assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) + } + + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> noCodegen) { + val proj = SafeProjection.createObject(exprs) + assert(proj(input).toSeq(StructType.fromDDL("c0 int, c1 int")) === expected) + } + } } 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 2e0adbb465008..603073b40d7aa 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 @@ -25,6 +25,7 @@ import scala.util.Random 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 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -108,101 +109,64 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper } test("Map Concat") { - val m0 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType, + val m0 = Literal.create(create_map("a" -> "1", "b" -> "2"), MapType(StringType, StringType, valueContainsNull = false)) - val m1 = Literal.create(Map("c" -> "3", "a" -> "4"), MapType(StringType, StringType, + val m1 = Literal.create(create_map("c" -> "3", "a" -> "4"), MapType(StringType, StringType, valueContainsNull = false)) - val m2 = Literal.create(Map("d" -> "4", "e" -> "5"), MapType(StringType, StringType)) - val m3 = Literal.create(Map("a" -> "1", "b" -> "2"), MapType(StringType, StringType)) - val m4 = Literal.create(Map("a" -> null, "c" -> "3"), MapType(StringType, StringType)) - val m5 = Literal.create(Map("a" -> 1, "b" -> 2), MapType(StringType, IntegerType)) - val m6 = Literal.create(Map("a" -> null, "c" -> 3), MapType(StringType, IntegerType)) - val m7 = Literal.create(Map(List(1, 2) -> 1, List(3, 4) -> 2), + val m2 = Literal.create(create_map("d" -> "4", "e" -> "5"), MapType(StringType, StringType)) + val m3 = Literal.create(create_map("f" -> "1", "g" -> "2"), MapType(StringType, StringType)) + val m4 = Literal.create(create_map("a" -> null, "c" -> "3"), MapType(StringType, StringType)) + val m5 = Literal.create(create_map("a" -> 1, "b" -> 2), MapType(StringType, IntegerType)) + val m6 = Literal.create(create_map("c" -> null, "d" -> 3), MapType(StringType, IntegerType)) + val m7 = Literal.create(create_map(List(1, 2) -> 1, List(3, 4) -> 2), MapType(ArrayType(IntegerType), IntegerType)) - val m8 = Literal.create(Map(List(5, 6) -> 3, List(1, 2) -> 4), + val m8 = Literal.create(create_map(List(5, 6) -> 3, List(7, 8) -> 4), MapType(ArrayType(IntegerType), IntegerType)) - val m9 = Literal.create(Map(Map(1 -> 2, 3 -> 4) -> 1, Map(5 -> 6, 7 -> 8) -> 2), - MapType(MapType(IntegerType, IntegerType), IntegerType)) - val m10 = Literal.create(Map(Map(9 -> 10, 11 -> 12) -> 3, Map(1 -> 2, 3 -> 4) -> 4), - MapType(MapType(IntegerType, IntegerType), IntegerType)) - val m11 = Literal.create(Map(1 -> "1", 2 -> "2"), MapType(IntegerType, StringType, + val m9 = Literal.create(create_map(1 -> "1", 2 -> "2"), MapType(IntegerType, StringType, valueContainsNull = false)) - val m12 = Literal.create(Map(3 -> "3", 4 -> "4"), MapType(IntegerType, StringType, + val m10 = Literal.create(create_map(3 -> "3", 4 -> "4"), MapType(IntegerType, StringType, valueContainsNull = false)) - val m13 = Literal.create(Map(1 -> 2, 3 -> 4), + val m11 = Literal.create(create_map(1 -> 2, 3 -> 4), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val m14 = Literal.create(Map(5 -> 6), + val m12 = Literal.create(create_map(5 -> 6), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val m15 = Literal.create(Map(7 -> null), + val m13 = Literal.create(create_map(7 -> null), MapType(IntegerType, IntegerType, valueContainsNull = true)) val mNull = Literal.create(null, MapType(StringType, StringType)) - // overlapping maps - checkEvaluation(MapConcat(Seq(m0, m1)), - ( - Array("a", "b", "c", "a"), // keys - Array("1", "2", "3", "4") // values - ) - ) + // overlapping maps should remove duplicated map keys w.r.t. last win policy. + checkEvaluation(MapConcat(Seq(m0, m1)), create_map("a" -> "4", "b" -> "2", "c" -> "3")) // maps with no overlap checkEvaluation(MapConcat(Seq(m0, m2)), - Map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5")) + create_map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5")) // 3 maps - checkEvaluation(MapConcat(Seq(m0, m1, m2)), - ( - Array("a", "b", "c", "a", "d", "e"), // keys - Array("1", "2", "3", "4", "4", "5") // values - ) - ) + checkEvaluation(MapConcat(Seq(m0, m2, m3)), + create_map("a" -> "1", "b" -> "2", "d" -> "4", "e" -> "5", "f" -> "1", "g" -> "2")) // null reference values - checkEvaluation(MapConcat(Seq(m3, m4)), - ( - Array("a", "b", "a", "c"), // keys - Array("1", "2", null, "3") // values - ) - ) + checkEvaluation(MapConcat(Seq(m2, m4)), + create_map("d" -> "4", "e" -> "5", "a" -> null, "c" -> "3")) // null primitive values checkEvaluation(MapConcat(Seq(m5, m6)), - ( - Array("a", "b", "a", "c"), // keys - Array(1, 2, null, 3) // values - ) - ) + create_map("a" -> 1, "b" -> 2, "c" -> null, "d" -> 3)) // keys that are primitive - checkEvaluation(MapConcat(Seq(m11, m12)), - ( - Array(1, 2, 3, 4), // keys - Array("1", "2", "3", "4") // values - ) - ) + checkEvaluation(MapConcat(Seq(m9, m10)), + create_map(1 -> "1", 2 -> "2", 3 -> "3", 4 -> "4")) - // keys that are arrays, with overlap + // keys that are arrays checkEvaluation(MapConcat(Seq(m7, m8)), - ( - Array(List(1, 2), List(3, 4), List(5, 6), List(1, 2)), // keys - Array(1, 2, 3, 4) // values - ) - ) + create_map(List(1, 2) -> 1, List(3, 4) -> 2, List(5, 6) -> 3, List(7, 8) -> 4)) - // keys that are maps, with overlap - checkEvaluation(MapConcat(Seq(m9, m10)), - ( - Array(Map(1 -> 2, 3 -> 4), Map(5 -> 6, 7 -> 8), Map(9 -> 10, 11 -> 12), - Map(1 -> 2, 3 -> 4)), // keys - Array(1, 2, 3, 4) // values - ) - ) // both keys and value are primitive and valueContainsNull = false - checkEvaluation(MapConcat(Seq(m13, m14)), Map(1 -> 2, 3 -> 4, 5 -> 6)) + checkEvaluation(MapConcat(Seq(m11, m12)), create_map(1 -> 2, 3 -> 4, 5 -> 6)) // both keys and value are primitive and valueContainsNull = true - checkEvaluation(MapConcat(Seq(m13, m15)), Map(1 -> 2, 3 -> 4, 7 -> null)) + checkEvaluation(MapConcat(Seq(m11, m13)), create_map(1 -> 2, 3 -> 4, 7 -> null)) // null map checkEvaluation(MapConcat(Seq(m0, mNull)), null) @@ -211,21 +175,20 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(MapConcat(Seq(mNull)), null) // single map - checkEvaluation(MapConcat(Seq(m0)), Map("a" -> "1", "b" -> "2")) + checkEvaluation(MapConcat(Seq(m0)), create_map("a" -> "1", "b" -> "2")) // no map checkEvaluation(MapConcat(Seq.empty), Map.empty) // force split expressions for input in generated code - val expectedKeys = Array.fill(65)(Seq("a", "b")).flatten ++ Array("d", "e") - val expectedValues = Array.fill(65)(Seq("1", "2")).flatten ++ Array("4", "5") - checkEvaluation(MapConcat( - Seq( - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, - m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m0, m2 - )), - (expectedKeys, expectedValues)) + val expectedKeys = (1 to 65).map(_.toString) + val expectedValues = (1 to 65).map(_.toString) + checkEvaluation( + MapConcat( + expectedKeys.zip(expectedValues).map { + case (k, v) => Literal.create(create_map(k -> v), MapType(StringType, StringType)) + }), + create_map(expectedKeys.zip(expectedValues): _*)) // argument checking assert(MapConcat(Seq(m0, m1)).checkInputDataTypes().isSuccess) @@ -245,12 +208,12 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper assert(MapConcat(Seq(m1, mNull)).nullable) val mapConcat = MapConcat(Seq( - Literal.create(Map(Seq(1, 2) -> Seq("a", "b")), + Literal.create(create_map(Seq(1, 2) -> Seq("a", "b")), MapType( ArrayType(IntegerType, containsNull = false), ArrayType(StringType, containsNull = false), valueContainsNull = false)), - Literal.create(Map(Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null), + Literal.create(create_map(Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null), MapType( ArrayType(IntegerType, containsNull = true), ArrayType(StringType, containsNull = true), @@ -260,10 +223,22 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayType(IntegerType, containsNull = true), ArrayType(StringType, containsNull = true), valueContainsNull = true)) - checkEvaluation(mapConcat, Map( + checkEvaluation(mapConcat, create_map( Seq(1, 2) -> Seq("a", "b"), Seq(3, 4, null) -> Seq("c", "d", null), Seq(6) -> null)) + + // map key can't be map + val mapOfMap = Literal.create(Map(Map(1 -> 2, 3 -> 4) -> 1, Map(5 -> 6, 7 -> 8) -> 2), + MapType(MapType(IntegerType, IntegerType), IntegerType)) + val mapOfMap2 = Literal.create(Map(Map(9 -> 10, 11 -> 12) -> 3, Map(1 -> 2, 3 -> 4) -> 4), + MapType(MapType(IntegerType, IntegerType), IntegerType)) + val map = MapConcat(Seq(mapOfMap, mapOfMap2)) + map.checkInputDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("The key of map cannot be/contain map")) + } } test("MapFromEntries") { @@ -274,47 +249,64 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper StructField("b", valueType))), true) } - def r(values: Any*): InternalRow = create_row(values: _*) + def row(values: Any*): InternalRow = create_row(values: _*) // Primitive-type keys and values val aiType = arrayType(IntegerType, IntegerType) - val ai0 = Literal.create(Seq(r(1, 10), r(2, 20), r(3, 20)), aiType) - val ai1 = Literal.create(Seq(r(1, null), r(2, 20), r(3, null)), aiType) + val ai0 = Literal.create(Seq(row(1, 10), row(2, 20), row(3, 20)), aiType) + val ai1 = Literal.create(Seq(row(1, null), row(2, 20), row(3, null)), aiType) val ai2 = Literal.create(Seq.empty, aiType) val ai3 = Literal.create(null, aiType) - val ai4 = Literal.create(Seq(r(1, 10), r(1, 20)), aiType) - val ai5 = Literal.create(Seq(r(1, 10), r(null, 20)), aiType) - val ai6 = Literal.create(Seq(null, r(2, 20), null), aiType) - - checkEvaluation(MapFromEntries(ai0), Map(1 -> 10, 2 -> 20, 3 -> 20)) - checkEvaluation(MapFromEntries(ai1), Map(1 -> null, 2 -> 20, 3 -> null)) + // The map key is duplicated + val ai4 = Literal.create(Seq(row(1, 10), row(1, 20)), aiType) + // The map key is null + val ai5 = Literal.create(Seq(row(1, 10), row(null, 20)), aiType) + val ai6 = Literal.create(Seq(null, row(2, 20), null), aiType) + + checkEvaluation(MapFromEntries(ai0), create_map(1 -> 10, 2 -> 20, 3 -> 20)) + checkEvaluation(MapFromEntries(ai1), create_map(1 -> null, 2 -> 20, 3 -> null)) checkEvaluation(MapFromEntries(ai2), Map.empty) checkEvaluation(MapFromEntries(ai3), null) - checkEvaluation(MapKeys(MapFromEntries(ai4)), Seq(1, 1)) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(MapFromEntries(ai4), create_map(1 -> 20)) + // Map key can't be null checkExceptionInExpression[RuntimeException]( MapFromEntries(ai5), - "The first field from a struct (key) can't be null.") + "Cannot use null as map key") checkEvaluation(MapFromEntries(ai6), null) // Non-primitive-type keys and values val asType = arrayType(StringType, StringType) - val as0 = Literal.create(Seq(r("a", "aa"), r("b", "bb"), r("c", "bb")), asType) - val as1 = Literal.create(Seq(r("a", null), r("b", "bb"), r("c", null)), asType) + val as0 = Literal.create(Seq(row("a", "aa"), row("b", "bb"), row("c", "bb")), asType) + val as1 = Literal.create(Seq(row("a", null), row("b", "bb"), row("c", null)), asType) val as2 = Literal.create(Seq.empty, asType) val as3 = Literal.create(null, asType) - val as4 = Literal.create(Seq(r("a", "aa"), r("a", "bb")), asType) - val as5 = Literal.create(Seq(r("a", "aa"), r(null, "bb")), asType) - val as6 = Literal.create(Seq(null, r("b", "bb"), null), asType) + val as4 = Literal.create(Seq(row("a", "aa"), row("a", "bb")), asType) + val as5 = Literal.create(Seq(row("a", "aa"), row(null, "bb")), asType) + val as6 = Literal.create(Seq(null, row("b", "bb"), null), asType) - checkEvaluation(MapFromEntries(as0), Map("a" -> "aa", "b" -> "bb", "c" -> "bb")) - checkEvaluation(MapFromEntries(as1), Map("a" -> null, "b" -> "bb", "c" -> null)) + checkEvaluation(MapFromEntries(as0), create_map("a" -> "aa", "b" -> "bb", "c" -> "bb")) + checkEvaluation(MapFromEntries(as1), create_map("a" -> null, "b" -> "bb", "c" -> null)) checkEvaluation(MapFromEntries(as2), Map.empty) checkEvaluation(MapFromEntries(as3), null) - checkEvaluation(MapKeys(MapFromEntries(as4)), Seq("a", "a")) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(MapFromEntries(as4), create_map("a" -> "bb")) + // Map key can't be null checkExceptionInExpression[RuntimeException]( MapFromEntries(as5), - "The first field from a struct (key) can't be null.") + "Cannot use null as map key") checkEvaluation(MapFromEntries(as6), null) + + // map key can't be map + val structOfMap = row(create_map(1 -> 1), 1) + val map = MapFromEntries(Literal.create( + Seq(structOfMap), + arrayType(keyType = MapType(IntegerType, IntegerType), valueType = IntegerType))) + map.checkInputDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("The key of map cannot be/contain map")) + } } test("Sort Array") { @@ -1100,6 +1092,39 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(ElementAt(mb0, Literal(Array[Byte](3, 4))), null) } + test("correctly handles ElementAt nullability for arrays") { + // CreateArray case + val a = AttributeReference("a", IntegerType, nullable = false)() + val b = AttributeReference("b", IntegerType, nullable = true)() + val array = CreateArray(a :: b :: Nil) + assert(!ElementAt(array, Literal(0)).nullable) + assert(ElementAt(array, Literal(1)).nullable) + assert(!ElementAt(array, Subtract(Literal(2), Literal(2))).nullable) + assert(ElementAt(array, AttributeReference("ordinal", IntegerType)()).nullable) + + // GetArrayStructFields case + val f1 = StructField("a", IntegerType, nullable = false) + val f2 = StructField("b", IntegerType, nullable = true) + val structType = StructType(f1 :: f2 :: Nil) + val c = AttributeReference("c", structType, nullable = false)() + val inputArray1 = CreateArray(c :: Nil) + val inputArray1ContainsNull = c.nullable + val stArray1 = GetArrayStructFields(inputArray1, f1, 0, 2, inputArray1ContainsNull) + assert(!ElementAt(stArray1, Literal(0)).nullable) + val stArray2 = GetArrayStructFields(inputArray1, f2, 1, 2, inputArray1ContainsNull) + assert(ElementAt(stArray2, Literal(0)).nullable) + + val d = AttributeReference("d", structType, nullable = true)() + val inputArray2 = CreateArray(c :: d :: Nil) + val inputArray2ContainsNull = c.nullable || d.nullable + val stArray3 = GetArrayStructFields(inputArray2, f1, 0, 2, inputArray2ContainsNull) + assert(!ElementAt(stArray3, Literal(0)).nullable) + assert(ElementAt(stArray3, Literal(1)).nullable) + val stArray4 = GetArrayStructFields(inputArray2, f2, 1, 2, inputArray2ContainsNull) + assert(ElementAt(stArray4, Literal(0)).nullable) + assert(ElementAt(stArray4, Literal(1)).nullable) + } + test("Concat") { // Primitive-type elements val ai0 = Literal.create(Seq(1, 2, 3), ArrayType(IntegerType, containsNull = false)) @@ -1145,9 +1170,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(Concat(Seq(aa0, aa1)), Seq(Seq("a", "b"), Seq("c"), Seq("d"), Seq("e", "f"))) assert(Concat(Seq(ai0, ai1)).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(Concat(Seq(ai0, ai2)).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(Concat(Seq(ai0, ai2)).dataType.asInstanceOf[ArrayType].containsNull) assert(Concat(Seq(as0, as1)).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(Concat(Seq(as0, as2)).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(Concat(Seq(as0, as2)).dataType.asInstanceOf[ArrayType].containsNull) assert(Concat(Seq(aa0, aa1)).dataType === ArrayType(ArrayType(StringType, containsNull = false), containsNull = false)) assert(Concat(Seq(aa0, aa2)).dataType === @@ -1339,6 +1364,8 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayType(DoubleType)) val a7 = Literal.create(Seq(1.123f, 0.1234f, 1.121f, 1.123f, 1.1230f, 1.121f, 0.1234f), ArrayType(FloatType)) + val a8 = + Literal.create(Seq(2, 1, 2, 3, 4, 4, 5).map(_.toString.getBytes), ArrayType(BinaryType)) checkEvaluation(new ArrayDistinct(a0), Seq(2, 1, 3, 4, 5)) checkEvaluation(new ArrayDistinct(a1), Seq.empty[Integer]) @@ -1348,6 +1375,7 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(new ArrayDistinct(a5), Seq(true, false)) checkEvaluation(new ArrayDistinct(a6), Seq(1.123, 0.1234, 1.121)) checkEvaluation(new ArrayDistinct(a7), Seq(1.123f, 0.1234f, 1.121f)) + checkEvaluation(new ArrayDistinct(a8), Seq(2, 1, 3, 4, 5).map(_.toString.getBytes)) // complex data types val b0 = Literal.create(Seq[Array[Byte]](Array[Byte](5, 6), Array[Byte](1, 2), @@ -1368,9 +1396,17 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper ArrayType(ArrayType(IntegerType))) val c2 = Literal.create(Seq[Seq[Int]](null, Seq[Int](2, 1), null, null, Seq[Int](2, 1), null), ArrayType(ArrayType(IntegerType))) + val c3 = Literal.create(Seq[Seq[Int]](Seq[Int](1, 2), Seq[Int](1, 2), Seq[Int](1, 2), + Seq[Int](3, 4), Seq[Int](4, 5)), ArrayType(ArrayType(IntegerType))) + val c4 = Literal.create(Seq[Seq[Int]](null, Seq[Int](1, 2), Seq[Int](1, 2), + Seq[Int](3, 4), Seq[Int](4, 5), null), ArrayType(ArrayType(IntegerType))) checkEvaluation(ArrayDistinct(c0), Seq[Seq[Int]](Seq[Int](1, 2), Seq[Int](3, 4))) checkEvaluation(ArrayDistinct(c1), Seq[Seq[Int]](Seq[Int](5, 6), Seq[Int](2, 1))) checkEvaluation(ArrayDistinct(c2), Seq[Seq[Int]](null, Seq[Int](2, 1))) + checkEvaluation(ArrayDistinct(c3), Seq[Seq[Int]](Seq[Int](1, 2), Seq[Int](3, 4), + Seq[Int](4, 5))) + checkEvaluation(ArrayDistinct(c4), Seq[Seq[Int]](null, Seq[Int](1, 2), Seq[Int](3, 4), + Seq[Int](4, 5))) } test("Array Union") { @@ -1458,9 +1494,9 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper Seq[Seq[Int]](Seq[Int](1, 2), Seq[Int](3, 4), Seq[Int](5, 6), Seq[Int](2, 1))) assert(ArrayUnion(a00, a01).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayUnion(a00, a02).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayUnion(a00, a02).dataType.asInstanceOf[ArrayType].containsNull) assert(ArrayUnion(a20, a21).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayUnion(a20, a22).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayUnion(a20, a22).dataType.asInstanceOf[ArrayType].containsNull) } test("Shuffle") { @@ -1639,10 +1675,23 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(ArrayExcept(aa1, aa0), Seq[Seq[Int]](Seq[Int](2, 1))) assert(ArrayExcept(a00, a01).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayExcept(a04, a02).dataType.asInstanceOf[ArrayType].containsNull === true) - assert(ArrayExcept(a04, a05).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayExcept(a04, a02).dataType.asInstanceOf[ArrayType].containsNull) + assert(ArrayExcept(a04, a05).dataType.asInstanceOf[ArrayType].containsNull) assert(ArrayExcept(a20, a21).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayExcept(a24, a22).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayExcept(a24, a22).dataType.asInstanceOf[ArrayType].containsNull) + } + + test("Array Except - null handling") { + val empty = Literal.create(Seq.empty[Int], ArrayType(IntegerType, containsNull = false)) + val oneNull = Literal.create(Seq(null), ArrayType(IntegerType)) + val twoNulls = Literal.create(Seq(null, null), ArrayType(IntegerType)) + + checkEvaluation(ArrayExcept(oneNull, oneNull), Seq.empty) + checkEvaluation(ArrayExcept(twoNulls, twoNulls), Seq.empty) + checkEvaluation(ArrayExcept(twoNulls, oneNull), Seq.empty) + checkEvaluation(ArrayExcept(empty, oneNull), Seq.empty) + checkEvaluation(ArrayExcept(oneNull, empty), Seq(null)) + checkEvaluation(ArrayExcept(twoNulls, empty), Seq(null)) } test("Array Intersect") { @@ -1752,8 +1801,21 @@ class CollectionExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper assert(ArrayIntersect(a00, a01).dataType.asInstanceOf[ArrayType].containsNull === false) assert(ArrayIntersect(a00, a04).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayIntersect(a04, a05).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayIntersect(a04, a05).dataType.asInstanceOf[ArrayType].containsNull) assert(ArrayIntersect(a20, a21).dataType.asInstanceOf[ArrayType].containsNull === false) - assert(ArrayIntersect(a23, a24).dataType.asInstanceOf[ArrayType].containsNull === true) + assert(ArrayIntersect(a23, a24).dataType.asInstanceOf[ArrayType].containsNull) + } + + test("Array Intersect - null handling") { + val empty = Literal.create(Seq.empty[Int], ArrayType(IntegerType, containsNull = false)) + val oneNull = Literal.create(Seq(null), ArrayType(IntegerType)) + val twoNulls = Literal.create(Seq(null, null), ArrayType(IntegerType)) + + checkEvaluation(ArrayIntersect(oneNull, oneNull), Seq(null)) + checkEvaluation(ArrayIntersect(twoNulls, twoNulls), Seq(null)) + checkEvaluation(ArrayIntersect(twoNulls, oneNull), Seq(null)) + checkEvaluation(ArrayIntersect(oneNull, twoNulls), Seq(null)) + checkEvaluation(ArrayIntersect(empty, oneNull), Seq.empty) + checkEvaluation(ArrayIntersect(oneNull, empty), Seq.empty) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala index 77aaf55480ec2..0c4438987cd2a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ComplexTypeSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.analysis.UnresolvedExtractValue +import org.apache.spark.sql.catalyst.analysis.{TypeCheckResult, UnresolvedExtractValue} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.types._ @@ -59,6 +59,39 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(GetArrayItem(nestedArray, Literal(0)), Seq(1)) } + test("SPARK-26637 handles GetArrayItem nullability correctly when input array size is constant") { + // CreateArray case + val a = AttributeReference("a", IntegerType, nullable = false)() + val b = AttributeReference("b", IntegerType, nullable = true)() + val array = CreateArray(a :: b :: Nil) + assert(!GetArrayItem(array, Literal(0)).nullable) + assert(GetArrayItem(array, Literal(1)).nullable) + assert(!GetArrayItem(array, Subtract(Literal(2), Literal(2))).nullable) + assert(GetArrayItem(array, AttributeReference("ordinal", IntegerType)()).nullable) + + // GetArrayStructFields case + val f1 = StructField("a", IntegerType, nullable = false) + val f2 = StructField("b", IntegerType, nullable = true) + val structType = StructType(f1 :: f2 :: Nil) + val c = AttributeReference("c", structType, nullable = false)() + val inputArray1 = CreateArray(c :: Nil) + val inputArray1ContainsNull = c.nullable + val stArray1 = GetArrayStructFields(inputArray1, f1, 0, 2, inputArray1ContainsNull) + assert(!GetArrayItem(stArray1, Literal(0)).nullable) + val stArray2 = GetArrayStructFields(inputArray1, f2, 1, 2, inputArray1ContainsNull) + assert(GetArrayItem(stArray2, Literal(0)).nullable) + + val d = AttributeReference("d", structType, nullable = true)() + val inputArray2 = CreateArray(c :: d :: Nil) + val inputArray2ContainsNull = c.nullable || d.nullable + val stArray3 = GetArrayStructFields(inputArray2, f1, 0, 2, inputArray2ContainsNull) + assert(!GetArrayItem(stArray3, Literal(0)).nullable) + assert(GetArrayItem(stArray3, Literal(1)).nullable) + val stArray4 = GetArrayStructFields(inputArray2, f2, 1, 2, inputArray2ContainsNull) + assert(GetArrayItem(stArray4, Literal(0)).nullable) + assert(GetArrayItem(stArray4, Literal(1)).nullable) + } + test("GetMapValue") { val typeM = MapType(StringType, StringType) val map = Literal.create(Map("a" -> "b"), typeM) @@ -102,9 +135,9 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val nullStruct_fieldNotNullable = Literal.create(null, typeS_fieldNotNullable) assert(getStructField(struct_fieldNotNullable, "a").nullable === false) - assert(getStructField(struct, "a").nullable === true) - assert(getStructField(nullStruct_fieldNotNullable, "a").nullable === true) - assert(getStructField(nullStruct, "a").nullable === true) + assert(getStructField(struct, "a").nullable) + assert(getStructField(nullStruct_fieldNotNullable, "a").nullable) + assert(getStructField(nullStruct, "a").nullable) } test("GetArrayStructFields") { @@ -158,40 +191,37 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { keys.zip(values).flatMap { case (k, v) => Seq(k, v) } } - def createMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = { - // catalyst map is order-sensitive, so we create ListMap here to preserve the elements order. - scala.collection.immutable.ListMap(keys.zip(values): _*) - } - val intSeq = Seq(5, 10, 15, 20, 25) val longSeq = intSeq.map(_.toLong) val strSeq = intSeq.map(_.toString) + checkEvaluation(CreateMap(Nil), Map.empty) checkEvaluation( CreateMap(interlace(intSeq.map(Literal(_)), longSeq.map(Literal(_)))), - createMap(intSeq, longSeq)) + create_map(intSeq, longSeq)) checkEvaluation( CreateMap(interlace(strSeq.map(Literal(_)), longSeq.map(Literal(_)))), - createMap(strSeq, longSeq)) + create_map(strSeq, longSeq)) checkEvaluation( CreateMap(interlace(longSeq.map(Literal(_)), strSeq.map(Literal(_)))), - createMap(longSeq, strSeq)) + create_map(longSeq, strSeq)) val strWithNull = strSeq.drop(1).map(Literal(_)) :+ Literal.create(null, StringType) checkEvaluation( CreateMap(interlace(intSeq.map(Literal(_)), strWithNull)), - createMap(intSeq, strWithNull.map(_.value))) - intercept[RuntimeException] { - checkEvaluationWithoutCodegen( - CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))), - null, null) - } - intercept[RuntimeException] { - checkEvaluationWithUnsafeProjection( - CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))), - null, null) - } + create_map(intSeq, strWithNull.map(_.value))) + + // Map key can't be null + checkExceptionInExpression[RuntimeException]( + CreateMap(interlace(strWithNull, intSeq.map(Literal(_)))), + "Cannot use null as map key") + + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + CreateMap(Seq(Literal(1), Literal(2), Literal(1), Literal(3))), + create_map(1 -> 3)) + // ArrayType map key and value val map = CreateMap(Seq( Literal.create(intSeq, ArrayType(IntegerType, containsNull = false)), Literal.create(strSeq, ArrayType(StringType, containsNull = false)), @@ -202,15 +232,21 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { ArrayType(IntegerType, containsNull = true), ArrayType(StringType, containsNull = true), valueContainsNull = false)) - checkEvaluation(map, createMap(Seq(intSeq, intSeq :+ null), Seq(strSeq, strSeq :+ null))) + checkEvaluation(map, create_map(intSeq -> strSeq, (intSeq :+ null) -> (strSeq :+ null))) + + // map key can't be map + val map2 = CreateMap(Seq( + Literal.create(create_map(1 -> 1), MapType(IntegerType, IntegerType)), + Literal(1) + )) + map2.checkInputDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("The key of map cannot be/contain map")) + } } test("MapFromArrays") { - def createMap(keys: Seq[Any], values: Seq[Any]): Map[Any, Any] = { - // catalyst map is order-sensitive, so we create ListMap here to preserve the elements order. - scala.collection.immutable.ListMap(keys.zip(values): _*) - } - val intSeq = Seq(5, 10, 15, 20, 25) val longSeq = intSeq.map(_.toLong) val strSeq = intSeq.map(_.toString) @@ -228,24 +264,39 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val nullArray = Literal.create(null, ArrayType(StringType, false)) - checkEvaluation(MapFromArrays(intArray, longArray), createMap(intSeq, longSeq)) - checkEvaluation(MapFromArrays(intArray, strArray), createMap(intSeq, strSeq)) - checkEvaluation(MapFromArrays(integerArray, strArray), createMap(integerSeq, strSeq)) + checkEvaluation(MapFromArrays(intArray, longArray), create_map(intSeq, longSeq)) + checkEvaluation(MapFromArrays(intArray, strArray), create_map(intSeq, strSeq)) + checkEvaluation(MapFromArrays(integerArray, strArray), create_map(integerSeq, strSeq)) checkEvaluation( - MapFromArrays(strArray, intWithNullArray), createMap(strSeq, intWithNullSeq)) + MapFromArrays(strArray, intWithNullArray), create_map(strSeq, intWithNullSeq)) checkEvaluation( - MapFromArrays(strArray, longWithNullArray), createMap(strSeq, longWithNullSeq)) + MapFromArrays(strArray, longWithNullArray), create_map(strSeq, longWithNullSeq)) checkEvaluation( - MapFromArrays(strArray, longWithNullArray), createMap(strSeq, longWithNullSeq)) + MapFromArrays(strArray, longWithNullArray), create_map(strSeq, longWithNullSeq)) checkEvaluation(MapFromArrays(nullArray, nullArray), null) - intercept[RuntimeException] { - checkEvaluation(MapFromArrays(intWithNullArray, strArray), null) - } - intercept[RuntimeException] { - checkEvaluation( - MapFromArrays(intArray, Literal.create(Seq(1), ArrayType(IntegerType))), null) + // Map key can't be null + checkExceptionInExpression[RuntimeException]( + MapFromArrays(intWithNullArray, strArray), + "Cannot use null as map key") + + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + MapFromArrays( + Literal.create(Seq(1, 1), ArrayType(IntegerType)), + Literal.create(Seq(2, 3), ArrayType(IntegerType))), + create_map(1 -> 3)) + + // map key can't be map + val arrayOfMap = Seq(create_map(1 -> "a", 2 -> "b")) + val map = MapFromArrays( + Literal.create(arrayOfMap, ArrayType(MapType(IntegerType, StringType))), + Literal.create(Seq(1), ArrayType(IntegerType))) + map.checkInputDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("The key of map cannot be/contain map")) } } @@ -349,6 +400,11 @@ class ComplexTypeSuite extends SparkFunSuite with ExpressionEvalHelper { val m5 = Map("a" -> null) checkEvaluation(new StringToMap(s5), m5) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation( + new StringToMap(Literal("a:1,b:2,a:3")), + create_map("a" -> "3", "b" -> "2")) + // arguments checking assert(new StringToMap(Literal("a:1,b:2,c:3")).checkInputDataTypes().isSuccess) assert(new StringToMap(Literal(null)).checkInputDataTypes().isFailure) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala index f489d330cf453..51ba7c95b3146 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ConditionalExpressionSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.types._ @@ -90,27 +91,27 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5)), c6), "c", row) checkEvaluation(CaseWhen(Seq((c1, c4), (c2, c5))), null, row) - assert(CaseWhen(Seq((c2, c4)), c6).nullable === true) - assert(CaseWhen(Seq((c2, c4), (c3, c5)), c6).nullable === true) - assert(CaseWhen(Seq((c2, c4), (c3, c5))).nullable === true) + assert(CaseWhen(Seq((c2, c4)), c6).nullable) + assert(CaseWhen(Seq((c2, c4), (c3, c5)), c6).nullable) + assert(CaseWhen(Seq((c2, c4), (c3, c5))).nullable) val c4_notNull = 'a.boolean.notNull.at(3) val c5_notNull = 'a.boolean.notNull.at(4) val c6_notNull = 'a.boolean.notNull.at(5) assert(CaseWhen(Seq((c2, c4_notNull)), c6_notNull).nullable === false) - assert(CaseWhen(Seq((c2, c4)), c6_notNull).nullable === true) - assert(CaseWhen(Seq((c2, c4_notNull))).nullable === true) - assert(CaseWhen(Seq((c2, c4_notNull)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4)), c6_notNull).nullable) + assert(CaseWhen(Seq((c2, c4_notNull))).nullable) + assert(CaseWhen(Seq((c2, c4_notNull)), c6).nullable) assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6_notNull).nullable === false) - assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull)), c6_notNull).nullable === true) - assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5)), c6_notNull).nullable === true) - assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6).nullable === true) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull)), c6_notNull).nullable) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5)), c6_notNull).nullable) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull)), c6).nullable) - assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull))).nullable === true) - assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull))).nullable === true) - assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5))).nullable === true) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5_notNull))).nullable) + assert(CaseWhen(Seq((c2, c4), (c3, c5_notNull))).nullable) + assert(CaseWhen(Seq((c2, c4_notNull), (c3, c5))).nullable) } test("if/case when - null flags of non-primitive types") { @@ -222,4 +223,53 @@ class ConditionalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper CaseWhen(Seq((Literal.create(false, BooleanType), Literal(1))), Literal(-1)).genCode(ctx) assert(ctx.inlinedMutableStates.size == 1) } + + test("SPARK-27551: informative error message of mismatched types for case when") { + val caseVal1 = Literal.create( + create_row(1), + StructType(Seq(StructField("x", IntegerType, false)))) + val caseVal2 = Literal.create( + create_row(1), + StructType(Seq(StructField("y", IntegerType, false)))) + val elseVal = Literal.create( + create_row(1), + StructType(Seq(StructField("z", IntegerType, false)))) + + val checkResult1 = CaseWhen(Seq((Literal.FalseLiteral, caseVal1), + (Literal.FalseLiteral, caseVal2))).checkInputDataTypes() + assert(checkResult1.isInstanceOf[TypeCheckResult.TypeCheckFailure]) + assert(checkResult1.asInstanceOf[TypeCheckResult.TypeCheckFailure].message + .contains("CASE WHEN ... THEN struct WHEN ... THEN struct END")) + + val checkResult2 = CaseWhen(Seq((Literal.FalseLiteral, caseVal1), + (Literal.FalseLiteral, caseVal2)), Some(elseVal)).checkInputDataTypes() + assert(checkResult2.isInstanceOf[TypeCheckResult.TypeCheckFailure]) + assert(checkResult2.asInstanceOf[TypeCheckResult.TypeCheckFailure].message + .contains("CASE WHEN ... THEN struct WHEN ... THEN struct " + + "ELSE struct END")) + } + + test("SPARK-27917 test semantic equals of CaseWhen") { + val attrRef = AttributeReference("ACCESS_CHECK", StringType)() + val aliasAttrRef = attrRef.withName("access_check") + // Test for Equality + var caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A")))) + var caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A")))) + assert(caseWhenObj1.semanticEquals(caseWhenObj2)) + assert(caseWhenObj2.semanticEquals(caseWhenObj1)) + // Test for inEquality + caseWhenObj2 = CaseWhen(Seq((attrRef, Literal("a")))) + assert(!caseWhenObj1.semanticEquals(caseWhenObj2)) + assert(!caseWhenObj2.semanticEquals(caseWhenObj1)) + // Test with elseValue with Equality + caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))), attrRef.withName("ELSEVALUE")) + caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A"))), aliasAttrRef.withName("elsevalue")) + assert(caseWhenObj1.semanticEquals(caseWhenObj2)) + assert(caseWhenObj2.semanticEquals(caseWhenObj1)) + caseWhenObj1 = CaseWhen(Seq((attrRef, Literal("A"))), Literal("ELSEVALUE")) + caseWhenObj2 = CaseWhen(Seq((aliasAttrRef, Literal("A"))), Literal("elsevalue")) + // Test with elseValue with inEquality + assert(!caseWhenObj1.semanticEquals(caseWhenObj2)) + assert(!caseWhenObj2.semanticEquals(caseWhenObj1)) + } } 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 new file mode 100644 index 0000000000000..98c93a4946f4f --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/CsvExpressionsSuite.scala @@ -0,0 +1,240 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import java.text.SimpleDateFormat +import java.util.{Calendar, Locale} + +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.SparkFunSuite +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.types._ +import org.apache.spark.unsafe.types.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), + InternalRow(1) + ) + } + + test("from_csv - invalid data") { + val csvData = "---" + val schema = StructType(StructField("a", DoubleType) :: Nil) + checkEvaluation( + CsvToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(csvData), gmtId), + InternalRow(null)) + + // Default mode is Permissive + checkEvaluation(CsvToStructs(schema, Map.empty, Literal(csvData), gmtId), 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), + null + ) + } + + test("from_csv bad UTF-8") { + val schema = StructType(StructField("a", IntegerType) :: Nil) + checkEvaluation( + CsvToStructs(schema, Map.empty, Literal(badCsv), gmtId), + InternalRow(null)) + } + + test("from_csv with timestamp") { + val schema = StructType(StructField("t", TimestampType) :: Nil) + + val csvData1 = "2016-01-01T00:00:00.123Z" + var c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + c.set(2016, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 123) + checkEvaluation( + CsvToStructs(schema, Map.empty, Literal(csvData1), gmtId), + 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")), + InternalRow(c.getTimeInMillis * 1000L) + ) + + val csvData2 = "2016-01-01T00:00:00" + for (tz <- DateTimeTestUtils.outstandingTimezones) { + c = Calendar.getInstance(tz) + c.set(2016, 0, 1, 0, 0, 0) + c.set(Calendar.MILLISECOND, 0) + checkEvaluation( + CsvToStructs( + schema, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss"), + Literal(csvData2), + Option(tz.getID)), + InternalRow(c.getTimeInMillis * 1000L) + ) + checkEvaluation( + CsvToStructs( + schema, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", + DateTimeUtils.TIMEZONE_OPTION -> tz.getID), + Literal(csvData2), + gmtId), + InternalRow(c.getTimeInMillis * 1000L) + ) + } + } + + test("from_csv empty input column") { + val schema = StructType(StructField("a", IntegerType) :: Nil) + checkEvaluation( + CsvToStructs(schema, Map.empty, Literal.create(" ", StringType), gmtId), + InternalRow(null) + ) + } + + test("forcing schema nullability") { + val input = """1,,"foo"""" + val csvSchema = new StructType() + .add("a", LongType, nullable = false) + .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) + checkEvaluation(expr, output) + val schema = expr.dataType + val schemaToCompare = csvSchema.asNullable + assert(schemaToCompare == schema) + } + + + test("from_csv missing columns") { + val schema = new StructType() + .add("a", IntegerType) + .add("b", IntegerType) + checkEvaluation( + CsvToStructs(schema, Map.empty, Literal.create("1"), gmtId), + InternalRow(1, null) + ) + } + + test("unsupported mode") { + val csvData = "---" + val schema = StructType(StructField("a", DoubleType) :: Nil) + val exception = intercept[TestFailedException] { + checkEvaluation( + CsvToStructs(schema, Map("mode" -> DropMalformedMode.name), Literal(csvData), gmtId), + InternalRow(null)) + }.getCause + assert(exception.getMessage.contains("from_csv() doesn't support the DROPMALFORMED mode")) + } + + test("infer schema of CSV strings") { + checkEvaluation(new SchemaOfCsv(Literal.create("1,abc")), "struct<_c0:int,_c1:string>") + } + + test("infer schema of CSV strings by using options") { + checkEvaluation( + new SchemaOfCsv(Literal.create("1|abc"), Map("delimiter" -> "|")), + "struct<_c0:int,_c1:string>") + } + + 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") + } + + 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), + null + ) + } + + test("to_csv with timestamp") { + val schema = StructType(StructField("t", TimestampType) :: Nil) + val c = Calendar.getInstance(DateTimeUtils.TimeZoneGMT) + c.set(2016, 0, 1, 0, 0, 0) + 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, Option("PST")), "2015-12-31T16:00:00.000-08:00") + + checkEvaluation( + StructsToCsv( + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", + DateTimeUtils.TIMEZONE_OPTION -> gmtId.get), + struct, + gmtId), + "2016-01-01T00:00:00" + ) + checkEvaluation( + StructsToCsv( + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss", + DateTimeUtils.TIMEZONE_OPTION -> "PST"), + struct, + gmtId), + "2015-12-31T16:00:00" + ) + } + + test("parse date with locale") { + Seq("en-US", "ru-RU").foreach { langTag => + val locale = Locale.forLanguageTag(langTag) + val date = new SimpleDateFormat("yyyy-MM-dd").parse("2018-11-05") + val schema = new StructType().add("d", DateType) + val dateFormat = "MMM yyyy" + val sdf = new SimpleDateFormat(dateFormat, locale) + val dateStr = sdf.format(date) + val options = Map("dateFormat" -> dateFormat, "locale" -> langTag) + + checkEvaluation( + CsvToStructs(schema, options, Literal.create(dateStr), gmtId), + InternalRow(17836)) // number of days from 1970-01-01 + } + } + + test("verify corrupt column") { + checkExceptionInExpression[AnalysisException]( + CsvToStructs( + schema = StructType.fromDDL("i int, _unparsed boolean"), + options = Map("columnNameOfCorruptRecord" -> "_unparsed"), + child = Literal.create("a"), + timeZoneId = gmtId), + 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 c9d733726ff2c..4e8322d3c55d7 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,15 +19,21 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.time.ZoneOffset import java.util.{Calendar, Locale, TimeZone} +import java.util.concurrent.TimeUnit +import java.util.concurrent.TimeUnit._ import org.apache.spark.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.util.{DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneGMT +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -40,12 +46,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val pstId = Option(TimeZonePST.getID) val jstId = Option(TimeZoneJST.getID) - val sdf = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) - sdf.setTimeZone(TimeZoneGMT) - val sdfDate = new SimpleDateFormat("yyyy-MM-dd", Locale.US) - sdfDate.setTimeZone(TimeZoneGMT) - val d = new Date(sdf.parse("2015-04-08 13:10:15").getTime) - val ts = new Timestamp(sdf.parse("2013-11-08 13:10:15").getTime) + def toMillis(timestamp: String): Long = { + val tf = TimestampFormatter("yyyy-MM-dd HH:mm:ss", ZoneOffset.UTC) + TimeUnit.MICROSECONDS.toMillis(tf.parse(timestamp)) + } + val date = "2015-04-08 13:10:15" + val d = new Date(toMillis(date)) + val time = "2013-11-08 13:10:15" + val ts = new Timestamp(toMillis(time)) test("datetime function current_date") { val d0 = DateTimeUtils.millisToDays(System.currentTimeMillis(), TimeZoneGMT) @@ -78,15 +86,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } checkEvaluation(DayOfYear(Literal.create(null, DateType)), null) - checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 288) - checkEvaluation(DayOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 277) + checkEvaluation(DayOfYear(Cast(Literal("1582-10-15 13:10:15"), DateType)), 288) + checkEvaluation(DayOfYear(Cast(Literal("1582-10-04 13:10:15"), DateType)), 277) checkConsistencyBetweenInterpretedAndCodegen(DayOfYear, DateType) } test("Year") { checkEvaluation(Year(Literal.create(null, DateType)), null) checkEvaluation(Year(Literal(d)), 2015) - checkEvaluation(Year(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2015) + checkEvaluation(Year(Cast(Literal(date), DateType, gmtId)), 2015) checkEvaluation(Year(Cast(Literal(ts), DateType, gmtId)), 2013) val c = Calendar.getInstance() @@ -100,15 +108,15 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } } - checkEvaluation(Year(Literal(new Date(sdf.parse("1582-01-01 13:10:15").getTime))), 1582) - checkEvaluation(Year(Literal(new Date(sdf.parse("1581-12-31 13:10:15").getTime))), 1581) + checkEvaluation(Year(Cast(Literal("1582-01-01 13:10:15"), DateType)), 1582) + checkEvaluation(Year(Cast(Literal("1581-12-31 13:10:15"), DateType)), 1581) checkConsistencyBetweenInterpretedAndCodegen(Year, DateType) } test("Quarter") { checkEvaluation(Quarter(Literal.create(null, DateType)), null) checkEvaluation(Quarter(Literal(d)), 2) - checkEvaluation(Quarter(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 2) + checkEvaluation(Quarter(Cast(Literal(date), DateType, gmtId)), 2) checkEvaluation(Quarter(Cast(Literal(ts), DateType, gmtId)), 4) val c = Calendar.getInstance() @@ -123,20 +131,20 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } } - checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-10-01 13:10:15").getTime))), 4) - checkEvaluation(Quarter(Literal(new Date(sdf.parse("1582-09-30 13:10:15").getTime))), 3) + checkEvaluation(Quarter(Cast(Literal("1582-10-01 13:10:15"), DateType)), 4) + checkEvaluation(Quarter(Cast(Literal("1582-09-30 13:10:15"), DateType)), 3) checkConsistencyBetweenInterpretedAndCodegen(Quarter, DateType) } test("Month") { checkEvaluation(Month(Literal.create(null, DateType)), null) checkEvaluation(Month(Literal(d)), 4) - checkEvaluation(Month(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 4) + checkEvaluation(Month(Cast(Literal(date), DateType, gmtId)), 4) checkEvaluation(Month(Cast(Literal(ts), DateType, gmtId)), 11) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 4) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 10) - checkEvaluation(Month(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 10) + checkEvaluation(Month(Cast(Literal("1582-04-28 13:10:15"), DateType)), 4) + checkEvaluation(Month(Cast(Literal("1582-10-04 13:10:15"), DateType)), 10) + checkEvaluation(Month(Cast(Literal("1582-10-15 13:10:15"), DateType)), 10) val c = Calendar.getInstance() (2003 to 2004).foreach { y => @@ -156,12 +164,12 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(DayOfMonth(Cast(Literal("2000-02-29"), DateType)), 29) checkEvaluation(DayOfMonth(Literal.create(null, DateType)), null) checkEvaluation(DayOfMonth(Literal(d)), 8) - checkEvaluation(DayOfMonth(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 8) + checkEvaluation(DayOfMonth(Cast(Literal(date), DateType, gmtId)), 8) checkEvaluation(DayOfMonth(Cast(Literal(ts), DateType, gmtId)), 8) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-04-28 13:10:15").getTime))), 28) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 15) - checkEvaluation(DayOfMonth(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 4) + checkEvaluation(DayOfMonth(Cast(Literal("1582-04-28 13:10:15"), DateType)), 28) + checkEvaluation(DayOfMonth(Cast(Literal("1582-10-15 13:10:15"), DateType)), 15) + checkEvaluation(DayOfMonth(Cast(Literal("1582-10-04 13:10:15"), DateType)), 4) val c = Calendar.getInstance() (1999 to 2000).foreach { y => @@ -177,9 +185,9 @@ 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 === true) + assert(Second(Cast(Literal(d), TimestampType, gmtId), gmtId).resolved ) checkEvaluation(Second(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Second(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 15) + checkEvaluation(Second(Cast(Literal(date), TimestampType, gmtId), gmtId), 15) checkEvaluation(Second(Literal(ts), gmtId), 15) val c = Calendar.getInstance() @@ -200,13 +208,13 @@ 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(sdfDate.format(d)), DateType, gmtId)), + checkEvaluation(DayOfWeek(Cast(Literal(date), DateType, gmtId)), Calendar.WEDNESDAY) checkEvaluation(DayOfWeek(Cast(Literal(ts), DateType, gmtId)), Calendar.FRIDAY) checkEvaluation(DayOfWeek(Cast(Literal("2011-05-06"), DateType, gmtId)), Calendar.FRIDAY) - checkEvaluation(DayOfWeek(Literal(new Date(sdf.parse("2017-05-27 13:10:15").getTime))), + checkEvaluation(DayOfWeek(Literal(new Date(toMillis("2017-05-27 13:10:15")))), Calendar.SATURDAY) - checkEvaluation(DayOfWeek(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), + checkEvaluation(DayOfWeek(Literal(new Date(toMillis("1582-10-15 13:10:15")))), Calendar.FRIDAY) checkConsistencyBetweenInterpretedAndCodegen(DayOfWeek, DateType) } @@ -214,22 +222,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("WeekDay") { checkEvaluation(WeekDay(Literal.create(null, DateType)), null) checkEvaluation(WeekDay(Literal(d)), 2) - checkEvaluation(WeekDay(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 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(Literal(new Date(sdf.parse("2017-05-27 13:10:15").getTime))), 5) - checkEvaluation(WeekDay(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 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) } test("WeekOfYear") { checkEvaluation(WeekOfYear(Literal.create(null, DateType)), null) checkEvaluation(WeekOfYear(Literal(d)), 15) - checkEvaluation(WeekOfYear(Cast(Literal(sdfDate.format(d)), DateType, gmtId)), 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(Literal(new Date(sdf.parse("1582-10-15 13:10:15").getTime))), 40) - checkEvaluation(WeekOfYear(Literal(new Date(sdf.parse("1582-10-04 13:10:15").getTime))), 40) + 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) checkConsistencyBetweenInterpretedAndCodegen(WeekOfYear, DateType) } @@ -260,13 +268,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(DateFormatClass(Cast(Literal(d), TimestampType, jstId), Literal("H"), jstId), "0") checkEvaluation(DateFormatClass(Literal(ts), Literal("H"), jstId), "22") + + // 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), + expected = "22", + inputRow = InternalRow(DateTimeUtils.fromJavaTimestamp(ts), UTF8String.fromString("H"))) } test("Hour") { assert(Hour(Literal.create(null, DateType), gmtId).resolved === false) - assert(Hour(Literal(ts), gmtId).resolved === true) + assert(Hour(Literal(ts), gmtId).resolved) checkEvaluation(Hour(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) - checkEvaluation(Hour(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 13) + checkEvaluation(Hour(Cast(Literal(date), TimestampType, gmtId), gmtId), 13) checkEvaluation(Hour(Literal(ts), gmtId), 13) val c = Calendar.getInstance() @@ -290,10 +307,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("Minute") { assert(Minute(Literal.create(null, DateType), gmtId).resolved === false) - assert(Minute(Literal(ts), gmtId).resolved === true) + assert(Minute(Literal(ts), gmtId).resolved) checkEvaluation(Minute(Cast(Literal(d), TimestampType, gmtId), gmtId), 0) checkEvaluation( - Minute(Cast(Literal(sdf.format(d)), TimestampType, gmtId), gmtId), 10) + Minute(Cast(Literal(date), TimestampType, gmtId), gmtId), 10) checkEvaluation(Minute(Literal(ts), gmtId), 10) val c = Calendar.getInstance() @@ -445,13 +462,19 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(AddMonths(Literal.create(null, DateType), Literal(1)), null) checkEvaluation(AddMonths(Literal.create(null, DateType), Literal.create(null, IntegerType)), null) + // Valid range of DateType is [0001-01-01, 9999-12-31] + val maxMonthInterval = 10000 * 12 checkEvaluation( - AddMonths(Literal(Date.valueOf("2015-01-30")), Literal(Int.MinValue)), -7293498) - checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), positiveIntLit), 1014213) + AddMonths(Literal(Date.valueOf("0001-01-01")), Literal(maxMonthInterval)), 2933261) checkEvaluation( - AddMonths(Literal(Date.valueOf("2016-02-28")), negativeIntLit), -980528) - checkConsistencyBetweenInterpretedAndCodegen(AddMonths, DateType, IntegerType) + AddMonths(Literal(Date.valueOf("9999-12-31")), Literal(-1 * maxMonthInterval)), -719529) + // Test evaluation results between Interpreted mode and Codegen mode + forAll ( + LiteralGenerator.randomGen(DateType), + LiteralGenerator.monthIntervalLiterGen + ) { (l1: Literal, l2: Literal) => + cmpInterpretWithCodegen(EmptyRow, AddMonths(l1, l2)) + } } test("months_between") { @@ -645,7 +668,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("from_unixtime") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt1 = "yyyy-MM-dd HH:mm:ss" + 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)) { @@ -654,10 +678,10 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { sdf2.setTimeZone(tz) checkEvaluation( - FromUnixTime(Literal(0L), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + FromUnixTime(Literal(0L), Literal(fmt1), timeZoneId), sdf1.format(new Timestamp(0))) checkEvaluation(FromUnixTime( - Literal(1000L), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + Literal(1000L), Literal(fmt1), timeZoneId), sdf1.format(new Timestamp(1000000))) checkEvaluation( FromUnixTime(Literal(-1000L), Literal(fmt2), timeZoneId), @@ -666,13 +690,22 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { FromUnixTime(Literal.create(null, LongType), Literal.create(null, StringType), timeZoneId), null) checkEvaluation( - FromUnixTime(Literal.create(null, LongType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + FromUnixTime(Literal.create(null, LongType), Literal(fmt1), timeZoneId), null) checkEvaluation( FromUnixTime(Literal(1000L), Literal.create(null, StringType), timeZoneId), null) checkEvaluation( FromUnixTime(Literal(0L), Literal("not a valid format"), timeZoneId), null) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = FromUnixTime( + BoundReference(ordinal = 0, dataType = LongType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = UTF8String.fromString(sdf1.format(new Timestamp(0))), + inputRow = InternalRow(0L, UTF8String.fromString(fmt1))) } } @@ -702,14 +735,14 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { 1000L) checkEvaluation( UnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) checkEvaluation( UnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), -1000L) checkEvaluation(UnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), - DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz) / 1000L) + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) val t1 = UnixTimestamp( CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] val t2 = UnixTimestamp( @@ -724,7 +757,7 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { null) checkEvaluation( UnixTimestamp(Literal(date1), Literal.create(null, StringType), timeZoneId), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) checkEvaluation( UnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) } @@ -732,7 +765,8 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("to_unix_timestamp") { - val sdf1 = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) + val fmt1 = "yyyy-MM-dd HH:mm:ss" + val sdf1 = new SimpleDateFormat(fmt1, Locale.US) val fmt2 = "yyyy-MM-dd HH:mm:ss.SSS" val sdf2 = new SimpleDateFormat(fmt2, Locale.US) val fmt3 = "yy-MM-dd" @@ -747,39 +781,49 @@ class DateExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val date1 = Date.valueOf("2015-07-24") checkEvaluation(ToUnixTimestamp( - Literal(sdf1.format(new Timestamp(0))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), 0L) + Literal(sdf1.format(new Timestamp(0))), Literal(fmt1), timeZoneId), 0L) checkEvaluation(ToUnixTimestamp( - Literal(sdf1.format(new Timestamp(1000000))), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + Literal(sdf1.format(new Timestamp(1000000))), Literal(fmt1), timeZoneId), 1000L) checkEvaluation(ToUnixTimestamp( - Literal(new Timestamp(1000000)), Literal("yyyy-MM-dd HH:mm:ss")), + Literal(new Timestamp(1000000)), Literal(fmt1)), 1000L) checkEvaluation( - ToUnixTimestamp(Literal(date1), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + ToUnixTimestamp(Literal(date1), Literal(fmt1), timeZoneId), + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) checkEvaluation( ToUnixTimestamp(Literal(sdf2.format(new Timestamp(-1000000))), Literal(fmt2), timeZoneId), -1000L) checkEvaluation(ToUnixTimestamp( Literal(sdf3.format(Date.valueOf("2015-07-24"))), Literal(fmt3), timeZoneId), - DateTimeUtils.daysToMillis( - DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz) / 1000L) + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis( + DateTimeUtils.fromJavaDate(Date.valueOf("2015-07-24")), tz))) val t1 = ToUnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] val t2 = ToUnixTimestamp( - CurrentTimestamp(), Literal("yyyy-MM-dd HH:mm:ss")).eval().asInstanceOf[Long] + CurrentTimestamp(), Literal(fmt1)).eval().asInstanceOf[Long] assert(t2 - t1 <= 1) checkEvaluation(ToUnixTimestamp( Literal.create(null, DateType), Literal.create(null, StringType), timeZoneId), null) checkEvaluation( ToUnixTimestamp( - Literal.create(null, DateType), Literal("yyyy-MM-dd HH:mm:ss"), timeZoneId), + Literal.create(null, DateType), Literal(fmt1), timeZoneId), null) checkEvaluation(ToUnixTimestamp( Literal(date1), Literal.create(null, StringType), timeZoneId), - DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz) / 1000L) + MILLISECONDS.toSeconds(DateTimeUtils.daysToMillis(DateTimeUtils.fromJavaDate(date1), tz))) checkEvaluation( ToUnixTimestamp(Literal("2015-07-24"), Literal("not a valid format"), timeZoneId), null) + + // SPARK-28072 The codegen path for non-literal input should also work + checkEvaluation( + expression = ToUnixTimestamp( + BoundReference(ordinal = 0, dataType = StringType, nullable = true), + BoundReference(ordinal = 1, dataType = StringType, nullable = true), + timeZoneId), + expected = 0L, + inputRow = InternalRow( + UTF8String.fromString(sdf1.format(new Timestamp(0))), UTF8String.fromString(fmt1))) } } } @@ -811,14 +855,30 @@ 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(null, "UTC", null) - test("2015-07-24 00:00:00", null, null) - test(null, null, null) - // Test escaping of timezone - GenerateUnsafeProjection.generate( - ToUTCTimestamp(Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal("\"quote")) :: Nil) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + 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(null, "UTC", null) + test("2015-07-24 00:00:00", null, null) + test(null, null, null) + } + val msg = intercept[AnalysisException] { + test("2015-07-24 00:00:00", "PST", "2015-07-24 07:00:00") + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + } + + test("to_utc_timestamp - invalid time zone id") { + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => + val msg = intercept[java.time.DateTimeException] { + GenerateUnsafeProjection.generate( + ToUTCTimestamp( + Literal(Timestamp.valueOf("2015-07-24 00:00:00")), Literal(invalidTz)) :: Nil) + }.getMessage + assert(msg.contains(invalidTz)) + } + } } test("from_utc_timestamp") { @@ -834,12 +894,28 @@ 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(null, "UTC", null) - test("2015-07-24 00:00:00", null, null) - test(null, null, null) - // Test escaping of timezone - GenerateUnsafeProjection.generate(FromUTCTimestamp(Literal(0), Literal("\"quote")) :: Nil) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + 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(null, "UTC", null) + test("2015-07-24 00:00:00", null, null) + test(null, null, null) + } + val msg = intercept[AnalysisException] { + test("2015-07-24 00:00:00", "PST", "2015-07-23 17:00:00") + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) + } + + test("from_utc_timestamp - invalid time zone id") { + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + Seq("Invalid time zone", "\"quote", "UTC*42").foreach { invalidTz => + val msg = intercept[java.time.DateTimeException] { + GenerateUnsafeProjection.generate( + FromUTCTimestamp(Literal(0), Literal(invalidTz)) :: Nil) + }.getMessage + assert(msg.contains(invalidTz)) + } + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala index a8f758d625a02..fc5e8dc5ee7f1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/DecimalExpressionSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{Decimal, DecimalType, LongType} class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -31,8 +32,23 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { } test("MakeDecimal") { - checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) - checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "true") { + checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) + checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) + val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) + checkEvaluation(overflowExpr, null) + checkEvaluationWithMutableProjection(overflowExpr, null) + evaluateWithoutCodegen(overflowExpr, null) + checkEvaluationWithUnsafeProjection(overflowExpr, null) + } + withSQLConf(SQLConf.DECIMAL_OPERATIONS_NULL_ON_OVERFLOW.key -> "false") { + checkEvaluation(MakeDecimal(Literal(101L), 3, 1), Decimal("10.1")) + checkEvaluation(MakeDecimal(Literal.create(null, LongType), 3, 1), null) + val overflowExpr = MakeDecimal(Literal.create(1000L, LongType), 3, 1) + intercept[ArithmeticException](checkEvaluationWithMutableProjection(overflowExpr, null)) + intercept[ArithmeticException](evaluateWithoutCodegen(overflowExpr, null)) + intercept[ArithmeticException](checkEvaluationWithUnsafeProjection(overflowExpr, null)) + } } test("PromotePrecision") { @@ -45,18 +61,26 @@ class DecimalExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { test("CheckOverflow") { val d1 = Decimal("10.1") - checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 0)), Decimal("10")) - checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 1)), d1) - checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 2)), d1) - checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 3)), null) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 0), true), Decimal("10")) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 1), true), d1) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 2), true), d1) + checkEvaluation(CheckOverflow(Literal(d1), DecimalType(4, 3), true), null) + intercept[ArithmeticException](CheckOverflow(Literal(d1), DecimalType(4, 3), false).eval()) + intercept[ArithmeticException](checkEvaluationWithMutableProjection( + CheckOverflow(Literal(d1), DecimalType(4, 3), false), null)) val d2 = Decimal(101, 3, 1) - checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 0)), Decimal("10")) - checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 1)), d2) - checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 2)), d2) - checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 3)), null) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 0), true), Decimal("10")) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 1), true), d2) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 2), true), d2) + checkEvaluation(CheckOverflow(Literal(d2), DecimalType(4, 3), true), null) + intercept[ArithmeticException](CheckOverflow(Literal(d2), DecimalType(4, 3), false).eval()) + intercept[ArithmeticException](checkEvaluationWithMutableProjection( + CheckOverflow(Literal(d2), DecimalType(4, 3), false), null)) - checkEvaluation(CheckOverflow(Literal.create(null, DecimalType(2, 1)), DecimalType(3, 2)), null) + checkEvaluation(CheckOverflow( + Literal.create(null, DecimalType(2, 1)), DecimalType(3, 2), true), null) + checkEvaluation(CheckOverflow( + Literal.create(null, DecimalType(2, 1)), DecimalType(3, 2), false), null) } - } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala index da18475276a13..a2c0ce35df23c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvalHelper.scala @@ -28,12 +28,12 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.JavaSerializer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} -import org.apache.spark.sql.catalyst.analysis.{ResolveTimeZone, SimpleAnalyzer} +import org.apache.spark.sql.catalyst.analysis.ResolveTimeZone import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.optimizer.SimpleTestOptimizer import org.apache.spark.sql.catalyst.plans.PlanTestBase import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Project} -import org.apache.spark.sql.catalyst.util.{ArrayData, MapData} +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, ArrayData, MapData} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -48,10 +48,31 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa InternalRow.fromSeq(values.map(CatalystTypeConverters.convertToCatalyst)) } + // Currently MapData just stores the key and value arrays. Its equality is not well implemented, + // as the order of the map entries should not matter for equality. This method creates MapData + // with the entries ordering preserved, so that we can deterministically test expressions with + // map input/output. + protected def create_map(entries: (_, _)*): ArrayBasedMapData = { + create_map(entries.map(_._1), entries.map(_._2)) + } + + protected def create_map(keys: Seq[_], values: Seq[_]): ArrayBasedMapData = { + assert(keys.length == values.length) + val keyArray = CatalystTypeConverters + .convertToCatalyst(keys) + .asInstanceOf[ArrayData] + val valueArray = CatalystTypeConverters + .convertToCatalyst(values) + .asInstanceOf[ArrayData] + new ArrayBasedMapData(keyArray, valueArray) + } + private def prepareEvaluation(expression: Expression): Expression = { val serializer = new JavaSerializer(new SparkConf()).newInstance val resolver = ResolveTimeZone(new SQLConf) - resolver.resolveTimeZones(serializer.deserialize(serializer.serialize(expression))) + val expr = resolver.resolveTimeZones(expression) + assert(expr.resolved) + serializer.deserialize(serializer.serialize(expr)) } protected def checkEvaluation( @@ -277,9 +298,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa expected: Any, inputRow: InternalRow = EmptyRow): Unit = { val plan = Project(Alias(expression, s"Optimized($expression)")() :: Nil, OneRowRelation()) - // We should analyze the plan first, otherwise we possibly optimize an unresolved plan. - val analyzedPlan = SimpleAnalyzer.execute(plan) - val optimizedPlan = SimpleTestOptimizer.execute(analyzedPlan) + val optimizedPlan = SimpleTestOptimizer.execute(plan) checkEvaluationWithoutCodegen(optimizedPlan.expressions.head, expected, inputRow) } @@ -302,8 +321,8 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa GenerateUnsafeProjection.generate(Alias(expression, s"Optimized($expression)")() :: Nil), expression) plan.initialize(0) - actual = FromUnsafeProjection(expression.dataType :: Nil)( - plan(inputRow)).get(0, expression.dataType) + val ref = new BoundReference(0, expression.dataType, nullable = true) + actual = GenerateSafeProjection.generate(ref :: Nil)(plan(inputRow)).get(0, expression.dataType) assert(checkResult(actual, expected, expression)) } @@ -379,7 +398,7 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa } } - private def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { + def cmpInterpretWithCodegen(inputRow: InternalRow, expr: Expression): Unit = { val interpret = try { evaluateWithoutCodegen(expr, inputRow) } catch { @@ -437,4 +456,15 @@ trait ExpressionEvalHelper extends GeneratorDrivenPropertyChecks with PlanTestBa diff < eps * math.min(absX, absY) } } + + def testBothCodegenAndInterpreted(name: String)(f: => Unit): Unit = { + val modes = Seq(CodegenObjectFactoryMode.CODEGEN_ONLY, CodegenObjectFactoryMode.NO_CODEGEN) + for (fallbackMode <- modes) { + test(s"$name with $fallbackMode") { + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { + f + } + } + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala index 4281c89ac475d..b5cfaf8f4b0fd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HashExpressionsSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.charset.StandardCharsets -import java.util.TimeZone +import java.time.{ZoneId, ZoneOffset} import scala.collection.mutable.ArrayBuffer @@ -183,7 +183,7 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkHiveHashForDateType("2017-01-01", 17167) // boundary cases - checkHiveHashForDateType("0000-01-01", -719530) + checkHiveHashForDateType("0000-01-01", -719528) checkHiveHashForDateType("9999-12-31", 2932896) // epoch @@ -208,9 +208,9 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { def checkHiveHashForTimestampType( timestamp: String, expected: Long, - timeZone: TimeZone = TimeZone.getTimeZone("UTC")): Unit = { + zoneId: ZoneId = ZoneOffset.UTC): Unit = { checkHiveHash( - DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), timeZone).get, + DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), zoneId).get, TimestampType, expected) } @@ -223,10 +223,10 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // with different timezone checkHiveHashForTimestampType("2017-02-24 10:56:29", 1445732471, - TimeZone.getTimeZone("US/Pacific")) + DateTimeUtils.getZoneId("US/Pacific")) // boundary cases - checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645926784) + checkHiveHashForTimestampType("0001-01-01 00:00:00", 1645969984) checkHiveHashForTimestampType("9999-01-01 00:00:00", -1081818240) // epoch @@ -630,6 +630,11 @@ class HashExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val murmursHashEval = Murmur3Hash(exprs, 42).eval(input) assert(murmur3HashPlan(input).getInt(0) == murmursHashEval) + val xxHash64Expr = XxHash64(exprs, 42) + val xxHash64Plan = GenerateMutableProjection.generate(Seq(xxHash64Expr)) + val xxHash64Eval = XxHash64(exprs, 42).eval(input) + assert(xxHash64Plan(input).getLong(0) == xxHash64Eval) + val hiveHashExpr = HiveHash(exprs) val hiveHashPlan = GenerateMutableProjection.generate(Seq(hiveHashExpr)) val hiveHashEval = HiveHash(exprs).eval(input) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala index e13f4d98295be..1411be8007deb 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/HigherOrderFunctionsSuite.scala @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.ArrayBasedMapData +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper { @@ -254,13 +255,26 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val isEven: Expression => Expression = x => x % 2 === 0 val isNullOrOdd: Expression => Expression = x => x.isNull || x % 2 === 1 - - checkEvaluation(exists(ai0, isEven), true) - checkEvaluation(exists(ai0, isNullOrOdd), true) - checkEvaluation(exists(ai1, isEven), false) - checkEvaluation(exists(ai1, isNullOrOdd), true) - checkEvaluation(exists(ain, isEven), null) - checkEvaluation(exists(ain, isNullOrOdd), null) + val alwaysFalse: Expression => Expression = _ => Literal.FalseLiteral + val alwaysNull: Expression => Expression = _ => Literal(null, BooleanType) + + for (followThreeValuedLogic <- Seq(false, true)) { + withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key + -> followThreeValuedLogic.toString) { + checkEvaluation(exists(ai0, isEven), true) + checkEvaluation(exists(ai0, isNullOrOdd), true) + checkEvaluation(exists(ai0, alwaysFalse), false) + checkEvaluation(exists(ai0, alwaysNull), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(ai1, isEven), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(ai1, isNullOrOdd), true) + checkEvaluation(exists(ai1, alwaysFalse), false) + checkEvaluation(exists(ai1, alwaysNull), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(ain, isEven), null) + checkEvaluation(exists(ain, isNullOrOdd), null) + checkEvaluation(exists(ain, alwaysFalse), null) + checkEvaluation(exists(ain, alwaysNull), null) + } + } val as0 = Literal.create(Seq("a0", "b1", "a2", "c3"), ArrayType(StringType, containsNull = false)) @@ -269,9 +283,20 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val startsWithA: Expression => Expression = x => x.startsWith("a") - checkEvaluation(exists(as0, startsWithA), true) - checkEvaluation(exists(as1, startsWithA), false) - checkEvaluation(exists(asn, startsWithA), null) + for (followThreeValuedLogic <- Seq(false, true)) { + withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key + -> followThreeValuedLogic.toString) { + checkEvaluation(exists(as0, startsWithA), true) + checkEvaluation(exists(as0, alwaysFalse), false) + checkEvaluation(exists(as0, alwaysNull), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(as1, startsWithA), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(as1, alwaysFalse), false) + checkEvaluation(exists(as1, alwaysNull), if (followThreeValuedLogic) null else false) + checkEvaluation(exists(asn, startsWithA), null) + checkEvaluation(exists(asn, alwaysFalse), null) + checkEvaluation(exists(asn, alwaysNull), null) + } + } val aai = Literal.create(Seq(Seq(1, 2, 3), null, Seq(4, 5)), ArrayType(ArrayType(IntegerType, containsNull = false), containsNull = true)) @@ -310,13 +335,13 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper test("TransformKeys") { val ai0 = Literal.create( - Map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4), + create_map(1 -> 1, 2 -> 2, 3 -> 3, 4 -> 4), MapType(IntegerType, IntegerType, valueContainsNull = false)) val ai1 = Literal.create( Map.empty[Int, Int], MapType(IntegerType, IntegerType, valueContainsNull = true)) val ai2 = Literal.create( - Map(1 -> 1, 2 -> null, 3 -> 3), + create_map(1 -> 1, 2 -> null, 3 -> 3), MapType(IntegerType, IntegerType, valueContainsNull = true)) val ai3 = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) @@ -324,26 +349,27 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val plusValue: (Expression, Expression) => Expression = (k, v) => k + v val modKey: (Expression, Expression) => Expression = (k, v) => k % 3 - checkEvaluation(transformKeys(ai0, plusOne), Map(2 -> 1, 3 -> 2, 4 -> 3, 5 -> 4)) - checkEvaluation(transformKeys(ai0, plusValue), Map(2 -> 1, 4 -> 2, 6 -> 3, 8 -> 4)) + checkEvaluation(transformKeys(ai0, plusOne), create_map(2 -> 1, 3 -> 2, 4 -> 3, 5 -> 4)) + checkEvaluation(transformKeys(ai0, plusValue), create_map(2 -> 1, 4 -> 2, 6 -> 3, 8 -> 4)) checkEvaluation( - transformKeys(transformKeys(ai0, plusOne), plusValue), Map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4)) - checkEvaluation(transformKeys(ai0, modKey), - ArrayBasedMapData(Array(1, 2, 0, 1), Array(1, 2, 3, 4))) + transformKeys(transformKeys(ai0, plusOne), plusValue), + create_map(3 -> 1, 5 -> 2, 7 -> 3, 9 -> 4)) + // Duplicated map keys will be removed w.r.t. the last wins policy. + checkEvaluation(transformKeys(ai0, modKey), create_map(1 -> 4, 2 -> 2, 0 -> 3)) checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) checkEvaluation(transformKeys(ai1, plusOne), Map.empty[Int, Int]) checkEvaluation( transformKeys(transformKeys(ai1, plusOne), plusValue), Map.empty[Int, Int]) - checkEvaluation(transformKeys(ai2, plusOne), Map(2 -> 1, 3 -> null, 4 -> 3)) + checkEvaluation(transformKeys(ai2, plusOne), create_map(2 -> 1, 3 -> null, 4 -> 3)) checkEvaluation( - transformKeys(transformKeys(ai2, plusOne), plusOne), Map(3 -> 1, 4 -> null, 5 -> 3)) + transformKeys(transformKeys(ai2, plusOne), plusOne), create_map(3 -> 1, 4 -> null, 5 -> 3)) checkEvaluation(transformKeys(ai3, plusOne), null) val as0 = Literal.create( - Map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"), + create_map("a" -> "xy", "bb" -> "yz", "ccc" -> "zx"), MapType(StringType, StringType, valueContainsNull = false)) val as1 = Literal.create( - Map("a" -> "xy", "bb" -> "yz", "ccc" -> null), + create_map("a" -> "xy", "bb" -> "yz", "ccc" -> null), MapType(StringType, StringType, valueContainsNull = true)) val as2 = Literal.create(null, MapType(StringType, StringType, valueContainsNull = false)) @@ -355,26 +381,35 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper (k, v) => Length(k) + 1 checkEvaluation( - transformKeys(as0, concatValue), Map("axy" -> "xy", "bbyz" -> "yz", "ccczx" -> "zx")) + transformKeys(as0, concatValue), create_map("axy" -> "xy", "bbyz" -> "yz", "ccczx" -> "zx")) checkEvaluation( transformKeys(transformKeys(as0, concatValue), concatValue), - Map("axyxy" -> "xy", "bbyzyz" -> "yz", "ccczxzx" -> "zx")) + create_map("axyxy" -> "xy", "bbyzyz" -> "yz", "ccczxzx" -> "zx")) checkEvaluation(transformKeys(as3, concatValue), Map.empty[String, String]) checkEvaluation( transformKeys(transformKeys(as3, concatValue), convertKeyToKeyLength), Map.empty[Int, String]) checkEvaluation(transformKeys(as0, convertKeyToKeyLength), - Map(2 -> "xy", 3 -> "yz", 4 -> "zx")) + create_map(2 -> "xy", 3 -> "yz", 4 -> "zx")) checkEvaluation(transformKeys(as1, convertKeyToKeyLength), - Map(2 -> "xy", 3 -> "yz", 4 -> null)) + create_map(2 -> "xy", 3 -> "yz", 4 -> null)) checkEvaluation(transformKeys(as2, convertKeyToKeyLength), null) checkEvaluation(transformKeys(as3, convertKeyToKeyLength), Map.empty[Int, String]) val ax0 = Literal.create( - Map(1 -> "x", 2 -> "y", 3 -> "z"), + create_map(1 -> "x", 2 -> "y", 3 -> "z"), MapType(IntegerType, StringType, valueContainsNull = false)) - checkEvaluation(transformKeys(ax0, plusOne), Map(2 -> "x", 3 -> "y", 4 -> "z")) + checkEvaluation(transformKeys(ax0, plusOne), create_map(2 -> "x", 3 -> "y", 4 -> "z")) + + // map key can't be map + val makeMap: (Expression, Expression) => Expression = (k, v) => CreateMap(Seq(k, v)) + val map = transformKeys(ai0, makeMap) + map.checkInputDataTypes() match { + case TypeCheckResult.TypeCheckSuccess => fail("should not allow map as map key") + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("The key of map cannot be/contain map")) + } } test("TransformValues") { @@ -456,16 +491,13 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper .bind(validateBinding) } - val mii0 = Literal.create(Map(1 -> 10, 2 -> 20, 3 -> 30), + val mii0 = Literal.create(create_map(1 -> 10, 2 -> 20, 3 -> 30), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii1 = Literal.create(Map(1 -> -1, 2 -> -2, 4 -> -4), + val mii1 = Literal.create(create_map(1 -> -1, 2 -> -2, 4 -> -4), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii2 = Literal.create(Map(1 -> null, 2 -> -2, 3 -> null), + val mii2 = Literal.create(create_map(1 -> null, 2 -> -2, 3 -> null), MapType(IntegerType, IntegerType, valueContainsNull = true)) val mii3 = Literal.create(Map(), MapType(IntegerType, IntegerType, valueContainsNull = false)) - val mii4 = MapFromArrays( - Literal.create(Seq(2, 2), ArrayType(IntegerType, false)), - Literal.create(Seq(20, 200), ArrayType(IntegerType, false))) val miin = Literal.create(null, MapType(IntegerType, IntegerType, valueContainsNull = false)) val multiplyKeyWithValues: (Expression, Expression, Expression) => Expression = { @@ -481,12 +513,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mii0, mii3, multiplyKeyWithValues), Map(1 -> null, 2 -> null, 3 -> null)) - checkEvaluation( - map_zip_with(mii0, mii4, multiplyKeyWithValues), - Map(1 -> null, 2 -> 800, 3 -> null)) - checkEvaluation( - map_zip_with(mii4, mii0, multiplyKeyWithValues), - Map(2 -> 800, 1 -> null, 3 -> null)) checkEvaluation( map_zip_with(mii0, miin, multiplyKeyWithValues), null) @@ -500,9 +526,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val mss2 = Literal.create(Map("c" -> null, "b" -> "t", "a" -> null), MapType(StringType, StringType, valueContainsNull = true)) val mss3 = Literal.create(Map(), MapType(StringType, StringType, valueContainsNull = false)) - val mss4 = MapFromArrays( - Literal.create(Seq("a", "a"), ArrayType(StringType, false)), - Literal.create(Seq("a", "n"), ArrayType(StringType, false))) val mssn = Literal.create(null, MapType(StringType, StringType, valueContainsNull = false)) val concat: (Expression, Expression, Expression) => Expression = { @@ -518,12 +541,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mss0, mss3, concat), Map("a" -> null, "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss0, mss4, concat), - Map("a" -> "axa", "b" -> null, "d" -> null)) - checkEvaluation( - map_zip_with(mss4, mss0, concat), - Map("a" -> "aax", "b" -> null, "d" -> null)) checkEvaluation( map_zip_with(mss0, mssn, concat), null) @@ -539,9 +556,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper val mbb2 = Literal.create(Map(b(1, 3) -> null, b(1, 2) -> b(2), b(2, 1) -> null), MapType(BinaryType, BinaryType, valueContainsNull = true)) val mbb3 = Literal.create(Map(), MapType(BinaryType, BinaryType, valueContainsNull = false)) - val mbb4 = MapFromArrays( - Literal.create(Seq(b(2, 1), b(2, 1)), ArrayType(BinaryType, false)), - Literal.create(Seq(b(1), b(9)), ArrayType(BinaryType, false))) val mbbn = Literal.create(null, MapType(BinaryType, BinaryType, valueContainsNull = false)) checkEvaluation( @@ -553,12 +567,6 @@ class HigherOrderFunctionsSuite extends SparkFunSuite with ExpressionEvalHelper checkEvaluation( map_zip_with(mbb0, mbb3, concat), Map(b(1, 2) -> null, b(2, 1) -> null, b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb0, mbb4, concat), - Map(b(1, 2) -> null, b(2, 1) -> b(2, 1, 5, 1), b(1, 3) -> null)) - checkEvaluation( - map_zip_with(mbb4, mbb0, concat), - Map(b(2, 1) -> b(2, 1, 1, 5), b(1, 2) -> null, b(1, 3) -> null)) checkEvaluation( map_zip_with(mbb0, mbbn, concat), 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 81ab7d690396a..b190d6f5caa1c 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 @@ -17,13 +17,17 @@ package org.apache.spark.sql.catalyst.expressions -import java.util.Calendar +import java.text.{DecimalFormat, DecimalFormatSymbols, SimpleDateFormat} +import java.util.{Calendar, Locale} -import org.apache.spark.SparkFunSuite +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.errors.TreeNodeException +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.plans.PlanTestBase -import org.apache.spark.sql.catalyst.util.{ArrayBasedMapData, DateTimeTestUtils, DateTimeUtils, GenericArrayData, PermissiveMode} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.UTF8String @@ -409,14 +413,18 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( JsonToStructs(schema, Map.empty, Literal(jsonData), gmtId), - null + InternalRow(null) ) - // Other modes should still return `null`. - checkEvaluation( - JsonToStructs(schema, Map("mode" -> PermissiveMode.name), Literal(jsonData), gmtId), - null - ) + val exception = intercept[TestFailedException] { + checkEvaluation( + JsonToStructs(schema, Map("mode" -> FailFastMode.name), Literal(jsonData), gmtId), + InternalRow(null) + ) + }.getCause + assert(exception.isInstanceOf[SparkException]) + assert(exception.getMessage.contains( + "Malformed records are detected in record parsing. Parse Mode: FAILFAST")) } test("from_json - input=array, schema=array, output=array") { @@ -450,21 +458,23 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with 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(1) + val output = InternalRow(null) checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) } - test("from_json - input=array, schema=struct, output=null") { + test("from_json - input=array, schema=struct, output=single row") { val input = """[{"a": 1}, {"a": 2}]""" - val schema = StructType(StructField("a", IntegerType) :: Nil) - val output = null - checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) + val corrupted = "corrupted" + 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) } - test("from_json - input=empty array, schema=struct, output=null") { + test("from_json - input=empty array, schema=struct, output=single row with null") { val input = """[]""" val schema = StructType(StructField("a", IntegerType) :: Nil) - val output = null + val output = InternalRow(null) checkEvaluation(JsonToStructs(schema, Map.empty, Literal(input), gmtId), output) } @@ -487,7 +497,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val schema = StructType(StructField("a", IntegerType) :: Nil) checkEvaluation( JsonToStructs(schema, Map.empty, Literal(badJson), gmtId), - null) + InternalRow(null)) } test("from_json with timestamp") { @@ -510,7 +520,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with ) val jsonData2 = """{"t": "2016-01-01T00:00:00"}""" - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + for (tz <- DateTimeTestUtils.outstandingTimezones) { c = Calendar.getInstance(tz) c.set(2016, 0, 1, 0, 0, 0) c.set(Calendar.MILLISECOND, 0) @@ -623,7 +633,8 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[string, struct] to json") { val schema = MapType(StringType, StructType(StructField("a", IntegerType) :: Nil)) - val input = Literal.create(ArrayBasedMapData(Map("test" -> InternalRow(1))), schema) + val input = Literal( + ArrayBasedMapData(Map(UTF8String.fromString("test") -> InternalRow(1))), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"test":{"a":1}}""" @@ -633,7 +644,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[struct, struct] to json") { val schema = MapType(StructType(StructField("a", IntegerType) :: Nil), StructType(StructField("b", IntegerType) :: Nil)) - val input = Literal.create(ArrayBasedMapData(Map(InternalRow(1) -> InternalRow(2))), schema) + val input = Literal(ArrayBasedMapData(Map(InternalRow(1) -> InternalRow(2))), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"[1]":{"b":2}}""" @@ -642,7 +653,7 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("SPARK-21513: to_json support map[string, integer] to json") { val schema = MapType(StringType, IntegerType) - val input = Literal.create(ArrayBasedMapData(Map("a" -> 1)), schema) + val input = Literal(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)), schema) checkEvaluation( StructsToJson(Map.empty, input), """{"a":1}""" @@ -651,17 +662,18 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with test("to_json - array with maps") { val inputSchema = ArrayType(MapType(StringType, IntegerType)) - val input = new GenericArrayData(ArrayBasedMapData( - Map("a" -> 1)) :: ArrayBasedMapData(Map("b" -> 2)) :: Nil) + val input = new GenericArrayData( + ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: + ArrayBasedMapData(Map(UTF8String.fromString("b") -> 2)) :: Nil) val output = """[{"a":1},{"b":2}]""" checkEvaluation( - StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), + StructsToJson(Map.empty, Literal(input, inputSchema), gmtId), output) } test("to_json - array with single map") { val inputSchema = ArrayType(MapType(StringType, IntegerType)) - val input = new GenericArrayData(ArrayBasedMapData(Map("a" -> 1)) :: Nil) + val input = new GenericArrayData(ArrayBasedMapData(Map(UTF8String.fromString("a") -> 1)) :: Nil) val output = """[{"a":1}]""" checkEvaluation( StructsToJson(Map.empty, Literal.create(input, inputSchema), gmtId), @@ -682,11 +694,10 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with val mapType2 = MapType(IntegerType, CalendarIntervalType) val schema2 = StructType(StructField("a", mapType2) :: Nil) val struct2 = Literal.create(null, schema2) - intercept[TreeNodeException[_]] { - checkEvaluation( - StructsToJson(Map.empty, struct2, gmtId), - null - ) + StructsToJson(Map.empty, struct2, gmtId).checkInputDataTypes() match { + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("Unable to convert column a of type calendarinterval to JSON")) + case _ => fail("from_json should not work on interval map value type.") } } @@ -727,4 +738,70 @@ class JsonExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper with CreateMap(Seq(Literal.create("allowNumericLeadingZeros"), Literal.create("true")))), "struct") } + + test("parse date with locale") { + Seq("en-US", "ru-RU").foreach { langTag => + val locale = Locale.forLanguageTag(langTag) + val date = new SimpleDateFormat("yyyy-MM-dd").parse("2018-11-05") + val schema = new StructType().add("d", DateType) + val dateFormat = "MMM yyyy" + val sdf = new SimpleDateFormat(dateFormat, locale) + val dateStr = s"""{"d":"${sdf.format(date)}"}""" + val options = Map("dateFormat" -> dateFormat, "locale" -> langTag) + + checkEvaluation( + JsonToStructs(schema, options, Literal.create(dateStr), gmtId), + InternalRow(17836)) // number of days from 1970-01-01 + } + } + + test("verify corrupt column") { + checkExceptionInExpression[AnalysisException]( + JsonToStructs( + schema = StructType.fromDDL("i int, _unparsed boolean"), + options = Map("columnNameOfCorruptRecord" -> "_unparsed"), + child = Literal.create("""{"i":"a"}"""), + timeZoneId = gmtId), + expectedErrMsg = "The field for corrupt records must be string type and nullable") + } + + def decimalInput(langTag: String): (Decimal, String) = { + val decimalVal = new java.math.BigDecimal("1000.001") + val decimalType = new DecimalType(10, 5) + val expected = Decimal(decimalVal, decimalType.precision, decimalType.scale) + val decimalFormat = new DecimalFormat("", + new DecimalFormatSymbols(Locale.forLanguageTag(langTag))) + val input = s"""{"d": "${decimalFormat.format(expected.toBigDecimal)}"}""" + + (expected, input) + } + + test("parse decimals using locale") { + def checkDecimalParsing(langTag: String): Unit = { + val schema = new StructType().add("d", DecimalType(10, 5)) + val options = Map("locale" -> langTag) + val (expected, input) = decimalInput(langTag) + + checkEvaluation( + JsonToStructs(schema, options, Literal.create(input), gmtId), + InternalRow(expected)) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach(checkDecimalParsing) + } + + test("inferring the decimal type using locale") { + def checkDecimalInfer(langTag: String, expectedType: String): Unit = { + val options = Map("locale" -> langTag, "prefersDecimal" -> "true") + val (_, input) = decimalInput(langTag) + + checkEvaluation( + SchemaOfJson(Literal.create(input), options), + expectedType) + } + + Seq("en-US", "ko-KR", "ru-RU", "de-DE").foreach { + checkDecimalInfer(_, """struct""") + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala index 3ea6bfac9ddca..269f1a09ac533 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralExpressionSuite.scala @@ -18,14 +18,17 @@ package org.apache.spark.sql.catalyst.expressions import java.nio.charset.StandardCharsets +import java.time.{Instant, LocalDate, LocalDateTime, ZoneOffset} +import java.util.TimeZone -import scala.reflect.runtime.universe.{typeTag, TypeTag} +import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkFunSuite import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.{CatalystTypeConverters, ScalaReflection} import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -64,8 +67,14 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.default(BinaryType), "".getBytes(StandardCharsets.UTF_8)) checkEvaluation(Literal.default(DecimalType.USER_DEFAULT), Decimal(0)) checkEvaluation(Literal.default(DecimalType.SYSTEM_DEFAULT), Decimal(0)) - checkEvaluation(Literal.default(DateType), DateTimeUtils.toJavaDate(0)) - checkEvaluation(Literal.default(TimestampType), DateTimeUtils.toJavaTimestamp(0L)) + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "false") { + checkEvaluation(Literal.default(DateType), DateTimeUtils.toJavaDate(0)) + checkEvaluation(Literal.default(TimestampType), DateTimeUtils.toJavaTimestamp(0L)) + } + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + checkEvaluation(Literal.default(DateType), LocalDate.ofEpochDay(0)) + checkEvaluation(Literal.default(TimestampType), Instant.ofEpochSecond(0)) + } checkEvaluation(Literal.default(CalendarIntervalType), new CalendarInterval(0, 0L)) checkEvaluation(Literal.default(ArrayType(StringType)), Array()) checkEvaluation(Literal.default(MapType(IntegerType, StringType)), Map()) @@ -179,6 +188,8 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkArrayLiteral(Array("a", "b", "c")) checkArrayLiteral(Array(1.0, 4.0)) checkArrayLiteral(Array(CalendarInterval.MICROS_PER_DAY, CalendarInterval.MICROS_PER_HOUR)) + val arr = collection.mutable.WrappedArray.make(Array(1.0, 4.0)) + checkEvaluation(Literal(arr), toCatalyst(arr)) } test("seq") { @@ -227,24 +238,82 @@ class LiteralExpressionSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Literal.create('\n'), "\n") } - test("fromString converts String/DataType input correctly") { - checkEvaluation(Literal.fromString(false.toString, BooleanType), false) - checkEvaluation(Literal.fromString(null, NullType), null) - checkEvaluation(Literal.fromString(Int.MaxValue.toByte.toString, ByteType), Int.MaxValue.toByte) - checkEvaluation(Literal.fromString(Short.MaxValue.toShort.toString, ShortType), Short.MaxValue - .toShort) - checkEvaluation(Literal.fromString(Int.MaxValue.toString, IntegerType), Int.MaxValue) - checkEvaluation(Literal.fromString(Long.MaxValue.toString, LongType), Long.MaxValue) - checkEvaluation(Literal.fromString(Float.MaxValue.toString, FloatType), Float.MaxValue) - checkEvaluation(Literal.fromString(Double.MaxValue.toString, DoubleType), Double.MaxValue) - checkEvaluation(Literal.fromString("1.23456", DecimalType(10, 5)), Decimal(1.23456)) - checkEvaluation(Literal.fromString("Databricks", StringType), "Databricks") - val dateString = "1970-01-01" - checkEvaluation(Literal.fromString(dateString, DateType), java.sql.Date.valueOf(dateString)) - val timestampString = "0000-01-01 00:00:00" - checkEvaluation(Literal.fromString(timestampString, TimestampType), - java.sql.Timestamp.valueOf(timestampString)) - val calInterval = new CalendarInterval(1, 1) - checkEvaluation(Literal.fromString(calInterval.toString, CalendarIntervalType), calInterval) + test("construct literals from java.time.LocalDate") { + Seq( + LocalDate.of(1, 1, 1), + LocalDate.of(1582, 10, 1), + LocalDate.of(1600, 7, 30), + LocalDate.of(1969, 12, 31), + LocalDate.of(1970, 1, 1), + LocalDate.of(2019, 3, 20), + LocalDate.of(2100, 5, 17)).foreach { localDate => + checkEvaluation(Literal(localDate), localDate) + } + } + + test("construct literals from arrays of java.time.LocalDate") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val localDate0 = LocalDate.of(2019, 3, 20) + checkEvaluation(Literal(Array(localDate0)), Array(localDate0)) + val localDate1 = LocalDate.of(2100, 4, 22) + checkEvaluation(Literal(Array(localDate0, localDate1)), Array(localDate0, localDate1)) + } + } + + test("construct literals from java.time.Instant") { + Seq( + Instant.parse("0001-01-01T00:00:00Z"), + Instant.parse("1582-10-01T01:02:03Z"), + Instant.parse("1970-02-28T11:12:13Z"), + Instant.ofEpochMilli(0), + Instant.parse("2019-03-20T10:15:30Z"), + Instant.parse("2100-12-31T22:17:31Z")).foreach { instant => + checkEvaluation(Literal(instant), instant) + } + } + + test("construct literals from arrays of java.time.Instant") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val instant0 = Instant.ofEpochMilli(0) + checkEvaluation(Literal(Array(instant0)), Array(instant0)) + val instant1 = Instant.parse("2019-03-20T10:15:30Z") + checkEvaluation(Literal(Array(instant0, instant1)), Array(instant0, instant1)) + } + } + + private def withTimeZones( + sessionTimeZone: String, + systemTimeZone: String)(f: => Unit): Unit = { + withSQLConf( + SQLConf.SESSION_LOCAL_TIMEZONE.key -> sessionTimeZone, + SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val originTimeZone = TimeZone.getDefault + try { + TimeZone.setDefault(TimeZone.getTimeZone(systemTimeZone)) + f + } finally { + TimeZone.setDefault(originTimeZone) + } + } + } + + test("format timestamp literal using spark.sql.session.timeZone") { + withTimeZones(sessionTimeZone = "GMT+01:00", systemTimeZone = "GMT-08:00") { + val timestamp = LocalDateTime.of(2019, 3, 21, 0, 2, 3, 456000000) + .atZone(ZoneOffset.UTC) + .toInstant + val expected = "TIMESTAMP('2019-03-21 01:02:03.456')" + val literalStr = Literal.create(timestamp).sql + assert(literalStr === expected) + } + } + + test("format date literal independently from time zone") { + withTimeZones(sessionTimeZone = "GMT-11:00", systemTimeZone = "GMT-10:00") { + val date = LocalDate.of(2019, 3, 21) + val expected = "DATE '2019-03-21'" + val literalStr = Literal.create(date).sql + assert(literalStr === expected) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala index 032aec01782f0..b111797c3588e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/LiteralGenerator.scala @@ -18,9 +18,12 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} +import java.time.{Duration, Instant, LocalDate} +import java.util.concurrent.TimeUnit import org.scalacheck.{Arbitrary, Gen} +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -99,15 +102,44 @@ object LiteralGenerator { lazy val booleanLiteralGen: Gen[Literal] = for { b <- Arbitrary.arbBool.arbitrary } yield Literal.create(b, BooleanType) - lazy val dateLiteralGen: Gen[Literal] = - for { d <- Arbitrary.arbInt.arbitrary } yield Literal.create(new Date(d), DateType) + lazy val dateLiteralGen: Gen[Literal] = { + // Valid range for DateType is [0001-01-01, 9999-12-31] + val minDay = LocalDate.of(1, 1, 1).toEpochDay + val maxDay = LocalDate.of(9999, 12, 31).toEpochDay + for { day <- Gen.choose(minDay, maxDay) } + yield Literal.create(new Date(day * DateTimeUtils.MILLIS_PER_DAY), DateType) + } + + lazy val timestampLiteralGen: Gen[Literal] = { + // Catalyst's Timestamp type stores number of microseconds since epoch in + // a variable of Long type. To prevent arithmetic overflow of Long on + // conversion from milliseconds to microseconds, the range of random milliseconds + // since epoch is restricted here. + // Valid range for TimestampType is [0001-01-01T00:00:00.000000Z, 9999-12-31T23:59:59.999999Z] + val minMillis = Instant.parse("0001-01-01T00:00:00.000000Z").toEpochMilli + val maxMillis = Instant.parse("9999-12-31T23:59:59.999999Z").toEpochMilli + for { millis <- Gen.choose(minMillis, maxMillis) } + yield Literal.create(new Timestamp(millis), TimestampType) + } - lazy val timestampLiteralGen: Gen[Literal] = - for { t <- Arbitrary.arbLong.arbitrary } yield Literal.create(new Timestamp(t), TimestampType) + // Valid range for DateType and TimestampType is [0001-01-01, 9999-12-31] + private val maxIntervalInMonths: Int = 10000 * 12 - lazy val calendarIntervalLiterGen: Gen[Literal] = - for { m <- Arbitrary.arbInt.arbitrary; s <- Arbitrary.arbLong.arbitrary} - yield Literal.create(new CalendarInterval(m, s), CalendarIntervalType) + lazy val monthIntervalLiterGen: Gen[Literal] = { + for { months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) } + yield Literal.create(months, IntegerType) + } + + lazy val calendarIntervalLiterGen: Gen[Literal] = { + val maxDurationInSec = Duration.between( + Instant.parse("0001-01-01T00:00:00.000000Z"), + Instant.parse("9999-12-31T23:59:59.999999Z")).getSeconds + val maxMicros = TimeUnit.SECONDS.toMicros(maxDurationInSec) + for { + months <- Gen.choose(-1 * maxIntervalInMonths, maxIntervalInMonths) + micros <- Gen.choose(-1 * maxMicros, maxMicros) + } yield Literal.create(new CalendarInterval(months, micros), CalendarIntervalType) + } // Sometimes, it would be quite expensive when unlimited value is used, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala index 3a094079380fd..4c048f79741bc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MathExpressionsSuite.scala @@ -199,6 +199,18 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Sinh, DoubleType) } + test("asinh") { + testUnary(Asinh, (x: Double) => math.log(x + math.sqrt(x * x + 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Asinh, DoubleType) + + checkEvaluation(Asinh(Double.NegativeInfinity), Double.NegativeInfinity) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Asinh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Asinh(doubleNullLit)), null, EmptyRow) + } + test("cos") { testUnary(Cos, math.cos) checkConsistencyBetweenInterpretedAndCodegen(Cos, DoubleType) @@ -215,6 +227,16 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) } + test("acosh") { + testUnary(Acosh, (x: Double) => math.log(x + math.sqrt(x * x - 1.0))) + checkConsistencyBetweenInterpretedAndCodegen(Cosh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Acosh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Acosh(doubleNullLit)), null, EmptyRow) + } + test("tan") { testUnary(Tan, math.tan) checkConsistencyBetweenInterpretedAndCodegen(Tan, DoubleType) @@ -244,9 +266,19 @@ class MathExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkConsistencyBetweenInterpretedAndCodegen(Tanh, DoubleType) } + test("atanh") { + testUnary(Atanh, (x: Double) => 0.5 * math.log((1.0 + x) / (1.0 - x))) + checkConsistencyBetweenInterpretedAndCodegen(Atanh, DoubleType) + + val nullLit = Literal.create(null, NullType) + val doubleNullLit = Literal.create(null, DoubleType) + checkEvaluation(checkDataTypeAndCast(Atanh(nullLit)), null, EmptyRow) + checkEvaluation(checkDataTypeAndCast(Atanh(doubleNullLit)), null, EmptyRow) + } + test("toDegrees") { testUnary(ToDegrees, math.toDegrees) - checkConsistencyBetweenInterpretedAndCodegen(Acos, DoubleType) + checkConsistencyBetweenInterpretedAndCodegen(ToDegrees, DoubleType) } test("toRadians") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala new file mode 100644 index 0000000000000..0d594eb10962e --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/MutableProjectionSuite.scala @@ -0,0 +1,81 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.CalendarInterval + +class MutableProjectionSuite extends SparkFunSuite with ExpressionEvalHelper { + + val fixedLengthTypes = Array[DataType]( + BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, + DateType, TimestampType) + + val variableLengthTypes = Array( + StringType, DecimalType.defaultConcreteType, CalendarIntervalType, BinaryType, + ArrayType(StringType), MapType(IntegerType, StringType), + StructType.fromDDL("a INT, b STRING"), ObjectType(classOf[java.lang.Integer])) + + def createMutableProjection(dataTypes: Array[DataType]): MutableProjection = { + MutableProjection.create(dataTypes.zipWithIndex.map(x => BoundReference(x._2, x._1, true))) + } + + testBothCodegenAndInterpreted("fixed-length types") { + val inputRow = InternalRow.fromSeq(Seq(true, 3.toByte, 15.toShort, -83, 129L, 1.0f, 5.0, 1, 2L)) + val proj = createMutableProjection(fixedLengthTypes) + assert(proj(inputRow) === inputRow) + } + + testBothCodegenAndInterpreted("unsafe buffer") { + val inputRow = InternalRow.fromSeq(Seq(false, 1.toByte, 9.toShort, -18, 53L, 3.2f, 7.8, 4, 9L)) + val numBytes = UnsafeRow.calculateBitSetWidthInBytes(fixedLengthTypes.length) + val unsafeBuffer = UnsafeRow.createFromByteArray(numBytes, fixedLengthTypes.length) + val proj = createMutableProjection(fixedLengthTypes) + val projUnsafeRow = proj.target(unsafeBuffer)(inputRow) + assert(SafeProjection.create(fixedLengthTypes)(projUnsafeRow) === inputRow) + } + + testBothCodegenAndInterpreted("variable-length types") { + val proj = createMutableProjection(variableLengthTypes) + val scalaValues = Seq("abc", BigDecimal(10), CalendarInterval.fromString("interval 1 day"), + Array[Byte](1, 2), Array("123", "456"), Map(1 -> "a", 2 -> "b"), Row(1, "a"), + new java.lang.Integer(5)) + val inputRow = InternalRow.fromSeq(scalaValues.zip(variableLengthTypes).map { + case (v, dataType) => CatalystTypeConverters.createToCatalystConverter(dataType)(v) + }) + val projRow = proj(inputRow) + variableLengthTypes.zipWithIndex.foreach { case (dataType, index) => + val toScala = CatalystTypeConverters.createToScalaConverter(dataType) + assert(toScala(projRow.get(index, dataType)) === toScala(inputRow.get(index, dataType))) + } + } + + test("unsupported types for unsafe buffer") { + withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> CodegenObjectFactoryMode.NO_CODEGEN.toString) { + val proj = createMutableProjection(Array(StringType)) + val errMsg = intercept[IllegalArgumentException] { + proj.target(new UnsafeRow(1)) + }.getMessage + assert(errMsg.contains("MutableProjection cannot use UnsafeRow for output data types:")) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala index 6e07f7a59b730..49fd59c8694f1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/NullExpressionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.Timestamp + import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.analysis.SimpleAnalyzer import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext @@ -51,7 +53,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("AssertNotNUll") { val ex = intercept[RuntimeException] { - evaluateWithoutCodegen(AssertNotNull(Literal(null), Seq.empty[String])) + evaluateWithoutCodegen(AssertNotNull(Literal(null))) }.getMessage assert(ex.contains("Null value appeared in non-nullable field")) } @@ -107,8 +109,8 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val nullLit = Literal.create(null, NullType) val floatNullLit = Literal.create(null, FloatType) val floatLit = Literal.create(1.01f, FloatType) - val timestampLit = Literal.create("2017-04-12", TimestampType) - val decimalLit = Literal.create(10.2, DecimalType(20, 2)) + val timestampLit = Literal.create(Timestamp.valueOf("2017-04-12 00:00:00"), TimestampType) + val decimalLit = Literal.create(BigDecimal.valueOf(10.2), DecimalType(20, 2)) assert(analyze(new Nvl(decimalLit, stringLit)).dataType == StringType) assert(analyze(new Nvl(doubleLit, decimalLit)).dataType == DoubleType) @@ -158,7 +160,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(AtLeastNNonNulls(4, nullOnly), false, EmptyRow) } - test("Coalesce should not throw 64kb exception") { + test("Coalesce should not throw 64KiB exception") { val inputs = (1 to 2500).map(x => Literal(s"x_$x")) checkEvaluation(Coalesce(inputs), "x_1") } @@ -169,7 +171,7 @@ class NullExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx.inlinedMutableStates.size == 1) } - test("AtLeastNNonNulls should not throw 64kb exception") { + test("AtLeastNNonNulls should not throw 64KiB exception") { val inputs = (1 to 4000).map(x => Literal(s"x_$x")) checkEvaluation(AtLeastNNonNulls(1, inputs), true) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala index d145fd0aaba47..b6ca52f1d9678 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ObjectExpressionsSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.expressions import java.sql.{Date, Timestamp} import scala.collection.JavaConverters._ -import scala.language.existentials import scala.reflect.ClassTag import scala.reflect.runtime.universe.TypeTag import scala.util.Random @@ -29,6 +28,7 @@ import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, JavaTypeInference, ScalaReflection} +import org.apache.spark.sql.catalyst.ScroogeLikeExample import org.apache.spark.sql.catalyst.analysis.{ResolveTimeZone, SimpleAnalyzer, UnresolvedDeserializer} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.encoders._ @@ -307,7 +307,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val conf = new SparkConf() Seq(true, false).foreach { useKryo => val serializer = if (useKryo) new KryoSerializer(conf) else new JavaSerializer(conf) - val expected = serializer.newInstance().serialize(new Integer(1)).array() + val expected = serializer.newInstance().serialize(Integer.valueOf(1)).array() val encodeUsingSerializer = EncodeUsingSerializer(inputObject, useKryo) checkEvaluation(encodeUsingSerializer, expected, InternalRow.fromSeq(Seq(1))) checkEvaluation(encodeUsingSerializer, null, InternalRow.fromSeq(Seq(null))) @@ -315,7 +315,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { } test("SPARK-23587: MapObjects should support interpreted execution") { - def testMapObjects(collection: Any, collectionCls: Class[_], inputType: DataType): Unit = { + def testMapObjects[T](collection: Any, collectionCls: Class[T], inputType: DataType): Unit = { val function = (lambda: Expression) => Add(lambda, Literal(1)) val elementType = IntegerType val expected = Seq(2, 3, 4) @@ -384,9 +384,9 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val conf = new SparkConf() Seq(true, false).foreach { useKryo => val serializer = if (useKryo) new KryoSerializer(conf) else new JavaSerializer(conf) - val input = serializer.newInstance().serialize(new Integer(1)).array() + val input = serializer.newInstance().serialize(Integer.valueOf(1)).array() val decodeUsingSerializer = DecodeUsingSerializer(inputObject, ClassTag(cls), useKryo) - checkEvaluation(decodeUsingSerializer, new Integer(1), InternalRow.fromSeq(Seq(input))) + checkEvaluation(decodeUsingSerializer, Integer.valueOf(1), InternalRow.fromSeq(Seq(input))) checkEvaluation(decodeUsingSerializer, null, InternalRow.fromSeq(Seq(null))) } } @@ -410,6 +410,15 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { dataType = ObjectType(classOf[outerObj.Inner]), outerPointer = Some(() => outerObj)) checkObjectExprEvaluation(newInst2, new outerObj.Inner(1)) + + // SPARK-8288: A class with only a companion object constructor + val newInst3 = NewInstance( + cls = classOf[ScroogeLikeExample], + arguments = Literal(1) :: Nil, + propagateNull = false, + dataType = ObjectType(classOf[ScroogeLikeExample]), + outerPointer = Some(() => outerObj)) + checkObjectExprEvaluation(newInst3, ScroogeLikeExample(1)) } test("LambdaVariable should support interpreted execution") { @@ -439,7 +448,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val row = RandomDataGenerator.randomRow(random, schema) val rowConverter = RowEncoder(schema) val internalRow = rowConverter.toRow(row) - val lambda = LambdaVariable("dummy", "dummuIsNull", schema(0).dataType, schema(0).nullable) + val lambda = LambdaVariable("dummy", schema(0).dataType, schema(0).nullable, id = 0) checkEvaluationWithoutCodegen(lambda, internalRow.get(0, schema(0).dataType), internalRow) } } @@ -575,7 +584,7 @@ class ObjectExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // NULL key test val scalaMapHasNullKey = scala.collection.Map[java.lang.Integer, String]( - null.asInstanceOf[java.lang.Integer] -> "v0", new java.lang.Integer(1) -> "v1") + null.asInstanceOf[java.lang.Integer] -> "v0", java.lang.Integer.valueOf(1) -> "v1") val javaMapHasNullKey = new java.util.HashMap[java.lang.Integer, java.lang.String]() { { put(null, "v0") diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala index d0604b8eb7675..94e251d90bcfa 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/OrderingSuite.scala @@ -128,7 +128,7 @@ class OrderingSuite extends SparkFunSuite with ExpressionEvalHelper { } } - test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KB") { + test("SPARK-16845: GeneratedClass$SpecificOrdering grows beyond 64 KiB") { val sortOrder = Literal("abc").asc // this is passing prior to SPARK-16845, and it should also be passing after SPARK-16845 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala index ac76b17ef4761..9b6896f65abfd 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/PredicateSuite.scala @@ -23,10 +23,13 @@ import scala.collection.immutable.HashSet import org.apache.spark.SparkFunSuite import org.apache.spark.sql.RandomDataGenerator -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.catalyst.analysis.TypeCheckResult import org.apache.spark.sql.catalyst.encoders.ExamplePointUDT import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ @@ -124,34 +127,43 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { (null, false, null) :: (null, null, null) :: Nil) - test("basic IN predicate test") { - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq(Literal(1), + private def checkInAndInSet(in: In, expected: Any): Unit = { + // expecting all in.list are Literal or NonFoldableLiteral. + checkEvaluation(in, expected) + checkEvaluation(InSet(in.value, HashSet() ++ in.list.map(_.eval())), expected) + } + + test("basic IN/INSET predicate test") { + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq(Literal(1), Literal(2))), null) - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq(NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(NonFoldableLiteral.create(null, IntegerType), Seq.empty), null) - checkEvaluation(In(Literal(1), Seq.empty), false) - checkEvaluation(In(Literal(1), Seq(NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(Literal(1), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + checkInAndInSet(In(NonFoldableLiteral.create(null, IntegerType), Seq.empty), null) + checkInAndInSet(In(Literal(1), Seq.empty), false) + checkInAndInSet(In(Literal(1), Seq(NonFoldableLiteral.create(null, IntegerType))), null) + checkInAndInSet(In(Literal(1), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), + checkInAndInSet(In(Literal(2), Seq(Literal(1), NonFoldableLiteral.create(null, IntegerType))), null) - checkEvaluation(In(Literal(1), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(2), Seq(Literal(1), Literal(2))), true) - checkEvaluation(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkInAndInSet(In(Literal(1), Seq(Literal(1), Literal(2))), true) + checkInAndInSet(In(Literal(2), Seq(Literal(1), Literal(2))), true) + checkInAndInSet(In(Literal(3), Seq(Literal(1), Literal(2))), false) + checkEvaluation( And(In(Literal(1), Seq(Literal(1), Literal(2))), In(Literal(2), Seq(Literal(1), Literal(2)))), true) + checkEvaluation( + And(InSet(Literal(1), HashSet(1, 2)), InSet(Literal(2), Set(1, 2))), + true) val ns = NonFoldableLiteral.create(null, StringType) - checkEvaluation(In(ns, Seq(Literal("1"), Literal("2"))), null) - checkEvaluation(In(ns, Seq(ns)), null) - checkEvaluation(In(Literal("a"), Seq(ns)), null) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"), ns)), true) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^Ba*n"))), true) - checkEvaluation(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^n"))), false) - + checkInAndInSet(In(ns, Seq(Literal("1"), Literal("2"))), null) + checkInAndInSet(In(ns, Seq(ns)), null) + checkInAndInSet(In(Literal("a"), Seq(ns)), null) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("^Ba*n"), ns)), true) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^Ba*n"))), true) + checkInAndInSet(In(Literal("^Ba*n"), Seq(Literal("aa"), Literal("^n"))), false) } test("IN with different types") { @@ -187,11 +199,12 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { } else { false } - checkEvaluation(In(input(0), input.slice(1, 10)), expected) + checkInAndInSet(In(input(0), input.slice(1, 10)), expected) } val atomicTypes = DataTypeTestUtils.atomicTypes.filter { t => - RandomDataGenerator.forType(t).isDefined && !t.isInstanceOf[DecimalType] + RandomDataGenerator.forType(t).isDefined && + !t.isInstanceOf[DecimalType] && !t.isInstanceOf[BinaryType] } ++ Seq(DecimalType.USER_DEFAULT) val atomicArrayTypes = atomicTypes.map(ArrayType(_, containsNull = true)) @@ -221,22 +234,58 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { testWithRandomDataGeneration(structType, nullable) } - // Map types: not supported - for ( - keyType <- atomicTypes; - valueType <- atomicTypes; - nullable <- Seq(true, false)) { - val mapType = MapType(keyType, valueType) - val e = intercept[Exception] { - testWithRandomDataGeneration(mapType, nullable) - } - if (e.getMessage.contains("Code generation of")) { - // If the `value` expression is null, `eval` will be short-circuited. - // Codegen version evaluation will be run then. - assert(e.getMessage.contains("cannot generate equality code for un-comparable type")) - } else { - assert(e.getMessage.contains("Exception evaluating")) - } + // In doesn't support map type and will fail the analyzer. + val map = Literal.create(create_map(1 -> 1), MapType(IntegerType, IntegerType)) + In(map, Seq(map)).checkInputDataTypes() match { + case TypeCheckResult.TypeCheckFailure(msg) => + assert(msg.contains("function in does not support ordering on type map")) + case _ => fail("In should not work on map type") + } + } + + test("switch statements in InSet for bytes, shorts, ints, dates") { + val byteValues = Set[Any](1.toByte, 2.toByte, Byte.MinValue, Byte.MaxValue) + val shortValues = Set[Any](-10.toShort, 20.toShort, Short.MinValue, Short.MaxValue) + val intValues = Set[Any](20, -100, 30, Int.MinValue, Int.MaxValue) + val dateValues = Set[Any]( + CatalystTypeConverters.convertToCatalyst(Date.valueOf("2017-01-01")), + CatalystTypeConverters.convertToCatalyst(Date.valueOf("1950-01-02"))) + + def check(presentValue: Expression, absentValue: Expression, values: Set[Any]): Unit = { + require(presentValue.dataType == absentValue.dataType) + + val nullLiteral = Literal(null, presentValue.dataType) + + checkEvaluation(InSet(nullLiteral, values), expected = null) + checkEvaluation(InSet(nullLiteral, values + null), expected = null) + checkEvaluation(InSet(presentValue, values), expected = true) + checkEvaluation(InSet(presentValue, values + null), expected = true) + checkEvaluation(InSet(absentValue, values), expected = false) + checkEvaluation(InSet(absentValue, values + null), expected = null) + } + + def checkAllTypes(): Unit = { + check(presentValue = Literal(2.toByte), absentValue = Literal(3.toByte), byteValues) + check(presentValue = Literal(Byte.MinValue), absentValue = Literal(5.toByte), byteValues) + check(presentValue = Literal(20.toShort), absentValue = Literal(-14.toShort), shortValues) + check(presentValue = Literal(Short.MaxValue), absentValue = Literal(30.toShort), shortValues) + check(presentValue = Literal(20), absentValue = Literal(-14), intValues) + check(presentValue = Literal(Int.MinValue), absentValue = Literal(2), intValues) + check( + presentValue = Literal(Date.valueOf("2017-01-01")), + absentValue = Literal(Date.valueOf("2017-01-02")), + dateValues) + check( + presentValue = Literal(Date.valueOf("1950-01-02")), + absentValue = Literal(Date.valueOf("2017-10-02")), + dateValues) + } + + withSQLConf(SQLConf.OPTIMIZER_INSET_SWITCH_THRESHOLD.key -> "0") { + checkAllTypes() + } + withSQLConf(SQLConf.OPTIMIZER_INSET_SWITCH_THRESHOLD.key -> "20") { + checkAllTypes() } } @@ -252,45 +301,55 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { assert(ctx.inlinedMutableStates.isEmpty) } - test("INSET") { - val hS = HashSet[Any]() + 1 + 2 - val nS = HashSet[Any]() + 1 + 2 + null - val one = Literal(1) - val two = Literal(2) - val three = Literal(3) - val nl = Literal(null) - checkEvaluation(InSet(one, hS), true) - checkEvaluation(InSet(two, hS), true) - checkEvaluation(InSet(two, nS), true) - checkEvaluation(InSet(three, hS), false) - checkEvaluation(InSet(three, nS), null) - checkEvaluation(InSet(nl, hS), null) - checkEvaluation(InSet(nl, nS), null) - - val primitiveTypes = Seq(IntegerType, FloatType, DoubleType, StringType, ByteType, ShortType, - LongType, BinaryType, BooleanType, DecimalType.USER_DEFAULT, TimestampType) - primitiveTypes.foreach { t => - val dataGen = RandomDataGenerator.forType(t, nullable = true).get - val inputData = Seq.fill(10) { - val value = dataGen.apply() - value match { - case d: Double if d.isNaN => 0.0d - case f: Float if f.isNaN => 0.0f - case _ => value - } - } - val input = inputData.map(Literal(_)) - val expected = if (inputData(0) == null) { - null - } else if (inputData.slice(1, 10).contains(inputData(0))) { - true - } else if (inputData.slice(1, 10).contains(null)) { - null - } else { - false - } - checkEvaluation(InSet(input(0), inputData.slice(1, 10).toSet), expected) - } + test("IN/INSET: binary") { + val onetwo = Literal(Array(1.toByte, 2.toByte)) + val three = Literal(Array(3.toByte)) + val threefour = Literal(Array(3.toByte, 4.toByte)) + val nl = NonFoldableLiteral.create(null, onetwo.dataType) + val hS = Seq(Literal(Array(1.toByte, 2.toByte)), Literal(Array(3.toByte))) + val nS = Seq(Literal(Array(1.toByte, 2.toByte)), Literal(Array(3.toByte)), + NonFoldableLiteral.create(null, onetwo.dataType)) + checkInAndInSet(In(onetwo, hS), true) + checkInAndInSet(In(three, hS), true) + checkInAndInSet(In(three, nS), true) + checkInAndInSet(In(threefour, hS), false) + checkInAndInSet(In(threefour, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) + } + + test("IN/INSET: struct") { + val oneA = Literal.create((1, "a")) + val twoB = Literal.create((2, "b")) + val twoC = Literal.create((2, "c")) + val nl = NonFoldableLiteral.create(null, oneA.dataType) + val hS = Seq(Literal.create((1, "a")), Literal.create((2, "b"))) + val nS = Seq(Literal.create((1, "a")), Literal.create((2, "b")), + NonFoldableLiteral.create(null, oneA.dataType)) + checkInAndInSet(In(oneA, hS), true) + checkInAndInSet(In(twoB, hS), true) + checkInAndInSet(In(twoB, nS), true) + checkInAndInSet(In(twoC, hS), false) + checkInAndInSet(In(twoC, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) + } + + test("IN/INSET: array") { + val onetwo = Literal.create(Seq(1, 2)) + val three = Literal.create(Seq(3)) + val threefour = Literal.create(Seq(3, 4)) + val nl = NonFoldableLiteral.create(null, onetwo.dataType) + val hS = Seq(Literal.create(Seq(1, 2)), Literal.create(Seq(3))) + val nS = Seq(Literal.create(Seq(1, 2)), Literal.create(Seq(3)), + NonFoldableLiteral.create(null, onetwo.dataType)) + checkInAndInSet(In(onetwo, hS), true) + checkInAndInSet(In(three, hS), true) + checkInAndInSet(In(three, nS), true) + checkInAndInSet(In(threefour, hS), false) + checkInAndInSet(In(threefour, nS), null) + checkInAndInSet(In(nl, hS), null) + checkInAndInSet(In(nl, nS), null) } private case class MyStruct(a: Long, b: String) @@ -455,4 +514,11 @@ class PredicateSuite extends SparkFunSuite with ExpressionEvalHelper { interpreted.initialize(0) assert(interpreted.eval(new UnsafeRow())) } + + test("SPARK-24872: Replace taking the $symbol with $sqlOperator in BinaryOperator's" + + " toString method") { + val expression = CatalystSqlParser.parseExpression("id=1 or id=2").toString() + val expected = "(('id = 1) OR ('id = 2))" + assert(expression == expected) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala index 752c9d5449ee2..469c24b3b5f49 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/RandomSuite.scala @@ -17,25 +17,21 @@ package org.apache.spark.sql.catalyst.expressions -import org.scalatest.Matchers._ - import org.apache.spark.SparkFunSuite import org.apache.spark.sql.types.{IntegerType, LongType} class RandomSuite extends SparkFunSuite with ExpressionEvalHelper { test("random") { - checkDoubleEvaluation(Rand(30), 0.31429268272540556 +- 0.001) - checkDoubleEvaluation(Randn(30), -0.4798519469521663 +- 0.001) + checkEvaluation(Rand(30), 0.2762195585886885) + checkEvaluation(Randn(30), -1.0451987154313813) - checkDoubleEvaluation( - new Rand(Literal.create(null, LongType)), 0.8446490682263027 +- 0.001) - checkDoubleEvaluation( - new Randn(Literal.create(null, IntegerType)), 1.1164209726833079 +- 0.001) + checkEvaluation(new Rand(Literal.create(null, LongType)), 0.7604953758285915) + checkEvaluation(new Randn(Literal.create(null, IntegerType)), 1.6034991609278433) } test("SPARK-9127 codegen with long seed") { - checkDoubleEvaluation(Rand(5419823303878592871L), 0.2304755080444375 +- 0.001) - checkDoubleEvaluation(Randn(5419823303878592871L), -1.2824262718225607 +- 0.001) + checkEvaluation(Rand(5419823303878592871L), 0.7145363364564755) + checkEvaluation(Randn(5419823303878592871L), 0.7816815274533012) } } 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 e083ae0089244..df92fa3475bd9 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 @@ -26,10 +26,10 @@ import org.apache.spark.sql.types.{IntegerType, StringType} class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { test("basic") { - val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil) + val intUdf = ScalaUDF((i: Int) => i + 1, IntegerType, Literal(1) :: Nil, true :: Nil) checkEvaluation(intUdf, 2) - val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil) + val stringUdf = ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, false :: Nil) checkEvaluation(stringUdf, "ax") } @@ -37,7 +37,8 @@ class ScalaUDFSuite extends SparkFunSuite with ExpressionEvalHelper { val udf = ScalaUDF( (s: String) => s.toLowerCase(Locale.ROOT), StringType, - Literal.create(null, StringType) :: Nil) + Literal.create(null, StringType) :: Nil, + false :: Nil) val e1 = intercept[SparkException](udf.eval()) assert(e1.getMessage.contains("Failed to execute user defined function")) @@ -50,7 +51,7 @@ 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).genCode(ctx) + ScalaUDF((s: String) => s + "x", StringType, Literal("a") :: Nil, false :: Nil).genCode(ctx) assert(ctx.inlinedMutableStates.isEmpty) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala new file mode 100644 index 0000000000000..c04f59ebb1b1b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SchemaPruningSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.types._ + +class SchemaPruningSuite extends SparkFunSuite { + test("prune schema by the requested fields") { + def testPrunedSchema( + schema: StructType, + requestedFields: StructField*): Unit = { + val requestedRootFields = requestedFields.map { f => + // `derivedFromAtt` doesn't affect the result of pruned schema. + SchemaPruning.RootField(field = f, derivedFromAtt = true) + } + val expectedSchema = SchemaPruning.pruneDataSchema(schema, requestedRootFields) + assert(expectedSchema == StructType(requestedFields)) + } + + testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("a", IntegerType)) + testPrunedSchema(StructType.fromDDL("a int, b int"), StructField("b", IntegerType)) + + val structOfStruct = StructType.fromDDL("a struct, b int") + testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("a int, b int"))) + testPrunedSchema(structOfStruct, StructField("b", IntegerType)) + testPrunedSchema(structOfStruct, StructField("a", StructType.fromDDL("b int"))) + + val arrayOfStruct = StructField("a", ArrayType(StructType.fromDDL("a int, b int, c string"))) + val mapOfStruct = StructField("d", MapType(StructType.fromDDL("a int, b int, c string"), + StructType.fromDDL("d int, e int, f string"))) + + val complexStruct = StructType( + arrayOfStruct :: StructField("b", structOfStruct) :: StructField("c", IntegerType) :: + mapOfStruct :: Nil) + + testPrunedSchema(complexStruct, StructField("a", ArrayType(StructType.fromDDL("b int"))), + StructField("b", StructType.fromDDL("a int"))) + testPrunedSchema(complexStruct, + StructField("a", ArrayType(StructType.fromDDL("b int, c string"))), + StructField("b", StructType.fromDDL("b int"))) + + val selectFieldInMap = StructField("d", MapType(StructType.fromDDL("a int, b int"), + StructType.fromDDL("e int, f string"))) + testPrunedSchema(complexStruct, StructField("c", IntegerType), selectFieldInMap) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala similarity index 80% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala index 05f7e3ce83880..6a3cc21804991 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SelectedFieldSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SelectedFieldSuite.scala @@ -15,19 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql.execution +package org.apache.spark.sql.catalyst.expressions -import org.scalatest.BeforeAndAfterAll import org.scalatest.exceptions.TestFailedException -import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.AnalysisTest import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.types._ -class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { +class SelectedFieldSuite extends AnalysisTest { private val ignoredField = StructField("col1", StringType, nullable = false) // The test schema as a tree string, i.e. `schema.treeString` @@ -318,6 +316,18 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) } + testSelect(arrayOfStruct, "map_values(col5[0]).field1.subfield1 as foo") { + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + } + + testSelect(arrayOfStruct, "map_values(col5[0]).field1.subfield2 as foo") { + StructField("col5", ArrayType(MapType(StringType, StructType( + StructField("field1", StructType( + StructField("subfield2", IntegerType) :: Nil)) :: Nil), valueContainsNull = false))) + } + // |-- col1: string (nullable = false) // |-- col6: map (nullable = true) // | |-- key: string @@ -395,6 +405,90 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { :: Nil))) } + // |-- col1: string (nullable = false) + // |-- col2: map (nullable = true) + // | |-- key: struct (containsNull = false) + // | | |-- field1: string (nullable = true) + // | | |-- field2: integer (nullable = true) + // | |-- value: array (valueContainsNull = true) + // | | |-- element: struct (containsNull = false) + // | | | |-- field3: struct (nullable = true) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: integer (nullable = true) + private val mapWithStructKey = StructType(Array(ignoredField, + StructField("col2", MapType( + StructType( + StructField("field1", StringType) :: + StructField("field2", IntegerType) :: Nil), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))))) + + testSelect(mapWithStructKey, "map_keys(col2).field1 as foo") { + StructField("col2", MapType( + StructType(StructField("field1", StringType) :: Nil), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) + } + + testSelect(mapWithStructKey, "map_keys(col2).field2 as foo") { + StructField("col2", MapType( + StructType(StructField("field2", IntegerType) :: Nil), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", IntegerType) :: Nil)) :: Nil), containsNull = false))) + } + + // |-- col1: string (nullable = false) + // |-- col2: map (nullable = true) + // | |-- key: array (valueContainsNull = true) + // | | |-- element: struct (containsNull = false) + // | | | |-- field1: string (nullable = true) + // | | | |-- field2: struct (containsNull = false) + // | | | | |-- subfield1: integer (nullable = true) + // | | | | |-- subfield2: long (nullable = true) + // | |-- value: array (valueContainsNull = true) + // | | |-- element: struct (containsNull = false) + // | | | |-- field3: struct (nullable = true) + // | | | | |-- subfield3: integer (nullable = true) + // | | | | |-- subfield4: integer (nullable = true) + private val mapWithArrayOfStructKey = StructType(Array(ignoredField, + StructField("col2", MapType( + ArrayType(StructType( + StructField("field1", StringType) :: + StructField("field2", StructType( + StructField("subfield1", IntegerType) :: + StructField("subfield2", LongType) :: Nil)) :: Nil), containsNull = false), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield3", IntegerType) :: + StructField("subfield4", IntegerType) :: Nil)) :: Nil), containsNull = false))))) + + testSelect(mapWithArrayOfStructKey, "map_keys(col2)[0].field1 as foo") { + StructField("col2", MapType( + ArrayType(StructType( + StructField("field1", StringType) :: Nil), containsNull = false), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield3", IntegerType) :: + StructField("subfield4", IntegerType) :: Nil)) :: Nil), containsNull = false))) + } + + testSelect(mapWithArrayOfStructKey, "map_keys(col2)[0].field2.subfield1 as foo") { + StructField("col2", MapType( + ArrayType(StructType( + StructField("field2", StructType( + StructField("subfield1", IntegerType) :: Nil)) :: Nil), containsNull = false), + ArrayType(StructType( + StructField("field3", StructType( + StructField("subfield3", IntegerType) :: + StructField("subfield4", IntegerType) :: Nil)) :: Nil), containsNull = false))) + } + def assertResult(expected: StructField)(actual: StructField)(selectExpr: String): Unit = { try { super.assertResult(expected)(actual) @@ -440,7 +534,7 @@ class SelectedFieldSuite extends SparkFunSuite with BeforeAndAfterAll { private def unapplySelect(expr: String, relation: LocalRelation) = { val parsedExpr = parseAsCatalystExpression(Seq(expr)).head val select = relation.select(parsedExpr) - val analyzed = select.analyze + val analyzed = caseSensitiveAnalyzer.execute(select) SelectedField.unapply(analyzed.expressions.head) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala index cc2e2a993d629..f2696849d7753 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/SortOrderExpressionsSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.expressions -import java.sql.{Date, Timestamp} +import java.sql.Timestamp import java.util.TimeZone import org.apache.spark.SparkFunSuite @@ -32,9 +32,9 @@ class SortOrderExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val b2 = Literal.create(true, BooleanType) val i1 = Literal.create(20132983, IntegerType) val i2 = Literal.create(-20132983, IntegerType) - val l1 = Literal.create(20132983, LongType) - val l2 = Literal.create(-20132983, LongType) - val millis = 1524954911000L; + val l1 = Literal.create(20132983L, LongType) + val l2 = Literal.create(-20132983L, LongType) + val millis = 1524954911000L // Explicitly choose a time zone, since Date objects can create different values depending on // local time zone of the machine on which the test is running val oldDefaultTZ = TimeZone.getDefault @@ -57,7 +57,7 @@ class SortOrderExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper val dec1 = Literal(Decimal(20132983L, 10, 2)) val dec2 = Literal(Decimal(20132983L, 19, 2)) val dec3 = Literal(Decimal(20132983L, 21, 2)) - val list1 = Literal(List(1, 2), ArrayType(IntegerType)) + val list1 = Literal.create(Seq(1, 2), ArrayType(IntegerType)) val nullVal = Literal.create(null, IntegerType) checkEvaluation(SortPrefix(SortOrder(b1, Ascending)), 0L) 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 aa334e040d5fc..1b5acf4b0abcc 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 @@ -231,15 +231,14 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { val s_notNull = 'a.string.notNull.at(0) - assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable - === true) + assert(Substring(s, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable) assert( Substring(s_notNull, Literal.create(0, IntegerType), Literal.create(2, IntegerType)).nullable === false) assert(Substring(s_notNull, - Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable === true) + Literal.create(null, IntegerType), Literal.create(2, IntegerType)).nullable) assert(Substring(s_notNull, - Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable === true) + Literal.create(0, IntegerType), Literal.create(null, IntegerType)).nullable) checkEvaluation(s.substr(0, 2), "ex", row) checkEvaluation(s.substr(0), "example", row) @@ -429,6 +428,30 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on } + test("overlay") { + checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("_"), + Literal.create(6, IntegerType)), "Spark_SQL") + checkEvaluation(new Overlay(Literal("Spark SQL"), Literal("CORE"), + Literal.create(7, IntegerType)), "Spark CORE") + checkEvaluation(Overlay(Literal("Spark SQL"), Literal("ANSI "), + Literal.create(7, IntegerType), Literal.create(0, IntegerType)), "Spark ANSI SQL") + checkEvaluation(Overlay(Literal("Spark SQL"), Literal("tructured"), + Literal.create(2, IntegerType), Literal.create(4, IntegerType)), "Structured SQL") + checkEvaluation(new Overlay(Literal.create(null, StringType), Literal("_"), + Literal.create(6, IntegerType)), null) + checkEvaluation(new Overlay(Literal.create(null, StringType), Literal("CORE"), + Literal.create(7, IntegerType)), null) + checkEvaluation(Overlay(Literal.create(null, StringType), Literal("ANSI "), + Literal.create(7, IntegerType), Literal.create(0, IntegerType)), null) + checkEvaluation(Overlay(Literal.create(null, StringType), Literal("tructured"), + Literal.create(2, IntegerType), Literal.create(4, IntegerType)), null) + // scalastyle:off + // non ascii characters are not allowed in the source code, so we disable the scalastyle. + checkEvaluation(new Overlay(Literal("Spark的SQL"), Literal("_"), + Literal.create(6, IntegerType)), "Spark_SQL") + // scalastyle:on + } + test("translate") { checkEvaluation( StringTranslate(Literal("translate"), Literal("rnlt"), Literal("123")), "1a2s3ae") @@ -466,6 +489,9 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on checkEvaluation(StringTrim(Literal("a"), Literal.create(null, StringType)), null) checkEvaluation(StringTrim(Literal.create(null, StringType), Literal("a")), null) + + checkEvaluation(StringTrim(Literal("yxTomxx"), Literal("xyz")), "Tom") + checkEvaluation(StringTrim(Literal("xxxbarxxx"), Literal("x")), "bar") } test("LTRIM") { @@ -490,6 +516,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on checkEvaluation(StringTrimLeft(Literal.create(null, StringType), Literal("a")), null) checkEvaluation(StringTrimLeft(Literal("a"), Literal.create(null, StringType)), null) + + checkEvaluation(StringTrimLeft(Literal("zzzytest"), Literal("xyz")), "test") + checkEvaluation(StringTrimLeft(Literal("zzzytestxyz"), Literal("xyz")), "testxyz") + checkEvaluation(StringTrimLeft(Literal("xyxXxyLAST WORD"), Literal("xy")), "XxyLAST WORD") } test("RTRIM") { @@ -515,6 +545,10 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { // scalastyle:on checkEvaluation(StringTrimRight(Literal("a"), Literal.create(null, StringType)), null) checkEvaluation(StringTrimRight(Literal.create(null, StringType), Literal("a")), null) + + checkEvaluation(StringTrimRight(Literal("testxxzx"), Literal("xyz")), "test") + checkEvaluation(StringTrimRight(Literal("xyztestxxzx"), Literal("xyz")), "xyztest") + checkEvaluation(StringTrimRight(Literal("TURNERyxXxy"), Literal("xy")), "TURNERyxX") } test("FORMAT") { @@ -604,6 +638,8 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(StringLPad(s1, s2, s3), null, row3) checkEvaluation(StringLPad(s1, s2, s3), null, row4) checkEvaluation(StringLPad(s1, s2, s3), null, row5) + checkEvaluation(StringLPad(Literal("hi"), Literal(5)), " hi") + checkEvaluation(StringLPad(Literal("hi"), Literal(1)), "h") checkEvaluation(StringRPad(Literal("hi"), Literal(5), Literal("??")), "hi???", row1) checkEvaluation(StringRPad(Literal("hi"), Literal(1), Literal("??")), "h", row1) @@ -612,6 +648,8 @@ 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) + checkEvaluation(StringRPad(Literal("hi"), Literal(5)), "hi ") + checkEvaluation(StringRPad(Literal("hi"), Literal(1)), "h") } test("REPEAT") { @@ -744,16 +782,14 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { test("ParseUrl") { def checkParseUrl(expected: String, urlStr: String, partToExtract: String): Unit = { - checkEvaluation( - ParseUrl(Seq(Literal(urlStr), Literal(partToExtract))), expected) + checkEvaluation(ParseUrl(Seq(urlStr, partToExtract)), expected) } def checkParseUrlWithKey( expected: String, urlStr: String, partToExtract: String, key: String): Unit = { - checkEvaluation( - ParseUrl(Seq(Literal(urlStr), Literal(partToExtract), Literal(key))), expected) + checkEvaluation(ParseUrl(Seq(urlStr, partToExtract, key)), expected) } checkParseUrl("spark.apache.org", "http://spark.apache.org/path?query=1", "HOST") @@ -798,7 +834,6 @@ class StringExpressionsSuite extends SparkFunSuite with ExpressionEvalHelper { checkEvaluation(Sentences(nullString, nullString, nullString), null) checkEvaluation(Sentences(nullString, nullString), null) checkEvaluation(Sentences(nullString), null) - checkEvaluation(Sentences(Literal.create(null, NullType)), null) checkEvaluation(Sentences("", nullString, nullString), Seq.empty) checkEvaluation(Sentences("", nullString), Seq.empty) checkEvaluation(Sentences(""), Seq.empty) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala index d46135c02bc01..d202c2f271d97 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/TimeWindowSuite.scala @@ -105,9 +105,9 @@ class TimeWindowSuite extends SparkFunSuite with ExpressionEvalHelper with Priva } test("parse sql expression for duration in microseconds - long") { - val dur = TimeWindow.invokePrivate(parseExpression(Literal.create(2 << 52, LongType))) + val dur = TimeWindow.invokePrivate(parseExpression(Literal.create(2L << 52, LongType))) assert(dur.isInstanceOf[Long]) - assert(dur === (2 << 52)) + assert(dur === (2L << 52)) } test("parse sql expression for duration in microseconds - invalid interval") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala index 5a646d9a850ac..69523fa81bc65 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/UnsafeRowConverterSuite.scala @@ -26,26 +26,15 @@ import org.apache.spark.SparkFunSuite 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.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, LongType, _} import org.apache.spark.unsafe.array.ByteArrayMethods -import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} -class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase { +class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestBase + with ExpressionEvalHelper { private def roundedSize(size: Int) = ByteArrayMethods.roundNumberOfBytesToNearestWord(size) - private def testBothCodegenAndInterpreted(name: String)(f: => Unit): Unit = { - val modes = Seq(CodegenObjectFactoryMode.CODEGEN_ONLY, CodegenObjectFactoryMode.NO_CODEGEN) - for (fallbackMode <- modes) { - test(s"$name with $fallbackMode") { - withSQLConf(SQLConf.CODEGEN_FACTORY_MODE.key -> fallbackMode.toString) { - f - } - } - } - } - testBothCodegenAndInterpreted("basic conversion with only primitive types") { val factory = UnsafeProjection val fieldTypes: Array[DataType] = Array(LongType, LongType, IntegerType) @@ -257,22 +246,6 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB // assert(setToNullAfterCreation.get(11) === rowWithNoNullColumns.get(11)) } - testBothCodegenAndInterpreted("NaN canonicalization") { - val factory = UnsafeProjection - val fieldTypes: Array[DataType] = Array(FloatType, DoubleType) - - val row1 = new SpecificInternalRow(fieldTypes) - row1.setFloat(0, java.lang.Float.intBitsToFloat(0x7f800001)) - row1.setDouble(1, java.lang.Double.longBitsToDouble(0x7ff0000000000001L)) - - val row2 = new SpecificInternalRow(fieldTypes) - row2.setFloat(0, java.lang.Float.intBitsToFloat(0x7fffffff)) - row2.setDouble(1, java.lang.Double.longBitsToDouble(0x7fffffffffffffffL)) - - val converter = factory.create(fieldTypes) - assert(converter.apply(row1).getBytes === converter.apply(row2).getBytes) - } - testBothCodegenAndInterpreted("basic conversion with struct type") { val factory = UnsafeProjection val fieldTypes: Array[DataType] = Array( @@ -546,4 +519,91 @@ class UnsafeRowConverterSuite extends SparkFunSuite with Matchers with PlanTestB assert(unsafeRow.getSizeInBytes == 8 + 8 * 2 + roundedSize(field1.getSizeInBytes) + roundedSize(field2.getSizeInBytes)) } + + testBothCodegenAndInterpreted("SPARK-25374 converts back into safe representation") { + def convertBackToInternalRow(inputRow: InternalRow, fields: Array[DataType]): InternalRow = { + val unsafeProj = UnsafeProjection.create(fields) + val unsafeRow = unsafeProj(inputRow) + val safeProj = SafeProjection.create(fields) + safeProj(unsafeRow) + } + + // Simple tests + val inputRow = InternalRow.fromSeq(Seq( + false, 3.toByte, 15.toShort, -83, 129L, 1.0f, 8.0, UTF8String.fromString("test"), + Decimal(255), CalendarInterval.fromString("interval 1 day"), Array[Byte](1, 2) + )) + val fields1 = Array( + BooleanType, ByteType, ShortType, IntegerType, LongType, FloatType, + DoubleType, StringType, DecimalType.defaultConcreteType, CalendarIntervalType, + BinaryType) + + assert(convertBackToInternalRow(inputRow, fields1) === inputRow) + + // Array tests + val arrayRow = InternalRow.fromSeq(Seq( + createArray(1, 2, 3), + createArray( + createArray(Seq("a", "b", "c").map(UTF8String.fromString): _*), + createArray(Seq("d").map(UTF8String.fromString): _*)) + )) + val fields2 = Array[DataType]( + ArrayType(IntegerType), + ArrayType(ArrayType(StringType))) + + assert(convertBackToInternalRow(arrayRow, fields2) === arrayRow) + + // Struct tests + val structRow = InternalRow.fromSeq(Seq( + InternalRow.fromSeq(Seq[Any](1, 4.0)), + InternalRow.fromSeq(Seq( + UTF8String.fromString("test"), + InternalRow.fromSeq(Seq( + 1, + createArray(Seq("2", "3").map(UTF8String.fromString): _*) + )) + )) + )) + val fields3 = Array[DataType]( + StructType( + StructField("c0", IntegerType) :: + StructField("c1", DoubleType) :: + Nil), + StructType( + StructField("c2", StringType) :: + StructField("c3", StructType( + StructField("c4", IntegerType) :: + StructField("c5", ArrayType(StringType)) :: + Nil)) :: + Nil)) + + assert(convertBackToInternalRow(structRow, fields3) === structRow) + + // Map tests + val mapRow = InternalRow.fromSeq(Seq( + createMap(Seq("k1", "k2").map(UTF8String.fromString): _*)(1, 2), + createMap( + createMap(3, 5)(Seq("v1", "v2").map(UTF8String.fromString): _*), + createMap(7, 9)(Seq("v3", "v4").map(UTF8String.fromString): _*) + )( + createMap(Seq("k3", "k4").map(UTF8String.fromString): _*)(3.toShort, 4.toShort), + createMap(Seq("k5", "k6").map(UTF8String.fromString): _*)(5.toShort, 6.toShort) + ))) + val fields4 = Array[DataType]( + MapType(StringType, IntegerType), + MapType(MapType(IntegerType, StringType), MapType(StringType, ShortType))) + + val mapResultRow = convertBackToInternalRow(mapRow, fields4) + val mapExpectedRow = mapRow + checkResult(mapExpectedRow, mapResultRow, + exprDataType = StructType(fields4.zipWithIndex.map(f => StructField(s"c${f._2}", f._1))), + exprNullable = false) + + // UDT tests + val vector = new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) + val udt = new TestUDT.MyDenseVectorUDT() + val udtRow = InternalRow.fromSeq(Seq(udt.serialize(vector))) + val fields5 = Array[DataType](udt) + assert(convertBackToInternalRow(udtRow, fields5) === udtRow) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala new file mode 100644 index 0000000000000..8e9c9972071ad --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/AggregateExpressionSuite.scala @@ -0,0 +1,34 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.expressions.aggregate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions.{Add, AttributeSet} + +class AggregateExpressionSuite extends SparkFunSuite { + + test("test references from unresolved aggregate functions") { + val x = UnresolvedAttribute("x") + val y = UnresolvedAttribute("y") + val actual = AggregateExpression(Sum(Add(x, y)), mode = Complete, isDistinct = false).references + val expected = AttributeSet(x :: y :: Nil) + assert(expected == actual, s"Expected: $expected. Actual: $actual") + } + +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala index 614f24db0aafb..b0f55b3b5c443 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/DeclarativeAggregateEvaluator.scala @@ -17,25 +17,24 @@ package org.apache.spark.sql.catalyst.expressions.aggregate import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow} -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection +import org.apache.spark.sql.catalyst.expressions.{Attribute, JoinedRow, SafeProjection} /** * Evaluator for a [[DeclarativeAggregate]]. */ case class DeclarativeAggregateEvaluator(function: DeclarativeAggregate, input: Seq[Attribute]) { - lazy val initializer = GenerateSafeProjection.generate(function.initialValues) + lazy val initializer = SafeProjection.create(function.initialValues) - lazy val updater = GenerateSafeProjection.generate( + lazy val updater = SafeProjection.create( function.updateExpressions, function.aggBufferAttributes ++ input) - lazy val merger = GenerateSafeProjection.generate( + lazy val merger = SafeProjection.create( function.mergeExpressions, function.aggBufferAttributes ++ function.inputAggBufferAttributes) - lazy val evaluator = GenerateSafeProjection.generate( + lazy val evaluator = SafeProjection.create( function.evaluateExpression :: Nil, function.aggBufferAttributes) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala index 2420ba513f287..0e0c8e167a0a7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/aggregate/PercentileSuite.scala @@ -41,9 +41,9 @@ class PercentileSuite extends SparkFunSuite { val buffer = new OpenHashMap[AnyRef, Long]() assert(compareEquals(agg.deserialize(agg.serialize(buffer)), buffer)) - // Check non-empty buffer serializa and deserialize. + // Check non-empty buffer serialize and deserialize. data.foreach { key => - buffer.changeValue(new Integer(key), 1L, _ + 1L) + buffer.changeValue(Integer.valueOf(key), 1L, _ + 1L) } assert(compareEquals(agg.deserialize(agg.serialize(buffer)), buffer)) } @@ -215,7 +215,7 @@ class PercentileSuite extends SparkFunSuite { val percentile2 = new Percentile(child, percentage) assertEqual(percentile2.checkInputDataTypes(), TypeCheckFailure(s"Percentage(s) must be between 0.0 and 1.0, " + - s"but got ${percentage.simpleString}")) + s"but got ${percentage.simpleString(100)}")) } val nonFoldablePercentage = Seq(NonFoldableLiteral(0.5), @@ -232,11 +232,14 @@ class PercentileSuite extends SparkFunSuite { BooleanType, StringType, DateType, TimestampType, CalendarIntervalType, NullType) invalidDataTypes.foreach { dataType => - val percentage = Literal(0.5, dataType) + val percentage = Literal.default(dataType) val percentile4 = new Percentile(child, percentage) - assertEqual(percentile4.checkInputDataTypes(), - TypeCheckFailure(s"argument 2 requires double type, however, " + - s"'0.5' is of ${dataType.simpleString} type.")) + val checkResult = percentile4.checkInputDataTypes() + assert(checkResult.isFailure) + Seq("argument 2 requires double type, however, ", + s"is of ${dataType.simpleString} type.").foreach { errMsg => + assert(checkResult.asInstanceOf[TypeCheckFailure].message.contains(errMsg)) + } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala index fe5cb8eda824f..ee0167b4005a5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodegenExpressionCachingSuite.scala @@ -58,7 +58,7 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { expr2.mutableState = true val instance2 = UnsafeProjection.create(Seq(expr2)) assert(instance1.apply(null).getBoolean(0) === false) - assert(instance2.apply(null).getBoolean(0) === true) + assert(instance2.apply(null).getBoolean(0)) } test("GenerateMutableProjection should not share expression instances") { @@ -70,7 +70,7 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { expr2.mutableState = true val instance2 = GenerateMutableProjection.generate(Seq(expr2)) assert(instance1.apply(null).getBoolean(0) === false) - assert(instance2.apply(null).getBoolean(0) === true) + assert(instance2.apply(null).getBoolean(0)) } test("GeneratePredicate should not share expression instances") { @@ -82,7 +82,7 @@ class CodegenExpressionCachingSuite extends SparkFunSuite { expr2.mutableState = true val instance2 = GeneratePredicate.generate(expr2) assert(instance1.eval(null) === false) - assert(instance2.eval(null) === true) + assert(instance2.eval(null)) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala index 2c45b3b0c73d1..4c9bcfe8f93a6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/GeneratedProjectionSuite.scala @@ -58,7 +58,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { } // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(unsafe) // Can't compare GenericInternalRow with JoinedRow directly (0 until N).foreach { i => @@ -109,7 +109,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { } // test generated SafeProjection - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(unsafe) // Can't compare GenericInternalRow with JoinedRow directly (0 until N).foreach { i => @@ -147,7 +147,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { assert(unsafeRow.getArray(1).getBinary(1) === null) assert(java.util.Arrays.equals(unsafeRow.getArray(1).getBinary(2), Array[Byte](3, 4))) - val safeProj = FromUnsafeProjection(fields) + val safeProj = SafeProjection.create(fields) val row2 = safeProj(unsafeRow) assert(row2 === row) } @@ -233,7 +233,7 @@ class GeneratedProjectionSuite extends SparkFunSuite { val nestedSchema = StructType( Seq(StructField("", joinedSchema), StructField("", joinedSchema)) ++ joinedSchema) - val safeProj = FromUnsafeProjection(nestedSchema) + val safeProj = SafeProjection.create(nestedSchema) val result = safeProj(nested) // test generated MutableProjection diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala index fb651b76fc16d..86b8fa54c0fd4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/codegen/UnsafeRowWriterSuite.scala @@ -49,5 +49,4 @@ class UnsafeRowWriterSuite extends SparkFunSuite { // The two rows should be the equal assert(res1 == res2) } - } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala index 0fec15bc42c17..8de972f25f6a3 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/xml/UDFXPathUtilSuite.scala @@ -57,7 +57,7 @@ class UDFXPathUtilSuite extends SparkFunSuite { test("boolean eval") { var ret = util.evalBoolean("truefalseb3c1c2", "a/b[1]/text()") - assert(ret == true) + assert(ret) ret = util.evalBoolean("truefalseb3c1c2", "a/b[4]") assert(ret == false) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala new file mode 100644 index 0000000000000..8ce45f06ba65d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/json/JsonInferSchemaSuite.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.json + +import com.fasterxml.jackson.core.JsonFactory + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.types._ + +class JsonInferSchemaSuite extends SparkFunSuite with SQLHelper { + + def checkType(options: Map[String, String], json: String, dt: DataType): Unit = { + val jsonOptions = new JSONOptions(options, "UTC", "") + val inferSchema = new JsonInferSchema(jsonOptions) + val factory = new JsonFactory() + jsonOptions.setJacksonOptions(factory) + val parser = CreateJacksonParser.string(factory, json) + parser.nextToken() + val expectedType = StructType(Seq(StructField("a", dt, true))) + + assert(inferSchema.inferField(parser) === expectedType) + } + + def checkTimestampType(pattern: String, json: String): Unit = { + checkType(Map("timestampFormat" -> pattern), json, TimestampType) + } + + test("inferring timestamp type") { + checkTimestampType("yyyy", """{"a": "2018"}""") + checkTimestampType("yyyy=MM", """{"a": "2018=12"}""") + checkTimestampType("yyyy MM dd", """{"a": "2018 12 02"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSS", + """{"a": "2018-12-02T21:04:00.123"}""") + checkTimestampType( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXX", + """{"a": "2018-12-02T21:04:00.123567+01:00"}""") + } + + test("prefer decimals over timestamps") { + checkType( + options = Map( + "prefersDecimal" -> "true", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = DecimalType(17, 9) + ) + } + + test("skip decimal type inferring") { + checkType( + options = Map( + "prefersDecimal" -> "false", + "timestampFormat" -> "yyyyMMdd.HHmmssSSS" + ), + json = """{"a": "20181202.210400123"}""", + dt = TimestampType + ) + } + + test("fallback to string type") { + checkType( + options = Map("timestampFormat" -> "yyyy,MM,dd.HHmmssSSS"), + json = """{"a": "20181202.210400123"}""", + dt = StringType + ) + } + + test("disable timestamp inferring") { + val json = """{"a": "2019-01-04T21:11:10.123Z"}""" + checkType(Map("inferTimestamp" -> "true"), json, TimestampType) + checkType(Map("inferTimestamp" -> "false"), json, StringType) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala index a313681eeb8f0..5794691a365a9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala @@ -25,6 +25,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper { @@ -92,4 +93,33 @@ class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper val correctAnswer = nonNullableRelation.analyze comparePlans(actual, correctAnswer) } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + val expId = NamedExpression.newExprId + val qualifier = Seq.empty[String] + val structType = StructType( + StructField("a", StructType(StructField("b", IntegerType, false) :: Nil), false) :: Nil) + + val fieldA1 = GetStructField( + GetStructField( + AttributeReference("data1", structType, false)(expId, qualifier), + 0, Some("a1")), + 0, Some("b1")) + val fieldA2 = GetStructField( + GetStructField( + AttributeReference("data2", structType, false)(expId, qualifier), + 0, Some("a2")), + 0, Some("b2")) + + // GetStructField with different names are semantically equal; thus, `EqualTo(fieldA1, fieldA2)` + // will be optimized to `TrueLiteral` by `SimplifyBinaryComparison`. + val originalQuery = nonNullableRelation + .where(EqualTo(fieldA1, fieldA2)) + .analyze + + val optimized = Optimize.execute(originalQuery) + val correctAnswer = nonNullableRelation.analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala index 6cd1108eef333..a0de5f6930958 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.BooleanType -class BooleanSimplificationSuite extends PlanTest with PredicateHelper { +class BooleanSimplificationSuite extends PlanTest with ExpressionEvalHelper with PredicateHelper { object Optimize extends RuleExecutor[LogicalPlan] { val batches = @@ -71,6 +71,14 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { comparePlans(actual, correctAnswer) } + private def checkConditionInNotNullableRelation( + input: Expression, expected: Expression): Unit = { + val plan = testNotNullableRelationWithData.where(input).analyze + val actual = Optimize.execute(plan) + val correctAnswer = testNotNullableRelationWithData.where(expected).analyze + comparePlans(actual, correctAnswer) + } + private def checkConditionInNotNullableRelation( input: Expression, expected: LogicalPlan): Unit = { val plan = testNotNullableRelationWithData.where(input).analyze @@ -119,42 +127,55 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { 'a === 'b || 'b > 3 && 'a > 3 && 'a < 5) } - test("e && (!e || f)") { - checkCondition('e && (!'e || 'f ), 'e && 'f) + test("e && (!e || f) - not nullable") { + checkConditionInNotNullableRelation('e && (!'e || 'f ), 'e && 'f) - checkCondition('e && ('f || !'e ), 'e && 'f) + checkConditionInNotNullableRelation('e && ('f || !'e ), 'e && 'f) - checkCondition((!'e || 'f ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation((!'e || 'f ) && 'e, 'f && 'e) - checkCondition(('f || !'e ) && 'e, 'f && 'e) + checkConditionInNotNullableRelation(('f || !'e ) && 'e, 'f && 'e) } - test("a < 1 && (!(a < 1) || f)") { - checkCondition('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) - checkCondition('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) + test("e && (!e || f) - nullable") { + Seq ('e && (!'e || 'f ), + 'e && ('f || !'e ), + (!'e || 'f ) && 'e, + ('f || !'e ) && 'e, + 'e || (!'e && 'f), + 'e || ('f && !'e), + ('e && 'f) || !'e, + ('f && 'e) || !'e).foreach { expr => + checkCondition(expr, expr) + } + } - checkCondition('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) - checkCondition('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) + test("a < 1 && (!(a < 1) || f) - not nullable") { + checkConditionInNotNullableRelation('a < 1 && (!('a < 1) || 'f), ('a < 1) && 'f) + checkConditionInNotNullableRelation('a < 1 && ('f || !('a < 1)), ('a < 1) && 'f) - checkCondition('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) - checkCondition('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && (!('a <= 1) || 'f), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('f || !('a <= 1)), ('a <= 1) && 'f) - checkCondition('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) - checkCondition('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && (!('a > 1) || 'f), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && ('f || !('a > 1)), ('a > 1) && 'f) + + checkConditionInNotNullableRelation('a >= 1 && (!('a >= 1) || 'f), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && ('f || !('a >= 1)), ('a >= 1) && 'f) } - test("a < 1 && ((a >= 1) || f)") { - checkCondition('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) - checkCondition('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) + test("a < 1 && ((a >= 1) || f) - not nullable") { + checkConditionInNotNullableRelation('a < 1 && ('a >= 1 || 'f ), ('a < 1) && 'f) + checkConditionInNotNullableRelation('a < 1 && ('f || 'a >= 1), ('a < 1) && 'f) - checkCondition('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) - checkCondition('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('a > 1 || 'f ), ('a <= 1) && 'f) + checkConditionInNotNullableRelation('a <= 1 && ('f || 'a > 1), ('a <= 1) && 'f) - checkCondition('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) - checkCondition('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && (('a <= 1) || 'f), ('a > 1) && 'f) + checkConditionInNotNullableRelation('a > 1 && ('f || ('a <= 1)), ('a > 1) && 'f) - checkCondition('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) - checkCondition('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && (('a < 1) || 'f), ('a >= 1) && 'f) + checkConditionInNotNullableRelation('a >= 1 && ('f || ('a < 1)), ('a >= 1) && 'f) } test("DeMorgan's law") { @@ -217,4 +238,46 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper { checkCondition('e || !'f, testRelationWithData.where('e || !'f).analyze) checkCondition(!'f || 'e, testRelationWithData.where(!'f || 'e).analyze) } + + protected def assertEquivalent(e1: Expression, e2: Expression): Unit = { + val correctAnswer = Project(Alias(e2, "out")() :: Nil, OneRowRelation()).analyze + val actual = Optimize.execute(Project(Alias(e1, "out")() :: Nil, OneRowRelation()).analyze) + comparePlans(actual, correctAnswer) + } + + test("filter reduction - positive cases") { + val fields = Seq( + 'col1NotNULL.boolean.notNull, + 'col2NotNULL.boolean.notNull + ) + val Seq(col1NotNULL, col2NotNULL) = fields.zipWithIndex.map { case (f, i) => f.at(i) } + + val exprs = Seq( + // actual expressions of the transformations: original -> transformed + (col1NotNULL && (!col1NotNULL || col2NotNULL)) -> (col1NotNULL && col2NotNULL), + (col1NotNULL && (col2NotNULL || !col1NotNULL)) -> (col1NotNULL && col2NotNULL), + ((!col1NotNULL || col2NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), + ((col2NotNULL || !col1NotNULL) && col1NotNULL) -> (col2NotNULL && col1NotNULL), + + (col1NotNULL || (!col1NotNULL && col2NotNULL)) -> (col1NotNULL || col2NotNULL), + (col1NotNULL || (col2NotNULL && !col1NotNULL)) -> (col1NotNULL || col2NotNULL), + ((!col1NotNULL && col2NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL), + ((col2NotNULL && !col1NotNULL) || col1NotNULL) -> (col2NotNULL || col1NotNULL) + ) + + // check plans + for ((originalExpr, expectedExpr) <- exprs) { + assertEquivalent(originalExpr, expectedExpr) + } + + // check evaluation + val binaryBooleanValues = Seq(true, false) + for (col1NotNULLVal <- binaryBooleanValues; + col2NotNULLVal <- binaryBooleanValues; + (originalExpr, expectedExpr) <- exprs) { + val inputRow = create_row(col1NotNULLVal, col2NotNULLVal) + val optimizedVal = evaluateWithoutCodegen(expectedExpr, inputRow) + checkEvaluation(originalExpr, optimizedVal, inputRow) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala index e7a5bcee420f5..42bcd13ee378d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseProjectSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.expressions.{Alias, Rand} import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.types.MetadataBuilder @@ -138,4 +138,36 @@ class CollapseProjectSuite extends PlanTest { assert(projects.size === 1) assert(hasMetadata(optimized)) } + + test("collapse redundant alias through limit") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation.select('a as 'b).limit(1).select('b as 'c).analyze + val optimized = Optimize.execute(query) + val expected = relation.select('a as 'c).limit(1).analyze + comparePlans(optimized, expected) + } + + test("collapse redundant alias through local limit") { + val relation = LocalRelation('a.int, 'b.int) + val query = LocalLimit(1, relation.select('a as 'b)).select('b as 'c).analyze + val optimized = Optimize.execute(query) + val expected = LocalLimit(1, relation.select('a as 'c)).analyze + comparePlans(optimized, expected) + } + + test("collapse redundant alias through repartition") { + val relation = LocalRelation('a.int, 'b.int) + val query = relation.select('a as 'b).repartition(1).select('b as 'c).analyze + val optimized = Optimize.execute(query) + val expected = relation.select('a as 'c).repartition(1).analyze + comparePlans(optimized, expected) + } + + test("collapse redundant alias through sample") { + val relation = LocalRelation('a.int, 'b.int) + val query = Sample(0.0, 0.6, false, 11L, relation.select('a as 'b)).select('b as 'c).analyze + val optimized = Optimize.execute(query) + val expected = Sample(0.0, 0.6, false, 11L, relation.select('a as 'c)).analyze + comparePlans(optimized, expected) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala index 52054c2f8bd8d..3b3b4907eea8d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CollapseWindowSuite.scala @@ -89,4 +89,15 @@ class CollapseWindowSuite extends PlanTest { val optimized = Optimize.execute(query.analyze) comparePlans(optimized, expected) } + + test("Skip windows with empty window expressions") { + val query = testRelation + .window(Seq(), partitionSpec1, orderSpec1) + .window(Seq(sum(a).as('sum_a)), partitionSpec1, orderSpec1) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = query.analyze + + comparePlans(optimized, correctAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala index 8d7c9bf220bc2..75ff07637fccc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ColumnPruningSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.optimizer import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -26,14 +27,16 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{StringType, StructType} class ColumnPruningSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Column pruning", FixedPoint(100), - PushDownPredicate, + PushPredicateThroughNonJoin, ColumnPruning, + RemoveNoopOperators, CollapseProject) :: Nil } @@ -100,6 +103,81 @@ class ColumnPruningSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + test("Nested column pruning for Generate") { + def runTest( + origGenerator: Generator, + replacedGenerator: Seq[String] => Generator, + aliasedExprs: Seq[String] => Seq[Expression], + unrequiredChildIndex: Seq[Int], + generatorOutputNames: Seq[String]) { + withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { + val structType = StructType.fromDDL("d double, e array, f double, g double, " + + "h array>") + val input = LocalRelation('a.int, 'b.int, 'c.struct(structType)) + val generatorOutputs = generatorOutputNames.map(UnresolvedAttribute(_)) + + val selectedExprs = Seq(UnresolvedAttribute("a"), 'c.getField("d")) ++ + generatorOutputs + + val query = + input + .generate(origGenerator, outputNames = generatorOutputNames) + .select(selectedExprs: _*) + .analyze + + val optimized = Optimize.execute(query) + + val aliases = NestedColumnAliasingSuite.collectGeneratedAliases(optimized) + + val selectedFields = UnresolvedAttribute("a") +: aliasedExprs(aliases) + val finalSelectedExprs = Seq(UnresolvedAttribute("a"), $"${aliases(0)}".as("c.d")) ++ + generatorOutputs + + val correctAnswer = + input + .select(selectedFields: _*) + .generate(replacedGenerator(aliases), + unrequiredChildIndex = unrequiredChildIndex, + outputNames = generatorOutputNames) + .select(finalSelectedExprs: _*) + .analyze + + comparePlans(optimized, correctAnswer) + } + } + + runTest( + Explode('c.getField("e")), + aliases => Explode($"${aliases(1)}".as("c.e")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + Seq(2), + Seq("explode") + ) + runTest(Stack(2 :: 'c.getField("f") :: 'c.getField("g") :: Nil), + aliases => Stack(2 :: $"${aliases(1)}".as("c.f") :: $"${aliases(2)}".as("c.g") :: Nil), + aliases => Seq( + 'c.getField("d").as(aliases(0)), + 'c.getField("f").as(aliases(1)), + 'c.getField("g").as(aliases(2))), + Seq(2, 3), + Seq("stack") + ) + runTest( + PosExplode('c.getField("e")), + aliases => PosExplode($"${aliases(1)}".as("c.e")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("e").as(aliases(1))), + Seq(2), + Seq("pos", "explode") + ) + runTest( + Inline('c.getField("h")), + aliases => Inline($"${aliases(1)}".as("c.h")), + aliases => Seq('c.getField("d").as(aliases(0)), 'c.getField("h").as(aliases(1))), + Seq(2), + Seq("h1", "h2") + ) + } + test("Column pruning for Project on Sort") { val input = LocalRelation('a.int, 'b.string, 'c.double) @@ -340,10 +418,8 @@ class ColumnPruningSuite extends PlanTest { test("Column pruning on Union") { val input1 = LocalRelation('a.int, 'b.string, 'c.double) val input2 = LocalRelation('c.int, 'd.string, 'e.double) - val query = Project('b :: Nil, - Union(input1 :: input2 :: Nil)).analyze - val expected = Project('b :: Nil, - Union(Project('b :: Nil, input1) :: Project('d :: Nil, input2) :: Nil)).analyze + val query = Project('b :: Nil, Union(input1 :: input2 :: Nil)).analyze + val expected = Union(Project('b :: Nil, input1) :: Project('d :: Nil, input2) :: Nil).analyze comparePlans(Optimize.execute(query), expected) } @@ -354,15 +430,15 @@ class ColumnPruningSuite extends PlanTest { Project(Seq($"x.key", $"y.key"), Join( SubqueryAlias("x", input), - ResolvedHint(SubqueryAlias("y", input)), Inner, None)).analyze + SubqueryAlias("y", input), Inner, None, JoinHint.NONE)).analyze val optimized = Optimize.execute(query) val expected = Join( Project(Seq($"x.key"), SubqueryAlias("x", input)), - ResolvedHint(Project(Seq($"y.key"), SubqueryAlias("y", input))), - Inner, None).analyze + Project(Seq($"y.key"), SubqueryAlias("y", input)), + Inner, None, JoinHint.NONE).analyze comparePlans(optimized, expected) } @@ -389,7 +465,7 @@ class ColumnPruningSuite extends PlanTest { val query2 = Sample(0.0, 0.6, false, 11L, x).select('a as 'aa) val optimized2 = Optimize.execute(query2.analyze) - val expected2 = Sample(0.0, 0.6, false, 11L, x.select('a)).select('a as 'aa) + val expected2 = Sample(0.0, 0.6, false, 11L, x.select('a as 'aa)) comparePlans(optimized2, expected2.analyze) } @@ -400,6 +476,5 @@ class ColumnPruningSuite extends PlanTest { val expected = input.where(rand(0L) > 0.5).where('key < 10).select('key).analyze comparePlans(optimized, expected) } - // todo: add more tests for column pruning } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala index ef4b848924f06..b190dd5a7c220 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala @@ -27,8 +27,9 @@ class CombiningLimitsSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Filter Pushdown", FixedPoint(100), - ColumnPruning) :: + Batch("Column Pruning", FixedPoint(100), + ColumnPruning, + RemoveNoopOperators) :: Batch("Combine Limit", FixedPoint(10), CombineLimits) :: Batch("Constant Folding", FixedPoint(10), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala index 0c015f88e1e84..43579d4c903a1 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConvertToLocalRelationSuite.scala @@ -21,10 +21,12 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{LessThan, Literal} +import org.apache.spark.sql.catalyst.expressions.{Expression, GenericInternalRow, LessThan, Literal, UnaryExpression} +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.{DataType, StructType} class ConvertToLocalRelationSuite extends PlanTest { @@ -70,4 +72,36 @@ class ConvertToLocalRelationSuite extends PlanTest { comparePlans(optimized, correctAnswer) } + + test("SPARK-27798: Expression reusing output shouldn't override values in local relation") { + val testRelation = LocalRelation( + LocalRelation('a.int).output, + InternalRow(1) :: InternalRow(2) :: Nil) + + val correctAnswer = LocalRelation( + LocalRelation('a.struct('a1.int)).output, + InternalRow(InternalRow(1)) :: InternalRow(InternalRow(2)) :: Nil) + + val projected = testRelation.select(ExprReuseOutput(UnresolvedAttribute("a")).as("a")) + val optimized = Optimize.execute(projected.analyze) + + comparePlans(optimized, correctAnswer) + } +} + + +// Dummy expression used for testing. It reuses output row. Assumes child expr outputs an integer. +case class ExprReuseOutput(child: Expression) extends UnaryExpression { + override def dataType: DataType = StructType.fromDDL("a1 int") + override def nullable: Boolean = true + + override protected def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = + throw new UnsupportedOperationException("Should not trigger codegen") + + private val row: InternalRow = new GenericInternalRow(1) + + override def eval(input: InternalRow): Any = { + row.update(0, child.eval(input)) + row + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala new file mode 100644 index 0000000000000..6f1280c90e9de --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownOnePassSuite.scala @@ -0,0 +1,183 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + +/** + * This test suite ensures that the [[PushDownPredicates]] actually does predicate pushdown in + * an efficient manner. This is enforced by asserting that a single predicate pushdown can push + * all predicate to bottom as much as possible. + */ +class FilterPushdownOnePassSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + // this batch must reach expected state in one pass + Batch("Filter Pushdown One Pass", Once, + ReorderJoin, + PushDownPredicates + ) :: Nil + } + + val testRelation1 = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('a.int, 'd.int, 'e.int) + + test("really simple predicate push down") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val originalQuery = x.join(y).where("x.a".attr === 1) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = x.where("x.a".attr === 1).join(y).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down conjunctive predicates") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val originalQuery = x.join(y).where("x.a".attr === 1 && "y.d".attr < 1) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = x.where("x.a".attr === 1).join(y.where("y.d".attr < 1)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down predicates for simple joins") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val originalQuery = + x.where("x.c".attr < 0) + .join(y.where("y.d".attr > 1)) + .where("x.a".attr === 1 && "y.d".attr < 2) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + x.where("x.c".attr < 0 && "x.a".attr === 1) + .join(y.where("y.d".attr > 1 && "y.d".attr < 2)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down top-level filters for cascading joins") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val originalQuery = + y.join(x).join(x).join(x).join(x).join(x).where("y.d".attr === 0) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = y.where("y.d".attr === 0).join(x).join(x).join(x).join(x).join(x).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down predicates for tree-like joins") { + val x = testRelation1.subquery('x) + val y1 = testRelation2.subquery('y1) + val y2 = testRelation2.subquery('y2) + + val originalQuery = + y1.join(x).join(x) + .join(y2.join(x).join(x)) + .where("y1.d".attr === 0 && "y2.d".attr === 3) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + y1.where("y1.d".attr === 0).join(x).join(x) + .join(y2.where("y2.d".attr === 3).join(x).join(x)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down through join and project") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val originalQuery = + x.where('a > 0).select('a, 'b) + .join(y.where('d < 100).select('e)) + .where("x.a".attr < 100) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + x.where('a > 0 && 'a < 100).select('a, 'b) + .join(y.where('d < 100).select('e)).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down through deep projects") { + val x = testRelation1.subquery('x) + + val originalQuery = + x.select(('a + 1) as 'a1, 'b) + .select(('a1 + 1) as 'a2, 'b) + .select(('a2 + 1) as 'a3, 'b) + .select(('a3 + 1) as 'a4, 'b) + .select('b) + .where('b > 0) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + x.where('b > 0) + .select(('a + 1) as 'a1, 'b) + .select(('a1 + 1) as 'a2, 'b) + .select(('a2 + 1) as 'a3, 'b) + .select(('a3 + 1) as 'a4, 'b) + .select('b).analyze + + comparePlans(optimized, correctAnswer) + } + + test("push down through aggregate and join") { + val x = testRelation1.subquery('x) + val y = testRelation2.subquery('y) + + val left = x + .where('c > 0) + .groupBy('a)('a, count('b)) + .subquery('left) + val right = y + .where('d < 0) + .groupBy('a)('a, count('d)) + .subquery('right) + val originalQuery = left + .join(right).where("left.a".attr < 100 && "right.a".attr < 100) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = + x.where('c > 0 && 'a < 100).groupBy('a)('a, count('b)) + .join(y.where('d < 0 && 'a < 100).groupBy('a)('a, count('d))) + .analyze + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala index 82a10254d846d..3ec8d18bc871d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -25,7 +25,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ -import org.apache.spark.sql.types.IntegerType +import org.apache.spark.sql.types.{BooleanType, IntegerType} import org.apache.spark.unsafe.types.CalendarInterval class FilterPushdownSuite extends PlanTest { @@ -36,15 +36,20 @@ class FilterPushdownSuite extends PlanTest { EliminateSubqueryAliases) :: Batch("Filter Pushdown", FixedPoint(10), CombineFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, BooleanSimplification, PushPredicateThroughJoin, CollapseProject) :: Nil } - val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int - val testRelation1 = LocalRelation('d.int) + val testRelation = LocalRelation(attrA, attrB, attrC) + + val testRelation1 = LocalRelation(attrD) // This test already passes. test("eliminate subqueries") { @@ -822,19 +827,6 @@ class FilterPushdownSuite extends PlanTest { comparePlans(optimized, correctAnswer) } - test("broadcast hint") { - val originalQuery = ResolvedHint(testRelation) - .where('a === 2L && 'b + Rand(10).as("rnd") === 3) - - val optimized = Optimize.execute(originalQuery.analyze) - - val correctAnswer = ResolvedHint(testRelation.where('a === 2L)) - .where('b + Rand(10).as("rnd") === 3) - .analyze - - comparePlans(optimized, correctAnswer) - } - test("union") { val testRelation2 = LocalRelation('d.int, 'e.int, 'f.int) @@ -1216,4 +1208,26 @@ class FilterPushdownSuite extends PlanTest { comparePlans(Optimize.execute(originalQuery.analyze), correctAnswer.analyze, checkAnalysis = false) } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + val pythonUDFJoinCond = { + val pythonUDF = PythonUDF("pythonUDF", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrD + } + + val query = testRelation.join( + testRelation1, + joinType = Cross).where(pythonUDFJoinCond) + + val expected = testRelation.join( + testRelation1, + joinType = Cross, + condition = Some(pythonUDFJoinCond)).analyze + + comparePlans(Optimize.execute(query.analyze), expected) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala index c28844642aed0..0d48ecb31cfa4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FoldablePropagationSuite.scala @@ -132,10 +132,10 @@ class FoldablePropagationSuite extends PlanTest { test("Propagate in inner join") { val ta = testRelation.select('a, Literal(1).as('tag)) - .union(testRelation.select('a, Literal(2).as('tag))) + .union(testRelation.select('a.as('a), Literal(2).as('tag))) .subquery('ta) val tb = testRelation.select('a, Literal(1).as('tag)) - .union(testRelation.select('a, Literal(2).as('tag))) + .union(testRelation.select('a.as('a), Literal(2).as('tag))) .subquery('tb) val query = ta.join(tb, Inner, Some("ta.a".attr === "tb.a".attr && "ta.tag".attr === "tb.tag".attr)) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala index a40ba2dc38b70..974bc781d36ab 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala @@ -31,7 +31,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest { val batches = Batch("InferAndPushDownFilters", FixedPoint(100), PushPredicateThroughJoin, - PushDownPredicate, + PushPredicateThroughNonJoin, InferFiltersFromConstraints, CombineFilters, SimplifyBinaryComparison, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala index ccd9d8dd4d213..0f93305565224 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala @@ -34,11 +34,12 @@ class JoinOptimizationSuite extends PlanTest { EliminateSubqueryAliases) :: Batch("Filter Pushdown", FixedPoint(100), CombineFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, BooleanSimplification, ReorderJoin, PushPredicateThroughJoin, ColumnPruning, + RemoveNoopOperators, CollapseProject) :: Nil } @@ -64,7 +65,8 @@ class JoinOptimizationSuite extends PlanTest { def testExtractCheckCross (plan: LogicalPlan, expected: Option[(Seq[(LogicalPlan, InnerLike)], Seq[Expression])]) { - assert(ExtractFiltersAndInnerJoins.unapply(plan) === expected) + assert( + ExtractFiltersAndInnerJoins.unapply(plan) === expected.map(e => (e._1, e._2))) } testExtract(x, None) @@ -102,16 +104,19 @@ class JoinOptimizationSuite extends PlanTest { x.join(y).join(z).where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, condition = Some("x.b".attr === "z.b".attr)) .join(y, condition = Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Cross).join(z, Cross) .where(("x.b".attr === "z.b".attr) && ("y.d".attr === "z.a".attr)), x.join(z, Cross, Some("x.b".attr === "z.b".attr)) .join(y, Cross, Some("y.d".attr === "z.a".attr)) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ), ( x.join(y, Inner).join(z, Cross).where("x.b".attr === "z.a".attr), x.join(z, Cross, Some("x.b".attr === "z.a".attr)).join(y, Inner) + .select(Seq("x.a", "x.b", "x.c", "y.d", "z.a", "z.b", "z.c").map(_.attr): _*) ) ) @@ -120,29 +125,4 @@ class JoinOptimizationSuite extends PlanTest { comparePlans(optimized, queryAnswerPair._2.analyze) } } - - test("broadcasthint sets relation statistics to smallest value") { - val input = LocalRelation('key.int, 'value.string) - - val query = - Project(Seq($"x.key", $"y.key"), - Join( - SubqueryAlias("x", input), - ResolvedHint(SubqueryAlias("y", input)), Cross, None)).analyze - - val optimized = Optimize.execute(query) - - val expected = - Join( - Project(Seq($"x.key"), SubqueryAlias("x", input)), - ResolvedHint(Project(Seq($"y.key"), SubqueryAlias("y", input))), - Cross, None).analyze - - comparePlans(optimized, expected) - - val broadcastChildren = optimized.collect { - case Join(_, r, _, _) if r.stats.sizeInBytes == 1 => r - } - assert(broadcastChildren.size == 1) - } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala index 565b0a10154a8..43e5bad3fb415 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala @@ -20,8 +20,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.{Attribute, AttributeMap} -import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} -import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.{Cross, Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} import org.apache.spark.sql.internal.SQLConf.{CBO_ENABLED, JOIN_REORDER_ENABLED} @@ -31,9 +31,11 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { object Optimize extends RuleExecutor[LogicalPlan] { val batches = + Batch("Resolve Hints", Once, + EliminateResolvedHint) :: Batch("Operator Optimizations", FixedPoint(100), CombineFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, ReorderJoin, PushPredicateThroughJoin, ColumnPruning, @@ -42,6 +44,12 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { CostBasedJoinReorder) :: Nil } + object ResolveHints extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Resolve Hints", Once, + EliminateResolvedHint) :: Nil + } + var originalConfCBOEnabled = false var originalConfJoinReorderEnabled = false @@ -124,7 +132,8 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { // the original order (t1 J t2) J t3. val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) - .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -139,7 +148,9 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { val bestPlan = t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) // this is redundant but we'll take it for now .join(t4) + .select(outputsOf(t1, t2, t4, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -202,6 +213,7 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) .join(t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))), Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + .select(outputsOf(t1, t4, t2, t3): _*) assertEqualPlans(originalPlan, bestPlan) } @@ -219,6 +231,23 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { } } + test("SPARK-26352: join reordering should not change the order of attributes") { + // This test case does not rely on CBO. + // It's similar to the test case above, but catches a reordering bug that the one above doesn't + val tab1 = LocalRelation('x.int, 'y.int) + val tab2 = LocalRelation('i.int, 'j.int) + val tab3 = LocalRelation('a.int, 'b.int) + val original = + tab1.join(tab2, Cross) + .join(tab3, Inner, Some('a === 'x && 'b === 'i)) + val expected = + tab1.join(tab3, Inner, Some('a === 'x)) + .join(tab2, Cross, Some('b === 'i)) + .select(outputsOf(tab1, tab2, tab3): _*) + + assertEqualPlans(original, expected) + } + test("reorder recursively") { // Original order: // Join @@ -263,11 +292,97 @@ class JoinReorderSuite extends PlanTest with StatsEstimationTestBase { assertEqualPlans(originalPlan, bestPlan) } + test("don't reorder if hints present") { + val originalPlan = + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .hint("broadcast") + .join( + t4.join(t3, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast"), + Inner, + Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + + assertEqualPlans(originalPlan, originalPlan) + + val originalPlan2 = + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .hint("broadcast") + .join(t4, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast") + .join(t3, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t4.k-1-2"))) + + assertEqualPlans(originalPlan2, originalPlan2) + + val originalPlan3 = + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join(t4).hint("broadcast") + .join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t5, Inner, Some(nameToAttr("t5.v-1-5") === nameToAttr("t3.v-1-100"))) + + assertEqualPlans(originalPlan3, originalPlan3) + } + + test("reorder below and above the hint node") { + val originalPlan = + t1.join(t2).join(t3) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast").join(t4) + + val bestPlan = + t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) + .hint("broadcast").join(t4) + + assertEqualPlans(originalPlan, bestPlan) + + val originalPlan2 = + t1.join(t2).join(t3) + .where((nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5")) && + (nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t4.hint("broadcast")) + + val bestPlan2 = + t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) + .join(t4.hint("broadcast")) + + assertEqualPlans(originalPlan2, bestPlan2) + + val originalPlan3 = + t1.join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .hint("broadcast") + .join(t4, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t5, Inner, Some(nameToAttr("t5.v-1-5") === nameToAttr("t3.v-1-100"))) + + val bestPlan3 = + t1.join(t3, Inner, Some(nameToAttr("t1.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t2, Inner, Some(nameToAttr("t1.k-1-2") === nameToAttr("t2.k-1-5"))) + .select(outputsOf(t1, t2, t3): _*) + .hint("broadcast") + .join(t4, Inner, Some(nameToAttr("t4.v-1-10") === nameToAttr("t3.v-1-100"))) + .join(t5, Inner, Some(nameToAttr("t5.v-1-5") === nameToAttr("t3.v-1-100"))) + + assertEqualPlans(originalPlan3, bestPlan3) + } + private def assertEqualPlans( originalPlan: LogicalPlan, groundTruthBestPlan: LogicalPlan): Unit = { - val optimized = Optimize.execute(originalPlan.analyze) - val expected = groundTruthBestPlan.analyze + val analyzed = originalPlan.analyze + val optimized = Optimize.execute(analyzed) + val expected = ResolveHints.execute(groundTruthBestPlan.analyze) + + assert(analyzed.sameOutput(expected)) // if this fails, the expected plan itself is incorrect + assert(analyzed.sameOutput(optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala new file mode 100644 index 0000000000000..f6d1898dc64a8 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LeftSemiAntiJoinPushDownSuite.scala @@ -0,0 +1,432 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.unsafe.types.CalendarInterval + +class LeftSemiPushdownSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueryAliases) :: + Batch("Filter Pushdown", FixedPoint(10), + CombineFilters, + PushPredicateThroughNonJoin, + PushDownLeftSemiAntiJoin, + PushLeftSemiLeftAntiThroughJoin, + BooleanSimplification, + CollapseProject) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation1 = LocalRelation('d.int) + val testRelation2 = LocalRelation('e.int) + + test("Project: LeftSemiAnti join pushdown") { + val originalQuery = testRelation + .select(star()) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .select('a, 'b, 'c) + .analyze + comparePlans(optimized, correctAnswer) + } + + test("Project: LeftSemiAnti join no pushdown because of non-deterministic proj exprs") { + val originalQuery = testRelation + .select(Rand('a), 'b, 'c) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Project: LeftSemiAnti join non correlated scalar subq") { + val subq = ScalarSubquery(testRelation.groupBy('b)(sum('c).as("sum")).analyze) + val originalQuery = testRelation + .select(subq.as("sum")) + .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some(subq === 'd)) + .select(subq.as("sum")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Project: LeftSemiAnti join no pushdown - correlated scalar subq in projection list") { + val testRelation2 = LocalRelation('e.int, 'f.int) + val subqPlan = testRelation2.groupBy('e)(sum('f).as("sum")).where('e === 'a) + val subqExpr = ScalarSubquery(subqPlan) + val originalQuery = testRelation + .select(subqExpr.as("sum")) + .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Aggregate: LeftSemiAnti join pushdown") { + val originalQuery = testRelation + .groupBy('b)('b, sum('c)) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .groupBy('b)('b, sum('c)) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Aggregate: LeftSemiAnti join no pushdown due to non-deterministic aggr expressions") { + val originalQuery = testRelation + .groupBy('b)('b, Rand(10).as('c)) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Aggregate: LeftSemi join partial pushdown") { + val originalQuery = testRelation + .groupBy('b)('b, sum('c).as('sum)) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'sum === 10)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .groupBy('b)('b, sum('c).as('sum)) + .where('sum === 10) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Aggregate: LeftAnti join no pushdown") { + val originalQuery = testRelation + .groupBy('b)('b, sum('c).as('sum)) + .join(testRelation1, joinType = LeftAnti, condition = Some('b === 'd && 'sum === 10)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("LeftSemiAnti join over aggregate - no pushdown") { + val originalQuery = testRelation + .groupBy('b)('b, sum('c).as('sum)) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'sum === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Aggregate: LeftSemiAnti join non-correlated scalar subq aggr exprs") { + val subq = ScalarSubquery(testRelation.groupBy('b)(sum('c).as("sum")).analyze) + val originalQuery = testRelation + .groupBy('a) ('a, subq.as("sum")) + .join(testRelation1, joinType = LeftSemi, condition = Some('sum === 'd && 'a === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some(subq === 'd && 'a === 'd)) + .groupBy('a) ('a, subq.as("sum")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("LeftSemiAnti join over Window") { + val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + + val originalQuery = testRelation + .select('a, 'b, 'c, winExpr.as('window)) + .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + .select('a, 'b, 'c) + .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) + .select('a, 'b, 'c, 'window).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Window: LeftSemi partial pushdown") { + // Attributes from join condition which does not refer to the window partition spec + // are kept up in the plan as a Filter operator above Window. + val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + + val originalQuery = testRelation + .select('a, 'b, 'c, winExpr.as('window)) + .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd && 'b > 5)) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + .select('a, 'b, 'c) + .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) + .where('b > 5) + .select('a, 'b, 'c, 'window).analyze + + comparePlans(optimized, correctAnswer) + } + + test("Window: LeftAnti no pushdown") { + // Attributes from join condition which does not refer to the window partition spec + // are kept up in the plan as a Filter operator above Window. + val winExpr = windowExpr(count('b), windowSpec('a :: Nil, 'b.asc :: Nil, UnspecifiedFrame)) + + val originalQuery = testRelation + .select('a, 'b, 'c, winExpr.as('window)) + .join(testRelation1, joinType = LeftAnti, condition = Some('a === 'd && 'b > 5)) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = testRelation + .select('a, 'b, 'c) + .window(winExpr.as('window) :: Nil, 'a :: Nil, 'b.asc :: Nil) + .join(testRelation1, joinType = LeftAnti, condition = Some('a === 'd && 'b > 5)) + .select('a, 'b, 'c, 'window).analyze + comparePlans(optimized, correctAnswer) + } + + test("Union: LeftSemiAnti join pushdown") { + val testRelation2 = LocalRelation('x.int, 'y.int, 'z.int) + + val originalQuery = Union(Seq(testRelation, testRelation2)) + .join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = Union(Seq( + testRelation.join(testRelation1, joinType = LeftSemi, condition = Some('a === 'd)), + testRelation2.join(testRelation1, joinType = LeftSemi, condition = Some('x === 'd)))) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Union: LeftSemiAnti join no pushdown in self join scenario") { + val testRelation2 = LocalRelation('x.int, 'y.int, 'z.int) + + val originalQuery = Union(Seq(testRelation, testRelation2)) + .join(testRelation2, joinType = LeftSemi, condition = Some('a === 'x)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Unary: LeftSemiAnti join pushdown") { + val originalQuery = testRelation + .select(star()) + .repartition(1) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .select('a, 'b, 'c) + .repartition(1) + .analyze + comparePlans(optimized, correctAnswer) + } + + test("Unary: LeftSemiAnti join pushdown - empty join condition") { + val originalQuery = testRelation + .select(star()) + .repartition(1) + .join(testRelation1, joinType = LeftSemi, condition = None) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelation + .join(testRelation1, joinType = LeftSemi, condition = None) + .select('a, 'b, 'c) + .repartition(1) + .analyze + comparePlans(optimized, correctAnswer) + } + + test("Unary: LeftSemi join pushdown - partial pushdown") { + val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val originalQuery = testRelationWithArrayType + .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'b === 'out_col)) + + val optimized = Optimize.execute(originalQuery.analyze) + val correctAnswer = testRelationWithArrayType + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd)) + .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) + .where('b === 'out_col) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Unary: LeftAnti join pushdown - no pushdown") { + val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val originalQuery = testRelationWithArrayType + .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftAnti, condition = Some('b === 'd && 'b === 'out_col)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + test("Unary: LeftSemiAnti join pushdown - no pushdown") { + val testRelationWithArrayType = LocalRelation('a.int, 'b.int, 'c_arr.array(IntegerType)) + val originalQuery = testRelationWithArrayType + .generate(Explode('c_arr), alias = Some("arr"), outputNames = Seq("out_col")) + .join(testRelation1, joinType = LeftSemi, condition = Some('b === 'd && 'd === 'out_col)) + + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + + Seq(Some('d === 'e), None).foreach { case innerJoinCond => + Seq(LeftSemi, LeftAnti).foreach { case outerJT => + Seq(Inner, LeftOuter, Cross, RightOuter).foreach { case innerJT => + test(s"$outerJT pushdown empty join cond join type $innerJT join cond $innerJoinCond") { + val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, innerJoinCond) + val originalQuery = joinedRelation.join(testRelation, joinType = outerJT, None) + val optimized = Optimize.execute(originalQuery.analyze) + + val correctAnswer = if (innerJT == RightOuter) { + val pushedDownJoin = testRelation2.join(testRelation, joinType = outerJT, None) + testRelation1.join(pushedDownJoin, joinType = innerJT, innerJoinCond).analyze + } else { + val pushedDownJoin = testRelation1.join(testRelation, joinType = outerJT, None) + pushedDownJoin.join(testRelation2, joinType = innerJT, innerJoinCond).analyze + } + comparePlans(optimized, correctAnswer) + } + } + } + } + + Seq(Some('d === 'e), None).foreach { case innerJoinCond => + Seq(LeftSemi, LeftAnti).foreach { case outerJT => + Seq(Inner, LeftOuter, Cross).foreach { case innerJT => + test(s"$outerJT pushdown to left of join type: $innerJT join condition $innerJoinCond") { + val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, innerJoinCond) + val originalQuery = + joinedRelation.join(testRelation, joinType = outerJT, condition = Some('a === 'd)) + val optimized = Optimize.execute(originalQuery.analyze) + + val pushedDownJoin = + testRelation1.join(testRelation, joinType = outerJT, condition = Some('a === 'd)) + val correctAnswer = + pushedDownJoin.join(testRelation2, joinType = innerJT, innerJoinCond).analyze + comparePlans(optimized, correctAnswer) + } + } + } + } + + Seq(Some('e === 'd), None).foreach { case innerJoinCond => + Seq(LeftSemi, LeftAnti).foreach { case outerJT => + Seq(Inner, RightOuter, Cross).foreach { case innerJT => + test(s"$outerJT pushdown to right of join type: $innerJT join condition $innerJoinCond") { + val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, innerJoinCond) + val originalQuery = + joinedRelation.join(testRelation, joinType = outerJT, condition = Some('a === 'e)) + val optimized = Optimize.execute(originalQuery.analyze) + + val pushedDownJoin = + testRelation2.join(testRelation, joinType = outerJT, condition = Some('a === 'e)) + val correctAnswer = + testRelation1.join(pushedDownJoin, joinType = innerJT, innerJoinCond).analyze + comparePlans(optimized, correctAnswer) + } + } + } + } + + Seq(LeftSemi, LeftAnti).foreach { case jt => + test(s"$jt no pushdown - join condition refers left leg - join type for RightOuter") { + val joinedRelation = testRelation1.join(testRelation2, joinType = RightOuter, None) + val originalQuery = + joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'd)) + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + } + + Seq(LeftSemi, LeftAnti).foreach { case jt => + test(s"$jt no pushdown - join condition refers right leg - join type for LeftOuter") { + val joinedRelation = testRelation1.join(testRelation2, joinType = LeftOuter, None) + val originalQuery = + joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'e)) + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + } + + Seq(LeftSemi, LeftAnti).foreach { case outerJT => + Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT => + test(s"$outerJT no pushdown - join condition refers both leg - join type $innerJT") { + val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, None) + val originalQuery = joinedRelation + .join(testRelation, joinType = outerJT, condition = Some('a === 'd && 'a === 'e)) + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + } + } + + Seq(LeftSemi, LeftAnti).foreach { case outerJT => + Seq(Inner, LeftOuter, RightOuter, Cross).foreach { case innerJT => + test(s"$outerJT no pushdown - join condition refers none of the leg - join type $innerJT") { + val joinedRelation = testRelation1.join(testRelation2, joinType = innerJT, None) + val originalQuery = joinedRelation + .join(testRelation, joinType = outerJT, condition = Some('d + 'e === 'a)) + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + } + } + + Seq(LeftSemi, LeftAnti).foreach { case jt => + test(s"$jt no pushdown when child join type is FullOuter") { + val joinedRelation = testRelation1.join(testRelation2, joinType = FullOuter, None) + val originalQuery = + joinedRelation.join(testRelation, joinType = jt, condition = Some('a === 'e)) + val optimized = Optimize.execute(originalQuery.analyze) + comparePlans(optimized, originalQuery.analyze) + } + } + +} 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 new file mode 100644 index 0000000000000..2351d8321c5f3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NestedColumnAliasingSuite.scala @@ -0,0 +1,281 @@ +/* + * 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.optimizer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.SchemaPruningTest +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.{StringType, StructField, StructType} + +class NestedColumnAliasingSuite extends SchemaPruningTest { + + import NestedColumnAliasingSuite._ + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("Nested column pruning", FixedPoint(100), + ColumnPruning, + CollapseProject, + RemoveNoopOperators) :: Nil + } + + private val name = StructType.fromDDL("first string, middle string, last string") + private val employer = StructType.fromDDL("id int, company struct") + private val contact = LocalRelation( + 'id.int, + 'name.struct(name), + 'address.string, + 'friends.array(name), + 'relatives.map(StringType, name), + 'employer.struct(employer)) + + test("Pushing a single nested field projection") { + def testSingleFieldPushDown(op: LogicalPlan => LogicalPlan): Unit = { + val middle = GetStructField('name, 1, Some("middle")) + val query = op(contact).select(middle).analyze + val optimized = Optimize.execute(query) + val expected = op(contact.select(middle)).analyze + comparePlans(optimized, expected) + } + + testSingleFieldPushDown((input: LogicalPlan) => input.limit(5)) + testSingleFieldPushDown((input: LogicalPlan) => input.repartition(1)) + testSingleFieldPushDown((input: LogicalPlan) => Sample(0.0, 0.6, false, 11L, input)) + } + + test("Pushing multiple nested field projection") { + val first = GetStructField('name, 0, Some("first")) + val last = GetStructField('name, 2, Some("last")) + + val query = contact + .limit(5) + .select('id, first, last) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select('id, first, last) + .limit(5) + .analyze + + comparePlans(optimized, expected) + } + + test("function with nested field inputs") { + val first = GetStructField('name, 0, Some("first")) + val last = GetStructField('name, 2, Some("last")) + + val query = contact + .limit(5) + .select('id, ConcatWs(Seq(first, last))) + .analyze + + val optimized = Optimize.execute(query) + + val aliases = collectGeneratedAliases(optimized) + + val expected = contact + .select('id, first.as(aliases(0)), last.as(aliases(1))) + .limit(5) + .select( + 'id, + ConcatWs(Seq($"${aliases(0)}", $"${aliases(1)}")).as("concat_ws(name.first, name.last)")) + .analyze + comparePlans(optimized, expected) + } + + test("multi-level nested field") { + val field1 = GetStructField(GetStructField('employer, 1, Some("company")), 0, Some("name")) + val field2 = GetStructField('employer, 0, Some("id")) + + val query = contact + .limit(5) + .select(field1, field2) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select(field1, field2) + .limit(5) + .analyze + comparePlans(optimized, expected) + } + + test("Push original case-sensitive names") { + val first1 = GetStructField('name, 0, Some("first")) + val first2 = GetStructField('name, 1, Some("FIRST")) + + val query = contact + .limit(5) + .select('id, first1, first2) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select('id, first1, first2) + .limit(5) + .analyze + + comparePlans(optimized, expected) + } + + test("Pushing a single nested field projection - negative") { + val ops = Seq( + (input: LogicalPlan) => input.distribute('name)(1), + (input: LogicalPlan) => input.distribute($"name.middle")(1), + (input: LogicalPlan) => input.orderBy('name.asc), + (input: LogicalPlan) => input.orderBy($"name.middle".asc), + (input: LogicalPlan) => input.sortBy('name.asc), + (input: LogicalPlan) => input.sortBy($"name.middle".asc), + (input: LogicalPlan) => input.union(input) + ) + + val queries = ops.map { op => + op(contact.select('name)) + .select(GetStructField('name, 1, Some("middle"))) + .analyze + } + + val optimizedQueries :+ optimizedUnion = queries.map(Optimize.execute) + val expectedQueries = queries.init + optimizedQueries.zip(expectedQueries).foreach { case (optimized, expected) => + comparePlans(optimized, expected) + } + val expectedUnion = + contact.select('name).union(contact.select('name.as('name))) + .select(GetStructField('name, 1, Some("middle"))).analyze + comparePlans(optimizedUnion, expectedUnion) + } + + test("Pushing a single nested field projection through filters - negative") { + val ops = Array( + (input: LogicalPlan) => input.where('name.isNotNull), + (input: LogicalPlan) => input.where($"name.middle".isNotNull) + ) + + val queries = ops.map { op => + op(contact) + .select(GetStructField('name, 1, Some("middle"))) + .analyze + } + + val optimizedQueries = queries.map(Optimize.execute) + val expectedQueries = queries + + optimizedQueries.zip(expectedQueries).foreach { case (optimized, expected) => + comparePlans(optimized, expected) + } + } + + test("Do not optimize when parent field is used") { + val query = contact + .limit(5) + .select('id, GetStructField('name, 0, Some("first")), 'name) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select('id, 'name) + .limit(5) + .select('id, GetStructField('name, 0, Some("first")), 'name) + .analyze + comparePlans(optimized, expected) + } + + test("Some nested column means the whole structure") { + val nestedRelation = LocalRelation('a.struct('b.struct('c.int, 'd.int, 'e.int))) + + val query = nestedRelation + .limit(5) + .select(GetStructField('a, 0, Some("b"))) + .analyze + + val optimized = Optimize.execute(query) + + val expected = nestedRelation + .select(GetStructField('a, 0, Some("b"))) + .limit(5) + .analyze + + comparePlans(optimized, expected) + } + + test("nested field pruning for getting struct field in array of struct") { + val field1 = GetArrayStructFields(child = 'friends, + field = StructField("first", StringType), + ordinal = 0, + numFields = 3, + containsNull = true) + val field2 = GetStructField('employer, 0, Some("id")) + + val query = contact + .limit(5) + .select(field1, field2) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select(field1, field2) + .limit(5) + .analyze + comparePlans(optimized, expected) + } + + test("nested field pruning for getting struct field in map") { + val field1 = GetStructField(GetMapValue('relatives, Literal("key")), 0, Some("first")) + val field2 = GetArrayStructFields(child = MapValues('relatives), + field = StructField("middle", StringType), + ordinal = 1, + numFields = 3, + containsNull = true) + + val query = contact + .limit(5) + .select(field1, field2) + .analyze + + val optimized = Optimize.execute(query) + + val expected = contact + .select(field1, field2) + .limit(5) + .analyze + comparePlans(optimized, expected) + } +} + +object NestedColumnAliasingSuite { + def collectGeneratedAliases(query: LogicalPlan): ArrayBuffer[String] = { + val aliases = ArrayBuffer[String]() + query.transformAllExpressions { + case a @ Alias(_, name) if name.startsWith("_gen_alias_") => + aliases += name + a + } + aliases + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala new file mode 100644 index 0000000000000..f5af416602c9d --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/NormalizeFloatingPointNumbersSuite.scala @@ -0,0 +1,95 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{And, IsNull, KnownFloatingPointNormalized} +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor + +class NormalizeFloatingPointNumbersSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("NormalizeFloatingPointNumbers", Once, NormalizeFloatingNumbers) :: Nil + } + + val testRelation1 = LocalRelation('a.double) + val a = testRelation1.output(0) + val testRelation2 = LocalRelation('a.double) + val b = testRelation2.output(0) + + test("normalize floating points in window function expressions") { + val query = testRelation1.window(Seq(sum(a).as("sum")), Seq(a), Seq(a.asc)) + + val optimized = Optimize.execute(query) + val correctAnswer = testRelation1.window(Seq(sum(a).as("sum")), + Seq(KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), Seq(a.asc)) + + comparePlans(optimized, correctAnswer) + } + + test("normalize floating points in window function expressions - idempotence") { + val query = testRelation1.window(Seq(sum(a).as("sum")), Seq(a), Seq(a.asc)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val correctAnswer = testRelation1.window(Seq(sum(a).as("sum")), + Seq(KnownFloatingPointNormalized(NormalizeNaNAndZero(a))), Seq(a.asc)) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("normalize floating points in join keys") { + val query = testRelation1.join(testRelation2, condition = Some(a === b)) + + val optimized = Optimize.execute(query) + val joinCond = Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)) + === KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) + val correctAnswer = testRelation1.join(testRelation2, condition = joinCond) + + comparePlans(optimized, correctAnswer) + } + + test("normalize floating points in join keys - idempotence") { + val query = testRelation1.join(testRelation2, condition = Some(a === b)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val joinCond = Some(KnownFloatingPointNormalized(NormalizeNaNAndZero(a)) + === KnownFloatingPointNormalized(NormalizeNaNAndZero(b))) + val correctAnswer = testRelation1.join(testRelation2, condition = joinCond) + + comparePlans(doubleOptimized, correctAnswer) + } + + test("normalize floating points in join keys (equal null safe) - idempotence") { + val query = testRelation1.join(testRelation2, condition = Some(a <=> b)) + + val optimized = Optimize.execute(query) + val doubleOptimized = Optimize.execute(optimized) + val joinCond = IsNull(a) === IsNull(b) && + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(a, 0.0))) === + KnownFloatingPointNormalized(NormalizeNaNAndZero(coalesce(b, 0.0))) + val correctAnswer = testRelation1.join(testRelation2, condition = Some(joinCond)) + + comparePlans(doubleOptimized, correctAnswer) + } +} + diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala new file mode 100644 index 0000000000000..0dd4d6a245f5c --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ObjectSerializerPruningSuite.scala @@ -0,0 +1,110 @@ +/* + * 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.optimizer + +import scala.collection.mutable.ArrayBuffer +import scala.reflect.runtime.universe.TypeTag + +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.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +class ObjectSerializerPruningSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("Object serializer pruning", FixedPoint(100), + ObjectSerializerPruning, + RemoveNoopOperators) :: Nil + } + + implicit private def productEncoder[T <: Product : TypeTag] = ExpressionEncoder[T]() + + test("collect struct types") { + val dataTypes = Seq( + IntegerType, + ArrayType(IntegerType), + StructType.fromDDL("a int, b int"), + ArrayType(StructType.fromDDL("a int, b int, c string")), + StructType.fromDDL("a struct, b int"), + MapType(IntegerType, StructType.fromDDL("a int, b int, c string")), + MapType(StructType.fromDDL("a struct, b int"), IntegerType), + MapType(StructType.fromDDL("a int, b int"), StructType.fromDDL("c long, d string")) + ) + + val expectedTypes = Seq( + Seq.empty[StructType], + Seq.empty[StructType], + Seq(StructType.fromDDL("a int, b int")), + Seq(StructType.fromDDL("a int, b int, c string")), + Seq(StructType.fromDDL("a struct, b int"), + StructType.fromDDL("a int, b int")), + Seq(StructType.fromDDL("a int, b int, c string")), + Seq(StructType.fromDDL("a struct, b int"), + StructType.fromDDL("a int, b int")), + Seq(StructType.fromDDL("a int, b int"), StructType.fromDDL("c long, d string")) + ) + + dataTypes.zipWithIndex.foreach { case (dt, idx) => + val structs = ObjectSerializerPruning.collectStructType(dt, ArrayBuffer.empty[StructType]) + assert(structs === expectedTypes(idx)) + } + } + + test("SPARK-26619: Prune the unused serializers from SerializeFromObject") { + val testRelation = LocalRelation('_1.int, '_2.int) + val serializerObject = CatalystSerde.serialize[(Int, Int)]( + CatalystSerde.deserialize[(Int, Int)](testRelation)) + val query = serializerObject.select('_1) + val optimized = Optimize.execute(query.analyze) + val expected = serializerObject.copy(serializer = Seq(serializerObject.serializer.head)).analyze + comparePlans(optimized, expected) + } + + test("Prune nested serializers") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val testRelation = LocalRelation('_1.struct(StructType.fromDDL("_1 int, _2 string")), '_2.int) + val serializerObject = CatalystSerde.serialize[((Int, String), Int)]( + CatalystSerde.deserialize[((Int, String), Int)](testRelation)) + val query = serializerObject.select($"_1._1") + val optimized = Optimize.execute(query.analyze) + + val prunedSerializer = serializerObject.serializer.head.transformDown { + case CreateNamedStruct(children) => + CreateNamedStruct(children.take(2)) + }.transformUp { + // Aligns null literal in `If` expression to make it resolvable. + case i @ If(_: IsNull, Literal(null, dt), ser) if !dt.sameType(ser.dataType) => + i.copy(trueValue = Literal(null, ser.dataType)) + }.asInstanceOf[NamedExpression] + + // `name` in `GetStructField` affects `comparePlans`. Maybe we can ignore + // `name` in `GetStructField.equals`? + val expected = serializerObject.copy(serializer = Seq(prunedSerializer)) + .select($"_1._1").analyze.transformAllExpressions { + case g: GetStructField => g.copy(name = None) + } + comparePlans(optimized, expected) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala new file mode 100644 index 0000000000000..cf875efc62c98 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeLimitZeroSuite.scala @@ -0,0 +1,108 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{Distinct, GlobalLimit, LocalLimit, LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.IntegerType + +// Test class to verify correct functioning of OptimizeLimitZero rule in various scenarios +class OptimizeLimitZeroSuite extends PlanTest { + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("OptimizeLimitZero", Once, + ReplaceIntersectWithSemiJoin, + OptimizeLimitZero, + PropagateEmptyRelation) :: Nil + } + + val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1))) + val testRelation2 = LocalRelation.fromExternalRows(Seq('b.int), data = Seq(Row(1))) + + test("Limit 0: return empty local relation") { + val query = testRelation1.limit(0) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = LocalRelation('a.int) + + comparePlans(optimized, correctAnswer) + } + + test("Limit 0: individual LocalLimit 0 node") { + val query = LocalLimit(0, testRelation1) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = LocalRelation('a.int) + + comparePlans(optimized, correctAnswer) + } + + test("Limit 0: individual GlobalLimit 0 node") { + val query = GlobalLimit(0, testRelation1) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = LocalRelation('a.int) + + comparePlans(optimized, correctAnswer) + } + + Seq( + (Inner, LocalRelation('a.int, 'b.int)), + (LeftOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze), + (RightOuter, LocalRelation('a.int, 'b.int)), + (FullOuter, Project(Seq('a, Literal(null).cast(IntegerType).as('b)), testRelation1).analyze) + ).foreach { case (jt, correctAnswer) => + test(s"Limit 0: for join type $jt") { + val query = testRelation1 + .join(testRelation2.limit(0), joinType = jt, condition = Some('a.attr == 'b.attr)) + + val optimized = Optimize.execute(query.analyze) + + comparePlans(optimized, correctAnswer) + } + } + + test("Limit 0: 3-way join") { + val testRelation3 = LocalRelation.fromExternalRows(Seq('c.int), data = Seq(Row(1))) + + val subJoinQuery = testRelation1 + .join(testRelation2, joinType = Inner, condition = Some('a.attr == 'b.attr)) + val query = subJoinQuery + .join(testRelation3.limit(0), joinType = Inner, condition = Some('a.attr == 'c.attr)) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = LocalRelation('a.int, 'b.int, 'c.int) + + comparePlans(optimized, correctAnswer) + } + + test("Limit 0: intersect") { + val query = testRelation1 + .intersect(testRelation1.limit(0), isAll = false) + + val optimized = Optimize.execute(query.analyze) + val correctAnswer = Distinct(LocalRelation('a.int)) + + comparePlans(optimized, correctAnswer) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala index 915f408089fe9..7a432d269abe6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerLoggingSuite.scala @@ -32,17 +32,20 @@ import org.apache.spark.sql.internal.SQLConf class OptimizerLoggingSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { - val batches = Batch("Optimizer Batch", FixedPoint(100), - PushDownPredicate, - ColumnPruning, - CollapseProject) :: Nil + val batches = + Batch("Optimizer Batch", FixedPoint(100), + PushPredicateThroughNonJoin, ColumnPruning, CollapseProject) :: + Batch("Batch Has No Effect", Once, + ColumnPruning) :: Nil } class MockAppender extends AppenderSkeleton { val loggingEvents = new ArrayBuffer[LoggingEvent]() override def append(loggingEvent: LoggingEvent): Unit = { - if (loggingEvent.getRenderedMessage().contains("Applying Rule")) { + if (loggingEvent.getRenderedMessage().contains("Applying Rule") || + loggingEvent.getRenderedMessage().contains("Result of Batch") || + loggingEvent.getRenderedMessage().contains("has no effect")) { loggingEvents.append(loggingEvent) } } @@ -62,16 +65,18 @@ class OptimizerLoggingSuite extends PlanTest { } } - private def verifyLog(expectedLevel: Level, expectedRules: Seq[String]): Unit = { + private def verifyLog(expectedLevel: Level, expectedRulesOrBatches: Seq[String]): Unit = { val logAppender = new MockAppender() - withLogLevelAndAppender(Level.TRACE, logAppender) { + withLogAppender(logAppender, + loggerName = Some(Optimize.getClass.getName.dropRight(1)), level = Some(Level.TRACE)) { val input = LocalRelation('a.int, 'b.string, 'c.double) val query = input.select('a, 'b).select('a).where('a > 1).analyze val expected = input.where('a > 1).select('a).analyze comparePlans(Optimize.execute(query), expected) } val logMessages = logAppender.loggingEvents.map(_.getRenderedMessage) - assert(expectedRules.forall(rule => logMessages.exists(_.contains(rule)))) + assert(expectedRulesOrBatches.forall + (ruleOrBatch => logMessages.exists(_.contains(ruleOrBatch)))) assert(logAppender.loggingEvents.forall(_.getLevel == expectedLevel)) } @@ -94,7 +99,7 @@ class OptimizerLoggingSuite extends PlanTest { verifyLog( level._2, Seq( - PushDownPredicate.ruleName, + PushPredicateThroughNonJoin.ruleName, ColumnPruning.ruleName, CollapseProject.ruleName)) } @@ -118,15 +123,15 @@ class OptimizerLoggingSuite extends PlanTest { test("test log rules") { val rulesSeq = Seq( - Seq(PushDownPredicate.ruleName, + Seq(PushPredicateThroughNonJoin.ruleName, ColumnPruning.ruleName, CollapseProject.ruleName).reduce(_ + "," + _) -> - Seq(PushDownPredicate.ruleName, + Seq(PushPredicateThroughNonJoin.ruleName, ColumnPruning.ruleName, CollapseProject.ruleName), - Seq(PushDownPredicate.ruleName, + Seq(PushPredicateThroughNonJoin.ruleName, ColumnPruning.ruleName).reduce(_ + "," + _) -> - Seq(PushDownPredicate.ruleName, + Seq(PushPredicateThroughNonJoin.ruleName, ColumnPruning.ruleName), CollapseProject.ruleName -> Seq(CollapseProject.ruleName), @@ -145,4 +150,20 @@ class OptimizerLoggingSuite extends PlanTest { } } } + + test("test log batches which change the plan") { + withSQLConf( + SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_BATCHES.key -> "Optimizer Batch", + SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_LEVEL.key -> "INFO") { + verifyLog(Level.INFO, Seq("Optimizer Batch")) + } + } + + test("test log batches which do not change the plan") { + withSQLConf( + SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_BATCHES.key -> "Batch Has No Effect", + SQLConf.OPTIMIZER_PLAN_CHANGE_LOG_LEVEL.key -> "INFO") { + verifyLog(Level.INFO, Seq("Batch Has No Effect")) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala index 4fa4a7aadc8f2..2a8780346d99e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerRuleExclusionSuite.scala @@ -96,21 +96,21 @@ class OptimizerRuleExclusionSuite extends PlanTest { val optimizer = new SimpleTestOptimizer() { override def defaultBatches: Seq[Batch] = Batch("push", Once, - PushDownPredicate, + PushPredicateThroughNonJoin, PushPredicateThroughJoin, PushProjectionThroughUnion) :: Batch("pull", Once, PullupCorrelatedPredicates) :: Nil override def nonExcludableRules: Seq[String] = - PushDownPredicate.ruleName :: + PushPredicateThroughNonJoin.ruleName :: PullupCorrelatedPredicates.ruleName :: Nil } verifyExcludedRules( optimizer, Seq( - PushDownPredicate.ruleName, + PushPredicateThroughNonJoin.ruleName, PushProjectionThroughUnion.ruleName, PullupCorrelatedPredicates.ruleName)) } @@ -121,10 +121,14 @@ class OptimizerRuleExclusionSuite extends PlanTest { PropagateEmptyRelation.ruleName, CombineUnions.ruleName) + val testRelation1 = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + val testRelation3 = LocalRelation('a.int, 'b.int, 'c.int) + withSQLConf( OPTIMIZER_EXCLUDED_RULES.key -> excludedRules.foldLeft("")((l, r) => l + "," + r)) { val optimizer = new SimpleTestOptimizer() - val originalQuery = testRelation.union(testRelation.union(testRelation)).analyze + val originalQuery = testRelation1.union(testRelation2.union(testRelation3)).analyze val optimized = optimizer.execute(originalQuery) comparePlans(originalQuery, optimized) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala index a22a81e9844d3..5e0d2041fac5d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerStructuralIntegrityCheckerSuite.scala @@ -19,11 +19,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.{EmptyFunctionRegistry, UnresolvedAttribute} 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.errors.TreeNodeException -import org.apache.spark.sql.catalyst.expressions.{Alias, Literal} +import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, NamedExpression} import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation, Project} +import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LocalRelation, LogicalPlan, OneRowRelation, Project} import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf @@ -35,6 +36,9 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { case Project(projectList, child) => val newAttr = UnresolvedAttribute("unresolvedAttr") Project(projectList ++ Seq(newAttr), child) + case agg @ Aggregate(Nil, aggregateExpressions, child) => + // Project cannot host AggregateExpression + Project(aggregateExpressions, child) } } @@ -47,7 +51,7 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { override def defaultBatches: Seq[Batch] = Seq(newBatch) ++ super.defaultBatches } - test("check for invalid plan after execution of rule") { + test("check for invalid plan after execution of rule - unresolved attribute") { val analyzed = Project(Alias(Literal(10), "attr")() :: Nil, OneRowRelation()).analyze assert(analyzed.resolved) val message = intercept[TreeNodeException[LogicalPlan]] { @@ -57,4 +61,35 @@ class OptimizerStructuralIntegrityCheckerSuite extends PlanTest { assert(message.contains(s"After applying rule $ruleName in batch OptimizeRuleBreakSI")) assert(message.contains("the structural integrity of the plan is broken")) } + + test("check for invalid plan after execution of rule - special expression in wrong operator") { + val analyzed = + Aggregate(Nil, Seq[NamedExpression](max('id) as 'm), + LocalRelation('id.long)).analyze + assert(analyzed.resolved) + + // Should fail verification with the OptimizeRuleBreakSI rule + val message = intercept[TreeNodeException[LogicalPlan]] { + Optimize.execute(analyzed) + }.getMessage + val ruleName = OptimizeRuleBreakSI.ruleName + assert(message.contains(s"After applying rule $ruleName in batch OptimizeRuleBreakSI")) + assert(message.contains("the structural integrity of the plan is broken")) + + // Should not fail verification with the regular optimizer + SimpleTestOptimizer.execute(analyzed) + } + + test("check for invalid plan before execution of any rule") { + val analyzed = + Aggregate(Nil, Seq[NamedExpression](max('id) as 'm), + LocalRelation('id.long)).analyze + val invalidPlan = OptimizeRuleBreakSI.apply(analyzed) + + // Should fail verification right at the beginning + val message = intercept[TreeNodeException[LogicalPlan]] { + Optimize.execute(invalidPlan) + }.getMessage + assert(message.contains("The structural integrity of the input plan is broken")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala index d395bba105a7b..9c7d4c7d8d233 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala @@ -35,7 +35,7 @@ class PropagateEmptyRelationSuite extends PlanTest { ReplaceDistinctWithAggregate, ReplaceExceptWithAntiJoin, ReplaceIntersectWithSemiJoin, - PushDownPredicate, + PushPredicateThroughNonJoin, PruneFilters, PropagateEmptyRelation, CollapseProject) :: Nil @@ -48,7 +48,7 @@ class PropagateEmptyRelationSuite extends PlanTest { ReplaceDistinctWithAggregate, ReplaceExceptWithAntiJoin, ReplaceIntersectWithSemiJoin, - PushDownPredicate, + PushPredicateThroughNonJoin, PruneFilters, CollapseProject) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala index 6d1a05f3c998e..526a5b01939df 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala @@ -35,7 +35,7 @@ class PruneFiltersSuite extends PlanTest { Batch("Filter Pushdown and Pruning", Once, CombineFilters, PruneFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, PushPredicateThroughJoin) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala new file mode 100644 index 0000000000000..4a25ddf3ed9e9 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PullOutPythonUDFInJoinConditionSuite.scala @@ -0,0 +1,198 @@ +/* + * 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.optimizer + +import org.apache.spark.api.python.PythonEvalType +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.PythonUDF +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf._ +import org.apache.spark.sql.types.{BooleanType, IntegerType} + +class PullOutPythonUDFInJoinConditionSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Extract PythonUDF From JoinCondition", Once, + PullOutPythonUDFInJoinCondition) :: + Batch("Check Cartesian Products", Once, + CheckCartesianProducts) :: Nil + } + + val attrA = 'a.int + val attrB = 'b.int + val attrC = 'c.int + val attrD = 'd.int + + val testRelationLeft = LocalRelation(attrA, attrB) + val testRelationRight = LocalRelation(attrC, attrD) + + // This join condition refers to attributes from 2 tables, but the PythonUDF inside it only + // refer to attributes from one side. + val evaluableJoinCond = { + val pythonUDF = PythonUDF("evaluable", null, + IntegerType, + Seq(attrA), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + pythonUDF === attrC + } + + // This join condition is a PythonUDF which refers to attributes from 2 tables. + val unevaluableJoinCond = PythonUDF("unevaluable", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + + val unsupportedJoinTypes = Seq(LeftOuter, RightOuter, FullOuter, LeftAnti, LeftSemi) + + private def comparePlanWithCrossJoinEnable(query: LogicalPlan, expected: LogicalPlan): Unit = { + // AnalysisException thrown by CheckCartesianProducts while spark.sql.crossJoin.enabled=false + val exception = intercept[AnalysisException] { + Optimize.execute(query.analyze) + } + assert(exception.message.startsWith("Detected implicit cartesian product")) + + // pull out the python udf while set spark.sql.crossJoin.enabled=true + withSQLConf(CROSS_JOINS_ENABLED.key -> "true") { + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + } + + test("inner join condition with python udf") { + val query1 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond)) + val expected1 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(unevaluableJoinCond).analyze + comparePlanWithCrossJoinEnable(query1, expected1) + + // evaluable PythonUDF will not be touched + val query2 = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) + } + + test("unevaluable python udf and common condition") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond && 'a.attr === 'c.attr)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("unevaluable python udf or common condition") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(unevaluableJoinCond || 'a.attr === 'c.attr)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(unevaluableJoinCond || 'a.attr === 'c.attr).analyze + comparePlanWithCrossJoinEnable(query, expected) + } + + test("pull out whole complex condition with multiple unevaluable python udf") { + val pythonUDF1 = PythonUDF("pythonUDF1", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + val condition = (unevaluableJoinCond || 'a.attr === 'c.attr) && pythonUDF1 + + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(condition)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = None).where(condition).analyze + comparePlanWithCrossJoinEnable(query, expected) + } + + test("partial pull out complex condition with multiple unevaluable python udf") { + val pythonUDF1 = PythonUDF("pythonUDF1", null, + BooleanType, + Seq(attrA, attrC), + PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) + val condition = (unevaluableJoinCond || pythonUDF1) && 'a.attr === 'c.attr + + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(condition)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some('a.attr === 'c.attr)).where(unevaluableJoinCond || pythonUDF1).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("pull out unevaluable python udf when it's mixed with evaluable one") { + val query = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond && unevaluableJoinCond)) + val expected = testRelationLeft.join( + testRelationRight, + joinType = Inner, + condition = Some(evaluableJoinCond)).where(unevaluableJoinCond).analyze + val optimized = Optimize.execute(query.analyze) + comparePlans(optimized, expected) + } + + test("throw an exception for not supported join types") { + for (joinType <- unsupportedJoinTypes) { + val e = intercept[AnalysisException] { + val query = testRelationLeft.join( + testRelationRight, + joinType, + condition = Some(unevaluableJoinCond)) + Optimize.execute(query.analyze) + } + assert(e.message.contentEquals( + s"Using PythonUDF in join condition of join type $joinType is not supported.")) + + val query2 = testRelationLeft.join( + testRelationRight, + joinType, + condition = Some(evaluableJoinCond)) + comparePlans(Optimize.execute(query2), query2) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala new file mode 100644 index 0000000000000..06a32c77ac5ec --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReassignLambdaVariableIDSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.objects.LambdaVariable +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.BooleanType + +class ReassignLambdaVariableIDSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("Optimizer Batch", FixedPoint(100), ReassignLambdaVariableID) :: Nil + } + + test("basic: replace positive IDs with unique negative IDs") { + val testRelation = LocalRelation('col.int) + val var1 = LambdaVariable("a", BooleanType, true, id = 2) + val var2 = LambdaVariable("b", BooleanType, true, id = 4) + val query = testRelation.where(var1 && var2) + val optimized = Optimize.execute(query) + val expected = testRelation.where(var1.copy(id = -1) && var2.copy(id = -2)) + comparePlans(optimized, expected) + } + + test("ignore LambdaVariable with negative IDs") { + val testRelation = LocalRelation('col.int) + val var1 = LambdaVariable("a", BooleanType, true, id = -2) + val var2 = LambdaVariable("b", BooleanType, true, id = -4) + val query = testRelation.where(var1 && var2) + val optimized = Optimize.execute(query) + comparePlans(optimized, query) + } + + test("fail if positive ID LambdaVariable and negative LambdaVariable both exist") { + val testRelation = LocalRelation('col.int) + val var1 = LambdaVariable("a", BooleanType, true, id = -2) + val var2 = LambdaVariable("b", BooleanType, true, id = 4) + val query = testRelation.where(var1 && var2) + val e = intercept[IllegalStateException](Optimize.execute(query)) + assert(e.getMessage.contains("should be all positive or negative")) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala index 1973b5abb462d..3802dbf5d6e06 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RemoveRedundantAliasAndProjectSuite.scala @@ -33,7 +33,7 @@ class RemoveRedundantAliasAndProjectSuite extends PlanTest with PredicateHelper FixedPoint(50), PushProjectionThroughUnion, RemoveRedundantAliases, - RemoveRedundantProject) :: Nil + RemoveNoopOperators) :: Nil } test("all expressions in project list are aliased child output") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala new file mode 100644 index 0000000000000..b692c3fee53c7 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceNullWithFalseInPredicateSuite.scala @@ -0,0 +1,384 @@ +/* + * 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.optimizer + +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{And, ArrayExists, ArrayFilter, ArrayTransform, CaseWhen, Expression, GreaterThan, If, LambdaFunction, Literal, MapFilter, NamedExpression, Or, UnresolvedNamedLambdaVariable} +import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{BooleanType, IntegerType} + +class ReplaceNullWithFalseInPredicateSuite extends PlanTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Replace null literals", FixedPoint(10), + NullPropagation, + ConstantFolding, + BooleanSimplification, + SimplifyConditionals, + ReplaceNullWithFalseInPredicate) :: Nil + } + + private val testRelation = + LocalRelation('i.int, 'b.boolean, 'a.array(IntegerType), 'm.map(IntegerType, IntegerType)) + private val anotherTestRelation = LocalRelation('d.int) + + test("replace null inside filter and join conditions") { + testFilter(originalCond = Literal(null, BooleanType), expectedCond = FalseLiteral) + testJoin(originalCond = Literal(null, BooleanType), expectedCond = FalseLiteral) + } + + test("Not expected type - replaceNullWithFalse") { + val e = intercept[IllegalArgumentException] { + testFilter(originalCond = Literal(null, IntegerType), expectedCond = FalseLiteral) + }.getMessage + assert(e.contains("but got the type `int` in `CAST(NULL AS INT)")) + } + + test("replace null in branches of If") { + val originalCond = If( + UnresolvedAttribute("i") > Literal(10), + FalseLiteral, + Literal(null, BooleanType)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace nulls in nested expressions in branches of If") { + val originalCond = If( + UnresolvedAttribute("i") > Literal(10), + TrueLiteral && Literal(null, BooleanType), + UnresolvedAttribute("b") && Literal(null, BooleanType)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in elseValue of CaseWhen") { + val branches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, + (UnresolvedAttribute("i") > Literal(40)) -> FalseLiteral) + val originalCond = CaseWhen(branches, Literal(null, BooleanType)) + val expectedCond = CaseWhen(branches, FalseLiteral) + testFilter(originalCond, expectedCond) + testJoin(originalCond, expectedCond) + } + + test("replace null in branch values of CaseWhen") { + val branches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> Literal(null, BooleanType), + (UnresolvedAttribute("i") > Literal(40)) -> FalseLiteral) + val originalCond = CaseWhen(branches, Literal(null)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in branches of If inside CaseWhen") { + val originalBranches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> + If(UnresolvedAttribute("i") < Literal(20), Literal(null, BooleanType), FalseLiteral), + (UnresolvedAttribute("i") > Literal(40)) -> TrueLiteral) + val originalCond = CaseWhen(originalBranches) + + val expectedBranches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> FalseLiteral, + (UnresolvedAttribute("i") > Literal(40)) -> TrueLiteral) + val expectedCond = CaseWhen(expectedBranches) + + testFilter(originalCond, expectedCond) + testJoin(originalCond, expectedCond) + } + + test("replace null in complex CaseWhen expressions") { + val originalBranches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, + (Literal(6) <= Literal(1)) -> FalseLiteral, + (Literal(4) === Literal(5)) -> FalseLiteral, + (UnresolvedAttribute("i") > Literal(10)) -> Literal(null, BooleanType), + (Literal(4) === Literal(4)) -> TrueLiteral) + val originalCond = CaseWhen(originalBranches) + + val expectedBranches = Seq( + (UnresolvedAttribute("i") < Literal(10)) -> TrueLiteral, + (UnresolvedAttribute("i") > Literal(10)) -> FalseLiteral, + TrueLiteral -> TrueLiteral) + val expectedCond = CaseWhen(expectedBranches) + + testFilter(originalCond, expectedCond) + testJoin(originalCond, expectedCond) + } + + test("replace null in Or") { + val originalCond = Or(UnresolvedAttribute("b"), Literal(null)) + val expectedCond = UnresolvedAttribute("b") + testFilter(originalCond, expectedCond) + testJoin(originalCond, expectedCond) + } + + test("replace null in And") { + val originalCond = And(UnresolvedAttribute("b"), Literal(null)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace nulls in nested And/Or expressions") { + val originalCond = And( + And(UnresolvedAttribute("b"), Literal(null)), + Or(Literal(null), And(Literal(null), And(UnresolvedAttribute("b"), Literal(null))))) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in And inside branches of If") { + val originalCond = If( + UnresolvedAttribute("i") > Literal(10), + FalseLiteral, + And(UnresolvedAttribute("b"), Literal(null, BooleanType))) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in branches of If inside And") { + val originalCond = And( + UnresolvedAttribute("b"), + If( + UnresolvedAttribute("i") > Literal(10), + Literal(null), + And(FalseLiteral, UnresolvedAttribute("b")))) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in branches of If inside another If") { + val originalCond = If( + If(UnresolvedAttribute("b"), Literal(null), FalseLiteral), + TrueLiteral, + Literal(null)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in CaseWhen inside another CaseWhen") { + val nestedCaseWhen = CaseWhen(Seq(UnresolvedAttribute("b") -> FalseLiteral), Literal(null)) + val originalCond = CaseWhen(Seq(nestedCaseWhen -> TrueLiteral), Literal(null)) + testFilter(originalCond, expectedCond = FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("inability to replace null in non-boolean branches of If") { + val condition = If( + UnresolvedAttribute("i") > Literal(10), + Literal(5) > If( + UnresolvedAttribute("i") === Literal(15), + Literal(null, IntegerType), + Literal(3)), + FalseLiteral) + testFilter(originalCond = condition, expectedCond = condition) + testJoin(originalCond = condition, expectedCond = condition) + } + + test("inability to replace null in non-boolean values of CaseWhen") { + val nestedCaseWhen = CaseWhen( + Seq((UnresolvedAttribute("i") > Literal(20)) -> Literal(2)), + Literal(null, IntegerType)) + val branchValue = If( + Literal(2) === nestedCaseWhen, + TrueLiteral, + FalseLiteral) + val branches = Seq((UnresolvedAttribute("i") > Literal(10)) -> branchValue) + val condition = CaseWhen(branches) + testFilter(originalCond = condition, expectedCond = condition) + testJoin(originalCond = condition, expectedCond = condition) + } + + test("inability to replace null in non-boolean branches of If inside another If") { + val condition = If( + Literal(5) > If( + UnresolvedAttribute("i") === Literal(15), + Literal(null, IntegerType), + Literal(3)), + TrueLiteral, + FalseLiteral) + testFilter(originalCond = condition, expectedCond = condition) + testJoin(originalCond = condition, expectedCond = condition) + } + + test("replace null in If used as a join condition") { + // this test is only for joins as the condition involves columns from different relations + val originalCond = If( + UnresolvedAttribute("d") > UnresolvedAttribute("i"), + Literal(null), + FalseLiteral) + testJoin(originalCond, expectedCond = FalseLiteral) + } + + test("replace null in CaseWhen used as a join condition") { + // this test is only for joins as the condition involves columns from different relations + val originalBranches = Seq( + (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> Literal(null), + (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) + + val expectedBranches = Seq( + (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> FalseLiteral, + (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) + + testJoin( + originalCond = CaseWhen(originalBranches, FalseLiteral), + expectedCond = CaseWhen(expectedBranches, FalseLiteral)) + } + + test("inability to replace null in CaseWhen inside EqualTo used as a join condition") { + // this test is only for joins as the condition involves columns from different relations + val branches = Seq( + (UnresolvedAttribute("d") > UnresolvedAttribute("i")) -> Literal(null, BooleanType), + (UnresolvedAttribute("d") === UnresolvedAttribute("i")) -> TrueLiteral) + val condition = UnresolvedAttribute("b") === CaseWhen(branches, FalseLiteral) + testJoin(originalCond = condition, expectedCond = condition) + } + + test("replace null in predicates of If") { + val predicate = And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)) + testProjection( + originalExpr = If(predicate, Literal(5), Literal(1)).as("out"), + expectedExpr = Literal(1).as("out")) + } + + test("replace null in predicates of If inside another If") { + val predicate = If( + And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)), + TrueLiteral, + FalseLiteral) + testProjection( + originalExpr = If(predicate, Literal(5), Literal(1)).as("out"), + expectedExpr = Literal(1).as("out")) + } + + test("inability to replace null in non-boolean expressions inside If predicates") { + val predicate = GreaterThan( + UnresolvedAttribute("i"), + If(UnresolvedAttribute("b"), Literal(null, IntegerType), Literal(4))) + val column = If(predicate, Literal(5), Literal(1)).as("out") + testProjection(originalExpr = column, expectedExpr = column) + } + + test("replace null in conditions of CaseWhen") { + val branches = Seq( + And(GreaterThan(UnresolvedAttribute("i"), Literal(0.5)), Literal(null)) -> Literal(5)) + testProjection( + originalExpr = CaseWhen(branches, Literal(2)).as("out"), + expectedExpr = Literal(2).as("out")) + } + + test("replace null in conditions of CaseWhen inside another CaseWhen") { + val nestedCaseWhen = CaseWhen( + Seq(And(UnresolvedAttribute("b"), Literal(null)) -> Literal(5)), + Literal(2)) + val branches = Seq(GreaterThan(Literal(3), nestedCaseWhen) -> Literal(1)) + testProjection( + originalExpr = CaseWhen(branches).as("out"), + expectedExpr = Literal(1).as("out")) + } + + test("inability to replace null in non-boolean exprs inside CaseWhen conditions") { + val condition = GreaterThan( + UnresolvedAttribute("i"), + If(UnresolvedAttribute("b"), Literal(null, IntegerType), Literal(4))) + val column = CaseWhen(Seq(condition -> Literal(5)), Literal(2)).as("out") + testProjection(originalExpr = column, expectedExpr = column) + } + + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + + test("replace nulls in lambda function of ArrayFilter") { + testHigherOrderFunc('a, ArrayFilter, Seq(lv('e))) + } + + test("replace nulls in lambda function of ArrayExists") { + withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key -> "true") { + val lambdaArgs = Seq(lv('e)) + val cond = GreaterThan(lambdaArgs.last, Literal(0)) + val lambda = LambdaFunction( + function = If(cond, Literal(null, BooleanType), TrueLiteral), + arguments = lambdaArgs) + val expr = ArrayExists('a, lambda) + testProjection(originalExpr = expr, expectedExpr = expr) + } + withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key -> "false") { + testHigherOrderFunc('a, ArrayExists, Seq(lv('e))) + } + } + + test("replace nulls in lambda function of MapFilter") { + testHigherOrderFunc('m, MapFilter, Seq(lv('k), lv('v))) + } + + test("inability to replace nulls in arbitrary higher-order function") { + val lambdaFunc = LambdaFunction( + function = If(lv('e) > 0, Literal(null, BooleanType), TrueLiteral), + arguments = Seq[NamedExpression](lv('e))) + val column = ArrayTransform('a, lambdaFunc) + testProjection(originalExpr = column, expectedExpr = column) + } + + private def testFilter(originalCond: Expression, expectedCond: Expression): Unit = { + test((rel, exp) => rel.where(exp), originalCond, expectedCond) + } + + private def testJoin(originalCond: Expression, expectedCond: Expression): Unit = { + test((rel, exp) => rel.join(anotherTestRelation, Inner, Some(exp)), originalCond, expectedCond) + } + + private def testProjection(originalExpr: Expression, expectedExpr: Expression): Unit = { + test((rel, exp) => rel.select(exp), originalExpr, expectedExpr) + } + + private def testHigherOrderFunc( + argument: Expression, + createExpr: (Expression, Expression) => Expression, + lambdaArgs: Seq[NamedExpression]): Unit = { + val condArg = lambdaArgs.last + // the lambda body is: if(arg > 0, null, true) + val cond = GreaterThan(condArg, Literal(0)) + val lambda1 = LambdaFunction( + function = If(cond, Literal(null, BooleanType), TrueLiteral), + arguments = lambdaArgs) + // the optimized lambda body is: if(arg > 0, false, true) + val lambda2 = LambdaFunction( + function = If(cond, FalseLiteral, TrueLiteral), + arguments = lambdaArgs) + testProjection( + originalExpr = createExpr(argument, lambda1) as 'x, + expectedExpr = createExpr(argument, lambda2) as 'x) + } + + private def test( + func: (LogicalPlan, Expression) => LogicalPlan, + originalExpr: Expression, + expectedExpr: Expression): Unit = { + + val originalPlan = func(testRelation, originalExpr).analyze + val optimizedPlan = Optimize.execute(originalPlan) + val expectedPlan = func(testRelation, expectedExpr).analyze + comparePlans(optimizedPlan, expectedPlan) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala index 3b1b2d588ef67..9bf864f5201ff 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ReplaceOperatorSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, Literal, Not} +import org.apache.spark.sql.catalyst.expressions.{Alias, Coalesce, If, Literal, Not} import org.apache.spark.sql.catalyst.expressions.aggregate.First import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi, PlanTest} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.types.BooleanType class ReplaceOperatorSuite extends PlanTest { @@ -47,7 +48,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftSemi, Option('a <=> 'c && 'b <=> 'd))).analyze + Join(table1, table2, LeftSemi, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -65,8 +66,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -84,8 +84,8 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), table1)).analyze + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), + table1)).analyze comparePlans(optimized, correctAnswer) } @@ -104,8 +104,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), table1))).analyze comparePlans(optimized, correctAnswer) @@ -125,8 +124,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA >= 2 && attributeB < 1)), + Filter(Not(Coalesce(Seq(attributeA >= 2 && attributeB < 1, Literal.FalseLiteral))), Filter(attributeB === 2, Filter(attributeA === 1, table1)))).analyze comparePlans(optimized, correctAnswer) @@ -146,8 +144,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Filter(Not((attributeA.isNotNull && attributeB.isNotNull) && - (attributeA === 1 && attributeB === 2)), + Filter(Not(Coalesce(Seq(attributeA === 1 && attributeB === 2, Literal.FalseLiteral))), Project(Seq(attributeA, attributeB), Filter(attributeB < 1, Filter(attributeA >= 2, table1))))).analyze @@ -163,7 +160,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(table1.output, table1.output, - Join(table1, table2, LeftAnti, Option('a <=> 'c && 'b <=> 'd))).analyze + Join(table1, table2, LeftAnti, Option('a <=> 'c && 'b <=> 'd), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -178,7 +175,7 @@ class ReplaceOperatorSuite extends PlanTest { val correctAnswer = Aggregate(left.output, right.output, - Join(left, right, LeftAnti, Option($"left.a" <=> $"right.a"))).analyze + Join(left, right, LeftAnti, Option($"left.a" <=> $"right.a"), JoinHint.NONE)).analyze comparePlans(optimized, correctAnswer) } @@ -229,4 +226,29 @@ class ReplaceOperatorSuite extends PlanTest { comparePlans(optimized, query) } + + test("SPARK-26366: ReplaceExceptWithFilter should handle properly NULL") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a.in(1, 2) || 'b.in()) + val except = Except(basePlan, otherPlan, false) + val result = OptimizeIn(Optimize.execute(except.analyze)) + val correctAnswer = Aggregate(basePlan.output, basePlan.output, + Filter(!Coalesce(Seq( + 'a.in(1, 2) || If('b.isNotNull, Literal.FalseLiteral, Literal(null, BooleanType)), + Literal.FalseLiteral)), + basePlan)).analyze + comparePlans(result, correctAnswer) + } + + test("SPARK-26366: ReplaceExceptWithFilter should not transform non-deterministic") { + val basePlan = LocalRelation(Seq('a.int, 'b.int)) + val otherPlan = basePlan.where('a > rand(1L)) + val except = Except(basePlan, otherPlan, false) + val result = Optimize.execute(except.analyze) + val condition = basePlan.output.zip(otherPlan.output).map { case (a1, a2) => + a1 <=> a2 }.reduce( _ && _) + val correctAnswer = Aggregate(basePlan.output, otherPlan.output, + Join(basePlan, otherPlan, LeftAnti, Option(condition), JoinHint.NONE)).analyze + comparePlans(result, correctAnswer) + } } 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 6b3739c372c3a..f00d22e6e96a6 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 @@ -34,7 +34,7 @@ class RewriteSubquerySuite extends PlanTest { RewritePredicateSubquery, ColumnPruning, CollapseProject, - RemoveRedundantProject) :: Nil + RemoveNoopOperators) :: Nil } test("Column pruning after rewriting predicate subquery") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala index da3923f8d6477..ccc30b1d2f8ce 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{Alias, And, GreaterThan, GreaterThanOrEqual, If, Literal, ReplicateRows} +import org.apache.spark.sql.catalyst.expressions.{And, GreaterThan, GreaterThanOrEqual, If, Literal, Rand, ReplicateRows} import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -31,10 +31,10 @@ class SetOperationSuite extends PlanTest { val batches = Batch("Subqueries", Once, EliminateSubqueryAliases) :: - Batch("Union Pushdown", Once, + Batch("Union Pushdown", FixedPoint(5), CombineUnions, PushProjectionThroughUnion, - PushDownPredicate, + PushPredicateThroughNonJoin, PruneFilters) :: Nil } @@ -44,8 +44,8 @@ class SetOperationSuite extends PlanTest { val testUnion = Union(testRelation :: testRelation2 :: testRelation3 :: Nil) test("union: combine unions into one unions") { - val unionQuery1 = Union(Union(testRelation, testRelation2), testRelation) - val unionQuery2 = Union(testRelation, Union(testRelation2, testRelation)) + val unionQuery1 = Union(Union(testRelation, testRelation2), testRelation3) + val unionQuery2 = Union(testRelation, Union(testRelation2, testRelation3)) val unionOptimized1 = Optimize.execute(unionQuery1.analyze) val unionOptimized2 = Optimize.execute(unionQuery2.analyze) @@ -93,7 +93,7 @@ class SetOperationSuite extends PlanTest { val unionQuery1 = Distinct(Union(Distinct(Union(query1, query2)), query3)).analyze val optimized1 = Optimize.execute(unionQuery1) val distinctUnionCorrectAnswer1 = - Distinct(Union(query1 :: query2 :: query3 :: Nil)).analyze + Distinct(Union(query1 :: query2 :: query3 :: Nil)) comparePlans(distinctUnionCorrectAnswer1, optimized1) // query1 @@ -107,7 +107,7 @@ class SetOperationSuite extends PlanTest { Distinct(Union(query2, query3)))).analyze val optimized2 = Optimize.execute(unionQuery2) val distinctUnionCorrectAnswer2 = - Distinct(Union(query1 :: query2 :: query2 :: query3 :: Nil)).analyze + Distinct(Union(query1 :: query2 :: query2 :: query3 :: Nil)) comparePlans(distinctUnionCorrectAnswer2, optimized2) } @@ -196,4 +196,31 @@ class SetOperationSuite extends PlanTest { )) comparePlans(expectedPlan, rewrittenPlan) } + + test("SPARK-23356 union: expressions with literal in project list are pushed down") { + val unionQuery = testUnion.select(('a + 1).as("aa")) + val unionOptimized = Optimize.execute(unionQuery.analyze) + val unionCorrectAnswer = + Union(testRelation.select(('a + 1).as("aa")) :: + testRelation2.select(('d + 1).as("aa")) :: + testRelation3.select(('g + 1).as("aa")) :: Nil).analyze + comparePlans(unionOptimized, unionCorrectAnswer) + } + + test("SPARK-23356 union: expressions in project list are pushed down") { + val unionQuery = testUnion.select(('a + 'b).as("ab")) + val unionOptimized = Optimize.execute(unionQuery.analyze) + val unionCorrectAnswer = + Union(testRelation.select(('a + 'b).as("ab")) :: + testRelation2.select(('d + 'e).as("ab")) :: + testRelation3.select(('g + 'h).as("ab")) :: Nil).analyze + comparePlans(unionOptimized, unionCorrectAnswer) + } + + test("SPARK-23356 union: no pushdown for non-deterministic expression") { + val unionQuery = testUnion.select('a, Rand(10).as("rnd")) + val unionOptimized = Optimize.execute(unionQuery.analyze) + val unionCorrectAnswer = unionQuery.analyze + comparePlans(unionOptimized, unionCorrectAnswer) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala index d4d23ad69b2c2..f8c48d56f1e57 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala @@ -33,7 +33,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas val batches = Batch("Operator Optimizations", FixedPoint(100), CombineFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, ReorderJoin, PushPredicateThroughJoin, ColumnPruning, @@ -218,6 +218,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) .join(t2, Inner, Some(nameToAttr("f1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_c1") === nameToAttr("t1_c1"))) + .select(outputsOf(f1, t1, t2, d1, d2): _*) assertEqualPlans(query, expected) } @@ -256,6 +257,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas .join(t3.join(t2, Inner, Some(nameToAttr("t2_c2") === nameToAttr("t3_c1"))), Inner, Some(nameToAttr("d1_c2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("t1_c1") === nameToAttr("f1_c1"))) + .select(outputsOf(d1, t1, t2, f1, d2, t3): _*) assertEqualPlans(query, expected) } @@ -297,6 +299,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("t3_c1") === nameToAttr("t4_c1"))) .join(t1.join(t2, Inner, Some(nameToAttr("t1_c1") === nameToAttr("t2_c1"))), Inner, Some(nameToAttr("t1_c2") === nameToAttr("t4_c2"))) + .select(outputsOf(d1, t1, t2, t3, t4, f1, d2): _*) assertEqualPlans(query, expected) } @@ -347,6 +350,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas Some(nameToAttr("d3_c2") === nameToAttr("t1_c1"))) .join(t5.join(t6, Inner, Some(nameToAttr("t5_c2") === nameToAttr("t6_c2"))), Inner, Some(nameToAttr("d2_c2") === nameToAttr("t5_c1"))) + .select(outputsOf(d1, t3, t4, f1, d2, t5, t6, d3, t1, t2): _*) assertEqualPlans(query, expected) } @@ -375,6 +379,7 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk"))) .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk"))) + .select(outputsOf(d1, d2, f1, d3): _*) assertEqualPlans(query, expected) } @@ -400,13 +405,27 @@ class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBas f1.join(t3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("t3_c1"))) .join(t2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("t2_c1"))) .join(t1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("t1_c1"))) + .select(outputsOf(t1, f1, t2, t3): _*) assertEqualPlans(query, expected) } private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala index 4e0883e91e84a..10e970d534c49 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala @@ -52,7 +52,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { val batches = Batch("Operator Optimizations", FixedPoint(100), CombineFilters, - PushDownPredicate, + PushPredicateThroughNonJoin, ReorderJoin, PushPredicateThroughJoin, ColumnPruning, @@ -182,6 +182,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d1, Inner, Some(nameToAttr("f1_fk1") === nameToAttr("d1_pk1"))) .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d2, f1, d3, s3): _*) assertEqualPlans(query, expected) } @@ -220,6 +221,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -255,7 +257,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("s3_c2"))) - + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -292,6 +294,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c2"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f1, d2, s3, d3): _*) assertEqualPlans(query, expected) } @@ -395,6 +398,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("f11_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, f11, f1, d2, s3): _*) assertEqualPlans(query, equivQuery) } @@ -430,6 +434,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_c4"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -465,6 +470,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -499,6 +505,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d2.where(nameToAttr("d2_c2") === 2), Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -532,6 +539,7 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") < nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") < nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") < nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } @@ -565,13 +573,27 @@ class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase { .join(d3, Inner, Some(nameToAttr("f1_fk3") === nameToAttr("d3_pk1"))) .join(d2, Inner, Some(nameToAttr("f1_fk2") === nameToAttr("d2_pk1"))) .join(s3, Inner, Some(nameToAttr("d3_fk1") === nameToAttr("s3_pk1"))) + .select(outputsOf(d1, d3, f1, d2, s3): _*) assertEqualPlans(query, expected) } - private def assertEqualPlans( plan1: LogicalPlan, plan2: LogicalPlan): Unit = { - val optimized = Optimize.execute(plan1.analyze) + private def assertEqualPlans(plan1: LogicalPlan, plan2: LogicalPlan): Unit = { + val analyzed = plan1.analyze + val optimized = Optimize.execute(analyzed) val expected = plan2.analyze + + assert(equivalentOutput(analyzed, expected)) // if this fails, the expected itself is incorrect + assert(equivalentOutput(analyzed, optimized)) + compareJoinOrder(optimized, expected) } + + private def outputsOf(plans: LogicalPlan*): Seq[Attribute] = { + plans.map(_.output).reduce(_ ++ _) + } + + private def equivalentOutput(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { + normalizeExprIds(plan1).output == normalizeExprIds(plan2).output + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala index 58b3d1c98f3cd..4acd57832d2f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/TransposeWindowSuite.scala @@ -27,7 +27,7 @@ import org.apache.spark.sql.catalyst.rules.RuleExecutor class TransposeWindowSuite extends PlanTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("CollapseProject", FixedPoint(100), CollapseProject, RemoveRedundantProject) :: + Batch("CollapseProject", FixedPoint(100), CollapseProject, RemoveNoopOperators) :: Batch("FlipWindow", Once, CollapseWindow, TransposeWindow) :: Nil } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UpdateNullabilityInAttributeReferencesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UpdateNullabilityInAttributeReferencesSuite.scala deleted file mode 100644 index 09b11f5aba2a0..0000000000000 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/UpdateNullabilityInAttributeReferencesSuite.scala +++ /dev/null @@ -1,57 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.dsl.expressions._ -import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.expressions.{CreateArray, GetArrayItem} -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} -import org.apache.spark.sql.catalyst.rules.RuleExecutor - - -class UpdateNullabilityInAttributeReferencesSuite extends PlanTest { - - object Optimizer extends RuleExecutor[LogicalPlan] { - val batches = - Batch("Constant Folding", FixedPoint(10), - NullPropagation, - ConstantFolding, - BooleanSimplification, - SimplifyConditionals, - SimplifyBinaryComparison, - SimplifyExtractValueOps) :: - Batch("UpdateAttributeReferences", Once, - UpdateNullabilityInAttributeReferences) :: Nil - } - - test("update nullability in AttributeReference") { - val rel = LocalRelation('a.long.notNull) - // In the 'original' plans below, the Aggregate node produced by groupBy() has a - // nullable AttributeReference to `b`, because both array indexing and map lookup are - // nullable expressions. After optimization, the same attribute is now non-nullable, - // but the AttributeReference is not updated to reflect this. So, we need to update nullability - // by the `UpdateNullabilityInAttributeReferences` rule. - val original = rel - .select(GetArrayItem(CreateArray(Seq('a, 'a + 1L)), 0) as "b") - .groupBy($"b")("1") - val expected = rel.select('a as "b").groupBy($"b")("1").analyze - val optimized = Optimizer.execute(original.analyze) - comparePlans(optimized, expected) - } -} 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 new file mode 100644 index 0000000000000..d008b3c78fac3 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/DDLParserSuite.scala @@ -0,0 +1,596 @@ +/* + * 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.parser + +import java.util.Locale + +import org.apache.spark.sql.catalog.v2.expressions.{ApplyTransform, BucketTransform, DaysTransform, FieldReference, HoursTransform, IdentityTransform, LiteralValue, MonthsTransform, YearsTransform} +import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.catalog.BucketSpec +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableAlterColumnStatement, AlterTableDropColumnsStatement, AlterTableRenameColumnStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.types.{IntegerType, LongType, StringType, StructType, TimestampType} +import org.apache.spark.unsafe.types.UTF8String + +class DDLParserSuite extends AnalysisTest { + import CatalystSqlParser._ + + private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { + val e = intercept[ParseException] { + parsePlan(sql) + } + assert(e.getMessage.toLowerCase(Locale.ROOT).contains("operation not allowed")) + containsThesePhrases.foreach { p => + assert(e.getMessage.toLowerCase(Locale.ROOT).contains(p.toLowerCase(Locale.ROOT))) + } + } + + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parsePlan)(sqlCommand, messages: _*) + + private def parseCompare(sql: String, expected: LogicalPlan): Unit = { + comparePlans(parsePlan(sql), expected, checkAnalysis = false) + } + + test("create table using - schema") { + val sql = "CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + + intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", + "no viable alternative at input") + } + + test("create table - with IF NOT EXISTS") { + val sql = "CREATE TABLE IF NOT EXISTS my_tab(a INT, b STRING) USING parquet" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with partitioned by") { + val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" + + parsePlan(query) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType)) + assert(create.partitioning == Seq(IdentityTransform(FieldReference("a")))) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - partitioned by transforms") { + val sql = + """ + |CREATE TABLE my_tab (a INT, b STRING, ts TIMESTAMP) USING parquet + |PARTITIONED BY ( + | a, + | bucket(16, b), + | years(ts), + | months(ts), + | days(ts), + | hours(ts), + | foo(a, "bar", 34)) + """.stripMargin + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType() + .add("a", IntegerType) + .add("b", StringType) + .add("ts", TimestampType)) + assert(create.partitioning == Seq( + IdentityTransform(FieldReference("a")), + BucketTransform(LiteralValue(16, IntegerType), Seq(FieldReference("b"))), + YearsTransform(FieldReference("ts")), + MonthsTransform(FieldReference("ts")), + DaysTransform(FieldReference("ts")), + HoursTransform(FieldReference("ts")), + ApplyTransform("foo", Seq( + FieldReference("a"), + LiteralValue(UTF8String.fromString("bar"), StringType), + LiteralValue(34, IntegerType))))) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with bucket") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + + parsePlan(query) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.contains(BucketSpec(5, Seq("a"), Seq("b")))) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - with comment") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.contains("abc")) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with table properties") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties == Map("test" -> "test")) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with location") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("my_tab")) + assert(create.tableSchema == new StructType().add("a", IntegerType).add("b", StringType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.contains("/tmp/file")) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - byte length literal table name") { + val sql = "CREATE TABLE 1m.2g(a INT) USING parquet" + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("1m", "2g")) + assert(create.tableSchema == new StructType().add("a", IntegerType)) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Duplicate clauses - create table") { + def createTableHeader(duplicateClause: String): String = { + s"CREATE TABLE my_tab(a INT, b STRING) USING parquet $duplicateClause $duplicateClause" + } + + intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')"), + "Found duplicate clauses: TBLPROPERTIES") + intercept(createTableHeader("LOCATION '/tmp/file'"), + "Found duplicate clauses: LOCATION") + intercept(createTableHeader("COMMENT 'a table'"), + "Found duplicate clauses: COMMENT") + intercept(createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), + "Found duplicate clauses: CLUSTERED BY") + intercept(createTableHeader("PARTITIONED BY (b)"), + "Found duplicate clauses: PARTITIONED BY") + } + + test("support for other types in OPTIONS") { + val sql = + """ + |CREATE TABLE table_name USING json + |OPTIONS (a 1, b 0.1, c TRUE) + """.stripMargin + + parsePlan(sql) match { + case create: CreateTableStatement => + assert(create.tableName == Seq("table_name")) + assert(create.tableSchema == new StructType) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties.isEmpty) + assert(create.provider == "json") + assert(create.options == Map("a" -> "1", "b" -> "0.1", "c" -> "true")) + assert(create.location.isEmpty) + assert(create.comment.isEmpty) + assert(!create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableStatement].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test CTAS against native tables") { + val s1 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + val s2 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |LOCATION '/user/external/page_view' + |COMMENT 'This is the staging page view table' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + val s3 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + checkParsing(s1) + checkParsing(s2) + checkParsing(s3) + + def checkParsing(sql: String): Unit = { + parsePlan(sql) match { + case create: CreateTableAsSelectStatement => + assert(create.tableName == Seq("mydb", "page_view")) + assert(create.partitioning.isEmpty) + assert(create.bucketSpec.isEmpty) + assert(create.properties == Map("p1" -> "v1", "p2" -> "v2")) + assert(create.provider == "parquet") + assert(create.options.isEmpty) + assert(create.location.contains("/user/external/page_view")) + assert(create.comment.contains("This is the staging page view table")) + assert(create.ifNotExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableAsSelectStatement].getClass.getName} " + + s"from query, got ${other.getClass.getName}: $sql") + } + } + } + + test("drop table") { + parseCompare("DROP TABLE testcat.ns1.ns2.tbl", + DropTableStatement(Seq("testcat", "ns1", "ns2", "tbl"), ifExists = false, purge = false)) + parseCompare(s"DROP TABLE db.tab", + DropTableStatement(Seq("db", "tab"), ifExists = false, purge = false)) + parseCompare(s"DROP TABLE IF EXISTS db.tab", + DropTableStatement(Seq("db", "tab"), ifExists = true, purge = false)) + parseCompare(s"DROP TABLE tab", + DropTableStatement(Seq("tab"), ifExists = false, purge = false)) + parseCompare(s"DROP TABLE IF EXISTS tab", + DropTableStatement(Seq("tab"), ifExists = true, purge = false)) + parseCompare(s"DROP TABLE tab PURGE", + DropTableStatement(Seq("tab"), ifExists = false, purge = true)) + parseCompare(s"DROP TABLE IF EXISTS tab PURGE", + DropTableStatement(Seq("tab"), ifExists = true, purge = true)) + } + + test("drop view") { + parseCompare(s"DROP VIEW testcat.db.view", + DropViewStatement(Seq("testcat", "db", "view"), ifExists = false)) + parseCompare(s"DROP VIEW db.view", DropViewStatement(Seq("db", "view"), ifExists = false)) + parseCompare(s"DROP VIEW IF EXISTS db.view", + DropViewStatement(Seq("db", "view"), ifExists = true)) + parseCompare(s"DROP VIEW view", DropViewStatement(Seq("view"), ifExists = false)) + parseCompare(s"DROP VIEW IF EXISTS view", DropViewStatement(Seq("view"), ifExists = true)) + } + + // ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter view: alter view properties") { + val sql1_view = "ALTER VIEW table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2_view = "ALTER VIEW table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3_view = "ALTER VIEW table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + comparePlans(parsePlan(sql1_view), + AlterViewSetPropertiesStatement( + Seq("table_name"), Map("test" -> "test", "comment" -> "new_comment"))) + comparePlans(parsePlan(sql2_view), + AlterViewUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = false)) + comparePlans(parsePlan(sql3_view), + AlterViewUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = true)) + } + + // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter table: alter table properties") { + val sql1_table = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + comparePlans( + parsePlan(sql1_table), + AlterTableSetPropertiesStatement( + Seq("table_name"), Map("test" -> "test", "comment" -> "new_comment"))) + comparePlans( + parsePlan(sql2_table), + AlterTableUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = false)) + comparePlans( + parsePlan(sql3_table), + AlterTableUnsetPropertiesStatement( + Seq("table_name"), Seq("comment", "test"), ifExists = true)) + } + + test("alter table: add column") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x int"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None) + ))) + } + + test("alter table: add multiple columns") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMNS x int, y string"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None), + QualifiedColType(Seq("y"), StringType, None) + ))) + } + + test("alter table: add column with COLUMNS") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMNS x int"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None) + ))) + } + + test("alter table: add column with COLUMNS (...)") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMNS (x int)"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, None) + ))) + } + + test("alter table: add column with COLUMNS (...) and COMMENT") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMNS (x int COMMENT 'doc')"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, Some("doc")) + ))) + } + + test("alter table: add column with COMMENT") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x int COMMENT 'doc'"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x"), IntegerType, Some("doc")) + ))) + } + + test("alter table: add column with nested column name") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc'"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc")) + ))) + } + + test("alter table: add multiple columns with nested column name") { + comparePlans( + parsePlan("ALTER TABLE table_name ADD COLUMN x.y.z int COMMENT 'doc', a.b string"), + AlterTableAddColumnsStatement(Seq("table_name"), Seq( + QualifiedColType(Seq("x", "y", "z"), IntegerType, Some("doc")), + QualifiedColType(Seq("a", "b"), StringType, None) + ))) + } + + test("alter table: add column at position (not supported)") { + assertUnsupported("ALTER TABLE table_name ADD COLUMNS name bigint COMMENT 'doc' FIRST, a.b int") + assertUnsupported("ALTER TABLE table_name ADD COLUMN name bigint COMMENT 'doc' FIRST") + assertUnsupported("ALTER TABLE table_name ADD COLUMN name string AFTER a.b") + } + + test("alter table: set location") { + val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + val parsed1 = parsePlan(sql1) + val expected1 = AlterTableSetLocationStatement(Seq("table_name"), "new location") + comparePlans(parsed1, expected1) + } + + test("alter table: rename column") { + comparePlans( + parsePlan("ALTER TABLE table_name RENAME COLUMN a.b.c TO d"), + AlterTableRenameColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + "d")) + } + + test("alter table: update column type using ALTER") { + comparePlans( + parsePlan("ALTER TABLE table_name ALTER COLUMN a.b.c TYPE bigint"), + AlterTableAlterColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + Some(LongType), + None)) + } + + test("alter table: update column type") { + comparePlans( + parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint"), + AlterTableAlterColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + Some(LongType), + None)) + } + + test("alter table: update column comment") { + comparePlans( + parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c COMMENT 'new comment'"), + AlterTableAlterColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + None, + Some("new comment"))) + } + + test("alter table: update column type and comment") { + comparePlans( + parsePlan("ALTER TABLE table_name CHANGE COLUMN a.b.c TYPE bigint COMMENT 'new comment'"), + AlterTableAlterColumnStatement( + Seq("table_name"), + Seq("a", "b", "c"), + Some(LongType), + Some("new comment"))) + } + + test("alter table: change column position (not supported)") { + assertUnsupported("ALTER TABLE table_name CHANGE COLUMN name COMMENT 'doc' FIRST") + assertUnsupported("ALTER TABLE table_name CHANGE COLUMN name TYPE INT AFTER other_col") + } + + test("alter table: drop column") { + comparePlans( + parsePlan("ALTER TABLE table_name DROP COLUMN a.b.c"), + AlterTableDropColumnsStatement(Seq("table_name"), Seq(Seq("a", "b", "c")))) + } + + test("alter table: drop multiple columns") { + val sql = "ALTER TABLE table_name DROP COLUMN x, y, a.b.c" + Seq(sql, sql.replace("COLUMN", "COLUMNS")).foreach { drop => + comparePlans( + parsePlan(drop), + AlterTableDropColumnsStatement( + Seq("table_name"), + Seq(Seq("x"), Seq("y"), Seq("a", "b", "c")))) + } + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala index baaf01800b33b..478953fb1b571 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ErrorParserSuite.scala @@ -16,13 +16,26 @@ */ package org.apache.spark.sql.catalyst.parser -import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** * Test various parser errors. */ -class ErrorParserSuite extends SparkFunSuite { - def intercept(sql: String, line: Int, startPosition: Int, messages: String*): Unit = { +class ErrorParserSuite extends AnalysisTest { + import CatalystSqlParser._ + import org.apache.spark.sql.catalyst.dsl.expressions._ + import org.apache.spark.sql.catalyst.dsl.plans._ + + private def assertEqual(sqlCommand: String, plan: LogicalPlan): Unit = { + assert(parsePlan(sqlCommand) == plan) + } + + def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(CatalystSqlParser.parsePlan)(sqlCommand, messages: _*) + + def intercept(sql: String, line: Int, startPosition: Int, stopPosition: Int, + messages: String*): Unit = { val e = intercept[ParseException](CatalystSqlParser.parsePlan(sql)) // Check position. @@ -30,6 +43,8 @@ class ErrorParserSuite extends SparkFunSuite { assert(e.line.get === line) assert(e.startPosition.isDefined) assert(e.startPosition.get === startPosition) + assert(e.stop.startPosition.isDefined) + assert(e.stop.startPosition.get === stopPosition) // Check messages. val error = e.getMessage @@ -39,24 +54,132 @@ class ErrorParserSuite extends SparkFunSuite { } test("no viable input") { - intercept("select ((r + 1) ", 1, 16, "no viable alternative at input", "----------------^^^") + intercept("select ((r + 1) ", 1, 16, 16, + "no viable alternative at input", "----------------^^^") } test("extraneous input") { - intercept("select 1 1", 1, 9, "extraneous input '1' expecting", "---------^^^") - intercept("select *\nfrom r as q t", 2, 12, "extraneous input", "------------^^^") + intercept("select 1 1", 1, 9, 10, "extraneous input '1' expecting", "---------^^^") + intercept("select *\nfrom r as q t", 2, 12, 13, "extraneous input", "------------^^^") } test("mismatched input") { - intercept("select * from r order by q from t", 1, 27, + intercept("select * from r order by q from t", 1, 27, 31, "mismatched input", "---------------------------^^^") - intercept("select *\nfrom r\norder by q\nfrom t", 4, 0, "mismatched input", "^^^") + intercept("select *\nfrom r\norder by q\nfrom t", 4, 0, 4, "mismatched input", "^^^") } test("semantic errors") { - intercept("select *\nfrom r\norder by q\ncluster by q", 3, 0, + intercept("select *\nfrom r\norder by q\ncluster by q", 3, 0, 11, "Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY is not supported", "^^^") } + + test("SPARK-21136: misleading error message due to problematic antlr grammar") { + intercept("select * from a left joinn b on a.id = b.id", "missing 'JOIN' at 'joinn'") + intercept("select * from test where test.t is like 'test'", "mismatched input 'is' expecting") + intercept("SELECT * FROM test WHERE x NOT NULL", "mismatched input 'NOT' expecting") + } + + test("hyphen in identifier - DDL tests") { + val msg = "unquoted identifier" + intercept("USE test-test", 1, 8, 9, msg + " test-test") + intercept("CREATE DATABASE IF NOT EXISTS my-database", 1, 32, 33, msg + " my-database") + intercept( + """ + |ALTER DATABASE my-database + |SET DBPROPERTIES ('p1'='v1')""".stripMargin, 2, 17, 18, msg + " my-database") + intercept("DROP DATABASE my-database", 1, 16, 17, msg + " my-database") + intercept( + """ + |ALTER TABLE t + |CHANGE COLUMN + |test-col BIGINT + """.stripMargin, 4, 4, 5, msg + " test-col") + intercept("CREATE TABLE test (attri-bute INT)", 1, 24, 25, msg + " attri-bute") + intercept( + """ + |CREATE TABLE IF NOT EXISTS mydb.page-view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src""".stripMargin, 2, 36, 37, msg + " page-view") + intercept("SHOW TABLES IN hyphen-database", 1, 21, 22, msg + " hyphen-database") + intercept("SHOW TABLE EXTENDED IN hyphen-db LIKE \"str\"", 1, 29, 30, msg + " hyphen-db") + intercept("SHOW COLUMNS IN t FROM test-db", 1, 27, 28, msg + " test-db") + intercept("DESC SCHEMA EXTENDED test-db", 1, 25, 26, msg + " test-db") + intercept("ANALYZE TABLE test-table PARTITION (part1)", 1, 18, 19, msg + " test-table") + intercept("LOAD DATA INPATH \"path\" INTO TABLE my-tab", 1, 37, 38, msg + " my-tab") + } + + test("hyphen in identifier - DML tests") { + val msg = "unquoted identifier" + // dml tests + intercept("SELECT * FROM table-with-hyphen", 1, 19, 25, msg + " table-with-hyphen") + // special test case: minus in expression shouldn't be treated as hyphen in identifiers + intercept("SELECT a-b FROM table-with-hyphen", 1, 21, 27, msg + " table-with-hyphen") + intercept("SELECT a-b AS a-b FROM t", 1, 15, 16, msg + " a-b") + intercept("SELECT a-b FROM table-hyphen WHERE a-b = 0", 1, 21, 22, msg + " table-hyphen") + intercept("SELECT (a - test_func(b-c)) FROM test-table", 1, 37, 38, msg + " test-table") + intercept("WITH a-b AS (SELECT 1 FROM s) SELECT * FROM s;", 1, 6, 7, msg + " a-b") + intercept( + """ + |SELECT a, b + |FROM t1 JOIN t2 + |USING (a, b, at-tr) + """.stripMargin, 4, 15, 16, msg + " at-tr" + ) + intercept( + """ + |SELECT product, category, dense_rank() + |OVER (PARTITION BY category ORDER BY revenue DESC) as hyphen-rank + |FROM productRevenue + """.stripMargin, 3, 60, 61, msg + " hyphen-rank" + ) + intercept( + """ + |SELECT a, b + |FROM grammar-breaker + |WHERE a-b > 10 + |GROUP BY fake-breaker + |ORDER BY c + """.stripMargin, 3, 12, 13, msg + " grammar-breaker") + assertEqual( + """ + |SELECT a, b + |FROM t + |WHERE a-b > 10 + |GROUP BY fake-breaker + |ORDER BY c + """.stripMargin, + table("t") + .where('a - 'b > 10) + .groupBy('fake - 'breaker)('a, 'b) + .orderBy('c.asc)) + intercept( + """ + |SELECT * FROM tab + |WINDOW hyphen-window AS + | (PARTITION BY a, b ORDER BY c rows BETWEEN 1 PRECEDING AND 1 FOLLOWING) + """.stripMargin, 3, 13, 14, msg + " hyphen-window") + intercept( + """ + |SELECT * FROM tab + |WINDOW window_ref AS window-ref + """.stripMargin, 3, 27, 28, msg + " window-ref") + intercept( + """ + |SELECT tb.* + |FROM t-a INNER JOIN tb + |ON ta.a = tb.a AND ta.tag = tb.tag + """.stripMargin, 3, 6, 7, msg + " t-a") + intercept( + """ + |FROM test-table + |SELECT a + |SELECT b + """.stripMargin, 2, 9, 10, msg + " test-table") + } } 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 b4df22c5b29fa..6248e5724f063 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 @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst.parser import java.sql.{Date, Timestamp} +import java.time.LocalDateTime +import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.{UnresolvedAttribute, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.{First, Last} -import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -35,8 +37,7 @@ import org.apache.spark.unsafe.types.CalendarInterval * structure needs to be valid. Unsound expressions should be caught by the Analyzer or * CheckAnalysis classes. */ -class ExpressionParserSuite extends PlanTest { - import CatalystSqlParser._ +class ExpressionParserSuite extends AnalysisTest { import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ @@ -49,11 +50,14 @@ class ExpressionParserSuite extends PlanTest { compareExpressions(parser.parseExpression(sqlCommand), e) } - def intercept(sqlCommand: String, messages: String*): Unit = { - val e = intercept[ParseException](defaultParser.parseExpression(sqlCommand)) - messages.foreach { message => - assert(e.message.contains(message)) - } + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(defaultParser.parseExpression)(sqlCommand, messages: _*) + + def assertEval( + sqlCommand: String, + expect: Any, + parser: ParserInterface = defaultParser): Unit = { + assert(parser.parseExpression(sqlCommand).eval() === expect) } test("star expressions") { @@ -246,9 +250,11 @@ class ExpressionParserSuite extends PlanTest { intercept("foo(a x)", "extraneous input 'x'") } + private def lv(s: Symbol) = UnresolvedNamedLambdaVariable(Seq(s.name)) + test("lambda functions") { - assertEqual("x -> x + 1", LambdaFunction('x + 1, Seq('x.attr))) - assertEqual("(x, y) -> x + y", LambdaFunction('x + 'y, Seq('x.attr, 'y.attr))) + assertEqual("x -> x + 1", LambdaFunction(lv('x) + 1, Seq(lv('x)))) + assertEqual("(x, y) -> x + y", LambdaFunction(lv('x) + lv('y), Seq(lv('x), lv('y)))) } test("window function expressions") { @@ -573,49 +579,60 @@ class ExpressionParserSuite extends PlanTest { } } + val intervalUnits = Seq( + "year", + "month", + "week", + "day", + "hour", + "minute", + "second", + "millisecond", + "microsecond") + + def intervalLiteral(u: String, s: String): Literal = { + Literal(CalendarInterval.fromSingleUnitString(u, s)) + } + test("intervals") { - def intervalLiteral(u: String, s: String): Literal = { - Literal(CalendarInterval.fromSingleUnitString(u, s)) + def checkIntervals(intervalValue: String, expected: Literal): Unit = { + assertEqual(s"interval $intervalValue", expected) + + // SPARK-23264 Support interval values without INTERVAL clauses if ANSI SQL enabled + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + assertEqual(intervalValue, expected) + } } // Empty interval statement intercept("interval", "at least one time unit should be given for interval literal") // Single Intervals. - val units = Seq( - "year", - "month", - "week", - "day", - "hour", - "minute", - "second", - "millisecond", - "microsecond") val forms = Seq("", "s") val values = Seq("0", "10", "-7", "21") - units.foreach { unit => + intervalUnits.foreach { unit => forms.foreach { form => values.foreach { value => val expected = intervalLiteral(unit, value) - assertEqual(s"interval $value $unit$form", expected) - assertEqual(s"interval '$value' $unit$form", expected) + checkIntervals(s"$value $unit$form", expected) + checkIntervals(s"'$value' $unit$form", expected) } } } // Hive nanosecond notation. - assertEqual("interval 13.123456789 seconds", intervalLiteral("second", "13.123456789")) - assertEqual("interval -13.123456789 second", intervalLiteral("second", "-13.123456789")) + checkIntervals("13.123456789 seconds", intervalLiteral("second", "13.123456789")) + checkIntervals("-13.123456789 second", intervalLiteral("second", "-13.123456789")) // Non Existing unit - intercept("interval 10 nanoseconds", "No interval can be constructed") + intercept("interval 10 nanoseconds", + "no viable alternative at input 'interval 10 nanoseconds'") // Year-Month intervals. val yearMonthValues = Seq("123-10", "496-0", "-2-3", "-123-0") yearMonthValues.foreach { value => val result = Literal(CalendarInterval.fromYearMonthString(value)) - assertEqual(s"interval '$value' year to month", result) + checkIntervals(s"'$value' year to month", result) } // Day-Time intervals. @@ -628,22 +645,63 @@ class ExpressionParserSuite extends PlanTest { "1 0:0:1") datTimeValues.foreach { value => val result = Literal(CalendarInterval.fromDayTimeString(value)) - assertEqual(s"interval '$value' day to second", result) + checkIntervals(s"'$value' day to second", result) + } + + // Hour-Time intervals. + val hourTimeValues = Seq( + "11:22:33.123456789", + "9:8:7.123456789", + "-19:18:17.123456789", + "0:0:0", + "0:0:1") + hourTimeValues.foreach { value => + val result = Literal(CalendarInterval.fromDayTimeString(value)) + checkIntervals(s"'$value' hour to second", result) } // Unknown FROM TO intervals - intercept("interval 10 month to second", "Intervals FROM month TO second are not supported.") + intercept("interval 10 month to second", + "Intervals FROM month TO second are not supported.") // Composed intervals. - assertEqual( - "interval 3 months 22 seconds 1 millisecond", + checkIntervals( + "3 months 22 seconds 1 millisecond", Literal(new CalendarInterval(3, 22001000L))) - assertEqual( - "interval 3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second", + checkIntervals( + "3 years '-1-10' year to month 3 weeks '1 0:0:2' day to second", Literal(new CalendarInterval(14, 22 * CalendarInterval.MICROS_PER_DAY + 2 * CalendarInterval.MICROS_PER_SECOND))) } + test("SPARK-23264 Interval Compatibility tests") { + def checkIntervals(intervalValue: String, expected: Literal): Unit = { + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + assertEqual(intervalValue, expected) + } + + // Compatibility tests: If ANSI SQL disabled, `intervalValue` should be parsed as an alias + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + val aliases = defaultParser.parseExpression(intervalValue).collect { + case a @ Alias(_: Literal, name) + if intervalUnits.exists { unit => name.startsWith(unit) } => a + } + assert(aliases.size === 1) + } + } + val forms = Seq("", "s") + val values = Seq("5", "1", "-11", "8") + intervalUnits.foreach { unit => + forms.foreach { form => + values.foreach { value => + val expected = intervalLiteral(unit, value) + checkIntervals(s"$value $unit$form", expected) + checkIntervals(s"'$value' $unit$form", expected) + } + } + } + } + test("composed expressions") { assertEqual("1 + r.r As q", (Literal(1) + UnresolvedAttribute("r.r")).as("q")) assertEqual("1 - f('o', o(bar))", Literal(1) - 'f.function("o", 'o.function('bar))) @@ -678,4 +736,57 @@ class ExpressionParserSuite extends PlanTest { assertEqual("last(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) assertEqual("last(a)", Last('a, Literal(false)).toAggregateExpression()) } + + test("Support respect nulls keywords for first_value and last_value") { + assertEqual("first_value(a ignore nulls)", First('a, Literal(true)).toAggregateExpression()) + assertEqual("first_value(a respect nulls)", First('a, Literal(false)).toAggregateExpression()) + assertEqual("first_value(a)", First('a, Literal(false)).toAggregateExpression()) + assertEqual("last_value(a ignore nulls)", Last('a, Literal(true)).toAggregateExpression()) + assertEqual("last_value(a respect nulls)", Last('a, Literal(false)).toAggregateExpression()) + assertEqual("last_value(a)", Last('a, Literal(false)).toAggregateExpression()) + } + + test("timestamp literals") { + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone.getID) { + def toMicros(time: LocalDateTime): Long = { + val seconds = time.atZone(timeZone.toZoneId).toInstant.getEpochSecond + TimeUnit.SECONDS.toMicros(seconds) + } + assertEval( + sqlCommand = "TIMESTAMP '2019-01-14 20:54:00.000'", + expect = toMicros(LocalDateTime.of(2019, 1, 14, 20, 54))) + assertEval( + sqlCommand = "Timestamp '2000-01-01T00:55:00'", + expect = toMicros(LocalDateTime.of(2000, 1, 1, 0, 55))) + // Parsing of the string does not depend on the SQL config because the string contains + // time zone offset already. + assertEval( + sqlCommand = "TIMESTAMP '2019-01-16 20:50:00.567000+01:00'", + expect = 1547668200567000L) + } + } + } + + test("date literals") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + assertEval("DATE '2019-01-14'", 17910) + assertEval("DATE '2019-01'", 17897) + assertEval("DATE '2019'", 17897) + } + } + } + + test("current date/timestamp braceless expressions") { + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + assertEqual("current_date", CurrentDate()) + assertEqual("current_timestamp", CurrentTimestamp()) + } + + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "false") { + assertEqual("current_date", UnresolvedAttribute.quoted("current_date")) + assertEqual("current_timestamp", UnresolvedAttribute.quoted("current_timestamp")) + } + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala index 768030f0a9bc4..cba24fbe2a65b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/parser/ParserUtilsSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst.parser import org.antlr.v4.runtime.{CharStreams, CommonTokenStream, ParserRuleContext} +import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.parser.SqlBaseParser._ @@ -152,14 +153,14 @@ class ParserUtilsSuite extends SparkFunSuite { assert(string(showDbsContext.pattern) == "identifier_with_wildcards") assert(string(createDbContext.comment) == "database_comment") - assert(string(createDbContext.locationSpec.STRING) == "/home/user/db") + assert(string(createDbContext.locationSpec.asScala.head.STRING) == "/home/user/db") } test("position") { assert(position(setConfContext.start) == Origin(Some(1), Some(0))) assert(position(showFuncContext.stop) == Origin(Some(1), Some(19))) assert(position(descFuncContext.describeFuncName.start) == Origin(Some(1), Some(27))) - assert(position(createDbContext.locationSpec.start) == Origin(Some(3), Some(27))) + assert(position(createDbContext.locationSpec.asScala.head.start) == Origin(Some(3), Some(27))) assert(position(emptyContext.stop) == Origin(None, None)) } @@ -177,7 +178,7 @@ class ParserUtilsSuite extends SparkFunSuite { } test("withOrigin") { - val ctx = createDbContext.locationSpec + val ctx = createDbContext.locationSpec.asScala.head val current = CurrentOrigin.get val (location, origin) = withOrigin(ctx) { (string(ctx.STRING), CurrentOrigin.get) 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 f5da90f7cf0c6..fb245eef5e4be 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute, UnresolvedFunction, UnresolvedGenerator, UnresolvedInlineTable, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction} +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction, UnresolvedGenerator, UnresolvedInlineTable, UnresolvedRelation, UnresolvedSubqueryColumnAliases, UnresolvedTableValuedFunction} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -39,11 +39,20 @@ class PlanParserSuite extends AnalysisTest { comparePlans(parsePlan(sqlCommand), plan, checkAnalysis = false) } - private def intercept(sqlCommand: String, messages: String*): Unit = { - val e = intercept[ParseException](parsePlan(sqlCommand)) - messages.foreach { message => - assert(e.message.contains(message)) + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parsePlan)(sqlCommand, messages: _*) + + private def cte(plan: LogicalPlan, namedPlans: (String, (LogicalPlan, Seq[String]))*): With = { + val ctes = namedPlans.map { + case (name, (cte, columnAliases)) => + val subquery = if (columnAliases.isEmpty) { + cte + } else { + UnresolvedSubqueryColumnAliases(columnAliases, cte) + } + name -> SubqueryAlias(name, subquery) } + With(plan, ctes) } test("case insensitive") { @@ -78,27 +87,20 @@ class PlanParserSuite extends AnalysisTest { } test("common table expressions") { - def cte(plan: LogicalPlan, namedPlans: (String, LogicalPlan)*): With = { - val ctes = namedPlans.map { - case (name, cte) => - name -> SubqueryAlias(name, cte) - } - With(plan, ctes) - } assertEqual( "with cte1 as (select * from a) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> table("a").select(star()))) + cte(table("cte1").select(star()), "cte1" -> ((table("a").select(star()), Seq.empty)))) assertEqual( "with cte1 (select 1) select * from cte1", - cte(table("cte1").select(star()), "cte1" -> OneRowRelation().select(1))) + cte(table("cte1").select(star()), "cte1" -> ((OneRowRelation().select(1), Seq.empty)))) assertEqual( "with cte1 (select 1), cte2 as (select * from cte1) select * from cte2", cte(table("cte2").select(star()), - "cte1" -> OneRowRelation().select(1), - "cte2" -> table("cte1").select(star()))) + "cte1" -> ((OneRowRelation().select(1), Seq.empty)), + "cte2" -> ((table("cte1").select(star()), Seq.empty)))) intercept( "with cte1 (select 1), cte1 as (select 1 from cte1) select * from cte1", - "Found duplicate keys 'cte1'") + "CTE definition can't have duplicate names: 'cte1'.") } test("simple select query") { @@ -115,15 +117,15 @@ class PlanParserSuite extends AnalysisTest { assertEqual("select a from 1k.2m", table("1k", "2m").select('a)) } - test("reverse select query") { - assertEqual("from a", table("a")) + test("hive-style single-FROM statement") { assertEqual("from a select b, c", table("a").select('b, 'c)) assertEqual( "from db.a select b, c where d < 1", table("db", "a").where('d < 1).select('b, 'c)) assertEqual("from a select distinct b, c", Distinct(table("a").select('b, 'c))) - assertEqual( - "from (from a union all from b) c select *", - table("a").union(table("b")).as("c").select(star())) + + // Weird "FROM table" queries, should be invalid anyway + intercept("from a", "no viable alternative at input 'from a'") + intercept("from (from a union all from b) c select *", "no viable alternative at input 'from") } test("multi select query") { @@ -132,11 +134,19 @@ class PlanParserSuite extends AnalysisTest { table("a").select(star()).union(table("a").where('s < 10).select(star()))) intercept( "from a select * select * from x where a.s < 10", - "Multi-Insert queries cannot have a FROM clause in their individual SELECT statements") + "mismatched input 'from' expecting") + intercept( + "from a select * from b", + "mismatched input 'from' expecting") assertEqual( "from a insert into tbl1 select * insert into tbl2 select * where s < 10", table("a").select(star()).insertInto("tbl1").union( table("a").where('s < 10).select(star()).insertInto("tbl2"))) + assertEqual( + "select * from (from a select * select *)", + table("a").select(star()) + .union(table("a").select(star())) + .as("__auto_generated_subquery_name").select(star())) } test("query organization") { @@ -370,6 +380,7 @@ class PlanParserSuite extends AnalysisTest { test("full join", FullOuter, testAll) test("full outer join", FullOuter, testAll) test("left semi join", LeftSemi, testExistence) + test("semi join", LeftSemi, testExistence) test("left anti join", LeftAnti, testExistence) test("anti join", LeftAnti, testExistence) @@ -691,20 +702,74 @@ class PlanParserSuite extends AnalysisTest { } test("TRIM function") { - intercept("select ltrim(both 'S' from 'SS abc S'", "missing ')' at ''") - intercept("select rtrim(trailing 'S' from 'SS abc S'", "missing ')' at ''") + def assertTrimPlans(inputSQL: String, expectedExpression: Expression): Unit = { + comparePlans( + parsePlan(inputSQL), + Project(Seq(UnresolvedAlias(expectedExpression)), OneRowRelation()) + ) + } - assertEqual( + intercept("select ltrim(both 'S' from 'SS abc S'", "mismatched input 'from' expecting {')'") + intercept("select rtrim(trailing 'S' from 'SS abc S'", "mismatched input 'from' expecting {')'") + + assertTrimPlans( "SELECT TRIM(BOTH '@$%&( )abc' FROM '@ $ % & ()abc ' )", - OneRowRelation().select('TRIM.function("@$%&( )abc", "@ $ % & ()abc ")) + StringTrim(Literal("@ $ % & ()abc "), Some(Literal("@$%&( )abc"))) ) - assertEqual( + assertTrimPlans( "SELECT TRIM(LEADING 'c []' FROM '[ ccccbcc ')", - OneRowRelation().select('ltrim.function("c []", "[ ccccbcc ")) + StringTrimLeft(Literal("[ ccccbcc "), Some(Literal("c []"))) ) - assertEqual( + assertTrimPlans( "SELECT TRIM(TRAILING 'c&^,.' FROM 'bc...,,,&&&ccc')", - OneRowRelation().select('rtrim.function("c&^,.", "bc...,,,&&&ccc")) + StringTrimRight(Literal("bc...,,,&&&ccc"), Some(Literal("c&^,."))) + ) + + assertTrimPlans( + "SELECT TRIM(BOTH FROM ' bunch o blanks ')", + StringTrim(Literal(" bunch o blanks "), None) + ) + assertTrimPlans( + "SELECT TRIM(LEADING FROM ' bunch o blanks ')", + StringTrimLeft(Literal(" bunch o blanks "), None) + ) + assertTrimPlans( + "SELECT TRIM(TRAILING FROM ' bunch o blanks ')", + StringTrimRight(Literal(" bunch o blanks "), None) + ) + + assertTrimPlans( + "SELECT TRIM('xyz' FROM 'yxTomxx')", + StringTrim(Literal("yxTomxx"), Some(Literal("xyz"))) + ) + } + + test("OVERLAY function") { + def assertOverlayPlans(inputSQL: String, expectedExpression: Expression): Unit = { + comparePlans( + parsePlan(inputSQL), + Project(Seq(UnresolvedAlias(expectedExpression)), OneRowRelation()) + ) + } + + assertOverlayPlans( + "SELECT OVERLAY('Spark SQL' PLACING '_' FROM 6)", + new Overlay(Literal("Spark SQL"), Literal("_"), Literal(6)) + ) + + assertOverlayPlans( + "SELECT OVERLAY('Spark SQL' PLACING 'CORE' FROM 7)", + new Overlay(Literal("Spark SQL"), Literal("CORE"), Literal(7)) + ) + + assertOverlayPlans( + "SELECT OVERLAY('Spark SQL' PLACING 'ANSI ' FROM 7 FOR 0)", + Overlay(Literal("Spark SQL"), Literal("ANSI "), Literal(7), Literal(0)) + ) + + assertOverlayPlans( + "SELECT OVERLAY('Spark SQL' PLACING 'tructured' FROM 2 FOR 4)", + Overlay(Literal("Spark SQL"), Literal("tructured"), Literal(2), Literal(4)) ) } @@ -753,4 +818,70 @@ class PlanParserSuite extends AnalysisTest { assertEqual(query2, Distinct(a.union(b)).except(c.intersect(d, isAll = true), isAll = true)) } } + + test("create/alter view as insert into table") { + val m1 = intercept[ParseException] { + parsePlan("CREATE VIEW testView AS INSERT INTO jt VALUES(1, 1)") + }.getMessage + assert(m1.contains("mismatched input 'INSERT' expecting")) + // Multi insert query + val m2 = intercept[ParseException] { + parsePlan( + """ + |CREATE VIEW testView AS FROM jt + |INSERT INTO tbl1 SELECT * WHERE jt.id < 5 + |INSERT INTO tbl2 SELECT * WHERE jt.id > 4 + """.stripMargin) + }.getMessage + assert(m2.contains("mismatched input 'INSERT' expecting")) + val m3 = intercept[ParseException] { + parsePlan("ALTER VIEW testView AS INSERT INTO jt VALUES(1, 1)") + }.getMessage + assert(m3.contains("mismatched input 'INSERT' expecting")) + // Multi insert query + val m4 = intercept[ParseException] { + parsePlan( + """ + |ALTER VIEW testView AS FROM jt + |INSERT INTO tbl1 SELECT * WHERE jt.id < 5 + |INSERT INTO tbl2 SELECT * WHERE jt.id > 4 + """.stripMargin + ) + }.getMessage + assert(m4.contains("mismatched input 'INSERT' expecting")) + } + + test("Invalid insert constructs in the query") { + val m1 = intercept[ParseException] { + parsePlan("SELECT * FROM (INSERT INTO BAR VALUES (2))") + }.getMessage + assert(m1.contains("missing ')' at 'BAR'")) + val m2 = intercept[ParseException] { + parsePlan("SELECT * FROM S WHERE C1 IN (INSERT INTO T VALUES (2))") + }.getMessage + assert(m2.contains("mismatched input 'IN' expecting")) + } + + test("relation in v2 catalog") { + assertEqual("TABLE testcat.db.tab", table("testcat", "db", "tab")) + assertEqual("SELECT * FROM testcat.db.tab", table("testcat", "db", "tab").select(star())) + + assertEqual( + """ + |WITH cte1 AS (SELECT * FROM testcat.db.tab) + |SELECT * FROM cte1 + """.stripMargin, + cte(table("cte1").select(star()), + "cte1" -> ((table("testcat", "db", "tab").select(star()), Seq.empty)))) + + assertEqual( + "SELECT /*+ BROADCAST(tab) */ * FROM testcat.db.tab", + table("testcat", "db", "tab").select(star()).hint("BROADCAST", $"tab")) + } + + test("CTE with column alias") { + assertEqual( + "WITH t(x) AS (SELECT c FROM a) SELECT * FROM t", + cte(table("t").select(star()), "t" -> ((table("a").select('c), Seq("x"))))) + } } 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 ff0de0fb7c1f0..fc2ce12092190 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 @@ -18,44 +18,622 @@ package org.apache.spark.sql.catalyst.parser import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.internal.SQLConf -class TableIdentifierParserSuite extends SparkFunSuite { +class TableIdentifierParserSuite extends SparkFunSuite with SQLHelper { import CatalystSqlParser._ // Add "$elem$", "$value$" & "$key$" - val hiveNonReservedKeyword = Array("add", "admin", "after", "analyze", "archive", "asc", "before", - "bucket", "buckets", "cascade", "change", "cluster", "clustered", "clusterstatus", "collection", - "columns", "comment", "compact", "compactions", "compute", "concatenate", "continue", "cost", - "data", "day", "databases", "datetime", "dbproperties", "deferred", "defined", "delimited", - "dependency", "desc", "directories", "directory", "disable", "distribute", - "enable", "escaped", "exclusive", "explain", "export", "fields", "file", "fileformat", "first", - "format", "formatted", "functions", "hold_ddltime", "hour", "idxproperties", "ignore", "index", - "indexes", "inpath", "inputdriver", "inputformat", "items", "jar", "keys", "key_type", "last", - "limit", "offset", "lines", "load", "location", "lock", "locks", "logical", "long", "mapjoin", - "materialized", "metadata", "minus", "minute", "month", "msck", "noscan", "no_drop", "nulls", - "offline", "option", "outputdriver", "outputformat", "overwrite", "owner", "partitioned", - "partitions", "plus", "pretty", "principals", "protection", "purge", "read", "readonly", - "rebuild", "recordreader", "recordwriter", "reload", "rename", "repair", "replace", - "replication", "restrict", "rewrite", "role", "roles", "schemas", "second", - "serde", "serdeproperties", "server", "sets", "shared", "show", "show_database", "skewed", - "sort", "sorted", "ssl", "statistics", "stored", "streamtable", "string", "struct", "tables", - "tblproperties", "temporary", "terminated", "tinyint", "touch", "transactions", "unarchive", - "undo", "uniontype", "unlock", "unset", "unsigned", "uri", "use", "utc", "utctimestamp", - "view", "while", "year", "work", "transaction", "write", "isolation", "level", "snapshot", - "autocommit", "all", "any", "alter", "array", "as", "authorization", "between", "bigint", - "binary", "boolean", "both", "by", "create", "cube", "current_date", "current_timestamp", - "cursor", "date", "decimal", "delete", "describe", "double", "drop", "exists", "external", - "false", "fetch", "float", "for", "grant", "group", "grouping", "import", "in", - "insert", "int", "into", "is", "pivot", "lateral", "like", "local", "none", "null", - "of", "order", "out", "outer", "partition", "percent", "procedure", "range", "reads", "revoke", - "rollup", "row", "rows", "set", "smallint", "table", "timestamp", "to", "trigger", - "true", "truncate", "update", "user", "values", "with", "regexp", "rlike", - "bigint", "binary", "boolean", "current_date", "current_timestamp", "date", "double", "float", - "int", "smallint", "timestamp", "at", "position", "both", "leading", "trailing", "extract") + // It is recommended to list them in alphabetical order. + val hiveNonReservedKeyword = Array( + "add", + "admin", + "after", + "all", + "alter", + "analyze", + "any", + "archive", + "array", + "as", + "asc", + "at", + "authorization", + "autocommit", + "before", + "between", + "bigint", + "binary", + "boolean", + "both", + "bucket", + "buckets", + "by", + "cascade", + "change", + "cluster", + "clustered", + "clusterstatus", + "collection", + "columns", + "comment", + "compact", + "compactions", + "compute", + "concatenate", + "continue", + "cost", + "create", + "cube", + "current_date", + "current_timestamp", + "cursor", + "data", + "databases", + "date", + "datetime", + "day", + "days", + "dbproperties", + "decimal", + "deferred", + "defined", + "delete", + "delimited", + "dependency", + "desc", + "describe", + "directories", + "directory", + "disable", + "distribute", + "double", + "drop", + "enable", + "escaped", + "exclusive", + "exists", + "explain", + "export", + "external", + "extract", + "false", + "fetch", + "fields", + "file", + "fileformat", + "first", + "float", + "for", + "format", + "formatted", + "functions", + "grant", + "group", + "grouping", + "hold_ddltime", + "hour", + "hours", + "idxproperties", + "ignore", + "import", + "in", + "index", + "indexes", + "inpath", + "inputdriver", + "inputformat", + "insert", + "int", + "into", + "is", + "isolation", + "items", + "jar", + "key_type", + "keys", + "last", + "lateral", + "leading", + "level", + "like", + "limit", + "lines", + "load", + "local", + "location", + "lock", + "locks", + "logical", + "long", + "mapjoin", + "materialized", + "metadata", + "microsecond", + "microseconds", + "millisecond", + "milliseconds", + "minus", + "minute", + "minutes", + "month", + "months", + "msck", + "no_drop", + "none", + "noscan", + "null", + "nulls", + "of", + "offline", + "offset", + "option", + "order", + "out", + "outer", + "outputdriver", + "outputformat", + "overwrite", + "owner", + "partition", + "partitioned", + "partitions", + "percent", + "pivot", + "plus", + "position", + "pretty", + "principals", + "procedure", + "protection", + "purge", + "query", + "range", + "read", + "readonly", + "reads", + "rebuild", + "recordreader", + "recordwriter", + "regexp", + "reload", + "rename", + "repair", + "replace", + "replication", + "restrict", + "revoke", + "rewrite", + "rlike", + "role", + "roles", + "rollup", + "row", + "rows", + "schemas", + "second", + "seconds", + "serde", + "serdeproperties", + "server", + "set", + "sets", + "shared", + "show", + "show_database", + "skewed", + "smallint", + "snapshot", + "sort", + "sorted", + "ssl", + "statistics", + "stored", + "streamtable", + "string", + "struct", + "table", + "tables", + "tblproperties", + "temporary", + "terminated", + "timestamp", + "tinyint", + "to", + "touch", + "trailing", + "transaction", + "transactions", + "trigger", + "trim", + "true", + "truncate", + "unarchive", + "undo", + "uniontype", + "unlock", + "unset", + "unsigned", + "update", + "uri", + "use", + "user", + "utc", + "utctimestamp", + "values", + "view", + "week", + "weeks", + "while", + "with", + "work", + "write", + "year", + "years") - val hiveStrictNonReservedKeyword = Seq("anti", "full", "inner", "left", "semi", "right", - "natural", "union", "intersect", "except", "database", "on", "join", "cross", "select", "from", - "where", "having", "from", "to", "table", "with", "not") + val hiveStrictNonReservedKeyword = Seq( + "anti", + "cross", + "database", + "except", + "from", + "full", + "having", + "inner", + "intersect", + "join", + "left", + "natural", + "not", + "on", + "right", + "select", + "semi", + "table", + "to", + "union", + "where", + "with") + + // All the keywords in `docs/sql-keywords.md` are listed below: + val allCandidateKeywords = Set( + "add", + "after", + "all", + "alter", + "analyze", + "and", + "anti", + "any", + "archive", + "array", + "as", + "asc", + "at", + "authorization", + "between", + "both", + "bucket", + "buckets", + "by", + "cache", + "cascade", + "case", + "cast", + "change", + "check", + "clear", + "cluster", + "clustered", + "codegen", + "collate", + "collection", + "column", + "columns", + "comment", + "commit", + "compact", + "compactions", + "compute", + "concatenate", + "constraint", + "cost", + "create", + "cross", + "cube", + "current", + "current_date", + "current_time", + "current_timestamp", + "current_user", + "data", + "database", + "databases", + "day", + "days", + "dbproperties", + "defined", + "delete", + "delimited", + "desc", + "describe", + "dfs", + "directories", + "directory", + "distinct", + "distribute", + "div", + "drop", + "else", + "end", + "escaped", + "except", + "exchange", + "exists", + "explain", + "export", + "extended", + "external", + "extract", + "false", + "fetch", + "fields", + "fileformat", + "first", + "first_value", + "following", + "for", + "foreign", + "format", + "formatted", + "from", + "full", + "function", + "functions", + "global", + "grant", + "group", + "grouping", + "having", + "hour", + "hours", + "if", + "ignore", + "import", + "in", + "index", + "indexes", + "inner", + "inpath", + "inputformat", + "insert", + "intersect", + "interval", + "into", + "is", + "items", + "join", + "keys", + "last", + "last_value", + "lateral", + "lazy", + "leading", + "left", + "like", + "limit", + "lines", + "list", + "load", + "local", + "location", + "lock", + "locks", + "logical", + "macro", + "map", + "microsecond", + "microseconds", + "millisecond", + "milliseconds", + "minus", + "minute", + "minutes", + "month", + "months", + "msck", + "natural", + "no", + "not", + "null", + "nulls", + "of", + "on", + "only", + "option", + "options", + "or", + "order", + "out", + "outer", + "outputformat", + "over", + "overlaps", + "overlay", + "overwrite", + "partition", + "partitioned", + "partitions", + "percent", + "pivot", + "placing", + "position", + "preceding", + "primary", + "principals", + "purge", + "query", + "range", + "recordreader", + "recordwriter", + "recover", + "reduce", + "references", + "refresh", + "rename", + "repair", + "replace", + "reset", + "respect", + "restrict", + "revoke", + "right", + "rlike", + "role", + "roles", + "rollback", + "rollup", + "row", + "rows", + "schema", + "second", + "seconds", + "select", + "semi", + "separated", + "serde", + "serdeproperties", + "session_user", + "set", + "sets", + "show", + "skewed", + "some", + "sort", + "sorted", + "start", + "statistics", + "stored", + "stratify", + "struct", + "substr", + "substring", + "table", + "tables", + "tablesample", + "tblproperties", + "temporary", + "terminated", + "then", + "to", + "touch", + "trailing", + "transaction", + "transactions", + "transform", + "true", + "truncate", + "unarchive", + "unbounded", + "uncache", + "union", + "unique", + "unlock", + "unset", + "use", + "user", + "using", + "values", + "view", + "week", + "weeks", + "when", + "where", + "window", + "with", + "year", + "years") + + val reservedKeywordsInAnsiMode = Set( + "all", + "and", + "anti", + "any", + "as", + "authorization", + "both", + "case", + "cast", + "check", + "collate", + "column", + "constraint", + "create", + "cross", + "current_date", + "current_time", + "current_timestamp", + "current_user", + "day", + "distinct", + "else", + "end", + "except", + "false", + "fetch", + "first_value", + "for", + "foreign", + "from", + "full", + "grant", + "group", + "having", + "hour", + "in", + "inner", + "intersect", + "into", + "join", + "is", + "last_value", + "leading", + "left", + "minute", + "month", + "natural", + "not", + "null", + "on", + "only", + "or", + "order", + "outer", + "overlaps", + "primary", + "references", + "right", + "select", + "semi", + "session_user", + "minus", + "second", + "some", + "table", + "then", + "to", + "trailing", + "union", + "unique", + "user", + "using", + "when", + "where", + "with", + "year") + + val nonReservedKeywordsInAnsiMode = allCandidateKeywords -- reservedKeywordsInAnsiMode test("table identifier") { // Regular names. @@ -76,6 +654,23 @@ class TableIdentifierParserSuite extends SparkFunSuite { assert(TableIdentifier("x.y.z", None) === parseTableIdentifier("`x.y.z`")) } + test("table identifier - reserved/non-reserved keywords if ANSI mode enabled") { + withSQLConf(SQLConf.ANSI_SQL_PARSER.key -> "true") { + reservedKeywordsInAnsiMode.foreach { keyword => + val errMsg = intercept[ParseException] { + parseTableIdentifier(keyword) + }.getMessage + assert(errMsg.contains("no viable alternative at input")) + assert(TableIdentifier(keyword) === parseTableIdentifier(s"`$keyword`")) + assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.`$keyword`")) + } + nonReservedKeywordsInAnsiMode.foreach { keyword => + assert(TableIdentifier(keyword) === parseTableIdentifier(s"$keyword")) + assert(TableIdentifier(keyword, Option("db")) === parseTableIdentifier(s"db.$keyword")) + } + } + } + test("table identifier - strict keywords") { // SQL Keywords. hiveStrictNonReservedKeyword.foreach { keyword => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala index d56f97970122c..43221bf60ca34 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/JoinTypesTest.scala @@ -48,11 +48,13 @@ class JoinTypesTest extends SparkFunSuite { test("construct a LeftSemi type") { assert(JoinType("leftsemi") === LeftSemi) assert(JoinType("left_semi") === LeftSemi) + assert(JoinType("semi") === LeftSemi) } test("construct a LeftAnti type") { assert(JoinType("leftanti") === LeftAnti) assert(JoinType("left_anti") === LeftAnti) + assert(JoinType("anti") === LeftAnti) } test("construct a Cross type") { diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala index aaab3ff1bf128..84452399de824 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/LogicalPlanSuite.scala @@ -69,10 +69,10 @@ class LogicalPlanSuite extends SparkFunSuite { } require(relation.isStreaming === false) - require(incrementalRelation.isStreaming === true) + require(incrementalRelation.isStreaming) assert(TestBinaryRelation(relation, relation).isStreaming === false) - assert(TestBinaryRelation(incrementalRelation, relation).isStreaming === true) - assert(TestBinaryRelation(relation, incrementalRelation).isStreaming === true) + assert(TestBinaryRelation(incrementalRelation, relation).isStreaming) + assert(TestBinaryRelation(relation, incrementalRelation).isStreaming) assert(TestBinaryRelation(incrementalRelation, incrementalRelation).isStreaming) } 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 3081ff935f043..5394732f41f2d 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 @@ -99,11 +99,11 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => .reduce(And), child) case sample: Sample => sample.copy(seed = 0L) - case Join(left, right, joinType, condition) if condition.isDefined => + case Join(left, right, joinType, condition, hint) if condition.isDefined => val newCondition = splitConjunctivePredicates(condition.get).map(rewriteEqual).sortBy(_.hashCode()) .reduce(And) - Join(left, right, joinType, Some(newCondition)) + Join(left, right, joinType, Some(newCondition), hint) } } @@ -165,8 +165,10 @@ trait PlanTestBase extends PredicateHelper with SQLHelper { self: Suite => private def sameJoinPlan(plan1: LogicalPlan, plan2: LogicalPlan): Boolean = { (plan1, plan2) match { case (j1: Join, j2: Join) => - (sameJoinPlan(j1.left, j2.left) && sameJoinPlan(j1.right, j2.right)) || - (sameJoinPlan(j1.left, j2.right) && sameJoinPlan(j1.right, j2.left)) + (sameJoinPlan(j1.left, j2.left) && sameJoinPlan(j1.right, j2.right) + && j1.hint.leftHint == j2.hint.leftHint && j1.hint.rightHint == j2.hint.rightHint) || + (sameJoinPlan(j1.left, j2.right) && sameJoinPlan(j1.right, j2.left) + && j1.hint.leftHint == j2.hint.rightHint && j1.hint.rightHint == j2.hint.leftHint) case (p1: Project, p2: Project) => p1.projectList == p2.projectList && sameJoinPlan(p1.child, p2.child) case _ => diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala index 7c8ed78a49116..fbaaf807af5d6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/SameResultSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.catalyst.plans import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, ResolvedHint, Union} +import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.util._ /** @@ -30,6 +32,10 @@ class SameResultSuite extends SparkFunSuite { val testRelation = LocalRelation('a.int, 'b.int, 'c.int) val testRelation2 = LocalRelation('a.int, 'b.int, 'c.int) + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = Batch("EliminateResolvedHint", Once, EliminateResolvedHint) :: Nil + } + def assertSameResult(a: LogicalPlan, b: LogicalPlan, result: Boolean = true): Unit = { val aAnalyzed = a.analyze val bAnalyzed = b.analyze @@ -72,4 +78,12 @@ class SameResultSuite extends SparkFunSuite { val df2 = testRelation.join(testRelation) assertSameResult(df1, df2) } + + test("join hint") { + val df1 = testRelation.join(testRelation.hint("broadcast")) + val df2 = testRelation.join(testRelation) + val df1Optimized = Optimize.execute(df1.analyze) + val df2Optimized = Optimize.execute(df2.analyze) + assertSameResult(df1Optimized, df2Optimized) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala index 8213d568fe85e..32bf20b8c17f6 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala @@ -38,13 +38,41 @@ class AggregateEstimationSuite extends StatsEstimationTestBase with PlanTest { attr("key22") -> ColumnStat(distinctCount = Some(2), min = Some(10), max = Some(20), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)), attr("key31") -> ColumnStat(distinctCount = Some(0), min = None, max = None, - nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) + nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)), + attr("key32") -> ColumnStat(distinctCount = Some(0), min = None, max = None, + nullCount = Some(4), avgLen = Some(4), maxLen = Some(4)), + attr("key33") -> ColumnStat(distinctCount = Some(2), min = None, max = None, + nullCount = Some(2), avgLen = Some(4), maxLen = Some(4)) )) private val nameToAttr: Map[String, Attribute] = columnInfo.map(kv => kv._1.name -> kv._1) private val nameToColInfo: Map[String, (Attribute, ColumnStat)] = columnInfo.map(kv => kv._1.name -> kv) + test("SPARK-26894: propagate child stats for aliases in Aggregate") { + val tableColumns = Seq("key11", "key12") + val groupByColumns = Seq("key11") + val attributes = groupByColumns.map(nameToAttr) + + val rowCount = 2 + val child = StatsTestPlan( + outputList = tableColumns.map(nameToAttr), + rowCount, + // rowCount * (overhead + column size) + size = Some(4 * (8 + 4)), + attributeStats = AttributeMap(tableColumns.map(nameToColInfo))) + + val testAgg = Aggregate( + groupingExpressions = attributes, + aggregateExpressions = Seq(Alias(nameToAttr("key12"), "abc")()), + child) + + val expectedColStats = Seq("abc" -> nameToColInfo("key12")._2) + val expectedAttrStats = toAttributeMap(expectedColStats, testAgg) + + assert(testAgg.stats.attributeStats == expectedAttrStats) + } + test("set an upper bound if the product of ndv's of group-by columns is too large") { // Suppose table1 (key11 int, key12 int) has 4 records: (1, 10), (1, 20), (2, 30), (2, 40) checkAggStats( @@ -92,6 +120,23 @@ class AggregateEstimationSuite extends StatsEstimationTestBase with PlanTest { expectedOutputRowCount = 0) } + test("group-by column with only null value") { + checkAggStats( + tableColumns = Seq("key22", "key32"), + tableRowCount = 6, + groupByColumns = Seq("key22", "key32"), + expectedOutputRowCount = nameToColInfo("key22")._2.distinctCount.get) + } + + test("group-by column with null value") { + checkAggStats( + tableColumns = Seq("key21", "key33"), + tableRowCount = 6, + groupByColumns = Seq("key21", "key33"), + expectedOutputRowCount = nameToColInfo("key21")._2.distinctCount.get * + (nameToColInfo("key33")._2.distinctCount.get + 1)) + } + test("non-cbo estimation") { val attributes = Seq("key12").map(nameToAttr) val child = StatsTestPlan( diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala index 953094cb0dd52..16a5c2d3001a7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala @@ -38,24 +38,6 @@ class BasicStatsEstimationSuite extends PlanTest with StatsEstimationTestBase { // row count * (overhead + column size) size = Some(10 * (8 + 4))) - test("BroadcastHint estimation") { - val filter = Filter(Literal(true), plan) - val filterStatsCboOn = Statistics(sizeInBytes = 10 * (8 +4), - rowCount = Some(10), attributeStats = AttributeMap(Seq(attribute -> colStat))) - val filterStatsCboOff = Statistics(sizeInBytes = 10 * (8 +4)) - checkStats( - filter, - expectedStatsCboOn = filterStatsCboOn, - expectedStatsCboOff = filterStatsCboOff) - - val broadcastHint = ResolvedHint(filter, HintInfo(broadcast = true)) - checkStats( - broadcastHint, - expectedStatsCboOn = filterStatsCboOn.copy(hints = HintInfo(broadcast = true)), - expectedStatsCboOff = filterStatsCboOff.copy(hints = HintInfo(broadcast = true)) - ) - } - test("range") { val range = Range(1, 5, 1, None) val rangeStats = Statistics(sizeInBytes = 4 * 8) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala index 47bfa62569583..1cf888519077a 100755 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral} import org.apache.spark.sql.catalyst.plans.LeftOuter import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.{ColumnStatsMap, FilterEstimation} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.types._ @@ -527,7 +528,7 @@ class FilterEstimationSuite extends StatsEstimationTestBase { rowCount = 30, attributeStats = AttributeMap(Seq(attrIntLargerRange -> colStatIntLargerRange))) val nonLeafChild = Join(largerTable, smallerTable, LeftOuter, - Some(EqualTo(attrIntLargerRange, attrInt))) + Some(EqualTo(attrIntLargerRange, attrInt)), JoinHint.NONE) Seq(IsNull(attrIntLargerRange), IsNotNull(attrIntLargerRange)).foreach { predicate => validateEstimatedStats( @@ -821,6 +822,32 @@ class FilterEstimationSuite extends StatsEstimationTestBase { expectedRowCount = 3) } + test("ColumnStatsMap tests") { + val attrNoDistinct = AttributeReference("att_without_distinct", IntegerType)() + val attrNoCount = AttributeReference("att_without_count", BooleanType)() + val attrNoMinMax = AttributeReference("att_without_min_max", DateType)() + val colStatNoDistinct = ColumnStat(distinctCount = None, min = Some(1), max = Some(10), + nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) + val colStatNoCount = ColumnStat(distinctCount = Some(2), min = Some(false), max = Some(true), + nullCount = None, avgLen = Some(1), maxLen = Some(1)) + val colStatNoMinMax = ColumnStat(distinctCount = Some(1), min = None, max = None, + nullCount = Some(1), avgLen = None, maxLen = None) + val columnStatsMap = ColumnStatsMap(AttributeMap(Seq( + attrNoDistinct -> colStatNoDistinct, + attrNoCount -> colStatNoCount, + attrNoMinMax -> colStatNoMinMax + ))) + assert(!columnStatsMap.hasDistinctCount(attrNoDistinct)) + assert(columnStatsMap.hasDistinctCount(attrNoCount)) + assert(columnStatsMap.hasDistinctCount(attrNoMinMax)) + assert(!columnStatsMap.hasCountStats(attrNoDistinct)) + assert(!columnStatsMap.hasCountStats(attrNoCount)) + assert(columnStatsMap.hasCountStats(attrNoMinMax)) + assert(columnStatsMap.hasMinMaxStats(attrNoDistinct)) + assert(columnStatsMap.hasMinMaxStats(attrNoCount)) + assert(!columnStatsMap.hasMinMaxStats(attrNoMinMax)) + } + private def childStatsTestPlan(outList: Seq[Attribute], tableRowCount: BigInt): StatsTestPlan = { StatsTestPlan( outputList = outList, diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala index 12c0a7be21292..6c5a2b247fc23 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/JoinEstimationSuite.scala @@ -79,8 +79,8 @@ class JoinEstimationSuite extends StatsEstimationTestBase { val c1 = generateJoinChild(col1, leftHistogram, expectedMin, expectedMax) val c2 = generateJoinChild(col2, rightHistogram, expectedMin, expectedMax) - val c1JoinC2 = Join(c1, c2, Inner, Some(EqualTo(col1, col2))) - val c2JoinC1 = Join(c2, c1, Inner, Some(EqualTo(col2, col1))) + val c1JoinC2 = Join(c1, c2, Inner, Some(EqualTo(col1, col2)), JoinHint.NONE) + val c2JoinC1 = Join(c2, c1, Inner, Some(EqualTo(col2, col1)), JoinHint.NONE) val expectedStatsAfterJoin = Statistics( sizeInBytes = expectedRows * (8 + 2 * 4), rowCount = Some(expectedRows), @@ -284,7 +284,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { test("cross join") { // table1 (key-1-5 int, key-5-9 int): (1, 9), (2, 8), (3, 7), (4, 6), (5, 5) // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) - val join = Join(table1, table2, Cross, None) + val join = Join(table1, table2, Cross, None, JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 5 * 3 * (8 + 4 * 4), rowCount = Some(5 * 3), @@ -299,7 +299,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, Inner, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1, rowCount = Some(0), @@ -312,7 +312,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, LeftOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 5 * (8 + 4 * 4), rowCount = Some(5), @@ -328,7 +328,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, RightOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 4), rowCount = Some(3), @@ -344,7 +344,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // key-5-9 and key-2-4 are disjoint val join = Join(table1, table2, FullOuter, - Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-5-9"), nameToAttr("key-2-4"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = (5 + 3) * (8 + 4 * 4), rowCount = Some(5 + 3), @@ -361,7 +361,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table1 (key-1-5 int, key-5-9 int): (1, 9), (2, 8), (3, 7), (4, 6), (5, 5) // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) val join = Join(table1, table2, Inner, - Some(EqualTo(nameToAttr("key-1-5"), nameToAttr("key-1-2")))) + Some(EqualTo(nameToAttr("key-1-5"), nameToAttr("key-1-2"))), JoinHint.NONE) // Update column stats for equi-join keys (key-1-5 and key-1-2). val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(1), max = Some(2), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -383,7 +383,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, Inner, Some( And(EqualTo(nameToAttr("key-1-2"), nameToAttr("key-1-2")), - EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))))) + EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))), JoinHint.NONE) // Update column stats for join keys. val joinedColStat1 = ColumnStat(distinctCount = Some(2), min = Some(1), max = Some(2), @@ -404,7 +404,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table3, table2, LeftOuter, - Some(EqualTo(nameToAttr("key-2-3"), nameToAttr("key-2-4")))) + Some(EqualTo(nameToAttr("key-2-3"), nameToAttr("key-2-4"))), JoinHint.NONE) val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(2), max = Some(3), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -422,7 +422,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, RightOuter, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) val joinedColStat = ColumnStat(distinctCount = Some(2), min = Some(2), max = Some(3), nullCount = Some(0), avgLen = Some(4), maxLen = Some(4)) @@ -440,7 +440,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table2 (key-1-2 int, key-2-4 int): (1, 2), (2, 3), (2, 4) // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) val join = Join(table2, table3, FullOuter, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 4), @@ -456,7 +456,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { // table3 (key-1-2 int, key-2-3 int): (1, 2), (2, 3) Seq(LeftSemi, LeftAnti).foreach { jt => val join = Join(table2, table3, jt, - Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3")))) + Some(EqualTo(nameToAttr("key-2-4"), nameToAttr("key-2-3"))), JoinHint.NONE) // For now we just propagate the statistics from left side for left semi/anti join. val expectedStats = Statistics( sizeInBytes = 3 * (8 + 4 * 2), @@ -525,7 +525,7 @@ class JoinEstimationSuite extends StatsEstimationTestBase { withClue(s"For data type ${key1.dataType}") { // All values in two tables are the same, so column stats after join are also the same. val join = Join(Project(Seq(key1), table1), Project(Seq(key2), table2), Inner, - Some(EqualTo(key1, key2))) + Some(EqualTo(key1, key2)), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1 * (8 + 2 * getColSize(key1, columnInfo1(key1))), rowCount = Some(1), @@ -543,7 +543,8 @@ class JoinEstimationSuite extends StatsEstimationTestBase { outputList = Seq(nullColumn), rowCount = 1, attributeStats = AttributeMap(Seq(nullColumn -> nullColStat))) - val join = Join(table1, nullTable, Inner, Some(EqualTo(nameToAttr("key-1-5"), nullColumn))) + val join = Join(table1, nullTable, Inner, + Some(EqualTo(nameToAttr("key-1-5"), nullColumn)), JoinHint.NONE) val expectedStats = Statistics( sizeInBytes = 1, rowCount = Some(0), diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index a67f54b263cc9..8dbe198e54c4c 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -57,7 +57,7 @@ class RuleExecutorSuite extends SparkFunSuite { assert(message.contains("Max iterations (10) reached for batch fixedPoint")) } - test("structural integrity checker") { + test("structural integrity checker - verify initial input") { object WithSIChecker extends RuleExecutor[Expression] { override protected def isPlanIntegral(expr: Expression): Boolean = expr match { case IntegerLiteral(_) => true @@ -69,8 +69,32 @@ class RuleExecutorSuite extends SparkFunSuite { assert(WithSIChecker.execute(Literal(10)) === Literal(9)) val message = intercept[TreeNodeException[LogicalPlan]] { + // The input is already invalid as determined by WithSIChecker.isPlanIntegral WithSIChecker.execute(Literal(10.1)) }.getMessage + assert(message.contains("The structural integrity of the input plan is broken")) + } + + test("structural integrity checker - verify rule execution result") { + object WithSICheckerForPositiveLiteral extends RuleExecutor[Expression] { + override protected def isPlanIntegral(expr: Expression): Boolean = expr match { + case IntegerLiteral(i) if i > 0 => true + case _ => false + } + val batches = Batch("once", Once, DecrementLiterals) :: Nil + } + + assert(WithSICheckerForPositiveLiteral.execute(Literal(2)) === Literal(1)) + + val message = intercept[TreeNodeException[LogicalPlan]] { + WithSICheckerForPositiveLiteral.execute(Literal(1)) + }.getMessage assert(message.contains("the structural integrity of the plan is broken")) } + + test("SPARK-27243: dumpTimeSpent when no rule has run") { + RuleExecutor.resetMetrics() + // This should not throw an exception + RuleExecutor.dumpTimeSpent() + } } 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 b7092f4c42d4c..fbaa5527a7058 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 @@ -33,9 +33,10 @@ import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions.DslString 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} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Union} +import org.apache.spark.sql.catalyst.plans.{LeftOuter, NaturalJoin, SQLHelper} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{IdentityBroadcastMode, RoundRobinPartitioning, SinglePartition} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.storage.StorageLevel @@ -81,7 +82,12 @@ case class SelfReferenceUDF( def apply(key: String): Boolean = config.contains(key) } -class TreeNodeSuite extends SparkFunSuite { +case class FakeLeafPlan(child: LogicalPlan) + extends org.apache.spark.sql.catalyst.plans.logical.LeafNode { + override def output: Seq[Attribute] = child.output +} + +class TreeNodeSuite extends SparkFunSuite with SQLHelper { test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } assert(after === Literal(2)) @@ -564,7 +570,7 @@ class TreeNodeSuite extends SparkFunSuite { } test("toJSON should not throws java.lang.StackOverflowError") { - val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr)) + val udf = ScalaUDF(SelfReferenceUDF(), BooleanType, Seq("col1".attr), false :: Nil) // Should not throw java.lang.StackOverflowError udf.toJSON } @@ -595,4 +601,111 @@ class TreeNodeSuite extends SparkFunSuite { val expected = Coalesce(Stream(Literal(1), Literal(3))) assert(result === expected) } + + test("treeString limits plan length") { + withSQLConf(SQLConf.MAX_PLAN_STRING_LENGTH.key -> "200") { + val ds = (1 until 20).foldLeft(Literal("TestLiteral"): Expression) { case (treeNode, x) => + Add(Literal(x), treeNode) + } + + val planString = ds.treeString + logWarning("Plan string: " + planString) + assert(planString.endsWith(" more characters")) + assert(planString.length <= SQLConf.get.maxPlanStringLength) + } + } + + test("treeString limit at zero") { + withSQLConf(SQLConf.MAX_PLAN_STRING_LENGTH.key -> "0") { + val ds = (1 until 2).foldLeft(Literal("TestLiteral"): Expression) { case (treeNode, x) => + Add(Literal(x), treeNode) + } + + val planString = ds.treeString + assert(planString.startsWith("Truncated plan of")) + } + } + + test("tags will be carried over after copy & transform") { + val tag = TreeNodeTag[String]("test") + + withClue("makeCopy") { + val node = Dummy(None) + node.setTagValue(tag, "a") + val copied = node.makeCopy(Array(Some(Literal(1)))) + assert(copied.getTagValue(tag) == Some("a")) + } + + def checkTransform( + sameTypeTransform: Expression => Expression, + differentTypeTransform: Expression => Expression): Unit = { + val child = Dummy(None) + child.setTagValue(tag, "child") + val node = Dummy(Some(child)) + node.setTagValue(tag, "parent") + + val transformed = sameTypeTransform(node) + // Both the child and parent keep the tags + assert(transformed.getTagValue(tag) == Some("parent")) + assert(transformed.children.head.getTagValue(tag) == Some("child")) + + val transformed2 = differentTypeTransform(node) + // Both the child and parent keep the tags, even if we transform the node to a new one of + // different type. + assert(transformed2.getTagValue(tag) == Some("parent")) + assert(transformed2.children.head.getTagValue(tag) == Some("child")) + } + + withClue("transformDown") { + checkTransform( + sameTypeTransform = _ transformDown { + case Dummy(None) => Dummy(Some(Literal(1))) + }, + differentTypeTransform = _ transformDown { + case Dummy(None) => Literal(1) + + }) + } + + withClue("transformUp") { + checkTransform( + sameTypeTransform = _ transformUp { + case Dummy(None) => Dummy(Some(Literal(1))) + }, + differentTypeTransform = _ transformUp { + case Dummy(None) => Literal(1) + + }) + } + } + + test("clone") { + def assertDifferentInstance(before: AnyRef, after: AnyRef): Unit = { + assert(before.ne(after) && before == after) + before.asInstanceOf[TreeNode[_]].children.zip( + after.asInstanceOf[TreeNode[_]].children).foreach { + case (beforeChild: AnyRef, afterChild: AnyRef) => + assertDifferentInstance(beforeChild, afterChild) + } + } + + // Empty constructor + val rowNumber = RowNumber() + assertDifferentInstance(rowNumber, rowNumber.clone()) + + // Overridden `makeCopy` + val oneRowRelation = OneRowRelation() + assertDifferentInstance(oneRowRelation, oneRowRelation.clone()) + + // Multi-way operators + val intersect = + Intersect(oneRowRelation, Union(Seq(oneRowRelation, oneRowRelation)), isAll = false) + assertDifferentInstance(intersect, intersect.clone()) + + // Leaf node with an inner child + val leaf = FakeLeafPlan(intersect) + val leafCloned = leaf.clone() + assertDifferentInstance(leaf, leafCloned) + assert(leaf.child.eq(leafCloned.asInstanceOf[FakeLeafPlan].child)) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala new file mode 100644 index 0000000000000..8509bce177129 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayBasedMapBuilderSuite.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeRow} +import org.apache.spark.sql.types.{ArrayType, BinaryType, IntegerType, StructType} +import org.apache.spark.unsafe.Platform + +class ArrayBasedMapBuilderSuite extends SparkFunSuite { + + test("basic") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, 1) + builder.put(InternalRow(2, 2)) + builder.putAll(new GenericArrayData(Seq(3)), new GenericArrayData(Seq(3))) + val map = builder.build() + assert(map.numElements() == 3) + assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 1, 2 -> 2, 3 -> 3)) + } + + test("fail with null key") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, null) // null value is OK + val e = intercept[RuntimeException](builder.put(null, 1)) + assert(e.getMessage.contains("Cannot use null as map key")) + } + + test("remove duplicated keys with last wins policy") { + val builder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + builder.put(1, 1) + builder.put(2, 2) + builder.put(1, 2) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == Map(1 -> 2, 2 -> 2)) + } + + test("binary type key") { + val builder = new ArrayBasedMapBuilder(BinaryType, IntegerType) + builder.put(Array(1.toByte), 1) + builder.put(Array(2.toByte), 2) + builder.put(Array(1.toByte), 3) + val map = builder.build() + assert(map.numElements() == 2) + val entries = ArrayBasedMapData.toScalaMap(map).iterator.toSeq + assert(entries(0)._1.asInstanceOf[Array[Byte]].toSeq == Seq(1)) + assert(entries(0)._2 == 3) + assert(entries(1)._1.asInstanceOf[Array[Byte]].toSeq == Seq(2)) + assert(entries(1)._2 == 2) + } + + test("struct type key") { + val builder = new ArrayBasedMapBuilder(new StructType().add("i", "int"), IntegerType) + builder.put(InternalRow(1), 1) + builder.put(InternalRow(2), 2) + val unsafeRow = { + val row = new UnsafeRow(1) + val bytes = new Array[Byte](16) + row.pointTo(bytes, 16) + row.setInt(0, 1) + row + } + builder.put(unsafeRow, 3) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == Map(InternalRow(1) -> 3, InternalRow(2) -> 2)) + } + + test("array type key") { + val builder = new ArrayBasedMapBuilder(ArrayType(IntegerType), IntegerType) + builder.put(new GenericArrayData(Seq(1, 1)), 1) + builder.put(new GenericArrayData(Seq(2, 2)), 2) + val unsafeArray = { + val array = new UnsafeArrayData() + val bytes = new Array[Byte](24) + Platform.putLong(bytes, Platform.BYTE_ARRAY_OFFSET, 2) + array.pointTo(bytes, Platform.BYTE_ARRAY_OFFSET, 24) + array.setInt(0, 1) + array.setInt(1, 1) + array + } + builder.put(unsafeArray, 3) + val map = builder.build() + assert(map.numElements() == 2) + assert(ArrayBasedMapData.toScalaMap(map) == + Map(new GenericArrayData(Seq(1, 1)) -> 3, new GenericArrayData(Seq(2, 2)) -> 2)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala index 6400898343ae7..da71e3a4d53e2 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/ArrayDataIndexedSeqSuite.scala @@ -22,7 +22,7 @@ import scala.util.Random import org.apache.spark.SparkFunSuite import org.apache.spark.sql.RandomDataGenerator import org.apache.spark.sql.catalyst.encoders.{ExamplePointUDT, RowEncoder} -import org.apache.spark.sql.catalyst.expressions.{FromUnsafeProjection, UnsafeArrayData, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.{SafeProjection, UnsafeArrayData, UnsafeProjection} import org.apache.spark.sql.types._ class ArrayDataIndexedSeqSuite extends SparkFunSuite { @@ -77,7 +77,7 @@ class ArrayDataIndexedSeqSuite extends SparkFunSuite { val internalRow = rowConverter.toRow(row) val unsafeRowConverter = UnsafeProjection.create(schema) - val safeRowConverter = FromUnsafeProjection(schema) + val safeRowConverter = SafeProjection.create(schema) val unsafeRow = unsafeRowConverter(internalRow) val safeRow = safeRowConverter(unsafeRow) 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 0c1feb3aa0882..4dfeb85c74f93 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 @@ -17,7 +17,11 @@ package org.apache.spark.sql.catalyst.util +import java.time.{LocalDate, LocalDateTime, LocalTime, ZoneId} import java.util.TimeZone +import java.util.concurrent.TimeUnit + +import org.apache.spark.sql.catalyst.util.DateTimeUtils.TimeZoneUTC /** * Helper functions for testing date and time functionality. @@ -26,15 +30,71 @@ object DateTimeTestUtils { val ALL_TIMEZONES: Seq[TimeZone] = TimeZone.getAvailableIDs.toSeq.map(TimeZone.getTimeZone) + val outstandingTimezonesIds: Seq[String] = Seq( + "UTC", + "PST", + "CET", + "Africa/Dakar", + "America/Los_Angeles", + "Antarctica/Vostok", + "Asia/Hong_Kong", + "Europe/Amsterdam") + val outstandingTimezones: Seq[TimeZone] = outstandingTimezonesIds.map(TimeZone.getTimeZone) + val outstandingZoneIds: Seq[ZoneId] = outstandingTimezonesIds.map(DateTimeUtils.getZoneId) + def withDefaultTimeZone[T](newDefaultTimeZone: TimeZone)(block: => T): T = { val originalDefaultTimeZone = TimeZone.getDefault try { - DateTimeUtils.resetThreadLocals() TimeZone.setDefault(newDefaultTimeZone) block } finally { TimeZone.setDefault(originalDefaultTimeZone) - DateTimeUtils.resetThreadLocals() } } + + def localDateTimeToMicros(localDateTime: LocalDateTime, tz: TimeZone): Long = { + val instant = localDateTime.atZone(tz.toZoneId).toInstant + DateTimeUtils.instantToMicros(instant) + } + + // Returns microseconds since epoch for the given date + def date( + year: Int, + month: Byte = 1, + day: Byte = 1, + hour: Byte = 0, + minute: Byte = 0, + sec: Byte = 0, + micros: Int = 0, + tz: TimeZone = TimeZoneUTC): Long = { + val nanos = TimeUnit.MICROSECONDS.toNanos(micros).toInt + val localDateTime = LocalDateTime.of(year, month, day, hour, minute, sec, nanos) + localDateTimeToMicros(localDateTime, tz) + } + + // Returns number of days since epoch for the given date + def days( + year: Int, + month: Byte = 1, + day: Byte = 1, + hour: Byte = 0, + minute: Byte = 0, + sec: Byte = 0): Int = { + val micros = date(year, month, day, hour, minute, sec) + TimeUnit.MICROSECONDS.toDays(micros).toInt + } + + // Returns microseconds since epoch for current date and give time + def time( + hour: Byte = 0, + minute: Byte = 0, + sec: Byte = 0, + micros: Int = 0, + tz: TimeZone = TimeZoneUTC): Long = { + val nanos = TimeUnit.MICROSECONDS.toNanos(micros).toInt + val localDate = LocalDate.now(tz.toZoneId) + val localTime = LocalTime.of(hour, minute, sec, nanos) + val localDateTime = LocalDateTime.of(localDate, localTime) + localDateTimeToMicros(localDateTime, tz) + } } 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 2423668392231..8ff691fb17f27 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,26 +19,27 @@ package org.apache.spark.sql.catalyst.util import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat -import java.util.{Calendar, Locale, TimeZone} +import java.time.ZoneId +import java.util.{Locale, TimeZone} +import java.util.concurrent.TimeUnit import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.unsafe.types.UTF8String class DateTimeUtilsSuite extends SparkFunSuite { val TimeZonePST = TimeZone.getTimeZone("PST") - - private[this] def getInUTCDays(timestamp: Long): Int = { - val tz = TimeZone.getDefault - ((timestamp + tz.getOffset(timestamp)) / MILLIS_PER_DAY).toInt - } + private def defaultZoneId = ZoneId.systemDefault() test("nanoseconds truncation") { + val tf = TimestampFormatter.getFractionFormatter(DateTimeUtils.defaultTimeZone.toZoneId) def checkStringToTimestamp(originalTime: String, expectedParsedTime: String) { - val parsedTimestampOp = DateTimeUtils.stringToTimestamp(UTF8String.fromString(originalTime)) + val parsedTimestampOp = DateTimeUtils.stringToTimestamp( + UTF8String.fromString(originalTime), defaultZoneId) assert(parsedTimestampOp.isDefined, "timestamp with nanoseconds was not parsed correctly") - assert(DateTimeUtils.timestampToString(parsedTimestampOp.get) === expectedParsedTime) + assert(DateTimeUtils.timestampToString(tf, parsedTimestampOp.get) === expectedParsedTime) } checkStringToTimestamp("2015-01-02 00:00:00.123456789", "2015-01-02 00:00:00.123456") @@ -118,38 +119,14 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("string to date") { - - var c = Calendar.getInstance() - c.set(2015, 0, 28, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToDate(UTF8String.fromString("2015-01-28")).get === - millisToDays(c.getTimeInMillis)) - c.set(2015, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToDate(UTF8String.fromString("2015")).get === - millisToDays(c.getTimeInMillis)) - c.set(1, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToDate(UTF8String.fromString("0001")).get === - millisToDays(c.getTimeInMillis)) - c = Calendar.getInstance() - c.set(2015, 2, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToDate(UTF8String.fromString("2015-03")).get === - millisToDays(c.getTimeInMillis)) - c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(stringToDate(UTF8String.fromString("2015-03-18")).get === - millisToDays(c.getTimeInMillis)) - assert(stringToDate(UTF8String.fromString("2015-03-18 ")).get === - millisToDays(c.getTimeInMillis)) - assert(stringToDate(UTF8String.fromString("2015-03-18 123142")).get === - millisToDays(c.getTimeInMillis)) - assert(stringToDate(UTF8String.fromString("2015-03-18T123123")).get === - millisToDays(c.getTimeInMillis)) - assert(stringToDate(UTF8String.fromString("2015-03-18T")).get === - millisToDays(c.getTimeInMillis)) + assert(stringToDate(UTF8String.fromString("2015-01-28")).get === days(2015, 1, 28)) + assert(stringToDate(UTF8String.fromString("2015")).get === days(2015, 1, 1)) + assert(stringToDate(UTF8String.fromString("0001")).get === days(1, 1, 1)) + assert(stringToDate(UTF8String.fromString("2015-03")).get === days(2015, 3, 1)) + Seq("2015-03-18", "2015-03-18 ", " 2015-03-18", " 2015-03-18 ", "2015-03-18 123142", + "2015-03-18T123123", "2015-03-18T").foreach { s => + assert(stringToDate(UTF8String.fromString(s)).get === days(2015, 3, 18)) + } assert(stringToDate(UTF8String.fromString("2015-03-18X")).isEmpty) assert(stringToDate(UTF8String.fromString("2015/03/18")).isEmpty) @@ -160,167 +137,98 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(stringToDate(UTF8String.fromString("015-03-18")).isEmpty) assert(stringToDate(UTF8String.fromString("015")).isEmpty) assert(stringToDate(UTF8String.fromString("02015")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999 08 01")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999-08 01")).isEmpty) + assert(stringToDate(UTF8String.fromString("1999 08")).isEmpty) } - test("string to time") { - // Tests with UTC. - val c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(Calendar.MILLISECOND, 0) - - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00GMT-00:00") === c.getTime()) - - c.set(2000, 11, 30, 10, 0, 0) - assert(stringToTime("2000-12-30T10:00:00Z") === c.getTime()) + test("string to timestamp") { + for (tz <- ALL_TIMEZONES) { + def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { + assert(stringToTimestamp(UTF8String.fromString(str), tz.toZoneId) === expected) + } - // Tests with set time zone. - c.setTimeZone(TimeZone.getTimeZone("GMT-04:00")) - c.set(Calendar.MILLISECOND, 0) + checkStringToTimestamp("1969-12-31 16:00:00", Option(date(1969, 12, 31, 16, tz = tz))) + checkStringToTimestamp("0001", Option(date(1, 1, 1, 0, tz = tz))) + checkStringToTimestamp("2015-03", Option(date(2015, 3, 1, tz = tz))) + Seq("2015-03-18", "2015-03-18 ", " 2015-03-18", " 2015-03-18 ", "2015-03-18T").foreach { s => + checkStringToTimestamp(s, Option(date(2015, 3, 18, tz = tz))) + } - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00-04:00") === c.getTime()) + var expected = Option(date(2015, 3, 18, 12, 3, 17, tz = tz)) + checkStringToTimestamp("2015-03-18 12:03:17", expected) + checkStringToTimestamp("2015-03-18T12:03:17", expected) - c.set(1900, 0, 1, 0, 0, 0) - assert(stringToTime("1900-01-01T00:00:00GMT-04:00") === c.getTime()) + // If the string value includes timezone string, it represents the timestamp string + // in the timezone regardless of the tz parameter. + var timeZone = TimeZone.getTimeZone("GMT-13:53") + expected = Option(date(2015, 3, 18, 12, 3, 17, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17-13:53", expected) - // Tests with local time zone. - c.setTimeZone(TimeZone.getDefault()) - c.set(Calendar.MILLISECOND, 0) + timeZone = TimeZone.getTimeZone("UTC") + expected = Option(date(2015, 3, 18, 12, 3, 17, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17Z", expected) + checkStringToTimestamp("2015-03-18 12:03:17Z", expected) - c.set(2000, 11, 30, 0, 0, 0) - assert(stringToTime("2000-12-30") === new Date(c.getTimeInMillis())) + timeZone = TimeZone.getTimeZone("GMT-01:00") + expected = Option(date(2015, 3, 18, 12, 3, 17, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17-1:0", expected) + checkStringToTimestamp("2015-03-18T12:03:17-01:00", expected) - c.set(2000, 11, 30, 10, 0, 0) - assert(stringToTime("2000-12-30 10:00:00") === new Timestamp(c.getTimeInMillis())) - } + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(date(2015, 3, 18, 12, 3, 17, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17+07:30", expected) - test("string to timestamp") { - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { - def checkStringToTimestamp(str: String, expected: Option[Long]): Unit = { - assert(stringToTimestamp(UTF8String.fromString(str), tz) === expected) - } + timeZone = TimeZone.getTimeZone("GMT+07:03") + expected = Option(date(2015, 3, 18, 12, 3, 17, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17+07:03", expected) - var c = Calendar.getInstance(tz) - c.set(1969, 11, 31, 16, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("1969-12-31 16:00:00", Option(c.getTimeInMillis * 1000)) - c.set(1, 0, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("0001", Option(c.getTimeInMillis * 1000)) - c = Calendar.getInstance(tz) - c.set(2015, 2, 1, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03", Option(c.getTimeInMillis * 1000)) - c = Calendar.getInstance(tz) - c.set(2015, 2, 18, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18 ", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18T", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(tz) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18 12:03:17", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18T12:03:17", Option(c.getTimeInMillis * 1000)) + // tests for the string including milliseconds. + expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, tz = tz)) + checkStringToTimestamp("2015-03-18 12:03:17.123", expected) + checkStringToTimestamp("2015-03-18T12:03:17.123", expected) // If the string value includes timezone string, it represents the timestamp string // in the timezone regardless of the tz parameter. - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-13:53")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18T12:03:17-13:53", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18T12:03:17Z", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18 12:03:17Z", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18T12:03:17-1:0", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18T12:03:17-01:00", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18T12:03:17+07:30", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:03")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("2015-03-18T12:03:17+07:03", Option(c.getTimeInMillis * 1000)) + timeZone = TimeZone.getTimeZone("UTC") + expected = Option(date(2015, 3, 18, 12, 3, 17, 456000, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.456Z", expected) + checkStringToTimestamp("2015-03-18 12:03:17.456Z", expected) - // tests for the string including milliseconds. - c = Calendar.getInstance(tz) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("2015-03-18 12:03:17.123", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18T12:03:17.123", Option(c.getTimeInMillis * 1000)) + timeZone = TimeZone.getTimeZone("GMT-01:00") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.123-1:0", expected) + checkStringToTimestamp("2015-03-18T12:03:17.123-01:00", expected) - // If the string value includes timezone string, it represents the timestamp string - // in the timezone regardless of the tz parameter. - c = Calendar.getInstance(TimeZone.getTimeZone("UTC")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 456) - checkStringToTimestamp("2015-03-18T12:03:17.456Z", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18 12:03:17.456Z", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT-01:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("2015-03-18T12:03:17.123-1:0", Option(c.getTimeInMillis * 1000)) - checkStringToTimestamp("2015-03-18T12:03:17.123-01:00", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp( - "2015-03-18T12:03:17.123121+7:30", Option(c.getTimeInMillis * 1000 + 121)) + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected) - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp( - "2015-03-18T12:03:17.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) - - c = Calendar.getInstance(tz) - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 0) - checkStringToTimestamp("18:12:15", Option(c.getTimeInMillis * 1000)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("T18:12:15.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) - - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+07:30")) - c.set(Calendar.HOUR_OF_DAY, 18) - c.set(Calendar.MINUTE, 12) - c.set(Calendar.SECOND, 15) - c.set(Calendar.MILLISECOND, 123) - checkStringToTimestamp("18:12:15.12312+7:30", Option(c.getTimeInMillis * 1000 + 120)) - - c = Calendar.getInstance(tz) - c.set(2011, 4, 6, 7, 8, 9) - c.set(Calendar.MILLISECOND, 100) - checkStringToTimestamp("2011-05-06 07:08:09.1000", Option(c.getTimeInMillis * 1000)) + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123000, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.123+07:30", expected) + + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123121, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.123121+7:30", expected) + + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123120, tz = timeZone)) + checkStringToTimestamp("2015-03-18T12:03:17.12312+7:30", expected) + + expected = Option(time(18, 12, 15, tz = tz)) + checkStringToTimestamp("18:12:15", expected) + + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(time(18, 12, 15, 123120, tz = timeZone)) + checkStringToTimestamp("T18:12:15.12312+7:30", expected) + + timeZone = TimeZone.getTimeZone("GMT+07:30") + expected = Option(time(18, 12, 15, 123120, tz = timeZone)) + checkStringToTimestamp("18:12:15.12312+7:30", expected) + + expected = Option(date(2011, 5, 6, 7, 8, 9, 100000, tz = tz)) + checkStringToTimestamp("2011-05-06 07:08:09.1000", expected) checkStringToTimestamp("238", None) checkStringToTimestamp("00238", None) @@ -336,13 +244,15 @@ class DateTimeUtilsSuite extends SparkFunSuite { checkStringToTimestamp("2015-03-18T12:03.17-20:0", None) checkStringToTimestamp("2015-03-18T12:03.17-0:70", None) checkStringToTimestamp("2015-03-18T12:03.17-1:0:0", None) + checkStringToTimestamp("1999 08 01", None) + checkStringToTimestamp("1999-08 01", None) + checkStringToTimestamp("1999 08", None) // Truncating the fractional seconds - c = Calendar.getInstance(TimeZone.getTimeZone("GMT+00:00")) - c.set(2015, 2, 18, 12, 3, 17) - c.set(Calendar.MILLISECOND, 0) + timeZone = TimeZone.getTimeZone("GMT+00:00") + expected = Option(date(2015, 3, 18, 12, 3, 17, 123456, tz = timeZone)) checkStringToTimestamp( - "2015-03-18T12:03:17.123456789+0:00", Option(c.getTimeInMillis * 1000 + 123456)) + "2015-03-18T12:03:17.123456789+0:00", expected) } } @@ -358,43 +268,40 @@ class DateTimeUtilsSuite extends SparkFunSuite { // Test stringToTimestamp assert(stringToTimestamp( - UTF8String.fromString("2015-02-29 00:00:00")).isEmpty) + UTF8String.fromString("2015-02-29 00:00:00"), defaultZoneId).isEmpty) assert(stringToTimestamp( - UTF8String.fromString("2015-04-31 00:00:00")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-02-29")).isEmpty) - assert(stringToTimestamp(UTF8String.fromString("2015-04-31")).isEmpty) + UTF8String.fromString("2015-04-31 00:00:00"), defaultZoneId).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-02-29"), defaultZoneId).isEmpty) + assert(stringToTimestamp(UTF8String.fromString("2015-04-31"), defaultZoneId).isEmpty) } test("hours") { - val c = Calendar.getInstance(TimeZonePST) - c.set(2015, 2, 18, 13, 2, 11) - assert(getHours(c.getTimeInMillis * 1000, TimeZonePST) === 13) - assert(getHours(c.getTimeInMillis * 1000, TimeZoneGMT) === 20) - c.set(2015, 12, 8, 2, 7, 9) - assert(getHours(c.getTimeInMillis * 1000, TimeZonePST) === 2) - assert(getHours(c.getTimeInMillis * 1000, TimeZoneGMT) === 10) + var input = date(2015, 3, 18, 13, 2, 11, 0, TimeZonePST) + assert(getHours(input, TimeZonePST) === 13) + assert(getHours(input, TimeZoneGMT) === 20) + input = date(2015, 12, 8, 2, 7, 9, 0, TimeZonePST) + assert(getHours(input, TimeZonePST) === 2) + assert(getHours(input, TimeZoneGMT) === 10) } test("minutes") { - val c = Calendar.getInstance(TimeZonePST) - c.set(2015, 2, 18, 13, 2, 11) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZonePST) === 2) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZoneGMT) === 2) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZone.getTimeZone("Australia/North")) === 32) - c.set(2015, 2, 8, 2, 7, 9) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZonePST) === 7) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZoneGMT) === 7) - assert(getMinutes(c.getTimeInMillis * 1000, TimeZone.getTimeZone("Australia/North")) === 37) + var input = date(2015, 3, 18, 13, 2, 11, 0, TimeZonePST) + assert(getMinutes(input, TimeZonePST) === 2) + assert(getMinutes(input, TimeZoneGMT) === 2) + assert(getMinutes(input, TimeZone.getTimeZone("Australia/North")) === 32) + input = date(2015, 3, 8, 2, 7, 9, 0, TimeZonePST) + assert(getMinutes(input, TimeZonePST) === 7) + assert(getMinutes(input, TimeZoneGMT) === 7) + assert(getMinutes(input, TimeZone.getTimeZone("Australia/North")) === 37) } test("seconds") { - val c = Calendar.getInstance(TimeZonePST) - c.set(2015, 2, 18, 13, 2, 11) - assert(getSeconds(c.getTimeInMillis * 1000, TimeZonePST) === 11) - assert(getSeconds(c.getTimeInMillis * 1000, TimeZoneGMT) === 11) - c.set(2015, 2, 8, 2, 7, 9) - assert(getSeconds(c.getTimeInMillis * 1000, TimeZonePST) === 9) - assert(getSeconds(c.getTimeInMillis * 1000, TimeZoneGMT) === 9) + var input = date(2015, 3, 18, 13, 2, 11, 0, TimeZonePST) + assert(getSeconds(input, TimeZonePST) === 11) + assert(getSeconds(input, TimeZoneGMT) === 11) + input = date(2015, 3, 8, 2, 7, 9, 0, TimeZonePST) + assert(getSeconds(input, TimeZonePST) === 9) + assert(getSeconds(input, TimeZoneGMT) === 9) } test("hours / minutes / seconds") { @@ -408,118 +315,84 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("get day in year") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 77) - c.set(2012, 2, 18, 0, 0, 0) - assert(getDayInYear(getInUTCDays(c.getTimeInMillis)) === 78) + assert(getDayInYear(days(2015, 3, 18)) === 77) + assert(getDayInYear(days(2012, 3, 18)) === 78) + } + + test("day of year calculations for old years") { + assert(getDayInYear(days(1582, 3)) === 60) + + (1000 to 1600 by 10).foreach { year => + // January 1 is the 1st day of year. + assert(getYear(days(year)) === year) + assert(getMonth(days(year, 1)) === 1) + assert(getDayInYear(days(year, 1, 1)) === 1) + + // December 31 is the 1st day of year. + val date = days(year, 12, 31) + assert(getYear(date) === year) + assert(getMonth(date) === 12) + assert(getDayOfMonth(date) === 31) + } } test("get year") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2015) - c.set(2012, 2, 18, 0, 0, 0) - assert(getYear(getInUTCDays(c.getTimeInMillis)) === 2012) + assert(getYear(days(2015, 2, 18)) === 2015) + assert(getYear(days(2012, 2, 18)) === 2012) } test("get quarter") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 1) - c.set(2012, 11, 18, 0, 0, 0) - assert(getQuarter(getInUTCDays(c.getTimeInMillis)) === 4) + assert(getQuarter(days(2015, 2, 18)) === 1) + assert(getQuarter(days(2012, 11, 18)) === 4) } test("get month") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 3) - c.set(2012, 11, 18, 0, 0, 0) - assert(getMonth(getInUTCDays(c.getTimeInMillis)) === 12) + assert(getMonth(days(2015, 3, 18)) === 3) + assert(getMonth(days(2012, 12, 18)) === 12) } test("get day of month") { - val c = Calendar.getInstance() - c.set(2015, 2, 18, 0, 0, 0) - assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 18) - c.set(2012, 11, 24, 0, 0, 0) - assert(getDayOfMonth(getInUTCDays(c.getTimeInMillis)) === 24) + assert(getDayOfMonth(days(2015, 3, 18)) === 18) + assert(getDayOfMonth(days(2012, 12, 24)) === 24) } test("date add months") { - val c1 = Calendar.getInstance() - c1.set(1997, 1, 28, 10, 30, 0) - val days1 = millisToDays(c1.getTimeInMillis) - val c2 = Calendar.getInstance() - c2.set(2000, 1, 29) - assert(dateAddMonths(days1, 36) === millisToDays(c2.getTimeInMillis)) - c2.set(1996, 0, 31) - assert(dateAddMonths(days1, -13) === millisToDays(c2.getTimeInMillis)) + val input = days(1997, 2, 28, 10, 30) + assert(dateAddMonths(input, 36) === days(2000, 2, 28)) + assert(dateAddMonths(input, -13) === days(1996, 1, 28)) } test("timestamp add months") { - val c1 = Calendar.getInstance() - c1.set(1997, 1, 28, 10, 30, 0) - c1.set(Calendar.MILLISECOND, 0) - val ts1 = c1.getTimeInMillis * 1000L - val c2 = Calendar.getInstance() - c2.set(2000, 1, 29, 10, 30, 0) - c2.set(Calendar.MILLISECOND, 123) - val ts2 = c2.getTimeInMillis * 1000L - assert(timestampAddInterval(ts1, 36, 123000) === ts2) - - val c3 = Calendar.getInstance(TimeZonePST) - c3.set(1997, 1, 27, 16, 0, 0) - c3.set(Calendar.MILLISECOND, 0) - val ts3 = c3.getTimeInMillis * 1000L - val c4 = Calendar.getInstance(TimeZonePST) - c4.set(2000, 1, 27, 16, 0, 0) - c4.set(Calendar.MILLISECOND, 123) - val ts4 = c4.getTimeInMillis * 1000L - val c5 = Calendar.getInstance(TimeZoneGMT) - c5.set(2000, 1, 29, 0, 0, 0) - c5.set(Calendar.MILLISECOND, 123) - val ts5 = c5.getTimeInMillis * 1000L - assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST) === ts4) - assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT) === ts5) + val ts1 = date(1997, 2, 28, 10, 30, 0) + val ts2 = date(2000, 2, 28, 10, 30, 0, 123000) + assert(timestampAddInterval(ts1, 36, 123000, defaultZoneId) === ts2) + + val ts3 = date(1997, 2, 27, 16, 0, 0, 0, TimeZonePST) + val ts4 = date(2000, 2, 27, 16, 0, 0, 123000, TimeZonePST) + val ts5 = date(2000, 2, 28, 0, 0, 0, 123000, TimeZoneGMT) + assert(timestampAddInterval(ts3, 36, 123000, TimeZonePST.toZoneId) === ts4) + assert(timestampAddInterval(ts3, 36, 123000, TimeZoneGMT.toZoneId) === ts5) } test("monthsBetween") { - val c1 = Calendar.getInstance() - c1.set(1997, 1, 28, 10, 30, 0) - val c2 = Calendar.getInstance() - c2.set(1996, 9, 30, 0, 0, 0) - assert(monthsBetween( - c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L, true, c1.getTimeZone) === 3.94959677) - assert(monthsBetween( - c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L, false, c1.getTimeZone) - === 3.9495967741935485) + val date1 = date(1997, 2, 28, 10, 30, 0) + var date2 = date(1996, 10, 30) + assert(monthsBetween(date1, date2, true, TimeZoneUTC) === 3.94959677) + assert(monthsBetween(date1, date2, false, TimeZoneUTC) === 3.9495967741935485) Seq(true, false).foreach { roundOff => - c2.set(2000, 1, 28, 0, 0, 0) - assert(monthsBetween( - c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L, roundOff, c1.getTimeZone) === -36) - c2.set(2000, 1, 29, 0, 0, 0) - assert(monthsBetween( - c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L, roundOff, c1.getTimeZone) === -36) - c2.set(1996, 2, 31, 0, 0, 0) - assert(monthsBetween( - c1.getTimeInMillis * 1000L, c2.getTimeInMillis * 1000L, roundOff, c1.getTimeZone) === 11) + date2 = date(2000, 2, 28) + assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + date2 = date(2000, 2, 29) + assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === -36) + date2 = date(1996, 3, 31) + assert(monthsBetween(date1, date2, roundOff, TimeZoneUTC) === 11) } - val c3 = Calendar.getInstance(TimeZonePST) - c3.set(2000, 1, 28, 16, 0, 0) - val c4 = Calendar.getInstance(TimeZonePST) - c4.set(1997, 1, 28, 16, 0, 0) - assert( - monthsBetween(c3.getTimeInMillis * 1000L, c4.getTimeInMillis * 1000L, true, TimeZonePST) - === 36.0) - assert( - monthsBetween(c3.getTimeInMillis * 1000L, c4.getTimeInMillis * 1000L, true, TimeZoneGMT) - === 35.90322581) - assert( - monthsBetween(c3.getTimeInMillis * 1000L, c4.getTimeInMillis * 1000L, false, TimeZoneGMT) - === 35.903225806451616) + val date3 = date(2000, 2, 28, 16, tz = TimeZonePST) + val date4 = date(1997, 2, 28, 16, tz = TimeZonePST) + assert(monthsBetween(date3, date4, true, TimeZonePST) === 36.0) + assert(monthsBetween(date3, date4, true, TimeZoneGMT) === 35.90322581) + assert(monthsBetween(date3, date4, false, TimeZoneGMT) === 35.903225806451616) } test("from UTC timestamp") { @@ -527,8 +400,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { assert(toJavaTimestamp(fromUTCTime(fromJavaTimestamp(Timestamp.valueOf(utc)), tz)).toString === expected) } - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { - DateTimeTestUtils.withDefaultTimeZone(tz) { + 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") @@ -536,7 +409,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { } } - DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + withDefaultTimeZone(TimeZone.getTimeZone("PST")) { // 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") @@ -552,8 +425,8 @@ class DateTimeUtilsSuite extends SparkFunSuite { === expected) } - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { - DateTimeTestUtils.withDefaultTimeZone(tz) { + 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 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") @@ -561,7 +434,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { } } - DateTimeTestUtils.withDefaultTimeZone(TimeZone.getTimeZone("PST")) { + withDefaultTimeZone(TimeZone.getTimeZone("PST")) { // 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 @@ -584,20 +457,20 @@ class DateTimeUtilsSuite extends SparkFunSuite { val truncated = DateTimeUtils.truncTimestamp(inputTS, level, timezone) val expectedTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected)) + DateTimeUtils.stringToTimestamp(UTF8String.fromString(expected), defaultZoneId) assert(truncated === expectedTS.get) } - val defaultInputTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359")) - val defaultInputTS1 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359")) - val defaultInputTS2 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359")) - val defaultInputTS3 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359")) - val defaultInputTS4 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359")) + val defaultInputTS = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId) + val defaultInputTS1 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-31T20:32:05.359"), defaultZoneId) + val defaultInputTS2 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-04-01T02:32:05.359"), defaultZoneId) + val defaultInputTS3 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-30T02:32:05.359"), defaultZoneId) + val defaultInputTS4 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-29T02:32:05.359"), defaultZoneId) testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", defaultInputTS.get) testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", defaultInputTS.get) @@ -614,18 +487,18 @@ class DateTimeUtilsSuite extends SparkFunSuite { testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-01-01T00:00:00", defaultInputTS1.get) testTrunc(DateTimeUtils.TRUNC_TO_QUARTER, "2015-04-01T00:00:00", defaultInputTS2.get) - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { - DateTimeTestUtils.withDefaultTimeZone(tz) { - val inputTS = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-05T09:32:05.359")) - val inputTS1 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-31T20:32:05.359")) - val inputTS2 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-04-01T02:32:05.359")) - val inputTS3 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-30T02:32:05.359")) - val inputTS4 = - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2015-03-29T02:32:05.359")) + for (tz <- ALL_TIMEZONES) { + withDefaultTimeZone(tz) { + val inputTS = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-05T09:32:05.359"), defaultZoneId) + val inputTS1 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-31T20:32:05.359"), defaultZoneId) + val inputTS2 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-04-01T02:32:05.359"), defaultZoneId) + val inputTS3 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-30T02:32:05.359"), defaultZoneId) + val inputTS4 = DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2015-03-29T02:32:05.359"), defaultZoneId) testTrunc(DateTimeUtils.TRUNC_TO_YEAR, "2015-01-01T00:00:00", inputTS.get, tz) testTrunc(DateTimeUtils.TRUNC_TO_MONTH, "2015-03-01T00:00:00", inputTS.get, tz) @@ -646,20 +519,16 @@ class DateTimeUtilsSuite extends SparkFunSuite { } test("daysToMillis and millisToDays") { - val c = Calendar.getInstance(TimeZonePST) - - c.set(2015, 11, 31, 16, 0, 0) - assert(millisToDays(c.getTimeInMillis, TimeZonePST) === 16800) - assert(millisToDays(c.getTimeInMillis, TimeZoneGMT) === 16801) + val input = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, 16, tz = TimeZonePST)) + assert(millisToDays(input, TimeZonePST) === 16800) + assert(millisToDays(input, TimeZoneGMT) === 16801) + assert(millisToDays(-1 * MILLIS_PER_DAY + 1, TimeZoneGMT) == -1) - c.set(2015, 11, 31, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(daysToMillis(16800, TimeZonePST) === c.getTimeInMillis) + var expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZonePST)) + assert(daysToMillis(16800, TimeZonePST) === expected) - c.setTimeZone(TimeZoneGMT) - c.set(2015, 11, 31, 0, 0, 0) - c.set(Calendar.MILLISECOND, 0) - assert(daysToMillis(16800, TimeZoneGMT) === c.getTimeInMillis) + expected = TimeUnit.MICROSECONDS.toMillis(date(2015, 12, 31, tz = TimeZoneGMT)) + assert(daysToMillis(16800, TimeZoneGMT) === expected) // There are some days are skipped entirely in some timezone, skip them here. val skipped_days = Map[String, Set[Int]]( @@ -670,7 +539,7 @@ class DateTimeUtilsSuite extends SparkFunSuite { "Pacific/Kiritimati" -> Set(9130, 9131), "Pacific/Kwajalein" -> Set(8632), "MIT" -> Set(15338)) - for (tz <- DateTimeTestUtils.ALL_TIMEZONES) { + for (tz <- ALL_TIMEZONES) { val skipped = skipped_days.getOrElse(tz.getID, Set.empty) (-20000 to 20000).foreach { d => if (!skipped.contains(d)) { @@ -680,4 +549,9 @@ class DateTimeUtilsSuite extends SparkFunSuite { } } } + + test("toMillis") { + assert(DateTimeUtils.toMillis(-9223372036844776001L) === -9223372036844777L) + assert(DateTimeUtils.toMillis(-157700927876544L) === -157700927877L) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala index a0c1d97bfc3a8..a16fcfc850ead 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/MetadataSuite.scala @@ -59,29 +59,29 @@ class MetadataSuite extends SparkFunSuite { test("metadata builder and getters") { assert(age.contains("summary") === false) - assert(age.contains("index") === true) + assert(age.contains("index")) assert(age.getLong("index") === 1L) - assert(age.contains("average") === true) + assert(age.contains("average")) assert(age.getDouble("average") === 45.0) - assert(age.contains("categorical") === true) + assert(age.contains("categorical")) assert(age.getBoolean("categorical") === false) - assert(age.contains("name") === true) + assert(age.contains("name")) assert(age.getString("name") === "age") - assert(metadata.contains("purpose") === true) + assert(metadata.contains("purpose")) assert(metadata.getString("purpose") === "ml") - assert(metadata.contains("isBase") === true) + assert(metadata.contains("isBase")) assert(metadata.getBoolean("isBase") === false) - assert(metadata.contains("summary") === true) + assert(metadata.contains("summary")) assert(metadata.getMetadata("summary") === summary) - assert(metadata.contains("long[]") === true) + assert(metadata.contains("long[]")) assert(metadata.getLongArray("long[]").toSeq === Seq(0L, 1L)) - assert(metadata.contains("double[]") === true) + assert(metadata.contains("double[]")) assert(metadata.getDoubleArray("double[]").toSeq === Seq(3.0, 4.0)) - assert(metadata.contains("boolean[]") === true) + assert(metadata.contains("boolean[]")) assert(metadata.getBooleanArray("boolean[]").toSeq === Seq(true, false)) - assert(gender.contains("categories") === true) + assert(gender.contains("categories")) assert(gender.getStringArray("categories").toSeq === Seq("male", "female")) - assert(metadata.contains("features") === true) + assert(metadata.contains("features")) assert(metadata.getMetadataArray("features").toSeq === Seq(age, gender)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala index 13265a1ff1c7f..ec73f4518737d 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/NumberConverterSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark.sql.catalyst.util +import java.nio.ByteBuffer +import java.nio.ByteOrder.BIG_ENDIAN + import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.NumberConverter.convert +import org.apache.spark.sql.catalyst.util.NumberConverter.{convert, toBinary} import org.apache.spark.unsafe.types.UTF8String class NumberConverterSuite extends SparkFunSuite { @@ -37,4 +40,47 @@ class NumberConverterSuite extends SparkFunSuite { checkConv("11abc", 10, 16, "B") } + test("byte to binary") { + checkToBinary(0.toByte) + checkToBinary(1.toByte) + checkToBinary(-1.toByte) + checkToBinary(Byte.MaxValue) + checkToBinary(Byte.MinValue) + } + + test("short to binary") { + checkToBinary(0.toShort) + checkToBinary(1.toShort) + checkToBinary(-1.toShort) + checkToBinary(Short.MaxValue) + checkToBinary(Short.MinValue) + } + + test("integer to binary") { + checkToBinary(0) + checkToBinary(1) + checkToBinary(-1) + checkToBinary(Int.MaxValue) + checkToBinary(Int.MinValue) + } + + test("long to binary") { + checkToBinary(0L) + checkToBinary(1L) + checkToBinary(-1L) + checkToBinary(Long.MaxValue) + checkToBinary(Long.MinValue) + } + + def checkToBinary[T](in: T): Unit = in match { + case b: Byte => + assert(toBinary(b) === ByteBuffer.allocate(1).order(BIG_ENDIAN).put(b).array()) + case s: Short => + assert(toBinary(s) === ByteBuffer.allocate(2).order(BIG_ENDIAN).putShort(s).array()) + case i: Int => + assert(toBinary(i) === ByteBuffer.allocate(4).order(BIG_ENDIAN).putInt(i).array()) + case l: Long => + assert(toBinary(l) === ByteBuffer.allocate(8).order(BIG_ENDIAN).putLong(l).array()) + } + } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala index 78fee5135c3ae..63d3831404d47 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/StringUtilsSuite.scala @@ -43,4 +43,38 @@ class StringUtilsSuite extends SparkFunSuite { assert(filterPattern(names, " a. ") === Seq("a1", "a2")) assert(filterPattern(names, " d* ") === Nil) } + + test("string concatenation") { + def concat(seq: String*): String = { + seq.foldLeft(new StringConcat()) { (acc, s) => acc.append(s); acc }.toString + } + + assert(new StringConcat().toString == "") + assert(concat("") === "") + assert(concat(null) === "") + assert(concat("a") === "a") + assert(concat("1", "2") === "12") + assert(concat("abc", "\n", "123") === "abc\n123") + } + + test("string concatenation with limit") { + def concat(seq: String*): String = { + seq.foldLeft(new StringConcat(7)) { (acc, s) => acc.append(s); acc }.toString + } + assert(concat("under") === "under") + assert(concat("under", "over", "extra") === "underov") + assert(concat("underover") === "underov") + assert(concat("under", "ov") === "underov") + } + + test("string concatenation return value") { + def checkLimit(s: String): Boolean = { + val sc = new StringConcat(7) + sc.append(s) + sc.atLimit + } + assert(!checkLimit("under")) + assert(checkLimit("1234567")) + assert(checkLimit("1234567890")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala index bc6852ca7e1fd..d6d1e418d74ea 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/TypeUtilsSuite.scala @@ -43,4 +43,22 @@ class TypeUtilsSuite extends SparkFunSuite { typeCheckPass(ArrayType(StringType, containsNull = true) :: ArrayType(StringType, containsNull = false) :: Nil) } + + test("compareBinary") { + val x1 = Array[Byte]() + val y1 = Array(1, 2, 3).map(_.toByte) + assert(TypeUtils.compareBinary(x1, y1) < 0) + + val x2 = Array(200, 100).map(_.toByte) + val y2 = Array(100, 100).map(_.toByte) + assert(TypeUtils.compareBinary(x2, y2) > 0) + + val x3 = Array(100, 200, 12).map(_.toByte) + val y3 = Array(100, 200).map(_.toByte) + assert(TypeUtils.compareBinary(x3, y3) > 0) + + val x4 = Array(100, 200).map(_.toByte) + val y4 = Array(100, 200).map(_.toByte) + assert(TypeUtils.compareBinary(x4, y4) == 0) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala index 755c8897cada2..61ce63faf0d26 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeArraySuite.scala @@ -17,11 +17,15 @@ package org.apache.spark.sql.catalyst.util -import org.apache.spark.SparkFunSuite +import java.time.ZoneId + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} import org.apache.spark.sql.catalyst.expressions.UnsafeArrayData import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.Platform import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class UnsafeArraySuite extends SparkFunSuite { @@ -36,9 +40,13 @@ class UnsafeArraySuite extends SparkFunSuite { val dateArray = Array( DateTimeUtils.stringToDate(UTF8String.fromString("1970-1-1")).get, DateTimeUtils.stringToDate(UTF8String.fromString("2016-7-26")).get) + private def defaultTz = DateTimeUtils.defaultTimeZone() + private def defaultZoneId = ZoneId.systemDefault() val timestampArray = Array( - DateTimeUtils.stringToTimestamp(UTF8String.fromString("1970-1-1 00:00:00")).get, - DateTimeUtils.stringToTimestamp(UTF8String.fromString("2016-7-26 00:00:00")).get) + DateTimeUtils.stringToTimestamp( + UTF8String.fromString("1970-1-1 00:00:00"), defaultZoneId).get, + DateTimeUtils.stringToTimestamp( + UTF8String.fromString("2016-7-26 00:00:00"), defaultZoneId).get) val decimalArray4_1 = Array( BigDecimal("123.4").setScale(1, BigDecimal.RoundingMode.FLOOR), BigDecimal("567.8").setScale(1, BigDecimal.RoundingMode.FLOOR)) @@ -52,6 +60,16 @@ class UnsafeArraySuite extends SparkFunSuite { val doubleMultiDimArray = Array( Array(1.1, 11.1), Array(2.2, 22.2, 222.2), Array(3.3, 33.3, 333.3, 3333.3)) + val serialArray = { + val offset = 32 + val data = new Array[Byte](1024) + Platform.putLong(data, offset, 1) + val arrayData = new UnsafeArrayData() + arrayData.pointTo(data, offset, data.length) + arrayData.setLong(0, 19285) + arrayData + } + test("read array") { val unsafeBoolean = ExpressionEncoder[Array[Boolean]].resolveAndBind(). toRow(booleanArray).getArray(0) @@ -204,4 +222,18 @@ class UnsafeArraySuite extends SparkFunSuite { val doubleEncoder = ExpressionEncoder[Array[Double]].resolveAndBind() assert(doubleEncoder.toRow(doubleArray).getArray(0).toDoubleArray.sameElements(doubleArray)) } + + test("unsafe java serialization") { + val ser = new JavaSerializer(new SparkConf).newInstance() + val arrayDataSer = ser.deserialize[UnsafeArrayData](ser.serialize(serialArray)) + assert(arrayDataSer.getLong(0) == 19285) + assert(arrayDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) + } + + test("unsafe Kryo serialization") { + val ser = new KryoSerializer(new SparkConf).newInstance() + val arrayDataSer = ser.deserialize[UnsafeArrayData](ser.serialize(serialArray)) + assert(arrayDataSer.getLong(0) == 19285) + assert(arrayDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala new file mode 100644 index 0000000000000..ebc88612be22a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/util/UnsafeMapSuite.scala @@ -0,0 +1,64 @@ +/* + * 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.{SparkConf, SparkFunSuite} +import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} +import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData} +import org.apache.spark.unsafe.Platform + +class UnsafeMapSuite extends SparkFunSuite { + + val unsafeMapData = { + val offset = 32 + val keyArraySize = 256 + val baseObject = new Array[Byte](1024) + Platform.putLong(baseObject, offset, keyArraySize) + + val unsafeMap = new UnsafeMapData + Platform.putLong(baseObject, offset + 8, 1) + val keyArray = new UnsafeArrayData() + keyArray.pointTo(baseObject, offset + 8, keyArraySize) + keyArray.setLong(0, 19285) + + val valueArray = new UnsafeArrayData() + Platform.putLong(baseObject, offset + 8 + keyArray.getSizeInBytes, 1) + valueArray.pointTo(baseObject, offset + 8 + keyArray.getSizeInBytes, keyArraySize) + valueArray.setLong(0, 19286) + unsafeMap.pointTo(baseObject, offset, baseObject.length) + unsafeMap + } + + test("unsafe java serialization") { + val ser = new JavaSerializer(new SparkConf).newInstance() + val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) + assert(mapDataSer.numElements() == 1) + assert(mapDataSer.keyArray().getInt(0) == 19285) + assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) + } + + test("unsafe Kryo serialization") { + val ser = new KryoSerializer(new SparkConf).newInstance() + val mapDataSer = ser.deserialize[UnsafeMapData](ser.serialize(unsafeMapData)) + assert(mapDataSer.numElements() == 1) + assert(mapDataSer.keyArray().getInt(0) == 19285) + assert(mapDataSer.valueArray().getInt(0) == 19286) + assert(mapDataSer.getBaseObject.asInstanceOf[Array[Byte]].length == 1024) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala index d92f52f3248aa..6b5fc5f0d4434 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DataTypeWriteCompatibilitySuite.scala @@ -67,7 +67,7 @@ class DataTypeWriteCompatibilitySuite extends SparkFunSuite { test("Check atomic types: write allowed only when casting is safe") { atomicTypes.foreach { w => atomicTypes.foreach { r => - if (Cast.canSafeCast(w, r)) { + if (Cast.canUpCast(w, r)) { assertAllowed(w, r, "t", s"Should allow writing $w to $r because cast is safe") } else { @@ -189,6 +189,14 @@ class DataTypeWriteCompatibilitySuite extends SparkFunSuite { "Should allow widening float fields x and y to double") } + test("Check struct type: ignore field name mismatch with byPosition mode") { + val nameMismatchFields = StructType(Seq( + StructField("a", FloatType, nullable = false), + StructField("b", FloatType, nullable = false))) + assertAllowed(nameMismatchFields, point2, "t", + "Should allow field name mismatch with byPosition mode", false) + } + ignore("Check struct types: missing optional field is allowed") { // built-in data sources do not yet support missing fields when optional assertAllowed(point2, point3, "t", @@ -370,10 +378,15 @@ class DataTypeWriteCompatibilitySuite extends SparkFunSuite { // Helper functions - def assertAllowed(writeType: DataType, readType: DataType, name: String, desc: String): Unit = { + def assertAllowed( + writeType: DataType, + readType: DataType, + name: String, + desc: String, + byName: Boolean = true): Unit = { assert( - DataType.canWrite(writeType, readType, analysis.caseSensitiveResolution, name, - errMsg => fail(s"Should not produce errors but was called with: $errMsg")) === true, desc) + DataType.canWrite(writeType, readType, byName, analysis.caseSensitiveResolution, name, + errMsg => fail(s"Should not produce errors but was called with: $errMsg")), desc) } def assertSingleError( @@ -392,13 +405,14 @@ class DataTypeWriteCompatibilitySuite extends SparkFunSuite { readType: DataType, name: String, desc: String, - numErrs: Int) - (errFunc: Seq[String] => Unit): Unit = { + numErrs: Int, + byName: Boolean = true) + (checkErrors: Seq[String] => Unit): Unit = { val errs = new mutable.ArrayBuffer[String]() assert( - DataType.canWrite(writeType, readType, analysis.caseSensitiveResolution, name, + DataType.canWrite(writeType, readType, byName, analysis.caseSensitiveResolution, name, errMsg => errs += errMsg) === false, desc) assert(errs.size === numErrs, s"Should produce $numErrs error messages") - errFunc(errs) + checkErrors(errs) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala index 10de90c6a44ca..d69bb2f0b6bcc 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/DecimalSuite.scala @@ -56,11 +56,11 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { checkDecimal(Decimal(1000000000000000000L, 20, 2), "10000000000000000.00", 20, 2) checkDecimal(Decimal(Long.MaxValue), Long.MaxValue.toString, 20, 0) checkDecimal(Decimal(Long.MinValue), Long.MinValue.toString, 20, 0) - intercept[IllegalArgumentException](Decimal(170L, 2, 1)) - intercept[IllegalArgumentException](Decimal(170L, 2, 0)) - intercept[IllegalArgumentException](Decimal(BigDecimal("10.030"), 2, 1)) - intercept[IllegalArgumentException](Decimal(BigDecimal("-9.95"), 2, 1)) - intercept[IllegalArgumentException](Decimal(1e17.toLong, 17, 0)) + intercept[ArithmeticException](Decimal(170L, 2, 1)) + intercept[ArithmeticException](Decimal(170L, 2, 0)) + intercept[ArithmeticException](Decimal(BigDecimal("10.030"), 2, 1)) + intercept[ArithmeticException](Decimal(BigDecimal("-9.95"), 2, 1)) + intercept[ArithmeticException](Decimal(1e17.toLong, 17, 0)) } test("creating decimals with negative scale") { @@ -228,4 +228,15 @@ class DecimalSuite extends SparkFunSuite with PrivateMethodTester { val decimal = Decimal.apply(bigInt) assert(decimal.toJavaBigDecimal.unscaledValue.toString === "9223372036854775808") } + + test("SPARK-26038: toScalaBigInt/toJavaBigInteger") { + // not fitting long + val decimal = Decimal("1234568790123456789012348790.1234879012345678901234568790") + assert(decimal.toScalaBigInt == scala.math.BigInt("1234568790123456789012348790")) + assert(decimal.toJavaBigInteger == new java.math.BigInteger("1234568790123456789012348790")) + // fitting long + val decimalLong = Decimal(123456789123456789L, 18, 9) + assert(decimalLong.toScalaBigInt == scala.math.BigInt("123456789")) + assert(decimalLong.toJavaBigInteger == new java.math.BigInteger("123456789")) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala index 210e65708170f..c3ae798e85fa7 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/MetadataSuite.scala @@ -26,6 +26,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getString("key") === "value") assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getString("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -36,6 +37,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getLong("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -46,6 +48,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.## !== 0) assert(meta.getDouble("key") === 12) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getDouble("no_such_key")) intercept[ClassCastException](meta.getBoolean("key")) } @@ -54,8 +57,9 @@ class MetadataSuite extends SparkFunSuite { val meta = new MetadataBuilder().putBoolean("key", true).build() assert(meta === meta) assert(meta.## !== 0) - assert(meta.getBoolean("key") === true) + assert(meta.getBoolean("key")) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getBoolean("no_such_key")) intercept[ClassCastException](meta.getString("key")) } @@ -69,6 +73,7 @@ class MetadataSuite extends SparkFunSuite { assert(meta.getLong("key") === 0) assert(meta.getBoolean("key") === false) assert(meta.contains("key")) + assert(meta === Metadata.fromJson(meta.json)) intercept[NoSuchElementException](meta.getLong("no_such_key")) } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala index 53a78c94aa6fb..c49308838a19e 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/StructTypeSuite.scala @@ -22,21 +22,21 @@ import org.apache.spark.sql.types.StructType.fromDDL class StructTypeSuite extends SparkFunSuite { - val s = StructType.fromDDL("a INT, b STRING") + private val s = StructType.fromDDL("a INT, b STRING") test("lookup a single missing field should output existing fields") { val e = intercept[IllegalArgumentException](s("c")).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("lookup a set of missing fields should output existing fields") { val e = intercept[IllegalArgumentException](s(Set("a", "c"))).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("lookup fieldIndex for missing field should output existing fields") { val e = intercept[IllegalArgumentException](s.fieldIndex("c")).getMessage - assert(e.contains("Available fields: a, b")) + assert(e.contains("Available: a, b")) } test("SPARK-24849: toDDL - simple struct") { @@ -70,4 +70,16 @@ class StructTypeSuite extends SparkFunSuite { assert(struct.toDDL == """`b` BOOLEAN COMMENT 'Field\'s comment'""") } + + + test("Print up to the given level") { + val schema = StructType.fromDDL( + "c1 INT, c2 STRUCT>") + + assert(5 == schema.treeString(2).split("\n").length) + assert(3 == schema.treeString(1).split("\n").length) + assert(7 == schema.treeString.split("\n").length) + assert(7 == schema.treeString(0).split("\n").length) + assert(7 == schema.treeString(-1).split("\n").length) + } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala new file mode 100644 index 0000000000000..1be8ee9dfa92b --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/types/TestUDT.scala @@ -0,0 +1,61 @@ +/* + * 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.types + +import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} + + +// Wrapped in an object to check Scala compatibility. See SPARK-13929 +object TestUDT { + + @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) + private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { + override def hashCode(): Int = java.util.Arrays.hashCode(data) + + override def equals(other: Any): Boolean = other match { + case v: MyDenseVector => java.util.Arrays.equals(this.data, v.data) + case _ => false + } + + override def toString: String = data.mkString("(", ", ", ")") + } + + private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { + + override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) + + override def serialize(features: MyDenseVector): ArrayData = { + new GenericArrayData(features.data.map(_.asInstanceOf[Any])) + } + + override def deserialize(datum: Any): MyDenseVector = { + datum match { + case data: ArrayData => + new MyDenseVector(data.toDoubleArray()) + } + } + + override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] + + private[spark] override def asNullable: MyDenseVectorUDT = this + + override def hashCode(): Int = getClass.hashCode() + + override def equals(other: Any): Boolean = other.isInstanceOf[MyDenseVectorUDT] + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala index d801f62b62323..4439a7bb3ae87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/arrow/ArrowUtilsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/ArrowUtilsSuite.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql.execution.arrow +package org.apache.spark.sql.util -import org.apache.arrow.vector.types.pojo.{ArrowType, Field, FieldType, Schema} +import org.apache.arrow.vector.types.pojo.ArrowType import org.apache.spark.SparkFunSuite import org.apache.spark.sql.catalyst.util.DateTimeUtils diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceOptionsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala similarity index 53% rename from sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceOptionsSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala index cfa69a86de1a7..0accb471cada3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceOptionsSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/CaseInsensitiveStringMapSuite.scala @@ -15,31 +15,38 @@ * limitations under the License. */ -package org.apache.spark.sql.sources.v2 +package org.apache.spark.sql.util + +import java.util import scala.collection.JavaConverters._ import org.apache.spark.SparkFunSuite -/** - * A simple test suite to verify `DataSourceOptions`. - */ -class DataSourceOptionsSuite extends SparkFunSuite { +class CaseInsensitiveStringMapSuite extends SparkFunSuite { + + test("put and get") { + val options = CaseInsensitiveStringMap.empty() + intercept[UnsupportedOperationException] { + options.put("kEy", "valUE") + } + } - test("key is case-insensitive") { - val options = new DataSourceOptions(Map("foo" -> "bar").asJava) - assert(options.get("foo").get() == "bar") - assert(options.get("FoO").get() == "bar") - assert(!options.get("abc").isPresent) + test("clear") { + val options = new CaseInsensitiveStringMap(Map("kEy" -> "valUE").asJava) + intercept[UnsupportedOperationException] { + options.clear() + } } - test("value is case-sensitive") { - val options = new DataSourceOptions(Map("foo" -> "bAr").asJava) - assert(options.get("foo").get == "bAr") + test("key and value set") { + val options = new CaseInsensitiveStringMap(Map("kEy" -> "valUE").asJava) + assert(options.keySet().asScala == Set("key")) + assert(options.values().asScala.toSeq == Seq("valUE")) } test("getInt") { - val options = new DataSourceOptions(Map("numFOo" -> "1", "foo" -> "bar").asJava) + val options = new CaseInsensitiveStringMap(Map("numFOo" -> "1", "foo" -> "bar").asJava) assert(options.getInt("numFOO", 10) == 1) assert(options.getInt("numFOO2", 10) == 10) @@ -49,17 +56,20 @@ class DataSourceOptionsSuite extends SparkFunSuite { } test("getBoolean") { - val options = new DataSourceOptions( + val options = new CaseInsensitiveStringMap( Map("isFoo" -> "true", "isFOO2" -> "false", "foo" -> "bar").asJava) assert(options.getBoolean("isFoo", false)) assert(!options.getBoolean("isFoo2", true)) assert(options.getBoolean("isBar", true)) assert(!options.getBoolean("isBar", false)) - assert(!options.getBoolean("FOO", true)) + + intercept[IllegalArgumentException] { + options.getBoolean("FOO", true) + } } test("getLong") { - val options = new DataSourceOptions(Map("numFoo" -> "9223372036854775807", + val options = new CaseInsensitiveStringMap(Map("numFoo" -> "9223372036854775807", "foo" -> "bar").asJava) assert(options.getLong("numFOO", 0L) == 9223372036854775807L) assert(options.getLong("numFoo2", -1L) == -1L) @@ -70,7 +80,7 @@ class DataSourceOptionsSuite extends SparkFunSuite { } test("getDouble") { - val options = new DataSourceOptions(Map("numFoo" -> "922337.1", + val options = new CaseInsensitiveStringMap(Map("numFoo" -> "922337.1", "foo" -> "bar").asJava) assert(options.getDouble("numFOO", 0d) == 922337.1d) assert(options.getDouble("numFoo2", -1.02d) == -1.02d) @@ -80,28 +90,19 @@ class DataSourceOptionsSuite extends SparkFunSuite { } } - test("standard options") { - val options = new DataSourceOptions(Map( - DataSourceOptions.PATH_KEY -> "abc", - DataSourceOptions.TABLE_KEY -> "tbl").asJava) - - assert(options.paths().toSeq == Seq("abc")) - assert(options.tableName().get() == "tbl") - assert(!options.databaseName().isPresent) - } - - test("standard options with both singular path and multi-paths") { - val options = new DataSourceOptions(Map( - DataSourceOptions.PATH_KEY -> "abc", - DataSourceOptions.PATHS_KEY -> """["c", "d"]""").asJava) - - assert(options.paths().toSeq == Seq("abc", "c", "d")) - } - - test("standard options with only multi-paths") { - val options = new DataSourceOptions(Map( - DataSourceOptions.PATHS_KEY -> """["c", "d\"e"]""").asJava) + test("asCaseSensitiveMap") { + val originalMap = new util.HashMap[String, String] { + put("Foo", "Bar") + put("OFO", "ABR") + put("OoF", "bar") + } - assert(options.paths().toSeq == Seq("c", "d\"e")) + val options = new CaseInsensitiveStringMap(originalMap) + val caseSensitiveMap = options.asCaseSensitiveMap + assert(caseSensitiveMap.equals(originalMap)) + // The result of `asCaseSensitiveMap` is read-only. + intercept[UnsupportedOperationException] { + caseSensitiveMap.put("kEy", "valUE") + } } } 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 new file mode 100644 index 0000000000000..602542fb33db0 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/DateFormatterSuite.scala @@ -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.sql.util + +import java.time.LocalDate + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.internal.SQLConf + +class DateFormatterSuite extends SparkFunSuite with SQLHelper { + test("parsing dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val daysSinceEpoch = formatter.parse("2018-12-02") + assert(daysSinceEpoch === 17867) + } + } + } + + test("format dates") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val date = formatter.format(17867) + assert(date === "2018-12-02") + } + } + } + + test("roundtrip date -> days -> date") { + Seq( + "0050-01-01", + "0953-02-02", + "1423-03-08", + "1969-12-31", + "1972-08-25", + "1975-09-26", + "2018-12-12", + "2038-01-01", + "5010-11-17").foreach { date => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val days = formatter.parse(date) + val formatted = formatter.format(days) + assert(date === formatted) + } + } + } + } + + test("roundtrip days -> date -> days") { + Seq( + -701265, + -371419, + -199722, + -1, + 0, + 967, + 2094, + 17877, + 24837, + 1110657).foreach { days => + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + val formatter = DateFormatter() + val date = formatter.format(days) + val parsed = formatter.parse(date) + assert(days === parsed) + } + } + } + } + + test("parsing date without explicit day") { + val formatter = DateFormatter("yyyy MMM") + val daysSinceEpoch = formatter.parse("2018 Dec") + assert(daysSinceEpoch === LocalDate.of(2018, 12, 1).toEpochDay) + } +} 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 new file mode 100644 index 0000000000000..d10c30c14d876 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/TimestampFormatterSuite.scala @@ -0,0 +1,126 @@ +/* + * 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.util + +import java.time.{LocalDateTime, ZoneId, ZoneOffset} +import java.util.concurrent.TimeUnit + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils, TimestampFormatter} + +class TimestampFormatterSuite extends SparkFunSuite with SQLHelper { + + test("parsing timestamps using time zones") { + val localDate = "2018-12-02T10:11:12.001234" + val expectedMicros = Map( + "UTC" -> 1543745472001234L, + "PST" -> 1543774272001234L, + "CET" -> 1543741872001234L, + "Africa/Dakar" -> 1543745472001234L, + "America/Los_Angeles" -> 1543774272001234L, + "Antarctica/Vostok" -> 1543723872001234L, + "Asia/Hong_Kong" -> 1543716672001234L, + "Europe/Amsterdam" -> 1543741872001234L) + DateTimeTestUtils.outstandingTimezonesIds.foreach { zoneId => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + DateTimeUtils.getZoneId(zoneId)) + val microsSinceEpoch = formatter.parse(localDate) + assert(microsSinceEpoch === expectedMicros(zoneId)) + } + } + + test("format timestamps using time zones") { + 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", + "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", + "Asia/Hong_Kong" -> "2018-12-02T18:11:12.001234", + "Europe/Amsterdam" -> "2018-12-02T11:11:12.001234") + DateTimeTestUtils.outstandingTimezonesIds.foreach { zoneId => + val formatter = TimestampFormatter( + "yyyy-MM-dd'T'HH:mm:ss.SSSSSS", + DateTimeUtils.getZoneId(zoneId)) + val timestamp = formatter.format(microsSinceEpoch) + assert(timestamp === expectedTimestamp(zoneId)) + } + } + + test("roundtrip micros -> timestamp -> micros using timezones") { + Seq("yyyy-MM-dd'T'HH:mm:ss.SSSSSS", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSXXXXX").foreach { pattern => + Seq( + -58710115316212000L, + -18926315945345679L, + -9463427405253013L, + -244000001L, + 0L, + 99628200102030L, + 1543749753123456L, + 2177456523456789L, + 11858049903010203L).foreach { micros => + DateTimeTestUtils.outstandingZoneIds.foreach { zoneId => + val formatter = TimestampFormatter(pattern, zoneId) + val timestamp = formatter.format(micros) + val parsed = formatter.parse(timestamp) + assert(micros === parsed) + } + } + } + } + + test("roundtrip timestamp -> micros -> timestamp using timezones") { + Seq( + "0109-07-20T18:38:03.788000", + "1370-04-01T10:00:54.654321", + "1670-02-11T14:09:54.746987", + "1969-12-31T23:55:55.999999", + "1970-01-01T00:00:00.000000", + "1973-02-27T02:30:00.102030", + "2018-12-02T11:22:33.123456", + "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) + assert(timestamp === formatted) + } + } + } + + 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( + LocalDateTime.of(2009, 3, 20, 11, 30, 1).toEpochSecond(ZoneOffset.UTC))) + } + + test("format fraction of second") { + val formatter = TimestampFormatter.getFractionFormatter(ZoneOffset.UTC) + assert(formatter.format(0) === "1970-01-01 00:00:00") + assert(formatter.format(1) === "1970-01-01 00:00:00.000001") + assert(formatter.format(1000) === "1970-01-01 00:00:00.001") + assert(formatter.format(900000) === "1970-01-01 00:00:00.9") + assert(formatter.format(1000000) === "1970-01-01 00:00:01") + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala new file mode 100644 index 0000000000000..d95de71e897a2 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/util/UtilSuite.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.util + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalyst.util.truncatedString + +class UtilSuite extends SparkFunSuite { + test("truncatedString") { + assert(truncatedString(Nil, "[", ", ", "]", 2) == "[]") + assert(truncatedString(Seq(1, 2), "[", ", ", "]", 2) == "[1, 2]") + assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", 2) == "[1, ... 2 more fields]") + assert(truncatedString(Seq(1, 2, 3), "[", ", ", "]", -5) == "[, ... 3 more fields]") + assert(truncatedString(Seq(1, 2, 3), ", ", 10) == "1, 2, 3") + } +} diff --git a/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt new file mode 100644 index 0000000000000..9d656fc10dce4 --- /dev/null +++ b/sql/core/benchmarks/BuiltInDataSourceWriteBenchmark-results.txt @@ -0,0 +1,60 @@ +================================================================================================ +Parquet writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Parquet writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Output Single Int Column 2354 / 2438 6.7 149.7 1.0X +Output Single Double Column 2462 / 2485 6.4 156.5 1.0X +Output Int and String Column 8083 / 8100 1.9 513.9 0.3X +Output Partitions 5015 / 5027 3.1 318.8 0.5X +Output Buckets 6883 / 6887 2.3 437.6 0.3X + + +================================================================================================ +ORC writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +ORC writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Output Single Int Column 1769 / 1789 8.9 112.4 1.0X +Output Single Double Column 1989 / 2009 7.9 126.5 0.9X +Output Int and String Column 7323 / 7400 2.1 465.6 0.2X +Output Partitions 4374 / 4381 3.6 278.1 0.4X +Output Buckets 6086 / 6104 2.6 386.9 0.3X + + +================================================================================================ +JSON writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +JSON writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Output Single Int Column 2954 / 4085 5.3 187.8 1.0X +Output Single Double Column 3832 / 3837 4.1 243.6 0.8X +Output Int and String Column 9591 / 10336 1.6 609.8 0.3X +Output Partitions 4956 / 4994 3.2 315.1 0.6X +Output Buckets 6608 / 6676 2.4 420.1 0.4X + + +================================================================================================ +CSV writer benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +CSV writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Output Single Int Column 4118 / 4125 3.8 261.8 1.0X +Output Single Double Column 4888 / 4891 3.2 310.8 0.8X +Output Int and String Column 9788 / 9872 1.6 622.3 0.4X +Output Partitions 6578 / 6640 2.4 418.2 0.6X +Output Buckets 9125 / 9171 1.7 580.2 0.5X + + diff --git a/sql/core/benchmarks/CSVBenchmark-results.txt b/sql/core/benchmarks/CSVBenchmark-results.txt new file mode 100644 index 0000000000000..888c2ce9f2845 --- /dev/null +++ b/sql/core/benchmarks/CSVBenchmark-results.txt @@ -0,0 +1,59 @@ +================================================================================================ +Benchmark to measure CSV read/write performance +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Parsing quoted values: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +One quoted string 36998 37134 120 0.0 739953.1 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Wide rows with 1000 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 1000 columns 140620 141162 737 0.0 140620.5 1.0X +Select 100 columns 35170 35287 183 0.0 35170.0 4.0X +Select one column 27711 27927 187 0.0 27710.9 5.1X +count() 7707 7804 84 0.1 7707.4 18.2X +Select 100 columns, one bad input field 41762 41851 117 0.0 41761.8 3.4X +Select 100 columns, corrupt record field 48717 48761 44 0.0 48717.4 2.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +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() 16001 16053 53 0.6 1600.1 1.0X +Select 1 column + count() 11571 11614 58 0.9 1157.1 1.4X +count() 4752 4766 18 2.1 475.2 3.4X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 1070 1072 2 9.3 107.0 1.0X +to_csv(timestamp) 10446 10746 344 1.0 1044.6 0.1X +write timestamps to files 9573 9659 101 1.0 957.3 0.1X +Create a dataset of dates 1245 1260 17 8.0 124.5 0.9X +to_csv(date) 7157 7167 11 1.4 715.7 0.1X +write dates to files 5415 5450 57 1.8 541.5 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 1880 1887 8 5.3 188.0 1.0X +read timestamps from files 27135 27180 43 0.4 2713.5 0.1X +infer timestamps from files 51426 51534 97 0.2 5142.6 0.0X +read date text from files 1618 1622 4 6.2 161.8 1.2X +read date from files 20207 20218 13 0.5 2020.7 0.1X +infer date from files 19418 19479 94 0.5 1941.8 0.1X +timestamp strings 2289 2300 13 4.4 228.9 0.8X +parse timestamps from Dataset[String] 29367 29391 24 0.3 2936.7 0.1X +infer timestamps from Dataset[String] 54782 54902 126 0.2 5478.2 0.0X +date strings 2508 2524 16 4.0 250.8 0.7X +parse dates from Dataset[String] 21884 21902 19 0.5 2188.4 0.1X +from_csv(timestamp) 27188 27723 477 0.4 2718.8 0.1X +from_csv(date) 21137 21191 84 0.5 2113.7 0.1X + + diff --git a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt index 2d3bae442cc50..f547f61654b5f 100644 --- a/sql/core/benchmarks/DataSourceReadBenchmark-results.txt +++ b/sql/core/benchmarks/DataSourceReadBenchmark-results.txt @@ -2,268 +2,251 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 21508 / 22112 0.7 1367.5 1.0X -SQL Json 8705 / 8825 1.8 553.4 2.5X -SQL Parquet Vectorized 157 / 186 100.0 10.0 136.7X -SQL Parquet MR 1789 / 1794 8.8 113.8 12.0X -SQL ORC Vectorized 156 / 166 100.9 9.9 138.0X -SQL ORC Vectorized with copy 218 / 225 72.1 13.9 98.6X -SQL ORC MR 1448 / 1492 10.9 92.0 14.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 26366 / 26562 0.6 1676.3 1.0X +SQL Json 8709 / 8724 1.8 553.7 3.0X +SQL Parquet Vectorized 166 / 187 94.8 10.5 159.0X +SQL Parquet MR 1706 / 1720 9.2 108.4 15.5X +SQL ORC Vectorized 167 / 174 94.2 10.6 157.9X +SQL ORC MR 1433 / 1465 11.0 91.1 18.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 202 / 211 77.7 12.9 1.0X -ParquetReader Vectorized -> Row 118 / 120 133.5 7.5 1.7X +ParquetReader Vectorized 200 / 207 78.7 12.7 1.0X +ParquetReader Vectorized -> Row 117 / 119 134.7 7.4 1.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 23282 / 23312 0.7 1480.2 1.0X -SQL Json 9187 / 9189 1.7 584.1 2.5X -SQL Parquet Vectorized 204 / 218 77.0 13.0 114.0X -SQL Parquet MR 1941 / 1953 8.1 123.4 12.0X -SQL ORC Vectorized 217 / 225 72.6 13.8 107.5X -SQL ORC Vectorized with copy 279 / 289 56.3 17.8 83.4X -SQL ORC MR 1541 / 1549 10.2 98.0 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 26489 / 26547 0.6 1684.1 1.0X +SQL Json 8990 / 8998 1.7 571.5 2.9X +SQL Parquet Vectorized 209 / 221 75.1 13.3 126.5X +SQL Parquet MR 1949 / 1949 8.1 123.9 13.6X +SQL ORC Vectorized 221 / 228 71.3 14.0 120.1X +SQL ORC MR 1527 / 1549 10.3 97.1 17.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 288 / 297 54.6 18.3 1.0X -ParquetReader Vectorized -> Row 255 / 257 61.7 16.2 1.1X +ParquetReader Vectorized 286 / 296 54.9 18.2 1.0X +ParquetReader Vectorized -> Row 249 / 253 63.1 15.8 1.1X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 24990 / 25012 0.6 1588.8 1.0X -SQL Json 9837 / 9865 1.6 625.4 2.5X -SQL Parquet Vectorized 170 / 180 92.3 10.8 146.6X -SQL Parquet MR 2319 / 2328 6.8 147.4 10.8X -SQL ORC Vectorized 293 / 301 53.7 18.6 85.3X -SQL ORC Vectorized with copy 297 / 309 52.9 18.9 84.0X -SQL ORC MR 1667 / 1674 9.4 106.0 15.0X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 27701 / 27744 0.6 1761.2 1.0X +SQL Json 9703 / 9733 1.6 616.9 2.9X +SQL Parquet Vectorized 176 / 182 89.2 11.2 157.0X +SQL Parquet MR 2164 / 2173 7.3 137.6 12.8X +SQL ORC Vectorized 307 / 314 51.2 19.5 90.2X +SQL ORC MR 1690 / 1700 9.3 107.4 16.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 257 / 274 61.3 16.3 1.0X -ParquetReader Vectorized -> Row 259 / 264 60.8 16.4 1.0X +ParquetReader Vectorized 259 / 277 60.7 16.5 1.0X +ParquetReader Vectorized -> Row 261 / 265 60.3 16.6 1.0X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 32537 / 32554 0.5 2068.7 1.0X -SQL Json 12610 / 12668 1.2 801.7 2.6X -SQL Parquet Vectorized 258 / 276 61.0 16.4 126.2X -SQL Parquet MR 2422 / 2435 6.5 154.0 13.4X -SQL ORC Vectorized 378 / 385 41.6 24.0 86.2X -SQL ORC Vectorized with copy 381 / 389 41.3 24.2 85.4X -SQL ORC MR 1797 / 1819 8.8 114.3 18.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 34813 / 34900 0.5 2213.3 1.0X +SQL Json 12570 / 12617 1.3 799.2 2.8X +SQL Parquet Vectorized 270 / 308 58.2 17.2 128.9X +SQL Parquet MR 2427 / 2431 6.5 154.3 14.3X +SQL ORC Vectorized 388 / 398 40.6 24.6 89.8X +SQL ORC MR 1819 / 1851 8.6 115.7 19.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 352 / 368 44.7 22.4 1.0X -ParquetReader Vectorized -> Row 351 / 359 44.8 22.3 1.0X +ParquetReader Vectorized 372 / 379 42.3 23.7 1.0X +ParquetReader Vectorized -> Row 357 / 368 44.1 22.7 1.0X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 27179 / 27184 0.6 1728.0 1.0X -SQL Json 12578 / 12585 1.3 799.7 2.2X -SQL Parquet Vectorized 161 / 171 97.5 10.3 168.5X -SQL Parquet MR 2361 / 2395 6.7 150.1 11.5X -SQL ORC Vectorized 473 / 480 33.3 30.0 57.5X -SQL ORC Vectorized with copy 478 / 483 32.9 30.4 56.8X -SQL ORC MR 1858 / 1859 8.5 118.2 14.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 28753 / 28781 0.5 1828.0 1.0X +SQL Json 12039 / 12215 1.3 765.4 2.4X +SQL Parquet Vectorized 170 / 177 92.4 10.8 169.0X +SQL Parquet MR 2184 / 2196 7.2 138.9 13.2X +SQL ORC Vectorized 432 / 440 36.4 27.5 66.5X +SQL ORC MR 1812 / 1833 8.7 115.2 15.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 251 / 255 62.7 15.9 1.0X -ParquetReader Vectorized -> Row 255 / 259 61.8 16.2 1.0X +ParquetReader Vectorized 253 / 260 62.2 16.1 1.0X +ParquetReader Vectorized -> Row 256 / 257 61.6 16.2 1.0X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 34797 / 34830 0.5 2212.3 1.0X -SQL Json 17806 / 17828 0.9 1132.1 2.0X -SQL Parquet Vectorized 260 / 269 60.6 16.5 134.0X -SQL Parquet MR 2512 / 2534 6.3 159.7 13.9X -SQL ORC Vectorized 582 / 593 27.0 37.0 59.8X -SQL ORC Vectorized with copy 576 / 584 27.3 36.6 60.4X -SQL ORC MR 2309 / 2313 6.8 146.8 15.1X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 36177 / 36188 0.4 2300.1 1.0X +SQL Json 18895 / 18898 0.8 1201.3 1.9X +SQL Parquet Vectorized 267 / 276 58.9 17.0 135.6X +SQL Parquet MR 2355 / 2363 6.7 149.7 15.4X +SQL ORC Vectorized 543 / 546 29.0 34.5 66.6X +SQL ORC MR 2246 / 2258 7.0 142.8 16.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Parquet Reader Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -ParquetReader Vectorized 350 / 363 44.9 22.3 1.0X -ParquetReader Vectorized -> Row 350 / 366 44.9 22.3 1.0X +ParquetReader Vectorized 353 / 367 44.6 22.4 1.0X +ParquetReader Vectorized -> Row 351 / 357 44.7 22.3 1.0X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 22486 / 22590 0.5 2144.5 1.0X -SQL Json 14124 / 14195 0.7 1347.0 1.6X -SQL Parquet Vectorized 2342 / 2347 4.5 223.4 9.6X -SQL Parquet MR 4660 / 4664 2.2 444.4 4.8X -SQL ORC Vectorized 2378 / 2379 4.4 226.8 9.5X -SQL ORC Vectorized with copy 2548 / 2571 4.1 243.0 8.8X -SQL ORC MR 4206 / 4211 2.5 401.1 5.3X +SQL CSV 21130 / 21246 0.5 2015.1 1.0X +SQL Json 12145 / 12174 0.9 1158.2 1.7X +SQL Parquet Vectorized 2363 / 2377 4.4 225.3 8.9X +SQL Parquet MR 4555 / 4557 2.3 434.4 4.6X +SQL ORC Vectorized 2361 / 2388 4.4 225.1 9.0X +SQL ORC MR 4186 / 4209 2.5 399.2 5.0X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 12150 / 12178 0.9 1158.7 1.0X -SQL Json 7012 / 7014 1.5 668.7 1.7X -SQL Parquet Vectorized 792 / 796 13.2 75.5 15.3X -SQL Parquet MR 1961 / 1975 5.3 187.0 6.2X -SQL ORC Vectorized 482 / 485 21.8 46.0 25.2X -SQL ORC Vectorized with copy 710 / 715 14.8 67.7 17.1X -SQL ORC MR 2081 / 2083 5.0 198.5 5.8X +SQL CSV 11693 / 11729 0.9 1115.1 1.0X +SQL Json 7025 / 7025 1.5 669.9 1.7X +SQL Parquet Vectorized 803 / 821 13.1 76.6 14.6X +SQL Parquet MR 1776 / 1790 5.9 169.4 6.6X +SQL ORC Vectorized 491 / 494 21.4 46.8 23.8X +SQL ORC MR 2050 / 2063 5.1 195.5 5.7X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Data column - CSV 31789 / 31791 0.5 2021.1 1.0X -Data column - Json 12873 / 12918 1.2 818.4 2.5X -Data column - Parquet Vectorized 267 / 280 58.9 17.0 119.1X -Data column - Parquet MR 3387 / 3402 4.6 215.3 9.4X -Data column - ORC Vectorized 391 / 453 40.2 24.9 81.2X -Data column - ORC Vectorized with copy 392 / 398 40.2 24.9 81.2X -Data column - ORC MR 2508 / 2512 6.3 159.4 12.7X -Partition column - CSV 6965 / 6977 2.3 442.8 4.6X -Partition column - Json 5563 / 5576 2.8 353.7 5.7X -Partition column - Parquet Vectorized 65 / 78 241.1 4.1 487.2X -Partition column - Parquet MR 1811 / 1811 8.7 115.1 17.6X -Partition column - ORC Vectorized 66 / 73 239.0 4.2 483.0X -Partition column - ORC Vectorized with copy 65 / 70 241.1 4.1 487.3X -Partition column - ORC MR 1775 / 1778 8.9 112.8 17.9X -Both columns - CSV 30032 / 30113 0.5 1909.4 1.1X -Both columns - Json 13941 / 13959 1.1 886.3 2.3X -Both columns - Parquet Vectorized 312 / 330 50.3 19.9 101.7X -Both columns - Parquet MR 3858 / 3862 4.1 245.3 8.2X -Both columns - ORC Vectorized 431 / 437 36.5 27.4 73.8X -Both column - ORC Vectorized with copy 523 / 529 30.1 33.3 60.7X -Both columns - ORC MR 2712 / 2805 5.8 172.4 11.7X +Data column - CSV 30965 / 31041 0.5 1968.7 1.0X +Data column - Json 12876 / 12882 1.2 818.6 2.4X +Data column - Parquet Vectorized 277 / 282 56.7 17.6 111.6X +Data column - Parquet MR 3398 / 3402 4.6 216.0 9.1X +Data column - ORC Vectorized 399 / 407 39.4 25.4 77.5X +Data column - ORC MR 2583 / 2589 6.1 164.2 12.0X +Partition column - CSV 7403 / 7427 2.1 470.7 4.2X +Partition column - Json 5587 / 5625 2.8 355.2 5.5X +Partition column - Parquet Vectorized 71 / 78 222.6 4.5 438.3X +Partition column - Parquet MR 1798 / 1808 8.7 114.3 17.2X +Partition column - ORC Vectorized 72 / 75 219.0 4.6 431.2X +Partition column - ORC MR 1772 / 1778 8.9 112.6 17.5X +Both columns - CSV 30211 / 30212 0.5 1920.7 1.0X +Both columns - Json 13382 / 13391 1.2 850.8 2.3X +Both columns - Parquet Vectorized 321 / 333 49.0 20.4 96.4X +Both columns - Parquet MR 3656 / 3661 4.3 232.4 8.5X +Both columns - ORC Vectorized 443 / 448 35.5 28.2 69.9X +Both columns - ORC MR 2626 / 2633 6.0 167.0 11.8X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 13525 / 13823 0.8 1289.9 1.0X -SQL Json 9913 / 9921 1.1 945.3 1.4X -SQL Parquet Vectorized 1517 / 1517 6.9 144.7 8.9X -SQL Parquet MR 3996 / 4008 2.6 381.1 3.4X -ParquetReader Vectorized 1120 / 1128 9.4 106.8 12.1X -SQL ORC Vectorized 1203 / 1224 8.7 114.7 11.2X -SQL ORC Vectorized with copy 1639 / 1646 6.4 156.3 8.3X -SQL ORC MR 3720 / 3780 2.8 354.7 3.6X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 13918 / 13979 0.8 1327.3 1.0X +SQL Json 10068 / 10068 1.0 960.1 1.4X +SQL Parquet Vectorized 1563 / 1564 6.7 149.0 8.9X +SQL Parquet MR 3835 / 3836 2.7 365.8 3.6X +ParquetReader Vectorized 1115 / 1118 9.4 106.4 12.5X +SQL ORC Vectorized 1172 / 1208 8.9 111.8 11.9X +SQL ORC MR 3708 / 3711 2.8 353.6 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 15860 / 15877 0.7 1512.5 1.0X -SQL Json 7676 / 7688 1.4 732.0 2.1X -SQL Parquet Vectorized 1072 / 1084 9.8 102.2 14.8X -SQL Parquet MR 2890 / 2897 3.6 275.6 5.5X -ParquetReader Vectorized 1052 / 1053 10.0 100.4 15.1X -SQL ORC Vectorized 1248 / 1248 8.4 119.0 12.7X -SQL ORC Vectorized with copy 1627 / 1637 6.4 155.2 9.7X -SQL ORC MR 3365 / 3369 3.1 320.9 4.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 13972 / 14043 0.8 1332.5 1.0X +SQL Json 7436 / 7469 1.4 709.1 1.9X +SQL Parquet Vectorized 1103 / 1112 9.5 105.2 12.7X +SQL Parquet MR 2841 / 2847 3.7 271.0 4.9X +ParquetReader Vectorized 992 / 1012 10.6 94.6 14.1X +SQL ORC Vectorized 1275 / 1349 8.2 121.6 11.0X +SQL ORC MR 3244 / 3259 3.2 309.3 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 13401 / 13561 0.8 1278.1 1.0X -SQL Json 5253 / 5303 2.0 500.9 2.6X -SQL Parquet Vectorized 233 / 242 45.0 22.2 57.6X -SQL Parquet MR 1791 / 1796 5.9 170.8 7.5X -ParquetReader Vectorized 236 / 238 44.4 22.5 56.7X -SQL ORC Vectorized 453 / 473 23.2 43.2 29.6X -SQL ORC Vectorized with copy 573 / 577 18.3 54.7 23.4X -SQL ORC MR 1846 / 1850 5.7 176.0 7.3X +SQL CSV 11228 / 11244 0.9 1070.8 1.0X +SQL Json 5200 / 5247 2.0 495.9 2.2X +SQL Parquet Vectorized 238 / 242 44.1 22.7 47.2X +SQL Parquet MR 1730 / 1734 6.1 165.0 6.5X +ParquetReader Vectorized 237 / 238 44.3 22.6 47.4X +SQL ORC Vectorized 459 / 462 22.8 43.8 24.4X +SQL ORC MR 1767 / 1783 5.9 168.5 6.4X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 3147 / 3148 0.3 3001.1 1.0X -SQL Json 2666 / 2693 0.4 2542.9 1.2X -SQL Parquet Vectorized 54 / 58 19.5 51.3 58.5X -SQL Parquet MR 220 / 353 4.8 209.9 14.3X -SQL ORC Vectorized 63 / 77 16.8 59.7 50.3X -SQL ORC Vectorized with copy 63 / 66 16.7 59.8 50.2X -SQL ORC MR 317 / 321 3.3 302.2 9.9X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 3322 / 3356 0.3 3167.9 1.0X +SQL Json 2808 / 2843 0.4 2678.2 1.2X +SQL Parquet Vectorized 56 / 63 18.9 52.9 59.8X +SQL Parquet MR 215 / 219 4.9 205.4 15.4X +SQL ORC Vectorized 64 / 76 16.4 60.9 52.0X +SQL ORC MR 314 / 316 3.3 299.6 10.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 50 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 7902 / 7921 0.1 7536.2 1.0X -SQL Json 9467 / 9491 0.1 9028.6 0.8X -SQL Parquet Vectorized 73 / 79 14.3 69.8 108.0X -SQL Parquet MR 239 / 247 4.4 228.0 33.1X -SQL ORC Vectorized 78 / 84 13.4 74.6 101.0X -SQL ORC Vectorized with copy 78 / 88 13.4 74.4 101.3X -SQL ORC MR 910 / 918 1.2 867.6 8.7X - -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +SQL CSV 7978 / 7989 0.1 7608.5 1.0X +SQL Json 10294 / 10325 0.1 9816.9 0.8X +SQL Parquet Vectorized 72 / 85 14.5 69.0 110.3X +SQL Parquet MR 237 / 241 4.4 226.4 33.6X +SQL ORC Vectorized 82 / 92 12.7 78.5 97.0X +SQL ORC MR 900 / 909 1.2 858.5 8.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -SQL CSV 13539 / 13543 0.1 12912.0 1.0X -SQL Json 17420 / 17446 0.1 16613.1 0.8X -SQL Parquet Vectorized 103 / 120 10.2 98.1 131.6X -SQL Parquet MR 250 / 258 4.2 238.9 54.1X -SQL ORC Vectorized 99 / 104 10.6 94.6 136.5X -SQL ORC Vectorized with copy 100 / 106 10.5 95.6 135.1X -SQL ORC MR 1653 / 1659 0.6 1576.3 8.2X +SQL CSV 13489 / 13508 0.1 12864.3 1.0X +SQL Json 18813 / 18827 0.1 17941.4 0.7X +SQL Parquet Vectorized 107 / 111 9.8 101.8 126.3X +SQL Parquet MR 275 / 286 3.8 262.3 49.0X +SQL ORC Vectorized 107 / 115 9.8 101.7 126.4X +SQL ORC MR 1659 / 1664 0.6 1582.3 8.1X diff --git a/sql/core/benchmarks/DateTimeBenchmark-results.txt b/sql/core/benchmarks/DateTimeBenchmark-results.txt new file mode 100644 index 0000000000000..1a58b05a2abba --- /dev/null +++ b/sql/core/benchmarks/DateTimeBenchmark-results.txt @@ -0,0 +1,425 @@ +================================================================================================ +Extract components +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +cast to timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +cast to timestamp wholestage off 275 / 287 36.4 27.5 1.0X +cast to timestamp wholestage on 243 / 253 41.2 24.3 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +year of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +year of timestamp wholestage off 661 / 667 15.1 66.1 1.0X +year of timestamp wholestage on 659 / 669 15.2 65.9 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +quarter of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +quarter of timestamp wholestage off 820 / 822 12.2 82.0 1.0X +quarter of timestamp wholestage on 768 / 776 13.0 76.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +month of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +month of timestamp wholestage off 636 / 638 15.7 63.6 1.0X +month of timestamp wholestage on 648 / 654 15.4 64.8 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +weekofyear of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +weekofyear of timestamp wholestage off 1093 / 1097 9.2 109.3 1.0X +weekofyear of timestamp wholestage on 1101 / 1107 9.1 110.1 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +day of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +day of timestamp wholestage off 643 / 644 15.6 64.3 1.0X +day of timestamp wholestage on 655 / 657 15.3 65.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +dayofyear of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +dayofyear of timestamp wholestage off 681 / 692 14.7 68.1 1.0X +dayofyear of timestamp wholestage on 675 / 680 14.8 67.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +dayofmonth of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +dayofmonth of timestamp wholestage off 656 / 657 15.2 65.6 1.0X +dayofmonth of timestamp wholestage on 651 / 658 15.4 65.1 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +dayofweek of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +dayofweek of timestamp wholestage off 775 / 776 12.9 77.5 1.0X +dayofweek of timestamp wholestage on 777 / 781 12.9 77.7 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +weekday of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +weekday of timestamp wholestage off 737 / 737 13.6 73.7 1.0X +weekday of timestamp wholestage on 737 / 739 13.6 73.7 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +hour of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +hour of timestamp wholestage off 425 / 426 23.5 42.5 1.0X +hour of timestamp wholestage on 430 / 434 23.2 43.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +minute of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +minute of timestamp wholestage off 430 / 439 23.3 43.0 1.0X +minute of timestamp wholestage on 436 / 438 23.0 43.6 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +second of timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +second of timestamp wholestage off 413 / 413 24.2 41.3 1.0X +second of timestamp wholestage on 413 / 425 24.2 41.3 1.0X + + +================================================================================================ +Current date and time +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +current_date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +current_date wholestage off 205 / 206 48.7 20.5 1.0X +current_date wholestage on 219 / 224 45.8 21.9 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +current_timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +current_timestamp wholestage off 212 / 213 47.3 21.2 1.0X +current_timestamp wholestage on 202 / 205 49.6 20.2 1.0X + + +================================================================================================ +Date arithmetic +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +cast to date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +cast to date wholestage off 459 / 462 21.8 45.9 1.0X +cast to date wholestage on 493 / 500 20.3 49.3 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +last_day: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +last_day wholestage off 680 / 686 14.7 68.0 1.0X +last_day wholestage on 671 / 681 14.9 67.1 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +next_day: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +next_day wholestage off 532 / 533 18.8 53.2 1.0X +next_day wholestage on 576 / 580 17.4 57.6 0.9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_add: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_add wholestage off 445 / 446 22.5 44.5 1.0X +date_add wholestage on 455 / 457 22.0 45.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_sub: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_sub wholestage off 454 / 457 22.0 45.4 1.0X +date_sub wholestage on 455 / 458 22.0 45.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +add_months: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +add_months wholestage off 898 / 900 11.1 89.8 1.0X +add_months wholestage on 894 / 909 11.2 89.4 1.0X + + +================================================================================================ +Formatting dates +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +format date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +format date wholestage off 7180 / 7181 1.4 718.0 1.0X +format date wholestage on 7051 / 7194 1.4 705.1 1.0X + + +================================================================================================ +Formatting timestamps +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +from_unixtime: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +from_unixtime wholestage off 7136 / 7163 1.4 713.6 1.0X +from_unixtime wholestage on 7144 / 7174 1.4 714.4 1.0X + + +================================================================================================ +Convert timestamps +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +from_utc_timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +from_utc_timestamp wholestage off 880 / 888 11.4 88.0 1.0X +from_utc_timestamp wholestage on 841 / 854 11.9 84.1 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to_utc_timestamp: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +to_utc_timestamp wholestage off 879 / 884 11.4 87.9 1.0X +to_utc_timestamp wholestage on 862 / 876 11.6 86.2 1.0X + + +================================================================================================ +Intervals +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +cast interval: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +cast interval wholestage off 242 / 250 41.3 24.2 1.0X +cast interval wholestage on 221 / 223 45.3 22.1 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +datediff: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +datediff wholestage off 697 / 698 14.3 69.7 1.0X +datediff wholestage on 680 / 683 14.7 68.0 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +months_between: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +months_between wholestage off 1675 / 1677 6.0 167.5 1.0X +months_between wholestage on 1636 / 1649 6.1 163.6 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +window: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +window wholestage off 1600 / 1627 0.6 1599.7 1.0X +window wholestage on 19480 / 19530 0.1 19479.6 0.1X + + +================================================================================================ +Truncation +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc YEAR: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc YEAR wholestage off 863 / 864 11.6 86.3 1.0X +date_trunc YEAR wholestage on 812 / 814 12.3 81.2 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc YYYY: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc YYYY wholestage off 865 / 926 11.6 86.5 1.0X +date_trunc YYYY wholestage on 811 / 820 12.3 81.1 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc YY: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc YY wholestage off 863 / 867 11.6 86.3 1.0X +date_trunc YY wholestage on 810 / 822 12.3 81.0 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc MON: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc MON wholestage off 917 / 921 10.9 91.7 1.0X +date_trunc MON wholestage on 857 / 860 11.7 85.7 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc MONTH: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc MONTH wholestage off 919 / 919 10.9 91.9 1.0X +date_trunc MONTH wholestage on 862 / 863 11.6 86.2 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc MM: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc MM wholestage off 923 / 924 10.8 92.3 1.0X +date_trunc MM wholestage on 855 / 859 11.7 85.5 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc DAY: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc DAY wholestage off 444 / 444 22.5 44.4 1.0X +date_trunc DAY wholestage on 404 / 406 24.7 40.4 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc DD: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc DD wholestage off 445 / 446 22.5 44.5 1.0X +date_trunc DD wholestage on 404 / 406 24.7 40.4 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc HOUR: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc HOUR wholestage off 462 / 464 21.6 46.2 1.0X +date_trunc HOUR wholestage on 416 / 422 24.1 41.6 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc MINUTE: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc MINUTE wholestage off 294 / 294 34.0 29.4 1.0X +date_trunc MINUTE wholestage on 258 / 266 38.8 25.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc SECOND: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc SECOND wholestage off 292 / 295 34.2 29.2 1.0X +date_trunc SECOND wholestage on 271 / 276 36.9 27.1 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc WEEK: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc WEEK wholestage off 739 / 740 13.5 73.9 1.0X +date_trunc WEEK wholestage on 712 / 715 14.0 71.2 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +date_trunc QUARTER: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +date_trunc QUARTER wholestage off 1465 / 1467 6.8 146.5 1.0X +date_trunc QUARTER wholestage on 1419 / 1423 7.0 141.9 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc year: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc year wholestage off 222 / 222 45.0 22.2 1.0X +trunc year wholestage on 207 / 214 48.3 20.7 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc yyyy: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc yyyy wholestage off 221 / 225 45.2 22.1 1.0X +trunc yyyy wholestage on 208 / 212 48.0 20.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc yy: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc yy wholestage off 221 / 222 45.3 22.1 1.0X +trunc yy wholestage on 208 / 210 48.0 20.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc mon: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc mon wholestage off 231 / 239 43.3 23.1 1.0X +trunc mon wholestage on 208 / 214 48.0 20.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc month: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc month wholestage off 222 / 222 45.1 22.2 1.0X +trunc month wholestage on 208 / 224 48.1 20.8 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +trunc mm: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +trunc mm wholestage off 222 / 226 45.1 22.2 1.0X +trunc mm wholestage on 208 / 216 48.0 20.8 1.1X + + +================================================================================================ +Parsing +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to timestamp str: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +to timestamp str wholestage off 165 / 166 6.1 164.7 1.0X +to timestamp str wholestage on 160 / 163 6.2 160.5 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_timestamp wholestage off 1308 1353 64 0.8 1307.9 1.0X +to_timestamp wholestage on 1197 1230 21 0.8 1197.0 1.1X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to_unix_timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +to_unix_timestamp wholestage off 1221 1224 4 0.8 1221.0 1.0X +to_unix_timestamp wholestage on 1224 1228 4 0.8 1223.8 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to date str: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +to date str wholestage off 155 / 157 6.4 155.4 1.0X +to date str wholestage on 154 / 156 6.5 154.3 1.0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.3 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +to_date: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +to_date wholestage off 1477 / 1479 0.7 1477.3 1.0X +to_date wholestage on 1468 / 1473 0.7 1468.2 1.0X + + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +To/from java.sql.Timestamp: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +From java.sql.Timestamp 283 301 19 17.7 56.6 1.0X +Collect longs 1048 1087 36 4.8 209.6 0.3X +Collect timestamps 1425 1479 56 3.5 285.1 0.2X + + diff --git a/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt new file mode 100644 index 0000000000000..02c6b72f32216 --- /dev/null +++ b/sql/core/benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt @@ -0,0 +1,45 @@ +================================================================================================ +WITHOUT SPILL +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Array with 100000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ArrayBuffer 6378 / 6550 16.1 62.3 1.0X +ExternalAppendOnlyUnsafeRowArray 6196 / 6242 16.5 60.5 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Array with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ArrayBuffer 11988 / 12027 21.9 45.7 1.0X +ExternalAppendOnlyUnsafeRowArray 37480 / 37574 7.0 143.0 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Array with 30000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +ArrayBuffer 23536 / 23538 20.9 47.9 1.0X +ExternalAppendOnlyUnsafeRowArray 31275 / 31277 15.7 63.6 0.8X + + +================================================================================================ +WITH SPILL +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Spilling with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 29241 / 29279 9.0 111.5 1.0X +ExternalAppendOnlyUnsafeRowArray 14309 / 14313 18.3 54.6 2.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Spilling with 10000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +UnsafeExternalSorter 11 / 11 14.8 67.4 1.0X +ExternalAppendOnlyUnsafeRowArray 9 / 9 17.6 56.8 1.2X + + diff --git a/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt new file mode 100644 index 0000000000000..338244ad542f4 --- /dev/null +++ b/sql/core/benchmarks/HashedRelationMetricsBenchmark-results.txt @@ -0,0 +1,11 @@ +================================================================================================ +LongToUnsafeRowMap metrics +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_181-b13 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-7700HQ CPU @ 2.80GHz +LongToUnsafeRowMap metrics: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +LongToUnsafeRowMap 234 / 315 2.1 467.3 1.0X + + diff --git a/sql/core/benchmarks/InExpressionBenchmark-results.txt b/sql/core/benchmarks/InExpressionBenchmark-results.txt new file mode 100644 index 0000000000000..f6685bfc45089 --- /dev/null +++ b/sql/core/benchmarks/InExpressionBenchmark-results.txt @@ -0,0 +1,740 @@ +================================================================================================ +In Expression Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 105 148 42 94.8 10.5 1.0X +InSet expression 79 98 19 126.9 7.9 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 101 115 20 99.3 10.1 1.0X +InSet expression 76 84 8 131.4 7.6 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 172 176 3 58.0 17.2 1.0X +InSet expression 100 107 9 99.6 10.0 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 299 302 4 33.5 29.9 1.0X +InSet expression 145 149 5 69.0 14.5 2.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 518 524 11 19.3 51.8 1.0X +InSet expression 240 250 12 41.6 24.0 2.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 bytes: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 831 844 14 12.0 83.1 1.0X +InSet expression 425 432 4 23.5 42.5 2.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 58 62 5 171.9 5.8 1.0X +InSet expression 56 58 5 178.0 5.6 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 76 79 5 131.9 7.6 1.0X +InSet expression 50 55 7 198.2 5.0 1.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 129 139 23 77.3 12.9 1.0X +InSet expression 48 50 5 210.5 4.8 2.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 225 226 0 44.4 22.5 1.0X +InSet expression 52 56 7 191.2 5.2 4.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 400 406 11 25.0 40.0 1.0X +InSet expression 54 58 7 185.0 5.4 7.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 761 762 1 13.1 76.1 1.0X +InSet expression 60 61 2 167.1 6.0 12.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1118 1119 1 8.9 111.8 1.0X +InSet expression 66 67 2 152.2 6.6 17.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1478 1487 19 6.8 147.8 1.0X +InSet expression 71 75 11 141.7 7.1 20.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 shorts: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1836 1854 27 5.4 183.6 1.0X +InSet expression 248 253 3 40.2 24.8 7.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 55 68 19 180.3 5.5 1.0X +InSet expression 60 63 7 167.0 6.0 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 72 76 5 138.0 7.2 1.0X +InSet expression 63 68 11 157.7 6.3 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 133 136 5 75.0 13.3 1.0X +InSet expression 73 78 10 137.2 7.3 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 223 225 2 44.8 22.3 1.0X +InSet expression 81 84 14 124.1 8.1 2.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 402 404 1 24.9 40.2 1.0X +InSet expression 90 91 2 111.6 9.0 4.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 758 759 0 13.2 75.8 1.0X +InSet expression 110 119 20 91.0 11.0 6.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1121 1123 3 8.9 112.1 1.0X +InSet expression 121 122 2 82.6 12.1 9.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1482 1484 2 6.7 148.2 1.0X +InSet expression 134 135 2 74.6 13.4 11.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 shorts (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1838 1882 92 5.4 183.8 1.0X +InSet expression 251 254 3 39.8 25.1 7.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 51 52 2 197.1 5.1 1.0X +InSet expression 61 63 3 162.8 6.1 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 69 73 10 145.0 6.9 1.0X +InSet expression 43 46 7 231.2 4.3 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 123 129 19 81.4 12.3 1.0X +InSet expression 43 46 8 230.0 4.3 2.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 222 223 1 45.1 22.2 1.0X +InSet expression 49 50 2 206.2 4.9 4.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 401 402 0 24.9 40.1 1.0X +InSet expression 51 56 11 196.6 5.1 7.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 755 756 1 13.2 75.5 1.0X +InSet expression 56 57 2 179.5 5.6 13.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1115 1116 1 9.0 111.5 1.0X +InSet expression 61 62 4 165.2 6.1 18.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1476 1478 1 6.8 147.6 1.0X +InSet expression 66 67 2 152.2 6.6 22.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 ints: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1834 1873 85 5.5 183.4 1.0X +InSet expression 230 233 3 43.5 23.0 8.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 40 42 2 247.6 4.0 1.0X +InSet expression 37 39 3 271.6 3.7 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 59 60 3 170.0 5.9 1.0X +InSet expression 42 44 3 237.6 4.2 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 114 116 6 87.5 11.4 1.0X +InSet expression 53 58 10 188.0 5.3 2.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 207 214 14 48.3 20.7 1.0X +InSet expression 62 63 2 162.1 6.2 3.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 385 391 6 26.0 38.5 1.0X +InSet expression 71 73 2 140.4 7.1 5.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 746 750 2 13.4 74.6 1.0X +InSet expression 101 105 8 98.5 10.1 7.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1100 1106 4 9.1 110.0 1.0X +InSet expression 109 111 2 91.6 10.9 10.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1470 1480 7 6.8 147.0 1.0X +InSet expression 115 116 2 87.1 11.5 12.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 ints (non-compact): Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1838 1907 152 5.4 183.8 1.0X +InSet expression 231 233 2 43.3 23.1 8.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 48 52 6 206.5 4.8 1.0X +InSet expression 150 152 4 66.8 15.0 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 62 63 1 161.3 6.2 1.0X +InSet expression 165 168 5 60.7 16.5 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 116 116 0 86.1 11.6 1.0X +InSet expression 173 175 3 57.9 17.3 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 206 208 6 48.6 20.6 1.0X +InSet expression 212 214 2 47.1 21.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 384 386 2 26.0 38.4 1.0X +InSet expression 183 185 2 54.6 18.3 2.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 longs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 742 748 13 13.5 74.2 1.0X +InSet expression 175 177 2 57.1 17.5 4.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 88 89 1 114.2 8.8 1.0X +InSet expression 168 170 2 59.5 16.8 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 128 129 3 78.0 12.8 1.0X +InSet expression 187 188 2 53.6 18.7 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 242 242 0 41.3 24.2 1.0X +InSet expression 192 194 2 52.0 19.2 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 419 420 0 23.8 41.9 1.0X +InSet expression 235 236 1 42.5 23.5 1.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 774 775 1 12.9 77.4 1.0X +InSet expression 205 206 3 48.9 20.5 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 floats: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 3036 3123 191 3.3 303.6 1.0X +InSet expression 197 198 1 50.8 19.7 15.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 83 84 2 120.9 8.3 1.0X +InSet expression 167 168 2 60.0 16.7 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 125 126 3 80.3 12.5 1.0X +InSet expression 186 188 2 53.7 18.6 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 237 238 1 42.1 23.7 1.0X +InSet expression 192 195 3 52.0 19.2 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 414 415 0 24.1 41.4 1.0X +InSet expression 239 242 3 41.9 23.9 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 769 771 3 13.0 76.9 1.0X +InSet expression 203 213 22 49.3 20.3 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 doubles: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 3757 3796 85 2.7 375.7 1.0X +InSet expression 193 194 2 51.9 19.3 19.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 47 48 3 21.3 47.0 1.0X +InSet expression 155 168 29 6.4 155.3 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 58 59 2 17.4 57.6 1.0X +InSet expression 157 160 2 6.4 157.4 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 92 92 2 10.9 91.5 1.0X +InSet expression 160 162 2 6.3 159.6 0.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 171 171 0 5.9 170.8 1.0X +InSet expression 169 172 3 5.9 169.3 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 429 429 0 2.3 428.6 1.0X +InSet expression 170 172 2 5.9 170.4 2.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 small decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 996 1144 328 1.0 996.3 1.0X +InSet expression 177 179 3 5.7 176.8 5.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 248 251 2 4.0 248.0 1.0X +InSet expression 175 177 2 5.7 174.9 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 420 426 11 2.4 420.0 1.0X +InSet expression 177 180 3 5.7 176.9 2.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1005 1008 4 1.0 1004.9 1.0X +InSet expression 184 187 3 5.4 183.7 5.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1922 1933 13 0.5 1922.2 1.0X +InSet expression 189 193 7 5.3 188.9 10.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 3861 3871 12 0.3 3860.5 1.0X +InSet expression 213 225 30 4.7 213.5 18.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 large decimals: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 7731 7774 25 0.1 7731.5 1.0X +InSet expression 222 225 3 4.5 222.4 34.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 127 133 9 7.9 126.8 1.0X +InSet expression 142 143 2 7.0 141.9 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 132 133 2 7.6 131.7 1.0X +InSet expression 144 146 2 6.9 144.1 0.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 153 154 2 6.5 152.9 1.0X +InSet expression 151 153 2 6.6 151.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 186 187 2 5.4 185.8 1.0X +InSet expression 154 156 3 6.5 153.7 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 260 263 3 3.8 260.3 1.0X +InSet expression 151 153 2 6.6 151.3 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 731 891 352 1.4 731.4 1.0X +InSet expression 155 157 3 6.4 155.4 4.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 42 43 2 240.1 4.2 1.0X +InSet expression 159 160 2 63.0 15.9 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 58 59 2 171.4 5.8 1.0X +InSet expression 174 183 21 57.5 17.4 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 111 113 3 90.0 11.1 1.0X +InSet expression 228 229 2 43.9 22.8 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 192 193 1 52.1 19.2 1.0X +InSet expression 250 250 1 40.1 25.0 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 373 384 12 26.8 37.3 1.0X +InSet expression 229 236 7 43.7 22.9 1.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 694 707 25 14.4 69.4 1.0X +InSet expression 221 226 7 45.2 22.1 3.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 196 198 2 50.9 19.6 1.0X +InSet expression 169 170 0 59.2 16.9 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 212 212 0 47.3 21.2 1.0X +InSet expression 197 197 0 50.8 19.7 1.1X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 266 266 1 37.7 26.6 1.0X +InSet expression 203 217 23 49.4 20.3 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 356 367 12 28.1 35.6 1.0X +InSet expression 212 213 1 47.1 21.2 1.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 559 573 26 17.9 55.9 1.0X +InSet expression 221 223 2 45.2 22.1 2.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 901 916 9 11.1 90.1 1.0X +InSet expression 238 241 9 42.1 23.8 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +300 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1264 1282 10 7.9 126.4 1.0X +InSet expression 253 262 15 39.5 25.3 5.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +400 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1628 1646 11 6.1 162.8 1.0X +InSet expression 264 265 1 37.8 26.4 6.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +500 dates: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1993 2015 15 5.0 199.3 1.0X +InSet expression 355 368 10 28.2 35.5 5.6X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 52 63 14 19.3 51.8 1.0X +InSet expression 96 98 2 10.4 95.9 0.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 78 80 3 12.8 77.9 1.0X +InSet expression 97 154 48 10.3 97.1 0.8X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 269 279 27 3.7 268.7 1.0X +InSet expression 120 124 13 8.3 119.9 2.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 579 626 96 1.7 579.2 1.0X +InSet expression 165 167 3 6.1 165.1 3.5X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 2582 2775 415 0.4 2582.1 1.0X +InSet expression 196 201 10 5.1 196.0 13.2X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 arrays: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 9438 9939 763 0.1 9437.9 1.0X +InSet expression 256 258 3 3.9 255.8 36.9X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +5 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 47 48 2 21.4 46.8 1.0X +InSet expression 158 160 2 6.3 157.6 0.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +10 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 62 63 4 16.2 61.9 1.0X +InSet expression 158 161 4 6.3 158.4 0.4X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +25 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 138 140 3 7.3 137.9 1.0X +InSet expression 202 219 43 5.0 201.7 0.7X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +50 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 366 367 1 2.7 365.7 1.0X +InSet expression 286 289 4 3.5 285.6 1.3X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +100 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 1055 1212 346 0.9 1054.7 1.0X +InSet expression 348 354 6 2.9 347.9 3.0X + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +200 structs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +In expression 6463 6772 650 0.2 6463.3 1.0X +InSet expression 450 455 4 2.2 449.6 14.4X + + diff --git a/sql/core/benchmarks/JSONBenchmark-results.txt b/sql/core/benchmarks/JSONBenchmark-results.txt new file mode 100644 index 0000000000000..7846983b44fb3 --- /dev/null +++ b/sql/core/benchmarks/JSONBenchmark-results.txt @@ -0,0 +1,112 @@ +================================================================================================ +Benchmark for performance of JSON parsing +================================================================================================ + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +JSON schema inferring: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 50949 51086 150 2.0 509.5 1.0X +UTF-8 is set 72012 72147 120 1.4 720.1 0.7X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +count a short column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 36799 36891 80 2.7 368.0 1.0X +UTF-8 is set 59796 59880 74 1.7 598.0 0.6X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +count a wide column: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 55803 55967 152 0.2 5580.3 1.0X +UTF-8 is set 80623 80825 178 0.1 8062.3 0.7X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +select wide row: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +No encoding 84263 85750 1476 0.0 168526.2 1.0X +UTF-8 is set 98848 100183 1592 0.0 197696.0 0.9X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Select a subset of 10 columns: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Select 10 columns 13930 13996 60 0.7 1393.0 1.0X +Select 1 column 17092 17394 360 0.6 1709.2 0.8X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +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 5596 5711 101 1.8 559.6 1.0X +Short column with UTF-8 7983 8158 160 1.3 798.3 0.7X +Wide column without encoding 110189 118451 NaN 0.1 11018.9 0.1X +Wide column with UTF-8 137827 142813 NaN 0.1 13782.7 0.0X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +JSON functions: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 951 953 2 10.5 95.1 1.0X +from_json 13015 13045 27 0.8 1301.5 0.1X +json_tuple 16257 16306 43 0.6 1625.7 0.1X +get_json_object 13195 13225 39 0.8 1319.5 0.1X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Dataset of json strings: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 4632 4687 49 10.8 92.6 1.0X +schema inferring 29176 29297 146 1.7 583.5 0.2X +parsing 24268 24457 175 2.1 485.4 0.2X + +Preparing data for benchmarking ... +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Json files in the per-line mode: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Text read 8264 8272 7 6.1 165.3 1.0X +Schema inferring 31910 32375 543 1.6 638.2 0.3X +Parsing without charset 29290 29397 124 1.7 585.8 0.3X +Parsing with UTF-8 41301 41390 81 1.2 826.0 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Write dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Create a dataset of timestamps 1149 1160 11 8.7 114.9 1.0X +to_json(timestamp) 11585 11688 140 0.9 1158.5 0.1X +write timestamps to files 10212 10260 49 1.0 1021.2 0.1X +Create a dataset of dates 1322 1328 10 7.6 132.2 0.9X +to_json(date) 7226 7241 14 1.4 722.6 0.2X +write dates to files 5634 5648 20 1.8 563.4 0.2X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_202-b08 on Mac OS X 10.14.4 +Intel(R) Core(TM) i7-4850HQ CPU @ 2.30GHz +Read dates and timestamps: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +read timestamp text from files 2097 2137 41 4.8 209.7 1.0X +read timestamps from files 20438 20451 11 0.5 2043.8 0.1X +infer timestamps from files 41694 41770 66 0.2 4169.4 0.1X +read date text from files 1832 1847 16 5.5 183.2 1.1X +read date from files 13796 13837 49 0.7 1379.6 0.2X +timestamp strings 3213 3233 26 3.1 321.3 0.7X +parse timestamps from Dataset[String] 22686 22743 53 0.4 2268.6 0.1X +infer timestamps from Dataset[String] 45301 45368 58 0.2 4530.1 0.0X +date strings 3431 3439 7 2.9 343.1 0.6X +parse dates from Dataset[String] 17688 17734 41 0.6 1768.8 0.1X +from_json(timestamp) 33439 33456 24 0.3 3343.9 0.1X +from_json(date) 24055 24164 107 0.4 2405.5 0.1X + + diff --git a/sql/core/benchmarks/JoinBenchmark-results.txt b/sql/core/benchmarks/JoinBenchmark-results.txt new file mode 100644 index 0000000000000..8ceb5e7a7fe94 --- /dev/null +++ b/sql/core/benchmarks/JoinBenchmark-results.txt @@ -0,0 +1,75 @@ +================================================================================================ +Join Benchmark +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Join w long wholestage off 4464 / 4483 4.7 212.9 1.0X +Join w long wholestage on 289 / 339 72.6 13.8 15.5X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Join w long duplicated wholestage off 5662 / 5678 3.7 270.0 1.0X +Join w long duplicated wholestage on 332 / 345 63.1 15.8 17.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Join w 2 ints wholestage off 173174 / 173183 0.1 8257.6 1.0X +Join w 2 ints wholestage on 166350 / 198362 0.1 7932.2 1.0X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Join w 2 longs wholestage off 7055 / 7214 3.0 336.4 1.0X +Join w 2 longs wholestage on 1869 / 1985 11.2 89.1 3.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +Join w 2 longs duplicated wholestage off 19256 / 20283 1.1 918.2 1.0X +Join w 2 longs duplicated wholestage on 2467 / 2544 8.5 117.7 7.8X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +outer join w long wholestage off 3756 / 3761 5.6 179.1 1.0X +outer join w long wholestage on 218 / 250 96.2 10.4 17.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +semi join w long wholestage off 2393 / 2416 8.8 114.1 1.0X +semi join w long wholestage on 214 / 218 97.9 10.2 11.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sort merge join wholestage off 2318 / 2392 0.9 1105.3 1.0X +sort merge join wholestage on 1669 / 1811 1.3 795.9 1.4X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +sort merge join with duplicates: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sort merge join with duplicates wholestage off 2966 / 2976 0.7 1414.5 1.0X +sort merge join with duplicates wholestage on 2413 / 2641 0.9 1150.5 1.2X + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +shuffle hash join wholestage off 1475 / 1479 2.8 351.7 1.0X +shuffle hash join wholestage on 1209 / 1238 3.5 288.3 1.2X + + diff --git a/sql/core/benchmarks/MiscBenchmark-results.txt b/sql/core/benchmarks/MiscBenchmark-results.txt index 85acd57893655..c4ae052095656 100644 --- a/sql/core/benchmarks/MiscBenchmark-results.txt +++ b/sql/core/benchmarks/MiscBenchmark-results.txt @@ -105,6 +105,13 @@ generate big struct array: Best/Avg Time(ms) Rate(M/s) Per Ro generate big struct array wholestage off 708 / 776 0.1 11803.5 1.0X generate big struct array wholestage on 535 / 589 0.1 8913.9 1.3X +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +generate big nested struct array: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +generate big nested struct array wholestage off 540 553 19 0.1 8997.4 1.0X +generate big nested struct array wholestage on 523 554 31 0.1 8725.0 1.0X + ================================================================================================ generate regular generator diff --git a/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt new file mode 100644 index 0000000000000..765193d6c6436 --- /dev/null +++ b/sql/core/benchmarks/OrcNestedSchemaPruningBenchmark-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For ORC v1 +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 127 163 24 7.9 127.1 1.0X +Nested column 974 1023 39 1.0 974.2 0.1X +Nested column in array 4834 4857 23 0.2 4834.1 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 454 488 45 2.2 454.3 1.0X +Nested column 1539 1602 80 0.6 1539.3 0.3X +Nested column in array 5765 5848 69 0.2 5764.7 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 365 395 58 2.7 364.9 1.0X +Nested column 1456 1477 23 0.7 1456.0 0.3X +Nested column in array 5734 5842 91 0.2 5734.4 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 373 387 15 2.7 372.8 1.0X +Nested column 4349 4397 59 0.2 4348.8 0.1X +Nested column in array 8893 8971 73 0.1 8893.2 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 130 159 24 7.7 129.9 1.0X +Nested column 1160 1216 50 0.9 1159.8 0.1X +Nested column in array 5297 5420 176 0.2 5296.8 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 585 615 60 1.7 585.5 1.0X +Nested column 4972 5213 156 0.2 4972.2 0.1X +Nested column in array 10095 10156 32 0.1 10095.4 0.1X + + diff --git a/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt new file mode 100644 index 0000000000000..fdd347f4bad9b --- /dev/null +++ b/sql/core/benchmarks/OrcV2NestedSchemaPruningBenchmark-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For ORC v2 +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 122 161 29 8.2 121.9 1.0X +Nested column 1255 1279 23 0.8 1255.4 0.1X +Nested column in array 5352 5393 37 0.2 5352.3 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 132 162 32 7.6 131.8 1.0X +Nested column 1246 1286 32 0.8 1245.6 0.1X +Nested column in array 5395 5542 143 0.2 5394.9 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 385 403 20 2.6 385.4 1.0X +Nested column 1663 1691 52 0.6 1663.2 0.2X +Nested column in array 6264 6335 73 0.2 6264.4 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 392 422 58 2.5 392.2 1.0X +Nested column 4104 4153 57 0.2 4104.0 0.1X +Nested column in array 8668 8748 55 0.1 8668.3 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 130 146 22 7.7 130.1 1.0X +Nested column 1127 1166 53 0.9 1127.3 0.1X +Nested column in array 4906 4968 40 0.2 4905.8 0.0X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 291 308 25 3.4 290.5 1.0X +Nested column 3016 3091 58 0.3 3016.0 0.1X +Nested column in array 7730 7821 140 0.1 7729.5 0.0X + + diff --git a/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt new file mode 100644 index 0000000000000..4e0c368b5370e --- /dev/null +++ b/sql/core/benchmarks/ParquetNestedSchemaPruningBenchmark-results.txt @@ -0,0 +1,53 @@ +================================================================================================ +Nested Schema Pruning Benchmark For Parquet +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Selection: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 151 174 16 6.6 151.3 1.0X +Nested column 316 375 88 3.2 315.7 0.5X +Nested column in array 1277 1292 11 0.8 1277.0 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Limiting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 452 501 67 2.2 451.9 1.0X +Nested column 664 722 77 1.5 664.4 0.7X +Nested column in array 1906 1997 91 0.5 1905.6 0.2X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 385 410 39 2.6 385.5 1.0X +Nested column 612 620 10 1.6 611.9 0.6X +Nested column in array 1790 1845 80 0.6 1789.5 0.2X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Repartitioning by exprs: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 386 402 26 2.6 386.0 1.0X +Nested column 2982 3057 64 0.3 2982.0 0.1X +Nested column in array 3504 3690 248 0.3 3503.7 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sample: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 138 152 10 7.2 138.3 1.0X +Nested column 345 369 16 2.9 344.8 0.4X +Nested column in array 1358 1405 50 0.7 1358.5 0.1X + +OpenJDK 64-Bit Server VM 1.8.0_212-b04 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +Sorting: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Top-level column 606 632 45 1.6 606.3 1.0X +Nested column 3586 3679 107 0.3 3585.8 0.2X +Nested column in array 4452 4831 244 0.2 4451.8 0.1X + + diff --git a/sql/core/benchmarks/RangeBenchmark-results.txt b/sql/core/benchmarks/RangeBenchmark-results.txt new file mode 100644 index 0000000000000..21766e0fd8664 --- /dev/null +++ b/sql/core/benchmarks/RangeBenchmark-results.txt @@ -0,0 +1,16 @@ +================================================================================================ +range +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_161-b12 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz + +range: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +full scan 12674 / 12840 41.4 24.2 1.0X +limit after range 33 / 37 15900.2 0.1 384.4X +filter after range 969 / 985 541.0 1.8 13.1X +count after range 42 / 42 12510.5 0.1 302.4X +count after limit after range 32 / 33 16337.0 0.1 394.9X + + diff --git a/sql/core/benchmarks/UDFBenchmark-results.txt b/sql/core/benchmarks/UDFBenchmark-results.txt new file mode 100644 index 0000000000000..3dfd0c1caeb28 --- /dev/null +++ b/sql/core/benchmarks/UDFBenchmark-results.txt @@ -0,0 +1,59 @@ +================================================================================================ +UDF with mixed input types +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int/string to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int/string to string wholestage off 194 248 76 0,5 1941,4 1,0X +long/nullable int/string to string wholestage on 127 136 8 0,8 1269,5 1,5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int/string to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int/string to option wholestage off 91 97 8 1,1 910,1 1,0X +long/nullable int/string to option wholestage on 60 79 29 1,7 603,8 1,5X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int/string to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int/string to primitive wholestage off 55 63 12 1,8 547,9 1,0X +long/nullable int/string to primitive wholestage on 43 44 2 2,3 428,0 1,3X + + +================================================================================================ +UDF with primitive types +================================================================================================ + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int to string: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int to string wholestage off 46 48 2 2,2 461,2 1,0X +long/nullable int to string wholestage on 49 56 8 2,0 488,9 0,9X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int to option: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int to option wholestage off 41 47 9 2,4 408,2 1,0X +long/nullable int to option wholestage on 26 28 2 3,9 256,7 1,6X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +long/nullable int to primitive: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +long/nullable int to primitive wholestage off 26 27 0 3,8 263,7 1,0X +long/nullable int to primitive wholestage on 26 31 5 3,8 262,2 1,0X + +Java HotSpot(TM) 64-Bit Server VM 1.8.0_152-b16 on Mac OS X 10.13.6 +Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz +UDF identity overhead: Best Time(ms) Avg Time(ms) Stdev(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------------------------------ +Baseline 20 22 1 4,9 204,3 1,0X +With identity UDF 24 26 2 4,1 241,3 0,8X + + diff --git a/sql/core/benchmarks/WideSchemaBenchmark-results.txt b/sql/core/benchmarks/WideSchemaBenchmark-results.txt index 0b9f791ac85e4..6347a6ac6b67c 100644 --- a/sql/core/benchmarks/WideSchemaBenchmark-results.txt +++ b/sql/core/benchmarks/WideSchemaBenchmark-results.txt @@ -1,117 +1,145 @@ -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +================================================================================================ +parsing large select expressions +================================================================================================ +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz parsing large select: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 select expressions 2 / 4 0.0 2050147.0 1.0X -100 select expressions 6 / 7 0.0 6123412.0 0.3X -2500 select expressions 135 / 141 0.0 134623148.0 0.0X +1 select expressions 6 / 13 0.0 5997373.0 1.0X +100 select expressions 7 / 10 0.0 7204596.0 0.8X +2500 select expressions 103 / 107 0.0 102962705.0 0.1X -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +================================================================================================ +many column field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz many column field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 cols x 100000 rows (read in-mem) 16 / 18 6.3 158.6 1.0X -1 cols x 100000 rows (exec in-mem) 17 / 19 6.0 166.7 1.0X -1 cols x 100000 rows (read parquet) 24 / 26 4.3 235.1 0.7X -1 cols x 100000 rows (write parquet) 81 / 85 1.2 811.3 0.2X -100 cols x 1000 rows (read in-mem) 17 / 19 6.0 166.2 1.0X -100 cols x 1000 rows (exec in-mem) 25 / 27 4.0 249.2 0.6X -100 cols x 1000 rows (read parquet) 23 / 25 4.4 226.0 0.7X -100 cols x 1000 rows (write parquet) 83 / 87 1.2 831.0 0.2X -2500 cols x 40 rows (read in-mem) 132 / 137 0.8 1322.9 0.1X -2500 cols x 40 rows (exec in-mem) 326 / 330 0.3 3260.6 0.0X -2500 cols x 40 rows (read parquet) 831 / 839 0.1 8305.8 0.0X -2500 cols x 40 rows (write parquet) 237 / 245 0.4 2372.6 0.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +1 cols x 100000 rows (read in-mem) 40 / 51 2.5 396.5 1.0X +1 cols x 100000 rows (exec in-mem) 41 / 48 2.4 414.4 1.0X +1 cols x 100000 rows (read parquet) 61 / 70 1.6 610.2 0.6X +1 cols x 100000 rows (write parquet) 209 / 233 0.5 2086.1 0.2X +100 cols x 1000 rows (read in-mem) 43 / 49 2.3 433.8 0.9X +100 cols x 1000 rows (exec in-mem) 57 / 66 1.8 568.4 0.7X +100 cols x 1000 rows (read parquet) 60 / 66 1.7 599.0 0.7X +100 cols x 1000 rows (write parquet) 212 / 224 0.5 2120.6 0.2X +2500 cols x 40 rows (read in-mem) 268 / 275 0.4 2676.5 0.1X +2500 cols x 40 rows (exec in-mem) 494 / 504 0.2 4936.9 0.1X +2500 cols x 40 rows (read parquet) 132 / 139 0.8 1319.7 0.3X +2500 cols x 40 rows (write parquet) 371 / 381 0.3 3710.1 0.1X + + +================================================================================================ +wide shallowly nested struct field read and write +================================================================================================ +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide shallowly nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 15 / 17 6.6 151.0 1.0X -1 wide x 100000 rows (exec in-mem) 20 / 22 5.1 196.6 0.8X -1 wide x 100000 rows (read parquet) 59 / 63 1.7 592.8 0.3X -1 wide x 100000 rows (write parquet) 81 / 87 1.2 814.6 0.2X -100 wide x 1000 rows (read in-mem) 21 / 25 4.8 208.7 0.7X -100 wide x 1000 rows (exec in-mem) 72 / 81 1.4 718.5 0.2X -100 wide x 1000 rows (read parquet) 75 / 85 1.3 752.6 0.2X -100 wide x 1000 rows (write parquet) 88 / 95 1.1 876.7 0.2X -2500 wide x 40 rows (read in-mem) 28 / 34 3.5 282.2 0.5X -2500 wide x 40 rows (exec in-mem) 1269 / 1284 0.1 12688.1 0.0X -2500 wide x 40 rows (read parquet) 549 / 578 0.2 5493.4 0.0X -2500 wide x 40 rows (write parquet) 96 / 104 1.0 959.1 0.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +1 wide x 100000 rows (read in-mem) 37 / 43 2.7 373.6 1.0X +1 wide x 100000 rows (exec in-mem) 47 / 54 2.1 472.7 0.8X +1 wide x 100000 rows (read parquet) 132 / 145 0.8 1316.5 0.3X +1 wide x 100000 rows (write parquet) 205 / 232 0.5 2046.3 0.2X +100 wide x 1000 rows (read in-mem) 68 / 79 1.5 676.3 0.6X +100 wide x 1000 rows (exec in-mem) 88 / 97 1.1 882.2 0.4X +100 wide x 1000 rows (read parquet) 197 / 234 0.5 1971.8 0.2X +100 wide x 1000 rows (write parquet) 236 / 249 0.4 2359.6 0.2X +2500 wide x 40 rows (read in-mem) 77 / 85 1.3 768.0 0.5X +2500 wide x 40 rows (exec in-mem) 386 / 393 0.3 3855.2 0.1X +2500 wide x 40 rows (read parquet) 1741 / 1765 0.1 17408.3 0.0X +2500 wide x 40 rows (write parquet) 243 / 256 0.4 2425.2 0.2X + +================================================================================================ +deeply nested struct field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz deeply nested struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 deep x 100000 rows (read in-mem) 14 / 16 7.0 143.8 1.0X -1 deep x 100000 rows (exec in-mem) 17 / 19 5.9 169.7 0.8X -1 deep x 100000 rows (read parquet) 33 / 35 3.1 327.0 0.4X -1 deep x 100000 rows (write parquet) 79 / 84 1.3 786.9 0.2X -100 deep x 1000 rows (read in-mem) 21 / 24 4.7 211.3 0.7X -100 deep x 1000 rows (exec in-mem) 221 / 235 0.5 2214.5 0.1X -100 deep x 1000 rows (read parquet) 1928 / 1952 0.1 19277.1 0.0X -100 deep x 1000 rows (write parquet) 91 / 96 1.1 909.5 0.2X -250 deep x 400 rows (read in-mem) 57 / 61 1.8 567.1 0.3X -250 deep x 400 rows (exec in-mem) 1329 / 1385 0.1 13291.8 0.0X -250 deep x 400 rows (read parquet) 36563 / 36750 0.0 365630.2 0.0X -250 deep x 400 rows (write parquet) 126 / 130 0.8 1262.0 0.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +1 deep x 100000 rows (read in-mem) 35 / 42 2.9 350.2 1.0X +1 deep x 100000 rows (exec in-mem) 40 / 43 2.5 399.5 0.9X +1 deep x 100000 rows (read parquet) 69 / 73 1.4 691.6 0.5X +1 deep x 100000 rows (write parquet) 203 / 224 0.5 2025.9 0.2X +100 deep x 1000 rows (read in-mem) 70 / 75 1.4 703.7 0.5X +100 deep x 1000 rows (exec in-mem) 654 / 684 0.2 6539.9 0.1X +100 deep x 1000 rows (read parquet) 10503 / 10550 0.0 105030.5 0.0X +100 deep x 1000 rows (write parquet) 235 / 243 0.4 2353.2 0.1X +250 deep x 400 rows (read in-mem) 249 / 259 0.4 2492.6 0.1X +250 deep x 400 rows (exec in-mem) 3842 / 3854 0.0 38424.8 0.0X +250 deep x 400 rows (read parquet) 153080 / 153444 0.0 1530796.1 0.0X +250 deep x 400 rows (write parquet) 434 / 441 0.2 4344.6 0.1X + +================================================================================================ +bushy struct field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz bushy struct field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 x 1 deep x 100000 rows (read in-mem) 13 / 15 7.8 127.7 1.0X -1 x 1 deep x 100000 rows (exec in-mem) 15 / 17 6.6 151.5 0.8X -1 x 1 deep x 100000 rows (read parquet) 20 / 23 5.0 198.3 0.6X -1 x 1 deep x 100000 rows (write parquet) 77 / 82 1.3 770.4 0.2X -128 x 8 deep x 1000 rows (read in-mem) 12 / 14 8.2 122.5 1.0X -128 x 8 deep x 1000 rows (exec in-mem) 124 / 140 0.8 1241.2 0.1X -128 x 8 deep x 1000 rows (read parquet) 69 / 74 1.4 693.9 0.2X -128 x 8 deep x 1000 rows (write parquet) 78 / 83 1.3 777.7 0.2X -1024 x 11 deep x 100 rows (read in-mem) 25 / 29 4.1 246.1 0.5X -1024 x 11 deep x 100 rows (exec in-mem) 1197 / 1223 0.1 11974.6 0.0X -1024 x 11 deep x 100 rows (read parquet) 426 / 433 0.2 4263.7 0.0X -1024 x 11 deep x 100 rows (write parquet) 91 / 98 1.1 913.5 0.1X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +1 x 1 deep x 100000 rows (read in-mem) 37 / 42 2.7 370.2 1.0X +1 x 1 deep x 100000 rows (exec in-mem) 43 / 47 2.4 425.3 0.9X +1 x 1 deep x 100000 rows (read parquet) 48 / 51 2.1 478.7 0.8X +1 x 1 deep x 100000 rows (write parquet) 204 / 215 0.5 2042.0 0.2X +128 x 8 deep x 1000 rows (read in-mem) 32 / 37 3.1 318.6 1.2X +128 x 8 deep x 1000 rows (exec in-mem) 91 / 96 1.1 906.6 0.4X +128 x 8 deep x 1000 rows (read parquet) 351 / 379 0.3 3510.3 0.1X +128 x 8 deep x 1000 rows (write parquet) 199 / 203 0.5 1988.3 0.2X +1024 x 11 deep x 100 rows (read in-mem) 73 / 76 1.4 730.4 0.5X +1024 x 11 deep x 100 rows (exec in-mem) 327 / 334 0.3 3267.2 0.1X +1024 x 11 deep x 100 rows (read parquet) 2063 / 2078 0.0 20629.2 0.0X +1024 x 11 deep x 100 rows (write parquet) 248 / 266 0.4 2475.1 0.1X + + +================================================================================================ +wide array field read and write +================================================================================================ +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide array field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 14 / 16 7.0 143.2 1.0X -1 wide x 100000 rows (exec in-mem) 17 / 19 5.9 170.9 0.8X -1 wide x 100000 rows (read parquet) 43 / 46 2.3 434.1 0.3X -1 wide x 100000 rows (write parquet) 78 / 83 1.3 777.6 0.2X -100 wide x 1000 rows (read in-mem) 11 / 13 9.0 111.5 1.3X -100 wide x 1000 rows (exec in-mem) 13 / 15 7.8 128.3 1.1X -100 wide x 1000 rows (read parquet) 24 / 27 4.1 245.0 0.6X -100 wide x 1000 rows (write parquet) 74 / 80 1.4 740.5 0.2X -2500 wide x 40 rows (read in-mem) 11 / 13 9.1 109.5 1.3X -2500 wide x 40 rows (exec in-mem) 13 / 15 7.7 129.4 1.1X -2500 wide x 40 rows (read parquet) 24 / 26 4.1 241.3 0.6X -2500 wide x 40 rows (write parquet) 75 / 81 1.3 751.8 0.2X - -Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.11.6 -Intel(R) Core(TM) i7-4980HQ CPU @ 2.80GHz +1 wide x 100000 rows (read in-mem) 33 / 38 3.0 328.4 1.0X +1 wide x 100000 rows (exec in-mem) 40 / 44 2.5 402.7 0.8X +1 wide x 100000 rows (read parquet) 83 / 91 1.2 826.6 0.4X +1 wide x 100000 rows (write parquet) 204 / 218 0.5 2039.1 0.2X +100 wide x 1000 rows (read in-mem) 28 / 31 3.6 277.2 1.2X +100 wide x 1000 rows (exec in-mem) 34 / 37 2.9 343.2 1.0X +100 wide x 1000 rows (read parquet) 56 / 61 1.8 556.4 0.6X +100 wide x 1000 rows (write parquet) 202 / 206 0.5 2017.3 0.2X +2500 wide x 40 rows (read in-mem) 29 / 30 3.5 286.4 1.1X +2500 wide x 40 rows (exec in-mem) 33 / 39 3.0 330.2 1.0X +2500 wide x 40 rows (read parquet) 54 / 66 1.8 544.0 0.6X +2500 wide x 40 rows (write parquet) 196 / 208 0.5 1959.2 0.2X + +================================================================================================ +wide map field read and write +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz wide map field r/w: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -1 wide x 100000 rows (read in-mem) 16 / 18 6.2 162.6 1.0X -1 wide x 100000 rows (exec in-mem) 21 / 23 4.8 208.2 0.8X -1 wide x 100000 rows (read parquet) 54 / 59 1.8 543.6 0.3X -1 wide x 100000 rows (write parquet) 80 / 86 1.2 804.5 0.2X -100 wide x 1000 rows (read in-mem) 11 / 13 8.7 114.5 1.4X -100 wide x 1000 rows (exec in-mem) 14 / 16 7.0 143.5 1.1X -100 wide x 1000 rows (read parquet) 30 / 32 3.3 300.4 0.5X -100 wide x 1000 rows (write parquet) 75 / 80 1.3 749.9 0.2X -2500 wide x 40 rows (read in-mem) 13 / 15 7.8 128.1 1.3X -2500 wide x 40 rows (exec in-mem) 15 / 18 6.5 153.6 1.1X -2500 wide x 40 rows (read parquet) 30 / 33 3.3 304.4 0.5X -2500 wide x 40 rows (write parquet) 77 / 83 1.3 768.5 0.2X +1 wide x 100000 rows (read in-mem) 31 / 34 3.3 305.7 1.0X +1 wide x 100000 rows (exec in-mem) 39 / 44 2.6 390.0 0.8X +1 wide x 100000 rows (read parquet) 125 / 132 0.8 1250.5 0.2X +1 wide x 100000 rows (write parquet) 198 / 213 0.5 1979.9 0.2X +100 wide x 1000 rows (read in-mem) 21 / 23 4.7 212.7 1.4X +100 wide x 1000 rows (exec in-mem) 28 / 32 3.5 283.3 1.1X +100 wide x 1000 rows (read parquet) 68 / 73 1.5 683.0 0.4X +100 wide x 1000 rows (write parquet) 188 / 206 0.5 1882.1 0.2X +2500 wide x 40 rows (read in-mem) 25 / 28 4.0 252.2 1.2X +2500 wide x 40 rows (exec in-mem) 32 / 34 3.1 318.5 1.0X +2500 wide x 40 rows (read parquet) 69 / 73 1.4 691.5 0.4X +2500 wide x 40 rows (write parquet) 193 / 202 0.5 1932.8 0.2X + diff --git a/sql/core/benchmarks/WideTableBenchmark-results.txt b/sql/core/benchmarks/WideTableBenchmark-results.txt new file mode 100644 index 0000000000000..8c09f9ca11307 --- /dev/null +++ b/sql/core/benchmarks/WideTableBenchmark-results.txt @@ -0,0 +1,17 @@ +================================================================================================ +projection on wide table +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +projection on wide table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +split threshold 10 40571 / 40937 0.0 38691.7 1.0X +split threshold 100 31116 / 31669 0.0 29674.6 1.3X +split threshold 1024 10077 / 10199 0.1 9609.7 4.0X +split threshold 2048 8654 / 8692 0.1 8253.2 4.7X +split threshold 4096 8006 / 8038 0.1 7634.7 5.1X +split threshold 8192 8069 / 8107 0.1 7695.3 5.0X +split threshold 65536 56973 / 57204 0.0 54333.7 0.7X + + diff --git a/sql/core/pom.xml b/sql/core/pom.xml index 2f72ff6cfdbfb..02ed6f8adaa62 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-sql_2.11 + spark-sql_2.12 jar Spark Project SQL http://spark.apache.org/ @@ -112,13 +112,9 @@ com.fasterxml.jackson.core jackson-databind - - org.apache.arrow - arrow-vector - org.apache.xbean - xbean-asm6-shaded + xbean-asm7-shaded org.scalacheck @@ -189,6 +185,19 @@ org.codehaus.mojo build-helper-maven-plugin + + add-sources + generate-sources + + add-source + + + + v${hive.version.short}/src/main/scala + v${hive.version.short}/src/main/java + + + add-scala-test-sources generate-test-sources @@ -197,6 +206,7 @@ + v${hive.version.short}/src/test/scala src/test/gen-java diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java index 802949c0ddb60..d4e1d89491f43 100644 --- a/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java +++ b/sql/core/src/main/java/org/apache/spark/api/java/function/FlatMapGroupsWithStateFunction.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.streaming.GroupState; /** @@ -33,7 +33,7 @@ * @since 2.1.1 */ @Experimental -@InterfaceStability.Evolving +@Evolving public interface FlatMapGroupsWithStateFunction extends Serializable { Iterator call(K key, Iterator values, GroupState state) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java index 353e9886a8a57..f0abfde843cc5 100644 --- a/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java +++ b/sql/core/src/main/java/org/apache/spark/api/java/function/MapGroupsWithStateFunction.java @@ -20,8 +20,8 @@ import java.io.Serializable; import java.util.Iterator; +import org.apache.spark.annotation.Evolving; import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.sql.streaming.GroupState; /** @@ -32,7 +32,7 @@ * @since 2.1.1 */ @Experimental -@InterfaceStability.Evolving +@Evolving public interface MapGroupsWithStateFunction extends Serializable { R call(K key, Iterator values, GroupState state) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java index 1c3c9794fb6bb..9cc073f53a3eb 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java +++ b/sql/core/src/main/java/org/apache/spark/sql/SaveMode.java @@ -16,14 +16,14 @@ */ package org.apache.spark.sql; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * SaveMode is used to specify the expected behavior of saving a DataFrame to a data source. * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable public enum SaveMode { /** * Append mode means that when saving a DataFrame to a data source, if data/table already exists, diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java index 4eeb7be3f5abb..631d6eb1cfb03 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF0.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 0 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF0 extends Serializable { R call() throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java index 1460daf27dc20..a5d01406edd8c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF1.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 1 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF1 extends Serializable { R call(T1 t1) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java index 7c4f1e4897084..effe99e30b2a5 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF10.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 10 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF10 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java index 26a05106aebd6..e70b18b84b08f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF11.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 11 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF11 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java index 8ef7a99042025..339feb34135e1 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF12.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 12 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF12 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java index 5c3b2ec1222e2..d346e5c908c6f 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF13.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 13 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF13 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java index 97e744d843466..d27f9f5270f4b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF14.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 14 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF14 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java index 7ddbf914fc11a..b99b57a91d465 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF15.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 15 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF15 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java index 0ae5dc7195ad6..7899fc4b7ad65 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF16.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 16 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF16 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java index 03543a556c614..40a7e95724fc2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF17.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 17 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF17 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java index 46740d3443916..47935a935891c 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF18.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 18 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF18 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java index 33fefd8ecaf1d..578b796ff03a3 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF19.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 19 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF19 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java index 9822f19217d76..2f856aa3cf630 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF2.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 2 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF2 extends Serializable { R call(T1 t1, T2 t2) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java index 8c5e90182da1c..aa8a9fa897040 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF20.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 20 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF20 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java index e3b09f5167cff..0fe52bce2eca2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF21.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 21 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF21 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java index dc6cfa9097bab..69fd8ca422833 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF22.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 22 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF22 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9, T10 t10, T11 t11, T12 t12, T13 t13, T14 t14, T15 t15, T16 t16, T17 t17, T18 t18, T19 t19, T20 t20, T21 t21, T22 t22) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java index 7c264b69ba195..84ffd655672a2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF3.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 3 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF3 extends Serializable { R call(T1 t1, T2 t2, T3 t3) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java index 58df38fc3c911..dd2dc285c226d 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF4.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 4 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF4 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java index 4146f96e2eed5..795cc21c3f76e 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF5.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 5 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF5 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java index 25d39654c1095..a954684c3c9a9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF6.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 6 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF6 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java index ce63b6a91adbb..03761f2c9ebbf 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF7.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 7 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF7 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java index 0e00209ef6b9f..8cd3583b2cbf0 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF8.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 8 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF8 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8) throws Exception; } diff --git a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java index 077981bb3e3ee..78a7097791963 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java +++ b/sql/core/src/main/java/org/apache/spark/sql/api/java/UDF9.java @@ -19,12 +19,12 @@ import java.io.Serializable; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Stable; /** * A Spark SQL UDF that has 9 arguments. */ -@InterfaceStability.Stable +@Stable public interface UDF9 extends Serializable { R call(T1 t1, T2 t2, T3 t3, T4 t4, T5 t5, T6 t6, T7 t7, T8 t8, T9 t9) throws Exception; } diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java b/sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java similarity index 100% rename from sql/catalyst/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java rename to sql/core/src/main/java/org/apache/spark/sql/execution/RecordBinaryComparator.java diff --git a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java similarity index 95% rename from sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java rename to sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java index 1b2f5eee5ccdd..863d80b5cb9c5 100644 --- a/sql/catalyst/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeExternalRowSorter.java @@ -40,7 +40,6 @@ public final class UnsafeExternalRowSorter { - static final int DEFAULT_INITIAL_SORT_BUFFER_SIZE = 4096; /** * If positive, forces records to be spilled to disk at the given frequency (measured in numbers * of records). This is only intended to be used in tests. @@ -50,7 +49,7 @@ public final class UnsafeExternalRowSorter { private long numRowsInserted = 0; private final StructType schema; - private final PrefixComputer prefixComputer; + private final UnsafeExternalRowSorter.PrefixComputer prefixComputer; private final UnsafeExternalSorter sorter; public abstract static class PrefixComputer { @@ -74,7 +73,7 @@ public static UnsafeExternalRowSorter createWithRecordComparator( StructType schema, Supplier recordComparatorSupplier, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, boolean canUseRadixSort) throws IOException { return new UnsafeExternalRowSorter(schema, recordComparatorSupplier, prefixComparator, @@ -85,7 +84,7 @@ public static UnsafeExternalRowSorter create( StructType schema, Ordering ordering, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, boolean canUseRadixSort) throws IOException { Supplier recordComparatorSupplier = @@ -98,9 +97,9 @@ private UnsafeExternalRowSorter( StructType schema, Supplier recordComparatorSupplier, PrefixComparator prefixComparator, - PrefixComputer prefixComputer, + UnsafeExternalRowSorter.PrefixComputer prefixComputer, long pageSizeBytes, - boolean canUseRadixSort) throws IOException { + boolean canUseRadixSort) { this.schema = schema; this.prefixComputer = prefixComputer; final SparkEnv sparkEnv = SparkEnv.get(); @@ -112,8 +111,7 @@ private UnsafeExternalRowSorter( taskContext, recordComparatorSupplier, prefixComparator, - sparkEnv.conf().getInt("spark.shuffle.sort.initialBufferSize", - DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) (long) sparkEnv.conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, (int) SparkEnv.get().conf().get( package$.MODULE$.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD()), diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java index c8cf44b51df77..117e98f33a0ec 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeFixedWidthAggregationMap.java @@ -98,7 +98,7 @@ public UnsafeFixedWidthAggregationMap( this.groupingKeyProjection = UnsafeProjection.create(groupingKeySchema); this.groupingKeySchema = groupingKeySchema; this.map = new BytesToBytesMap( - taskContext.taskMemoryManager(), initialCapacity, pageSizeBytes, true); + taskContext.taskMemoryManager(), initialCapacity, pageSizeBytes); // Initialize the buffer for aggregation value final UnsafeProjection valueProjection = UnsafeProjection.create(aggregationBufferSchema); @@ -226,10 +226,10 @@ public void free() { } /** - * Gets the average hash map probe per looking up for the underlying `BytesToBytesMap`. + * Gets the average bucket list iterations per lookup in the underlying `BytesToBytesMap`. */ - public double getAverageProbesPerLookup() { - return map.getAverageProbesPerLookup(); + public double getAvgHashProbeBucketListIterations() { + return map.getAvgHashProbeBucketListIterations(); } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java index 9eb03430a7db2..09426117a24b9 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/UnsafeKVExternalSorter.java @@ -25,6 +25,7 @@ import org.apache.spark.SparkEnv; import org.apache.spark.TaskContext; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.serializer.SerializerManager; import org.apache.spark.sql.catalyst.expressions.UnsafeRow; @@ -93,8 +94,7 @@ public UnsafeKVExternalSorter( taskContext, comparatorSupplier, prefixComparator, - SparkEnv.get().conf().getInt("spark.shuffle.sort.initialBufferSize", - UnsafeExternalRowSorter.DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) (long) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, numElementsForSpillThreshold, canUseRadixSort); @@ -160,8 +160,7 @@ public UnsafeKVExternalSorter( taskContext, comparatorSupplier, prefixComparator, - SparkEnv.get().conf().getInt("spark.shuffle.sort.initialBufferSize", - UnsafeExternalRowSorter.DEFAULT_INITIAL_SORT_BUFFER_SIZE), + (int) (long) SparkEnv.get().conf().get(package$.MODULE$.SHUFFLE_SORT_INIT_BUFFER_SIZE()), pageSizeBytes, numElementsForSpillThreshold, inMemSorter); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java index 82a1169cbe7ae..7d1fbe64fc960 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/SchemaColumnConvertNotSupportedException.java @@ -17,12 +17,12 @@ package org.apache.spark.sql.execution.datasources; -import org.apache.spark.annotation.InterfaceStability; +import org.apache.spark.annotation.Unstable; /** * Exception thrown when the parquet reader find column type mismatches. */ -@InterfaceStability.Unstable +@Unstable public class SchemaColumnConvertNotSupportedException extends RuntimeException { /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java index a0d9578a377b1..6a4b116cdef0b 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReader.java @@ -18,8 +18,8 @@ package org.apache.spark.sql.execution.datasources.orc; import java.io.IOException; -import java.util.stream.IntStream; +import com.google.common.annotations.VisibleForTesting; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.mapreduce.InputSplit; import org.apache.hadoop.mapreduce.RecordReader; @@ -30,16 +30,11 @@ import org.apache.orc.Reader; import org.apache.orc.TypeDescription; import org.apache.orc.mapred.OrcInputFormat; -import org.apache.orc.storage.common.type.HiveDecimal; -import org.apache.orc.storage.ql.exec.vector.*; -import org.apache.orc.storage.serde2.io.HiveDecimalWritable; -import org.apache.spark.memory.MemoryMode; import org.apache.spark.sql.catalyst.InternalRow; +import org.apache.spark.sql.execution.datasources.orc.OrcShimUtils.VectorizedRowBatchWrap; import org.apache.spark.sql.execution.vectorized.ColumnVectorUtils; -import org.apache.spark.sql.execution.vectorized.OffHeapColumnVector; import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.execution.vectorized.WritableColumnVector; import org.apache.spark.sql.types.*; import org.apache.spark.sql.vectorized.ColumnarBatch; @@ -53,14 +48,19 @@ public class OrcColumnarBatchReader extends RecordReader { // The capacity of vectorized batch. private int capacity; - // Vectorized ORC Row Batch - private VectorizedRowBatch batch; + // Vectorized ORC Row Batch wrap. + private VectorizedRowBatchWrap wrap; /** * The column IDs of the physical ORC file schema which are required by this reader. - * -1 means this required column doesn't exist in the ORC file. + * -1 means this required column is partition column, or it doesn't exist in the ORC file. + * Ideally partition column should never appear in the physical file, and should only appear + * in the directory name. However, Spark allows partition columns inside physical file, + * but Spark will discard the values from the file, and use the partition value got from + * directory name. The column order will be reserved though. */ - private int[] requestedColIds; + @VisibleForTesting + public int[] requestedDataColIds; // Record reader from ORC row batch. private org.apache.orc.RecordReader recordReader; @@ -68,23 +68,13 @@ public class OrcColumnarBatchReader extends RecordReader { private StructField[] requiredFields; // The result columnar batch for vectorized execution by whole-stage codegen. - private ColumnarBatch columnarBatch; + @VisibleForTesting + public ColumnarBatch columnarBatch; - // Writable column vectors of the result columnar batch. - private WritableColumnVector[] columnVectors; - - // The wrapped ORC column vectors. It should be null if `copyToSpark` is true. + // The wrapped ORC column vectors. private org.apache.spark.sql.vectorized.ColumnVector[] orcVectorWrappers; - // The memory mode of the columnarBatch - private final MemoryMode MEMORY_MODE; - - // Whether or not to copy the ORC columnar batch to Spark columnar batch. - private final boolean copyToSpark; - - public OrcColumnarBatchReader(boolean useOffHeap, boolean copyToSpark, int capacity) { - MEMORY_MODE = useOffHeap ? MemoryMode.OFF_HEAP : MemoryMode.ON_HEAP; - this.copyToSpark = copyToSpark; + public OrcColumnarBatchReader(int capacity) { this.capacity = capacity; } @@ -143,56 +133,46 @@ public void initialize( /** * Initialize columnar batch by setting required schema and partition information. * With this information, this creates ColumnarBatch with the full schema. + * + * @param orcSchema Schema from ORC file reader. + * @param requiredFields All the fields that are required to return, including partition fields. + * @param requestedDataColIds Requested column ids from orcSchema. -1 if not existed. + * @param requestedPartitionColIds Requested column ids from partition schema. -1 if not existed. + * @param partitionValues Values of partition columns. */ public void initBatch( TypeDescription orcSchema, - int[] requestedColIds, StructField[] requiredFields, - StructType partitionSchema, + int[] requestedDataColIds, + int[] requestedPartitionColIds, InternalRow partitionValues) { - batch = orcSchema.createRowBatch(capacity); - assert(!batch.selectedInUse); // `selectedInUse` should be initialized with `false`. - + wrap = new VectorizedRowBatchWrap(orcSchema.createRowBatch(capacity)); + assert(!wrap.batch().selectedInUse); // `selectedInUse` should be initialized with `false`. + assert(requiredFields.length == requestedDataColIds.length); + assert(requiredFields.length == requestedPartitionColIds.length); + // If a required column is also partition column, use partition value and don't read from file. + for (int i = 0; i < requiredFields.length; i++) { + if (requestedPartitionColIds[i] != -1) { + requestedDataColIds[i] = -1; + } + } this.requiredFields = requiredFields; - this.requestedColIds = requestedColIds; - assert(requiredFields.length == requestedColIds.length); + this.requestedDataColIds = requestedDataColIds; StructType resultSchema = new StructType(requiredFields); - for (StructField f : partitionSchema.fields()) { - resultSchema = resultSchema.add(f); - } - - if (copyToSpark) { - if (MEMORY_MODE == MemoryMode.OFF_HEAP) { - columnVectors = OffHeapColumnVector.allocateColumns(capacity, resultSchema); - } else { - columnVectors = OnHeapColumnVector.allocateColumns(capacity, resultSchema); - } - - // Initialize the missing columns once. - for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] == -1) { - columnVectors[i].putNulls(0, capacity); - columnVectors[i].setIsConstant(); - } - } - - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - ColumnVectorUtils.populate(columnVectors[i + partitionIdx], partitionValues, i); - columnVectors[i + partitionIdx].setIsConstant(); - } - } - columnarBatch = new ColumnarBatch(columnVectors); - } else { - // Just wrap the ORC column vector instead of copying it to Spark column vector. - orcVectorWrappers = new org.apache.spark.sql.vectorized.ColumnVector[resultSchema.length()]; + // Just wrap the ORC column vector instead of copying it to Spark column vector. + orcVectorWrappers = new org.apache.spark.sql.vectorized.ColumnVector[resultSchema.length()]; - for (int i = 0; i < requiredFields.length; i++) { - DataType dt = requiredFields[i].dataType(); - int colId = requestedColIds[i]; + for (int i = 0; i < requiredFields.length; i++) { + DataType dt = requiredFields[i].dataType(); + if (requestedPartitionColIds[i] != -1) { + OnHeapColumnVector partitionCol = new OnHeapColumnVector(capacity, dt); + ColumnVectorUtils.populate(partitionCol, partitionValues, requestedPartitionColIds[i]); + partitionCol.setIsConstant(); + orcVectorWrappers[i] = partitionCol; + } else { + int colId = requestedDataColIds[i]; // Initialize the missing columns once. if (colId == -1) { OnHeapColumnVector missingCol = new OnHeapColumnVector(capacity, dt); @@ -200,23 +180,12 @@ public void initBatch( missingCol.setIsConstant(); orcVectorWrappers[i] = missingCol; } else { - orcVectorWrappers[i] = new OrcColumnVector(dt, batch.cols[colId]); + orcVectorWrappers[i] = new OrcColumnVector(dt, wrap.batch().cols[colId]); } } - - if (partitionValues.numFields() > 0) { - int partitionIdx = requiredFields.length; - for (int i = 0; i < partitionValues.numFields(); i++) { - DataType dt = partitionSchema.fields()[i].dataType(); - OnHeapColumnVector partitionCol = new OnHeapColumnVector(capacity, dt); - ColumnVectorUtils.populate(partitionCol, partitionValues, i); - partitionCol.setIsConstant(); - orcVectorWrappers[partitionIdx + i] = partitionCol; - } - } - - columnarBatch = new ColumnarBatch(orcVectorWrappers); } + + columnarBatch = new ColumnarBatch(orcVectorWrappers); } /** @@ -224,332 +193,18 @@ public void initBatch( * by copying from ORC VectorizedRowBatch columns to Spark ColumnarBatch columns. */ private boolean nextBatch() throws IOException { - recordReader.nextBatch(batch); - int batchSize = batch.size; + recordReader.nextBatch(wrap.batch()); + int batchSize = wrap.batch().size; if (batchSize == 0) { return false; } columnarBatch.setNumRows(batchSize); - if (!copyToSpark) { - for (int i = 0; i < requiredFields.length; i++) { - if (requestedColIds[i] != -1) { - ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); - } - } - return true; - } - - for (WritableColumnVector vector : columnVectors) { - vector.reset(); - } - for (int i = 0; i < requiredFields.length; i++) { - StructField field = requiredFields[i]; - WritableColumnVector toColumn = columnVectors[i]; - - if (requestedColIds[i] >= 0) { - ColumnVector fromColumn = batch.cols[requestedColIds[i]]; - - if (fromColumn.isRepeating) { - putRepeatingValues(batchSize, field, fromColumn, toColumn); - } else if (fromColumn.noNulls) { - putNonNullValues(batchSize, field, fromColumn, toColumn); - } else { - putValues(batchSize, field, fromColumn, toColumn); - } + if (requestedDataColIds[i] != -1) { + ((OrcColumnVector) orcVectorWrappers[i]).setBatchSize(batchSize); } } return true; } - - private void putRepeatingValues( - int batchSize, - StructField field, - ColumnVector fromColumn, - WritableColumnVector toColumn) { - if (fromColumn.isNull[0]) { - toColumn.putNulls(0, batchSize); - } else { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - toColumn.putBooleans(0, batchSize, ((LongColumnVector)fromColumn).vector[0] == 1); - } else if (type instanceof ByteType) { - toColumn.putBytes(0, batchSize, (byte)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof ShortType) { - toColumn.putShorts(0, batchSize, (short)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof IntegerType || type instanceof DateType) { - toColumn.putInts(0, batchSize, (int)((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof LongType) { - toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector[0]); - } else if (type instanceof TimestampType) { - toColumn.putLongs(0, batchSize, - fromTimestampColumnVector((TimestampColumnVector)fromColumn, 0)); - } else if (type instanceof FloatType) { - toColumn.putFloats(0, batchSize, (float)((DoubleColumnVector)fromColumn).vector[0]); - } else if (type instanceof DoubleType) { - toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector[0]); - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = (BytesColumnVector)fromColumn; - int size = data.vector[0].length; - toColumn.arrayData().reserve(size); - toColumn.arrayData().putBytes(0, size, data.vector[0], 0); - for (int index = 0; index < batchSize; index++) { - toColumn.putArray(index, 0, size); - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - putDecimalWritables( - toColumn, - batchSize, - decimalType.precision(), - decimalType.scale(), - ((DecimalColumnVector)fromColumn).vector[0]); - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } - } - } - - private void putNonNullValues( - int batchSize, - StructField field, - ColumnVector fromColumn, - WritableColumnVector toColumn) { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putBoolean(index, data[index] == 1); - } - } else if (type instanceof ByteType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putByte(index, (byte)data[index]); - } - } else if (type instanceof ShortType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putShort(index, (short)data[index]); - } - } else if (type instanceof IntegerType || type instanceof DateType) { - long[] data = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putInt(index, (int)data[index]); - } - } else if (type instanceof LongType) { - toColumn.putLongs(0, batchSize, ((LongColumnVector)fromColumn).vector, 0); - } else if (type instanceof TimestampType) { - TimestampColumnVector data = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - toColumn.putLong(index, fromTimestampColumnVector(data, index)); - } - } else if (type instanceof FloatType) { - double[] data = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - toColumn.putFloat(index, (float)data[index]); - } - } else if (type instanceof DoubleType) { - toColumn.putDoubles(0, batchSize, ((DoubleColumnVector)fromColumn).vector, 0); - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector data = ((BytesColumnVector)fromColumn); - WritableColumnVector arrayData = toColumn.arrayData(); - int totalNumBytes = IntStream.of(data.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < batchSize; pos += data.length[index], index++) { - arrayData.putBytes(pos, data.length[index], data.vector[index], data.start[index]); - toColumn.putArray(index, pos, data.length[index]); - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - DecimalColumnVector data = ((DecimalColumnVector)fromColumn); - if (decimalType.precision() > Decimal.MAX_LONG_DIGITS()) { - toColumn.arrayData().reserve(batchSize * 16); - } - for (int index = 0; index < batchSize; index++) { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - data.vector[index]); - } - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } - } - - private void putValues( - int batchSize, - StructField field, - ColumnVector fromColumn, - WritableColumnVector toColumn) { - DataType type = field.dataType(); - if (type instanceof BooleanType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putBoolean(index, vector[index] == 1); - } - } - } else if (type instanceof ByteType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putByte(index, (byte)vector[index]); - } - } - } else if (type instanceof ShortType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putShort(index, (short)vector[index]); - } - } - } else if (type instanceof IntegerType || type instanceof DateType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putInt(index, (int)vector[index]); - } - } - } else if (type instanceof LongType) { - long[] vector = ((LongColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, vector[index]); - } - } - } else if (type instanceof TimestampType) { - TimestampColumnVector vector = ((TimestampColumnVector)fromColumn); - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putLong(index, fromTimestampColumnVector(vector, index)); - } - } - } else if (type instanceof FloatType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putFloat(index, (float)vector[index]); - } - } - } else if (type instanceof DoubleType) { - double[] vector = ((DoubleColumnVector)fromColumn).vector; - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - toColumn.putDouble(index, vector[index]); - } - } - } else if (type instanceof StringType || type instanceof BinaryType) { - BytesColumnVector vector = (BytesColumnVector)fromColumn; - WritableColumnVector arrayData = toColumn.arrayData(); - int totalNumBytes = IntStream.of(vector.length).sum(); - arrayData.reserve(totalNumBytes); - for (int index = 0, pos = 0; index < batchSize; pos += vector.length[index], index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - arrayData.putBytes(pos, vector.length[index], vector.vector[index], vector.start[index]); - toColumn.putArray(index, pos, vector.length[index]); - } - } - } else if (type instanceof DecimalType) { - DecimalType decimalType = (DecimalType)type; - HiveDecimalWritable[] vector = ((DecimalColumnVector)fromColumn).vector; - if (decimalType.precision() > Decimal.MAX_LONG_DIGITS()) { - toColumn.arrayData().reserve(batchSize * 16); - } - for (int index = 0; index < batchSize; index++) { - if (fromColumn.isNull[index]) { - toColumn.putNull(index); - } else { - putDecimalWritable( - toColumn, - index, - decimalType.precision(), - decimalType.scale(), - vector[index]); - } - } - } else { - throw new UnsupportedOperationException("Unsupported Data Type: " + type); - } - } - - /** - * Returns the number of micros since epoch from an element of TimestampColumnVector. - */ - private static long fromTimestampColumnVector(TimestampColumnVector vector, int index) { - return vector.time[index] * 1000 + (vector.nanos[index] / 1000 % 1000); - } - - /** - * Put a `HiveDecimalWritable` to a `WritableColumnVector`. - */ - private static void putDecimalWritable( - WritableColumnVector toColumn, - int index, - int precision, - int scale, - HiveDecimalWritable decimalWritable) { - HiveDecimal decimal = decimalWritable.getHiveDecimal(); - Decimal value = - Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); - value.changePrecision(precision, scale); - - if (precision <= Decimal.MAX_INT_DIGITS()) { - toColumn.putInt(index, (int) value.toUnscaledLong()); - } else if (precision <= Decimal.MAX_LONG_DIGITS()) { - toColumn.putLong(index, value.toUnscaledLong()); - } else { - byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); - toColumn.arrayData().putBytes(index * 16, bytes.length, bytes, 0); - toColumn.putArray(index, index * 16, bytes.length); - } - } - - /** - * Put `HiveDecimalWritable`s to a `WritableColumnVector`. - */ - private static void putDecimalWritables( - WritableColumnVector toColumn, - int size, - int precision, - int scale, - HiveDecimalWritable decimalWritable) { - HiveDecimal decimal = decimalWritable.getHiveDecimal(); - Decimal value = - Decimal.apply(decimal.bigDecimalValue(), decimal.precision(), decimal.scale()); - value.changePrecision(precision, scale); - - if (precision <= Decimal.MAX_INT_DIGITS()) { - toColumn.putInts(0, size, (int) value.toUnscaledLong()); - } else if (precision <= Decimal.MAX_LONG_DIGITS()) { - toColumn.putLongs(0, size, value.toUnscaledLong()); - } else { - byte[] bytes = value.toJavaBigDecimal().unscaledValue().toByteArray(); - toColumn.arrayData().reserve(bytes.length); - toColumn.arrayData().putBytes(0, bytes.length, bytes, 0); - for (int index = 0; index < size; index++) { - toColumn.putArray(index, 0, bytes.length); - } - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java similarity index 68% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java rename to sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java index c44b8af2552f0..7c167dc012329 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/BaseStreamingSource.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/streaming/Offset.java @@ -18,12 +18,10 @@ package org.apache.spark.sql.execution.streaming; /** - * The shared interface between V1 streaming sources and V2 streaming readers. + * This class is an alias of {@link org.apache.spark.sql.sources.v2.reader.streaming.Offset}. It's + * internal and deprecated. New streaming data source implementations should use data source v2 API, + * which will be supported in the long term. * - * This is a temporary interface for compatibility during migration. It should not be implemented - * directly, and will be removed in future versions. + * This class will be removed in a future release. */ -public interface BaseStreamingSource { - /** Stop this source and free any resources it has allocated. */ - void stop(); -} +public abstract class Offset extends org.apache.spark.sql.sources.v2.reader.streaming.Offset {} diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java index 4e4242fe8d9b9..fca7e36859126 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/MutableColumnarRow.java @@ -26,7 +26,6 @@ import org.apache.spark.sql.vectorized.ColumnarBatch; import org.apache.spark.sql.vectorized.ColumnarMap; import org.apache.spark.sql.vectorized.ColumnarRow; -import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.unsafe.types.CalendarInterval; import org.apache.spark.unsafe.types.UTF8String; @@ -39,17 +38,10 @@ */ public final class MutableColumnarRow extends InternalRow { public int rowId; - private final ColumnVector[] columns; - private final WritableColumnVector[] writableColumns; - - public MutableColumnarRow(ColumnVector[] columns) { - this.columns = columns; - this.writableColumns = null; - } + private final WritableColumnVector[] columns; public MutableColumnarRow(WritableColumnVector[] writableColumns) { this.columns = writableColumns; - this.writableColumns = writableColumns; } @Override @@ -228,54 +220,54 @@ public void update(int ordinal, Object value) { @Override public void setNullAt(int ordinal) { - writableColumns[ordinal].putNull(rowId); + columns[ordinal].putNull(rowId); } @Override public void setBoolean(int ordinal, boolean value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putBoolean(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putBoolean(rowId, value); } @Override public void setByte(int ordinal, byte value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putByte(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putByte(rowId, value); } @Override public void setShort(int ordinal, short value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putShort(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putShort(rowId, value); } @Override public void setInt(int ordinal, int value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putInt(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putInt(rowId, value); } @Override public void setLong(int ordinal, long value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putLong(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putLong(rowId, value); } @Override public void setFloat(int ordinal, float value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putFloat(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putFloat(rowId, value); } @Override public void setDouble(int ordinal, double value) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putDouble(rowId, value); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putDouble(rowId, value); } @Override public void setDecimal(int ordinal, Decimal value, int precision) { - writableColumns[ordinal].putNotNull(rowId); - writableColumns[ordinal].putDecimal(rowId, value, precision); + columns[ordinal].putNotNull(rowId); + columns[ordinal].putDecimal(rowId, value, precision); } } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java index 5e0cf7d370dd1..3b919c7d471f4 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OffHeapColumnVector.java @@ -417,7 +417,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 4L, count * 4L); } else { - ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); long offset = data + 4L * rowId; for (int i = 0; i < count; ++i, offset += 4) { Platform.putFloat(null, offset, bb.getFloat(srcIndex + (4 * i))); @@ -472,7 +472,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, null, data + rowId * 8L, count * 8L); } else { - ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); long offset = data + 8L * rowId; for (int i = 0; i < count; ++i, offset += 8) { Platform.putDouble(null, offset, bb.getDouble(srcIndex + (8 * i))); diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java index 577eab6ed14c8..1bf3126664177 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/OnHeapColumnVector.java @@ -396,7 +396,7 @@ public void putFloats(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, floatData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 4L, count * 4L); } else { - ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); for (int i = 0; i < count; ++i) { floatData[i + rowId] = bb.getFloat(srcIndex + (4 * i)); } @@ -445,7 +445,7 @@ public void putDoubles(int rowId, int count, byte[] src, int srcIndex) { Platform.copyMemory(src, Platform.BYTE_ARRAY_OFFSET + srcIndex, doubleData, Platform.DOUBLE_ARRAY_OFFSET + rowId * 8L, count * 8L); } else { - ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.LITTLE_ENDIAN); + ByteBuffer bb = ByteBuffer.wrap(src).order(ByteOrder.BIG_ENDIAN); for (int i = 0; i < count; ++i) { doubleData[i + rowId] = bb.getDouble(srcIndex + (8 * i)); } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java index b0e119d658cb4..14fac72847af2 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java +++ b/sql/core/src/main/java/org/apache/spark/sql/execution/vectorized/WritableColumnVector.java @@ -101,10 +101,11 @@ private void throwUnsupportedException(int requiredCapacity, Throwable cause) { String message = "Cannot reserve additional contiguous bytes in the vectorized reader (" + (requiredCapacity >= 0 ? "requested " + requiredCapacity + " bytes" : "integer overflow") + "). As a workaround, you can reduce the vectorized reader batch size, or disable the " + - "vectorized reader. For parquet file format, refer to " + + "vectorized reader, or disable " + SQLConf.BUCKETING_ENABLED().key() + " if you read " + + "from bucket table. For Parquet file format, refer to " + SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE().key() + " (default " + SQLConf.PARQUET_VECTORIZED_READER_BATCH_SIZE().defaultValueString() + - ") and " + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + "; for orc file format, " + + ") and " + SQLConf.PARQUET_VECTORIZED_READER_ENABLED().key() + "; for ORC file format, " + "refer to " + SQLConf.ORC_VECTORIZED_READER_BATCH_SIZE().key() + " (default " + SQLConf.ORC_VECTORIZED_READER_BATCH_SIZE().defaultValueString() + ") and " + SQLConf.ORC_VECTORIZED_READER_ENABLED().key() + "."; @@ -603,7 +604,10 @@ public final int appendArray(int length) { */ public final int appendStruct(boolean isNull) { if (isNull) { - appendNull(); + // This is the same as appendNull but without the assertion for struct types + reserve(elementsAppended + 1); + putNull(elementsAppended); + elementsAppended++; for (WritableColumnVector c: childColumns) { if (c.type instanceof StructType) { c.appendStruct(true); diff --git a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java index ec9c107b1c119..859b936acdf70 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java +++ b/sql/core/src/main/java/org/apache/spark/sql/expressions/javalang/typed.java @@ -17,8 +17,6 @@ package org.apache.spark.sql.expressions.javalang; -import org.apache.spark.annotation.Experimental; -import org.apache.spark.annotation.InterfaceStability; import org.apache.spark.api.java.function.MapFunction; import org.apache.spark.sql.TypedColumn; import org.apache.spark.sql.execution.aggregate.TypedAverage; @@ -33,9 +31,9 @@ * Scala users should use {@link org.apache.spark.sql.expressions.scalalang.typed}. * * @since 2.0.0 + * @deprecated As of release 3.0.0, please use the untyped builtin aggregate functions. */ -@Experimental -@InterfaceStability.Evolving +@Deprecated public class typed { // Note: make sure to keep in sync with typed.scala diff --git a/sql/core/src/main/java/org/apache/spark/sql/internal/NonClosableMutableURLClassLoader.java b/sql/core/src/main/java/org/apache/spark/sql/internal/NonClosableMutableURLClassLoader.java new file mode 100644 index 0000000000000..db77d9136aaba --- /dev/null +++ b/sql/core/src/main/java/org/apache/spark/sql/internal/NonClosableMutableURLClassLoader.java @@ -0,0 +1,39 @@ +/* + * 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.net.URL; + +import org.apache.spark.util.MutableURLClassLoader; + +/** + * This class loader cannot be closed (its `close` method is a no-op). + */ +public class NonClosableMutableURLClassLoader extends MutableURLClassLoader { + + static { + ClassLoader.registerAsParallelCapable(); + } + + public NonClosableMutableURLClassLoader(ClassLoader parent) { + super(new URL[]{}, parent); + } + + @Override + public void close() {} +} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java deleted file mode 100644 index f403dc619e86c..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchReadSupportProvider.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.BatchReadSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for batch processing. - * - * This interface is used to create {@link BatchReadSupport} instances when end users run - * {@code SparkSession.read.format(...).option(...).load()}. - */ -@InterfaceStability.Evolving -public interface BatchReadSupportProvider extends DataSourceV2 { - - /** - * Creates a {@link BatchReadSupport} instance to load the data from this data source with a user - * specified schema, which is called by Spark at the beginning of each batch query. - * - * Spark will call this method at the beginning of each batch query to create a - * {@link BatchReadSupport} instance. - * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. - * - * @param schema the user specified schema. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - default BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); - } - - /** - * Creates a {@link BatchReadSupport} instance to scan the data from this data source, which is - * called by Spark at the beginning of each batch query. - * - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - BatchReadSupport createBatchReadSupport(DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java deleted file mode 100644 index bd10c3353bf12..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/BatchWriteSupportProvider.java +++ /dev/null @@ -1,59 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import java.util.Optional; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.SaveMode; -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data writing ability for batch processing. - * - * This interface is used to create {@link BatchWriteSupport} instances when end users run - * {@code Dataset.write.format(...).option(...).save()}. - */ -@InterfaceStability.Evolving -public interface BatchWriteSupportProvider extends DataSourceV2 { - - /** - * Creates an optional {@link BatchWriteSupport} instance to save the data to this data source, - * which is called by Spark at the beginning of each batch query. - * - * Data sources can return None if there is no writing needed to be done according to the save - * mode. - * - * @param queryId A unique string for the writing query. It's possible that there are many - * writing queries running at the same time, and the returned - * {@link BatchWriteSupport} can use this id to distinguish itself from others. - * @param schema the schema of the data to be written. - * @param mode the save mode which determines what to do when the data are already in this data - * source, please refer to {@link SaveMode} for more details. - * @param options the options for the returned data source writer, which is an immutable - * case-insensitive string-to-string map. - * @return a write support to write data to this data source. - */ - Optional createBatchWriteSupport( - String queryId, - StructType schema, - SaveMode mode, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java deleted file mode 100644 index 824c290518acf..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/ContinuousReadSupportProvider.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for continuous stream processing. - * - * This interface is used to create {@link ContinuousReadSupport} instances when end users run - * {@code SparkSession.readStream.format(...).option(...).load()} with a continuous trigger. - */ -@InterfaceStability.Evolving -public interface ContinuousReadSupportProvider extends DataSourceV2 { - - /** - * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data - * source with a user specified schema, which is called by Spark at the beginning of each - * continuous streaming query. - * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. - * - * @param schema the user provided schema. - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - default ContinuousReadSupport createContinuousReadSupport( - StructType schema, - String checkpointLocation, - DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); - } - - /** - * Creates a {@link ContinuousReadSupport} instance to scan the data from this streaming data - * source, which is called by Spark at the beginning of each continuous streaming query. - * - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - ContinuousReadSupport createContinuousReadSupport( - String checkpointLocation, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java deleted file mode 100644 index 83df3be747085..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/DataSourceOptions.java +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import java.io.IOException; -import java.util.HashMap; -import java.util.Locale; -import java.util.Map; -import java.util.Optional; -import java.util.stream.Stream; - -import com.fasterxml.jackson.databind.ObjectMapper; - -import org.apache.spark.annotation.InterfaceStability; - -/** - * An immutable string-to-string map in which keys are case-insensitive. This is used to represent - * data source options. - * - * Each data source implementation can define its own options and teach its users how to set them. - * Spark doesn't have any restrictions about what options a data source should or should not have. - * Instead Spark defines some standard options that data sources can optionally adopt. It's possible - * that some options are very common and many data sources use them. However different data - * sources may define the common options(key and meaning) differently, which is quite confusing to - * end users. - * - * The standard options defined by Spark: - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - * - *
    Option keyOption value
    pathA path string of the data files/directories, like - * path1, /absolute/file2, path3/*. The path can - * either be relative or absolute, points to either file or directory, and can contain - * wildcards. This option is commonly used by file-based data sources.
    pathsA JSON array style paths string of the data files/directories, like - * ["path1", "/absolute/file2"]. The format of each path is same as the - * path option, plus it should follow JSON string literal format, e.g. quotes - * should be escaped, pa\"th means pa"th. - *
    tableA table name string representing the table name directly without any interpretation. - * For example, db.tbl means a table called db.tbl, not a table called tbl - * inside database db. `t*b.l` means a table called `t*b.l`, not t*b.l.
    databaseA database name string representing the database name directly without any - * interpretation, which is very similar to the table name option.
    - */ -@InterfaceStability.Evolving -public class DataSourceOptions { - private final Map keyLowerCasedMap; - - private String toLowerCase(String key) { - return key.toLowerCase(Locale.ROOT); - } - - public static DataSourceOptions empty() { - return new DataSourceOptions(new HashMap<>()); - } - - public DataSourceOptions(Map originalMap) { - keyLowerCasedMap = new HashMap<>(originalMap.size()); - for (Map.Entry entry : originalMap.entrySet()) { - keyLowerCasedMap.put(toLowerCase(entry.getKey()), entry.getValue()); - } - } - - public Map asMap() { - return new HashMap<>(keyLowerCasedMap); - } - - /** - * Returns the option value to which the specified key is mapped, case-insensitively. - */ - public Optional get(String key) { - return Optional.ofNullable(keyLowerCasedMap.get(toLowerCase(key))); - } - - /** - * Returns the boolean value to which the specified key is mapped, - * or defaultValue if there is no mapping for the key. The key match is case-insensitive - */ - public boolean getBoolean(String key, boolean defaultValue) { - String lcaseKey = toLowerCase(key); - return keyLowerCasedMap.containsKey(lcaseKey) ? - Boolean.parseBoolean(keyLowerCasedMap.get(lcaseKey)) : defaultValue; - } - - /** - * Returns the integer value to which the specified key is mapped, - * or defaultValue if there is no mapping for the key. The key match is case-insensitive - */ - public int getInt(String key, int defaultValue) { - String lcaseKey = toLowerCase(key); - return keyLowerCasedMap.containsKey(lcaseKey) ? - Integer.parseInt(keyLowerCasedMap.get(lcaseKey)) : defaultValue; - } - - /** - * Returns the long value to which the specified key is mapped, - * or defaultValue if there is no mapping for the key. The key match is case-insensitive - */ - public long getLong(String key, long defaultValue) { - String lcaseKey = toLowerCase(key); - return keyLowerCasedMap.containsKey(lcaseKey) ? - Long.parseLong(keyLowerCasedMap.get(lcaseKey)) : defaultValue; - } - - /** - * Returns the double value to which the specified key is mapped, - * or defaultValue if there is no mapping for the key. The key match is case-insensitive - */ - public double getDouble(String key, double defaultValue) { - String lcaseKey = toLowerCase(key); - return keyLowerCasedMap.containsKey(lcaseKey) ? - Double.parseDouble(keyLowerCasedMap.get(lcaseKey)) : defaultValue; - } - - /** - * The option key for singular path. - */ - public static final String PATH_KEY = "path"; - - /** - * The option key for multiple paths. - */ - public static final String PATHS_KEY = "paths"; - - /** - * The option key for table name. - */ - public static final String TABLE_KEY = "table"; - - /** - * The option key for database name. - */ - public static final String DATABASE_KEY = "database"; - - /** - * Returns all the paths specified by both the singular path option and the multiple - * paths option. - */ - public String[] paths() { - String[] singularPath = - get(PATH_KEY).map(s -> new String[]{s}).orElseGet(() -> new String[0]); - Optional pathsStr = get(PATHS_KEY); - if (pathsStr.isPresent()) { - ObjectMapper objectMapper = new ObjectMapper(); - try { - String[] paths = objectMapper.readValue(pathsStr.get(), String[].class); - return Stream.of(singularPath, paths).flatMap(Stream::of).toArray(String[]::new); - } catch (IOException e) { - return singularPath; - } - } else { - return singularPath; - } - } - - /** - * Returns the value of the table name option. - */ - public Optional tableName() { - return get(TABLE_KEY); - } - - /** - * Returns the value of the database name option. - */ - public Optional databaseName() { - return get(DATABASE_KEY); - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java deleted file mode 100644 index 61c08e7fa89df..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/MicroBatchReadSupportProvider.java +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils; -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data reading ability for micro-batch stream processing. - * - * This interface is used to create {@link MicroBatchReadSupport} instances when end users run - * {@code SparkSession.readStream.format(...).option(...).load()} with a micro-batch trigger. - */ -@InterfaceStability.Evolving -public interface MicroBatchReadSupportProvider extends DataSourceV2 { - - /** - * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data - * source with a user specified schema, which is called by Spark at the beginning of each - * micro-batch streaming query. - * - * By default this method throws {@link UnsupportedOperationException}, implementations should - * override this method to handle user specified schema. - * - * @param schema the user provided schema. - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - default MicroBatchReadSupport createMicroBatchReadSupport( - StructType schema, - String checkpointLocation, - DataSourceOptions options) { - return DataSourceV2Utils.failForUserSpecifiedSchema(this); - } - - /** - * Creates a {@link MicroBatchReadSupport} instance to scan the data from this streaming data - * source, which is called by Spark at the beginning of each micro-batch streaming query. - * - * @param checkpointLocation a path to Hadoop FS scratch space that can be used for failure - * recovery. Readers for the same logical source in the same query - * will be given the same checkpointLocation. - * @param options the options for the returned data source reader, which is an immutable - * case-insensitive string-to-string map. - */ - MicroBatchReadSupport createMicroBatchReadSupport( - String checkpointLocation, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java deleted file mode 100644 index f9ca85d8089b4..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/StreamingWriteSupportProvider.java +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSink; -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport; -import org.apache.spark.sql.streaming.OutputMode; -import org.apache.spark.sql.types.StructType; - -/** - * A mix-in interface for {@link DataSourceV2}. Data sources can implement this interface to - * provide data writing ability for structured streaming. - * - * This interface is used to create {@link StreamingWriteSupport} instances when end users run - * {@code Dataset.writeStream.format(...).option(...).start()}. - */ -@InterfaceStability.Evolving -public interface StreamingWriteSupportProvider extends DataSourceV2, BaseStreamingSink { - - /** - * Creates a {@link StreamingWriteSupport} instance to save the data to this data source, which is - * called by Spark at the beginning of each streaming query. - * - * @param queryId A unique string for the writing query. It's possible that there are many - * writing queries running at the same time, and the returned - * {@link StreamingWriteSupport} can use this id to distinguish itself from others. - * @param schema the schema of the data to be written. - * @param mode the output mode which determines what successive epoch output means to this - * sink, please refer to {@link OutputMode} for more details. - * @param options the options for the returned data source writer, which is an immutable - * case-insensitive string-to-string map. - */ - StreamingWriteSupport createStreamingWriteSupport( - String queryId, - StructType schema, - OutputMode mode, - DataSourceOptions options); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java deleted file mode 100644 index 452ee86675b42..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/BatchReadSupport.java +++ /dev/null @@ -1,51 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2.reader; - -import org.apache.spark.annotation.InterfaceStability; - -/** - * An interface that defines how to load the data from data source for batch processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.BatchReadSupportProvider}) at the start of a batch - * query, then call {@link #newScanConfigBuilder()} and create an instance of {@link ScanConfig}. - * The {@link ScanConfigBuilder} can apply operator pushdown and keep the pushdown result in - * {@link ScanConfig}. The {@link ScanConfig} will be used to create input partitions and reader - * factory to scan data from the data source with a Spark job. - */ -@InterfaceStability.Evolving -public interface BatchReadSupport extends ReadSupport { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, and keep these - * information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link BatchReadSupport} needs - * to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(); - - /** - * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}. - */ - PartitionReaderFactory createReaderFactory(ScanConfig config); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java deleted file mode 100644 index 7462ce2820585..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/ScanConfig.java +++ /dev/null @@ -1,45 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2.reader; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.types.StructType; - -/** - * An interface that carries query specific information for the data scanning job, like operator - * pushdown information and streaming query offsets. This is defined as an empty interface, and data - * sources should define their own {@link ScanConfig} classes. - * - * For APIs that take a {@link ScanConfig} as input, like - * {@link ReadSupport#planInputPartitions(ScanConfig)}, - * {@link BatchReadSupport#createReaderFactory(ScanConfig)} and - * {@link SupportsReportStatistics#estimateStatistics(ScanConfig)}, implementations mostly need to - * cast the input {@link ScanConfig} to the concrete {@link ScanConfig} class of the data source. - */ -@InterfaceStability.Evolving -public interface ScanConfig { - - /** - * Returns the actual schema of this data source reader, which may be different from the physical - * schema of the underlying storage, as column pruning or other optimizations may happen. - * - * If this method fails (by throwing an exception), the action will fail and no Spark job will be - * submitted. - */ - StructType readSchema(); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java deleted file mode 100644 index 9a3ad2eb8a801..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/ContinuousReadSupport.java +++ /dev/null @@ -1,77 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2.reader.streaming; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import org.apache.spark.sql.sources.v2.reader.InputPartition; -import org.apache.spark.sql.sources.v2.reader.ScanConfig; -import org.apache.spark.sql.sources.v2.reader.ScanConfigBuilder; - -/** - * An interface that defines how to load the data from data source for continuous streaming - * processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.ContinuousReadSupportProvider}) at the start of a - * streaming query, then call {@link #newScanConfigBuilder(Offset)} and create an instance of - * {@link ScanConfig} for the duration of the streaming query or until - * {@link #needsReconfiguration(ScanConfig)} is true. The {@link ScanConfig} will be used to create - * input partitions and reader factory to scan data with a Spark job for its duration. At the end - * {@link #stop()} will be called when the streaming execution is completed. Note that a single - * query may have multiple executions due to restart or failure recovery. - */ -@InterfaceStability.Evolving -public interface ContinuousReadSupport extends StreamingReadSupport, BaseStreamingSource { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, store streaming - * offsets, etc., and keep these information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link ContinuousReadSupport} - * needs to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(Offset start); - - /** - * Returns a factory, which produces one {@link ContinuousPartitionReader} for one - * {@link InputPartition}. - */ - ContinuousPartitionReaderFactory createContinuousReaderFactory(ScanConfig config); - - /** - * Merge partitioned offsets coming from {@link ContinuousPartitionReader} instances - * for each partition to a single global offset. - */ - Offset mergeOffsets(PartitionOffset[] offsets); - - /** - * The execution engine will call this method in every epoch to determine if new input - * partitions need to be generated, which may be required if for example the underlying - * source system has had partitions added or removed. - * - * If true, the query will be shut down and restarted with a new {@link ContinuousReadSupport} - * instance. - */ - default boolean needsReconfiguration(ScanConfig config) { - return false; - } -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java b/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java deleted file mode 100644 index edb0db11bff2c..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/sources/v2/reader/streaming/MicroBatchReadSupport.java +++ /dev/null @@ -1,60 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.sources.v2.reader.streaming; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.BaseStreamingSource; -import org.apache.spark.sql.sources.v2.reader.*; - -/** - * An interface that defines how to scan the data from data source for micro-batch streaming - * processing. - * - * The execution engine will get an instance of this interface from a data source provider - * (e.g. {@link org.apache.spark.sql.sources.v2.MicroBatchReadSupportProvider}) at the start of a - * streaming query, then call {@link #newScanConfigBuilder(Offset, Offset)} and create an instance - * of {@link ScanConfig} for each micro-batch. The {@link ScanConfig} will be used to create input - * partitions and reader factory to scan a micro-batch with a Spark job. At the end {@link #stop()} - * will be called when the streaming execution is completed. Note that a single query may have - * multiple executions due to restart or failure recovery. - */ -@InterfaceStability.Evolving -public interface MicroBatchReadSupport extends StreamingReadSupport, BaseStreamingSource { - - /** - * Returns a builder of {@link ScanConfig}. Spark will call this method and create a - * {@link ScanConfig} for each data scanning job. - * - * The builder can take some query specific information to do operators pushdown, store streaming - * offsets, etc., and keep these information in the created {@link ScanConfig}. - * - * This is the first step of the data scan. All other methods in {@link MicroBatchReadSupport} - * needs to take {@link ScanConfig} as an input. - */ - ScanConfigBuilder newScanConfigBuilder(Offset start, Offset end); - - /** - * Returns a factory, which produces one {@link PartitionReader} for one {@link InputPartition}. - */ - PartitionReaderFactory createReaderFactory(ScanConfig config); - - /** - * Returns the most recent offset available. - */ - Offset latestOffset(); -} diff --git a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java index 5371a23230c98..1bd7b825328db 100644 --- a/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java +++ b/sql/core/src/main/java/org/apache/spark/sql/streaming/Trigger.java @@ -19,10 +19,11 @@ import java.util.concurrent.TimeUnit; +import org.apache.spark.annotation.Evolving; +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger; import scala.concurrent.duration.Duration; -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger; +import org.apache.spark.sql.execution.streaming.ContinuousTrigger; import org.apache.spark.sql.execution.streaming.OneTimeTrigger$; /** @@ -30,7 +31,7 @@ * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving public class Trigger { /** @@ -40,7 +41,7 @@ public class Trigger { * @since 2.2.0 */ public static Trigger ProcessingTime(long intervalMs) { - return ProcessingTime.create(intervalMs, TimeUnit.MILLISECONDS); + return ProcessingTimeTrigger.create(intervalMs, TimeUnit.MILLISECONDS); } /** @@ -56,7 +57,7 @@ public static Trigger ProcessingTime(long intervalMs) { * @since 2.2.0 */ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { - return ProcessingTime.create(interval, timeUnit); + return ProcessingTimeTrigger.create(interval, timeUnit); } /** @@ -71,7 +72,7 @@ public static Trigger ProcessingTime(long interval, TimeUnit timeUnit) { * @since 2.2.0 */ public static Trigger ProcessingTime(Duration interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** @@ -84,7 +85,7 @@ public static Trigger ProcessingTime(Duration interval) { * @since 2.2.0 */ public static Trigger ProcessingTime(String interval) { - return ProcessingTime.apply(interval); + return ProcessingTimeTrigger.apply(interval); } /** diff --git a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java b/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java deleted file mode 100644 index d206c1df42abb..0000000000000 --- a/sql/core/src/main/java/org/apache/spark/sql/vectorized/ColumnarBatch.java +++ /dev/null @@ -1,113 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.vectorized; - -import java.util.*; - -import org.apache.spark.annotation.InterfaceStability; -import org.apache.spark.sql.catalyst.InternalRow; -import org.apache.spark.sql.execution.vectorized.MutableColumnarRow; - -/** - * This class wraps multiple ColumnVectors as a row-wise table. It provides a row view of this - * batch so that Spark can access the data row by row. Instance of it is meant to be reused during - * the entire data loading process. - */ -@InterfaceStability.Evolving -public final class ColumnarBatch { - private int numRows; - private final ColumnVector[] columns; - - // Staging row returned from `getRow`. - private final MutableColumnarRow row; - - /** - * Called to close all the columns in this batch. It is not valid to access the data after - * calling this. This must be called at the end to clean up memory allocations. - */ - public void close() { - for (ColumnVector c: columns) { - c.close(); - } - } - - /** - * Returns an iterator over the rows in this batch. - */ - public Iterator rowIterator() { - final int maxRows = numRows; - final MutableColumnarRow row = new MutableColumnarRow(columns); - return new Iterator() { - int rowId = 0; - - @Override - public boolean hasNext() { - return rowId < maxRows; - } - - @Override - public InternalRow next() { - if (rowId >= maxRows) { - throw new NoSuchElementException(); - } - row.rowId = rowId++; - return row; - } - - @Override - public void remove() { - throw new UnsupportedOperationException(); - } - }; - } - - /** - * Sets the number of rows in this batch. - */ - public void setNumRows(int numRows) { - this.numRows = numRows; - } - - /** - * Returns the number of columns that make up this batch. - */ - public int numCols() { return columns.length; } - - /** - * Returns the number of rows for read, including filtered rows. - */ - public int numRows() { return numRows; } - - /** - * Returns the column at `ordinal`. - */ - public ColumnVector column(int ordinal) { return columns[ordinal]; } - - /** - * Returns the row in this batch at `rowId`. Returned row is reused across calls. - */ - public InternalRow getRow(int rowId) { - assert(rowId >= 0 && rowId < numRows); - row.rowId = rowId; - return row; - } - - public ColumnarBatch(ColumnVector[] columns) { - this.columns = columns; - this.row = new MutableColumnarRow(columns); - } -} diff --git a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index 1b37905543b4e..c0b8b270bab1f 100644 --- a/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/main/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -1,9 +1,11 @@ -org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2 org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider -org.apache.spark.sql.execution.datasources.json.JsonFileFormat +org.apache.spark.sql.execution.datasources.v2.json.JsonDataSourceV2 +org.apache.spark.sql.execution.datasources.noop.NoopDataSource org.apache.spark.sql.execution.datasources.orc.OrcFileFormat -org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat -org.apache.spark.sql.execution.datasources.text.TextFileFormat +org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +org.apache.spark.sql.execution.datasources.v2.text.TextDataSourceV2 org.apache.spark.sql.execution.streaming.ConsoleSinkProvider org.apache.spark.sql.execution.streaming.sources.RateStreamProvider org.apache.spark.sql.execution.streaming.sources.TextSocketSourceProvider +org.apache.spark.sql.execution.datasources.binaryfile.BinaryFileFormat 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 5161fcde669e7..46d3fbc8c3cb4 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 @@ -42,7 +42,7 @@ function renderPlanViz() { setupTooltipForSparkPlanNode(i); } - resizeSvg(svg) + resizeSvg(svg); } /* -------------------- * 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 ae27690f2e5ba..5a408b29f9337 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ import scala.language.implicitConversions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} @@ -60,7 +60,7 @@ private[sql] object Column { * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable class TypedColumn[-T, U]( expr: Expression, private[sql] val encoder: ExpressionEncoder[U]) @@ -74,6 +74,9 @@ class TypedColumn[-T, U]( inputEncoder: ExpressionEncoder[_], inputAttributes: Seq[Attribute]): TypedColumn[T, U] = { val unresolvedDeserializer = UnresolvedDeserializer(inputEncoder.deserializer, inputAttributes) + + // This only inserts inputs into typed aggregate expressions. For untyped aggregate expressions, + // the resolving is handled in the analyzer directly. val newExpr = expr transform { case ta: TypedAggregateExpression if ta.inputDeserializer.isEmpty => ta.withInputInfo( @@ -127,7 +130,7 @@ class TypedColumn[-T, U]( * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class Column(val expr: Expression) extends Logging { def this(name: String) = this(name match { @@ -199,13 +202,13 @@ class Column(val expr: Expression) extends Logging { /** * Extracts a value or values from a complex type. * The following types of extraction are supported: - * - * - Given an Array, an integer ordinal can be used to retrieve a single value. - * - Given a Map, a key of the correct type can be used to retrieve an individual value. - * - Given a Struct, a string fieldName can be used to extract that field. - * - Given an Array of Structs, a string fieldName can be used to extract filed - * of every struct in that array, and return an Array of fields - * + *

      + *
    • Given an Array, an integer ordinal can be used to retrieve a single value.
    • + *
    • Given a Map, a key of the correct type can be used to retrieve an individual value.
    • + *
    • Given a Struct, a string fieldName can be used to extract that field.
    • + *
    • Given an Array of Structs, a string fieldName can be used to extract filed + * of every struct in that array, and return an Array of fields.
    • + *
    * @group expr_ops * @since 1.4.0 */ @@ -302,24 +305,6 @@ 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. * {{{ @@ -1242,7 +1227,7 @@ class Column(val expr: Expression) extends Logging { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class ColumnName(name: String) extends Column(name) { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala index 5288907b7d7ff..53e9f810d7c85 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameNaFunctions.scala @@ -22,18 +22,17 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.types._ - /** * Functionality for working with missing data in `DataFrame`s. * * @since 1.3.1 */ -@InterfaceStability.Stable +@Stable final class DataFrameNaFunctions private[sql](df: DataFrame) { /** 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 3af70b5153c83..e5cb7a51902ce 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 @@ -22,23 +22,27 @@ import java.util.{Locale, Properties} import scala.collection.JavaConverters._ import com.fasterxml.jackson.databind.ObjectMapper -import com.univocity.parsers.csv.CsvParser import org.apache.spark.Partition -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} +import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JSONOptions} +import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{DataSource, FailureSafeParser} +import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.csv._ import org.apache.spark.sql.execution.datasources.jdbc._ import org.apache.spark.sql.execution.datasources.json.TextInputJsonDataSource -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, DataSourceOptions, DataSourceV2} -import org.apache.spark.sql.types.{StringType, StructType} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils} +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.unsafe.types.UTF8String /** @@ -47,7 +51,7 @@ import org.apache.spark.unsafe.types.UTF8String * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** @@ -95,6 +99,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 1.4.0 @@ -132,6 +139,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 1.4.0 @@ -148,6 +158,9 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 1.4.0 @@ -175,7 +188,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { */ def load(path: String): DataFrame = { // force invocation of `load(...varargs...)` - option(DataSourceOptions.PATH_KEY, path).load(Seq.empty: _*) + option("path", path).load(Seq.empty: _*) } /** @@ -191,21 +204,37 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { "read files of Hive data source directly.") } + val useV1Sources = + sparkSession.sessionState.conf.useV1SourceReaderList.toLowerCase(Locale.ROOT).split(",") val cls = DataSource.lookupDataSource(source, sparkSession.sessionState.conf) - if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val ds = cls.newInstance().asInstanceOf[DataSourceV2] - if (ds.isInstanceOf[BatchReadSupportProvider]) { - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - ds = ds, conf = sparkSession.sessionState.conf) - val pathsOption = { - val objectMapper = new ObjectMapper() - DataSourceOptions.PATHS_KEY -> objectMapper.writeValueAsString(paths.toArray) - } - Dataset.ofRows(sparkSession, DataSourceV2Relation.create( - ds, sessionOptions ++ extraOptions.toMap + pathsOption, - userSpecifiedSchema = userSpecifiedSchema)) + val shouldUseV1Source = cls.newInstance() match { + case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true + case _ => useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT)) + } + + if (!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(cls)) { + val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + source = provider, conf = sparkSession.sessionState.conf) + val pathsOption = if (paths.isEmpty) { + None } else { - loadV1Source(paths: _*) + val objectMapper = new ObjectMapper() + Some("paths" -> objectMapper.writeValueAsString(paths.toArray)) + } + + val finalOptions = sessionOptions ++ extraOptions.toMap ++ pathsOption + val dsOptions = new CaseInsensitiveStringMap(finalOptions.asJava) + val table = userSpecifiedSchema match { + case Some(schema) => provider.getTable(dsOptions, schema) + case _ => provider.getTable(dsOptions) + } + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + table match { + case _: SupportsRead if table.supports(BATCH_READ) => + Dataset.ofRows(sparkSession, DataSourceV2Relation.create(table, dsOptions)) + + case _ => loadV1Source(paths: _*) } } else { loadV1Source(paths: _*) @@ -354,7 +383,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * during parsing. *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. To * keep corrupt records, an user can set a string type field named * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have the * field, it drops corrupt records during parsing. When inferring a schema, it implicitly @@ -367,11 +396,11 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
    • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + * `java.time.format.DateTimeFormatter`. 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 @@ -383,6 +412,8 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * for schema inferring.
    • *
    • `dropFieldIfAllNull` (default `false`): whether to ignore column of all null values or * empty array/struct during schema inference.
    • + *
    • `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. + * For instance, this is used while parsing dates and timestamps.
    • *
    * * @since 2.0.0 @@ -439,19 +470,18 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { TextInputJsonDataSource.inferFromDataset(jsonDataset, parsedOptions) } - verifyColumnNameOfCorruptRecord(schema, parsedOptions.columnNameOfCorruptRecord) + ExprUtils.verifyColumnNameOfCorruptRecord(schema, parsedOptions.columnNameOfCorruptRecord) val actualSchema = StructType(schema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) val createParser = CreateJacksonParser.string _ val parsed = jsonDataset.rdd.mapPartitions { iter => - val rawParser = new JacksonParser(actualSchema, parsedOptions) + val rawParser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) val parser = new FailureSafeParser[String]( input => rawParser.parse(input, createParser, UTF8String.fromString), parsedOptions.parseMode, schema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) + parsedOptions.columnNameOfCorruptRecord) iter.flatMap(parser.parse) } sparkSession.internalCreateDataFrame(parsed, schema, isStreaming = jsonDataset.isStreaming) @@ -491,7 +521,19 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { sparkSession.sessionState.conf.sessionLocalTimeZone) val filteredLines: Dataset[String] = CSVUtils.filterCommentAndEmpty(csvDataset, parsedOptions) - val maybeFirstLine: Option[String] = filteredLines.take(1).headOption + + // For performance, short-circuit the collection of the first line when it won't be used: + // - TextInputCSVDataSource - Only uses firstLine to infer an unspecified schema + // - CSVHeaderChecker - Only uses firstLine to check header, when headerFlag is true + // - CSVUtils - Only uses firstLine to filter headers, when headerFlag is true + // (If the downstream logic grows more complicated, consider refactoring to an approach that + // delegates this decision to the constituent consumers themselves.) + val maybeFirstLine: Option[String] = + if (userSpecifiedSchema.isEmpty || parsedOptions.headerFlag) { + filteredLines.take(1).headOption + } else { + None + } val schema = userSpecifiedSchema.getOrElse { TextInputCSVDataSource.inferFromDataset( @@ -501,7 +543,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { parsedOptions) } - verifyColumnNameOfCorruptRecord(schema, parsedOptions.columnNameOfCorruptRecord) + ExprUtils.verifyColumnNameOfCorruptRecord(schema, parsedOptions.columnNameOfCorruptRecord) val actualSchema = StructType(schema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) @@ -520,8 +562,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { input => Seq(rawParser.parse(input)), parsedOptions.parseMode, schema, - parsedOptions.columnNameOfCorruptRecord, - parsedOptions.multiLine) + parsedOptions.columnNameOfCorruptRecord) iter.flatMap(parser.parse) } sparkSession.internalCreateDataFrame(parsed, schema, isStreaming = csvDataset.isStreaming) @@ -575,26 +616,29 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. 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 * for any given value being read. By default, it is -1 meaning unlimited length
  • *
  • `mode` (default `PERMISSIVE`): allows a mode for dealing with corrupt records - * during parsing. It supports the following case-insensitive modes. + * during parsing. It supports the following case-insensitive modes. Note that Spark tries + * to parse only required columns in CSV under column pruning. Therefore, corrupt records + * can be different based on required set of fields. This behavior can be controlled by + * `spark.sql.csv.parser.columnPruning.enabled` (enabled by default). *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To keep - * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` - * in an user-defined schema. If a schema does not have the field, it drops corrupt records - * during parsing. A record with less/more tokens than schema is not a corrupted record to - * CSV. When it meets a record having fewer tokens than the length of the schema, sets - * `null` to extra fields. When the record has more tokens than the length of the schema, - * it drops extra tokens.
    • + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. + * To keep corrupt records, an user can set a string type field named + * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have + * the field, it drops corrupt records during parsing. A record with less/more tokens + * than schema is not a corrupted record to CSV. When it meets a record having fewer + * tokens than the length of the schema, sets `null` to extra fields. When the record + * has more tokens than the length of the schema, it drops extra tokens. *
    • `DROPMALFORMED` : ignores the whole corrupted records.
    • *
    • `FAILFAST` : throws an exception when it meets corrupted records.
    • *
    @@ -603,6 +647,10 @@ 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`.
  • *
  • `multiLine` (default `false`): parse one record, which may span multiple lines.
  • + *
  • `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. + * For instance, this is used while parsing dates and timestamps.
  • + *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator + * that should be used for parsing. Maximum length is 1 character.
  • * * * @since 2.0.0 @@ -642,7 +690,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * @param path input path * @since 1.5.0 - * @note Currently, this method can only be used after enabling Hive support. */ def orc(path: String): DataFrame = { // This method ensures that calls that explicit need single argument works, see SPARK-16009 @@ -654,7 +701,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { * * @param paths input paths * @since 2.0.0 - * @note Currently, this method can only be used after enabling Hive support. */ @scala.annotation.varargs def orc(paths: String*): DataFrame = format("orc").load(paths: _*) @@ -684,6 +730,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Loads text files and returns a `DataFrame` whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. + * The text files must be encoded as UTF-8. * * By default, each line in the text files is a new row in the resulting DataFrame. For example: * {{{ @@ -721,6 +768,7 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { /** * Loads text files and returns a [[Dataset]] of String. The underlying schema of the Dataset * contains a single string column named "value". + * The text files must be encoded as UTF-8. * * If the directory structure of the text files contains partitioning information, those are * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. @@ -760,22 +808,6 @@ class DataFrameReader private[sql](sparkSession: SparkSession) extends Logging { } } - /** - * A convenient function for schema validation in datasources supporting - * `columnNameOfCorruptRecord` as an option. - */ - private def verifyColumnNameOfCorruptRecord( - schema: StructType, - columnNameOfCorruptRecord: String): Unit = { - schema.getFieldIndex(columnNameOfCorruptRecord).foreach { corruptFieldIndex => - val f = schema(corruptFieldIndex) - if (f.dataType != StringType || !f.nullable) { - throw new AnalysisException( - "The field for corrupt records must be string type and nullable") - } - } - } - /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options /////////////////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala index 7c12432d33c33..0b22b898557f7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrameStatFunctions.scala @@ -21,7 +21,7 @@ import java.{lang => jl, util => ju} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.stat._ import org.apache.spark.sql.functions.col @@ -33,7 +33,7 @@ import org.apache.spark.util.sketch.{BloomFilter, CountMinSketch} * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable final class DataFrameStatFunctions private[sql](df: DataFrame) { /** @@ -51,7 +51,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { * * This method implements a variation of the Greenwald-Khanna algorithm (with some speed * optimizations). - * The algorithm was first present in + * The algorithm was first present in * Space-efficient Online Computation of Quantile Summaries by Greenwald and Khanna. * * @param col the name of the numerical column @@ -218,7 +218,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { /** * Finding frequent items for columns, possibly with false positives. Using the * frequent element count algorithm described in - * here, proposed by Karp, + * here, proposed by Karp, * Schenker, and Papadimitriou. * The `support` should be greater than 1e-4. * @@ -265,7 +265,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { /** * Finding frequent items for columns, possibly with false positives. Using the * frequent element count algorithm described in - * here, proposed by Karp, + * here, proposed by Karp, * Schenker, and Papadimitriou. * Uses a `default` support of 1%. * @@ -284,7 +284,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { /** * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the * frequent element count algorithm described in - * here, proposed by Karp, Schenker, + * here, proposed by Karp, Schenker, * and Papadimitriou. * * This function is meant for exploratory data analysis, as we make no guarantee about the @@ -328,7 +328,7 @@ final class DataFrameStatFunctions private[sql](df: DataFrame) { /** * (Scala-specific) Finding frequent items for columns, possibly with false positives. Using the * frequent element count algorithm described in - * here, proposed by Karp, Schenker, + * here, proposed by Karp, Schenker, * and Papadimitriou. * Uses a `default` support of 1%. * 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 55e538f49feda..d1d0d83bacb4c 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 @@ -21,18 +21,21 @@ import java.util.{Locale, Properties, UUID} import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, InsertIntoTable, LogicalPlan, OverwriteByExpression} import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command.DDLUtils -import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, LogicalRelation} -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2Utils, WriteToDataSourceV2} -import org.apache.spark.sql.sources.BaseRelation +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSource, DataSourceUtils, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.{BaseRelation, DataSourceRegister} import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * Interface used to write a [[Dataset]] to external storage systems (e.g. file systems, @@ -40,44 +43,51 @@ import org.apache.spark.sql.types.StructType * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() /** * Specifies the behavior when data or table already exists. Options include: - * - `SaveMode.Overwrite`: overwrite the existing data. - * - `SaveMode.Append`: append the data. - * - `SaveMode.Ignore`: ignore the operation (i.e. no-op). - * - `SaveMode.ErrorIfExists`: default option, throw an exception at runtime. + *
      + *
    • `SaveMode.Overwrite`: overwrite the existing data.
    • + *
    • `SaveMode.Append`: append the data.
    • + *
    • `SaveMode.Ignore`: ignore the operation (i.e. no-op).
    • + *
    • `SaveMode.ErrorIfExists`: throw an exception at runtime.
    • + *
    + *

    + * When writing to data source v1, the default option is `ErrorIfExists`. When writing to data + * source v2, the default option is `Append`. * * @since 1.4.0 */ def mode(saveMode: SaveMode): DataFrameWriter[T] = { - this.mode = saveMode + this.mode = Some(saveMode) this } /** * Specifies the behavior when data or table already exists. Options include: - * - `overwrite`: overwrite the existing data. - * - `append`: append the data. - * - `ignore`: ignore the operation (i.e. no-op). - * - `error` or `errorifexists`: default option, throw an exception at runtime. + *

      + *
    • `overwrite`: overwrite the existing data.
    • + *
    • `append`: append the data.
    • + *
    • `ignore`: ignore the operation (i.e. no-op).
    • + *
    • `error` or `errorifexists`: default option, throw an exception at runtime.
    • + *
    * * @since 1.4.0 */ def mode(saveMode: String): DataFrameWriter[T] = { - this.mode = saveMode.toLowerCase(Locale.ROOT) match { - case "overwrite" => SaveMode.Overwrite - case "append" => SaveMode.Append - case "ignore" => SaveMode.Ignore - case "error" | "errorifexists" | "default" => SaveMode.ErrorIfExists + saveMode.toLowerCase(Locale.ROOT) match { + case "overwrite" => mode(SaveMode.Overwrite) + case "append" => mode(SaveMode.Append) + case "ignore" => mode(SaveMode.Ignore) + case "error" | "errorifexists" => mode(SaveMode.ErrorIfExists) + case "default" => this case _ => throw new IllegalArgumentException(s"Unknown save mode: $saveMode. " + "Accepted save modes are 'overwrite', 'append', 'ignore', 'error', 'errorifexists'.") } - this } /** @@ -163,9 +173,10 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * Partitions the output by the given columns on the file system. If specified, the output is * laid out on the file system similar to Hive's partitioning scheme. As an example, when we * partition a dataset by year and then month, the directory layout would look like: - * - * - year=2016/month=01/ - * - year=2016/month=02/ + *
      + *
    • year=2016/month=01/
    • + *
    • year=2016/month=02/
    • + *
    * * Partitioning is one of the most widely used techniques to optimize physical data layout. * It provides a coarse-grained index for skipping unnecessary data reads when queries have @@ -236,34 +247,61 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { assertNotBucketed("save") - val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) - if (classOf[DataSourceV2].isAssignableFrom(cls)) { - val source = cls.newInstance().asInstanceOf[DataSourceV2] - source match { - case provider: BatchWriteSupportProvider => - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - source, - df.sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions - - if (mode == SaveMode.Append) { - val relation = DataSourceV2Relation.create(source, options) + val session = df.sparkSession + val useV1Sources = + session.sessionState.conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",") + val cls = DataSource.lookupDataSource(source, session.sessionState.conf) + val shouldUseV1Source = cls.newInstance() match { + case d: DataSourceRegister if useV1Sources.contains(d.shortName()) => true + case _ => useV1Sources.contains(cls.getCanonicalName.toLowerCase(Locale.ROOT)) + } + + // In Data Source V2 project, partitioning is still under development. + // Here we fallback to V1 if partitioning columns are specified. + // TODO(SPARK-26778): use V2 implementations when partitioning feature is supported. + if (!shouldUseV1Source && classOf[TableProvider].isAssignableFrom(cls) && + partitioningColumns.isEmpty) { + val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + provider, session.sessionState.conf) + val options = sessionOptions ++ extraOptions + val dsOptions = new CaseInsensitiveStringMap(options.asJava) + + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + provider.getTable(dsOptions) match { + // TODO (SPARK-27815): To not break existing tests, here we treat file source as a special + // case, and pass the save mode to file source directly. This hack should be removed. + case table: FileTable => + val write = table.newWriteBuilder(dsOptions).asInstanceOf[FileWriteBuilder] + .mode(modeForDSV1) // should not change default mode for file source. + .withQueryId(UUID.randomUUID().toString) + .withInputDataSchema(df.logicalPlan.schema) + .buildForBatch() + // The returned `Write` can be null, which indicates that we can skip writing. + if (write != null) { runCommand(df.sparkSession, "save") { - AppendData.byName(relation, df.logicalPlan) + WriteToDataSourceV2(write, df.logicalPlan) } + } - } else { - val writer = provider.createBatchWriteSupport( - UUID.randomUUID().toString, - df.logicalPlan.output.toStructType, - mode, - new DataSourceOptions(options.asJava)) + case table: SupportsWrite if table.supports(BATCH_WRITE) => + lazy val relation = DataSourceV2Relation.create(table, dsOptions) + modeForDSV2 match { + case SaveMode.Append => + runCommand(df.sparkSession, "save") { + AppendData.byName(relation, df.logicalPlan) + } - if (writer.isPresent) { + case SaveMode.Overwrite if table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER) => + // truncate the table runCommand(df.sparkSession, "save") { - WriteToDataSourceV2(writer.get, df.logicalPlan) + OverwriteByExpression.byName(relation, df.logicalPlan, Literal(true)) } - } + + case other => + throw new AnalysisException(s"TableProvider implementation $source cannot be " + + s"written with $other mode, please use Append or Overwrite " + + "modes instead.") } // Streaming also uses the data source V2 API. So it may be that the data source implements @@ -277,13 +315,18 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { } private def saveToV1Source(): Unit = { + partitioningColumns.foreach { columns => + extraOptions += (DataSourceUtils.PARTITIONING_COLUMNS_KEY -> + DataSourceUtils.encodePartitioningColumns(columns)) + } + // Code path for data source v1. runCommand(df.sparkSession, "save") { DataSource( sparkSession = df.sparkSession, className = source, partitionColumns = partitioningColumns.getOrElse(Nil), - options = extraOptions.toMap).planForWriting(mode, df.logicalPlan) + options = extraOptions.toMap).planForWriting(modeForDSV1, df.logicalPlan) } } @@ -332,7 +375,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { table = UnresolvedRelation(tableIdent), partition = Map.empty[String, Option[String]], query = df.logicalPlan, - overwrite = mode == SaveMode.Overwrite, + overwrite = modeForDSV1 == SaveMode.Overwrite, ifPartitionNotExists = false) } } @@ -412,7 +455,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { val tableIdentWithDB = tableIdent.copy(database = Some(db)) val tableName = tableIdentWithDB.unquotedString - (tableExists, mode) match { + (tableExists, modeForDSV1) match { case (true, SaveMode.Ignore) => // Do nothing @@ -467,7 +510,8 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { partitionColumnNames = partitioningColumns.getOrElse(Nil), bucketSpec = getBucketSpec) - runCommand(df.sparkSession, "saveAsTable")(CreateTable(tableDesc, mode, Some(df.logicalPlan))) + runCommand(df.sparkSession, "saveAsTable")( + CreateTable(tableDesc, modeForDSV1, Some(df.logicalPlan))) } /** @@ -525,11 +569,11 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. 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.
  • @@ -579,7 +623,6 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * * * @since 1.5.0 - * @note Currently, this method can only be used after enabling Hive support */ def orc(path: String): Unit = { format("orc").save(path) @@ -596,6 +639,7 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { * // Java: * df.write().text("/path/to/output") * }}} + * The text files will be encoded as UTF-8. * * You can set the following option(s) for writing text files: *
      @@ -644,15 +688,17 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
    • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + * `java.time.format.DateTimeFormatter`. 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 * trailing whitespaces from values being written should be skipped.
    • + *
    • `lineSep` (default `\n`): defines the line separator that should be used for writing. + * Maximum length is 1 character.
    • *
    * * @since 2.0.0 @@ -667,26 +713,21 @@ final class DataFrameWriter[T] private[sql](ds: Dataset[T]) { */ private def runCommand(session: SparkSession, name: String)(command: LogicalPlan): Unit = { val qe = session.sessionState.executePlan(command) - try { - val start = System.nanoTime() - // call `QueryExecution.toRDD` to trigger the execution of commands. - SQLExecution.withNewExecutionId(session, qe)(qe.toRdd) - val end = System.nanoTime() - session.listenerManager.onSuccess(name, qe, end - start) - } catch { - case e: Exception => - session.listenerManager.onFailure(name, qe, e) - throw e - } + // call `QueryExecution.toRDD` to trigger the execution of commands. + SQLExecution.withNewExecutionId(session, qe, Some(name))(qe.toRdd) } + private def modeForDSV1 = mode.getOrElse(SaveMode.ErrorIfExists) + + private def modeForDSV2 = mode.getOrElse(SaveMode.Append) + /////////////////////////////////////////////////////////////////////////////////////// // Builder pattern config options /////////////////////////////////////////////////////////////////////////////////////// private var source: String = df.sparkSession.sessionState.conf.defaultDataSourceName - private var mode: SaveMode = SaveMode.ErrorIfExists + private var mode: Option[SaveMode] = None private val extraOptions = new scala.collection.mutable.HashMap[String, String] 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 fa14aa14ee968..ef03a09bba0b9 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 @@ -17,46 +17,46 @@ package org.apache.spark.sql -import java.io.CharArrayWriter -import java.sql.{Date, Timestamp} +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 import org.apache.spark.TaskContext -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.api.java.function._ -import org.apache.spark.api.python.{PythonRDD, SerDeUtil} +import org.apache.spark.api.python.{PythonEvalType, 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._ +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog.HiveTableRelation import org.apache.spark.sql.catalyst.encoders._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.GenerateSafeProjection import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions} import org.apache.spark.sql.catalyst.optimizer.CombineUnions import org.apache.spark.sql.catalyst.parser.{ParseException, ParserUtils} import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, PartitioningCollection} -import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.arrow.{ArrowBatchStreamWriter, ArrowConverters} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} import org.apache.spark.sql.execution.python.EvaluatePython import org.apache.spark.sql.execution.stat.StatFunctions import org.apache.spark.sql.streaming.DataStreamWriter import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils import org.apache.spark.storage.StorageLevel +import org.apache.spark.unsafe.array.ByteArrayMethods import org.apache.spark.unsafe.types.CalendarInterval import org.apache.spark.util.Utils @@ -69,7 +69,7 @@ private[sql] object Dataset { // do not do this check in that case. this check can be expensive since it requires running // the whole [[Analyzer]] to resolve the deserializer if (dataset.exprEnc.clsTag.runtimeClass != classOf[Row]) { - dataset.deserializer + dataset.resolvedEnc } dataset } @@ -79,6 +79,14 @@ private[sql] object Dataset { qe.assertAnalyzed() new Dataset[Row](sparkSession, qe, RowEncoder(qe.analyzed.schema)) } + + /** A variant of ofRows that allows passing in a tracker so we can track query parsing time. */ + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan, tracker: QueryPlanningTracker) + : DataFrame = { + val qe = new QueryExecution(sparkSession, logicalPlan, tracker) + qe.assertAnalyzed() + new Dataset[Row](sparkSession, qe, RowEncoder(qe.analyzed.schema)) + } } /** @@ -167,11 +175,11 @@ private[sql] object Dataset { * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable class Dataset[T] private[sql]( @transient val sparkSession: SparkSession, - @DeveloperApi @InterfaceStability.Unstable @transient val queryExecution: QueryExecution, - encoder: Encoder[T]) + @DeveloperApi @Unstable @transient val queryExecution: QueryExecution, + @DeveloperApi @Unstable @transient val encoder: Encoder[T]) extends Serializable { queryExecution.assertAnalyzed() @@ -208,10 +216,11 @@ class Dataset[T] private[sql]( */ private[sql] implicit val exprEnc: ExpressionEncoder[T] = encoderFor(encoder) - // The deserializer expression which can be used to build a projection and turn rows to objects - // of type T, after collecting rows to the driver side. - private lazy val deserializer = - exprEnc.resolveAndBind(logicalPlan.output, sparkSession.sessionState.analyzer).deserializer + // The resolved `ExpressionEncoder` which can be used to turn rows to objects of type T, after + // collecting rows to the driver side. + private lazy val resolvedEnc = { + exprEnc.resolveAndBind(logicalPlan.output, sparkSession.sessionState.analyzer) + } private implicit def classTag = exprEnc.clsTag @@ -287,7 +296,7 @@ class Dataset[T] private[sql]( _numRows: Int, truncate: Int = 20, vertical: Boolean = false): String = { - val numRows = _numRows.max(0).min(Int.MaxValue - 1) + val numRows = _numRows.max(0).min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH - 1) // Get rows represented by Seq[Seq[String]], we may get one more line if it has more data. val tmpRows = getRows(numRows, truncate) @@ -427,7 +436,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def as[U : Encoder]: Dataset[U] = Dataset[U](sparkSession, logicalPlan) /** @@ -470,8 +479,16 @@ class Dataset[T] private[sql]( * @group basic * @since 1.6.0 */ + def printSchema(): Unit = printSchema(Int.MaxValue) + // scalastyle:off println - def printSchema(): Unit = println(schema.treeString) + /** + * Prints the schema up to the given level to the console in a nice tree format. + * + * @group basic + * @since 3.0.0 + */ + def printSchema(level: Int): Unit = println(schema.treeString(level)) // scalastyle:on println /** @@ -481,12 +498,21 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def explain(extended: Boolean): Unit = { - val explain = ExplainCommand(queryExecution.logical, extended = extended) - sparkSession.sessionState.executePlan(explain).executedPlan.executeCollect().foreach { - // scalastyle:off println - r => println(r.getString(0)) - // scalastyle:on println - } + // Because temporary views are resolved during analysis when we create a Dataset, and + // `ExplainCommand` analyzes input query plan and resolves temporary views again. Using + // `ExplainCommand` here will probably output different query plans, compared to the results + // of evaluation of the Dataset. So just output QueryExecution's query plans here. + val qe = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, queryExecution) + + val outputString = + if (extended) { + qe.toString + } else { + qe.simpleString + } + // scalastyle:off println + println(outputString) + // scalastyle:on println } /** @@ -545,7 +571,7 @@ class Dataset[T] private[sql]( * @group streaming * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def isStreaming: Boolean = logicalPlan.isStreaming /** @@ -558,7 +584,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def checkpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = true) /** @@ -571,7 +597,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def checkpoint(eager: Boolean): Dataset[T] = checkpoint(eager = eager, reliableCheckpoint = true) /** @@ -584,7 +610,7 @@ class Dataset[T] private[sql]( * @since 2.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def localCheckpoint(): Dataset[T] = checkpoint(eager = true, reliableCheckpoint = false) /** @@ -597,7 +623,7 @@ class Dataset[T] private[sql]( * @since 2.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def localCheckpoint(eager: Boolean): Dataset[T] = checkpoint( eager = eager, reliableCheckpoint = false @@ -612,40 +638,41 @@ class Dataset[T] private[sql]( * the caching subsystem */ private def checkpoint(eager: Boolean, reliableCheckpoint: Boolean): Dataset[T] = { - val internalRdd = queryExecution.toRdd.map(_.copy()) - if (reliableCheckpoint) { - internalRdd.checkpoint() - } else { - internalRdd.localCheckpoint() - } - - if (eager) { - internalRdd.count() - } - - val physicalPlan = queryExecution.executedPlan + val actionName = if (reliableCheckpoint) "checkpoint" else "localCheckpoint" + withAction(actionName, queryExecution) { physicalPlan => + val internalRdd = physicalPlan.execute().map(_.copy()) + if (reliableCheckpoint) { + internalRdd.checkpoint() + } else { + internalRdd.localCheckpoint() + } - // Takes the first leaf partitioning whenever we see a `PartitioningCollection`. Otherwise the - // size of `PartitioningCollection` may grow exponentially for queries involving deep inner - // joins. - def firstLeafPartitioning(partitioning: Partitioning): Partitioning = { - partitioning match { - case p: PartitioningCollection => firstLeafPartitioning(p.partitionings.head) - case p => p + if (eager) { + internalRdd.count() } - } - val outputPartitioning = firstLeafPartitioning(physicalPlan.outputPartitioning) + // Takes the first leaf partitioning whenever we see a `PartitioningCollection`. Otherwise the + // size of `PartitioningCollection` may grow exponentially for queries involving deep inner + // joins. + def firstLeafPartitioning(partitioning: Partitioning): Partitioning = { + partitioning match { + case p: PartitioningCollection => firstLeafPartitioning(p.partitionings.head) + case p => p + } + } - Dataset.ofRows( - sparkSession, - LogicalRDD( - logicalPlan.output, - internalRdd, - outputPartitioning, - physicalPlan.outputOrdering, - isStreaming - )(sparkSession)).as[T] + val outputPartitioning = firstLeafPartitioning(physicalPlan.outputPartitioning) + + Dataset.ofRows( + sparkSession, + LogicalRDD( + logicalPlan.output, + internalRdd, + outputPartitioning, + physicalPlan.outputOrdering, + isStreaming + )(sparkSession)).as[T] + } } /** @@ -672,13 +699,19 @@ class Dataset[T] private[sql]( * @group streaming * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving // We only accept an existing column name, not a derived column here as a watermark that is // defined on a derived column cannot referenced elsewhere in the plan. def withWatermark(eventTime: String, delayThreshold: String): Dataset[T] = withTypedPlan { val parsedDelay = - Option(CalendarInterval.fromString("interval " + delayThreshold)) - .getOrElse(throw new AnalysisException(s"Unable to parse time delay '$delayThreshold'")) + try { + CalendarInterval.fromCaseInsensitiveString(delayThreshold) + } catch { + case e: IllegalArgumentException => + throw new AnalysisException( + s"Unable to parse time delay '$delayThreshold'", + cause = Some(e)) + } require(parsedDelay.milliseconds >= 0 && parsedDelay.months >= 0, s"delay threshold ($delayThreshold) should not be negative.") EliminateEventTimeWatermark( @@ -855,7 +888,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ def join(right: Dataset[_]): DataFrame = withPlan { - Join(logicalPlan, right.logicalPlan, joinType = Inner, None) + Join(logicalPlan, right.logicalPlan, joinType = Inner, None, JoinHint.NONE) } /** @@ -919,8 +952,9 @@ class Dataset[T] private[sql]( * @param right Right side of the join operation. * @param usingColumns Names of the columns to join on. This columns must exist on both sides. * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `full_outer`, `left`, `left_outer`, - * `right`, `right_outer`, `left_semi`, `left_anti`. + * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, + * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, + * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, left_anti`. * * @note If you perform a self-join using this function without aliasing the input * `DataFrame`s, you will NOT be able to reference any columns after the join, since @@ -933,7 +967,7 @@ class Dataset[T] private[sql]( // Analyze the self join. The assumption is that the analyzer will disambiguate left vs right // by creating a new instance for one of the branch. val joined = sparkSession.sessionState.executePlan( - Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None)) + Join(logicalPlan, right.logicalPlan, joinType = JoinType(joinType), None, JoinHint.NONE)) .analyzed.asInstanceOf[Join] withPlan { @@ -941,7 +975,8 @@ class Dataset[T] private[sql]( joined.left, joined.right, UsingJoin(JoinType(joinType), usingColumns), - None) + None, + JoinHint.NONE) } } @@ -976,8 +1011,9 @@ class Dataset[T] private[sql]( * @param right Right side of the join. * @param joinExprs Join expression. * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `full_outer`, `left`, `left_outer`, - * `right`, `right_outer`, `left_semi`, `left_anti`. + * `inner`, `cross`, `outer`, `full`, `fullouter`, `full_outer`, `left`, + * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`, + * `semi`, `leftsemi`, `left_semi`, `anti`, `leftanti`, left_anti`. * * @group untypedrel * @since 2.0.0 @@ -994,7 +1030,7 @@ class Dataset[T] private[sql]( // Trigger analysis so in the case of self-join, the analyzer will clone the plan. // After the cloning, left and right side will have distinct expression ids. val plan = withPlan( - Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr))) + Join(logicalPlan, right.logicalPlan, JoinType(joinType), Some(joinExprs.expr), JoinHint.NONE)) .queryExecution.analyzed.asInstanceOf[Join] // If auto self join alias is disabled, return the plan. @@ -1041,7 +1077,7 @@ class Dataset[T] private[sql]( * @since 2.1.0 */ def crossJoin(right: Dataset[_]): DataFrame = withPlan { - Join(logicalPlan, right.logicalPlan, joinType = Cross, None) + Join(logicalPlan, right.logicalPlan, joinType = Cross, None, JoinHint.NONE) } /** @@ -1060,14 +1096,14 @@ class Dataset[T] private[sql]( * @param other Right side of the join. * @param condition Join expression. * @param joinType Type of join to perform. Default `inner`. Must be one of: - * `inner`, `cross`, `outer`, `full`, `full_outer`, `left`, `left_outer`, - * `right`, `right_outer`. + * `inner`, `cross`, `outer`, `full`, `fullouter`,`full_outer`, `left`, + * `leftouter`, `left_outer`, `right`, `rightouter`, `right_outer`. * * @group typedrel * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def joinWith[U](other: Dataset[U], condition: Column, joinType: String): Dataset[(T, U)] = { // Creates a Join node and resolve it first, to get join condition resolved, self-join resolved, // etc. @@ -1076,59 +1112,68 @@ class Dataset[T] private[sql]( this.logicalPlan, other.logicalPlan, JoinType(joinType), - Some(condition.expr))).analyzed.asInstanceOf[Join] + Some(condition.expr), + JoinHint.NONE)).analyzed.asInstanceOf[Join] if (joined.joinType == LeftSemi || joined.joinType == LeftAnti) { throw new AnalysisException("Invalid join type in joinWith: " + joined.joinType.sql) } - // For both join side, combine all outputs into a single column and alias it with "_1" or "_2", - // to match the schema for the encoder of the join result. - // Note that we do this before joining them, to enable the join operator to return null for one - // side, in cases like outer-join. - val left = { - val combined = if (this.exprEnc.flat) { + implicit val tuple2Encoder: Encoder[(T, U)] = + ExpressionEncoder.tuple(this.exprEnc, other.exprEnc) + + val leftResultExpr = { + if (!this.exprEnc.isSerializedAsStructForTopLevel) { assert(joined.left.output.length == 1) Alias(joined.left.output.head, "_1")() } else { Alias(CreateStruct(joined.left.output), "_1")() } - Project(combined :: Nil, joined.left) } - val right = { - val combined = if (other.exprEnc.flat) { + val rightResultExpr = { + if (!other.exprEnc.isSerializedAsStructForTopLevel) { assert(joined.right.output.length == 1) Alias(joined.right.output.head, "_2")() } else { Alias(CreateStruct(joined.right.output), "_2")() } - Project(combined :: Nil, joined.right) } - // Rewrites the join condition to make the attribute point to correct column/field, after we - // combine the outputs of each join side. - val conditionExpr = joined.condition.get transformUp { - case a: Attribute if joined.left.outputSet.contains(a) => - if (this.exprEnc.flat) { - left.output.head - } else { - val index = joined.left.output.indexWhere(_.exprId == a.exprId) - GetStructField(left.output.head, index) - } - case a: Attribute if joined.right.outputSet.contains(a) => - if (other.exprEnc.flat) { - right.output.head - } else { - val index = joined.right.output.indexWhere(_.exprId == a.exprId) - GetStructField(right.output.head, index) - } - } - - implicit val tuple2Encoder: Encoder[(T, U)] = - ExpressionEncoder.tuple(this.exprEnc, other.exprEnc) + if (joined.joinType.isInstanceOf[InnerLike]) { + // For inner joins, we can directly perform the join and then can project the join + // results into structs. This ensures that data remains flat during shuffles / + // exchanges (unlike the outer join path, which nests the data before shuffling). + withTypedPlan(Project(Seq(leftResultExpr, rightResultExpr), joined)) + } else { // outer joins + // For both join sides, combine all outputs into a single column and alias it with "_1 + // or "_2", to match the schema for the encoder of the join result. + // Note that we do this before joining them, to enable the join operator to return null + // for one side, in cases like outer-join. + val left = Project(leftResultExpr :: Nil, joined.left) + val right = Project(rightResultExpr :: Nil, joined.right) + + // Rewrites the join condition to make the attribute point to correct column/field, + // after we combine the outputs of each join side. + val conditionExpr = joined.condition.get transformUp { + case a: Attribute if joined.left.outputSet.contains(a) => + if (!this.exprEnc.isSerializedAsStructForTopLevel) { + left.output.head + } else { + val index = joined.left.output.indexWhere(_.exprId == a.exprId) + GetStructField(left.output.head, index) + } + case a: Attribute if joined.right.outputSet.contains(a) => + if (!other.exprEnc.isSerializedAsStructForTopLevel) { + right.output.head + } else { + val index = joined.right.output.indexWhere(_.exprId == a.exprId) + GetStructField(right.output.head, index) + } + } - withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr))) + withTypedPlan(Join(left, right, joined.joinType, Some(conditionExpr), JoinHint.NONE)) + } } /** @@ -1143,7 +1188,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def joinWith[U](other: Dataset[U], condition: Column): Dataset[(T, U)] = { joinWith(other, condition, "inner") } @@ -1385,12 +1430,12 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1](c1: TypedColumn[T, U1]): Dataset[U1] = { implicit val encoder = c1.encoder val project = Project(c1.withInputType(exprEnc, logicalPlan.output).named :: Nil, logicalPlan) - if (encoder.flat) { + if (!encoder.isSerializedAsStructForTopLevel) { new Dataset[U1](sparkSession, project, encoder) } else { // Flattens inner fields of U1 @@ -1419,7 +1464,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2](c1: TypedColumn[T, U1], c2: TypedColumn[T, U2]): Dataset[(U1, U2)] = selectUntyped(c1, c2).asInstanceOf[Dataset[(U1, U2)]] @@ -1431,7 +1476,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1446,7 +1491,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3, U4]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1462,7 +1507,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def select[U1, U2, U3, U4, U5]( c1: TypedColumn[T, U1], c2: TypedColumn[T, U2], @@ -1633,7 +1678,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def reduce(func: (T, T) => T): T = withNewRDDExecutionId { rdd.reduce(func) } @@ -1648,7 +1693,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def reduce(func: ReduceFunction[T]): T = reduce(func.call(_, _)) /** @@ -1660,7 +1705,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def groupByKey[K: Encoder](func: T => K): KeyValueGroupedDataset[K, T] = { val withGroupingKey = AppendColumns(func, logicalPlan) val executed = sparkSession.sessionState.executePlan(withGroupingKey) @@ -1682,7 +1727,7 @@ class Dataset[T] private[sql]( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def groupByKey[K](func: MapFunction[T, K], encoder: Encoder[K]): KeyValueGroupedDataset[K, T] = groupByKey(func.call(_))(encoder) @@ -1810,20 +1855,6 @@ class Dataset[T] private[sql]( Limit(Literal(n), logicalPlan) } - /** - * Returns a new Dataset containing union of rows in this Dataset and another Dataset. - * - * This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does - * deduplication of elements), use this function followed by a [[distinct]]. - * - * Also as standard in SQL, this function resolves columns by position (not by name). - * - * @group typedrel - * @since 2.0.0 - */ - @deprecated("use union()", "2.0.0") - def unionAll(other: Dataset[T]): Dataset[T] = union(other) - /** * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * @@ -1860,6 +1891,20 @@ class Dataset[T] private[sql]( CombineUnions(Union(logicalPlan, other.logicalPlan)) } + /** + * Returns a new Dataset containing union of rows in this Dataset and another Dataset. + * This is an alias for `union`. + * + * This is equivalent to `UNION ALL` in SQL. To do a SQL-style set union (that does + * deduplication of elements), use this function followed by a [[distinct]]. + * + * Also as standard in SQL, this function resolves columns by position (not by name). + * + * @group typedrel + * @since 2.0.0 + */ + def unionAll(other: Dataset[T]): Dataset[T] = union(other) + /** * Returns a new Dataset containing union of rows in this Dataset and another Dataset. * @@ -2123,90 +2168,6 @@ 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. @@ -2214,6 +2175,11 @@ class Dataset[T] private[sql]( * `column`'s expression must only refer to attributes supplied by this Dataset. It is an * error to add a column that refers to some other Dataset. * + * @note this method introduces a projection internally. Therefore, calling it multiple times, + * for instance, via loops in order to add multiple columns can generate big plans which + * can cause performance issues and even `StackOverflowException`. To avoid this, + * use `select` with the multiple columns at once. + * * @group untypedrel * @since 2.0.0 */ @@ -2356,7 +2322,7 @@ class Dataset[T] private[sql]( } val attrs = this.logicalPlan.output val colsAfterDrop = attrs.filter { attr => - attr != expression + !attr.semanticEquals(expression) }.map(attr => Column(attr)) select(colsAfterDrop : _*) } @@ -2582,7 +2548,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def filter(func: T => Boolean): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2596,7 +2562,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def filter(func: FilterFunction[T]): Dataset[T] = { withTypedPlan(TypedFilter(func, logicalPlan)) } @@ -2610,7 +2576,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def map[U : Encoder](func: T => U): Dataset[U] = withTypedPlan { MapElements[T, U](func, logicalPlan) } @@ -2624,7 +2590,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def map[U](func: MapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { implicit val uEnc = encoder withTypedPlan(MapElements[T, U](func, logicalPlan)) @@ -2639,7 +2605,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapPartitions[U : Encoder](func: Iterator[T] => Iterator[U]): Dataset[U] = { new Dataset[U]( sparkSession, @@ -2656,7 +2622,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapPartitions[U](f: MapPartitionsFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (Iterator[T]) => Iterator[U] = x => f.call(x.asJava).asScala mapPartitions(func)(encoder) @@ -2677,6 +2643,23 @@ class Dataset[T] private[sql]( MapPartitionsInR(func, packageNames, broadcastVars, schema, rowEncoder, logicalPlan)) } + /** + * Applies a Scalar iterator Pandas UDF to each partition. The user-defined function + * defines a transformation: `iter(pandas.DataFrame)` -> `iter(pandas.DataFrame)`. + * Each partition is each iterator consisting of DataFrames as batches. + * + * This function uses Apache Arrow as serialization format between Java executors and Python + * workers. + */ + private[sql] def mapInPandas(func: PythonUDF): DataFrame = { + Dataset.ofRows( + sparkSession, + MapInPandas( + func, + func.dataType.asInstanceOf[StructType].toAttributes, + logicalPlan)) + } + /** * :: Experimental :: * (Scala-specific) @@ -2687,7 +2670,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMap[U : Encoder](func: T => TraversableOnce[U]): Dataset[U] = mapPartitions(_.flatMap(func)) @@ -2701,7 +2684,7 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMap[U](f: FlatMapFunction[T, U], encoder: Encoder[U]): Dataset[U] = { val func: (T) => Iterator[U] = x => f.call(x).asScala flatMap(func)(encoder) @@ -2810,15 +2793,9 @@ class Dataset[T] private[sql]( */ def toLocalIterator(): java.util.Iterator[T] = { withAction("toLocalIterator", queryExecution) { plan => - // This projection writes output to a `InternalRow`, which means applying this projection is - // not thread-safe. Here we create the projection inside this method to make `Dataset` - // thread-safe. - val objProj = GenerateSafeProjection.generate(deserializer :: Nil) - plan.executeToIterator().map { row => - // The row returned by SafeProjection is `SpecificInternalRow`, which ignore the data type - // parameter of its `get` method, so it's safe to use null here. - objProj(row).get(0, null).asInstanceOf[T] - }.asJava + // `ExpressionEncoder` is not thread-safe, here we create a new encoder. + val enc = resolvedEnc.copy() + plan.executeToIterator().map(enc.fromRow).asJava } } @@ -2888,6 +2865,12 @@ class Dataset[T] private[sql]( * When no explicit sort order is specified, "ascending nulls first" is assumed. * Note, the rows are not sorted in each partition of the resulting Dataset. * + * + * Note that due to performance reasons this method uses sampling to estimate the ranges. + * Hence, the output may not be consistent, since sampling can return different values. + * The sample size can be controlled by the config + * `spark.sql.execution.rangeExchange.sampleSizePerPartition`. + * * @group typedrel * @since 2.3.0 */ @@ -2912,6 +2895,11 @@ class Dataset[T] private[sql]( * When no explicit sort order is specified, "ascending nulls first" is assumed. * Note, the rows are not sorted in each partition of the resulting Dataset. * + * Note that due to performance reasons this method uses sampling to estimate the ranges. + * Hence, the output may not be consistent, since sampling can return different values. + * The sample size can be controlled by the config + * `spark.sql.execution.rangeExchange.sampleSizePerPartition`. + * * @group typedrel * @since 2.3.0 */ @@ -3008,7 +2996,8 @@ class Dataset[T] private[sql]( * @since 1.6.0 */ def unpersist(blocking: Boolean): this.type = { - sparkSession.sharedState.cacheManager.uncacheQuery(this, cascade = false, blocking) + sparkSession.sharedState.cacheManager.uncacheQuery( + sparkSession, logicalPlan, cascade = false, blocking) this } @@ -3054,18 +3043,6 @@ 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. @@ -3175,7 +3152,7 @@ class Dataset[T] private[sql]( * @group basic * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def writeStream: DataStreamWriter[T] = { if (!isStreaming) { logicalPlan.failAnalysis( @@ -3233,6 +3210,8 @@ class Dataset[T] private[sql]( fr.inputFiles case r: HiveTableRelation => r.tableMeta.storage.locationUri.map(_.toString).toArray + case DataSourceV2Relation(table: FileTable, _, _) => + table.fileIndex.inputFiles }.flatten files.toSet.toArray } @@ -3264,7 +3243,7 @@ class Dataset[T] private[sql]( _numRows: Int, truncate: Int): Array[Any] = { EvaluatePython.registerPicklers() - val numRows = _numRows.max(0).min(Int.MaxValue - 1) + val numRows = _numRows.max(0).min(ByteArrayMethods.MAX_ROUNDED_ARRAY_LENGTH - 1) val rows = getRows(numRows, truncate).map(_.toArray).toArray val toJava: (Any) => Any = EvaluatePython.toJava(_, ArrayType(ArrayType(StringType))) val iter: Iterator[Array[Byte]] = new SerDeUtil.AutoBatchedPickler( @@ -3273,14 +3252,19 @@ class Dataset[T] private[sql]( } /** - * Collect a Dataset as Arrow batches and serve stream to PySpark. + * Collect a Dataset as Arrow batches and serve stream to SparkR. It sends + * arrow batches in an ordered manner with buffering. This is inevitable + * due to missing R API that reads batches from socket directly. See ARROW-4512. + * Eventually, this code should be deduplicated by `collectAsArrowToPython`. */ - private[sql] def collectAsArrowToPython(): Array[Any] = { + private[sql] def collectAsArrowToR(): Array[Any] = { val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone - withAction("collectAsArrowToPython", queryExecution) { plan => - PythonRDD.serveToStream("serve-Arrow") { out => - val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId) + withAction("collectAsArrowToR", queryExecution) { plan => + RRDD.serveToStream("serve-Arrow") { outputStream => + val buffer = new ByteArrayOutputStream() + val out = new DataOutputStream(outputStream) + val batchWriter = new ArrowBatchStreamWriter(schema, buffer, timeZoneId) val arrowBatchRdd = toArrowBatchRdd(plan) val numPartitions = arrowBatchRdd.partitions.length @@ -3303,6 +3287,9 @@ class Dataset[T] private[sql]( // After last batch, end the stream if (lastIndex == results.length) { batchWriter.end() + val batches = buffer.toByteArray + out.writeInt(batches.length) + out.write(batches) } } else { // Store partitions received out of order @@ -3319,6 +3306,55 @@ class Dataset[T] private[sql]( } } + /** + * Collect a Dataset as Arrow batches and serve stream to PySpark. It sends + * arrow batches in an un-ordered manner without buffering, and then batch order + * information at the end. The batches should be reordered at Python side. + */ + private[sql] def collectAsArrowToPython: Array[Any] = { + val timeZoneId = sparkSession.sessionState.conf.sessionLocalTimeZone + + withAction("collectAsArrowToPython", queryExecution) { plan => + PythonRDD.serveToStream("serve-Arrow") { outputStream => + val out = new DataOutputStream(outputStream) + val batchWriter = new ArrowBatchStreamWriter(schema, out, timeZoneId) + + // Batches ordered by (index of partition, batch index in that partition) tuple + val batchOrder = ArrayBuffer.empty[(Int, Int)] + + // Handler to eagerly write batches to Python as they arrive, un-ordered + val handlePartitionBatches = (index: Int, arrowBatches: Array[Array[Byte]]) => + if (arrowBatches.nonEmpty) { + // Write all batches (can be more than 1) in the partition, store the batch order tuple + batchWriter.writeBatches(arrowBatches.iterator) + arrowBatches.indices.foreach { + partitionBatchIndex => batchOrder.append((index, partitionBatchIndex)) + } + } + + Utils.tryWithSafeFinally { + val arrowBatchRdd = toArrowBatchRdd(plan) + sparkSession.sparkContext.runJob( + arrowBatchRdd, + (it: Iterator[Array[Byte]]) => it.toArray, + handlePartitionBatches) + } { + // After processing all partitions, end the batch stream + batchWriter.end() + + // Write batch order indices + out.writeInt(batchOrder.length) + // Sort by (index of partition, batch index in that partition) tuple to get the + // overall_batch_index from 0 to N-1 batches, which can be used to put the + // transferred batches in the correct order + batchOrder.zipWithIndex.sortBy(_._1).foreach { case (_, overallBatchIndex) => + out.writeInt(overallBatchIndex) + } + } + } + } + } + private[sql] def toPythonIterator(): Array[Any] = { withNewExecutionId { PythonRDD.toLocalIteratorAndServe(javaToPython.rdd) @@ -3356,21 +3392,11 @@ class Dataset[T] private[sql]( * user-registered callback functions. */ private def withAction[U](name: String, qe: QueryExecution)(action: SparkPlan => U) = { - try { + SQLExecution.withNewExecutionId(sparkSession, qe, Some(name)) { qe.executedPlan.foreach { plan => plan.resetMetrics() } - val start = System.nanoTime() - val result = SQLExecution.withNewExecutionId(sparkSession, qe) { - action(qe.executedPlan) - } - val end = System.nanoTime() - sparkSession.listenerManager.onSuccess(name, qe, end - start) - result - } catch { - case e: Exception => - sparkSession.listenerManager.onFailure(name, qe, e) - throw e + action(qe.executedPlan) } } @@ -3378,14 +3404,9 @@ class Dataset[T] private[sql]( * Collect all elements from a spark plan. */ private def collectFromPlan(plan: SparkPlan): Array[T] = { - // This projection writes output to a `InternalRow`, which means applying this projection is not - // thread-safe. Here we create the projection inside this method to make `Dataset` thread-safe. - val objProj = GenerateSafeProjection.generate(deserializer :: Nil) - plan.executeCollect().map { row => - // The row returned by SafeProjection is `SpecificInternalRow`, which ignore the data type - // parameter of its `get` method, so it's safe to use null here. - objProj(row).get(0, null).asInstanceOf[T] - } + // `ExpressionEncoder` is not thread-safe, here we create a new encoder. + val enc = resolvedEnc.copy() + plan.executeCollect().map(enc.fromRow) } private def sortInternal(global: Boolean, sortExprs: Seq[Column]): Dataset[T] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala index 08aa1bbe78fae..1c4ffefb897ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DatasetHolder.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable /** * A container for a [[Dataset]], used for implicit conversions in Scala. @@ -30,7 +30,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 1.6.0 */ -@InterfaceStability.Stable +@Stable case class DatasetHolder[T] private[sql](private val ds: Dataset[T]) { // This is declared with parentheses to prevent the Scala compiler from treating diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala index bd8dd6ea3fe0f..302d38cde1430 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ExperimentalMethods.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule @@ -33,7 +33,7 @@ import org.apache.spark.sql.catalyst.rules.Rule * @since 1.3.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable class ExperimentalMethods private[sql]() { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala index b21c50af18433..5c0fe798b1044 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/ForeachWriter.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * The abstract class for writing custom logic to process data generated by a query. @@ -104,7 +104,7 @@ import org.apache.spark.annotation.InterfaceStability * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving abstract class ForeachWriter[T] extends Serializable { // TODO: Move this to org.apache.spark.sql.util or consolidate this with batch API. @@ -130,8 +130,11 @@ abstract class ForeachWriter[T] extends Serializable { * Called when stopping to process one partition of new data in the executor side. This is * guaranteed to be called either `open` returns `true` or `false`. However, * `close` won't be called in the following cases: - * - JVM crashes without throwing a `Throwable` - * - `open` throws a `Throwable`. + * + *
      + *
    • JVM crashes without throwing a `Throwable`
    • + *
    • `open` throws a `Throwable`.
    • + *
    * * @param errorOrNull the error thrown during processing data or null if there was no error. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala index 6bab21dca0cbd..0da52d432d25d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/KeyValueGroupedDataset.scala @@ -19,13 +19,14 @@ package org.apache.spark.sql import scala.collection.JavaConverters._ -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.api.java.function._ import org.apache.spark.sql.catalyst.encoders.{encoderFor, ExpressionEncoder} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CreateStruct} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.expressions.ReduceAggregator +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode} /** @@ -37,7 +38,7 @@ import org.apache.spark.sql.streaming.{GroupState, GroupStateTimeout, OutputMode * @since 2.0.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving class KeyValueGroupedDataset[K, V] private[sql]( kEncoder: Encoder[K], vEncoder: Encoder[V], @@ -237,7 +238,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S: Encoder, U: Encoder]( func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = { val flatMapFunc = (key: K, it: Iterator[V], s: GroupState[S]) => Iterator(func(key, it, s)) @@ -272,7 +273,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S: Encoder, U: Encoder]( timeoutConf: GroupStateTimeout)( func: (K, Iterator[V], GroupState[S]) => U): Dataset[U] = { @@ -309,7 +310,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S, U]( func: MapGroupsWithStateFunction[K, V, S, U], stateEncoder: Encoder[S], @@ -340,7 +341,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def mapGroupsWithState[S, U]( func: MapGroupsWithStateFunction[K, V, S, U], stateEncoder: Encoder[S], @@ -371,7 +372,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMapGroupsWithState[S: Encoder, U: Encoder]( outputMode: OutputMode, timeoutConf: GroupStateTimeout)( @@ -413,7 +414,7 @@ class KeyValueGroupedDataset[K, V] private[sql]( * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def flatMapGroupsWithState[S, U]( func: FlatMapGroupsWithStateFunction[K, V, S, U], outputMode: OutputMode, @@ -457,9 +458,13 @@ class KeyValueGroupedDataset[K, V] private[sql]( val encoders = columns.map(_.encoder) val namedColumns = columns.map(_.withInputType(vExprEnc, dataAttributes).named) - val keyColumn = if (kExprEnc.flat) { + val keyColumn = if (!kExprEnc.isSerializedAsStructForTopLevel) { assert(groupingAttributes.length == 1) - groupingAttributes.head + if (SQLConf.get.nameNonStructGroupingKeyAsValue) { + groupingAttributes.head + } else { + Alias(groupingAttributes.head, "key")() + } } else { Alias(CreateStruct(groupingAttributes), "key")() } @@ -515,6 +520,71 @@ class KeyValueGroupedDataset[K, V] private[sql]( col4: TypedColumn[V, U4]): Dataset[(K, U1, U2, U3, U4)] = aggUntyped(col1, col2, col3, col4).asInstanceOf[Dataset[(K, U1, U2, U3, U4)]] + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5]): Dataset[(K, U1, U2, U3, U4, U5)] = + aggUntyped(col1, col2, col3, col4, col5).asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6]): Dataset[(K, U1, U2, U3, U4, U5, U6)] = + aggUntyped(col1, col2, col3, col4, col5, col6) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7)]] + + /** + * Computes the given aggregations, returning a [[Dataset]] of tuples for each unique key + * and the result of computing these aggregations over all elements in the group. + * + * @since 3.0.0 + */ + def agg[U1, U2, U3, U4, U5, U6, U7, U8]( + col1: TypedColumn[V, U1], + col2: TypedColumn[V, U2], + col3: TypedColumn[V, U3], + col4: TypedColumn[V, U4], + col5: TypedColumn[V, U5], + col6: TypedColumn[V, U6], + col7: TypedColumn[V, U7], + col8: TypedColumn[V, U8]): Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)] = + aggUntyped(col1, col2, col3, col4, col5, col6, col7, col8) + .asInstanceOf[Dataset[(K, U1, U2, U3, U4, U5, U6, U7, U8)]] + /** * Returns a [[Dataset]] that contains a tuple with each key and the number of items present * for that key. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala index d4e75b5ebd405..e85636d82a62c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RelationalGroupedDataset.scala @@ -22,7 +22,7 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.language.implicitConversions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.broadcast.Broadcast import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedAlias, UnresolvedAttribute, UnresolvedFunction} @@ -45,7 +45,7 @@ import org.apache.spark.sql.types.{NumericType, StructType} * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class RelationalGroupedDataset protected[sql]( df: DataFrame, groupingExprs: Seq[Expression], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala index 3c39579149fff..0f5aab7f47d0d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/RuntimeConfig.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.internal.config.{ConfigEntry, OptionalConfigEntry} import org.apache.spark.sql.internal.SQLConf - /** * Runtime configuration interface for Spark. To access this, use `SparkSession.conf`. * @@ -29,7 +28,7 @@ import org.apache.spark.sql.internal.SQLConf * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { /** @@ -154,5 +153,9 @@ class RuntimeConfig private[sql](sqlConf: SQLConf = new SQLConf) { if (SQLConf.staticConfKeys.contains(key)) { throw new AnalysisException(s"Cannot modify the value of a static config: $key") } + if (sqlConf.setCommandRejectsSparkCoreConfs && + ConfigEntry.findEntry(key) != null && !SQLConf.sqlConfEntries.containsKey(key)) { + throw new AnalysisException(s"Cannot modify the value of a Spark config: $key") + } } } 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 dfb12f272eb2f..08b7521de9573 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 @@ -23,8 +23,8 @@ import scala.collection.immutable import scala.reflect.runtime.universe.TypeTag import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} -import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.annotation._ +import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.internal.config.ConfigEntry import org.apache.spark.rdd.RDD @@ -54,7 +54,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * @groupname Ungrouped Support functions for language integrated queries * @since 1.0.0 */ -@InterfaceStability.Stable +@Stable class SQLContext private[sql](val sparkSession: SparkSession) extends Logging with Serializable { @@ -64,15 +64,6 @@ 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 @@ -95,7 +86,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * that listen for execution metrics. */ @Experimental - @InterfaceStability.Evolving + @Evolving def listenerManager: ExecutionListenerManager = sparkSession.listenerManager /** @@ -167,7 +158,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) */ @Experimental @transient - @InterfaceStability.Unstable + @Unstable def experimental: ExperimentalMethods = sparkSession.experimental /** @@ -253,7 +244,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = self } @@ -267,7 +258,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { sparkSession.createDataFrame(rdd) } @@ -280,7 +271,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { sparkSession.createDataFrame(data) } @@ -328,20 +319,11 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } - /** - * Creates a DataFrame from an RDD[Row]. User can specify whether the input rows should be - * converted to Catalyst rows. - */ - private[sql] - def createDataFrame(rowRDD: RDD[Row], schema: StructType, needsConversion: Boolean) = { - sparkSession.createDataFrame(rowRDD, schema, needsConversion) - } - /** * :: Experimental :: * Creates a [[Dataset]] from a local Seq of data of a given type. This method requires an @@ -372,7 +354,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -410,7 +392,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataset */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { sparkSession.createDataset(data) } @@ -437,7 +419,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.3.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rowRDD, schema) } @@ -452,7 +434,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @since 1.6.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { sparkSession.createDataFrame(rows, schema) } @@ -516,7 +498,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def readStream: DataStreamReader = sparkSession.readStream @@ -640,7 +622,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(end: Long): DataFrame = sparkSession.range(end).toDF() /** @@ -652,7 +634,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long): DataFrame = sparkSession.range(start, end).toDF() /** @@ -664,7 +646,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long): DataFrame = { sparkSession.range(start, end, step).toDF() } @@ -679,7 +661,7 @@ class SQLContext private[sql](val sparkSession: SparkSession) * @group dataframe */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): DataFrame = { sparkSession.range(start, end, step, numPartitions).toDF() } @@ -755,289 +737,6 @@ 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) - } } /** @@ -1050,45 +749,6 @@ 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/SQLImplicits.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala index 05db292bd41b1..c997b7d8e0bfb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLImplicits.scala @@ -21,7 +21,7 @@ import scala.collection.Map import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder @@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder * * @since 1.6.0 */ -@InterfaceStability.Evolving +@Evolving abstract class SQLImplicits extends LowPrioritySQLImplicits { protected def _sqlContext: SQLContext @@ -81,9 +81,14 @@ abstract class SQLImplicits extends LowPrioritySQLImplicits { /** @since 2.2.0 */ implicit def newDateEncoder: Encoder[java.sql.Date] = Encoders.DATE + /** @since 3.0.0 */ + implicit def newLocalDateEncoder: Encoder[java.time.LocalDate] = Encoders.LOCALDATE + /** @since 2.2.0 */ implicit def newTimeStampEncoder: Encoder[java.sql.Timestamp] = Encoders.TIMESTAMP + /** @since 3.0.0 */ + implicit def newInstantEncoder: Encoder[java.time.Instant] = Encoders.INSTANT // Boxed primitives diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala index 2b847fb6f9458..90d1b9205787b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala @@ -18,19 +18,22 @@ package org.apache.spark.sql import java.io.Closeable +import java.util.concurrent.TimeUnit._ import java.util.concurrent.atomic.AtomicReference import scala.collection.JavaConverters._ +import scala.collection.mutable import scala.reflect.runtime.universe.TypeTag import scala.util.control.NonFatal import org.apache.spark.{SPARK_VERSION, SparkConf, SparkContext, TaskContext} -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental, Stable, Unstable} import org.apache.spark.api.java.JavaRDD import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.catalog.Catalog +import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Catalogs} import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.encoders._ @@ -73,7 +76,7 @@ import org.apache.spark.util.{CallSite, Utils} * @param parentSessionState If supplied, inherit all session state (i.e. temporary * views, SQL config, UDFs etc) from parent. */ -@InterfaceStability.Stable +@Stable class SparkSession private( @transient val sparkContext: SparkContext, @transient private val existingSharedState: Option[SharedState], @@ -84,8 +87,17 @@ class SparkSession private( // The call site where this SparkSession was constructed. private val creationSite: CallSite = Utils.getCallSite() + /** + * Constructor used in Pyspark. Contains explicit application of Spark Session Extensions + * which otherwise only occurs during getOrCreate. We cannot add this to the default constructor + * since that would cause every new session to reinvoke Spark Session Extensions on the currently + * running extensions. + */ private[sql] def this(sc: SparkContext) { - this(sc, None, None, new SparkSessionExtensions) + this(sc, None, None, + SparkSession.applyExtensions( + sc.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), + new SparkSessionExtensions)) } sparkContext.assertNotStopped() @@ -115,10 +127,10 @@ class SparkSession private( * * @since 2.2.0 */ - @InterfaceStability.Unstable + @Unstable @transient lazy val sharedState: SharedState = { - existingSharedState.getOrElse(new SharedState(sparkContext)) + existingSharedState.getOrElse(new SharedState(sparkContext, initialSessionOptions)) } /** @@ -136,7 +148,7 @@ class SparkSession private( * * @since 2.2.0 */ - @InterfaceStability.Unstable + @Unstable @transient lazy val sessionState: SessionState = { parentSessionState @@ -177,7 +189,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def listenerManager: ExecutionListenerManager = sessionState.listenerManager /** @@ -188,7 +200,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Unstable + @Unstable def experimental: ExperimentalMethods = sessionState.experimentalMethods /** @@ -222,7 +234,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Unstable + @Unstable def streams: StreamingQueryManager = sessionState.streamingQueryManager /** @@ -280,7 +292,7 @@ class SparkSession private( * @return 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def emptyDataset[T: Encoder]: Dataset[T] = { val encoder = implicitly[Encoder[T]] new Dataset(self, LocalRelation(encoder.schema.toAttributes), encoder) @@ -293,7 +305,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](rdd: RDD[A]): DataFrame = { SparkSession.setActiveSession(this) val encoder = Encoders.product[A] @@ -307,7 +319,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataFrame[A <: Product : TypeTag](data: Seq[A]): DataFrame = { SparkSession.setActiveSession(this) val schema = ScalaReflection.schemaFor[A].dataType.asInstanceOf[StructType] @@ -347,9 +359,13 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: RDD[Row], schema: StructType): DataFrame = { - createDataFrame(rowRDD, schema, needsConversion = true) + // TODO: use MutableProjection when rowRDD is another DataFrame and the applied + // schema differs from the existing schema on any field data type. + val encoder = RowEncoder(schema) + val catalystRows = rowRDD.map(encoder.toRow) + internalCreateDataFrame(catalystRows.setName(rowRDD.name), schema) } /** @@ -361,7 +377,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rowRDD: JavaRDD[Row], schema: StructType): DataFrame = { createDataFrame(rowRDD.rdd, schema) } @@ -375,7 +391,7 @@ class SparkSession private( * @since 2.0.0 */ @DeveloperApi - @InterfaceStability.Evolving + @Evolving def createDataFrame(rows: java.util.List[Row], schema: StructType): DataFrame = { Dataset.ofRows(self, LocalRelation.fromExternalRows(schema.toAttributes, rows.asScala)) } @@ -465,7 +481,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: Seq[T]): Dataset[T] = { val enc = encoderFor[T] val attributes = enc.schema.toAttributes @@ -484,7 +500,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: RDD[T]): Dataset[T] = { Dataset[T](self, ExternalRDD(data, self)) } @@ -506,7 +522,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createDataset[T : Encoder](data: java.util.List[T]): Dataset[T] = { createDataset(data.asScala) } @@ -519,7 +535,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(end: Long): Dataset[java.lang.Long] = range(0, end) /** @@ -530,7 +546,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long): Dataset[java.lang.Long] = { range(start, end, step = 1, numPartitions = sparkContext.defaultParallelism) } @@ -543,7 +559,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long): Dataset[java.lang.Long] = { range(start, end, step, numPartitions = sparkContext.defaultParallelism) } @@ -557,7 +573,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def range(start: Long, end: Long, step: Long, numPartitions: Int): Dataset[java.lang.Long] = { new Dataset(self, Range(start, end, step, numPartitions), Encoders.LONG) } @@ -578,25 +594,6 @@ class SparkSession private( Dataset.ofRows(self, logicalPlan) } - /** - * Creates a `DataFrame` from an `RDD[Row]`. - * User can specify whether the input rows should be converted to Catalyst rows. - */ - private[sql] def createDataFrame( - rowRDD: RDD[Row], - schema: StructType, - needsConversion: Boolean) = { - // TODO: use MutableProjection when rowRDD is another DataFrame and the applied - // schema differs from the existing schema on any field data type. - val catalystRows = if (needsConversion) { - val encoder = RowEncoder(schema) - rowRDD.map(encoder.toRow) - } else { - rowRDD.map { r: Row => InternalRow.fromSeq(r.toSeq) } - } - internalCreateDataFrame(catalystRows.setName(rowRDD.name), schema) - } - /* ------------------------- * | Catalog-related methods | @@ -610,6 +607,12 @@ class SparkSession private( */ @transient lazy val catalog: Catalog = new CatalogImpl(self) + @transient private lazy val catalogs = new mutable.HashMap[String, CatalogPlugin]() + + private[sql] def catalog(name: String): CatalogPlugin = synchronized { + catalogs.getOrElseUpdate(name, Catalogs.load(name, sessionState.conf)) + } + /** * Returns the specified table/view as a `DataFrame`. * @@ -621,7 +624,11 @@ class SparkSession private( * @since 2.0.0 */ def table(tableName: String): DataFrame = { - table(sessionState.sqlParser.parseTableIdentifier(tableName)) + table(sessionState.sqlParser.parseMultipartIdentifier(tableName)) + } + + private[sql] def table(multipartIdentifier: Seq[String]): DataFrame = { + Dataset.ofRows(self, UnresolvedRelation(multipartIdentifier)) } private[sql] def table(tableIdent: TableIdentifier): DataFrame = { @@ -639,7 +646,11 @@ class SparkSession private( * @since 2.0.0 */ def sql(sqlText: String): DataFrame = { - Dataset.ofRows(self, sessionState.sqlParser.parsePlan(sqlText)) + val tracker = new QueryPlanningTracker + val plan = tracker.measurePhase(QueryPlanningTracker.PARSING) { + sessionState.sqlParser.parsePlan(sqlText) + } + Dataset.ofRows(self, plan, tracker) } /** @@ -663,7 +674,7 @@ class SparkSession private( * * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving def readStream: DataStreamReader = new DataStreamReader(self) /** @@ -677,7 +688,7 @@ class SparkSession private( val ret = f val end = System.nanoTime() // scalastyle:off println - println(s"Time taken: ${(end - start) / 1000 / 1000} ms") + println(s"Time taken: ${NANOSECONDS.toMillis(end - start)} ms") // scalastyle:on println ret } @@ -697,7 +708,7 @@ class SparkSession private( * @since 2.0.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving object implicits extends SQLImplicits with Serializable { protected override def _sqlContext: SQLContext = SparkSession.this.sqlContext } @@ -766,13 +777,13 @@ class SparkSession private( } -@InterfaceStability.Stable +@Stable object SparkSession extends Logging { /** * Builder for [[SparkSession]]. */ - @InterfaceStability.Stable + @Stable class Builder extends Logging { private[this] val options = new scala.collection.mutable.HashMap[String, String] @@ -936,23 +947,9 @@ object SparkSession extends Logging { // Do not update `SparkConf` for existing `SparkContext`, as it's shared by all sessions. } - // Initialize extensions if the user has defined a configurator class. - val extensionConfOption = sparkContext.conf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS) - if (extensionConfOption.isDefined) { - val extensionConfClassName = extensionConfOption.get - try { - val extensionConfClass = Utils.classForName(extensionConfClassName) - val extensionConf = extensionConfClass.newInstance() - .asInstanceOf[SparkSessionExtensions => Unit] - extensionConf(extensions) - } catch { - // Ignore the error if we cannot find the class or when the class has the wrong type. - case e @ (_: ClassCastException | - _: ClassNotFoundException | - _: NoClassDefFoundError) => - logWarning(s"Cannot use $extensionConfClassName to configure session extensions.", e) - } - } + applyExtensions( + sparkContext.getConf.get(StaticSQLConf.SPARK_SESSION_EXTENSIONS).getOrElse(Seq.empty), + extensions) session = new SparkSession(sparkContext, None, None, extensions) options.foreach { case (k, v) => session.initialSessionOptions.put(k, v) } @@ -1137,4 +1134,28 @@ object SparkSession extends Logging { SparkSession.clearDefaultSession() } } + + /** + * Initialize extensions for given extension classnames. The classes will be applied to the + * extensions passed into this function. + */ + private def applyExtensions( + extensionConfClassNames: Seq[String], + extensions: SparkSessionExtensions): SparkSessionExtensions = { + extensionConfClassNames.foreach { extensionConfClassName => + try { + val extensionConfClass = Utils.classForName(extensionConfClassName) + val extensionConf = extensionConfClass.getConstructor().newInstance() + .asInstanceOf[SparkSessionExtensions => Unit] + extensionConf(extensions) + } catch { + // Ignore the error if we cannot find the class or when the class has the wrong type. + case e@(_: ClassCastException | + _: ClassNotFoundException | + _: NoClassDefFoundError) => + logWarning(s"Cannot use $extensionConfClassName to configure session extensions.", e) + } + } + extensions + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala index f99c108161f94..1c2bf9e7c2a57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSessionExtensions.scala @@ -19,10 +19,15 @@ package org.apache.spark.sql import scala.collection.mutable -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Experimental, Unstable} +import org.apache.spark.sql.catalyst.FunctionIdentifier +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry +import org.apache.spark.sql.catalyst.analysis.FunctionRegistry.FunctionBuilder +import org.apache.spark.sql.catalyst.expressions.ExpressionInfo import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.ColumnarRule /** * :: Experimental :: @@ -30,19 +35,23 @@ import org.apache.spark.sql.catalyst.rules.Rule * regarding binary compatibility and source compatibility of methods here. * * This current provides the following extension points: - * - Analyzer Rules. - * - Check Analysis Rules - * - Optimizer Rules. - * - Planning Strategies. - * - Customized Parser. - * - (External) Catalog listeners. * - * The extensions can be used by calling withExtension on the [[SparkSession.Builder]], for + *
      + *
    • Analyzer Rules.
    • + *
    • Check Analysis Rules.
    • + *
    • Optimizer Rules.
    • + *
    • Planning Strategies.
    • + *
    • Customized Parser.
    • + *
    • (External) Catalog listeners.
    • + *
    • Columnar Rules.
    • + *
    + * + * The extensions can be used by calling `withExtensions` on the [[SparkSession.Builder]], for * example: * {{{ * SparkSession.builder() * .master("...") - * .conf("...", true) + * .config("...", true) * .withExtensions { extensions => * extensions.injectResolutionRule { session => * ... @@ -54,17 +63,55 @@ import org.apache.spark.sql.catalyst.rules.Rule * .getOrCreate() * }}} * + * The extensions can also be used by setting the Spark SQL configuration property + * `spark.sql.extensions`. Multiple extensions can be set using a comma-separated list. For example: + * {{{ + * SparkSession.builder() + * .master("...") + * .config("spark.sql.extensions", "org.example.MyExtensions") + * .getOrCreate() + * + * class MyExtensions extends Function1[SparkSessionExtensions, Unit] { + * override def apply(extensions: SparkSessionExtensions): Unit = { + * extensions.injectResolutionRule { session => + * ... + * } + * extensions.injectParser { (session, parser) => + * ... + * } + * } + * } + * }}} + * * Note that none of the injected builders should assume that the [[SparkSession]] is fully * initialized and should not touch the session's internals (e.g. the SessionState). */ @DeveloperApi @Experimental -@InterfaceStability.Unstable +@Unstable class SparkSessionExtensions { type RuleBuilder = SparkSession => Rule[LogicalPlan] type CheckRuleBuilder = SparkSession => LogicalPlan => Unit type StrategyBuilder = SparkSession => Strategy type ParserBuilder = (SparkSession, ParserInterface) => ParserInterface + type FunctionDescription = (FunctionIdentifier, ExpressionInfo, FunctionBuilder) + type ColumnarRuleBuilder = SparkSession => ColumnarRule + + private[this] val columnarRuleBuilders = mutable.Buffer.empty[ColumnarRuleBuilder] + + /** + * Build the override rules for columnar execution. + */ + private[sql] def buildColumnarRules(session: SparkSession): Seq[ColumnarRule] = { + columnarRuleBuilders.map(_.apply(session)) + } + + /** + * Inject a rule that can override the columnar execution of an executor. + */ + def injectColumnar(builder: ColumnarRuleBuilder): Unit = { + columnarRuleBuilders += builder + } private[this] val resolutionRuleBuilders = mutable.Buffer.empty[RuleBuilder] @@ -168,4 +215,21 @@ class SparkSessionExtensions { def injectParser(builder: ParserBuilder): Unit = { parserBuilders += builder } + + private[this] val injectedFunctions = mutable.Buffer.empty[FunctionDescription] + + private[sql] def registerFunctions(functionRegistry: FunctionRegistry) = { + for ((name, expressionInfo, function) <- injectedFunctions) { + functionRegistry.registerFunction(name, expressionInfo, function) + } + functionRegistry + } + + /** + * Injects a custom function into the [[org.apache.spark.sql.catalyst.analysis.FunctionRegistry]] + * at runtime for all sessions. + */ + def injectFunction(functionDescription: FunctionDescription): Unit = { + injectedFunctions += functionDescription + } } 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 c37ba0c60c3d4..bb05c76cfee6d 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 @@ -22,7 +22,7 @@ import java.lang.reflect.ParameterizedType import scala.reflect.runtime.universe.TypeTag import scala.util.Try -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.api.python.PythonEvalType import org.apache.spark.internal.Logging import org.apache.spark.sql.api.java._ @@ -44,7 +44,7 @@ import org.apache.spark.util.Utils * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends Logging { protected[sql] def registerPython(name: String, udf: UserDefinedPythonFunction): Unit = { @@ -113,7 +113,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"ScalaReflection.schemaFor[A$i] :: $s"}) + val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor[A$i]).toOption :: $s"}) println(s""" |/** | * Registers a deterministic Scala closure of $x arguments as user-defined function (UDF). @@ -122,17 +122,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try($inputSchemas).toOption + | val inputSchemas: Seq[Option[ScalaReflection.Schema]] = $inputSchemas + | val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + | val finalUdf = if (nullable) udf else udf.asNonNullable() | def builder(e: Seq[Expression]) = if (e.length == $x) { - | ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - | udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + | finalUdf.createScalaUDF(e) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $x; Found: " + e.length) | } | functionRegistry.createOrReplaceTempFunction(name, builder) - | val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - | if (nullable) udf else udf.asNonNullable() + | finalUdf |}""".stripMargin) } @@ -142,16 +142,16 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") val version = if (i == 0) "2.3.0" else "1.3.0" - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Register a deterministic Java UDF$i instance as user-defined function (UDF). | * @since $version | */ |def register(name: String, f: UDF$i[$extTypeArgs], returnType: DataType): Unit = { - | val func = f$anyCast.call($anyParams) + | val func = $funcCall | def builder(e: Seq[Expression]) = if (e.length == $i) { - | ScalaUDF($funcCall, returnType, e, udfName = Some(name)) + | ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) | } else { | throw new AnalysisException("Invalid number of arguments for function " + name + | ". Expected: $i; Found: " + e.length) @@ -168,17 +168,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -188,17 +188,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: Nil).toOption + val inputSchemas: Seq[Option[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1]).toOption :: Nil + val udf = SparkUserDefinedFunction(func, dataType, inputSchemas).withName(name) + val finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 1) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -208,17 +208,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 2) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -228,17 +228,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 3) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -248,17 +248,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 4) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -268,17 +268,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 5) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -288,17 +288,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 6) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -308,17 +308,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 7) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -328,17 +328,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 8) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -348,17 +348,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 9) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -368,17 +368,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 10) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -388,17 +388,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 11) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -408,17 +408,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 12) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -428,17 +428,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 13) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -448,17 +448,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 14) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -468,17 +468,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 15) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -488,17 +488,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 16) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -508,17 +508,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 17) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -528,17 +528,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 18) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -548,17 +548,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 19) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -568,17 +568,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 20) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -588,17 +588,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 21) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } /** @@ -608,17 +608,17 @@ 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: Option[Seq[ScalaReflection.Schema]] = Try(ScalaReflection.schemaFor[A1] :: ScalaReflection.schemaFor[A2] :: ScalaReflection.schemaFor[A3] :: ScalaReflection.schemaFor[A4] :: ScalaReflection.schemaFor[A5] :: ScalaReflection.schemaFor[A6] :: ScalaReflection.schemaFor[A7] :: ScalaReflection.schemaFor[A8] :: ScalaReflection.schemaFor[A9] :: ScalaReflection.schemaFor[A10] :: ScalaReflection.schemaFor[A11] :: ScalaReflection.schemaFor[A12] :: ScalaReflection.schemaFor[A13] :: ScalaReflection.schemaFor[A14] :: ScalaReflection.schemaFor[A15] :: ScalaReflection.schemaFor[A16] :: ScalaReflection.schemaFor[A17] :: ScalaReflection.schemaFor[A18] :: ScalaReflection.schemaFor[A19] :: ScalaReflection.schemaFor[A20] :: ScalaReflection.schemaFor[A21] :: ScalaReflection.schemaFor[A22] :: Nil).toOption + 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 finalUdf = if (nullable) udf else udf.asNonNullable() def builder(e: Seq[Expression]) = if (e.length == 22) { - ScalaUDF(func, dataType, e, inputSchemas.map(_.map(_.dataType)).getOrElse(Nil), Some(name), nullable, - udfDeterministic = true, nullableTypes = inputSchemas.map(_.map(_.nullable)).getOrElse(Nil)) + finalUdf.createScalaUDF(e) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 22; Found: " + e.length) } functionRegistry.createOrReplaceTempFunction(name, builder) - val udf = SparkUserDefinedFunction.create(func, dataType, inputSchemas).withName(name) - if (nullable) udf else udf.asNonNullable() + finalUdf } ////////////////////////////////////////////////////////////////////////////////////////////// @@ -635,7 +635,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends private[sql] def registerJava(name: String, className: String, returnDataType: DataType): Unit = { try { - val clazz = Utils.classForName(className) + val clazz = Utils.classForName[AnyRef](className) val udfInterfaces = clazz.getGenericInterfaces .filter(_.isInstanceOf[ParameterizedType]) .map(_.asInstanceOf[ParameterizedType]) @@ -646,7 +646,7 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends throw new AnalysisException(s"It is invalid to implement multiple UDF interfaces, UDF class $className") } else { try { - val udf = clazz.newInstance() + val udf = clazz.getConstructor().newInstance() val udfReturnType = udfInterfaces(0).getActualTypeArguments.last var returnType = returnDataType if (returnType == null) { @@ -699,11 +699,11 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends */ private[sql] def registerJavaUDAF(name: String, className: String): Unit = { try { - val clazz = Utils.classForName(className) + val clazz = Utils.classForName[AnyRef](className) if (!classOf[UserDefinedAggregateFunction].isAssignableFrom(clazz)) { throw new AnalysisException(s"class $className doesn't implement interface UserDefinedAggregateFunction") } - val udaf = clazz.newInstance().asInstanceOf[UserDefinedAggregateFunction] + val udaf = clazz.getConstructor().newInstance().asInstanceOf[UserDefinedAggregateFunction] register(name, udaf) } catch { case e: ClassNotFoundException => throw new AnalysisException(s"Can not load class ${className}, please make sure it is on the classpath") @@ -717,9 +717,9 @@ class UDFRegistration private[sql] (functionRegistry: FunctionRegistry) extends * @since 2.3.0 */ def register(name: String, f: UDF0[_], returnType: DataType): Unit = { - val func = f.asInstanceOf[UDF0[Any]].call() + val func = () => f.asInstanceOf[UDF0[Any]].call() def builder(e: Seq[Expression]) = if (e.length == 0) { - ScalaUDF(() => func, returnType, e, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 0; Found: " + e.length) @@ -734,7 +734,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 1; Found: " + e.length) @@ -749,7 +749,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 2; Found: " + e.length) @@ -764,7 +764,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 3; Found: " + e.length) @@ -779,7 +779,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 4; Found: " + e.length) @@ -794,7 +794,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 5; Found: " + e.length) @@ -809,7 +809,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 6; Found: " + e.length) @@ -824,7 +824,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 7; Found: " + e.length) @@ -839,7 +839,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 8; Found: " + e.length) @@ -854,7 +854,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 9; Found: " + e.length) @@ -869,7 +869,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 10; Found: " + e.length) @@ -884,7 +884,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 11; Found: " + e.length) @@ -899,7 +899,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 12; Found: " + e.length) @@ -914,7 +914,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 13; Found: " + e.length) @@ -929,7 +929,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 14; Found: " + e.length) @@ -944,7 +944,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 15; Found: " + e.length) @@ -959,7 +959,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 16; Found: " + e.length) @@ -974,7 +974,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 17; Found: " + e.length) @@ -989,7 +989,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 18; Found: " + e.length) @@ -1004,7 +1004,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 19; Found: " + e.length) @@ -1019,7 +1019,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 20; Found: " + e.length) @@ -1034,7 +1034,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), udfName = Some(name)) } else { throw new AnalysisException("Invalid number of arguments for function " + name + ". Expected: 21; Found: " + e.length) @@ -1049,7 +1049,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, udfName = Some(name)) + ScalaUDF(func, returnType, e, e.map(_ => false), 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/api/r/SQLUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala index af20764f9a968..693be99d47495 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/r/SQLUtils.scala @@ -30,8 +30,9 @@ import org.apache.spark.broadcast.Broadcast import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema +import org.apache.spark.sql.catalyst.expressions.{ExprUtils, GenericRowWithSchema} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.arrow.ArrowConverters import org.apache.spark.sql.execution.command.ShowTablesCommand import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types._ @@ -49,9 +50,17 @@ private[sql] object SQLUtils extends Logging { sparkConfigMap: JMap[Object, Object], enableHiveSupport: Boolean): SparkSession = { val spark = - if (SparkSession.hiveClassesArePresent && enableHiveSupport && + if (enableHiveSupport && jsc.sc.conf.get(CATALOG_IMPLEMENTATION.key, "hive").toLowerCase(Locale.ROOT) == - "hive") { + "hive" && + // Note that the order of conditions here are on purpose. + // `SparkSession.hiveClassesArePresent` checks if Hive's `HiveConf` is loadable or not; + // however, `HiveConf` itself has some static logic to check if Hadoop version is + // supported or not, which throws an `IllegalArgumentException` if unsupported. + // If this is checked first, there's no way to disable Hive support in the case above. + // So, we intentionally check if Hive classes are loadable or not only when + // Hive support is explicitly enabled by short-circuiting. See also SPARK-26422. + SparkSession.hiveClassesArePresent) { SparkSession.builder().sparkContext(withHiveExternalCatalog(jsc.sc)).getOrCreate() } else { if (enableHiveSupport) { @@ -111,7 +120,7 @@ private[sql] object SQLUtils extends Logging { private[this] def doConversion(data: Object, dataType: DataType): Object = { data match { case d: java.lang.Double if dataType == FloatType => - new java.lang.Float(d) + java.lang.Float.valueOf(d.toFloat) // Scala Map is the only allowed external type of map type in Row. case m: java.util.Map[_, _] => m.asScala case _ => data @@ -225,4 +234,29 @@ private[sql] object SQLUtils extends Logging { } sparkSession.sessionState.catalog.listTables(db).map(_.table).toArray } + + def createArrayType(column: Column): ArrayType = { + new ArrayType(ExprUtils.evalTypeExpr(column.expr), true) + } + + /** + * R callable function to read a file in Arrow stream format and create an `RDD` + * using each serialized ArrowRecordBatch as a partition. + */ + def readArrowStreamFromFile( + sparkSession: SparkSession, + filename: String): JavaRDD[Array[Byte]] = { + ArrowConverters.readArrowStreamFromFile(sparkSession.sqlContext, filename) + } + + /** + * R callable function to create a `DataFrame` from a `JavaRDD` of serialized + * ArrowRecordBatches. + */ + def toDataFrame( + arrowBatchRDD: JavaRDD[Array[Byte]], + schema: StructType, + sparkSession: SparkSession): DataFrame = { + ArrowConverters.toDataFrame(arrowBatchRDD, schema.json, sparkSession.sqlContext) + } } 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 ab81725def3f4..44668610d8052 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.{Experimental, InterfaceStability} +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 @@ -29,7 +29,7 @@ import org.apache.spark.storage.StorageLevel * * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable abstract class Catalog { /** @@ -233,7 +233,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable(tableName: String, path: String): DataFrame /** @@ -261,7 +261,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable(tableName: String, path: String, source: String): DataFrame /** @@ -292,7 +292,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -330,7 +330,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -366,7 +366,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, @@ -406,7 +406,7 @@ abstract class Catalog { * @since 2.2.0 */ @Experimental - @InterfaceStability.Evolving + @Evolving def createTable( tableName: String, source: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala index c0c5ebc2ba2d6..cb270875228ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/interface.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.catalog import javax.annotation.Nullable -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.catalyst.DefinedByConstructorParams @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.DefinedByConstructorParams * @param locationUri path (in the form of a uri) to data files. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Database( val name: String, @Nullable val description: String, @@ -61,7 +61,7 @@ class Database( * @param isTemporary whether the table is a temporary table. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Table( val name: String, @Nullable val database: String, @@ -93,7 +93,7 @@ class Table( * @param isBucket whether the column is a bucket column. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Column( val name: String, @Nullable val description: String, @@ -126,7 +126,7 @@ class Column( * @param isTemporary whether the function is a temporary function or not. * @since 2.0.0 */ -@InterfaceStability.Stable +@Stable class Function( val name: String, @Nullable val database: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala index c9929935fb8ac..a13e6ada83dd0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/CacheManager.scala @@ -17,18 +17,19 @@ package org.apache.spark.sql.execution -import java.util.concurrent.locks.ReentrantReadWriteLock - -import scala.collection.JavaConverters._ +import scala.collection.immutable.IndexedSeq import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} -import org.apache.spark.sql.catalyst.expressions.SubqueryExpression -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ResolvedHint} +import org.apache.spark.sql.catalyst.expressions.{Attribute, SubqueryExpression} +import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan, ResolvedHint} import org.apache.spark.sql.execution.columnar.InMemoryRelation -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.command.CommandUtils +import org.apache.spark.sql.execution.datasources.{FileIndex, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StorageLevel.MEMORY_AND_DISK @@ -45,38 +46,22 @@ case class CachedData(plan: LogicalPlan, cachedRepresentation: InMemoryRelation) */ class CacheManager extends Logging { - @transient - private val cachedData = new java.util.LinkedList[CachedData] - - @transient - private val cacheLock = new ReentrantReadWriteLock - - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { - val lock = cacheLock.readLock() - lock.lock() - try f finally { - lock.unlock() - } - } - - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { - val lock = cacheLock.writeLock() - lock.lock() - try f finally { - lock.unlock() - } - } + /** + * Maintains the list of cached plans as an immutable sequence. Any updates to the list + * should be protected in a "this.synchronized" block which includes the reading of the + * existing value and the update of the cachedData var. + */ + @transient @volatile + private var cachedData = IndexedSeq[CachedData]() /** Clears all cached tables. */ - def clearCache(): Unit = writeLock { - cachedData.asScala.foreach(_.cachedRepresentation.cacheBuilder.clearCache()) - cachedData.clear() + def clearCache(): Unit = this.synchronized { + cachedData.foreach(_.cachedRepresentation.cacheBuilder.clearCache()) + cachedData = IndexedSeq[CachedData]() } /** Checks if the cache is empty. */ - def isEmpty: Boolean = readLock { + def isEmpty: Boolean = { cachedData.isEmpty } @@ -88,7 +73,7 @@ class CacheManager extends Logging { def cacheQuery( query: Dataset[_], tableName: Option[String] = None, - storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = writeLock { + storageLevel: StorageLevel = MEMORY_AND_DISK): Unit = { val planToCache = query.logicalPlan if (lookupCachedData(planToCache).nonEmpty) { logWarning("Asked to cache already cached data.") @@ -100,7 +85,13 @@ class CacheManager extends Logging { sparkSession.sessionState.executePlan(planToCache).executedPlan, tableName, planToCache) - cachedData.add(CachedData(planToCache, inMemoryRelation)) + this.synchronized { + if (lookupCachedData(planToCache).nonEmpty) { + logWarning("Data has already been cached.") + } else { + cachedData = CachedData(planToCache, inMemoryRelation) +: cachedData + } + } } } @@ -109,13 +100,11 @@ class CacheManager extends Logging { * @param query The [[Dataset]] to be un-cached. * @param cascade If true, un-cache all the cache entries that refer to the given * [[Dataset]]; otherwise un-cache the given [[Dataset]] only. - * @param blocking Whether to block until all blocks are deleted. */ def uncacheQuery( query: Dataset[_], - cascade: Boolean, - blocking: Boolean = true): Unit = writeLock { - uncacheQuery(query.sparkSession, query.logicalPlan, cascade, blocking) + cascade: Boolean): Unit = { + uncacheQuery(query.sparkSession, query.logicalPlan, cascade) } /** @@ -130,84 +119,112 @@ class CacheManager extends Logging { spark: SparkSession, plan: LogicalPlan, cascade: Boolean, - blocking: Boolean): Unit = writeLock { + blocking: Boolean = false): Unit = { val shouldRemove: LogicalPlan => Boolean = if (cascade) { _.find(_.sameResult(plan)).isDefined } else { _.sameResult(plan) } - val it = cachedData.iterator() - while (it.hasNext) { - val cd = it.next() - if (shouldRemove(cd.plan)) { - cd.cachedRepresentation.cacheBuilder.clearCache(blocking) - it.remove() - } + val plansToUncache = cachedData.filter(cd => shouldRemove(cd.plan)) + this.synchronized { + cachedData = cachedData.filterNot(cd => plansToUncache.exists(_ eq cd)) } + plansToUncache.foreach { _.cachedRepresentation.cacheBuilder.clearCache(blocking) } + // Re-compile dependent cached queries after removing the cached query. if (!cascade) { - recacheByCondition(spark, _.find(_.sameResult(plan)).isDefined, clearCache = false) + recacheByCondition(spark, cd => { + // If the cache buffer has already been loaded, we don't need to recompile the cached plan, + // as it does not rely on the plan that has been uncached anymore, it will just produce + // data from the cache buffer. + // Note that the `CachedRDDBuilder.isCachedColumnBuffersLoaded` call is a non-locking + // status test and may not return the most accurate cache buffer state. So the worse case + // scenario can be: + // 1) The buffer has been loaded, but `isCachedColumnBuffersLoaded` returns false, then we + // will clear the buffer and re-compiled the plan. It is inefficient but doesn't affect + // correctness. + // 2) The buffer has been cleared, but `isCachedColumnBuffersLoaded` returns true, then we + // will keep it as it is. It means the physical plan has been re-compiled already in the + // other thread. + val cacheAlreadyLoaded = cd.cachedRepresentation.cacheBuilder.isCachedColumnBuffersLoaded + cd.plan.find(_.sameResult(plan)).isDefined && !cacheAlreadyLoaded + }) } } + // Analyzes column statistics in the given cache data + private[sql] def analyzeColumnCacheQuery( + sparkSession: SparkSession, + cachedData: CachedData, + column: Seq[Attribute]): Unit = { + val relation = cachedData.cachedRepresentation + val (rowCount, newColStats) = + CommandUtils.computeColumnStats(sparkSession, relation, column) + relation.updateStats(rowCount, newColStats) + } + /** * Tries to re-cache all the cache entries that refer to the given plan. */ - def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit = writeLock { - recacheByCondition(spark, _.find(_.sameResult(plan)).isDefined) + def recacheByPlan(spark: SparkSession, plan: LogicalPlan): Unit = { + recacheByCondition(spark, _.plan.find(_.sameResult(plan)).isDefined) } + /** + * Re-caches all the cache entries that satisfies the given `condition`. + */ private def recacheByCondition( spark: SparkSession, - condition: LogicalPlan => Boolean, - clearCache: Boolean = true): Unit = { - val it = cachedData.iterator() - val needToRecache = scala.collection.mutable.ArrayBuffer.empty[CachedData] - while (it.hasNext) { - val cd = it.next() - if (condition(cd.plan)) { - if (clearCache) { - cd.cachedRepresentation.cacheBuilder.clearCache() + condition: CachedData => Boolean): Unit = { + val needToRecache = cachedData.filter(condition) + this.synchronized { + // Remove the cache entry before creating a new ones. + cachedData = cachedData.filterNot(cd => needToRecache.exists(_ eq cd)) + } + needToRecache.map { cd => + cd.cachedRepresentation.cacheBuilder.clearCache() + val plan = spark.sessionState.executePlan(cd.plan).executedPlan + val newCache = InMemoryRelation( + cacheBuilder = cd.cachedRepresentation.cacheBuilder.copy(cachedPlan = plan), + logicalPlan = cd.plan) + val recomputedPlan = cd.copy(cachedRepresentation = newCache) + this.synchronized { + if (lookupCachedData(recomputedPlan.plan).nonEmpty) { + logWarning("While recaching, data was already added to cache.") + } else { + cachedData = recomputedPlan +: cachedData } - // Remove the cache entry before we create a new one, so that we can have a different - // physical plan. - it.remove() - val plan = spark.sessionState.executePlan(cd.plan).executedPlan - val newCache = InMemoryRelation( - cacheBuilder = cd.cachedRepresentation.cacheBuilder.withCachedPlan(plan), - logicalPlan = cd.plan) - needToRecache += cd.copy(cachedRepresentation = newCache) } } - - needToRecache.foreach(cachedData.add) } /** Optionally returns cached data for the given [[Dataset]] */ - def lookupCachedData(query: Dataset[_]): Option[CachedData] = readLock { + def lookupCachedData(query: Dataset[_]): Option[CachedData] = { lookupCachedData(query.logicalPlan) } /** Optionally returns cached data for the given [[LogicalPlan]]. */ - def lookupCachedData(plan: LogicalPlan): Option[CachedData] = readLock { - cachedData.asScala.find(cd => plan.sameResult(cd.plan)) + def lookupCachedData(plan: LogicalPlan): Option[CachedData] = { + cachedData.find(cd => plan.sameResult(cd.plan)) } /** Replaces segments of the given logical plan with cached versions where possible. */ def useCachedData(plan: LogicalPlan): LogicalPlan = { val newPlan = plan transformDown { - // Do not lookup the cache by hint node. Hint node is special, we should ignore it when - // canonicalizing plans, so that plans which are same except hint can hit the same cache. - // However, we also want to keep the hint info after cache lookup. Here we skip the hint - // node, so that the returned caching plan won't replace the hint node and drop the hint info - // from the original plan. - case hint: ResolvedHint => hint + case command: IgnoreCachedData => command case currentFragment => - lookupCachedData(currentFragment) - .map(_.cachedRepresentation.withOutput(currentFragment.output)) - .getOrElse(currentFragment) + lookupCachedData(currentFragment).map { cached => + // After cache lookup, we should still keep the hints from the input plan. + val hints = EliminateResolvedHint.extractHintsFromPlan(currentFragment)._2 + val cachedPlan = cached.cachedRepresentation.withOutput(currentFragment.output) + // The returned hint list is in top-down order, we should create the hint nodes from + // right to left. + hints.foldRight[LogicalPlan](cachedPlan) { case (hint, p) => + ResolvedHint(p, hint) + } + }.getOrElse(currentFragment) } newPlan transformAllExpressions { @@ -219,14 +236,14 @@ class CacheManager extends Logging { * Tries to re-cache all the cache entries that contain `resourcePath` in one or more * `HadoopFsRelation` node(s) as part of its logical plan. */ - def recacheByPath(spark: SparkSession, resourcePath: String): Unit = writeLock { + def recacheByPath(spark: SparkSession, resourcePath: String): Unit = { val (fs, qualifiedPath) = { val path = new Path(resourcePath) val fs = path.getFileSystem(spark.sessionState.newHadoopConf()) (fs, fs.makeQualified(path)) } - recacheByCondition(spark, _.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined) + recacheByCondition(spark, _.plan.find(lookupAndRefresh(_, fs, qualifiedPath)).isDefined) } /** @@ -239,15 +256,30 @@ class CacheManager extends Logging { plan match { case lr: LogicalRelation => lr.relation match { case hr: HadoopFsRelation => - val prefixToInvalidate = qualifiedPath.toString - val invalidate = hr.location.rootPaths - .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory).toString) - .exists(_.startsWith(prefixToInvalidate)) - if (invalidate) hr.location.refresh() - invalidate + refreshFileIndexIfNecessary(hr.location, fs, qualifiedPath) case _ => false } + + case DataSourceV2Relation(fileTable: FileTable, _, _) => + refreshFileIndexIfNecessary(fileTable.fileIndex, fs, qualifiedPath) + case _ => false } } + + /** + * Refresh the given [[FileIndex]] if any of its root paths starts with `qualifiedPath`. + * @return whether the [[FileIndex]] is refreshed. + */ + private def refreshFileIndexIfNecessary( + fileIndex: FileIndex, + fs: FileSystem, + qualifiedPath: Path): Boolean = { + val prefixToInvalidate = qualifiedPath.toString + val needToRefresh = fileIndex.rootPaths + .map(_.makeQualified(fs.getUri, fs.getWorkingDirectory).toString) + .exists(_.startsWith(prefixToInvalidate)) + if (needToRefresh) fileIndex.refresh() + needToRefresh + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala new file mode 100644 index 0000000000000..4385843d90112 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Columnar.scala @@ -0,0 +1,528 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import scala.collection.JavaConverters._ + +import org.apache.spark.{broadcast, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder, SpecializedGetters, UnsafeProjection} +import org.apache.spark.sql.catalyst.expressions.codegen._ +import org.apache.spark.sql.catalyst.expressions.codegen.Block._ +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} +import org.apache.spark.sql.execution.vectorized.{OffHeapColumnVector, OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} + +/** + * Holds a user defined rule that can be used to inject columnar implementations of various + * operators in the plan. The [[preColumnarTransitions]] [[Rule]] can be used to replace + * [[SparkPlan]] instances with versions that support a columnar implementation. After this + * Spark will insert any transitions necessary. This includes transitions from row to columnar + * [[RowToColumnarExec]] and from columnar to row [[ColumnarToRowExec]]. At this point the + * [[postColumnarTransitions]] [[Rule]] is called to allow replacing any of the implementations + * of the transitions or doing cleanup of the plan, like inserting stages to build larger batches + * for more efficient processing, or stages that transition the data to/from an accelerator's + * memory. + */ +class ColumnarRule { + def preColumnarTransitions: Rule[SparkPlan] = plan => plan + def postColumnarTransitions: Rule[SparkPlan] = plan => plan +} + +/** + * Provides a common executor to translate an [[RDD]] of [[ColumnarBatch]] into an [[RDD]] of + * [[InternalRow]]. This is inserted whenever such a transition is determined to be needed. + * + * The implementation is based off of similar implementations in + * [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] and + * [[MapPartitionsInRWithArrowExec]]. Eventually this should replace those implementations. + */ +case class ColumnarToRowExec(child: SparkPlan) + extends UnaryExecNode with CodegenSupport { + + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), + "numInputBatches" -> SQLMetrics.createMetric(sparkContext, "number of input batches"), + "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time") + ) + + override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + val numInputBatches = longMetric("numInputBatches") + val scanTime = longMetric("scanTime") + // UnsafeProjection is not serializable so do it on the executor side, which is why it is lazy + @transient lazy val outputProject = UnsafeProjection.create(output, output) + val batches = child.executeColumnar() + batches.flatMap(batch => { + val batchStartNs = System.nanoTime() + numInputBatches += 1 + // In order to match the numOutputRows metric in the generated code we update + // numOutputRows for each batch. This is less accurate than doing it at output + // because it will over count the number of rows output in the case of a limit, + // but it is more efficient. + numOutputRows += batch.numRows() + val ret = batch.rowIterator().asScala + scanTime += ((System.nanoTime() - batchStartNs) / (1000 * 1000)) + ret.map(outputProject) + }) + } + + /** + * Generate [[ColumnVector]] expressions for our parent to consume as rows. + * This is called once per [[ColumnVector]] in the batch. + */ + private def genCodeColumnVector( + ctx: CodegenContext, + columnVar: String, + ordinal: String, + dataType: DataType, + nullable: Boolean): ExprCode = { + val javaType = CodeGenerator.javaType(dataType) + val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal) + val isNullVar = if (nullable) { + JavaCode.isNullVariable(ctx.freshName("isNull")) + } else { + FalseLiteral + } + val valueVar = ctx.freshName("value") + val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" + val code = code"${ctx.registerComment(str)}" + (if (nullable) { + code""" + boolean $isNullVar = $columnVar.isNullAt($ordinal); + $javaType $valueVar = $isNullVar ? ${CodeGenerator.defaultValue(dataType)} : ($value); + """ + } else { + code"$javaType $valueVar = $value;" + }) + ExprCode(code, isNullVar, JavaCode.variable(valueVar, dataType)) + } + + /** + * Produce code to process the input iterator as [[ColumnarBatch]]es. + * This produces an [[org.apache.spark.sql.catalyst.expressions.UnsafeRow]] for each row in + * each batch. + */ + override protected def doProduce(ctx: CodegenContext): String = { + // PhysicalRDD always just has one input + val input = ctx.addMutableState("scala.collection.Iterator", "input", + v => s"$v = inputs[0];") + + // metrics + val numOutputRows = metricTerm(ctx, "numOutputRows") + val numInputBatches = metricTerm(ctx, "numInputBatches") + val scanTimeMetric = metricTerm(ctx, "scanTime") + val scanTimeTotalNs = + ctx.addMutableState(CodeGenerator.JAVA_LONG, "scanTime") // init as scanTime = 0 + + val columnarBatchClz = classOf[ColumnarBatch].getName + val batch = ctx.addMutableState(columnarBatchClz, "batch") + + val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init as batchIdx = 0 + val columnVectorClzs = child.vectorTypes.getOrElse( + Seq.fill(output.indices.size)(classOf[ColumnVector].getName)) + val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map { + case (columnVectorClz, i) => + val name = ctx.addMutableState(columnVectorClz, s"colInstance$i") + (name, s"$name = ($columnVectorClz) $batch.column($i);") + }.unzip + + val nextBatch = ctx.freshName("nextBatch") + val nextBatchFuncName = ctx.addNewFunction(nextBatch, + s""" + |private void $nextBatch() throws java.io.IOException { + | long getBatchStart = System.nanoTime(); + | if ($input.hasNext()) { + | $batch = ($columnarBatchClz)$input.next(); + | $numOutputRows.add($batch.numRows()); + | $idx = 0; + | ${columnAssigns.mkString("", "\n", "\n")} + | ${numInputBatches}.add(1); + | } + | $scanTimeTotalNs += System.nanoTime() - getBatchStart; + |}""".stripMargin) + + ctx.currentVars = null + val rowidx = ctx.freshName("rowIdx") + val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => + genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) + } + val localIdx = ctx.freshName("localIdx") + val localEnd = ctx.freshName("localEnd") + val numRows = ctx.freshName("numRows") + val shouldStop = if (parent.needStopCheck) { + s"if (shouldStop()) { $idx = $rowidx + 1; return; }" + } else { + "// shouldStop check is eliminated" + } + s""" + |if ($batch == null) { + | $nextBatchFuncName(); + |} + |while ($batch != null) { + | int $numRows = $batch.numRows(); + | int $localEnd = $numRows - $idx; + | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { + | int $rowidx = $idx + $localIdx; + | ${consume(ctx, columnsBatchInput).trim} + | $shouldStop + | } + | $idx = $numRows; + | $batch = null; + | $nextBatchFuncName(); + |} + |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); + |$scanTimeTotalNs = 0; + """.stripMargin + } + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + child.asInstanceOf[CodegenSupport].inputRDDs() + } +} + +/** + * Provides an optimized set of APIs to append row based data to an array of + * [[WritableColumnVector]]. + */ +private[execution] class RowToColumnConverter(schema: StructType) extends Serializable { + private val converters = schema.fields.map { + f => RowToColumnConverter.getConverterForType(f.dataType, f.nullable) + } + + final def convert(row: InternalRow, vectors: Array[WritableColumnVector]): Unit = { + var idx = 0 + while (idx < row.numFields) { + converters(idx).append(row, idx, vectors(idx)) + idx += 1 + } + } +} + +/** + * Provides an optimized set of APIs to extract a column from a row and append it to a + * [[WritableColumnVector]]. + */ +private object RowToColumnConverter { + private abstract class TypeConverter extends Serializable { + def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit + } + + private final case class BasicNullableTypeConverter(base: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + if (row.isNullAt(column)) { + cv.appendNull + } else { + base.append(row, column, cv) + } + } + } + + private final case class StructNullableTypeConverter(base: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + if (row.isNullAt(column)) { + cv.appendStruct(true) + } else { + base.append(row, column, cv) + } + } + } + + private def getConverterForType(dataType: DataType, nullable: Boolean): TypeConverter = { + val core = dataType match { + case BooleanType => BooleanConverter + case ByteType => ByteConverter + case ShortType => ShortConverter + case IntegerType | DateType => IntConverter + case FloatType => FloatConverter + case LongType | TimestampType => LongConverter + case DoubleType => DoubleConverter + case StringType => StringConverter + case CalendarIntervalType => CalendarConverter + case at: ArrayType => new ArrayConverter(getConverterForType(at.elementType, nullable)) + case st: StructType => new StructConverter(st.fields.map( + (f) => getConverterForType(f.dataType, f.nullable))) + case dt: DecimalType => new DecimalConverter(dt) + case mt: MapType => new MapConverter(getConverterForType(mt.keyType, nullable), + getConverterForType(mt.valueType, nullable)) + case unknown => throw new UnsupportedOperationException( + s"Type $unknown not supported") + } + + if (nullable) { + dataType match { + case CalendarIntervalType => new StructNullableTypeConverter(core) + case st: StructType => new StructNullableTypeConverter(core) + case _ => new BasicNullableTypeConverter(core) + } + } else { + core + } + } + + private object BooleanConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendBoolean(row.getBoolean(column)) + } + + private object ByteConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendByte(row.getByte(column)) + } + + private object ShortConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendShort(row.getShort(column)) + } + + private object IntConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendInt(row.getInt(column)) + } + + private object FloatConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendFloat(row.getFloat(column)) + } + + private object LongConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendLong(row.getLong(column)) + } + + private object DoubleConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = + cv.appendDouble(row.getDouble(column)) + } + + private object StringConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val data = row.getUTF8String(column).getBytes + cv.appendByteArray(data, 0, data.length) + } + } + + private object CalendarConverter extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val c = row.getInterval(column) + cv.appendStruct(false) + cv.getChild(0).appendInt(c.months) + cv.getChild(1).appendLong(c.microseconds) + } + } + + private case class ArrayConverter(childConverter: TypeConverter) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val values = row.getArray(column) + val numElements = values.numElements() + cv.appendArray(numElements) + val arrData = cv.arrayData() + for (i <- 0 until numElements) { + childConverter.append(values, i, arrData) + } + } + } + + private case class StructConverter(childConverters: Array[TypeConverter]) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + cv.appendStruct(false) + val data = row.getStruct(column, childConverters.length) + for (i <- 0 until childConverters.length) { + childConverters(i).append(data, i, cv.getChild(i)) + } + } + } + + private case class DecimalConverter(dt: DecimalType) extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val d = row.getDecimal(column, dt.precision, dt.scale) + if (dt.precision <= Decimal.MAX_INT_DIGITS) { + cv.appendInt(d.toUnscaledLong.toInt) + } else if (dt.precision <= Decimal.MAX_LONG_DIGITS) { + cv.appendLong(d.toUnscaledLong) + } else { + val integer = d.toJavaBigDecimal.unscaledValue + val bytes = integer.toByteArray + cv.appendByteArray(bytes, 0, bytes.length) + } + } + } + + private case class MapConverter(keyConverter: TypeConverter, valueConverter: TypeConverter) + extends TypeConverter { + override def append(row: SpecializedGetters, column: Int, cv: WritableColumnVector): Unit = { + val m = row.getMap(column) + val keys = cv.getChild(0) + val values = cv.getChild(1) + val numElements = m.numElements() + cv.appendArray(numElements) + + val srcKeys = m.keyArray() + val srcValues = m.valueArray() + + for (i <- 0 until numElements) { + keyConverter.append(srcKeys, i, keys) + valueConverter.append(srcValues, i, values) + } + } + } +} + +/** + * Provides a common executor to translate an [[RDD]] of [[InternalRow]] into an [[RDD]] of + * [[ColumnarBatch]]. This is inserted whenever such a transition is determined to be needed. + * + * This is similar to some of the code in ArrowConverters.scala and + * [[org.apache.spark.sql.execution.arrow.ArrowWriter]]. That code is more specialized + * to convert [[InternalRow]] to Arrow formatted data, but in the future if we make + * [[OffHeapColumnVector]] internally Arrow formatted we may be able to replace much of that code. + * + * This is also similar to + * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.populate()]] and + * [[org.apache.spark.sql.execution.vectorized.ColumnVectorUtils.toBatch()]] toBatch is only ever + * called from tests and can probably be removed, but populate is used by both Orc and Parquet + * to initialize partition and missing columns. There is some chance that we could replace + * populate with [[RowToColumnConverter]], but the performance requirements are different and it + * would only be to reduce code. + */ +case class RowToColumnarExec(child: SparkPlan) extends UnaryExecNode { + override def output: Seq[Attribute] = child.output + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + + override def doExecute(): RDD[InternalRow] = { + child.execute() + } + + override def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { + child.doExecuteBroadcast() + } + + override def supportsColumnar: Boolean = true + + override lazy val metrics: Map[String, SQLMetric] = Map( + "numInputRows" -> SQLMetrics.createMetric(sparkContext, "number of input rows"), + "numOutputBatches" -> SQLMetrics.createMetric(sparkContext, "number of output batches") + ) + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val enableOffHeapColumnVector = sqlContext.conf.offHeapColumnVectorEnabled + val numInputRows = longMetric("numInputRows") + val numOutputBatches = longMetric("numOutputBatches") + // Instead of creating a new config we are reusing columnBatchSize. In the future if we do + // combine with some of the Arrow conversion tools we will need to unify some of the configs. + val numRows = conf.columnBatchSize + val converters = new RowToColumnConverter(schema) + val rowBased = child.execute() + rowBased.mapPartitions(rowIterator => { + new Iterator[ColumnarBatch] { + var cb: ColumnarBatch = null + + TaskContext.get().addTaskCompletionListener[Unit] { _ => + if (cb != null) { + cb.close() + cb = null + } + } + + override def hasNext: Boolean = { + rowIterator.hasNext + } + + override def next(): ColumnarBatch = { + if (cb != null) { + cb.close() + cb = null + } + val columnVectors : Array[WritableColumnVector] = + if (enableOffHeapColumnVector) { + OffHeapColumnVector.allocateColumns(numRows, schema).toArray + } else { + OnHeapColumnVector.allocateColumns(numRows, schema).toArray + } + var rowCount = 0 + while (rowCount < numRows && rowIterator.hasNext) { + val row = rowIterator.next() + converters.convert(row, columnVectors) + rowCount += 1 + } + cb = new ColumnarBatch(columnVectors.toArray, rowCount) + numInputRows += rowCount + numOutputBatches += 1 + cb + } + } + }) + } +} + +/** + * Apply any user defined [[ColumnarRule]]s and find the correct place to insert transitions + * to/from columnar formatted data. + */ +case class ApplyColumnarRulesAndInsertTransitions(conf: SQLConf, columnarRules: Seq[ColumnarRule]) + extends Rule[SparkPlan] { + + /** + * Inserts an transition to columnar formatted data. + */ + private def insertRowToColumnar(plan: SparkPlan): SparkPlan = { + if (!plan.supportsColumnar) { + // The tree feels kind of backwards + // Columnar Processing will start here, so transition from row to columnar + RowToColumnarExec(insertTransitions(plan)) + } else { + plan.withNewChildren(plan.children.map(insertRowToColumnar)) + } + } + + /** + * Inserts RowToColumnarExecs and ColumnarToRowExecs where needed. + */ + private def insertTransitions(plan: SparkPlan): SparkPlan = { + if (plan.supportsColumnar) { + // The tree feels kind of backwards + // This is the end of the columnar processing so go back to rows + ColumnarToRowExec(insertRowToColumnar(plan)) + } else { + plan.withNewChildren(plan.children.map(insertTransitions)) + } + } + + def apply(plan: SparkPlan): SparkPlan = { + var preInsertPlan: SparkPlan = plan + columnarRules.foreach((r : ColumnarRule) => + preInsertPlan = r.preColumnarTransitions(preInsertPlan)) + var postInsertPlan = insertTransitions(preInsertPlan) + columnarRules.reverse.foreach((r : ColumnarRule) => + postInsertPlan = r.postColumnarTransitions(postInsertPlan)) + postInsertPlan + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala deleted file mode 100644 index 9f6b593360802..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ColumnarBatchScan.scala +++ /dev/null @@ -1,177 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import org.apache.spark.sql.catalyst.expressions.{BoundReference, UnsafeRow} -import org.apache.spark.sql.catalyst.expressions.codegen._ -import org.apache.spark.sql.catalyst.expressions.codegen.Block._ -import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.DataType -import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} - - -/** - * Helper trait for abstracting scan functionality using [[ColumnarBatch]]es. - */ -private[sql] trait ColumnarBatchScan extends CodegenSupport { - - def vectorTypes: Option[Seq[String]] = None - - protected def supportsBatch: Boolean = true - - protected def needsUnsafeRowConversion: Boolean = true - - override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) - - /** - * Generate [[ColumnVector]] expressions for our parent to consume as rows. - * This is called once per [[ColumnarBatch]]. - */ - private def genCodeColumnVector( - ctx: CodegenContext, - columnVar: String, - ordinal: String, - dataType: DataType, - nullable: Boolean): ExprCode = { - val javaType = CodeGenerator.javaType(dataType) - val value = CodeGenerator.getValueFromVector(columnVar, dataType, ordinal) - val isNullVar = if (nullable) { - JavaCode.isNullVariable(ctx.freshName("isNull")) - } else { - FalseLiteral - } - val valueVar = ctx.freshName("value") - val str = s"columnVector[$columnVar, $ordinal, ${dataType.simpleString}]" - val code = code"${ctx.registerComment(str)}" + (if (nullable) { - code""" - boolean $isNullVar = $columnVar.isNullAt($ordinal); - $javaType $valueVar = $isNullVar ? ${CodeGenerator.defaultValue(dataType)} : ($value); - """ - } else { - code"$javaType $valueVar = $value;" - }) - ExprCode(code, isNullVar, JavaCode.variable(valueVar, dataType)) - } - - /** - * Produce code to process the input iterator as [[ColumnarBatch]]es. - * This produces an [[UnsafeRow]] for each row in each batch. - */ - // TODO: return ColumnarBatch.Rows instead - override protected def doProduce(ctx: CodegenContext): String = { - // PhysicalRDD always just has one input - val input = ctx.addMutableState("scala.collection.Iterator", "input", - v => s"$v = inputs[0];") - if (supportsBatch) { - produceBatches(ctx, input) - } else { - produceRows(ctx, input) - } - } - - private def produceBatches(ctx: CodegenContext, input: String): String = { - // metrics - val numOutputRows = metricTerm(ctx, "numOutputRows") - val scanTimeMetric = metricTerm(ctx, "scanTime") - val scanTimeTotalNs = - ctx.addMutableState(CodeGenerator.JAVA_LONG, "scanTime") // init as scanTime = 0 - - val columnarBatchClz = classOf[ColumnarBatch].getName - val batch = ctx.addMutableState(columnarBatchClz, "batch") - - val idx = ctx.addMutableState(CodeGenerator.JAVA_INT, "batchIdx") // init as batchIdx = 0 - val columnVectorClzs = vectorTypes.getOrElse( - Seq.fill(output.indices.size)(classOf[ColumnVector].getName)) - val (colVars, columnAssigns) = columnVectorClzs.zipWithIndex.map { - case (columnVectorClz, i) => - val name = ctx.addMutableState(columnVectorClz, s"colInstance$i") - (name, s"$name = ($columnVectorClz) $batch.column($i);") - }.unzip - - val nextBatch = ctx.freshName("nextBatch") - val nextBatchFuncName = ctx.addNewFunction(nextBatch, - s""" - |private void $nextBatch() throws java.io.IOException { - | long getBatchStart = System.nanoTime(); - | if ($input.hasNext()) { - | $batch = ($columnarBatchClz)$input.next(); - | $numOutputRows.add($batch.numRows()); - | $idx = 0; - | ${columnAssigns.mkString("", "\n", "\n")} - | } - | $scanTimeTotalNs += System.nanoTime() - getBatchStart; - |}""".stripMargin) - - ctx.currentVars = null - val rowidx = ctx.freshName("rowIdx") - val columnsBatchInput = (output zip colVars).map { case (attr, colVar) => - genCodeColumnVector(ctx, colVar, rowidx, attr.dataType, attr.nullable) - } - val localIdx = ctx.freshName("localIdx") - val localEnd = ctx.freshName("localEnd") - val numRows = ctx.freshName("numRows") - val shouldStop = if (parent.needStopCheck) { - s"if (shouldStop()) { $idx = $rowidx + 1; return; }" - } else { - "// shouldStop check is eliminated" - } - s""" - |if ($batch == null) { - | $nextBatchFuncName(); - |} - |while ($limitNotReachedCond $batch != null) { - | int $numRows = $batch.numRows(); - | int $localEnd = $numRows - $idx; - | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { - | int $rowidx = $idx + $localIdx; - | ${consume(ctx, columnsBatchInput).trim} - | $shouldStop - | } - | $idx = $numRows; - | $batch = null; - | $nextBatchFuncName(); - |} - |$scanTimeMetric.add($scanTimeTotalNs / (1000 * 1000)); - |$scanTimeTotalNs = 0; - """.stripMargin - } - - private def produceRows(ctx: CodegenContext, input: String): String = { - val numOutputRows = metricTerm(ctx, "numOutputRows") - val row = ctx.freshName("row") - - ctx.INPUT_ROW = row - ctx.currentVars = null - // Always provide `outputVars`, so that the framework can help us build unsafe row if the input - // row is not unsafe row, i.e. `needsUnsafeRowConversion` is true. - val outputVars = output.zipWithIndex.map { case (a, i) => - BoundReference(i, a.dataType, a.nullable).genCode(ctx) - } - val inputRow = if (needsUnsafeRowConversion) null else row - s""" - |while ($limitNotReachedCond $input.hasNext()) { - | InternalRow $row = (InternalRow) $input.next(); - | $numOutputRows.add(1); - | ${consume(ctx, outputVars, inputRow).trim} - | if (shouldStop()) return; - |} - """.stripMargin - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/DataSourceScanExec.scala index 738c0666bc3fd..728ac3a466fbf 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 @@ -17,28 +17,31 @@ package org.apache.spark.sql.execution -import scala.collection.mutable.ArrayBuffer +import java.util.concurrent.TimeUnit._ + +import scala.collection.mutable.HashMap import org.apache.commons.lang3.StringUtils -import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} +import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat => ParquetSource} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.sources.{BaseRelation, Filter} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils import org.apache.spark.util.collection.BitSet -trait DataSourceScanExec extends LeafExecNode with CodegenSupport { +trait DataSourceScanExec extends LeafExecNode { val relation: BaseRelation val tableIdentifier: Option[TableIdentifier] @@ -51,19 +54,14 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { // Metadata that describes more details of this scan. protected def metadata: Map[String, String] - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { val metadataEntries = metadata.toSeq.sorted.map { case (key, value) => key + ": " + StringUtils.abbreviate(redact(value), 100) } - val metadataStr = Utils.truncatedString(metadataEntries, " ", ", ", "") - s"$nodeNamePrefix$nodeName${Utils.truncatedString(output, "[", ",", "]")}$metadataStr" - } - - override def verboseString: String = redact(super.verboseString) - - override def treeString(verbose: Boolean, addSuffix: Boolean): String = { - redact(super.treeString(verbose, addSuffix)) + val metadataStr = truncatedString(metadataEntries, " ", ", ", "", maxFields) + redact( + s"$nodeNamePrefix$nodeName${truncatedString(output, "[", ",", "]", maxFields)}$metadataStr") } /** @@ -72,6 +70,12 @@ trait DataSourceScanExec extends LeafExecNode with CodegenSupport { private def redact(text: String): String = { Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, text) } + + /** + * The data being read in. This is to provide input to the tests in a way compatible with + * [[InputRDDCodegen]] which all implementations used to extend. + */ + def inputRDDs(): Seq[RDD[InternalRow]] } /** Physical plan node for scanning data from a relation. */ @@ -83,7 +87,7 @@ case class RowDataSourceScanExec( rdd: RDD[InternalRow], @transient relation: BaseRelation, override val tableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec { + extends DataSourceScanExec with InputRDDCodegen { def output: Seq[Attribute] = requiredColumnsIndex.map(fullOutput) @@ -103,30 +107,10 @@ case class RowDataSourceScanExec( } } - override def inputRDDs(): Seq[RDD[InternalRow]] = { - rdd :: Nil - } + // Input can be InternalRow, has to be turned into UnsafeRows. + override protected val createUnsafeProjection: Boolean = true - override protected def doProduce(ctx: CodegenContext): String = { - val numOutputRows = metricTerm(ctx, "numOutputRows") - // PhysicalRDD always just has one input - val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];") - val exprRows = output.zipWithIndex.map{ case (a, i) => - BoundReference(i, a.dataType, a.nullable) - } - val row = ctx.freshName("row") - ctx.INPUT_ROW = row - ctx.currentVars = null - val columnsRowInput = exprRows.map(_.genCode(ctx)) - s""" - |while ($input.hasNext()) { - | InternalRow $row = (InternalRow) $input.next(); - | $numOutputRows.add(1); - | ${consume(ctx, columnsRowInput).trim} - | if (shouldStop()) return; - |} - """.stripMargin - } + override def inputRDD: RDD[InternalRow] = rdd override val metadata: Map[String, String] = { val markedFilters = for (filter <- filters) yield { @@ -164,14 +148,15 @@ case class FileSourceScanExec( optionalBucketSet: Option[BitSet], dataFilters: Seq[Expression], override val tableIdentifier: Option[TableIdentifier]) - extends DataSourceScanExec with ColumnarBatchScan { + extends DataSourceScanExec { // Note that some vals referring the file-based relation are lazy intentionally // so that this plan can be canonicalized on executor side too. See SPARK-23731. - override lazy val supportsBatch: Boolean = relation.fileFormat.supportBatch( - relation.sparkSession, StructType.fromAttributes(output)) + override lazy val supportsColumnar: Boolean = { + relation.fileFormat.supportBatch(relation.sparkSession, schema) + } - override lazy val needsUnsafeRowConversion: Boolean = { + private lazy val needsUnsafeRowConversion: Boolean = { if (relation.fileFormat.isInstanceOf[ParquetSource]) { SparkSession.getActiveSession.get.sessionState.conf.parquetVectorizedReaderEnabled } else { @@ -185,20 +170,36 @@ case class FileSourceScanExec( partitionSchema = relation.partitionSchema, relation.sparkSession.sessionState.conf) - @transient private lazy val selectedPartitions: Seq[PartitionDirectory] = { - val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) - val startTime = System.nanoTime() - val ret = relation.location.listFiles(partitionFilters, dataFilters) - val timeTakenMs = ((System.nanoTime() - startTime) + optimizerMetadataTimeNs) / 1000 / 1000 - - metrics("numFiles").add(ret.map(_.files.size.toLong).sum) - metrics("metadataTime").add(timeTakenMs) + val driverMetrics: HashMap[String, Long] = HashMap.empty + /** + * Send the driver-side metrics. Before calling this function, selectedPartitions has + * been initialized. See SPARK-26327 for more details. + */ + private def sendDriverMetrics(): Unit = { + driverMetrics.foreach(e => metrics(e._1).add(e._2)) val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, - metrics("numFiles") :: metrics("metadataTime") :: Nil) + metrics.filter(e => driverMetrics.contains(e._1)).values.toSeq) + } + @transient private lazy val selectedPartitions: Array[PartitionDirectory] = { + val optimizerMetadataTimeNs = relation.location.metadataOpsTimeNs.getOrElse(0L) + val startTime = System.nanoTime() + val ret = relation.location.listFiles(partitionFilters, dataFilters) + driverMetrics("numFiles") = ret.map(_.files.size.toLong).sum + val timeTakenMs = NANOSECONDS.toMillis( + (System.nanoTime() - startTime) + optimizerMetadataTimeNs) + driverMetrics("metadataTime") = timeTakenMs ret + }.toArray + + /** + * [[partitionFilters]] can contain subqueries whose results are available only at runtime so + * accessing [[selectedPartitions]] should be guarded by this method during planning + */ + private def hasPartitionsAvailableAtRunTime: Boolean = { + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) } override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = { @@ -237,7 +238,7 @@ case class FileSourceScanExec( val sortColumns = spec.sortColumnNames.map(x => toAttribute(x)).takeWhile(x => x.isDefined).map(_.get) - val sortOrder = if (sortColumns.nonEmpty) { + val sortOrder = if (sortColumns.nonEmpty && !hasPartitionsAvailableAtRunTime) { // 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, @@ -281,17 +282,17 @@ case class FileSourceScanExec( Map( "Format" -> relation.fileFormat.toString, "ReadSchema" -> requiredSchema.catalogString, - "Batched" -> supportsBatch.toString, + "Batched" -> supportsColumnar.toString, "PartitionFilters" -> seqToString(partitionFilters), "PushedFilters" -> seqToString(pushedDownFilters), "DataFilters" -> seqToString(dataFilters), "Location" -> locationDesc) - val withOptPartitionCount = - relation.partitionSchemaOption.map { _ => - metadata + ("PartitionCount" -> selectedPartitions.size.toString) - } getOrElse { - metadata - } + val withOptPartitionCount = if (relation.partitionSchemaOption.isDefined && + !hasPartitionsAvailableAtRunTime) { + metadata + ("PartitionCount" -> selectedPartitions.size.toString) + } else { + metadata + } val withSelectedBucketsCount = relation.bucketSpec.map { spec => val numSelectedBuckets = optionalBucketSet.map { b => @@ -308,7 +309,7 @@ case class FileSourceScanExec( withSelectedBucketsCount } - private lazy val inputRDD: RDD[InternalRow] = { + lazy val inputRDD: RDD[InternalRow] = { val readFile: (PartitionedFile) => Iterator[InternalRow] = relation.fileFormat.buildReaderWithPartitionValues( sparkSession = relation.sparkSession, @@ -319,12 +320,14 @@ case class FileSourceScanExec( options = relation.options, hadoopConf = relation.sparkSession.sessionState.newHadoopConfWithOptions(relation.options)) - relation.bucketSpec match { + val readRDD = relation.bucketSpec match { case Some(bucketing) if relation.sparkSession.sessionState.conf.bucketingEnabled => createBucketedReadRDD(bucketing, readFile, selectedPartitions, relation) case _ => createNonBucketedReadRDD(readFile, selectedPartitions, relation) } + sendDriverMetrics() + readRDD } override def inputRDDs(): Seq[RDD[InternalRow]] = { @@ -333,37 +336,38 @@ case class FileSourceScanExec( override lazy val metrics = Map("numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files"), - "metadataTime" -> SQLMetrics.createMetric(sparkContext, "metadata time (ms)"), + "numFiles" -> SQLMetrics.createMetric(sparkContext, "number of files read"), + "metadataTime" -> SQLMetrics.createTimingMetric(sparkContext, "metadata time"), "scanTime" -> SQLMetrics.createTimingMetric(sparkContext, "scan time")) protected override def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - // in the case of fallback, this batched scan should never fail because of: - // 1) only primitive types are supported - // 2) the number of columns should be smaller than spark.sql.codegen.maxFields - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - val numOutputRows = longMetric("numOutputRows") - - if (needsUnsafeRowConversion) { - inputRDD.mapPartitionsWithIndexInternal { (index, iter) => - val proj = UnsafeProjection.create(schema) - proj.initialize(index) - iter.map( r => { - numOutputRows += 1 - proj(r) - }) - } - } else { - inputRDD.map { r => + val numOutputRows = longMetric("numOutputRows") + + if (needsUnsafeRowConversion) { + inputRDD.mapPartitionsWithIndexInternal { (index, iter) => + val proj = UnsafeProjection.create(schema) + proj.initialize(index) + iter.map( r => { numOutputRows += 1 - r - } + proj(r) + }) + } + } else { + inputRDD.map { r => + numOutputRows += 1 + r } } } + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].map { batch => + numOutputRows += batch.numRows() + batch + } + } + override val nodeNamePrefix: String = "File" /** @@ -381,14 +385,13 @@ case class FileSourceScanExec( private def createBucketedReadRDD( bucketSpec: BucketSpec, readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[PartitionDirectory], + selectedPartitions: Array[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { logInfo(s"Planning with ${bucketSpec.numBuckets} buckets") val filesGroupedToBuckets = selectedPartitions.flatMap { p => p.files.map { f => - val hosts = getBlockHosts(getBlockLocations(f), 0, f.getLen) - PartitionedFile(p.values, f.getPath.toUri.toString, 0, f.getLen, hosts) + PartitionedFileUtil.getPartitionedFile(f, f.getPath, p.values) } }.groupBy { f => BucketingUtils @@ -406,7 +409,7 @@ case class FileSourceScanExec( } val filePartitions = Seq.tabulate(bucketSpec.numBuckets) { bucketId => - FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Nil)) + FilePartition(bucketId, prunedFilesGroupedToBuckets.getOrElse(bucketId, Array.empty)) } new FileScanRDD(fsRelation.sparkSession, readFile, filePartitions) @@ -422,109 +425,37 @@ case class FileSourceScanExec( */ private def createNonBucketedReadRDD( readFile: (PartitionedFile) => Iterator[InternalRow], - selectedPartitions: Seq[PartitionDirectory], + selectedPartitions: Array[PartitionDirectory], fsRelation: HadoopFsRelation): RDD[InternalRow] = { - val defaultMaxSplitBytes = - fsRelation.sparkSession.sessionState.conf.filesMaxPartitionBytes val openCostInBytes = fsRelation.sparkSession.sessionState.conf.filesOpenCostInBytes - val defaultParallelism = fsRelation.sparkSession.sparkContext.defaultParallelism - val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum - val bytesPerCore = totalBytes / defaultParallelism - - val maxSplitBytes = Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + val maxSplitBytes = + FilePartition.maxSplitBytes(fsRelation.sparkSession, selectedPartitions) logInfo(s"Planning scan with bin packing, max size: $maxSplitBytes bytes, " + s"open cost is considered as scanning $openCostInBytes bytes.") val splitFiles = selectedPartitions.flatMap { partition => partition.files.flatMap { file => - val blockLocations = getBlockLocations(file) - if (fsRelation.fileFormat.isSplitable( - fsRelation.sparkSession, fsRelation.options, file.getPath)) { - (0L until file.getLen by maxSplitBytes).map { offset => - val remaining = file.getLen - offset - val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining - val hosts = getBlockHosts(blockLocations, offset, size) - PartitionedFile( - partition.values, file.getPath.toUri.toString, offset, size, hosts) - } - } else { - val hosts = getBlockHosts(blockLocations, 0, file.getLen) - Seq(PartitionedFile( - partition.values, file.getPath.toUri.toString, 0, file.getLen, hosts)) - } + // getPath() is very expensive so we only want to call it once in this block: + val filePath = file.getPath + val isSplitable = relation.fileFormat.isSplitable( + relation.sparkSession, relation.options, filePath) + PartitionedFileUtil.splitFiles( + sparkSession = relation.sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable, + maxSplitBytes = maxSplitBytes, + partitionValues = partition.values + ) } - }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - - val partitions = new ArrayBuffer[FilePartition] - val currentFiles = new ArrayBuffer[PartitionedFile] - var currentSize = 0L - - /** Close the current partition and move to the next. */ - def closePartition(): Unit = { - if (currentFiles.nonEmpty) { - val newPartition = - FilePartition( - partitions.size, - currentFiles.toArray.toSeq) // Copy to a new Array. - partitions += newPartition - } - currentFiles.clear() - currentSize = 0 - } + }.sortBy(_.length)(implicitly[Ordering[Long]].reverse) - // Assign files to partitions using "Next Fit Decreasing" - splitFiles.foreach { file => - if (currentSize + file.length > maxSplitBytes) { - closePartition() - } - // Add the given file to the current partition. - currentSize += file.length + openCostInBytes - currentFiles += file - } - closePartition() + val partitions = + FilePartition.getFilePartitions(relation.sparkSession, splitFiles, maxSplitBytes) new FileScanRDD(fsRelation.sparkSession, readFile, partitions) } - private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { - case f: LocatedFileStatus => f.getBlockLocations - case f => Array.empty[BlockLocation] - } - - // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` - // pair that represents a segment of the same file, find out the block that contains the largest - // fraction the segment, and returns location hosts of that block. If no such block can be found, - // returns an empty array. - private def getBlockHosts( - blockLocations: Array[BlockLocation], offset: Long, length: Long): Array[String] = { - val candidates = blockLocations.map { - // The fragment starts from a position within this block - case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => - b.getHosts -> (b.getOffset + b.getLength - offset).min(length) - - // The fragment ends at a position within this block - case b if offset <= b.getOffset && offset + length < b.getLength => - b.getHosts -> (offset + length - b.getOffset).min(length) - - // The fragment fully contains this block - case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => - b.getHosts -> b.getLength - - // The fragment doesn't intersect with this block - case b => - b.getHosts -> 0L - }.filter { case (hosts, size) => - size > 0L - } - - if (candidates.isEmpty) { - Array.empty[String] - } else { - val (hosts, _) = candidates.maxBy { case (_, size) => size } - hosts - } - } - override def doCanonicalize(): FileSourceScanExec = { FileSourceScanExec( relation, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala index 2962becb64e88..1ab183fe843ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala @@ -18,54 +18,14 @@ package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{Encoder, Row, SparkSession} -import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} +import org.apache.spark.sql.{Encoder, SparkSession} +import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.DataType -import org.apache.spark.util.Utils - -object RDDConversions { - def productToRowRdd[A <: Product](data: RDD[A], outputTypes: Seq[DataType]): RDD[InternalRow] = { - data.mapPartitions { iterator => - val numColumns = outputTypes.length - val mutableRow = new GenericInternalRow(numColumns) - val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) - iterator.map { r => - var i = 0 - while (i < numColumns) { - mutableRow(i) = converters(i)(r.productElement(i)) - i += 1 - } - - mutableRow - } - } - } - - /** - * Convert the objects inside Row into the types Catalyst expected. - */ - def rowToRowRdd(data: RDD[Row], outputTypes: Seq[DataType]): RDD[InternalRow] = { - data.mapPartitions { iterator => - val numColumns = outputTypes.length - val mutableRow = new GenericInternalRow(numColumns) - val converters = outputTypes.map(CatalystTypeConverters.createToCatalystConverter) - iterator.map { r => - var i = 0 - while (i < numColumns) { - mutableRow(i) = converters(i)(r(i)) - i += 1 - } - - mutableRow - } - } - } -} object ExternalRDD { @@ -109,9 +69,8 @@ case class ExternalRDDScanExec[T]( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val outputDataType = outputObjAttr.dataType rdd.mapPartitionsInternal { iter => - val outputObject = ObjectOperator.wrapObjectToRow(outputDataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) iter.map { value => numOutputRows += 1 outputObject(value) @@ -119,7 +78,7 @@ case class ExternalRDDScanExec[T]( } } - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { s"$nodeName${output.mkString("[", ",", "]")}" } } @@ -175,7 +134,7 @@ case class RDDScanExec( rdd: RDD[InternalRow], name: String, override val outputPartitioning: Partitioning = UnknownPartitioning(0), - override val outputOrdering: Seq[SortOrder] = Nil) extends LeafExecNode { + override val outputOrdering: Seq[SortOrder] = Nil) extends LeafExecNode with InputRDDCodegen { private def rddName: String = Option(rdd.name).map(n => s" $n").getOrElse("") @@ -196,7 +155,12 @@ case class RDDScanExec( } } - override def simpleString: String = { - s"$nodeName${Utils.truncatedString(output, "[", ",", "]")}" + override def simpleString(maxFields: Int): String = { + s"$nodeName${truncatedString(output, "[", ",", "]", maxFields)}" } + + // Input can be InternalRow, has to be turned into UnsafeRows. + override protected val createUnsafeProjection: Boolean = true + + override def inputRDD: RDD[InternalRow] = rdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala index 5b4edf5136e3f..0615324b84305 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExpandExec.scala @@ -47,7 +47,8 @@ case class ExpandExec( // as UNKNOWN partitioning override def outputPartitioning: Partitioning = UnknownPartitioning(0) - override def references: AttributeSet = + @transient + override lazy val references: AttributeSet = AttributeSet(projections.flatten.flatMap(_.references)) private[this] val projection = @@ -145,11 +146,12 @@ case class ExpandExec( // Part 1: declare variables for each column // If a column has the same value for all output rows, then we also generate its computation // right after declaration. Otherwise its value is computed in the part 2. + lazy val attributeSeq: AttributeSeq = child.output val outputColumns = output.indices.map { col => val firstExpr = projections.head(col) if (sameOutput(col)) { // This column is the same across all output rows. Just generate code for it here. - BindReferences.bindReference(firstExpr, child.output).genCode(ctx) + BindReferences.bindReference(firstExpr, attributeSeq).genCode(ctx) } else { val isNull = ctx.freshName("isNull") val value = ctx.freshName("value") @@ -170,7 +172,7 @@ case class ExpandExec( var updateCode = "" for (col <- exprs.indices) { if (!sameOutput(col)) { - val ev = BindReferences.bindReference(exprs(col), child.output).genCode(ctx) + val ev = BindReferences.bindReference(exprs(col), attributeSeq).genCode(ctx) updateCode += s""" |${ev.code} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala index 2549b9e1537a0..4c9efdbf2ba62 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/GenerateExec.scala @@ -68,7 +68,7 @@ case class GenerateExec( override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) - override def producedAttributes: AttributeSet = AttributeSet(output) + override def producedAttributes: AttributeSet = AttributeSet(generatorOutput) override def outputPartitioning: Partitioning = child.outputPartitioning 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 new file mode 100644 index 0000000000000..eec8d70b5adf0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/HiveResult.scala @@ -0,0 +1,127 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} + +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.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * Runs a query returning the result in Hive compatible form. + */ +object HiveResult { + /** + * Returns the result as a hive compatible sequence of strings. This is used in tests and + * `SparkSQLDriver` for CLI applications. + */ + 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. + case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => + 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 + val types = executedPlan.output.map(_.dataType) + // Reformat to match hive tab delimited output. + result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) + } + + private def formatDecimal(d: java.math.BigDecimal): String = { + if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { + java.math.BigDecimal.ZERO.toPlainString + } else { + d.stripTrailingZeros().toPlainString // Hive strips trailing zeros + } + } + + private val primitiveTypes = Seq( + StringType, + IntegerType, + LongType, + DoubleType, + FloatType, + BooleanType, + ByteType, + ShortType, + DateType, + TimestampType, + BinaryType) + + private lazy val dateFormatter = DateFormatter() + private lazy val timestampFormatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + + /** Hive outputs fields of structs slightly differently than top level attributes. */ + private def toHiveStructString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "null" + case (s: String, StringType) => "\"" + s + "\"" + case (decimal, DecimalType()) => decimal.toString + case (interval, CalendarIntervalType) => interval.toString + case (other, tpe) if primitiveTypes contains tpe => other.toString + } + + /** Formats a datum (based on the given data type) and returns the string representation. */ + def toHiveString(a: (Any, DataType)): String = a match { + case (struct: Row, StructType(fields)) => + struct.toSeq.zip(fields).map { + case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" + }.mkString("{", ",", "}") + case (seq: Seq[_], ArrayType(typ, _)) => + seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") + case (map: Map[_, _], MapType(kType, vType, _)) => + map.map { + case (key, value) => + toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) + }.toSeq.sorted.mkString("{", ",", "}") + case (null, _) => "NULL" + case (d: Date, DateType) => dateFormatter.format(DateTimeUtils.fromJavaDate(d)) + case (t: Timestamp, TimestampType) => + DateTimeUtils.timestampToString(timestampFormatter, DateTimeUtils.fromJavaTimestamp(t)) + case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) + case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) + case (interval, CalendarIntervalType) => interval.toString + case (other, _ : UserDefinedType[_]) => other.toString + case (other, tpe) if primitiveTypes.contains(tpe) => other.toString + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala index 448eb703eacde..31640db3722ec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/LocalTableScanExec.scala @@ -31,7 +31,7 @@ import org.apache.spark.sql.execution.metric.SQLMetrics */ case class LocalTableScanExec( output: Seq[Attribute], - @transient rows: Seq[InternalRow]) extends LeafExecNode { + @transient rows: Seq[InternalRow]) extends LeafExecNode with InputRDDCodegen { override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) @@ -76,4 +76,12 @@ case class LocalTableScanExec( longMetric("numOutputRows").add(taken.size) taken } + + // Input is already UnsafeRows. + override protected val createUnsafeProjection: Boolean = false + + // Do not codegen when there is no parent - to support the fast driver-local collect/take paths. + override def supportCodegen: Boolean = (parent != null) + + override def inputRDD: RDD[InternalRow] = rdd } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala index 3ca03ab2939aa..45e5f415e8da1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala @@ -72,6 +72,11 @@ case class OptimizeMetadataOnlyQuery(catalog: SessionCatalog) extends Rule[Logic }) } if (isAllDistinctAgg) { + logWarning("Since configuration `spark.sql.optimizer.metadataOnly` is enabled, " + + "Spark will scan partition-level metadata without scanning data files. " + + "This could result in wrong results when the partition metadata exists but the " + + "inclusive data files are empty." + ) a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, rel, filters))) } else { a diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala new file mode 100644 index 0000000000000..3196624f7c7c3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/PartitionedFileUtil.scala @@ -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.execution + +import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources._ + +object PartitionedFileUtil { + def splitFiles( + sparkSession: SparkSession, + file: FileStatus, + filePath: Path, + isSplitable: Boolean, + maxSplitBytes: Long, + partitionValues: InternalRow): Seq[PartitionedFile] = { + if (isSplitable) { + (0L until file.getLen by maxSplitBytes).map { offset => + val remaining = file.getLen - offset + val size = if (remaining > maxSplitBytes) maxSplitBytes else remaining + val hosts = getBlockHosts(getBlockLocations(file), offset, size) + PartitionedFile(partitionValues, filePath.toUri.toString, offset, size, hosts) + } + } else { + Seq(getPartitionedFile(file, filePath, partitionValues)) + } + } + + def getPartitionedFile( + file: FileStatus, + filePath: Path, + partitionValues: InternalRow): PartitionedFile = { + val hosts = getBlockHosts(getBlockLocations(file), 0, file.getLen) + PartitionedFile(partitionValues, filePath.toUri.toString, 0, file.getLen, hosts) + } + + private def getBlockLocations(file: FileStatus): Array[BlockLocation] = file match { + case f: LocatedFileStatus => f.getBlockLocations + case f => Array.empty[BlockLocation] + } + // Given locations of all blocks of a single file, `blockLocations`, and an `(offset, length)` + // pair that represents a segment of the same file, find out the block that contains the largest + // fraction the segment, and returns location hosts of that block. If no such block can be found, + // returns an empty array. + private def getBlockHosts( + blockLocations: Array[BlockLocation], + offset: Long, + length: Long): Array[String] = { + val candidates = blockLocations.map { + // The fragment starts from a position within this block + case b if b.getOffset <= offset && offset < b.getOffset + b.getLength => + b.getHosts -> (b.getOffset + b.getLength - offset).min(length) + + // The fragment ends at a position within this block + case b if offset <= b.getOffset && offset + length < b.getLength => + b.getHosts -> (offset + length - b.getOffset).min(length) + + // The fragment fully contains this block + case b if offset <= b.getOffset && b.getOffset + b.getLength <= offset + length => + b.getHosts -> b.getLength + + // The fragment doesn't intersect with this block + case b => + b.getHosts -> 0L + }.filter { case (hosts, size) => + size > 0L + } + + if (candidates.isEmpty) { + Array.empty[String] + } else { + val (hosts, _) = candidates.maxBy { case (_, size) => size } + hosts + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala index 64f49e2d0d4e6..c8531e9a046a4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecution.scala @@ -17,19 +17,22 @@ package org.apache.spark.sql.execution -import java.nio.charset.StandardCharsets -import java.sql.{Date, Timestamp} +import java.io.{BufferedWriter, OutputStreamWriter} + +import org.apache.hadoop.fs.Path import org.apache.spark.rdd.RDD -import org.apache.spark.sql.{AnalysisException, Row, SparkSession} -import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.{InternalRow, QueryPlanningTracker} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.command.{DescribeTableCommand, ExecutedCommandExec, ShowTablesCommand} +import org.apache.spark.sql.catalyst.util.StringUtils.{PlanStringConcat, StringConcat} +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.adaptive.InsertAdaptiveSparkPlan import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReuseExchange} -import org.apache.spark.sql.types.{BinaryType, DateType, DecimalType, TimestampType, _} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.Utils /** @@ -39,7 +42,10 @@ import org.apache.spark.util.Utils * While this is not a public class, we should avoid changing the function names for the sake of * changing them, because a lot of developers use the feature for debugging. */ -class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { +class QueryExecution( + val sparkSession: SparkSession, + val logical: LogicalPlan, + val tracker: QueryPlanningTracker = new QueryPlanningTracker) { // TODO: Move the planner an optimizer into here from SessionState. protected def planner = sparkSession.sessionState.planner @@ -52,9 +58,9 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { } } - lazy val analyzed: LogicalPlan = { + lazy val analyzed: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.ANALYSIS) { SparkSession.setActiveSession(sparkSession) - sparkSession.sessionState.analyzer.executeAndCheck(logical) + sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } lazy val withCachedData: LogicalPlan = { @@ -63,20 +69,39 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { sparkSession.sharedState.cacheManager.useCachedData(analyzed) } - lazy val optimizedPlan: LogicalPlan = sparkSession.sessionState.optimizer.execute(withCachedData) + lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { + sparkSession.sessionState.optimizer.executeAndTrack(withCachedData, tracker) + } - lazy val sparkPlan: SparkPlan = { + lazy val sparkPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { SparkSession.setActiveSession(sparkSession) + // Runtime re-optimization requires a unique instance of every node in the logical plan. + val logicalPlan = if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { + optimizedPlan.clone() + } else { + optimizedPlan + } // TODO: We use next(), i.e. take the first plan returned by the planner, here for now, // but we will implement to choose the best plan. - planner.plan(ReturnAnswer(optimizedPlan)).next() + planner.plan(ReturnAnswer(logicalPlan)).next() } // executedPlan should not be used to initialize any SparkPlan. It should be // only used for execution. - lazy val executedPlan: SparkPlan = prepareForExecution(sparkPlan) + lazy val executedPlan: SparkPlan = tracker.measurePhase(QueryPlanningTracker.PLANNING) { + prepareForExecution(sparkPlan) + } - /** Internal version of the RDD. Avoids copies and has no schema */ + /** + * Internal version of the RDD. Avoids copies and has no schema. + * Note for callers: Spark may apply various optimization including reusing object: this means + * the row is valid only for the iteration it is retrieved. You should avoid storing row and + * accessing after iteration. (Calling `collect()` is one of known bad usage.) + * If you want to store these rows into collection, please apply some converter or copy row + * which produces new object per iteration. + * Given QueryExecution is not a public class, end users are discouraged to use this: please + * use `Dataset.rdd` instead where conversion will be applied. + */ lazy val toRdd: RDD[InternalRow] = executedPlan.execute() /** @@ -89,135 +114,65 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { /** A sequence of rules that will be applied in order to the physical plan before execution. */ protected def preparations: Seq[Rule[SparkPlan]] = Seq( + // `AdaptiveSparkPlanExec` is a leaf node. If inserted, all the following rules will be no-op + // as the original plan is hidden behind `AdaptiveSparkPlanExec`. + InsertAdaptiveSparkPlan(sparkSession), PlanSubqueries(sparkSession), EnsureRequirements(sparkSession.sessionState.conf), + ApplyColumnarRulesAndInsertTransitions(sparkSession.sessionState.conf, + sparkSession.sessionState.columnarRules), CollapseCodegenStages(sparkSession.sessionState.conf), ReuseExchange(sparkSession.sessionState.conf), ReuseSubquery(sparkSession.sessionState.conf)) - protected def stringOrError[A](f: => A): String = - try f.toString catch { case e: AnalysisException => e.toString } - - - /** - * Returns the result as a hive compatible sequence of strings. This is used in tests and - * `SparkSQLDriver` for CLI applications. - */ - def hiveResultString(): Seq[String] = executedPlan match { - case ExecutedCommandExec(desc: DescribeTableCommand) => - // If it is a describe command for a Hive table, we want to have the output format - // be similar with Hive. - desc.run(sparkSession).map { - case Row(name: String, dataType: String, comment) => - Seq(name, dataType, - Option(comment.asInstanceOf[String]).getOrElse("")) - .map(s => String.format(s"%-20s", s)) - .mkString("\t") - } - // SHOW TABLES in Hive only output table names, while ours output database, table name, isTemp. - case command @ ExecutedCommandExec(s: ShowTablesCommand) if !s.isExtended => - 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 - val types = analyzed.output.map(_.dataType) - // Reformat to match hive tab delimited output. - result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")) + def simpleString: String = withRedaction { + val concat = new PlanStringConcat() + concat.append("== Physical Plan ==\n") + QueryPlan.append(executedPlan, concat.append, verbose = false, addSuffix = false) + concat.append("\n") + concat.toString } - /** Formats a datum (based on the given data type) and returns the string representation. */ - private def toHiveString(a: (Any, DataType)): String = { - val primitiveTypes = Seq(StringType, IntegerType, LongType, DoubleType, FloatType, - BooleanType, ByteType, ShortType, DateType, TimestampType, BinaryType) - - def formatDecimal(d: java.math.BigDecimal): String = { - if (d.compareTo(java.math.BigDecimal.ZERO) == 0) { - java.math.BigDecimal.ZERO.toPlainString - } else { - d.stripTrailingZeros().toPlainString - } - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Date, DateType) => - DateTimeUtils.dateToString(DateTimeUtils.fromJavaDate(d)) - case (t: Timestamp, TimestampType) => - DateTimeUtils.timestampToString(DateTimeUtils.fromJavaTimestamp(t), - DateTimeUtils.getTimeZone(sparkSession.sessionState.conf.sessionLocalTimeZone)) - case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => formatDecimal(decimal) - case (interval, CalendarIntervalType) => interval.toString - case (other, tpe) if primitiveTypes.contains(tpe) => other.toString + private def writePlans(append: String => Unit, maxFields: Int): Unit = { + val (verbose, addSuffix) = (true, false) + append("== Parsed Logical Plan ==\n") + QueryPlan.append(logical, append, verbose, addSuffix, maxFields) + append("\n== Analyzed Logical Plan ==\n") + val analyzedOutput = try { + truncatedString( + analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ", maxFields) + } catch { + case e: AnalysisException => e.toString } - } - - def simpleString: String = withRedaction { - s"""== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = false))} - """.stripMargin.trim + append(analyzedOutput) + append("\n") + QueryPlan.append(analyzed, append, verbose, addSuffix, maxFields) + append("\n== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, append, verbose, addSuffix, maxFields) + append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, append, verbose, addSuffix, maxFields) } override def toString: String = withRedaction { - def output = Utils.truncatedString( - analyzed.output.map(o => s"${o.name}: ${o.dataType.simpleString}"), ", ") - val analyzedPlan = Seq( - stringOrError(output), - stringOrError(analyzed.treeString(verbose = true)) - ).filter(_.nonEmpty).mkString("\n") - - s"""== Parsed Logical Plan == - |${stringOrError(logical.treeString(verbose = true))} - |== Analyzed Logical Plan == - |$analyzedPlan - |== Optimized Logical Plan == - |${stringOrError(optimizedPlan.treeString(verbose = true))} - |== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = true))} - """.stripMargin.trim + val concat = new PlanStringConcat() + writePlans(concat.append, SQLConf.get.maxToStringFields) + concat.toString } def stringWithStats: String = withRedaction { + val concat = new PlanStringConcat() + val maxFields = SQLConf.get.maxToStringFields + // trigger to compute stats for logical plans optimizedPlan.stats // only show optimized logical plan and physical plan - s"""== Optimized Logical Plan == - |${stringOrError(optimizedPlan.treeString(verbose = true, addSuffix = true))} - |== Physical Plan == - |${stringOrError(executedPlan.treeString(verbose = true))} - """.stripMargin.trim + concat.append("== Optimized Logical Plan ==\n") + QueryPlan.append(optimizedPlan, concat.append, verbose = true, addSuffix = true, maxFields) + concat.append("\n== Physical Plan ==\n") + QueryPlan.append(executedPlan, concat.append, verbose = true, addSuffix = false, maxFields) + concat.append("\n") + concat.toString } /** @@ -250,5 +205,26 @@ class QueryExecution(val sparkSession: SparkSession, val logical: LogicalPlan) { def codegenToSeq(): Seq[(String, String)] = { org.apache.spark.sql.execution.debug.codegenStringSeq(executedPlan) } + + /** + * Dumps debug information about query execution into the specified file. + * + * @param maxFields maximum number of fields converted to string representation. + */ + def toFile(path: String, maxFields: Int = Int.MaxValue): Unit = { + val filePath = new Path(path) + val fs = filePath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + val writer = new BufferedWriter(new OutputStreamWriter(fs.create(filePath))) + val append = (s: String) => { + writer.write(s) + } + try { + writePlans(append, maxFields) + writer.write("\n== Whole Stage Codegen ==\n") + org.apache.spark.sql.execution.debug.writeCodegen(writer.write, executedPlan) + } finally { + writer.close() + } + } } } 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 439932b0cc3ac..ca66337846a0d 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 @@ -21,8 +21,11 @@ import java.util.concurrent.ConcurrentHashMap import java.util.concurrent.atomic.AtomicLong import org.apache.spark.SparkContext +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.sql.internal.StaticSQLConf.SQL_EVENT_TRUNCATE_LENGTH +import org.apache.spark.util.Utils object SQLExecution { @@ -38,7 +41,7 @@ object SQLExecution { executionIdToQueryExecution.get(executionId) } - private val testing = sys.props.contains("spark.testing") + private val testing = sys.props.contains(IS_TESTING.key) private[sql] def checkSQLExecutionId(sparkSession: SparkSession): Unit = { val sc = sparkSession.sparkContext @@ -58,7 +61,8 @@ object SQLExecution { */ def withNewExecutionId[T]( sparkSession: SparkSession, - queryExecution: QueryExecution)(body: => T): T = { + queryExecution: QueryExecution, + name: Option[String] = None)(body: => T): T = { val sc = sparkSession.sparkContext val oldExecutionId = sc.getLocalProperty(EXECUTION_ID_KEY) val executionId = SQLExecution.nextExecutionId @@ -70,15 +74,46 @@ object SQLExecution { // streaming queries would give us call site like "run at :0" val callSite = sc.getCallSite() + val truncateLength = sc.conf.get(SQL_EVENT_TRUNCATE_LENGTH) + + val desc = Option(sc.getLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION)) + .filter(_ => truncateLength > 0) + .map { sqlStr => + val redactedStr = Utils + .redact(sparkSession.sessionState.conf.stringRedactionPattern, sqlStr) + redactedStr.substring(0, Math.min(truncateLength, redactedStr.length)) + }.getOrElse(callSite.shortForm) + withSQLConfPropagated(sparkSession) { - sc.listenerBus.post(SparkListenerSQLExecutionStart( - executionId, callSite.shortForm, callSite.longForm, queryExecution.toString, - SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), System.currentTimeMillis())) + var ex: Option[Exception] = None + val startTime = System.nanoTime() try { + sc.listenerBus.post(SparkListenerSQLExecutionStart( + executionId = executionId, + description = desc, + details = callSite.longForm, + physicalPlanDescription = queryExecution.toString, + // `queryExecution.executedPlan` triggers query planning. If it fails, the exception + // will be caught and reported in the `SparkListenerSQLExecutionEnd` + sparkPlanInfo = SparkPlanInfo.fromSparkPlan(queryExecution.executedPlan), + time = System.currentTimeMillis())) body + } catch { + case e: Exception => + ex = Some(e) + throw e } finally { - sc.listenerBus.post(SparkListenerSQLExecutionEnd( - executionId, System.currentTimeMillis())) + val endTime = System.nanoTime() + val event = SparkListenerSQLExecutionEnd(executionId, System.currentTimeMillis()) + // Currently only `Dataset.withAction` and `DataFrameWriter.runCommand` specify the `name` + // parameter. The `ExecutionListenerManager` only watches SQL executions with name. We + // can specify the execution name in more places in the future, so that + // `QueryExecutionListener` can track more cases. + event.executionName = name + event.duration = endTime - startTime + event.qe = queryExecution + event.executionFailure = ex + sc.listenerBus.post(event) } } } finally { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala deleted file mode 100644 index 0e7c593f9fb67..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SelectedField.scala +++ /dev/null @@ -1,134 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types._ - -/** - * A Scala extractor that builds a [[org.apache.spark.sql.types.StructField]] from a Catalyst - * complex type extractor. For example, consider a relation with the following schema: - * - * {{{ - * root - * |-- name: struct (nullable = true) - * | |-- first: string (nullable = true) - * | |-- last: string (nullable = true) - * }}} - * - * Further, suppose we take the select expression `name.first`. This will parse into an - * `Alias(child, "first")`. Ignoring the alias, `child` matches the following pattern: - * - * {{{ - * GetStructFieldObject( - * AttributeReference("name", StructType(_), _, _), - * StructField("first", StringType, _, _)) - * }}} - * - * [[SelectedField]] converts that expression into - * - * {{{ - * StructField("name", StructType(Array(StructField("first", StringType)))) - * }}} - * - * by mapping each complex type extractor to a [[org.apache.spark.sql.types.StructField]] with the - * same name as its child (or "parent" going right to left in the select expression) and a data - * type appropriate to the complex type extractor. In our example, the name of the child expression - * is "name" and its data type is a [[org.apache.spark.sql.types.StructType]] with a single string - * field named "first". - * - * @param expr the top-level complex type extractor - */ -private[execution] object SelectedField { - def unapply(expr: Expression): Option[StructField] = { - // If this expression is an alias, work on its child instead - val unaliased = expr match { - case Alias(child, _) => child - case expr => expr - } - selectField(unaliased, None) - } - - private def selectField(expr: Expression, fieldOpt: Option[StructField]): Option[StructField] = { - expr match { - // No children. Returns a StructField with the attribute name or None if fieldOpt is None. - case AttributeReference(name, dataType, nullable, metadata) => - fieldOpt.map(field => - StructField(name, wrapStructType(dataType, field), nullable, metadata)) - // Handles case "expr0.field[n]", where "expr0" is of struct type and "expr0.field" is of - // array type. - case GetArrayItem(x @ GetStructFieldObject(child, field @ StructField(name, - dataType, nullable, metadata)), _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), nullable, metadata)).getOrElse(field) - selectField(child, Some(childField)) - // Handles case "expr0.field[n]", where "expr0.field" is of array type. - case GetArrayItem(child, _) => - selectField(child, fieldOpt) - // Handles case "expr0.field.subfield", where "expr0" and "expr0.field" are of array type. - case GetArrayStructFields(child: GetArrayStructFields, - field @ StructField(name, dataType, nullable, metadata), _, _, _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field", where "expr0" is of array type. - case GetArrayStructFields(child, - field @ StructField(name, dataType, nullable, metadata), _, _, _) => - val childField = - fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field[key]", where "expr0" is of struct type and "expr0.field" is of - // map type. - case GetMapValue(x @ GetStructFieldObject(child, field @ StructField(name, - dataType, - nullable, metadata)), _) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - // Handles case "expr0.field[key]", where "expr0.field" is of map type. - case GetMapValue(child, _) => - selectField(child, fieldOpt) - // Handles case "expr0.field", where expr0 is of struct type. - case GetStructFieldObject(child, - field @ StructField(name, dataType, nullable, metadata)) => - val childField = fieldOpt.map(field => StructField(name, - wrapStructType(dataType, field), - nullable, metadata)).orElse(Some(field)) - selectField(child, childField) - case _ => - None - } - } - - // Constructs a composition of complex types with a StructType(Array(field)) at its core. Returns - // a StructType for a StructType, an ArrayType for an ArrayType and a MapType for a MapType. - private def wrapStructType(dataType: DataType, field: StructField): DataType = { - dataType match { - case _: StructType => - StructType(Array(field)) - case ArrayType(elementType, containsNull) => - ArrayType(wrapStructType(elementType, field), containsNull) - case MapType(keyType, valueType, valueContainsNull) => - MapType(keyType, wrapStructType(valueType, field), valueContainsNull) - } - } -} 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 862ee05392f37..079ff25fcb67e 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 @@ -22,6 +22,7 @@ import java.util.Arrays import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLShuffleReadMetricsReporter} /** * The [[Partition]] used by [[ShuffledRowRDD]]. A post-shuffle partition @@ -112,6 +113,7 @@ class CoalescedPartitioner(val parent: Partitioner, val partitionStartIndices: A */ class ShuffledRowRDD( var dependency: ShuffleDependency[Int, InternalRow, InternalRow], + metrics: Map[String, SQLMetric], specifiedPartitionStartIndices: Option[Array[Int]] = None) extends RDD[InternalRow](dependency.rdd.context, Nil) { @@ -154,6 +156,10 @@ class ShuffledRowRDD( override def compute(split: Partition, context: TaskContext): Iterator[InternalRow] = { val shuffledRowPartition = split.asInstanceOf[ShuffledRowRDDPartition] + val tempMetrics = context.taskMetrics().createTempShuffleReadMetrics() + // `SQLShuffleReadMetricsReporter` will update its own metrics for SQL exchange operator, + // as well as the `tempMetrics` for basic shuffle metrics. + val sqlMetricsReporter = new SQLShuffleReadMetricsReporter(tempMetrics, metrics) // The range of pre-shuffle partitions that we are fetching at here is // [startPreShufflePartitionIndex, endPreShufflePartitionIndex - 1]. val reader = @@ -161,7 +167,8 @@ class ShuffledRowRDD( dependency.shuffleHandle, shuffledRowPartition.startPreShufflePartitionIndex, shuffledRowPartition.endPreShufflePartitionIndex, - context) + context, + sqlMetricsReporter) reader.read().asInstanceOf[Iterator[Product2[Int, InternalRow]]].map(_._2) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala index f1470e45f1292..0a955d6a75235 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortExec.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util.concurrent.TimeUnit._ + import org.apache.spark.{SparkEnv, TaskContext} import org.apache.spark.executor.TaskMetrics import org.apache.spark.rdd.RDD @@ -24,6 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.metric.SQLMetrics /** @@ -106,7 +109,7 @@ case class SortExec( // figure out how many bytes we spilled for this operator. val spillSizeBefore = metrics.memoryBytesSpilled val sortedIterator = sorter.sort(iter.asInstanceOf[Iterator[UnsafeRow]]) - sortTime += sorter.getSortTimeNanos / 1000000 + sortTime += NANOSECONDS.toMillis(sorter.getSortTimeNanos) peakMemory += sorter.getPeakMemoryUsage spillSize += metrics.memoryBytesSpilled - spillSizeBefore metrics.incPeakExecutionMemory(sorter.getPeakMemoryUsage) @@ -157,7 +160,7 @@ case class SortExec( | long $spillSizeBefore = $metrics.memoryBytesSpilled(); | $addToSorterFuncName(); | $sortedIterator = $sorterVariable.sort(); - | $sortTime.add($sorterVariable.getSortTimeNanos() / 1000000); + | $sortTime.add($sorterVariable.getSortTimeNanos() / $NANOS_PER_MILLIS); | $peakMemory.add($sorterVariable.getPeakMemoryUsage()); | $spillSize.add($metrics.memoryBytesSpilled() - $spillSizeBefore); | $metrics.incPeakExecutionMemory($sorterVariable.getPeakMemoryUsage()); diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala index c6665d273fd27..2bd5cad43a386 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SortPrefixUtils.scala @@ -112,9 +112,7 @@ object SortPrefixUtils { val field = schema.head getPrefixComparator(SortOrder(BoundReference(0, field.dataType, field.nullable), Ascending)) } else { - new PrefixComparator { - override def compare(prefix1: Long, prefix2: Long): Int = 0 - } + (_: Long, _: Long) => 0 } } @@ -164,12 +162,7 @@ object SortPrefixUtils { } } } else { - new UnsafeExternalRowSorter.PrefixComputer { - override def computePrefix(row: InternalRow): - UnsafeExternalRowSorter.PrefixComputer.Prefix = { - emptyPrefix - } - } + _: InternalRow => emptyPrefix } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala index 6c6d344240cea..4ae2194f5b006 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkOptimizer.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.ExperimentalMethods import org.apache.spark.sql.catalyst.catalog.SessionCatalog -import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.optimizer.{ColumnPruning, Optimizer, PushPredicateThroughNonJoin, RemoveNoopOperators} import org.apache.spark.sql.execution.datasources.PruneFileSourcePartitions -import org.apache.spark.sql.execution.datasources.parquet.ParquetSchemaPruning +import org.apache.spark.sql.execution.datasources.SchemaPruning import org.apache.spark.sql.execution.python.{ExtractPythonUDFFromAggregate, ExtractPythonUDFs} class SparkOptimizer( @@ -32,14 +32,21 @@ class SparkOptimizer( override def defaultBatches: Seq[Batch] = (preOptimizationBatches ++ super.defaultBatches :+ Batch("Optimize Metadata Only Query", Once, OptimizeMetadataOnlyQuery(catalog)) :+ Batch("Extract Python UDFs", Once, - Seq(ExtractPythonUDFFromAggregate, ExtractPythonUDFs): _*) :+ + ExtractPythonUDFFromAggregate, + ExtractPythonUDFs, + // The eval-python node may be between Project/Filter and the scan node, which breaks + // column pruning and filter push-down. Here we rerun the related optimizer rules. + ColumnPruning, + PushPredicateThroughNonJoin, + RemoveNoopOperators) :+ Batch("Prune File Source Table Partitions", Once, PruneFileSourcePartitions) :+ - Batch("Parquet Schema Pruning", Once, ParquetSchemaPruning)) ++ + Batch("Schema Pruning", Once, SchemaPruning)) ++ postHocOptimizationBatches :+ Batch("User Provided Optimizers", fixedPoint, experimentalMethods.extraOptimizations: _*) - override def nonExcludableRules: Seq[String] = - super.nonExcludableRules :+ ExtractPythonUDFFromAggregate.ruleName + override def nonExcludableRules: Seq[String] = super.nonExcludableRules :+ + ExtractPythonUDFFromAggregate.ruleName :+ + ExtractPythonUDFs.ruleName /** * Optimization batches that are executed before the regular optimization batches (also before diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 9d9b020309d9f..2baf2e52bf24e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.execution import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataInputStream, DataOutputStream} import scala.collection.mutable.ArrayBuffer -import scala.concurrent.ExecutionContext import org.codehaus.commons.compiler.CompileException import org.codehaus.janino.InternalCompilerException @@ -34,10 +33,20 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{Predicate => GenPredicate, _} import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.trees.TreeNodeTag import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.types.DataType -import org.apache.spark.util.ThreadUtils +import org.apache.spark.sql.vectorized.ColumnarBatch + +object SparkPlan { + /** The original [[LogicalPlan]] from which this [[SparkPlan]] is converted. */ + val LOGICAL_PLAN_TAG = TreeNodeTag[LogicalPlan]("logical_plan") + + /** The [[LogicalPlan]] inherited from its ancestor. */ + val LOGICAL_PLAN_INHERITED_TAG = TreeNodeTag[LogicalPlan]("logical_plan_inherited") +} /** * The base class for physical operators. @@ -65,6 +74,17 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ // whether we should fallback when hitting compilation errors caused by codegen private val codeGenFallBack = (sqlContext == null) || sqlContext.conf.codegenFallback + /** + * Return true if this stage of the plan supports columnar execution. + */ + def supportsColumnar: Boolean = false + + /** + * The exact java types of the columns that are output in columnar processing mode. This + * is a performance optimization for code generation and is optional. + */ + def vectorTypes: Option[Seq[String]] = None + /** Overridden make copy also propagates sqlContext to copied plan. */ override def makeCopy(newArgs: Array[AnyRef]): SparkPlan = { if (sqlContext != null) { @@ -73,6 +93,35 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ super.makeCopy(newArgs) } + /** + * @return The logical plan this plan is linked to. + */ + def logicalLink: Option[LogicalPlan] = + getTagValue(SparkPlan.LOGICAL_PLAN_TAG) + .orElse(getTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG)) + + /** + * Set logical plan link recursively if unset. + */ + def setLogicalLink(logicalPlan: LogicalPlan): Unit = { + setLogicalLink(logicalPlan, false) + } + + private def setLogicalLink(logicalPlan: LogicalPlan, inherited: Boolean = false): Unit = { + // Stop at a descendant which is the root of a sub-tree transformed from another logical node. + if (inherited && getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isDefined) { + return + } + + val tag = if (inherited) { + SparkPlan.LOGICAL_PLAN_INHERITED_TAG + } else { + SparkPlan.LOGICAL_PLAN_TAG + } + setTagValue(tag, logicalPlan) + children.foreach(_.setLogicalLink(logicalPlan, true)) + } + /** * @return All metrics containing metrics of this SparkPlan. */ @@ -144,6 +193,20 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ doExecuteBroadcast() } + /** + * Returns the result of this query as an RDD[ColumnarBatch] by delegating to `doColumnarExecute` + * after preparations. + * + * Concrete implementations of SparkPlan should override `doColumnarExecute` if `supportsColumnar` + * returns true. + */ + final def executeColumnar(): RDD[ColumnarBatch] = executeQuery { + if (isCanonicalizedPlan) { + throw new IllegalStateException("A canonicalized plan is not supposed to be executed.") + } + doExecuteColumnar() + } + /** * Executes a query after preparing the query and adding query plan information to created RDDs * for visualization. @@ -235,6 +298,16 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ throw new UnsupportedOperationException(s"$nodeName does not implement doExecuteBroadcast") } + /** + * Produces the result of the query as an `RDD[ColumnarBatch]` if [[supportsColumnar]] returns + * true. By convention the executor that creates a ColumnarBatch is responsible for closing it + * when it is no longer needed. This allows input formats to be able to reuse batches if needed. + */ + protected def doExecuteColumnar(): RDD[ColumnarBatch] = { + throw new IllegalStateException(s"Internal Error ${this.getClass} has column support" + + s" mismatch:\n${this}") + } + /** * Packing the UnsafeRows into byte array for faster serialization. * The byte arrays are in the following format: @@ -336,12 +409,12 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ return new Array[InternalRow](0) } - val childRDD = getByteArrayRdd(n).map(_._2) + val childRDD = getByteArrayRdd(n) val buf = new ArrayBuffer[InternalRow] val totalParts = childRDD.partitions.length var partsScanned = 0 - while (buf.size < n && partsScanned < totalParts) { + while (buf.length < n && partsScanned < totalParts) { // The number of partitions to try in this iteration. It is ok for this number to be // greater than totalParts because we actually cap it at totalParts in runJob. var numPartsToTry = 1L @@ -353,28 +426,32 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ if (buf.isEmpty) { numPartsToTry = partsScanned * limitScaleUpFactor } else { - val left = n - buf.size + val left = n - buf.length // As left > 0, numPartsToTry is always >= 1 - numPartsToTry = Math.ceil(1.5 * left * partsScanned / buf.size).toInt + numPartsToTry = Math.ceil(1.5 * left * partsScanned / buf.length).toInt numPartsToTry = Math.min(numPartsToTry, partsScanned * limitScaleUpFactor) } } val p = partsScanned.until(math.min(partsScanned + numPartsToTry, totalParts).toInt) val sc = sqlContext.sparkContext - val res = sc.runJob(childRDD, - (it: Iterator[Array[Byte]]) => if (it.hasNext) it.next() else Array.empty[Byte], p) - - buf ++= res.flatMap(decodeUnsafeRows) - + val res = sc.runJob(childRDD, (it: Iterator[(Long, Array[Byte])]) => + if (it.hasNext) it.next() else (0L, Array.empty[Byte]), p) + + var i = 0 + while (buf.length < n && i < res.length) { + val rows = decodeUnsafeRows(res(i)._2) + val rowsToTake = if (n - buf.length >= res(i)._1) { + rows.toArray + } else { + rows.take(n - buf.length).toArray + } + buf ++= rowsToTake + i += 1 + } partsScanned += p.size } - - if (buf.size > n) { - buf.take(n).toArray - } else { - buf.toArray - } + buf.toArray } protected def newMutableProjection( @@ -423,11 +500,6 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging with Serializ } } -object SparkPlan { - private[execution] val subqueryExecutionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) -} - trait LeafExecNode extends SparkPlan { override final def children: Seq[SparkPlan] = Nil override def producedAttributes: AttributeSet = outputSet diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala index 59ffd16381116..8c7752c4bb742 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanInfo.scala @@ -18,8 +18,10 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.sql.execution.adaptive.{AdaptiveSparkPlanExec, QueryStageExec} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.metric.SQLMetricInfo +import org.apache.spark.sql.internal.SQLConf /** * :: DeveloperApi :: @@ -51,6 +53,9 @@ private[execution] object SparkPlanInfo { def fromSparkPlan(plan: SparkPlan): SparkPlanInfo = { val children = plan match { case ReusedExchangeExec(_, child) => child :: Nil + case ReusedSubqueryExec(child) => child :: Nil + case a: AdaptiveSparkPlanExec => a.executedPlan :: Nil + case stage: QueryStageExec => stage.plan :: Nil case _ => plan.children ++ plan.subqueries } val metrics = plan.metrics.toSeq.map { case (key, metric) => @@ -62,7 +67,10 @@ private[execution] object SparkPlanInfo { case fileScan: FileSourceScanExec => fileScan.metadata case _ => Map[String, String]() } - new SparkPlanInfo(plan.nodeName, plan.simpleString, children.map(fromSparkPlan), + new SparkPlanInfo( + plan.nodeName, + plan.simpleString(SQLConf.get.maxToStringFields), + children.map(fromSparkPlan), metadata, metrics) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala index 2a4a1c8ef3438..dc7fb7741e7a7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlanner.scala @@ -21,6 +21,7 @@ import org.apache.spark.SparkContext import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.adaptive.LogicalQueryStageStrategy import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileSourceStrategy} import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Strategy import org.apache.spark.sql.internal.SQLConf @@ -36,6 +37,7 @@ class SparkPlanner( override def strategies: Seq[Strategy] = experimentalMethods.extraStrategies ++ extraPlanningStrategies ++ ( + LogicalQueryStageStrategy :: PythonEvals :: DataSourceV2Strategy :: FileSourceStrategy :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala index a246ed84c34ee..7f69c7fd8a94f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala @@ -282,7 +282,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { throw new ParseException(s"It is not allowed to add database prefix `$database` to " + s"the table name in CACHE TABLE AS SELECT", ctx) } - CacheTableCommand(tableIdent, query, ctx.LAZY != null) + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + CacheTableCommand(tableIdent, query, ctx.LAZY != null, options) } /** @@ -296,7 +297,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Create a [[ClearCacheCommand]] logical plan. */ override def visitClearCache(ctx: ClearCacheContext): LogicalPlan = withOrigin(ctx) { - ClearCacheCommand() + ClearCacheCommand } /** @@ -317,25 +318,15 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { val statement = plan(ctx.statement) if (statement == null) { null // This is enough since ParseException will raise later. - } else if (isExplainableStatement(statement)) { + } else { ExplainCommand( logicalPlan = statement, extended = ctx.EXTENDED != null, codegen = ctx.CODEGEN != null, cost = ctx.COST != null) - } else { - ExplainCommand(OneRowRelation()) } } - /** - * Determine if a plan should be explained at all. - */ - protected def isExplainableStatement(plan: LogicalPlan): Boolean = plan match { - case _: DescribeTableCommand => false - case _ => true - } - /** * Create a [[DescribeColumnCommand]] or [[DescribeTableCommand]] logical commands. */ @@ -369,127 +360,72 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Type to keep track of a table header: (identifier, isTemporary, ifNotExists, isExternal). + * Create a [[DescribeQueryCommand]] logical command. */ - type TableHeader = (TableIdentifier, Boolean, Boolean, Boolean) + override def visitDescribeQuery(ctx: DescribeQueryContext): LogicalPlan = withOrigin(ctx) { + DescribeQueryCommand(source(ctx.query), visitQuery(ctx.query)) + } /** - * Validate a create table statement and return the [[TableIdentifier]]. - */ - override def visitCreateTableHeader( - ctx: CreateTableHeaderContext): TableHeader = withOrigin(ctx) { - val temporary = ctx.TEMPORARY != null - val ifNotExists = ctx.EXISTS != null - if (temporary && ifNotExists) { - operationNotAllowed("CREATE TEMPORARY TABLE ... IF NOT EXISTS", ctx) + * Converts a multi-part identifier to a TableIdentifier. + * + * If the multi-part identifier has too many parts, this will throw a ParseException. + */ + def tableIdentifier( + multipart: Seq[String], + command: String, + ctx: ParserRuleContext): TableIdentifier = { + multipart match { + case Seq(tableName) => + TableIdentifier(tableName) + case Seq(database, tableName) => + TableIdentifier(tableName, Some(database)) + case _ => + operationNotAllowed(s"$command does not support multi-part identifiers", ctx) } - (visitTableIdentifier(ctx.tableIdentifier), temporary, ifNotExists, ctx.EXTERNAL != null) } /** * Create a table, returning a [[CreateTable]] logical plan. * - * Expected format: - * {{{ - * CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name - * USING table_provider - * create_table_clauses - * [[AS] select_statement]; + * This is used to produce CreateTempViewUsing from CREATE TEMPORARY TABLE. * - * create_table_clauses (order insensitive): - * [OPTIONS table_property_list] - * [PARTITIONED BY (col_name, col_name, ...)] - * [CLUSTERED BY (col_name, col_name, ...) - * [SORTED BY (col_name [ASC|DESC], ...)] - * INTO num_buckets BUCKETS - * ] - * [LOCATION path] - * [COMMENT table_comment] - * [TBLPROPERTIES (property_name=property_value, ...)] - * }}} + * TODO: Remove this. It is used because CreateTempViewUsing is not a Catalyst plan. + * Either move CreateTempViewUsing into catalyst as a parsed logical plan, or remove it because + * it is deprecated. */ override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = withOrigin(ctx) { - val (table, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) - if (external) { - operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) - } - - checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) - checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) - checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) - checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) - checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + val (ident, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) - val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) - val provider = ctx.tableProvider.qualifiedName.getText - val schema = Option(ctx.colTypeList()).map(createSchema) - val partitionColumnNames = - Option(ctx.partitionColumnNames) - .map(visitIdentifierList(_).toArray) - .getOrElse(Array.empty[String]) - val properties = Option(ctx.tableProps).map(visitPropertyKeyValues).getOrElse(Map.empty) - val bucketSpec = ctx.bucketSpec().asScala.headOption.map(visitBucketSpec) - - val location = ctx.locationSpec.asScala.headOption.map(visitLocationSpec) - val storage = DataSource.buildStorageFormatFromOptions(options) - - if (location.isDefined && storage.locationUri.isDefined) { - throw new ParseException( - "LOCATION and 'path' in OPTIONS are both used to indicate the custom table path, " + - "you can only specify one of them.", ctx) - } - val customLocation = storage.locationUri.orElse(location.map(CatalogUtils.stringToURI)) - - val tableType = if (customLocation.isDefined) { - CatalogTableType.EXTERNAL + if (!temp || ctx.query != null) { + super.visitCreateTable(ctx) } else { - CatalogTableType.MANAGED - } - - val tableDesc = CatalogTable( - identifier = table, - tableType = tableType, - storage = storage.copy(locationUri = customLocation), - schema = schema.getOrElse(new StructType), - provider = Some(provider), - partitionColumnNames = partitionColumnNames, - bucketSpec = bucketSpec, - properties = properties, - comment = Option(ctx.comment).map(string)) - - // Determine the storage mode. - val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists - - if (ctx.query != null) { - // Get the backing query. - val query = plan(ctx.query) - - if (temp) { - operationNotAllowed("CREATE TEMPORARY TABLE ... USING ... AS query", ctx) + if (external) { + operationNotAllowed("CREATE EXTERNAL TABLE ... USING", ctx) } - // Don't allow explicit specification of schema for CTAS - if (schema.nonEmpty) { - operationNotAllowed( - "Schema may not be specified in a Create Table As Select (CTAS) statement", - ctx) - } - CreateTable(tableDesc, mode, Some(query)) - } else { - if (temp) { - if (ifNotExists) { - operationNotAllowed("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) - } + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) + checkDuplicateClauses(ctx.OPTIONS, "OPTIONS", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED BY", ctx) + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.bucketSpec(), "CLUSTERED BY", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) - logWarning(s"CREATE TEMPORARY TABLE ... USING ... is deprecated, please use " + - "CREATE TEMPORARY VIEW ... USING ... instead") + if (ifNotExists) { // Unlike CREATE TEMPORARY VIEW USING, CREATE TEMPORARY TABLE USING does not support // IF NOT EXISTS. Users are not allowed to replace the existing temp table. - CreateTempViewUsing(table, schema, replace = false, global = false, provider, options) - } else { - CreateTable(tableDesc, mode, None) + operationNotAllowed("CREATE TEMPORARY TABLE IF NOT EXISTS", ctx) } + + val options = Option(ctx.options).map(visitPropertyKeyValues).getOrElse(Map.empty) + val provider = ctx.tableProvider.qualifiedName.getText + val schema = Option(ctx.colTypeList()).map(createSchema) + + logWarning(s"CREATE TEMPORARY TABLE ... USING ... is deprecated, please use " + + "CREATE TEMPORARY VIEW ... USING ... instead") + + val table = tableIdentifier(ident, "CREATE TEMPORARY VIEW", ctx) + CreateTempViewUsing(table, schema, replace = false, global = false, provider, options) } } @@ -554,93 +490,31 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { "MSCK REPAIR TABLE") } - /** - * Convert a table property list into a key-value map. - * This should be called through [[visitPropertyKeyValues]] or [[visitPropertyKeys]]. - */ - override def visitTablePropertyList( - ctx: TablePropertyListContext): Map[String, String] = withOrigin(ctx) { - val properties = ctx.tableProperty.asScala.map { property => - val key = visitTablePropertyKey(property.key) - val value = visitTablePropertyValue(property.value) - key -> value - } - // Check for duplicate property names. - checkDuplicateKeys(properties, ctx) - properties.toMap - } - - /** - * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified. - */ - private def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { - val props = visitTablePropertyList(ctx) - val badKeys = props.collect { case (key, null) => key } - if (badKeys.nonEmpty) { - operationNotAllowed( - s"Values must be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) - } - props - } - - /** - * Parse a list of keys from a [[TablePropertyListContext]], assuming no values are specified. - */ - private def visitPropertyKeys(ctx: TablePropertyListContext): Seq[String] = { - val props = visitTablePropertyList(ctx) - val badKeys = props.filter { case (_, v) => v != null }.keys - if (badKeys.nonEmpty) { - operationNotAllowed( - s"Values should not be specified for key(s): ${badKeys.mkString("[", ",", "]")}", ctx) - } - props.keys.toSeq - } - - /** - * A table property key can either be String or a collection of dot separated elements. This - * function extracts the property key based on whether its a string literal or a table property - * identifier. - */ - override def visitTablePropertyKey(key: TablePropertyKeyContext): String = { - if (key.STRING != null) { - string(key.STRING) - } else { - key.getText - } - } - - /** - * A table property value can be String, Integer, Boolean or Decimal. This function extracts - * the property value based on whether its a string, integer, boolean or decimal literal. - */ - override def visitTablePropertyValue(value: TablePropertyValueContext): String = { - if (value == null) { - null - } else if (value.STRING != null) { - string(value.STRING) - } else if (value.booleanValue != null) { - value.getText.toLowerCase(Locale.ROOT) - } else { - value.getText - } - } - /** * Create a [[CreateDatabaseCommand]] command. * * For example: * {{{ - * CREATE DATABASE [IF NOT EXISTS] database_name [COMMENT database_comment] - * [LOCATION path] [WITH DBPROPERTIES (key1=val1, key2=val2, ...)] + * CREATE DATABASE [IF NOT EXISTS] database_name + * create_database_clauses; + * + * create_database_clauses (order insensitive): + * [COMMENT database_comment] + * [LOCATION path] + * [WITH DBPROPERTIES (key1=val1, key2=val2, ...)] * }}} */ override def visitCreateDatabase(ctx: CreateDatabaseContext): LogicalPlan = withOrigin(ctx) { + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.locationSpec, "LOCATION", ctx) + checkDuplicateClauses(ctx.DBPROPERTIES, "WITH DBPROPERTIES", ctx) + CreateDatabaseCommand( - ctx.identifier.getText, + ctx.db.getText, ctx.EXISTS != null, - Option(ctx.locationSpec).map(visitLocationSpec), + ctx.locationSpec.asScala.headOption.map(visitLocationSpec), Option(ctx.comment).map(string), - Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty)) + ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues).getOrElse(Map.empty)) } /** @@ -654,7 +528,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { override def visitSetDatabaseProperties( ctx: SetDatabasePropertiesContext): LogicalPlan = withOrigin(ctx) { AlterDatabasePropertiesCommand( - ctx.identifier.getText, + ctx.db.getText, visitPropertyKeyValues(ctx.tablePropertyList)) } @@ -667,7 +541,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * }}} */ override def visitDropDatabase(ctx: DropDatabaseContext): LogicalPlan = withOrigin(ctx) { - DropDatabaseCommand(ctx.identifier.getText, ctx.EXISTS != null, ctx.CASCADE != null) + DropDatabaseCommand(ctx.db.getText, ctx.EXISTS != null, ctx.CASCADE != null) } /** @@ -679,7 +553,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * }}} */ override def visitDescribeDatabase(ctx: DescribeDatabaseContext): LogicalPlan = withOrigin(ctx) { - DescribeDatabaseCommand(ctx.identifier.getText, ctx.EXTENDED != null) + DescribeDatabaseCommand(ctx.db.getText, ctx.EXTENDED != null) } /** @@ -771,17 +645,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.TEMPORARY != null) } - /** - * Create a [[DropTableCommand]] command. - */ - override def visitDropTable(ctx: DropTableContext): LogicalPlan = withOrigin(ctx) { - DropTableCommand( - visitTableIdentifier(ctx.tableIdentifier), - ctx.EXISTS != null, - ctx.VIEW != null, - ctx.PURGE != null) - } - /** * Create a [[AlterTableRenameCommand]] command. * @@ -798,57 +661,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { ctx.VIEW != null) } - /** - * Create a [[AlterTableAddColumnsCommand]] command. - * - * For example: - * {{{ - * ALTER TABLE table1 - * ADD COLUMNS (col_name data_type [COMMENT col_comment], ...); - * }}} - */ - override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = withOrigin(ctx) { - AlterTableAddColumnsCommand( - visitTableIdentifier(ctx.tableIdentifier), - visitColTypeList(ctx.columns) - ) - } - - /** - * Create an [[AlterTableSetPropertiesCommand]] command. - * - * For example: - * {{{ - * ALTER TABLE table SET TBLPROPERTIES ('comment' = new_comment); - * ALTER VIEW view SET TBLPROPERTIES ('comment' = new_comment); - * }}} - */ - override def visitSetTableProperties( - ctx: SetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableSetPropertiesCommand( - visitTableIdentifier(ctx.tableIdentifier), - visitPropertyKeyValues(ctx.tablePropertyList), - ctx.VIEW != null) - } - - /** - * Create an [[AlterTableUnsetPropertiesCommand]] command. - * - * For example: - * {{{ - * ALTER TABLE table UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); - * ALTER VIEW view UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); - * }}} - */ - override def visitUnsetTableProperties( - ctx: UnsetTablePropertiesContext): LogicalPlan = withOrigin(ctx) { - AlterTableUnsetPropertiesCommand( - visitTableIdentifier(ctx.tableIdentifier), - visitPropertyKeys(ctx.tablePropertyList), - ctx.EXISTS != null, - ctx.VIEW != null) - } - /** * Create an [[AlterTableSerDePropertiesCommand]] command. * @@ -957,17 +769,18 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { } /** - * Create an [[AlterTableSetLocationCommand]] command + * Create an [[AlterTableSetLocationCommand]] command for a partition. * * For example: * {{{ - * ALTER TABLE table [PARTITION spec] SET LOCATION "loc"; + * ALTER TABLE table PARTITION spec SET LOCATION "loc"; * }}} */ - override def visitSetTableLocation(ctx: SetTableLocationContext): LogicalPlan = withOrigin(ctx) { + override def visitSetPartitionLocation( + ctx: SetPartitionLocationContext): LogicalPlan = withOrigin(ctx) { AlterTableSetLocationCommand( visitTableIdentifier(ctx.tableIdentifier), - Option(ctx.partitionSpec).map(visitNonOptionalPartitionSpec), + Some(visitNonOptionalPartitionSpec(ctx.partitionSpec)), visitLocationSpec(ctx.locationSpec)) } @@ -994,38 +807,10 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { AlterTableChangeColumnCommand( tableName = visitTableIdentifier(ctx.tableIdentifier), - columnName = ctx.identifier.getText, + columnName = ctx.colName.getText, newColumn = visitColType(ctx.colType)) } - /** - * Create location string. - */ - override def visitLocationSpec(ctx: LocationSpecContext): String = withOrigin(ctx) { - string(ctx.STRING) - } - - /** - * Create a [[BucketSpec]]. - */ - override def visitBucketSpec(ctx: BucketSpecContext): BucketSpec = withOrigin(ctx) { - BucketSpec( - ctx.INTEGER_VALUE.getText.toInt, - visitIdentifierList(ctx.identifierList), - Option(ctx.orderedIdentifierList) - .toSeq - .flatMap(_.orderedIdentifier.asScala) - .map { orderedIdCtx => - Option(orderedIdCtx.ordering).map(_.getText).foreach { dir => - if (dir.toLowerCase(Locale.ROOT) != "asc") { - operationNotAllowed(s"Column ordering must be ASC, was '$dir'", ctx) - } - } - - orderedIdCtx.identifier.getText - }) - } - /** * Convert a nested constants list into a sequence of string sequences. */ @@ -1121,7 +906,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * }}} */ override def visitCreateHiveTable(ctx: CreateHiveTableContext): LogicalPlan = withOrigin(ctx) { - val (name, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) + val (ident, temp, ifNotExists, external) = visitCreateTableHeader(ctx.createTableHeader) // TODO: implement temporary tables if (temp) { throw new ParseException( @@ -1179,6 +964,8 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { CatalogTableType.MANAGED } + val name = tableIdentifier(ident, "CREATE TABLE ... STORED AS ...", ctx) + // TODO support the sql text - have a proper location for this! val tableDesc = CatalogTable( identifier = name, @@ -1195,33 +982,40 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { selectQuery match { case Some(q) => - // Hive does not allow to use a CTAS statement to create a partitioned table. - if (tableDesc.partitionColumnNames.nonEmpty) { - val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " + - "create a partitioned table using Hive's file formats. " + - "Please use the syntax of \"CREATE TABLE tableName USING dataSource " + - "OPTIONS (...) PARTITIONED BY ...\" to create a partitioned table through a " + - "CTAS statement." - operationNotAllowed(errorMessage, ctx) - } - // Don't allow explicit specification of schema for CTAS. - if (schema.nonEmpty) { + if (dataCols.nonEmpty) { operationNotAllowed( "Schema may not be specified in a Create Table As Select (CTAS) statement", ctx) } + // When creating partitioned table with CTAS statement, we can't specify data type for the + // partition columns. + if (partitionCols.nonEmpty) { + val errorMessage = "Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table." + operationNotAllowed(errorMessage, ctx) + } + + // Hive CTAS supports dynamic partition by specifying partition column names. + val partitionColumnNames = + Option(ctx.partitionColumnNames) + .map(visitIdentifierList(_).toArray) + .getOrElse(Array.empty[String]) + + val tableDescWithPartitionColNames = + tableDesc.copy(partitionColumnNames = partitionColumnNames) + val hasStorageProperties = (ctx.createFileFormat.size != 0) || (ctx.rowFormat.size != 0) if (conf.convertCTAS && !hasStorageProperties) { // At here, both rowStorage.serdeProperties and fileStorage.serdeProperties // are empty Maps. - val newTableDesc = tableDesc.copy( + val newTableDesc = tableDescWithPartitionColNames.copy( storage = CatalogStorageFormat.empty.copy(locationUri = locUri), provider = Some(conf.defaultDataSourceName)) CreateTable(newTableDesc, mode, Some(q)) } else { - CreateTable(tableDesc, mode, Some(q)) + CreateTable(tableDescWithPartitionColNames, mode, Some(q)) } case None => CreateTable(tableDesc, mode, None) } @@ -1414,49 +1208,49 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * {{{ * CREATE [OR REPLACE] [[GLOBAL] TEMPORARY] VIEW [IF NOT EXISTS] [db_name.]view_name * [(column_name [COMMENT column_comment], ...) ] - * [COMMENT view_comment] - * [TBLPROPERTIES (property_name = property_value, ...)] + * create_view_clauses + * * AS SELECT ...; + * + * create_view_clauses (order insensitive): + * [COMMENT view_comment] + * [TBLPROPERTIES (property_name = property_value, ...)] * }}} */ override def visitCreateView(ctx: CreateViewContext): LogicalPlan = withOrigin(ctx) { - if (ctx.identifierList != null) { + if (!ctx.identifierList.isEmpty) { operationNotAllowed("CREATE VIEW ... PARTITIONED ON", ctx) - } else { - // CREATE VIEW ... AS INSERT INTO is not allowed. - ctx.query.queryNoWith match { - case s: SingleInsertQueryContext if s.insertInto != null => - operationNotAllowed("CREATE VIEW ... AS INSERT INTO", ctx) - case _: MultiInsertQueryContext => - operationNotAllowed("CREATE VIEW ... AS FROM ... [INSERT INTO ...]+", ctx) - case _ => // OK - } + } - val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => - icl.identifierComment.asScala.map { ic => - ic.identifier.getText -> Option(ic.STRING).map(string) - } - } + checkDuplicateClauses(ctx.COMMENT, "COMMENT", ctx) + checkDuplicateClauses(ctx.PARTITIONED, "PARTITIONED ON", ctx) + checkDuplicateClauses(ctx.TBLPROPERTIES, "TBLPROPERTIES", ctx) - val viewType = if (ctx.TEMPORARY == null) { - PersistedView - } else if (ctx.GLOBAL != null) { - GlobalTempView - } else { - LocalTempView + val userSpecifiedColumns = Option(ctx.identifierCommentList).toSeq.flatMap { icl => + icl.identifierComment.asScala.map { ic => + ic.identifier.getText -> Option(ic.STRING).map(string) } + } - CreateViewCommand( - name = visitTableIdentifier(ctx.tableIdentifier), - userSpecifiedColumns = userSpecifiedColumns, - comment = Option(ctx.STRING).map(string), - properties = Option(ctx.tablePropertyList).map(visitPropertyKeyValues).getOrElse(Map.empty), - originalText = Option(source(ctx.query)), - child = plan(ctx.query), - allowExisting = ctx.EXISTS != null, - replace = ctx.REPLACE != null, - viewType = viewType) + val viewType = if (ctx.TEMPORARY == null) { + PersistedView + } else if (ctx.GLOBAL != null) { + GlobalTempView + } else { + LocalTempView } + + CreateViewCommand( + name = visitTableIdentifier(ctx.tableIdentifier), + userSpecifiedColumns = userSpecifiedColumns, + comment = ctx.STRING.asScala.headOption.map(string), + properties = ctx.tablePropertyList.asScala.headOption.map(visitPropertyKeyValues) + .getOrElse(Map.empty), + originalText = Option(source(ctx.query)), + child = plan(ctx.query), + allowExisting = ctx.EXISTS != null, + replace = ctx.REPLACE != null, + viewType = viewType) } /** @@ -1468,14 +1262,6 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * }}} */ override def visitAlterViewQuery(ctx: AlterViewQueryContext): LogicalPlan = withOrigin(ctx) { - // ALTER VIEW ... AS INSERT INTO is not allowed. - ctx.query.queryNoWith match { - case s: SingleInsertQueryContext if s.insertInto != null => - operationNotAllowed("ALTER VIEW ... AS INSERT INTO", ctx) - case _: MultiInsertQueryContext => - operationNotAllowed("ALTER VIEW ... AS FROM ... [INSERT INTO ...]+", ctx) - case _ => // OK - } AlterViewAsCommand( name = visitTableIdentifier(ctx.tableIdentifier), originalText = source(ctx.query), @@ -1486,7 +1272,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder(conf) { * Create a [[ScriptInputOutputSchema]]. */ override protected def withScriptIOSchema( - ctx: QuerySpecificationContext, + ctx: ParserRuleContext, inRowFormat: RowFormatContext, recordWriter: Token, outRowFormat: RowFormatContext, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala index dbc6db62bd820..5500941936442 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -23,18 +23,20 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.catalyst.optimizer.NormalizeFloatingNumbers import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.execution.adaptive.LogicalQueryStage import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide} import org.apache.spark.sql.execution.python._ import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.sources.MemoryPlanV2 +import org.apache.spark.sql.execution.streaming.sources.MemoryPlan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery} import org.apache.spark.sql.types.StructType @@ -57,11 +59,24 @@ case class PlanLater(plan: LogicalPlan) extends LeafExecNode { protected override def doExecute(): RDD[InternalRow] = { throw new UnsupportedOperationException() } + + override def setLogicalLink(logicalPlan: LogicalPlan): Unit = {} } abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SparkPlanner => + override def plan(plan: LogicalPlan): Iterator[SparkPlan] = { + super.plan(plan).map { p => + val logicalPlan = plan match { + case ReturnAnswer(rootPlan) => rootPlan + case _ => plan + } + p.setLogicalLink(logicalPlan) + p + } + } + /** * Plans special cases of limit operators. */ @@ -89,61 +104,36 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } /** - * Select the proper physical plan for join based on joining keys and size of logical plan. - * - * At first, uses the [[ExtractEquiJoinKeys]] pattern to find joins where at least some of the - * predicates can be evaluated by matching join keys. If found, join implementations are chosen - * with the following precedence: + * Select the proper physical plan for join based on join strategy hints, the availability of + * equi-join keys and the sizes of joining relations. Below are the existing join strategies, + * their characteristics and their limitations. * * - Broadcast hash join (BHJ): - * BHJ is not supported for full outer join. For right outer join, we only can broadcast the - * left side. For left outer, left semi, left anti and the internal join type ExistenceJoin, - * we only can broadcast the right side. For inner like join, we can broadcast both sides. - * Normally, BHJ can perform faster than the other join algorithms when the broadcast side is - * small. However, broadcasting tables is a network-intensive operation. It could cause OOM - * or perform worse than the other join algorithms, especially when the build/broadcast side - * is big. - * - * For the supported cases, users can specify the broadcast hint (e.g. the user applied the - * [[org.apache.spark.sql.functions.broadcast()]] function to a DataFrame) and session-based - * [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold to adjust whether BHJ is used and - * which join side is broadcast. - * - * 1) Broadcast the join side with the broadcast hint, even if the size is larger than - * [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]]. If both sides have the hint (only when the type - * is inner like join), the side with a smaller estimated physical size will be broadcast. - * 2) Respect the [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold and broadcast the side - * whose estimated physical size is smaller than the threshold. If both sides are below the - * threshold, broadcast the smaller side. If neither is smaller, BHJ is not used. + * Only supported for equi-joins, while the join keys do not need to be sortable. + * Supported for all join types except full outer joins. + * BHJ usually performs faster than the other join algorithms when the broadcast side is + * small. However, broadcasting tables is a network-intensive operation and it could cause + * OOM or perform badly in some cases, especially when the build/broadcast side is big. * - * - Shuffle hash join: if the average size of a single partition is small enough to build a hash - * table. + * - Shuffle hash join: + * Only supported for equi-joins, while the join keys do not need to be sortable. + * Supported for all join types except full outer joins. * - * - Sort merge: if the matching join keys are sortable. + * - Shuffle sort merge join (SMJ): + * Only supported for equi-joins and the join keys have to be sortable. + * Supported for all join types. * - * If there is no joining keys, Join implementations are chosen with the following precedence: - * - BroadcastNestedLoopJoin (BNLJ): - * BNLJ supports all the join types but the impl is OPTIMIZED for the following scenarios: - * For right outer join, the left side is broadcast. For left outer, left semi, left anti - * and the internal join type ExistenceJoin, the right side is broadcast. For inner like - * joins, either side is broadcast. + * - Broadcast nested loop join (BNLJ): + * Supports both equi-joins and non-equi-joins. + * Supports all the join types, but the implementation is optimized for: + * 1) broadcasting the left side in a right outer join; + * 2) broadcasting the right side in a left outer, left semi, left anti or existence join; + * 3) broadcasting either side in an inner-like join. + * For other cases, we need to scan the data multiple times, which can be rather slow. * - * Like BHJ, users still can specify the broadcast hint and session-based - * [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold to impact which side is broadcast. - * - * 1) Broadcast the join side with the broadcast hint, even if the size is larger than - * [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]]. If both sides have the hint (i.e., just for - * inner-like join), the side with a smaller estimated physical size will be broadcast. - * 2) Respect the [[SQLConf.AUTO_BROADCASTJOIN_THRESHOLD]] threshold and broadcast the side - * whose estimated physical size is smaller than the threshold. If both sides are below the - * threshold, broadcast the smaller side. If neither is smaller, BNLJ is not used. - * - * - CartesianProduct: for inner like join, CartesianProduct is the fallback option. - * - * - BroadcastNestedLoopJoin (BNLJ): - * For the other join types, BNLJ is the fallback option. Here, we just pick the broadcast - * side with the broadcast hint. If neither side has a hint, we broadcast the side with - * the smaller estimated physical size. + * - Shuffle-and-replicate nested loop join (a.k.a. cartesian product join): + * Supports both equi-joins and non-equi-joins. + * Supports only inner like joins. */ object JoinSelection extends Strategy with PredicateHelper { @@ -185,126 +175,218 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case _ => false } - private def broadcastSide( - canBuildLeft: Boolean, - canBuildRight: Boolean, + private def getBuildSide( + wantToBuildLeft: Boolean, + wantToBuildRight: Boolean, left: LogicalPlan, - right: LogicalPlan): BuildSide = { - - def smallerSide = - if (right.stats.sizeInBytes <= left.stats.sizeInBytes) BuildRight else BuildLeft - - if (canBuildRight && canBuildLeft) { - // Broadcast smaller side base on its estimated physical size - // if both sides have broadcast hint - smallerSide - } else if (canBuildRight) { - BuildRight - } else if (canBuildLeft) { - BuildLeft + right: LogicalPlan): Option[BuildSide] = { + if (wantToBuildLeft && wantToBuildRight) { + // returns the smaller side base on its estimated physical size, if we want to build the + // both sides. + Some(getSmallerSide(left, right)) + } else if (wantToBuildLeft) { + Some(BuildLeft) + } else if (wantToBuildRight) { + Some(BuildRight) } else { - // for the last default broadcast nested loop join - smallerSide + None } } - private def canBroadcastByHints(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : Boolean = { - val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast - val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast - buildLeft || buildRight + private def getSmallerSide(left: LogicalPlan, right: LogicalPlan) = { + if (right.stats.sizeInBytes <= left.stats.sizeInBytes) BuildRight else BuildLeft + } + + private def hintToBroadcastLeft(hint: JoinHint): Boolean = { + hint.leftHint.exists(_.strategy.contains(BROADCAST)) + } + + private def hintToBroadcastRight(hint: JoinHint): Boolean = { + hint.rightHint.exists(_.strategy.contains(BROADCAST)) } - private def broadcastSideByHints(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : BuildSide = { - val buildLeft = canBuildLeft(joinType) && left.stats.hints.broadcast - val buildRight = canBuildRight(joinType) && right.stats.hints.broadcast - broadcastSide(buildLeft, buildRight, left, right) + private def hintToShuffleHashLeft(hint: JoinHint): Boolean = { + hint.leftHint.exists(_.strategy.contains(SHUFFLE_HASH)) } - private def canBroadcastBySizes(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : Boolean = { - val buildLeft = canBuildLeft(joinType) && canBroadcast(left) - val buildRight = canBuildRight(joinType) && canBroadcast(right) - buildLeft || buildRight + private def hintToShuffleHashRight(hint: JoinHint): Boolean = { + hint.rightHint.exists(_.strategy.contains(SHUFFLE_HASH)) } - private def broadcastSideBySizes(joinType: JoinType, left: LogicalPlan, right: LogicalPlan) - : BuildSide = { - val buildLeft = canBuildLeft(joinType) && canBroadcast(left) - val buildRight = canBuildRight(joinType) && canBroadcast(right) - broadcastSide(buildLeft, buildRight, left, right) + private def hintToSortMergeJoin(hint: JoinHint): Boolean = { + hint.leftHint.exists(_.strategy.contains(SHUFFLE_MERGE)) || + hint.rightHint.exists(_.strategy.contains(SHUFFLE_MERGE)) + } + + private def hintToShuffleReplicateNL(hint: JoinHint): Boolean = { + hint.leftHint.exists(_.strategy.contains(SHUFFLE_REPLICATE_NL)) || + hint.rightHint.exists(_.strategy.contains(SHUFFLE_REPLICATE_NL)) } def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - // --- BroadcastHashJoin -------------------------------------------------------------------- - - // broadcast hints were specified - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if canBroadcastByHints(joinType, left, right) => - val buildSide = broadcastSideByHints(joinType, left, right) - Seq(joins.BroadcastHashJoinExec( - leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) - - // broadcast hints were not specified, so need to infer it from size and configuration. - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if canBroadcastBySizes(joinType, left, right) => - val buildSide = broadcastSideBySizes(joinType, left, right) - Seq(joins.BroadcastHashJoinExec( - leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) - - // --- ShuffledHashJoin --------------------------------------------------------------------- - - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if !conf.preferSortMergeJoin && canBuildRight(joinType) && canBuildLocalHashMap(right) - && muchSmaller(right, left) || - !RowOrdering.isOrderable(leftKeys) => - Seq(joins.ShuffledHashJoinExec( - leftKeys, rightKeys, joinType, BuildRight, condition, planLater(left), planLater(right))) - - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if !conf.preferSortMergeJoin && canBuildLeft(joinType) && canBuildLocalHashMap(left) - && muchSmaller(left, right) || - !RowOrdering.isOrderable(leftKeys) => - Seq(joins.ShuffledHashJoinExec( - leftKeys, rightKeys, joinType, BuildLeft, condition, planLater(left), planLater(right))) - - // --- SortMergeJoin ------------------------------------------------------------ - - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) - if RowOrdering.isOrderable(leftKeys) => - joins.SortMergeJoinExec( - leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil - - // --- Without joining keys ------------------------------------------------------------ - - // Pick BroadcastNestedLoopJoin if one side could be broadcast - case j @ logical.Join(left, right, joinType, condition) - if canBroadcastByHints(joinType, left, right) => - val buildSide = broadcastSideByHints(joinType, left, right) - joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition) :: Nil - - case j @ logical.Join(left, right, joinType, condition) - if canBroadcastBySizes(joinType, left, right) => - val buildSide = broadcastSideBySizes(joinType, left, right) - joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition) :: Nil - - // Pick CartesianProduct for InnerJoin - case logical.Join(left, right, _: InnerLike, condition) => - joins.CartesianProductExec(planLater(left), planLater(right), condition) :: Nil - - case logical.Join(left, right, joinType, condition) => - val buildSide = broadcastSide( - left.stats.hints.broadcast, right.stats.hints.broadcast, left, right) - // This join could be very slow or OOM - joins.BroadcastNestedLoopJoinExec( - planLater(left), planLater(right), buildSide, joinType, condition) :: Nil + // If it is an equi-join, we first look at the join hints w.r.t. the following order: + // 1. broadcast hint: pick broadcast hash join if the join type is supported. If both sides + // have the broadcast hints, choose the smaller side (based on stats) to broadcast. + // 2. sort merge hint: pick sort merge join if join keys are sortable. + // 3. shuffle hash hint: We pick shuffle hash join if the join type is supported. If both + // sides have the shuffle hash hints, choose the smaller side (based on stats) as the + // build side. + // 4. shuffle replicate NL hint: pick cartesian product if join type is inner like. + // + // If there is no hint or the hints are not applicable, we follow these rules one by one: + // 1. Pick broadcast hash join if one side is small enough to broadcast, and the join type + // is supported. If both sides are small, choose the smaller side (based on stats) + // to broadcast. + // 2. Pick shuffle hash join if one side is small enough to build local hash map, and is + // much smaller than the other side, and `spark.sql.join.preferSortMergeJoin` is false. + // 3. Pick sort merge join if the join keys are sortable. + // 4. Pick cartesian product if join type is inner like. + // 5. Pick broadcast nested loop join as the final solution. It may OOM but we don't have + // other choice. + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) => + def createBroadcastHashJoin(buildLeft: Boolean, buildRight: Boolean) = { + val wantToBuildLeft = canBuildLeft(joinType) && buildLeft + val wantToBuildRight = canBuildRight(joinType) && buildRight + getBuildSide(wantToBuildLeft, wantToBuildRight, left, right).map { buildSide => + Seq(joins.BroadcastHashJoinExec( + leftKeys, + rightKeys, + joinType, + buildSide, + condition, + planLater(left), + planLater(right))) + } + } - // --- Cases where this strategy does not apply --------------------------------------------- + def createShuffleHashJoin(buildLeft: Boolean, buildRight: Boolean) = { + val wantToBuildLeft = canBuildLeft(joinType) && buildLeft + val wantToBuildRight = canBuildRight(joinType) && buildRight + getBuildSide(wantToBuildLeft, wantToBuildRight, left, right).map { buildSide => + Seq(joins.ShuffledHashJoinExec( + leftKeys, + rightKeys, + joinType, + buildSide, + condition, + planLater(left), + planLater(right))) + } + } + + def createSortMergeJoin() = { + if (RowOrdering.isOrderable(leftKeys)) { + Some(Seq(joins.SortMergeJoinExec( + leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)))) + } else { + None + } + } + + def createCartesianProduct() = { + if (joinType.isInstanceOf[InnerLike]) { + Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), condition))) + } else { + None + } + } + + def createJoinWithoutHint() = { + createBroadcastHashJoin(canBroadcast(left), canBroadcast(right)) + .orElse { + if (!conf.preferSortMergeJoin) { + createShuffleHashJoin( + canBuildLocalHashMap(left) && muchSmaller(left, right), + canBuildLocalHashMap(right) && muchSmaller(right, left)) + } else { + None + } + } + .orElse(createSortMergeJoin()) + .orElse(createCartesianProduct()) + .getOrElse { + // This join could be very slow or OOM + val buildSide = getSmallerSide(left, right) + Seq(joins.BroadcastNestedLoopJoinExec( + planLater(left), planLater(right), buildSide, joinType, condition)) + } + } + + createBroadcastHashJoin(hintToBroadcastLeft(hint), hintToBroadcastRight(hint)) + .orElse { if (hintToSortMergeJoin(hint)) createSortMergeJoin() else None } + .orElse(createShuffleHashJoin(hintToShuffleHashLeft(hint), hintToShuffleHashRight(hint))) + .orElse { if (hintToShuffleReplicateNL(hint)) createCartesianProduct() else None } + .getOrElse(createJoinWithoutHint()) + + // If it is not an equi-join, we first look at the join hints w.r.t. the following order: + // 1. broadcast hint: pick broadcast nested loop join. If both sides have the broadcast + // hints, choose the smaller side (based on stats) to broadcast for inner and full joins, + // choose the left side for right join, and choose right side for left join. + // 2. shuffle replicate NL hint: pick cartesian product if join type is inner like. + // + // If there is no hint or the hints are not applicable, we follow these rules one by one: + // 1. Pick broadcast nested loop join if one side is small enough to broadcast. If only left + // side is broadcast-able and it's left join, or only right side is broadcast-able and + // it's right join, we skip this rule. If both sides are small, broadcasts the smaller + // side for inner and full joins, broadcasts the left side for right join, and broadcasts + // right side for left join. + // 2. Pick cartesian product if join type is inner like. + // 3. Pick broadcast nested loop join as the final solution. It may OOM but we don't have + // other choice. It broadcasts the smaller side for inner and full joins, broadcasts the + // left side for right join, and broadcasts right side for left join. + case logical.Join(left, right, joinType, condition, hint) => + val desiredBuildSide = if (joinType.isInstanceOf[InnerLike] || joinType == FullOuter) { + getSmallerSide(left, right) + } else { + // For perf reasons, `BroadcastNestedLoopJoinExec` prefers to broadcast left side if + // it's a right join, and broadcast right side if it's a left join. + // TODO: revisit it. If left side is much smaller than the right side, it may be better + // to broadcast the left side even if it's a left join. + if (canBuildLeft(joinType)) BuildLeft else BuildRight + } + def createBroadcastNLJoin(buildLeft: Boolean, buildRight: Boolean) = { + val maybeBuildSide = if (buildLeft && buildRight) { + Some(desiredBuildSide) + } else if (buildLeft) { + Some(BuildLeft) + } else if (buildRight) { + Some(BuildRight) + } else { + None + } + + maybeBuildSide.map { buildSide => + Seq(joins.BroadcastNestedLoopJoinExec( + planLater(left), planLater(right), buildSide, joinType, condition)) + } + } + + def createCartesianProduct() = { + if (joinType.isInstanceOf[InnerLike]) { + Some(Seq(joins.CartesianProductExec(planLater(left), planLater(right), condition))) + } else { + None + } + } + + def createJoinWithoutHint() = { + createBroadcastNLJoin(canBroadcast(left), canBroadcast(right)) + .orElse(createCartesianProduct()) + .getOrElse { + // This join could be very slow or OOM + Seq(joins.BroadcastNestedLoopJoinExec( + planLater(left), planLater(right), desiredBuildSide, joinType, condition)) + } + } + + createBroadcastNLJoin(hintToBroadcastLeft(hint), hintToBroadcastRight(hint)) + .orElse { if (hintToShuffleReplicateNL(hint)) createCartesianProduct() else None } + .getOrElse(createJoinWithoutHint()) + + + // --- Cases where this strategy does not apply --------------------------------------------- case _ => Nil } } @@ -331,8 +413,17 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val stateVersion = conf.getConf(SQLConf.STREAMING_AGGREGATION_STATE_FORMAT_VERSION) + // Ideally this should be done in `NormalizeFloatingNumbers`, but we do it here because + // `groupingExpressions` is not extracted during logical phase. + val normalizedGroupingExpressions = namedGroupingExpressions.map { e => + NormalizeFloatingNumbers.normalize(e) match { + case n: NamedExpression => n + case other => Alias(other, e.name)(exprId = e.exprId) + } + } + aggregate.AggUtils.planStreamingAggregation( - namedGroupingExpressions, + normalizedGroupingExpressions, aggregateExpressions.map(expr => expr.asInstanceOf[AggregateExpression]), rewrittenResultExpressions, stateVersion, @@ -380,13 +471,13 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { object StreamingJoinStrategy extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = { plan match { - case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right) + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, _) if left.isStreaming && right.isStreaming => new StreamingSymmetricHashJoinExec( leftKeys, rightKeys, joinType, condition, planLater(left), planLater(right)) :: Nil - case Join(left, right, _, _) if left.isStreaming && right.isStreaming => + case Join(left, right, _, _, _) if left.isStreaming && right.isStreaming => throw new AnalysisException( "Stream-stream join without equality predicate is not supported", plan = Some(plan)) @@ -414,16 +505,25 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { "Spark user mailing list.") } + // Ideally this should be done in `NormalizeFloatingNumbers`, but we do it here because + // `groupingExpressions` is not extracted during logical phase. + val normalizedGroupingExpressions = groupingExpressions.map { e => + NormalizeFloatingNumbers.normalize(e) match { + case n: NamedExpression => n + case other => Alias(other, e.name)(exprId = e.exprId) + } + } + val aggregateOperator = if (functionsWithDistinct.isEmpty) { aggregate.AggUtils.planAggregateWithoutDistinct( - groupingExpressions, + normalizedGroupingExpressions, aggregateExpressions, resultExpressions, planLater(child)) } else { aggregate.AggUtils.planAggregateWithOneDistinct( - groupingExpressions, + normalizedGroupingExpressions, functionsWithDistinct, functionsWithoutDistinct, resultExpressions, @@ -523,8 +623,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { */ object PythonEvals extends Strategy { override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case ArrowEvalPython(udfs, output, child) => - ArrowEvalPythonExec(udfs, output, planLater(child)) :: Nil + case ArrowEvalPython(udfs, output, child, evalType) => + ArrowEvalPythonExec(udfs, output, planLater(child), evalType) :: Nil case BatchEvalPython(udfs, output, child) => BatchEvalPythonExec(udfs, output, planLater(child)) :: Nil case _ => @@ -538,9 +638,6 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: RunnableCommand => ExecutedCommandExec(r) :: Nil case MemoryPlan(sink, output) => - val encoder = RowEncoder(sink.schema) - LocalTableScanExec(output, sink.allData.map(r => encoder.toRow(r).copy())) :: Nil - case MemoryPlanV2(sink, output) => val encoder = RowEncoder(StructType.fromAttributes(output)) LocalTableScanExec(output, sink.allData.map(r => encoder.toRow(r).copy())) :: Nil @@ -561,6 +658,9 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { throw new IllegalStateException( "logical except (all) operator should have been replaced by union, aggregate" + " and generate operators in the optimizer") + case logical.ResolvedHint(child, hints) => + throw new IllegalStateException( + "ResolvedHint operator should have been replaced by join hint in the optimizer") case logical.DeserializeToObject(deserializer, objAttr, child) => execution.DeserializeToObjectExec(deserializer, objAttr, planLater(child)) :: Nil @@ -574,8 +674,16 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.FlatMapGroupsInR(f, p, b, is, os, key, value, grouping, data, objAttr, child) => execution.FlatMapGroupsInRExec(f, p, b, is, os, key, value, grouping, data, objAttr, planLater(child)) :: Nil + case logical.FlatMapGroupsInRWithArrow(f, p, b, is, ot, key, grouping, child) => + execution.FlatMapGroupsInRWithArrowExec( + f, p, b, is, ot, key, grouping, planLater(child)) :: Nil + case logical.MapPartitionsInRWithArrow(f, p, b, is, ot, child) => + execution.MapPartitionsInRWithArrowExec( + f, p, b, is, ot, planLater(child)) :: Nil case logical.FlatMapGroupsInPandas(grouping, func, output, child) => execution.python.FlatMapGroupsInPandasExec(grouping, func, output, planLater(child)) :: Nil + case logical.MapInPandas(func, output, child) => + execution.python.MapInPandasExec(func, output, planLater(child)) :: Nil case logical.MapElements(f, _, _, objAttr, child) => execution.MapElementsExec(f, objAttr, planLater(child)) :: Nil case logical.AppendColumns(f, _, _, in, out, child) => @@ -595,7 +703,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case logical.Repartition(numPartitions, shuffle, child) => if (shuffle) { - ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), planLater(child)) :: Nil + ShuffleExchangeExec(RoundRobinPartitioning(numPartitions), + planLater(child), canChangeNumPartitions = false) :: Nil } else { execution.CoalesceExec(numPartitions, planLater(child)) :: Nil } @@ -628,11 +737,11 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case r: logical.Range => execution.RangeExec(r) :: Nil case r: logical.RepartitionByExpression => - exchange.ShuffleExchangeExec(r.partitioning, planLater(r.child)) :: Nil + exchange.ShuffleExchangeExec( + r.partitioning, planLater(r.child), canChangeNumPartitions = false) :: Nil case ExternalRDD(outputObjAttr, rdd) => ExternalRDDScanExec(outputObjAttr, rdd) :: Nil case r: LogicalRDD => RDDScanExec(r.output, r.rdd, "ExistingRDD", r.outputPartitioning, r.outputOrdering) :: Nil - case h: ResolvedHint => planLater(h.child) :: Nil case _ => Nil } } 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 f5aee627fe901..a0afa9a26fd51 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 @@ -17,8 +17,9 @@ package org.apache.spark.sql.execution +import java.io.Writer import java.util.Locale -import java.util.function.Supplier +import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable import scala.util.control.NonFatal @@ -32,10 +33,12 @@ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.aggregate.HashAggregateExec +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.Utils /** @@ -50,6 +53,7 @@ trait CodegenSupport extends SparkPlan { case _: SortMergeJoinExec => "smj" case _: RDDScanExec => "rdd" case _: DataSourceScanExec => "scan" + case _: InMemoryTableScanExec => "memoryScan" case _ => nodeName.toLowerCase(Locale.ROOT) } @@ -86,7 +90,7 @@ trait CodegenSupport extends SparkPlan { this.parent = parent ctx.freshNamePrefix = variablePrefix s""" - |${ctx.registerComment(s"PRODUCE: ${this.simpleString}")} + |${ctx.registerComment(s"PRODUCE: ${this.simpleString(SQLConf.get.maxToStringFields)}")} |${doProduce(ctx)} """.stripMargin } @@ -142,7 +146,7 @@ trait CodegenSupport extends SparkPlan { * Note that `outputVars` and `row` can't both be null. */ final def consume(ctx: CodegenContext, outputVars: Seq[ExprCode], row: String = null): String = { - val inputVars = + val inputVarsCandidate = if (outputVars != null) { assert(outputVars.length == output.length) // outputVars will be used to generate the code for UnsafeRow, so we should copy them @@ -156,6 +160,11 @@ trait CodegenSupport extends SparkPlan { } } + val inputVars = inputVarsCandidate match { + case stream: Stream[ExprCode] => stream.force + case other => other + } + val rowVar = prepareRowVar(ctx, row, outputVars) // Set up the `currentVars` in the codegen context, as we generate the code of `inputVars` @@ -184,7 +193,7 @@ trait CodegenSupport extends SparkPlan { parent.doConsume(ctx, inputVars, rowVar) } s""" - |${ctx.registerComment(s"CONSUME: ${parent.simpleString}")} + |${ctx.registerComment(s"CONSUME: ${parent.simpleString(SQLConf.get.maxToStringFields)}")} |$evaluated |$consumeFunc """.stripMargin @@ -284,6 +293,18 @@ trait CodegenSupport extends SparkPlan { evaluateVars.toString() } + /** + * Returns source code to evaluate the variables for non-deterministic expressions, and clear the + * code of evaluated variables, to prevent them to be evaluated twice. + */ + protected def evaluateNondeterministicVariables( + attributes: Seq[Attribute], + variables: Seq[ExprCode], + expressions: Seq[NamedExpression]): String = { + val nondeterministicAttrs = expressions.filterNot(_.deterministic).map(_.toAttribute) + evaluateRequiredVariables(attributes, variables, AttributeSet(nondeterministicAttrs)) + } + /** * The subset of inputSet those should be evaluated before this plan. * @@ -346,6 +367,15 @@ trait CodegenSupport extends SparkPlan { */ def needStopCheck: Boolean = parent.needStopCheck + /** + * Helper default should stop check code. + */ + def shouldStopCheckCode: String = if (needStopCheck) { + "if (shouldStop()) return;" + } else { + "// shouldStop check is eliminated" + } + /** * A sequence of checks which evaluate to true if the downstream Limit operators have not received * enough records and reached the limit. If current node is a data producing node, it can leverage @@ -355,14 +385,17 @@ trait CodegenSupport extends SparkPlan { */ def limitNotReachedChecks: Seq[String] = parent.limitNotReachedChecks + /** + * Check if the node is supposed to produce limit not reached checks. + */ + protected def canCheckLimitNotReached: Boolean = children.isEmpty + /** * A helper method to generate the data producing loop condition according to the * limit-not-reached checks. */ final def limitNotReachedCond: String = { - // InputAdapter is also a leaf node. - val isLeafNode = children.isEmpty || this.isInstanceOf[InputAdapter] - if (!isLeafNode && !this.isInstanceOf[BlockingOperatorWithCodegen]) { + if (!canCheckLimitNotReached) { val errMsg = "Only leaf nodes and blocking nodes need to call 'limitNotReachedCond' " + "in its data producing loop." if (Utils.isTesting) { @@ -400,16 +433,70 @@ trait BlockingOperatorWithCodegen extends CodegenSupport { // that upstream operators will not generate useless conditions (which are always evaluated to // false) for the Limit operators after this blocking operator. override def limitNotReachedChecks: Seq[String] = Nil + + // This is a blocking node so the node can produce these checks + override protected def canCheckLimitNotReached: Boolean = true } +/** + * Leaf codegen node reading from a single RDD. + */ +trait InputRDDCodegen extends CodegenSupport { + + def inputRDD: RDD[InternalRow] + + // If the input can be InternalRows, an UnsafeProjection needs to be created. + protected val createUnsafeProjection: Boolean + + override def inputRDDs(): Seq[RDD[InternalRow]] = { + inputRDD :: Nil + } + + override def doProduce(ctx: CodegenContext): String = { + // Inline mutable state since an InputRDDCodegen is used once in a task for WholeStageCodegen + val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];", + forceInline = true) + val row = ctx.freshName("row") + + val outputVars = if (createUnsafeProjection) { + // creating the vars will make the parent consume add an unsafe projection. + ctx.INPUT_ROW = row + ctx.currentVars = null + output.zipWithIndex.map { case (a, i) => + BoundReference(i, a.dataType, a.nullable).genCode(ctx) + } + } else { + null + } + + val updateNumOutputRowsMetrics = if (metrics.contains("numOutputRows")) { + val numOutputRows = metricTerm(ctx, "numOutputRows") + s"$numOutputRows.add(1);" + } else { + "" + } + s""" + | while ($limitNotReachedCond $input.hasNext()) { + | InternalRow $row = (InternalRow) $input.next(); + | ${updateNumOutputRowsMetrics} + | ${consume(ctx, outputVars, if (createUnsafeProjection) null else row).trim} + | ${shouldStopCheckCode} + | } + """.stripMargin + } +} /** * InputAdapter is used to hide a SparkPlan from a subtree that supports codegen. * * This is the leaf node of a tree with WholeStageCodegen that is used to generate code * that consumes an RDD iterator of InternalRow. + * + * @param isChildColumnar true if the inputRDD is really columnar data hidden by type erasure, + * false if inputRDD is really an RDD[InternalRow] */ -case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupport { +case class InputAdapter(child: SparkPlan, isChildColumnar: Boolean) + extends UnaryExecNode with InputRDDCodegen { override def output: Seq[Attribute] = child.output @@ -417,6 +504,12 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def vectorTypes: Option[Seq[String]] = child.vectorTypes + + // This is not strictly needed because the codegen transformation happens after the columnar + // transformation but just for consistency + override def supportsColumnar: Boolean = child.supportsColumnar + override def doExecute(): RDD[InternalRow] = { child.execute() } @@ -425,33 +518,40 @@ case class InputAdapter(child: SparkPlan) extends UnaryExecNode with CodegenSupp child.doExecuteBroadcast() } - override def inputRDDs(): Seq[RDD[InternalRow]] = { - child.execute() :: Nil + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + child.executeColumnar() } - override def doProduce(ctx: CodegenContext): String = { - // Right now, InputAdapter is only used when there is one input RDD. - // Inline mutable state since an InputAdapter is used once in a task for WholeStageCodegen - val input = ctx.addMutableState("scala.collection.Iterator", "input", v => s"$v = inputs[0];", - forceInline = true) - val row = ctx.freshName("row") - s""" - | while ($limitNotReachedCond $input.hasNext()) { - | InternalRow $row = (InternalRow) $input.next(); - | ${consume(ctx, null, row).trim} - | if (shouldStop()) return; - | } - """.stripMargin + override def inputRDD: RDD[InternalRow] = { + if (isChildColumnar) { + child.executeColumnar().asInstanceOf[RDD[InternalRow]] // Hack because of type erasure + } else { + child.execute() + } } + // This is a leaf node so the node can produce limit not reached checks. + override protected def canCheckLimitNotReached: Boolean = true + + // InputAdapter does not need UnsafeProjection. + protected val createUnsafeProjection: Boolean = false + override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + append: String => Unit, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, verbose, "") + addSuffix: Boolean = false, + maxFields: Int): Unit = { + child.generateTreeString( + depth, + lastChildren, + append, + verbose, + prefix = "", + addSuffix = false, + maxFields) } override def needCopyResult: Boolean = false @@ -473,58 +573,6 @@ object WholeStageCodegenExec { } } -object WholeStageCodegenId { - // codegenStageId: ID for codegen stages within a query plan. - // It does not affect equality, nor does it participate in destructuring pattern matching - // of WholeStageCodegenExec. - // - // This ID is used to help differentiate between codegen stages. It is included as a part - // of the explain output for physical plans, e.g. - // - // == Physical Plan == - // *(5) SortMergeJoin [x#3L], [y#9L], Inner - // :- *(2) Sort [x#3L ASC NULLS FIRST], false, 0 - // : +- Exchange hashpartitioning(x#3L, 200) - // : +- *(1) Project [(id#0L % 2) AS x#3L] - // : +- *(1) Filter isnotnull((id#0L % 2)) - // : +- *(1) Range (0, 5, step=1, splits=8) - // +- *(4) Sort [y#9L ASC NULLS FIRST], false, 0 - // +- Exchange hashpartitioning(y#9L, 200) - // +- *(3) Project [(id#6L % 2) AS y#9L] - // +- *(3) Filter isnotnull((id#6L % 2)) - // +- *(3) Range (0, 5, step=1, splits=8) - // - // where the ID makes it obvious that not all adjacent codegen'd plan operators are of the - // same codegen stage. - // - // The codegen stage ID is also optionally included in the name of the generated classes as - // a suffix, so that it's easier to associate a generated class back to the physical operator. - // This is controlled by SQLConf: spark.sql.codegen.useIdInClassName - // - // The ID is also included in various log messages. - // - // Within a query, a codegen stage in a plan starts counting from 1, in "insertion order". - // WholeStageCodegenExec operators are inserted into a plan in depth-first post-order. - // See CollapseCodegenStages.insertWholeStageCodegen for the definition of insertion order. - // - // 0 is reserved as a special ID value to indicate a temporary WholeStageCodegenExec object - // is created, e.g. for special fallback handling when an existing WholeStageCodegenExec - // failed to generate/compile code. - - private val codegenStageCounter = ThreadLocal.withInitial(new Supplier[Integer] { - override def get() = 1 // TODO: change to Scala lambda syntax when upgraded to Scala 2.12+ - }) - - def resetPerQuery(): Unit = codegenStageCounter.set(1) - - def getNextStageId(): Int = { - val counter = codegenStageCounter - val id = counter.get() - counter.set(id + 1) - id - } -} - /** * WholeStageCodegen compiles a subtree of plans that support codegen together into single Java * function. @@ -562,6 +610,10 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def outputOrdering: Seq[SortOrder] = child.outputOrdering + // This is not strictly needed because the codegen transformation happens after the columnar + // transformation but just for consistency + override def supportsColumnar: Boolean = child.supportsColumnar + override lazy val metrics = Map( "pipelineTime" -> SQLMetrics.createTimingMetric(sparkContext, WholeStageCodegenExec.PIPELINE_DURATION_METRIC)) @@ -632,6 +684,12 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) (ctx, cleanedSource) } + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + // Code generation is not currently supported for columnar output, so just fall back to + // the interpreted path + child.executeColumnar() + } + override def doExecute(): RDD[InternalRow] = { val (ctx, cleanedSource) = doCodeGen() // try to compile and fallback if it failed @@ -651,17 +709,16 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) s"${sqlContext.conf.hugeMethodLimit}, and the whole-stage codegen was disabled " + s"for this plan (id=$codegenStageId). To avoid this, you can raise the limit " + s"`${SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key}`:\n$treeString") - child match { - // The fallback solution of batch file source scan still uses WholeStageCodegenExec - case f: FileSourceScanExec if f.supportsBatch => // do nothing - case _ => return child.execute() - } + return child.execute() } val references = ctx.references.toArray val durationMs = longMetric("pipelineTime") + // Even though rdds is an RDD[InternalRow] it may actually be an RDD[ColumnarBatch] with + // type erasure hiding that. This allows for the input to a code gen stage to be columnar, + // but the output must be rows. val rdds = child.asInstanceOf[CodegenSupport].inputRDDs() assert(rdds.size <= 2, "Up to two input RDDs can be supported") if (rdds.length == 1) { @@ -723,11 +780,19 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) override def generateTreeString( depth: Int, lastChildren: Seq[Boolean], - builder: StringBuilder, + append: String => Unit, verbose: Boolean, prefix: String = "", - addSuffix: Boolean = false): StringBuilder = { - child.generateTreeString(depth, lastChildren, builder, verbose, s"*($codegenStageId) ") + addSuffix: Boolean = false, + maxFields: Int): Unit = { + child.generateTreeString( + depth, + lastChildren, + append, + verbose, + s"*($codegenStageId) ", + false, + maxFields) } override def needStopCheck: Boolean = true @@ -740,8 +805,48 @@ case class WholeStageCodegenExec(child: SparkPlan)(val codegenStageId: Int) /** * Find the chained plans that support codegen, collapse them together as WholeStageCodegen. + * + * The `codegenStageCounter` generates ID for codegen stages within a query plan. + * It does not affect equality, nor does it participate in destructuring pattern matching + * of WholeStageCodegenExec. + * + * This ID is used to help differentiate between codegen stages. It is included as a part + * of the explain output for physical plans, e.g. + * + * == Physical Plan == + * *(5) SortMergeJoin [x#3L], [y#9L], Inner + * :- *(2) Sort [x#3L ASC NULLS FIRST], false, 0 + * : +- Exchange hashpartitioning(x#3L, 200) + * : +- *(1) Project [(id#0L % 2) AS x#3L] + * : +- *(1) Filter isnotnull((id#0L % 2)) + * : +- *(1) Range (0, 5, step=1, splits=8) + * +- *(4) Sort [y#9L ASC NULLS FIRST], false, 0 + * +- Exchange hashpartitioning(y#9L, 200) + * +- *(3) Project [(id#6L % 2) AS y#9L] + * +- *(3) Filter isnotnull((id#6L % 2)) + * +- *(3) Range (0, 5, step=1, splits=8) + * + * where the ID makes it obvious that not all adjacent codegen'd plan operators are of the + * same codegen stage. + * + * The codegen stage ID is also optionally included in the name of the generated classes as + * a suffix, so that it's easier to associate a generated class back to the physical operator. + * This is controlled by SQLConf: spark.sql.codegen.useIdInClassName + * + * The ID is also included in various log messages. + * + * Within a query, a codegen stage in a plan starts counting from 1, in "insertion order". + * WholeStageCodegenExec operators are inserted into a plan in depth-first post-order. + * See CollapseCodegenStages.insertWholeStageCodegen for the definition of insertion order. + * + * 0 is reserved as a special ID value to indicate a temporary WholeStageCodegenExec object + * is created, e.g. for special fallback handling when an existing WholeStageCodegenExec + * failed to generate/compile code. */ -case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { +case class CollapseCodegenStages( + conf: SQLConf, + codegenStageCounter: AtomicInteger = new AtomicInteger(0)) + extends Rule[SparkPlan] { private def supportCodegen(e: Expression): Boolean = e match { case e: LeafExpression => true @@ -765,35 +870,55 @@ case class CollapseCodegenStages(conf: SQLConf) extends Rule[SparkPlan] { /** * Inserts an InputAdapter on top of those that do not support codegen. */ - private def insertInputAdapter(plan: SparkPlan): SparkPlan = plan match { - case p if !supportCodegen(p) => - // collapse them recursively - InputAdapter(insertWholeStageCodegen(p)) - case j: SortMergeJoinExec => - // The children of SortMergeJoin should do codegen separately. - j.withNewChildren(j.children.map(child => InputAdapter(insertWholeStageCodegen(child)))) - case p => - p.withNewChildren(p.children.map(insertInputAdapter)) + private def insertInputAdapter(plan: SparkPlan, isColumnarInput: Boolean): SparkPlan = { + val isColumnar = adjustColumnar(plan, isColumnarInput) + plan match { + case p if !supportCodegen(p) => + // collapse them recursively + InputAdapter(insertWholeStageCodegen(p, isColumnar), isColumnar) + case j: SortMergeJoinExec => + // The children of SortMergeJoin should do codegen separately. + j.withNewChildren(j.children.map( + child => InputAdapter(insertWholeStageCodegen(child, isColumnar), isColumnar))) + case p => + p.withNewChildren(p.children.map(insertInputAdapter(_, isColumnar))) + } } /** * Inserts a WholeStageCodegen on top of those that support codegen. */ - private def insertWholeStageCodegen(plan: SparkPlan): SparkPlan = plan match { - // For operators that will output domain object, do not insert WholeStageCodegen for it as - // domain object can not be written into unsafe row. - case plan if plan.output.length == 1 && plan.output.head.dataType.isInstanceOf[ObjectType] => - plan.withNewChildren(plan.children.map(insertWholeStageCodegen)) - case plan: CodegenSupport if supportCodegen(plan) => - WholeStageCodegenExec(insertInputAdapter(plan))(WholeStageCodegenId.getNextStageId()) - case other => - other.withNewChildren(other.children.map(insertWholeStageCodegen)) + private def insertWholeStageCodegen(plan: SparkPlan, isColumnarInput: Boolean): SparkPlan = { + val isColumnar = adjustColumnar(plan, isColumnarInput) + plan match { + // For operators that will output domain object, do not insert WholeStageCodegen for it as + // domain object can not be written into unsafe row. + case plan if plan.output.length == 1 && plan.output.head.dataType.isInstanceOf[ObjectType] => + plan.withNewChildren(plan.children.map(insertWholeStageCodegen(_, isColumnar))) + case plan: CodegenSupport if supportCodegen(plan) => + WholeStageCodegenExec( + insertInputAdapter(plan, isColumnar))(codegenStageCounter.incrementAndGet()) + case other => + other.withNewChildren(other.children.map(insertWholeStageCodegen(_, isColumnar))) + } + } + + /** + * Depending on the stage in the plan and if we currently are columnar or not + * return if we are still columnar or not. + */ + private def adjustColumnar(plan: SparkPlan, isColumnar: Boolean): Boolean = + // We are walking up the plan, so columnar starts when we transition to rows + // and ends when we transition to columns + plan match { + case c2r: ColumnarToRowExec => true + case r2c: RowToColumnarExec => false + case _ => isColumnar } def apply(plan: SparkPlan): SparkPlan = { if (conf.wholeStageEnabled) { - WholeStageCodegenId.resetPerQuery() - insertWholeStageCodegen(plan) + insertWholeStageCodegen(plan, false) } else { plan } 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 new file mode 100644 index 0000000000000..61dbc58297388 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala @@ -0,0 +1,429 @@ +/* + * 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.util +import java.util.concurrent.LinkedBlockingQueue + +import scala.collection.JavaConverters._ +import scala.collection.concurrent.TrieMap +import scala.collection.mutable +import scala.concurrent.ExecutionContext +import scala.util.control.NonFatal + +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer} +import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.adaptive.AdaptiveSparkPlanExec._ +import org.apache.spark.sql.execution.adaptive.rule.ReduceNumShufflePartitions +import org.apache.spark.sql.execution.exchange._ +import org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.ThreadUtils + +/** + * A root node to execute the query plan adaptively. It splits the query plan into independent + * stages and executes them in order according to their dependencies. The query stage + * materializes its output at the end. When one stage completes, the data statistics of the + * materialized output will be used to optimize the remainder of the query. + * + * To create query stages, we traverse the query tree bottom up. When we hit an exchange node, + * and if all the child query stages of this exchange node are materialized, we create a new + * query stage for this exchange node. The new stage is then materialized asynchronously once it + * is created. + * + * When one query stage finishes materialization, the rest query is re-optimized and planned based + * on the latest statistics provided by all materialized stages. Then we traverse the query plan + * again and create more stages if possible. After all stages have been materialized, we execute + * the rest of the plan. + */ +case class AdaptiveSparkPlanExec( + initialPlan: SparkPlan, + @transient session: SparkSession, + @transient subqueryMap: Map[Long, ExecSubqueryExpression], + @transient stageCache: TrieMap[SparkPlan, QueryStageExec]) + extends LeafExecNode { + + @transient private val lock = new Object() + + // The logical plan optimizer for re-optimizing the current logical plan. + @transient private val optimizer = new RuleExecutor[LogicalPlan] { + // TODO add more optimization rules + override protected def batches: Seq[Batch] = Seq() + } + + // A list of physical plan rules to be applied before creation of query stages. The physical + // plan should reach a final status of query stages (i.e., no more addition or removal of + // Exchange nodes) after running these rules. + @transient private val queryStagePreparationRules: Seq[Rule[SparkPlan]] = Seq( + PlanAdaptiveSubqueries(subqueryMap), + EnsureRequirements(conf) + ) + + // A list of physical optimizer rules to be applied to a new stage before its execution. These + // optimizations should be stage-independent. + @transient private val queryStageOptimizerRules: Seq[Rule[SparkPlan]] = Seq( + ReduceNumShufflePartitions(conf), + ApplyColumnarRulesAndInsertTransitions(session.sessionState.conf, + session.sessionState.columnarRules), + CollapseCodegenStages(conf) + ) + + @volatile private var currentPhysicalPlan = initialPlan + + private var isFinalPlan = false + + private var currentStageId = 0 + + /** + * Return type for `createQueryStages` + * @param newPlan the new plan with created query stages. + * @param allChildStagesMaterialized whether all child stages have been materialized. + * @param newStages the newly created query stages, including new reused query stages. + */ + private case class CreateStageResult( + newPlan: SparkPlan, + allChildStagesMaterialized: Boolean, + newStages: Seq[(Exchange, QueryStageExec)]) + + def executedPlan: SparkPlan = currentPhysicalPlan + + override def conf: SQLConf = session.sessionState.conf + + override def output: Seq[Attribute] = initialPlan.output + + override def doCanonicalize(): SparkPlan = initialPlan.canonicalized + + override def doExecute(): RDD[InternalRow] = lock.synchronized { + if (isFinalPlan) { + currentPhysicalPlan.execute() + } else { + val executionId = Option( + session.sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)).map(_.toLong) + var currentLogicalPlan = currentPhysicalPlan.logicalLink.get + var result = createQueryStages(currentPhysicalPlan) + val events = new LinkedBlockingQueue[StageMaterializationEvent]() + val errors = new mutable.ArrayBuffer[SparkException]() + while (!result.allChildStagesMaterialized) { + currentPhysicalPlan = result.newPlan + currentLogicalPlan = updateLogicalPlan(currentLogicalPlan, result.newStages) + currentPhysicalPlan.setTagValue(SparkPlan.LOGICAL_PLAN_TAG, currentLogicalPlan) + executionId.foreach(onUpdatePlan) + + // Start materialization of all new stages. + result.newStages.map(_._2).foreach { stage => + stage.materialize().onComplete { res => + if (res.isSuccess) { + events.offer(StageSuccess(stage, res.get)) + } else { + events.offer(StageFailure(stage, res.failed.get)) + } + }(AdaptiveSparkPlanExec.executionContext) + } + + // Wait on the next completed stage, which indicates new stats are available and probably + // new stages can be created. There might be other stages that finish at around the same + // time, so we process those stages too in order to reduce re-planning. + val nextMsg = events.take() + val rem = new util.ArrayList[StageMaterializationEvent]() + events.drainTo(rem) + (Seq(nextMsg) ++ rem.asScala).foreach { + case StageSuccess(stage, res) => + stage.resultOption = Some(res) + case StageFailure(stage, ex) => + errors.append( + new SparkException(s"Failed to materialize query stage: ${stage.treeString}", ex)) + } + + // In case of errors, we cancel all running stages and throw exception. + if (errors.nonEmpty) { + cleanUpAndThrowException(errors) + } + + // Do re-planning and try creating new stages on the new physical plan. + val (newPhysicalPlan, newLogicalPlan) = reOptimize(currentLogicalPlan) + currentPhysicalPlan = newPhysicalPlan + currentLogicalPlan = newLogicalPlan + result = createQueryStages(currentPhysicalPlan) + } + + // Run the final plan when there's no more unfinished stages. + currentPhysicalPlan = applyPhysicalRules(result.newPlan, queryStageOptimizerRules) + currentPhysicalPlan.setTagValue(SparkPlan.LOGICAL_PLAN_TAG, currentLogicalPlan) + isFinalPlan = true + logDebug(s"Final plan: $currentPhysicalPlan") + executionId.foreach(onUpdatePlan) + + currentPhysicalPlan.execute() + } + } + + override def verboseString(maxFields: Int): String = simpleString(maxFields) + + override def simpleString(maxFields: Int): String = + s"AdaptiveSparkPlan(isFinalPlan=$isFinalPlan)" + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + append: String => Unit, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false, + maxFields: Int): Unit = { + super.generateTreeString(depth, lastChildren, append, verbose, prefix, addSuffix, maxFields) + currentPhysicalPlan.generateTreeString( + depth + 1, lastChildren :+ true, append, verbose, "", addSuffix = false, maxFields) + } + + /** + * This method is called recursively to traverse the plan tree bottom-up and create a new query + * stage or try reusing an existing stage if the current node is an [[Exchange]] node and all of + * its child stages have been materialized. + * + * With each call, it returns: + * 1) The new plan replaced with [[QueryStageExec]] nodes where new stages are created. + * 2) Whether the child query stages (if any) of the current node have all been materialized. + * 3) A list of the new query stages that have been created. + */ + private def createQueryStages(plan: SparkPlan): CreateStageResult = plan match { + case e: Exchange => + // First have a quick check in the `stageCache` without having to traverse down the node. + stageCache.get(e.canonicalized) match { + case Some(existingStage) if conf.exchangeReuseEnabled => + val reusedStage = reuseQueryStage(existingStage, e.output) + // When reusing a stage, we treat it a new stage regardless of whether the existing stage + // has been materialized or not. Thus we won't skip re-optimization for a reused stage. + CreateStageResult(newPlan = reusedStage, + allChildStagesMaterialized = false, newStages = Seq((e, reusedStage))) + + case _ => + val result = createQueryStages(e.child) + val newPlan = e.withNewChildren(Seq(result.newPlan)).asInstanceOf[Exchange] + // Create a query stage only when all the child query stages are ready. + if (result.allChildStagesMaterialized) { + var newStage = newQueryStage(newPlan) + if (conf.exchangeReuseEnabled) { + // Check the `stageCache` again for reuse. If a match is found, ditch the new stage + // and reuse the existing stage found in the `stageCache`, otherwise update the + // `stageCache` with the new stage. + val queryStage = stageCache.getOrElseUpdate(e.canonicalized, newStage) + if (queryStage.ne(newStage)) { + newStage = reuseQueryStage(queryStage, e.output) + } + } + + // We've created a new stage, which is obviously not ready yet. + CreateStageResult(newPlan = newStage, + allChildStagesMaterialized = false, newStages = Seq((e, newStage))) + } else { + CreateStageResult(newPlan = newPlan, + allChildStagesMaterialized = false, newStages = result.newStages) + } + } + + case q: QueryStageExec => + CreateStageResult(newPlan = q, + allChildStagesMaterialized = q.resultOption.isDefined, newStages = Seq.empty) + + case _ => + if (plan.children.isEmpty) { + CreateStageResult(newPlan = plan, allChildStagesMaterialized = true, newStages = Seq.empty) + } else { + val results = plan.children.map(createQueryStages) + CreateStageResult( + newPlan = plan.withNewChildren(results.map(_.newPlan)), + allChildStagesMaterialized = results.forall(_.allChildStagesMaterialized), + newStages = results.flatMap(_.newStages)) + } + } + + private def newQueryStage(e: Exchange): QueryStageExec = { + val optimizedPlan = applyPhysicalRules(e.child, queryStageOptimizerRules) + val queryStage = e match { + case s: ShuffleExchangeExec => + ShuffleQueryStageExec(currentStageId, s.copy(child = optimizedPlan)) + case b: BroadcastExchangeExec => + BroadcastQueryStageExec(currentStageId, b.copy(child = optimizedPlan)) + } + currentStageId += 1 + queryStage + } + + private def reuseQueryStage(s: QueryStageExec, output: Seq[Attribute]): QueryStageExec = { + val queryStage = ReusedQueryStageExec(currentStageId, s, output) + currentStageId += 1 + queryStage + } + + /** + * Returns the updated logical plan after new query stages have been created and the physical + * plan has been updated with the newly created stages. + * 1. If the new query stage can be mapped to an integral logical sub-tree, replace the + * corresponding logical sub-tree with a leaf node [[LogicalQueryStage]] referencing the new + * query stage. For example: + * Join SMJ SMJ + * / \ / \ / \ + * r1 r2 => Xchg1 Xchg2 => Stage1 Stage2 + * | | + * r1 r2 + * The updated plan node will be: + * Join + * / \ + * LogicalQueryStage1(Stage1) LogicalQueryStage2(Stage2) + * + * 2. Otherwise (which means the new query stage can only be mapped to part of a logical + * sub-tree), replace the corresponding logical sub-tree with a leaf node + * [[LogicalQueryStage]] referencing to the top physical node into which this logical node is + * transformed during physical planning. For example: + * Agg HashAgg HashAgg + * | | | + * child => Xchg => Stage1 + * | + * HashAgg + * | + * child + * The updated plan node will be: + * LogicalQueryStage(HashAgg - Stage1) + */ + private def updateLogicalPlan( + logicalPlan: LogicalPlan, + newStages: Seq[(Exchange, QueryStageExec)]): LogicalPlan = { + var currentLogicalPlan = logicalPlan + newStages.foreach { + case (exchange, stage) => + // Get the corresponding logical node for `exchange`. If `exchange` has been transformed + // from a `Repartition`, it should have `logicalLink` available by itself; otherwise + // traverse down to find the first node that is not generated by `EnsureRequirements`. + val logicalNodeOpt = exchange.logicalLink.orElse(exchange.collectFirst { + case p if p.logicalLink.isDefined => p.logicalLink.get + }) + assert(logicalNodeOpt.isDefined) + val logicalNode = logicalNodeOpt.get + val physicalNode = currentPhysicalPlan.collectFirst { + case p if p.eq(stage) || p.logicalLink.exists(logicalNode.eq) => p + } + assert(physicalNode.isDefined) + // Replace the corresponding logical node with LogicalQueryStage + val newLogicalNode = LogicalQueryStage(logicalNode, physicalNode.get) + val newLogicalPlan = currentLogicalPlan.transformDown { + case p if p.eq(logicalNode) => newLogicalNode + } + assert(newLogicalPlan != currentLogicalPlan, + s"logicalNode: $logicalNode; " + + s"logicalPlan: $currentLogicalPlan " + + s"physicalPlan: $currentPhysicalPlan" + + s"stage: $stage") + currentLogicalPlan = newLogicalPlan + } + currentLogicalPlan + } + + /** + * Re-optimize and run physical planning on the current logical plan based on the latest stats. + */ + private def reOptimize(logicalPlan: LogicalPlan): (SparkPlan, LogicalPlan) = { + logicalPlan.invalidateStatsCache() + val optimized = optimizer.execute(logicalPlan) + SparkSession.setActiveSession(session) + val sparkPlan = session.sessionState.planner.plan(ReturnAnswer(optimized)).next() + val newPlan = applyPhysicalRules(sparkPlan, queryStagePreparationRules) + (newPlan, optimized) + } + + /** + * Notify the listeners of the physical plan change. + */ + private def onUpdatePlan(executionId: Long): Unit = { + session.sparkContext.listenerBus.post(SparkListenerSQLAdaptiveExecutionUpdate( + executionId, + SQLExecution.getQueryExecution(executionId).toString, + SparkPlanInfo.fromSparkPlan(this))) + } + + /** + * Cancel all running stages with best effort and throw an Exception containing all stage + * materialization errors and stage cancellation errors. + */ + private def cleanUpAndThrowException(errors: Seq[SparkException]): Unit = { + val runningStages = currentPhysicalPlan.collect { + case s: QueryStageExec => s + } + val cancelErrors = new mutable.ArrayBuffer[SparkException]() + try { + runningStages.foreach { s => + try { + s.cancel() + } catch { + case NonFatal(t) => + cancelErrors.append( + new SparkException(s"Failed to cancel query stage: ${s.treeString}", t)) + } + } + } finally { + val ex = new SparkException( + "Adaptive execution failed due to stage materialization failures.", errors.head) + errors.tail.foreach(ex.addSuppressed) + cancelErrors.foreach(ex.addSuppressed) + throw ex + } + } +} + +object AdaptiveSparkPlanExec { + private val executionContext = ExecutionContext.fromExecutorService( + ThreadUtils.newDaemonCachedThreadPool("QueryStageCreator", 16)) + + /** + * Creates the list of physical plan rules to be applied before creation of query stages. + */ + def createQueryStagePreparationRules( + conf: SQLConf, + subqueryMap: Map[Long, ExecSubqueryExpression]): Seq[Rule[SparkPlan]] = { + Seq( + PlanAdaptiveSubqueries(subqueryMap), + EnsureRequirements(conf)) + } + + /** + * Apply a list of physical operator rules on a [[SparkPlan]]. + */ + def applyPhysicalRules(plan: SparkPlan, rules: Seq[Rule[SparkPlan]]): SparkPlan = { + rules.foldLeft(plan) { case (sp, rule) => rule.apply(sp) } + } +} + +/** + * The event type for stage materialization. + */ +sealed trait StageMaterializationEvent + +/** + * The materialization of a query stage completed with success. + */ +case class StageSuccess(stage: QueryStageExec, result: Any) extends StageMaterializationEvent + +/** + * The materialization of a query stage hit an error and failed. + */ +case class StageFailure(stage: QueryStageExec, error: Throwable) extends StageMaterializationEvent 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 new file mode 100644 index 0000000000000..14ca2b41a442b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/InsertAdaptiveSparkPlan.scala @@ -0,0 +1,131 @@ +/* + * 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 scala.collection.concurrent.TrieMap +import scala.collection.mutable + +import org.apache.spark.sql.{execution, SparkSession} +import org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +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.internal.SQLConf +import org.apache.spark.sql.types.StructType + +/** + * This rule wraps the query plan with an [[AdaptiveSparkPlanExec]], which executes the query plan + * and re-optimize the plan during execution based on runtime data statistics. + * + * Note that this rule is stateful and thus should not be reused across query executions. + */ +case class InsertAdaptiveSparkPlan(session: SparkSession) extends Rule[SparkPlan] { + + private val conf = session.sessionState.conf + + // Exchange-reuse is shared across the entire query, including sub-queries. + private val stageCache = new TrieMap[SparkPlan, QueryStageExec]() + + override def apply(plan: SparkPlan): SparkPlan = plan match { + case _: ExecutedCommandExec => plan + case _ if conf.adaptiveExecutionEnabled && supportAdaptive(plan) => + try { + // Plan sub-queries recursively and pass in the shared stage cache for exchange reuse. Fall + // back to non-adaptive mode if adaptive execution is supported in any of the sub-queries. + val subqueryMap = buildSubqueryMap(plan) + // Run preparation rules. + val preparations = AdaptiveSparkPlanExec.createQueryStagePreparationRules( + session.sessionState.conf, subqueryMap) + val newPlan = AdaptiveSparkPlanExec.applyPhysicalRules(plan, preparations) + logDebug(s"Adaptive execution enabled for plan: $plan") + AdaptiveSparkPlanExec(newPlan, session, subqueryMap, stageCache) + } catch { + case SubqueryAdaptiveNotSupportedException(subquery) => + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + + s"but is not supported for sub-query: $subquery.") + plan + } + case _ => + if (conf.adaptiveExecutionEnabled) { + logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} is enabled " + + s"but is not supported for query: $plan.") + } + plan + } + + private def supportAdaptive(plan: SparkPlan): Boolean = { + sanityCheck(plan) && + !plan.logicalLink.exists(_.isStreaming) && + plan.children.forall(supportAdaptive) + } + + private def sanityCheck(plan: SparkPlan): Boolean = + plan.logicalLink.isDefined + + /** + * Returns an expression-id-to-execution-plan map for all the sub-queries. + * For each sub-query, generate the adaptive execution plan for each sub-query by applying this + * rule, or reuse the execution plan from another sub-query of the same semantics if possible. + */ + private def buildSubqueryMap(plan: SparkPlan): Map[Long, ExecSubqueryExpression] = { + val subqueryMapBuilder = mutable.HashMap.empty[Long, ExecSubqueryExpression] + plan.foreach(_.expressions.foreach(_.foreach { + case expressions.ScalarSubquery(p, _, exprId) + if !subqueryMapBuilder.contains(exprId.id) => + val executedPlan = getExecutedPlan(p) + val scalarSubquery = execution.ScalarSubquery( + SubqueryExec(s"subquery${exprId.id}", executedPlan), exprId) + subqueryMapBuilder.put(exprId.id, scalarSubquery) + case _ => + })) + + // Reuse subqueries + if (session.sessionState.conf.subqueryReuseEnabled) { + // Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls. + val reuseMap = mutable.HashMap[StructType, mutable.ArrayBuffer[BaseSubqueryExec]]() + subqueryMapBuilder.keySet.foreach { exprId => + val sub = subqueryMapBuilder(exprId) + val sameSchema = + reuseMap.getOrElseUpdate(sub.plan.schema, mutable.ArrayBuffer.empty) + val sameResult = sameSchema.find(_.sameResult(sub.plan)) + if (sameResult.isDefined) { + val newExpr = sub.withNewPlan(ReusedSubqueryExec(sameResult.get)) + subqueryMapBuilder.update(exprId, newExpr) + } else { + sameSchema += sub.plan + } + } + } + + subqueryMapBuilder.toMap + } + + private def getExecutedPlan(plan: LogicalPlan): SparkPlan = { + val queryExec = new QueryExecution(session, plan) + // Apply the same instance of this rule to sub-queries so that sub-queries all share the + // same `stageCache` for Exchange reuse. + val adaptivePlan = this.apply(queryExec.sparkPlan) + if (!adaptivePlan.isInstanceOf[AdaptiveSparkPlanExec]) { + throw SubqueryAdaptiveNotSupportedException(plan) + } + adaptivePlan + } +} + +private case class SubqueryAdaptiveNotSupportedException(plan: LogicalPlan) extends Exception {} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala new file mode 100644 index 0000000000000..9914eddd53a3d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStage.scala @@ -0,0 +1,56 @@ +/* + * 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 org.apache.spark.sql.catalyst.expressions.{Attribute, SortOrder} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution.SparkPlan + +/** + * The LogicalPlan wrapper for a [[QueryStageExec]], or a snippet of physical plan containing + * a [[QueryStageExec]], in which all ancestor nodes of the [[QueryStageExec]] are linked to + * the same logical node. + * + * For example, a logical Aggregate can be transformed into FinalAgg - Shuffle - PartialAgg, in + * which the Shuffle will be wrapped into a [[QueryStageExec]], thus the [[LogicalQueryStage]] + * will have FinalAgg - QueryStageExec as its physical plan. + */ +// TODO we can potentially include only [[QueryStageExec]] in this class if we make the aggregation +// planning aware of partitioning. +case class LogicalQueryStage( + logicalPlan: LogicalPlan, + physicalPlan: SparkPlan) extends LeafNode { + + override def output: Seq[Attribute] = logicalPlan.output + override val isStreaming: Boolean = logicalPlan.isStreaming + override val outputOrdering: Seq[SortOrder] = physicalPlan.outputOrdering + + override def computeStats(): Statistics = { + // TODO this is not accurate when there is other physical nodes above QueryStageExec. + val physicalStats = physicalPlan.collectFirst { + case s: QueryStageExec => s + }.flatMap(_.computeStats()) + if (physicalStats.isDefined) { + logDebug(s"Physical stats available as ${physicalStats.get} for plan: $physicalPlan") + } else { + logDebug(s"Physical stats not available for plan: $physicalPlan") + } + physicalStats.getOrElse(logicalPlan.stats) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.scala new file mode 100644 index 0000000000000..a0d07a68ab0f4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/LogicalQueryStageStrategy.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.execution.adaptive + +import org.apache.spark.sql.Strategy +import org.apache.spark.sql.catalyst.expressions.PredicateHelper +import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys +import org.apache.spark.sql.catalyst.plans.logical.{Join, LogicalPlan} +import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, BroadcastNestedLoopJoinExec, BuildLeft, BuildRight} + +/** + * Strategy for plans containing [[LogicalQueryStage]] nodes: + * 1. Transforms [[LogicalQueryStage]] to its corresponding physical plan that is either being + * executed or has already completed execution. + * 2. Transforms [[Join]] which has one child relation already planned and executed as a + * [[BroadcastQueryStageExec]]. This is to prevent reversing a broadcast stage into a shuffle + * stage in case of the larger join child relation finishes before the smaller relation. Note + * that this rule needs to applied before regular join strategies. + */ +object LogicalQueryStageStrategy extends Strategy with PredicateHelper { + + private def isBroadcastStage(plan: LogicalPlan): Boolean = plan match { + case LogicalQueryStage(_, physicalPlan) + if BroadcastQueryStageExec.isBroadcastQueryStageExec(physicalPlan) => + true + case _ => false + } + + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case ExtractEquiJoinKeys(joinType, leftKeys, rightKeys, condition, left, right, hint) + if isBroadcastStage(left) || isBroadcastStage(right) => + val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight + Seq(BroadcastHashJoinExec( + leftKeys, rightKeys, joinType, buildSide, condition, planLater(left), planLater(right))) + + case j @ Join(left, right, joinType, condition, _) + if isBroadcastStage(left) || isBroadcastStage(right) => + val buildSide = if (isBroadcastStage(left)) BuildLeft else BuildRight + BroadcastNestedLoopJoinExec( + planLater(left), planLater(right), buildSide, joinType, condition) :: Nil + + case q: LogicalQueryStage => + q.physicalPlan :: Nil + + case _ => Nil + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala new file mode 100644 index 0000000000000..4af7432d7bed0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/PlanAdaptiveSubqueries.scala @@ -0,0 +1,36 @@ +/* + * 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 org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.expressions.ListQuery +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ExecSubqueryExpression, SparkPlan} + +case class PlanAdaptiveSubqueries( + subqueryMap: Map[Long, ExecSubqueryExpression]) extends Rule[SparkPlan] { + + def apply(plan: SparkPlan): SparkPlan = { + plan.transformAllExpressions { + case expressions.ScalarSubquery(_, _, exprId) => + subqueryMap(exprId.id) + case expressions.InSubquery(_, ListQuery(_, _, exprId, _)) => + subqueryMap(exprId.id) + } + } +} 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 new file mode 100644 index 0000000000000..c803ca3638b4a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/QueryStageExec.scala @@ -0,0 +1,222 @@ +/* + * 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 scala.concurrent.Future + +import org.apache.spark.{FutureAction, MapOutputStatistics} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.Statistics +import org.apache.spark.sql.catalyst.plans.physical.Partitioning +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.exchange._ + + +/** + * A query stage is an independent subgraph of the query plan. Query stage materializes its output + * before proceeding with further operators of the query plan. The data statistics of the + * materialized output can be used to optimize subsequent query stages. + * + * There are 2 kinds of query stages: + * 1. Shuffle query stage. This stage materializes its output to shuffle files, and Spark launches + * another job to execute the further operators. + * 2. Broadcast query stage. This stage materializes its output to an array in driver JVM. Spark + * broadcasts the array before executing the further operators. + */ +abstract class QueryStageExec extends LeafExecNode { + + /** + * An id of this query stage which is unique in the entire query plan. + */ + val id: Int + + /** + * The sub-tree of the query plan that belongs to this query stage. + */ + val plan: SparkPlan + + /** + * Materialize this query stage, to prepare for the execution, like submitting map stages, + * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this + * stage is ready. + */ + def doMaterialize(): Future[Any] + + /** + * Cancel the stage materialization if in progress; otherwise do nothing. + */ + def cancel(): Unit + + /** + * Materialize this query stage, to prepare for the execution, like submitting map stages, + * broadcasting data, etc. The caller side can use the returned [[Future]] to wait until this + * stage is ready. + */ + final def materialize(): Future[Any] = executeQuery { + doMaterialize() + } + + /** + * Compute the statistics of the query stage if executed, otherwise None. + */ + def computeStats(): Option[Statistics] = resultOption.map { _ => + // Metrics `dataSize` are available in both `ShuffleExchangeExec` and `BroadcastExchangeExec`. + Statistics(sizeInBytes = plan.metrics("dataSize").value) + } + + @transient + @volatile + private[adaptive] var resultOption: Option[Any] = None + + override def output: Seq[Attribute] = plan.output + override def outputPartitioning: Partitioning = plan.outputPartitioning + override def outputOrdering: Seq[SortOrder] = plan.outputOrdering + override def executeCollect(): Array[InternalRow] = plan.executeCollect() + override def executeTake(n: Int): Array[InternalRow] = plan.executeTake(n) + override def executeToIterator(): Iterator[InternalRow] = plan.executeToIterator() + + override def doPrepare(): Unit = plan.prepare() + override def doExecute(): RDD[InternalRow] = plan.execute() + override def doExecuteBroadcast[T](): Broadcast[T] = plan.executeBroadcast() + override def doCanonicalize(): SparkPlan = plan.canonicalized + + protected override def stringArgs: Iterator[Any] = Iterator.single(id) + + override def generateTreeString( + depth: Int, + lastChildren: Seq[Boolean], + append: String => Unit, + verbose: Boolean, + prefix: String = "", + addSuffix: Boolean = false, + maxFields: Int): Unit = { + super.generateTreeString(depth, lastChildren, append, verbose, prefix, addSuffix, maxFields) + plan.generateTreeString( + depth + 1, lastChildren :+ true, append, verbose, "", false, maxFields) + } +} + +/** + * A shuffle query stage whose child is a [[ShuffleExchangeExec]]. + */ +case class ShuffleQueryStageExec( + override val id: Int, + override val plan: ShuffleExchangeExec) extends QueryStageExec { + + @transient lazy val mapOutputStatisticsFuture: Future[MapOutputStatistics] = { + if (plan.inputRDD.getNumPartitions == 0) { + Future.successful(null) + } else { + sparkContext.submitMapStage(plan.shuffleDependency) + } + } + + override def doMaterialize(): Future[Any] = { + mapOutputStatisticsFuture + } + + override def cancel(): Unit = { + mapOutputStatisticsFuture match { + case action: FutureAction[MapOutputStatistics] if !mapOutputStatisticsFuture.isCompleted => + action.cancel() + case _ => + } + } +} + +/** + * A broadcast query stage whose child is a [[BroadcastExchangeExec]]. + */ +case class BroadcastQueryStageExec( + override val id: Int, + override val plan: BroadcastExchangeExec) extends QueryStageExec { + + override def doMaterialize(): Future[Any] = { + plan.completionFuture + } + + override def cancel(): Unit = { + if (!plan.relationFuture.isDone) { + sparkContext.cancelJobGroup(plan.runId.toString) + plan.relationFuture.cancel(true) + } + } +} + +object ShuffleQueryStageExec { + /** + * Returns true if the plan is a [[ShuffleQueryStageExec]] or a reused [[ShuffleQueryStageExec]]. + */ + def isShuffleQueryStageExec(plan: SparkPlan): Boolean = plan match { + case r: ReusedQueryStageExec => isShuffleQueryStageExec(r.plan) + case _: ShuffleQueryStageExec => true + case _ => false + } +} + +object BroadcastQueryStageExec { + /** + * Returns true if the plan is a [[BroadcastQueryStageExec]] or a reused + * [[BroadcastQueryStageExec]]. + */ + def isBroadcastQueryStageExec(plan: SparkPlan): Boolean = plan match { + case r: ReusedQueryStageExec => isBroadcastQueryStageExec(r.plan) + case _: BroadcastQueryStageExec => true + case _ => false + } +} + +/** + * A wrapper for reused query stage to have different output. + */ +case class ReusedQueryStageExec( + override val id: Int, + override val plan: QueryStageExec, + override val output: Seq[Attribute]) extends QueryStageExec { + + override def doMaterialize(): Future[Any] = { + plan.materialize() + } + + override def cancel(): Unit = { + plan.cancel() + } + + // `ReusedQueryStageExec` can have distinct set of output attribute ids from its child, we need + // to update the attribute ids in `outputPartitioning` and `outputOrdering`. + private lazy val updateAttr: Expression => Expression = { + val originalAttrToNewAttr = AttributeMap(plan.output.zip(output)) + e => e.transform { + case attr: Attribute => originalAttrToNewAttr.getOrElse(attr, attr) + } + } + + override def outputPartitioning: Partitioning = plan.outputPartitioning match { + case e: Expression => updateAttr(e).asInstanceOf[Partitioning] + case other => other + } + + override def outputOrdering: Seq[SortOrder] = { + plan.outputOrdering.map(updateAttr(_).asInstanceOf[SortOrder]) + } + + override def computeStats(): Option[Statistics] = plan.computeStats() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.scala new file mode 100644 index 0000000000000..78923433eaab9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/ReduceNumShufflePartitions.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.sql.execution.adaptive.rule + +import scala.collection.mutable.ArrayBuffer +import scala.concurrent.duration.Duration + +import org.apache.spark.MapOutputStatistics +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan, UnaryExecNode} +import org.apache.spark.sql.execution.adaptive.{QueryStageExec, ReusedQueryStageExec, ShuffleQueryStageExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.util.ThreadUtils + +/** + * A rule to adjust the post shuffle partitions based on the map output statistics. + * + * The strategy used to determine the number of post-shuffle partitions is described as follows. + * To determine the number of post-shuffle partitions, we have a target input size for a + * post-shuffle partition. Once we have size statistics of all pre-shuffle partitions, we will do + * a pass of those statistics and pack pre-shuffle partitions with continuous indices to a single + * post-shuffle partition until adding another pre-shuffle partition would cause the size of a + * post-shuffle partition to be greater than the target size. + * + * For example, we have two stages with the following pre-shuffle partition size statistics: + * stage 1: [100 MiB, 20 MiB, 100 MiB, 10MiB, 30 MiB] + * stage 2: [10 MiB, 10 MiB, 70 MiB, 5 MiB, 5 MiB] + * assuming the target input size is 128 MiB, we will have four post-shuffle partitions, + * which are: + * - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MiB) + * - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MiB) + * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MiB) + * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MiB) + */ +case class ReduceNumShufflePartitions(conf: SQLConf) extends Rule[SparkPlan] { + + override def apply(plan: SparkPlan): SparkPlan = { + if (!conf.reducePostShufflePartitionsEnabled) { + return plan + } + if (!plan.collectLeaves().forall(_.isInstanceOf[QueryStageExec])) { + // If not all leaf nodes are query stages, it's not safe to reduce the number of + // shuffle partitions, because we may break the assumption that all children of a spark plan + // have same number of output partitions. + return plan + } + + val shuffleStages = plan.collect { + case stage: ShuffleQueryStageExec => stage + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => stage + } + // ShuffleExchanges introduced by repartition do not support changing the number of partitions. + // We change the number of partitions in the stage only if all the ShuffleExchanges support it. + if (!shuffleStages.forall(_.plan.canChangeNumPartitions)) { + plan + } else { + val shuffleMetrics = shuffleStages.map { stage => + val metricsFuture = stage.mapOutputStatisticsFuture + assert(metricsFuture.isCompleted, "ShuffleQueryStageExec should already be ready") + ThreadUtils.awaitResult(metricsFuture, Duration.Zero) + } + + // `ShuffleQueryStageExec` gives null mapOutputStatistics when the input RDD has 0 partitions, + // we should skip it when calculating the `partitionStartIndices`. + val validMetrics = shuffleMetrics.filter(_ != null) + if (validMetrics.nonEmpty) { + val partitionStartIndices = estimatePartitionStartIndices(validMetrics.toArray) + // This transformation adds new nodes, so we must use `transformUp` here. + plan.transformUp { + // even for shuffle exchange whose input RDD has 0 partition, we should still update its + // `partitionStartIndices`, so that all the leaf shuffles in a stage have the same + // number of output partitions. + case stage: QueryStageExec if ShuffleQueryStageExec.isShuffleQueryStageExec(stage) => + CoalescedShuffleReaderExec(stage, partitionStartIndices) + } + } else { + plan + } + } + } + + /** + * Estimates partition start indices for post-shuffle partitions based on + * mapOutputStatistics provided by all pre-shuffle stages. + */ + // visible for testing. + private[sql] def estimatePartitionStartIndices( + mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { + val minNumPostShufflePartitions = conf.minNumPostShufflePartitions + val advisoryTargetPostShuffleInputSize = conf.targetPostShuffleInputSize + // If minNumPostShufflePartitions is defined, it is possible that we need to use a + // value less than advisoryTargetPostShuffleInputSize as the target input size of + // a post shuffle task. + val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum + // The max at here is to make sure that when we have an empty table, we + // only have a single post-shuffle partition. + // There is no particular reason that we pick 16. We just need a number to + // prevent maxPostShuffleInputSize from being set to 0. + val maxPostShuffleInputSize = math.max( + math.ceil(totalPostShuffleInputSize / minNumPostShufflePartitions.toDouble).toLong, 16) + val targetPostShuffleInputSize = + math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize) + + logInfo( + s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + + s"targetPostShuffleInputSize $targetPostShuffleInputSize.") + + // Make sure we do get the same number of pre-shuffle partitions for those stages. + val distinctNumPreShufflePartitions = + mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct + // The reason that we are expecting a single value of the number of pre-shuffle partitions + // is that when we add Exchanges, we set the number of pre-shuffle partitions + // (i.e. map output partitions) using a static setting, which is the value of + // spark.sql.shuffle.partitions. Even if two input RDDs are having different + // number of partitions, they will have the same number of pre-shuffle partitions + // (i.e. map output partitions). + assert( + distinctNumPreShufflePartitions.length == 1, + "There should be only one distinct value of the number pre-shuffle partitions " + + "among registered Exchange operator.") + val numPreShufflePartitions = distinctNumPreShufflePartitions.head + + val partitionStartIndices = ArrayBuffer[Int]() + // The first element of partitionStartIndices is always 0. + partitionStartIndices += 0 + + var postShuffleInputSize = 0L + + var i = 0 + while (i < numPreShufflePartitions) { + // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. + // Then, we add the total size to postShuffleInputSize. + var nextShuffleInputSize = 0L + var j = 0 + while (j < mapOutputStatistics.length) { + nextShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i) + j += 1 + } + + // If including the nextShuffleInputSize would exceed the target partition size, then start a + // new partition. + if (i > 0 && postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize) { + partitionStartIndices += i + // reset postShuffleInputSize. + postShuffleInputSize = nextShuffleInputSize + } else { + postShuffleInputSize += nextShuffleInputSize + } + + i += 1 + } + + partitionStartIndices.toArray + } +} + +case class CoalescedShuffleReaderExec( + child: QueryStageExec, + partitionStartIndices: Array[Int]) extends UnaryExecNode { + + override def output: Seq[Attribute] = child.output + + override def doCanonicalize(): SparkPlan = child.canonicalized + + override def outputPartitioning: Partitioning = { + UnknownPartitioning(partitionStartIndices.length) + } + + private var cachedShuffleRDD: ShuffledRowRDD = null + + override protected def doExecute(): RDD[InternalRow] = { + if (cachedShuffleRDD == null) { + cachedShuffleRDD = child match { + case stage: ShuffleQueryStageExec => + stage.plan.createShuffledRDD(Some(partitionStartIndices)) + case ReusedQueryStageExec(_, stage: ShuffleQueryStageExec, _) => + stage.plan.createShuffledRDD(Some(partitionStartIndices)) + } + } + cachedShuffleRDD + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala index 6be88c463dbd9..4d762c5ea9f34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggUtils.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.optimizer.NormalizeFloatingNumbers import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.execution.streaming.{StateStoreRestoreExec, StateStoreSaveExec} -import org.apache.spark.sql.internal.SQLConf /** * Utility functions used by the query planner to convert our plan to new aggregation code path. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala index 98c4a51299958..d03de1507fbbd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/AggregationIterator.scala @@ -74,19 +74,19 @@ abstract class AggregationIterator( startingInputBufferOffset: Int): Array[AggregateFunction] = { var mutableBufferOffset = 0 var inputBufferOffset: Int = startingInputBufferOffset - val expressionsLength = expressions.length - val functions = new Array[AggregateFunction](expressionsLength) + val functions = new Array[AggregateFunction](expressions.length) var i = 0 - while (i < expressionsLength) { - val func = expressions(i).aggregateFunction - val funcWithBoundReferences: AggregateFunction = expressions(i).mode match { + val inputAttributeSeq: AttributeSeq = inputAttributes + for (expression <- expressions) { + val func = expression.aggregateFunction + val funcWithBoundReferences: AggregateFunction = expression.mode match { case Partial | Complete if func.isInstanceOf[ImperativeAggregate] => // We need to create BoundReferences if the function is not an // expression-based aggregate function (it does not support code-gen) and the mode of // this function is Partial or Complete because we will call eval of this // function's children in the update method of this aggregate function. // Those eval calls require BoundReferences to work. - BindReferences.bindReference(func, inputAttributes) + BindReferences.bindReference(func, inputAttributeSeq) case _ => // We only need to set inputBufferOffset for aggregate functions with mode // PartialMerge and Final. @@ -157,9 +157,9 @@ abstract class AggregationIterator( inputAttributes: Seq[Attribute]): (InternalRow, InternalRow) => Unit = { val joinedRow = new JoinedRow if (expressions.nonEmpty) { - val mergeExpressions = functions.zipWithIndex.flatMap { - case (ae: DeclarativeAggregate, i) => - expressions(i).mode match { + val mergeExpressions = functions.zip(expressions).flatMap { + case (ae: DeclarativeAggregate, expression) => + expression.mode match { case Partial | Complete => ae.updateExpressions case PartialMerge | Final => ae.mergeExpressions } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala index 6155ec9d30db4..4a95f76381339 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala @@ -17,19 +17,25 @@ package org.apache.spark.sql.execution.aggregate +import java.util.concurrent.TimeUnit._ + import org.apache.spark.TaskContext -import org.apache.spark.memory.TaskMemoryManager +import org.apache.spark.memory.{SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.execution.vectorized.MutableColumnarRow +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, StringType, StructType} import org.apache.spark.unsafe.KVIterator import org.apache.spark.util.Utils @@ -61,8 +67,9 @@ case class HashAggregateExec( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "peakMemory" -> SQLMetrics.createSizeMetric(sparkContext, "peak memory"), "spillSize" -> SQLMetrics.createSizeMetric(sparkContext, "spill size"), - "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "aggregate time"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "time in aggregation build"), + "avgHashProbe" -> + SQLMetrics.createAverageMetric(sparkContext, "avg hash probe bucket list iters")) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) @@ -132,7 +139,7 @@ case class HashAggregateExec( aggregationIterator } } - aggTime += (System.nanoTime() - beforeAgg) / 1000000 + aggTime += NANOSECONDS.toMillis(System.nanoTime() - beforeAgg) res } } @@ -198,15 +205,13 @@ case class HashAggregateExec( val (resultVars, genResult) = if (modes.contains(Final) || modes.contains(Complete)) { // evaluate aggregate results ctx.currentVars = bufVars - val aggResults = functions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) - } + val aggResults = bindReferences( + functions.map(_.evaluateExpression), + aggregateBufferAttributes).map(_.genCode(ctx)) val evaluateAggResults = evaluateVariables(aggResults) // evaluate result expressions ctx.currentVars = aggResults - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, aggregateAttributes).genCode(ctx) - } + val resultVars = bindReferences(resultExpressions, aggregateAttributes).map(_.genCode(ctx)) (resultVars, s""" |$evaluateAggResults |${evaluateVariables(resultVars)} @@ -239,7 +244,7 @@ case class HashAggregateExec( | $initAgg = true; | long $beforeAgg = System.nanoTime(); | $doAggFuncName(); - | $aggTime.add((System.nanoTime() - $beforeAgg) / 1000000); + | $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); | | // output the result | ${genResult.trim} @@ -263,7 +268,7 @@ case class HashAggregateExec( } } ctx.currentVars = bufVars ++ input - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttrs)) + val boundUpdateExpr = bindReferences(updateExpr, inputAttrs) val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) val effectiveCodes = subExprs.codes.mkString("\n") val aggVals = ctx.withSubExprEliminationExprs(subExprs.states) { @@ -362,7 +367,7 @@ case class HashAggregateExec( metrics.incPeakExecutionMemory(maxMemory) // Update average hashmap probe - avgHashProbe.set(hashMap.getAverageProbesPerLookup()) + avgHashProbe.set(hashMap.getAvgHashProbeBucketListIterations) if (sorter == null) { // not spilled @@ -455,20 +460,23 @@ case class HashAggregateExec( val evaluateBufferVars = evaluateVariables(bufferVars) // evaluate the aggregation result ctx.currentVars = bufferVars - val aggResults = declFunctions.map(_.evaluateExpression).map { e => - BindReferences.bindReference(e, aggregateBufferAttributes).genCode(ctx) - } + val aggResults = bindReferences( + declFunctions.map(_.evaluateExpression), + aggregateBufferAttributes).map(_.genCode(ctx)) val evaluateAggResults = evaluateVariables(aggResults) // generate the final result ctx.currentVars = keyVars ++ aggResults val inputAttrs = groupingAttributes ++ aggregateAttributes - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).genCode(ctx) - } + val resultVars = bindReferences[Expression]( + resultExpressions, + inputAttrs).map(_.genCode(ctx)) + val evaluateNondeterministicResults = + evaluateNondeterministicVariables(output, resultVars, resultExpressions) s""" $evaluateKeyVars $evaluateBufferVars $evaluateAggResults + $evaluateNondeterministicResults ${consume(ctx, resultVars)} """ } else if (modes.contains(Partial) || modes.contains(PartialMerge)) { @@ -493,9 +501,9 @@ case class HashAggregateExec( ctx.currentVars = keyVars ++ resultBufferVars val inputAttrs = resultExpressions.map(_.toAttribute) - val resultVars = resultExpressions.map { e => - BindReferences.bindReference(e, inputAttrs).genCode(ctx) - } + val resultVars = bindReferences[Expression]( + resultExpressions, + inputAttrs).map(_.genCode(ctx)) s""" $evaluateKeyVars $evaluateResultBufferVars @@ -505,10 +513,15 @@ case class HashAggregateExec( // generate result based on grouping key ctx.INPUT_ROW = keyTerm ctx.currentVars = null - val eval = resultExpressions.map{ e => - BindReferences.bindReference(e, groupingAttributes).genCode(ctx) - } - consume(ctx, eval) + val resultVars = bindReferences[Expression]( + resultExpressions, + groupingAttributes).map(_.genCode(ctx)) + val evaluateNondeterministicResults = + evaluateNondeterministicVariables(output, resultVars, resultExpressions) + s""" + $evaluateNondeterministicResults + ${consume(ctx, resultVars)} + """ } ctx.addNewFunction(funcName, s""" @@ -547,7 +560,7 @@ case class HashAggregateExec( private def enableTwoLevelHashMap(ctx: CodegenContext): Unit = { if (!checkIfFastHashMapSupported(ctx)) { if (modes.forall(mode => mode == Partial || mode == PartialMerge) && !Utils.isTesting) { - logInfo("spark.sql.codegen.aggregate.map.twolevel.enabled is set to true, but" + logInfo(s"${SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key} is set to true, but" + " current version of codegened fast hashmap does not support this aggregate.") } } else { @@ -555,8 +568,7 @@ case class HashAggregateExec( // This is for testing/benchmarking only. // We enforce to first level to be a vectorized hashmap, instead of the default row-based one. - isVectorizedHashMapEnabled = sqlContext.getConf( - "spark.sql.codegen.aggregate.map.vectorized.enable", "false") == "true" + isVectorizedHashMapEnabled = sqlContext.conf.enableVectorizedHashMap } } @@ -564,12 +576,8 @@ case class HashAggregateExec( val initAgg = ctx.addMutableState(CodeGenerator.JAVA_BOOLEAN, "initAgg") if (sqlContext.conf.enableTwoLevelAggMap) { enableTwoLevelHashMap(ctx) - } else { - sqlContext.getConf("spark.sql.codegen.aggregate.map.vectorized.enable", null) match { - case "true" => - logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") - case _ => - } + } else if (sqlContext.conf.enableVectorizedHashMap) { + logWarning("Two level hashmap is disabled but vectorized hashmap is enabled.") } val bitMaxCapacity = sqlContext.conf.fastHashAggregateRowMaxCapacityBit @@ -717,7 +725,7 @@ case class HashAggregateExec( $initAgg = true; long $beforeAgg = System.nanoTime(); $doAggFuncName(); - $aggTime.add((System.nanoTime() - $beforeAgg) / 1000000); + $aggTime.add((System.nanoTime() - $beforeAgg) / $NANOS_PER_MILLIS); } // output the result @@ -729,10 +737,11 @@ case class HashAggregateExec( private def doConsumeWithKeys(ctx: CodegenContext, input: Seq[ExprCode]): String = { // create grouping key val unsafeRowKeyCode = GenerateUnsafeProjection.createCode( - ctx, groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) + ctx, bindReferences[Expression](groupingExpressions, child.output)) val fastRowKeys = ctx.generateExpressions( - groupingExpressions.map(e => BindReferences.bindReference[Expression](e, child.output))) + bindReferences[Expression](groupingExpressions, child.output)) val unsafeRowKeys = unsafeRowKeyCode.value + val unsafeRowKeyHash = ctx.freshName("unsafeRowKeyHash") val unsafeRowBuffer = ctx.freshName("unsafeRowAggBuffer") val fastRowBuffer = ctx.freshName("fastAggBuffer") @@ -746,13 +755,6 @@ case class HashAggregateExec( } } - // generate hash code for key - // SPARK-24076: HashAggregate uses the same hash algorithm on the same expressions - // as ShuffleExchange, it may lead to bad hash conflict when shuffle.partitions=8192*n, - // pick a different seed to avoid this conflict - val hashExpr = Murmur3Hash(groupingExpressions, 48) - val hashEval = BindReferences.bindReference(hashExpr, child.output).genCode(ctx) - val (checkFallbackForGeneratedHashMap, checkFallbackForBytesToBytesMap, resetCounter, incCounter) = if (testFallbackStartsAt.isDefined) { val countTerm = ctx.addMutableState(CodeGenerator.JAVA_INT, "fallbackCounter") @@ -762,15 +764,17 @@ case class HashAggregateExec( ("true", "true", "", "") } + val oomeClassName = classOf[SparkOutOfMemoryError].getName + val findOrInsertRegularHashMap: String = s""" |// generate grouping key |${unsafeRowKeyCode.code} - |${hashEval.code} + |int $unsafeRowKeyHash = ${unsafeRowKeyCode.value}.hashCode(); |if ($checkFallbackForBytesToBytesMap) { | // try to get the buffer from hash map | $unsafeRowBuffer = - | $hashMapTerm.getAggregationBufferFromUnsafeRow($unsafeRowKeys, ${hashEval.value}); + | $hashMapTerm.getAggregationBufferFromUnsafeRow($unsafeRowKeys, $unsafeRowKeyHash); |} |// Can't allocate buffer from the hash map. Spill the map and fallback to sort-based |// aggregation after processing all input rows. @@ -784,10 +788,10 @@ case class HashAggregateExec( | // the hash map had be spilled, it should have enough memory now, | // try to allocate buffer again. | $unsafeRowBuffer = $hashMapTerm.getAggregationBufferFromUnsafeRow( - | $unsafeRowKeys, ${hashEval.value}); + | $unsafeRowKeys, $unsafeRowKeyHash); | if ($unsafeRowBuffer == null) { | // failed to allocate the first page - | throw new OutOfMemoryError("No enough memory for aggregation"); + | throw new $oomeClassName("No enough memory for aggregation"); | } |} """.stripMargin @@ -822,7 +826,7 @@ case class HashAggregateExec( val updateRowInRegularHashMap: String = { ctx.INPUT_ROW = unsafeRowBuffer - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) + val boundUpdateExpr = bindReferences(updateExpr, inputAttr) val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) val effectiveCodes = subExprs.codes.mkString("\n") val unsafeRowBufferEvals = ctx.withSubExprEliminationExprs(subExprs.states) { @@ -844,33 +848,47 @@ case class HashAggregateExec( val updateRowInHashMap: String = { if (isFastHashMapEnabled) { - ctx.INPUT_ROW = fastRowBuffer - val boundUpdateExpr = updateExpr.map(BindReferences.bindReference(_, inputAttr)) - val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) - val effectiveCodes = subExprs.codes.mkString("\n") - val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { - boundUpdateExpr.map(_.genCode(ctx)) - } - val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) => - val dt = updateExpr(i).dataType - CodeGenerator.updateColumn( - fastRowBuffer, dt, i, ev, updateExpr(i).nullable, isVectorizedHashMapEnabled) - } + if (isVectorizedHashMapEnabled) { + ctx.INPUT_ROW = fastRowBuffer + val boundUpdateExpr = bindReferences(updateExpr, inputAttr) + val subExprs = ctx.subexpressionEliminationForWholeStageCodegen(boundUpdateExpr) + val effectiveCodes = subExprs.codes.mkString("\n") + val fastRowEvals = ctx.withSubExprEliminationExprs(subExprs.states) { + boundUpdateExpr.map(_.genCode(ctx)) + } + val updateFastRow = fastRowEvals.zipWithIndex.map { case (ev, i) => + val dt = updateExpr(i).dataType + CodeGenerator.updateColumn( + fastRowBuffer, dt, i, ev, updateExpr(i).nullable, isVectorized = true) + } - // If fast hash map is on, we first generate code to update row in fast hash map, if the - // previous loop up hit fast hash map. Otherwise, update row in regular hash map. - s""" - |if ($fastRowBuffer != null) { - | // common sub-expressions - | $effectiveCodes - | // evaluate aggregate function - | ${evaluateVariables(fastRowEvals)} - | // update fast row - | ${updateFastRow.mkString("\n").trim} - |} else { - | $updateRowInRegularHashMap - |} - """.stripMargin + // If vectorized fast hash map is on, we first generate code to update row + // in vectorized fast hash map, if the previous loop up hit vectorized fast hash map. + // Otherwise, update row in regular hash map. + s""" + |if ($fastRowBuffer != null) { + | // common sub-expressions + | $effectiveCodes + | // evaluate aggregate function + | ${evaluateVariables(fastRowEvals)} + | // update fast row + | ${updateFastRow.mkString("\n").trim} + |} else { + | $updateRowInRegularHashMap + |} + """.stripMargin + } else { + // If row-based hash map is on and the previous loop up hit fast hash map, + // we reuse regular hash buffer to update row of fast hash map. + // Otherwise, update row in regular hash map. + s""" + |// Updates the proper row buffer + |if ($fastRowBuffer != null) { + | $unsafeRowBuffer = $fastRowBuffer; + |} + |$updateRowInRegularHashMap + """.stripMargin + } } else { updateRowInRegularHashMap } @@ -905,18 +923,18 @@ case class HashAggregateExec( """ } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions testFallbackStartsAt match { case None => - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"HashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala index 66955b8ef723c..151da241144be 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/ObjectHashAggregateExec.scala @@ -17,15 +17,17 @@ package org.apache.spark.sql.execution.aggregate +import java.util.concurrent.TimeUnit._ + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.Utils /** * A hash-based aggregate operator that supports [[TypedImperativeAggregate]] functions that may @@ -77,7 +79,7 @@ case class ObjectHashAggregateExec( override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "aggregate time") + "aggTime" -> SQLMetrics.createTimingMetric(sparkContext, "time in aggregation build") ) override def output: Seq[Attribute] = resultExpressions.map(_.toAttribute) @@ -132,20 +134,20 @@ case class ObjectHashAggregateExec( aggregationIterator } } - aggTime += (System.nanoTime() - beforeAgg) / 1000000 + aggTime += NANOSECONDS.toMillis(System.nanoTime() - beforeAgg) res } } - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"ObjectHashAggregate(keys=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala index fc87de2c52e41..7ab6ecc08a7bc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/SortAggregateExec.scala @@ -23,9 +23,9 @@ import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.util.Utils /** * Sort-based aggregate operator. @@ -107,16 +107,16 @@ case class SortAggregateExec( } } - override def simpleString: String = toString(verbose = false) + override def simpleString(maxFields: Int): String = toString(verbose = false, maxFields) - override def verboseString: String = toString(verbose = true) + override def verboseString(maxFields: Int): String = toString(verbose = true, maxFields) - private def toString(verbose: Boolean): String = { + private def toString(verbose: Boolean, maxFields: Int): String = { val allAggregateExpressions = aggregateExpressions - val keyString = Utils.truncatedString(groupingExpressions, "[", ", ", "]") - val functionString = Utils.truncatedString(allAggregateExpressions, "[", ", ", "]") - val outputString = Utils.truncatedString(output, "[", ", ", "]") + val keyString = truncatedString(groupingExpressions, "[", ", ", "]", maxFields) + val functionString = truncatedString(allAggregateExpressions, "[", ", ", "]", maxFields) + val outputString = truncatedString(output, "[", ", ", "]", maxFields) if (verbose) { s"SortAggregate(key=$keyString, functions=$functionString, output=$outputString)" } else { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index 72505f7fac0c6..6dc64657ebf1f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -206,7 +206,9 @@ class TungstenAggregationIterator( buffer = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) if (buffer == null) { // failed to allocate the first page + // scalastyle:off throwerror throw new SparkOutOfMemoryError("No enough memory for aggregation") + // scalastyle:on throwerror } } processRow(buffer, newInput) @@ -385,7 +387,7 @@ class TungstenAggregationIterator( metrics.incPeakExecutionMemory(maxMemory) // Updating average hashmap probe - avgHashProbe.set(hashMap.getAverageProbesPerLookup()) + avgHashProbe.set(hashMap.getAvgHashProbeBucketListIterations) }) /////////////////////////////////////////////////////////////////////////// diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala index 6d44890704f49..ea44c6013b7d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TypedAggregateExpression.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.aggregate -import scala.language.existentials - import org.apache.spark.sql.Encoder import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedDeserializer @@ -38,18 +36,14 @@ object TypedAggregateExpression { val bufferSerializer = bufferEncoder.namedExpressions val outputEncoder = encoderFor[OUT] - val outputType = if (outputEncoder.flat) { - outputEncoder.schema.head.dataType - } else { - outputEncoder.schema - } + val outputType = outputEncoder.objSerializer.dataType - // Checks if the buffer object is simple, i.e. the buffer encoder is flat and the serializer - // expression is an alias of `BoundReference`, which means the buffer object doesn't need - // serialization. + // Checks if the buffer object is simple, i.e. the `BUF` type is not serialized as struct + // and the serializer expression is an alias of `BoundReference`, which means the buffer + // object doesn't need serialization. val isSimpleBuffer = { bufferSerializer.head match { - case Alias(_: BoundReference, _) if bufferEncoder.flat => true + case Alias(_: BoundReference, _) if !bufferEncoder.isSerializedAsStruct => true case _ => false } } @@ -57,9 +51,8 @@ object TypedAggregateExpression { // If the buffer object is simple, use `SimpleTypedAggregateExpression`, which supports whole // stage codegen. if (isSimpleBuffer) { - val bufferDeserializer = UnresolvedDeserializer( - bufferEncoder.deserializer, - bufferSerializer.map(_.toAttribute)) + val bufferAttrs = bufferSerializer.map(_.toAttribute) + val bufferDeserializer = UnresolvedDeserializer(bufferEncoder.deserializer, bufferAttrs) SimpleTypedAggregateExpression( aggregator.asInstanceOf[Aggregator[Any, Any, Any]], @@ -67,11 +60,12 @@ object TypedAggregateExpression { None, None, bufferSerializer, + bufferAttrs.map(_.asInstanceOf[AttributeReference]), bufferDeserializer, outputEncoder.serializer, outputEncoder.deserializer.dataType, outputType, - !outputEncoder.flat || outputEncoder.schema.head.nullable) + outputEncoder.objSerializer.nullable) } else { ComplexTypedAggregateExpression( aggregator.asInstanceOf[Aggregator[Any, Any, Any]], @@ -80,9 +74,9 @@ object TypedAggregateExpression { None, bufferSerializer, bufferEncoder.resolveAndBind().deserializer, - outputEncoder.serializer, + outputEncoder.objSerializer, outputType, - !outputEncoder.flat || outputEncoder.schema.head.nullable) + outputEncoder.objSerializer.nullable) } } } @@ -122,7 +116,8 @@ case class SimpleTypedAggregateExpression( inputDeserializer: Option[Expression], inputClass: Option[Class[_]], inputSchema: Option[StructType], - bufferSerializer: Seq[NamedExpression], + bufferSerializer: Seq[Expression], + aggBufferAttributes: Seq[AttributeReference], bufferDeserializer: Expression, outputSerializer: Seq[Expression], outputExternalType: DataType, @@ -132,7 +127,10 @@ case class SimpleTypedAggregateExpression( override lazy val deterministic: Boolean = true - override def children: Seq[Expression] = inputDeserializer.toSeq :+ bufferDeserializer + override def children: Seq[Expression] = { + inputDeserializer.toSeq ++ bufferSerializer ++ aggBufferAttributes ++ + Seq(bufferDeserializer) ++ outputSerializer + } override lazy val resolved: Boolean = inputDeserializer.isDefined && childrenResolved @@ -143,9 +141,6 @@ case class SimpleTypedAggregateExpression( private def bufferExternalType = bufferDeserializer.dataType - override lazy val aggBufferAttributes: Seq[AttributeReference] = - bufferSerializer.map(_.toAttribute.asInstanceOf[AttributeReference]) - private def serializeToBuffer(expr: Expression): Seq[Expression] = { bufferSerializer.map(_.transform { case _: BoundReference => expr @@ -215,9 +210,9 @@ case class ComplexTypedAggregateExpression( inputDeserializer: Option[Expression], inputClass: Option[Class[_]], inputSchema: Option[StructType], - bufferSerializer: Seq[NamedExpression], + bufferSerializer: Seq[Expression], bufferDeserializer: Expression, - outputSerializer: Seq[Expression], + outputSerializer: Expression, dataType: DataType, nullable: Boolean, mutableAggBufferOffset: Int = 0, @@ -226,7 +221,9 @@ case class ComplexTypedAggregateExpression( override lazy val deterministic: Boolean = true - override def children: Seq[Expression] = inputDeserializer.toSeq + override def children: Seq[Expression] = { + inputDeserializer.toSeq ++ bufferSerializer :+ bufferDeserializer :+ outputSerializer + } override lazy val resolved: Boolean = inputDeserializer.isDefined && childrenResolved @@ -249,13 +246,7 @@ case class ComplexTypedAggregateExpression( aggregator.merge(buffer, input) } - private lazy val resultObjToRow = dataType match { - case _: StructType => - UnsafeProjection.create(CreateStruct(outputSerializer)) - case _ => - assert(outputSerializer.length == 1) - UnsafeProjection.create(outputSerializer.head) - } + private lazy val resultObjToRow = UnsafeProjection.create(outputSerializer) override def eval(buffer: Any): Any = { val resultObj = aggregator.finish(buffer) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala index 2bf6a58b55658..1a6f4acb63521 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowConverters.scala @@ -31,10 +31,10 @@ import org.apache.arrow.vector.ipc.message.{ArrowRecordBatch, MessageSerializer} import org.apache.spark.TaskContext import org.apache.spark.api.java.JavaRDD import org.apache.spark.network.util.JavaUtils -import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.{ByteBufferOutputStream, Utils} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala index 8dd484af6e908..6147d6fefd52a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/arrow/ArrowWriter.scala @@ -25,6 +25,7 @@ import org.apache.arrow.vector.complex._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils object ArrowWriter { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala index 4cd2e788ade07..7204548181f6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicPhysicalOperators.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution +import java.util.concurrent.TimeUnit._ + import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration.Duration @@ -24,10 +26,11 @@ import org.apache.spark.{InterruptibleIterator, Partition, SparkContext, TaskCon import org.apache.spark.rdd.{EmptyRDD, PartitionwiseSampledRDD, RDD} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution.metric.SQLMetrics -import org.apache.spark.sql.types.LongType +import org.apache.spark.sql.types.{LongType, StructType} import org.apache.spark.util.ThreadUtils import org.apache.spark.util.random.{BernoulliCellSampler, PoissonSampler} @@ -56,7 +59,7 @@ case class ProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) } override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { - val exprs = projectList.map(x => BindReferences.bindReference[Expression](x, child.output)) + val exprs = bindReferences[Expression](projectList, child.output) val resultVars = exprs.map(_.genCode(ctx)) // Evaluation of non-deterministic expressions can't be deferred. val nonDeterministicAttrs = projectList.filterNot(_.deterministic).map(_.toAttribute) @@ -452,8 +455,15 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) val localIdx = ctx.freshName("localIdx") val localEnd = ctx.freshName("localEnd") - val shouldStop = if (parent.needStopCheck) { - s"if (shouldStop()) { $nextIndex = $value + ${step}L; return; }" + val stopCheck = if (parent.needStopCheck) { + s""" + |if (shouldStop()) { + | $nextIndex = $value + ${step}L; + | $numOutput.add($localIdx + 1); + | $inputMetrics.incRecordsRead($localIdx + 1); + | return; + |} + """.stripMargin } else { "// shouldStop check is eliminated" } @@ -506,8 +516,6 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | $numElementsTodo = 0; | if ($nextBatchTodo == 0) break; | } - | $numOutput.add($nextBatchTodo); - | $inputMetrics.incRecordsRead($nextBatchTodo); | $batchEnd += $nextBatchTodo * ${step}L; | } | @@ -515,9 +523,11 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) | for (int $localIdx = 0; $localIdx < $localEnd; $localIdx++) { | long $value = ((long)$localIdx * ${step}L) + $nextIndex; | ${consume(ctx, Seq(ev))} - | $shouldStop + | $stopCheck | } | $nextIndex = $batchEnd; + | $numOutput.add($localEnd); + | $inputMetrics.incRecordsRead($localEnd); | $taskContext.killTaskIfInterrupted(); | } """.stripMargin @@ -579,7 +589,9 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) } } - override def simpleString: String = s"Range ($start, $end, step=$step, splits=$numSlices)" + override def simpleString(maxFields: Int): String = { + s"Range ($start, $end, step=$step, splits=$numSlices)" + } } /** @@ -589,9 +601,20 @@ case class RangeExec(range: org.apache.spark.sql.catalyst.plans.logical.Range) * [[org.apache.spark.sql.catalyst.plans.logical.Union.maxRowsPerPartition]]. */ case class UnionExec(children: Seq[SparkPlan]) extends SparkPlan { - override def output: Seq[Attribute] = - children.map(_.output).transpose.map(attrs => - attrs.head.withNullability(attrs.exists(_.nullable))) + // updating nullability to make all the children consistent + override def output: Seq[Attribute] = { + children.map(_.output).transpose.map { attrs => + val firstAttr = attrs.head + val nullable = attrs.exists(_.nullable) + val newDt = attrs.map(_.dataType).reduce(StructType.merge) + if (firstAttr.dataType == newDt) { + firstAttr.withNullability(nullable) + } else { + AttributeReference(firstAttr.name, newDt, nullable, firstAttr.metadata)( + firstAttr.exprId, firstAttr.qualifier) + } + } + } protected override def doExecute(): RDD[InternalRow] = sparkContext.union(children.map(_.execute())) @@ -648,19 +671,28 @@ object CoalesceExec { } /** - * Physical plan for a subquery. + * Parent class for different types of subquery plans */ -case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { - - override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), - "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)")) +abstract class BaseSubqueryExec extends SparkPlan { + def name: String + def child: SparkPlan override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = child.outputPartitioning override def outputOrdering: Seq[SortOrder] = child.outputOrdering +} + +/** + * Physical plan for a subquery. + */ +case class SubqueryExec(name: String, child: SparkPlan) + extends BaseSubqueryExec with UnaryExecNode { + + override lazy val metrics = Map( + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect")) @transient private lazy val relationFuture: Future[Array[InternalRow]] = { @@ -674,7 +706,7 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { // Note that we use .executeCollect() because we don't want to convert data to Scala types val rows: Array[InternalRow] = child.executeCollect() val beforeBuild = System.nanoTime() - longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 + longMetric("collectTime") += NANOSECONDS.toMillis(beforeBuild - beforeCollect) val dataSize = rows.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum longMetric("dataSize") += dataSize @@ -684,6 +716,10 @@ case class SubqueryExec(name: String, child: SparkPlan) extends UnaryExecNode { }(SubqueryExec.executionContext) } + protected override def doCanonicalize(): SparkPlan = { + SubqueryExec("Subquery", child.canonicalized) + } + protected override def doPrepare(): Unit = { relationFuture } @@ -701,3 +737,23 @@ object SubqueryExec { private[execution] val executionContext = ExecutionContext.fromExecutorService( ThreadUtils.newDaemonCachedThreadPool("subquery", 16)) } + +/** + * A wrapper for reused [[BaseSubqueryExec]]. + */ +case class ReusedSubqueryExec(child: BaseSubqueryExec) + extends BaseSubqueryExec with LeafExecNode { + + override def name: String = child.name + + override def output: Seq[Attribute] = child.output + override def doCanonicalize(): SparkPlan = child.canonicalized + override def outputOrdering: Seq[SortOrder] = child.outputOrdering + override def outputPartitioning: Partitioning = child.outputPartitioning + + protected override def doPrepare(): Unit = child.prepare() + + protected override def doExecute(): RDD[InternalRow] = child.execute() + + override def executeCollect(): Array[InternalRow] = child.executeCollect() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala index 1a8fbaca53f59..1de2b6e0a85de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala @@ -26,10 +26,11 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.{LongAccumulator, Utils} +import org.apache.spark.util.LongAccumulator /** @@ -47,10 +48,15 @@ case class CachedRDDBuilder( batchSize: Int, storageLevel: StorageLevel, @transient cachedPlan: SparkPlan, - tableName: Option[String])( - @transient private var _cachedColumnBuffers: RDD[CachedBatch] = null) { + tableName: Option[String]) { + + @transient @volatile private var _cachedColumnBuffers: RDD[CachedBatch] = null val sizeInBytesStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator + val rowCountStats: LongAccumulator = cachedPlan.sqlContext.sparkContext.longAccumulator + + val cachedName = tableName.map(n => s"In-memory table $n") + .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024)) def cachedColumnBuffers: RDD[CachedBatch] = { if (_cachedColumnBuffers == null) { @@ -63,7 +69,7 @@ case class CachedRDDBuilder( _cachedColumnBuffers } - def clearCache(blocking: Boolean = true): Unit = { + def clearCache(blocking: Boolean = false): Unit = { if (_cachedColumnBuffers != null) { synchronized { if (_cachedColumnBuffers != null) { @@ -74,14 +80,8 @@ case class CachedRDDBuilder( } } - def withCachedPlan(cachedPlan: SparkPlan): CachedRDDBuilder = { - new CachedRDDBuilder( - useCompression, - batchSize, - storageLevel, - cachedPlan = cachedPlan, - tableName - )(_cachedColumnBuffers) + def isCachedColumnBuffersLoaded: Boolean = { + _cachedColumnBuffers != null } private def buildBuffers(): RDD[CachedBatch] = { @@ -120,6 +120,7 @@ case class CachedRDDBuilder( } sizeInBytesStats.add(totalSize) + rowCountStats.add(rowCount) val stats = InternalRow.fromSeq( columnBuilders.flatMap(_.columnStats.collectedStatistics)) @@ -132,9 +133,7 @@ case class CachedRDDBuilder( } }.persist(storageLevel) - cached.setName( - tableName.map(n => s"In-memory table $n") - .getOrElse(StringUtils.abbreviate(cachedPlan.toString, 1024))) + cached.setName(cachedName) cached } } @@ -148,66 +147,82 @@ object InMemoryRelation { child: SparkPlan, tableName: Option[String], logicalPlan: LogicalPlan): InMemoryRelation = { - val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName)() - new InMemoryRelation(child.output, cacheBuilder)( - statsOfPlanToCache = logicalPlan.stats, outputOrdering = logicalPlan.outputOrdering) + val cacheBuilder = CachedRDDBuilder(useCompression, batchSize, storageLevel, child, tableName) + val relation = new InMemoryRelation(child.output, cacheBuilder, logicalPlan.outputOrdering) + relation.statsOfPlanToCache = logicalPlan.stats + relation } def apply(cacheBuilder: CachedRDDBuilder, logicalPlan: LogicalPlan): InMemoryRelation = { - new InMemoryRelation(cacheBuilder.cachedPlan.output, cacheBuilder)( - statsOfPlanToCache = logicalPlan.stats, outputOrdering = logicalPlan.outputOrdering) + val relation = new InMemoryRelation( + cacheBuilder.cachedPlan.output, cacheBuilder, logicalPlan.outputOrdering) + relation.statsOfPlanToCache = logicalPlan.stats + relation + } + + def apply( + output: Seq[Attribute], + cacheBuilder: CachedRDDBuilder, + outputOrdering: Seq[SortOrder], + statsOfPlanToCache: Statistics): InMemoryRelation = { + val relation = InMemoryRelation(output, cacheBuilder, outputOrdering) + relation.statsOfPlanToCache = statsOfPlanToCache + relation } } case class InMemoryRelation( output: Seq[Attribute], - @transient cacheBuilder: CachedRDDBuilder)( - statsOfPlanToCache: Statistics, + @transient cacheBuilder: CachedRDDBuilder, override val outputOrdering: Seq[SortOrder]) extends logical.LeafNode with MultiInstanceRelation { + @volatile var statsOfPlanToCache: Statistics = null + override protected def innerChildren: Seq[SparkPlan] = Seq(cachedPlan) override def doCanonicalize(): logical.LogicalPlan = copy(output = output.map(QueryPlan.normalizeExprId(_, cachedPlan.output)), - cacheBuilder)( - statsOfPlanToCache, + cacheBuilder, outputOrdering) - override def producedAttributes: AttributeSet = outputSet - @transient val partitionStatistics = new PartitionStatistics(output) def cachedPlan: SparkPlan = cacheBuilder.cachedPlan + private[sql] def updateStats( + rowCount: Long, + newColStats: Map[Attribute, ColumnStat]): Unit = this.synchronized { + val newStats = statsOfPlanToCache.copy( + rowCount = Some(rowCount), + attributeStats = AttributeMap((statsOfPlanToCache.attributeStats ++ newColStats).toSeq) + ) + statsOfPlanToCache = newStats + } + override def computeStats(): Statistics = { - if (cacheBuilder.sizeInBytesStats.value == 0L) { + if (!cacheBuilder.isCachedColumnBuffersLoaded) { // Underlying columnar RDD hasn't been materialized, use the stats from the plan to cache. - // Note that we should drop the hint info here. We may cache a plan whose root node is a hint - // node. When we lookup the cache with a semantically same plan without hint info, the plan - // returned by cache lookup should not have hint info. If we lookup the cache with a - // semantically same plan with a different hint info, `CacheManager.useCachedData` will take - // care of it and retain the hint info in the lookup input plan. - statsOfPlanToCache.copy(hints = HintInfo()) + statsOfPlanToCache } else { - Statistics(sizeInBytes = cacheBuilder.sizeInBytesStats.value.longValue) + statsOfPlanToCache.copy( + sizeInBytes = cacheBuilder.sizeInBytesStats.value.longValue, + rowCount = Some(cacheBuilder.rowCountStats.value.longValue) + ) } } - def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = { - InMemoryRelation(newOutput, cacheBuilder)(statsOfPlanToCache, outputOrdering) - } + def withOutput(newOutput: Seq[Attribute]): InMemoryRelation = + InMemoryRelation(newOutput, cacheBuilder, outputOrdering, statsOfPlanToCache) override def newInstance(): this.type = { - new InMemoryRelation( + InMemoryRelation( output.map(_.newInstance()), - cacheBuilder)( - statsOfPlanToCache, - outputOrdering).asInstanceOf[this.type] + cacheBuilder, + outputOrdering, + statsOfPlanToCache).asInstanceOf[this.type] } - override protected def otherCopyArgs: Seq[AnyRef] = Seq(statsOfPlanToCache) - - override def simpleString: String = - s"InMemoryRelation [${Utils.truncatedString(output, ", ")}], ${cacheBuilder.storageLevel}" + override def simpleString(maxFields: Int): String = + s"InMemoryRelation [${truncatedString(output, ", ", maxFields)}], ${cacheBuilder.storageLevel}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala index 196d057c2de1b..3566ab1aa5a33 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryTableScanExec.scala @@ -24,7 +24,8 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.physical.Partitioning -import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan, WholeStageCodegenExec} +import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.execution.vectorized._ import org.apache.spark.sql.types._ import org.apache.spark.sql.vectorized.{ColumnarBatch, ColumnVector} @@ -34,7 +35,19 @@ case class InMemoryTableScanExec( attributes: Seq[Attribute], predicates: Seq[Expression], @transient relation: InMemoryRelation) - extends LeafExecNode with ColumnarBatchScan { + extends LeafExecNode { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + override val nodeName: String = { + relation.cacheBuilder.tableName match { + case Some(_) => + "Scan " + relation.cacheBuilder.cachedName + case _ => + super.nodeName + } + } override protected def innerChildren: Seq[QueryPlan[_]] = Seq(relation) ++ super.innerChildren @@ -56,7 +69,7 @@ case class InMemoryTableScanExec( * If true, get data from ColumnVector in ColumnarBatch, which are generally faster. * If false, get data from UnsafeRow build from CachedBatch */ - override val supportsBatch: Boolean = { + override val supportsColumnar: Boolean = { // In the initial implementation, for ease of review // support only primitive data types and # of fields is less than wholeStageMaxNumFields conf.cacheVectorizedReaderEnabled && relation.schema.fields.forall(f => f.dataType match { @@ -66,11 +79,6 @@ case class InMemoryTableScanExec( }) && !WholeStageCodegenExec.isTooManyFields(conf, relation.schema) } - // TODO: revisit this. Shall we always turn off whole stage codegen if the output data are rows? - override def supportCodegen: Boolean = supportsBatch - - override protected def needsUnsafeRowConversion: Boolean = false - private val columnIndices = attributes.map(a => relation.output.map(o => o.exprId).indexOf(a.exprId)).toArray @@ -101,59 +109,61 @@ case class InMemoryTableScanExec( columnarBatch } + private lazy val columnarInputRDD: RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + val buffers = filteredCachedBatches() + val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled + buffers + .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) + .map(b => { + numOutputRows += b.numRows() + b + }) + } + private lazy val inputRDD: RDD[InternalRow] = { val buffers = filteredCachedBatches() val offHeapColumnVectorEnabled = conf.offHeapColumnVectorEnabled - if (supportsBatch) { - // HACK ALERT: This is actually an RDD[ColumnarBatch]. - // We're taking advantage of Scala's type erasure here to pass these batches along. - buffers - .map(createAndDecompressColumn(_, offHeapColumnVectorEnabled)) - .asInstanceOf[RDD[InternalRow]] - } else { - val numOutputRows = longMetric("numOutputRows") + val numOutputRows = longMetric("numOutputRows") - if (enableAccumulatorsForTest) { - readPartitions.setValue(0) - readBatches.setValue(0) - } + if (enableAccumulatorsForTest) { + readPartitions.setValue(0) + readBatches.setValue(0) + } - // Using these variables here to avoid serialization of entire objects (if referenced - // directly) within the map Partitions closure. - val relOutput: AttributeSeq = relation.output - - filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => - // Find the ordinals and data types of the requested columns. - val (requestedColumnIndices, requestedColumnDataTypes) = - attributes.map { a => - relOutput.indexOf(a.exprId) -> a.dataType - }.unzip - - // update SQL metrics - val withMetrics = cachedBatchIterator.map { batch => - if (enableAccumulatorsForTest) { - readBatches.add(1) - } - numOutputRows += batch.numRows - batch + // Using these variables here to avoid serialization of entire objects (if referenced + // directly) within the map Partitions closure. + val relOutput: AttributeSeq = relation.output + + filteredCachedBatches().mapPartitionsInternal { cachedBatchIterator => + // Find the ordinals and data types of the requested columns. + val (requestedColumnIndices, requestedColumnDataTypes) = + attributes.map { a => + relOutput.indexOf(a.exprId) -> a.dataType + }.unzip + + // update SQL metrics + val withMetrics = cachedBatchIterator.map { batch => + if (enableAccumulatorsForTest) { + readBatches.add(1) } + numOutputRows += batch.numRows + batch + } - val columnTypes = requestedColumnDataTypes.map { - case udt: UserDefinedType[_] => udt.sqlType - case other => other - }.toArray - val columnarIterator = GenerateColumnAccessor.generate(columnTypes) - columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) - if (enableAccumulatorsForTest && columnarIterator.hasNext) { - readPartitions.add(1) - } - columnarIterator + val columnTypes = requestedColumnDataTypes.map { + case udt: UserDefinedType[_] => udt.sqlType + case other => other + }.toArray + val columnarIterator = GenerateColumnAccessor.generate(columnTypes) + columnarIterator.initialize(withMetrics, columnTypes, requestedColumnIndices.toArray) + if (enableAccumulatorsForTest && columnarIterator.hasNext) { + readPartitions.add(1) } + columnarIterator } } - override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) - override def output: Seq[Attribute] = attributes private def updateAttribute(expr: Expression): Expression = { @@ -196,7 +206,7 @@ case class InMemoryTableScanExec( } // Returned filter predicate should return false iff it is impossible for the input expression - // to evaluate to `true' based on statistics collected about this partition batch. + // to evaluate to `true` based on statistics collected about this partition batch. @transient lazy val buildFilter: PartialFunction[Expression, Expression] = { case And(lhs: Expression, rhs: Expression) if buildFilter.isDefinedAt(lhs) || buildFilter.isDefinedAt(rhs) => @@ -239,6 +249,34 @@ case class InMemoryTableScanExec( if list.forall(ExtractableLiteral.unapply(_).isDefined) && list.nonEmpty => list.map(l => statsFor(a).lowerBound <= l.asInstanceOf[Literal] && l.asInstanceOf[Literal] <= statsFor(a).upperBound).reduce(_ || _) + + // This is an example to explain how it works, imagine that the id column stored as follows: + // __________________________________________ + // | Partition ID | lowerBound | upperBound | + // |--------------|------------|------------| + // | p1 | '1' | '9' | + // | p2 | '10' | '19' | + // | p3 | '20' | '29' | + // | p4 | '30' | '39' | + // | p5 | '40' | '49' | + // |______________|____________|____________| + // + // A filter: df.filter($"id".startsWith("2")). + // In this case it substr lowerBound and upperBound: + // ________________________________________________________________________________________ + // | Partition ID | lowerBound.substr(0, Length("2")) | upperBound.substr(0, Length("2")) | + // |--------------|-----------------------------------|-----------------------------------| + // | p1 | '1' | '9' | + // | p2 | '1' | '1' | + // | p3 | '2' | '2' | + // | p4 | '3' | '3' | + // | p5 | '4' | '4' | + // |______________|___________________________________|___________________________________| + // + // We can see that we only need to read p1 and p3. + case StartsWith(a: AttributeReference, ExtractableLiteral(l)) => + statsFor(a).lowerBound.substr(0, Length(l)) <= l && + l <= statsFor(a).upperBound.substr(0, Length(l)) } lazy val partitionFilters: Seq[Expression] = { @@ -259,8 +297,7 @@ case class InMemoryTableScanExec( } } - lazy val enableAccumulatorsForTest: Boolean = - sqlContext.getConf("spark.sql.inMemoryTableScanStatistics.enable", "false").toBoolean + lazy val enableAccumulatorsForTest: Boolean = sqlContext.conf.inMemoryTableScanStatisticsEnabled // Accumulators used for testing purposes lazy val readPartitions = sparkContext.longAccumulator @@ -304,10 +341,10 @@ case class InMemoryTableScanExec( } protected override def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - inputRDD - } + inputRDD + } + + protected override def doExecuteColumnar(): RDD[ColumnarBatch] = { + columnarInputRDD } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala index 93447a52097ce..5017893077922 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala @@ -17,17 +17,12 @@ package org.apache.spark.sql.execution.command -import scala.collection.mutable - import org.apache.spark.sql._ -import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} -import org.apache.spark.sql.catalyst.catalog.{CatalogColumnStat, CatalogStatistics, CatalogTableType} -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.ArrayData -import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTableType} +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.types._ @@ -42,33 +37,44 @@ case class AnalyzeColumnCommand( allColumns: Boolean) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { - require((columnNames.isDefined ^ allColumns), "Parameter `columnNames` or `allColumns` are " + + require(columnNames.isDefined ^ allColumns, "Parameter `columnNames` or `allColumns` are " + "mutually exclusive. Only one of them should be specified.") val sessionState = sparkSession.sessionState - val db = tableIdent.database.getOrElse(sessionState.catalog.getCurrentDatabase) - val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) - val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) - if (tableMeta.tableType == CatalogTableType.VIEW) { - throw new AnalysisException("ANALYZE TABLE is not supported on views.") - } - val sizeInBytes = CommandUtils.calculateTotalSize(sparkSession, tableMeta) - val relation = sparkSession.table(tableIdent).logicalPlan - val columnsToAnalyze = getColumnsToAnalyze(tableIdent, relation, columnNames, allColumns) - // Compute stats for the computed list of columns. - val (rowCount, newColStats) = - computeColumnStats(sparkSession, relation, columnsToAnalyze) + tableIdent.database match { + case Some(db) if db == sparkSession.sharedState.globalTempViewManager.database => + val plan = sessionState.catalog.getGlobalTempView(tableIdent.identifier).getOrElse { + throw new NoSuchTableException(db = db, table = tableIdent.identifier) + } + analyzeColumnInTempView(plan, sparkSession) + case Some(_) => + analyzeColumnInCatalog(sparkSession) + case None => + sessionState.catalog.getTempView(tableIdent.identifier) match { + case Some(tempView) => analyzeColumnInTempView(tempView, sparkSession) + case _ => analyzeColumnInCatalog(sparkSession) + } + } - // We also update table-level stats in order to keep them consistent with column-level stats. - val statistics = CatalogStatistics( - sizeInBytes = sizeInBytes, - rowCount = Some(rowCount), - // Newly computed column stats should override the existing ones. - colStats = tableMeta.stats.map(_.colStats).getOrElse(Map.empty) ++ newColStats) + Seq.empty[Row] + } - sessionState.catalog.alterTableStats(tableIdentWithDB, Some(statistics)) + private def analyzeColumnInCachedData(plan: LogicalPlan, sparkSession: SparkSession): Boolean = { + val cacheManager = sparkSession.sharedState.cacheManager + cacheManager.lookupCachedData(plan).map { cachedData => + val columnsToAnalyze = getColumnsToAnalyze( + tableIdent, cachedData.plan, columnNames, allColumns) + cacheManager.analyzeColumnCacheQuery(sparkSession, cachedData, columnsToAnalyze) + cachedData + }.isDefined + } - Seq.empty[Row] + private def analyzeColumnInTempView(plan: LogicalPlan, sparkSession: SparkSession): Unit = { + if (!analyzeColumnInCachedData(plan, sparkSession)) { + val catalog = sparkSession.sessionState.catalog + val db = tableIdent.database.getOrElse(catalog.getCurrentDatabase) + throw new NoSuchTableException(db = db, table = tableIdent.identifier) + } } private def getColumnsToAnalyze( @@ -95,80 +101,38 @@ case class AnalyzeColumnCommand( columnsToAnalyze } - /** - * Compute stats for the given columns. - * @return (row count, map from column name to CatalogColumnStats) - */ - private def computeColumnStats( - sparkSession: SparkSession, - relation: LogicalPlan, - columns: Seq[Attribute]): (Long, Map[String, CatalogColumnStat]) = { - val conf = sparkSession.sessionState.conf - - // Collect statistics per column. - // If no histogram is required, we run a job to compute basic column stats such as - // min, max, ndv, etc. Otherwise, besides basic column stats, histogram will also be - // generated. Currently we only support equi-height histogram. - // To generate an equi-height histogram, we need two jobs: - // 1. compute percentiles p(0), p(1/n) ... p((n-1)/n), p(1). - // 2. use the percentiles as value intervals of bins, e.g. [p(0), p(1/n)], - // [p(1/n), p(2/n)], ..., [p((n-1)/n), p(1)], and then count ndv in each bin. - // Basic column stats will be computed together in the second job. - val attributePercentiles = computePercentiles(columns, sparkSession, relation) - - // The first element in the result will be the overall row count, the following elements - // will be structs containing all column stats. - // The layout of each struct follows the layout of the ColumnStats. - val expressions = Count(Literal(1)).toAggregateExpression() +: - columns.map(statExprs(_, conf, attributePercentiles)) - - val namedExpressions = expressions.map(e => Alias(e, e.toString)()) - val statsRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExpressions, relation)) - .executedPlan.executeTake(1).head - - val rowCount = statsRow.getLong(0) - val columnStats = columns.zipWithIndex.map { case (attr, i) => - // according to `statExprs`, the stats struct always have 7 fields. - (attr.name, rowToColumnStat(statsRow.getStruct(i + 1, 7), attr, rowCount, - attributePercentiles.get(attr)).toCatalogColumnStat(attr.name, attr.dataType)) - }.toMap - (rowCount, columnStats) - } - - /** Computes percentiles for each attribute. */ - private def computePercentiles( - attributesToAnalyze: Seq[Attribute], - sparkSession: SparkSession, - relation: LogicalPlan): AttributeMap[ArrayData] = { - val attrsToGenHistogram = if (conf.histogramEnabled) { - attributesToAnalyze.filter(a => supportsHistogram(a.dataType)) + private def analyzeColumnInCatalog(sparkSession: SparkSession): Unit = { + val sessionState = sparkSession.sessionState + val tableMeta = sessionState.catalog.getTableMetadata(tableIdent) + if (tableMeta.tableType == CatalogTableType.VIEW) { + // Analyzes a catalog view if the view is cached + val plan = sparkSession.table(tableIdent.quotedString).logicalPlan + if (!analyzeColumnInCachedData(plan, sparkSession)) { + throw new AnalysisException("ANALYZE TABLE is not supported on views.") + } } else { - Nil - } - val attributePercentiles = mutable.HashMap[Attribute, ArrayData]() - if (attrsToGenHistogram.nonEmpty) { - val percentiles = (0 to conf.histogramNumBins) - .map(i => i.toDouble / conf.histogramNumBins).toArray + val sizeInBytes = CommandUtils.calculateTotalSize(sparkSession, tableMeta) + val relation = sparkSession.table(tableIdent).logicalPlan + val columnsToAnalyze = getColumnsToAnalyze(tableIdent, relation, columnNames, allColumns) - val namedExprs = attrsToGenHistogram.map { attr => - val aggFunc = - new ApproximatePercentile(attr, Literal(percentiles), Literal(conf.percentileAccuracy)) - val expr = aggFunc.toAggregateExpression() - Alias(expr, expr.toString)() - } + // Compute stats for the computed list of columns. + val (rowCount, newColStats) = + CommandUtils.computeColumnStats(sparkSession, relation, columnsToAnalyze) - val percentilesRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExprs, relation)) - .executedPlan.executeTake(1).head - attrsToGenHistogram.zipWithIndex.foreach { case (attr, i) => - val percentiles = percentilesRow.getArray(i) - // When there is no non-null value, `percentiles` is null. In such case, there is no - // need to generate histogram. - if (percentiles != null) { - attributePercentiles += attr -> percentiles - } + val newColCatalogStats = newColStats.map { + case (attr, columnStat) => + attr.name -> columnStat.toCatalogColumnStat(attr.name, attr.dataType) } + + // We also update table-level stats in order to keep them consistent with column-level stats. + val statistics = CatalogStatistics( + sizeInBytes = sizeInBytes, + rowCount = Some(rowCount), + // Newly computed column stats should override the existing ones. + colStats = tableMeta.stats.map(_.colStats).getOrElse(Map.empty) ++ newColCatalogStats) + + sessionState.catalog.alterTableStats(tableIdent, Some(statistics)) } - AttributeMap(attributePercentiles.toSeq) } /** Returns true iff the we support gathering column statistics on column of the given type. */ @@ -182,109 +146,4 @@ case class AnalyzeColumnCommand( case BinaryType | StringType => true case _ => false } - - /** Returns true iff the we support gathering histogram on column of the given type. */ - private def supportsHistogram(dataType: DataType): Boolean = dataType match { - case _: IntegralType => true - case _: DecimalType => true - case DoubleType | FloatType => true - case DateType => true - case TimestampType => true - case _ => false - } - - /** - * Constructs an expression to compute column statistics for a given column. - * - * The expression should create a single struct column with the following schema: - * distinctCount: Long, min: T, max: T, nullCount: Long, avgLen: Long, maxLen: Long, - * distinctCountsForIntervals: Array[Long] - * - * Together with [[rowToColumnStat]], this function is used to create [[ColumnStat]] and - * as a result should stay in sync with it. - */ - private def statExprs( - col: Attribute, - conf: SQLConf, - colPercentiles: AttributeMap[ArrayData]): CreateNamedStruct = { - def struct(exprs: Expression*): CreateNamedStruct = CreateStruct(exprs.map { expr => - expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() } - }) - val one = Literal(1, LongType) - - // the approximate ndv (num distinct value) should never be larger than the number of rows - val numNonNulls = if (col.nullable) Count(col) else Count(one) - val ndv = Least(Seq(HyperLogLogPlusPlus(col, conf.ndvMaxError), numNonNulls)) - val numNulls = Subtract(Count(one), numNonNulls) - val defaultSize = Literal(col.dataType.defaultSize, LongType) - val nullArray = Literal(null, ArrayType(LongType)) - - def fixedLenTypeStruct: CreateNamedStruct = { - val genHistogram = - supportsHistogram(col.dataType) && colPercentiles.contains(col) - val intervalNdvsExpr = if (genHistogram) { - ApproxCountDistinctForIntervals(col, - Literal(colPercentiles(col), ArrayType(col.dataType)), conf.ndvMaxError) - } else { - nullArray - } - // For fixed width types, avg size should be the same as max size. - struct(ndv, Cast(Min(col), col.dataType), Cast(Max(col), col.dataType), numNulls, - defaultSize, defaultSize, intervalNdvsExpr) - } - - col.dataType match { - case _: IntegralType => fixedLenTypeStruct - case _: DecimalType => fixedLenTypeStruct - case DoubleType | FloatType => fixedLenTypeStruct - case BooleanType => fixedLenTypeStruct - case DateType => fixedLenTypeStruct - case TimestampType => fixedLenTypeStruct - case BinaryType | StringType => - // For string and binary type, we don't compute min, max or histogram - val nullLit = Literal(null, col.dataType) - struct( - ndv, nullLit, nullLit, numNulls, - // Set avg/max size to default size if all the values are null or there is no value. - Coalesce(Seq(Ceil(Average(Length(col))), defaultSize)), - Coalesce(Seq(Cast(Max(Length(col)), LongType), defaultSize)), - nullArray) - case _ => - throw new AnalysisException("Analyzing column statistics is not supported for column " + - s"${col.name} of data type: ${col.dataType}.") - } - } - - /** Convert a struct for column stats (defined in `statExprs`) into [[ColumnStat]]. */ - private def rowToColumnStat( - row: InternalRow, - attr: Attribute, - rowCount: Long, - percentiles: Option[ArrayData]): ColumnStat = { - // The first 6 fields are basic column stats, the 7th is ndvs for histogram bins. - val cs = ColumnStat( - distinctCount = Option(BigInt(row.getLong(0))), - // for string/binary min/max, get should return null - min = Option(row.get(1, attr.dataType)), - max = Option(row.get(2, attr.dataType)), - nullCount = Option(BigInt(row.getLong(3))), - avgLen = Option(row.getLong(4)), - maxLen = Option(row.getLong(5)) - ) - if (row.isNullAt(6) || cs.nullCount.isEmpty) { - cs - } else { - val ndvs = row.getArray(6).toLongArray() - assert(percentiles.get.numElements() == ndvs.length + 1) - val endpoints = percentiles.get.toArray[Any](attr.dataType).map(_.toString.toDouble) - // Construct equi-height histogram - val bins = ndvs.zipWithIndex.map { case (ndv, i) => - HistogramBin(endpoints(i), endpoints(i + 1), ndv) - } - val nonNullRows = rowCount - cs.nullCount.get - val histogram = Histogram(nonNullRows.toDouble / ndvs.length, bins) - cs.copy(histogram = Some(histogram)) - } - } - } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala index 3076e919dd61f..67cfcebec187c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeTableCommand.scala @@ -35,19 +35,29 @@ case class AnalyzeTableCommand( val tableIdentWithDB = TableIdentifier(tableIdent.table, Some(db)) val tableMeta = sessionState.catalog.getTableMetadata(tableIdentWithDB) if (tableMeta.tableType == CatalogTableType.VIEW) { - throw new AnalysisException("ANALYZE TABLE is not supported on views.") - } - - // Compute stats for the whole table - val newTotalSize = CommandUtils.calculateTotalSize(sparkSession, tableMeta) - val newRowCount = - if (noscan) None else Some(BigInt(sparkSession.table(tableIdentWithDB).count())) + // Analyzes a catalog view if the view is cached + val table = sparkSession.table(tableIdent.quotedString) + val cacheManager = sparkSession.sharedState.cacheManager + if (cacheManager.lookupCachedData(table.logicalPlan).isDefined) { + if (!noscan) { + // To collect table stats, materializes an underlying columnar RDD + table.count() + } + } else { + throw new AnalysisException("ANALYZE TABLE is not supported on views.") + } + } else { + // Compute stats for the whole table + val newTotalSize = CommandUtils.calculateTotalSize(sparkSession, tableMeta) + val newRowCount = + if (noscan) None else Some(BigInt(sparkSession.table(tableIdentWithDB).count())) - // Update the metastore if the above statistics of the table are different from those - // recorded in the metastore. - val newStats = CommandUtils.compareAndGetNewStats(tableMeta.stats, newTotalSize, newRowCount) - if (newStats.isDefined) { - sessionState.catalog.alterTableStats(tableIdentWithDB, newStats) + // Update the metastore if the above statistics of the table are different from those + // recorded in the metastore. + val newStats = CommandUtils.compareAndGetNewStats(tableMeta.stats, newTotalSize, newRowCount) + if (newStats.isDefined) { + sessionState.catalog.alterTableStats(tableIdentWithDB, newStats) + } } Seq.empty[Row] 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 df71bc9effb3e..b644e6dc471d6 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 @@ -19,32 +19,37 @@ package org.apache.spark.sql.execution.command import java.net.URI +import scala.collection.mutable import scala.util.control.NonFatal import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.catalog.{CatalogStatistics, CatalogTable, CatalogTablePartition} +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} +import org.apache.spark.sql.catalyst.catalog.{CatalogColumnStat, CatalogStatistics, CatalogTable} +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.ArrayData +import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.datasources.{DataSourceUtils, InMemoryFileIndex} -import org.apache.spark.sql.internal.SessionState +import org.apache.spark.sql.internal.{SessionState, SQLConf} +import org.apache.spark.sql.types._ object CommandUtils extends Logging { /** Change statistics after changing data by commands. */ def updateTableStats(sparkSession: SparkSession, table: CatalogTable): Unit = { - if (table.stats.nonEmpty) { - val catalog = sparkSession.sessionState.catalog - if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { - val newTable = catalog.getTableMetadata(table.identifier) - val newSize = CommandUtils.calculateTotalSize(sparkSession, newTable) - val newStats = CatalogStatistics(sizeInBytes = newSize) - catalog.alterTableStats(table.identifier, Some(newStats)) - } else { - catalog.alterTableStats(table.identifier, None) - } + val catalog = sparkSession.sessionState.catalog + if (sparkSession.sessionState.conf.autoSizeUpdateEnabled) { + val newTable = catalog.getTableMetadata(table.identifier) + val newSize = CommandUtils.calculateTotalSize(sparkSession, newTable) + val newStats = CatalogStatistics(sizeInBytes = newSize) + catalog.alterTableStats(table.identifier, Some(newStats)) + } else if (table.stats.nonEmpty) { + catalog.alterTableStats(table.identifier, None) } } @@ -59,12 +64,10 @@ object CommandUtils extends Logging { val paths = partitions.map(x => new Path(x.storage.locationUri.get)) val stagingDir = sessionState.conf.getConfString("hive.exec.stagingdir", ".hive-staging") val pathFilter = new PathFilter with Serializable { - override def accept(path: Path): Boolean = { - DataSourceUtils.isDataPath(path) && !path.getName.startsWith(stagingDir) - } + override def accept(path: Path): Boolean = isDataPath(path, stagingDir) } val fileStatusSeq = InMemoryFileIndex.bulkListLeafFiles( - paths, sessionState.newHadoopConf(), pathFilter, spark) + paths, sessionState.newHadoopConf(), pathFilter, spark, areRootPaths = true) fileStatusSeq.flatMap(_._2.map(_.getLen)).sum } else { partitions.map { p => @@ -93,8 +96,7 @@ object CommandUtils extends Logging { val size = if (fileStatus.isDirectory) { fs.listStatus(path) .map { status => - if (!status.getPath.getName.startsWith(stagingDir) && - DataSourceUtils.isDataPath(path)) { + if (isDataPath(status.getPath, stagingDir)) { getPathSize(fs, status.getPath) } else { 0L @@ -153,4 +155,193 @@ object CommandUtils extends Logging { } newStats } + + /** + * Compute stats for the given columns. + * @return (row count, map from column name to CatalogColumnStats) + */ + private[sql] def computeColumnStats( + sparkSession: SparkSession, + relation: LogicalPlan, + columns: Seq[Attribute]): (Long, Map[Attribute, ColumnStat]) = { + val conf = sparkSession.sessionState.conf + + // Collect statistics per column. + // If no histogram is required, we run a job to compute basic column stats such as + // min, max, ndv, etc. Otherwise, besides basic column stats, histogram will also be + // generated. Currently we only support equi-height histogram. + // To generate an equi-height histogram, we need two jobs: + // 1. compute percentiles p(0), p(1/n) ... p((n-1)/n), p(1). + // 2. use the percentiles as value intervals of bins, e.g. [p(0), p(1/n)], + // [p(1/n), p(2/n)], ..., [p((n-1)/n), p(1)], and then count ndv in each bin. + // Basic column stats will be computed together in the second job. + val attributePercentiles = computePercentiles(columns, sparkSession, relation) + + // The first element in the result will be the overall row count, the following elements + // will be structs containing all column stats. + // The layout of each struct follows the layout of the ColumnStats. + val expressions = Count(Literal(1)).toAggregateExpression() +: + columns.map(statExprs(_, conf, attributePercentiles)) + + val namedExpressions = expressions.map(e => Alias(e, e.toString)()) + val statsRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExpressions, relation)) + .executedPlan.executeTake(1).head + + val rowCount = statsRow.getLong(0) + val columnStats = columns.zipWithIndex.map { case (attr, i) => + // according to `statExprs`, the stats struct always have 7 fields. + (attr, rowToColumnStat(statsRow.getStruct(i + 1, 7), attr, rowCount, + attributePercentiles.get(attr))) + }.toMap + (rowCount, columnStats) + } + + /** Computes percentiles for each attribute. */ + private def computePercentiles( + attributesToAnalyze: Seq[Attribute], + sparkSession: SparkSession, + relation: LogicalPlan): AttributeMap[ArrayData] = { + val conf = sparkSession.sessionState.conf + val attrsToGenHistogram = if (conf.histogramEnabled) { + attributesToAnalyze.filter(a => supportsHistogram(a.dataType)) + } else { + Nil + } + val attributePercentiles = mutable.HashMap[Attribute, ArrayData]() + if (attrsToGenHistogram.nonEmpty) { + val percentiles = (0 to conf.histogramNumBins) + .map(i => i.toDouble / conf.histogramNumBins).toArray + + val namedExprs = attrsToGenHistogram.map { attr => + val aggFunc = + new ApproximatePercentile(attr, Literal(percentiles), Literal(conf.percentileAccuracy)) + val expr = aggFunc.toAggregateExpression() + Alias(expr, expr.toString)() + } + + val percentilesRow = new QueryExecution(sparkSession, Aggregate(Nil, namedExprs, relation)) + .executedPlan.executeTake(1).head + attrsToGenHistogram.zipWithIndex.foreach { case (attr, i) => + val percentiles = percentilesRow.getArray(i) + // When there is no non-null value, `percentiles` is null. In such case, there is no + // need to generate histogram. + if (percentiles != null) { + attributePercentiles += attr -> percentiles + } + } + } + AttributeMap(attributePercentiles.toSeq) + } + + + /** Returns true iff the we support gathering histogram on column of the given type. */ + private def supportsHistogram(dataType: DataType): Boolean = dataType match { + case _: IntegralType => true + case _: DecimalType => true + case DoubleType | FloatType => true + case DateType => true + case TimestampType => true + case _ => false + } + + /** + * Constructs an expression to compute column statistics for a given column. + * + * The expression should create a single struct column with the following schema: + * distinctCount: Long, min: T, max: T, nullCount: Long, avgLen: Long, maxLen: Long, + * distinctCountsForIntervals: Array[Long] + * + * Together with [[rowToColumnStat]], this function is used to create [[ColumnStat]] and + * as a result should stay in sync with it. + */ + private def statExprs( + col: Attribute, + conf: SQLConf, + colPercentiles: AttributeMap[ArrayData]): CreateNamedStruct = { + def struct(exprs: Expression*): CreateNamedStruct = CreateStruct(exprs.map { expr => + expr.transformUp { case af: AggregateFunction => af.toAggregateExpression() } + }) + val one = Literal(1.toLong, LongType) + + // the approximate ndv (num distinct value) should never be larger than the number of rows + val numNonNulls = if (col.nullable) Count(col) else Count(one) + val ndv = Least(Seq(HyperLogLogPlusPlus(col, conf.ndvMaxError), numNonNulls)) + val numNulls = Subtract(Count(one), numNonNulls) + val defaultSize = Literal(col.dataType.defaultSize.toLong, LongType) + val nullArray = Literal(null, ArrayType(LongType)) + + def fixedLenTypeStruct: CreateNamedStruct = { + val genHistogram = + supportsHistogram(col.dataType) && colPercentiles.contains(col) + val intervalNdvsExpr = if (genHistogram) { + ApproxCountDistinctForIntervals(col, + Literal(colPercentiles(col), ArrayType(col.dataType)), conf.ndvMaxError) + } else { + nullArray + } + // For fixed width types, avg size should be the same as max size. + struct(ndv, Cast(Min(col), col.dataType), Cast(Max(col), col.dataType), numNulls, + defaultSize, defaultSize, intervalNdvsExpr) + } + + col.dataType match { + case _: IntegralType => fixedLenTypeStruct + case _: DecimalType => fixedLenTypeStruct + case DoubleType | FloatType => fixedLenTypeStruct + case BooleanType => fixedLenTypeStruct + case DateType => fixedLenTypeStruct + case TimestampType => fixedLenTypeStruct + case BinaryType | StringType => + // For string and binary type, we don't compute min, max or histogram + val nullLit = Literal(null, col.dataType) + struct( + ndv, nullLit, nullLit, numNulls, + // Set avg/max size to default size if all the values are null or there is no value. + Coalesce(Seq(Ceil(Average(Length(col))), defaultSize)), + Coalesce(Seq(Cast(Max(Length(col)), LongType), defaultSize)), + nullArray) + case _ => + throw new AnalysisException("Analyzing column statistics is not supported for column " + + s"${col.name} of data type: ${col.dataType}.") + } + } + + /** + * Convert a struct for column stats (defined in `statExprs`) into + * [[org.apache.spark.sql.catalyst.plans.logical.ColumnStat]]. + */ + private def rowToColumnStat( + row: InternalRow, + attr: Attribute, + rowCount: Long, + percentiles: Option[ArrayData]): ColumnStat = { + // The first 6 fields are basic column stats, the 7th is ndvs for histogram bins. + val cs = ColumnStat( + distinctCount = Option(BigInt(row.getLong(0))), + // for string/binary min/max, get should return null + min = Option(row.get(1, attr.dataType)), + max = Option(row.get(2, attr.dataType)), + nullCount = Option(BigInt(row.getLong(3))), + avgLen = Option(row.getLong(4)), + maxLen = Option(row.getLong(5)) + ) + if (row.isNullAt(6) || cs.nullCount.isEmpty) { + cs + } else { + val ndvs = row.getArray(6).toLongArray() + assert(percentiles.get.numElements() == ndvs.length + 1) + val endpoints = percentiles.get.toArray[Any](attr.dataType).map(_.toString.toDouble) + // Construct equi-height histogram + val bins = ndvs.zipWithIndex.map { case (ndv, i) => + HistogramBin(endpoints(i), endpoints(i + 1), ndv) + } + val nonNullRows = rowCount - cs.nullCount.get + val histogram = Histogram(nonNullRows.toDouble / ndvs.length, bins) + cs.copy(histogram = Some(histogram)) + } + } + + private def isDataPath(path: Path, stagingDir: String): Boolean = { + !path.getName.startsWith(stagingDir) && DataSourceUtils.isDataPath(path) + } } 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 3c900be839aa9..45c62b4676570 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.command import org.apache.spark.internal.Logging import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.IgnoreCachedData import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION import org.apache.spark.sql.types.{StringType, StructField, StructType} @@ -161,7 +162,7 @@ object SetCommand { * reset; * }}} */ -case object ResetCommand extends RunnableCommand with Logging { +case object ResetCommand extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.conf.clear() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala index 6b00426d2fa91..7b00769308a41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/cache.scala @@ -17,16 +17,21 @@ package org.apache.spark.sql.execution.command +import java.util.Locale + import org.apache.spark.sql.{Dataset, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{IgnoreCachedData, LogicalPlan} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.storage.StorageLevel case class CacheTableCommand( tableIdent: TableIdentifier, plan: Option[LogicalPlan], - isLazy: Boolean) extends RunnableCommand { + isLazy: Boolean, + options: Map[String, String]) extends RunnableCommand { require(plan.isEmpty || tableIdent.database.isEmpty, "Database name is not allowed in CACHE TABLE AS SELECT") @@ -36,7 +41,21 @@ case class CacheTableCommand( plan.foreach { logicalPlan => Dataset.ofRows(sparkSession, logicalPlan).createTempView(tableIdent.quotedString) } - sparkSession.catalog.cacheTable(tableIdent.quotedString) + + val storageLevelKey = "storagelevel" + val storageLevelValue = + CaseInsensitiveMap(options).get(storageLevelKey).map(_.toUpperCase(Locale.ROOT)) + val withoutStorageLevel = options.filterKeys(_.toLowerCase(Locale.ROOT) != storageLevelKey) + if (withoutStorageLevel.nonEmpty) { + logWarning(s"Invalid options: ${withoutStorageLevel.mkString(", ")}") + } + + if (storageLevelValue.nonEmpty) { + sparkSession.catalog.cacheTable( + tableIdent.quotedString, StorageLevel.fromString(storageLevelValue.get)) + } else { + sparkSession.catalog.cacheTable(tableIdent.quotedString) + } if (!isLazy) { // Performs eager caching @@ -64,13 +83,10 @@ case class UncacheTableCommand( /** * Clear all cached data from the in-memory cache. */ -case class ClearCacheCommand() extends RunnableCommand { +case object ClearCacheCommand extends RunnableCommand with IgnoreCachedData { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.catalog.clearCache() Seq.empty[Row] } - - /** [[org.apache.spark.sql.catalyst.trees.TreeNode.makeCopy()]] does not support 0-arg ctor. */ - override def makeCopy(newArgs: Array[AnyRef]): ClearCacheCommand = ClearCacheCommand() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala index 2cc0e38adc2ee..bb8a982f0d317 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/commands.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{Command, LogicalPlan} -import org.apache.spark.sql.execution.{LeafExecNode, SparkPlan} +import org.apache.spark.sql.execution.{LeafExecNode, QueryExecution, SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.debug._ import org.apache.spark.sql.execution.metric.SQLMetric import org.apache.spark.sql.execution.streaming.{IncrementalExecution, OffsetSeqMetadata} @@ -95,7 +95,7 @@ case class ExecutedCommandExec(cmd: RunnableCommand) extends LeafExecNode { * @param child the physical plan child ran by the `DataWritingCommand`. */ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) - extends SparkPlan { + extends UnaryExecNode { override lazy val metrics: Map[String, SQLMetric] = cmd.metrics @@ -106,12 +106,13 @@ case class DataWritingCommandExec(cmd: DataWritingCommand, child: SparkPlan) rows.map(converter(_).asInstanceOf[InternalRow]) } - override def children: Seq[SparkPlan] = child :: Nil - override def output: Seq[Attribute] = cmd.output override def nodeName: String = "Execute " + cmd.nodeName + // override the default one, otherwise the `cmd.nodeName` will appear twice from simpleString + override def argString(maxFields: Int): String = cmd.argString(maxFields) + override def executeCollect(): Array[InternalRow] = sideEffectResult.toArray override def executeToIterator: Iterator[InternalRow] = sideEffectResult.toIterator @@ -150,16 +151,8 @@ case class ExplainCommand( // Run through the optimizer to generate the physical plan. override def run(sparkSession: SparkSession): Seq[Row] = try { - val queryExecution = - if (logicalPlan.isStreaming) { - // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the - // output mode does not matter since there is no `Sink`. - new IncrementalExecution( - sparkSession, logicalPlan, OutputMode.Append(), "", - UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) - } else { - sparkSession.sessionState.executePlan(logicalPlan) - } + val queryExecution = ExplainCommandUtil.explainedQueryExecution(sparkSession, logicalPlan, + sparkSession.sessionState.executePlan(logicalPlan)) val outputString = if (codegen) { codegenString(queryExecution.executedPlan) @@ -176,6 +169,24 @@ case class ExplainCommand( } } +object ExplainCommandUtil { + // Returns `QueryExecution` which is used to explain a logical plan. + def explainedQueryExecution( + sparkSession: SparkSession, + logicalPlan: LogicalPlan, + queryExecution: => QueryExecution): QueryExecution = { + if (logicalPlan.isStreaming) { + // This is used only by explaining `Dataset/DataFrame` created by `spark.readStream`, so the + // output mode does not matter since there is no `Sink`. + new IncrementalExecution( + sparkSession, logicalPlan, OutputMode.Append(), "", + UUID.randomUUID, UUID.randomUUID, 0, OffsetSeqMetadata(0, 0)) + } else { + queryExecution + } + } +} + /** An explain command for users to see how a streaming batch is executed. */ case class StreamingExplainCommand( queryExecution: IncrementalExecution, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala index b2e1f530b5328..68c47d6a6dfaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/createDataSourceTables.scala @@ -193,6 +193,8 @@ case class CreateDataSourceTableAsSelectCommand( } } + CommandUtils.updateTableStats(sparkSession, table) + Seq.empty[Row] } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala index c419926e5b423..4827a10375f2e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/ddl.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.command import java.util.Locale +import java.util.concurrent.TimeUnit._ import scala.collection.{GenMap, GenSeq} import scala.collection.parallel.ForkJoinTaskSupport @@ -27,6 +28,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ import org.apache.hadoop.mapred.{FileInputFormat, JobConf} +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{Resolver, UnresolvedAttribute} @@ -71,7 +73,7 @@ case class CreateDatabaseCommand( CatalogDatabase( databaseName, comment.getOrElse(""), - path.map(CatalogUtils.stringToURI(_)).getOrElse(catalog.getDefaultDBPath(databaseName)), + path.map(CatalogUtils.stringToURI).getOrElse(catalog.getDefaultDBPath(databaseName)), props), ifNotExists) Seq.empty[Row] @@ -351,9 +353,8 @@ case class AlterTableChangeColumnCommand( } // Add the comment to a column, if comment is empty, return the original column. - private def addComment(column: StructField, comment: Option[String]): StructField = { - comment.map(column.withComment(_)).getOrElse(column) - } + private def addComment(column: StructField, comment: Option[String]): StructField = + comment.map(column.withComment).getOrElse(column) // Compare a [[StructField]] to another, return true if they have the same column // name(by resolver) and dataType. @@ -662,14 +663,12 @@ case class AlterTableRecoverPartitionsCommand( // It's very expensive to create a JobConf(ClassUtil.findContainingJar() is slow) val jobConf = new JobConf(hadoopConf, this.getClass) val pathFilter = FileInputFormat.getInputPathFilter(jobConf) - new PathFilter { - override def accept(path: Path): Boolean = { - val name = path.getName - if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { - pathFilter == null || pathFilter.accept(path) - } else { - false - } + path: Path => { + val name = path.getName + if (name != "_SUCCESS" && name != "_temporary" && !name.startsWith(".")) { + pathFilter == null || pathFilter.accept(path) + } else { + false } } } @@ -694,7 +693,7 @@ case class AlterTableRecoverPartitionsCommand( val hadoopConf = spark.sessionState.newHadoopConf() val fs = root.getFileSystem(hadoopConf) - val threshold = spark.conf.get("spark.rdd.parallelListingThreshold", "10").toInt + val threshold = spark.sparkContext.conf.get(RDD_PARALLEL_LISTING_THRESHOLD) val pathFilter = getPathFilter(hadoopConf) val evalPool = ThreadUtils.newForkJoinPool("AlterTableRecoverPartitionsCommand", 8) @@ -818,7 +817,7 @@ case class AlterTableRecoverPartitionsCommand( // do this in parallel. val batchSize = 100 partitionSpecsAndLocs.toIterator.grouped(batchSize).foreach { batch => - val now = System.currentTimeMillis() / 1000 + val now = MILLISECONDS.toSeconds(System.currentTimeMillis()) val parts = batch.map { case (spec, location) => val params = partitionStats.get(location.toString).map { case PartitionStatistics(numFiles, totalSize) => @@ -899,6 +898,14 @@ object DDLUtils { table.provider.isDefined && table.provider.get.toLowerCase(Locale.ROOT) != HIVE_PROVIDER } + def readHiveTable(table: CatalogTable): HiveTableRelation = { + HiveTableRelation( + table, + // Hive table columns are always nullable. + table.dataSchema.asNullable.toAttributes, + table.partitionSchema.asNullable.toAttributes) + } + /** * Throws a standard error for actions that require partitionProvider = hive. */ @@ -956,7 +963,8 @@ object DDLUtils { if (serde == HiveSerDe.sourceToSerDe("orc").get.serde) { OrcFileFormat.checkFieldNames(colNames) } else if (serde == HiveSerDe.sourceToSerDe("parquet").get.serde || - serde == Some("parquet.hive.serde.ParquetHiveSerDe")) { + serde == Some("parquet.hive.serde.ParquetHiveSerDe") || + serde == Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) { ParquetSchemaConverter.checkFieldNames(colNames) } case "parquet" => ParquetSchemaConverter.checkFieldNames(colNames) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala index 2e859cf1ef253..8fee02a8f6c82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/resources.scala @@ -38,7 +38,7 @@ case class AddJarCommand(path: String) extends RunnableCommand { override def run(sparkSession: SparkSession): Seq[Row] = { sparkSession.sessionState.resourceLoader.addJar(path) - Seq(Row(0)) + 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 2eca1c40a5b3f..03aca89bc642e 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 @@ -29,21 +29,24 @@ import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.spark.sql.{AnalysisException, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedAttribute} +import org.apache.spark.sql.catalyst.analysis.{NoSuchPartitionException, UnresolvedAttribute, UnresolvedRelation} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference} -import org.apache.spark.sql.catalyst.plans.logical.Histogram +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util.{escapeSingleQuotedString, quoteIdentifier} import org.apache.spark.sql.execution.datasources.{DataSource, PartitioningUtils} import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.json.JsonDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils -import org.apache.spark.util.Utils /** * A command to create a table with the same definition of the given existing table. @@ -214,7 +217,7 @@ case class AlterTableAddColumnsCommand( /** * ALTER TABLE ADD COLUMNS command does not support temporary view/table, * view, or datasource table with text, orc formats or external provider. - * For datasource table, it currently only supports parquet, json, csv. + * For datasource table, it currently only supports parquet, json, csv, orc. */ private def verifyAlterTableAddColumn( conf: SQLConf, @@ -231,12 +234,15 @@ case class AlterTableAddColumnsCommand( } if (DDLUtils.isDatasourceTable(catalogTable)) { - DataSource.lookupDataSource(catalogTable.provider.get, conf).newInstance() match { + DataSource.lookupDataSource(catalogTable.provider.get, conf). + getConstructor().newInstance() match { // For datasource table, this command can only support the following File format. // TextFileFormat only default to one column "value" // Hive type is already considered as hive serde table, so the logic will not // come in here. - case _: JsonFileFormat | _: CSVFileFormat | _: ParquetFileFormat => + case _: CSVFileFormat | _: JsonFileFormat | _: ParquetFileFormat => + case _: JsonDataSourceV2 | _: CSVDataSourceV2 | + _: OrcDataSourceV2 | _: ParquetDataSourceV2 => case s if s.getClass.getCanonicalName.endsWith("OrcFileFormat") => case s => throw new AnalysisException( @@ -306,7 +312,8 @@ case class LoadDataCommand( val loadPath = { if (isLocal) { val localFS = FileContext.getLocalFSFileContext() - makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), new Path(path)) + LoadDataCommand.makeQualified(FsConstants.LOCAL_FS_URI, localFS.getWorkingDirectory(), + new Path(path)) } else { val loadPath = new Path(path) // Follow Hive's behavior: @@ -323,7 +330,7 @@ case class LoadDataCommand( // by considering the wild card scenario in mind.as per old logic query param is // been considered while creating URI instance and if path contains wild card char '?' // the remaining charecters after '?' will be removed while forming URI instance - makeQualified(defaultFS, uriPath, loadPath) + LoadDataCommand.makeQualified(defaultFS, uriPath, loadPath) } } val fs = loadPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) @@ -363,7 +370,9 @@ case class LoadDataCommand( CommandUtils.updateTableStats(sparkSession, targetTable) Seq.empty[Row] } +} +object LoadDataCommand { /** * Returns a qualified path object. Method ported from org.apache.hadoop.fs.Path class. * @@ -372,8 +381,9 @@ case class LoadDataCommand( * @param path Path instance based on the path string specified by the user. * @return qualified path object */ - private def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { - val pathUri = if (path.isAbsolute()) path.toUri() else new Path(workingDir, path).toUri() + private[sql] def makeQualified(defaultUri: URI, workingDir: Path, path: Path): Path = { + val newPath = new Path(workingDir, path) + val pathUri = if (path.isAbsolute()) path.toUri() else newPath.toUri() if (pathUri.getScheme == null || pathUri.getAuthority == null && defaultUri.getAuthority != null) { val scheme = if (pathUri.getScheme == null) defaultUri.getScheme else pathUri.getScheme @@ -383,14 +393,14 @@ case class LoadDataCommand( pathUri.getAuthority } try { - val newUri = new URI(scheme, authority, pathUri.getPath, pathUri.getFragment) + val newUri = new URI(scheme, authority, pathUri.getPath, null, pathUri.getFragment) new Path(newUri) } catch { case e: URISyntaxException => throw new IllegalArgumentException(e) } } else { - path + newPath } } } @@ -488,6 +498,34 @@ case class TruncateTableCommand( } } +abstract class DescribeCommandBase extends RunnableCommand { + override val output: Seq[Attribute] = Seq( + // Column names are based on Hive. + AttributeReference("col_name", StringType, nullable = false, + new MetadataBuilder().putString("comment", "name of the column").build())(), + AttributeReference("data_type", StringType, nullable = false, + new MetadataBuilder().putString("comment", "data type of the column").build())(), + AttributeReference("comment", StringType, nullable = true, + new MetadataBuilder().putString("comment", "comment of the column").build())() + ) + + protected def describeSchema( + schema: StructType, + buffer: ArrayBuffer[Row], + header: Boolean): Unit = { + if (header) { + append(buffer, s"# ${output.head.name}", output(1).name, output(2).name) + } + schema.foreach { column => + append(buffer, column.name, column.dataType.simpleString, column.getComment().orNull) + } + } + + protected def append( + buffer: ArrayBuffer[Row], column: String, dataType: String, comment: String): Unit = { + buffer += Row(column, dataType, comment) + } +} /** * Command that looks like * {{{ @@ -498,17 +536,7 @@ case class DescribeTableCommand( table: TableIdentifier, partitionSpec: TablePartitionSpec, isExtended: Boolean) - extends RunnableCommand { - - override val output: Seq[Attribute] = Seq( - // Column names are based on Hive. - AttributeReference("col_name", StringType, nullable = false, - new MetadataBuilder().putString("comment", "name of the column").build())(), - AttributeReference("data_type", StringType, nullable = false, - new MetadataBuilder().putString("comment", "data type of the column").build())(), - AttributeReference("comment", StringType, nullable = true, - new MetadataBuilder().putString("comment", "comment of the column").build())() - ) + extends DescribeCommandBase { override def run(sparkSession: SparkSession): Seq[Row] = { val result = new ArrayBuffer[Row] @@ -597,22 +625,34 @@ case class DescribeTableCommand( } table.storage.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, "")) } +} - private def describeSchema( - schema: StructType, - buffer: ArrayBuffer[Row], - header: Boolean): Unit = { - if (header) { - append(buffer, s"# ${output.head.name}", output(1).name, output(2).name) - } - schema.foreach { column => - append(buffer, column.name, column.dataType.simpleString, column.getComment().orNull) - } - } +/** + * Command that looks like + * {{{ + * DESCRIBE [QUERY] statement + * }}} + * + * Parameter 'statement' can be one of the following types : + * 1. SELECT statements + * 2. SELECT statements inside set operators (UNION, INTERSECT etc) + * 3. VALUES statement. + * 4. TABLE statement. Example : TABLE table_name + * 5. statements of the form 'FROM table SELECT *' + * 6. Multi select statements of the following form: + * select * from (from a select * select *) + * 7. Common table expressions (CTEs) + */ +case class DescribeQueryCommand(queryText: String, plan: LogicalPlan) + extends DescribeCommandBase { - private def append( - buffer: ArrayBuffer[Row], column: String, dataType: String, comment: String): Unit = { - buffer += Row(column, dataType, comment) + override def simpleString(maxFields: Int): String = s"$nodeName $queryText".trim + + override def run(sparkSession: SparkSession): Seq[Row] = { + val result = new ArrayBuffer[Row] + val queryExecution = sparkSession.sessionState.executePlan(plan) + describeSchema(queryExecution.analyzed.schema, result, header = false) + result } } @@ -954,9 +994,11 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman builder ++= metadata.viewText.mkString(" AS\n", "", "\n") } else { showHiveTableHeader(metadata, builder) + showTableComment(metadata, builder) showHiveTableNonDataColumns(metadata, builder) showHiveTableStorageInfo(metadata, builder) - showHiveTableProperties(metadata, builder) + showTableLocation(metadata, builder) + showTableProperties(metadata, builder) } builder.toString() @@ -970,14 +1012,8 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman if (columns.nonEmpty) { builder ++= columns.mkString("(", ", ", ")\n") } - - metadata - .comment - .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") - .foreach(builder.append) } - private def showHiveTableNonDataColumns(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.partitionColumnNames.nonEmpty) { val partCols = metadata.partitionSchema.map(_.toDDL) @@ -1020,15 +1056,24 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman builder ++= s" OUTPUTFORMAT '${escapeSingleQuotedString(format)}'\n" } } + } + private def showTableLocation(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.tableType == EXTERNAL) { - storage.locationUri.foreach { uri => - builder ++= s"LOCATION '$uri'\n" + metadata.storage.locationUri.foreach { location => + builder ++= s"LOCATION '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'\n" } } } - private def showHiveTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { + private def showTableComment(metadata: CatalogTable, builder: StringBuilder): Unit = { + metadata + .comment + .map("COMMENT '" + escapeSingleQuotedString(_) + "'\n") + .foreach(builder.append) + } + + private def showTableProperties(metadata: CatalogTable, builder: StringBuilder): Unit = { if (metadata.properties.nonEmpty) { val props = metadata.properties.map { case (key, value) => s"'${escapeSingleQuotedString(key)}' = '${escapeSingleQuotedString(value)}'" @@ -1045,6 +1090,9 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman showDataSourceTableDataColumns(metadata, builder) showDataSourceTableOptions(metadata, builder) showDataSourceTableNonDataColumns(metadata, builder) + showTableComment(metadata, builder) + showTableLocation(metadata, builder) + showTableProperties(metadata, builder) builder.toString() } @@ -1060,14 +1108,6 @@ case class ShowCreateTableCommand(table: TableIdentifier) extends RunnableComman val dataSourceOptions = metadata.storage.properties.map { case (key, value) => s"${quoteIdentifier(key)} '${escapeSingleQuotedString(value)}'" - } ++ metadata.storage.locationUri.flatMap { location => - if (metadata.tableType == MANAGED) { - // If it's a managed table, omit PATH option. Spark SQL always creates external table - // when the table creation DDL contains the PATH option. - None - } else { - Some(s"path '${escapeSingleQuotedString(CatalogUtils.URIToString(location))}'") - } } if (dataSourceOptions.nonEmpty) { 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 cd34dfafd1320..4d3eb11250c3f 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 @@ -181,6 +181,8 @@ case class CreateViewCommand( * Permanent views are not allowed to reference temp objects, including temp function and views */ private def verifyTemporaryObjectsNotExists(sparkSession: SparkSession): Unit = { + import sparkSession.sessionState.analyzer.AsTableIdentifier + if (!isTemporary) { // This func traverses the unresolved plan `child`. Below are the reasons: // 1) Analyzer replaces unresolved temporary views by a SubqueryAlias with the corresponding @@ -190,10 +192,11 @@ case class CreateViewCommand( // package (e.g., HiveGenericUDF). child.collect { // Disallow creating permanent views based on temporary views. - case s: UnresolvedRelation - if sparkSession.sessionState.catalog.isTemporaryTable(s.tableIdentifier) => + case UnresolvedRelation(AsTableIdentifier(ident)) + if sparkSession.sessionState.catalog.isTemporaryTable(ident) => + // temporary views are only stored in the session catalog throw new AnalysisException(s"Not allowed to create a permanent view $name by " + - s"referencing a temporary view ${s.tableIdentifier}") + s"referencing a temporary view $ident") case other if !other.resolved => other.expressions.flatMap(_.collect { // Disallow creating permanent views based on temporary UDFs. case e: UnresolvedFunction diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala index ba7d2b7cbdb1a..b71c2d12f02b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/BasicWriteStatsTracker.scala @@ -173,7 +173,7 @@ object BasicWriteJobStatsTracker { val sparkContext = SparkContext.getActive.get Map( NUM_FILES_KEY -> SQLMetrics.createMetric(sparkContext, "number of written files"), - NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createMetric(sparkContext, "bytes of written output"), + NUM_OUTPUT_BYTES_KEY -> SQLMetrics.createSizeMetric(sparkContext, "written output"), NUM_OUTPUT_ROWS_KEY -> SQLMetrics.createMetric(sparkContext, "number of output rows"), NUM_PARTS_KEY -> SQLMetrics.createMetric(sparkContext, "number of dynamic part") ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala index a66a07673e25f..8736d0713e0b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/CatalogFileIndex.scala @@ -84,8 +84,8 @@ class CatalogFileIndex( new PrunedInMemoryFileIndex( sparkSession, new Path(baseLocation.get), fileStatusCache, partitionSpec, Option(timeNs)) } else { - new InMemoryFileIndex( - sparkSession, rootPaths, table.storage.properties, userSpecifiedSchema = None) + new InMemoryFileIndex(sparkSession, rootPaths, table.storage.properties, + userSpecifiedSchema = None, fileStatusCache = fileStatusCache) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala index ce3bc3dd48327..04ae528a1f6b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala @@ -20,9 +20,10 @@ package org.apache.spark.sql.execution.datasources import java.util.{Locale, ServiceConfigurationError, ServiceLoader} import scala.collection.JavaConverters._ -import scala.language.{existentials, implicitConversions} +import scala.language.implicitConversions import scala.util.{Failure, Success, Try} +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.deploy.SparkHadoopUtil @@ -30,7 +31,6 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogUtils} -import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.SparkPlan @@ -40,6 +40,8 @@ import org.apache.spark.sql.execution.datasources.jdbc.JdbcRelationProvider import org.apache.spark.sql.execution.datasources.json.JsonFileFormat import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.sources.{RateStreamProvider, TextSocketSourceProvider} import org.apache.spark.sql.internal.SQLConf @@ -90,8 +92,22 @@ case class DataSource( case class SourceInfo(name: String, schema: StructType, partitionColumns: Seq[String]) - lazy val providingClass: Class[_] = - DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + lazy val providingClass: Class[_] = { + val cls = DataSource.lookupDataSource(className, sparkSession.sessionState.conf) + // `providingClass` is used for resolving data source relation for catalog tables. + // As now catalog for data source V2 is under development, here we fall back all the + // [[FileDataSourceV2]] to [[FileFormat]] to guarantee the current catalog works. + // [[FileDataSourceV2]] will still be used if we call the load()/save() method in + // [[DataFrameReader]]/[[DataFrameWriter]], since they use method `lookupDataSource` + // instead of `providingClass`. + cls.newInstance() match { + case f: FileDataSourceV2 => f.fallbackFileFormat + case _ => cls + } + } + + private def providingInstance() = providingClass.getConstructor().newInstance() + lazy val sourceInfo: SourceInfo = sourceSchema() private val caseInsensitiveOptions = CaseInsensitiveMap(options) private val equality = sparkSession.sessionState.conf.resolver @@ -122,21 +138,14 @@ case class DataSource( * be any further inference in any triggers. * * @param format the file format object for this DataSource - * @param fileIndex optional [[InMemoryFileIndex]] for getting partition schema and file list + * @param getFileIndex [[InMemoryFileIndex]] for getting partition schema and file list * @return A pair of the data schema (excluding partition columns) and the schema of the partition * columns. */ private def getOrInferFileFormatSchema( format: FileFormat, - fileIndex: Option[InMemoryFileIndex] = None): (StructType, StructType) = { - // The operations below are expensive therefore try not to do them if we don't need to, e.g., - // in streaming mode, we have already inferred and registered partition columns, we will - // never have to materialize the lazy val below - lazy val tempFileIndex = fileIndex.getOrElse { - val globbedPaths = - checkAndGlobPathIfNecessary(checkEmptyGlobPath = false, checkFilesExist = false) - createInMemoryFileIndex(globbedPaths) - } + getFileIndex: () => InMemoryFileIndex): (StructType, StructType) = { + lazy val tempFileIndex = getFileIndex() val partitionSchema = if (partitionColumns.isEmpty) { // Try to infer partitioning, because no DataSource in the read path provides the partitioning @@ -204,7 +213,7 @@ case class DataSource( /** Returns the name and schema of the source that can be used to continually read data. */ private def sourceSchema(): SourceInfo = { - providingClass.newInstance() match { + providingInstance() match { case s: StreamSourceProvider => val (name, schema) = s.sourceSchema( sparkSession.sqlContext, userSpecifiedSchema, className, caseInsensitiveOptions) @@ -236,7 +245,15 @@ case class DataSource( "you may be able to create a static DataFrame on that directory with " + "'spark.read.load(directory)' and infer schema from it.") } - val (dataSchema, partitionSchema) = getOrInferFileFormatSchema(format) + + val (dataSchema, partitionSchema) = getOrInferFileFormatSchema(format, () => { + // The operations below are expensive therefore try not to do them if we don't need to, + // e.g., in streaming mode, we have already inferred and registered partition columns, + // we will never have to materialize the lazy val below + val globbedPaths = + checkAndGlobPathIfNecessary(checkEmptyGlobPath = false, checkFilesExist = false) + createInMemoryFileIndex(globbedPaths) + }) SourceInfo( s"FileSource[$path]", StructType(dataSchema ++ partitionSchema), @@ -250,7 +267,7 @@ case class DataSource( /** Returns a source that can be used to continually read data. */ def createSource(metadataPath: String): Source = { - providingClass.newInstance() match { + providingInstance() match { case s: StreamSourceProvider => s.createSource( sparkSession.sqlContext, @@ -279,7 +296,7 @@ case class DataSource( /** Returns a sink that can be used to continually write data. */ def createSink(outputMode: OutputMode): Sink = { - providingClass.newInstance() match { + providingInstance() match { case s: StreamSinkProvider => s.createSink(sparkSession.sqlContext, caseInsensitiveOptions, partitionColumns, outputMode) @@ -310,7 +327,7 @@ case class DataSource( * that files already exist, we don't need to check them again. */ def resolveRelation(checkFilesExist: Boolean = true): BaseRelation = { - val relation = (providingClass.newInstance(), userSpecifiedSchema) match { + val relation = (providingInstance(), userSpecifiedSchema) match { // TODO: Throw when too much is given. case (dataSource: SchemaRelationProvider, Some(schema)) => dataSource.createRelation(sparkSession.sqlContext, caseInsensitiveOptions, schema) @@ -331,9 +348,11 @@ case class DataSource( case (format: FileFormat, _) if FileStreamSink.hasMetadata( caseInsensitiveOptions.get("path").toSeq ++ paths, - sparkSession.sessionState.newHadoopConf()) => + sparkSession.sessionState.newHadoopConf(), + sparkSession.sessionState.conf) => val basePath = new Path((caseInsensitiveOptions.get("path").toSeq ++ paths).head) - val fileCatalog = new MetadataLogFileIndex(sparkSession, basePath, userSpecifiedSchema) + val fileCatalog = new MetadataLogFileIndex(sparkSession, basePath, + caseInsensitiveOptions, userSpecifiedSchema) val dataSchema = userSpecifiedSchema.orElse { format.inferSchema( sparkSession, @@ -370,7 +389,7 @@ case class DataSource( } else { val index = createInMemoryFileIndex(globbedPaths) val (resultDataSchema, resultPartitionSchema) = - getOrInferFileFormatSchema(format, Some(index)) + getOrInferFileFormatSchema(format, () => index) (index, resultDataSchema, resultPartitionSchema) } @@ -397,7 +416,7 @@ case class DataSource( hs.partitionSchema.map(_.name), "in the partition schema", equality) - DataSourceUtils.verifyReadSchema(hs.fileFormat, hs.dataSchema) + DataSourceUtils.verifySchema(hs.fileFormat, hs.dataSchema) case _ => SchemaUtils.checkColumnNameDuplication( relation.schema.map(_.name), @@ -479,7 +498,7 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - providingClass.newInstance() match { + providingInstance() match { case dataSource: CreatableRelationProvider => dataSource.createRelation( sparkSession.sqlContext, mode, caseInsensitiveOptions, Dataset.ofRows(sparkSession, data)) @@ -516,7 +535,7 @@ case class DataSource( throw new AnalysisException("Cannot save interval data type into external storage.") } - providingClass.newInstance() match { + providingInstance() match { case dataSource: CreatableRelationProvider => SaveIntoDataSourceCommand(data, dataSource, caseInsensitiveOptions, mode) case format: FileFormat => @@ -542,23 +561,9 @@ case class DataSource( checkFilesExist: Boolean): Seq[Path] = { val allPaths = caseInsensitiveOptions.get("path") ++ paths val hadoopConf = sparkSession.sessionState.newHadoopConf() - allPaths.flatMap { path => - val hdfsPath = new Path(path) - val fs = hdfsPath.getFileSystem(hadoopConf) - val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) - val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) - if (checkEmptyGlobPath && globPath.isEmpty) { - throw new AnalysisException(s"Path does not exist: $qualified") - } - - // Sufficient to check head of the globPath seq for non-glob scenario - // Don't need to check once again if files exist in streaming mode - if (checkFilesExist && !fs.exists(globPath.head)) { - throw new AnalysisException(s"Path does not exist: ${globPath.head}") - } - globPath - }.toSeq + DataSource.checkAndGlobPathIfNecessary(allPaths.toSeq, hadoopConf, + checkEmptyGlobPath, checkFilesExist) } } @@ -614,7 +619,7 @@ object DataSource extends Logging { val provider1 = backwardCompatibilityMap.getOrElse(provider, provider) match { case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native" => - classOf[OrcFileFormat].getCanonicalName + classOf[OrcDataSourceV2].getCanonicalName case name if name.equalsIgnoreCase("orc") && conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "hive" => "org.apache.spark.sql.hive.orc.OrcFileFormat" @@ -703,6 +708,48 @@ object DataSource extends Logging { } } + /** + * Checks and returns files in all the paths. + */ + private[sql] def checkAndGlobPathIfNecessary( + paths: Seq[String], + hadoopConf: Configuration, + checkEmptyGlobPath: Boolean, + checkFilesExist: Boolean): Seq[Path] = { + val allGlobPath = paths.flatMap { path => + val hdfsPath = new Path(path) + val fs = hdfsPath.getFileSystem(hadoopConf) + val qualified = hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + val globPath = SparkHadoopUtil.get.globPathIfNecessary(fs, qualified) + + if (checkEmptyGlobPath && globPath.isEmpty) { + throw new AnalysisException(s"Path does not exist: $qualified") + } + + // Sufficient to check head of the globPath seq for non-glob scenario + // Don't need to check once again if files exist in streaming mode + if (checkFilesExist && !fs.exists(globPath.head)) { + throw new AnalysisException(s"Path does not exist: ${globPath.head}") + } + globPath + } + + if (checkFilesExist) { + val (filteredOut, filteredIn) = allGlobPath.partition { path => + InMemoryFileIndex.shouldFilterOut(path.getName) + } + if (filteredIn.isEmpty) { + logWarning( + s"All paths were ignored:\n ${filteredOut.mkString("\n ")}") + } else { + logDebug( + s"Some paths were ignored:\n ${filteredOut.mkString("\n ")}") + } + } + + allGlobPath + } + /** * When creating a data source table, the `path` option has a special meaning: the table location. * This method extracts the `path` option and treat it as table location to build a @@ -720,7 +767,7 @@ object DataSource extends Logging { * supplied schema is not empty. * @param schema */ - private def validateSchema(schema: StructType): Unit = { + def validateSchema(schema: StructType): Unit = { def hasEmptySchema(schema: StructType): Boolean = { schema.size == 0 || schema.find { case StructField(_, b: StructType, _, _) => hasEmptySchema(b) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala new file mode 100644 index 0000000000000..1b7bb169b36fd --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceResolution.scala @@ -0,0 +1,284 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import java.util.Locale + +import scala.collection.mutable + +import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalog.v2.{CatalogPlugin, Identifier, LookupCatalog, TableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.CastSupport +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, UnresolvedCatalogRelation} +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.sql.{AlterTableAddColumnsStatement, AlterTableSetLocationStatement, AlterTableSetPropertiesStatement, AlterTableUnsetPropertiesStatement, AlterViewSetPropertiesStatement, AlterViewUnsetPropertiesStatement, CreateTableAsSelectStatement, CreateTableStatement, DropTableStatement, DropViewStatement, QualifiedColType} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.command.{AlterTableAddColumnsCommand, AlterTableSetLocationCommand, AlterTableSetPropertiesCommand, AlterTableUnsetPropertiesCommand, DropTableCommand} +import org.apache.spark.sql.execution.datasources.v2.{CatalogTableAsV2, DataSourceV2Relation} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.TableProvider +import org.apache.spark.sql.types.{HIVE_TYPE_STRING, HiveStringType, MetadataBuilder, StructField, StructType} + +case class DataSourceResolution( + conf: SQLConf, + lookup: LookupCatalog) + extends Rule[LogicalPlan] with CastSupport { + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + import lookup._ + + lazy val v2SessionCatalog: CatalogPlugin = lookup.sessionCatalog + .getOrElse(throw new AnalysisException("No v2 session catalog implementation is available")) + + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case CreateTableStatement( + AsTableIdentifier(table), schema, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment, ifNotExists) => + // the source is v1, the identifier has no catalog, and there is no default v2 catalog + val tableDesc = buildCatalogTable(table, schema, partitionCols, bucketSpec, properties, + provider, options, location, comment, ifNotExists) + val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists + + CreateTable(tableDesc, mode, None) + + case create: CreateTableStatement => + // the provider was not a v1 source or a v2 catalog is the default, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = create.tableName + maybeCatalog match { + case Some(catalog) => + // the identifier had a catalog, or there is a default v2 catalog + convertCreateTable(catalog.asTableCatalog, identifier, create) + case _ => + // the identifier had no catalog and no default catalog is set, but the source is v2. + // use the v2 session catalog, which delegates to the global v1 session catalog + convertCreateTable(v2SessionCatalog.asTableCatalog, identifier, create) + } + + case CreateTableAsSelectStatement( + AsTableIdentifier(table), query, partitionCols, bucketSpec, properties, + V1WriteProvider(provider), options, location, comment, ifNotExists) => + // the source is v1, the identifier has no catalog, and there is no default v2 catalog + val tableDesc = buildCatalogTable(table, new StructType, partitionCols, bucketSpec, + properties, provider, options, location, comment, ifNotExists) + val mode = if (ifNotExists) SaveMode.Ignore else SaveMode.ErrorIfExists + + CreateTable(tableDesc, mode, Some(query)) + + case create: CreateTableAsSelectStatement => + // the provider was not a v1 source or a v2 catalog is the default, convert to a v2 plan + val CatalogObjectIdentifier(maybeCatalog, identifier) = create.tableName + maybeCatalog match { + case Some(catalog) => + // the identifier had a catalog, or there is a default v2 catalog + convertCTAS(catalog.asTableCatalog, identifier, create) + case _ => + // the identifier had no catalog and no default catalog is set, but the source is v2. + // use the v2 session catalog, which delegates to the global v1 session catalog + convertCTAS(v2SessionCatalog.asTableCatalog, identifier, create) + } + + case DropTableStatement(CatalogObjectIdentifier(Some(catalog), ident), ifExists, _) => + DropTable(catalog.asTableCatalog, ident, ifExists) + + case DropTableStatement(AsTableIdentifier(tableName), ifExists, purge) => + DropTableCommand(tableName, ifExists, isView = false, purge) + + case DropViewStatement(CatalogObjectIdentifier(Some(catalog), ident), _) => + throw new AnalysisException( + s"Can not specify catalog `${catalog.name}` for view $ident " + + s"because view support in catalog has not been implemented yet") + + case DropViewStatement(AsTableIdentifier(tableName), ifExists) => + DropTableCommand(tableName, ifExists, isView = true, purge = false) + + case AlterTableSetPropertiesStatement(AsTableIdentifier(table), properties) => + AlterTableSetPropertiesCommand(table, properties, isView = false) + + case AlterViewSetPropertiesStatement(AsTableIdentifier(table), properties) => + AlterTableSetPropertiesCommand(table, properties, isView = true) + + case AlterTableUnsetPropertiesStatement(AsTableIdentifier(table), propertyKeys, ifExists) => + AlterTableUnsetPropertiesCommand(table, propertyKeys, ifExists, isView = false) + + case AlterViewUnsetPropertiesStatement(AsTableIdentifier(table), propertyKeys, ifExists) => + AlterTableUnsetPropertiesCommand(table, propertyKeys, ifExists, isView = true) + + case AlterTableSetLocationStatement(AsTableIdentifier(table), newLocation) => + AlterTableSetLocationCommand(table, None, newLocation) + + case AlterTableAddColumnsStatement(AsTableIdentifier(table), newColumns) + if newColumns.forall(_.name.size == 1) => + // only top-level adds are supported using AlterTableAddColumnsCommand + AlterTableAddColumnsCommand(table, newColumns.map(convertToStructField)) + + case DataSourceV2Relation(CatalogTableAsV2(catalogTable), _, _) => + UnresolvedCatalogRelation(catalogTable) + } + + object V1WriteProvider { + private val v1WriteOverrideSet = + conf.useV1SourceWriterList.toLowerCase(Locale.ROOT).split(",").toSet + + def unapply(provider: String): Option[String] = { + if (v1WriteOverrideSet.contains(provider.toLowerCase(Locale.ROOT))) { + Some(provider) + } else { + lazy val providerClass = DataSource.lookupDataSource(provider, conf) + provider match { + case _ if classOf[TableProvider].isAssignableFrom(providerClass) => + None + case _ => + Some(provider) + } + } + } + } + + private def buildCatalogTable( + table: TableIdentifier, + schema: StructType, + partitioning: Seq[Transform], + bucketSpec: Option[BucketSpec], + properties: Map[String, String], + provider: String, + options: Map[String, String], + location: Option[String], + comment: Option[String], + ifNotExists: Boolean): CatalogTable = { + + val storage = DataSource.buildStorageFormatFromOptions(options) + if (location.isDefined && storage.locationUri.isDefined) { + throw new AnalysisException( + "LOCATION and 'path' in OPTIONS are both used to indicate the custom table path, " + + "you can only specify one of them.") + } + val customLocation = storage.locationUri.orElse(location.map(CatalogUtils.stringToURI)) + + val tableType = if (customLocation.isDefined) { + CatalogTableType.EXTERNAL + } else { + CatalogTableType.MANAGED + } + + CatalogTable( + identifier = table, + tableType = tableType, + storage = storage.copy(locationUri = customLocation), + schema = schema, + provider = Some(provider), + partitionColumnNames = partitioning.asPartitionColumns, + bucketSpec = bucketSpec, + properties = properties, + comment = comment) + } + + private def convertCTAS( + catalog: TableCatalog, + identifier: Identifier, + ctas: CreateTableAsSelectStatement): CreateTableAsSelect = { + // convert the bucket spec and add it as a transform + val partitioning = ctas.partitioning ++ ctas.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + ctas.properties, ctas.options, ctas.location, ctas.comment, ctas.provider) + + CreateTableAsSelect( + catalog, + identifier, + partitioning, + ctas.asSelect, + properties, + writeOptions = ctas.options.filterKeys(_ != "path"), + ignoreIfExists = ctas.ifNotExists) + } + + private def convertCreateTable( + catalog: TableCatalog, + identifier: Identifier, + create: CreateTableStatement): CreateV2Table = { + // convert the bucket spec and add it as a transform + val partitioning = create.partitioning ++ create.bucketSpec.map(_.asTransform) + val properties = convertTableProperties( + create.properties, create.options, create.location, create.comment, create.provider) + + CreateV2Table( + catalog, + identifier, + create.tableSchema, + partitioning, + properties, + ignoreIfExists = create.ifNotExists) + } + + private def convertTableProperties( + properties: Map[String, String], + options: Map[String, String], + location: Option[String], + comment: Option[String], + provider: String): Map[String, String] = { + if (options.contains("path") && location.isDefined) { + throw new AnalysisException( + "LOCATION and 'path' in OPTIONS are both used to indicate the custom table path, " + + "you can only specify one of them.") + } + + if ((options.contains("comment") || properties.contains("comment")) + && comment.isDefined) { + throw new AnalysisException( + "COMMENT and option/property 'comment' are both used to set the table comment, you can " + + "only specify one of them.") + } + + if (options.contains("provider") || properties.contains("provider")) { + throw new AnalysisException( + "USING and option/property 'provider' are both used to set the provider implementation, " + + "you can only specify one of them.") + } + + val filteredOptions = options.filterKeys(_ != "path") + + // create table properties from TBLPROPERTIES and OPTIONS clauses + val tableProperties = new mutable.HashMap[String, String]() + tableProperties ++= properties + tableProperties ++= filteredOptions + + // convert USING, LOCATION, and COMMENT clauses to table properties + tableProperties += ("provider" -> provider) + comment.map(text => tableProperties += ("comment" -> text)) + location.orElse(options.get("path")).map(loc => tableProperties += ("location" -> loc)) + + tableProperties.toMap + } + + private def convertToStructField(col: QualifiedColType): StructField = { + val builder = new MetadataBuilder + col.comment.foreach(builder.putString("comment", _)) + + val cleanedDataType = HiveStringType.replaceCharType(col.dataType) + if (col.dataType != cleanedDataType) { + builder.putString(HIVE_TYPE_STRING, col.dataType.catalogString) + } + + StructField( + col.name.head, + cleanedDataType, + nullable = true, + builder.build()) + } +} 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 c6000442fae76..b8bed8569ace0 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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.execution.datasources import java.util.Locale -import java.util.concurrent.Callable + +import scala.collection.mutable import org.apache.hadoop.fs.Path @@ -29,11 +30,11 @@ import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, Quali import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.catalog._ +import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTable, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan} import org.apache.spark.sql.execution.command._ @@ -133,12 +134,10 @@ case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] with Cast override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case CreateTable(tableDesc, mode, None) if DDLUtils.isDatasourceTable(tableDesc) => - DDLUtils.checkDataColNames(tableDesc) CreateDataSourceTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) case CreateTable(tableDesc, mode, Some(query)) if query.resolved && DDLUtils.isDatasourceTable(tableDesc) => - DDLUtils.checkDataColNames(tableDesc.copy(schema = query.schema)) CreateDataSourceTableAsSelectCommand(tableDesc, mode, query, query.output.map(_.name)) case InsertIntoTable(l @ LogicalRelation(_: InsertableRelation, _, _, _), @@ -224,47 +223,36 @@ class FindDataSourceTable(sparkSession: SparkSession) extends Rule[LogicalPlan] private def readDataSourceTable(table: CatalogTable): LogicalPlan = { val qualifiedTableName = QualifiedTableName(table.database, table.identifier.table) val catalog = sparkSession.sessionState.catalog - catalog.getCachedPlan(qualifiedTableName, new Callable[LogicalPlan]() { - override def call(): LogicalPlan = { - val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) - val dataSource = - DataSource( - sparkSession, - // In older version(prior to 2.1) of Spark, the table schema can be empty and should be - // inferred at runtime. We should still support it. - userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema), - partitionColumns = table.partitionColumnNames, - bucketSpec = table.bucketSpec, - className = table.provider.get, - options = table.storage.properties ++ pathOption, - catalogTable = Some(table)) - - LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table) - } + catalog.getCachedPlan(qualifiedTableName, () => { + val pathOption = table.storage.locationUri.map("path" -> CatalogUtils.URIToString(_)) + val dataSource = + DataSource( + sparkSession, + // In older version(prior to 2.1) of Spark, the table schema can be empty and should be + // inferred at runtime. We should still support it. + userSpecifiedSchema = if (table.schema.isEmpty) None else Some(table.schema), + partitionColumns = table.partitionColumnNames, + bucketSpec = table.bucketSpec, + className = table.provider.get, + options = table.storage.properties ++ pathOption, + catalogTable = Some(table)) + LogicalRelation(dataSource.resolveRelation(checkFilesExist = false), table) }) } - private def readHiveTable(table: CatalogTable): LogicalPlan = { - HiveTableRelation( - table, - // Hive table columns are always nullable. - table.dataSchema.asNullable.toAttributes, - table.partitionSchema.asNullable.toAttributes) - } - override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) if DDLUtils.isDatasourceTable(tableMeta) => i.copy(table = readDataSourceTable(tableMeta)) case i @ InsertIntoTable(UnresolvedCatalogRelation(tableMeta), _, _, _, _) => - i.copy(table = readHiveTable(tableMeta)) + i.copy(table = DDLUtils.readHiveTable(tableMeta)) case UnresolvedCatalogRelation(tableMeta) if DDLUtils.isDatasourceTable(tableMeta) => readDataSourceTable(tableMeta) case UnresolvedCatalogRelation(tableMeta) => - readHiveTable(tableMeta) + DDLUtils.readHiveTable(tableMeta) } } @@ -416,7 +404,10 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with output: Seq[Attribute], rdd: RDD[Row]): RDD[InternalRow] = { if (relation.relation.needConversion) { - execution.RDDConversions.rowToRowRdd(rdd, output.map(_.dataType)) + val converters = RowEncoder(StructType.fromAttributes(output)) + rdd.mapPartitions { iterator => + iterator.map(converters.toRow) + } } else { rdd.asInstanceOf[RDD[InternalRow]] } @@ -431,60 +422,110 @@ case class DataSourceStrategy(conf: SQLConf) extends Strategy with Logging with } object DataSourceStrategy { + /** + * The attribute name of predicate could be different than the one in schema in case of + * case insensitive, we should change them to match the one in schema, so we do not need to + * worry about case sensitivity anymore. + */ + protected[sql] def normalizeFilters( + filters: Seq[Expression], + attributes: Seq[AttributeReference]): Seq[Expression] = { + filters.map { e => + e transform { + case a: AttributeReference => + a.withName(attributes.find(_.semanticEquals(a)).get.name) + } + } + } + + private def translateLeafNodeFilter(predicate: Expression): Option[Filter] = predicate match { + case expressions.EqualTo(a: Attribute, Literal(v, t)) => + Some(sources.EqualTo(a.name, convertToScala(v, t))) + case expressions.EqualTo(Literal(v, t), a: Attribute) => + Some(sources.EqualTo(a.name, convertToScala(v, t))) + + case expressions.EqualNullSafe(a: Attribute, Literal(v, t)) => + Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) + case expressions.EqualNullSafe(Literal(v, t), a: Attribute) => + Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) + + case expressions.GreaterThan(a: Attribute, Literal(v, t)) => + Some(sources.GreaterThan(a.name, convertToScala(v, t))) + case expressions.GreaterThan(Literal(v, t), a: Attribute) => + Some(sources.LessThan(a.name, convertToScala(v, t))) + + case expressions.LessThan(a: Attribute, Literal(v, t)) => + Some(sources.LessThan(a.name, convertToScala(v, t))) + case expressions.LessThan(Literal(v, t), a: Attribute) => + Some(sources.GreaterThan(a.name, convertToScala(v, t))) + + case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, t)) => + Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) + case expressions.GreaterThanOrEqual(Literal(v, t), a: Attribute) => + Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) + + case expressions.LessThanOrEqual(a: Attribute, Literal(v, t)) => + Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) + case expressions.LessThanOrEqual(Literal(v, t), a: Attribute) => + Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) + + case expressions.InSet(a: Attribute, set) => + val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) + Some(sources.In(a.name, set.toArray.map(toScala))) + + // Because we only convert In to InSet in Optimizer when there are more than certain + // items. So it is possible we still get an In expression here that needs to be pushed + // down. + case expressions.In(a: Attribute, list) if list.forall(_.isInstanceOf[Literal]) => + val hSet = list.map(_.eval(EmptyRow)) + val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) + Some(sources.In(a.name, hSet.toArray.map(toScala))) + + case expressions.IsNull(a: Attribute) => + Some(sources.IsNull(a.name)) + case expressions.IsNotNull(a: Attribute) => + Some(sources.IsNotNull(a.name)) + case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringStartsWith(a.name, v.toString)) + + case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringEndsWith(a.name, v.toString)) + + case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => + Some(sources.StringContains(a.name, v.toString)) + + case expressions.Literal(true, BooleanType) => + Some(sources.AlwaysTrue) + + case expressions.Literal(false, BooleanType) => + Some(sources.AlwaysFalse) + + case _ => None + } + /** * Tries to translate a Catalyst [[Expression]] into data source [[Filter]]. * * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`. */ protected[sql] def translateFilter(predicate: Expression): Option[Filter] = { - predicate match { - case expressions.EqualTo(a: Attribute, Literal(v, t)) => - Some(sources.EqualTo(a.name, convertToScala(v, t))) - case expressions.EqualTo(Literal(v, t), a: Attribute) => - Some(sources.EqualTo(a.name, convertToScala(v, t))) - - case expressions.EqualNullSafe(a: Attribute, Literal(v, t)) => - Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) - case expressions.EqualNullSafe(Literal(v, t), a: Attribute) => - Some(sources.EqualNullSafe(a.name, convertToScala(v, t))) - - case expressions.GreaterThan(a: Attribute, Literal(v, t)) => - Some(sources.GreaterThan(a.name, convertToScala(v, t))) - case expressions.GreaterThan(Literal(v, t), a: Attribute) => - Some(sources.LessThan(a.name, convertToScala(v, t))) - - case expressions.LessThan(a: Attribute, Literal(v, t)) => - Some(sources.LessThan(a.name, convertToScala(v, t))) - case expressions.LessThan(Literal(v, t), a: Attribute) => - Some(sources.GreaterThan(a.name, convertToScala(v, t))) - - case expressions.GreaterThanOrEqual(a: Attribute, Literal(v, t)) => - Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) - case expressions.GreaterThanOrEqual(Literal(v, t), a: Attribute) => - Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) - - case expressions.LessThanOrEqual(a: Attribute, Literal(v, t)) => - Some(sources.LessThanOrEqual(a.name, convertToScala(v, t))) - case expressions.LessThanOrEqual(Literal(v, t), a: Attribute) => - Some(sources.GreaterThanOrEqual(a.name, convertToScala(v, t))) - - case expressions.InSet(a: Attribute, set) => - val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) - Some(sources.In(a.name, set.toArray.map(toScala))) - - // Because we only convert In to InSet in Optimizer when there are more than certain - // items. So it is possible we still get an In expression here that needs to be pushed - // down. - case expressions.In(a: Attribute, list) if !list.exists(!_.isInstanceOf[Literal]) => - val hSet = list.map(e => e.eval(EmptyRow)) - val toScala = CatalystTypeConverters.createToScalaConverter(a.dataType) - Some(sources.In(a.name, hSet.toArray.map(toScala))) - - case expressions.IsNull(a: Attribute) => - Some(sources.IsNull(a.name)) - case expressions.IsNotNull(a: Attribute) => - Some(sources.IsNotNull(a.name)) + translateFilterWithMapping(predicate, None) + } + /** + * Tries to translate a Catalyst [[Expression]] into data source [[Filter]]. + * + * @param predicate The input [[Expression]] to be translated as [[Filter]] + * @param translatedFilterToExpr An optional map from leaf node filter expressions to its + * translated [[Filter]]. The map is used for rebuilding + * [[Expression]] from [[Filter]]. + * @return a `Some[Filter]` if the input [[Expression]] is convertible, otherwise a `None`. + */ + protected[sql] def translateFilterWithMapping( + predicate: Expression, + translatedFilterToExpr: Option[mutable.HashMap[sources.Filter, Expression]]) + : Option[Filter] = { + predicate match { case expressions.And(left, right) => // See SPARK-12218 for detailed discussion // It is not safe to just convert one side if we do not understand the @@ -496,29 +537,44 @@ object DataSourceStrategy { // Pushing one leg of AND down is only safe to do at the top level. // You can see ParquetFilters' createFilter for more details. for { - leftFilter <- translateFilter(left) - rightFilter <- translateFilter(right) + leftFilter <- translateFilterWithMapping(left, translatedFilterToExpr) + rightFilter <- translateFilterWithMapping(right, translatedFilterToExpr) } yield sources.And(leftFilter, rightFilter) case expressions.Or(left, right) => for { - leftFilter <- translateFilter(left) - rightFilter <- translateFilter(right) + leftFilter <- translateFilterWithMapping(left, translatedFilterToExpr) + rightFilter <- translateFilterWithMapping(right, translatedFilterToExpr) } yield sources.Or(leftFilter, rightFilter) case expressions.Not(child) => - translateFilter(child).map(sources.Not) + translateFilterWithMapping(child, translatedFilterToExpr).map(sources.Not) - case expressions.StartsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringStartsWith(a.name, v.toString)) - - case expressions.EndsWith(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringEndsWith(a.name, v.toString)) - - case expressions.Contains(a: Attribute, Literal(v: UTF8String, StringType)) => - Some(sources.StringContains(a.name, v.toString)) + case other => + val filter = translateLeafNodeFilter(other) + if (filter.isDefined && translatedFilterToExpr.isDefined) { + translatedFilterToExpr.get(filter.get) = predicate + } + filter + } + } - case _ => None + protected[sql] def rebuildExpressionFromFilter( + filter: Filter, + translatedFilterToExpr: mutable.HashMap[sources.Filter, Expression]): Expression = { + filter match { + case sources.And(left, right) => + expressions.And(rebuildExpressionFromFilter(left, translatedFilterToExpr), + rebuildExpressionFromFilter(right, translatedFilterToExpr)) + case sources.Or(left, right) => + expressions.Or(rebuildExpressionFromFilter(left, translatedFilterToExpr), + rebuildExpressionFromFilter(right, translatedFilterToExpr)) + case sources.Not(pred) => + expressions.Not(rebuildExpressionFromFilter(pred, translatedFilterToExpr)) + case other => + translatedFilterToExpr.getOrElse(other, + throw new AnalysisException( + s"Fail to rebuild expression: missing key $filter in `translatedFilterToExpr`")) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala index 90cec5e72c1a7..bd56635084c34 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceUtils.scala @@ -18,34 +18,39 @@ package org.apache.spark.sql.execution.datasources import org.apache.hadoop.fs.Path +import org.json4s.NoTypeHints +import org.json4s.jackson.Serialization import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.types._ object DataSourceUtils { - /** - * Verify if the schema is supported in datasource in write path. + * The key to use for storing partitionBy columns as options. */ - def verifyWriteSchema(format: FileFormat, schema: StructType): Unit = { - verifySchema(format, schema, isReadPath = false) - } + val PARTITIONING_COLUMNS_KEY = "__partition_columns" /** - * Verify if the schema is supported in datasource in read path. + * Utility methods for converting partitionBy columns to options and back. */ - def verifyReadSchema(format: FileFormat, schema: StructType): Unit = { - verifySchema(format, schema, isReadPath = true) + private implicit val formats = Serialization.formats(NoTypeHints) + + def encodePartitioningColumns(columns: Seq[String]): String = { + Serialization.write(columns) + } + + def decodePartitioningColumns(str: String): Seq[String] = { + Serialization.read[Seq[String]](str) } /** * Verify if the schema is supported in datasource. This verification should be done * in a driver side. */ - private def verifySchema(format: FileFormat, schema: StructType, isReadPath: Boolean): Unit = { + def verifySchema(format: FileFormat, schema: StructType): Unit = { schema.foreach { field => - if (!format.supportDataType(field.dataType, isReadPath)) { + if (!format.supportDataType(field.dataType)) { throw new AnalysisException( s"$format data source does not support ${field.dataType.catalogString} data type.") } @@ -55,8 +60,8 @@ object DataSourceUtils { // SPARK-24626: Metadata files and temporary files should not be // counted as data files, so that they shouldn't participate in tasks like // location size calculation. - private[sql] def isDataPath(path: Path): Boolean = { - val name = path.getName - !(name.startsWith("_") || name.startsWith(".")) - } + private[sql] def isDataPath(path: Path): Boolean = isDataFile(path.getName) + + private[sql] def isDataFile(fileName: String) = + !(fileName.startsWith("_") || fileName.startsWith(".")) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala new file mode 100644 index 0000000000000..813af8203c2c5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FallBackFileSourceV2.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoTable, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileDataSourceV2, FileTable} + +/** + * Replace the File source V2 table in [[InsertIntoTable]] to V1 [[FileFormat]]. + * E.g, with temporary view `t` using [[FileDataSourceV2]], inserting into view `t` fails + * since there is no corresponding physical plan. + * This is a temporary hack for making current data source V2 work. It should be + * removed when Catalog support of file data source v2 is finished. + */ +class FallBackFileSourceV2(sparkSession: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { + case i @ InsertIntoTable(d @ DataSourceV2Relation(table: FileTable, _, _), _, _, _, _) => + val v1FileFormat = table.fallbackFileFormat.newInstance() + val relation = HadoopFsRelation( + table.fileIndex, + table.fileIndex.partitionSchema, + table.schema, + None, + v1FileFormat, + d.options.asScala.toMap)(sparkSession) + i.copy(table = LogicalRelation(relation)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala index 2c162e23644ef..a72a9c27a9894 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormat.scala @@ -132,8 +132,6 @@ trait FileFormat { new (PartitionedFile => Iterator[InternalRow]) with Serializable { private val fullSchema = requiredSchema.toAttributes ++ partitionSchema.toAttributes - private val joinedRow = new JoinedRow() - // Using lazy val to avoid serialization private lazy val appendPartitionColumns = GenerateUnsafeProjection.generate(fullSchema, fullSchema) @@ -145,8 +143,15 @@ trait FileFormat { // Note that we have to apply the converter even though `file.partitionValues` is empty. // This is because the converter is also responsible for converting safe `InternalRow`s into // `UnsafeRow`s. - dataReader(file).map { dataRow => - converter(joinedRow(dataRow, file.partitionValues)) + if (partitionSchema.isEmpty) { + dataReader(file).map { dataRow => + converter(dataRow) + } + } else { + val joinedRow = new JoinedRow() + dataReader(file).map { dataRow => + converter(joinedRow(dataRow, file.partitionValues)) + } } } } @@ -156,7 +161,7 @@ trait FileFormat { * Returns whether this format supports the given [[DataType]] in read/write path. * By default all data types are supported. */ - def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = true + def supportDataType(dataType: DataType): Boolean = true } /** 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 6499328e89ce7..2595cc6371bc2 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} import org.apache.spark.sql.types.StringType import org.apache.spark.util.SerializableConfiguration @@ -37,7 +38,7 @@ import org.apache.spark.util.SerializableConfiguration abstract class FileFormatDataWriter( description: WriteJobDescription, taskAttemptContext: TaskAttemptContext, - committer: FileCommitProtocol) { + committer: FileCommitProtocol) extends DataWriter[InternalRow] { /** * Max number of files a single task writes out due to file size. In most cases the number of * files written should be very small. This is just a safe guard to protect some really bad @@ -70,7 +71,7 @@ abstract class FileFormatDataWriter( * to the driver and used to update the catalog. Other information will be sent back to the * driver too and used to e.g. update the metrics in UI. */ - def commit(): WriteTaskResult = { + override def commit(): WriteTaskResult = { releaseResources() val summary = ExecutedWriteSummary( updatedPartitions = updatedPartitions.toSet, @@ -179,7 +180,8 @@ class DynamicPartitionDataWriter( val partitionName = ScalaUDF( ExternalCatalogUtils.getPartitionPathString _, StringType, - Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId)))) + Seq(Literal(c.name), Cast(c, StringType, Option(description.timeZoneId))), + Seq(false, false)) if (i == 0) Seq(partitionName) else Seq(Literal(Path.SEPARATOR), partitionName) }) @@ -300,6 +302,7 @@ class WriteJobDescription( /** The result of a successful write task. */ case class WriteTaskResult(commitMsg: TaskCommitMessage, summary: ExecutedWriteSummary) + extends WriterCommitMessage /** * Wrapper class for the metrics of writing data out. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala index 774fe38f5c2e6..f1fc5d762ad56 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileFormatWriter.scala @@ -34,9 +34,13 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.BucketSpec import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences +import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} -import org.apache.spark.sql.execution.{SortExec, SparkPlan, SQLExecution} +import org.apache.spark.sql.execution.{ProjectExec, SortExec, SparkPlan, SQLExecution} +import org.apache.spark.sql.types.StringType +import org.apache.spark.unsafe.types.UTF8String import org.apache.spark.util.{SerializableConfiguration, Utils} @@ -48,6 +52,22 @@ object FileFormatWriter extends Logging { customPartitionLocations: Map[TablePartitionSpec, String], outputColumns: Seq[Attribute]) + /** A function that converts the empty string to null for partition values. */ + case class Empty2Null(child: Expression) extends UnaryExpression with String2StringExpression { + override def convert(v: UTF8String): UTF8String = if (v.numBytes() == 0) null else v + override def nullable: Boolean = true + override def doGenCode(ctx: CodegenContext, ev: ExprCode): ExprCode = { + nullSafeCodeGen(ctx, ev, c => { + s"""if ($c.numBytes() == 0) { + | ${ev.isNull} = true; + | ${ev.value} = null; + |} else { + | ${ev.value} = $c; + |}""".stripMargin + }) + } + } + /** * Basic work flow of this command is: * 1. Driver side setup, including output committer initialization and data source specific @@ -83,6 +103,15 @@ object FileFormatWriter extends Logging { val partitionSet = AttributeSet(partitionColumns) val dataColumns = outputSpec.outputColumns.filterNot(partitionSet.contains) + var needConvert = false + val projectList: Seq[NamedExpression] = plan.output.map { + case p if partitionSet.contains(p) && p.dataType == StringType && p.nullable => + needConvert = true + Alias(Empty2Null(p), p.name)() + case attr => attr + } + val empty2NullPlan = if (needConvert) ProjectExec(projectList, plan) else plan + val bucketIdExpression = bucketSpec.map { spec => val bucketColumns = spec.bucketColumnNames.map(c => dataColumns.find(_.name == c).get) // Use `HashPartitioning.partitionIdExpression` as our bucket id expression, so that we can @@ -97,7 +126,7 @@ object FileFormatWriter extends Logging { val caseInsensitiveOptions = CaseInsensitiveMap(options) val dataSchema = dataColumns.toStructType - DataSourceUtils.verifyWriteSchema(fileFormat, dataSchema) + DataSourceUtils.verifySchema(fileFormat, dataSchema) // Note: prepareWrite has side effect. It sets "job". val outputWriterFactory = fileFormat.prepareWrite(sparkSession, job, caseInsensitiveOptions, dataSchema) @@ -122,7 +151,7 @@ object FileFormatWriter extends Logging { // We should first sort by partition columns, then bucket id, and finally sorting columns. val requiredOrdering = partitionColumns ++ bucketIdExpression ++ sortColumns // the sort order doesn't matter - val actualOrdering = plan.outputOrdering.map(_.child) + val actualOrdering = empty2NullPlan.outputOrdering.map(_.child) val orderingMatched = if (requiredOrdering.length > actualOrdering.length) { false } else { @@ -140,18 +169,17 @@ object FileFormatWriter extends Logging { try { val rdd = if (orderingMatched) { - plan.execute() + empty2NullPlan.execute() } else { // SPARK-21165: the `requiredOrdering` is based on the attributes from analyzed plan, and // the physical plan may have different attribute ids due to optimizer removing some // aliases. Here we bind the expression ahead to avoid potential attribute ids mismatch. - val orderingExpr = requiredOrdering - .map(SortOrder(_, Ascending)) - .map(BindReferences.bindReference(_, outputSpec.outputColumns)) + val orderingExpr = bindReferences( + requiredOrdering.map(SortOrder(_, Ascending)), outputSpec.outputColumns) SortExec( orderingExpr, global = false, - child = plan).execute() + child = empty2NullPlan).execute() } // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single @@ -162,15 +190,17 @@ object FileFormatWriter extends Logging { rdd } + val jobIdInstant = new Date().getTime val ret = new Array[WriteTaskResult](rddWithNonEmptyPartitions.partitions.length) sparkSession.sparkContext.runJob( rddWithNonEmptyPartitions, (taskContext: TaskContext, iter: Iterator[InternalRow]) => { executeTask( description = description, + jobIdInstant = jobIdInstant, sparkStageId = taskContext.stageId(), sparkPartitionId = taskContext.partitionId(), - sparkAttemptNumber = taskContext.attemptNumber(), + sparkAttemptNumber = taskContext.taskAttemptId().toInt & Integer.MAX_VALUE, committer, iterator = iter) }, @@ -200,13 +230,14 @@ object FileFormatWriter extends Logging { /** Writes data out in a single Spark task. */ private def executeTask( description: WriteJobDescription, + jobIdInstant: Long, sparkStageId: Int, sparkPartitionId: Int, sparkAttemptNumber: Int, committer: FileCommitProtocol, iterator: Iterator[InternalRow]): WriteTaskResult = { - val jobId = SparkHadoopWriterUtils.createJobID(new Date, sparkStageId) + val jobId = SparkHadoopWriterUtils.createJobID(new Date(jobIdInstant), sparkStageId) val taskId = new TaskID(jobId, TaskType.MAP, sparkPartitionId) val taskAttemptId = new TaskAttemptID(taskId, sparkAttemptNumber) @@ -259,7 +290,7 @@ object FileFormatWriter extends Logging { * For every registered [[WriteJobStatsTracker]], call `processStats()` on it, passing it * the corresponding [[WriteTaskStats]] from all executors. */ - private def processStats( + private[datasources] def processStats( statsTrackers: Seq[WriteJobStatsTracker], statsPerTask: Seq[Seq[WriteTaskStats]]) : Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala new file mode 100644 index 0000000000000..14bee173cc116 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FilePartition.scala @@ -0,0 +1,97 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.Partition +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.v2.reader.InputPartition + +/** + * A collection of file blocks that should be read as a single task + * (possibly from multiple partitioned directories). + */ +case class FilePartition(index: Int, files: Array[PartitionedFile]) + extends Partition with InputPartition { + override def preferredLocations(): Array[String] = { + // Computes total number of bytes can be retrieved from each host. + val hostToNumBytes = mutable.HashMap.empty[String, Long] + files.foreach { file => + file.locations.filter(_ != "localhost").foreach { host => + hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length + } + } + + // Takes the first 3 hosts with the most data to be retrieved + hostToNumBytes.toSeq.sortBy { + case (host, numBytes) => numBytes + }.reverse.take(3).map { + case (host, numBytes) => host + }.toArray + } +} + +object FilePartition extends Logging { + + def getFilePartitions( + sparkSession: SparkSession, + partitionedFiles: Seq[PartitionedFile], + maxSplitBytes: Long): Seq[FilePartition] = { + val partitions = new ArrayBuffer[FilePartition] + val currentFiles = new ArrayBuffer[PartitionedFile] + var currentSize = 0L + + /** Close the current partition and move to the next. */ + def closePartition(): Unit = { + if (currentFiles.nonEmpty) { + // Copy to a new Array. + val newPartition = FilePartition(partitions.size, currentFiles.toArray) + partitions += newPartition + } + currentFiles.clear() + currentSize = 0 + } + + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + // Assign files to partitions using "Next Fit Decreasing" + partitionedFiles.foreach { file => + if (currentSize + file.length > maxSplitBytes) { + closePartition() + } + // Add the given file to the current partition. + currentSize += file.length + openCostInBytes + currentFiles += file + } + closePartition() + partitions + } + + def maxSplitBytes( + sparkSession: SparkSession, + selectedPartitions: Seq[PartitionDirectory]): Long = { + val defaultMaxSplitBytes = sparkSession.sessionState.conf.filesMaxPartitionBytes + val openCostInBytes = sparkSession.sessionState.conf.filesOpenCostInBytes + val defaultParallelism = sparkSession.sparkContext.defaultParallelism + val totalBytes = selectedPartitions.flatMap(_.files.map(_.getLen + openCostInBytes)).sum + val bytesPerCore = totalBytes / defaultParallelism + + Math.min(defaultMaxSplitBytes, Math.max(openCostInBytes, bytesPerCore)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala index dd3c154259c73..9e98b0bbfabc9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileScanRDD.scala @@ -29,6 +29,7 @@ import org.apache.spark.rdd.{InputFileBlockHolder, RDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.vectorized.ColumnarBatch import org.apache.spark.util.NextIterator @@ -37,7 +38,7 @@ import org.apache.spark.util.NextIterator * that need to be prepended to each row. * * @param partitionValues value of partition columns to be prepended to each row. - * @param filePath path of the file to read + * @param filePath URI of the file to read * @param start the beginning offset (in bytes) of the block. * @param length number of bytes to read. * @param locations locality information (list of nodes that have the data). @@ -53,12 +54,6 @@ case class PartitionedFile( } } -/** - * A collection of file blocks that should be read as a single task - * (possibly from multiple partitioned directories). - */ -case class FilePartition(index: Int, files: Seq[PartitionedFile]) extends RDDPartition - /** * An RDD that scans a list of file partitions. */ @@ -85,7 +80,7 @@ class FileScanRDD( // If we do a coalesce, however, we are likely to compute multiple partitions in the same // task and in the same thread, in which case we need to avoid override values written by // previous partitions (SPARK-13071). - private def updateBytesRead(): Unit = { + private def incTaskInputMetricsBytesRead(): Unit = { inputMetrics.setBytesRead(existingBytesRead + getBytesReadCallback()) } @@ -106,15 +101,16 @@ class FileScanRDD( // don't need to run this `if` for every record. val preNumRecordsRead = inputMetrics.recordsRead if (nextElement.isInstanceOf[ColumnarBatch]) { + incTaskInputMetricsBytesRead() inputMetrics.incRecordsRead(nextElement.asInstanceOf[ColumnarBatch].numRows()) } else { + // too costly to update every record + if (inputMetrics.recordsRead % + SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS == 0) { + incTaskInputMetricsBytesRead() + } inputMetrics.incRecordsRead(1) } - // The records may be incremented by more than 1 at a time. - if (preNumRecordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS != - inputMetrics.recordsRead / SparkHadoopUtil.UPDATE_INPUT_METRICS_INTERVAL_RECORDS) { - updateBytesRead() - } nextElement } @@ -201,7 +197,7 @@ class FileScanRDD( } override def close(): Unit = { - updateBytesRead() + incTaskInputMetricsBytesRead() InputFileBlockHolder.unset() } } @@ -215,21 +211,6 @@ class FileScanRDD( override protected def getPartitions: Array[RDDPartition] = filePartitions.toArray override protected def getPreferredLocations(split: RDDPartition): Seq[String] = { - val files = split.asInstanceOf[FilePartition].files - - // Computes total number of bytes can be retrieved from each host. - val hostToNumBytes = mutable.HashMap.empty[String, Long] - files.foreach { file => - file.locations.filter(_ != "localhost").foreach { host => - hostToNumBytes(host) = hostToNumBytes.getOrElse(host, 0L) + file.length - } - } - - // Takes the first 3 hosts with the most data to be retrieved - hostToNumBytes.toSeq.sortBy { - case (host, numBytes) => numBytes - }.reverse.take(3).map { - case (host, numBytes) => host - } + split.asInstanceOf[FilePartition].preferredLocations() } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala index fe27b78bf3360..c8a42f043f15f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategy.scala @@ -147,15 +147,7 @@ object FileSourceStrategy extends Strategy with Logging { // - filters that need to be evaluated again after the scan val filterSet = ExpressionSet(filters) - // The attribute name of predicate could be different than the one in schema in case of - // case insensitive, we should change them to match the one in schema, so we do not need to - // worry about case sensitivity anymore. - val normalizedFilters = filters.map { e => - e transform { - case a: AttributeReference => - a.withName(l.output.find(_.semanticEquals(a)).get.name) - } - } + val normalizedFilters = DataSourceStrategy.normalizeFilters(filters, l.output) val partitionColumns = l.resolve( @@ -163,14 +155,18 @@ object FileSourceStrategy extends Strategy with Logging { val partitionSet = AttributeSet(partitionColumns) val partitionKeyFilters = ExpressionSet(normalizedFilters - .filterNot(SubqueryExpression.hasSubquery(_)) .filter(_.references.subsetOf(partitionSet))) logInfo(s"Pruning directories with: ${partitionKeyFilters.mkString(",")}") + // subquery expressions are filtered out because they can't be used to prune buckets or pushed + // down as data filters, yet they would be executed + val normalizedFiltersWithoutSubqueries = + normalizedFilters.filterNot(SubqueryExpression.hasSubquery) + val bucketSpec: Option[BucketSpec] = fsRelation.bucketSpec val bucketSet = if (shouldPruneBuckets(bucketSpec)) { - genBucketSet(normalizedFilters, bucketSpec.get) + genBucketSet(normalizedFiltersWithoutSubqueries, bucketSpec.get) } else { None } @@ -179,7 +175,8 @@ object FileSourceStrategy extends Strategy with Logging { l.resolve(fsRelation.dataSchema, fsRelation.sparkSession.sessionState.analyzer.resolver) // Partition keys are not available in the statistics of the files. - val dataFilters = normalizedFilters.filter(_.references.intersect(partitionSet).isEmpty) + val dataFilters = + normalizedFiltersWithoutSubqueries.filter(_.references.intersect(partitionSet).isEmpty) // Predicates with both partition keys and attributes need to be evaluated after the scan. val afterScanFilters = filterSet -- partitionKeyFilters.filter(_.references.nonEmpty) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala index 00a78f7343c59..57082b40e1132 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileLinesReader.scala @@ -51,7 +51,7 @@ class HadoopFileLinesReader( new Path(new URI(file.filePath)), file.start, file.length, - // TODO: Implement Locality + // The locality is decided by `getPreferredLocations` in `FileScanRDD`. Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala index c61a89e6e8c3f..f5724f7c5955d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFileWholeTextReader.scala @@ -40,7 +40,7 @@ class HadoopFileWholeTextReader(file: PartitionedFile, conf: Configuration) Array(new Path(new URI(file.filePath))), Array(file.start), Array(file.length), - // TODO: Implement Locality + // The locality is decided by `getPreferredLocations` in `FileScanRDD`. Array.empty[String]) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala index b2f73b7f8d1fc..d278802e6c9f2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelation.scala @@ -52,28 +52,12 @@ case class HadoopFsRelation( override def sqlContext: SQLContext = sparkSession.sqlContext - private def getColName(f: StructField): String = { - if (sparkSession.sessionState.conf.caseSensitiveAnalysis) { - f.name - } else { - f.name.toLowerCase(Locale.ROOT) - } - } - - val overlappedPartCols = mutable.Map.empty[String, StructField] - partitionSchema.foreach { partitionField => - if (dataSchema.exists(getColName(_) == getColName(partitionField))) { - overlappedPartCols += getColName(partitionField) -> partitionField - } - } - // When data and partition schemas have overlapping columns, the output // schema respects the order of the data schema for the overlapping columns, and it // respects the data types of the partition schema. - val schema: StructType = { - StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f), f)) ++ - partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f)))) - } + val (schema: StructType, overlappedPartCols: Map[String, StructField]) = + PartitioningUtils.mergeDataAndPartitionSchema(dataSchema, + partitionSchema, sparkSession.sessionState.conf.caseSensitiveAnalysis) def partitionSchemaOption: Option[StructType] = if (partitionSchema.isEmpty) None else Some(partitionSchema) 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 fe418e610da8f..cf7a13050f66c 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.hdfs.DistributedFileSystem import org.apache.hadoop.mapred.{FileInputFormat, JobConf} import org.apache.spark.SparkContext @@ -124,7 +125,7 @@ class InMemoryFileIndex( } val filter = FileInputFormat.getInputPathFilter(new JobConf(hadoopConf, this.getClass)) val discovered = InMemoryFileIndex.bulkListLeafFiles( - pathsToFetch, hadoopConf, filter, sparkSession) + pathsToFetch, hadoopConf, filter, sparkSession, areRootPaths = true) discovered.foreach { case (path, leafFiles) => HiveCatalogMetrics.incrementFilesDiscovered(leafFiles.size) fileStatusCache.putLeafFiles(path, leafFiles.toArray) @@ -166,12 +167,22 @@ object InMemoryFileIndex extends Logging { paths: Seq[Path], hadoopConf: Configuration, filter: PathFilter, - sparkSession: SparkSession): Seq[(Path, Seq[FileStatus])] = { + sparkSession: SparkSession, + areRootPaths: Boolean): Seq[(Path, Seq[FileStatus])] = { + + val ignoreMissingFiles = sparkSession.sessionState.conf.ignoreMissingFiles // Short-circuits parallel listing when serial listing is likely to be faster. if (paths.size <= sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold) { return paths.map { path => - (path, listLeafFiles(path, hadoopConf, filter, Some(sparkSession))) + val leafFiles = listLeafFiles( + path, + hadoopConf, + filter, + Some(sparkSession), + ignoreMissingFiles = ignoreMissingFiles, + isRootPath = areRootPaths) + (path, leafFiles) } } @@ -204,7 +215,14 @@ object InMemoryFileIndex extends Logging { .mapPartitions { pathStrings => val hadoopConf = serializableConfiguration.value pathStrings.map(new Path(_)).toSeq.map { path => - (path, listLeafFiles(path, hadoopConf, filter, None)) + val leafFiles = listLeafFiles( + path, + hadoopConf, + filter, + None, + ignoreMissingFiles = ignoreMissingFiles, + isRootPath = areRootPaths) + (path, leafFiles) }.iterator }.map { case (path, statuses) => val serializableStatuses = statuses.map { status => @@ -267,15 +285,49 @@ object InMemoryFileIndex extends Logging { path: Path, hadoopConf: Configuration, filter: PathFilter, - sessionOpt: Option[SparkSession]): Seq[FileStatus] = { + sessionOpt: Option[SparkSession], + ignoreMissingFiles: Boolean, + isRootPath: Boolean): Seq[FileStatus] = { logTrace(s"Listing $path") val fs = path.getFileSystem(hadoopConf) - // [SPARK-17599] Prevent InMemoryFileIndex from failing if path doesn't exist // Note that statuses only include FileStatus for the files and dirs directly under path, // and does not include anything else recursively. - val statuses = try fs.listStatus(path) catch { - case _: FileNotFoundException => + val statuses: Array[FileStatus] = try { + fs match { + // DistributedFileSystem overrides listLocatedStatus to make 1 single call to namenode + // 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 => + val remoteIter = fs.listLocatedStatus(path) + new Iterator[LocatedFileStatus]() { + def next(): LocatedFileStatus = remoteIter.next + def hasNext(): Boolean = remoteIter.hasNext + }.toArray + case _ => fs.listStatus(path) + } + } catch { + // If we are listing a root path (e.g. a top level directory of a table), we need to + // ignore FileNotFoundExceptions during this root level of the listing because + // + // (a) certain code paths might construct an InMemoryFileIndex with root paths that + // might not exist (i.e. not all callers are guaranteed to have checked + // path existence prior to constructing InMemoryFileIndex) and, + // (b) we need to ignore deleted root paths during REFRESH TABLE, otherwise we break + // existing behavior and break the ability drop SessionCatalog tables when tables' + // root directories have been deleted (which breaks a number of Spark's own tests). + // + // If we are NOT listing a root path then a FileNotFoundException here means that the + // directory was present in a previous level of file listing but is absent in this + // listing, likely indicating a race condition (e.g. concurrent table overwrite or S3 + // list inconsistency). + // + // The trade-off in supporting existing behaviors / use-cases is that we won't be + // able to detect race conditions involving root paths being deleted during + // InMemoryFileIndex construction. However, it's still a net improvement to detect and + // fail-fast on the non-root cases. For more info see the SPARK-27676 review discussion. + case _: FileNotFoundException if isRootPath || ignoreMissingFiles => logWarning(s"The directory $path was not found. Was it deleted very recently?") Array.empty[FileStatus] } @@ -286,9 +338,23 @@ object InMemoryFileIndex extends Logging { val (dirs, topLevelFiles) = filteredStatuses.partition(_.isDirectory) val nestedFiles: Seq[FileStatus] = sessionOpt match { case Some(session) => - bulkListLeafFiles(dirs.map(_.getPath), hadoopConf, filter, session).flatMap(_._2) + bulkListLeafFiles( + dirs.map(_.getPath), + hadoopConf, + filter, + session, + areRootPaths = false + ).flatMap(_._2) case _ => - dirs.flatMap(dir => listLeafFiles(dir.getPath, hadoopConf, filter, sessionOpt)) + dirs.flatMap { dir => + listLeafFiles( + dir.getPath, + hadoopConf, + filter, + sessionOpt, + ignoreMissingFiles = ignoreMissingFiles, + isRootPath = false) + } } val allFiles = topLevelFiles ++ nestedFiles if (filter != null) allFiles.filter(f => filter.accept(f.getPath)) else allFiles @@ -308,7 +374,7 @@ object InMemoryFileIndex extends Logging { // implementations don't actually issue RPC for this method. // // - Here we are calling `getFileBlockLocations` in a sequential manner, but it should not - // be a big deal since we always use to `listLeafFilesInParallel` when the number of + // be a big deal since we always use to `bulkListLeafFiles` when the number of // paths exceeds threshold. case f => // The other constructor of LocatedFileStatus will call FileStatus.getPermission(), @@ -330,7 +396,7 @@ object InMemoryFileIndex extends Logging { } Some(lfs) } catch { - case _: FileNotFoundException => + case _: FileNotFoundException if ignoreMissingFiles => missingFiles += f.getPath.toString None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala index 8d715f6342988..db3604fe92cc0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala @@ -21,8 +21,8 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.expressions.{AttributeMap, AttributeReference} import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.sources.BaseRelation -import org.apache.spark.util.Utils /** * Used to link a [[BaseRelation]] in to a logical query plan. @@ -63,7 +63,9 @@ case class LogicalRelation( case _ => // Do nothing. } - override def simpleString: String = s"Relation[${Utils.truncatedString(output, ",")}] $relation" + override def simpleString(maxFields: Int): String = { + s"Relation[${truncatedString(output, ",", maxFields)}] $relation" + } } object LogicalRelation { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala index cc8af7b92c454..3adec2f790730 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PartitioningAwareFileIndex.scala @@ -56,17 +56,34 @@ abstract class PartitioningAwareFileIndex( protected def leafDirToChildrenFiles: Map[Path, Array[FileStatus]] + protected lazy val pathGlobFilter = parameters.get("pathGlobFilter").map(new GlobFilter(_)) + + protected def matchGlobPattern(file: FileStatus): Boolean = { + pathGlobFilter.forall(_.accept(file.getPath)) + } + + protected lazy val recursiveFileLookup = { + parameters.getOrElse("recursiveFileLookup", "false").toBoolean + } + override def listFiles( partitionFilters: Seq[Expression], dataFilters: Seq[Expression]): Seq[PartitionDirectory] = { + def isNonEmptyFile(f: FileStatus): Boolean = { + isDataPath(f.getPath) && f.getLen > 0 + } val selectedPartitions = if (partitionSpec().partitionColumns.isEmpty) { - PartitionDirectory(InternalRow.empty, allFiles().filter(f => isDataPath(f.getPath))) :: Nil + PartitionDirectory(InternalRow.empty, allFiles().filter(isNonEmptyFile)) :: Nil } else { + if (recursiveFileLookup) { + throw new IllegalArgumentException( + "Datasource with partition do not allow recursive file loading.") + } prunePartitions(partitionFilters, partitionSpec()).map { case PartitionPath(values, path) => val files: Seq[FileStatus] = leafDirToChildrenFiles.get(path) match { case Some(existingDir) => // Directory has children files in it, return them - existingDir.filter(f => isDataPath(f.getPath)) + existingDir.filter(f => matchGlobPattern(f) && isNonEmptyFile(f)) case None => // Directory does not exist, or has no children files @@ -86,10 +103,10 @@ abstract class PartitioningAwareFileIndex( override def sizeInBytes: Long = allFiles().map(_.getLen).sum def allFiles(): Seq[FileStatus] = { - if (partitionSpec().partitionColumns.isEmpty) { + val files = if (partitionSpec().partitionColumns.isEmpty && !recursiveFileLookup) { // For each of the root input paths, get the list of files inside them rootPaths.flatMap { path => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + // Make the path qualified (consistent with listLeafFiles and bulkListLeafFiles). val fs = path.getFileSystem(hadoopConf) val qualifiedPathPre = fs.makeQualified(path) val qualifiedPath: Path = if (qualifiedPathPre.isRoot && !qualifiedPathPre.isAbsolute) { @@ -115,43 +132,30 @@ abstract class PartitioningAwareFileIndex( } else { leafFiles.values.toSeq } + files.filter(matchGlobPattern) } protected def inferPartitioning(): PartitionSpec = { - // We use leaf dirs containing data files to discover the schema. - val leafDirs = leafDirToChildrenFiles.filter { case (_, files) => - files.exists(f => isDataPath(f.getPath)) - }.keys.toSeq - - val caseInsensitiveOptions = CaseInsensitiveMap(parameters) - val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) - .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) - val inferredPartitionSpec = PartitioningUtils.parsePartitions( - leafDirs, - typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, - basePaths = basePaths, - timeZoneId = timeZoneId) - userSpecifiedSchema match { - case Some(userProvidedSchema) if userProvidedSchema.nonEmpty => - val userPartitionSchema = - combineInferredAndUserSpecifiedPartitionSchema(inferredPartitionSpec) - - // we need to cast into the data type that user specified. - def castPartitionValuesToUserSchema(row: InternalRow) = { - InternalRow((0 until row.numFields).map { i => - val dt = inferredPartitionSpec.partitionColumns.fields(i).dataType - Cast( - Literal.create(row.get(i, dt), dt), - userPartitionSchema.fields(i).dataType, - Option(timeZoneId)).eval() - }: _*) - } - - PartitionSpec(userPartitionSchema, inferredPartitionSpec.partitions.map { part => - part.copy(values = castPartitionValuesToUserSchema(part.values)) - }) - case _ => - inferredPartitionSpec + if (recursiveFileLookup) { + PartitionSpec.emptySpec + } else { + // We use leaf dirs containing data files to discover the schema. + val leafDirs = leafDirToChildrenFiles.filter { case (_, files) => + files.exists(f => isDataPath(f.getPath)) + }.keys.toSeq + + val caseInsensitiveOptions = CaseInsensitiveMap(parameters) + val timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) + .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone) + + PartitioningUtils.parsePartitions( + leafDirs, + typeInference = sparkSession.sessionState.conf.partitionColumnTypeInferenceEnabled, + basePaths = basePaths, + userSpecifiedSchema = userSpecifiedSchema, + caseSensitive = sparkSession.sqlContext.conf.caseSensitiveAnalysis, + validatePartitionColumns = sparkSession.sqlContext.conf.validatePartitionColumns, + timeZoneId = timeZoneId) } } @@ -221,7 +225,7 @@ abstract class PartitioningAwareFileIndex( case None => rootPaths.map { path => - // Make the path qualified (consistent with listLeafFiles and listLeafFilesInParallel). + // Make the path qualified (consistent with listLeafFiles and bulkListLeafFiles). val qualifiedPath = path.getFileSystem(hadoopConf).makeQualified(path) if (leafFiles.contains(qualifiedPath)) qualifiedPath.getParent else qualifiedPath }.toSet } @@ -233,25 +237,6 @@ abstract class PartitioningAwareFileIndex( val name = path.getName !((name.startsWith("_") && !name.contains("=")) || name.startsWith(".")) } - - /** - * In the read path, only managed tables by Hive provide the partition columns properly when - * initializing this class. All other file based data sources will try to infer the partitioning, - * and then cast the inferred types to user specified dataTypes if the partition columns exist - * inside `userSpecifiedSchema`, otherwise we can hit data corruption bugs like SPARK-18510, or - * inconsistent data types as reported in SPARK-21463. - * @param spec A partition inference result - * @return The PartitionSchema resolved from inference and cast according to `userSpecifiedSchema` - */ - private def combineInferredAndUserSpecifiedPartitionSchema(spec: PartitionSpec): StructType = { - val equality = sparkSession.sessionState.conf.resolver - val resolved = spec.partitionColumns.map { partitionField => - // SPARK-18510: try to get schema from userSpecifiedSchema, otherwise fallback to inferred - userSpecifiedSchema.flatMap(_.find(f => equality(f.name, partitionField.name))).getOrElse( - partitionField) - } - StructType(resolved) - } } object PartitioningAwareFileIndex { 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 43ba12dd04765..b00a5fa50c45e 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 @@ -19,8 +19,10 @@ package org.apache.spark.sql.execution.datasources import java.lang.{Double => JDouble, Long => JLong} import java.math.{BigDecimal => JBigDecimal} -import java.util.{Locale, TimeZone} +import java.time.ZoneId +import java.util.Locale +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.Try @@ -30,8 +32,8 @@ import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.{Resolver, TypeCoercion} import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec -import org.apache.spark.sql.catalyst.expressions.{Attribute, Cast, Literal} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal} +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.types._ import org.apache.spark.sql.util.SchemaUtils @@ -58,14 +60,14 @@ object PartitionSpec { object PartitioningUtils { + val timestampPartitionPattern = "yyyy-MM-dd HH:mm:ss[.S]" + private[datasources] case class PartitionValues(columnNames: Seq[String], literals: Seq[Literal]) { require(columnNames.size == literals.size) } - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.DEFAULT_PARTITION_NAME - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.escapePathName - import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.unescapePathName + import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils.{escapePathName, unescapePathName, DEFAULT_PARTITION_NAME} /** * Given a group of qualified paths, tries to parse them and returns a partition specification. @@ -94,18 +96,46 @@ object PartitioningUtils { paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, + validatePartitionColumns: Boolean, timeZoneId: String): PartitionSpec = { - parsePartitions(paths, typeInference, basePaths, DateTimeUtils.getTimeZone(timeZoneId)) + parsePartitions(paths, typeInference, basePaths, userSpecifiedSchema, caseSensitive, + validatePartitionColumns, DateTimeUtils.getZoneId(timeZoneId)) } private[datasources] def parsePartitions( paths: Seq[Path], typeInference: Boolean, basePaths: Set[Path], - timeZone: TimeZone): PartitionSpec = { + userSpecifiedSchema: Option[StructType], + caseSensitive: Boolean, + validatePartitionColumns: Boolean, + zoneId: ZoneId): PartitionSpec = { + val userSpecifiedDataTypes = if (userSpecifiedSchema.isDefined) { + val nameToDataType = userSpecifiedSchema.get.fields.map(f => f.name -> f.dataType).toMap + if (!caseSensitive) { + CaseInsensitiveMap(nameToDataType) + } else { + nameToDataType + } + } else { + Map.empty[String, DataType] + } + + // SPARK-26990: use user specified field names if case insensitive. + val userSpecifiedNames = if (userSpecifiedSchema.isDefined && !caseSensitive) { + CaseInsensitiveMap(userSpecifiedSchema.get.fields.map(f => f.name -> f.name).toMap) + } else { + Map.empty[String, String] + } + + val dateFormatter = DateFormatter() + val timestampFormatter = TimestampFormatter(timestampPartitionPattern, zoneId) // 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, timeZone) + parsePartition(path, typeInference, basePaths, userSpecifiedDataTypes, + validatePartitionColumns, zoneId, dateFormatter, timestampFormatter) }.unzip // We create pairs of (path -> path's partition value) here @@ -139,7 +169,8 @@ object PartitioningUtils { "root directory of the table. If there are multiple root directories, " + "please load them separately and then union them.") - val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues, timeZone) + val resolvedPartitionValues = + resolvePartitions(pathsWithPartitionValues, caseSensitive, zoneId) // Creates the StructType which represents the partition columns. val fields = { @@ -147,7 +178,9 @@ object PartitioningUtils { columnNames.zip(literals).map { case (name, Literal(_, dataType)) => // We always assume partition columns are nullable since we've no idea whether null values // will be appended in the future. - StructField(name, dataType, nullable = true) + val resultName = userSpecifiedNames.getOrElse(name, name) + val resultDataType = userSpecifiedDataTypes.getOrElse(name, dataType) + StructField(resultName, resultDataType, nullable = true) } } @@ -185,7 +218,11 @@ object PartitioningUtils { path: Path, typeInference: Boolean, basePaths: Set[Path], - timeZone: TimeZone): (Option[PartitionValues], Option[Path]) = { + userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): (Option[PartitionValues], Option[Path]) = { val columns = ArrayBuffer.empty[(String, Literal)] // Old Hadoop versions don't have `Path.isRoot` var finished = path.getParent == null @@ -206,7 +243,8 @@ object PartitioningUtils { // Let's say currentPath is a path of "/table/a=1/", currentPath.getName will give us a=1. // Once we get the string, we try to parse it and find the partition column and value. val maybeColumn = - parsePartitionColumn(currentPath.getName, typeInference, timeZone) + parsePartitionColumn(currentPath.getName, typeInference, userSpecifiedDataTypes, + validatePartitionColumns, zoneId, dateFormatter, timestampFormatter) maybeColumn.foreach(columns += _) // Now, we determine if we should stop. @@ -239,7 +277,11 @@ object PartitioningUtils { private def parsePartitionColumn( columnSpec: String, typeInference: Boolean, - timeZone: TimeZone): Option[(String, Literal)] = { + userSpecifiedDataTypes: Map[String, DataType], + validatePartitionColumns: Boolean, + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): Option[(String, Literal)] = { val equalSignIndex = columnSpec.indexOf('=') if (equalSignIndex == -1) { None @@ -250,7 +292,31 @@ object PartitioningUtils { val rawColumnValue = columnSpec.drop(equalSignIndex + 1) assert(rawColumnValue.nonEmpty, s"Empty partition column value in '$columnSpec'") - val literal = inferPartitionColumnValue(rawColumnValue, typeInference, timeZone) + val literal = if (userSpecifiedDataTypes.contains(columnName)) { + // SPARK-26188: if user provides corresponding column schema, get the column value without + // inference, and then cast it as user specified data type. + val dataType = userSpecifiedDataTypes(columnName) + val columnValueLiteral = inferPartitionColumnValue( + rawColumnValue, + false, + zoneId, + dateFormatter, + timestampFormatter) + val columnValue = columnValueLiteral.eval() + val castedValue = Cast(columnValueLiteral, dataType, Option(zoneId.getId)).eval() + if (validatePartitionColumns && columnValue != null && castedValue == null) { + throw new RuntimeException(s"Failed to cast value `$columnValue` to `$dataType` " + + s"for partition column `$columnName`") + } + Literal.create(castedValue, dataType) + } else { + inferPartitionColumnValue( + rawColumnValue, + typeInference, + zoneId, + dateFormatter, + timestampFormatter) + } Some(columnName -> literal) } } @@ -326,22 +392,25 @@ object PartitioningUtils { */ def resolvePartitions( pathsWithPartitionValues: Seq[(Path, PartitionValues)], - timeZone: TimeZone): Seq[PartitionValues] = { + caseSensitive: Boolean, + zoneId: ZoneId): Seq[PartitionValues] = { if (pathsWithPartitionValues.isEmpty) { Seq.empty } else { - // TODO: Selective case sensitivity. - val distinctPartColNames = - pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())).distinct + val partColNames = if (caseSensitive) { + pathsWithPartitionValues.map(_._2.columnNames) + } else { + pathsWithPartitionValues.map(_._2.columnNames.map(_.toLowerCase())) + } assert( - distinctPartColNames.size == 1, + partColNames.distinct.size == 1, listConflictingPartitionColumns(pathsWithPartitionValues)) // Resolves possible type conflicts for each column val values = pathsWithPartitionValues.map(_._2) val columnCount = values.head.columnNames.size val resolvedValues = (0 until columnCount).map { i => - resolveTypeConflicts(values.map(_.literals(i)), timeZone) + resolveTypeConflicts(values.map(_.literals(i)), zoneId) } // Fills resolved literals back to each partition @@ -407,7 +476,9 @@ object PartitioningUtils { private[datasources] def inferPartitionColumnValue( raw: String, typeInference: Boolean, - timeZone: TimeZone): Literal = { + zoneId: ZoneId, + dateFormatter: DateFormatter, + timestampFormatter: TimestampFormatter): Literal = { val decimalTry = Try { // `BigDecimal` conversion can fail when the `field` is not a form of number. val bigDecimal = new JBigDecimal(raw) @@ -422,7 +493,7 @@ object PartitioningUtils { val dateTry = Try { // try and parse the date, if no exception occurs this is a candidate to be resolved as // DateType - DateTimeUtils.getThreadLocalDateFormat(DateTimeUtils.defaultTimeZone()).parse(raw) + dateFormatter.parse(raw) // SPARK-23436: Casting the string to date may still return null if a bad Date is provided. // This can happen since DateFormat.parse may not use the entire text of the given string: // so if there are extra-characters after the date, it returns correctly. @@ -439,9 +510,9 @@ object PartitioningUtils { val unescapedRaw = unescapePathName(raw) // try and parse the date, if no exception occurs this is a candidate to be resolved as // TimestampType - DateTimeUtils.getThreadLocalTimestampFormat(timeZone).parse(unescapedRaw) + timestampFormatter.parse(unescapedRaw) // SPARK-23436: see comment for date - val timestampValue = Cast(Literal(unescapedRaw), TimestampType, Some(timeZone.getID)).eval() + val timestampValue = Cast(Literal(unescapedRaw), TimestampType, Some(zoneId.getId)).eval() // Disallow TimestampType if the cast returned null require(timestampValue != null) Literal.create(timestampValue, TimestampType) @@ -504,6 +575,35 @@ object PartitioningUtils { }).asNullable } + def mergeDataAndPartitionSchema( + dataSchema: StructType, + partitionSchema: StructType, + caseSensitive: Boolean): (StructType, Map[String, StructField]) = { + val overlappedPartCols = mutable.Map.empty[String, StructField] + partitionSchema.foreach { partitionField => + val partitionFieldName = getColName(partitionField, caseSensitive) + if (dataSchema.exists(getColName(_, caseSensitive) == partitionFieldName)) { + overlappedPartCols += partitionFieldName -> partitionField + } + } + + // When data and partition schemas have overlapping columns, the output + // schema respects the order of the data schema for the overlapping columns, and it + // respects the data types of the partition schema. + val fullSchema = + StructType(dataSchema.map(f => overlappedPartCols.getOrElse(getColName(f, caseSensitive), f)) ++ + partitionSchema.filterNot(f => overlappedPartCols.contains(getColName(f, caseSensitive)))) + (fullSchema, overlappedPartCols.toMap) + } + + def getColName(f: StructField, caseSensitive: Boolean): String = { + if (caseSensitive) { + f.name + } else { + f.name.toLowerCase(Locale.ROOT) + } + } + private def columnNameEquality(caseSensitive: Boolean): (String, String) => Boolean = { if (caseSensitive) { org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution @@ -516,12 +616,12 @@ object PartitioningUtils { * Given a collection of [[Literal]]s, resolves possible type conflicts by * [[findWiderTypeForPartitionColumn]]. */ - private def resolveTypeConflicts(literals: Seq[Literal], timeZone: TimeZone): Seq[Literal] = { + private def resolveTypeConflicts(literals: Seq[Literal], zoneId: ZoneId): Seq[Literal] = { val litTypes = literals.map(_.dataType) val desiredType = litTypes.reduce(findWiderTypeForPartitionColumn) literals.map { case l @ Literal(_, dataType) => - Literal.create(Cast(l, desiredType, Some(timeZone.getID)).eval(), desiredType) + Literal.create(Cast(l, desiredType, Some(zoneId.getId)).eval(), desiredType) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala index 16b2367bfdd5c..9db7c30b23207 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/PruneFileSourcePartitions.scala @@ -39,15 +39,8 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { _, _)) if filters.nonEmpty && fsRelation.partitionSchemaOption.isDefined => - // The attribute name of predicate could be different than the one in schema in case of - // case insensitive, we should change them to match the one in schema, so we donot need to - // worry about case sensitivity anymore. - val normalizedFilters = filters.map { e => - e transform { - case a: AttributeReference => - a.withName(logicalRelation.output.find(_.semanticEquals(a)).get.name) - } - } + val normalizedFilters = DataSourceStrategy.normalizeFilters( + filters.filterNot(SubqueryExpression.hasSubquery), logicalRelation.output) val sparkSession = fsRelation.sparkSession val partitionColumns = @@ -56,7 +49,6 @@ private[sql] object PruneFileSourcePartitions extends Rule[LogicalPlan] { val partitionSet = AttributeSet(partitionColumns) val partitionKeyFilters = ExpressionSet(normalizedFilters - .filterNot(SubqueryExpression.hasSubquery(_)) .filter(_.references.subsetOf(partitionSet))) if (partitionKeyFilters.nonEmpty) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala index 00b1b5dedb593..f29e7869fb27c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommand.scala @@ -48,7 +48,7 @@ case class SaveIntoDataSourceCommand( Seq.empty[Row] } - override def simpleString: String = { + override def simpleString(maxFields: Int): String = { val redacted = SQLConf.get.redactOptions(options) s"SaveIntoDataSourceCommand ${dataSource}, ${redacted}, ${mode}" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala new file mode 100644 index 0000000000000..99882b0f7c7b0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaMergeUtils.scala @@ -0,0 +1,106 @@ +/* + * 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 org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.SparkException +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +object SchemaMergeUtils extends Logging { + /** + * Figures out a merged Parquet/ORC schema with a distributed Spark job. + */ + def mergeSchemasInParallel( + sparkSession: SparkSession, + files: Seq[FileStatus], + schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]) + : Option[StructType] = { + val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) + + // !! HACK ALERT !! + // Here is a hack for Parquet, but it can be used by Orc as well. + // + // Parquet requires `FileStatus`es to read footers. + // Here we try to send cached `FileStatus`es to executor side to avoid fetching them again. + // However, `FileStatus` is not `Serializable` + // but only `Writable`. What makes it worse, for some reason, `FileStatus` doesn't play well + // with `SerializableWritable[T]` and always causes a weird `IllegalStateException`. These + // facts virtually prevents us to serialize `FileStatus`es. + // + // Since Parquet only relies on path and length information of those `FileStatus`es to read + // footers, here we just extract them (which can be easily serialized), send them to executor + // side, and resemble fake `FileStatus`es there. + val partialFileStatusInfo = files.map(f => (f.getPath.toString, f.getLen)) + + // Set the number of partitions to prevent following schema reads from generating many tasks + // in case of a small number of orc files. + val numParallelism = Math.min(Math.max(partialFileStatusInfo.size, 1), + sparkSession.sparkContext.defaultParallelism) + + val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles + + // Issues a Spark job to read Parquet/ORC schema in parallel. + val partiallyMergedSchemas = + sparkSession + .sparkContext + .parallelize(partialFileStatusInfo, numParallelism) + .mapPartitions { iterator => + // Resembles fake `FileStatus`es with serialized path and length information. + val fakeFileStatuses = iterator.map { case (path, length) => + new FileStatus(length, false, 0, 0, 0, 0, null, null, null, new Path(path)) + }.toSeq + + val schemas = schemaReader(fakeFileStatuses, serializedConf.value, ignoreCorruptFiles) + + if (schemas.isEmpty) { + Iterator.empty + } else { + var mergedSchema = schemas.head + schemas.tail.foreach { schema => + try { + mergedSchema = mergedSchema.merge(schema) + } catch { case cause: SparkException => + throw new SparkException( + s"Failed merging schema:\n${schema.treeString}", cause) + } + } + Iterator.single(mergedSchema) + } + }.collect() + + if (partiallyMergedSchemas.isEmpty) { + None + } else { + var finalSchema = partiallyMergedSchemas.head + partiallyMergedSchemas.tail.foreach { schema => + try { + finalSchema = finalSchema.merge(schema) + } catch { case cause: SparkException => + throw new SparkException( + s"Failed merging schema:\n${schema.treeString}", cause) + } + } + Some(finalSchema) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala new file mode 100644 index 0000000000000..c2211cccb501c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/SchemaPruning.scala @@ -0,0 +1,238 @@ +/* + * 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 org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans.logical.{Filter, LeafNode, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} + +/** + * Prunes unnecessary physical columns given a [[PhysicalOperation]] over a data source relation. + * By "physical column", we mean a column as defined in the data source format like Parquet format + * or ORC format. For example, in Spark SQL, a root-level Parquet column corresponds to a SQL + * column, and a nested Parquet column corresponds to a [[StructField]]. + */ +object SchemaPruning extends Rule[LogicalPlan] { + import org.apache.spark.sql.catalyst.expressions.SchemaPruning._ + + override def apply(plan: LogicalPlan): LogicalPlan = + if (SQLConf.get.nestedSchemaPruningEnabled) { + apply0(plan) + } else { + plan + } + + private def apply0(plan: LogicalPlan): LogicalPlan = + plan transformDown { + case op @ PhysicalOperation(projects, filters, + l @ LogicalRelation(hadoopFsRelation: HadoopFsRelation, _, _, _)) + if canPruneRelation(hadoopFsRelation) => + + prunePhysicalColumns(l.output, projects, filters, hadoopFsRelation.dataSchema, + prunedDataSchema => { + val prunedHadoopRelation = + hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession) + buildPrunedRelation(l, prunedHadoopRelation) + }).getOrElse(op) + + case op @ PhysicalOperation(projects, filters, + d @ DataSourceV2Relation(table: FileTable, output, _)) if canPruneTable(table) => + + prunePhysicalColumns(output, projects, filters, table.dataSchema, + prunedDataSchema => { + val prunedFileTable = table match { + case o: OrcTable => o.copy(userSpecifiedSchema = Some(prunedDataSchema)) + case p: ParquetTable => p.copy(userSpecifiedSchema = Some(prunedDataSchema)) + case _ => + val message = s"${table.formatName} data source doesn't support schema pruning." + throw new AnalysisException(message) + } + buildPrunedRelationV2(d, prunedFileTable) + }).getOrElse(op) + } + + /** + * This method returns optional logical plan. `None` is returned if no nested field is required or + * all nested fields are required. + */ + private def prunePhysicalColumns( + output: Seq[AttributeReference], + projects: Seq[NamedExpression], + filters: Seq[Expression], + dataSchema: StructType, + leafNodeBuilder: StructType => LeafNode): Option[LogicalPlan] = { + val (normalizedProjects, normalizedFilters) = + normalizeAttributeRefNames(output, projects, filters) + val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) + + // If requestedRootFields includes a nested field, continue. Otherwise, + // return op + if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { + val prunedDataSchema = pruneDataSchema(dataSchema, requestedRootFields) + + // If the data schema is different from the pruned data schema, continue. Otherwise, + // return op. We effect this comparison by counting the number of "leaf" fields in + // each schemata, assuming the fields in prunedDataSchema are a subset of the fields + // in dataSchema. + if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) { + val prunedRelation = leafNodeBuilder(prunedDataSchema) + val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) + + Some(buildNewProjection(normalizedProjects, normalizedFilters, prunedRelation, + projectionOverSchema)) + } else { + None + } + } else { + None + } + } + + /** + * Checks to see if the given relation can be pruned. Currently we support Parquet and ORC v1. + */ + private def canPruneRelation(fsRelation: HadoopFsRelation) = + fsRelation.fileFormat.isInstanceOf[ParquetFileFormat] || + fsRelation.fileFormat.isInstanceOf[OrcFileFormat] + + /** + * Checks to see if the given [[FileTable]] can be pruned. Currently we support ORC v2. + */ + private def canPruneTable(table: FileTable) = + table.isInstanceOf[OrcTable] || table.isInstanceOf[ParquetTable] + + /** + * Normalizes the names of the attribute references in the given projects and filters to reflect + * the names in the given logical relation. This makes it possible to compare attributes and + * fields by name. Returns a tuple with the normalized projects and filters, respectively. + */ + private def normalizeAttributeRefNames( + output: Seq[AttributeReference], + projects: Seq[NamedExpression], + filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { + val normalizedAttNameMap = output.map(att => (att.exprId, att.name)).toMap + val normalizedProjects = projects.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }).map { case expr: NamedExpression => expr } + val normalizedFilters = filters.map(_.transform { + case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => + att.withName(normalizedAttNameMap(att.exprId)) + }) + (normalizedProjects, normalizedFilters) + } + + /** + * Builds the new output [[Project]] Spark SQL operator that has the `leafNode`. + */ + private def buildNewProjection( + projects: Seq[NamedExpression], + filters: Seq[Expression], + leafNode: LeafNode, + projectionOverSchema: ProjectionOverSchema): Project = { + // Construct a new target for our projection by rewriting and + // including the original filters where available + val projectionChild = + if (filters.nonEmpty) { + val projectedFilters = filters.map(_.transformDown { + case projectionOverSchema(expr) => expr + }) + val newFilterCondition = projectedFilters.reduce(And) + Filter(newFilterCondition, leafNode) + } else { + leafNode + } + + // Construct the new projections of our Project by + // rewriting the original projections + val newProjects = projects.map(_.transformDown { + case projectionOverSchema(expr) => expr + }).map { case expr: NamedExpression => expr } + + if (log.isDebugEnabled) { + logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") + } + + Project(newProjects, projectionChild) + } + + /** + * Builds a pruned logical relation from the output of the output relation and the schema of the + * pruned base relation. + */ + private def buildPrunedRelation( + outputRelation: LogicalRelation, + prunedBaseRelation: HadoopFsRelation) = { + val prunedOutput = getPrunedOutput(outputRelation.output, prunedBaseRelation.schema) + outputRelation.copy(relation = prunedBaseRelation, output = prunedOutput) + } + + /** + * Builds a pruned data source V2 relation from the output of the relation and the schema + * of the pruned [[FileTable]]. + */ + private def buildPrunedRelationV2( + outputRelation: DataSourceV2Relation, + prunedFileTable: FileTable) = { + val prunedOutput = getPrunedOutput(outputRelation.output, prunedFileTable.schema) + outputRelation.copy(table = prunedFileTable, output = prunedOutput) + } + + // Prune the given output to make it consistent with `requiredSchema`. + private def getPrunedOutput( + output: Seq[AttributeReference], + requiredSchema: StructType): Seq[AttributeReference] = { + // We need to replace the expression ids of the pruned relation output attributes + // with the expression ids of the original relation output attributes so that + // references to the original relation's output are not broken + val outputIdMap = output.map(att => (att.name, att.exprId)).toMap + requiredSchema + .toAttributes + .map { + case att if outputIdMap.contains(att.name) => + att.withExprId(outputIdMap(att.name)) + case att => att + } + } + + /** + * Counts the "leaf" fields of the given dataType. Informally, this is the + * number of fields of non-complex data type in the tree representation of + * [[DataType]]. + */ + private def countLeaves(dataType: DataType): Int = { + dataType match { + case array: ArrayType => countLeaves(array.elementType) + case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) + case struct: StructType => + struct.map(field => countLeaves(field.dataType)).sum + case _ => 1 + } + } + + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala new file mode 100644 index 0000000000000..fda4e148b640f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormat.scala @@ -0,0 +1,198 @@ +/* + * 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.binaryfile + +import java.net.URI +import java.sql.Timestamp + +import com.google.common.io.{ByteStreams, Closeables} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileStatus, GlobFilter, Path} +import org.apache.hadoop.mapreduce.Job + +import org.apache.spark.SparkException +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriterFactory, PartitionedFile} +import org.apache.spark.sql.internal.SQLConf.SOURCES_BINARY_FILE_MAX_LENGTH +import org.apache.spark.sql.sources.{And, DataSourceRegister, EqualTo, Filter, GreaterThan, GreaterThanOrEqual, LessThan, LessThanOrEqual, Not, Or} +import org.apache.spark.sql.types._ +import org.apache.spark.unsafe.types.UTF8String +import org.apache.spark.util.SerializableConfiguration + + +/** + * The binary file data source. + * + * It reads binary files and converts each file into a single record that contains the raw content + * and metadata of the file. + * + * Example: + * {{{ + * // Scala + * val df = spark.read.format("binaryFile") + * .load("/path/to/fileDir") + * + * // Java + * Dataset df = spark.read().format("binaryFile") + * .load("/path/to/fileDir"); + * }}} + */ +class BinaryFileFormat extends FileFormat with DataSourceRegister { + + import BinaryFileFormat._ + + override def inferSchema( + sparkSession: SparkSession, + options: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = Some(schema) + + override def prepareWrite( + sparkSession: SparkSession, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + throw new UnsupportedOperationException("Write is not supported for binary file data source") + } + + override def isSplitable( + sparkSession: SparkSession, + options: Map[String, String], + path: Path): Boolean = { + false + } + + override def shortName(): String = BINARY_FILE + + override protected def buildReader( + sparkSession: SparkSession, + dataSchema: StructType, + partitionSchema: StructType, + requiredSchema: StructType, + filters: Seq[Filter], + options: Map[String, String], + hadoopConf: Configuration): PartitionedFile => Iterator[InternalRow] = { + require(dataSchema.sameType(schema), + s""" + |Binary file data source expects dataSchema: $schema, + |but got: $dataSchema. + """.stripMargin) + + val broadcastedHadoopConf = + sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) + val filterFuncs = filters.map(filter => createFilterFunction(filter)) + val maxLength = sparkSession.conf.get(SOURCES_BINARY_FILE_MAX_LENGTH) + + file: PartitionedFile => { + val path = new Path(new URI(file.filePath)) + val fs = path.getFileSystem(broadcastedHadoopConf.value.value) + val status = fs.getFileStatus(path) + if (filterFuncs.forall(_.apply(status))) { + val writer = new UnsafeRowWriter(requiredSchema.length) + writer.resetRowWriter() + requiredSchema.fieldNames.zipWithIndex.foreach { + case (PATH, i) => writer.write(i, UTF8String.fromString(status.getPath.toString)) + case (LENGTH, i) => writer.write(i, status.getLen) + case (MODIFICATION_TIME, i) => + writer.write(i, DateTimeUtils.fromMillis(status.getModificationTime)) + case (CONTENT, i) => + if (status.getLen > maxLength) { + throw new SparkException( + s"The length of ${status.getPath} is ${status.getLen}, " + + s"which exceeds the max length allowed: ${maxLength}.") + } + val stream = fs.open(status.getPath) + try { + writer.write(i, ByteStreams.toByteArray(stream)) + } finally { + Closeables.close(stream, true) + } + case (other, _) => + throw new RuntimeException(s"Unsupported field name: ${other}") + } + Iterator.single(writer.getRow) + } else { + Iterator.empty + } + } + } +} + +object BinaryFileFormat { + + private[binaryfile] val PATH = "path" + private[binaryfile] val MODIFICATION_TIME = "modificationTime" + private[binaryfile] val LENGTH = "length" + private[binaryfile] val CONTENT = "content" + private[binaryfile] val BINARY_FILE = "binaryFile" + + /** + * Schema for the binary file data source. + * + * Schema: + * - path (StringType): The path of the file. + * - modificationTime (TimestampType): The modification time of the file. + * In some Hadoop FileSystem implementation, this might be unavailable and fallback to some + * default value. + * - length (LongType): The length of the file in bytes. + * - content (BinaryType): The content of the file. + */ + val schema = StructType( + StructField(PATH, StringType, false) :: + StructField(MODIFICATION_TIME, TimestampType, false) :: + StructField(LENGTH, LongType, false) :: + StructField(CONTENT, BinaryType, true) :: Nil) + + private[binaryfile] def createFilterFunction(filter: Filter): FileStatus => Boolean = { + filter match { + case And(left, right) => + s => createFilterFunction(left)(s) && createFilterFunction(right)(s) + case Or(left, right) => + s => createFilterFunction(left)(s) || createFilterFunction(right)(s) + case Not(child) => + s => !createFilterFunction(child)(s) + + case LessThan(LENGTH, value: Long) => + _.getLen < value + case LessThanOrEqual(LENGTH, value: Long) => + _.getLen <= value + case GreaterThan(LENGTH, value: Long) => + _.getLen > value + case GreaterThanOrEqual(LENGTH, value: Long) => + _.getLen >= value + case EqualTo(LENGTH, value: Long) => + _.getLen == value + + case LessThan(MODIFICATION_TIME, value: Timestamp) => + _.getModificationTime < value.getTime + case LessThanOrEqual(MODIFICATION_TIME, value: Timestamp) => + _.getModificationTime <= value.getTime + case GreaterThan(MODIFICATION_TIME, value: Timestamp) => + _.getModificationTime > value.getTime + case GreaterThanOrEqual(MODIFICATION_TIME, value: Timestamp) => + _.getModificationTime >= value.getTime + case EqualTo(MODIFICATION_TIME, value: Timestamp) => + _.getModificationTime == value.getTime + + case _ => (_ => true) + } + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala index 0b5a719d427c9..375cec597166c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVDataSource.scala @@ -34,6 +34,8 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVInferSchema, CSVOptions, UnivocityParser} +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat import org.apache.spark.sql.types.StructType @@ -94,7 +96,7 @@ object TextInputCSVDataSource extends CSVDataSource { headerChecker: CSVHeaderChecker, requiredSchema: StructType): Iterator[InternalRow] = { val lines = { - val linesReader = new HadoopFileLinesReader(file, conf) + val linesReader = new HadoopFileLinesReader(file, parser.options.lineSeparatorInRead, conf) Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => linesReader.close())) linesReader.map { line => new String(line.getBytes, 0, line.getLength, parser.options.charset) @@ -134,7 +136,9 @@ object TextInputCSVDataSource extends CSVDataSource { val parser = new CsvParser(parsedOptions.asParserSettings) linesWithoutHeader.map(parser.parseLine) } - CSVInferSchema.infer(tokenRDD, header, parsedOptions) + SQLExecution.withSQLConfPropagated(csv.sparkSession) { + new CSVInferSchema(parsedOptions).infer(tokenRDD, header) + } case _ => // If the first line could not be read, just return the empty schema. StructType(Nil) @@ -191,7 +195,8 @@ object MultiLineCSVDataSource extends CSVDataSource { UnivocityParser.tokenizeStream( CodecStreams.createInputStreamWithCloseResource(lines.getConfiguration, path), shouldDropHeader = false, - new CsvParser(parsedOptions.asParserSettings)) + new CsvParser(parsedOptions.asParserSettings), + encoding = parsedOptions.charset) }.take(1).headOption match { case Some(firstRow) => val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis @@ -202,10 +207,13 @@ object MultiLineCSVDataSource extends CSVDataSource { lines.getConfiguration, new Path(lines.getPath())), parsedOptions.headerFlag, - new CsvParser(parsedOptions.asParserSettings)) + new CsvParser(parsedOptions.asParserSettings), + encoding = parsedOptions.charset) } val sampled = CSVUtils.sample(tokenRDD, parsedOptions) - CSVInferSchema.infer(sampled, header, parsedOptions) + SQLExecution.withSQLConfPropagated(sparkSession) { + new CSVInferSchema(parsedOptions).infer(sampled, header) + } case None => // If the first row could not be read, just return the empty schema. StructType(Nil) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala index 3de1c2d955d20..8abc6fcacd4c5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala @@ -26,6 +26,8 @@ import org.apache.hadoop.mapreduce._ import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityGenerator, UnivocityParser} +import org.apache.spark.sql.catalyst.expressions.ExprUtils import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ @@ -109,20 +111,14 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { sparkSession.sessionState.conf.columnNameOfCorruptRecord) // Check a field requirement for corrupt records here to throw an exception in a driver side - dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex => - val f = dataSchema(corruptFieldIndex) - if (f.dataType != StringType || !f.nullable) { - throw new AnalysisException( - "The field for corrupt records must be string type and nullable") - } - } + ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) if (requiredSchema.length == 1 && requiredSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { throw new AnalysisException( "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the\n" + "referenced columns only include the internal corrupt record column\n" + - s"(named _corrupt_record by default). For example:\n" + + "(named _corrupt_record by default). For example:\n" + "spark.read.schema(schema).csv(file).filter($\"_corrupt_record\".isNotNull).count()\n" + "and spark.read.schema(schema).csv(file).select(\"_corrupt_record\").show().\n" + "Instead, you can cache or save the parsed results and then send the same query.\n" + @@ -134,11 +130,12 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { (file: PartitionedFile) => { val conf = broadcastedHadoopConf.value.value - val parser = new UnivocityParser( - StructType(dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), - StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)), - parsedOptions) - val schema = if (columnPruning) requiredSchema else dataSchema + val actualDataSchema = StructType( + dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val actualRequiredSchema = StructType( + requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val parser = new UnivocityParser(actualDataSchema, actualRequiredSchema, parsedOptions) + val schema = if (columnPruning) actualRequiredSchema else actualDataSchema val isStartOfFile = file.start == 0 val headerChecker = new CSVHeaderChecker( schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) @@ -157,29 +154,13 @@ class CSVFileFormat extends TextBasedFileFormat with DataSourceRegister { override def equals(other: Any): Boolean = other.isInstanceOf[CSVFileFormat] - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match { + override def supportDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true - case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath) + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) case _ => false } } -private[csv] class CsvOutputWriter( - path: String, - dataSchema: StructType, - context: TaskAttemptContext, - params: CSVOptions) extends OutputWriter with Logging { - - private val charset = Charset.forName(params.charset) - - private val writer = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) - - private val gen = new UnivocityGenerator(dataSchema, writer, params) - - override def write(row: InternalRow): Unit = gen.write(row) - - override def close(): Unit = gen.close() -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala index b912f8add3afd..21fabac472f4b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution.datasources.csv import org.apache.spark.rdd.RDD import org.apache.spark.sql.Dataset +import org.apache.spark.sql.catalyst.csv.CSVExprUtils +import org.apache.spark.sql.catalyst.csv.CSVOptions import org.apache.spark.sql.functions._ object CSVUtils { @@ -39,16 +41,6 @@ object CSVUtils { } } - /** - * Filter ignorable rows for CSV iterator (lines empty and starting with `comment`). - * This is currently being used in CSV reading path and CSV schema inference. - */ - def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = { - iter.filter { line => - line.trim.nonEmpty && !line.startsWith(options.comment.toString) - } - } - /** * Skip the given first line so that only data can remain in a dataset. * This is similar with `dropHeaderLine` below and currently being used in CSV schema inference. @@ -67,29 +59,6 @@ object CSVUtils { } } - def skipComments(iter: Iterator[String], options: CSVOptions): Iterator[String] = { - if (options.isCommentSet) { - val commentPrefix = options.comment.toString - iter.dropWhile { line => - line.trim.isEmpty || line.trim.startsWith(commentPrefix) - } - } else { - iter.dropWhile(_.trim.isEmpty) - } - } - - /** - * Extracts header and moves iterator forward so that only data remains in it - */ - def extractHeader(iter: Iterator[String], options: CSVOptions): Option[String] = { - val nonEmptyLines = skipComments(iter, options) - if (nonEmptyLines.hasNext) { - Some(nonEmptyLines.next()) - } else { - None - } - } - /** * Generates a header from the given row which is null-safe and duplicate-safe. */ @@ -132,33 +101,6 @@ object CSVUtils { } } - /** - * Helper method that converts string representation of a character to actual character. - * It handles some Java escaped strings and throws exception if given string is longer than one - * character. - */ - @throws[IllegalArgumentException] - def toChar(str: String): Char = { - if (str.charAt(0) == '\\') { - str.charAt(1) - match { - case 't' => '\t' - case 'r' => '\r' - case 'b' => '\b' - case 'f' => '\f' - case '\"' => '\"' // In case user changes quote char and uses \" as delimiter in options - case '\'' => '\'' - case 'u' if str == """\u0000""" => '\u0000' - case _ => - throw new IllegalArgumentException(s"Unsupported special character for delimiter: $str") - } - } else if (str.length == 1) { - str.charAt(0) - } else { - throw new IllegalArgumentException(s"Delimiter cannot be more than one character: $str") - } - } - /** * Sample CSV dataset as configured by `samplingRatio`. */ @@ -184,4 +126,7 @@ object CSVUtils { csv.sample(withReplacement = false, options.samplingRatio, 1) } } + + def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): Iterator[String] = + CSVExprUtils.filterCommentAndEmpty(iter, options) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala new file mode 100644 index 0000000000000..3ff36bfde3cca --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CsvOutputWriter.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.csv + +import java.nio.charset.Charset + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.{CSVOptions, UnivocityGenerator} +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter} +import org.apache.spark.sql.types.StructType + +class CsvOutputWriter( + path: String, + dataSchema: StructType, + context: TaskAttemptContext, + params: CSVOptions) extends OutputWriter with Logging { + + private var univocityGenerator: Option[UnivocityGenerator] = None + + if (params.headerFlag) { + val gen = getGen() + gen.writeHeaders() + } + + private def getGen(): UnivocityGenerator = univocityGenerator.getOrElse { + val charset = Charset.forName(params.charset) + val os = CodecStreams.createOutputStreamWriter(context, new Path(path), charset) + val newGen = new UnivocityGenerator(dataSchema, os, params) + univocityGenerator = Some(newGen) + newGen + } + + override def write(row: InternalRow): Unit = { + val gen = getGen() + gen.write(row) + } + + override def close(): Unit = univocityGenerator.foreach(_.close()) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala index fdc5e85f3c2ea..042320edea4f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ddl.scala @@ -68,7 +68,7 @@ case class CreateTempViewUsing( s"Temporary view '$tableIdent' should not have specified a database") } - override def argString: String = { + override def argString(maxFields: Int): String = { s"[tableIdent:$tableIdent " + userSpecifiedSchema.map(_ + " ").getOrElse("") + s"replace:$replace " + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala index 1723596de1db2..530d836d9fde3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/DriverRegistry.scala @@ -50,7 +50,7 @@ object DriverRegistry extends Logging { } else { synchronized { if (wrapperMap.get(className).isEmpty) { - val wrapper = new DriverWrapper(cls.newInstance().asInstanceOf[Driver]) + val wrapper = new DriverWrapper(cls.getConstructor().newInstance().asInstanceOf[Driver]) DriverManager.registerDriver(wrapper) wrapperMap(className) = wrapper logTrace(s"Wrapper for $className registered") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala index 7dfbb9d8b5c05..d184f3cb71b1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCOptions.scala @@ -87,7 +87,7 @@ class JDBCOptions( if (subquery.isEmpty) { throw new IllegalArgumentException(s"Option `$JDBC_QUERY_STRING` can not be empty.") } else { - s"(${subquery}) __SPARK_GEN_JDBC_SUBQUERY_NAME_${curId.getAndIncrement()}" + s"(${subquery}) SPARK_GEN_SUBQ_${curId.getAndIncrement()}" } } @@ -137,9 +137,13 @@ class JDBCOptions( |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin ) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala index f15014442e3fb..3cd5cb1647923 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JDBCRelation.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.execution.datasources.jdbc -import java.sql.{Date, Timestamp} - import scala.collection.mutable.ArrayBuffer import org.apache.spark.Partition @@ -26,11 +24,13 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, DataFrame, Row, SaveMode, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} +import org.apache.spark.sql.catalyst.util.DateTimeUtils.{getZoneId, stringToDate, stringToTimestamp} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.jdbc.JdbcDialects import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, DateType, NumericType, StructType, TimestampType} -import org.apache.spark.util.Utils +import org.apache.spark.unsafe.types.UTF8String /** * Instructions on how to partition the table among workers. @@ -85,8 +85,8 @@ private[sql] object JDBCRelation extends Logging { val (column, columnType) = verifyAndGetNormalizedPartitionColumn( schema, partitionColumn.get, resolver, jdbcOptions) - val lowerBoundValue = toInternalBoundValue(lowerBound.get, columnType) - val upperBoundValue = toInternalBoundValue(upperBound.get, columnType) + val lowerBoundValue = toInternalBoundValue(lowerBound.get, columnType, timeZoneId) + val upperBoundValue = toInternalBoundValue(upperBound.get, columnType, timeZoneId) JDBCPartitioningInfo( column, columnType, lowerBoundValue, upperBoundValue, numPartitions.get) } @@ -159,8 +159,9 @@ private[sql] object JDBCRelation extends Logging { val column = schema.find { f => resolver(f.name, columnName) || resolver(dialect.quoteIdentifier(f.name), columnName) }.getOrElse { + val maxNumToStringFields = SQLConf.get.maxToStringFields throw new AnalysisException(s"User-defined partition column $columnName not " + - s"found in the JDBC relation: ${schema.simpleString(Utils.maxNumToStringFields)}") + s"found in the JDBC relation: ${schema.simpleString(maxNumToStringFields)}") } column.dataType match { case _: NumericType | DateType | TimestampType => @@ -173,10 +174,21 @@ private[sql] object JDBCRelation extends Logging { (dialect.quoteIdentifier(column.name), column.dataType) } - private def toInternalBoundValue(value: String, columnType: DataType): Long = columnType match { - case _: NumericType => value.toLong - case DateType => DateTimeUtils.fromJavaDate(Date.valueOf(value)).toLong - case TimestampType => DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(value)) + private def toInternalBoundValue( + value: String, + columnType: DataType, + timeZoneId: String): Long = { + def parse[T](f: UTF8String => Option[T]): T = { + f(UTF8String.fromString(value)).getOrElse { + throw new IllegalArgumentException( + s"Cannot parse the bound value $value as ${columnType.catalogString}") + } + } + columnType match { + case _: NumericType => value.toLong + case DateType => parse(stringToDate).toLong + case TimestampType => parse(stringToTimestamp(_, getZoneId(timeZoneId))) + } } private def toBoundValueInWhereClause( @@ -184,10 +196,12 @@ private[sql] object JDBCRelation extends Logging { columnType: DataType, timeZoneId: String): String = { def dateTimeToString(): String = { - val timeZone = DateTimeUtils.getTimeZone(timeZoneId) val dateTimeStr = columnType match { - case DateType => DateTimeUtils.dateToString(value.toInt, timeZone) - case TimestampType => DateTimeUtils.timestampToString(value, timeZone) + case DateType => DateFormatter().format(value.toInt) + case TimestampType => + val timestampFormatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(timeZoneId)) + DateTimeUtils.timestampToString(timestampFormatter, value) } s"'$dateTimeStr'" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala index edea549748b47..86a27b5afc250 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/jdbc/JdbcUtils.scala @@ -48,6 +48,7 @@ object JdbcUtils extends Logging { * Returns a factory for creating connections to the given JDBC URL. * * @param options - JDBC options that contains url, table and other information. + * @throws IllegalArgumentException if the driver could not open a JDBC connection. */ def createConnectionFactory(options: JDBCOptions): () => Connection = { val driverClass: String = options.driverClass @@ -60,7 +61,11 @@ object JdbcUtils extends Logging { throw new IllegalStateException( s"Did not find registered driver with class $driverClass") } - driver.connect(options.url, options.asConnectionProperties) + val connection: Connection = driver.connect(options.url, options.asConnectionProperties) + require(connection != null, + s"The driver could not open a JDBC connection. Check the URL: ${options.url}") + + connection } } @@ -438,6 +443,10 @@ object JdbcUtils extends Logging { (rs: ResultSet, row: InternalRow, pos: Int) => row.setShort(pos, rs.getShort(pos + 1)) + case ByteType => + (rs: ResultSet, row: InternalRow, pos: Int) => + row.update(pos, rs.getByte(pos + 1)) + case StringType => (rs: ResultSet, row: InternalRow, pos: Int) => // TODO(davies): use getBytes for better performance, if the encoding is UTF-8 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala index 76f58371ae264..7ec2267e3461f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonDataSource.scala @@ -34,6 +34,7 @@ import org.apache.spark.rdd.{BinaryFileRDD, RDD} import org.apache.spark.sql.{Dataset, Encoders, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions} +import org.apache.spark.sql.catalyst.util.FailureSafeParser import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.text.TextFileFormat @@ -106,7 +107,7 @@ object TextInputJsonDataSource extends JsonDataSource { }.getOrElse(CreateJacksonParser.internalRow(_: JsonFactory, _: InternalRow)) SQLExecution.withSQLConfPropagated(json.sparkSession) { - JsonInferSchema.infer(rdd, parsedOptions, rowParser) + new JsonInferSchema(parsedOptions).infer(rdd, rowParser) } } @@ -139,8 +140,7 @@ object TextInputJsonDataSource extends JsonDataSource { input => parser.parse(input, textParser, textToUTF8String), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) linesReader.flatMap(safeParser.parse) } @@ -165,7 +165,7 @@ object MultiLineJsonDataSource extends JsonDataSource { .getOrElse(createParser(_: JsonFactory, _: PortableDataStream)) SQLExecution.withSQLConfPropagated(sparkSession) { - JsonInferSchema.infer[PortableDataStream](sampled, parsedOptions, parser) + new JsonInferSchema(parsedOptions).infer[PortableDataStream](sampled, parser) } } @@ -224,8 +224,7 @@ object MultiLineJsonDataSource extends JsonDataSource { input => parser.parse[InputStream](input, streamParser, partitionedFileString), parser.options.parseMode, schema, - parser.options.columnNameOfCorruptRecord, - parser.options.multiLine) + parser.options.columnNameOfCorruptRecord) safeParser.parse( CodecStreams.createInputStreamWithCloseResource(conf, new Path(new URI(file.filePath)))) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala index a9241afba537b..95a63c3d1e2d9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonFileFormat.scala @@ -26,7 +26,8 @@ import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JacksonParser, JSONOptions, JSONOptionsInRead} +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.json._ import org.apache.spark.sql.catalyst.util.CompressionCodecs import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.sources._ @@ -107,13 +108,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { val actualSchema = StructType(requiredSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) // Check a field requirement for corrupt records here to throw an exception in a driver side - dataSchema.getFieldIndex(parsedOptions.columnNameOfCorruptRecord).foreach { corruptFieldIndex => - val f = dataSchema(corruptFieldIndex) - if (f.dataType != StringType || !f.nullable) { - throw new AnalysisException( - "The field for corrupt records must be string type and nullable") - } - } + ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) if (requiredSchema.length == 1 && requiredSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { @@ -130,7 +125,7 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { } (file: PartitionedFile) => { - val parser = new JacksonParser(actualSchema, parsedOptions) + val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) JsonDataSource(parsedOptions).readFile( broadcastedHadoopConf.value.value, file, @@ -145,54 +140,20 @@ class JsonFileFormat extends TextBasedFileFormat with DataSourceRegister { override def equals(other: Any): Boolean = other.isInstanceOf[JsonFileFormat] - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match { + override def supportDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true - case st: StructType => st.forall { f => supportDataType(f.dataType, isReadPath) } + case st: StructType => st.forall { f => supportDataType(f.dataType) } - case ArrayType(elementType, _) => supportDataType(elementType, isReadPath) + case ArrayType(elementType, _) => supportDataType(elementType) case MapType(keyType, valueType, _) => - supportDataType(keyType, isReadPath) && supportDataType(valueType, isReadPath) + supportDataType(keyType) && supportDataType(valueType) - case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath) + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) case _: NullType => true case _ => false } } - -private[json] class JsonOutputWriter( - path: String, - options: JSONOptions, - dataSchema: StructType, - context: TaskAttemptContext) - extends OutputWriter with Logging { - - private val encoding = options.encoding match { - case Some(charsetName) => Charset.forName(charsetName) - case None => StandardCharsets.UTF_8 - } - - if (JSONOptionsInRead.blacklist.contains(encoding)) { - logWarning(s"The JSON file ($path) was written in the encoding ${encoding.displayName()}" + - " which can be read back by Spark only if multiLine is enabled.") - } - - private val writer = CodecStreams.createOutputStreamWriter( - context, new Path(path), encoding) - - // create the Generator without separator inserted between 2 records - private[this] val gen = new JacksonGenerator(dataSchema, writer, options) - - override def write(row: InternalRow): Unit = { - gen.write(row) - gen.writeLineEnding() - } - - override def close(): Unit = { - gen.close() - writer.close() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.scala new file mode 100644 index 0000000000000..b3cd570cfb1cf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/json/JsonOutputWriter.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.execution.datasources.json + +import java.nio.charset.{Charset, StandardCharsets} + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.json.{JacksonGenerator, JSONOptions, JSONOptionsInRead} +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter} +import org.apache.spark.sql.types.StructType + +class JsonOutputWriter( + path: String, + options: JSONOptions, + dataSchema: StructType, + context: TaskAttemptContext) + extends OutputWriter with Logging { + + private val encoding = options.encoding match { + case Some(charsetName) => Charset.forName(charsetName) + case None => StandardCharsets.UTF_8 + } + + if (JSONOptionsInRead.blacklist.contains(encoding)) { + logWarning(s"The JSON file ($path) was written in the encoding ${encoding.displayName()}" + + " which can be read back by Spark only if multiLine is enabled.") + } + + private var jacksonGenerator: Option[JacksonGenerator] = None + + override def write(row: InternalRow): Unit = { + val gen = jacksonGenerator.getOrElse { + val os = CodecStreams.createOutputStreamWriter(context, new Path(path), encoding) + // create the Generator without separator inserted between 2 records + val newGen = new JacksonGenerator(dataSchema, os, options) + jacksonGenerator = Some(newGen) + newGen + } + + gen.write(row) + gen.writeLineEnding() + } + + override def close(): Unit = jacksonGenerator.foreach(_.close()) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala new file mode 100644 index 0000000000000..e4f9e49c4dd28 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/noop/NoopDataSource.scala @@ -0,0 +1,88 @@ +/* + * 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.noop + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * This is no-op datasource. It does not do anything besides consuming its input. + * This can be useful for benchmarking or to cache data without any additional overhead. + */ +class NoopDataSource extends TableProvider with DataSourceRegister { + override def shortName(): String = "noop" + override def getTable(options: CaseInsensitiveStringMap): Table = NoopTable +} + +private[noop] object NoopTable extends Table with SupportsWrite { + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = NoopWriteBuilder + override def name(): String = "noop-table" + override def schema(): StructType = new StructType() + override def capabilities(): util.Set[TableCapability] = { + Set( + TableCapability.BATCH_WRITE, + TableCapability.STREAMING_WRITE, + TableCapability.TRUNCATE, + TableCapability.ACCEPT_ANY_SCHEMA).asJava + } +} + +private[noop] object NoopWriteBuilder extends WriteBuilder with SupportsTruncate { + override def truncate(): WriteBuilder = this + override def buildForBatch(): BatchWrite = NoopBatchWrite + override def buildForStreaming(): StreamingWrite = NoopStreamingWrite +} + +private[noop] object NoopBatchWrite extends BatchWrite { + override def createBatchWriterFactory(): DataWriterFactory = NoopWriterFactory + override def commit(messages: Array[WriterCommitMessage]): Unit = {} + override def abort(messages: Array[WriterCommitMessage]): Unit = {} +} + +private[noop] object NoopWriterFactory extends DataWriterFactory { + override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = NoopWriter +} + +private[noop] object NoopWriter extends DataWriter[InternalRow] { + override def write(record: InternalRow): Unit = {} + override def commit(): WriterCommitMessage = null + override def abort(): Unit = {} +} + +private[noop] object NoopStreamingWrite extends StreamingWrite { + override def createStreamingWriterFactory(): StreamingDataWriterFactory = + NoopStreamingDataWriterFactory + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} + override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} +} + +private[noop] object NoopStreamingDataWriterFactory extends StreamingDataWriterFactory { + override def createWriter( + partitionId: Int, + taskId: Long, + epochId: Long): DataWriter[InternalRow] = NoopWriter +} 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 4ecc54bd2fd96..6d52d40d6dd03 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 @@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.hadoop.io._ import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} -import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{SpecificInternalRow, UnsafeArrayData} @@ -37,28 +36,34 @@ class OrcDeserializer( private val resultRow = new SpecificInternalRow(requiredSchema.map(_.dataType)) + // `fieldWriters(index)` is + // - null if the respective source column is missing, since the output value + // is always null in this case + // - a function that updates target column `index` otherwise. private val fieldWriters: Array[WritableComparable[_] => Unit] = { requiredSchema.zipWithIndex - // The value of missing columns are always null, do not need writers. - .filterNot { case (_, index) => requestedColIds(index) == -1 } .map { case (f, index) => - val writer = newWriter(f.dataType, new RowUpdater(resultRow)) - (value: WritableComparable[_]) => writer(index, value) + if (requestedColIds(index) == -1) { + null + } else { + val writer = newWriter(f.dataType, new RowUpdater(resultRow)) + (value: WritableComparable[_]) => writer(index, value) + } }.toArray } - private val validColIds = requestedColIds.filterNot(_ == -1) - def deserialize(orcStruct: OrcStruct): InternalRow = { - var i = 0 - while (i < validColIds.length) { - val value = orcStruct.getFieldValue(validColIds(i)) - if (value == null) { - resultRow.setNullAt(i) - } else { - fieldWriters(i)(value) + var targetColumnIndex = 0 + while (targetColumnIndex < fieldWriters.length) { + if (fieldWriters(targetColumnIndex) != null) { + val value = orcStruct.getFieldValue(requestedColIds(targetColumnIndex)) + if (value == null) { + resultRow.setNullAt(targetColumnIndex) + } else { + fieldWriters(targetColumnIndex)(value) + } } - i += 1 + targetColumnIndex += 1 } resultRow } @@ -103,14 +108,13 @@ class OrcDeserializer( updater.set(ordinal, bytes) case DateType => (ordinal, value) => - updater.setInt(ordinal, DateTimeUtils.fromJavaDate(value.asInstanceOf[DateWritable].get)) + updater.setInt(ordinal, DateTimeUtils.fromJavaDate(OrcShimUtils.getSqlDate(value))) case TimestampType => (ordinal, value) => updater.setLong(ordinal, DateTimeUtils.fromJavaTimestamp(value.asInstanceOf[OrcTimestamp])) case DecimalType.Fixed(precision, scale) => (ordinal, value) => - val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() - val v = Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + val v = OrcShimUtils.getDecimal(value) v.changePrecision(precision, scale) updater.set(ordinal, v) @@ -179,6 +183,8 @@ class OrcDeserializer( i += 1 } + // The ORC map will never have null or duplicated map keys, it's safe to create a + // ArrayBasedMapData directly here. updater.set(ordinal, new ArrayBasedMapData(keyArray, valueArray)) case udt: UserDefinedType[_] => newWriter(udt.sqlType, updater) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala index 4574f8247af54..f7c12598da209 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFileFormat.scala @@ -94,7 +94,7 @@ class OrcFileFormat sparkSession: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { - OrcUtils.readSchema(sparkSession, files) + OrcUtils.inferSchema(sparkSession, files, options) } override def prepareWrite( @@ -162,10 +162,12 @@ class OrcFileFormat val resultSchema = StructType(requiredSchema.fields ++ partitionSchema.fields) val sqlConf = sparkSession.sessionState.conf - val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled val enableVectorizedReader = supportBatch(sparkSession, resultSchema) val capacity = sqlConf.orcVectorizedReaderBatchSize - val copyToSpark = sparkSession.sessionState.conf.getConf(SQLConf.ORC_COPY_BATCH_TO_SPARK) + + val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) + OrcConf.MAPRED_INPUT_SCHEMA.setString(hadoopConf, resultSchemaString) + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(hadoopConf, sqlConf.caseSensitiveAnalysis) val broadcastedConf = sparkSession.sparkContext.broadcast(new SerializableConfiguration(hadoopConf)) @@ -190,29 +192,27 @@ class OrcFileFormat assert(requestedColIds.length == requiredSchema.length, "[BUG] requested column IDs do not match required schema") val taskConf = new Configuration(conf) - taskConf.set(OrcConf.INCLUDE_COLUMNS.getAttribute, - requestedColIds.filter(_ != -1).sorted.mkString(",")) val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) - val taskContext = Option(TaskContext.get()) if (enableVectorizedReader) { - val batchReader = new OrcColumnarBatchReader( - enableOffHeapColumnVector && taskContext.isDefined, copyToSpark, capacity) + val batchReader = new OrcColumnarBatchReader(capacity) // SPARK-23399 Register a task completion listener first to call `close()` in all cases. // There is a possibility that `initialize` and `initBatch` hit some errors (like OOM) // after opening a file. val iter = new RecordReaderIterator(batchReader) Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) - + val requestedDataColIds = requestedColIds ++ Array.fill(partitionSchema.length)(-1) + val requestedPartitionColIds = + Array.fill(requiredSchema.length)(-1) ++ Range(0, partitionSchema.length) batchReader.initialize(fileSplit, taskAttemptContext) batchReader.initBatch( - reader.getSchema, - requestedColIds, - requiredSchema.fields, - partitionSchema, + TypeDescription.fromString(resultSchemaString), + resultSchema.fields, + requestedDataColIds, + requestedPartitionColIds, file.partitionValues) iter.asInstanceOf[Iterator[InternalRow]] @@ -238,19 +238,17 @@ class OrcFileFormat } } - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match { + override def supportDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true - case st: StructType => st.forall { f => supportDataType(f.dataType, isReadPath) } + case st: StructType => st.forall { f => supportDataType(f.dataType) } - case ArrayType(elementType, _) => supportDataType(elementType, isReadPath) + case ArrayType(elementType, _) => supportDataType(elementType) case MapType(keyType, valueType, _) => - supportDataType(keyType, isReadPath) && supportDataType(valueType, isReadPath) - - case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath) + supportDataType(keyType) && supportDataType(valueType) - case _: NullType => isReadPath + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) case _ => false } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala deleted file mode 100644 index 2b17b479432fb..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala +++ /dev/null @@ -1,255 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.orc - -import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder -import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder -import org.apache.orc.storage.serde2.io.HiveDecimalWritable - -import org.apache.spark.sql.sources.{And, Filter} -import org.apache.spark.sql.types._ - -/** - * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. - * - * Due to limitation of ORC `SearchArgument` builder, we had to end up with a pretty weird double- - * checking pattern when converting `And`/`Or`/`Not` filters. - * - * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't - * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite - * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using - * existing simpler ones. - * - * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and - * `startNot()` mutate internal state of the builder instance. This forces us to translate all - * convertible filters with a single builder instance. However, before actually converting a filter, - * we've no idea whether it can be recognized by ORC or not. Thus, when an inconvertible filter is - * found, we may already end up with a builder whose internal state is inconsistent. - * - * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then - * try to convert its children. Say we convert `left` child successfully, but find that `right` - * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent - * now. - * - * The workaround employed here is that, for `And`/`Or`/`Not`, we first try to convert their - * children with brand new builders, and only do the actual conversion with the right builder - * instance when the children are proven to be convertible. - * - * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of - * builder methods mentioned above can only be found in test code, where all tested filters are - * known to be convertible. - */ -private[sql] object OrcFilters { - private[sql] def buildTree(filters: Seq[Filter]): Option[Filter] = { - filters match { - case Seq() => None - case Seq(filter) => Some(filter) - case Seq(filter1, filter2) => Some(And(filter1, filter2)) - case _ => // length > 2 - val (left, right) = filters.splitAt(filters.length / 2) - Some(And(buildTree(left).get, buildTree(right).get)) - } - } - - /** - * 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 - - // First, tries to convert each filter individually to see whether it's convertible, and then - // collect all convertible ones to build the final `SearchArgument`. - val convertibleFilters = for { - filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder) - } yield filter - - for { - // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters) - // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) - } yield builder.build() - } - - /** - * Return true if this is a searchable type in ORC. - * Both CharType and VarcharType are cleaned at AstBuilder. - */ - private def isSearchableType(dataType: DataType) = dataType match { - case BinaryType => false - case _: AtomicType => true - case _ => false - } - - /** - * Get PredicateLeafType which is corresponding to the given DataType. - */ - private def getPredicateLeafType(dataType: DataType) = dataType match { - case BooleanType => PredicateLeaf.Type.BOOLEAN - case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG - case FloatType | DoubleType => PredicateLeaf.Type.FLOAT - case StringType => PredicateLeaf.Type.STRING - case DateType => PredicateLeaf.Type.DATE - case TimestampType => PredicateLeaf.Type.TIMESTAMP - case _: DecimalType => PredicateLeaf.Type.DECIMAL - case _ => throw new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") - } - - /** - * Cast literal values for filters. - * - * We need to cast to long because ORC raises exceptions - * at 'checkLiteralType' of SearchArgumentImpl.java. - */ - private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { - case ByteType | ShortType | IntegerType | LongType => - value.asInstanceOf[Number].longValue - case FloatType | DoubleType => - value.asInstanceOf[Number].doubleValue() - case _: DecimalType => - val decimal = value.asInstanceOf[java.math.BigDecimal] - val decimalWritable = new HiveDecimalWritable(decimal.longValue) - decimalWritable.mutateEnforcePrecisionScale(decimal.precision, decimal.scale) - decimalWritable - case _ => value - } - - /** - * Build a SearchArgument and return the builder so far. - */ - private def buildSearchArgument( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) - } - - /** - * @param dataTypeMap a map from the attribute name to its data type. - * @param expression the input filter predicates. - * @param builder the input SearchArgument.Builder. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. - * @return the builder so far. - */ - private def createBuilder( - dataTypeMap: Map[String, DataType], - expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { - def getType(attribute: String): PredicateLeaf.Type = - getPredicateLeafType(dataTypeMap(attribute)) - - import org.apache.spark.sql.sources._ - - expression match { - case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => - for { - lhs <- createBuilder(dataTypeMap, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() - - case (Some(_), None) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) - - case (None, Some(_)) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) - - case _ => None - } - - case Or(left, right) => - for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts = false) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts = false) - lhs <- createBuilder(dataTypeMap, left, - builder.startOr(), canPartialPushDownConjuncts = false) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts = false) - } yield rhs.end() - - case Not(child) => - for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) - negate <- createBuilder(dataTypeMap, - child, builder.startNot(), canPartialPushDownConjuncts = false) - } yield negate.end() - - // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` - // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be - // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). - - case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().equals(attribute, getType(attribute), castedValue).end()) - - case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().nullSafeEquals(attribute, getType(attribute), castedValue).end()) - - case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThan(attribute, getType(attribute), castedValue).end()) - - case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startAnd().lessThanEquals(attribute, getType(attribute), castedValue).end()) - - case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThanEquals(attribute, getType(attribute), castedValue).end()) - - case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - val castedValue = castLiteralValue(value, dataTypeMap(attribute)) - Some(builder.startNot().lessThan(attribute, getType(attribute), castedValue).end()) - - case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().isNull(attribute, getType(attribute)).end()) - - case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startNot().isNull(attribute, getType(attribute)).end()) - - case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) - Some(builder.startAnd().in(attribute, getType(attribute), - castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) - - case _ => None - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala new file mode 100644 index 0000000000000..0b5658715377a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFiltersBase.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.sql.sources.{And, Filter} +import org.apache.spark.sql.types.{AtomicType, BinaryType, DataType} + +/** + * Methods that can be shared when upgrading the built-in Hive. + */ +trait OrcFiltersBase { + + private[sql] def buildTree(filters: Seq[Filter]): Option[Filter] = { + filters match { + case Seq() => None + case Seq(filter) => Some(filter) + case Seq(filter1, filter2) => Some(And(filter1, filter2)) + case _ => // length > 2 + val (left, right) = filters.splitAt(filters.length / 2) + Some(And(buildTree(left).get, buildTree(right).get)) + } + } + + // 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. + protected[sql] def quoteAttributeNameIfNeeded(name: String) : String = { + if (!name.contains("`") && name.contains(".")) { + s"`$name`" + } else { + name + } + } + + /** + * Return true if this is a searchable type in ORC. + * Both CharType and VarcharType are cleaned at AstBuilder. + */ + protected[sql] def isSearchableType(dataType: DataType) = dataType match { + case BinaryType => false + case _: AtomicType => true + case _ => false + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala index 0ad3862f6cf01..25f022bcdde89 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOptions.scala @@ -57,9 +57,20 @@ class OrcOptions( } shortOrcCompressionCodecNames(codecName) } + + /** + * Whether it merges schemas or not. When the given Orc files have different schemas, + * the schemas can be merged. By default use the value specified in SQLConf. + */ + val mergeSchema: Boolean = parameters + .get(MERGE_SCHEMA) + .map(_.toBoolean) + .getOrElse(sqlConf.isOrcSchemaMergingEnabled) } object OrcOptions { + val MERGE_SCHEMA = "mergeSchema" + // The ORC compression short names private val shortOrcCompressionCodecNames = Map( "none" -> "NONE", diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala index 84755bfa301f0..08086bcd91f6e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcOutputWriter.scala @@ -20,14 +20,15 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.hadoop.fs.Path import org.apache.hadoop.io.NullWritable import org.apache.hadoop.mapreduce.TaskAttemptContext -import org.apache.orc.mapred.OrcStruct -import org.apache.orc.mapreduce.OrcOutputFormat +import org.apache.orc.OrcFile +import org.apache.orc.mapred.{OrcOutputFormat => OrcMapRedOutputFormat, OrcStruct} +import org.apache.orc.mapreduce.{OrcMapreduceRecordWriter, OrcOutputFormat} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriter import org.apache.spark.sql.types._ -private[orc] class OrcOutputWriter( +private[sql] class OrcOutputWriter( path: String, dataSchema: StructType, context: TaskAttemptContext) @@ -36,11 +37,17 @@ private[orc] class OrcOutputWriter( private[this] val serializer = new OrcSerializer(dataSchema) private val recordWriter = { - new OrcOutputFormat[OrcStruct]() { + val orcOutputFormat = new OrcOutputFormat[OrcStruct]() { override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { new Path(path) } - }.getRecordWriter(context) + } + val filename = orcOutputFormat.getDefaultWorkFile(context, ".orc") + val options = OrcMapRedOutputFormat.buildOptions(context.getConfiguration) + val writer = OrcFile.createWriter(filename, options) + val recordWriter = new OrcMapreduceRecordWriter[OrcStruct](writer) + OrcUtils.addSparkVersionMetadata(writer) + recordWriter } override def write(row: InternalRow): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala index 90d1268028096..0b9cbecd0d329 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcSerializer.scala @@ -20,8 +20,6 @@ package org.apache.spark.sql.execution.datasources.orc import org.apache.hadoop.io._ import org.apache.orc.TypeDescription import org.apache.orc.mapred.{OrcList, OrcMap, OrcStruct, OrcTimestamp} -import org.apache.orc.storage.common.`type`.HiveDecimal -import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.SpecializedGetters @@ -139,14 +137,7 @@ class OrcSerializer(dataSchema: StructType) { new BytesWritable(getter.getBinary(ordinal)) case DateType => - if (reuseObj) { - val result = new DateWritable() - (getter, ordinal) => - result.set(getter.getInt(ordinal)) - result - } else { - (getter, ordinal) => new DateWritable(getter.getInt(ordinal)) - } + OrcShimUtils.getDateWritable(reuseObj) // The following cases are already expensive, reusing object or not doesn't matter. @@ -156,9 +147,8 @@ class OrcSerializer(dataSchema: StructType) { result.setNanos(ts.getNanos) result - case DecimalType.Fixed(precision, scale) => (getter, ordinal) => - val d = getter.getDecimal(ordinal, precision, scale) - new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) + case DecimalType.Fixed(precision, scale) => + OrcShimUtils.getHiveDecimalWritable(precision, scale) case st: StructType => (getter, ordinal) => val result = createOrcValue(st).asInstanceOf[OrcStruct] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala index 95fb25bf5addb..12d4244e19812 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcUtils.scala @@ -17,21 +17,25 @@ package org.apache.spark.sql.execution.datasources.orc +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Locale import scala.collection.JavaConverters._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} -import org.apache.orc.{OrcFile, Reader, TypeDescription} +import org.apache.orc.{OrcFile, Reader, TypeDescription, Writer} -import org.apache.spark.SparkException +import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.{SPARK_VERSION_METADATA_KEY, SparkSession} import org.apache.spark.sql.catalyst.analysis.caseSensitiveResolution import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.util.quoteIdentifier +import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.types._ +import org.apache.spark.util.{SerializableConfiguration, ThreadUtils} object OrcUtils extends Logging { @@ -80,7 +84,6 @@ object OrcUtils extends Logging { : Option[StructType] = { val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles val conf = sparkSession.sessionState.newHadoopConf() - // TODO: We need to support merge schema. Please see SPARK-11412. files.toIterator.map(file => readSchema(file.getPath, conf, ignoreCorruptFiles)).collectFirst { case Some(schema) => logDebug(s"Reading schema from file $files, got Hive schema string: $schema") @@ -88,6 +91,29 @@ object OrcUtils extends Logging { } } + /** + * Reads ORC file schemas in multi-threaded manner, using native version of ORC. + * This is visible for testing. + */ + def readOrcSchemasInParallel( + files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean): Seq[StructType] = { + ThreadUtils.parmap(files, "readingOrcSchemas", 8) { currentFile => + OrcUtils.readSchema(currentFile.getPath, conf, ignoreCorruptFiles) + .map(s => CatalystSqlParser.parseDataType(s.toString).asInstanceOf[StructType]) + }.flatten + } + + def inferSchema(sparkSession: SparkSession, files: Seq[FileStatus], options: Map[String, String]) + : Option[StructType] = { + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + if (orcOptions.mergeSchema) { + SchemaMergeUtils.mergeSchemasInParallel( + sparkSession, files, OrcUtils.readOrcSchemasInParallel) + } else { + OrcUtils.readSchema(sparkSession, files) + } + } + /** * Returns the requested column ids from the given ORC file. Column id can be -1, which means the * requested column doesn't exist in the ORC file. Returns None if the given ORC file is empty. @@ -119,24 +145,27 @@ object OrcUtils extends Logging { }) } else { if (isCaseSensitive) { - Some(requiredSchema.fieldNames.map { name => - orcFieldNames.indexWhere(caseSensitiveResolution(_, name)) + Some(requiredSchema.fieldNames.zipWithIndex.map { case (name, idx) => + if (orcFieldNames.indexWhere(caseSensitiveResolution(_, name)) != -1) { + idx + } else { + -1 + } }) } else { // Do case-insensitive resolution only if in case-insensitive mode - val caseInsensitiveOrcFieldMap = - orcFieldNames.zipWithIndex.groupBy(_._1.toLowerCase(Locale.ROOT)) - Some(requiredSchema.fieldNames.map { requiredFieldName => + val caseInsensitiveOrcFieldMap = orcFieldNames.groupBy(_.toLowerCase(Locale.ROOT)) + Some(requiredSchema.fieldNames.zipWithIndex.map { case (requiredFieldName, idx) => caseInsensitiveOrcFieldMap .get(requiredFieldName.toLowerCase(Locale.ROOT)) .map { matchedOrcFields => if (matchedOrcFields.size > 1) { // Need to fail if there is ambiguity, i.e. more than one field is matched. - val matchedOrcFieldsString = matchedOrcFields.map(_._1).mkString("[", ", ", "]") + val matchedOrcFieldsString = matchedOrcFields.mkString("[", ", ", "]") throw new RuntimeException(s"""Found duplicate field(s) "$requiredFieldName": """ + s"$matchedOrcFieldsString in case-insensitive mode") } else { - matchedOrcFields.head._2 + idx } }.getOrElse(-1) }) @@ -144,4 +173,28 @@ object OrcUtils extends Logging { } } } + + /** + * Add a metadata specifying Spark version. + */ + def addSparkVersionMetadata(writer: Writer): Unit = { + writer.addUserMetadata(SPARK_VERSION_METADATA_KEY, UTF_8.encode(SPARK_VERSION_SHORT)) + } + + /** + * Given a `StructType` object, this methods converts it to corresponding string representation + * in ORC. + */ + def orcTypeDescriptionString(dt: DataType): String = dt match { + case s: StructType => + val fieldTypes = s.fields.map { f => + s"${quoteIdentifier(f.name)}:${orcTypeDescriptionString(f.dataType)}" + } + s"struct<${fieldTypes.mkString(",")}>" + case a: ArrayType => + s"array<${orcTypeDescriptionString(a.elementType)}>" + case m: MapType => + s"map<${orcTypeDescriptionString(m.keyType)},${orcTypeDescriptionString(m.valueType)}>" + case _ => dt.catalogString + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala index ea4f1592a7c2e..9caa34b2d9652 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala @@ -129,7 +129,7 @@ class ParquetFileFormat conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE) } - if (ParquetOutputFormat.getJobSummaryLevel(conf) == JobSummaryLevel.NONE + if (ParquetOutputFormat.getJobSummaryLevel(conf) != JobSummaryLevel.NONE && !classOf[ParquetOutputCommitter].isAssignableFrom(committerClass)) { // output summary is requested, but the class is not a Parquet Committer logWarning(s"Committer $committerClass is not a ParquetOutputCommitter and cannot" + @@ -161,105 +161,7 @@ class ParquetFileFormat sparkSession: SparkSession, parameters: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { - val parquetOptions = new ParquetOptions(parameters, sparkSession.sessionState.conf) - - // Should we merge schemas from all Parquet part-files? - val shouldMergeSchemas = parquetOptions.mergeSchema - - val mergeRespectSummaries = sparkSession.sessionState.conf.isParquetSchemaRespectSummaries - - val filesByType = splitFiles(files) - - // Sees which file(s) we need to touch in order to figure out the schema. - // - // Always tries the summary files first if users don't require a merged schema. In this case, - // "_common_metadata" is more preferable than "_metadata" because it doesn't contain row - // groups information, and could be much smaller for large Parquet files with lots of row - // groups. If no summary file is available, falls back to some random part-file. - // - // NOTE: Metadata stored in the summary files are merged from all part-files. However, for - // user defined key-value metadata (in which we store Spark SQL schema), Parquet doesn't know - // how to merge them correctly if some key is associated with different values in different - // part-files. When this happens, Parquet simply gives up generating the summary file. This - // implies that if a summary file presents, then: - // - // 1. Either all part-files have exactly the same Spark SQL schema, or - // 2. Some part-files don't contain Spark SQL schema in the key-value metadata at all (thus - // their schemas may differ from each other). - // - // Here we tend to be pessimistic and take the second case into account. Basically this means - // we can't trust the summary files if users require a merged schema, and must touch all part- - // files to do the merge. - val filesToTouch = - if (shouldMergeSchemas) { - // Also includes summary files, 'cause there might be empty partition directories. - - // If mergeRespectSummaries config is true, we assume that all part-files are the same for - // their schema with summary files, so we ignore them when merging schema. - // If the config is disabled, which is the default setting, we merge all part-files. - // In this mode, we only need to merge schemas contained in all those summary files. - // You should enable this configuration only if you are very sure that for the parquet - // part-files to read there are corresponding summary files containing correct schema. - - // As filed in SPARK-11500, the order of files to touch is a matter, which might affect - // the ordering of the output columns. There are several things to mention here. - // - // 1. If mergeRespectSummaries config is false, then it merges schemas by reducing from - // the first part-file so that the columns of the lexicographically first file show - // first. - // - // 2. If mergeRespectSummaries config is true, then there should be, at least, - // "_metadata"s for all given files, so that we can ensure the columns of - // the lexicographically first file show first. - // - // 3. If shouldMergeSchemas is false, but when multiple files are given, there is - // no guarantee of the output order, since there might not be a summary file for the - // lexicographically first file, which ends up putting ahead the columns of - // the other files. However, this should be okay since not enabling - // shouldMergeSchemas means (assumes) all the files have the same schemas. - - val needMerged: Seq[FileStatus] = - if (mergeRespectSummaries) { - Seq.empty - } else { - filesByType.data - } - needMerged ++ filesByType.metadata ++ filesByType.commonMetadata - } else { - // Tries any "_common_metadata" first. Parquet files written by old versions or Parquet - // don't have this. - filesByType.commonMetadata.headOption - // Falls back to "_metadata" - .orElse(filesByType.metadata.headOption) - // Summary file(s) not found, the Parquet file is either corrupted, or different part- - // files contain conflicting user defined metadata (two or more values are associated - // with a same key in different files). In either case, we fall back to any of the - // first part-file, and just assume all schemas are consistent. - .orElse(filesByType.data.headOption) - .toSeq - } - ParquetFileFormat.mergeSchemasInParallel(filesToTouch, sparkSession) - } - - case class FileTypes( - data: Seq[FileStatus], - metadata: Seq[FileStatus], - commonMetadata: Seq[FileStatus]) - - private def splitFiles(allFiles: Seq[FileStatus]): FileTypes = { - val leaves = allFiles.toArray.sortBy(_.getPath.toString) - - FileTypes( - data = leaves.filterNot(f => isSummaryFile(f.getPath)), - metadata = - leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE), - commonMetadata = - leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE)) - } - - private def isSummaryFile(file: Path): Boolean = { - file.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE || - file.getName == ParquetFileWriter.PARQUET_METADATA_FILE + ParquetUtils.inferSchema(sparkSession, parameters, files) } /** @@ -310,6 +212,9 @@ class ParquetFileFormat hadoopConf.set( SQLConf.SESSION_LOCAL_TIMEZONE.key, sparkSession.sessionState.conf.sessionLocalTimeZone) + hadoopConf.setBoolean( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, + sparkSession.sessionState.conf.nestedSchemaPruningEnabled) hadoopConf.setBoolean( SQLConf.CASE_SENSITIVE.key, sparkSession.sessionState.conf.caseSensitiveAnalysis) @@ -352,17 +257,14 @@ class ParquetFileFormat (file: PartitionedFile) => { assert(file.partitionValues.numFields == partitionSchema.size) - val fileSplit = - new FileSplit(new Path(new URI(file.filePath)), file.start, file.length, Array.empty) - val filePath = fileSplit.getPath - + val filePath = new Path(new URI(file.filePath)) val split = new org.apache.parquet.hadoop.ParquetInputSplit( filePath, - fileSplit.getStart, - fileSplit.getStart + fileSplit.getLength, - fileSplit.getLength, - fileSplit.getLocations, + file.start, + file.start + file.length, + file.length, + Array.empty, null) val sharedConf = broadcastedHadoopConf.value.value @@ -372,13 +274,13 @@ class ParquetFileFormat // Try to push down filters when filter push-down is enabled. val pushed = if (enableParquetFilterPushDown) { val parquetSchema = footerFileMetaData.getSchema - val parquetFilters = new ParquetFilters(pushDownDate, pushDownTimestamp, pushDownDecimal, - pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, + pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) filters // Collects all converted Parquet filter predicates. Notice that not all predicates can be // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` // is used here. - .flatMap(parquetFilters.createFilter(parquetSchema, _)) + .flatMap(parquetFilters.createFilter(_)) .reduceOption(FilterApi.and) } else { None @@ -427,11 +329,12 @@ class ParquetFileFormat } else { logDebug(s"Falling back to parquet-mr") // ParquetRecordReader returns UnsafeRow + val readSupport = new ParquetReadSupport(convertTz, enableVectorizedReader = false) val reader = if (pushed.isDefined && enableRecordFilter) { val parquetFilter = FilterCompat.get(pushed.get, null) - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz), parquetFilter) + new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) } else { - new ParquetRecordReader[UnsafeRow](new ParquetReadSupport(convertTz)) + new ParquetRecordReader[UnsafeRow](readSupport) } val iter = new RecordReaderIterator(reader) // SPARK-23457 Register a task completion lister before `initialization`. @@ -456,17 +359,17 @@ class ParquetFileFormat } } - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match { + override def supportDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true - case st: StructType => st.forall { f => supportDataType(f.dataType, isReadPath) } + case st: StructType => st.forall { f => supportDataType(f.dataType) } - case ArrayType(elementType, _) => supportDataType(elementType, isReadPath) + case ArrayType(elementType, _) => supportDataType(elementType) case MapType(keyType, valueType, _) => - supportDataType(keyType, isReadPath) && supportDataType(valueType, isReadPath) + supportDataType(keyType) && supportDataType(valueType) - case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath) + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) case _ => false } @@ -573,79 +476,18 @@ object ParquetFileFormat extends Logging { sparkSession: SparkSession): Option[StructType] = { val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp - val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf()) - - // !! HACK ALERT !! - // - // Parquet requires `FileStatus`es to read footers. Here we try to send cached `FileStatus`es - // to executor side to avoid fetching them again. However, `FileStatus` is not `Serializable` - // but only `Writable`. What makes it worse, for some reason, `FileStatus` doesn't play well - // with `SerializableWritable[T]` and always causes a weird `IllegalStateException`. These - // facts virtually prevents us to serialize `FileStatus`es. - // - // Since Parquet only relies on path and length information of those `FileStatus`es to read - // footers, here we just extract them (which can be easily serialized), send them to executor - // side, and resemble fake `FileStatus`es there. - val partialFileStatusInfo = filesToTouch.map(f => (f.getPath.toString, f.getLen)) - - // Set the number of partitions to prevent following schema reads from generating many tasks - // in case of a small number of parquet files. - val numParallelism = Math.min(Math.max(partialFileStatusInfo.size, 1), - sparkSession.sparkContext.defaultParallelism) - - val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles - - // Issues a Spark job to read Parquet schema in parallel. - val partiallyMergedSchemas = - sparkSession - .sparkContext - .parallelize(partialFileStatusInfo, numParallelism) - .mapPartitions { iterator => - // Resembles fake `FileStatus`es with serialized path and length information. - val fakeFileStatuses = iterator.map { case (path, length) => - new FileStatus(length, false, 0, 0, 0, 0, null, null, null, new Path(path)) - }.toSeq - - // Reads footers in multi-threaded manner within each task - val footers = - ParquetFileFormat.readParquetFootersInParallel( - serializedConf.value, fakeFileStatuses, ignoreCorruptFiles) - - // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` - val converter = new ParquetToSparkSchemaConverter( - assumeBinaryIsString = assumeBinaryIsString, - assumeInt96IsTimestamp = assumeInt96IsTimestamp) - if (footers.isEmpty) { - Iterator.empty - } else { - var mergedSchema = ParquetFileFormat.readSchemaFromFooter(footers.head, converter) - footers.tail.foreach { footer => - val schema = ParquetFileFormat.readSchemaFromFooter(footer, converter) - try { - mergedSchema = mergedSchema.merge(schema) - } catch { case cause: SparkException => - throw new SparkException( - s"Failed merging schema of file ${footer.getFile}:\n${schema.treeString}", cause) - } - } - Iterator.single(mergedSchema) - } - }.collect() - if (partiallyMergedSchemas.isEmpty) { - None - } else { - var finalSchema = partiallyMergedSchemas.head - partiallyMergedSchemas.tail.foreach { schema => - try { - finalSchema = finalSchema.merge(schema) - } catch { case cause: SparkException => - throw new SparkException( - s"Failed merging schema:\n${schema.treeString}", cause) - } - } - Some(finalSchema) + val reader = (files: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) => { + // Converter used to convert Parquet `MessageType` to Spark SQL `StructType` + val converter = new ParquetToSparkSchemaConverter( + assumeBinaryIsString = assumeBinaryIsString, + assumeInt96IsTimestamp = assumeInt96IsTimestamp) + + readParquetFootersInParallel(conf, files, ignoreCorruptFiles) + .map(ParquetFileFormat.readSchemaFromFooter(_, converter)) } + + SchemaMergeUtils.mergeSchemasInParallel(sparkSession, filesToTouch, reader) } /** 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 21ab9c78e53d9..b9b86adb438e6 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 @@ -40,13 +40,39 @@ import org.apache.spark.unsafe.types.UTF8String /** * Some utility function to convert Spark data source filters to Parquet filters. */ -private[parquet] class ParquetFilters( +class ParquetFilters( + schema: MessageType, pushDownDate: Boolean, pushDownTimestamp: Boolean, pushDownDecimal: Boolean, pushDownStartWith: Boolean, 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)) + } + if (caseSensitive) { + primitiveFields.toMap + } else { + // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive + // mode, just skip pushdown for these fields, they will trigger Exception when reading, + // See: SPARK-25132. + val dedupPrimitiveFields = + primitiveFields + .groupBy(_._1.toLowerCase(Locale.ROOT)) + .filter(_._2.size == 1) + .mapValues(_.head._2) + CaseInsensitiveMap(dedupPrimitiveFields) + } + } /** * Holds a single field information stored in the underlying parquet file. @@ -361,45 +387,95 @@ private[parquet] class ParquetFilters( FilterApi.gtEq(binaryColumn(n), decimalToByteArray(v.asInstanceOf[JBigDecimal], length)) } - /** - * Returns a map, which contains parquet field name and data type, if predicate push down applies. - */ - private def getFieldMap(dataType: MessageType): 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 = - dataType.getFields.asScala.filter(_.isPrimitive).map(_.asPrimitiveType()).map { f => - f.getName -> ParquetField(f.getName, - ParquetSchemaType(f.getOriginalType, - f.getPrimitiveTypeName, f.getTypeLength, f.getDecimalMetadata)) - } - if (caseSensitive) { - primitiveFields.toMap - } else { - // Don't consider ambiguity here, i.e. more than one field is matched in case insensitive - // mode, just skip pushdown for these fields, they will trigger Exception when reading, - // See: SPARK-25132. - val dedupPrimitiveFields = - primitiveFields - .groupBy(_._1.toLowerCase(Locale.ROOT)) - .filter(_._2.size == 1) - .mapValues(_.head._2) - CaseInsensitiveMap(dedupPrimitiveFields) + // Returns filters that can be pushed down when reading Parquet files. + def convertibleFilters(filters: Seq[sources.Filter]): Seq[sources.Filter] = { + filters.flatMap(convertibleFiltersHelper(_, canPartialPushDown = true)) + } + + private def convertibleFiltersHelper( + predicate: sources.Filter, + canPartialPushDown: Boolean): Option[sources.Filter] = { + predicate match { + case sources.And(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(sources.And(leftResult, rightResult)) + case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) + case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) + case _ => None + } + + case sources.Or(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + if (leftResultOptional.isEmpty || rightResultOptional.isEmpty) { + None + } else { + Some(sources.Or(leftResultOptional.get, rightResultOptional.get)) + } + case sources.Not(pred) => + val resultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) + resultOptional.map(sources.Not) + + case other => + if (createFilter(other).isDefined) { + Some(other) + } else { + None + } } } /** * Converts data sources filters to Parquet filter predicates. */ - def createFilter(schema: MessageType, predicate: sources.Filter): Option[FilterPredicate] = { - val nameToParquetField = getFieldMap(schema) - createFilterHelper(nameToParquetField, predicate, canPartialPushDownConjuncts = true) + def createFilter(predicate: sources.Filter): Option[FilterPredicate] = { + createFilterHelper(predicate, canPartialPushDownConjuncts = true) + } + + // Parquet's type in the given file should be matched to the value's type + // in the pushed filter in order to push down the filter to Parquet. + private def valueCanMakeFilterOn(name: String, value: Any): Boolean = { + value == null || (nameToParquetField(name).fieldType match { + case ParquetBooleanType => value.isInstanceOf[JBoolean] + case ParquetByteType | ParquetShortType | ParquetIntegerType => value.isInstanceOf[Number] + case ParquetLongType => value.isInstanceOf[JLong] + case ParquetFloatType => value.isInstanceOf[JFloat] + case ParquetDoubleType => value.isInstanceOf[JDouble] + case ParquetStringType => value.isInstanceOf[String] + case ParquetBinaryType => value.isInstanceOf[Array[Byte]] + case ParquetDateType => value.isInstanceOf[Date] + case ParquetTimestampMicrosType | ParquetTimestampMillisType => + value.isInstanceOf[Timestamp] + case ParquetSchemaType(DECIMAL, INT32, _, decimalMeta) => + isDecimalMatched(value, decimalMeta) + case ParquetSchemaType(DECIMAL, INT64, _, decimalMeta) => + isDecimalMatched(value, decimalMeta) + case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, _, decimalMeta) => + isDecimalMatched(value, decimalMeta) + case _ => false + }) + } + + // Decimal type must make sure that filter value's scale matched the file. + // If doesn't matched, which would cause data corruption. + private def isDecimalMatched(value: Any, decimalMeta: DecimalMetadata): Boolean = value match { + case decimal: JBigDecimal => + decimal.scale == decimalMeta.getScale + 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) } /** - * @param nameToParquetField a map from the field name to its field name and data type. - * This only includes the root fields whose types are primitive types. * @param predicate the input filter predicates. Not all the predicates can be pushed down. * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed * down safely. Pushing ONLY one side of AND down is safe to @@ -407,50 +483,8 @@ private[parquet] class ParquetFilters( * @return the Parquet-native filter predicates that are eligible for pushdown. */ private def createFilterHelper( - nameToParquetField: Map[String, ParquetField], predicate: sources.Filter, canPartialPushDownConjuncts: Boolean): Option[FilterPredicate] = { - // Decimal type must make sure that filter value's scale matched the file. - // If doesn't matched, which would cause data corruption. - def isDecimalMatched(value: Any, decimalMeta: DecimalMetadata): Boolean = value match { - case decimal: JBigDecimal => - decimal.scale == decimalMeta.getScale - case _ => false - } - - // Parquet's type in the given file should be matched to the value's type - // in the pushed filter in order to push down the filter to Parquet. - def valueCanMakeFilterOn(name: String, value: Any): Boolean = { - value == null || (nameToParquetField(name).fieldType match { - case ParquetBooleanType => value.isInstanceOf[JBoolean] - case ParquetByteType | ParquetShortType | ParquetIntegerType => value.isInstanceOf[Number] - case ParquetLongType => value.isInstanceOf[JLong] - case ParquetFloatType => value.isInstanceOf[JFloat] - case ParquetDoubleType => value.isInstanceOf[JDouble] - case ParquetStringType => value.isInstanceOf[String] - case ParquetBinaryType => value.isInstanceOf[Array[Byte]] - case ParquetDateType => value.isInstanceOf[Date] - case ParquetTimestampMicrosType | ParquetTimestampMillisType => - value.isInstanceOf[Timestamp] - case ParquetSchemaType(DECIMAL, INT32, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) - case ParquetSchemaType(DECIMAL, INT64, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) - case ParquetSchemaType(DECIMAL, FIXED_LEN_BYTE_ARRAY, _, decimalMeta) => - isDecimalMatched(value, decimalMeta) - 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. - def canMakeFilterOn(name: String, value: Any): Boolean = { - nameToParquetField.contains(name) && !name.contains(".") && valueCanMakeFilterOn(name, value) - } - // NOTE: // // For any comparison operator `cmp`, both `a cmp NULL` and `NULL cmp a` evaluate to `NULL`, @@ -515,9 +549,9 @@ private[parquet] class ParquetFilters( // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate // can be safely removed. val lhsFilterOption = - createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts) + createFilterHelper(lhs, canPartialPushDownConjuncts) val rhsFilterOption = - createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts) + createFilterHelper(rhs, canPartialPushDownConjuncts) (lhsFilterOption, rhsFilterOption) match { case (Some(lhsFilter), Some(rhsFilter)) => Some(FilterApi.and(lhsFilter, rhsFilter)) @@ -527,15 +561,24 @@ private[parquet] class ParquetFilters( } case sources.Or(lhs, rhs) => + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). for { - lhsFilter <- - createFilterHelper(nameToParquetField, lhs, canPartialPushDownConjuncts = false) - rhsFilter <- - createFilterHelper(nameToParquetField, rhs, canPartialPushDownConjuncts = false) + lhsFilter <- createFilterHelper(lhs, canPartialPushDownConjuncts) + rhsFilter <- createFilterHelper(rhs, canPartialPushDownConjuncts) } yield FilterApi.or(lhsFilter, rhsFilter) case sources.Not(pred) => - createFilterHelper(nameToParquetField, pred, canPartialPushDownConjuncts = false) + createFilterHelper(pred, canPartialPushDownConjuncts = false) .map(FilterApi.not) case sources.In(name, values) if canMakeFilterOn(name, values.head) @@ -570,7 +613,7 @@ private[parquet] class ParquetFilters( } override def keep(value: Binary): Boolean = { - UTF8String.fromBytes(value.getBytes).startsWith( + value != null && UTF8String.fromBytes(value.getBytes).startsWith( UTF8String.fromBytes(strToBinary.getBytes)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala index 8361762b09703..e7753cec681cf 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetOutputWriter.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.OutputWriter // NOTE: This class is instantiated and used on executor side only, no need to be serializable. -private[parquet] class ParquetOutputWriter(path: String, context: TaskAttemptContext) +class ParquetOutputWriter(path: String, context: TaskAttemptContext) extends OutputWriter { private val recordWriter: RecordWriter[Void, InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala index 3319e73f2b313..2c7231d2c3e0a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetReadSupport.scala @@ -49,15 +49,16 @@ import org.apache.spark.sql.types._ * Due to this reason, we no longer rely on [[ReadContext]] to pass requested schema from [[init()]] * to [[prepareForRead()]], but use a private `var` for simplicity. */ -private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) - extends ReadSupport[UnsafeRow] with Logging { +class ParquetReadSupport(val convertTz: Option[TimeZone], + enableVectorizedReader: Boolean) + extends ReadSupport[UnsafeRow] with Logging { private var catalystRequestedSchema: StructType = _ def this() { // We need a zero-arg constructor for SpecificParquetRecordReaderBase. But that is only // used in the vectorized reader, where we get the convertTz value directly, and the value here // is ignored. - this(None) + this(None, enableVectorizedReader = true) } /** @@ -65,18 +66,48 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) * readers. Responsible for figuring out Parquet requested schema used for column pruning. */ override def init(context: InitContext): ReadContext = { + val conf = context.getConfiguration catalystRequestedSchema = { - val conf = context.getConfiguration val schemaString = conf.get(ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA) assert(schemaString != null, "Parquet requested schema not set.") StructType.fromString(schemaString) } - val caseSensitive = context.getConfiguration.getBoolean(SQLConf.CASE_SENSITIVE.key, + val caseSensitive = conf.getBoolean(SQLConf.CASE_SENSITIVE.key, SQLConf.CASE_SENSITIVE.defaultValue.get) - val parquetRequestedSchema = ParquetReadSupport.clipParquetSchema( - context.getFileSchema, catalystRequestedSchema, caseSensitive) - + val schemaPruningEnabled = conf.getBoolean(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.defaultValue.get) + val parquetFileSchema = context.getFileSchema + val parquetClippedSchema = ParquetReadSupport.clipParquetSchema(parquetFileSchema, + catalystRequestedSchema, caseSensitive) + + // We pass two schema to ParquetRecordMaterializer: + // - parquetRequestedSchema: the schema of the file data we want to read + // - catalystRequestedSchema: the schema of the rows we want to return + // The reader is responsible for reconciling the differences between the two. + val parquetRequestedSchema = if (schemaPruningEnabled && !enableVectorizedReader) { + // Parquet-MR reader requires that parquetRequestedSchema include only those fields present + // in the underlying parquetFileSchema. Therefore, we intersect the parquetClippedSchema + // with the parquetFileSchema + ParquetReadSupport.intersectParquetGroups(parquetClippedSchema, parquetFileSchema) + .map(groupType => new MessageType(groupType.getName, groupType.getFields)) + .getOrElse(ParquetSchemaConverter.EMPTY_MESSAGE) + } else { + // Spark's vectorized reader only support atomic types currently. It also skip fields + // in parquetRequestedSchema which are not present in the file. + parquetClippedSchema + } + logDebug( + s"""Going to read the following fields from the Parquet file with the following schema: + |Parquet file schema: + |$parquetFileSchema + |Parquet clipped schema: + |$parquetClippedSchema + |Parquet requested schema: + |$parquetRequestedSchema + |Catalyst requested schema: + |${catalystRequestedSchema.treeString} + """.stripMargin) new ReadContext(parquetRequestedSchema, Map.empty[String, String].asJava) } @@ -90,19 +121,7 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) keyValueMetaData: JMap[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[UnsafeRow] = { - log.debug(s"Preparing for read Parquet file with message type: $fileSchema") val parquetRequestedSchema = readContext.getRequestedSchema - - logInfo { - s"""Going to read the following fields from the Parquet file: - | - |Parquet form: - |$parquetRequestedSchema - |Catalyst form: - |$catalystRequestedSchema - """.stripMargin - } - new ParquetRecordMaterializer( parquetRequestedSchema, ParquetReadSupport.expandUDT(catalystRequestedSchema), @@ -111,7 +130,7 @@ private[parquet] class ParquetReadSupport(val convertTz: Option[TimeZone]) } } -private[parquet] object ParquetReadSupport { +object ParquetReadSupport { val SPARK_ROW_REQUESTED_SCHEMA = "org.apache.spark.sql.parquet.row.requested_schema" val SPARK_METADATA_KEY = "org.apache.spark.sql.parquet.row.metadata" @@ -322,6 +341,35 @@ private[parquet] object ParquetReadSupport { } } + /** + * Computes the structural intersection between two Parquet group types. + * This is used to create a requestedSchema for ReadContext of Parquet-MR reader. + * Parquet-MR reader does not support the nested field access to non-existent field + * while parquet library does support to read the non-existent field by regular field access. + */ + private def intersectParquetGroups( + groupType1: GroupType, groupType2: GroupType): Option[GroupType] = { + val fields = + groupType1.getFields.asScala + .filter(field => groupType2.containsField(field.getName)) + .flatMap { + case field1: GroupType => + val field2 = groupType2.getType(field1.getName) + if (field2.isPrimitive) { + None + } else { + intersectParquetGroups(field1, field2.asGroupType) + } + case field1 => Some(field1) + } + + if (fields.nonEmpty) { + Some(groupType1.withNewFields(fields.asJava)) + } else { + None + } + } + def expandUDT(schema: StructType): StructType = { def expand(dataType: DataType): DataType = { dataType match { 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 1199725941842..b772b6b77d1ce 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 @@ -130,8 +130,8 @@ private[parquet] class ParquetRowConverter( extends ParquetGroupConverter(updater) with Logging { assert( - parquetType.getFieldCount == catalystType.length, - s"""Field counts of the Parquet schema and the Catalyst schema don't match: + parquetType.getFieldCount <= catalystType.length, + s"""Field count of the Parquet schema is greater than the field count of the Catalyst schema: | |Parquet schema: |$parquetType @@ -182,10 +182,11 @@ private[parquet] class ParquetRowConverter( // Converters for each field. private val fieldConverters: Array[Converter with HasParentContainerUpdater] = { - parquetType.getFields.asScala.zip(catalystType).zipWithIndex.map { - case ((parquetFieldType, catalystField), ordinal) => - // Converted field value should be set to the `ordinal`-th cell of `currentRow` - newConverter(parquetFieldType, catalystField.dataType, new RowUpdater(currentRow, ordinal)) + parquetType.getFields.asScala.map { parquetField => + val fieldIndex = catalystType.fieldIndex(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)) }.toArray } @@ -193,7 +194,7 @@ private[parquet] class ParquetRowConverter( override def end(): Unit = { var i = 0 - while (i < currentRow.numFields) { + while (i < fieldConverters.length) { fieldConverters(i).updater.end() i += 1 } @@ -203,10 +204,14 @@ private[parquet] class ParquetRowConverter( override def start(): Unit = { var i = 0 while (i < currentRow.numFields) { - fieldConverters(i).updater.start() currentRow.setNullAt(i) i += 1 } + i = 0 + while (i < fieldConverters.length) { + fieldConverters(i).updater.start() + i += 1 + } } /** @@ -558,8 +563,12 @@ private[parquet] class ParquetRowConverter( override def getConverter(fieldIndex: Int): Converter = keyValueConverter - override def end(): Unit = + override def end(): Unit = { + // The parquet map may contains null or duplicated map keys. When it happens, the behavior is + // undefined. + // TODO (SPARK-26174): disallow it with a config. updater.set(ArrayBasedMapData(currentKeys.toArray, currentValues.toArray)) + } // NOTE: We can't reuse the mutable Map here and must instantiate a new `Map` for the next // value. `Row.copy()` only copies row cells, it doesn't do deep copy to objects stored in row diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala deleted file mode 100644 index 91080b15727d6..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruning.scala +++ /dev/null @@ -1,279 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.parquet - -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{ProjectionOverSchema, SelectedField} -import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} -import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.types.{ArrayType, DataType, MapType, StructField, StructType} - -/** - * Prunes unnecessary Parquet columns given a [[PhysicalOperation]] over a - * [[ParquetRelation]]. By "Parquet column", we mean a column as defined in the - * Parquet format. In Spark SQL, a root-level Parquet column corresponds to a - * SQL column, and a nested Parquet column corresponds to a [[StructField]]. - */ -private[sql] object ParquetSchemaPruning extends Rule[LogicalPlan] { - override def apply(plan: LogicalPlan): LogicalPlan = - if (SQLConf.get.nestedSchemaPruningEnabled) { - apply0(plan) - } else { - plan - } - - private def apply0(plan: LogicalPlan): LogicalPlan = - plan transformDown { - case op @ PhysicalOperation(projects, filters, - l @ LogicalRelation(hadoopFsRelation: HadoopFsRelation, _, _, _)) - if canPruneRelation(hadoopFsRelation) => - val (normalizedProjects, normalizedFilters) = - normalizeAttributeRefNames(l, projects, filters) - val requestedRootFields = identifyRootFields(normalizedProjects, normalizedFilters) - - // If requestedRootFields includes a nested field, continue. Otherwise, - // return op - if (requestedRootFields.exists { root: RootField => !root.derivedFromAtt }) { - val dataSchema = hadoopFsRelation.dataSchema - val prunedDataSchema = pruneDataSchema(dataSchema, requestedRootFields) - - // If the data schema is different from the pruned data schema, continue. Otherwise, - // return op. We effect this comparison by counting the number of "leaf" fields in - // each schemata, assuming the fields in prunedDataSchema are a subset of the fields - // in dataSchema. - if (countLeaves(dataSchema) > countLeaves(prunedDataSchema)) { - val prunedParquetRelation = - hadoopFsRelation.copy(dataSchema = prunedDataSchema)(hadoopFsRelation.sparkSession) - - val prunedRelation = buildPrunedRelation(l, prunedParquetRelation) - val projectionOverSchema = ProjectionOverSchema(prunedDataSchema) - - buildNewProjection(normalizedProjects, normalizedFilters, prunedRelation, - projectionOverSchema) - } else { - op - } - } else { - op - } - } - - /** - * Checks to see if the given relation is Parquet and can be pruned. - */ - private def canPruneRelation(fsRelation: HadoopFsRelation) = - fsRelation.fileFormat.isInstanceOf[ParquetFileFormat] - - /** - * Normalizes the names of the attribute references in the given projects and filters to reflect - * the names in the given logical relation. This makes it possible to compare attributes and - * fields by name. Returns a tuple with the normalized projects and filters, respectively. - */ - private def normalizeAttributeRefNames( - logicalRelation: LogicalRelation, - projects: Seq[NamedExpression], - filters: Seq[Expression]): (Seq[NamedExpression], Seq[Expression]) = { - val normalizedAttNameMap = logicalRelation.output.map(att => (att.exprId, att.name)).toMap - val normalizedProjects = projects.map(_.transform { - case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => - att.withName(normalizedAttNameMap(att.exprId)) - }).map { case expr: NamedExpression => expr } - val normalizedFilters = filters.map(_.transform { - case att: AttributeReference if normalizedAttNameMap.contains(att.exprId) => - att.withName(normalizedAttNameMap(att.exprId)) - }) - (normalizedProjects, normalizedFilters) - } - - /** - * Returns the set of fields from the Parquet file that the query plan needs. - */ - private def identifyRootFields(projects: Seq[NamedExpression], filters: Seq[Expression]) = { - val projectionRootFields = projects.flatMap(getRootFields) - val filterRootFields = filters.flatMap(getRootFields) - - // Kind of expressions don't need to access any fields of a root fields, e.g., `IsNotNull`. - // For them, if there are any nested fields accessed in the query, we don't need to add root - // field access of above expressions. - // For example, for a query `SELECT name.first FROM contacts WHERE name IS NOT NULL`, - // we don't need to read nested fields of `name` struct other than `first` field. - val (rootFields, optRootFields) = (projectionRootFields ++ filterRootFields) - .distinct.partition(_.contentAccessed) - - optRootFields.filter { opt => - !rootFields.exists(_.field.name == opt.field.name) - } ++ rootFields - } - - /** - * Builds the new output [[Project]] Spark SQL operator that has the pruned output relation. - */ - private def buildNewProjection( - projects: Seq[NamedExpression], filters: Seq[Expression], prunedRelation: LogicalRelation, - projectionOverSchema: ProjectionOverSchema) = { - // Construct a new target for our projection by rewriting and - // including the original filters where available - val projectionChild = - if (filters.nonEmpty) { - val projectedFilters = filters.map(_.transformDown { - case projectionOverSchema(expr) => expr - }) - val newFilterCondition = projectedFilters.reduce(And) - Filter(newFilterCondition, prunedRelation) - } else { - prunedRelation - } - - // Construct the new projections of our Project by - // rewriting the original projections - val newProjects = projects.map(_.transformDown { - case projectionOverSchema(expr) => expr - }).map { case expr: NamedExpression => expr } - - if (log.isDebugEnabled) { - logDebug(s"New projects:\n${newProjects.map(_.treeString).mkString("\n")}") - } - - Project(newProjects, projectionChild) - } - - /** - * Filters the schema from the given file by the requested fields. - * Schema field ordering from the file is preserved. - */ - private def pruneDataSchema( - fileDataSchema: StructType, - requestedRootFields: Seq[RootField]) = { - // Merge the requested root fields into a single schema. Note the ordering of the fields - // in the resulting schema may differ from their ordering in the logical relation's - // original schema - val mergedSchema = requestedRootFields - .map { case root: RootField => StructType(Array(root.field)) } - .reduceLeft(_ merge _) - val dataSchemaFieldNames = fileDataSchema.fieldNames.toSet - val mergedDataSchema = - StructType(mergedSchema.filter(f => dataSchemaFieldNames.contains(f.name))) - // Sort the fields of mergedDataSchema according to their order in dataSchema, - // recursively. This makes mergedDataSchema a pruned schema of dataSchema - sortLeftFieldsByRight(mergedDataSchema, fileDataSchema).asInstanceOf[StructType] - } - - /** - * Builds a pruned logical relation from the output of the output relation and the schema of the - * pruned base relation. - */ - private def buildPrunedRelation( - outputRelation: LogicalRelation, - prunedBaseRelation: HadoopFsRelation) = { - // We need to replace the expression ids of the pruned relation output attributes - // with the expression ids of the original relation output attributes so that - // references to the original relation's output are not broken - val outputIdMap = outputRelation.output.map(att => (att.name, att.exprId)).toMap - val prunedRelationOutput = - prunedBaseRelation - .schema - .toAttributes - .map { - case att if outputIdMap.contains(att.name) => - att.withExprId(outputIdMap(att.name)) - case att => att - } - outputRelation.copy(relation = prunedBaseRelation, output = prunedRelationOutput) - } - - /** - * Gets the root (aka top-level, no-parent) [[StructField]]s for the given [[Expression]]. - * When expr is an [[Attribute]], construct a field around it and indicate that that - * field was derived from an attribute. - */ - private def getRootFields(expr: Expression): Seq[RootField] = { - expr match { - case att: Attribute => - RootField(StructField(att.name, att.dataType, att.nullable), derivedFromAtt = true) :: Nil - case SelectedField(field) => RootField(field, derivedFromAtt = false) :: Nil - // Root field accesses by `IsNotNull` and `IsNull` are special cases as the expressions - // don't actually use any nested fields. These root field accesses might be excluded later - // if there are any nested fields accesses in the query plan. - case IsNotNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil - case IsNull(SelectedField(field)) => - RootField(field, derivedFromAtt = false, contentAccessed = false) :: Nil - case IsNotNull(_: Attribute) | IsNull(_: Attribute) => - expr.children.flatMap(getRootFields).map(_.copy(contentAccessed = false)) - case _ => - expr.children.flatMap(getRootFields) - } - } - - /** - * Counts the "leaf" fields of the given dataType. Informally, this is the - * number of fields of non-complex data type in the tree representation of - * [[DataType]]. - */ - private def countLeaves(dataType: DataType): Int = { - dataType match { - case array: ArrayType => countLeaves(array.elementType) - case map: MapType => countLeaves(map.keyType) + countLeaves(map.valueType) - case struct: StructType => - struct.map(field => countLeaves(field.dataType)).sum - case _ => 1 - } - } - - /** - * Sorts the fields and descendant fields of structs in left according to their order in - * right. This function assumes that the fields of left are a subset of the fields of - * right, recursively. That is, left is a "subschema" of right, ignoring order of - * fields. - */ - private def sortLeftFieldsByRight(left: DataType, right: DataType): DataType = - (left, right) match { - case (ArrayType(leftElementType, containsNull), ArrayType(rightElementType, _)) => - ArrayType( - sortLeftFieldsByRight(leftElementType, rightElementType), - containsNull) - case (MapType(leftKeyType, leftValueType, containsNull), - MapType(rightKeyType, rightValueType, _)) => - MapType( - sortLeftFieldsByRight(leftKeyType, rightKeyType), - sortLeftFieldsByRight(leftValueType, rightValueType), - containsNull) - case (leftStruct: StructType, rightStruct: StructType) => - val filteredRightFieldNames = rightStruct.fieldNames.filter(leftStruct.fieldNames.contains) - val sortedLeftFields = filteredRightFieldNames.map { fieldName => - val leftFieldType = leftStruct(fieldName).dataType - val rightFieldType = rightStruct(fieldName).dataType - val sortedLeftFieldType = sortLeftFieldsByRight(leftFieldType, rightFieldType) - StructField(fieldName, sortedLeftFieldType) - } - StructType(sortedLeftFields) - case _ => left - } - - /** - * This represents a "root" schema field (aka top-level, no-parent). `field` is the - * `StructField` for field name and datatype. `derivedFromAtt` indicates whether it - * was derived from an attribute or had a proper child. `contentAccessed` means whether - * it was accessed with its content by the expressions refer it. - */ - private case class RootField(field: StructField, derivedFromAtt: Boolean, - contentAccessed: Boolean = true) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala new file mode 100644 index 0000000000000..7e7dba92f37b5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetUtils.scala @@ -0,0 +1,130 @@ +/* + * 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.parquet + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.apache.parquet.hadoop.ParquetFileWriter + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.types.StructType + +object ParquetUtils { + def inferSchema( + sparkSession: SparkSession, + parameters: Map[String, String], + files: Seq[FileStatus]): Option[StructType] = { + val parquetOptions = new ParquetOptions(parameters, sparkSession.sessionState.conf) + + // Should we merge schemas from all Parquet part-files? + val shouldMergeSchemas = parquetOptions.mergeSchema + + val mergeRespectSummaries = sparkSession.sessionState.conf.isParquetSchemaRespectSummaries + + val filesByType = splitFiles(files) + + // Sees which file(s) we need to touch in order to figure out the schema. + // + // Always tries the summary files first if users don't require a merged schema. In this case, + // "_common_metadata" is more preferable than "_metadata" because it doesn't contain row + // groups information, and could be much smaller for large Parquet files with lots of row + // groups. If no summary file is available, falls back to some random part-file. + // + // NOTE: Metadata stored in the summary files are merged from all part-files. However, for + // user defined key-value metadata (in which we store Spark SQL schema), Parquet doesn't know + // how to merge them correctly if some key is associated with different values in different + // part-files. When this happens, Parquet simply gives up generating the summary file. This + // implies that if a summary file presents, then: + // + // 1. Either all part-files have exactly the same Spark SQL schema, or + // 2. Some part-files don't contain Spark SQL schema in the key-value metadata at all (thus + // their schemas may differ from each other). + // + // Here we tend to be pessimistic and take the second case into account. Basically this means + // we can't trust the summary files if users require a merged schema, and must touch all part- + // files to do the merge. + val filesToTouch = + if (shouldMergeSchemas) { + // Also includes summary files, 'cause there might be empty partition directories. + + // If mergeRespectSummaries config is true, we assume that all part-files are the same for + // their schema with summary files, so we ignore them when merging schema. + // If the config is disabled, which is the default setting, we merge all part-files. + // In this mode, we only need to merge schemas contained in all those summary files. + // You should enable this configuration only if you are very sure that for the parquet + // part-files to read there are corresponding summary files containing correct schema. + + // As filed in SPARK-11500, the order of files to touch is a matter, which might affect + // the ordering of the output columns. There are several things to mention here. + // + // 1. If mergeRespectSummaries config is false, then it merges schemas by reducing from + // the first part-file so that the columns of the lexicographically first file show + // first. + // + // 2. If mergeRespectSummaries config is true, then there should be, at least, + // "_metadata"s for all given files, so that we can ensure the columns of + // the lexicographically first file show first. + // + // 3. If shouldMergeSchemas is false, but when multiple files are given, there is + // no guarantee of the output order, since there might not be a summary file for the + // lexicographically first file, which ends up putting ahead the columns of + // the other files. However, this should be okay since not enabling + // shouldMergeSchemas means (assumes) all the files have the same schemas. + + val needMerged: Seq[FileStatus] = + if (mergeRespectSummaries) { + Seq.empty + } else { + filesByType.data + } + needMerged ++ filesByType.metadata ++ filesByType.commonMetadata + } else { + // Tries any "_common_metadata" first. Parquet files written by old versions or Parquet + // don't have this. + filesByType.commonMetadata.headOption + // Falls back to "_metadata" + .orElse(filesByType.metadata.headOption) + // Summary file(s) not found, the Parquet file is either corrupted, or different part- + // files contain conflicting user defined metadata (two or more values are associated + // with a same key in different files). In either case, we fall back to any of the + // first part-file, and just assume all schemas are consistent. + .orElse(filesByType.data.headOption) + .toSeq + } + ParquetFileFormat.mergeSchemasInParallel(filesToTouch, sparkSession) + } + + case class FileTypes( + data: Seq[FileStatus], + metadata: Seq[FileStatus], + commonMetadata: Seq[FileStatus]) + + private def splitFiles(allFiles: Seq[FileStatus]): FileTypes = { + val leaves = allFiles.toArray.sortBy(_.getPath.toString) + + FileTypes( + data = leaves.filterNot(f => isSummaryFile(f.getPath)), + metadata = + leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_METADATA_FILE), + commonMetadata = + leaves.filter(_.getPath.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE)) + } + + private def isSummaryFile(file: Path): Boolean = { + file.getName == ParquetFileWriter.PARQUET_COMMON_METADATA_FILE || + file.getName == ParquetFileWriter.PARQUET_METADATA_FILE + } +} 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 b40b8c2e61f33..f6490614ab05b 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 @@ -29,7 +29,9 @@ import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext import org.apache.parquet.io.api.{Binary, RecordConsumer} +import org.apache.spark.SPARK_VERSION_SHORT import org.apache.spark.internal.Logging +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 @@ -47,7 +49,7 @@ import org.apache.spark.sql.types._ * of this option is propagated to this class by the `init()` method and its Hadoop configuration * argument. */ -private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { +class ParquetWriteSupport extends WriteSupport[InternalRow] with Logging { // A `ValueWriter` is responsible for writing a field of an `InternalRow` to the record consumer. // Here we are using `SpecializedGetters` rather than `InternalRow` so that we can directly access // data in `ArrayData` without the help of `SpecificMutableRow`. @@ -93,7 +95,10 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit this.rootFieldWriters = schema.map(_.dataType).map(makeWriter).toArray[ValueWriter] val messageType = new SparkToParquetSchemaConverter(configuration).convert(schema) - val metadata = Map(ParquetReadSupport.SPARK_METADATA_KEY -> schemaString).asJava + val metadata = Map( + SPARK_VERSION_METADATA_KEY -> SPARK_VERSION_SHORT, + ParquetReadSupport.SPARK_METADATA_KEY -> schemaString + ).asJava logInfo( s"""Initialized Parquet WriteSupport with Catalyst schema: @@ -437,7 +442,7 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit } } -private[parquet] object ParquetWriteSupport { +object ParquetWriteSupport { val SPARK_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.attributes" def setSchema(schema: StructType, configuration: Configuration): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala index 949aa665527ac..e8951bc8e7164 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/rules.scala @@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expres import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.command.DDLUtils +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.InsertableRelation import org.apache.spark.sql.types.{AtomicType, StructType} @@ -36,7 +37,7 @@ import org.apache.spark.sql.util.SchemaUtils */ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { private def maybeSQLFile(u: UnresolvedRelation): Boolean = { - sparkSession.sessionState.conf.runSQLonFile && u.tableIdentifier.database.isDefined + sparkSession.sessionState.conf.runSQLonFile && u.multipartIdentifier.size == 2 } def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators { @@ -44,8 +45,8 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { try { val dataSource = DataSource( sparkSession, - paths = u.tableIdentifier.table :: Nil, - className = u.tableIdentifier.database.get) + paths = u.multipartIdentifier.last :: Nil, + className = u.multipartIdentifier.head) // `dataSource.providingClass` may throw ClassNotFoundException, then the outer try-catch // will catch it and return the original plan, so that the analyzer can report table not @@ -54,7 +55,7 @@ class ResolveSQLOnFile(sparkSession: SparkSession) extends Rule[LogicalPlan] { if (!isFileFormat || dataSource.className.toLowerCase(Locale.ROOT) == DDLUtils.HIVE_PROVIDER) { throw new AnalysisException("Unsupported data source type for direct query on files: " + - s"${u.tableIdentifier.database.get}") + s"${dataSource.className}") } LogicalRelation(dataSource.resolveRelation()) } catch { @@ -113,7 +114,9 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi val specifiedProvider = DataSource.lookupDataSource(tableDesc.provider.get, conf) // TODO: Check that options from the resolved relation match the relation that we are // inserting into (i.e. using the same compression). - if (existingProvider != specifiedProvider) { + // If the one of the provider is [[FileDataSourceV2]] and the other one is its corresponding + // [[FileFormat]], the two providers are considered compatible. + if (fallBackV2ToV1(existingProvider) != fallBackV2ToV1(specifiedProvider)) { throw new AnalysisException(s"The format of the existing table $tableName is " + s"`${existingProvider.getSimpleName}`. It doesn't match the specified format " + s"`${specifiedProvider.getSimpleName}`.") @@ -206,6 +209,8 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi val analyzedQuery = query.get val normalizedTable = normalizeCatalogTable(analyzedQuery.schema, tableDesc) + DDLUtils.checkDataColNames(tableDesc.copy(schema = analyzedQuery.schema)) + val output = analyzedQuery.output val partitionAttrs = normalizedTable.partitionColumnNames.map { partCol => output.find(_.name == partCol).get @@ -219,6 +224,7 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi c.copy(tableDesc = normalizedTable, query = Some(reorderedQuery)) } else { + DDLUtils.checkDataColNames(tableDesc) val normalizedTable = normalizeCatalogTable(tableDesc.schema, tableDesc) val partitionSchema = normalizedTable.partitionColumnNames.map { partCol => @@ -232,6 +238,11 @@ case class PreprocessTableCreation(sparkSession: SparkSession) extends Rule[Logi } } + private def fallBackV2ToV1(cls: Class[_]): Class[_] = cls.newInstance match { + case f: FileDataSourceV2 => f.fallbackFileFormat + case _ => cls + } + private def normalizeCatalogTable(schema: StructType, table: CatalogTable): CatalogTable = { SchemaUtils.checkSchemaColumnNameDuplication( schema, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala index 268297148b522..d8811c708a6b9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextFileFormat.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.types.{DataType, StringType, StructType} import org.apache.spark.util.SerializableConfiguration /** - * A data source for reading text files. + * A data source for reading text files. The text files must be encoded as UTF-8. */ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { @@ -137,28 +137,7 @@ class TextFileFormat extends TextBasedFileFormat with DataSourceRegister { } } - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = + override def supportDataType(dataType: DataType): Boolean = dataType == StringType } -class TextOutputWriter( - path: String, - dataSchema: StructType, - lineSeparator: Array[Byte], - context: TaskAttemptContext) - extends OutputWriter { - - private val writer = CodecStreams.createOutputStream(context, new Path(path)) - - override def write(row: InternalRow): Unit = { - if (!row.isNullAt(0)) { - val utf8string = row.getUTF8String(0) - utf8string.writeTo(writer) - } - writer.write(lineSeparator) - } - - override def close(): Unit = { - writer.close() - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala index e4e201995faa2..ef132162750b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOptions.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CompressionCodecs /** * Options for the Text data source. */ -private[text] class TextOptions(@transient private val parameters: CaseInsensitiveMap[String]) +class TextOptions(@transient private val parameters: CaseInsensitiveMap[String]) extends Serializable { import TextOptions._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.scala new file mode 100644 index 0000000000000..faf6e573105f2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/text/TextOutputWriter.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.text + +import java.io.OutputStream + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.TaskAttemptContext + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter} +import org.apache.spark.sql.types.StructType + +class TextOutputWriter( + path: String, + dataSchema: StructType, + lineSeparator: Array[Byte], + context: TaskAttemptContext) + extends OutputWriter { + + private var outputStream: Option[OutputStream] = None + + override def write(row: InternalRow): Unit = { + val os = outputStream.getOrElse { + val newStream = CodecStreams.createOutputStream(context, new Path(path)) + outputStream = Some(newStream) + newStream + } + + if (!row.isNullAt(0)) { + val utf8string = row.getUTF8String(0) + utf8string.writeTo(os) + } + os.write(lineSeparator) + } + + override def close(): Unit = { + outputStream.foreach(_.close()) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala new file mode 100644 index 0000000000000..a3fa82b12e938 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/AlterTableExec.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.SparkException +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for altering a table. + */ +case class AlterTableExec( + catalog: TableCatalog, + ident: Identifier, + changes: Seq[TableChange]) extends LeafExecNode { + + override def output: Seq[Attribute] = Seq.empty + + override protected def doExecute(): RDD[InternalRow] = { + try { + catalog.alterTable(ident, changes: _*) + } catch { + case e: IllegalArgumentException => + throw new SparkException(s"Unsupported table change: ${e.getMessage}", e) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.scala new file mode 100644 index 0000000000000..c3cbb9d2af4ed --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/BatchScanExec.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.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.sources.v2.reader._ + +/** + * Physical plan node for scanning a batch of data from a data source v2. + */ +case class BatchScanExec( + output: Seq[AttributeReference], + @transient scan: Scan) extends DataSourceV2ScanExecBase { + + @transient lazy val batch = scan.toBatch + + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: BatchScanExec => this.batch == other.batch + case _ => false + } + + override def hashCode(): Int = batch.hashCode() + + override lazy val partitions: Seq[InputPartition] = batch.planInputPartitions() + + override lazy val readerFactory: PartitionReaderFactory = batch.createReaderFactory() + + override lazy val inputRDD: RDD[InternalRow] = { + new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar) + } + + override def doCanonicalize(): BatchScanExec = { + this.copy(output = output.map(QueryPlan.normalizeExprId(_, output))) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.scala new file mode 100644 index 0000000000000..f54ff608a53e3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/ContinuousScanExec.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.execution.datasources.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.execution.streaming.continuous._ +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousStream, Offset} + +/** + * Physical plan node for scanning data from a streaming data source with continuous mode. + */ +case class ContinuousScanExec( + output: Seq[Attribute], + @transient scan: Scan, + @transient stream: ContinuousStream, + @transient start: Offset) extends DataSourceV2ScanExecBase { + + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: ContinuousScanExec => this.stream == other.stream + case _ => false + } + + override def hashCode(): Int = stream.hashCode() + + override lazy val partitions: Seq[InputPartition] = stream.planInputPartitions(start) + + override lazy val readerFactory: ContinuousPartitionReaderFactory = { + stream.createContinuousReaderFactory() + } + + override lazy val inputRDD: RDD[InternalRow] = { + EpochCoordinatorRef.get( + sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), + sparkContext.env) + .askSync[Unit](SetReaderPartitions(partitions.size)) + new ContinuousDataSourceRDD( + sparkContext, + sqlContext.conf.continuousStreamingExecutorQueueSize, + sqlContext.conf.continuousStreamingExecutorPollIntervalMs, + partitions, + schema, + readerFactory.asInstanceOf[ContinuousPartitionReaderFactory]) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala new file mode 100644 index 0000000000000..f35758bf08c67 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/CreateTableExec.scala @@ -0,0 +1,56 @@ +/* + * 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.v2 + +import scala.collection.JavaConverters._ + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.types.StructType + +case class CreateTableExec( + catalog: TableCatalog, + identifier: Identifier, + tableSchema: StructType, + partitioning: Seq[Transform], + tableProperties: Map[String, String], + ignoreIfExists: Boolean) extends LeafExecNode { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + override protected def doExecute(): RDD[InternalRow] = { + if (!catalog.tableExists(identifier)) { + try { + catalog.createTable(identifier, tableSchema, partitioning.toArray, tableProperties.asJava) + } catch { + case _: TableAlreadyExistsException if ignoreIfExists => + logWarning(s"Table ${identifier.quoted} was created concurrently. Ignoring.") + } + } else if (!ignoreIfExists) { + throw new TableAlreadyExistsException(identifier) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala deleted file mode 100644 index f7e29593a6353..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Relation.scala +++ /dev/null @@ -1,189 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.v2 - -import java.util.UUID - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql.{AnalysisException, SaveMode} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, NamedRelation} -import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Expression} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} -import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{BatchReadSupportProvider, BatchWriteSupportProvider, DataSourceOptions, DataSourceV2} -import org.apache.spark.sql.sources.v2.reader.{BatchReadSupport, ReadSupport, ScanConfigBuilder, SupportsReportStatistics} -import org.apache.spark.sql.sources.v2.writer.BatchWriteSupport -import org.apache.spark.sql.types.StructType - -/** - * A logical plan representing a data source v2 scan. - * - * @param source An instance of a [[DataSourceV2]] implementation. - * @param options The options for this scan. Used to create fresh [[BatchWriteSupport]]. - * @param userSpecifiedSchema The user-specified schema for this scan. - */ -case class DataSourceV2Relation( - source: DataSourceV2, - readSupport: BatchReadSupport, - output: Seq[AttributeReference], - options: Map[String, String], - tableIdent: Option[TableIdentifier] = None, - userSpecifiedSchema: Option[StructType] = None) - extends LeafNode with MultiInstanceRelation with NamedRelation with DataSourceV2StringFormat { - - import DataSourceV2Relation._ - - override def name: String = { - tableIdent.map(_.unquotedString).getOrElse(s"${source.name}:unknown") - } - - override def pushedFilters: Seq[Expression] = Seq.empty - - override def simpleString: String = "RelationV2 " + metadataString - - def newWriteSupport(): BatchWriteSupport = source.createWriteSupport(options, schema) - - override def computeStats(): Statistics = readSupport match { - case r: SupportsReportStatistics => - val statistics = r.estimateStatistics(readSupport.newScanConfigBuilder().build()) - Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) - case _ => - Statistics(sizeInBytes = conf.defaultSizeInBytes) - } - - override def newInstance(): DataSourceV2Relation = { - copy(output = output.map(_.newInstance())) - } -} - -/** - * A specialization of [[DataSourceV2Relation]] with the streaming bit set to true. - * - * Note that, this plan has a mutable reader, so Spark won't apply operator push-down for this plan, - * to avoid making the plan mutable. We should consolidate this plan and [[DataSourceV2Relation]] - * after we figure out how to apply operator push-down for streaming data sources. - */ -case class StreamingDataSourceV2Relation( - output: Seq[AttributeReference], - source: DataSourceV2, - options: Map[String, String], - readSupport: ReadSupport, - scanConfigBuilder: ScanConfigBuilder) - extends LeafNode with MultiInstanceRelation with DataSourceV2StringFormat { - - override def isStreaming: Boolean = true - - override def simpleString: String = "Streaming RelationV2 " + metadataString - - override def pushedFilters: Seq[Expression] = Nil - - override def newInstance(): LogicalPlan = copy(output = output.map(_.newInstance())) - - // TODO: unify the equal/hashCode implementation for all data source v2 query plans. - override def equals(other: Any): Boolean = other match { - case other: StreamingDataSourceV2Relation => - output == other.output && readSupport.getClass == other.readSupport.getClass && - options == other.options - case _ => false - } - - override def hashCode(): Int = { - Seq(output, source, options).hashCode() - } - - override def computeStats(): Statistics = readSupport match { - case r: SupportsReportStatistics => - val statistics = r.estimateStatistics(scanConfigBuilder.build()) - Statistics(sizeInBytes = statistics.sizeInBytes().orElse(conf.defaultSizeInBytes)) - case _ => - Statistics(sizeInBytes = conf.defaultSizeInBytes) - } -} - -object DataSourceV2Relation { - private implicit class SourceHelpers(source: DataSourceV2) { - def asReadSupportProvider: BatchReadSupportProvider = { - source match { - case provider: BatchReadSupportProvider => - provider - case _ => - throw new AnalysisException(s"Data source is not readable: $name") - } - } - - def asWriteSupportProvider: BatchWriteSupportProvider = { - source match { - case provider: BatchWriteSupportProvider => - provider - case _ => - throw new AnalysisException(s"Data source is not writable: $name") - } - } - - def name: String = { - source match { - case registered: DataSourceRegister => - registered.shortName() - case _ => - source.getClass.getSimpleName - } - } - - def createReadSupport( - options: Map[String, String], - userSpecifiedSchema: Option[StructType]): BatchReadSupport = { - val v2Options = new DataSourceOptions(options.asJava) - userSpecifiedSchema match { - case Some(s) => - asReadSupportProvider.createBatchReadSupport(s, v2Options) - case _ => - asReadSupportProvider.createBatchReadSupport(v2Options) - } - } - - def createWriteSupport( - options: Map[String, String], - schema: StructType): BatchWriteSupport = { - asWriteSupportProvider.createBatchWriteSupport( - UUID.randomUUID().toString, - schema, - SaveMode.Append, - new DataSourceOptions(options.asJava)).get - } - } - - def create( - source: DataSourceV2, - options: Map[String, String], - tableIdent: Option[TableIdentifier] = None, - userSpecifiedSchema: Option[StructType] = None): DataSourceV2Relation = { - val readSupport = source.createReadSupport(options, userSpecifiedSchema) - val output = readSupport.fullSchema().toAttributes - val ident = tableIdent.orElse(tableFromOptions(options)) - DataSourceV2Relation( - source, readSupport, output, options, ident, userSpecifiedSchema) - } - - private def tableFromOptions(options: Map[String, String]): Option[TableIdentifier] = { - options - .get(DataSourceOptions.TABLE_KEY) - .map(TableIdentifier(_, options.get(DataSourceOptions.DATABASE_KEY))) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala deleted file mode 100644 index 04a97735d024d..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExec.scala +++ /dev/null @@ -1,122 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.v2 - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.plans.physical.SinglePartition -import org.apache.spark.sql.execution.{ColumnarBatchScan, LeafExecNode, WholeStageCodegenExec} -import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.DataSourceV2 -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReaderFactory, ContinuousReadSupport, MicroBatchReadSupport} - -/** - * Physical plan node for scanning data from a data source. - */ -case class DataSourceV2ScanExec( - output: Seq[AttributeReference], - @transient source: DataSourceV2, - @transient options: Map[String, String], - @transient pushedFilters: Seq[Expression], - @transient readSupport: ReadSupport, - @transient scanConfig: ScanConfig) - extends LeafExecNode with DataSourceV2StringFormat with ColumnarBatchScan { - - override def simpleString: String = "ScanV2 " + metadataString - - // TODO: unify the equal/hashCode implementation for all data source v2 query plans. - override def equals(other: Any): Boolean = other match { - case other: DataSourceV2ScanExec => - output == other.output && readSupport.getClass == other.readSupport.getClass && - options == other.options - case _ => false - } - - override def hashCode(): Int = { - Seq(output, source, options).hashCode() - } - - override def outputPartitioning: physical.Partitioning = readSupport match { - case _ if partitions.length == 1 => - SinglePartition - - case s: SupportsReportPartitioning => - new DataSourcePartitioning( - s.outputPartitioning(scanConfig), AttributeMap(output.map(a => a -> a.name))) - - case _ => super.outputPartitioning - } - - private lazy val partitions: Seq[InputPartition] = readSupport.planInputPartitions(scanConfig) - - private lazy val readerFactory = readSupport match { - case r: BatchReadSupport => r.createReaderFactory(scanConfig) - case r: MicroBatchReadSupport => r.createReaderFactory(scanConfig) - case r: ContinuousReadSupport => r.createContinuousReaderFactory(scanConfig) - case _ => throw new IllegalStateException("unknown read support: " + readSupport) - } - - // TODO: clean this up when we have dedicated scan plan for continuous streaming. - override val supportsBatch: Boolean = { - require(partitions.forall(readerFactory.supportColumnarReads) || - !partitions.exists(readerFactory.supportColumnarReads), - "Cannot mix row-based and columnar input partitions.") - - partitions.exists(readerFactory.supportColumnarReads) - } - - private lazy val inputRDD: RDD[InternalRow] = readSupport match { - case _: ContinuousReadSupport => - assert(!supportsBatch, - "continuous stream reader does not support columnar read yet.") - EpochCoordinatorRef.get( - sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), - sparkContext.env) - .askSync[Unit](SetReaderPartitions(partitions.size)) - new ContinuousDataSourceRDD( - sparkContext, - sqlContext.conf.continuousStreamingExecutorQueueSize, - sqlContext.conf.continuousStreamingExecutorPollIntervalMs, - partitions, - schema, - readerFactory.asInstanceOf[ContinuousPartitionReaderFactory]) - - case _ => - new DataSourceRDD( - sparkContext, partitions, readerFactory.asInstanceOf[PartitionReaderFactory], supportsBatch) - } - - override def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) - - override protected def needsUnsafeRowConversion: Boolean = false - - override protected def doExecute(): RDD[InternalRow] = { - if (supportsBatch) { - WholeStageCodegenExec(this)(codegenStageId = 0).execute() - } else { - val numOutputRows = longMetric("numOutputRows") - inputRDD.map { r => - numOutputRows += 1 - r - } - } - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala new file mode 100644 index 0000000000000..c5c902ffc4104 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2ScanExecBase.scala @@ -0,0 +1,88 @@ +/* + * 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.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.AttributeMap +import org.apache.spark.sql.catalyst.plans.physical +import org.apache.spark.sql.catalyst.plans.physical.SinglePartition +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.execution.LeafExecNode +import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan, SupportsReportPartitioning} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.Utils + +trait DataSourceV2ScanExecBase extends LeafExecNode { + + override lazy val metrics = Map( + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) + + def scan: Scan + + def partitions: Seq[InputPartition] + + def readerFactory: PartitionReaderFactory + + override def simpleString(maxFields: Int): String = { + val result = + s"$nodeName${truncatedString(output, "[", ", ", "]", maxFields)} ${scan.description()}" + Utils.redact(sqlContext.sessionState.conf.stringRedactionPattern, result) + } + + override def outputPartitioning: physical.Partitioning = scan match { + case _ if partitions.length == 1 => + SinglePartition + + case s: SupportsReportPartitioning => + new DataSourcePartitioning( + s.outputPartitioning(), AttributeMap(output.map(a => a -> a.name))) + + case _ => super.outputPartitioning + } + + override def supportsColumnar: Boolean = { + require(partitions.forall(readerFactory.supportColumnarReads) || + !partitions.exists(readerFactory.supportColumnarReads), + "Cannot mix row-based and columnar input partitions.") + + partitions.exists(readerFactory.supportColumnarReads) + } + + def inputRDD: RDD[InternalRow] + + def inputRDDs(): Seq[RDD[InternalRow]] = Seq(inputRDD) + + override def doExecute(): RDD[InternalRow] = { + val numOutputRows = longMetric("numOutputRows") + inputRDD.map { r => + numOutputRows += 1 + r + } + } + + override def doExecuteColumnar(): RDD[ColumnarBatch] = { + val numOutputRows = longMetric("numOutputRows") + inputRDD.asInstanceOf[RDD[ColumnarBatch]].map { + b => + numOutputRows += b.numRows() + b + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala index 9a3109e7c199e..4f8507da39240 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Strategy.scala @@ -17,19 +17,22 @@ package org.apache.spark.sql.execution.datasources.v2 +import scala.collection.JavaConverters._ import scala.collection.mutable -import org.apache.spark.sql.{sources, Strategy} -import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression} +import org.apache.spark.sql.{AnalysisException, Strategy} +import org.apache.spark.sql.catalyst.expressions.{And, AttributeReference, AttributeSet, Expression, PredicateHelper, SubqueryExpression} import org.apache.spark.sql.catalyst.planning.PhysicalOperation -import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, Repartition} +import org.apache.spark.sql.catalyst.plans.logical.{AlterTable, AppendData, CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic, Repartition} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan} import org.apache.spark.sql.execution.datasources.DataSourceStrategy import org.apache.spark.sql.execution.streaming.continuous.{ContinuousCoalesceExec, WriteToContinuousDataSource, WriteToContinuousDataSourceExec} +import org.apache.spark.sql.sources import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.ContinuousReadSupport +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} +import org.apache.spark.sql.util.CaseInsensitiveStringMap -object DataSourceV2Strategy extends Strategy { +object DataSourceV2Strategy extends Strategy with PredicateHelper { /** * Pushes down filters to the data source reader @@ -37,31 +40,39 @@ object DataSourceV2Strategy extends Strategy { * @return pushed filter and post-scan filters. */ private def pushFilters( - configBuilder: ScanConfigBuilder, + scanBuilder: ScanBuilder, filters: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - configBuilder match { + scanBuilder match { case r: SupportsPushDownFilters => - // A map from translated data source filters to original catalyst filter expressions. + // A map from translated data source leaf node filters to original catalyst filter + // expressions. For a `And`/`Or` predicate, it is possible that the predicate is partially + // pushed down. This map can be used to construct a catalyst filter expression from the + // input filter, or a superset(partial push down filter) of the input filter. val translatedFilterToExpr = mutable.HashMap.empty[sources.Filter, Expression] + val translatedFilters = mutable.ArrayBuffer.empty[sources.Filter] // Catalyst filter expression that can't be translated to data source filters. val untranslatableExprs = mutable.ArrayBuffer.empty[Expression] for (filterExpr <- filters) { - val translated = DataSourceStrategy.translateFilter(filterExpr) - if (translated.isDefined) { - translatedFilterToExpr(translated.get) = filterExpr - } else { + val translated = + DataSourceStrategy.translateFilterWithMapping(filterExpr, Some(translatedFilterToExpr)) + if (translated.isEmpty) { untranslatableExprs += filterExpr + } else { + translatedFilters += translated.get } } // Data source filters that need to be evaluated again after scanning. which means // the data source cannot guarantee the rows returned can pass these filters. // As a result we must return it so Spark can plan an extra filter operator. - val postScanFilters = r.pushFilters(translatedFilterToExpr.keys.toArray) - .map(translatedFilterToExpr) + val postScanFilters = r.pushFilters(translatedFilters.toArray).map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } // The filters which are marked as pushed to this data source - val pushedFilters = r.pushedFilters().map(translatedFilterToExpr) + val pushedFilters = r.pushedFilters().map { filter => + DataSourceStrategy.rebuildExpressionFromFilter(filter, translatedFilterToExpr) + } (pushedFilters, untranslatableExprs ++ postScanFilters) case _ => (Nil, filters) @@ -76,18 +87,18 @@ object DataSourceV2Strategy extends Strategy { */ // TODO: nested column pruning. private def pruneColumns( - configBuilder: ScanConfigBuilder, + scanBuilder: ScanBuilder, relation: DataSourceV2Relation, - exprs: Seq[Expression]): (ScanConfig, Seq[AttributeReference]) = { - configBuilder match { + exprs: Seq[Expression]): (Scan, Seq[AttributeReference]) = { + scanBuilder match { case r: SupportsPushDownRequiredColumns => val requiredColumns = AttributeSet(exprs.flatMap(_.references)) val neededOutput = relation.output.filter(requiredColumns.contains) if (neededOutput != relation.output) { r.pruneColumns(neededOutput.toStructType) - val config = r.build() + val scan = r.build() val nameToAttr = relation.output.map(_.name).zip(relation.output).toMap - config -> config.readSchema().toAttributes.map { + scan -> scan.readSchema().toAttributes.map { // We have to keep the attribute id during transformation. a => a.withExprId(nameToAttr(a.name).exprId) } @@ -95,61 +106,90 @@ object DataSourceV2Strategy extends Strategy { r.build() -> relation.output } - case _ => configBuilder.build() -> relation.output + case _ => scanBuilder.build() -> relation.output } } + import DataSourceV2Implicits._ override def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case PhysicalOperation(project, filters, relation: DataSourceV2Relation) => - val configBuilder = relation.readSupport.newScanConfigBuilder() + val scanBuilder = relation.newScanBuilder() + + val (withSubquery, withoutSubquery) = filters.partition(SubqueryExpression.hasSubquery) + val normalizedFilters = DataSourceStrategy.normalizeFilters( + withoutSubquery, relation.output) + // `pushedFilters` will be pushed down and evaluated in the underlying data sources. // `postScanFilters` need to be evaluated after the scan. // `postScanFilters` and `pushedFilters` can overlap, e.g. the parquet row group filter. - val (pushedFilters, postScanFilters) = pushFilters(configBuilder, filters) - val (config, output) = pruneColumns(configBuilder, relation, project ++ postScanFilters) + val (pushedFilters, postScanFiltersWithoutSubquery) = + pushFilters(scanBuilder, normalizedFilters) + val postScanFilters = postScanFiltersWithoutSubquery ++ withSubquery + val (scan, output) = pruneColumns(scanBuilder, relation, project ++ postScanFilters) logInfo( s""" - |Pushing operators to ${relation.source.getClass} + |Pushing operators to ${relation.name} |Pushed Filters: ${pushedFilters.mkString(", ")} |Post-Scan Filters: ${postScanFilters.mkString(",")} |Output: ${output.mkString(", ")} """.stripMargin) - val scan = DataSourceV2ScanExec( - output, - relation.source, - relation.options, - pushedFilters, - relation.readSupport, - config) + val plan = BatchScanExec(output, scan) val filterCondition = postScanFilters.reduceLeftOption(And) - val withFilter = filterCondition.map(FilterExec(_, scan)).getOrElse(scan) + val withFilter = filterCondition.map(FilterExec(_, plan)).getOrElse(plan) // always add the projection, which will produce unsafe rows required by some operators ProjectExec(project, withFilter) :: Nil - case r: StreamingDataSourceV2Relation => - // TODO: support operator pushdown for streaming data sources. - val scanConfig = r.scanConfigBuilder.build() + case r: StreamingDataSourceV2Relation if r.startOffset.isDefined && r.endOffset.isDefined => + val microBatchStream = r.stream.asInstanceOf[MicroBatchStream] + // ensure there is a projection, which will produce unsafe rows required by some operators + ProjectExec(r.output, + MicroBatchScanExec( + r.output, r.scan, microBatchStream, r.startOffset.get, r.endOffset.get)) :: Nil + + case r: StreamingDataSourceV2Relation if r.startOffset.isDefined && r.endOffset.isEmpty => + val continuousStream = r.stream.asInstanceOf[ContinuousStream] // ensure there is a projection, which will produce unsafe rows required by some operators ProjectExec(r.output, - DataSourceV2ScanExec( - r.output, r.source, r.options, r.pushedFilters, r.readSupport, scanConfig)) :: Nil + ContinuousScanExec( + r.output, r.scan, continuousStream, r.startOffset.get)) :: Nil case WriteToDataSourceV2(writer, query) => WriteToDataSourceV2Exec(writer, planLater(query)) :: Nil + case CreateV2Table(catalog, ident, schema, parts, props, ifNotExists) => + CreateTableExec(catalog, ident, schema, parts, props, ifNotExists) :: Nil + + case CreateTableAsSelect(catalog, ident, parts, query, props, options, ifNotExists) => + val writeOptions = new CaseInsensitiveStringMap(options.asJava) + CreateTableAsSelectExec( + catalog, ident, parts, planLater(query), props, writeOptions, ifNotExists) :: Nil + case AppendData(r: DataSourceV2Relation, query, _) => - WriteToDataSourceV2Exec(r.newWriteSupport(), planLater(query)) :: Nil + AppendDataExec(r.table.asWritable, r.options, planLater(query)) :: Nil + + case OverwriteByExpression(r: DataSourceV2Relation, deleteExpr, query, _) => + // fail if any filter cannot be converted. correctness depends on removing all matching data. + val filters = splitConjunctivePredicates(deleteExpr).map { + filter => DataSourceStrategy.translateFilter(deleteExpr).getOrElse( + throw new AnalysisException(s"Cannot translate expression to source filter: $filter")) + }.toArray + + OverwriteByExpressionExec( + r.table.asWritable, filters, r.options, planLater(query)) :: Nil + + case OverwritePartitionsDynamic(r: DataSourceV2Relation, query, _) => + OverwritePartitionsDynamicExec(r.table.asWritable, r.options, planLater(query)) :: Nil case WriteToContinuousDataSource(writer, query) => WriteToContinuousDataSourceExec(writer, planLater(query)) :: Nil case Repartition(1, false, child) => val isContinuous = child.find { - case s: StreamingDataSourceV2Relation => s.readSupport.isInstanceOf[ContinuousReadSupport] + case r: StreamingDataSourceV2Relation => r.stream.isInstanceOf[ContinuousStream] case _ => false }.isDefined @@ -159,6 +199,12 @@ object DataSourceV2Strategy extends Strategy { Nil } + case DropTable(catalog, ident, ifExists) => + DropTableExec(catalog, ident, ifExists) :: Nil + + case AlterTable(catalog, ident, _, changes) => + AlterTableExec(catalog, ident, changes) :: Nil + case _ => Nil } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala deleted file mode 100644 index 97e6c6d702acb..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2StringFormat.scala +++ /dev/null @@ -1,87 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.v2 - -import org.apache.commons.lang3.StringUtils - -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.DataSourceV2 -import org.apache.spark.util.Utils - -/** - * A trait that can be used by data source v2 related query plans(both logical and physical), to - * provide a string format of the data source information for explain. - */ -trait DataSourceV2StringFormat { - - /** - * The instance of this data source implementation. Note that we only consider its class in - * equals/hashCode, not the instance itself. - */ - def source: DataSourceV2 - - /** - * The output of the data source reader, w.r.t. column pruning. - */ - def output: Seq[Attribute] - - /** - * The options for this data source reader. - */ - def options: Map[String, String] - - /** - * The filters which have been pushed to the data source. - */ - def pushedFilters: Seq[Expression] - - private def sourceName: String = source match { - case registered: DataSourceRegister => registered.shortName() - // source.getClass.getSimpleName can cause Malformed class name error, - // call safer `Utils.getSimpleName` instead - case _ => Utils.getSimpleName(source.getClass) - } - - def metadataString: String = { - val entries = scala.collection.mutable.ArrayBuffer.empty[(String, String)] - - if (pushedFilters.nonEmpty) { - entries += "Filters" -> pushedFilters.mkString("[", ", ", "]") - } - - // TODO: we should only display some standard options like path, table, etc. - if (options.nonEmpty) { - entries += "Options" -> Utils.redact(options).map { - case (k, v) => s"$k=$v" - }.mkString("[", ",", "]") - } - - val outputStr = Utils.truncatedString(output, "[", ", ", "]") - - val entriesStr = if (entries.nonEmpty) { - Utils.truncatedString(entries.map { - case (key, value) => key + ": " + StringUtils.abbreviate(value, 100) - }, " (", ", ", ")") - } else { - "" - } - - s"$sourceName$outputStr$entriesStr" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala index e9cc3991155c4..30897d86f8179 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DataSourceV2Utils.scala @@ -21,8 +21,7 @@ import java.util.regex.Pattern import org.apache.spark.internal.Logging import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{DataSourceV2, SessionConfigSupport} +import org.apache.spark.sql.sources.v2.{SessionConfigSupport, TableProvider} private[sql] object DataSourceV2Utils extends Logging { @@ -34,34 +33,28 @@ private[sql] object DataSourceV2Utils extends Logging { * `spark.datasource.$keyPrefix`. A session config `spark.datasource.$keyPrefix.xxx -> yyy` will * be transformed into `xxx -> yyy`. * - * @param ds a [[DataSourceV2]] object + * @param source a [[TableProvider]] object * @param conf the session conf * @return an immutable map that contains all the extracted and transformed k/v pairs. */ - def extractSessionConfigs(ds: DataSourceV2, conf: SQLConf): Map[String, String] = ds match { - case cs: SessionConfigSupport => - val keyPrefix = cs.keyPrefix() - require(keyPrefix != null, "The data source config key prefix can't be null.") - - val pattern = Pattern.compile(s"^spark\\.datasource\\.$keyPrefix\\.(.+)") - - conf.getAllConfs.flatMap { case (key, value) => - val m = pattern.matcher(key) - if (m.matches() && m.groupCount() > 0) { - Seq((m.group(1), value)) - } else { - Seq.empty + def extractSessionConfigs(source: TableProvider, conf: SQLConf): Map[String, String] = { + source match { + case cs: SessionConfigSupport => + val keyPrefix = cs.keyPrefix() + require(keyPrefix != null, "The data source config key prefix can't be null.") + + val pattern = Pattern.compile(s"^spark\\.datasource\\.$keyPrefix\\.(.+)") + + conf.getAllConfs.flatMap { case (key, value) => + val m = pattern.matcher(key) + if (m.matches() && m.groupCount() > 0) { + Seq((m.group(1), value)) + } else { + Seq.empty + } } - } - - case _ => Map.empty - } - def failForUserSpecifiedSchema[T](ds: DataSourceV2): T = { - val name = ds match { - case register: DataSourceRegister => register.shortName() - case _ => ds.getClass.getName + case _ => Map.empty } - throw new UnsupportedOperationException(name + " source does not support user-specified schema") } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala new file mode 100644 index 0000000000000..d325e0205f9d8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/DropTableExec.scala @@ -0,0 +1,44 @@ +/* + * 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.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.execution.LeafExecNode + +/** + * Physical plan node for dropping a table. + */ +case class DropTableExec(catalog: TableCatalog, ident: Identifier, ifExists: Boolean) + extends LeafExecNode { + + override def doExecute(): RDD[InternalRow] = { + if (catalog.tableExists(ident)) { + catalog.dropTable(ident) + } else if (!ifExists) { + throw new NoSuchTableException(ident) + } + + sqlContext.sparkContext.parallelize(Seq.empty, 1) + } + + override def output: Seq[Attribute] = Seq.empty +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala new file mode 100644 index 0000000000000..b177d15e1fe32 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/EmptyPartitionReader.scala @@ -0,0 +1,34 @@ +/* + * 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.v2 + +import java.io.IOException + +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +/** + * A [[PartitionReader]] with empty output. + */ +class EmptyPartitionReader[T] extends PartitionReader[T] { + override def next(): Boolean = false + + override def get(): T = + throw new IOException("No records should be returned from EmptyDataReader") + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala new file mode 100644 index 0000000000000..db31927fa73bb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileBatchWrite.scala @@ -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. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.hadoop.mapreduce.Job + +import org.apache.spark.internal.Logging +import org.apache.spark.internal.io.FileCommitProtocol +import org.apache.spark.sql.execution.datasources.{WriteJobDescription, WriteTaskResult} +import org.apache.spark.sql.execution.datasources.FileFormatWriter.processStats +import org.apache.spark.sql.sources.v2.writer._ +import org.apache.spark.util.SerializableConfiguration + +class FileBatchWrite( + job: Job, + description: WriteJobDescription, + committer: FileCommitProtocol) + extends BatchWrite with Logging { + override def commit(messages: Array[WriterCommitMessage]): Unit = { + val results = messages.map(_.asInstanceOf[WriteTaskResult]) + committer.commitJob(job, results.map(_.commitMsg)) + logInfo(s"Write Job ${description.uuid} committed.") + + processStats(description.statsTrackers, results.map(_.summary.stats)) + logInfo(s"Finished processing stats for write job ${description.uuid}.") + } + + override def useCommitCoordinator(): Boolean = false + + override def abort(messages: Array[WriterCommitMessage]): Unit = { + committer.abortJob(job) + } + + override def createBatchWriterFactory(): DataWriterFactory = { + FileWriterFactory(description, committer) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.scala new file mode 100644 index 0000000000000..bcb10ae5999fc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileDataSourceV2.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.execution.datasources.v2 + +import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2.TableProvider +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils + +/** + * A base interface for data source v2 implementations of the built-in file-based data sources. + */ +trait FileDataSourceV2 extends TableProvider with DataSourceRegister { + /** + * Returns a V1 [[FileFormat]] class of the same file data source. + * This is a solution for the following cases: + * 1. File datasource V2 implementations cause regression. Users can disable the problematic data + * source via SQL configuration and fall back to FileFormat. + * 2. Catalog support is required, which is still under development for data source V2. + */ + def fallbackFileFormat: Class[_ <: FileFormat] + + lazy val sparkSession = SparkSession.active + + protected def getPaths(map: CaseInsensitiveStringMap): Seq[String] = { + val objectMapper = new ObjectMapper() + Option(map.get("paths")).map { pathStr => + objectMapper.readValue(pathStr, classOf[Array[String]]).toSeq + }.getOrElse { + Option(map.get("path")).toSeq + } + } + + protected def getTableName(paths: Seq[String]): String = { + val name = shortName() + " " + paths.map(qualifiedPathName).mkString(",") + Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, name) + } + + private def qualifiedPathName(path: String): String = { + val hdfsPath = new Path(path) + val fs = hdfsPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) + hdfsPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toString + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala new file mode 100644 index 0000000000000..836eae88e4da7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala @@ -0,0 +1,116 @@ +/* + * 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.v2 + +import java.io.{FileNotFoundException, IOException} + +import org.apache.parquet.io.ParquetDecodingException + +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.InputFileBlockHolder +import org.apache.spark.sql.execution.QueryExecutionException +import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +class FilePartitionReader[T](readers: Iterator[PartitionedFileReader[T]]) + extends PartitionReader[T] with Logging { + private var currentReader: PartitionedFileReader[T] = null + + private val sqlConf = SQLConf.get + private def ignoreMissingFiles = sqlConf.ignoreMissingFiles + private def ignoreCorruptFiles = sqlConf.ignoreCorruptFiles + + override def next(): Boolean = { + if (currentReader == null) { + if (readers.hasNext) { + try { + currentReader = getNextReader() + } catch { + case e: FileNotFoundException if ignoreMissingFiles => + logWarning(s"Skipped missing file: $currentReader", e) + currentReader = null + return false + // Throw FileNotFoundException even if `ignoreCorruptFiles` is true + case e: FileNotFoundException if !ignoreMissingFiles => + throw new FileNotFoundException( + e.getMessage + "\n" + + "It is possible the underlying files have been updated. " + + "You can explicitly invalidate the cache in Spark by " + + "recreating the Dataset/DataFrame involved.") + case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: $currentReader", e) + currentReader = null + InputFileBlockHolder.unset() + return false + } + } else { + return false + } + } + + // In PartitionReader.next(), the current reader proceeds to next record. + // It might throw RuntimeException/IOException and Spark should handle these exceptions. + val hasNext = try { + currentReader.next() + } catch { + case e: SchemaColumnConvertNotSupportedException => + val message = "Parquet column cannot be converted in " + + s"file ${currentReader.file.filePath}. Column: ${e.getColumn}, " + + s"Expected: ${e.getLogicalType}, Found: ${e.getPhysicalType}" + throw new QueryExecutionException(message, e) + case e: ParquetDecodingException => + if (e.getMessage.contains("Can not read value at")) { + val message = "Encounter error while reading parquet files. " + + "One possible cause: Parquet column cannot be converted in the " + + "corresponding files. Details: " + throw new QueryExecutionException(message, e) + } + throw e + case e @ (_: RuntimeException | _: IOException) if ignoreCorruptFiles => + logWarning( + s"Skipped the rest of the content in the corrupted file: $currentReader", e) + false + } + if (hasNext) { + true + } else { + close() + currentReader = null + next() + } + } + + override def get(): T = currentReader.get() + + override def close(): Unit = { + if (currentReader != null) { + currentReader.close() + } + InputFileBlockHolder.unset() + } + + private def getNextReader(): PartitionedFileReader[T] = { + val reader = readers.next() + logInfo(s"Reading file $reader") + // Sets InputFileBlockHolder for the file block's information + val file = reader.file + InputFileBlockHolder.set(file.filePath, file.start, file.length) + reader + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala new file mode 100644 index 0000000000000..5a19412c90334 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReaderFactory.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.vectorized.ColumnarBatch + +abstract class FilePartitionReaderFactory extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map { file => + PartitionedFileReader(file, buildReader(file)) + } + new FilePartitionReader[InternalRow](iter) + } + + override def createColumnarReader(partition: InputPartition): PartitionReader[ColumnarBatch] = { + assert(partition.isInstanceOf[FilePartition]) + val filePartition = partition.asInstanceOf[FilePartition] + val iter = filePartition.files.toIterator.map { file => + PartitionedFileReader(file, buildColumnarReader(file)) + } + new FilePartitionReader[ColumnarBatch](iter) + } + + def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] + + def buildColumnarReader(partitionedFile: PartitionedFile): PartitionReader[ColumnarBatch] = { + throw new UnsupportedOperationException("Cannot create columnar reader.") + } +} + +// A compound class for combining file and its corresponding reader. +private[v2] case class PartitionedFileReader[T]( + file: PartitionedFile, + reader: PartitionReader[T]) extends PartitionReader[T] { + override def next(): Boolean = reader.next() + + override def get(): T = reader.get() + + override def close(): Unit = reader.close() + + override def toString: String = file.toString +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala new file mode 100644 index 0000000000000..b2f3c4d256448 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScan.scala @@ -0,0 +1,132 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import java.util.{Locale, OptionalLong} + +import org.apache.commons.lang3.StringUtils +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.PartitionedFileUtil +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader._ +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.Utils + +abstract class FileScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + readDataSchema: StructType, + readPartitionSchema: StructType) extends Scan with Batch with SupportsReportStatistics { + /** + * Returns whether a file with `path` could be split or not. + */ + def isSplitable(path: Path): Boolean = { + false + } + + override def description(): String = { + val locationDesc = + fileIndex.getClass.getSimpleName + fileIndex.rootPaths.mkString("[", ", ", "]") + val metadata: Map[String, String] = Map( + "ReadSchema" -> readDataSchema.catalogString, + "Location" -> locationDesc) + val metadataStr = metadata.toSeq.sorted.map { + case (key, value) => + val redactedValue = + Utils.redact(sparkSession.sessionState.conf.stringRedactionPattern, value) + key + ": " + StringUtils.abbreviate(redactedValue, 100) + }.mkString(", ") + s"${this.getClass.getSimpleName} $metadataStr" + } + + protected def partitions: Seq[FilePartition] = { + val selectedPartitions = fileIndex.listFiles(Seq.empty, Seq.empty) + val maxSplitBytes = FilePartition.maxSplitBytes(sparkSession, selectedPartitions) + val partitionAttributes = fileIndex.partitionSchema.toAttributes + val attributeMap = partitionAttributes.map(a => normalizeName(a.name) -> a).toMap + val readPartitionAttributes = readPartitionSchema.map { readField => + attributeMap.get(normalizeName(readField.name)).getOrElse { + throw new AnalysisException(s"Can't find required partition column ${readField.name} " + + s"in partition schema ${fileIndex.partitionSchema}") + } + } + lazy val partitionValueProject = + GenerateUnsafeProjection.generate(readPartitionAttributes, partitionAttributes) + val splitFiles = selectedPartitions.flatMap { partition => + // Prune partition values if part of the partition columns are not required. + val partitionValues = if (readPartitionAttributes != partitionAttributes) { + partitionValueProject(partition.values).copy() + } else { + partition.values + } + partition.files.flatMap { file => + val filePath = file.getPath + PartitionedFileUtil.splitFiles( + sparkSession = sparkSession, + file = file, + filePath = filePath, + isSplitable = isSplitable(filePath), + maxSplitBytes = maxSplitBytes, + partitionValues = partitionValues + ) + }.toArray.sortBy(_.length)(implicitly[Ordering[Long]].reverse) + } + FilePartition.getFilePartitions(sparkSession, splitFiles, maxSplitBytes) + } + + override def planInputPartitions(): Array[InputPartition] = { + partitions.toArray + } + + override def estimateStatistics(): Statistics = { + new Statistics { + override def sizeInBytes(): OptionalLong = { + val compressionFactor = sparkSession.sessionState.conf.fileCompressionFactor + val size = (compressionFactor * fileIndex.sizeInBytes).toLong + OptionalLong.of(size) + } + + override def numRows(): OptionalLong = OptionalLong.empty() + } + } + + override def toBatch: Batch = this + + override def readSchema(): StructType = + StructType(readDataSchema.fields ++ readPartitionSchema.fields) + + // Returns whether the two given arrays of [[Filter]]s are equivalent. + protected def equivalentFilters(a: Array[Filter], b: Array[Filter]): Boolean = { + a.sortBy(_.hashCode()).sameElements(b.sortBy(_.hashCode())) + } + + private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + + private def normalizeName(name: String): String = { + if (isCaseSensitive) { + name + } else { + name.toLowerCase(Locale.ROOT) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala new file mode 100644 index 0000000000000..3b236be90e6ff --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileScanBuilder.scala @@ -0,0 +1,59 @@ +/* + * 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.v2 + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.{PartitioningAwareFileIndex, PartitioningUtils} +import org.apache.spark.sql.sources.v2.reader.{ScanBuilder, SupportsPushDownRequiredColumns} +import org.apache.spark.sql.types.StructType + +abstract class FileScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType) extends ScanBuilder with SupportsPushDownRequiredColumns { + private val partitionSchema = fileIndex.partitionSchema + private val isCaseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + protected var requiredSchema = StructType(dataSchema.fields ++ partitionSchema.fields) + + override def pruneColumns(requiredSchema: StructType): Unit = { + this.requiredSchema = requiredSchema + } + + protected def readDataSchema(): StructType = { + val requiredNameSet = createRequiredNameSet() + val fields = dataSchema.fields.filter { field => + val colName = PartitioningUtils.getColName(field, isCaseSensitive) + requiredNameSet.contains(colName) && !partitionNameSet.contains(colName) + } + StructType(fields) + } + + protected def readPartitionSchema(): StructType = { + val requiredNameSet = createRequiredNameSet() + val fields = partitionSchema.fields.filter { field => + val colName = PartitioningUtils.getColName(field, isCaseSensitive) + requiredNameSet.contains(colName) + } + StructType(fields) + } + + private def createRequiredNameSet(): Set[String] = + requiredSchema.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet + + private val partitionNameSet: Set[String] = + partitionSchema.fields.map(PartitioningUtils.getColName(_, isCaseSensitive)).toSet +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala new file mode 100644 index 0000000000000..4483f5b1dd30c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileTable.scala @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.{FileStatus, Path} + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.streaming.{FileStreamSink, MetadataLogFileIndex} +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.SchemaUtils + +abstract class FileTable( + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType]) + extends Table with SupportsRead with SupportsWrite { + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + lazy val fileIndex: PartitioningAwareFileIndex = { + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + if (FileStreamSink.hasMetadata(paths, hadoopConf, sparkSession.sessionState.conf)) { + // We are reading from the results of a streaming query. We will load files from + // the metadata log instead of listing them using HDFS APIs. + new MetadataLogFileIndex(sparkSession, new Path(paths.head), + options.asScala.toMap, userSpecifiedSchema) + } else { + // This is a non-streaming file based datasource. + val rootPathsSpecified = DataSource.checkAndGlobPathIfNecessary(paths, hadoopConf, + checkEmptyGlobPath = true, checkFilesExist = true) + val fileStatusCache = FileStatusCache.getOrCreate(sparkSession) + new InMemoryFileIndex( + sparkSession, rootPathsSpecified, caseSensitiveMap, userSpecifiedSchema, fileStatusCache) + } + } + + lazy val dataSchema: StructType = { + val schema = userSpecifiedSchema.map { schema => + val partitionSchema = fileIndex.partitionSchema + val resolver = sparkSession.sessionState.conf.resolver + StructType(schema.filterNot(f => partitionSchema.exists(p => resolver(p.name, f.name)))) + }.orElse { + inferSchema(fileIndex.allFiles()) + }.getOrElse { + throw new AnalysisException( + s"Unable to infer schema for $formatName. It must be specified manually.") + } + fileIndex match { + case _: MetadataLogFileIndex => schema + case _ => schema.asNullable + } + } + + override lazy val schema: StructType = { + val caseSensitive = sparkSession.sessionState.conf.caseSensitiveAnalysis + SchemaUtils.checkColumnNameDuplication(dataSchema.fieldNames, + "in the data schema", caseSensitive) + dataSchema.foreach { field => + if (!supportsDataType(field.dataType)) { + throw new AnalysisException( + s"$formatName data source does not support ${field.dataType.catalogString} data type.") + } + } + val partitionSchema = fileIndex.partitionSchema + SchemaUtils.checkColumnNameDuplication(partitionSchema.fieldNames, + "in the partition schema", caseSensitive) + val partitionNameSet: Set[String] = + partitionSchema.fields.map(PartitioningUtils.getColName(_, caseSensitive)).toSet + + // When data and partition schemas have overlapping columns, + // tableSchema = dataSchema - overlapSchema + partitionSchema + val fields = dataSchema.fields.filterNot { field => + val colName = PartitioningUtils.getColName(field, caseSensitive) + partitionNameSet.contains(colName) + } ++ partitionSchema.fields + StructType(fields) + } + + override def partitioning: Array[Transform] = fileIndex.partitionSchema.asTransforms + + override def properties: util.Map[String, String] = options.asCaseSensitiveMap + + override def capabilities: java.util.Set[TableCapability] = FileTable.CAPABILITIES + + /** + * When possible, this method should return the schema of the given `files`. When the format + * does not support inference, or no valid files are given should return None. In these cases + * Spark will require that user specify the schema manually. + */ + def inferSchema(files: Seq[FileStatus]): Option[StructType] + + /** + * Returns whether this format supports the given [[DataType]] in read/write path. + * By default all data types are supported. + */ + def supportsDataType(dataType: DataType): Boolean = true + + /** + * The string that represents the format that this data source provider uses. This is + * overridden by children to provide a nice alias for the data source. For example: + * + * {{{ + * override def formatName(): String = "ORC" + * }}} + */ + def formatName: String + + /** + * Returns a V1 [[FileFormat]] class of the same file data source. + * This is a solution for the following cases: + * 1. File datasource V2 implementations cause regression. Users can disable the problematic data + * source via SQL configuration and fall back to FileFormat. + * 2. Catalog support is required, which is still under development for data source V2. + */ + def fallbackFileFormat: Class[_ <: FileFormat] +} + +object FileTable { + private val CAPABILITIES = Set(BATCH_READ, BATCH_WRITE, TRUNCATE).asJava +} 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 new file mode 100644 index 0000000000000..eacc4cb3ac4a9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriteBuilder.scala @@ -0,0 +1,179 @@ +/* + * 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.v2 + +import java.io.IOException +import java.util.UUID + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +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.catalyst.InternalRow +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils} +import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, DataSource, OutputWriterFactory, WriteJobDescription} +import org.apache.spark.sql.execution.metric.SQLMetric +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, WriteBuilder} +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.sql.util.SchemaUtils +import org.apache.spark.util.SerializableConfiguration + +abstract class FileWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) extends WriteBuilder { + private var schema: StructType = _ + private var queryId: String = _ + private var mode: SaveMode = _ + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + this.schema = schema + this + } + + override def withQueryId(queryId: String): WriteBuilder = { + this.queryId = queryId + this + } + + def mode(mode: SaveMode): WriteBuilder = { + this.mode = mode + this + } + + override def buildForBatch(): BatchWrite = { + val sparkSession = SparkSession.active + validateInputs(sparkSession.sessionState.conf.caseSensitiveAnalysis) + val path = new Path(paths.head) + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + val job = getJobInstance(hadoopConf, path) + val committer = FileCommitProtocol.instantiate( + sparkSession.sessionState.conf.fileCommitProtocolClass, + jobId = java.util.UUID.randomUUID().toString, + outputPath = paths.head) + 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) + } + } + + /** + * Prepares a write job and returns an [[OutputWriterFactory]]. Client side job preparation can + * be put here. For example, user defined output committer can be configured here + * by setting the output committer class in the conf of spark.sql.sources.outputCommitterClass. + */ + def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory + + 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 " + + s"got: ${paths.mkString(", ")}") + } + val pathName = paths.head + SchemaUtils.checkColumnNameDuplication(schema.fields.map(_.name), + s"when inserting into $pathName", caseSensitiveAnalysis) + DataSource.validateSchema(schema) + + schema.foreach { field => + if (!supportsDataType(field.dataType)) { + throw new AnalysisException( + s"$formatName data source does not support ${field.dataType.catalogString} data type.") + } + } + } + + private def getJobInstance(hadoopConf: Configuration, path: Path): Job = { + val job = Job.getInstance(hadoopConf) + job.setOutputKeyClass(classOf[Void]) + job.setOutputValueClass(classOf[InternalRow]) + FileOutputFormat.setOutputPath(job, path) + job + } + + private def createWriteJobDescription( + sparkSession: SparkSession, + hadoopConf: Configuration, + job: Job, + pathName: String, + options: Map[String, String]): WriteJobDescription = { + val caseInsensitiveOptions = CaseInsensitiveMap(options) + // Note: prepareWrite has side effect. It sets "job". + val outputWriterFactory = + prepareWrite(sparkSession.sessionState.conf, job, caseInsensitiveOptions, schema) + val allColumns = schema.toAttributes + val metrics: Map[String, SQLMetric] = BasicWriteJobStatsTracker.metrics + val serializableHadoopConf = new SerializableConfiguration(hadoopConf) + val statsTracker = new BasicWriteJobStatsTracker(serializableHadoopConf, metrics) + // TODO: after partitioning is supported in V2: + // 1. filter out partition columns in `dataColumns`. + // 2. Don't use Seq.empty for `partitionColumns`. + new WriteJobDescription( + uuid = UUID.randomUUID().toString, + serializableHadoopConf = new SerializableConfiguration(job.getConfiguration), + outputWriterFactory = outputWriterFactory, + allColumns = allColumns, + dataColumns = allColumns, + partitionColumns = Seq.empty, + bucketIdExpression = None, + path = pathName, + customPartitionLocations = Map.empty, + maxRecordsPerFile = caseInsensitiveOptions.get("maxRecordsPerFile").map(_.toLong) + .getOrElse(sparkSession.sessionState.conf.maxRecordsPerFile), + timeZoneId = caseInsensitiveOptions.get(DateTimeUtils.TIMEZONE_OPTION) + .getOrElse(sparkSession.sessionState.conf.sessionLocalTimeZone), + statsTrackers = Seq(statsTracker) + ) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala new file mode 100644 index 0000000000000..eb573b317142a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FileWriterFactory.scala @@ -0,0 +1,57 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import java.util.Date + +import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl + +import org.apache.spark.internal.io.{FileCommitProtocol, SparkHadoopWriterUtils} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.datasources.{DynamicPartitionDataWriter, SingleDirectoryDataWriter, WriteJobDescription} +import org.apache.spark.sql.sources.v2.writer.{DataWriter, DataWriterFactory} +import org.apache.spark.util.SerializableConfiguration + +case class FileWriterFactory ( + description: WriteJobDescription, + committer: FileCommitProtocol) extends DataWriterFactory { + override def createWriter(partitionId: Int, realTaskId: Long): DataWriter[InternalRow] = { + val taskAttemptContext = createTaskAttemptContext(partitionId) + committer.setupTask(taskAttemptContext) + if (description.partitionColumns.isEmpty) { + new SingleDirectoryDataWriter(description, taskAttemptContext, committer) + } else { + new DynamicPartitionDataWriter(description, taskAttemptContext, committer) + } + } + + private def createTaskAttemptContext(partitionId: Int): TaskAttemptContextImpl = { + val jobId = SparkHadoopWriterUtils.createJobID(new Date, 0) + val taskId = new TaskID(jobId, TaskType.MAP, partitionId) + val taskAttemptId = new TaskAttemptID(taskId, 0) + // Set up the configuration object + val hadoopConf = description.serializableHadoopConf.value + hadoopConf.set("mapreduce.job.id", jobId.toString) + hadoopConf.set("mapreduce.task.id", taskId.toString) + hadoopConf.set("mapreduce.task.attempt.id", taskAttemptId.toString) + hadoopConf.setBoolean("mapreduce.task.ismap", true) + hadoopConf.setInt("mapreduce.task.partition", 0) + + new TaskAttemptContextImpl(hadoopConf, taskAttemptId) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala new file mode 100644 index 0000000000000..a9b0f5bce1b09 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/MicroBatchScanExec.scala @@ -0,0 +1,51 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, Scan} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} + +/** + * Physical plan node for scanning a micro-batch of data from a data source. + */ +case class MicroBatchScanExec( + output: Seq[Attribute], + @transient scan: Scan, + @transient stream: MicroBatchStream, + @transient start: Offset, + @transient end: Offset) extends DataSourceV2ScanExecBase { + + // TODO: unify the equal/hashCode implementation for all data source v2 query plans. + override def equals(other: Any): Boolean = other match { + case other: MicroBatchScanExec => this.stream == other.stream + case _ => false + } + + override def hashCode(): Int = stream.hashCode() + + override lazy val partitions: Seq[InputPartition] = stream.planInputPartitions(start, end) + + override lazy val readerFactory: PartitionReaderFactory = stream.createReaderFactory() + + override lazy val inputRDD: RDD[InternalRow] = { + new DataSourceRDD(sparkContext, partitions, readerFactory, supportsColumnar) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala new file mode 100644 index 0000000000000..f9dfcf448a3ea --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderFromIterator.scala @@ -0,0 +1,37 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2 + +import org.apache.spark.sql.sources.v2.reader.PartitionReader + +class PartitionReaderFromIterator[InternalRow]( + iter: Iterator[InternalRow]) extends PartitionReader[InternalRow] { + private var currentValue: InternalRow = _ + + override def next(): Boolean = { + if (iter.hasNext) { + currentValue = iter.next() + true + } else { + false + } + } + + override def get(): InternalRow = currentValue + + override def close(): Unit = {} +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala new file mode 100644 index 0000000000000..072465b56857d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionReaderWithPartitionValues.scala @@ -0,0 +1,53 @@ +/* + * 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.v2 + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType + +/** + * A wrapper reader that always appends partition values to [[InternalRow]]s produced by the input + * reader [[fileReader]]. + */ +class PartitionReaderWithPartitionValues( + fileReader: PartitionReader[InternalRow], + readDataSchema: StructType, + partitionSchema: StructType, + partitionValues: InternalRow) extends PartitionReader[InternalRow] { + private val fullSchema = readDataSchema.toAttributes ++ partitionSchema.toAttributes + private val unsafeProjection = GenerateUnsafeProjection.generate(fullSchema, fullSchema) + // Note that we have to apply the converter even though `file.partitionValues` is empty. + // This is because the converter is also responsible for converting safe `InternalRow`s into + // `UnsafeRow`s + private val rowConverter = { + if (partitionSchema.isEmpty) { + () => unsafeProjection(fileReader.get())} + else { + val joinedRow = new JoinedRow() + () => unsafeProjection(joinedRow(fileReader.get(), partitionValues)) + } + } + + override def next(): Boolean = fileReader.next() + + override def get(): InternalRow = rowConverter() + + override def close(): Unit = fileReader.close() +} diff --git a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala similarity index 64% rename from core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala index af67cbbce4e51..baa8cb6b24659 100644 --- a/core/src/main/scala/org/apache/spark/api/r/SparkRDefaults.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/PartitionRecordReader.scala @@ -15,16 +15,17 @@ * limitations under the License. */ -package org.apache.spark.api.r +package org.apache.spark.sql.execution.datasources.v2 -private[spark] object SparkRDefaults { +import org.apache.hadoop.mapreduce.RecordReader - // Default value for spark.r.backendConnectionTimeout config - val DEFAULT_CONNECTION_TIMEOUT: Int = 6000 +import org.apache.spark.sql.sources.v2.reader.PartitionReader - // Default value for spark.r.heartBeatInterval config - val DEFAULT_HEARTBEAT_INTERVAL: Int = 100 +class PartitionRecordReader[T]( + private[this] var rowReader: RecordReader[_, T]) extends PartitionReader[T] { + override def next(): Boolean = rowReader.nextKeyValue() - // Default value for spark.r.numRBackendThreads config - val DEFAULT_NUM_RBACKEND_THREADS = 2 + override def get(): T = rowReader.getCurrentValue + + override def close(): Unit = rowReader.close() } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TextBasedFileScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TextBasedFileScan.scala new file mode 100644 index 0000000000000..d6b84dcdfd15d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/TextBasedFileScan.scala @@ -0,0 +1,46 @@ +/* + * 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.v2 + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.compress.{CompressionCodecFactory, SplittableCompressionCodec} + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +abstract class TextBasedFileScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema) { + private var codecFactory: CompressionCodecFactory = _ + + override def isSplitable(path: Path): Boolean = { + if (codecFactory == null) { + codecFactory = new CompressionCodecFactory( + sparkSession.sessionState.newHadoopConfWithOptions(options.asScala.toMap)) + } + val codec = codecFactory.getCodec(path) + codec == null || codec.isInstanceOf[SplittableCompressionCodec] + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala new file mode 100644 index 0000000000000..4cd0346b57e7e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalog.scala @@ -0,0 +1,255 @@ +/* + * 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.v2 + +import java.util +import java.util.Locale + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.expressions.{BucketTransform, FieldReference, IdentityTransform, LogicalExpressions, Transform} +import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchNamespaceException, NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogTable, CatalogTableType, CatalogUtils, SessionCatalog} +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.internal.SessionState +import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +/** + * A [[TableCatalog]] that translates calls to the v1 SessionCatalog. + */ +class V2SessionCatalog(sessionState: SessionState) extends TableCatalog { + def this() = { + this(SparkSession.active.sessionState) + } + + private lazy val catalog: SessionCatalog = sessionState.catalog + + private var _name: String = _ + + override def name: String = _name + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + this._name = name + } + + override def listTables(namespace: Array[String]): Array[Identifier] = { + namespace match { + case Array(db) => + catalog.listTables(db).map(ident => Identifier.of(Array(db), ident.table)).toArray + case _ => + throw new NoSuchNamespaceException(namespace) + } + } + + override def loadTable(ident: Identifier): Table = { + val catalogTable = try { + catalog.getTableMetadata(ident.asTableIdentifier) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + CatalogTableAsV2(catalogTable) + } + + override def invalidateTable(ident: Identifier): Unit = { + catalog.refreshTable(ident.asTableIdentifier) + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + val (partitionColumns, maybeBucketSpec) = V2SessionCatalog.convertTransforms(partitions) + val provider = properties.getOrDefault("provider", sessionState.conf.defaultDataSourceName) + val tableProperties = properties.asScala + val location = Option(properties.get("location")) + val storage = DataSource.buildStorageFormatFromOptions(tableProperties.toMap) + .copy(locationUri = location.map(CatalogUtils.stringToURI)) + + val tableDesc = CatalogTable( + identifier = ident.asTableIdentifier, + tableType = CatalogTableType.MANAGED, + storage = storage, + schema = schema, + provider = Some(provider), + partitionColumnNames = partitionColumns, + bucketSpec = maybeBucketSpec, + properties = tableProperties.toMap, + tracksPartitionsInCatalog = sessionState.conf.manageFilesourcePartitions, + comment = Option(properties.get("comment"))) + + try { + catalog.createTable(tableDesc, ignoreIfExists = false) + } catch { + case _: TableAlreadyExistsException => + throw new TableAlreadyExistsException(ident) + } + + loadTable(ident) + } + + override def alterTable( + ident: Identifier, + changes: TableChange*): Table = { + val catalogTable = try { + catalog.getTableMetadata(ident.asTableIdentifier) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + val properties = CatalogV2Util.applyPropertiesChanges(catalogTable.properties, changes) + val schema = CatalogV2Util.applySchemaChanges(catalogTable.schema, changes) + + try { + catalog.alterTable(catalogTable.copy(properties = properties, schema = schema)) + } catch { + case _: NoSuchTableException => + throw new NoSuchTableException(ident) + } + + loadTable(ident) + } + + override def dropTable(ident: Identifier): Boolean = { + try { + if (loadTable(ident) != null) { + catalog.dropTable( + ident.asTableIdentifier, + ignoreIfNotExists = true, + purge = true /* skip HDFS trash */) + true + } else { + false + } + } catch { + case _: NoSuchTableException => + false + } + } + + implicit class TableIdentifierHelper(ident: Identifier) { + def asTableIdentifier: TableIdentifier = { + ident.namespace match { + case Array(db) => + TableIdentifier(ident.name, Some(db)) + case Array() => + TableIdentifier(ident.name, Some(catalog.getCurrentDatabase)) + case _ => + throw new NoSuchTableException(ident) + } + } + } + + override def toString: String = s"V2SessionCatalog($name)" +} + +/** + * An implementation of catalog v2 [[Table]] to expose v1 table metadata. + */ +case class CatalogTableAsV2(v1Table: CatalogTable) extends Table { + implicit class IdentifierHelper(identifier: TableIdentifier) { + def quoted: String = { + identifier.database match { + case Some(db) => + Seq(db, identifier.table).map(quote).mkString(".") + case _ => + quote(identifier.table) + + } + } + + private def quote(part: String): String = { + if (part.contains(".") || part.contains("`")) { + s"`${part.replace("`", "``")}`" + } else { + part + } + } + } + + def catalogTable: CatalogTable = v1Table + + lazy val options: Map[String, String] = { + v1Table.storage.locationUri match { + case Some(uri) => + v1Table.storage.properties + ("path" -> uri.toString) + case _ => + v1Table.storage.properties + } + } + + override lazy val properties: util.Map[String, String] = v1Table.properties.asJava + + override lazy val schema: StructType = v1Table.schema + + override lazy val partitioning: Array[Transform] = { + val partitions = new mutable.ArrayBuffer[Transform]() + + v1Table.partitionColumnNames.foreach { col => + partitions += LogicalExpressions.identity(col) + } + + v1Table.bucketSpec.foreach { spec => + partitions += LogicalExpressions.bucket(spec.numBuckets, spec.bucketColumnNames: _*) + } + + partitions.toArray + } + + override def name: String = v1Table.identifier.quoted + + override def capabilities: util.Set[TableCapability] = new util.HashSet[TableCapability]() + + override def toString: String = s"CatalogTableAsV2($name)" +} + +private[sql] object V2SessionCatalog { + /** + * Convert v2 Transforms to v1 partition columns and an optional bucket spec. + */ + private def convertTransforms(partitions: Seq[Transform]): (Seq[String], Option[BucketSpec]) = { + val identityCols = new mutable.ArrayBuffer[String] + var bucketSpec = Option.empty[BucketSpec] + + partitions.map { + case IdentityTransform(FieldReference(Seq(col))) => + identityCols += col + + case BucketTransform(numBuckets, FieldReference(Seq(col))) => + bucketSpec = Some(BucketSpec(numBuckets, col :: Nil, Nil)) + + case transform => + throw new UnsupportedOperationException( + s"SessionCatalog does not support partition transform: $transform") + } + + (identityCols, bucketSpec) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheck.scala new file mode 100644 index 0000000000000..c029acc0bb2df --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheck.scala @@ -0,0 +1,64 @@ +/* + * 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.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} +import org.apache.spark.sql.sources.v2.TableCapability.{CONTINUOUS_READ, MICRO_BATCH_READ} + +/** + * This rules adds some basic table capability check for streaming scan, without knowing the actual + * streaming execution mode. + */ +object V2StreamingScanSupportCheck extends (LogicalPlan => Unit) { + import DataSourceV2Implicits._ + + override def apply(plan: LogicalPlan): Unit = { + plan.foreach { + case r: StreamingRelationV2 if !r.table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => + throw new AnalysisException( + s"Table ${r.table.name()} does not support either micro-batch or continuous scan.") + case _ => + } + + val streamingSources = plan.collect { + case r: StreamingRelationV2 => r.table + } + val v1StreamingRelations = plan.collect { + case r: StreamingRelation => r + } + + if (streamingSources.length + v1StreamingRelations.length > 1) { + val allSupportsMicroBatch = streamingSources.forall(_.supports(MICRO_BATCH_READ)) + // v1 streaming data source only supports micro-batch. + val allSupportsContinuous = streamingSources.forall(_.supports(CONTINUOUS_READ)) && + v1StreamingRelations.isEmpty + if (!allSupportsMicroBatch && !allSupportsContinuous) { + val microBatchSources = + streamingSources.filter(_.supports(MICRO_BATCH_READ)).map(_.name()) ++ + v1StreamingRelations.map(_.sourceName) + val continuousSources = streamingSources.filter(_.supports(CONTINUOUS_READ)).map(_.name()) + throw new AnalysisException( + "The streaming sources in a query do not have a common supported execution mode.\n" + + "Sources support micro-batch: " + microBatchSources.mkString(", ") + "\n" + + "Sources support continuous: " + continuousSources.mkString(", ")) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala new file mode 100644 index 0000000000000..cf77998c122f8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/V2WriteSupportCheck.scala @@ -0,0 +1,56 @@ +/* + * 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.v2 + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.types.BooleanType + +object V2WriteSupportCheck extends (LogicalPlan => Unit) { + import DataSourceV2Implicits._ + + def failAnalysis(msg: String): Unit = throw new AnalysisException(msg) + + override def apply(plan: LogicalPlan): Unit = plan foreach { + case AppendData(rel: DataSourceV2Relation, _, _) if !rel.table.supports(BATCH_WRITE) => + failAnalysis(s"Table does not support append in batch mode: ${rel.table}") + + case OverwritePartitionsDynamic(rel: DataSourceV2Relation, _, _) + if !rel.table.supports(BATCH_WRITE) || !rel.table.supports(OVERWRITE_DYNAMIC) => + failAnalysis(s"Table does not support dynamic overwrite in batch mode: ${rel.table}") + + case OverwriteByExpression(rel: DataSourceV2Relation, expr, _, _) => + expr match { + case Literal(true, BooleanType) => + if (!rel.table.supports(BATCH_WRITE) || + !rel.table.supportsAny(TRUNCATE, OVERWRITE_BY_FILTER)) { + failAnalysis( + s"Table does not support truncate in batch mode: ${rel.table}") + } + case _ => + if (!rel.table.supports(BATCH_WRITE) || !rel.table.supports(OVERWRITE_BY_FILTER)) { + failAnalysis(s"Table does not support overwrite expression ${expr.sql} " + + s"in batch mode: ${rel.table}") + } + } + + case _ => // OK + } +} 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 c3f7b690ef636..6c771ea988324 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 @@ -17,75 +17,250 @@ package org.apache.spark.sql.execution.datasources.v2 +import java.util.UUID + +import scala.collection.JavaConverters._ import scala.util.control.NonFatal import org.apache.spark.{SparkEnv, SparkException, TaskContext} import org.apache.spark.executor.CommitDeniedException import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalog.v2.{Identifier, TableCatalog} +import org.apache.spark.sql.catalog.v2.expressions.Transform import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.TableAlreadyExistsException import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.util.Utils +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.sources.{AlwaysTrue, Filter} +import org.apache.spark.sql.sources.v2.SupportsWrite +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriterFactory, SupportsDynamicOverwrite, SupportsOverwrite, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.{LongAccumulator, Utils} /** * Deprecated logical plan for writing data into data source v2. This is being replaced by more * specific logical plans, like [[org.apache.spark.sql.catalyst.plans.logical.AppendData]]. */ @deprecated("Use specific logical plans like AppendData instead", "2.4.0") -case class WriteToDataSourceV2(writeSupport: BatchWriteSupport, query: LogicalPlan) +case class WriteToDataSourceV2(batchWrite: BatchWrite, query: LogicalPlan) extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(query) override def output: Seq[Attribute] = Nil } /** - * The physical plan for writing data into data source v2. + * Physical plan node for v2 create table as select. + * + * A new table will be created using the schema of the query, and rows from the query are appended. + * If either table creation or the append fails, the table will be deleted. This implementation does + * not provide an atomic CTAS. */ -case class WriteToDataSourceV2Exec(writeSupport: BatchWriteSupport, query: SparkPlan) - extends SparkPlan { +case class CreateTableAsSelectExec( + catalog: TableCatalog, + ident: Identifier, + partitioning: Seq[Transform], + query: SparkPlan, + properties: Map[String, String], + writeOptions: CaseInsensitiveStringMap, + ifNotExists: Boolean) extends V2TableWriteExec { - override def children: Seq[SparkPlan] = Seq(query) - override def output: Seq[Attribute] = Nil + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits.IdentifierHelper override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = writeSupport.createBatchWriterFactory() - val useCommitCoordinator = writeSupport.useCommitCoordinator + if (catalog.tableExists(ident)) { + if (ifNotExists) { + return sparkContext.parallelize(Seq.empty, 1) + } + + throw new TableAlreadyExistsException(ident) + } + + Utils.tryWithSafeFinallyAndFailureCallbacks({ + catalog.createTable(ident, query.schema, partitioning.toArray, properties.asJava) match { + case table: SupportsWrite => + val batchWrite = table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + .buildForBatch() + + doWrite(batchWrite) + + case _ => + // table does not support writes + throw new SparkException(s"Table implementation does not support writes: ${ident.quoted}") + } + + })(catchBlock = { + catalog.dropTable(ident) + }) + } +} + +/** + * Physical plan node for append into a v2 table. + * + * Rows in the output data set are appended. + */ +case class AppendDataExec( + table: SupportsWrite, + writeOptions: CaseInsensitiveStringMap, + query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper { + + override protected def doExecute(): RDD[InternalRow] = { + val batchWrite = newWriteBuilder().buildForBatch() + doWrite(batchWrite) + } +} + +/** + * Physical plan node for overwrite into a v2 table. + * + * Overwrites data in a table matched by a set of filters. Rows matching all of the filters will be + * deleted and rows in the output data set are appended. + * + * This plan is used to implement SaveMode.Overwrite. The behavior of SaveMode.Overwrite is to + * truncate the table -- delete all rows -- and append the output data set. This uses the filter + * AlwaysTrue to delete all rows. + */ +case class OverwriteByExpressionExec( + table: SupportsWrite, + deleteWhere: Array[Filter], + writeOptions: CaseInsensitiveStringMap, + query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper { + + private def isTruncate(filters: Array[Filter]): Boolean = { + filters.length == 1 && filters(0).isInstanceOf[AlwaysTrue] + } + + override protected def doExecute(): RDD[InternalRow] = { + val batchWrite = newWriteBuilder() match { + case builder: SupportsTruncate if isTruncate(deleteWhere) => + builder.truncate().buildForBatch() + + case builder: SupportsOverwrite => + builder.overwrite(deleteWhere).buildForBatch() + + case _ => + throw new SparkException(s"Table does not support overwrite by expression: $table") + } + + doWrite(batchWrite) + } +} + +/** + * Physical plan node for dynamic partition overwrite into a v2 table. + * + * Dynamic partition overwrite is the behavior of Hive INSERT OVERWRITE ... PARTITION queries, and + * Spark INSERT OVERWRITE queries when spark.sql.sources.partitionOverwriteMode=dynamic. Each + * partition in the output data set replaces the corresponding existing partition in the table or + * creates a new partition. Existing partitions for which there is no data in the output data set + * are not modified. + */ +case class OverwritePartitionsDynamicExec( + table: SupportsWrite, + writeOptions: CaseInsensitiveStringMap, + query: SparkPlan) extends V2TableWriteExec with BatchWriteHelper { + + override protected def doExecute(): RDD[InternalRow] = { + val batchWrite = newWriteBuilder() match { + case builder: SupportsDynamicOverwrite => + builder.overwriteDynamicPartitions().buildForBatch() + + case _ => + throw new SparkException(s"Table does not support dynamic partition overwrite: $table") + } + + doWrite(batchWrite) + } +} + +case class WriteToDataSourceV2Exec( + batchWrite: BatchWrite, + query: SparkPlan) extends V2TableWriteExec { + + def writeOptions: CaseInsensitiveStringMap = CaseInsensitiveStringMap.empty() + + override protected def doExecute(): RDD[InternalRow] = { + doWrite(batchWrite) + } +} + +/** + * Helper for physical plans that build batch writes. + */ +trait BatchWriteHelper { + def table: SupportsWrite + def query: SparkPlan + def writeOptions: CaseInsensitiveStringMap + + def newWriteBuilder(): WriteBuilder = { + table.newWriteBuilder(writeOptions) + .withInputDataSchema(query.schema) + .withQueryId(UUID.randomUUID().toString) + } +} + +/** + * The base physical plan for writing data into data source v2. + */ +trait V2TableWriteExec extends UnaryExecNode { + def query: SparkPlan + + var commitProgress: Option[StreamWriterCommitProgress] = None + + override def child: SparkPlan = query + override def output: Seq[Attribute] = Nil + + protected def doWrite(batchWrite: BatchWrite): RDD[InternalRow] = { + val writerFactory = batchWrite.createBatchWriterFactory() + val useCommitCoordinator = batchWrite.useCommitCoordinator val rdd = query.execute() - val messages = new Array[WriterCommitMessage](rdd.partitions.length) + // SPARK-23271 If we are attempting to write a zero partition rdd, create a dummy single + // partition rdd to make sure we at least set up one write task to write the metadata. + val rddWithNonEmptyPartitions = if (rdd.partitions.length == 0) { + sparkContext.parallelize(Array.empty[InternalRow], 1) + } else { + rdd + } + val messages = new Array[WriterCommitMessage](rddWithNonEmptyPartitions.partitions.length) + val totalNumRowsAccumulator = new LongAccumulator() - logInfo(s"Start processing data source write support: $writeSupport. " + + logInfo(s"Start processing data source write support: $batchWrite. " + s"The input RDD has ${messages.length} partitions.") try { sparkContext.runJob( - rdd, + rddWithNonEmptyPartitions, (context: TaskContext, iter: Iterator[InternalRow]) => DataWritingSparkTask.run(writerFactory, context, iter, useCommitCoordinator), - rdd.partitions.indices, - (index, message: WriterCommitMessage) => { - messages(index) = message - writeSupport.onDataWriterCommit(message) + rddWithNonEmptyPartitions.partitions.indices, + (index, result: DataWritingSparkTaskResult) => { + val commitMessage = result.writerCommitMessage + messages(index) = commitMessage + totalNumRowsAccumulator.add(result.numRows) + batchWrite.onDataWriterCommit(commitMessage) } ) - logInfo(s"Data source write support $writeSupport is committing.") - writeSupport.commit(messages) - logInfo(s"Data source write support $writeSupport committed.") + logInfo(s"Data source write support $batchWrite is committing.") + batchWrite.commit(messages) + logInfo(s"Data source write support $batchWrite committed.") + commitProgress = Some(StreamWriterCommitProgress(totalNumRowsAccumulator.value)) } catch { case cause: Throwable => - logError(s"Data source write support $writeSupport is aborting.") + logError(s"Data source write support $batchWrite is aborting.") try { - writeSupport.abort(messages) + batchWrite.abort(messages) } catch { case t: Throwable => - logError(s"Data source write support $writeSupport failed to abort.") + logError(s"Data source write support $batchWrite failed to abort.") cause.addSuppressed(t) throw new SparkException("Writing job failed.", cause) } - logError(s"Data source write support $writeSupport aborted.") + logError(s"Data source write support $batchWrite aborted.") cause match { // Only wrap non fatal exceptions. case NonFatal(e) => throw new SparkException("Writing job aborted.", e) @@ -102,7 +277,7 @@ object DataWritingSparkTask extends Logging { writerFactory: DataWriterFactory, context: TaskContext, iter: Iterator[InternalRow], - useCommitCoordinator: Boolean): WriterCommitMessage = { + useCommitCoordinator: Boolean): DataWritingSparkTaskResult = { val stageId = context.stageId() val stageAttempt = context.stageAttemptNumber() val partId = context.partitionId() @@ -110,9 +285,12 @@ object DataWritingSparkTask extends Logging { val attemptId = context.attemptNumber() val dataWriter = writerFactory.createWriter(partId, taskId) + var count = 0L // write the data and commit this writer. Utils.tryWithSafeFinallyAndFailureCallbacks(block = { while (iter.hasNext) { + // Count is here. + count += 1 dataWriter.write(iter.next()) } @@ -139,7 +317,7 @@ object DataWritingSparkTask extends Logging { logInfo(s"Committed partition $partId (task $taskId, attempt $attemptId" + s"stage $stageId.$stageAttempt)") - msg + DataWritingSparkTaskResult(count, msg) })(catchBlock = { // If there is an error, abort this writer @@ -151,3 +329,12 @@ object DataWritingSparkTask extends Logging { }) } } + +private[v2] case class DataWritingSparkTaskResult( + numRows: Long, + writerCommitMessage: WriterCommitMessage) + +/** + * Sink progress information collected after commit. + */ +private[sql] case class StreamWriterCommitProgress(numOutputRows: Long) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala new file mode 100644 index 0000000000000..045f41e670ad3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVDataSourceV2.scala @@ -0,0 +1,43 @@ +/* + * 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.v2.csv + +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.csv.CSVFileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class CSVDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[CSVFileFormat] + + override def shortName(): String = "csv" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + CSVTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + CSVTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala new file mode 100644 index 0000000000000..828594ffb10af --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVPartitionReaderFactory.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.csv + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.csv.{CSVHeaderChecker, CSVOptions, UnivocityParser} +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.csv.CSVDataSource +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create CSV readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcasted serializable Hadoop Configuration. + * @param dataSchema Schema of CSV files. + * @param readDataSchema Required data schema in the batch scan. + * @param partitionSchema Schema of partitions. + * @param parsedOptions Options for parsing CSV files. + */ +case class CSVPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType, + parsedOptions: CSVOptions) extends FilePartitionReaderFactory { + private val columnPruning = sqlConf.csvColumnPruning + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + val conf = broadcastedConf.value.value + val actualDataSchema = StructType( + dataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val actualReadDataSchema = StructType( + readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val parser = new UnivocityParser( + actualDataSchema, + actualReadDataSchema, + parsedOptions) + val schema = if (columnPruning) actualReadDataSchema else actualDataSchema + val isStartOfFile = file.start == 0 + val headerChecker = new CSVHeaderChecker( + schema, parsedOptions, source = s"CSV file: ${file.filePath}", isStartOfFile) + val iter = CSVDataSource(parsedOptions).readFile( + conf, + file, + parser, + headerChecker, + readDataSchema) + val fileReader = new PartitionReaderFromIterator[InternalRow](iter) + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, file.partitionValues) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala new file mode 100644 index 0000000000000..5bc8029b4068a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScan.scala @@ -0,0 +1,81 @@ +/* + * 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.v2.csv + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.csv.CSVDataSource +import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class CSVScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap) + extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { + + private lazy val parsedOptions: CSVOptions = new CSVOptions( + options.asScala.toMap, + columnPruning = sparkSession.sessionState.conf.csvColumnPruning, + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + + override def isSplitable(path: Path): Boolean = { + CSVDataSource(parsedOptions).isSplitable && super.isSplitable(path) + } + + override def createReaderFactory(): PartitionReaderFactory = { + // Check a field requirement for corrupt records here to throw an exception in a driver side + ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) + + if (readDataSchema.length == 1 && + readDataSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { + throw new AnalysisException( + "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the\n" + + "referenced columns only include the internal corrupt record column\n" + + "(named _corrupt_record by default). For example:\n" + + "spark.read.schema(schema).csv(file).filter($\"_corrupt_record\".isNotNull).count()\n" + + "and spark.read.schema(schema).csv(file).select(\"_corrupt_record\").show().\n" + + "Instead, you can cache or save the parsed results and then send the same query.\n" + + "For example, val df = spark.read.schema(schema).csv(file).cache() and then\n" + + "df.filter($\"_corrupt_record\".isNotNull).count()." + ) + } + + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + // The partition values are already truncated in `FileScan.partitions`. + // We should use `readPartitionSchema` as the partition schema here. + CSVPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.scala new file mode 100644 index 0000000000000..28c5b3d81a3d5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVScanBuilder.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.sql.execution.datasources.v2.csv + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class CSVScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + + override def build(): Scan = { + CSVScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala new file mode 100644 index 0000000000000..8170661a70172 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVTable.scala @@ -0,0 +1,64 @@ +/* + * 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.v2.csv + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.csv.CSVDataSource +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{AtomicType, DataType, StructType, UserDefinedType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class CSVTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + override def newScanBuilder(options: CaseInsensitiveStringMap): CSVScanBuilder = + CSVScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { + val parsedOptions = new CSVOptions( + options.asScala.toMap, + columnPruning = sparkSession.sessionState.conf.csvColumnPruning, + sparkSession.sessionState.conf.sessionLocalTimeZone) + + CSVDataSource(parsedOptions).inferSchema(sparkSession, files, parsedOptions) + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new CSVWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) + + case _ => false + } + + override def formatName: String = "CSV" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWriteBuilder.scala new file mode 100644 index 0000000000000..92b47e4354807 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/csv/CSVWriteBuilder.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.execution.datasources.v2.csv + +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.sql.catalyst.csv.CSVOptions +import org.apache.spark.sql.catalyst.util.CompressionCodecs +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.csv.CsvOutputWriter +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class CSVWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) { + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val conf = job.getConfiguration + val csvOptions = new CSVOptions( + options, + columnPruning = sqlConf.csvColumnPruning, + sqlConf.sessionLocalTimeZone) + csvOptions.compressionCodec.foreach { codec => + CompressionCodecs.setCodecConfiguration(conf, codec) + } + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new CsvOutputWriter(path, dataSchema, context, csvOptions) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".csv" + CodecStreams.getCompressionExtension(context) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala new file mode 100644 index 0000000000000..610bd4c1b9d85 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonDataSourceV2.scala @@ -0,0 +1,44 @@ +/* + * 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.v2.json + +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.json.JsonFileFormat +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JsonDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[JsonFileFormat] + + override def shortName(): String = "json" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + JsonTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + JsonTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala new file mode 100644 index 0000000000000..e5b7ae0bd228a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonPartitionReaderFactory.scala @@ -0,0 +1,61 @@ +/* + * 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.v2.json + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.json.{JacksonParser, JSONOptionsInRead} +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.execution.datasources.json.JsonDataSource +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create JSON readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of JSON files. + * @param readDataSchema Required schema of JSON files. + * @param partitionSchema Schema of partitions. + * @param parsedOptions Options for parsing JSON files. + */ +case class JsonPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType, + parsedOptions: JSONOptionsInRead) extends FilePartitionReaderFactory { + + override def buildReader(partitionedFile: PartitionedFile): PartitionReader[InternalRow] = { + val actualSchema = + StructType(readDataSchema.filterNot(_.name == parsedOptions.columnNameOfCorruptRecord)) + val parser = new JacksonParser(actualSchema, parsedOptions, allowArrayAsStructs = true) + val iter = JsonDataSource(parsedOptions).readFile( + broadcastedConf.value.value, + partitionedFile, + parser, + readDataSchema) + val fileReader = new PartitionReaderFromIterator[InternalRow](iter) + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, partitionedFile.partitionValues) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.scala new file mode 100644 index 0000000000000..201572b4338b6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScan.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.sql.execution.datasources.v2.json + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.{AnalysisException, SparkSession} +import org.apache.spark.sql.catalyst.expressions.ExprUtils +import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.json.JsonDataSource +import org.apache.spark.sql.execution.datasources.v2.{FileScan, TextBasedFileScan} +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class JsonScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap) + extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { + + private val parsedOptions = new JSONOptionsInRead( + CaseInsensitiveMap(options.asScala.toMap), + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + + override def isSplitable(path: Path): Boolean = { + JsonDataSource(parsedOptions).isSplitable && super.isSplitable(path) + } + + override def createReaderFactory(): PartitionReaderFactory = { + // Check a field requirement for corrupt records here to throw an exception in a driver side + ExprUtils.verifyColumnNameOfCorruptRecord(dataSchema, parsedOptions.columnNameOfCorruptRecord) + + if (readDataSchema.length == 1 && + readDataSchema.head.name == parsedOptions.columnNameOfCorruptRecord) { + throw new AnalysisException( + "Since Spark 2.3, the queries from raw JSON/CSV files are disallowed when the\n" + + "referenced columns only include the internal corrupt record column\n" + + s"(named _corrupt_record by default). For example:\n" + + "spark.read.schema(schema).json(file).filter($\"_corrupt_record\".isNotNull).count()\n" + + "and spark.read.schema(schema).json(file).select(\"_corrupt_record\").show().\n" + + "Instead, you can cache or save the parsed results and then send the same query.\n" + + "For example, val df = spark.read.schema(schema).json(file).cache() and then\n" + + "df.filter($\"_corrupt_record\".isNotNull).count()." + ) + } + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + val hadoopConf = sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + // The partition values are already truncated in `FileScan.partitions`. + // We should use `readPartitionSchema` as the partition schema here. + JsonPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, readDataSchema, readPartitionSchema, parsedOptions) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala new file mode 100644 index 0000000000000..bb3c0366bdc2f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonScanBuilder.scala @@ -0,0 +1,36 @@ +/* + * 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.v2.json + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JsonScanBuilder ( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + override def build(): Scan = { + JsonScan(sparkSession, fileIndex, dataSchema, readDataSchema(), readPartitionSchema(), options) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala new file mode 100644 index 0000000000000..bbdd3ae69222a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonTable.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.json + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.catalyst.json.JSONOptionsInRead +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.json.JsonDataSource +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class JsonTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + override def newScanBuilder(options: CaseInsensitiveStringMap): JsonScanBuilder = + new JsonScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = { + val parsedOptions = new JSONOptionsInRead( + options.asScala.toMap, + sparkSession.sessionState.conf.sessionLocalTimeZone, + sparkSession.sessionState.conf.columnNameOfCorruptRecord) + JsonDataSource(parsedOptions).inferSchema( + sparkSession, files, parsedOptions) + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new JsonWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall { f => supportsDataType(f.dataType) } + + case ArrayType(elementType, _) => supportsDataType(elementType) + + case MapType(keyType, valueType, _) => + supportsDataType(keyType) && supportsDataType(valueType) + + case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) + + case _: NullType => true + + case _ => false + } + + override def formatName: String = "JSON" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonWriteBuilder.scala new file mode 100644 index 0000000000000..3c99e07489a77 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/json/JsonWriteBuilder.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.execution.datasources.v2.json + +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.sql.catalyst.json.JSONOptions +import org.apache.spark.sql.catalyst.util.CompressionCodecs +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.json.JsonOutputWriter +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class JsonWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) { + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val conf = job.getConfiguration + val parsedOptions = new JSONOptions( + options, + sqlConf.sessionLocalTimeZone, + sqlConf.columnNameOfCorruptRecord) + parsedOptions.compressionCodec.foreach { codec => + CompressionCodecs.setCodecConfiguration(conf, codec) + } + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new JsonOutputWriter(path, parsedOptions, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".json" + CodecStreams.getCompressionExtension(context) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala new file mode 100644 index 0000000000000..1ea80d2ba5fbc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcDataSourceV2.scala @@ -0,0 +1,44 @@ +/* + * 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.v2.orc + +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.orc.OrcFileFormat +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class OrcDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[OrcFileFormat] + + override def shortName(): String = "orc" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + OrcTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + OrcTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala new file mode 100644 index 0000000000000..ec923797e2691 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcPartitionReaderFactory.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import java.net.URI + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.{JobID, TaskAttemptID, TaskID, TaskType} +import org.apache.hadoop.mapreduce.lib.input.FileSplit +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.orc.{OrcConf, OrcFile, TypeDescription} +import org.apache.orc.mapred.OrcStruct +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.JoinedRow +import org.apache.spark.sql.catalyst.expressions.codegen.GenerateUnsafeProjection +import org.apache.spark.sql.execution.datasources.{PartitionedFile, PartitioningUtils} +import org.apache.spark.sql.execution.datasources.orc.{OrcColumnarBatchReader, OrcDeserializer, OrcUtils} +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} +import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create Orc readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of orc files. + * @param readDataSchema Required data schema in the batch scan. + * @param partitionSchema Schema of partitions. + */ +case class OrcPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType) extends FilePartitionReaderFactory { + private val resultSchema = StructType(readDataSchema.fields ++ partitionSchema.fields) + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + private val capacity = sqlConf.orcVectorizedReaderBatchSize + + override def supportColumnarReads(partition: InputPartition): Boolean = { + sqlConf.orcVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + resultSchema.length <= sqlConf.wholeStageMaxNumFields && + resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + val conf = broadcastedConf.value.value + + val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) + OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyPartitionReader[InternalRow] + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get + assert(requestedColIds.length == readDataSchema.length, + "[BUG] requested column IDs do not match required schema") + + val taskConf = new Configuration(conf) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val orcRecordReader = new OrcInputFormat[OrcStruct] + .createRecordReader(fileSplit, taskAttemptContext) + val deserializer = new OrcDeserializer(dataSchema, readDataSchema, requestedColIds) + val fileReader = new PartitionReader[InternalRow] { + override def next(): Boolean = orcRecordReader.nextKeyValue() + + override def get(): InternalRow = deserializer.deserialize(orcRecordReader.getCurrentValue) + + override def close(): Unit = orcRecordReader.close() + } + + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, file.partitionValues) + } + } + + override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { + val conf = broadcastedConf.value.value + + val resultSchemaString = OrcUtils.orcTypeDescriptionString(resultSchema) + OrcConf.MAPRED_INPUT_SCHEMA.setString(conf, resultSchemaString) + OrcConf.IS_SCHEMA_EVOLUTION_CASE_SENSITIVE.setBoolean(conf, isCaseSensitive) + + val filePath = new Path(new URI(file.filePath)) + + val fs = filePath.getFileSystem(conf) + val readerOptions = OrcFile.readerOptions(conf).filesystem(fs) + val reader = OrcFile.createReader(filePath, readerOptions) + + val requestedColIdsOrEmptyFile = OrcUtils.requestedColumnIds( + isCaseSensitive, dataSchema, readDataSchema, reader, conf) + + if (requestedColIdsOrEmptyFile.isEmpty) { + new EmptyPartitionReader + } else { + val requestedColIds = requestedColIdsOrEmptyFile.get ++ Array.fill(partitionSchema.length)(-1) + assert(requestedColIds.length == resultSchema.length, + "[BUG] requested column IDs do not match required schema") + val taskConf = new Configuration(conf) + + val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val taskAttemptContext = new TaskAttemptContextImpl(taskConf, attemptId) + + val batchReader = new OrcColumnarBatchReader(capacity) + batchReader.initialize(fileSplit, taskAttemptContext) + val requestedPartitionColIds = + Array.fill(readDataSchema.length)(-1) ++ Range(0, partitionSchema.length) + + batchReader.initBatch( + TypeDescription.fromString(resultSchemaString), + resultSchema.fields, + requestedColIds, + requestedPartitionColIds, + file.partitionValues) + new PartitionRecordReader(batchReader) + } + } + +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala new file mode 100644 index 0000000000000..a4fb03405d162 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScan.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class OrcScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap, + pushedFilters: Array[Filter]) + extends FileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema) { + override def isSplitable(path: Path): Boolean = true + + override def createReaderFactory(): PartitionReaderFactory = { + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + // The partition values are already truncated in `FileScan.partitions`. + // We should use `readPartitionSchema` as the partition schema here. + OrcPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, readDataSchema, readPartitionSchema) + } + + override def equals(obj: Any): Boolean = obj match { + case o: OrcScan => + fileIndex == o.fileIndex && dataSchema == o.dataSchema && + readDataSchema == o.readDataSchema && readPartitionSchema == o.readPartitionSchema && + options == o.options && equivalentFilters(pushedFilters, o.pushedFilters) + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() + + override def description(): String = { + super.description() + ", PushedFilters: " + pushedFilters.mkString("[", ", ", "]") + } +} 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 new file mode 100644 index 0000000000000..458b98c627be4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcScanBuilder.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.execution.datasources.v2.orc + +import scala.collection.JavaConverters._ + +import org.apache.orc.mapreduce.OrcInputFormat + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.orc.OrcFilters +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class OrcScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { + lazy val hadoopConf = { + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + } + + override def build(): Scan = { + OrcScan(sparkSession, hadoopConf, fileIndex, dataSchema, + readDataSchema(), readPartitionSchema(), options, pushedFilters()) + } + + private var _pushedFilters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + if (sparkSession.sessionState.conf.orcFilterPushDown) { + OrcFilters.createFilter(schema, filters).foreach { f => + // The pushed filters will be set in `hadoopConf`. After that, we can simply use the + // 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 + } + filters + } + + override def pushedFilters(): Array[Filter] = _pushedFilters +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala new file mode 100644 index 0000000000000..3fe433861a3c4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcTable.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.orc + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.orc.OrcUtils +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class OrcTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + + override def newScanBuilder(options: CaseInsensitiveStringMap): OrcScanBuilder = + new OrcScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + OrcUtils.inferSchema(sparkSession, files, options.asScala.toMap) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new OrcWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall { f => supportsDataType(f.dataType) } + + case ArrayType(elementType, _) => supportsDataType(elementType) + + case MapType(keyType, valueType, _) => + supportsDataType(keyType) && supportsDataType(valueType) + + case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) + + case _ => false + } + + override def formatName: String = "ORC" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcWriteBuilder.scala new file mode 100644 index 0000000000000..f5b06e11c8bd7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/orc/OrcWriteBuilder.scala @@ -0,0 +1,72 @@ +/* + * 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.v2.orc + +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} +import org.apache.orc.OrcConf.{COMPRESS, MAPRED_OUTPUT_SCHEMA} +import org.apache.orc.mapred.OrcStruct + +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.orc.{OrcFileFormat, OrcOptions, OrcOutputWriter, OrcUtils} +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class OrcWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) { + + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val orcOptions = new OrcOptions(options, sqlConf) + + val conf = job.getConfiguration + + conf.set(MAPRED_OUTPUT_SCHEMA.getAttribute, OrcFileFormat.getQuotedSchemaString(dataSchema)) + + conf.set(COMPRESS.getAttribute, orcOptions.compressionCodec) + + conf.asInstanceOf[JobConf] + .setOutputFormat(classOf[org.apache.orc.mapred.OrcOutputFormat[OrcStruct]]) + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new OrcOutputWriter(path, dataSchema, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + val compressionExtension: String = { + val name = context.getConfiguration.get(COMPRESS.getAttribute) + OrcUtils.extensionsForCompressionCodecNames.getOrElse(name, "") + } + + compressionExtension + ".orc" + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala new file mode 100644 index 0000000000000..0b6d5a960374b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetDataSourceV2.scala @@ -0,0 +1,44 @@ +/* + * 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.v2.parquet + +import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class ParquetDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + ParquetTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + ParquetTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala new file mode 100644 index 0000000000000..4a281ba46eb5f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetPartitionReaderFactory.scala @@ -0,0 +1,227 @@ +/* + * 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.v2.parquet + +import java.net.URI +import java.util.TimeZone + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl +import org.apache.parquet.filter2.compat.FilterCompat +import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate} +import org.apache.parquet.format.converter.ParquetMetadataConverter.SKIP_ROW_GROUPS +import org.apache.parquet.hadoop.{ParquetFileReader, ParquetInputFormat, ParquetInputSplit, ParquetRecordReader} + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.Logging +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.execution.datasources.{PartitionedFile, RecordReaderIterator} +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader} +import org.apache.spark.sql.types.{AtomicType, StructType} +import org.apache.spark.sql.vectorized.ColumnarBatch +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create Parquet readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcast serializable Hadoop Configuration. + * @param dataSchema Schema of Parquet files. + * @param readDataSchema Required schema of Parquet files. + * @param partitionSchema Schema of partitions. + * @param filters Filters to be pushed down in the batch scan. + */ +case class ParquetPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + dataSchema: StructType, + readDataSchema: StructType, + partitionSchema: StructType, + filters: Array[Filter]) extends FilePartitionReaderFactory with Logging { + private val isCaseSensitive = sqlConf.caseSensitiveAnalysis + private val resultSchema = StructType(partitionSchema.fields ++ readDataSchema.fields) + private val enableOffHeapColumnVector = sqlConf.offHeapColumnVectorEnabled + private val enableVectorizedReader: Boolean = sqlConf.parquetVectorizedReaderEnabled && + resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) + private val enableRecordFilter: Boolean = sqlConf.parquetRecordFilterEnabled + private val timestampConversion: Boolean = sqlConf.isParquetINT96TimestampConversion + private val capacity = sqlConf.parquetVectorizedReaderBatchSize + private val enableParquetFilterPushDown: Boolean = sqlConf.parquetFilterPushDown + private val pushDownDate = sqlConf.parquetFilterPushDownDate + private val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp + private val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal + private val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith + private val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold + + override def supportColumnarReads(partition: InputPartition): Boolean = { + sqlConf.parquetVectorizedReaderEnabled && sqlConf.wholeStageEnabled && + resultSchema.length <= sqlConf.wholeStageMaxNumFields && + resultSchema.forall(_.dataType.isInstanceOf[AtomicType]) + } + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + val reader = if (enableVectorizedReader) { + createVectorizedReader(file) + } else { + createRowBaseReader(file) + } + + val fileReader = new PartitionReader[InternalRow] { + override def next(): Boolean = reader.nextKeyValue() + + override def get(): InternalRow = reader.getCurrentValue.asInstanceOf[InternalRow] + + override def close(): Unit = reader.close() + } + + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, file.partitionValues) + } + + override def buildColumnarReader(file: PartitionedFile): PartitionReader[ColumnarBatch] = { + val vectorizedReader = createVectorizedReader(file) + vectorizedReader.enableReturningBatches() + + new PartitionReader[ColumnarBatch] { + override def next(): Boolean = vectorizedReader.nextKeyValue() + + override def get(): ColumnarBatch = + vectorizedReader.getCurrentValue.asInstanceOf[ColumnarBatch] + + override def close(): Unit = vectorizedReader.close() + } + } + + private def buildReaderBase[T]( + file: PartitionedFile, + buildReaderFunc: ( + ParquetInputSplit, InternalRow, TaskAttemptContextImpl, Option[FilterPredicate], + Option[TimeZone]) => RecordReader[Void, T]): RecordReader[Void, T] = { + val conf = broadcastedConf.value.value + + val filePath = new Path(new URI(file.filePath)) + val split = + new org.apache.parquet.hadoop.ParquetInputSplit( + filePath, + file.start, + file.start + file.length, + file.length, + Array.empty, + null) + + lazy val footerFileMetaData = + ParquetFileReader.readFooter(conf, filePath, SKIP_ROW_GROUPS).getFileMetaData + // Try to push down filters when filter push-down is enabled. + val pushed = if (enableParquetFilterPushDown) { + val parquetSchema = footerFileMetaData.getSchema + val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, + pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + filters + // Collects all converted Parquet filter predicates. Notice that not all predicates can be + // converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` + // is used here. + .flatMap(parquetFilters.createFilter) + .reduceOption(FilterApi.and) + } else { + None + } + // PARQUET_INT96_TIMESTAMP_CONVERSION says to apply timezone conversions to int96 timestamps' + // *only* if the file was created by something other than "parquet-mr", so check the actual + // writer here for this file. We have to do this per-file, as each file in the table may + // have different writers. + // Define isCreatedByParquetMr as function to avoid unnecessary parquet footer reads. + def isCreatedByParquetMr: Boolean = + footerFileMetaData.getCreatedBy().startsWith("parquet-mr") + + val convertTz = + if (timestampConversion && !isCreatedByParquetMr) { + Some(DateTimeUtils.getTimeZone(conf.get(SQLConf.SESSION_LOCAL_TIMEZONE.key))) + } else { + None + } + + val attemptId = new TaskAttemptID(new TaskID(new JobID(), TaskType.MAP, 0), 0) + val hadoopAttemptContext = new TaskAttemptContextImpl(conf, attemptId) + + // Try to push down filters when filter push-down is enabled. + // Notice: This push-down is RowGroups level, not individual records. + if (pushed.isDefined) { + ParquetInputFormat.setFilterPredicate(hadoopAttemptContext.getConfiguration, pushed.get) + } + val reader = + buildReaderFunc(split, file.partitionValues, hadoopAttemptContext, pushed, convertTz) + reader.initialize(split, hadoopAttemptContext) + reader + } + + private def createRowBaseReader(file: PartitionedFile): RecordReader[Void, UnsafeRow] = { + buildReaderBase(file, createRowBaseParquetReader) + } + + private def createRowBaseParquetReader( + split: ParquetInputSplit, + partitionValues: InternalRow, + hadoopAttemptContext: TaskAttemptContextImpl, + pushed: Option[FilterPredicate], + convertTz: Option[TimeZone]): RecordReader[Void, UnsafeRow] = { + logDebug(s"Falling back to parquet-mr") + val taskContext = Option(TaskContext.get()) + // ParquetRecordReader returns UnsafeRow + val readSupport = new ParquetReadSupport(convertTz, enableVectorizedReader = false) + val reader = if (pushed.isDefined && enableRecordFilter) { + val parquetFilter = FilterCompat.get(pushed.get, null) + new ParquetRecordReader[UnsafeRow](readSupport, parquetFilter) + } else { + new ParquetRecordReader[UnsafeRow](readSupport) + } + val iter = new RecordReaderIterator(reader) + // SPARK-23457 Register a task completion lister before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + reader + } + + private def createVectorizedReader(file: PartitionedFile): VectorizedParquetRecordReader = { + val vectorizedReader = buildReaderBase(file, createParquetVectorizedReader) + .asInstanceOf[VectorizedParquetRecordReader] + vectorizedReader.initBatch(partitionSchema, file.partitionValues) + vectorizedReader + } + + private def createParquetVectorizedReader( + split: ParquetInputSplit, + partitionValues: InternalRow, + hadoopAttemptContext: TaskAttemptContextImpl, + pushed: Option[FilterPredicate], + convertTz: Option[TimeZone]): VectorizedParquetRecordReader = { + val taskContext = Option(TaskContext.get()) + val vectorizedReader = new VectorizedParquetRecordReader( + convertTz.orNull, enableOffHeapColumnVector && taskContext.isDefined, capacity) + val iter = new RecordReaderIterator(vectorizedReader) + // SPARK-23457 Register a task completion lister before `initialization`. + taskContext.foreach(_.addTaskCompletionListener[Unit](_ => iter.close())) + logDebug(s"Appending $partitionSchema $partitionValues") + vectorizedReader + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala new file mode 100644 index 0000000000000..a67aa3b92ce82 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScan.scala @@ -0,0 +1,91 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.parquet + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetInputFormat + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.parquet.{ParquetReadSupport, ParquetWriteSupport} +import org.apache.spark.sql.execution.datasources.v2.FileScan +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class ParquetScan( + sparkSession: SparkSession, + hadoopConf: Configuration, + fileIndex: PartitioningAwareFileIndex, + dataSchema: StructType, + readDataSchema: StructType, + readPartitionSchema: StructType, + pushedFilters: Array[Filter], + options: CaseInsensitiveStringMap) + extends FileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema) { + override def isSplitable(path: Path): Boolean = true + + override def createReaderFactory(): PartitionReaderFactory = { + val readDataSchemaAsJson = readDataSchema.json + hadoopConf.set(ParquetInputFormat.READ_SUPPORT_CLASS, classOf[ParquetReadSupport].getName) + hadoopConf.set( + ParquetReadSupport.SPARK_ROW_REQUESTED_SCHEMA, + readDataSchemaAsJson) + hadoopConf.set( + ParquetWriteSupport.SPARK_ROW_SCHEMA, + readDataSchemaAsJson) + hadoopConf.set( + SQLConf.SESSION_LOCAL_TIMEZONE.key, + sparkSession.sessionState.conf.sessionLocalTimeZone) + hadoopConf.setBoolean( + SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key, + sparkSession.sessionState.conf.nestedSchemaPruningEnabled) + hadoopConf.setBoolean( + SQLConf.CASE_SENSITIVE.key, + sparkSession.sessionState.conf.caseSensitiveAnalysis) + + ParquetWriteSupport.setSchema(readDataSchema, hadoopConf) + + // Sets flags for `ParquetToSparkSchemaConverter` + hadoopConf.setBoolean( + SQLConf.PARQUET_BINARY_AS_STRING.key, + sparkSession.sessionState.conf.isParquetBinaryAsString) + hadoopConf.setBoolean( + SQLConf.PARQUET_INT96_AS_TIMESTAMP.key, + sparkSession.sessionState.conf.isParquetINT96AsTimestamp) + + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + ParquetPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, + dataSchema, readDataSchema, readPartitionSchema, pushedFilters) + } + + override def equals(obj: Any): Boolean = obj match { + case p: ParquetScan => + fileIndex == p.fileIndex && dataSchema == p.dataSchema && + readDataSchema == p.readDataSchema && readPartitionSchema == p.readPartitionSchema && + options == p.options && equivalentFilters(pushedFilters, p.pushedFilters) + case _ => false + } + + override def hashCode(): Int = getClass.hashCode() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.scala new file mode 100644 index 0000000000000..4b8b434af88e6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetScanBuilder.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.v2.parquet + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFilters, SparkToParquetSchemaConverter} +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.sources.v2.reader.{Scan, SupportsPushDownFilters} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class ParquetScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) with SupportsPushDownFilters { + lazy val hadoopConf = { + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + } + + lazy val pushedParquetFilters = { + val sqlConf = sparkSession.sessionState.conf + val pushDownDate = sqlConf.parquetFilterPushDownDate + val pushDownTimestamp = sqlConf.parquetFilterPushDownTimestamp + val pushDownDecimal = sqlConf.parquetFilterPushDownDecimal + val pushDownStringStartWith = sqlConf.parquetFilterPushDownStringStartWith + val pushDownInFilterThreshold = sqlConf.parquetFilterPushDownInFilterThreshold + val isCaseSensitive = sqlConf.caseSensitiveAnalysis + val parquetSchema = + new SparkToParquetSchemaConverter(sparkSession.sessionState.conf).convert(schema) + val parquetFilters = new ParquetFilters(parquetSchema, pushDownDate, pushDownTimestamp, + pushDownDecimal, pushDownStringStartWith, pushDownInFilterThreshold, isCaseSensitive) + parquetFilters.convertibleFilters(this.filters).toArray + } + + private var filters: Array[Filter] = Array.empty + + override def pushFilters(filters: Array[Filter]): Array[Filter] = { + this.filters = filters + this.filters + } + + // Note: for Parquet, the actual filter push down happens in [[ParquetPartitionReaderFactory]]. + // It requires the Parquet physical schema to determine whether a filter is convertible. + // All filters that can be converted to Parquet are pushed down. + override def pushedFilters(): Array[Filter] = pushedParquetFilters + + override def build(): Scan = { + ParquetScan(sparkSession, hadoopConf, fileIndex, dataSchema, readDataSchema(), + readPartitionSchema(), pushedParquetFilters, options) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala new file mode 100644 index 0000000000000..dce851dbcd336 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetTable.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.parquet + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.parquet.ParquetUtils +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class ParquetTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + + override def newScanBuilder(options: CaseInsensitiveStringMap): ParquetScanBuilder = + new ParquetScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + ParquetUtils.inferSchema(sparkSession, options.asScala.toMap, files) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new ParquetWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = dataType match { + case _: AtomicType => true + + case st: StructType => st.forall { f => supportsDataType(f.dataType) } + + case ArrayType(elementType, _) => supportsDataType(elementType) + + case MapType(keyType, valueType, _) => + supportsDataType(keyType) && supportsDataType(valueType) + + case udt: UserDefinedType[_] => supportsDataType(udt.sqlType) + + case _ => false + } + + override def formatName: String = "Parquet" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetWriteBuilder.scala new file mode 100644 index 0000000000000..bfe2084299df3 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/parquet/ParquetWriteBuilder.scala @@ -0,0 +1,116 @@ +/* + * 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.v2.parquet + +import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.mapreduce.{Job, OutputCommitter, TaskAttemptContext} +import org.apache.parquet.hadoop.{ParquetOutputCommitter, ParquetOutputFormat} +import org.apache.parquet.hadoop.ParquetOutputFormat.JobSummaryLevel +import org.apache.parquet.hadoop.codec.CodecConfig +import org.apache.parquet.hadoop.util.ContextUtil + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.Row +import org.apache.spark.sql.execution.datasources.{OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.parquet._ +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class ParquetWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) with Logging { + + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + val parquetOptions = new ParquetOptions(options, sqlConf) + + val conf = ContextUtil.getConfiguration(job) + + val committerClass = + conf.getClass( + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key, + classOf[ParquetOutputCommitter], + classOf[OutputCommitter]) + + if (conf.get(SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key) == null) { + logInfo("Using default output committer for Parquet: " + + classOf[ParquetOutputCommitter].getCanonicalName) + } else { + logInfo("Using user defined output committer for Parquet: " + committerClass.getCanonicalName) + } + + conf.setClass( + SQLConf.OUTPUT_COMMITTER_CLASS.key, + committerClass, + classOf[OutputCommitter]) + + // We're not really using `ParquetOutputFormat[Row]` for writing data here, because we override + // it in `ParquetOutputWriter` to support appending and dynamic partitioning. The reason why + // we set it here is to setup the output committer class to `ParquetOutputCommitter`, which is + // bundled with `ParquetOutputFormat[Row]`. + job.setOutputFormatClass(classOf[ParquetOutputFormat[Row]]) + + ParquetOutputFormat.setWriteSupportClass(job, classOf[ParquetWriteSupport]) + + // This metadata is useful for keeping UDTs like Vector/Matrix. + ParquetWriteSupport.setSchema(dataSchema, conf) + + // Sets flags for `ParquetWriteSupport`, which converts Catalyst schema to Parquet + // schema and writes actual rows to Parquet files. + conf.set(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key, sqlConf.writeLegacyParquetFormat.toString) + + conf.set(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key, sqlConf.parquetOutputTimestampType.toString) + + // Sets compression scheme + conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName) + + // SPARK-15719: Disables writing Parquet summary files by default. + if (conf.get(ParquetOutputFormat.JOB_SUMMARY_LEVEL) == null + && conf.get(ParquetOutputFormat.ENABLE_JOB_SUMMARY) == null) { + conf.setEnum(ParquetOutputFormat.JOB_SUMMARY_LEVEL, JobSummaryLevel.NONE) + } + + if (ParquetOutputFormat.getJobSummaryLevel(conf) == JobSummaryLevel.NONE + && !classOf[ParquetOutputCommitter].isAssignableFrom(committerClass)) { + // output summary is requested, but the class is not a Parquet Committer + logWarning(s"Committer $committerClass is not a ParquetOutputCommitter and cannot" + + s" create job summaries. " + + s"Set Parquet option ${ParquetOutputFormat.JOB_SUMMARY_LEVEL} to NONE.") + } + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new ParquetOutputWriter(path, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + CodecConfig.from(context).getCodec.getExtension + ".parquet" + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala new file mode 100644 index 0000000000000..f6aa1e9c898b9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextDataSourceV2.scala @@ -0,0 +1,44 @@ +/* + * 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.v2.text + +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.text.TextFileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class TextDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[TextFileFormat] + + override def shortName(): String = "text" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + TextTable(tableName, sparkSession, options, paths, None, fallbackFileFormat) + } + + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val paths = getPaths(options) + val tableName = getTableName(paths) + TextTable(tableName, sparkSession, options, paths, Some(schema), fallbackFileFormat) + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala new file mode 100644 index 0000000000000..8788887111880 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextPartitionReaderFactory.scala @@ -0,0 +1,73 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.text + +import org.apache.spark.TaskContext +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.expressions.codegen.UnsafeRowWriter +import org.apache.spark.sql.execution.datasources.{HadoopFileLinesReader, HadoopFileWholeTextReader, PartitionedFile} +import org.apache.spark.sql.execution.datasources.text.TextOptions +import org.apache.spark.sql.execution.datasources.v2._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.PartitionReader +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.SerializableConfiguration + +/** + * A factory used to create Text readers. + * + * @param sqlConf SQL configuration. + * @param broadcastedConf Broadcasted serializable Hadoop Configuration. + * @param readDataSchema Required schema in the batch scan. + * @param partitionSchema Schema of partitions. + * @param textOptions Options for reading a text file. + * */ +case class TextPartitionReaderFactory( + sqlConf: SQLConf, + broadcastedConf: Broadcast[SerializableConfiguration], + readDataSchema: StructType, + partitionSchema: StructType, + textOptions: TextOptions) extends FilePartitionReaderFactory { + + override def buildReader(file: PartitionedFile): PartitionReader[InternalRow] = { + val confValue = broadcastedConf.value.value + val reader = if (!textOptions.wholeText) { + new HadoopFileLinesReader(file, textOptions.lineSeparatorInRead, confValue) + } else { + new HadoopFileWholeTextReader(file, confValue) + } + Option(TaskContext.get()).foreach(_.addTaskCompletionListener[Unit](_ => reader.close())) + val iter = if (readDataSchema.isEmpty) { + val emptyUnsafeRow = new UnsafeRow(0) + reader.map(_ => emptyUnsafeRow) + } else { + val unsafeRowWriter = new UnsafeRowWriter(1) + + reader.map { line => + // Writes to an UnsafeRow directly + unsafeRowWriter.reset() + unsafeRowWriter.write(0, line.getBytes, 0, line.getLength) + unsafeRowWriter.getRow() + } + } + val fileReader = new PartitionReaderFromIterator[InternalRow](iter) + new PartitionReaderWithPartitionValues(fileReader, readDataSchema, + partitionSchema, file.partitionValues) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala new file mode 100644 index 0000000000000..202723db27421 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScan.scala @@ -0,0 +1,61 @@ +/* + * 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.v2.text + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.Path + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.text.TextOptions +import org.apache.spark.sql.execution.datasources.v2.TextBasedFileScan +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap +import org.apache.spark.util.SerializableConfiguration + +case class TextScan( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + readDataSchema: StructType, + readPartitionSchema: StructType, + options: CaseInsensitiveStringMap) + extends TextBasedFileScan(sparkSession, fileIndex, readDataSchema, readPartitionSchema, options) { + + private val optionsAsScala = options.asScala.toMap + private lazy val textOptions: TextOptions = new TextOptions(optionsAsScala) + + override def isSplitable(path: Path): Boolean = { + super.isSplitable(path) && !textOptions.wholeText + } + + override def createReaderFactory(): PartitionReaderFactory = { + assert( + readDataSchema.length <= 1, + "Text data source only produces a single data column named \"value\".") + val hadoopConf = { + val caseSensitiveMap = options.asCaseSensitiveMap.asScala.toMap + // Hadoop Configurations are case sensitive. + sparkSession.sessionState.newHadoopConfWithOptions(caseSensitiveMap) + } + val broadcastedConf = sparkSession.sparkContext.broadcast( + new SerializableConfiguration(hadoopConf)) + TextPartitionReaderFactory(sparkSession.sessionState.conf, broadcastedConf, readDataSchema, + readPartitionSchema, textOptions) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.scala new file mode 100644 index 0000000000000..fbe5e1688b836 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextScanBuilder.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.sql.execution.datasources.v2.text + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.PartitioningAwareFileIndex +import org.apache.spark.sql.execution.datasources.v2.FileScanBuilder +import org.apache.spark.sql.sources.v2.reader.Scan +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class TextScanBuilder( + sparkSession: SparkSession, + fileIndex: PartitioningAwareFileIndex, + schema: StructType, + dataSchema: StructType, + options: CaseInsensitiveStringMap) + extends FileScanBuilder(sparkSession, fileIndex, dataSchema) { + + override def build(): Scan = { + TextScan(sparkSession, fileIndex, readDataSchema(), readPartitionSchema(), options) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.scala new file mode 100644 index 0000000000000..b8cb61a6c646e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextTable.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.execution.datasources.v2.text + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.v2.FileTable +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.types.{DataType, StringType, StructField, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +case class TextTable( + name: String, + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + userSpecifiedSchema: Option[StructType], + fallbackFileFormat: Class[_ <: FileFormat]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + override def newScanBuilder(options: CaseInsensitiveStringMap): TextScanBuilder = + TextScanBuilder(sparkSession, fileIndex, schema, dataSchema, options) + + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = + Some(StructType(Seq(StructField("value", StringType)))) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + new TextWriteBuilder(options, paths, formatName, supportsDataType) + + override def supportsDataType(dataType: DataType): Boolean = dataType == StringType + + override def formatName: String = "Text" +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextWriteBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextWriteBuilder.scala new file mode 100644 index 0000000000000..c00dbc20be64a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/text/TextWriteBuilder.scala @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.v2.text + +import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.util.CompressionCodecs +import org.apache.spark.sql.execution.datasources.{CodecStreams, OutputWriter, OutputWriterFactory} +import org.apache.spark.sql.execution.datasources.text.{TextOptions, TextOutputWriter} +import org.apache.spark.sql.execution.datasources.v2.FileWriteBuilder +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class TextWriteBuilder( + options: CaseInsensitiveStringMap, + paths: Seq[String], + formatName: String, + supportsDataType: DataType => Boolean) + extends FileWriteBuilder(options, paths, formatName, supportsDataType) { + private def verifySchema(schema: StructType): Unit = { + if (schema.size != 1) { + throw new AnalysisException( + s"Text data source supports only a single column, and you have ${schema.size} columns.") + } + } + + override def prepareWrite( + sqlConf: SQLConf, + job: Job, + options: Map[String, String], + dataSchema: StructType): OutputWriterFactory = { + verifySchema(dataSchema) + + val textOptions = new TextOptions(options) + val conf = job.getConfiguration + + textOptions.compressionCodec.foreach { codec => + CompressionCodecs.setCodecConfiguration(conf, codec) + } + + new OutputWriterFactory { + override def newInstance( + path: String, + dataSchema: StructType, + context: TaskAttemptContext): OutputWriter = { + new TextOutputWriter(path, dataSchema, textOptions.lineSeparatorInWrite, context) + } + + override def getFileExtension(context: TaskAttemptContext): String = { + ".txt" + CodecStreams.getCompressionExtension(context) + } + } + } +} 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 366e1fe6a4aaa..53b74c7c85594 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 @@ -29,8 +29,9 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.catalyst.trees.TreeNodeRef +import org.apache.spark.sql.catalyst.util.StringUtils.StringConcat import org.apache.spark.sql.execution.streaming.{StreamExecution, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.continuous.WriteToContinuousDataSourceExec +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.StreamingQuery import org.apache.spark.util.{AccumulatorV2, LongAccumulator} @@ -70,15 +71,20 @@ package object debug { * @return single String containing all WholeStageCodegen subtrees and corresponding codegen */ def codegenString(plan: SparkPlan): String = { + val concat = new StringConcat() + writeCodegen(concat.append, plan) + concat.toString + } + + def writeCodegen(append: String => Unit, plan: SparkPlan): Unit = { val codegenSeq = codegenStringSeq(plan) - var output = s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n" + append(s"Found ${codegenSeq.size} WholeStageCodegen subtrees.\n") for (((subtree, code), i) <- codegenSeq.zipWithIndex) { - output += s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n" - output += subtree - output += "\nGenerated code:\n" - output += s"${code}\n" + append(s"== Subtree ${i + 1} / ${codegenSeq.size} ==\n") + append(subtree) + append("\nGenerated code:\n") + append(s"${code}\n") } - output } /** @@ -204,7 +210,7 @@ package object debug { val columnStats: Array[ColumnMetrics] = Array.fill(child.output.size)(new ColumnMetrics()) def dumpStats(): Unit = { - debugPrint(s"== ${child.simpleString} ==") + debugPrint(s"== ${child.simpleString(SQLConf.get.maxToStringFields)} ==") debugPrint(s"Tuples output: ${tupleCount.value}") child.output.zip(columnStats).foreach { case (attr, metric) => // This is called on driver. All accumulator updates have a fixed value. So it's safe to use 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 a80673c705f1a..36f0d173cd0b0 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 @@ -17,10 +17,11 @@ package org.apache.spark.sql.execution.exchange -import java.util.concurrent.TimeoutException +import java.util.UUID +import java.util.concurrent._ -import scala.concurrent.{ExecutionContext, Future} -import scala.concurrent.duration._ +import scala.concurrent.{ExecutionContext, Promise} +import scala.concurrent.duration.NANOSECONDS import scala.util.control.NonFatal import org.apache.spark.{broadcast, SparkException} @@ -32,7 +33,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastMode, BroadcastPar 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 +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.util.{SparkFatalException, ThreadUtils} /** @@ -43,11 +44,13 @@ case class BroadcastExchangeExec( mode: BroadcastMode, child: SparkPlan) extends Exchange { + private[sql] val runId: UUID = UUID.randomUUID + override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createMetric(sparkContext, "data size (bytes)"), - "collectTime" -> SQLMetrics.createMetric(sparkContext, "time to collect (ms)"), - "buildTime" -> SQLMetrics.createMetric(sparkContext, "time to build (ms)"), - "broadcastTime" -> SQLMetrics.createMetric(sparkContext, "time to broadcast (ms)")) + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size"), + "collectTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to collect"), + "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build"), + "broadcastTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to broadcast")) override def outputPartitioning: Partitioning = BroadcastPartitioning(mode) @@ -56,79 +59,97 @@ case class BroadcastExchangeExec( } @transient - private val timeout: Duration = { - val timeoutValue = sqlContext.conf.broadcastTimeout - if (timeoutValue < 0) { - Duration.Inf - } else { - timeoutValue.seconds - } - } + private lazy val promise = Promise[broadcast.Broadcast[Any]]() + /** + * For registering callbacks on `relationFuture`. + * Note that calling this field will not start the execution of broadcast job. + */ @transient - private lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { - // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here. - val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) - Future { - // This will run in another thread. Set the execution id so that we can connect these jobs - // with the correct execution. - SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { - try { - val beforeCollect = System.nanoTime() - // Use executeCollect/executeCollectIterator to avoid conversion to Scala types - val (numRows, input) = child.executeCollectIterator() - if (numRows >= 512000000) { - throw new SparkException( - s"Cannot broadcast the table with more than 512 millions rows: $numRows rows") - } - - val beforeBuild = System.nanoTime() - longMetric("collectTime") += (beforeBuild - beforeCollect) / 1000000 - - // Construct the relation. - val relation = mode.transform(input, Some(numRows)) + lazy val completionFuture: scala.concurrent.Future[broadcast.Broadcast[Any]] = promise.future - val dataSize = relation match { - case map: HashedRelation => - map.estimatedSize - case arr: Array[InternalRow] => - arr.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum - case _ => - throw new SparkException("[BUG] BroadcastMode.transform returned unexpected type: " + - relation.getClass.getName) - } + @transient + private val timeout: Long = SQLConf.get.broadcastTimeout - longMetric("dataSize") += dataSize - if (dataSize >= (8L << 30)) { - throw new SparkException( - s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") + @transient + private[sql] lazy val relationFuture: Future[broadcast.Broadcast[Any]] = { + // relationFuture is used in "doExecute". Therefore we can get the execution id correctly here. + val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY) + val task = new Callable[broadcast.Broadcast[Any]]() { + override def call(): broadcast.Broadcast[Any] = { + // This will run in another thread. Set the execution id so that we can connect these jobs + // with the correct execution. + SQLExecution.withExecutionId(sqlContext.sparkSession, executionId) { + try { + // Setup a job group here so later it may get cancelled by groupId if necessary. + sparkContext.setJobGroup(runId.toString, s"broadcast exchange (runId $runId)", + interruptOnCancel = true) + val beforeCollect = System.nanoTime() + // Use executeCollect/executeCollectIterator to avoid conversion to Scala types + val (numRows, input) = child.executeCollectIterator() + if (numRows >= 512000000) { + throw new SparkException( + s"Cannot broadcast the table with 512 million or more rows: $numRows rows") + } + + val beforeBuild = System.nanoTime() + longMetric("collectTime") += NANOSECONDS.toMillis(beforeBuild - beforeCollect) + + // Construct the relation. + val relation = mode.transform(input, Some(numRows)) + + val dataSize = relation match { + case map: HashedRelation => + map.estimatedSize + case arr: Array[InternalRow] => + arr.map(_.asInstanceOf[UnsafeRow].getSizeInBytes.toLong).sum + case _ => + throw new SparkException("[BUG] BroadcastMode.transform returned unexpected " + + s"type: ${relation.getClass.getName}") + } + + longMetric("dataSize") += dataSize + if (dataSize >= (8L << 30)) { + throw new SparkException( + s"Cannot broadcast the table that is larger than 8GB: ${dataSize >> 30} GB") + } + + val beforeBroadcast = System.nanoTime() + longMetric("buildTime") += NANOSECONDS.toMillis(beforeBroadcast - beforeBuild) + + // Broadcast the relation + val broadcasted = sparkContext.broadcast(relation) + longMetric("broadcastTime") += NANOSECONDS.toMillis( + System.nanoTime() - beforeBroadcast) + + SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) + promise.success(broadcasted) + broadcasted + } catch { + // SPARK-24294: To bypass scala bug: https://github.com/scala/bug/issues/9554, we throw + // SparkFatalException, which is a subclass of Exception. ThreadUtils.awaitResult + // will catch this exception and re-throw the wrapped fatal throwable. + case oe: OutOfMemoryError => + val ex = new SparkFatalException( + new OutOfMemoryError("Not enough memory to build and broadcast the table to all " + + "worker nodes. As a workaround, you can either disable broadcast by setting " + + s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark " + + s"driver memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value.") + .initCause(oe.getCause)) + promise.failure(ex) + throw ex + case e if !NonFatal(e) => + val ex = new SparkFatalException(e) + promise.failure(ex) + throw ex + case e: Throwable => + promise.failure(e) + throw e } - - val beforeBroadcast = System.nanoTime() - longMetric("buildTime") += (beforeBroadcast - beforeBuild) / 1000000 - - // Broadcast the relation - val broadcasted = sparkContext.broadcast(relation) - longMetric("broadcastTime") += (System.nanoTime() - beforeBroadcast) / 1000000 - - SQLMetrics.postDriverMetricUpdates(sparkContext, executionId, metrics.values.toSeq) - broadcasted - } catch { - // SPARK-24294: To bypass scala bug: https://github.com/scala/bug/issues/9554, we throw - // SparkFatalException, which is a subclass of Exception. ThreadUtils.awaitResult - // will catch this exception and re-throw the wrapped fatal throwable. - case oe: OutOfMemoryError => - throw new SparkFatalException( - new OutOfMemoryError(s"Not enough memory to build and broadcast the table to " + - s"all worker nodes. As a workaround, you can either disable broadcast by setting " + - s"${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1 or increase the spark driver " + - s"memory by setting ${SparkLauncher.DRIVER_MEMORY} to a higher value") - .initCause(oe.getCause)) - case e if !NonFatal(e) => - throw new SparkFatalException(e) } } - }(BroadcastExchangeExec.executionContext) + } + BroadcastExchangeExec.executionContext.submit[broadcast.Broadcast[Any]](task) } override protected def doPrepare(): Unit = { @@ -143,11 +164,15 @@ case class BroadcastExchangeExec( override protected[sql] def doExecuteBroadcast[T](): broadcast.Broadcast[T] = { try { - ThreadUtils.awaitResult(relationFuture, timeout).asInstanceOf[broadcast.Broadcast[T]] + relationFuture.get(timeout, TimeUnit.SECONDS).asInstanceOf[broadcast.Broadcast[T]] } catch { case ex: TimeoutException => - logError(s"Could not execute broadcast in ${timeout.toSeconds} secs.", ex) - throw new SparkException(s"Could not execute broadcast in ${timeout.toSeconds} secs. " + + logError(s"Could not execute broadcast in $timeout secs.", ex) + if (!relationFuture.isDone) { + sparkContext.cancelJobGroup(runId.toString) + relationFuture.cancel(true) + } + throw new SparkException(s"Could not execute broadcast in $timeout secs. " + s"You can increase the timeout for broadcasts via ${SQLConf.BROADCAST_TIMEOUT.key} or " + s"disable broadcast join by setting ${SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key} to -1", ex) @@ -157,5 +182,6 @@ case class BroadcastExchangeExec( object BroadcastExchangeExec { private[execution] val executionContext = ExecutionContext.fromExecutorService( - ThreadUtils.newDaemonCachedThreadPool("broadcast-exchange", 128)) + 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/exchange/EnsureRequirements.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala index d2d5011bbcb97..c56a5c015f32d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/EnsureRequirements.scala @@ -24,8 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, ShuffledHashJoinExec, - SortMergeJoinExec} +import org.apache.spark.sql.execution.joins.{ShuffledHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.internal.SQLConf /** @@ -36,107 +35,12 @@ import org.apache.spark.sql.internal.SQLConf * the input partition ordering requirements are met. */ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { - private def defaultNumPreShufflePartitions: Int = conf.numShufflePartitions - - private def targetPostShuffleInputSize: Long = conf.targetPostShuffleInputSize - - private def adaptiveExecutionEnabled: Boolean = conf.adaptiveExecutionEnabled - - private def minNumPostShufflePartitions: Option[Int] = { - val minNumPostShufflePartitions = conf.minNumPostShufflePartitions - if (minNumPostShufflePartitions > 0) Some(minNumPostShufflePartitions) else None - } - - /** - * Adds [[ExchangeCoordinator]] to [[ShuffleExchangeExec]]s if adaptive query execution is enabled - * and partitioning schemes of these [[ShuffleExchangeExec]]s support [[ExchangeCoordinator]]. - */ - private def withExchangeCoordinator( - children: Seq[SparkPlan], - requiredChildDistributions: Seq[Distribution]): Seq[SparkPlan] = { - val supportsCoordinator = - if (children.exists(_.isInstanceOf[ShuffleExchangeExec])) { - // Right now, ExchangeCoordinator only support HashPartitionings. - children.forall { - case e @ ShuffleExchangeExec(hash: HashPartitioning, _, _) => true - case child => - child.outputPartitioning match { - case hash: HashPartitioning => true - case collection: PartitioningCollection => - collection.partitionings.forall(_.isInstanceOf[HashPartitioning]) - case _ => false - } - } - } else { - // In this case, although we do not have Exchange operators, we may still need to - // shuffle data when we have more than one children because data generated by - // these children may not be partitioned in the same way. - // Please see the comment in withCoordinator for more details. - val supportsDistribution = requiredChildDistributions.forall { dist => - dist.isInstanceOf[ClusteredDistribution] || dist.isInstanceOf[HashClusteredDistribution] - } - children.length > 1 && supportsDistribution - } - - val withCoordinator = - if (adaptiveExecutionEnabled && supportsCoordinator) { - val coordinator = - new ExchangeCoordinator( - targetPostShuffleInputSize, - minNumPostShufflePartitions) - children.zip(requiredChildDistributions).map { - case (e: ShuffleExchangeExec, _) => - // This child is an Exchange, we need to add the coordinator. - e.copy(coordinator = Some(coordinator)) - case (child, distribution) => - // If this child is not an Exchange, we need to add an Exchange for now. - // Ideally, we can try to avoid this Exchange. However, when we reach here, - // there are at least two children operators (because if there is a single child - // and we can avoid Exchange, supportsCoordinator will be false and we - // will not reach here.). Although we can make two children have the same number of - // post-shuffle partitions. Their numbers of pre-shuffle partitions may be different. - // For example, let's say we have the following plan - // Join - // / \ - // Agg Exchange - // / \ - // Exchange t2 - // / - // t1 - // In this case, because a post-shuffle partition can include multiple pre-shuffle - // partitions, a HashPartitioning will not be strictly partitioned by the hashcodes - // after shuffle. So, even we can use the child Exchange operator of the Join to - // have a number of post-shuffle partitions that matches the number of partitions of - // Agg, we cannot say these two children are partitioned in the same way. - // Here is another case - // Join - // / \ - // Agg1 Agg2 - // / \ - // Exchange1 Exchange2 - // / \ - // t1 t2 - // In this case, two Aggs shuffle data with the same column of the join condition. - // After we use ExchangeCoordinator, these two Aggs may not be partitioned in the same - // way. Let's say that Agg1 and Agg2 both have 5 pre-shuffle partitions and 2 - // post-shuffle partitions. It is possible that Agg1 fetches those pre-shuffle - // partitions by using a partitionStartIndices [0, 3]. However, Agg2 may fetch its - // pre-shuffle partitions by using another partitionStartIndices [0, 4]. - // So, Agg1 and Agg2 are actually not co-partitioned. - // - // It will be great to introduce a new Partitioning to represent the post-shuffle - // partitions when one post-shuffle partition includes multiple pre-shuffle partitions. - val targetPartitioning = distribution.createPartitioning(defaultNumPreShufflePartitions) - assert(targetPartitioning.isInstanceOf[HashPartitioning]) - ShuffleExchangeExec(targetPartitioning, child, Some(coordinator)) - } - } else { - // If we do not need ExchangeCoordinator, the original children are returned. - children - } - - withCoordinator - } + private def defaultNumPreShufflePartitions: Int = + if (conf.adaptiveExecutionEnabled) { + conf.maxNumPostShufflePartitions + } else { + conf.numShufflePartitions + } private def ensureDistributionAndOrdering(operator: SparkPlan): SparkPlan = { val requiredChildDistributions: Seq[Distribution] = operator.requiredChildDistribution @@ -198,15 +102,6 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { } } - // Now, we need to add ExchangeCoordinator if necessary. - // Actually, it is not a good idea to add ExchangeCoordinators while we are adding Exchanges. - // However, with the way that we plan the query, we do not have a place where we have a - // global picture of all shuffle dependencies of a post-shuffle stage. So, we add coordinator - // at here for now. - // Once we finish https://issues.apache.org/jira/browse/SPARK-10665, - // we can first add Exchanges and then add coordinator once we have a DAG of query fragments. - children = withExchangeCoordinator(children, requiredChildDistributions) - // Now that we've performed any necessary shuffles, add sorts to guarantee output orderings: children = children.zip(requiredChildOrderings).map { case (child, requiredOrdering) => // If child.outputOrdering already satisfies the requiredOrdering, we do not need to sort. @@ -221,25 +116,41 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { } private def reorder( - leftKeys: Seq[Expression], - rightKeys: Seq[Expression], + leftKeys: IndexedSeq[Expression], + rightKeys: IndexedSeq[Expression], expectedOrderOfKeys: Seq[Expression], currentOrderOfKeys: Seq[Expression]): (Seq[Expression], Seq[Expression]) = { - val leftKeysBuffer = ArrayBuffer[Expression]() - val rightKeysBuffer = ArrayBuffer[Expression]() - val pickedIndexes = mutable.Set[Int]() - val keysAndIndexes = currentOrderOfKeys.zipWithIndex + if (expectedOrderOfKeys.size != currentOrderOfKeys.size) { + return (leftKeys, rightKeys) + } - expectedOrderOfKeys.foreach(expression => { - val index = keysAndIndexes.find { case (e, idx) => - // As we may have the same key used many times, we need to filter out its occurrence we - // have already used. - e.semanticEquals(expression) && !pickedIndexes.contains(idx) - }.map(_._2).get - pickedIndexes += index - leftKeysBuffer.append(leftKeys(index)) - rightKeysBuffer.append(rightKeys(index)) - }) + // Build a lookup between an expression and the positions its holds in the current key seq. + val keyToIndexMap = mutable.Map.empty[Expression, mutable.BitSet] + currentOrderOfKeys.zipWithIndex.foreach { + case (key, index) => + keyToIndexMap.getOrElseUpdate(key.canonicalized, mutable.BitSet.empty).add(index) + } + + // Reorder the keys. + val leftKeysBuffer = new ArrayBuffer[Expression](leftKeys.size) + val rightKeysBuffer = new ArrayBuffer[Expression](rightKeys.size) + val iterator = expectedOrderOfKeys.iterator + while (iterator.hasNext) { + // Lookup the current index of this key. + keyToIndexMap.get(iterator.next().canonicalized) match { + case Some(indices) if indices.nonEmpty => + // Take the first available index from the map. + val index = indices.firstKey + indices.remove(index) + + // Add the keys for that index to the reordered keys. + leftKeysBuffer += leftKeys(index) + rightKeysBuffer += rightKeys(index) + case _ => + // The expression cannot be found, or we have exhausted all indices for that expression. + return (leftKeys, rightKeys) + } + } (leftKeysBuffer, rightKeysBuffer) } @@ -249,20 +160,13 @@ case class EnsureRequirements(conf: SQLConf) extends Rule[SparkPlan] { leftPartitioning: Partitioning, rightPartitioning: Partitioning): (Seq[Expression], Seq[Expression]) = { if (leftKeys.forall(_.deterministic) && rightKeys.forall(_.deterministic)) { - leftPartitioning match { - case HashPartitioning(leftExpressions, _) - if leftExpressions.length == leftKeys.length && - leftKeys.forall(x => leftExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, leftExpressions, leftKeys) - - case _ => rightPartitioning match { - case HashPartitioning(rightExpressions, _) - if rightExpressions.length == rightKeys.length && - rightKeys.forall(x => rightExpressions.exists(_.semanticEquals(x))) => - reorder(leftKeys, rightKeys, rightExpressions, rightKeys) - - case _ => (leftKeys, rightKeys) - } + (leftPartitioning, rightPartitioning) match { + case (HashPartitioning(leftExpressions, _), _) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, leftExpressions, leftKeys) + case (_, HashPartitioning(rightExpressions, _)) => + reorder(leftKeys.toIndexedSeq, rightKeys.toIndexedSeq, rightExpressions, rightKeys) + case _ => + (leftKeys, rightKeys) } } else { (leftKeys, rightKeys) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala deleted file mode 100644 index f5d93ee5fa914..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ExchangeCoordinator.scala +++ /dev/null @@ -1,277 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.exchange - -import java.util.{HashMap => JHashMap, Map => JMap} -import javax.annotation.concurrent.GuardedBy - -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{MapOutputStatistics, ShuffleDependency, SimpleFutureAction} -import org.apache.spark.internal.Logging -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.{ShuffledRowRDD, SparkPlan} - -/** - * A coordinator used to determines how we shuffle data between stages generated by Spark SQL. - * Right now, the work of this coordinator is to determine the number of post-shuffle partitions - * for a stage that needs to fetch shuffle data from one or multiple stages. - * - * A coordinator is constructed with three parameters, `numExchanges`, - * `targetPostShuffleInputSize`, and `minNumPostShufflePartitions`. - * - `numExchanges` is used to indicated that how many [[ShuffleExchangeExec]]s that will be - * registered to this coordinator. So, when we start to do any actual work, we have a way to - * make sure that we have got expected number of [[ShuffleExchangeExec]]s. - * - `targetPostShuffleInputSize` is the targeted size of a post-shuffle partition's - * input data size. With this parameter, we can estimate the number of post-shuffle partitions. - * This parameter is configured through - * `spark.sql.adaptive.shuffle.targetPostShuffleInputSize`. - * - `minNumPostShufflePartitions` is an optional parameter. If it is defined, this coordinator - * will try to make sure that there are at least `minNumPostShufflePartitions` post-shuffle - * partitions. - * - * The workflow of this coordinator is described as follows: - * - Before the execution of a [[SparkPlan]], for a [[ShuffleExchangeExec]] operator, - * if an [[ExchangeCoordinator]] is assigned to it, it registers itself to this coordinator. - * This happens in the `doPrepare` method. - * - Once we start to execute a physical plan, a [[ShuffleExchangeExec]] registered to this - * coordinator will call `postShuffleRDD` to get its corresponding post-shuffle - * [[ShuffledRowRDD]]. - * If this coordinator has made the decision on how to shuffle data, this [[ShuffleExchangeExec]] - * will immediately get its corresponding post-shuffle [[ShuffledRowRDD]]. - * - If this coordinator has not made the decision on how to shuffle data, it will ask those - * registered [[ShuffleExchangeExec]]s to submit their pre-shuffle stages. Then, based on the - * size statistics of pre-shuffle partitions, this coordinator will determine the number of - * post-shuffle partitions and pack multiple pre-shuffle partitions with continuous indices - * to a single post-shuffle partition whenever necessary. - * - Finally, this coordinator will create post-shuffle [[ShuffledRowRDD]]s for all registered - * [[ShuffleExchangeExec]]s. So, when a [[ShuffleExchangeExec]] calls `postShuffleRDD`, this - * coordinator can lookup the corresponding [[RDD]]. - * - * The strategy used to determine the number of post-shuffle partitions is described as follows. - * To determine the number of post-shuffle partitions, we have a target input size for a - * post-shuffle partition. Once we have size statistics of pre-shuffle partitions from stages - * corresponding to the registered [[ShuffleExchangeExec]]s, we will do a pass of those statistics - * and pack pre-shuffle partitions with continuous indices to a single post-shuffle partition until - * adding another pre-shuffle partition would cause the size of a post-shuffle partition to be - * greater than the target size. - * - * For example, we have two stages with the following pre-shuffle partition size statistics: - * stage 1: [100 MB, 20 MB, 100 MB, 10MB, 30 MB] - * stage 2: [10 MB, 10 MB, 70 MB, 5 MB, 5 MB] - * assuming the target input size is 128 MB, we will have four post-shuffle partitions, - * which are: - * - post-shuffle partition 0: pre-shuffle partition 0 (size 110 MB) - * - post-shuffle partition 1: pre-shuffle partition 1 (size 30 MB) - * - post-shuffle partition 2: pre-shuffle partition 2 (size 170 MB) - * - post-shuffle partition 3: pre-shuffle partition 3 and 4 (size 50 MB) - */ -class ExchangeCoordinator( - advisoryTargetPostShuffleInputSize: Long, - minNumPostShufflePartitions: Option[Int] = None) - extends Logging { - - // The registered Exchange operators. - private[this] val exchanges = ArrayBuffer[ShuffleExchangeExec]() - - // `lazy val` is used here so that we could notice the wrong use of this class, e.g., all the - // exchanges should be registered before `postShuffleRDD` called first time. If a new exchange is - // registered after the `postShuffleRDD` call, `assert(exchanges.length == numExchanges)` fails - // in `doEstimationIfNecessary`. - private[this] lazy val numExchanges = exchanges.size - - // This map is used to lookup the post-shuffle ShuffledRowRDD for an Exchange operator. - private[this] lazy val postShuffleRDDs: JMap[ShuffleExchangeExec, ShuffledRowRDD] = - new JHashMap[ShuffleExchangeExec, ShuffledRowRDD](numExchanges) - - // A boolean that indicates if this coordinator has made decision on how to shuffle data. - // This variable will only be updated by doEstimationIfNecessary, which is protected by - // synchronized. - @volatile private[this] var estimated: Boolean = false - - /** - * Registers a [[ShuffleExchangeExec]] operator to this coordinator. This method is only allowed - * to be called in the `doPrepare` method of a [[ShuffleExchangeExec]] operator. - */ - @GuardedBy("this") - def registerExchange(exchange: ShuffleExchangeExec): Unit = synchronized { - exchanges += exchange - } - - def isEstimated: Boolean = estimated - - /** - * Estimates partition start indices for post-shuffle partitions based on - * mapOutputStatistics provided by all pre-shuffle stages. - */ - def estimatePartitionStartIndices( - mapOutputStatistics: Array[MapOutputStatistics]): Array[Int] = { - // If minNumPostShufflePartitions is defined, it is possible that we need to use a - // value less than advisoryTargetPostShuffleInputSize as the target input size of - // a post shuffle task. - val targetPostShuffleInputSize = minNumPostShufflePartitions match { - case Some(numPartitions) => - val totalPostShuffleInputSize = mapOutputStatistics.map(_.bytesByPartitionId.sum).sum - // The max at here is to make sure that when we have an empty table, we - // only have a single post-shuffle partition. - // There is no particular reason that we pick 16. We just need a number to - // prevent maxPostShuffleInputSize from being set to 0. - val maxPostShuffleInputSize = - math.max(math.ceil(totalPostShuffleInputSize / numPartitions.toDouble).toLong, 16) - math.min(maxPostShuffleInputSize, advisoryTargetPostShuffleInputSize) - - case None => advisoryTargetPostShuffleInputSize - } - - logInfo( - s"advisoryTargetPostShuffleInputSize: $advisoryTargetPostShuffleInputSize, " + - s"targetPostShuffleInputSize $targetPostShuffleInputSize.") - - // Make sure we do get the same number of pre-shuffle partitions for those stages. - val distinctNumPreShufflePartitions = - mapOutputStatistics.map(stats => stats.bytesByPartitionId.length).distinct - // The reason that we are expecting a single value of the number of pre-shuffle partitions - // is that when we add Exchanges, we set the number of pre-shuffle partitions - // (i.e. map output partitions) using a static setting, which is the value of - // spark.sql.shuffle.partitions. Even if two input RDDs are having different - // number of partitions, they will have the same number of pre-shuffle partitions - // (i.e. map output partitions). - assert( - distinctNumPreShufflePartitions.length == 1, - "There should be only one distinct value of the number pre-shuffle partitions " + - "among registered Exchange operator.") - val numPreShufflePartitions = distinctNumPreShufflePartitions.head - - val partitionStartIndices = ArrayBuffer[Int]() - // The first element of partitionStartIndices is always 0. - partitionStartIndices += 0 - - var postShuffleInputSize = 0L - - var i = 0 - while (i < numPreShufflePartitions) { - // We calculate the total size of ith pre-shuffle partitions from all pre-shuffle stages. - // Then, we add the total size to postShuffleInputSize. - var nextShuffleInputSize = 0L - var j = 0 - while (j < mapOutputStatistics.length) { - nextShuffleInputSize += mapOutputStatistics(j).bytesByPartitionId(i) - j += 1 - } - - // If including the nextShuffleInputSize would exceed the target partition size, then start a - // new partition. - if (i > 0 && postShuffleInputSize + nextShuffleInputSize > targetPostShuffleInputSize) { - partitionStartIndices += i - // reset postShuffleInputSize. - postShuffleInputSize = nextShuffleInputSize - } else postShuffleInputSize += nextShuffleInputSize - - i += 1 - } - - partitionStartIndices.toArray - } - - @GuardedBy("this") - private def doEstimationIfNecessary(): Unit = synchronized { - // It is unlikely that this method will be called from multiple threads - // (when multiple threads trigger the execution of THIS physical) - // because in common use cases, we will create new physical plan after - // users apply operations (e.g. projection) to an existing DataFrame. - // However, if it happens, we have synchronized to make sure only one - // thread will trigger the job submission. - if (!estimated) { - // Make sure we have the expected number of registered Exchange operators. - assert(exchanges.length == numExchanges) - - val newPostShuffleRDDs = new JHashMap[ShuffleExchangeExec, ShuffledRowRDD](numExchanges) - - // Submit all map stages - val shuffleDependencies = ArrayBuffer[ShuffleDependency[Int, InternalRow, InternalRow]]() - val submittedStageFutures = ArrayBuffer[SimpleFutureAction[MapOutputStatistics]]() - var i = 0 - while (i < numExchanges) { - val exchange = exchanges(i) - val shuffleDependency = exchange.prepareShuffleDependency() - shuffleDependencies += shuffleDependency - if (shuffleDependency.rdd.partitions.length != 0) { - // submitMapStage does not accept RDD with 0 partition. - // So, we will not submit this dependency. - submittedStageFutures += - exchange.sqlContext.sparkContext.submitMapStage(shuffleDependency) - } - i += 1 - } - - // Wait for the finishes of those submitted map stages. - val mapOutputStatistics = new Array[MapOutputStatistics](submittedStageFutures.length) - var j = 0 - while (j < submittedStageFutures.length) { - // This call is a blocking call. If the stage has not finished, we will wait at here. - mapOutputStatistics(j) = submittedStageFutures(j).get() - j += 1 - } - - // If we have mapOutputStatistics.length < numExchange, it is because we do not submit - // a stage when the number of partitions of this dependency is 0. - assert(mapOutputStatistics.length <= numExchanges) - - // Now, we estimate partitionStartIndices. partitionStartIndices.length will be the - // number of post-shuffle partitions. - val partitionStartIndices = - if (mapOutputStatistics.length == 0) { - Array.empty[Int] - } else { - estimatePartitionStartIndices(mapOutputStatistics) - } - - var k = 0 - while (k < numExchanges) { - val exchange = exchanges(k) - val rdd = - exchange.preparePostShuffleRDD(shuffleDependencies(k), Some(partitionStartIndices)) - newPostShuffleRDDs.put(exchange, rdd) - - k += 1 - } - - // Finally, we set postShuffleRDDs and estimated. - assert(postShuffleRDDs.isEmpty) - assert(newPostShuffleRDDs.size() == numExchanges) - postShuffleRDDs.putAll(newPostShuffleRDDs) - estimated = true - } - } - - def postShuffleRDD(exchange: ShuffleExchangeExec): ShuffledRowRDD = { - doEstimationIfNecessary() - - if (!postShuffleRDDs.containsKey(exchange)) { - throw new IllegalStateException( - s"The given $exchange is not registered in this coordinator.") - } - - postShuffleRDDs.get(exchange) - } - - override def toString: String = { - s"coordinator[target post-shuffle partition size: $advisoryTargetPostShuffleInputSize]" - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala index aba94885f941c..fec05a76b4516 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/exchange/ShuffleExchangeExec.scala @@ -21,94 +21,65 @@ import java.util.Random import java.util.function.Supplier import org.apache.spark._ +import org.apache.spark.internal.config import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer +import org.apache.spark.shuffle.{ShuffleWriteMetricsReporter, ShuffleWriteProcessor} import org.apache.spark.shuffle.sort.SortShuffleManager import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.errors._ -import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{Attribute, BoundReference, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.expressions.codegen.LazilyGeneratedOrdering import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.execution._ -import org.apache.spark.sql.execution.metric.SQLMetrics +import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics, SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.StructType import org.apache.spark.util.MutablePair import org.apache.spark.util.collection.unsafe.sort.{PrefixComparators, RecordComparator} /** - * Performs a shuffle that will result in the desired `newPartitioning`. + * Performs a shuffle that will result in the desired partitioning. */ case class ShuffleExchangeExec( - var newPartitioning: Partitioning, + override val outputPartitioning: Partitioning, child: SparkPlan, - @transient coordinator: Option[ExchangeCoordinator]) extends Exchange { + canChangeNumPartitions: Boolean = true) extends Exchange { // NOTE: coordinator can be null after serialization/deserialization, // e.g. it can be null on the Executor side - + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) override lazy val metrics = Map( - "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size")) - - override def nodeName: String = { - val extraInfo = coordinator match { - case Some(exchangeCoordinator) => - s"(coordinator id: ${System.identityHashCode(exchangeCoordinator)})" - case _ => "" - } + "dataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size") + ) ++ readMetrics ++ writeMetrics - val simpleNodeName = "Exchange" - s"$simpleNodeName$extraInfo" - } - - override def outputPartitioning: Partitioning = newPartitioning + override def nodeName: String = "Exchange" private val serializer: Serializer = new UnsafeRowSerializer(child.output.size, longMetric("dataSize")) - override protected def doPrepare(): Unit = { - // If an ExchangeCoordinator is needed, we register this Exchange operator - // to the coordinator when we do prepare. It is important to make sure - // we register this operator right before the execution instead of register it - // in the constructor because it is possible that we create new instances of - // Exchange operators when we transform the physical plan - // (then the ExchangeCoordinator will hold references of unneeded Exchanges). - // So, we should only call registerExchange just before we start to execute - // the plan. - coordinator match { - case Some(exchangeCoordinator) => exchangeCoordinator.registerExchange(this) - case _ => - } - } + @transient lazy val inputRDD: RDD[InternalRow] = child.execute() /** - * Returns a [[ShuffleDependency]] that will partition rows of its child based on + * A [[ShuffleDependency]] that will partition rows of its child based on * the partitioning scheme defined in `newPartitioning`. Those partitions of * the returned ShuffleDependency will be the input of shuffle. */ - private[exchange] def prepareShuffleDependency() - : ShuffleDependency[Int, InternalRow, InternalRow] = { + @transient + lazy val shuffleDependency : ShuffleDependency[Int, InternalRow, InternalRow] = { ShuffleExchangeExec.prepareShuffleDependency( - child.execute(), child.output, newPartitioning, serializer) + inputRDD, + child.output, + outputPartitioning, + serializer, + writeMetrics) } - /** - * Returns a [[ShuffledRowRDD]] that represents the post-shuffle dataset. - * This [[ShuffledRowRDD]] is created based on a given [[ShuffleDependency]] and an optional - * partition start indices array. If this optional array is defined, the returned - * [[ShuffledRowRDD]] will fetch pre-shuffle partitions based on indices of this array. - */ - private[exchange] def preparePostShuffleRDD( - shuffleDependency: ShuffleDependency[Int, InternalRow, InternalRow], - specifiedPartitionStartIndices: Option[Array[Int]] = None): ShuffledRowRDD = { - // If an array of partition start indices is provided, we need to use this array - // to create the ShuffledRowRDD. Also, we need to update newPartitioning to - // update the number of post-shuffle partitions. - specifiedPartitionStartIndices.foreach { indices => - assert(newPartitioning.isInstanceOf[HashPartitioning]) - newPartitioning = UnknownPartitioning(indices.length) - } - new ShuffledRowRDD(shuffleDependency, specifiedPartitionStartIndices) + def createShuffledRDD(partitionStartIndices: Option[Array[Int]]): ShuffledRowRDD = { + new ShuffledRowRDD(shuffleDependency, readMetrics, partitionStartIndices) } /** @@ -119,24 +90,13 @@ case class ShuffleExchangeExec( protected override def doExecute(): RDD[InternalRow] = attachTree(this, "execute") { // Returns the same ShuffleRowRDD if this plan is used by multiple plans. if (cachedShuffleRDD == null) { - cachedShuffleRDD = coordinator match { - case Some(exchangeCoordinator) => - val shuffleRDD = exchangeCoordinator.postShuffleRDD(this) - assert(shuffleRDD.partitions.length == newPartitioning.numPartitions) - shuffleRDD - case _ => - val shuffleDependency = prepareShuffleDependency() - preparePostShuffleRDD(shuffleDependency) - } + cachedShuffleRDD = createShuffledRDD(None) } cachedShuffleRDD } } object ShuffleExchangeExec { - def apply(newPartitioning: Partitioning, child: SparkPlan): ShuffleExchangeExec = { - ShuffleExchangeExec(newPartitioning, child, coordinator = Option.empty[ExchangeCoordinator]) - } /** * Determines whether records must be defensively copied before being sent to the shuffle. @@ -163,7 +123,7 @@ object ShuffleExchangeExec { val conf = SparkEnv.get.conf val shuffleManager = SparkEnv.get.shuffleManager val sortBasedShuffleOn = shuffleManager.isInstanceOf[SortShuffleManager] - val bypassMergeThreshold = conf.getInt("spark.shuffle.sort.bypassMergeThreshold", 200) + val bypassMergeThreshold = conf.get(config.SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD) val numParts = partitioner.numPartitions if (sortBasedShuffleOn) { if (numParts <= bypassMergeThreshold) { @@ -203,7 +163,9 @@ object ShuffleExchangeExec { rdd: RDD[InternalRow], outputAttributes: Seq[Attribute], newPartitioning: Partitioning, - serializer: Serializer): ShuffleDependency[Int, InternalRow, InternalRow] = { + serializer: Serializer, + writeMetrics: Map[String, SQLMetric]) + : ShuffleDependency[Int, InternalRow, InternalRow] = { val part: Partitioner = newPartitioning match { case RoundRobinPartitioning(numPartitions) => new HashPartitioner(numPartitions) case HashPartitioning(_, n) => @@ -214,13 +176,21 @@ object ShuffleExchangeExec { override def getPartition(key: Any): Int = key.asInstanceOf[Int] } case RangePartitioning(sortingExpressions, numPartitions) => - // Internally, RangePartitioner runs a job on the RDD that samples keys to compute - // partition bounds. To get accurate samples, we need to copy the mutable keys. + // Extract only fields used for sorting to avoid collecting large fields that does not + // affect sorting result when deciding partition bounds in RangePartitioner val rddForSampling = rdd.mapPartitionsInternal { iter => + val projection = + UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) val mutablePair = new MutablePair[InternalRow, Null]() - iter.map(row => mutablePair.update(row.copy(), null)) + // Internally, RangePartitioner runs a job on the RDD that samples keys to compute + // partition bounds. To get accurate samples, we need to copy the mutable keys. + iter.map(row => mutablePair.update(projection(row).copy(), null)) + } + // Construct ordering on extracted sort key. + val orderingAttributes = sortingExpressions.zipWithIndex.map { case (ord, i) => + ord.copy(child = BoundReference(i, ord.dataType, ord.nullable)) } - implicit val ordering = new LazilyGeneratedOrdering(sortingExpressions, outputAttributes) + implicit val ordering = new LazilyGeneratedOrdering(orderingAttributes) new RangePartitioner( numPartitions, rddForSampling, @@ -246,7 +216,10 @@ object ShuffleExchangeExec { case h: HashPartitioning => val projection = UnsafeProjection.create(h.partitionIdExpression :: Nil, outputAttributes) row => projection(row).getInt(0) - case RangePartitioning(_, _) | SinglePartition => identity + case RangePartitioning(sortingExpressions, _) => + val projection = UnsafeProjection.create(sortingExpressions.map(_.child), outputAttributes) + row => projection(row) + case SinglePartition => identity case _ => sys.error(s"Exchange not implemented for $newPartitioning") } @@ -269,7 +242,7 @@ object ShuffleExchangeExec { } // The comparator for comparing row hashcode, which should always be Integer. val prefixComparator = PrefixComparators.LONG - val canUseRadixSort = SparkEnv.get.conf.get(SQLConf.RADIX_SORT_ENABLED) + val canUseRadixSort = SQLConf.get.enableRadixSort // The prefix computer generates row hashcode as the prefix, so we may decrease the // probability that the prefixes are equal when input rows choose column values from a // limited range. @@ -321,8 +294,22 @@ object ShuffleExchangeExec { new ShuffleDependency[Int, InternalRow, InternalRow]( rddWithPartitionIds, new PartitionIdPassthrough(part.numPartitions), - serializer) + serializer, + shuffleWriterProcessor = createShuffleWriteProcessor(writeMetrics)) dependency } + + /** + * Create a customized [[ShuffleWriteProcessor]] for SQL which wrap the default metrics reporter + * with [[SQLShuffleWriteMetricsReporter]] as new reporter for [[ShuffleWriteProcessor]]. + */ + def createShuffleWriteProcessor(metrics: Map[String, SQLMetric]): ShuffleWriteProcessor = { + new ShuffleWriteProcessor { + override protected def createMetricsReporter( + context: TaskContext): ShuffleWriteMetricsReporter = { + new SQLShuffleWriteMetricsReporter(context.taskMetrics().shuffleWriteMetrics, metrics) + } + } + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala index a6f3ea47c8492..fd4a7897c7ad1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashJoinExec.scala @@ -29,7 +29,6 @@ import org.apache.spark.sql.catalyst.plans.physical.{BroadcastDistribution, Dist import org.apache.spark.sql.execution.{BinaryExecNode, CodegenSupport, SparkPlan} import org.apache.spark.sql.execution.metric.SQLMetrics import org.apache.spark.sql.types.{BooleanType, LongType} -import org.apache.spark.util.TaskCompletionListener /** * Performs an inner hash join of two child relations. When the output RDD of this operator is @@ -48,8 +47,7 @@ case class BroadcastHashJoinExec( extends BinaryExecNode with HashJoin with CodegenSupport { override lazy val metrics = Map( - "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows")) override def requiredChildDistribution: Seq[Distribution] = { val mode = HashedRelationBroadcastMode(buildKeys) @@ -63,13 +61,12 @@ case class BroadcastHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") val broadcastRelation = buildPlan.executeBroadcast[HashedRelation]() streamedPlan.execute().mapPartitions { streamedIter => val hashed = broadcastRelation.value.asReadOnlyCopy() TaskContext.get().taskMetrics().incPeakExecutionMemory(hashed.estimatedSize) - join(streamedIter, hashed, numOutputRows, avgHashProbe) + join(streamedIter, hashed, numOutputRows) } } @@ -111,23 +108,6 @@ case class BroadcastHashJoinExec( } } - /** - * Returns the codes used to add a task completion listener to update avg hash probe - * at the end of the task. - */ - private def genTaskListener(avgHashProbe: String, relationTerm: String): String = { - val listenerClass = classOf[TaskCompletionListener].getName - val taskContextClass = classOf[TaskContext].getName - s""" - | $taskContextClass$$.MODULE$$.get().addTaskCompletionListener(new $listenerClass() { - | @Override - | public void onTaskCompletion($taskContextClass context) { - | $avgHashProbe.set($relationTerm.getAverageProbesPerLookup()); - | } - | }); - """.stripMargin - } - /** * Returns a tuple of Broadcast of HashedRelation and the variable name for it. */ @@ -137,15 +117,11 @@ case class BroadcastHashJoinExec( val broadcast = ctx.addReferenceObj("broadcast", broadcastRelation) val clsName = broadcastRelation.value.getClass.getName - // At the end of the task, we update the avg hash probe. - val avgHashProbe = metricTerm(ctx, "avgHashProbe") - // Inline mutable state since not many join operations in a task val relationTerm = ctx.addMutableState(clsName, "relation", v => s""" | $v = (($clsName) $broadcast.value()).asReadOnlyCopy(); | incPeakExecutionMemory($v.estimatedSize()); - | ${genTaskListener(avgHashProbe, v)} """.stripMargin, forceInline = true) (broadcastRelation, relationTerm) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala index dab873bf9b9a0..5ee4c7ffb1911 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashJoin.scala @@ -17,9 +17,9 @@ package org.apache.spark.sql.execution.joins -import org.apache.spark.TaskContext import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.Partitioning import org.apache.spark.sql.execution.{RowIterator, SparkPlan} @@ -64,9 +64,8 @@ trait HashJoin { protected lazy val (buildKeys, streamedKeys) = { require(leftKeys.map(_.dataType) == rightKeys.map(_.dataType), "Join keys from two sides should have same types") - val lkeys = HashJoin.rewriteKeyExpr(leftKeys).map(BindReferences.bindReference(_, left.output)) - val rkeys = HashJoin.rewriteKeyExpr(rightKeys) - .map(BindReferences.bindReference(_, right.output)) + val lkeys = bindReferences(HashJoin.rewriteKeyExpr(leftKeys), left.output) + val rkeys = bindReferences(HashJoin.rewriteKeyExpr(rightKeys), right.output) buildSide match { case BuildLeft => (lkeys, rkeys) case BuildRight => (rkeys, lkeys) @@ -194,8 +193,7 @@ trait HashJoin { protected def join( streamedIter: Iterator[InternalRow], hashed: HashedRelation, - numOutputRows: SQLMetric, - avgHashProbe: SQLMetric): Iterator[InternalRow] = { + numOutputRows: SQLMetric): Iterator[InternalRow] = { val joinedIter = joinType match { case _: InnerLike => @@ -213,10 +211,6 @@ trait HashJoin { s"BroadcastHashJoin should not take $x as the JoinType") } - // At the end of the task, we update the avg hash probe. - TaskContext.get().addTaskCompletionListener[Unit](_ => - avgHashProbe.set(hashed.getAverageProbesPerLookup)) - val resultProj = createResultProjection joinedIter.map { r => numOutputRows += 1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala index 86eb47a70f1ad..9d8063d53b5ab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/HashedRelation.scala @@ -23,8 +23,8 @@ import com.esotericsoftware.kryo.{Kryo, KryoSerializable} import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.{SparkConf, SparkEnv, SparkException} -import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED -import org.apache.spark.memory.{MemoryConsumer, StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.internal.config.{BUFFER_PAGESIZE, MEMORY_OFFHEAP_ENABLED} +import org.apache.spark.memory._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.BroadcastMode @@ -80,11 +80,6 @@ private[execution] sealed trait HashedRelation extends KnownSizeEstimation { * Release any used resources. */ def close(): Unit - - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double } private[execution] object HashedRelation { @@ -99,10 +94,10 @@ private[execution] object HashedRelation { taskMemoryManager: TaskMemoryManager = null): HashedRelation = { val mm = Option(taskMemoryManager).getOrElse { new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) } @@ -232,15 +227,15 @@ private[joins] class UnsafeHashedRelation( // TODO(josh): This needs to be revisited before we merge this patch; making this change now // so that tests compile: val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val pageSizeBytes = Option(SparkEnv.get).map(_.memoryManager.pageSizeBytes) - .getOrElse(new SparkConf().getSizeAsBytes("spark.buffer.pageSize", "16m")) + .getOrElse(new SparkConf().get(BUFFER_PAGESIZE).getOrElse(16L * 1024 * 1024)) // TODO(josh): We won't need this dummy memory manager after future refactorings; revisit // during code review @@ -248,8 +243,7 @@ private[joins] class UnsafeHashedRelation( binaryMap = new BytesToBytesMap( taskMemoryManager, (nKeys * 1.5 + 1).toInt, // reduce hash collision - pageSizeBytes, - true) + pageSizeBytes) var i = 0 var keyBuffer = new Array[Byte](1024) @@ -280,8 +274,6 @@ private[joins] class UnsafeHashedRelation( override def read(kryo: Kryo, in: Input): Unit = Utils.tryOrIOException { read(() => in.readInt(), () => in.readLong(), in.readBytes) } - - override def getAverageProbesPerLookup: Double = binaryMap.getAverageProbesPerLookup } private[joins] object UnsafeHashedRelation { @@ -293,14 +285,12 @@ private[joins] object UnsafeHashedRelation { taskMemoryManager: TaskMemoryManager): HashedRelation = { val pageSizeBytes = Option(SparkEnv.get).map(_.memoryManager.pageSizeBytes) - .getOrElse(new SparkConf().getSizeAsBytes("spark.buffer.pageSize", "16m")) - + .getOrElse(new SparkConf().get(BUFFER_PAGESIZE).getOrElse(16L * 1024 * 1024)) val binaryMap = new BytesToBytesMap( taskMemoryManager, // Only 70% of the slots can be used before growing, more capacity help to reduce collision (sizeEstimate * 1.5 + 1).toInt, - pageSizeBytes, - true) + pageSizeBytes) // Create a mapping of buildKeys -> rows val keyGenerator = UnsafeProjection.create(key) @@ -316,7 +306,9 @@ private[joins] object UnsafeHashedRelation { row.getBaseObject, row.getBaseOffset, row.getSizeInBytes) if (!success) { binaryMap.free() - throw new SparkException("There is no enough memory to build hash map") + // scalastyle:off throwerror + throw new SparkOutOfMemoryError("There is no enough memory to build hash map") + // scalastyle:on throwerror } } } @@ -395,18 +387,14 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap // The number of unique keys. private var numKeys = 0L - // Tracking average number of probes per key lookup. - private var numKeyLookups = 0L - private var numProbes = 0L - // needed by serializer def this() = { this( new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0), 0) @@ -424,7 +412,7 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def init(): Unit = { if (mm != null) { - require(capacity < 512000000, "Cannot broadcast more than 512 millions rows") + require(capacity < 512000000, "Cannot broadcast 512 million or more rows") var n = 1 while (n < capacity) n *= 2 ensureAcquireMemory(n * 2L * 8 + (1 << 20)) @@ -483,8 +471,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def getValue(key: Long, resultRow: UnsafeRow): UnsafeRow = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -493,14 +479,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return getRow(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -528,8 +511,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap */ def get(key: Long, resultRow: UnsafeRow): Iterator[UnsafeRow] = { if (isDense) { - numKeyLookups += 1 - numProbes += 1 if (key >= minKey && key <= maxKey) { val value = array((key - minKey).toInt) if (value > 0) { @@ -538,14 +519,11 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap } } else { var pos = firstSlot(key) - numKeyLookups += 1 - numProbes += 1 while (array(pos + 1) != 0) { if (array(pos) == key) { return valueIter(array(pos + 1), resultRow) } pos = nextSlot(pos) - numProbes += 1 } } null @@ -585,11 +563,8 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap private def updateIndex(key: Long, address: Long): Unit = { var pos = firstSlot(key) assert(numKeys < array.length / 2) - numKeyLookups += 1 - numProbes += 1 while (array(pos) != key && array(pos + 1) != 0) { pos = nextSlot(pos) - numProbes += 1 } if (array(pos + 1) == 0) { // this is the first value for this key, put the address in array. @@ -721,8 +696,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap writeLong(maxKey) writeLong(numKeys) writeLong(numValues) - writeLong(numKeyLookups) - writeLong(numProbes) writeLong(array.length) writeLongArray(writeBuffer, array, array.length) @@ -764,8 +737,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap maxKey = readLong() numKeys = readLong() numValues = readLong() - numKeyLookups = readLong() - numProbes = readLong() val length = readLong().toInt mask = length - 2 @@ -783,11 +754,6 @@ private[execution] final class LongToUnsafeRowMap(val mm: TaskMemoryManager, cap override def read(kryo: Kryo, in: Input): Unit = { read(() => in.readBoolean(), () => in.readLong(), in.readBytes) } - - /** - * Returns the average number of probes per key lookup. - */ - def getAverageProbesPerLookup: Double = numProbes.toDouble / numKeyLookups } private[joins] class LongHashedRelation( @@ -839,8 +805,6 @@ private[joins] class LongHashedRelation( resultRow = new UnsafeRow(nFields) map = in.readObject().asInstanceOf[LongToUnsafeRowMap] } - - override def getAverageProbesPerLookup: Double = map.getAverageProbesPerLookup } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala index 2b59ed6e4d16b..a8361fd7dd559 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashJoinExec.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.execution.joins +import java.util.concurrent.TimeUnit._ + import org.apache.spark.TaskContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow @@ -42,8 +44,7 @@ case class ShuffledHashJoinExec( override lazy val metrics = Map( "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "buildDataSize" -> SQLMetrics.createSizeMetric(sparkContext, "data size of build side"), - "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map"), - "avgHashProbe" -> SQLMetrics.createAverageMetric(sparkContext, "avg hash probe")) + "buildTime" -> SQLMetrics.createTimingMetric(sparkContext, "time to build hash map")) override def requiredChildDistribution: Seq[Distribution] = HashClusteredDistribution(leftKeys) :: HashClusteredDistribution(rightKeys) :: Nil @@ -54,7 +55,7 @@ case class ShuffledHashJoinExec( val start = System.nanoTime() val context = TaskContext.get() val relation = HashedRelation(iter, buildKeys, taskMemoryManager = context.taskMemoryManager()) - buildTime += (System.nanoTime() - start) / 1000000 + buildTime += NANOSECONDS.toMillis(System.nanoTime() - start) buildDataSize += relation.estimatedSize // This relation is usually used until the end of task. context.addTaskCompletionListener[Unit](_ => relation.close()) @@ -63,10 +64,9 @@ case class ShuffledHashJoinExec( protected override def doExecute(): RDD[InternalRow] = { val numOutputRows = longMetric("numOutputRows") - val avgHashProbe = longMetric("avgHashProbe") streamedPlan.execute().zipPartitions(buildPlan.execute()) { (streamIter, buildIter) => val hashed = buildHashedRelation(buildIter) - join(streamIter, hashed, numOutputRows, avgHashProbe) + join(streamIter, hashed, numOutputRows) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala index d7d3f6d6078b4..f829f07e80720 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoinExec.scala @@ -22,6 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ import org.apache.spark.sql.catalyst.plans._ @@ -393,7 +394,7 @@ case class SortMergeJoinExec( input: Seq[Attribute]): Seq[ExprCode] = { ctx.INPUT_ROW = row ctx.currentVars = null - keys.map(BindReferences.bindReference(_, input).genCode(ctx)) + bindReferences(keys, input).map(_.genCode(ctx)) } private def copyKeys(ctx: CodegenContext, vars: Seq[ExprCode]): Seq[ExprCode] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala index 9bfe1a79fc1e1..2ff08883d5cab 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/limit.scala @@ -23,8 +23,17 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, CodeGenerator, ExprCode, LazilyGeneratedOrdering} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec -import org.apache.spark.util.Utils +import org.apache.spark.sql.execution.metric.{SQLShuffleReadMetricsReporter, SQLShuffleWriteMetricsReporter} + +/** + * The operator takes limited number of elements from its child operator. + */ +trait LimitExec extends UnaryExecNode { + /** Number of element should be taken from child operator */ + def limit: Int +} /** * Take the first `limit` elements and collect them to a single partition. @@ -32,16 +41,26 @@ import org.apache.spark.util.Utils * This operator will be used when a logical `Limit` operation is the final operator in an * logical plan, which happens when the user is collecting results back to the driver. */ -case class CollectLimitExec(limit: Int, child: SparkPlan) extends UnaryExecNode { +case class CollectLimitExec(limit: Int, child: SparkPlan) extends LimitExec { override def output: Seq[Attribute] = child.output override def outputPartitioning: Partitioning = SinglePartition override def executeCollect(): Array[InternalRow] = child.executeTake(limit) private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics = readMetrics ++ writeMetrics protected override def doExecute(): RDD[InternalRow] = { val locallyLimited = child.execute().mapPartitionsInternal(_.take(limit)) val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - locallyLimited, child.output, SinglePartition, serializer)) + locallyLimited, + child.output, + SinglePartition, + serializer, + writeMetrics), + readMetrics) shuffled.mapPartitionsInternal(_.take(limit)) } } @@ -59,8 +78,7 @@ object BaseLimitExec { * Helper trait which defines methods that are shared by both * [[LocalLimitExec]] and [[GlobalLimitExec]]. */ -trait BaseLimitExec extends UnaryExecNode with CodegenSupport { - val limit: Int +trait BaseLimitExec extends LimitExec with CodegenSupport { override def output: Seq[Attribute] = child.output protected override def doExecute(): RDD[InternalRow] = child.execute().mapPartitions { iter => @@ -151,6 +169,12 @@ case class TakeOrderedAndProjectExec( private val serializer: Serializer = new UnsafeRowSerializer(child.output.size) + private lazy val writeMetrics = + SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext) + private lazy val readMetrics = + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(sparkContext) + override lazy val metrics = readMetrics ++ writeMetrics + protected override def doExecute(): RDD[InternalRow] = { val ord = new LazilyGeneratedOrdering(sortOrder, child.output) val localTopK: RDD[InternalRow] = { @@ -160,7 +184,12 @@ case class TakeOrderedAndProjectExec( } val shuffled = new ShuffledRowRDD( ShuffleExchangeExec.prepareShuffleDependency( - localTopK, child.output, SinglePartition, serializer)) + localTopK, + child.output, + SinglePartition, + serializer, + writeMetrics), + readMetrics) shuffled.mapPartitions { iter => val topK = org.apache.spark.util.collection.Utils.takeOrdered(iter.map(_.copy()), limit)(ord) if (projectList != child.output) { @@ -176,9 +205,9 @@ case class TakeOrderedAndProjectExec( override def outputPartitioning: Partitioning = SinglePartition - override def simpleString: String = { - val orderByString = Utils.truncatedString(sortOrder, "[", ",", "]") - val outputString = Utils.truncatedString(output, "[", ",", "]") + override def simpleString(maxFields: Int): String = { + val orderByString = truncatedString(sortOrder, "[", ",", "]", maxFields) + val outputString = truncatedString(output, "[", ",", "]", maxFields) s"TakeOrderedAndProject(limit=$limit, orderBy=$orderByString, output=$outputString)" } 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 cbf707f4a9cfd..19809b07508d9 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 @@ -20,6 +20,8 @@ package org.apache.spark.sql.execution.metric import java.text.NumberFormat import java.util.Locale +import scala.concurrent.duration._ + import org.apache.spark.SparkContext import org.apache.spark.scheduler.AccumulableInfo import org.apache.spark.sql.execution.ui.SparkListenerDriverAccumUpdates @@ -78,6 +80,7 @@ object SQLMetrics { private val SUM_METRIC = "sum" private val SIZE_METRIC = "size" private val TIMING_METRIC = "timing" + private val NS_TIMING_METRIC = "nsTiming" private val AVERAGE_METRIC = "average" private val baseForAvgMetric: Int = 10 @@ -121,6 +124,13 @@ object SQLMetrics { 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)"), countFailedValues = false) + acc + } + /** * Create a metric to report the average information (including min, med, max) like * avg hash probe. As average metrics are double values, this kind of metrics should be @@ -163,6 +173,8 @@ object SQLMetrics { Utils.bytesToString } else if (metricsType == TIMING_METRIC) { Utils.msDurationToString + } else if (metricsType == NS_TIMING_METRIC) { + duration => Utils.msDurationToString(duration.nanos.toMillis) } else { throw new IllegalStateException("unexpected metrics type: " + metricsType) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala new file mode 100644 index 0000000000000..2c0ea80495abb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/metric/SQLShuffleMetricsReporter.scala @@ -0,0 +1,152 @@ +/* + * 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.metric + +import org.apache.spark.SparkContext +import org.apache.spark.executor.TempShuffleReadMetrics +import org.apache.spark.shuffle.ShuffleWriteMetricsReporter + +/** + * A shuffle metrics reporter for SQL exchange operators. + * @param tempMetrics [[TempShuffleReadMetrics]] created in TaskContext. + * @param metrics All metrics in current SparkPlan. This param should not empty and + * contains all shuffle metrics defined in createShuffleReadMetrics. + */ +class SQLShuffleReadMetricsReporter( + tempMetrics: TempShuffleReadMetrics, + metrics: Map[String, SQLMetric]) extends TempShuffleReadMetrics { + private[this] val _remoteBlocksFetched = + metrics(SQLShuffleReadMetricsReporter.REMOTE_BLOCKS_FETCHED) + private[this] val _localBlocksFetched = + metrics(SQLShuffleReadMetricsReporter.LOCAL_BLOCKS_FETCHED) + private[this] val _remoteBytesRead = + metrics(SQLShuffleReadMetricsReporter.REMOTE_BYTES_READ) + private[this] val _remoteBytesReadToDisk = + metrics(SQLShuffleReadMetricsReporter.REMOTE_BYTES_READ_TO_DISK) + private[this] val _localBytesRead = + metrics(SQLShuffleReadMetricsReporter.LOCAL_BYTES_READ) + private[this] val _fetchWaitTime = + metrics(SQLShuffleReadMetricsReporter.FETCH_WAIT_TIME) + private[this] val _recordsRead = + metrics(SQLShuffleReadMetricsReporter.RECORDS_READ) + + override def incRemoteBlocksFetched(v: Long): Unit = { + _remoteBlocksFetched.add(v) + tempMetrics.incRemoteBlocksFetched(v) + } + override def incLocalBlocksFetched(v: Long): Unit = { + _localBlocksFetched.add(v) + tempMetrics.incLocalBlocksFetched(v) + } + override def incRemoteBytesRead(v: Long): Unit = { + _remoteBytesRead.add(v) + tempMetrics.incRemoteBytesRead(v) + } + override def incRemoteBytesReadToDisk(v: Long): Unit = { + _remoteBytesReadToDisk.add(v) + tempMetrics.incRemoteBytesReadToDisk(v) + } + override def incLocalBytesRead(v: Long): Unit = { + _localBytesRead.add(v) + tempMetrics.incLocalBytesRead(v) + } + override def incFetchWaitTime(v: Long): Unit = { + _fetchWaitTime.add(v) + tempMetrics.incFetchWaitTime(v) + } + override def incRecordsRead(v: Long): Unit = { + _recordsRead.add(v) + tempMetrics.incRecordsRead(v) + } +} + +object SQLShuffleReadMetricsReporter { + val REMOTE_BLOCKS_FETCHED = "remoteBlocksFetched" + val LOCAL_BLOCKS_FETCHED = "localBlocksFetched" + val REMOTE_BYTES_READ = "remoteBytesRead" + val REMOTE_BYTES_READ_TO_DISK = "remoteBytesReadToDisk" + val LOCAL_BYTES_READ = "localBytesRead" + val FETCH_WAIT_TIME = "fetchWaitTime" + val RECORDS_READ = "recordsRead" + + /** + * Create all shuffle read relative metrics and return the Map. + */ + def createShuffleReadMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( + REMOTE_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "remote blocks read"), + LOCAL_BLOCKS_FETCHED -> SQLMetrics.createMetric(sc, "local blocks read"), + REMOTE_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "remote bytes read"), + REMOTE_BYTES_READ_TO_DISK -> SQLMetrics.createSizeMetric(sc, "remote bytes read to disk"), + LOCAL_BYTES_READ -> SQLMetrics.createSizeMetric(sc, "local bytes read"), + FETCH_WAIT_TIME -> SQLMetrics.createTimingMetric(sc, "fetch wait time"), + RECORDS_READ -> SQLMetrics.createMetric(sc, "records read")) +} + +/** + * A shuffle write metrics reporter for SQL exchange operators. + * @param metricsReporter Other reporter need to be updated in this SQLShuffleWriteMetricsReporter. + * @param metrics Shuffle write metrics in current SparkPlan. + */ +class SQLShuffleWriteMetricsReporter( + metricsReporter: ShuffleWriteMetricsReporter, + metrics: Map[String, SQLMetric]) extends ShuffleWriteMetricsReporter { + private[this] val _bytesWritten = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_BYTES_WRITTEN) + private[this] val _recordsWritten = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_RECORDS_WRITTEN) + private[this] val _writeTime = + metrics(SQLShuffleWriteMetricsReporter.SHUFFLE_WRITE_TIME) + + override def incBytesWritten(v: Long): Unit = { + metricsReporter.incBytesWritten(v) + _bytesWritten.add(v) + } + override def decRecordsWritten(v: Long): Unit = { + metricsReporter.decBytesWritten(v) + _recordsWritten.set(_recordsWritten.value - v) + } + override def incRecordsWritten(v: Long): Unit = { + metricsReporter.incRecordsWritten(v) + _recordsWritten.add(v) + } + override def incWriteTime(v: Long): Unit = { + metricsReporter.incWriteTime(v) + _writeTime.add(v) + } + override def decBytesWritten(v: Long): Unit = { + metricsReporter.decBytesWritten(v) + _bytesWritten.set(_bytesWritten.value - v) + } +} + +object SQLShuffleWriteMetricsReporter { + val SHUFFLE_BYTES_WRITTEN = "shuffleBytesWritten" + val SHUFFLE_RECORDS_WRITTEN = "shuffleRecordsWritten" + val SHUFFLE_WRITE_TIME = "shuffleWriteTime" + + /** + * Create all shuffle write relative metrics and return the Map. + */ + def createShuffleWriteMetrics(sc: SparkContext): Map[String, SQLMetric] = Map( + SHUFFLE_BYTES_WRITTEN -> + SQLMetrics.createSizeMetric(sc, "shuffle bytes written"), + SHUFFLE_RECORDS_WRITTEN -> + SQLMetrics.createMetric(sc, "shuffle records written"), + SHUFFLE_WRITE_TIME -> + SQLMetrics.createNanoTimingMetric(sc, "shuffle write time")) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala index 03d1bbf2ab882..d05113431df41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/objects.scala @@ -17,6 +17,9 @@ package org.apache.spark.sql.execution +import java.io.{ByteArrayOutputStream, DataOutputStream} + +import scala.collection.JavaConverters._ import scala.language.existentials import org.apache.spark.api.java.function.MapFunction @@ -31,7 +34,10 @@ import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.objects.Invoke import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, FunctionUtils, LogicalGroupState} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.python.BatchIterator +import org.apache.spark.sql.execution.r.ArrowRRunner import org.apache.spark.sql.execution.streaming.GroupStateImpl +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.GroupStateTimeout import org.apache.spark.sql.types._ @@ -56,7 +62,8 @@ trait ObjectConsumerExec extends UnaryExecNode { assert(child.output.length == 1) // This operator always need all columns of its child, even it doesn't reference to. - override def references: AttributeSet = child.outputSet + @transient + override lazy val references: AttributeSet = child.outputSet def inputObjectType: DataType = child.output.head.dataType } @@ -184,12 +191,69 @@ case class MapPartitionsExec( override protected def doExecute(): RDD[InternalRow] = { child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) func(iter.map(getObject)).map(outputObject) } } } +/** + * Similar with [[MapPartitionsExec]] and + * [[org.apache.spark.sql.execution.r.MapPartitionsRWrapper]] but serializes and deserializes + * input/output in Arrow format. + * + * This is somewhat similar with [[org.apache.spark.sql.execution.python.ArrowEvalPythonExec]] + */ +case class MapPartitionsInRWithArrowExec( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + output: Seq[Attribute], + child: SparkPlan) extends UnaryExecNode { + override def producedAttributes: AttributeSet = AttributeSet(output) + + private val batchSize = conf.arrowMaxRecordsPerBatch + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { inputIter => + val outputTypes = schema.map(_.dataType) + + // DO NOT use iter.grouped(). See BatchIterator. + val batchIter = + if (batchSize > 0) new BatchIterator(inputIter, batchSize) else Iterator(inputIter) + + val runner = new ArrowRRunner(func, packageNames, broadcastVars, inputSchema, + SQLConf.get.sessionLocalTimeZone, RRunnerModes.DATAFRAME_DAPPLY) + + // The communication mechanism is as follows: + // + // JVM side R side + // + // 1. Internal rows --------> Arrow record batches + // 2. Converts each Arrow record batch to each R data frame + // 3. Combine R data frames into one R data frame + // 4. Computes R native function on the data frame + // 5. Converts the R data frame to Arrow record batches + // 6. Columnar batches <-------- Arrow record batches + // 7. Each row from each batch + // + // Note that, unlike Python vectorization implementation, R side sends Arrow formatted + // binary in a batch due to the limitation of R API. See also ARROW-4512. + val columnarBatchIter = runner.compute(batchIter, -1) + val outputProject = UnsafeProjection.create(output, output) + columnarBatchIter.flatMap { batch => + val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType()) + assert(outputTypes == actualDataTypes, "Invalid schema from dapply(): " + + s"expected ${outputTypes.mkString(", ")}, got ${actualDataTypes.mkString(", ")}") + batch.rowIterator.asScala + }.map(outputProject) + } + } +} + /** * Applies the given function to each input object. * The output of its child must be a single-field row containing the input object. @@ -214,10 +278,10 @@ case class MapElementsExec( override def doConsume(ctx: CodegenContext, input: Seq[ExprCode], row: ExprCode): String = { val (funcClass, methodName) = func match { case m: MapFunction[_, _] => classOf[MapFunction[_, _]] -> "call" - case _ => FunctionUtils.getFunctionOneName(outputObjAttr.dataType, child.output(0).dataType) + case _ => FunctionUtils.getFunctionOneName(outputObjectType, child.output(0).dataType) } val funcObj = Literal.create(func, ObjectType(funcClass)) - val callFunc = Invoke(funcObj, methodName, outputObjAttr.dataType, child.output) + val callFunc = Invoke(funcObj, methodName, outputObjectType, child.output) val result = BindReferences.bindReference(callFunc, child.output).genCode(ctx) @@ -232,7 +296,7 @@ case class MapElementsExec( child.execute().mapPartitionsInternal { iter => val getObject = ObjectOperator.unwrapObjectFromRow(child.output.head.dataType) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) iter.map(row => outputObject(callFunc(getObject(row)))) } } @@ -331,7 +395,7 @@ case class MapGroupsExec( val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) grouped.flatMap { case (key, rowIter) => val result = func( @@ -383,12 +447,8 @@ case class FlatMapGroupsInRExec( outputObjAttr: Attribute, child: SparkPlan) extends UnaryExecNode with ObjectProducerExec { - override def output: Seq[Attribute] = outputObjAttr :: Nil - override def outputPartitioning: Partitioning = child.outputPartitioning - override def producedAttributes: AttributeSet = AttributeSet(outputObjAttr) - override def requiredChildDistribution: Seq[Distribution] = if (groupingAttributes.isEmpty) { AllTuples :: Nil @@ -411,8 +471,8 @@ case class FlatMapGroupsInRExec( val grouped = GroupedIterator(iter, groupingAttributes, child.output) val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) val getValue = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) - val runner = new RRunner[Array[Byte]]( + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) + val runner = new RRunner[(Array[Byte], Iterator[Array[Byte]]), Array[Byte]]( func, SerializationFormats.ROW, serializerForR, packageNames, broadcastVars, isDataFrame = true, colNames = inputSchema.fieldNames, mode = RRunnerModes.DATAFRAME_GAPPLY) @@ -437,6 +497,81 @@ case class FlatMapGroupsInRExec( } } +/** + * Similar with [[FlatMapGroupsInRExec]] but serializes and deserializes input/output in + * Arrow format. + * This is also somewhat similar with + * [[org.apache.spark.sql.execution.python.FlatMapGroupsInPandasExec]]. + */ +case class FlatMapGroupsInRWithArrowExec( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + inputSchema: StructType, + output: Seq[Attribute], + keyDeserializer: Expression, + groupingAttributes: Seq[Attribute], + child: SparkPlan) extends UnaryExecNode { + override def outputPartitioning: Partitioning = child.outputPartitioning + + override def producedAttributes: AttributeSet = AttributeSet(output) + + override def requiredChildDistribution: Seq[Distribution] = + if (groupingAttributes.isEmpty) { + AllTuples :: Nil + } else { + ClusteredDistribution(groupingAttributes) :: Nil + } + + override def requiredChildOrdering: Seq[Seq[SortOrder]] = + Seq(groupingAttributes.map(SortOrder(_, Ascending))) + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { iter => + val grouped = GroupedIterator(iter, groupingAttributes, child.output) + val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) + + val keys = collection.mutable.ArrayBuffer.empty[Array[Byte]] + val groupedByRKey: Iterator[Iterator[InternalRow]] = + grouped.map { case (key, rowIter) => + keys.append(rowToRBytes(getKey(key).asInstanceOf[Row])) + rowIter + } + + val runner = new ArrowRRunner(func, packageNames, broadcastVars, inputSchema, + SQLConf.get.sessionLocalTimeZone, RRunnerModes.DATAFRAME_GAPPLY) { + protected override def bufferedWrite( + dataOut: DataOutputStream)(writeFunc: ByteArrayOutputStream => Unit): Unit = { + super.bufferedWrite(dataOut)(writeFunc) + // Don't forget we're sending keys additionally. + keys.foreach(dataOut.write) + } + } + + // The communication mechanism is as follows: + // + // JVM side R side + // + // 1. Group internal rows + // 2. Grouped internal rows --------> Arrow record batches + // 3. Grouped keys --------> Regular serialized keys + // 4. Converts each Arrow record batch to each R data frame + // 5. Deserializes keys + // 6. Maps each key to each R Data frame + // 7. Computes R native function on each key/R data frame + // 8. Converts all R data frames to Arrow record batches + // 9. Columnar batches <-------- Arrow record batches + // 10. Each row from each batch + // + // Note that, unlike Python vectorization implementation, R side sends Arrow formatted + // binary in a batch due to the limitation of R API. See also ARROW-4512. + val columnarBatchIter = runner.compute(groupedByRKey, -1) + val outputProject = UnsafeProjection.create(output, output) + columnarBatchIter.flatMap(_.rowIterator().asScala).map(outputProject) + } + } +} + /** * Co-groups the data from left and right children, and calls the function with each group and 2 * iterators containing all elements in the group from left and right side. @@ -469,7 +604,7 @@ case class CoGroupExec( val getKey = ObjectOperator.deserializeRowToObject(keyDeserializer, leftGroup) val getLeft = ObjectOperator.deserializeRowToObject(leftDeserializer, leftAttr) val getRight = ObjectOperator.deserializeRowToObject(rightDeserializer, rightAttr) - val outputObject = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + val outputObject = ObjectOperator.wrapObjectToRow(outputObjectType) new CoGroupedIterator(leftGrouped, rightGrouped, leftGroup).flatMap { case (key, leftResult, rightResult) => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala index 2ab7240556aaa..fcbd0b19515b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/AggregateInPandasExec.scala @@ -28,8 +28,8 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils /** @@ -105,7 +105,8 @@ case class AggregateInPandasExec( StructField(s"_$i", dt) }) - inputRDD.mapPartitionsInternal { iter => + // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty + inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { val prunedProj = UnsafeProjection.create(allInputs, child.output) val grouped = if (groupingExpressions.isEmpty) { @@ -151,6 +152,6 @@ case class AggregateInPandasExec( val joinedRow = joined(leftRow, aggOutputRow) resultProj(joinedRow) } - } + }} } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala index 2b87796dc6833..e714554f108ff 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowEvalPythonExec.scala @@ -23,10 +23,9 @@ import org.apache.spark.TaskContext import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils /** * Grouped a iterator into batches. @@ -34,7 +33,7 @@ import org.apache.spark.sql.types.StructType * This is necessary because sometimes we cannot hold reference of input rows * because the some input rows are mutable and can be reused. */ -private class BatchIterator[T](iter: Iterator[T], batchSize: Int) +private[spark] class BatchIterator[T](iter: Iterator[T], batchSize: Int) extends Iterator[Iterator[T]] { override def hasNext: Boolean = iter.hasNext @@ -57,17 +56,12 @@ private class BatchIterator[T](iter: Iterator[T], batchSize: Int) } } -/** - * A logical plan that evaluates a [[PythonUDF]]. - */ -case class ArrowEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan) - extends UnaryNode - /** * A physical plan that evaluates a [[PythonUDF]]. */ -case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) - extends EvalPythonExec(udfs, output, child) { +case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan, + evalType: Int) + extends EvalPythonExec(udfs, resultAttrs, child) { private val batchSize = conf.arrowMaxRecordsPerBatch private val sessionLocalTimeZone = conf.sessionLocalTimeZone @@ -87,34 +81,17 @@ case class ArrowEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], chi val columnarBatchIter = new ArrowPythonRunner( funcs, - PythonEvalType.SQL_SCALAR_PANDAS_UDF, + evalType, argOffsets, schema, sessionLocalTimeZone, pythonRunnerConf).compute(batchIter, context.partitionId(), context) - new Iterator[InternalRow] { - - private var currentIter = if (columnarBatchIter.hasNext) { - val batch = columnarBatchIter.next() - val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType()) - assert(outputTypes == actualDataTypes, "Invalid schema from pandas_udf: " + - s"expected ${outputTypes.mkString(", ")}, got ${actualDataTypes.mkString(", ")}") - batch.rowIterator.asScala - } else { - Iterator.empty - } - - override def hasNext: Boolean = currentIter.hasNext || { - if (columnarBatchIter.hasNext) { - currentIter = columnarBatchIter.next().rowIterator.asScala - hasNext - } else { - false - } - } - - override def next(): InternalRow = currentIter.next() + columnarBatchIter.flatMap { batch => + val actualDataTypes = (0 until batch.numCols()).map(i => batch.column(i).dataType()) + assert(outputTypes == actualDataTypes, "Invalid schema from pandas_udf: " + + s"expected ${outputTypes.mkString(", ")}, got ${actualDataTypes.mkString(", ")}") + batch.rowIterator.asScala } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala index 18992d7a9f974..5101f7e871af2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ArrowPythonRunner.scala @@ -29,8 +29,10 @@ import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} import org.apache.spark._ import org.apache.spark.api.python._ import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.arrow.{ArrowUtils, ArrowWriter} +import org.apache.spark.sql.execution.arrow.ArrowWriter +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} import org.apache.spark.util.Utils @@ -47,6 +49,12 @@ class ArrowPythonRunner( extends BasePythonRunner[Iterator[InternalRow], ColumnarBatch]( funcs, evalType, argOffsets) { + override val bufferSize: Int = SQLConf.get.pandasUDFBufferSize + require( + bufferSize >= 4, + "Pandas execution requires more than 4 bytes. Please set higher buffer. " + + s"Please change '${SQLConf.PANDAS_UDF_BUFFER_SIZE.key}'.") + protected override def newWriterThread( env: SparkEnv, worker: Socket, @@ -117,9 +125,9 @@ class ArrowPythonRunner( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[ColumnarBatch] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { private val allocator = ArrowUtils.rootAllocator.newChildAllocator( s"stdin reader for $pythonExec", 0, Long.MaxValue) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala index b08b7e60e130b..02bfbc4949b37 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExec.scala @@ -25,21 +25,14 @@ import org.apache.spark.TaskContext import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, UnaryNode} import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.types.{StructField, StructType} -/** - * A logical plan that evaluates a [[PythonUDF]] - */ -case class BatchEvalPython(udfs: Seq[PythonUDF], output: Seq[Attribute], child: LogicalPlan) - extends UnaryNode - /** * A physical plan that evaluates a [[PythonUDF]] */ -case class BatchEvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) - extends EvalPythonExec(udfs, output, child) { +case class BatchEvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan) + extends EvalPythonExec(udfs, resultAttrs, child) { protected override def evaluate( funcs: Seq[ChainedPythonFunctions], diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala index 942a6db57416e..3554bdb5c9e0c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/EvalPythonExec.scala @@ -26,7 +26,7 @@ import org.apache.spark.api.python.ChainedPythonFunctions import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.types.{DataType, StructField, StructType} import org.apache.spark.util.Utils @@ -57,12 +57,12 @@ import org.apache.spark.util.Utils * there should be always some rows buffered in the socket or Python process, so the pulling from * RowQueue ALWAYS happened after pushing into it. */ -abstract class EvalPythonExec(udfs: Seq[PythonUDF], output: Seq[Attribute], child: SparkPlan) - extends SparkPlan { +abstract class EvalPythonExec(udfs: Seq[PythonUDF], resultAttrs: Seq[Attribute], child: SparkPlan) + extends UnaryExecNode { - def children: Seq[SparkPlan] = child :: Nil + override def output: Seq[Attribute] = child.output ++ resultAttrs - override def producedAttributes: AttributeSet = AttributeSet(output.drop(child.output.length)) + override def producedAttributes: AttributeSet = AttributeSet(resultAttrs) private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = { udf.children match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala index 90b5325919e96..fc4ded376bf1b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFs.scala @@ -24,7 +24,7 @@ import org.apache.spark.api.python.PythonEvalType import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression -import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Filter, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules.Rule @@ -111,19 +111,27 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { } private def collectEvaluableUDFsFromExpressions(expressions: Seq[Expression]): Seq[PythonUDF] = { - // Eval type checker is set once when we find the first evaluable UDF and its value - // shouldn't change later. - // Used to check if subsequent UDFs are of the same type as the first UDF. (since we can only + // If fisrt UDF is SQL_SCALAR_PANDAS_ITER_UDF, then only return this UDF, + // otherwise check if subsequent UDFs are of the same type as the first UDF. (since we can only // extract UDFs of the same eval type) - var evalTypeChecker: Option[EvalTypeChecker] = None + + var firstVisitedScalarUDFEvalType: Option[Int] = None + + def canChainUDF(evalType: Int): Boolean = { + if (evalType == PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF) { + false + } else { + evalType == firstVisitedScalarUDFEvalType.get + } + } def collectEvaluableUDFs(expr: Expression): Seq[PythonUDF] = expr match { case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf) && canEvaluateInPython(udf) - && evalTypeChecker.isEmpty => - evalTypeChecker = Some((otherEvalType: EvalType) => otherEvalType == udf.evalType) + && firstVisitedScalarUDFEvalType.isEmpty => + firstVisitedScalarUDFEvalType = Some(udf.evalType) Seq(udf) case udf: PythonUDF if PythonUDF.isScalarPythonUDF(udf) && canEvaluateInPython(udf) - && evalTypeChecker.get(udf.evalType) => + && canChainUDF(udf.evalType) => Seq(udf) case e => e.children.flatMap(collectEvaluableUDFs) } @@ -131,8 +139,20 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { expressions.flatMap(collectEvaluableUDFs) } - def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { - case plan: LogicalPlan => extract(plan) + def apply(plan: LogicalPlan): LogicalPlan = plan match { + // SPARK-26293: A subquery will be rewritten into join later, and will go through this rule + // eventually. Here we skip subquery, as Python UDF only needs to be extracted once. + case _: Subquery => plan + + case _ => plan transformUp { + // A safe guard. `ExtractPythonUDFs` only runs once, so we will not hit `BatchEvalPython` and + // `ArrowEvalPython` in the input plan. However if we hit them, we must skip them, as we can't + // extract Python UDFs from them. + case p: BatchEvalPython => p + case p: ArrowEvalPython => p + + case plan: LogicalPlan => extract(plan) + } } /** @@ -146,21 +166,9 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { // If there aren't any, we are done. plan } else { - val inputsForPlan = plan.references ++ plan.outputSet - val prunedChildren = plan.children.map { child => - val allNeededOutput = inputsForPlan.intersect(child.outputSet).toSeq - if (allNeededOutput.length != child.output.length) { - Project(allNeededOutput, child) - } else { - child - } - } - val planWithNewChildren = plan.withNewChildren(prunedChildren) - val attributeMap = mutable.HashMap[PythonUDF, Expression]() - val splitFilter = trySplitFilter(planWithNewChildren) // Rewrite the child that has the input required for the UDF - val newChildren = splitFilter.children.map { child => + val newChildren = plan.children.map { child => // Pick the UDF we are going to evaluate val validUdfs = udfs.filter { udf => // Check to make sure that the UDF can be evaluated with only the input of this child. @@ -171,20 +179,24 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { validUdfs.forall(PythonUDF.isScalarPythonUDF), "Can only extract scalar vectorized udf or sql batch udf") - val resultAttrs = udfs.zipWithIndex.map { case (u, i) => + val resultAttrs = validUdfs.zipWithIndex.map { case (u, i) => AttributeReference(s"pythonUDF$i", u.dataType)() } - val evaluation = validUdfs.partition( - _.evalType == PythonEvalType.SQL_SCALAR_PANDAS_UDF - ) match { - case (vectorizedUdfs, plainUdfs) if plainUdfs.isEmpty => - ArrowEvalPython(vectorizedUdfs, child.output ++ resultAttrs, child) - case (vectorizedUdfs, plainUdfs) if vectorizedUdfs.isEmpty => - BatchEvalPython(plainUdfs, child.output ++ resultAttrs, child) + val evalTypes = validUdfs.map(_.evalType).toSet + if (evalTypes.size != 1) { + throw new AnalysisException( + s"Expected udfs have the same evalType but got different evalTypes: " + + s"${evalTypes.mkString(",")}") + } + val evalType = evalTypes.head + val evaluation = evalType match { + case PythonEvalType.SQL_BATCHED_UDF => + BatchEvalPython(validUdfs, resultAttrs, child) + case PythonEvalType.SQL_SCALAR_PANDAS_UDF | PythonEvalType.SQL_SCALAR_PANDAS_ITER_UDF => + ArrowEvalPython(validUdfs, resultAttrs, child, evalType) case _ => - throw new AnalysisException( - "Expected either Scalar Pandas UDFs or Batched UDFs but got both") + throw new AnalysisException("Unexcepted UDF evalType") } attributeMap ++= validUdfs.zip(resultAttrs) @@ -199,7 +211,7 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { sys.error(s"Invalid PythonUDF $udf, requires attributes from more than one child.") } - val rewritten = splitFilter.withNewChildren(newChildren).transformExpressions { + val rewritten = plan.withNewChildren(newChildren).transformExpressions { case p: PythonUDF if attributeMap.contains(p) => attributeMap(p) } @@ -214,22 +226,4 @@ object ExtractPythonUDFs extends Rule[LogicalPlan] with PredicateHelper { } } } - - // Split the original FilterExec to two FilterExecs. Only push down the first few predicates - // that are all deterministic. - private def trySplitFilter(plan: LogicalPlan): LogicalPlan = { - plan match { - case filter: Filter => - val (candidates, nonDeterministic) = - splitConjunctivePredicates(filter.condition).partition(_.deterministic) - val (pushDown, rest) = candidates.partition(!hasScalarPythonUDF(_)) - if (pushDown.nonEmpty) { - val newChild = Filter(pushDown.reduceLeft(And), filter.child) - Filter((rest ++ nonDeterministic).reduceLeft(And), newChild) - } else { - filter - } - case o => o - } - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala index e9cff1a5a2007..267698d1bca50 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/FlatMapGroupsInPandasExec.scala @@ -27,8 +27,9 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} /** * Physical node for [[org.apache.spark.sql.catalyst.plans.logical.FlatMapGroupsInPandas]] @@ -124,7 +125,8 @@ case class FlatMapGroupsInPandasExec( val dedupAttributes = nonDupGroupingAttributes ++ dataAttributes val dedupSchema = StructType.fromAttributes(dedupAttributes) - inputRDD.mapPartitionsInternal { iter => + // Map grouped rows to ArrowPythonRunner results, Only execute if partition is not empty + inputRDD.mapPartitionsInternal { iter => if (iter.isEmpty) iter else { val grouped = if (groupingAttributes.isEmpty) { Iterator(iter) } else { @@ -145,7 +147,16 @@ case class FlatMapGroupsInPandasExec( sessionLocalTimeZone, pythonRunnerConf).compute(grouped, context.partitionId(), context) - columnarBatchIter.flatMap(_.rowIterator.asScala).map(UnsafeProjection.create(output, output)) - } + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // Grouped Map UDF returns a StructType column in ColumnarBatch, select the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + }} } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInPandasExec.scala new file mode 100644 index 0000000000000..2bb808119c0ae --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/MapInPandasExec.scala @@ -0,0 +1,95 @@ +/* + * 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.python + +import scala.collection.JavaConverters._ + +import org.apache.spark.TaskContext +import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} + +/** + * A relation produced by applying a function that takes an iterator of pandas DataFrames + * and outputs an iterator of pandas DataFrames. + * + * This is somewhat similar with [[FlatMapGroupsInPandasExec]] and + * `org.apache.spark.sql.catalyst.plans.logical.MapPartitionsInRWithArrow` + * + */ +case class MapInPandasExec( + func: Expression, + output: Seq[Attribute], + child: SparkPlan) + extends UnaryExecNode { + + private val pandasFunction = func.asInstanceOf[PythonUDF].func + + override def producedAttributes: AttributeSet = AttributeSet(output) + + private val batchSize = conf.arrowMaxRecordsPerBatch + + override def outputPartitioning: Partitioning = child.outputPartitioning + + override protected def doExecute(): RDD[InternalRow] = { + child.execute().mapPartitionsInternal { inputIter => + // Single function with one struct. + val argOffsets = Array(Array(0)) + val chainedFunc = Seq(ChainedPythonFunctions(Seq(pandasFunction))) + val sessionLocalTimeZone = conf.sessionLocalTimeZone + val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) + val outputTypes = child.schema + + // Here we wrap it via another row so that Python sides understand it + // as a DataFrame. + val wrappedIter = inputIter.map(InternalRow(_)) + + // DO NOT use iter.grouped(). See BatchIterator. + val batchIter = + if (batchSize > 0) new BatchIterator(wrappedIter, batchSize) else Iterator(wrappedIter) + + val context = TaskContext.get() + + val columnarBatchIter = new ArrowPythonRunner( + chainedFunc, + PythonEvalType.SQL_MAP_PANDAS_ITER_UDF, + argOffsets, + StructType(StructField("struct", outputTypes) :: Nil), + sessionLocalTimeZone, + pythonRunnerConf).compute(batchIter, context.partitionId(), context) + + val unsafeProj = UnsafeProjection.create(output, output) + + columnarBatchIter.flatMap { batch => + // Scalar Iterator UDF returns a StructType column in ColumnarBatch, select + // the children here + val structVector = batch.column(0).asInstanceOf[ArrowColumnVector] + val outputVectors = output.indices.map(structVector.getChild) + val flattenedBatch = new ColumnarBatch(outputVectors.toArray) + flattenedBatch.setNumRows(batch.numRows()) + flattenedBatch.rowIterator.asScala + }.map(unsafeProj) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala index cc61faa7e7051..752d271c4cc35 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/PythonUDFRunner.scala @@ -59,9 +59,9 @@ class PythonUDFRunner( startTime: Long, env: SparkEnv, worker: Socket, - released: AtomicBoolean, + releasedOrClosed: AtomicBoolean, context: TaskContext): Iterator[Array[Byte]] = { - new ReaderIterator(stream, writerThread, startTime, env, worker, released, context) { + new ReaderIterator(stream, writerThread, startTime, env, worker, releasedOrClosed, context) { protected override def read(): Array[Byte] = { if (writerThread.exception.isDefined) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala index d2820ff335ecf..eb12641f548ad 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/RowQueue.scala @@ -23,7 +23,7 @@ import com.google.common.io.Closeables import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.io.NioBufferedFileInputStream -import org.apache.spark.memory.{MemoryConsumer, TaskMemoryManager} +import org.apache.spark.memory.{MemoryConsumer, SparkOutOfMemoryError, TaskMemoryManager} import org.apache.spark.serializer.SerializerManager import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.unsafe.Platform @@ -226,7 +226,7 @@ private[python] case class HybridRowQueue( val page = try { allocatePage(required) } catch { - case _: OutOfMemoryError => + case _: SparkOutOfMemoryError => null } val buffer = if (page != null) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala index f4c2d02ee9420..41521bfae1add 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/UserDefinedPythonFunction.scala @@ -32,7 +32,7 @@ case class UserDefinedPythonFunction( pythonEvalType: Int, udfDeterministic: Boolean) { - def builder(e: Seq[Expression]): PythonUDF = { + def builder(e: Seq[Expression]): Expression = { PythonUDF(name, func, dataType, e, pythonEvalType, udfDeterministic) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala index 27bed1137e5b3..01ce07b133ffd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/python/WindowInPandasExec.scala @@ -27,24 +27,71 @@ import org.apache.spark.api.python.{ChainedPythonFunctions, PythonEvalType} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.execution.{GroupedIterator, SparkPlan, UnaryExecNode} -import org.apache.spark.sql.execution.arrow.ArrowUtils -import org.apache.spark.sql.types.{DataType, StructField, StructType} +import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, Distribution, Partitioning} +import org.apache.spark.sql.execution.{ExternalAppendOnlyUnsafeRowArray, SparkPlan} +import org.apache.spark.sql.execution.window._ +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils +/** + * This class calculates and outputs windowed aggregates over the rows in a single partition. + * + * This is similar to [[WindowExec]]. The main difference is that this node does not compute + * any window aggregation values. Instead, it computes the lower and upper bound for each window + * (i.e. window bounds) and pass the data and indices to Python worker to do the actual window + * aggregation. + * + * It currently materializes all data associated with the same partition key and passes them to + * Python worker. This is not strictly necessary for sliding windows and can be improved (by + * possibly slicing data into overlapping chunks and stitching them together). + * + * This class groups window expressions by their window boundaries so that window expressions + * with the same window boundaries can share the same window bounds. The window bounds are + * prepended to the data passed to the python worker. + * + * For example, if we have: + * avg(v) over specifiedwindowframe(RowFrame, -5, 5), + * avg(v) over specifiedwindowframe(RowFrame, UnboundedPreceding, UnboundedFollowing), + * avg(v) over specifiedwindowframe(RowFrame, -3, 3), + * max(v) over specifiedwindowframe(RowFrame, -3, 3) + * + * The python input will look like: + * (lower_bound_w1, upper_bound_w1, lower_bound_w3, upper_bound_w3, v) + * + * where w1 is specifiedwindowframe(RowFrame, -5, 5) + * w2 is specifiedwindowframe(RowFrame, UnboundedPreceding, UnboundedFollowing) + * w3 is specifiedwindowframe(RowFrame, -3, 3) + * + * Note that w2 doesn't have bound indices in the python input because it's unbounded window + * so it's bound indices will always be the same. + * + * Bounded window and Unbounded window are evaluated differently in Python worker: + * (1) Bounded window takes the window bound indices in addition to the input columns. + * Unbounded window takes only input columns. + * (2) Bounded window evaluates the udf once per input row. + * Unbounded window evaluates the udf once per window partition. + * This is controlled by Python runner conf "pandas_window_bound_types" + * + * The logic to compute window bounds is delegated to [[WindowFunctionFrame]] and shared with + * [[WindowExec]] + * + * Note this doesn't support partial aggregation and all aggregation is computed from the entire + * window. + */ case class WindowInPandasExec( windowExpression: Seq[NamedExpression], partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], - child: SparkPlan) extends UnaryExecNode { + child: SparkPlan) + extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) { override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) override def requiredChildDistribution: Seq[Distribution] = { if (partitionSpec.isEmpty) { - // Only show warning when the number of bytes is larger than 100 MB? + // Only show warning when the number of bytes is larger than 100 MiB? logWarning("No Partition Defined for Window operation! Moving all data to a single " + "partition, this can cause serious performance degradation.") AllTuples :: Nil @@ -60,6 +107,26 @@ case class WindowInPandasExec( override def outputPartitioning: Partitioning = child.outputPartitioning + /** + * Helper functions and data structures for window bounds + * + * It contains: + * (1) Total number of window bound indices in the python input row + * (2) Function from frame index to its lower bound column index in the python input row + * (3) Function from frame index to its upper bound column index in the python input row + * (4) Seq from frame index to its window bound type + */ + private type WindowBoundHelpers = (Int, Int => Int, Int => Int, Seq[WindowBoundType]) + + /** + * Enum for window bound types. Used only inside this class. + */ + private sealed case class WindowBoundType(value: String) + private object UnboundedWindow extends WindowBoundType("unbounded") + private object BoundedWindow extends WindowBoundType("bounded") + + private val windowBoundTypeConf = "pandas_window_bound_types" + private def collectFunctions(udf: PythonUDF): (ChainedPythonFunctions, Seq[Expression]) = { udf.children match { case Seq(u: PythonUDF) => @@ -73,68 +140,150 @@ case class WindowInPandasExec( } /** - * Create the resulting projection. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param expressions unbound ordered function expressions. - * @return the final resulting projection. + * See [[WindowBoundHelpers]] for details. */ - private[this] def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { - val references = expressions.zipWithIndex.map { case (e, i) => - // Results of window expressions will be on the right side of child's output - BoundReference(child.output.size + i, e.dataType, e.nullable) + private def computeWindowBoundHelpers( + factories: Seq[InternalRow => WindowFunctionFrame] + ): WindowBoundHelpers = { + val functionFrames = factories.map(_(EmptyRow)) + + val windowBoundTypes = functionFrames.map { + case _: UnboundedWindowFunctionFrame => UnboundedWindow + case _: UnboundedFollowingWindowFunctionFrame | + _: SlidingWindowFunctionFrame | + _: UnboundedPrecedingWindowFunctionFrame => BoundedWindow + // It should be impossible to get other types of window function frame here + case frame => throw new RuntimeException(s"Unexpected window function frame $frame.") } - val unboundToRefMap = expressions.zip(references).toMap - val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) - UnsafeProjection.create( - child.output ++ patchedWindowExpression, - child.output) + + val requiredIndices = functionFrames.map { + case _: UnboundedWindowFunctionFrame => 0 + case _ => 2 + } + + val upperBoundIndices = requiredIndices.scan(0)(_ + _).tail + + val boundIndices = requiredIndices.zip(upperBoundIndices).map { case (num, upperBoundIndex) => + if (num == 0) { + // Sentinel values for unbounded window + (-1, -1) + } else { + (upperBoundIndex - 2, upperBoundIndex - 1) + } + } + + def lowerBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._1 + def upperBoundIndex(frameIndex: Int) = boundIndices(frameIndex)._2 + + (requiredIndices.sum, lowerBoundIndex, upperBoundIndex, windowBoundTypes) } protected override def doExecute(): RDD[InternalRow] = { - val inputRDD = child.execute() + // Unwrap the expressions and factories from the map. + val expressionsWithFrameIndex = + windowFrameExpressionFactoryPairs.map(_._1).zipWithIndex.flatMap { + case (buffer, frameIndex) => buffer.map(expr => (expr, frameIndex)) + } + + val expressions = expressionsWithFrameIndex.map(_._1) + val expressionIndexToFrameIndex = + expressionsWithFrameIndex.map(_._2).zipWithIndex.map(_.swap).toMap + + val factories = windowFrameExpressionFactoryPairs.map(_._2).toArray + // Helper functions + val (numBoundIndices, lowerBoundIndex, upperBoundIndex, frameWindowBoundTypes) = + computeWindowBoundHelpers(factories) + val isBounded = { frameIndex: Int => lowerBoundIndex(frameIndex) >= 0 } + val numFrames = factories.length + + val inMemoryThreshold = conf.windowExecBufferInMemoryThreshold + val spillThreshold = conf.windowExecBufferSpillThreshold val sessionLocalTimeZone = conf.sessionLocalTimeZone - val pythonRunnerConf = ArrowUtils.getPythonRunnerConfMap(conf) // Extract window expressions and window functions - val expressions = windowExpression.flatMap(_.collect { case e: WindowExpression => e }) - - val udfExpressions = expressions.map(_.windowFunction.asInstanceOf[PythonUDF]) + val windowExpressions = expressions.flatMap(_.collect { case e: WindowExpression => e }) + val udfExpressions = windowExpressions.map(_.windowFunction.asInstanceOf[PythonUDF]) + // We shouldn't be chaining anything here. + // All chained python functions should only contain one function. val (pyFuncs, inputs) = udfExpressions.map(collectFunctions).unzip + require(pyFuncs.length == expressions.length) + + val udfWindowBoundTypes = pyFuncs.indices.map(i => + frameWindowBoundTypes(expressionIndexToFrameIndex(i))) + val pythonRunnerConf: Map[String, String] = (ArrowUtils.getPythonRunnerConfMap(conf) + + (windowBoundTypeConf -> udfWindowBoundTypes.map(_.value).mkString(","))) // Filter child output attributes down to only those that are UDF inputs. - // Also eliminate duplicate UDF inputs. - val allInputs = new ArrayBuffer[Expression] - val dataTypes = new ArrayBuffer[DataType] + // Also eliminate duplicate UDF inputs. This is similar to how other Python UDF node + // handles UDF inputs. + val dataInputs = new ArrayBuffer[Expression] + val dataInputTypes = new ArrayBuffer[DataType] val argOffsets = inputs.map { input => input.map { e => - if (allInputs.exists(_.semanticEquals(e))) { - allInputs.indexWhere(_.semanticEquals(e)) + if (dataInputs.exists(_.semanticEquals(e))) { + dataInputs.indexWhere(_.semanticEquals(e)) } else { - allInputs += e - dataTypes += e.dataType - allInputs.length - 1 + dataInputs += e + dataInputTypes += e.dataType + dataInputs.length - 1 } }.toArray }.toArray - // Schema of input rows to the python runner - val windowInputSchema = StructType(dataTypes.zipWithIndex.map { case (dt, i) => - StructField(s"_$i", dt) - }) + // In addition to UDF inputs, we will prepend window bounds for each UDFs. + // For bounded windows, we prepend lower bound and upper bound. For unbounded windows, + // we no not add window bounds. (strictly speaking, we only need to lower or upper bound + // if the window is bounded only on one side, this can be improved in the future) - inputRDD.mapPartitionsInternal { iter => - val context = TaskContext.get() + // Setting window bounds for each window frames. Each window frame has different bounds so + // each has its own window bound columns. + val windowBoundsInput = factories.indices.flatMap { frameIndex => + if (isBounded(frameIndex)) { + Seq( + BoundReference(lowerBoundIndex(frameIndex), IntegerType, nullable = false), + BoundReference(upperBoundIndex(frameIndex), IntegerType, nullable = false) + ) + } else { + Seq.empty + } + } - val grouped = if (partitionSpec.isEmpty) { - // Use an empty unsafe row as a place holder for the grouping key - Iterator((new UnsafeRow(), iter)) + // Setting the window bounds argOffset for each UDF. For UDFs with bounded window, argOffset + // for the UDF is (lowerBoundOffet, upperBoundOffset, inputOffset1, inputOffset2, ...) + // For UDFs with unbounded window, argOffset is (inputOffset1, inputOffset2, ...) + pyFuncs.indices.foreach { exprIndex => + val frameIndex = expressionIndexToFrameIndex(exprIndex) + if (isBounded(frameIndex)) { + argOffsets(exprIndex) = + Array(lowerBoundIndex(frameIndex), upperBoundIndex(frameIndex)) ++ + argOffsets(exprIndex).map(_ + windowBoundsInput.length) } else { - GroupedIterator(iter, partitionSpec, child.output) + argOffsets(exprIndex) = argOffsets(exprIndex).map(_ + windowBoundsInput.length) } + } + + val allInputs = windowBoundsInput ++ dataInputs + val allInputTypes = allInputs.map(_.dataType) + + // Start processing. + child.execute().mapPartitions { iter => + val context = TaskContext.get() + + // Get all relevant projections. + val resultProj = createResultProjection(expressions) + val pythonInputProj = UnsafeProjection.create( + allInputs, + windowBoundsInput.map(ref => + AttributeReference(s"i_${ref.ordinal}", ref.dataType)()) ++ child.output + ) + val pythonInputSchema = StructType( + allInputTypes.zipWithIndex.map { case (dt, i) => + StructField(s"_$i", dt) + } + ) + val grouping = UnsafeProjection.create(partitionSpec, child.output) // The queue used to buffer input rows so we can drain it to // combine input with output from Python. @@ -144,11 +293,94 @@ case class WindowInPandasExec( queue.close() } - val inputProj = UnsafeProjection.create(allInputs, child.output) - val pythonInput = grouped.map { case (_, rows) => - rows.map { row => - queue.add(row.asInstanceOf[UnsafeRow]) - inputProj(row) + val stream = iter.map { row => + queue.add(row.asInstanceOf[UnsafeRow]) + row + } + + val pythonInput = new Iterator[Iterator[UnsafeRow]] { + + // Manage the stream and the grouping. + var nextRow: UnsafeRow = null + var nextGroup: UnsafeRow = null + var nextRowAvailable: Boolean = false + private[this] def fetchNextRow() { + nextRowAvailable = stream.hasNext + if (nextRowAvailable) { + nextRow = stream.next().asInstanceOf[UnsafeRow] + nextGroup = grouping(nextRow) + } else { + nextRow = null + nextGroup = null + } + } + fetchNextRow() + + // Manage the current partition. + val buffer: ExternalAppendOnlyUnsafeRowArray = + new ExternalAppendOnlyUnsafeRowArray(inMemoryThreshold, spillThreshold) + var bufferIterator: Iterator[UnsafeRow] = _ + + val indexRow = new SpecificInternalRow(Array.fill(numBoundIndices)(IntegerType)) + + val frames = factories.map(_(indexRow)) + + private[this] def fetchNextPartition() { + // Collect all the rows in the current partition. + // Before we start to fetch new input rows, make a copy of nextGroup. + val currentGroup = nextGroup.copy() + + // clear last partition + buffer.clear() + + while (nextRowAvailable && nextGroup == currentGroup) { + buffer.add(nextRow) + fetchNextRow() + } + + // Setup the frames. + var i = 0 + while (i < numFrames) { + frames(i).prepare(buffer) + i += 1 + } + + // Setup iteration + rowIndex = 0 + bufferIterator = buffer.generateIterator() + } + + // Iteration + var rowIndex = 0 + + override final def hasNext: Boolean = + (bufferIterator != null && bufferIterator.hasNext) || nextRowAvailable + + override final def next(): Iterator[UnsafeRow] = { + // Load the next partition if we need to. + if ((bufferIterator == null || !bufferIterator.hasNext) && nextRowAvailable) { + fetchNextPartition() + } + + val join = new JoinedRow + + bufferIterator.zipWithIndex.map { + case (current, index) => + var frameIndex = 0 + while (frameIndex < numFrames) { + frames(frameIndex).write(index, current) + // If the window is unbounded we don't need to write out window bounds. + if (isBounded(frameIndex)) { + indexRow.setInt( + lowerBoundIndex(frameIndex), frames(frameIndex).currentLowerBound()) + indexRow.setInt( + upperBoundIndex(frameIndex), frames(frameIndex).currentUpperBound()) + } + frameIndex += 1 + } + + pythonInputProj(join(indexRow, current)) + } } } @@ -156,12 +388,11 @@ case class WindowInPandasExec( pyFuncs, PythonEvalType.SQL_WINDOW_AGG_PANDAS_UDF, argOffsets, - windowInputSchema, + pythonInputSchema, sessionLocalTimeZone, pythonRunnerConf).compute(pythonInput, context.partitionId(), context) val joined = new JoinedRow - val resultProj = createResultProjection(expressions) windowFunctionResult.flatMap(_.rowIterator.asScala).map { windowOutput => val leftRow = queue.remove() diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.scala new file mode 100644 index 0000000000000..0fe2b628fa38b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/ArrowRRunner.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.sql.execution.r + +import java.io._ +import java.nio.channels.Channels + +import scala.collection.JavaConverters._ + +import org.apache.arrow.vector.VectorSchemaRoot +import org.apache.arrow.vector.ipc.{ArrowStreamReader, ArrowStreamWriter} +import org.apache.arrow.vector.util.ByteArrayReadableSeekableByteChannel + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.api.r._ +import org.apache.spark.api.r.SpecialLengths +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.arrow.ArrowWriter +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} +import org.apache.spark.util.Utils + + +/** + * Similar to `ArrowPythonRunner`, but exchange data with R worker via Arrow stream. + */ +class ArrowRRunner( + func: Array[Byte], + packageNames: Array[Byte], + broadcastVars: Array[Broadcast[Object]], + schema: StructType, + timeZoneId: String, + mode: Int) + extends BaseRRunner[Iterator[InternalRow], ColumnarBatch]( + func, + "arrow", + "arrow", + packageNames, + broadcastVars, + numPartitions = -1, + isDataFrame = true, + schema.fieldNames, + mode) { + + protected def bufferedWrite( + dataOut: DataOutputStream)(writeFunc: ByteArrayOutputStream => Unit): Unit = { + val out = new ByteArrayOutputStream() + writeFunc(out) + + // Currently, there looks no way to read batch by batch by socket connection in R side, + // See ARROW-4512. Therefore, it writes the whole Arrow streaming-formatted binary at + // once for now. + val data = out.toByteArray + dataOut.writeInt(data.length) + dataOut.write(data) + } + + protected override def newWriterThread( + output: OutputStream, + inputIterator: Iterator[Iterator[InternalRow]], + partitionIndex: Int): WriterThread = { + new WriterThread(output, inputIterator, partitionIndex) { + + /** + * Writes input data to the stream connected to the R worker. + */ + override protected def writeIteratorToStream(dataOut: DataOutputStream): Unit = { + if (inputIterator.hasNext) { + val arrowSchema = ArrowUtils.toArrowSchema(schema, timeZoneId) + val allocator = ArrowUtils.rootAllocator.newChildAllocator( + "stdout writer for R", 0, Long.MaxValue) + val root = VectorSchemaRoot.create(arrowSchema, allocator) + + bufferedWrite(dataOut) { out => + Utils.tryWithSafeFinally { + val arrowWriter = ArrowWriter.create(root) + val writer = new ArrowStreamWriter(root, null, Channels.newChannel(out)) + writer.start() + + while (inputIterator.hasNext) { + val nextBatch: Iterator[InternalRow] = inputIterator.next() + + while (nextBatch.hasNext) { + arrowWriter.write(nextBatch.next()) + } + + arrowWriter.finish() + writer.writeBatch() + arrowWriter.reset() + } + writer.end() + } { + // Don't close root and allocator in TaskCompletionListener to prevent + // a race condition. See `ArrowPythonRunner`. + root.close() + allocator.close() + } + } + } + } + } + } + + protected override def newReaderIterator( + dataStream: DataInputStream, errThread: BufferedStreamThread): ReaderIterator = { + new ReaderIterator(dataStream, errThread) { + private val allocator = ArrowUtils.rootAllocator.newChildAllocator( + "stdin reader for R", 0, Long.MaxValue) + + private var reader: ArrowStreamReader = _ + private var root: VectorSchemaRoot = _ + private var vectors: Array[ColumnVector] = _ + + TaskContext.get().addTaskCompletionListener[Unit] { _ => + if (reader != null) { + reader.close(false) + } + allocator.close() + } + + private var batchLoaded = true + + protected override def read(): ColumnarBatch = try { + if (reader != null && batchLoaded) { + batchLoaded = reader.loadNextBatch() + if (batchLoaded) { + val batch = new ColumnarBatch(vectors) + batch.setNumRows(root.getRowCount) + batch + } else { + reader.close(false) + allocator.close() + // Should read timing data after this. + read() + } + } else { + dataStream.readInt() match { + case SpecialLengths.TIMING_DATA => + // Timing data from R worker + val boot = dataStream.readDouble - bootTime + val init = dataStream.readDouble + val broadcast = dataStream.readDouble + val input = dataStream.readDouble + val compute = dataStream.readDouble + val output = dataStream.readDouble + logInfo( + ("Times: boot = %.3f s, init = %.3f s, broadcast = %.3f s, " + + "read-input = %.3f s, compute = %.3f s, write-output = %.3f s, " + + "total = %.3f s").format( + boot, + init, + broadcast, + input, + compute, + output, + boot + init + broadcast + input + compute + output)) + read() + case length if length > 0 => + // Likewise, there looks no way to send each batch in streaming format via socket + // connection. See ARROW-4512. + // So, it reads the whole Arrow streaming-formatted binary at once for now. + val buffer = new Array[Byte](length) + dataStream.readFully(buffer) + val in = new ByteArrayReadableSeekableByteChannel(buffer) + reader = new ArrowStreamReader(in, allocator) + root = reader.getVectorSchemaRoot + vectors = root.getFieldVectors.asScala.map { vector => + new ArrowColumnVector(vector) + }.toArray[ColumnVector] + read() + case length if length == 0 => + // End of stream + eos = true + null + } + } + } catch { + case eof: EOFException => + throw new SparkException( + "R worker exited unexpectedly (crashed)\n " + errThread.getLines(), eof) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala index a62016dac1229..a3a40886f5f3f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/r/MapPartitionsRWrapper.scala @@ -51,7 +51,7 @@ case class MapPartitionsRWrapper( SerializationFormats.BYTE } - val runner = new RRunner[Array[Byte]]( + val runner = new RRunner[Any, Array[Byte]]( func, deserializer, serializer, packageNames, broadcastVars, isDataFrame = true, colNames = colNames, mode = RRunnerModes.DATAFRAME_DAPPLY) // Partition index is ignored. Dataset has no support for mapPartitionsWithIndex. 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 86f6307254332..420faa6f24734 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 @@ -69,7 +69,7 @@ object FrequentItems extends Logging { /** * Finding frequent items for columns, possibly with false positives. Using the * frequent element count algorithm described in - * here, proposed by Karp, Schenker, + * here, proposed by Karp, Schenker, * and Papadimitriou. * The `support` should be greater than 1e-4. * For Internal use only. 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 bea652cc33076..a6c9c2972df6c 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 @@ -45,7 +45,7 @@ object StatFunctions extends Logging { * * This method implements a variation of the Greenwald-Khanna algorithm (with some speed * optimizations). - * The algorithm was first present in + * The algorithm was first present in * Space-efficient Online Computation of Quantile Summaries by Greenwald and Khanna. * * @param df the dataframe @@ -203,7 +203,7 @@ object StatFunctions extends Logging { // row.get(0) is column 1 // row.get(1) is column 2 // row.get(2) is the frequency - val columnIndex = distinctCol2.get(cleanElement(row.get(1))).get + val columnIndex = distinctCol2(cleanElement(row.get(1))) countsRow.setLong(columnIndex + 1, row.getLong(2)) } // the value of col1 is the first value, the rest are the counts diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala index 606ba250ad9d2..fe6362d878d53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CheckpointFileManager.scala @@ -56,7 +56,7 @@ trait CheckpointFileManager { * @param overwriteIfPossible If true, then the implementations must do a best-effort attempt to * overwrite the file if it already exists. It should not throw * any exception if the file exists. However, if false, then the - * implementation must not overwrite if the file alraedy exists and + * implementation must not overwrite if the file already exists and * must throw `FileAlreadyExistsException` in that case. */ def createAtomic(path: Path, overwriteIfPossible: Boolean): CancellableFSDataOutputStream @@ -69,7 +69,7 @@ trait CheckpointFileManager { /** List all the files in a path. */ def list(path: Path): Array[FileStatus] = { - list(path, new PathFilter { override def accept(path: Path): Boolean = true }) + list(path, (_: Path) => true) } /** Make directory at the give path and all its parent directories as needed. */ @@ -103,7 +103,7 @@ object CheckpointFileManager extends Logging { * @param overwriteIfPossible If true, then the implementations must do a best-effort attempt to * overwrite the file if it already exists. It should not throw * any exception if the file exists. However, if false, then the - * implementation must not overwrite if the file alraedy exists and + * implementation must not overwrite if the file already exists and * must throw `FileAlreadyExistsException` in that case. */ def renameTempFile(srcPath: Path, dstPath: Path, overwriteIfPossible: Boolean): Unit @@ -236,14 +236,12 @@ class FileSystemBasedCheckpointFileManager(path: Path, hadoopConf: Configuration fs.open(path) } - override def exists(path: Path): Boolean = { - try - return fs.getFileStatus(path) != null - catch { - case e: FileNotFoundException => - return false + override def exists(path: Path): Boolean = + try { + fs.getFileStatus(path) != null + } catch { + case _: FileNotFoundException => false } - } override def renameTempFile(srcPath: Path, dstPath: Path, overwriteIfPossible: Boolean): Unit = { if (!overwriteIfPossible && fs.exists(dstPath)) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala index 0063318db332d..ad7c59bbd9f52 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CommitLog.scala @@ -56,7 +56,7 @@ class CommitLog(sparkSession: SparkSession, path: String) if (!lines.hasNext) { throw new IllegalStateException("Incomplete log file in the offset commit log") } - parseVersion(lines.next.trim, VERSION) + validateVersion(lines.next.trim, VERSION) val metadataJson = if (lines.hasNext) lines.next else EMPTY_JSON CommitMetadata(metadataJson) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala index 77bc0ba5548dd..905bce4d614e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLog.scala @@ -23,7 +23,7 @@ import java.nio.charset.StandardCharsets.UTF_8 import scala.io.{Source => IOSource} import scala.reflect.ClassTag -import org.apache.hadoop.fs.{Path, PathFilter} +import org.apache.hadoop.fs.Path import org.json4s.NoTypeHints import org.json4s.jackson.Serialization @@ -85,8 +85,7 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( val compactibleBatchIds = fileManager.list(metadataPath, batchFilesFilter) .filter(f => f.getPath.toString.endsWith(CompactibleFileStreamLog.COMPACT_FILE_SUFFIX)) .map(f => pathToBatchId(f.getPath)) - .sorted - .reverse + .sorted(Ordering.Long.reverse) // Case 1 var interval = defaultCompactInterval @@ -146,7 +145,7 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( if (!lines.hasNext) { throw new IllegalStateException("Incomplete log file") } - val version = parseVersion(lines.next(), metadataLogVersion) + validateVersion(lines.next(), metadataLogVersion) lines.map(Serialization.read[T]).toArray } @@ -163,19 +162,29 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( batchAdded } + /** + * CompactibleFileStreamLog maintains logs by itself, and manual purging might break internal + * state, specifically which latest compaction batch is purged. + * + * To simplify the situation, this method just throws UnsupportedOperationException regardless + * of given parameter, and let CompactibleFileStreamLog handles purging by itself. + */ + override def purge(thresholdBatchId: Long): Unit = throw new UnsupportedOperationException( + s"Cannot purge as it might break internal state.") + /** * Compacts all logs before `batchId` plus the provided `logs`, and writes them into the * corresponding `batchId` file. It will delete expired files as well if enabled. */ private def compact(batchId: Long, logs: Array[T]): Boolean = { val validBatches = getValidBatchesBeforeCompactionBatch(batchId, compactInterval) - val allLogs = validBatches.map { id => + val allLogs = validBatches.flatMap { id => super.get(id).getOrElse { throw new IllegalStateException( s"${batchIdToPath(id)} doesn't exist when compacting batch $batchId " + s"(compactInterval: $compactInterval)") } - }.flatten ++ logs + } ++ logs // Return false as there is another writer. super.add(batchId, compactLogs(allLogs).toArray) } @@ -192,13 +201,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( if (latestId >= 0) { try { val logs = - getAllValidBatches(latestId, compactInterval).map { id => + getAllValidBatches(latestId, compactInterval).flatMap { id => super.get(id).getOrElse { throw new IllegalStateException( s"${batchIdToPath(id)} doesn't exist " + s"(latestId: $latestId, compactInterval: $compactInterval)") } - }.flatten + } return compactLogs(logs).toArray } catch { case e: IOException => @@ -240,15 +249,13 @@ abstract class CompactibleFileStreamLog[T <: AnyRef : ClassTag]( s"min compaction batch id to delete = $minCompactionBatchId") val expiredTime = System.currentTimeMillis() - fileCleanupDelayMs - fileManager.list(metadataPath, new PathFilter { - override def accept(path: Path): Boolean = { - try { - val batchId = getBatchIdFromFileName(path.getName) - batchId < minCompactionBatchId - } catch { - case _: NumberFormatException => - false - } + fileManager.list(metadataPath, (path: Path) => { + try { + val batchId = getBatchIdFromFileName(path.getName) + batchId < minCompactionBatchId + } catch { + case _: NumberFormatException => + false } }).foreach { f => if (f.getModificationTime <= expiredTime) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala index b161651c4e6a3..6d1131e6939db 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/EventTimeWatermarkExec.scala @@ -21,6 +21,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeProjection} import org.apache.spark.sql.catalyst.plans.logical.EventTimeWatermark +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.types.MetadataBuilder import org.apache.spark.unsafe.types.CalendarInterval @@ -36,10 +37,19 @@ case class EventTimeStats(var max: Long, var min: Long, var avg: Double, var cou } def merge(that: EventTimeStats): Unit = { - this.max = math.max(this.max, that.max) - this.min = math.min(this.min, that.min) - this.count += that.count - this.avg += (that.avg - this.avg) * that.count / this.count + if (that.count == 0) { + // no-op + } else if (this.count == 0) { + this.max = that.max + this.min = that.min + this.count = that.count + this.avg = that.avg + } else { + this.max = math.max(this.max, that.max) + this.min = math.min(this.min, that.min) + this.count += that.count + this.avg += (that.avg - this.avg) * that.count / this.count + } } } @@ -90,7 +100,7 @@ case class EventTimeWatermarkExec( child.execute().mapPartitions { iter => val getEventTime = UnsafeProjection.create(eventTime :: Nil, child.output) iter.map { row => - eventTimeStats.add(getEventTime(row).getLong(0) / 1000) + eventTimeStats.add(getEventTime(row).getLong(0) / MICROS_PER_MILLIS) row } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala index b3d12f67b5d63..b679f163fc561 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSink.scala @@ -20,13 +20,15 @@ package org.apache.spark.sql.execution.streaming import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.SparkException import org.apache.spark.internal.Logging import org.apache.spark.internal.io.FileCommitProtocol import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution.datasources.{BasicWriteJobStatsTracker, FileFormat, FileFormatWriter} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.SerializableConfiguration object FileStreamSink extends Logging { @@ -37,23 +39,54 @@ object FileStreamSink extends Logging { * Returns true if there is a single path that has a metadata log indicating which files should * be read. */ - def hasMetadata(path: Seq[String], hadoopConf: Configuration): Boolean = { + def hasMetadata(path: Seq[String], hadoopConf: Configuration, sqlConf: SQLConf): Boolean = { path match { case Seq(singlePath) => + val hdfsPath = new Path(singlePath) + val fs = hdfsPath.getFileSystem(hadoopConf) + if (fs.isDirectory(hdfsPath)) { + val metadataPath = new Path(hdfsPath, metadataDir) + checkEscapedMetadataPath(fs, metadataPath, sqlConf) + fs.exists(metadataPath) + } else { + false + } + case _ => false + } + } + + def checkEscapedMetadataPath(fs: FileSystem, metadataPath: Path, sqlConf: SQLConf): Unit = { + if (sqlConf.getConf(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED) + && StreamExecution.containsSpecialCharsInPath(metadataPath)) { + val legacyMetadataPath = new Path(metadataPath.toUri.toString) + val legacyMetadataPathExists = try { - val hdfsPath = new Path(singlePath) - val fs = hdfsPath.getFileSystem(hadoopConf) - if (fs.isDirectory(hdfsPath)) { - fs.exists(new Path(hdfsPath, metadataDir)) - } else { - false - } + fs.exists(legacyMetadataPath) } catch { case NonFatal(e) => - logWarning(s"Error while looking for metadata directory.") + // We may not have access to this directory. Don't fail the query if that happens. + logWarning(e.getMessage, e) false } - case _ => false + if (legacyMetadataPathExists) { + throw new SparkException( + s"""Error: we detected a possible problem with the location of your "_spark_metadata" + |directory and you likely need to move it before restarting this query. + | + |Earlier version of Spark incorrectly escaped paths when writing out the + |"_spark_metadata" directory for structured streaming. While this was corrected in + |Spark 3.0, it appears that your query was started using an earlier version that + |incorrectly handled the "_spark_metadata" path. + | + |Correct "_spark_metadata" Directory: $metadataPath + |Incorrect "_spark_metadata" Directory: $legacyMetadataPath + | + |Please move the data from the incorrect directory to the correct one, delete the + |incorrect directory, and then restart this query. If you believe you are receiving + |this message in error, you can disable it with the SQL conf + |${SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED.key}.""" + .stripMargin) + } } } @@ -92,11 +125,16 @@ class FileStreamSink( partitionColumnNames: Seq[String], options: Map[String, String]) extends Sink with Logging { + private val hadoopConf = sparkSession.sessionState.newHadoopConf() private val basePath = new Path(path) - private val logPath = new Path(basePath, FileStreamSink.metadataDir) + private val logPath = { + val metadataDir = new Path(basePath, FileStreamSink.metadataDir) + val fs = metadataDir.getFileSystem(hadoopConf) + FileStreamSink.checkEscapedMetadataPath(fs, metadataDir, sparkSession.sessionState.conf) + metadataDir + } private val fileLog = - new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, logPath.toUri.toString) - private val hadoopConf = sparkSession.sessionState.newHadoopConf() + new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, logPath.toString) private def basicWriteJobStatsTracker: BasicWriteJobStatsTracker = { val serializableHadoopConf = new SerializableConfiguration(hadoopConf) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala index 103fa7ce9066d..67e26dc1a2dbc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSource.scala @@ -18,14 +18,14 @@ package org.apache.spark.sql.execution.streaming import java.net.URI - -import scala.collection.JavaConverters._ +import java.util.concurrent.TimeUnit._ import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Dataset, SparkSession} +import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap import org.apache.spark.sql.execution.datasources.{DataSource, InMemoryFileIndex, LogicalRelation} import org.apache.spark.sql.types.StructType @@ -196,7 +196,8 @@ class FileStreamSource( private def allFilesUsingMetadataLogFileIndex() = { // Note if `sourceHasMetadata` holds, then `qualifiedBasePath` is guaranteed to be a // non-glob path - new MetadataLogFileIndex(sparkSession, qualifiedBasePath, None).allFiles() + new MetadataLogFileIndex(sparkSession, qualifiedBasePath, + CaseInsensitiveMap(options), None).allFiles() } /** @@ -208,7 +209,7 @@ class FileStreamSource( var allFiles: Seq[FileStatus] = null sourceHasMetadata match { case None => - if (FileStreamSink.hasMetadata(Seq(path), hadoopConf)) { + if (FileStreamSink.hasMetadata(Seq(path), hadoopConf, sparkSession.sessionState.conf)) { sourceHasMetadata = Some(true) allFiles = allFilesUsingMetadataLogFileIndex() } else { @@ -220,7 +221,7 @@ class FileStreamSource( // double check whether source has metadata, preventing the extreme corner case that // metadata log and data files are only generated after the previous // `FileStreamSink.hasMetadata` check - if (FileStreamSink.hasMetadata(Seq(path), hadoopConf)) { + if (FileStreamSink.hasMetadata(Seq(path), hadoopConf, sparkSession.sessionState.conf)) { sourceHasMetadata = Some(true) allFiles = allFilesUsingMetadataLogFileIndex() } else { @@ -237,7 +238,7 @@ class FileStreamSource( (status.getPath.toUri.toString, status.getModificationTime) } val endTime = System.nanoTime - val listingTimeMs = (endTime.toDouble - startTime) / 1000000 + val listingTimeMs = NANOSECONDS.toMillis(endTime - startTime) if (listingTimeMs > 2000) { // Output a warning when listing files uses more than 2 seconds. logWarning(s"Listed ${files.size} file(s) in $listingTimeMs ms") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala index 8628471fdb925..7b2ea9627a98e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FileStreamSourceLog.scala @@ -117,7 +117,9 @@ class FileStreamSourceLog( val batches = (existedBatches ++ retrievedBatches).map(i => i._1 -> i._2.get).toArray.sortBy(_._1) - HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + if (startBatchId <= endBatchId) { + HDFSMetadataLog.verifyBatchIds(batches.map(_._1), startId, endId) + } batches } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala index bfe7d00f56048..fe91d24912222 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/FlatMapGroupsWithStateExec.scala @@ -28,14 +28,14 @@ import org.apache.spark.sql.streaming.{GroupStateTimeout, OutputMode} import org.apache.spark.util.CompletionIterator /** - * Physical operator for executing `FlatMapGroupsWithState.` + * Physical operator for executing `FlatMapGroupsWithState` * * @param func function called on each group * @param keyDeserializer used to extract the key object for each group. * @param valueDeserializer used to extract the items in the iterator from an input row. * @param groupingAttributes used to group the data * @param dataAttributes used to read the data - * @param outputObjAttr used to define the output object + * @param outputObjAttr Defines the output object * @param stateEncoder used to serialize/deserialize state before calling `func` * @param outputMode the output mode of `func` * @param timeoutConf used to timeout groups that have not received data in a while @@ -111,28 +111,35 @@ case class FlatMapGroupsWithStateExec( indexOrdinal = None, sqlContext.sessionState, Some(sqlContext.streams.stateStoreCoordinator)) { case (store, iter) => + val allUpdatesTimeMs = longMetric("allUpdatesTimeMs") + val commitTimeMs = longMetric("commitTimeMs") + val processor = new InputProcessor(store) - // If timeout is based on event time, then filter late data based on watermark - val filteredIter = watermarkPredicateForData match { - case Some(predicate) if timeoutConf == EventTimeTimeout => - iter.filter(row => !predicate.eval(row)) - case _ => - iter + // variable `outputIterator` is no chance to be unassigned, so setting to null is safe + var outputIterator: Iterator[InternalRow] = null + allUpdatesTimeMs += timeTakenMs { + // If timeout is based on event time, then filter late data based on watermark + val filteredIter = watermarkPredicateForData match { + case Some(predicate) if timeoutConf == EventTimeTimeout => + iter.filter(row => !predicate.eval(row)) + case _ => + iter + } + // Generate a iterator that returns the rows grouped by the grouping function + // Note that this code ensures that the filtering for timeout occurs only after + // all the data has been processed. This is to ensure that the timeout information of all + // the keys with data is updated before they are processed for timeouts. + outputIterator = processor.processNewData(filteredIter) ++ + processor.processTimedOutState() } - // Generate a iterator that returns the rows grouped by the grouping function - // Note that this code ensures that the filtering for timeout occurs only after - // all the data has been processed. This is to ensure that the timeout information of all - // the keys with data is updated before they are processed for timeouts. - val outputIterator = - processor.processNewData(filteredIter) ++ processor.processTimedOutState() // Return an iterator of all the rows generated by all the keys, such that when fully // consumed, all the state updates will be committed by the state store - CompletionIterator[InternalRow, Iterator[InternalRow]]( - outputIterator, - { - store.commit() + CompletionIterator[InternalRow, Iterator[InternalRow]](outputIterator, { + commitTimeMs += timeTakenMs { + store.commit() + } setStoreMetrics(store) } ) @@ -147,7 +154,7 @@ case class FlatMapGroupsWithStateExec( ObjectOperator.deserializeRowToObject(keyDeserializer, groupingAttributes) private val getValueObj = ObjectOperator.deserializeRowToObject(valueDeserializer, dataAttributes) - private val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjAttr.dataType) + private val getOutputRow = ObjectOperator.wrapObjectToRow(outputObjectType) // Metrics private val numUpdatedStateRows = longMetric("numUpdatedStateRows") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala index 7f65e3ea9dd5e..dda9d41f630e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/GroupStateImpl.scala @@ -18,8 +18,7 @@ package org.apache.spark.sql.execution.streaming import java.sql.Date - -import org.apache.commons.lang3.StringUtils +import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.plans.logical.{EventTimeTimeout, ProcessingTimeTimeout} import org.apache.spark.sql.execution.streaming.GroupStateImpl._ @@ -160,25 +159,12 @@ private[sql] class GroupStateImpl[S] private( def getTimeoutTimestamp: Long = timeoutTimestamp private def parseDuration(duration: String): Long = { - if (StringUtils.isBlank(duration)) { - throw new IllegalArgumentException( - "Provided duration is null or blank.") - } - val intervalString = if (duration.startsWith("interval")) { - duration - } else { - "interval " + duration - } - val cal = CalendarInterval.fromString(intervalString) - if (cal == null) { - throw new IllegalArgumentException( - s"Provided duration ($duration) is not valid.") - } + val cal = CalendarInterval.fromCaseInsensitiveString(duration) if (cal.milliseconds < 0 || cal.months < 0) { throw new IllegalArgumentException(s"Provided duration ($duration) is not positive") } - val millisPerMonth = CalendarInterval.MICROS_PER_DAY / 1000 * 31 + val millisPerMonth = TimeUnit.MICROSECONDS.toMillis(CalendarInterval.MICROS_PER_DAY) * 31 cal.milliseconds + cal.months * millisPerMonth } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala index bd0a46115ceb0..5c9249fb16343 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLog.scala @@ -177,8 +177,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: override def getLatest(): Option[(Long, T)] = { val batchIds = fileManager.list(metadataPath, batchFilesFilter) .map(f => pathToBatchId(f.getPath)) - .sorted - .reverse + .sorted(Ordering.Long.reverse) for (batchId <- batchIds) { val batch = get(batchId) if (batch.isDefined) { @@ -232,7 +231,7 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: * exceeds `maxSupportedVersion`, or when `text` is malformed (such as "xyz", "v", "v-1", * "v123xyz" etc.) */ - private[sql] def parseVersion(text: String, maxSupportedVersion: Int): Int = { + private[sql] def validateVersion(text: String, maxSupportedVersion: Int): Int = { if (text.length > 0 && text(0) == 'v') { val version = try { @@ -262,7 +261,8 @@ class HDFSMetadataLog[T <: AnyRef : ClassTag](sparkSession: SparkSession, path: object HDFSMetadataLog { /** - * Verify if batchIds are continuous and between `startId` and `endId`. + * Verify if batchIds are continuous and between `startId` and `endId` (both inclusive and + * startId assumed to be <= endId). * * @param batchIds the sorted ids to verify. * @param startId the start id. If it's set, batchIds should start with this id. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala index fad287e28877d..af52af0d1d7e6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/IncrementalExecution.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SparkSession, Strategy} +import org.apache.spark.sql.catalyst.QueryPlanningTracker import org.apache.spark.sql.catalyst.expressions.{CurrentBatchTimestamp, ExpressionWithRandomSeed} import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.physical.{AllTuples, ClusteredDistribution, HashPartitioning, SinglePartition} @@ -41,6 +42,7 @@ class IncrementalExecution( logicalPlan: LogicalPlan, val outputMode: OutputMode, val checkpointLocation: String, + val queryId: UUID, val runId: UUID, val currentBatchId: Long, val offsetSeqMetadata: OffsetSeqMetadata) @@ -73,7 +75,8 @@ class IncrementalExecution( * Walk the optimized logical plan and replace CurrentBatchTimestamp * with the desired literal */ - override lazy val optimizedPlan: LogicalPlan = { + override + lazy val optimizedPlan: LogicalPlan = tracker.measurePhase(QueryPlanningTracker.OPTIMIZATION) { sparkSession.sessionState.optimizer.execute(withCachedData) transformAllExpressions { case ts @ CurrentBatchTimestamp(timestamp, _, _) => logInfo(s"Current batch timestamp = $timestamp") diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala index 3ff5b86ac45d6..a27898cb0c9fc 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/LongOffset.scala @@ -17,12 +17,10 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} - /** * A simple offset for sources that produce a single linear stream of data. */ -case class LongOffset(offset: Long) extends OffsetV2 { +case class LongOffset(offset: Long) extends Offset { override val json = offset.toString @@ -37,14 +35,4 @@ object LongOffset { * @return new LongOffset */ def apply(offset: SerializedOffset) : LongOffset = new LongOffset(offset.json.toLong) - - /** - * Convert generic Offset to LongOffset if possible. - * @return converted LongOffset - */ - def convert(offset: Offset): Option[LongOffset] = offset match { - case lo: LongOffset => Some(lo) - case so: SerializedOffset => Some(LongOffset(so)) - case _ => None - } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala index 92191c8b64b72..916bd2ddbc818 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/ManifestFileCommitProtocol.scala @@ -114,7 +114,10 @@ class ManifestFileCommitProtocol(jobId: String, path: String) } override def abortTask(taskContext: TaskAttemptContext): Unit = { - // Do nothing - // TODO: we can also try delete the addedFiles as a best-effort cleanup. + // best effort cleanup of incomplete files + if (addedFiles.nonEmpty) { + val fs = new Path(addedFiles.head).getFileSystem(taskContext.getConfiguration) + addedFiles.foreach { file => fs.delete(new Path(file), false) } + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala index 5cacdd070b735..6eaccfb6d9347 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/MetadataLogFileIndex.scala @@ -36,13 +36,20 @@ import org.apache.spark.sql.types.StructType class MetadataLogFileIndex( sparkSession: SparkSession, path: Path, + parameters: Map[String, String], userSpecifiedSchema: Option[StructType]) - extends PartitioningAwareFileIndex(sparkSession, Map.empty, userSpecifiedSchema) { + extends PartitioningAwareFileIndex(sparkSession, parameters, userSpecifiedSchema) { + + private val metadataDirectory = { + val metadataDir = new Path(path, FileStreamSink.metadataDir) + val fs = metadataDir.getFileSystem(sparkSession.sessionState.newHadoopConf()) + FileStreamSink.checkEscapedMetadataPath(fs, metadataDir, sparkSession.sessionState.conf) + metadataDir + } - private val metadataDirectory = new Path(path, FileStreamSink.metadataDir) logInfo(s"Reading streaming file log from $metadataDirectory") private val metadataLog = - new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, metadataDirectory.toUri.toString) + new FileStreamSinkLog(FileStreamSinkLog.VERSION, sparkSession, metadataDirectory.toString) private val allFilesFromLog = metadataLog.allFiles().map(_.toFileStatus).filterNot(_.isDirectory) private var cachedPartitionSpec: PartitionSpec = _ 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 2cac86599ef19..e7eb2cb558cdb 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 @@ -17,27 +17,28 @@ package org.apache.spark.sql.execution.streaming -import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.encoders.RowEncoder -import org.apache.spark.sql.catalyst.expressions.{Alias, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} -import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, CurrentBatchTimestamp, CurrentDate, CurrentTimestamp} +import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LocalRelation, LogicalPlan, Project} +import org.apache.spark.sql.catalyst.util.truncatedString import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, WriteToDataSourceV2} -import org.apache.spark.sql.execution.streaming.sources.{MicroBatchWritSupport, RateControlMicroBatchReadSupport} +import org.apache.spark.sql.execution.datasources.v2.{StreamingDataSourceV2Relation, StreamWriterCommitProgress, WriteToDataSourceV2Exec} +import org.apache.spark.sql.execution.streaming.sources.{RateControlMicroBatchStream, WriteToMicroBatchDataSource} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} -import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} +import org.apache.spark.util.Clock class MicroBatchExecution( sparkSession: SparkSession, name: String, checkpointRoot: String, analyzedPlan: LogicalPlan, - sink: BaseStreamingSink, + sink: Table, trigger: Trigger, triggerClock: Clock, outputMode: OutputMode, @@ -47,13 +48,10 @@ class MicroBatchExecution( sparkSession, name, checkpointRoot, analyzedPlan, sink, trigger, triggerClock, outputMode, deleteCheckpointOnStop) { - @volatile protected var sources: Seq[BaseStreamingSource] = Seq.empty - - private val readSupportToDataSourceMap = - MutableMap.empty[MicroBatchReadSupport, (DataSourceV2, Map[String, String])] + @volatile protected var sources: Seq[SparkDataStream] = Seq.empty private val triggerExecutor = trigger match { - case t: ProcessingTime => ProcessingTimeExecutor(t, triggerClock) + case t: ProcessingTimeTrigger => ProcessingTimeExecutor(t, triggerClock) case OneTimeTrigger => OneTimeExecutor() case _ => throw new IllegalStateException(s"Unknown type of trigger: $trigger") } @@ -67,6 +65,7 @@ class MicroBatchExecution( var nextSourceId = 0L val toExecutionRelationMap = MutableMap[StreamingRelation, StreamingExecutionRelation]() val v2ToExecutionRelationMap = MutableMap[StreamingRelationV2, StreamingExecutionRelation]() + val v2ToRelationMap = MutableMap[StreamingRelationV2, StreamingDataSourceV2Relation]() // We transform each distinct streaming relation into a StreamingExecutionRelation, keeping a // map as we go to ensure each identical relation gets the same StreamingExecutionRelation // object. For each microbatch, the StreamingExecutionRelation will be replaced with a logical @@ -78,6 +77,7 @@ class MicroBatchExecution( val disabledSources = sparkSession.sqlContext.conf.disabledV2StreamingMicroBatchReaders.split(",") + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ val _logicalPlan = analyzedPlan.transform { case streamingRelation@StreamingRelation(dataSourceV1, sourceName, output) => toExecutionRelationMap.getOrElseUpdate(streamingRelation, { @@ -88,38 +88,50 @@ class MicroBatchExecution( logInfo(s"Using Source [$source] from DataSourceV1 named '$sourceName' [$dataSourceV1]") StreamingExecutionRelation(source, output)(sparkSession) }) - case s @ StreamingRelationV2( - dataSourceV2: MicroBatchReadSupportProvider, sourceName, options, output, _) if - !disabledSources.contains(dataSourceV2.getClass.getCanonicalName) => - v2ToExecutionRelationMap.getOrElseUpdate(s, { - // Materialize source to avoid creating it in every batch - val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" - val readSupport = dataSourceV2.createMicroBatchReadSupport( - metadataPath, - new DataSourceOptions(options.asJava)) - nextSourceId += 1 - readSupportToDataSourceMap(readSupport) = dataSourceV2 -> options - logInfo(s"Using MicroBatchReadSupport [$readSupport] from " + - s"DataSourceV2 named '$sourceName' [$dataSourceV2]") - StreamingExecutionRelation(readSupport, output)(sparkSession) - }) - case s @ StreamingRelationV2(dataSourceV2, sourceName, _, output, v1Relation) => - v2ToExecutionRelationMap.getOrElseUpdate(s, { - // Materialize source to avoid creating it in every batch - val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" - if (v1Relation.isEmpty) { - throw new UnsupportedOperationException( - s"Data source $sourceName does not support microbatch processing.") - } - val source = v1Relation.get.dataSource.createSource(metadataPath) - nextSourceId += 1 - logInfo(s"Using Source [$source] from DataSourceV2 named '$sourceName' [$dataSourceV2]") - StreamingExecutionRelation(source, output)(sparkSession) - }) + + case s @ StreamingRelationV2(src, srcName, table: SupportsRead, options, output, v1) => + val v2Disabled = disabledSources.contains(src.getClass.getCanonicalName) + if (!v2Disabled && table.supports(TableCapability.MICRO_BATCH_READ)) { + v2ToRelationMap.getOrElseUpdate(s, { + // Materialize source to avoid creating it in every batch + val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" + nextSourceId += 1 + logInfo(s"Reading table [$table] from DataSourceV2 named '$srcName' [$src]") + // TODO: operator pushdown. + val scan = table.newScanBuilder(options).build() + val stream = scan.toMicroBatchStream(metadataPath) + StreamingDataSourceV2Relation(output, scan, stream) + }) + } else if (v1.isEmpty) { + throw new UnsupportedOperationException( + s"Data source $srcName does not support microbatch processing.") + } else { + v2ToExecutionRelationMap.getOrElseUpdate(s, { + // Materialize source to avoid creating it in every batch + val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" + val source = v1.get.dataSource.createSource(metadataPath) + nextSourceId += 1 + logInfo(s"Using Source [$source] from DataSourceV2 named '$srcName' [$src]") + StreamingExecutionRelation(source, output)(sparkSession) + }) + } + } + sources = _logicalPlan.collect { + // v1 source + case s: StreamingExecutionRelation => s.source + // v2 source + case r: StreamingDataSourceV2Relation => r.stream } - sources = _logicalPlan.collect { case s: StreamingExecutionRelation => s.source } uniqueSources = sources.distinct - _logicalPlan + + // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. + sink match { + case s: SupportsWrite => + val streamingWrite = createStreamingWrite(s, extraOptions, _logicalPlan) + WriteToMicroBatchDataSource(streamingWrite, _logicalPlan) + + case _ => _logicalPlan + } } /** @@ -146,6 +158,12 @@ class MicroBatchExecution( logInfo(s"Query $prettyIdString was stopped") } + /** Begins recording statistics about query progress for a given trigger. */ + override protected def startTrigger(): Unit = { + super.startTrigger() + currentStatus = currentStatus.copy(isTriggerActive = true) + } + /** * Repeatedly attempts to run batches as data arrives. */ @@ -239,6 +257,7 @@ class MicroBatchExecution( * DONE */ private def populateStartOffsets(sparkSessionToRunBatches: SparkSession): Unit = { + sinkCommitProgress = None offsetLog.getLatest() match { case Some((latestBatchId, nextOffsets)) => /* First assume that we are re-executing the latest known batch @@ -249,10 +268,10 @@ class MicroBatchExecution( /* Initialize committed offsets to a committed batch, which at this * is the second latest batch id in the offset log. */ if (latestBatchId != 0) { - val secondLatestBatchId = offsetLog.get(latestBatchId - 1).getOrElse { + val secondLatestOffsets = offsetLog.get(latestBatchId - 1).getOrElse { throw new IllegalStateException(s"batch ${latestBatchId - 1} doesn't exist") } - committedOffsets = secondLatestBatchId.toStreamProgress(sources) + committedOffsets = secondLatestOffsets.toStreamProgress(sources) } // update offset metadata @@ -277,7 +296,7 @@ class MicroBatchExecution( * batch will be executed before getOffset is called again. */ availableOffsets.foreach { case (source: Source, end: Offset) => - val start = committedOffsets.get(source) + val start = committedOffsets.get(source).map(_.asInstanceOf[Offset]) source.getBatch(start, end) case nonV1Tuple => // The V2 API does not have the same edge case requiring getBatch to be called @@ -335,13 +354,13 @@ class MicroBatchExecution( if (isCurrentBatchConstructed) return true // Generate a map from each unique source to the next available offset. - val latestOffsets: Map[BaseStreamingSource, Option[Offset]] = uniqueSources.map { + val latestOffsets: Map[SparkDataStream, Option[OffsetV2]] = uniqueSources.map { case s: Source => updateStatusMessage(s"Getting offsets from $s") reportTimeTaken("getOffset") { (s, s.getOffset) } - case s: RateControlMicroBatchReadSupport => + case s: RateControlMicroBatchStream => updateStatusMessage(s"Getting offsets from $s") reportTimeTaken("latestOffset") { val startOffset = availableOffsets @@ -349,7 +368,7 @@ class MicroBatchExecution( .getOrElse(s.initialOffset()) (s, Option(s.latestOffset(startOffset))) } - case s: MicroBatchReadSupport => + case s: MicroBatchStream => updateStatusMessage(s"Getting offsets from $s") reportTimeTaken("latestOffset") { (s, Option(s.latestOffset())) @@ -392,9 +411,9 @@ class MicroBatchExecution( val prevBatchOff = offsetLog.get(currentBatchId - 1) if (prevBatchOff.isDefined) { prevBatchOff.get.toStreamProgress(sources).foreach { - case (src: Source, off) => src.commit(off) - case (readSupport: MicroBatchReadSupport, off) => - readSupport.commit(readSupport.deserializeOffset(off.json)) + case (src: Source, off: Offset) => src.commit(off) + case (stream: MicroBatchStream, off) => + stream.commit(stream.deserializeOffset(off.json)) case (src, _) => throw new IllegalArgumentException( s"Unknown source is found at constructNextBatch: $src") @@ -407,8 +426,7 @@ class MicroBatchExecution( // It is now safe to discard the metadata beyond the minimum number to retain. // Note that purge is exclusive, i.e. it purges everything before the target ID. if (minLogEntriesToMaintain < currentBatchId) { - offsetLog.purge(currentBatchId - minLogEntriesToMaintain) - commitLog.purge(currentBatchId - minLogEntriesToMaintain) + purge(currentBatchId - minLogEntriesToMaintain) } } noNewData = false @@ -429,9 +447,9 @@ class MicroBatchExecution( // Request unprocessed data from all sources. newData = reportTimeTaken("getBatch") { availableOffsets.flatMap { - case (source: Source, available) + case (source: Source, available: Offset) if committedOffsets.get(source).map(_ != available).getOrElse(true) => - val current = committedOffsets.get(source) + val current = committedOffsets.get(source).map(_.asInstanceOf[Offset]) val batch = source.getBatch(current, available) assert(batch.isStreaming, s"DataFrame returned by getBatch from $source did not have isStreaming=true\n" + @@ -439,44 +457,36 @@ class MicroBatchExecution( logDebug(s"Retrieving data from $source: $current -> $available") Some(source -> batch.logicalPlan) - // TODO(cloud-fan): for data source v2, the new batch is just a new `ScanConfigBuilder`, but - // to be compatible with streaming source v1, we return a logical plan as a new batch here. - case (readSupport: MicroBatchReadSupport, available) - if committedOffsets.get(readSupport).map(_ != available).getOrElse(true) => - val current = committedOffsets.get(readSupport).map { - off => readSupport.deserializeOffset(off.json) + case (stream: MicroBatchStream, available) + if committedOffsets.get(stream).map(_ != available).getOrElse(true) => + val current = committedOffsets.get(stream).map { + off => stream.deserializeOffset(off.json) } val endOffset: OffsetV2 = available match { - case v1: SerializedOffset => readSupport.deserializeOffset(v1.json) + case v1: SerializedOffset => stream.deserializeOffset(v1.json) case v2: OffsetV2 => v2 } - val startOffset = current.getOrElse(readSupport.initialOffset) - val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset, endOffset) - logDebug(s"Retrieving data from $readSupport: $current -> $endOffset") - - val (source, options) = readSupport match { - // `MemoryStream` is special. It's for test only and doesn't have a `DataSourceV2` - // implementation. We provide a fake one here for explain. - case _: MemoryStream[_] => MemoryStreamDataSource -> Map.empty[String, String] - // Provide a fake value here just in case something went wrong, e.g. the reader gives - // a wrong `equals` implementation. - case _ => readSupportToDataSourceMap.getOrElse(readSupport, { - FakeDataSourceV2 -> Map.empty[String, String] - }) - } - Some(readSupport -> StreamingDataSourceV2Relation( - readSupport.fullSchema().toAttributes, source, options, readSupport, scanConfigBuilder)) + val startOffset = current.getOrElse(stream.initialOffset) + logDebug(s"Retrieving data from $stream: $current -> $endOffset") + + // To be compatible with the v1 source, the `newData` is represented as a logical plan, + // while the `newData` of v2 source is just the start and end offsets. Here we return a + // fake logical plan to carry the offsets. + Some(stream -> OffsetHolder(startOffset, endOffset)) + case _ => None } } // Replace sources in the logical plan with data that has arrived since the last batch. val newBatchesPlan = logicalPlan transform { + // For v1 sources. case StreamingExecutionRelation(source, output) => newData.get(source).map { dataPlan => + val maxFields = SQLConf.get.maxToStringFields assert(output.size == dataPlan.output.size, - s"Invalid batch: ${Utils.truncatedString(output, ",")} != " + - s"${Utils.truncatedString(dataPlan.output, ",")}") + s"Invalid batch: ${truncatedString(output, ",", maxFields)} != " + + s"${truncatedString(dataPlan.output, ",", maxFields)}") val aliases = output.zip(dataPlan.output).map { case (to, from) => Alias(from, to.name)(exprId = to.exprId, explicitMetadata = Some(from.metadata)) @@ -485,13 +495,25 @@ class MicroBatchExecution( }.getOrElse { LocalRelation(output, isStreaming = true) } + + // For v2 sources. + case r: StreamingDataSourceV2Relation => + newData.get(r.stream).map { + case OffsetHolder(start, end) => + r.copy(startOffset = Some(start), endOffset = Some(end)) + }.getOrElse { + LocalRelation(r.output, isStreaming = true) + } } // Rewire the plan to use the new attributes that were returned by the source. val newAttributePlan = newBatchesPlan transformAllExpressions { case ct: CurrentTimestamp => + // CurrentTimestamp is not TimeZoneAwareExpression while CurrentBatchTimestamp is. + // Without TimeZoneId, CurrentBatchTimestamp is unresolved. Here, we use an explicit + // dummy string to prevent UnresolvedException and to prevent to be used in the future. CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, - ct.dataType) + ct.dataType, Some("Dummy TimeZoneId")) case cd: CurrentDate => CurrentBatchTimestamp(offsetSeqMetadata.batchTimestampMs, cd.dataType, cd.timeZoneId) @@ -499,13 +521,8 @@ class MicroBatchExecution( val triggerLogicalPlan = sink match { case _: Sink => newAttributePlan - case s: StreamingWriteSupportProvider => - val writer = s.createStreamingWriteSupport( - s"$runId", - newAttributePlan.schema, - outputMode, - new DataSourceOptions(extraOptions.asJava)) - WriteToDataSourceV2(new MicroBatchWritSupport(currentBatchId, writer), newAttributePlan) + case _: SupportsWrite => + newAttributePlan.asInstanceOf[WriteToMicroBatchDataSource].createPlan(currentBatchId) case _ => throw new IllegalArgumentException(s"unknown sink type for $sink") } @@ -520,6 +537,7 @@ class MicroBatchExecution( triggerLogicalPlan, outputMode, checkpointFile("state"), + id, runId, currentBatchId, offsetSeqMetadata) @@ -529,18 +547,24 @@ class MicroBatchExecution( val nextBatch = new Dataset(sparkSessionToRunBatch, lastExecution, RowEncoder(lastExecution.analyzed.schema)) - reportTimeTaken("addBatch") { + val batchSinkProgress: Option[StreamWriterCommitProgress] = + reportTimeTaken("addBatch") { SQLExecution.withNewExecutionId(sparkSessionToRunBatch, lastExecution) { sink match { case s: Sink => s.addBatch(currentBatchId, nextBatch) - case _: StreamingWriteSupportProvider => + case _: SupportsWrite => // This doesn't accumulate any data - it just forces execution of the microbatch writer. nextBatch.collect() } + lastExecution.executedPlan match { + case w: WriteToDataSourceV2Exec => w.commitProgress + case _ => None + } } } withProgressLocked { + sinkCommitProgress = batchSinkProgress watermarkTracker.updateWatermark(lastExecution.executedPlan) commitLog.add(currentBatchId, CommitMetadata(watermarkTracker.currentWatermark)) committedOffsets ++= availableOffsets @@ -563,6 +587,6 @@ object MicroBatchExecution { val BATCH_ID_KEY = "streaming.sql.batchId" } -object MemoryStreamDataSource extends DataSourceV2 - -object FakeDataSourceV2 extends DataSourceV2 +case class OffsetHolder(start: OffsetV2, end: OffsetV2) extends LeafNode { + override def output: Seq[Attribute] = Nil +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java deleted file mode 100644 index 43ad4b3384ec3..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Offset.java +++ /dev/null @@ -1,61 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming; - -/** - * This is an internal, deprecated interface. New source implementations should use the - * org.apache.spark.sql.sources.v2.reader.streaming.Offset class, which is the one that will be - * supported in the long term. - * - * This class will be removed in a future release. - */ -public abstract class Offset { - /** - * A JSON-serialized representation of an Offset that is - * used for saving offsets to the offset log. - * Note: We assume that equivalent/equal offsets serialize to - * identical JSON strings. - * - * @return JSON string encoding - */ - public abstract String json(); - - /** - * Equality based on JSON string representation. We leverage the - * JSON representation for normalization between the Offset's - * in memory and on disk representations. - */ - @Override - public boolean equals(Object obj) { - if (obj instanceof Offset) { - return this.json().equals(((Offset) obj).json()); - } else { - return false; - } - } - - @Override - public int hashCode() { - return this.json().hashCode(); - } - - @Override - public String toString() { - return this.json(); - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala index 73cf355dbe758..b6fa2e9dc3612 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeq.scala @@ -24,13 +24,15 @@ import org.apache.spark.internal.Logging import org.apache.spark.sql.RuntimeConfig import org.apache.spark.sql.execution.streaming.state.{FlatMapGroupsWithStateExecHelper, StreamingAggregationStateManager} import org.apache.spark.sql.internal.SQLConf.{FLATMAPGROUPSWITHSTATE_STATE_FORMAT_VERSION, _} +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} + /** * An ordered collection of offsets, used to track the progress of processing data from one or more * [[Source]]s that are present in a streaming query. This is similar to simplified, single-instance * vector clock that must progress linearly forward. */ -case class OffsetSeq(offsets: Seq[Option[Offset]], metadata: Option[OffsetSeqMetadata] = None) { +case class OffsetSeq(offsets: Seq[Option[OffsetV2]], metadata: Option[OffsetSeqMetadata] = None) { /** * Unpacks an offset into [[StreamProgress]] by associating each offset with the ordered list of @@ -39,7 +41,7 @@ case class OffsetSeq(offsets: Seq[Option[Offset]], metadata: Option[OffsetSeqMet * This method is typically used to associate a serialized offset with actual sources (which * cannot be serialized). */ - def toStreamProgress(sources: Seq[BaseStreamingSource]): StreamProgress = { + def toStreamProgress(sources: Seq[SparkDataStream]): StreamProgress = { assert(sources.size == offsets.size, s"There are [${offsets.size}] sources in the " + s"checkpoint offsets and now there are [${sources.size}] sources requested by the query. " + s"Cannot continue.") @@ -56,13 +58,13 @@ object OffsetSeq { * Returns a [[OffsetSeq]] with a variable sequence of offsets. * `nulls` in the sequence are converted to `None`s. */ - def fill(offsets: Offset*): OffsetSeq = OffsetSeq.fill(None, offsets: _*) + def fill(offsets: OffsetV2*): OffsetSeq = OffsetSeq.fill(None, offsets: _*) /** * Returns a [[OffsetSeq]] with metadata and a variable sequence of offsets. * `nulls` in the sequence are converted to `None`s. */ - def fill(metadata: Option[String], offsets: Offset*): OffsetSeq = { + def fill(metadata: Option[String], offsets: OffsetV2*): OffsetSeq = { OffsetSeq(offsets.map(Option(_)), metadata.map(OffsetSeqMetadata.apply)) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala index 2c8d7c7b0f3c5..b40426aff0e79 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/OffsetSeqLog.scala @@ -24,6 +24,7 @@ import java.nio.charset.StandardCharsets._ import scala.io.{Source => IOSource} import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} /** * This class is used to log offsets to persistent files in HDFS. @@ -47,7 +48,7 @@ class OffsetSeqLog(sparkSession: SparkSession, path: String) override protected def deserialize(in: InputStream): OffsetSeq = { // called inside a try-finally where the underlying stream is closed in the caller - def parseOffset(value: String): Offset = value match { + def parseOffset(value: String): OffsetV2 = value match { case OffsetSeqLog.SERIALIZED_VOID_OFFSET => null case json => SerializedOffset(json) } @@ -56,7 +57,7 @@ class OffsetSeqLog(sparkSession: SparkSession, path: String) throw new IllegalStateException("Incomplete log file") } - val version = parseVersion(lines.next(), OffsetSeqLog.VERSION) + validateVersion(lines.next(), OffsetSeqLog.VERSION) // read metadata val metadata = lines.next().trim match { 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 392229bcb5f55..6cb75083d0c0b 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 @@ -26,10 +26,11 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.plans.logical.{EventTimeWatermark, LogicalPlan} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeUtils._ import org.apache.spark.sql.execution.QueryExecution -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec -import org.apache.spark.sql.sources.v2.reader.streaming.MicroBatchReadSupport +import org.apache.spark.sql.execution.datasources.v2.{MicroBatchScanExec, StreamingDataSourceV2Relation, StreamWriterCommitProgress} +import org.apache.spark.sql.sources.v2.Table +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, SparkDataStream} import org.apache.spark.sql.streaming._ import org.apache.spark.sql.streaming.StreamingQueryListener.QueryProgressEvent import org.apache.spark.util.Clock @@ -44,7 +45,7 @@ import org.apache.spark.util.Clock trait ProgressReporter extends Logging { case class ExecutionStats( - inputRows: Map[BaseStreamingSource, Long], + inputRows: Map[SparkDataStream, Long], stateOperators: Seq[StateOperatorProgress], eventTimeStats: Map[String, String]) @@ -55,9 +56,10 @@ trait ProgressReporter extends Logging { protected def triggerClock: Clock protected def logicalPlan: LogicalPlan protected def lastExecution: QueryExecution - protected def newData: Map[BaseStreamingSource, LogicalPlan] - protected def sources: Seq[BaseStreamingSource] - protected def sink: BaseStreamingSink + protected def newData: Map[SparkDataStream, LogicalPlan] + protected def sinkCommitProgress: Option[StreamWriterCommitProgress] + protected def sources: Seq[SparkDataStream] + protected def sink: Table protected def offsetSeqMetadata: OffsetSeqMetadata protected def currentBatchId: Long protected def sparkSession: SparkSession @@ -66,8 +68,8 @@ trait ProgressReporter extends Logging { // Local timestamps and counters. private var currentTriggerStartTimestamp = -1L private var currentTriggerEndTimestamp = -1L - private var currentTriggerStartOffsets: Map[BaseStreamingSource, String] = _ - private var currentTriggerEndOffsets: Map[BaseStreamingSource, String] = _ + private var currentTriggerStartOffsets: Map[SparkDataStream, String] = _ + private var currentTriggerEndOffsets: Map[SparkDataStream, String] = _ // TODO: Restore this from the checkpoint when possible. private var lastTriggerStartTimestamp = -1L @@ -86,7 +88,7 @@ trait ProgressReporter extends Logging { private var lastNoDataProgressEventTime = Long.MinValue private val timestampFormat = new SimpleDateFormat("yyyy-MM-dd'T'HH:mm:ss.SSS'Z'") // ISO8601 - timestampFormat.setTimeZone(DateTimeUtils.getTimeZone("UTC")) + timestampFormat.setTimeZone(getTimeZone("UTC")) @volatile protected var currentStatus: StreamingQueryStatus = { @@ -114,7 +116,6 @@ trait ProgressReporter extends Logging { logDebug("Starting Trigger Calculation") lastTriggerStartTimestamp = currentTriggerStartTimestamp currentTriggerStartTimestamp = triggerClock.getTimeMillis() - currentStatus = currentStatus.copy(isTriggerActive = true) currentTriggerStartOffsets = null currentTriggerEndOffsets = null currentDurationsMs.clear() @@ -147,10 +148,10 @@ trait ProgressReporter extends Logging { val executionStats = extractExecutionStats(hasNewData) val processingTimeSec = - (currentTriggerEndTimestamp - currentTriggerStartTimestamp).toDouble / 1000 + (currentTriggerEndTimestamp - currentTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND val inputTimeSec = if (lastTriggerStartTimestamp >= 0) { - (currentTriggerStartTimestamp - lastTriggerStartTimestamp).toDouble / 1000 + (currentTriggerStartTimestamp - lastTriggerStartTimestamp).toDouble / MILLIS_PER_SECOND } else { Double.NaN } @@ -168,7 +169,9 @@ trait ProgressReporter extends Logging { ) } - val sinkProgress = new SinkProgress(sink.toString) + val sinkProgress = SinkProgress( + sink.toString, + sinkCommitProgress.map(_.numOutputRows)) val newProgress = new StreamingQueryProgress( id = id, @@ -238,17 +241,19 @@ trait ProgressReporter extends Logging { } /** Extract number of input sources for each streaming source in plan */ - private def extractSourceToNumInputRows(): Map[BaseStreamingSource, Long] = { + private def extractSourceToNumInputRows(): Map[SparkDataStream, Long] = { - def sumRows(tuples: Seq[(BaseStreamingSource, Long)]): Map[BaseStreamingSource, Long] = { + def sumRows(tuples: Seq[(SparkDataStream, Long)]): Map[SparkDataStream, Long] = { tuples.groupBy(_._1).mapValues(_.map(_._2).sum) // sum up rows for each source } val onlyDataSourceV2Sources = { - // Check whether the streaming query's logical plan has only V2 data sources - val allStreamingLeaves = - logicalPlan.collect { case s: StreamingExecutionRelation => s } - allStreamingLeaves.forall { _.source.isInstanceOf[MicroBatchReadSupport] } + // Check whether the streaming query's logical plan has only V2 micro-batch data sources + val allStreamingLeaves = logicalPlan.collect { + case s: StreamingDataSourceV2Relation => s.stream.isInstanceOf[MicroBatchStream] + case _: StreamingExecutionRelation => false + } + allStreamingLeaves.forall(_ == true) } if (onlyDataSourceV2Sources) { @@ -256,9 +261,9 @@ trait ProgressReporter extends Logging { // (can happen with self-unions or self-joins). This means the source is scanned multiple // times in the query, we should count the numRows for each scan. val sourceToInputRowsTuples = lastExecution.executedPlan.collect { - case s: DataSourceV2ScanExec if s.readSupport.isInstanceOf[BaseStreamingSource] => + case s: MicroBatchScanExec => val numRows = s.metrics.get("numOutputRows").map(_.value).getOrElse(0L) - val source = s.readSupport.asInstanceOf[BaseStreamingSource] + val source = s.stream source -> numRows } logDebug("Source -> # input rows\n\t" + sourceToInputRowsTuples.mkString("\n\t")) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala index 34bc085d920c1..190325fb7ec25 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Sink.scala @@ -17,14 +17,21 @@ package org.apache.spark.sql.execution.streaming +import java.util + import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.sources.v2.{Table, TableCapability} +import org.apache.spark.sql.types.StructType /** * An interface for systems that can collect the results of a streaming query. In order to preserve * exactly once semantics a sink must be idempotent in the face of multiple attempts to add the same * batch. + * + * Note that, we extends `Table` here, to make the v1 streaming sink API be compatible with + * data source v2. */ -trait Sink extends BaseStreamingSink { +trait Sink extends Table { /** * Adds a batch of data to this sink. The data for a given `batchId` is deterministic and if @@ -38,4 +45,16 @@ trait Sink extends BaseStreamingSink { * after data is consumed by sink successfully. */ def addBatch(batchId: Long, data: DataFrame): Unit + + override def name: String = { + throw new IllegalStateException("should not be called.") + } + + override def schema: StructType = { + throw new IllegalStateException("should not be called.") + } + + override def capabilities: util.Set[TableCapability] = { + throw new IllegalStateException("should not be called.") + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala index dbbd59e06909c..7f66d0b055cc3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Source.scala @@ -18,14 +18,19 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} +import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream import org.apache.spark.sql.types.StructType /** * A source of continually arriving data for a streaming query. A [[Source]] must have a * monotonically increasing notion of progress that can be represented as an [[Offset]]. Spark * will regularly query each [[Source]] to see if any more data is available. + * + * Note that, we extends `SparkDataStream` here, to make the v1 streaming source API be compatible + * with data source v2. */ -trait Source extends BaseStreamingSource { +trait Source extends SparkDataStream { /** Returns the schema of the data from this source */ def schema: StructType @@ -62,6 +67,15 @@ trait Source extends BaseStreamingSource { */ def commit(end: Offset) : Unit = {} - /** Stop this source and free any resources it has allocated. */ - def stop(): Unit + override def initialOffset(): OffsetV2 = { + throw new IllegalStateException("should not be called.") + } + + override def deserializeOffset(json: String): OffsetV2 = { + throw new IllegalStateException("should not be called.") + } + + override def commit(end: OffsetV2): Unit = { + throw new IllegalStateException("should not be called.") + } } 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 631a6eb649ffb..7c1f6ca42c1f2 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 @@ -24,6 +24,7 @@ import java.util.concurrent.{CountDownLatch, ExecutionException, TimeUnit} import java.util.concurrent.atomic.AtomicReference import java.util.concurrent.locks.{Condition, ReentrantLock} +import scala.collection.JavaConverters._ import scala.collection.mutable.{Map => MutableMap} import scala.util.control.NonFatal @@ -34,10 +35,17 @@ import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.execution.command.StreamingExplainCommand +import org.apache.spark.sql.execution.datasources.v2.StreamWriterCommitProgress import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} +import org.apache.spark.sql.sources.v2.writer.SupportsTruncate +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{Clock, UninterruptibleThread, Utils} /** States for [[StreamExecution]]'s lifecycle. */ @@ -54,14 +62,15 @@ case object RECONFIGURING extends State * and the results are committed transactionally to the given [[Sink]]. * * @param deleteCheckpointOnStop whether to delete the checkpoint if the query is stopped without - * errors + * errors. Checkpoint deletion can be forced with the appropriate + * Spark configuration. */ abstract class StreamExecution( override val sparkSession: SparkSession, override val name: String, private val checkpointRoot: String, analyzedPlan: LogicalPlan, - val sink: BaseStreamingSink, + val sink: Table, val trigger: Trigger, val triggerClock: Clock, val outputMode: OutputMode, @@ -88,8 +97,47 @@ abstract class StreamExecution( val resolvedCheckpointRoot = { val checkpointPath = new Path(checkpointRoot) val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) - checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory).toUri.toString + if (sparkSession.conf.get(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED) + && StreamExecution.containsSpecialCharsInPath(checkpointPath)) { + // In Spark 2.4 and earlier, the checkpoint path is escaped 3 times (3 `Path.toUri.toString` + // calls). If this legacy checkpoint path exists, we will throw an error to tell the user how + // to migrate. + val legacyCheckpointDir = + new Path(new Path(checkpointPath.toUri.toString).toUri.toString).toUri.toString + val legacyCheckpointDirExists = + try { + fs.exists(new Path(legacyCheckpointDir)) + } catch { + case NonFatal(e) => + // We may not have access to this directory. Don't fail the query if that happens. + logWarning(e.getMessage, e) + false + } + if (legacyCheckpointDirExists) { + throw new SparkException( + s"""Error: we detected a possible problem with the location of your checkpoint and you + |likely need to move it before restarting this query. + | + |Earlier version of Spark incorrectly escaped paths when writing out checkpoints for + |structured streaming. While this was corrected in Spark 3.0, it appears that your + |query was started using an earlier version that incorrectly handled the checkpoint + |path. + | + |Correct Checkpoint Directory: $checkpointPath + |Incorrect Checkpoint Directory: $legacyCheckpointDir + | + |Please move the data from the incorrect directory to the correct one, delete the + |incorrect directory, and then restart this query. If you believe you are receiving + |this message in error, you can disable it with the SQL conf + |${SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED.key}.""" + .stripMargin) + } + } + val checkpointDir = checkpointPath.makeQualified(fs.getUri, fs.getWorkingDirectory) + fs.mkdirs(checkpointDir) + checkpointDir.toString } + logInfo(s"Checkpoint root $checkpointRoot resolved to $resolvedCheckpointRoot.") def logicalPlan: LogicalPlan @@ -113,6 +161,9 @@ abstract class StreamExecution( @volatile var availableOffsets = new StreamProgress + @volatile + var sinkCommitProgress: Option[StreamWriterCommitProgress] = None + /** The current batchId or -1 if execution has not yet been initialized. */ protected var currentBatchId: Long = -1 @@ -155,7 +206,7 @@ abstract class StreamExecution( /** * A list of unique sources in the query plan. This will be set when generating logical plan. */ - @volatile protected var uniqueSources: Seq[BaseStreamingSource] = Seq.empty + @volatile protected var uniqueSources: Seq[SparkDataStream] = Seq.empty /** Defines the internal state of execution */ protected val state = new AtomicReference[State](INITIALIZING) @@ -164,7 +215,7 @@ abstract class StreamExecution( var lastExecution: IncrementalExecution = _ /** Holds the most recent input data for each source. */ - protected var newData: Map[BaseStreamingSource, LogicalPlan] = _ + protected var newData: Map[SparkDataStream, LogicalPlan] = _ @volatile protected var streamDeathCause: StreamingQueryException = null @@ -176,6 +227,9 @@ abstract class StreamExecution( lazy val streamMetrics = new MetricsReporter( this, s"spark.streaming.${Option(name).getOrElse(id)}") + /** Isolated spark session to run the batches with. */ + private val sparkSessionForStream = sparkSession.cloneSession() + /** * The thread that runs the micro-batches of this stream. Note that this thread must be * [[org.apache.spark.util.UninterruptibleThread]] to workaround KAFKA-1894: interrupting a @@ -217,7 +271,7 @@ abstract class StreamExecution( /** Returns the path of a file with `name` in the checkpoint directory. */ protected def checkpointFile(name: String): String = - new Path(new Path(resolvedCheckpointRoot), name).toUri.toString + new Path(new Path(resolvedCheckpointRoot), name).toString /** * Starts the execution. This returns only after the thread has started and [[QueryStartedEvent]] @@ -265,8 +319,6 @@ abstract class StreamExecution( // force initialization of the logical plan so that the sources can be created logicalPlan - // Isolated spark session to run the batches with. - val sparkSessionForStream = sparkSession.cloneSession() // Adaptive execution can change num shuffle partitions, disallow sparkSessionForStream.conf.set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "false") // Disable cost-based join optimization as we do not want stateful operations to be rearranged @@ -329,10 +381,13 @@ abstract class StreamExecution( postEvent( new QueryTerminatedEvent(id, runId, exception.map(_.cause).map(Utils.exceptionString))) - // Delete the temp checkpoint only when the query didn't fail - if (deleteCheckpointOnStop && exception.isEmpty) { + // Delete the temp checkpoint when either force delete enabled or the query didn't fail + if (deleteCheckpointOnStop && + (sparkSession.sessionState.conf + .getConf(SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION) || exception.isEmpty)) { val checkpointPath = new Path(resolvedCheckpointRoot) try { + logInfo(s"Deleting checkpoint $checkpointPath.") val fs = checkpointPath.getFileSystem(sparkSession.sessionState.newHadoopConf()) fs.delete(checkpointPath, true) } catch { @@ -383,7 +438,7 @@ abstract class StreamExecution( * Blocks the current thread until processing for data from the given `source` has reached at * least the given `Offset`. This method is intended for use primarily when writing tests. */ - private[sql] def awaitOffset(sourceIndex: Int, newOffset: Offset, timeoutMs: Long): Unit = { + private[sql] def awaitOffset(sourceIndex: Int, newOffset: OffsetV2, timeoutMs: Long): Unit = { assertAwaitThread() def notDone = { val localCommittedOffsets = committedOffsets @@ -526,6 +581,41 @@ abstract class StreamExecution( Option(name).map(_ + "
    ").getOrElse("") + s"id = $id
    runId = $runId
    batch = $batchDescription" } + + protected def createStreamingWrite( + table: SupportsWrite, + options: Map[String, String], + inputPlan: LogicalPlan): StreamingWrite = { + val writeBuilder = table.newWriteBuilder(new CaseInsensitiveStringMap(options.asJava)) + .withQueryId(id.toString) + .withInputDataSchema(inputPlan.schema) + outputMode match { + case Append => + writeBuilder.buildForStreaming() + + case Complete => + // TODO: we should do this check earlier when we have capability API. + require(writeBuilder.isInstanceOf[SupportsTruncate], + table.name + " does not support Complete mode.") + writeBuilder.asInstanceOf[SupportsTruncate].truncate().buildForStreaming() + + case Update => + // Although no v2 sinks really support Update mode now, but during tests we do want them + // to pretend to support Update mode, and treat Update mode same as Append mode. + if (Utils.isTesting) { + writeBuilder.buildForStreaming() + } else { + throw new IllegalArgumentException( + "Data source v2 streaming sinks does not support Update mode.") + } + } + } + + protected def purge(threshold: Long): Unit = { + logDebug(s"Purging metadata at threshold=$threshold") + offsetLog.purge(threshold) + commitLog.purge(threshold) + } } object StreamExecution { @@ -562,6 +652,11 @@ object StreamExecution { case _ => false } + + /** Whether the path contains special chars that will be escaped when converting to a `URI`. */ + def containsSpecialCharsInPath(path: Path): Boolean = { + path.toUri.getPath != new Path(path.toUri.toString).toUri.getPath + } } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala index 0bc54eac4ee8e..516afbea5d9de 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamMetadata.scala @@ -19,16 +19,18 @@ package org.apache.spark.sql.execution.streaming import java.io.{InputStreamReader, OutputStreamWriter} import java.nio.charset.StandardCharsets +import java.util.ConcurrentModificationException import scala.util.control.NonFatal import org.apache.commons.io.IOUtils import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.{FileSystem, FSDataInputStream, FSDataOutputStream, Path} +import org.apache.hadoop.fs.{FileAlreadyExistsException, FSDataInputStream, Path} import org.json4s.NoTypeHints import org.json4s.jackson.Serialization import org.apache.spark.internal.Logging +import org.apache.spark.sql.execution.streaming.CheckpointFileManager.CancellableFSDataOutputStream import org.apache.spark.sql.streaming.StreamingQuery /** @@ -70,19 +72,26 @@ object StreamMetadata extends Logging { metadata: StreamMetadata, metadataFile: Path, hadoopConf: Configuration): Unit = { - var output: FSDataOutputStream = null + var output: CancellableFSDataOutputStream = null try { - val fs = metadataFile.getFileSystem(hadoopConf) - output = fs.create(metadataFile) + val fileManager = CheckpointFileManager.create(metadataFile.getParent, hadoopConf) + output = fileManager.createAtomic(metadataFile, overwriteIfPossible = false) val writer = new OutputStreamWriter(output) Serialization.write(metadata, writer) writer.close() } catch { - case NonFatal(e) => + case e: FileAlreadyExistsException => + if (output != null) { + output.cancel() + } + throw new ConcurrentModificationException( + s"Multiple streaming queries are concurrently using $metadataFile", e) + case e: Throwable => + if (output != null) { + output.cancel() + } logError(s"Error writing stream metadata $metadata to $metadataFile", e) throw e - } finally { - IOUtils.closeQuietly(output) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala index 8531070b1bc49..8783eaa0e68b3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamProgress.scala @@ -19,32 +19,35 @@ package org.apache.spark.sql.execution.streaming import scala.collection.{immutable, GenTraversableOnce} +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} + + /** * A helper class that looks like a Map[Source, Offset]. */ class StreamProgress( - val baseMap: immutable.Map[BaseStreamingSource, Offset] = - new immutable.HashMap[BaseStreamingSource, Offset]) - extends scala.collection.immutable.Map[BaseStreamingSource, Offset] { + val baseMap: immutable.Map[SparkDataStream, OffsetV2] = + new immutable.HashMap[SparkDataStream, OffsetV2]) + extends scala.collection.immutable.Map[SparkDataStream, OffsetV2] { - def toOffsetSeq(source: Seq[BaseStreamingSource], metadata: OffsetSeqMetadata): OffsetSeq = { + def toOffsetSeq(source: Seq[SparkDataStream], metadata: OffsetSeqMetadata): OffsetSeq = { OffsetSeq(source.map(get), Some(metadata)) } override def toString: String = baseMap.map { case (k, v) => s"$k: $v"}.mkString("{", ",", "}") - override def +[B1 >: Offset](kv: (BaseStreamingSource, B1)): Map[BaseStreamingSource, B1] = { + override def +[B1 >: OffsetV2](kv: (SparkDataStream, B1)): Map[SparkDataStream, B1] = { baseMap + kv } - override def get(key: BaseStreamingSource): Option[Offset] = baseMap.get(key) + override def get(key: SparkDataStream): Option[OffsetV2] = baseMap.get(key) - override def iterator: Iterator[(BaseStreamingSource, Offset)] = baseMap.iterator + override def iterator: Iterator[(SparkDataStream, OffsetV2)] = baseMap.iterator - override def -(key: BaseStreamingSource): Map[BaseStreamingSource, Offset] = baseMap - key + override def -(key: SparkDataStream): Map[SparkDataStream, OffsetV2] = baseMap - key - def ++(updates: GenTraversableOnce[(BaseStreamingSource, Offset)]): StreamProgress = { + def ++(updates: GenTraversableOnce[(SparkDataStream, OffsetV2)]): StreamProgress = { new StreamProgress(baseMap ++ updates) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala index 4b696dfa57359..142b6e7d18068 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/StreamingRelation.scala @@ -25,7 +25,9 @@ import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} import org.apache.spark.sql.execution.LeafExecNode import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceV2} +import org.apache.spark.sql.sources.v2.{Table, TableProvider} +import org.apache.spark.sql.sources.v2.reader.streaming.SparkDataStream +import org.apache.spark.sql.util.CaseInsensitiveStringMap object StreamingRelation { def apply(dataSource: DataSource): StreamingRelation = { @@ -62,7 +64,7 @@ case class StreamingRelation(dataSource: DataSource, sourceName: String, output: * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. */ case class StreamingExecutionRelation( - source: BaseStreamingSource, + source: SparkDataStream, output: Seq[Attribute])(session: SparkSession) extends LeafNode with MultiInstanceRelation { @@ -86,15 +88,16 @@ case class StreamingExecutionRelation( // know at read time whether the query is continuous or not, so we need to be able to // swap a V1 relation back in. /** - * Used to link a [[DataSourceV2]] into a streaming + * Used to link a [[TableProvider]] into a streaming * [[org.apache.spark.sql.catalyst.plans.logical.LogicalPlan]]. This is only used for creating * a streaming [[org.apache.spark.sql.DataFrame]] from [[org.apache.spark.sql.DataFrameReader]], * and should be converted before passing to [[StreamExecution]]. */ case class StreamingRelationV2( - dataSource: DataSourceV2, + source: TableProvider, sourceName: String, - extraOptions: Map[String, String], + table: Table, + extraOptions: CaseInsensitiveStringMap, output: Seq[Attribute], v1Relation: Option[StreamingRelation])(session: SparkSession) extends LeafNode with MultiInstanceRelation { @@ -109,30 +112,6 @@ case class StreamingRelationV2( override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))(session) } -/** - * Used to link a [[DataSourceV2]] into a continuous processing execution. - */ -case class ContinuousExecutionRelation( - source: ContinuousReadSupportProvider, - extraOptions: Map[String, String], - output: Seq[Attribute])(session: SparkSession) - extends LeafNode with MultiInstanceRelation { - - override def otherCopyArgs: Seq[AnyRef] = session :: Nil - override def isStreaming: Boolean = true - override def toString: String = source.toString - - // There's no sensible value here. On the execution path, this relation will be - // swapped out with microbatches. But some dataframe operations (in particular explain) do lead - // to this node surviving analysis. So we satisfy the LeafNode contract with the session default - // value. - override def computeStats(): Statistics = Statistics( - sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes) - ) - - override def newInstance(): LogicalPlan = this.copy(output = output.map(_.newInstance()))(session) -} - /** * A dummy physical plan for [[StreamingRelation]] to support * [[org.apache.spark.sql.Dataset.explain]] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala index d188566f822b4..088471053b6f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/TriggerExecutor.scala @@ -18,7 +18,6 @@ package org.apache.spark.sql.execution.streaming import org.apache.spark.internal.Logging -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.util.{Clock, SystemClock} trait TriggerExecutor { @@ -43,10 +42,12 @@ case class OneTimeExecutor() extends TriggerExecutor { /** * A trigger executor that runs a batch every `intervalMs` milliseconds. */ -case class ProcessingTimeExecutor(processingTime: ProcessingTime, clock: Clock = new SystemClock()) +case class ProcessingTimeExecutor( + processingTimeTrigger: ProcessingTimeTrigger, + clock: Clock = new SystemClock()) extends TriggerExecutor with Logging { - private val intervalMs = processingTime.intervalMs + private val intervalMs = processingTimeTrigger.intervalMs require(intervalMs >= 0) override def execute(triggerHandler: () => Boolean): Unit = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala index 19e3e55cb2829..2bdb3402c14b1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/Triggers.scala @@ -17,13 +17,89 @@ package org.apache.spark.sql.execution.streaming -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import java.util.concurrent.TimeUnit + +import scala.concurrent.duration.Duration + import org.apache.spark.sql.streaming.Trigger +import org.apache.spark.unsafe.types.CalendarInterval + +private object Triggers { + def validate(intervalMs: Long): Unit = { + require(intervalMs >= 0, "the interval of trigger should not be negative") + } + + def convert(interval: String): Long = { + val cal = CalendarInterval.fromCaseInsensitiveString(interval) + if (cal.months > 0) { + throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") + } + TimeUnit.MICROSECONDS.toMillis(cal.microseconds) + } + + def convert(interval: Duration): Long = interval.toMillis + + def convert(interval: Long, unit: TimeUnit): Long = unit.toMillis(interval) +} /** * A [[Trigger]] that processes only one batch of data in a streaming query then terminates * the query. */ -@Experimental -@InterfaceStability.Evolving -case object OneTimeTrigger extends Trigger +private[sql] case object OneTimeTrigger extends Trigger + +/** + * A [[Trigger]] that runs a query periodically based on the processing time. If `interval` is 0, + * the query will run as fast as possible. + */ +private[sql] case class ProcessingTimeTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ProcessingTimeTrigger { + import Triggers._ + + def apply(interval: String): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def apply(interval: Duration): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval)) + } + + def create(interval: String): ProcessingTimeTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ProcessingTimeTrigger = { + ProcessingTimeTrigger(convert(interval, unit)) + } +} + +/** + * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at + * the specified interval. + */ +private[sql] case class ContinuousTrigger(intervalMs: Long) extends Trigger { + Triggers.validate(intervalMs) +} + +private[sql] object ContinuousTrigger { + import Triggers._ + + def apply(interval: String): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def apply(interval: Duration): ContinuousTrigger = { + ContinuousTrigger(convert(interval)) + } + + def create(interval: String): ContinuousTrigger = { + apply(interval) + } + + def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { + ContinuousTrigger(convert(interval, unit)) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala index 9c5c16f4f5d13..9ae39c79c5156 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/console.scala @@ -17,30 +17,30 @@ package org.apache.spark.sql.execution.streaming +import java.util + +import scala.collection.JavaConverters._ + import org.apache.spark.sql._ -import org.apache.spark.sql.execution.streaming.sources.ConsoleWriteSupport +import org.apache.spark.sql.execution.streaming.sources.ConsoleWrite import org.apache.spark.sql.sources.{BaseRelation, CreatableRelationProvider, DataSourceRegister} -import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.writer.{SupportsTruncate, WriteBuilder} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap case class ConsoleRelation(override val sqlContext: SQLContext, data: DataFrame) extends BaseRelation { override def schema: StructType = data.schema } -class ConsoleSinkProvider extends DataSourceV2 - with StreamingWriteSupportProvider +class ConsoleSinkProvider extends TableProvider with DataSourceRegister with CreatableRelationProvider { - override def createStreamingWriteSupport( - queryId: String, - schema: StructType, - mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new ConsoleWriteSupport(schema, options) + override def getTable(options: CaseInsensitiveStringMap): Table = { + ConsoleTable } def createRelation( @@ -60,3 +60,33 @@ class ConsoleSinkProvider extends DataSourceV2 def shortName(): String = "console" } + +object ConsoleTable extends Table with SupportsWrite { + + override def name(): String = "console" + + override def schema(): StructType = StructType(Nil) + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.STREAMING_WRITE).asJava + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new WriteBuilder with SupportsTruncate { + private var inputSchema: StructType = _ + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + this.inputSchema = schema + this + } + + // Do nothing for truncate. Console sink is special that it just prints all the records. + override def truncate(): WriteBuilder = this + + override def buildForStreaming(): StreamingWrite = { + assert(inputSchema != null) + new ConsoleWrite(inputSchema, options) + } + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala index 5f60343bacfaa..4c621890c9793 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousCoalesceExec.scala @@ -17,26 +17,20 @@ package org.apache.spark.sql.execution.streaming.continuous -import java.util.UUID - -import org.apache.spark.{HashPartitioner, SparkEnv} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, SinglePartition} -import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.streaming.continuous.shuffle.{ContinuousShuffleReadPartition, ContinuousShuffleReadRDD} +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} /** * Physical plan for coalescing a continuous processing plan. * * Currently, only coalesces to a single partition are supported. `numPartitions` must be 1. */ -case class ContinuousCoalesceExec(numPartitions: Int, child: SparkPlan) extends SparkPlan { +case class ContinuousCoalesceExec(numPartitions: Int, child: SparkPlan) extends UnaryExecNode { override def output: Seq[Attribute] = child.output - override def children: Seq[SparkPlan] = child :: Nil - override def outputPartitioning: Partitioning = SinglePartition override def doExecute(): RDD[InternalRow] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala index f009c52449adc..f6d156ded7663 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousExecution.scala @@ -19,30 +19,30 @@ package org.apache.spark.sql.execution.streaming.continuous import java.util.UUID import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic.AtomicReference import java.util.function.UnaryOperator -import scala.collection.JavaConverters._ -import scala.collection.mutable.{ArrayBuffer, Map => MutableMap} +import scala.collection.mutable.{Map => MutableMap} import org.apache.spark.SparkEnv import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, CurrentDate, CurrentTimestamp} +import org.apache.spark.sql.catalyst.expressions.{CurrentDate, CurrentTimestamp} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SQLExecution -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2ScanExec, StreamingDataSourceV2Relation} -import org.apache.spark.sql.execution.streaming.{ContinuousExecutionRelation, StreamingRelationV2, _} +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation +import org.apache.spark.sql.execution.streaming.{StreamingRelationV2, _} import org.apache.spark.sql.sources.v2 -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} -import org.apache.spark.sql.streaming.{OutputMode, ProcessingTime, Trigger} -import org.apache.spark.util.{Clock, Utils} +import org.apache.spark.sql.sources.v2.{SupportsRead, SupportsWrite, TableCapability} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} +import org.apache.spark.sql.streaming.{OutputMode, Trigger} +import org.apache.spark.util.Clock class ContinuousExecution( sparkSession: SparkSession, name: String, checkpointRoot: String, analyzedPlan: LogicalPlan, - sink: StreamingWriteSupportProvider, + sink: SupportsWrite, trigger: Trigger, triggerClock: Clock, outputMode: OutputMode, @@ -52,29 +52,48 @@ class ContinuousExecution( sparkSession, name, checkpointRoot, analyzedPlan, sink, trigger, triggerClock, outputMode, deleteCheckpointOnStop) { - @volatile protected var continuousSources: Seq[ContinuousReadSupport] = Seq() - override protected def sources: Seq[BaseStreamingSource] = continuousSources + @volatile protected var sources: Seq[ContinuousStream] = Seq() // For use only in test harnesses. private[sql] var currentEpochCoordinatorId: String = _ - override val logicalPlan: LogicalPlan = { - val toExecutionRelationMap = MutableMap[StreamingRelationV2, ContinuousExecutionRelation]() - analyzedPlan.transform { - case r @ StreamingRelationV2( - source: ContinuousReadSupportProvider, _, extraReaderOptions, output, _) => - // TODO: shall we create `ContinuousReadSupport` here instead of each reconfiguration? - toExecutionRelationMap.getOrElseUpdate(r, { - ContinuousExecutionRelation(source, extraReaderOptions, output)(sparkSession) + // Throwable that caused the execution to fail + private val failure: AtomicReference[Throwable] = new AtomicReference[Throwable](null) + + override val logicalPlan: WriteToContinuousDataSource = { + val v2ToRelationMap = MutableMap[StreamingRelationV2, StreamingDataSourceV2Relation]() + var nextSourceId = 0 + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + val _logicalPlan = analyzedPlan.transform { + case s @ StreamingRelationV2(ds, sourceName, table: SupportsRead, options, output, _) => + if (!table.supports(TableCapability.CONTINUOUS_READ)) { + throw new UnsupportedOperationException( + s"Data source $sourceName does not support continuous processing.") + } + + v2ToRelationMap.getOrElseUpdate(s, { + val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" + nextSourceId += 1 + logInfo(s"Reading table [$table] from DataSourceV2 named '$sourceName' [$ds]") + // TODO: operator pushdown. + val scan = table.newScanBuilder(options).build() + val stream = scan.toContinuousStream(metadataPath) + StreamingDataSourceV2Relation(output, scan, stream) }) - case StreamingRelationV2(_, sourceName, _, _, _) => - throw new UnsupportedOperationException( - s"Data source $sourceName does not support continuous processing.") } + + sources = _logicalPlan.collect { + case r: StreamingDataSourceV2Relation => r.stream.asInstanceOf[ContinuousStream] + } + uniqueSources = sources.distinct + + // TODO (SPARK-27484): we should add the writing node before the plan is analyzed. + WriteToContinuousDataSource( + createStreamingWrite(sink, extraOptions, _logicalPlan), _logicalPlan) } private val triggerExecutor = trigger match { - case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTime(t), triggerClock) + case ContinuousTrigger(t) => ProcessingTimeExecutor(ProcessingTimeTrigger(t), triggerClock) case _ => throw new IllegalStateException(s"Unsupported type of trigger: $trigger") } @@ -90,6 +109,8 @@ class ContinuousExecution( do { runContinuous(sparkSessionForStream) } while (state.updateAndGet(stateUpdate) == ACTIVE) + + stopSources() } /** @@ -117,6 +138,8 @@ class ContinuousExecution( // For at least once, we can just ignore those reports and risk duplicates. commitLog.getLatest() match { case Some((latestEpochId, _)) => + updateStatusMessage("Starting new streaming query " + + s"and getting offsets from latest epoch $latestEpochId") val nextOffsets = offsetLog.get(latestEpochId).getOrElse { throw new IllegalStateException( s"Batch $latestEpochId was committed without end epoch offsets!") @@ -128,9 +151,10 @@ class ContinuousExecution( nextOffsets case None => // We are starting this stream for the first time. Offsets are all None. + updateStatusMessage("Starting new streaming query") logInfo(s"Starting new streaming query.") currentBatchId = 0 - OffsetSeq.fill(continuousSources.map(_ => null): _*) + OffsetSeq.fill(sources.map(_ => null): _*) } } @@ -139,74 +163,38 @@ class ContinuousExecution( * @param sparkSessionForQuery Isolated [[SparkSession]] to run the continuous query with. */ private def runContinuous(sparkSessionForQuery: SparkSession): Unit = { - // A list of attributes that will need to be updated. - val replacements = new ArrayBuffer[(Attribute, Attribute)] - // Translate from continuous relation to the underlying data source. - var nextSourceId = 0 - continuousSources = logicalPlan.collect { - case ContinuousExecutionRelation(dataSource, extraReaderOptions, output) => - val metadataPath = s"$resolvedCheckpointRoot/sources/$nextSourceId" - nextSourceId += 1 - - dataSource.createContinuousReadSupport( - metadataPath, - new DataSourceOptions(extraReaderOptions.asJava)) - } - uniqueSources = continuousSources.distinct - val offsets = getStartOffsets(sparkSessionForQuery) - var insertedSourceId = 0 - val withNewSources = logicalPlan transform { - case ContinuousExecutionRelation(source, options, output) => - val readSupport = continuousSources(insertedSourceId) - insertedSourceId += 1 - val newOutput = readSupport.fullSchema().toAttributes - - assert(output.size == newOutput.size, - s"Invalid reader: ${Utils.truncatedString(output, ",")} != " + - s"${Utils.truncatedString(newOutput, ",")}") - replacements ++= output.zip(newOutput) - + val withNewSources: LogicalPlan = logicalPlan transform { + case relation: StreamingDataSourceV2Relation => val loggedOffset = offsets.offsets(0) - val realOffset = loggedOffset.map(off => readSupport.deserializeOffset(off.json)) - val startOffset = realOffset.getOrElse(readSupport.initialOffset) - val scanConfigBuilder = readSupport.newScanConfigBuilder(startOffset) - StreamingDataSourceV2Relation(newOutput, source, options, readSupport, scanConfigBuilder) + val realOffset = loggedOffset.map(off => relation.stream.deserializeOffset(off.json)) + val startOffset = realOffset.getOrElse(relation.stream.initialOffset) + relation.copy(startOffset = Some(startOffset)) } - // Rewire the plan to use the new attributes that were returned by the source. - val replacementMap = AttributeMap(replacements) - val triggerLogicalPlan = withNewSources transformAllExpressions { - case a: Attribute if replacementMap.contains(a) => - replacementMap(a).withMetadata(a.metadata) + withNewSources.transformAllExpressions { case (_: CurrentTimestamp | _: CurrentDate) => throw new IllegalStateException( "CurrentTimestamp and CurrentDate not yet supported for continuous processing") } - val writer = sink.createStreamingWriteSupport( - s"$runId", - triggerLogicalPlan.schema, - outputMode, - new DataSourceOptions(extraOptions.asJava)) - val withSink = WriteToContinuousDataSource(writer, triggerLogicalPlan) - reportTimeTaken("queryPlanning") { lastExecution = new IncrementalExecution( sparkSessionForQuery, - withSink, + withNewSources, outputMode, checkpointFile("state"), + id, runId, currentBatchId, offsetSeqMetadata) lastExecution.executedPlan // Force the lazy generation of execution plan } - val (readSupport, scanConfig) = lastExecution.executedPlan.collect { - case scan: DataSourceV2ScanExec if scan.readSupport.isInstanceOf[ContinuousReadSupport] => - scan.readSupport.asInstanceOf[ContinuousReadSupport] -> scan.scanConfig + val stream = withNewSources.collect { + case relation: StreamingDataSourceV2Relation => + relation.stream.asInstanceOf[ContinuousStream] }.head sparkSessionForQuery.sparkContext.setLocalProperty( @@ -224,18 +212,21 @@ class ContinuousExecution( trigger.asInstanceOf[ContinuousTrigger].intervalMs.toString) // Use the parent Spark session for the endpoint since it's where this query ID is registered. - val epochEndpoint = - EpochCoordinatorRef.create( - writer, readSupport, this, epochCoordinatorId, currentBatchId, sparkSession, SparkEnv.get) + val epochEndpoint = EpochCoordinatorRef.create( + logicalPlan.write, + stream, + this, + epochCoordinatorId, + currentBatchId, + sparkSession, + SparkEnv.get) val epochUpdateThread = new Thread(new Runnable { override def run: Unit = { try { triggerExecutor.execute(() => { startTrigger() - val shouldReconfigure = readSupport.needsReconfiguration(scanConfig) && - state.compareAndSet(ACTIVE, RECONFIGURING) - if (shouldReconfigure) { + if (stream.needsReconfiguration && state.compareAndSet(ACTIVE, RECONFIGURING)) { if (queryExecutionThread.isAlive) { queryExecutionThread.interrupt() } @@ -260,9 +251,16 @@ class ContinuousExecution( epochUpdateThread.setDaemon(true) epochUpdateThread.start() + updateStatusMessage("Running") reportTimeTaken("runContinuous") { - SQLExecution.withNewExecutionId( - sparkSessionForQuery, lastExecution)(lastExecution.toRdd) + SQLExecution.withNewExecutionId(sparkSessionForQuery, lastExecution) { + lastExecution.executedPlan.execute() + } + } + + val f = failure.get() + if (f != null) { + throw f } } catch { case t: Throwable if StreamExecution.isInterruptionException(t, sparkSession.sparkContext) && @@ -270,14 +268,29 @@ class ContinuousExecution( logInfo(s"Query $id ignoring exception from reconfiguring: $t") // interrupted by reconfiguration - swallow exception so we can restart the query } finally { - epochEndpoint.askSync[Unit](StopContinuousExecutionWrites) - SparkEnv.get.rpcEnv.stop(epochEndpoint) - - epochUpdateThread.interrupt() - epochUpdateThread.join() - - stopSources() - sparkSession.sparkContext.cancelJobGroup(runId.toString) + // The above execution may finish before getting interrupted, for example, a Spark job having + // 0 partitions will complete immediately. Then the interrupted status will sneak here. + // + // To handle this case, we do the two things here: + // + // 1. Clean up the resources in `queryExecutionThread.runUninterruptibly`. This may increase + // the waiting time of `stop` but should be minor because the operations here are very fast + // (just sending an RPC message in the same process and stopping a very simple thread). + // 2. Clear the interrupted status at the end so that it won't impact the `runContinuous` + // call. We may clear the interrupted status set by `stop`, but it doesn't affect the query + // termination because `runActivatedStream` will check `state` and exit accordingly. + queryExecutionThread.runUninterruptibly { + try { + epochEndpoint.askSync[Unit](StopContinuousExecutionWrites) + } finally { + SparkEnv.get.rpcEnv.stop(epochEndpoint) + epochUpdateThread.interrupt() + epochUpdateThread.join() + // The following line must be the last line because it may fail if SparkContext is stopped + sparkSession.sparkContext.cancelJobGroup(runId.toString) + } + } + Thread.interrupted() } } @@ -286,11 +299,11 @@ class ContinuousExecution( */ def addOffset( epoch: Long, - readSupport: ContinuousReadSupport, + stream: ContinuousStream, partitionOffsets: Seq[PartitionOffset]): Unit = { - assert(continuousSources.length == 1, "only one continuous source supported currently") + assert(sources.length == 1, "only one continuous source supported currently") - val globalOffset = readSupport.mergeOffsets(partitionOffsets.toArray) + val globalOffset = stream.mergeOffsets(partitionOffsets.toArray) val oldOffset = synchronized { offsetLog.add(epoch, OffsetSeq.fill(globalOffset)) offsetLog.get(epoch - 1) @@ -314,7 +327,9 @@ class ContinuousExecution( * before this is called. */ def commit(epoch: Long): Unit = { - assert(continuousSources.length == 1, "only one continuous source supported currently") + updateStatusMessage(s"Committing epoch $epoch") + + assert(sources.length == 1, "only one continuous source supported currently") assert(offsetLog.get(epoch).isDefined, s"offset for epoch $epoch not reported before commit") synchronized { @@ -323,9 +338,9 @@ class ContinuousExecution( if (queryExecutionThread.isAlive) { commitLog.add(epoch, CommitMetadata()) val offset = - continuousSources(0).deserializeOffset(offsetLog.get(epoch).get.offsets(0).get.json) - committedOffsets ++= Seq(continuousSources(0) -> offset) - continuousSources(0).commit(offset.asInstanceOf[v2.reader.streaming.Offset]) + sources(0).deserializeOffset(offsetLog.get(epoch).get.offsets(0).get.json) + committedOffsets ++= Seq(sources(0) -> offset) + sources(0).commit(offset.asInstanceOf[v2.reader.streaming.Offset]) } else { return } @@ -337,8 +352,7 @@ class ContinuousExecution( // number of batches that must be retained and made recoverable, so we should keep the // specified number of metadata that have been committed. if (minLogEntriesToMaintain <= epoch) { - offsetLog.purge(epoch + 1 - minLogEntriesToMaintain) - commitLog.purge(epoch + 1 - minLogEntriesToMaintain) + purge(epoch + 1 - minLogEntriesToMaintain) } awaitProgressLock.lock() @@ -375,6 +389,35 @@ class ContinuousExecution( } } + /** + * Stores error and stops the query execution thread to terminate the query in new thread. + */ + def stopInNewThread(error: Throwable): Unit = { + if (failure.compareAndSet(null, error)) { + logError(s"Query $prettyIdString received exception $error") + stopInNewThread() + } + } + + /** + * Stops the query execution thread to terminate the query in new thread. + */ + private def stopInNewThread(): Unit = { + new Thread("stop-continuous-execution") { + setDaemon(true) + + override def run(): Unit = { + try { + ContinuousExecution.this.stop() + } catch { + case e: Throwable => + logError(e.getMessage, e) + throw e + } + } + }.start() + } + /** * Stops the query execution thread to terminate the query. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala index a6cde2b8a710f..e1b7a8fc283d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousRateStreamSource.scala @@ -22,23 +22,18 @@ import org.json4s.jackson.Serialization import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.{RateStreamOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder, ValueRunTimeMsPair} -import org.apache.spark.sql.execution.streaming.sources.RateStreamProvider -import org.apache.spark.sql.sources.v2.DataSourceOptions +import org.apache.spark.sql.execution.streaming.{RateStreamOffset, ValueRunTimeMsPair} import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.types.StructType case class RateStreamPartitionOffset( partition: Int, currentValue: Long, currentTimeMs: Long) extends PartitionOffset -class RateStreamContinuousReadSupport(options: DataSourceOptions) extends ContinuousReadSupport { +class RateStreamContinuousStream(rowsPerSecond: Long, numPartitions: Int) extends ContinuousStream { implicit val defaultFormats: DefaultFormats = DefaultFormats val creationTime = System.currentTimeMillis() - val numPartitions = options.get(RateStreamProvider.NUM_PARTITIONS).orElse("5").toInt - val rowsPerSecond = options.get(RateStreamProvider.ROWS_PER_SECOND).orElse("6").toLong val perPartitionRate = rowsPerSecond.toDouble / numPartitions.toDouble override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = { @@ -54,18 +49,10 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin RateStreamOffset(Serialization.read[Map[Int, ValueRunTimeMsPair]](json)) } - override def fullSchema(): StructType = RateStreamProvider.SCHEMA - - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } - override def initialOffset: Offset = createInitialOffset(numPartitions, creationTime) - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig].start - - val partitionStartMap = startOffset match { + override def planInputPartitions(start: Offset): Array[InputPartition] = { + val partitionStartMap = start match { case off: RateStreamOffset => off.partitionToValueAndRunTimeMs case off => throw new IllegalArgumentException( @@ -91,8 +78,7 @@ class RateStreamContinuousReadSupport(options: DataSourceOptions) extends Contin }.toArray } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { RateStreamContinuousReaderFactory } @@ -148,8 +134,10 @@ class RateStreamContinuousPartitionReader( nextReadTime += readTimeIncrement try { - while (System.currentTimeMillis < nextReadTime) { - Thread.sleep(nextReadTime - System.currentTimeMillis) + var toWaitMs = nextReadTime - System.currentTimeMillis + while (toWaitMs > 0) { + Thread.sleep(toWaitMs) + toWaitMs = nextReadTime - System.currentTimeMillis } } catch { case _: InterruptedException => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala index 28ab2448a6633..2263b42870a65 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTextSocketSource.scala @@ -31,37 +31,29 @@ import org.json4s.jackson.Serialization import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.streaming.{Offset => _, _} import org.apache.spark.sql.execution.streaming.sources.TextSocketReader -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.RpcUtils /** - * A ContinuousReadSupport that reads text lines through a TCP socket, designed only for tutorials - * and debugging. This ContinuousReadSupport will *not* work in production applications due to + * A [[ContinuousStream]] that reads text lines through a TCP socket, designed only for tutorials + * and debugging. This ContinuousStream will *not* work in production applications due to * multiple reasons, including no support for fault recovery. * * The driver maintains a socket connection to the host-port, keeps the received messages in * buckets and serves the messages to the executors via a RPC endpoint. */ -class TextSocketContinuousReadSupport(options: DataSourceOptions) - extends ContinuousReadSupport with Logging { +class TextSocketContinuousStream( + host: String, port: Int, numPartitions: Int, options: CaseInsensitiveStringMap) + extends ContinuousStream with Logging { implicit val defaultFormats: DefaultFormats = DefaultFormats - private val host: String = options.get("host").get() - private val port: Int = options.get("port").get().toInt - - assert(SparkSession.getActiveSession.isDefined) - private val spark = SparkSession.getActiveSession.get - private val numPartitions = spark.sparkContext.defaultParallelism - @GuardedBy("this") private var socket: Socket = _ @@ -101,21 +93,9 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) startOffset } - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } - - override def fullSchema(): StructType = { - if (includeTimestamp) { - TextSocketReader.SCHEMA_TIMESTAMP - } else { - TextSocketReader.SCHEMA_REGULAR - } - } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig] - .start.asInstanceOf[TextSocketOffset] + override def planInputPartitions(start: Offset): Array[InputPartition] = { + val startOffset = start.asInstanceOf[TextSocketOffset] recordEndpoint.setStartOffsets(startOffset.offsets) val endpointName = s"TextSocketContinuousReaderEndpoint-${java.util.UUID.randomUUID()}" endpointRef = recordEndpoint.rpcEnv.setupEndpoint(endpointName, recordEndpoint) @@ -140,8 +120,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) }.toArray } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { TextSocketReaderFactory } @@ -197,7 +176,7 @@ class TextSocketContinuousReadSupport(options: DataSourceOptions) logWarning(s"Stream closed by $host:$port") return } - TextSocketContinuousReadSupport.this.synchronized { + TextSocketContinuousStream.this.synchronized { currentOffset += 1 val newData = (line, Timestamp.valueOf( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala deleted file mode 100644 index 90e1766c4d9f1..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/ContinuousTrigger.scala +++ /dev/null @@ -1,70 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming.continuous - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.commons.lang3.StringUtils - -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A [[Trigger]] that continuously processes streaming data, asynchronously checkpointing at - * the specified interval. - */ -@InterfaceStability.Evolving -case class ContinuousTrigger(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -private[sql] object ContinuousTrigger { - def apply(interval: String): ContinuousTrigger = { - if (StringUtils.isBlank(interval)) { - throw new IllegalArgumentException( - "interval cannot be null or blank.") - } - val cal = if (interval.startsWith("interval")) { - CalendarInterval.fromString(interval) - } else { - CalendarInterval.fromString("interval " + interval) - } - if (cal == null) { - throw new IllegalArgumentException(s"Invalid interval: $interval") - } - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ContinuousTrigger(cal.microseconds / 1000) - } - - def apply(interval: Duration): ContinuousTrigger = { - ContinuousTrigger(interval.toMillis) - } - - def create(interval: String): ContinuousTrigger = { - apply(interval) - } - - def create(interval: Long, unit: TimeUnit): ContinuousTrigger = { - ContinuousTrigger(unit.toMillis(interval)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala index 2238ce26e7b46..decf524f7167c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/EpochCoordinator.scala @@ -23,9 +23,9 @@ import org.apache.spark.SparkEnv import org.apache.spark.internal.Logging import org.apache.spark.rpc.{RpcCallContext, RpcEndpointRef, RpcEnv, ThreadSafeRpcEndpoint} import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.util.RpcUtils private[continuous] sealed trait EpochCoordinatorMessage extends Serializable @@ -82,15 +82,15 @@ private[sql] object EpochCoordinatorRef extends Logging { * Create a reference to a new [[EpochCoordinator]]. */ def create( - writeSupport: StreamingWriteSupport, - readSupport: ContinuousReadSupport, + writeSupport: StreamingWrite, + stream: ContinuousStream, query: ContinuousExecution, epochCoordinatorId: String, startEpoch: Long, session: SparkSession, env: SparkEnv): RpcEndpointRef = synchronized { val coordinator = new EpochCoordinator( - writeSupport, readSupport, query, startEpoch, session, env.rpcEnv) + writeSupport, stream, query, startEpoch, session, env.rpcEnv) val ref = env.rpcEnv.setupEndpoint(endpointName(epochCoordinatorId), coordinator) logInfo("Registered EpochCoordinator endpoint") ref @@ -115,14 +115,17 @@ private[sql] object EpochCoordinatorRef extends Logging { * have both committed and reported an end offset for a given epoch. */ private[continuous] class EpochCoordinator( - writeSupport: StreamingWriteSupport, - readSupport: ContinuousReadSupport, + writeSupport: StreamingWrite, + stream: ContinuousStream, query: ContinuousExecution, startEpoch: Long, session: SparkSession, override val rpcEnv: RpcEnv) extends ThreadSafeRpcEndpoint with Logging { + private val epochBacklogQueueSize = + session.sqlContext.conf.continuousStreamingEpochBacklogQueueSize + private var queryWritesStopped: Boolean = false private var numReaderPartitions: Int = _ @@ -212,6 +215,7 @@ private[continuous] class EpochCoordinator( if (!partitionCommits.isDefinedAt((epoch, partitionId))) { partitionCommits.put((epoch, partitionId), message) resolveCommitsAtEpoch(epoch) + checkProcessingQueueBoundaries() } case ReportPartitionOffset(partitionId, epoch, offset) => @@ -220,9 +224,25 @@ private[continuous] class EpochCoordinator( partitionOffsets.collect { case ((e, _), o) if e == epoch => o } if (thisEpochOffsets.size == numReaderPartitions) { logDebug(s"Epoch $epoch has offsets reported from all partitions: $thisEpochOffsets") - query.addOffset(epoch, readSupport, thisEpochOffsets.toSeq) + query.addOffset(epoch, stream, thisEpochOffsets.toSeq) resolveCommitsAtEpoch(epoch) } + checkProcessingQueueBoundaries() + } + + private def checkProcessingQueueBoundaries() = { + if (partitionOffsets.size > epochBacklogQueueSize) { + query.stopInNewThread(new IllegalStateException("Size of the partition offset queue has " + + "exceeded its maximum")) + } + if (partitionCommits.size > epochBacklogQueueSize) { + query.stopInNewThread(new IllegalStateException("Size of the partition commit queue has " + + "exceeded its maximum")) + } + if (epochsWaitingToBeCommitted.size > epochBacklogQueueSize) { + query.stopInNewThread(new IllegalStateException("Size of the epoch queue has " + + "exceeded its maximum")) + } } override def receiveAndReply(context: RpcCallContext): PartialFunction[Any, Unit] = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala index 7ad21cc304e7c..54f484c4adae3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSource.scala @@ -19,13 +19,13 @@ package org.apache.spark.sql.execution.streaming.continuous import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** * The logical plan for writing data in a continuous stream. */ -case class WriteToContinuousDataSource( - writeSupport: StreamingWriteSupport, query: LogicalPlan) extends LogicalPlan { +case class WriteToContinuousDataSource(write: StreamingWrite, query: LogicalPlan) + extends LogicalPlan { override def children: Seq[LogicalPlan] = Seq(query) override def output: Seq[Attribute] = Nil } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala index a797ac1879f41..2f3af6a6544c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/continuous/WriteToContinuousDataSourceExec.scala @@ -24,23 +24,24 @@ import org.apache.spark.internal.Logging import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.Attribute -import org.apache.spark.sql.execution.SparkPlan +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.streaming.StreamExecution -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite /** - * The physical plan for writing data into a continuous processing [[StreamingWriteSupport]]. + * The physical plan for writing data into a continuous processing [[StreamingWrite]]. */ -case class WriteToContinuousDataSourceExec(writeSupport: StreamingWriteSupport, query: SparkPlan) - extends SparkPlan with Logging { - override def children: Seq[SparkPlan] = Seq(query) +case class WriteToContinuousDataSourceExec(write: StreamingWrite, query: SparkPlan) + extends UnaryExecNode with Logging { + + override def child: SparkPlan = query override def output: Seq[Attribute] = Nil override protected def doExecute(): RDD[InternalRow] = { - val writerFactory = writeSupport.createStreamingWriterFactory() + val writerFactory = write.createStreamingWriterFactory() val rdd = new ContinuousWriteRDD(query.execute(), writerFactory) - logInfo(s"Start processing data source write support: $writeSupport. " + + logInfo(s"Start processing data source write support: $write. " + s"The input RDD has ${rdd.partitions.length} partitions.") EpochCoordinatorRef.get( sparkContext.getLocalProperty(ContinuousExecution.EPOCH_COORDINATOR_ID_KEY), diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala index adf52aba21a04..df149552dfb30 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala @@ -17,25 +17,26 @@ package org.apache.spark.sql.execution.streaming +import java.util import java.util.concurrent.atomic.AtomicInteger import javax.annotation.concurrent.GuardedBy -import scala.collection.mutable.{ArrayBuffer, ListBuffer} -import scala.util.control.NonFatal +import scala.collection.JavaConverters._ +import scala.collection.mutable.ListBuffer import org.apache.spark.internal.Logging import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.encoderFor -import org.apache.spark.sql.catalyst.expressions.{Attribute, UnsafeRow} -import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics} -import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils -import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._ +import org.apache.spark.sql.catalyst.expressions.UnsafeRow +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util.truncatedString +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2._ import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset => OffsetV2} -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream, Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils +import org.apache.spark.sql.util.CaseInsensitiveStringMap object MemoryStream { protected val currentBlockId = new AtomicInteger(0) @@ -48,8 +49,8 @@ object MemoryStream { /** * A base class for memory stream implementations. Supports adding data and resetting. */ -abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends BaseStreamingSource { - protected val encoder = encoderFor[A] +abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends SparkDataStream { + val encoder = encoderFor[A] protected val attributes = encoder.schema.toAttributes def toDS(): Dataset[A] = { @@ -60,15 +61,75 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas Dataset.ofRows(sqlContext.sparkSession, logicalPlan) } - def addData(data: A*): Offset = { + def addData(data: A*): OffsetV2 = { addData(data.toTraversable) } + def addData(data: TraversableOnce[A]): OffsetV2 + def fullSchema(): StructType = encoder.schema - protected def logicalPlan: LogicalPlan + protected val logicalPlan: LogicalPlan = { + StreamingRelationV2( + MemoryStreamTableProvider, + "memory", + new MemoryStreamTable(this), + CaseInsensitiveStringMap.empty(), + attributes, + None)(sqlContext.sparkSession) + } + + override def initialOffset(): OffsetV2 = { + throw new IllegalStateException("should not be called.") + } + + override def deserializeOffset(json: String): OffsetV2 = { + throw new IllegalStateException("should not be called.") + } + + override def commit(end: OffsetV2): Unit = { + throw new IllegalStateException("should not be called.") + } +} + +// This class is used to indicate the memory stream data source. We don't actually use it, as +// memory stream is for test only and we never look it up by name. +object MemoryStreamTableProvider extends TableProvider { + override def getTable(options: CaseInsensitiveStringMap): Table = { + throw new IllegalStateException("MemoryStreamTableProvider should not be used.") + } +} + +class MemoryStreamTable(val stream: MemoryStreamBase[_]) extends Table with SupportsRead { + + override def name(): String = "MemoryStreamDataSource" + + override def schema(): StructType = stream.fullSchema() + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.MICRO_BATCH_READ, TableCapability.CONTINUOUS_READ).asJava + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MemoryStreamScanBuilder(stream) + } +} + +class MemoryStreamScanBuilder(stream: MemoryStreamBase[_]) extends ScanBuilder with Scan { + + override def build(): Scan = this + + override def description(): String = "MemoryStreamDataSource" + + override def readSchema(): StructType = stream.fullSchema() + + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { + stream.asInstanceOf[MicroBatchStream] + } - def addData(data: TraversableOnce[A]): Offset + override def toContinuousStream(checkpointLocation: String): ContinuousStream = { + stream.asInstanceOf[ContinuousStream] + } } /** @@ -77,10 +138,8 @@ abstract class MemoryStreamBase[A : Encoder](sqlContext: SQLContext) extends Bas * available. */ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) - extends MemoryStreamBase[A](sqlContext) with MicroBatchReadSupport with Logging { + extends MemoryStreamBase[A](sqlContext) with MicroBatchStream with Logging { - protected val logicalPlan: LogicalPlan = - StreamingExecutionRelation(this, attributes)(sqlContext.sparkSession) protected val output = logicalPlan.output /** @@ -117,7 +176,9 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def toString: String = s"MemoryStream[${Utils.truncatedString(output, ",")}]" + override def toString: String = { + s"MemoryStream[${truncatedString(output, ",", SQLConf.get.maxToStringFields)}]" + } override def deserializeOffset(json: String): OffsetV2 = LongOffset(json.toLong) @@ -127,14 +188,9 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) if (currentOffset.offset == -1) null else currentOffset } - override def newScanConfigBuilder(start: OffsetV2, end: OffsetV2): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) - } - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startOffset = sc.start.asInstanceOf[LongOffset] - val endOffset = sc.end.get.asInstanceOf[LongOffset] + override def planInputPartitions(start: OffsetV2, end: OffsetV2): Array[InputPartition] = { + val startOffset = start.asInstanceOf[LongOffset] + val endOffset = end.asInstanceOf[LongOffset] synchronized { // Compute the internal batch numbers to fetch: [startOrdinal, endOrdinal) val startOrdinal = startOffset.offset.toInt + 1 @@ -156,7 +212,7 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { MemoryStreamReaderFactory } @@ -170,22 +226,15 @@ case class MemoryStream[A : Encoder](id: Int, sqlContext: SQLContext) } override def commit(end: OffsetV2): Unit = synchronized { - def check(newOffset: LongOffset): Unit = { - val offsetDiff = (newOffset.offset - lastOffsetCommitted.offset).toInt + val newOffset = end.asInstanceOf[LongOffset] + val offsetDiff = (newOffset.offset - lastOffsetCommitted.offset).toInt - if (offsetDiff < 0) { - sys.error(s"Offsets committed out of order: $lastOffsetCommitted followed by $end") - } - - batches.trimStart(offsetDiff) - lastOffsetCommitted = newOffset + if (offsetDiff < 0) { + sys.error(s"Offsets committed out of order: $lastOffsetCommitted followed by $end") } - LongOffset.convert(end) match { - case Some(lo) => check(lo) - case None => sys.error(s"MemoryStream.commit() received an offset ($end) " + - "that did not originate with an instance of this class") - } + batches.trimStart(offsetDiff) + lastOffsetCommitted = newOffset } override def stop() {} @@ -220,93 +269,3 @@ object MemoryStreamReaderFactory extends PartitionReaderFactory { } } } - -/** A common trait for MemorySinks with methods used for testing */ -trait MemorySinkBase extends BaseStreamingSink { - def allData: Seq[Row] - def latestBatchData: Seq[Row] - def dataSinceBatch(sinceBatchId: Long): Seq[Row] - def latestBatchId: Option[Long] -} - -/** - * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit - * tests and does not provide durability. - */ -class MemorySink(val schema: StructType, outputMode: OutputMode) extends Sink - with MemorySinkBase with Logging { - - private case class AddedData(batchId: Long, data: Array[Row]) - - /** An order list of batches that have been written to this [[Sink]]. */ - @GuardedBy("this") - private val batches = new ArrayBuffer[AddedData]() - - /** Returns all rows that are stored in this [[Sink]]. */ - def allData: Seq[Row] = synchronized { - batches.flatMap(_.data) - } - - def latestBatchId: Option[Long] = synchronized { - batches.lastOption.map(_.batchId) - } - - def latestBatchData: Seq[Row] = synchronized { batches.lastOption.toSeq.flatten(_.data) } - - def dataSinceBatch(sinceBatchId: Long): Seq[Row] = synchronized { - batches.filter(_.batchId > sinceBatchId).flatMap(_.data) - } - - def toDebugString: String = synchronized { - batches.map { case AddedData(batchId, data) => - val dataStr = try data.mkString(" ") catch { - case NonFatal(e) => "[Error converting to string]" - } - s"$batchId: $dataStr" - }.mkString("\n") - } - - override def addBatch(batchId: Long, data: DataFrame): Unit = { - val notCommitted = synchronized { - latestBatchId.isEmpty || batchId > latestBatchId.get - } - if (notCommitted) { - logDebug(s"Committing batch $batchId to $this") - outputMode match { - case Append | Update => - val rows = AddedData(batchId, data.collect()) - synchronized { batches += rows } - - case Complete => - val rows = AddedData(batchId, data.collect()) - synchronized { - batches.clear() - batches += rows - } - - case _ => - throw new IllegalArgumentException( - s"Output mode $outputMode is not supported by MemorySink") - } - } else { - logDebug(s"Skipping already committed batch: $batchId") - } - } - - def clear(): Unit = synchronized { - batches.clear() - } - - override def toString(): String = "MemorySink" -} - -/** - * Used to query the data that has been written into a [[MemorySink]]. - */ -case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode { - def this(sink: MemorySink) = this(sink, sink.schema.toAttributes) - - private val sizePerRow = EstimationUtils.getSizePerRow(sink.schema.toAttributes) - - override def computeStats(): Statistics = Statistics(sizePerRow * sink.allData.size) -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala similarity index 92% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala index 833e62f35ede1..dbe242784986d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWriteSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ConsoleWrite.scala @@ -20,14 +20,14 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.internal.Logging import org.apache.spark.sql.{Dataset, SparkSession} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** Common methods used to create writes for the the console sink */ -class ConsoleWriteSupport(schema: StructType, options: DataSourceOptions) - extends StreamingWriteSupport with Logging { +class ConsoleWrite(schema: StructType, options: CaseInsensitiveStringMap) + extends StreamingWrite with Logging { // Number of rows to display, by default 20 rows protected val numRowsToShow = options.getInt("numRows", 20) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala index dbcc4483e5770..41eaf84b7f9ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ContinuousMemoryStream.scala @@ -30,8 +30,7 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.{Encoder, SQLContext} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.streaming.{Offset => _, _} -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions} -import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig, ScanConfigBuilder} +import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.sources.v2.reader.streaming._ import org.apache.spark.util.RpcUtils @@ -44,14 +43,10 @@ import org.apache.spark.util.RpcUtils * the specified offset within the list, or null if that offset doesn't yet have a record. */ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPartitions: Int = 2) - extends MemoryStreamBase[A](sqlContext) - with ContinuousReadSupportProvider with ContinuousReadSupport { + extends MemoryStreamBase[A](sqlContext) with ContinuousStream { private implicit val formats = Serialization.formats(NoTypeHints) - protected val logicalPlan = - StreamingRelationV2(this, "memory", Map(), attributes, None)(sqlContext.sparkSession) - // ContinuousReader implementation @GuardedBy("this") @@ -86,13 +81,9 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa ) } - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start) - } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val startOffset = config.asInstanceOf[SimpleStreamingScanConfig] - .start.asInstanceOf[ContinuousMemoryStreamOffset] + override def planInputPartitions(start: Offset): Array[InputPartition] = { + val startOffset = start.asInstanceOf[ContinuousMemoryStreamOffset] synchronized { val endpointName = s"ContinuousMemoryStreamRecordEndpoint-${java.util.UUID.randomUUID()}-$id" endpointRef = @@ -104,8 +95,7 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa } } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { ContinuousMemoryStreamReaderFactory } @@ -114,12 +104,6 @@ class ContinuousMemoryStream[A : Encoder](id: Int, sqlContext: SQLContext, numPa } override def commit(end: Offset): Unit = {} - - // ContinuousReadSupportProvider implementation - // This is necessary because of how StreamTest finds the source for AddDataMemory steps. - override def createContinuousReadSupport( - checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = this } object ContinuousMemoryStream { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala similarity index 60% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala index 4218fd51ad206..6da1b3a49c442 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriteSupportProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/ForeachWriterTable.scala @@ -17,68 +17,88 @@ package org.apache.spark.sql.execution.streaming.sources +import java.util + +import scala.collection.JavaConverters._ + import org.apache.spark.sql.{ForeachWriter, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.python.PythonForeachWriter -import org.apache.spark.sql.sources.v2.{DataSourceOptions, StreamingWriteSupportProvider} -import org.apache.spark.sql.sources.v2.writer.{DataWriter, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} +import org.apache.spark.sql.sources.v2.writer.{DataWriter, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** - * A [[org.apache.spark.sql.sources.v2.DataSourceV2]] for forwarding data into the specified - * [[ForeachWriter]]. + * A write-only table for forwarding data into the specified [[ForeachWriter]]. * * @param writer The [[ForeachWriter]] to process all data. * @param converter An object to convert internal rows to target type T. Either it can be * a [[ExpressionEncoder]] or a direct converter function. * @tparam T The expected type of the sink. */ -case class ForeachWriteSupportProvider[T]( +case class ForeachWriterTable[T]( writer: ForeachWriter[T], converter: Either[ExpressionEncoder[T], InternalRow => T]) - extends StreamingWriteSupportProvider { - - override def createStreamingWriteSupport( - queryId: String, - schema: StructType, - mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new StreamingWriteSupport { - override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} - - override def createStreamingWriterFactory(): StreamingDataWriterFactory = { - val rowConverter: InternalRow => T = converter match { - case Left(enc) => - val boundEnc = enc.resolveAndBind( - schema.toAttributes, - SparkSession.getActiveSession.get.sessionState.analyzer) - boundEnc.fromRow - case Right(func) => - func - } - ForeachWriterFactory(writer, rowConverter) + extends Table with SupportsWrite { + + override def name(): String = "ForeachSink" + + override def schema(): StructType = StructType(Nil) + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.STREAMING_WRITE).asJava + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new WriteBuilder with SupportsTruncate { + private var inputSchema: StructType = _ + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + this.inputSchema = schema + this } - override def toString: String = "ForeachSink" + // Do nothing for truncate. Foreach sink is special that it just forwards all the records to + // ForeachWriter. + override def truncate(): WriteBuilder = this + + override def buildForStreaming(): StreamingWrite = { + new StreamingWrite { + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} + override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = {} + + override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + val rowConverter: InternalRow => T = converter match { + case Left(enc) => + val boundEnc = enc.resolveAndBind( + inputSchema.toAttributes, + SparkSession.getActiveSession.get.sessionState.analyzer) + boundEnc.fromRow + case Right(func) => + func + } + ForeachWriterFactory(writer, rowConverter) + } + } + } } } } -object ForeachWriteSupportProvider { +object ForeachWriterTable { def apply[T]( writer: ForeachWriter[T], - encoder: ExpressionEncoder[T]): ForeachWriteSupportProvider[_] = { + encoder: ExpressionEncoder[T]): ForeachWriterTable[_] = { writer match { case pythonWriter: PythonForeachWriter => - new ForeachWriteSupportProvider[UnsafeRow]( + new ForeachWriterTable[UnsafeRow]( pythonWriter, Right((x: InternalRow) => x.asInstanceOf[UnsafeRow])) case _ => - new ForeachWriteSupportProvider[T](writer, Left(encoder)) + new ForeachWriterTable[T](writer, Left(encoder)) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala similarity index 82% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala index 9f88416871f8e..f3951897ea747 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWritSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/MicroBatchWrite.scala @@ -18,16 +18,15 @@ package org.apache.spark.sql.execution.streaming.sources import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} /** - * A [[BatchWriteSupport]] used to hook V2 stream writers into a microbatch plan. It implements + * A [[BatchWrite]] used to hook V2 stream writers into a microbatch plan. It implements * the non-streaming interface, forwarding the epoch ID determined at construction to a wrapped * streaming write support. */ -class MicroBatchWritSupport(eppchId: Long, val writeSupport: StreamingWriteSupport) - extends BatchWriteSupport { +class MicroBatchWrite(eppchId: Long, val writeSupport: StreamingWrite) extends BatchWrite { override def commit(messages: Array[WriterCommitMessage]): Unit = { writeSupport.commit(eppchId, messages) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala index ac3c71cc222b1..fd4cb444ce580 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/PackedRowWriterFactory.scala @@ -21,12 +21,12 @@ import scala.collection.mutable import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.sources.v2.writer.{BatchWriteSupport, DataWriter, DataWriterFactory, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, WriterCommitMessage} import org.apache.spark.sql.sources.v2.writer.streaming.StreamingDataWriterFactory /** * A simple [[DataWriterFactory]] whose tasks just pack rows into the commit message for delivery - * to a [[BatchWriteSupport]] on the driver. + * to a [[BatchWrite]] on the driver. * * Note that, because it sends all rows to the driver, this factory will generally be unsuitable * for production-quality sinks. It's intended for use in tests. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala similarity index 86% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala index 90680ea38fbd6..6a66f52c8f732 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateControlMicroBatchStream.scala @@ -17,10 +17,10 @@ package org.apache.spark.sql.execution.streaming.sources -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} -// A special `MicroBatchReadSupport` that can get latestOffset with a start offset. -trait RateControlMicroBatchReadSupport extends MicroBatchReadSupport { +// A special `MicroBatchStream` that can get latestOffset with a start offset. +trait RateControlMicroBatchStream extends MicroBatchStream { override def latestOffset(): Offset = { throw new IllegalAccessException( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala similarity index 80% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala index f5364047adff1..156ba95ab9733 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamMicroBatchStream.scala @@ -24,19 +24,23 @@ import java.util.concurrent.TimeUnit import org.apache.commons.io.IOUtils import org.apache.spark.internal.Logging -import org.apache.spark.network.util.JavaUtils import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.sources.v2.DataSourceOptions import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchReadSupport, Offset} -import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.{ManualClock, SystemClock} -class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLocation: String) - extends MicroBatchReadSupport with Logging { +class RateStreamMicroBatchStream( + rowsPerSecond: Long, + // The default values here are used in tests. + rampUpTimeSeconds: Long = 0, + numPartitions: Int = 1, + options: CaseInsensitiveStringMap, + checkpointLocation: String) + extends MicroBatchStream with Logging { import RateStreamProvider._ private[sources] val clock = { @@ -44,14 +48,6 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca if (options.getBoolean("useManualClock", false)) new ManualClock else new SystemClock } - private val rowsPerSecond = - options.get(ROWS_PER_SECOND).orElse("1").toLong - - private val rampUpTimeSeconds = - Option(options.get(RAMP_UP_TIME).orElse(null.asInstanceOf[String])) - .map(JavaUtils.timeStringAsSec(_)) - .getOrElse(0L) - private val maxSeconds = Long.MaxValue / rowsPerSecond if (rampUpTimeSeconds > maxSeconds) { @@ -80,7 +76,7 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca if (content(0) == 'v') { val indexOfNewLine = content.indexOf("\n") if (indexOfNewLine > 0) { - parseVersion(content.substring(0, indexOfNewLine), VERSION) + validateVersion(content.substring(0, indexOfNewLine), VERSION) LongOffset(SerializedOffset(content.substring(indexOfNewLine + 1))) } else { throw new IllegalStateException( @@ -117,16 +113,10 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca LongOffset(json.toLong) } - override def fullSchema(): StructType = SCHEMA - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) - } - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startSeconds = sc.start.asInstanceOf[LongOffset].offset - val endSeconds = sc.end.get.asInstanceOf[LongOffset].offset + override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = { + val startSeconds = start.asInstanceOf[LongOffset].offset + val endSeconds = end.asInstanceOf[LongOffset].offset assert(startSeconds <= endSeconds, s"startSeconds($startSeconds) > endSeconds($endSeconds)") if (endSeconds > maxSeconds) { throw new ArithmeticException("Integer overflow. Max offset with " + @@ -148,21 +138,14 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca val localStartTimeMs = creationTimeMs + TimeUnit.SECONDS.toMillis(startSeconds) val relativeMsPerValue = TimeUnit.SECONDS.toMillis(endSeconds - startSeconds).toDouble / (rangeEnd - rangeStart) - val numPartitions = { - val activeSession = SparkSession.getActiveSession - require(activeSession.isDefined) - Option(options.get(NUM_PARTITIONS).orElse(null.asInstanceOf[String])) - .map(_.toInt) - .getOrElse(activeSession.get.sparkContext.defaultParallelism) - } (0 until numPartitions).map { p => - new RateStreamMicroBatchInputPartition( + RateStreamMicroBatchInputPartition( p, numPartitions, rangeStart, rangeEnd, localStartTimeMs, relativeMsPerValue) }.toArray } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { RateStreamMicroBatchReaderFactory } @@ -172,7 +155,7 @@ class RateStreamMicroBatchReadSupport(options: DataSourceOptions, checkpointLoca override def toString: String = s"RateStreamV2[rowsPerSecond=$rowsPerSecond, " + s"rampUpTimeSeconds=$rampUpTimeSeconds, " + - s"numPartitions=${options.get(NUM_PARTITIONS).orElse("default")}" + s"numPartitions=${options.getOrDefault(NUM_PARTITIONS, "default")}" } case class RateStreamMicroBatchInputPartition( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala index 6942dfbfe0ecf..f61e9dbecd4ea 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProvider.scala @@ -17,12 +17,19 @@ package org.apache.spark.sql.execution.streaming.sources +import java.util + +import scala.collection.JavaConverters._ + import org.apache.spark.network.util.JavaUtils -import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousReadSupport +import org.apache.spark.sql.SparkSession +import org.apache.spark.sql.execution.streaming.continuous.RateStreamContinuousStream import org.apache.spark.sql.sources.DataSourceRegister import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * A source that generates increment long values with timestamps. Each generated row has two @@ -38,48 +45,63 @@ import org.apache.spark.sql.types._ * generated rows. The source will try its best to reach `rowsPerSecond`, but the query may * be resource constrained, and `numPartitions` can be tweaked to help reach the desired speed. */ -class RateStreamProvider extends DataSourceV2 - with MicroBatchReadSupportProvider with ContinuousReadSupportProvider with DataSourceRegister { +class RateStreamProvider extends TableProvider with DataSourceRegister { import RateStreamProvider._ - override def createMicroBatchReadSupport( - checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = { - if (options.get(ROWS_PER_SECOND).isPresent) { - val rowsPerSecond = options.get(ROWS_PER_SECOND).get().toLong - if (rowsPerSecond <= 0) { - throw new IllegalArgumentException( - s"Invalid value '$rowsPerSecond'. The option 'rowsPerSecond' must be positive") - } + override def getTable(options: CaseInsensitiveStringMap): Table = { + val rowsPerSecond = options.getLong(ROWS_PER_SECOND, 1) + if (rowsPerSecond <= 0) { + throw new IllegalArgumentException( + s"Invalid value '$rowsPerSecond'. The option 'rowsPerSecond' must be positive") } - if (options.get(RAMP_UP_TIME).isPresent) { - val rampUpTimeSeconds = - JavaUtils.timeStringAsSec(options.get(RAMP_UP_TIME).get()) - if (rampUpTimeSeconds < 0) { - throw new IllegalArgumentException( - s"Invalid value '$rampUpTimeSeconds'. The option 'rampUpTime' must not be negative") - } + val rampUpTimeSeconds = Option(options.get(RAMP_UP_TIME)) + .map(JavaUtils.timeStringAsSec) + .getOrElse(0L) + if (rampUpTimeSeconds < 0) { + throw new IllegalArgumentException( + s"Invalid value '$rampUpTimeSeconds'. The option 'rampUpTime' must not be negative") } - if (options.get(NUM_PARTITIONS).isPresent) { - val numPartitions = options.get(NUM_PARTITIONS).get().toInt - if (numPartitions <= 0) { - throw new IllegalArgumentException( - s"Invalid value '$numPartitions'. The option 'numPartitions' must be positive") - } + val numPartitions = options.getInt( + NUM_PARTITIONS, SparkSession.active.sparkContext.defaultParallelism) + if (numPartitions <= 0) { + throw new IllegalArgumentException( + s"Invalid value '$numPartitions'. The option 'numPartitions' must be positive") } + new RateStreamTable(rowsPerSecond, rampUpTimeSeconds, numPartitions) + } - new RateStreamMicroBatchReadSupport(options, checkpointLocation) + override def shortName(): String = "rate" +} + +class RateStreamTable( + rowsPerSecond: Long, + rampUpTimeSeconds: Long, + numPartitions: Int) + extends Table with SupportsRead { + + override def name(): String = { + s"RateStream(rowsPerSecond=$rowsPerSecond, rampUpTimeSeconds=$rampUpTimeSeconds, " + + s"numPartitions=$numPartitions)" } - override def createContinuousReadSupport( - checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = { - new RateStreamContinuousReadSupport(options) + override def schema(): StructType = RateStreamProvider.SCHEMA + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.MICRO_BATCH_READ, TableCapability.CONTINUOUS_READ).asJava } - override def shortName(): String = "rate" + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = () => new Scan { + override def readSchema(): StructType = RateStreamProvider.SCHEMA + + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = + new RateStreamMicroBatchStream( + rowsPerSecond, rampUpTimeSeconds, numPartitions, options, checkpointLocation) + + override def toContinuousStream(checkpointLocation: String): ContinuousStream = + new RateStreamContinuousStream(rowsPerSecond, numPartitions) + } } object RateStreamProvider { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala similarity index 52% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala index b2a573eae504a..25e9af2bc2927 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/socket.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketMicroBatchStream.scala @@ -19,44 +19,27 @@ package org.apache.spark.sql.execution.streaming.sources import java.io.{BufferedReader, InputStreamReader, IOException} import java.net.Socket -import java.text.SimpleDateFormat -import java.util.{Calendar, Locale} +import java.util.Calendar import java.util.concurrent.atomic.AtomicBoolean import javax.annotation.concurrent.GuardedBy import scala.collection.mutable.ListBuffer -import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging -import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.execution.streaming.{LongOffset, SimpleStreamingScanConfig, SimpleStreamingScanConfigBuilder} -import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousReadSupport -import org.apache.spark.sql.sources.DataSourceRegister -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, DataSourceV2, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport, Offset} -import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.execution.streaming.LongOffset +import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReader, PartitionReaderFactory} +import org.apache.spark.sql.sources.v2.reader.streaming.{MicroBatchStream, Offset} import org.apache.spark.unsafe.types.UTF8String -object TextSocketReader { - val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) - val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: - StructField("timestamp", TimestampType) :: Nil) - val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) -} - /** * A MicroBatchReadSupport that reads text lines through a TCP socket, designed only for tutorials * and debugging. This MicroBatchReadSupport will *not* work in production applications due to * multiple reasons, including no support for fault recovery. */ -class TextSocketMicroBatchReadSupport(options: DataSourceOptions) - extends MicroBatchReadSupport with Logging { - - private val host: String = options.get("host").get() - private val port: Int = options.get("port").get().toInt +class TextSocketMicroBatchStream(host: String, port: Int, numPartitions: Int) + extends MicroBatchStream with Logging { @GuardedBy("this") private var socket: Socket = null @@ -99,7 +82,7 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) logWarning(s"Stream closed by $host:$port") return } - TextSocketMicroBatchReadSupport.this.synchronized { + TextSocketMicroBatchStream.this.synchronized { val newData = ( UTF8String.fromString(line), DateTimeUtils.fromMillis(Calendar.getInstance().getTimeInMillis) @@ -124,22 +107,9 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) LongOffset(json.toLong) } - override def fullSchema(): StructType = { - if (options.getBoolean("includeTimestamp", false)) { - TextSocketReader.SCHEMA_TIMESTAMP - } else { - TextSocketReader.SCHEMA_REGULAR - } - } - - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = { - new SimpleStreamingScanConfigBuilder(fullSchema(), start, Some(end)) - } - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val sc = config.asInstanceOf[SimpleStreamingScanConfig] - val startOrdinal = sc.start.asInstanceOf[LongOffset].offset.toInt + 1 - val endOrdinal = sc.end.get.asInstanceOf[LongOffset].offset.toInt + 1 + override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = { + val startOrdinal = start.asInstanceOf[LongOffset].offset.toInt + 1 + val endOrdinal = end.asInstanceOf[LongOffset].offset.toInt + 1 // Internal buffer only holds the batches after lastOffsetCommitted val rawList = synchronized { @@ -152,10 +122,6 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) batches.slice(sliceStart, sliceEnd) } - assert(SparkSession.getActiveSession.isDefined) - val spark = SparkSession.getActiveSession.get - val numPartitions = spark.sparkContext.defaultParallelism - val slices = Array.fill(numPartitions)(new ListBuffer[(UTF8String, Long)]) rawList.zipWithIndex.foreach { case (r, idx) => slices(idx % numPartitions).append(r) @@ -164,33 +130,27 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) slices.map(TextSocketInputPartition) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - new PartitionReaderFactory { - override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { - val slice = partition.asInstanceOf[TextSocketInputPartition].slice - new PartitionReader[InternalRow] { - private var currentIdx = -1 - - override def next(): Boolean = { - currentIdx += 1 - currentIdx < slice.size - } + override def createReaderFactory(): PartitionReaderFactory = + (partition: InputPartition) => { + val slice = partition.asInstanceOf[TextSocketInputPartition].slice + new PartitionReader[InternalRow] { + private var currentIdx = -1 - override def get(): InternalRow = { - InternalRow(slice(currentIdx)._1, slice(currentIdx)._2) - } + override def next(): Boolean = { + currentIdx += 1 + currentIdx < slice.size + } - override def close(): Unit = {} + override def get(): InternalRow = { + InternalRow(slice(currentIdx)._1, slice(currentIdx)._2) } + + override def close(): Unit = {} } } - } override def commit(end: Offset): Unit = synchronized { - val newOffset = LongOffset.convert(end).getOrElse( - sys.error(s"TextSocketStream.commit() received an offset ($end) that did not " + - s"originate with an instance of this class") - ) + val newOffset = end.asInstanceOf[LongOffset] val offsetDiff = (newOffset.offset - lastOffsetCommitted.offset).toInt @@ -220,43 +180,3 @@ class TextSocketMicroBatchReadSupport(options: DataSourceOptions) } case class TextSocketInputPartition(slice: ListBuffer[(UTF8String, Long)]) extends InputPartition - -class TextSocketSourceProvider extends DataSourceV2 - with MicroBatchReadSupportProvider with ContinuousReadSupportProvider - with DataSourceRegister with Logging { - - private def checkParameters(params: DataSourceOptions): Unit = { - logWarning("The socket source should not be used for production applications! " + - "It does not support recovery.") - if (!params.get("host").isPresent) { - throw new AnalysisException("Set a host to read from with option(\"host\", ...).") - } - if (!params.get("port").isPresent) { - throw new AnalysisException("Set a port to read from with option(\"port\", ...).") - } - Try { - params.get("includeTimestamp").orElse("false").toBoolean - } match { - case Success(_) => - case Failure(_) => - throw new AnalysisException("includeTimestamp must be set to either \"true\" or \"false\"") - } - } - - override def createMicroBatchReadSupport( - checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = { - checkParameters(options) - new TextSocketMicroBatchReadSupport(options) - } - - override def createContinuousReadSupport( - checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = { - checkParameters(options) - new TextSocketContinuousReadSupport(options) - } - - /** String that represents the format that this data source provider uses. */ - override def shortName(): String = "socket" -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala new file mode 100644 index 0000000000000..0f807e235661a --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketSourceProvider.scala @@ -0,0 +1,105 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.streaming.sources + +import java.text.SimpleDateFormat +import java.util +import java.util.Locale + +import scala.collection.JavaConverters._ +import scala.util.{Failure, Success, Try} + +import org.apache.spark.internal.Logging +import org.apache.spark.sql._ +import org.apache.spark.sql.execution.streaming.continuous.TextSocketContinuousStream +import org.apache.spark.sql.sources.DataSourceRegister +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.reader.{Scan, ScanBuilder} +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, MicroBatchStream} +import org.apache.spark.sql.types.{StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class TextSocketSourceProvider extends TableProvider with DataSourceRegister with Logging { + + private def checkParameters(params: CaseInsensitiveStringMap): Unit = { + logWarning("The socket source should not be used for production applications! " + + "It does not support recovery.") + if (!params.containsKey("host")) { + throw new AnalysisException("Set a host to read from with option(\"host\", ...).") + } + if (!params.containsKey("port")) { + throw new AnalysisException("Set a port to read from with option(\"port\", ...).") + } + Try { + params.getBoolean("includeTimestamp", false) + } match { + case Success(_) => + case Failure(_) => + throw new AnalysisException("includeTimestamp must be set to either \"true\" or \"false\"") + } + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + checkParameters(options) + new TextSocketTable( + options.get("host"), + options.getInt("port", -1), + options.getInt("numPartitions", SparkSession.active.sparkContext.defaultParallelism), + options.getBoolean("includeTimestamp", false)) + } + + /** String that represents the format that this data source provider uses. */ + override def shortName(): String = "socket" +} + +class TextSocketTable(host: String, port: Int, numPartitions: Int, includeTimestamp: Boolean) + extends Table with SupportsRead { + + override def name(): String = s"Socket[$host:$port]" + + override def schema(): StructType = { + if (includeTimestamp) { + TextSocketReader.SCHEMA_TIMESTAMP + } else { + TextSocketReader.SCHEMA_REGULAR + } + } + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.MICRO_BATCH_READ, TableCapability.CONTINUOUS_READ).asJava + } + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = () => new Scan { + override def readSchema(): StructType = schema() + + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = { + new TextSocketMicroBatchStream(host, port, numPartitions) + } + + override def toContinuousStream(checkpointLocation: String): ContinuousStream = { + new TextSocketContinuousStream(host, port, numPartitions, options) + } + } +} + +object TextSocketReader { + val SCHEMA_REGULAR = StructType(StructField("value", StringType) :: Nil) + val SCHEMA_TIMESTAMP = StructType(StructField("value", StringType) :: + StructField("timestamp", TimestampType) :: Nil) + val DATE_FORMAT = new SimpleDateFormat("yyyy-MM-dd HH:mm:ss", Locale.US) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala new file mode 100644 index 0000000000000..a3f58fa966fe8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/WriteToMicroBatchDataSource.scala @@ -0,0 +1,39 @@ +/* + * 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.streaming.sources + +import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.datasources.v2.WriteToDataSourceV2 +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite + +/** + * The logical plan for writing data to a micro-batch stream. + * + * Note that this logical plan does not have a corresponding physical plan, as it will be converted + * to [[WriteToDataSourceV2]] with [[MicroBatchWrite]] before execution. + */ +case class WriteToMicroBatchDataSource(write: StreamingWrite, query: LogicalPlan) + extends LogicalPlan { + override def children: Seq[LogicalPlan] = Seq(query) + override def output: Seq[Attribute] = Nil + + def createPlan(batchId: Long): WriteToDataSourceV2 = { + WriteToDataSourceV2(new MicroBatchWrite(batchId, write), query) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala similarity index 69% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala index c50dc7bcb8da1..de8d00d4ac348 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memoryV2.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/sources/memory.scala @@ -17,8 +17,10 @@ package org.apache.spark.sql.execution.streaming.sources +import java.util import javax.annotation.concurrent.GuardedBy +import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.util.control.NonFatal @@ -30,27 +32,46 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics} import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils -import org.apache.spark.sql.catalyst.streaming.InternalOutputModes.{Append, Complete, Update} -import org.apache.spark.sql.execution.streaming.{MemorySinkBase, Sink} -import org.apache.spark.sql.sources.v2.{DataSourceOptions, DataSourceV2, StreamingWriteSupportProvider} +import org.apache.spark.sql.execution.streaming.Sink +import org.apache.spark.sql.sources.v2.{SupportsWrite, Table, TableCapability} import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWriteSupport} -import org.apache.spark.sql.streaming.OutputMode +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * A sink that stores the results in memory. This [[Sink]] is primarily intended for use in unit * tests and does not provide durability. */ -class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider - with MemorySinkBase with Logging { +class MemorySink extends Table with SupportsWrite with Logging { - override def createStreamingWriteSupport( - queryId: String, - schema: StructType, - mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - new MemoryStreamingWriteSupport(this, mode, schema) + override def name(): String = "MemorySink" + + override def schema(): StructType = StructType(Nil) + + override def capabilities(): util.Set[TableCapability] = { + Set(TableCapability.STREAMING_WRITE).asJava + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new WriteBuilder with SupportsTruncate { + private var needTruncate: Boolean = false + private var inputSchema: StructType = _ + + override def truncate(): WriteBuilder = { + this.needTruncate = true + this + } + + override def withInputDataSchema(schema: StructType): WriteBuilder = { + this.inputSchema = schema + this + } + + override def buildForStreaming(): StreamingWrite = { + new MemoryStreamingWrite(MemorySink.this, inputSchema, needTruncate) + } + } } private case class AddedData(batchId: Long, data: Array[Row]) @@ -85,27 +106,20 @@ class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider }.mkString("\n") } - def write(batchId: Long, outputMode: OutputMode, newRows: Array[Row]): Unit = { + def write(batchId: Long, needTruncate: Boolean, newRows: Array[Row]): Unit = { val notCommitted = synchronized { latestBatchId.isEmpty || batchId > latestBatchId.get } if (notCommitted) { logDebug(s"Committing batch $batchId to $this") - outputMode match { - case Append | Update => - val rows = AddedData(batchId, newRows) - synchronized { batches += rows } - - case Complete => - val rows = AddedData(batchId, newRows) - synchronized { - batches.clear() - batches += rows - } - - case _ => - throw new IllegalArgumentException( - s"Output mode $outputMode is not supported by MemorySinkV2") + val rows = AddedData(batchId, newRows) + if (needTruncate) { + synchronized { + batches.clear() + batches += rows + } + } else { + synchronized { batches += rows } } } else { logDebug(s"Skipping already committed batch: $batchId") @@ -116,25 +130,25 @@ class MemorySinkV2 extends DataSourceV2 with StreamingWriteSupportProvider batches.clear() } - override def toString(): String = "MemorySinkV2" + override def toString(): String = "MemorySink" } case class MemoryWriterCommitMessage(partition: Int, data: Seq[Row]) extends WriterCommitMessage {} -class MemoryStreamingWriteSupport( - val sink: MemorySinkV2, outputMode: OutputMode, schema: StructType) - extends StreamingWriteSupport { +class MemoryStreamingWrite( + val sink: MemorySink, schema: StructType, needTruncate: Boolean) + extends StreamingWrite { override def createStreamingWriterFactory: MemoryWriterFactory = { - MemoryWriterFactory(outputMode, schema) + MemoryWriterFactory(schema) } override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { val newRows = messages.flatMap { case message: MemoryWriterCommitMessage => message.data } - sink.write(epochId, outputMode, newRows) + sink.write(epochId, needTruncate, newRows) } override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { @@ -142,13 +156,13 @@ class MemoryStreamingWriteSupport( } } -case class MemoryWriterFactory(outputMode: OutputMode, schema: StructType) +case class MemoryWriterFactory(schema: StructType) extends DataWriterFactory with StreamingDataWriterFactory { override def createWriter( partitionId: Int, taskId: Long): DataWriter[InternalRow] = { - new MemoryDataWriter(partitionId, outputMode, schema) + new MemoryDataWriter(partitionId, schema) } override def createWriter( @@ -159,7 +173,7 @@ case class MemoryWriterFactory(outputMode: OutputMode, schema: StructType) } } -class MemoryDataWriter(partition: Int, outputMode: OutputMode, schema: StructType) +class MemoryDataWriter(partition: Int, schema: StructType) extends DataWriter[InternalRow] with Logging { private val data = mutable.Buffer[Row]() @@ -181,9 +195,9 @@ class MemoryDataWriter(partition: Int, outputMode: OutputMode, schema: StructTyp /** - * Used to query the data that has been written into a [[MemorySinkV2]]. + * Used to query the data that has been written into a [[MemorySink]]. */ -case class MemoryPlanV2(sink: MemorySinkV2, override val output: Seq[Attribute]) extends LeafNode { +case class MemoryPlan(sink: MemorySink, override val output: Seq[Attribute]) extends LeafNode { private val sizePerRow = EstimationUtils.getSizePerRow(output) override def computeStats(): Statistics = Statistics(sizePerRow * sink.allData.size) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala index 92a2480e8b017..6ee54b948a7d4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/HDFSBackedStateStoreProvider.scala @@ -629,7 +629,7 @@ private[state] class HDFSBackedStateStoreProvider extends StateStoreProvider wit require(allFiles.exists(_.version == version)) val latestSnapshotFileBeforeVersion = allFiles - .filter(_.isSnapshot == true) + .filter(_.isSnapshot) .takeWhile(_.version <= version) .lastOption val deltaBatchFiles = latestSnapshotFileBeforeVersion match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala index d3313b8a315c9..7d80fd0c591f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/state/StateStore.scala @@ -213,7 +213,7 @@ object StateStoreProvider { */ def create(providerClassName: String): StateStoreProvider = { val providerClass = Utils.classForName(providerClassName) - providerClass.newInstance().asInstanceOf[StateStoreProvider] + providerClass.getConstructor().newInstance().asInstanceOf[StateStoreProvider] } /** @@ -462,8 +462,7 @@ object StateStore extends Logging { val env = SparkEnv.get if (env != null) { val isDriver = - env.executorId == SparkContext.DRIVER_IDENTIFIER || - env.executorId == SparkContext.LEGACY_DRIVER_IDENTIFIER + env.executorId == SparkContext.DRIVER_IDENTIFIER // If running locally, then the coordinator reference in _coordRef may be have become inactive // as SparkContext + SparkEnv may have been restarted. Hence, when running in driver, // always recreate the reference. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala index c11af345b0248..d689a6f3c9819 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/statefulOperators.scala @@ -79,8 +79,8 @@ trait StateStoreWriter extends StatefulOperator { self: SparkPlan => "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"), "numTotalStateRows" -> SQLMetrics.createMetric(sparkContext, "number of total state rows"), "numUpdatedStateRows" -> SQLMetrics.createMetric(sparkContext, "number of updated state rows"), - "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "total time to update rows"), - "allRemovalsTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "total time to remove rows"), + "allUpdatesTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to update"), + "allRemovalsTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to remove"), "commitTimeMs" -> SQLMetrics.createTimingMetric(sparkContext, "time to commit changes"), "stateMemory" -> SQLMetrics.createSizeMetric(sparkContext, "memory used by state") ) ++ stateStoreCustomMetrics diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala index 310ebcdf67686..064e0a098955c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/subquery.scala @@ -22,7 +22,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.{expressions, InternalRow} -import org.apache.spark.sql.catalyst.expressions.{Expression, ExprId, InSet, Literal, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{AttributeSeq, Expression, ExprId, InSet, Literal, PlanExpression} import org.apache.spark.sql.catalyst.expressions.codegen.{CodegenContext, ExprCode} import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.internal.SQLConf @@ -31,11 +31,31 @@ import org.apache.spark.sql.types.{BooleanType, DataType, StructType} /** * The base class for subquery that is used in SparkPlan. */ -abstract class ExecSubqueryExpression extends PlanExpression[SubqueryExec] { +abstract class ExecSubqueryExpression extends PlanExpression[BaseSubqueryExec] { /** * Fill the expression with collected result from executed plan. */ def updateResult(): Unit + + /** Updates the expression with a new plan. */ + override def withNewPlan(plan: BaseSubqueryExec): ExecSubqueryExpression + + override def canonicalize(attrs: AttributeSeq): ExecSubqueryExpression = { + withNewPlan(plan.canonicalized.asInstanceOf[BaseSubqueryExec]) + .asInstanceOf[ExecSubqueryExpression] + } +} + +object ExecSubqueryExpression { + /** + * Returns true when an expression contains a subquery + */ + def hasSubquery(e: Expression): Boolean = { + e.find { + case _: ExecSubqueryExpression => true + case _ => false + }.isDefined + } } /** @@ -44,15 +64,15 @@ abstract class ExecSubqueryExpression extends PlanExpression[SubqueryExec] { * This is the physical copy of ScalarSubquery to be used inside SparkPlan. */ case class ScalarSubquery( - plan: SubqueryExec, + plan: BaseSubqueryExec, exprId: ExprId) extends ExecSubqueryExpression { override def dataType: DataType = plan.schema.fields.head.dataType override def children: Seq[Expression] = Nil override def nullable: Boolean = true - override def toString: String = plan.simpleString - override def withNewPlan(query: SubqueryExec): ScalarSubquery = copy(plan = query) + override def toString: String = plan.simpleString(SQLConf.get.maxToStringFields) + override def withNewPlan(query: BaseSubqueryExec): ScalarSubquery = copy(plan = query) override def semanticEquals(other: Expression): Boolean = other match { case s: ScalarSubquery => plan.sameResult(s.plan) @@ -99,7 +119,7 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { case subquery: expressions.ScalarSubquery => val executedPlan = new QueryExecution(sparkSession, subquery.plan).executedPlan ScalarSubquery( - SubqueryExec(s"subquery${subquery.exprId.id}", executedPlan), + SubqueryExec(s"scalar-subquery#${subquery.exprId.id}", executedPlan), subquery.exprId) } } @@ -113,17 +133,18 @@ case class PlanSubqueries(sparkSession: SparkSession) extends Rule[SparkPlan] { case class ReuseSubquery(conf: SQLConf) extends Rule[SparkPlan] { def apply(plan: SparkPlan): SparkPlan = { - if (!conf.exchangeReuseEnabled) { + if (!conf.subqueryReuseEnabled) { return plan } // Build a hash map using schema of subqueries to avoid O(N*N) sameResult calls. - val subqueries = mutable.HashMap[StructType, ArrayBuffer[SubqueryExec]]() + val subqueries = mutable.HashMap[StructType, ArrayBuffer[BaseSubqueryExec]]() plan transformAllExpressions { case sub: ExecSubqueryExpression => - val sameSchema = subqueries.getOrElseUpdate(sub.plan.schema, ArrayBuffer[SubqueryExec]()) + val sameSchema = + subqueries.getOrElseUpdate(sub.plan.schema, ArrayBuffer[BaseSubqueryExec]()) val sameResult = sameSchema.find(_.sameResult(sub.plan)) if (sameResult.isDefined) { - sub.withNewPlan(sameResult.get) + sub.withNewPlan(ReusedSubqueryExec(sameResult.get)) } else { sameSchema += sub.plan sub diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala index 1b2d8a821b364..824c094220e82 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/AllExecutionsPage.scala @@ -17,16 +17,17 @@ package org.apache.spark.sql.execution.ui +import java.net.URLEncoder import javax.servlet.http.HttpServletRequest +import scala.collection.JavaConverters._ import scala.collection.mutable -import scala.xml.{Node, NodeSeq} - -import org.apache.commons.lang3.StringEscapeUtils +import scala.xml.{Node, NodeSeq, Unparsed} import org.apache.spark.JobExecutionStatus import org.apache.spark.internal.Logging -import org.apache.spark.ui.{UIUtils, WebUIPage} +import org.apache.spark.ui.{PagedDataSource, PagedTable, UIUtils, WebUIPage} +import org.apache.spark.util.Utils private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with Logging { @@ -55,8 +56,8 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L val _content = mutable.ListBuffer[Node]() if (running.nonEmpty) { - val runningPageTable = new RunningExecutionTable( - parent, currentTime, running.sortBy(_.submissionTime).reverse).toNodeSeq(request) + val runningPageTable = + executionsTable(request, "running", running, currentTime, true, true, true) _content ++= - Running Queries: + Running Queries: {running.size} } @@ -129,7 +130,7 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L { if (completed.nonEmpty) {
  • - Completed Queries: + Completed Queries: {completed.size}
  • } @@ -137,50 +138,216 @@ private[ui] class AllExecutionsPage(parent: SQLTab) extends WebUIPage("") with L { if (failed.nonEmpty) {
  • - Failed Queries: + Failed Queries: {failed.size}
  • } } - UIUtils.headerSparkPage(request, "SQL", summary ++ content, parent, Some(5000)) + + UIUtils.headerSparkPage(request, "SQL", summary ++ content, parent) + } + + private def executionsTable( + request: HttpServletRequest, + executionTag: String, + executionData: Seq[SQLExecutionUIData], + currentTime: Long, + showRunningJobs: Boolean, + showSucceededJobs: Boolean, + showFailedJobs: Boolean): Seq[Node] = { + + val parameterOtherTable = request.getParameterMap().asScala + .filterNot(_._1.startsWith(executionTag)) + .map { case (name, vals) => + name + "=" + vals(0) + } + + val parameterExecutionPage = request.getParameter(s"$executionTag.page") + val parameterExecutionSortColumn = request.getParameter(s"$executionTag.sort") + val parameterExecutionSortDesc = request.getParameter(s"$executionTag.desc") + val parameterExecutionPageSize = request.getParameter(s"$executionTag.pageSize") + + val executionPage = Option(parameterExecutionPage).map(_.toInt).getOrElse(1) + val executionSortColumn = Option(parameterExecutionSortColumn).map { sortColumn => + UIUtils.decodeURLParameter(sortColumn) + }.getOrElse("ID") + val executionSortDesc = Option(parameterExecutionSortDesc).map(_.toBoolean).getOrElse( + // New executions should be shown above old executions by default. + executionSortColumn == "ID" + ) + val executionPageSize = Option(parameterExecutionPageSize).map(_.toInt).getOrElse(100) + + val tableHeaderId = executionTag // "running", "completed" or "failed" + + try { + new ExecutionPagedTable( + request, + parent, + executionData, + tableHeaderId, + executionTag, + UIUtils.prependBaseUri(request, parent.basePath), + "SQL", // subPath + parameterOtherTable, + currentTime, + pageSize = executionPageSize, + sortColumn = executionSortColumn, + desc = executionSortDesc, + showRunningJobs, + showSucceededJobs, + showFailedJobs).table(executionPage) + } catch { + case e@(_: IllegalArgumentException | _: IndexOutOfBoundsException) => +
    +

    Error while rendering execution table:

    +
    +            {Utils.exceptionString(e)}
    +          
    +
    + } } } -private[ui] abstract class ExecutionTable( +private[ui] class ExecutionPagedTable( + request: HttpServletRequest, parent: SQLTab, - tableId: String, + data: Seq[SQLExecutionUIData], + tableHeaderId: String, + executionTag: String, + basePath: String, + subPath: String, + parameterOtherTable: Iterable[String], currentTime: Long, - executionUIDatas: Seq[SQLExecutionUIData], + pageSize: Int, + sortColumn: String, + desc: Boolean, showRunningJobs: Boolean, showSucceededJobs: Boolean, - showFailedJobs: Boolean) { + showFailedJobs: Boolean) extends PagedTable[ExecutionTableRowData] { + + override val dataSource = new ExecutionDataSource( + request, + parent, + data, + basePath, + currentTime, + pageSize, + sortColumn, + desc, + showRunningJobs, + showSucceededJobs, + showFailedJobs) + + private val parameterPath = s"$basePath/$subPath/?${parameterOtherTable.mkString("&")}" + + override def tableId: String = s"$executionTag-table" + + override def tableCssClass: String = + "table table-bordered table-condensed table-striped " + + "table-head-clickable table-cell-width-limited" + + override def pageLink(page: Int): String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + parameterPath + + s"&$pageNumberFormField=$page" + + s"&$executionTag.sort=$encodedSortColumn" + + s"&$executionTag.desc=$desc" + + s"&$pageSizeFormField=$pageSize" + + s"#$tableHeaderId" + } - protected def baseHeader: Seq[String] = Seq( - "ID", - "Description", - "Submitted", - "Duration") + override def pageSizeFormField: String = s"$executionTag.pageSize" - protected def header: Seq[String] + override def pageNumberFormField: String = s"$executionTag.page" - protected def row( - request: HttpServletRequest, - currentTime: Long, - executionUIData: SQLExecutionUIData): Seq[Node] = { - val submissionTime = executionUIData.submissionTime - val duration = executionUIData.completionTime.map(_.getTime()).getOrElse(currentTime) - - submissionTime + override def goButtonFormPath: String = { + val encodedSortColumn = URLEncoder.encode(sortColumn, "UTF-8") + s"$parameterPath&$executionTag.sort=$encodedSortColumn&$executionTag.desc=$desc#$tableHeaderId" + } + + override def headers: Seq[Node] = { + // Information for each header: title, sortable + val executionHeadersAndCssClasses: Seq[(String, Boolean)] = + Seq( + ("ID", true), + ("Description", true), + ("Submitted", true), + ("Duration", true)) ++ { + if (showRunningJobs && showSucceededJobs && showFailedJobs) { + Seq( + ("Running Job IDs", true), + ("Succeeded Job IDs", true), + ("Failed Job IDs", true)) + } else if (showSucceededJobs && showFailedJobs) { + Seq( + ("Succeeded Job IDs", true), + ("Failed Job IDs", true)) + } else { + Seq(("Job IDs", true)) + } + } - def jobLinks(status: JobExecutionStatus): Seq[Node] = { - executionUIData.jobs.flatMap { case (jobId, jobStatus) => - if (jobStatus == status) { - [{jobId.toString}] + val sortableColumnHeaders = executionHeadersAndCssClasses.filter { + case (_, sortable) => sortable + }.map { case (title, _) => title } + + require(sortableColumnHeaders.contains(sortColumn), s"Unknown column: $sortColumn") + + val headerRow: Seq[Node] = { + executionHeadersAndCssClasses.map { case (header, sortable) => + if (header == sortColumn) { + val headerLink = Unparsed( + parameterPath + + s"&$executionTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$executionTag.desc=${!desc}" + + s"&$executionTag.pageSize=$pageSize" + + s"#$tableHeaderId") + val arrow = if (desc) "▾" else "▴" // UP or DOWN + + + + {header} +  {Unparsed(arrow)} + + + } else { - None + if (sortable) { + val headerLink = Unparsed( + parameterPath + + s"&$executionTag.sort=${URLEncoder.encode(header, "UTF-8")}" + + s"&$executionTag.pageSize=$pageSize" + + s"#$tableHeaderId") + + + + {header} + + + } else { + + {header} + + } } - }.toSeq + } + } + + {headerRow} + + } + + override def row(executionTableRow: ExecutionTableRowData): Seq[Node] = { + val executionUIData = executionTableRow.executionUIData + val submissionTime = executionUIData.submissionTime + val duration = executionTableRow.duration + + def jobLinks(jobData: Seq[Int]): Seq[Node] = { + jobData.map { jobId => + [{jobId.toString}] + } } @@ -188,7 +355,7 @@ private[ui] abstract class ExecutionTable( {executionUIData.executionId.toString} - {descriptionCell(request, executionUIData)} + {descriptionCell(executionUIData)} {UIUtils.formatDate(submissionTime)} @@ -198,104 +365,138 @@ private[ui] abstract class ExecutionTable( {if (showRunningJobs) { - {jobLinks(JobExecutionStatus.RUNNING)} + {jobLinks(executionTableRow.runningJobData)} }} {if (showSucceededJobs) { - {jobLinks(JobExecutionStatus.SUCCEEDED)} + {jobLinks(executionTableRow.completedJobData)} }} {if (showFailedJobs) { - {jobLinks(JobExecutionStatus.FAILED)} + {jobLinks(executionTableRow.failedJobData)} }} } - private def descriptionCell( - request: HttpServletRequest, - execution: SQLExecutionUIData): Seq[Node] = { + private def descriptionCell(execution: SQLExecutionUIData): Seq[Node] = { val details = if (execution.details != null && execution.details.nonEmpty) { - + +details ++ } else { Nil } val desc = if (execution.description != null && execution.description.nonEmpty) { - {execution.description} + + {execution.description} } else { - {execution.executionId} + {execution.executionId} } -
    {desc} {details}
    - } - - def toNodeSeq(request: HttpServletRequest): Seq[Node] = { - UIUtils.listingTable[SQLExecutionUIData]( - header, row(request, currentTime, _), executionUIDatas, id = Some(tableId)) +
    {desc}{details}
    } private def jobURL(request: HttpServletRequest, jobId: Long): String = "%s/jobs/job/?id=%s".format(UIUtils.prependBaseUri(request, parent.basePath), jobId) - private def executionURL(request: HttpServletRequest, executionID: Long): String = + private def executionURL(executionID: Long): String = s"${UIUtils.prependBaseUri( request, parent.basePath)}/${parent.prefix}/execution/?id=$executionID" } -private[ui] class RunningExecutionTable( - parent: SQLTab, - currentTime: Long, - executionUIDatas: Seq[SQLExecutionUIData]) - extends ExecutionTable( - parent, - "running-execution-table", - currentTime, - executionUIDatas, - showRunningJobs = true, - showSucceededJobs = true, - showFailedJobs = true) { - override protected def header: Seq[String] = - baseHeader ++ Seq("Running Job IDs", "Succeeded Job IDs", "Failed Job IDs") -} +private[ui] class ExecutionTableRowData( + val submissionTime: Long, + val duration: Long, + val executionUIData: SQLExecutionUIData, + val runningJobData: Seq[Int], + val completedJobData: Seq[Int], + val failedJobData: Seq[Int]) + -private[ui] class CompletedExecutionTable( +private[ui] class ExecutionDataSource( + request: HttpServletRequest, parent: SQLTab, + executionData: Seq[SQLExecutionUIData], + basePath: String, currentTime: Long, - executionUIDatas: Seq[SQLExecutionUIData]) - extends ExecutionTable( - parent, - "completed-execution-table", - currentTime, - executionUIDatas, - showRunningJobs = false, - showSucceededJobs = true, - showFailedJobs = false) { + pageSize: Int, + sortColumn: String, + desc: Boolean, + showRunningJobs: Boolean, + showSucceededJobs: Boolean, + showFailedJobs: Boolean) extends PagedDataSource[ExecutionTableRowData](pageSize) { - override protected def header: Seq[String] = baseHeader ++ Seq("Job IDs") -} + // Convert ExecutionData to ExecutionTableRowData which contains the final contents to show + // in the table so that we can avoid creating duplicate contents during sorting the data + private val data = executionData.map(executionRow).sorted(ordering(sortColumn, desc)) -private[ui] class FailedExecutionTable( - parent: SQLTab, - currentTime: Long, - executionUIDatas: Seq[SQLExecutionUIData]) - extends ExecutionTable( - parent, - "failed-execution-table", - currentTime, - executionUIDatas, - showRunningJobs = false, - showSucceededJobs = true, - showFailedJobs = true) { + private var _sliceExecutionIds: Set[Int] = _ + + override def dataSize: Int = data.size + + override def sliceData(from: Int, to: Int): Seq[ExecutionTableRowData] = { + val r = data.slice(from, to) + _sliceExecutionIds = r.map(_.executionUIData.executionId.toInt).toSet + r + } - override protected def header: Seq[String] = - baseHeader ++ Seq("Succeeded Job IDs", "Failed Job IDs") + private def executionRow(executionUIData: SQLExecutionUIData): ExecutionTableRowData = { + val submissionTime = executionUIData.submissionTime + val duration = executionUIData.completionTime.map(_.getTime()) + .getOrElse(currentTime) - submissionTime + + val runningJobData = if (showRunningJobs) { + executionUIData.jobs.filter { + case (_, jobStatus) => jobStatus == JobExecutionStatus.RUNNING + }.map { case (jobId, _) => jobId }.toSeq.sorted + } else Seq.empty + + val completedJobData = if (showSucceededJobs) { + executionUIData.jobs.filter { + case (_, jobStatus) => jobStatus == JobExecutionStatus.SUCCEEDED + }.map { case (jobId, _) => jobId }.toSeq.sorted + } else Seq.empty + + val failedJobData = if (showFailedJobs) { + executionUIData.jobs.filter { + case (_, jobStatus) => jobStatus == JobExecutionStatus.FAILED + }.map { case (jobId, _) => jobId }.toSeq.sorted + } else Seq.empty + + new ExecutionTableRowData( + submissionTime, + duration, + executionUIData, + runningJobData, + completedJobData, + failedJobData) + } + + /** Return Ordering according to sortColumn and desc. */ + private def ordering(sortColumn: String, desc: Boolean): Ordering[ExecutionTableRowData] = { + val ordering: Ordering[ExecutionTableRowData] = sortColumn match { + case "ID" => Ordering.by(_.executionUIData.executionId) + case "Description" => Ordering.by(_.executionUIData.description) + case "Submitted" => Ordering.by(_.executionUIData.submissionTime) + case "Duration" => Ordering.by(_.duration) + case "Job IDs" | "Succeeded Job IDs" => Ordering by (_.completedJobData.headOption) + case "Running Job IDs" => Ordering.by(_.runningJobData.headOption) + case "Failed Job IDs" => Ordering.by(_.failedJobData.headOption) + case unknownColumn => throw new IllegalArgumentException(s"Unknown column: $unknownColumn") + } + if (desc) { + ordering.reverse + } else { + ordering + } + } } 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 877176b030f8b..875086cda258d 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 @@ -30,8 +30,7 @@ class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging private val sqlStore = parent.sqlStore override def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterExecutionId = UIUtils.stripXSS(request.getParameter("id")) + val parameterExecutionId = request.getParameter("id") require(parameterExecutionId != null && parameterExecutionId.nonEmpty, "Missing execution id parameter") @@ -84,7 +83,7 @@ class ExecutionPage(parent: SQLTab) extends WebUIPage("execution") with Logging } UIUtils.headerSparkPage( - request, s"Details for Query $executionId", content, parent, Some(5000)) + request, s"Details for Query $executionId", content, parent) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala index 1199eeca959d5..2c4a7eacdf10b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLAppStatusListener.scala @@ -16,20 +16,19 @@ */ package org.apache.spark.sql.execution.ui -import java.util.Date +import java.util.{Date, NoSuchElementException} import java.util.concurrent.ConcurrentHashMap -import java.util.function.Function import scala.collection.JavaConverters._ import org.apache.spark.{JobExecutionStatus, SparkConf} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Status._ import org.apache.spark.scheduler._ import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.metric._ import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.status.{ElementTrackingStore, KVUtils, LiveEntity} -import org.apache.spark.status.config._ class SQLAppStatusListener( conf: SparkConf, @@ -77,7 +76,29 @@ class SQLAppStatusListener( val executionId = executionIdString.toLong val jobId = event.jobId - val exec = getOrCreateExecution(executionId) + val exec = Option(liveExecutions.get(executionId)) + .orElse { + try { + // Should not overwrite the kvstore with new entry, if it already has the SQLExecution + // data corresponding to the execId. + val sqlStoreData = kvstore.read(classOf[SQLExecutionUIData], executionId) + val executionData = new LiveExecutionData(executionId) + executionData.description = sqlStoreData.description + executionData.details = sqlStoreData.details + executionData.physicalPlanDescription = sqlStoreData.physicalPlanDescription + executionData.metrics = sqlStoreData.metrics + executionData.submissionTime = sqlStoreData.submissionTime + executionData.completionTime = sqlStoreData.completionTime + executionData.jobs = sqlStoreData.jobs + executionData.stages = sqlStoreData.stages + executionData.metricsValues = sqlStoreData.metricValues + executionData.endEvents = sqlStoreData.jobs.size + 1 + liveExecutions.put(executionId, executionData) + Some(executionData) + } catch { + case _: NoSuchElementException => None + } + }.getOrElse(getOrCreateExecution(executionId)) // Record the accumulator IDs for the stages of this job, so that the code that keeps // track of the metrics knows which accumulators to look at. @@ -159,7 +180,6 @@ class SQLAppStatusListener( } private def aggregateMetrics(exec: LiveExecutionData): Map[Long, String] = { - val metricIds = exec.metrics.map(_.accumulatorId).sorted val metricTypes = exec.metrics.map { m => (m.accumulatorId, m.metricType) }.toMap val metrics = exec.stages.toSeq .flatMap { stageId => Option(stageMetrics.get(stageId)) } @@ -167,15 +187,15 @@ class SQLAppStatusListener( .flatMap { metrics => metrics.ids.zip(metrics.values) } val aggregatedMetrics = (metrics ++ exec.driverAccumUpdates.toSeq) - .filter { case (id, _) => metricIds.contains(id) } + .filter { case (id, _) => metricTypes.contains(id) } .groupBy(_._1) .map { case (id, values) => - id -> SQLMetrics.stringValue(metricTypes(id), values.map(_._2).toSeq) + id -> SQLMetrics.stringValue(metricTypes(id), values.map(_._2)) } // Check the execution again for whether the aggregated metrics data has been calculated. // This can happen if the UI is requesting this data, and the onExecutionEnd handler is - // running at the same time. The metrics calculated for the UI can be innacurate in that + // running at the same time. The metrics calculated for the UI can be inaccurate in that // case, since the onExecutionEnd handler will clean up tracked stage metrics. if (exec.metricsValues != null) { exec.metricsValues @@ -228,26 +248,26 @@ class SQLAppStatusListener( } } + private def toStoredNodes(nodes: Seq[SparkPlanGraphNode]): Seq[SparkPlanGraphNodeWrapper] = { + nodes.map { + case cluster: SparkPlanGraphCluster => + val storedCluster = new SparkPlanGraphClusterWrapper( + cluster.id, + cluster.name, + cluster.desc, + toStoredNodes(cluster.nodes), + cluster.metrics) + new SparkPlanGraphNodeWrapper(null, storedCluster) + + case node => + new SparkPlanGraphNodeWrapper(node, null) + } + } + private def onExecutionStart(event: SparkListenerSQLExecutionStart): Unit = { val SparkListenerSQLExecutionStart(executionId, description, details, physicalPlanDescription, sparkPlanInfo, time) = event - def toStoredNodes(nodes: Seq[SparkPlanGraphNode]): Seq[SparkPlanGraphNodeWrapper] = { - nodes.map { - case cluster: SparkPlanGraphCluster => - val storedCluster = new SparkPlanGraphClusterWrapper( - cluster.id, - cluster.name, - cluster.desc, - toStoredNodes(cluster.nodes), - cluster.metrics) - new SparkPlanGraphNodeWrapper(null, storedCluster) - - case node => - new SparkPlanGraphNodeWrapper(node, null) - } - } - val planGraph = SparkPlanGraph(sparkPlanInfo) val sqlPlanMetrics = planGraph.allNodes.flatMap { node => node.metrics.map { metric => (metric.accumulatorId, metric) } @@ -268,6 +288,27 @@ class SQLAppStatusListener( update(exec) } + private def onAdaptiveExecutionUpdate(event: SparkListenerSQLAdaptiveExecutionUpdate): Unit = { + val SparkListenerSQLAdaptiveExecutionUpdate( + executionId, physicalPlanDescription, sparkPlanInfo) = event + + val planGraph = SparkPlanGraph(sparkPlanInfo) + val sqlPlanMetrics = planGraph.allNodes.flatMap { node => + node.metrics.map { metric => (metric.accumulatorId, metric) } + }.toMap.values.toList + + val graphToStore = new SparkPlanGraphWrapper( + executionId, + toStoredNodes(planGraph.nodes), + planGraph.edges) + kvstore.write(graphToStore) + + val exec = getOrCreateExecution(executionId) + exec.physicalPlanDescription = physicalPlanDescription + exec.metrics = sqlPlanMetrics + update(exec) + } + private def onExecutionEnd(event: SparkListenerSQLExecutionEnd): Unit = { val SparkListenerSQLExecutionEnd(executionId, time) = event Option(liveExecutions.get(executionId)).foreach { exec => @@ -276,16 +317,20 @@ class SQLAppStatusListener( exec.endEvents += 1 update(exec) - // Remove stale LiveStageMetrics objects for stages that are not active anymore. - val activeStages = liveExecutions.values().asScala.flatMap { other => - if (other != exec) other.stages else Nil - }.toSet - stageMetrics.keySet().asScala - .filter(!activeStages.contains(_)) - .foreach(stageMetrics.remove) + removeStaleMetricsData(exec) } } + private def removeStaleMetricsData(exec: LiveExecutionData): Unit = { + // Remove stale LiveStageMetrics objects for stages that are not active anymore. + val activeStages = liveExecutions.values().asScala.flatMap { other => + if (other != exec) other.stages else Nil + }.toSet + stageMetrics.keySet().asScala + .filter(!activeStages.contains(_)) + .foreach(stageMetrics.remove) + } + private def onDriverAccumUpdates(event: SparkListenerDriverAccumUpdates): Unit = { val SparkListenerDriverAccumUpdates(executionId, accumUpdates) = event Option(liveExecutions.get(executionId)).foreach { exec => @@ -296,6 +341,7 @@ class SQLAppStatusListener( override def onOtherEvent(event: SparkListenerEvent): Unit = event match { case e: SparkListenerSQLExecutionStart => onExecutionStart(e) + case e: SparkListenerSQLAdaptiveExecutionUpdate => onAdaptiveExecutionUpdate(e) case e: SparkListenerSQLExecutionEnd => onExecutionEnd(e) case e: SparkListenerDriverAccumUpdates => onDriverAccumUpdates(e) case _ => // Ignore @@ -303,15 +349,14 @@ class SQLAppStatusListener( private def getOrCreateExecution(executionId: Long): LiveExecutionData = { liveExecutions.computeIfAbsent(executionId, - new Function[Long, LiveExecutionData]() { - override def apply(key: Long): LiveExecutionData = new LiveExecutionData(executionId) - }) + (_: Long) => new LiveExecutionData(executionId)) } private def update(exec: LiveExecutionData, force: Boolean = false): Unit = { val now = System.nanoTime() if (exec.endEvents >= exec.jobs.size + 1) { exec.write(kvstore, now) + removeStaleMetricsData(exec) liveExecutions.remove(exec.executionId) } else if (force) { exec.write(kvstore, now) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala index b58b8c6d45e5b..67d1f27271b21 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ui/SQLListener.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.ui +import com.fasterxml.jackson.annotation.JsonIgnore import com.fasterxml.jackson.databind.JavaType import com.fasterxml.jackson.databind.`type`.TypeFactory import com.fasterxml.jackson.databind.annotation.JsonDeserialize @@ -24,8 +25,14 @@ import com.fasterxml.jackson.databind.util.Converter import org.apache.spark.annotation.DeveloperApi import org.apache.spark.scheduler._ -import org.apache.spark.sql.execution.SparkPlanInfo -import org.apache.spark.sql.execution.metric._ +import org.apache.spark.sql.execution.{QueryExecution, SparkPlanInfo} + +@DeveloperApi +case class SparkListenerSQLAdaptiveExecutionUpdate( + executionId: Long, + physicalPlanDescription: String, + sparkPlanInfo: SparkPlanInfo) + extends SparkListenerEvent @DeveloperApi case class SparkListenerSQLExecutionStart( @@ -39,7 +46,22 @@ case class SparkListenerSQLExecutionStart( @DeveloperApi case class SparkListenerSQLExecutionEnd(executionId: Long, time: Long) - extends SparkListenerEvent + extends SparkListenerEvent { + + // The name of the execution, e.g. `df.collect` will trigger a SQL execution with name "collect". + @JsonIgnore private[sql] var executionName: Option[String] = None + + // The following 3 fields are only accessed when `executionName` is defined. + + // The duration of the SQL execution, in nanoseconds. + @JsonIgnore private[sql] var duration: Long = 0L + + // The `QueryExecution` instance that represents the SQL execution + @JsonIgnore private[sql] var qe: QueryExecution = null + + // The exception object that caused this execution to fail. None if the execution doesn't fail. + @JsonIgnore private[sql] var executionFailure: Option[Exception] = None +} /** * A message used to update SQL metric value for driver-side updates (which doesn't get reflected @@ -74,12 +96,12 @@ private class LongLongTupleConverter extends Converter[(Object, Object), (Long, } override def getInputType(typeFactory: TypeFactory): JavaType = { - val objectType = typeFactory.uncheckedSimpleType(classOf[Object]) - typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(objectType, objectType)) + val objectType = typeFactory.constructType(classOf[Object]) + typeFactory.constructSimpleType(classOf[(_, _)], Array(objectType, objectType)) } override def getOutputType(typeFactory: TypeFactory): JavaType = { - val longType = typeFactory.uncheckedSimpleType(classOf[Long]) - typeFactory.constructSimpleType(classOf[(_, _)], classOf[(_, _)], Array(longType, longType)) + val longType = typeFactory.constructType(classOf[Long]) + typeFactory.constructSimpleType(classOf[(_, _)], Array(longType, longType)) } } 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 e57d080dadf78..2b7597ee66d43 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 @@ -96,6 +96,15 @@ object SparkPlanGraph { case "InputAdapter" => buildSparkPlanGraphNode( planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) + case "BroadcastQueryStage" | "ShuffleQueryStage" => + if (exchanges.contains(planInfo.children.head)) { + // Point to the re-used exchange + val node = exchanges(planInfo.children.head) + edges += SparkPlanGraphEdge(node.id, parent.id) + } else { + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, null, exchanges) + } case "Subquery" if subgraph != null => // Subquery should not be included in WholeStageCodegen buildSparkPlanGraphNode(planInfo, nodeIdGenerator, nodes, edges, parent, null, exchanges) @@ -103,6 +112,11 @@ object SparkPlanGraph { // Point to the re-used subquery val node = exchanges(planInfo) edges += SparkPlanGraphEdge(node.id, parent.id) + case "ReusedSubquery" => + // Re-used subquery might appear before the original subquery, so skip this node and let + // the previous `case` make sure the re-used and the original point to the same node. + buildSparkPlanGraphNode( + planInfo.children.head, nodeIdGenerator, nodes, edges, parent, subgraph, exchanges) case "ReusedExchange" if exchanges.contains(planInfo.children.head) => // Point to the re-used exchange val node = exchanges(planInfo.children.head) @@ -184,7 +198,7 @@ private[ui] class SparkPlanGraphCluster( require(duration.length == 1) val id = duration(0).accumulatorId if (metricsValue.contains(duration(0).accumulatorId)) { - name + "\n\n" + metricsValue.get(id).get + name + "\n\n" + metricsValue(id) } else { name } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala index fede0f3e92d67..89f6edda2ef57 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExec.scala @@ -83,14 +83,14 @@ case class WindowExec( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], child: SparkPlan) - extends UnaryExecNode { + extends WindowExecBase(windowExpression, partitionSpec, orderSpec, child) { override def output: Seq[Attribute] = child.output ++ windowExpression.map(_.toAttribute) override def requiredChildDistribution: Seq[Distribution] = { if (partitionSpec.isEmpty) { - // Only show warning when the number of bytes is larger than 100 MB? + // Only show warning when the number of bytes is larger than 100 MiB? logWarning("No Partition Defined for Window operation! Moving all data to a single " + "partition, this can cause serious performance degradation.") AllTuples :: Nil @@ -104,193 +104,6 @@ case class WindowExec( override def outputPartitioning: Partitioning = child.outputPartitioning - /** - * Create a bound ordering object for a given frame type and offset. A bound ordering object is - * used to determine which input row lies within the frame boundaries of an output row. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param frame to evaluate. This can either be a Row or Range frame. - * @param bound with respect to the row. - * @param timeZone the session local timezone for time related calculations. - * @return a bound ordering object. - */ - private[this] def createBoundOrdering( - frame: FrameType, bound: Expression, timeZone: String): BoundOrdering = { - (frame, bound) match { - case (RowFrame, CurrentRow) => - RowBoundOrdering(0) - - case (RowFrame, IntegerLiteral(offset)) => - RowBoundOrdering(offset) - - case (RangeFrame, CurrentRow) => - val ordering = newOrdering(orderSpec, child.output) - RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) - - case (RangeFrame, offset: Expression) if orderSpec.size == 1 => - // Use only the first order expression when the offset is non-null. - val sortExpr = orderSpec.head - val expr = sortExpr.child - - // Create the projection which returns the current 'value'. - val current = newMutableProjection(expr :: Nil, child.output) - - // Flip the sign of the offset when processing the order is descending - val boundOffset = sortExpr.direction match { - case Descending => UnaryMinus(offset) - case Ascending => offset - } - - // Create the projection which returns the current 'value' modified by adding the offset. - val boundExpr = (expr.dataType, boundOffset.dataType) match { - case (DateType, IntegerType) => DateAdd(expr, boundOffset) - case (TimestampType, CalendarIntervalType) => - TimeAdd(expr, boundOffset, Some(timeZone)) - case (a, b) if a== b => Add(expr, boundOffset) - } - val bound = newMutableProjection(boundExpr :: Nil, child.output) - - // Construct the ordering. This is used to compare the result of current value projection - // to the result of bound value projection. This is done manually because we want to use - // Code Generation (if it is enabled). - val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil - val ordering = newOrdering(boundSortExprs, Nil) - RangeBoundOrdering(ordering, current, bound) - - case (RangeFrame, _) => - sys.error("Non-Zero range offsets are not supported for windows " + - "with multiple order expressions.") - } - } - - /** - * Collection containing an entry for each window frame to process. Each entry contains a frame's - * [[WindowExpression]]s and factory function for the WindowFrameFunction. - */ - private[this] lazy val windowFrameExpressionFactoryPairs = { - type FrameKey = (String, FrameType, Expression, Expression) - type ExpressionBuffer = mutable.Buffer[Expression] - val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)] - - // Add a function and its function to the map for a given frame. - def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = { - val key = (tpe, fr.frameType, fr.lower, fr.upper) - val (es, fns) = framedFunctions.getOrElseUpdate( - key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) - es += e - fns += fn - } - - // Collect all valid window functions and group them by their frame. - windowExpression.foreach { x => - x.foreach { - case e @ WindowExpression(function, spec) => - val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] - function match { - case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f) - case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f) - case f: OffsetWindowFunction => collect("OFFSET", frame, e, f) - case f => sys.error(s"Unsupported window function: $f") - } - case _ => - } - } - - // Map the groups to a (unbound) expression and frame factory pair. - var numExpressions = 0 - val timeZone = conf.sessionLocalTimeZone - framedFunctions.toSeq.map { - case (key, (expressions, functionSeq)) => - val ordinal = numExpressions - val functions = functionSeq.toArray - - // Construct an aggregate processor if we need one. - def processor = AggregateProcessor( - functions, - ordinal, - child.output, - (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) - - // Create the factory - val factory = key match { - // Offset Frame - case ("OFFSET", _, IntegerLiteral(offset), _) => - target: InternalRow => - new OffsetWindowFunctionFrame( - target, - ordinal, - // OFFSET frame functions are guaranteed be OffsetWindowFunctions. - functions.map(_.asInstanceOf[OffsetWindowFunction]), - child.output, - (expressions, schema) => - newMutableProjection(expressions, schema, subexpressionEliminationEnabled), - offset) - - // Entire Partition Frame. - case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing) => - target: InternalRow => { - new UnboundedWindowFunctionFrame(target, processor) - } - - // Growing Frame. - case ("AGGREGATE", frameType, UnboundedPreceding, upper) => - target: InternalRow => { - new UnboundedPrecedingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, upper, timeZone)) - } - - // Shrinking Frame. - case ("AGGREGATE", frameType, lower, UnboundedFollowing) => - target: InternalRow => { - new UnboundedFollowingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, lower, timeZone)) - } - - // Moving Frame. - case ("AGGREGATE", frameType, lower, upper) => - target: InternalRow => { - new SlidingWindowFunctionFrame( - target, - processor, - createBoundOrdering(frameType, lower, timeZone), - createBoundOrdering(frameType, upper, timeZone)) - } - } - - // Keep track of the number of expressions. This is a side-effect in a map... - numExpressions += expressions.size - - // Create the Frame Expression - Factory pair. - (expressions, factory) - } - } - - /** - * Create the resulting projection. - * - * This method uses Code Generation. It can only be used on the executor side. - * - * @param expressions unbound ordered function expressions. - * @return the final resulting projection. - */ - private[this] def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { - val references = expressions.zipWithIndex.map{ case (e, i) => - // Results of window expressions will be on the right side of child's output - BoundReference(child.output.size + i, e.dataType, e.nullable) - } - val unboundToRefMap = expressions.zip(references).toMap - val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) - UnsafeProjection.create( - child.output ++ patchedWindowExpression, - child.output) - } - protected override def doExecute(): RDD[InternalRow] = { // Unwrap the expressions and factories from the map. val expressions = windowFrameExpressionFactoryPairs.flatMap(_._1) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala new file mode 100644 index 0000000000000..dcb86f48bdf32 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowExecBase.scala @@ -0,0 +1,230 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.window + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression +import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} +import org.apache.spark.sql.types.{CalendarIntervalType, DateType, IntegerType, TimestampType} + +abstract class WindowExecBase( + windowExpression: Seq[NamedExpression], + partitionSpec: Seq[Expression], + orderSpec: Seq[SortOrder], + child: SparkPlan) extends UnaryExecNode { + + /** + * Create the resulting projection. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param expressions unbound ordered function expressions. + * @return the final resulting projection. + */ + protected def createResultProjection(expressions: Seq[Expression]): UnsafeProjection = { + val references = expressions.zipWithIndex.map { case (e, i) => + // Results of window expressions will be on the right side of child's output + BoundReference(child.output.size + i, e.dataType, e.nullable) + } + val unboundToRefMap = expressions.zip(references).toMap + val patchedWindowExpression = windowExpression.map(_.transform(unboundToRefMap)) + UnsafeProjection.create( + child.output ++ patchedWindowExpression, + child.output) + } + + /** + * Create a bound ordering object for a given frame type and offset. A bound ordering object is + * used to determine which input row lies within the frame boundaries of an output row. + * + * This method uses Code Generation. It can only be used on the executor side. + * + * @param frame to evaluate. This can either be a Row or Range frame. + * @param bound with respect to the row. + * @param timeZone the session local timezone for time related calculations. + * @return a bound ordering object. + */ + private def createBoundOrdering( + frame: FrameType, bound: Expression, timeZone: String): BoundOrdering = { + (frame, bound) match { + case (RowFrame, CurrentRow) => + RowBoundOrdering(0) + + case (RowFrame, IntegerLiteral(offset)) => + RowBoundOrdering(offset) + + case (RangeFrame, CurrentRow) => + val ordering = newOrdering(orderSpec, child.output) + RangeBoundOrdering(ordering, IdentityProjection, IdentityProjection) + + case (RangeFrame, offset: Expression) if orderSpec.size == 1 => + // Use only the first order expression when the offset is non-null. + val sortExpr = orderSpec.head + val expr = sortExpr.child + + // Create the projection which returns the current 'value'. + val current = newMutableProjection(expr :: Nil, child.output) + + // Flip the sign of the offset when processing the order is descending + val boundOffset = sortExpr.direction match { + case Descending => UnaryMinus(offset) + case Ascending => offset + } + + // Create the projection which returns the current 'value' modified by adding the offset. + val boundExpr = (expr.dataType, boundOffset.dataType) match { + case (DateType, IntegerType) => DateAdd(expr, boundOffset) + case (TimestampType, CalendarIntervalType) => + TimeAdd(expr, boundOffset, Some(timeZone)) + case (a, b) if a == b => Add(expr, boundOffset) + } + val bound = newMutableProjection(boundExpr :: Nil, child.output) + + // Construct the ordering. This is used to compare the result of current value projection + // to the result of bound value projection. This is done manually because we want to use + // Code Generation (if it is enabled). + val boundSortExprs = sortExpr.copy(BoundReference(0, expr.dataType, expr.nullable)) :: Nil + val ordering = newOrdering(boundSortExprs, Nil) + RangeBoundOrdering(ordering, current, bound) + + case (RangeFrame, _) => + sys.error("Non-Zero range offsets are not supported for windows " + + "with multiple order expressions.") + } + } + + /** + * Collection containing an entry for each window frame to process. Each entry contains a frame's + * [[WindowExpression]]s and factory function for the WindowFrameFunction. + */ + protected lazy val windowFrameExpressionFactoryPairs = { + type FrameKey = (String, FrameType, Expression, Expression) + type ExpressionBuffer = mutable.Buffer[Expression] + val framedFunctions = mutable.Map.empty[FrameKey, (ExpressionBuffer, ExpressionBuffer)] + + // Add a function and its function to the map for a given frame. + def collect(tpe: String, fr: SpecifiedWindowFrame, e: Expression, fn: Expression): Unit = { + val key = (tpe, fr.frameType, fr.lower, fr.upper) + val (es, fns) = framedFunctions.getOrElseUpdate( + key, (ArrayBuffer.empty[Expression], ArrayBuffer.empty[Expression])) + es += e + fns += fn + } + + // Collect all valid window functions and group them by their frame. + windowExpression.foreach { x => + x.foreach { + case e @ WindowExpression(function, spec) => + val frame = spec.frameSpecification.asInstanceOf[SpecifiedWindowFrame] + function match { + case AggregateExpression(f, _, _, _) => collect("AGGREGATE", frame, e, f) + case f: AggregateWindowFunction => collect("AGGREGATE", frame, e, f) + case f: OffsetWindowFunction => collect("OFFSET", frame, e, f) + case f: PythonUDF => collect("AGGREGATE", frame, e, f) + case f => sys.error(s"Unsupported window function: $f") + } + case _ => + } + } + + // Map the groups to a (unbound) expression and frame factory pair. + var numExpressions = 0 + val timeZone = conf.sessionLocalTimeZone + framedFunctions.toSeq.map { + case (key, (expressions, functionSeq)) => + val ordinal = numExpressions + val functions = functionSeq.toArray + + // Construct an aggregate processor if we need one. + // Currently we don't allow mixing of Pandas UDF and SQL aggregation functions + // in a single Window physical node. Therefore, we can assume no SQL aggregation + // functions if Pandas UDF exists. In the future, we might mix Pandas UDF and SQL + // aggregation function in a single physical node. + def processor = if (functions.exists(_.isInstanceOf[PythonUDF])) { + null + } else { + AggregateProcessor( + functions, + ordinal, + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled)) + } + + // Create the factory + val factory = key match { + // Offset Frame + case ("OFFSET", _, IntegerLiteral(offset), _) => + target: InternalRow => + new OffsetWindowFunctionFrame( + target, + ordinal, + // OFFSET frame functions are guaranteed be OffsetWindowFunctions. + functions.map(_.asInstanceOf[OffsetWindowFunction]), + child.output, + (expressions, schema) => + newMutableProjection(expressions, schema, subexpressionEliminationEnabled), + offset) + + // Entire Partition Frame. + case ("AGGREGATE", _, UnboundedPreceding, UnboundedFollowing) => + target: InternalRow => { + new UnboundedWindowFunctionFrame(target, processor) + } + + // Growing Frame. + case ("AGGREGATE", frameType, UnboundedPreceding, upper) => + target: InternalRow => { + new UnboundedPrecedingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, upper, timeZone)) + } + + // Shrinking Frame. + case ("AGGREGATE", frameType, lower, UnboundedFollowing) => + target: InternalRow => { + new UnboundedFollowingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower, timeZone)) + } + + // Moving Frame. + case ("AGGREGATE", frameType, lower, upper) => + target: InternalRow => { + new SlidingWindowFunctionFrame( + target, + processor, + createBoundOrdering(frameType, lower, timeZone), + createBoundOrdering(frameType, upper, timeZone)) + } + } + + // Keep track of the number of expressions. This is a side-effect in a map... + numExpressions += expressions.size + + // Create the Frame Expression - Factory pair. + (expressions, factory) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala index 156002ef58fbe..d5f2ffa5573a9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/window/WindowFunctionFrame.scala @@ -21,6 +21,7 @@ import java.util import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReferences import org.apache.spark.sql.catalyst.expressions.aggregate.NoOp import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray @@ -30,7 +31,7 @@ import org.apache.spark.sql.execution.ExternalAppendOnlyUnsafeRowArray * Before use a frame must be prepared by passing it all the rows in the current partition. After * preparation the update method can be called to fill the output rows. */ -private[window] abstract class WindowFunctionFrame { +abstract class WindowFunctionFrame { /** * Prepare the frame for calculating the results for a partition. * @@ -42,6 +43,20 @@ private[window] abstract class WindowFunctionFrame { * Write the current results to the target row. */ def write(index: Int, current: InternalRow): Unit + + /** + * The current lower window bound in the row array (inclusive). + * + * This should be called after the current row is updated via [[write]] + */ + def currentLowerBound(): Int + + /** + * The current row index of the upper window bound in the row array (exclusive) + * + * This should be called after the current row is updated via [[write]] + */ + def currentUpperBound(): Int } object WindowFunctionFrame { @@ -62,7 +77,7 @@ object WindowFunctionFrame { * @param newMutableProjection function used to create the projection. * @param offset by which rows get moved within a partition. */ -private[window] final class OffsetWindowFunctionFrame( +final class OffsetWindowFunctionFrame( target: InternalRow, ordinal: Int, expressions: Array[OffsetWindowFunction], @@ -89,9 +104,8 @@ private[window] final class OffsetWindowFunctionFrame( private[this] val projection = { // Collect the expressions and bind them. val inputAttrs = inputSchema.map(_.withNullability(true)) - val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => - BindReferences.bindReference(e.input, inputAttrs) - } + val boundExpressions = Seq.fill(ordinal)(NoOp) ++ bindReferences( + expressions.toSeq.map(_.input), inputAttrs) // Create the projection. newMutableProjection(boundExpressions, Nil).target(target) @@ -100,7 +114,7 @@ private[window] final class OffsetWindowFunctionFrame( /** Create the projection used when the offset row DOES NOT exists. */ private[this] val fillDefaultValue = { // Collect the expressions and bind them. - val inputAttrs = inputSchema.map(_.withNullability(true)) + val inputAttrs: AttributeSeq = inputSchema.map(_.withNullability(true)) val boundExpressions = Seq.fill(ordinal)(NoOp) ++ expressions.toSeq.map { e => if (e.default == null || e.default.foldable && e.default.eval() == null) { // The default value is null. @@ -137,6 +151,10 @@ private[window] final class OffsetWindowFunctionFrame( } inputIndex += 1 } + + override def currentLowerBound(): Int = throw new UnsupportedOperationException() + + override def currentUpperBound(): Int = throw new UnsupportedOperationException() } /** @@ -148,7 +166,7 @@ private[window] final class OffsetWindowFunctionFrame( * @param lbound comparator used to identify the lower bound of an output row. * @param ubound comparator used to identify the upper bound of an output row. */ -private[window] final class SlidingWindowFunctionFrame( +final class SlidingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering, @@ -170,24 +188,24 @@ private[window] final class SlidingWindowFunctionFrame( private[this] val buffer = new util.ArrayDeque[InternalRow]() /** - * Index of the first input row with a value greater than the upper bound of the current - * output row. + * Index of the first input row with a value equal to or greater than the lower bound of the + * current output row. */ - private[this] var inputHighIndex = 0 + private[this] var lowerBound = 0 /** - * Index of the first input row with a value equal to or greater than the lower bound of the - * current output row. + * Index of the first input row with a value greater than the upper bound of the current + * output row. */ - private[this] var inputLowIndex = 0 + private[this] var upperBound = 0 /** Prepare the frame for calculating a new partition. Reset all variables. */ override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { input = rows inputIterator = input.generateIterator() nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputHighIndex = 0 - inputLowIndex = 0 + lowerBound = 0 + upperBound = 0 buffer.clear() } @@ -197,27 +215,27 @@ private[window] final class SlidingWindowFunctionFrame( // Drop all rows from the buffer for which the input row value is smaller than // the output row lower bound. - while (!buffer.isEmpty && lbound.compare(buffer.peek(), inputLowIndex, current, index) < 0) { + while (!buffer.isEmpty && lbound.compare(buffer.peek(), lowerBound, current, index) < 0) { buffer.remove() - inputLowIndex += 1 + lowerBound += 1 bufferUpdated = true } // Add all rows to the buffer for which the input row value is equal to or less than // the output row upper bound. - while (nextRow != null && ubound.compare(nextRow, inputHighIndex, current, index) <= 0) { - if (lbound.compare(nextRow, inputLowIndex, current, index) < 0) { - inputLowIndex += 1 + while (nextRow != null && ubound.compare(nextRow, upperBound, current, index) <= 0) { + if (lbound.compare(nextRow, lowerBound, current, index) < 0) { + lowerBound += 1 } else { buffer.add(nextRow.copy()) bufferUpdated = true } nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) - inputHighIndex += 1 + upperBound += 1 } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.initialize(input.length) val iter = buffer.iterator() while (iter.hasNext) { @@ -226,6 +244,10 @@ private[window] final class SlidingWindowFunctionFrame( processor.evaluate(target) } } + + override def currentLowerBound(): Int = lowerBound + + override def currentUpperBound(): Int = upperBound } /** @@ -239,27 +261,39 @@ private[window] final class SlidingWindowFunctionFrame( * @param target to write results to. * @param processor to calculate the row values with. */ -private[window] final class UnboundedWindowFunctionFrame( +final class UnboundedWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor) extends WindowFunctionFrame { + val lowerBound: Int = 0 + var upperBound: Int = 0 + /** Prepare the frame for calculating a new partition. Process all rows eagerly. */ override def prepare(rows: ExternalAppendOnlyUnsafeRowArray): Unit = { - processor.initialize(rows.length) - - val iterator = rows.generateIterator() - while (iterator.hasNext) { - processor.update(iterator.next()) + if (processor != null) { + processor.initialize(rows.length) + val iterator = rows.generateIterator() + while (iterator.hasNext) { + processor.update(iterator.next()) + } } + + upperBound = rows.length } /** Write the frame columns for the current row to the given target row. */ override def write(index: Int, current: InternalRow): Unit = { // Unfortunately we cannot assume that evaluation is deterministic. So we need to re-evaluate // for each row. - processor.evaluate(target) + if (processor != null) { + processor.evaluate(target) + } } + + override def currentLowerBound(): Int = lowerBound + + override def currentUpperBound(): Int = upperBound } /** @@ -276,7 +310,7 @@ private[window] final class UnboundedWindowFunctionFrame( * @param processor to calculate the row values with. * @param ubound comparator used to identify the upper bound of an output row. */ -private[window] final class UnboundedPrecedingWindowFunctionFrame( +final class UnboundedPrecedingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, ubound: BoundOrdering) @@ -308,7 +342,9 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( nextRow = inputIterator.next() } - processor.initialize(input.length) + if (processor != null) { + processor.initialize(input.length) + } } /** Write the frame columns for the current row to the given target row. */ @@ -318,17 +354,23 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( // Add all rows to the aggregates for which the input row value is equal to or less than // the output row upper bound. while (nextRow != null && ubound.compare(nextRow, inputIndex, current, index) <= 0) { - processor.update(nextRow) + if (processor != null) { + processor.update(nextRow) + } nextRow = WindowFunctionFrame.getNextOrNull(inputIterator) inputIndex += 1 bufferUpdated = true } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.evaluate(target) } } + + override def currentLowerBound(): Int = 0 + + override def currentUpperBound(): Int = inputIndex } /** @@ -347,7 +389,7 @@ private[window] final class UnboundedPrecedingWindowFunctionFrame( * @param processor to calculate the row values with. * @param lbound comparator used to identify the lower bound of an output row. */ -private[window] final class UnboundedFollowingWindowFunctionFrame( +final class UnboundedFollowingWindowFunctionFrame( target: InternalRow, processor: AggregateProcessor, lbound: BoundOrdering) @@ -384,7 +426,7 @@ private[window] final class UnboundedFollowingWindowFunctionFrame( } // Only recalculate and update when the buffer changes. - if (bufferUpdated) { + if (processor != null && bufferUpdated) { processor.initialize(input.length) if (nextRow != null) { processor.update(nextRow) @@ -395,4 +437,8 @@ private[window] final class UnboundedFollowingWindowFunctionFrame( processor.evaluate(target) } } + + override def currentLowerBound(): Int = inputIndex + + override def currentUpperBound(): Int = input.length } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala index 1e076207bc607..6b4def35e1955 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Aggregator.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.{Dataset, Encoder, TypedColumn} +import org.apache.spark.annotation.{Evolving, Experimental} +import org.apache.spark.sql.{Encoder, TypedColumn} import org.apache.spark.sql.catalyst.encoders.encoderFor import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression @@ -51,7 +51,7 @@ import org.apache.spark.sql.execution.aggregate.TypedAggregateExpression * @since 1.6.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving abstract class Aggregator[-IN, BUF, OUT] extends Serializable { /** 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 697757f8a73ce..0c956ecbf936e 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 @@ -17,11 +17,11 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.ScalaReflection -import org.apache.spark.sql.catalyst.expressions.ScalaUDF -import org.apache.spark.sql.types.DataType +import org.apache.spark.sql.catalyst.expressions.{Expression, ScalaUDF} +import org.apache.spark.sql.types.{AnyDataType, DataType} /** * A user-defined function. To create one, use the `udf` functions in `functions`. @@ -37,26 +37,15 @@ import org.apache.spark.sql.types.DataType * * @since 1.3.0 */ -@InterfaceStability.Stable -case class UserDefinedFunction protected[sql] ( - f: AnyRef, - dataType: DataType, - inputTypes: Option[Seq[DataType]]) { - - private var _nameOption: Option[String] = None - private var _nullable: Boolean = true - private var _deterministic: Boolean = true - - // This is a `var` instead of in the constructor for backward compatibility of this case class. - // TODO: revisit this case class in Spark 3.0, and narrow down the public surface. - private[sql] var nullableTypes: Option[Seq[Boolean]] = None +@Stable +sealed abstract class UserDefinedFunction { /** * Returns true when the UDF can return a nullable value. * * @since 2.3.0 */ - def nullable: Boolean = _nullable + def nullable: Boolean /** * Returns true iff the UDF is deterministic, i.e. the UDF produces the same output given the same @@ -64,7 +53,7 @@ case class UserDefinedFunction protected[sql] ( * * @since 2.3.0 */ - def deterministic: Boolean = _deterministic + def deterministic: Boolean /** * Returns an expression that invokes the UDF, using the given arguments. @@ -72,83 +61,78 @@ case class UserDefinedFunction protected[sql] ( * @since 1.3.0 */ @scala.annotation.varargs - def apply(exprs: Column*): Column = { - if (inputTypes.isDefined && nullableTypes.isDefined) { - require(inputTypes.get.length == nullableTypes.get.length) - } - - Column(ScalaUDF( - f, - dataType, - exprs.map(_.expr), - inputTypes.getOrElse(Nil), - udfName = _nameOption, - nullable = _nullable, - udfDeterministic = _deterministic, - nullableTypes = nullableTypes.getOrElse(Nil))) - } - - private def copyAll(): UserDefinedFunction = { - val udf = copy() - udf._nameOption = _nameOption - udf._nullable = _nullable - udf._deterministic = _deterministic - udf.nullableTypes = nullableTypes - udf - } + def apply(exprs: Column*): Column /** * Updates UserDefinedFunction with a given name. * * @since 2.3.0 */ - def withName(name: String): UserDefinedFunction = { - val udf = copyAll() - udf._nameOption = Option(name) - udf - } + def withName(name: String): UserDefinedFunction /** * Updates UserDefinedFunction to non-nullable. * * @since 2.3.0 */ - def asNonNullable(): UserDefinedFunction = { - if (!nullable) { - this - } else { - val udf = copyAll() - udf._nullable = false - udf - } - } + def asNonNullable(): UserDefinedFunction /** * Updates UserDefinedFunction to nondeterministic. * * @since 2.3.0 */ - def asNondeterministic(): UserDefinedFunction = { - if (!_deterministic) { + def asNondeterministic(): UserDefinedFunction +} + +private[sql] case class SparkUserDefinedFunction( + f: AnyRef, + dataType: DataType, + inputSchemas: Seq[Option[ScalaReflection.Schema]], + name: Option[String] = None, + nullable: Boolean = true, + deterministic: Boolean = true) extends UserDefinedFunction { + + @scala.annotation.varargs + override def apply(exprs: Column*): Column = { + Column(createScalaUDF(exprs.map(_.expr))) + } + + 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, + udfName = name, + nullable = nullable, + udfDeterministic = deterministic) + } + + override def withName(name: String): SparkUserDefinedFunction = { + copy(name = Option(name)) + } + + override def asNonNullable(): SparkUserDefinedFunction = { + if (!nullable) { this } else { - val udf = copyAll() - udf._deterministic = false - udf + copy(nullable = false) } } -} -// We have to use a name different than `UserDefinedFunction` here, to avoid breaking the binary -// compatibility of the auto-generate UserDefinedFunction object. -private[sql] object SparkUserDefinedFunction { - - def create( - f: AnyRef, - dataType: DataType, - inputSchemas: Option[Seq[ScalaReflection.Schema]]): UserDefinedFunction = { - val udf = new UserDefinedFunction(f, dataType, inputSchemas.map(_.map(_.dataType))) - udf.nullableTypes = inputSchemas.map(_.map(_.nullable)) - udf + override def asNondeterministic(): SparkUserDefinedFunction = { + if (!deterministic) { + this + } else { + copy(deterministic = false) + } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala index cd819bab1b14c..cd1c198ddebf0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/Window.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.Column import org.apache.spark.sql.catalyst.expressions._ @@ -39,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable object Window { /** @@ -129,7 +129,7 @@ object Window { * An offset indicates the number of rows above or below the current row, the frame for the * current row starts or ends. For instance, given a row based sliding frame with a lower bound * offset of -1 and a upper bound offset of +2. The frame for row with index 5 would range from - * index 4 to index 6. + * index 4 to index 7. * * {{{ * import org.apache.spark.sql.expressions.Window @@ -175,8 +175,8 @@ object Window { * directly. * * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset + * expression(s). An offset is used to alter the value of the ORDER BY expression, + * for instance if the current ORDER BY expression has a value of 10 and the lower bound offset * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a * number of constraints on the ORDER BY expressions: there can be only one expression and this * expression must have a numerical data type. An exception can be made when the offset is @@ -214,57 +214,6 @@ object Window { spec.rangeBetween(start, end) } - /** - * Creates a [[WindowSpec]] with the frame boundaries defined, - * from `start` (inclusive) to `end` (inclusive). - * - * Both `start` and `end` are relative to the current row. For example, "lit(0)" means - * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the - * five off after the current row. - * - * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from - * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not - * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. - * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical/date/timestamp data type. An exception can be made when the - * offset is unbounded, because no value modification is needed, in this case multiple and - * non-numerical/date/timestamp data type ORDER BY expression are allowed. - * - * {{{ - * import org.apache.spark.sql.expressions.Window - * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) - * .toDF("id", "category") - * val byCategoryOrderedById = - * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) - * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() - * - * +---+--------+---+ - * | id|category|sum| - * +---+--------+---+ - * | 1| b| 3| - * | 2| b| 5| - * | 3| b| 3| - * | 1| a| 4| - * | 1| a| 4| - * | 2| a| 2| - * +---+--------+---+ - * }}} - * - * @param start boundary start, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. - * @param end boundary end, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. - * @since 2.3.0 - */ - def rangeBetween(start: Column, end: Column): WindowSpec = { - spec.rangeBetween(start, end) - } - private[sql] def spec: WindowSpec = { new WindowSpec(Seq.empty, Seq.empty, UnspecifiedFrame) } @@ -285,5 +234,5 @@ object Window { * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class Window private() // So we can see Window in JavaDoc. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala index 4c41aa3c5fb67..58227f075f2c7 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/WindowSpec.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.{AnalysisException, Column} import org.apache.spark.sql.catalyst.expressions._ @@ -28,7 +28,7 @@ import org.apache.spark.sql.catalyst.expressions._ * * @since 1.4.0 */ -@InterfaceStability.Stable +@Stable class WindowSpec private[sql]( partitionSpec: Seq[Expression], orderSpec: Seq[SortOrder], @@ -209,59 +209,6 @@ class WindowSpec private[sql]( SpecifiedWindowFrame(RangeFrame, boundaryStart, boundaryEnd)) } - /** - * Defines the frame boundaries, from `start` (inclusive) to `end` (inclusive). - * - * Both `start` and `end` are relative to the current row. For example, "lit(0)" means - * "current row", while "lit(-1)" means one off before the current row, and "lit(5)" means the - * five off after the current row. - * - * Users should use `unboundedPreceding()`, `unboundedFollowing()`, and `currentRow()` from - * [[org.apache.spark.sql.functions]] to specify special boundary values, literals are not - * transformed to [[org.apache.spark.sql.catalyst.expressions.SpecialFrameBoundary]]s. - * - * A range-based boundary is based on the actual value of the ORDER BY - * expression(s). An offset is used to alter the value of the ORDER BY expression, for - * instance if the current order by expression has a value of 10 and the lower bound offset - * is -3, the resulting lower bound for the current row will be 10 - 3 = 7. This however puts a - * number of constraints on the ORDER BY expressions: there can be only one expression and this - * expression must have a numerical/date/timestamp data type. An exception can be made when the - * offset is unbounded, because no value modification is needed, in this case multiple and - * non-numerical/date/timestamp data type ORDER BY expression are allowed. - * - * {{{ - * import org.apache.spark.sql.expressions.Window - * val df = Seq((1, "a"), (1, "a"), (2, "a"), (1, "b"), (2, "b"), (3, "b")) - * .toDF("id", "category") - * val byCategoryOrderedById = - * Window.partitionBy('category).orderBy('id).rangeBetween(currentRow(), lit(1)) - * df.withColumn("sum", sum('id) over byCategoryOrderedById).show() - * - * +---+--------+---+ - * | id|category|sum| - * +---+--------+---+ - * | 1| b| 3| - * | 2| b| 5| - * | 3| b| 3| - * | 1| a| 4| - * | 1| a| 4| - * | 2| a| 2| - * +---+--------+---+ - * }}} - * - * @param start boundary start, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedPreceding]]. - * @param end boundary end, inclusive. The frame is unbounded if the expression is - * [[org.apache.spark.sql.catalyst.expressions.UnboundedFollowing]]. - * @since 2.3.0 - */ - def rangeBetween(start: Column, end: Column): WindowSpec = { - new WindowSpec( - partitionSpec, - orderSpec, - SpecifiedWindowFrame(RangeFrame, start.expr, end.expr)) - } - /** * Converts this [[WindowSpec]] into a [[Column]] with an aggregate expression. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala index 650ffd4586592..da7ed69e48390 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/scalalang/typed.scala @@ -17,7 +17,6 @@ package org.apache.spark.sql.expressions.scalalang -import org.apache.spark.annotation.{Experimental, InterfaceStability} import org.apache.spark.sql._ import org.apache.spark.sql.execution.aggregate._ @@ -29,8 +28,7 @@ import org.apache.spark.sql.execution.aggregate._ * * @since 2.0.0 */ -@Experimental -@InterfaceStability.Evolving +@deprecated("please use untyped builtin aggregate functions.", "3.0.0") // scalastyle:off object typed { // scalastyle:on @@ -76,15 +74,4 @@ object typed { * @since 2.0.0 */ def sumLong[IN](f: IN => Long): TypedColumn[IN, Long] = new TypedSumLong[IN](f).toColumn - - // TODO: - // stddevOf: Double - // varianceOf: Double - // approxCountDistinct: Long - - // minOf: T - // maxOf: T - - // firstOf: T - // lastOf: T } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala index 4976b875fa298..4e8cb3a6ddd66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/udaf.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.expressions -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.{Column, Row} import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Complete} import org.apache.spark.sql.execution.aggregate.ScalaUDAF @@ -28,7 +28,7 @@ import org.apache.spark.sql.types._ * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable abstract class UserDefinedAggregateFunction extends Serializable { /** @@ -159,7 +159,7 @@ abstract class UserDefinedAggregateFunction extends Serializable { * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable abstract class MutableAggregationBuffer extends Row { /** Update the ith value of this buffer. */ 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 4247d3110f1e1..72a197bdbcfc5 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 @@ -23,14 +23,14 @@ import scala.reflect.runtime.universe.{typeTag, TypeTag} import scala.util.Try import scala.util.control.NonFatal -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Stable import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.{Star, UnresolvedFunction} import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.aggregate._ -import org.apache.spark.sql.catalyst.plans.logical.{HintInfo, ResolvedHint} +import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, HintInfo, ResolvedHint} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.expressions.{SparkUserDefinedFunction, UserDefinedFunction} import org.apache.spark.sql.internal.SQLConf @@ -68,7 +68,7 @@ import org.apache.spark.util.Utils * @groupname Ungrouped Support functions for DataFrames * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable // scalastyle:off object functions { // scalastyle:on @@ -206,36 +206,6 @@ 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. * @@ -388,9 +358,10 @@ object functions { * @since 1.3.0 */ @scala.annotation.varargs - def countDistinct(expr: Column, exprs: Column*): Column = { - withAggregateFunction(Count.apply((expr +: exprs).map(_.expr)), isDistinct = true) - } + def countDistinct(expr: Column, exprs: Column*): Column = + // For usage like countDistinct("*"), we should let analyzer expand star and + // resolve function. + Column(UnresolvedFunction("count", (expr +: exprs).map(_.expr), isDistinct = true)) /** * Aggregate function: returns the number of distinct items in a group. @@ -829,32 +800,6 @@ object functions { ////////////////////////////////////////////////////////////////////////////////////////////// // Window functions ////////////////////////////////////////////////////////////////////////////////////////////// - /** - * Window function: returns the special frame boundary that represents the first row in the - * window partition. - * - * @group window_funcs - * @since 2.3.0 - */ - def unboundedPreceding(): Column = Column(UnboundedPreceding) - - /** - * Window function: returns the special frame boundary that represents the last row in the - * window partition. - * - * @group window_funcs - * @since 2.3.0 - */ - def unboundedFollowing(): Column = Column(UnboundedFollowing) - - /** - * Window function: returns the special frame boundary that represents the current row in the - * window partition. - * - * @group window_funcs - * @since 2.3.0 - */ - def currentRow(): Column = Column(CurrentRow) /** * Window function: returns the cumulative distribution of values within a window partition, @@ -1101,7 +1046,7 @@ object functions { */ def broadcast[T](df: Dataset[T]): Dataset[T] = { Dataset[T](df.sparkSession, - ResolvedHint(df.logicalPlan, HintInfo(broadcast = true)))(df.exprEnc) + ResolvedHint(df.logicalPlan, HintInfo(strategy = Some(BROADCAST))))(df.exprEnc) } /** @@ -1140,27 +1085,6 @@ 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. * @@ -2142,20 +2066,6 @@ 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. * @@ -2178,20 +2088,6 @@ 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. * @@ -2272,6 +2168,19 @@ object functions { new Murmur3Hash(cols.map(_.expr)) } + /** + * Calculates the hash code of given columns using the 64-bit + * variant of the xxHash algorithm, and returns the result as a long + * column. + * + * @group misc_funcs + * @since 3.0.0 + */ + @scala.annotation.varargs + def xxhash64(cols: Column*): Column = withExpr { + new XxHash64(cols.map(_.expr)) + } + ////////////////////////////////////////////////////////////////////////////////////////////// // String functions ////////////////////////////////////////////////////////////////////////////////////////////// @@ -2607,6 +2516,28 @@ object functions { SubstringIndex(str.expr, lit(delim).expr, lit(count).expr) } + /** + * Overlay the specified portion of `src` with `replaceString`, + * starting from byte position `pos` of `inputString` and proceeding for `len` bytes. + * + * @group string_funcs + * @since 3.0.0 + */ + def overlay(src: Column, replaceString: String, pos: Int, len: Int): Column = withExpr { + Overlay(src.expr, lit(replaceString).expr, lit(pos).expr, lit(len).expr) + } + + /** + * Overlay the specified portion of `src` with `replaceString`, + * starting from byte position `pos` of `inputString`. + * + * @group string_funcs + * @since 3.0.0 + */ + def overlay(src: Column, replaceString: String, pos: Int): Column = withExpr { + new Overlay(src.expr, lit(replaceString).expr, lit(pos).expr) + } + /** * Translate any character in the src by a character in replaceString. * The characters in replaceString correspond to the characters in matchingString. @@ -2683,7 +2614,7 @@ 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.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] 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` @@ -2916,7 +2847,7 @@ object functions { * representing the timestamp of that moment in the current system time zone in the given * format. * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] 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 @@ -2960,7 +2891,7 @@ object functions { /** * Converts time string with given pattern to Unix timestamp (in seconds). * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] 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` @@ -2988,7 +2919,7 @@ object functions { /** * Converts time string with the given pattern to timestamp. * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] 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` @@ -3013,7 +2944,7 @@ object functions { /** * Converts the column into a `DateType` with a specified format * - * See [[java.text.SimpleDateFormat]] for valid date and time format patterns + * See [[java.time.format.DateTimeFormatter]] 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` @@ -3034,8 +2965,8 @@ object functions { * * @param date 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` - * @param format: 'year', 'yyyy', 'yy' for truncate by year, - * or 'month', 'mon', 'mm' for truncate by month + * @param format: 'year', 'yyyy', 'yy' to truncate by year, + * or 'month', 'mon', 'mm' to truncate by month * * @return A date, or null if `date` was a string that could not be cast to a date or `format` * was an invalid value @@ -3049,11 +2980,11 @@ object functions { /** * Returns timestamp truncated to the unit specified by the format. * - * For example, `date_tunc("2018-11-19 12:01:19", "year")` returns 2018-01-01 00:00:00 + * For example, `date_trunc("year", "2018-11-19 12:01:19")` returns 2018-01-01 00:00:00 * - * @param format: 'year', 'yyyy', 'yy' for truncate by year, - * 'month', 'mon', 'mm' for truncate by month, - * 'day', 'dd' for truncate by day, + * @param format: 'year', 'yyyy', 'yy' to truncate by year, + * 'month', 'mon', 'mm' to truncate by month, + * 'day', 'dd' to truncate by day, * Other options are: 'second', 'minute', 'hour', 'week', 'month', 'quarter' * @param timestamp 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` @@ -3080,6 +3011,7 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ + @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def from_utc_timestamp(ts: Column, tz: String): Column = withExpr { FromUTCTimestamp(ts.expr, Literal(tz)) } @@ -3091,6 +3023,7 @@ object functions { * @group datetime_funcs * @since 2.4.0 */ + @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def from_utc_timestamp(ts: Column, tz: Column): Column = withExpr { FromUTCTimestamp(ts.expr, tz.expr) } @@ -3109,6 +3042,7 @@ object functions { * @group datetime_funcs * @since 1.5.0 */ + @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def to_utc_timestamp(ts: Column, tz: String): Column = withExpr { ToUTCTimestamp(ts.expr, Literal(tz)) } @@ -3120,6 +3054,7 @@ object functions { * @group datetime_funcs * @since 2.4.0 */ + @deprecated("This function is deprecated and will be removed in future versions.", "3.0.0") def to_utc_timestamp(ts: Column, tz: Column): Column = withExpr { ToUTCTimestamp(ts.expr, tz.expr) } @@ -3410,6 +3345,8 @@ object functions { /** * Creates a new row for each element in the given array or map column. + * Uses the default column name `col` for elements in the array and + * `key` and `value` for elements in the map unless specified otherwise. * * @group collection_funcs * @since 1.3.0 @@ -3418,6 +3355,8 @@ object functions { /** * Creates a new row for each element in the given array or map column. + * Uses the default column name `col` for elements in the array and + * `key` and `value` for elements in the map unless specified otherwise. * Unlike explode, if the array/map is null or empty then null is produced. * * @group collection_funcs @@ -3427,6 +3366,8 @@ object functions { /** * Creates a new row for each element with position in the given array or map column. + * Uses the default column name `pos` for position, and `col` for elements in the array + * and `key` and `value` for elements in the map unless specified otherwise. * * @group collection_funcs * @since 2.1.0 @@ -3435,6 +3376,8 @@ object functions { /** * Creates a new row for each element with position in the given array or map column. + * Uses the default column name `pos` for position, and `col` for elements in the array + * and `key` and `value` for elements in the map unless specified otherwise. * Unlike posexplode, if the array/map is null or empty then the row (null, null) is produced. * * @group collection_funcs @@ -3626,19 +3569,29 @@ object functions { } /** - * Parses a column containing a JSON string and infers its schema. + * Parses a JSON string and infers its schema in DDL format. * - * @param e a string column containing JSON data. + * @param json a JSON string. * * @group collection_funcs * @since 2.4.0 */ - def schema_of_json(e: Column): Column = withExpr(new SchemaOfJson(e.expr)) + def schema_of_json(json: String): Column = schema_of_json(lit(json)) /** - * Parses a column containing a JSON string and infers its schema using options. + * Parses a JSON string and infers its schema in DDL format. * - * @param e a string column containing JSON data. + * @param json a string literal containing a JSON string. + * + * @group collection_funcs + * @since 2.4.0 + */ + def schema_of_json(json: Column): Column = withExpr(new SchemaOfJson(json.expr)) + + /** + * Parses a JSON string and infers its schema in DDL format using options. + * + * @param json a string column containing JSON data. * @param options options to control how the json is parsed. accepts the same options and the * json data source. See [[DataFrameReader#json]]. * @return a column with string literal containing schema in DDL format. @@ -3646,8 +3599,8 @@ object functions { * @group collection_funcs * @since 3.0.0 */ - def schema_of_json(e: Column, options: java.util.Map[String, String]): Column = { - withExpr(SchemaOfJson(e.expr, options.asScala.toMap)) + def schema_of_json(json: Column, options: java.util.Map[String, String]): Column = { + withExpr(SchemaOfJson(json.expr, options.asScala.toMap)) } /** @@ -3826,7 +3779,7 @@ object functions { /** * Returns an unordered array of all entries in the given map. * @group collection_funcs - * @since 2.4.0 + * @since 3.0.0 */ def map_entries(e: Column): Column = withExpr { MapEntries(e.expr) } @@ -3854,6 +3807,99 @@ object functions { @scala.annotation.varargs def map_concat(cols: Column*): Column = withExpr { MapConcat(cols.map(_.expr)) } + /** + * Parses a column containing a CSV string into a `StructType` with the specified schema. + * Returns `null`, in the case of an unparseable string. + * + * @param e a string column containing CSV data. + * @param schema the schema to use when parsing the CSV string + * @param options options to control how the CSV is parsed. accepts the same options and the + * CSV data source. + * + * @group collection_funcs + * @since 3.0.0 + */ + def from_csv(e: Column, schema: StructType, options: Map[String, String]): Column = withExpr { + CsvToStructs(schema, options, e.expr) + } + + /** + * (Java-specific) Parses a column containing a CSV string into a `StructType` + * with the specified schema. Returns `null`, in the case of an unparseable string. + * + * @param e a string column containing CSV data. + * @param schema the schema to use when parsing the CSV string + * @param options options to control how the CSV is parsed. accepts the same options and the + * CSV data source. + * + * @group collection_funcs + * @since 3.0.0 + */ + def from_csv(e: Column, schema: Column, options: java.util.Map[String, String]): Column = { + withExpr(new CsvToStructs(e.expr, schema.expr, options.asScala.toMap)) + } + + /** + * Parses a CSV string and infers its schema in DDL format. + * + * @param csv a CSV string. + * + * @group collection_funcs + * @since 3.0.0 + */ + def schema_of_csv(csv: String): Column = schema_of_csv(lit(csv)) + + /** + * Parses a CSV string and infers its schema in DDL format. + * + * @param csv a string literal containing a CSV string. + * + * @group collection_funcs + * @since 3.0.0 + */ + def schema_of_csv(csv: Column): Column = withExpr(new SchemaOfCsv(csv.expr)) + + /** + * Parses a CSV string and infers its schema in DDL format using options. + * + * @param csv a string literal containing a CSV string. + * @param options options to control how the CSV is parsed. accepts the same options and the + * json data source. See [[DataFrameReader#csv]]. + * @return a column with string literal containing schema in DDL format. + * + * @group collection_funcs + * @since 3.0.0 + */ + def schema_of_csv(csv: Column, options: java.util.Map[String, String]): Column = { + withExpr(SchemaOfCsv(csv.expr, options.asScala.toMap)) + } + + /** + * (Java-specific) Converts a column containing a `StructType` into a CSV string with + * the specified schema. Throws an exception, in the case of an unsupported type. + * + * @param e a column containing a struct. + * @param options options to control how the struct column is converted into a CSV string. + * It accepts the same options and the json data source. + * + * @group collection_funcs + * @since 3.0.0 + */ + def to_csv(e: Column, options: java.util.Map[String, String]): Column = withExpr { + StructsToCsv(options.asScala.toMap, e.expr) + } + + /** + * Converts a column containing a `StructType` into a CSV string with the specified schema. + * Throws an exception, in the case of an unsupported type. + * + * @param e a column containing a struct. + * + * @group collection_funcs + * @since 3.0.0 + */ + def to_csv(e: Column): Column = to_csv(e, Map.empty[String, String].asJava) + // scalastyle:off line.size.limit // scalastyle:off parameter.number @@ -3862,7 +3908,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"ScalaReflection.schemaFor(typeTag[A$i]) :: $s"}) + val inputSchemas = (1 to x).foldRight("Nil")((i, s) => {s"Try(ScalaReflection.schemaFor(typeTag[A$i])).toOption :: $s"}) println(s""" |/** | * Defines a Scala closure of $x arguments as user-defined function (UDF). @@ -3875,8 +3921,8 @@ object functions { | */ |def udf[$typeTags](f: Function$x[$types]): UserDefinedFunction = { | val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - | val inputSchemas = Try($inputTypes).toOption - | val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + | val inputSchemas = $inputSchemas + | val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) | if (nullable) udf else udf.asNonNullable() |}""".stripMargin) } @@ -3886,7 +3932,7 @@ object functions { val anyTypeArgs = (0 to i).map(_ => "Any").mkString(", ") val anyCast = s".asInstanceOf[UDF$i[$anyTypeArgs]]" val anyParams = (1 to i).map(_ => "_: Any").mkString(", ") - val funcCall = if (i == 0) "() => func" else "func" + val funcCall = if (i == 0) s"() => f$anyCast.call($anyParams)" else s"f$anyCast.call($anyParams)" println(s""" |/** | * Defines a Java UDF$i instance as user-defined function (UDF). @@ -3898,8 +3944,8 @@ object functions { | * @since 2.3.0 | */ |def udf(f: UDF$i[$extTypeArgs], returnType: DataType): UserDefinedFunction = { - | val func = f$anyCast.call($anyParams) - | SparkUserDefinedFunction.create($funcCall, returnType, inputSchemas = None) + | val func = $funcCall + | SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill($i)(None)) |}""".stripMargin) } @@ -3920,8 +3966,8 @@ object functions { */ def udf[RT: TypeTag](f: Function0[RT]): UserDefinedFunction = { val ScalaReflection.Schema(dataType, nullable) = ScalaReflection.schemaFor[RT] - val inputSchemas = Try(Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val inputSchemas = Nil + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3936,8 +3982,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]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3952,8 +3998,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + val inputSchemas = Try(ScalaReflection.schemaFor(typeTag[A1])).toOption :: Try(ScalaReflection.schemaFor(typeTag[A2])).toOption :: Nil + val udf = SparkUserDefinedFunction(f, dataType, inputSchemas) if (nullable) udf else udf.asNonNullable() } @@ -3968,8 +4014,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -3984,8 +4030,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4000,8 +4046,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4016,8 +4062,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4032,8 +4078,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4048,8 +4094,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4064,8 +4110,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: ScalaReflection.schemaFor(typeTag[A9]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4080,8 +4126,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]) :: ScalaReflection.schemaFor(typeTag[A2]) :: ScalaReflection.schemaFor(typeTag[A3]) :: ScalaReflection.schemaFor(typeTag[A4]) :: ScalaReflection.schemaFor(typeTag[A5]) :: ScalaReflection.schemaFor(typeTag[A6]) :: ScalaReflection.schemaFor(typeTag[A7]) :: ScalaReflection.schemaFor(typeTag[A8]) :: ScalaReflection.schemaFor(typeTag[A9]) :: ScalaReflection.schemaFor(typeTag[A10]) :: Nil).toOption - val udf = SparkUserDefinedFunction.create(f, dataType, inputSchemas) + 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) if (nullable) udf else udf.asNonNullable() } @@ -4099,8 +4145,8 @@ object functions { * @since 2.3.0 */ def udf(f: UDF0[_], returnType: DataType): UserDefinedFunction = { - val func = f.asInstanceOf[UDF0[Any]].call() - SparkUserDefinedFunction.create(() => func, returnType, inputSchemas = None) + val func = () => f.asInstanceOf[UDF0[Any]].call() + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(0)(None)) } /** @@ -4114,7 +4160,7 @@ object functions { */ def udf(f: UDF1[_, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF1[Any, Any]].call(_: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(1)(None)) } /** @@ -4128,7 +4174,7 @@ object functions { */ def udf(f: UDF2[_, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF2[Any, Any, Any]].call(_: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(2)(None)) } /** @@ -4142,7 +4188,7 @@ object functions { */ def udf(f: UDF3[_, _, _, _], returnType: DataType): UserDefinedFunction = { val func = f.asInstanceOf[UDF3[Any, Any, Any, Any]].call(_: Any, _: Any, _: Any) - SparkUserDefinedFunction.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(3)(None)) } /** @@ -4156,7 +4202,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(4)(None)) } /** @@ -4170,7 +4216,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(5)(None)) } /** @@ -4184,7 +4230,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(6)(None)) } /** @@ -4198,7 +4244,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(7)(None)) } /** @@ -4212,7 +4258,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(8)(None)) } /** @@ -4226,7 +4272,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(9)(None)) } /** @@ -4240,7 +4286,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.create(func, returnType, inputSchemas = None) + SparkUserDefinedFunction(func, returnType, inputSchemas = Seq.fill(10)(None)) } // scalastyle:on parameter.number @@ -4252,6 +4298,13 @@ object functions { * By default the returned UDF is deterministic. To change it to nondeterministic, call the * API `UserDefinedFunction.asNondeterministic()`. * + * Note that, although the Scala closure can have primitive-type function argument, it doesn't + * work well with null values. Because the Scala closure is passed in as Any type, there is no + * type information for the function arguments. Without the type 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. + * * @param f A closure in Scala * @param dataType The output data type of the UDF * @@ -4259,7 +4312,7 @@ object functions { * @since 2.0.0 */ def udf(f: AnyRef, dataType: DataType): UserDefinedFunction = { - SparkUserDefinedFunction.create(f, dataType, inputSchemas = None) + SparkUserDefinedFunction(f, dataType, inputSchemas = 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 3a0db7e16c23a..b05a5dfea3fff 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 @@ -17,16 +17,18 @@ package org.apache.spark.sql.internal import org.apache.spark.SparkConf -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql.{ExperimentalMethods, SparkSession, UDFRegistration, _} +import org.apache.spark.sql.catalog.v2.CatalogPlugin import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog.SessionCatalog import org.apache.spark.sql.catalyst.optimizer.Optimizer import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} +import org.apache.spark.sql.execution.{ColumnarRule, QueryExecution, SparkOptimizer, SparkPlanner, SparkSqlParser} import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.v2.{V2StreamingScanSupportCheck, V2WriteSupportCheck} import org.apache.spark.sql.streaming.StreamingQueryManager import org.apache.spark.sql.util.ExecutionListenerManager @@ -50,7 +52,7 @@ import org.apache.spark.sql.util.ExecutionListenerManager * and `catalog` fields. Note that the state is cloned when `build` is called, and not before. */ @Experimental -@InterfaceStability.Unstable +@Unstable abstract class BaseSessionStateBuilder( val session: SparkSession, val parentState: Option[SessionState] = None) { @@ -80,13 +82,21 @@ abstract class BaseSessionStateBuilder( /** * SQL-specific key-value configurations. * - * These either get cloned from a pre-existing instance or newly created. The conf is always - * merged with its [[SparkConf]]. + * These either get cloned from a pre-existing instance or newly created. The conf is merged + * with its [[SparkConf]] only when there is no parent session. */ protected lazy val conf: SQLConf = { - val conf = parentState.map(_.conf.clone()).getOrElse(new SQLConf) - mergeSparkConf(conf, session.sparkContext.conf) - conf + parentState.map { s => + val cloned = s.conf.clone() + if (session.sparkContext.conf.get(StaticSQLConf.SQL_LEGACY_SESSION_INIT_WITH_DEFAULTS)) { + mergeSparkConf(cloned, session.sparkContext.conf) + } + cloned + }.getOrElse { + val conf = new SQLConf + mergeSparkConf(conf, session.sparkContext.conf) + conf + } } /** @@ -95,7 +105,8 @@ abstract class BaseSessionStateBuilder( * This either gets cloned from a pre-existing version or cloned from the built-in registry. */ protected lazy val functionRegistry: FunctionRegistry = { - parentState.map(_.functionRegistry).getOrElse(FunctionRegistry.builtin).clone() + parentState.map(_.functionRegistry.clone()) + .getOrElse(extensions.registerFunctions(FunctionRegistry.builtin.clone())) } /** @@ -158,6 +169,8 @@ abstract class BaseSessionStateBuilder( override val extendedResolutionRules: Seq[Rule[LogicalPlan]] = new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallBackFileSourceV2(session) +: + DataSourceResolution(conf, this) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -170,7 +183,11 @@ abstract class BaseSessionStateBuilder( PreWriteCheck +: PreReadCheck +: HiveOnlyCheck +: + V2WriteSupportCheck +: + V2StreamingScanSupportCheck +: customCheckRules + + override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name) } /** @@ -247,6 +264,10 @@ abstract class BaseSessionStateBuilder( extensions.buildPlannerStrategies(session) } + protected def columnarRules: Seq[ColumnarRule] = { + extensions.buildColumnarRules(session) + } + /** * Create a query execution object. */ @@ -266,8 +287,8 @@ abstract class BaseSessionStateBuilder( * This gets cloned from parent if available, otherwise a new instance is created. */ protected def listenerManager: ExecutionListenerManager = { - parentState.map(_.listenerManager.clone()).getOrElse( - new ExecutionListenerManager(session.sparkContext.conf)) + parentState.map(_.listenerManager.clone(session)).getOrElse( + new ExecutionListenerManager(session, loadExtensions = true)) } /** @@ -297,7 +318,8 @@ abstract class BaseSessionStateBuilder( listenerManager, () => resourceLoader, createQueryExecution, - createClone) + createClone, + columnarRules) } } @@ -308,13 +330,14 @@ private[sql] trait WithTestConf { self: BaseSessionStateBuilder => def overrideConfs: Map[String, String] override protected lazy val conf: SQLConf = { + val overrideConfigurations = overrideConfs val conf = parentState.map(_.conf.clone()).getOrElse { new SQLConf { clear() override def clear(): Unit = { super.clear() // Make sure we start with the default test configs even after clear - overrideConfs.foreach { case (key, value) => setConfString(key, value) } + overrideConfigurations.foreach { case (key, value) => setConfString(key, value) } } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala index 4698e8ab13ce3..baa8ebee66108 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala @@ -365,7 +365,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { override def dropTempView(viewName: String): Boolean = { sparkSession.sessionState.catalog.getTempView(viewName).exists { viewDef => sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession, viewDef, cascade = false, blocking = true) + sparkSession, viewDef, cascade = false) sessionCatalog.dropTempView(viewName) } } @@ -381,7 +381,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { override def dropGlobalTempView(viewName: String): Boolean = { sparkSession.sessionState.catalog.getGlobalTempView(viewName).exists { viewDef => sparkSession.sharedState.cacheManager.uncacheQuery( - sparkSession, viewDef, cascade = false, blocking = true) + sparkSession, viewDef, cascade = false) sessionCatalog.dropGlobalTempView(viewName) } } @@ -492,11 +492,17 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog { // If this table is cached as an InMemoryRelation, drop the original // cached version and make the new version cached lazily. - if (isCached(table)) { - // Uncache the logicalPlan. - sparkSession.sharedState.cacheManager.uncacheQuery(table, cascade = true, blocking = true) - // Cache it again. - sparkSession.sharedState.cacheManager.cacheQuery(table, Some(tableIdent.table)) + val cache = sparkSession.sharedState.cacheManager.lookupCachedData(table) + if (cache.nonEmpty) { + // save the cache name and cache level for recreation + val cacheName = cache.get.cachedRepresentation.cacheBuilder.tableName + val cacheLevel = cache.get.cachedRepresentation.cacheBuilder.storageLevel + + // uncache the logical plan. + sparkSession.sharedState.cacheManager.uncacheQuery(table, cascade = true) + + // recache with the same name and cache level. + sparkSession.sharedState.cacheManager.cacheQuery(table, cacheName, cacheLevel) } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala index eca612f06f9bb..4921e3ca903c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/HiveSerDe.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.internal import java.util.Locale +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat case class HiveSerDe( @@ -74,8 +75,10 @@ object HiveSerDe { def sourceToSerDe(source: String): Option[HiveSerDe] = { val key = source.toLowerCase(Locale.ROOT) match { case s if s.startsWith("org.apache.spark.sql.parquet") => "parquet" + case s if s.startsWith("org.apache.spark.sql.execution.datasources.parquet") => "parquet" case s if s.startsWith("org.apache.spark.sql.orc") => "orc" case s if s.startsWith("org.apache.spark.sql.hive.orc") => "orc" + case s if s.startsWith("org.apache.spark.sql.execution.datasources.orc") => "orc" case s if s.equals("orcfile") => "orc" case s if s.equals("parquetfile") => "parquet" case s if s.equals("avrofile") => "avro" @@ -86,7 +89,16 @@ object HiveSerDe { } def getDefaultStorage(conf: SQLConf): CatalogStorageFormat = { - val defaultStorageType = conf.getConfString("hive.default.fileformat", "textfile") + // To respect hive-site.xml, it peeks Hadoop configuration from existing Spark session, + // as an easy workaround. See SPARK-27555. + val defaultFormatKey = "hive.default.fileformat" + val defaultValue = { + val defaultFormatValue = "textfile" + SparkSession.getActiveSession.map { session => + session.sessionState.newHadoopConf().get(defaultFormatKey, defaultFormatValue) + }.getOrElse(defaultFormatValue) + } + val defaultStorageType = conf.getConfString("hive.default.fileformat", defaultValue) val defaultHiveSerde = sourceToSerDe(defaultStorageType) CatalogStorageFormat.empty.copy( inputFormat = defaultHiveSerde.flatMap(_.inputFormat) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala index accbea41b9603..b962ab6feabcb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SessionState.scala @@ -23,7 +23,7 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.spark.SparkContext -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry} import org.apache.spark.sql.catalyst.catalog._ @@ -71,7 +71,8 @@ private[sql] class SessionState( val listenerManager: ExecutionListenerManager, resourceLoaderBuilder: () => SessionResourceLoader, createQueryExecution: LogicalPlan => QueryExecution, - createClone: (SparkSession, SessionState) => SessionState) { + createClone: (SparkSession, SessionState) => SessionState, + val columnarRules: Seq[ColumnarRule]) { // The following fields are lazy to avoid creating the Hive client when creating SessionState. lazy val catalog: SessionCatalog = catalogBuilder() @@ -124,7 +125,7 @@ private[sql] object SessionState { * Concrete implementation of a [[BaseSessionStateBuilder]]. */ @Experimental -@InterfaceStability.Unstable +@Unstable class SessionStateBuilder( session: SparkSession, parentState: Option[SessionState] = None) @@ -135,7 +136,7 @@ class SessionStateBuilder( /** * Session shared [[FunctionResourceLoader]]. */ -@InterfaceStability.Unstable +@Unstable class SessionResourceLoader(session: SparkSession) extends FunctionResourceLoader { override def loadResource(resource: FunctionResource): Unit = { resource.resourceType match { 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 5b6160e2b408f..f1a648176c3b3 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 @@ -34,13 +34,19 @@ import org.apache.spark.sql.execution.CacheManager import org.apache.spark.sql.execution.ui.{SQLAppStatusListener, SQLAppStatusStore, SQLTab} import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.status.ElementTrackingStore -import org.apache.spark.util.{MutableURLClassLoader, Utils} +import org.apache.spark.util.Utils /** * A class that holds all state shared across sessions in a given [[SQLContext]]. + * + * @param sparkContext The Spark context associated with this SharedState + * @param initialConfigs The configs from the very first created SparkSession */ -private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { +private[sql] class SharedState( + val sparkContext: SparkContext, + initialConfigs: scala.collection.Map[String, String]) + extends Logging { // 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. @@ -77,6 +83,27 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { } 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) = { + val confClone = sparkContext.conf.clone() + val hadoopConfClone = new Configuration(sparkContext.hadoopConfiguration) + // If `SparkSession` is instantiated using an existing `SparkContext` instance and no existing + // `SharedState`, all `SparkSession` level configurations have higher priority to generate a + // `SharedState` instance. This will be done only once then shared across `SparkSession`s + initialConfigs.foreach { + case (k, _) if k == "hive.metastore.warehouse.dir" || k == WAREHOUSE_PATH.key => + logWarning(s"Not allowing to set ${WAREHOUSE_PATH.key} or hive.metastore.warehouse.dir " + + s"in SparkSession's options, it should be set statically for cross-session usages") + case (k, v) => + logDebug(s"Applying initial SparkSession options to SparkConf/HadoopConf: $k -> $v") + confClone.set(k, v) + hadoopConfClone.set(k, v) + + } + (confClone, hadoopConfClone) + } /** * Class for caching query results reused in future executions. @@ -89,7 +116,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { */ val statusStore: SQLAppStatusStore = { val kvStore = sparkContext.statusStore.store.asInstanceOf[ElementTrackingStore] - val listener = new SQLAppStatusListener(sparkContext.conf, kvStore, live = true) + val listener = new SQLAppStatusListener(conf, kvStore, live = true) sparkContext.listenerBus.addToStatusQueue(listener) val statusStore = new SQLAppStatusStore(kvStore, Some(listener)) sparkContext.ui.foreach(new SQLTab(statusStore, _)) @@ -101,9 +128,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { */ lazy val externalCatalog: ExternalCatalogWithListener = { val externalCatalog = SharedState.reflect[ExternalCatalog, SparkConf, Configuration]( - SharedState.externalCatalogClassName(sparkContext.conf), - sparkContext.conf, - sparkContext.hadoopConfiguration) + SharedState.externalCatalogClassName(conf), conf, hadoopConf) val defaultDbDefinition = CatalogDatabase( SessionCatalog.DEFAULT_DATABASE, @@ -121,11 +146,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { val wrapped = new ExternalCatalogWithListener(externalCatalog) // Make sure we propagate external catalog events to the spark listener bus - wrapped.addListener(new ExternalCatalogEventListener { - override def onEvent(event: ExternalCatalogEvent): Unit = { - sparkContext.listenerBus.post(event) - } - }) + wrapped.addListener((event: ExternalCatalogEvent) => sparkContext.listenerBus.post(event)) wrapped } @@ -137,7 +158,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging { // System preserved database should not exists in metastore. However it's hard to guarantee it // for every session, because case-sensitivity differs. Here we always lowercase it to make our // life easier. - val globalTempDB = sparkContext.conf.get(GLOBAL_TEMP_DATABASE).toLowerCase(Locale.ROOT) + val globalTempDB = conf.get(GLOBAL_TEMP_DATABASE) if (externalCatalog.databaseExists(globalTempDB)) { throw new SparkException( s"$globalTempDB is a system preserved database, please rename your existing database " + @@ -193,14 +214,3 @@ object SharedState extends Logging { } } } - - -/** - * URL class loader that exposes the `addURL` and `getURLs` methods in URLClassLoader. - * This class loader cannot be closed (its `close` method is a no-op). - */ -private[sql] class NonClosableMutableURLClassLoader(parent: ClassLoader) - extends MutableURLClassLoader(Array.empty, parent) { - - override def close(): Unit = {} -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala index f76c1fae562c6..230b43022b02b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JdbcDialects.scala @@ -21,8 +21,7 @@ import java.sql.{Connection, Date, Timestamp} import org.apache.commons.lang3.StringUtils -import org.apache.spark.annotation.{DeveloperApi, InterfaceStability, Since} -import org.apache.spark.sql.execution.datasources.jdbc.JDBCOptions +import org.apache.spark.annotation.{DeveloperApi, Evolving, Since} import org.apache.spark.sql.types._ /** @@ -34,7 +33,7 @@ import org.apache.spark.sql.types._ * send a null value to the database. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) /** @@ -57,7 +56,7 @@ case class JdbcType(databaseTypeDefinition : String, jdbcNullType : Int) * for the given Catalyst type. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving abstract class JdbcDialect extends Serializable { /** * Check if this dialect instance can handle a certain jdbc url. @@ -197,7 +196,7 @@ abstract class JdbcDialect extends Serializable { * sure to register your dialects first. */ @DeveloperApi -@InterfaceStability.Evolving +@Evolving object JdbcDialects { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala index da787b4859a73..805f73dee141b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/MsSqlServerDialect.scala @@ -30,7 +30,11 @@ private object MsSqlServerDialect extends JdbcDialect { // String is recommend by Microsoft SQL Server for datetimeoffset types in non-MS clients Option(StringType) } else { - None + sqlType match { + case java.sql.Types.SMALLINT => Some(ShortType) + case java.sql.Types.REAL => Some(FloatType) + case _ => None + } } } @@ -38,6 +42,8 @@ private object MsSqlServerDialect extends JdbcDialect { case TimestampType => Some(JdbcType("DATETIME", java.sql.Types.TIMESTAMP)) case StringType => Some(JdbcType("NVARCHAR(MAX)", java.sql.Types.NVARCHAR)) case BooleanType => Some(JdbcType("BIT", java.sql.Types.BIT)) + case BinaryType => Some(JdbcType("VARBINARY(MAX)", java.sql.Types.VARBINARY)) + case ShortType => Some(JdbcType("SMALLINT", java.sql.Types.SMALLINT)) case _ => None } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala index f8d2bc8e0f13f..2645e4c9d528b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/PostgresDialect.scala @@ -60,7 +60,10 @@ private object PostgresDialect extends JdbcDialect { case "bytea" => Some(BinaryType) case "timestamp" | "timestamptz" | "time" | "timetz" => Some(TimestampType) case "date" => Some(DateType) - case "numeric" | "decimal" => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" if precision > 0 => Some(DecimalType.bounded(precision, scale)) + case "numeric" | "decimal" => + // SPARK-26538: handle numeric without explicit precision and scale. + Some(DecimalType. SYSTEM_DEFAULT) case _ => None } @@ -70,14 +73,13 @@ private object PostgresDialect extends JdbcDialect { case BooleanType => Some(JdbcType("BOOLEAN", Types.BOOLEAN)) case FloatType => Some(JdbcType("FLOAT4", Types.FLOAT)) case DoubleType => Some(JdbcType("FLOAT8", Types.DOUBLE)) - case ShortType => Some(JdbcType("SMALLINT", Types.SMALLINT)) + case ShortType | ByteType => Some(JdbcType("SMALLINT", Types.SMALLINT)) case t: DecimalType => Some( JdbcType(s"NUMERIC(${t.precision},${t.scale})", java.sql.Types.NUMERIC)) case ArrayType(et, _) if et.isInstanceOf[AtomicType] => getJDBCType(et).map(_.databaseTypeDefinition) .orElse(JdbcUtils.getCommonJDBCType(et).map(_.databaseTypeDefinition)) .map(typeName => JdbcType(s"$typeName[]", java.sql.Types.ARRAY)) - case ByteType => throw new IllegalArgumentException(s"Unsupported type in postgresql: $dt"); case _ => None } 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 161e0102f0b43..61875931d226e 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 @@ -17,7 +17,7 @@ package org.apache.spark -import org.apache.spark.annotation.{DeveloperApi, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Unstable} import org.apache.spark.sql.execution.SparkStrategy /** @@ -40,8 +40,17 @@ package object sql { * [[org.apache.spark.sql.sources]] */ @DeveloperApi - @InterfaceStability.Unstable + @Unstable type Strategy = SparkStrategy type DataFrame = Dataset[Row] + + /** + * Metadata key which is used to write Spark version in the followings: + * - Parquet file metadata + * - ORC file metadata + * + * Note that Hive table property `spark.sql.create.version` also has Spark version. + */ + private[sql] val SPARK_VERSION_METADATA_KEY = "org.apache.spark.version" } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index 6057a795c8bf5..6ad054c9f6403 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.sources -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation._ import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow @@ -35,7 +35,7 @@ import org.apache.spark.sql.types.StructType * * @since 1.5.0 */ -@InterfaceStability.Stable +@Stable trait DataSourceRegister { /** @@ -65,7 +65,7 @@ trait DataSourceRegister { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait RelationProvider { /** * Returns a new base relation with the given parameters. @@ -96,7 +96,7 @@ trait RelationProvider { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait SchemaRelationProvider { /** * Returns a new base relation with the given parameters and user defined schema. @@ -117,7 +117,7 @@ trait SchemaRelationProvider { * @since 2.0.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait StreamSourceProvider { /** @@ -148,7 +148,7 @@ trait StreamSourceProvider { * @since 2.0.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait StreamSinkProvider { def createSink( sqlContext: SQLContext, @@ -160,7 +160,7 @@ trait StreamSinkProvider { /** * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait CreatableRelationProvider { /** * Saves a DataFrame to a destination (using data source-specific parameters) @@ -192,7 +192,7 @@ trait CreatableRelationProvider { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable abstract class BaseRelation { def sqlContext: SQLContext def schema: StructType @@ -242,7 +242,7 @@ abstract class BaseRelation { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait TableScan { def buildScan(): RDD[Row] } @@ -253,7 +253,7 @@ trait TableScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait PrunedScan { def buildScan(requiredColumns: Array[String]): RDD[Row] } @@ -271,7 +271,7 @@ trait PrunedScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait PrunedFilteredScan { def buildScan(requiredColumns: Array[String], filters: Array[Filter]): RDD[Row] } @@ -293,7 +293,7 @@ trait PrunedFilteredScan { * * @since 1.3.0 */ -@InterfaceStability.Stable +@Stable trait InsertableRelation { def insert(data: DataFrame, overwrite: Boolean): Unit } @@ -309,7 +309,7 @@ trait InsertableRelation { * @since 1.3.0 */ @Experimental -@InterfaceStability.Unstable +@Unstable trait CatalystScan { def buildScan(requiredColumns: Seq[Attribute], filters: Seq[Expression]): RDD[Row] } 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 4c7dcedafeeae..bb536b6fee47c 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 @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, Dataset, SparkSession} import org.apache.spark.sql.execution.command.DDLUtils @@ -29,10 +29,10 @@ import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming.{StreamingRelation, StreamingRelationV2} import org.apache.spark.sql.sources.StreamSourceProvider -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, MicroBatchReadSupport} +import org.apache.spark.sql.sources.v2._ +import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.types.StructType -import org.apache.spark.util.Utils +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * Interface used to load a streaming `Dataset` from external storage systems (e.g. file systems, @@ -40,7 +40,7 @@ import org.apache.spark.util.Utils * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving final class DataStreamReader private[sql](sparkSession: SparkSession) extends Logging { /** * Specifies the input data source format. @@ -83,6 +83,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 2.0.0 @@ -120,6 +123,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 2.0.0 @@ -136,6 +142,9 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo *
      *
    • `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.
    • + *
    • `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.
    • *
    * * @since 2.0.0 @@ -158,7 +167,8 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo "read files of Hive data source directly.") } - val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf).newInstance() + val ds = DataSource.lookupDataSource(source, sparkSession.sqlContext.conf). + getConstructor().newInstance() // We need to generate the V1 data source so we can pass it to the V2 relation as a shim. // We can't be sure at this point whether we'll actually want to use V2, since we don't know the // writer or whether the query is continuous. @@ -172,60 +182,29 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo case _ => None } ds match { - case s: MicroBatchReadSupportProvider => + case provider: TableProvider => val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - ds = s, conf = sparkSession.sessionState.conf) + source = provider, conf = sparkSession.sessionState.conf) val options = sessionOptions ++ extraOptions - val dataSourceOptions = new DataSourceOptions(options.asJava) - var tempReadSupport: MicroBatchReadSupport = null - val schema = try { - val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath - tempReadSupport = if (userSpecifiedSchema.isDefined) { - s.createMicroBatchReadSupport( - userSpecifiedSchema.get, tmpCheckpointPath, dataSourceOptions) - } else { - s.createMicroBatchReadSupport(tmpCheckpointPath, dataSourceOptions) - } - tempReadSupport.fullSchema() - } finally { - // Stop tempReader to avoid side-effect thing - if (tempReadSupport != null) { - tempReadSupport.stop() - tempReadSupport = null - } + val dsOptions = new CaseInsensitiveStringMap(options.asJava) + val table = userSpecifiedSchema match { + case Some(schema) => provider.getTable(dsOptions, schema) + case _ => provider.getTable(dsOptions) } - Dataset.ofRows( - sparkSession, - StreamingRelationV2( - s, source, options, - schema.toAttributes, v1Relation)(sparkSession)) - case s: ContinuousReadSupportProvider => - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - ds = s, conf = sparkSession.sessionState.conf) - val options = sessionOptions ++ extraOptions - val dataSourceOptions = new DataSourceOptions(options.asJava) - var tempReadSupport: ContinuousReadSupport = null - val schema = try { - val tmpCheckpointPath = Utils.createTempDir(namePrefix = s"tempCP").getCanonicalPath - tempReadSupport = if (userSpecifiedSchema.isDefined) { - s.createContinuousReadSupport( - userSpecifiedSchema.get, tmpCheckpointPath, dataSourceOptions) - } else { - s.createContinuousReadSupport(tmpCheckpointPath, dataSourceOptions) - } - tempReadSupport.fullSchema() - } finally { - // Stop tempReader to avoid side-effect thing - if (tempReadSupport != null) { - tempReadSupport.stop() - tempReadSupport = null - } + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + table match { + case _: SupportsRead if table.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => + Dataset.ofRows( + sparkSession, + StreamingRelationV2( + provider, source, table, dsOptions, table.schema.toAttributes, v1Relation)( + sparkSession)) + + // fallback to v1 + // TODO (SPARK-27483): we should move this fallback logic to an analyzer rule. + case _ => Dataset.ofRows(sparkSession, StreamingRelation(v1DataSource)) } - Dataset.ofRows( - sparkSession, - StreamingRelationV2( - s, source, options, - schema.toAttributes, v1Relation)(sparkSession)) + case _ => // Code path for data source v1. Dataset.ofRows(sparkSession, StreamingRelation(v1DataSource)) @@ -272,7 +251,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * during parsing. *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. To * keep corrupt records, an user can set a string type field named * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have the * field, it drops corrupt records during parsing. When inferring a schema, it implicitly @@ -285,17 +264,19 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
    • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
    • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
    • + * `java.time.format.DateTimeFormatter`. 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 * that should be used for parsing.
    • *
    • `dropFieldIfAllNull` (default `false`): whether to ignore column of all null values or * empty array/struct during schema inference.
    • + *
    • `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. + * For instance, this is used while parsing dates and timestamps.
    • *
    * * @since 2.0.0 @@ -344,11 +325,11 @@ 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.text.SimpleDateFormat`. This applies to - * date type.
  • + * Custom date formats follow the formats at `java.time.format.DateTimeFormatter`. + * This applies to date type. *
  • `timestampFormat` (default `yyyy-MM-dd'T'HH:mm:ss.SSSXXX`): sets the string that * indicates a timestamp format. Custom date formats follow the formats at - * `java.text.SimpleDateFormat`. This applies to timestamp type.
  • + * `java.time.format.DateTimeFormatter`. 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 @@ -357,13 +338,13 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo * during parsing. It supports the following case-insensitive modes. *
      *
    • `PERMISSIVE` : when it meets a corrupted record, puts the malformed string into a - * field configured by `columnNameOfCorruptRecord`, and sets other fields to `null`. To keep - * corrupt records, an user can set a string type field named `columnNameOfCorruptRecord` - * in an user-defined schema. If a schema does not have the field, it drops corrupt records - * during parsing. A record with less/more tokens than schema is not a corrupted record to - * CSV. When it meets a record having fewer tokens than the length of the schema, sets - * `null` to extra fields. When the record has more tokens than the length of the schema, - * it drops extra tokens.
    • + * field configured by `columnNameOfCorruptRecord`, and sets malformed fields to `null`. + * To keep corrupt records, an user can set a string type field named + * `columnNameOfCorruptRecord` in an user-defined schema. If a schema does not have + * the field, it drops corrupt records during parsing. A record with less/more tokens + * than schema is not a corrupted record to CSV. When it meets a record having fewer + * tokens than the length of the schema, sets `null` to extra fields. When the record + * has more tokens than the length of the schema, it drops extra tokens. *
    • `DROPMALFORMED` : ignores the whole corrupted records.
    • *
    • `FAILFAST` : throws an exception when it meets corrupted records.
    • *
    @@ -372,6 +353,10 @@ 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`.
  • *
  • `multiLine` (default `false`): parse one record, which may span multiple lines.
  • + *
  • `locale` (default is `en-US`): sets a locale as language tag in IETF BCP 47 format. + * For instance, this is used while parsing dates and timestamps.
  • + *
  • `lineSep` (default covers all `\r`, `\r\n` and `\n`): defines the line separator + * that should be used for parsing. Maximum length is 1 character.
  • * * * @since 2.0.0 @@ -415,6 +400,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo /** * Loads text files and returns a `DataFrame` whose schema starts with a string column named * "value", and followed by partitioned columns if there are any. + * The text files must be encoded as UTF-8. * * By default, each line in the text files is a new row in the resulting DataFrame. For example: * {{{ @@ -442,6 +428,7 @@ final class DataStreamReader private[sql](sparkSession: SparkSession) extends Lo /** * Loads text file(s) and returns a `Dataset` of String. The underlying schema of the Dataset * contains a single string column named "value". + * The text files must be encoded as UTF-8. * * If the directory structure of the text files contains partitioning information, those are * ignored in the resulting Dataset. To include partitioning information as columns, use `text`. 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 b23e86a786459..36104d7a70443 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 @@ -21,7 +21,7 @@ import java.util.Locale import scala.collection.JavaConverters._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.api.java.function.VoidFunction2 import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.streaming.InternalOutputModes @@ -29,9 +29,10 @@ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.execution.datasources.DataSource import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Utils import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger import org.apache.spark.sql.execution.streaming.sources._ -import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider +import org.apache.spark.sql.sources.v2.{SupportsWrite, TableProvider} +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap /** * Interface used to write a streaming `Dataset` to external storage systems (e.g. file systems, @@ -39,21 +40,23 @@ import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { private val df = ds.toDF() /** * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be - * written to the sink - * - `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written - * to the sink every time these is some updates - * - `OutputMode.Update()`: only the rows that were updated in the streaming DataFrame/Dataset - * will be written to the sink every time there are some updates. If - * the query doesn't contain aggregations, it will be equivalent to - * `OutputMode.Append()` mode. + *
      + *
    • `OutputMode.Append()`: only the new rows in the streaming DataFrame/Dataset will be + * written to the sink.
    • + *
    • `OutputMode.Complete()`: all the rows in the streaming DataFrame/Dataset will be written + * to the sink every time there are some updates.
    • + *
    • `OutputMode.Update()`: only the rows that were updated in the streaming + * DataFrame/Dataset will be written to the sink every time there are some updates. + * If the query doesn't contain aggregations, it will be equivalent to + * `OutputMode.Append()` mode.
    • + *
    * * @since 2.0.0 */ @@ -64,13 +67,16 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { /** * Specifies how data of a streaming DataFrame/Dataset is written to a streaming sink. - * - `append`: only the new rows in the streaming DataFrame/Dataset will be written to - * the sink - * - `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink - * every time these is some updates - * - `update`: only the rows that were updated in the streaming DataFrame/Dataset will - * be written to the sink every time there are some updates. If the query doesn't - * contain aggregations, it will be equivalent to `append` mode. + *
      + *
    • `append`: only the new rows in the streaming DataFrame/Dataset will be written to + * the sink.
    • + *
    • `complete`: all the rows in the streaming DataFrame/Dataset will be written to the sink + * every time there are some updates.
    • + *
    • `update`: only the rows that were updated in the streaming DataFrame/Dataset will + * be written to the sink every time there are some updates. If the query doesn't + * contain aggregations, it will be equivalent to `append` mode.
    • + *
    + * * @since 2.0.0 */ def outputMode(outputMode: String): DataStreamWriter[T] = { @@ -131,8 +137,10 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * laid out on the file system similar to Hive's partitioning scheme. As an example, when we * partition a dataset by year and then month, the directory layout would look like: * - * - year=2016/month=01/ - * - year=2016/month=02/ + *
      + *
    • year=2016/month=01/
    • + *
    • year=2016/month=02/
    • + *
    * * Partitioning is one of the most widely used techniques to optimize physical data layout. * It provides a coarse-grained index for skipping unnecessary data reads when queries have @@ -245,16 +253,8 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { if (extraOptions.get("queryName").isEmpty) { throw new AnalysisException("queryName must be specified for memory sink") } - val (sink, resultDf) = trigger match { - case _: ContinuousTrigger => - val s = new MemorySinkV2() - val r = Dataset.ofRows(df.sparkSession, new MemoryPlanV2(s, df.schema.toAttributes)) - (s, r) - case _ => - val s = new MemorySink(df.schema, outputMode) - val r = Dataset.ofRows(df.sparkSession, new MemoryPlan(s)) - (s, r) - } + val sink = new MemorySink() + val resultDf = Dataset.ofRows(df.sparkSession, new MemoryPlan(sink, df.schema.toAttributes)) val chkpointLoc = extraOptions.get("checkpointLocation") val recoverFromChkpoint = outputMode == OutputMode.Complete() val query = df.sparkSession.sessionState.streamingQueryManager.startQuery( @@ -271,7 +271,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { query } else if (source == "foreach") { assertNotPartitioned("foreach") - val sink = ForeachWriteSupportProvider[T](foreachWriter, ds.exprEnc) + val sink = ForeachWriterTable[T](foreachWriter, ds.exprEnc) df.sparkSession.sessionState.streamingQueryManager.startQuery( extraOptions.get("queryName"), extraOptions.get("checkpointLocation"), @@ -297,38 +297,48 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { useTempCheckpointLocation = true, trigger = trigger) } else { - val ds = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) + val cls = DataSource.lookupDataSource(source, df.sparkSession.sessionState.conf) val disabledSources = df.sparkSession.sqlContext.conf.disabledV2StreamingWriters.split(",") - var options = extraOptions.toMap - val sink = ds.newInstance() match { - case w: StreamingWriteSupportProvider - if !disabledSources.contains(w.getClass.getCanonicalName) => - val sessionOptions = DataSourceV2Utils.extractSessionConfigs( - w, df.sparkSession.sessionState.conf) - options = sessionOptions ++ extraOptions - w - case _ => - val ds = DataSource( - df.sparkSession, - className = source, - options = options, - partitionColumns = normalizedParCols.getOrElse(Nil)) - ds.createSink(outputMode) + val useV1Source = disabledSources.contains(cls.getCanonicalName) + + val sink = if (classOf[TableProvider].isAssignableFrom(cls) && !useV1Source) { + val provider = cls.getConstructor().newInstance().asInstanceOf[TableProvider] + val sessionOptions = DataSourceV2Utils.extractSessionConfigs( + source = provider, conf = df.sparkSession.sessionState.conf) + val options = sessionOptions ++ extraOptions + val dsOptions = new CaseInsensitiveStringMap(options.asJava) + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + provider.getTable(dsOptions) match { + case table: SupportsWrite if table.supports(STREAMING_WRITE) => + table + case _ => createV1Sink() + } + } else { + createV1Sink() } df.sparkSession.sessionState.streamingQueryManager.startQuery( - options.get("queryName"), - options.get("checkpointLocation"), + extraOptions.get("queryName"), + extraOptions.get("checkpointLocation"), df, - options, + extraOptions.toMap, sink, outputMode, - useTempCheckpointLocation = source == "console", + useTempCheckpointLocation = source == "console" || source == "noop", recoverFromCheckpointLocation = true, trigger = trigger) } } + private def createV1Sink(): Sink = { + val ds = DataSource( + df.sparkSession, + className = source, + options = extraOptions.toMap, + partitionColumns = normalizedParCols.getOrElse(Nil)) + ds.createSink(outputMode) + } + /** * Sets the output of the streaming query to be processed using the provided writer object. * object. See [[org.apache.spark.sql.ForeachWriter]] for more details on the lifecycle and @@ -358,7 +368,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.4.0 */ - @InterfaceStability.Evolving + @Evolving def foreachBatch(function: (Dataset[T], Long) => Unit): DataStreamWriter[T] = { this.source = "foreachBatch" if (function == null) throw new IllegalArgumentException("foreachBatch function cannot be null") @@ -379,7 +389,7 @@ final class DataStreamWriter[T] private[sql](ds: Dataset[T]) { * * @since 2.4.0 */ - @InterfaceStability.Evolving + @Evolving def foreachBatch(function: VoidFunction2[Dataset[T], java.lang.Long]): DataStreamWriter[T] = { foreachBatch((batchDs: Dataset[T], batchId: Long) => function.call(batchDs, batchId)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala index e9510c903acae..ab68eba81b843 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/GroupState.scala @@ -17,8 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.annotation.{Experimental, InterfaceStability} -import org.apache.spark.sql.KeyValueGroupedDataset +import org.apache.spark.annotation.{Evolving, Experimental} import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState /** @@ -192,7 +191,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalGroupState * @since 2.2.0 */ @Experimental -@InterfaceStability.Evolving +@Evolving trait GroupState[S] extends LogicalGroupState[S] { /** Whether state exists or not. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala deleted file mode 100644 index a033575d3d38f..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/ProcessingTime.scala +++ /dev/null @@ -1,146 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.streaming - -import java.util.concurrent.TimeUnit - -import scala.concurrent.duration.Duration - -import org.apache.commons.lang3.StringUtils - -import org.apache.spark.annotation.InterfaceStability -import org.apache.spark.unsafe.types.CalendarInterval - -/** - * A trigger that runs a query periodically based on the processing time. If `interval` is 0, - * the query will run as fast as possible. - * - * Scala Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * Java Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - */ -@InterfaceStability.Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -case class ProcessingTime(intervalMs: Long) extends Trigger { - require(intervalMs >= 0, "the interval of trigger should not be negative") -} - -/** - * Used to create [[ProcessingTime]] triggers for [[StreamingQuery]]s. - * - * @since 2.0.0 - */ -@InterfaceStability.Evolving -@deprecated("use Trigger.ProcessingTime(intervalMs)", "2.2.0") -object ProcessingTime { - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: String): ProcessingTime = { - if (StringUtils.isBlank(interval)) { - throw new IllegalArgumentException( - "interval cannot be null or blank.") - } - val cal = if (interval.startsWith("interval")) { - CalendarInterval.fromString(interval) - } else { - CalendarInterval.fromString("interval " + interval) - } - if (cal == null) { - throw new IllegalArgumentException(s"Invalid interval: $interval") - } - if (cal.months > 0) { - throw new IllegalArgumentException(s"Doesn't support month or year interval: $interval") - } - new ProcessingTime(cal.microseconds / 1000) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import scala.concurrent.duration._ - * df.writeStream.trigger(ProcessingTime(10.seconds)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def apply(interval: Duration): ProcessingTime = { - new ProcessingTime(interval.toMillis) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * df.writeStream.trigger(ProcessingTime.create("10 seconds")) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval) - */ - @deprecated("use Trigger.ProcessingTime(interval)", "2.2.0") - def create(interval: String): ProcessingTime = { - apply(interval) - } - - /** - * Create a [[ProcessingTime]]. If `interval` is 0, the query will run as fast as possible. - * - * Example: - * {{{ - * import java.util.concurrent.TimeUnit - * df.writeStream.trigger(ProcessingTime.create(10, TimeUnit.SECONDS)) - * }}} - * - * @since 2.0.0 - * @deprecated use Trigger.ProcessingTime(interval, unit) - */ - @deprecated("use Trigger.ProcessingTime(interval, unit)", "2.2.0") - def create(interval: Long, unit: TimeUnit): ProcessingTime = { - new ProcessingTime(unit.toMillis(interval)) - } -} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala index f2dfbe42260d7..47ddc88e964e8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQuery.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import java.util.UUID -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.sql.SparkSession /** @@ -27,7 +27,7 @@ import org.apache.spark.sql.SparkSession * All these methods are thread-safe. * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving trait StreamingQuery { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala index 03aeb14de502a..646d6888b2a16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryException.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.streaming -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * Exception that stopped a [[StreamingQuery]]. Use `cause` get the actual exception @@ -28,7 +28,7 @@ import org.apache.spark.annotation.InterfaceStability * @param endOffset Ending offset in json of the range of data in exception occurred * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryException private[sql]( private val queryDebugString: String, val message: String, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala index 6aa82b89ede81..916d6a0365965 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryListener.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.streaming import java.util.UUID -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.scheduler.SparkListenerEvent /** @@ -28,7 +28,7 @@ import org.apache.spark.scheduler.SparkListenerEvent * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving abstract class StreamingQueryListener { import StreamingQueryListener._ @@ -67,14 +67,14 @@ abstract class StreamingQueryListener { * Companion object of [[StreamingQueryListener]] that defines the listener events. * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving object StreamingQueryListener { /** * Base type of [[StreamingQueryListener]] events * @since 2.0.0 */ - @InterfaceStability.Evolving + @Evolving trait Event extends SparkListenerEvent /** @@ -84,7 +84,7 @@ object StreamingQueryListener { * @param name User-specified name of the query, null if not specified. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryStartedEvent private[sql]( val id: UUID, val runId: UUID, @@ -95,7 +95,7 @@ object StreamingQueryListener { * @param progress The query progress updates. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryProgressEvent private[sql](val progress: StreamingQueryProgress) extends Event /** @@ -107,7 +107,7 @@ object StreamingQueryListener { * with an exception. Otherwise, it will be `None`. * @since 2.1.0 */ - @InterfaceStability.Evolving + @Evolving class QueryTerminatedEvent private[sql]( val id: UUID, val runId: UUID, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala index cd52d991d55c9..abee5f6017df8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryManager.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.streaming import java.util.UUID +import java.util.concurrent.TimeUnit import javax.annotation.concurrent.GuardedBy import scala.collection.mutable @@ -25,16 +26,16 @@ import scala.collection.mutable import org.apache.hadoop.fs.Path import org.apache.spark.SparkException -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, DataFrame, SparkSession} import org.apache.spark.sql.catalyst.analysis.UnsupportedOperationChecker import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, ContinuousTrigger} +import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution import org.apache.spark.sql.execution.streaming.state.StateStoreCoordinatorRef import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS -import org.apache.spark.sql.sources.v2.StreamingWriteSupportProvider +import org.apache.spark.sql.sources.v2.{SupportsWrite, Table} import org.apache.spark.util.{Clock, SystemClock, Utils} /** @@ -42,7 +43,7 @@ import org.apache.spark.util.{Clock, SystemClock, Utils} * * @since 2.0.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Logging { private[sql] val stateStoreCoordinator = @@ -151,8 +152,10 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo @throws[StreamingQueryException] def awaitAnyTermination(timeoutMs: Long): Boolean = { - val startTime = System.currentTimeMillis - def isTimedout = System.currentTimeMillis - startTime >= timeoutMs + val startTime = System.nanoTime() + def isTimedout = { + System.nanoTime() - startTime >= TimeUnit.MILLISECONDS.toNanos(timeoutMs) + } awaitTerminationLock.synchronized { while (!isTimedout && lastTerminatedQuery == null) { @@ -206,7 +209,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo userSpecifiedCheckpointLocation: Option[String], df: DataFrame, extraOptions: Map[String, String], - sink: BaseStreamingSink, + sink: Table, outputMode: OutputMode, useTempCheckpointLocation: Boolean, recoverFromCheckpointLocation: Boolean, @@ -214,16 +217,20 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo triggerClock: Clock): StreamingQueryWrapper = { var deleteCheckpointOnStop = false val checkpointLocation = userSpecifiedCheckpointLocation.map { userSpecified => - new Path(userSpecified).toUri.toString + new Path(userSpecified).toString }.orElse { df.sparkSession.sessionState.conf.checkpointLocation.map { location => - new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toUri.toString + new Path(location, userSpecifiedName.getOrElse(UUID.randomUUID().toString)).toString } }.getOrElse { if (useTempCheckpointLocation) { - // Delete the temp checkpoint when a query is being stopped without errors. deleteCheckpointOnStop = true - Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath + val tempDir = Utils.createTempDir(namePrefix = s"temporary").getCanonicalPath + logWarning("Temporary checkpoint location created which is deleted normally when" + + s" the query didn't fail: $tempDir. If it's required to delete it under any" + + s" circumstances, please set ${SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key} to" + + s" true. Important to know deleting temp checkpoint folder is best effort.") + tempDir } else { throw new AnalysisException( "checkpointLocation must be specified either " + @@ -246,9 +253,7 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo val analyzedPlan = df.queryExecution.analyzed df.queryExecution.assertAnalyzed() - if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { - UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) - } + val operationCheckEnabled = sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled if (sparkSession.sessionState.conf.adaptiveExecutionEnabled) { logWarning(s"${SQLConf.ADAPTIVE_EXECUTION_ENABLED.key} " + @@ -256,8 +261,8 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo } (sink, trigger) match { - case (v2Sink: StreamingWriteSupportProvider, trigger: ContinuousTrigger) => - if (sparkSession.sessionState.conf.isUnsupportedOperationCheckEnabled) { + case (table: SupportsWrite, trigger: ContinuousTrigger) => + if (operationCheckEnabled) { UnsupportedOperationChecker.checkForContinuous(analyzedPlan, outputMode) } new StreamingQueryWrapper(new ContinuousExecution( @@ -265,13 +270,16 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo userSpecifiedName.orNull, checkpointLocation, analyzedPlan, - v2Sink, + table, trigger, triggerClock, outputMode, extraOptions, deleteCheckpointOnStop)) case _ => + if (operationCheckEnabled) { + UnsupportedOperationChecker.checkForStreaming(analyzedPlan, outputMode) + } new StreamingQueryWrapper(new MicroBatchExecution( sparkSession, userSpecifiedName.orNull, @@ -307,11 +315,11 @@ class StreamingQueryManager private[sql] (sparkSession: SparkSession) extends Lo userSpecifiedCheckpointLocation: Option[String], df: DataFrame, extraOptions: Map[String, String], - sink: BaseStreamingSink, + sink: Table, outputMode: OutputMode, useTempCheckpointLocation: Boolean = false, recoverFromCheckpointLocation: Boolean = true, - trigger: Trigger = ProcessingTime(0), + trigger: Trigger = Trigger.ProcessingTime(0), triggerClock: Clock = new SystemClock()): StreamingQuery = { val query = createQuery( userSpecifiedName, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala index a0c9bcc8929eb..6ca9aacab7247 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/StreamingQueryStatus.scala @@ -22,19 +22,21 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving /** * Reports information about the instantaneous status of a streaming query. * * @param message A human readable description of what the stream is currently doing. - * @param isDataAvailable True when there is new data to be processed. + * @param isDataAvailable True when there is new data to be processed. Doesn't apply + * to ContinuousExecution where it is always false. * @param isTriggerActive True when the trigger is actively firing, false when waiting for the - * next trigger time. + * next trigger time. Doesn't apply to ContinuousExecution where it is + * always false. * * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryStatus protected[sql]( val message: String, val isDataAvailable: Boolean, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala index f2173aa1e59c2..0b3945cbd1323 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/streaming/progress.scala @@ -29,12 +29,13 @@ import org.json4s.JsonAST.JValue import org.json4s.JsonDSL._ import org.json4s.jackson.JsonMethods._ -import org.apache.spark.annotation.InterfaceStability +import org.apache.spark.annotation.Evolving +import org.apache.spark.sql.streaming.SinkProgress.DEFAULT_NUM_OUTPUT_ROWS /** * Information about updates made to stateful operators in a [[StreamingQuery]] during a trigger. */ -@InterfaceStability.Evolving +@Evolving class StateOperatorProgress private[sql]( val numRowsTotal: Long, val numRowsUpdated: Long, @@ -94,7 +95,7 @@ class StateOperatorProgress private[sql]( * @param sources detailed statistics on data being read from each of the streaming sources. * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class StreamingQueryProgress private[sql]( val id: UUID, val runId: UUID, @@ -165,7 +166,7 @@ class StreamingQueryProgress private[sql]( * Spark. * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class SourceProgress protected[sql]( val description: String, val startOffset: String, @@ -207,11 +208,19 @@ class SourceProgress protected[sql]( * during a trigger. See [[StreamingQueryProgress]] for more information. * * @param description Description of the source corresponding to this status. + * @param numOutputRows Number of rows written to the sink or -1 for Continuous Mode (temporarily) + * or Sink V1 (until decommissioned). * @since 2.1.0 */ -@InterfaceStability.Evolving +@Evolving class SinkProgress protected[sql]( - val description: String) extends Serializable { + val description: String, + val numOutputRows: Long) extends Serializable { + + /** SinkProgress without custom metrics. */ + protected[sql] def this(description: String) { + this(description, DEFAULT_NUM_OUTPUT_ROWS) + } /** The compact JSON representation of this progress. */ def json: String = compact(render(jsonValue)) @@ -222,6 +231,14 @@ class SinkProgress protected[sql]( override def toString: String = prettyJson private[sql] def jsonValue: JValue = { - ("description" -> JString(description)) + ("description" -> JString(description)) ~ + ("numOutputRows" -> JInt(numOutputRows)) } } + +private[sql] object SinkProgress { + val DEFAULT_NUM_OUTPUT_ROWS: Long = -1L + + def apply(description: String, numOutputRows: Option[Long]): SinkProgress = + new SinkProgress(description, numOutputRows.getOrElse(DEFAULT_NUM_OUTPUT_ROWS)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala index 8bab7e1c58762..7beac16599de5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/ExamplePointUDT.scala @@ -45,7 +45,7 @@ private[sql] class ExamplePointUDT extends UserDefinedType[ExamplePoint] { override def sqlType: DataType = ArrayType(DoubleType, false) - override def pyUDT: String = "pyspark.sql.tests.ExamplePointUDT" + override def pyUDT: String = "pyspark.testing.sqlutils.ExamplePointUDT" override def serialize(p: ExamplePoint): GenericArrayData = { val output = new Array[Any](2) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala index 2b46233e1a5df..77ae047705de0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/util/QueryExecutionListener.scala @@ -17,17 +17,16 @@ package org.apache.spark.sql.util -import java.util.concurrent.locks.ReentrantReadWriteLock +import scala.collection.JavaConverters._ -import scala.collection.mutable.ListBuffer -import scala.util.control.NonFatal - -import org.apache.spark.SparkConf -import org.apache.spark.annotation.{DeveloperApi, Experimental, InterfaceStability} +import org.apache.spark.annotation.{DeveloperApi, Evolving, Experimental} import org.apache.spark.internal.Logging +import org.apache.spark.scheduler.{SparkListener, SparkListenerEvent} +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd import org.apache.spark.sql.internal.StaticSQLConf._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{ListenerBus, Utils} /** * :: Experimental :: @@ -37,7 +36,7 @@ import org.apache.spark.util.Utils * multiple different threads. */ @Experimental -@InterfaceStability.Evolving +@Evolving trait QueryExecutionListener { /** @@ -74,11 +73,19 @@ trait QueryExecutionListener { * Manager for [[QueryExecutionListener]]. See `org.apache.spark.sql.SQLContext.listenerManager`. */ @Experimental -@InterfaceStability.Evolving -class ExecutionListenerManager private extends Logging { - - private[sql] def this(conf: SparkConf) = { - this() +@Evolving +// The `session` is used to indicate which session carries this listener manager, and we only +// catch SQL executions which are launched by the same session. +// The `loadExtensions` flag is used to indicate whether we should load the pre-defined, +// user-specified listeners during construction. We should not do it when cloning this listener +// manager, as we will copy all listeners to the cloned listener manager. +class ExecutionListenerManager private[sql](session: SparkSession, loadExtensions: Boolean) + extends Logging { + + private val listenerBus = new ExecutionListenerBus(session) + + if (loadExtensions) { + val conf = session.sparkContext.conf conf.get(QUERY_EXECUTION_LISTENERS).foreach { classNames => Utils.loadExtensions(classOf[QueryExecutionListener], classNames, conf).foreach(register) } @@ -88,82 +95,63 @@ class ExecutionListenerManager private extends Logging { * Registers the specified [[QueryExecutionListener]]. */ @DeveloperApi - def register(listener: QueryExecutionListener): Unit = writeLock { - listeners += listener + def register(listener: QueryExecutionListener): Unit = { + listenerBus.addListener(listener) } /** * Unregisters the specified [[QueryExecutionListener]]. */ @DeveloperApi - def unregister(listener: QueryExecutionListener): Unit = writeLock { - listeners -= listener + def unregister(listener: QueryExecutionListener): Unit = { + listenerBus.removeListener(listener) } /** * Removes all the registered [[QueryExecutionListener]]. */ @DeveloperApi - def clear(): Unit = writeLock { - listeners.clear() + def clear(): Unit = { + listenerBus.removeAllListeners() } /** * Get an identical copy of this listener manager. */ - @DeveloperApi - override def clone(): ExecutionListenerManager = writeLock { - val newListenerManager = new ExecutionListenerManager - listeners.foreach(newListenerManager.register) + private[sql] def clone(session: SparkSession): ExecutionListenerManager = { + val newListenerManager = new ExecutionListenerManager(session, loadExtensions = false) + listenerBus.listeners.asScala.foreach(newListenerManager.register) newListenerManager } +} - private[sql] def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { - readLock { - withErrorHandling { listener => - listener.onSuccess(funcName, qe, duration) - } - } - } - - private[sql] def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = { - readLock { - withErrorHandling { listener => - listener.onFailure(funcName, qe, exception) - } - } - } - - private[this] val listeners = ListBuffer.empty[QueryExecutionListener] +private[sql] class ExecutionListenerBus(session: SparkSession) + extends SparkListener with ListenerBus[QueryExecutionListener, SparkListenerSQLExecutionEnd] { - /** A lock to prevent updating the list of listeners while we are traversing through them. */ - private[this] val lock = new ReentrantReadWriteLock() + session.sparkContext.listenerBus.addToSharedQueue(this) - private def withErrorHandling(f: QueryExecutionListener => Unit): Unit = { - for (listener <- listeners) { - try { - f(listener) - } catch { - case NonFatal(e) => logWarning("Error executing query execution listener", e) - } - } + override def onOtherEvent(event: SparkListenerEvent): Unit = event match { + case e: SparkListenerSQLExecutionEnd => postToAll(e) + case _ => } - /** Acquires a read lock on the cache for the duration of `f`. */ - private def readLock[A](f: => A): A = { - val rl = lock.readLock() - rl.lock() - try f finally { - rl.unlock() + override protected def doPostEvent( + listener: QueryExecutionListener, + event: SparkListenerSQLExecutionEnd): Unit = { + if (shouldReport(event)) { + val funcName = event.executionName.get + event.executionFailure match { + case Some(ex) => + listener.onFailure(funcName, event.qe, ex) + case _ => + listener.onSuccess(funcName, event.qe, event.duration) + } } } - /** Acquires a write lock on the cache for the duration of `f`. */ - private def writeLock[A](f: => A): A = { - val wl = lock.writeLock() - wl.lock() - try f finally { - wl.unlock() - } + private def shouldReport(e: SparkListenerSQLExecutionEnd): Boolean = { + // Only catch SQL execution with a name, and triggered by the same spark session that this + // listener manager belongs. + e.executionName.isDefined && e.qe.sparkSession.eq(this.session) } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java new file mode 100644 index 0000000000000..7bf0789b43d63 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaBeanDeserializationSuite.java @@ -0,0 +1,608 @@ +/* + * 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 test.org.apache.spark.sql; + +import java.io.Serializable; +import java.time.Instant; +import java.time.LocalDate; +import java.util.*; + +import org.apache.spark.sql.*; +import org.apache.spark.sql.catalyst.expressions.GenericRow; +import org.apache.spark.sql.catalyst.util.DateTimeUtils; +import org.apache.spark.sql.catalyst.util.TimestampFormatter; +import org.apache.spark.sql.internal.SQLConf; +import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.types.StructType; +import org.junit.*; + +import org.apache.spark.sql.test.TestSparkSession; + +public class JavaBeanDeserializationSuite implements Serializable { + + private TestSparkSession spark; + + @Before + public void setUp() { + spark = new TestSparkSession(); + } + + @After + public void tearDown() { + spark.stop(); + spark = null; + } + + private static final List ARRAY_RECORDS = new ArrayList<>(); + + static { + ARRAY_RECORDS.add( + new ArrayRecord(1, Arrays.asList(new Interval(111, 211), new Interval(121, 221)), + new int[] { 11, 12, 13, 14 }) + ); + ARRAY_RECORDS.add( + new ArrayRecord(2, Arrays.asList(new Interval(112, 212), new Interval(122, 222)), + new int[] { 21, 22, 23, 24 }) + ); + ARRAY_RECORDS.add( + new ArrayRecord(3, Arrays.asList(new Interval(113, 213), new Interval(123, 223)), + new int[] { 31, 32, 33, 34 }) + ); + } + + @Test + public void testBeanWithArrayFieldDeserialization() { + Encoder encoder = Encoders.bean(ArrayRecord.class); + + Dataset dataset = spark + .read() + .format("json") + .schema("id int, intervals array>, " + + "ints array") + .load("src/test/resources/test-data/with-array-fields.json") + .as(encoder); + + List records = dataset.collectAsList(); + Assert.assertEquals(records, ARRAY_RECORDS); + } + + private static final List MAP_RECORDS = new ArrayList<>(); + + static { + MAP_RECORDS.add(new MapRecord(1, + toMap(Arrays.asList("a", "b"), Arrays.asList(new Interval(111, 211), new Interval(121, 221))) + )); + MAP_RECORDS.add(new MapRecord(2, + toMap(Arrays.asList("a", "b"), Arrays.asList(new Interval(112, 212), new Interval(122, 222))) + )); + MAP_RECORDS.add(new MapRecord(3, + toMap(Arrays.asList("a", "b"), Arrays.asList(new Interval(113, 213), new Interval(123, 223))) + )); + MAP_RECORDS.add(new MapRecord(4, new HashMap<>())); + MAP_RECORDS.add(new MapRecord(5, null)); + } + + private static Map toMap(Collection keys, Collection values) { + Map map = new HashMap<>(); + Iterator keyI = keys.iterator(); + Iterator valueI = values.iterator(); + while (keyI.hasNext() && valueI.hasNext()) { + map.put(keyI.next(), valueI.next()); + } + return map; + } + + @Test + public void testBeanWithMapFieldsDeserialization() { + + Encoder encoder = Encoders.bean(MapRecord.class); + + Dataset dataset = spark + .read() + .format("json") + .schema("id int, intervals map>") + .load("src/test/resources/test-data/with-map-fields.json") + .as(encoder); + + List records = dataset.collectAsList(); + + Assert.assertEquals(records, MAP_RECORDS); + } + + @Test + public void testSpark22000() { + List inputRows = new ArrayList<>(); + List expectedRecords = new ArrayList<>(); + + for (long idx = 0 ; idx < 5 ; idx++) { + Row row = createRecordSpark22000Row(idx); + inputRows.add(row); + expectedRecords.add(createRecordSpark22000(row)); + } + + // Here we try to convert the fields, from any types to string. + // Before applying SPARK-22000, Spark called toString() against variable which type might + // be primitive. + // SPARK-22000 it calls String.valueOf() which finally calls toString() but handles boxing + // if the type is primitive. + Encoder encoder = Encoders.bean(RecordSpark22000.class); + + StructType schema = new StructType() + .add("shortField", DataTypes.ShortType) + .add("intField", DataTypes.IntegerType) + .add("longField", DataTypes.LongType) + .add("floatField", DataTypes.FloatType) + .add("doubleField", DataTypes.DoubleType) + .add("stringField", DataTypes.StringType) + .add("booleanField", DataTypes.BooleanType) + .add("timestampField", DataTypes.TimestampType) + // explicitly setting nullable = true to make clear the intention + .add("nullIntField", DataTypes.IntegerType, true); + + Dataset dataFrame = spark.createDataFrame(inputRows, schema); + Dataset dataset = dataFrame.as(encoder); + + List records = dataset.collectAsList(); + + Assert.assertEquals(expectedRecords, records); + } + + @Test + public void testSpark22000FailToUpcast() { + List inputRows = new ArrayList<>(); + for (long idx = 0 ; idx < 5 ; idx++) { + Row row = createRecordSpark22000FailToUpcastRow(idx); + inputRows.add(row); + } + + // Here we try to convert the fields, from string type to int, which upcast doesn't help. + Encoder encoder = + Encoders.bean(RecordSpark22000FailToUpcast.class); + + StructType schema = new StructType().add("id", DataTypes.StringType); + + Dataset dataFrame = spark.createDataFrame(inputRows, schema); + + try { + dataFrame.as(encoder).collect(); + Assert.fail("Expected AnalysisException, but passed."); + } catch (Throwable e) { + // Here we need to handle weird case: compiler complains AnalysisException never be thrown + // in try statement, but it can be thrown actually. Maybe Scala-Java interop issue? + if (e instanceof AnalysisException) { + Assert.assertTrue(e.getMessage().contains("Cannot up cast ")); + } else { + throw e; + } + } + } + + private static Row createRecordSpark22000Row(Long index) { + Object[] values = new Object[] { + index.shortValue(), + index.intValue(), + index, + index.floatValue(), + index.doubleValue(), + String.valueOf(index), + index % 2 == 0, + new java.sql.Timestamp(System.currentTimeMillis()), + null + }; + return new GenericRow(values); + } + + private static RecordSpark22000 createRecordSpark22000(Row recordRow) { + RecordSpark22000 record = new RecordSpark22000(); + record.setShortField(String.valueOf(recordRow.getShort(0))); + record.setIntField(String.valueOf(recordRow.getInt(1))); + record.setLongField(String.valueOf(recordRow.getLong(2))); + record.setFloatField(String.valueOf(recordRow.getFloat(3))); + record.setDoubleField(String.valueOf(recordRow.getDouble(4))); + record.setStringField(recordRow.getString(5)); + record.setBooleanField(String.valueOf(recordRow.getBoolean(6))); + record.setTimestampField(String.valueOf(recordRow.getTimestamp(7))); + // This would figure out that null value will not become "null". + record.setNullIntField(null); + return record; + } + + private static Row createRecordSpark22000FailToUpcastRow(Long index) { + Object[] values = new Object[] { String.valueOf(index) }; + return new GenericRow(values); + } + + public static class ArrayRecord { + + private int id; + private List intervals; + private int[] ints; + + public ArrayRecord() { } + + ArrayRecord(int id, List intervals, int[] ints) { + this.id = id; + this.intervals = intervals; + this.ints = ints; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + public List getIntervals() { + return intervals; + } + + public void setIntervals(List intervals) { + this.intervals = intervals; + } + + public int[] getInts() { + return ints; + } + + public void setInts(int[] ints) { + this.ints = ints; + } + + @Override + public int hashCode() { + return id ^ Objects.hashCode(intervals) ^ Objects.hashCode(ints); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof ArrayRecord)) return false; + ArrayRecord other = (ArrayRecord) obj; + return (other.id == this.id) && Objects.equals(other.intervals, this.intervals) && + Arrays.equals(other.ints, ints); + } + + @Override + public String toString() { + return String.format("{ id: %d, intervals: %s, ints: %s }", id, intervals, + Arrays.toString(ints)); + } + } + + public static class MapRecord { + + private int id; + private Map intervals; + + public MapRecord() { } + + MapRecord(int id, Map intervals) { + this.id = id; + this.intervals = intervals; + } + + public int getId() { + return id; + } + + public void setId(int id) { + this.id = id; + } + + public Map getIntervals() { + return intervals; + } + + public void setIntervals(Map intervals) { + this.intervals = intervals; + } + + @Override + public int hashCode() { + return id ^ Objects.hashCode(intervals); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof MapRecord)) return false; + MapRecord other = (MapRecord) obj; + return (other.id == this.id) && Objects.equals(other.intervals, this.intervals); + } + + @Override + public String toString() { + return String.format("{ id: %d, intervals: %s }", id, intervals); + } + } + + public static class Interval { + + private long startTime; + private long endTime; + + public Interval() { } + + Interval(long startTime, long endTime) { + this.startTime = startTime; + this.endTime = endTime; + } + + public long getStartTime() { + return startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + } + + public long getEndTime() { + return endTime; + } + + public void setEndTime(long endTime) { + this.endTime = endTime; + } + + @Override + public int hashCode() { + return Long.hashCode(startTime) ^ Long.hashCode(endTime); + } + + @Override + public boolean equals(Object obj) { + if (!(obj instanceof Interval)) return false; + Interval other = (Interval) obj; + return (other.startTime == this.startTime) && (other.endTime == this.endTime); + } + + @Override + public String toString() { + return String.format("[%d,%d]", startTime, endTime); + } + } + + public static final class RecordSpark22000 { + private String shortField; + private String intField; + private String longField; + private String floatField; + private String doubleField; + private String stringField; + private String booleanField; + private String timestampField; + private String nullIntField; + + public RecordSpark22000() { } + + public String getShortField() { + return shortField; + } + + public void setShortField(String shortField) { + this.shortField = shortField; + } + + public String getIntField() { + return intField; + } + + public void setIntField(String intField) { + this.intField = intField; + } + + public String getLongField() { + return longField; + } + + public void setLongField(String longField) { + this.longField = longField; + } + + public String getFloatField() { + return floatField; + } + + public void setFloatField(String floatField) { + this.floatField = floatField; + } + + public String getDoubleField() { + return doubleField; + } + + public void setDoubleField(String doubleField) { + this.doubleField = doubleField; + } + + public String getStringField() { + return stringField; + } + + public void setStringField(String stringField) { + this.stringField = stringField; + } + + public String getBooleanField() { + return booleanField; + } + + public void setBooleanField(String booleanField) { + this.booleanField = booleanField; + } + + public String getTimestampField() { + return timestampField; + } + + public void setTimestampField(String timestampField) { + this.timestampField = timestampField; + } + + public String getNullIntField() { + return nullIntField; + } + + public void setNullIntField(String nullIntField) { + this.nullIntField = nullIntField; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + RecordSpark22000 that = (RecordSpark22000) o; + return Objects.equals(shortField, that.shortField) && + Objects.equals(intField, that.intField) && + Objects.equals(longField, that.longField) && + Objects.equals(floatField, that.floatField) && + Objects.equals(doubleField, that.doubleField) && + Objects.equals(stringField, that.stringField) && + Objects.equals(booleanField, that.booleanField) && + Objects.equals(timestampField, that.timestampField) && + Objects.equals(nullIntField, that.nullIntField); + } + + @Override + public int hashCode() { + return Objects.hash(shortField, intField, longField, floatField, doubleField, stringField, + booleanField, timestampField, nullIntField); + } + + @Override + public String toString() { + return com.google.common.base.Objects.toStringHelper(this) + .add("shortField", shortField) + .add("intField", intField) + .add("longField", longField) + .add("floatField", floatField) + .add("doubleField", doubleField) + .add("stringField", stringField) + .add("booleanField", booleanField) + .add("timestampField", timestampField) + .add("nullIntField", nullIntField) + .toString(); + } + } + + public static final class RecordSpark22000FailToUpcast { + private Integer id; + + public RecordSpark22000FailToUpcast() { + } + + public Integer getId() { + return id; + } + + public void setId(Integer id) { + this.id = id; + } + } + + @Test + public void testBeanWithLocalDateAndInstant() { + String originConf = spark.conf().get(SQLConf.DATETIME_JAVA8API_ENABLED().key()); + try { + spark.conf().set(SQLConf.DATETIME_JAVA8API_ENABLED().key(), "true"); + List inputRows = new ArrayList<>(); + List expectedRecords = new ArrayList<>(); + + for (long idx = 0 ; idx < 5 ; idx++) { + Row row = createLocalDateInstantRow(idx); + inputRows.add(row); + expectedRecords.add(createLocalDateInstantRecord(row)); + } + + Encoder encoder = Encoders.bean(LocalDateInstantRecord.class); + + StructType schema = new StructType() + .add("localDateField", DataTypes.DateType) + .add("instantField", DataTypes.TimestampType); + + Dataset dataFrame = spark.createDataFrame(inputRows, schema); + Dataset dataset = dataFrame.as(encoder); + + List records = dataset.collectAsList(); + + Assert.assertEquals(expectedRecords, records); + } finally { + spark.conf().set(SQLConf.DATETIME_JAVA8API_ENABLED().key(), originConf); + } + } + + public static final class LocalDateInstantRecord { + private String localDateField; + private String instantField; + + public LocalDateInstantRecord() { } + + public String getLocalDateField() { + return localDateField; + } + + public void setLocalDateField(String localDateField) { + this.localDateField = localDateField; + } + + public String getInstantField() { + return instantField; + } + + public void setInstantField(String instantField) { + this.instantField = instantField; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + LocalDateInstantRecord that = (LocalDateInstantRecord) o; + return Objects.equals(localDateField, that.localDateField) && + Objects.equals(instantField, that.instantField); + } + + @Override + public int hashCode() { + return Objects.hash(localDateField, instantField); + } + + @Override + public String toString() { + return com.google.common.base.Objects.toStringHelper(this) + .add("localDateField", localDateField) + .add("instantField", instantField) + .toString(); + } + } + + private static Row createLocalDateInstantRow(Long index) { + Object[] values = new Object[] { LocalDate.ofEpochDay(42), Instant.ofEpochSecond(42) }; + return new GenericRow(values); + } + + private static LocalDateInstantRecord createLocalDateInstantRecord(Row recordRow) { + LocalDateInstantRecord record = new LocalDateInstantRecord(); + record.setLocalDateField(String.valueOf(recordRow.getLocalDate(0))); + Instant instant = recordRow.getInstant(1); + TimestampFormatter formatter = TimestampFormatter.getFractionFormatter( + DateTimeUtils.getZoneId(SQLConf.get().sessionLocalTimeZone())); + record.setInstantField(formatter.format(DateTimeUtils.instantToMicros(instant))); + return record; + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java index 2c695fc58fd8c..1e5f55e494b70 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaDatasetSuite.java @@ -21,6 +21,8 @@ import java.math.BigDecimal; import java.sql.Date; import java.sql.Timestamp; +import java.time.Instant; +import java.time.LocalDate; import java.util.*; import org.apache.spark.sql.streaming.GroupStateTimeout; @@ -399,6 +401,16 @@ public void testPrimitiveEncoder() { Assert.assertEquals(data, ds.collectAsList()); } + @Test + public void testLocalDateAndInstantEncoders() { + Encoder> encoder = + Encoders.tuple(Encoders.LOCALDATE(), Encoders.INSTANT()); + List> data = + Arrays.asList(new Tuple2<>(LocalDate.ofEpochDay(0), Instant.ofEpochSecond(0))); + Dataset> ds = spark.createDataset(data, encoder); + Assert.assertEquals(data, ds.collectAsList()); + } + public static class KryoSerializable { String value; diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java index 3ab4db2a035d3..ca78d6489ef5c 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaRowSuite.java @@ -67,20 +67,20 @@ public void setUp() { public void constructSimpleRow() { Row simpleRow = RowFactory.create( byteValue, // ByteType - new Byte(byteValue), + Byte.valueOf(byteValue), shortValue, // ShortType - new Short(shortValue), + Short.valueOf(shortValue), intValue, // IntegerType - new Integer(intValue), + Integer.valueOf(intValue), longValue, // LongType - new Long(longValue), + Long.valueOf(longValue), floatValue, // FloatType - new Float(floatValue), + Float.valueOf(floatValue), doubleValue, // DoubleType - new Double(doubleValue), + Double.valueOf(doubleValue), decimalValue, // DecimalType booleanValue, // BooleanType - new Boolean(booleanValue), + Boolean.valueOf(booleanValue), stringValue, // StringType binaryValue, // BinaryType dateValue, // DateType diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java index b90224f2ae397..5955eabe496df 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaStringLength.java @@ -25,6 +25,6 @@ public class JavaStringLength implements UDF1 { @Override public Integer call(String str) throws Exception { - return new Integer(str.length()); + return Integer.valueOf(str.length()); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java index 5bf1888826186..9af5023acf391 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/JavaUDFSuite.java @@ -18,8 +18,10 @@ package test.org.apache.spark.sql; import java.io.Serializable; +import java.time.LocalDate; import java.util.List; +import org.apache.spark.sql.internal.SQLConf; import org.junit.After; import org.junit.Assert; import org.junit.Before; @@ -121,4 +123,20 @@ public void udf6Test() { Row result = spark.sql("SELECT returnOne()").head(); Assert.assertEquals(1, result.getInt(0)); } + + @SuppressWarnings("unchecked") + @Test + public void udf7Test() { + String originConf = spark.conf().get(SQLConf.DATETIME_JAVA8API_ENABLED().key()); + try { + spark.conf().set(SQLConf.DATETIME_JAVA8API_ENABLED().key(), "true"); + spark.udf().register( + "plusDay", + (java.time.LocalDate ld) -> ld.plusDays(1), DataTypes.DateType); + Row result = spark.sql("SELECT plusDay(DATE '2019-02-26')").head(); + Assert.assertEquals(LocalDate.parse("2019-02-27"), result.get(0)); + } finally { + spark.conf().set(SQLConf.DATETIME_JAVA8API_ENABLED().key(), originConf); + } + } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java index 97f3dc588ecc5..92dabc79d2bff 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/execution/sort/RecordBinaryComparatorSuite.java @@ -18,6 +18,7 @@ package test.org.apache.spark.sql.execution.sort; import org.apache.spark.SparkConf; +import org.apache.spark.internal.config.package$; import org.apache.spark.memory.TaskMemoryManager; import org.apache.spark.memory.TestMemoryConsumer; import org.apache.spark.memory.TestMemoryManager; @@ -41,7 +42,8 @@ public class RecordBinaryComparatorSuite { private final TaskMemoryManager memoryManager = new TaskMemoryManager( - new TestMemoryManager(new SparkConf().set("spark.memory.offHeap.enabled", "false")), 0); + new TestMemoryManager( + new SparkConf().set(package$.MODULE$.MEMORY_OFFHEAP_ENABLED(), false)), 0); private final TestMemoryConsumer consumer = new TestMemoryConsumer(memoryManager); private final int uaoSize = UnsafeAlignedOffset.getUaoSize(); diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java index 5602310219a74..255a9f887878b 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaAdvancedDataSourceV2.java @@ -24,62 +24,29 @@ import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; import org.apache.spark.sql.sources.Filter; import org.apache.spark.sql.sources.GreaterThan; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaAdvancedDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaAdvancedDataSourceV2 implements TableProvider { - public class ReadSupport extends JavaSimpleReadSupport { - @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new AdvancedScanConfigBuilder(); - } - - @Override - public InputPartition[] planInputPartitions(ScanConfig config) { - Filter[] filters = ((AdvancedScanConfigBuilder) config).filters; - List res = new ArrayList<>(); - - Integer lowerBound = null; - for (Filter filter : filters) { - if (filter instanceof GreaterThan) { - GreaterThan f = (GreaterThan) filter; - if ("i".equals(f.attribute()) && f.value() instanceof Integer) { - lowerBound = (Integer) f.value(); - break; - } - } - } - - if (lowerBound == null) { - res.add(new JavaRangeInputPartition(0, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 4) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 5)); - res.add(new JavaRangeInputPartition(5, 10)); - } else if (lowerBound < 9) { - res.add(new JavaRangeInputPartition(lowerBound + 1, 10)); + @Override + public Table getTable(CaseInsensitiveStringMap options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new AdvancedScanBuilder(); } - - return res.stream().toArray(InputPartition[]::new); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - StructType requiredSchema = ((AdvancedScanConfigBuilder) config).requiredSchema; - return new AdvancedReaderFactory(requiredSchema); - } + }; } - public static class AdvancedScanConfigBuilder implements ScanConfigBuilder, ScanConfig, + static class AdvancedScanBuilder implements ScanBuilder, Scan, SupportsPushDownFilters, SupportsPushDownRequiredColumns { - // Exposed for testing. - public StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); - public Filter[] filters = new Filter[0]; + private StructType requiredSchema = new StructType().add("i", "int").add("j", "int"); + private Filter[] filters = new Filter[0]; @Override public void pruneColumns(StructType requiredSchema) { @@ -121,9 +88,58 @@ public Filter[] pushedFilters() { } @Override - public ScanConfig build() { + public Scan build() { return this; } + + @Override + public Batch toBatch() { + return new AdvancedBatch(requiredSchema, filters); + } + } + + public static class AdvancedBatch implements Batch { + // Exposed for testing. + public StructType requiredSchema; + public Filter[] filters; + + AdvancedBatch(StructType requiredSchema, Filter[] filters) { + this.requiredSchema = requiredSchema; + this.filters = filters; + } + + @Override + public InputPartition[] planInputPartitions() { + List res = new ArrayList<>(); + + Integer lowerBound = null; + for (Filter filter : filters) { + if (filter instanceof GreaterThan) { + GreaterThan f = (GreaterThan) filter; + if ("i".equals(f.attribute()) && f.value() instanceof Integer) { + lowerBound = (Integer) f.value(); + break; + } + } + } + + if (lowerBound == null) { + res.add(new JavaRangeInputPartition(0, 5)); + res.add(new JavaRangeInputPartition(5, 10)); + } else if (lowerBound < 4) { + res.add(new JavaRangeInputPartition(lowerBound + 1, 5)); + res.add(new JavaRangeInputPartition(5, 10)); + } else if (lowerBound < 9) { + res.add(new JavaRangeInputPartition(lowerBound + 1, 10)); + } + + return res.stream().toArray(InputPartition[]::new); + } + + @Override + public PartitionReaderFactory createReaderFactory() { + return new AdvancedReaderFactory(requiredSchema); + } } static class AdvancedReaderFactory implements PartitionReaderFactory { @@ -165,10 +181,4 @@ public void close() throws IOException { }; } } - - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java index 28a9330398310..699859cfaebe1 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaColumnarDataSourceV2.java @@ -21,21 +21,21 @@ import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.DataTypes; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; import org.apache.spark.sql.vectorized.ColumnVector; import org.apache.spark.sql.vectorized.ColumnarBatch; -public class JavaColumnarDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaColumnarDataSourceV2 implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { + class MyScanBuilder extends JavaSimpleScanBuilder { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new JavaRangeInputPartition(0, 50); partitions[1] = new JavaRangeInputPartition(50, 90); @@ -43,11 +43,21 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { + public PartitionReaderFactory createReaderFactory() { return new ColumnarReaderFactory(); } } + @Override + public Table getTable(CaseInsensitiveStringMap options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new MyScanBuilder(); + } + }; + } + static class ColumnarReaderFactory implements PartitionReaderFactory { private static final int BATCH_SIZE = 20; @@ -106,9 +116,4 @@ public void close() throws IOException { }; } } - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java index 18a11dde82198..391af5a306a16 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaPartitionAwareDataSource.java @@ -20,20 +20,24 @@ import java.io.IOException; import java.util.Arrays; +import org.apache.spark.sql.catalog.v2.expressions.Expressions; +import org.apache.spark.sql.catalog.v2.expressions.Transform; import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.*; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.sources.v2.reader.partitioning.ClusteredDistribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Distribution; import org.apache.spark.sql.sources.v2.reader.partitioning.Partitioning; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaPartitionAwareDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaPartitionAwareDataSource implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport implements SupportsReportPartitioning { + class MyScanBuilder extends JavaSimpleScanBuilder implements SupportsReportPartitioning { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new SpecificInputPartition(new int[]{1, 1, 3}, new int[]{4, 4, 6}); partitions[1] = new SpecificInputPartition(new int[]{2, 4, 4}, new int[]{6, 2, 2}); @@ -41,16 +45,31 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { + public PartitionReaderFactory createReaderFactory() { return new SpecificReaderFactory(); } @Override - public Partitioning outputPartitioning(ScanConfig config) { + public Partitioning outputPartitioning() { return new MyPartitioning(); } } + @Override + public Table getTable(CaseInsensitiveStringMap options) { + return new JavaSimpleBatchTable() { + @Override + public Transform[] partitioning() { + return new Transform[] { Expressions.identity("i") }; + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new MyScanBuilder(); + } + }; + } + static class MyPartitioning implements Partitioning { @Override @@ -106,9 +125,4 @@ public void close() throws IOException { }; } } - - @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); - } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java new file mode 100644 index 0000000000000..438f489a3eea7 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaRangeInputPartition.java @@ -0,0 +1,30 @@ +/* + * 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 test.org.apache.spark.sql.sources.v2; + +import org.apache.spark.sql.sources.v2.reader.InputPartition; + +class JavaRangeInputPartition implements InputPartition { + int start; + int end; + + JavaRangeInputPartition(int start, int end) { + this.start = start; + this.end = end; + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java new file mode 100644 index 0000000000000..f3755e18b58d5 --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaReportStatisticsDataSource.java @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.util.OptionalLong; + +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; +import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.sources.v2.reader.Statistics; +import org.apache.spark.sql.sources.v2.reader.SupportsReportStatistics; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; + +public class JavaReportStatisticsDataSource implements TableProvider { + class MyScanBuilder extends JavaSimpleScanBuilder implements SupportsReportStatistics { + @Override + public Statistics estimateStatistics() { + return new Statistics() { + @Override + public OptionalLong sizeInBytes() { + return OptionalLong.of(80); + } + + @Override + public OptionalLong numRows() { + return OptionalLong.of(10); + } + }; + } + + @Override + public InputPartition[] planInputPartitions() { + InputPartition[] partitions = new InputPartition[2]; + partitions[0] = new JavaRangeInputPartition(0, 5); + partitions[1] = new JavaRangeInputPartition(5, 10); + return partitions; + } + } + + @Override + public Table getTable(CaseInsensitiveStringMap options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new MyScanBuilder(); + } + }; + } +} diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java index cc9ac04a0dad3..3800a94f88898 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSchemaRequiredDataSource.java @@ -17,39 +17,51 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceOptions; -import org.apache.spark.sql.sources.v2.DataSourceV2; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; import org.apache.spark.sql.types.StructType; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaSchemaRequiredDataSource implements DataSourceV2, BatchReadSupportProvider { +public class JavaSchemaRequiredDataSource implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { - private final StructType schema; + class MyScanBuilder extends JavaSimpleScanBuilder { - ReadSupport(StructType schema) { + private StructType schema; + + MyScanBuilder(StructType schema) { this.schema = schema; } @Override - public StructType fullSchema() { + public StructType readSchema() { return schema; } @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { return new InputPartition[0]; } } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - throw new IllegalArgumentException("requires a user-supplied schema"); + public Table getTable(CaseInsensitiveStringMap options, StructType schema) { + return new JavaSimpleBatchTable() { + + @Override + public StructType schema() { + return schema; + } + + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new MyScanBuilder(schema); + } + }; } @Override - public BatchReadSupport createBatchReadSupport(StructType schema, DataSourceOptions options) { - return new ReadSupport(schema); + public Table getTable(CaseInsensitiveStringMap options) { + throw new IllegalArgumentException("requires a user-supplied schema"); } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java new file mode 100644 index 0000000000000..64663d5db4bed --- /dev/null +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleBatchTable.java @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package test.org.apache.spark.sql.sources.v2; + +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import org.apache.spark.sql.sources.v2.SupportsRead; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableCapability; +import org.apache.spark.sql.types.StructType; + +abstract class JavaSimpleBatchTable implements Table, SupportsRead { + private static final Set CAPABILITIES = new HashSet<>(Arrays.asList( + TableCapability.BATCH_READ, + TableCapability.BATCH_WRITE, + TableCapability.TRUNCATE)); + + @Override + public StructType schema() { + return new StructType().add("i", "int").add("j", "int"); + } + + @Override + public String name() { + return this.getClass().toString(); + } + + @Override + public Set capabilities() { + return CAPABILITIES; + } +} + diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java index 2cdbba84ec4a4..7474f36c97f75 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleDataSourceV2.java @@ -17,17 +17,17 @@ package test.org.apache.spark.sql.sources.v2; -import org.apache.spark.sql.sources.v2.BatchReadSupportProvider; -import org.apache.spark.sql.sources.v2.DataSourceV2; -import org.apache.spark.sql.sources.v2.DataSourceOptions; +import org.apache.spark.sql.sources.v2.Table; +import org.apache.spark.sql.sources.v2.TableProvider; import org.apache.spark.sql.sources.v2.reader.*; +import org.apache.spark.sql.util.CaseInsensitiveStringMap; -public class JavaSimpleDataSourceV2 implements DataSourceV2, BatchReadSupportProvider { +public class JavaSimpleDataSourceV2 implements TableProvider { - class ReadSupport extends JavaSimpleReadSupport { + class MyScanBuilder extends JavaSimpleScanBuilder { @Override - public InputPartition[] planInputPartitions(ScanConfig config) { + public InputPartition[] planInputPartitions() { InputPartition[] partitions = new InputPartition[2]; partitions[0] = new JavaRangeInputPartition(0, 5); partitions[1] = new JavaRangeInputPartition(5, 10); @@ -36,7 +36,12 @@ public InputPartition[] planInputPartitions(ScanConfig config) { } @Override - public BatchReadSupport createBatchReadSupport(DataSourceOptions options) { - return new ReadSupport(); + public Table getTable(CaseInsensitiveStringMap options) { + return new JavaSimpleBatchTable() { + @Override + public ScanBuilder newScanBuilder(CaseInsensitiveStringMap options) { + return new MyScanBuilder(); + } + }; } } diff --git a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java similarity index 56% rename from sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java rename to sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java index 685f9b9747e85..740279033c416 100644 --- a/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReadSupport.java +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleReaderFactory.java @@ -17,49 +17,11 @@ package test.org.apache.spark.sql.sources.v2; -import java.io.IOException; - import org.apache.spark.sql.catalyst.InternalRow; import org.apache.spark.sql.catalyst.expressions.GenericInternalRow; -import org.apache.spark.sql.sources.v2.reader.*; -import org.apache.spark.sql.types.StructType; - -abstract class JavaSimpleReadSupport implements BatchReadSupport { - - @Override - public StructType fullSchema() { - return new StructType().add("i", "int").add("j", "int"); - } - - @Override - public ScanConfigBuilder newScanConfigBuilder() { - return new JavaNoopScanConfigBuilder(fullSchema()); - } - - @Override - public PartitionReaderFactory createReaderFactory(ScanConfig config) { - return new JavaSimpleReaderFactory(); - } -} - -class JavaNoopScanConfigBuilder implements ScanConfigBuilder, ScanConfig { - - private StructType schema; - - JavaNoopScanConfigBuilder(StructType schema) { - this.schema = schema; - } - - @Override - public ScanConfig build() { - return this; - } - - @Override - public StructType readSchema() { - return schema; - } -} +import org.apache.spark.sql.sources.v2.reader.InputPartition; +import org.apache.spark.sql.sources.v2.reader.PartitionReader; +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; class JavaSimpleReaderFactory implements PartitionReaderFactory { @@ -70,7 +32,7 @@ public PartitionReader createReader(InputPartition partition) { private int current = p.start - 1; @Override - public boolean next() throws IOException { + public boolean next() { current += 1; return current < p.end; } @@ -81,19 +43,8 @@ public InternalRow get() { } @Override - public void close() throws IOException { - + public void close() { } }; } } - -class JavaRangeInputPartition implements InputPartition { - int start; - int end; - - JavaRangeInputPartition(int start, int end) { - this.start = start; - this.end = end; - } -} diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java similarity index 53% rename from resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala rename to sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java index 4b57b9509a655..217e66950d146 100644 --- a/resource-managers/yarn/src/test/scala/org/apache/spark/scheduler/cluster/StubApplicationAttemptId.scala +++ b/sql/core/src/test/java/test/org/apache/spark/sql/sources/v2/JavaSimpleScanBuilder.java @@ -15,34 +15,33 @@ * limitations under the License. */ -package org.apache.spark.scheduler.cluster +package test.org.apache.spark.sql.sources.v2; -import org.apache.hadoop.yarn.api.records.{ApplicationAttemptId, ApplicationId} +import org.apache.spark.sql.sources.v2.reader.Batch; +import org.apache.spark.sql.sources.v2.reader.PartitionReaderFactory; +import org.apache.spark.sql.sources.v2.reader.Scan; +import org.apache.spark.sql.sources.v2.reader.ScanBuilder; +import org.apache.spark.sql.types.StructType; -/** - * A stub application ID; can be set in constructor and/or updated later. - * @param applicationId application ID - * @param attempt an attempt counter - */ -class StubApplicationAttemptId(var applicationId: ApplicationId, var attempt: Int) - extends ApplicationAttemptId { - - override def setApplicationId(appID: ApplicationId): Unit = { - applicationId = appID - } +abstract class JavaSimpleScanBuilder implements ScanBuilder, Scan, Batch { - override def getAttemptId: Int = { - attempt + @Override + public Scan build() { + return this; } - override def setAttemptId(attemptId: Int): Unit = { - attempt = attemptId + @Override + public Batch toBatch() { + return this; } - override def getApplicationId: ApplicationId = { - applicationId + @Override + public StructType readSchema() { + return new StructType().add("i", "int").add("j", "int"); } - override def build(): Unit = { + @Override + public PartitionReaderFactory createReaderFactory() { + return new JavaSimpleReaderFactory(); } } diff --git a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister index a36b0cfa6ff18..914af589384df 100644 --- a/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister +++ b/sql/core/src/test/resources/META-INF/services/org.apache.spark.sql.sources.DataSourceRegister @@ -9,6 +9,6 @@ org.apache.spark.sql.streaming.sources.FakeReadMicroBatchOnly org.apache.spark.sql.streaming.sources.FakeReadContinuousOnly org.apache.spark.sql.streaming.sources.FakeReadBothModes org.apache.spark.sql.streaming.sources.FakeReadNeitherMode -org.apache.spark.sql.streaming.sources.FakeWriteSupportProvider +org.apache.spark.sql.streaming.sources.FakeWriteOnly org.apache.spark.sql.streaming.sources.FakeNoWrite org.apache.spark.sql.streaming.sources.FakeWriteSupportProviderV1Fallback diff --git a/sql/core/src/test/resources/sample.json b/sql/core/src/test/resources/sample.json new file mode 100644 index 0000000000000..a2c2ffd5e0330 --- /dev/null +++ b/sql/core/src/test/resources/sample.json @@ -0,0 +1,2 @@ +{"a" : "2" ,"b" : "blah", "c_!@(3)":1} +{"" : {"d!" : [4, 5], "=" : [{"Dd2": null}, {"Dd2" : true}]}} diff --git a/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql new file mode 100644 index 0000000000000..f2f4b02c8634b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/ansi/interval.sql @@ -0,0 +1,188 @@ +-- Turns on ANSI mode +SET spark.sql.parser.ansi.enabled=true; + +select + '1' second, + 2 seconds, + '1' minute, + 2 minutes, + '1' hour, + 2 hours, + '1' day, + 2 days, + '1' month, + 2 months, + '1' year, + 2 years; + +select + interval '10-11' year to month, + interval '10' year, + interval '11' month; + +select + '10-11' year to month, + '10' year, + '11' month; + +select + interval '10 9:8:7.987654321' day to second, + interval '10' day, + interval '11' hour, + interval '12' minute, + interval '13' second, + interval '13.123456789' second; + +select + '10 9:8:7.987654321' day to second, + '10' day, + '11' hour, + '12' minute, + '13' second, + '13.123456789' second; + +select map(1, interval 1 day, 2, interval 3 week); + +select map(1, 1 day, 2, 3 week); + +-- Interval year-month arithmetic + +create temporary view interval_arithmetic as + select CAST(dateval AS date), CAST(tsval AS timestamp) from values + ('2012-01-01', '2012-01-01') + as interval_arithmetic(dateval, tsval); + +select + dateval, + dateval - interval '2-2' year to month, + dateval - interval '-2-2' year to month, + dateval + interval '2-2' year to month, + dateval + interval '-2-2' year to month, + - interval '2-2' year to month + dateval, + interval '2-2' year to month + dateval +from interval_arithmetic; + +select + dateval, + dateval - '2-2' year to month, + dateval - '-2-2' year to month, + dateval + '2-2' year to month, + dateval + '-2-2' year to month, + - '2-2' year to month + dateval, + '2-2' year to month + dateval +from interval_arithmetic; + +select + tsval, + tsval - interval '2-2' year to month, + tsval - interval '-2-2' year to month, + tsval + interval '2-2' year to month, + tsval + interval '-2-2' year to month, + - interval '2-2' year to month + tsval, + interval '2-2' year to month + tsval +from interval_arithmetic; + +select + tsval, + tsval - '2-2' year to month, + tsval - '-2-2' year to month, + tsval + '2-2' year to month, + tsval + '-2-2' year to month, + - '2-2' year to month + tsval, + '2-2' year to month + tsval +from interval_arithmetic; + +select + interval '2-2' year to month + interval '3-3' year to month, + interval '2-2' year to month - interval '3-3' year to month +from interval_arithmetic; + +select + '2-2' year to month + '3-3' year to month, + '2-2' year to month - '3-3' year to month +from interval_arithmetic; + +-- Interval day-time arithmetic + +select + dateval, + dateval - interval '99 11:22:33.123456789' day to second, + dateval - interval '-99 11:22:33.123456789' day to second, + dateval + interval '99 11:22:33.123456789' day to second, + dateval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + dateval, + interval '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic; + +select + dateval, + dateval - '99 11:22:33.123456789' day to second, + dateval - '-99 11:22:33.123456789' day to second, + dateval + '99 11:22:33.123456789' day to second, + dateval + '-99 11:22:33.123456789' day to second, + - '99 11:22:33.123456789' day to second + dateval, + '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic; + +select + tsval, + tsval - interval '99 11:22:33.123456789' day to second, + tsval - interval '-99 11:22:33.123456789' day to second, + tsval + interval '99 11:22:33.123456789' day to second, + tsval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + tsval, + interval '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic; + +select + tsval, + tsval - '99 11:22:33.123456789' day to second, + tsval - '-99 11:22:33.123456789' day to second, + tsval + '99 11:22:33.123456789' day to second, + tsval + '-99 11:22:33.123456789' day to second, + - '99 11:22:33.123456789' day to second + tsval, + '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic; + +select + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second +from interval_arithmetic; + +select + '99 11:22:33.123456789' day to second + '10 9:8:7.123456789' day to second, + '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second +from interval_arithmetic; + +-- More tests for interval syntax alternatives + +select 30 day; + +select 30 day day; + +select 30 day day day; + +select date '2012-01-01' - 30 day; + +select date '2012-01-01' - 30 day day; + +select date '2012-01-01' - 30 day day day; + +select date '2012-01-01' + '-30' day; + +select date '2012-01-01' + interval '-30' day; + +-- Unsupported syntax for intervals + +select date '2012-01-01' + interval (-30) day; + +select date '2012-01-01' + (-30) day; + +create temporary view t as select * from values (1), (2) as t(a); + +select date '2012-01-01' + interval (a + 1) day from t; + +select date '2012-01-01' + (a + 1) day from t; + +-- Turns off ANSI mode +SET spark.sql.parser.ansi.enabled=false; diff --git a/sql/core/src/test/resources/sql-tests/inputs/cast.sql b/sql/core/src/test/resources/sql-tests/inputs/cast.sql index 629df59cff8b3..7244cd31bba5e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cast.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cast.sql @@ -40,6 +40,19 @@ SELECT CAST('-9223372036854775809' AS long); SELECT CAST('9223372036854775807' AS long); SELECT CAST('9223372036854775808' AS long); +-- cast string to its binary representation +SELECT HEX(CAST('abc' AS binary)); + +-- cast integral values to their corresponding binary representation +SELECT HEX(CAST(CAST(123 AS byte) AS binary)); +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)); +SELECT HEX(CAST(123S AS binary)); +SELECT HEX(CAST(-123S AS binary)); +SELECT HEX(CAST(123 AS binary)); +SELECT HEX(CAST(-123 AS binary)); +SELECT HEX(CAST(123L AS binary)); +SELECT HEX(CAST(-123L AS binary)); + DESC FUNCTION boolean; DESC FUNCTION EXTENDED boolean; -- TODO: migrate all cast tests here. diff --git a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql index 2909024e4c9f7..6f5ac221ce79c 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/change-column.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/change-column.sql @@ -54,3 +54,4 @@ ALTER TABLE partition_table CHANGE COLUMN c c INT COMMENT 'this is column C'; -- DROP TEST TABLE DROP TABLE test_change; DROP TABLE partition_table; +DROP VIEW global_temp.global_temp_view; diff --git a/sql/core/src/test/resources/sql-tests/inputs/count.sql b/sql/core/src/test/resources/sql-tests/inputs/count.sql new file mode 100644 index 0000000000000..9f9ee4a873d4f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/count.sql @@ -0,0 +1,27 @@ +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData; + +-- distinct count with single expression +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData; + +-- count with multiple expressions +SELECT count(a, b), count(b, a), count(testData.*) FROM testData; + +-- distinct count with multiple expressions +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) +FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql new file mode 100644 index 0000000000000..a1a4bc9de3f97 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/csv-functions.sql @@ -0,0 +1,23 @@ +-- from_csv +select from_csv('1, 3.14', 'a INT, f FLOAT'); +select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select from_csv('1', 1); +select from_csv('1', 'a InvalidType'); +select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')); +select from_csv('1', 'a INT', map('mode', 1)); +select from_csv(); +-- infer schema of json literal +select from_csv('1,abc', schema_of_csv('1,abc')); +select schema_of_csv('1|abc', map('delimiter', '|')); +select schema_of_csv(null); +CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a'); +SELECT schema_of_csv(csvField) FROM csvTable; +-- Clean up +DROP VIEW IF EXISTS csvTable; +-- to_csv +select to_csv(named_struct('a', 1, 'b', 2)); +select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); +-- Check if errors handled +select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')); +select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql new file mode 100644 index 0000000000000..2f2606d44d910 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/cte-legacy.sql @@ -0,0 +1,115 @@ +create temporary view t as select * from values 0, 1, 2 as t(id); +create temporary view t2 as select * from values 0, 1 as t(id); + +-- CTE legacy substitution +SET spark.sql.legacy.ctePrecedence.enabled=true; + +-- CTE in CTE definition +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t; + +-- CTE in subquery +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in subquery expression +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in CTE definition shadows outer +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 2 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 3 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2; + +-- CTE in subquery shadows outer +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery shadows outer 2 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery shadows outer 3 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery expression shadows outer 2 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 3 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +); + +-- Clean up +DROP VIEW IF EXISTS t; +DROP VIEW IF EXISTS t2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/cte.sql b/sql/core/src/test/resources/sql-tests/inputs/cte.sql index d34d89f23575a..d0e145c35a9fe 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/cte.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/cte.sql @@ -4,6 +4,9 @@ create temporary view t2 as select * from values 0, 1 as t(id); -- WITH clause should not fall into infinite loop by referencing self WITH s AS (SELECT 1 FROM s) SELECT * FROM s; +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r; + -- WITH clause should reference the base table WITH t AS (SELECT 1 FROM t) SELECT * FROM t; @@ -24,6 +27,134 @@ SELECT t1.id AS c1, FROM CTE1 t1 CROSS JOIN CTE1 t2; +-- CTE with column alias +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1; + +-- CTE with multiple column aliases +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2; + +-- CTE with duplicate column aliases +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t; + +-- CTE with empty column alias list is not allowed +WITH t() AS (SELECT 1) +SELECT * FROM t; + +-- CTEs with duplicate names are not allowed +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t; + +-- CTE in CTE definition +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t; + +-- CTE in subquery +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in subquery expression +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +); + +-- CTE in CTE definition shadows outer +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 2 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2; + +-- CTE in CTE definition shadows outer 3 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2; + +-- CTE in subquery shadows outer +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery shadows outer 2 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery shadows outer 3 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +); + +-- CTE in subquery expression shadows outer 2 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +); + +-- CTE in subquery expression shadows outer 3 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +); + -- Clean up DROP VIEW IF EXISTS t; DROP VIEW IF EXISTS t2; 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 547c2bef02b24..2f7ffb73e86b8 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/datetime.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/datetime.sql @@ -27,3 +27,10 @@ select current_date = current_date(), current_timestamp = current_timestamp(), a select a, b from ttf2 order by a, current_date; select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), weekday(null), weekday('1582-10-15 13:10:15'); + +select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01'); + +select date '2001-09-28' + 7; +select 7 + date '2001-09-28'; +select date '2001-10-01' - 7; +select date '2001-10-01' - date '2001-09-28'; diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql similarity index 79% rename from sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql rename to sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql index 28a0e20c0f495..35f2be46cd130 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/decimalArithmeticOperations.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/decimalArithmeticOperations.sql @@ -83,4 +83,28 @@ select 12345678912345678912345678912.1234567 + 9999999999999999999999999999999.1 select 123456789123456789.1234567890 * 1.123456789123456789; select 12345678912345.123456789123 / 0.000000012345678; +-- throw an exception instead of returning NULL, according to SQL ANSI 2011 +set spark.sql.decimalOperations.nullOnOverflow=false; + +-- test operations between decimals and constants +select id, a*10, b/10 from decimals_test order by id; + +-- test operations on constants +select 10.3 * 3.0; +select 10.3000 * 3.0; +select 10.30000 * 30.0; +select 10.300000000000000000 * 3.000000000000000000; +select 10.300000000000000000 * 3.0000000000000000000; + +-- arithmetic operations causing an overflow throw exception +select (5e36 + 0.1) + 5e36; +select (-4e36 - 0.1) - 7e36; +select 12345678901234567890.0 * 12345678901234567890.0; +select 1e35 / 0.1; + +-- arithmetic operations causing a precision loss throw exception +select 123456789123456789.1234567890 * 1.123456789123456789; +select 123456789123456789.1234567890 * 1.123456789123456789; +select 12345678912345.123456789123 / 0.000000012345678; + drop table decimals_test; diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe-query.sql b/sql/core/src/test/resources/sql-tests/inputs/describe-query.sql new file mode 100644 index 0000000000000..f8fbb43ba2f8f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/describe-query.sql @@ -0,0 +1,31 @@ +-- Test tables +CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET; +CREATE table desc_temp2 (key int, val string) USING PARQUET; + +-- Simple Describe query +DESC SELECT key, key + 1 as plusone FROM desc_temp1; +DESC QUERY SELECT * FROM desc_temp2; +DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key; +DESC SELECT 10.00D as col1; +DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE); +DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2); +DESC QUERY FROM desc_temp1 a SELECT *; +DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s; +DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s; +DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *); + +-- Error cases. +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1'); +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2; +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * + insert into desc_temp2 select *; + +-- Explain +EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0; +EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s; + +-- cleanup +DROP TABLE desc_temp1; +DROP TABLE desc_temp2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql index f26d5efec076c..a0ee9322372ba 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql @@ -80,6 +80,13 @@ DESC EXTENDED v; -- AnalysisException DESC PARTITION is not allowed on a view DESC v PARTITION (c='Us', d=1); +-- Explain Describe Table +EXPLAIN DESC t; +EXPLAIN DESC EXTENDED t; +EXPLAIN EXTENDED DESC t; +EXPLAIN DESCRIBE t b; +EXPLAIN DESCRIBE t PARTITION (c='Us', d=2); + -- DROP TEST TABLES/VIEWS DROP TABLE t; diff --git a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql index 433db71527437..66bc90914e0d4 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/group-by.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/group-by.sql @@ -80,3 +80,77 @@ SELECT 1 FROM range(10) HAVING true; SELECT 1 FROM range(10) HAVING MAX(id) > 0; SELECT id FROM range(10) HAVING id > 0; + +-- Test data +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v); + +-- empty table +SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0; + +-- all null values +SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4; + +-- aggregates are null Filtering +SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5; + +-- group by +SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k; + +-- having +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false; +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- basic subquery path to make sure rewrite happens in both parent and child plans. +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k; + +-- input type checking Int +SELECT every(1); + +-- input type checking Short +SELECT some(1S); + +-- input type checking Long +SELECT any(1L); + +-- input type checking String +SELECT every("true"); + +-- every/some/any aggregates are supported as windows expression. +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg; + +-- Having referencing aggregate expressions is ok. +SELECT count(*) FROM test_agg HAVING count(*) > 1L; +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true; + +-- Aggrgate expressions can be referenced through an alias +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L; + +-- Error when aggregate expressions are in where clause directly +SELECT count(*) FROM test_agg WHERE count(*) > 1L; +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L; +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1; + diff --git a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql index 41d316444ed6b..b3ec956cd178e 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/inline-table.sql @@ -49,6 +49,3 @@ select * from values ("one", count(1)), ("two", 2) as data(a, b); -- string to timestamp select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); - --- cross-join inline tables -EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null); 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 bdd1fe4074f3c..6c14eee2e4e61 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 @@ -1,6 +1,4 @@ -- to_json -describe function to_json; -describe function extended to_json; select to_json(named_struct('a', 1, 'b', 2)); select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); select to_json(array(named_struct('a', 1, 'b', 2))); @@ -15,8 +13,6 @@ select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)); select to_json(); -- from_json -describe function from_json; -describe function extended from_json; select from_json('{"a":1}', 'a INT'); select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); -- Check if errors handled @@ -59,4 +55,8 @@ select to_json(array(array(1, 2, 3), array(4))); -- infer schema of json literal using options select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')); select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')); - +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; +-- Clean up +DROP VIEW IF EXISTS jsonTable; diff --git a/sql/core/src/test/resources/sql-tests/inputs/operators.sql b/sql/core/src/test/resources/sql-tests/inputs/operators.sql index 37f9cd44da7f2..ba14789d48db6 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/operators.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/operators.sql @@ -29,27 +29,6 @@ select 2 * 5; select 5 % 3; select pmod(-7, 3); --- check operator precedence. --- We follow Oracle operator precedence in the table below that lists the levels of precedence --- among SQL operators from high to low: ------------------------------------------------------------------------------------------- --- Operator Operation ------------------------------------------------------------------------------------------- --- +, - identity, negation --- *, / multiplication, division --- +, -, || addition, subtraction, concatenation --- =, !=, <, >, <=, >=, IS NULL, LIKE, BETWEEN, IN comparison --- NOT exponentiation, logical negation --- AND conjunction --- OR disjunction ------------------------------------------------------------------------------------------- -explain select 'a' || 1 + 2; -explain select 1 - 2 || 'b'; -explain select 2 * 4 + 3 || 'b'; -explain select 3 + 1 || 'a' || 4 / 2; -explain select 1 == 1 OR 'a' || 'b' == 'ab'; -explain select 'a' || 'c' == 'ac' AND 2 == 3; - -- math functions select cot(1); select cot(null); diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql new file mode 100644 index 0000000000000..801a16cf41f54 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part1.sql @@ -0,0 +1,151 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 1] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +SELECT avg(four) AS avg_1 FROM onek; + +SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100; + +-- In 7.1, avg(float4) is computed using float8 arithmetic. +-- Round the result to 3 digits to avoid platform-specific results. + +select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; + +SELECT sum(four) AS sum_1500 FROM onek; +SELECT sum(a) AS sum_198 FROM aggtest; +SELECT sum(b) AS avg_431_773 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; + +SELECT max(four) AS max_3 FROM onek; +SELECT max(a) AS max_100 FROM aggtest; +SELECT max(aggtest.b) AS max_324_78 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT max(student.gpa) AS max_3_7 FROM student; + +SELECT stddev_pop(b) FROM aggtest; +SELECT stddev_samp(b) FROM aggtest; +SELECT var_pop(b) FROM aggtest; +SELECT var_samp(b) FROM aggtest; + +SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest; +SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest; + +-- population variance is defined for a single tuple, sample variance +-- is not +SELECT var_pop(1.0), var_samp(2.0); +SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))); + + +-- verify correct results for null and NaN inputs +select sum(CAST(null AS int)) from range(1,4); +select sum(CAST(null AS long)) from range(1,4); +select sum(CAST(null AS Decimal(38,0))) from range(1,4); +select sum(CAST(null AS DOUBLE)) from range(1,4); +select avg(CAST(null AS int)) from range(1,4); +select avg(CAST(null AS long)) from range(1,4); +select avg(CAST(null AS Decimal(38,0))) from range(1,4); +select avg(CAST(null AS DOUBLE)) from range(1,4); +select sum(CAST('NaN' AS DOUBLE)) from range(1,4); +select avg(CAST('NaN' AS DOUBLE)) from range(1,4); + +-- [SPARK-27768] verify correct results for infinite inputs +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('Infinity' AS DOUBLE))) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x); + + +-- test accuracy with a large input offset +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (7000000000005), (7000000000007)) v(x); + +-- SQL2003 binary aggregates [SPARK-23907] +-- SELECT regr_count(b, a) FROM aggtest; +-- SELECT regr_sxx(b, a) FROM aggtest; +-- SELECT regr_syy(b, a) FROM aggtest; +-- SELECT regr_sxy(b, a) FROM aggtest; +-- SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; +-- SELECT regr_r2(b, a) FROM aggtest; +-- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; +SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest; +SELECT corr(b, a) FROM aggtest; + + +-- test accum and combine functions directly [SPARK-23907] +-- CREATE TABLE regr_test (x float8, y float8); +-- INSERT INTO regr_test VALUES (10,150),(20,250),(30,350),(80,540),(100,200); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30,80); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test; +-- SELECT float8_accum('{4,140,2900}'::float8[], 100); +-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (80,100); +-- SELECT float8_combine('{3,60,200}'::float8[], '{0,0,0}'::float8[]); +-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{0,0,0,0,0,0}'::float8[]); +-- SELECT float8_regr_combine('{0,0,0,0,0,0}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- DROP TABLE regr_test; + + +-- test count, distinct +SELECT count(four) AS cnt_1000 FROM onek; +SELECT count(DISTINCT four) AS cnt_4 FROM onek; + +select ten, count(*), sum(four) from onek +group by ten order by ten; + +select ten, count(four), sum(DISTINCT four) from onek +group by ten order by ten; + +-- user-defined aggregates +-- SELECT newavg(four) AS avg_1 FROM onek; +-- SELECT newsum(four) AS sum_1500 FROM onek; +-- SELECT newcnt(four) AS cnt_1000 FROM onek; +-- SELECT newcnt(*) AS cnt_1000 FROM onek; +-- SELECT oldcnt(*) AS cnt_1000 FROM onek; +-- SELECT sum2(q1,q2) FROM int8_tbl; + +-- test for outer-level aggregates + +-- this should work +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four); + +-- this should fail because subquery has an agg of its own in WHERE +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = b.four); + +-- [SPARK-27769] Test handling of sublinks within outer-level aggregates. +-- Per bug report from Daniel Grace. +select + (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) +from tenk1 o; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql new file mode 100644 index 0000000000000..47f9d2f373069 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/aggregates_part2.sql @@ -0,0 +1,228 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 2] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- Test handling of Params within aggregate arguments in hashed aggregation. +-- Per bug report from Jeevan Chalke. +-- [SPARK-27877] Implement SQL-standard LATERAL subqueries +-- explain (verbose, costs off) +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- explain (verbose, costs off) +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; + +-- [SPARK-27879] Implement bitwise integer aggregates(BIT_AND and BIT_OR) +-- +-- test for bitwise integer aggregates +-- +-- CREATE TEMPORARY TABLE bitwise_test( +-- i2 INT2, +-- i4 INT4, +-- i8 INT8, +-- i INTEGER, +-- x INT2, +-- y BIT(4) +-- ); + +-- empty case +-- SELECT +-- BIT_AND(i2) AS "?", +-- BIT_OR(i4) AS "?" +-- FROM bitwise_test; + +-- COPY bitwise_test FROM STDIN NULL 'null'; +-- 1 1 1 1 1 B0101 +-- 3 3 3 null 2 B0100 +-- 7 7 7 3 4 B1100 +-- \. + +-- SELECT +-- BIT_AND(i2) AS "1", +-- BIT_AND(i4) AS "1", +-- BIT_AND(i8) AS "1", +-- BIT_AND(i) AS "?", +-- BIT_AND(x) AS "0", +-- BIT_AND(y) AS "0100", +-- +-- BIT_OR(i2) AS "7", +-- BIT_OR(i4) AS "7", +-- BIT_OR(i8) AS "7", +-- BIT_OR(i) AS "?", +-- BIT_OR(x) AS "7", +-- BIT_OR(y) AS "1101" +-- FROM bitwise_test; + +-- +-- test boolean aggregates +-- +-- first test all possible transition and final states + +-- The result is inconsistent with PostgreSQL because our AND does not have strict mode +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t`; + +-- The result is inconsistent with PostgreSQL because our OR does not have strict mode +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t`; + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- CREATE TEMPORARY TABLE bool_test( +-- b1 BOOL, +-- b2 BOOL, +-- b3 BOOL, +-- b4 BOOL); + +-- empty case +-- SELECT +-- BOOL_AND(b1) AS "n", +-- BOOL_OR(b3) AS "n" +-- FROM bool_test; + +-- COPY bool_test FROM STDIN NULL 'null'; +-- TRUE null FALSE null +-- FALSE TRUE null null +-- null TRUE FALSE null +-- \. + +-- SELECT +-- BOOL_AND(b1) AS "f", +-- BOOL_AND(b2) AS "t", +-- BOOL_AND(b3) AS "f", +-- BOOL_AND(b4) AS "n", +-- BOOL_AND(NOT b2) AS "f", +-- BOOL_AND(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- EVERY(b1) AS "f", +-- EVERY(b2) AS "t", +-- EVERY(b3) AS "f", +-- EVERY(b4) AS "n", +-- EVERY(NOT b2) AS "f", +-- EVERY(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- BOOL_OR(b1) AS "t", +-- BOOL_OR(b2) AS "t", +-- BOOL_OR(b3) AS "f", +-- BOOL_OR(b4) AS "n", +-- BOOL_OR(NOT b2) AS "f", +-- BOOL_OR(NOT b3) AS "t" +-- FROM bool_test; + +-- +-- Test cases that should be optimized into indexscans instead of +-- the generic aggregate implementation. +-- + +-- Basic cases +-- explain +-- select min(unique1) from tenk1; +select min(unique1) from tenk1; +-- explain +-- select max(unique1) from tenk1; +select max(unique1) from tenk1; +-- explain +-- select max(unique1) from tenk1 where unique1 < 42; +select max(unique1) from tenk1 where unique1 < 42; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42; +select max(unique1) from tenk1 where unique1 > 42; + +-- the planner may choose a generic aggregate here if parallel query is +-- enabled, since that plan will be parallel safe and the "optimized" +-- plan, which has almost identical cost, will not be. we want to test +-- the optimized plan, so temporarily disable parallel query. +-- begin; +-- set local max_parallel_workers_per_gather = 0; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42000; +select max(unique1) from tenk1 where unique1 > 42000; +-- rollback; + +-- multi-column index (uses tenk1_thous_tenthous) +-- explain +-- select max(tenthous) from tenk1 where thousand = 33; +select max(tenthous) from tenk1 where thousand = 33; +-- explain +-- select min(tenthous) from tenk1 where thousand = 33; +select min(tenthous) from tenk1 where thousand = 33; + +-- [SPARK-17348] Correlated column is not allowed in a non-equality predicate +-- check parameter propagation into an indexscan subquery +-- explain +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; + +-- check some cases that were handled incorrectly in 8.3.0 +-- explain +-- select distinct max(unique2) from tenk1; +select distinct max(unique2) from tenk1; +-- explain +-- select max(unique2) from tenk1 order by 1; +select max(unique2) from tenk1 order by 1; +-- explain +-- select max(unique2) from tenk1 order by max(unique2); +select max(unique2) from tenk1 order by max(unique2); +-- explain +-- select max(unique2) from tenk1 order by max(unique2)+1; +select max(unique2) from tenk1 order by max(unique2)+1; +-- explain +-- select max(unique2), generate_series(1,3) as g from tenk1 order by g desc; +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc; + +-- interesting corner case: constant gets optimized into a seqscan +-- explain +-- select max(100) from tenk1; +select max(100) from tenk1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql new file mode 100644 index 0000000000000..fd0d299d7b0ff --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/boolean.sql @@ -0,0 +1,285 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- BOOLEAN +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/boolean.sql + +-- +-- sanity check - if this fails go insane! +-- +SELECT 1 AS one; + + +-- ******************testing built-in type bool******************** + +-- check bool input syntax + +SELECT true AS true; + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT false AS `false`; + +SELECT boolean('t') AS true; + +-- [SPARK-27931] Trim the string when cast string type to boolean type +SELECT boolean(' f ') AS `false`; + +SELECT boolean('true') AS true; + +-- [SPARK-27923] PostgreSQL does not accept 'test' but Spark SQL accepts it and sets it to NULL +SELECT boolean('test') AS error; + +SELECT boolean('false') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'foo' but Spark SQL accepts it and sets it to NULL +SELECT boolean('foo') AS error; + +SELECT boolean('y') AS true; + +SELECT boolean('yes') AS true; + +-- [SPARK-27923] PostgreSQL does not accept 'yeah' but Spark SQL accepts it and sets it to NULL +SELECT boolean('yeah') AS error; + +SELECT boolean('n') AS `false`; + +SELECT boolean('no') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'nay' but Spark SQL accepts it and sets it to NULL +SELECT boolean('nay') AS error; + +-- [SPARK-27931] Accept 'on' and 'off' as input for boolean data type +SELECT boolean('on') AS true; + +SELECT boolean('off') AS `false`; + +-- [SPARK-27931] Accept unique prefixes thereof +SELECT boolean('of') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept 'o' but Spark SQL accepts it and sets it to NULL +SELECT boolean('o') AS error; + +-- [SPARK-27923] PostgreSQL does not accept 'on_' but Spark SQL accepts it and sets it to NULL +SELECT boolean('on_') AS error; + +-- [SPARK-27923] PostgreSQL does not accept 'off_' but Spark SQL accepts it and sets it to NULL +SELECT boolean('off_') AS error; + +SELECT boolean('1') AS true; + +-- [SPARK-27923] PostgreSQL does not accept '11' but Spark SQL accepts it and sets it to NULL +SELECT boolean('11') AS error; + +SELECT boolean('0') AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept '000' but Spark SQL accepts it and sets it to NULL +SELECT boolean('000') AS error; + +-- [SPARK-27923] PostgreSQL does not accept '' but Spark SQL accepts it and sets it to NULL +SELECT boolean('') AS error; + +-- and, or, not in qualifications + +SELECT boolean('t') or boolean('f') AS true; + +SELECT boolean('t') and boolean('f') AS `false`; + +SELECT not boolean('f') AS true; + +SELECT boolean('t') = boolean('f') AS `false`; + +SELECT boolean('t') <> boolean('f') AS true; + +SELECT boolean('t') > boolean('f') AS true; + +SELECT boolean('t') >= boolean('f') AS true; + +SELECT boolean('f') < boolean('t') AS true; + +SELECT boolean('f') <= boolean('t') AS true; + +-- explicit casts to/from text +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false`; +-- [SPARK-27931] Trim the string when cast to boolean type +SELECT boolean(string(' true ')) AS true, + boolean(string(' FALSE')) AS `false`; +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false`; + +-- [SPARK-27923] PostgreSQL does not accept ' tru e ' but Spark SQL accepts it and sets it to NULL +SELECT boolean(string(' tru e ')) AS invalid; -- error +-- [SPARK-27923] PostgreSQL does not accept '' but Spark SQL accepts it and sets it to NULL +SELECT boolean(string('')) AS invalid; -- error + +CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet; + +INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)); + +INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)); + +INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)); + + +-- BOOLTBL1 should be full of true's at this point +SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1; + + +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('true'); + + +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 <> boolean('false'); + +SELECT '' AS zero, BOOLTBL1.* + FROM BOOLTBL1 + WHERE booleq(boolean('false'), f1); + +INSERT INTO BOOLTBL1 VALUES (boolean('f')); + +SELECT '' AS f_1, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('false'); + + +CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet; + +INSERT INTO BOOLTBL2 VALUES (boolean('f')); + +INSERT INTO BOOLTBL2 VALUES (boolean('false')); + +INSERT INTO BOOLTBL2 VALUES (boolean('False')); + +INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')); + +-- [SPARK-27923] PostgreSQL does not accept 'XXX' but Spark SQL accepts it and sets it to NULL +-- This is now an invalid expression +-- For pre-v6.3 this evaluated to false - thomas 1997-10-23 +INSERT INTO BOOLTBL2 + VALUES (boolean('XXX')); + +-- BOOLTBL2 should be full of false's at this point +SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2; + + +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 <> BOOLTBL1.f1; + + +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1); + + +SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false'); + + +SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') + ORDER BY BOOLTBL1.f1, BOOLTBL2.f1; + +-- [SPARK-27924] E061-14: Search Conditions +-- +-- SQL syntax +-- Try all combinations to ensure that we get nothing when we expect nothing +-- - thomas 2000-01-04 +-- + +-- SELECT '' AS True, f1 +-- FROM BOOLTBL1 +-- WHERE f1 IS TRUE; + +-- SELECT '' AS "Not False", f1 +-- FROM BOOLTBL1 +-- WHERE f1 IS NOT FALSE; + +-- SELECT '' AS "False", f1 +-- FROM BOOLTBL1 +-- WHERE f1 IS FALSE; + +-- SELECT '' AS "Not True", f1 +-- FROM BOOLTBL1 +-- WHERE f1 IS NOT TRUE; + +-- SELECT '' AS "True", f1 +-- FROM BOOLTBL2 +-- WHERE f1 IS TRUE; + +-- SELECT '' AS "Not False", f1 +-- FROM BOOLTBL2 +-- WHERE f1 IS NOT FALSE; + +-- SELECT '' AS "False", f1 +-- FROM BOOLTBL2 +-- WHERE f1 IS FALSE; + +-- SELECT '' AS "Not True", f1 +-- FROM BOOLTBL2 +-- WHERE f1 IS NOT TRUE; + +-- +-- Tests for BooleanTest +-- +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet; +INSERT INTO BOOLTBL3 VALUES ('true', true, 1); +INSERT INTO BOOLTBL3 VALUES ('false', false, 2); +INSERT INTO BOOLTBL3 VALUES ('null', null, 3); + +-- [SPARK-27924] E061-14: Search Conditions +-- SELECT +-- d, +-- b IS TRUE AS istrue, +-- b IS NOT TRUE AS isnottrue, +-- b IS FALSE AS isfalse, +-- b IS NOT FALSE AS isnotfalse, +-- b IS UNKNOWN AS isunknown, +-- b IS NOT UNKNOWN AS isnotunknown +-- FROM booltbl3 ORDER BY o; + + +-- Test to make sure short-circuiting and NULL handling is +-- correct. Use a table as source to prevent constant simplification +-- to interfer. +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet; +INSERT INTO booltbl4 VALUES (false, true, null); +-- \pset null '(null)' + +-- AND expression need to return null if there's any nulls and not all +-- of the value are true +SELECT istrue AND isnul AND istrue FROM booltbl4; +SELECT istrue AND istrue AND isnul FROM booltbl4; +SELECT isnul AND istrue AND istrue FROM booltbl4; +SELECT isfalse AND isnul AND istrue FROM booltbl4; +SELECT istrue AND isfalse AND isnul FROM booltbl4; +SELECT isnul AND istrue AND isfalse FROM booltbl4; + +-- OR expression need to return null if there's any nulls and none +-- of the value is true +SELECT isfalse OR isnul OR isfalse FROM booltbl4; +SELECT isfalse OR isfalse OR isnul FROM booltbl4; +SELECT isnul OR isfalse OR isfalse FROM booltbl4; +SELECT isfalse OR isnul OR istrue FROM booltbl4; +SELECT istrue OR isfalse OR isnul FROM booltbl4; +SELECT isnul OR istrue OR isfalse FROM booltbl4; + + +-- +-- Clean up +-- Many tables are retained by the regression test, but these do not seem +-- particularly useful so just get rid of them for now. +-- - thomas 1997-11-30 +-- + +DROP TABLE BOOLTBL1; + +DROP TABLE BOOLTBL2; + +DROP TABLE BOOLTBL3; + +DROP TABLE BOOLTBL4; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql new file mode 100644 index 0000000000000..6d9c44c67a96b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/case.sql @@ -0,0 +1,263 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- CASE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql +-- Test the CASE statement +-- +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet; + +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet; + +INSERT INTO CASE_TBL VALUES (1, 10.1); +INSERT INTO CASE_TBL VALUES (2, 20.2); +INSERT INTO CASE_TBL VALUES (3, -30.3); +INSERT INTO CASE_TBL VALUES (4, NULL); + +INSERT INTO CASE2_TBL VALUES (1, -1); +INSERT INTO CASE2_TBL VALUES (2, -2); +INSERT INTO CASE2_TBL VALUES (3, -3); +INSERT INTO CASE2_TBL VALUES (2, -4); +INSERT INTO CASE2_TBL VALUES (1, NULL); +INSERT INTO CASE2_TBL VALUES (NULL, -6); + +-- +-- Simplest examples without tables +-- + +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + END AS `Simple WHEN`; + +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + END AS `Simple default`; + +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + ELSE 4 + END AS `Simple ELSE`; + +SELECT '4' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default`; + +SELECT '6' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + WHEN 4 < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default`; + +SELECT '7' AS `None`, + CASE WHEN rand() < 0 THEN 1 + END AS `NULL on no matches`; + +-- Constant-expression folding shouldn't evaluate unreachable subexpressions +SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END; + +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- However we do not currently suppress folding of potentially +-- reachable subexpressions +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl; + +-- Test for cases involving untyped literals in test expression +SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END; + +-- +-- Examples of targets involving tables +-- + +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL; + +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +-- +-- Examples of qualifications involving tables +-- + +-- +-- NULLIF() and COALESCE() +-- Shorthand forms for typical CASE constructs +-- defined in the SQL standard. +-- + +SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4; + +SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2; + +SELECT COALESCE(a.f, b.i, b.j) + FROM CASE_TBL a, CASE2_TBL b; + +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(a.f, b.i, b.j) = 2; + +SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b; + +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(f,b.i) = 2; + +-- We don't support update now. +-- +-- Examples of updates involving tables +-- + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 3 THEN (- i) +-- ELSE (2 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 2 THEN (2 * i) +-- ELSE (3 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN b.i >= 2 THEN (2 * j) +-- ELSE (3 * j) END +-- FROM CASE2_TBL b +-- WHERE j = -CASE_TBL.i; + +-- SELECT * FROM CASE_TBL; + +-- +-- Nested CASE expressions +-- + +-- This test exercises a bug caused by aliasing econtext->caseValue_isNull +-- with the isNull argument of the inner CASE's CaseExpr evaluation. After +-- evaluating the vol(null) expression in the inner CASE's second WHEN-clause, +-- the isNull flag for the case test value incorrectly became true, causing +-- the third WHEN-clause not to match. The volatile function calls are needed +-- to prevent constant-folding in the planner, which would hide the bug. + +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE FUNCTION vol(text) returns text as +-- 'begin return $1; end' language plpgsql volatile; + +SELECT CASE + (CASE vol('bar') + WHEN 'foo' THEN 'it was foo!' + WHEN vol(null) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN 'it was foo!' THEN 'foo recognized' + WHEN 'it was bar!' THEN 'bar recognized' + ELSE 'unrecognized' END; + +-- We don't support the features below: +-- 1. CREATE DOMAIN ... +-- 2. CREATE OPERATOR ... +-- 3. CREATE TYPE ... + +-- In this case, we can't inline the SQL function without confusing things. +-- CREATE DOMAIN foodomain AS text; + +-- CREATE FUNCTION volfoo(text) returns foodomain as +-- 'begin return $1::foodomain; end' language plpgsql volatile; + +-- CREATE FUNCTION inline_eq(foodomain, foodomain) returns boolean as +-- 'SELECT CASE $2::text WHEN $1::text THEN true ELSE false END' language sql; + +-- CREATE OPERATOR = (procedure = inline_eq, +-- leftarg = foodomain, rightarg = foodomain); + +-- SELECT CASE volfoo('bar') WHEN 'foo'::foodomain THEN 'is foo' ELSE 'is not foo' END; + +-- ROLLBACK; + +-- Test multiple evaluation of a CASE arg that is a read/write object (#14472) +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE DOMAIN arrdomain AS int[]; + +-- CREATE FUNCTION make_ad(int,int) returns arrdomain as +-- 'declare x arrdomain; +-- begin +-- x := array[$1,$2]; +-- return x; +-- end' language plpgsql volatile; + +-- CREATE FUNCTION ad_eq(arrdomain, arrdomain) returns boolean as +-- 'begin return array_eq($1, $2); end' language plpgsql; + +-- CREATE OPERATOR = (procedure = ad_eq, +-- leftarg = arrdomain, rightarg = arrdomain); + +-- SELECT CASE make_ad(1,2) +-- WHEN array[2,4]::arrdomain THEN 'wrong' +-- WHEN array[2,5]::arrdomain THEN 'still wrong' +-- WHEN array[1,2]::arrdomain THEN 'right' +-- END; + +-- ROLLBACK; + +-- Test interaction of CASE with ArrayCoerceExpr (bug #15471) +-- BEGIN; + +-- CREATE TYPE casetestenum AS ENUM ('e', 'f', 'g'); + +-- SELECT +-- CASE 'foo'::text +-- WHEN 'foo' THEN ARRAY['a', 'b', 'c', 'd'] || enum_range(NULL::casetestenum)::text[] +-- ELSE ARRAY['x', 'y'] +-- END; + +-- ROLLBACK; + +-- +-- Clean up +-- + +DROP TABLE CASE_TBL; +DROP TABLE CASE2_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql new file mode 100644 index 0000000000000..e6943803da399 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/date.sql @@ -0,0 +1,364 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- DATE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/date.sql + +CREATE TABLE DATE_TBL (f1 date) USING parquet; + +INSERT INTO DATE_TBL VALUES ('1957-04-09'); +INSERT INTO DATE_TBL VALUES ('1957-06-13'); +INSERT INTO DATE_TBL VALUES ('1996-02-28'); +INSERT INTO DATE_TBL VALUES ('1996-02-29'); +INSERT INTO DATE_TBL VALUES ('1996-03-01'); +INSERT INTO DATE_TBL VALUES ('1996-03-02'); +INSERT INTO DATE_TBL VALUES ('1997-02-28'); +-- [SPARK-27923] Skip invalid date: 1997-02-29 +-- INSERT INTO DATE_TBL VALUES ('1997-02-29'); +INSERT INTO DATE_TBL VALUES ('1997-03-01'); +INSERT INTO DATE_TBL VALUES ('1997-03-02'); +INSERT INTO DATE_TBL VALUES ('2000-04-01'); +INSERT INTO DATE_TBL VALUES ('2000-04-02'); +INSERT INTO DATE_TBL VALUES ('2000-04-03'); +INSERT INTO DATE_TBL VALUES ('2038-04-08'); +INSERT INTO DATE_TBL VALUES ('2039-04-09'); +INSERT INTO DATE_TBL VALUES ('2040-04-10'); + +SELECT f1 AS `Fifteen` FROM DATE_TBL; + +SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01'; + +SELECT f1 AS `Three` FROM DATE_TBL + WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01'; + +-- Skip the formats that we do not supported. Please check [SPARK-8995] for all supported formats +-- +-- Check all the documented input formats +-- +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- SET datestyle TO iso; -- display results in ISO + +-- SET datestyle TO ymd; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- SET datestyle TO dmy; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +-- SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- SET datestyle TO mdy; + +-- SELECT date 'January 8, 1999'; +SELECT date '1999-01-08'; +SELECT date '1999-01-18'; +-- SELECT date '1/8/1999'; +-- SELECT date '1/18/1999'; +-- SELECT date '18/1/1999'; +-- SELECT date '01/02/03'; +-- SELECT date '19990108'; +-- SELECT date '990108'; +-- SELECT date '1999.008'; +-- SELECT date 'J2451187'; +-- SELECT date 'January 8, 99 BC'; + +-- SELECT date '99-Jan-08'; +-- SELECT date '1999-Jan-08'; +-- SELECT date '08-Jan-99'; +-- SELECT date '08-Jan-1999'; +-- SELECT date 'Jan-08-99'; +-- SELECT date 'Jan-08-1999'; +-- SELECT date '99-08-Jan'; +-- SELECT date '1999-08-Jan'; + +-- SELECT date '99 Jan 08'; +SELECT date '1999 Jan 08'; +-- SELECT date '08 Jan 99'; +-- SELECT date '08 Jan 1999'; +-- SELECT date 'Jan 08 99'; +-- SELECT date 'Jan 08 1999'; +-- SELECT date '99 08 Jan'; +SELECT date '1999 08 Jan'; + +-- SELECT date '99-01-08'; +SELECT date '1999-01-08'; +-- SELECT date '08-01-99'; +-- SELECT date '08-01-1999'; +-- SELECT date '01-08-99'; +-- SELECT date '01-08-1999'; +-- SELECT date '99-08-01'; +SELECT date '1999-08-01'; + +-- SELECT date '99 01 08'; +SELECT date '1999 01 08'; +-- SELECT date '08 01 99'; +-- SELECT date '08 01 1999'; +-- SELECT date '01 08 99'; +-- SELECT date '01 08 1999'; +-- SELECT date '99 08 01'; +SELECT date '1999 08 01'; + +-- [SPARK-28253] Date type have different low value and high value +-- Check upper and lower limits of date range +SELECT date '4714-11-24 BC'; +SELECT date '4714-11-23 BC'; -- out of range +SELECT date '5874897-12-31'; +SELECT date '5874898-01-01'; -- out of range + +-- RESET datestyle; + +-- +-- Simple math +-- Leave most of it for the horology tests +-- + +SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL; + +-- [SPARK-28141] Date type can not accept special values +-- SELECT f1 - date 'epoch' AS "Days From Epoch" FROM DATE_TBL; + +-- SELECT date 'yesterday' - date 'today' AS "One day"; + +-- SELECT date 'today' - date 'tomorrow' AS "One day"; + +-- SELECT date 'yesterday' - date 'tomorrow' AS "Two days"; + +-- SELECT date 'tomorrow' - date 'today' AS "One day"; + +-- SELECT date 'today' - date 'yesterday' AS "One day"; + +-- SELECT date 'tomorrow' - date 'yesterday' AS "Two days"; + +-- [SPARK-28017] Enhance date EXTRACT +-- +-- test extract! +-- +-- epoch +-- +-- SELECT EXTRACT(EPOCH FROM DATE '1970-01-01'); -- 0 +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP '1970-01-01'); -- 0 +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '1970-01-01+00'); -- 0 +-- +-- century +-- +-- SELECT EXTRACT(CENTURY FROM DATE '0101-12-31 BC'); -- -2 +-- SELECT EXTRACT(CENTURY FROM DATE '0100-12-31 BC'); -- -1 +-- SELECT EXTRACT(CENTURY FROM DATE '0001-12-31 BC'); -- -1 +-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01'); -- 1 +-- SELECT EXTRACT(CENTURY FROM DATE '0001-01-01 AD'); -- 1 +-- SELECT EXTRACT(CENTURY FROM DATE '1900-12-31'); -- 19 +-- SELECT EXTRACT(CENTURY FROM DATE '1901-01-01'); -- 20 +-- SELECT EXTRACT(CENTURY FROM DATE '2000-12-31'); -- 20 +-- SELECT EXTRACT(CENTURY FROM DATE '2001-01-01'); -- 21 +-- SELECT EXTRACT(CENTURY FROM CURRENT_DATE)>=21 AS True; -- true +-- +-- millennium +-- +-- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-12-31 BC'); -- -1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '0001-01-01 AD'); -- 1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '1000-12-31'); -- 1 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '1001-01-01'); -- 2 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '2000-12-31'); -- 2 +-- SELECT EXTRACT(MILLENNIUM FROM DATE '2001-01-01'); -- 3 +-- next test to be fixed on the turn of the next millennium;-) +-- SELECT EXTRACT(MILLENNIUM FROM CURRENT_DATE); -- 3 +-- +-- decade +-- +-- SELECT EXTRACT(DECADE FROM DATE '1994-12-25'); -- 199 +-- SELECT EXTRACT(DECADE FROM DATE '0010-01-01'); -- 1 +-- SELECT EXTRACT(DECADE FROM DATE '0009-12-31'); -- 0 +-- SELECT EXTRACT(DECADE FROM DATE '0001-01-01 BC'); -- 0 +-- SELECT EXTRACT(DECADE FROM DATE '0002-12-31 BC'); -- -1 +-- SELECT EXTRACT(DECADE FROM DATE '0011-01-01 BC'); -- -1 +-- SELECT EXTRACT(DECADE FROM DATE '0012-12-31 BC'); -- -2 +-- +-- some other types: +-- +-- on a timestamp. +-- SELECT EXTRACT(CENTURY FROM NOW())>=21 AS True; -- true +-- SELECT EXTRACT(CENTURY FROM TIMESTAMP '1970-03-20 04:30:00.00000'); -- 20 +-- on an interval +-- SELECT EXTRACT(CENTURY FROM INTERVAL '100 y'); -- 1 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '99 y'); -- 0 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '-99 y'); -- 0 +-- SELECT EXTRACT(CENTURY FROM INTERVAL '-100 y'); -- -1 + +-- [SPARK-28017] Enhance date DATE_TRUNC +-- +-- test trunc function! +-- +-- SELECT DATE_TRUNC('MILLENNIUM', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1001 +-- SELECT DATE_TRUNC('MILLENNIUM', DATE '1970-03-20'); -- 1001-01-01 +-- SELECT DATE_TRUNC('CENTURY', TIMESTAMP '1970-03-20 04:30:00.00000'); -- 1901 +-- SELECT DATE_TRUNC('CENTURY', DATE '1970-03-20'); -- 1901 +-- SELECT DATE_TRUNC('CENTURY', DATE '2004-08-10'); -- 2001-01-01 +-- SELECT DATE_TRUNC('CENTURY', DATE '0002-02-04'); -- 0001-01-01 +-- SELECT DATE_TRUNC('CENTURY', DATE '0055-08-10 BC'); -- 0100-01-01 BC +-- SELECT DATE_TRUNC('DECADE', DATE '1993-12-25'); -- 1990-01-01 +-- SELECT DATE_TRUNC('DECADE', DATE '0004-12-25'); -- 0001-01-01 BC +-- SELECT DATE_TRUNC('DECADE', DATE '0002-12-31 BC'); -- 0011-01-01 BC + +-- [SPARK-28141] Date type can not accept special values +-- +-- test infinity +-- +-- select 'infinity'::date, '-infinity'::date; +-- select 'infinity'::date > 'today'::date as t; +-- select '-infinity'::date < 'today'::date as t; +-- select isfinite('infinity'::date), isfinite('-infinity'::date), isfinite('today'::date); +-- +-- oscillating fields from non-finite date/timestamptz: +-- +-- SELECT EXTRACT(HOUR FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM DATE '-infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMP 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMP '-infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMPTZ 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM TIMESTAMPTZ '-infinity'); -- NULL +-- all possible fields +-- SELECT EXTRACT(MICROSECONDS FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MILLISECONDS FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(SECOND FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MINUTE FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(HOUR FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DAY FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(MONTH FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(QUARTER FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(WEEK FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DOW FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(ISODOW FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(DOY FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE_M FROM DATE 'infinity'); -- NULL +-- SELECT EXTRACT(TIMEZONE_H FROM DATE 'infinity'); -- NULL +-- +-- monotonic fields from non-finite date/timestamptz: +-- +-- SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM DATE '-infinity'); -- -Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMP '-infinity'); -- -Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM TIMESTAMPTZ '-infinity'); -- -Infinity +-- all possible fields +-- SELECT EXTRACT(YEAR FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(DECADE FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(CENTURY FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(MILLENNIUM FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(JULIAN FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(ISOYEAR FROM DATE 'infinity'); -- Infinity +-- SELECT EXTRACT(EPOCH FROM DATE 'infinity'); -- Infinity +-- +-- wrong fields from non-finite date: +-- +-- SELECT EXTRACT(MICROSEC FROM DATE 'infinity'); -- ERROR: timestamp units "microsec" not recognized +-- SELECT EXTRACT(UNDEFINED FROM DATE 'infinity'); -- ERROR: timestamp units "undefined" not supported + +-- skip test constructors +-- test constructors +-- select make_date(2013, 7, 15); +-- select make_date(-44, 3, 15); +-- select make_time(8, 20, 0.0); +-- should fail +-- select make_date(2013, 2, 30); +-- select make_date(2013, 13, 1); +-- select make_date(2013, 11, -1); +-- select make_time(10, 55, 100.1); +-- select make_time(24, 0, 2.1); + +DROP TABLE DATE_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql new file mode 100644 index 0000000000000..3dad5cd56ba02 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float4.sql @@ -0,0 +1,363 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT4 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float4.sql + +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet; + +INSERT INTO FLOAT4_TBL VALUES (' 0.0'); +INSERT INTO FLOAT4_TBL VALUES ('1004.30 '); +INSERT INTO FLOAT4_TBL VALUES (' -34.84 '); +INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20'); +INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20'); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over and under flow +-- INSERT INTO FLOAT4_TBL VALUES ('10e70'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e70'); +-- INSERT INTO FLOAT4_TBL VALUES ('10e-70'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e-70'); + +-- INSERT INTO FLOAT4_TBL VALUES ('10e400'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e400'); +-- INSERT INTO FLOAT4_TBL VALUES ('10e-400'); +-- INSERT INTO FLOAT4_TBL VALUES ('-10e-400'); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT4_TBL VALUES (''); +-- INSERT INTO FLOAT4_TBL VALUES (' '); +-- INSERT INTO FLOAT4_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT4_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT4_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT4_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT4_TBL VALUES (' - 3.0'); +-- INSERT INTO FLOAT4_TBL VALUES ('123 5'); + +-- special inputs +SELECT float('NaN'); +-- [SPARK-28060] Float type can not accept some special inputs +SELECT float('nan'); +SELECT float(' NAN '); +SELECT float('infinity'); +SELECT float(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT float('N A N'); +SELECT float('NaN x'); +SELECT float(' INFINITY x'); + +-- [SPARK-28060] Float type can not accept some special inputs +SELECT float('Infinity') + 100.0; +SELECT float('Infinity') / float('Infinity'); +SELECT float('nan') / float('nan'); +SELECT float(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT4_TBL; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-27923] Spark SQL returns NULL +-- test divide by zero +-- SELECT '' AS bad, f.f1 / '0.0' from FLOAT4_TBL f; + +SELECT '' AS five, * FROM FLOAT4_TBL; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- test the unary float4abs operator +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 FROM FLOAT4_TBL f; + +-- Spark SQL does not support update now. +-- UPDATE FLOAT4_TBL +-- SET f1 = FLOAT4_TBL.f1 * '-1' +-- WHERE FLOAT4_TBL.f1 > '0.0'; + +-- SELECT '' AS five, * FROM FLOAT4_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(float('32767.4')); +SELECT smallint(float('32767.6')); +SELECT smallint(float('-32768.4')); +SELECT smallint(float('-32768.6')); +SELECT int(float('2147483520')); +SELECT int(float('2147483647')); +SELECT int(float('-2147483648.5')); +SELECT int(float('-2147483900')); +SELECT bigint(float('9223369837831520256')); +SELECT bigint(float('9223372036854775807')); +SELECT bigint(float('-9223372036854775808.5')); +SELECT bigint(float('-9223380000000000000')); + +-- [SPARK-28061] Support for converting float to binary format +-- Test for correct input rounding in edge cases. +-- These lists are from Paxson 1991, excluding subnormals and +-- inputs of over 9 sig. digits. + +-- SELECT float4send('5e-20'::float4); +-- SELECT float4send('67e14'::float4); +-- SELECT float4send('985e15'::float4); +-- SELECT float4send('55895e-16'::float4); +-- SELECT float4send('7038531e-32'::float4); +-- SELECT float4send('702990899e-20'::float4); + +-- SELECT float4send('3e-23'::float4); +-- SELECT float4send('57e18'::float4); +-- SELECT float4send('789e-35'::float4); +-- SELECT float4send('2539e-18'::float4); +-- SELECT float4send('76173e28'::float4); +-- SELECT float4send('887745e-11'::float4); +-- SELECT float4send('5382571e-37'::float4); +-- SELECT float4send('82381273e-35'::float4); +-- SELECT float4send('750486563e-38'::float4); + +-- Test that the smallest possible normalized input value inputs +-- correctly, either in 9-significant-digit or shortest-decimal +-- format. +-- +-- exact val is 1.1754943508... +-- shortest val is 1.1754944000 +-- midpoint to next val is 1.1754944208... + +-- SELECT float4send('1.17549435e-38'::float4); +-- SELECT float4send('1.1754944e-38'::float4); + +-- We do not support creating types, skip the test below +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat4; +-- create function xfloat4in(cstring) returns xfloat4 immutable strict +-- language internal as 'int4in'; +-- create function xfloat4out(xfloat4) returns cstring immutable strict +-- language internal as 'int4out'; +-- create type xfloat4 (input = xfloat4in, output = xfloat4out, like = float4); +-- create cast (xfloat4 as float4) without function; +-- create cast (float4 as xfloat4) without function; +-- create cast (xfloat4 as integer) without function; +-- create cast (integer as xfloat4) without function; + +-- float4: seeeeeee emmmmmmm mmmmmmmm mmmmmmmm + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'00000001'), +-- (x'00000002'), (x'00000003'), +-- (x'00000010'), (x'00000011'), (x'00000100'), (x'00000101'), +-- (x'00004000'), (x'00004001'), (x'00080000'), (x'00080001'), +-- -- stress values +-- (x'0053c4f4'), -- 7693e-42 +-- (x'006c85c4'), -- 996622e-44 +-- (x'0041ca76'), -- 60419369e-46 +-- (x'004b7678'), -- 6930161142e-48 +-- -- taken from upstream testsuite +-- (x'00000007'), +-- (x'00424fe2'), +-- -- borderline between subnormal and normal +-- (x'007ffff0'), (x'007ffff1'), (x'007ffffe'), (x'007fffff')) +-- select float4send(flt) as ibits, +-- flt +-- from (select bits::integer::xfloat4::float4 as flt +-- from testdata +-- offset 0) s; + +-- with testdata(bits) as (values +-- (x'00000000'), +-- -- smallest normal values +-- (x'00800000'), (x'00800001'), (x'00800004'), (x'00800005'), +-- (x'00800006'), +-- -- small normal values chosen for short vs. long output +-- (x'008002f1'), (x'008002f2'), (x'008002f3'), +-- (x'00800e17'), (x'00800e18'), (x'00800e19'), +-- -- assorted values (random mantissae) +-- (x'01000001'), (x'01102843'), (x'01a52c98'), +-- (x'0219c229'), (x'02e4464d'), (x'037343c1'), (x'03a91b36'), +-- (x'047ada65'), (x'0496fe87'), (x'0550844f'), (x'05999da3'), +-- (x'060ea5e2'), (x'06e63c45'), (x'07f1e548'), (x'0fc5282b'), +-- (x'1f850283'), (x'2874a9d6'), +-- -- values around 5e-08 +-- (x'3356bf94'), (x'3356bf95'), (x'3356bf96'), +-- -- around 1e-07 +-- (x'33d6bf94'), (x'33d6bf95'), (x'33d6bf96'), +-- -- around 3e-07 .. 1e-04 +-- (x'34a10faf'), (x'34a10fb0'), (x'34a10fb1'), +-- (x'350637bc'), (x'350637bd'), (x'350637be'), +-- (x'35719786'), (x'35719787'), (x'35719788'), +-- (x'358637bc'), (x'358637bd'), (x'358637be'), +-- (x'36a7c5ab'), (x'36a7c5ac'), (x'36a7c5ad'), +-- (x'3727c5ab'), (x'3727c5ac'), (x'3727c5ad'), +-- -- format crossover at 1e-04 +-- (x'38d1b714'), (x'38d1b715'), (x'38d1b716'), +-- (x'38d1b717'), (x'38d1b718'), (x'38d1b719'), +-- (x'38d1b71a'), (x'38d1b71b'), (x'38d1b71c'), +-- (x'38d1b71d'), +-- -- +-- (x'38dffffe'), (x'38dfffff'), (x'38e00000'), +-- (x'38efffff'), (x'38f00000'), (x'38f00001'), +-- (x'3a83126e'), (x'3a83126f'), (x'3a831270'), +-- (x'3c23d709'), (x'3c23d70a'), (x'3c23d70b'), +-- (x'3dcccccc'), (x'3dcccccd'), (x'3dccccce'), +-- -- chosen to need 9 digits for 3dcccd70 +-- (x'3dcccd6f'), (x'3dcccd70'), (x'3dcccd71'), +-- -- +-- (x'3effffff'), (x'3f000000'), (x'3f000001'), +-- (x'3f333332'), (x'3f333333'), (x'3f333334'), +-- -- approach 1.0 with increasing numbers of 9s +-- (x'3f666665'), (x'3f666666'), (x'3f666667'), +-- (x'3f7d70a3'), (x'3f7d70a4'), (x'3f7d70a5'), +-- (x'3f7fbe76'), (x'3f7fbe77'), (x'3f7fbe78'), +-- (x'3f7ff971'), (x'3f7ff972'), (x'3f7ff973'), +-- (x'3f7fff57'), (x'3f7fff58'), (x'3f7fff59'), +-- (x'3f7fffee'), (x'3f7fffef'), +-- -- values very close to 1 +-- (x'3f7ffff0'), (x'3f7ffff1'), (x'3f7ffff2'), +-- (x'3f7ffff3'), (x'3f7ffff4'), (x'3f7ffff5'), +-- (x'3f7ffff6'), (x'3f7ffff7'), (x'3f7ffff8'), +-- (x'3f7ffff9'), (x'3f7ffffa'), (x'3f7ffffb'), +-- (x'3f7ffffc'), (x'3f7ffffd'), (x'3f7ffffe'), +-- (x'3f7fffff'), +-- (x'3f800000'), +-- (x'3f800001'), (x'3f800002'), (x'3f800003'), +-- (x'3f800004'), (x'3f800005'), (x'3f800006'), +-- (x'3f800007'), (x'3f800008'), (x'3f800009'), +-- -- values 1 to 1.1 +-- (x'3f80000f'), (x'3f800010'), (x'3f800011'), +-- (x'3f800012'), (x'3f800013'), (x'3f800014'), +-- (x'3f800017'), (x'3f800018'), (x'3f800019'), +-- (x'3f80001a'), (x'3f80001b'), (x'3f80001c'), +-- (x'3f800029'), (x'3f80002a'), (x'3f80002b'), +-- (x'3f800053'), (x'3f800054'), (x'3f800055'), +-- (x'3f800346'), (x'3f800347'), (x'3f800348'), +-- (x'3f8020c4'), (x'3f8020c5'), (x'3f8020c6'), +-- (x'3f8147ad'), (x'3f8147ae'), (x'3f8147af'), +-- (x'3f8ccccc'), (x'3f8ccccd'), (x'3f8cccce'), +-- -- +-- (x'3fc90fdb'), -- pi/2 +-- (x'402df854'), -- e +-- (x'40490fdb'), -- pi +-- -- +-- (x'409fffff'), (x'40a00000'), (x'40a00001'), +-- (x'40afffff'), (x'40b00000'), (x'40b00001'), +-- (x'411fffff'), (x'41200000'), (x'41200001'), +-- (x'42c7ffff'), (x'42c80000'), (x'42c80001'), +-- (x'4479ffff'), (x'447a0000'), (x'447a0001'), +-- (x'461c3fff'), (x'461c4000'), (x'461c4001'), +-- (x'47c34fff'), (x'47c35000'), (x'47c35001'), +-- (x'497423ff'), (x'49742400'), (x'49742401'), +-- (x'4b18967f'), (x'4b189680'), (x'4b189681'), +-- (x'4cbebc1f'), (x'4cbebc20'), (x'4cbebc21'), +-- (x'4e6e6b27'), (x'4e6e6b28'), (x'4e6e6b29'), +-- (x'501502f8'), (x'501502f9'), (x'501502fa'), +-- (x'51ba43b6'), (x'51ba43b7'), (x'51ba43b8'), +-- -- stress values +-- (x'1f6c1e4a'), -- 5e-20 +-- (x'59be6cea'), -- 67e14 +-- (x'5d5ab6c4'), -- 985e15 +-- (x'2cc4a9bd'), -- 55895e-16 +-- (x'15ae43fd'), -- 7038531e-32 +-- (x'2cf757ca'), -- 702990899e-20 +-- (x'665ba998'), -- 25933168707e13 +-- (x'743c3324'), -- 596428896559e20 +-- -- exercise fixed-point memmoves +-- (x'47f1205a'), +-- (x'4640e6ae'), +-- (x'449a5225'), +-- (x'42f6e9d5'), +-- (x'414587dd'), +-- (x'3f9e064b'), +-- -- these cases come from the upstream's testsuite +-- -- BoundaryRoundEven +-- (x'4c000004'), +-- (x'50061c46'), +-- (x'510006a8'), +-- -- ExactValueRoundEven +-- (x'48951f84'), +-- (x'45fd1840'), +-- -- LotsOfTrailingZeros +-- (x'39800000'), +-- (x'3b200000'), +-- (x'3b900000'), +-- (x'3bd00000'), +-- -- Regression +-- (x'63800000'), +-- (x'4b000000'), +-- (x'4b800000'), +-- (x'4c000001'), +-- (x'4c800b0d'), +-- (x'00d24584'), +-- (x'00d90b88'), +-- (x'45803f34'), +-- (x'4f9f24f7'), +-- (x'3a8722c3'), +-- (x'5c800041'), +-- (x'15ae43fd'), +-- (x'5d4cccfb'), +-- (x'4c800001'), +-- (x'57800ed8'), +-- (x'5f000000'), +-- (x'700000f0'), +-- (x'5f23e9ac'), +-- (x'5e9502f9'), +-- (x'5e8012b1'), +-- (x'3c000028'), +-- (x'60cde861'), +-- (x'03aa2a50'), +-- (x'43480000'), +-- (x'4c000000'), +-- -- LooksLikePow5 +-- (x'5D1502F9'), +-- (x'5D9502F9'), +-- (x'5E1502F9'), +-- -- OutputLength +-- (x'3f99999a'), +-- (x'3f9d70a4'), +-- (x'3f9df3b6'), +-- (x'3f9e0419'), +-- (x'3f9e0610'), +-- (x'3f9e064b'), +-- (x'3f9e0651'), +-- (x'03d20cfe') +-- ) +-- select float4send(flt) as ibits, +-- flt, +-- flt::text::float4 as r_flt, +-- float4send(flt::text::float4) as obits, +-- float4send(flt::text::float4) = float4send(flt) as correct +-- from (select bits::integer::xfloat4::float4 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat4 cascade; +DROP TABLE FLOAT4_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql new file mode 100644 index 0000000000000..6f8e3b596e60e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/float8.sql @@ -0,0 +1,500 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- FLOAT8 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/float8.sql + +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet; + +INSERT INTO FLOAT8_TBL VALUES (' 0.0 '); +INSERT INTO FLOAT8_TBL VALUES ('1004.30 '); +INSERT INTO FLOAT8_TBL VALUES (' -34.84'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200'); +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200'); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for underflow and overflow handling +SELECT double('10e400'); +SELECT double('-10e400'); +SELECT double('10e-400'); +SELECT double('-10e-400'); + +-- [SPARK-28061] Support for converting float to binary format +-- test smallest normalized input +-- SELECT float8send('2.2250738585072014E-308'::float8); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad input +-- INSERT INTO FLOAT8_TBL VALUES (''); +-- INSERT INTO FLOAT8_TBL VALUES (' '); +-- INSERT INTO FLOAT8_TBL VALUES ('xyz'); +-- INSERT INTO FLOAT8_TBL VALUES ('5.0.0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5 . 0'); +-- INSERT INTO FLOAT8_TBL VALUES ('5. 0'); +-- INSERT INTO FLOAT8_TBL VALUES (' - 3'); +-- INSERT INTO FLOAT8_TBL VALUES ('123 5'); + +-- special inputs +SELECT double('NaN'); +-- [SPARK-28060] Double type can not accept some special inputs +SELECT double('nan'); +SELECT double(' NAN '); +SELECT double('infinity'); +SELECT double(' -INFINiTY '); +-- [SPARK-27923] Spark SQL insert there bad special inputs to NULL +-- bad special inputs +SELECT double('N A N'); +SELECT double('NaN x'); +SELECT double(' INFINITY x'); + +SELECT double('Infinity') + 100.0; +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +SELECT double('Infinity') / double('Infinity'); +SELECT double('NaN') / double('NaN'); +-- [SPARK-28315] Decimal can not accept NaN as input +SELECT double(decimal('nan')); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3'; + +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3'; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1; + +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3'; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1; + +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3'; + +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS one, f.f1 ^ '2.0' AS square_f1 +-- FROM FLOAT8_TBL f where f.f1 = '1004.3'; + +-- [SPARK-28027] Spark SQL does not support prefix operator @ +-- absolute value +-- SELECT '' AS five, f.f1, @f.f1 AS abs_f1 +-- FROM FLOAT8_TBL f; + +-- [SPARK-23906] Support Truncate number +-- truncate +-- SELECT '' AS five, f.f1, trunc(f.f1) AS trunc_f1 +-- FROM FLOAT8_TBL f; + +-- round +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f; + +-- [SPARK-28135] ceil/ceiling/floor returns incorrect values +-- ceil / ceiling +select ceil(f1) as ceil_f1 from float8_tbl f; +select ceiling(f1) as ceiling_f1 from float8_tbl f; + +-- floor +select floor(f1) as floor_f1 from float8_tbl f; + +-- sign +select sign(f1) as sign_f1 from float8_tbl f; + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- square root +SELECT sqrt(double('64')) AS eight; + +-- [SPARK-28027] Spark SQL does not support prefix operator |/ +-- SELECT |/ float8 '64' AS eight; + +-- SELECT '' AS three, f.f1, |/f.f1 AS sqrt_f1 +-- FROM FLOAT8_TBL f +-- WHERE f.f1 > '0.0'; + +-- power +SELECT power(double('144'), double('0.5')); +SELECT power(double('NaN'), double('0.5')); +SELECT power(double('144'), double('NaN')); +SELECT power(double('NaN'), double('NaN')); +SELECT power(double('-1'), double('NaN')); +-- [SPARK-28135] power returns incorrect values +SELECT power(double('1'), double('NaN')); +SELECT power(double('NaN'), double('0')); + +-- take exp of ln(f.f1) +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0'; + +-- [SPARK-28027] Spark SQL does not support prefix operator ||/ +-- cube root +-- SELECT ||/ float8 '27' AS three; + +-- SELECT '' AS five, f.f1, ||/f.f1 AS cbrt_f1 FROM FLOAT8_TBL f; + + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- UPDATE FLOAT8_TBL +-- SET f1 = FLOAT8_TBL.f1 * '-1' +-- WHERE FLOAT8_TBL.f1 > '0.0'; +-- Update the FLOAT8_TBL to UPDATED_FLOAT8_TBL +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL; + +-- [SPARK-27923] Out of range, Spark SQL returns Infinity +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark/Hive and exponentiation in Postgres +-- SELECT '' AS bad, f.f1 ^ '1e200' from UPDATED_FLOAT8_TBL f; + +-- SELECT 0 ^ 0 + 0 ^ 1 + 0 ^ 0.0 + 0 ^ 0.5; + +-- [SPARK-27923] Cannot take logarithm of zero +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 = '0.0' ; + +-- [SPARK-27923] Cannot take logarithm of a negative number +-- SELECT '' AS bad, ln(f.f1) from UPDATED_FLOAT8_TBL f where f.f1 < '0.0' ; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT '' AS bad, exp(f.f1) from UPDATED_FLOAT8_TBL f; + +-- [SPARK-27923] Divide by zero, Spark SQL returns NULL +-- SELECT '' AS bad, f.f1 / '0.0' from UPDATED_FLOAT8_TBL f; + +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL; + +-- hyperbolic functions +-- we run these with extra_float_digits = 0 too, since different platforms +-- tend to produce results that vary in the last place. +SELECT sinh(double('1')); +SELECT cosh(double('1')); +SELECT tanh(double('1')); +SELECT asinh(double('1')); +SELECT acosh(double('2')); +SELECT atanh(double('0.5')); +-- [SPARK-27768] Infinity, -Infinity, NaN should be recognized in a case insensitive manner +-- test Inf/NaN cases for hyperbolic functions +SELECT sinh(double('Infinity')); +SELECT sinh(double('-Infinity')); +SELECT sinh(double('NaN')); +SELECT cosh(double('Infinity')); +SELECT cosh(double('-Infinity')); +SELECT cosh(double('NaN')); +SELECT tanh(double('Infinity')); +SELECT tanh(double('-Infinity')); +SELECT tanh(double('NaN')); +SELECT asinh(double('Infinity')); +SELECT asinh(double('-Infinity')); +SELECT asinh(double('NaN')); +-- acosh(Inf) should be Inf, but some mingw versions produce NaN, so skip test +SELECT acosh(double('Infinity')); +SELECT acosh(double('-Infinity')); +SELECT acosh(double('NaN')); +SELECT atanh(double('Infinity')); +SELECT atanh(double('-Infinity')); +SELECT atanh(double('NaN')); + +-- RESET extra_float_digits; + +-- [SPARK-28024] Incorrect numeric values when out of range +-- test for over- and underflow +-- INSERT INTO FLOAT8_TBL VALUES ('10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('10e-400'); + +-- INSERT INTO FLOAT8_TBL VALUES ('-10e-400'); + +-- maintain external table consistency across platforms +-- delete all values and reinsert well-behaved ones + +TRUNCATE TABLE FLOAT8_TBL; + +INSERT INTO FLOAT8_TBL VALUES ('0.0'); + +INSERT INTO FLOAT8_TBL VALUES ('-34.84'); + +INSERT INTO FLOAT8_TBL VALUES ('-1004.30'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200'); + +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200'); + +SELECT '' AS five, * FROM FLOAT8_TBL; + +-- [SPARK-28028] Cast numeric to integral type need round +-- [SPARK-28024] Incorrect numeric values when out of range +-- test edge-case coercions to integer +SELECT smallint(double('32767.4')); +SELECT smallint(double('32767.6')); +SELECT smallint(double('-32768.4')); +SELECT smallint(double('-32768.6')); +SELECT int(double('2147483647.4')); +SELECT int(double('2147483647.6')); +SELECT int(double('-2147483648.4')); +SELECT int(double('-2147483648.6')); +SELECT bigint(double('9223372036854773760')); +SELECT bigint(double('9223372036854775807')); +SELECT bigint(double('-9223372036854775808.5')); +SELECT bigint(double('-9223372036854780000')); + +-- [SPARK-28134] Missing Trigonometric Functions +-- test exact cases for trigonometric functions in degrees + +-- SELECT x, +-- sind(x), +-- sind(x) IN (-1,-0.5,0,0.5,1) AS sind_exact +-- FROM (VALUES (0), (30), (90), (150), (180), +-- (210), (270), (330), (360)) AS t(x); + +-- SELECT x, +-- cosd(x), +-- cosd(x) IN (-1,-0.5,0,0.5,1) AS cosd_exact +-- FROM (VALUES (0), (60), (90), (120), (180), +-- (240), (270), (300), (360)) AS t(x); + +-- SELECT x, +-- tand(x), +-- tand(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS tand_exact, +-- cotd(x), +-- cotd(x) IN ('-Infinity'::float8,-1,0, +-- 1,'Infinity'::float8) AS cotd_exact +-- FROM (VALUES (0), (45), (90), (135), (180), +-- (225), (270), (315), (360)) AS t(x); + +-- SELECT x, +-- asind(x), +-- asind(x) IN (-90,-30,0,30,90) AS asind_exact, +-- acosd(x), +-- acosd(x) IN (0,60,90,120,180) AS acosd_exact +-- FROM (VALUES (-1), (-0.5), (0), (0.5), (1)) AS t(x); + +-- SELECT x, +-- atand(x), +-- atand(x) IN (-90,-45,0,45,90) AS atand_exact +-- FROM (VALUES ('-Infinity'::float8), (-1), (0), (1), +-- ('Infinity'::float8)) AS t(x); + +-- SELECT x, y, +-- atan2d(y, x), +-- atan2d(y, x) IN (-90,0,90,180) AS atan2d_exact +-- FROM (SELECT 10*cosd(a), 10*sind(a) +-- FROM generate_series(0, 360, 90) AS t(a)) AS t(x,y); + +-- We do not support creating types, skip the test below +-- +-- test output (and round-trip safety) of various values. +-- To ensure we're testing what we think we're testing, start with +-- float values specified by bit patterns (as a useful side effect, +-- this means we'll fail on non-IEEE platforms). + +-- create type xfloat8; +-- create function xfloat8in(cstring) returns xfloat8 immutable strict +-- language internal as 'int8in'; +-- create function xfloat8out(xfloat8) returns cstring immutable strict +-- language internal as 'int8out'; +-- create type xfloat8 (input = xfloat8in, output = xfloat8out, like = float8); +-- create cast (xfloat8 as float8) without function; +-- create cast (float8 as xfloat8) without function; +-- create cast (xfloat8 as bigint) without function; +-- create cast (bigint as xfloat8) without function; + +-- float8: seeeeeee eeeeeeee eeeeeeee mmmmmmmm mmmmmmmm(x4) + +-- we don't care to assume the platform's strtod() handles subnormals +-- correctly; those are "use at your own risk". However we do test +-- subnormal outputs, since those are under our control. + +-- with testdata(bits) as (values +-- -- small subnormals +-- (x'0000000000000001'), +-- (x'0000000000000002'), (x'0000000000000003'), +-- (x'0000000000001000'), (x'0000000100000000'), +-- (x'0000010000000000'), (x'0000010100000000'), +-- (x'0000400000000000'), (x'0000400100000000'), +-- (x'0000800000000000'), (x'0000800000000001'), +-- -- these values taken from upstream testsuite +-- (x'00000000000f4240'), +-- (x'00000000016e3600'), +-- (x'0000008cdcdea440'), +-- -- borderline between subnormal and normal +-- (x'000ffffffffffff0'), (x'000ffffffffffff1'), +-- (x'000ffffffffffffe'), (x'000fffffffffffff')) +-- select float8send(flt) as ibits, +-- flt +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- round-trip tests + +-- with testdata(bits) as (values +-- (x'0000000000000000'), +-- -- smallest normal values +-- (x'0010000000000000'), (x'0010000000000001'), +-- (x'0010000000000002'), (x'0018000000000000'), +-- -- +-- (x'3ddb7cdfd9d7bdba'), (x'3ddb7cdfd9d7bdbb'), (x'3ddb7cdfd9d7bdbc'), +-- (x'3e112e0be826d694'), (x'3e112e0be826d695'), (x'3e112e0be826d696'), +-- (x'3e45798ee2308c39'), (x'3e45798ee2308c3a'), (x'3e45798ee2308c3b'), +-- (x'3e7ad7f29abcaf47'), (x'3e7ad7f29abcaf48'), (x'3e7ad7f29abcaf49'), +-- (x'3eb0c6f7a0b5ed8c'), (x'3eb0c6f7a0b5ed8d'), (x'3eb0c6f7a0b5ed8e'), +-- (x'3ee4f8b588e368ef'), (x'3ee4f8b588e368f0'), (x'3ee4f8b588e368f1'), +-- (x'3f1a36e2eb1c432c'), (x'3f1a36e2eb1c432d'), (x'3f1a36e2eb1c432e'), +-- (x'3f50624dd2f1a9fb'), (x'3f50624dd2f1a9fc'), (x'3f50624dd2f1a9fd'), +-- (x'3f847ae147ae147a'), (x'3f847ae147ae147b'), (x'3f847ae147ae147c'), +-- (x'3fb9999999999999'), (x'3fb999999999999a'), (x'3fb999999999999b'), +-- -- values very close to 1 +-- (x'3feffffffffffff0'), (x'3feffffffffffff1'), (x'3feffffffffffff2'), +-- (x'3feffffffffffff3'), (x'3feffffffffffff4'), (x'3feffffffffffff5'), +-- (x'3feffffffffffff6'), (x'3feffffffffffff7'), (x'3feffffffffffff8'), +-- (x'3feffffffffffff9'), (x'3feffffffffffffa'), (x'3feffffffffffffb'), +-- (x'3feffffffffffffc'), (x'3feffffffffffffd'), (x'3feffffffffffffe'), +-- (x'3fefffffffffffff'), +-- (x'3ff0000000000000'), +-- (x'3ff0000000000001'), (x'3ff0000000000002'), (x'3ff0000000000003'), +-- (x'3ff0000000000004'), (x'3ff0000000000005'), (x'3ff0000000000006'), +-- (x'3ff0000000000007'), (x'3ff0000000000008'), (x'3ff0000000000009'), +-- -- +-- (x'3ff921fb54442d18'), +-- (x'4005bf0a8b14576a'), +-- (x'400921fb54442d18'), +-- -- +-- (x'4023ffffffffffff'), (x'4024000000000000'), (x'4024000000000001'), +-- (x'4058ffffffffffff'), (x'4059000000000000'), (x'4059000000000001'), +-- (x'408f3fffffffffff'), (x'408f400000000000'), (x'408f400000000001'), +-- (x'40c387ffffffffff'), (x'40c3880000000000'), (x'40c3880000000001'), +-- (x'40f869ffffffffff'), (x'40f86a0000000000'), (x'40f86a0000000001'), +-- (x'412e847fffffffff'), (x'412e848000000000'), (x'412e848000000001'), +-- (x'416312cfffffffff'), (x'416312d000000000'), (x'416312d000000001'), +-- (x'4197d783ffffffff'), (x'4197d78400000000'), (x'4197d78400000001'), +-- (x'41cdcd64ffffffff'), (x'41cdcd6500000000'), (x'41cdcd6500000001'), +-- (x'4202a05f1fffffff'), (x'4202a05f20000000'), (x'4202a05f20000001'), +-- (x'42374876e7ffffff'), (x'42374876e8000000'), (x'42374876e8000001'), +-- (x'426d1a94a1ffffff'), (x'426d1a94a2000000'), (x'426d1a94a2000001'), +-- (x'42a2309ce53fffff'), (x'42a2309ce5400000'), (x'42a2309ce5400001'), +-- (x'42d6bcc41e8fffff'), (x'42d6bcc41e900000'), (x'42d6bcc41e900001'), +-- (x'430c6bf52633ffff'), (x'430c6bf526340000'), (x'430c6bf526340001'), +-- (x'4341c37937e07fff'), (x'4341c37937e08000'), (x'4341c37937e08001'), +-- (x'4376345785d89fff'), (x'4376345785d8a000'), (x'4376345785d8a001'), +-- (x'43abc16d674ec7ff'), (x'43abc16d674ec800'), (x'43abc16d674ec801'), +-- (x'43e158e460913cff'), (x'43e158e460913d00'), (x'43e158e460913d01'), +-- (x'4415af1d78b58c3f'), (x'4415af1d78b58c40'), (x'4415af1d78b58c41'), +-- (x'444b1ae4d6e2ef4f'), (x'444b1ae4d6e2ef50'), (x'444b1ae4d6e2ef51'), +-- (x'4480f0cf064dd591'), (x'4480f0cf064dd592'), (x'4480f0cf064dd593'), +-- (x'44b52d02c7e14af5'), (x'44b52d02c7e14af6'), (x'44b52d02c7e14af7'), +-- (x'44ea784379d99db3'), (x'44ea784379d99db4'), (x'44ea784379d99db5'), +-- (x'45208b2a2c280290'), (x'45208b2a2c280291'), (x'45208b2a2c280292'), +-- -- +-- (x'7feffffffffffffe'), (x'7fefffffffffffff'), +-- -- round to even tests (+ve) +-- (x'4350000000000002'), +-- (x'4350000000002e06'), +-- (x'4352000000000003'), +-- (x'4352000000000004'), +-- (x'4358000000000003'), +-- (x'4358000000000004'), +-- (x'435f000000000020'), +-- -- round to even tests (-ve) +-- (x'c350000000000002'), +-- (x'c350000000002e06'), +-- (x'c352000000000003'), +-- (x'c352000000000004'), +-- (x'c358000000000003'), +-- (x'c358000000000004'), +-- (x'c35f000000000020'), +-- -- exercise fixed-point memmoves +-- (x'42dc12218377de66'), +-- (x'42a674e79c5fe51f'), +-- (x'4271f71fb04cb74c'), +-- (x'423cbe991a145879'), +-- (x'4206fee0e1a9e061'), +-- (x'41d26580b487e6b4'), +-- (x'419d6f34540ca453'), +-- (x'41678c29dcd6e9dc'), +-- (x'4132d687e3df217d'), +-- (x'40fe240c9fcb68c8'), +-- (x'40c81cd6e63c53d3'), +-- (x'40934a4584fd0fdc'), +-- (x'405edd3c07fb4c93'), +-- (x'4028b0fcd32f7076'), +-- (x'3ff3c0ca428c59f8'), +-- -- these cases come from the upstream's testsuite +-- -- LotsOfTrailingZeros) +-- (x'3e60000000000000'), +-- -- Regression +-- (x'c352bd2668e077c4'), +-- (x'434018601510c000'), +-- (x'43d055dc36f24000'), +-- (x'43e052961c6f8000'), +-- (x'3ff3c0ca2a5b1d5d'), +-- -- LooksLikePow5 +-- (x'4830f0cf064dd592'), +-- (x'4840f0cf064dd592'), +-- (x'4850f0cf064dd592'), +-- -- OutputLength +-- (x'3ff3333333333333'), +-- (x'3ff3ae147ae147ae'), +-- (x'3ff3be76c8b43958'), +-- (x'3ff3c083126e978d'), +-- (x'3ff3c0c1fc8f3238'), +-- (x'3ff3c0c9539b8887'), +-- (x'3ff3c0ca2a5b1d5d'), +-- (x'3ff3c0ca4283de1b'), +-- (x'3ff3c0ca43db770a'), +-- (x'3ff3c0ca428abd53'), +-- (x'3ff3c0ca428c1d2b'), +-- (x'3ff3c0ca428c51f2'), +-- (x'3ff3c0ca428c58fc'), +-- (x'3ff3c0ca428c59dd'), +-- (x'3ff3c0ca428c59f8'), +-- (x'3ff3c0ca428c59fb'), +-- -- 32-bit chunking +-- (x'40112e0be8047a7d'), +-- (x'40112e0be815a889'), +-- (x'40112e0be826d695'), +-- (x'40112e0be83804a1'), +-- (x'40112e0be84932ad'), +-- -- MinMaxShift +-- (x'0040000000000000'), +-- (x'007fffffffffffff'), +-- (x'0290000000000000'), +-- (x'029fffffffffffff'), +-- (x'4350000000000000'), +-- (x'435fffffffffffff'), +-- (x'1330000000000000'), +-- (x'133fffffffffffff'), +-- (x'3a6fa7161a4d6e0c') +-- ) +-- select float8send(flt) as ibits, +-- flt, +-- flt::text::float8 as r_flt, +-- float8send(flt::text::float8) as obits, +-- float8send(flt::text::float8) = float8send(flt) as correct +-- from (select bits::bigint::xfloat8::float8 as flt +-- from testdata +-- offset 0) s; + +-- clean up, lest opr_sanity complain +-- drop type xfloat8 cascade; +DROP TABLE FLOAT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql new file mode 100644 index 0000000000000..f64ec5d75afcf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int2.sql @@ -0,0 +1,127 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT2 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int2.sql + +CREATE TABLE INT2_TBL(f1 smallint) USING parquet; + +-- [SPARK-28023] Trim the string when cast string type to other types +INSERT INTO INT2_TBL VALUES (trim('0 ')); + +INSERT INTO INT2_TBL VALUES (trim(' 1234 ')); + +INSERT INTO INT2_TBL VALUES (trim(' -1234')); + +-- [SPARK-27923] Invalid input syntax for type short throws exception at PostgreSQL +-- INSERT INTO INT2_TBL VALUES ('34.5'); + +-- largest and smallest values +INSERT INTO INT2_TBL VALUES ('32767'); + +INSERT INTO INT2_TBL VALUES ('-32767'); + +-- bad input values -- should give errors +-- INSERT INTO INT2_TBL VALUES ('100000'); +-- INSERT INTO INT2_TBL VALUES ('asdf'); +-- INSERT INTO INT2_TBL VALUES (' '); +-- INSERT INTO INT2_TBL VALUES ('- 1234'); +-- INSERT INTO INT2_TBL VALUES ('4 444'); +-- INSERT INTO INT2_TBL VALUES ('123 dt'); +-- INSERT INTO INT2_TBL VALUES (''); + + +SELECT '' AS five, * FROM INT2_TBL; + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0'); + +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0'); + +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0'); + +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0'); + +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0'); + +-- positive odds +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1'); + +-- any evens +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0'); + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384; + +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766; + +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i; + +-- [SPARK-28024] Incorrect value when out of range +-- SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767; + +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i; + +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i; + +-- corner cases +SELECT string(shiftleft(smallint(-1), 15)); +SELECT string(smallint(shiftleft(smallint(-1), 15))+1); + +-- check sane handling of INT16_MIN overflow cases +-- [SPARK-28024] Incorrect numeric values when out of range +-- SELECT smallint((-32768)) * smallint(-1); +-- SELECT smallint(-32768) / smallint(-1); +SELECT smallint(-32768) % smallint(-1); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from numeric +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT2_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql new file mode 100644 index 0000000000000..86432a845b6e1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int4.sql @@ -0,0 +1,178 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT4 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int4.sql +-- + +CREATE TABLE INT4_TBL(f1 int) USING parquet; + +-- [SPARK-28023] Trim the string when cast string type to other types +INSERT INTO INT4_TBL VALUES (trim(' 0 ')); + +INSERT INTO INT4_TBL VALUES (trim('123456 ')); + +INSERT INTO INT4_TBL VALUES (trim(' -123456')); + +-- [SPARK-27923] Invalid input syntax for integer: "34.5" at PostgreSQL +-- INSERT INTO INT4_TBL(f1) VALUES ('34.5'); + +-- largest and smallest values +INSERT INTO INT4_TBL VALUES ('2147483647'); + +INSERT INTO INT4_TBL VALUES ('-2147483647'); + +-- [SPARK-27923] Spark SQL insert these bad inputs to NULL +-- bad input values +-- INSERT INTO INT4_TBL(f1) VALUES ('1000000000000'); +-- INSERT INTO INT4_TBL(f1) VALUES ('asdf'); +-- INSERT INTO INT4_TBL(f1) VALUES (' '); +-- INSERT INTO INT4_TBL(f1) VALUES (' asdf '); +-- INSERT INTO INT4_TBL(f1) VALUES ('- 1234'); +-- INSERT INTO INT4_TBL(f1) VALUES ('123 5'); +-- INSERT INTO INT4_TBL(f1) VALUES (''); + + +SELECT '' AS five, * FROM INT4_TBL; + +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0'); + +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0'); + +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0'); + +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0'); + +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0'); + +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0'); + +-- positive odds +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1'); + +-- any evens +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0'); + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647; + +-- [SPARK-28024] Incorrect value when out of range +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647; + +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i; + +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i; + +-- +-- more complex expressions +-- + +-- variations on unary minus parsing +SELECT -2+3 AS one; + +SELECT 4-2 AS two; + +SELECT 2- -1 AS three; + +SELECT 2 - -2 AS four; + +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true; + +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true; + +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true; + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT int('1000') < int('999') AS `false`; + +-- [SPARK-28027] Our ! and !! has different meanings +-- SELECT 4! AS twenty_four; + +-- SELECT !!3 AS six; + +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten; + +SELECT 2 + 2 / 2 AS three; + +SELECT (2 + 2) / 2 AS two; + +-- [SPARK-28027] Add bitwise shift left/right operators +-- corner case +SELECT string(shiftleft(int(-1), 31)); +SELECT string(int(shiftleft(int(-1), 31))+1); + +-- [SPARK-28024] Incorrect numeric values when out of range +-- check sane handling of INT_MIN overflow cases +-- SELECT (-2147483648)::int4 * (-1)::int4; +-- SELECT (-2147483648)::int4 / (-1)::int4; +SELECT int(-2147483648) % int(-1); +-- SELECT (-2147483648)::int4 * (-1)::int2; +-- SELECT (-2147483648)::int4 / (-1)::int2; +SELECT int(-2147483648) % smallint(-1); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, int(x) AS int4_value +FROM (VALUES double(-2.5), + double(-1.5), + double(-0.5), + double(0.0), + double(0.5), + double(1.5), + double(2.5)) t(x); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from numeric +SELECT x, int(x) AS int4_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT4_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql new file mode 100644 index 0000000000000..d29bf3bfad4ca --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/int8.sql @@ -0,0 +1,239 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- INT8 +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/int8.sql +-- +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet; + +INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')); +INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789'); +INSERT INTO INT8_TBL VALUES('4567890123456789','123'); +INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789'); +INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789'); + +-- [SPARK-27923] Spark SQL insert there bad inputs to NULL +-- bad inputs +-- INSERT INTO INT8_TBL(q1) VALUES (' '); +-- INSERT INTO INT8_TBL(q1) VALUES ('xxx'); +-- INSERT INTO INT8_TBL(q1) VALUES ('3908203590239580293850293850329485'); +-- INSERT INTO INT8_TBL(q1) VALUES ('-1204982019841029840928340329840934'); +-- INSERT INTO INT8_TBL(q1) VALUES ('- 123'); +-- INSERT INTO INT8_TBL(q1) VALUES (' 345 5'); +-- INSERT INTO INT8_TBL(q1) VALUES (''); + +SELECT * FROM INT8_TBL; + +-- int8/int8 cmp +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789; +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789; + +-- int8/int4 cmp +SELECT * FROM INT8_TBL WHERE q2 = 456; +SELECT * FROM INT8_TBL WHERE q2 <> 456; +SELECT * FROM INT8_TBL WHERE q2 < 456; +SELECT * FROM INT8_TBL WHERE q2 > 456; +SELECT * FROM INT8_TBL WHERE q2 <= 456; +SELECT * FROM INT8_TBL WHERE q2 >= 456; + +-- int4/int8 cmp +SELECT * FROM INT8_TBL WHERE 123 = q1; +SELECT * FROM INT8_TBL WHERE 123 <> q1; +SELECT * FROM INT8_TBL WHERE 123 < q1; +SELECT * FROM INT8_TBL WHERE 123 > q1; +SELECT * FROM INT8_TBL WHERE 123 <= q1; +SELECT * FROM INT8_TBL WHERE 123 >= q1; + +-- int8/int2 cmp +SELECT * FROM INT8_TBL WHERE q2 = smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 < smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 > smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456'); +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456'); + +-- int2/int8 cmp +SELECT * FROM INT8_TBL WHERE smallint('123') = q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1; +SELECT * FROM INT8_TBL WHERE smallint('123') < q1; +SELECT * FROM INT8_TBL WHERE smallint('123') > q1; +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1; +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1; + + +-- [SPARK-28349] We do not need to follow PostgreSQL to support reserved words in column alias +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL; + +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL; +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL; +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000); +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL; + +SELECT '' AS five, q1, double(q1) FROM INT8_TBL; +SELECT '' AS five, q2, double(q2) FROM INT8_TBL; + +SELECT 37 + q1 AS plus4 FROM INT8_TBL; +SELECT 37 - q1 AS minus4 FROM INT8_TBL; +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL; +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL; + +-- int8 op int4 +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL; +-- int4 op int8 +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL; + +-- int8 op int2 +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL; +-- int2 op int8 +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL; + +SELECT q2, abs(q2) FROM INT8_TBL; +SELECT min(q1), min(q2) FROM INT8_TBL; +SELECT max(q1), max(q2) FROM INT8_TBL; + +-- [SPARK-28137] Missing Data Type Formatting Functions +-- TO_CHAR() +-- +-- SELECT '' AS to_char_1, to_char(q1, '9G999G999G999G999G999'), to_char(q2, '9,999,999,999,999,999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_2, to_char(q1, '9G999G999G999G999G999D999G999'), to_char(q2, '9,999,999,999,999,999.999,999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_3, to_char( (q1 * -1), '9999999999999999PR'), to_char( (q2 * -1), '9999999999999999.999PR') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_4, to_char( (q1 * -1), '9999999999999999S'), to_char( (q2 * -1), 'S9999999999999999') +-- FROM INT8_TBL; + +-- SELECT '' AS to_char_5, to_char(q2, 'MI9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_6, to_char(q2, 'FMS9999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_7, to_char(q2, 'FM9999999999999999THPR') FROM INT8_TBL; +-- SELECT '' AS to_char_8, to_char(q2, 'SG9999999999999999th') FROM INT8_TBL; +-- SELECT '' AS to_char_9, to_char(q2, '0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_10, to_char(q2, 'S0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_11, to_char(q2, 'FM0999999999999999') FROM INT8_TBL; +-- SELECT '' AS to_char_12, to_char(q2, 'FM9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_13, to_char(q2, 'L9999999999999999.000') FROM INT8_TBL; +-- SELECT '' AS to_char_14, to_char(q2, 'FM9999999999999999.999') FROM INT8_TBL; +-- SELECT '' AS to_char_15, to_char(q2, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9') FROM INT8_TBL; +-- SELECT '' AS to_char_16, to_char(q2, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM INT8_TBL; +-- SELECT '' AS to_char_17, to_char(q2, '999999SG9999999999') FROM INT8_TBL; + +-- [SPARK-26218] Throw exception on overflow for integers +-- check min/max values and overflow behavior + +-- select bigint('-9223372036854775808'); +-- select bigint('-9223372036854775809'); +-- select bigint('9223372036854775807'); +-- select bigint('9223372036854775808'); + +-- select bigint('9223372036854775808'); + +-- select -(bigint('-9223372036854775807')); +-- select -(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + bigint('9223372036854775800'); +-- select bigint('-9223372036854775800') + bigint('-9223372036854775800'); + +-- select bigint('9223372036854775800') - bigint('-9223372036854775800'); +-- select bigint('-9223372036854775800') - bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') * bigint('9223372036854775800'); + +select bigint('9223372036854775800') / bigint('0'); +-- select bigint('9223372036854775800') % bigint('0'); + +-- select abs(bigint('-9223372036854775808')); + +-- select bigint('9223372036854775800') + int('100'); +-- select bigint('-9223372036854775800') - int('100'); +-- select bigint('9223372036854775800') * int('100'); + +-- select int('100') + bigint('9223372036854775800'); +-- select int('-100') - bigint('9223372036854775800'); +-- select int('100') * bigint('9223372036854775800'); + +-- select bigint('9223372036854775800') + smallint('100'); +-- select bigint('-9223372036854775800') - smallint('100'); +-- select bigint('9223372036854775800') * smallint('100'); +select bigint('-9223372036854775808') / smallint('0'); + +-- select smallint('100') + bigint('9223372036854775800'); +-- select smallint('-100') - bigint('9223372036854775800'); +-- select smallint('100') * bigint('9223372036854775800'); +select smallint('100') / bigint('0'); + +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456; +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456; + +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint); + +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL; +SELECT CAST(float('36854775807.0') AS bigint); +SELECT CAST(double('922337203685477580700.0') AS bigint); + + +-- [SPARK-28027] Missing some mathematical operators +-- bit operations + +-- SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, q1 # q2 AS `xor`, ~q1 AS `not` FROM INT8_TBL; +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL; +-- SELECT q1, q1 << 2 AS `shl`, q1 >> 3 AS `shr` FROM INT8_TBL; + + +-- generate_series + +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0); +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2); + +-- corner case +SELECT string(shiftleft(bigint(-1), 63)); +SELECT string(int(shiftleft(bigint(-1), 63))+1); + +-- [SPARK-26218] Throw exception on overflow for integers +-- check sane handling of INT64_MIN overflow cases +SELECT bigint((-9223372036854775808)) * bigint((-1)); +SELECT bigint((-9223372036854775808)) / bigint((-1)); +SELECT bigint((-9223372036854775808)) % bigint((-1)); +SELECT bigint((-9223372036854775808)) * int((-1)); +SELECT bigint((-9223372036854775808)) / int((-1)); +SELECT bigint((-9223372036854775808)) % int((-1)); +SELECT bigint((-9223372036854775808)) * smallint((-1)); +SELECT bigint((-9223372036854775808)) / smallint((-1)); +SELECT bigint((-9223372036854775808)) % smallint((-1)); + +-- [SPARK-28028] Cast numeric to integral type need round +-- check rounding when casting from float +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x); + +-- check rounding when casting from numeric +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x); + +DROP TABLE INT8_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql new file mode 100644 index 0000000000000..0fe47a4d583ef --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/numeric.sql @@ -0,0 +1,1096 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- NUMERIC +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/numeric.sql +-- + +-- [SPARK-28318] Decimal can only support precision up to 38. We rewrite numeric(210,10) to decimal(38,10). +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet; +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet; +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet; + +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet; + + +-- ****************************** +-- * The following EXPECTED results are computed by bc(1) +-- * with a scale of 200 +-- ****************************** + +-- BEGIN TRANSACTION; +INSERT INTO num_exp_add VALUES (0,0,'0'); +INSERT INTO num_exp_sub VALUES (0,0,'0'); +INSERT INTO num_exp_mul VALUES (0,0,'0'); +INSERT INTO num_exp_div VALUES (0,0,'NaN'); +INSERT INTO num_exp_add VALUES (0,1,'0'); +INSERT INTO num_exp_sub VALUES (0,1,'0'); +INSERT INTO num_exp_mul VALUES (0,1,'0'); +INSERT INTO num_exp_div VALUES (0,1,'NaN'); +INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047'); +INSERT INTO num_exp_mul VALUES (0,2,'0'); +INSERT INTO num_exp_div VALUES (0,2,'0'); +INSERT INTO num_exp_add VALUES (0,3,'4.31'); +INSERT INTO num_exp_sub VALUES (0,3,'-4.31'); +INSERT INTO num_exp_mul VALUES (0,3,'0'); +INSERT INTO num_exp_div VALUES (0,3,'0'); +INSERT INTO num_exp_add VALUES (0,4,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119'); +INSERT INTO num_exp_mul VALUES (0,4,'0'); +INSERT INTO num_exp_div VALUES (0,4,'0'); +INSERT INTO num_exp_add VALUES (0,5,'16397.038491'); +INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491'); +INSERT INTO num_exp_mul VALUES (0,5,'0'); +INSERT INTO num_exp_div VALUES (0,5,'0'); +INSERT INTO num_exp_add VALUES (0,6,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026'); +INSERT INTO num_exp_mul VALUES (0,6,'0'); +INSERT INTO num_exp_div VALUES (0,6,'0'); +INSERT INTO num_exp_add VALUES (0,7,'-83028485'); +INSERT INTO num_exp_sub VALUES (0,7,'83028485'); +INSERT INTO num_exp_mul VALUES (0,7,'0'); +INSERT INTO num_exp_div VALUES (0,7,'0'); +INSERT INTO num_exp_add VALUES (0,8,'74881'); +INSERT INTO num_exp_sub VALUES (0,8,'-74881'); +INSERT INTO num_exp_mul VALUES (0,8,'0'); +INSERT INTO num_exp_div VALUES (0,8,'0'); +INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420'); +INSERT INTO num_exp_mul VALUES (0,9,'0'); +INSERT INTO num_exp_div VALUES (0,9,'0'); +INSERT INTO num_exp_add VALUES (1,0,'0'); +INSERT INTO num_exp_sub VALUES (1,0,'0'); +INSERT INTO num_exp_mul VALUES (1,0,'0'); +INSERT INTO num_exp_div VALUES (1,0,'NaN'); +INSERT INTO num_exp_add VALUES (1,1,'0'); +INSERT INTO num_exp_sub VALUES (1,1,'0'); +INSERT INTO num_exp_mul VALUES (1,1,'0'); +INSERT INTO num_exp_div VALUES (1,1,'NaN'); +INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047'); +INSERT INTO num_exp_mul VALUES (1,2,'0'); +INSERT INTO num_exp_div VALUES (1,2,'0'); +INSERT INTO num_exp_add VALUES (1,3,'4.31'); +INSERT INTO num_exp_sub VALUES (1,3,'-4.31'); +INSERT INTO num_exp_mul VALUES (1,3,'0'); +INSERT INTO num_exp_div VALUES (1,3,'0'); +INSERT INTO num_exp_add VALUES (1,4,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119'); +INSERT INTO num_exp_mul VALUES (1,4,'0'); +INSERT INTO num_exp_div VALUES (1,4,'0'); +INSERT INTO num_exp_add VALUES (1,5,'16397.038491'); +INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491'); +INSERT INTO num_exp_mul VALUES (1,5,'0'); +INSERT INTO num_exp_div VALUES (1,5,'0'); +INSERT INTO num_exp_add VALUES (1,6,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026'); +INSERT INTO num_exp_mul VALUES (1,6,'0'); +INSERT INTO num_exp_div VALUES (1,6,'0'); +INSERT INTO num_exp_add VALUES (1,7,'-83028485'); +INSERT INTO num_exp_sub VALUES (1,7,'83028485'); +INSERT INTO num_exp_mul VALUES (1,7,'0'); +INSERT INTO num_exp_div VALUES (1,7,'0'); +INSERT INTO num_exp_add VALUES (1,8,'74881'); +INSERT INTO num_exp_sub VALUES (1,8,'-74881'); +INSERT INTO num_exp_mul VALUES (1,8,'0'); +INSERT INTO num_exp_div VALUES (1,8,'0'); +INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420'); +INSERT INTO num_exp_mul VALUES (1,9,'0'); +INSERT INTO num_exp_div VALUES (1,9,'0'); +INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047'); +INSERT INTO num_exp_mul VALUES (2,0,'0'); +INSERT INTO num_exp_div VALUES (2,0,'NaN'); +INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047'); +INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047'); +INSERT INTO num_exp_mul VALUES (2,1,'0'); +INSERT INTO num_exp_div VALUES (2,1,'NaN'); +INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094'); +INSERT INTO num_exp_sub VALUES (2,2,'0'); +INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209'); +INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047'); +INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047'); +INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257'); +INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266'); +INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047'); +INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047'); +INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593'); +INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685'); +INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047'); +INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047'); +INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077'); +INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429'); +INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787'); +INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307'); +INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222'); +INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940'); +INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047'); +INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953'); +INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795'); +INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518'); +INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047'); +INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047'); +INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407'); +INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476'); +INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467'); +INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627'); +INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740'); +INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811'); +INSERT INTO num_exp_add VALUES (3,0,'4.31'); +INSERT INTO num_exp_sub VALUES (3,0,'4.31'); +INSERT INTO num_exp_mul VALUES (3,0,'0'); +INSERT INTO num_exp_div VALUES (3,0,'NaN'); +INSERT INTO num_exp_add VALUES (3,1,'4.31'); +INSERT INTO num_exp_sub VALUES (3,1,'4.31'); +INSERT INTO num_exp_mul VALUES (3,1,'0'); +INSERT INTO num_exp_div VALUES (3,1,'NaN'); +INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047'); +INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047'); +INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257'); +INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352'); +INSERT INTO num_exp_add VALUES (3,3,'8.62'); +INSERT INTO num_exp_sub VALUES (3,3,'0'); +INSERT INTO num_exp_mul VALUES (3,3,'18.5761'); +INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (3,4,'7799465.7219'); +INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019'); +INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289'); +INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552'); +INSERT INTO num_exp_add VALUES (3,5,'16401.348491'); +INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491'); +INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621'); +INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504'); +INSERT INTO num_exp_add VALUES (3,6,'93905.88763026'); +INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026'); +INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206'); +INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595'); +INSERT INTO num_exp_add VALUES (3,7,'-83028480.69'); +INSERT INTO num_exp_sub VALUES (3,7,'83028489.31'); +INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35'); +INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240'); +INSERT INTO num_exp_add VALUES (3,8,'74885.31'); +INSERT INTO num_exp_sub VALUES (3,8,'-74876.69'); +INSERT INTO num_exp_mul VALUES (3,8,'322737.11'); +INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553'); +INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420'); +INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420'); +INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020'); +INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854'); +INSERT INTO num_exp_add VALUES (4,0,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119'); +INSERT INTO num_exp_mul VALUES (4,0,'0'); +INSERT INTO num_exp_div VALUES (4,0,'NaN'); +INSERT INTO num_exp_add VALUES (4,1,'7799461.4119'); +INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119'); +INSERT INTO num_exp_mul VALUES (4,1,'0'); +INSERT INTO num_exp_div VALUES (4,1,'NaN'); +INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047'); +INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047'); +INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593'); +INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385'); +INSERT INTO num_exp_add VALUES (4,3,'7799465.7219'); +INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019'); +INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289'); +INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883'); +INSERT INTO num_exp_add VALUES (4,4,'15598922.8238'); +INSERT INTO num_exp_sub VALUES (4,4,'0'); +INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161'); +INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (4,5,'7815858.450391'); +INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409'); +INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429'); +INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061'); +INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026'); +INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974'); +INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094'); +INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606'); +INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881'); +INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119'); +INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715'); +INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637'); +INSERT INTO num_exp_add VALUES (4,8,'7874342.4119'); +INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119'); +INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839'); +INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143'); +INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420'); +INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420'); +INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980'); +INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409'); +INSERT INTO num_exp_add VALUES (5,0,'16397.038491'); +INSERT INTO num_exp_sub VALUES (5,0,'16397.038491'); +INSERT INTO num_exp_mul VALUES (5,0,'0'); +INSERT INTO num_exp_div VALUES (5,0,'NaN'); +INSERT INTO num_exp_add VALUES (5,1,'16397.038491'); +INSERT INTO num_exp_sub VALUES (5,1,'16397.038491'); +INSERT INTO num_exp_mul VALUES (5,1,'0'); +INSERT INTO num_exp_div VALUES (5,1,'NaN'); +INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047'); +INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047'); +INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077'); +INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446'); +INSERT INTO num_exp_add VALUES (5,3,'16401.348491'); +INSERT INTO num_exp_sub VALUES (5,3,'16392.728491'); +INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621'); +INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584'); +INSERT INTO num_exp_add VALUES (5,4,'7815858.450391'); +INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409'); +INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429'); +INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192'); +INSERT INTO num_exp_add VALUES (5,5,'32794.076982'); +INSERT INTO num_exp_sub VALUES (5,5,'0'); +INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081'); +INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (5,6,'110298.61612126'); +INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926'); +INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766'); +INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689'); +INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509'); +INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491'); +INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135'); +INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710'); +INSERT INTO num_exp_add VALUES (5,8,'91278.038491'); +INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509'); +INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571'); +INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228'); +INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420'); +INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420'); +INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220'); +INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427'); +INSERT INTO num_exp_add VALUES (6,0,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026'); +INSERT INTO num_exp_mul VALUES (6,0,'0'); +INSERT INTO num_exp_div VALUES (6,0,'NaN'); +INSERT INTO num_exp_add VALUES (6,1,'93901.57763026'); +INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026'); +INSERT INTO num_exp_mul VALUES (6,1,'0'); +INSERT INTO num_exp_div VALUES (6,1,'NaN'); +INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787'); +INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307'); +INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222'); +INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823'); +INSERT INTO num_exp_add VALUES (6,3,'93905.88763026'); +INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026'); +INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206'); +INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907'); +INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026'); +INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974'); +INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094'); +INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469'); +INSERT INTO num_exp_add VALUES (6,5,'110298.61612126'); +INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926'); +INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766'); +INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679'); +INSERT INTO num_exp_add VALUES (6,6,'187803.15526052'); +INSERT INTO num_exp_sub VALUES (6,6,'0'); +INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676'); +INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974'); +INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026'); +INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610'); +INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980'); +INSERT INTO num_exp_add VALUES (6,8,'168782.57763026'); +INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026'); +INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906'); +INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184'); +INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160'); +INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680'); +INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920'); +INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789'); +INSERT INTO num_exp_add VALUES (7,0,'-83028485'); +INSERT INTO num_exp_sub VALUES (7,0,'-83028485'); +INSERT INTO num_exp_mul VALUES (7,0,'0'); +INSERT INTO num_exp_div VALUES (7,0,'NaN'); +INSERT INTO num_exp_add VALUES (7,1,'-83028485'); +INSERT INTO num_exp_sub VALUES (7,1,'-83028485'); +INSERT INTO num_exp_mul VALUES (7,1,'0'); +INSERT INTO num_exp_div VALUES (7,1,'NaN'); +INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047'); +INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953'); +INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795'); +INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700'); +INSERT INTO num_exp_add VALUES (7,3,'-83028480.69'); +INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31'); +INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35'); +INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974'); +INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881'); +INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119'); +INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715'); +INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686'); +INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509'); +INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491'); +INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135'); +INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574'); +INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974'); +INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026'); +INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610'); +INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294'); +INSERT INTO num_exp_add VALUES (7,7,'-166056970'); +INSERT INTO num_exp_sub VALUES (7,7,'0'); +INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225'); +INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (7,8,'-82953604'); +INSERT INTO num_exp_sub VALUES (7,8,'-83103366'); +INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285'); +INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118'); +INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420'); +INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580'); +INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700'); +INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382'); +INSERT INTO num_exp_add VALUES (8,0,'74881'); +INSERT INTO num_exp_sub VALUES (8,0,'74881'); +INSERT INTO num_exp_mul VALUES (8,0,'0'); +INSERT INTO num_exp_div VALUES (8,0,'NaN'); +INSERT INTO num_exp_add VALUES (8,1,'74881'); +INSERT INTO num_exp_sub VALUES (8,1,'74881'); +INSERT INTO num_exp_mul VALUES (8,1,'0'); +INSERT INTO num_exp_div VALUES (8,1,'NaN'); +INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047'); +INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047'); +INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407'); +INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615'); +INSERT INTO num_exp_add VALUES (8,3,'74885.31'); +INSERT INTO num_exp_sub VALUES (8,3,'74876.69'); +INSERT INTO num_exp_mul VALUES (8,3,'322737.11'); +INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410'); +INSERT INTO num_exp_add VALUES (8,4,'7874342.4119'); +INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119'); +INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839'); +INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956'); +INSERT INTO num_exp_add VALUES (8,5,'91278.038491'); +INSERT INTO num_exp_sub VALUES (8,5,'58483.961509'); +INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571'); +INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456'); +INSERT INTO num_exp_add VALUES (8,6,'168782.57763026'); +INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026'); +INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906'); +INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424'); +INSERT INTO num_exp_add VALUES (8,7,'-82953604'); +INSERT INTO num_exp_sub VALUES (8,7,'83103366'); +INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285'); +INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172'); +INSERT INTO num_exp_add VALUES (8,8,'149762'); +INSERT INTO num_exp_sub VALUES (8,8,'0'); +INSERT INTO num_exp_mul VALUES (8,8,'5607164161'); +INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000'); +INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420'); +INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420'); +INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020'); +INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735'); +INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420'); +INSERT INTO num_exp_mul VALUES (9,0,'0'); +INSERT INTO num_exp_div VALUES (9,0,'NaN'); +INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420'); +INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420'); +INSERT INTO num_exp_mul VALUES (9,1,'0'); +INSERT INTO num_exp_div VALUES (9,1,'NaN'); +INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467'); +INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627'); +INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740'); +INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526'); +INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420'); +INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420'); +INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020'); +INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677'); +INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420'); +INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420'); +INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980'); +INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484'); +INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420'); +INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420'); +INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220'); +INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807'); +INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160'); +INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680'); +INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920'); +INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280'); +INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420'); +INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580'); +INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700'); +INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689'); +INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420'); +INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420'); +INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020'); +INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748'); +INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840'); +INSERT INTO num_exp_sub VALUES (9,9,'0'); +INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400'); +INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_sqrt VALUES (0,'0'); +INSERT INTO num_exp_sqrt VALUES (1,'0'); +INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505'); +INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396'); +INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923'); +INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473'); +INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406'); +INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230'); +INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542'); +INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_ln VALUES (0,'NaN'); +INSERT INTO num_exp_ln VALUES (1,'NaN'); +INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514'); +INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971'); +INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464'); +INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038'); +INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127'); +INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991'); +INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668'); +INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_log10 VALUES (0,'NaN'); +INSERT INTO num_exp_log10 VALUES (1,'NaN'); +INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459'); +INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075'); +INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345'); +INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626'); +INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671'); +INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914'); +INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138'); +INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN'); +INSERT INTO num_exp_power_10_ln VALUES (1,'NaN'); +INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184'); +INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393'); +INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636'); +INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098'); +INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067'); +INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627'); +INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952'); +INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457'); +-- COMMIT TRANSACTION; +-- BEGIN TRANSACTION; +INSERT INTO num_data VALUES (0, '0'); +INSERT INTO num_data VALUES (1, '0'); +INSERT INTO num_data VALUES (2, '-34338492.215397047'); +INSERT INTO num_data VALUES (3, '4.31'); +INSERT INTO num_data VALUES (4, '7799461.4119'); +INSERT INTO num_data VALUES (5, '16397.038491'); +INSERT INTO num_data VALUES (6, '93901.57763026'); +INSERT INTO num_data VALUES (7, '-83028485'); +INSERT INTO num_data VALUES (8, '74881'); +INSERT INTO num_data VALUES (9, '-24926804.045047420'); +-- COMMIT TRANSACTION; + +SELECT * FROM num_data; + +-- ****************************** +-- * Create indices for faster checks +-- ****************************** + +-- CREATE UNIQUE INDEX num_exp_add_idx ON num_exp_add (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sub_idx ON num_exp_sub (id1, id2); +-- CREATE UNIQUE INDEX num_exp_div_idx ON num_exp_div (id1, id2); +-- CREATE UNIQUE INDEX num_exp_mul_idx ON num_exp_mul (id1, id2); +-- CREATE UNIQUE INDEX num_exp_sqrt_idx ON num_exp_sqrt (id); +-- CREATE UNIQUE INDEX num_exp_ln_idx ON num_exp_ln (id); +-- CREATE UNIQUE INDEX num_exp_log10_idx ON num_exp_log10 (id); +-- CREATE UNIQUE INDEX num_exp_power_10_ln_idx ON num_exp_power_10_ln (id); + +-- VACUUM ANALYZE num_exp_add; +-- VACUUM ANALYZE num_exp_sub; +-- VACUUM ANALYZE num_exp_div; +-- VACUUM ANALYZE num_exp_mul; +-- VACUUM ANALYZE num_exp_sqrt; +-- VACUUM ANALYZE num_exp_ln; +-- VACUUM ANALYZE num_exp_log10; +-- VACUUM ANALYZE num_exp_power_10_ln; + +-- ****************************** +-- * Now check the behaviour of the NUMERIC type +-- ****************************** + +-- ****************************** +-- * Addition check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10); + +-- ****************************** +-- * Subtraction check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40); + +-- ****************************** +-- * Multiply check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30); + +-- ****************************** +-- * Division check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected; + +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0'; +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80); + +-- ****************************** +-- * Square root check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Natural logarithm check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * Logarithm base 10 check +-- ****************************** +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * POWER(10, LN(value)) check +-- ****************************** +-- [SPARK-28316] Decimal precision issue +TRUNCATE TABLE num_result; +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0'; +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected; + +-- ****************************** +-- * miscellaneous checks for things that have been broken in the past... +-- ****************************** +-- numeric AVG used to fail on some platforms +SELECT AVG(val) FROM num_data; +-- [SPARK-28316] STDDEV and VARIANCE returns double type +-- Skip it because: Expected "2.779120328758835[]E7", but got "2.779120328758835[4]E7" +-- SELECT STDDEV(val) FROM num_data; +-- Skip it because: Expected "7.72350980172061[8]E14", but got "7.72350980172061[6]E14" +-- SELECT VARIANCE(val) FROM num_data; + +-- Check for appropriate rounding and overflow +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet; +INSERT INTO fract_only VALUES (1, '0.0'); +INSERT INTO fract_only VALUES (2, '0.1'); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (3, '1.0'); -- should fail +INSERT INTO fract_only VALUES (4, '-0.9999'); +INSERT INTO fract_only VALUES (5, '0.99994'); +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- INSERT INTO fract_only VALUES (6, '0.99995'); -- should fail +INSERT INTO fract_only VALUES (7, '0.00001'); +INSERT INTO fract_only VALUES (8, '0.00017'); +SELECT * FROM fract_only; +DROP TABLE fract_only; + +-- [SPARK-28315] Decimal can not accept NaN as input +-- [SPARK-27923] Decimal type can not accept Infinity and -Infinity +-- Check inf/nan conversion behavior +SELECT decimal(double('NaN')); +SELECT decimal(double('Infinity')); +SELECT decimal(double('-Infinity')); +SELECT decimal(float('NaN')); +SELECT decimal(float('Infinity')); +SELECT decimal(float('-Infinity')); + +-- Simple check that ceil(), floor(), and round() work correctly +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet; +INSERT INTO ceil_floor_round VALUES ('-5.5'); +INSERT INTO ceil_floor_round VALUES ('-5.499999'); +INSERT INTO ceil_floor_round VALUES ('9.5'); +INSERT INTO ceil_floor_round VALUES ('9.4999999'); +INSERT INTO ceil_floor_round VALUES ('0.0'); +INSERT INTO ceil_floor_round VALUES ('0.0000001'); +INSERT INTO ceil_floor_round VALUES ('-0.000001'); +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round; +DROP TABLE ceil_floor_round; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Check rounding, it should round ties away from zero. +-- SELECT i as pow, +-- round((-2.5 * 10 ^ i)::numeric, -i), +-- round((-1.5 * 10 ^ i)::numeric, -i), +-- round((-0.5 * 10 ^ i)::numeric, -i), +-- round((0.5 * 10 ^ i)::numeric, -i), +-- round((1.5 * 10 ^ i)::numeric, -i), +-- round((2.5 * 10 ^ i)::numeric, -i) +-- FROM generate_series(-5,5) AS t(i); + +-- [SPARK-21117] Built-in SQL Function Support - WIDTH_BUCKET +-- Testing for width_bucket(). For convenience, we test both the +-- numeric and float8 versions of the function in this file. + +-- errors +-- SELECT width_bucket(5.0, 3.0, 4.0, 0); +-- SELECT width_bucket(5.0, 3.0, 4.0, -5); +-- SELECT width_bucket(3.5, 3.0, 3.0, 888); +-- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, 0); +-- SELECT width_bucket(5.0::float8, 3.0::float8, 4.0::float8, -5); +-- SELECT width_bucket(3.5::float8, 3.0::float8, 3.0::float8, 888); +-- SELECT width_bucket('NaN', 3.0, 4.0, 888); +-- SELECT width_bucket(0::float8, 'NaN', 4.0::float8, 888); + +-- normal operation +-- CREATE TABLE width_bucket_test (operand_num numeric, operand_f8 float8); + +-- COPY width_bucket_test (operand_num) FROM stdin; +-- -5.2 +-- -0.0000000001 +-- 0.000000000001 +-- 1 +-- 1.99999999999999 +-- 2 +-- 2.00000000000001 +-- 3 +-- 4 +-- 4.5 +-- 5 +-- 5.5 +-- 6 +-- 7 +-- 8 +-- 9 +-- 9.99999999999999 +-- 10 +-- 10.0000000000001 +-- \. + +-- UPDATE width_bucket_test SET operand_f8 = operand_num::float8; + +-- SELECT +-- operand_num, +-- width_bucket(operand_num, 0, 10, 5) AS wb_1, +-- width_bucket(operand_f8, 0, 10, 5) AS wb_1f, +-- width_bucket(operand_num, 10, 0, 5) AS wb_2, +-- width_bucket(operand_f8, 10, 0, 5) AS wb_2f, +-- width_bucket(operand_num, 2, 8, 4) AS wb_3, +-- width_bucket(operand_f8, 2, 8, 4) AS wb_3f, +-- width_bucket(operand_num, 5.0, 5.5, 20) AS wb_4, +-- width_bucket(operand_f8, 5.0, 5.5, 20) AS wb_4f, +-- width_bucket(operand_num, -25, 25, 10) AS wb_5, +-- width_bucket(operand_f8, -25, 25, 10) AS wb_5f +-- FROM width_bucket_test; + +-- for float8 only, check positive and negative infinity: we require +-- finite bucket bounds, but allow an infinite operand +-- SELECT width_bucket(0.0::float8, 'Infinity'::float8, 5, 10); -- error +-- SELECT width_bucket(0.0::float8, 5, '-Infinity'::float8, 20); -- error +-- SELECT width_bucket('Infinity'::float8, 1, 10, 10), +-- width_bucket('-Infinity'::float8, 1, 10, 10); + +-- DROP TABLE width_bucket_test; + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_CHAR +-- TO_CHAR() +-- +-- SELECT '' AS to_char_1, to_char(val, '9G999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_2, to_char(val, '9G999G999G999G999G999D999G999G999G999G999') +-- FROM num_data; + +-- SELECT '' AS to_char_3, to_char(val, '9999999999999999.999999999999999PR') +-- FROM num_data; + +-- SELECT '' AS to_char_4, to_char(val, '9999999999999999.999999999999999S') +-- FROM num_data; + +-- SELECT '' AS to_char_5, to_char(val, 'MI9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_6, to_char(val, 'FMS9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_7, to_char(val, 'FM9999999999999999.999999999999999THPR') FROM num_data; +-- SELECT '' AS to_char_8, to_char(val, 'SG9999999999999999.999999999999999th') FROM num_data; +-- SELECT '' AS to_char_9, to_char(val, '0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_10, to_char(val, 'S0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_11, to_char(val, 'FM0999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_12, to_char(val, 'FM9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_13, to_char(val, 'FM9999999999990999.990999999999999') FROM num_data; +-- SELECT '' AS to_char_14, to_char(val, 'FM0999999999999999.999909999999999') FROM num_data; +-- SELECT '' AS to_char_15, to_char(val, 'FM9999999990999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_16, to_char(val, 'L9999999999999999.099999999999999') FROM num_data; +-- SELECT '' AS to_char_17, to_char(val, 'FM9999999999999999.99999999999999') FROM num_data; +-- SELECT '' AS to_char_18, to_char(val, 'S 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data; +-- SELECT '' AS to_char_19, to_char(val, 'FMS 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 . 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9 9') FROM num_data; +-- SELECT '' AS to_char_20, to_char(val, E'99999 "text" 9999 "9999" 999 "\\"text between quote marks\\"" 9999') FROM num_data; +-- SELECT '' AS to_char_21, to_char(val, '999999SG9999999999') FROM num_data; +-- SELECT '' AS to_char_22, to_char(val, 'FM9999999999999999.999999999999999') FROM num_data; +-- SELECT '' AS to_char_23, to_char(val, '9.999EEEE') FROM num_data; + +-- SELECT '' AS to_char_24, to_char('100'::numeric, 'FM999.9'); +-- SELECT '' AS to_char_25, to_char('100'::numeric, 'FM999.'); +-- SELECT '' AS to_char_26, to_char('100'::numeric, 'FM999'); + +-- Check parsing of literal text in a format string +-- SELECT '' AS to_char_27, to_char('100'::numeric, 'foo999'); +-- SELECT '' AS to_char_28, to_char('100'::numeric, 'f\oo999'); +-- SELECT '' AS to_char_29, to_char('100'::numeric, 'f\\oo999'); +-- SELECT '' AS to_char_30, to_char('100'::numeric, 'f\"oo999'); +-- SELECT '' AS to_char_31, to_char('100'::numeric, 'f\\"oo999'); +-- SELECT '' AS to_char_32, to_char('100'::numeric, 'f"ool"999'); +-- SELECT '' AS to_char_33, to_char('100'::numeric, 'f"\ool"999'); +-- SELECT '' AS to_char_34, to_char('100'::numeric, 'f"\\ool"999'); +-- SELECT '' AS to_char_35, to_char('100'::numeric, 'f"ool\"999'); +-- SELECT '' AS to_char_36, to_char('100'::numeric, 'f"ool\\"999'); + +-- [SPARK-28137] Missing Data Type Formatting Functions: TO_NUMBER +-- TO_NUMBER() +-- +-- SET lc_numeric = 'C'; +-- SELECT '' AS to_number_1, to_number('-34,338,492', '99G999G999'); +-- SELECT '' AS to_number_2, to_number('-34,338,492.654,878', '99G999G999D999G999'); +-- SELECT '' AS to_number_3, to_number('<564646.654564>', '999999.999999PR'); +-- SELECT '' AS to_number_4, to_number('0.00001-', '9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999S'); +-- SELECT '' AS to_number_5, to_number('5.01-', 'FM9.999999MI'); +-- SELECT '' AS to_number_7, to_number('5 4 4 4 4 8 . 7 8', '9 9 9 9 9 9 . 9 9'); +-- SELECT '' AS to_number_8, to_number('.01', 'FM9.99'); +-- SELECT '' AS to_number_9, to_number('.0', '99999999.99999999'); +-- SELECT '' AS to_number_10, to_number('0', '99.99'); +-- SELECT '' AS to_number_11, to_number('.-01', 'S99.99'); +-- SELECT '' AS to_number_12, to_number('.01-', '99.99S'); +-- SELECT '' AS to_number_13, to_number(' . 0 1-', ' 9 9 . 9 9 S'); +-- SELECT '' AS to_number_14, to_number('34,50','999,99'); +-- SELECT '' AS to_number_15, to_number('123,000','999G'); +-- SELECT '' AS to_number_16, to_number('123456','999G999'); +-- SELECT '' AS to_number_17, to_number('$1234.56','L9,999.99'); +-- SELECT '' AS to_number_18, to_number('$1234.56','L99,999.99'); +-- SELECT '' AS to_number_19, to_number('$1,234.56','L99,999.99'); +-- SELECT '' AS to_number_20, to_number('1234.56','L99,999.99'); +-- SELECT '' AS to_number_21, to_number('1,234.56','L99,999.99'); +-- SELECT '' AS to_number_22, to_number('42nd', '99th'); +-- RESET lc_numeric; + +-- +-- Input syntax +-- + +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet; + +-- good inputs +INSERT INTO num_input_test VALUES (trim(' 123')); +INSERT INTO num_input_test VALUES (trim(' 3245874 ')); +INSERT INTO num_input_test VALUES (trim(' -93853')); +INSERT INTO num_input_test VALUES ('555.50'); +INSERT INTO num_input_test VALUES ('-555.50'); +-- [SPARK-28315] Decimal can not accept NaN as input +-- INSERT INTO num_input_test VALUES (trim('NaN ')); +-- INSERT INTO num_input_test VALUES (trim(' nan')); + +-- [SPARK-27923] Spark SQL accept bad inputs to NULL +-- bad inputs +-- INSERT INTO num_input_test VALUES (' '); +-- INSERT INTO num_input_test VALUES (' 1234 %'); +-- INSERT INTO num_input_test VALUES ('xyz'); +-- INSERT INTO num_input_test VALUES ('- 1234'); +-- INSERT INTO num_input_test VALUES ('5 . 0'); +-- INSERT INTO num_input_test VALUES ('5. 0 '); +-- INSERT INTO num_input_test VALUES (''); +-- INSERT INTO num_input_test VALUES (' N aN '); + +SELECT * FROM num_input_test; + +-- [SPARK-28318] Decimal can only support precision up to 38 +-- +-- Test some corner cases for multiplication +-- + +-- select 4790999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4789999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4770999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- select 4769999999999999999999999999999999999999999999999999999999999999999999999999999999999999 * 9999999999999999999999999999999999999999999999999999999999999999999999999999999999999999; + +-- +-- Test some corner cases for division +-- +-- 999999999999999999999 is overflow for SYSTEM_DEFAULT(decimal(38, 18)), we use BigIntDecimal(decimal(38, 0)). +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000; +-- [SPARK-28322] DIV support decimal type +-- select div(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000); +-- select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +-- select div(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000)*1000000000000000000000 + mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000); +select mod (70.0,70) ; +-- select div (70.0,70) ; +select 70.0 / 70 ; +select 12345678901234567890 % 123; +-- [SPARK-2659] HiveQL: Division operator should always perform fractional division +-- select 12345678901234567890 DIV 123; +-- select div(12345678901234567890, 123); +-- select div(12345678901234567890, 123) * 123 + 12345678901234567890 % 123; + +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- +-- Test code path for raising to integer powers +-- + +-- select 10.0 ^ -2147483648 as rounds_to_zero; +-- select 10.0 ^ -2147483647 as rounds_to_zero; +-- select 10.0 ^ 2147483647 as overflows; +-- select 117743296169.0 ^ 1000000000 as overflows; + +-- cases that used to return inaccurate results +-- select 3.789 ^ 21; +-- select 3.789 ^ 35; +-- select 1.2 ^ 345; +-- select 0.12 ^ (-20); + +-- cases that used to error out +-- select 0.12 ^ (-25); +-- select 0.5678 ^ (-85); + +-- +-- Tests for raising to non-integer powers +-- + +-- special cases +-- select 0.0 ^ 0.0; +-- select (-12.34) ^ 0.0; +-- select 12.34 ^ 0.0; +-- select 0.0 ^ 12.34; + +-- NaNs +-- select 'NaN'::numeric ^ 'NaN'::numeric; +-- select 'NaN'::numeric ^ 0; +-- select 'NaN'::numeric ^ 1; +-- select 0 ^ 'NaN'::numeric; +-- select 1 ^ 'NaN'::numeric; + +-- invalid inputs +-- select 0.0 ^ (-12.34); +-- select (-12.34) ^ 1.2; + +-- cases that used to generate inaccurate results +-- select 32.1 ^ 9.8; +-- select 32.1 ^ (-9.8); +-- select 12.3 ^ 45.6; +-- select 12.3 ^ (-45.6); + +-- big test +-- select 1.234 ^ 5678; + +-- +-- Tests for EXP() +-- + +-- special cases +select exp(0.0); +select exp(1.0); +-- [SPARK-28316] EXP returns double type for decimal input +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select exp(1.0::numeric(71,70)); + +-- cases that used to generate inaccurate results +select exp(32.999); +select exp(-32.999); +select exp(123.456); +select exp(-123.456); + +-- big test +select exp(1234.5678); + +-- +-- Tests for generate_series +-- +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))); +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))); +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))); +-- Trigger errors +-- select * from generate_series(-100::numeric, 100::numeric, 0::numeric); +-- select * from generate_series(-100::numeric, 100::numeric, 'nan'::numeric); +-- select * from generate_series('nan'::numeric, 100::numeric, 10::numeric); +-- select * from generate_series(0::numeric, 'nan'::numeric, 10::numeric); +-- [SPARK-28007] Caret operator (^) means bitwise XOR in Spark and exponentiation in Postgres +-- Checks maximum, output is truncated +-- select (i / (10::numeric ^ 131071))::numeric(1,0) +-- from generate_series(6 * (10::numeric ^ 131071), +-- 9 * (10::numeric ^ 131071), +-- 10::numeric ^ 131071) as a(i); +-- Check usage with variables +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(i,3) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,i) j; +-- select * from generate_series(1::numeric, 3::numeric) i, generate_series(1,5,i) j; + +-- +-- Tests for LN() +-- + +-- [SPARK-27923] Invalid inputs for LN throws exception at PostgreSQL +-- Invalid inputs +-- select ln(-12.34); +-- select ln(0.0); + +-- Some random tests +select ln(1.2345678e-28); +select ln(0.0456789); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select ln(0.349873948359354029493948309745709580730482050975); +select ln(0.99949452); +select ln(1.00049687395); +select ln(1234.567890123456789); +select ln(5.80397490724e5); +select ln(9.342536355e34); + +-- +-- Tests for LOG() (base 10) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34); +-- select log(0.0); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.234567e-89); +-- [SPARK-28324] The LOG function using 10 as the base, but Spark using E +select log(3.4634998359873254962349856073435545); +select log(9.999999999999999999); +select log(10.00000000000000000); +select log(10.00000000000000001); +select log(590489.45235237); + +-- +-- Tests for LOG() (arbitrary base) +-- + +-- [SPARK-27923] Invalid inputs for LOG throws exception at PostgreSQL +-- invalid inputs +-- select log(-12.34, 56.78); +-- select log(-12.34, -56.78); +-- select log(12.34, -56.78); +-- select log(0.0, 12.34); +-- select log(12.34, 0.0); +-- select log(1.0, 12.34); + +-- some random tests +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(1.23e-89, 6.4689e45); +select log(0.99923, 4.58934e34); +select log(1.000016, 8.452010e18); +-- [SPARK-28318] Decimal can only support precision up to 38 +-- select log(3.1954752e47, 9.4792021e-73); + +-- [SPARK-28317] Built-in Mathematical Functions: SCALE +-- +-- Tests for scale() +-- + +-- select scale(numeric 'NaN'); +-- select scale(NULL::numeric); +-- select scale(1.12); +-- select scale(0); +-- select scale(0.00); +-- select scale(1.12345); +-- select scale(110123.12475871856128); +-- select scale(-1123.12471856128); +-- select scale(-13.000000000000000); + +-- +-- Tests for SUM() +-- + +-- cases that need carry propagation +SELECT SUM(decimal(9999)) FROM range(1, 100001); +SELECT SUM(decimal(-9999)) FROM range(1, 100001); + +DROP TABLE num_data; +DROP TABLE num_exp_add; +DROP TABLE num_exp_sub; +DROP TABLE num_exp_div; +DROP TABLE num_exp_mul; +DROP TABLE num_exp_sqrt; +DROP TABLE num_exp_ln; +DROP TABLE num_exp_log10; +DROP TABLE num_exp_power_10_ln; +DROP TABLE num_result; +DROP TABLE num_input_test; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql new file mode 100644 index 0000000000000..1f83d6c41661f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select.sql @@ -0,0 +1,285 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT +-- Test int8 64-bit integers. +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select.sql +-- +create or replace temporary view onek2 as select * from onek; +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2); + +-- btree index +-- awk '{if($1<10){print;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1d -2 +0nr -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$16;}else{next;}}' onek.data | +-- sort +1dr -2 +0n -1 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0nr -1 +1d -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC; + +-- +-- awk '{if($1<20){print $1,$16;}else{next;}}' onek.data | +-- sort +0n -1 +1dr -2 +-- +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC; + +-- +-- test partial btree indexes +-- +-- As of 7.2, planner probably won't pick an indexscan without stats, +-- so ANALYZE first. Also, we want to prevent it from picking a bitmapscan +-- followed by sort, because that could hide index ordering problems. +-- +-- ANALYZE onek2; + +-- SET enable_seqscan TO off; +-- SET enable_bitmapscan TO off; +-- SET enable_sort TO off; + +-- +-- awk '{if($1<10){print $0;}else{next;}}' onek.data | sort +0n -1 +-- +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10; + +-- +-- awk '{if($1<20){print $1,$14;}else{next;}}' onek.data | sort +0nr -1 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC; + +-- +-- awk '{if($1>980){print $1,$14;}else{next;}}' onek.data | sort +1d -2 +-- +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980; + +-- RESET enable_seqscan; +-- RESET enable_bitmapscan; +-- RESET enable_sort; + +-- [SPARK-28329] SELECT INTO syntax +-- SELECT two, stringu1, ten, string4 +-- INTO TABLE tmp +-- FROM onek; +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=2){print $4,$5;}else{print;}}' - stud_emp.data +-- +-- SELECT name, age FROM person*; ??? check if different +-- SELECT p.name, p.age FROM person* p; + +-- +-- awk '{print $1,$2;}' person.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - emp.data | +-- awk '{if(NF!=2){print $3,$2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $4,$5;}else{print;}}' - stud_emp.data | +-- sort +1nr -2 +-- +-- SELECT p.name, p.age FROM person* p ORDER BY age DESC, name; + +-- [SPARK-28330] Enhance query limit +-- +-- Test some cases involving whole-row Var referencing a subquery +-- +select foo.* from (select 1) as foo; +select foo.* from (select null) as foo; +select foo.* from (select 'xyzzy',1,null) as foo; + +-- +-- Test VALUES lists +-- +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j; + +-- [SPARK-28296] Improved VALUES support +-- a more complex case +-- looks like we're coding lisp :-) +-- select * from onek, +-- (values ((select i from +-- (values(10000), (2), (389), (1000), (2000), ((select 10029))) as foo(i) +-- order by i asc limit 1))) bar (i) +-- where onek.unique1 = bar.i; + +-- try VALUES in a subquery +-- select * from onek +-- where (unique1,ten) in (values (1,1), (20,0), (99,9), (17,99)) +-- order by unique1; + +-- VALUES is also legal as a standalone query or a set-operation member +VALUES (1,2), (3,4+4), (7,77.7); + +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl; + +-- +-- Test ORDER BY options +-- + +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1); + +-- [SPARK-28333] NULLS FIRST for DESC and NULLS LAST for ASC +SELECT * FROM foo ORDER BY f1; +SELECT * FROM foo ORDER BY f1 ASC; -- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST; +SELECT * FROM foo ORDER BY f1 DESC; +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- check if indexscans do the right things +-- CREATE INDEX fooi ON foo (f1); +-- SET enable_sort = false; + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- DROP INDEX fooi; +-- CREATE INDEX fooi ON foo (f1 DESC NULLS LAST); + +-- SELECT * FROM foo ORDER BY f1; +-- SELECT * FROM foo ORDER BY f1 NULLS FIRST; +-- SELECT * FROM foo ORDER BY f1 DESC; +-- SELECT * FROM foo ORDER BY f1 DESC NULLS LAST; + +-- +-- Test planning of some cases with partial indexes +-- + +-- partial index is usable +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- actually run the query with an analyze to use the partial index +-- explain (costs off, analyze on, timing off, summary off) +-- select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA'; +-- partial index predicate implies clause, so no need for retest +-- explain (costs off) +-- select * from onek2 where unique2 = 11 and stringu1 < 'B'; +select * from onek2 where unique2 = 11 and stringu1 < 'B'; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- but if it's an update target, must retest anyway +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' for update; +-- partial index is not applicable +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C'; +-- partial index implies clause, but bitmap scan must recheck predicate anyway +-- SET enable_indexscan TO off; +-- explain (costs off) +-- select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B'; +-- RESET enable_indexscan; +-- check multi-index cases too +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B'; +-- explain (costs off) +-- select unique1, unique2 from onek2 +-- where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0; + +-- +-- Test some corner cases that have been known to confuse the planner +-- + +-- ORDER BY on a constant doesn't really need any sorting +SELECT 1 AS x ORDER BY x; + +-- But ORDER BY on a set-valued expression does +-- create function sillysrf(int) returns setof int as +-- 'values (1),(10),(2),($1)' language sql immutable; + +-- select sillysrf(42); +-- select sillysrf(-1) order by 1; + +-- drop function sillysrf(int); + +-- X = X isn't a no-op, it's effectively X IS NOT NULL assuming = is strict +-- (see bug #5084) +select * from (values (2),(null),(1)) v(k) where k = k order by k; +select * from (values (2),(null),(1)) v(k) where k = k; + +-- Test partitioned tables with no partitions, which should be handled the +-- same as the non-inheritance case when expanding its RTE. +-- create table list_parted_tbl (a int,b int) partition by list (a); +-- create table list_parted_tbl1 partition of list_parted_tbl +-- for values in (1) partition by list(b); +-- explain (costs off) select * from list_parted_tbl; +-- drop table list_parted_tbl; +drop table tmp; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql new file mode 100644 index 0000000000000..5306028e5bd7b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_distinct.sql @@ -0,0 +1,86 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_DISTINCT +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_distinct.sql +-- + +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek; + +-- +-- awk '{print $3;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT two FROM tmp ORDER BY 1; + +-- +-- awk '{print $5;}' onek.data | sort -n | uniq +-- +SELECT DISTINCT ten FROM tmp ORDER BY 1; + +-- +-- awk '{print $16;}' onek.data | sort -d | uniq +-- +SELECT DISTINCT string4 FROM tmp ORDER BY 1; + +-- [SPARK-28010] Support ORDER BY ... USING syntax +-- +-- awk '{print $3,$16,$5;}' onek.data | sort -d | uniq | +-- sort +0n -1 +1d -2 +2n -3 +-- +-- SELECT DISTINCT two, string4, ten +-- FROM tmp +-- ORDER BY two using <, string4 using <, ten using <; +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC; + +-- Skip the person table because there is a point data type that we don't support. +-- +-- awk '{print $2;}' person.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - emp.data | +-- awk '{if(NF!=1){print $2;}else{print;}}' - student.data | +-- awk 'BEGIN{FS=" ";}{if(NF!=1){print $5;}else{print;}}' - stud_emp.data | +-- sort -n -r | uniq +-- +-- SELECT DISTINCT p.age FROM person* p ORDER BY age using >; + +-- +-- Check mentioning same column more than once +-- + +-- EXPLAIN (VERBOSE, COSTS OFF) +-- SELECT count(*) FROM +-- (SELECT DISTINCT two, four, two FROM tenk1) ss; + +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss; + +-- +-- Also, some tests of IS DISTINCT FROM, which doesn't quite deserve its +-- very own regression file. +-- + +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1); + +-- basic cases +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable; +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable; + +-- check that optimizer constant-folds it properly +SELECT 1 IS DISTINCT FROM 2 as `yes`; +SELECT 2 IS DISTINCT FROM 2 as `no`; +SELECT 2 IS DISTINCT FROM null as `yes`; +SELECT null IS DISTINCT FROM null as `no`; + +-- negated form +SELECT 1 IS NOT DISTINCT FROM 2 as `no`; +SELECT 2 IS NOT DISTINCT FROM 2 as `yes`; +SELECT 2 IS NOT DISTINCT FROM null as `no`; +SELECT null IS NOT DISTINCT FROM null as `yes`; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql new file mode 100644 index 0000000000000..2edde8df08047 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_having.sql @@ -0,0 +1,55 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_HAVING +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_having.sql +-- + +-- load test data +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet; +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b'); +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c'); +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j'); + +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c; + +-- HAVING is effectively equivalent to WHERE in this case +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c; + +-- [SPARK-28386] Cannot resolve ORDER BY columns with GROUP BY and HAVING +-- SELECT lower(c), count(c) FROM test_having +-- GROUP BY lower(c) HAVING count(*) > 2 OR min(a) = max(a) +-- ORDER BY lower(c); + +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c; + +-- test degenerate cases involving HAVING without GROUP BY +-- Per SQL spec, these should generate 0 or 1 row, even without aggregates + +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a); +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a); + +-- errors: ungrouped column references +SELECT a FROM test_having HAVING min(a) < max(a); +SELECT 1 AS one FROM test_having HAVING a > 1; + +-- the really degenerate case: need not scan table at all +SELECT 1 AS one FROM test_having HAVING 1 > 2; +SELECT 1 AS one FROM test_having HAVING 1 < 2; + +-- and just to prove that we aren't scanning the table: +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2; + +DROP TABLE test_having; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql new file mode 100644 index 0000000000000..54b3083a9f4a2 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/select_implicit.sql @@ -0,0 +1,160 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- SELECT_IMPLICIT +-- Test cases for queries with ordering terms missing from the target list. +-- This used to be called "junkfilter.sql". +-- The parser uses the term "resjunk" to handle these cases. +-- - thomas 1998-07-09 +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/select_implicit.sql +-- + +-- load test data +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet; +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A'); +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b'); +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c'); +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D'); +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e'); +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F'); +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g'); +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h'); +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I'); +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j'); + + +-- w/ existing GROUP BY target +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c; + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/ existing GROUP BY target using a relation name in target +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b; + +-- w/o existing GROUP BY target +SELECT c FROM test_missing_target ORDER BY a; + +-- w/o existing ORDER BY target +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc; + +-- group using reference number +SELECT count(*) FROM test_missing_target ORDER BY 1 desc; + +-- order using reference number +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1; + +-- group using reference number out of range +-- failure expected +SELECT c, count(*) FROM test_missing_target GROUP BY 3; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b; + +-- order w/ target under ambiguous condition +-- failure NOT expected +SELECT a, a FROM test_missing_target + ORDER BY a; + +-- order expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2; + +-- group expression w/ target under ambiguous condition +-- failure NOT expected +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- group w/o existing GROUP BY target under ambiguous condition +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(*) INTO TABLE test_missing_target2 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b ORDER BY x.b; +-- SELECT * FROM test_missing_target2; + + +-- Functions and expressions + +-- w/ existing GROUP BY target +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2; + +-- w/o existing GROUP BY target using a relation name in GROUP BY clause +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c); + +-- w/o existing GROUP BY target and w/o existing a different ORDER BY target +-- failure expected +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b; + +-- w/o existing GROUP BY target and w/o existing same ORDER BY target +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2; + +-- w/ existing GROUP BY target using a relation name in target +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c); + +-- w/o existing GROUP BY target +SELECT a FROM test_missing_target ORDER BY upper(d); + +-- w/o existing ORDER BY target +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc; + +-- group w/o existing GROUP BY and ORDER BY target under ambiguous condition +-- failure expected +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2; + +-- group w/ existing GROUP BY target under ambiguous condition +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2; + +-- group w/o existing GROUP BY target under ambiguous condition +-- failure expected due to ambiguous b in count(b) +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2; + +-- [SPARK-28329] SELECT INTO syntax +-- group w/o existing GROUP BY target under ambiguous condition +-- into a table +-- SELECT count(x.b) INTO TABLE test_missing_target3 +-- FROM test_missing_target x, test_missing_target y +-- WHERE x.a = y.a +-- GROUP BY x.b/2 ORDER BY x.b/2; +-- SELECT * FROM test_missing_target3; + +-- Cleanup +DROP TABLE test_missing_target; +-- DROP TABLE test_missing_target2; +-- DROP TABLE test_missing_target3; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql new file mode 100644 index 0000000000000..02af15a879c84 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/timestamp.sql @@ -0,0 +1,247 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- TIMESTAMP +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/timestamp.sql + +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet; + +-- [SPARK-28141] Timestamp type can not accept special values +-- Test shorthand input values +-- We can't just "select" the results since they aren't constants; test for +-- equality instead. We can do that by running the test inside a transaction +-- block, within which the value of 'now' shouldn't change. We also check +-- that 'now' *does* change over a reasonable interval such as 100 msec. +-- NOTE: it is possible for this part of the test to fail if the transaction +-- block is entered exactly at local midnight; then 'now' and 'today' have +-- the same values and the counts will come out different. + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); + +-- BEGIN; + +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('today'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('yesterday'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow'); +-- time zone should be ignored by this data type +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow EST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('tomorrow zulu'); + +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'today'; +-- SELECT count(*) AS Three FROM TIMESTAMP_TBL WHERE d1 = timestamp 'tomorrow'; +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp 'yesterday'; +-- SELECT count(*) AS One FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; + +-- COMMIT; + +-- DELETE FROM TIMESTAMP_TBL; + +-- verify uniform transaction time within transaction block +-- BEGIN; +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- INSERT INTO TIMESTAMP_TBL VALUES ('now'); +-- SELECT pg_sleep(0.1); +-- SELECT count(*) AS two FROM TIMESTAMP_TBL WHERE d1 = timestamp(2) 'now'; +-- COMMIT; + +-- TRUNCATE TIMESTAMP_TBL; + +-- Special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('-infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('infinity'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('epoch'); +-- [SPARK-27923] Spark SQL insert there obsolete special values to NULL +-- Obsolete special values +-- INSERT INTO TIMESTAMP_TBL VALUES ('invalid'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('undefined'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('current'); + +-- [SPARK-28259] Date/Time Output Styles and Date Order Conventions +-- Postgres v6.0 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01 1997 PST'); + +-- Variations on Postgres v6.1 standard output format +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.000001 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.999999 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.4 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.5 1997 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mon Feb 10 17:32:01.6 1997 PST'); + +-- ISO 8601 format +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05'); +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 -08:00'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 17:32:01 -07:00'); +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20'); + +-- POSIX format (note that the timezone abbrev is just decoration here) +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 08:14:01 GMT+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 13:14:02 GMT-1'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 12:14:03 GMT-2'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 03:14:04 PST+8'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('2000-03-15 02:14:05 MST+7:00'); + +-- Variations for acceptable input formats +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997 -0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 5:32PM 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997/02/10 17:32:01-0800'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('02-10-1997 17:32:01 PST'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 PST'); +-- set datestyle to ymd; +-- INSERT INTO TIMESTAMP_TBL VALUES ('97FEB10 5:32:01PM UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('97/02/10 17:32:01 UTC'); +-- reset datestyle; +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997.041 17:32:01 UTC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970210 173201 America/New_York'); +-- this fails (even though TZ is a no-op, we still look it up) +-- INSERT INTO TIMESTAMP_TBL VALUES ('19970710 173201 America/Does_not_exist'); + +-- Check date conversion and date arithmetic +-- INSERT INTO TIMESTAMP_TBL VALUES ('1997-06-10 18:32:01 PDT'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 10 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 11 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 12 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 13 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 14 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 15 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097 BC'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 0597'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1697'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1797'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1897'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 2097'); + +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1996'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 28 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 29 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Mar 01 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 30 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1997'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 1999'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Dec 31 17:32:01 2000'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Jan 01 17:32:01 2001'); + +-- Currently unsupported syntax and ranges +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 -0097'); +-- INSERT INTO TIMESTAMP_TBL VALUES ('Feb 16 17:32:01 5097 BC'); + +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL; + +-- [SPARK-28253] Date/Timestamp type have different low value and high value with Spark +-- Check behavior at the lower boundary of the timestamp range +-- SELECT '4714-11-24 00:00:00 BC'::timestamp; +-- SELECT '4714-11-23 23:59:59 BC'::timestamp; -- out of range +-- The upper boundary differs between integer and float timestamps, so no check + +-- Demonstrate functions and operators +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02'; + +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02'; + +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02'; + +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02'; + +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02'; + +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02'; + +-- [SPARK-28425] Add more Date/Time Operators +-- SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc; + +-- [SPARK-28425] Add more Date/Time Operators +-- Test casting within a BETWEEN qualifier +-- SELECT '' AS `54`, d1 - timestamp '1997-01-02' AS diff +-- FROM TIMESTAMP_TBL +-- WHERE d1 BETWEEN timestamp '1902-01-01' +-- AND timestamp '2038-01-01'; + +-- [SPARK-28420] Date/Time Functions: date_part +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'year', d1) AS year, date_part( 'month', d1) AS month, +-- date_part( 'day', d1) AS day, date_part( 'hour', d1) AS hour, +-- date_part( 'minute', d1) AS minute, date_part( 'second', d1) AS second +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'quarter', d1) AS quarter, date_part( 'msec', d1) AS msec, +-- date_part( 'usec', d1) AS usec +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- SELECT '' AS "54", d1 as "timestamp", +-- date_part( 'isoyear', d1) AS isoyear, date_part( 'week', d1) AS week, +-- date_part( 'dow', d1) AS dow +-- FROM TIMESTAMP_TBL WHERE d1 BETWEEN '1902-01-01' AND '2038-01-01'; + +-- [SPARK-28137] Data Type Formatting Functions +-- TO_CHAR() +-- SELECT '' AS to_char_1, to_char(d1, 'DAY Day day DY Dy dy MONTH Month month RM MON Mon mon') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_2, to_char(d1, 'FMDAY FMDay FMday FMMONTH FMMonth FMmonth FMRM') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_3, to_char(d1, 'Y,YYY YYYY YYY YY Y CC Q MM WW DDD DD D J') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_4, to_char(d1, 'FMY,YYY FMYYYY FMYYY FMYY FMY FMCC FMQ FMMM FMWW FMDDD FMDD FMD FMJ') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_5, to_char(d1, 'HH HH12 HH24 MI SS SSSS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_6, to_char(d1, E'"HH:MI:SS is" HH:MI:SS "\\"text between quote marks\\""') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_7, to_char(d1, 'HH24--text--MI--text--SS') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_8, to_char(d1, 'YYYYTH YYYYth Jth') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_9, to_char(d1, 'YYYY A.D. YYYY a.d. YYYY bc HH:MI:SS P.M. HH:MI:SS p.m. HH:MI:SS pm') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_10, to_char(d1, 'IYYY IYY IY I IW IDDD ID') +-- FROM TIMESTAMP_TBL; + +-- SELECT '' AS to_char_11, to_char(d1, 'FMIYYY FMIYY FMIY FMI FMIW FMIDDD FMID') +-- FROM TIMESTAMP_TBL; + + +--[SPARK-28432] Missing Date/Time Functions: make_timestamp +-- timestamp numeric fields constructor +-- SELECT make_timestamp(2014,12,28,6,30,45.887); + +DROP TABLE TIMESTAMP_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql new file mode 100644 index 0000000000000..83c6fd8cbac91 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/pgSQL/with.sql @@ -0,0 +1,1208 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- WITH +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/with.sql +-- +-- This test uses the generate_series(...) function which is rewritten to EXPLODE(SEQUENCE(...)) as +-- it's feature tracking ticket SPARK-27767 is closed as Won't Do. + +-- +-- Tests for common table expressions (WITH query, ... SELECT ...) +-- + +-- Basic WITH +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2; + +-- Multiple uses are evaluated only once +-- [SPARK-28299] Evaluation of multiple CTE uses +-- [ORIGINAL SQL] +--SELECT count(*) FROM ( +-- WITH q1(x) AS (SELECT random() FROM generate_series(1, 5)) +-- SELECT * FROM q1 +-- UNION +-- SELECT * FROM q1 +--) ss; +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss; + +-- WITH RECURSIVE + +-- sum of 1..100 +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT (VALUES(1)) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 5 +--) +--SELECT * FROM t; + +-- recursive view +-- [SPARK-24497] Support recursive SQL query +--CREATE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 5; +-- +--SELECT * FROM nums; + +-- [SPARK-24497] Support recursive SQL query +--CREATE OR REPLACE RECURSIVE VIEW nums (n) AS +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM nums WHERE n < 6; +-- +--SELECT * FROM nums; + +-- This is an infinite loop with UNION ALL, but not with UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT 10-n FROM t) +--SELECT * FROM t; + +-- This'd be an infinite loop, but outside query reads only as much as needed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- UNION case should have same property +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(n) AS ( +-- SELECT 1 +--UNION +-- SELECT n+1 FROM t) +--SELECT * FROM t LIMIT 10; + +-- Test behavior with an unknown-type literal in the WITH +-- [SPARK-28146] Support IS OF type predicate +--WITH q AS (SELECT 'foo' AS x) +--SELECT x, x IS OF (text) AS is_text FROM q; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT 'foo' +--UNION ALL +-- SELECT n || ' bar' FROM t WHERE length(n) < 20 +--) +--SELECT n, n IS OF (text) AS is_text FROM t; + +-- In a perfect world, this would work and resolve the literal as int ... +-- but for now, we have to be content with resolving to text too soon. +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28146] Support IS OF type predicate +--WITH RECURSIVE t(n) AS ( +-- SELECT '7' +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 10 +--) +--SELECT n, n IS OF (int) AS is_int FROM t; + +-- +-- Some examples with a tree +-- +-- department structure represented here is as follows: +-- +-- ROOT-+->A-+->B-+->C +-- | | +-- | +->D-+->F +-- +->E-+->G + + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE department ( +-- id INTEGER PRIMARY KEY, -- department ID +-- parent_department INTEGER REFERENCES department, -- upper department ID +-- name string -- department name +--); +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet; + +INSERT INTO department VALUES (0, NULL, 'ROOT'); +INSERT INTO department VALUES (1, 0, 'A'); +INSERT INTO department VALUES (2, 1, 'B'); +INSERT INTO department VALUES (3, 2, 'C'); +INSERT INTO department VALUES (4, 2, 'D'); +INSERT INTO department VALUES (5, 0, 'E'); +INSERT INTO department VALUES (6, 4, 'F'); +INSERT INTO department VALUES (7, 5, 'G'); + + +-- extract all departments under 'A'. Result should be A, B, C, D and F +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- non recursive term +-- SELECT name as root_name, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.root_name, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- extract all departments under 'A' with "level" number. +-- Only shows level 2 or more +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment(level, id, parent_department, name) AS +--( +-- -- non recursive term +-- SELECT 1, * FROM department WHERE name = 'A' +-- +-- UNION ALL +-- +-- -- recursive term +-- SELECT sd.level + 1, d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +--) +--SELECT * FROM subdepartment WHERE level >= 2 ORDER BY name; + +-- "RECURSIVE" is ignored if the query has no self-reference +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE subdepartment AS +--( +-- -- note lack of recursive UNION structure +-- SELECT * FROM department WHERE name = 'A' +--) +--SELECT * FROM subdepartment ORDER BY name; + +-- inside subqueries +-- [SPARK-24497] Support recursive SQL query +--SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 500 +-- ) +-- SELECT * FROM t) AS t WHERE n < ( +-- SELECT count(*) FROM ( +-- WITH RECURSIVE t(n) AS ( +-- SELECT 1 UNION ALL SELECT n + 1 FROM t WHERE n < 100 +-- ) +-- SELECT * FROM t WHERE n < 50000 +-- ) AS t WHERE n < 100); + +-- use same CTE twice at different subquery levels +-- [SPARK-24497] Support recursive SQL query +--WITH q1(x,y) AS ( +-- SELECT hundred, sum(ten) FROM tenk1 GROUP BY hundred +-- ) +--SELECT count(*) FROM q1 WHERE y > (SELECT sum(y)/100 FROM q1 qsub); + +-- via a VIEW +-- [SPARK-24497] Support recursive SQL query +--CREATE TEMPORARY VIEW vsubdepartment AS +-- WITH RECURSIVE subdepartment AS +-- ( +-- -- non recursive term +-- SELECT * FROM department WHERE name = 'A' +-- UNION ALL +-- -- recursive term +-- SELECT d.* FROM department AS d, subdepartment AS sd +-- WHERE d.parent_department = sd.id +-- ) +-- SELECT * FROM subdepartment; +-- +--SELECT * FROM vsubdepartment ORDER BY name; +-- +---- Check reverse listing +--SELECT pg_get_viewdef('vsubdepartment'::regclass); +--SELECT pg_get_viewdef('vsubdepartment'::regclass, true); + +-- Another reverse-listing example +-- [SPARK-24497] Support recursive SQL query +--CREATE VIEW sums_1_100 AS +--WITH RECURSIVE t(n) AS ( +-- VALUES (1) +--UNION ALL +-- SELECT n+1 FROM t WHERE n < 100 +--) +--SELECT sum(n) FROM t; +-- +--\d+ sums_1_100 + +-- corner case in which sub-WITH gets initialized first +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with x as (select * from q) +-- select * from x) +-- ) +--select * from q limit 24; + +-- [SPARK-24497] Support recursive SQL query +--with recursive q as ( +-- select * from department +-- union all +-- (with recursive x as ( +-- select * from department +-- union all +-- (select * from q union all select * from x) +-- ) +-- select * from x) +-- ) +--select * from q limit 32; + +-- recursive term has sub-UNION +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(i,j) AS ( +-- VALUES (1,2) +-- UNION ALL +-- SELECT t2.i, t.j+1 FROM +-- (SELECT 2 AS i UNION ALL SELECT 3 AS i) AS t2 +-- JOIN t ON (t2.i = t.i+1)) +-- +-- SELECT * FROM t; + +-- +-- different tree example +-- +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE tree( +-- id INTEGER PRIMARY KEY, +-- parent_id INTEGER REFERENCES tree(id) +--); +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet; + +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11); + +-- +-- get all paths from "second level" nodes to leaf nodes +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.*, t2.* FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- ORDER BY t1.id, t2.id; + +-- just count 'em +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, count(t2.*) FROM t AS t1 JOIN t AS t2 ON +-- (t1.path[1] = t2.path[1] AND +-- array_upper(t1.path,1) = 1 AND +-- array_upper(t2.path,1) > 1) +-- GROUP BY t1.id +-- ORDER BY t1.id; + +-- this variant tickled a whole-row-variable bug in 8.4devel +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(id, path) AS ( +-- VALUES(1,ARRAY[]::integer[]) +--UNION ALL +-- SELECT tree.id, t.path || tree.id +-- FROM tree JOIN t ON (tree.parent_id = t.id) +--) +--SELECT t1.id, t2.path, t2 FROM t AS t1 JOIN t AS t2 ON +--(t1.id=t2.id); + +-- +-- test cycle detection +-- +-- [ORIGINAL SQL] +--create temp table graph( f int, t int, label text ); +create table graph( f int, t int, label string ) USING parquet; + +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1'); + +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph; + +-- ordering by the path column has same effect as SEARCH DEPTH FIRST +-- [SPARK-24497] Support recursive SQL query +--with recursive search_graph(f, t, label, path, cycle) as ( +-- select *, array[row(g.f, g.t)], false from graph g +-- union all +-- select g.*, path || row(g.f, g.t), row(g.f, g.t) = any(path) +-- from graph g, search_graph sg +-- where g.f = sg.t and not cycle +--) +--select * from search_graph order by path; + +-- +-- test multiple WITH queries +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- y (id) AS (VALUES (1)), +-- x (id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- forward reference OK +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS (SELECT * FROM y UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS (values (1)) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM y WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 5), +-- y(id) AS +-- (VALUES (1) UNION ALL SELECT id+1 FROM x WHERE id < 10) +-- SELECT y.*, x.* FROM y LEFT JOIN x USING (id); + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM x UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x(id) AS +-- (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 3 ), +-- y(id) AS +-- (SELECT * FROM x UNION ALL SELECT * FROM x), +-- z(id) AS +-- (SELECT * FROM y UNION ALL SELECT id+1 FROM z WHERE id < 10) +-- SELECT * FROM z; + +-- +-- Test WITH attached to a data-modifying statement +-- + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- SELECT a FROM y +--) +--INSERT INTO y +--SELECT a+20 FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- SELECT a FROM y +--) +--UPDATE y SET a = y.a-10 FROM t WHERE y.a > 20 AND t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH RECURSIVE t(a) AS ( +-- SELECT 11 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a < 50 +--) +--DELETE FROM y USING t WHERE t.a = y.a RETURNING y.a; +-- +--SELECT * FROM y; + +DROP TABLE y; + +-- +-- error cases +-- + +-- INTERSECT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 INTERSECT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- EXCEPT +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 EXCEPT ALL SELECT n+1 FROM x) +-- SELECT * FROM x; + +-- no non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x) +-- SELECT * FROM x; + +-- recursive term in the left hand side (strictly speaking, should allow this) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT n FROM x UNION ALL SELECT 1) +-- SELECT * FROM x; + +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE y (a INTEGER); +CREATE TABLE y (a INTEGER) USING parquet; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 10); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)); + +-- LEFT JOIN + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM y LEFT JOIN x ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- RIGHT JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x RIGHT JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- FULL JOIN +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT a FROM y WHERE a = 1 +-- UNION ALL +-- SELECT x.n+1 FROM x FULL JOIN y ON x.n = y.a WHERE n < 10) +--SELECT * FROM x; + +-- subquery +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x +-- WHERE n IN (SELECT * FROM x)) +-- SELECT * FROM x; + +-- aggregate functions +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT count(*) FROM x) +-- SELECT * FROM x; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT sum(n) FROM x) +-- SELECT * FROM x; + +-- ORDER BY +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x ORDER BY 1) +-- SELECT * FROM x; + +-- LIMIT/OFFSET +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x LIMIT 10 OFFSET 1) +-- SELECT * FROM x; + +-- FOR UPDATE +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(n) AS (SELECT 1 UNION ALL SELECT n+1 FROM x FOR UPDATE) +-- SELECT * FROM x; + +-- target list has a recursive query name +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE x(id) AS (values (1) +-- UNION ALL +-- SELECT (SELECT * FROM x) FROM x WHERE id < 5 +--) SELECT * FROM x; + +-- mutual recursive query (not implemented) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- x (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM y WHERE id < 5), +-- y (id) AS (SELECT 1 UNION ALL SELECT id+1 FROM x WHERE id < 5) +--SELECT * FROM x; + +-- non-linear recursion is not allowed +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- SELECT * FROM +-- (SELECT i+1 FROM foo WHERE i < 10 +-- UNION ALL +-- SELECT i+1 FROM foo WHERE i < 5) AS t +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- EXCEPT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (values (1) +-- UNION ALL +-- (SELECT i+1 FROM foo WHERE i < 10 +-- INTERSECT +-- SELECT i+1 FROM foo WHERE i < 5) +--) SELECT * FROM foo; + +-- Wrong type induced from non-recursive term +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- rejects different typmod, too (should we allow this?) +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE foo(i) AS +-- (SELECT i::numeric(3,0) FROM (VALUES(1),(2)) t(i) +-- UNION ALL +-- SELECT (i+1)::numeric(10,0) FROM foo WHERE i < 10) +--SELECT * FROM foo; + +-- [NOTE] Spark SQL doesn't support RULEs +-- disallow OLD/NEW reference in CTE +--CREATE TABLE x (n integer) USING parquet; +--CREATE RULE r2 AS ON UPDATE TO x DO INSTEAD +-- WITH t AS (SELECT OLD.*) UPDATE y SET a = t.n FROM t; + +-- +-- test for bug #4902 +-- +-- [SPARK-28296] Improved VALUES support +--with cte(foo) as ( values(42) ) values((select foo from cte)); +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q; + +-- test CTE referencing an outer-level variable (to see that changed-parameter +-- signaling still works properly after fixing this bug) +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- select (select foo from cte) ) +--from int4_tbl; + +-- [SPARK-28296] Improved VALUES support +-- [SPARK-28297] Handling outer links in CTE subquery expressions +--select ( with cte(foo) as ( values(f1) ) +-- values((select foo from cte)) ) +--from int4_tbl; + +-- +-- test for nested-recursive-WITH bug +-- +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(j) AS ( +-- WITH RECURSIVE s(i) AS ( +-- VALUES (1) +-- UNION ALL +-- SELECT i+1 FROM s WHERE i < 10 +-- ) +-- SELECT i FROM s +-- UNION ALL +-- SELECT j+1 FROM t WHERE j < 10 +--) +--SELECT * FROM t; + +-- +-- test WITH attached to intermediate-level set operation +-- + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1; + +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- SELECT 1 +-- UNION (WITH innermost as (SELECT 2) +-- SELECT * FROM outermost +-- UNION SELECT * FROM innermost) +--) +--SELECT * FROM outermost ORDER BY 1; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE outermost(x) AS ( +-- WITH innermost as (SELECT 2 FROM outermost) -- fail +-- SELECT * FROM innermost +-- UNION SELECT * from outermost +--) +--SELECT * FROM outermost ORDER BY 1; + +-- +-- This test will fail with the old implementation of PARAM_EXEC parameter +-- assignment, because the "q1" Var passed down to A's targetlist subselect +-- looks exactly like the "A.id" Var passed down to C's subselect, causing +-- the old code to give them the same runtime PARAM_EXEC slot. But the +-- lifespans of the two parameters overlap, thanks to B also reading A. +-- + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +--with +--A as ( select q2 as id, (select q1) as x from int8_tbl ), +--B as ( select id, row_number() over (partition by id) as r from A ), +--C as ( select A.id, array(select B.id from B where B.id = A.id) from A ) +--select * from C; + +-- +-- Test CTEs read in non-initialization orders +-- + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ALL ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE +-- tab(id_key,link) AS (VALUES (1,17), (2,17), (3,17), (4,17), (6,17), (5,17)), +-- iter (id_key, row_type, link) AS ( +-- SELECT 0, 'base', 17 +-- UNION ( +-- WITH remaining(id_key, row_type, link, min) AS ( +-- SELECT tab.id_key, 'true'::text, iter.link, MIN(tab.id_key) OVER () +-- FROM tab INNER JOIN iter USING (link) +-- WHERE tab.id_key > iter.id_key +-- ), +-- first_remaining AS ( +-- SELECT id_key, row_type, link +-- FROM remaining +-- WHERE id_key=min +-- ), +-- effect AS ( +-- SELECT tab.id_key, 'new'::text, tab.link +-- FROM first_remaining e INNER JOIN tab ON e.id_key=tab.id_key +-- WHERE e.row_type = 'false' +-- ) +-- SELECT * FROM first_remaining +-- UNION ALL SELECT * FROM effect +-- ) +-- ) +--SELECT * FROM iter; + +-- +-- Data-modifying statements in WITH +-- + +-- INSERT ... RETURNING +-- [SPARK-28147] Support RETURNING clause +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (11), +-- (12), +-- (13), +-- (14), +-- (15), +-- (16), +-- (17), +-- (18), +-- (19), +-- (20) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- UPDATE ... RETURNING +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y +-- SET a=a+1 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- DELETE ... RETURNING +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +-- forward reference +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT a+5 FROM t2 WHERE a > 5 +-- RETURNING * +--), t2 AS ( +-- UPDATE y SET a=a-11 RETURNING * +--) +--SELECT * FROM t +--UNION ALL +--SELECT * FROM t2; +-- +--SELECT * FROM y; + +-- unconditional DO INSTEAD rule +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON DELETE TO y DO INSTEAD +-- INSERT INTO y VALUES(42) RETURNING *; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; + +--DROP RULE y_rule ON y; + +-- check merging of outer CTE with CTE in a rule action +--CREATE TEMP TABLE bug6051 AS +-- select i from generate_series(1,3) as t(i); + +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; + +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE TEMP TABLE bug6051_2 (i int); +-- +--CREATE RULE bug6051_ins AS ON INSERT TO bug6051 DO INSTEAD +-- INSERT INTO bug6051_2 +-- SELECT NEW.i; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t1 AS ( DELETE FROM bug6051 RETURNING * ) +--INSERT INTO bug6051 SELECT * FROM t1; +-- +--SELECT * FROM bug6051; +--SELECT * FROM bug6051_2; + +-- a truly recursive CTE in the same list +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t(a) AS ( +-- SELECT 0 +-- UNION ALL +-- SELECT a+1 FROM t WHERE a+1 < 5 +--), t2 as ( +-- INSERT INTO y +-- SELECT * FROM t RETURNING * +--) +--SELECT * FROM t2 JOIN y USING (a) ORDER BY a; +-- +--SELECT * FROM y; + +-- data-modifying WITH in a modifying statement +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH t AS ( +-- DELETE FROM y +-- WHERE a <= 10 +-- RETURNING * +--) +--INSERT INTO y SELECT -a FROM t RETURNING *; +-- +--SELECT * FROM y; + +-- check that WITH query is run to completion even if outer query isn't +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- UPDATE y SET a = a * 100 RETURNING * +--) +--SELECT * FROM t LIMIT 10; +-- +--SELECT * FROM y; + +-- data-modifying WITH containing INSERT...ON CONFLICT DO UPDATE +-- [ORIGINAL SQL] +--CREATE TABLE withz AS SELECT i AS k, (i || ' v')::text v FROM generate_series(1, 16, 3) i; +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i); +-- [NOTE] Spark SQL doesn't support UNIQUE constraints +--ALTER TABLE withz ADD UNIQUE (k); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH t AS ( +-- INSERT INTO withz SELECT i, 'insert' +-- FROM generate_series(0, 16) i +-- ON CONFLICT (k) DO UPDATE SET v = withz.v || ', now update' +-- RETURNING * +--) +--SELECT * FROM t JOIN y ON t.k = y.a ORDER BY a, k; + +-- Test EXCLUDED.* reference within CTE +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS ( +-- INSERT INTO withz VALUES(1, 5) ON CONFLICT (k) DO UPDATE SET v = EXCLUDED.v +-- WHERE withz.k != EXCLUDED.k +-- RETURNING * +--) +--SELECT * FROM aa; + +-- New query/snapshot demonstrates side-effects of previous query. +SELECT * FROM withz ORDER BY k; + +-- +-- Ensure subqueries within the update clause work, even if they +-- reference outside values +-- +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = ' update' WHERE withz.k = (SELECT a FROM aa); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); +--WITH aa AS (SELECT 'a' a, 'b' b UNION ALL SELECT 'a' a, 'b' b) +--INSERT INTO withz VALUES(1, 'insert') +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 'a' LIMIT 1); +--WITH aa AS (SELECT 1 a, 2 b) +--INSERT INTO withz VALUES(1, (SELECT b || ' insert' FROM aa WHERE a = 1 )) +--ON CONFLICT (k) DO UPDATE SET v = (SELECT b || ' update' FROM aa WHERE a = 1 LIMIT 1); + +-- Update a row more than once, in different parts of a wCTE. That is +-- an allowed, presumably very rare, edge case, but since it was +-- broken in the past, having a test seems worthwhile. +-- [NOTE] Spark SQL doesn't support ON CONFLICT clause +--WITH simpletup AS ( +-- SELECT 2 k, 'Green' v), +--upsert_cte AS ( +-- INSERT INTO withz VALUES(2, 'Blue') ON CONFLICT (k) DO +-- UPDATE SET (k, v) = (SELECT k, v FROM simpletup WHERE simpletup.k = withz.k) +-- RETURNING k, v) +--INSERT INTO withz VALUES(2, 'Red') ON CONFLICT (k) DO +--UPDATE SET (k, v) = (SELECT k, v FROM upsert_cte WHERE upsert_cte.k = withz.k) +--RETURNING k, v; + +DROP TABLE withz; + +-- check that run to completion happens in proper ordering + +TRUNCATE TABLE y; +-- [ORIGINAL SQL] +--INSERT INTO y SELECT generate_series(1, 3); +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)); +-- [ORIGINAL SQL] +--CREATE TEMPORARY TABLE yy (a INTEGER); +CREATE TABLE yy (a INTEGER) USING parquet; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--), t2 AS ( +-- INSERT INTO yy SELECT * FROM t1 RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [SPARK-24497] Support recursive SQL query +-- [SPARK-28147] Support RETURNING clause +--WITH RECURSIVE t1 AS ( +-- INSERT INTO yy SELECT * FROM t2 RETURNING * +--), t2 AS ( +-- INSERT INTO y SELECT * FROM y RETURNING * +--) +--SELECT 1; + +SELECT * FROM y; +SELECT * FROM yy; + +-- [NOTE] Spark SQL doesn't support TRIGGERs +-- triggers +-- +--TRUNCATE TABLE y; +--INSERT INTO y SELECT generate_series(1, 10); +-- +--CREATE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger: a = %', new.a; +-- return new; +--end; +--$$ LANGUAGE plpgsql; +-- +-- +--CREATE TRIGGER y_trig BEFORE INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (21), +-- (22), +-- (23) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH ROW +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (31), +-- (32), +-- (33) +-- RETURNING * +--) +--SELECT * FROM t LIMIT 1; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +-- +--CREATE OR REPLACE FUNCTION y_trigger() RETURNS trigger AS $$ +--begin +-- raise notice 'y_trigger'; +-- return null; +--end; +--$$ LANGUAGE plpgsql; +-- +--CREATE TRIGGER y_trig AFTER INSERT ON y FOR EACH STATEMENT +-- EXECUTE PROCEDURE y_trigger(); +-- +--WITH t AS ( +-- INSERT INTO y +-- VALUES +-- (41), +-- (42), +-- (43) +-- RETURNING * +--) +--SELECT * FROM t; +-- +--SELECT * FROM y; +-- +--DROP TRIGGER y_trig ON y; +--DROP FUNCTION y_trigger(); + +-- WITH attached to inherited UPDATE or DELETE + +-- [ORIGINAL SQL] +--CREATE TEMP TABLE parent ( id int, val text ); +CREATE TABLE parent ( id int, val string ) USING parquet; +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child1 ( ) INHERITS ( parent ); +-- [NOTE] Spark SQL doesn't support INHERITS clause +--CREATE TEMP TABLE child2 ( ) INHERITS ( parent ); + +INSERT INTO parent VALUES ( 1, 'p1' ); +--INSERT INTO child1 VALUES ( 11, 'c11' ),( 12, 'c12' ); +--INSERT INTO child2 VALUES ( 23, 'c21' ),( 24, 'c22' ); + +-- [NOTE] Spark SQL doesn't support UPDATE statement +--WITH rcte AS ( SELECT sum(id) AS totalid FROM parent ) +--UPDATE parent SET id = id + totalid FROM rcte; + +SELECT * FROM parent; + +-- [SPARK-28147] Support RETURNING clause +--WITH wcte AS ( INSERT INTO child1 VALUES ( 42, 'new' ) RETURNING id AS newid ) +--UPDATE parent SET id = id + newid FROM wcte; +-- +--SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH rcte AS ( SELECT max(id) AS maxid FROM parent ) +--DELETE FROM parent USING rcte WHERE id = maxid; + +SELECT * FROM parent; + +-- [NOTE] Spark SQL doesn't support DELETE statement +--WITH wcte AS ( INSERT INTO child2 VALUES ( 42, 'new2' ) RETURNING id AS newid ) +--DELETE FROM parent USING wcte WHERE id = newid; +-- +--SELECT * FROM parent; + +-- check EXPLAIN VERBOSE for a wCTE with RETURNING + +-- [NOTE] Spark SQL doesn't support DELETE statement +--EXPLAIN (VERBOSE, COSTS OFF) +--WITH wcte AS ( INSERT INTO int8_tbl VALUES ( 42, 47 ) RETURNING q2 ) +--DELETE FROM a USING wcte WHERE aa = q2; + +-- error cases + +-- data-modifying WITH tries to use its own output +-- [SPARK-24497] Support recursive SQL query +--WITH RECURSIVE t AS ( +-- INSERT INTO y +-- SELECT * FROM t +--) +--VALUES(FALSE); + +-- no RETURNING in a referenced data-modifying WITH +-- [SPARK-24497] Support recursive SQL query +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--SELECT * FROM t; + +-- data-modifying WITH allowed only at the top level +-- [SPARK-28147] Support RETURNING clause +--SELECT * FROM ( +-- WITH t AS (UPDATE y SET a=a+1 RETURNING *) +-- SELECT * FROM t +--) ss; + +-- most variants of rules aren't allowed +-- [NOTE] Spark SQL doesn't support RULEs +--CREATE RULE y_rule AS ON INSERT TO y WHERE a=0 DO INSTEAD DELETE FROM y; +--WITH t AS ( +-- INSERT INTO y VALUES(0) +--) +--VALUES(FALSE); +--DROP RULE y_rule ON y; + +-- check that parser lookahead for WITH doesn't cause any odd behavior +create table foo (with baz); -- fail, WITH is a reserved word +create table foo (with ordinality); -- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality; + +-- check sane response to attempt to modify CTE relation +WITH test AS (SELECT 42) INSERT INTO test VALUES (1); + +-- check response to attempt to modify table with same name as a CTE (perhaps +-- surprisingly it works, because CTEs don't hide tables from data-modifying +-- statements) +-- [ORIGINAL SQL] +--create temp table test (i int); +create table test (i int) USING parquet; +with test as (select 42) insert into test select * from test; +select * from test; +drop table test; + +-- +-- Clean up +-- + +DROP TABLE department; +DROP TABLE tree; +DROP TABLE graph; +DROP TABLE y; +DROP TABLE yy; +DROP TABLE parent; 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 new file mode 100644 index 0000000000000..852bfbd63847d --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/show-create-table.sql @@ -0,0 +1,61 @@ +-- simple +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- options +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('a' 1); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- path option +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('path' '/path/to/table'); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- location +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +LOCATION '/path/to/table'; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- partition by +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +PARTITIONED BY (a); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- clustered by +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- comment +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +COMMENT 'This is a comment'; + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; + + +-- tblproperties +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES ('a' = '1'); + +SHOW CREATE TABLE tbl; +DROP TABLE tbl; diff --git a/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql index f1461032065ad..1ae49c8bfc76a 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/sql-compatibility-functions.sql @@ -12,11 +12,6 @@ SELECT nullif(1, 2.1d), nullif(1, 1.0d); SELECT nvl(1, 2.1d), nvl(null, 2.1d); SELECT nvl2(null, 1, 2.1d), nvl2('n', 1, 2.1d); --- explain for these functions; use range to avoid constant folding -explain extended -select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') -from range(2); - -- SPARK-16730 cast alias functions for Hive compatibility SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1); SELECT float(1), double(1), decimal(1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql index 2effb43183d75..59987b9792e25 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/string-functions.sql @@ -5,10 +5,6 @@ select format_string(); -- A pipe operator for string concatenation select 'a' || 'b' || 'c'; --- Check if catalyst combine nested `Concat`s -EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col -FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)); - -- replace function select replace('abc', 'b', '123'); select replace('abc', 'b'); @@ -25,29 +21,30 @@ select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a'); select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null); select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a'); --- turn off concatBinaryAsString -set spark.sql.function.concatBinaryAsString=false; - --- Check if catalyst combine nested `Concat`s if concatBinaryAsString=false -EXPLAIN SELECT ((col1 || col2) || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - string(id + 1) col2, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -); - -EXPLAIN SELECT (col1 || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -); - -- split function SELECT split('aa1cc2ee3', '[1-9]+'); SELECT split('aa1cc2ee3', '[1-9]+', 2); + +-- substring function +SELECT substr('Spark SQL', 5); +SELECT substr('Spark SQL', -3); +SELECT substr('Spark SQL', 5, 1); +SELECT substr('Spark SQL' from 5); +SELECT substr('Spark SQL' from -3); +SELECT substr('Spark SQL' from 5 for 1); +SELECT substring('Spark SQL', 5); +SELECT substring('Spark SQL', -3); +SELECT substring('Spark SQL', 5, 1); +SELECT substring('Spark SQL' from 5); +SELECT substring('Spark SQL' from -3); +SELECT substring('Spark SQL' from 5 for 1); + +-- trim/ltrim/rtrim +SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx'); +SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx'); +SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest'); +SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz'); +SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD'); +SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx'); +SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx'); +SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy'); diff --git a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql index e22cade936792..109ffa77d6219 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/subquery/negative-cases/invalid-correlation.sql @@ -46,9 +46,10 @@ WHERE t1a IN (SELECT min(t2a) SELECT t1a FROM t1 GROUP BY 1 -HAVING EXISTS (SELECT 1 +HAVING EXISTS (SELECT t2a FROM t2 - WHERE t2a < min(t1a + t2a)); + GROUP BY 1 + HAVING t2a < min(t1a + t2a)); -- TC 01.04 -- Invalid due to mixure of outer and local references under an AggegatedExpression diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql index 72cd8ca9d8722..6f14c8ca87821 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql @@ -21,9 +21,3 @@ select * from range(1, null); -- range call with a mixed-case function name select * from RaNgE(2); - --- Explain -EXPLAIN select * from RaNgE(2); - --- cross-join table valued functions -EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3); diff --git a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql index 69da67fc66fc0..60895020fcc83 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/typeCoercion/native/mapconcat.sql @@ -13,7 +13,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( map('a', 'b'), map('c', 'd'), map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), - map(map('a', 1), map('b', 2)), map(map('c', 3), map('d', 4)), map('a', 1), map('c', 2), map(1, 'a'), map(2, 'c') ) AS various_maps ( @@ -31,7 +30,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( string_map1, string_map2, array_map1, array_map2, struct_map1, struct_map2, - map_map1, map_map2, string_int_map1, string_int_map2, int_string_map1, int_string_map2 ); @@ -51,7 +49,6 @@ SELECT map_concat(string_map1, string_map2) string_map, map_concat(array_map1, array_map2) array_map, map_concat(struct_map1, struct_map2) struct_map, - map_concat(map_map1, map_map2) map_map, map_concat(string_int_map1, string_int_map2) string_int_map, map_concat(int_string_map1, int_string_map2) int_string_map FROM various_maps; @@ -71,7 +68,7 @@ FROM various_maps; -- Concatenate map of incompatible types 1 SELECT - map_concat(tinyint_map1, map_map2) tm_map + map_concat(tinyint_map1, array_map1) tm_map FROM various_maps; -- Concatenate map of incompatible types 2 @@ -86,10 +83,10 @@ FROM various_maps; -- Concatenate map of incompatible types 4 SELECT - map_concat(map_map1, array_map2) ma_map + map_concat(struct_map1, array_map2) ma_map FROM various_maps; -- Concatenate map of incompatible types 5 SELECT - map_concat(map_map1, struct_map2) ms_map + map_concat(int_map1, array_map2) ms_map FROM various_maps; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql index 2183ba23afc38..58613a1325dfa 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/udaf.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/udaf.sql @@ -11,3 +11,6 @@ SELECT default.myDoubleAvg(int_col1, 3) as my_avg from t1; CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; SELECT default.udaf1(int_col1) as udaf1 from t1; + +DROP FUNCTION myDoubleAvg; +DROP FUNCTION udaf1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql new file mode 100644 index 0000000000000..d829a5c1159fd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part1.sql @@ -0,0 +1,154 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 1] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L1-L143 + +-- avoid bit-exact output here because operations may not be bit-exact. +-- SET extra_float_digits = 0; + +-- This test file was converted from pgSQL/aggregates_part1.sql. + +SELECT avg(udf(four)) AS avg_1 FROM onek; + +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100; + +-- In 7.1, avg(float4) is computed using float8 arithmetic. +-- Round the result to 3 digits to avoid platform-specific results. + +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT avg(gpa) AS avg_3_4 FROM ONLY student; + +SELECT sum(udf(four)) AS sum_1500 FROM onek; +SELECT udf(sum(a)) AS sum_198 FROM aggtest; +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT sum(gpa) AS avg_6_8 FROM ONLY student; + +SELECT udf(max(four)) AS max_3 FROM onek; +SELECT max(udf(a)) AS max_100 FROM aggtest; +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest; +-- `student` has a column with data type POINT, which is not supported by Spark [SPARK-27766] +-- SELECT max(student.gpa) AS max_3_7 FROM student; + +SELECT stddev_pop(udf(b)) FROM aggtest; +SELECT udf(stddev_samp(b)) FROM aggtest; +SELECT var_pop(udf(b)) FROM aggtest; +SELECT udf(var_samp(b)) FROM aggtest; + +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest; +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest; +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest; + +-- population variance is defined for a single tuple, sample variance +-- is not +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)); +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))); + + +-- verify correct results for null and NaN inputs +select sum(udf(CAST(null AS int))) from range(1,4); +select sum(udf(CAST(null AS long))) from range(1,4); +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select sum(udf(CAST(null AS DOUBLE))) from range(1,4); +select avg(udf(CAST(null AS int))) from range(1,4); +select avg(udf(CAST(null AS long))) from range(1,4); +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4); +select avg(udf(CAST(null AS DOUBLE))) from range(1,4); +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4); +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4); + +-- [SPARK-27768] verify correct results for infinite inputs +-- [SPARK-28291] UDFs cannot be evaluated within inline table definition +-- SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +-- FROM (VALUES (CAST(udf('1') AS DOUBLE)), (CAST(udf('Infinity') AS DOUBLE))) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x); +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x); + + +-- test accuracy with a large input offset +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x); +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x); + +-- SQL2003 binary aggregates [SPARK-23907] +-- SELECT regr_count(b, a) FROM aggtest; +-- SELECT regr_sxx(b, a) FROM aggtest; +-- SELECT regr_syy(b, a) FROM aggtest; +-- SELECT regr_sxy(b, a) FROM aggtest; +-- SELECT regr_avgx(b, a), regr_avgy(b, a) FROM aggtest; +-- SELECT regr_r2(b, a) FROM aggtest; +-- SELECT regr_slope(b, a), regr_intercept(b, a) FROM aggtest; +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest; +SELECT corr(b, udf(a)) FROM aggtest; + + +-- test accum and combine functions directly [SPARK-23907] +-- CREATE TABLE regr_test (x float8, y float8); +-- INSERT INTO regr_test VALUES (10,150),(20,250),(30,350),(80,540),(100,200); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30,80); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test; +-- SELECT float8_accum('{4,140,2900}'::float8[], 100); +-- SELECT float8_regr_accum('{4,140,2900,1290,83075,15050}'::float8[], 200, 100); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (10,20,30); +-- SELECT count(*), sum(x), regr_sxx(y,x), sum(y),regr_syy(y,x), regr_sxy(y,x) +-- FROM regr_test WHERE x IN (80,100); +-- SELECT float8_combine('{3,60,200}'::float8[],ELECT CAST(udf(covar_pop(b, udf(a))) AS '{0,0,0}'::float8[]); +-- SELECT float8_combine('{0,0,0}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_combine('{3,60,200}'::float8[], '{2,180,200}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{0,0,0,0,0,0}'::float8[]); +-- SELECT float8_regr_combine('{0,0,0,0,0,0}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- SELECT float8_regr_combine('{3,60,200,750,20000,2000}'::float8[], +-- '{2,180,200,740,57800,-3400}'::float8[]); +-- DROP TABLE regr_test; + + +-- test count, distinct +SELECT count(udf(four)) AS cnt_1000 FROM onek; +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek; + +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten; + +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten; + +-- user-defined aggregates +-- SELECT newavg(four) AS avg_1 FROM onek; +-- SELECT newsum(four) AS sum_1500 FROM onek; +-- SELECT newcnt(four) AS cnt_1000 FROM onek; +-- SELECT newcnt(*) AS cnt_1000 FROM onek; +-- SELECT oldcnt(*) AS cnt_1000 FROM onek; +-- SELECT sum2(q1,q2) FROM int8_tbl; + +-- test for outer-level aggregates + +-- this should work +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four); + +-- this should fail because subquery has an agg of its own in WHERE +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)); + +-- [SPARK-27769] Test handling of sublinks within outer-level aggregates. +-- Per bug report from Daniel Grace. +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql new file mode 100644 index 0000000000000..5636537398a86 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-aggregates_part2.sql @@ -0,0 +1,230 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- AGGREGATES [Part 2] +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/aggregates.sql#L145-L350 +-- +-- This test file was converted from pgSQL/aggregates_part2.sql. + +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1); + +-- Test handling of Params within aggregate arguments in hashed aggregation. +-- Per bug report from Jeevan Chalke. +-- [SPARK-27877] Implement SQL-standard LATERAL subqueries +-- explain (verbose, costs off) +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; +-- select s1, s2, sm +-- from generate_series(1, 3) s1, +-- lateral (select s2, sum(s1 + s2) sm +-- from generate_series(1, 3) s2 group by s2) ss +-- order by 1, 2; + +-- [SPARK-27878] Support ARRAY(sub-SELECT) expressions +-- explain (verbose, costs off) +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; +-- select array(select sum(x+y) s +-- from generate_series(1,3) y group by y order by s) +-- from generate_series(1,3) x; + +-- [SPARK-27879] Implement bitwise integer aggregates(BIT_AND and BIT_OR) +-- +-- test for bitwise integer aggregates +-- +-- CREATE TEMPORARY TABLE bitwise_test( +-- i2 INT2, +-- i4 INT4, +-- i8 INT8, +-- i INTEGER, +-- x INT2, +-- y BIT(4) +-- ); + +-- empty case +-- SELECT +-- BIT_AND(i2) AS "?", +-- BIT_OR(i4) AS "?" +-- FROM bitwise_test; + +-- COPY bitwise_test FROM STDIN NULL 'null'; +-- 1 1 1 1 1 B0101 +-- 3 3 3 null 2 B0100 +-- 7 7 7 3 4 B1100 +-- \. + +-- SELECT +-- BIT_AND(i2) AS "1", +-- BIT_AND(i4) AS "1", +-- BIT_AND(i8) AS "1", +-- BIT_AND(i) AS "?", +-- BIT_AND(x) AS "0", +-- BIT_AND(y) AS "0100", +-- +-- BIT_OR(i2) AS "7", +-- BIT_OR(i4) AS "7", +-- BIT_OR(i8) AS "7", +-- BIT_OR(i) AS "?", +-- BIT_OR(x) AS "7", +-- BIT_OR(y) AS "1101" +-- FROM bitwise_test; + +-- +-- test boolean aggregates +-- +-- first test all possible transition and final states + +-- The result is inconsistent with PostgreSQL because our AND does not have strict mode +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t`; + +-- The result is inconsistent with PostgreSQL because our OR does not have strict mode +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t`; + +-- [SPARK-27880] Implement boolean aggregates(BOOL_AND, BOOL_OR and EVERY) +-- CREATE TEMPORARY TABLE bool_test( +-- b1 BOOL, +-- b2 BOOL, +-- b3 BOOL, +-- b4 BOOL); + +-- empty case +-- SELECT +-- BOOL_AND(b1) AS "n", +-- BOOL_OR(b3) AS "n" +-- FROM bool_test; + +-- COPY bool_test FROM STDIN NULL 'null'; +-- TRUE null FALSE null +-- FALSE TRUE null null +-- null TRUE FALSE null +-- \. + +-- SELECT +-- BOOL_AND(b1) AS "f", +-- BOOL_AND(b2) AS "t", +-- BOOL_AND(b3) AS "f", +-- BOOL_AND(b4) AS "n", +-- BOOL_AND(NOT b2) AS "f", +-- BOOL_AND(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- EVERY(b1) AS "f", +-- EVERY(b2) AS "t", +-- EVERY(b3) AS "f", +-- EVERY(b4) AS "n", +-- EVERY(NOT b2) AS "f", +-- EVERY(NOT b3) AS "t" +-- FROM bool_test; + +-- SELECT +-- BOOL_OR(b1) AS "t", +-- BOOL_OR(b2) AS "t", +-- BOOL_OR(b3) AS "f", +-- BOOL_OR(b4) AS "n", +-- BOOL_OR(NOT b2) AS "f", +-- BOOL_OR(NOT b3) AS "t" +-- FROM bool_test; + +-- +-- Test cases that should be optimized into indexscans instead of +-- the generic aggregate implementation. +-- + +-- Basic cases +-- explain +-- select min(unique1) from tenk1; +select min(udf(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1; +select udf(max(unique1)) from tenk1; +-- explain +-- select max(unique1) from tenk1 where unique1 < 42; +select max(unique1) from tenk1 where udf(unique1) < 42; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42; +select max(unique1) from tenk1 where unique1 > udf(42); + +-- the planner may choose a generic aggregate here if parallel query is +-- enabled, since that plan will be parallel safe and the "optimized" +-- plan, which has almost identical cost, will not be. we want to test +-- the optimized plan, so temporarily disable parallel query. +-- begin; +-- set local max_parallel_workers_per_gather = 0; +-- explain +-- select max(unique1) from tenk1 where unique1 > 42000; +select max(unique1) from tenk1 where udf(unique1) > 42000; +-- rollback; + +-- multi-column index (uses tenk1_thous_tenthous) +-- explain +-- select max(tenthous) from tenk1 where thousand = 33; +select max(tenthous) from tenk1 where udf(thousand) = 33; +-- explain +-- select min(tenthous) from tenk1 where thousand = 33; +select min(tenthous) from tenk1 where udf(thousand) = 33; + +-- [SPARK-17348] Correlated column is not allowed in a non-equality predicate +-- check parameter propagation into an indexscan subquery +-- explain +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; +-- select f1, (select min(unique1) from tenk1 where unique1 > f1) AS gt +-- from int4_tbl; + +-- check some cases that were handled incorrectly in 8.3.0 +-- explain +-- select distinct max(unique2) from tenk1; +select distinct max(udf(unique2)) from tenk1; +-- explain +-- select max(unique2) from tenk1 order by 1; +select max(unique2) from tenk1 order by udf(1); +-- explain +-- select max(unique2) from tenk1 order by max(unique2); +select max(unique2) from tenk1 order by max(udf(unique2)); +-- explain +-- select max(unique2) from tenk1 order by max(unique2)+1; +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1; +-- explain +-- select max(unique2), generate_series(1,3) as g from tenk1 order by g desc; +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc; + +-- interesting corner case: constant gets optimized into a seqscan +-- explain +-- select max(100) from tenk1; +select udf(max(100)) from tenk1; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql new file mode 100644 index 0000000000000..1865ee94ec1f9 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/pgSQL/udf-case.sql @@ -0,0 +1,265 @@ +-- +-- Portions Copyright (c) 1996-2019, PostgreSQL Global Development Group +-- +-- +-- CASE +-- https://github.com/postgres/postgres/blob/REL_12_BETA2/src/test/regress/sql/case.sql +-- Test the CASE statement +-- +-- This test file was converted from pgSQL/case.sql. + +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet; + +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet; + +INSERT INTO CASE_TBL VALUES (1, 10.1); +INSERT INTO CASE_TBL VALUES (2, 20.2); +INSERT INTO CASE_TBL VALUES (3, -30.3); +INSERT INTO CASE_TBL VALUES (4, NULL); + +INSERT INTO CASE2_TBL VALUES (1, -1); +INSERT INTO CASE2_TBL VALUES (2, -2); +INSERT INTO CASE2_TBL VALUES (3, -3); +INSERT INTO CASE2_TBL VALUES (2, -4); +INSERT INTO CASE2_TBL VALUES (1, NULL); +INSERT INTO CASE2_TBL VALUES (NULL, -6); + +-- +-- Simplest examples without tables +-- + +SELECT '3' AS `One`, + CASE + WHEN udf(1 < 2) THEN 3 + END AS `Simple WHEN`; + +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN udf(3) + END AS `Simple default`; + +SELECT '3' AS `One`, + CASE + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) + END AS `Simple ELSE`; + +SELECT udf('4') AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default`; + +SELECT udf('6') AS `One`, + CASE + WHEN udf(1 > 2) THEN 3 + WHEN udf(4) < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default`; + +SELECT '7' AS `None`, + CASE WHEN rand() < udf(0) THEN 1 + END AS `NULL on no matches`; + +-- Constant-expression folding shouldn't evaluate unreachable subexpressions +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END; +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END; + +-- [SPARK-27923] PostgreSQL throws an exception but Spark SQL is NULL +-- However we do not currently suppress folding of potentially +-- reachable subexpressions +SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl; + +-- Test for cases involving untyped literals in test expression +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END; + +-- +-- Examples of targets involving tables +-- + +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL; + +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL; + +-- +-- Examples of qualifications involving tables +-- + +-- +-- NULLIF() and COALESCE() +-- Shorthand forms for typical CASE constructs +-- defined in the SQL standard. +-- + +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4; + +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2; + +SELECT udf(COALESCE(a.f, b.i, b.j)) + FROM CASE_TBL a, CASE2_TBL b; + +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2; + +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b; + +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(f,b.i) = 2); + +-- We don't support update now. +-- +-- Examples of updates involving tables +-- + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 3 THEN (- i) +-- ELSE (2 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN i >= 2 THEN (2 * i) +-- ELSE (3 * i) END; + +-- SELECT * FROM CASE_TBL; + +-- UPDATE CASE_TBL +-- SET i = CASE WHEN b.i >= 2 THEN (2 * j) +-- ELSE (3 * j) END +-- FROM CASE2_TBL b +-- WHERE j = -CASE_TBL.i; + +-- SELECT * FROM CASE_TBL; + +-- +-- Nested CASE expressions +-- + +-- This test exercises a bug caused by aliasing econtext->caseValue_isNull +-- with the isNull argument of the inner CASE's CaseExpr evaluation. After +-- evaluating the vol(null) expression in the inner CASE's second WHEN-clause, +-- the isNull flag for the case test value incorrectly became true, causing +-- the third WHEN-clause not to match. The volatile function calls are needed +-- to prevent constant-folding in the planner, which would hide the bug. + +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE FUNCTION vol(text) returns text as +-- 'begin return $1; end' language plpgsql volatile; + +SELECT CASE + (CASE vol('bar') + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col; + +-- We don't support the features below: +-- 1. CREATE DOMAIN ... +-- 2. CREATE OPERATOR ... +-- 3. CREATE TYPE ... + +-- In this case, we can't inline the SQL function without confusing things. +-- CREATE DOMAIN foodomain AS text; + +-- CREATE FUNCTION volfoo(text) returns foodomain as +-- 'begin return $1::foodomain; end' language plpgsql volatile; + +-- CREATE FUNCTION inline_eq(foodomain, foodomain) returns boolean as +-- 'SELECT CASE $2::text WHEN $1::text THEN true ELSE false END' language sql; + +-- CREATE OPERATOR = (procedure = inline_eq, +-- leftarg = foodomain, rightarg = foodomain); + +-- SELECT CASE volfoo('bar') WHEN 'foo'::foodomain THEN 'is foo' ELSE 'is not foo' END; + +-- ROLLBACK; + +-- Test multiple evaluation of a CASE arg that is a read/write object (#14472) +-- Wrap this in a single transaction so the transient '=' operator doesn't +-- cause problems in concurrent sessions +-- BEGIN; + +-- CREATE DOMAIN arrdomain AS int[]; + +-- CREATE FUNCTION make_ad(int,int) returns arrdomain as +-- 'declare x arrdomain; +-- begin +-- x := array[$1,$2]; +-- return x; +-- end' language plpgsql volatile; + +-- CREATE FUNCTION ad_eq(arrdomain, arrdomain) returns boolean as +-- 'begin return array_eq($1, $2); end' language plpgsql; + +-- CREATE OPERATOR = (procedure = ad_eq, +-- leftarg = arrdomain, rightarg = arrdomain); + +-- SELECT CASE make_ad(1,2) +-- WHEN array[2,4]::arrdomain THEN 'wrong' +-- WHEN array[2,5]::arrdomain THEN 'still wrong' +-- WHEN array[1,2]::arrdomain THEN 'right' +-- END; + +-- ROLLBACK; + +-- Test interaction of CASE with ArrayCoerceExpr (bug #15471) +-- BEGIN; + +-- CREATE TYPE casetestenum AS ENUM ('e', 'f', 'g'); + +-- SELECT +-- CASE 'foo'::text +-- WHEN 'foo' THEN ARRAY['a', 'b', 'c', 'd'] || enum_range(NULL::casetestenum)::text[] +-- ELSE ARRAY['x', 'y'] +-- END; + +-- ROLLBACK; + +-- +-- Clean up +-- + +DROP TABLE CASE_TBL; +DROP TABLE CASE2_TBL; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql new file mode 100644 index 0000000000000..6ab207b93f561 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-count.sql @@ -0,0 +1,28 @@ +-- This test file was converted from count.sql +-- Test data. +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b); + +-- count with single expression +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData; + +-- distinct count with single expression +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData; + +-- count with multiple expressions +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData; + +-- distinct count with multiple expressions +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) +FROM testData; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql new file mode 100644 index 0000000000000..9aa6de7e92b8e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-cross-join.sql @@ -0,0 +1,37 @@ +-- Cross join detection and error checking is done in JoinSuite since explain output is +-- used in the error message and the ids are not stable. Only positive cases are checked here. +-- This test file was converted from cross-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + +-- Cross joins with and without predicates +SELECT * FROM nt1 cross join nt2; +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k); +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)); +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22"; + +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b; + +-- Join reordering +create temporary view A(a, va) as select * from nt1; +create temporary view B(b, vb) as select * from nt1; +create temporary view C(c, vc) as select * from nt1; +create temporary view D(d, vd) as select * from nt1; + +-- Allowed since cross join with C is explicit +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)); +-- Cross joins with non-equal predicates +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql new file mode 100644 index 0000000000000..71e05e67698c8 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from except-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1); +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v); +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v); + +-- Basic EXCEPT ALL +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- MINUS ALL (synonym for EXCEPT) +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2; + +-- EXCEPT ALL same table in both branches +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL; + +-- Empty left relation +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2; + +-- Empty right relation +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)); + +-- Type Coerced ExceptAll +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1); + +-- Basic +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4; + +-- Basic +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3; + +-- EXCEPT ALL + INTERSECT +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- EXCEPT ALL + EXCEPT +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Mismatch on number of columns across both branches +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4; + +-- Chain of set operations +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4; + +-- Using MINUS ALL +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4; + +-- Chain of set operations +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4; + +-- Join under except all. Should produce empty resultset since both left and right sets +-- are same. +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)); + +-- Join under except all (2) +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)); + +-- Group by under ExceptAll +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; +DROP VIEW IF EXISTS tab3; +DROP VIEW IF EXISTS tab4; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql new file mode 100644 index 0000000000000..17e24880ea1bd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-except.sql @@ -0,0 +1,59 @@ +-- This test file was converted from except.sql. +-- Tests different scenarios of except operation +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v); + +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v); + + +-- Except operation that will be replaced by left anti join +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2; + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2); + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3); + + +-- Except operation that will be replaced by Filter: SPARK-22181 +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one'); + + +-- Except operation that will be replaced by left anti join +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1; + + +-- Except operation that will be replaced by left anti join +SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one'); + + +-- Except operation that will be replaced by left anti join +--- [SPARK-28441] udf(max(udf(column))) throws java.lang.UnsupportedOperationException: Cannot evaluate expression: udf(null) +--- SELECT t1.k +--- FROM t1 +--- WHERE t1.v <= (SELECT udf(max(udf(t2.v))) +--- FROM t2 +--- WHERE udf(t2.k) = udf(t1.k)) +--- MINUS +--- SELECT t1.k +--- FROM t1 +--- WHERE udf(t1.v) >= (SELECT min(udf(t2.v)) +--- FROM t2 +--- WHERE t2.k = t1.k); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql new file mode 100644 index 0000000000000..618c8df977476 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-group-analytics.sql @@ -0,0 +1,64 @@ +-- This test file was converted from group-analytics.sql. +-- TODO: UDF should be inserted and tested at GROUP BY clause after SPARK-28445 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b); + +-- CUBE on overlapping columns +SELECT a + b, b, udf(SUM(a - b)) FROM testData GROUP BY a + b, b WITH CUBE; + +SELECT a, udf(b), SUM(b) FROM testData GROUP BY a, b WITH CUBE; + +-- ROLLUP on overlapping columns +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP; + +SELECT a, b, udf(SUM(b)) FROM testData GROUP BY a, b WITH ROLLUP; + +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings); + +-- ROLLUP +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year; + +-- CUBE +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year); + +-- GROUPING SETS +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year); +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course); +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year); + +-- GROUPING SETS with aggregate functions containing groupBy columns +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum); +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum; + +-- GROUPING/GROUPING_ID +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year); +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, year; +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY course, year; +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year); + +-- GROUPING/GROUPING_ID in having clause +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year); +SELECT course, udf(year) FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0; +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0; +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0; + +-- GROUPING/GROUPING_ID in orderBy clause +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year); +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year); +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course); +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course); +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year; + +-- Aliases in SELECT could be used in ROLLUP/CUBE/GROUPING SETS +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2); +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b); +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql new file mode 100644 index 0000000000000..ff8573ad7e562 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-having.sql @@ -0,0 +1,20 @@ +-- This test file was converted from having.sql. + +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v); + +-- having clause +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2; + +-- having condition contains grouping column +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2); + +-- SPARK-11032: resolve having correctly +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0); + +-- SPARK-20329: make sure we handle timezones correctly +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql new file mode 100644 index 0000000000000..179e528d74ca7 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inline-table.sql @@ -0,0 +1,54 @@ +-- This test file was converted from inline-table.sql. +-- [SPARK-28291] UDFs cannot be evaluated within inline table definition +-- TODO: We should add UDFs in VALUES clause when [SPARK-28291] is resolved. + +-- single row, without table and column alias +select udf(col1), udf(col2) from values ("one", 1); + +-- single row, without column alias +select udf(col1), udf(udf(col2)) from values ("one", 1) as data; + +-- single row +select udf(a), b from values ("one", 1) as data(a, b); + +-- single column multiple rows +select udf(a) from values 1, 2, 3 as data(a); + +-- three rows +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b); + +-- null type +select udf(a), b from values ("one", null), ("two", null) as data(a, b); + +-- int and long coercion +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b); + +-- foldable expressions +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b); + +-- complex types +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b); + +-- decimal and double coercion +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b); + +-- error reporting: nondeterministic function rand +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b); + +-- error reporting: different number of columns +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b); + +-- error reporting: types that are incompatible +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b); + +-- error reporting: number aliases different from number data values +select udf(a), udf(b) from values ("one"), ("two") as data(a, b); + +-- error reporting: unresolved expression +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b); + +-- error reporting: aggregate expression +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b); + +-- string to timestamp +select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inner-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inner-join.sql new file mode 100644 index 0000000000000..8bd61b804ab66 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-inner-join.sql @@ -0,0 +1,17 @@ +-- This test file was converted from inner-join.sql. +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW ta AS +SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2; + +CREATE TEMPORARY VIEW tb AS +SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4; + +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql new file mode 100644 index 0000000000000..028d4c7695995 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-intersect-all.sql @@ -0,0 +1,162 @@ +-- This test file was converted from intersect-all.sql. + +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v); +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v); + +-- Basic INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2; + +-- INTERSECT ALL same table in both branches +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1; + +-- Empty left relation +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Empty right relation +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3; + +-- Type Coerced INTERSECT ALL +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT); + +-- Error as types of two side are not compatible +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2); + +-- Mismatch on number of columns across both branches +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Basic +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2; + +-- Chain of different `set operations +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +; + +-- Chain of different `set operations +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +; + +-- test use parenthesis to control order of evaluation +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +; + +-- Join under intersect all +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))); + +-- Join under intersect all (2) +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)); + +-- Group by under intersect all +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k; + +-- Test pre spark2.4 behaviour of set operation precedence +-- All the set operators are given equal precedence and are evaluated +-- from left to right as they appear in the query. + +-- Set the property +SET spark.sql.legacy.setopsPrecedence.enabled= true; + +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2; + +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2; + +-- Restore the property +SET spark.sql.legacy.setopsPrecedence.enabled = false; + +-- Clean-up +DROP VIEW IF EXISTS tab1; +DROP VIEW IF EXISTS tab2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql new file mode 100644 index 0000000000000..47fb70d02394b --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-join-empty-relation.sql @@ -0,0 +1,35 @@ +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- This test file was converted from join-empty-relation.sql. + +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a); +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a); + +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false; + +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))); +SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)); +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))); +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)); + +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)); +SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))); +SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))); +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)); + +SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))); +SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)); +SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))); +SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); +SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql new file mode 100644 index 0000000000000..e5eb812d69a1c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-natural-join.sql @@ -0,0 +1,27 @@ +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- This test file was converted from natural-join.sql. + +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1); + +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2); + + +SELECT * FROM nt1 natural join nt2 where udf(k) = "one"; + +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2; + +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2; + +SELECT udf(count(*)) FROM nt1 natural full outer join nt2; diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql new file mode 100644 index 0000000000000..4eb0805c9cc67 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-outer-join.sql @@ -0,0 +1,45 @@ +-- This test file was converted from outer-join.sql. +-- List of configuration the test suite is run against: +--SET spark.sql.autoBroadcastJoinThreshold=10485760 +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=true +--SET spark.sql.autoBroadcastJoinThreshold=-1,spark.sql.join.preferSortMergeJoin=false + +-- SPARK-17099: Incorrect result when HAVING clause is added to group by query +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1); + +SELECT + (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), + (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON udf(t2.int_col0) = udf(t1.int_col1) +GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) + > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2); + + +-- SPARK-17120: Analyzer incorrectly optimizes plan to empty LocalRelation +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1); + +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1); + +-- Set the cross join enabled flag for the LEFT JOIN test since there's no join condition. +-- Ultimately the join should be optimized away. +set spark.sql.crossJoin.enabled = true; +SELECT * +FROM ( +SELECT + udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (udf(t.int_col)) is not null; +set spark.sql.crossJoin.enabled = false; + + diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql new file mode 100644 index 0000000000000..93937930de7f5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-pivot.sql @@ -0,0 +1,307 @@ +-- This test file was converted from pivot.sql. + +-- Note some test cases have been commented as the current integrated UDFs cannot handle complex types + +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings); + +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s); + +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s); + +-- pivot courses +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot years with no subquery +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot courses with multiple aggregations +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with no group by column and with multiple aggregations on different columns +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple group by columns +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +); + +-- pivot on join query with multiple aggregations on different columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on join query with multiple columns in one aggregation +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +); + +-- pivot with aliases and projection +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +); + +-- pivot with projection and value aliases +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +); + +-- pivot years with non-aggregate function +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with one of the expressions as non-aggregate function +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +); + +-- pivot with unresolvable columns +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +); + +-- pivot with complex aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +); + +-- pivot with invalid arguments in aggregate expressions +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +); + +-- pivot on multiple pivot columns +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +); + +-- pivot on multiple pivot columns with aliased values +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +); + +-- pivot on multiple pivot columns with values of wrong data types +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +); + +-- pivot with unresolvable values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +); + +-- pivot with non-literal values +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on join query with columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(min(a)) +-- FOR course IN ('dotNET', 'Java') +--); + +-- Complex type is not supported in the current UDF. Skipped for now. +-- pivot on multiple pivot columns with agg columns of complex data types +-- SELECT * FROM ( +-- SELECT course, year, y, a +-- FROM courseSales +-- JOIN yearsWithComplexTypes ON year = y +--) +--PIVOT ( +-- udf(max(a)) +-- FOR (y, course) IN ((2012, 'dotNET'), (2013, 'Java')) +--); + +-- pivot on pivot column of array type +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +); + +-- pivot on multiple pivot columns containing array type +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +); + +-- pivot on pivot column of struct type +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +); + +-- pivot on multiple pivot columns containing struct type +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +); + +-- pivot on pivot column of map type +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +); + +-- pivot on multiple pivot columns containing map type +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +); + +-- grouping columns output in the same order as input +-- correctly handle pivot columns with different cases +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql new file mode 100644 index 0000000000000..9cd15369bb164 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-special-values.sql @@ -0,0 +1,8 @@ +-- This file tests special values such as NaN, Infinity and NULL. + +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x); +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x); +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x); diff --git a/sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql new file mode 100644 index 0000000000000..c22d552a0ebe6 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/inputs/udf/udf-udaf.sql @@ -0,0 +1,18 @@ + -- This test file was converted from udaf.sql. + +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1); + +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg'; + +SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1; + +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1; + +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf'; + +SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1; + +DROP FUNCTION myDoubleAvg; +DROP FUNCTION udaf1; \ No newline at end of file diff --git a/sql/core/src/test/resources/sql-tests/inputs/window.sql b/sql/core/src/test/resources/sql-tests/inputs/window.sql index cda4db4b449fe..faab4c61c8640 100644 --- a/sql/core/src/test/resources/sql-tests/inputs/window.sql +++ b/sql/core/src/test/resources/sql-tests/inputs/window.sql @@ -109,3 +109,9 @@ last_value(false, false) OVER w AS last_value_contain_null FROM testData WINDOW w AS () ORDER BY cate, val; + +-- parentheses around window reference +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val); diff --git a/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out new file mode 100644 index 0000000000000..1f8b5b6ebee79 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/ansi/interval.sql.out @@ -0,0 +1,439 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 35 + + +-- !query 0 +SET spark.sql.parser.ansi.enabled=true +-- !query 0 schema +struct +-- !query 0 output +spark.sql.parser.ansi.enabled true + + +-- !query 1 +select + '1' second, + 2 seconds, + '1' minute, + 2 minutes, + '1' hour, + 2 hours, + '1' day, + 2 days, + '1' month, + 2 months, + '1' year, + 2 years +-- !query 1 schema +struct +-- !query 1 output +interval 1 seconds interval 2 seconds interval 1 minutes interval 2 minutes interval 1 hours interval 2 hours interval 1 days interval 2 days interval 1 months interval 2 months interval 1 years interval 2 years + + +-- !query 2 +select + interval '10-11' year to month, + interval '10' year, + interval '11' month +-- !query 2 schema +struct +-- !query 2 output +interval 10 years 11 months interval 10 years interval 11 months + + +-- !query 3 +select + '10-11' year to month, + '10' year, + '11' month +-- !query 3 schema +struct +-- !query 3 output +interval 10 years 11 months interval 10 years interval 11 months + + +-- !query 4 +select + interval '10 9:8:7.987654321' day to second, + interval '10' day, + interval '11' hour, + interval '12' minute, + interval '13' second, + interval '13.123456789' second +-- !query 4 schema +struct +-- !query 4 output +interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds + + +-- !query 5 +select + '10 9:8:7.987654321' day to second, + '10' day, + '11' hour, + '12' minute, + '13' second, + '13.123456789' second +-- !query 5 schema +struct +-- !query 5 output +interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 987 milliseconds 654 microseconds interval 1 weeks 3 days interval 11 hours interval 12 minutes interval 13 seconds interval 13 seconds 123 milliseconds 456 microseconds + + +-- !query 6 +select map(1, interval 1 day, 2, interval 3 week) +-- !query 6 schema +struct> +-- !query 6 output +{1:interval 1 days,2:interval 3 weeks} + + +-- !query 7 +select map(1, 1 day, 2, 3 week) +-- !query 7 schema +struct> +-- !query 7 output +{1:interval 1 days,2:interval 3 weeks} + + +-- !query 8 +create temporary view interval_arithmetic as + select CAST(dateval AS date), CAST(tsval AS timestamp) from values + ('2012-01-01', '2012-01-01') + as interval_arithmetic(dateval, tsval) +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +select + dateval, + dateval - interval '2-2' year to month, + dateval - interval '-2-2' year to month, + dateval + interval '2-2' year to month, + dateval + interval '-2-2' year to month, + - interval '2-2' year to month + dateval, + interval '2-2' year to month + dateval +from interval_arithmetic +-- !query 9 schema +struct +-- !query 9 output +2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 + + +-- !query 10 +select + dateval, + dateval - '2-2' year to month, + dateval - '-2-2' year to month, + dateval + '2-2' year to month, + dateval + '-2-2' year to month, + - '2-2' year to month + dateval, + '2-2' year to month + dateval +from interval_arithmetic +-- !query 10 schema +struct +-- !query 10 output +2012-01-01 2009-11-01 2014-03-01 2014-03-01 2009-11-01 2009-11-01 2014-03-01 + + +-- !query 11 +select + tsval, + tsval - interval '2-2' year to month, + tsval - interval '-2-2' year to month, + tsval + interval '2-2' year to month, + tsval + interval '-2-2' year to month, + - interval '2-2' year to month + tsval, + interval '2-2' year to month + tsval +from interval_arithmetic +-- !query 11 schema +struct +-- !query 11 output +2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 + + +-- !query 12 +select + tsval, + tsval - '2-2' year to month, + tsval - '-2-2' year to month, + tsval + '2-2' year to month, + tsval + '-2-2' year to month, + - '2-2' year to month + tsval, + '2-2' year to month + tsval +from interval_arithmetic +-- !query 12 schema +struct +-- !query 12 output +2012-01-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 2014-03-01 00:00:00 2009-11-01 00:00:00 2009-11-01 00:00:00 2014-03-01 00:00:00 + + +-- !query 13 +select + interval '2-2' year to month + interval '3-3' year to month, + interval '2-2' year to month - interval '3-3' year to month +from interval_arithmetic +-- !query 13 schema +struct<(interval 2 years 2 months + interval 3 years 3 months):calendarinterval,(interval 2 years 2 months - interval 3 years 3 months):calendarinterval> +-- !query 13 output +interval 5 years 5 months interval -1 years -1 months + + +-- !query 14 +select + '2-2' year to month + '3-3' year to month, + '2-2' year to month - '3-3' year to month +from interval_arithmetic +-- !query 14 schema +struct<(interval 2 years 2 months + interval 3 years 3 months):calendarinterval,(interval 2 years 2 months - interval 3 years 3 months):calendarinterval> +-- !query 14 output +interval 5 years 5 months interval -1 years -1 months + + +-- !query 15 +select + dateval, + dateval - interval '99 11:22:33.123456789' day to second, + dateval - interval '-99 11:22:33.123456789' day to second, + dateval + interval '99 11:22:33.123456789' day to second, + dateval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + dateval, + interval '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic +-- !query 15 schema +struct +-- !query 15 output +2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 + + +-- !query 16 +select + dateval, + dateval - '99 11:22:33.123456789' day to second, + dateval - '-99 11:22:33.123456789' day to second, + dateval + '99 11:22:33.123456789' day to second, + dateval + '-99 11:22:33.123456789' day to second, + - '99 11:22:33.123456789' day to second + dateval, + '99 11:22:33.123456789' day to second + dateval +from interval_arithmetic +-- !query 16 schema +struct +-- !query 16 output +2012-01-01 2011-09-23 2012-04-09 2012-04-09 2011-09-23 2011-09-23 2012-04-09 + + +-- !query 17 +select + tsval, + tsval - interval '99 11:22:33.123456789' day to second, + tsval - interval '-99 11:22:33.123456789' day to second, + tsval + interval '99 11:22:33.123456789' day to second, + tsval + interval '-99 11:22:33.123456789' day to second, + -interval '99 11:22:33.123456789' day to second + tsval, + interval '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic +-- !query 17 schema +struct +-- !query 17 output +2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 + + +-- !query 18 +select + tsval, + tsval - '99 11:22:33.123456789' day to second, + tsval - '-99 11:22:33.123456789' day to second, + tsval + '99 11:22:33.123456789' day to second, + tsval + '-99 11:22:33.123456789' day to second, + - '99 11:22:33.123456789' day to second + tsval, + '99 11:22:33.123456789' day to second + tsval +from interval_arithmetic +-- !query 18 schema +struct +-- !query 18 output +2012-01-01 00:00:00 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 2012-04-09 12:22:33.123456 2011-09-23 13:37:26.876544 2011-09-23 13:37:26.876544 2012-04-09 12:22:33.123456 + + +-- !query 19 +select + interval '99 11:22:33.123456789' day to second + interval '10 9:8:7.123456789' day to second, + interval '99 11:22:33.123456789' day to second - interval '10 9:8:7.123456789' day to second +from interval_arithmetic +-- !query 19 schema +struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval> +-- !query 19 output +interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds + + +-- !query 20 +select + '99 11:22:33.123456789' day to second + '10 9:8:7.123456789' day to second, + '99 11:22:33.123456789' day to second - '10 9:8:7.123456789' day to second +from interval_arithmetic +-- !query 20 schema +struct<(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds + interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval,(interval 14 weeks 1 days 11 hours 22 minutes 33 seconds 123 milliseconds 456 microseconds - interval 1 weeks 3 days 9 hours 8 minutes 7 seconds 123 milliseconds 456 microseconds):calendarinterval> +-- !query 20 output +interval 15 weeks 4 days 20 hours 30 minutes 40 seconds 246 milliseconds 912 microseconds interval 12 weeks 5 days 2 hours 14 minutes 26 seconds + + +-- !query 21 +select 30 day +-- !query 21 schema +struct +-- !query 21 output +interval 4 weeks 2 days + + +-- !query 22 +select 30 day day +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 14) + +== SQL == +select 30 day day +--------------^^^ + + +-- !query 23 +select 30 day day day +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 14) + +== SQL == +select 30 day day day +--------------^^^ + + +-- !query 24 +select date '2012-01-01' - 30 day +-- !query 24 schema +struct +-- !query 24 output +2011-12-02 + + +-- !query 25 +select date '2012-01-01' - 30 day day +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 34) + +== SQL == +select date '2012-01-01' - 30 day day +----------------------------------^^^ + + +-- !query 26 +select date '2012-01-01' - 30 day day day +-- !query 26 schema +struct<> +-- !query 26 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 34) + +== SQL == +select date '2012-01-01' - 30 day day day +----------------------------------^^^ + + +-- !query 27 +select date '2012-01-01' + '-30' day +-- !query 27 schema +struct +-- !query 27 output +2011-12-02 + + +-- !query 28 +select date '2012-01-01' + interval '-30' day +-- !query 28 schema +struct +-- !query 28 output +2011-12-02 + + +-- !query 29 +select date '2012-01-01' + interval (-30) day +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 42) + +== SQL == +select date '2012-01-01' + interval (-30) day +------------------------------------------^^^ + + +-- !query 30 +select date '2012-01-01' + (-30) day +-- !query 30 schema +struct<> +-- !query 30 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 33) + +== SQL == +select date '2012-01-01' + (-30) day +---------------------------------^^^ + + +-- !query 31 +create temporary view t as select * from values (1), (2) as t(a) +-- !query 31 schema +struct<> +-- !query 31 output + + + +-- !query 32 +select date '2012-01-01' + interval (a + 1) day from t +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 44) + +== SQL == +select date '2012-01-01' + interval (a + 1) day from t +--------------------------------------------^^^ + + +-- !query 33 +select date '2012-01-01' + (a + 1) day from t +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'day'(line 1, pos 35) + +== SQL == +select date '2012-01-01' + (a + 1) day from t +-----------------------------------^^^ + + +-- !query 34 +SET spark.sql.parser.ansi.enabled=false +-- !query 34 schema +struct +-- !query 34 output +spark.sql.parser.ansi.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/array.sql.out b/sql/core/src/test/resources/sql-tests/results/array.sql.out index 981b2504bcaad..5f5d988771847 100644 --- a/sql/core/src/test/resources/sql-tests/results/array.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/array.sql.out @@ -92,7 +92,7 @@ select array_contains(timestamp_array, timestamp '2016-11-15 20:54:00.000'), array_contains(timestamp_array, timestamp '2016-01-01 20:54:00.000') from primitive_arrays -- !query 6 schema -struct +struct -- !query 6 output true false true false true false true false true false true false true false true false true false true false diff --git a/sql/core/src/test/resources/sql-tests/results/cast.sql.out b/sql/core/src/test/resources/sql-tests/results/cast.sql.out index 9c5f4554d9fe9..21b18e9b0f84f 100644 --- a/sql/core/src/test/resources/sql-tests/results/cast.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cast.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 24 +-- Number of queries: 33 -- !query 0 @@ -179,20 +179,92 @@ NULL -- !query 22 -DESC FUNCTION boolean +SELECT HEX(CAST('abc' AS binary)) -- !query 22 schema -struct +struct -- !query 22 output +616263 + + +-- !query 23 +SELECT HEX(CAST(CAST(123 AS byte) AS binary)) +-- !query 23 schema +struct +-- !query 23 output +7B + + +-- !query 24 +SELECT HEX(CAST(CAST(-123 AS byte) AS binary)) +-- !query 24 schema +struct +-- !query 24 output +85 + + +-- !query 25 +SELECT HEX(CAST(123S AS binary)) +-- !query 25 schema +struct +-- !query 25 output +007B + + +-- !query 26 +SELECT HEX(CAST(-123S AS binary)) +-- !query 26 schema +struct +-- !query 26 output +FF85 + + +-- !query 27 +SELECT HEX(CAST(123 AS binary)) +-- !query 27 schema +struct +-- !query 27 output +0000007B + + +-- !query 28 +SELECT HEX(CAST(-123 AS binary)) +-- !query 28 schema +struct +-- !query 28 output +FFFFFF85 + + +-- !query 29 +SELECT HEX(CAST(123L AS binary)) +-- !query 29 schema +struct +-- !query 29 output +000000000000007B + + +-- !query 30 +SELECT HEX(CAST(-123L AS binary)) +-- !query 30 schema +struct +-- !query 30 output +FFFFFFFFFFFFFF85 + + +-- !query 31 +DESC FUNCTION boolean +-- !query 31 schema +struct +-- !query 31 output Class: org.apache.spark.sql.catalyst.expressions.Cast Function: boolean Usage: boolean(expr) - Casts the value `expr` to the target data type `boolean`. --- !query 23 +-- !query 32 DESC FUNCTION EXTENDED boolean --- !query 23 schema +-- !query 32 schema struct --- !query 23 output +-- !query 32 output Class: org.apache.spark.sql.catalyst.expressions.Cast Extended Usage: No example/argument for boolean. diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out index ff1ecbcc44c23..114617873af47 100644 --- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 33 +-- Number of queries: 34 -- !query 0 @@ -313,3 +313,11 @@ DROP TABLE partition_table struct<> -- !query 32 output + + +-- !query 33 +DROP VIEW global_temp.global_temp_view +-- !query 33 schema +struct<> +-- !query 33 output + diff --git a/sql/core/src/test/resources/sql-tests/results/count.sql.out b/sql/core/src/test/resources/sql-tests/results/count.sql.out new file mode 100644 index 0000000000000..b8a86d4c44594 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/count.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + count(*), count(1), count(null), count(a), count(b), count(a + b), count((a, b)) +FROM testData +-- !query 1 schema +struct +-- !query 1 output +7 7 0 5 5 4 7 + + +-- !query 2 +SELECT + count(DISTINCT 1), + count(DISTINCT null), + count(DISTINCT a), + count(DISTINCT b), + count(DISTINCT (a + b)), + count(DISTINCT (a, b)) +FROM testData +-- !query 2 schema +struct +-- !query 2 output +1 0 2 2 2 6 + + +-- !query 3 +SELECT count(a, b), count(b, a), count(testData.*) FROM testData +-- !query 3 schema +struct +-- !query 3 output +4 4 4 + + +-- !query 4 +SELECT + count(DISTINCT a, b), count(DISTINCT b, a), count(DISTINCT *), count(DISTINCT testData.*) +FROM testData +-- !query 4 schema +struct +-- !query 4 output +3 3 3 3 diff --git a/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out new file mode 100644 index 0000000000000..03d4bfffa8923 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/csv-functions.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +select from_csv('1, 3.14', 'a INT, f FLOAT') +-- !query 0 schema +struct> +-- !query 0 output +{"a":1,"f":3.14} + + +-- !query 1 +select from_csv('26/08/2015', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +-- !query 1 schema +struct> +-- !query 1 output +{"time":2015-08-26 00:00:00.0} + + +-- !query 2 +select from_csv('1', 1) +-- !query 2 schema +struct<> +-- !query 2 output +org.apache.spark.sql.AnalysisException +Schema should be specified in DDL format as a string literal or output of the schema_of_csv function instead of 1;; line 1 pos 7 + + +-- !query 3 +select from_csv('1', 'a InvalidType') +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException + +DataType invalidtype is not supported.(line 1, pos 2) + +== SQL == +a InvalidType +--^^^ +; line 1 pos 7 + + +-- !query 4 +select from_csv('1', 'a INT', named_struct('mode', 'PERMISSIVE')) +-- !query 4 schema +struct<> +-- !query 4 output +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 + + +-- !query 5 +select from_csv('1', 'a INT', map('mode', 1)) +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got map;; line 1 pos 7 + + +-- !query 6 +select from_csv() +-- !query 6 schema +struct<> +-- !query 6 output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function from_csv. Expected: one of 2 and 3; Found: 0; line 1 pos 7 + + +-- !query 7 +select from_csv('1,abc', schema_of_csv('1,abc')) +-- !query 7 schema +struct> +-- !query 7 output +{"_c0":1,"_c1":"abc"} + + +-- !query 8 +select schema_of_csv('1|abc', map('delimiter', '|')) +-- !query 8 schema +struct +-- !query 8 output +struct<_c0:int,_c1:string> + + +-- !query 9 +select schema_of_csv(null) +-- !query 9 schema +struct<> +-- !query 9 output +org.apache.spark.sql.AnalysisException +cannot resolve 'schema_of_csv(NULL)' due to data type mismatch: The input csv should be a string literal and not null; however, got NULL.; line 1 pos 7 + + +-- !query 10 +CREATE TEMPORARY VIEW csvTable(csvField, a) AS SELECT * FROM VALUES ('1,abc', 'a') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT schema_of_csv(csvField) FROM csvTable +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +cannot resolve 'schema_of_csv(csvtable.`csvField`)' due to data type mismatch: The input csv should be a string literal and not null; however, got csvtable.`csvField`.; line 1 pos 7 + + +-- !query 12 +DROP VIEW IF EXISTS csvTable +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select to_csv(named_struct('a', 1, 'b', 2)) +-- !query 13 schema +struct +-- !query 13 output +1,2 + + +-- !query 14 +select to_csv(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) +-- !query 14 schema +struct +-- !query 14 output +26/08/2015 + + +-- !query 15 +select to_csv(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 + + +-- !query 16 +select to_csv(named_struct('a', 1, 'b', 2), map('mode', 1)) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got map;; line 1 pos 7 diff --git a/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out new file mode 100644 index 0000000000000..5193e2536c0cc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/cte-legacy.sql.out @@ -0,0 +1,208 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +create temporary view t as select * from values 0, 1, 2 as t(id) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values 0, 1 as t(id) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SET spark.sql.legacy.ctePrecedence.enabled=true +-- !query 2 schema +struct +-- !query 2 output +spark.sql.legacy.ctePrecedence.enabled true + + +-- !query 3 +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query 3 schema +struct<1:int> +-- !query 3 output +1 + + +-- !query 4 +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query 4 schema +struct +-- !query 4 output +1 + + +-- !query 5 +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query 5 schema +struct +-- !query 5 output +1 + + +-- !query 6 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query 6 schema +struct<1:int> +-- !query 6 output +1 + + +-- !query 7 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query 7 schema +struct +-- !query 7 output +1 + + +-- !query 8 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query 8 schema +struct<2:int> +-- !query 8 output +2 + + +-- !query 9 +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query 9 schema +struct +-- !query 9 output +2 + + +-- !query 10 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 10 schema +struct +-- !query 10 output +2 + + +-- !query 11 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 11 schema +struct +-- !query 11 output +3 + + +-- !query 12 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query 12 schema +struct +-- !query 12 output +1 + + +-- !query 13 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 13 schema +struct +-- !query 13 output +1 + + +-- !query 14 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +DROP VIEW IF EXISTS t +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +DROP VIEW IF EXISTS t2 +-- !query 16 schema +struct<> +-- !query 16 output + diff --git a/sql/core/src/test/resources/sql-tests/results/cte.sql.out b/sql/core/src/test/resources/sql-tests/results/cte.sql.out index a446c2cd183da..b7dd76c725209 100644 --- a/sql/core/src/test/resources/sql-tests/results/cte.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/cte.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 9 +-- Number of queries: 27 -- !query 0 @@ -28,36 +28,46 @@ Table or view not found: s; line 1 pos 25 -- !query 3 -WITH t AS (SELECT 1 FROM t) SELECT * FROM t +WITH r AS (SELECT (SELECT * FROM r)) +SELECT * FROM r -- !query 3 schema -struct<1:int> +struct<> -- !query 3 output +org.apache.spark.sql.AnalysisException +Table or view not found: r; line 1 pos 33 + + +-- !query 4 +WITH t AS (SELECT 1 FROM t) SELECT * FROM t +-- !query 4 schema +struct<1:int> +-- !query 4 output 1 1 1 --- !query 4 +-- !query 5 WITH s1 AS (SELECT 1 FROM s2), s2 AS (SELECT 1 FROM s1) SELECT * FROM s1, s2 --- !query 4 schema +-- !query 5 schema struct<> --- !query 4 output +-- !query 5 output org.apache.spark.sql.AnalysisException Table or view not found: s2; line 1 pos 26 --- !query 5 +-- !query 6 WITH t1 AS (SELECT * FROM t2), t2 AS (SELECT 2 FROM t1) SELECT * FROM t1 cross join t2 --- !query 5 schema +-- !query 6 schema struct --- !query 5 output +-- !query 6 output 0 2 0 2 1 2 1 2 --- !query 6 +-- !query 7 WITH CTE1 AS ( SELECT b.id AS id FROM T2 a @@ -67,9 +77,9 @@ SELECT t1.id AS c1, t2.id AS c2 FROM CTE1 t1 CROSS JOIN CTE1 t2 --- !query 6 schema +-- !query 7 schema struct --- !query 6 output +-- !query 7 output 0 0 0 0 0 0 @@ -88,17 +98,246 @@ struct 1 1 --- !query 7 +-- !query 8 +WITH t(x) AS (SELECT 1) +SELECT * FROM t WHERE x = 1 +-- !query 8 schema +struct +-- !query 8 output +1 + + +-- !query 9 +WITH t(x, y) AS (SELECT 1, 2) +SELECT * FROM t WHERE x = 1 AND y = 2 +-- !query 9 schema +struct +-- !query 9 output +1 2 + + +-- !query 10 +WITH t(x, x) AS (SELECT 1, 2) +SELECT * FROM t +-- !query 10 schema +struct +-- !query 10 output +1 2 + + +-- !query 11 +WITH t() AS (SELECT 1) +SELECT * FROM t +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'WITH t()'(line 1, pos 7) + +== SQL == +WITH t() AS (SELECT 1) +-------^^^ +SELECT * FROM t + + +-- !query 12 +WITH + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.catalyst.parser.ParseException + +CTE definition can't have duplicate names: 't'.(line 1, pos 0) + +== SQL == +WITH +^^^ + t(x) AS (SELECT 1), + t(x) AS (SELECT 2) +SELECT * FROM t + + +-- !query 13 +WITH t as ( + WITH t2 AS (SELECT 1) + SELECT * FROM t2 +) +SELECT * FROM t +-- !query 13 schema +struct<1:int> +-- !query 13 output +1 + + +-- !query 14 +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 1) + SELECT * FROM t +) +-- !query 14 schema +struct +-- !query 14 output +1 + + +-- !query 15 +SELECT ( + WITH t AS (SELECT 1) + SELECT * FROM t +) +-- !query 15 schema +struct +-- !query 15 output +1 + + +-- !query 16 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +SELECT * FROM t2 +-- !query 16 schema +struct<2:int> +-- !query 16 output +2 + + +-- !query 17 +WITH + t(c) AS (SELECT 1), + t2 AS ( + SELECT ( + SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) + ) + ) +SELECT * FROM t2 +-- !query 17 schema +struct +-- !query 17 output +2 + + +-- !query 18 +WITH + t AS (SELECT 1), + t2 AS ( + WITH t AS (SELECT 2), + t2 AS ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) + SELECT * FROM t2 + ) +SELECT * FROM t2 +-- !query 18 schema +struct<3:int> +-- !query 18 output +3 + + +-- !query 19 +WITH t(c) AS (SELECT 1) +SELECT max(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t +) +-- !query 19 schema +struct +-- !query 19 output +2 + + +-- !query 20 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 20 schema +struct +-- !query 20 output +2 + + +-- !query 21 +WITH t(c) AS (SELECT 1) +SELECT sum(c) FROM ( + WITH t(c) AS (SELECT 2) + SELECT max(c) AS c FROM ( + WITH t(c) AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 21 schema +struct +-- !query 21 output +3 + + +-- !query 22 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t +) +-- !query 22 schema +struct +-- !query 22 output +2 + + +-- !query 23 +WITH t AS (SELECT 1) +SELECT ( + SELECT ( + WITH t AS (SELECT 2) + SELECT * FROM t + ) +) +-- !query 23 schema +struct +-- !query 23 output +2 + + +-- !query 24 +WITH t AS (SELECT 1) +SELECT ( + WITH t AS (SELECT 2) + SELECT ( + WITH t AS (SELECT 3) + SELECT * FROM t + ) +) +-- !query 24 schema +struct +-- !query 24 output +3 + + +-- !query 25 DROP VIEW IF EXISTS t --- !query 7 schema +-- !query 25 schema struct<> --- !query 7 output +-- !query 25 output --- !query 8 +-- !query 26 DROP VIEW IF EXISTS t2 --- !query 8 schema +-- !query 26 schema struct<> --- !query 8 output +-- !query 26 output 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 63aa00426ea32..178400e5706b8 100644 --- 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: 10 +-- Number of queries: 15 -- !query 0 @@ -89,3 +89,43 @@ select weekday('2007-02-03'), weekday('2009-07-30'), weekday('2017-05-27'), week struct -- !query 9 output 5 3 5 NULL 4 + + +-- !query 10 +select year('1500-01-01'), month('1500-01-01'), dayOfYear('1500-01-01') +-- !query 10 schema +struct +-- !query 10 output +1500 1 1 + + +-- !query 11 +select date '2001-09-28' + 7 +-- !query 11 schema +struct +-- !query 11 output +2001-10-05 + + +-- !query 12 +select 7 + date '2001-09-28' +-- !query 12 schema +struct +-- !query 12 output +2001-10-05 + + +-- !query 13 +select date '2001-10-01' - 7 +-- !query 13 schema +struct +-- !query 13 output +2001-09-24 + + +-- !query 14 +select date '2001-10-01' - date '2001-09-28' +-- !query 14 schema +struct +-- !query 14 output +3 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalArithmeticOperations.sql.out b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out similarity index 73% rename from sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalArithmeticOperations.sql.out rename to sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out index cbf44548b3cce..217233bfad378 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/decimalArithmeticOperations.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/decimalArithmeticOperations.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 40 +-- Number of queries: 54 -- !query 0 @@ -328,8 +328,131 @@ NULL -- !query 39 -drop table decimals_test +set spark.sql.decimalOperations.nullOnOverflow=false -- !query 39 schema -struct<> +struct -- !query 39 output +spark.sql.decimalOperations.nullOnOverflow false + + +-- !query 40 +select id, a*10, b/10 from decimals_test order by id +-- !query 40 schema +struct +-- !query 40 output +1 1000 99.9 +2 123451.23 1234.5123 +3 1.234567891011 123.41 +4 1234567891234567890 0.1123456789123456789 + + +-- !query 41 +select 10.3 * 3.0 +-- !query 41 schema +struct<(CAST(10.3 AS DECIMAL(3,1)) * CAST(3.0 AS DECIMAL(3,1))):decimal(6,2)> +-- !query 41 output +30.9 + + +-- !query 42 +select 10.3000 * 3.0 +-- !query 42 schema +struct<(CAST(10.3000 AS DECIMAL(6,4)) * CAST(3.0 AS DECIMAL(6,4))):decimal(9,5)> +-- !query 42 output +30.9 + + +-- !query 43 +select 10.30000 * 30.0 +-- !query 43 schema +struct<(CAST(10.30000 AS DECIMAL(7,5)) * CAST(30.0 AS DECIMAL(7,5))):decimal(11,6)> +-- !query 43 output +309 + + +-- !query 44 +select 10.300000000000000000 * 3.000000000000000000 +-- !query 44 schema +struct<(CAST(10.300000000000000000 AS DECIMAL(20,18)) * CAST(3.000000000000000000 AS DECIMAL(20,18))):decimal(38,36)> +-- !query 44 output +30.9 + + +-- !query 45 +select 10.300000000000000000 * 3.0000000000000000000 +-- !query 45 schema +struct<> +-- !query 45 output +java.lang.ArithmeticException +Decimal(expanded,30.900000000000000000000000000000000000,38,36}) cannot be represented as Decimal(38, 37). + + +-- !query 46 +select (5e36 + 0.1) + 5e36 +-- !query 46 schema +struct<> +-- !query 46 output +java.lang.ArithmeticException +Decimal(expanded,10000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). + + +-- !query 47 +select (-4e36 - 0.1) - 7e36 +-- !query 47 schema +struct<> +-- !query 47 output +java.lang.ArithmeticException +Decimal(expanded,-11000000000000000000000000000000000000.1,39,1}) cannot be represented as Decimal(38, 1). + + +-- !query 48 +select 12345678901234567890.0 * 12345678901234567890.0 +-- !query 48 schema +struct<> +-- !query 48 output +java.lang.ArithmeticException +Decimal(expanded,1.5241578753238836750190519987501905210E+38,38,-1}) cannot be represented as Decimal(38, 2). + + +-- !query 49 +select 1e35 / 0.1 +-- !query 49 schema +struct<> +-- !query 49 output +java.lang.ArithmeticException +Decimal(expanded,1000000000000000000000000000000000000,37,0}) cannot be represented as Decimal(38, 3). + + +-- !query 50 +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query 50 schema +struct<> +-- !query 50 output +java.lang.ArithmeticException +Decimal(expanded,138698367904130467.65432098851562262075,38,20}) cannot be represented as Decimal(38, 28). + + +-- !query 51 +select 123456789123456789.1234567890 * 1.123456789123456789 +-- !query 51 schema +struct<> +-- !query 51 output +java.lang.ArithmeticException +Decimal(expanded,138698367904130467.65432098851562262075,38,20}) cannot be represented as Decimal(38, 28). + + +-- !query 52 +select 12345678912345.123456789123 / 0.000000012345678 +-- !query 52 schema +struct<> +-- !query 52 output +java.lang.ArithmeticException +Decimal(expanded,1000000073899961059796.7258663315210392,38,16}) cannot be represented as Decimal(38, 18). + + +-- !query 53 +drop table decimals_test +-- !query 53 schema +struct<> +-- !query 53 output diff --git a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out index 8ba69c698b551..17dd317f63b70 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe-part-after-analyze.sql.out @@ -93,7 +93,7 @@ Partition Values [ds=2017-08-01, hr=10] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1121 bytes, 3 rows +Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t @@ -128,7 +128,7 @@ Partition Values [ds=2017-08-01, hr=10] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1121 bytes, 3 rows +Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t @@ -155,7 +155,7 @@ Partition Values [ds=2017-08-01, hr=11] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1098 bytes, 4 rows +Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t @@ -190,7 +190,7 @@ Partition Values [ds=2017-08-01, hr=10] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=10 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1121 bytes, 3 rows +Partition Statistics [not included in comparison] bytes, 3 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t @@ -217,7 +217,7 @@ Partition Values [ds=2017-08-01, hr=11] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-08-01/hr=11 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1098 bytes, 4 rows +Partition Statistics [not included in comparison] bytes, 4 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t @@ -244,7 +244,7 @@ Partition Values [ds=2017-09-01, hr=5] Location [not included in comparison]sql/core/spark-warehouse/t/ds=2017-09-01/hr=5 Created Time [not included in comparison] Last Access [not included in comparison] -Partition Statistics 1144 bytes, 2 rows +Partition Statistics [not included in comparison] bytes, 2 rows # Storage Information Location [not included in comparison]sql/core/spark-warehouse/t 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 new file mode 100644 index 0000000000000..e41534681dc91 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/describe-query.sql.out @@ -0,0 +1,199 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 19 + + +-- !query 0 +CREATE table desc_temp1 (key int COMMENT 'column_comment', val string) USING PARQUET +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE table desc_temp2 (key int, val string) USING PARQUET +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +DESC SELECT key, key + 1 as plusone FROM desc_temp1 +-- !query 2 schema +struct +-- !query 2 output +key int column_comment +plusone int + + +-- !query 3 +DESC QUERY SELECT * FROM desc_temp2 +-- !query 3 schema +struct +-- !query 3 output +key int +val string + + +-- !query 4 +DESC SELECT key, COUNT(*) as count FROM desc_temp1 group by key +-- !query 4 schema +struct +-- !query 4 output +key int column_comment +count bigint + + +-- !query 5 +DESC SELECT 10.00D as col1 +-- !query 5 schema +struct +-- !query 5 output +col1 double + + +-- !query 6 +DESC QUERY SELECT key FROM desc_temp1 UNION ALL select CAST(1 AS DOUBLE) +-- !query 6 schema +struct +-- !query 6 output +key double + + +-- !query 7 +DESC QUERY VALUES(1.00D, 'hello') as tab1(col1, col2) +-- !query 7 schema +struct +-- !query 7 output +col1 double +col2 string + + +-- !query 8 +DESC QUERY FROM desc_temp1 a SELECT * +-- !query 8 schema +struct +-- !query 8 output +key int column_comment +val string + + +-- !query 9 +DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query 9 schema +struct +-- !query 9 output +col1 string + + +-- !query 10 +DESCRIBE QUERY WITH s AS (SELECT * from desc_temp1) SELECT * FROM s +-- !query 10 schema +struct +-- !query 10 output +key int column_comment +val string + + +-- !query 11 +DESCRIBE SELECT * FROM (FROM desc_temp2 select * select *) +-- !query 11 schema +struct +-- !query 11 output +key int +val string + + +-- !query 12 +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) + +== SQL == +DESCRIBE INSERT INTO desc_temp1 values (1, 'val1') +---------------------^^^ + + +-- !query 13 +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input 'desc_temp1' expecting {, '.'}(line 1, pos 21) + +== SQL == +DESCRIBE INSERT INTO desc_temp1 SELECT * FROM desc_temp2 +---------------------^^^ + + +-- !query 14 +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * + insert into desc_temp2 select * +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.catalyst.parser.ParseException + +mismatched input 'insert' expecting {'MAP', 'REDUCE', 'SELECT'}(line 3, pos 5) + +== SQL == +DESCRIBE + FROM desc_temp1 a + insert into desc_temp1 select * +-----^^^ + insert into desc_temp2 select * + + +-- !query 15 +EXPLAIN DESC QUERY SELECT * FROM desc_temp2 WHERE key > 0 +-- !query 15 schema +struct +-- !query 15 output +== Physical Plan == +Execute DescribeQueryCommand + +- DescribeQueryCommand SELECT * FROM desc_temp2 WHERE key > 0 + + +-- !query 16 +EXPLAIN EXTENDED DESC WITH s AS (SELECT 'hello' as col1) SELECT * FROM s +-- !query 16 schema +struct +-- !query 16 output +== Parsed Logical Plan == +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Analyzed Logical Plan == +col_name: string, data_type: string, comment: string +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Optimized Logical Plan == +DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + +== Physical Plan == +Execute DescribeQueryCommand + +- DescribeQueryCommand WITH s AS (SELECT 'hello' as col1) SELECT * FROM s + + +-- !query 17 +DROP TABLE desc_temp1 +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +DROP TABLE desc_temp2 +-- !query 18 schema +struct<> +-- !query 18 output + diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out index 9c4b70d1b1ab7..46d9ec30a8a77 100644 --- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 36 +-- Number of queries: 41 -- !query 0 @@ -514,32 +514,92 @@ DESC PARTITION is not allowed on a view: v; -- !query 32 -DROP TABLE t +EXPLAIN DESC t -- !query 32 schema -struct<> +struct -- !query 32 output - +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `t`, false -- !query 33 -DROP VIEW temp_v +EXPLAIN DESC EXTENDED t -- !query 33 schema -struct<> +struct -- !query 33 output - +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `t`, true -- !query 34 -DROP VIEW temp_Data_Source_View +EXPLAIN EXTENDED DESC t -- !query 34 schema -struct<> +struct -- !query 34 output +== Parsed Logical Plan == +DescribeTableCommand `t`, false + +== Analyzed Logical Plan == +col_name: string, data_type: string, comment: string +DescribeTableCommand `t`, false +== Optimized Logical Plan == +DescribeTableCommand `t`, false + +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `t`, false -- !query 35 -DROP VIEW v +EXPLAIN DESCRIBE t b -- !query 35 schema -struct<> +struct -- !query 35 output +== Physical Plan == +Execute DescribeColumnCommand + +- DescribeColumnCommand `t`, [b], false + + +-- !query 36 +EXPLAIN DESCRIBE t PARTITION (c='Us', d=2) +-- !query 36 schema +struct +-- !query 36 output +== Physical Plan == +Execute DescribeTableCommand + +- DescribeTableCommand `t`, Map(c -> Us, d -> 2), false + + +-- !query 37 +DROP TABLE t +-- !query 37 schema +struct<> +-- !query 37 output + + + +-- !query 38 +DROP VIEW temp_v +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +DROP VIEW temp_Data_Source_View +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +DROP VIEW v +-- !query 40 schema +struct<> +-- !query 40 output diff --git a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out index cf5add6a71af2..09e2c632f6386 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by-ordinal.sql.out @@ -141,12 +141,12 @@ from -- !query 13 schema struct -- !query 13 output -1 0.4048454303385226 2 -1 0.8446490682263027 1 -2 0.5871875724155838 1 -2 0.8865128837019473 2 -3 0.742083829230211 1 -3 0.9179913208300406 2 +1 0.5234194256885571 2 +1 0.7604953758285915 1 +2 0.0953472826424725 1 +2 0.3163249920547614 2 +3 0.2710259815484829 2 +3 0.7141011170991605 1 -- !query 14 diff --git a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out index f9d1ee8a6bcdb..3a5df254f2cd9 100644 --- a/sql/core/src/test/resources/sql-tests/results/group-by.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/group-by.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 30 +-- Number of queries: 52 -- !query 0 @@ -275,3 +275,249 @@ struct<> -- !query 29 output org.apache.spark.sql.AnalysisException grouping expressions sequence is empty, and '`id`' is not an aggregate function. Wrap '()' in windowing function(s) or wrap '`id`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 30 +CREATE OR REPLACE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + (1, true), (1, false), + (2, true), + (3, false), (3, null), + (4, null), (4, null), + (5, null), (5, true), (5, false) AS test_agg(k, v) +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +SELECT every(v), some(v), any(v) FROM test_agg WHERE 1 = 0 +-- !query 31 schema +struct +-- !query 31 output +NULL NULL NULL + + +-- !query 32 +SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 4 +-- !query 32 schema +struct +-- !query 32 output +NULL NULL NULL + + +-- !query 33 +SELECT every(v), some(v), any(v) FROM test_agg WHERE k = 5 +-- !query 33 schema +struct +-- !query 33 output +false true true + + +-- !query 34 +SELECT k, every(v), some(v), any(v) FROM test_agg GROUP BY k +-- !query 34 schema +struct +-- !query 34 output +1 false true true +2 true true true +3 false false false +4 NULL NULL NULL +5 false true true + + +-- !query 35 +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) = false +-- !query 35 schema +struct +-- !query 35 output +1 false +3 false +5 false + + +-- !query 36 +SELECT k, every(v) FROM test_agg GROUP BY k HAVING every(v) IS NULL +-- !query 36 schema +struct +-- !query 36 output +4 NULL + + +-- !query 37 +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Any(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query 37 schema +struct +-- !query 37 output +2 true + + +-- !query 38 +SELECT k, + Every(v) AS every +FROM test_agg +WHERE k = 2 + AND v IN (SELECT Every(v) + FROM test_agg + WHERE k = 1) +GROUP BY k +-- !query 38 schema +struct +-- !query 38 output + + + +-- !query 39 +SELECT every(1) +-- !query 39 schema +struct<> +-- !query 39 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every(1)' due to data type mismatch: Input to function 'every' should have been boolean, but it's [int].; line 1 pos 7 + + +-- !query 40 +SELECT some(1S) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +cannot resolve 'some(1S)' due to data type mismatch: Input to function 'some' should have been boolean, but it's [smallint].; line 1 pos 7 + + +-- !query 41 +SELECT any(1L) +-- !query 41 schema +struct<> +-- !query 41 output +org.apache.spark.sql.AnalysisException +cannot resolve 'any(1L)' due to data type mismatch: Input to function 'any' should have been boolean, but it's [bigint].; line 1 pos 7 + + +-- !query 42 +SELECT every("true") +-- !query 42 schema +struct<> +-- !query 42 output +org.apache.spark.sql.AnalysisException +cannot resolve 'every('true')' due to data type mismatch: Input to function 'every' should have been boolean, but it's [string].; line 1 pos 7 + + +-- !query 43 +SELECT k, v, every(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 43 schema +struct +-- !query 43 output +1 false false +1 true false +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true false + + +-- !query 44 +SELECT k, v, some(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 44 schema +struct +-- !query 44 output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query 45 +SELECT k, v, any(v) OVER (PARTITION BY k ORDER BY v) FROM test_agg +-- !query 45 schema +struct +-- !query 45 output +1 false false +1 true true +2 true true +3 NULL NULL +3 false false +4 NULL NULL +4 NULL NULL +5 NULL NULL +5 false false +5 true true + + +-- !query 46 +SELECT count(*) FROM test_agg HAVING count(*) > 1L +-- !query 46 schema +struct +-- !query 46 output +10 + + +-- !query 47 +SELECT k, max(v) FROM test_agg GROUP BY k HAVING max(v) = true +-- !query 47 schema +struct +-- !query 47 output +1 true +2 true +5 true + + +-- !query 48 +SELECT * FROM (SELECT COUNT(*) AS cnt FROM test_agg) WHERE cnt > 1L +-- !query 48 schema +struct +-- !query 48 output +10 + + +-- !query 49 +SELECT count(*) FROM test_agg WHERE count(*) > 1L +-- !query 49 schema +struct<> +-- !query 49 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(count(1) > 1L)] +Invalid expressions: [count(1)]; + + +-- !query 50 +SELECT count(*) FROM test_agg WHERE count(*) + 1L > 1L +-- !query 50 schema +struct<> +-- !query 50 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [((count(1) + 1L) > 1L)] +Invalid expressions: [count(1)]; + + +-- !query 51 +SELECT count(*) FROM test_agg WHERE k = 1 or k = 2 or count(*) + 1L > 1L or max(k) > 1 +-- !query 51 schema +struct<> +-- !query 51 output +org.apache.spark.sql.AnalysisException + +Aggregate/Window/Generate expressions are not valid in where clause of the query. +Expression in where clause: [(((test_agg.`k` = 1) OR (test_agg.`k` = 2)) OR (((count(1) + 1L) > 1L) OR (max(test_agg.`k`) > 1)))] +Invalid expressions: [count(1), max(test_agg.`k`)]; diff --git a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out index c065ce5012929..4e80f0bda5513 100644 --- a/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/inline-table.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 18 +-- Number of queries: 17 -- !query 0 @@ -151,33 +151,3 @@ select * from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991- struct> -- !query 16 output 1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] - - --- !query 17 -EXPLAIN EXTENDED SELECT * FROM VALUES ('one', 1), ('three', null) CROSS JOIN VALUES ('one', 1), ('three', null) --- !query 17 schema -struct --- !query 17 output -== Parsed Logical Plan == -'Project [*] -+- 'Join Cross - :- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] - +- 'UnresolvedInlineTable [col1, col2], [List(one, 1), List(three, null)] - -== Analyzed Logical Plan == -col1: string, col2: int, col1: string, col2: int -Project [col1#x, col2#x, col1#x, col2#x] -+- Join Cross - :- LocalRelation [col1#x, col2#x] - +- LocalRelation [col1#x, col2#x] - -== Optimized Logical Plan == -Join Cross -:- LocalRelation [col1#x, col2#x] -+- LocalRelation [col1#x, col2#x] - -== Physical Plan == -BroadcastNestedLoopJoin BuildRight, Cross -:- LocalTableScan [col1#x, col2#x] -+- BroadcastExchange IdentityBroadcastMode - +- LocalTableScan [col1#x, col2#x] 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 77e9000401141..ca0cd90d94fa7 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 @@ -3,386 +3,352 @@ -- !query 0 -describe function to_json +select to_json(named_struct('a', 1, 'b', 2)) -- !query 0 schema -struct +struct -- !query 0 output -Class: org.apache.spark.sql.catalyst.expressions.StructsToJson -Function: to_json -Usage: to_json(expr[, options]) - Returns a JSON string with a given struct value +{"a":1,"b":2} -- !query 1 -describe function extended to_json +select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) -- !query 1 schema -struct +struct -- !query 1 output -Class: org.apache.spark.sql.catalyst.expressions.StructsToJson -Extended Usage: - Examples: - > SELECT to_json(named_struct('a', 1, 'b', 2)); - {"a":1,"b":2} - > SELECT to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')); - {"time":"26/08/2015"} - > SELECT to_json(array(named_struct('a', 1, 'b', 2))); - [{"a":1,"b":2}] - > SELECT to_json(map('a', named_struct('b', 1))); - {"a":{"b":1}} - > SELECT to_json(map(named_struct('a', 1),named_struct('b', 2))); - {"[1]":{"b":2}} - > SELECT to_json(map('a', 1)); - {"a":1} - > SELECT to_json(array((map('a', 1)))); - [{"a":1}] - - Since: 2.2.0 - -Function: to_json -Usage: to_json(expr[, options]) - Returns a JSON string with a given struct value +{"time":"26/08/2015"} -- !query 2 -select to_json(named_struct('a', 1, 'b', 2)) +select to_json(array(named_struct('a', 1, 'b', 2))) -- !query 2 schema -struct +struct -- !query 2 output -{"a":1,"b":2} +[{"a":1,"b":2}] -- !query 3 -select to_json(named_struct('time', to_timestamp('2015-08-26', 'yyyy-MM-dd')), map('timestampFormat', 'dd/MM/yyyy')) +select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) -- !query 3 schema -struct +struct -- !query 3 output -{"time":"26/08/2015"} +{"[1,2]":{"a":1,"b":2}} -- !query 4 -select to_json(array(named_struct('a', 1, 'b', 2))) +select to_json(map('a', named_struct('a', 1, 'b', 2))) -- !query 4 schema -struct +struct -- !query 4 output -[{"a":1,"b":2}] +{"a":{"a":1,"b":2}} -- !query 5 -select to_json(map(named_struct('a', 1, 'b', 2), named_struct('a', 1, 'b', 2))) +select to_json(map('a', 1)) -- !query 5 schema -struct +struct -- !query 5 output -{"[1,2]":{"a":1,"b":2}} +{"a":1} -- !query 6 -select to_json(map('a', named_struct('a', 1, 'b', 2))) +select to_json(array(map('a',1))) -- !query 6 schema -struct +struct -- !query 6 output -{"a":{"a":1,"b":2}} +[{"a":1}] -- !query 7 -select to_json(map('a', 1)) +select to_json(array(map('a',1), map('b',2))) -- !query 7 schema -struct +struct -- !query 7 output -{"a":1} +[{"a":1},{"b":2}] -- !query 8 -select to_json(array(map('a',1))) +select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) -- !query 8 schema -struct +struct<> -- !query 8 output -[{"a":1}] +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 -- !query 9 -select to_json(array(map('a',1), map('b',2))) +select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) -- !query 9 schema -struct +struct<> -- !query 9 output -[{"a":1},{"b":2}] +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got map;; line 1 pos 7 -- !query 10 -select to_json(named_struct('a', 1, 'b', 2), named_struct('mode', 'PERMISSIVE')) +select to_json() -- !query 10 schema struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -Must use a map() function for options;; line 1 pos 7 +Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 -- !query 11 -select to_json(named_struct('a', 1, 'b', 2), map('mode', 1)) +select from_json('{"a":1}', 'a INT') -- !query 11 schema -struct<> +struct> -- !query 11 output -org.apache.spark.sql.AnalysisException -A type of keys and values in map() must be string, but got map;; line 1 pos 7 +{"a":1} -- !query 12 -select to_json() +select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) -- !query 12 schema -struct<> +struct> -- !query 12 output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function to_json. Expected: one of 1 and 2; Found: 0; line 1 pos 7 +{"time":2015-08-26 00:00:00.0} -- !query 13 -describe function from_json +select from_json('{"a":1}', 1) -- !query 13 schema -struct +struct<> -- !query 13 output -Class: org.apache.spark.sql.catalyst.expressions.JsonToStructs -Function: from_json -Usage: from_json(jsonStr, schema[, options]) - Returns a struct value with the given `jsonStr` and `schema`. +org.apache.spark.sql.AnalysisException +Schema should be specified in DDL format as a string literal or output of the schema_of_json function instead of 1;; line 1 pos 7 -- !query 14 -describe function extended from_json +select from_json('{"a":1}', 'a InvalidType') -- !query 14 schema -struct +struct<> -- !query 14 output -Class: org.apache.spark.sql.catalyst.expressions.JsonToStructs -Extended Usage: - Examples: - > SELECT from_json('{"a":1, "b":0.8}', 'a INT, b DOUBLE'); - {"a":1, "b":0.8} - > SELECT from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')); - {"time":"2015-08-26 00:00:00.0"} - - Since: 2.2.0 +org.apache.spark.sql.AnalysisException + +DataType invalidtype is not supported.(line 1, pos 2) -Function: from_json -Usage: from_json(jsonStr, schema[, options]) - Returns a struct value with the given `jsonStr` and `schema`. +== SQL == +a InvalidType +--^^^ +; line 1 pos 7 -- !query 15 -select from_json('{"a":1}', 'a INT') +select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) -- !query 15 schema -struct> +struct<> -- !query 15 output -{"a":1} +org.apache.spark.sql.AnalysisException +Must use a map() function for options;; line 1 pos 7 -- !query 16 -select from_json('{"time":"26/08/2015"}', 'time Timestamp', map('timestampFormat', 'dd/MM/yyyy')) +select from_json('{"a":1}', 'a INT', map('mode', 1)) -- !query 16 schema -struct> +struct<> -- !query 16 output -{"time":2015-08-26 00:00:00.0} +org.apache.spark.sql.AnalysisException +A type of keys and values in map() must be string, but got map;; line 1 pos 7 -- !query 17 -select from_json('{"a":1}', 1) +select from_json() -- !query 17 schema struct<> -- !query 17 output org.apache.spark.sql.AnalysisException -Schema should be specified in DDL format as a string literal or output of the schema_of_json function instead of 1;; line 1 pos 7 +Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 -- !query 18 -select from_json('{"a":1}', 'a InvalidType') +SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') -- !query 18 schema -struct<> +struct -- !query 18 output -org.apache.spark.sql.AnalysisException - -DataType invalidtype is not supported.(line 1, pos 2) - -== SQL == -a InvalidType ---^^^ -; line 1 pos 7 +NULL 2 NULL 1 -- !query 19 -select from_json('{"a":1}', 'a INT', named_struct('mode', 'PERMISSIVE')) +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') -- !query 19 schema struct<> -- !query 19 output -org.apache.spark.sql.AnalysisException -Must use a map() function for options;; line 1 pos 7 + -- !query 20 -select from_json('{"a":1}', 'a INT', map('mode', 1)) +SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable -- !query 20 schema -struct<> +struct -- !query 20 output -org.apache.spark.sql.AnalysisException -A type of keys and values in map() must be string, but got map;; line 1 pos 7 +2 NULL 1 -- !query 21 -select from_json() +DROP VIEW IF EXISTS jsonTable -- !query 21 schema struct<> -- !query 21 output -org.apache.spark.sql.AnalysisException -Invalid number of arguments for function from_json. Expected: one of 2 and 3; Found: 0; line 1 pos 7 + -- !query 22 -SELECT json_tuple('{"a" : 1, "b" : 2}', CAST(NULL AS STRING), 'b', CAST(NULL AS STRING), 'a') +select from_json('{"a":1, "b":2}', 'map') -- !query 22 schema -struct +struct> -- !query 22 output -NULL 2 NULL 1 +{"a":1,"b":2} -- !query 23 -CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') +select from_json('{"a":1, "b":"2"}', 'struct') -- !query 23 schema -struct<> +struct> -- !query 23 output - +{"a":1,"b":"2"} -- !query 24 -SELECT json_tuple(jsonField, 'b', CAST(NULL AS STRING), a) FROM jsonTable +select schema_of_json('{"c1":0, "c2":[1]}') -- !query 24 schema -struct +struct -- !query 24 output -2 NULL 1 +struct> -- !query 25 -DROP VIEW IF EXISTS jsonTable +select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')) -- !query 25 schema -struct<> +struct>> -- !query 25 output - +{"c1":[1,2,3]} -- !query 26 -select from_json('{"a":1, "b":2}', 'map') +select from_json('[1, 2, 3]', 'array') -- !query 26 schema -struct> +struct> -- !query 26 output -{"a":1,"b":2} +[1,2,3] -- !query 27 -select from_json('{"a":1, "b":"2"}', 'struct') +select from_json('[1, "2", 3]', 'array') -- !query 27 schema -struct> +struct> -- !query 27 output -{"a":1,"b":"2"} +NULL -- !query 28 -select schema_of_json('{"c1":0, "c2":[1]}') +select from_json('[1, 2, null]', 'array') -- !query 28 schema -struct +struct> -- !query 28 output -struct> +[1,2,null] -- !query 29 -select from_json('{"c1":[1, 2, 3]}', schema_of_json('{"c1":[0]}')) +select from_json('[{"a": 1}, {"a":2}]', 'array>') -- !query 29 schema -struct>> +struct>> -- !query 29 output -{"c1":[1,2,3]} +[{"a":1},{"a":2}] -- !query 30 -select from_json('[1, 2, 3]', 'array') +select from_json('{"a": 1}', 'array>') -- !query 30 schema -struct> +struct>> -- !query 30 output -[1,2,3] +[{"a":1}] -- !query 31 -select from_json('[1, "2", 3]', 'array') +select from_json('[null, {"a":2}]', 'array>') -- !query 31 schema -struct> +struct>> -- !query 31 output -NULL +[null,{"a":2}] -- !query 32 -select from_json('[1, 2, null]', 'array') +select from_json('[{"a": 1}, {"b":2}]', 'array>') -- !query 32 schema -struct> +struct>> -- !query 32 output -[1,2,null] +[{"a":1},{"b":2}] -- !query 33 -select from_json('[{"a": 1}, {"a":2}]', 'array>') +select from_json('[{"a": 1}, 2]', 'array>') -- !query 33 schema -struct>> +struct>> -- !query 33 output -[{"a":1},{"a":2}] +NULL -- !query 34 -select from_json('{"a": 1}', 'array>') +select to_json(array('1', '2', '3')) -- !query 34 schema -struct>> +struct -- !query 34 output -[{"a":1}] +["1","2","3"] -- !query 35 -select from_json('[null, {"a":2}]', 'array>') +select to_json(array(array(1, 2, 3), array(4))) -- !query 35 schema -struct>> +struct -- !query 35 output -[null,{"a":2}] +[[1,2,3],[4]] -- !query 36 -select from_json('[{"a": 1}, {"b":2}]', 'array>') +select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')) -- !query 36 schema -struct>> +struct -- !query 36 output -[{"a":1},{"b":2}] +struct -- !query 37 -select from_json('[{"a": 1}, 2]', 'array>') +select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')) -- !query 37 schema -struct>> +struct -- !query 37 output -NULL +struct -- !query 38 -select to_json(array('1', '2', '3')) +select schema_of_json(null) -- !query 38 schema -struct +struct<> -- !query 38 output -["1","2","3"] +org.apache.spark.sql.AnalysisException +cannot resolve 'schema_of_json(NULL)' due to data type mismatch: The input json should be a string literal and not null; however, got NULL.; line 1 pos 7 -- !query 39 -select to_json(array(array(1, 2, 3), array(4))) +CREATE TEMPORARY VIEW jsonTable(jsonField, a) AS SELECT * FROM VALUES ('{"a": 1, "b": 2}', 'a') -- !query 39 schema -struct +struct<> -- !query 39 output -[[1,2,3],[4]] + -- !query 40 -select schema_of_json('{"c1":1}', map('primitivesAsString', 'true')) +SELECT schema_of_json(jsonField) FROM jsonTable -- !query 40 schema -struct +struct<> -- !query 40 output -struct +org.apache.spark.sql.AnalysisException +cannot resolve 'schema_of_json(jsontable.`jsonField`)' due to data type mismatch: The input json should be a string literal and not null; however, got jsontable.`jsonField`.; line 1 pos 7 -- !query 41 -select schema_of_json('{"c1":01, "c2":0.1}', map('allowNumericLeadingZeros', 'true', 'prefersDecimal', 'true')) +DROP VIEW IF EXISTS jsonTable -- !query 41 schema -struct +struct<> -- !query 41 output -struct + diff --git a/sql/core/src/test/resources/sql-tests/results/literals.sql.out b/sql/core/src/test/resources/sql-tests/results/literals.sql.out index 7f301614523b2..9d3668d49d0a0 100644 --- a/sql/core/src/test/resources/sql-tests/results/literals.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/literals.sql.out @@ -291,7 +291,7 @@ struct<> -- !query 31 output org.apache.spark.sql.catalyst.parser.ParseException -Exception parsing DATE(line 1, pos 7) +Cannot parse the DATE value: mar 11 2016(line 1, pos 7) == SQL == select date 'mar 11 2016' @@ -301,7 +301,7 @@ select date 'mar 11 2016' -- !query 32 select tImEstAmp '2016-03-11 20:54:00.000' -- !query 32 schema -struct +struct -- !query 32 output 2016-03-11 20:54:00 @@ -313,7 +313,7 @@ struct<> -- !query 33 output org.apache.spark.sql.catalyst.parser.ParseException -Timestamp format must be yyyy-mm-dd hh:mm:ss[.fffffffff](line 1, pos 7) +Cannot parse the TIMESTAMP value: 2016-33-11 20:54:00.000(line 1, pos 7) == SQL == select timestamp '2016-33-11 20:54:00.000' @@ -343,11 +343,11 @@ struct<> -- !query 36 output org.apache.spark.sql.catalyst.parser.ParseException -No interval can be constructed(line 1, pos 16) +no viable alternative at input 'interval 10 nanoseconds'(line 1, pos 19) == SQL == select interval 10 nanoseconds -----------------^^^ +-------------------^^^ -- !query 37 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 fd1d0db9e3f78..e0cbd575bc346 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 55 +-- Number of queries: 49 -- !query 0 @@ -195,260 +195,200 @@ struct -- !query 24 -explain select 'a' || 1 + 2 +select cot(1) -- !query 24 schema -struct +struct -- !query 24 output -== Physical Plan == -*Project [null AS (CAST(concat(a, CAST(1 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE))#x] -+- Scan OneRowRelation[] +0.6420926159343306 -- !query 25 -explain select 1 - 2 || 'b' +select cot(null) -- !query 25 schema -struct +struct -- !query 25 output -== Physical Plan == -*Project [-1b AS concat(CAST((1 - 2) AS STRING), b)#x] -+- Scan OneRowRelation[] +NULL -- !query 26 -explain select 2 * 4 + 3 || 'b' +select cot(0) -- !query 26 schema -struct +struct -- !query 26 output -== Physical Plan == -*Project [11b AS concat(CAST(((2 * 4) + 3) AS STRING), b)#x] -+- Scan OneRowRelation[] +Infinity -- !query 27 -explain select 3 + 1 || 'a' || 4 / 2 +select cot(-1) -- !query 27 schema -struct +struct -- !query 27 output -== Physical Plan == -*Project [4a2.0 AS concat(concat(CAST((3 + 1) AS STRING), a), CAST((CAST(4 AS DOUBLE) / CAST(2 AS DOUBLE)) AS STRING))#x] -+- Scan OneRowRelation[] +-0.6420926159343306 -- !query 28 -explain select 1 == 1 OR 'a' || 'b' == 'ab' +select ceiling(0) -- !query 28 schema -struct +struct -- !query 28 output -== Physical Plan == -*Project [true AS ((1 = 1) OR (concat(a, b) = ab))#x] -+- Scan OneRowRelation[] +0 -- !query 29 -explain select 'a' || 'c' == 'ac' AND 2 == 3 +select ceiling(1) -- !query 29 schema -struct +struct -- !query 29 output -== Physical Plan == -*Project [false AS ((concat(a, c) = ac) AND (2 = 3))#x] -+- Scan OneRowRelation[] +1 -- !query 30 -select cot(1) +select ceil(1234567890123456) -- !query 30 schema -struct +struct -- !query 30 output -0.6420926159343306 +1234567890123456 -- !query 31 -select cot(null) +select ceiling(1234567890123456) -- !query 31 schema -struct +struct -- !query 31 output -NULL +1234567890123456 -- !query 32 -select cot(0) +select ceil(0.01) -- !query 32 schema -struct +struct -- !query 32 output -Infinity +1 -- !query 33 -select cot(-1) +select ceiling(-0.10) -- !query 33 schema -struct +struct -- !query 33 output --0.6420926159343306 +0 -- !query 34 -select ceiling(0) +select floor(0) -- !query 34 schema -struct +struct -- !query 34 output 0 -- !query 35 -select ceiling(1) +select floor(1) -- !query 35 schema -struct +struct -- !query 35 output 1 -- !query 36 -select ceil(1234567890123456) +select floor(1234567890123456) -- !query 36 schema -struct +struct -- !query 36 output 1234567890123456 -- !query 37 -select ceiling(1234567890123456) --- !query 37 schema -struct --- !query 37 output -1234567890123456 - - --- !query 38 -select ceil(0.01) --- !query 38 schema -struct --- !query 38 output -1 - - --- !query 39 -select ceiling(-0.10) --- !query 39 schema -struct --- !query 39 output -0 - - --- !query 40 -select floor(0) --- !query 40 schema -struct --- !query 40 output -0 - - --- !query 41 -select floor(1) --- !query 41 schema -struct --- !query 41 output -1 - - --- !query 42 -select floor(1234567890123456) --- !query 42 schema -struct --- !query 42 output -1234567890123456 - - --- !query 43 select floor(0.01) --- !query 43 schema +-- !query 37 schema struct --- !query 43 output +-- !query 37 output 0 --- !query 44 +-- !query 38 select floor(-0.10) --- !query 44 schema +-- !query 38 schema struct --- !query 44 output +-- !query 38 output -1 --- !query 45 +-- !query 39 select 1 > 0.00001 --- !query 45 schema +-- !query 39 schema struct<(CAST(1 AS BIGINT) > 0):boolean> --- !query 45 output +-- !query 39 output true --- !query 46 +-- !query 40 select mod(7, 2), mod(7, 0), mod(0, 2), mod(7, null), mod(null, 2), mod(null, null) --- !query 46 schema +-- !query 40 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> --- !query 46 output +-- !query 40 output 1 NULL 0 NULL NULL NULL --- !query 47 +-- !query 41 select BIT_LENGTH('abc') --- !query 47 schema +-- !query 41 schema struct --- !query 47 output +-- !query 41 output 24 --- !query 48 +-- !query 42 select CHAR_LENGTH('abc') --- !query 48 schema +-- !query 42 schema struct --- !query 48 output +-- !query 42 output 3 --- !query 49 +-- !query 43 select CHARACTER_LENGTH('abc') --- !query 49 schema +-- !query 43 schema struct --- !query 49 output +-- !query 43 output 3 --- !query 50 +-- !query 44 select OCTET_LENGTH('abc') --- !query 50 schema +-- !query 44 schema struct --- !query 50 output +-- !query 44 output 3 --- !query 51 +-- !query 45 select abs(-3.13), abs('-2.19') --- !query 51 schema +-- !query 45 schema struct --- !query 51 output +-- !query 45 output 3.13 2.19 --- !query 52 +-- !query 46 select positive('-1.11'), positive(-1.11), negative('-1.11'), negative(-1.11) --- !query 52 schema +-- !query 46 schema struct<(+ CAST(-1.11 AS DOUBLE)):double,(+ -1.11):decimal(3,2),(- CAST(-1.11 AS DOUBLE)):double,(- -1.11):decimal(3,2)> --- !query 52 output +-- !query 46 output -1.11 -1.11 1.11 1.11 --- !query 53 +-- !query 47 select pmod(-7, 2), pmod(0, 2), pmod(7, 0), pmod(7, null), pmod(null, 2), pmod(null, null) --- !query 53 schema +-- !query 47 schema struct --- !query 53 output +-- !query 47 output 1 0 NULL NULL NULL NULL --- !query 54 +-- !query 48 select pmod(cast(3.13 as decimal), cast(0 as decimal)), pmod(cast(2 as smallint), cast(0 as smallint)) --- !query 54 schema +-- !query 48 schema struct --- !query 54 output +-- !query 48 output NULL NULL diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out new file mode 100644 index 0000000000000..51ca1d558691c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part1.sql.out @@ -0,0 +1,396 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 44 + + +-- !query 0 +SELECT avg(four) AS avg_1 FROM onek +-- !query 0 schema +struct +-- !query 0 output +1.5 + + +-- !query 1 +SELECT avg(a) AS avg_32 FROM aggtest WHERE a < 100 +-- !query 1 schema +struct +-- !query 1 output +32.666666666666664 + + +-- !query 2 +select CAST(avg(b) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query 2 schema +struct +-- !query 2 output +107.943 + + +-- !query 3 +SELECT sum(four) AS sum_1500 FROM onek +-- !query 3 schema +struct +-- !query 3 output +1500 + + +-- !query 4 +SELECT sum(a) AS sum_198 FROM aggtest +-- !query 4 schema +struct +-- !query 4 output +198 + + +-- !query 5 +SELECT sum(b) AS avg_431_773 FROM aggtest +-- !query 5 schema +struct +-- !query 5 output +431.77260909229517 + + +-- !query 6 +SELECT max(four) AS max_3 FROM onek +-- !query 6 schema +struct +-- !query 6 output +3 + + +-- !query 7 +SELECT max(a) AS max_100 FROM aggtest +-- !query 7 schema +struct +-- !query 7 output +100 + + +-- !query 8 +SELECT max(aggtest.b) AS max_324_78 FROM aggtest +-- !query 8 schema +struct +-- !query 8 output +324.78 + + +-- !query 9 +SELECT stddev_pop(b) FROM aggtest +-- !query 9 schema +struct +-- !query 9 output +131.10703231895047 + + +-- !query 10 +SELECT stddev_samp(b) FROM aggtest +-- !query 10 schema +struct +-- !query 10 output +151.38936080399804 + + +-- !query 11 +SELECT var_pop(b) FROM aggtest +-- !query 11 schema +struct +-- !query 11 output +17189.053923482323 + + +-- !query 12 +SELECT var_samp(b) FROM aggtest +-- !query 12 schema +struct +-- !query 12 output +22918.738564643096 + + +-- !query 13 +SELECT stddev_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query 13 schema +struct +-- !query 13 output +131.18117242958306 + + +-- !query 14 +SELECT stddev_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query 14 schema +struct +-- !query 14 output +151.47497042966097 + + +-- !query 15 +SELECT var_pop(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query 15 schema +struct +-- !query 15 output +17208.5 + + +-- !query 16 +SELECT var_samp(CAST(b AS Decimal(38,0))) FROM aggtest +-- !query 16 schema +struct +-- !query 16 output +22944.666666666668 + + +-- !query 17 +SELECT var_pop(1.0), var_samp(2.0) +-- !query 17 schema +struct +-- !query 17 output +0.0 NaN + + +-- !query 18 +SELECT stddev_pop(CAST(3.0 AS Decimal(38,0))), stddev_samp(CAST(4.0 AS Decimal(38,0))) +-- !query 18 schema +struct +-- !query 18 output +0.0 NaN + + +-- !query 19 +select sum(CAST(null AS int)) from range(1,4) +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +select sum(CAST(null AS long)) from range(1,4) +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +select sum(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +select sum(CAST(null AS DOUBLE)) from range(1,4) +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +select avg(CAST(null AS int)) from range(1,4) +-- !query 23 schema +struct +-- !query 23 output +NULL + + +-- !query 24 +select avg(CAST(null AS long)) from range(1,4) +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +select avg(CAST(null AS Decimal(38,0))) from range(1,4) +-- !query 25 schema +struct +-- !query 25 output +NULL + + +-- !query 26 +select avg(CAST(null AS DOUBLE)) from range(1,4) +-- !query 26 schema +struct +-- !query 26 output +NULL + + +-- !query 27 +select sum(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query 27 schema +struct +-- !query 27 output +NaN + + +-- !query 28 +select avg(CAST('NaN' AS DOUBLE)) from range(1,4) +-- !query 28 schema +struct +-- !query 28 output +NaN + + +-- !query 29 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (CAST('1' AS DOUBLE)), (CAST('Infinity' AS DOUBLE))) v(x) +-- !query 29 schema +struct +-- !query 29 output +Infinity NaN + + +-- !query 30 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x) +-- !query 30 schema +struct +-- !query 30 output +Infinity NaN + + +-- !query 31 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x) +-- !query 31 schema +struct +-- !query 31 output +Infinity NaN + + +-- !query 32 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +-- !query 32 schema +struct +-- !query 32 output +NaN NaN + + +-- !query 33 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query 33 schema +struct +-- !query 33 output +1.00000005E8 2.5 + + +-- !query 34 +SELECT avg(CAST(x AS DOUBLE)), var_pop(CAST(x AS DOUBLE)) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query 34 schema +struct +-- !query 34 output +7.000000000006E12 1.0 + + +-- !query 35 +SELECT covar_pop(b, a), covar_samp(b, a) FROM aggtest +-- !query 35 schema +struct +-- !query 35 output +653.6289553875104 871.5052738500139 + + +-- !query 36 +SELECT corr(b, a) FROM aggtest +-- !query 36 schema +struct +-- !query 36 output +0.1396345165178734 + + +-- !query 37 +SELECT count(four) AS cnt_1000 FROM onek +-- !query 37 schema +struct +-- !query 37 output +1000 + + +-- !query 38 +SELECT count(DISTINCT four) AS cnt_4 FROM onek +-- !query 38 schema +struct +-- !query 38 output +4 + + +-- !query 39 +select ten, count(*), sum(four) from onek +group by ten order by ten +-- !query 39 schema +struct +-- !query 39 output +0 100 100 +1 100 200 +2 100 100 +3 100 200 +4 100 100 +5 100 200 +6 100 100 +7 100 200 +8 100 100 +9 100 200 + + +-- !query 40 +select ten, count(four), sum(DISTINCT four) from onek +group by ten order by ten +-- !query 40 schema +struct +-- !query 40 output +0 100 2 +1 100 4 +2 100 2 +3 100 4 +4 100 2 +5 100 4 +6 100 2 +7 100 4 +8 100 2 +9 100 4 + + +-- !query 41 +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b where sum(distinct a.four) = b.four) +-- !query 41 schema +struct +-- !query 41 output +0 2 +2 2 +4 2 +6 2 +8 2 + + +-- !query 42 +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = b.four) +-- !query 42 schema +struct<> +-- !query 42 output +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))]; + + +-- !query 43 +select + (select max((select i.unique2 from tenk1 i where i.unique1 = o.unique1))) +from tenk1 o +-- !query 43 schema +struct<> +-- !query 43 output +org.apache.spark.sql.AnalysisException +cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 63 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out new file mode 100644 index 0000000000000..2606d2eba7468 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/aggregates_part2.sql.out @@ -0,0 +1,162 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query 1 schema +struct +-- !query 1 output +true true false true false true true true true + + +-- !query 2 +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query 2 schema +struct +-- !query 2 output +true false true false true true true true true + + +-- !query 3 +select min(unique1) from tenk1 +-- !query 3 schema +struct +-- !query 3 output +0 + + +-- !query 4 +select max(unique1) from tenk1 +-- !query 4 schema +struct +-- !query 4 output +9999 + + +-- !query 5 +select max(unique1) from tenk1 where unique1 < 42 +-- !query 5 schema +struct +-- !query 5 output +41 + + +-- !query 6 +select max(unique1) from tenk1 where unique1 > 42 +-- !query 6 schema +struct +-- !query 6 output +9999 + + +-- !query 7 +select max(unique1) from tenk1 where unique1 > 42000 +-- !query 7 schema +struct +-- !query 7 output +NULL + + +-- !query 8 +select max(tenthous) from tenk1 where thousand = 33 +-- !query 8 schema +struct +-- !query 8 output +9033 + + +-- !query 9 +select min(tenthous) from tenk1 where thousand = 33 +-- !query 9 schema +struct +-- !query 9 output +33 + + +-- !query 10 +select distinct max(unique2) from tenk1 +-- !query 10 schema +struct +-- !query 10 output +9999 + + +-- !query 11 +select max(unique2) from tenk1 order by 1 +-- !query 11 schema +struct +-- !query 11 output +9999 + + +-- !query 12 +select max(unique2) from tenk1 order by max(unique2) +-- !query 12 schema +struct +-- !query 12 output +9999 + + +-- !query 13 +select max(unique2) from tenk1 order by max(unique2)+1 +-- !query 13 schema +struct +-- !query 13 output +9999 + + +-- !query 14 +select t1.max_unique2, g from (select max(unique2) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query 14 schema +struct +-- !query 14 output +9999 3 +9999 2 +9999 1 + + +-- !query 15 +select max(100) from tenk1 +-- !query 15 schema +struct +-- !query 15 output +100 diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out new file mode 100644 index 0000000000000..b7cf3a9f1ad84 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/boolean.sql.out @@ -0,0 +1,741 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 83 + + +-- !query 0 +SELECT 1 AS one +-- !query 0 schema +struct +-- !query 0 output +1 + + +-- !query 1 +SELECT true AS true +-- !query 1 schema +struct +-- !query 1 output +true + + +-- !query 2 +SELECT false AS `false` +-- !query 2 schema +struct +-- !query 2 output +false + + +-- !query 3 +SELECT boolean('t') AS true +-- !query 3 schema +struct +-- !query 3 output +true + + +-- !query 4 +SELECT boolean(' f ') AS `false` +-- !query 4 schema +struct +-- !query 4 output +NULL + + +-- !query 5 +SELECT boolean('true') AS true +-- !query 5 schema +struct +-- !query 5 output +true + + +-- !query 6 +SELECT boolean('test') AS error +-- !query 6 schema +struct +-- !query 6 output +NULL + + +-- !query 7 +SELECT boolean('false') AS `false` +-- !query 7 schema +struct +-- !query 7 output +false + + +-- !query 8 +SELECT boolean('foo') AS error +-- !query 8 schema +struct +-- !query 8 output +NULL + + +-- !query 9 +SELECT boolean('y') AS true +-- !query 9 schema +struct +-- !query 9 output +true + + +-- !query 10 +SELECT boolean('yes') AS true +-- !query 10 schema +struct +-- !query 10 output +true + + +-- !query 11 +SELECT boolean('yeah') AS error +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT boolean('n') AS `false` +-- !query 12 schema +struct +-- !query 12 output +false + + +-- !query 13 +SELECT boolean('no') AS `false` +-- !query 13 schema +struct +-- !query 13 output +false + + +-- !query 14 +SELECT boolean('nay') AS error +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +SELECT boolean('on') AS true +-- !query 15 schema +struct +-- !query 15 output +NULL + + +-- !query 16 +SELECT boolean('off') AS `false` +-- !query 16 schema +struct +-- !query 16 output +NULL + + +-- !query 17 +SELECT boolean('of') AS `false` +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT boolean('o') AS error +-- !query 18 schema +struct +-- !query 18 output +NULL + + +-- !query 19 +SELECT boolean('on_') AS error +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +SELECT boolean('off_') AS error +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +SELECT boolean('1') AS true +-- !query 21 schema +struct +-- !query 21 output +true + + +-- !query 22 +SELECT boolean('11') AS error +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +SELECT boolean('0') AS `false` +-- !query 23 schema +struct +-- !query 23 output +false + + +-- !query 24 +SELECT boolean('000') AS error +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +SELECT boolean('') AS error +-- !query 25 schema +struct +-- !query 25 output +NULL + + +-- !query 26 +SELECT boolean('t') or boolean('f') AS true +-- !query 26 schema +struct +-- !query 26 output +true + + +-- !query 27 +SELECT boolean('t') and boolean('f') AS `false` +-- !query 27 schema +struct +-- !query 27 output +false + + +-- !query 28 +SELECT not boolean('f') AS true +-- !query 28 schema +struct +-- !query 28 output +true + + +-- !query 29 +SELECT boolean('t') = boolean('f') AS `false` +-- !query 29 schema +struct +-- !query 29 output +false + + +-- !query 30 +SELECT boolean('t') <> boolean('f') AS true +-- !query 30 schema +struct +-- !query 30 output +true + + +-- !query 31 +SELECT boolean('t') > boolean('f') AS true +-- !query 31 schema +struct +-- !query 31 output +true + + +-- !query 32 +SELECT boolean('t') >= boolean('f') AS true +-- !query 32 schema +struct +-- !query 32 output +true + + +-- !query 33 +SELECT boolean('f') < boolean('t') AS true +-- !query 33 schema +struct +-- !query 33 output +true + + +-- !query 34 +SELECT boolean('f') <= boolean('t') AS true +-- !query 34 schema +struct +-- !query 34 output +true + + +-- !query 35 +SELECT boolean(string('TrUe')) AS true, boolean(string('fAlse')) AS `false` +-- !query 35 schema +struct +-- !query 35 output +true false + + +-- !query 36 +SELECT boolean(string(' true ')) AS true, + boolean(string(' FALSE')) AS `false` +-- !query 36 schema +struct +-- !query 36 output +NULL NULL + + +-- !query 37 +SELECT string(boolean(true)) AS true, string(boolean(false)) AS `false` +-- !query 37 schema +struct +-- !query 37 output +true false + + +-- !query 38 +SELECT boolean(string(' tru e ')) AS invalid +-- !query 38 schema +struct +-- !query 38 output +NULL + + +-- !query 39 +-- error +SELECT boolean(string('')) AS invalid +-- !query 39 schema +struct +-- !query 39 output +NULL + + +-- !query 40 +-- error + +CREATE TABLE BOOLTBL1 (f1 boolean) USING parquet +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +INSERT INTO BOOLTBL1 VALUES (cast('t' as boolean)) +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +INSERT INTO BOOLTBL1 VALUES (cast('True' as boolean)) +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +INSERT INTO BOOLTBL1 VALUES (cast('true' as boolean)) +-- !query 43 schema +struct<> +-- !query 43 output + + + +-- !query 44 +SELECT '' AS t_3, BOOLTBL1.* FROM BOOLTBL1 +-- !query 44 schema +struct +-- !query 44 output +true + true + true + + +-- !query 45 +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('true') +-- !query 45 schema +struct +-- !query 45 output +true + true + true + + +-- !query 46 +SELECT '' AS t_3, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 <> boolean('false') +-- !query 46 schema +struct +-- !query 46 output +true + true + true + + +-- !query 47 +SELECT '' AS zero, BOOLTBL1.* + FROM BOOLTBL1 + WHERE booleq(boolean('false'), f1) +-- !query 47 schema +struct +-- !query 47 output + + + +-- !query 48 +INSERT INTO BOOLTBL1 VALUES (boolean('f')) +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +SELECT '' AS f_1, BOOLTBL1.* + FROM BOOLTBL1 + WHERE f1 = boolean('false') +-- !query 49 schema +struct +-- !query 49 output +false + + +-- !query 50 +CREATE TABLE BOOLTBL2 (f1 boolean) USING parquet +-- !query 50 schema +struct<> +-- !query 50 output + + + +-- !query 51 +INSERT INTO BOOLTBL2 VALUES (boolean('f')) +-- !query 51 schema +struct<> +-- !query 51 output + + + +-- !query 52 +INSERT INTO BOOLTBL2 VALUES (boolean('false')) +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +INSERT INTO BOOLTBL2 VALUES (boolean('False')) +-- !query 53 schema +struct<> +-- !query 53 output + + + +-- !query 54 +INSERT INTO BOOLTBL2 VALUES (boolean('FALSE')) +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +INSERT INTO BOOLTBL2 + VALUES (boolean('XXX')) +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +SELECT '' AS f_4, BOOLTBL2.* FROM BOOLTBL2 +-- !query 56 schema +struct +-- !query 56 output +NULL + false + false + false + false + + +-- !query 57 +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 <> BOOLTBL1.f1 +-- !query 57 schema +struct +-- !query 57 output +true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query 58 +SELECT '' AS tf_12, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE boolne(BOOLTBL2.f1,BOOLTBL1.f1) +-- !query 58 schema +struct +-- !query 58 output +true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query 59 +SELECT '' AS ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 and BOOLTBL1.f1 = boolean('false') +-- !query 59 schema +struct +-- !query 59 output +false false + false false + false false + false false + + +-- !query 60 +SELECT '' AS tf_12_ff_4, BOOLTBL1.*, BOOLTBL2.* + FROM BOOLTBL1, BOOLTBL2 + WHERE BOOLTBL2.f1 = BOOLTBL1.f1 or BOOLTBL1.f1 = boolean('true') + ORDER BY BOOLTBL1.f1, BOOLTBL2.f1 +-- !query 60 schema +struct +-- !query 60 output +false false + false false + false false + false false + true NULL + true NULL + true NULL + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + true false + + +-- !query 61 +CREATE TABLE BOOLTBL3 (d string, b boolean, o int) USING parquet +-- !query 61 schema +struct<> +-- !query 61 output + + + +-- !query 62 +INSERT INTO BOOLTBL3 VALUES ('true', true, 1) +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +INSERT INTO BOOLTBL3 VALUES ('false', false, 2) +-- !query 63 schema +struct<> +-- !query 63 output + + + +-- !query 64 +INSERT INTO BOOLTBL3 VALUES ('null', null, 3) +-- !query 64 schema +struct<> +-- !query 64 output + + + +-- !query 65 +CREATE TABLE booltbl4(isfalse boolean, istrue boolean, isnul boolean) USING parquet +-- !query 65 schema +struct<> +-- !query 65 output + + + +-- !query 66 +INSERT INTO booltbl4 VALUES (false, true, null) +-- !query 66 schema +struct<> +-- !query 66 output + + + +-- !query 67 +SELECT istrue AND isnul AND istrue FROM booltbl4 +-- !query 67 schema +struct<((istrue AND isnul) AND istrue):boolean> +-- !query 67 output +NULL + + +-- !query 68 +SELECT istrue AND istrue AND isnul FROM booltbl4 +-- !query 68 schema +struct<((istrue AND istrue) AND isnul):boolean> +-- !query 68 output +NULL + + +-- !query 69 +SELECT isnul AND istrue AND istrue FROM booltbl4 +-- !query 69 schema +struct<((isnul AND istrue) AND istrue):boolean> +-- !query 69 output +NULL + + +-- !query 70 +SELECT isfalse AND isnul AND istrue FROM booltbl4 +-- !query 70 schema +struct<((isfalse AND isnul) AND istrue):boolean> +-- !query 70 output +false + + +-- !query 71 +SELECT istrue AND isfalse AND isnul FROM booltbl4 +-- !query 71 schema +struct<((istrue AND isfalse) AND isnul):boolean> +-- !query 71 output +false + + +-- !query 72 +SELECT isnul AND istrue AND isfalse FROM booltbl4 +-- !query 72 schema +struct<((isnul AND istrue) AND isfalse):boolean> +-- !query 72 output +false + + +-- !query 73 +SELECT isfalse OR isnul OR isfalse FROM booltbl4 +-- !query 73 schema +struct<((isfalse OR isnul) OR isfalse):boolean> +-- !query 73 output +NULL + + +-- !query 74 +SELECT isfalse OR isfalse OR isnul FROM booltbl4 +-- !query 74 schema +struct<((isfalse OR isfalse) OR isnul):boolean> +-- !query 74 output +NULL + + +-- !query 75 +SELECT isnul OR isfalse OR isfalse FROM booltbl4 +-- !query 75 schema +struct<((isnul OR isfalse) OR isfalse):boolean> +-- !query 75 output +NULL + + +-- !query 76 +SELECT isfalse OR isnul OR istrue FROM booltbl4 +-- !query 76 schema +struct<((isfalse OR isnul) OR istrue):boolean> +-- !query 76 output +true + + +-- !query 77 +SELECT istrue OR isfalse OR isnul FROM booltbl4 +-- !query 77 schema +struct<((istrue OR isfalse) OR isnul):boolean> +-- !query 77 output +true + + +-- !query 78 +SELECT isnul OR istrue OR isfalse FROM booltbl4 +-- !query 78 schema +struct<((isnul OR istrue) OR isfalse):boolean> +-- !query 78 output +true + + +-- !query 79 +DROP TABLE BOOLTBL1 +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +DROP TABLE BOOLTBL2 +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +DROP TABLE BOOLTBL3 +-- !query 81 schema +struct<> +-- !query 81 output + + + +-- !query 82 +DROP TABLE BOOLTBL4 +-- !query 82 schema +struct<> +-- !query 82 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out new file mode 100644 index 0000000000000..f95adcde81b3f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/case.sql.out @@ -0,0 +1,409 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 35 + + +-- !query 0 +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + END AS `Simple WHEN` +-- !query 12 schema +struct +-- !query 12 output +3 3 + + +-- !query 13 +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + END AS `Simple default` +-- !query 13 schema +struct +-- !query 13 output + NULL + + +-- !query 14 +SELECT '3' AS `One`, + CASE + WHEN 1 < 2 THEN 3 + ELSE 4 + END AS `Simple ELSE` +-- !query 14 schema +struct +-- !query 14 output +3 3 + + +-- !query 15 +SELECT '4' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query 15 schema +struct +-- !query 15 output +4 4 + + +-- !query 16 +SELECT '6' AS `One`, + CASE + WHEN 1 > 2 THEN 3 + WHEN 4 < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query 16 schema +struct +-- !query 16 output +6 6 + + +-- !query 17 +SELECT '7' AS `None`, + CASE WHEN rand() < 0 THEN 1 + END AS `NULL on no matches` +-- !query 17 schema +struct +-- !query 17 output +7 NULL + + +-- !query 18 +SELECT CASE WHEN 1=0 THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + +-- !query 19 +SELECT CASE 1 WHEN 0 THEN 1/0 WHEN 1 THEN 1 ELSE 2/0 END +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT CASE WHEN i > 100 THEN 1/0 ELSE 0 END FROM case_tbl +-- !query 20 schema +struct 100) THEN (1 div 0) ELSE 0 END:int> +-- !query 20 output +0 +0 +0 +0 + + +-- !query 21 +SELECT CASE 'a' WHEN 'a' THEN 1 ELSE 2 END +-- !query 21 schema +struct +-- !query 21 output +1 + + +-- !query 22 +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query 22 schema +struct= 3 or Null:int> +-- !query 22 output +3 + 4 + NULL + NULL + + +-- !query 23 +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query 23 schema +struct +-- !query 23 output +1 + 2 + 6 + 8 + + +-- !query 24 +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query 24 schema +struct +-- !query 24 output +1 one + 2 two + 3 big + 4 big + + +-- !query 25 +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query 25 schema +struct +-- !query 25 output +big + big + one + two + + +-- !query 26 +SELECT * FROM CASE_TBL WHERE COALESCE(f,i) = 4 +-- !query 26 schema +struct +-- !query 26 output +4 NULL + + +-- !query 27 +SELECT * FROM CASE_TBL WHERE NULLIF(f,i) = 2 +-- !query 27 schema +struct +-- !query 27 output + + + +-- !query 28 +SELECT COALESCE(a.f, b.i, b.j) + FROM CASE_TBL a, CASE2_TBL b +-- !query 28 schema +struct +-- !query 28 output +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-6.0 +1.0 +1.0 +10.1 +10.1 +10.1 +10.1 +10.1 +10.1 +2.0 +2.0 +20.2 +20.2 +20.2 +20.2 +20.2 +20.2 +3.0 + + +-- !query 29 +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(a.f, b.i, b.j) = 2 +-- !query 29 schema +struct +-- !query 29 output +4 NULL 2 -2 +4 NULL 2 -4 + + +-- !query 30 +SELECT '' AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query 30 schema +struct +-- !query 30 output +1 2 + 1 2 + 1 3 + 1 NULL + 2 1 + 2 1 + 2 3 + 2 NULL + 3 1 + 3 1 + 3 2 + 3 2 + 3 NULL + 4 1 + 4 1 + 4 2 + 4 2 + 4 3 + 4 NULL + NULL 1 + NULL 1 + NULL 2 + NULL 2 + NULL 3 + + +-- !query 31 +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE COALESCE(f,b.i) = 2 +-- !query 31 schema +struct +-- !query 31 output +4 NULL 2 -2 + 4 NULL 2 -4 + + +-- !query 32 +SELECT CASE + (CASE vol('bar') + WHEN 'foo' THEN 'it was foo!' + WHEN vol(null) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN 'it was foo!' THEN 'foo recognized' + WHEN 'it was bar!' THEN 'bar recognized' + ELSE 'unrecognized' END +-- !query 32 schema +struct +-- !query 32 output +bar recognized + + +-- !query 33 +DROP TABLE CASE_TBL +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +DROP TABLE CASE2_TBL +-- !query 34 schema +struct<> +-- !query 34 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out new file mode 100644 index 0000000000000..0d669ae7ce5b5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/date.sql.out @@ -0,0 +1,515 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 48 + + +-- !query 0 +CREATE TABLE DATE_TBL (f1 date) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO DATE_TBL VALUES ('1957-04-09') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO DATE_TBL VALUES ('1957-06-13') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO DATE_TBL VALUES ('1996-02-28') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO DATE_TBL VALUES ('1996-02-29') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO DATE_TBL VALUES ('1996-03-01') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO DATE_TBL VALUES ('1996-03-02') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO DATE_TBL VALUES ('1997-02-28') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO DATE_TBL VALUES ('1997-03-01') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO DATE_TBL VALUES ('1997-03-02') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO DATE_TBL VALUES ('2000-04-01') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO DATE_TBL VALUES ('2000-04-02') +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO DATE_TBL VALUES ('2000-04-03') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO DATE_TBL VALUES ('2038-04-08') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO DATE_TBL VALUES ('2039-04-09') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO DATE_TBL VALUES ('2040-04-10') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +SELECT f1 AS `Fifteen` FROM DATE_TBL +-- !query 16 schema +struct +-- !query 16 output +1957-04-09 +1957-06-13 +1996-02-28 +1996-02-29 +1996-03-01 +1996-03-02 +1997-02-28 +1997-03-01 +1997-03-02 +2000-04-01 +2000-04-02 +2000-04-03 +2038-04-08 +2039-04-09 +2040-04-10 + + +-- !query 17 +SELECT f1 AS `Nine` FROM DATE_TBL WHERE f1 < '2000-01-01' +-- !query 17 schema +struct +-- !query 17 output +1957-04-09 +1957-06-13 +1996-02-28 +1996-02-29 +1996-03-01 +1996-03-02 +1997-02-28 +1997-03-01 +1997-03-02 + + +-- !query 18 +SELECT f1 AS `Three` FROM DATE_TBL + WHERE f1 BETWEEN '2000-01-01' AND '2001-01-01' +-- !query 18 schema +struct +-- !query 18 output +2000-04-01 +2000-04-02 +2000-04-03 + + +-- !query 19 +SELECT date '1999-01-08' +-- !query 19 schema +struct +-- !query 19 output +1999-01-08 + + +-- !query 20 +SELECT date '1999-01-18' +-- !query 20 schema +struct +-- !query 20 output +1999-01-18 + + +-- !query 21 +SELECT date '1999 Jan 08' +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ + + +-- !query 22 +SELECT date '1999 08 Jan' +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ + + +-- !query 23 +SELECT date '1999-01-08' +-- !query 23 schema +struct +-- !query 23 output +1999-01-08 + + +-- !query 24 +SELECT date '1999-08-01' +-- !query 24 schema +struct +-- !query 24 output +1999-08-01 + + +-- !query 25 +SELECT date '1999 01 08' +-- !query 25 schema +struct<> +-- !query 25 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ + + +-- !query 26 +SELECT date '1999 08 01' +-- !query 26 schema +struct<> +-- !query 26 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ + + +-- !query 27 +SELECT date '1999-01-08' +-- !query 27 schema +struct +-- !query 27 output +1999-01-08 + + +-- !query 28 +SELECT date '1999 Jan 08' +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ + + +-- !query 29 +SELECT date '1999 08 Jan' +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ + + +-- !query 30 +SELECT date '1999-01-08' +-- !query 30 schema +struct +-- !query 30 output +1999-01-08 + + +-- !query 31 +SELECT date '1999-08-01' +-- !query 31 schema +struct +-- !query 31 output +1999-08-01 + + +-- !query 32 +SELECT date '1999 01 08' +-- !query 32 schema +struct<> +-- !query 32 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ + + +-- !query 33 +SELECT date '1999 08 01' +-- !query 33 schema +struct<> +-- !query 33 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ + + +-- !query 34 +SELECT date '1999-01-08' +-- !query 34 schema +struct +-- !query 34 output +1999-01-08 + + +-- !query 35 +SELECT date '1999-01-18' +-- !query 35 schema +struct +-- !query 35 output +1999-01-18 + + +-- !query 36 +SELECT date '1999 Jan 08' +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 Jan 08(line 1, pos 7) + +== SQL == +SELECT date '1999 Jan 08' +-------^^^ + + +-- !query 37 +SELECT date '1999 08 Jan' +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 Jan(line 1, pos 7) + +== SQL == +SELECT date '1999 08 Jan' +-------^^^ + + +-- !query 38 +SELECT date '1999-01-08' +-- !query 38 schema +struct +-- !query 38 output +1999-01-08 + + +-- !query 39 +SELECT date '1999-08-01' +-- !query 39 schema +struct +-- !query 39 output +1999-08-01 + + +-- !query 40 +SELECT date '1999 01 08' +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 01 08(line 1, pos 7) + +== SQL == +SELECT date '1999 01 08' +-------^^^ + + +-- !query 41 +SELECT date '1999 08 01' +-- !query 41 schema +struct<> +-- !query 41 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 1999 08 01(line 1, pos 7) + +== SQL == +SELECT date '1999 08 01' +-------^^^ + + +-- !query 42 +SELECT date '4714-11-24 BC' +-- !query 42 schema +struct +-- !query 42 output +4714-11-24 + + +-- !query 43 +SELECT date '4714-11-23 BC' +-- !query 43 schema +struct +-- !query 43 output +4714-11-23 + + +-- !query 44 +-- out of range +SELECT date '5874897-12-31' +-- !query 44 schema +struct<> +-- !query 44 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 5874897-12-31(line 2, pos 7) + +== SQL == +-- out of range +SELECT date '5874897-12-31' +-------^^^ + + +-- !query 45 +SELECT date '5874898-01-01' +-- !query 45 schema +struct<> +-- !query 45 output +org.apache.spark.sql.catalyst.parser.ParseException + +Cannot parse the DATE value: 5874898-01-01(line 1, pos 7) + +== SQL == +SELECT date '5874898-01-01' +-------^^^ + + +-- !query 46 +-- out of range + + + +SELECT f1 - date '2000-01-01' AS `Days From 2K` FROM DATE_TBL +-- !query 46 schema +struct +-- !query 46 output +-1035 +-1036 +-1037 +-1400 +-1401 +-1402 +-1403 +-15542 +-15607 +13977 +14343 +14710 +91 +92 +93 + + +-- !query 47 +DROP TABLE DATE_TBL +-- !query 47 schema +struct<> +-- !query 47 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out new file mode 100644 index 0000000000000..f25b7f5911aeb --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float4.sql.out @@ -0,0 +1,379 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +CREATE TABLE FLOAT4_TBL (f1 float) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO FLOAT4_TBL VALUES (' 0.0') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO FLOAT4_TBL VALUES ('1004.30 ') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO FLOAT4_TBL VALUES (' -34.84 ') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e+20') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO FLOAT4_TBL VALUES ('1.2345678901234e-20') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT float('NaN') +-- !query 6 schema +struct +-- !query 6 output +NaN + + +-- !query 7 +SELECT float('nan') +-- !query 7 schema +struct +-- !query 7 output +NULL + + +-- !query 8 +SELECT float(' NAN ') +-- !query 8 schema +struct +-- !query 8 output +NULL + + +-- !query 9 +SELECT float('infinity') +-- !query 9 schema +struct +-- !query 9 output +NULL + + +-- !query 10 +SELECT float(' -INFINiTY ') +-- !query 10 schema +struct +-- !query 10 output +NULL + + +-- !query 11 +SELECT float('N A N') +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT float('NaN x') +-- !query 12 schema +struct +-- !query 12 output +NULL + + +-- !query 13 +SELECT float(' INFINITY x') +-- !query 13 schema +struct +-- !query 13 output +NULL + + +-- !query 14 +SELECT float('Infinity') + 100.0 +-- !query 14 schema +struct<(CAST(CAST(Infinity AS FLOAT) AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> +-- !query 14 output +Infinity + + +-- !query 15 +SELECT float('Infinity') / float('Infinity') +-- !query 15 schema +struct<(CAST(CAST(Infinity AS FLOAT) AS DOUBLE) / CAST(CAST(Infinity AS FLOAT) AS DOUBLE)):double> +-- !query 15 output +NaN + + +-- !query 16 +SELECT float('nan') / float('nan') +-- !query 16 schema +struct<(CAST(CAST(nan AS FLOAT) AS DOUBLE) / CAST(CAST(nan AS FLOAT) AS DOUBLE)):double> +-- !query 16 output +NULL + + +-- !query 17 +SELECT float(decimal('nan')) +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query 18 schema +struct +-- !query 18 output +-34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query 19 +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <> '1004.3' +-- !query 19 schema +struct +-- !query 19 output +-34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + + +-- !query 20 +SELECT '' AS one, f.* FROM FLOAT4_TBL f WHERE f.f1 = '1004.3' +-- !query 20 schema +struct +-- !query 20 output +1004.3 + + +-- !query 21 +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE '1004.3' > f.f1 +-- !query 21 schema +struct +-- !query 21 output +-34.84 + 0.0 + 1.2345679E-20 + + +-- !query 22 +SELECT '' AS three, f.* FROM FLOAT4_TBL f WHERE f.f1 < '1004.3' +-- !query 22 schema +struct +-- !query 22 output +-34.84 + 0.0 + 1.2345679E-20 + + +-- !query 23 +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE '1004.3' >= f.f1 +-- !query 23 schema +struct +-- !query 23 output +-34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query 24 +SELECT '' AS four, f.* FROM FLOAT4_TBL f WHERE f.f1 <= '1004.3' +-- !query 24 schema +struct +-- !query 24 output +-34.84 + 0.0 + 1.2345679E-20 + 1004.3 + + +-- !query 25 +SELECT '' AS three, f.f1, f.f1 * '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query 25 schema +struct +-- !query 25 output +1.2345679E-20 -1.2345678720289608E-19 + 1.2345679E20 -1.2345678955701443E21 + 1004.3 -10042.999877929688 + + +-- !query 26 +SELECT '' AS three, f.f1, f.f1 + '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query 26 schema +struct +-- !query 26 output +1.2345679E-20 -10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 994.2999877929688 + + +-- !query 27 +SELECT '' AS three, f.f1, f.f1 / '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query 27 schema +struct +-- !query 27 output +1.2345679E-20 -1.2345678720289608E-21 + 1.2345679E20 -1.2345678955701443E19 + 1004.3 -100.42999877929688 + + +-- !query 28 +SELECT '' AS three, f.f1, f.f1 - '-10' AS x FROM FLOAT4_TBL f + WHERE f.f1 > '0.0' +-- !query 28 schema +struct +-- !query 28 output +1.2345679E-20 10.0 + 1.2345679E20 1.2345678955701443E20 + 1004.3 1014.2999877929688 + + +-- !query 29 +SELECT '' AS five, * FROM FLOAT4_TBL +-- !query 29 schema +struct +-- !query 29 output +-34.84 + 0.0 + 1.2345679E-20 + 1.2345679E20 + 1004.3 + + +-- !query 30 +SELECT smallint(float('32767.4')) +-- !query 30 schema +struct +-- !query 30 output +32767 + + +-- !query 31 +SELECT smallint(float('32767.6')) +-- !query 31 schema +struct +-- !query 31 output +32767 + + +-- !query 32 +SELECT smallint(float('-32768.4')) +-- !query 32 schema +struct +-- !query 32 output +-32768 + + +-- !query 33 +SELECT smallint(float('-32768.6')) +-- !query 33 schema +struct +-- !query 33 output +-32768 + + +-- !query 34 +SELECT int(float('2147483520')) +-- !query 34 schema +struct +-- !query 34 output +2147483520 + + +-- !query 35 +SELECT int(float('2147483647')) +-- !query 35 schema +struct +-- !query 35 output +2147483647 + + +-- !query 36 +SELECT int(float('-2147483648.5')) +-- !query 36 schema +struct +-- !query 36 output +-2147483648 + + +-- !query 37 +SELECT int(float('-2147483900')) +-- !query 37 schema +struct +-- !query 37 output +-2147483648 + + +-- !query 38 +SELECT bigint(float('9223369837831520256')) +-- !query 38 schema +struct +-- !query 38 output +9223369837831520256 + + +-- !query 39 +SELECT bigint(float('9223372036854775807')) +-- !query 39 schema +struct +-- !query 39 output +9223372036854775807 + + +-- !query 40 +SELECT bigint(float('-9223372036854775808.5')) +-- !query 40 schema +struct +-- !query 40 output +-9223372036854775808 + + +-- !query 41 +SELECT bigint(float('-9223380000000000000')) +-- !query 41 schema +struct +-- !query 41 output +-9223372036854775808 + + +-- !query 42 +DROP TABLE FLOAT4_TBL +-- !query 42 schema +struct<> +-- !query 42 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out new file mode 100644 index 0000000000000..3e3f24d603ff0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/float8.sql.out @@ -0,0 +1,839 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 95 + + +-- !query 0 +CREATE TABLE FLOAT8_TBL(f1 double) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO FLOAT8_TBL VALUES (' 0.0 ') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO FLOAT8_TBL VALUES ('1004.30 ') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO FLOAT8_TBL VALUES (' -34.84') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e+200') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO FLOAT8_TBL VALUES ('1.2345678901234e-200') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT double('10e400') +-- !query 6 schema +struct +-- !query 6 output +Infinity + + +-- !query 7 +SELECT double('-10e400') +-- !query 7 schema +struct +-- !query 7 output +-Infinity + + +-- !query 8 +SELECT double('10e-400') +-- !query 8 schema +struct +-- !query 8 output +0.0 + + +-- !query 9 +SELECT double('-10e-400') +-- !query 9 schema +struct +-- !query 9 output +-0.0 + + +-- !query 10 +SELECT double('NaN') +-- !query 10 schema +struct +-- !query 10 output +NaN + + +-- !query 11 +SELECT double('nan') +-- !query 11 schema +struct +-- !query 11 output +NULL + + +-- !query 12 +SELECT double(' NAN ') +-- !query 12 schema +struct +-- !query 12 output +NULL + + +-- !query 13 +SELECT double('infinity') +-- !query 13 schema +struct +-- !query 13 output +NULL + + +-- !query 14 +SELECT double(' -INFINiTY ') +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +SELECT double('N A N') +-- !query 15 schema +struct +-- !query 15 output +NULL + + +-- !query 16 +SELECT double('NaN x') +-- !query 16 schema +struct +-- !query 16 output +NULL + + +-- !query 17 +SELECT double(' INFINITY x') +-- !query 17 schema +struct +-- !query 17 output +NULL + + +-- !query 18 +SELECT double('Infinity') + 100.0 +-- !query 18 schema +struct<(CAST(Infinity AS DOUBLE) + CAST(100.0 AS DOUBLE)):double> +-- !query 18 output +Infinity + + +-- !query 19 +SELECT double('Infinity') / double('Infinity') +-- !query 19 schema +struct<(CAST(Infinity AS DOUBLE) / CAST(Infinity AS DOUBLE)):double> +-- !query 19 output +NaN + + +-- !query 20 +SELECT double('NaN') / double('NaN') +-- !query 20 schema +struct<(CAST(NaN AS DOUBLE) / CAST(NaN AS DOUBLE)):double> +-- !query 20 output +NaN + + +-- !query 21 +SELECT double(decimal('nan')) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 22 schema +struct +-- !query 22 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 23 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <> '1004.3' +-- !query 23 schema +struct +-- !query 23 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + + +-- !query 24 +SELECT '' AS one, f.* FROM FLOAT8_TBL f WHERE f.f1 = '1004.3' +-- !query 24 schema +struct +-- !query 24 output +1004.3 + + +-- !query 25 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE '1004.3' > f.f1 +-- !query 25 schema +struct +-- !query 25 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 26 +SELECT '' AS three, f.* FROM FLOAT8_TBL f WHERE f.f1 < '1004.3' +-- !query 26 schema +struct +-- !query 26 output +-34.84 + 0.0 + 1.2345678901234E-200 + + +-- !query 27 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE '1004.3' >= f.f1 +-- !query 27 schema +struct +-- !query 27 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 28 +SELECT '' AS four, f.* FROM FLOAT8_TBL f WHERE f.f1 <= '1004.3' +-- !query 28 schema +struct +-- !query 28 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1004.3 + + +-- !query 29 +SELECT '' AS three, f.f1, f.f1 * '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 29 schema +struct +-- !query 29 output +1.2345678901234E-200 -1.2345678901234E-199 + 1.2345678901234E200 -1.2345678901234E201 + 1004.3 -10043.0 + + +-- !query 30 +SELECT '' AS three, f.f1, f.f1 + '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 30 schema +struct +-- !query 30 output +1.2345678901234E-200 -10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 994.3 + + +-- !query 31 +SELECT '' AS three, f.f1, f.f1 / '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 31 schema +struct +-- !query 31 output +1.2345678901234E-200 -1.2345678901234E-201 + 1.2345678901234E200 -1.2345678901234E199 + 1004.3 -100.42999999999999 + + +-- !query 32 +SELECT '' AS three, f.f1, f.f1 - '-10' AS x + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 32 schema +struct +-- !query 32 output +1.2345678901234E-200 10.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1014.3 + + +-- !query 33 +SELECT '' AS five, f.f1, round(f.f1) AS round_f1 + FROM FLOAT8_TBL f +-- !query 33 schema +struct +-- !query 33 output +-34.84 -35.0 + 0.0 0.0 + 1.2345678901234E-200 0.0 + 1.2345678901234E200 1.2345678901234E200 + 1004.3 1004.0 + + +-- !query 34 +select ceil(f1) as ceil_f1 from float8_tbl f +-- !query 34 schema +struct +-- !query 34 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 35 +select ceiling(f1) as ceiling_f1 from float8_tbl f +-- !query 35 schema +struct +-- !query 35 output +-34 +0 +1 +1005 +9223372036854775807 + + +-- !query 36 +select floor(f1) as floor_f1 from float8_tbl f +-- !query 36 schema +struct +-- !query 36 output +-35 +0 +0 +1004 +9223372036854775807 + + +-- !query 37 +select sign(f1) as sign_f1 from float8_tbl f +-- !query 37 schema +struct +-- !query 37 output +-1.0 +0.0 +1.0 +1.0 +1.0 + + +-- !query 38 +SELECT sqrt(double('64')) AS eight +-- !query 38 schema +struct +-- !query 38 output +8.0 + + +-- !query 39 +SELECT power(double('144'), double('0.5')) +-- !query 39 schema +struct +-- !query 39 output +12.0 + + +-- !query 40 +SELECT power(double('NaN'), double('0.5')) +-- !query 40 schema +struct +-- !query 40 output +NaN + + +-- !query 41 +SELECT power(double('144'), double('NaN')) +-- !query 41 schema +struct +-- !query 41 output +NaN + + +-- !query 42 +SELECT power(double('NaN'), double('NaN')) +-- !query 42 schema +struct +-- !query 42 output +NaN + + +-- !query 43 +SELECT power(double('-1'), double('NaN')) +-- !query 43 schema +struct +-- !query 43 output +NaN + + +-- !query 44 +SELECT power(double('1'), double('NaN')) +-- !query 44 schema +struct +-- !query 44 output +NaN + + +-- !query 45 +SELECT power(double('NaN'), double('0')) +-- !query 45 schema +struct +-- !query 45 output +1.0 + + +-- !query 46 +SELECT '' AS three, f.f1, exp(ln(f.f1)) AS exp_ln_f1 + FROM FLOAT8_TBL f + WHERE f.f1 > '0.0' +-- !query 46 schema +struct +-- !query 46 output +1.2345678901234E-200 1.2345678901233948E-200 + 1.2345678901234E200 1.234567890123379E200 + 1004.3 1004.3000000000004 + + +-- !query 47 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 47 schema +struct +-- !query 47 output +-34.84 + 0.0 + 1.2345678901234E-200 + 1.2345678901234E200 + 1004.3 + + +-- !query 48 +CREATE TEMPORARY VIEW UPDATED_FLOAT8_TBL as +SELECT + CASE WHEN FLOAT8_TBL.f1 > '0.0' THEN FLOAT8_TBL.f1 * '-1' ELSE FLOAT8_TBL.f1 END AS f1 +FROM FLOAT8_TBL +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +SELECT '' AS bad, f.f1 * '1e200' from UPDATED_FLOAT8_TBL f +-- !query 49 schema +struct +-- !query 49 output +-1.0042999999999999E203 + -1.2345678901234 + -3.484E201 + -Infinity + 0.0 + + +-- !query 50 +SELECT '' AS five, * FROM UPDATED_FLOAT8_TBL +-- !query 50 schema +struct +-- !query 50 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 51 +SELECT sinh(double('1')) +-- !query 51 schema +struct +-- !query 51 output +1.1752011936438014 + + +-- !query 52 +SELECT cosh(double('1')) +-- !query 52 schema +struct +-- !query 52 output +1.543080634815244 + + +-- !query 53 +SELECT tanh(double('1')) +-- !query 53 schema +struct +-- !query 53 output +0.7615941559557649 + + +-- !query 54 +SELECT asinh(double('1')) +-- !query 54 schema +struct +-- !query 54 output +0.8813735870195429 + + +-- !query 55 +SELECT acosh(double('2')) +-- !query 55 schema +struct +-- !query 55 output +1.3169578969248166 + + +-- !query 56 +SELECT atanh(double('0.5')) +-- !query 56 schema +struct +-- !query 56 output +0.5493061443340549 + + +-- !query 57 +SELECT sinh(double('Infinity')) +-- !query 57 schema +struct +-- !query 57 output +Infinity + + +-- !query 58 +SELECT sinh(double('-Infinity')) +-- !query 58 schema +struct +-- !query 58 output +-Infinity + + +-- !query 59 +SELECT sinh(double('NaN')) +-- !query 59 schema +struct +-- !query 59 output +NaN + + +-- !query 60 +SELECT cosh(double('Infinity')) +-- !query 60 schema +struct +-- !query 60 output +Infinity + + +-- !query 61 +SELECT cosh(double('-Infinity')) +-- !query 61 schema +struct +-- !query 61 output +Infinity + + +-- !query 62 +SELECT cosh(double('NaN')) +-- !query 62 schema +struct +-- !query 62 output +NaN + + +-- !query 63 +SELECT tanh(double('Infinity')) +-- !query 63 schema +struct +-- !query 63 output +1.0 + + +-- !query 64 +SELECT tanh(double('-Infinity')) +-- !query 64 schema +struct +-- !query 64 output +-1.0 + + +-- !query 65 +SELECT tanh(double('NaN')) +-- !query 65 schema +struct +-- !query 65 output +NaN + + +-- !query 66 +SELECT asinh(double('Infinity')) +-- !query 66 schema +struct +-- !query 66 output +Infinity + + +-- !query 67 +SELECT asinh(double('-Infinity')) +-- !query 67 schema +struct +-- !query 67 output +-Infinity + + +-- !query 68 +SELECT asinh(double('NaN')) +-- !query 68 schema +struct +-- !query 68 output +NaN + + +-- !query 69 +SELECT acosh(double('Infinity')) +-- !query 69 schema +struct +-- !query 69 output +Infinity + + +-- !query 70 +SELECT acosh(double('-Infinity')) +-- !query 70 schema +struct +-- !query 70 output +NaN + + +-- !query 71 +SELECT acosh(double('NaN')) +-- !query 71 schema +struct +-- !query 71 output +NaN + + +-- !query 72 +SELECT atanh(double('Infinity')) +-- !query 72 schema +struct +-- !query 72 output +NaN + + +-- !query 73 +SELECT atanh(double('-Infinity')) +-- !query 73 schema +struct +-- !query 73 output +NaN + + +-- !query 74 +SELECT atanh(double('NaN')) +-- !query 74 schema +struct +-- !query 74 output +NaN + + +-- !query 75 +TRUNCATE TABLE FLOAT8_TBL +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO FLOAT8_TBL VALUES ('0.0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO FLOAT8_TBL VALUES ('-34.84') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO FLOAT8_TBL VALUES ('-1004.30') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e+200') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO FLOAT8_TBL VALUES ('-1.2345678901234e-200') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +SELECT '' AS five, * FROM FLOAT8_TBL +-- !query 81 schema +struct +-- !query 81 output +-1.2345678901234E-200 + -1.2345678901234E200 + -1004.3 + -34.84 + 0.0 + + +-- !query 82 +SELECT smallint(double('32767.4')) +-- !query 82 schema +struct +-- !query 82 output +32767 + + +-- !query 83 +SELECT smallint(double('32767.6')) +-- !query 83 schema +struct +-- !query 83 output +32767 + + +-- !query 84 +SELECT smallint(double('-32768.4')) +-- !query 84 schema +struct +-- !query 84 output +-32768 + + +-- !query 85 +SELECT smallint(double('-32768.6')) +-- !query 85 schema +struct +-- !query 85 output +-32768 + + +-- !query 86 +SELECT int(double('2147483647.4')) +-- !query 86 schema +struct +-- !query 86 output +2147483647 + + +-- !query 87 +SELECT int(double('2147483647.6')) +-- !query 87 schema +struct +-- !query 87 output +2147483647 + + +-- !query 88 +SELECT int(double('-2147483648.4')) +-- !query 88 schema +struct +-- !query 88 output +-2147483648 + + +-- !query 89 +SELECT int(double('-2147483648.6')) +-- !query 89 schema +struct +-- !query 89 output +-2147483648 + + +-- !query 90 +SELECT bigint(double('9223372036854773760')) +-- !query 90 schema +struct +-- !query 90 output +9223372036854773760 + + +-- !query 91 +SELECT bigint(double('9223372036854775807')) +-- !query 91 schema +struct +-- !query 91 output +9223372036854775807 + + +-- !query 92 +SELECT bigint(double('-9223372036854775808.5')) +-- !query 92 schema +struct +-- !query 92 output +-9223372036854775808 + + +-- !query 93 +SELECT bigint(double('-9223372036854780000')) +-- !query 93 schema +struct +-- !query 93 output +-9223372036854775808 + + +-- !query 94 +DROP TABLE FLOAT8_TBL +-- !query 94 schema +struct<> +-- !query 94 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out new file mode 100644 index 0000000000000..7a7ce5f37dea4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int2.sql.out @@ -0,0 +1,363 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 35 + + +-- !query 0 +CREATE TABLE INT2_TBL(f1 smallint) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO INT2_TBL VALUES (trim('0 ')) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO INT2_TBL VALUES (trim(' 1234 ')) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO INT2_TBL VALUES (trim(' -1234')) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO INT2_TBL VALUES ('32767') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO INT2_TBL VALUES ('-32767') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT '' AS five, * FROM INT2_TBL +-- !query 6 schema +struct +-- !query 6 output +-1234 + -32767 + 0 + 1234 + 32767 + + +-- !query 7 +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> smallint('0') +-- !query 7 schema +struct +-- !query 7 output +-1234 + -32767 + 1234 + 32767 + + +-- !query 8 +SELECT '' AS four, i.* FROM INT2_TBL i WHERE i.f1 <> int('0') +-- !query 8 schema +struct +-- !query 8 output +-1234 + -32767 + 1234 + 32767 + + +-- !query 9 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = smallint('0') +-- !query 9 schema +struct +-- !query 9 output +0 + + +-- !query 10 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE i.f1 = int('0') +-- !query 10 schema +struct +-- !query 10 output +0 + + +-- !query 11 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < smallint('0') +-- !query 11 schema +struct +-- !query 11 output +-1234 + -32767 + + +-- !query 12 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 < int('0') +-- !query 12 schema +struct +-- !query 12 output +-1234 + -32767 + + +-- !query 13 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= smallint('0') +-- !query 13 schema +struct +-- !query 13 output +-1234 + -32767 + 0 + + +-- !query 14 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 <= int('0') +-- !query 14 schema +struct +-- !query 14 output +-1234 + -32767 + 0 + + +-- !query 15 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > smallint('0') +-- !query 15 schema +struct +-- !query 15 output +1234 + 32767 + + +-- !query 16 +SELECT '' AS two, i.* FROM INT2_TBL i WHERE i.f1 > int('0') +-- !query 16 schema +struct +-- !query 16 output +1234 + 32767 + + +-- !query 17 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= smallint('0') +-- !query 17 schema +struct +-- !query 17 output +0 + 1234 + 32767 + + +-- !query 18 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE i.f1 >= int('0') +-- !query 18 schema +struct +-- !query 18 output +0 + 1234 + 32767 + + +-- !query 19 +SELECT '' AS one, i.* FROM INT2_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query 19 schema +struct +-- !query 19 output +32767 + + +-- !query 20 +SELECT '' AS three, i.* FROM INT2_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query 20 schema +struct +-- !query 20 output +-1234 + 0 + 1234 + + +-- !query 21 +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT2_TBL i +WHERE abs(f1) < 16384 +-- !query 21 schema +struct +-- !query 21 output +-1234 -2468 + 0 0 + 1234 2468 + + +-- !query 22 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT2_TBL i +-- !query 22 schema +struct +-- !query 22 output +-1234 -2468 + -32767 -65534 + 0 0 + 1234 2468 + 32767 65534 + + +-- !query 23 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT2_TBL i +WHERE f1 < 32766 +-- !query 23 schema +struct +-- !query 23 output +-1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + + +-- !query 24 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT2_TBL i +-- !query 24 schema +struct +-- !query 24 output +-1234 -1232 + -32767 -32765 + 0 2 + 1234 1236 + 32767 32769 + + +-- !query 25 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT2_TBL i +WHERE f1 > -32767 +-- !query 25 schema +struct +-- !query 25 output +-1234 -1236 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query 26 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT2_TBL i +-- !query 26 schema +struct +-- !query 26 output +-1234 -1236 + -32767 -32769 + 0 -2 + 1234 1232 + 32767 32765 + + +-- !query 27 +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT2_TBL i +-- !query 27 schema +struct +-- !query 27 output +-1234 -617 + -32767 -16383 + 0 0 + 1234 617 + 32767 16383 + + +-- !query 28 +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT2_TBL i +-- !query 28 schema +struct +-- !query 28 output +-1234 -617 + -32767 -16383 + 0 0 + 1234 617 + 32767 16383 + + +-- !query 29 +SELECT string(shiftleft(smallint(-1), 15)) +-- !query 29 schema +struct +-- !query 29 output +-32768 + + +-- !query 30 +SELECT string(smallint(shiftleft(smallint(-1), 15))+1) +-- !query 30 schema +struct +-- !query 30 output +-32767 + + +-- !query 31 +SELECT smallint(-32768) % smallint(-1) +-- !query 31 schema +struct<(CAST(-32768 AS SMALLINT) % CAST(-1 AS SMALLINT)):smallint> +-- !query 31 output +0 + + +-- !query 32 +SELECT x, smallint(x) AS int2_value +FROM (VALUES float(-2.5), + float(-1.5), + float(-0.5), + float(0.0), + float(0.5), + float(1.5), + float(2.5)) t(x) +-- !query 32 schema +struct +-- !query 32 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 33 +SELECT x, smallint(x) AS int2_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query 33 schema +struct +-- !query 33 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 34 +DROP TABLE INT2_TBL +-- !query 34 schema +struct<> +-- !query 34 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out new file mode 100644 index 0000000000000..456b1ef962d46 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int4.sql.out @@ -0,0 +1,530 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 53 + + +-- !query 0 +CREATE TABLE INT4_TBL(f1 int) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO INT4_TBL VALUES (trim(' 0 ')) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO INT4_TBL VALUES (trim('123456 ')) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO INT4_TBL VALUES (trim(' -123456')) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO INT4_TBL VALUES ('2147483647') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO INT4_TBL VALUES ('-2147483647') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT '' AS five, * FROM INT4_TBL +-- !query 6 schema +struct +-- !query 6 output +-123456 + -2147483647 + 0 + 123456 + 2147483647 + + +-- !query 7 +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> smallint('0') +-- !query 7 schema +struct +-- !query 7 output +-123456 + -2147483647 + 123456 + 2147483647 + + +-- !query 8 +SELECT '' AS four, i.* FROM INT4_TBL i WHERE i.f1 <> int('0') +-- !query 8 schema +struct +-- !query 8 output +-123456 + -2147483647 + 123456 + 2147483647 + + +-- !query 9 +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = smallint('0') +-- !query 9 schema +struct +-- !query 9 output +0 + + +-- !query 10 +SELECT '' AS one, i.* FROM INT4_TBL i WHERE i.f1 = int('0') +-- !query 10 schema +struct +-- !query 10 output +0 + + +-- !query 11 +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < smallint('0') +-- !query 11 schema +struct +-- !query 11 output +-123456 + -2147483647 + + +-- !query 12 +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 < int('0') +-- !query 12 schema +struct +-- !query 12 output +-123456 + -2147483647 + + +-- !query 13 +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= smallint('0') +-- !query 13 schema +struct +-- !query 13 output +-123456 + -2147483647 + 0 + + +-- !query 14 +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 <= int('0') +-- !query 14 schema +struct +-- !query 14 output +-123456 + -2147483647 + 0 + + +-- !query 15 +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > smallint('0') +-- !query 15 schema +struct +-- !query 15 output +123456 + 2147483647 + + +-- !query 16 +SELECT '' AS two, i.* FROM INT4_TBL i WHERE i.f1 > int('0') +-- !query 16 schema +struct +-- !query 16 output +123456 + 2147483647 + + +-- !query 17 +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= smallint('0') +-- !query 17 schema +struct +-- !query 17 output +0 + 123456 + 2147483647 + + +-- !query 18 +SELECT '' AS three, i.* FROM INT4_TBL i WHERE i.f1 >= int('0') +-- !query 18 schema +struct +-- !query 18 output +0 + 123456 + 2147483647 + + +-- !query 19 +SELECT '' AS one, i.* FROM INT4_TBL i WHERE (i.f1 % smallint('2')) = smallint('1') +-- !query 19 schema +struct +-- !query 19 output +2147483647 + + +-- !query 20 +SELECT '' AS three, i.* FROM INT4_TBL i WHERE (i.f1 % int('2')) = smallint('0') +-- !query 20 schema +struct +-- !query 20 output +-123456 + 0 + 123456 + + +-- !query 21 +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +-- !query 21 schema +struct +-- !query 21 output +-123456 -246912 + -2147483647 2 + 0 0 + 123456 246912 + 2147483647 -2 + + +-- !query 22 +SELECT '' AS five, i.f1, i.f1 * smallint('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query 22 schema +struct +-- !query 22 output +-123456 -246912 + 0 0 + 123456 246912 + + +-- !query 23 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +-- !query 23 schema +struct +-- !query 23 output +-123456 -246912 + -2147483647 2 + 0 0 + 123456 246912 + 2147483647 -2 + + +-- !query 24 +SELECT '' AS five, i.f1, i.f1 * int('2') AS x FROM INT4_TBL i +WHERE abs(f1) < 1073741824 +-- !query 24 schema +struct +-- !query 24 output +-123456 -246912 + 0 0 + 123456 246912 + + +-- !query 25 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +-- !query 25 schema +struct +-- !query 25 output +-123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + 2147483647 -2147483647 + + +-- !query 26 +SELECT '' AS five, i.f1, i.f1 + smallint('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query 26 schema +struct +-- !query 26 output +-123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + + +-- !query 27 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +-- !query 27 schema +struct +-- !query 27 output +-123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + 2147483647 -2147483647 + + +-- !query 28 +SELECT '' AS five, i.f1, i.f1 + int('2') AS x FROM INT4_TBL i +WHERE f1 < 2147483646 +-- !query 28 schema +struct +-- !query 28 output +-123456 -123454 + -2147483647 -2147483645 + 0 2 + 123456 123458 + + +-- !query 29 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +-- !query 29 schema +struct +-- !query 29 output +-123456 -123458 + -2147483647 2147483647 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query 30 +SELECT '' AS five, i.f1, i.f1 - smallint('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query 30 schema +struct +-- !query 30 output +-123456 -123458 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query 31 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +-- !query 31 schema +struct +-- !query 31 output +-123456 -123458 + -2147483647 2147483647 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query 32 +SELECT '' AS five, i.f1, i.f1 - int('2') AS x FROM INT4_TBL i +WHERE f1 > -2147483647 +-- !query 32 schema +struct +-- !query 32 output +-123456 -123458 + 0 -2 + 123456 123454 + 2147483647 2147483645 + + +-- !query 33 +SELECT '' AS five, i.f1, i.f1 / smallint('2') AS x FROM INT4_TBL i +-- !query 33 schema +struct +-- !query 33 output +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 + + +-- !query 34 +SELECT '' AS five, i.f1, i.f1 / int('2') AS x FROM INT4_TBL i +-- !query 34 schema +struct +-- !query 34 output +-123456 -61728 + -2147483647 -1073741823 + 0 0 + 123456 61728 + 2147483647 1073741823 + + +-- !query 35 +SELECT -2+3 AS one +-- !query 35 schema +struct +-- !query 35 output +1 + + +-- !query 36 +SELECT 4-2 AS two +-- !query 36 schema +struct +-- !query 36 output +2 + + +-- !query 37 +SELECT 2- -1 AS three +-- !query 37 schema +struct +-- !query 37 output +3 + + +-- !query 38 +SELECT 2 - -2 AS four +-- !query 38 schema +struct +-- !query 38 output +4 + + +-- !query 39 +SELECT smallint('2') * smallint('2') = smallint('16') / smallint('4') AS true +-- !query 39 schema +struct +-- !query 39 output +true + + +-- !query 40 +SELECT int('2') * smallint('2') = smallint('16') / int('4') AS true +-- !query 40 schema +struct +-- !query 40 output +true + + +-- !query 41 +SELECT smallint('2') * int('2') = int('16') / smallint('4') AS true +-- !query 41 schema +struct +-- !query 41 output +true + + +-- !query 42 +SELECT int('1000') < int('999') AS `false` +-- !query 42 schema +struct +-- !query 42 output +false + + +-- !query 43 +SELECT 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 + 1 AS ten +-- !query 43 schema +struct +-- !query 43 output +10 + + +-- !query 44 +SELECT 2 + 2 / 2 AS three +-- !query 44 schema +struct +-- !query 44 output +3 + + +-- !query 45 +SELECT (2 + 2) / 2 AS two +-- !query 45 schema +struct +-- !query 45 output +2 + + +-- !query 46 +SELECT string(shiftleft(int(-1), 31)) +-- !query 46 schema +struct +-- !query 46 output +-2147483648 + + +-- !query 47 +SELECT string(int(shiftleft(int(-1), 31))+1) +-- !query 47 schema +struct +-- !query 47 output +-2147483647 + + +-- !query 48 +SELECT int(-2147483648) % int(-1) +-- !query 48 schema +struct<(CAST(-2147483648 AS INT) % CAST(-1 AS INT)):int> +-- !query 48 output +0 + + +-- !query 49 +SELECT int(-2147483648) % smallint(-1) +-- !query 49 schema +struct<(CAST(-2147483648 AS INT) % CAST(CAST(-1 AS SMALLINT) AS INT)):int> +-- !query 49 output +0 + + +-- !query 50 +SELECT x, int(x) AS int4_value +FROM (VALUES double(-2.5), + double(-1.5), + double(-0.5), + double(0.0), + double(0.5), + double(1.5), + double(2.5)) t(x) +-- !query 50 schema +struct +-- !query 50 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 51 +SELECT x, int(x) AS int4_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query 51 schema +struct +-- !query 51 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 52 +DROP TABLE INT4_TBL +-- !query 52 schema +struct<> +-- !query 52 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out new file mode 100644 index 0000000000000..6d7fae19aa7e4 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/int8.sql.out @@ -0,0 +1,855 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 85 + + +-- !query 0 +CREATE TABLE INT8_TBL(q1 bigint, q2 bigint) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO INT8_TBL VALUES(trim(' 123 '),trim(' 456')) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO INT8_TBL VALUES(trim('123 '),'4567890123456789') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO INT8_TBL VALUES('4567890123456789','123') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO INT8_TBL VALUES(+4567890123456789,'4567890123456789') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO INT8_TBL VALUES('+4567890123456789','-4567890123456789') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT * FROM INT8_TBL +-- !query 6 schema +struct +-- !query 6 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 7 +SELECT * FROM INT8_TBL WHERE q2 = 4567890123456789 +-- !query 7 schema +struct +-- !query 7 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 8 +SELECT * FROM INT8_TBL WHERE q2 <> 4567890123456789 +-- !query 8 schema +struct +-- !query 8 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 9 +SELECT * FROM INT8_TBL WHERE q2 < 4567890123456789 +-- !query 9 schema +struct +-- !query 9 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 10 +SELECT * FROM INT8_TBL WHERE q2 > 4567890123456789 +-- !query 10 schema +struct +-- !query 10 output + + + +-- !query 11 +SELECT * FROM INT8_TBL WHERE q2 <= 4567890123456789 +-- !query 11 schema +struct +-- !query 11 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 12 +SELECT * FROM INT8_TBL WHERE q2 >= 4567890123456789 +-- !query 12 schema +struct +-- !query 12 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 13 +SELECT * FROM INT8_TBL WHERE q2 = 456 +-- !query 13 schema +struct +-- !query 13 output +123 456 + + +-- !query 14 +SELECT * FROM INT8_TBL WHERE q2 <> 456 +-- !query 14 schema +struct +-- !query 14 output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 15 +SELECT * FROM INT8_TBL WHERE q2 < 456 +-- !query 15 schema +struct +-- !query 15 output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 16 +SELECT * FROM INT8_TBL WHERE q2 > 456 +-- !query 16 schema +struct +-- !query 16 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 17 +SELECT * FROM INT8_TBL WHERE q2 <= 456 +-- !query 17 schema +struct +-- !query 17 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 18 +SELECT * FROM INT8_TBL WHERE q2 >= 456 +-- !query 18 schema +struct +-- !query 18 output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 19 +SELECT * FROM INT8_TBL WHERE 123 = q1 +-- !query 19 schema +struct +-- !query 19 output +123 456 +123 4567890123456789 + + +-- !query 20 +SELECT * FROM INT8_TBL WHERE 123 <> q1 +-- !query 20 schema +struct +-- !query 20 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 21 +SELECT * FROM INT8_TBL WHERE 123 < q1 +-- !query 21 schema +struct +-- !query 21 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 22 +SELECT * FROM INT8_TBL WHERE 123 > q1 +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT * FROM INT8_TBL WHERE 123 <= q1 +-- !query 23 schema +struct +-- !query 23 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 24 +SELECT * FROM INT8_TBL WHERE 123 >= q1 +-- !query 24 schema +struct +-- !query 24 output +123 456 +123 4567890123456789 + + +-- !query 25 +SELECT * FROM INT8_TBL WHERE q2 = smallint('456') +-- !query 25 schema +struct +-- !query 25 output +123 456 + + +-- !query 26 +SELECT * FROM INT8_TBL WHERE q2 <> smallint('456') +-- !query 26 schema +struct +-- !query 26 output +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 27 +SELECT * FROM INT8_TBL WHERE q2 < smallint('456') +-- !query 27 schema +struct +-- !query 27 output +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 28 +SELECT * FROM INT8_TBL WHERE q2 > smallint('456') +-- !query 28 schema +struct +-- !query 28 output +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 29 +SELECT * FROM INT8_TBL WHERE q2 <= smallint('456') +-- !query 29 schema +struct +-- !query 29 output +123 456 +4567890123456789 -4567890123456789 +4567890123456789 123 + + +-- !query 30 +SELECT * FROM INT8_TBL WHERE q2 >= smallint('456') +-- !query 30 schema +struct +-- !query 30 output +123 456 +123 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 31 +SELECT * FROM INT8_TBL WHERE smallint('123') = q1 +-- !query 31 schema +struct +-- !query 31 output +123 456 +123 4567890123456789 + + +-- !query 32 +SELECT * FROM INT8_TBL WHERE smallint('123') <> q1 +-- !query 32 schema +struct +-- !query 32 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 33 +SELECT * FROM INT8_TBL WHERE smallint('123') < q1 +-- !query 33 schema +struct +-- !query 33 output +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 34 +SELECT * FROM INT8_TBL WHERE smallint('123') > q1 +-- !query 34 schema +struct +-- !query 34 output + + + +-- !query 35 +SELECT * FROM INT8_TBL WHERE smallint('123') <= q1 +-- !query 35 schema +struct +-- !query 35 output +123 456 +123 4567890123456789 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 + + +-- !query 36 +SELECT * FROM INT8_TBL WHERE smallint('123') >= q1 +-- !query 36 schema +struct +-- !query 36 output +123 456 +123 4567890123456789 + + +-- !query 37 +SELECT '' AS five, q1 AS plus, -q1 AS `minus` FROM INT8_TBL +-- !query 37 schema +struct +-- !query 37 output +123 -123 + 123 -123 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + 4567890123456789 -4567890123456789 + + +-- !query 38 +SELECT '' AS five, q1, q2, q1 + q2 AS plus FROM INT8_TBL +-- !query 38 schema +struct +-- !query 38 output +123 456 579 + 123 4567890123456789 4567890123456912 + 4567890123456789 -4567890123456789 0 + 4567890123456789 123 4567890123456912 + 4567890123456789 4567890123456789 9135780246913578 + + +-- !query 39 +SELECT '' AS five, q1, q2, q1 - q2 AS `minus` FROM INT8_TBL +-- !query 39 schema +struct +-- !query 39 output +123 456 -333 + 123 4567890123456789 -4567890123456666 + 4567890123456789 -4567890123456789 9135780246913578 + 4567890123456789 123 4567890123456666 + 4567890123456789 4567890123456789 0 + + +-- !query 40 +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL +-- !query 40 schema +struct +-- !query 40 output +123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 -4567890123456789 -4868582358072306617 + 4567890123456789 123 561850485185185047 + 4567890123456789 4567890123456789 4868582358072306617 + + +-- !query 41 +SELECT '' AS three, q1, q2, q1 * q2 AS multiply FROM INT8_TBL + WHERE q1 < 1000 or (q2 > 0 and q2 < 1000) +-- !query 41 schema +struct +-- !query 41 output +123 456 56088 + 123 4567890123456789 561850485185185047 + 4567890123456789 123 561850485185185047 + + +-- !query 42 +SELECT '' AS five, q1, q2, q1 / q2 AS divide, q1 % q2 AS mod FROM INT8_TBL +-- !query 42 schema +struct +-- !query 42 output +123 456 0 123 + 123 4567890123456789 0 123 + 4567890123456789 -4567890123456789 -1 0 + 4567890123456789 123 37137318076884 57 + 4567890123456789 4567890123456789 1 0 + + +-- !query 43 +SELECT '' AS five, q1, double(q1) FROM INT8_TBL +-- !query 43 schema +struct +-- !query 43 output +123 123.0 + 123 123.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query 44 +SELECT '' AS five, q2, double(q2) FROM INT8_TBL +-- !query 44 schema +struct +-- !query 44 output +-4567890123456789 -4.567890123456789E15 + 123 123.0 + 456 456.0 + 4567890123456789 4.567890123456789E15 + 4567890123456789 4.567890123456789E15 + + +-- !query 45 +SELECT 37 + q1 AS plus4 FROM INT8_TBL +-- !query 45 schema +struct +-- !query 45 output +160 +160 +4567890123456826 +4567890123456826 +4567890123456826 + + +-- !query 46 +SELECT 37 - q1 AS minus4 FROM INT8_TBL +-- !query 46 schema +struct +-- !query 46 output +-4567890123456752 +-4567890123456752 +-4567890123456752 +-86 +-86 + + +-- !query 47 +SELECT '' AS five, 2 * q1 AS `twice int4` FROM INT8_TBL +-- !query 47 schema +struct +-- !query 47 output +246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query 48 +SELECT '' AS five, q1 * 2 AS `twice int4` FROM INT8_TBL +-- !query 48 schema +struct +-- !query 48 output +246 + 246 + 9135780246913578 + 9135780246913578 + 9135780246913578 + + +-- !query 49 +SELECT q1 + int(42) AS `8plus4`, q1 - int(42) AS `8minus4`, q1 * int(42) AS `8mul4`, q1 / int(42) AS `8div4` FROM INT8_TBL +-- !query 49 schema +struct<8plus4:bigint,8minus4:bigint,8mul4:bigint,8div4:bigint> +-- !query 49 output +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 + + +-- !query 50 +SELECT int(246) + q1 AS `4plus8`, int(246) - q1 AS `4minus8`, int(246) * q1 AS `4mul8`, int(246) / q1 AS `4div8` FROM INT8_TBL +-- !query 50 schema +struct<4plus8:bigint,4minus8:bigint,4mul8:bigint,4div8:bigint> +-- !query 50 output +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 + + +-- !query 51 +SELECT q1 + smallint(42) AS `8plus2`, q1 - smallint(42) AS `8minus2`, q1 * smallint(42) AS `8mul2`, q1 / smallint(42) AS `8div2` FROM INT8_TBL +-- !query 51 schema +struct<8plus2:bigint,8minus2:bigint,8mul2:bigint,8div2:bigint> +-- !query 51 output +165 81 5166 2 +165 81 5166 2 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 +4567890123456831 4567890123456747 191851385185185138 108759288653733 + + +-- !query 52 +SELECT smallint(246) + q1 AS `2plus8`, smallint(246) - q1 AS `2minus8`, smallint(246) * q1 AS `2mul8`, smallint(246) / q1 AS `2div8` FROM INT8_TBL +-- !query 52 schema +struct<2plus8:bigint,2minus8:bigint,2mul8:bigint,2div8:bigint> +-- !query 52 output +369 123 30258 2 +369 123 30258 2 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 +4567890123457035 -4567890123456543 1123700970370370094 0 + + +-- !query 53 +SELECT q2, abs(q2) FROM INT8_TBL +-- !query 53 schema +struct +-- !query 53 output +-4567890123456789 4567890123456789 +123 123 +456 456 +4567890123456789 4567890123456789 +4567890123456789 4567890123456789 + + +-- !query 54 +SELECT min(q1), min(q2) FROM INT8_TBL +-- !query 54 schema +struct +-- !query 54 output +123 -4567890123456789 + + +-- !query 55 +SELECT max(q1), max(q2) FROM INT8_TBL +-- !query 55 schema +struct +-- !query 55 output +4567890123456789 4567890123456789 + + +-- !query 56 +select bigint('9223372036854775800') / bigint('0') +-- !query 56 schema +struct<(CAST(9223372036854775800 AS BIGINT) div CAST(0 AS BIGINT)):bigint> +-- !query 56 output +NULL + + +-- !query 57 +select bigint('-9223372036854775808') / smallint('0') +-- !query 57 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(0 AS SMALLINT) AS BIGINT)):bigint> +-- !query 57 output +NULL + + +-- !query 58 +select smallint('100') / bigint('0') +-- !query 58 schema +struct<(CAST(CAST(100 AS SMALLINT) AS BIGINT) div CAST(0 AS BIGINT)):bigint> +-- !query 58 output +NULL + + +-- !query 59 +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 = 456 +-- !query 59 schema +struct +-- !query 59 output +123 + + +-- !query 60 +SELECT CAST(q1 AS int) FROM int8_tbl WHERE q2 <> 456 +-- !query 60 schema +struct +-- !query 60 output +-869367531 +-869367531 +-869367531 +123 + + +-- !query 61 +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 = 456 +-- !query 61 schema +struct +-- !query 61 output +123 + + +-- !query 62 +SELECT CAST(q1 AS smallint) FROM int8_tbl WHERE q2 <> 456 +-- !query 62 schema +struct +-- !query 62 output +-32491 +-32491 +-32491 +123 + + +-- !query 63 +SELECT CAST(smallint('42') AS bigint), CAST(smallint('-37') AS bigint) +-- !query 63 schema +struct +-- !query 63 output +42 -37 + + +-- !query 64 +SELECT CAST(q1 AS float), CAST(q2 AS double) FROM INT8_TBL +-- !query 64 schema +struct +-- !query 64 output +123.0 4.567890123456789E15 +123.0 456.0 +4.5678899E15 -4.567890123456789E15 +4.5678899E15 123.0 +4.5678899E15 4.567890123456789E15 + + +-- !query 65 +SELECT CAST(float('36854775807.0') AS bigint) +-- !query 65 schema +struct +-- !query 65 output +36854775808 + + +-- !query 66 +SELECT CAST(double('922337203685477580700.0') AS bigint) +-- !query 66 schema +struct +-- !query 66 output +9223372036854775807 + + +-- !query 67 +SELECT q1, q2, q1 & q2 AS `and`, q1 | q2 AS `or`, ~q1 AS `not` FROM INT8_TBL +-- !query 67 schema +struct +-- !query 67 output +123 456 72 507 -124 +123 4567890123456789 17 4567890123456895 -124 +4567890123456789 -4567890123456789 1 -1 -4567890123456790 +4567890123456789 123 17 4567890123456895 -4567890123456790 +4567890123456789 4567890123456789 4567890123456789 4567890123456789 -4567890123456790 + + +-- !query 68 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799')) +-- !query 68 schema +struct +-- !query 68 output +4567890123456789 +4567890123456790 +4567890123456791 +4567890123456792 +4567890123456793 +4567890123456794 +4567890123456795 +4567890123456796 +4567890123456797 +4567890123456798 + + +-- !query 69 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 0) +-- !query 69 schema +struct<> +-- !query 69 output +java.lang.IllegalArgumentException +requirement failed: step (0) cannot be 0 + + +-- !query 70 +SELECT * FROM range(bigint('+4567890123456789'), bigint('+4567890123456799'), 2) +-- !query 70 schema +struct +-- !query 70 output +4567890123456789 +4567890123456791 +4567890123456793 +4567890123456795 +4567890123456797 + + +-- !query 71 +SELECT string(shiftleft(bigint(-1), 63)) +-- !query 71 schema +struct +-- !query 71 output +-9223372036854775808 + + +-- !query 72 +SELECT string(int(shiftleft(bigint(-1), 63))+1) +-- !query 72 schema +struct +-- !query 72 output +1 + + +-- !query 73 +SELECT bigint((-9223372036854775808)) * bigint((-1)) +-- !query 73 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(-1 AS BIGINT)):bigint> +-- !query 73 output +-9223372036854775808 + + +-- !query 74 +SELECT bigint((-9223372036854775808)) / bigint((-1)) +-- !query 74 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(-1 AS BIGINT)):bigint> +-- !query 74 output +-9223372036854775808 + + +-- !query 75 +SELECT bigint((-9223372036854775808)) % bigint((-1)) +-- !query 75 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(-1 AS BIGINT)):bigint> +-- !query 75 output +0 + + +-- !query 76 +SELECT bigint((-9223372036854775808)) * int((-1)) +-- !query 76 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 76 output +-9223372036854775808 + + +-- !query 77 +SELECT bigint((-9223372036854775808)) / int((-1)) +-- !query 77 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 77 output +-9223372036854775808 + + +-- !query 78 +SELECT bigint((-9223372036854775808)) % int((-1)) +-- !query 78 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS INT) AS BIGINT)):bigint> +-- !query 78 output +0 + + +-- !query 79 +SELECT bigint((-9223372036854775808)) * smallint((-1)) +-- !query 79 schema +struct<(CAST(-9223372036854775808 AS BIGINT) * CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 79 output +-9223372036854775808 + + +-- !query 80 +SELECT bigint((-9223372036854775808)) / smallint((-1)) +-- !query 80 schema +struct<(CAST(-9223372036854775808 AS BIGINT) div CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 80 output +-9223372036854775808 + + +-- !query 81 +SELECT bigint((-9223372036854775808)) % smallint((-1)) +-- !query 81 schema +struct<(CAST(-9223372036854775808 AS BIGINT) % CAST(CAST(-1 AS SMALLINT) AS BIGINT)):bigint> +-- !query 81 output +0 + + +-- !query 82 +SELECT x, bigint(x) AS int8_value +FROM (VALUES (double(-2.5)), + (double(-1.5)), + (double(-0.5)), + (double(0.0)), + (double(0.5)), + (double(1.5)), + (double(2.5))) t(x) +-- !query 82 schema +struct +-- !query 82 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0.0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 83 +SELECT x, bigint(x) AS int8_value +FROM (VALUES cast(-2.5 as decimal(38, 18)), + cast(-1.5 as decimal(38, 18)), + cast(-0.5 as decimal(38, 18)), + cast(-0.0 as decimal(38, 18)), + cast(0.5 as decimal(38, 18)), + cast(1.5 as decimal(38, 18)), + cast(2.5 as decimal(38, 18))) t(x) +-- !query 83 schema +struct +-- !query 83 output +-0.5 0 +-1.5 -1 +-2.5 -2 +0 0 +0.5 0 +1.5 1 +2.5 2 + + +-- !query 84 +DROP TABLE INT8_TBL +-- !query 84 schema +struct<> +-- !query 84 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out new file mode 100644 index 0000000000000..04425fd4b5444 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/numeric.sql.out @@ -0,0 +1,4832 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 573 + + +-- !query 0 +CREATE TABLE num_data (id int, val decimal(38,10)) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE num_exp_add (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TABLE num_exp_sub (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE num_exp_div (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TABLE num_exp_mul (id1 int, id2 int, expected decimal(38,10)) USING parquet +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TABLE num_exp_sqrt (id int, expected decimal(38,10)) USING parquet +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE TABLE num_exp_ln (id int, expected decimal(38,10)) USING parquet +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +CREATE TABLE num_exp_log10 (id int, expected decimal(38,10)) USING parquet +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +CREATE TABLE num_exp_power_10_ln (id int, expected decimal(38,10)) USING parquet +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE TABLE num_result (id1 int, id2 int, result decimal(38,10)) USING parquet +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO num_exp_add VALUES (0,0,'0') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO num_exp_sub VALUES (0,0,'0') +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO num_exp_mul VALUES (0,0,'0') +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +INSERT INTO num_exp_div VALUES (0,0,'NaN') +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +INSERT INTO num_exp_add VALUES (0,1,'0') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +INSERT INTO num_exp_sub VALUES (0,1,'0') +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO num_exp_mul VALUES (0,1,'0') +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +INSERT INTO num_exp_div VALUES (0,1,'NaN') +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +INSERT INTO num_exp_add VALUES (0,2,'-34338492.215397047') +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO num_exp_sub VALUES (0,2,'34338492.215397047') +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +INSERT INTO num_exp_mul VALUES (0,2,'0') +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +INSERT INTO num_exp_div VALUES (0,2,'0') +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +INSERT INTO num_exp_add VALUES (0,3,'4.31') +-- !query 22 schema +struct<> +-- !query 22 output + + + +-- !query 23 +INSERT INTO num_exp_sub VALUES (0,3,'-4.31') +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +INSERT INTO num_exp_mul VALUES (0,3,'0') +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +INSERT INTO num_exp_div VALUES (0,3,'0') +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +INSERT INTO num_exp_add VALUES (0,4,'7799461.4119') +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO num_exp_sub VALUES (0,4,'-7799461.4119') +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +INSERT INTO num_exp_mul VALUES (0,4,'0') +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +INSERT INTO num_exp_div VALUES (0,4,'0') +-- !query 29 schema +struct<> +-- !query 29 output + + + +-- !query 30 +INSERT INTO num_exp_add VALUES (0,5,'16397.038491') +-- !query 30 schema +struct<> +-- !query 30 output + + + +-- !query 31 +INSERT INTO num_exp_sub VALUES (0,5,'-16397.038491') +-- !query 31 schema +struct<> +-- !query 31 output + + + +-- !query 32 +INSERT INTO num_exp_mul VALUES (0,5,'0') +-- !query 32 schema +struct<> +-- !query 32 output + + + +-- !query 33 +INSERT INTO num_exp_div VALUES (0,5,'0') +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO num_exp_add VALUES (0,6,'93901.57763026') +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +INSERT INTO num_exp_sub VALUES (0,6,'-93901.57763026') +-- !query 35 schema +struct<> +-- !query 35 output + + + +-- !query 36 +INSERT INTO num_exp_mul VALUES (0,6,'0') +-- !query 36 schema +struct<> +-- !query 36 output + + + +-- !query 37 +INSERT INTO num_exp_div VALUES (0,6,'0') +-- !query 37 schema +struct<> +-- !query 37 output + + + +-- !query 38 +INSERT INTO num_exp_add VALUES (0,7,'-83028485') +-- !query 38 schema +struct<> +-- !query 38 output + + + +-- !query 39 +INSERT INTO num_exp_sub VALUES (0,7,'83028485') +-- !query 39 schema +struct<> +-- !query 39 output + + + +-- !query 40 +INSERT INTO num_exp_mul VALUES (0,7,'0') +-- !query 40 schema +struct<> +-- !query 40 output + + + +-- !query 41 +INSERT INTO num_exp_div VALUES (0,7,'0') +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +INSERT INTO num_exp_add VALUES (0,8,'74881') +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +INSERT INTO num_exp_sub VALUES (0,8,'-74881') +-- !query 43 schema +struct<> +-- !query 43 output + + + +-- !query 44 +INSERT INTO num_exp_mul VALUES (0,8,'0') +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +INSERT INTO num_exp_div VALUES (0,8,'0') +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +INSERT INTO num_exp_add VALUES (0,9,'-24926804.045047420') +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +INSERT INTO num_exp_sub VALUES (0,9,'24926804.045047420') +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +INSERT INTO num_exp_mul VALUES (0,9,'0') +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +INSERT INTO num_exp_div VALUES (0,9,'0') +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +INSERT INTO num_exp_add VALUES (1,0,'0') +-- !query 50 schema +struct<> +-- !query 50 output + + + +-- !query 51 +INSERT INTO num_exp_sub VALUES (1,0,'0') +-- !query 51 schema +struct<> +-- !query 51 output + + + +-- !query 52 +INSERT INTO num_exp_mul VALUES (1,0,'0') +-- !query 52 schema +struct<> +-- !query 52 output + + + +-- !query 53 +INSERT INTO num_exp_div VALUES (1,0,'NaN') +-- !query 53 schema +struct<> +-- !query 53 output + + + +-- !query 54 +INSERT INTO num_exp_add VALUES (1,1,'0') +-- !query 54 schema +struct<> +-- !query 54 output + + + +-- !query 55 +INSERT INTO num_exp_sub VALUES (1,1,'0') +-- !query 55 schema +struct<> +-- !query 55 output + + + +-- !query 56 +INSERT INTO num_exp_mul VALUES (1,1,'0') +-- !query 56 schema +struct<> +-- !query 56 output + + + +-- !query 57 +INSERT INTO num_exp_div VALUES (1,1,'NaN') +-- !query 57 schema +struct<> +-- !query 57 output + + + +-- !query 58 +INSERT INTO num_exp_add VALUES (1,2,'-34338492.215397047') +-- !query 58 schema +struct<> +-- !query 58 output + + + +-- !query 59 +INSERT INTO num_exp_sub VALUES (1,2,'34338492.215397047') +-- !query 59 schema +struct<> +-- !query 59 output + + + +-- !query 60 +INSERT INTO num_exp_mul VALUES (1,2,'0') +-- !query 60 schema +struct<> +-- !query 60 output + + + +-- !query 61 +INSERT INTO num_exp_div VALUES (1,2,'0') +-- !query 61 schema +struct<> +-- !query 61 output + + + +-- !query 62 +INSERT INTO num_exp_add VALUES (1,3,'4.31') +-- !query 62 schema +struct<> +-- !query 62 output + + + +-- !query 63 +INSERT INTO num_exp_sub VALUES (1,3,'-4.31') +-- !query 63 schema +struct<> +-- !query 63 output + + + +-- !query 64 +INSERT INTO num_exp_mul VALUES (1,3,'0') +-- !query 64 schema +struct<> +-- !query 64 output + + + +-- !query 65 +INSERT INTO num_exp_div VALUES (1,3,'0') +-- !query 65 schema +struct<> +-- !query 65 output + + + +-- !query 66 +INSERT INTO num_exp_add VALUES (1,4,'7799461.4119') +-- !query 66 schema +struct<> +-- !query 66 output + + + +-- !query 67 +INSERT INTO num_exp_sub VALUES (1,4,'-7799461.4119') +-- !query 67 schema +struct<> +-- !query 67 output + + + +-- !query 68 +INSERT INTO num_exp_mul VALUES (1,4,'0') +-- !query 68 schema +struct<> +-- !query 68 output + + + +-- !query 69 +INSERT INTO num_exp_div VALUES (1,4,'0') +-- !query 69 schema +struct<> +-- !query 69 output + + + +-- !query 70 +INSERT INTO num_exp_add VALUES (1,5,'16397.038491') +-- !query 70 schema +struct<> +-- !query 70 output + + + +-- !query 71 +INSERT INTO num_exp_sub VALUES (1,5,'-16397.038491') +-- !query 71 schema +struct<> +-- !query 71 output + + + +-- !query 72 +INSERT INTO num_exp_mul VALUES (1,5,'0') +-- !query 72 schema +struct<> +-- !query 72 output + + + +-- !query 73 +INSERT INTO num_exp_div VALUES (1,5,'0') +-- !query 73 schema +struct<> +-- !query 73 output + + + +-- !query 74 +INSERT INTO num_exp_add VALUES (1,6,'93901.57763026') +-- !query 74 schema +struct<> +-- !query 74 output + + + +-- !query 75 +INSERT INTO num_exp_sub VALUES (1,6,'-93901.57763026') +-- !query 75 schema +struct<> +-- !query 75 output + + + +-- !query 76 +INSERT INTO num_exp_mul VALUES (1,6,'0') +-- !query 76 schema +struct<> +-- !query 76 output + + + +-- !query 77 +INSERT INTO num_exp_div VALUES (1,6,'0') +-- !query 77 schema +struct<> +-- !query 77 output + + + +-- !query 78 +INSERT INTO num_exp_add VALUES (1,7,'-83028485') +-- !query 78 schema +struct<> +-- !query 78 output + + + +-- !query 79 +INSERT INTO num_exp_sub VALUES (1,7,'83028485') +-- !query 79 schema +struct<> +-- !query 79 output + + + +-- !query 80 +INSERT INTO num_exp_mul VALUES (1,7,'0') +-- !query 80 schema +struct<> +-- !query 80 output + + + +-- !query 81 +INSERT INTO num_exp_div VALUES (1,7,'0') +-- !query 81 schema +struct<> +-- !query 81 output + + + +-- !query 82 +INSERT INTO num_exp_add VALUES (1,8,'74881') +-- !query 82 schema +struct<> +-- !query 82 output + + + +-- !query 83 +INSERT INTO num_exp_sub VALUES (1,8,'-74881') +-- !query 83 schema +struct<> +-- !query 83 output + + + +-- !query 84 +INSERT INTO num_exp_mul VALUES (1,8,'0') +-- !query 84 schema +struct<> +-- !query 84 output + + + +-- !query 85 +INSERT INTO num_exp_div VALUES (1,8,'0') +-- !query 85 schema +struct<> +-- !query 85 output + + + +-- !query 86 +INSERT INTO num_exp_add VALUES (1,9,'-24926804.045047420') +-- !query 86 schema +struct<> +-- !query 86 output + + + +-- !query 87 +INSERT INTO num_exp_sub VALUES (1,9,'24926804.045047420') +-- !query 87 schema +struct<> +-- !query 87 output + + + +-- !query 88 +INSERT INTO num_exp_mul VALUES (1,9,'0') +-- !query 88 schema +struct<> +-- !query 88 output + + + +-- !query 89 +INSERT INTO num_exp_div VALUES (1,9,'0') +-- !query 89 schema +struct<> +-- !query 89 output + + + +-- !query 90 +INSERT INTO num_exp_add VALUES (2,0,'-34338492.215397047') +-- !query 90 schema +struct<> +-- !query 90 output + + + +-- !query 91 +INSERT INTO num_exp_sub VALUES (2,0,'-34338492.215397047') +-- !query 91 schema +struct<> +-- !query 91 output + + + +-- !query 92 +INSERT INTO num_exp_mul VALUES (2,0,'0') +-- !query 92 schema +struct<> +-- !query 92 output + + + +-- !query 93 +INSERT INTO num_exp_div VALUES (2,0,'NaN') +-- !query 93 schema +struct<> +-- !query 93 output + + + +-- !query 94 +INSERT INTO num_exp_add VALUES (2,1,'-34338492.215397047') +-- !query 94 schema +struct<> +-- !query 94 output + + + +-- !query 95 +INSERT INTO num_exp_sub VALUES (2,1,'-34338492.215397047') +-- !query 95 schema +struct<> +-- !query 95 output + + + +-- !query 96 +INSERT INTO num_exp_mul VALUES (2,1,'0') +-- !query 96 schema +struct<> +-- !query 96 output + + + +-- !query 97 +INSERT INTO num_exp_div VALUES (2,1,'NaN') +-- !query 97 schema +struct<> +-- !query 97 output + + + +-- !query 98 +INSERT INTO num_exp_add VALUES (2,2,'-68676984.430794094') +-- !query 98 schema +struct<> +-- !query 98 output + + + +-- !query 99 +INSERT INTO num_exp_sub VALUES (2,2,'0') +-- !query 99 schema +struct<> +-- !query 99 output + + + +-- !query 100 +INSERT INTO num_exp_mul VALUES (2,2,'1179132047626883.596862135856320209') +-- !query 100 schema +struct<> +-- !query 100 output + + + +-- !query 101 +INSERT INTO num_exp_div VALUES (2,2,'1.00000000000000000000') +-- !query 101 schema +struct<> +-- !query 101 output + + + +-- !query 102 +INSERT INTO num_exp_add VALUES (2,3,'-34338487.905397047') +-- !query 102 schema +struct<> +-- !query 102 output + + + +-- !query 103 +INSERT INTO num_exp_sub VALUES (2,3,'-34338496.525397047') +-- !query 103 schema +struct<> +-- !query 103 output + + + +-- !query 104 +INSERT INTO num_exp_mul VALUES (2,3,'-147998901.44836127257') +-- !query 104 schema +struct<> +-- !query 104 output + + + +-- !query 105 +INSERT INTO num_exp_div VALUES (2,3,'-7967167.56737750510440835266') +-- !query 105 schema +struct<> +-- !query 105 output + + + +-- !query 106 +INSERT INTO num_exp_add VALUES (2,4,'-26539030.803497047') +-- !query 106 schema +struct<> +-- !query 106 output + + + +-- !query 107 +INSERT INTO num_exp_sub VALUES (2,4,'-42137953.627297047') +-- !query 107 schema +struct<> +-- !query 107 output + + + +-- !query 108 +INSERT INTO num_exp_mul VALUES (2,4,'-267821744976817.8111137106593') +-- !query 108 schema +struct<> +-- !query 108 output + + + +-- !query 109 +INSERT INTO num_exp_div VALUES (2,4,'-4.40267480046830116685') +-- !query 109 schema +struct<> +-- !query 109 output + + + +-- !query 110 +INSERT INTO num_exp_add VALUES (2,5,'-34322095.176906047') +-- !query 110 schema +struct<> +-- !query 110 output + + + +-- !query 111 +INSERT INTO num_exp_sub VALUES (2,5,'-34354889.253888047') +-- !query 111 schema +struct<> +-- !query 111 output + + + +-- !query 112 +INSERT INTO num_exp_mul VALUES (2,5,'-563049578578.769242506736077') +-- !query 112 schema +struct<> +-- !query 112 output + + + +-- !query 113 +INSERT INTO num_exp_div VALUES (2,5,'-2094.18866914563535496429') +-- !query 113 schema +struct<> +-- !query 113 output + + + +-- !query 114 +INSERT INTO num_exp_add VALUES (2,6,'-34244590.637766787') +-- !query 114 schema +struct<> +-- !query 114 output + + + +-- !query 115 +INSERT INTO num_exp_sub VALUES (2,6,'-34432393.793027307') +-- !query 115 schema +struct<> +-- !query 115 output + + + +-- !query 116 +INSERT INTO num_exp_mul VALUES (2,6,'-3224438592470.18449811926184222') +-- !query 116 schema +struct<> +-- !query 116 output + + + +-- !query 117 +INSERT INTO num_exp_div VALUES (2,6,'-365.68599891479766440940') +-- !query 117 schema +struct<> +-- !query 117 output + + + +-- !query 118 +INSERT INTO num_exp_add VALUES (2,7,'-117366977.215397047') +-- !query 118 schema +struct<> +-- !query 118 output + + + +-- !query 119 +INSERT INTO num_exp_sub VALUES (2,7,'48689992.784602953') +-- !query 119 schema +struct<> +-- !query 119 output + + + +-- !query 120 +INSERT INTO num_exp_mul VALUES (2,7,'2851072985828710.485883795') +-- !query 120 schema +struct<> +-- !query 120 output + + + +-- !query 121 +INSERT INTO num_exp_div VALUES (2,7,'.41357483778485235518') +-- !query 121 schema +struct<> +-- !query 121 output + + + +-- !query 122 +INSERT INTO num_exp_add VALUES (2,8,'-34263611.215397047') +-- !query 122 schema +struct<> +-- !query 122 output + + + +-- !query 123 +INSERT INTO num_exp_sub VALUES (2,8,'-34413373.215397047') +-- !query 123 schema +struct<> +-- !query 123 output + + + +-- !query 124 +INSERT INTO num_exp_mul VALUES (2,8,'-2571300635581.146276407') +-- !query 124 schema +struct<> +-- !query 124 output + + + +-- !query 125 +INSERT INTO num_exp_div VALUES (2,8,'-458.57416721727870888476') +-- !query 125 schema +struct<> +-- !query 125 output + + + +-- !query 126 +INSERT INTO num_exp_add VALUES (2,9,'-59265296.260444467') +-- !query 126 schema +struct<> +-- !query 126 output + + + +-- !query 127 +INSERT INTO num_exp_sub VALUES (2,9,'-9411688.170349627') +-- !query 127 schema +struct<> +-- !query 127 output + + + +-- !query 128 +INSERT INTO num_exp_mul VALUES (2,9,'855948866655588.453741509242968740') +-- !query 128 schema +struct<> +-- !query 128 output + + + +-- !query 129 +INSERT INTO num_exp_div VALUES (2,9,'1.37757299946438931811') +-- !query 129 schema +struct<> +-- !query 129 output + + + +-- !query 130 +INSERT INTO num_exp_add VALUES (3,0,'4.31') +-- !query 130 schema +struct<> +-- !query 130 output + + + +-- !query 131 +INSERT INTO num_exp_sub VALUES (3,0,'4.31') +-- !query 131 schema +struct<> +-- !query 131 output + + + +-- !query 132 +INSERT INTO num_exp_mul VALUES (3,0,'0') +-- !query 132 schema +struct<> +-- !query 132 output + + + +-- !query 133 +INSERT INTO num_exp_div VALUES (3,0,'NaN') +-- !query 133 schema +struct<> +-- !query 133 output + + + +-- !query 134 +INSERT INTO num_exp_add VALUES (3,1,'4.31') +-- !query 134 schema +struct<> +-- !query 134 output + + + +-- !query 135 +INSERT INTO num_exp_sub VALUES (3,1,'4.31') +-- !query 135 schema +struct<> +-- !query 135 output + + + +-- !query 136 +INSERT INTO num_exp_mul VALUES (3,1,'0') +-- !query 136 schema +struct<> +-- !query 136 output + + + +-- !query 137 +INSERT INTO num_exp_div VALUES (3,1,'NaN') +-- !query 137 schema +struct<> +-- !query 137 output + + + +-- !query 138 +INSERT INTO num_exp_add VALUES (3,2,'-34338487.905397047') +-- !query 138 schema +struct<> +-- !query 138 output + + + +-- !query 139 +INSERT INTO num_exp_sub VALUES (3,2,'34338496.525397047') +-- !query 139 schema +struct<> +-- !query 139 output + + + +-- !query 140 +INSERT INTO num_exp_mul VALUES (3,2,'-147998901.44836127257') +-- !query 140 schema +struct<> +-- !query 140 output + + + +-- !query 141 +INSERT INTO num_exp_div VALUES (3,2,'-.00000012551512084352') +-- !query 141 schema +struct<> +-- !query 141 output + + + +-- !query 142 +INSERT INTO num_exp_add VALUES (3,3,'8.62') +-- !query 142 schema +struct<> +-- !query 142 output + + + +-- !query 143 +INSERT INTO num_exp_sub VALUES (3,3,'0') +-- !query 143 schema +struct<> +-- !query 143 output + + + +-- !query 144 +INSERT INTO num_exp_mul VALUES (3,3,'18.5761') +-- !query 144 schema +struct<> +-- !query 144 output + + + +-- !query 145 +INSERT INTO num_exp_div VALUES (3,3,'1.00000000000000000000') +-- !query 145 schema +struct<> +-- !query 145 output + + + +-- !query 146 +INSERT INTO num_exp_add VALUES (3,4,'7799465.7219') +-- !query 146 schema +struct<> +-- !query 146 output + + + +-- !query 147 +INSERT INTO num_exp_sub VALUES (3,4,'-7799457.1019') +-- !query 147 schema +struct<> +-- !query 147 output + + + +-- !query 148 +INSERT INTO num_exp_mul VALUES (3,4,'33615678.685289') +-- !query 148 schema +struct<> +-- !query 148 output + + + +-- !query 149 +INSERT INTO num_exp_div VALUES (3,4,'.00000055260225961552') +-- !query 149 schema +struct<> +-- !query 149 output + + + +-- !query 150 +INSERT INTO num_exp_add VALUES (3,5,'16401.348491') +-- !query 150 schema +struct<> +-- !query 150 output + + + +-- !query 151 +INSERT INTO num_exp_sub VALUES (3,5,'-16392.728491') +-- !query 151 schema +struct<> +-- !query 151 output + + + +-- !query 152 +INSERT INTO num_exp_mul VALUES (3,5,'70671.23589621') +-- !query 152 schema +struct<> +-- !query 152 output + + + +-- !query 153 +INSERT INTO num_exp_div VALUES (3,5,'.00026285234387695504') +-- !query 153 schema +struct<> +-- !query 153 output + + + +-- !query 154 +INSERT INTO num_exp_add VALUES (3,6,'93905.88763026') +-- !query 154 schema +struct<> +-- !query 154 output + + + +-- !query 155 +INSERT INTO num_exp_sub VALUES (3,6,'-93897.26763026') +-- !query 155 schema +struct<> +-- !query 155 output + + + +-- !query 156 +INSERT INTO num_exp_mul VALUES (3,6,'404715.7995864206') +-- !query 156 schema +struct<> +-- !query 156 output + + + +-- !query 157 +INSERT INTO num_exp_div VALUES (3,6,'.00004589912234457595') +-- !query 157 schema +struct<> +-- !query 157 output + + + +-- !query 158 +INSERT INTO num_exp_add VALUES (3,7,'-83028480.69') +-- !query 158 schema +struct<> +-- !query 158 output + + + +-- !query 159 +INSERT INTO num_exp_sub VALUES (3,7,'83028489.31') +-- !query 159 schema +struct<> +-- !query 159 output + + + +-- !query 160 +INSERT INTO num_exp_mul VALUES (3,7,'-357852770.35') +-- !query 160 schema +struct<> +-- !query 160 output + + + +-- !query 161 +INSERT INTO num_exp_div VALUES (3,7,'-.00000005190989574240') +-- !query 161 schema +struct<> +-- !query 161 output + + + +-- !query 162 +INSERT INTO num_exp_add VALUES (3,8,'74885.31') +-- !query 162 schema +struct<> +-- !query 162 output + + + +-- !query 163 +INSERT INTO num_exp_sub VALUES (3,8,'-74876.69') +-- !query 163 schema +struct<> +-- !query 163 output + + + +-- !query 164 +INSERT INTO num_exp_mul VALUES (3,8,'322737.11') +-- !query 164 schema +struct<> +-- !query 164 output + + + +-- !query 165 +INSERT INTO num_exp_div VALUES (3,8,'.00005755799201399553') +-- !query 165 schema +struct<> +-- !query 165 output + + + +-- !query 166 +INSERT INTO num_exp_add VALUES (3,9,'-24926799.735047420') +-- !query 166 schema +struct<> +-- !query 166 output + + + +-- !query 167 +INSERT INTO num_exp_sub VALUES (3,9,'24926808.355047420') +-- !query 167 schema +struct<> +-- !query 167 output + + + +-- !query 168 +INSERT INTO num_exp_mul VALUES (3,9,'-107434525.43415438020') +-- !query 168 schema +struct<> +-- !query 168 output + + + +-- !query 169 +INSERT INTO num_exp_div VALUES (3,9,'-.00000017290624149854') +-- !query 169 schema +struct<> +-- !query 169 output + + + +-- !query 170 +INSERT INTO num_exp_add VALUES (4,0,'7799461.4119') +-- !query 170 schema +struct<> +-- !query 170 output + + + +-- !query 171 +INSERT INTO num_exp_sub VALUES (4,0,'7799461.4119') +-- !query 171 schema +struct<> +-- !query 171 output + + + +-- !query 172 +INSERT INTO num_exp_mul VALUES (4,0,'0') +-- !query 172 schema +struct<> +-- !query 172 output + + + +-- !query 173 +INSERT INTO num_exp_div VALUES (4,0,'NaN') +-- !query 173 schema +struct<> +-- !query 173 output + + + +-- !query 174 +INSERT INTO num_exp_add VALUES (4,1,'7799461.4119') +-- !query 174 schema +struct<> +-- !query 174 output + + + +-- !query 175 +INSERT INTO num_exp_sub VALUES (4,1,'7799461.4119') +-- !query 175 schema +struct<> +-- !query 175 output + + + +-- !query 176 +INSERT INTO num_exp_mul VALUES (4,1,'0') +-- !query 176 schema +struct<> +-- !query 176 output + + + +-- !query 177 +INSERT INTO num_exp_div VALUES (4,1,'NaN') +-- !query 177 schema +struct<> +-- !query 177 output + + + +-- !query 178 +INSERT INTO num_exp_add VALUES (4,2,'-26539030.803497047') +-- !query 178 schema +struct<> +-- !query 178 output + + + +-- !query 179 +INSERT INTO num_exp_sub VALUES (4,2,'42137953.627297047') +-- !query 179 schema +struct<> +-- !query 179 output + + + +-- !query 180 +INSERT INTO num_exp_mul VALUES (4,2,'-267821744976817.8111137106593') +-- !query 180 schema +struct<> +-- !query 180 output + + + +-- !query 181 +INSERT INTO num_exp_div VALUES (4,2,'-.22713465002993920385') +-- !query 181 schema +struct<> +-- !query 181 output + + + +-- !query 182 +INSERT INTO num_exp_add VALUES (4,3,'7799465.7219') +-- !query 182 schema +struct<> +-- !query 182 output + + + +-- !query 183 +INSERT INTO num_exp_sub VALUES (4,3,'7799457.1019') +-- !query 183 schema +struct<> +-- !query 183 output + + + +-- !query 184 +INSERT INTO num_exp_mul VALUES (4,3,'33615678.685289') +-- !query 184 schema +struct<> +-- !query 184 output + + + +-- !query 185 +INSERT INTO num_exp_div VALUES (4,3,'1809619.81714617169373549883') +-- !query 185 schema +struct<> +-- !query 185 output + + + +-- !query 186 +INSERT INTO num_exp_add VALUES (4,4,'15598922.8238') +-- !query 186 schema +struct<> +-- !query 186 output + + + +-- !query 187 +INSERT INTO num_exp_sub VALUES (4,4,'0') +-- !query 187 schema +struct<> +-- !query 187 output + + + +-- !query 188 +INSERT INTO num_exp_mul VALUES (4,4,'60831598315717.14146161') +-- !query 188 schema +struct<> +-- !query 188 output + + + +-- !query 189 +INSERT INTO num_exp_div VALUES (4,4,'1.00000000000000000000') +-- !query 189 schema +struct<> +-- !query 189 output + + + +-- !query 190 +INSERT INTO num_exp_add VALUES (4,5,'7815858.450391') +-- !query 190 schema +struct<> +-- !query 190 output + + + +-- !query 191 +INSERT INTO num_exp_sub VALUES (4,5,'7783064.373409') +-- !query 191 schema +struct<> +-- !query 191 output + + + +-- !query 192 +INSERT INTO num_exp_mul VALUES (4,5,'127888068979.9935054429') +-- !query 192 schema +struct<> +-- !query 192 output + + + +-- !query 193 +INSERT INTO num_exp_div VALUES (4,5,'475.66281046305802686061') +-- !query 193 schema +struct<> +-- !query 193 output + + + +-- !query 194 +INSERT INTO num_exp_add VALUES (4,6,'7893362.98953026') +-- !query 194 schema +struct<> +-- !query 194 output + + + +-- !query 195 +INSERT INTO num_exp_sub VALUES (4,6,'7705559.83426974') +-- !query 195 schema +struct<> +-- !query 195 output + + + +-- !query 196 +INSERT INTO num_exp_mul VALUES (4,6,'732381731243.745115764094') +-- !query 196 schema +struct<> +-- !query 196 output + + + +-- !query 197 +INSERT INTO num_exp_div VALUES (4,6,'83.05996138436129499606') +-- !query 197 schema +struct<> +-- !query 197 output + + + +-- !query 198 +INSERT INTO num_exp_add VALUES (4,7,'-75229023.5881') +-- !query 198 schema +struct<> +-- !query 198 output + + + +-- !query 199 +INSERT INTO num_exp_sub VALUES (4,7,'90827946.4119') +-- !query 199 schema +struct<> +-- !query 199 output + + + +-- !query 200 +INSERT INTO num_exp_mul VALUES (4,7,'-647577464846017.9715') +-- !query 200 schema +struct<> +-- !query 200 output + + + +-- !query 201 +INSERT INTO num_exp_div VALUES (4,7,'-.09393717604145131637') +-- !query 201 schema +struct<> +-- !query 201 output + + + +-- !query 202 +INSERT INTO num_exp_add VALUES (4,8,'7874342.4119') +-- !query 202 schema +struct<> +-- !query 202 output + + + +-- !query 203 +INSERT INTO num_exp_sub VALUES (4,8,'7724580.4119') +-- !query 203 schema +struct<> +-- !query 203 output + + + +-- !query 204 +INSERT INTO num_exp_mul VALUES (4,8,'584031469984.4839') +-- !query 204 schema +struct<> +-- !query 204 output + + + +-- !query 205 +INSERT INTO num_exp_div VALUES (4,8,'104.15808298366741897143') +-- !query 205 schema +struct<> +-- !query 205 output + + + +-- !query 206 +INSERT INTO num_exp_add VALUES (4,9,'-17127342.633147420') +-- !query 206 schema +struct<> +-- !query 206 output + + + +-- !query 207 +INSERT INTO num_exp_sub VALUES (4,9,'32726265.456947420') +-- !query 207 schema +struct<> +-- !query 207 output + + + +-- !query 208 +INSERT INTO num_exp_mul VALUES (4,9,'-194415646271340.1815956522980') +-- !query 208 schema +struct<> +-- !query 208 output + + + +-- !query 209 +INSERT INTO num_exp_div VALUES (4,9,'-.31289456112403769409') +-- !query 209 schema +struct<> +-- !query 209 output + + + +-- !query 210 +INSERT INTO num_exp_add VALUES (5,0,'16397.038491') +-- !query 210 schema +struct<> +-- !query 210 output + + + +-- !query 211 +INSERT INTO num_exp_sub VALUES (5,0,'16397.038491') +-- !query 211 schema +struct<> +-- !query 211 output + + + +-- !query 212 +INSERT INTO num_exp_mul VALUES (5,0,'0') +-- !query 212 schema +struct<> +-- !query 212 output + + + +-- !query 213 +INSERT INTO num_exp_div VALUES (5,0,'NaN') +-- !query 213 schema +struct<> +-- !query 213 output + + + +-- !query 214 +INSERT INTO num_exp_add VALUES (5,1,'16397.038491') +-- !query 214 schema +struct<> +-- !query 214 output + + + +-- !query 215 +INSERT INTO num_exp_sub VALUES (5,1,'16397.038491') +-- !query 215 schema +struct<> +-- !query 215 output + + + +-- !query 216 +INSERT INTO num_exp_mul VALUES (5,1,'0') +-- !query 216 schema +struct<> +-- !query 216 output + + + +-- !query 217 +INSERT INTO num_exp_div VALUES (5,1,'NaN') +-- !query 217 schema +struct<> +-- !query 217 output + + + +-- !query 218 +INSERT INTO num_exp_add VALUES (5,2,'-34322095.176906047') +-- !query 218 schema +struct<> +-- !query 218 output + + + +-- !query 219 +INSERT INTO num_exp_sub VALUES (5,2,'34354889.253888047') +-- !query 219 schema +struct<> +-- !query 219 output + + + +-- !query 220 +INSERT INTO num_exp_mul VALUES (5,2,'-563049578578.769242506736077') +-- !query 220 schema +struct<> +-- !query 220 output + + + +-- !query 221 +INSERT INTO num_exp_div VALUES (5,2,'-.00047751189505192446') +-- !query 221 schema +struct<> +-- !query 221 output + + + +-- !query 222 +INSERT INTO num_exp_add VALUES (5,3,'16401.348491') +-- !query 222 schema +struct<> +-- !query 222 output + + + +-- !query 223 +INSERT INTO num_exp_sub VALUES (5,3,'16392.728491') +-- !query 223 schema +struct<> +-- !query 223 output + + + +-- !query 224 +INSERT INTO num_exp_mul VALUES (5,3,'70671.23589621') +-- !query 224 schema +struct<> +-- !query 224 output + + + +-- !query 225 +INSERT INTO num_exp_div VALUES (5,3,'3804.41728329466357308584') +-- !query 225 schema +struct<> +-- !query 225 output + + + +-- !query 226 +INSERT INTO num_exp_add VALUES (5,4,'7815858.450391') +-- !query 226 schema +struct<> +-- !query 226 output + + + +-- !query 227 +INSERT INTO num_exp_sub VALUES (5,4,'-7783064.373409') +-- !query 227 schema +struct<> +-- !query 227 output + + + +-- !query 228 +INSERT INTO num_exp_mul VALUES (5,4,'127888068979.9935054429') +-- !query 228 schema +struct<> +-- !query 228 output + + + +-- !query 229 +INSERT INTO num_exp_div VALUES (5,4,'.00210232958726897192') +-- !query 229 schema +struct<> +-- !query 229 output + + + +-- !query 230 +INSERT INTO num_exp_add VALUES (5,5,'32794.076982') +-- !query 230 schema +struct<> +-- !query 230 output + + + +-- !query 231 +INSERT INTO num_exp_sub VALUES (5,5,'0') +-- !query 231 schema +struct<> +-- !query 231 output + + + +-- !query 232 +INSERT INTO num_exp_mul VALUES (5,5,'268862871.275335557081') +-- !query 232 schema +struct<> +-- !query 232 output + + + +-- !query 233 +INSERT INTO num_exp_div VALUES (5,5,'1.00000000000000000000') +-- !query 233 schema +struct<> +-- !query 233 output + + + +-- !query 234 +INSERT INTO num_exp_add VALUES (5,6,'110298.61612126') +-- !query 234 schema +struct<> +-- !query 234 output + + + +-- !query 235 +INSERT INTO num_exp_sub VALUES (5,6,'-77504.53913926') +-- !query 235 schema +struct<> +-- !query 235 output + + + +-- !query 236 +INSERT INTO num_exp_mul VALUES (5,6,'1539707782.76899778633766') +-- !query 236 schema +struct<> +-- !query 236 output + + + +-- !query 237 +INSERT INTO num_exp_div VALUES (5,6,'.17461941433576102689') +-- !query 237 schema +struct<> +-- !query 237 output + + + +-- !query 238 +INSERT INTO num_exp_add VALUES (5,7,'-83012087.961509') +-- !query 238 schema +struct<> +-- !query 238 output + + + +-- !query 239 +INSERT INTO num_exp_sub VALUES (5,7,'83044882.038491') +-- !query 239 schema +struct<> +-- !query 239 output + + + +-- !query 240 +INSERT INTO num_exp_mul VALUES (5,7,'-1361421264394.416135') +-- !query 240 schema +struct<> +-- !query 240 output + + + +-- !query 241 +INSERT INTO num_exp_div VALUES (5,7,'-.00019748690453643710') +-- !query 241 schema +struct<> +-- !query 241 output + + + +-- !query 242 +INSERT INTO num_exp_add VALUES (5,8,'91278.038491') +-- !query 242 schema +struct<> +-- !query 242 output + + + +-- !query 243 +INSERT INTO num_exp_sub VALUES (5,8,'-58483.961509') +-- !query 243 schema +struct<> +-- !query 243 output + + + +-- !query 244 +INSERT INTO num_exp_mul VALUES (5,8,'1227826639.244571') +-- !query 244 schema +struct<> +-- !query 244 output + + + +-- !query 245 +INSERT INTO num_exp_div VALUES (5,8,'.21897461960978085228') +-- !query 245 schema +struct<> +-- !query 245 output + + + +-- !query 246 +INSERT INTO num_exp_add VALUES (5,9,'-24910407.006556420') +-- !query 246 schema +struct<> +-- !query 246 output + + + +-- !query 247 +INSERT INTO num_exp_sub VALUES (5,9,'24943201.083538420') +-- !query 247 schema +struct<> +-- !query 247 output + + + +-- !query 248 +INSERT INTO num_exp_mul VALUES (5,9,'-408725765384.257043660243220') +-- !query 248 schema +struct<> +-- !query 248 output + + + +-- !query 249 +INSERT INTO num_exp_div VALUES (5,9,'-.00065780749354660427') +-- !query 249 schema +struct<> +-- !query 249 output + + + +-- !query 250 +INSERT INTO num_exp_add VALUES (6,0,'93901.57763026') +-- !query 250 schema +struct<> +-- !query 250 output + + + +-- !query 251 +INSERT INTO num_exp_sub VALUES (6,0,'93901.57763026') +-- !query 251 schema +struct<> +-- !query 251 output + + + +-- !query 252 +INSERT INTO num_exp_mul VALUES (6,0,'0') +-- !query 252 schema +struct<> +-- !query 252 output + + + +-- !query 253 +INSERT INTO num_exp_div VALUES (6,0,'NaN') +-- !query 253 schema +struct<> +-- !query 253 output + + + +-- !query 254 +INSERT INTO num_exp_add VALUES (6,1,'93901.57763026') +-- !query 254 schema +struct<> +-- !query 254 output + + + +-- !query 255 +INSERT INTO num_exp_sub VALUES (6,1,'93901.57763026') +-- !query 255 schema +struct<> +-- !query 255 output + + + +-- !query 256 +INSERT INTO num_exp_mul VALUES (6,1,'0') +-- !query 256 schema +struct<> +-- !query 256 output + + + +-- !query 257 +INSERT INTO num_exp_div VALUES (6,1,'NaN') +-- !query 257 schema +struct<> +-- !query 257 output + + + +-- !query 258 +INSERT INTO num_exp_add VALUES (6,2,'-34244590.637766787') +-- !query 258 schema +struct<> +-- !query 258 output + + + +-- !query 259 +INSERT INTO num_exp_sub VALUES (6,2,'34432393.793027307') +-- !query 259 schema +struct<> +-- !query 259 output + + + +-- !query 260 +INSERT INTO num_exp_mul VALUES (6,2,'-3224438592470.18449811926184222') +-- !query 260 schema +struct<> +-- !query 260 output + + + +-- !query 261 +INSERT INTO num_exp_div VALUES (6,2,'-.00273458651128995823') +-- !query 261 schema +struct<> +-- !query 261 output + + + +-- !query 262 +INSERT INTO num_exp_add VALUES (6,3,'93905.88763026') +-- !query 262 schema +struct<> +-- !query 262 output + + + +-- !query 263 +INSERT INTO num_exp_sub VALUES (6,3,'93897.26763026') +-- !query 263 schema +struct<> +-- !query 263 output + + + +-- !query 264 +INSERT INTO num_exp_mul VALUES (6,3,'404715.7995864206') +-- !query 264 schema +struct<> +-- !query 264 output + + + +-- !query 265 +INSERT INTO num_exp_div VALUES (6,3,'21786.90896293735498839907') +-- !query 265 schema +struct<> +-- !query 265 output + + + +-- !query 266 +INSERT INTO num_exp_add VALUES (6,4,'7893362.98953026') +-- !query 266 schema +struct<> +-- !query 266 output + + + +-- !query 267 +INSERT INTO num_exp_sub VALUES (6,4,'-7705559.83426974') +-- !query 267 schema +struct<> +-- !query 267 output + + + +-- !query 268 +INSERT INTO num_exp_mul VALUES (6,4,'732381731243.745115764094') +-- !query 268 schema +struct<> +-- !query 268 output + + + +-- !query 269 +INSERT INTO num_exp_div VALUES (6,4,'.01203949512295682469') +-- !query 269 schema +struct<> +-- !query 269 output + + + +-- !query 270 +INSERT INTO num_exp_add VALUES (6,5,'110298.61612126') +-- !query 270 schema +struct<> +-- !query 270 output + + + +-- !query 271 +INSERT INTO num_exp_sub VALUES (6,5,'77504.53913926') +-- !query 271 schema +struct<> +-- !query 271 output + + + +-- !query 272 +INSERT INTO num_exp_mul VALUES (6,5,'1539707782.76899778633766') +-- !query 272 schema +struct<> +-- !query 272 output + + + +-- !query 273 +INSERT INTO num_exp_div VALUES (6,5,'5.72674008674192359679') +-- !query 273 schema +struct<> +-- !query 273 output + + + +-- !query 274 +INSERT INTO num_exp_add VALUES (6,6,'187803.15526052') +-- !query 274 schema +struct<> +-- !query 274 output + + + +-- !query 275 +INSERT INTO num_exp_sub VALUES (6,6,'0') +-- !query 275 schema +struct<> +-- !query 275 output + + + +-- !query 276 +INSERT INTO num_exp_mul VALUES (6,6,'8817506281.4517452372676676') +-- !query 276 schema +struct<> +-- !query 276 output + + + +-- !query 277 +INSERT INTO num_exp_div VALUES (6,6,'1.00000000000000000000') +-- !query 277 schema +struct<> +-- !query 277 output + + + +-- !query 278 +INSERT INTO num_exp_add VALUES (6,7,'-82934583.42236974') +-- !query 278 schema +struct<> +-- !query 278 output + + + +-- !query 279 +INSERT INTO num_exp_sub VALUES (6,7,'83122386.57763026') +-- !query 279 schema +struct<> +-- !query 279 output + + + +-- !query 280 +INSERT INTO num_exp_mul VALUES (6,7,'-7796505729750.37795610') +-- !query 280 schema +struct<> +-- !query 280 output + + + +-- !query 281 +INSERT INTO num_exp_div VALUES (6,7,'-.00113095617281538980') +-- !query 281 schema +struct<> +-- !query 281 output + + + +-- !query 282 +INSERT INTO num_exp_add VALUES (6,8,'168782.57763026') +-- !query 282 schema +struct<> +-- !query 282 output + + + +-- !query 283 +INSERT INTO num_exp_sub VALUES (6,8,'19020.57763026') +-- !query 283 schema +struct<> +-- !query 283 output + + + +-- !query 284 +INSERT INTO num_exp_mul VALUES (6,8,'7031444034.53149906') +-- !query 284 schema +struct<> +-- !query 284 output + + + +-- !query 285 +INSERT INTO num_exp_div VALUES (6,8,'1.25401073209839612184') +-- !query 285 schema +struct<> +-- !query 285 output + + + +-- !query 286 +INSERT INTO num_exp_add VALUES (6,9,'-24832902.467417160') +-- !query 286 schema +struct<> +-- !query 286 output + + + +-- !query 287 +INSERT INTO num_exp_sub VALUES (6,9,'25020705.622677680') +-- !query 287 schema +struct<> +-- !query 287 output + + + +-- !query 288 +INSERT INTO num_exp_mul VALUES (6,9,'-2340666225110.29929521292692920') +-- !query 288 schema +struct<> +-- !query 288 output + + + +-- !query 289 +INSERT INTO num_exp_div VALUES (6,9,'-.00376709254265256789') +-- !query 289 schema +struct<> +-- !query 289 output + + + +-- !query 290 +INSERT INTO num_exp_add VALUES (7,0,'-83028485') +-- !query 290 schema +struct<> +-- !query 290 output + + + +-- !query 291 +INSERT INTO num_exp_sub VALUES (7,0,'-83028485') +-- !query 291 schema +struct<> +-- !query 291 output + + + +-- !query 292 +INSERT INTO num_exp_mul VALUES (7,0,'0') +-- !query 292 schema +struct<> +-- !query 292 output + + + +-- !query 293 +INSERT INTO num_exp_div VALUES (7,0,'NaN') +-- !query 293 schema +struct<> +-- !query 293 output + + + +-- !query 294 +INSERT INTO num_exp_add VALUES (7,1,'-83028485') +-- !query 294 schema +struct<> +-- !query 294 output + + + +-- !query 295 +INSERT INTO num_exp_sub VALUES (7,1,'-83028485') +-- !query 295 schema +struct<> +-- !query 295 output + + + +-- !query 296 +INSERT INTO num_exp_mul VALUES (7,1,'0') +-- !query 296 schema +struct<> +-- !query 296 output + + + +-- !query 297 +INSERT INTO num_exp_div VALUES (7,1,'NaN') +-- !query 297 schema +struct<> +-- !query 297 output + + + +-- !query 298 +INSERT INTO num_exp_add VALUES (7,2,'-117366977.215397047') +-- !query 298 schema +struct<> +-- !query 298 output + + + +-- !query 299 +INSERT INTO num_exp_sub VALUES (7,2,'-48689992.784602953') +-- !query 299 schema +struct<> +-- !query 299 output + + + +-- !query 300 +INSERT INTO num_exp_mul VALUES (7,2,'2851072985828710.485883795') +-- !query 300 schema +struct<> +-- !query 300 output + + + +-- !query 301 +INSERT INTO num_exp_div VALUES (7,2,'2.41794207151503385700') +-- !query 301 schema +struct<> +-- !query 301 output + + + +-- !query 302 +INSERT INTO num_exp_add VALUES (7,3,'-83028480.69') +-- !query 302 schema +struct<> +-- !query 302 output + + + +-- !query 303 +INSERT INTO num_exp_sub VALUES (7,3,'-83028489.31') +-- !query 303 schema +struct<> +-- !query 303 output + + + +-- !query 304 +INSERT INTO num_exp_mul VALUES (7,3,'-357852770.35') +-- !query 304 schema +struct<> +-- !query 304 output + + + +-- !query 305 +INSERT INTO num_exp_div VALUES (7,3,'-19264149.65197215777262180974') +-- !query 305 schema +struct<> +-- !query 305 output + + + +-- !query 306 +INSERT INTO num_exp_add VALUES (7,4,'-75229023.5881') +-- !query 306 schema +struct<> +-- !query 306 output + + + +-- !query 307 +INSERT INTO num_exp_sub VALUES (7,4,'-90827946.4119') +-- !query 307 schema +struct<> +-- !query 307 output + + + +-- !query 308 +INSERT INTO num_exp_mul VALUES (7,4,'-647577464846017.9715') +-- !query 308 schema +struct<> +-- !query 308 output + + + +-- !query 309 +INSERT INTO num_exp_div VALUES (7,4,'-10.64541262725136247686') +-- !query 309 schema +struct<> +-- !query 309 output + + + +-- !query 310 +INSERT INTO num_exp_add VALUES (7,5,'-83012087.961509') +-- !query 310 schema +struct<> +-- !query 310 output + + + +-- !query 311 +INSERT INTO num_exp_sub VALUES (7,5,'-83044882.038491') +-- !query 311 schema +struct<> +-- !query 311 output + + + +-- !query 312 +INSERT INTO num_exp_mul VALUES (7,5,'-1361421264394.416135') +-- !query 312 schema +struct<> +-- !query 312 output + + + +-- !query 313 +INSERT INTO num_exp_div VALUES (7,5,'-5063.62688881730941836574') +-- !query 313 schema +struct<> +-- !query 313 output + + + +-- !query 314 +INSERT INTO num_exp_add VALUES (7,6,'-82934583.42236974') +-- !query 314 schema +struct<> +-- !query 314 output + + + +-- !query 315 +INSERT INTO num_exp_sub VALUES (7,6,'-83122386.57763026') +-- !query 315 schema +struct<> +-- !query 315 output + + + +-- !query 316 +INSERT INTO num_exp_mul VALUES (7,6,'-7796505729750.37795610') +-- !query 316 schema +struct<> +-- !query 316 output + + + +-- !query 317 +INSERT INTO num_exp_div VALUES (7,6,'-884.20756174009028770294') +-- !query 317 schema +struct<> +-- !query 317 output + + + +-- !query 318 +INSERT INTO num_exp_add VALUES (7,7,'-166056970') +-- !query 318 schema +struct<> +-- !query 318 output + + + +-- !query 319 +INSERT INTO num_exp_sub VALUES (7,7,'0') +-- !query 319 schema +struct<> +-- !query 319 output + + + +-- !query 320 +INSERT INTO num_exp_mul VALUES (7,7,'6893729321395225') +-- !query 320 schema +struct<> +-- !query 320 output + + + +-- !query 321 +INSERT INTO num_exp_div VALUES (7,7,'1.00000000000000000000') +-- !query 321 schema +struct<> +-- !query 321 output + + + +-- !query 322 +INSERT INTO num_exp_add VALUES (7,8,'-82953604') +-- !query 322 schema +struct<> +-- !query 322 output + + + +-- !query 323 +INSERT INTO num_exp_sub VALUES (7,8,'-83103366') +-- !query 323 schema +struct<> +-- !query 323 output + + + +-- !query 324 +INSERT INTO num_exp_mul VALUES (7,8,'-6217255985285') +-- !query 324 schema +struct<> +-- !query 324 output + + + +-- !query 325 +INSERT INTO num_exp_div VALUES (7,8,'-1108.80577182462841041118') +-- !query 325 schema +struct<> +-- !query 325 output + + + +-- !query 326 +INSERT INTO num_exp_add VALUES (7,9,'-107955289.045047420') +-- !query 326 schema +struct<> +-- !query 326 output + + + +-- !query 327 +INSERT INTO num_exp_sub VALUES (7,9,'-58101680.954952580') +-- !query 327 schema +struct<> +-- !query 327 output + + + +-- !query 328 +INSERT INTO num_exp_mul VALUES (7,9,'2069634775752159.035758700') +-- !query 328 schema +struct<> +-- !query 328 output + + + +-- !query 329 +INSERT INTO num_exp_div VALUES (7,9,'3.33089171198810413382') +-- !query 329 schema +struct<> +-- !query 329 output + + + +-- !query 330 +INSERT INTO num_exp_add VALUES (8,0,'74881') +-- !query 330 schema +struct<> +-- !query 330 output + + + +-- !query 331 +INSERT INTO num_exp_sub VALUES (8,0,'74881') +-- !query 331 schema +struct<> +-- !query 331 output + + + +-- !query 332 +INSERT INTO num_exp_mul VALUES (8,0,'0') +-- !query 332 schema +struct<> +-- !query 332 output + + + +-- !query 333 +INSERT INTO num_exp_div VALUES (8,0,'NaN') +-- !query 333 schema +struct<> +-- !query 333 output + + + +-- !query 334 +INSERT INTO num_exp_add VALUES (8,1,'74881') +-- !query 334 schema +struct<> +-- !query 334 output + + + +-- !query 335 +INSERT INTO num_exp_sub VALUES (8,1,'74881') +-- !query 335 schema +struct<> +-- !query 335 output + + + +-- !query 336 +INSERT INTO num_exp_mul VALUES (8,1,'0') +-- !query 336 schema +struct<> +-- !query 336 output + + + +-- !query 337 +INSERT INTO num_exp_div VALUES (8,1,'NaN') +-- !query 337 schema +struct<> +-- !query 337 output + + + +-- !query 338 +INSERT INTO num_exp_add VALUES (8,2,'-34263611.215397047') +-- !query 338 schema +struct<> +-- !query 338 output + + + +-- !query 339 +INSERT INTO num_exp_sub VALUES (8,2,'34413373.215397047') +-- !query 339 schema +struct<> +-- !query 339 output + + + +-- !query 340 +INSERT INTO num_exp_mul VALUES (8,2,'-2571300635581.146276407') +-- !query 340 schema +struct<> +-- !query 340 output + + + +-- !query 341 +INSERT INTO num_exp_div VALUES (8,2,'-.00218067233500788615') +-- !query 341 schema +struct<> +-- !query 341 output + + + +-- !query 342 +INSERT INTO num_exp_add VALUES (8,3,'74885.31') +-- !query 342 schema +struct<> +-- !query 342 output + + + +-- !query 343 +INSERT INTO num_exp_sub VALUES (8,3,'74876.69') +-- !query 343 schema +struct<> +-- !query 343 output + + + +-- !query 344 +INSERT INTO num_exp_mul VALUES (8,3,'322737.11') +-- !query 344 schema +struct<> +-- !query 344 output + + + +-- !query 345 +INSERT INTO num_exp_div VALUES (8,3,'17373.78190255220417633410') +-- !query 345 schema +struct<> +-- !query 345 output + + + +-- !query 346 +INSERT INTO num_exp_add VALUES (8,4,'7874342.4119') +-- !query 346 schema +struct<> +-- !query 346 output + + + +-- !query 347 +INSERT INTO num_exp_sub VALUES (8,4,'-7724580.4119') +-- !query 347 schema +struct<> +-- !query 347 output + + + +-- !query 348 +INSERT INTO num_exp_mul VALUES (8,4,'584031469984.4839') +-- !query 348 schema +struct<> +-- !query 348 output + + + +-- !query 349 +INSERT INTO num_exp_div VALUES (8,4,'.00960079113741758956') +-- !query 349 schema +struct<> +-- !query 349 output + + + +-- !query 350 +INSERT INTO num_exp_add VALUES (8,5,'91278.038491') +-- !query 350 schema +struct<> +-- !query 350 output + + + +-- !query 351 +INSERT INTO num_exp_sub VALUES (8,5,'58483.961509') +-- !query 351 schema +struct<> +-- !query 351 output + + + +-- !query 352 +INSERT INTO num_exp_mul VALUES (8,5,'1227826639.244571') +-- !query 352 schema +struct<> +-- !query 352 output + + + +-- !query 353 +INSERT INTO num_exp_div VALUES (8,5,'4.56673929509287019456') +-- !query 353 schema +struct<> +-- !query 353 output + + + +-- !query 354 +INSERT INTO num_exp_add VALUES (8,6,'168782.57763026') +-- !query 354 schema +struct<> +-- !query 354 output + + + +-- !query 355 +INSERT INTO num_exp_sub VALUES (8,6,'-19020.57763026') +-- !query 355 schema +struct<> +-- !query 355 output + + + +-- !query 356 +INSERT INTO num_exp_mul VALUES (8,6,'7031444034.53149906') +-- !query 356 schema +struct<> +-- !query 356 output + + + +-- !query 357 +INSERT INTO num_exp_div VALUES (8,6,'.79744134113322314424') +-- !query 357 schema +struct<> +-- !query 357 output + + + +-- !query 358 +INSERT INTO num_exp_add VALUES (8,7,'-82953604') +-- !query 358 schema +struct<> +-- !query 358 output + + + +-- !query 359 +INSERT INTO num_exp_sub VALUES (8,7,'83103366') +-- !query 359 schema +struct<> +-- !query 359 output + + + +-- !query 360 +INSERT INTO num_exp_mul VALUES (8,7,'-6217255985285') +-- !query 360 schema +struct<> +-- !query 360 output + + + +-- !query 361 +INSERT INTO num_exp_div VALUES (8,7,'-.00090187120721280172') +-- !query 361 schema +struct<> +-- !query 361 output + + + +-- !query 362 +INSERT INTO num_exp_add VALUES (8,8,'149762') +-- !query 362 schema +struct<> +-- !query 362 output + + + +-- !query 363 +INSERT INTO num_exp_sub VALUES (8,8,'0') +-- !query 363 schema +struct<> +-- !query 363 output + + + +-- !query 364 +INSERT INTO num_exp_mul VALUES (8,8,'5607164161') +-- !query 364 schema +struct<> +-- !query 364 output + + + +-- !query 365 +INSERT INTO num_exp_div VALUES (8,8,'1.00000000000000000000') +-- !query 365 schema +struct<> +-- !query 365 output + + + +-- !query 366 +INSERT INTO num_exp_add VALUES (8,9,'-24851923.045047420') +-- !query 366 schema +struct<> +-- !query 366 output + + + +-- !query 367 +INSERT INTO num_exp_sub VALUES (8,9,'25001685.045047420') +-- !query 367 schema +struct<> +-- !query 367 output + + + +-- !query 368 +INSERT INTO num_exp_mul VALUES (8,9,'-1866544013697.195857020') +-- !query 368 schema +struct<> +-- !query 368 output + + + +-- !query 369 +INSERT INTO num_exp_div VALUES (8,9,'-.00300403532938582735') +-- !query 369 schema +struct<> +-- !query 369 output + + + +-- !query 370 +INSERT INTO num_exp_add VALUES (9,0,'-24926804.045047420') +-- !query 370 schema +struct<> +-- !query 370 output + + + +-- !query 371 +INSERT INTO num_exp_sub VALUES (9,0,'-24926804.045047420') +-- !query 371 schema +struct<> +-- !query 371 output + + + +-- !query 372 +INSERT INTO num_exp_mul VALUES (9,0,'0') +-- !query 372 schema +struct<> +-- !query 372 output + + + +-- !query 373 +INSERT INTO num_exp_div VALUES (9,0,'NaN') +-- !query 373 schema +struct<> +-- !query 373 output + + + +-- !query 374 +INSERT INTO num_exp_add VALUES (9,1,'-24926804.045047420') +-- !query 374 schema +struct<> +-- !query 374 output + + + +-- !query 375 +INSERT INTO num_exp_sub VALUES (9,1,'-24926804.045047420') +-- !query 375 schema +struct<> +-- !query 375 output + + + +-- !query 376 +INSERT INTO num_exp_mul VALUES (9,1,'0') +-- !query 376 schema +struct<> +-- !query 376 output + + + +-- !query 377 +INSERT INTO num_exp_div VALUES (9,1,'NaN') +-- !query 377 schema +struct<> +-- !query 377 output + + + +-- !query 378 +INSERT INTO num_exp_add VALUES (9,2,'-59265296.260444467') +-- !query 378 schema +struct<> +-- !query 378 output + + + +-- !query 379 +INSERT INTO num_exp_sub VALUES (9,2,'9411688.170349627') +-- !query 379 schema +struct<> +-- !query 379 output + + + +-- !query 380 +INSERT INTO num_exp_mul VALUES (9,2,'855948866655588.453741509242968740') +-- !query 380 schema +struct<> +-- !query 380 output + + + +-- !query 381 +INSERT INTO num_exp_div VALUES (9,2,'.72591434384152961526') +-- !query 381 schema +struct<> +-- !query 381 output + + + +-- !query 382 +INSERT INTO num_exp_add VALUES (9,3,'-24926799.735047420') +-- !query 382 schema +struct<> +-- !query 382 output + + + +-- !query 383 +INSERT INTO num_exp_sub VALUES (9,3,'-24926808.355047420') +-- !query 383 schema +struct<> +-- !query 383 output + + + +-- !query 384 +INSERT INTO num_exp_mul VALUES (9,3,'-107434525.43415438020') +-- !query 384 schema +struct<> +-- !query 384 output + + + +-- !query 385 +INSERT INTO num_exp_div VALUES (9,3,'-5783481.21694835730858468677') +-- !query 385 schema +struct<> +-- !query 385 output + + + +-- !query 386 +INSERT INTO num_exp_add VALUES (9,4,'-17127342.633147420') +-- !query 386 schema +struct<> +-- !query 386 output + + + +-- !query 387 +INSERT INTO num_exp_sub VALUES (9,4,'-32726265.456947420') +-- !query 387 schema +struct<> +-- !query 387 output + + + +-- !query 388 +INSERT INTO num_exp_mul VALUES (9,4,'-194415646271340.1815956522980') +-- !query 388 schema +struct<> +-- !query 388 output + + + +-- !query 389 +INSERT INTO num_exp_div VALUES (9,4,'-3.19596478892958416484') +-- !query 389 schema +struct<> +-- !query 389 output + + + +-- !query 390 +INSERT INTO num_exp_add VALUES (9,5,'-24910407.006556420') +-- !query 390 schema +struct<> +-- !query 390 output + + + +-- !query 391 +INSERT INTO num_exp_sub VALUES (9,5,'-24943201.083538420') +-- !query 391 schema +struct<> +-- !query 391 output + + + +-- !query 392 +INSERT INTO num_exp_mul VALUES (9,5,'-408725765384.257043660243220') +-- !query 392 schema +struct<> +-- !query 392 output + + + +-- !query 393 +INSERT INTO num_exp_div VALUES (9,5,'-1520.20159364322004505807') +-- !query 393 schema +struct<> +-- !query 393 output + + + +-- !query 394 +INSERT INTO num_exp_add VALUES (9,6,'-24832902.467417160') +-- !query 394 schema +struct<> +-- !query 394 output + + + +-- !query 395 +INSERT INTO num_exp_sub VALUES (9,6,'-25020705.622677680') +-- !query 395 schema +struct<> +-- !query 395 output + + + +-- !query 396 +INSERT INTO num_exp_mul VALUES (9,6,'-2340666225110.29929521292692920') +-- !query 396 schema +struct<> +-- !query 396 output + + + +-- !query 397 +INSERT INTO num_exp_div VALUES (9,6,'-265.45671195426965751280') +-- !query 397 schema +struct<> +-- !query 397 output + + + +-- !query 398 +INSERT INTO num_exp_add VALUES (9,7,'-107955289.045047420') +-- !query 398 schema +struct<> +-- !query 398 output + + + +-- !query 399 +INSERT INTO num_exp_sub VALUES (9,7,'58101680.954952580') +-- !query 399 schema +struct<> +-- !query 399 output + + + +-- !query 400 +INSERT INTO num_exp_mul VALUES (9,7,'2069634775752159.035758700') +-- !query 400 schema +struct<> +-- !query 400 output + + + +-- !query 401 +INSERT INTO num_exp_div VALUES (9,7,'.30021990699995814689') +-- !query 401 schema +struct<> +-- !query 401 output + + + +-- !query 402 +INSERT INTO num_exp_add VALUES (9,8,'-24851923.045047420') +-- !query 402 schema +struct<> +-- !query 402 output + + + +-- !query 403 +INSERT INTO num_exp_sub VALUES (9,8,'-25001685.045047420') +-- !query 403 schema +struct<> +-- !query 403 output + + + +-- !query 404 +INSERT INTO num_exp_mul VALUES (9,8,'-1866544013697.195857020') +-- !query 404 schema +struct<> +-- !query 404 output + + + +-- !query 405 +INSERT INTO num_exp_div VALUES (9,8,'-332.88556569820675471748') +-- !query 405 schema +struct<> +-- !query 405 output + + + +-- !query 406 +INSERT INTO num_exp_add VALUES (9,9,'-49853608.090094840') +-- !query 406 schema +struct<> +-- !query 406 output + + + +-- !query 407 +INSERT INTO num_exp_sub VALUES (9,9,'0') +-- !query 407 schema +struct<> +-- !query 407 output + + + +-- !query 408 +INSERT INTO num_exp_mul VALUES (9,9,'621345559900192.420120630048656400') +-- !query 408 schema +struct<> +-- !query 408 output + + + +-- !query 409 +INSERT INTO num_exp_div VALUES (9,9,'1.00000000000000000000') +-- !query 409 schema +struct<> +-- !query 409 output + + + +-- !query 410 +INSERT INTO num_exp_sqrt VALUES (0,'0') +-- !query 410 schema +struct<> +-- !query 410 output + + + +-- !query 411 +INSERT INTO num_exp_sqrt VALUES (1,'0') +-- !query 411 schema +struct<> +-- !query 411 output + + + +-- !query 412 +INSERT INTO num_exp_sqrt VALUES (2,'5859.90547836712524903505') +-- !query 412 schema +struct<> +-- !query 412 output + + + +-- !query 413 +INSERT INTO num_exp_sqrt VALUES (3,'2.07605394920266944396') +-- !query 413 schema +struct<> +-- !query 413 output + + + +-- !query 414 +INSERT INTO num_exp_sqrt VALUES (4,'2792.75158435189147418923') +-- !query 414 schema +struct<> +-- !query 414 output + + + +-- !query 415 +INSERT INTO num_exp_sqrt VALUES (5,'128.05092147657509145473') +-- !query 415 schema +struct<> +-- !query 415 output + + + +-- !query 416 +INSERT INTO num_exp_sqrt VALUES (6,'306.43364311096782703406') +-- !query 416 schema +struct<> +-- !query 416 output + + + +-- !query 417 +INSERT INTO num_exp_sqrt VALUES (7,'9111.99676251039939975230') +-- !query 417 schema +struct<> +-- !query 417 output + + + +-- !query 418 +INSERT INTO num_exp_sqrt VALUES (8,'273.64392922189960397542') +-- !query 418 schema +struct<> +-- !query 418 output + + + +-- !query 419 +INSERT INTO num_exp_sqrt VALUES (9,'4992.67503899937593364766') +-- !query 419 schema +struct<> +-- !query 419 output + + + +-- !query 420 +INSERT INTO num_exp_ln VALUES (0,'NaN') +-- !query 420 schema +struct<> +-- !query 420 output + + + +-- !query 421 +INSERT INTO num_exp_ln VALUES (1,'NaN') +-- !query 421 schema +struct<> +-- !query 421 output + + + +-- !query 422 +INSERT INTO num_exp_ln VALUES (2,'17.35177750493897715514') +-- !query 422 schema +struct<> +-- !query 422 output + + + +-- !query 423 +INSERT INTO num_exp_ln VALUES (3,'1.46093790411565641971') +-- !query 423 schema +struct<> +-- !query 423 output + + + +-- !query 424 +INSERT INTO num_exp_ln VALUES (4,'15.86956523951936572464') +-- !query 424 schema +struct<> +-- !query 424 output + + + +-- !query 425 +INSERT INTO num_exp_ln VALUES (5,'9.70485601768871834038') +-- !query 425 schema +struct<> +-- !query 425 output + + + +-- !query 426 +INSERT INTO num_exp_ln VALUES (6,'11.45000246622944403127') +-- !query 426 schema +struct<> +-- !query 426 output + + + +-- !query 427 +INSERT INTO num_exp_ln VALUES (7,'18.23469429965478772991') +-- !query 427 schema +struct<> +-- !query 427 output + + + +-- !query 428 +INSERT INTO num_exp_ln VALUES (8,'11.22365546576315513668') +-- !query 428 schema +struct<> +-- !query 428 output + + + +-- !query 429 +INSERT INTO num_exp_ln VALUES (9,'17.03145425013166006962') +-- !query 429 schema +struct<> +-- !query 429 output + + + +-- !query 430 +INSERT INTO num_exp_log10 VALUES (0,'NaN') +-- !query 430 schema +struct<> +-- !query 430 output + + + +-- !query 431 +INSERT INTO num_exp_log10 VALUES (1,'NaN') +-- !query 431 schema +struct<> +-- !query 431 output + + + +-- !query 432 +INSERT INTO num_exp_log10 VALUES (2,'7.53578122160797276459') +-- !query 432 schema +struct<> +-- !query 432 output + + + +-- !query 433 +INSERT INTO num_exp_log10 VALUES (3,'.63447727016073160075') +-- !query 433 schema +struct<> +-- !query 433 output + + + +-- !query 434 +INSERT INTO num_exp_log10 VALUES (4,'6.89206461372691743345') +-- !query 434 schema +struct<> +-- !query 434 output + + + +-- !query 435 +INSERT INTO num_exp_log10 VALUES (5,'4.21476541614777768626') +-- !query 435 schema +struct<> +-- !query 435 output + + + +-- !query 436 +INSERT INTO num_exp_log10 VALUES (6,'4.97267288886207207671') +-- !query 436 schema +struct<> +-- !query 436 output + + + +-- !query 437 +INSERT INTO num_exp_log10 VALUES (7,'7.91922711353275546914') +-- !query 437 schema +struct<> +-- !query 437 output + + + +-- !query 438 +INSERT INTO num_exp_log10 VALUES (8,'4.87437163556421004138') +-- !query 438 schema +struct<> +-- !query 438 output + + + +-- !query 439 +INSERT INTO num_exp_log10 VALUES (9,'7.39666659961986567059') +-- !query 439 schema +struct<> +-- !query 439 output + + + +-- !query 440 +INSERT INTO num_exp_power_10_ln VALUES (0,'NaN') +-- !query 440 schema +struct<> +-- !query 440 output + + + +-- !query 441 +INSERT INTO num_exp_power_10_ln VALUES (1,'NaN') +-- !query 441 schema +struct<> +-- !query 441 output + + + +-- !query 442 +INSERT INTO num_exp_power_10_ln VALUES (2,'224790267919917955.13261618583642653184') +-- !query 442 schema +struct<> +-- !query 442 output + + + +-- !query 443 +INSERT INTO num_exp_power_10_ln VALUES (3,'28.90266599445155957393') +-- !query 443 schema +struct<> +-- !query 443 output + + + +-- !query 444 +INSERT INTO num_exp_power_10_ln VALUES (4,'7405685069594999.07733999469386277636') +-- !query 444 schema +struct<> +-- !query 444 output + + + +-- !query 445 +INSERT INTO num_exp_power_10_ln VALUES (5,'5068226527.32127265408584640098') +-- !query 445 schema +struct<> +-- !query 445 output + + + +-- !query 446 +INSERT INTO num_exp_power_10_ln VALUES (6,'281839893606.99372343357047819067') +-- !query 446 schema +struct<> +-- !query 446 output + + + +-- !query 447 +INSERT INTO num_exp_power_10_ln VALUES (7,'1716699575118597095.42330819910640247627') +-- !query 447 schema +struct<> +-- !query 447 output + + + +-- !query 448 +INSERT INTO num_exp_power_10_ln VALUES (8,'167361463828.07491320069016125952') +-- !query 448 schema +struct<> +-- !query 448 output + + + +-- !query 449 +INSERT INTO num_exp_power_10_ln VALUES (9,'107511333880052007.04141124673540337457') +-- !query 449 schema +struct<> +-- !query 449 output + + + +-- !query 450 +INSERT INTO num_data VALUES (0, '0') +-- !query 450 schema +struct<> +-- !query 450 output + + + +-- !query 451 +INSERT INTO num_data VALUES (1, '0') +-- !query 451 schema +struct<> +-- !query 451 output + + + +-- !query 452 +INSERT INTO num_data VALUES (2, '-34338492.215397047') +-- !query 452 schema +struct<> +-- !query 452 output + + + +-- !query 453 +INSERT INTO num_data VALUES (3, '4.31') +-- !query 453 schema +struct<> +-- !query 453 output + + + +-- !query 454 +INSERT INTO num_data VALUES (4, '7799461.4119') +-- !query 454 schema +struct<> +-- !query 454 output + + + +-- !query 455 +INSERT INTO num_data VALUES (5, '16397.038491') +-- !query 455 schema +struct<> +-- !query 455 output + + + +-- !query 456 +INSERT INTO num_data VALUES (6, '93901.57763026') +-- !query 456 schema +struct<> +-- !query 456 output + + + +-- !query 457 +INSERT INTO num_data VALUES (7, '-83028485') +-- !query 457 schema +struct<> +-- !query 457 output + + + +-- !query 458 +INSERT INTO num_data VALUES (8, '74881') +-- !query 458 schema +struct<> +-- !query 458 output + + + +-- !query 459 +INSERT INTO num_data VALUES (9, '-24926804.045047420') +-- !query 459 schema +struct<> +-- !query 459 output + + + +-- !query 460 +SELECT * FROM num_data +-- !query 460 schema +struct +-- !query 460 output +0 0 +1 0 +2 -34338492.215397047 +3 4.31 +4 7799461.4119 +5 16397.038491 +6 93901.57763026 +7 -83028485 +8 74881 +9 -24926804.04504742 + + +-- !query 461 +TRUNCATE TABLE num_result +-- !query 461 schema +struct<> +-- !query 461 output + + + +-- !query 462 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val + t2.val + FROM num_data t1, num_data t2 +-- !query 462 schema +struct<> +-- !query 462 output + + + +-- !query 463 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 463 schema +struct +-- !query 463 output + + + +-- !query 464 +TRUNCATE TABLE num_result +-- !query 464 schema +struct<> +-- !query 464 output + + + +-- !query 465 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val + t2.val, 10) + FROM num_data t1, num_data t2 +-- !query 465 schema +struct<> +-- !query 465 output + + + +-- !query 466 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 10) as expected + FROM num_result t1, num_exp_add t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 10) +-- !query 466 schema +struct +-- !query 466 output + + + +-- !query 467 +TRUNCATE TABLE num_result +-- !query 467 schema +struct<> +-- !query 467 output + + + +-- !query 468 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val - t2.val + FROM num_data t1, num_data t2 +-- !query 468 schema +struct<> +-- !query 468 output + + + +-- !query 469 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 469 schema +struct +-- !query 469 output + + + +-- !query 470 +TRUNCATE TABLE num_result +-- !query 470 schema +struct<> +-- !query 470 output + + + +-- !query 471 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val - t2.val, 40) + FROM num_data t1, num_data t2 +-- !query 471 schema +struct<> +-- !query 471 output + + + +-- !query 472 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 40) + FROM num_result t1, num_exp_sub t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 40) +-- !query 472 schema +struct +-- !query 472 output + + + +-- !query 473 +TRUNCATE TABLE num_result +-- !query 473 schema +struct<> +-- !query 473 output + + + +-- !query 474 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val, t2.val, t1.val * t2.val + FROM num_data t1, num_data t2 +-- !query 474 schema +struct<> +-- !query 474 output +org.apache.spark.sql.AnalysisException +`default`.`num_result` requires that the data to be inserted have the same number of columns as the target table: target table has 3 column(s) but the inserted data has 5 column(s), including 0 partition column(s) having constant value(s).; + + +-- !query 475 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 475 schema +struct +-- !query 475 output + + + +-- !query 476 +TRUNCATE TABLE num_result +-- !query 476 schema +struct<> +-- !query 476 output + + + +-- !query 477 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val * t2.val, 30) + FROM num_data t1, num_data t2 +-- !query 477 schema +struct<> +-- !query 477 output + + + +-- !query 478 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 30) as expected + FROM num_result t1, num_exp_mul t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 30) +-- !query 478 schema +struct +-- !query 478 output +2 2 1179132047626883.596862 1179132047626883.5968621359 +2 3 -147998901.448361 -147998901.4483612726 +2 4 -267821744976817.811114 -267821744976817.8111137107 +2 5 -563049578578.769243 -563049578578.7692425067 +2 6 -3224438592470.184498 -3224438592470.1844981193 +2 7 2851072985828710.485884 2851072985828710.485883795 +2 8 -2571300635581.146276 -2571300635581.146276407 +2 9 855948866655588.453742 855948866655588.4537415092 +3 2 -147998901.448361 -147998901.4483612726 +3 5 70671.235896 70671.23589621 +3 6 404715.799586 404715.7995864206 +3 9 -107434525.434154 -107434525.4341543802 +4 2 -267821744976817.811114 -267821744976817.8111137107 +4 4 60831598315717.141462 60831598315717.14146161 +4 5 127888068979.993505 127888068979.9935054429 +4 6 732381731243.745116 732381731243.7451157641 +4 9 -194415646271340.181596 -194415646271340.1815956523 +5 2 -563049578578.769243 -563049578578.7692425067 +5 3 70671.235896 70671.23589621 +5 4 127888068979.993505 127888068979.9935054429 +5 5 268862871.275336 268862871.2753355571 +5 6 1539707782.768998 1539707782.7689977863 +5 9 -408725765384.257044 -408725765384.2570436602 +6 2 -3224438592470.184498 -3224438592470.1844981193 +6 3 404715.799586 404715.7995864206 +6 4 732381731243.745116 732381731243.7451157641 +6 5 1539707782.768998 1539707782.7689977863 +6 6 8817506281.451745 8817506281.4517452373 +6 7 -7796505729750.377956 -7796505729750.3779561 +6 8 7031444034.531499 7031444034.53149906 +6 9 -2340666225110.299295 -2340666225110.2992952129 +7 2 2851072985828710.485884 2851072985828710.485883795 +7 6 -7796505729750.377956 -7796505729750.3779561 +7 9 2069634775752159.035759 2069634775752159.0357587 +8 2 -2571300635581.146276 -2571300635581.146276407 +8 6 7031444034.531499 7031444034.53149906 +8 9 -1866544013697.195857 -1866544013697.19585702 +9 2 855948866655588.453742 855948866655588.4537415092 +9 3 -107434525.434154 -107434525.4341543802 +9 4 -194415646271340.181596 -194415646271340.1815956523 +9 5 -408725765384.257044 -408725765384.2570436602 +9 6 -2340666225110.299295 -2340666225110.2992952129 +9 7 2069634775752159.035759 2069634775752159.0357587 +9 8 -1866544013697.195857 -1866544013697.19585702 +9 9 621345559900192.420121 621345559900192.42012063 + + +-- !query 479 +TRUNCATE TABLE num_result +-- !query 479 schema +struct<> +-- !query 479 output + + + +-- !query 480 +INSERT INTO num_result SELECT t1.id, t2.id, t1.val / t2.val + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query 480 schema +struct<> +-- !query 480 output + + + +-- !query 481 +SELECT t1.id1, t1.id2, t1.result, t2.expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != t2.expected +-- !query 481 schema +struct +-- !query 481 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 482 +TRUNCATE TABLE num_result +-- !query 482 schema +struct<> +-- !query 482 output + + + +-- !query 483 +INSERT INTO num_result SELECT t1.id, t2.id, round(t1.val / t2.val, 80) + FROM num_data t1, num_data t2 + WHERE t2.val != '0.0' +-- !query 483 schema +struct<> +-- !query 483 output + + + +-- !query 484 +SELECT t1.id1, t1.id2, t1.result, round(t2.expected, 80) as expected + FROM num_result t1, num_exp_div t2 + WHERE t1.id1 = t2.id1 AND t1.id2 = t2.id2 + AND t1.result != round(t2.expected, 80) +-- !query 484 schema +struct +-- !query 484 output +2 3 -7967167.567378 -7967167.5673775051 +2 4 -4.402675 -4.4026748005 +2 5 -2094.188669 -2094.1886691456 +2 6 -365.685999 -365.6859989148 +2 7 0.413575 0.4135748378 +2 8 -458.574167 -458.5741672173 +2 9 1.377573 1.3775729995 +3 2 0 -0.0000001255 +3 4 0.000001 0.0000005526 +3 5 0.000263 0.0002628523 +3 6 0.000046 0.0000458991 +3 7 0 -0.0000000519 +3 8 0.000058 0.000057558 +3 9 0 -0.0000001729 +4 2 -0.227135 -0.22713465 +4 3 1809619.817146 1809619.8171461717 +4 5 475.66281 475.6628104631 +4 6 83.059961 83.0599613844 +4 7 -0.093937 -0.093937176 +4 8 104.158083 104.1580829837 +4 9 -0.312895 -0.3128945611 +5 2 -0.000478 -0.0004775119 +5 3 3804.417283 3804.4172832947 +5 4 0.002102 0.0021023296 +5 6 0.174619 0.1746194143 +5 7 -0.000197 -0.0001974869 +5 8 0.218975 0.2189746196 +5 9 -0.000658 -0.0006578075 +6 2 -0.002735 -0.0027345865 +6 3 21786.908963 21786.9089629374 +6 4 0.012039 0.0120394951 +6 5 5.72674 5.7267400867 +6 7 -0.001131 -0.0011309562 +6 8 1.254011 1.2540107321 +6 9 -0.003767 -0.0037670925 +7 2 2.417942 2.4179420715 +7 3 -19264149.651972 -19264149.6519721578 +7 4 -10.645413 -10.6454126273 +7 5 -5063.626889 -5063.6268888173 +7 6 -884.207562 -884.2075617401 +7 8 -1108.805772 -1108.8057718246 +7 9 3.330892 3.330891712 +8 2 -0.002181 -0.0021806723 +8 3 17373.781903 17373.7819025522 +8 4 0.009601 0.0096007911 +8 5 4.566739 4.5667392951 +8 6 0.797441 0.7974413411 +8 7 -0.000902 -0.0009018712 +8 9 -0.003004 -0.0030040353 +9 2 0.725914 0.7259143438 +9 3 -5783481.216948 -5783481.2169483573 +9 4 -3.195965 -3.1959647889 +9 5 -1520.201594 -1520.2015936432 +9 6 -265.456712 -265.4567119543 +9 7 0.30022 0.300219907 +9 8 -332.885566 -332.8855656982 + + +-- !query 485 +TRUNCATE TABLE num_result +-- !query 485 schema +struct<> +-- !query 485 output + + + +-- !query 486 +INSERT INTO num_result SELECT id, 0, SQRT(ABS(val)) + FROM num_data +-- !query 486 schema +struct<> +-- !query 486 output + + + +-- !query 487 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_sqrt t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 487 schema +struct +-- !query 487 output + + + +-- !query 488 +TRUNCATE TABLE num_result +-- !query 488 schema +struct<> +-- !query 488 output + + + +-- !query 489 +INSERT INTO num_result SELECT id, 0, LN(ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 489 schema +struct<> +-- !query 489 output + + + +-- !query 490 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 490 schema +struct +-- !query 490 output + + + +-- !query 491 +TRUNCATE TABLE num_result +-- !query 491 schema +struct<> +-- !query 491 output + + + +-- !query 492 +INSERT INTO num_result SELECT id, 0, LOG(cast('10' as decimal(38, 18)), ABS(val)) + FROM num_data + WHERE val != '0.0' +-- !query 492 schema +struct<> +-- !query 492 output + + + +-- !query 493 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_log10 t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 493 schema +struct +-- !query 493 output + + + +-- !query 494 +TRUNCATE TABLE num_result +-- !query 494 schema +struct<> +-- !query 494 output + + + +-- !query 495 +INSERT INTO num_result SELECT id, 0, POWER(cast('10' as decimal(38, 18)), LN(ABS(round(val,200)))) + FROM num_data + WHERE val != '0.0' +-- !query 495 schema +struct<> +-- !query 495 output + + + +-- !query 496 +SELECT t1.id1, t1.result, t2.expected + FROM num_result t1, num_exp_power_10_ln t2 + WHERE t1.id1 = t2.id + AND t1.result != t2.expected +-- !query 496 schema +struct +-- !query 496 output +2 224790267919917472 224790267919917955.1326161858 +4 7405685069595001 7405685069594999.0773399947 +5 5068226527.321263 5068226527.3212726541 +6 281839893606.99365 281839893606.9937234336 +7 1716699575118595840 1716699575118597095.4233081991 +8 167361463828.0749 167361463828.0749132007 +9 107511333880051856 107511333880052007.0414112467 + + +-- !query 497 +SELECT AVG(val) FROM num_data +-- !query 497 schema +struct +-- !query 497 output +-13430913.5922423207 + + +-- !query 498 +CREATE TABLE fract_only (id int, val decimal(4,4)) USING parquet +-- !query 498 schema +struct<> +-- !query 498 output + + + +-- !query 499 +INSERT INTO fract_only VALUES (1, '0.0') +-- !query 499 schema +struct<> +-- !query 499 output + + + +-- !query 500 +INSERT INTO fract_only VALUES (2, '0.1') +-- !query 500 schema +struct<> +-- !query 500 output + + + +-- !query 501 +INSERT INTO fract_only VALUES (4, '-0.9999') +-- !query 501 schema +struct<> +-- !query 501 output + + + +-- !query 502 +INSERT INTO fract_only VALUES (5, '0.99994') +-- !query 502 schema +struct<> +-- !query 502 output + + + +-- !query 503 +INSERT INTO fract_only VALUES (7, '0.00001') +-- !query 503 schema +struct<> +-- !query 503 output + + + +-- !query 504 +INSERT INTO fract_only VALUES (8, '0.00017') +-- !query 504 schema +struct<> +-- !query 504 output + + + +-- !query 505 +SELECT * FROM fract_only +-- !query 505 schema +struct +-- !query 505 output +1 0 +2 0.1 +4 -0.9999 +5 0.9999 +7 0 +8 0.0002 + + +-- !query 506 +DROP TABLE fract_only +-- !query 506 schema +struct<> +-- !query 506 output + + + +-- !query 507 +SELECT decimal(double('NaN')) +-- !query 507 schema +struct +-- !query 507 output +NULL + + +-- !query 508 +SELECT decimal(double('Infinity')) +-- !query 508 schema +struct +-- !query 508 output +NULL + + +-- !query 509 +SELECT decimal(double('-Infinity')) +-- !query 509 schema +struct +-- !query 509 output +NULL + + +-- !query 510 +SELECT decimal(float('NaN')) +-- !query 510 schema +struct +-- !query 510 output +NULL + + +-- !query 511 +SELECT decimal(float('Infinity')) +-- !query 511 schema +struct +-- !query 511 output +NULL + + +-- !query 512 +SELECT decimal(float('-Infinity')) +-- !query 512 schema +struct +-- !query 512 output +NULL + + +-- !query 513 +CREATE TABLE ceil_floor_round (a decimal(38, 18)) USING parquet +-- !query 513 schema +struct<> +-- !query 513 output + + + +-- !query 514 +INSERT INTO ceil_floor_round VALUES ('-5.5') +-- !query 514 schema +struct<> +-- !query 514 output + + + +-- !query 515 +INSERT INTO ceil_floor_round VALUES ('-5.499999') +-- !query 515 schema +struct<> +-- !query 515 output + + + +-- !query 516 +INSERT INTO ceil_floor_round VALUES ('9.5') +-- !query 516 schema +struct<> +-- !query 516 output + + + +-- !query 517 +INSERT INTO ceil_floor_round VALUES ('9.4999999') +-- !query 517 schema +struct<> +-- !query 517 output + + + +-- !query 518 +INSERT INTO ceil_floor_round VALUES ('0.0') +-- !query 518 schema +struct<> +-- !query 518 output + + + +-- !query 519 +INSERT INTO ceil_floor_round VALUES ('0.0000001') +-- !query 519 schema +struct<> +-- !query 519 output + + + +-- !query 520 +INSERT INTO ceil_floor_round VALUES ('-0.000001') +-- !query 520 schema +struct<> +-- !query 520 output + + + +-- !query 521 +SELECT a, ceil(a), ceiling(a), floor(a), round(a) FROM ceil_floor_round +-- !query 521 schema +struct +-- !query 521 output +-0.000001 0 0 -1 0 +-5.499999 -5 -5 -6 -5 +-5.5 -5 -5 -6 -6 +0 0 0 0 0 +0.0000001 1 1 0 0 +9.4999999 10 10 9 9 +9.5 10 10 9 10 + + +-- !query 522 +DROP TABLE ceil_floor_round +-- !query 522 schema +struct<> +-- !query 522 output + + + +-- !query 523 +CREATE TABLE num_input_test (n1 decimal(38, 18)) USING parquet +-- !query 523 schema +struct<> +-- !query 523 output + + + +-- !query 524 +INSERT INTO num_input_test VALUES (trim(' 123')) +-- !query 524 schema +struct<> +-- !query 524 output + + + +-- !query 525 +INSERT INTO num_input_test VALUES (trim(' 3245874 ')) +-- !query 525 schema +struct<> +-- !query 525 output + + + +-- !query 526 +INSERT INTO num_input_test VALUES (trim(' -93853')) +-- !query 526 schema +struct<> +-- !query 526 output + + + +-- !query 527 +INSERT INTO num_input_test VALUES ('555.50') +-- !query 527 schema +struct<> +-- !query 527 output + + + +-- !query 528 +INSERT INTO num_input_test VALUES ('-555.50') +-- !query 528 schema +struct<> +-- !query 528 output + + + +-- !query 529 +SELECT * FROM num_input_test +-- !query 529 schema +struct +-- !query 529 output +-555.5 +-93853 +123 +3245874 +555.5 + + +-- !query 530 +select cast(999999999999999999999 as decimal(38, 0))/1000000000000000000000 +-- !query 530 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) / CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(38,6)> +-- !query 530 output +1 + + +-- !query 531 +select mod(cast(999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 531 schema +struct<(CAST(CAST(999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 531 output +999999999999999999999 + + +-- !query 532 +select mod(cast(-9999999999999999999999 as decimal(38, 0)),1000000000000000000000) +-- !query 532 schema +struct<(CAST(CAST(-9999999999999999999999 AS DECIMAL(38,0)) AS DECIMAL(38,0)) % CAST(1000000000000000000000 AS DECIMAL(38,0))):decimal(22,0)> +-- !query 532 output +-999999999999999999999 + + +-- !query 533 +select mod (70.0,70) +-- !query 533 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) % CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(3,1)> +-- !query 533 output +0 + + +-- !query 534 +select 70.0 / 70 +-- !query 534 schema +struct<(CAST(70.0 AS DECIMAL(3,1)) / CAST(CAST(70 AS DECIMAL(2,0)) AS DECIMAL(3,1))):decimal(8,6)> +-- !query 534 output +1 + + +-- !query 535 +select 12345678901234567890 % 123 +-- !query 535 schema +struct<(CAST(12345678901234567890 AS DECIMAL(20,0)) % CAST(CAST(123 AS DECIMAL(3,0)) AS DECIMAL(20,0))):decimal(3,0)> +-- !query 535 output +78 + + +-- !query 536 +select exp(0.0) +-- !query 536 schema +struct +-- !query 536 output +1.0 + + +-- !query 537 +select exp(1.0) +-- !query 537 schema +struct +-- !query 537 output +2.718281828459045 + + +-- !query 538 +select exp(32.999) +-- !query 538 schema +struct +-- !query 538 output +2.1442904349215556E14 + + +-- !query 539 +select exp(-32.999) +-- !query 539 schema +struct +-- !query 539 output +4.663547361468238E-15 + + +-- !query 540 +select exp(123.456) +-- !query 540 schema +struct +-- !query 540 output +4.132944352778106E53 + + +-- !query 541 +select exp(-123.456) +-- !query 541 schema +struct +-- !query 541 output +2.4195825412645934E-54 + + +-- !query 542 +select exp(1234.5678) +-- !query 542 schema +struct +-- !query 542 output +Infinity + + +-- !query 543 +select * from range(cast(0.0 as decimal(38, 18)), cast(4.0 as decimal(38, 18))) +-- !query 543 schema +struct +-- !query 543 output +0 +1 +2 +3 + + +-- !query 544 +select * from range(cast(0.1 as decimal(38, 18)), cast(4.0 as decimal(38, 18)), cast(1.3 as decimal(38, 18))) +-- !query 544 schema +struct +-- !query 544 output +0 +1 +2 +3 + + +-- !query 545 +select * from range(cast(4.0 as decimal(38, 18)), cast(-1.5 as decimal(38, 18)), cast(-2.2 as decimal(38, 18))) +-- !query 545 schema +struct +-- !query 545 output +0 +2 +4 + + +-- !query 546 +select ln(1.2345678e-28) +-- !query 546 schema +struct +-- !query 546 output +-64.26166165451762 + + +-- !query 547 +select ln(0.0456789) +-- !query 547 schema +struct +-- !query 547 output +-3.0861187944847437 + + +-- !query 548 +select ln(0.99949452) +-- !query 548 schema +struct +-- !query 548 output +-5.056077980832118E-4 + + +-- !query 549 +select ln(1.00049687395) +-- !query 549 schema +struct +-- !query 549 output +4.967505490136803E-4 + + +-- !query 550 +select ln(1234.567890123456789) +-- !query 550 schema +struct +-- !query 550 output +7.11847630129779 + + +-- !query 551 +select ln(5.80397490724e5) +-- !query 551 schema +struct +-- !query 551 output +13.271468476626518 + + +-- !query 552 +select ln(9.342536355e34) +-- !query 552 schema +struct +-- !query 552 output +80.52247093552418 + + +-- !query 553 +select log(3.4634998359873254962349856073435545) +-- !query 553 schema +struct +-- !query 553 output +1.2422795911259166 + + +-- !query 554 +select log(9.999999999999999999) +-- !query 554 schema +struct +-- !query 554 output +2.302585092994046 + + +-- !query 555 +select log(10.00000000000000000) +-- !query 555 schema +struct +-- !query 555 output +2.302585092994046 + + +-- !query 556 +select log(10.00000000000000001) +-- !query 556 schema +struct +-- !query 556 output +2.302585092994046 + + +-- !query 557 +select log(590489.45235237) +-- !query 557 schema +struct +-- !query 557 output +13.288707052228641 + + +-- !query 558 +select log(0.99923, 4.58934e34) +-- !query 558 schema +struct +-- !query 558 output +-103611.55579543479 + + +-- !query 559 +select log(1.000016, 8.452010e18) +-- !query 559 schema +struct +-- !query 559 output +2723830.287707013 + + +-- !query 560 +SELECT SUM(decimal(9999)) FROM range(1, 100001) +-- !query 560 schema +struct +-- !query 560 output +999900000 + + +-- !query 561 +SELECT SUM(decimal(-9999)) FROM range(1, 100001) +-- !query 561 schema +struct +-- !query 561 output +-999900000 + + +-- !query 562 +DROP TABLE num_data +-- !query 562 schema +struct<> +-- !query 562 output + + + +-- !query 563 +DROP TABLE num_exp_add +-- !query 563 schema +struct<> +-- !query 563 output + + + +-- !query 564 +DROP TABLE num_exp_sub +-- !query 564 schema +struct<> +-- !query 564 output + + + +-- !query 565 +DROP TABLE num_exp_div +-- !query 565 schema +struct<> +-- !query 565 output + + + +-- !query 566 +DROP TABLE num_exp_mul +-- !query 566 schema +struct<> +-- !query 566 output + + + +-- !query 567 +DROP TABLE num_exp_sqrt +-- !query 567 schema +struct<> +-- !query 567 output + + + +-- !query 568 +DROP TABLE num_exp_ln +-- !query 568 schema +struct<> +-- !query 568 output + + + +-- !query 569 +DROP TABLE num_exp_log10 +-- !query 569 schema +struct<> +-- !query 569 output + + + +-- !query 570 +DROP TABLE num_exp_power_10_ln +-- !query 570 schema +struct<> +-- !query 570 output + + + +-- !query 571 +DROP TABLE num_result +-- !query 571 schema +struct<> +-- !query 571 output + + + +-- !query 572 +DROP TABLE num_input_test +-- !query 572 schema +struct<> +-- !query 572 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out new file mode 100644 index 0000000000000..797f808dad11e --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select.sql.out @@ -0,0 +1,543 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 37 + + +-- !query 0 +create or replace temporary view onek2 as select * from onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create or replace temporary view INT8_TBL as select * from values + (cast(trim(' 123 ') as bigint), cast(trim(' 456') as bigint)), + (cast(trim('123 ') as bigint),cast('4567890123456789' as bigint)), + (cast('4567890123456789' as bigint),cast('123' as bigint)), + (cast(+4567890123456789 as bigint),cast('4567890123456789' as bigint)), + (cast('+4567890123456789' as bigint),cast('-4567890123456789' as bigint)) + as INT8_TBL(q1, q2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM onek + WHERE onek.unique1 < 10 + ORDER BY onek.unique1 +-- !query 2 schema +struct +-- !query 2 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 3 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC +-- !query 3 schema +struct +-- !query 3 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 4 +SELECT onek.unique1, onek.stringu1 FROM onek + WHERE onek.unique1 > 980 + ORDER BY stringu1 ASC +-- !query 4 schema +struct +-- !query 4 output +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA + + +-- !query 5 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 ASC, unique1 DESC +-- !query 5 schema +struct +-- !query 5 output +999 AAAAxx +995 AAAAxx +983 AAAAxx +982 AAAAxx +981 AAAAxx +998 HHHHxx +997 HHHHxx +993 HHHHxx +990 HHHHxx +986 HHHHxx +996 OOOOxx +991 OOOOxx +988 OOOOxx +987 OOOOxx +985 OOOOxx +994 VVVVxx +992 VVVVxx +989 VVVVxx +984 VVVVxx + + +-- !query 6 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 > 980 + ORDER BY string4 DESC, unique1 ASC +-- !query 6 schema +struct +-- !query 6 output +984 VVVVxx +989 VVVVxx +992 VVVVxx +994 VVVVxx +985 OOOOxx +987 OOOOxx +988 OOOOxx +991 OOOOxx +996 OOOOxx +986 HHHHxx +990 HHHHxx +993 HHHHxx +997 HHHHxx +998 HHHHxx +981 AAAAxx +982 AAAAxx +983 AAAAxx +995 AAAAxx +999 AAAAxx + + +-- !query 7 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 DESC, string4 ASC +-- !query 7 schema +struct +-- !query 7 output +19 OOOOxx +18 VVVVxx +17 HHHHxx +16 OOOOxx +15 VVVVxx +14 AAAAxx +13 OOOOxx +12 AAAAxx +11 OOOOxx +10 AAAAxx +9 HHHHxx +8 HHHHxx +7 VVVVxx +6 OOOOxx +5 HHHHxx +4 HHHHxx +3 VVVVxx +2 OOOOxx +1 OOOOxx +0 OOOOxx + + +-- !query 8 +SELECT onek.unique1, onek.string4 FROM onek + WHERE onek.unique1 < 20 + ORDER BY unique1 ASC, string4 DESC +-- !query 8 schema +struct +-- !query 8 output +0 OOOOxx +1 OOOOxx +2 OOOOxx +3 VVVVxx +4 HHHHxx +5 HHHHxx +6 OOOOxx +7 VVVVxx +8 HHHHxx +9 HHHHxx +10 AAAAxx +11 OOOOxx +12 AAAAxx +13 OOOOxx +14 AAAAxx +15 VVVVxx +16 OOOOxx +17 HHHHxx +18 VVVVxx +19 OOOOxx + + +-- !query 9 +SELECT onek2.* FROM onek2 WHERE onek2.unique1 < 10 +-- !query 9 schema +struct +-- !query 9 output +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx + + +-- !query 10 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 < 20 + ORDER BY unique1 DESC +-- !query 10 schema +struct +-- !query 10 output +19 TAAAAA +18 SAAAAA +17 RAAAAA +16 QAAAAA +15 PAAAAA +14 OAAAAA +13 NAAAAA +12 MAAAAA +11 LAAAAA +10 KAAAAA +9 JAAAAA +8 IAAAAA +7 HAAAAA +6 GAAAAA +5 FAAAAA +4 EAAAAA +3 DAAAAA +2 CAAAAA +1 BAAAAA +0 AAAAAA + + +-- !query 11 +SELECT onek2.unique1, onek2.stringu1 FROM onek2 + WHERE onek2.unique1 > 980 +-- !query 11 schema +struct +-- !query 11 output +981 TLAAAA +982 ULAAAA +983 VLAAAA +984 WLAAAA +985 XLAAAA +986 YLAAAA +987 ZLAAAA +988 AMAAAA +989 BMAAAA +990 CMAAAA +991 DMAAAA +992 EMAAAA +993 FMAAAA +994 GMAAAA +995 HMAAAA +996 IMAAAA +997 JMAAAA +998 KMAAAA +999 LMAAAA + + +-- !query 12 +CREATE TABLE tmp USING parquet AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +select foo.* from (select 1) as foo +-- !query 13 schema +struct<1:int> +-- !query 13 output +1 + + +-- !query 14 +select foo.* from (select null) as foo +-- !query 14 schema +struct +-- !query 14 output +NULL + + +-- !query 15 +select foo.* from (select 'xyzzy',1,null) as foo +-- !query 15 schema +struct +-- !query 15 output +xyzzy 1 NULL + + +-- !query 16 +select * from onek, values(147, 'RFAAAA'), (931, 'VJAAAA') as v (i, j) + WHERE onek.unique1 = v.i and onek.stringu1 = v.j +-- !query 16 schema +struct +-- !query 16 output +147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx 147 RFAAAA +931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx 931 VJAAAA + + +-- !query 17 +VALUES (1,2), (3,4+4), (7,77.7) +-- !query 17 schema +struct +-- !query 17 output +1 2 +3 8 +7 77.7 + + +-- !query 18 +VALUES (1,2), (3,4+4), (7,77.7) +UNION ALL +SELECT 2+2, 57 +UNION ALL +TABLE int8_tbl +-- !query 18 schema +struct +-- !query 18 output +1 2 +123 456 +123 4567890123456789 +3 8 +4 57 +4567890123456789 -4567890123456789 +4567890123456789 123 +4567890123456789 4567890123456789 +7 77.7 + + +-- !query 19 +CREATE OR REPLACE TEMPORARY VIEW foo AS +SELECT * FROM (values(42),(3),(10),(7),(null),(null),(1)) as foo (f1) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +SELECT * FROM foo ORDER BY f1 +-- !query 20 schema +struct +-- !query 20 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 21 +SELECT * FROM foo ORDER BY f1 ASC +-- !query 21 schema +struct +-- !query 21 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 22 +-- same thing +SELECT * FROM foo ORDER BY f1 NULLS FIRST +-- !query 22 schema +struct +-- !query 22 output +NULL +NULL +1 +3 +7 +10 +42 + + +-- !query 23 +SELECT * FROM foo ORDER BY f1 DESC +-- !query 23 schema +struct +-- !query 23 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 24 +SELECT * FROM foo ORDER BY f1 DESC NULLS LAST +-- !query 24 schema +struct +-- !query 24 output +42 +10 +7 +3 +1 +NULL +NULL + + +-- !query 25 +select * from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 25 schema +struct +-- !query 25 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 26 +select unique2 from onek2 where unique2 = 11 and stringu1 = 'ATAAAA' +-- !query 26 schema +struct +-- !query 26 output +11 + + +-- !query 27 +select * from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 27 schema +struct +-- !query 27 output +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx + + +-- !query 28 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 28 schema +struct +-- !query 28 output +11 + + +-- !query 29 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'C' +-- !query 29 schema +struct +-- !query 29 output +11 + + +-- !query 30 +select unique2 from onek2 where unique2 = 11 and stringu1 < 'B' +-- !query 30 schema +struct +-- !query 30 output +11 + + +-- !query 31 +select unique1, unique2 from onek2 + where (unique2 = 11 or unique1 = 0) and stringu1 < 'B' +-- !query 31 schema +struct +-- !query 31 output +0 998 +494 11 + + +-- !query 32 +select unique1, unique2 from onek2 + where (unique2 = 11 and stringu1 < 'B') or unique1 = 0 +-- !query 32 schema +struct +-- !query 32 output +0 998 +494 11 + + +-- !query 33 +SELECT 1 AS x ORDER BY x +-- !query 33 schema +struct +-- !query 33 output +1 + + +-- !query 34 +select * from (values (2),(null),(1)) v(k) where k = k order by k +-- !query 34 schema +struct +-- !query 34 output +1 +2 + + +-- !query 35 +select * from (values (2),(null),(1)) v(k) where k = k +-- !query 35 schema +struct +-- !query 35 output +1 +2 + + +-- !query 36 +drop table tmp +-- !query 36 schema +struct<> +-- !query 36 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out new file mode 100644 index 0000000000000..38eae1739f553 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_distinct.sql.out @@ -0,0 +1,225 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 19 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW tmp AS +SELECT two, stringu1, ten, string4 +FROM onek +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT DISTINCT two FROM tmp ORDER BY 1 +-- !query 1 schema +struct +-- !query 1 output +0 +1 + + +-- !query 2 +SELECT DISTINCT ten FROM tmp ORDER BY 1 +-- !query 2 schema +struct +-- !query 2 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 3 +SELECT DISTINCT string4 FROM tmp ORDER BY 1 +-- !query 3 schema +struct +-- !query 3 output +AAAAxx +HHHHxx +OOOOxx +VVVVxx + + +-- !query 4 +SELECT DISTINCT two, string4, ten + FROM tmp + ORDER BY two ASC, string4 ASC, ten ASC +-- !query 4 schema +struct +-- !query 4 output +0 AAAAxx 0 +0 AAAAxx 2 +0 AAAAxx 4 +0 AAAAxx 6 +0 AAAAxx 8 +0 HHHHxx 0 +0 HHHHxx 2 +0 HHHHxx 4 +0 HHHHxx 6 +0 HHHHxx 8 +0 OOOOxx 0 +0 OOOOxx 2 +0 OOOOxx 4 +0 OOOOxx 6 +0 OOOOxx 8 +0 VVVVxx 0 +0 VVVVxx 2 +0 VVVVxx 4 +0 VVVVxx 6 +0 VVVVxx 8 +1 AAAAxx 1 +1 AAAAxx 3 +1 AAAAxx 5 +1 AAAAxx 7 +1 AAAAxx 9 +1 HHHHxx 1 +1 HHHHxx 3 +1 HHHHxx 5 +1 HHHHxx 7 +1 HHHHxx 9 +1 OOOOxx 1 +1 OOOOxx 3 +1 OOOOxx 5 +1 OOOOxx 7 +1 OOOOxx 9 +1 VVVVxx 1 +1 VVVVxx 3 +1 VVVVxx 5 +1 VVVVxx 7 +1 VVVVxx 9 + + +-- !query 5 +SELECT count(*) FROM + (SELECT DISTINCT two, four, two FROM tenk1) ss +-- !query 5 schema +struct +-- !query 5 output +4 + + +-- !query 6 +CREATE OR REPLACE TEMPORARY VIEW disttable AS SELECT * FROM + (VALUES (1), (2), (3), (NULL)) + AS v(f1) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SELECT f1, f1 IS DISTINCT FROM 2 as `not 2` FROM disttable +-- !query 7 schema +struct +-- !query 7 output +1 true +2 false +3 true +NULL true + + +-- !query 8 +SELECT f1, f1 IS DISTINCT FROM NULL as `not null` FROM disttable +-- !query 8 schema +struct +-- !query 8 output +1 true +2 true +3 true +NULL false + + +-- !query 9 +SELECT f1, f1 IS DISTINCT FROM f1 as `false` FROM disttable +-- !query 9 schema +struct +-- !query 9 output +1 false +2 false +3 false +NULL false + + +-- !query 10 +SELECT f1, f1 IS DISTINCT FROM f1+1 as `not null` FROM disttable +-- !query 10 schema +struct +-- !query 10 output +1 true +2 true +3 true +NULL false + + +-- !query 11 +SELECT 1 IS DISTINCT FROM 2 as `yes` +-- !query 11 schema +struct +-- !query 11 output +true + + +-- !query 12 +SELECT 2 IS DISTINCT FROM 2 as `no` +-- !query 12 schema +struct +-- !query 12 output +false + + +-- !query 13 +SELECT 2 IS DISTINCT FROM null as `yes` +-- !query 13 schema +struct +-- !query 13 output +true + + +-- !query 14 +SELECT null IS DISTINCT FROM null as `no` +-- !query 14 schema +struct +-- !query 14 output +false + + +-- !query 15 +SELECT 1 IS NOT DISTINCT FROM 2 as `no` +-- !query 15 schema +struct +-- !query 15 output +false + + +-- !query 16 +SELECT 2 IS NOT DISTINCT FROM 2 as `yes` +-- !query 16 schema +struct +-- !query 16 output +true + + +-- !query 17 +SELECT 2 IS NOT DISTINCT FROM null as `no` +-- !query 17 schema +struct +-- !query 17 output +false + + +-- !query 18 +SELECT null IS NOT DISTINCT FROM null as `yes` +-- !query 18 schema +struct +-- !query 18 output +true diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out new file mode 100644 index 0000000000000..02536ebd8ebea --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_having.sql.out @@ -0,0 +1,187 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE TABLE test_having (a int, b int, c string, d string) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_having VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_having VALUES (1, 2, 'AAAA', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_having VALUES (2, 2, 'AAAA', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_having VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_having VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_having VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_having VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_having VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_having VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_having VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT b, c FROM test_having + GROUP BY b, c HAVING count(*) = 1 ORDER BY b, c +-- !query 11 schema +struct +-- !query 11 output +1 XXXX +3 bbbb + + +-- !query 12 +SELECT b, c FROM test_having + GROUP BY b, c HAVING b = 3 ORDER BY b, c +-- !query 12 schema +struct +-- !query 12 output +3 BBBB +3 bbbb + + +-- !query 13 +SELECT c, max(a) FROM test_having + GROUP BY c HAVING count(*) > 2 OR min(a) = max(a) + ORDER BY c +-- !query 13 schema +struct +-- !query 13 output +XXXX 0 +bbbb 5 + + +-- !query 14 +SELECT min(a), max(a) FROM test_having HAVING min(a) = max(a) +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT min(a), max(a) FROM test_having HAVING min(a) < max(a) +-- !query 15 schema +struct +-- !query 15 output +0 9 + + +-- !query 16 +SELECT a FROM test_having HAVING min(a) < max(a) +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +grouping expressions sequence is empty, and 'default.test_having.`a`' is not an aggregate function. Wrap '(min(default.test_having.`a`) AS `min(a#x)`, max(default.test_having.`a`) AS `max(a#x)`)' in windowing function(s) or wrap 'default.test_having.`a`' in first() (or first_value) if you don't care which value you get.; + + +-- !query 17 +SELECT 1 AS one FROM test_having HAVING a > 1 +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +cannot resolve '`a`' given input columns: [one]; line 1 pos 40 + + +-- !query 18 +SELECT 1 AS one FROM test_having HAVING 1 > 2 +-- !query 18 schema +struct +-- !query 18 output + + + +-- !query 19 +SELECT 1 AS one FROM test_having HAVING 1 < 2 +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT 1 AS one FROM test_having WHERE 1/a = 1 HAVING 1 < 2 +-- !query 20 schema +struct +-- !query 20 output +1 + + +-- !query 21 +DROP TABLE test_having +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out new file mode 100644 index 0000000000000..0675820b381da --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/select_implicit.sql.out @@ -0,0 +1,416 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 38 + + +-- !query 0 +CREATE TABLE test_missing_target (a int, b int, c string, d string) using parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO test_missing_target VALUES (0, 1, 'XXXX', 'A') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO test_missing_target VALUES (1, 2, 'ABAB', 'b') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO test_missing_target VALUES (2, 2, 'ABAB', 'c') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO test_missing_target VALUES (3, 3, 'BBBB', 'D') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO test_missing_target VALUES (4, 3, 'BBBB', 'e') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO test_missing_target VALUES (5, 3, 'bbbb', 'F') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO test_missing_target VALUES (6, 4, 'cccc', 'g') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO test_missing_target VALUES (7, 4, 'cccc', 'h') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO test_missing_target VALUES (8, 4, 'CCCC', 'I') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO test_missing_target VALUES (9, 4, 'CCCC', 'j') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +SELECT c, count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 11 schema +struct +-- !query 11 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 12 +SELECT count(*) FROM test_missing_target GROUP BY test_missing_target.c ORDER BY c +-- !query 12 schema +struct +-- !query 12 output +2 +2 +2 +1 +1 +2 + + +-- !query 13 +SELECT count(*) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(1)]; line 1 pos 61 + + +-- !query 14 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b +-- !query 14 schema +struct +-- !query 14 output +1 +2 +3 +4 + + +-- !query 15 +SELECT test_missing_target.b, count(*) + FROM test_missing_target GROUP BY b ORDER BY b +-- !query 15 schema +struct +-- !query 15 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 16 +SELECT c FROM test_missing_target ORDER BY a +-- !query 16 schema +struct +-- !query 16 output +XXXX +ABAB +ABAB +BBBB +BBBB +bbbb +cccc +cccc +CCCC +CCCC + + +-- !query 17 +SELECT count(*) FROM test_missing_target GROUP BY b ORDER BY b desc +-- !query 17 schema +struct +-- !query 17 output +4 +3 +2 +1 + + +-- !query 18 +SELECT count(*) FROM test_missing_target ORDER BY 1 desc +-- !query 18 schema +struct +-- !query 18 output +10 + + +-- !query 19 +SELECT c, count(*) FROM test_missing_target GROUP BY 1 ORDER BY 1 +-- !query 19 schema +struct +-- !query 19 output +ABAB 2 +BBBB 2 +CCCC 2 +XXXX 1 +bbbb 1 +cccc 2 + + +-- !query 20 +SELECT c, count(*) FROM test_missing_target GROUP BY 3 +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +GROUP BY position 3 is not in select list (valid range is [1, 2]); line 1 pos 53 + + +-- !query 21 +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b ORDER BY b +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 22 +SELECT a, a FROM test_missing_target + ORDER BY a +-- !query 22 schema +struct +-- !query 22 output +0 0 +1 1 +2 2 +3 3 +4 4 +5 5 +6 6 +7 7 +8 8 +9 9 + + +-- !query 23 +SELECT a/2, a/2 FROM test_missing_target + ORDER BY a/2 +-- !query 23 schema +struct<(a div 2):int,(a div 2):int> +-- !query 23 output +0 0 +0 0 +1 1 +1 1 +2 2 +2 2 +3 3 +3 3 +4 4 +4 4 + + +-- !query 24 +SELECT a/2, a/2 FROM test_missing_target + GROUP BY a/2 ORDER BY a/2 +-- !query 24 schema +struct<(a div 2):int,(a div 2):int> +-- !query 24 output +0 0 +1 1 +2 2 +3 3 +4 4 + + +-- !query 25 +SELECT x.b, count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query 25 schema +struct +-- !query 25 output +1 1 +2 2 +3 3 +4 4 + + +-- !query 26 +SELECT count(*) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b ORDER BY x.b +-- !query 26 schema +struct +-- !query 26 output +1 +2 +3 +4 + + +-- !query 27 +SELECT a%2, count(b) FROM test_missing_target +GROUP BY test_missing_target.a%2 +ORDER BY test_missing_target.a%2 +-- !query 27 schema +struct<(a % 2):int,count(b):bigint> +-- !query 27 output +0 5 +1 5 + + +-- !query 28 +SELECT count(c) FROM test_missing_target +GROUP BY lower(test_missing_target.c) +ORDER BY lower(test_missing_target.c) +-- !query 28 schema +struct +-- !query 28 output +2 +3 +4 +1 + + +-- !query 29 +SELECT count(a) FROM test_missing_target GROUP BY a ORDER BY b +-- !query 29 schema +struct<> +-- !query 29 output +org.apache.spark.sql.AnalysisException +cannot resolve '`b`' given input columns: [count(a)]; line 1 pos 61 + + +-- !query 30 +SELECT count(b) FROM test_missing_target GROUP BY b/2 ORDER BY b/2 +-- !query 30 schema +struct +-- !query 30 output +1 +5 +4 + + +-- !query 31 +SELECT lower(test_missing_target.c), count(c) + FROM test_missing_target GROUP BY lower(c) ORDER BY lower(c) +-- !query 31 schema +struct +-- !query 31 output +abab 2 +bbbb 3 +cccc 4 +xxxx 1 + + +-- !query 32 +SELECT a FROM test_missing_target ORDER BY upper(d) +-- !query 32 schema +struct +-- !query 32 output +0 +1 +2 +3 +4 +5 +6 +7 +8 +9 + + +-- !query 33 +SELECT count(b) FROM test_missing_target + GROUP BY (b + 1) / 2 ORDER BY (b + 1) / 2 desc +-- !query 33 schema +struct +-- !query 33 output +7 +3 + + +-- !query 34 +SELECT count(x.a) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY b/2 ORDER BY b/2 +-- !query 34 schema +struct<> +-- !query 34 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 3 pos 10 + + +-- !query 35 +SELECT x.b/2, count(x.b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 ORDER BY x.b/2 +-- !query 35 schema +struct<(b div 2):int,count(b):bigint> +-- !query 35 output +0 1 +1 5 +2 4 + + +-- !query 36 +SELECT count(b) FROM test_missing_target x, test_missing_target y + WHERE x.a = y.a + GROUP BY x.b/2 +-- !query 36 schema +struct<> +-- !query 36 output +org.apache.spark.sql.AnalysisException +Reference 'b' is ambiguous, could be: x.b, y.b.; line 1 pos 13 + + +-- !query 37 +DROP TABLE test_missing_target +-- !query 37 schema +struct<> +-- !query 37 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out new file mode 100644 index 0000000000000..200fecce17bc0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/timestamp.sql.out @@ -0,0 +1,130 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 14 + + +-- !query 0 +CREATE TABLE TIMESTAMP_TBL (d1 timestamp) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02') +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-01-02 03:04:05') +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO TIMESTAMP_TBL VALUES ('1997-02-10 17:32:01-08') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO TIMESTAMP_TBL VALUES ('2001-09-22T18:19:20') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT '' AS `64`, d1 FROM TIMESTAMP_TBL +-- !query 5 schema +struct<64:string,d1:timestamp> +-- !query 5 output +1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 6 +SELECT '' AS `48`, d1 FROM TIMESTAMP_TBL + WHERE d1 > timestamp '1997-01-02' +-- !query 6 schema +struct<48:string,d1:timestamp> +-- !query 6 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 7 +SELECT '' AS `15`, d1 FROM TIMESTAMP_TBL + WHERE d1 < timestamp '1997-01-02' +-- !query 7 schema +struct<15:string,d1:timestamp> +-- !query 7 output + + + +-- !query 8 +SELECT '' AS one, d1 FROM TIMESTAMP_TBL + WHERE d1 = timestamp '1997-01-02' +-- !query 8 schema +struct +-- !query 8 output +1997-01-02 00:00:00 + + +-- !query 9 +SELECT '' AS `63`, d1 FROM TIMESTAMP_TBL + WHERE d1 != timestamp '1997-01-02' +-- !query 9 schema +struct<63:string,d1:timestamp> +-- !query 9 output +1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 10 +SELECT '' AS `16`, d1 FROM TIMESTAMP_TBL + WHERE d1 <= timestamp '1997-01-02' +-- !query 10 schema +struct<16:string,d1:timestamp> +-- !query 10 output +1997-01-02 00:00:00 + + +-- !query 11 +SELECT '' AS `49`, d1 FROM TIMESTAMP_TBL + WHERE d1 >= timestamp '1997-01-02' +-- !query 11 schema +struct<49:string,d1:timestamp> +-- !query 11 output +1997-01-02 00:00:00 + 1997-01-02 03:04:05 + 1997-02-10 17:32:01 + 2001-09-22 18:19:20 + + +-- !query 12 +SELECT '' AS date_trunc_week, date_trunc( 'week', timestamp '2004-02-29 15:44:17.71393' ) AS week_trunc +-- !query 12 schema +struct +-- !query 12 output +2004-02-23 00:00:00 + + +-- !query 13 +DROP TABLE TIMESTAMP_TBL +-- !query 13 schema +struct<> +-- !query 13 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out new file mode 100644 index 0000000000000..366b65f3659cd --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/pgSQL/with.sql.out @@ -0,0 +1,471 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 51 + + +-- !query 0 +WITH q1(x,y) AS (SELECT 1,2) +SELECT * FROM q1, q1 AS q2 +-- !query 0 schema +struct +-- !query 0 output +1 2 1 2 + + +-- !query 1 +SELECT count(*) FROM ( + WITH q1(x) AS (SELECT rand() FROM (SELECT EXPLODE(SEQUENCE(1, 5)))) + SELECT * FROM q1 + UNION + SELECT * FROM q1 +) ss +-- !query 1 schema +struct +-- !query 1 output +10 + + +-- !query 2 +CREATE TABLE department ( + id INTEGER, -- department ID + parent_department INTEGER, -- upper department ID + name string -- department name +) USING parquet +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO department VALUES (0, NULL, 'ROOT') +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO department VALUES (1, 0, 'A') +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO department VALUES (2, 1, 'B') +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO department VALUES (3, 2, 'C') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO department VALUES (4, 2, 'D') +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO department VALUES (5, 0, 'E') +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO department VALUES (6, 4, 'F') +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO department VALUES (7, 5, 'G') +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +CREATE TABLE tree( + id INTEGER, + parent_id INTEGER +) USING parquet +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +INSERT INTO tree +VALUES (1, NULL), (2, 1), (3,1), (4,2), (5,2), (6,2), (7,3), (8,3), + (9,4), (10,4), (11,7), (12,7), (13,7), (14, 9), (15,11), (16,11) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +create table graph( f int, t int, label string ) USING parquet +-- !query 13 schema +struct<> +-- !query 13 output + + + +-- !query 14 +insert into graph values + (1, 2, 'arc 1 -> 2'), + (1, 3, 'arc 1 -> 3'), + (2, 3, 'arc 2 -> 3'), + (1, 4, 'arc 1 -> 4'), + (4, 5, 'arc 4 -> 5'), + (5, 1, 'arc 5 -> 1') +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 16 schema +struct<> +-- !query 16 output + + + +-- !query 17 +DROP TABLE y +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +CREATE TABLE y (a INTEGER) USING parquet +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 10)) +-- !query 19 schema +struct<> +-- !query 19 output + + + +-- !query 20 +with cte(foo) as ( select 42 ) select * from ((select foo from cte)) q +-- !query 20 schema +struct +-- !query 20 output +42 + + +-- !query 21 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM innermost + UNION SELECT 3) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 21 schema +struct +-- !query 21 output +1 +2 +3 + + +-- !query 22 +WITH outermost(x) AS ( + SELECT 1 + UNION (WITH innermost as (SELECT 2) + SELECT * FROM outermost -- fail + UNION SELECT * FROM innermost) +) +SELECT * FROM outermost ORDER BY 1 +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Table or view not found: outermost; line 4 pos 23 + + +-- !query 23 +CREATE TABLE withz USING parquet AS SELECT i AS k, CAST(i || ' v' AS string) v FROM (SELECT EXPLODE(SEQUENCE(1, 16, 3)) i) +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +SELECT * FROM withz ORDER BY k +-- !query 24 schema +struct +-- !query 24 output +1 1 v +4 4 v +7 7 v +10 10 v +13 13 v +16 16 v + + +-- !query 25 +DROP TABLE withz +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +TRUNCATE TABLE y +-- !query 26 schema +struct<> +-- !query 26 output + + + +-- !query 27 +INSERT INTO y SELECT EXPLODE(SEQUENCE(1, 3)) +-- !query 27 schema +struct<> +-- !query 27 output + + + +-- !query 28 +CREATE TABLE yy (a INTEGER) USING parquet +-- !query 28 schema +struct<> +-- !query 28 output + + + +-- !query 29 +SELECT * FROM y +-- !query 29 schema +struct +-- !query 29 output +1 +2 +3 + + +-- !query 30 +SELECT * FROM yy +-- !query 30 schema +struct +-- !query 30 output + + + +-- !query 31 +SELECT * FROM y +-- !query 31 schema +struct +-- !query 31 output +1 +2 +3 + + +-- !query 32 +SELECT * FROM yy +-- !query 32 schema +struct +-- !query 32 output + + + +-- !query 33 +CREATE TABLE parent ( id int, val string ) USING parquet +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +INSERT INTO parent VALUES ( 1, 'p1' ) +-- !query 34 schema +struct<> +-- !query 34 output + + + +-- !query 35 +SELECT * FROM parent +-- !query 35 schema +struct +-- !query 35 output +1 p1 + + +-- !query 36 +SELECT * FROM parent +-- !query 36 schema +struct +-- !query 36 output +1 p1 + + +-- !query 37 +create table foo (with baz) +-- !query 37 schema +struct<> +-- !query 37 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 1, pos 18) + +== SQL == +create table foo (with baz) +------------------^^^ + + +-- !query 38 +-- fail, WITH is a reserved word +create table foo (with ordinality) +-- !query 38 schema +struct<> +-- !query 38 output +org.apache.spark.sql.catalyst.parser.ParseException + +no viable alternative at input 'with'(line 2, pos 18) + +== SQL == +-- fail, WITH is a reserved word +create table foo (with ordinality) +------------------^^^ + + +-- !query 39 +-- fail, WITH is a reserved word +with ordinality as (select 1 as x) select * from ordinality +-- !query 39 schema +struct +-- !query 39 output +1 + + +-- !query 40 +WITH test AS (SELECT 42) INSERT INTO test VALUES (1) +-- !query 40 schema +struct<> +-- !query 40 output +org.apache.spark.sql.AnalysisException +Table not found: test; + + +-- !query 41 +create table test (i int) USING parquet +-- !query 41 schema +struct<> +-- !query 41 output + + + +-- !query 42 +with test as (select 42) insert into test select * from test +-- !query 42 schema +struct<> +-- !query 42 output + + + +-- !query 43 +select * from test +-- !query 43 schema +struct +-- !query 43 output +42 + + +-- !query 44 +drop table test +-- !query 44 schema +struct<> +-- !query 44 output + + + +-- !query 45 +DROP TABLE department +-- !query 45 schema +struct<> +-- !query 45 output + + + +-- !query 46 +DROP TABLE tree +-- !query 46 schema +struct<> +-- !query 46 output + + + +-- !query 47 +DROP TABLE graph +-- !query 47 schema +struct<> +-- !query 47 output + + + +-- !query 48 +DROP TABLE y +-- !query 48 schema +struct<> +-- !query 48 output + + + +-- !query 49 +DROP TABLE yy +-- !query 49 schema +struct<> +-- !query 49 output + + + +-- !query 50 +DROP TABLE parent +-- !query 50 schema +struct<> +-- !query 50 output + diff --git a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out index 595ce1f8efcd2..9a8f783da4369 100644 --- a/sql/core/src/test/resources/sql-tests/results/pivot.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/pivot.sql.out @@ -326,7 +326,7 @@ PIVOT ( struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve '`s`' given input columns: [coursesales.course, coursesales.year, coursesales.earnings]; line 4 pos 15 +cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 -- !query 22 diff --git a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out index cf828c69af62a..d38cab8fa7862 100644 --- a/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/predicate-functions.sql.out @@ -85,7 +85,7 @@ false -- !query 10 select to_date('2009-07-30 04:17:52') > '2009-07-30 04:17:52' -- !query 10 schema -struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) > 2009-07-30 04:17:52):boolean> +struct<(to_date('2009-07-30 04:17:52') > CAST(2009-07-30 04:17:52 AS DATE)):boolean> -- !query 10 output false @@ -141,9 +141,9 @@ true -- !query 17 select to_date('2009-07-30 04:17:52') >= '2009-07-30 04:17:52' -- !query 17 schema -struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) >= 2009-07-30 04:17:52):boolean> +struct<(to_date('2009-07-30 04:17:52') >= CAST(2009-07-30 04:17:52 AS DATE)):boolean> -- !query 17 output -false +true -- !query 18 @@ -197,9 +197,9 @@ false -- !query 24 select to_date('2009-07-30 04:17:52') < '2009-07-30 04:17:52' -- !query 24 schema -struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) < 2009-07-30 04:17:52):boolean> +struct<(to_date('2009-07-30 04:17:52') < CAST(2009-07-30 04:17:52 AS DATE)):boolean> -- !query 24 output -true +false -- !query 25 @@ -253,7 +253,7 @@ true -- !query 31 select to_date('2009-07-30 04:17:52') <= '2009-07-30 04:17:52' -- !query 31 schema -struct<(CAST(to_date('2009-07-30 04:17:52') AS STRING) <= 2009-07-30 04:17:52):boolean> +struct<(to_date('2009-07-30 04:17:52') <= CAST(2009-07-30 04:17:52 AS DATE)):boolean> -- !query 31 output true diff --git a/sql/core/src/test/resources/sql-tests/results/random.sql.out b/sql/core/src/test/resources/sql-tests/results/random.sql.out index bca67320fe7bb..acd0609aabb16 100644 --- a/sql/core/src/test/resources/sql-tests/results/random.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/random.sql.out @@ -7,7 +7,7 @@ SELECT rand(0) -- !query 0 schema struct -- !query 0 output -0.8446490682263027 +0.7604953758285915 -- !query 1 @@ -15,7 +15,7 @@ SELECT rand(cast(3 / 7 AS int)) -- !query 1 schema struct -- !query 1 output -0.8446490682263027 +0.7604953758285915 -- !query 2 @@ -23,7 +23,7 @@ SELECT rand(NULL) -- !query 2 schema struct -- !query 2 output -0.8446490682263027 +0.7604953758285915 -- !query 3 @@ -31,7 +31,7 @@ SELECT rand(cast(NULL AS int)) -- !query 3 schema struct -- !query 3 output -0.8446490682263027 +0.7604953758285915 -- !query 4 @@ -48,7 +48,7 @@ SELECT randn(0L) -- !query 5 schema struct -- !query 5 output -1.1164209726833079 +1.6034991609278433 -- !query 6 @@ -56,7 +56,7 @@ SELECT randn(cast(3 / 7 AS long)) -- !query 6 schema struct -- !query 6 output -1.1164209726833079 +1.6034991609278433 -- !query 7 @@ -64,7 +64,7 @@ SELECT randn(NULL) -- !query 7 schema struct -- !query 7 output -1.1164209726833079 +1.6034991609278433 -- !query 8 @@ -72,7 +72,7 @@ SELECT randn(cast(NULL AS long)) -- !query 8 schema struct -- !query 8 output -1.1164209726833079 +1.6034991609278433 -- !query 9 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 new file mode 100644 index 0000000000000..1faf16cc30509 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/show-create-table.sql.out @@ -0,0 +1,222 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 24 + + +-- !query 0 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SHOW CREATE TABLE tbl +-- !query 1 schema +struct +-- !query 1 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet + + +-- !query 2 +DROP TABLE tbl +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('a' 1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +SHOW CREATE TABLE tbl +-- !query 4 schema +struct +-- !query 4 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +OPTIONS ( + `a` '1' +) + + +-- !query 5 +DROP TABLE tbl +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +OPTIONS ('path' '/path/to/table') +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +SHOW CREATE TABLE tbl +-- !query 7 schema +struct +-- !query 7 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +LOCATION 'file:/path/to/table' + + +-- !query 8 +DROP TABLE tbl +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +LOCATION '/path/to/table' +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +SHOW CREATE TABLE tbl +-- !query 10 schema +struct +-- !query 10 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +LOCATION 'file:/path/to/table' + + +-- !query 11 +DROP TABLE tbl +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +PARTITIONED BY (a) +-- !query 12 schema +struct<> +-- !query 12 output + + + +-- !query 13 +SHOW CREATE TABLE tbl +-- !query 13 schema +struct +-- !query 13 output +CREATE TABLE `tbl` (`b` STRING, `c` INT, `a` INT) +USING parquet +PARTITIONED BY (a) + + +-- !query 14 +DROP TABLE tbl +-- !query 14 schema +struct<> +-- !query 14 output + + + +-- !query 15 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS +-- !query 15 schema +struct<> +-- !query 15 output + + + +-- !query 16 +SHOW CREATE TABLE tbl +-- !query 16 schema +struct +-- !query 16 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +CLUSTERED BY (a) +SORTED BY (b) +INTO 2 BUCKETS + + +-- !query 17 +DROP TABLE tbl +-- !query 17 schema +struct<> +-- !query 17 output + + + +-- !query 18 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +COMMENT 'This is a comment' +-- !query 18 schema +struct<> +-- !query 18 output + + + +-- !query 19 +SHOW CREATE TABLE tbl +-- !query 19 schema +struct +-- !query 19 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +COMMENT 'This is a comment' + + +-- !query 20 +DROP TABLE tbl +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +CREATE TABLE tbl (a INT, b STRING, c INT) USING parquet +TBLPROPERTIES ('a' = '1') +-- !query 21 schema +struct<> +-- !query 21 output + + + +-- !query 22 +SHOW CREATE TABLE tbl +-- !query 22 schema +struct +-- !query 22 output +CREATE TABLE `tbl` (`a` INT, `b` STRING, `c` INT) +USING parquet +TBLPROPERTIES ( + 'a' = '1' +) + + +-- !query 23 +DROP TABLE tbl +-- !query 23 schema +struct<> +-- !query 23 output + diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out index abeb7e18f031e..8efeae3a1bd09 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 @@ -63,11 +63,14 @@ SHOW TABLES -- !query 7 schema struct -- !query 7 output +aggtest arraydata mapdata +onek show_t1 show_t2 show_t3 +tenk1 testdata @@ -76,11 +79,14 @@ SHOW TABLES IN showdb -- !query 8 schema struct -- !query 8 output +aggtest arraydata mapdata +onek show_t1 show_t2 show_t3 +tenk1 testdata diff --git a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out index e035505f15d28..69a8e958000db 100644 --- a/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/sql-compatibility-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 15 +-- Number of queries: 14 -- !query 0 @@ -67,74 +67,49 @@ struct -- !query 8 -explain extended -select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') -from range(2) --- !query 8 schema -struct --- !query 8 output -== Parsed Logical Plan == -'Project [unresolvedalias('ifnull('id, x), None), unresolvedalias('nullif('id, x), None), unresolvedalias('nvl('id, x), None), unresolvedalias('nvl2('id, x, y), None)] -+- 'UnresolvedTableValuedFunction range, [2] - -== Analyzed Logical Plan == -ifnull(`id`, 'x'): string, nullif(`id`, 'x'): bigint, nvl(`id`, 'x'): string, nvl2(`id`, 'x', 'y'): string -Project [ifnull(id#xL, x) AS ifnull(`id`, 'x')#x, nullif(id#xL, x) AS nullif(`id`, 'x')#xL, nvl(id#xL, x) AS nvl(`id`, 'x')#x, nvl2(id#xL, x, y) AS nvl2(`id`, 'x', 'y')#x] -+- Range (0, 2, step=1, splits=None) - -== Optimized Logical Plan == -Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] -+- Range (0, 2, step=1, splits=None) - -== Physical Plan == -*Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, x AS nvl2(`id`, 'x', 'y')#x] -+- *Range (0, 2, step=1, splits=2) - - --- !query 9 SELECT boolean(1), tinyint(1), smallint(1), int(1), bigint(1) --- !query 9 schema +-- !query 8 schema struct --- !query 9 output +-- !query 8 output true 1 1 1 1 --- !query 10 +-- !query 9 SELECT float(1), double(1), decimal(1) --- !query 10 schema +-- !query 9 schema struct --- !query 10 output +-- !query 9 output 1.0 1.0 1 --- !query 11 +-- !query 10 SELECT date("2014-04-04"), timestamp(date("2014-04-04")) --- !query 11 schema +-- !query 10 schema struct --- !query 11 output +-- !query 10 output 2014-04-04 2014-04-04 00:00:00 --- !query 12 +-- !query 11 SELECT string(1, 2) --- !query 12 schema +-- !query 11 schema struct<> --- !query 12 output +-- !query 11 output org.apache.spark.sql.AnalysisException Function string accepts only one argument; line 1 pos 7 --- !query 13 +-- !query 12 CREATE TEMPORARY VIEW tempView1 AS VALUES (1, NAMED_STRUCT('col1', 'gamma', 'col2', 'delta')) AS T(id, st) --- !query 13 schema +-- !query 12 schema struct<> --- !query 13 output +-- !query 12 output --- !query 14 +-- !query 13 SELECT nvl(st.col1, "value"), count(*) FROM from tempView1 GROUP BY nvl(st.col1, "value") --- !query 14 schema +-- !query 13 schema struct --- !query 14 output +-- !query 13 output gamma 1 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 e8f2e0a81455a..87c3e04017643 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 @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 17 +-- Number of queries: 33 -- !query 0 @@ -29,151 +29,240 @@ abc -- !query 3 -EXPLAIN EXTENDED SELECT (col1 || col2 || col3 || col4) col -FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)) +select replace('abc', 'b', '123') -- !query 3 schema -struct +struct -- !query 3 output -== Parsed Logical Plan == -'Project [concat(concat(concat('col1, 'col2), 'col3), 'col4) AS col#x] -+- 'SubqueryAlias `__auto_generated_subquery_name` - +- 'Project ['id AS col1#x, 'id AS col2#x, 'id AS col3#x, 'id AS col4#x] - +- 'UnresolvedTableValuedFunction range, [10] - -== Analyzed Logical Plan == -col: string -Project [concat(concat(concat(cast(col1#xL as string), cast(col2#xL as string)), cast(col3#xL as string)), cast(col4#xL as string)) AS col#x] -+- SubqueryAlias `__auto_generated_subquery_name` - +- Project [id#xL AS col1#xL, id#xL AS col2#xL, id#xL AS col3#xL, id#xL AS col4#xL] - +- Range (0, 10, step=1, splits=None) - -== Optimized Logical Plan == -Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] -+- Range (0, 10, step=1, splits=None) - -== Physical Plan == -*Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) +a123c -- !query 4 -select replace('abc', 'b', '123') +select replace('abc', 'b') -- !query 4 schema -struct +struct -- !query 4 output -a123c +ac -- !query 5 -select replace('abc', 'b') +select length(uuid()), (uuid() <> uuid()) -- !query 5 schema -struct +struct -- !query 5 output -ac +36 true -- !query 6 -select length(uuid()), (uuid() <> uuid()) +select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) -- !query 6 schema -struct +struct -- !query 6 output -36 true +4 NULL NULL -- !query 7 -select position('bar' in 'foobarbar'), position(null, 'foobarbar'), position('aaads', null) +select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) -- !query 7 schema -struct +struct -- !query 7 output -4 NULL NULL +ab abcd ab NULL -- !query 8 -select left("abcd", 2), left("abcd", 5), left("abcd", '2'), left("abcd", null) +select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') -- !query 8 schema -struct +struct -- !query 8 output -ab abcd ab NULL +NULL NULL -- !query 9 -select left(null, -2), left("abcd", -2), left("abcd", 0), left("abcd", 'a') +select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) -- !query 9 schema -struct +struct -- !query 9 output -NULL NULL +cd abcd cd NULL -- !query 10 -select right("abcd", 2), right("abcd", 5), right("abcd", '2'), right("abcd", null) +select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') -- !query 10 schema -struct +struct -- !query 10 output -cd abcd cd NULL +NULL NULL -- !query 11 -select right(null, -2), right("abcd", -2), right("abcd", 0), right("abcd", 'a') +SELECT split('aa1cc2ee3', '[1-9]+') -- !query 11 schema -struct +struct> -- !query 11 output -NULL NULL +["aa","cc","ee",""] -- !query 12 -set spark.sql.function.concatBinaryAsString=false +SELECT split('aa1cc2ee3', '[1-9]+', 2) -- !query 12 schema -struct +struct> -- !query 12 output -spark.sql.function.concatBinaryAsString false +["aa","cc2ee3"] -- !query 13 -EXPLAIN SELECT ((col1 || col2) || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - string(id + 1) col2, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -) +SELECT substr('Spark SQL', 5) -- !query 13 schema -struct +struct -- !query 13 output -== Physical Plan == -*Project [concat(cast(id#xL as string), cast((id#xL + 1) as string), cast(encode(cast((id#xL + 2) as string), utf-8) as string), cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) +k SQL -- !query 14 -EXPLAIN SELECT (col1 || (col3 || col4)) col -FROM ( - SELECT - string(id) col1, - encode(string(id + 2), 'utf-8') col3, - encode(string(id + 3), 'utf-8') col4 - FROM range(10) -) +SELECT substr('Spark SQL', -3) -- !query 14 schema -struct +struct -- !query 14 output -== Physical Plan == -*Project [concat(cast(id#xL as string), cast(encode(cast((id#xL + 2) as string), utf-8) as string), cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x] -+- *Range (0, 10, step=1, splits=2) +SQL -- !query 15 -SELECT split('aa1cc2ee3', '[1-9]+') +SELECT substr('Spark SQL', 5, 1) -- !query 15 schema -struct> +struct -- !query 15 output -["aa","cc","ee",""] +k -- !query 16 -SELECT split('aa1cc2ee3', '[1-9]+', 2) +SELECT substr('Spark SQL' from 5) -- !query 16 schema -struct> +struct -- !query 16 output -["aa","cc2ee3"] +k SQL + + +-- !query 17 +SELECT substr('Spark SQL' from -3) +-- !query 17 schema +struct +-- !query 17 output +SQL + + +-- !query 18 +SELECT substr('Spark SQL' from 5 for 1) +-- !query 18 schema +struct +-- !query 18 output +k + + +-- !query 19 +SELECT substring('Spark SQL', 5) +-- !query 19 schema +struct +-- !query 19 output +k SQL + + +-- !query 20 +SELECT substring('Spark SQL', -3) +-- !query 20 schema +struct +-- !query 20 output +SQL + + +-- !query 21 +SELECT substring('Spark SQL', 5, 1) +-- !query 21 schema +struct +-- !query 21 output +k + + +-- !query 22 +SELECT substring('Spark SQL' from 5) +-- !query 22 schema +struct +-- !query 22 output +k SQL + + +-- !query 23 +SELECT substring('Spark SQL' from -3) +-- !query 23 schema +struct +-- !query 23 output +SQL + + +-- !query 24 +SELECT substring('Spark SQL' from 5 for 1) +-- !query 24 schema +struct +-- !query 24 output +k + + +-- !query 25 +SELECT trim('yxTomxx', 'xyz'), trim(BOTH 'xyz' FROM 'yxTomxx'), trim('xyz' FROM 'yxTomxx') +-- !query 25 schema +struct +-- !query 25 output +Tom Tom Tom + + +-- !query 26 +SELECT trim('xxxbarxxx', 'x'), trim(BOTH 'x' FROM 'xxxbarxxx'), trim('x' FROM 'xxxbarxxx') +-- !query 26 schema +struct +-- !query 26 output +bar bar bar + + +-- !query 27 +SELECT ltrim('zzzytest', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytest') +-- !query 27 schema +struct +-- !query 27 output +test test + + +-- !query 28 +SELECT ltrim('zzzytestxyz', 'xyz'), trim(LEADING 'xyz' FROM 'zzzytestxyz') +-- !query 28 schema +struct +-- !query 28 output +testxyz testxyz + + +-- !query 29 +SELECT ltrim('xyxXxyLAST WORD', 'xy'), trim(LEADING 'xy' FROM 'xyxXxyLAST WORD') +-- !query 29 schema +struct +-- !query 29 output +XxyLAST WORD XxyLAST WORD + + +-- !query 30 +SELECT rtrim('testxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'testxxzx') +-- !query 30 schema +struct +-- !query 30 output +test test + + +-- !query 31 +SELECT rtrim('xyztestxxzx', 'xyz'), trim(TRAILING 'xyz' FROM 'xyztestxxzx') +-- !query 31 schema +struct +-- !query 31 output +xyztest xyztest + + +-- !query 32 +SELECT rtrim('TURNERyxXxy', 'xy'), trim(TRAILING 'xy' FROM 'TURNERyxXxy') +-- !query 32 schema +struct +-- !query 32 output +TURNERyxX TURNERyxX 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 e49978ddb1ce2..7b47a6139f60a 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 @@ -70,9 +70,10 @@ Resolved attribute(s) t2b#x missing from min(t2a)#x,t2c#x in operator !Filter t2 SELECT t1a FROM t1 GROUP BY 1 -HAVING EXISTS (SELECT 1 +HAVING EXISTS (SELECT t2a FROM t2 - WHERE t2a < min(t1a + t2a)) + GROUP BY 1 + HAVING t2a < min(t1a + t2a)) -- !query 5 schema struct<> -- !query 5 output diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out index 94af9181225d6..fdbea0ee90720 100644 --- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 10 +-- Number of queries: 8 -- !query 0 @@ -99,42 +99,3 @@ struct -- !query 7 output 0 1 - - --- !query 8 -EXPLAIN select * from RaNgE(2) --- !query 8 schema -struct --- !query 8 output -== Physical Plan == -*Range (0, 2, step=1, splits=2) - - --- !query 9 -EXPLAIN EXTENDED SELECT * FROM range(3) CROSS JOIN range(3) --- !query 9 schema -struct --- !query 9 output -== Parsed Logical Plan == -'Project [*] -+- 'Join Cross - :- 'UnresolvedTableValuedFunction range, [3] - +- 'UnresolvedTableValuedFunction range, [3] - -== Analyzed Logical Plan == -id: bigint, id: bigint -Project [id#xL, id#xL] -+- Join Cross - :- Range (0, 3, step=1, splits=None) - +- Range (0, 3, step=1, splits=None) - -== Optimized Logical Plan == -Join Cross -:- Range (0, 3, step=1, splits=None) -+- Range (0, 3, step=1, splits=None) - -== Physical Plan == -BroadcastNestedLoopJoin BuildRight, Cross -:- *Range (0, 3, step=1, splits=2) -+- BroadcastExchange IdentityBroadcastMode - +- *Range (0, 3, step=1, splits=2) diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out index b23a62dacef7c..c3d5fad0870bc 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/arrayJoin.sql.out @@ -77,7 +77,7 @@ struct -- !query 9 SELECT array_join(array(timestamp '2016-11-15 20:54:00.000', timestamp '2016-11-12 20:54:00.000'), ', ') -- !query 9 schema -struct +struct -- !query 9 output 2016-11-15 20:54:00, 2016-11-12 20:54:00 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out index 2914d6015ea88..55caab8528fa9 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/binaryComparison.sql.out @@ -13,217 +13,193 @@ struct<> -- !query 1 SELECT cast(1 as binary) = '1' FROM t -- !query 1 schema -struct<> +struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> -- !query 1 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +false -- !query 2 SELECT cast(1 as binary) > '2' FROM t -- !query 2 schema -struct<> +struct<(CAST(1 AS BINARY) > CAST(2 AS BINARY)):boolean> -- !query 2 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +false -- !query 3 SELECT cast(1 as binary) >= '2' FROM t -- !query 3 schema -struct<> +struct<(CAST(1 AS BINARY) >= CAST(2 AS BINARY)):boolean> -- !query 3 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +false -- !query 4 SELECT cast(1 as binary) < '2' FROM t -- !query 4 schema -struct<> +struct<(CAST(1 AS BINARY) < CAST(2 AS BINARY)):boolean> -- !query 4 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +true -- !query 5 SELECT cast(1 as binary) <= '2' FROM t -- !query 5 schema -struct<> +struct<(CAST(1 AS BINARY) <= CAST(2 AS BINARY)):boolean> -- !query 5 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +true -- !query 6 SELECT cast(1 as binary) <> '2' FROM t -- !query 6 schema -struct<> +struct<(NOT (CAST(1 AS BINARY) = CAST(2 AS BINARY))):boolean> -- !query 6 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +true -- !query 7 SELECT cast(1 as binary) = cast(null as string) FROM t -- !query 7 schema -struct<> +struct<(CAST(1 AS BINARY) = CAST(CAST(NULL AS STRING) AS BINARY)):boolean> -- !query 7 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 8 SELECT cast(1 as binary) > cast(null as string) FROM t -- !query 8 schema -struct<> +struct<(CAST(1 AS BINARY) > CAST(CAST(NULL AS STRING) AS BINARY)):boolean> -- !query 8 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 9 SELECT cast(1 as binary) >= cast(null as string) FROM t -- !query 9 schema -struct<> +struct<(CAST(1 AS BINARY) >= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> -- !query 9 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 10 SELECT cast(1 as binary) < cast(null as string) FROM t -- !query 10 schema -struct<> +struct<(CAST(1 AS BINARY) < CAST(CAST(NULL AS STRING) AS BINARY)):boolean> -- !query 10 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 11 SELECT cast(1 as binary) <= cast(null as string) FROM t -- !query 11 schema -struct<> +struct<(CAST(1 AS BINARY) <= CAST(CAST(NULL AS STRING) AS BINARY)):boolean> -- !query 11 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 12 SELECT cast(1 as binary) <> cast(null as string) FROM t -- !query 12 schema -struct<> +struct<(NOT (CAST(1 AS BINARY) = CAST(CAST(NULL AS STRING) AS BINARY))):boolean> -- !query 12 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 7 +NULL -- !query 13 SELECT '1' = cast(1 as binary) FROM t -- !query 13 schema -struct<> +struct<(CAST(1 AS BINARY) = CAST(1 AS BINARY)):boolean> -- !query 13 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 13 +false -- !query 14 SELECT '2' > cast(1 as binary) FROM t -- !query 14 schema -struct<> +struct<(CAST(2 AS BINARY) > CAST(1 AS BINARY)):boolean> -- !query 14 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 13 +true -- !query 15 SELECT '2' >= cast(1 as binary) FROM t -- !query 15 schema -struct<> +struct<(CAST(2 AS BINARY) >= CAST(1 AS BINARY)):boolean> -- !query 15 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 14 +true -- !query 16 SELECT '2' < cast(1 as binary) FROM t -- !query 16 schema -struct<> +struct<(CAST(2 AS BINARY) < CAST(1 AS BINARY)):boolean> -- !query 16 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 13 +false -- !query 17 SELECT '2' <= cast(1 as binary) FROM t -- !query 17 schema -struct<> +struct<(CAST(2 AS BINARY) <= CAST(1 AS BINARY)):boolean> -- !query 17 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 14 +false -- !query 18 SELECT '2' <> cast(1 as binary) FROM t -- !query 18 schema -struct<> +struct<(NOT (CAST(2 AS BINARY) = CAST(1 AS BINARY))):boolean> -- !query 18 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 14 +true -- !query 19 SELECT cast(null as string) = cast(1 as binary) FROM t -- !query 19 schema -struct<> +struct<(CAST(CAST(NULL AS STRING) AS BINARY) = CAST(1 AS BINARY)):boolean> -- !query 19 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 30 +NULL -- !query 20 SELECT cast(null as string) > cast(1 as binary) FROM t -- !query 20 schema -struct<> +struct<(CAST(CAST(NULL AS STRING) AS BINARY) > CAST(1 AS BINARY)):boolean> -- !query 20 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 30 +NULL -- !query 21 SELECT cast(null as string) >= cast(1 as binary) FROM t -- !query 21 schema -struct<> +struct<(CAST(CAST(NULL AS STRING) AS BINARY) >= CAST(1 AS BINARY)):boolean> -- !query 21 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 31 +NULL -- !query 22 SELECT cast(null as string) < cast(1 as binary) FROM t -- !query 22 schema -struct<> +struct<(CAST(CAST(NULL AS STRING) AS BINARY) < CAST(1 AS BINARY)):boolean> -- !query 22 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 30 +NULL -- !query 23 SELECT cast(null as string) <= cast(1 as binary) FROM t -- !query 23 schema -struct<> +struct<(CAST(CAST(NULL AS STRING) AS BINARY) <= CAST(1 AS BINARY)):boolean> -- !query 23 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 31 +NULL -- !query 24 SELECT cast(null as string) <> cast(1 as binary) FROM t -- !query 24 schema -struct<> +struct<(NOT (CAST(CAST(NULL AS STRING) AS BINARY) = CAST(1 AS BINARY))):boolean> -- !query 24 output -org.apache.spark.sql.AnalysisException -cannot resolve 'CAST(1 AS BINARY)' due to data type mismatch: cannot cast int to binary; line 1 pos 31 +NULL -- !query 25 @@ -1573,7 +1549,7 @@ NULL -- !query 193 SELECT '1996-09-09' = date('1996-09-09') FROM t -- !query 193 schema -struct<(1996-09-09 = CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(1996-09-09 AS DATE) = CAST(1996-09-09 AS DATE)):boolean> -- !query 193 output true @@ -1581,7 +1557,7 @@ true -- !query 194 SELECT '1996-9-10' > date('1996-09-09') FROM t -- !query 194 schema -struct<(1996-9-10 > CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) > CAST(1996-09-09 AS DATE)):boolean> -- !query 194 output true @@ -1589,7 +1565,7 @@ true -- !query 195 SELECT '1996-9-10' >= date('1996-09-09') FROM t -- !query 195 schema -struct<(1996-9-10 >= CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> -- !query 195 output true @@ -1597,7 +1573,7 @@ true -- !query 196 SELECT '1996-9-10' < date('1996-09-09') FROM t -- !query 196 schema -struct<(1996-9-10 < CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) < CAST(1996-09-09 AS DATE)):boolean> -- !query 196 output false @@ -1605,7 +1581,7 @@ false -- !query 197 SELECT '1996-9-10' <= date('1996-09-09') FROM t -- !query 197 schema -struct<(1996-9-10 <= CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> -- !query 197 output false @@ -1613,7 +1589,7 @@ false -- !query 198 SELECT '1996-9-10' <> date('1996-09-09') FROM t -- !query 198 schema -struct<(NOT (1996-9-10 = CAST(CAST(1996-09-09 AS DATE) AS STRING))):boolean> +struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(1996-09-09 AS DATE))):boolean> -- !query 198 output true @@ -1621,7 +1597,7 @@ true -- !query 199 SELECT cast(null as string) = date('1996-09-09') FROM t -- !query 199 schema -struct<(CAST(NULL AS STRING) = CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS DATE) = CAST(1996-09-09 AS DATE)):boolean> -- !query 199 output NULL @@ -1629,7 +1605,7 @@ NULL -- !query 200 SELECT cast(null as string)> date('1996-09-09') FROM t -- !query 200 schema -struct<(CAST(NULL AS STRING) > CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS DATE) > CAST(1996-09-09 AS DATE)):boolean> -- !query 200 output NULL @@ -1637,7 +1613,7 @@ NULL -- !query 201 SELECT cast(null as string)>= date('1996-09-09') FROM t -- !query 201 schema -struct<(CAST(NULL AS STRING) >= CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> -- !query 201 output NULL @@ -1645,7 +1621,7 @@ NULL -- !query 202 SELECT cast(null as string)< date('1996-09-09') FROM t -- !query 202 schema -struct<(CAST(NULL AS STRING) < CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS DATE) < CAST(1996-09-09 AS DATE)):boolean> -- !query 202 output NULL @@ -1653,7 +1629,7 @@ NULL -- !query 203 SELECT cast(null as string)<= date('1996-09-09') FROM t -- !query 203 schema -struct<(CAST(NULL AS STRING) <= CAST(CAST(1996-09-09 AS DATE) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> -- !query 203 output NULL @@ -1661,7 +1637,7 @@ NULL -- !query 204 SELECT cast(null as string)<> date('1996-09-09') FROM t -- !query 204 schema -struct<(NOT (CAST(NULL AS STRING) = CAST(CAST(1996-09-09 AS DATE) AS STRING))):boolean> +struct<(NOT (CAST(CAST(NULL AS STRING) AS DATE) = CAST(1996-09-09 AS DATE))):boolean> -- !query 204 output NULL @@ -1669,7 +1645,7 @@ NULL -- !query 205 SELECT date('1996-09-09') = '1996-09-09' FROM t -- !query 205 schema -struct<(CAST(CAST(1996-09-09 AS DATE) AS STRING) = 1996-09-09):boolean> +struct<(CAST(1996-09-09 AS DATE) = CAST(1996-09-09 AS DATE)):boolean> -- !query 205 output true @@ -1677,7 +1653,7 @@ true -- !query 206 SELECT date('1996-9-10') > '1996-09-09' FROM t -- !query 206 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) > 1996-09-09):boolean> +struct<(CAST(1996-9-10 AS DATE) > CAST(1996-09-09 AS DATE)):boolean> -- !query 206 output true @@ -1685,7 +1661,7 @@ true -- !query 207 SELECT date('1996-9-10') >= '1996-09-09' FROM t -- !query 207 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) >= 1996-09-09):boolean> +struct<(CAST(1996-9-10 AS DATE) >= CAST(1996-09-09 AS DATE)):boolean> -- !query 207 output true @@ -1693,7 +1669,7 @@ true -- !query 208 SELECT date('1996-9-10') < '1996-09-09' FROM t -- !query 208 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) < 1996-09-09):boolean> +struct<(CAST(1996-9-10 AS DATE) < CAST(1996-09-09 AS DATE)):boolean> -- !query 208 output false @@ -1701,7 +1677,7 @@ false -- !query 209 SELECT date('1996-9-10') <= '1996-09-09' FROM t -- !query 209 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) <= 1996-09-09):boolean> +struct<(CAST(1996-9-10 AS DATE) <= CAST(1996-09-09 AS DATE)):boolean> -- !query 209 output false @@ -1709,7 +1685,7 @@ false -- !query 210 SELECT date('1996-9-10') <> '1996-09-09' FROM t -- !query 210 schema -struct<(NOT (CAST(CAST(1996-9-10 AS DATE) AS STRING) = 1996-09-09)):boolean> +struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(1996-09-09 AS DATE))):boolean> -- !query 210 output true @@ -1717,7 +1693,7 @@ true -- !query 211 SELECT date('1996-09-09') = cast(null as string) FROM t -- !query 211 schema -struct<(CAST(CAST(1996-09-09 AS DATE) AS STRING) = CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-09-09 AS DATE) = CAST(CAST(NULL AS STRING) AS DATE)):boolean> -- !query 211 output NULL @@ -1725,7 +1701,7 @@ NULL -- !query 212 SELECT date('1996-9-10') > cast(null as string) FROM t -- !query 212 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) > CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) > CAST(CAST(NULL AS STRING) AS DATE)):boolean> -- !query 212 output NULL @@ -1733,7 +1709,7 @@ NULL -- !query 213 SELECT date('1996-9-10') >= cast(null as string) FROM t -- !query 213 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) >= CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) >= CAST(CAST(NULL AS STRING) AS DATE)):boolean> -- !query 213 output NULL @@ -1741,7 +1717,7 @@ NULL -- !query 214 SELECT date('1996-9-10') < cast(null as string) FROM t -- !query 214 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) < CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) < CAST(CAST(NULL AS STRING) AS DATE)):boolean> -- !query 214 output NULL @@ -1749,7 +1725,7 @@ NULL -- !query 215 SELECT date('1996-9-10') <= cast(null as string) FROM t -- !query 215 schema -struct<(CAST(CAST(1996-9-10 AS DATE) AS STRING) <= CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-9-10 AS DATE) <= CAST(CAST(NULL AS STRING) AS DATE)):boolean> -- !query 215 output NULL @@ -1757,7 +1733,7 @@ NULL -- !query 216 SELECT date('1996-9-10') <> cast(null as string) FROM t -- !query 216 schema -struct<(NOT (CAST(CAST(1996-9-10 AS DATE) AS STRING) = CAST(NULL AS STRING))):boolean> +struct<(NOT (CAST(1996-9-10 AS DATE) = CAST(CAST(NULL AS STRING) AS DATE))):boolean> -- !query 216 output NULL @@ -1773,7 +1749,7 @@ true -- !query 218 SELECT '1996-09-09 12:12:12.5' > timestamp('1996-09-09 12:12:12.4') FROM t -- !query 218 schema -struct<(1996-09-09 12:12:12.5 > CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 218 output true @@ -1781,7 +1757,7 @@ true -- !query 219 SELECT '1996-09-09 12:12:12.5' >= timestamp('1996-09-09 12:12:12.4') FROM t -- !query 219 schema -struct<(1996-09-09 12:12:12.5 >= CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 219 output true @@ -1789,7 +1765,7 @@ true -- !query 220 SELECT '1996-09-09 12:12:12.5' < timestamp('1996-09-09 12:12:12.4') FROM t -- !query 220 schema -struct<(1996-09-09 12:12:12.5 < CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 220 output false @@ -1797,7 +1773,7 @@ false -- !query 221 SELECT '1996-09-09 12:12:12.5' <= timestamp('1996-09-09 12:12:12.4') FROM t -- !query 221 schema -struct<(1996-09-09 12:12:12.5 <= CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 221 output false @@ -1821,7 +1797,7 @@ NULL -- !query 224 SELECT cast(null as string) > timestamp('1996-09-09 12:12:12.4') FROM t -- !query 224 schema -struct<(CAST(NULL AS STRING) > CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 224 output NULL @@ -1829,7 +1805,7 @@ NULL -- !query 225 SELECT cast(null as string) >= timestamp('1996-09-09 12:12:12.4') FROM t -- !query 225 schema -struct<(CAST(NULL AS STRING) >= CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 225 output NULL @@ -1837,7 +1813,7 @@ NULL -- !query 226 SELECT cast(null as string) < timestamp('1996-09-09 12:12:12.4') FROM t -- !query 226 schema -struct<(CAST(NULL AS STRING) < CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 226 output NULL @@ -1845,7 +1821,7 @@ NULL -- !query 227 SELECT cast(null as string) <= timestamp('1996-09-09 12:12:12.4') FROM t -- !query 227 schema -struct<(CAST(NULL AS STRING) <= CAST(CAST(1996-09-09 12:12:12.4 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(CAST(NULL AS STRING) AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 227 output NULL @@ -1869,7 +1845,7 @@ true -- !query 230 SELECT timestamp('1996-09-09 12:12:12.5' )> '1996-09-09 12:12:12.4' FROM t -- !query 230 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) > 1996-09-09 12:12:12.4):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 230 output true @@ -1877,7 +1853,7 @@ true -- !query 231 SELECT timestamp('1996-09-09 12:12:12.5' )>= '1996-09-09 12:12:12.4' FROM t -- !query 231 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) >= 1996-09-09 12:12:12.4):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 231 output true @@ -1885,7 +1861,7 @@ true -- !query 232 SELECT timestamp('1996-09-09 12:12:12.5' )< '1996-09-09 12:12:12.4' FROM t -- !query 232 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) < 1996-09-09 12:12:12.4):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 232 output false @@ -1893,7 +1869,7 @@ false -- !query 233 SELECT timestamp('1996-09-09 12:12:12.5' )<= '1996-09-09 12:12:12.4' FROM t -- !query 233 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) <= 1996-09-09 12:12:12.4):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(1996-09-09 12:12:12.4 AS TIMESTAMP)):boolean> -- !query 233 output false @@ -1917,7 +1893,7 @@ NULL -- !query 236 SELECT timestamp('1996-09-09 12:12:12.5' )> cast(null as string) FROM t -- !query 236 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) > CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) > CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> -- !query 236 output NULL @@ -1925,7 +1901,7 @@ NULL -- !query 237 SELECT timestamp('1996-09-09 12:12:12.5' )>= cast(null as string) FROM t -- !query 237 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) >= CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) >= CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> -- !query 237 output NULL @@ -1933,7 +1909,7 @@ NULL -- !query 238 SELECT timestamp('1996-09-09 12:12:12.5' )< cast(null as string) FROM t -- !query 238 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) < CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) < CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> -- !query 238 output NULL @@ -1941,7 +1917,7 @@ NULL -- !query 239 SELECT timestamp('1996-09-09 12:12:12.5' )<= cast(null as string) FROM t -- !query 239 schema -struct<(CAST(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) AS STRING) <= CAST(NULL AS STRING)):boolean> +struct<(CAST(1996-09-09 12:12:12.5 AS TIMESTAMP) <= CAST(CAST(NULL AS STRING) AS TIMESTAMP)):boolean> -- !query 239 output NULL diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out index a739f8d73181c..1e1cbc3304141 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/caseWhenCoercion.sql.out @@ -80,7 +80,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2' as binary) END FROM struct<> -- !query 9 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE binary END; line 1 pos 7 -- !query 10 @@ -89,7 +89,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast(2 as boolean) END FROM t struct<> -- !query 10 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE boolean END; line 1 pos 7 -- !query 11 @@ -98,7 +98,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00.0' struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE timestamp END; line 1 pos 7 -- !query 12 @@ -107,7 +107,7 @@ SELECT CASE WHEN true THEN cast(1 as tinyint) ELSE cast('2017-12-11 09:30:00' as struct<> -- !query 12 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS TINYINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN tinyint ELSE date END; line 1 pos 7 -- !query 13 @@ -180,7 +180,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2' as binary) END FROM struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE binary END; line 1 pos 7 -- !query 22 @@ -189,7 +189,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast(2 as boolean) END FROM struct<> -- !query 22 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE boolean END; line 1 pos 7 -- !query 23 @@ -198,7 +198,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00.0' struct<> -- !query 23 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE timestamp END; line 1 pos 7 -- !query 24 @@ -207,7 +207,7 @@ SELECT CASE WHEN true THEN cast(1 as smallint) ELSE cast('2017-12-11 09:30:00' a struct<> -- !query 24 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS SMALLINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN smallint ELSE date END; line 1 pos 7 -- !query 25 @@ -280,7 +280,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2' as binary) END FROM t struct<> -- !query 33 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE binary END; line 1 pos 7 -- !query 34 @@ -289,7 +289,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast(2 as boolean) END FROM t struct<> -- !query 34 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE boolean END; line 1 pos 7 -- !query 35 @@ -298,7 +298,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00.0' as t struct<> -- !query 35 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE timestamp END; line 1 pos 7 -- !query 36 @@ -307,7 +307,7 @@ SELECT CASE WHEN true THEN cast(1 as int) ELSE cast('2017-12-11 09:30:00' as dat struct<> -- !query 36 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS INT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN int ELSE date END; line 1 pos 7 -- !query 37 @@ -380,7 +380,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2' as binary) END FROM t struct<> -- !query 45 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE binary END; line 1 pos 7 -- !query 46 @@ -389,7 +389,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast(2 as boolean) END FROM t struct<> -- !query 46 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE boolean END; line 1 pos 7 -- !query 47 @@ -398,7 +398,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00.0' a struct<> -- !query 47 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE timestamp END; line 1 pos 7 -- !query 48 @@ -407,7 +407,7 @@ SELECT CASE WHEN true THEN cast(1 as bigint) ELSE cast('2017-12-11 09:30:00' as struct<> -- !query 48 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BIGINT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN bigint ELSE date END; line 1 pos 7 -- !query 49 @@ -480,7 +480,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2' as binary) END FROM t struct<> -- !query 57 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE binary END; line 1 pos 7 -- !query 58 @@ -489,7 +489,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast(2 as boolean) END FROM t struct<> -- !query 58 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE boolean END; line 1 pos 7 -- !query 59 @@ -498,7 +498,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00.0' as struct<> -- !query 59 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE timestamp END; line 1 pos 7 -- !query 60 @@ -507,7 +507,7 @@ SELECT CASE WHEN true THEN cast(1 as float) ELSE cast('2017-12-11 09:30:00' as d struct<> -- !query 60 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS FLOAT) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN float ELSE date END; line 1 pos 7 -- !query 61 @@ -580,7 +580,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2' as binary) END FROM t struct<> -- !query 69 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE binary END; line 1 pos 7 -- !query 70 @@ -589,7 +589,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast(2 as boolean) END FROM t struct<> -- !query 70 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE boolean END; line 1 pos 7 -- !query 71 @@ -598,7 +598,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00.0' a struct<> -- !query 71 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE timestamp END; line 1 pos 7 -- !query 72 @@ -607,7 +607,7 @@ SELECT CASE WHEN true THEN cast(1 as double) ELSE cast('2017-12-11 09:30:00' as struct<> -- !query 72 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DOUBLE) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN double ELSE date END; line 1 pos 7 -- !query 73 @@ -680,7 +680,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2' as binary) EN struct<> -- !query 81 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE binary END; line 1 pos 7 -- !query 82 @@ -689,7 +689,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast(2 as boolean) END struct<> -- !query 82 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE boolean END; line 1 pos 7 -- !query 83 @@ -698,7 +698,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30 struct<> -- !query 83 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE timestamp END; line 1 pos 7 -- !query 84 @@ -707,7 +707,7 @@ SELECT CASE WHEN true THEN cast(1 as decimal(10, 0)) ELSE cast('2017-12-11 09:30 struct<> -- !query 84 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS DECIMAL(10,0)) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN decimal(10,0) ELSE date END; line 1 pos 7 -- !query 85 @@ -780,7 +780,7 @@ SELECT CASE WHEN true THEN cast(1 as string) ELSE cast('2' as binary) END FROM t struct<> -- !query 93 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE binary END; line 1 pos 7 -- !query 94 @@ -789,7 +789,7 @@ SELECT CASE WHEN true THEN cast(1 as string) ELSE cast(2 as boolean) END FROM t struct<> -- !query 94 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS STRING) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN string ELSE boolean END; line 1 pos 7 -- !query 95 @@ -814,7 +814,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as tinyint) END FROM struct<> -- !query 97 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE tinyint END; line 1 pos 7 -- !query 98 @@ -823,7 +823,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as smallint) END FROM struct<> -- !query 98 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE smallint END; line 1 pos 7 -- !query 99 @@ -832,7 +832,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as int) END FROM t struct<> -- !query 99 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE int END; line 1 pos 7 -- !query 100 @@ -841,7 +841,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as bigint) END FROM t struct<> -- !query 100 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE bigint END; line 1 pos 7 -- !query 101 @@ -850,7 +850,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as float) END FROM t struct<> -- !query 101 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE float END; line 1 pos 7 -- !query 102 @@ -859,7 +859,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as double) END FROM t struct<> -- !query 102 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE double END; line 1 pos 7 -- !query 103 @@ -868,7 +868,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as decimal(10, 0)) EN struct<> -- !query 103 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE decimal(10,0) END; line 1 pos 7 -- !query 104 @@ -877,7 +877,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as string) END FROM t struct<> -- !query 104 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE string END; line 1 pos 7 -- !query 105 @@ -894,7 +894,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast(2 as boolean) END FROM struct<> -- !query 106 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE boolean END; line 1 pos 7 -- !query 107 @@ -903,7 +903,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00.0' struct<> -- !query 107 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE timestamp END; line 1 pos 7 -- !query 108 @@ -912,7 +912,7 @@ SELECT CASE WHEN true THEN cast('1' as binary) ELSE cast('2017-12-11 09:30:00' a struct<> -- !query 108 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('1' AS BINARY) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN binary ELSE date END; line 1 pos 7 -- !query 109 @@ -921,7 +921,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as tinyint) END FROM t struct<> -- !query 109 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE tinyint END; line 1 pos 7 -- !query 110 @@ -930,7 +930,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as smallint) END FROM struct<> -- !query 110 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE smallint END; line 1 pos 7 -- !query 111 @@ -939,7 +939,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as int) END FROM t struct<> -- !query 111 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE int END; line 1 pos 7 -- !query 112 @@ -948,7 +948,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as bigint) END FROM t struct<> -- !query 112 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE bigint END; line 1 pos 7 -- !query 113 @@ -957,7 +957,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as float) END FROM t struct<> -- !query 113 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE float END; line 1 pos 7 -- !query 114 @@ -966,7 +966,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as double) END FROM t struct<> -- !query 114 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE double END; line 1 pos 7 -- !query 115 @@ -975,7 +975,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as decimal(10, 0)) END struct<> -- !query 115 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE decimal(10,0) END; line 1 pos 7 -- !query 116 @@ -984,7 +984,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast(2 as string) END FROM t struct<> -- !query 116 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST(2 AS STRING) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE string END; line 1 pos 7 -- !query 117 @@ -993,7 +993,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2' as binary) END FROM struct<> -- !query 117 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE binary END; line 1 pos 7 -- !query 118 @@ -1010,7 +1010,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00.0' struct<> -- !query 119 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00.0' AS TIMESTAMP) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE timestamp END; line 1 pos 7 -- !query 120 @@ -1019,7 +1019,7 @@ SELECT CASE WHEN true THEN cast(1 as boolean) ELSE cast('2017-12-11 09:30:00' as struct<> -- !query 120 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST(1 AS BOOLEAN) ELSE CAST('2017-12-11 09:30:00' AS DATE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN boolean ELSE date END; line 1 pos 7 -- !query 121 @@ -1028,7 +1028,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 121 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE tinyint END; line 1 pos 7 -- !query 122 @@ -1037,7 +1037,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 122 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE smallint END; line 1 pos 7 -- !query 123 @@ -1046,7 +1046,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 123 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE int END; line 1 pos 7 -- !query 124 @@ -1055,7 +1055,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 124 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE bigint END; line 1 pos 7 -- !query 125 @@ -1064,7 +1064,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 125 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE float END; line 1 pos 7 -- !query 126 @@ -1073,7 +1073,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 126 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE double END; line 1 pos 7 -- !query 127 @@ -1082,7 +1082,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 127 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE decimal(10,0) END; line 1 pos 7 -- !query 128 @@ -1099,7 +1099,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 129 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE binary END; line 1 pos 7 -- !query 130 @@ -1108,7 +1108,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00.0' as timestamp) ELSE cast( struct<> -- !query 130 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00.0' AS TIMESTAMP) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN timestamp ELSE boolean END; line 1 pos 7 -- !query 131 @@ -1133,7 +1133,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as ti struct<> -- !query 133 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS TINYINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE tinyint END; line 1 pos 7 -- !query 134 @@ -1142,7 +1142,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as sm struct<> -- !query 134 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS SMALLINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE smallint END; line 1 pos 7 -- !query 135 @@ -1151,7 +1151,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as in struct<> -- !query 135 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS INT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE int END; line 1 pos 7 -- !query 136 @@ -1160,7 +1160,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bi struct<> -- !query 136 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BIGINT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE bigint END; line 1 pos 7 -- !query 137 @@ -1169,7 +1169,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as fl struct<> -- !query 137 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS FLOAT) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE float END; line 1 pos 7 -- !query 138 @@ -1178,7 +1178,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as do struct<> -- !query 138 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DOUBLE) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE double END; line 1 pos 7 -- !query 139 @@ -1187,7 +1187,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as de struct<> -- !query 139 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS DECIMAL(10,0)) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE decimal(10,0) END; line 1 pos 7 -- !query 140 @@ -1204,7 +1204,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast('2' as struct<> -- !query 141 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST('2' AS BINARY) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE binary END; line 1 pos 7 -- !query 142 @@ -1213,7 +1213,7 @@ SELECT CASE WHEN true THEN cast('2017-12-12 09:30:00' as date) ELSE cast(2 as bo struct<> -- !query 142 output org.apache.spark.sql.AnalysisException -cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type; line 1 pos 7 +cannot resolve 'CASE WHEN true THEN CAST('2017-12-12 09:30:00' AS DATE) ELSE CAST(2 AS BOOLEAN) END' due to data type mismatch: THEN and ELSE expressions should all be same type or coercible to a common type, got CASE WHEN ... THEN date ELSE boolean END; line 1 pos 7 -- !query 143 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out index 35740094ba53e..86a578ca013df 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapZipWith.sql.out @@ -85,7 +85,7 @@ FROM various_maps struct<> -- !query 5 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map1`, various_maps.`decimal_map2`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,0), decimal(36,35)].; line 1 pos 7 -- !query 6 @@ -113,7 +113,7 @@ FROM various_maps struct<> -- !query 8 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), `k`, NamePlaceholder(), `v1`, NamePlaceholder(), `v2`), `k`, `v1`, `v2`))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 +cannot resolve 'map_zip_with(various_maps.`decimal_map2`, various_maps.`int_map`, lambdafunction(named_struct(NamePlaceholder(), k, NamePlaceholder(), v1, NamePlaceholder(), v2), k, v1, v2))' due to argument data type mismatch: The input to function map_zip_with should have been two maps with compatible key types, but the key types are [decimal(36,35), int].; line 1 pos 7 -- !query 9 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out index efc88e47209a6..79e00860e4c05 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/mapconcat.sql.out @@ -18,7 +18,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( map('a', 'b'), map('c', 'd'), map(array('a', 'b'), array('c', 'd')), map(array('e'), array('f')), map(struct('a', 1), struct('b', 2)), map(struct('c', 3), struct('d', 4)), - map(map('a', 1), map('b', 2)), map(map('c', 3), map('d', 4)), map('a', 1), map('c', 2), map(1, 'a'), map(2, 'c') ) AS various_maps ( @@ -36,7 +35,6 @@ CREATE TEMPORARY VIEW various_maps AS SELECT * FROM VALUES ( string_map1, string_map2, array_map1, array_map2, struct_map1, struct_map2, - map_map1, map_map2, string_int_map1, string_int_map2, int_string_map1, int_string_map2 ) @@ -61,14 +59,13 @@ SELECT map_concat(string_map1, string_map2) string_map, map_concat(array_map1, array_map2) array_map, map_concat(struct_map1, struct_map2) struct_map, - map_concat(map_map1, map_map2) map_map, map_concat(string_int_map1, string_int_map2) string_int_map, map_concat(int_string_map1, int_string_map2) int_string_map FROM various_maps -- !query 1 schema -struct,tinyint_map:map,smallint_map:map,int_map:map,bigint_map:map,decimal_map:map,float_map:map,double_map:map,date_map:map,timestamp_map:map,string_map:map,array_map:map,array>,struct_map:map,struct>,map_map:map,map>,string_int_map:map,int_string_map:map> +struct,tinyint_map:map,smallint_map:map,int_map:map,bigint_map:map,decimal_map:map,float_map:map,double_map:map,date_map:map,timestamp_map:map,string_map:map,array_map:map,array>,struct_map:map,struct>,string_int_map:map,int_string_map:map> -- !query 1 output -{false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00.0:2016-11-09 20:54:00.0,2016-11-15 20:54:00.0:2016-11-12 20:54:00.0} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {{"a":1}:{"b":2},{"c":3}:{"d":4}} {"a":1,"c":2} {1:"a",2:"c"} +{false:true,true:false} {1:2,3:4} {1:2,3:4} {4:6,7:8} {6:7,8:9} {9223372036854775808:9223372036854775809,9223372036854775809:9223372036854775808} {1.0:2.0,3.0:4.0} {1.0:2.0,3.0:4.0} {2016-03-12:2016-03-11,2016-03-14:2016-03-13} {2016-11-11 20:54:00.0:2016-11-09 20:54:00.0,2016-11-15 20:54:00.0:2016-11-12 20:54:00.0} {"a":"b","c":"d"} {["a","b"]:["c","d"],["e"]:["f"]} {{"col1":"a","col2":1}:{"col1":"b","col2":2},{"col1":"c","col2":3}:{"col1":"d","col2":4}} {"a":1,"c":2} {1:"a",2:"c"} -- !query 2 @@ -91,13 +88,13 @@ struct,si_map:map,ib_map:map -- !query 3 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_concat(various_maps.`tinyint_map1`, various_maps.`map_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,map>]; line 2 pos 4 +cannot resolve 'map_concat(various_maps.`tinyint_map1`, various_maps.`array_map1`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,array>]; line 2 pos 4 -- !query 4 @@ -124,21 +121,21 @@ cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`struct_map2`)' -- !query 6 SELECT - map_concat(map_map1, array_map2) ma_map + map_concat(struct_map1, array_map2) ma_map FROM various_maps -- !query 6 schema struct<> -- !query 6 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_concat(various_maps.`map_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map,map>, map,array>]; line 2 pos 4 +cannot resolve 'map_concat(various_maps.`struct_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map,struct>, map,array>]; line 2 pos 4 -- !query 7 SELECT - map_concat(map_map1, struct_map2) ms_map + map_concat(int_map1, array_map2) ms_map FROM various_maps -- !query 7 schema struct<> -- !query 7 output org.apache.spark.sql.AnalysisException -cannot resolve 'map_concat(various_maps.`map_map1`, various_maps.`struct_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map,map>, map,struct>]; line 2 pos 4 +cannot resolve 'map_concat(various_maps.`int_map1`, various_maps.`array_map2`)' due to data type mismatch: input to function map_concat should all be the same type, but it's [map, map,array>]; line 2 pos 4 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 0beb1f6263d2c..c54ceba85ce79 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 @@ -1253,9 +1253,9 @@ NULL -- !query 150 SELECT '1' = cast('2017-12-11 09:30:00' as date) FROM t -- !query 150 schema -struct<(1 = CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) = CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 150 output -false +NULL -- !query 151 @@ -1341,9 +1341,9 @@ NULL -- !query 161 SELECT cast('2017-12-11 09:30:00' as date) = '1' FROM t -- !query 161 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) = 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DATE)):boolean> -- !query 161 output -false +NULL -- !query 162 @@ -1437,7 +1437,7 @@ false -- !query 173 SELECT '1' <=> cast('2017-12-11 09:30:00' as date) FROM t -- !query 173 schema -struct<(1 <=> CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) <=> CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 173 output false @@ -1525,7 +1525,7 @@ false -- !query 184 SELECT cast('2017-12-11 09:30:00' as date) <=> '1' FROM t -- !query 184 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) <=> 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) <=> CAST(1 AS DATE)):boolean> -- !query 184 output false @@ -1613,17 +1613,17 @@ false -- !query 195 SELECT '1' < cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query 195 schema -struct<(1 < CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1 AS TIMESTAMP) < CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> -- !query 195 output -true +NULL -- !query 196 SELECT '1' < cast('2017-12-11 09:30:00' as date) FROM t -- !query 196 schema -struct<(1 < CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) < CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 196 output -true +NULL -- !query 197 @@ -1709,17 +1709,17 @@ true -- !query 207 SELECT '1' <= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query 207 schema -struct<(1 <= CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1 AS TIMESTAMP) <= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> -- !query 207 output -true +NULL -- !query 208 SELECT '1' <= cast('2017-12-11 09:30:00' as date) FROM t -- !query 208 schema -struct<(1 <= CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) <= CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 208 output -true +NULL -- !query 209 @@ -1805,17 +1805,17 @@ false -- !query 219 SELECT '1' > cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query 219 schema -struct<(1 > CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1 AS TIMESTAMP) > CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> -- !query 219 output -false +NULL -- !query 220 SELECT '1' > cast('2017-12-11 09:30:00' as date) FROM t -- !query 220 schema -struct<(1 > CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) > CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 220 output -false +NULL -- !query 221 @@ -1901,17 +1901,17 @@ true -- !query 231 SELECT '1' >= cast('2017-12-11 09:30:00.0' as timestamp) FROM t -- !query 231 schema -struct<(1 >= CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING)):boolean> +struct<(CAST(1 AS TIMESTAMP) >= CAST(2017-12-11 09:30:00.0 AS TIMESTAMP)):boolean> -- !query 231 output -false +NULL -- !query 232 SELECT '1' >= cast('2017-12-11 09:30:00' as date) FROM t -- !query 232 schema -struct<(1 >= CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING)):boolean> +struct<(CAST(1 AS DATE) >= CAST(2017-12-11 09:30:00 AS DATE)):boolean> -- !query 232 output -false +NULL -- !query 233 @@ -2005,9 +2005,9 @@ NULL -- !query 244 SELECT '1' <> cast('2017-12-11 09:30:00' as date) FROM t -- !query 244 schema -struct<(NOT (1 = CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING))):boolean> +struct<(NOT (CAST(1 AS DATE) = CAST(2017-12-11 09:30:00 AS DATE))):boolean> -- !query 244 output -true +NULL -- !query 245 @@ -2093,17 +2093,17 @@ false -- !query 255 SELECT cast('2017-12-11 09:30:00.0' as timestamp) < '1' FROM t -- !query 255 schema -struct<(CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING) < 1):boolean> +struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) < CAST(1 AS TIMESTAMP)):boolean> -- !query 255 output -false +NULL -- !query 256 SELECT cast('2017-12-11 09:30:00' as date) < '1' FROM t -- !query 256 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) < 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) < CAST(1 AS DATE)):boolean> -- !query 256 output -false +NULL -- !query 257 @@ -2189,17 +2189,17 @@ true -- !query 267 SELECT cast('2017-12-11 09:30:00.0' as timestamp) <= '1' FROM t -- !query 267 schema -struct<(CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING) <= 1):boolean> +struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) <= CAST(1 AS TIMESTAMP)):boolean> -- !query 267 output -false +NULL -- !query 268 SELECT cast('2017-12-11 09:30:00' as date) <= '1' FROM t -- !query 268 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) <= 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) <= CAST(1 AS DATE)):boolean> -- !query 268 output -false +NULL -- !query 269 @@ -2285,17 +2285,17 @@ false -- !query 279 SELECT cast('2017-12-11 09:30:00.0' as timestamp) > '1' FROM t -- !query 279 schema -struct<(CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING) > 1):boolean> +struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) > CAST(1 AS TIMESTAMP)):boolean> -- !query 279 output -true +NULL -- !query 280 SELECT cast('2017-12-11 09:30:00' as date) > '1' FROM t -- !query 280 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) > 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) > CAST(1 AS DATE)):boolean> -- !query 280 output -true +NULL -- !query 281 @@ -2381,17 +2381,17 @@ true -- !query 291 SELECT cast('2017-12-11 09:30:00.0' as timestamp) >= '1' FROM t -- !query 291 schema -struct<(CAST(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) AS STRING) >= 1):boolean> +struct<(CAST(2017-12-11 09:30:00.0 AS TIMESTAMP) >= CAST(1 AS TIMESTAMP)):boolean> -- !query 291 output -true +NULL -- !query 292 SELECT cast('2017-12-11 09:30:00' as date) >= '1' FROM t -- !query 292 schema -struct<(CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) >= 1):boolean> +struct<(CAST(2017-12-11 09:30:00 AS DATE) >= CAST(1 AS DATE)):boolean> -- !query 292 output -true +NULL -- !query 293 @@ -2485,9 +2485,9 @@ NULL -- !query 304 SELECT cast('2017-12-11 09:30:00' as date) <> '1' FROM t -- !query 304 schema -struct<(NOT (CAST(CAST(2017-12-11 09:30:00 AS DATE) AS STRING) = 1)):boolean> +struct<(NOT (CAST(2017-12-11 09:30:00 AS DATE) = CAST(1 AS DATE))):boolean> -- !query 304 output -true +NULL -- !query 305 diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out index 8ed2820244412..31ee700a8db95 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/stringCastAndExpressions.sql.out @@ -256,6 +256,6 @@ NULL -- !query 31 select from_json(a, 'a INT') from t -- !query 31 schema -struct> +struct> -- !query 31 output -NULL +{"a":null} diff --git a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out index 01d83938031fe..4fa2032c66c60 100644 --- a/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/typeCoercion/native/windowFrameCoercion.sql.out @@ -177,7 +177,7 @@ SELECT COUNT(*) OVER (PARTITION BY 1 ORDER BY cast('1' as binary) DESC RANGE BET struct<> -- !query 21 output org.apache.spark.sql.AnalysisException -cannot resolve '(PARTITION BY 1 ORDER BY CAST('1' AS BINARY) DESC NULLS LAST RANGE BETWEEN CURRENT ROW AND 1 FOLLOWING)' due to data type mismatch: The data type 'binary' used in the order specification does not match the data type 'int' which is used in the range frame.; line 1 pos 21 +cannot resolve 'RANGE BETWEEN CURRENT ROW AND CAST(1 AS BINARY) FOLLOWING' due to data type mismatch: The data type of the upper bound 'binary' does not match the expected data type '(numeric or calendarinterval)'.; line 1 pos 21 -- !query 22 diff --git a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out index 87824ab81cdf7..f4455bb717578 100644 --- a/sql/core/src/test/resources/sql-tests/results/udaf.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/udaf.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 6 +-- Number of queries: 8 -- !query 0 @@ -52,3 +52,19 @@ struct<> -- !query 5 output org.apache.spark.sql.AnalysisException Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 7 + + +-- !query 6 +DROP FUNCTION myDoubleAvg +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +DROP FUNCTION udaf1 +-- !query 7 schema +struct<> +-- !query 7 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out new file mode 100644 index 0000000000000..a2f64717d73a1 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part1.sql.out @@ -0,0 +1,387 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 43 + + +-- !query 0 +SELECT avg(udf(four)) AS avg_1 FROM onek +-- !query 0 schema +struct +-- !query 0 output +1.5 + + +-- !query 1 +SELECT udf(avg(a)) AS avg_32 FROM aggtest WHERE a < 100 +-- !query 1 schema +struct +-- !query 1 output +32.666666666666664 + + +-- !query 2 +select CAST(avg(udf(b)) AS Decimal(10,3)) AS avg_107_943 FROM aggtest +-- !query 2 schema +struct +-- !query 2 output +107.943 + + +-- !query 3 +SELECT sum(udf(four)) AS sum_1500 FROM onek +-- !query 3 schema +struct +-- !query 3 output +1500 + + +-- !query 4 +SELECT udf(sum(a)) AS sum_198 FROM aggtest +-- !query 4 schema +struct +-- !query 4 output +198 + + +-- !query 5 +SELECT udf(udf(sum(b))) AS avg_431_773 FROM aggtest +-- !query 5 schema +struct +-- !query 5 output +431.77260909229517 + + +-- !query 6 +SELECT udf(max(four)) AS max_3 FROM onek +-- !query 6 schema +struct +-- !query 6 output +3 + + +-- !query 7 +SELECT max(udf(a)) AS max_100 FROM aggtest +-- !query 7 schema +struct +-- !query 7 output +100 + + +-- !query 8 +SELECT udf(udf(max(aggtest.b))) AS max_324_78 FROM aggtest +-- !query 8 schema +struct +-- !query 8 output +324.78 + + +-- !query 9 +SELECT stddev_pop(udf(b)) FROM aggtest +-- !query 9 schema +struct +-- !query 9 output +131.10703231895047 + + +-- !query 10 +SELECT udf(stddev_samp(b)) FROM aggtest +-- !query 10 schema +struct +-- !query 10 output +151.38936080399804 + + +-- !query 11 +SELECT var_pop(udf(b)) FROM aggtest +-- !query 11 schema +struct +-- !query 11 output +17189.053923482323 + + +-- !query 12 +SELECT udf(var_samp(b)) FROM aggtest +-- !query 12 schema +struct +-- !query 12 output +22918.738564643096 + + +-- !query 13 +SELECT udf(stddev_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 13 schema +struct +-- !query 13 output +131.18117242958306 + + +-- !query 14 +SELECT stddev_samp(CAST(udf(b) AS Decimal(38,0))) FROM aggtest +-- !query 14 schema +struct +-- !query 14 output +151.47497042966097 + + +-- !query 15 +SELECT udf(var_pop(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 15 schema +struct +-- !query 15 output +17208.5 + + +-- !query 16 +SELECT var_samp(udf(CAST(b AS Decimal(38,0)))) FROM aggtest +-- !query 16 schema +struct +-- !query 16 output +22944.666666666668 + + +-- !query 17 +SELECT udf(var_pop(1.0)), var_samp(udf(2.0)) +-- !query 17 schema +struct +-- !query 17 output +0.0 NaN + + +-- !query 18 +SELECT stddev_pop(udf(CAST(3.0 AS Decimal(38,0)))), stddev_samp(CAST(udf(4.0) AS Decimal(38,0))) +-- !query 18 schema +struct +-- !query 18 output +0.0 NaN + + +-- !query 19 +select sum(udf(CAST(null AS int))) from range(1,4) +-- !query 19 schema +struct +-- !query 19 output +NULL + + +-- !query 20 +select sum(udf(CAST(null AS long))) from range(1,4) +-- !query 20 schema +struct +-- !query 20 output +NULL + + +-- !query 21 +select sum(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query 21 schema +struct +-- !query 21 output +NULL + + +-- !query 22 +select sum(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query 22 schema +struct +-- !query 22 output +NULL + + +-- !query 23 +select avg(udf(CAST(null AS int))) from range(1,4) +-- !query 23 schema +struct +-- !query 23 output +NULL + + +-- !query 24 +select avg(udf(CAST(null AS long))) from range(1,4) +-- !query 24 schema +struct +-- !query 24 output +NULL + + +-- !query 25 +select avg(udf(CAST(null AS Decimal(38,0)))) from range(1,4) +-- !query 25 schema +struct +-- !query 25 output +NULL + + +-- !query 26 +select avg(udf(CAST(null AS DOUBLE))) from range(1,4) +-- !query 26 schema +struct +-- !query 26 output +NULL + + +-- !query 27 +select sum(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query 27 schema +struct +-- !query 27 output +NaN + + +-- !query 28 +select avg(CAST(udf('NaN') AS DOUBLE)) from range(1,4) +-- !query 28 schema +struct +-- !query 28 output +NaN + + +-- !query 29 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('1')) v(x) +-- !query 29 schema +struct +-- !query 29 output +Infinity NaN + + +-- !query 30 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('Infinity'), ('Infinity')) v(x) +-- !query 30 schema +struct +-- !query 30 output +Infinity NaN + + +-- !query 31 +SELECT avg(CAST(udf(x) AS DOUBLE)), var_pop(CAST(udf(x) AS DOUBLE)) +FROM (VALUES ('-Infinity'), ('Infinity')) v(x) +-- !query 31 schema +struct +-- !query 31 output +NaN NaN + + +-- !query 32 +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (100000003), (100000004), (100000006), (100000007)) v(x) +-- !query 32 schema +struct +-- !query 32 output +1.00000005E8 2.5 + + +-- !query 33 +SELECT avg(udf(CAST(x AS DOUBLE))), udf(var_pop(CAST(x AS DOUBLE))) +FROM (VALUES (7000000000005), (7000000000007)) v(x) +-- !query 33 schema +struct +-- !query 33 output +7.000000000006E12 1.0 + + +-- !query 34 +SELECT udf(covar_pop(b, udf(a))), covar_samp(udf(b), a) FROM aggtest +-- !query 34 schema +struct +-- !query 34 output +653.6289553875104 871.5052738500139 + + +-- !query 35 +SELECT corr(b, udf(a)) FROM aggtest +-- !query 35 schema +struct +-- !query 35 output +0.1396345165178734 + + +-- !query 36 +SELECT count(udf(four)) AS cnt_1000 FROM onek +-- !query 36 schema +struct +-- !query 36 output +1000 + + +-- !query 37 +SELECT udf(count(DISTINCT four)) AS cnt_4 FROM onek +-- !query 37 schema +struct +-- !query 37 output +4 + + +-- !query 38 +select ten, udf(count(*)), sum(udf(four)) from onek +group by ten order by ten +-- !query 38 schema +struct +-- !query 38 output +0 100 100 +1 100 200 +2 100 100 +3 100 200 +4 100 100 +5 100 200 +6 100 100 +7 100 200 +8 100 100 +9 100 200 + + +-- !query 39 +select ten, count(udf(four)), udf(sum(DISTINCT four)) from onek +group by ten order by ten +-- !query 39 schema +struct +-- !query 39 output +0 100 2 +1 100 4 +2 100 2 +3 100 4 +4 100 2 +5 100 4 +6 100 2 +7 100 4 +8 100 2 +9 100 4 + + +-- !query 40 +select ten, udf(sum(distinct four)) from onek a +group by ten +having exists (select 1 from onek b where udf(sum(distinct a.four)) = b.four) +-- !query 40 schema +struct +-- !query 40 output +0 2 +2 2 +4 2 +6 2 +8 2 + + +-- !query 41 +select ten, sum(distinct four) from onek a +group by ten +having exists (select 1 from onek b + where sum(distinct a.four + b.four) = udf(b.four)) +-- !query 41 schema +struct<> +-- !query 41 output +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(cast(four as string)) AS INT) AS BIGINT))] +Invalid expressions: [sum(DISTINCT CAST((outer() + b.`four`) AS BIGINT))]; + + +-- !query 42 +select + (select udf(max((select i.unique2 from tenk1 i where i.unique1 = o.unique1)))) +from tenk1 o +-- !query 42 schema +struct<> +-- !query 42 output +org.apache.spark.sql.AnalysisException +cannot resolve '`o.unique1`' given input columns: [i.even, i.fivethous, i.four, i.hundred, i.odd, i.string4, i.stringu1, i.stringu2, i.ten, i.tenthous, i.thousand, i.twenty, i.two, i.twothousand, i.unique1, i.unique2]; line 2 pos 67 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out new file mode 100644 index 0000000000000..9fe943874c3e5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-aggregates_part2.sql.out @@ -0,0 +1,162 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 16 + + +-- !query 0 +create temporary view int4_tbl as select * from values + (0), + (123456), + (-123456), + (2147483647), + (-2147483647) + as int4_tbl(f1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + -- boolean and transitions + -- null because strict + (NULL AND NULL) IS NULL AS `t`, + (TRUE AND NULL) IS NULL AS `t`, + (FALSE AND NULL) IS NULL AS `t`, + (NULL AND TRUE) IS NULL AS `t`, + (NULL AND FALSE) IS NULL AS `t`, + -- and actual computations + (TRUE AND TRUE) AS `t`, + NOT (TRUE AND FALSE) AS `t`, + NOT (FALSE AND TRUE) AS `t`, + NOT (FALSE AND FALSE) AS `t` +-- !query 1 schema +struct +-- !query 1 output +true true false true false true true true true + + +-- !query 2 +SELECT + -- boolean or transitions + -- null because strict + (NULL OR NULL) IS NULL AS `t`, + (TRUE OR NULL) IS NULL AS `t`, + (FALSE OR NULL) IS NULL AS `t`, + (NULL OR TRUE) IS NULL AS `t`, + (NULL OR FALSE) IS NULL AS `t`, + -- actual computations + (TRUE OR TRUE) AS `t`, + (TRUE OR FALSE) AS `t`, + (FALSE OR TRUE) AS `t`, + NOT (FALSE OR FALSE) AS `t` +-- !query 2 schema +struct +-- !query 2 output +true false true false true true true true true + + +-- !query 3 +select min(udf(unique1)) from tenk1 +-- !query 3 schema +struct +-- !query 3 output +0 + + +-- !query 4 +select udf(max(unique1)) from tenk1 +-- !query 4 schema +struct +-- !query 4 output +9999 + + +-- !query 5 +select max(unique1) from tenk1 where udf(unique1) < 42 +-- !query 5 schema +struct +-- !query 5 output +41 + + +-- !query 6 +select max(unique1) from tenk1 where unique1 > udf(42) +-- !query 6 schema +struct +-- !query 6 output +9999 + + +-- !query 7 +select max(unique1) from tenk1 where udf(unique1) > 42000 +-- !query 7 schema +struct +-- !query 7 output +NULL + + +-- !query 8 +select max(tenthous) from tenk1 where udf(thousand) = 33 +-- !query 8 schema +struct +-- !query 8 output +9033 + + +-- !query 9 +select min(tenthous) from tenk1 where udf(thousand) = 33 +-- !query 9 schema +struct +-- !query 9 output +33 + + +-- !query 10 +select distinct max(udf(unique2)) from tenk1 +-- !query 10 schema +struct +-- !query 10 output +9999 + + +-- !query 11 +select max(unique2) from tenk1 order by udf(1) +-- !query 11 schema +struct +-- !query 11 output +9999 + + +-- !query 12 +select max(unique2) from tenk1 order by max(udf(unique2)) +-- !query 12 schema +struct +-- !query 12 output +9999 + + +-- !query 13 +select udf(max(udf(unique2))) from tenk1 order by udf(max(unique2))+1 +-- !query 13 schema +struct +-- !query 13 output +9999 + + +-- !query 14 +select t1.max_unique2, udf(g) from (select max(udf(unique2)) as max_unique2 FROM tenk1) t1 LATERAL VIEW explode(array(1,2,3)) t2 AS g order by g desc +-- !query 14 schema +struct +-- !query 14 output +9999 3 +9999 2 +9999 1 + + +-- !query 15 +select udf(max(100)) from tenk1 +-- !query 15 schema +struct +-- !query 15 output +100 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out new file mode 100644 index 0000000000000..d9a8ca86361fc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/pgSQL/udf-case.sql.out @@ -0,0 +1,409 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 35 + + +-- !query 0 +CREATE TABLE CASE_TBL ( + i integer, + f double +) USING parquet +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TABLE CASE2_TBL ( + i integer, + j integer +) USING parquet +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +INSERT INTO CASE_TBL VALUES (1, 10.1) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +INSERT INTO CASE_TBL VALUES (2, 20.2) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +INSERT INTO CASE_TBL VALUES (3, -30.3) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +INSERT INTO CASE_TBL VALUES (4, NULL) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +INSERT INTO CASE2_TBL VALUES (1, -1) +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +INSERT INTO CASE2_TBL VALUES (2, -2) +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +INSERT INTO CASE2_TBL VALUES (3, -3) +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +INSERT INTO CASE2_TBL VALUES (2, -4) +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +INSERT INTO CASE2_TBL VALUES (1, NULL) +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +INSERT INTO CASE2_TBL VALUES (NULL, -6) +-- !query 11 schema +struct<> +-- !query 11 output + + + +-- !query 12 +SELECT '3' AS `One`, + CASE + WHEN udf(1 < 2) THEN 3 + END AS `Simple WHEN` +-- !query 12 schema +struct +-- !query 12 output +3 3 + + +-- !query 13 +SELECT '' AS `One`, + CASE + WHEN 1 > 2 THEN udf(3) + END AS `Simple default` +-- !query 13 schema +struct +-- !query 13 output + NULL + + +-- !query 14 +SELECT '3' AS `One`, + CASE + WHEN udf(1) < 2 THEN udf(3) + ELSE udf(4) + END AS `Simple ELSE` +-- !query 14 schema +struct +-- !query 14 output +3 3 + + +-- !query 15 +SELECT udf('4') AS `One`, + CASE + WHEN 1 > 2 THEN 3 + ELSE 4 + END AS `ELSE default` +-- !query 15 schema +struct +-- !query 15 output +4 4 + + +-- !query 16 +SELECT udf('6') AS `One`, + CASE + WHEN udf(1 > 2) THEN 3 + WHEN udf(4) < 5 THEN 6 + ELSE 7 + END AS `Two WHEN with default` +-- !query 16 schema +struct +-- !query 16 output +6 6 + + +-- !query 17 +SELECT '7' AS `None`, + CASE WHEN rand() < udf(0) THEN 1 + END AS `NULL on no matches` +-- !query 17 schema +struct +-- !query 17 output +7 NULL + + +-- !query 18 +SELECT CASE WHEN udf(1=0) THEN 1/0 WHEN 1=1 THEN 1 ELSE 2/0 END +-- !query 18 schema +struct +-- !query 18 output +1 + + +-- !query 19 +SELECT CASE 1 WHEN 0 THEN 1/udf(0) WHEN 1 THEN 1 ELSE 2/0 END +-- !query 19 schema +struct +-- !query 19 output +1 + + +-- !query 20 +SELECT CASE WHEN i > 100 THEN udf(1/0) ELSE udf(0) END FROM case_tbl +-- !query 20 schema +struct 100) THEN CAST(udf(cast((1 div 0) as string)) AS INT) ELSE CAST(udf(cast(0 as string)) AS INT) END:int> +-- !query 20 output +0 +0 +0 +0 + + +-- !query 21 +SELECT CASE 'a' WHEN 'a' THEN udf(1) ELSE udf(2) END +-- !query 21 schema +struct +-- !query 21 output +1 + + +-- !query 22 +SELECT '' AS `Five`, + CASE + WHEN i >= 3 THEN i + END AS `>= 3 or Null` + FROM CASE_TBL +-- !query 22 schema +struct= 3 or Null:int> +-- !query 22 output +3 + 4 + NULL + NULL + + +-- !query 23 +SELECT '' AS `Five`, + CASE WHEN i >= 3 THEN (i + i) + ELSE i + END AS `Simplest Math` + FROM CASE_TBL +-- !query 23 schema +struct +-- !query 23 output +1 + 2 + 6 + 8 + + +-- !query 24 +SELECT '' AS `Five`, i AS `Value`, + CASE WHEN (i < 0) THEN 'small' + WHEN (i = 0) THEN 'zero' + WHEN (i = 1) THEN 'one' + WHEN (i = 2) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query 24 schema +struct +-- !query 24 output +1 one + 2 two + 3 big + 4 big + + +-- !query 25 +SELECT '' AS `Five`, + CASE WHEN ((i < 0) or (i < 0)) THEN 'small' + WHEN ((i = 0) or (i = 0)) THEN 'zero' + WHEN ((i = 1) or (i = 1)) THEN 'one' + WHEN ((i = 2) or (i = 2)) THEN 'two' + ELSE 'big' + END AS `Category` + FROM CASE_TBL +-- !query 25 schema +struct +-- !query 25 output +big + big + one + two + + +-- !query 26 +SELECT * FROM CASE_TBL WHERE udf(COALESCE(f,i)) = 4 +-- !query 26 schema +struct +-- !query 26 output +4 NULL + + +-- !query 27 +SELECT * FROM CASE_TBL WHERE udf(NULLIF(f,i)) = 2 +-- !query 27 schema +struct +-- !query 27 output + + + +-- !query 28 +SELECT udf(COALESCE(a.f, b.i, b.j)) + FROM CASE_TBL a, CASE2_TBL b +-- !query 28 schema +struct +-- !query 28 output +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-30.3 +-6.0 +1.0 +1.0 +10.1 +10.1 +10.1 +10.1 +10.1 +10.1 +2.0 +2.0 +20.2 +20.2 +20.2 +20.2 +20.2 +20.2 +3.0 + + +-- !query 29 +SELECT * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(a.f, b.i, b.j)) = 2 +-- !query 29 schema +struct +-- !query 29 output +4 NULL 2 -2 +4 NULL 2 -4 + + +-- !query 30 +SELECT udf('') AS Five, NULLIF(a.i,b.i) AS `NULLIF(a.i,b.i)`, + NULLIF(b.i, 4) AS `NULLIF(b.i,4)` + FROM CASE_TBL a, CASE2_TBL b +-- !query 30 schema +struct +-- !query 30 output +1 2 + 1 2 + 1 3 + 1 NULL + 2 1 + 2 1 + 2 3 + 2 NULL + 3 1 + 3 1 + 3 2 + 3 2 + 3 NULL + 4 1 + 4 1 + 4 2 + 4 2 + 4 3 + 4 NULL + NULL 1 + NULL 1 + NULL 2 + NULL 2 + NULL 3 + + +-- !query 31 +SELECT '' AS `Two`, * + FROM CASE_TBL a, CASE2_TBL b + WHERE udf(COALESCE(f,b.i) = 2) +-- !query 31 schema +struct +-- !query 31 output +4 NULL 2 -2 + 4 NULL 2 -4 + + +-- !query 32 +SELECT CASE + (CASE vol('bar') + WHEN udf('foo') THEN 'it was foo!' + WHEN udf(vol(null)) THEN 'null input' + WHEN 'bar' THEN 'it was bar!' END + ) + WHEN udf('it was foo!') THEN 'foo recognized' + WHEN 'it was bar!' THEN udf('bar recognized') + ELSE 'unrecognized' END AS col +-- !query 32 schema +struct +-- !query 32 output +bar recognized + + +-- !query 33 +DROP TABLE CASE_TBL +-- !query 33 schema +struct<> +-- !query 33 output + + + +-- !query 34 +DROP TABLE CASE2_TBL +-- !query 34 schema +struct<> +-- !query 34 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out new file mode 100644 index 0000000000000..3d7c64054a6ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-count.sql.out @@ -0,0 +1,55 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (1, 1), (null, 2), (1, null), (null, null) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT + udf(count(*)), udf(count(1)), udf(count(null)), udf(count(a)), udf(count(b)), udf(count(a + b)), udf(count((a, b))) +FROM testData +-- !query 1 schema +struct +-- !query 1 output +7 7 0 5 5 4 7 + + +-- !query 2 +SELECT + udf(count(DISTINCT 1)), + udf(count(DISTINCT null)), + udf(count(DISTINCT a)), + udf(count(DISTINCT b)), + udf(count(DISTINCT (a + b))), + udf(count(DISTINCT (a, b))) +FROM testData +-- !query 2 schema +struct +-- !query 2 output +1 0 2 2 2 6 + + +-- !query 3 +SELECT udf(count(a, b)), udf(count(b, a)), udf(count(testData.*)) FROM testData +-- !query 3 schema +struct +-- !query 3 output +4 4 4 + + +-- !query 4 +SELECT + udf(count(DISTINCT a, b)), udf(count(DISTINCT b, a)), udf(count(DISTINCT *)), udf(count(DISTINCT testData.*)) +FROM testData +-- !query 4 schema +struct +-- !query 4 output +3 3 3 3 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out new file mode 100644 index 0000000000000..98d3ad37a8dfa --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-cross-join.sql.out @@ -0,0 +1,140 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 13 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 cross join nt2 +-- !query 2 schema +struct +-- !query 2 output +one 1 one 1 +one 1 one 5 +one 1 two 22 +three 3 one 1 +three 3 one 5 +three 3 two 22 +two 2 one 1 +two 2 one 5 +two 2 two 22 + + +-- !query 3 +SELECT * FROM nt1 cross join nt2 where udf(nt1.k) = udf(nt2.k) +-- !query 3 schema +struct +-- !query 3 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 4 +SELECT * FROM nt1 cross join nt2 on (udf(nt1.k) = udf(nt2.k)) +-- !query 4 schema +struct +-- !query 4 output +one 1 one 1 +one 1 one 5 +two 2 two 22 + + +-- !query 5 +SELECT * FROM nt1 cross join nt2 where udf(nt1.v1) = "1" and udf(nt2.v2) = "22" +-- !query 5 schema +struct +-- !query 5 output +one 1 two 22 + + +-- !query 6 +SELECT udf(a.key), udf(b.key) FROM +(SELECT udf(k) key FROM nt1 WHERE v1 < 2) a +CROSS JOIN +(SELECT udf(k) key FROM nt2 WHERE v2 = 22) b +-- !query 6 schema +struct +-- !query 6 output +one two + + +-- !query 7 +create temporary view A(a, va) as select * from nt1 +-- !query 7 schema +struct<> +-- !query 7 output + + + +-- !query 8 +create temporary view B(b, vb) as select * from nt1 +-- !query 8 schema +struct<> +-- !query 8 output + + + +-- !query 9 +create temporary view C(c, vc) as select * from nt1 +-- !query 9 schema +struct<> +-- !query 9 output + + + +-- !query 10 +create temporary view D(d, vd) as select * from nt1 +-- !query 10 schema +struct<> +-- !query 10 output + + + +-- !query 11 +select * from ((A join B on (udf(a) = udf(b))) cross join C) join D on (udf(a) = udf(d)) +-- !query 11 schema +struct +-- !query 11 output +one 1 one 1 one 1 one 1 +one 1 one 1 three 3 one 1 +one 1 one 1 two 2 one 1 +three 3 three 3 one 1 three 3 +three 3 three 3 three 3 three 3 +three 3 three 3 two 2 three 3 +two 2 two 2 one 1 two 2 +two 2 two 2 three 3 two 2 +two 2 two 2 two 2 two 2 + + +-- !query 12 +SELECT * FROM nt1 CROSS JOIN nt2 ON (udf(nt1.k) > udf(nt2.k)) +-- !query 12 schema +struct +-- !query 12 output +three 3 one 1 +three 3 one 5 +two 2 one 1 +two 2 one 5 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out new file mode 100644 index 0000000000000..b7bfad0e538ac --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except-all.sql.out @@ -0,0 +1,346 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 27 + + +-- !query 0 +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (0), (1), (2), (2), (2), (2), (3), (null), (null) AS tab1(c1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1), (2), (2), (3), (5), (5), (null) AS tab2(c1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW tab3 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (2, 3), + (2, 2) + AS tab3(k, v) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TEMPORARY VIEW tab4 AS SELECT * FROM VALUES + (1, 2), + (2, 3), + (2, 2), + (2, 2), + (2, 20) + AS tab4(k, v) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output +0 +2 +2 +NULL + + +-- !query 5 +SELECT udf(c1) FROM tab1 +MINUS ALL +SELECT udf(c1) FROM tab2 +-- !query 5 schema +struct +-- !query 5 output +0 +2 +2 +NULL + + +-- !query 6 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1) IS NOT NULL +-- !query 6 schema +struct +-- !query 6 output +0 +2 +2 +NULL +NULL + + +-- !query 7 +SELECT udf(c1) FROM tab1 WHERE udf(c1) > 5 +EXCEPT ALL +SELECT udf(c1) FROM tab2 +-- !query 7 schema +struct +-- !query 7 output + + + +-- !query 8 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT udf(c1) FROM tab2 WHERE udf(c1 > udf(6)) +-- !query 8 schema +struct +-- !query 8 output +0 +1 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 9 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT CAST(udf(1) AS BIGINT) +-- !query 9 schema +struct +-- !query 9 output +0 +2 +2 +2 +2 +3 +NULL +NULL + + +-- !query 10 +SELECT udf(c1) FROM tab1 +EXCEPT ALL +SELECT array(1) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 11 +SELECT udf(k), v FROM tab3 +EXCEPT ALL +SELECT k, udf(v) FROM tab4 +-- !query 11 schema +struct +-- !query 11 output +1 2 +1 3 + + +-- !query 12 +SELECT k, udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), v FROM tab3 +-- !query 12 schema +struct +-- !query 12 output +2 2 +2 20 + + +-- !query 13 +SELECT udf(k), udf(v) FROM tab4 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab3 +INTERSECT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 13 schema +struct +-- !query 13 output +2 2 +2 20 + + +-- !query 14 +SELECT udf(k), v FROM tab4 +EXCEPT ALL +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION ALL +SELECT udf(k), v FROM tab3 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query 15 schema +struct +-- !query 15 output +1 3 + + +-- !query 16 +SELECT k FROM tab3 +EXCEPT ALL +SELECT k, v FROM tab4 +-- !query 16 schema +struct<> +-- !query 16 output +org.apache.spark.sql.AnalysisException +ExceptAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 17 +SELECT udf(k), udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), udf(v) FROM tab4 +-- !query 17 schema +struct +-- !query 17 output +1 3 + + +-- !query 18 +SELECT udf(k), udf(v) FROM tab3 +MINUS ALL +SELECT k, udf(v) FROM tab4 +UNION +SELECT udf(k), udf(v) FROM tab3 +MINUS DISTINCT +SELECT k, udf(v) FROM tab4 +-- !query 18 schema +struct +-- !query 18 output +1 3 + + +-- !query 19 +SELECT k, udf(v) FROM tab3 +EXCEPT ALL +SELECT udf(k), v FROM tab4 +EXCEPT DISTINCT +SELECT k, udf(v) FROM tab3 +EXCEPT DISTINCT +SELECT udf(k), v FROM tab4 +-- !query 19 schema +struct +-- !query 19 output + + + +-- !query 20 +SELECT * +FROM (SELECT tab3.k, + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = tab4.k) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab3.k), + tab4.v + FROM tab3 + JOIN tab4 + ON tab3.k = udf(tab4.k)) +-- !query 20 schema +struct +-- !query 20 output + + + +-- !query 21 +SELECT * +FROM (SELECT udf(udf(tab3.k)), + udf(tab4.v) + FROM tab3 + JOIN tab4 + ON udf(udf(tab3.k)) = udf(tab4.k)) +EXCEPT ALL +SELECT * +FROM (SELECT udf(tab4.v) AS k, + udf(udf(tab3.k)) AS v + FROM tab3 + JOIN tab4 + ON udf(tab3.k) = udf(tab4.k)) +-- !query 21 schema +struct +-- !query 21 output +1 2 +1 2 +1 2 +2 20 +2 20 +2 3 +2 3 + + +-- !query 22 +SELECT udf(v) FROM tab3 GROUP BY v +EXCEPT ALL +SELECT udf(k) FROM tab4 GROUP BY k +-- !query 22 schema +struct +-- !query 22 output +3 + + +-- !query 23 +DROP VIEW IF EXISTS tab1 +-- !query 23 schema +struct<> +-- !query 23 output + + + +-- !query 24 +DROP VIEW IF EXISTS tab2 +-- !query 24 schema +struct<> +-- !query 24 output + + + +-- !query 25 +DROP VIEW IF EXISTS tab3 +-- !query 25 schema +struct<> +-- !query 25 output + + + +-- !query 26 +DROP VIEW IF EXISTS tab4 +-- !query 26 schema +struct<> +-- !query 26 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out new file mode 100644 index 0000000000000..cb8a4e84a95cf --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-except.sql.out @@ -0,0 +1,87 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +create temporary view t1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", NULL) + as t1(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view t2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5), + ("one", NULL), + (NULL, 5) + as t2(k, v) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT udf(k), udf(v) FROM t1 EXCEPT SELECT udf(k), udf(v) FROM t2 +-- !query 2 schema +struct +-- !query 2 output +three 3 +two 2 + + +-- !query 3 +SELECT * FROM t1 EXCEPT SELECT * FROM t1 where udf(v) <> 1 and v <> udf(2) +-- !query 3 schema +struct +-- !query 3 output +one 1 +one NULL +two 2 + + +-- !query 4 +SELECT * FROM t1 where udf(v) <> 1 and v <> udf(22) EXCEPT SELECT * FROM t1 where udf(v) <> 2 and v >= udf(3) +-- !query 4 schema +struct +-- !query 4 output +two 2 + + +-- !query 5 +SELECT t1.* FROM t1, t2 where t1.k = t2.k +EXCEPT +SELECT t1.* FROM t1, t2 where t1.k = t2.k and t1.k != udf('one') +-- !query 5 schema +struct +-- !query 5 output +one 1 +one NULL + + +-- !query 6 +SELECT * FROM t2 where v >= udf(1) and udf(v) <> 22 EXCEPT SELECT * FROM t1 +-- !query 6 schema +struct +-- !query 6 output +NULL 5 +one 5 + + +-- !query 7 +SELECT (SELECT min(udf(k)) FROM t2 WHERE t2.k = t1.k) min_t2 FROM t1 +MINUS +SELECT (SELECT udf(min(k)) FROM t2) abs_min_t2 FROM t1 WHERE t1.k = udf('one') +-- !query 7 schema +struct +-- !query 7 output +NULL +two diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out new file mode 100644 index 0000000000000..31e9e08e2ca8c --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-group-analytics.sql.out @@ -0,0 +1,394 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 29 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW testData AS SELECT * FROM VALUES +(1, 1), (1, 2), (2, 1), (2, 2), (3, 1), (3, 2) +AS testData(a, b) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT a + b, b, udf(SUM(a - b)) FROM testData GROUP BY a + b, b WITH CUBE +-- !query 1 schema +struct<(a + b):int,b:int,CAST(udf(cast(sum(cast((a - b) as bigint)) as string)) AS BIGINT):bigint> +-- !query 1 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query 2 +SELECT a, udf(b), SUM(b) FROM testData GROUP BY a, b WITH CUBE +-- !query 2 schema +struct +-- !query 2 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL 1 3 +NULL 2 6 +NULL NULL 9 + + +-- !query 3 +SELECT udf(a + b), b, SUM(a - b) FROM testData GROUP BY a + b, b WITH ROLLUP +-- !query 3 schema +struct +-- !query 3 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 4 +SELECT a, b, udf(SUM(b)) FROM testData GROUP BY a, b WITH ROLLUP +-- !query 4 schema +struct +-- !query 4 output +1 1 1 +1 2 2 +1 NULL 3 +2 1 1 +2 2 2 +2 NULL 3 +3 1 1 +3 2 2 +3 NULL 3 +NULL NULL 9 + + +-- !query 5 +CREATE OR REPLACE TEMPORARY VIEW courseSales AS SELECT * FROM VALUES +("dotNET", 2012, 10000), ("Java", 2012, 20000), ("dotNET", 2012, 5000), ("dotNET", 2013, 48000), ("Java", 2013, 30000) +AS courseSales(course, year, earnings) +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY ROLLUP(course, year) ORDER BY udf(course), year +-- !query 6 schema +struct +-- !query 6 output +NULL NULL 113000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query 7 +SELECT course, year, SUM(earnings) FROM courseSales GROUP BY CUBE(course, year) ORDER BY course, udf(year) +-- !query 7 schema +struct +-- !query 7 output +NULL NULL 113000 +NULL 2012 35000 +NULL 2013 78000 +Java NULL 50000 +Java 2012 20000 +Java 2013 30000 +dotNET NULL 63000 +dotNET 2012 15000 +dotNET 2013 48000 + + +-- !query 8 +SELECT course, udf(year), SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(course, year) +-- !query 8 schema +struct +-- !query 8 output +Java NULL 50000 +NULL 2012 35000 +NULL 2013 78000 +dotNET NULL 63000 + + +-- !query 9 +SELECT course, year, udf(SUM(earnings)) FROM courseSales GROUP BY course, year GROUPING SETS(course) +-- !query 9 schema +struct +-- !query 9 output +Java NULL 50000 +dotNET NULL 63000 + + +-- !query 10 +SELECT udf(course), year, SUM(earnings) FROM courseSales GROUP BY course, year GROUPING SETS(year) +-- !query 10 schema +struct +-- !query 10 output +NULL 2012 35000 +NULL 2013 78000 + + +-- !query 11 +SELECT course, udf(SUM(earnings)) AS sum FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY course, udf(sum) +-- !query 11 schema +struct +-- !query 11 output +NULL 113000 +Java 20000 +Java 30000 +Java 50000 +dotNET 5000 +dotNET 10000 +dotNET 48000 +dotNET 63000 + + +-- !query 12 +SELECT course, SUM(earnings) AS sum, GROUPING_ID(course, earnings) FROM courseSales +GROUP BY course, earnings GROUPING SETS((), (course), (course, earnings)) ORDER BY udf(course), sum +-- !query 12 schema +struct +-- !query 12 output +NULL 113000 3 +Java 20000 0 +Java 30000 0 +Java 50000 1 +dotNET 5000 0 +dotNET 10000 0 +dotNET 48000 0 +dotNET 63000 1 + + +-- !query 13 +SELECT udf(course), udf(year), GROUPING(course), GROUPING(year), GROUPING_ID(course, year) FROM courseSales +GROUP BY CUBE(course, year) +-- !query 13 schema +struct +-- !query 13 output +Java 2012 0 0 0 +Java 2013 0 0 0 +Java NULL 0 1 1 +NULL 2012 1 0 2 +NULL 2013 1 0 2 +NULL NULL 1 1 3 +dotNET 2012 0 0 0 +dotNET 2013 0 0 0 +dotNET NULL 0 1 1 + + +-- !query 14 +SELECT course, udf(year), GROUPING(course) FROM courseSales GROUP BY course, year +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +grouping() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 15 +SELECT course, udf(year), GROUPING_ID(course, year) FROM courseSales GROUP BY course, year +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 16 +SELECT course, year, grouping__id FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, course, udf(year) +-- !query 16 schema +struct +-- !query 16 output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query 17 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) +HAVING GROUPING(year) = 1 AND GROUPING_ID(course, year) > 0 ORDER BY course, udf(year) +-- !query 17 schema +struct +-- !query 17 output +NULL NULL +Java NULL +dotNET NULL + + +-- !query 18 +SELECT course, udf(year) FROM courseSales GROUP BY course, year HAVING GROUPING(course) > 0 +-- !query 18 schema +struct<> +-- !query 18 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 19 +SELECT course, udf(udf(year)) FROM courseSales GROUP BY course, year HAVING GROUPING_ID(course) > 0 +-- !query 19 schema +struct<> +-- !query 19 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 20 +SELECT udf(course), year FROM courseSales GROUP BY CUBE(course, year) HAVING grouping__id > 0 +-- !query 20 schema +struct +-- !query 20 output +Java NULL +NULL 2012 +NULL 2013 +NULL NULL +dotNET NULL + + +-- !query 21 +SELECT course, year, GROUPING(course), GROUPING(year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query 21 schema +struct +-- !query 21 output +Java 2012 0 0 +Java 2013 0 0 +dotNET 2012 0 0 +dotNET 2013 0 0 +Java NULL 0 1 +dotNET NULL 0 1 +NULL 2012 1 0 +NULL 2013 1 0 +NULL NULL 1 1 + + +-- !query 22 +SELECT course, year, GROUPING_ID(course, year) FROM courseSales GROUP BY CUBE(course, year) +ORDER BY GROUPING(course), GROUPING(year), course, udf(year) +-- !query 22 schema +struct +-- !query 22 output +Java 2012 0 +Java 2013 0 +dotNET 2012 0 +dotNET 2013 0 +Java NULL 1 +dotNET NULL 1 +NULL 2012 2 +NULL 2013 2 +NULL NULL 3 + + +-- !query 23 +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING(course) +-- !query 23 schema +struct<> +-- !query 23 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 24 +SELECT course, udf(year) FROM courseSales GROUP BY course, udf(year) ORDER BY GROUPING_ID(course) +-- !query 24 schema +struct<> +-- !query 24 output +org.apache.spark.sql.AnalysisException +grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup; + + +-- !query 25 +SELECT course, year FROM courseSales GROUP BY CUBE(course, year) ORDER BY grouping__id, udf(course), year +-- !query 25 schema +struct +-- !query 25 output +Java 2012 +Java 2013 +dotNET 2012 +dotNET 2013 +Java NULL +dotNET NULL +NULL 2012 +NULL 2013 +NULL NULL + + +-- !query 26 +SELECT udf(a + b) AS k1, udf(b) AS k2, SUM(a - b) FROM testData GROUP BY CUBE(k1, k2) +-- !query 26 schema +struct +-- !query 26 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL 1 3 +NULL 2 0 +NULL NULL 3 + + +-- !query 27 +SELECT udf(udf(a + b)) AS k, b, SUM(a - b) FROM testData GROUP BY ROLLUP(k, b) +-- !query 27 schema +struct +-- !query 27 output +2 1 0 +2 NULL 0 +3 1 1 +3 2 -1 +3 NULL 0 +4 1 2 +4 2 0 +4 NULL 2 +5 2 1 +5 NULL 1 +NULL NULL 3 + + +-- !query 28 +SELECT udf(a + b), udf(udf(b)) AS k, SUM(a - b) FROM testData GROUP BY a + b, k GROUPING SETS(k) +-- !query 28 schema +struct +-- !query 28 output +NULL 1 3 +NULL 2 0 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out new file mode 100644 index 0000000000000..1effcc8470e19 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-having.sql.out @@ -0,0 +1,49 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 5 + + +-- !query 0 +create temporary view hav as select * from values + ("one", 1), + ("two", 2), + ("three", 3), + ("one", 5) + as hav(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +SELECT udf(k) AS k, udf(sum(v)) FROM hav GROUP BY k HAVING udf(sum(v)) > 2 +-- !query 1 schema +struct +-- !query 1 output +one 6 +three 3 + + +-- !query 2 +SELECT udf(count(udf(k))) FROM hav GROUP BY v + 1 HAVING v + 1 = udf(2) +-- !query 2 schema +struct +-- !query 2 output +1 + + +-- !query 3 +SELECT udf(MIN(t.v)) FROM (SELECT * FROM hav WHERE v > 0) t HAVING(udf(COUNT(udf(1))) > 0) +-- !query 3 schema +struct +-- !query 3 output +1 + + +-- !query 4 +SELECT udf(a + b) FROM VALUES (1L, 2), (3L, 4) AS T(a, b) GROUP BY a + b HAVING a + b > udf(1) +-- !query 4 schema +struct +-- !query 4 output +3 +7 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out new file mode 100644 index 0000000000000..2cf24e50c80a5 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inline-table.sql.out @@ -0,0 +1,153 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 17 + + +-- !query 0 +select udf(col1), udf(col2) from values ("one", 1) +-- !query 0 schema +struct +-- !query 0 output +one 1 + + +-- !query 1 +select udf(col1), udf(udf(col2)) from values ("one", 1) as data +-- !query 1 schema +struct +-- !query 1 output +one 1 + + +-- !query 2 +select udf(a), b from values ("one", 1) as data(a, b) +-- !query 2 schema +struct +-- !query 2 output +one 1 + + +-- !query 3 +select udf(a) from values 1, 2, 3 as data(a) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +3 + + +-- !query 4 +select udf(a), b from values ("one", 1), ("two", 2), ("three", null) as data(a, b) +-- !query 4 schema +struct +-- !query 4 output +one 1 +three NULL +two 2 + + +-- !query 5 +select udf(a), b from values ("one", null), ("two", null) as data(a, b) +-- !query 5 schema +struct +-- !query 5 output +one NULL +two NULL + + +-- !query 6 +select udf(a), b from values ("one", 1), ("two", 2L) as data(a, b) +-- !query 6 schema +struct +-- !query 6 output +one 1 +two 2 + + +-- !query 7 +select udf(udf(a)), udf(b) from values ("one", 1 + 0), ("two", 1 + 3L) as data(a, b) +-- !query 7 schema +struct +-- !query 7 output +one 1 +two 4 + + +-- !query 8 +select udf(a), b from values ("one", array(0, 1)), ("two", array(2, 3)) as data(a, b) +-- !query 8 schema +struct> +-- !query 8 output +one [0,1] +two [2,3] + + +-- !query 9 +select udf(a), b from values ("one", 2.0), ("two", 3.0D) as data(a, b) +-- !query 9 schema +struct +-- !query 9 output +one 2.0 +two 3.0 + + +-- !query 10 +select udf(a), b from values ("one", rand(5)), ("two", 3.0D) as data(a, b) +-- !query 10 schema +struct<> +-- !query 10 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression rand(5) in inline table definition; line 1 pos 37 + + +-- !query 11 +select udf(a), udf(b) from values ("one", 2.0), ("two") as data(a, b) +-- !query 11 schema +struct<> +-- !query 11 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 1; line 1 pos 27 + + +-- !query 12 +select udf(a), udf(b) from values ("one", array(0, 1)), ("two", struct(1, 2)) as data(a, b) +-- !query 12 schema +struct<> +-- !query 12 output +org.apache.spark.sql.AnalysisException +incompatible types found in column b for inline table; line 1 pos 27 + + +-- !query 13 +select udf(a), udf(b) from values ("one"), ("two") as data(a, b) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +expected 2 columns but found 1 columns in row 0; line 1 pos 27 + + +-- !query 14 +select udf(a), udf(b) from values ("one", random_not_exist_func(1)), ("two", 2) as data(a, b) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Undefined function: 'random_not_exist_func'. This function is neither a registered temporary function nor a permanent function registered in the database 'default'.; line 1 pos 42 + + +-- !query 15 +select udf(a), udf(b) from values ("one", count(1)), ("two", 2) as data(a, b) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot evaluate expression count(1) in inline table definition; line 1 pos 42 + + +-- !query 16 +select udf(a), b from values (timestamp('1991-12-06 00:00:00.0'), array(timestamp('1991-12-06 01:00:00.0'), timestamp('1991-12-06 12:00:00.0'))) as data(a, b) +-- !query 16 schema +struct> +-- !query 16 output +1991-12-06 00:00:00 [1991-12-06 01:00:00.0,1991-12-06 12:00:00.0] diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out new file mode 100644 index 0000000000000..120f2d39f73dc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-inner-join.sql.out @@ -0,0 +1,67 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 7 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW t3 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +CREATE TEMPORARY VIEW t4 AS SELECT * FROM VALUES (1), (1) AS GROUPING(a) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE TEMPORARY VIEW ta AS +SELECT udf(a) AS a, udf('a') AS tag FROM t1 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t2 +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +CREATE TEMPORARY VIEW tb AS +SELECT udf(a) AS a, udf('a') AS tag FROM t3 +UNION ALL +SELECT udf(a) AS a, udf('b') AS tag FROM t4 +-- !query 5 schema +struct<> +-- !query 5 output + + + +-- !query 6 +SELECT tb.* FROM ta INNER JOIN tb ON ta.a = tb.a AND ta.tag = tb.tag +-- !query 6 schema +struct +-- !query 6 output +1 a +1 a +1 b +1 b diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out new file mode 100644 index 0000000000000..0cb82be2da228 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-intersect-all.sql.out @@ -0,0 +1,307 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 22 + + +-- !query 0 +CREATE TEMPORARY VIEW tab1 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (1, 3), + (1, 3), + (2, 3), + (null, null), + (null, null) + AS tab1(k, v) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW tab2 AS SELECT * FROM VALUES + (1, 2), + (1, 2), + (2, 3), + (3, 4), + (null, null), + (null, null) + AS tab2(k, v) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT k, udf(v) FROM tab2 +-- !query 2 schema +struct +-- !query 2 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 3 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab1 WHERE udf(k) = 1 +-- !query 3 schema +struct +-- !query 3 output +1 2 +1 2 +1 3 +1 3 + + +-- !query 4 +SELECT udf(k), udf(v) FROM tab1 WHERE k > udf(2) +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 WHERE udf(udf(k)) > 3 +-- !query 5 schema +struct +-- !query 5 output + + + +-- !query 6 +SELECT udf(k), v FROM tab1 +INTERSECT ALL +SELECT CAST(udf(1) AS BIGINT), CAST(udf(2) AS BIGINT) +-- !query 6 schema +struct +-- !query 6 output +1 2 + + +-- !query 7 +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT array(1), udf(2) +-- !query 7 schema +struct<> +-- !query 7 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the compatible column types. array <> int at the first column of the second table; + + +-- !query 8 +SELECT udf(k) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 8 schema +struct<> +-- !query 8 output +org.apache.spark.sql.AnalysisException +IntersectAll can only be performed on tables with the same number of columns, but the first table has 1 columns and the second table has 2 columns; + + +-- !query 9 +SELECT udf(k), v FROM tab2 +INTERSECT ALL +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(v) FROM tab2 +-- !query 9 schema +struct +-- !query 9 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 10 +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT k, udf(udf(v)) FROM tab1 +INTERSECT ALL +SELECT udf(k), v FROM tab2 +-- !query 10 schema +struct +-- !query 10 output +1 2 +1 2 +1 3 +2 3 +NULL NULL +NULL NULL + + +-- !query 11 +SELECT udf(k), udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +EXCEPT +SELECT k, udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query 11 schema +struct +-- !query 11 output +1 3 + + +-- !query 12 +( + ( + ( + SELECT udf(k), v FROM tab1 + EXCEPT + SELECT k, udf(v) FROM tab2 + ) + EXCEPT + SELECT udf(k), udf(v) FROM tab1 + ) + INTERSECT ALL + SELECT udf(k), udf(v) FROM tab2 +) +-- !query 12 schema +struct +-- !query 12 output + + + +-- !query 13 +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(udf(tab1.k)) = tab2.k) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(udf(tab2.k))) +-- !query 13 schema +struct +-- !query 13 output +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +1 2 +2 3 + + +-- !query 14 +SELECT * +FROM (SELECT udf(tab1.k), + udf(tab2.v) + FROM tab1 + JOIN tab2 + ON udf(tab1.k) = udf(tab2.k)) +INTERSECT ALL +SELECT * +FROM (SELECT udf(tab2.v) AS k, + udf(tab1.k) AS v + FROM tab1 + JOIN tab2 + ON tab1.k = udf(tab2.k)) +-- !query 14 schema +struct +-- !query 14 output + + + +-- !query 15 +SELECT udf(v) FROM tab1 GROUP BY v +INTERSECT ALL +SELECT udf(udf(k)) FROM tab2 GROUP BY k +-- !query 15 schema +struct +-- !query 15 output +2 +3 +NULL + + +-- !query 16 +SET spark.sql.legacy.setopsPrecedence.enabled= true +-- !query 16 schema +struct +-- !query 16 output +spark.sql.legacy.setopsPrecedence.enabled true + + +-- !query 17 +SELECT udf(k), v FROM tab1 +EXCEPT +SELECT k, udf(v) FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT ALL +SELECT udf(udf(k)), udf(v) FROM tab2 +-- !query 17 schema +struct +-- !query 17 output +1 2 +1 2 +2 3 +NULL NULL +NULL NULL + + +-- !query 18 +SELECT k, udf(v) FROM tab1 +EXCEPT +SELECT udf(k), v FROM tab2 +UNION ALL +SELECT udf(k), udf(v) FROM tab1 +INTERSECT +SELECT udf(k), udf(udf(v)) FROM tab2 +-- !query 18 schema +struct +-- !query 18 output +1 2 +2 3 +NULL NULL + + +-- !query 19 +SET spark.sql.legacy.setopsPrecedence.enabled = false +-- !query 19 schema +struct +-- !query 19 output +spark.sql.legacy.setopsPrecedence.enabled false + + +-- !query 20 +DROP VIEW IF EXISTS tab1 +-- !query 20 schema +struct<> +-- !query 20 output + + + +-- !query 21 +DROP VIEW IF EXISTS tab2 +-- !query 21 schema +struct<> +-- !query 21 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out new file mode 100644 index 0000000000000..e79d01fb14d60 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-join-empty-relation.sql.out @@ -0,0 +1,194 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 24 + + +-- !query 0 +CREATE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (1) AS GROUPING(a) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +CREATE TEMPORARY VIEW empty_table as SELECT a FROM t2 WHERE false +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT udf(t1.a), udf(empty_table.a) FROM t1 INNER JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query 3 schema +struct +-- !query 3 output + + + +-- !query 4 +SELECT udf(t1.a), udf(udf(empty_table.a)) FROM t1 CROSS JOIN empty_table ON (udf(udf(t1.a)) = udf(empty_table.a)) +-- !query 4 schema +struct +-- !query 4 output + + + +-- !query 5 +SELECT udf(udf(t1.a)), empty_table.a FROM t1 LEFT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 5 schema +struct +-- !query 5 output +1 NULL + + +-- !query 6 +SELECT udf(t1.a), udf(empty_table.a) FROM t1 RIGHT OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 6 schema +struct +-- !query 6 output + + + +-- !query 7 +SELECT udf(t1.a), empty_table.a FROM t1 FULL OUTER JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 7 schema +struct +-- !query 7 output +1 NULL + + +-- !query 8 +SELECT udf(udf(t1.a)) FROM t1 LEFT SEMI JOIN empty_table ON (udf(t1.a) = udf(udf(empty_table.a))) +-- !query 8 schema +struct +-- !query 8 output + + + +-- !query 9 +SELECT udf(t1.a) FROM t1 LEFT ANTI JOIN empty_table ON (udf(t1.a) = udf(empty_table.a)) +-- !query 9 schema +struct +-- !query 9 output +1 + + +-- !query 10 +SELECT udf(empty_table.a), udf(t1.a) FROM empty_table INNER JOIN t1 ON (udf(udf(empty_table.a)) = udf(t1.a)) +-- !query 10 schema +struct +-- !query 10 output + + + +-- !query 11 +SELECT udf(empty_table.a), udf(udf(t1.a)) FROM empty_table CROSS JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query 11 schema +struct +-- !query 11 output + + + +-- !query 12 +SELECT udf(udf(empty_table.a)), udf(t1.a) FROM empty_table LEFT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 12 schema +struct +-- !query 12 output + + + +-- !query 13 +SELECT empty_table.a, udf(t1.a) FROM empty_table RIGHT OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 13 schema +struct +-- !query 13 output +NULL 1 + + +-- !query 14 +SELECT empty_table.a, udf(udf(t1.a)) FROM empty_table FULL OUTER JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 14 schema +struct +-- !query 14 output +NULL 1 + + +-- !query 15 +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN t1 ON (udf(empty_table.a) = udf(udf(t1.a))) +-- !query 15 schema +struct +-- !query 15 output + + + +-- !query 16 +SELECT empty_table.a FROM empty_table LEFT ANTI JOIN t1 ON (udf(empty_table.a) = udf(t1.a)) +-- !query 16 schema +struct +-- !query 16 output + + + +-- !query 17 +SELECT udf(empty_table.a) FROM empty_table INNER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query 17 schema +struct +-- !query 17 output + + + +-- !query 18 +SELECT udf(udf(empty_table.a)) FROM empty_table CROSS JOIN empty_table AS empty_table2 ON (udf(udf(empty_table.a)) = udf(empty_table2.a)) +-- !query 18 schema +struct +-- !query 18 output + + + +-- !query 19 +SELECT udf(empty_table.a) FROM empty_table LEFT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query 19 schema +struct +-- !query 19 output + + + +-- !query 20 +SELECT udf(udf(empty_table.a)) FROM empty_table RIGHT OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(udf(empty_table2.a))) +-- !query 20 schema +struct +-- !query 20 output + + + +-- !query 21 +SELECT udf(empty_table.a) FROM empty_table FULL OUTER JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query 21 schema +struct +-- !query 21 output + + + +-- !query 22 +SELECT udf(udf(empty_table.a)) FROM empty_table LEFT SEMI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query 22 schema +struct +-- !query 22 output + + + +-- !query 23 +SELECT udf(empty_table.a) FROM empty_table LEFT ANTI JOIN empty_table AS empty_table2 ON (udf(empty_table.a) = udf(empty_table2.a)) +-- !query 23 schema +struct +-- !query 23 output + diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out new file mode 100644 index 0000000000000..950809ddcaf25 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-natural-join.sql.out @@ -0,0 +1,64 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +create temporary view nt1 as select * from values + ("one", 1), + ("two", 2), + ("three", 3) + as nt1(k, v1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view nt2 as select * from values + ("one", 1), + ("two", 22), + ("one", 5) + as nt2(k, v2) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT * FROM nt1 natural join nt2 where udf(k) = "one" +-- !query 2 schema +struct +-- !query 2 output +one 1 1 +one 1 5 + + +-- !query 3 +SELECT * FROM nt1 natural left join nt2 where k <> udf("") order by v1, v2 +-- !query 3 schema +struct +-- !query 3 output +one 1 1 +one 1 5 +two 2 22 +three 3 NULL + + +-- !query 4 +SELECT * FROM nt1 natural right join nt2 where udf(k) <> udf("") order by v1, v2 +-- !query 4 schema +struct +-- !query 4 output +one 1 1 +one 1 5 +two 2 22 + + +-- !query 5 +SELECT udf(count(*)) FROM nt1 natural full outer join nt2 +-- !query 5 schema +struct +-- !query 5 output +4 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out new file mode 100644 index 0000000000000..819f786070882 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-outer-join.sql.out @@ -0,0 +1,88 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(-234), (145), (367), (975), (298) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES +(-769, -244), (-800, -409), (940, 86), (-507, 304), (-367, 158) +as t2(int_col0, int_col1) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT + (udf(SUM(udf(COALESCE(t1.int_col1, t2.int_col0))))), + (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +FROM t1 +RIGHT JOIN t2 + ON udf(t2.int_col0) = udf(t1.int_col1) +GROUP BY udf(GREATEST(COALESCE(udf(t2.int_col1), 109), COALESCE(t1.int_col1, udf(-449)))), + COALESCE(t1.int_col1, t2.int_col0) +HAVING (udf(SUM(COALESCE(udf(t1.int_col1), udf(t2.int_col0))))) + > (udf(COALESCE(t1.int_col1, t2.int_col0)) * 2) +-- !query 2 schema +struct +-- !query 2 output +-367 -734 +-507 -1014 +-769 -1538 +-800 -1600 + + +-- !query 3 +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES (97) as t1(int_col1) +-- !query 3 schema +struct<> +-- !query 3 output + + + +-- !query 4 +CREATE OR REPLACE TEMPORARY VIEW t2 AS SELECT * FROM VALUES (0) as t2(int_col1) +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +set spark.sql.crossJoin.enabled = true +-- !query 5 schema +struct +-- !query 5 output +spark.sql.crossJoin.enabled true + + +-- !query 6 +SELECT * +FROM ( +SELECT + udf(COALESCE(udf(t2.int_col1), udf(t1.int_col1))) AS int_col + FROM t1 + LEFT JOIN t2 ON false +) t where (udf(t.int_col)) is not null +-- !query 6 schema +struct +-- !query 6 output +97 + + +-- !query 7 +set spark.sql.crossJoin.enabled = false +-- !query 7 schema +struct +-- !query 7 output +spark.sql.crossJoin.enabled false diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out new file mode 100644 index 0000000000000..cb9e4d736c9a0 --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-pivot.sql.out @@ -0,0 +1,460 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 30 + + +-- !query 0 +create temporary view courseSales as select * from values + ("dotNET", 2012, 10000), + ("Java", 2012, 20000), + ("dotNET", 2012, 5000), + ("dotNET", 2013, 48000), + ("Java", 2013, 30000) + as courseSales(course, year, earnings) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +create temporary view years as select * from values + (2012, 1), + (2013, 2) + as years(y, s) +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +create temporary view yearsWithComplexTypes as select * from values + (2012, array(1, 1), map('1', 1), struct(1, 'a')), + (2013, array(2, 2), map('2', 2), struct(2, 'b')) + as yearsWithComplexTypes(y, a, m, s) +-- !query 2 schema +struct<> +-- !query 2 output + + + +-- !query 3 +SELECT * FROM ( + SELECT udf(year), course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 3 schema +struct +-- !query 3 output +2012 15000 20000 +2013 48000 30000 + + +-- !query 4 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 4 schema +struct +-- !query 4 output +Java 20000 30000 +dotNET 15000 48000 + + +-- !query 5 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), udf(avg(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 5 schema +struct +-- !query 5 output +2012 15000 7500.0 20000 20000.0 +2013 48000 48000.0 30000 30000.0 + + +-- !query 6 +SELECT * FROM ( + SELECT udf(course) as course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR course IN ('dotNET', 'Java') +) +-- !query 6 schema +struct +-- !query 6 output +63000 50000 + + +-- !query 7 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(udf(earnings))), udf(min(year)) + FOR course IN ('dotNET', 'Java') +) +-- !query 7 schema +struct +-- !query 7 output +63000 2012 50000 2012 + + +-- !query 8 +SELECT * FROM ( + SELECT course, year, earnings, udf(s) as s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN (1, 2) +) +-- !query 8 schema +struct +-- !query 8 output +Java 2012 20000 NULL +Java 2013 NULL 30000 +dotNET 2012 15000 NULL +dotNET 2013 NULL 48000 + + +-- !query 9 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)), udf(min(s)) + FOR course IN ('dotNET', 'Java') +) +-- !query 9 schema +struct +-- !query 9 output +2012 15000 1 20000 1 +2013 48000 2 30000 2 + + +-- !query 10 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings * s)) + FOR course IN ('dotNET', 'Java') +) +-- !query 10 schema +struct +-- !query 10 output +2012 15000 20000 +2013 96000 60000 + + +-- !query 11 +SELECT 2012_s, 2013_s, 2012_a, 2013_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012, 2013) +) +-- !query 11 schema +struct<2012_s:bigint,2013_s:bigint,2012_a:double,2013_a:double,c:string> +-- !query 11 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 12 +SELECT firstYear_s, secondYear_s, firstYear_a, secondYear_a, c FROM ( + SELECT year y, course c, earnings e FROM courseSales +) +PIVOT ( + udf(sum(e)) s, udf(avg(e)) a + FOR y IN (2012 as firstYear, 2013 secondYear) +) +-- !query 12 schema +struct +-- !query 12 output +15000 48000 7500.0 48000.0 dotNET +20000 30000 20000.0 30000.0 Java + + +-- !query 13 +SELECT * FROM courseSales +PIVOT ( + udf(abs(earnings)) + FOR year IN (2012, 2013) +) +-- !query 13 schema +struct<> +-- !query 13 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but 'coursesales.`earnings`' did not appear in any aggregate function.; + + +-- !query 14 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)), year + FOR course IN ('dotNET', 'Java') +) +-- !query 14 schema +struct<> +-- !query 14 output +org.apache.spark.sql.AnalysisException +Aggregate expression required for pivot, but '__auto_generated_subquery_name.`year`' did not appear in any aggregate function.; + + +-- !query 15 +SELECT * FROM ( + SELECT course, earnings FROM courseSales +) +PIVOT ( + udf(sum(earnings)) + FOR year IN (2012, 2013) +) +-- !query 15 schema +struct<> +-- !query 15 output +org.apache.spark.sql.AnalysisException +cannot resolve '`year`' given input columns: [__auto_generated_subquery_name.course, __auto_generated_subquery_name.earnings]; line 4 pos 0 + + +-- !query 16 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + udf(ceil(udf(sum(earnings)))), avg(earnings) + 1 as a1 + FOR course IN ('dotNET', 'Java') +) +-- !query 16 schema +struct +-- !query 16 output +2012 15000 7501.0 20000 20001.0 +2013 48000 48001.0 30000 30001.0 + + +-- !query 17 +SELECT * FROM ( + SELECT year, course, earnings FROM courseSales +) +PIVOT ( + sum(udf(avg(earnings))) + FOR course IN ('dotNET', 'Java') +) +-- !query 17 schema +struct<> +-- !query 17 output +org.apache.spark.sql.AnalysisException +It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query.; + + +-- !query 18 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN (('dotNET', 2012), ('Java', 2013)) +) +-- !query 18 schema +struct +-- !query 18 output +1 15000 NULL +2 NULL 30000 + + +-- !query 19 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', 2) as c1, ('Java', 1) as c2) +) +-- !query 19 schema +struct +-- !query 19 output +2012 NULL 20000 +2013 48000 NULL + + +-- !query 20 +SELECT * FROM ( + SELECT course, year, earnings, s + FROM courseSales + JOIN years ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, year) IN ('dotNET', 'Java') +) +-- !query 20 schema +struct<> +-- !query 20 output +org.apache.spark.sql.AnalysisException +Invalid pivot value 'dotNET': value data type string does not match pivot column data type struct; + + +-- !query 21 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (s, 2013) +) +-- !query 21 schema +struct<> +-- !query 21 output +org.apache.spark.sql.AnalysisException +cannot resolve '`s`' given input columns: [coursesales.course, coursesales.earnings, coursesales.year]; line 4 pos 15 + + +-- !query 22 +SELECT * FROM courseSales +PIVOT ( + udf(sum(earnings)) + FOR year IN (course, 2013) +) +-- !query 22 schema +struct<> +-- !query 22 output +org.apache.spark.sql.AnalysisException +Literal expressions required for pivot values, found 'course#x'; + + +-- !query 23 +SELECT * FROM ( + SELECT earnings, year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR a IN (array(1, 1), array(2, 2)) +) +-- !query 23 schema +struct +-- !query 23 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 24 +SELECT * FROM ( + SELECT course, earnings, udf(year) as year, a + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, a) IN (('dotNET', array(1, 1)), ('Java', array(2, 2))) +) +-- !query 24 schema +struct +-- !query 24 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 25 +SELECT * FROM ( + SELECT earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR s IN ((1, 'a'), (2, 'b')) +) +-- !query 25 schema +struct +-- !query 25 output +2012 35000 NULL +2013 NULL 78000 + + +-- !query 26 +SELECT * FROM ( + SELECT course, earnings, year, s + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, s) IN (('dotNET', (1, 'a')), ('Java', (2, 'b'))) +) +-- !query 26 schema +struct +-- !query 26 output +2012 15000 NULL +2013 NULL 30000 + + +-- !query 27 +SELECT * FROM ( + SELECT earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR m IN (map('1', 1), map('2', 2)) +) +-- !query 27 schema +struct<> +-- !query 27 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'm#x'. Pivot columns must be comparable.; + + +-- !query 28 +SELECT * FROM ( + SELECT course, earnings, year, m + FROM courseSales + JOIN yearsWithComplexTypes ON year = y +) +PIVOT ( + udf(sum(earnings)) + FOR (course, m) IN (('dotNET', map('1', 1)), ('Java', map('2', 2))) +) +-- !query 28 schema +struct<> +-- !query 28 output +org.apache.spark.sql.AnalysisException +Invalid pivot column 'named_struct(course, course#x, m, m#x)'. Pivot columns must be comparable.; + + +-- !query 29 +SELECT * FROM ( + SELECT course, earnings, udf("a") as a, udf("z") as z, udf("b") as b, udf("y") as y, + udf("c") as c, udf("x") as x, udf("d") as d, udf("w") as w + FROM courseSales +) +PIVOT ( + udf(sum(Earnings)) + FOR Course IN ('dotNET', 'Java') +) +-- !query 29 schema +struct +-- !query 29 output +a z b y c x d w 63000 50000 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out new file mode 100644 index 0000000000000..7b2b5dbe578cc --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-special-values.sql.out @@ -0,0 +1,62 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 6 + + +-- !query 0 +SELECT udf(x) FROM (VALUES (1), (2), (NULL)) v(x) +-- !query 0 schema +struct +-- !query 0 output +1 +2 +NULL + + +-- !query 1 +SELECT udf(x) FROM (VALUES ('A'), ('B'), (NULL)) v(x) +-- !query 1 schema +struct +-- !query 1 output +A +B +NULL + + +-- !query 2 +SELECT udf(x) FROM (VALUES ('NaN'), ('1'), ('2')) v(x) +-- !query 2 schema +struct +-- !query 2 output +1 +2 +NaN + + +-- !query 3 +SELECT udf(x) FROM (VALUES ('Infinity'), ('1'), ('2')) v(x) +-- !query 3 schema +struct +-- !query 3 output +1 +2 +Infinity + + +-- !query 4 +SELECT udf(x) FROM (VALUES ('-Infinity'), ('1'), ('2')) v(x) +-- !query 4 schema +struct +-- !query 4 output +-Infinity +1 +2 + + +-- !query 5 +SELECT udf(x) FROM (VALUES 0.00000001, 0.00000002, 0.00000003) v(x) +-- !query 5 schema +struct +-- !query 5 output +0.00000001 +0.00000002 +0.00000003 diff --git a/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out new file mode 100644 index 0000000000000..e1747f466779f --- /dev/null +++ b/sql/core/src/test/resources/sql-tests/results/udf/udf-udaf.sql.out @@ -0,0 +1,72 @@ +-- Automatically generated by SQLQueryTestSuite +-- Number of queries: 8 + + +-- !query 0 +-- This test file was converted from udaf.sql. + +CREATE OR REPLACE TEMPORARY VIEW t1 AS SELECT * FROM VALUES +(1), (2), (3), (4) +as t1(int_col1) +-- !query 0 schema +struct<> +-- !query 0 output + + + +-- !query 1 +CREATE FUNCTION myDoubleAvg AS 'test.org.apache.spark.sql.MyDoubleAvg' +-- !query 1 schema +struct<> +-- !query 1 output + + + +-- !query 2 +SELECT default.myDoubleAvg(udf(int_col1)) as my_avg, udf(default.myDoubleAvg(udf(int_col1))) as my_avg2, udf(default.myDoubleAvg(int_col1)) as my_avg3 from t1 +-- !query 2 schema +struct +-- !query 2 output +102.5 102.5 102.5 + + +-- !query 3 +SELECT default.myDoubleAvg(udf(int_col1), udf(3)) as my_avg from t1 +-- !query 3 schema +struct<> +-- !query 3 output +org.apache.spark.sql.AnalysisException +Invalid number of arguments for function default.myDoubleAvg. Expected: 1; Found: 2; line 1 pos 7 + + +-- !query 4 +CREATE FUNCTION udaf1 AS 'test.non.existent.udaf' +-- !query 4 schema +struct<> +-- !query 4 output + + + +-- !query 5 +SELECT default.udaf1(udf(int_col1)) as udaf1, udf(default.udaf1(udf(int_col1))) as udaf2, udf(default.udaf1(int_col1)) as udaf3 from t1 +-- !query 5 schema +struct<> +-- !query 5 output +org.apache.spark.sql.AnalysisException +Can not load class 'test.non.existent.udaf' when registering the function 'default.udaf1', please make sure it is on the classpath; line 1 pos 94 + + +-- !query 6 +DROP FUNCTION myDoubleAvg +-- !query 6 schema +struct<> +-- !query 6 output + + + +-- !query 7 +DROP FUNCTION udaf1 +-- !query 7 schema +struct<> +-- !query 7 output + diff --git a/sql/core/src/test/resources/sql-tests/results/window.sql.out b/sql/core/src/test/resources/sql-tests/results/window.sql.out index 4afbcd62853dc..367dc4f513635 100644 --- a/sql/core/src/test/resources/sql-tests/results/window.sql.out +++ b/sql/core/src/test/resources/sql-tests/results/window.sql.out @@ -1,5 +1,5 @@ -- Automatically generated by SQLQueryTestSuite --- Number of queries: 22 +-- Number of queries: 23 -- !query 0 @@ -191,7 +191,7 @@ ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING) FROM testData ORDER BY cate, v struct<> -- !query 11 output org.apache.spark.sql.AnalysisException -cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not followes the lower bound 'unboundedfollowing$()'.; line 1 pos 33 +cannot resolve 'ROWS BETWEEN UNBOUNDED FOLLOWING AND 1 FOLLOWING' due to data type mismatch: Window frame upper bound '1' does not follow the lower bound 'unboundedfollowing$()'.; line 1 pos 33 -- !query 12 @@ -363,3 +363,20 @@ NULL a false true false false true false 1 b false true false false true false 2 b false true false false true false 3 b false true false false true false + + +-- !query 22 +SELECT cate, sum(val) OVER (w) +FROM testData +WHERE val is not null +WINDOW w AS (PARTITION BY cate ORDER BY val) +-- !query 22 schema +struct +-- !query 22 output +NULL 3 +a 2 +a 2 +a 4 +b 1 +b 3 +b 6 diff --git a/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/commits/0 b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/commits/0 new file mode 100644 index 0000000000000..9c1e3021c3ead --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/commits/0 @@ -0,0 +1,2 @@ +v1 +{"nextBatchWatermarkMs":0} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/metadata b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/metadata new file mode 100644 index 0000000000000..3071b0dfc550b --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/metadata @@ -0,0 +1 @@ +{"id":"09be7fb3-49d8-48a6-840d-e9c2ad92a898"} \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/offsets/0 b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/offsets/0 new file mode 100644 index 0000000000000..a0a567631fd14 --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/chk%252520%252525@%252523chk/offsets/0 @@ -0,0 +1,3 @@ +v1 +{"batchWatermarkMs":0,"batchTimestampMs":1549649384149,"conf":{"spark.sql.streaming.stateStore.providerClass":"org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider","spark.sql.streaming.flatMapGroupsWithState.stateFormatVersion":"2","spark.sql.streaming.multipleWatermarkPolicy":"min","spark.sql.streaming.aggregation.stateFormatVersion":"2","spark.sql.shuffle.partitions":"200"}} +0 \ No newline at end of file diff --git a/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output %@#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output %@#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet new file mode 100644 index 0000000000000..1b2919b25c381 Binary files /dev/null and b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output %@#output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet differ diff --git a/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output%20%25@%23output/_spark_metadata/0 b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output%20%25@%23output/_spark_metadata/0 new file mode 100644 index 0000000000000..79768f89d6eca --- /dev/null +++ b/sql/core/src/test/resources/structured-streaming/escaped-path-2.4.0/output%20%25@%23output/_spark_metadata/0 @@ -0,0 +1,2 @@ +v1 +{"path":"file://TEMPDIR/output%20%25@%23output/part-00000-97f675a2-bb82-4201-8245-05f3dae4c372-c000.snappy.parquet","size":404,"isDir":false,"modificationTime":1549649385000,"blockReplication":1,"blockSize":33554432,"action":"add"} diff --git a/sql/core/src/test/resources/test-data/bad_after_good.csv b/sql/core/src/test/resources/test-data/bad_after_good.csv new file mode 100644 index 0000000000000..4621a7d23714d --- /dev/null +++ b/sql/core/src/test/resources/test-data/bad_after_good.csv @@ -0,0 +1,2 @@ +"good record",1999-08-01 +"bad record",1999-088-01 diff --git a/sql/core/src/test/resources/test-data/cars-crlf.csv b/sql/core/src/test/resources/test-data/cars-crlf.csv new file mode 100644 index 0000000000000..d018d08ebc6fc --- /dev/null +++ b/sql/core/src/test/resources/test-data/cars-crlf.csv @@ -0,0 +1,7 @@ + +year,make,model,comment,blank +"2012","Tesla","S","No comment", + +1997,Ford,E350,"Go get one now they are going fast", +2015,Chevy,Volt + diff --git a/sql/core/src/test/resources/test-data/postgresql/agg.data b/sql/core/src/test/resources/test-data/postgresql/agg.data new file mode 100755 index 0000000000000..d92c7df4d0d9e --- /dev/null +++ b/sql/core/src/test/resources/test-data/postgresql/agg.data @@ -0,0 +1,4 @@ +56 7.8 +100 99.097 +0 0.09561 +42 324.78 diff --git a/sql/core/src/test/resources/test-data/postgresql/onek.data b/sql/core/src/test/resources/test-data/postgresql/onek.data new file mode 100755 index 0000000000000..1605bbec8d570 --- /dev/null +++ b/sql/core/src/test/resources/test-data/postgresql/onek.data @@ -0,0 +1,1000 @@ +147 0 1 3 7 7 7 47 147 147 147 14 15 RFAAAA AAAAAA AAAAxx +931 1 1 3 1 11 1 31 131 431 931 2 3 VJAAAA BAAAAA HHHHxx +714 2 0 2 4 14 4 14 114 214 714 8 9 MBAAAA CAAAAA OOOOxx +711 3 1 3 1 11 1 11 111 211 711 2 3 JBAAAA DAAAAA VVVVxx +883 4 1 3 3 3 3 83 83 383 883 6 7 ZHAAAA EAAAAA AAAAxx +439 5 1 3 9 19 9 39 39 439 439 18 19 XQAAAA FAAAAA HHHHxx +670 6 0 2 0 10 0 70 70 170 670 0 1 UZAAAA GAAAAA OOOOxx +543 7 1 3 3 3 3 43 143 43 543 6 7 XUAAAA HAAAAA VVVVxx +425 8 1 1 5 5 5 25 25 425 425 10 11 JQAAAA IAAAAA AAAAxx +800 9 0 0 0 0 0 0 0 300 800 0 1 UEAAAA JAAAAA HHHHxx +489 10 1 1 9 9 9 89 89 489 489 18 19 VSAAAA KAAAAA OOOOxx +494 11 0 2 4 14 4 94 94 494 494 8 9 ATAAAA LAAAAA VVVVxx +880 12 0 0 0 0 0 80 80 380 880 0 1 WHAAAA MAAAAA AAAAxx +611 13 1 3 1 11 1 11 11 111 611 2 3 NXAAAA NAAAAA HHHHxx +226 14 0 2 6 6 6 26 26 226 226 12 13 SIAAAA OAAAAA OOOOxx +774 15 0 2 4 14 4 74 174 274 774 8 9 UDAAAA PAAAAA VVVVxx +298 16 0 2 8 18 8 98 98 298 298 16 17 MLAAAA QAAAAA AAAAxx +682 17 0 2 2 2 2 82 82 182 682 4 5 GAAAAA RAAAAA HHHHxx +864 18 0 0 4 4 4 64 64 364 864 8 9 GHAAAA SAAAAA OOOOxx +183 19 1 3 3 3 3 83 183 183 183 6 7 BHAAAA TAAAAA VVVVxx +885 20 1 1 5 5 5 85 85 385 885 10 11 BIAAAA UAAAAA AAAAxx +997 21 1 1 7 17 7 97 197 497 997 14 15 JMAAAA VAAAAA HHHHxx +966 22 0 2 6 6 6 66 166 466 966 12 13 ELAAAA WAAAAA OOOOxx +389 23 1 1 9 9 9 89 189 389 389 18 19 ZOAAAA XAAAAA VVVVxx +846 24 0 2 6 6 6 46 46 346 846 12 13 OGAAAA YAAAAA AAAAxx +206 25 0 2 6 6 6 6 6 206 206 12 13 YHAAAA ZAAAAA HHHHxx +239 26 1 3 9 19 9 39 39 239 239 18 19 FJAAAA ABAAAA OOOOxx +365 27 1 1 5 5 5 65 165 365 365 10 11 BOAAAA BBAAAA VVVVxx +204 28 0 0 4 4 4 4 4 204 204 8 9 WHAAAA CBAAAA AAAAxx +690 29 0 2 0 10 0 90 90 190 690 0 1 OAAAAA DBAAAA HHHHxx +69 30 1 1 9 9 9 69 69 69 69 18 19 RCAAAA EBAAAA OOOOxx +358 31 0 2 8 18 8 58 158 358 358 16 17 UNAAAA FBAAAA VVVVxx +269 32 1 1 9 9 9 69 69 269 269 18 19 JKAAAA GBAAAA AAAAxx +663 33 1 3 3 3 3 63 63 163 663 6 7 NZAAAA HBAAAA HHHHxx +608 34 0 0 8 8 8 8 8 108 608 16 17 KXAAAA IBAAAA OOOOxx +398 35 0 2 8 18 8 98 198 398 398 16 17 IPAAAA JBAAAA VVVVxx +330 36 0 2 0 10 0 30 130 330 330 0 1 SMAAAA KBAAAA AAAAxx +529 37 1 1 9 9 9 29 129 29 529 18 19 JUAAAA LBAAAA HHHHxx +555 38 1 3 5 15 5 55 155 55 555 10 11 JVAAAA MBAAAA OOOOxx +746 39 0 2 6 6 6 46 146 246 746 12 13 SCAAAA NBAAAA VVVVxx +558 40 0 2 8 18 8 58 158 58 558 16 17 MVAAAA OBAAAA AAAAxx +574 41 0 2 4 14 4 74 174 74 574 8 9 CWAAAA PBAAAA HHHHxx +343 42 1 3 3 3 3 43 143 343 343 6 7 FNAAAA QBAAAA OOOOxx +120 43 0 0 0 0 0 20 120 120 120 0 1 QEAAAA RBAAAA VVVVxx +461 44 1 1 1 1 1 61 61 461 461 2 3 TRAAAA SBAAAA AAAAxx +754 45 0 2 4 14 4 54 154 254 754 8 9 ADAAAA TBAAAA HHHHxx +772 46 0 0 2 12 2 72 172 272 772 4 5 SDAAAA UBAAAA OOOOxx +749 47 1 1 9 9 9 49 149 249 749 18 19 VCAAAA VBAAAA VVVVxx +386 48 0 2 6 6 6 86 186 386 386 12 13 WOAAAA WBAAAA AAAAxx +9 49 1 1 9 9 9 9 9 9 9 18 19 JAAAAA XBAAAA HHHHxx +771 50 1 3 1 11 1 71 171 271 771 2 3 RDAAAA YBAAAA OOOOxx +470 51 0 2 0 10 0 70 70 470 470 0 1 CSAAAA ZBAAAA VVVVxx +238 52 0 2 8 18 8 38 38 238 238 16 17 EJAAAA ACAAAA AAAAxx +86 53 0 2 6 6 6 86 86 86 86 12 13 IDAAAA BCAAAA HHHHxx +56 54 0 0 6 16 6 56 56 56 56 12 13 ECAAAA CCAAAA OOOOxx +767 55 1 3 7 7 7 67 167 267 767 14 15 NDAAAA DCAAAA VVVVxx +363 56 1 3 3 3 3 63 163 363 363 6 7 ZNAAAA ECAAAA AAAAxx +655 57 1 3 5 15 5 55 55 155 655 10 11 FZAAAA FCAAAA HHHHxx +394 58 0 2 4 14 4 94 194 394 394 8 9 EPAAAA GCAAAA OOOOxx +223 59 1 3 3 3 3 23 23 223 223 6 7 PIAAAA HCAAAA VVVVxx +946 60 0 2 6 6 6 46 146 446 946 12 13 KKAAAA ICAAAA AAAAxx +863 61 1 3 3 3 3 63 63 363 863 6 7 FHAAAA JCAAAA HHHHxx +913 62 1 1 3 13 3 13 113 413 913 6 7 DJAAAA KCAAAA OOOOxx +737 63 1 1 7 17 7 37 137 237 737 14 15 JCAAAA LCAAAA VVVVxx +65 64 1 1 5 5 5 65 65 65 65 10 11 NCAAAA MCAAAA AAAAxx +251 65 1 3 1 11 1 51 51 251 251 2 3 RJAAAA NCAAAA HHHHxx +686 66 0 2 6 6 6 86 86 186 686 12 13 KAAAAA OCAAAA OOOOxx +971 67 1 3 1 11 1 71 171 471 971 2 3 JLAAAA PCAAAA VVVVxx +775 68 1 3 5 15 5 75 175 275 775 10 11 VDAAAA QCAAAA AAAAxx +577 69 1 1 7 17 7 77 177 77 577 14 15 FWAAAA RCAAAA HHHHxx +830 70 0 2 0 10 0 30 30 330 830 0 1 YFAAAA SCAAAA OOOOxx +787 71 1 3 7 7 7 87 187 287 787 14 15 HEAAAA TCAAAA VVVVxx +898 72 0 2 8 18 8 98 98 398 898 16 17 OIAAAA UCAAAA AAAAxx +588 73 0 0 8 8 8 88 188 88 588 16 17 QWAAAA VCAAAA HHHHxx +872 74 0 0 2 12 2 72 72 372 872 4 5 OHAAAA WCAAAA OOOOxx +397 75 1 1 7 17 7 97 197 397 397 14 15 HPAAAA XCAAAA VVVVxx +51 76 1 3 1 11 1 51 51 51 51 2 3 ZBAAAA YCAAAA AAAAxx +381 77 1 1 1 1 1 81 181 381 381 2 3 ROAAAA ZCAAAA HHHHxx +632 78 0 0 2 12 2 32 32 132 632 4 5 IYAAAA ADAAAA OOOOxx +31 79 1 3 1 11 1 31 31 31 31 2 3 FBAAAA BDAAAA VVVVxx +855 80 1 3 5 15 5 55 55 355 855 10 11 XGAAAA CDAAAA AAAAxx +699 81 1 3 9 19 9 99 99 199 699 18 19 XAAAAA DDAAAA HHHHxx +562 82 0 2 2 2 2 62 162 62 562 4 5 QVAAAA EDAAAA OOOOxx +681 83 1 1 1 1 1 81 81 181 681 2 3 FAAAAA FDAAAA VVVVxx +585 84 1 1 5 5 5 85 185 85 585 10 11 NWAAAA GDAAAA AAAAxx +35 85 1 3 5 15 5 35 35 35 35 10 11 JBAAAA HDAAAA HHHHxx +962 86 0 2 2 2 2 62 162 462 962 4 5 ALAAAA IDAAAA OOOOxx +282 87 0 2 2 2 2 82 82 282 282 4 5 WKAAAA JDAAAA VVVVxx +254 88 0 2 4 14 4 54 54 254 254 8 9 UJAAAA KDAAAA AAAAxx +514 89 0 2 4 14 4 14 114 14 514 8 9 UTAAAA LDAAAA HHHHxx +406 90 0 2 6 6 6 6 6 406 406 12 13 QPAAAA MDAAAA OOOOxx +544 91 0 0 4 4 4 44 144 44 544 8 9 YUAAAA NDAAAA VVVVxx +704 92 0 0 4 4 4 4 104 204 704 8 9 CBAAAA ODAAAA AAAAxx +948 93 0 0 8 8 8 48 148 448 948 16 17 MKAAAA PDAAAA HHHHxx +412 94 0 0 2 12 2 12 12 412 412 4 5 WPAAAA QDAAAA OOOOxx +200 95 0 0 0 0 0 0 0 200 200 0 1 SHAAAA RDAAAA VVVVxx +583 96 1 3 3 3 3 83 183 83 583 6 7 LWAAAA SDAAAA AAAAxx +486 97 0 2 6 6 6 86 86 486 486 12 13 SSAAAA TDAAAA HHHHxx +666 98 0 2 6 6 6 66 66 166 666 12 13 QZAAAA UDAAAA OOOOxx +436 99 0 0 6 16 6 36 36 436 436 12 13 UQAAAA VDAAAA VVVVxx +842 100 0 2 2 2 2 42 42 342 842 4 5 KGAAAA WDAAAA AAAAxx +99 101 1 3 9 19 9 99 99 99 99 18 19 VDAAAA XDAAAA HHHHxx +656 102 0 0 6 16 6 56 56 156 656 12 13 GZAAAA YDAAAA OOOOxx +673 103 1 1 3 13 3 73 73 173 673 6 7 XZAAAA ZDAAAA VVVVxx +371 104 1 3 1 11 1 71 171 371 371 2 3 HOAAAA AEAAAA AAAAxx +869 105 1 1 9 9 9 69 69 369 869 18 19 LHAAAA BEAAAA HHHHxx +569 106 1 1 9 9 9 69 169 69 569 18 19 XVAAAA CEAAAA OOOOxx +616 107 0 0 6 16 6 16 16 116 616 12 13 SXAAAA DEAAAA VVVVxx +612 108 0 0 2 12 2 12 12 112 612 4 5 OXAAAA EEAAAA AAAAxx +505 109 1 1 5 5 5 5 105 5 505 10 11 LTAAAA FEAAAA HHHHxx +922 110 0 2 2 2 2 22 122 422 922 4 5 MJAAAA GEAAAA OOOOxx +221 111 1 1 1 1 1 21 21 221 221 2 3 NIAAAA HEAAAA VVVVxx +388 112 0 0 8 8 8 88 188 388 388 16 17 YOAAAA IEAAAA AAAAxx +567 113 1 3 7 7 7 67 167 67 567 14 15 VVAAAA JEAAAA HHHHxx +58 114 0 2 8 18 8 58 58 58 58 16 17 GCAAAA KEAAAA OOOOxx +316 115 0 0 6 16 6 16 116 316 316 12 13 EMAAAA LEAAAA VVVVxx +659 116 1 3 9 19 9 59 59 159 659 18 19 JZAAAA MEAAAA AAAAxx +501 117 1 1 1 1 1 1 101 1 501 2 3 HTAAAA NEAAAA HHHHxx +815 118 1 3 5 15 5 15 15 315 815 10 11 JFAAAA OEAAAA OOOOxx +638 119 0 2 8 18 8 38 38 138 638 16 17 OYAAAA PEAAAA VVVVxx +696 120 0 0 6 16 6 96 96 196 696 12 13 UAAAAA QEAAAA AAAAxx +734 121 0 2 4 14 4 34 134 234 734 8 9 GCAAAA REAAAA HHHHxx +237 122 1 1 7 17 7 37 37 237 237 14 15 DJAAAA SEAAAA OOOOxx +816 123 0 0 6 16 6 16 16 316 816 12 13 KFAAAA TEAAAA VVVVxx +917 124 1 1 7 17 7 17 117 417 917 14 15 HJAAAA UEAAAA AAAAxx +844 125 0 0 4 4 4 44 44 344 844 8 9 MGAAAA VEAAAA HHHHxx +657 126 1 1 7 17 7 57 57 157 657 14 15 HZAAAA WEAAAA OOOOxx +952 127 0 0 2 12 2 52 152 452 952 4 5 QKAAAA XEAAAA VVVVxx +519 128 1 3 9 19 9 19 119 19 519 18 19 ZTAAAA YEAAAA AAAAxx +792 129 0 0 2 12 2 92 192 292 792 4 5 MEAAAA ZEAAAA HHHHxx +275 130 1 3 5 15 5 75 75 275 275 10 11 PKAAAA AFAAAA OOOOxx +319 131 1 3 9 19 9 19 119 319 319 18 19 HMAAAA BFAAAA VVVVxx +487 132 1 3 7 7 7 87 87 487 487 14 15 TSAAAA CFAAAA AAAAxx +945 133 1 1 5 5 5 45 145 445 945 10 11 JKAAAA DFAAAA HHHHxx +584 134 0 0 4 4 4 84 184 84 584 8 9 MWAAAA EFAAAA OOOOxx +765 135 1 1 5 5 5 65 165 265 765 10 11 LDAAAA FFAAAA VVVVxx +814 136 0 2 4 14 4 14 14 314 814 8 9 IFAAAA GFAAAA AAAAxx +359 137 1 3 9 19 9 59 159 359 359 18 19 VNAAAA HFAAAA HHHHxx +548 138 0 0 8 8 8 48 148 48 548 16 17 CVAAAA IFAAAA OOOOxx +811 139 1 3 1 11 1 11 11 311 811 2 3 FFAAAA JFAAAA VVVVxx +531 140 1 3 1 11 1 31 131 31 531 2 3 LUAAAA KFAAAA AAAAxx +104 141 0 0 4 4 4 4 104 104 104 8 9 AEAAAA LFAAAA HHHHxx +33 142 1 1 3 13 3 33 33 33 33 6 7 HBAAAA MFAAAA OOOOxx +404 143 0 0 4 4 4 4 4 404 404 8 9 OPAAAA NFAAAA VVVVxx +995 144 1 3 5 15 5 95 195 495 995 10 11 HMAAAA OFAAAA AAAAxx +408 145 0 0 8 8 8 8 8 408 408 16 17 SPAAAA PFAAAA HHHHxx +93 146 1 1 3 13 3 93 93 93 93 6 7 PDAAAA QFAAAA OOOOxx +794 147 0 2 4 14 4 94 194 294 794 8 9 OEAAAA RFAAAA VVVVxx +833 148 1 1 3 13 3 33 33 333 833 6 7 BGAAAA SFAAAA AAAAxx +615 149 1 3 5 15 5 15 15 115 615 10 11 RXAAAA TFAAAA HHHHxx +333 150 1 1 3 13 3 33 133 333 333 6 7 VMAAAA UFAAAA OOOOxx +357 151 1 1 7 17 7 57 157 357 357 14 15 TNAAAA VFAAAA VVVVxx +999 152 1 3 9 19 9 99 199 499 999 18 19 LMAAAA WFAAAA AAAAxx +515 153 1 3 5 15 5 15 115 15 515 10 11 VTAAAA XFAAAA HHHHxx +685 154 1 1 5 5 5 85 85 185 685 10 11 JAAAAA YFAAAA OOOOxx +692 155 0 0 2 12 2 92 92 192 692 4 5 QAAAAA ZFAAAA VVVVxx +627 156 1 3 7 7 7 27 27 127 627 14 15 DYAAAA AGAAAA AAAAxx +654 157 0 2 4 14 4 54 54 154 654 8 9 EZAAAA BGAAAA HHHHxx +115 158 1 3 5 15 5 15 115 115 115 10 11 LEAAAA CGAAAA OOOOxx +75 159 1 3 5 15 5 75 75 75 75 10 11 XCAAAA DGAAAA VVVVxx +14 160 0 2 4 14 4 14 14 14 14 8 9 OAAAAA EGAAAA AAAAxx +148 161 0 0 8 8 8 48 148 148 148 16 17 SFAAAA FGAAAA HHHHxx +201 162 1 1 1 1 1 1 1 201 201 2 3 THAAAA GGAAAA OOOOxx +862 163 0 2 2 2 2 62 62 362 862 4 5 EHAAAA HGAAAA VVVVxx +634 164 0 2 4 14 4 34 34 134 634 8 9 KYAAAA IGAAAA AAAAxx +589 165 1 1 9 9 9 89 189 89 589 18 19 RWAAAA JGAAAA HHHHxx +142 166 0 2 2 2 2 42 142 142 142 4 5 MFAAAA KGAAAA OOOOxx +545 167 1 1 5 5 5 45 145 45 545 10 11 ZUAAAA LGAAAA VVVVxx +983 168 1 3 3 3 3 83 183 483 983 6 7 VLAAAA MGAAAA AAAAxx +87 169 1 3 7 7 7 87 87 87 87 14 15 JDAAAA NGAAAA HHHHxx +335 170 1 3 5 15 5 35 135 335 335 10 11 XMAAAA OGAAAA OOOOxx +915 171 1 3 5 15 5 15 115 415 915 10 11 FJAAAA PGAAAA VVVVxx +286 172 0 2 6 6 6 86 86 286 286 12 13 ALAAAA QGAAAA AAAAxx +361 173 1 1 1 1 1 61 161 361 361 2 3 XNAAAA RGAAAA HHHHxx +97 174 1 1 7 17 7 97 97 97 97 14 15 TDAAAA SGAAAA OOOOxx +98 175 0 2 8 18 8 98 98 98 98 16 17 UDAAAA TGAAAA VVVVxx +377 176 1 1 7 17 7 77 177 377 377 14 15 NOAAAA UGAAAA AAAAxx +525 177 1 1 5 5 5 25 125 25 525 10 11 FUAAAA VGAAAA HHHHxx +448 178 0 0 8 8 8 48 48 448 448 16 17 GRAAAA WGAAAA OOOOxx +154 179 0 2 4 14 4 54 154 154 154 8 9 YFAAAA XGAAAA VVVVxx +866 180 0 2 6 6 6 66 66 366 866 12 13 IHAAAA YGAAAA AAAAxx +741 181 1 1 1 1 1 41 141 241 741 2 3 NCAAAA ZGAAAA HHHHxx +172 182 0 0 2 12 2 72 172 172 172 4 5 QGAAAA AHAAAA OOOOxx +843 183 1 3 3 3 3 43 43 343 843 6 7 LGAAAA BHAAAA VVVVxx +378 184 0 2 8 18 8 78 178 378 378 16 17 OOAAAA CHAAAA AAAAxx +804 185 0 0 4 4 4 4 4 304 804 8 9 YEAAAA DHAAAA HHHHxx +596 186 0 0 6 16 6 96 196 96 596 12 13 YWAAAA EHAAAA OOOOxx +77 187 1 1 7 17 7 77 77 77 77 14 15 ZCAAAA FHAAAA VVVVxx +572 188 0 0 2 12 2 72 172 72 572 4 5 AWAAAA GHAAAA AAAAxx +444 189 0 0 4 4 4 44 44 444 444 8 9 CRAAAA HHAAAA HHHHxx +47 190 1 3 7 7 7 47 47 47 47 14 15 VBAAAA IHAAAA OOOOxx +274 191 0 2 4 14 4 74 74 274 274 8 9 OKAAAA JHAAAA VVVVxx +40 192 0 0 0 0 0 40 40 40 40 0 1 OBAAAA KHAAAA AAAAxx +339 193 1 3 9 19 9 39 139 339 339 18 19 BNAAAA LHAAAA HHHHxx +13 194 1 1 3 13 3 13 13 13 13 6 7 NAAAAA MHAAAA OOOOxx +878 195 0 2 8 18 8 78 78 378 878 16 17 UHAAAA NHAAAA VVVVxx +53 196 1 1 3 13 3 53 53 53 53 6 7 BCAAAA OHAAAA AAAAxx +939 197 1 3 9 19 9 39 139 439 939 18 19 DKAAAA PHAAAA HHHHxx +928 198 0 0 8 8 8 28 128 428 928 16 17 SJAAAA QHAAAA OOOOxx +886 199 0 2 6 6 6 86 86 386 886 12 13 CIAAAA RHAAAA VVVVxx +267 200 1 3 7 7 7 67 67 267 267 14 15 HKAAAA SHAAAA AAAAxx +105 201 1 1 5 5 5 5 105 105 105 10 11 BEAAAA THAAAA HHHHxx +312 202 0 0 2 12 2 12 112 312 312 4 5 AMAAAA UHAAAA OOOOxx +552 203 0 0 2 12 2 52 152 52 552 4 5 GVAAAA VHAAAA VVVVxx +918 204 0 2 8 18 8 18 118 418 918 16 17 IJAAAA WHAAAA AAAAxx +114 205 0 2 4 14 4 14 114 114 114 8 9 KEAAAA XHAAAA HHHHxx +805 206 1 1 5 5 5 5 5 305 805 10 11 ZEAAAA YHAAAA OOOOxx +875 207 1 3 5 15 5 75 75 375 875 10 11 RHAAAA ZHAAAA VVVVxx +225 208 1 1 5 5 5 25 25 225 225 10 11 RIAAAA AIAAAA AAAAxx +495 209 1 3 5 15 5 95 95 495 495 10 11 BTAAAA BIAAAA HHHHxx +150 210 0 2 0 10 0 50 150 150 150 0 1 UFAAAA CIAAAA OOOOxx +759 211 1 3 9 19 9 59 159 259 759 18 19 FDAAAA DIAAAA VVVVxx +149 212 1 1 9 9 9 49 149 149 149 18 19 TFAAAA EIAAAA AAAAxx +480 213 0 0 0 0 0 80 80 480 480 0 1 MSAAAA FIAAAA HHHHxx +1 214 1 1 1 1 1 1 1 1 1 2 3 BAAAAA GIAAAA OOOOxx +557 215 1 1 7 17 7 57 157 57 557 14 15 LVAAAA HIAAAA VVVVxx +295 216 1 3 5 15 5 95 95 295 295 10 11 JLAAAA IIAAAA AAAAxx +854 217 0 2 4 14 4 54 54 354 854 8 9 WGAAAA JIAAAA HHHHxx +420 218 0 0 0 0 0 20 20 420 420 0 1 EQAAAA KIAAAA OOOOxx +414 219 0 2 4 14 4 14 14 414 414 8 9 YPAAAA LIAAAA VVVVxx +758 220 0 2 8 18 8 58 158 258 758 16 17 EDAAAA MIAAAA AAAAxx +879 221 1 3 9 19 9 79 79 379 879 18 19 VHAAAA NIAAAA HHHHxx +332 222 0 0 2 12 2 32 132 332 332 4 5 UMAAAA OIAAAA OOOOxx +78 223 0 2 8 18 8 78 78 78 78 16 17 ADAAAA PIAAAA VVVVxx +851 224 1 3 1 11 1 51 51 351 851 2 3 TGAAAA QIAAAA AAAAxx +592 225 0 0 2 12 2 92 192 92 592 4 5 UWAAAA RIAAAA HHHHxx +979 226 1 3 9 19 9 79 179 479 979 18 19 RLAAAA SIAAAA OOOOxx +989 227 1 1 9 9 9 89 189 489 989 18 19 BMAAAA TIAAAA VVVVxx +752 228 0 0 2 12 2 52 152 252 752 4 5 YCAAAA UIAAAA AAAAxx +214 229 0 2 4 14 4 14 14 214 214 8 9 GIAAAA VIAAAA HHHHxx +453 230 1 1 3 13 3 53 53 453 453 6 7 LRAAAA WIAAAA OOOOxx +540 231 0 0 0 0 0 40 140 40 540 0 1 UUAAAA XIAAAA VVVVxx +597 232 1 1 7 17 7 97 197 97 597 14 15 ZWAAAA YIAAAA AAAAxx +356 233 0 0 6 16 6 56 156 356 356 12 13 SNAAAA ZIAAAA HHHHxx +720 234 0 0 0 0 0 20 120 220 720 0 1 SBAAAA AJAAAA OOOOxx +367 235 1 3 7 7 7 67 167 367 367 14 15 DOAAAA BJAAAA VVVVxx +762 236 0 2 2 2 2 62 162 262 762 4 5 IDAAAA CJAAAA AAAAxx +986 237 0 2 6 6 6 86 186 486 986 12 13 YLAAAA DJAAAA HHHHxx +924 238 0 0 4 4 4 24 124 424 924 8 9 OJAAAA EJAAAA OOOOxx +779 239 1 3 9 19 9 79 179 279 779 18 19 ZDAAAA FJAAAA VVVVxx +684 240 0 0 4 4 4 84 84 184 684 8 9 IAAAAA GJAAAA AAAAxx +413 241 1 1 3 13 3 13 13 413 413 6 7 XPAAAA HJAAAA HHHHxx +479 242 1 3 9 19 9 79 79 479 479 18 19 LSAAAA IJAAAA OOOOxx +731 243 1 3 1 11 1 31 131 231 731 2 3 DCAAAA JJAAAA VVVVxx +409 244 1 1 9 9 9 9 9 409 409 18 19 TPAAAA KJAAAA AAAAxx +372 245 0 0 2 12 2 72 172 372 372 4 5 IOAAAA LJAAAA HHHHxx +139 246 1 3 9 19 9 39 139 139 139 18 19 JFAAAA MJAAAA OOOOxx +717 247 1 1 7 17 7 17 117 217 717 14 15 PBAAAA NJAAAA VVVVxx +539 248 1 3 9 19 9 39 139 39 539 18 19 TUAAAA OJAAAA AAAAxx +318 249 0 2 8 18 8 18 118 318 318 16 17 GMAAAA PJAAAA HHHHxx +208 250 0 0 8 8 8 8 8 208 208 16 17 AIAAAA QJAAAA OOOOxx +797 251 1 1 7 17 7 97 197 297 797 14 15 REAAAA RJAAAA VVVVxx +661 252 1 1 1 1 1 61 61 161 661 2 3 LZAAAA SJAAAA AAAAxx +50 253 0 2 0 10 0 50 50 50 50 0 1 YBAAAA TJAAAA HHHHxx +102 254 0 2 2 2 2 2 102 102 102 4 5 YDAAAA UJAAAA OOOOxx +484 255 0 0 4 4 4 84 84 484 484 8 9 QSAAAA VJAAAA VVVVxx +108 256 0 0 8 8 8 8 108 108 108 16 17 EEAAAA WJAAAA AAAAxx +140 257 0 0 0 0 0 40 140 140 140 0 1 KFAAAA XJAAAA HHHHxx +996 258 0 0 6 16 6 96 196 496 996 12 13 IMAAAA YJAAAA OOOOxx +687 259 1 3 7 7 7 87 87 187 687 14 15 LAAAAA ZJAAAA VVVVxx +241 260 1 1 1 1 1 41 41 241 241 2 3 HJAAAA AKAAAA AAAAxx +923 261 1 3 3 3 3 23 123 423 923 6 7 NJAAAA BKAAAA HHHHxx +500 262 0 0 0 0 0 0 100 0 500 0 1 GTAAAA CKAAAA OOOOxx +536 263 0 0 6 16 6 36 136 36 536 12 13 QUAAAA DKAAAA VVVVxx +490 264 0 2 0 10 0 90 90 490 490 0 1 WSAAAA EKAAAA AAAAxx +773 265 1 1 3 13 3 73 173 273 773 6 7 TDAAAA FKAAAA HHHHxx +19 266 1 3 9 19 9 19 19 19 19 18 19 TAAAAA GKAAAA OOOOxx +534 267 0 2 4 14 4 34 134 34 534 8 9 OUAAAA HKAAAA VVVVxx +941 268 1 1 1 1 1 41 141 441 941 2 3 FKAAAA IKAAAA AAAAxx +477 269 1 1 7 17 7 77 77 477 477 14 15 JSAAAA JKAAAA HHHHxx +173 270 1 1 3 13 3 73 173 173 173 6 7 RGAAAA KKAAAA OOOOxx +113 271 1 1 3 13 3 13 113 113 113 6 7 JEAAAA LKAAAA VVVVxx +526 272 0 2 6 6 6 26 126 26 526 12 13 GUAAAA MKAAAA AAAAxx +727 273 1 3 7 7 7 27 127 227 727 14 15 ZBAAAA NKAAAA HHHHxx +302 274 0 2 2 2 2 2 102 302 302 4 5 QLAAAA OKAAAA OOOOxx +789 275 1 1 9 9 9 89 189 289 789 18 19 JEAAAA PKAAAA VVVVxx +447 276 1 3 7 7 7 47 47 447 447 14 15 FRAAAA QKAAAA AAAAxx +884 277 0 0 4 4 4 84 84 384 884 8 9 AIAAAA RKAAAA HHHHxx +718 278 0 2 8 18 8 18 118 218 718 16 17 QBAAAA SKAAAA OOOOxx +818 279 0 2 8 18 8 18 18 318 818 16 17 MFAAAA TKAAAA VVVVxx +466 280 0 2 6 6 6 66 66 466 466 12 13 YRAAAA UKAAAA AAAAxx +131 281 1 3 1 11 1 31 131 131 131 2 3 BFAAAA VKAAAA HHHHxx +503 282 1 3 3 3 3 3 103 3 503 6 7 JTAAAA WKAAAA OOOOxx +364 283 0 0 4 4 4 64 164 364 364 8 9 AOAAAA XKAAAA VVVVxx +934 284 0 2 4 14 4 34 134 434 934 8 9 YJAAAA YKAAAA AAAAxx +542 285 0 2 2 2 2 42 142 42 542 4 5 WUAAAA ZKAAAA HHHHxx +146 286 0 2 6 6 6 46 146 146 146 12 13 QFAAAA ALAAAA OOOOxx +652 287 0 0 2 12 2 52 52 152 652 4 5 CZAAAA BLAAAA VVVVxx +566 288 0 2 6 6 6 66 166 66 566 12 13 UVAAAA CLAAAA AAAAxx +788 289 0 0 8 8 8 88 188 288 788 16 17 IEAAAA DLAAAA HHHHxx +168 290 0 0 8 8 8 68 168 168 168 16 17 MGAAAA ELAAAA OOOOxx +736 291 0 0 6 16 6 36 136 236 736 12 13 ICAAAA FLAAAA VVVVxx +795 292 1 3 5 15 5 95 195 295 795 10 11 PEAAAA GLAAAA AAAAxx +103 293 1 3 3 3 3 3 103 103 103 6 7 ZDAAAA HLAAAA HHHHxx +763 294 1 3 3 3 3 63 163 263 763 6 7 JDAAAA ILAAAA OOOOxx +256 295 0 0 6 16 6 56 56 256 256 12 13 WJAAAA JLAAAA VVVVxx +63 296 1 3 3 3 3 63 63 63 63 6 7 LCAAAA KLAAAA AAAAxx +702 297 0 2 2 2 2 2 102 202 702 4 5 ABAAAA LLAAAA HHHHxx +390 298 0 2 0 10 0 90 190 390 390 0 1 APAAAA MLAAAA OOOOxx +116 299 0 0 6 16 6 16 116 116 116 12 13 MEAAAA NLAAAA VVVVxx +354 300 0 2 4 14 4 54 154 354 354 8 9 QNAAAA OLAAAA AAAAxx +162 301 0 2 2 2 2 62 162 162 162 4 5 GGAAAA PLAAAA HHHHxx +71 302 1 3 1 11 1 71 71 71 71 2 3 TCAAAA QLAAAA OOOOxx +916 303 0 0 6 16 6 16 116 416 916 12 13 GJAAAA RLAAAA VVVVxx +565 304 1 1 5 5 5 65 165 65 565 10 11 TVAAAA SLAAAA AAAAxx +509 305 1 1 9 9 9 9 109 9 509 18 19 PTAAAA TLAAAA HHHHxx +20 306 0 0 0 0 0 20 20 20 20 0 1 UAAAAA ULAAAA OOOOxx +813 307 1 1 3 13 3 13 13 313 813 6 7 HFAAAA VLAAAA VVVVxx +80 308 0 0 0 0 0 80 80 80 80 0 1 CDAAAA WLAAAA AAAAxx +400 309 0 0 0 0 0 0 0 400 400 0 1 KPAAAA XLAAAA HHHHxx +888 310 0 0 8 8 8 88 88 388 888 16 17 EIAAAA YLAAAA OOOOxx +825 311 1 1 5 5 5 25 25 325 825 10 11 TFAAAA ZLAAAA VVVVxx +401 312 1 1 1 1 1 1 1 401 401 2 3 LPAAAA AMAAAA AAAAxx +158 313 0 2 8 18 8 58 158 158 158 16 17 CGAAAA BMAAAA HHHHxx +973 314 1 1 3 13 3 73 173 473 973 6 7 LLAAAA CMAAAA OOOOxx +324 315 0 0 4 4 4 24 124 324 324 8 9 MMAAAA DMAAAA VVVVxx +873 316 1 1 3 13 3 73 73 373 873 6 7 PHAAAA EMAAAA AAAAxx +676 317 0 0 6 16 6 76 76 176 676 12 13 AAAAAA FMAAAA HHHHxx +199 318 1 3 9 19 9 99 199 199 199 18 19 RHAAAA GMAAAA OOOOxx +304 319 0 0 4 4 4 4 104 304 304 8 9 SLAAAA HMAAAA VVVVxx +338 320 0 2 8 18 8 38 138 338 338 16 17 ANAAAA IMAAAA AAAAxx +743 321 1 3 3 3 3 43 143 243 743 6 7 PCAAAA JMAAAA HHHHxx +730 322 0 2 0 10 0 30 130 230 730 0 1 CCAAAA KMAAAA OOOOxx +130 323 0 2 0 10 0 30 130 130 130 0 1 AFAAAA LMAAAA VVVVxx +224 324 0 0 4 4 4 24 24 224 224 8 9 QIAAAA MMAAAA AAAAxx +216 325 0 0 6 16 6 16 16 216 216 12 13 IIAAAA NMAAAA HHHHxx +2 326 0 2 2 2 2 2 2 2 2 4 5 CAAAAA OMAAAA OOOOxx +836 327 0 0 6 16 6 36 36 336 836 12 13 EGAAAA PMAAAA VVVVxx +443 328 1 3 3 3 3 43 43 443 443 6 7 BRAAAA QMAAAA AAAAxx +777 329 1 1 7 17 7 77 177 277 777 14 15 XDAAAA RMAAAA HHHHxx +126 330 0 2 6 6 6 26 126 126 126 12 13 WEAAAA SMAAAA OOOOxx +117 331 1 1 7 17 7 17 117 117 117 14 15 NEAAAA TMAAAA VVVVxx +633 332 1 1 3 13 3 33 33 133 633 6 7 JYAAAA UMAAAA AAAAxx +310 333 0 2 0 10 0 10 110 310 310 0 1 YLAAAA VMAAAA HHHHxx +622 334 0 2 2 2 2 22 22 122 622 4 5 YXAAAA WMAAAA OOOOxx +268 335 0 0 8 8 8 68 68 268 268 16 17 IKAAAA XMAAAA VVVVxx +384 336 0 0 4 4 4 84 184 384 384 8 9 UOAAAA YMAAAA AAAAxx +460 337 0 0 0 0 0 60 60 460 460 0 1 SRAAAA ZMAAAA HHHHxx +475 338 1 3 5 15 5 75 75 475 475 10 11 HSAAAA ANAAAA OOOOxx +624 339 0 0 4 4 4 24 24 124 624 8 9 AYAAAA BNAAAA VVVVxx +826 340 0 2 6 6 6 26 26 326 826 12 13 UFAAAA CNAAAA AAAAxx +680 341 0 0 0 0 0 80 80 180 680 0 1 EAAAAA DNAAAA HHHHxx +306 342 0 2 6 6 6 6 106 306 306 12 13 ULAAAA ENAAAA OOOOxx +896 343 0 0 6 16 6 96 96 396 896 12 13 MIAAAA FNAAAA VVVVxx +30 344 0 2 0 10 0 30 30 30 30 0 1 EBAAAA GNAAAA AAAAxx +576 345 0 0 6 16 6 76 176 76 576 12 13 EWAAAA HNAAAA HHHHxx +551 346 1 3 1 11 1 51 151 51 551 2 3 FVAAAA INAAAA OOOOxx +639 347 1 3 9 19 9 39 39 139 639 18 19 PYAAAA JNAAAA VVVVxx +975 348 1 3 5 15 5 75 175 475 975 10 11 NLAAAA KNAAAA AAAAxx +882 349 0 2 2 2 2 82 82 382 882 4 5 YHAAAA LNAAAA HHHHxx +160 350 0 0 0 0 0 60 160 160 160 0 1 EGAAAA MNAAAA OOOOxx +522 351 0 2 2 2 2 22 122 22 522 4 5 CUAAAA NNAAAA VVVVxx +620 352 0 0 0 0 0 20 20 120 620 0 1 WXAAAA ONAAAA AAAAxx +719 353 1 3 9 19 9 19 119 219 719 18 19 RBAAAA PNAAAA HHHHxx +88 354 0 0 8 8 8 88 88 88 88 16 17 KDAAAA QNAAAA OOOOxx +614 355 0 2 4 14 4 14 14 114 614 8 9 QXAAAA RNAAAA VVVVxx +54 356 0 2 4 14 4 54 54 54 54 8 9 CCAAAA SNAAAA AAAAxx +209 357 1 1 9 9 9 9 9 209 209 18 19 BIAAAA TNAAAA HHHHxx +67 358 1 3 7 7 7 67 67 67 67 14 15 PCAAAA UNAAAA OOOOxx +809 359 1 1 9 9 9 9 9 309 809 18 19 DFAAAA VNAAAA VVVVxx +982 360 0 2 2 2 2 82 182 482 982 4 5 ULAAAA WNAAAA AAAAxx +817 361 1 1 7 17 7 17 17 317 817 14 15 LFAAAA XNAAAA HHHHxx +187 362 1 3 7 7 7 87 187 187 187 14 15 FHAAAA YNAAAA OOOOxx +992 363 0 0 2 12 2 92 192 492 992 4 5 EMAAAA ZNAAAA VVVVxx +580 364 0 0 0 0 0 80 180 80 580 0 1 IWAAAA AOAAAA AAAAxx +658 365 0 2 8 18 8 58 58 158 658 16 17 IZAAAA BOAAAA HHHHxx +222 366 0 2 2 2 2 22 22 222 222 4 5 OIAAAA COAAAA OOOOxx +667 367 1 3 7 7 7 67 67 167 667 14 15 RZAAAA DOAAAA VVVVxx +715 368 1 3 5 15 5 15 115 215 715 10 11 NBAAAA EOAAAA AAAAxx +990 369 0 2 0 10 0 90 190 490 990 0 1 CMAAAA FOAAAA HHHHxx +22 370 0 2 2 2 2 22 22 22 22 4 5 WAAAAA GOAAAA OOOOxx +362 371 0 2 2 2 2 62 162 362 362 4 5 YNAAAA HOAAAA VVVVxx +376 372 0 0 6 16 6 76 176 376 376 12 13 MOAAAA IOAAAA AAAAxx +246 373 0 2 6 6 6 46 46 246 246 12 13 MJAAAA JOAAAA HHHHxx +300 374 0 0 0 0 0 0 100 300 300 0 1 OLAAAA KOAAAA OOOOxx +231 375 1 3 1 11 1 31 31 231 231 2 3 XIAAAA LOAAAA VVVVxx +151 376 1 3 1 11 1 51 151 151 151 2 3 VFAAAA MOAAAA AAAAxx +29 377 1 1 9 9 9 29 29 29 29 18 19 DBAAAA NOAAAA HHHHxx +297 378 1 1 7 17 7 97 97 297 297 14 15 LLAAAA OOAAAA OOOOxx +403 379 1 3 3 3 3 3 3 403 403 6 7 NPAAAA POAAAA VVVVxx +716 380 0 0 6 16 6 16 116 216 716 12 13 OBAAAA QOAAAA AAAAxx +260 381 0 0 0 0 0 60 60 260 260 0 1 AKAAAA ROAAAA HHHHxx +170 382 0 2 0 10 0 70 170 170 170 0 1 OGAAAA SOAAAA OOOOxx +285 383 1 1 5 5 5 85 85 285 285 10 11 ZKAAAA TOAAAA VVVVxx +82 384 0 2 2 2 2 82 82 82 82 4 5 EDAAAA UOAAAA AAAAxx +958 385 0 2 8 18 8 58 158 458 958 16 17 WKAAAA VOAAAA HHHHxx +175 386 1 3 5 15 5 75 175 175 175 10 11 TGAAAA WOAAAA OOOOxx +671 387 1 3 1 11 1 71 71 171 671 2 3 VZAAAA XOAAAA VVVVxx +822 388 0 2 2 2 2 22 22 322 822 4 5 QFAAAA YOAAAA AAAAxx +573 389 1 1 3 13 3 73 173 73 573 6 7 BWAAAA ZOAAAA HHHHxx +723 390 1 3 3 3 3 23 123 223 723 6 7 VBAAAA APAAAA OOOOxx +195 391 1 3 5 15 5 95 195 195 195 10 11 NHAAAA BPAAAA VVVVxx +197 392 1 1 7 17 7 97 197 197 197 14 15 PHAAAA CPAAAA AAAAxx +755 393 1 3 5 15 5 55 155 255 755 10 11 BDAAAA DPAAAA HHHHxx +42 394 0 2 2 2 2 42 42 42 42 4 5 QBAAAA EPAAAA OOOOxx +897 395 1 1 7 17 7 97 97 397 897 14 15 NIAAAA FPAAAA VVVVxx +309 396 1 1 9 9 9 9 109 309 309 18 19 XLAAAA GPAAAA AAAAxx +724 397 0 0 4 4 4 24 124 224 724 8 9 WBAAAA HPAAAA HHHHxx +474 398 0 2 4 14 4 74 74 474 474 8 9 GSAAAA IPAAAA OOOOxx +345 399 1 1 5 5 5 45 145 345 345 10 11 HNAAAA JPAAAA VVVVxx +678 400 0 2 8 18 8 78 78 178 678 16 17 CAAAAA KPAAAA AAAAxx +757 401 1 1 7 17 7 57 157 257 757 14 15 DDAAAA LPAAAA HHHHxx +600 402 0 0 0 0 0 0 0 100 600 0 1 CXAAAA MPAAAA OOOOxx +184 403 0 0 4 4 4 84 184 184 184 8 9 CHAAAA NPAAAA VVVVxx +155 404 1 3 5 15 5 55 155 155 155 10 11 ZFAAAA OPAAAA AAAAxx +136 405 0 0 6 16 6 36 136 136 136 12 13 GFAAAA PPAAAA HHHHxx +889 406 1 1 9 9 9 89 89 389 889 18 19 FIAAAA QPAAAA OOOOxx +95 407 1 3 5 15 5 95 95 95 95 10 11 RDAAAA RPAAAA VVVVxx +549 408 1 1 9 9 9 49 149 49 549 18 19 DVAAAA SPAAAA AAAAxx +81 409 1 1 1 1 1 81 81 81 81 2 3 DDAAAA TPAAAA HHHHxx +679 410 1 3 9 19 9 79 79 179 679 18 19 DAAAAA UPAAAA OOOOxx +27 411 1 3 7 7 7 27 27 27 27 14 15 BBAAAA VPAAAA VVVVxx +748 412 0 0 8 8 8 48 148 248 748 16 17 UCAAAA WPAAAA AAAAxx +107 413 1 3 7 7 7 7 107 107 107 14 15 DEAAAA XPAAAA HHHHxx +870 414 0 2 0 10 0 70 70 370 870 0 1 MHAAAA YPAAAA OOOOxx +848 415 0 0 8 8 8 48 48 348 848 16 17 QGAAAA ZPAAAA VVVVxx +764 416 0 0 4 4 4 64 164 264 764 8 9 KDAAAA AQAAAA AAAAxx +535 417 1 3 5 15 5 35 135 35 535 10 11 PUAAAA BQAAAA HHHHxx +211 418 1 3 1 11 1 11 11 211 211 2 3 DIAAAA CQAAAA OOOOxx +625 419 1 1 5 5 5 25 25 125 625 10 11 BYAAAA DQAAAA VVVVxx +96 420 0 0 6 16 6 96 96 96 96 12 13 SDAAAA EQAAAA AAAAxx +828 421 0 0 8 8 8 28 28 328 828 16 17 WFAAAA FQAAAA HHHHxx +229 422 1 1 9 9 9 29 29 229 229 18 19 VIAAAA GQAAAA OOOOxx +602 423 0 2 2 2 2 2 2 102 602 4 5 EXAAAA HQAAAA VVVVxx +742 424 0 2 2 2 2 42 142 242 742 4 5 OCAAAA IQAAAA AAAAxx +451 425 1 3 1 11 1 51 51 451 451 2 3 JRAAAA JQAAAA HHHHxx +991 426 1 3 1 11 1 91 191 491 991 2 3 DMAAAA KQAAAA OOOOxx +301 427 1 1 1 1 1 1 101 301 301 2 3 PLAAAA LQAAAA VVVVxx +510 428 0 2 0 10 0 10 110 10 510 0 1 QTAAAA MQAAAA AAAAxx +299 429 1 3 9 19 9 99 99 299 299 18 19 NLAAAA NQAAAA HHHHxx +961 430 1 1 1 1 1 61 161 461 961 2 3 ZKAAAA OQAAAA OOOOxx +3 431 1 3 3 3 3 3 3 3 3 6 7 DAAAAA PQAAAA VVVVxx +106 432 0 2 6 6 6 6 106 106 106 12 13 CEAAAA QQAAAA AAAAxx +591 433 1 3 1 11 1 91 191 91 591 2 3 TWAAAA RQAAAA HHHHxx +700 434 0 0 0 0 0 0 100 200 700 0 1 YAAAAA SQAAAA OOOOxx +841 435 1 1 1 1 1 41 41 341 841 2 3 JGAAAA TQAAAA VVVVxx +829 436 1 1 9 9 9 29 29 329 829 18 19 XFAAAA UQAAAA AAAAxx +508 437 0 0 8 8 8 8 108 8 508 16 17 OTAAAA VQAAAA HHHHxx +750 438 0 2 0 10 0 50 150 250 750 0 1 WCAAAA WQAAAA OOOOxx +665 439 1 1 5 5 5 65 65 165 665 10 11 PZAAAA XQAAAA VVVVxx +157 440 1 1 7 17 7 57 157 157 157 14 15 BGAAAA YQAAAA AAAAxx +694 441 0 2 4 14 4 94 94 194 694 8 9 SAAAAA ZQAAAA HHHHxx +176 442 0 0 6 16 6 76 176 176 176 12 13 UGAAAA ARAAAA OOOOxx +950 443 0 2 0 10 0 50 150 450 950 0 1 OKAAAA BRAAAA VVVVxx +970 444 0 2 0 10 0 70 170 470 970 0 1 ILAAAA CRAAAA AAAAxx +496 445 0 0 6 16 6 96 96 496 496 12 13 CTAAAA DRAAAA HHHHxx +429 446 1 1 9 9 9 29 29 429 429 18 19 NQAAAA ERAAAA OOOOxx +907 447 1 3 7 7 7 7 107 407 907 14 15 XIAAAA FRAAAA VVVVxx +72 448 0 0 2 12 2 72 72 72 72 4 5 UCAAAA GRAAAA AAAAxx +186 449 0 2 6 6 6 86 186 186 186 12 13 EHAAAA HRAAAA HHHHxx +713 450 1 1 3 13 3 13 113 213 713 6 7 LBAAAA IRAAAA OOOOxx +432 451 0 0 2 12 2 32 32 432 432 4 5 QQAAAA JRAAAA VVVVxx +735 452 1 3 5 15 5 35 135 235 735 10 11 HCAAAA KRAAAA AAAAxx +516 453 0 0 6 16 6 16 116 16 516 12 13 WTAAAA LRAAAA HHHHxx +964 454 0 0 4 4 4 64 164 464 964 8 9 CLAAAA MRAAAA OOOOxx +840 455 0 0 0 0 0 40 40 340 840 0 1 IGAAAA NRAAAA VVVVxx +550 456 0 2 0 10 0 50 150 50 550 0 1 EVAAAA ORAAAA AAAAxx +360 457 0 0 0 0 0 60 160 360 360 0 1 WNAAAA PRAAAA HHHHxx +827 458 1 3 7 7 7 27 27 327 827 14 15 VFAAAA QRAAAA OOOOxx +959 459 1 3 9 19 9 59 159 459 959 18 19 XKAAAA RRAAAA VVVVxx +454 460 0 2 4 14 4 54 54 454 454 8 9 MRAAAA SRAAAA AAAAxx +819 461 1 3 9 19 9 19 19 319 819 18 19 NFAAAA TRAAAA HHHHxx +745 462 1 1 5 5 5 45 145 245 745 10 11 RCAAAA URAAAA OOOOxx +279 463 1 3 9 19 9 79 79 279 279 18 19 TKAAAA VRAAAA VVVVxx +426 464 0 2 6 6 6 26 26 426 426 12 13 KQAAAA WRAAAA AAAAxx +70 465 0 2 0 10 0 70 70 70 70 0 1 SCAAAA XRAAAA HHHHxx +637 466 1 1 7 17 7 37 37 137 637 14 15 NYAAAA YRAAAA OOOOxx +417 467 1 1 7 17 7 17 17 417 417 14 15 BQAAAA ZRAAAA VVVVxx +586 468 0 2 6 6 6 86 186 86 586 12 13 OWAAAA ASAAAA AAAAxx +314 469 0 2 4 14 4 14 114 314 314 8 9 CMAAAA BSAAAA HHHHxx +101 470 1 1 1 1 1 1 101 101 101 2 3 XDAAAA CSAAAA OOOOxx +205 471 1 1 5 5 5 5 5 205 205 10 11 XHAAAA DSAAAA VVVVxx +969 472 1 1 9 9 9 69 169 469 969 18 19 HLAAAA ESAAAA AAAAxx +217 473 1 1 7 17 7 17 17 217 217 14 15 JIAAAA FSAAAA HHHHxx +281 474 1 1 1 1 1 81 81 281 281 2 3 VKAAAA GSAAAA OOOOxx +984 475 0 0 4 4 4 84 184 484 984 8 9 WLAAAA HSAAAA VVVVxx +366 476 0 2 6 6 6 66 166 366 366 12 13 COAAAA ISAAAA AAAAxx +483 477 1 3 3 3 3 83 83 483 483 6 7 PSAAAA JSAAAA HHHHxx +838 478 0 2 8 18 8 38 38 338 838 16 17 GGAAAA KSAAAA OOOOxx +64 479 0 0 4 4 4 64 64 64 64 8 9 MCAAAA LSAAAA VVVVxx +981 480 1 1 1 1 1 81 181 481 981 2 3 TLAAAA MSAAAA AAAAxx +538 481 0 2 8 18 8 38 138 38 538 16 17 SUAAAA NSAAAA HHHHxx +39 482 1 3 9 19 9 39 39 39 39 18 19 NBAAAA OSAAAA OOOOxx +60 483 0 0 0 0 0 60 60 60 60 0 1 ICAAAA PSAAAA VVVVxx +874 484 0 2 4 14 4 74 74 374 874 8 9 QHAAAA QSAAAA AAAAxx +955 485 1 3 5 15 5 55 155 455 955 10 11 TKAAAA RSAAAA HHHHxx +347 486 1 3 7 7 7 47 147 347 347 14 15 JNAAAA SSAAAA OOOOxx +227 487 1 3 7 7 7 27 27 227 227 14 15 TIAAAA TSAAAA VVVVxx +44 488 0 0 4 4 4 44 44 44 44 8 9 SBAAAA USAAAA AAAAxx +446 489 0 2 6 6 6 46 46 446 446 12 13 ERAAAA VSAAAA HHHHxx +605 490 1 1 5 5 5 5 5 105 605 10 11 HXAAAA WSAAAA OOOOxx +570 491 0 2 0 10 0 70 170 70 570 0 1 YVAAAA XSAAAA VVVVxx +895 492 1 3 5 15 5 95 95 395 895 10 11 LIAAAA YSAAAA AAAAxx +760 493 0 0 0 0 0 60 160 260 760 0 1 GDAAAA ZSAAAA HHHHxx +428 494 0 0 8 8 8 28 28 428 428 16 17 MQAAAA ATAAAA OOOOxx +628 495 0 0 8 8 8 28 28 128 628 16 17 EYAAAA BTAAAA VVVVxx +933 496 1 1 3 13 3 33 133 433 933 6 7 XJAAAA CTAAAA AAAAxx +263 497 1 3 3 3 3 63 63 263 263 6 7 DKAAAA DTAAAA HHHHxx +729 498 1 1 9 9 9 29 129 229 729 18 19 BCAAAA ETAAAA OOOOxx +860 499 0 0 0 0 0 60 60 360 860 0 1 CHAAAA FTAAAA VVVVxx +76 500 0 0 6 16 6 76 76 76 76 12 13 YCAAAA GTAAAA AAAAxx +293 501 1 1 3 13 3 93 93 293 293 6 7 HLAAAA HTAAAA HHHHxx +296 502 0 0 6 16 6 96 96 296 296 12 13 KLAAAA ITAAAA OOOOxx +124 503 0 0 4 4 4 24 124 124 124 8 9 UEAAAA JTAAAA VVVVxx +568 504 0 0 8 8 8 68 168 68 568 16 17 WVAAAA KTAAAA AAAAxx +337 505 1 1 7 17 7 37 137 337 337 14 15 ZMAAAA LTAAAA HHHHxx +464 506 0 0 4 4 4 64 64 464 464 8 9 WRAAAA MTAAAA OOOOxx +582 507 0 2 2 2 2 82 182 82 582 4 5 KWAAAA NTAAAA VVVVxx +207 508 1 3 7 7 7 7 7 207 207 14 15 ZHAAAA OTAAAA AAAAxx +518 509 0 2 8 18 8 18 118 18 518 16 17 YTAAAA PTAAAA HHHHxx +513 510 1 1 3 13 3 13 113 13 513 6 7 TTAAAA QTAAAA OOOOxx +127 511 1 3 7 7 7 27 127 127 127 14 15 XEAAAA RTAAAA VVVVxx +396 512 0 0 6 16 6 96 196 396 396 12 13 GPAAAA STAAAA AAAAxx +781 513 1 1 1 1 1 81 181 281 781 2 3 BEAAAA TTAAAA HHHHxx +233 514 1 1 3 13 3 33 33 233 233 6 7 ZIAAAA UTAAAA OOOOxx +709 515 1 1 9 9 9 9 109 209 709 18 19 HBAAAA VTAAAA VVVVxx +325 516 1 1 5 5 5 25 125 325 325 10 11 NMAAAA WTAAAA AAAAxx +143 517 1 3 3 3 3 43 143 143 143 6 7 NFAAAA XTAAAA HHHHxx +824 518 0 0 4 4 4 24 24 324 824 8 9 SFAAAA YTAAAA OOOOxx +122 519 0 2 2 2 2 22 122 122 122 4 5 SEAAAA ZTAAAA VVVVxx +10 520 0 2 0 10 0 10 10 10 10 0 1 KAAAAA AUAAAA AAAAxx +41 521 1 1 1 1 1 41 41 41 41 2 3 PBAAAA BUAAAA HHHHxx +618 522 0 2 8 18 8 18 18 118 618 16 17 UXAAAA CUAAAA OOOOxx +161 523 1 1 1 1 1 61 161 161 161 2 3 FGAAAA DUAAAA VVVVxx +801 524 1 1 1 1 1 1 1 301 801 2 3 VEAAAA EUAAAA AAAAxx +768 525 0 0 8 8 8 68 168 268 768 16 17 ODAAAA FUAAAA HHHHxx +642 526 0 2 2 2 2 42 42 142 642 4 5 SYAAAA GUAAAA OOOOxx +803 527 1 3 3 3 3 3 3 303 803 6 7 XEAAAA HUAAAA VVVVxx +317 528 1 1 7 17 7 17 117 317 317 14 15 FMAAAA IUAAAA AAAAxx +938 529 0 2 8 18 8 38 138 438 938 16 17 CKAAAA JUAAAA HHHHxx +649 530 1 1 9 9 9 49 49 149 649 18 19 ZYAAAA KUAAAA OOOOxx +738 531 0 2 8 18 8 38 138 238 738 16 17 KCAAAA LUAAAA VVVVxx +344 532 0 0 4 4 4 44 144 344 344 8 9 GNAAAA MUAAAA AAAAxx +399 533 1 3 9 19 9 99 199 399 399 18 19 JPAAAA NUAAAA HHHHxx +609 534 1 1 9 9 9 9 9 109 609 18 19 LXAAAA OUAAAA OOOOxx +677 535 1 1 7 17 7 77 77 177 677 14 15 BAAAAA PUAAAA VVVVxx +478 536 0 2 8 18 8 78 78 478 478 16 17 KSAAAA QUAAAA AAAAxx +452 537 0 0 2 12 2 52 52 452 452 4 5 KRAAAA RUAAAA HHHHxx +261 538 1 1 1 1 1 61 61 261 261 2 3 BKAAAA SUAAAA OOOOxx +449 539 1 1 9 9 9 49 49 449 449 18 19 HRAAAA TUAAAA VVVVxx +433 540 1 1 3 13 3 33 33 433 433 6 7 RQAAAA UUAAAA AAAAxx +5 541 1 1 5 5 5 5 5 5 5 10 11 FAAAAA VUAAAA HHHHxx +664 542 0 0 4 4 4 64 64 164 664 8 9 OZAAAA WUAAAA OOOOxx +887 543 1 3 7 7 7 87 87 387 887 14 15 DIAAAA XUAAAA VVVVxx +546 544 0 2 6 6 6 46 146 46 546 12 13 AVAAAA YUAAAA AAAAxx +253 545 1 1 3 13 3 53 53 253 253 6 7 TJAAAA ZUAAAA HHHHxx +235 546 1 3 5 15 5 35 35 235 235 10 11 BJAAAA AVAAAA OOOOxx +258 547 0 2 8 18 8 58 58 258 258 16 17 YJAAAA BVAAAA VVVVxx +621 548 1 1 1 1 1 21 21 121 621 2 3 XXAAAA CVAAAA AAAAxx +998 549 0 2 8 18 8 98 198 498 998 16 17 KMAAAA DVAAAA HHHHxx +236 550 0 0 6 16 6 36 36 236 236 12 13 CJAAAA EVAAAA OOOOxx +537 551 1 1 7 17 7 37 137 37 537 14 15 RUAAAA FVAAAA VVVVxx +769 552 1 1 9 9 9 69 169 269 769 18 19 PDAAAA GVAAAA AAAAxx +921 553 1 1 1 1 1 21 121 421 921 2 3 LJAAAA HVAAAA HHHHxx +951 554 1 3 1 11 1 51 151 451 951 2 3 PKAAAA IVAAAA OOOOxx +240 555 0 0 0 0 0 40 40 240 240 0 1 GJAAAA JVAAAA VVVVxx +644 556 0 0 4 4 4 44 44 144 644 8 9 UYAAAA KVAAAA AAAAxx +352 557 0 0 2 12 2 52 152 352 352 4 5 ONAAAA LVAAAA HHHHxx +613 558 1 1 3 13 3 13 13 113 613 6 7 PXAAAA MVAAAA OOOOxx +784 559 0 0 4 4 4 84 184 284 784 8 9 EEAAAA NVAAAA VVVVxx +61 560 1 1 1 1 1 61 61 61 61 2 3 JCAAAA OVAAAA AAAAxx +144 561 0 0 4 4 4 44 144 144 144 8 9 OFAAAA PVAAAA HHHHxx +94 562 0 2 4 14 4 94 94 94 94 8 9 QDAAAA QVAAAA OOOOxx +270 563 0 2 0 10 0 70 70 270 270 0 1 KKAAAA RVAAAA VVVVxx +942 564 0 2 2 2 2 42 142 442 942 4 5 GKAAAA SVAAAA AAAAxx +756 565 0 0 6 16 6 56 156 256 756 12 13 CDAAAA TVAAAA HHHHxx +321 566 1 1 1 1 1 21 121 321 321 2 3 JMAAAA UVAAAA OOOOxx +36 567 0 0 6 16 6 36 36 36 36 12 13 KBAAAA VVAAAA VVVVxx +232 568 0 0 2 12 2 32 32 232 232 4 5 YIAAAA WVAAAA AAAAxx +430 569 0 2 0 10 0 30 30 430 430 0 1 OQAAAA XVAAAA HHHHxx +177 570 1 1 7 17 7 77 177 177 177 14 15 VGAAAA YVAAAA OOOOxx +220 571 0 0 0 0 0 20 20 220 220 0 1 MIAAAA ZVAAAA VVVVxx +109 572 1 1 9 9 9 9 109 109 109 18 19 FEAAAA AWAAAA AAAAxx +419 573 1 3 9 19 9 19 19 419 419 18 19 DQAAAA BWAAAA HHHHxx +135 574 1 3 5 15 5 35 135 135 135 10 11 FFAAAA CWAAAA OOOOxx +610 575 0 2 0 10 0 10 10 110 610 0 1 MXAAAA DWAAAA VVVVxx +956 576 0 0 6 16 6 56 156 456 956 12 13 UKAAAA EWAAAA AAAAxx +626 577 0 2 6 6 6 26 26 126 626 12 13 CYAAAA FWAAAA HHHHxx +375 578 1 3 5 15 5 75 175 375 375 10 11 LOAAAA GWAAAA OOOOxx +976 579 0 0 6 16 6 76 176 476 976 12 13 OLAAAA HWAAAA VVVVxx +152 580 0 0 2 12 2 52 152 152 152 4 5 WFAAAA IWAAAA AAAAxx +308 581 0 0 8 8 8 8 108 308 308 16 17 WLAAAA JWAAAA HHHHxx +445 582 1 1 5 5 5 45 45 445 445 10 11 DRAAAA KWAAAA OOOOxx +326 583 0 2 6 6 6 26 126 326 326 12 13 OMAAAA LWAAAA VVVVxx +422 584 0 2 2 2 2 22 22 422 422 4 5 GQAAAA MWAAAA AAAAxx +972 585 0 0 2 12 2 72 172 472 972 4 5 KLAAAA NWAAAA HHHHxx +45 586 1 1 5 5 5 45 45 45 45 10 11 TBAAAA OWAAAA OOOOxx +725 587 1 1 5 5 5 25 125 225 725 10 11 XBAAAA PWAAAA VVVVxx +753 588 1 1 3 13 3 53 153 253 753 6 7 ZCAAAA QWAAAA AAAAxx +493 589 1 1 3 13 3 93 93 493 493 6 7 ZSAAAA RWAAAA HHHHxx +601 590 1 1 1 1 1 1 1 101 601 2 3 DXAAAA SWAAAA OOOOxx +463 591 1 3 3 3 3 63 63 463 463 6 7 VRAAAA TWAAAA VVVVxx +303 592 1 3 3 3 3 3 103 303 303 6 7 RLAAAA UWAAAA AAAAxx +59 593 1 3 9 19 9 59 59 59 59 18 19 HCAAAA VWAAAA HHHHxx +595 594 1 3 5 15 5 95 195 95 595 10 11 XWAAAA WWAAAA OOOOxx +807 595 1 3 7 7 7 7 7 307 807 14 15 BFAAAA XWAAAA VVVVxx +424 596 0 0 4 4 4 24 24 424 424 8 9 IQAAAA YWAAAA AAAAxx +521 597 1 1 1 1 1 21 121 21 521 2 3 BUAAAA ZWAAAA HHHHxx +341 598 1 1 1 1 1 41 141 341 341 2 3 DNAAAA AXAAAA OOOOxx +571 599 1 3 1 11 1 71 171 71 571 2 3 ZVAAAA BXAAAA VVVVxx +165 600 1 1 5 5 5 65 165 165 165 10 11 JGAAAA CXAAAA AAAAxx +908 601 0 0 8 8 8 8 108 408 908 16 17 YIAAAA DXAAAA HHHHxx +351 602 1 3 1 11 1 51 151 351 351 2 3 NNAAAA EXAAAA OOOOxx +334 603 0 2 4 14 4 34 134 334 334 8 9 WMAAAA FXAAAA VVVVxx +636 604 0 0 6 16 6 36 36 136 636 12 13 MYAAAA GXAAAA AAAAxx +138 605 0 2 8 18 8 38 138 138 138 16 17 IFAAAA HXAAAA HHHHxx +438 606 0 2 8 18 8 38 38 438 438 16 17 WQAAAA IXAAAA OOOOxx +391 607 1 3 1 11 1 91 191 391 391 2 3 BPAAAA JXAAAA VVVVxx +395 608 1 3 5 15 5 95 195 395 395 10 11 FPAAAA KXAAAA AAAAxx +502 609 0 2 2 2 2 2 102 2 502 4 5 ITAAAA LXAAAA HHHHxx +85 610 1 1 5 5 5 85 85 85 85 10 11 HDAAAA MXAAAA OOOOxx +786 611 0 2 6 6 6 86 186 286 786 12 13 GEAAAA NXAAAA VVVVxx +619 612 1 3 9 19 9 19 19 119 619 18 19 VXAAAA OXAAAA AAAAxx +440 613 0 0 0 0 0 40 40 440 440 0 1 YQAAAA PXAAAA HHHHxx +949 614 1 1 9 9 9 49 149 449 949 18 19 NKAAAA QXAAAA OOOOxx +691 615 1 3 1 11 1 91 91 191 691 2 3 PAAAAA RXAAAA VVVVxx +348 616 0 0 8 8 8 48 148 348 348 16 17 KNAAAA SXAAAA AAAAxx +506 617 0 2 6 6 6 6 106 6 506 12 13 MTAAAA TXAAAA HHHHxx +192 618 0 0 2 12 2 92 192 192 192 4 5 KHAAAA UXAAAA OOOOxx +369 619 1 1 9 9 9 69 169 369 369 18 19 FOAAAA VXAAAA VVVVxx +311 620 1 3 1 11 1 11 111 311 311 2 3 ZLAAAA WXAAAA AAAAxx +273 621 1 1 3 13 3 73 73 273 273 6 7 NKAAAA XXAAAA HHHHxx +770 622 0 2 0 10 0 70 170 270 770 0 1 QDAAAA YXAAAA OOOOxx +191 623 1 3 1 11 1 91 191 191 191 2 3 JHAAAA ZXAAAA VVVVxx +90 624 0 2 0 10 0 90 90 90 90 0 1 MDAAAA AYAAAA AAAAxx +163 625 1 3 3 3 3 63 163 163 163 6 7 HGAAAA BYAAAA HHHHxx +350 626 0 2 0 10 0 50 150 350 350 0 1 MNAAAA CYAAAA OOOOxx +55 627 1 3 5 15 5 55 55 55 55 10 11 DCAAAA DYAAAA VVVVxx +488 628 0 0 8 8 8 88 88 488 488 16 17 USAAAA EYAAAA AAAAxx +215 629 1 3 5 15 5 15 15 215 215 10 11 HIAAAA FYAAAA HHHHxx +732 630 0 0 2 12 2 32 132 232 732 4 5 ECAAAA GYAAAA OOOOxx +688 631 0 0 8 8 8 88 88 188 688 16 17 MAAAAA HYAAAA VVVVxx +520 632 0 0 0 0 0 20 120 20 520 0 1 AUAAAA IYAAAA AAAAxx +62 633 0 2 2 2 2 62 62 62 62 4 5 KCAAAA JYAAAA HHHHxx +423 634 1 3 3 3 3 23 23 423 423 6 7 HQAAAA KYAAAA OOOOxx +242 635 0 2 2 2 2 42 42 242 242 4 5 IJAAAA LYAAAA VVVVxx +193 636 1 1 3 13 3 93 193 193 193 6 7 LHAAAA MYAAAA AAAAxx +648 637 0 0 8 8 8 48 48 148 648 16 17 YYAAAA NYAAAA HHHHxx +459 638 1 3 9 19 9 59 59 459 459 18 19 RRAAAA OYAAAA OOOOxx +196 639 0 0 6 16 6 96 196 196 196 12 13 OHAAAA PYAAAA VVVVxx +476 640 0 0 6 16 6 76 76 476 476 12 13 ISAAAA QYAAAA AAAAxx +903 641 1 3 3 3 3 3 103 403 903 6 7 TIAAAA RYAAAA HHHHxx +974 642 0 2 4 14 4 74 174 474 974 8 9 MLAAAA SYAAAA OOOOxx +603 643 1 3 3 3 3 3 3 103 603 6 7 FXAAAA TYAAAA VVVVxx +12 644 0 0 2 12 2 12 12 12 12 4 5 MAAAAA UYAAAA AAAAxx +599 645 1 3 9 19 9 99 199 99 599 18 19 BXAAAA VYAAAA HHHHxx +914 646 0 2 4 14 4 14 114 414 914 8 9 EJAAAA WYAAAA OOOOxx +7 647 1 3 7 7 7 7 7 7 7 14 15 HAAAAA XYAAAA VVVVxx +213 648 1 1 3 13 3 13 13 213 213 6 7 FIAAAA YYAAAA AAAAxx +174 649 0 2 4 14 4 74 174 174 174 8 9 SGAAAA ZYAAAA HHHHxx +392 650 0 0 2 12 2 92 192 392 392 4 5 CPAAAA AZAAAA OOOOxx +674 651 0 2 4 14 4 74 74 174 674 8 9 YZAAAA BZAAAA VVVVxx +650 652 0 2 0 10 0 50 50 150 650 0 1 AZAAAA CZAAAA AAAAxx +8 653 0 0 8 8 8 8 8 8 8 16 17 IAAAAA DZAAAA HHHHxx +492 654 0 0 2 12 2 92 92 492 492 4 5 YSAAAA EZAAAA OOOOxx +322 655 0 2 2 2 2 22 122 322 322 4 5 KMAAAA FZAAAA VVVVxx +315 656 1 3 5 15 5 15 115 315 315 10 11 DMAAAA GZAAAA AAAAxx +380 657 0 0 0 0 0 80 180 380 380 0 1 QOAAAA HZAAAA HHHHxx +353 658 1 1 3 13 3 53 153 353 353 6 7 PNAAAA IZAAAA OOOOxx +892 659 0 0 2 12 2 92 92 392 892 4 5 IIAAAA JZAAAA VVVVxx +932 660 0 0 2 12 2 32 132 432 932 4 5 WJAAAA KZAAAA AAAAxx +993 661 1 1 3 13 3 93 193 493 993 6 7 FMAAAA LZAAAA HHHHxx +859 662 1 3 9 19 9 59 59 359 859 18 19 BHAAAA MZAAAA OOOOxx +806 663 0 2 6 6 6 6 6 306 806 12 13 AFAAAA NZAAAA VVVVxx +145 664 1 1 5 5 5 45 145 145 145 10 11 PFAAAA OZAAAA AAAAxx +373 665 1 1 3 13 3 73 173 373 373 6 7 JOAAAA PZAAAA HHHHxx +418 666 0 2 8 18 8 18 18 418 418 16 17 CQAAAA QZAAAA OOOOxx +865 667 1 1 5 5 5 65 65 365 865 10 11 HHAAAA RZAAAA VVVVxx +462 668 0 2 2 2 2 62 62 462 462 4 5 URAAAA SZAAAA AAAAxx +24 669 0 0 4 4 4 24 24 24 24 8 9 YAAAAA TZAAAA HHHHxx +920 670 0 0 0 0 0 20 120 420 920 0 1 KJAAAA UZAAAA OOOOxx +672 671 0 0 2 12 2 72 72 172 672 4 5 WZAAAA VZAAAA VVVVxx +92 672 0 0 2 12 2 92 92 92 92 4 5 ODAAAA WZAAAA AAAAxx +721 673 1 1 1 1 1 21 121 221 721 2 3 TBAAAA XZAAAA HHHHxx +646 674 0 2 6 6 6 46 46 146 646 12 13 WYAAAA YZAAAA OOOOxx +910 675 0 2 0 10 0 10 110 410 910 0 1 AJAAAA ZZAAAA VVVVxx +909 676 1 1 9 9 9 9 109 409 909 18 19 ZIAAAA AABAAA AAAAxx +630 677 0 2 0 10 0 30 30 130 630 0 1 GYAAAA BABAAA HHHHxx +482 678 0 2 2 2 2 82 82 482 482 4 5 OSAAAA CABAAA OOOOxx +559 679 1 3 9 19 9 59 159 59 559 18 19 NVAAAA DABAAA VVVVxx +853 680 1 1 3 13 3 53 53 353 853 6 7 VGAAAA EABAAA AAAAxx +141 681 1 1 1 1 1 41 141 141 141 2 3 LFAAAA FABAAA HHHHxx +266 682 0 2 6 6 6 66 66 266 266 12 13 GKAAAA GABAAA OOOOxx +835 683 1 3 5 15 5 35 35 335 835 10 11 DGAAAA HABAAA VVVVxx +164 684 0 0 4 4 4 64 164 164 164 8 9 IGAAAA IABAAA AAAAxx +629 685 1 1 9 9 9 29 29 129 629 18 19 FYAAAA JABAAA HHHHxx +203 686 1 3 3 3 3 3 3 203 203 6 7 VHAAAA KABAAA OOOOxx +411 687 1 3 1 11 1 11 11 411 411 2 3 VPAAAA LABAAA VVVVxx +930 688 0 2 0 10 0 30 130 430 930 0 1 UJAAAA MABAAA AAAAxx +435 689 1 3 5 15 5 35 35 435 435 10 11 TQAAAA NABAAA HHHHxx +563 690 1 3 3 3 3 63 163 63 563 6 7 RVAAAA OABAAA OOOOxx +960 691 0 0 0 0 0 60 160 460 960 0 1 YKAAAA PABAAA VVVVxx +733 692 1 1 3 13 3 33 133 233 733 6 7 FCAAAA QABAAA AAAAxx +967 693 1 3 7 7 7 67 167 467 967 14 15 FLAAAA RABAAA HHHHxx +668 694 0 0 8 8 8 68 68 168 668 16 17 SZAAAA SABAAA OOOOxx +994 695 0 2 4 14 4 94 194 494 994 8 9 GMAAAA TABAAA VVVVxx +129 696 1 1 9 9 9 29 129 129 129 18 19 ZEAAAA UABAAA AAAAxx +954 697 0 2 4 14 4 54 154 454 954 8 9 SKAAAA VABAAA HHHHxx +68 698 0 0 8 8 8 68 68 68 68 16 17 QCAAAA WABAAA OOOOxx +79 699 1 3 9 19 9 79 79 79 79 18 19 BDAAAA XABAAA VVVVxx +121 700 1 1 1 1 1 21 121 121 121 2 3 REAAAA YABAAA AAAAxx +740 701 0 0 0 0 0 40 140 240 740 0 1 MCAAAA ZABAAA HHHHxx +902 702 0 2 2 2 2 2 102 402 902 4 5 SIAAAA ABBAAA OOOOxx +695 703 1 3 5 15 5 95 95 195 695 10 11 TAAAAA BBBAAA VVVVxx +455 704 1 3 5 15 5 55 55 455 455 10 11 NRAAAA CBBAAA AAAAxx +89 705 1 1 9 9 9 89 89 89 89 18 19 LDAAAA DBBAAA HHHHxx +893 706 1 1 3 13 3 93 93 393 893 6 7 JIAAAA EBBAAA OOOOxx +202 707 0 2 2 2 2 2 2 202 202 4 5 UHAAAA FBBAAA VVVVxx +132 708 0 0 2 12 2 32 132 132 132 4 5 CFAAAA GBBAAA AAAAxx +782 709 0 2 2 2 2 82 182 282 782 4 5 CEAAAA HBBAAA HHHHxx +512 710 0 0 2 12 2 12 112 12 512 4 5 STAAAA IBBAAA OOOOxx +857 711 1 1 7 17 7 57 57 357 857 14 15 ZGAAAA JBBAAA VVVVxx +248 712 0 0 8 8 8 48 48 248 248 16 17 OJAAAA KBBAAA AAAAxx +858 713 0 2 8 18 8 58 58 358 858 16 17 AHAAAA LBBAAA HHHHxx +527 714 1 3 7 7 7 27 127 27 527 14 15 HUAAAA MBBAAA OOOOxx +450 715 0 2 0 10 0 50 50 450 450 0 1 IRAAAA NBBAAA VVVVxx +712 716 0 0 2 12 2 12 112 212 712 4 5 KBAAAA OBBAAA AAAAxx +153 717 1 1 3 13 3 53 153 153 153 6 7 XFAAAA PBBAAA HHHHxx +587 718 1 3 7 7 7 87 187 87 587 14 15 PWAAAA QBBAAA OOOOxx +593 719 1 1 3 13 3 93 193 93 593 6 7 VWAAAA RBBAAA VVVVxx +249 720 1 1 9 9 9 49 49 249 249 18 19 PJAAAA SBBAAA AAAAxx +128 721 0 0 8 8 8 28 128 128 128 16 17 YEAAAA TBBAAA HHHHxx +675 722 1 3 5 15 5 75 75 175 675 10 11 ZZAAAA UBBAAA OOOOxx +929 723 1 1 9 9 9 29 129 429 929 18 19 TJAAAA VBBAAA VVVVxx +156 724 0 0 6 16 6 56 156 156 156 12 13 AGAAAA WBBAAA AAAAxx +415 725 1 3 5 15 5 15 15 415 415 10 11 ZPAAAA XBBAAA HHHHxx +28 726 0 0 8 8 8 28 28 28 28 16 17 CBAAAA YBBAAA OOOOxx +18 727 0 2 8 18 8 18 18 18 18 16 17 SAAAAA ZBBAAA VVVVxx +255 728 1 3 5 15 5 55 55 255 255 10 11 VJAAAA ACBAAA AAAAxx +793 729 1 1 3 13 3 93 193 293 793 6 7 NEAAAA BCBAAA HHHHxx +554 730 0 2 4 14 4 54 154 54 554 8 9 IVAAAA CCBAAA OOOOxx +467 731 1 3 7 7 7 67 67 467 467 14 15 ZRAAAA DCBAAA VVVVxx +410 732 0 2 0 10 0 10 10 410 410 0 1 UPAAAA ECBAAA AAAAxx +651 733 1 3 1 11 1 51 51 151 651 2 3 BZAAAA FCBAAA HHHHxx +287 734 1 3 7 7 7 87 87 287 287 14 15 BLAAAA GCBAAA OOOOxx +640 735 0 0 0 0 0 40 40 140 640 0 1 QYAAAA HCBAAA VVVVxx +245 736 1 1 5 5 5 45 45 245 245 10 11 LJAAAA ICBAAA AAAAxx +21 737 1 1 1 1 1 21 21 21 21 2 3 VAAAAA JCBAAA HHHHxx +83 738 1 3 3 3 3 83 83 83 83 6 7 FDAAAA KCBAAA OOOOxx +228 739 0 0 8 8 8 28 28 228 228 16 17 UIAAAA LCBAAA VVVVxx +323 740 1 3 3 3 3 23 123 323 323 6 7 LMAAAA MCBAAA AAAAxx +594 741 0 2 4 14 4 94 194 94 594 8 9 WWAAAA NCBAAA HHHHxx +528 742 0 0 8 8 8 28 128 28 528 16 17 IUAAAA OCBAAA OOOOxx +276 743 0 0 6 16 6 76 76 276 276 12 13 QKAAAA PCBAAA VVVVxx +598 744 0 2 8 18 8 98 198 98 598 16 17 AXAAAA QCBAAA AAAAxx +635 745 1 3 5 15 5 35 35 135 635 10 11 LYAAAA RCBAAA HHHHxx +868 746 0 0 8 8 8 68 68 368 868 16 17 KHAAAA SCBAAA OOOOxx +290 747 0 2 0 10 0 90 90 290 290 0 1 ELAAAA TCBAAA VVVVxx +468 748 0 0 8 8 8 68 68 468 468 16 17 ASAAAA UCBAAA AAAAxx +689 749 1 1 9 9 9 89 89 189 689 18 19 NAAAAA VCBAAA HHHHxx +799 750 1 3 9 19 9 99 199 299 799 18 19 TEAAAA WCBAAA OOOOxx +210 751 0 2 0 10 0 10 10 210 210 0 1 CIAAAA XCBAAA VVVVxx +346 752 0 2 6 6 6 46 146 346 346 12 13 INAAAA YCBAAA AAAAxx +957 753 1 1 7 17 7 57 157 457 957 14 15 VKAAAA ZCBAAA HHHHxx +905 754 1 1 5 5 5 5 105 405 905 10 11 VIAAAA ADBAAA OOOOxx +523 755 1 3 3 3 3 23 123 23 523 6 7 DUAAAA BDBAAA VVVVxx +899 756 1 3 9 19 9 99 99 399 899 18 19 PIAAAA CDBAAA AAAAxx +867 757 1 3 7 7 7 67 67 367 867 14 15 JHAAAA DDBAAA HHHHxx +11 758 1 3 1 11 1 11 11 11 11 2 3 LAAAAA EDBAAA OOOOxx +320 759 0 0 0 0 0 20 120 320 320 0 1 IMAAAA FDBAAA VVVVxx +766 760 0 2 6 6 6 66 166 266 766 12 13 MDAAAA GDBAAA AAAAxx +84 761 0 0 4 4 4 84 84 84 84 8 9 GDAAAA HDBAAA HHHHxx +507 762 1 3 7 7 7 7 107 7 507 14 15 NTAAAA IDBAAA OOOOxx +471 763 1 3 1 11 1 71 71 471 471 2 3 DSAAAA JDBAAA VVVVxx +517 764 1 1 7 17 7 17 117 17 517 14 15 XTAAAA KDBAAA AAAAxx +234 765 0 2 4 14 4 34 34 234 234 8 9 AJAAAA LDBAAA HHHHxx +988 766 0 0 8 8 8 88 188 488 988 16 17 AMAAAA MDBAAA OOOOxx +473 767 1 1 3 13 3 73 73 473 473 6 7 FSAAAA NDBAAA VVVVxx +66 768 0 2 6 6 6 66 66 66 66 12 13 OCAAAA ODBAAA AAAAxx +530 769 0 2 0 10 0 30 130 30 530 0 1 KUAAAA PDBAAA HHHHxx +834 770 0 2 4 14 4 34 34 334 834 8 9 CGAAAA QDBAAA OOOOxx +894 771 0 2 4 14 4 94 94 394 894 8 9 KIAAAA RDBAAA VVVVxx +481 772 1 1 1 1 1 81 81 481 481 2 3 NSAAAA SDBAAA AAAAxx +280 773 0 0 0 0 0 80 80 280 280 0 1 UKAAAA TDBAAA HHHHxx +705 774 1 1 5 5 5 5 105 205 705 10 11 DBAAAA UDBAAA OOOOxx +218 775 0 2 8 18 8 18 18 218 218 16 17 KIAAAA VDBAAA VVVVxx +560 776 0 0 0 0 0 60 160 60 560 0 1 OVAAAA WDBAAA AAAAxx +123 777 1 3 3 3 3 23 123 123 123 6 7 TEAAAA XDBAAA HHHHxx +289 778 1 1 9 9 9 89 89 289 289 18 19 DLAAAA YDBAAA OOOOxx +189 779 1 1 9 9 9 89 189 189 189 18 19 HHAAAA ZDBAAA VVVVxx +541 780 1 1 1 1 1 41 141 41 541 2 3 VUAAAA AEBAAA AAAAxx +876 781 0 0 6 16 6 76 76 376 876 12 13 SHAAAA BEBAAA HHHHxx +504 782 0 0 4 4 4 4 104 4 504 8 9 KTAAAA CEBAAA OOOOxx +643 783 1 3 3 3 3 43 43 143 643 6 7 TYAAAA DEBAAA VVVVxx +73 784 1 1 3 13 3 73 73 73 73 6 7 VCAAAA EEBAAA AAAAxx +465 785 1 1 5 5 5 65 65 465 465 10 11 XRAAAA FEBAAA HHHHxx +861 786 1 1 1 1 1 61 61 361 861 2 3 DHAAAA GEBAAA OOOOxx +355 787 1 3 5 15 5 55 155 355 355 10 11 RNAAAA HEBAAA VVVVxx +441 788 1 1 1 1 1 41 41 441 441 2 3 ZQAAAA IEBAAA AAAAxx +219 789 1 3 9 19 9 19 19 219 219 18 19 LIAAAA JEBAAA HHHHxx +839 790 1 3 9 19 9 39 39 339 839 18 19 HGAAAA KEBAAA OOOOxx +271 791 1 3 1 11 1 71 71 271 271 2 3 LKAAAA LEBAAA VVVVxx +212 792 0 0 2 12 2 12 12 212 212 4 5 EIAAAA MEBAAA AAAAxx +904 793 0 0 4 4 4 4 104 404 904 8 9 UIAAAA NEBAAA HHHHxx +244 794 0 0 4 4 4 44 44 244 244 8 9 KJAAAA OEBAAA OOOOxx +751 795 1 3 1 11 1 51 151 251 751 2 3 XCAAAA PEBAAA VVVVxx +944 796 0 0 4 4 4 44 144 444 944 8 9 IKAAAA QEBAAA AAAAxx +305 797 1 1 5 5 5 5 105 305 305 10 11 TLAAAA REBAAA HHHHxx +617 798 1 1 7 17 7 17 17 117 617 14 15 TXAAAA SEBAAA OOOOxx +891 799 1 3 1 11 1 91 91 391 891 2 3 HIAAAA TEBAAA VVVVxx +653 800 1 1 3 13 3 53 53 153 653 6 7 DZAAAA UEBAAA AAAAxx +845 801 1 1 5 5 5 45 45 345 845 10 11 NGAAAA VEBAAA HHHHxx +936 802 0 0 6 16 6 36 136 436 936 12 13 AKAAAA WEBAAA OOOOxx +91 803 1 3 1 11 1 91 91 91 91 2 3 NDAAAA XEBAAA VVVVxx +442 804 0 2 2 2 2 42 42 442 442 4 5 ARAAAA YEBAAA AAAAxx +498 805 0 2 8 18 8 98 98 498 498 16 17 ETAAAA ZEBAAA HHHHxx +987 806 1 3 7 7 7 87 187 487 987 14 15 ZLAAAA AFBAAA OOOOxx +194 807 0 2 4 14 4 94 194 194 194 8 9 MHAAAA BFBAAA VVVVxx +927 808 1 3 7 7 7 27 127 427 927 14 15 RJAAAA CFBAAA AAAAxx +607 809 1 3 7 7 7 7 7 107 607 14 15 JXAAAA DFBAAA HHHHxx +119 810 1 3 9 19 9 19 119 119 119 18 19 PEAAAA EFBAAA OOOOxx +182 811 0 2 2 2 2 82 182 182 182 4 5 AHAAAA FFBAAA VVVVxx +606 812 0 2 6 6 6 6 6 106 606 12 13 IXAAAA GFBAAA AAAAxx +849 813 1 1 9 9 9 49 49 349 849 18 19 RGAAAA HFBAAA HHHHxx +34 814 0 2 4 14 4 34 34 34 34 8 9 IBAAAA IFBAAA OOOOxx +683 815 1 3 3 3 3 83 83 183 683 6 7 HAAAAA JFBAAA VVVVxx +134 816 0 2 4 14 4 34 134 134 134 8 9 EFAAAA KFBAAA AAAAxx +331 817 1 3 1 11 1 31 131 331 331 2 3 TMAAAA LFBAAA HHHHxx +808 818 0 0 8 8 8 8 8 308 808 16 17 CFAAAA MFBAAA OOOOxx +703 819 1 3 3 3 3 3 103 203 703 6 7 BBAAAA NFBAAA VVVVxx +669 820 1 1 9 9 9 69 69 169 669 18 19 TZAAAA OFBAAA AAAAxx +264 821 0 0 4 4 4 64 64 264 264 8 9 EKAAAA PFBAAA HHHHxx +277 822 1 1 7 17 7 77 77 277 277 14 15 RKAAAA QFBAAA OOOOxx +877 823 1 1 7 17 7 77 77 377 877 14 15 THAAAA RFBAAA VVVVxx +783 824 1 3 3 3 3 83 183 283 783 6 7 DEAAAA SFBAAA AAAAxx +791 825 1 3 1 11 1 91 191 291 791 2 3 LEAAAA TFBAAA HHHHxx +171 826 1 3 1 11 1 71 171 171 171 2 3 PGAAAA UFBAAA OOOOxx +564 827 0 0 4 4 4 64 164 64 564 8 9 SVAAAA VFBAAA VVVVxx +230 828 0 2 0 10 0 30 30 230 230 0 1 WIAAAA WFBAAA AAAAxx +881 829 1 1 1 1 1 81 81 381 881 2 3 XHAAAA XFBAAA HHHHxx +890 830 0 2 0 10 0 90 90 390 890 0 1 GIAAAA YFBAAA OOOOxx +374 831 0 2 4 14 4 74 174 374 374 8 9 KOAAAA ZFBAAA VVVVxx +697 832 1 1 7 17 7 97 97 197 697 14 15 VAAAAA AGBAAA AAAAxx +4 833 0 0 4 4 4 4 4 4 4 8 9 EAAAAA BGBAAA HHHHxx +385 834 1 1 5 5 5 85 185 385 385 10 11 VOAAAA CGBAAA OOOOxx +739 835 1 3 9 19 9 39 139 239 739 18 19 LCAAAA DGBAAA VVVVxx +623 836 1 3 3 3 3 23 23 123 623 6 7 ZXAAAA EGBAAA AAAAxx +547 837 1 3 7 7 7 47 147 47 547 14 15 BVAAAA FGBAAA HHHHxx +532 838 0 0 2 12 2 32 132 32 532 4 5 MUAAAA GGBAAA OOOOxx +383 839 1 3 3 3 3 83 183 383 383 6 7 TOAAAA HGBAAA VVVVxx +181 840 1 1 1 1 1 81 181 181 181 2 3 ZGAAAA IGBAAA AAAAxx +327 841 1 3 7 7 7 27 127 327 327 14 15 PMAAAA JGBAAA HHHHxx +701 842 1 1 1 1 1 1 101 201 701 2 3 ZAAAAA KGBAAA OOOOxx +111 843 1 3 1 11 1 11 111 111 111 2 3 HEAAAA LGBAAA VVVVxx +977 844 1 1 7 17 7 77 177 477 977 14 15 PLAAAA MGBAAA AAAAxx +431 845 1 3 1 11 1 31 31 431 431 2 3 PQAAAA NGBAAA HHHHxx +456 846 0 0 6 16 6 56 56 456 456 12 13 ORAAAA OGBAAA OOOOxx +368 847 0 0 8 8 8 68 168 368 368 16 17 EOAAAA PGBAAA VVVVxx +32 848 0 0 2 12 2 32 32 32 32 4 5 GBAAAA QGBAAA AAAAxx +125 849 1 1 5 5 5 25 125 125 125 10 11 VEAAAA RGBAAA HHHHxx +847 850 1 3 7 7 7 47 47 347 847 14 15 PGAAAA SGBAAA OOOOxx +485 851 1 1 5 5 5 85 85 485 485 10 11 RSAAAA TGBAAA VVVVxx +387 852 1 3 7 7 7 87 187 387 387 14 15 XOAAAA UGBAAA AAAAxx +288 853 0 0 8 8 8 88 88 288 288 16 17 CLAAAA VGBAAA HHHHxx +919 854 1 3 9 19 9 19 119 419 919 18 19 JJAAAA WGBAAA OOOOxx +393 855 1 1 3 13 3 93 193 393 393 6 7 DPAAAA XGBAAA VVVVxx +953 856 1 1 3 13 3 53 153 453 953 6 7 RKAAAA YGBAAA AAAAxx +798 857 0 2 8 18 8 98 198 298 798 16 17 SEAAAA ZGBAAA HHHHxx +940 858 0 0 0 0 0 40 140 440 940 0 1 EKAAAA AHBAAA OOOOxx +198 859 0 2 8 18 8 98 198 198 198 16 17 QHAAAA BHBAAA VVVVxx +25 860 1 1 5 5 5 25 25 25 25 10 11 ZAAAAA CHBAAA AAAAxx +190 861 0 2 0 10 0 90 190 190 190 0 1 IHAAAA DHBAAA HHHHxx +820 862 0 0 0 0 0 20 20 320 820 0 1 OFAAAA EHBAAA OOOOxx +15 863 1 3 5 15 5 15 15 15 15 10 11 PAAAAA FHBAAA VVVVxx +427 864 1 3 7 7 7 27 27 427 427 14 15 LQAAAA GHBAAA AAAAxx +349 865 1 1 9 9 9 49 149 349 349 18 19 LNAAAA HHBAAA HHHHxx +785 866 1 1 5 5 5 85 185 285 785 10 11 FEAAAA IHBAAA OOOOxx +340 867 0 0 0 0 0 40 140 340 340 0 1 CNAAAA JHBAAA VVVVxx +292 868 0 0 2 12 2 92 92 292 292 4 5 GLAAAA KHBAAA AAAAxx +17 869 1 1 7 17 7 17 17 17 17 14 15 RAAAAA LHBAAA HHHHxx +985 870 1 1 5 5 5 85 185 485 985 10 11 XLAAAA MHBAAA OOOOxx +645 871 1 1 5 5 5 45 45 145 645 10 11 VYAAAA NHBAAA VVVVxx +631 872 1 3 1 11 1 31 31 131 631 2 3 HYAAAA OHBAAA AAAAxx +761 873 1 1 1 1 1 61 161 261 761 2 3 HDAAAA PHBAAA HHHHxx +707 874 1 3 7 7 7 7 107 207 707 14 15 FBAAAA QHBAAA OOOOxx +776 875 0 0 6 16 6 76 176 276 776 12 13 WDAAAA RHBAAA VVVVxx +856 876 0 0 6 16 6 56 56 356 856 12 13 YGAAAA SHBAAA AAAAxx +978 877 0 2 8 18 8 78 178 478 978 16 17 QLAAAA THBAAA HHHHxx +710 878 0 2 0 10 0 10 110 210 710 0 1 IBAAAA UHBAAA OOOOxx +604 879 0 0 4 4 4 4 4 104 604 8 9 GXAAAA VHBAAA VVVVxx +291 880 1 3 1 11 1 91 91 291 291 2 3 FLAAAA WHBAAA AAAAxx +747 881 1 3 7 7 7 47 147 247 747 14 15 TCAAAA XHBAAA HHHHxx +837 882 1 1 7 17 7 37 37 337 837 14 15 FGAAAA YHBAAA OOOOxx +722 883 0 2 2 2 2 22 122 222 722 4 5 UBAAAA ZHBAAA VVVVxx +925 884 1 1 5 5 5 25 125 425 925 10 11 PJAAAA AIBAAA AAAAxx +49 885 1 1 9 9 9 49 49 49 49 18 19 XBAAAA BIBAAA HHHHxx +832 886 0 0 2 12 2 32 32 332 832 4 5 AGAAAA CIBAAA OOOOxx +336 887 0 0 6 16 6 36 136 336 336 12 13 YMAAAA DIBAAA VVVVxx +185 888 1 1 5 5 5 85 185 185 185 10 11 DHAAAA EIBAAA AAAAxx +434 889 0 2 4 14 4 34 34 434 434 8 9 SQAAAA FIBAAA HHHHxx +284 890 0 0 4 4 4 84 84 284 284 8 9 YKAAAA GIBAAA OOOOxx +812 891 0 0 2 12 2 12 12 312 812 4 5 GFAAAA HIBAAA VVVVxx +810 892 0 2 0 10 0 10 10 310 810 0 1 EFAAAA IIBAAA AAAAxx +252 893 0 0 2 12 2 52 52 252 252 4 5 SJAAAA JIBAAA HHHHxx +965 894 1 1 5 5 5 65 165 465 965 10 11 DLAAAA KIBAAA OOOOxx +110 895 0 2 0 10 0 10 110 110 110 0 1 GEAAAA LIBAAA VVVVxx +698 896 0 2 8 18 8 98 98 198 698 16 17 WAAAAA MIBAAA AAAAxx +283 897 1 3 3 3 3 83 83 283 283 6 7 XKAAAA NIBAAA HHHHxx +533 898 1 1 3 13 3 33 133 33 533 6 7 NUAAAA OIBAAA OOOOxx +662 899 0 2 2 2 2 62 62 162 662 4 5 MZAAAA PIBAAA VVVVxx +329 900 1 1 9 9 9 29 129 329 329 18 19 RMAAAA QIBAAA AAAAxx +250 901 0 2 0 10 0 50 50 250 250 0 1 QJAAAA RIBAAA HHHHxx +407 902 1 3 7 7 7 7 7 407 407 14 15 RPAAAA SIBAAA OOOOxx +823 903 1 3 3 3 3 23 23 323 823 6 7 RFAAAA TIBAAA VVVVxx +852 904 0 0 2 12 2 52 52 352 852 4 5 UGAAAA UIBAAA AAAAxx +871 905 1 3 1 11 1 71 71 371 871 2 3 NHAAAA VIBAAA HHHHxx +118 906 0 2 8 18 8 18 118 118 118 16 17 OEAAAA WIBAAA OOOOxx +912 907 0 0 2 12 2 12 112 412 912 4 5 CJAAAA XIBAAA VVVVxx +458 908 0 2 8 18 8 58 58 458 458 16 17 QRAAAA YIBAAA AAAAxx +926 909 0 2 6 6 6 26 126 426 926 12 13 QJAAAA ZIBAAA HHHHxx +328 910 0 0 8 8 8 28 128 328 328 16 17 QMAAAA AJBAAA OOOOxx +980 911 0 0 0 0 0 80 180 480 980 0 1 SLAAAA BJBAAA VVVVxx +259 912 1 3 9 19 9 59 59 259 259 18 19 ZJAAAA CJBAAA AAAAxx +900 913 0 0 0 0 0 0 100 400 900 0 1 QIAAAA DJBAAA HHHHxx +137 914 1 1 7 17 7 37 137 137 137 14 15 HFAAAA EJBAAA OOOOxx +159 915 1 3 9 19 9 59 159 159 159 18 19 DGAAAA FJBAAA VVVVxx +243 916 1 3 3 3 3 43 43 243 243 6 7 JJAAAA GJBAAA AAAAxx +472 917 0 0 2 12 2 72 72 472 472 4 5 ESAAAA HJBAAA HHHHxx +796 918 0 0 6 16 6 96 196 296 796 12 13 QEAAAA IJBAAA OOOOxx +382 919 0 2 2 2 2 82 182 382 382 4 5 SOAAAA JJBAAA VVVVxx +911 920 1 3 1 11 1 11 111 411 911 2 3 BJAAAA KJBAAA AAAAxx +179 921 1 3 9 19 9 79 179 179 179 18 19 XGAAAA LJBAAA HHHHxx +778 922 0 2 8 18 8 78 178 278 778 16 17 YDAAAA MJBAAA OOOOxx +405 923 1 1 5 5 5 5 5 405 405 10 11 PPAAAA NJBAAA VVVVxx +265 924 1 1 5 5 5 65 65 265 265 10 11 FKAAAA OJBAAA AAAAxx +556 925 0 0 6 16 6 56 156 56 556 12 13 KVAAAA PJBAAA HHHHxx +16 926 0 0 6 16 6 16 16 16 16 12 13 QAAAAA QJBAAA OOOOxx +706 927 0 2 6 6 6 6 106 206 706 12 13 EBAAAA RJBAAA VVVVxx +497 928 1 1 7 17 7 97 97 497 497 14 15 DTAAAA SJBAAA AAAAxx +708 929 0 0 8 8 8 8 108 208 708 16 17 GBAAAA TJBAAA HHHHxx +46 930 0 2 6 6 6 46 46 46 46 12 13 UBAAAA UJBAAA OOOOxx +901 931 1 1 1 1 1 1 101 401 901 2 3 RIAAAA VJBAAA VVVVxx +416 932 0 0 6 16 6 16 16 416 416 12 13 AQAAAA WJBAAA AAAAxx +307 933 1 3 7 7 7 7 107 307 307 14 15 VLAAAA XJBAAA HHHHxx +166 934 0 2 6 6 6 66 166 166 166 12 13 KGAAAA YJBAAA OOOOxx +178 935 0 2 8 18 8 78 178 178 178 16 17 WGAAAA ZJBAAA VVVVxx +499 936 1 3 9 19 9 99 99 499 499 18 19 FTAAAA AKBAAA AAAAxx +257 937 1 1 7 17 7 57 57 257 257 14 15 XJAAAA BKBAAA HHHHxx +342 938 0 2 2 2 2 42 142 342 342 4 5 ENAAAA CKBAAA OOOOxx +850 939 0 2 0 10 0 50 50 350 850 0 1 SGAAAA DKBAAA VVVVxx +313 940 1 1 3 13 3 13 113 313 313 6 7 BMAAAA EKBAAA AAAAxx +831 941 1 3 1 11 1 31 31 331 831 2 3 ZFAAAA FKBAAA HHHHxx +57 942 1 1 7 17 7 57 57 57 57 14 15 FCAAAA GKBAAA OOOOxx +37 943 1 1 7 17 7 37 37 37 37 14 15 LBAAAA HKBAAA VVVVxx +511 944 1 3 1 11 1 11 111 11 511 2 3 RTAAAA IKBAAA AAAAxx +578 945 0 2 8 18 8 78 178 78 578 16 17 GWAAAA JKBAAA HHHHxx +100 946 0 0 0 0 0 0 100 100 100 0 1 WDAAAA KKBAAA OOOOxx +935 947 1 3 5 15 5 35 135 435 935 10 11 ZJAAAA LKBAAA VVVVxx +821 948 1 1 1 1 1 21 21 321 821 2 3 PFAAAA MKBAAA AAAAxx +294 949 0 2 4 14 4 94 94 294 294 8 9 ILAAAA NKBAAA HHHHxx +575 950 1 3 5 15 5 75 175 75 575 10 11 DWAAAA OKBAAA OOOOxx +272 951 0 0 2 12 2 72 72 272 272 4 5 MKAAAA PKBAAA VVVVxx +491 952 1 3 1 11 1 91 91 491 491 2 3 XSAAAA QKBAAA AAAAxx +43 953 1 3 3 3 3 43 43 43 43 6 7 RBAAAA RKBAAA HHHHxx +167 954 1 3 7 7 7 67 167 167 167 14 15 LGAAAA SKBAAA OOOOxx +457 955 1 1 7 17 7 57 57 457 457 14 15 PRAAAA TKBAAA VVVVxx +647 956 1 3 7 7 7 47 47 147 647 14 15 XYAAAA UKBAAA AAAAxx +180 957 0 0 0 0 0 80 180 180 180 0 1 YGAAAA VKBAAA HHHHxx +48 958 0 0 8 8 8 48 48 48 48 16 17 WBAAAA WKBAAA OOOOxx +553 959 1 1 3 13 3 53 153 53 553 6 7 HVAAAA XKBAAA VVVVxx +188 960 0 0 8 8 8 88 188 188 188 16 17 GHAAAA YKBAAA AAAAxx +262 961 0 2 2 2 2 62 62 262 262 4 5 CKAAAA ZKBAAA HHHHxx +728 962 0 0 8 8 8 28 128 228 728 16 17 ACAAAA ALBAAA OOOOxx +581 963 1 1 1 1 1 81 181 81 581 2 3 JWAAAA BLBAAA VVVVxx +937 964 1 1 7 17 7 37 137 437 937 14 15 BKAAAA CLBAAA AAAAxx +370 965 0 2 0 10 0 70 170 370 370 0 1 GOAAAA DLBAAA HHHHxx +590 966 0 2 0 10 0 90 190 90 590 0 1 SWAAAA ELBAAA OOOOxx +421 967 1 1 1 1 1 21 21 421 421 2 3 FQAAAA FLBAAA VVVVxx +693 968 1 1 3 13 3 93 93 193 693 6 7 RAAAAA GLBAAA AAAAxx +906 969 0 2 6 6 6 6 106 406 906 12 13 WIAAAA HLBAAA HHHHxx +802 970 0 2 2 2 2 2 2 302 802 4 5 WEAAAA ILBAAA OOOOxx +38 971 0 2 8 18 8 38 38 38 38 16 17 MBAAAA JLBAAA VVVVxx +790 972 0 2 0 10 0 90 190 290 790 0 1 KEAAAA KLBAAA AAAAxx +726 973 0 2 6 6 6 26 126 226 726 12 13 YBAAAA LLBAAA HHHHxx +23 974 1 3 3 3 3 23 23 23 23 6 7 XAAAAA MLBAAA OOOOxx +641 975 1 1 1 1 1 41 41 141 641 2 3 RYAAAA NLBAAA VVVVxx +524 976 0 0 4 4 4 24 124 24 524 8 9 EUAAAA OLBAAA AAAAxx +169 977 1 1 9 9 9 69 169 169 169 18 19 NGAAAA PLBAAA HHHHxx +6 978 0 2 6 6 6 6 6 6 6 12 13 GAAAAA QLBAAA OOOOxx +943 979 1 3 3 3 3 43 143 443 943 6 7 HKAAAA RLBAAA VVVVxx +26 980 0 2 6 6 6 26 26 26 26 12 13 ABAAAA SLBAAA AAAAxx +469 981 1 1 9 9 9 69 69 469 469 18 19 BSAAAA TLBAAA HHHHxx +968 982 0 0 8 8 8 68 168 468 968 16 17 GLAAAA ULBAAA OOOOxx +947 983 1 3 7 7 7 47 147 447 947 14 15 LKAAAA VLBAAA VVVVxx +133 984 1 1 3 13 3 33 133 133 133 6 7 DFAAAA WLBAAA AAAAxx +52 985 0 0 2 12 2 52 52 52 52 4 5 ACAAAA XLBAAA HHHHxx +660 986 0 0 0 0 0 60 60 160 660 0 1 KZAAAA YLBAAA OOOOxx +780 987 0 0 0 0 0 80 180 280 780 0 1 AEAAAA ZLBAAA VVVVxx +963 988 1 3 3 3 3 63 163 463 963 6 7 BLAAAA AMBAAA AAAAxx +561 989 1 1 1 1 1 61 161 61 561 2 3 PVAAAA BMBAAA HHHHxx +402 990 0 2 2 2 2 2 2 402 402 4 5 MPAAAA CMBAAA OOOOxx +437 991 1 1 7 17 7 37 37 437 437 14 15 VQAAAA DMBAAA VVVVxx +112 992 0 0 2 12 2 12 112 112 112 4 5 IEAAAA EMBAAA AAAAxx +247 993 1 3 7 7 7 47 47 247 247 14 15 NJAAAA FMBAAA HHHHxx +579 994 1 3 9 19 9 79 179 79 579 18 19 HWAAAA GMBAAA OOOOxx +379 995 1 3 9 19 9 79 179 379 379 18 19 POAAAA HMBAAA VVVVxx +74 996 0 2 4 14 4 74 74 74 74 8 9 WCAAAA IMBAAA AAAAxx +744 997 0 0 4 4 4 44 144 244 744 8 9 QCAAAA JMBAAA HHHHxx +0 998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA KMBAAA OOOOxx +278 999 0 2 8 18 8 78 78 278 278 16 17 SKAAAA LMBAAA VVVVxx diff --git a/sql/core/src/test/resources/test-data/postgresql/tenk.data b/sql/core/src/test/resources/test-data/postgresql/tenk.data new file mode 100755 index 0000000000000..c9064c9c0325f --- /dev/null +++ b/sql/core/src/test/resources/test-data/postgresql/tenk.data @@ -0,0 +1,10000 @@ +8800 0 0 0 0 0 0 800 800 3800 8800 0 1 MAAAAA AAAAAA AAAAxx +1891 1 1 3 1 11 91 891 1891 1891 1891 182 183 TUAAAA BAAAAA HHHHxx +3420 2 0 0 0 0 20 420 1420 3420 3420 40 41 OBAAAA CAAAAA OOOOxx +9850 3 0 2 0 10 50 850 1850 4850 9850 100 101 WOAAAA DAAAAA VVVVxx +7164 4 0 0 4 4 64 164 1164 2164 7164 128 129 OPAAAA EAAAAA AAAAxx +8009 5 1 1 9 9 9 9 9 3009 8009 18 19 BWAAAA FAAAAA HHHHxx +5057 6 1 1 7 17 57 57 1057 57 5057 114 115 NMAAAA GAAAAA OOOOxx +6701 7 1 1 1 1 1 701 701 1701 6701 2 3 TXAAAA HAAAAA VVVVxx +4321 8 1 1 1 1 21 321 321 4321 4321 42 43 FKAAAA IAAAAA AAAAxx +3043 9 1 3 3 3 43 43 1043 3043 3043 86 87 BNAAAA JAAAAA HHHHxx +1314 10 0 2 4 14 14 314 1314 1314 1314 28 29 OYAAAA KAAAAA OOOOxx +1504 11 0 0 4 4 4 504 1504 1504 1504 8 9 WFAAAA LAAAAA VVVVxx +5222 12 0 2 2 2 22 222 1222 222 5222 44 45 WSAAAA MAAAAA AAAAxx +6243 13 1 3 3 3 43 243 243 1243 6243 86 87 DGAAAA NAAAAA HHHHxx +5471 14 1 3 1 11 71 471 1471 471 5471 142 143 LCAAAA OAAAAA OOOOxx +5006 15 0 2 6 6 6 6 1006 6 5006 12 13 OKAAAA PAAAAA VVVVxx +5387 16 1 3 7 7 87 387 1387 387 5387 174 175 FZAAAA QAAAAA AAAAxx +5785 17 1 1 5 5 85 785 1785 785 5785 170 171 NOAAAA RAAAAA HHHHxx +6621 18 1 1 1 1 21 621 621 1621 6621 42 43 RUAAAA SAAAAA OOOOxx +6969 19 1 1 9 9 69 969 969 1969 6969 138 139 BIAAAA TAAAAA VVVVxx +9460 20 0 0 0 0 60 460 1460 4460 9460 120 121 WZAAAA UAAAAA AAAAxx +59 21 1 3 9 19 59 59 59 59 59 118 119 HCAAAA VAAAAA HHHHxx +8020 22 0 0 0 0 20 20 20 3020 8020 40 41 MWAAAA WAAAAA OOOOxx +7695 23 1 3 5 15 95 695 1695 2695 7695 190 191 ZJAAAA XAAAAA VVVVxx +3442 24 0 2 2 2 42 442 1442 3442 3442 84 85 KCAAAA YAAAAA AAAAxx +5119 25 1 3 9 19 19 119 1119 119 5119 38 39 XOAAAA ZAAAAA HHHHxx +646 26 0 2 6 6 46 646 646 646 646 92 93 WYAAAA ABAAAA OOOOxx +9605 27 1 1 5 5 5 605 1605 4605 9605 10 11 LFAAAA BBAAAA VVVVxx +263 28 1 3 3 3 63 263 263 263 263 126 127 DKAAAA CBAAAA AAAAxx +3269 29 1 1 9 9 69 269 1269 3269 3269 138 139 TVAAAA DBAAAA HHHHxx +1839 30 1 3 9 19 39 839 1839 1839 1839 78 79 TSAAAA EBAAAA OOOOxx +9144 31 0 0 4 4 44 144 1144 4144 9144 88 89 SNAAAA FBAAAA VVVVxx +2513 32 1 1 3 13 13 513 513 2513 2513 26 27 RSAAAA GBAAAA AAAAxx +8850 33 0 2 0 10 50 850 850 3850 8850 100 101 KCAAAA HBAAAA HHHHxx +236 34 0 0 6 16 36 236 236 236 236 72 73 CJAAAA IBAAAA OOOOxx +3162 35 0 2 2 2 62 162 1162 3162 3162 124 125 QRAAAA JBAAAA VVVVxx +4380 36 0 0 0 0 80 380 380 4380 4380 160 161 MMAAAA KBAAAA AAAAxx +8095 37 1 3 5 15 95 95 95 3095 8095 190 191 JZAAAA LBAAAA HHHHxx +209 38 1 1 9 9 9 209 209 209 209 18 19 BIAAAA MBAAAA OOOOxx +3055 39 1 3 5 15 55 55 1055 3055 3055 110 111 NNAAAA NBAAAA VVVVxx +6921 40 1 1 1 1 21 921 921 1921 6921 42 43 FGAAAA OBAAAA AAAAxx +7046 41 0 2 6 6 46 46 1046 2046 7046 92 93 ALAAAA PBAAAA HHHHxx +7912 42 0 0 2 12 12 912 1912 2912 7912 24 25 ISAAAA QBAAAA OOOOxx +7267 43 1 3 7 7 67 267 1267 2267 7267 134 135 NTAAAA RBAAAA VVVVxx +3599 44 1 3 9 19 99 599 1599 3599 3599 198 199 LIAAAA SBAAAA AAAAxx +923 45 1 3 3 3 23 923 923 923 923 46 47 NJAAAA TBAAAA HHHHxx +1437 46 1 1 7 17 37 437 1437 1437 1437 74 75 HDAAAA UBAAAA OOOOxx +6439 47 1 3 9 19 39 439 439 1439 6439 78 79 RNAAAA VBAAAA VVVVxx +6989 48 1 1 9 9 89 989 989 1989 6989 178 179 VIAAAA WBAAAA AAAAxx +8798 49 0 2 8 18 98 798 798 3798 8798 196 197 KAAAAA XBAAAA HHHHxx +5960 50 0 0 0 0 60 960 1960 960 5960 120 121 GVAAAA YBAAAA OOOOxx +5832 51 0 0 2 12 32 832 1832 832 5832 64 65 IQAAAA ZBAAAA VVVVxx +6066 52 0 2 6 6 66 66 66 1066 6066 132 133 IZAAAA ACAAAA AAAAxx +322 53 0 2 2 2 22 322 322 322 322 44 45 KMAAAA BCAAAA HHHHxx +8321 54 1 1 1 1 21 321 321 3321 8321 42 43 BIAAAA CCAAAA OOOOxx +734 55 0 2 4 14 34 734 734 734 734 68 69 GCAAAA DCAAAA VVVVxx +688 56 0 0 8 8 88 688 688 688 688 176 177 MAAAAA ECAAAA AAAAxx +4212 57 0 0 2 12 12 212 212 4212 4212 24 25 AGAAAA FCAAAA HHHHxx +9653 58 1 1 3 13 53 653 1653 4653 9653 106 107 HHAAAA GCAAAA OOOOxx +2677 59 1 1 7 17 77 677 677 2677 2677 154 155 ZYAAAA HCAAAA VVVVxx +5423 60 1 3 3 3 23 423 1423 423 5423 46 47 PAAAAA ICAAAA AAAAxx +2592 61 0 0 2 12 92 592 592 2592 2592 184 185 SVAAAA JCAAAA HHHHxx +3233 62 1 1 3 13 33 233 1233 3233 3233 66 67 JUAAAA KCAAAA OOOOxx +5032 63 0 0 2 12 32 32 1032 32 5032 64 65 OLAAAA LCAAAA VVVVxx +2525 64 1 1 5 5 25 525 525 2525 2525 50 51 DTAAAA MCAAAA AAAAxx +4450 65 0 2 0 10 50 450 450 4450 4450 100 101 EPAAAA NCAAAA HHHHxx +5778 66 0 2 8 18 78 778 1778 778 5778 156 157 GOAAAA OCAAAA OOOOxx +5852 67 0 0 2 12 52 852 1852 852 5852 104 105 CRAAAA PCAAAA VVVVxx +5404 68 0 0 4 4 4 404 1404 404 5404 8 9 WZAAAA QCAAAA AAAAxx +6223 69 1 3 3 3 23 223 223 1223 6223 46 47 JFAAAA RCAAAA HHHHxx +6133 70 1 1 3 13 33 133 133 1133 6133 66 67 XBAAAA SCAAAA OOOOxx +9112 71 0 0 2 12 12 112 1112 4112 9112 24 25 MMAAAA TCAAAA VVVVxx +7575 72 1 3 5 15 75 575 1575 2575 7575 150 151 JFAAAA UCAAAA AAAAxx +7414 73 0 2 4 14 14 414 1414 2414 7414 28 29 EZAAAA VCAAAA HHHHxx +9741 74 1 1 1 1 41 741 1741 4741 9741 82 83 RKAAAA WCAAAA OOOOxx +3767 75 1 3 7 7 67 767 1767 3767 3767 134 135 XOAAAA XCAAAA VVVVxx +9372 76 0 0 2 12 72 372 1372 4372 9372 144 145 MWAAAA YCAAAA AAAAxx +8976 77 0 0 6 16 76 976 976 3976 8976 152 153 GHAAAA ZCAAAA HHHHxx +4071 78 1 3 1 11 71 71 71 4071 4071 142 143 PAAAAA ADAAAA OOOOxx +1311 79 1 3 1 11 11 311 1311 1311 1311 22 23 LYAAAA BDAAAA VVVVxx +2604 80 0 0 4 4 4 604 604 2604 2604 8 9 EWAAAA CDAAAA AAAAxx +8840 81 0 0 0 0 40 840 840 3840 8840 80 81 ACAAAA DDAAAA HHHHxx +567 82 1 3 7 7 67 567 567 567 567 134 135 VVAAAA EDAAAA OOOOxx +5215 83 1 3 5 15 15 215 1215 215 5215 30 31 PSAAAA FDAAAA VVVVxx +5474 84 0 2 4 14 74 474 1474 474 5474 148 149 OCAAAA GDAAAA AAAAxx +3906 85 0 2 6 6 6 906 1906 3906 3906 12 13 GUAAAA HDAAAA HHHHxx +1769 86 1 1 9 9 69 769 1769 1769 1769 138 139 BQAAAA IDAAAA OOOOxx +1454 87 0 2 4 14 54 454 1454 1454 1454 108 109 YDAAAA JDAAAA VVVVxx +6877 88 1 1 7 17 77 877 877 1877 6877 154 155 NEAAAA KDAAAA AAAAxx +6501 89 1 1 1 1 1 501 501 1501 6501 2 3 BQAAAA LDAAAA HHHHxx +934 90 0 2 4 14 34 934 934 934 934 68 69 YJAAAA MDAAAA OOOOxx +4075 91 1 3 5 15 75 75 75 4075 4075 150 151 TAAAAA NDAAAA VVVVxx +3180 92 0 0 0 0 80 180 1180 3180 3180 160 161 ISAAAA ODAAAA AAAAxx +7787 93 1 3 7 7 87 787 1787 2787 7787 174 175 NNAAAA PDAAAA HHHHxx +6401 94 1 1 1 1 1 401 401 1401 6401 2 3 FMAAAA QDAAAA OOOOxx +4244 95 0 0 4 4 44 244 244 4244 4244 88 89 GHAAAA RDAAAA VVVVxx +4591 96 1 3 1 11 91 591 591 4591 4591 182 183 PUAAAA SDAAAA AAAAxx +4113 97 1 1 3 13 13 113 113 4113 4113 26 27 FCAAAA TDAAAA HHHHxx +5925 98 1 1 5 5 25 925 1925 925 5925 50 51 XTAAAA UDAAAA OOOOxx +1987 99 1 3 7 7 87 987 1987 1987 1987 174 175 LYAAAA VDAAAA VVVVxx +8248 100 0 0 8 8 48 248 248 3248 8248 96 97 GFAAAA WDAAAA AAAAxx +4151 101 1 3 1 11 51 151 151 4151 4151 102 103 RDAAAA XDAAAA HHHHxx +8670 102 0 2 0 10 70 670 670 3670 8670 140 141 MVAAAA YDAAAA OOOOxx +6194 103 0 2 4 14 94 194 194 1194 6194 188 189 GEAAAA ZDAAAA VVVVxx +88 104 0 0 8 8 88 88 88 88 88 176 177 KDAAAA AEAAAA AAAAxx +4058 105 0 2 8 18 58 58 58 4058 4058 116 117 CAAAAA BEAAAA HHHHxx +2742 106 0 2 2 2 42 742 742 2742 2742 84 85 MBAAAA CEAAAA OOOOxx +8275 107 1 3 5 15 75 275 275 3275 8275 150 151 HGAAAA DEAAAA VVVVxx +4258 108 0 2 8 18 58 258 258 4258 4258 116 117 UHAAAA EEAAAA AAAAxx +6129 109 1 1 9 9 29 129 129 1129 6129 58 59 TBAAAA FEAAAA HHHHxx +7243 110 1 3 3 3 43 243 1243 2243 7243 86 87 PSAAAA GEAAAA OOOOxx +2392 111 0 0 2 12 92 392 392 2392 2392 184 185 AOAAAA HEAAAA VVVVxx +9853 112 1 1 3 13 53 853 1853 4853 9853 106 107 ZOAAAA IEAAAA AAAAxx +6064 113 0 0 4 4 64 64 64 1064 6064 128 129 GZAAAA JEAAAA HHHHxx +4391 114 1 3 1 11 91 391 391 4391 4391 182 183 XMAAAA KEAAAA OOOOxx +726 115 0 2 6 6 26 726 726 726 726 52 53 YBAAAA LEAAAA VVVVxx +6957 116 1 1 7 17 57 957 957 1957 6957 114 115 PHAAAA MEAAAA AAAAxx +3853 117 1 1 3 13 53 853 1853 3853 3853 106 107 FSAAAA NEAAAA HHHHxx +4524 118 0 0 4 4 24 524 524 4524 4524 48 49 ASAAAA OEAAAA OOOOxx +5330 119 0 2 0 10 30 330 1330 330 5330 60 61 AXAAAA PEAAAA VVVVxx +6671 120 1 3 1 11 71 671 671 1671 6671 142 143 PWAAAA QEAAAA AAAAxx +5314 121 0 2 4 14 14 314 1314 314 5314 28 29 KWAAAA REAAAA HHHHxx +9202 122 0 2 2 2 2 202 1202 4202 9202 4 5 YPAAAA SEAAAA OOOOxx +4596 123 0 0 6 16 96 596 596 4596 4596 192 193 UUAAAA TEAAAA VVVVxx +8951 124 1 3 1 11 51 951 951 3951 8951 102 103 HGAAAA UEAAAA AAAAxx +9902 125 0 2 2 2 2 902 1902 4902 9902 4 5 WQAAAA VEAAAA HHHHxx +1440 126 0 0 0 0 40 440 1440 1440 1440 80 81 KDAAAA WEAAAA OOOOxx +5339 127 1 3 9 19 39 339 1339 339 5339 78 79 JXAAAA XEAAAA VVVVxx +3371 128 1 3 1 11 71 371 1371 3371 3371 142 143 RZAAAA YEAAAA AAAAxx +4467 129 1 3 7 7 67 467 467 4467 4467 134 135 VPAAAA ZEAAAA HHHHxx +6216 130 0 0 6 16 16 216 216 1216 6216 32 33 CFAAAA AFAAAA OOOOxx +5364 131 0 0 4 4 64 364 1364 364 5364 128 129 IYAAAA BFAAAA VVVVxx +7547 132 1 3 7 7 47 547 1547 2547 7547 94 95 HEAAAA CFAAAA AAAAxx +4338 133 0 2 8 18 38 338 338 4338 4338 76 77 WKAAAA DFAAAA HHHHxx +3481 134 1 1 1 1 81 481 1481 3481 3481 162 163 XDAAAA EFAAAA OOOOxx +826 135 0 2 6 6 26 826 826 826 826 52 53 UFAAAA FFAAAA VVVVxx +3647 136 1 3 7 7 47 647 1647 3647 3647 94 95 HKAAAA GFAAAA AAAAxx +3337 137 1 1 7 17 37 337 1337 3337 3337 74 75 JYAAAA HFAAAA HHHHxx +3591 138 1 3 1 11 91 591 1591 3591 3591 182 183 DIAAAA IFAAAA OOOOxx +7192 139 0 0 2 12 92 192 1192 2192 7192 184 185 QQAAAA JFAAAA VVVVxx +1078 140 0 2 8 18 78 78 1078 1078 1078 156 157 MPAAAA KFAAAA AAAAxx +1310 141 0 2 0 10 10 310 1310 1310 1310 20 21 KYAAAA LFAAAA HHHHxx +9642 142 0 2 2 2 42 642 1642 4642 9642 84 85 WGAAAA MFAAAA OOOOxx +39 143 1 3 9 19 39 39 39 39 39 78 79 NBAAAA NFAAAA VVVVxx +8682 144 0 2 2 2 82 682 682 3682 8682 164 165 YVAAAA OFAAAA AAAAxx +1794 145 0 2 4 14 94 794 1794 1794 1794 188 189 ARAAAA PFAAAA HHHHxx +5630 146 0 2 0 10 30 630 1630 630 5630 60 61 OIAAAA QFAAAA OOOOxx +6748 147 0 0 8 8 48 748 748 1748 6748 96 97 OZAAAA RFAAAA VVVVxx +3766 148 0 2 6 6 66 766 1766 3766 3766 132 133 WOAAAA SFAAAA AAAAxx +6403 149 1 3 3 3 3 403 403 1403 6403 6 7 HMAAAA TFAAAA HHHHxx +175 150 1 3 5 15 75 175 175 175 175 150 151 TGAAAA UFAAAA OOOOxx +2179 151 1 3 9 19 79 179 179 2179 2179 158 159 VFAAAA VFAAAA VVVVxx +7897 152 1 1 7 17 97 897 1897 2897 7897 194 195 TRAAAA WFAAAA AAAAxx +2760 153 0 0 0 0 60 760 760 2760 2760 120 121 ECAAAA XFAAAA HHHHxx +1675 154 1 3 5 15 75 675 1675 1675 1675 150 151 LMAAAA YFAAAA OOOOxx +2564 155 0 0 4 4 64 564 564 2564 2564 128 129 QUAAAA ZFAAAA VVVVxx +157 156 1 1 7 17 57 157 157 157 157 114 115 BGAAAA AGAAAA AAAAxx +8779 157 1 3 9 19 79 779 779 3779 8779 158 159 RZAAAA BGAAAA HHHHxx +9591 158 1 3 1 11 91 591 1591 4591 9591 182 183 XEAAAA CGAAAA OOOOxx +8732 159 0 0 2 12 32 732 732 3732 8732 64 65 WXAAAA DGAAAA VVVVxx +139 160 1 3 9 19 39 139 139 139 139 78 79 JFAAAA EGAAAA AAAAxx +5372 161 0 0 2 12 72 372 1372 372 5372 144 145 QYAAAA FGAAAA HHHHxx +1278 162 0 2 8 18 78 278 1278 1278 1278 156 157 EXAAAA GGAAAA OOOOxx +4697 163 1 1 7 17 97 697 697 4697 4697 194 195 RYAAAA HGAAAA VVVVxx +8610 164 0 2 0 10 10 610 610 3610 8610 20 21 ETAAAA IGAAAA AAAAxx +8180 165 0 0 0 0 80 180 180 3180 8180 160 161 QCAAAA JGAAAA HHHHxx +2399 166 1 3 9 19 99 399 399 2399 2399 198 199 HOAAAA KGAAAA OOOOxx +615 167 1 3 5 15 15 615 615 615 615 30 31 RXAAAA LGAAAA VVVVxx +7629 168 1 1 9 9 29 629 1629 2629 7629 58 59 LHAAAA MGAAAA AAAAxx +7628 169 0 0 8 8 28 628 1628 2628 7628 56 57 KHAAAA NGAAAA HHHHxx +4659 170 1 3 9 19 59 659 659 4659 4659 118 119 FXAAAA OGAAAA OOOOxx +5865 171 1 1 5 5 65 865 1865 865 5865 130 131 PRAAAA PGAAAA VVVVxx +3973 172 1 1 3 13 73 973 1973 3973 3973 146 147 VWAAAA QGAAAA AAAAxx +552 173 0 0 2 12 52 552 552 552 552 104 105 GVAAAA RGAAAA HHHHxx +708 174 0 0 8 8 8 708 708 708 708 16 17 GBAAAA SGAAAA OOOOxx +3550 175 0 2 0 10 50 550 1550 3550 3550 100 101 OGAAAA TGAAAA VVVVxx +5547 176 1 3 7 7 47 547 1547 547 5547 94 95 JFAAAA UGAAAA AAAAxx +489 177 1 1 9 9 89 489 489 489 489 178 179 VSAAAA VGAAAA HHHHxx +3794 178 0 2 4 14 94 794 1794 3794 3794 188 189 YPAAAA WGAAAA OOOOxx +9479 179 1 3 9 19 79 479 1479 4479 9479 158 159 PAAAAA XGAAAA VVVVxx +6435 180 1 3 5 15 35 435 435 1435 6435 70 71 NNAAAA YGAAAA AAAAxx +5120 181 0 0 0 0 20 120 1120 120 5120 40 41 YOAAAA ZGAAAA HHHHxx +3615 182 1 3 5 15 15 615 1615 3615 3615 30 31 BJAAAA AHAAAA OOOOxx +8399 183 1 3 9 19 99 399 399 3399 8399 198 199 BLAAAA BHAAAA VVVVxx +2155 184 1 3 5 15 55 155 155 2155 2155 110 111 XEAAAA CHAAAA AAAAxx +6690 185 0 2 0 10 90 690 690 1690 6690 180 181 IXAAAA DHAAAA HHHHxx +1683 186 1 3 3 3 83 683 1683 1683 1683 166 167 TMAAAA EHAAAA OOOOxx +6302 187 0 2 2 2 2 302 302 1302 6302 4 5 KIAAAA FHAAAA VVVVxx +516 188 0 0 6 16 16 516 516 516 516 32 33 WTAAAA GHAAAA AAAAxx +3901 189 1 1 1 1 1 901 1901 3901 3901 2 3 BUAAAA HHAAAA HHHHxx +6938 190 0 2 8 18 38 938 938 1938 6938 76 77 WGAAAA IHAAAA OOOOxx +7484 191 0 0 4 4 84 484 1484 2484 7484 168 169 WBAAAA JHAAAA VVVVxx +7424 192 0 0 4 4 24 424 1424 2424 7424 48 49 OZAAAA KHAAAA AAAAxx +9410 193 0 2 0 10 10 410 1410 4410 9410 20 21 YXAAAA LHAAAA HHHHxx +1714 194 0 2 4 14 14 714 1714 1714 1714 28 29 YNAAAA MHAAAA OOOOxx +8278 195 0 2 8 18 78 278 278 3278 8278 156 157 KGAAAA NHAAAA VVVVxx +3158 196 0 2 8 18 58 158 1158 3158 3158 116 117 MRAAAA OHAAAA AAAAxx +2511 197 1 3 1 11 11 511 511 2511 2511 22 23 PSAAAA PHAAAA HHHHxx +2912 198 0 0 2 12 12 912 912 2912 2912 24 25 AIAAAA QHAAAA OOOOxx +2648 199 0 0 8 8 48 648 648 2648 2648 96 97 WXAAAA RHAAAA VVVVxx +9385 200 1 1 5 5 85 385 1385 4385 9385 170 171 ZWAAAA SHAAAA AAAAxx +7545 201 1 1 5 5 45 545 1545 2545 7545 90 91 FEAAAA THAAAA HHHHxx +8407 202 1 3 7 7 7 407 407 3407 8407 14 15 JLAAAA UHAAAA OOOOxx +5893 203 1 1 3 13 93 893 1893 893 5893 186 187 RSAAAA VHAAAA VVVVxx +7049 204 1 1 9 9 49 49 1049 2049 7049 98 99 DLAAAA WHAAAA AAAAxx +6812 205 0 0 2 12 12 812 812 1812 6812 24 25 ACAAAA XHAAAA HHHHxx +3649 206 1 1 9 9 49 649 1649 3649 3649 98 99 JKAAAA YHAAAA OOOOxx +9275 207 1 3 5 15 75 275 1275 4275 9275 150 151 TSAAAA ZHAAAA VVVVxx +1179 208 1 3 9 19 79 179 1179 1179 1179 158 159 JTAAAA AIAAAA AAAAxx +969 209 1 1 9 9 69 969 969 969 969 138 139 HLAAAA BIAAAA HHHHxx +7920 210 0 0 0 0 20 920 1920 2920 7920 40 41 QSAAAA CIAAAA OOOOxx +998 211 0 2 8 18 98 998 998 998 998 196 197 KMAAAA DIAAAA VVVVxx +3958 212 0 2 8 18 58 958 1958 3958 3958 116 117 GWAAAA EIAAAA AAAAxx +6052 213 0 0 2 12 52 52 52 1052 6052 104 105 UYAAAA FIAAAA HHHHxx +8791 214 1 3 1 11 91 791 791 3791 8791 182 183 DAAAAA GIAAAA OOOOxx +5191 215 1 3 1 11 91 191 1191 191 5191 182 183 RRAAAA HIAAAA VVVVxx +4267 216 1 3 7 7 67 267 267 4267 4267 134 135 DIAAAA IIAAAA AAAAxx +2829 217 1 1 9 9 29 829 829 2829 2829 58 59 VEAAAA JIAAAA HHHHxx +6396 218 0 0 6 16 96 396 396 1396 6396 192 193 AMAAAA KIAAAA OOOOxx +9413 219 1 1 3 13 13 413 1413 4413 9413 26 27 BYAAAA LIAAAA VVVVxx +614 220 0 2 4 14 14 614 614 614 614 28 29 QXAAAA MIAAAA AAAAxx +4660 221 0 0 0 0 60 660 660 4660 4660 120 121 GXAAAA NIAAAA HHHHxx +8834 222 0 2 4 14 34 834 834 3834 8834 68 69 UBAAAA OIAAAA OOOOxx +2767 223 1 3 7 7 67 767 767 2767 2767 134 135 LCAAAA PIAAAA VVVVxx +2444 224 0 0 4 4 44 444 444 2444 2444 88 89 AQAAAA QIAAAA AAAAxx +4129 225 1 1 9 9 29 129 129 4129 4129 58 59 VCAAAA RIAAAA HHHHxx +3394 226 0 2 4 14 94 394 1394 3394 3394 188 189 OAAAAA SIAAAA OOOOxx +2705 227 1 1 5 5 5 705 705 2705 2705 10 11 BAAAAA TIAAAA VVVVxx +8499 228 1 3 9 19 99 499 499 3499 8499 198 199 XOAAAA UIAAAA AAAAxx +8852 229 0 0 2 12 52 852 852 3852 8852 104 105 MCAAAA VIAAAA HHHHxx +6174 230 0 2 4 14 74 174 174 1174 6174 148 149 MDAAAA WIAAAA OOOOxx +750 231 0 2 0 10 50 750 750 750 750 100 101 WCAAAA XIAAAA VVVVxx +8164 232 0 0 4 4 64 164 164 3164 8164 128 129 ACAAAA YIAAAA AAAAxx +4930 233 0 2 0 10 30 930 930 4930 4930 60 61 QHAAAA ZIAAAA HHHHxx +9904 234 0 0 4 4 4 904 1904 4904 9904 8 9 YQAAAA AJAAAA OOOOxx +7378 235 0 2 8 18 78 378 1378 2378 7378 156 157 UXAAAA BJAAAA VVVVxx +2927 236 1 3 7 7 27 927 927 2927 2927 54 55 PIAAAA CJAAAA AAAAxx +7155 237 1 3 5 15 55 155 1155 2155 7155 110 111 FPAAAA DJAAAA HHHHxx +1302 238 0 2 2 2 2 302 1302 1302 1302 4 5 CYAAAA EJAAAA OOOOxx +5904 239 0 0 4 4 4 904 1904 904 5904 8 9 CTAAAA FJAAAA VVVVxx +9687 240 1 3 7 7 87 687 1687 4687 9687 174 175 PIAAAA GJAAAA AAAAxx +3553 241 1 1 3 13 53 553 1553 3553 3553 106 107 RGAAAA HJAAAA HHHHxx +4447 242 1 3 7 7 47 447 447 4447 4447 94 95 BPAAAA IJAAAA OOOOxx +6878 243 0 2 8 18 78 878 878 1878 6878 156 157 OEAAAA JJAAAA VVVVxx +9470 244 0 2 0 10 70 470 1470 4470 9470 140 141 GAAAAA KJAAAA AAAAxx +9735 245 1 3 5 15 35 735 1735 4735 9735 70 71 LKAAAA LJAAAA HHHHxx +5967 246 1 3 7 7 67 967 1967 967 5967 134 135 NVAAAA MJAAAA OOOOxx +6601 247 1 1 1 1 1 601 601 1601 6601 2 3 XTAAAA NJAAAA VVVVxx +7631 248 1 3 1 11 31 631 1631 2631 7631 62 63 NHAAAA OJAAAA AAAAxx +3559 249 1 3 9 19 59 559 1559 3559 3559 118 119 XGAAAA PJAAAA HHHHxx +2247 250 1 3 7 7 47 247 247 2247 2247 94 95 LIAAAA QJAAAA OOOOxx +9649 251 1 1 9 9 49 649 1649 4649 9649 98 99 DHAAAA RJAAAA VVVVxx +808 252 0 0 8 8 8 808 808 808 808 16 17 CFAAAA SJAAAA AAAAxx +240 253 0 0 0 0 40 240 240 240 240 80 81 GJAAAA TJAAAA HHHHxx +5031 254 1 3 1 11 31 31 1031 31 5031 62 63 NLAAAA UJAAAA OOOOxx +9563 255 1 3 3 3 63 563 1563 4563 9563 126 127 VDAAAA VJAAAA VVVVxx +5656 256 0 0 6 16 56 656 1656 656 5656 112 113 OJAAAA WJAAAA AAAAxx +3886 257 0 2 6 6 86 886 1886 3886 3886 172 173 MTAAAA XJAAAA HHHHxx +2431 258 1 3 1 11 31 431 431 2431 2431 62 63 NPAAAA YJAAAA OOOOxx +5560 259 0 0 0 0 60 560 1560 560 5560 120 121 WFAAAA ZJAAAA VVVVxx +9065 260 1 1 5 5 65 65 1065 4065 9065 130 131 RKAAAA AKAAAA AAAAxx +8130 261 0 2 0 10 30 130 130 3130 8130 60 61 SAAAAA BKAAAA HHHHxx +4054 262 0 2 4 14 54 54 54 4054 4054 108 109 YZAAAA CKAAAA OOOOxx +873 263 1 1 3 13 73 873 873 873 873 146 147 PHAAAA DKAAAA VVVVxx +3092 264 0 0 2 12 92 92 1092 3092 3092 184 185 YOAAAA EKAAAA AAAAxx +6697 265 1 1 7 17 97 697 697 1697 6697 194 195 PXAAAA FKAAAA HHHHxx +2452 266 0 0 2 12 52 452 452 2452 2452 104 105 IQAAAA GKAAAA OOOOxx +7867 267 1 3 7 7 67 867 1867 2867 7867 134 135 PQAAAA HKAAAA VVVVxx +3753 268 1 1 3 13 53 753 1753 3753 3753 106 107 JOAAAA IKAAAA AAAAxx +7834 269 0 2 4 14 34 834 1834 2834 7834 68 69 IPAAAA JKAAAA HHHHxx +5846 270 0 2 6 6 46 846 1846 846 5846 92 93 WQAAAA KKAAAA OOOOxx +7604 271 0 0 4 4 4 604 1604 2604 7604 8 9 MGAAAA LKAAAA VVVVxx +3452 272 0 0 2 12 52 452 1452 3452 3452 104 105 UCAAAA MKAAAA AAAAxx +4788 273 0 0 8 8 88 788 788 4788 4788 176 177 ECAAAA NKAAAA HHHHxx +8600 274 0 0 0 0 0 600 600 3600 8600 0 1 USAAAA OKAAAA OOOOxx +8511 275 1 3 1 11 11 511 511 3511 8511 22 23 JPAAAA PKAAAA VVVVxx +4452 276 0 0 2 12 52 452 452 4452 4452 104 105 GPAAAA QKAAAA AAAAxx +1709 277 1 1 9 9 9 709 1709 1709 1709 18 19 TNAAAA RKAAAA HHHHxx +3440 278 0 0 0 0 40 440 1440 3440 3440 80 81 ICAAAA SKAAAA OOOOxx +9188 279 0 0 8 8 88 188 1188 4188 9188 176 177 KPAAAA TKAAAA VVVVxx +3058 280 0 2 8 18 58 58 1058 3058 3058 116 117 QNAAAA UKAAAA AAAAxx +5821 281 1 1 1 1 21 821 1821 821 5821 42 43 XPAAAA VKAAAA HHHHxx +3428 282 0 0 8 8 28 428 1428 3428 3428 56 57 WBAAAA WKAAAA OOOOxx +3581 283 1 1 1 1 81 581 1581 3581 3581 162 163 THAAAA XKAAAA VVVVxx +7523 284 1 3 3 3 23 523 1523 2523 7523 46 47 JDAAAA YKAAAA AAAAxx +3131 285 1 3 1 11 31 131 1131 3131 3131 62 63 LQAAAA ZKAAAA HHHHxx +2404 286 0 0 4 4 4 404 404 2404 2404 8 9 MOAAAA ALAAAA OOOOxx +5453 287 1 1 3 13 53 453 1453 453 5453 106 107 TBAAAA BLAAAA VVVVxx +1599 288 1 3 9 19 99 599 1599 1599 1599 198 199 NJAAAA CLAAAA AAAAxx +7081 289 1 1 1 1 81 81 1081 2081 7081 162 163 JMAAAA DLAAAA HHHHxx +1750 290 0 2 0 10 50 750 1750 1750 1750 100 101 IPAAAA ELAAAA OOOOxx +5085 291 1 1 5 5 85 85 1085 85 5085 170 171 PNAAAA FLAAAA VVVVxx +9777 292 1 1 7 17 77 777 1777 4777 9777 154 155 BMAAAA GLAAAA AAAAxx +574 293 0 2 4 14 74 574 574 574 574 148 149 CWAAAA HLAAAA HHHHxx +5984 294 0 0 4 4 84 984 1984 984 5984 168 169 EWAAAA ILAAAA OOOOxx +7039 295 1 3 9 19 39 39 1039 2039 7039 78 79 TKAAAA JLAAAA VVVVxx +7143 296 1 3 3 3 43 143 1143 2143 7143 86 87 TOAAAA KLAAAA AAAAxx +5702 297 0 2 2 2 2 702 1702 702 5702 4 5 ILAAAA LLAAAA HHHHxx +362 298 0 2 2 2 62 362 362 362 362 124 125 YNAAAA MLAAAA OOOOxx +6997 299 1 1 7 17 97 997 997 1997 6997 194 195 DJAAAA NLAAAA VVVVxx +2529 300 1 1 9 9 29 529 529 2529 2529 58 59 HTAAAA OLAAAA AAAAxx +6319 301 1 3 9 19 19 319 319 1319 6319 38 39 BJAAAA PLAAAA HHHHxx +954 302 0 2 4 14 54 954 954 954 954 108 109 SKAAAA QLAAAA OOOOxx +3413 303 1 1 3 13 13 413 1413 3413 3413 26 27 HBAAAA RLAAAA VVVVxx +9081 304 1 1 1 1 81 81 1081 4081 9081 162 163 HLAAAA SLAAAA AAAAxx +5599 305 1 3 9 19 99 599 1599 599 5599 198 199 JHAAAA TLAAAA HHHHxx +4772 306 0 0 2 12 72 772 772 4772 4772 144 145 OBAAAA ULAAAA OOOOxx +1124 307 0 0 4 4 24 124 1124 1124 1124 48 49 GRAAAA VLAAAA VVVVxx +7793 308 1 1 3 13 93 793 1793 2793 7793 186 187 TNAAAA WLAAAA AAAAxx +4201 309 1 1 1 1 1 201 201 4201 4201 2 3 PFAAAA XLAAAA HHHHxx +7015 310 1 3 5 15 15 15 1015 2015 7015 30 31 VJAAAA YLAAAA OOOOxx +5936 311 0 0 6 16 36 936 1936 936 5936 72 73 IUAAAA ZLAAAA VVVVxx +4625 312 1 1 5 5 25 625 625 4625 4625 50 51 XVAAAA AMAAAA AAAAxx +4989 313 1 1 9 9 89 989 989 4989 4989 178 179 XJAAAA BMAAAA HHHHxx +4949 314 1 1 9 9 49 949 949 4949 4949 98 99 JIAAAA CMAAAA OOOOxx +6273 315 1 1 3 13 73 273 273 1273 6273 146 147 HHAAAA DMAAAA VVVVxx +4478 316 0 2 8 18 78 478 478 4478 4478 156 157 GQAAAA EMAAAA AAAAxx +8854 317 0 2 4 14 54 854 854 3854 8854 108 109 OCAAAA FMAAAA HHHHxx +2105 318 1 1 5 5 5 105 105 2105 2105 10 11 ZCAAAA GMAAAA OOOOxx +8345 319 1 1 5 5 45 345 345 3345 8345 90 91 ZIAAAA HMAAAA VVVVxx +1941 320 1 1 1 1 41 941 1941 1941 1941 82 83 RWAAAA IMAAAA AAAAxx +1765 321 1 1 5 5 65 765 1765 1765 1765 130 131 XPAAAA JMAAAA HHHHxx +9592 322 0 0 2 12 92 592 1592 4592 9592 184 185 YEAAAA KMAAAA OOOOxx +1694 323 0 2 4 14 94 694 1694 1694 1694 188 189 ENAAAA LMAAAA VVVVxx +8940 324 0 0 0 0 40 940 940 3940 8940 80 81 WFAAAA MMAAAA AAAAxx +7264 325 0 0 4 4 64 264 1264 2264 7264 128 129 KTAAAA NMAAAA HHHHxx +4699 326 1 3 9 19 99 699 699 4699 4699 198 199 TYAAAA OMAAAA OOOOxx +4541 327 1 1 1 1 41 541 541 4541 4541 82 83 RSAAAA PMAAAA VVVVxx +5768 328 0 0 8 8 68 768 1768 768 5768 136 137 WNAAAA QMAAAA AAAAxx +6183 329 1 3 3 3 83 183 183 1183 6183 166 167 VDAAAA RMAAAA HHHHxx +7457 330 1 1 7 17 57 457 1457 2457 7457 114 115 VAAAAA SMAAAA OOOOxx +7317 331 1 1 7 17 17 317 1317 2317 7317 34 35 LVAAAA TMAAAA VVVVxx +1944 332 0 0 4 4 44 944 1944 1944 1944 88 89 UWAAAA UMAAAA AAAAxx +665 333 1 1 5 5 65 665 665 665 665 130 131 PZAAAA VMAAAA HHHHxx +5974 334 0 2 4 14 74 974 1974 974 5974 148 149 UVAAAA WMAAAA OOOOxx +7370 335 0 2 0 10 70 370 1370 2370 7370 140 141 MXAAAA XMAAAA VVVVxx +9196 336 0 0 6 16 96 196 1196 4196 9196 192 193 SPAAAA YMAAAA AAAAxx +6796 337 0 0 6 16 96 796 796 1796 6796 192 193 KBAAAA ZMAAAA HHHHxx +6180 338 0 0 0 0 80 180 180 1180 6180 160 161 SDAAAA ANAAAA OOOOxx +8557 339 1 1 7 17 57 557 557 3557 8557 114 115 DRAAAA BNAAAA VVVVxx +928 340 0 0 8 8 28 928 928 928 928 56 57 SJAAAA CNAAAA AAAAxx +6275 341 1 3 5 15 75 275 275 1275 6275 150 151 JHAAAA DNAAAA HHHHxx +409 342 1 1 9 9 9 409 409 409 409 18 19 TPAAAA ENAAAA OOOOxx +6442 343 0 2 2 2 42 442 442 1442 6442 84 85 UNAAAA FNAAAA VVVVxx +5889 344 1 1 9 9 89 889 1889 889 5889 178 179 NSAAAA GNAAAA AAAAxx +5180 345 0 0 0 0 80 180 1180 180 5180 160 161 GRAAAA HNAAAA HHHHxx +1629 346 1 1 9 9 29 629 1629 1629 1629 58 59 RKAAAA INAAAA OOOOxx +6088 347 0 0 8 8 88 88 88 1088 6088 176 177 EAAAAA JNAAAA VVVVxx +5598 348 0 2 8 18 98 598 1598 598 5598 196 197 IHAAAA KNAAAA AAAAxx +1803 349 1 3 3 3 3 803 1803 1803 1803 6 7 JRAAAA LNAAAA HHHHxx +2330 350 0 2 0 10 30 330 330 2330 2330 60 61 QLAAAA MNAAAA OOOOxx +5901 351 1 1 1 1 1 901 1901 901 5901 2 3 ZSAAAA NNAAAA VVVVxx +780 352 0 0 0 0 80 780 780 780 780 160 161 AEAAAA ONAAAA AAAAxx +7171 353 1 3 1 11 71 171 1171 2171 7171 142 143 VPAAAA PNAAAA HHHHxx +8778 354 0 2 8 18 78 778 778 3778 8778 156 157 QZAAAA QNAAAA OOOOxx +6622 355 0 2 2 2 22 622 622 1622 6622 44 45 SUAAAA RNAAAA VVVVxx +9938 356 0 2 8 18 38 938 1938 4938 9938 76 77 GSAAAA SNAAAA AAAAxx +8254 357 0 2 4 14 54 254 254 3254 8254 108 109 MFAAAA TNAAAA HHHHxx +1951 358 1 3 1 11 51 951 1951 1951 1951 102 103 BXAAAA UNAAAA OOOOxx +1434 359 0 2 4 14 34 434 1434 1434 1434 68 69 EDAAAA VNAAAA VVVVxx +7539 360 1 3 9 19 39 539 1539 2539 7539 78 79 ZDAAAA WNAAAA AAAAxx +600 361 0 0 0 0 0 600 600 600 600 0 1 CXAAAA XNAAAA HHHHxx +3122 362 0 2 2 2 22 122 1122 3122 3122 44 45 CQAAAA YNAAAA OOOOxx +5704 363 0 0 4 4 4 704 1704 704 5704 8 9 KLAAAA ZNAAAA VVVVxx +6300 364 0 0 0 0 0 300 300 1300 6300 0 1 IIAAAA AOAAAA AAAAxx +4585 365 1 1 5 5 85 585 585 4585 4585 170 171 JUAAAA BOAAAA HHHHxx +6313 366 1 1 3 13 13 313 313 1313 6313 26 27 VIAAAA COAAAA OOOOxx +3154 367 0 2 4 14 54 154 1154 3154 3154 108 109 IRAAAA DOAAAA VVVVxx +642 368 0 2 2 2 42 642 642 642 642 84 85 SYAAAA EOAAAA AAAAxx +7736 369 0 0 6 16 36 736 1736 2736 7736 72 73 OLAAAA FOAAAA HHHHxx +5087 370 1 3 7 7 87 87 1087 87 5087 174 175 RNAAAA GOAAAA OOOOxx +5708 371 0 0 8 8 8 708 1708 708 5708 16 17 OLAAAA HOAAAA VVVVxx +8169 372 1 1 9 9 69 169 169 3169 8169 138 139 FCAAAA IOAAAA AAAAxx +9768 373 0 0 8 8 68 768 1768 4768 9768 136 137 SLAAAA JOAAAA HHHHxx +3874 374 0 2 4 14 74 874 1874 3874 3874 148 149 ATAAAA KOAAAA OOOOxx +6831 375 1 3 1 11 31 831 831 1831 6831 62 63 TCAAAA LOAAAA VVVVxx +18 376 0 2 8 18 18 18 18 18 18 36 37 SAAAAA MOAAAA AAAAxx +6375 377 1 3 5 15 75 375 375 1375 6375 150 151 FLAAAA NOAAAA HHHHxx +7106 378 0 2 6 6 6 106 1106 2106 7106 12 13 INAAAA OOAAAA OOOOxx +5926 379 0 2 6 6 26 926 1926 926 5926 52 53 YTAAAA POAAAA VVVVxx +4956 380 0 0 6 16 56 956 956 4956 4956 112 113 QIAAAA QOAAAA AAAAxx +7042 381 0 2 2 2 42 42 1042 2042 7042 84 85 WKAAAA ROAAAA HHHHxx +6043 382 1 3 3 3 43 43 43 1043 6043 86 87 LYAAAA SOAAAA OOOOxx +2084 383 0 0 4 4 84 84 84 2084 2084 168 169 ECAAAA TOAAAA VVVVxx +6038 384 0 2 8 18 38 38 38 1038 6038 76 77 GYAAAA UOAAAA AAAAxx +7253 385 1 1 3 13 53 253 1253 2253 7253 106 107 ZSAAAA VOAAAA HHHHxx +2061 386 1 1 1 1 61 61 61 2061 2061 122 123 HBAAAA WOAAAA OOOOxx +7800 387 0 0 0 0 0 800 1800 2800 7800 0 1 AOAAAA XOAAAA VVVVxx +4970 388 0 2 0 10 70 970 970 4970 4970 140 141 EJAAAA YOAAAA AAAAxx +8580 389 0 0 0 0 80 580 580 3580 8580 160 161 ASAAAA ZOAAAA HHHHxx +9173 390 1 1 3 13 73 173 1173 4173 9173 146 147 VOAAAA APAAAA OOOOxx +8558 391 0 2 8 18 58 558 558 3558 8558 116 117 ERAAAA BPAAAA VVVVxx +3897 392 1 1 7 17 97 897 1897 3897 3897 194 195 XTAAAA CPAAAA AAAAxx +5069 393 1 1 9 9 69 69 1069 69 5069 138 139 ZMAAAA DPAAAA HHHHxx +2301 394 1 1 1 1 1 301 301 2301 2301 2 3 NKAAAA EPAAAA OOOOxx +9863 395 1 3 3 3 63 863 1863 4863 9863 126 127 JPAAAA FPAAAA VVVVxx +5733 396 1 1 3 13 33 733 1733 733 5733 66 67 NMAAAA GPAAAA AAAAxx +2338 397 0 2 8 18 38 338 338 2338 2338 76 77 YLAAAA HPAAAA HHHHxx +9639 398 1 3 9 19 39 639 1639 4639 9639 78 79 TGAAAA IPAAAA OOOOxx +1139 399 1 3 9 19 39 139 1139 1139 1139 78 79 VRAAAA JPAAAA VVVVxx +2293 400 1 1 3 13 93 293 293 2293 2293 186 187 FKAAAA KPAAAA AAAAxx +6125 401 1 1 5 5 25 125 125 1125 6125 50 51 PBAAAA LPAAAA HHHHxx +5374 402 0 2 4 14 74 374 1374 374 5374 148 149 SYAAAA MPAAAA OOOOxx +7216 403 0 0 6 16 16 216 1216 2216 7216 32 33 ORAAAA NPAAAA VVVVxx +2285 404 1 1 5 5 85 285 285 2285 2285 170 171 XJAAAA OPAAAA AAAAxx +2387 405 1 3 7 7 87 387 387 2387 2387 174 175 VNAAAA PPAAAA HHHHxx +5015 406 1 3 5 15 15 15 1015 15 5015 30 31 XKAAAA QPAAAA OOOOxx +2087 407 1 3 7 7 87 87 87 2087 2087 174 175 HCAAAA RPAAAA VVVVxx +4938 408 0 2 8 18 38 938 938 4938 4938 76 77 YHAAAA SPAAAA AAAAxx +3635 409 1 3 5 15 35 635 1635 3635 3635 70 71 VJAAAA TPAAAA HHHHxx +7737 410 1 1 7 17 37 737 1737 2737 7737 74 75 PLAAAA UPAAAA OOOOxx +8056 411 0 0 6 16 56 56 56 3056 8056 112 113 WXAAAA VPAAAA VVVVxx +4502 412 0 2 2 2 2 502 502 4502 4502 4 5 ERAAAA WPAAAA AAAAxx +54 413 0 2 4 14 54 54 54 54 54 108 109 CCAAAA XPAAAA HHHHxx +3182 414 0 2 2 2 82 182 1182 3182 3182 164 165 KSAAAA YPAAAA OOOOxx +3718 415 0 2 8 18 18 718 1718 3718 3718 36 37 ANAAAA ZPAAAA VVVVxx +3989 416 1 1 9 9 89 989 1989 3989 3989 178 179 LXAAAA AQAAAA AAAAxx +8028 417 0 0 8 8 28 28 28 3028 8028 56 57 UWAAAA BQAAAA HHHHxx +1426 418 0 2 6 6 26 426 1426 1426 1426 52 53 WCAAAA CQAAAA OOOOxx +3801 419 1 1 1 1 1 801 1801 3801 3801 2 3 FQAAAA DQAAAA VVVVxx +241 420 1 1 1 1 41 241 241 241 241 82 83 HJAAAA EQAAAA AAAAxx +8000 421 0 0 0 0 0 0 0 3000 8000 0 1 SVAAAA FQAAAA HHHHxx +8357 422 1 1 7 17 57 357 357 3357 8357 114 115 LJAAAA GQAAAA OOOOxx +7548 423 0 0 8 8 48 548 1548 2548 7548 96 97 IEAAAA HQAAAA VVVVxx +7307 424 1 3 7 7 7 307 1307 2307 7307 14 15 BVAAAA IQAAAA AAAAxx +2275 425 1 3 5 15 75 275 275 2275 2275 150 151 NJAAAA JQAAAA HHHHxx +2718 426 0 2 8 18 18 718 718 2718 2718 36 37 OAAAAA KQAAAA OOOOxx +7068 427 0 0 8 8 68 68 1068 2068 7068 136 137 WLAAAA LQAAAA VVVVxx +3181 428 1 1 1 1 81 181 1181 3181 3181 162 163 JSAAAA MQAAAA AAAAxx +749 429 1 1 9 9 49 749 749 749 749 98 99 VCAAAA NQAAAA HHHHxx +5195 430 1 3 5 15 95 195 1195 195 5195 190 191 VRAAAA OQAAAA OOOOxx +6136 431 0 0 6 16 36 136 136 1136 6136 72 73 ACAAAA PQAAAA VVVVxx +8012 432 0 0 2 12 12 12 12 3012 8012 24 25 EWAAAA QQAAAA AAAAxx +3957 433 1 1 7 17 57 957 1957 3957 3957 114 115 FWAAAA RQAAAA HHHHxx +3083 434 1 3 3 3 83 83 1083 3083 3083 166 167 POAAAA SQAAAA OOOOxx +9997 435 1 1 7 17 97 997 1997 4997 9997 194 195 NUAAAA TQAAAA VVVVxx +3299 436 1 3 9 19 99 299 1299 3299 3299 198 199 XWAAAA UQAAAA AAAAxx +846 437 0 2 6 6 46 846 846 846 846 92 93 OGAAAA VQAAAA HHHHxx +2985 438 1 1 5 5 85 985 985 2985 2985 170 171 VKAAAA WQAAAA OOOOxx +9238 439 0 2 8 18 38 238 1238 4238 9238 76 77 IRAAAA XQAAAA VVVVxx +1403 440 1 3 3 3 3 403 1403 1403 1403 6 7 ZBAAAA YQAAAA AAAAxx +5563 441 1 3 3 3 63 563 1563 563 5563 126 127 ZFAAAA ZQAAAA HHHHxx +7965 442 1 1 5 5 65 965 1965 2965 7965 130 131 JUAAAA ARAAAA OOOOxx +4512 443 0 0 2 12 12 512 512 4512 4512 24 25 ORAAAA BRAAAA VVVVxx +9730 444 0 2 0 10 30 730 1730 4730 9730 60 61 GKAAAA CRAAAA AAAAxx +1129 445 1 1 9 9 29 129 1129 1129 1129 58 59 LRAAAA DRAAAA HHHHxx +2624 446 0 0 4 4 24 624 624 2624 2624 48 49 YWAAAA ERAAAA OOOOxx +8178 447 0 2 8 18 78 178 178 3178 8178 156 157 OCAAAA FRAAAA VVVVxx +6468 448 0 0 8 8 68 468 468 1468 6468 136 137 UOAAAA GRAAAA AAAAxx +3027 449 1 3 7 7 27 27 1027 3027 3027 54 55 LMAAAA HRAAAA HHHHxx +3845 450 1 1 5 5 45 845 1845 3845 3845 90 91 XRAAAA IRAAAA OOOOxx +786 451 0 2 6 6 86 786 786 786 786 172 173 GEAAAA JRAAAA VVVVxx +4971 452 1 3 1 11 71 971 971 4971 4971 142 143 FJAAAA KRAAAA AAAAxx +1542 453 0 2 2 2 42 542 1542 1542 1542 84 85 IHAAAA LRAAAA HHHHxx +7967 454 1 3 7 7 67 967 1967 2967 7967 134 135 LUAAAA MRAAAA OOOOxx +443 455 1 3 3 3 43 443 443 443 443 86 87 BRAAAA NRAAAA VVVVxx +7318 456 0 2 8 18 18 318 1318 2318 7318 36 37 MVAAAA ORAAAA AAAAxx +4913 457 1 1 3 13 13 913 913 4913 4913 26 27 ZGAAAA PRAAAA HHHHxx +9466 458 0 2 6 6 66 466 1466 4466 9466 132 133 CAAAAA QRAAAA OOOOxx +7866 459 0 2 6 6 66 866 1866 2866 7866 132 133 OQAAAA RRAAAA VVVVxx +784 460 0 0 4 4 84 784 784 784 784 168 169 EEAAAA SRAAAA AAAAxx +9040 461 0 0 0 0 40 40 1040 4040 9040 80 81 SJAAAA TRAAAA HHHHxx +3954 462 0 2 4 14 54 954 1954 3954 3954 108 109 CWAAAA URAAAA OOOOxx +4183 463 1 3 3 3 83 183 183 4183 4183 166 167 XEAAAA VRAAAA VVVVxx +3608 464 0 0 8 8 8 608 1608 3608 3608 16 17 UIAAAA WRAAAA AAAAxx +7630 465 0 2 0 10 30 630 1630 2630 7630 60 61 MHAAAA XRAAAA HHHHxx +590 466 0 2 0 10 90 590 590 590 590 180 181 SWAAAA YRAAAA OOOOxx +3453 467 1 1 3 13 53 453 1453 3453 3453 106 107 VCAAAA ZRAAAA VVVVxx +7757 468 1 1 7 17 57 757 1757 2757 7757 114 115 JMAAAA ASAAAA AAAAxx +7394 469 0 2 4 14 94 394 1394 2394 7394 188 189 KYAAAA BSAAAA HHHHxx +396 470 0 0 6 16 96 396 396 396 396 192 193 GPAAAA CSAAAA OOOOxx +7873 471 1 1 3 13 73 873 1873 2873 7873 146 147 VQAAAA DSAAAA VVVVxx +1553 472 1 1 3 13 53 553 1553 1553 1553 106 107 THAAAA ESAAAA AAAAxx +598 473 0 2 8 18 98 598 598 598 598 196 197 AXAAAA FSAAAA HHHHxx +7191 474 1 3 1 11 91 191 1191 2191 7191 182 183 PQAAAA GSAAAA OOOOxx +8116 475 0 0 6 16 16 116 116 3116 8116 32 33 EAAAAA HSAAAA VVVVxx +2516 476 0 0 6 16 16 516 516 2516 2516 32 33 USAAAA ISAAAA AAAAxx +7750 477 0 2 0 10 50 750 1750 2750 7750 100 101 CMAAAA JSAAAA HHHHxx +6625 478 1 1 5 5 25 625 625 1625 6625 50 51 VUAAAA KSAAAA OOOOxx +8838 479 0 2 8 18 38 838 838 3838 8838 76 77 YBAAAA LSAAAA VVVVxx +4636 480 0 0 6 16 36 636 636 4636 4636 72 73 IWAAAA MSAAAA AAAAxx +7627 481 1 3 7 7 27 627 1627 2627 7627 54 55 JHAAAA NSAAAA HHHHxx +1690 482 0 2 0 10 90 690 1690 1690 1690 180 181 ANAAAA OSAAAA OOOOxx +7071 483 1 3 1 11 71 71 1071 2071 7071 142 143 ZLAAAA PSAAAA VVVVxx +2081 484 1 1 1 1 81 81 81 2081 2081 162 163 BCAAAA QSAAAA AAAAxx +7138 485 0 2 8 18 38 138 1138 2138 7138 76 77 OOAAAA RSAAAA HHHHxx +864 486 0 0 4 4 64 864 864 864 864 128 129 GHAAAA SSAAAA OOOOxx +6392 487 0 0 2 12 92 392 392 1392 6392 184 185 WLAAAA TSAAAA VVVVxx +7544 488 0 0 4 4 44 544 1544 2544 7544 88 89 EEAAAA USAAAA AAAAxx +5438 489 0 2 8 18 38 438 1438 438 5438 76 77 EBAAAA VSAAAA HHHHxx +7099 490 1 3 9 19 99 99 1099 2099 7099 198 199 BNAAAA WSAAAA OOOOxx +5157 491 1 1 7 17 57 157 1157 157 5157 114 115 JQAAAA XSAAAA VVVVxx +3391 492 1 3 1 11 91 391 1391 3391 3391 182 183 LAAAAA YSAAAA AAAAxx +3805 493 1 1 5 5 5 805 1805 3805 3805 10 11 JQAAAA ZSAAAA HHHHxx +2110 494 0 2 0 10 10 110 110 2110 2110 20 21 EDAAAA ATAAAA OOOOxx +3176 495 0 0 6 16 76 176 1176 3176 3176 152 153 ESAAAA BTAAAA VVVVxx +5918 496 0 2 8 18 18 918 1918 918 5918 36 37 QTAAAA CTAAAA AAAAxx +1218 497 0 2 8 18 18 218 1218 1218 1218 36 37 WUAAAA DTAAAA HHHHxx +6683 498 1 3 3 3 83 683 683 1683 6683 166 167 BXAAAA ETAAAA OOOOxx +914 499 0 2 4 14 14 914 914 914 914 28 29 EJAAAA FTAAAA VVVVxx +4737 500 1 1 7 17 37 737 737 4737 4737 74 75 FAAAAA GTAAAA AAAAxx +7286 501 0 2 6 6 86 286 1286 2286 7286 172 173 GUAAAA HTAAAA HHHHxx +9975 502 1 3 5 15 75 975 1975 4975 9975 150 151 RTAAAA ITAAAA OOOOxx +8030 503 0 2 0 10 30 30 30 3030 8030 60 61 WWAAAA JTAAAA VVVVxx +7364 504 0 0 4 4 64 364 1364 2364 7364 128 129 GXAAAA KTAAAA AAAAxx +1389 505 1 1 9 9 89 389 1389 1389 1389 178 179 LBAAAA LTAAAA HHHHxx +4025 506 1 1 5 5 25 25 25 4025 4025 50 51 VYAAAA MTAAAA OOOOxx +4835 507 1 3 5 15 35 835 835 4835 4835 70 71 ZDAAAA NTAAAA VVVVxx +8045 508 1 1 5 5 45 45 45 3045 8045 90 91 LXAAAA OTAAAA AAAAxx +1864 509 0 0 4 4 64 864 1864 1864 1864 128 129 STAAAA PTAAAA HHHHxx +3313 510 1 1 3 13 13 313 1313 3313 3313 26 27 LXAAAA QTAAAA OOOOxx +2384 511 0 0 4 4 84 384 384 2384 2384 168 169 SNAAAA RTAAAA VVVVxx +6115 512 1 3 5 15 15 115 115 1115 6115 30 31 FBAAAA STAAAA AAAAxx +5705 513 1 1 5 5 5 705 1705 705 5705 10 11 LLAAAA TTAAAA HHHHxx +9269 514 1 1 9 9 69 269 1269 4269 9269 138 139 NSAAAA UTAAAA OOOOxx +3379 515 1 3 9 19 79 379 1379 3379 3379 158 159 ZZAAAA VTAAAA VVVVxx +8205 516 1 1 5 5 5 205 205 3205 8205 10 11 PDAAAA WTAAAA AAAAxx +6575 517 1 3 5 15 75 575 575 1575 6575 150 151 XSAAAA XTAAAA HHHHxx +486 518 0 2 6 6 86 486 486 486 486 172 173 SSAAAA YTAAAA OOOOxx +4894 519 0 2 4 14 94 894 894 4894 4894 188 189 GGAAAA ZTAAAA VVVVxx +3090 520 0 2 0 10 90 90 1090 3090 3090 180 181 WOAAAA AUAAAA AAAAxx +759 521 1 3 9 19 59 759 759 759 759 118 119 FDAAAA BUAAAA HHHHxx +4864 522 0 0 4 4 64 864 864 4864 4864 128 129 CFAAAA CUAAAA OOOOxx +4083 523 1 3 3 3 83 83 83 4083 4083 166 167 BBAAAA DUAAAA VVVVxx +6918 524 0 2 8 18 18 918 918 1918 6918 36 37 CGAAAA EUAAAA AAAAxx +8146 525 0 2 6 6 46 146 146 3146 8146 92 93 IBAAAA FUAAAA HHHHxx +1523 526 1 3 3 3 23 523 1523 1523 1523 46 47 PGAAAA GUAAAA OOOOxx +1591 527 1 3 1 11 91 591 1591 1591 1591 182 183 FJAAAA HUAAAA VVVVxx +3343 528 1 3 3 3 43 343 1343 3343 3343 86 87 PYAAAA IUAAAA AAAAxx +1391 529 1 3 1 11 91 391 1391 1391 1391 182 183 NBAAAA JUAAAA HHHHxx +9963 530 1 3 3 3 63 963 1963 4963 9963 126 127 FTAAAA KUAAAA OOOOxx +2423 531 1 3 3 3 23 423 423 2423 2423 46 47 FPAAAA LUAAAA VVVVxx +1822 532 0 2 2 2 22 822 1822 1822 1822 44 45 CSAAAA MUAAAA AAAAxx +8706 533 0 2 6 6 6 706 706 3706 8706 12 13 WWAAAA NUAAAA HHHHxx +3001 534 1 1 1 1 1 1 1001 3001 3001 2 3 LLAAAA OUAAAA OOOOxx +6707 535 1 3 7 7 7 707 707 1707 6707 14 15 ZXAAAA PUAAAA VVVVxx +2121 536 1 1 1 1 21 121 121 2121 2121 42 43 PDAAAA QUAAAA AAAAxx +5814 537 0 2 4 14 14 814 1814 814 5814 28 29 QPAAAA RUAAAA HHHHxx +2659 538 1 3 9 19 59 659 659 2659 2659 118 119 HYAAAA SUAAAA OOOOxx +2016 539 0 0 6 16 16 16 16 2016 2016 32 33 OZAAAA TUAAAA VVVVxx +4286 540 0 2 6 6 86 286 286 4286 4286 172 173 WIAAAA UUAAAA AAAAxx +9205 541 1 1 5 5 5 205 1205 4205 9205 10 11 BQAAAA VUAAAA HHHHxx +3496 542 0 0 6 16 96 496 1496 3496 3496 192 193 MEAAAA WUAAAA OOOOxx +5333 543 1 1 3 13 33 333 1333 333 5333 66 67 DXAAAA XUAAAA VVVVxx +5571 544 1 3 1 11 71 571 1571 571 5571 142 143 HGAAAA YUAAAA AAAAxx +1696 545 0 0 6 16 96 696 1696 1696 1696 192 193 GNAAAA ZUAAAA HHHHxx +4871 546 1 3 1 11 71 871 871 4871 4871 142 143 JFAAAA AVAAAA OOOOxx +4852 547 0 0 2 12 52 852 852 4852 4852 104 105 QEAAAA BVAAAA VVVVxx +8483 548 1 3 3 3 83 483 483 3483 8483 166 167 HOAAAA CVAAAA AAAAxx +1376 549 0 0 6 16 76 376 1376 1376 1376 152 153 YAAAAA DVAAAA HHHHxx +5456 550 0 0 6 16 56 456 1456 456 5456 112 113 WBAAAA EVAAAA OOOOxx +499 551 1 3 9 19 99 499 499 499 499 198 199 FTAAAA FVAAAA VVVVxx +3463 552 1 3 3 3 63 463 1463 3463 3463 126 127 FDAAAA GVAAAA AAAAxx +7426 553 0 2 6 6 26 426 1426 2426 7426 52 53 QZAAAA HVAAAA HHHHxx +5341 554 1 1 1 1 41 341 1341 341 5341 82 83 LXAAAA IVAAAA OOOOxx +9309 555 1 1 9 9 9 309 1309 4309 9309 18 19 BUAAAA JVAAAA VVVVxx +2055 556 1 3 5 15 55 55 55 2055 2055 110 111 BBAAAA KVAAAA AAAAxx +2199 557 1 3 9 19 99 199 199 2199 2199 198 199 PGAAAA LVAAAA HHHHxx +7235 558 1 3 5 15 35 235 1235 2235 7235 70 71 HSAAAA MVAAAA OOOOxx +8661 559 1 1 1 1 61 661 661 3661 8661 122 123 DVAAAA NVAAAA VVVVxx +9494 560 0 2 4 14 94 494 1494 4494 9494 188 189 EBAAAA OVAAAA AAAAxx +935 561 1 3 5 15 35 935 935 935 935 70 71 ZJAAAA PVAAAA HHHHxx +7044 562 0 0 4 4 44 44 1044 2044 7044 88 89 YKAAAA QVAAAA OOOOxx +1974 563 0 2 4 14 74 974 1974 1974 1974 148 149 YXAAAA RVAAAA VVVVxx +9679 564 1 3 9 19 79 679 1679 4679 9679 158 159 HIAAAA SVAAAA AAAAxx +9822 565 0 2 2 2 22 822 1822 4822 9822 44 45 UNAAAA TVAAAA HHHHxx +4088 566 0 0 8 8 88 88 88 4088 4088 176 177 GBAAAA UVAAAA OOOOxx +1749 567 1 1 9 9 49 749 1749 1749 1749 98 99 HPAAAA VVAAAA VVVVxx +2116 568 0 0 6 16 16 116 116 2116 2116 32 33 KDAAAA WVAAAA AAAAxx +976 569 0 0 6 16 76 976 976 976 976 152 153 OLAAAA XVAAAA HHHHxx +8689 570 1 1 9 9 89 689 689 3689 8689 178 179 FWAAAA YVAAAA OOOOxx +2563 571 1 3 3 3 63 563 563 2563 2563 126 127 PUAAAA ZVAAAA VVVVxx +7195 572 1 3 5 15 95 195 1195 2195 7195 190 191 TQAAAA AWAAAA AAAAxx +9985 573 1 1 5 5 85 985 1985 4985 9985 170 171 BUAAAA BWAAAA HHHHxx +7699 574 1 3 9 19 99 699 1699 2699 7699 198 199 DKAAAA CWAAAA OOOOxx +5311 575 1 3 1 11 11 311 1311 311 5311 22 23 HWAAAA DWAAAA VVVVxx +295 576 1 3 5 15 95 295 295 295 295 190 191 JLAAAA EWAAAA AAAAxx +8214 577 0 2 4 14 14 214 214 3214 8214 28 29 YDAAAA FWAAAA HHHHxx +3275 578 1 3 5 15 75 275 1275 3275 3275 150 151 ZVAAAA GWAAAA OOOOxx +9646 579 0 2 6 6 46 646 1646 4646 9646 92 93 AHAAAA HWAAAA VVVVxx +1908 580 0 0 8 8 8 908 1908 1908 1908 16 17 KVAAAA IWAAAA AAAAxx +3858 581 0 2 8 18 58 858 1858 3858 3858 116 117 KSAAAA JWAAAA HHHHxx +9362 582 0 2 2 2 62 362 1362 4362 9362 124 125 CWAAAA KWAAAA OOOOxx +9307 583 1 3 7 7 7 307 1307 4307 9307 14 15 ZTAAAA LWAAAA VVVVxx +6124 584 0 0 4 4 24 124 124 1124 6124 48 49 OBAAAA MWAAAA AAAAxx +2405 585 1 1 5 5 5 405 405 2405 2405 10 11 NOAAAA NWAAAA HHHHxx +8422 586 0 2 2 2 22 422 422 3422 8422 44 45 YLAAAA OWAAAA OOOOxx +393 587 1 1 3 13 93 393 393 393 393 186 187 DPAAAA PWAAAA VVVVxx +8973 588 1 1 3 13 73 973 973 3973 8973 146 147 DHAAAA QWAAAA AAAAxx +5171 589 1 3 1 11 71 171 1171 171 5171 142 143 XQAAAA RWAAAA HHHHxx +4929 590 1 1 9 9 29 929 929 4929 4929 58 59 PHAAAA SWAAAA OOOOxx +6935 591 1 3 5 15 35 935 935 1935 6935 70 71 TGAAAA TWAAAA VVVVxx +8584 592 0 0 4 4 84 584 584 3584 8584 168 169 ESAAAA UWAAAA AAAAxx +1035 593 1 3 5 15 35 35 1035 1035 1035 70 71 VNAAAA VWAAAA HHHHxx +3734 594 0 2 4 14 34 734 1734 3734 3734 68 69 QNAAAA WWAAAA OOOOxx +1458 595 0 2 8 18 58 458 1458 1458 1458 116 117 CEAAAA XWAAAA VVVVxx +8746 596 0 2 6 6 46 746 746 3746 8746 92 93 KYAAAA YWAAAA AAAAxx +1677 597 1 1 7 17 77 677 1677 1677 1677 154 155 NMAAAA ZWAAAA HHHHxx +8502 598 0 2 2 2 2 502 502 3502 8502 4 5 APAAAA AXAAAA OOOOxx +7752 599 0 0 2 12 52 752 1752 2752 7752 104 105 EMAAAA BXAAAA VVVVxx +2556 600 0 0 6 16 56 556 556 2556 2556 112 113 IUAAAA CXAAAA AAAAxx +6426 601 0 2 6 6 26 426 426 1426 6426 52 53 ENAAAA DXAAAA HHHHxx +8420 602 0 0 0 0 20 420 420 3420 8420 40 41 WLAAAA EXAAAA OOOOxx +4462 603 0 2 2 2 62 462 462 4462 4462 124 125 QPAAAA FXAAAA VVVVxx +1378 604 0 2 8 18 78 378 1378 1378 1378 156 157 ABAAAA GXAAAA AAAAxx +1387 605 1 3 7 7 87 387 1387 1387 1387 174 175 JBAAAA HXAAAA HHHHxx +8094 606 0 2 4 14 94 94 94 3094 8094 188 189 IZAAAA IXAAAA OOOOxx +7247 607 1 3 7 7 47 247 1247 2247 7247 94 95 TSAAAA JXAAAA VVVVxx +4261 608 1 1 1 1 61 261 261 4261 4261 122 123 XHAAAA KXAAAA AAAAxx +5029 609 1 1 9 9 29 29 1029 29 5029 58 59 LLAAAA LXAAAA HHHHxx +3625 610 1 1 5 5 25 625 1625 3625 3625 50 51 LJAAAA MXAAAA OOOOxx +8068 611 0 0 8 8 68 68 68 3068 8068 136 137 IYAAAA NXAAAA VVVVxx +102 612 0 2 2 2 2 102 102 102 102 4 5 YDAAAA OXAAAA AAAAxx +5596 613 0 0 6 16 96 596 1596 596 5596 192 193 GHAAAA PXAAAA HHHHxx +5872 614 0 0 2 12 72 872 1872 872 5872 144 145 WRAAAA QXAAAA OOOOxx +4742 615 0 2 2 2 42 742 742 4742 4742 84 85 KAAAAA RXAAAA VVVVxx +2117 616 1 1 7 17 17 117 117 2117 2117 34 35 LDAAAA SXAAAA AAAAxx +3945 617 1 1 5 5 45 945 1945 3945 3945 90 91 TVAAAA TXAAAA HHHHxx +7483 618 1 3 3 3 83 483 1483 2483 7483 166 167 VBAAAA UXAAAA OOOOxx +4455 619 1 3 5 15 55 455 455 4455 4455 110 111 JPAAAA VXAAAA VVVVxx +609 620 1 1 9 9 9 609 609 609 609 18 19 LXAAAA WXAAAA AAAAxx +9829 621 1 1 9 9 29 829 1829 4829 9829 58 59 BOAAAA XXAAAA HHHHxx +4857 622 1 1 7 17 57 857 857 4857 4857 114 115 VEAAAA YXAAAA OOOOxx +3314 623 0 2 4 14 14 314 1314 3314 3314 28 29 MXAAAA ZXAAAA VVVVxx +5353 624 1 1 3 13 53 353 1353 353 5353 106 107 XXAAAA AYAAAA AAAAxx +4909 625 1 1 9 9 9 909 909 4909 4909 18 19 VGAAAA BYAAAA HHHHxx +7597 626 1 1 7 17 97 597 1597 2597 7597 194 195 FGAAAA CYAAAA OOOOxx +2683 627 1 3 3 3 83 683 683 2683 2683 166 167 FZAAAA DYAAAA VVVVxx +3223 628 1 3 3 3 23 223 1223 3223 3223 46 47 ZTAAAA EYAAAA AAAAxx +5363 629 1 3 3 3 63 363 1363 363 5363 126 127 HYAAAA FYAAAA HHHHxx +4578 630 0 2 8 18 78 578 578 4578 4578 156 157 CUAAAA GYAAAA OOOOxx +5544 631 0 0 4 4 44 544 1544 544 5544 88 89 GFAAAA HYAAAA VVVVxx +1589 632 1 1 9 9 89 589 1589 1589 1589 178 179 DJAAAA IYAAAA AAAAxx +7412 633 0 0 2 12 12 412 1412 2412 7412 24 25 CZAAAA JYAAAA HHHHxx +3803 634 1 3 3 3 3 803 1803 3803 3803 6 7 HQAAAA KYAAAA OOOOxx +6179 635 1 3 9 19 79 179 179 1179 6179 158 159 RDAAAA LYAAAA VVVVxx +5588 636 0 0 8 8 88 588 1588 588 5588 176 177 YGAAAA MYAAAA AAAAxx +2134 637 0 2 4 14 34 134 134 2134 2134 68 69 CEAAAA NYAAAA HHHHxx +4383 638 1 3 3 3 83 383 383 4383 4383 166 167 PMAAAA OYAAAA OOOOxx +6995 639 1 3 5 15 95 995 995 1995 6995 190 191 BJAAAA PYAAAA VVVVxx +6598 640 0 2 8 18 98 598 598 1598 6598 196 197 UTAAAA QYAAAA AAAAxx +8731 641 1 3 1 11 31 731 731 3731 8731 62 63 VXAAAA RYAAAA HHHHxx +7177 642 1 1 7 17 77 177 1177 2177 7177 154 155 BQAAAA SYAAAA OOOOxx +6578 643 0 2 8 18 78 578 578 1578 6578 156 157 ATAAAA TYAAAA VVVVxx +9393 644 1 1 3 13 93 393 1393 4393 9393 186 187 HXAAAA UYAAAA AAAAxx +1276 645 0 0 6 16 76 276 1276 1276 1276 152 153 CXAAAA VYAAAA HHHHxx +8766 646 0 2 6 6 66 766 766 3766 8766 132 133 EZAAAA WYAAAA OOOOxx +1015 647 1 3 5 15 15 15 1015 1015 1015 30 31 BNAAAA XYAAAA VVVVxx +4396 648 0 0 6 16 96 396 396 4396 4396 192 193 CNAAAA YYAAAA AAAAxx +5564 649 0 0 4 4 64 564 1564 564 5564 128 129 AGAAAA ZYAAAA HHHHxx +927 650 1 3 7 7 27 927 927 927 927 54 55 RJAAAA AZAAAA OOOOxx +3306 651 0 2 6 6 6 306 1306 3306 3306 12 13 EXAAAA BZAAAA VVVVxx +1615 652 1 3 5 15 15 615 1615 1615 1615 30 31 DKAAAA CZAAAA AAAAxx +4550 653 0 2 0 10 50 550 550 4550 4550 100 101 ATAAAA DZAAAA HHHHxx +2468 654 0 0 8 8 68 468 468 2468 2468 136 137 YQAAAA EZAAAA OOOOxx +5336 655 0 0 6 16 36 336 1336 336 5336 72 73 GXAAAA FZAAAA VVVVxx +4471 656 1 3 1 11 71 471 471 4471 4471 142 143 ZPAAAA GZAAAA AAAAxx +8085 657 1 1 5 5 85 85 85 3085 8085 170 171 ZYAAAA HZAAAA HHHHxx +540 658 0 0 0 0 40 540 540 540 540 80 81 UUAAAA IZAAAA OOOOxx +5108 659 0 0 8 8 8 108 1108 108 5108 16 17 MOAAAA JZAAAA VVVVxx +8015 660 1 3 5 15 15 15 15 3015 8015 30 31 HWAAAA KZAAAA AAAAxx +2857 661 1 1 7 17 57 857 857 2857 2857 114 115 XFAAAA LZAAAA HHHHxx +9472 662 0 0 2 12 72 472 1472 4472 9472 144 145 IAAAAA MZAAAA OOOOxx +5666 663 0 2 6 6 66 666 1666 666 5666 132 133 YJAAAA NZAAAA VVVVxx +3555 664 1 3 5 15 55 555 1555 3555 3555 110 111 TGAAAA OZAAAA AAAAxx +378 665 0 2 8 18 78 378 378 378 378 156 157 OOAAAA PZAAAA HHHHxx +4466 666 0 2 6 6 66 466 466 4466 4466 132 133 UPAAAA QZAAAA OOOOxx +3247 667 1 3 7 7 47 247 1247 3247 3247 94 95 XUAAAA RZAAAA VVVVxx +6570 668 0 2 0 10 70 570 570 1570 6570 140 141 SSAAAA SZAAAA AAAAxx +5655 669 1 3 5 15 55 655 1655 655 5655 110 111 NJAAAA TZAAAA HHHHxx +917 670 1 1 7 17 17 917 917 917 917 34 35 HJAAAA UZAAAA OOOOxx +3637 671 1 1 7 17 37 637 1637 3637 3637 74 75 XJAAAA VZAAAA VVVVxx +3668 672 0 0 8 8 68 668 1668 3668 3668 136 137 CLAAAA WZAAAA AAAAxx +5644 673 0 0 4 4 44 644 1644 644 5644 88 89 CJAAAA XZAAAA HHHHxx +8286 674 0 2 6 6 86 286 286 3286 8286 172 173 SGAAAA YZAAAA OOOOxx +6896 675 0 0 6 16 96 896 896 1896 6896 192 193 GFAAAA ZZAAAA VVVVxx +2870 676 0 2 0 10 70 870 870 2870 2870 140 141 KGAAAA AABAAA AAAAxx +8041 677 1 1 1 1 41 41 41 3041 8041 82 83 HXAAAA BABAAA HHHHxx +8137 678 1 1 7 17 37 137 137 3137 8137 74 75 ZAAAAA CABAAA OOOOxx +4823 679 1 3 3 3 23 823 823 4823 4823 46 47 NDAAAA DABAAA VVVVxx +2438 680 0 2 8 18 38 438 438 2438 2438 76 77 UPAAAA EABAAA AAAAxx +6329 681 1 1 9 9 29 329 329 1329 6329 58 59 LJAAAA FABAAA HHHHxx +623 682 1 3 3 3 23 623 623 623 623 46 47 ZXAAAA GABAAA OOOOxx +1360 683 0 0 0 0 60 360 1360 1360 1360 120 121 IAAAAA HABAAA VVVVxx +7987 684 1 3 7 7 87 987 1987 2987 7987 174 175 FVAAAA IABAAA AAAAxx +9788 685 0 0 8 8 88 788 1788 4788 9788 176 177 MMAAAA JABAAA HHHHxx +3212 686 0 0 2 12 12 212 1212 3212 3212 24 25 OTAAAA KABAAA OOOOxx +2725 687 1 1 5 5 25 725 725 2725 2725 50 51 VAAAAA LABAAA VVVVxx +7837 688 1 1 7 17 37 837 1837 2837 7837 74 75 LPAAAA MABAAA AAAAxx +4746 689 0 2 6 6 46 746 746 4746 4746 92 93 OAAAAA NABAAA HHHHxx +3986 690 0 2 6 6 86 986 1986 3986 3986 172 173 IXAAAA OABAAA OOOOxx +9128 691 0 0 8 8 28 128 1128 4128 9128 56 57 CNAAAA PABAAA VVVVxx +5044 692 0 0 4 4 44 44 1044 44 5044 88 89 AMAAAA QABAAA AAAAxx +8132 693 0 0 2 12 32 132 132 3132 8132 64 65 UAAAAA RABAAA HHHHxx +9992 694 0 0 2 12 92 992 1992 4992 9992 184 185 IUAAAA SABAAA OOOOxx +8468 695 0 0 8 8 68 468 468 3468 8468 136 137 SNAAAA TABAAA VVVVxx +6876 696 0 0 6 16 76 876 876 1876 6876 152 153 MEAAAA UABAAA AAAAxx +3532 697 0 0 2 12 32 532 1532 3532 3532 64 65 WFAAAA VABAAA HHHHxx +2140 698 0 0 0 0 40 140 140 2140 2140 80 81 IEAAAA WABAAA OOOOxx +2183 699 1 3 3 3 83 183 183 2183 2183 166 167 ZFAAAA XABAAA VVVVxx +9766 700 0 2 6 6 66 766 1766 4766 9766 132 133 QLAAAA YABAAA AAAAxx +7943 701 1 3 3 3 43 943 1943 2943 7943 86 87 NTAAAA ZABAAA HHHHxx +9243 702 1 3 3 3 43 243 1243 4243 9243 86 87 NRAAAA ABBAAA OOOOxx +6241 703 1 1 1 1 41 241 241 1241 6241 82 83 BGAAAA BBBAAA VVVVxx +9540 704 0 0 0 0 40 540 1540 4540 9540 80 81 YCAAAA CBBAAA AAAAxx +7418 705 0 2 8 18 18 418 1418 2418 7418 36 37 IZAAAA DBBAAA HHHHxx +1603 706 1 3 3 3 3 603 1603 1603 1603 6 7 RJAAAA EBBAAA OOOOxx +8950 707 0 2 0 10 50 950 950 3950 8950 100 101 GGAAAA FBBAAA VVVVxx +6933 708 1 1 3 13 33 933 933 1933 6933 66 67 RGAAAA GBBAAA AAAAxx +2646 709 0 2 6 6 46 646 646 2646 2646 92 93 UXAAAA HBBAAA HHHHxx +3447 710 1 3 7 7 47 447 1447 3447 3447 94 95 PCAAAA IBBAAA OOOOxx +9957 711 1 1 7 17 57 957 1957 4957 9957 114 115 ZSAAAA JBBAAA VVVVxx +4623 712 1 3 3 3 23 623 623 4623 4623 46 47 VVAAAA KBBAAA AAAAxx +9058 713 0 2 8 18 58 58 1058 4058 9058 116 117 KKAAAA LBBAAA HHHHxx +7361 714 1 1 1 1 61 361 1361 2361 7361 122 123 DXAAAA MBBAAA OOOOxx +2489 715 1 1 9 9 89 489 489 2489 2489 178 179 TRAAAA NBBAAA VVVVxx +7643 716 1 3 3 3 43 643 1643 2643 7643 86 87 ZHAAAA OBBAAA AAAAxx +9166 717 0 2 6 6 66 166 1166 4166 9166 132 133 OOAAAA PBBAAA HHHHxx +7789 718 1 1 9 9 89 789 1789 2789 7789 178 179 PNAAAA QBBAAA OOOOxx +2332 719 0 0 2 12 32 332 332 2332 2332 64 65 SLAAAA RBBAAA VVVVxx +1832 720 0 0 2 12 32 832 1832 1832 1832 64 65 MSAAAA SBBAAA AAAAxx +8375 721 1 3 5 15 75 375 375 3375 8375 150 151 DKAAAA TBBAAA HHHHxx +948 722 0 0 8 8 48 948 948 948 948 96 97 MKAAAA UBBAAA OOOOxx +5613 723 1 1 3 13 13 613 1613 613 5613 26 27 XHAAAA VBBAAA VVVVxx +6310 724 0 2 0 10 10 310 310 1310 6310 20 21 SIAAAA WBBAAA AAAAxx +4254 725 0 2 4 14 54 254 254 4254 4254 108 109 QHAAAA XBBAAA HHHHxx +4260 726 0 0 0 0 60 260 260 4260 4260 120 121 WHAAAA YBBAAA OOOOxx +2060 727 0 0 0 0 60 60 60 2060 2060 120 121 GBAAAA ZBBAAA VVVVxx +4831 728 1 3 1 11 31 831 831 4831 4831 62 63 VDAAAA ACBAAA AAAAxx +6176 729 0 0 6 16 76 176 176 1176 6176 152 153 ODAAAA BCBAAA HHHHxx +6688 730 0 0 8 8 88 688 688 1688 6688 176 177 GXAAAA CCBAAA OOOOxx +5752 731 0 0 2 12 52 752 1752 752 5752 104 105 GNAAAA DCBAAA VVVVxx +8714 732 0 2 4 14 14 714 714 3714 8714 28 29 EXAAAA ECBAAA AAAAxx +6739 733 1 3 9 19 39 739 739 1739 6739 78 79 FZAAAA FCBAAA HHHHxx +7066 734 0 2 6 6 66 66 1066 2066 7066 132 133 ULAAAA GCBAAA OOOOxx +7250 735 0 2 0 10 50 250 1250 2250 7250 100 101 WSAAAA HCBAAA VVVVxx +3161 736 1 1 1 1 61 161 1161 3161 3161 122 123 PRAAAA ICBAAA AAAAxx +1411 737 1 3 1 11 11 411 1411 1411 1411 22 23 HCAAAA JCBAAA HHHHxx +9301 738 1 1 1 1 1 301 1301 4301 9301 2 3 TTAAAA KCBAAA OOOOxx +8324 739 0 0 4 4 24 324 324 3324 8324 48 49 EIAAAA LCBAAA VVVVxx +9641 740 1 1 1 1 41 641 1641 4641 9641 82 83 VGAAAA MCBAAA AAAAxx +7077 741 1 1 7 17 77 77 1077 2077 7077 154 155 FMAAAA NCBAAA HHHHxx +9888 742 0 0 8 8 88 888 1888 4888 9888 176 177 IQAAAA OCBAAA OOOOxx +9909 743 1 1 9 9 9 909 1909 4909 9909 18 19 DRAAAA PCBAAA VVVVxx +2209 744 1 1 9 9 9 209 209 2209 2209 18 19 ZGAAAA QCBAAA AAAAxx +6904 745 0 0 4 4 4 904 904 1904 6904 8 9 OFAAAA RCBAAA HHHHxx +6608 746 0 0 8 8 8 608 608 1608 6608 16 17 EUAAAA SCBAAA OOOOxx +8400 747 0 0 0 0 0 400 400 3400 8400 0 1 CLAAAA TCBAAA VVVVxx +5124 748 0 0 4 4 24 124 1124 124 5124 48 49 CPAAAA UCBAAA AAAAxx +5484 749 0 0 4 4 84 484 1484 484 5484 168 169 YCAAAA VCBAAA HHHHxx +3575 750 1 3 5 15 75 575 1575 3575 3575 150 151 NHAAAA WCBAAA OOOOxx +9723 751 1 3 3 3 23 723 1723 4723 9723 46 47 ZJAAAA XCBAAA VVVVxx +360 752 0 0 0 0 60 360 360 360 360 120 121 WNAAAA YCBAAA AAAAxx +1059 753 1 3 9 19 59 59 1059 1059 1059 118 119 TOAAAA ZCBAAA HHHHxx +4941 754 1 1 1 1 41 941 941 4941 4941 82 83 BIAAAA ADBAAA OOOOxx +2535 755 1 3 5 15 35 535 535 2535 2535 70 71 NTAAAA BDBAAA VVVVxx +4119 756 1 3 9 19 19 119 119 4119 4119 38 39 LCAAAA CDBAAA AAAAxx +3725 757 1 1 5 5 25 725 1725 3725 3725 50 51 HNAAAA DDBAAA HHHHxx +4758 758 0 2 8 18 58 758 758 4758 4758 116 117 ABAAAA EDBAAA OOOOxx +9593 759 1 1 3 13 93 593 1593 4593 9593 186 187 ZEAAAA FDBAAA VVVVxx +4663 760 1 3 3 3 63 663 663 4663 4663 126 127 JXAAAA GDBAAA AAAAxx +7734 761 0 2 4 14 34 734 1734 2734 7734 68 69 MLAAAA HDBAAA HHHHxx +9156 762 0 0 6 16 56 156 1156 4156 9156 112 113 EOAAAA IDBAAA OOOOxx +8120 763 0 0 0 0 20 120 120 3120 8120 40 41 IAAAAA JDBAAA VVVVxx +4385 764 1 1 5 5 85 385 385 4385 4385 170 171 RMAAAA KDBAAA AAAAxx +2926 765 0 2 6 6 26 926 926 2926 2926 52 53 OIAAAA LDBAAA HHHHxx +4186 766 0 2 6 6 86 186 186 4186 4186 172 173 AFAAAA MDBAAA OOOOxx +2508 767 0 0 8 8 8 508 508 2508 2508 16 17 MSAAAA NDBAAA VVVVxx +4012 768 0 0 2 12 12 12 12 4012 4012 24 25 IYAAAA ODBAAA AAAAxx +6266 769 0 2 6 6 66 266 266 1266 6266 132 133 AHAAAA PDBAAA HHHHxx +3709 770 1 1 9 9 9 709 1709 3709 3709 18 19 RMAAAA QDBAAA OOOOxx +7289 771 1 1 9 9 89 289 1289 2289 7289 178 179 JUAAAA RDBAAA VVVVxx +8875 772 1 3 5 15 75 875 875 3875 8875 150 151 JDAAAA SDBAAA AAAAxx +4412 773 0 0 2 12 12 412 412 4412 4412 24 25 SNAAAA TDBAAA HHHHxx +3033 774 1 1 3 13 33 33 1033 3033 3033 66 67 RMAAAA UDBAAA OOOOxx +1645 775 1 1 5 5 45 645 1645 1645 1645 90 91 HLAAAA VDBAAA VVVVxx +3557 776 1 1 7 17 57 557 1557 3557 3557 114 115 VGAAAA WDBAAA AAAAxx +6316 777 0 0 6 16 16 316 316 1316 6316 32 33 YIAAAA XDBAAA HHHHxx +2054 778 0 2 4 14 54 54 54 2054 2054 108 109 ABAAAA YDBAAA OOOOxx +7031 779 1 3 1 11 31 31 1031 2031 7031 62 63 LKAAAA ZDBAAA VVVVxx +3405 780 1 1 5 5 5 405 1405 3405 3405 10 11 ZAAAAA AEBAAA AAAAxx +5343 781 1 3 3 3 43 343 1343 343 5343 86 87 NXAAAA BEBAAA HHHHxx +5240 782 0 0 0 0 40 240 1240 240 5240 80 81 OTAAAA CEBAAA OOOOxx +9650 783 0 2 0 10 50 650 1650 4650 9650 100 101 EHAAAA DEBAAA VVVVxx +3777 784 1 1 7 17 77 777 1777 3777 3777 154 155 HPAAAA EEBAAA AAAAxx +9041 785 1 1 1 1 41 41 1041 4041 9041 82 83 TJAAAA FEBAAA HHHHxx +6923 786 1 3 3 3 23 923 923 1923 6923 46 47 HGAAAA GEBAAA OOOOxx +2977 787 1 1 7 17 77 977 977 2977 2977 154 155 NKAAAA HEBAAA VVVVxx +5500 788 0 0 0 0 0 500 1500 500 5500 0 1 ODAAAA IEBAAA AAAAxx +1044 789 0 0 4 4 44 44 1044 1044 1044 88 89 EOAAAA JEBAAA HHHHxx +434 790 0 2 4 14 34 434 434 434 434 68 69 SQAAAA KEBAAA OOOOxx +611 791 1 3 1 11 11 611 611 611 611 22 23 NXAAAA LEBAAA VVVVxx +5760 792 0 0 0 0 60 760 1760 760 5760 120 121 ONAAAA MEBAAA AAAAxx +2445 793 1 1 5 5 45 445 445 2445 2445 90 91 BQAAAA NEBAAA HHHHxx +7098 794 0 2 8 18 98 98 1098 2098 7098 196 197 ANAAAA OEBAAA OOOOxx +2188 795 0 0 8 8 88 188 188 2188 2188 176 177 EGAAAA PEBAAA VVVVxx +4597 796 1 1 7 17 97 597 597 4597 4597 194 195 VUAAAA QEBAAA AAAAxx +1913 797 1 1 3 13 13 913 1913 1913 1913 26 27 PVAAAA REBAAA HHHHxx +8696 798 0 0 6 16 96 696 696 3696 8696 192 193 MWAAAA SEBAAA OOOOxx +3332 799 0 0 2 12 32 332 1332 3332 3332 64 65 EYAAAA TEBAAA VVVVxx +8760 800 0 0 0 0 60 760 760 3760 8760 120 121 YYAAAA UEBAAA AAAAxx +3215 801 1 3 5 15 15 215 1215 3215 3215 30 31 RTAAAA VEBAAA HHHHxx +1625 802 1 1 5 5 25 625 1625 1625 1625 50 51 NKAAAA WEBAAA OOOOxx +4219 803 1 3 9 19 19 219 219 4219 4219 38 39 HGAAAA XEBAAA VVVVxx +415 804 1 3 5 15 15 415 415 415 415 30 31 ZPAAAA YEBAAA AAAAxx +4242 805 0 2 2 2 42 242 242 4242 4242 84 85 EHAAAA ZEBAAA HHHHxx +8660 806 0 0 0 0 60 660 660 3660 8660 120 121 CVAAAA AFBAAA OOOOxx +6525 807 1 1 5 5 25 525 525 1525 6525 50 51 ZQAAAA BFBAAA VVVVxx +2141 808 1 1 1 1 41 141 141 2141 2141 82 83 JEAAAA CFBAAA AAAAxx +5152 809 0 0 2 12 52 152 1152 152 5152 104 105 EQAAAA DFBAAA HHHHxx +8560 810 0 0 0 0 60 560 560 3560 8560 120 121 GRAAAA EFBAAA OOOOxx +9835 811 1 3 5 15 35 835 1835 4835 9835 70 71 HOAAAA FFBAAA VVVVxx +2657 812 1 1 7 17 57 657 657 2657 2657 114 115 FYAAAA GFBAAA AAAAxx +6085 813 1 1 5 5 85 85 85 1085 6085 170 171 BAAAAA HFBAAA HHHHxx +6698 814 0 2 8 18 98 698 698 1698 6698 196 197 QXAAAA IFBAAA OOOOxx +5421 815 1 1 1 1 21 421 1421 421 5421 42 43 NAAAAA JFBAAA VVVVxx +6661 816 1 1 1 1 61 661 661 1661 6661 122 123 FWAAAA KFBAAA AAAAxx +5645 817 1 1 5 5 45 645 1645 645 5645 90 91 DJAAAA LFBAAA HHHHxx +1248 818 0 0 8 8 48 248 1248 1248 1248 96 97 AWAAAA MFBAAA OOOOxx +5690 819 0 2 0 10 90 690 1690 690 5690 180 181 WKAAAA NFBAAA VVVVxx +4762 820 0 2 2 2 62 762 762 4762 4762 124 125 EBAAAA OFBAAA AAAAxx +1455 821 1 3 5 15 55 455 1455 1455 1455 110 111 ZDAAAA PFBAAA HHHHxx +9846 822 0 2 6 6 46 846 1846 4846 9846 92 93 SOAAAA QFBAAA OOOOxx +5295 823 1 3 5 15 95 295 1295 295 5295 190 191 RVAAAA RFBAAA VVVVxx +2826 824 0 2 6 6 26 826 826 2826 2826 52 53 SEAAAA SFBAAA AAAAxx +7496 825 0 0 6 16 96 496 1496 2496 7496 192 193 ICAAAA TFBAAA HHHHxx +3024 826 0 0 4 4 24 24 1024 3024 3024 48 49 IMAAAA UFBAAA OOOOxx +4945 827 1 1 5 5 45 945 945 4945 4945 90 91 FIAAAA VFBAAA VVVVxx +4404 828 0 0 4 4 4 404 404 4404 4404 8 9 KNAAAA WFBAAA AAAAxx +9302 829 0 2 2 2 2 302 1302 4302 9302 4 5 UTAAAA XFBAAA HHHHxx +1286 830 0 2 6 6 86 286 1286 1286 1286 172 173 MXAAAA YFBAAA OOOOxx +8435 831 1 3 5 15 35 435 435 3435 8435 70 71 LMAAAA ZFBAAA VVVVxx +8969 832 1 1 9 9 69 969 969 3969 8969 138 139 ZGAAAA AGBAAA AAAAxx +3302 833 0 2 2 2 2 302 1302 3302 3302 4 5 AXAAAA BGBAAA HHHHxx +9753 834 1 1 3 13 53 753 1753 4753 9753 106 107 DLAAAA CGBAAA OOOOxx +9374 835 0 2 4 14 74 374 1374 4374 9374 148 149 OWAAAA DGBAAA VVVVxx +4907 836 1 3 7 7 7 907 907 4907 4907 14 15 TGAAAA EGBAAA AAAAxx +1659 837 1 3 9 19 59 659 1659 1659 1659 118 119 VLAAAA FGBAAA HHHHxx +5095 838 1 3 5 15 95 95 1095 95 5095 190 191 ZNAAAA GGBAAA OOOOxx +9446 839 0 2 6 6 46 446 1446 4446 9446 92 93 IZAAAA HGBAAA VVVVxx +8528 840 0 0 8 8 28 528 528 3528 8528 56 57 AQAAAA IGBAAA AAAAxx +4890 841 0 2 0 10 90 890 890 4890 4890 180 181 CGAAAA JGBAAA HHHHxx +1221 842 1 1 1 1 21 221 1221 1221 1221 42 43 ZUAAAA KGBAAA OOOOxx +5583 843 1 3 3 3 83 583 1583 583 5583 166 167 TGAAAA LGBAAA VVVVxx +7303 844 1 3 3 3 3 303 1303 2303 7303 6 7 XUAAAA MGBAAA AAAAxx +406 845 0 2 6 6 6 406 406 406 406 12 13 QPAAAA NGBAAA HHHHxx +7542 846 0 2 2 2 42 542 1542 2542 7542 84 85 CEAAAA OGBAAA OOOOxx +9507 847 1 3 7 7 7 507 1507 4507 9507 14 15 RBAAAA PGBAAA VVVVxx +9511 848 1 3 1 11 11 511 1511 4511 9511 22 23 VBAAAA QGBAAA AAAAxx +1373 849 1 1 3 13 73 373 1373 1373 1373 146 147 VAAAAA RGBAAA HHHHxx +6556 850 0 0 6 16 56 556 556 1556 6556 112 113 ESAAAA SGBAAA OOOOxx +4117 851 1 1 7 17 17 117 117 4117 4117 34 35 JCAAAA TGBAAA VVVVxx +7794 852 0 2 4 14 94 794 1794 2794 7794 188 189 UNAAAA UGBAAA AAAAxx +7170 853 0 2 0 10 70 170 1170 2170 7170 140 141 UPAAAA VGBAAA HHHHxx +5809 854 1 1 9 9 9 809 1809 809 5809 18 19 LPAAAA WGBAAA OOOOxx +7828 855 0 0 8 8 28 828 1828 2828 7828 56 57 CPAAAA XGBAAA VVVVxx +8046 856 0 2 6 6 46 46 46 3046 8046 92 93 MXAAAA YGBAAA AAAAxx +4833 857 1 1 3 13 33 833 833 4833 4833 66 67 XDAAAA ZGBAAA HHHHxx +2107 858 1 3 7 7 7 107 107 2107 2107 14 15 BDAAAA AHBAAA OOOOxx +4276 859 0 0 6 16 76 276 276 4276 4276 152 153 MIAAAA BHBAAA VVVVxx +9536 860 0 0 6 16 36 536 1536 4536 9536 72 73 UCAAAA CHBAAA AAAAxx +5549 861 1 1 9 9 49 549 1549 549 5549 98 99 LFAAAA DHBAAA HHHHxx +6427 862 1 3 7 7 27 427 427 1427 6427 54 55 FNAAAA EHBAAA OOOOxx +1382 863 0 2 2 2 82 382 1382 1382 1382 164 165 EBAAAA FHBAAA VVVVxx +3256 864 0 0 6 16 56 256 1256 3256 3256 112 113 GVAAAA GHBAAA AAAAxx +3270 865 0 2 0 10 70 270 1270 3270 3270 140 141 UVAAAA HHBAAA HHHHxx +4808 866 0 0 8 8 8 808 808 4808 4808 16 17 YCAAAA IHBAAA OOOOxx +7938 867 0 2 8 18 38 938 1938 2938 7938 76 77 ITAAAA JHBAAA VVVVxx +4405 868 1 1 5 5 5 405 405 4405 4405 10 11 LNAAAA KHBAAA AAAAxx +2264 869 0 0 4 4 64 264 264 2264 2264 128 129 CJAAAA LHBAAA HHHHxx +80 870 0 0 0 0 80 80 80 80 80 160 161 CDAAAA MHBAAA OOOOxx +320 871 0 0 0 0 20 320 320 320 320 40 41 IMAAAA NHBAAA VVVVxx +2383 872 1 3 3 3 83 383 383 2383 2383 166 167 RNAAAA OHBAAA AAAAxx +3146 873 0 2 6 6 46 146 1146 3146 3146 92 93 ARAAAA PHBAAA HHHHxx +6911 874 1 3 1 11 11 911 911 1911 6911 22 23 VFAAAA QHBAAA OOOOxx +7377 875 1 1 7 17 77 377 1377 2377 7377 154 155 TXAAAA RHBAAA VVVVxx +9965 876 1 1 5 5 65 965 1965 4965 9965 130 131 HTAAAA SHBAAA AAAAxx +8361 877 1 1 1 1 61 361 361 3361 8361 122 123 PJAAAA THBAAA HHHHxx +9417 878 1 1 7 17 17 417 1417 4417 9417 34 35 FYAAAA UHBAAA OOOOxx +2483 879 1 3 3 3 83 483 483 2483 2483 166 167 NRAAAA VHBAAA VVVVxx +9843 880 1 3 3 3 43 843 1843 4843 9843 86 87 POAAAA WHBAAA AAAAxx +6395 881 1 3 5 15 95 395 395 1395 6395 190 191 ZLAAAA XHBAAA HHHHxx +6444 882 0 0 4 4 44 444 444 1444 6444 88 89 WNAAAA YHBAAA OOOOxx +1820 883 0 0 0 0 20 820 1820 1820 1820 40 41 ASAAAA ZHBAAA VVVVxx +2768 884 0 0 8 8 68 768 768 2768 2768 136 137 MCAAAA AIBAAA AAAAxx +5413 885 1 1 3 13 13 413 1413 413 5413 26 27 FAAAAA BIBAAA HHHHxx +2923 886 1 3 3 3 23 923 923 2923 2923 46 47 LIAAAA CIBAAA OOOOxx +5286 887 0 2 6 6 86 286 1286 286 5286 172 173 IVAAAA DIBAAA VVVVxx +6126 888 0 2 6 6 26 126 126 1126 6126 52 53 QBAAAA EIBAAA AAAAxx +8343 889 1 3 3 3 43 343 343 3343 8343 86 87 XIAAAA FIBAAA HHHHxx +6010 890 0 2 0 10 10 10 10 1010 6010 20 21 EXAAAA GIBAAA OOOOxx +4177 891 1 1 7 17 77 177 177 4177 4177 154 155 REAAAA HIBAAA VVVVxx +5808 892 0 0 8 8 8 808 1808 808 5808 16 17 KPAAAA IIBAAA AAAAxx +4859 893 1 3 9 19 59 859 859 4859 4859 118 119 XEAAAA JIBAAA HHHHxx +9252 894 0 0 2 12 52 252 1252 4252 9252 104 105 WRAAAA KIBAAA OOOOxx +2941 895 1 1 1 1 41 941 941 2941 2941 82 83 DJAAAA LIBAAA VVVVxx +8693 896 1 1 3 13 93 693 693 3693 8693 186 187 JWAAAA MIBAAA AAAAxx +4432 897 0 0 2 12 32 432 432 4432 4432 64 65 MOAAAA NIBAAA HHHHxx +2371 898 1 3 1 11 71 371 371 2371 2371 142 143 FNAAAA OIBAAA OOOOxx +7546 899 0 2 6 6 46 546 1546 2546 7546 92 93 GEAAAA PIBAAA VVVVxx +1369 900 1 1 9 9 69 369 1369 1369 1369 138 139 RAAAAA QIBAAA AAAAxx +4687 901 1 3 7 7 87 687 687 4687 4687 174 175 HYAAAA RIBAAA HHHHxx +8941 902 1 1 1 1 41 941 941 3941 8941 82 83 XFAAAA SIBAAA OOOOxx +226 903 0 2 6 6 26 226 226 226 226 52 53 SIAAAA TIBAAA VVVVxx +3493 904 1 1 3 13 93 493 1493 3493 3493 186 187 JEAAAA UIBAAA AAAAxx +6433 905 1 1 3 13 33 433 433 1433 6433 66 67 LNAAAA VIBAAA HHHHxx +9189 906 1 1 9 9 89 189 1189 4189 9189 178 179 LPAAAA WIBAAA OOOOxx +6027 907 1 3 7 7 27 27 27 1027 6027 54 55 VXAAAA XIBAAA VVVVxx +4615 908 1 3 5 15 15 615 615 4615 4615 30 31 NVAAAA YIBAAA AAAAxx +5320 909 0 0 0 0 20 320 1320 320 5320 40 41 QWAAAA ZIBAAA HHHHxx +7002 910 0 2 2 2 2 2 1002 2002 7002 4 5 IJAAAA AJBAAA OOOOxx +7367 911 1 3 7 7 67 367 1367 2367 7367 134 135 JXAAAA BJBAAA VVVVxx +289 912 1 1 9 9 89 289 289 289 289 178 179 DLAAAA CJBAAA AAAAxx +407 913 1 3 7 7 7 407 407 407 407 14 15 RPAAAA DJBAAA HHHHxx +504 914 0 0 4 4 4 504 504 504 504 8 9 KTAAAA EJBAAA OOOOxx +8301 915 1 1 1 1 1 301 301 3301 8301 2 3 HHAAAA FJBAAA VVVVxx +1396 916 0 0 6 16 96 396 1396 1396 1396 192 193 SBAAAA GJBAAA AAAAxx +4794 917 0 2 4 14 94 794 794 4794 4794 188 189 KCAAAA HJBAAA HHHHxx +6400 918 0 0 0 0 0 400 400 1400 6400 0 1 EMAAAA IJBAAA OOOOxx +1275 919 1 3 5 15 75 275 1275 1275 1275 150 151 BXAAAA JJBAAA VVVVxx +5797 920 1 1 7 17 97 797 1797 797 5797 194 195 ZOAAAA KJBAAA AAAAxx +2221 921 1 1 1 1 21 221 221 2221 2221 42 43 LHAAAA LJBAAA HHHHxx +2504 922 0 0 4 4 4 504 504 2504 2504 8 9 ISAAAA MJBAAA OOOOxx +2143 923 1 3 3 3 43 143 143 2143 2143 86 87 LEAAAA NJBAAA VVVVxx +1083 924 1 3 3 3 83 83 1083 1083 1083 166 167 RPAAAA OJBAAA AAAAxx +6148 925 0 0 8 8 48 148 148 1148 6148 96 97 MCAAAA PJBAAA HHHHxx +3612 926 0 0 2 12 12 612 1612 3612 3612 24 25 YIAAAA QJBAAA OOOOxx +9499 927 1 3 9 19 99 499 1499 4499 9499 198 199 JBAAAA RJBAAA VVVVxx +5773 928 1 1 3 13 73 773 1773 773 5773 146 147 BOAAAA SJBAAA AAAAxx +1014 929 0 2 4 14 14 14 1014 1014 1014 28 29 ANAAAA TJBAAA HHHHxx +1427 930 1 3 7 7 27 427 1427 1427 1427 54 55 XCAAAA UJBAAA OOOOxx +6770 931 0 2 0 10 70 770 770 1770 6770 140 141 KAAAAA VJBAAA VVVVxx +9042 932 0 2 2 2 42 42 1042 4042 9042 84 85 UJAAAA WJBAAA AAAAxx +9892 933 0 0 2 12 92 892 1892 4892 9892 184 185 MQAAAA XJBAAA HHHHxx +1771 934 1 3 1 11 71 771 1771 1771 1771 142 143 DQAAAA YJBAAA OOOOxx +7392 935 0 0 2 12 92 392 1392 2392 7392 184 185 IYAAAA ZJBAAA VVVVxx +4465 936 1 1 5 5 65 465 465 4465 4465 130 131 TPAAAA AKBAAA AAAAxx +278 937 0 2 8 18 78 278 278 278 278 156 157 SKAAAA BKBAAA HHHHxx +7776 938 0 0 6 16 76 776 1776 2776 7776 152 153 CNAAAA CKBAAA OOOOxx +3763 939 1 3 3 3 63 763 1763 3763 3763 126 127 TOAAAA DKBAAA VVVVxx +7503 940 1 3 3 3 3 503 1503 2503 7503 6 7 PCAAAA EKBAAA AAAAxx +3793 941 1 1 3 13 93 793 1793 3793 3793 186 187 XPAAAA FKBAAA HHHHxx +6510 942 0 2 0 10 10 510 510 1510 6510 20 21 KQAAAA GKBAAA OOOOxx +7641 943 1 1 1 1 41 641 1641 2641 7641 82 83 XHAAAA HKBAAA VVVVxx +3228 944 0 0 8 8 28 228 1228 3228 3228 56 57 EUAAAA IKBAAA AAAAxx +194 945 0 2 4 14 94 194 194 194 194 188 189 MHAAAA JKBAAA HHHHxx +8555 946 1 3 5 15 55 555 555 3555 8555 110 111 BRAAAA KKBAAA OOOOxx +4997 947 1 1 7 17 97 997 997 4997 4997 194 195 FKAAAA LKBAAA VVVVxx +8687 948 1 3 7 7 87 687 687 3687 8687 174 175 DWAAAA MKBAAA AAAAxx +6632 949 0 0 2 12 32 632 632 1632 6632 64 65 CVAAAA NKBAAA HHHHxx +9607 950 1 3 7 7 7 607 1607 4607 9607 14 15 NFAAAA OKBAAA OOOOxx +6201 951 1 1 1 1 1 201 201 1201 6201 2 3 NEAAAA PKBAAA VVVVxx +857 952 1 1 7 17 57 857 857 857 857 114 115 ZGAAAA QKBAAA AAAAxx +5623 953 1 3 3 3 23 623 1623 623 5623 46 47 HIAAAA RKBAAA HHHHxx +5979 954 1 3 9 19 79 979 1979 979 5979 158 159 ZVAAAA SKBAAA OOOOxx +2201 955 1 1 1 1 1 201 201 2201 2201 2 3 RGAAAA TKBAAA VVVVxx +3166 956 0 2 6 6 66 166 1166 3166 3166 132 133 URAAAA UKBAAA AAAAxx +6249 957 1 1 9 9 49 249 249 1249 6249 98 99 JGAAAA VKBAAA HHHHxx +3271 958 1 3 1 11 71 271 1271 3271 3271 142 143 VVAAAA WKBAAA OOOOxx +7777 959 1 1 7 17 77 777 1777 2777 7777 154 155 DNAAAA XKBAAA VVVVxx +6732 960 0 0 2 12 32 732 732 1732 6732 64 65 YYAAAA YKBAAA AAAAxx +6297 961 1 1 7 17 97 297 297 1297 6297 194 195 FIAAAA ZKBAAA HHHHxx +5685 962 1 1 5 5 85 685 1685 685 5685 170 171 RKAAAA ALBAAA OOOOxx +9931 963 1 3 1 11 31 931 1931 4931 9931 62 63 ZRAAAA BLBAAA VVVVxx +7485 964 1 1 5 5 85 485 1485 2485 7485 170 171 XBAAAA CLBAAA AAAAxx +386 965 0 2 6 6 86 386 386 386 386 172 173 WOAAAA DLBAAA HHHHxx +8204 966 0 0 4 4 4 204 204 3204 8204 8 9 ODAAAA ELBAAA OOOOxx +3606 967 0 2 6 6 6 606 1606 3606 3606 12 13 SIAAAA FLBAAA VVVVxx +1692 968 0 0 2 12 92 692 1692 1692 1692 184 185 CNAAAA GLBAAA AAAAxx +3002 969 0 2 2 2 2 2 1002 3002 3002 4 5 MLAAAA HLBAAA HHHHxx +9676 970 0 0 6 16 76 676 1676 4676 9676 152 153 EIAAAA ILBAAA OOOOxx +915 971 1 3 5 15 15 915 915 915 915 30 31 FJAAAA JLBAAA VVVVxx +7706 972 0 2 6 6 6 706 1706 2706 7706 12 13 KKAAAA KLBAAA AAAAxx +6080 973 0 0 0 0 80 80 80 1080 6080 160 161 WZAAAA LLBAAA HHHHxx +1860 974 0 0 0 0 60 860 1860 1860 1860 120 121 OTAAAA MLBAAA OOOOxx +1444 975 0 0 4 4 44 444 1444 1444 1444 88 89 ODAAAA NLBAAA VVVVxx +7208 976 0 0 8 8 8 208 1208 2208 7208 16 17 GRAAAA OLBAAA AAAAxx +8554 977 0 2 4 14 54 554 554 3554 8554 108 109 ARAAAA PLBAAA HHHHxx +2028 978 0 0 8 8 28 28 28 2028 2028 56 57 AAAAAA QLBAAA OOOOxx +9893 979 1 1 3 13 93 893 1893 4893 9893 186 187 NQAAAA RLBAAA VVVVxx +4740 980 0 0 0 0 40 740 740 4740 4740 80 81 IAAAAA SLBAAA AAAAxx +6186 981 0 2 6 6 86 186 186 1186 6186 172 173 YDAAAA TLBAAA HHHHxx +6357 982 1 1 7 17 57 357 357 1357 6357 114 115 NKAAAA ULBAAA OOOOxx +3699 983 1 3 9 19 99 699 1699 3699 3699 198 199 HMAAAA VLBAAA VVVVxx +7620 984 0 0 0 0 20 620 1620 2620 7620 40 41 CHAAAA WLBAAA AAAAxx +921 985 1 1 1 1 21 921 921 921 921 42 43 LJAAAA XLBAAA HHHHxx +5506 986 0 2 6 6 6 506 1506 506 5506 12 13 UDAAAA YLBAAA OOOOxx +8851 987 1 3 1 11 51 851 851 3851 8851 102 103 LCAAAA ZLBAAA VVVVxx +3205 988 1 1 5 5 5 205 1205 3205 3205 10 11 HTAAAA AMBAAA AAAAxx +1956 989 0 0 6 16 56 956 1956 1956 1956 112 113 GXAAAA BMBAAA HHHHxx +6272 990 0 0 2 12 72 272 272 1272 6272 144 145 GHAAAA CMBAAA OOOOxx +1509 991 1 1 9 9 9 509 1509 1509 1509 18 19 BGAAAA DMBAAA VVVVxx +53 992 1 1 3 13 53 53 53 53 53 106 107 BCAAAA EMBAAA AAAAxx +213 993 1 1 3 13 13 213 213 213 213 26 27 FIAAAA FMBAAA HHHHxx +4924 994 0 0 4 4 24 924 924 4924 4924 48 49 KHAAAA GMBAAA OOOOxx +2097 995 1 1 7 17 97 97 97 2097 2097 194 195 RCAAAA HMBAAA VVVVxx +4607 996 1 3 7 7 7 607 607 4607 4607 14 15 FVAAAA IMBAAA AAAAxx +1582 997 0 2 2 2 82 582 1582 1582 1582 164 165 WIAAAA JMBAAA HHHHxx +6643 998 1 3 3 3 43 643 643 1643 6643 86 87 NVAAAA KMBAAA OOOOxx +2238 999 0 2 8 18 38 238 238 2238 2238 76 77 CIAAAA LMBAAA VVVVxx +2942 1000 0 2 2 2 42 942 942 2942 2942 84 85 EJAAAA MMBAAA AAAAxx +1655 1001 1 3 5 15 55 655 1655 1655 1655 110 111 RLAAAA NMBAAA HHHHxx +3226 1002 0 2 6 6 26 226 1226 3226 3226 52 53 CUAAAA OMBAAA OOOOxx +4263 1003 1 3 3 3 63 263 263 4263 4263 126 127 ZHAAAA PMBAAA VVVVxx +960 1004 0 0 0 0 60 960 960 960 960 120 121 YKAAAA QMBAAA AAAAxx +1213 1005 1 1 3 13 13 213 1213 1213 1213 26 27 RUAAAA RMBAAA HHHHxx +1845 1006 1 1 5 5 45 845 1845 1845 1845 90 91 ZSAAAA SMBAAA OOOOxx +6944 1007 0 0 4 4 44 944 944 1944 6944 88 89 CHAAAA TMBAAA VVVVxx +5284 1008 0 0 4 4 84 284 1284 284 5284 168 169 GVAAAA UMBAAA AAAAxx +188 1009 0 0 8 8 88 188 188 188 188 176 177 GHAAAA VMBAAA HHHHxx +748 1010 0 0 8 8 48 748 748 748 748 96 97 UCAAAA WMBAAA OOOOxx +2226 1011 0 2 6 6 26 226 226 2226 2226 52 53 QHAAAA XMBAAA VVVVxx +7342 1012 0 2 2 2 42 342 1342 2342 7342 84 85 KWAAAA YMBAAA AAAAxx +6120 1013 0 0 0 0 20 120 120 1120 6120 40 41 KBAAAA ZMBAAA HHHHxx +536 1014 0 0 6 16 36 536 536 536 536 72 73 QUAAAA ANBAAA OOOOxx +3239 1015 1 3 9 19 39 239 1239 3239 3239 78 79 PUAAAA BNBAAA VVVVxx +2832 1016 0 0 2 12 32 832 832 2832 2832 64 65 YEAAAA CNBAAA AAAAxx +5296 1017 0 0 6 16 96 296 1296 296 5296 192 193 SVAAAA DNBAAA HHHHxx +5795 1018 1 3 5 15 95 795 1795 795 5795 190 191 XOAAAA ENBAAA OOOOxx +6290 1019 0 2 0 10 90 290 290 1290 6290 180 181 YHAAAA FNBAAA VVVVxx +4916 1020 0 0 6 16 16 916 916 4916 4916 32 33 CHAAAA GNBAAA AAAAxx +8366 1021 0 2 6 6 66 366 366 3366 8366 132 133 UJAAAA HNBAAA HHHHxx +4248 1022 0 0 8 8 48 248 248 4248 4248 96 97 KHAAAA INBAAA OOOOxx +6460 1023 0 0 0 0 60 460 460 1460 6460 120 121 MOAAAA JNBAAA VVVVxx +9296 1024 0 0 6 16 96 296 1296 4296 9296 192 193 OTAAAA KNBAAA AAAAxx +3486 1025 0 2 6 6 86 486 1486 3486 3486 172 173 CEAAAA LNBAAA HHHHxx +5664 1026 0 0 4 4 64 664 1664 664 5664 128 129 WJAAAA MNBAAA OOOOxx +7624 1027 0 0 4 4 24 624 1624 2624 7624 48 49 GHAAAA NNBAAA VVVVxx +2790 1028 0 2 0 10 90 790 790 2790 2790 180 181 IDAAAA ONBAAA AAAAxx +682 1029 0 2 2 2 82 682 682 682 682 164 165 GAAAAA PNBAAA HHHHxx +6412 1030 0 0 2 12 12 412 412 1412 6412 24 25 QMAAAA QNBAAA OOOOxx +6882 1031 0 2 2 2 82 882 882 1882 6882 164 165 SEAAAA RNBAAA VVVVxx +1332 1032 0 0 2 12 32 332 1332 1332 1332 64 65 GZAAAA SNBAAA AAAAxx +4911 1033 1 3 1 11 11 911 911 4911 4911 22 23 XGAAAA TNBAAA HHHHxx +3528 1034 0 0 8 8 28 528 1528 3528 3528 56 57 SFAAAA UNBAAA OOOOxx +271 1035 1 3 1 11 71 271 271 271 271 142 143 LKAAAA VNBAAA VVVVxx +7007 1036 1 3 7 7 7 7 1007 2007 7007 14 15 NJAAAA WNBAAA AAAAxx +2198 1037 0 2 8 18 98 198 198 2198 2198 196 197 OGAAAA XNBAAA HHHHxx +4266 1038 0 2 6 6 66 266 266 4266 4266 132 133 CIAAAA YNBAAA OOOOxx +9867 1039 1 3 7 7 67 867 1867 4867 9867 134 135 NPAAAA ZNBAAA VVVVxx +7602 1040 0 2 2 2 2 602 1602 2602 7602 4 5 KGAAAA AOBAAA AAAAxx +7521 1041 1 1 1 1 21 521 1521 2521 7521 42 43 HDAAAA BOBAAA HHHHxx +7200 1042 0 0 0 0 0 200 1200 2200 7200 0 1 YQAAAA COBAAA OOOOxx +4816 1043 0 0 6 16 16 816 816 4816 4816 32 33 GDAAAA DOBAAA VVVVxx +1669 1044 1 1 9 9 69 669 1669 1669 1669 138 139 FMAAAA EOBAAA AAAAxx +4764 1045 0 0 4 4 64 764 764 4764 4764 128 129 GBAAAA FOBAAA HHHHxx +7393 1046 1 1 3 13 93 393 1393 2393 7393 186 187 JYAAAA GOBAAA OOOOxx +7434 1047 0 2 4 14 34 434 1434 2434 7434 68 69 YZAAAA HOBAAA VVVVxx +9079 1048 1 3 9 19 79 79 1079 4079 9079 158 159 FLAAAA IOBAAA AAAAxx +9668 1049 0 0 8 8 68 668 1668 4668 9668 136 137 WHAAAA JOBAAA HHHHxx +7184 1050 0 0 4 4 84 184 1184 2184 7184 168 169 IQAAAA KOBAAA OOOOxx +7347 1051 1 3 7 7 47 347 1347 2347 7347 94 95 PWAAAA LOBAAA VVVVxx +951 1052 1 3 1 11 51 951 951 951 951 102 103 PKAAAA MOBAAA AAAAxx +4513 1053 1 1 3 13 13 513 513 4513 4513 26 27 PRAAAA NOBAAA HHHHxx +2692 1054 0 0 2 12 92 692 692 2692 2692 184 185 OZAAAA OOBAAA OOOOxx +9930 1055 0 2 0 10 30 930 1930 4930 9930 60 61 YRAAAA POBAAA VVVVxx +4516 1056 0 0 6 16 16 516 516 4516 4516 32 33 SRAAAA QOBAAA AAAAxx +1592 1057 0 0 2 12 92 592 1592 1592 1592 184 185 GJAAAA ROBAAA HHHHxx +6312 1058 0 0 2 12 12 312 312 1312 6312 24 25 UIAAAA SOBAAA OOOOxx +185 1059 1 1 5 5 85 185 185 185 185 170 171 DHAAAA TOBAAA VVVVxx +1848 1060 0 0 8 8 48 848 1848 1848 1848 96 97 CTAAAA UOBAAA AAAAxx +5844 1061 0 0 4 4 44 844 1844 844 5844 88 89 UQAAAA VOBAAA HHHHxx +1666 1062 0 2 6 6 66 666 1666 1666 1666 132 133 CMAAAA WOBAAA OOOOxx +5864 1063 0 0 4 4 64 864 1864 864 5864 128 129 ORAAAA XOBAAA VVVVxx +1004 1064 0 0 4 4 4 4 1004 1004 1004 8 9 QMAAAA YOBAAA AAAAxx +1758 1065 0 2 8 18 58 758 1758 1758 1758 116 117 QPAAAA ZOBAAA HHHHxx +8823 1066 1 3 3 3 23 823 823 3823 8823 46 47 JBAAAA APBAAA OOOOxx +129 1067 1 1 9 9 29 129 129 129 129 58 59 ZEAAAA BPBAAA VVVVxx +5703 1068 1 3 3 3 3 703 1703 703 5703 6 7 JLAAAA CPBAAA AAAAxx +3331 1069 1 3 1 11 31 331 1331 3331 3331 62 63 DYAAAA DPBAAA HHHHxx +5791 1070 1 3 1 11 91 791 1791 791 5791 182 183 TOAAAA EPBAAA OOOOxx +4421 1071 1 1 1 1 21 421 421 4421 4421 42 43 BOAAAA FPBAAA VVVVxx +9740 1072 0 0 0 0 40 740 1740 4740 9740 80 81 QKAAAA GPBAAA AAAAxx +798 1073 0 2 8 18 98 798 798 798 798 196 197 SEAAAA HPBAAA HHHHxx +571 1074 1 3 1 11 71 571 571 571 571 142 143 ZVAAAA IPBAAA OOOOxx +7084 1075 0 0 4 4 84 84 1084 2084 7084 168 169 MMAAAA JPBAAA VVVVxx +650 1076 0 2 0 10 50 650 650 650 650 100 101 AZAAAA KPBAAA AAAAxx +1467 1077 1 3 7 7 67 467 1467 1467 1467 134 135 LEAAAA LPBAAA HHHHxx +5446 1078 0 2 6 6 46 446 1446 446 5446 92 93 MBAAAA MPBAAA OOOOxx +830 1079 0 2 0 10 30 830 830 830 830 60 61 YFAAAA NPBAAA VVVVxx +5516 1080 0 0 6 16 16 516 1516 516 5516 32 33 EEAAAA OPBAAA AAAAxx +8520 1081 0 0 0 0 20 520 520 3520 8520 40 41 SPAAAA PPBAAA HHHHxx +1152 1082 0 0 2 12 52 152 1152 1152 1152 104 105 ISAAAA QPBAAA OOOOxx +862 1083 0 2 2 2 62 862 862 862 862 124 125 EHAAAA RPBAAA VVVVxx +454 1084 0 2 4 14 54 454 454 454 454 108 109 MRAAAA SPBAAA AAAAxx +9956 1085 0 0 6 16 56 956 1956 4956 9956 112 113 YSAAAA TPBAAA HHHHxx +1654 1086 0 2 4 14 54 654 1654 1654 1654 108 109 QLAAAA UPBAAA OOOOxx +257 1087 1 1 7 17 57 257 257 257 257 114 115 XJAAAA VPBAAA VVVVxx +5469 1088 1 1 9 9 69 469 1469 469 5469 138 139 JCAAAA WPBAAA AAAAxx +9075 1089 1 3 5 15 75 75 1075 4075 9075 150 151 BLAAAA XPBAAA HHHHxx +7799 1090 1 3 9 19 99 799 1799 2799 7799 198 199 ZNAAAA YPBAAA OOOOxx +2001 1091 1 1 1 1 1 1 1 2001 2001 2 3 ZYAAAA ZPBAAA VVVVxx +9786 1092 0 2 6 6 86 786 1786 4786 9786 172 173 KMAAAA AQBAAA AAAAxx +7281 1093 1 1 1 1 81 281 1281 2281 7281 162 163 BUAAAA BQBAAA HHHHxx +5137 1094 1 1 7 17 37 137 1137 137 5137 74 75 PPAAAA CQBAAA OOOOxx +4053 1095 1 1 3 13 53 53 53 4053 4053 106 107 XZAAAA DQBAAA VVVVxx +7911 1096 1 3 1 11 11 911 1911 2911 7911 22 23 HSAAAA EQBAAA AAAAxx +4298 1097 0 2 8 18 98 298 298 4298 4298 196 197 IJAAAA FQBAAA HHHHxx +4805 1098 1 1 5 5 5 805 805 4805 4805 10 11 VCAAAA GQBAAA OOOOxx +9038 1099 0 2 8 18 38 38 1038 4038 9038 76 77 QJAAAA HQBAAA VVVVxx +8023 1100 1 3 3 3 23 23 23 3023 8023 46 47 PWAAAA IQBAAA AAAAxx +6595 1101 1 3 5 15 95 595 595 1595 6595 190 191 RTAAAA JQBAAA HHHHxx +9831 1102 1 3 1 11 31 831 1831 4831 9831 62 63 DOAAAA KQBAAA OOOOxx +788 1103 0 0 8 8 88 788 788 788 788 176 177 IEAAAA LQBAAA VVVVxx +902 1104 0 2 2 2 2 902 902 902 902 4 5 SIAAAA MQBAAA AAAAxx +9137 1105 1 1 7 17 37 137 1137 4137 9137 74 75 LNAAAA NQBAAA HHHHxx +1744 1106 0 0 4 4 44 744 1744 1744 1744 88 89 CPAAAA OQBAAA OOOOxx +7285 1107 1 1 5 5 85 285 1285 2285 7285 170 171 FUAAAA PQBAAA VVVVxx +7006 1108 0 2 6 6 6 6 1006 2006 7006 12 13 MJAAAA QQBAAA AAAAxx +9236 1109 0 0 6 16 36 236 1236 4236 9236 72 73 GRAAAA RQBAAA HHHHxx +5472 1110 0 0 2 12 72 472 1472 472 5472 144 145 MCAAAA SQBAAA OOOOxx +7975 1111 1 3 5 15 75 975 1975 2975 7975 150 151 TUAAAA TQBAAA VVVVxx +4181 1112 1 1 1 1 81 181 181 4181 4181 162 163 VEAAAA UQBAAA AAAAxx +7677 1113 1 1 7 17 77 677 1677 2677 7677 154 155 HJAAAA VQBAAA HHHHxx +35 1114 1 3 5 15 35 35 35 35 35 70 71 JBAAAA WQBAAA OOOOxx +6813 1115 1 1 3 13 13 813 813 1813 6813 26 27 BCAAAA XQBAAA VVVVxx +6618 1116 0 2 8 18 18 618 618 1618 6618 36 37 OUAAAA YQBAAA AAAAxx +8069 1117 1 1 9 9 69 69 69 3069 8069 138 139 JYAAAA ZQBAAA HHHHxx +3071 1118 1 3 1 11 71 71 1071 3071 3071 142 143 DOAAAA ARBAAA OOOOxx +4390 1119 0 2 0 10 90 390 390 4390 4390 180 181 WMAAAA BRBAAA VVVVxx +7764 1120 0 0 4 4 64 764 1764 2764 7764 128 129 QMAAAA CRBAAA AAAAxx +8163 1121 1 3 3 3 63 163 163 3163 8163 126 127 ZBAAAA DRBAAA HHHHxx +1961 1122 1 1 1 1 61 961 1961 1961 1961 122 123 LXAAAA ERBAAA OOOOxx +1103 1123 1 3 3 3 3 103 1103 1103 1103 6 7 LQAAAA FRBAAA VVVVxx +5486 1124 0 2 6 6 86 486 1486 486 5486 172 173 ADAAAA GRBAAA AAAAxx +9513 1125 1 1 3 13 13 513 1513 4513 9513 26 27 XBAAAA HRBAAA HHHHxx +7311 1126 1 3 1 11 11 311 1311 2311 7311 22 23 FVAAAA IRBAAA OOOOxx +4144 1127 0 0 4 4 44 144 144 4144 4144 88 89 KDAAAA JRBAAA VVVVxx +7901 1128 1 1 1 1 1 901 1901 2901 7901 2 3 XRAAAA KRBAAA AAAAxx +4629 1129 1 1 9 9 29 629 629 4629 4629 58 59 BWAAAA LRBAAA HHHHxx +6858 1130 0 2 8 18 58 858 858 1858 6858 116 117 UDAAAA MRBAAA OOOOxx +125 1131 1 1 5 5 25 125 125 125 125 50 51 VEAAAA NRBAAA VVVVxx +3834 1132 0 2 4 14 34 834 1834 3834 3834 68 69 MRAAAA ORBAAA AAAAxx +8155 1133 1 3 5 15 55 155 155 3155 8155 110 111 RBAAAA PRBAAA HHHHxx +8230 1134 0 2 0 10 30 230 230 3230 8230 60 61 OEAAAA QRBAAA OOOOxx +744 1135 0 0 4 4 44 744 744 744 744 88 89 QCAAAA RRBAAA VVVVxx +357 1136 1 1 7 17 57 357 357 357 357 114 115 TNAAAA SRBAAA AAAAxx +2159 1137 1 3 9 19 59 159 159 2159 2159 118 119 BFAAAA TRBAAA HHHHxx +8559 1138 1 3 9 19 59 559 559 3559 8559 118 119 FRAAAA URBAAA OOOOxx +6866 1139 0 2 6 6 66 866 866 1866 6866 132 133 CEAAAA VRBAAA VVVVxx +3863 1140 1 3 3 3 63 863 1863 3863 3863 126 127 PSAAAA WRBAAA AAAAxx +4193 1141 1 1 3 13 93 193 193 4193 4193 186 187 HFAAAA XRBAAA HHHHxx +3277 1142 1 1 7 17 77 277 1277 3277 3277 154 155 BWAAAA YRBAAA OOOOxx +5577 1143 1 1 7 17 77 577 1577 577 5577 154 155 NGAAAA ZRBAAA VVVVxx +9503 1144 1 3 3 3 3 503 1503 4503 9503 6 7 NBAAAA ASBAAA AAAAxx +7642 1145 0 2 2 2 42 642 1642 2642 7642 84 85 YHAAAA BSBAAA HHHHxx +6197 1146 1 1 7 17 97 197 197 1197 6197 194 195 JEAAAA CSBAAA OOOOxx +8995 1147 1 3 5 15 95 995 995 3995 8995 190 191 ZHAAAA DSBAAA VVVVxx +440 1148 0 0 0 0 40 440 440 440 440 80 81 YQAAAA ESBAAA AAAAxx +8418 1149 0 2 8 18 18 418 418 3418 8418 36 37 ULAAAA FSBAAA HHHHxx +8531 1150 1 3 1 11 31 531 531 3531 8531 62 63 DQAAAA GSBAAA OOOOxx +3790 1151 0 2 0 10 90 790 1790 3790 3790 180 181 UPAAAA HSBAAA VVVVxx +7610 1152 0 2 0 10 10 610 1610 2610 7610 20 21 SGAAAA ISBAAA AAAAxx +1252 1153 0 0 2 12 52 252 1252 1252 1252 104 105 EWAAAA JSBAAA HHHHxx +7559 1154 1 3 9 19 59 559 1559 2559 7559 118 119 TEAAAA KSBAAA OOOOxx +9945 1155 1 1 5 5 45 945 1945 4945 9945 90 91 NSAAAA LSBAAA VVVVxx +9023 1156 1 3 3 3 23 23 1023 4023 9023 46 47 BJAAAA MSBAAA AAAAxx +3516 1157 0 0 6 16 16 516 1516 3516 3516 32 33 GFAAAA NSBAAA HHHHxx +4671 1158 1 3 1 11 71 671 671 4671 4671 142 143 RXAAAA OSBAAA OOOOxx +1465 1159 1 1 5 5 65 465 1465 1465 1465 130 131 JEAAAA PSBAAA VVVVxx +9515 1160 1 3 5 15 15 515 1515 4515 9515 30 31 ZBAAAA QSBAAA AAAAxx +3242 1161 0 2 2 2 42 242 1242 3242 3242 84 85 SUAAAA RSBAAA HHHHxx +1732 1162 0 0 2 12 32 732 1732 1732 1732 64 65 QOAAAA SSBAAA OOOOxx +1678 1163 0 2 8 18 78 678 1678 1678 1678 156 157 OMAAAA TSBAAA VVVVxx +1464 1164 0 0 4 4 64 464 1464 1464 1464 128 129 IEAAAA USBAAA AAAAxx +6546 1165 0 2 6 6 46 546 546 1546 6546 92 93 URAAAA VSBAAA HHHHxx +4448 1166 0 0 8 8 48 448 448 4448 4448 96 97 CPAAAA WSBAAA OOOOxx +9847 1167 1 3 7 7 47 847 1847 4847 9847 94 95 TOAAAA XSBAAA VVVVxx +8264 1168 0 0 4 4 64 264 264 3264 8264 128 129 WFAAAA YSBAAA AAAAxx +1620 1169 0 0 0 0 20 620 1620 1620 1620 40 41 IKAAAA ZSBAAA HHHHxx +9388 1170 0 0 8 8 88 388 1388 4388 9388 176 177 CXAAAA ATBAAA OOOOxx +6445 1171 1 1 5 5 45 445 445 1445 6445 90 91 XNAAAA BTBAAA VVVVxx +4789 1172 1 1 9 9 89 789 789 4789 4789 178 179 FCAAAA CTBAAA AAAAxx +1562 1173 0 2 2 2 62 562 1562 1562 1562 124 125 CIAAAA DTBAAA HHHHxx +7305 1174 1 1 5 5 5 305 1305 2305 7305 10 11 ZUAAAA ETBAAA OOOOxx +6344 1175 0 0 4 4 44 344 344 1344 6344 88 89 AKAAAA FTBAAA VVVVxx +5130 1176 0 2 0 10 30 130 1130 130 5130 60 61 IPAAAA GTBAAA AAAAxx +3284 1177 0 0 4 4 84 284 1284 3284 3284 168 169 IWAAAA HTBAAA HHHHxx +6346 1178 0 2 6 6 46 346 346 1346 6346 92 93 CKAAAA ITBAAA OOOOxx +1061 1179 1 1 1 1 61 61 1061 1061 1061 122 123 VOAAAA JTBAAA VVVVxx +872 1180 0 0 2 12 72 872 872 872 872 144 145 OHAAAA KTBAAA AAAAxx +123 1181 1 3 3 3 23 123 123 123 123 46 47 TEAAAA LTBAAA HHHHxx +7903 1182 1 3 3 3 3 903 1903 2903 7903 6 7 ZRAAAA MTBAAA OOOOxx +560 1183 0 0 0 0 60 560 560 560 560 120 121 OVAAAA NTBAAA VVVVxx +4446 1184 0 2 6 6 46 446 446 4446 4446 92 93 APAAAA OTBAAA AAAAxx +3909 1185 1 1 9 9 9 909 1909 3909 3909 18 19 JUAAAA PTBAAA HHHHxx +669 1186 1 1 9 9 69 669 669 669 669 138 139 TZAAAA QTBAAA OOOOxx +7843 1187 1 3 3 3 43 843 1843 2843 7843 86 87 RPAAAA RTBAAA VVVVxx +2546 1188 0 2 6 6 46 546 546 2546 2546 92 93 YTAAAA STBAAA AAAAxx +6757 1189 1 1 7 17 57 757 757 1757 6757 114 115 XZAAAA TTBAAA HHHHxx +466 1190 0 2 6 6 66 466 466 466 466 132 133 YRAAAA UTBAAA OOOOxx +5556 1191 0 0 6 16 56 556 1556 556 5556 112 113 SFAAAA VTBAAA VVVVxx +7196 1192 0 0 6 16 96 196 1196 2196 7196 192 193 UQAAAA WTBAAA AAAAxx +2947 1193 1 3 7 7 47 947 947 2947 2947 94 95 JJAAAA XTBAAA HHHHxx +6493 1194 1 1 3 13 93 493 493 1493 6493 186 187 TPAAAA YTBAAA OOOOxx +7203 1195 1 3 3 3 3 203 1203 2203 7203 6 7 BRAAAA ZTBAAA VVVVxx +3716 1196 0 0 6 16 16 716 1716 3716 3716 32 33 YMAAAA AUBAAA AAAAxx +8058 1197 0 2 8 18 58 58 58 3058 8058 116 117 YXAAAA BUBAAA HHHHxx +433 1198 1 1 3 13 33 433 433 433 433 66 67 RQAAAA CUBAAA OOOOxx +7649 1199 1 1 9 9 49 649 1649 2649 7649 98 99 FIAAAA DUBAAA VVVVxx +6966 1200 0 2 6 6 66 966 966 1966 6966 132 133 YHAAAA EUBAAA AAAAxx +553 1201 1 1 3 13 53 553 553 553 553 106 107 HVAAAA FUBAAA HHHHxx +3677 1202 1 1 7 17 77 677 1677 3677 3677 154 155 LLAAAA GUBAAA OOOOxx +2344 1203 0 0 4 4 44 344 344 2344 2344 88 89 EMAAAA HUBAAA VVVVxx +7439 1204 1 3 9 19 39 439 1439 2439 7439 78 79 DAAAAA IUBAAA AAAAxx +3910 1205 0 2 0 10 10 910 1910 3910 3910 20 21 KUAAAA JUBAAA HHHHxx +3638 1206 0 2 8 18 38 638 1638 3638 3638 76 77 YJAAAA KUBAAA OOOOxx +6637 1207 1 1 7 17 37 637 637 1637 6637 74 75 HVAAAA LUBAAA VVVVxx +4438 1208 0 2 8 18 38 438 438 4438 4438 76 77 SOAAAA MUBAAA AAAAxx +171 1209 1 3 1 11 71 171 171 171 171 142 143 PGAAAA NUBAAA HHHHxx +310 1210 0 2 0 10 10 310 310 310 310 20 21 YLAAAA OUBAAA OOOOxx +2714 1211 0 2 4 14 14 714 714 2714 2714 28 29 KAAAAA PUBAAA VVVVxx +5199 1212 1 3 9 19 99 199 1199 199 5199 198 199 ZRAAAA QUBAAA AAAAxx +8005 1213 1 1 5 5 5 5 5 3005 8005 10 11 XVAAAA RUBAAA HHHHxx +3188 1214 0 0 8 8 88 188 1188 3188 3188 176 177 QSAAAA SUBAAA OOOOxx +1518 1215 0 2 8 18 18 518 1518 1518 1518 36 37 KGAAAA TUBAAA VVVVxx +6760 1216 0 0 0 0 60 760 760 1760 6760 120 121 AAAAAA UUBAAA AAAAxx +9373 1217 1 1 3 13 73 373 1373 4373 9373 146 147 NWAAAA VUBAAA HHHHxx +1938 1218 0 2 8 18 38 938 1938 1938 1938 76 77 OWAAAA WUBAAA OOOOxx +2865 1219 1 1 5 5 65 865 865 2865 2865 130 131 FGAAAA XUBAAA VVVVxx +3203 1220 1 3 3 3 3 203 1203 3203 3203 6 7 FTAAAA YUBAAA AAAAxx +6025 1221 1 1 5 5 25 25 25 1025 6025 50 51 TXAAAA ZUBAAA HHHHxx +8684 1222 0 0 4 4 84 684 684 3684 8684 168 169 AWAAAA AVBAAA OOOOxx +7732 1223 0 0 2 12 32 732 1732 2732 7732 64 65 KLAAAA BVBAAA VVVVxx +3218 1224 0 2 8 18 18 218 1218 3218 3218 36 37 UTAAAA CVBAAA AAAAxx +525 1225 1 1 5 5 25 525 525 525 525 50 51 FUAAAA DVBAAA HHHHxx +601 1226 1 1 1 1 1 601 601 601 601 2 3 DXAAAA EVBAAA OOOOxx +6091 1227 1 3 1 11 91 91 91 1091 6091 182 183 HAAAAA FVBAAA VVVVxx +4498 1228 0 2 8 18 98 498 498 4498 4498 196 197 ARAAAA GVBAAA AAAAxx +8192 1229 0 0 2 12 92 192 192 3192 8192 184 185 CDAAAA HVBAAA HHHHxx +8006 1230 0 2 6 6 6 6 6 3006 8006 12 13 YVAAAA IVBAAA OOOOxx +6157 1231 1 1 7 17 57 157 157 1157 6157 114 115 VCAAAA JVBAAA VVVVxx +312 1232 0 0 2 12 12 312 312 312 312 24 25 AMAAAA KVBAAA AAAAxx +8652 1233 0 0 2 12 52 652 652 3652 8652 104 105 UUAAAA LVBAAA HHHHxx +2787 1234 1 3 7 7 87 787 787 2787 2787 174 175 FDAAAA MVBAAA OOOOxx +1782 1235 0 2 2 2 82 782 1782 1782 1782 164 165 OQAAAA NVBAAA VVVVxx +23 1236 1 3 3 3 23 23 23 23 23 46 47 XAAAAA OVBAAA AAAAxx +1206 1237 0 2 6 6 6 206 1206 1206 1206 12 13 KUAAAA PVBAAA HHHHxx +1076 1238 0 0 6 16 76 76 1076 1076 1076 152 153 KPAAAA QVBAAA OOOOxx +5379 1239 1 3 9 19 79 379 1379 379 5379 158 159 XYAAAA RVBAAA VVVVxx +2047 1240 1 3 7 7 47 47 47 2047 2047 94 95 TAAAAA SVBAAA AAAAxx +6262 1241 0 2 2 2 62 262 262 1262 6262 124 125 WGAAAA TVBAAA HHHHxx +1840 1242 0 0 0 0 40 840 1840 1840 1840 80 81 USAAAA UVBAAA OOOOxx +2106 1243 0 2 6 6 6 106 106 2106 2106 12 13 ADAAAA VVBAAA VVVVxx +1307 1244 1 3 7 7 7 307 1307 1307 1307 14 15 HYAAAA WVBAAA AAAAxx +735 1245 1 3 5 15 35 735 735 735 735 70 71 HCAAAA XVBAAA HHHHxx +3657 1246 1 1 7 17 57 657 1657 3657 3657 114 115 RKAAAA YVBAAA OOOOxx +3006 1247 0 2 6 6 6 6 1006 3006 3006 12 13 QLAAAA ZVBAAA VVVVxx +1538 1248 0 2 8 18 38 538 1538 1538 1538 76 77 EHAAAA AWBAAA AAAAxx +6098 1249 0 2 8 18 98 98 98 1098 6098 196 197 OAAAAA BWBAAA HHHHxx +5267 1250 1 3 7 7 67 267 1267 267 5267 134 135 PUAAAA CWBAAA OOOOxx +9757 1251 1 1 7 17 57 757 1757 4757 9757 114 115 HLAAAA DWBAAA VVVVxx +1236 1252 0 0 6 16 36 236 1236 1236 1236 72 73 OVAAAA EWBAAA AAAAxx +83 1253 1 3 3 3 83 83 83 83 83 166 167 FDAAAA FWBAAA HHHHxx +9227 1254 1 3 7 7 27 227 1227 4227 9227 54 55 XQAAAA GWBAAA OOOOxx +8772 1255 0 0 2 12 72 772 772 3772 8772 144 145 KZAAAA HWBAAA VVVVxx +8822 1256 0 2 2 2 22 822 822 3822 8822 44 45 IBAAAA IWBAAA AAAAxx +7167 1257 1 3 7 7 67 167 1167 2167 7167 134 135 RPAAAA JWBAAA HHHHxx +6909 1258 1 1 9 9 9 909 909 1909 6909 18 19 TFAAAA KWBAAA OOOOxx +1439 1259 1 3 9 19 39 439 1439 1439 1439 78 79 JDAAAA LWBAAA VVVVxx +2370 1260 0 2 0 10 70 370 370 2370 2370 140 141 ENAAAA MWBAAA AAAAxx +4577 1261 1 1 7 17 77 577 577 4577 4577 154 155 BUAAAA NWBAAA HHHHxx +2575 1262 1 3 5 15 75 575 575 2575 2575 150 151 BVAAAA OWBAAA OOOOxx +2795 1263 1 3 5 15 95 795 795 2795 2795 190 191 NDAAAA PWBAAA VVVVxx +5520 1264 0 0 0 0 20 520 1520 520 5520 40 41 IEAAAA QWBAAA AAAAxx +382 1265 0 2 2 2 82 382 382 382 382 164 165 SOAAAA RWBAAA HHHHxx +6335 1266 1 3 5 15 35 335 335 1335 6335 70 71 RJAAAA SWBAAA OOOOxx +8430 1267 0 2 0 10 30 430 430 3430 8430 60 61 GMAAAA TWBAAA VVVVxx +4131 1268 1 3 1 11 31 131 131 4131 4131 62 63 XCAAAA UWBAAA AAAAxx +9332 1269 0 0 2 12 32 332 1332 4332 9332 64 65 YUAAAA VWBAAA HHHHxx +293 1270 1 1 3 13 93 293 293 293 293 186 187 HLAAAA WWBAAA OOOOxx +2276 1271 0 0 6 16 76 276 276 2276 2276 152 153 OJAAAA XWBAAA VVVVxx +5687 1272 1 3 7 7 87 687 1687 687 5687 174 175 TKAAAA YWBAAA AAAAxx +5862 1273 0 2 2 2 62 862 1862 862 5862 124 125 MRAAAA ZWBAAA HHHHxx +5073 1274 1 1 3 13 73 73 1073 73 5073 146 147 DNAAAA AXBAAA OOOOxx +4170 1275 0 2 0 10 70 170 170 4170 4170 140 141 KEAAAA BXBAAA VVVVxx +5039 1276 1 3 9 19 39 39 1039 39 5039 78 79 VLAAAA CXBAAA AAAAxx +3294 1277 0 2 4 14 94 294 1294 3294 3294 188 189 SWAAAA DXBAAA HHHHxx +6015 1278 1 3 5 15 15 15 15 1015 6015 30 31 JXAAAA EXBAAA OOOOxx +9015 1279 1 3 5 15 15 15 1015 4015 9015 30 31 TIAAAA FXBAAA VVVVxx +9785 1280 1 1 5 5 85 785 1785 4785 9785 170 171 JMAAAA GXBAAA AAAAxx +4312 1281 0 0 2 12 12 312 312 4312 4312 24 25 WJAAAA HXBAAA HHHHxx +6343 1282 1 3 3 3 43 343 343 1343 6343 86 87 ZJAAAA IXBAAA OOOOxx +2161 1283 1 1 1 1 61 161 161 2161 2161 122 123 DFAAAA JXBAAA VVVVxx +4490 1284 0 2 0 10 90 490 490 4490 4490 180 181 SQAAAA KXBAAA AAAAxx +4454 1285 0 2 4 14 54 454 454 4454 4454 108 109 IPAAAA LXBAAA HHHHxx +7647 1286 1 3 7 7 47 647 1647 2647 7647 94 95 DIAAAA MXBAAA OOOOxx +1028 1287 0 0 8 8 28 28 1028 1028 1028 56 57 ONAAAA NXBAAA VVVVxx +2965 1288 1 1 5 5 65 965 965 2965 2965 130 131 BKAAAA OXBAAA AAAAxx +9900 1289 0 0 0 0 0 900 1900 4900 9900 0 1 UQAAAA PXBAAA HHHHxx +5509 1290 1 1 9 9 9 509 1509 509 5509 18 19 XDAAAA QXBAAA OOOOxx +7751 1291 1 3 1 11 51 751 1751 2751 7751 102 103 DMAAAA RXBAAA VVVVxx +9594 1292 0 2 4 14 94 594 1594 4594 9594 188 189 AFAAAA SXBAAA AAAAxx +7632 1293 0 0 2 12 32 632 1632 2632 7632 64 65 OHAAAA TXBAAA HHHHxx +6528 1294 0 0 8 8 28 528 528 1528 6528 56 57 CRAAAA UXBAAA OOOOxx +1041 1295 1 1 1 1 41 41 1041 1041 1041 82 83 BOAAAA VXBAAA VVVVxx +1534 1296 0 2 4 14 34 534 1534 1534 1534 68 69 AHAAAA WXBAAA AAAAxx +4229 1297 1 1 9 9 29 229 229 4229 4229 58 59 RGAAAA XXBAAA HHHHxx +84 1298 0 0 4 4 84 84 84 84 84 168 169 GDAAAA YXBAAA OOOOxx +2189 1299 1 1 9 9 89 189 189 2189 2189 178 179 FGAAAA ZXBAAA VVVVxx +7566 1300 0 2 6 6 66 566 1566 2566 7566 132 133 AFAAAA AYBAAA AAAAxx +707 1301 1 3 7 7 7 707 707 707 707 14 15 FBAAAA BYBAAA HHHHxx +581 1302 1 1 1 1 81 581 581 581 581 162 163 JWAAAA CYBAAA OOOOxx +6753 1303 1 1 3 13 53 753 753 1753 6753 106 107 TZAAAA DYBAAA VVVVxx +8604 1304 0 0 4 4 4 604 604 3604 8604 8 9 YSAAAA EYBAAA AAAAxx +373 1305 1 1 3 13 73 373 373 373 373 146 147 JOAAAA FYBAAA HHHHxx +9635 1306 1 3 5 15 35 635 1635 4635 9635 70 71 PGAAAA GYBAAA OOOOxx +9277 1307 1 1 7 17 77 277 1277 4277 9277 154 155 VSAAAA HYBAAA VVVVxx +7117 1308 1 1 7 17 17 117 1117 2117 7117 34 35 TNAAAA IYBAAA AAAAxx +8564 1309 0 0 4 4 64 564 564 3564 8564 128 129 KRAAAA JYBAAA HHHHxx +1697 1310 1 1 7 17 97 697 1697 1697 1697 194 195 HNAAAA KYBAAA OOOOxx +7840 1311 0 0 0 0 40 840 1840 2840 7840 80 81 OPAAAA LYBAAA VVVVxx +3646 1312 0 2 6 6 46 646 1646 3646 3646 92 93 GKAAAA MYBAAA AAAAxx +368 1313 0 0 8 8 68 368 368 368 368 136 137 EOAAAA NYBAAA HHHHxx +4797 1314 1 1 7 17 97 797 797 4797 4797 194 195 NCAAAA OYBAAA OOOOxx +5300 1315 0 0 0 0 0 300 1300 300 5300 0 1 WVAAAA PYBAAA VVVVxx +7664 1316 0 0 4 4 64 664 1664 2664 7664 128 129 UIAAAA QYBAAA AAAAxx +1466 1317 0 2 6 6 66 466 1466 1466 1466 132 133 KEAAAA RYBAAA HHHHxx +2477 1318 1 1 7 17 77 477 477 2477 2477 154 155 HRAAAA SYBAAA OOOOxx +2036 1319 0 0 6 16 36 36 36 2036 2036 72 73 IAAAAA TYBAAA VVVVxx +3624 1320 0 0 4 4 24 624 1624 3624 3624 48 49 KJAAAA UYBAAA AAAAxx +5099 1321 1 3 9 19 99 99 1099 99 5099 198 199 DOAAAA VYBAAA HHHHxx +1308 1322 0 0 8 8 8 308 1308 1308 1308 16 17 IYAAAA WYBAAA OOOOxx +3704 1323 0 0 4 4 4 704 1704 3704 3704 8 9 MMAAAA XYBAAA VVVVxx +2451 1324 1 3 1 11 51 451 451 2451 2451 102 103 HQAAAA YYBAAA AAAAxx +4898 1325 0 2 8 18 98 898 898 4898 4898 196 197 KGAAAA ZYBAAA HHHHxx +4959 1326 1 3 9 19 59 959 959 4959 4959 118 119 TIAAAA AZBAAA OOOOxx +5942 1327 0 2 2 2 42 942 1942 942 5942 84 85 OUAAAA BZBAAA VVVVxx +2425 1328 1 1 5 5 25 425 425 2425 2425 50 51 HPAAAA CZBAAA AAAAxx +7760 1329 0 0 0 0 60 760 1760 2760 7760 120 121 MMAAAA DZBAAA HHHHxx +6294 1330 0 2 4 14 94 294 294 1294 6294 188 189 CIAAAA EZBAAA OOOOxx +6785 1331 1 1 5 5 85 785 785 1785 6785 170 171 ZAAAAA FZBAAA VVVVxx +3542 1332 0 2 2 2 42 542 1542 3542 3542 84 85 GGAAAA GZBAAA AAAAxx +1809 1333 1 1 9 9 9 809 1809 1809 1809 18 19 PRAAAA HZBAAA HHHHxx +130 1334 0 2 0 10 30 130 130 130 130 60 61 AFAAAA IZBAAA OOOOxx +8672 1335 0 0 2 12 72 672 672 3672 8672 144 145 OVAAAA JZBAAA VVVVxx +2125 1336 1 1 5 5 25 125 125 2125 2125 50 51 TDAAAA KZBAAA AAAAxx +7683 1337 1 3 3 3 83 683 1683 2683 7683 166 167 NJAAAA LZBAAA HHHHxx +7842 1338 0 2 2 2 42 842 1842 2842 7842 84 85 QPAAAA MZBAAA OOOOxx +9584 1339 0 0 4 4 84 584 1584 4584 9584 168 169 QEAAAA NZBAAA VVVVxx +7963 1340 1 3 3 3 63 963 1963 2963 7963 126 127 HUAAAA OZBAAA AAAAxx +8581 1341 1 1 1 1 81 581 581 3581 8581 162 163 BSAAAA PZBAAA HHHHxx +2135 1342 1 3 5 15 35 135 135 2135 2135 70 71 DEAAAA QZBAAA OOOOxx +7352 1343 0 0 2 12 52 352 1352 2352 7352 104 105 UWAAAA RZBAAA VVVVxx +5789 1344 1 1 9 9 89 789 1789 789 5789 178 179 ROAAAA SZBAAA AAAAxx +8490 1345 0 2 0 10 90 490 490 3490 8490 180 181 OOAAAA TZBAAA HHHHxx +2145 1346 1 1 5 5 45 145 145 2145 2145 90 91 NEAAAA UZBAAA OOOOxx +7021 1347 1 1 1 1 21 21 1021 2021 7021 42 43 BKAAAA VZBAAA VVVVxx +3736 1348 0 0 6 16 36 736 1736 3736 3736 72 73 SNAAAA WZBAAA AAAAxx +7396 1349 0 0 6 16 96 396 1396 2396 7396 192 193 MYAAAA XZBAAA HHHHxx +6334 1350 0 2 4 14 34 334 334 1334 6334 68 69 QJAAAA YZBAAA OOOOxx +5461 1351 1 1 1 1 61 461 1461 461 5461 122 123 BCAAAA ZZBAAA VVVVxx +5337 1352 1 1 7 17 37 337 1337 337 5337 74 75 HXAAAA AACAAA AAAAxx +7440 1353 0 0 0 0 40 440 1440 2440 7440 80 81 EAAAAA BACAAA HHHHxx +6879 1354 1 3 9 19 79 879 879 1879 6879 158 159 PEAAAA CACAAA OOOOxx +2432 1355 0 0 2 12 32 432 432 2432 2432 64 65 OPAAAA DACAAA VVVVxx +8529 1356 1 1 9 9 29 529 529 3529 8529 58 59 BQAAAA EACAAA AAAAxx +7859 1357 1 3 9 19 59 859 1859 2859 7859 118 119 HQAAAA FACAAA HHHHxx +15 1358 1 3 5 15 15 15 15 15 15 30 31 PAAAAA GACAAA OOOOxx +7475 1359 1 3 5 15 75 475 1475 2475 7475 150 151 NBAAAA HACAAA VVVVxx +717 1360 1 1 7 17 17 717 717 717 717 34 35 PBAAAA IACAAA AAAAxx +250 1361 0 2 0 10 50 250 250 250 250 100 101 QJAAAA JACAAA HHHHxx +4700 1362 0 0 0 0 0 700 700 4700 4700 0 1 UYAAAA KACAAA OOOOxx +7510 1363 0 2 0 10 10 510 1510 2510 7510 20 21 WCAAAA LACAAA VVVVxx +4562 1364 0 2 2 2 62 562 562 4562 4562 124 125 MTAAAA MACAAA AAAAxx +8075 1365 1 3 5 15 75 75 75 3075 8075 150 151 PYAAAA NACAAA HHHHxx +871 1366 1 3 1 11 71 871 871 871 871 142 143 NHAAAA OACAAA OOOOxx +7161 1367 1 1 1 1 61 161 1161 2161 7161 122 123 LPAAAA PACAAA VVVVxx +9109 1368 1 1 9 9 9 109 1109 4109 9109 18 19 JMAAAA QACAAA AAAAxx +8675 1369 1 3 5 15 75 675 675 3675 8675 150 151 RVAAAA RACAAA HHHHxx +1025 1370 1 1 5 5 25 25 1025 1025 1025 50 51 LNAAAA SACAAA OOOOxx +4065 1371 1 1 5 5 65 65 65 4065 4065 130 131 JAAAAA TACAAA VVVVxx +3511 1372 1 3 1 11 11 511 1511 3511 3511 22 23 BFAAAA UACAAA AAAAxx +9840 1373 0 0 0 0 40 840 1840 4840 9840 80 81 MOAAAA VACAAA HHHHxx +7495 1374 1 3 5 15 95 495 1495 2495 7495 190 191 HCAAAA WACAAA OOOOxx +55 1375 1 3 5 15 55 55 55 55 55 110 111 DCAAAA XACAAA VVVVxx +6151 1376 1 3 1 11 51 151 151 1151 6151 102 103 PCAAAA YACAAA AAAAxx +2512 1377 0 0 2 12 12 512 512 2512 2512 24 25 QSAAAA ZACAAA HHHHxx +5881 1378 1 1 1 1 81 881 1881 881 5881 162 163 FSAAAA ABCAAA OOOOxx +1442 1379 0 2 2 2 42 442 1442 1442 1442 84 85 MDAAAA BBCAAA VVVVxx +1270 1380 0 2 0 10 70 270 1270 1270 1270 140 141 WWAAAA CBCAAA AAAAxx +959 1381 1 3 9 19 59 959 959 959 959 118 119 XKAAAA DBCAAA HHHHxx +8251 1382 1 3 1 11 51 251 251 3251 8251 102 103 JFAAAA EBCAAA OOOOxx +3051 1383 1 3 1 11 51 51 1051 3051 3051 102 103 JNAAAA FBCAAA VVVVxx +5052 1384 0 0 2 12 52 52 1052 52 5052 104 105 IMAAAA GBCAAA AAAAxx +1863 1385 1 3 3 3 63 863 1863 1863 1863 126 127 RTAAAA HBCAAA HHHHxx +344 1386 0 0 4 4 44 344 344 344 344 88 89 GNAAAA IBCAAA OOOOxx +3590 1387 0 2 0 10 90 590 1590 3590 3590 180 181 CIAAAA JBCAAA VVVVxx +4223 1388 1 3 3 3 23 223 223 4223 4223 46 47 LGAAAA KBCAAA AAAAxx +2284 1389 0 0 4 4 84 284 284 2284 2284 168 169 WJAAAA LBCAAA HHHHxx +9425 1390 1 1 5 5 25 425 1425 4425 9425 50 51 NYAAAA MBCAAA OOOOxx +6221 1391 1 1 1 1 21 221 221 1221 6221 42 43 HFAAAA NBCAAA VVVVxx +195 1392 1 3 5 15 95 195 195 195 195 190 191 NHAAAA OBCAAA AAAAxx +1517 1393 1 1 7 17 17 517 1517 1517 1517 34 35 JGAAAA PBCAAA HHHHxx +3791 1394 1 3 1 11 91 791 1791 3791 3791 182 183 VPAAAA QBCAAA OOOOxx +572 1395 0 0 2 12 72 572 572 572 572 144 145 AWAAAA RBCAAA VVVVxx +46 1396 0 2 6 6 46 46 46 46 46 92 93 UBAAAA SBCAAA AAAAxx +9451 1397 1 3 1 11 51 451 1451 4451 9451 102 103 NZAAAA TBCAAA HHHHxx +3359 1398 1 3 9 19 59 359 1359 3359 3359 118 119 FZAAAA UBCAAA OOOOxx +8867 1399 1 3 7 7 67 867 867 3867 8867 134 135 BDAAAA VBCAAA VVVVxx +674 1400 0 2 4 14 74 674 674 674 674 148 149 YZAAAA WBCAAA AAAAxx +2674 1401 0 2 4 14 74 674 674 2674 2674 148 149 WYAAAA XBCAAA HHHHxx +6523 1402 1 3 3 3 23 523 523 1523 6523 46 47 XQAAAA YBCAAA OOOOxx +6210 1403 0 2 0 10 10 210 210 1210 6210 20 21 WEAAAA ZBCAAA VVVVxx +7564 1404 0 0 4 4 64 564 1564 2564 7564 128 129 YEAAAA ACCAAA AAAAxx +4776 1405 0 0 6 16 76 776 776 4776 4776 152 153 SBAAAA BCCAAA HHHHxx +2993 1406 1 1 3 13 93 993 993 2993 2993 186 187 DLAAAA CCCAAA OOOOxx +2969 1407 1 1 9 9 69 969 969 2969 2969 138 139 FKAAAA DCCAAA VVVVxx +1762 1408 0 2 2 2 62 762 1762 1762 1762 124 125 UPAAAA ECCAAA AAAAxx +685 1409 1 1 5 5 85 685 685 685 685 170 171 JAAAAA FCCAAA HHHHxx +5312 1410 0 0 2 12 12 312 1312 312 5312 24 25 IWAAAA GCCAAA OOOOxx +3264 1411 0 0 4 4 64 264 1264 3264 3264 128 129 OVAAAA HCCAAA VVVVxx +7008 1412 0 0 8 8 8 8 1008 2008 7008 16 17 OJAAAA ICCAAA AAAAxx +5167 1413 1 3 7 7 67 167 1167 167 5167 134 135 TQAAAA JCCAAA HHHHxx +3060 1414 0 0 0 0 60 60 1060 3060 3060 120 121 SNAAAA KCCAAA OOOOxx +1752 1415 0 0 2 12 52 752 1752 1752 1752 104 105 KPAAAA LCCAAA VVVVxx +1016 1416 0 0 6 16 16 16 1016 1016 1016 32 33 CNAAAA MCCAAA AAAAxx +7365 1417 1 1 5 5 65 365 1365 2365 7365 130 131 HXAAAA NCCAAA HHHHxx +4358 1418 0 2 8 18 58 358 358 4358 4358 116 117 QLAAAA OCCAAA OOOOxx +2819 1419 1 3 9 19 19 819 819 2819 2819 38 39 LEAAAA PCCAAA VVVVxx +6727 1420 1 3 7 7 27 727 727 1727 6727 54 55 TYAAAA QCCAAA AAAAxx +1459 1421 1 3 9 19 59 459 1459 1459 1459 118 119 DEAAAA RCCAAA HHHHxx +1708 1422 0 0 8 8 8 708 1708 1708 1708 16 17 SNAAAA SCCAAA OOOOxx +471 1423 1 3 1 11 71 471 471 471 471 142 143 DSAAAA TCCAAA VVVVxx +387 1424 1 3 7 7 87 387 387 387 387 174 175 XOAAAA UCCAAA AAAAxx +1166 1425 0 2 6 6 66 166 1166 1166 1166 132 133 WSAAAA VCCAAA HHHHxx +2400 1426 0 0 0 0 0 400 400 2400 2400 0 1 IOAAAA WCCAAA OOOOxx +3584 1427 0 0 4 4 84 584 1584 3584 3584 168 169 WHAAAA XCCAAA VVVVxx +6423 1428 1 3 3 3 23 423 423 1423 6423 46 47 BNAAAA YCCAAA AAAAxx +9520 1429 0 0 0 0 20 520 1520 4520 9520 40 41 ECAAAA ZCCAAA HHHHxx +8080 1430 0 0 0 0 80 80 80 3080 8080 160 161 UYAAAA ADCAAA OOOOxx +5709 1431 1 1 9 9 9 709 1709 709 5709 18 19 PLAAAA BDCAAA VVVVxx +1131 1432 1 3 1 11 31 131 1131 1131 1131 62 63 NRAAAA CDCAAA AAAAxx +8562 1433 0 2 2 2 62 562 562 3562 8562 124 125 IRAAAA DDCAAA HHHHxx +5766 1434 0 2 6 6 66 766 1766 766 5766 132 133 UNAAAA EDCAAA OOOOxx +245 1435 1 1 5 5 45 245 245 245 245 90 91 LJAAAA FDCAAA VVVVxx +9869 1436 1 1 9 9 69 869 1869 4869 9869 138 139 PPAAAA GDCAAA AAAAxx +3533 1437 1 1 3 13 33 533 1533 3533 3533 66 67 XFAAAA HDCAAA HHHHxx +5109 1438 1 1 9 9 9 109 1109 109 5109 18 19 NOAAAA IDCAAA OOOOxx +977 1439 1 1 7 17 77 977 977 977 977 154 155 PLAAAA JDCAAA VVVVxx +1651 1440 1 3 1 11 51 651 1651 1651 1651 102 103 NLAAAA KDCAAA AAAAxx +1357 1441 1 1 7 17 57 357 1357 1357 1357 114 115 FAAAAA LDCAAA HHHHxx +9087 1442 1 3 7 7 87 87 1087 4087 9087 174 175 NLAAAA MDCAAA OOOOxx +3399 1443 1 3 9 19 99 399 1399 3399 3399 198 199 TAAAAA NDCAAA VVVVxx +7543 1444 1 3 3 3 43 543 1543 2543 7543 86 87 DEAAAA ODCAAA AAAAxx +2469 1445 1 1 9 9 69 469 469 2469 2469 138 139 ZQAAAA PDCAAA HHHHxx +8305 1446 1 1 5 5 5 305 305 3305 8305 10 11 LHAAAA QDCAAA OOOOxx +3265 1447 1 1 5 5 65 265 1265 3265 3265 130 131 PVAAAA RDCAAA VVVVxx +9977 1448 1 1 7 17 77 977 1977 4977 9977 154 155 TTAAAA SDCAAA AAAAxx +3961 1449 1 1 1 1 61 961 1961 3961 3961 122 123 JWAAAA TDCAAA HHHHxx +4952 1450 0 0 2 12 52 952 952 4952 4952 104 105 MIAAAA UDCAAA OOOOxx +5173 1451 1 1 3 13 73 173 1173 173 5173 146 147 ZQAAAA VDCAAA VVVVxx +860 1452 0 0 0 0 60 860 860 860 860 120 121 CHAAAA WDCAAA AAAAxx +4523 1453 1 3 3 3 23 523 523 4523 4523 46 47 ZRAAAA XDCAAA HHHHxx +2361 1454 1 1 1 1 61 361 361 2361 2361 122 123 VMAAAA YDCAAA OOOOxx +7877 1455 1 1 7 17 77 877 1877 2877 7877 154 155 ZQAAAA ZDCAAA VVVVxx +3422 1456 0 2 2 2 22 422 1422 3422 3422 44 45 QBAAAA AECAAA AAAAxx +5781 1457 1 1 1 1 81 781 1781 781 5781 162 163 JOAAAA BECAAA HHHHxx +4752 1458 0 0 2 12 52 752 752 4752 4752 104 105 UAAAAA CECAAA OOOOxx +1786 1459 0 2 6 6 86 786 1786 1786 1786 172 173 SQAAAA DECAAA VVVVxx +1892 1460 0 0 2 12 92 892 1892 1892 1892 184 185 UUAAAA EECAAA AAAAxx +6389 1461 1 1 9 9 89 389 389 1389 6389 178 179 TLAAAA FECAAA HHHHxx +8644 1462 0 0 4 4 44 644 644 3644 8644 88 89 MUAAAA GECAAA OOOOxx +9056 1463 0 0 6 16 56 56 1056 4056 9056 112 113 IKAAAA HECAAA VVVVxx +1423 1464 1 3 3 3 23 423 1423 1423 1423 46 47 TCAAAA IECAAA AAAAxx +4901 1465 1 1 1 1 1 901 901 4901 4901 2 3 NGAAAA JECAAA HHHHxx +3859 1466 1 3 9 19 59 859 1859 3859 3859 118 119 LSAAAA KECAAA OOOOxx +2324 1467 0 0 4 4 24 324 324 2324 2324 48 49 KLAAAA LECAAA VVVVxx +8101 1468 1 1 1 1 1 101 101 3101 8101 2 3 PZAAAA MECAAA AAAAxx +8016 1469 0 0 6 16 16 16 16 3016 8016 32 33 IWAAAA NECAAA HHHHxx +5826 1470 0 2 6 6 26 826 1826 826 5826 52 53 CQAAAA OECAAA OOOOxx +8266 1471 0 2 6 6 66 266 266 3266 8266 132 133 YFAAAA PECAAA VVVVxx +7558 1472 0 2 8 18 58 558 1558 2558 7558 116 117 SEAAAA QECAAA AAAAxx +6976 1473 0 0 6 16 76 976 976 1976 6976 152 153 IIAAAA RECAAA HHHHxx +222 1474 0 2 2 2 22 222 222 222 222 44 45 OIAAAA SECAAA OOOOxx +1624 1475 0 0 4 4 24 624 1624 1624 1624 48 49 MKAAAA TECAAA VVVVxx +1250 1476 0 2 0 10 50 250 1250 1250 1250 100 101 CWAAAA UECAAA AAAAxx +1621 1477 1 1 1 1 21 621 1621 1621 1621 42 43 JKAAAA VECAAA HHHHxx +2350 1478 0 2 0 10 50 350 350 2350 2350 100 101 KMAAAA WECAAA OOOOxx +5239 1479 1 3 9 19 39 239 1239 239 5239 78 79 NTAAAA XECAAA VVVVxx +6681 1480 1 1 1 1 81 681 681 1681 6681 162 163 ZWAAAA YECAAA AAAAxx +4983 1481 1 3 3 3 83 983 983 4983 4983 166 167 RJAAAA ZECAAA HHHHxx +7149 1482 1 1 9 9 49 149 1149 2149 7149 98 99 ZOAAAA AFCAAA OOOOxx +3502 1483 0 2 2 2 2 502 1502 3502 3502 4 5 SEAAAA BFCAAA VVVVxx +3133 1484 1 1 3 13 33 133 1133 3133 3133 66 67 NQAAAA CFCAAA AAAAxx +8342 1485 0 2 2 2 42 342 342 3342 8342 84 85 WIAAAA DFCAAA HHHHxx +3041 1486 1 1 1 1 41 41 1041 3041 3041 82 83 ZMAAAA EFCAAA OOOOxx +5383 1487 1 3 3 3 83 383 1383 383 5383 166 167 BZAAAA FFCAAA VVVVxx +3916 1488 0 0 6 16 16 916 1916 3916 3916 32 33 QUAAAA GFCAAA AAAAxx +1438 1489 0 2 8 18 38 438 1438 1438 1438 76 77 IDAAAA HFCAAA HHHHxx +9408 1490 0 0 8 8 8 408 1408 4408 9408 16 17 WXAAAA IFCAAA OOOOxx +5783 1491 1 3 3 3 83 783 1783 783 5783 166 167 LOAAAA JFCAAA VVVVxx +683 1492 1 3 3 3 83 683 683 683 683 166 167 HAAAAA KFCAAA AAAAxx +9381 1493 1 1 1 1 81 381 1381 4381 9381 162 163 VWAAAA LFCAAA HHHHxx +5676 1494 0 0 6 16 76 676 1676 676 5676 152 153 IKAAAA MFCAAA OOOOxx +3224 1495 0 0 4 4 24 224 1224 3224 3224 48 49 AUAAAA NFCAAA VVVVxx +8332 1496 0 0 2 12 32 332 332 3332 8332 64 65 MIAAAA OFCAAA AAAAxx +3372 1497 0 0 2 12 72 372 1372 3372 3372 144 145 SZAAAA PFCAAA HHHHxx +7436 1498 0 0 6 16 36 436 1436 2436 7436 72 73 AAAAAA QFCAAA OOOOxx +5010 1499 0 2 0 10 10 10 1010 10 5010 20 21 SKAAAA RFCAAA VVVVxx +7256 1500 0 0 6 16 56 256 1256 2256 7256 112 113 CTAAAA SFCAAA AAAAxx +961 1501 1 1 1 1 61 961 961 961 961 122 123 ZKAAAA TFCAAA HHHHxx +4182 1502 0 2 2 2 82 182 182 4182 4182 164 165 WEAAAA UFCAAA OOOOxx +639 1503 1 3 9 19 39 639 639 639 639 78 79 PYAAAA VFCAAA VVVVxx +8836 1504 0 0 6 16 36 836 836 3836 8836 72 73 WBAAAA WFCAAA AAAAxx +8705 1505 1 1 5 5 5 705 705 3705 8705 10 11 VWAAAA XFCAAA HHHHxx +32 1506 0 0 2 12 32 32 32 32 32 64 65 GBAAAA YFCAAA OOOOxx +7913 1507 1 1 3 13 13 913 1913 2913 7913 26 27 JSAAAA ZFCAAA VVVVxx +229 1508 1 1 9 9 29 229 229 229 229 58 59 VIAAAA AGCAAA AAAAxx +2393 1509 1 1 3 13 93 393 393 2393 2393 186 187 BOAAAA BGCAAA HHHHxx +2815 1510 1 3 5 15 15 815 815 2815 2815 30 31 HEAAAA CGCAAA OOOOxx +4858 1511 0 2 8 18 58 858 858 4858 4858 116 117 WEAAAA DGCAAA VVVVxx +6283 1512 1 3 3 3 83 283 283 1283 6283 166 167 RHAAAA EGCAAA AAAAxx +4147 1513 1 3 7 7 47 147 147 4147 4147 94 95 NDAAAA FGCAAA HHHHxx +6801 1514 1 1 1 1 1 801 801 1801 6801 2 3 PBAAAA GGCAAA OOOOxx +1011 1515 1 3 1 11 11 11 1011 1011 1011 22 23 XMAAAA HGCAAA VVVVxx +2527 1516 1 3 7 7 27 527 527 2527 2527 54 55 FTAAAA IGCAAA AAAAxx +381 1517 1 1 1 1 81 381 381 381 381 162 163 ROAAAA JGCAAA HHHHxx +3366 1518 0 2 6 6 66 366 1366 3366 3366 132 133 MZAAAA KGCAAA OOOOxx +9636 1519 0 0 6 16 36 636 1636 4636 9636 72 73 QGAAAA LGCAAA VVVVxx +2239 1520 1 3 9 19 39 239 239 2239 2239 78 79 DIAAAA MGCAAA AAAAxx +5911 1521 1 3 1 11 11 911 1911 911 5911 22 23 JTAAAA NGCAAA HHHHxx +449 1522 1 1 9 9 49 449 449 449 449 98 99 HRAAAA OGCAAA OOOOxx +5118 1523 0 2 8 18 18 118 1118 118 5118 36 37 WOAAAA PGCAAA VVVVxx +7684 1524 0 0 4 4 84 684 1684 2684 7684 168 169 OJAAAA QGCAAA AAAAxx +804 1525 0 0 4 4 4 804 804 804 804 8 9 YEAAAA RGCAAA HHHHxx +8378 1526 0 2 8 18 78 378 378 3378 8378 156 157 GKAAAA SGCAAA OOOOxx +9855 1527 1 3 5 15 55 855 1855 4855 9855 110 111 BPAAAA TGCAAA VVVVxx +1995 1528 1 3 5 15 95 995 1995 1995 1995 190 191 TYAAAA UGCAAA AAAAxx +1979 1529 1 3 9 19 79 979 1979 1979 1979 158 159 DYAAAA VGCAAA HHHHxx +4510 1530 0 2 0 10 10 510 510 4510 4510 20 21 MRAAAA WGCAAA OOOOxx +3792 1531 0 0 2 12 92 792 1792 3792 3792 184 185 WPAAAA XGCAAA VVVVxx +3541 1532 1 1 1 1 41 541 1541 3541 3541 82 83 FGAAAA YGCAAA AAAAxx +8847 1533 1 3 7 7 47 847 847 3847 8847 94 95 HCAAAA ZGCAAA HHHHxx +1336 1534 0 0 6 16 36 336 1336 1336 1336 72 73 KZAAAA AHCAAA OOOOxx +6780 1535 0 0 0 0 80 780 780 1780 6780 160 161 UAAAAA BHCAAA VVVVxx +8711 1536 1 3 1 11 11 711 711 3711 8711 22 23 BXAAAA CHCAAA AAAAxx +7839 1537 1 3 9 19 39 839 1839 2839 7839 78 79 NPAAAA DHCAAA HHHHxx +677 1538 1 1 7 17 77 677 677 677 677 154 155 BAAAAA EHCAAA OOOOxx +1574 1539 0 2 4 14 74 574 1574 1574 1574 148 149 OIAAAA FHCAAA VVVVxx +2905 1540 1 1 5 5 5 905 905 2905 2905 10 11 THAAAA GHCAAA AAAAxx +1879 1541 1 3 9 19 79 879 1879 1879 1879 158 159 HUAAAA HHCAAA HHHHxx +7820 1542 0 0 0 0 20 820 1820 2820 7820 40 41 UOAAAA IHCAAA OOOOxx +4308 1543 0 0 8 8 8 308 308 4308 4308 16 17 SJAAAA JHCAAA VVVVxx +4474 1544 0 2 4 14 74 474 474 4474 4474 148 149 CQAAAA KHCAAA AAAAxx +6985 1545 1 1 5 5 85 985 985 1985 6985 170 171 RIAAAA LHCAAA HHHHxx +6929 1546 1 1 9 9 29 929 929 1929 6929 58 59 NGAAAA MHCAAA OOOOxx +777 1547 1 1 7 17 77 777 777 777 777 154 155 XDAAAA NHCAAA VVVVxx +8271 1548 1 3 1 11 71 271 271 3271 8271 142 143 DGAAAA OHCAAA AAAAxx +2389 1549 1 1 9 9 89 389 389 2389 2389 178 179 XNAAAA PHCAAA HHHHxx +946 1550 0 2 6 6 46 946 946 946 946 92 93 KKAAAA QHCAAA OOOOxx +9682 1551 0 2 2 2 82 682 1682 4682 9682 164 165 KIAAAA RHCAAA VVVVxx +8722 1552 0 2 2 2 22 722 722 3722 8722 44 45 MXAAAA SHCAAA AAAAxx +470 1553 0 2 0 10 70 470 470 470 470 140 141 CSAAAA THCAAA HHHHxx +7425 1554 1 1 5 5 25 425 1425 2425 7425 50 51 PZAAAA UHCAAA OOOOxx +2372 1555 0 0 2 12 72 372 372 2372 2372 144 145 GNAAAA VHCAAA VVVVxx +508 1556 0 0 8 8 8 508 508 508 508 16 17 OTAAAA WHCAAA AAAAxx +163 1557 1 3 3 3 63 163 163 163 163 126 127 HGAAAA XHCAAA HHHHxx +6579 1558 1 3 9 19 79 579 579 1579 6579 158 159 BTAAAA YHCAAA OOOOxx +2355 1559 1 3 5 15 55 355 355 2355 2355 110 111 PMAAAA ZHCAAA VVVVxx +70 1560 0 2 0 10 70 70 70 70 70 140 141 SCAAAA AICAAA AAAAxx +651 1561 1 3 1 11 51 651 651 651 651 102 103 BZAAAA BICAAA HHHHxx +4436 1562 0 0 6 16 36 436 436 4436 4436 72 73 QOAAAA CICAAA OOOOxx +4240 1563 0 0 0 0 40 240 240 4240 4240 80 81 CHAAAA DICAAA VVVVxx +2722 1564 0 2 2 2 22 722 722 2722 2722 44 45 SAAAAA EICAAA AAAAxx +8937 1565 1 1 7 17 37 937 937 3937 8937 74 75 TFAAAA FICAAA HHHHxx +8364 1566 0 0 4 4 64 364 364 3364 8364 128 129 SJAAAA GICAAA OOOOxx +8317 1567 1 1 7 17 17 317 317 3317 8317 34 35 XHAAAA HICAAA VVVVxx +8872 1568 0 0 2 12 72 872 872 3872 8872 144 145 GDAAAA IICAAA AAAAxx +5512 1569 0 0 2 12 12 512 1512 512 5512 24 25 AEAAAA JICAAA HHHHxx +6651 1570 1 3 1 11 51 651 651 1651 6651 102 103 VVAAAA KICAAA OOOOxx +5976 1571 0 0 6 16 76 976 1976 976 5976 152 153 WVAAAA LICAAA VVVVxx +3301 1572 1 1 1 1 1 301 1301 3301 3301 2 3 ZWAAAA MICAAA AAAAxx +6784 1573 0 0 4 4 84 784 784 1784 6784 168 169 YAAAAA NICAAA HHHHxx +573 1574 1 1 3 13 73 573 573 573 573 146 147 BWAAAA OICAAA OOOOxx +3015 1575 1 3 5 15 15 15 1015 3015 3015 30 31 ZLAAAA PICAAA VVVVxx +8245 1576 1 1 5 5 45 245 245 3245 8245 90 91 DFAAAA QICAAA AAAAxx +5251 1577 1 3 1 11 51 251 1251 251 5251 102 103 ZTAAAA RICAAA HHHHxx +2281 1578 1 1 1 1 81 281 281 2281 2281 162 163 TJAAAA SICAAA OOOOxx +518 1579 0 2 8 18 18 518 518 518 518 36 37 YTAAAA TICAAA VVVVxx +9839 1580 1 3 9 19 39 839 1839 4839 9839 78 79 LOAAAA UICAAA AAAAxx +4526 1581 0 2 6 6 26 526 526 4526 4526 52 53 CSAAAA VICAAA HHHHxx +1261 1582 1 1 1 1 61 261 1261 1261 1261 122 123 NWAAAA WICAAA OOOOxx +4259 1583 1 3 9 19 59 259 259 4259 4259 118 119 VHAAAA XICAAA VVVVxx +9098 1584 0 2 8 18 98 98 1098 4098 9098 196 197 YLAAAA YICAAA AAAAxx +6037 1585 1 1 7 17 37 37 37 1037 6037 74 75 FYAAAA ZICAAA HHHHxx +4284 1586 0 0 4 4 84 284 284 4284 4284 168 169 UIAAAA AJCAAA OOOOxx +3267 1587 1 3 7 7 67 267 1267 3267 3267 134 135 RVAAAA BJCAAA VVVVxx +5908 1588 0 0 8 8 8 908 1908 908 5908 16 17 GTAAAA CJCAAA AAAAxx +1549 1589 1 1 9 9 49 549 1549 1549 1549 98 99 PHAAAA DJCAAA HHHHxx +8736 1590 0 0 6 16 36 736 736 3736 8736 72 73 AYAAAA EJCAAA OOOOxx +2008 1591 0 0 8 8 8 8 8 2008 2008 16 17 GZAAAA FJCAAA VVVVxx +548 1592 0 0 8 8 48 548 548 548 548 96 97 CVAAAA GJCAAA AAAAxx +8846 1593 0 2 6 6 46 846 846 3846 8846 92 93 GCAAAA HJCAAA HHHHxx +8374 1594 0 2 4 14 74 374 374 3374 8374 148 149 CKAAAA IJCAAA OOOOxx +7986 1595 0 2 6 6 86 986 1986 2986 7986 172 173 EVAAAA JJCAAA VVVVxx +6819 1596 1 3 9 19 19 819 819 1819 6819 38 39 HCAAAA KJCAAA AAAAxx +4418 1597 0 2 8 18 18 418 418 4418 4418 36 37 YNAAAA LJCAAA HHHHxx +833 1598 1 1 3 13 33 833 833 833 833 66 67 BGAAAA MJCAAA OOOOxx +4416 1599 0 0 6 16 16 416 416 4416 4416 32 33 WNAAAA NJCAAA VVVVxx +4902 1600 0 2 2 2 2 902 902 4902 4902 4 5 OGAAAA OJCAAA AAAAxx +6828 1601 0 0 8 8 28 828 828 1828 6828 56 57 QCAAAA PJCAAA HHHHxx +1118 1602 0 2 8 18 18 118 1118 1118 1118 36 37 ARAAAA QJCAAA OOOOxx +9993 1603 1 1 3 13 93 993 1993 4993 9993 186 187 JUAAAA RJCAAA VVVVxx +1430 1604 0 2 0 10 30 430 1430 1430 1430 60 61 ADAAAA SJCAAA AAAAxx +5670 1605 0 2 0 10 70 670 1670 670 5670 140 141 CKAAAA TJCAAA HHHHxx +5424 1606 0 0 4 4 24 424 1424 424 5424 48 49 QAAAAA UJCAAA OOOOxx +5561 1607 1 1 1 1 61 561 1561 561 5561 122 123 XFAAAA VJCAAA VVVVxx +2027 1608 1 3 7 7 27 27 27 2027 2027 54 55 ZZAAAA WJCAAA AAAAxx +6924 1609 0 0 4 4 24 924 924 1924 6924 48 49 IGAAAA XJCAAA HHHHxx +5946 1610 0 2 6 6 46 946 1946 946 5946 92 93 SUAAAA YJCAAA OOOOxx +4294 1611 0 2 4 14 94 294 294 4294 4294 188 189 EJAAAA ZJCAAA VVVVxx +2936 1612 0 0 6 16 36 936 936 2936 2936 72 73 YIAAAA AKCAAA AAAAxx +3855 1613 1 3 5 15 55 855 1855 3855 3855 110 111 HSAAAA BKCAAA HHHHxx +455 1614 1 3 5 15 55 455 455 455 455 110 111 NRAAAA CKCAAA OOOOxx +2918 1615 0 2 8 18 18 918 918 2918 2918 36 37 GIAAAA DKCAAA VVVVxx +448 1616 0 0 8 8 48 448 448 448 448 96 97 GRAAAA EKCAAA AAAAxx +2149 1617 1 1 9 9 49 149 149 2149 2149 98 99 REAAAA FKCAAA HHHHxx +8890 1618 0 2 0 10 90 890 890 3890 8890 180 181 YDAAAA GKCAAA OOOOxx +8919 1619 1 3 9 19 19 919 919 3919 8919 38 39 BFAAAA HKCAAA VVVVxx +4957 1620 1 1 7 17 57 957 957 4957 4957 114 115 RIAAAA IKCAAA AAAAxx +4 1621 0 0 4 4 4 4 4 4 4 8 9 EAAAAA JKCAAA HHHHxx +4837 1622 1 1 7 17 37 837 837 4837 4837 74 75 BEAAAA KKCAAA OOOOxx +3976 1623 0 0 6 16 76 976 1976 3976 3976 152 153 YWAAAA LKCAAA VVVVxx +9459 1624 1 3 9 19 59 459 1459 4459 9459 118 119 VZAAAA MKCAAA AAAAxx +7097 1625 1 1 7 17 97 97 1097 2097 7097 194 195 ZMAAAA NKCAAA HHHHxx +9226 1626 0 2 6 6 26 226 1226 4226 9226 52 53 WQAAAA OKCAAA OOOOxx +5803 1627 1 3 3 3 3 803 1803 803 5803 6 7 FPAAAA PKCAAA VVVVxx +21 1628 1 1 1 1 21 21 21 21 21 42 43 VAAAAA QKCAAA AAAAxx +5275 1629 1 3 5 15 75 275 1275 275 5275 150 151 XUAAAA RKCAAA HHHHxx +3488 1630 0 0 8 8 88 488 1488 3488 3488 176 177 EEAAAA SKCAAA OOOOxx +1595 1631 1 3 5 15 95 595 1595 1595 1595 190 191 JJAAAA TKCAAA VVVVxx +5212 1632 0 0 2 12 12 212 1212 212 5212 24 25 MSAAAA UKCAAA AAAAxx +6574 1633 0 2 4 14 74 574 574 1574 6574 148 149 WSAAAA VKCAAA HHHHxx +7524 1634 0 0 4 4 24 524 1524 2524 7524 48 49 KDAAAA WKCAAA OOOOxx +6100 1635 0 0 0 0 0 100 100 1100 6100 0 1 QAAAAA XKCAAA VVVVxx +1198 1636 0 2 8 18 98 198 1198 1198 1198 196 197 CUAAAA YKCAAA AAAAxx +7345 1637 1 1 5 5 45 345 1345 2345 7345 90 91 NWAAAA ZKCAAA HHHHxx +5020 1638 0 0 0 0 20 20 1020 20 5020 40 41 CLAAAA ALCAAA OOOOxx +6925 1639 1 1 5 5 25 925 925 1925 6925 50 51 JGAAAA BLCAAA VVVVxx +8915 1640 1 3 5 15 15 915 915 3915 8915 30 31 XEAAAA CLCAAA AAAAxx +3088 1641 0 0 8 8 88 88 1088 3088 3088 176 177 UOAAAA DLCAAA HHHHxx +4828 1642 0 0 8 8 28 828 828 4828 4828 56 57 SDAAAA ELCAAA OOOOxx +7276 1643 0 0 6 16 76 276 1276 2276 7276 152 153 WTAAAA FLCAAA VVVVxx +299 1644 1 3 9 19 99 299 299 299 299 198 199 NLAAAA GLCAAA AAAAxx +76 1645 0 0 6 16 76 76 76 76 76 152 153 YCAAAA HLCAAA HHHHxx +8458 1646 0 2 8 18 58 458 458 3458 8458 116 117 INAAAA ILCAAA OOOOxx +7207 1647 1 3 7 7 7 207 1207 2207 7207 14 15 FRAAAA JLCAAA VVVVxx +5585 1648 1 1 5 5 85 585 1585 585 5585 170 171 VGAAAA KLCAAA AAAAxx +3234 1649 0 2 4 14 34 234 1234 3234 3234 68 69 KUAAAA LLCAAA HHHHxx +8001 1650 1 1 1 1 1 1 1 3001 8001 2 3 TVAAAA MLCAAA OOOOxx +1319 1651 1 3 9 19 19 319 1319 1319 1319 38 39 TYAAAA NLCAAA VVVVxx +6342 1652 0 2 2 2 42 342 342 1342 6342 84 85 YJAAAA OLCAAA AAAAxx +9199 1653 1 3 9 19 99 199 1199 4199 9199 198 199 VPAAAA PLCAAA HHHHxx +5696 1654 0 0 6 16 96 696 1696 696 5696 192 193 CLAAAA QLCAAA OOOOxx +2562 1655 0 2 2 2 62 562 562 2562 2562 124 125 OUAAAA RLCAAA VVVVxx +4226 1656 0 2 6 6 26 226 226 4226 4226 52 53 OGAAAA SLCAAA AAAAxx +1184 1657 0 0 4 4 84 184 1184 1184 1184 168 169 OTAAAA TLCAAA HHHHxx +5807 1658 1 3 7 7 7 807 1807 807 5807 14 15 JPAAAA ULCAAA OOOOxx +1890 1659 0 2 0 10 90 890 1890 1890 1890 180 181 SUAAAA VLCAAA VVVVxx +451 1660 1 3 1 11 51 451 451 451 451 102 103 JRAAAA WLCAAA AAAAxx +1049 1661 1 1 9 9 49 49 1049 1049 1049 98 99 JOAAAA XLCAAA HHHHxx +5272 1662 0 0 2 12 72 272 1272 272 5272 144 145 UUAAAA YLCAAA OOOOxx +4588 1663 0 0 8 8 88 588 588 4588 4588 176 177 MUAAAA ZLCAAA VVVVxx +5213 1664 1 1 3 13 13 213 1213 213 5213 26 27 NSAAAA AMCAAA AAAAxx +9543 1665 1 3 3 3 43 543 1543 4543 9543 86 87 BDAAAA BMCAAA HHHHxx +6318 1666 0 2 8 18 18 318 318 1318 6318 36 37 AJAAAA CMCAAA OOOOxx +7992 1667 0 0 2 12 92 992 1992 2992 7992 184 185 KVAAAA DMCAAA VVVVxx +4619 1668 1 3 9 19 19 619 619 4619 4619 38 39 RVAAAA EMCAAA AAAAxx +7189 1669 1 1 9 9 89 189 1189 2189 7189 178 179 NQAAAA FMCAAA HHHHxx +2178 1670 0 2 8 18 78 178 178 2178 2178 156 157 UFAAAA GMCAAA OOOOxx +4928 1671 0 0 8 8 28 928 928 4928 4928 56 57 OHAAAA HMCAAA VVVVxx +3966 1672 0 2 6 6 66 966 1966 3966 3966 132 133 OWAAAA IMCAAA AAAAxx +9790 1673 0 2 0 10 90 790 1790 4790 9790 180 181 OMAAAA JMCAAA HHHHxx +9150 1674 0 2 0 10 50 150 1150 4150 9150 100 101 YNAAAA KMCAAA OOOOxx +313 1675 1 1 3 13 13 313 313 313 313 26 27 BMAAAA LMCAAA VVVVxx +1614 1676 0 2 4 14 14 614 1614 1614 1614 28 29 CKAAAA MMCAAA AAAAxx +1581 1677 1 1 1 1 81 581 1581 1581 1581 162 163 VIAAAA NMCAAA HHHHxx +3674 1678 0 2 4 14 74 674 1674 3674 3674 148 149 ILAAAA OMCAAA OOOOxx +3444 1679 0 0 4 4 44 444 1444 3444 3444 88 89 MCAAAA PMCAAA VVVVxx +1050 1680 0 2 0 10 50 50 1050 1050 1050 100 101 KOAAAA QMCAAA AAAAxx +8241 1681 1 1 1 1 41 241 241 3241 8241 82 83 ZEAAAA RMCAAA HHHHxx +3382 1682 0 2 2 2 82 382 1382 3382 3382 164 165 CAAAAA SMCAAA OOOOxx +7105 1683 1 1 5 5 5 105 1105 2105 7105 10 11 HNAAAA TMCAAA VVVVxx +2957 1684 1 1 7 17 57 957 957 2957 2957 114 115 TJAAAA UMCAAA AAAAxx +6162 1685 0 2 2 2 62 162 162 1162 6162 124 125 ADAAAA VMCAAA HHHHxx +5150 1686 0 2 0 10 50 150 1150 150 5150 100 101 CQAAAA WMCAAA OOOOxx +2622 1687 0 2 2 2 22 622 622 2622 2622 44 45 WWAAAA XMCAAA VVVVxx +2240 1688 0 0 0 0 40 240 240 2240 2240 80 81 EIAAAA YMCAAA AAAAxx +8880 1689 0 0 0 0 80 880 880 3880 8880 160 161 ODAAAA ZMCAAA HHHHxx +9250 1690 0 2 0 10 50 250 1250 4250 9250 100 101 URAAAA ANCAAA OOOOxx +7010 1691 0 2 0 10 10 10 1010 2010 7010 20 21 QJAAAA BNCAAA VVVVxx +1098 1692 0 2 8 18 98 98 1098 1098 1098 196 197 GQAAAA CNCAAA AAAAxx +648 1693 0 0 8 8 48 648 648 648 648 96 97 YYAAAA DNCAAA HHHHxx +5536 1694 0 0 6 16 36 536 1536 536 5536 72 73 YEAAAA ENCAAA OOOOxx +7858 1695 0 2 8 18 58 858 1858 2858 7858 116 117 GQAAAA FNCAAA VVVVxx +7053 1696 1 1 3 13 53 53 1053 2053 7053 106 107 HLAAAA GNCAAA AAAAxx +8681 1697 1 1 1 1 81 681 681 3681 8681 162 163 XVAAAA HNCAAA HHHHxx +8832 1698 0 0 2 12 32 832 832 3832 8832 64 65 SBAAAA INCAAA OOOOxx +6836 1699 0 0 6 16 36 836 836 1836 6836 72 73 YCAAAA JNCAAA VVVVxx +4856 1700 0 0 6 16 56 856 856 4856 4856 112 113 UEAAAA KNCAAA AAAAxx +345 1701 1 1 5 5 45 345 345 345 345 90 91 HNAAAA LNCAAA HHHHxx +6559 1702 1 3 9 19 59 559 559 1559 6559 118 119 HSAAAA MNCAAA OOOOxx +3017 1703 1 1 7 17 17 17 1017 3017 3017 34 35 BMAAAA NNCAAA VVVVxx +4176 1704 0 0 6 16 76 176 176 4176 4176 152 153 QEAAAA ONCAAA AAAAxx +2839 1705 1 3 9 19 39 839 839 2839 2839 78 79 FFAAAA PNCAAA HHHHxx +6065 1706 1 1 5 5 65 65 65 1065 6065 130 131 HZAAAA QNCAAA OOOOxx +7360 1707 0 0 0 0 60 360 1360 2360 7360 120 121 CXAAAA RNCAAA VVVVxx +9527 1708 1 3 7 7 27 527 1527 4527 9527 54 55 LCAAAA SNCAAA AAAAxx +8849 1709 1 1 9 9 49 849 849 3849 8849 98 99 JCAAAA TNCAAA HHHHxx +7274 1710 0 2 4 14 74 274 1274 2274 7274 148 149 UTAAAA UNCAAA OOOOxx +4368 1711 0 0 8 8 68 368 368 4368 4368 136 137 AMAAAA VNCAAA VVVVxx +2488 1712 0 0 8 8 88 488 488 2488 2488 176 177 SRAAAA WNCAAA AAAAxx +4674 1713 0 2 4 14 74 674 674 4674 4674 148 149 UXAAAA XNCAAA HHHHxx +365 1714 1 1 5 5 65 365 365 365 365 130 131 BOAAAA YNCAAA OOOOxx +5897 1715 1 1 7 17 97 897 1897 897 5897 194 195 VSAAAA ZNCAAA VVVVxx +8918 1716 0 2 8 18 18 918 918 3918 8918 36 37 AFAAAA AOCAAA AAAAxx +1988 1717 0 0 8 8 88 988 1988 1988 1988 176 177 MYAAAA BOCAAA HHHHxx +1210 1718 0 2 0 10 10 210 1210 1210 1210 20 21 OUAAAA COCAAA OOOOxx +2945 1719 1 1 5 5 45 945 945 2945 2945 90 91 HJAAAA DOCAAA VVVVxx +555 1720 1 3 5 15 55 555 555 555 555 110 111 JVAAAA EOCAAA AAAAxx +9615 1721 1 3 5 15 15 615 1615 4615 9615 30 31 VFAAAA FOCAAA HHHHxx +9939 1722 1 3 9 19 39 939 1939 4939 9939 78 79 HSAAAA GOCAAA OOOOxx +1216 1723 0 0 6 16 16 216 1216 1216 1216 32 33 UUAAAA HOCAAA VVVVxx +745 1724 1 1 5 5 45 745 745 745 745 90 91 RCAAAA IOCAAA AAAAxx +3326 1725 0 2 6 6 26 326 1326 3326 3326 52 53 YXAAAA JOCAAA HHHHxx +953 1726 1 1 3 13 53 953 953 953 953 106 107 RKAAAA KOCAAA OOOOxx +444 1727 0 0 4 4 44 444 444 444 444 88 89 CRAAAA LOCAAA VVVVxx +280 1728 0 0 0 0 80 280 280 280 280 160 161 UKAAAA MOCAAA AAAAxx +3707 1729 1 3 7 7 7 707 1707 3707 3707 14 15 PMAAAA NOCAAA HHHHxx +1351 1730 1 3 1 11 51 351 1351 1351 1351 102 103 ZZAAAA OOCAAA OOOOxx +1280 1731 0 0 0 0 80 280 1280 1280 1280 160 161 GXAAAA POCAAA VVVVxx +628 1732 0 0 8 8 28 628 628 628 628 56 57 EYAAAA QOCAAA AAAAxx +6198 1733 0 2 8 18 98 198 198 1198 6198 196 197 KEAAAA ROCAAA HHHHxx +1957 1734 1 1 7 17 57 957 1957 1957 1957 114 115 HXAAAA SOCAAA OOOOxx +9241 1735 1 1 1 1 41 241 1241 4241 9241 82 83 LRAAAA TOCAAA VVVVxx +303 1736 1 3 3 3 3 303 303 303 303 6 7 RLAAAA UOCAAA AAAAxx +1945 1737 1 1 5 5 45 945 1945 1945 1945 90 91 VWAAAA VOCAAA HHHHxx +3634 1738 0 2 4 14 34 634 1634 3634 3634 68 69 UJAAAA WOCAAA OOOOxx +4768 1739 0 0 8 8 68 768 768 4768 4768 136 137 KBAAAA XOCAAA VVVVxx +9262 1740 0 2 2 2 62 262 1262 4262 9262 124 125 GSAAAA YOCAAA AAAAxx +2610 1741 0 2 0 10 10 610 610 2610 2610 20 21 KWAAAA ZOCAAA HHHHxx +6640 1742 0 0 0 0 40 640 640 1640 6640 80 81 KVAAAA APCAAA OOOOxx +3338 1743 0 2 8 18 38 338 1338 3338 3338 76 77 KYAAAA BPCAAA VVVVxx +6560 1744 0 0 0 0 60 560 560 1560 6560 120 121 ISAAAA CPCAAA AAAAxx +5986 1745 0 2 6 6 86 986 1986 986 5986 172 173 GWAAAA DPCAAA HHHHxx +2970 1746 0 2 0 10 70 970 970 2970 2970 140 141 GKAAAA EPCAAA OOOOxx +4731 1747 1 3 1 11 31 731 731 4731 4731 62 63 ZZAAAA FPCAAA VVVVxx +9486 1748 0 2 6 6 86 486 1486 4486 9486 172 173 WAAAAA GPCAAA AAAAxx +7204 1749 0 0 4 4 4 204 1204 2204 7204 8 9 CRAAAA HPCAAA HHHHxx +6685 1750 1 1 5 5 85 685 685 1685 6685 170 171 DXAAAA IPCAAA OOOOxx +6852 1751 0 0 2 12 52 852 852 1852 6852 104 105 ODAAAA JPCAAA VVVVxx +2325 1752 1 1 5 5 25 325 325 2325 2325 50 51 LLAAAA KPCAAA AAAAxx +1063 1753 1 3 3 3 63 63 1063 1063 1063 126 127 XOAAAA LPCAAA HHHHxx +6810 1754 0 2 0 10 10 810 810 1810 6810 20 21 YBAAAA MPCAAA OOOOxx +7718 1755 0 2 8 18 18 718 1718 2718 7718 36 37 WKAAAA NPCAAA VVVVxx +1680 1756 0 0 0 0 80 680 1680 1680 1680 160 161 QMAAAA OPCAAA AAAAxx +7402 1757 0 2 2 2 2 402 1402 2402 7402 4 5 SYAAAA PPCAAA HHHHxx +4134 1758 0 2 4 14 34 134 134 4134 4134 68 69 ADAAAA QPCAAA OOOOxx +8232 1759 0 0 2 12 32 232 232 3232 8232 64 65 QEAAAA RPCAAA VVVVxx +6682 1760 0 2 2 2 82 682 682 1682 6682 164 165 AXAAAA SPCAAA AAAAxx +7952 1761 0 0 2 12 52 952 1952 2952 7952 104 105 WTAAAA TPCAAA HHHHxx +5943 1762 1 3 3 3 43 943 1943 943 5943 86 87 PUAAAA UPCAAA OOOOxx +5394 1763 0 2 4 14 94 394 1394 394 5394 188 189 MZAAAA VPCAAA VVVVxx +6554 1764 0 2 4 14 54 554 554 1554 6554 108 109 CSAAAA WPCAAA AAAAxx +8186 1765 0 2 6 6 86 186 186 3186 8186 172 173 WCAAAA XPCAAA HHHHxx +199 1766 1 3 9 19 99 199 199 199 199 198 199 RHAAAA YPCAAA OOOOxx +3386 1767 0 2 6 6 86 386 1386 3386 3386 172 173 GAAAAA ZPCAAA VVVVxx +8974 1768 0 2 4 14 74 974 974 3974 8974 148 149 EHAAAA AQCAAA AAAAxx +8140 1769 0 0 0 0 40 140 140 3140 8140 80 81 CBAAAA BQCAAA HHHHxx +3723 1770 1 3 3 3 23 723 1723 3723 3723 46 47 FNAAAA CQCAAA OOOOxx +8827 1771 1 3 7 7 27 827 827 3827 8827 54 55 NBAAAA DQCAAA VVVVxx +1998 1772 0 2 8 18 98 998 1998 1998 1998 196 197 WYAAAA EQCAAA AAAAxx +879 1773 1 3 9 19 79 879 879 879 879 158 159 VHAAAA FQCAAA HHHHxx +892 1774 0 0 2 12 92 892 892 892 892 184 185 IIAAAA GQCAAA OOOOxx +9468 1775 0 0 8 8 68 468 1468 4468 9468 136 137 EAAAAA HQCAAA VVVVxx +3797 1776 1 1 7 17 97 797 1797 3797 3797 194 195 BQAAAA IQCAAA AAAAxx +8379 1777 1 3 9 19 79 379 379 3379 8379 158 159 HKAAAA JQCAAA HHHHxx +2817 1778 1 1 7 17 17 817 817 2817 2817 34 35 JEAAAA KQCAAA OOOOxx +789 1779 1 1 9 9 89 789 789 789 789 178 179 JEAAAA LQCAAA VVVVxx +3871 1780 1 3 1 11 71 871 1871 3871 3871 142 143 XSAAAA MQCAAA AAAAxx +7931 1781 1 3 1 11 31 931 1931 2931 7931 62 63 BTAAAA NQCAAA HHHHxx +3636 1782 0 0 6 16 36 636 1636 3636 3636 72 73 WJAAAA OQCAAA OOOOxx +699 1783 1 3 9 19 99 699 699 699 699 198 199 XAAAAA PQCAAA VVVVxx +6850 1784 0 2 0 10 50 850 850 1850 6850 100 101 MDAAAA QQCAAA AAAAxx +6394 1785 0 2 4 14 94 394 394 1394 6394 188 189 YLAAAA RQCAAA HHHHxx +3475 1786 1 3 5 15 75 475 1475 3475 3475 150 151 RDAAAA SQCAAA OOOOxx +3026 1787 0 2 6 6 26 26 1026 3026 3026 52 53 KMAAAA TQCAAA VVVVxx +876 1788 0 0 6 16 76 876 876 876 876 152 153 SHAAAA UQCAAA AAAAxx +1992 1789 0 0 2 12 92 992 1992 1992 1992 184 185 QYAAAA VQCAAA HHHHxx +3079 1790 1 3 9 19 79 79 1079 3079 3079 158 159 LOAAAA WQCAAA OOOOxx +8128 1791 0 0 8 8 28 128 128 3128 8128 56 57 QAAAAA XQCAAA VVVVxx +8123 1792 1 3 3 3 23 123 123 3123 8123 46 47 LAAAAA YQCAAA AAAAxx +3285 1793 1 1 5 5 85 285 1285 3285 3285 170 171 JWAAAA ZQCAAA HHHHxx +9315 1794 1 3 5 15 15 315 1315 4315 9315 30 31 HUAAAA ARCAAA OOOOxx +9862 1795 0 2 2 2 62 862 1862 4862 9862 124 125 IPAAAA BRCAAA VVVVxx +2764 1796 0 0 4 4 64 764 764 2764 2764 128 129 ICAAAA CRCAAA AAAAxx +3544 1797 0 0 4 4 44 544 1544 3544 3544 88 89 IGAAAA DRCAAA HHHHxx +7747 1798 1 3 7 7 47 747 1747 2747 7747 94 95 ZLAAAA ERCAAA OOOOxx +7725 1799 1 1 5 5 25 725 1725 2725 7725 50 51 DLAAAA FRCAAA VVVVxx +2449 1800 1 1 9 9 49 449 449 2449 2449 98 99 FQAAAA GRCAAA AAAAxx +8967 1801 1 3 7 7 67 967 967 3967 8967 134 135 XGAAAA HRCAAA HHHHxx +7371 1802 1 3 1 11 71 371 1371 2371 7371 142 143 NXAAAA IRCAAA OOOOxx +2158 1803 0 2 8 18 58 158 158 2158 2158 116 117 AFAAAA JRCAAA VVVVxx +5590 1804 0 2 0 10 90 590 1590 590 5590 180 181 AHAAAA KRCAAA AAAAxx +8072 1805 0 0 2 12 72 72 72 3072 8072 144 145 MYAAAA LRCAAA HHHHxx +1971 1806 1 3 1 11 71 971 1971 1971 1971 142 143 VXAAAA MRCAAA OOOOxx +772 1807 0 0 2 12 72 772 772 772 772 144 145 SDAAAA NRCAAA VVVVxx +3433 1808 1 1 3 13 33 433 1433 3433 3433 66 67 BCAAAA ORCAAA AAAAxx +8419 1809 1 3 9 19 19 419 419 3419 8419 38 39 VLAAAA PRCAAA HHHHxx +1493 1810 1 1 3 13 93 493 1493 1493 1493 186 187 LFAAAA QRCAAA OOOOxx +2584 1811 0 0 4 4 84 584 584 2584 2584 168 169 KVAAAA RRCAAA VVVVxx +9502 1812 0 2 2 2 2 502 1502 4502 9502 4 5 MBAAAA SRCAAA AAAAxx +4673 1813 1 1 3 13 73 673 673 4673 4673 146 147 TXAAAA TRCAAA HHHHxx +7403 1814 1 3 3 3 3 403 1403 2403 7403 6 7 TYAAAA URCAAA OOOOxx +7103 1815 1 3 3 3 3 103 1103 2103 7103 6 7 FNAAAA VRCAAA VVVVxx +7026 1816 0 2 6 6 26 26 1026 2026 7026 52 53 GKAAAA WRCAAA AAAAxx +8574 1817 0 2 4 14 74 574 574 3574 8574 148 149 URAAAA XRCAAA HHHHxx +1366 1818 0 2 6 6 66 366 1366 1366 1366 132 133 OAAAAA YRCAAA OOOOxx +5787 1819 1 3 7 7 87 787 1787 787 5787 174 175 POAAAA ZRCAAA VVVVxx +2552 1820 0 0 2 12 52 552 552 2552 2552 104 105 EUAAAA ASCAAA AAAAxx +4557 1821 1 1 7 17 57 557 557 4557 4557 114 115 HTAAAA BSCAAA HHHHxx +3237 1822 1 1 7 17 37 237 1237 3237 3237 74 75 NUAAAA CSCAAA OOOOxx +6901 1823 1 1 1 1 1 901 901 1901 6901 2 3 LFAAAA DSCAAA VVVVxx +7708 1824 0 0 8 8 8 708 1708 2708 7708 16 17 MKAAAA ESCAAA AAAAxx +2011 1825 1 3 1 11 11 11 11 2011 2011 22 23 JZAAAA FSCAAA HHHHxx +9455 1826 1 3 5 15 55 455 1455 4455 9455 110 111 RZAAAA GSCAAA OOOOxx +5228 1827 0 0 8 8 28 228 1228 228 5228 56 57 CTAAAA HSCAAA VVVVxx +4043 1828 1 3 3 3 43 43 43 4043 4043 86 87 NZAAAA ISCAAA AAAAxx +8242 1829 0 2 2 2 42 242 242 3242 8242 84 85 AFAAAA JSCAAA HHHHxx +6351 1830 1 3 1 11 51 351 351 1351 6351 102 103 HKAAAA KSCAAA OOOOxx +5899 1831 1 3 9 19 99 899 1899 899 5899 198 199 XSAAAA LSCAAA VVVVxx +4849 1832 1 1 9 9 49 849 849 4849 4849 98 99 NEAAAA MSCAAA AAAAxx +9583 1833 1 3 3 3 83 583 1583 4583 9583 166 167 PEAAAA NSCAAA HHHHxx +4994 1834 0 2 4 14 94 994 994 4994 4994 188 189 CKAAAA OSCAAA OOOOxx +9787 1835 1 3 7 7 87 787 1787 4787 9787 174 175 LMAAAA PSCAAA VVVVxx +243 1836 1 3 3 3 43 243 243 243 243 86 87 JJAAAA QSCAAA AAAAxx +3931 1837 1 3 1 11 31 931 1931 3931 3931 62 63 FVAAAA RSCAAA HHHHxx +5945 1838 1 1 5 5 45 945 1945 945 5945 90 91 RUAAAA SSCAAA OOOOxx +1325 1839 1 1 5 5 25 325 1325 1325 1325 50 51 ZYAAAA TSCAAA VVVVxx +4142 1840 0 2 2 2 42 142 142 4142 4142 84 85 IDAAAA USCAAA AAAAxx +1963 1841 1 3 3 3 63 963 1963 1963 1963 126 127 NXAAAA VSCAAA HHHHxx +7041 1842 1 1 1 1 41 41 1041 2041 7041 82 83 VKAAAA WSCAAA OOOOxx +3074 1843 0 2 4 14 74 74 1074 3074 3074 148 149 GOAAAA XSCAAA VVVVxx +3290 1844 0 2 0 10 90 290 1290 3290 3290 180 181 OWAAAA YSCAAA AAAAxx +4146 1845 0 2 6 6 46 146 146 4146 4146 92 93 MDAAAA ZSCAAA HHHHxx +3832 1846 0 0 2 12 32 832 1832 3832 3832 64 65 KRAAAA ATCAAA OOOOxx +2217 1847 1 1 7 17 17 217 217 2217 2217 34 35 HHAAAA BTCAAA VVVVxx +635 1848 1 3 5 15 35 635 635 635 635 70 71 LYAAAA CTCAAA AAAAxx +6967 1849 1 3 7 7 67 967 967 1967 6967 134 135 ZHAAAA DTCAAA HHHHxx +3522 1850 0 2 2 2 22 522 1522 3522 3522 44 45 MFAAAA ETCAAA OOOOxx +2471 1851 1 3 1 11 71 471 471 2471 2471 142 143 BRAAAA FTCAAA VVVVxx +4236 1852 0 0 6 16 36 236 236 4236 4236 72 73 YGAAAA GTCAAA AAAAxx +853 1853 1 1 3 13 53 853 853 853 853 106 107 VGAAAA HTCAAA HHHHxx +3754 1854 0 2 4 14 54 754 1754 3754 3754 108 109 KOAAAA ITCAAA OOOOxx +796 1855 0 0 6 16 96 796 796 796 796 192 193 QEAAAA JTCAAA VVVVxx +4640 1856 0 0 0 0 40 640 640 4640 4640 80 81 MWAAAA KTCAAA AAAAxx +9496 1857 0 0 6 16 96 496 1496 4496 9496 192 193 GBAAAA LTCAAA HHHHxx +6873 1858 1 1 3 13 73 873 873 1873 6873 146 147 JEAAAA MTCAAA OOOOxx +4632 1859 0 0 2 12 32 632 632 4632 4632 64 65 EWAAAA NTCAAA VVVVxx +5758 1860 0 2 8 18 58 758 1758 758 5758 116 117 MNAAAA OTCAAA AAAAxx +6514 1861 0 2 4 14 14 514 514 1514 6514 28 29 OQAAAA PTCAAA HHHHxx +9510 1862 0 2 0 10 10 510 1510 4510 9510 20 21 UBAAAA QTCAAA OOOOxx +8411 1863 1 3 1 11 11 411 411 3411 8411 22 23 NLAAAA RTCAAA VVVVxx +7762 1864 0 2 2 2 62 762 1762 2762 7762 124 125 OMAAAA STCAAA AAAAxx +2225 1865 1 1 5 5 25 225 225 2225 2225 50 51 PHAAAA TTCAAA HHHHxx +4373 1866 1 1 3 13 73 373 373 4373 4373 146 147 FMAAAA UTCAAA OOOOxx +7326 1867 0 2 6 6 26 326 1326 2326 7326 52 53 UVAAAA VTCAAA VVVVxx +8651 1868 1 3 1 11 51 651 651 3651 8651 102 103 TUAAAA WTCAAA AAAAxx +9825 1869 1 1 5 5 25 825 1825 4825 9825 50 51 XNAAAA XTCAAA HHHHxx +2988 1870 0 0 8 8 88 988 988 2988 2988 176 177 YKAAAA YTCAAA OOOOxx +8138 1871 0 2 8 18 38 138 138 3138 8138 76 77 ABAAAA ZTCAAA VVVVxx +7792 1872 0 0 2 12 92 792 1792 2792 7792 184 185 SNAAAA AUCAAA AAAAxx +1232 1873 0 0 2 12 32 232 1232 1232 1232 64 65 KVAAAA BUCAAA HHHHxx +8221 1874 1 1 1 1 21 221 221 3221 8221 42 43 FEAAAA CUCAAA OOOOxx +4044 1875 0 0 4 4 44 44 44 4044 4044 88 89 OZAAAA DUCAAA VVVVxx +1204 1876 0 0 4 4 4 204 1204 1204 1204 8 9 IUAAAA EUCAAA AAAAxx +5145 1877 1 1 5 5 45 145 1145 145 5145 90 91 XPAAAA FUCAAA HHHHxx +7791 1878 1 3 1 11 91 791 1791 2791 7791 182 183 RNAAAA GUCAAA OOOOxx +8270 1879 0 2 0 10 70 270 270 3270 8270 140 141 CGAAAA HUCAAA VVVVxx +9427 1880 1 3 7 7 27 427 1427 4427 9427 54 55 PYAAAA IUCAAA AAAAxx +2152 1881 0 0 2 12 52 152 152 2152 2152 104 105 UEAAAA JUCAAA HHHHxx +7790 1882 0 2 0 10 90 790 1790 2790 7790 180 181 QNAAAA KUCAAA OOOOxx +5301 1883 1 1 1 1 1 301 1301 301 5301 2 3 XVAAAA LUCAAA VVVVxx +626 1884 0 2 6 6 26 626 626 626 626 52 53 CYAAAA MUCAAA AAAAxx +260 1885 0 0 0 0 60 260 260 260 260 120 121 AKAAAA NUCAAA HHHHxx +4369 1886 1 1 9 9 69 369 369 4369 4369 138 139 BMAAAA OUCAAA OOOOxx +5457 1887 1 1 7 17 57 457 1457 457 5457 114 115 XBAAAA PUCAAA VVVVxx +3468 1888 0 0 8 8 68 468 1468 3468 3468 136 137 KDAAAA QUCAAA AAAAxx +2257 1889 1 1 7 17 57 257 257 2257 2257 114 115 VIAAAA RUCAAA HHHHxx +9318 1890 0 2 8 18 18 318 1318 4318 9318 36 37 KUAAAA SUCAAA OOOOxx +8762 1891 0 2 2 2 62 762 762 3762 8762 124 125 AZAAAA TUCAAA VVVVxx +9153 1892 1 1 3 13 53 153 1153 4153 9153 106 107 BOAAAA UUCAAA AAAAxx +9220 1893 0 0 0 0 20 220 1220 4220 9220 40 41 QQAAAA VUCAAA HHHHxx +8003 1894 1 3 3 3 3 3 3 3003 8003 6 7 VVAAAA WUCAAA OOOOxx +7257 1895 1 1 7 17 57 257 1257 2257 7257 114 115 DTAAAA XUCAAA VVVVxx +3930 1896 0 2 0 10 30 930 1930 3930 3930 60 61 EVAAAA YUCAAA AAAAxx +2976 1897 0 0 6 16 76 976 976 2976 2976 152 153 MKAAAA ZUCAAA HHHHxx +2531 1898 1 3 1 11 31 531 531 2531 2531 62 63 JTAAAA AVCAAA OOOOxx +2250 1899 0 2 0 10 50 250 250 2250 2250 100 101 OIAAAA BVCAAA VVVVxx +8549 1900 1 1 9 9 49 549 549 3549 8549 98 99 VQAAAA CVCAAA AAAAxx +7197 1901 1 1 7 17 97 197 1197 2197 7197 194 195 VQAAAA DVCAAA HHHHxx +5916 1902 0 0 6 16 16 916 1916 916 5916 32 33 OTAAAA EVCAAA OOOOxx +5287 1903 1 3 7 7 87 287 1287 287 5287 174 175 JVAAAA FVCAAA VVVVxx +9095 1904 1 3 5 15 95 95 1095 4095 9095 190 191 VLAAAA GVCAAA AAAAxx +7137 1905 1 1 7 17 37 137 1137 2137 7137 74 75 NOAAAA HVCAAA HHHHxx +7902 1906 0 2 2 2 2 902 1902 2902 7902 4 5 YRAAAA IVCAAA OOOOxx +7598 1907 0 2 8 18 98 598 1598 2598 7598 196 197 GGAAAA JVCAAA VVVVxx +5652 1908 0 0 2 12 52 652 1652 652 5652 104 105 KJAAAA KVCAAA AAAAxx +2017 1909 1 1 7 17 17 17 17 2017 2017 34 35 PZAAAA LVCAAA HHHHxx +7255 1910 1 3 5 15 55 255 1255 2255 7255 110 111 BTAAAA MVCAAA OOOOxx +7999 1911 1 3 9 19 99 999 1999 2999 7999 198 199 RVAAAA NVCAAA VVVVxx +5388 1912 0 0 8 8 88 388 1388 388 5388 176 177 GZAAAA OVCAAA AAAAxx +8754 1913 0 2 4 14 54 754 754 3754 8754 108 109 SYAAAA PVCAAA HHHHxx +5415 1914 1 3 5 15 15 415 1415 415 5415 30 31 HAAAAA QVCAAA OOOOxx +8861 1915 1 1 1 1 61 861 861 3861 8861 122 123 VCAAAA RVCAAA VVVVxx +2874 1916 0 2 4 14 74 874 874 2874 2874 148 149 OGAAAA SVCAAA AAAAxx +9910 1917 0 2 0 10 10 910 1910 4910 9910 20 21 ERAAAA TVCAAA HHHHxx +5178 1918 0 2 8 18 78 178 1178 178 5178 156 157 ERAAAA UVCAAA OOOOxx +5698 1919 0 2 8 18 98 698 1698 698 5698 196 197 ELAAAA VVCAAA VVVVxx +8500 1920 0 0 0 0 0 500 500 3500 8500 0 1 YOAAAA WVCAAA AAAAxx +1814 1921 0 2 4 14 14 814 1814 1814 1814 28 29 URAAAA XVCAAA HHHHxx +4968 1922 0 0 8 8 68 968 968 4968 4968 136 137 CJAAAA YVCAAA OOOOxx +2642 1923 0 2 2 2 42 642 642 2642 2642 84 85 QXAAAA ZVCAAA VVVVxx +1578 1924 0 2 8 18 78 578 1578 1578 1578 156 157 SIAAAA AWCAAA AAAAxx +4774 1925 0 2 4 14 74 774 774 4774 4774 148 149 QBAAAA BWCAAA HHHHxx +7062 1926 0 2 2 2 62 62 1062 2062 7062 124 125 QLAAAA CWCAAA OOOOxx +5381 1927 1 1 1 1 81 381 1381 381 5381 162 163 ZYAAAA DWCAAA VVVVxx +7985 1928 1 1 5 5 85 985 1985 2985 7985 170 171 DVAAAA EWCAAA AAAAxx +3850 1929 0 2 0 10 50 850 1850 3850 3850 100 101 CSAAAA FWCAAA HHHHxx +5624 1930 0 0 4 4 24 624 1624 624 5624 48 49 IIAAAA GWCAAA OOOOxx +8948 1931 0 0 8 8 48 948 948 3948 8948 96 97 EGAAAA HWCAAA VVVVxx +995 1932 1 3 5 15 95 995 995 995 995 190 191 HMAAAA IWCAAA AAAAxx +5058 1933 0 2 8 18 58 58 1058 58 5058 116 117 OMAAAA JWCAAA HHHHxx +9670 1934 0 2 0 10 70 670 1670 4670 9670 140 141 YHAAAA KWCAAA OOOOxx +3115 1935 1 3 5 15 15 115 1115 3115 3115 30 31 VPAAAA LWCAAA VVVVxx +4935 1936 1 3 5 15 35 935 935 4935 4935 70 71 VHAAAA MWCAAA AAAAxx +4735 1937 1 3 5 15 35 735 735 4735 4735 70 71 DAAAAA NWCAAA HHHHxx +1348 1938 0 0 8 8 48 348 1348 1348 1348 96 97 WZAAAA OWCAAA OOOOxx +2380 1939 0 0 0 0 80 380 380 2380 2380 160 161 ONAAAA PWCAAA VVVVxx +4246 1940 0 2 6 6 46 246 246 4246 4246 92 93 IHAAAA QWCAAA AAAAxx +522 1941 0 2 2 2 22 522 522 522 522 44 45 CUAAAA RWCAAA HHHHxx +1701 1942 1 1 1 1 1 701 1701 1701 1701 2 3 LNAAAA SWCAAA OOOOxx +9709 1943 1 1 9 9 9 709 1709 4709 9709 18 19 LJAAAA TWCAAA VVVVxx +8829 1944 1 1 9 9 29 829 829 3829 8829 58 59 PBAAAA UWCAAA AAAAxx +7936 1945 0 0 6 16 36 936 1936 2936 7936 72 73 GTAAAA VWCAAA HHHHxx +8474 1946 0 2 4 14 74 474 474 3474 8474 148 149 YNAAAA WWCAAA OOOOxx +4676 1947 0 0 6 16 76 676 676 4676 4676 152 153 WXAAAA XWCAAA VVVVxx +6303 1948 1 3 3 3 3 303 303 1303 6303 6 7 LIAAAA YWCAAA AAAAxx +3485 1949 1 1 5 5 85 485 1485 3485 3485 170 171 BEAAAA ZWCAAA HHHHxx +2695 1950 1 3 5 15 95 695 695 2695 2695 190 191 RZAAAA AXCAAA OOOOxx +8830 1951 0 2 0 10 30 830 830 3830 8830 60 61 QBAAAA BXCAAA VVVVxx +898 1952 0 2 8 18 98 898 898 898 898 196 197 OIAAAA CXCAAA AAAAxx +7268 1953 0 0 8 8 68 268 1268 2268 7268 136 137 OTAAAA DXCAAA HHHHxx +6568 1954 0 0 8 8 68 568 568 1568 6568 136 137 QSAAAA EXCAAA OOOOxx +9724 1955 0 0 4 4 24 724 1724 4724 9724 48 49 AKAAAA FXCAAA VVVVxx +3329 1956 1 1 9 9 29 329 1329 3329 3329 58 59 BYAAAA GXCAAA AAAAxx +9860 1957 0 0 0 0 60 860 1860 4860 9860 120 121 GPAAAA HXCAAA HHHHxx +6833 1958 1 1 3 13 33 833 833 1833 6833 66 67 VCAAAA IXCAAA OOOOxx +5956 1959 0 0 6 16 56 956 1956 956 5956 112 113 CVAAAA JXCAAA VVVVxx +3963 1960 1 3 3 3 63 963 1963 3963 3963 126 127 LWAAAA KXCAAA AAAAxx +883 1961 1 3 3 3 83 883 883 883 883 166 167 ZHAAAA LXCAAA HHHHxx +2761 1962 1 1 1 1 61 761 761 2761 2761 122 123 FCAAAA MXCAAA OOOOxx +4644 1963 0 0 4 4 44 644 644 4644 4644 88 89 QWAAAA NXCAAA VVVVxx +1358 1964 0 2 8 18 58 358 1358 1358 1358 116 117 GAAAAA OXCAAA AAAAxx +2049 1965 1 1 9 9 49 49 49 2049 2049 98 99 VAAAAA PXCAAA HHHHxx +2193 1966 1 1 3 13 93 193 193 2193 2193 186 187 JGAAAA QXCAAA OOOOxx +9435 1967 1 3 5 15 35 435 1435 4435 9435 70 71 XYAAAA RXCAAA VVVVxx +5890 1968 0 2 0 10 90 890 1890 890 5890 180 181 OSAAAA SXCAAA AAAAxx +8149 1969 1 1 9 9 49 149 149 3149 8149 98 99 LBAAAA TXCAAA HHHHxx +423 1970 1 3 3 3 23 423 423 423 423 46 47 HQAAAA UXCAAA OOOOxx +7980 1971 0 0 0 0 80 980 1980 2980 7980 160 161 YUAAAA VXCAAA VVVVxx +9019 1972 1 3 9 19 19 19 1019 4019 9019 38 39 XIAAAA WXCAAA AAAAxx +1647 1973 1 3 7 7 47 647 1647 1647 1647 94 95 JLAAAA XXCAAA HHHHxx +9495 1974 1 3 5 15 95 495 1495 4495 9495 190 191 FBAAAA YXCAAA OOOOxx +3904 1975 0 0 4 4 4 904 1904 3904 3904 8 9 EUAAAA ZXCAAA VVVVxx +5838 1976 0 2 8 18 38 838 1838 838 5838 76 77 OQAAAA AYCAAA AAAAxx +3866 1977 0 2 6 6 66 866 1866 3866 3866 132 133 SSAAAA BYCAAA HHHHxx +3093 1978 1 1 3 13 93 93 1093 3093 3093 186 187 ZOAAAA CYCAAA OOOOxx +9666 1979 0 2 6 6 66 666 1666 4666 9666 132 133 UHAAAA DYCAAA VVVVxx +1246 1980 0 2 6 6 46 246 1246 1246 1246 92 93 YVAAAA EYCAAA AAAAxx +9759 1981 1 3 9 19 59 759 1759 4759 9759 118 119 JLAAAA FYCAAA HHHHxx +7174 1982 0 2 4 14 74 174 1174 2174 7174 148 149 YPAAAA GYCAAA OOOOxx +7678 1983 0 2 8 18 78 678 1678 2678 7678 156 157 IJAAAA HYCAAA VVVVxx +3004 1984 0 0 4 4 4 4 1004 3004 3004 8 9 OLAAAA IYCAAA AAAAxx +5607 1985 1 3 7 7 7 607 1607 607 5607 14 15 RHAAAA JYCAAA HHHHxx +8510 1986 0 2 0 10 10 510 510 3510 8510 20 21 IPAAAA KYCAAA OOOOxx +1483 1987 1 3 3 3 83 483 1483 1483 1483 166 167 BFAAAA LYCAAA VVVVxx +2915 1988 1 3 5 15 15 915 915 2915 2915 30 31 DIAAAA MYCAAA AAAAxx +1548 1989 0 0 8 8 48 548 1548 1548 1548 96 97 OHAAAA NYCAAA HHHHxx +5767 1990 1 3 7 7 67 767 1767 767 5767 134 135 VNAAAA OYCAAA OOOOxx +3214 1991 0 2 4 14 14 214 1214 3214 3214 28 29 QTAAAA PYCAAA VVVVxx +8663 1992 1 3 3 3 63 663 663 3663 8663 126 127 FVAAAA QYCAAA AAAAxx +5425 1993 1 1 5 5 25 425 1425 425 5425 50 51 RAAAAA RYCAAA HHHHxx +8530 1994 0 2 0 10 30 530 530 3530 8530 60 61 CQAAAA SYCAAA OOOOxx +821 1995 1 1 1 1 21 821 821 821 821 42 43 PFAAAA TYCAAA VVVVxx +8816 1996 0 0 6 16 16 816 816 3816 8816 32 33 CBAAAA UYCAAA AAAAxx +9367 1997 1 3 7 7 67 367 1367 4367 9367 134 135 HWAAAA VYCAAA HHHHxx +4138 1998 0 2 8 18 38 138 138 4138 4138 76 77 EDAAAA WYCAAA OOOOxx +94 1999 0 2 4 14 94 94 94 94 94 188 189 QDAAAA XYCAAA VVVVxx +1858 2000 0 2 8 18 58 858 1858 1858 1858 116 117 MTAAAA YYCAAA AAAAxx +5513 2001 1 1 3 13 13 513 1513 513 5513 26 27 BEAAAA ZYCAAA HHHHxx +9620 2002 0 0 0 0 20 620 1620 4620 9620 40 41 AGAAAA AZCAAA OOOOxx +4770 2003 0 2 0 10 70 770 770 4770 4770 140 141 MBAAAA BZCAAA VVVVxx +5193 2004 1 1 3 13 93 193 1193 193 5193 186 187 TRAAAA CZCAAA AAAAxx +198 2005 0 2 8 18 98 198 198 198 198 196 197 QHAAAA DZCAAA HHHHxx +417 2006 1 1 7 17 17 417 417 417 417 34 35 BQAAAA EZCAAA OOOOxx +173 2007 1 1 3 13 73 173 173 173 173 146 147 RGAAAA FZCAAA VVVVxx +6248 2008 0 0 8 8 48 248 248 1248 6248 96 97 IGAAAA GZCAAA AAAAxx +302 2009 0 2 2 2 2 302 302 302 302 4 5 QLAAAA HZCAAA HHHHxx +8983 2010 1 3 3 3 83 983 983 3983 8983 166 167 NHAAAA IZCAAA OOOOxx +4840 2011 0 0 0 0 40 840 840 4840 4840 80 81 EEAAAA JZCAAA VVVVxx +2876 2012 0 0 6 16 76 876 876 2876 2876 152 153 QGAAAA KZCAAA AAAAxx +5841 2013 1 1 1 1 41 841 1841 841 5841 82 83 RQAAAA LZCAAA HHHHxx +2766 2014 0 2 6 6 66 766 766 2766 2766 132 133 KCAAAA MZCAAA OOOOxx +9482 2015 0 2 2 2 82 482 1482 4482 9482 164 165 SAAAAA NZCAAA VVVVxx +5335 2016 1 3 5 15 35 335 1335 335 5335 70 71 FXAAAA OZCAAA AAAAxx +1502 2017 0 2 2 2 2 502 1502 1502 1502 4 5 UFAAAA PZCAAA HHHHxx +9291 2018 1 3 1 11 91 291 1291 4291 9291 182 183 JTAAAA QZCAAA OOOOxx +8655 2019 1 3 5 15 55 655 655 3655 8655 110 111 XUAAAA RZCAAA VVVVxx +1687 2020 1 3 7 7 87 687 1687 1687 1687 174 175 XMAAAA SZCAAA AAAAxx +8171 2021 1 3 1 11 71 171 171 3171 8171 142 143 HCAAAA TZCAAA HHHHxx +5699 2022 1 3 9 19 99 699 1699 699 5699 198 199 FLAAAA UZCAAA OOOOxx +1462 2023 0 2 2 2 62 462 1462 1462 1462 124 125 GEAAAA VZCAAA VVVVxx +608 2024 0 0 8 8 8 608 608 608 608 16 17 KXAAAA WZCAAA AAAAxx +6860 2025 0 0 0 0 60 860 860 1860 6860 120 121 WDAAAA XZCAAA HHHHxx +6063 2026 1 3 3 3 63 63 63 1063 6063 126 127 FZAAAA YZCAAA OOOOxx +1422 2027 0 2 2 2 22 422 1422 1422 1422 44 45 SCAAAA ZZCAAA VVVVxx +1932 2028 0 0 2 12 32 932 1932 1932 1932 64 65 IWAAAA AADAAA AAAAxx +5065 2029 1 1 5 5 65 65 1065 65 5065 130 131 VMAAAA BADAAA HHHHxx +432 2030 0 0 2 12 32 432 432 432 432 64 65 QQAAAA CADAAA OOOOxx +4680 2031 0 0 0 0 80 680 680 4680 4680 160 161 AYAAAA DADAAA VVVVxx +8172 2032 0 0 2 12 72 172 172 3172 8172 144 145 ICAAAA EADAAA AAAAxx +8668 2033 0 0 8 8 68 668 668 3668 8668 136 137 KVAAAA FADAAA HHHHxx +256 2034 0 0 6 16 56 256 256 256 256 112 113 WJAAAA GADAAA OOOOxx +2500 2035 0 0 0 0 0 500 500 2500 2500 0 1 ESAAAA HADAAA VVVVxx +274 2036 0 2 4 14 74 274 274 274 274 148 149 OKAAAA IADAAA AAAAxx +5907 2037 1 3 7 7 7 907 1907 907 5907 14 15 FTAAAA JADAAA HHHHxx +8587 2038 1 3 7 7 87 587 587 3587 8587 174 175 HSAAAA KADAAA OOOOxx +9942 2039 0 2 2 2 42 942 1942 4942 9942 84 85 KSAAAA LADAAA VVVVxx +116 2040 0 0 6 16 16 116 116 116 116 32 33 MEAAAA MADAAA AAAAxx +7134 2041 0 2 4 14 34 134 1134 2134 7134 68 69 KOAAAA NADAAA HHHHxx +9002 2042 0 2 2 2 2 2 1002 4002 9002 4 5 GIAAAA OADAAA OOOOxx +1209 2043 1 1 9 9 9 209 1209 1209 1209 18 19 NUAAAA PADAAA VVVVxx +9983 2044 1 3 3 3 83 983 1983 4983 9983 166 167 ZTAAAA QADAAA AAAAxx +1761 2045 1 1 1 1 61 761 1761 1761 1761 122 123 TPAAAA RADAAA HHHHxx +7723 2046 1 3 3 3 23 723 1723 2723 7723 46 47 BLAAAA SADAAA OOOOxx +6518 2047 0 2 8 18 18 518 518 1518 6518 36 37 SQAAAA TADAAA VVVVxx +1372 2048 0 0 2 12 72 372 1372 1372 1372 144 145 UAAAAA UADAAA AAAAxx +3587 2049 1 3 7 7 87 587 1587 3587 3587 174 175 ZHAAAA VADAAA HHHHxx +5323 2050 1 3 3 3 23 323 1323 323 5323 46 47 TWAAAA WADAAA OOOOxx +5902 2051 0 2 2 2 2 902 1902 902 5902 4 5 ATAAAA XADAAA VVVVxx +3749 2052 1 1 9 9 49 749 1749 3749 3749 98 99 FOAAAA YADAAA AAAAxx +5965 2053 1 1 5 5 65 965 1965 965 5965 130 131 LVAAAA ZADAAA HHHHxx +663 2054 1 3 3 3 63 663 663 663 663 126 127 NZAAAA ABDAAA OOOOxx +36 2055 0 0 6 16 36 36 36 36 36 72 73 KBAAAA BBDAAA VVVVxx +9782 2056 0 2 2 2 82 782 1782 4782 9782 164 165 GMAAAA CBDAAA AAAAxx +5412 2057 0 0 2 12 12 412 1412 412 5412 24 25 EAAAAA DBDAAA HHHHxx +9961 2058 1 1 1 1 61 961 1961 4961 9961 122 123 DTAAAA EBDAAA OOOOxx +6492 2059 0 0 2 12 92 492 492 1492 6492 184 185 SPAAAA FBDAAA VVVVxx +4234 2060 0 2 4 14 34 234 234 4234 4234 68 69 WGAAAA GBDAAA AAAAxx +4922 2061 0 2 2 2 22 922 922 4922 4922 44 45 IHAAAA HBDAAA HHHHxx +6166 2062 0 2 6 6 66 166 166 1166 6166 132 133 EDAAAA IBDAAA OOOOxx +7019 2063 1 3 9 19 19 19 1019 2019 7019 38 39 ZJAAAA JBDAAA VVVVxx +7805 2064 1 1 5 5 5 805 1805 2805 7805 10 11 FOAAAA KBDAAA AAAAxx +9808 2065 0 0 8 8 8 808 1808 4808 9808 16 17 GNAAAA LBDAAA HHHHxx +2550 2066 0 2 0 10 50 550 550 2550 2550 100 101 CUAAAA MBDAAA OOOOxx +8626 2067 0 2 6 6 26 626 626 3626 8626 52 53 UTAAAA NBDAAA VVVVxx +5649 2068 1 1 9 9 49 649 1649 649 5649 98 99 HJAAAA OBDAAA AAAAxx +3117 2069 1 1 7 17 17 117 1117 3117 3117 34 35 XPAAAA PBDAAA HHHHxx +866 2070 0 2 6 6 66 866 866 866 866 132 133 IHAAAA QBDAAA OOOOxx +2323 2071 1 3 3 3 23 323 323 2323 2323 46 47 JLAAAA RBDAAA VVVVxx +5132 2072 0 0 2 12 32 132 1132 132 5132 64 65 KPAAAA SBDAAA AAAAxx +9222 2073 0 2 2 2 22 222 1222 4222 9222 44 45 SQAAAA TBDAAA HHHHxx +3934 2074 0 2 4 14 34 934 1934 3934 3934 68 69 IVAAAA UBDAAA OOOOxx +4845 2075 1 1 5 5 45 845 845 4845 4845 90 91 JEAAAA VBDAAA VVVVxx +7714 2076 0 2 4 14 14 714 1714 2714 7714 28 29 SKAAAA WBDAAA AAAAxx +9818 2077 0 2 8 18 18 818 1818 4818 9818 36 37 QNAAAA XBDAAA HHHHxx +2219 2078 1 3 9 19 19 219 219 2219 2219 38 39 JHAAAA YBDAAA OOOOxx +6573 2079 1 1 3 13 73 573 573 1573 6573 146 147 VSAAAA ZBDAAA VVVVxx +4555 2080 1 3 5 15 55 555 555 4555 4555 110 111 FTAAAA ACDAAA AAAAxx +7306 2081 0 2 6 6 6 306 1306 2306 7306 12 13 AVAAAA BCDAAA HHHHxx +9313 2082 1 1 3 13 13 313 1313 4313 9313 26 27 FUAAAA CCDAAA OOOOxx +3924 2083 0 0 4 4 24 924 1924 3924 3924 48 49 YUAAAA DCDAAA VVVVxx +5176 2084 0 0 6 16 76 176 1176 176 5176 152 153 CRAAAA ECDAAA AAAAxx +9767 2085 1 3 7 7 67 767 1767 4767 9767 134 135 RLAAAA FCDAAA HHHHxx +905 2086 1 1 5 5 5 905 905 905 905 10 11 VIAAAA GCDAAA OOOOxx +8037 2087 1 1 7 17 37 37 37 3037 8037 74 75 DXAAAA HCDAAA VVVVxx +8133 2088 1 1 3 13 33 133 133 3133 8133 66 67 VAAAAA ICDAAA AAAAxx +2954 2089 0 2 4 14 54 954 954 2954 2954 108 109 QJAAAA JCDAAA HHHHxx +7262 2090 0 2 2 2 62 262 1262 2262 7262 124 125 ITAAAA KCDAAA OOOOxx +8768 2091 0 0 8 8 68 768 768 3768 8768 136 137 GZAAAA LCDAAA VVVVxx +6953 2092 1 1 3 13 53 953 953 1953 6953 106 107 LHAAAA MCDAAA AAAAxx +1984 2093 0 0 4 4 84 984 1984 1984 1984 168 169 IYAAAA NCDAAA HHHHxx +9348 2094 0 0 8 8 48 348 1348 4348 9348 96 97 OVAAAA OCDAAA OOOOxx +7769 2095 1 1 9 9 69 769 1769 2769 7769 138 139 VMAAAA PCDAAA VVVVxx +2994 2096 0 2 4 14 94 994 994 2994 2994 188 189 ELAAAA QCDAAA AAAAxx +5938 2097 0 2 8 18 38 938 1938 938 5938 76 77 KUAAAA RCDAAA HHHHxx +556 2098 0 0 6 16 56 556 556 556 556 112 113 KVAAAA SCDAAA OOOOxx +2577 2099 1 1 7 17 77 577 577 2577 2577 154 155 DVAAAA TCDAAA VVVVxx +8733 2100 1 1 3 13 33 733 733 3733 8733 66 67 XXAAAA UCDAAA AAAAxx +3108 2101 0 0 8 8 8 108 1108 3108 3108 16 17 OPAAAA VCDAAA HHHHxx +4166 2102 0 2 6 6 66 166 166 4166 4166 132 133 GEAAAA WCDAAA OOOOxx +3170 2103 0 2 0 10 70 170 1170 3170 3170 140 141 YRAAAA XCDAAA VVVVxx +8118 2104 0 2 8 18 18 118 118 3118 8118 36 37 GAAAAA YCDAAA AAAAxx +8454 2105 0 2 4 14 54 454 454 3454 8454 108 109 ENAAAA ZCDAAA HHHHxx +5338 2106 0 2 8 18 38 338 1338 338 5338 76 77 IXAAAA ADDAAA OOOOxx +402 2107 0 2 2 2 2 402 402 402 402 4 5 MPAAAA BDDAAA VVVVxx +5673 2108 1 1 3 13 73 673 1673 673 5673 146 147 FKAAAA CDDAAA AAAAxx +4324 2109 0 0 4 4 24 324 324 4324 4324 48 49 IKAAAA DDDAAA HHHHxx +1943 2110 1 3 3 3 43 943 1943 1943 1943 86 87 TWAAAA EDDAAA OOOOxx +7703 2111 1 3 3 3 3 703 1703 2703 7703 6 7 HKAAAA FDDAAA VVVVxx +7180 2112 0 0 0 0 80 180 1180 2180 7180 160 161 EQAAAA GDDAAA AAAAxx +5478 2113 0 2 8 18 78 478 1478 478 5478 156 157 SCAAAA HDDAAA HHHHxx +5775 2114 1 3 5 15 75 775 1775 775 5775 150 151 DOAAAA IDDAAA OOOOxx +6952 2115 0 0 2 12 52 952 952 1952 6952 104 105 KHAAAA JDDAAA VVVVxx +9022 2116 0 2 2 2 22 22 1022 4022 9022 44 45 AJAAAA KDDAAA AAAAxx +547 2117 1 3 7 7 47 547 547 547 547 94 95 BVAAAA LDDAAA HHHHxx +5877 2118 1 1 7 17 77 877 1877 877 5877 154 155 BSAAAA MDDAAA OOOOxx +9580 2119 0 0 0 0 80 580 1580 4580 9580 160 161 MEAAAA NDDAAA VVVVxx +6094 2120 0 2 4 14 94 94 94 1094 6094 188 189 KAAAAA ODDAAA AAAAxx +3398 2121 0 2 8 18 98 398 1398 3398 3398 196 197 SAAAAA PDDAAA HHHHxx +4574 2122 0 2 4 14 74 574 574 4574 4574 148 149 YTAAAA QDDAAA OOOOxx +3675 2123 1 3 5 15 75 675 1675 3675 3675 150 151 JLAAAA RDDAAA VVVVxx +6413 2124 1 1 3 13 13 413 413 1413 6413 26 27 RMAAAA SDDAAA AAAAxx +9851 2125 1 3 1 11 51 851 1851 4851 9851 102 103 XOAAAA TDDAAA HHHHxx +126 2126 0 2 6 6 26 126 126 126 126 52 53 WEAAAA UDDAAA OOOOxx +6803 2127 1 3 3 3 3 803 803 1803 6803 6 7 RBAAAA VDDAAA VVVVxx +6949 2128 1 1 9 9 49 949 949 1949 6949 98 99 HHAAAA WDDAAA AAAAxx +115 2129 1 3 5 15 15 115 115 115 115 30 31 LEAAAA XDDAAA HHHHxx +4165 2130 1 1 5 5 65 165 165 4165 4165 130 131 FEAAAA YDDAAA OOOOxx +201 2131 1 1 1 1 1 201 201 201 201 2 3 THAAAA ZDDAAA VVVVxx +9324 2132 0 0 4 4 24 324 1324 4324 9324 48 49 QUAAAA AEDAAA AAAAxx +6562 2133 0 2 2 2 62 562 562 1562 6562 124 125 KSAAAA BEDAAA HHHHxx +1917 2134 1 1 7 17 17 917 1917 1917 1917 34 35 TVAAAA CEDAAA OOOOxx +558 2135 0 2 8 18 58 558 558 558 558 116 117 MVAAAA DEDAAA VVVVxx +8515 2136 1 3 5 15 15 515 515 3515 8515 30 31 NPAAAA EEDAAA AAAAxx +6321 2137 1 1 1 1 21 321 321 1321 6321 42 43 DJAAAA FEDAAA HHHHxx +6892 2138 0 0 2 12 92 892 892 1892 6892 184 185 CFAAAA GEDAAA OOOOxx +1001 2139 1 1 1 1 1 1 1001 1001 1001 2 3 NMAAAA HEDAAA VVVVxx +2858 2140 0 2 8 18 58 858 858 2858 2858 116 117 YFAAAA IEDAAA AAAAxx +2434 2141 0 2 4 14 34 434 434 2434 2434 68 69 QPAAAA JEDAAA HHHHxx +4460 2142 0 0 0 0 60 460 460 4460 4460 120 121 OPAAAA KEDAAA OOOOxx +5447 2143 1 3 7 7 47 447 1447 447 5447 94 95 NBAAAA LEDAAA VVVVxx +3799 2144 1 3 9 19 99 799 1799 3799 3799 198 199 DQAAAA MEDAAA AAAAxx +4310 2145 0 2 0 10 10 310 310 4310 4310 20 21 UJAAAA NEDAAA HHHHxx +405 2146 1 1 5 5 5 405 405 405 405 10 11 PPAAAA OEDAAA OOOOxx +4573 2147 1 1 3 13 73 573 573 4573 4573 146 147 XTAAAA PEDAAA VVVVxx +706 2148 0 2 6 6 6 706 706 706 706 12 13 EBAAAA QEDAAA AAAAxx +7619 2149 1 3 9 19 19 619 1619 2619 7619 38 39 BHAAAA REDAAA HHHHxx +7959 2150 1 3 9 19 59 959 1959 2959 7959 118 119 DUAAAA SEDAAA OOOOxx +6712 2151 0 0 2 12 12 712 712 1712 6712 24 25 EYAAAA TEDAAA VVVVxx +6959 2152 1 3 9 19 59 959 959 1959 6959 118 119 RHAAAA UEDAAA AAAAxx +9791 2153 1 3 1 11 91 791 1791 4791 9791 182 183 PMAAAA VEDAAA HHHHxx +2112 2154 0 0 2 12 12 112 112 2112 2112 24 25 GDAAAA WEDAAA OOOOxx +9114 2155 0 2 4 14 14 114 1114 4114 9114 28 29 OMAAAA XEDAAA VVVVxx +3506 2156 0 2 6 6 6 506 1506 3506 3506 12 13 WEAAAA YEDAAA AAAAxx +5002 2157 0 2 2 2 2 2 1002 2 5002 4 5 KKAAAA ZEDAAA HHHHxx +3518 2158 0 2 8 18 18 518 1518 3518 3518 36 37 IFAAAA AFDAAA OOOOxx +602 2159 0 2 2 2 2 602 602 602 602 4 5 EXAAAA BFDAAA VVVVxx +9060 2160 0 0 0 0 60 60 1060 4060 9060 120 121 MKAAAA CFDAAA AAAAxx +3292 2161 0 0 2 12 92 292 1292 3292 3292 184 185 QWAAAA DFDAAA HHHHxx +77 2162 1 1 7 17 77 77 77 77 77 154 155 ZCAAAA EFDAAA OOOOxx +1420 2163 0 0 0 0 20 420 1420 1420 1420 40 41 QCAAAA FFDAAA VVVVxx +6001 2164 1 1 1 1 1 1 1 1001 6001 2 3 VWAAAA GFDAAA AAAAxx +7477 2165 1 1 7 17 77 477 1477 2477 7477 154 155 PBAAAA HFDAAA HHHHxx +6655 2166 1 3 5 15 55 655 655 1655 6655 110 111 ZVAAAA IFDAAA OOOOxx +7845 2167 1 1 5 5 45 845 1845 2845 7845 90 91 TPAAAA JFDAAA VVVVxx +8484 2168 0 0 4 4 84 484 484 3484 8484 168 169 IOAAAA KFDAAA AAAAxx +4345 2169 1 1 5 5 45 345 345 4345 4345 90 91 DLAAAA LFDAAA HHHHxx +4250 2170 0 2 0 10 50 250 250 4250 4250 100 101 MHAAAA MFDAAA OOOOxx +2391 2171 1 3 1 11 91 391 391 2391 2391 182 183 ZNAAAA NFDAAA VVVVxx +6884 2172 0 0 4 4 84 884 884 1884 6884 168 169 UEAAAA OFDAAA AAAAxx +7270 2173 0 2 0 10 70 270 1270 2270 7270 140 141 QTAAAA PFDAAA HHHHxx +2499 2174 1 3 9 19 99 499 499 2499 2499 198 199 DSAAAA QFDAAA OOOOxx +7312 2175 0 0 2 12 12 312 1312 2312 7312 24 25 GVAAAA RFDAAA VVVVxx +7113 2176 1 1 3 13 13 113 1113 2113 7113 26 27 PNAAAA SFDAAA AAAAxx +6695 2177 1 3 5 15 95 695 695 1695 6695 190 191 NXAAAA TFDAAA HHHHxx +6521 2178 1 1 1 1 21 521 521 1521 6521 42 43 VQAAAA UFDAAA OOOOxx +272 2179 0 0 2 12 72 272 272 272 272 144 145 MKAAAA VFDAAA VVVVxx +9976 2180 0 0 6 16 76 976 1976 4976 9976 152 153 STAAAA WFDAAA AAAAxx +992 2181 0 0 2 12 92 992 992 992 992 184 185 EMAAAA XFDAAA HHHHxx +6158 2182 0 2 8 18 58 158 158 1158 6158 116 117 WCAAAA YFDAAA OOOOxx +3281 2183 1 1 1 1 81 281 1281 3281 3281 162 163 FWAAAA ZFDAAA VVVVxx +7446 2184 0 2 6 6 46 446 1446 2446 7446 92 93 KAAAAA AGDAAA AAAAxx +4679 2185 1 3 9 19 79 679 679 4679 4679 158 159 ZXAAAA BGDAAA HHHHxx +5203 2186 1 3 3 3 3 203 1203 203 5203 6 7 DSAAAA CGDAAA OOOOxx +9874 2187 0 2 4 14 74 874 1874 4874 9874 148 149 UPAAAA DGDAAA VVVVxx +8371 2188 1 3 1 11 71 371 371 3371 8371 142 143 ZJAAAA EGDAAA AAAAxx +9086 2189 0 2 6 6 86 86 1086 4086 9086 172 173 MLAAAA FGDAAA HHHHxx +430 2190 0 2 0 10 30 430 430 430 430 60 61 OQAAAA GGDAAA OOOOxx +8749 2191 1 1 9 9 49 749 749 3749 8749 98 99 NYAAAA HGDAAA VVVVxx +577 2192 1 1 7 17 77 577 577 577 577 154 155 FWAAAA IGDAAA AAAAxx +4884 2193 0 0 4 4 84 884 884 4884 4884 168 169 WFAAAA JGDAAA HHHHxx +3421 2194 1 1 1 1 21 421 1421 3421 3421 42 43 PBAAAA KGDAAA OOOOxx +2812 2195 0 0 2 12 12 812 812 2812 2812 24 25 EEAAAA LGDAAA VVVVxx +5958 2196 0 2 8 18 58 958 1958 958 5958 116 117 EVAAAA MGDAAA AAAAxx +9901 2197 1 1 1 1 1 901 1901 4901 9901 2 3 VQAAAA NGDAAA HHHHxx +8478 2198 0 2 8 18 78 478 478 3478 8478 156 157 COAAAA OGDAAA OOOOxx +6545 2199 1 1 5 5 45 545 545 1545 6545 90 91 TRAAAA PGDAAA VVVVxx +1479 2200 1 3 9 19 79 479 1479 1479 1479 158 159 XEAAAA QGDAAA AAAAxx +1046 2201 0 2 6 6 46 46 1046 1046 1046 92 93 GOAAAA RGDAAA HHHHxx +6372 2202 0 0 2 12 72 372 372 1372 6372 144 145 CLAAAA SGDAAA OOOOxx +8206 2203 0 2 6 6 6 206 206 3206 8206 12 13 QDAAAA TGDAAA VVVVxx +9544 2204 0 0 4 4 44 544 1544 4544 9544 88 89 CDAAAA UGDAAA AAAAxx +9287 2205 1 3 7 7 87 287 1287 4287 9287 174 175 FTAAAA VGDAAA HHHHxx +6786 2206 0 2 6 6 86 786 786 1786 6786 172 173 ABAAAA WGDAAA OOOOxx +6511 2207 1 3 1 11 11 511 511 1511 6511 22 23 LQAAAA XGDAAA VVVVxx +603 2208 1 3 3 3 3 603 603 603 603 6 7 FXAAAA YGDAAA AAAAxx +2022 2209 0 2 2 2 22 22 22 2022 2022 44 45 UZAAAA ZGDAAA HHHHxx +2086 2210 0 2 6 6 86 86 86 2086 2086 172 173 GCAAAA AHDAAA OOOOxx +1969 2211 1 1 9 9 69 969 1969 1969 1969 138 139 TXAAAA BHDAAA VVVVxx +4841 2212 1 1 1 1 41 841 841 4841 4841 82 83 FEAAAA CHDAAA AAAAxx +5845 2213 1 1 5 5 45 845 1845 845 5845 90 91 VQAAAA DHDAAA HHHHxx +4635 2214 1 3 5 15 35 635 635 4635 4635 70 71 HWAAAA EHDAAA OOOOxx +4658 2215 0 2 8 18 58 658 658 4658 4658 116 117 EXAAAA FHDAAA VVVVxx +2896 2216 0 0 6 16 96 896 896 2896 2896 192 193 KHAAAA GHDAAA AAAAxx +5179 2217 1 3 9 19 79 179 1179 179 5179 158 159 FRAAAA HHDAAA HHHHxx +8667 2218 1 3 7 7 67 667 667 3667 8667 134 135 JVAAAA IHDAAA OOOOxx +7294 2219 0 2 4 14 94 294 1294 2294 7294 188 189 OUAAAA JHDAAA VVVVxx +3706 2220 0 2 6 6 6 706 1706 3706 3706 12 13 OMAAAA KHDAAA AAAAxx +8389 2221 1 1 9 9 89 389 389 3389 8389 178 179 RKAAAA LHDAAA HHHHxx +2486 2222 0 2 6 6 86 486 486 2486 2486 172 173 QRAAAA MHDAAA OOOOxx +8743 2223 1 3 3 3 43 743 743 3743 8743 86 87 HYAAAA NHDAAA VVVVxx +2777 2224 1 1 7 17 77 777 777 2777 2777 154 155 VCAAAA OHDAAA AAAAxx +2113 2225 1 1 3 13 13 113 113 2113 2113 26 27 HDAAAA PHDAAA HHHHxx +2076 2226 0 0 6 16 76 76 76 2076 2076 152 153 WBAAAA QHDAAA OOOOxx +2300 2227 0 0 0 0 0 300 300 2300 2300 0 1 MKAAAA RHDAAA VVVVxx +6894 2228 0 2 4 14 94 894 894 1894 6894 188 189 EFAAAA SHDAAA AAAAxx +6939 2229 1 3 9 19 39 939 939 1939 6939 78 79 XGAAAA THDAAA HHHHxx +446 2230 0 2 6 6 46 446 446 446 446 92 93 ERAAAA UHDAAA OOOOxx +6218 2231 0 2 8 18 18 218 218 1218 6218 36 37 EFAAAA VHDAAA VVVVxx +1295 2232 1 3 5 15 95 295 1295 1295 1295 190 191 VXAAAA WHDAAA AAAAxx +5135 2233 1 3 5 15 35 135 1135 135 5135 70 71 NPAAAA XHDAAA HHHHxx +8122 2234 0 2 2 2 22 122 122 3122 8122 44 45 KAAAAA YHDAAA OOOOxx +316 2235 0 0 6 16 16 316 316 316 316 32 33 EMAAAA ZHDAAA VVVVxx +514 2236 0 2 4 14 14 514 514 514 514 28 29 UTAAAA AIDAAA AAAAxx +7970 2237 0 2 0 10 70 970 1970 2970 7970 140 141 OUAAAA BIDAAA HHHHxx +9350 2238 0 2 0 10 50 350 1350 4350 9350 100 101 QVAAAA CIDAAA OOOOxx +3700 2239 0 0 0 0 0 700 1700 3700 3700 0 1 IMAAAA DIDAAA VVVVxx +582 2240 0 2 2 2 82 582 582 582 582 164 165 KWAAAA EIDAAA AAAAxx +9722 2241 0 2 2 2 22 722 1722 4722 9722 44 45 YJAAAA FIDAAA HHHHxx +7398 2242 0 2 8 18 98 398 1398 2398 7398 196 197 OYAAAA GIDAAA OOOOxx +2265 2243 1 1 5 5 65 265 265 2265 2265 130 131 DJAAAA HIDAAA VVVVxx +3049 2244 1 1 9 9 49 49 1049 3049 3049 98 99 HNAAAA IIDAAA AAAAxx +9121 2245 1 1 1 1 21 121 1121 4121 9121 42 43 VMAAAA JIDAAA HHHHxx +4275 2246 1 3 5 15 75 275 275 4275 4275 150 151 LIAAAA KIDAAA OOOOxx +6567 2247 1 3 7 7 67 567 567 1567 6567 134 135 PSAAAA LIDAAA VVVVxx +6755 2248 1 3 5 15 55 755 755 1755 6755 110 111 VZAAAA MIDAAA AAAAxx +4535 2249 1 3 5 15 35 535 535 4535 4535 70 71 LSAAAA NIDAAA HHHHxx +7968 2250 0 0 8 8 68 968 1968 2968 7968 136 137 MUAAAA OIDAAA OOOOxx +3412 2251 0 0 2 12 12 412 1412 3412 3412 24 25 GBAAAA PIDAAA VVVVxx +6112 2252 0 0 2 12 12 112 112 1112 6112 24 25 CBAAAA QIDAAA AAAAxx +6805 2253 1 1 5 5 5 805 805 1805 6805 10 11 TBAAAA RIDAAA HHHHxx +2880 2254 0 0 0 0 80 880 880 2880 2880 160 161 UGAAAA SIDAAA OOOOxx +7710 2255 0 2 0 10 10 710 1710 2710 7710 20 21 OKAAAA TIDAAA VVVVxx +7949 2256 1 1 9 9 49 949 1949 2949 7949 98 99 TTAAAA UIDAAA AAAAxx +7043 2257 1 3 3 3 43 43 1043 2043 7043 86 87 XKAAAA VIDAAA HHHHxx +9012 2258 0 0 2 12 12 12 1012 4012 9012 24 25 QIAAAA WIDAAA OOOOxx +878 2259 0 2 8 18 78 878 878 878 878 156 157 UHAAAA XIDAAA VVVVxx +7930 2260 0 2 0 10 30 930 1930 2930 7930 60 61 ATAAAA YIDAAA AAAAxx +667 2261 1 3 7 7 67 667 667 667 667 134 135 RZAAAA ZIDAAA HHHHxx +1905 2262 1 1 5 5 5 905 1905 1905 1905 10 11 HVAAAA AJDAAA OOOOxx +4958 2263 0 2 8 18 58 958 958 4958 4958 116 117 SIAAAA BJDAAA VVVVxx +2973 2264 1 1 3 13 73 973 973 2973 2973 146 147 JKAAAA CJDAAA AAAAxx +3631 2265 1 3 1 11 31 631 1631 3631 3631 62 63 RJAAAA DJDAAA HHHHxx +5868 2266 0 0 8 8 68 868 1868 868 5868 136 137 SRAAAA EJDAAA OOOOxx +2873 2267 1 1 3 13 73 873 873 2873 2873 146 147 NGAAAA FJDAAA VVVVxx +6941 2268 1 1 1 1 41 941 941 1941 6941 82 83 ZGAAAA GJDAAA AAAAxx +6384 2269 0 0 4 4 84 384 384 1384 6384 168 169 OLAAAA HJDAAA HHHHxx +3806 2270 0 2 6 6 6 806 1806 3806 3806 12 13 KQAAAA IJDAAA OOOOxx +5079 2271 1 3 9 19 79 79 1079 79 5079 158 159 JNAAAA JJDAAA VVVVxx +1970 2272 0 2 0 10 70 970 1970 1970 1970 140 141 UXAAAA KJDAAA AAAAxx +7810 2273 0 2 0 10 10 810 1810 2810 7810 20 21 KOAAAA LJDAAA HHHHxx +4639 2274 1 3 9 19 39 639 639 4639 4639 78 79 LWAAAA MJDAAA OOOOxx +6527 2275 1 3 7 7 27 527 527 1527 6527 54 55 BRAAAA NJDAAA VVVVxx +8079 2276 1 3 9 19 79 79 79 3079 8079 158 159 TYAAAA OJDAAA AAAAxx +2740 2277 0 0 0 0 40 740 740 2740 2740 80 81 KBAAAA PJDAAA HHHHxx +2337 2278 1 1 7 17 37 337 337 2337 2337 74 75 XLAAAA QJDAAA OOOOxx +6670 2279 0 2 0 10 70 670 670 1670 6670 140 141 OWAAAA RJDAAA VVVVxx +2345 2280 1 1 5 5 45 345 345 2345 2345 90 91 FMAAAA SJDAAA AAAAxx +401 2281 1 1 1 1 1 401 401 401 401 2 3 LPAAAA TJDAAA HHHHxx +2704 2282 0 0 4 4 4 704 704 2704 2704 8 9 AAAAAA UJDAAA OOOOxx +5530 2283 0 2 0 10 30 530 1530 530 5530 60 61 SEAAAA VJDAAA VVVVxx +51 2284 1 3 1 11 51 51 51 51 51 102 103 ZBAAAA WJDAAA AAAAxx +4282 2285 0 2 2 2 82 282 282 4282 4282 164 165 SIAAAA XJDAAA HHHHxx +7336 2286 0 0 6 16 36 336 1336 2336 7336 72 73 EWAAAA YJDAAA OOOOxx +8320 2287 0 0 0 0 20 320 320 3320 8320 40 41 AIAAAA ZJDAAA VVVVxx +7772 2288 0 0 2 12 72 772 1772 2772 7772 144 145 YMAAAA AKDAAA AAAAxx +1894 2289 0 2 4 14 94 894 1894 1894 1894 188 189 WUAAAA BKDAAA HHHHxx +2320 2290 0 0 0 0 20 320 320 2320 2320 40 41 GLAAAA CKDAAA OOOOxx +6232 2291 0 0 2 12 32 232 232 1232 6232 64 65 SFAAAA DKDAAA VVVVxx +2833 2292 1 1 3 13 33 833 833 2833 2833 66 67 ZEAAAA EKDAAA AAAAxx +8265 2293 1 1 5 5 65 265 265 3265 8265 130 131 XFAAAA FKDAAA HHHHxx +4589 2294 1 1 9 9 89 589 589 4589 4589 178 179 NUAAAA GKDAAA OOOOxx +8182 2295 0 2 2 2 82 182 182 3182 8182 164 165 SCAAAA HKDAAA VVVVxx +8337 2296 1 1 7 17 37 337 337 3337 8337 74 75 RIAAAA IKDAAA AAAAxx +8210 2297 0 2 0 10 10 210 210 3210 8210 20 21 UDAAAA JKDAAA HHHHxx +1406 2298 0 2 6 6 6 406 1406 1406 1406 12 13 CCAAAA KKDAAA OOOOxx +4463 2299 1 3 3 3 63 463 463 4463 4463 126 127 RPAAAA LKDAAA VVVVxx +4347 2300 1 3 7 7 47 347 347 4347 4347 94 95 FLAAAA MKDAAA AAAAxx +181 2301 1 1 1 1 81 181 181 181 181 162 163 ZGAAAA NKDAAA HHHHxx +9986 2302 0 2 6 6 86 986 1986 4986 9986 172 173 CUAAAA OKDAAA OOOOxx +661 2303 1 1 1 1 61 661 661 661 661 122 123 LZAAAA PKDAAA VVVVxx +4105 2304 1 1 5 5 5 105 105 4105 4105 10 11 XBAAAA QKDAAA AAAAxx +2187 2305 1 3 7 7 87 187 187 2187 2187 174 175 DGAAAA RKDAAA HHHHxx +1628 2306 0 0 8 8 28 628 1628 1628 1628 56 57 QKAAAA SKDAAA OOOOxx +3119 2307 1 3 9 19 19 119 1119 3119 3119 38 39 ZPAAAA TKDAAA VVVVxx +6804 2308 0 0 4 4 4 804 804 1804 6804 8 9 SBAAAA UKDAAA AAAAxx +9918 2309 0 2 8 18 18 918 1918 4918 9918 36 37 MRAAAA VKDAAA HHHHxx +8916 2310 0 0 6 16 16 916 916 3916 8916 32 33 YEAAAA WKDAAA OOOOxx +6057 2311 1 1 7 17 57 57 57 1057 6057 114 115 ZYAAAA XKDAAA VVVVxx +3622 2312 0 2 2 2 22 622 1622 3622 3622 44 45 IJAAAA YKDAAA AAAAxx +9168 2313 0 0 8 8 68 168 1168 4168 9168 136 137 QOAAAA ZKDAAA HHHHxx +3720 2314 0 0 0 0 20 720 1720 3720 3720 40 41 CNAAAA ALDAAA OOOOxx +9927 2315 1 3 7 7 27 927 1927 4927 9927 54 55 VRAAAA BLDAAA VVVVxx +5616 2316 0 0 6 16 16 616 1616 616 5616 32 33 AIAAAA CLDAAA AAAAxx +5210 2317 0 2 0 10 10 210 1210 210 5210 20 21 KSAAAA DLDAAA HHHHxx +636 2318 0 0 6 16 36 636 636 636 636 72 73 MYAAAA ELDAAA OOOOxx +9936 2319 0 0 6 16 36 936 1936 4936 9936 72 73 ESAAAA FLDAAA VVVVxx +2316 2320 0 0 6 16 16 316 316 2316 2316 32 33 CLAAAA GLDAAA AAAAxx +4363 2321 1 3 3 3 63 363 363 4363 4363 126 127 VLAAAA HLDAAA HHHHxx +7657 2322 1 1 7 17 57 657 1657 2657 7657 114 115 NIAAAA ILDAAA OOOOxx +697 2323 1 1 7 17 97 697 697 697 697 194 195 VAAAAA JLDAAA VVVVxx +912 2324 0 0 2 12 12 912 912 912 912 24 25 CJAAAA KLDAAA AAAAxx +8806 2325 0 2 6 6 6 806 806 3806 8806 12 13 SAAAAA LLDAAA HHHHxx +9698 2326 0 2 8 18 98 698 1698 4698 9698 196 197 AJAAAA MLDAAA OOOOxx +6191 2327 1 3 1 11 91 191 191 1191 6191 182 183 DEAAAA NLDAAA VVVVxx +1188 2328 0 0 8 8 88 188 1188 1188 1188 176 177 STAAAA OLDAAA AAAAxx +7676 2329 0 0 6 16 76 676 1676 2676 7676 152 153 GJAAAA PLDAAA HHHHxx +7073 2330 1 1 3 13 73 73 1073 2073 7073 146 147 BMAAAA QLDAAA OOOOxx +8019 2331 1 3 9 19 19 19 19 3019 8019 38 39 LWAAAA RLDAAA VVVVxx +4726 2332 0 2 6 6 26 726 726 4726 4726 52 53 UZAAAA SLDAAA AAAAxx +4648 2333 0 0 8 8 48 648 648 4648 4648 96 97 UWAAAA TLDAAA HHHHxx +3227 2334 1 3 7 7 27 227 1227 3227 3227 54 55 DUAAAA ULDAAA OOOOxx +7232 2335 0 0 2 12 32 232 1232 2232 7232 64 65 ESAAAA VLDAAA VVVVxx +9761 2336 1 1 1 1 61 761 1761 4761 9761 122 123 LLAAAA WLDAAA AAAAxx +3105 2337 1 1 5 5 5 105 1105 3105 3105 10 11 LPAAAA XLDAAA HHHHxx +5266 2338 0 2 6 6 66 266 1266 266 5266 132 133 OUAAAA YLDAAA OOOOxx +6788 2339 0 0 8 8 88 788 788 1788 6788 176 177 CBAAAA ZLDAAA VVVVxx +2442 2340 0 2 2 2 42 442 442 2442 2442 84 85 YPAAAA AMDAAA AAAAxx +8198 2341 0 2 8 18 98 198 198 3198 8198 196 197 IDAAAA BMDAAA HHHHxx +5806 2342 0 2 6 6 6 806 1806 806 5806 12 13 IPAAAA CMDAAA OOOOxx +8928 2343 0 0 8 8 28 928 928 3928 8928 56 57 KFAAAA DMDAAA VVVVxx +1657 2344 1 1 7 17 57 657 1657 1657 1657 114 115 TLAAAA EMDAAA AAAAxx +9164 2345 0 0 4 4 64 164 1164 4164 9164 128 129 MOAAAA FMDAAA HHHHxx +1851 2346 1 3 1 11 51 851 1851 1851 1851 102 103 FTAAAA GMDAAA OOOOxx +4744 2347 0 0 4 4 44 744 744 4744 4744 88 89 MAAAAA HMDAAA VVVVxx +8055 2348 1 3 5 15 55 55 55 3055 8055 110 111 VXAAAA IMDAAA AAAAxx +1533 2349 1 1 3 13 33 533 1533 1533 1533 66 67 ZGAAAA JMDAAA HHHHxx +1260 2350 0 0 0 0 60 260 1260 1260 1260 120 121 MWAAAA KMDAAA OOOOxx +1290 2351 0 2 0 10 90 290 1290 1290 1290 180 181 QXAAAA LMDAAA VVVVxx +297 2352 1 1 7 17 97 297 297 297 297 194 195 LLAAAA MMDAAA AAAAxx +4145 2353 1 1 5 5 45 145 145 4145 4145 90 91 LDAAAA NMDAAA HHHHxx +863 2354 1 3 3 3 63 863 863 863 863 126 127 FHAAAA OMDAAA OOOOxx +3423 2355 1 3 3 3 23 423 1423 3423 3423 46 47 RBAAAA PMDAAA VVVVxx +8750 2356 0 2 0 10 50 750 750 3750 8750 100 101 OYAAAA QMDAAA AAAAxx +3546 2357 0 2 6 6 46 546 1546 3546 3546 92 93 KGAAAA RMDAAA HHHHxx +3678 2358 0 2 8 18 78 678 1678 3678 3678 156 157 MLAAAA SMDAAA OOOOxx +5313 2359 1 1 3 13 13 313 1313 313 5313 26 27 JWAAAA TMDAAA VVVVxx +6233 2360 1 1 3 13 33 233 233 1233 6233 66 67 TFAAAA UMDAAA AAAAxx +5802 2361 0 2 2 2 2 802 1802 802 5802 4 5 EPAAAA VMDAAA HHHHxx +7059 2362 1 3 9 19 59 59 1059 2059 7059 118 119 NLAAAA WMDAAA OOOOxx +6481 2363 1 1 1 1 81 481 481 1481 6481 162 163 HPAAAA XMDAAA VVVVxx +1596 2364 0 0 6 16 96 596 1596 1596 1596 192 193 KJAAAA YMDAAA AAAAxx +8181 2365 1 1 1 1 81 181 181 3181 8181 162 163 RCAAAA ZMDAAA HHHHxx +5368 2366 0 0 8 8 68 368 1368 368 5368 136 137 MYAAAA ANDAAA OOOOxx +9416 2367 0 0 6 16 16 416 1416 4416 9416 32 33 EYAAAA BNDAAA VVVVxx +9521 2368 1 1 1 1 21 521 1521 4521 9521 42 43 FCAAAA CNDAAA AAAAxx +1042 2369 0 2 2 2 42 42 1042 1042 1042 84 85 COAAAA DNDAAA HHHHxx +4503 2370 1 3 3 3 3 503 503 4503 4503 6 7 FRAAAA ENDAAA OOOOxx +3023 2371 1 3 3 3 23 23 1023 3023 3023 46 47 HMAAAA FNDAAA VVVVxx +1976 2372 0 0 6 16 76 976 1976 1976 1976 152 153 AYAAAA GNDAAA AAAAxx +5610 2373 0 2 0 10 10 610 1610 610 5610 20 21 UHAAAA HNDAAA HHHHxx +7410 2374 0 2 0 10 10 410 1410 2410 7410 20 21 AZAAAA INDAAA OOOOxx +7872 2375 0 0 2 12 72 872 1872 2872 7872 144 145 UQAAAA JNDAAA VVVVxx +8591 2376 1 3 1 11 91 591 591 3591 8591 182 183 LSAAAA KNDAAA AAAAxx +1804 2377 0 0 4 4 4 804 1804 1804 1804 8 9 KRAAAA LNDAAA HHHHxx +5299 2378 1 3 9 19 99 299 1299 299 5299 198 199 VVAAAA MNDAAA OOOOxx +4695 2379 1 3 5 15 95 695 695 4695 4695 190 191 PYAAAA NNDAAA VVVVxx +2672 2380 0 0 2 12 72 672 672 2672 2672 144 145 UYAAAA ONDAAA AAAAxx +585 2381 1 1 5 5 85 585 585 585 585 170 171 NWAAAA PNDAAA HHHHxx +8622 2382 0 2 2 2 22 622 622 3622 8622 44 45 QTAAAA QNDAAA OOOOxx +3780 2383 0 0 0 0 80 780 1780 3780 3780 160 161 KPAAAA RNDAAA VVVVxx +7941 2384 1 1 1 1 41 941 1941 2941 7941 82 83 LTAAAA SNDAAA AAAAxx +3305 2385 1 1 5 5 5 305 1305 3305 3305 10 11 DXAAAA TNDAAA HHHHxx +8653 2386 1 1 3 13 53 653 653 3653 8653 106 107 VUAAAA UNDAAA OOOOxx +5756 2387 0 0 6 16 56 756 1756 756 5756 112 113 KNAAAA VNDAAA VVVVxx +576 2388 0 0 6 16 76 576 576 576 576 152 153 EWAAAA WNDAAA AAAAxx +1915 2389 1 3 5 15 15 915 1915 1915 1915 30 31 RVAAAA XNDAAA HHHHxx +4627 2390 1 3 7 7 27 627 627 4627 4627 54 55 ZVAAAA YNDAAA OOOOxx +920 2391 0 0 0 0 20 920 920 920 920 40 41 KJAAAA ZNDAAA VVVVxx +2537 2392 1 1 7 17 37 537 537 2537 2537 74 75 PTAAAA AODAAA AAAAxx +50 2393 0 2 0 10 50 50 50 50 50 100 101 YBAAAA BODAAA HHHHxx +1313 2394 1 1 3 13 13 313 1313 1313 1313 26 27 NYAAAA CODAAA OOOOxx +8542 2395 0 2 2 2 42 542 542 3542 8542 84 85 OQAAAA DODAAA VVVVxx +6428 2396 0 0 8 8 28 428 428 1428 6428 56 57 GNAAAA EODAAA AAAAxx +4351 2397 1 3 1 11 51 351 351 4351 4351 102 103 JLAAAA FODAAA HHHHxx +2050 2398 0 2 0 10 50 50 50 2050 2050 100 101 WAAAAA GODAAA OOOOxx +5162 2399 0 2 2 2 62 162 1162 162 5162 124 125 OQAAAA HODAAA VVVVxx +8229 2400 1 1 9 9 29 229 229 3229 8229 58 59 NEAAAA IODAAA AAAAxx +7782 2401 0 2 2 2 82 782 1782 2782 7782 164 165 INAAAA JODAAA HHHHxx +1563 2402 1 3 3 3 63 563 1563 1563 1563 126 127 DIAAAA KODAAA OOOOxx +267 2403 1 3 7 7 67 267 267 267 267 134 135 HKAAAA LODAAA VVVVxx +5138 2404 0 2 8 18 38 138 1138 138 5138 76 77 QPAAAA MODAAA AAAAxx +7022 2405 0 2 2 2 22 22 1022 2022 7022 44 45 CKAAAA NODAAA HHHHxx +6705 2406 1 1 5 5 5 705 705 1705 6705 10 11 XXAAAA OODAAA OOOOxx +6190 2407 0 2 0 10 90 190 190 1190 6190 180 181 CEAAAA PODAAA VVVVxx +8226 2408 0 2 6 6 26 226 226 3226 8226 52 53 KEAAAA QODAAA AAAAxx +8882 2409 0 2 2 2 82 882 882 3882 8882 164 165 QDAAAA RODAAA HHHHxx +5181 2410 1 1 1 1 81 181 1181 181 5181 162 163 HRAAAA SODAAA OOOOxx +4598 2411 0 2 8 18 98 598 598 4598 4598 196 197 WUAAAA TODAAA VVVVxx +4882 2412 0 2 2 2 82 882 882 4882 4882 164 165 UFAAAA UODAAA AAAAxx +7490 2413 0 2 0 10 90 490 1490 2490 7490 180 181 CCAAAA VODAAA HHHHxx +5224 2414 0 0 4 4 24 224 1224 224 5224 48 49 YSAAAA WODAAA OOOOxx +2174 2415 0 2 4 14 74 174 174 2174 2174 148 149 QFAAAA XODAAA VVVVxx +3059 2416 1 3 9 19 59 59 1059 3059 3059 118 119 RNAAAA YODAAA AAAAxx +8790 2417 0 2 0 10 90 790 790 3790 8790 180 181 CAAAAA ZODAAA HHHHxx +2222 2418 0 2 2 2 22 222 222 2222 2222 44 45 MHAAAA APDAAA OOOOxx +5473 2419 1 1 3 13 73 473 1473 473 5473 146 147 NCAAAA BPDAAA VVVVxx +937 2420 1 1 7 17 37 937 937 937 937 74 75 BKAAAA CPDAAA AAAAxx +2975 2421 1 3 5 15 75 975 975 2975 2975 150 151 LKAAAA DPDAAA HHHHxx +9569 2422 1 1 9 9 69 569 1569 4569 9569 138 139 BEAAAA EPDAAA OOOOxx +3456 2423 0 0 6 16 56 456 1456 3456 3456 112 113 YCAAAA FPDAAA VVVVxx +6657 2424 1 1 7 17 57 657 657 1657 6657 114 115 BWAAAA GPDAAA AAAAxx +3776 2425 0 0 6 16 76 776 1776 3776 3776 152 153 GPAAAA HPDAAA HHHHxx +6072 2426 0 0 2 12 72 72 72 1072 6072 144 145 OZAAAA IPDAAA OOOOxx +8129 2427 1 1 9 9 29 129 129 3129 8129 58 59 RAAAAA JPDAAA VVVVxx +1085 2428 1 1 5 5 85 85 1085 1085 1085 170 171 TPAAAA KPDAAA AAAAxx +2079 2429 1 3 9 19 79 79 79 2079 2079 158 159 ZBAAAA LPDAAA HHHHxx +1200 2430 0 0 0 0 0 200 1200 1200 1200 0 1 EUAAAA MPDAAA OOOOxx +3276 2431 0 0 6 16 76 276 1276 3276 3276 152 153 AWAAAA NPDAAA VVVVxx +2608 2432 0 0 8 8 8 608 608 2608 2608 16 17 IWAAAA OPDAAA AAAAxx +702 2433 0 2 2 2 2 702 702 702 702 4 5 ABAAAA PPDAAA HHHHxx +5750 2434 0 2 0 10 50 750 1750 750 5750 100 101 ENAAAA QPDAAA OOOOxx +2776 2435 0 0 6 16 76 776 776 2776 2776 152 153 UCAAAA RPDAAA VVVVxx +9151 2436 1 3 1 11 51 151 1151 4151 9151 102 103 ZNAAAA SPDAAA AAAAxx +3282 2437 0 2 2 2 82 282 1282 3282 3282 164 165 GWAAAA TPDAAA HHHHxx +408 2438 0 0 8 8 8 408 408 408 408 16 17 SPAAAA UPDAAA OOOOxx +3473 2439 1 1 3 13 73 473 1473 3473 3473 146 147 PDAAAA VPDAAA VVVVxx +7095 2440 1 3 5 15 95 95 1095 2095 7095 190 191 XMAAAA WPDAAA AAAAxx +3288 2441 0 0 8 8 88 288 1288 3288 3288 176 177 MWAAAA XPDAAA HHHHxx +8215 2442 1 3 5 15 15 215 215 3215 8215 30 31 ZDAAAA YPDAAA OOOOxx +6244 2443 0 0 4 4 44 244 244 1244 6244 88 89 EGAAAA ZPDAAA VVVVxx +8440 2444 0 0 0 0 40 440 440 3440 8440 80 81 QMAAAA AQDAAA AAAAxx +3800 2445 0 0 0 0 0 800 1800 3800 3800 0 1 EQAAAA BQDAAA HHHHxx +7279 2446 1 3 9 19 79 279 1279 2279 7279 158 159 ZTAAAA CQDAAA OOOOxx +9206 2447 0 2 6 6 6 206 1206 4206 9206 12 13 CQAAAA DQDAAA VVVVxx +6465 2448 1 1 5 5 65 465 465 1465 6465 130 131 ROAAAA EQDAAA AAAAxx +4127 2449 1 3 7 7 27 127 127 4127 4127 54 55 TCAAAA FQDAAA HHHHxx +7463 2450 1 3 3 3 63 463 1463 2463 7463 126 127 BBAAAA GQDAAA OOOOxx +5117 2451 1 1 7 17 17 117 1117 117 5117 34 35 VOAAAA HQDAAA VVVVxx +4715 2452 1 3 5 15 15 715 715 4715 4715 30 31 JZAAAA IQDAAA AAAAxx +2010 2453 0 2 0 10 10 10 10 2010 2010 20 21 IZAAAA JQDAAA HHHHxx +6486 2454 0 2 6 6 86 486 486 1486 6486 172 173 MPAAAA KQDAAA OOOOxx +6434 2455 0 2 4 14 34 434 434 1434 6434 68 69 MNAAAA LQDAAA VVVVxx +2151 2456 1 3 1 11 51 151 151 2151 2151 102 103 TEAAAA MQDAAA AAAAxx +4821 2457 1 1 1 1 21 821 821 4821 4821 42 43 LDAAAA NQDAAA HHHHxx +6507 2458 1 3 7 7 7 507 507 1507 6507 14 15 HQAAAA OQDAAA OOOOxx +8741 2459 1 1 1 1 41 741 741 3741 8741 82 83 FYAAAA PQDAAA VVVVxx +6846 2460 0 2 6 6 46 846 846 1846 6846 92 93 IDAAAA QQDAAA AAAAxx +4525 2461 1 1 5 5 25 525 525 4525 4525 50 51 BSAAAA RQDAAA HHHHxx +8299 2462 1 3 9 19 99 299 299 3299 8299 198 199 FHAAAA SQDAAA OOOOxx +5465 2463 1 1 5 5 65 465 1465 465 5465 130 131 FCAAAA TQDAAA VVVVxx +7206 2464 0 2 6 6 6 206 1206 2206 7206 12 13 ERAAAA UQDAAA AAAAxx +2616 2465 0 0 6 16 16 616 616 2616 2616 32 33 QWAAAA VQDAAA HHHHxx +4440 2466 0 0 0 0 40 440 440 4440 4440 80 81 UOAAAA WQDAAA OOOOxx +6109 2467 1 1 9 9 9 109 109 1109 6109 18 19 ZAAAAA XQDAAA VVVVxx +7905 2468 1 1 5 5 5 905 1905 2905 7905 10 11 BSAAAA YQDAAA AAAAxx +6498 2469 0 2 8 18 98 498 498 1498 6498 196 197 YPAAAA ZQDAAA HHHHxx +2034 2470 0 2 4 14 34 34 34 2034 2034 68 69 GAAAAA ARDAAA OOOOxx +7693 2471 1 1 3 13 93 693 1693 2693 7693 186 187 XJAAAA BRDAAA VVVVxx +7511 2472 1 3 1 11 11 511 1511 2511 7511 22 23 XCAAAA CRDAAA AAAAxx +7531 2473 1 3 1 11 31 531 1531 2531 7531 62 63 RDAAAA DRDAAA HHHHxx +6869 2474 1 1 9 9 69 869 869 1869 6869 138 139 FEAAAA ERDAAA OOOOxx +2763 2475 1 3 3 3 63 763 763 2763 2763 126 127 HCAAAA FRDAAA VVVVxx +575 2476 1 3 5 15 75 575 575 575 575 150 151 DWAAAA GRDAAA AAAAxx +8953 2477 1 1 3 13 53 953 953 3953 8953 106 107 JGAAAA HRDAAA HHHHxx +5833 2478 1 1 3 13 33 833 1833 833 5833 66 67 JQAAAA IRDAAA OOOOxx +9035 2479 1 3 5 15 35 35 1035 4035 9035 70 71 NJAAAA JRDAAA VVVVxx +9123 2480 1 3 3 3 23 123 1123 4123 9123 46 47 XMAAAA KRDAAA AAAAxx +206 2481 0 2 6 6 6 206 206 206 206 12 13 YHAAAA LRDAAA HHHHxx +4155 2482 1 3 5 15 55 155 155 4155 4155 110 111 VDAAAA MRDAAA OOOOxx +532 2483 0 0 2 12 32 532 532 532 532 64 65 MUAAAA NRDAAA VVVVxx +1370 2484 0 2 0 10 70 370 1370 1370 1370 140 141 SAAAAA ORDAAA AAAAxx +7656 2485 0 0 6 16 56 656 1656 2656 7656 112 113 MIAAAA PRDAAA HHHHxx +7735 2486 1 3 5 15 35 735 1735 2735 7735 70 71 NLAAAA QRDAAA OOOOxx +2118 2487 0 2 8 18 18 118 118 2118 2118 36 37 MDAAAA RRDAAA VVVVxx +6914 2488 0 2 4 14 14 914 914 1914 6914 28 29 YFAAAA SRDAAA AAAAxx +6277 2489 1 1 7 17 77 277 277 1277 6277 154 155 LHAAAA TRDAAA HHHHxx +6347 2490 1 3 7 7 47 347 347 1347 6347 94 95 DKAAAA URDAAA OOOOxx +4030 2491 0 2 0 10 30 30 30 4030 4030 60 61 AZAAAA VRDAAA VVVVxx +9673 2492 1 1 3 13 73 673 1673 4673 9673 146 147 BIAAAA WRDAAA AAAAxx +2015 2493 1 3 5 15 15 15 15 2015 2015 30 31 NZAAAA XRDAAA HHHHxx +1317 2494 1 1 7 17 17 317 1317 1317 1317 34 35 RYAAAA YRDAAA OOOOxx +404 2495 0 0 4 4 4 404 404 404 404 8 9 OPAAAA ZRDAAA VVVVxx +1604 2496 0 0 4 4 4 604 1604 1604 1604 8 9 SJAAAA ASDAAA AAAAxx +1912 2497 0 0 2 12 12 912 1912 1912 1912 24 25 OVAAAA BSDAAA HHHHxx +5727 2498 1 3 7 7 27 727 1727 727 5727 54 55 HMAAAA CSDAAA OOOOxx +4538 2499 0 2 8 18 38 538 538 4538 4538 76 77 OSAAAA DSDAAA VVVVxx +6868 2500 0 0 8 8 68 868 868 1868 6868 136 137 EEAAAA ESDAAA AAAAxx +9801 2501 1 1 1 1 1 801 1801 4801 9801 2 3 ZMAAAA FSDAAA HHHHxx +1781 2502 1 1 1 1 81 781 1781 1781 1781 162 163 NQAAAA GSDAAA OOOOxx +7061 2503 1 1 1 1 61 61 1061 2061 7061 122 123 PLAAAA HSDAAA VVVVxx +2412 2504 0 0 2 12 12 412 412 2412 2412 24 25 UOAAAA ISDAAA AAAAxx +9191 2505 1 3 1 11 91 191 1191 4191 9191 182 183 NPAAAA JSDAAA HHHHxx +1958 2506 0 2 8 18 58 958 1958 1958 1958 116 117 IXAAAA KSDAAA OOOOxx +2203 2507 1 3 3 3 3 203 203 2203 2203 6 7 TGAAAA LSDAAA VVVVxx +9104 2508 0 0 4 4 4 104 1104 4104 9104 8 9 EMAAAA MSDAAA AAAAxx +3837 2509 1 1 7 17 37 837 1837 3837 3837 74 75 PRAAAA NSDAAA HHHHxx +7055 2510 1 3 5 15 55 55 1055 2055 7055 110 111 JLAAAA OSDAAA OOOOxx +4612 2511 0 0 2 12 12 612 612 4612 4612 24 25 KVAAAA PSDAAA VVVVxx +6420 2512 0 0 0 0 20 420 420 1420 6420 40 41 YMAAAA QSDAAA AAAAxx +613 2513 1 1 3 13 13 613 613 613 613 26 27 PXAAAA RSDAAA HHHHxx +1691 2514 1 3 1 11 91 691 1691 1691 1691 182 183 BNAAAA SSDAAA OOOOxx +33 2515 1 1 3 13 33 33 33 33 33 66 67 HBAAAA TSDAAA VVVVxx +875 2516 1 3 5 15 75 875 875 875 875 150 151 RHAAAA USDAAA AAAAxx +9030 2517 0 2 0 10 30 30 1030 4030 9030 60 61 IJAAAA VSDAAA HHHHxx +4285 2518 1 1 5 5 85 285 285 4285 4285 170 171 VIAAAA WSDAAA OOOOxx +6236 2519 0 0 6 16 36 236 236 1236 6236 72 73 WFAAAA XSDAAA VVVVxx +4702 2520 0 2 2 2 2 702 702 4702 4702 4 5 WYAAAA YSDAAA AAAAxx +3441 2521 1 1 1 1 41 441 1441 3441 3441 82 83 JCAAAA ZSDAAA HHHHxx +2150 2522 0 2 0 10 50 150 150 2150 2150 100 101 SEAAAA ATDAAA OOOOxx +1852 2523 0 0 2 12 52 852 1852 1852 1852 104 105 GTAAAA BTDAAA VVVVxx +7713 2524 1 1 3 13 13 713 1713 2713 7713 26 27 RKAAAA CTDAAA AAAAxx +6849 2525 1 1 9 9 49 849 849 1849 6849 98 99 LDAAAA DTDAAA HHHHxx +3425 2526 1 1 5 5 25 425 1425 3425 3425 50 51 TBAAAA ETDAAA OOOOxx +4681 2527 1 1 1 1 81 681 681 4681 4681 162 163 BYAAAA FTDAAA VVVVxx +1134 2528 0 2 4 14 34 134 1134 1134 1134 68 69 QRAAAA GTDAAA AAAAxx +7462 2529 0 2 2 2 62 462 1462 2462 7462 124 125 ABAAAA HTDAAA HHHHxx +2148 2530 0 0 8 8 48 148 148 2148 2148 96 97 QEAAAA ITDAAA OOOOxx +5921 2531 1 1 1 1 21 921 1921 921 5921 42 43 TTAAAA JTDAAA VVVVxx +118 2532 0 2 8 18 18 118 118 118 118 36 37 OEAAAA KTDAAA AAAAxx +3065 2533 1 1 5 5 65 65 1065 3065 3065 130 131 XNAAAA LTDAAA HHHHxx +6590 2534 0 2 0 10 90 590 590 1590 6590 180 181 MTAAAA MTDAAA OOOOxx +4993 2535 1 1 3 13 93 993 993 4993 4993 186 187 BKAAAA NTDAAA VVVVxx +6818 2536 0 2 8 18 18 818 818 1818 6818 36 37 GCAAAA OTDAAA AAAAxx +1449 2537 1 1 9 9 49 449 1449 1449 1449 98 99 TDAAAA PTDAAA HHHHxx +2039 2538 1 3 9 19 39 39 39 2039 2039 78 79 LAAAAA QTDAAA OOOOxx +2524 2539 0 0 4 4 24 524 524 2524 2524 48 49 CTAAAA RTDAAA VVVVxx +1481 2540 1 1 1 1 81 481 1481 1481 1481 162 163 ZEAAAA STDAAA AAAAxx +6984 2541 0 0 4 4 84 984 984 1984 6984 168 169 QIAAAA TTDAAA HHHHxx +3960 2542 0 0 0 0 60 960 1960 3960 3960 120 121 IWAAAA UTDAAA OOOOxx +1983 2543 1 3 3 3 83 983 1983 1983 1983 166 167 HYAAAA VTDAAA VVVVxx +6379 2544 1 3 9 19 79 379 379 1379 6379 158 159 JLAAAA WTDAAA AAAAxx +8975 2545 1 3 5 15 75 975 975 3975 8975 150 151 FHAAAA XTDAAA HHHHxx +1102 2546 0 2 2 2 2 102 1102 1102 1102 4 5 KQAAAA YTDAAA OOOOxx +2517 2547 1 1 7 17 17 517 517 2517 2517 34 35 VSAAAA ZTDAAA VVVVxx +712 2548 0 0 2 12 12 712 712 712 712 24 25 KBAAAA AUDAAA AAAAxx +5419 2549 1 3 9 19 19 419 1419 419 5419 38 39 LAAAAA BUDAAA HHHHxx +723 2550 1 3 3 3 23 723 723 723 723 46 47 VBAAAA CUDAAA OOOOxx +8057 2551 1 1 7 17 57 57 57 3057 8057 114 115 XXAAAA DUDAAA VVVVxx +7471 2552 1 3 1 11 71 471 1471 2471 7471 142 143 JBAAAA EUDAAA AAAAxx +8855 2553 1 3 5 15 55 855 855 3855 8855 110 111 PCAAAA FUDAAA HHHHxx +5074 2554 0 2 4 14 74 74 1074 74 5074 148 149 ENAAAA GUDAAA OOOOxx +7139 2555 1 3 9 19 39 139 1139 2139 7139 78 79 POAAAA HUDAAA VVVVxx +3833 2556 1 1 3 13 33 833 1833 3833 3833 66 67 LRAAAA IUDAAA AAAAxx +5186 2557 0 2 6 6 86 186 1186 186 5186 172 173 MRAAAA JUDAAA HHHHxx +9436 2558 0 0 6 16 36 436 1436 4436 9436 72 73 YYAAAA KUDAAA OOOOxx +8859 2559 1 3 9 19 59 859 859 3859 8859 118 119 TCAAAA LUDAAA VVVVxx +6943 2560 1 3 3 3 43 943 943 1943 6943 86 87 BHAAAA MUDAAA AAAAxx +2315 2561 1 3 5 15 15 315 315 2315 2315 30 31 BLAAAA NUDAAA HHHHxx +1394 2562 0 2 4 14 94 394 1394 1394 1394 188 189 QBAAAA OUDAAA OOOOxx +8863 2563 1 3 3 3 63 863 863 3863 8863 126 127 XCAAAA PUDAAA VVVVxx +8812 2564 0 0 2 12 12 812 812 3812 8812 24 25 YAAAAA QUDAAA AAAAxx +7498 2565 0 2 8 18 98 498 1498 2498 7498 196 197 KCAAAA RUDAAA HHHHxx +8962 2566 0 2 2 2 62 962 962 3962 8962 124 125 SGAAAA SUDAAA OOOOxx +2533 2567 1 1 3 13 33 533 533 2533 2533 66 67 LTAAAA TUDAAA VVVVxx +8188 2568 0 0 8 8 88 188 188 3188 8188 176 177 YCAAAA UUDAAA AAAAxx +6137 2569 1 1 7 17 37 137 137 1137 6137 74 75 BCAAAA VUDAAA HHHHxx +974 2570 0 2 4 14 74 974 974 974 974 148 149 MLAAAA WUDAAA OOOOxx +2751 2571 1 3 1 11 51 751 751 2751 2751 102 103 VBAAAA XUDAAA VVVVxx +4975 2572 1 3 5 15 75 975 975 4975 4975 150 151 JJAAAA YUDAAA AAAAxx +3411 2573 1 3 1 11 11 411 1411 3411 3411 22 23 FBAAAA ZUDAAA HHHHxx +3143 2574 1 3 3 3 43 143 1143 3143 3143 86 87 XQAAAA AVDAAA OOOOxx +8011 2575 1 3 1 11 11 11 11 3011 8011 22 23 DWAAAA BVDAAA VVVVxx +988 2576 0 0 8 8 88 988 988 988 988 176 177 AMAAAA CVDAAA AAAAxx +4289 2577 1 1 9 9 89 289 289 4289 4289 178 179 ZIAAAA DVDAAA HHHHxx +8105 2578 1 1 5 5 5 105 105 3105 8105 10 11 TZAAAA EVDAAA OOOOxx +9885 2579 1 1 5 5 85 885 1885 4885 9885 170 171 FQAAAA FVDAAA VVVVxx +1002 2580 0 2 2 2 2 2 1002 1002 1002 4 5 OMAAAA GVDAAA AAAAxx +5827 2581 1 3 7 7 27 827 1827 827 5827 54 55 DQAAAA HVDAAA HHHHxx +1228 2582 0 0 8 8 28 228 1228 1228 1228 56 57 GVAAAA IVDAAA OOOOxx +6352 2583 0 0 2 12 52 352 352 1352 6352 104 105 IKAAAA JVDAAA VVVVxx +8868 2584 0 0 8 8 68 868 868 3868 8868 136 137 CDAAAA KVDAAA AAAAxx +3643 2585 1 3 3 3 43 643 1643 3643 3643 86 87 DKAAAA LVDAAA HHHHxx +1468 2586 0 0 8 8 68 468 1468 1468 1468 136 137 MEAAAA MVDAAA OOOOxx +8415 2587 1 3 5 15 15 415 415 3415 8415 30 31 RLAAAA NVDAAA VVVVxx +9631 2588 1 3 1 11 31 631 1631 4631 9631 62 63 LGAAAA OVDAAA AAAAxx +7408 2589 0 0 8 8 8 408 1408 2408 7408 16 17 YYAAAA PVDAAA HHHHxx +1934 2590 0 2 4 14 34 934 1934 1934 1934 68 69 KWAAAA QVDAAA OOOOxx +996 2591 0 0 6 16 96 996 996 996 996 192 193 IMAAAA RVDAAA VVVVxx +8027 2592 1 3 7 7 27 27 27 3027 8027 54 55 TWAAAA SVDAAA AAAAxx +8464 2593 0 0 4 4 64 464 464 3464 8464 128 129 ONAAAA TVDAAA HHHHxx +5007 2594 1 3 7 7 7 7 1007 7 5007 14 15 PKAAAA UVDAAA OOOOxx +8356 2595 0 0 6 16 56 356 356 3356 8356 112 113 KJAAAA VVDAAA VVVVxx +4579 2596 1 3 9 19 79 579 579 4579 4579 158 159 DUAAAA WVDAAA AAAAxx +8513 2597 1 1 3 13 13 513 513 3513 8513 26 27 LPAAAA XVDAAA HHHHxx +383 2598 1 3 3 3 83 383 383 383 383 166 167 TOAAAA YVDAAA OOOOxx +9304 2599 0 0 4 4 4 304 1304 4304 9304 8 9 WTAAAA ZVDAAA VVVVxx +7224 2600 0 0 4 4 24 224 1224 2224 7224 48 49 WRAAAA AWDAAA AAAAxx +6023 2601 1 3 3 3 23 23 23 1023 6023 46 47 RXAAAA BWDAAA HHHHxx +2746 2602 0 2 6 6 46 746 746 2746 2746 92 93 QBAAAA CWDAAA OOOOxx +137 2603 1 1 7 17 37 137 137 137 137 74 75 HFAAAA DWDAAA VVVVxx +9441 2604 1 1 1 1 41 441 1441 4441 9441 82 83 DZAAAA EWDAAA AAAAxx +3690 2605 0 2 0 10 90 690 1690 3690 3690 180 181 YLAAAA FWDAAA HHHHxx +913 2606 1 1 3 13 13 913 913 913 913 26 27 DJAAAA GWDAAA OOOOxx +1768 2607 0 0 8 8 68 768 1768 1768 1768 136 137 AQAAAA HWDAAA VVVVxx +8492 2608 0 0 2 12 92 492 492 3492 8492 184 185 QOAAAA IWDAAA AAAAxx +8083 2609 1 3 3 3 83 83 83 3083 8083 166 167 XYAAAA JWDAAA HHHHxx +4609 2610 1 1 9 9 9 609 609 4609 4609 18 19 HVAAAA KWDAAA OOOOxx +7520 2611 0 0 0 0 20 520 1520 2520 7520 40 41 GDAAAA LWDAAA VVVVxx +4231 2612 1 3 1 11 31 231 231 4231 4231 62 63 TGAAAA MWDAAA AAAAxx +6022 2613 0 2 2 2 22 22 22 1022 6022 44 45 QXAAAA NWDAAA HHHHxx +9784 2614 0 0 4 4 84 784 1784 4784 9784 168 169 IMAAAA OWDAAA OOOOxx +1343 2615 1 3 3 3 43 343 1343 1343 1343 86 87 RZAAAA PWDAAA VVVVxx +7549 2616 1 1 9 9 49 549 1549 2549 7549 98 99 JEAAAA QWDAAA AAAAxx +269 2617 1 1 9 9 69 269 269 269 269 138 139 JKAAAA RWDAAA HHHHxx +1069 2618 1 1 9 9 69 69 1069 1069 1069 138 139 DPAAAA SWDAAA OOOOxx +4610 2619 0 2 0 10 10 610 610 4610 4610 20 21 IVAAAA TWDAAA VVVVxx +482 2620 0 2 2 2 82 482 482 482 482 164 165 OSAAAA UWDAAA AAAAxx +3025 2621 1 1 5 5 25 25 1025 3025 3025 50 51 JMAAAA VWDAAA HHHHxx +7914 2622 0 2 4 14 14 914 1914 2914 7914 28 29 KSAAAA WWDAAA OOOOxx +3198 2623 0 2 8 18 98 198 1198 3198 3198 196 197 ATAAAA XWDAAA VVVVxx +1187 2624 1 3 7 7 87 187 1187 1187 1187 174 175 RTAAAA YWDAAA AAAAxx +4707 2625 1 3 7 7 7 707 707 4707 4707 14 15 BZAAAA ZWDAAA HHHHxx +8279 2626 1 3 9 19 79 279 279 3279 8279 158 159 LGAAAA AXDAAA OOOOxx +6127 2627 1 3 7 7 27 127 127 1127 6127 54 55 RBAAAA BXDAAA VVVVxx +1305 2628 1 1 5 5 5 305 1305 1305 1305 10 11 FYAAAA CXDAAA AAAAxx +4804 2629 0 0 4 4 4 804 804 4804 4804 8 9 UCAAAA DXDAAA HHHHxx +6069 2630 1 1 9 9 69 69 69 1069 6069 138 139 LZAAAA EXDAAA OOOOxx +9229 2631 1 1 9 9 29 229 1229 4229 9229 58 59 ZQAAAA FXDAAA VVVVxx +4703 2632 1 3 3 3 3 703 703 4703 4703 6 7 XYAAAA GXDAAA AAAAxx +6410 2633 0 2 0 10 10 410 410 1410 6410 20 21 OMAAAA HXDAAA HHHHxx +944 2634 0 0 4 4 44 944 944 944 944 88 89 IKAAAA IXDAAA OOOOxx +3744 2635 0 0 4 4 44 744 1744 3744 3744 88 89 AOAAAA JXDAAA VVVVxx +1127 2636 1 3 7 7 27 127 1127 1127 1127 54 55 JRAAAA KXDAAA AAAAxx +6693 2637 1 1 3 13 93 693 693 1693 6693 186 187 LXAAAA LXDAAA HHHHxx +583 2638 1 3 3 3 83 583 583 583 583 166 167 LWAAAA MXDAAA OOOOxx +2684 2639 0 0 4 4 84 684 684 2684 2684 168 169 GZAAAA NXDAAA VVVVxx +6192 2640 0 0 2 12 92 192 192 1192 6192 184 185 EEAAAA OXDAAA AAAAxx +4157 2641 1 1 7 17 57 157 157 4157 4157 114 115 XDAAAA PXDAAA HHHHxx +6470 2642 0 2 0 10 70 470 470 1470 6470 140 141 WOAAAA QXDAAA OOOOxx +8965 2643 1 1 5 5 65 965 965 3965 8965 130 131 VGAAAA RXDAAA VVVVxx +1433 2644 1 1 3 13 33 433 1433 1433 1433 66 67 DDAAAA SXDAAA AAAAxx +4570 2645 0 2 0 10 70 570 570 4570 4570 140 141 UTAAAA TXDAAA HHHHxx +1806 2646 0 2 6 6 6 806 1806 1806 1806 12 13 MRAAAA UXDAAA OOOOxx +1230 2647 0 2 0 10 30 230 1230 1230 1230 60 61 IVAAAA VXDAAA VVVVxx +2283 2648 1 3 3 3 83 283 283 2283 2283 166 167 VJAAAA WXDAAA AAAAxx +6456 2649 0 0 6 16 56 456 456 1456 6456 112 113 IOAAAA XXDAAA HHHHxx +7427 2650 1 3 7 7 27 427 1427 2427 7427 54 55 RZAAAA YXDAAA OOOOxx +8310 2651 0 2 0 10 10 310 310 3310 8310 20 21 QHAAAA ZXDAAA VVVVxx +8103 2652 1 3 3 3 3 103 103 3103 8103 6 7 RZAAAA AYDAAA AAAAxx +3947 2653 1 3 7 7 47 947 1947 3947 3947 94 95 VVAAAA BYDAAA HHHHxx +3414 2654 0 2 4 14 14 414 1414 3414 3414 28 29 IBAAAA CYDAAA OOOOxx +2043 2655 1 3 3 3 43 43 43 2043 2043 86 87 PAAAAA DYDAAA VVVVxx +4393 2656 1 1 3 13 93 393 393 4393 4393 186 187 ZMAAAA EYDAAA AAAAxx +6664 2657 0 0 4 4 64 664 664 1664 6664 128 129 IWAAAA FYDAAA HHHHxx +4545 2658 1 1 5 5 45 545 545 4545 4545 90 91 VSAAAA GYDAAA OOOOxx +7637 2659 1 1 7 17 37 637 1637 2637 7637 74 75 THAAAA HYDAAA VVVVxx +1359 2660 1 3 9 19 59 359 1359 1359 1359 118 119 HAAAAA IYDAAA AAAAxx +5018 2661 0 2 8 18 18 18 1018 18 5018 36 37 ALAAAA JYDAAA HHHHxx +987 2662 1 3 7 7 87 987 987 987 987 174 175 ZLAAAA KYDAAA OOOOxx +1320 2663 0 0 0 0 20 320 1320 1320 1320 40 41 UYAAAA LYDAAA VVVVxx +9311 2664 1 3 1 11 11 311 1311 4311 9311 22 23 DUAAAA MYDAAA AAAAxx +7993 2665 1 1 3 13 93 993 1993 2993 7993 186 187 LVAAAA NYDAAA HHHHxx +7588 2666 0 0 8 8 88 588 1588 2588 7588 176 177 WFAAAA OYDAAA OOOOxx +5983 2667 1 3 3 3 83 983 1983 983 5983 166 167 DWAAAA PYDAAA VVVVxx +4070 2668 0 2 0 10 70 70 70 4070 4070 140 141 OAAAAA QYDAAA AAAAxx +8349 2669 1 1 9 9 49 349 349 3349 8349 98 99 DJAAAA RYDAAA HHHHxx +3810 2670 0 2 0 10 10 810 1810 3810 3810 20 21 OQAAAA SYDAAA OOOOxx +6948 2671 0 0 8 8 48 948 948 1948 6948 96 97 GHAAAA TYDAAA VVVVxx +7153 2672 1 1 3 13 53 153 1153 2153 7153 106 107 DPAAAA UYDAAA AAAAxx +5371 2673 1 3 1 11 71 371 1371 371 5371 142 143 PYAAAA VYDAAA HHHHxx +8316 2674 0 0 6 16 16 316 316 3316 8316 32 33 WHAAAA WYDAAA OOOOxx +5903 2675 1 3 3 3 3 903 1903 903 5903 6 7 BTAAAA XYDAAA VVVVxx +6718 2676 0 2 8 18 18 718 718 1718 6718 36 37 KYAAAA YYDAAA AAAAxx +4759 2677 1 3 9 19 59 759 759 4759 4759 118 119 BBAAAA ZYDAAA HHHHxx +2555 2678 1 3 5 15 55 555 555 2555 2555 110 111 HUAAAA AZDAAA OOOOxx +3457 2679 1 1 7 17 57 457 1457 3457 3457 114 115 ZCAAAA BZDAAA VVVVxx +9626 2680 0 2 6 6 26 626 1626 4626 9626 52 53 GGAAAA CZDAAA AAAAxx +2570 2681 0 2 0 10 70 570 570 2570 2570 140 141 WUAAAA DZDAAA HHHHxx +7964 2682 0 0 4 4 64 964 1964 2964 7964 128 129 IUAAAA EZDAAA OOOOxx +1543 2683 1 3 3 3 43 543 1543 1543 1543 86 87 JHAAAA FZDAAA VVVVxx +929 2684 1 1 9 9 29 929 929 929 929 58 59 TJAAAA GZDAAA AAAAxx +9244 2685 0 0 4 4 44 244 1244 4244 9244 88 89 ORAAAA HZDAAA HHHHxx +9210 2686 0 2 0 10 10 210 1210 4210 9210 20 21 GQAAAA IZDAAA OOOOxx +8334 2687 0 2 4 14 34 334 334 3334 8334 68 69 OIAAAA JZDAAA VVVVxx +9310 2688 0 2 0 10 10 310 1310 4310 9310 20 21 CUAAAA KZDAAA AAAAxx +5024 2689 0 0 4 4 24 24 1024 24 5024 48 49 GLAAAA LZDAAA HHHHxx +8794 2690 0 2 4 14 94 794 794 3794 8794 188 189 GAAAAA MZDAAA OOOOxx +4091 2691 1 3 1 11 91 91 91 4091 4091 182 183 JBAAAA NZDAAA VVVVxx +649 2692 1 1 9 9 49 649 649 649 649 98 99 ZYAAAA OZDAAA AAAAxx +8505 2693 1 1 5 5 5 505 505 3505 8505 10 11 DPAAAA PZDAAA HHHHxx +6652 2694 0 0 2 12 52 652 652 1652 6652 104 105 WVAAAA QZDAAA OOOOxx +8945 2695 1 1 5 5 45 945 945 3945 8945 90 91 BGAAAA RZDAAA VVVVxx +2095 2696 1 3 5 15 95 95 95 2095 2095 190 191 PCAAAA SZDAAA AAAAxx +8676 2697 0 0 6 16 76 676 676 3676 8676 152 153 SVAAAA TZDAAA HHHHxx +3994 2698 0 2 4 14 94 994 1994 3994 3994 188 189 QXAAAA UZDAAA OOOOxx +2859 2699 1 3 9 19 59 859 859 2859 2859 118 119 ZFAAAA VZDAAA VVVVxx +5403 2700 1 3 3 3 3 403 1403 403 5403 6 7 VZAAAA WZDAAA AAAAxx +3254 2701 0 2 4 14 54 254 1254 3254 3254 108 109 EVAAAA XZDAAA HHHHxx +7339 2702 1 3 9 19 39 339 1339 2339 7339 78 79 HWAAAA YZDAAA OOOOxx +7220 2703 0 0 0 0 20 220 1220 2220 7220 40 41 SRAAAA ZZDAAA VVVVxx +4154 2704 0 2 4 14 54 154 154 4154 4154 108 109 UDAAAA AAEAAA AAAAxx +7570 2705 0 2 0 10 70 570 1570 2570 7570 140 141 EFAAAA BAEAAA HHHHxx +2576 2706 0 0 6 16 76 576 576 2576 2576 152 153 CVAAAA CAEAAA OOOOxx +5764 2707 0 0 4 4 64 764 1764 764 5764 128 129 SNAAAA DAEAAA VVVVxx +4314 2708 0 2 4 14 14 314 314 4314 4314 28 29 YJAAAA EAEAAA AAAAxx +2274 2709 0 2 4 14 74 274 274 2274 2274 148 149 MJAAAA FAEAAA HHHHxx +9756 2710 0 0 6 16 56 756 1756 4756 9756 112 113 GLAAAA GAEAAA OOOOxx +8274 2711 0 2 4 14 74 274 274 3274 8274 148 149 GGAAAA HAEAAA VVVVxx +1289 2712 1 1 9 9 89 289 1289 1289 1289 178 179 PXAAAA IAEAAA AAAAxx +7335 2713 1 3 5 15 35 335 1335 2335 7335 70 71 DWAAAA JAEAAA HHHHxx +5351 2714 1 3 1 11 51 351 1351 351 5351 102 103 VXAAAA KAEAAA OOOOxx +8978 2715 0 2 8 18 78 978 978 3978 8978 156 157 IHAAAA LAEAAA VVVVxx +2 2716 0 2 2 2 2 2 2 2 2 4 5 CAAAAA MAEAAA AAAAxx +8906 2717 0 2 6 6 6 906 906 3906 8906 12 13 OEAAAA NAEAAA HHHHxx +6388 2718 0 0 8 8 88 388 388 1388 6388 176 177 SLAAAA OAEAAA OOOOxx +5675 2719 1 3 5 15 75 675 1675 675 5675 150 151 HKAAAA PAEAAA VVVVxx +255 2720 1 3 5 15 55 255 255 255 255 110 111 VJAAAA QAEAAA AAAAxx +9538 2721 0 2 8 18 38 538 1538 4538 9538 76 77 WCAAAA RAEAAA HHHHxx +1480 2722 0 0 0 0 80 480 1480 1480 1480 160 161 YEAAAA SAEAAA OOOOxx +4015 2723 1 3 5 15 15 15 15 4015 4015 30 31 LYAAAA TAEAAA VVVVxx +5166 2724 0 2 6 6 66 166 1166 166 5166 132 133 SQAAAA UAEAAA AAAAxx +91 2725 1 3 1 11 91 91 91 91 91 182 183 NDAAAA VAEAAA HHHHxx +2958 2726 0 2 8 18 58 958 958 2958 2958 116 117 UJAAAA WAEAAA OOOOxx +9131 2727 1 3 1 11 31 131 1131 4131 9131 62 63 FNAAAA XAEAAA VVVVxx +3944 2728 0 0 4 4 44 944 1944 3944 3944 88 89 SVAAAA YAEAAA AAAAxx +4514 2729 0 2 4 14 14 514 514 4514 4514 28 29 QRAAAA ZAEAAA HHHHxx +5661 2730 1 1 1 1 61 661 1661 661 5661 122 123 TJAAAA ABEAAA OOOOxx +8724 2731 0 0 4 4 24 724 724 3724 8724 48 49 OXAAAA BBEAAA VVVVxx +6408 2732 0 0 8 8 8 408 408 1408 6408 16 17 MMAAAA CBEAAA AAAAxx +5013 2733 1 1 3 13 13 13 1013 13 5013 26 27 VKAAAA DBEAAA HHHHxx +6156 2734 0 0 6 16 56 156 156 1156 6156 112 113 UCAAAA EBEAAA OOOOxx +7350 2735 0 2 0 10 50 350 1350 2350 7350 100 101 SWAAAA FBEAAA VVVVxx +9858 2736 0 2 8 18 58 858 1858 4858 9858 116 117 EPAAAA GBEAAA AAAAxx +895 2737 1 3 5 15 95 895 895 895 895 190 191 LIAAAA HBEAAA HHHHxx +8368 2738 0 0 8 8 68 368 368 3368 8368 136 137 WJAAAA IBEAAA OOOOxx +179 2739 1 3 9 19 79 179 179 179 179 158 159 XGAAAA JBEAAA VVVVxx +4048 2740 0 0 8 8 48 48 48 4048 4048 96 97 SZAAAA KBEAAA AAAAxx +3073 2741 1 1 3 13 73 73 1073 3073 3073 146 147 FOAAAA LBEAAA HHHHxx +321 2742 1 1 1 1 21 321 321 321 321 42 43 JMAAAA MBEAAA OOOOxx +5352 2743 0 0 2 12 52 352 1352 352 5352 104 105 WXAAAA NBEAAA VVVVxx +1940 2744 0 0 0 0 40 940 1940 1940 1940 80 81 QWAAAA OBEAAA AAAAxx +8803 2745 1 3 3 3 3 803 803 3803 8803 6 7 PAAAAA PBEAAA HHHHxx +791 2746 1 3 1 11 91 791 791 791 791 182 183 LEAAAA QBEAAA OOOOxx +9809 2747 1 1 9 9 9 809 1809 4809 9809 18 19 HNAAAA RBEAAA VVVVxx +5519 2748 1 3 9 19 19 519 1519 519 5519 38 39 HEAAAA SBEAAA AAAAxx +7420 2749 0 0 0 0 20 420 1420 2420 7420 40 41 KZAAAA TBEAAA HHHHxx +7541 2750 1 1 1 1 41 541 1541 2541 7541 82 83 BEAAAA UBEAAA OOOOxx +6538 2751 0 2 8 18 38 538 538 1538 6538 76 77 MRAAAA VBEAAA VVVVxx +710 2752 0 2 0 10 10 710 710 710 710 20 21 IBAAAA WBEAAA AAAAxx +9488 2753 0 0 8 8 88 488 1488 4488 9488 176 177 YAAAAA XBEAAA HHHHxx +3135 2754 1 3 5 15 35 135 1135 3135 3135 70 71 PQAAAA YBEAAA OOOOxx +4273 2755 1 1 3 13 73 273 273 4273 4273 146 147 JIAAAA ZBEAAA VVVVxx +629 2756 1 1 9 9 29 629 629 629 629 58 59 FYAAAA ACEAAA AAAAxx +9167 2757 1 3 7 7 67 167 1167 4167 9167 134 135 POAAAA BCEAAA HHHHxx +751 2758 1 3 1 11 51 751 751 751 751 102 103 XCAAAA CCEAAA OOOOxx +1126 2759 0 2 6 6 26 126 1126 1126 1126 52 53 IRAAAA DCEAAA VVVVxx +3724 2760 0 0 4 4 24 724 1724 3724 3724 48 49 GNAAAA ECEAAA AAAAxx +1789 2761 1 1 9 9 89 789 1789 1789 1789 178 179 VQAAAA FCEAAA HHHHxx +792 2762 0 0 2 12 92 792 792 792 792 184 185 MEAAAA GCEAAA OOOOxx +2771 2763 1 3 1 11 71 771 771 2771 2771 142 143 PCAAAA HCEAAA VVVVxx +4313 2764 1 1 3 13 13 313 313 4313 4313 26 27 XJAAAA ICEAAA AAAAxx +9312 2765 0 0 2 12 12 312 1312 4312 9312 24 25 EUAAAA JCEAAA HHHHxx +955 2766 1 3 5 15 55 955 955 955 955 110 111 TKAAAA KCEAAA OOOOxx +6382 2767 0 2 2 2 82 382 382 1382 6382 164 165 MLAAAA LCEAAA VVVVxx +7875 2768 1 3 5 15 75 875 1875 2875 7875 150 151 XQAAAA MCEAAA AAAAxx +7491 2769 1 3 1 11 91 491 1491 2491 7491 182 183 DCAAAA NCEAAA HHHHxx +8193 2770 1 1 3 13 93 193 193 3193 8193 186 187 DDAAAA OCEAAA OOOOxx +968 2771 0 0 8 8 68 968 968 968 968 136 137 GLAAAA PCEAAA VVVVxx +4951 2772 1 3 1 11 51 951 951 4951 4951 102 103 LIAAAA QCEAAA AAAAxx +2204 2773 0 0 4 4 4 204 204 2204 2204 8 9 UGAAAA RCEAAA HHHHxx +2066 2774 0 2 6 6 66 66 66 2066 2066 132 133 MBAAAA SCEAAA OOOOxx +2631 2775 1 3 1 11 31 631 631 2631 2631 62 63 FXAAAA TCEAAA VVVVxx +8947 2776 1 3 7 7 47 947 947 3947 8947 94 95 DGAAAA UCEAAA AAAAxx +8033 2777 1 1 3 13 33 33 33 3033 8033 66 67 ZWAAAA VCEAAA HHHHxx +6264 2778 0 0 4 4 64 264 264 1264 6264 128 129 YGAAAA WCEAAA OOOOxx +7778 2779 0 2 8 18 78 778 1778 2778 7778 156 157 ENAAAA XCEAAA VVVVxx +9701 2780 1 1 1 1 1 701 1701 4701 9701 2 3 DJAAAA YCEAAA AAAAxx +5091 2781 1 3 1 11 91 91 1091 91 5091 182 183 VNAAAA ZCEAAA HHHHxx +7577 2782 1 1 7 17 77 577 1577 2577 7577 154 155 LFAAAA ADEAAA OOOOxx +3345 2783 1 1 5 5 45 345 1345 3345 3345 90 91 RYAAAA BDEAAA VVVVxx +7329 2784 1 1 9 9 29 329 1329 2329 7329 58 59 XVAAAA CDEAAA AAAAxx +7551 2785 1 3 1 11 51 551 1551 2551 7551 102 103 LEAAAA DDEAAA HHHHxx +6207 2786 1 3 7 7 7 207 207 1207 6207 14 15 TEAAAA EDEAAA OOOOxx +8664 2787 0 0 4 4 64 664 664 3664 8664 128 129 GVAAAA FDEAAA VVVVxx +8394 2788 0 2 4 14 94 394 394 3394 8394 188 189 WKAAAA GDEAAA AAAAxx +7324 2789 0 0 4 4 24 324 1324 2324 7324 48 49 SVAAAA HDEAAA HHHHxx +2713 2790 1 1 3 13 13 713 713 2713 2713 26 27 JAAAAA IDEAAA OOOOxx +2230 2791 0 2 0 10 30 230 230 2230 2230 60 61 UHAAAA JDEAAA VVVVxx +9211 2792 1 3 1 11 11 211 1211 4211 9211 22 23 HQAAAA KDEAAA AAAAxx +1296 2793 0 0 6 16 96 296 1296 1296 1296 192 193 WXAAAA LDEAAA HHHHxx +8104 2794 0 0 4 4 4 104 104 3104 8104 8 9 SZAAAA MDEAAA OOOOxx +6916 2795 0 0 6 16 16 916 916 1916 6916 32 33 AGAAAA NDEAAA VVVVxx +2208 2796 0 0 8 8 8 208 208 2208 2208 16 17 YGAAAA ODEAAA AAAAxx +3935 2797 1 3 5 15 35 935 1935 3935 3935 70 71 JVAAAA PDEAAA HHHHxx +7814 2798 0 2 4 14 14 814 1814 2814 7814 28 29 OOAAAA QDEAAA OOOOxx +6508 2799 0 0 8 8 8 508 508 1508 6508 16 17 IQAAAA RDEAAA VVVVxx +1703 2800 1 3 3 3 3 703 1703 1703 1703 6 7 NNAAAA SDEAAA AAAAxx +5640 2801 0 0 0 0 40 640 1640 640 5640 80 81 YIAAAA TDEAAA HHHHxx +6417 2802 1 1 7 17 17 417 417 1417 6417 34 35 VMAAAA UDEAAA OOOOxx +1713 2803 1 1 3 13 13 713 1713 1713 1713 26 27 XNAAAA VDEAAA VVVVxx +5309 2804 1 1 9 9 9 309 1309 309 5309 18 19 FWAAAA WDEAAA AAAAxx +4364 2805 0 0 4 4 64 364 364 4364 4364 128 129 WLAAAA XDEAAA HHHHxx +619 2806 1 3 9 19 19 619 619 619 619 38 39 VXAAAA YDEAAA OOOOxx +9498 2807 0 2 8 18 98 498 1498 4498 9498 196 197 IBAAAA ZDEAAA VVVVxx +2804 2808 0 0 4 4 4 804 804 2804 2804 8 9 WDAAAA AEEAAA AAAAxx +2220 2809 0 0 0 0 20 220 220 2220 2220 40 41 KHAAAA BEEAAA HHHHxx +9542 2810 0 2 2 2 42 542 1542 4542 9542 84 85 ADAAAA CEEAAA OOOOxx +3349 2811 1 1 9 9 49 349 1349 3349 3349 98 99 VYAAAA DEEAAA VVVVxx +9198 2812 0 2 8 18 98 198 1198 4198 9198 196 197 UPAAAA EEEAAA AAAAxx +2727 2813 1 3 7 7 27 727 727 2727 2727 54 55 XAAAAA FEEAAA HHHHxx +3768 2814 0 0 8 8 68 768 1768 3768 3768 136 137 YOAAAA GEEAAA OOOOxx +2334 2815 0 2 4 14 34 334 334 2334 2334 68 69 ULAAAA HEEAAA VVVVxx +7770 2816 0 2 0 10 70 770 1770 2770 7770 140 141 WMAAAA IEEAAA AAAAxx +5963 2817 1 3 3 3 63 963 1963 963 5963 126 127 JVAAAA JEEAAA HHHHxx +4732 2818 0 0 2 12 32 732 732 4732 4732 64 65 AAAAAA KEEAAA OOOOxx +2448 2819 0 0 8 8 48 448 448 2448 2448 96 97 EQAAAA LEEAAA VVVVxx +5998 2820 0 2 8 18 98 998 1998 998 5998 196 197 SWAAAA MEEAAA AAAAxx +8577 2821 1 1 7 17 77 577 577 3577 8577 154 155 XRAAAA NEEAAA HHHHxx +266 2822 0 2 6 6 66 266 266 266 266 132 133 GKAAAA OEEAAA OOOOxx +2169 2823 1 1 9 9 69 169 169 2169 2169 138 139 LFAAAA PEEAAA VVVVxx +8228 2824 0 0 8 8 28 228 228 3228 8228 56 57 MEAAAA QEEAAA AAAAxx +4813 2825 1 1 3 13 13 813 813 4813 4813 26 27 DDAAAA REEAAA HHHHxx +2769 2826 1 1 9 9 69 769 769 2769 2769 138 139 NCAAAA SEEAAA OOOOxx +8382 2827 0 2 2 2 82 382 382 3382 8382 164 165 KKAAAA TEEAAA VVVVxx +1717 2828 1 1 7 17 17 717 1717 1717 1717 34 35 BOAAAA UEEAAA AAAAxx +7178 2829 0 2 8 18 78 178 1178 2178 7178 156 157 CQAAAA VEEAAA HHHHxx +9547 2830 1 3 7 7 47 547 1547 4547 9547 94 95 FDAAAA WEEAAA OOOOxx +8187 2831 1 3 7 7 87 187 187 3187 8187 174 175 XCAAAA XEEAAA VVVVxx +3168 2832 0 0 8 8 68 168 1168 3168 3168 136 137 WRAAAA YEEAAA AAAAxx +2180 2833 0 0 0 0 80 180 180 2180 2180 160 161 WFAAAA ZEEAAA HHHHxx +859 2834 1 3 9 19 59 859 859 859 859 118 119 BHAAAA AFEAAA OOOOxx +1554 2835 0 2 4 14 54 554 1554 1554 1554 108 109 UHAAAA BFEAAA VVVVxx +3567 2836 1 3 7 7 67 567 1567 3567 3567 134 135 FHAAAA CFEAAA AAAAxx +5985 2837 1 1 5 5 85 985 1985 985 5985 170 171 FWAAAA DFEAAA HHHHxx +1 2838 1 1 1 1 1 1 1 1 1 2 3 BAAAAA EFEAAA OOOOxx +5937 2839 1 1 7 17 37 937 1937 937 5937 74 75 JUAAAA FFEAAA VVVVxx +7594 2840 0 2 4 14 94 594 1594 2594 7594 188 189 CGAAAA GFEAAA AAAAxx +3783 2841 1 3 3 3 83 783 1783 3783 3783 166 167 NPAAAA HFEAAA HHHHxx +6841 2842 1 1 1 1 41 841 841 1841 6841 82 83 DDAAAA IFEAAA OOOOxx +9694 2843 0 2 4 14 94 694 1694 4694 9694 188 189 WIAAAA JFEAAA VVVVxx +4322 2844 0 2 2 2 22 322 322 4322 4322 44 45 GKAAAA KFEAAA AAAAxx +6012 2845 0 0 2 12 12 12 12 1012 6012 24 25 GXAAAA LFEAAA HHHHxx +108 2846 0 0 8 8 8 108 108 108 108 16 17 EEAAAA MFEAAA OOOOxx +3396 2847 0 0 6 16 96 396 1396 3396 3396 192 193 QAAAAA NFEAAA VVVVxx +8643 2848 1 3 3 3 43 643 643 3643 8643 86 87 LUAAAA OFEAAA AAAAxx +6087 2849 1 3 7 7 87 87 87 1087 6087 174 175 DAAAAA PFEAAA HHHHxx +2629 2850 1 1 9 9 29 629 629 2629 2629 58 59 DXAAAA QFEAAA OOOOxx +3009 2851 1 1 9 9 9 9 1009 3009 3009 18 19 TLAAAA RFEAAA VVVVxx +438 2852 0 2 8 18 38 438 438 438 438 76 77 WQAAAA SFEAAA AAAAxx +2480 2853 0 0 0 0 80 480 480 2480 2480 160 161 KRAAAA TFEAAA HHHHxx +936 2854 0 0 6 16 36 936 936 936 936 72 73 AKAAAA UFEAAA OOOOxx +6 2855 0 2 6 6 6 6 6 6 6 12 13 GAAAAA VFEAAA VVVVxx +768 2856 0 0 8 8 68 768 768 768 768 136 137 ODAAAA WFEAAA AAAAxx +1564 2857 0 0 4 4 64 564 1564 1564 1564 128 129 EIAAAA XFEAAA HHHHxx +3236 2858 0 0 6 16 36 236 1236 3236 3236 72 73 MUAAAA YFEAAA OOOOxx +3932 2859 0 0 2 12 32 932 1932 3932 3932 64 65 GVAAAA ZFEAAA VVVVxx +8914 2860 0 2 4 14 14 914 914 3914 8914 28 29 WEAAAA AGEAAA AAAAxx +119 2861 1 3 9 19 19 119 119 119 119 38 39 PEAAAA BGEAAA HHHHxx +6034 2862 0 2 4 14 34 34 34 1034 6034 68 69 CYAAAA CGEAAA OOOOxx +5384 2863 0 0 4 4 84 384 1384 384 5384 168 169 CZAAAA DGEAAA VVVVxx +6885 2864 1 1 5 5 85 885 885 1885 6885 170 171 VEAAAA EGEAAA AAAAxx +232 2865 0 0 2 12 32 232 232 232 232 64 65 YIAAAA FGEAAA HHHHxx +1293 2866 1 1 3 13 93 293 1293 1293 1293 186 187 TXAAAA GGEAAA OOOOxx +9204 2867 0 0 4 4 4 204 1204 4204 9204 8 9 AQAAAA HGEAAA VVVVxx +527 2868 1 3 7 7 27 527 527 527 527 54 55 HUAAAA IGEAAA AAAAxx +6539 2869 1 3 9 19 39 539 539 1539 6539 78 79 NRAAAA JGEAAA HHHHxx +3679 2870 1 3 9 19 79 679 1679 3679 3679 158 159 NLAAAA KGEAAA OOOOxx +8282 2871 0 2 2 2 82 282 282 3282 8282 164 165 OGAAAA LGEAAA VVVVxx +5027 2872 1 3 7 7 27 27 1027 27 5027 54 55 JLAAAA MGEAAA AAAAxx +7694 2873 0 2 4 14 94 694 1694 2694 7694 188 189 YJAAAA NGEAAA HHHHxx +473 2874 1 1 3 13 73 473 473 473 473 146 147 FSAAAA OGEAAA OOOOxx +6325 2875 1 1 5 5 25 325 325 1325 6325 50 51 HJAAAA PGEAAA VVVVxx +8761 2876 1 1 1 1 61 761 761 3761 8761 122 123 ZYAAAA QGEAAA AAAAxx +6184 2877 0 0 4 4 84 184 184 1184 6184 168 169 WDAAAA RGEAAA HHHHxx +419 2878 1 3 9 19 19 419 419 419 419 38 39 DQAAAA SGEAAA OOOOxx +6111 2879 1 3 1 11 11 111 111 1111 6111 22 23 BBAAAA TGEAAA VVVVxx +3836 2880 0 0 6 16 36 836 1836 3836 3836 72 73 ORAAAA UGEAAA AAAAxx +4086 2881 0 2 6 6 86 86 86 4086 4086 172 173 EBAAAA VGEAAA HHHHxx +5818 2882 0 2 8 18 18 818 1818 818 5818 36 37 UPAAAA WGEAAA OOOOxx +4528 2883 0 0 8 8 28 528 528 4528 4528 56 57 ESAAAA XGEAAA VVVVxx +7199 2884 1 3 9 19 99 199 1199 2199 7199 198 199 XQAAAA YGEAAA AAAAxx +1847 2885 1 3 7 7 47 847 1847 1847 1847 94 95 BTAAAA ZGEAAA HHHHxx +2875 2886 1 3 5 15 75 875 875 2875 2875 150 151 PGAAAA AHEAAA OOOOxx +2872 2887 0 0 2 12 72 872 872 2872 2872 144 145 MGAAAA BHEAAA VVVVxx +3972 2888 0 0 2 12 72 972 1972 3972 3972 144 145 UWAAAA CHEAAA AAAAxx +7590 2889 0 2 0 10 90 590 1590 2590 7590 180 181 YFAAAA DHEAAA HHHHxx +1914 2890 0 2 4 14 14 914 1914 1914 1914 28 29 QVAAAA EHEAAA OOOOxx +1658 2891 0 2 8 18 58 658 1658 1658 1658 116 117 ULAAAA FHEAAA VVVVxx +2126 2892 0 2 6 6 26 126 126 2126 2126 52 53 UDAAAA GHEAAA AAAAxx +645 2893 1 1 5 5 45 645 645 645 645 90 91 VYAAAA HHEAAA HHHHxx +6636 2894 0 0 6 16 36 636 636 1636 6636 72 73 GVAAAA IHEAAA OOOOxx +1469 2895 1 1 9 9 69 469 1469 1469 1469 138 139 NEAAAA JHEAAA VVVVxx +1377 2896 1 1 7 17 77 377 1377 1377 1377 154 155 ZAAAAA KHEAAA AAAAxx +8425 2897 1 1 5 5 25 425 425 3425 8425 50 51 BMAAAA LHEAAA HHHHxx +9300 2898 0 0 0 0 0 300 1300 4300 9300 0 1 STAAAA MHEAAA OOOOxx +5355 2899 1 3 5 15 55 355 1355 355 5355 110 111 ZXAAAA NHEAAA VVVVxx +840 2900 0 0 0 0 40 840 840 840 840 80 81 IGAAAA OHEAAA AAAAxx +5185 2901 1 1 5 5 85 185 1185 185 5185 170 171 LRAAAA PHEAAA HHHHxx +6467 2902 1 3 7 7 67 467 467 1467 6467 134 135 TOAAAA QHEAAA OOOOxx +58 2903 0 2 8 18 58 58 58 58 58 116 117 GCAAAA RHEAAA VVVVxx +5051 2904 1 3 1 11 51 51 1051 51 5051 102 103 HMAAAA SHEAAA AAAAxx +8901 2905 1 1 1 1 1 901 901 3901 8901 2 3 JEAAAA THEAAA HHHHxx +1550 2906 0 2 0 10 50 550 1550 1550 1550 100 101 QHAAAA UHEAAA OOOOxx +1698 2907 0 2 8 18 98 698 1698 1698 1698 196 197 INAAAA VHEAAA VVVVxx +802 2908 0 2 2 2 2 802 802 802 802 4 5 WEAAAA WHEAAA AAAAxx +2440 2909 0 0 0 0 40 440 440 2440 2440 80 81 WPAAAA XHEAAA HHHHxx +2260 2910 0 0 0 0 60 260 260 2260 2260 120 121 YIAAAA YHEAAA OOOOxx +8218 2911 0 2 8 18 18 218 218 3218 8218 36 37 CEAAAA ZHEAAA VVVVxx +5144 2912 0 0 4 4 44 144 1144 144 5144 88 89 WPAAAA AIEAAA AAAAxx +4822 2913 0 2 2 2 22 822 822 4822 4822 44 45 MDAAAA BIEAAA HHHHxx +9476 2914 0 0 6 16 76 476 1476 4476 9476 152 153 MAAAAA CIEAAA OOOOxx +7535 2915 1 3 5 15 35 535 1535 2535 7535 70 71 VDAAAA DIEAAA VVVVxx +8738 2916 0 2 8 18 38 738 738 3738 8738 76 77 CYAAAA EIEAAA AAAAxx +7946 2917 0 2 6 6 46 946 1946 2946 7946 92 93 QTAAAA FIEAAA HHHHxx +8143 2918 1 3 3 3 43 143 143 3143 8143 86 87 FBAAAA GIEAAA OOOOxx +2623 2919 1 3 3 3 23 623 623 2623 2623 46 47 XWAAAA HIEAAA VVVVxx +5209 2920 1 1 9 9 9 209 1209 209 5209 18 19 JSAAAA IIEAAA AAAAxx +7674 2921 0 2 4 14 74 674 1674 2674 7674 148 149 EJAAAA JIEAAA HHHHxx +1135 2922 1 3 5 15 35 135 1135 1135 1135 70 71 RRAAAA KIEAAA OOOOxx +424 2923 0 0 4 4 24 424 424 424 424 48 49 IQAAAA LIEAAA VVVVxx +942 2924 0 2 2 2 42 942 942 942 942 84 85 GKAAAA MIEAAA AAAAxx +7813 2925 1 1 3 13 13 813 1813 2813 7813 26 27 NOAAAA NIEAAA HHHHxx +3539 2926 1 3 9 19 39 539 1539 3539 3539 78 79 DGAAAA OIEAAA OOOOxx +2909 2927 1 1 9 9 9 909 909 2909 2909 18 19 XHAAAA PIEAAA VVVVxx +3748 2928 0 0 8 8 48 748 1748 3748 3748 96 97 EOAAAA QIEAAA AAAAxx +2996 2929 0 0 6 16 96 996 996 2996 2996 192 193 GLAAAA RIEAAA HHHHxx +1869 2930 1 1 9 9 69 869 1869 1869 1869 138 139 XTAAAA SIEAAA OOOOxx +8151 2931 1 3 1 11 51 151 151 3151 8151 102 103 NBAAAA TIEAAA VVVVxx +6361 2932 1 1 1 1 61 361 361 1361 6361 122 123 RKAAAA UIEAAA AAAAxx +5568 2933 0 0 8 8 68 568 1568 568 5568 136 137 EGAAAA VIEAAA HHHHxx +2796 2934 0 0 6 16 96 796 796 2796 2796 192 193 ODAAAA WIEAAA OOOOxx +8489 2935 1 1 9 9 89 489 489 3489 8489 178 179 NOAAAA XIEAAA VVVVxx +9183 2936 1 3 3 3 83 183 1183 4183 9183 166 167 FPAAAA YIEAAA AAAAxx +8227 2937 1 3 7 7 27 227 227 3227 8227 54 55 LEAAAA ZIEAAA HHHHxx +1844 2938 0 0 4 4 44 844 1844 1844 1844 88 89 YSAAAA AJEAAA OOOOxx +3975 2939 1 3 5 15 75 975 1975 3975 3975 150 151 XWAAAA BJEAAA VVVVxx +6490 2940 0 2 0 10 90 490 490 1490 6490 180 181 QPAAAA CJEAAA AAAAxx +8303 2941 1 3 3 3 3 303 303 3303 8303 6 7 JHAAAA DJEAAA HHHHxx +7334 2942 0 2 4 14 34 334 1334 2334 7334 68 69 CWAAAA EJEAAA OOOOxx +2382 2943 0 2 2 2 82 382 382 2382 2382 164 165 QNAAAA FJEAAA VVVVxx +177 2944 1 1 7 17 77 177 177 177 177 154 155 VGAAAA GJEAAA AAAAxx +8117 2945 1 1 7 17 17 117 117 3117 8117 34 35 FAAAAA HJEAAA HHHHxx +5485 2946 1 1 5 5 85 485 1485 485 5485 170 171 ZCAAAA IJEAAA OOOOxx +6544 2947 0 0 4 4 44 544 544 1544 6544 88 89 SRAAAA JJEAAA VVVVxx +8517 2948 1 1 7 17 17 517 517 3517 8517 34 35 PPAAAA KJEAAA AAAAxx +2252 2949 0 0 2 12 52 252 252 2252 2252 104 105 QIAAAA LJEAAA HHHHxx +4480 2950 0 0 0 0 80 480 480 4480 4480 160 161 IQAAAA MJEAAA OOOOxx +4785 2951 1 1 5 5 85 785 785 4785 4785 170 171 BCAAAA NJEAAA VVVVxx +9700 2952 0 0 0 0 0 700 1700 4700 9700 0 1 CJAAAA OJEAAA AAAAxx +2122 2953 0 2 2 2 22 122 122 2122 2122 44 45 QDAAAA PJEAAA HHHHxx +8783 2954 1 3 3 3 83 783 783 3783 8783 166 167 VZAAAA QJEAAA OOOOxx +1453 2955 1 1 3 13 53 453 1453 1453 1453 106 107 XDAAAA RJEAAA VVVVxx +3908 2956 0 0 8 8 8 908 1908 3908 3908 16 17 IUAAAA SJEAAA AAAAxx +7707 2957 1 3 7 7 7 707 1707 2707 7707 14 15 LKAAAA TJEAAA HHHHxx +9049 2958 1 1 9 9 49 49 1049 4049 9049 98 99 BKAAAA UJEAAA OOOOxx +654 2959 0 2 4 14 54 654 654 654 654 108 109 EZAAAA VJEAAA VVVVxx +3336 2960 0 0 6 16 36 336 1336 3336 3336 72 73 IYAAAA WJEAAA AAAAxx +622 2961 0 2 2 2 22 622 622 622 622 44 45 YXAAAA XJEAAA HHHHxx +8398 2962 0 2 8 18 98 398 398 3398 8398 196 197 ALAAAA YJEAAA OOOOxx +9193 2963 1 1 3 13 93 193 1193 4193 9193 186 187 PPAAAA ZJEAAA VVVVxx +7896 2964 0 0 6 16 96 896 1896 2896 7896 192 193 SRAAAA AKEAAA AAAAxx +9798 2965 0 2 8 18 98 798 1798 4798 9798 196 197 WMAAAA BKEAAA HHHHxx +2881 2966 1 1 1 1 81 881 881 2881 2881 162 163 VGAAAA CKEAAA OOOOxx +672 2967 0 0 2 12 72 672 672 672 672 144 145 WZAAAA DKEAAA VVVVxx +6743 2968 1 3 3 3 43 743 743 1743 6743 86 87 JZAAAA EKEAAA AAAAxx +8935 2969 1 3 5 15 35 935 935 3935 8935 70 71 RFAAAA FKEAAA HHHHxx +2426 2970 0 2 6 6 26 426 426 2426 2426 52 53 IPAAAA GKEAAA OOOOxx +722 2971 0 2 2 2 22 722 722 722 722 44 45 UBAAAA HKEAAA VVVVxx +5088 2972 0 0 8 8 88 88 1088 88 5088 176 177 SNAAAA IKEAAA AAAAxx +8677 2973 1 1 7 17 77 677 677 3677 8677 154 155 TVAAAA JKEAAA HHHHxx +6963 2974 1 3 3 3 63 963 963 1963 6963 126 127 VHAAAA KKEAAA OOOOxx +1653 2975 1 1 3 13 53 653 1653 1653 1653 106 107 PLAAAA LKEAAA VVVVxx +7295 2976 1 3 5 15 95 295 1295 2295 7295 190 191 PUAAAA MKEAAA AAAAxx +6675 2977 1 3 5 15 75 675 675 1675 6675 150 151 TWAAAA NKEAAA HHHHxx +7183 2978 1 3 3 3 83 183 1183 2183 7183 166 167 HQAAAA OKEAAA OOOOxx +4378 2979 0 2 8 18 78 378 378 4378 4378 156 157 KMAAAA PKEAAA VVVVxx +2157 2980 1 1 7 17 57 157 157 2157 2157 114 115 ZEAAAA QKEAAA AAAAxx +2621 2981 1 1 1 1 21 621 621 2621 2621 42 43 VWAAAA RKEAAA HHHHxx +9278 2982 0 2 8 18 78 278 1278 4278 9278 156 157 WSAAAA SKEAAA OOOOxx +79 2983 1 3 9 19 79 79 79 79 79 158 159 BDAAAA TKEAAA VVVVxx +7358 2984 0 2 8 18 58 358 1358 2358 7358 116 117 AXAAAA UKEAAA AAAAxx +3589 2985 1 1 9 9 89 589 1589 3589 3589 178 179 BIAAAA VKEAAA HHHHxx +1254 2986 0 2 4 14 54 254 1254 1254 1254 108 109 GWAAAA WKEAAA OOOOxx +3490 2987 0 2 0 10 90 490 1490 3490 3490 180 181 GEAAAA XKEAAA VVVVxx +7533 2988 1 1 3 13 33 533 1533 2533 7533 66 67 TDAAAA YKEAAA AAAAxx +2800 2989 0 0 0 0 0 800 800 2800 2800 0 1 SDAAAA ZKEAAA HHHHxx +351 2990 1 3 1 11 51 351 351 351 351 102 103 NNAAAA ALEAAA OOOOxx +4359 2991 1 3 9 19 59 359 359 4359 4359 118 119 RLAAAA BLEAAA VVVVxx +5788 2992 0 0 8 8 88 788 1788 788 5788 176 177 QOAAAA CLEAAA AAAAxx +5521 2993 1 1 1 1 21 521 1521 521 5521 42 43 JEAAAA DLEAAA HHHHxx +3351 2994 1 3 1 11 51 351 1351 3351 3351 102 103 XYAAAA ELEAAA OOOOxx +5129 2995 1 1 9 9 29 129 1129 129 5129 58 59 HPAAAA FLEAAA VVVVxx +315 2996 1 3 5 15 15 315 315 315 315 30 31 DMAAAA GLEAAA AAAAxx +7552 2997 0 0 2 12 52 552 1552 2552 7552 104 105 MEAAAA HLEAAA HHHHxx +9176 2998 0 0 6 16 76 176 1176 4176 9176 152 153 YOAAAA ILEAAA OOOOxx +7458 2999 0 2 8 18 58 458 1458 2458 7458 116 117 WAAAAA JLEAAA VVVVxx +279 3000 1 3 9 19 79 279 279 279 279 158 159 TKAAAA KLEAAA AAAAxx +738 3001 0 2 8 18 38 738 738 738 738 76 77 KCAAAA LLEAAA HHHHxx +2557 3002 1 1 7 17 57 557 557 2557 2557 114 115 JUAAAA MLEAAA OOOOxx +9395 3003 1 3 5 15 95 395 1395 4395 9395 190 191 JXAAAA NLEAAA VVVVxx +7214 3004 0 2 4 14 14 214 1214 2214 7214 28 29 MRAAAA OLEAAA AAAAxx +6354 3005 0 2 4 14 54 354 354 1354 6354 108 109 KKAAAA PLEAAA HHHHxx +4799 3006 1 3 9 19 99 799 799 4799 4799 198 199 PCAAAA QLEAAA OOOOxx +1231 3007 1 3 1 11 31 231 1231 1231 1231 62 63 JVAAAA RLEAAA VVVVxx +5252 3008 0 0 2 12 52 252 1252 252 5252 104 105 AUAAAA SLEAAA AAAAxx +5250 3009 0 2 0 10 50 250 1250 250 5250 100 101 YTAAAA TLEAAA HHHHxx +9319 3010 1 3 9 19 19 319 1319 4319 9319 38 39 LUAAAA ULEAAA OOOOxx +1724 3011 0 0 4 4 24 724 1724 1724 1724 48 49 IOAAAA VLEAAA VVVVxx +7947 3012 1 3 7 7 47 947 1947 2947 7947 94 95 RTAAAA WLEAAA AAAAxx +1105 3013 1 1 5 5 5 105 1105 1105 1105 10 11 NQAAAA XLEAAA HHHHxx +1417 3014 1 1 7 17 17 417 1417 1417 1417 34 35 NCAAAA YLEAAA OOOOxx +7101 3015 1 1 1 1 1 101 1101 2101 7101 2 3 DNAAAA ZLEAAA VVVVxx +1088 3016 0 0 8 8 88 88 1088 1088 1088 176 177 WPAAAA AMEAAA AAAAxx +979 3017 1 3 9 19 79 979 979 979 979 158 159 RLAAAA BMEAAA HHHHxx +7589 3018 1 1 9 9 89 589 1589 2589 7589 178 179 XFAAAA CMEAAA OOOOxx +8952 3019 0 0 2 12 52 952 952 3952 8952 104 105 IGAAAA DMEAAA VVVVxx +2864 3020 0 0 4 4 64 864 864 2864 2864 128 129 EGAAAA EMEAAA AAAAxx +234 3021 0 2 4 14 34 234 234 234 234 68 69 AJAAAA FMEAAA HHHHxx +7231 3022 1 3 1 11 31 231 1231 2231 7231 62 63 DSAAAA GMEAAA OOOOxx +6792 3023 0 0 2 12 92 792 792 1792 6792 184 185 GBAAAA HMEAAA VVVVxx +4311 3024 1 3 1 11 11 311 311 4311 4311 22 23 VJAAAA IMEAAA AAAAxx +3374 3025 0 2 4 14 74 374 1374 3374 3374 148 149 UZAAAA JMEAAA HHHHxx +3367 3026 1 3 7 7 67 367 1367 3367 3367 134 135 NZAAAA KMEAAA OOOOxx +2598 3027 0 2 8 18 98 598 598 2598 2598 196 197 YVAAAA LMEAAA VVVVxx +1033 3028 1 1 3 13 33 33 1033 1033 1033 66 67 TNAAAA MMEAAA AAAAxx +7803 3029 1 3 3 3 3 803 1803 2803 7803 6 7 DOAAAA NMEAAA HHHHxx +3870 3030 0 2 0 10 70 870 1870 3870 3870 140 141 WSAAAA OMEAAA OOOOxx +4962 3031 0 2 2 2 62 962 962 4962 4962 124 125 WIAAAA PMEAAA VVVVxx +4842 3032 0 2 2 2 42 842 842 4842 4842 84 85 GEAAAA QMEAAA AAAAxx +8814 3033 0 2 4 14 14 814 814 3814 8814 28 29 ABAAAA RMEAAA HHHHxx +3429 3034 1 1 9 9 29 429 1429 3429 3429 58 59 XBAAAA SMEAAA OOOOxx +6550 3035 0 2 0 10 50 550 550 1550 6550 100 101 YRAAAA TMEAAA VVVVxx +6317 3036 1 1 7 17 17 317 317 1317 6317 34 35 ZIAAAA UMEAAA AAAAxx +5023 3037 1 3 3 3 23 23 1023 23 5023 46 47 FLAAAA VMEAAA HHHHxx +5825 3038 1 1 5 5 25 825 1825 825 5825 50 51 BQAAAA WMEAAA OOOOxx +5297 3039 1 1 7 17 97 297 1297 297 5297 194 195 TVAAAA XMEAAA VVVVxx +8764 3040 0 0 4 4 64 764 764 3764 8764 128 129 CZAAAA YMEAAA AAAAxx +5084 3041 0 0 4 4 84 84 1084 84 5084 168 169 ONAAAA ZMEAAA HHHHxx +6808 3042 0 0 8 8 8 808 808 1808 6808 16 17 WBAAAA ANEAAA OOOOxx +1780 3043 0 0 0 0 80 780 1780 1780 1780 160 161 MQAAAA BNEAAA VVVVxx +4092 3044 0 0 2 12 92 92 92 4092 4092 184 185 KBAAAA CNEAAA AAAAxx +3618 3045 0 2 8 18 18 618 1618 3618 3618 36 37 EJAAAA DNEAAA HHHHxx +7299 3046 1 3 9 19 99 299 1299 2299 7299 198 199 TUAAAA ENEAAA OOOOxx +8544 3047 0 0 4 4 44 544 544 3544 8544 88 89 QQAAAA FNEAAA VVVVxx +2359 3048 1 3 9 19 59 359 359 2359 2359 118 119 TMAAAA GNEAAA AAAAxx +1939 3049 1 3 9 19 39 939 1939 1939 1939 78 79 PWAAAA HNEAAA HHHHxx +5834 3050 0 2 4 14 34 834 1834 834 5834 68 69 KQAAAA INEAAA OOOOxx +1997 3051 1 1 7 17 97 997 1997 1997 1997 194 195 VYAAAA JNEAAA VVVVxx +7917 3052 1 1 7 17 17 917 1917 2917 7917 34 35 NSAAAA KNEAAA AAAAxx +2098 3053 0 2 8 18 98 98 98 2098 2098 196 197 SCAAAA LNEAAA HHHHxx +7576 3054 0 0 6 16 76 576 1576 2576 7576 152 153 KFAAAA MNEAAA OOOOxx +376 3055 0 0 6 16 76 376 376 376 376 152 153 MOAAAA NNEAAA VVVVxx +8535 3056 1 3 5 15 35 535 535 3535 8535 70 71 HQAAAA ONEAAA AAAAxx +5659 3057 1 3 9 19 59 659 1659 659 5659 118 119 RJAAAA PNEAAA HHHHxx +2786 3058 0 2 6 6 86 786 786 2786 2786 172 173 EDAAAA QNEAAA OOOOxx +8820 3059 0 0 0 0 20 820 820 3820 8820 40 41 GBAAAA RNEAAA VVVVxx +1229 3060 1 1 9 9 29 229 1229 1229 1229 58 59 HVAAAA SNEAAA AAAAxx +9321 3061 1 1 1 1 21 321 1321 4321 9321 42 43 NUAAAA TNEAAA HHHHxx +7662 3062 0 2 2 2 62 662 1662 2662 7662 124 125 SIAAAA UNEAAA OOOOxx +5535 3063 1 3 5 15 35 535 1535 535 5535 70 71 XEAAAA VNEAAA VVVVxx +4889 3064 1 1 9 9 89 889 889 4889 4889 178 179 BGAAAA WNEAAA AAAAxx +8259 3065 1 3 9 19 59 259 259 3259 8259 118 119 RFAAAA XNEAAA HHHHxx +6789 3066 1 1 9 9 89 789 789 1789 6789 178 179 DBAAAA YNEAAA OOOOxx +5411 3067 1 3 1 11 11 411 1411 411 5411 22 23 DAAAAA ZNEAAA VVVVxx +6992 3068 0 0 2 12 92 992 992 1992 6992 184 185 YIAAAA AOEAAA AAAAxx +7698 3069 0 2 8 18 98 698 1698 2698 7698 196 197 CKAAAA BOEAAA HHHHxx +2342 3070 0 2 2 2 42 342 342 2342 2342 84 85 CMAAAA COEAAA OOOOxx +1501 3071 1 1 1 1 1 501 1501 1501 1501 2 3 TFAAAA DOEAAA VVVVxx +6322 3072 0 2 2 2 22 322 322 1322 6322 44 45 EJAAAA EOEAAA AAAAxx +9861 3073 1 1 1 1 61 861 1861 4861 9861 122 123 HPAAAA FOEAAA HHHHxx +9802 3074 0 2 2 2 2 802 1802 4802 9802 4 5 ANAAAA GOEAAA OOOOxx +4750 3075 0 2 0 10 50 750 750 4750 4750 100 101 SAAAAA HOEAAA VVVVxx +5855 3076 1 3 5 15 55 855 1855 855 5855 110 111 FRAAAA IOEAAA AAAAxx +4304 3077 0 0 4 4 4 304 304 4304 4304 8 9 OJAAAA JOEAAA HHHHxx +2605 3078 1 1 5 5 5 605 605 2605 2605 10 11 FWAAAA KOEAAA OOOOxx +1802 3079 0 2 2 2 2 802 1802 1802 1802 4 5 IRAAAA LOEAAA VVVVxx +9368 3080 0 0 8 8 68 368 1368 4368 9368 136 137 IWAAAA MOEAAA AAAAxx +7107 3081 1 3 7 7 7 107 1107 2107 7107 14 15 JNAAAA NOEAAA HHHHxx +8895 3082 1 3 5 15 95 895 895 3895 8895 190 191 DEAAAA OOEAAA OOOOxx +3750 3083 0 2 0 10 50 750 1750 3750 3750 100 101 GOAAAA POEAAA VVVVxx +8934 3084 0 2 4 14 34 934 934 3934 8934 68 69 QFAAAA QOEAAA AAAAxx +9464 3085 0 0 4 4 64 464 1464 4464 9464 128 129 AAAAAA ROEAAA HHHHxx +1928 3086 0 0 8 8 28 928 1928 1928 1928 56 57 EWAAAA SOEAAA OOOOxx +3196 3087 0 0 6 16 96 196 1196 3196 3196 192 193 YSAAAA TOEAAA VVVVxx +5256 3088 0 0 6 16 56 256 1256 256 5256 112 113 EUAAAA UOEAAA AAAAxx +7119 3089 1 3 9 19 19 119 1119 2119 7119 38 39 VNAAAA VOEAAA HHHHxx +4495 3090 1 3 5 15 95 495 495 4495 4495 190 191 XQAAAA WOEAAA OOOOxx +9292 3091 0 0 2 12 92 292 1292 4292 9292 184 185 KTAAAA XOEAAA VVVVxx +1617 3092 1 1 7 17 17 617 1617 1617 1617 34 35 FKAAAA YOEAAA AAAAxx +481 3093 1 1 1 1 81 481 481 481 481 162 163 NSAAAA ZOEAAA HHHHxx +56 3094 0 0 6 16 56 56 56 56 56 112 113 ECAAAA APEAAA OOOOxx +9120 3095 0 0 0 0 20 120 1120 4120 9120 40 41 UMAAAA BPEAAA VVVVxx +1306 3096 0 2 6 6 6 306 1306 1306 1306 12 13 GYAAAA CPEAAA AAAAxx +7773 3097 1 1 3 13 73 773 1773 2773 7773 146 147 ZMAAAA DPEAAA HHHHxx +4863 3098 1 3 3 3 63 863 863 4863 4863 126 127 BFAAAA EPEAAA OOOOxx +1114 3099 0 2 4 14 14 114 1114 1114 1114 28 29 WQAAAA FPEAAA VVVVxx +8124 3100 0 0 4 4 24 124 124 3124 8124 48 49 MAAAAA GPEAAA AAAAxx +6254 3101 0 2 4 14 54 254 254 1254 6254 108 109 OGAAAA HPEAAA HHHHxx +8109 3102 1 1 9 9 9 109 109 3109 8109 18 19 XZAAAA IPEAAA OOOOxx +1747 3103 1 3 7 7 47 747 1747 1747 1747 94 95 FPAAAA JPEAAA VVVVxx +6185 3104 1 1 5 5 85 185 185 1185 6185 170 171 XDAAAA KPEAAA AAAAxx +3388 3105 0 0 8 8 88 388 1388 3388 3388 176 177 IAAAAA LPEAAA HHHHxx +4905 3106 1 1 5 5 5 905 905 4905 4905 10 11 RGAAAA MPEAAA OOOOxx +5728 3107 0 0 8 8 28 728 1728 728 5728 56 57 IMAAAA NPEAAA VVVVxx +7507 3108 1 3 7 7 7 507 1507 2507 7507 14 15 TCAAAA OPEAAA AAAAxx +5662 3109 0 2 2 2 62 662 1662 662 5662 124 125 UJAAAA PPEAAA HHHHxx +1686 3110 0 2 6 6 86 686 1686 1686 1686 172 173 WMAAAA QPEAAA OOOOxx +5202 3111 0 2 2 2 2 202 1202 202 5202 4 5 CSAAAA RPEAAA VVVVxx +6905 3112 1 1 5 5 5 905 905 1905 6905 10 11 PFAAAA SPEAAA AAAAxx +9577 3113 1 1 7 17 77 577 1577 4577 9577 154 155 JEAAAA TPEAAA HHHHxx +7194 3114 0 2 4 14 94 194 1194 2194 7194 188 189 SQAAAA UPEAAA OOOOxx +7016 3115 0 0 6 16 16 16 1016 2016 7016 32 33 WJAAAA VPEAAA VVVVxx +8905 3116 1 1 5 5 5 905 905 3905 8905 10 11 NEAAAA WPEAAA AAAAxx +3419 3117 1 3 9 19 19 419 1419 3419 3419 38 39 NBAAAA XPEAAA HHHHxx +6881 3118 1 1 1 1 81 881 881 1881 6881 162 163 REAAAA YPEAAA OOOOxx +8370 3119 0 2 0 10 70 370 370 3370 8370 140 141 YJAAAA ZPEAAA VVVVxx +6117 3120 1 1 7 17 17 117 117 1117 6117 34 35 HBAAAA AQEAAA AAAAxx +1636 3121 0 0 6 16 36 636 1636 1636 1636 72 73 YKAAAA BQEAAA HHHHxx +6857 3122 1 1 7 17 57 857 857 1857 6857 114 115 TDAAAA CQEAAA OOOOxx +7163 3123 1 3 3 3 63 163 1163 2163 7163 126 127 NPAAAA DQEAAA VVVVxx +5040 3124 0 0 0 0 40 40 1040 40 5040 80 81 WLAAAA EQEAAA AAAAxx +6263 3125 1 3 3 3 63 263 263 1263 6263 126 127 XGAAAA FQEAAA HHHHxx +4809 3126 1 1 9 9 9 809 809 4809 4809 18 19 ZCAAAA GQEAAA OOOOxx +900 3127 0 0 0 0 0 900 900 900 900 0 1 QIAAAA HQEAAA VVVVxx +3199 3128 1 3 9 19 99 199 1199 3199 3199 198 199 BTAAAA IQEAAA AAAAxx +4156 3129 0 0 6 16 56 156 156 4156 4156 112 113 WDAAAA JQEAAA HHHHxx +3501 3130 1 1 1 1 1 501 1501 3501 3501 2 3 REAAAA KQEAAA OOOOxx +164 3131 0 0 4 4 64 164 164 164 164 128 129 IGAAAA LQEAAA VVVVxx +9548 3132 0 0 8 8 48 548 1548 4548 9548 96 97 GDAAAA MQEAAA AAAAxx +1149 3133 1 1 9 9 49 149 1149 1149 1149 98 99 FSAAAA NQEAAA HHHHxx +1962 3134 0 2 2 2 62 962 1962 1962 1962 124 125 MXAAAA OQEAAA OOOOxx +4072 3135 0 0 2 12 72 72 72 4072 4072 144 145 QAAAAA PQEAAA VVVVxx +4280 3136 0 0 0 0 80 280 280 4280 4280 160 161 QIAAAA QQEAAA AAAAxx +1398 3137 0 2 8 18 98 398 1398 1398 1398 196 197 UBAAAA RQEAAA HHHHxx +725 3138 1 1 5 5 25 725 725 725 725 50 51 XBAAAA SQEAAA OOOOxx +3988 3139 0 0 8 8 88 988 1988 3988 3988 176 177 KXAAAA TQEAAA VVVVxx +5059 3140 1 3 9 19 59 59 1059 59 5059 118 119 PMAAAA UQEAAA AAAAxx +2632 3141 0 0 2 12 32 632 632 2632 2632 64 65 GXAAAA VQEAAA HHHHxx +1909 3142 1 1 9 9 9 909 1909 1909 1909 18 19 LVAAAA WQEAAA OOOOxx +6827 3143 1 3 7 7 27 827 827 1827 6827 54 55 PCAAAA XQEAAA VVVVxx +8156 3144 0 0 6 16 56 156 156 3156 8156 112 113 SBAAAA YQEAAA AAAAxx +1192 3145 0 0 2 12 92 192 1192 1192 1192 184 185 WTAAAA ZQEAAA HHHHxx +9545 3146 1 1 5 5 45 545 1545 4545 9545 90 91 DDAAAA AREAAA OOOOxx +2249 3147 1 1 9 9 49 249 249 2249 2249 98 99 NIAAAA BREAAA VVVVxx +5580 3148 0 0 0 0 80 580 1580 580 5580 160 161 QGAAAA CREAAA AAAAxx +8403 3149 1 3 3 3 3 403 403 3403 8403 6 7 FLAAAA DREAAA HHHHxx +4024 3150 0 0 4 4 24 24 24 4024 4024 48 49 UYAAAA EREAAA OOOOxx +1866 3151 0 2 6 6 66 866 1866 1866 1866 132 133 UTAAAA FREAAA VVVVxx +9251 3152 1 3 1 11 51 251 1251 4251 9251 102 103 VRAAAA GREAAA AAAAxx +9979 3153 1 3 9 19 79 979 1979 4979 9979 158 159 VTAAAA HREAAA HHHHxx +9899 3154 1 3 9 19 99 899 1899 4899 9899 198 199 TQAAAA IREAAA OOOOxx +2540 3155 0 0 0 0 40 540 540 2540 2540 80 81 STAAAA JREAAA VVVVxx +8957 3156 1 1 7 17 57 957 957 3957 8957 114 115 NGAAAA KREAAA AAAAxx +7702 3157 0 2 2 2 2 702 1702 2702 7702 4 5 GKAAAA LREAAA HHHHxx +4211 3158 1 3 1 11 11 211 211 4211 4211 22 23 ZFAAAA MREAAA OOOOxx +6684 3159 0 0 4 4 84 684 684 1684 6684 168 169 CXAAAA NREAAA VVVVxx +3883 3160 1 3 3 3 83 883 1883 3883 3883 166 167 JTAAAA OREAAA AAAAxx +3531 3161 1 3 1 11 31 531 1531 3531 3531 62 63 VFAAAA PREAAA HHHHxx +9178 3162 0 2 8 18 78 178 1178 4178 9178 156 157 APAAAA QREAAA OOOOxx +3389 3163 1 1 9 9 89 389 1389 3389 3389 178 179 JAAAAA RREAAA VVVVxx +7874 3164 0 2 4 14 74 874 1874 2874 7874 148 149 WQAAAA SREAAA AAAAxx +4522 3165 0 2 2 2 22 522 522 4522 4522 44 45 YRAAAA TREAAA HHHHxx +9399 3166 1 3 9 19 99 399 1399 4399 9399 198 199 NXAAAA UREAAA OOOOxx +9083 3167 1 3 3 3 83 83 1083 4083 9083 166 167 JLAAAA VREAAA VVVVxx +1530 3168 0 2 0 10 30 530 1530 1530 1530 60 61 WGAAAA WREAAA AAAAxx +2360 3169 0 0 0 0 60 360 360 2360 2360 120 121 UMAAAA XREAAA HHHHxx +4908 3170 0 0 8 8 8 908 908 4908 4908 16 17 UGAAAA YREAAA OOOOxx +4628 3171 0 0 8 8 28 628 628 4628 4628 56 57 AWAAAA ZREAAA VVVVxx +3889 3172 1 1 9 9 89 889 1889 3889 3889 178 179 PTAAAA ASEAAA AAAAxx +1331 3173 1 3 1 11 31 331 1331 1331 1331 62 63 FZAAAA BSEAAA HHHHxx +1942 3174 0 2 2 2 42 942 1942 1942 1942 84 85 SWAAAA CSEAAA OOOOxx +4734 3175 0 2 4 14 34 734 734 4734 4734 68 69 CAAAAA DSEAAA VVVVxx +8386 3176 0 2 6 6 86 386 386 3386 8386 172 173 OKAAAA ESEAAA AAAAxx +3586 3177 0 2 6 6 86 586 1586 3586 3586 172 173 YHAAAA FSEAAA HHHHxx +2354 3178 0 2 4 14 54 354 354 2354 2354 108 109 OMAAAA GSEAAA OOOOxx +7108 3179 0 0 8 8 8 108 1108 2108 7108 16 17 KNAAAA HSEAAA VVVVxx +1857 3180 1 1 7 17 57 857 1857 1857 1857 114 115 LTAAAA ISEAAA AAAAxx +2544 3181 0 0 4 4 44 544 544 2544 2544 88 89 WTAAAA JSEAAA HHHHxx +819 3182 1 3 9 19 19 819 819 819 819 38 39 NFAAAA KSEAAA OOOOxx +2878 3183 0 2 8 18 78 878 878 2878 2878 156 157 SGAAAA LSEAAA VVVVxx +1772 3184 0 0 2 12 72 772 1772 1772 1772 144 145 EQAAAA MSEAAA AAAAxx +354 3185 0 2 4 14 54 354 354 354 354 108 109 QNAAAA NSEAAA HHHHxx +3259 3186 1 3 9 19 59 259 1259 3259 3259 118 119 JVAAAA OSEAAA OOOOxx +2170 3187 0 2 0 10 70 170 170 2170 2170 140 141 MFAAAA PSEAAA VVVVxx +1190 3188 0 2 0 10 90 190 1190 1190 1190 180 181 UTAAAA QSEAAA AAAAxx +3607 3189 1 3 7 7 7 607 1607 3607 3607 14 15 TIAAAA RSEAAA HHHHxx +4661 3190 1 1 1 1 61 661 661 4661 4661 122 123 HXAAAA SSEAAA OOOOxx +1796 3191 0 0 6 16 96 796 1796 1796 1796 192 193 CRAAAA TSEAAA VVVVxx +1561 3192 1 1 1 1 61 561 1561 1561 1561 122 123 BIAAAA USEAAA AAAAxx +4336 3193 0 0 6 16 36 336 336 4336 4336 72 73 UKAAAA VSEAAA HHHHxx +7550 3194 0 2 0 10 50 550 1550 2550 7550 100 101 KEAAAA WSEAAA OOOOxx +3238 3195 0 2 8 18 38 238 1238 3238 3238 76 77 OUAAAA XSEAAA VVVVxx +9870 3196 0 2 0 10 70 870 1870 4870 9870 140 141 QPAAAA YSEAAA AAAAxx +6502 3197 0 2 2 2 2 502 502 1502 6502 4 5 CQAAAA ZSEAAA HHHHxx +3903 3198 1 3 3 3 3 903 1903 3903 3903 6 7 DUAAAA ATEAAA OOOOxx +2869 3199 1 1 9 9 69 869 869 2869 2869 138 139 JGAAAA BTEAAA VVVVxx +5072 3200 0 0 2 12 72 72 1072 72 5072 144 145 CNAAAA CTEAAA AAAAxx +1201 3201 1 1 1 1 1 201 1201 1201 1201 2 3 FUAAAA DTEAAA HHHHxx +6245 3202 1 1 5 5 45 245 245 1245 6245 90 91 FGAAAA ETEAAA OOOOxx +1402 3203 0 2 2 2 2 402 1402 1402 1402 4 5 YBAAAA FTEAAA VVVVxx +2594 3204 0 2 4 14 94 594 594 2594 2594 188 189 UVAAAA GTEAAA AAAAxx +9171 3205 1 3 1 11 71 171 1171 4171 9171 142 143 TOAAAA HTEAAA HHHHxx +2620 3206 0 0 0 0 20 620 620 2620 2620 40 41 UWAAAA ITEAAA OOOOxx +6309 3207 1 1 9 9 9 309 309 1309 6309 18 19 RIAAAA JTEAAA VVVVxx +1285 3208 1 1 5 5 85 285 1285 1285 1285 170 171 LXAAAA KTEAAA AAAAxx +5466 3209 0 2 6 6 66 466 1466 466 5466 132 133 GCAAAA LTEAAA HHHHxx +168 3210 0 0 8 8 68 168 168 168 168 136 137 MGAAAA MTEAAA OOOOxx +1410 3211 0 2 0 10 10 410 1410 1410 1410 20 21 GCAAAA NTEAAA VVVVxx +6332 3212 0 0 2 12 32 332 332 1332 6332 64 65 OJAAAA OTEAAA AAAAxx +9530 3213 0 2 0 10 30 530 1530 4530 9530 60 61 OCAAAA PTEAAA HHHHxx +7749 3214 1 1 9 9 49 749 1749 2749 7749 98 99 BMAAAA QTEAAA OOOOxx +3656 3215 0 0 6 16 56 656 1656 3656 3656 112 113 QKAAAA RTEAAA VVVVxx +37 3216 1 1 7 17 37 37 37 37 37 74 75 LBAAAA STEAAA AAAAxx +2744 3217 0 0 4 4 44 744 744 2744 2744 88 89 OBAAAA TTEAAA HHHHxx +4206 3218 0 2 6 6 6 206 206 4206 4206 12 13 UFAAAA UTEAAA OOOOxx +1846 3219 0 2 6 6 46 846 1846 1846 1846 92 93 ATAAAA VTEAAA VVVVxx +9913 3220 1 1 3 13 13 913 1913 4913 9913 26 27 HRAAAA WTEAAA AAAAxx +4078 3221 0 2 8 18 78 78 78 4078 4078 156 157 WAAAAA XTEAAA HHHHxx +2080 3222 0 0 0 0 80 80 80 2080 2080 160 161 ACAAAA YTEAAA OOOOxx +4169 3223 1 1 9 9 69 169 169 4169 4169 138 139 JEAAAA ZTEAAA VVVVxx +2070 3224 0 2 0 10 70 70 70 2070 2070 140 141 QBAAAA AUEAAA AAAAxx +4500 3225 0 0 0 0 0 500 500 4500 4500 0 1 CRAAAA BUEAAA HHHHxx +4123 3226 1 3 3 3 23 123 123 4123 4123 46 47 PCAAAA CUEAAA OOOOxx +5594 3227 0 2 4 14 94 594 1594 594 5594 188 189 EHAAAA DUEAAA VVVVxx +9941 3228 1 1 1 1 41 941 1941 4941 9941 82 83 JSAAAA EUEAAA AAAAxx +7154 3229 0 2 4 14 54 154 1154 2154 7154 108 109 EPAAAA FUEAAA HHHHxx +8340 3230 0 0 0 0 40 340 340 3340 8340 80 81 UIAAAA GUEAAA OOOOxx +7110 3231 0 2 0 10 10 110 1110 2110 7110 20 21 MNAAAA HUEAAA VVVVxx +7795 3232 1 3 5 15 95 795 1795 2795 7795 190 191 VNAAAA IUEAAA AAAAxx +132 3233 0 0 2 12 32 132 132 132 132 64 65 CFAAAA JUEAAA HHHHxx +4603 3234 1 3 3 3 3 603 603 4603 4603 6 7 BVAAAA KUEAAA OOOOxx +9720 3235 0 0 0 0 20 720 1720 4720 9720 40 41 WJAAAA LUEAAA VVVVxx +1460 3236 0 0 0 0 60 460 1460 1460 1460 120 121 EEAAAA MUEAAA AAAAxx +4677 3237 1 1 7 17 77 677 677 4677 4677 154 155 XXAAAA NUEAAA HHHHxx +9272 3238 0 0 2 12 72 272 1272 4272 9272 144 145 QSAAAA OUEAAA OOOOxx +2279 3239 1 3 9 19 79 279 279 2279 2279 158 159 RJAAAA PUEAAA VVVVxx +4587 3240 1 3 7 7 87 587 587 4587 4587 174 175 LUAAAA QUEAAA AAAAxx +2244 3241 0 0 4 4 44 244 244 2244 2244 88 89 IIAAAA RUEAAA HHHHxx +742 3242 0 2 2 2 42 742 742 742 742 84 85 OCAAAA SUEAAA OOOOxx +4426 3243 0 2 6 6 26 426 426 4426 4426 52 53 GOAAAA TUEAAA VVVVxx +4571 3244 1 3 1 11 71 571 571 4571 4571 142 143 VTAAAA UUEAAA AAAAxx +4775 3245 1 3 5 15 75 775 775 4775 4775 150 151 RBAAAA VUEAAA HHHHxx +24 3246 0 0 4 4 24 24 24 24 24 48 49 YAAAAA WUEAAA OOOOxx +4175 3247 1 3 5 15 75 175 175 4175 4175 150 151 PEAAAA XUEAAA VVVVxx +9877 3248 1 1 7 17 77 877 1877 4877 9877 154 155 XPAAAA YUEAAA AAAAxx +7271 3249 1 3 1 11 71 271 1271 2271 7271 142 143 RTAAAA ZUEAAA HHHHxx +5468 3250 0 0 8 8 68 468 1468 468 5468 136 137 ICAAAA AVEAAA OOOOxx +6106 3251 0 2 6 6 6 106 106 1106 6106 12 13 WAAAAA BVEAAA VVVVxx +9005 3252 1 1 5 5 5 5 1005 4005 9005 10 11 JIAAAA CVEAAA AAAAxx +109 3253 1 1 9 9 9 109 109 109 109 18 19 FEAAAA DVEAAA HHHHxx +6365 3254 1 1 5 5 65 365 365 1365 6365 130 131 VKAAAA EVEAAA OOOOxx +7437 3255 1 1 7 17 37 437 1437 2437 7437 74 75 BAAAAA FVEAAA VVVVxx +7979 3256 1 3 9 19 79 979 1979 2979 7979 158 159 XUAAAA GVEAAA AAAAxx +6050 3257 0 2 0 10 50 50 50 1050 6050 100 101 SYAAAA HVEAAA HHHHxx +2853 3258 1 1 3 13 53 853 853 2853 2853 106 107 TFAAAA IVEAAA OOOOxx +7603 3259 1 3 3 3 3 603 1603 2603 7603 6 7 LGAAAA JVEAAA VVVVxx +483 3260 1 3 3 3 83 483 483 483 483 166 167 PSAAAA KVEAAA AAAAxx +5994 3261 0 2 4 14 94 994 1994 994 5994 188 189 OWAAAA LVEAAA HHHHxx +6708 3262 0 0 8 8 8 708 708 1708 6708 16 17 AYAAAA MVEAAA OOOOxx +5090 3263 0 2 0 10 90 90 1090 90 5090 180 181 UNAAAA NVEAAA VVVVxx +4608 3264 0 0 8 8 8 608 608 4608 4608 16 17 GVAAAA OVEAAA AAAAxx +4551 3265 1 3 1 11 51 551 551 4551 4551 102 103 BTAAAA PVEAAA HHHHxx +5437 3266 1 1 7 17 37 437 1437 437 5437 74 75 DBAAAA QVEAAA OOOOxx +4130 3267 0 2 0 10 30 130 130 4130 4130 60 61 WCAAAA RVEAAA VVVVxx +6363 3268 1 3 3 3 63 363 363 1363 6363 126 127 TKAAAA SVEAAA AAAAxx +1499 3269 1 3 9 19 99 499 1499 1499 1499 198 199 RFAAAA TVEAAA HHHHxx +384 3270 0 0 4 4 84 384 384 384 384 168 169 UOAAAA UVEAAA OOOOxx +2266 3271 0 2 6 6 66 266 266 2266 2266 132 133 EJAAAA VVEAAA VVVVxx +6018 3272 0 2 8 18 18 18 18 1018 6018 36 37 MXAAAA WVEAAA AAAAxx +7915 3273 1 3 5 15 15 915 1915 2915 7915 30 31 LSAAAA XVEAAA HHHHxx +6167 3274 1 3 7 7 67 167 167 1167 6167 134 135 FDAAAA YVEAAA OOOOxx +9988 3275 0 0 8 8 88 988 1988 4988 9988 176 177 EUAAAA ZVEAAA VVVVxx +6599 3276 1 3 9 19 99 599 599 1599 6599 198 199 VTAAAA AWEAAA AAAAxx +1693 3277 1 1 3 13 93 693 1693 1693 1693 186 187 DNAAAA BWEAAA HHHHxx +5971 3278 1 3 1 11 71 971 1971 971 5971 142 143 RVAAAA CWEAAA OOOOxx +8470 3279 0 2 0 10 70 470 470 3470 8470 140 141 UNAAAA DWEAAA VVVVxx +2807 3280 1 3 7 7 7 807 807 2807 2807 14 15 ZDAAAA EWEAAA AAAAxx +1120 3281 0 0 0 0 20 120 1120 1120 1120 40 41 CRAAAA FWEAAA HHHHxx +5924 3282 0 0 4 4 24 924 1924 924 5924 48 49 WTAAAA GWEAAA OOOOxx +9025 3283 1 1 5 5 25 25 1025 4025 9025 50 51 DJAAAA HWEAAA VVVVxx +9454 3284 0 2 4 14 54 454 1454 4454 9454 108 109 QZAAAA IWEAAA AAAAxx +2259 3285 1 3 9 19 59 259 259 2259 2259 118 119 XIAAAA JWEAAA HHHHxx +5249 3286 1 1 9 9 49 249 1249 249 5249 98 99 XTAAAA KWEAAA OOOOxx +6350 3287 0 2 0 10 50 350 350 1350 6350 100 101 GKAAAA LWEAAA VVVVxx +2930 3288 0 2 0 10 30 930 930 2930 2930 60 61 SIAAAA MWEAAA AAAAxx +6055 3289 1 3 5 15 55 55 55 1055 6055 110 111 XYAAAA NWEAAA HHHHxx +7691 3290 1 3 1 11 91 691 1691 2691 7691 182 183 VJAAAA OWEAAA OOOOxx +1573 3291 1 1 3 13 73 573 1573 1573 1573 146 147 NIAAAA PWEAAA VVVVxx +9943 3292 1 3 3 3 43 943 1943 4943 9943 86 87 LSAAAA QWEAAA AAAAxx +3085 3293 1 1 5 5 85 85 1085 3085 3085 170 171 ROAAAA RWEAAA HHHHxx +5928 3294 0 0 8 8 28 928 1928 928 5928 56 57 AUAAAA SWEAAA OOOOxx +887 3295 1 3 7 7 87 887 887 887 887 174 175 DIAAAA TWEAAA VVVVxx +4630 3296 0 2 0 10 30 630 630 4630 4630 60 61 CWAAAA UWEAAA AAAAxx +9827 3297 1 3 7 7 27 827 1827 4827 9827 54 55 ZNAAAA VWEAAA HHHHxx +8926 3298 0 2 6 6 26 926 926 3926 8926 52 53 IFAAAA WWEAAA OOOOxx +5726 3299 0 2 6 6 26 726 1726 726 5726 52 53 GMAAAA XWEAAA VVVVxx +1569 3300 1 1 9 9 69 569 1569 1569 1569 138 139 JIAAAA YWEAAA AAAAxx +8074 3301 0 2 4 14 74 74 74 3074 8074 148 149 OYAAAA ZWEAAA HHHHxx +7909 3302 1 1 9 9 9 909 1909 2909 7909 18 19 FSAAAA AXEAAA OOOOxx +8367 3303 1 3 7 7 67 367 367 3367 8367 134 135 VJAAAA BXEAAA VVVVxx +7217 3304 1 1 7 17 17 217 1217 2217 7217 34 35 PRAAAA CXEAAA AAAAxx +5254 3305 0 2 4 14 54 254 1254 254 5254 108 109 CUAAAA DXEAAA HHHHxx +1181 3306 1 1 1 1 81 181 1181 1181 1181 162 163 LTAAAA EXEAAA OOOOxx +6907 3307 1 3 7 7 7 907 907 1907 6907 14 15 RFAAAA FXEAAA VVVVxx +5508 3308 0 0 8 8 8 508 1508 508 5508 16 17 WDAAAA GXEAAA AAAAxx +4782 3309 0 2 2 2 82 782 782 4782 4782 164 165 YBAAAA HXEAAA HHHHxx +793 3310 1 1 3 13 93 793 793 793 793 186 187 NEAAAA IXEAAA OOOOxx +5740 3311 0 0 0 0 40 740 1740 740 5740 80 81 UMAAAA JXEAAA VVVVxx +3107 3312 1 3 7 7 7 107 1107 3107 3107 14 15 NPAAAA KXEAAA AAAAxx +1197 3313 1 1 7 17 97 197 1197 1197 1197 194 195 BUAAAA LXEAAA HHHHxx +4376 3314 0 0 6 16 76 376 376 4376 4376 152 153 IMAAAA MXEAAA OOOOxx +6226 3315 0 2 6 6 26 226 226 1226 6226 52 53 MFAAAA NXEAAA VVVVxx +5033 3316 1 1 3 13 33 33 1033 33 5033 66 67 PLAAAA OXEAAA AAAAxx +5494 3317 0 2 4 14 94 494 1494 494 5494 188 189 IDAAAA PXEAAA HHHHxx +3244 3318 0 0 4 4 44 244 1244 3244 3244 88 89 UUAAAA QXEAAA OOOOxx +7670 3319 0 2 0 10 70 670 1670 2670 7670 140 141 AJAAAA RXEAAA VVVVxx +9273 3320 1 1 3 13 73 273 1273 4273 9273 146 147 RSAAAA SXEAAA AAAAxx +5248 3321 0 0 8 8 48 248 1248 248 5248 96 97 WTAAAA TXEAAA HHHHxx +3381 3322 1 1 1 1 81 381 1381 3381 3381 162 163 BAAAAA UXEAAA OOOOxx +4136 3323 0 0 6 16 36 136 136 4136 4136 72 73 CDAAAA VXEAAA VVVVxx +4163 3324 1 3 3 3 63 163 163 4163 4163 126 127 DEAAAA WXEAAA AAAAxx +4270 3325 0 2 0 10 70 270 270 4270 4270 140 141 GIAAAA XXEAAA HHHHxx +1729 3326 1 1 9 9 29 729 1729 1729 1729 58 59 NOAAAA YXEAAA OOOOxx +2778 3327 0 2 8 18 78 778 778 2778 2778 156 157 WCAAAA ZXEAAA VVVVxx +5082 3328 0 2 2 2 82 82 1082 82 5082 164 165 MNAAAA AYEAAA AAAAxx +870 3329 0 2 0 10 70 870 870 870 870 140 141 MHAAAA BYEAAA HHHHxx +4192 3330 0 0 2 12 92 192 192 4192 4192 184 185 GFAAAA CYEAAA OOOOxx +308 3331 0 0 8 8 8 308 308 308 308 16 17 WLAAAA DYEAAA VVVVxx +6783 3332 1 3 3 3 83 783 783 1783 6783 166 167 XAAAAA EYEAAA AAAAxx +7611 3333 1 3 1 11 11 611 1611 2611 7611 22 23 TGAAAA FYEAAA HHHHxx +4221 3334 1 1 1 1 21 221 221 4221 4221 42 43 JGAAAA GYEAAA OOOOxx +6353 3335 1 1 3 13 53 353 353 1353 6353 106 107 JKAAAA HYEAAA VVVVxx +1830 3336 0 2 0 10 30 830 1830 1830 1830 60 61 KSAAAA IYEAAA AAAAxx +2437 3337 1 1 7 17 37 437 437 2437 2437 74 75 TPAAAA JYEAAA HHHHxx +3360 3338 0 0 0 0 60 360 1360 3360 3360 120 121 GZAAAA KYEAAA OOOOxx +1829 3339 1 1 9 9 29 829 1829 1829 1829 58 59 JSAAAA LYEAAA VVVVxx +9475 3340 1 3 5 15 75 475 1475 4475 9475 150 151 LAAAAA MYEAAA AAAAxx +4566 3341 0 2 6 6 66 566 566 4566 4566 132 133 QTAAAA NYEAAA HHHHxx +9944 3342 0 0 4 4 44 944 1944 4944 9944 88 89 MSAAAA OYEAAA OOOOxx +6054 3343 0 2 4 14 54 54 54 1054 6054 108 109 WYAAAA PYEAAA VVVVxx +4722 3344 0 2 2 2 22 722 722 4722 4722 44 45 QZAAAA QYEAAA AAAAxx +2779 3345 1 3 9 19 79 779 779 2779 2779 158 159 XCAAAA RYEAAA HHHHxx +8051 3346 1 3 1 11 51 51 51 3051 8051 102 103 RXAAAA SYEAAA OOOOxx +9671 3347 1 3 1 11 71 671 1671 4671 9671 142 143 ZHAAAA TYEAAA VVVVxx +6084 3348 0 0 4 4 84 84 84 1084 6084 168 169 AAAAAA UYEAAA AAAAxx +3729 3349 1 1 9 9 29 729 1729 3729 3729 58 59 LNAAAA VYEAAA HHHHxx +6627 3350 1 3 7 7 27 627 627 1627 6627 54 55 XUAAAA WYEAAA OOOOxx +4769 3351 1 1 9 9 69 769 769 4769 4769 138 139 LBAAAA XYEAAA VVVVxx +2224 3352 0 0 4 4 24 224 224 2224 2224 48 49 OHAAAA YYEAAA AAAAxx +1404 3353 0 0 4 4 4 404 1404 1404 1404 8 9 ACAAAA ZYEAAA HHHHxx +8532 3354 0 0 2 12 32 532 532 3532 8532 64 65 EQAAAA AZEAAA OOOOxx +6759 3355 1 3 9 19 59 759 759 1759 6759 118 119 ZZAAAA BZEAAA VVVVxx +6404 3356 0 0 4 4 4 404 404 1404 6404 8 9 IMAAAA CZEAAA AAAAxx +3144 3357 0 0 4 4 44 144 1144 3144 3144 88 89 YQAAAA DZEAAA HHHHxx +973 3358 1 1 3 13 73 973 973 973 973 146 147 LLAAAA EZEAAA OOOOxx +9789 3359 1 1 9 9 89 789 1789 4789 9789 178 179 NMAAAA FZEAAA VVVVxx +6181 3360 1 1 1 1 81 181 181 1181 6181 162 163 TDAAAA GZEAAA AAAAxx +1519 3361 1 3 9 19 19 519 1519 1519 1519 38 39 LGAAAA HZEAAA HHHHxx +9729 3362 1 1 9 9 29 729 1729 4729 9729 58 59 FKAAAA IZEAAA OOOOxx +8167 3363 1 3 7 7 67 167 167 3167 8167 134 135 DCAAAA JZEAAA VVVVxx +3830 3364 0 2 0 10 30 830 1830 3830 3830 60 61 IRAAAA KZEAAA AAAAxx +6286 3365 0 2 6 6 86 286 286 1286 6286 172 173 UHAAAA LZEAAA HHHHxx +3047 3366 1 3 7 7 47 47 1047 3047 3047 94 95 FNAAAA MZEAAA OOOOxx +3183 3367 1 3 3 3 83 183 1183 3183 3183 166 167 LSAAAA NZEAAA VVVVxx +6687 3368 1 3 7 7 87 687 687 1687 6687 174 175 FXAAAA OZEAAA AAAAxx +2783 3369 1 3 3 3 83 783 783 2783 2783 166 167 BDAAAA PZEAAA HHHHxx +9920 3370 0 0 0 0 20 920 1920 4920 9920 40 41 ORAAAA QZEAAA OOOOxx +4847 3371 1 3 7 7 47 847 847 4847 4847 94 95 LEAAAA RZEAAA VVVVxx +3645 3372 1 1 5 5 45 645 1645 3645 3645 90 91 FKAAAA SZEAAA AAAAxx +7406 3373 0 2 6 6 6 406 1406 2406 7406 12 13 WYAAAA TZEAAA HHHHxx +6003 3374 1 3 3 3 3 3 3 1003 6003 6 7 XWAAAA UZEAAA OOOOxx +3408 3375 0 0 8 8 8 408 1408 3408 3408 16 17 CBAAAA VZEAAA VVVVxx +4243 3376 1 3 3 3 43 243 243 4243 4243 86 87 FHAAAA WZEAAA AAAAxx +1622 3377 0 2 2 2 22 622 1622 1622 1622 44 45 KKAAAA XZEAAA HHHHxx +5319 3378 1 3 9 19 19 319 1319 319 5319 38 39 PWAAAA YZEAAA OOOOxx +4033 3379 1 1 3 13 33 33 33 4033 4033 66 67 DZAAAA ZZEAAA VVVVxx +8573 3380 1 1 3 13 73 573 573 3573 8573 146 147 TRAAAA AAFAAA AAAAxx +8404 3381 0 0 4 4 4 404 404 3404 8404 8 9 GLAAAA BAFAAA HHHHxx +6993 3382 1 1 3 13 93 993 993 1993 6993 186 187 ZIAAAA CAFAAA OOOOxx +660 3383 0 0 0 0 60 660 660 660 660 120 121 KZAAAA DAFAAA VVVVxx +1136 3384 0 0 6 16 36 136 1136 1136 1136 72 73 SRAAAA EAFAAA AAAAxx +3393 3385 1 1 3 13 93 393 1393 3393 3393 186 187 NAAAAA FAFAAA HHHHxx +9743 3386 1 3 3 3 43 743 1743 4743 9743 86 87 TKAAAA GAFAAA OOOOxx +9705 3387 1 1 5 5 5 705 1705 4705 9705 10 11 HJAAAA HAFAAA VVVVxx +6960 3388 0 0 0 0 60 960 960 1960 6960 120 121 SHAAAA IAFAAA AAAAxx +2753 3389 1 1 3 13 53 753 753 2753 2753 106 107 XBAAAA JAFAAA HHHHxx +906 3390 0 2 6 6 6 906 906 906 906 12 13 WIAAAA KAFAAA OOOOxx +999 3391 1 3 9 19 99 999 999 999 999 198 199 LMAAAA LAFAAA VVVVxx +6927 3392 1 3 7 7 27 927 927 1927 6927 54 55 LGAAAA MAFAAA AAAAxx +4846 3393 0 2 6 6 46 846 846 4846 4846 92 93 KEAAAA NAFAAA HHHHxx +676 3394 0 0 6 16 76 676 676 676 676 152 153 AAAAAA OAFAAA OOOOxx +8612 3395 0 0 2 12 12 612 612 3612 8612 24 25 GTAAAA PAFAAA VVVVxx +4111 3396 1 3 1 11 11 111 111 4111 4111 22 23 DCAAAA QAFAAA AAAAxx +9994 3397 0 2 4 14 94 994 1994 4994 9994 188 189 KUAAAA RAFAAA HHHHxx +4399 3398 1 3 9 19 99 399 399 4399 4399 198 199 FNAAAA SAFAAA OOOOxx +4464 3399 0 0 4 4 64 464 464 4464 4464 128 129 SPAAAA TAFAAA VVVVxx +7316 3400 0 0 6 16 16 316 1316 2316 7316 32 33 KVAAAA UAFAAA AAAAxx +8982 3401 0 2 2 2 82 982 982 3982 8982 164 165 MHAAAA VAFAAA HHHHxx +1871 3402 1 3 1 11 71 871 1871 1871 1871 142 143 ZTAAAA WAFAAA OOOOxx +4082 3403 0 2 2 2 82 82 82 4082 4082 164 165 ABAAAA XAFAAA VVVVxx +3949 3404 1 1 9 9 49 949 1949 3949 3949 98 99 XVAAAA YAFAAA AAAAxx +9352 3405 0 0 2 12 52 352 1352 4352 9352 104 105 SVAAAA ZAFAAA HHHHxx +9638 3406 0 2 8 18 38 638 1638 4638 9638 76 77 SGAAAA ABFAAA OOOOxx +8177 3407 1 1 7 17 77 177 177 3177 8177 154 155 NCAAAA BBFAAA VVVVxx +3499 3408 1 3 9 19 99 499 1499 3499 3499 198 199 PEAAAA CBFAAA AAAAxx +4233 3409 1 1 3 13 33 233 233 4233 4233 66 67 VGAAAA DBFAAA HHHHxx +1953 3410 1 1 3 13 53 953 1953 1953 1953 106 107 DXAAAA EBFAAA OOOOxx +7372 3411 0 0 2 12 72 372 1372 2372 7372 144 145 OXAAAA FBFAAA VVVVxx +5127 3412 1 3 7 7 27 127 1127 127 5127 54 55 FPAAAA GBFAAA AAAAxx +4384 3413 0 0 4 4 84 384 384 4384 4384 168 169 QMAAAA HBFAAA HHHHxx +9964 3414 0 0 4 4 64 964 1964 4964 9964 128 129 GTAAAA IBFAAA OOOOxx +5392 3415 0 0 2 12 92 392 1392 392 5392 184 185 KZAAAA JBFAAA VVVVxx +616 3416 0 0 6 16 16 616 616 616 616 32 33 SXAAAA KBFAAA AAAAxx +591 3417 1 3 1 11 91 591 591 591 591 182 183 TWAAAA LBFAAA HHHHxx +6422 3418 0 2 2 2 22 422 422 1422 6422 44 45 ANAAAA MBFAAA OOOOxx +6551 3419 1 3 1 11 51 551 551 1551 6551 102 103 ZRAAAA NBFAAA VVVVxx +9286 3420 0 2 6 6 86 286 1286 4286 9286 172 173 ETAAAA OBFAAA AAAAxx +3817 3421 1 1 7 17 17 817 1817 3817 3817 34 35 VQAAAA PBFAAA HHHHxx +7717 3422 1 1 7 17 17 717 1717 2717 7717 34 35 VKAAAA QBFAAA OOOOxx +8718 3423 0 2 8 18 18 718 718 3718 8718 36 37 IXAAAA RBFAAA VVVVxx +8608 3424 0 0 8 8 8 608 608 3608 8608 16 17 CTAAAA SBFAAA AAAAxx +2242 3425 0 2 2 2 42 242 242 2242 2242 84 85 GIAAAA TBFAAA HHHHxx +4811 3426 1 3 1 11 11 811 811 4811 4811 22 23 BDAAAA UBFAAA OOOOxx +6838 3427 0 2 8 18 38 838 838 1838 6838 76 77 ADAAAA VBFAAA VVVVxx +787 3428 1 3 7 7 87 787 787 787 787 174 175 HEAAAA WBFAAA AAAAxx +7940 3429 0 0 0 0 40 940 1940 2940 7940 80 81 KTAAAA XBFAAA HHHHxx +336 3430 0 0 6 16 36 336 336 336 336 72 73 YMAAAA YBFAAA OOOOxx +9859 3431 1 3 9 19 59 859 1859 4859 9859 118 119 FPAAAA ZBFAAA VVVVxx +3864 3432 0 0 4 4 64 864 1864 3864 3864 128 129 QSAAAA ACFAAA AAAAxx +7162 3433 0 2 2 2 62 162 1162 2162 7162 124 125 MPAAAA BCFAAA HHHHxx +2071 3434 1 3 1 11 71 71 71 2071 2071 142 143 RBAAAA CCFAAA OOOOxx +7469 3435 1 1 9 9 69 469 1469 2469 7469 138 139 HBAAAA DCFAAA VVVVxx +2917 3436 1 1 7 17 17 917 917 2917 2917 34 35 FIAAAA ECFAAA AAAAxx +7486 3437 0 2 6 6 86 486 1486 2486 7486 172 173 YBAAAA FCFAAA HHHHxx +3355 3438 1 3 5 15 55 355 1355 3355 3355 110 111 BZAAAA GCFAAA OOOOxx +6998 3439 0 2 8 18 98 998 998 1998 6998 196 197 EJAAAA HCFAAA VVVVxx +5498 3440 0 2 8 18 98 498 1498 498 5498 196 197 MDAAAA ICFAAA AAAAxx +5113 3441 1 1 3 13 13 113 1113 113 5113 26 27 ROAAAA JCFAAA HHHHxx +2846 3442 0 2 6 6 46 846 846 2846 2846 92 93 MFAAAA KCFAAA OOOOxx +6834 3443 0 2 4 14 34 834 834 1834 6834 68 69 WCAAAA LCFAAA VVVVxx +8925 3444 1 1 5 5 25 925 925 3925 8925 50 51 HFAAAA MCFAAA AAAAxx +2757 3445 1 1 7 17 57 757 757 2757 2757 114 115 BCAAAA NCFAAA HHHHxx +2775 3446 1 3 5 15 75 775 775 2775 2775 150 151 TCAAAA OCFAAA OOOOxx +6182 3447 0 2 2 2 82 182 182 1182 6182 164 165 UDAAAA PCFAAA VVVVxx +4488 3448 0 0 8 8 88 488 488 4488 4488 176 177 QQAAAA QCFAAA AAAAxx +8523 3449 1 3 3 3 23 523 523 3523 8523 46 47 VPAAAA RCFAAA HHHHxx +52 3450 0 0 2 12 52 52 52 52 52 104 105 ACAAAA SCFAAA OOOOxx +7251 3451 1 3 1 11 51 251 1251 2251 7251 102 103 XSAAAA TCFAAA VVVVxx +6130 3452 0 2 0 10 30 130 130 1130 6130 60 61 UBAAAA UCFAAA AAAAxx +205 3453 1 1 5 5 5 205 205 205 205 10 11 XHAAAA VCFAAA HHHHxx +1186 3454 0 2 6 6 86 186 1186 1186 1186 172 173 QTAAAA WCFAAA OOOOxx +1738 3455 0 2 8 18 38 738 1738 1738 1738 76 77 WOAAAA XCFAAA VVVVxx +9485 3456 1 1 5 5 85 485 1485 4485 9485 170 171 VAAAAA YCFAAA AAAAxx +4235 3457 1 3 5 15 35 235 235 4235 4235 70 71 XGAAAA ZCFAAA HHHHxx +7891 3458 1 3 1 11 91 891 1891 2891 7891 182 183 NRAAAA ADFAAA OOOOxx +4960 3459 0 0 0 0 60 960 960 4960 4960 120 121 UIAAAA BDFAAA VVVVxx +8911 3460 1 3 1 11 11 911 911 3911 8911 22 23 TEAAAA CDFAAA AAAAxx +1219 3461 1 3 9 19 19 219 1219 1219 1219 38 39 XUAAAA DDFAAA HHHHxx +9652 3462 0 0 2 12 52 652 1652 4652 9652 104 105 GHAAAA EDFAAA OOOOxx +9715 3463 1 3 5 15 15 715 1715 4715 9715 30 31 RJAAAA FDFAAA VVVVxx +6629 3464 1 1 9 9 29 629 629 1629 6629 58 59 ZUAAAA GDFAAA AAAAxx +700 3465 0 0 0 0 0 700 700 700 700 0 1 YAAAAA HDFAAA HHHHxx +9819 3466 1 3 9 19 19 819 1819 4819 9819 38 39 RNAAAA IDFAAA OOOOxx +5188 3467 0 0 8 8 88 188 1188 188 5188 176 177 ORAAAA JDFAAA VVVVxx +5367 3468 1 3 7 7 67 367 1367 367 5367 134 135 LYAAAA KDFAAA AAAAxx +6447 3469 1 3 7 7 47 447 447 1447 6447 94 95 ZNAAAA LDFAAA HHHHxx +720 3470 0 0 0 0 20 720 720 720 720 40 41 SBAAAA MDFAAA OOOOxx +9157 3471 1 1 7 17 57 157 1157 4157 9157 114 115 FOAAAA NDFAAA VVVVxx +1082 3472 0 2 2 2 82 82 1082 1082 1082 164 165 QPAAAA ODFAAA AAAAxx +3179 3473 1 3 9 19 79 179 1179 3179 3179 158 159 HSAAAA PDFAAA HHHHxx +4818 3474 0 2 8 18 18 818 818 4818 4818 36 37 IDAAAA QDFAAA OOOOxx +7607 3475 1 3 7 7 7 607 1607 2607 7607 14 15 PGAAAA RDFAAA VVVVxx +2352 3476 0 0 2 12 52 352 352 2352 2352 104 105 MMAAAA SDFAAA AAAAxx +1170 3477 0 2 0 10 70 170 1170 1170 1170 140 141 ATAAAA TDFAAA HHHHxx +4269 3478 1 1 9 9 69 269 269 4269 4269 138 139 FIAAAA UDFAAA OOOOxx +8767 3479 1 3 7 7 67 767 767 3767 8767 134 135 FZAAAA VDFAAA VVVVxx +3984 3480 0 0 4 4 84 984 1984 3984 3984 168 169 GXAAAA WDFAAA AAAAxx +3190 3481 0 2 0 10 90 190 1190 3190 3190 180 181 SSAAAA XDFAAA HHHHxx +7456 3482 0 0 6 16 56 456 1456 2456 7456 112 113 UAAAAA YDFAAA OOOOxx +4348 3483 0 0 8 8 48 348 348 4348 4348 96 97 GLAAAA ZDFAAA VVVVxx +3150 3484 0 2 0 10 50 150 1150 3150 3150 100 101 ERAAAA AEFAAA AAAAxx +8780 3485 0 0 0 0 80 780 780 3780 8780 160 161 SZAAAA BEFAAA HHHHxx +2553 3486 1 1 3 13 53 553 553 2553 2553 106 107 FUAAAA CEFAAA OOOOxx +7526 3487 0 2 6 6 26 526 1526 2526 7526 52 53 MDAAAA DEFAAA VVVVxx +2031 3488 1 3 1 11 31 31 31 2031 2031 62 63 DAAAAA EEFAAA AAAAxx +8793 3489 1 1 3 13 93 793 793 3793 8793 186 187 FAAAAA FEFAAA HHHHxx +1122 3490 0 2 2 2 22 122 1122 1122 1122 44 45 ERAAAA GEFAAA OOOOxx +1855 3491 1 3 5 15 55 855 1855 1855 1855 110 111 JTAAAA HEFAAA VVVVxx +6613 3492 1 1 3 13 13 613 613 1613 6613 26 27 JUAAAA IEFAAA AAAAxx +3231 3493 1 3 1 11 31 231 1231 3231 3231 62 63 HUAAAA JEFAAA HHHHxx +9101 3494 1 1 1 1 1 101 1101 4101 9101 2 3 BMAAAA KEFAAA OOOOxx +4937 3495 1 1 7 17 37 937 937 4937 4937 74 75 XHAAAA LEFAAA VVVVxx +666 3496 0 2 6 6 66 666 666 666 666 132 133 QZAAAA MEFAAA AAAAxx +8943 3497 1 3 3 3 43 943 943 3943 8943 86 87 ZFAAAA NEFAAA HHHHxx +6164 3498 0 0 4 4 64 164 164 1164 6164 128 129 CDAAAA OEFAAA OOOOxx +1081 3499 1 1 1 1 81 81 1081 1081 1081 162 163 PPAAAA PEFAAA VVVVxx +210 3500 0 2 0 10 10 210 210 210 210 20 21 CIAAAA QEFAAA AAAAxx +6024 3501 0 0 4 4 24 24 24 1024 6024 48 49 SXAAAA REFAAA HHHHxx +5715 3502 1 3 5 15 15 715 1715 715 5715 30 31 VLAAAA SEFAAA OOOOxx +8938 3503 0 2 8 18 38 938 938 3938 8938 76 77 UFAAAA TEFAAA VVVVxx +1326 3504 0 2 6 6 26 326 1326 1326 1326 52 53 AZAAAA UEFAAA AAAAxx +7111 3505 1 3 1 11 11 111 1111 2111 7111 22 23 NNAAAA VEFAAA HHHHxx +757 3506 1 1 7 17 57 757 757 757 757 114 115 DDAAAA WEFAAA OOOOxx +8933 3507 1 1 3 13 33 933 933 3933 8933 66 67 PFAAAA XEFAAA VVVVxx +6495 3508 1 3 5 15 95 495 495 1495 6495 190 191 VPAAAA YEFAAA AAAAxx +3134 3509 0 2 4 14 34 134 1134 3134 3134 68 69 OQAAAA ZEFAAA HHHHxx +1304 3510 0 0 4 4 4 304 1304 1304 1304 8 9 EYAAAA AFFAAA OOOOxx +1835 3511 1 3 5 15 35 835 1835 1835 1835 70 71 PSAAAA BFFAAA VVVVxx +7275 3512 1 3 5 15 75 275 1275 2275 7275 150 151 VTAAAA CFFAAA AAAAxx +7337 3513 1 1 7 17 37 337 1337 2337 7337 74 75 FWAAAA DFFAAA HHHHxx +1282 3514 0 2 2 2 82 282 1282 1282 1282 164 165 IXAAAA EFFAAA OOOOxx +6566 3515 0 2 6 6 66 566 566 1566 6566 132 133 OSAAAA FFFAAA VVVVxx +3786 3516 0 2 6 6 86 786 1786 3786 3786 172 173 QPAAAA GFFAAA AAAAxx +5741 3517 1 1 1 1 41 741 1741 741 5741 82 83 VMAAAA HFFAAA HHHHxx +6076 3518 0 0 6 16 76 76 76 1076 6076 152 153 SZAAAA IFFAAA OOOOxx +9998 3519 0 2 8 18 98 998 1998 4998 9998 196 197 OUAAAA JFFAAA VVVVxx +6268 3520 0 0 8 8 68 268 268 1268 6268 136 137 CHAAAA KFFAAA AAAAxx +9647 3521 1 3 7 7 47 647 1647 4647 9647 94 95 BHAAAA LFFAAA HHHHxx +4877 3522 1 1 7 17 77 877 877 4877 4877 154 155 PFAAAA MFFAAA OOOOxx +2652 3523 0 0 2 12 52 652 652 2652 2652 104 105 AYAAAA NFFAAA VVVVxx +1247 3524 1 3 7 7 47 247 1247 1247 1247 94 95 ZVAAAA OFFAAA AAAAxx +2721 3525 1 1 1 1 21 721 721 2721 2721 42 43 RAAAAA PFFAAA HHHHxx +5968 3526 0 0 8 8 68 968 1968 968 5968 136 137 OVAAAA QFFAAA OOOOxx +9570 3527 0 2 0 10 70 570 1570 4570 9570 140 141 CEAAAA RFFAAA VVVVxx +6425 3528 1 1 5 5 25 425 425 1425 6425 50 51 DNAAAA SFFAAA AAAAxx +5451 3529 1 3 1 11 51 451 1451 451 5451 102 103 RBAAAA TFFAAA HHHHxx +5668 3530 0 0 8 8 68 668 1668 668 5668 136 137 AKAAAA UFFAAA OOOOxx +9493 3531 1 1 3 13 93 493 1493 4493 9493 186 187 DBAAAA VFFAAA VVVVxx +7973 3532 1 1 3 13 73 973 1973 2973 7973 146 147 RUAAAA WFFAAA AAAAxx +8250 3533 0 2 0 10 50 250 250 3250 8250 100 101 IFAAAA XFFAAA HHHHxx +82 3534 0 2 2 2 82 82 82 82 82 164 165 EDAAAA YFFAAA OOOOxx +6258 3535 0 2 8 18 58 258 258 1258 6258 116 117 SGAAAA ZFFAAA VVVVxx +9978 3536 0 2 8 18 78 978 1978 4978 9978 156 157 UTAAAA AGFAAA AAAAxx +6930 3537 0 2 0 10 30 930 930 1930 6930 60 61 OGAAAA BGFAAA HHHHxx +3746 3538 0 2 6 6 46 746 1746 3746 3746 92 93 COAAAA CGFAAA OOOOxx +7065 3539 1 1 5 5 65 65 1065 2065 7065 130 131 TLAAAA DGFAAA VVVVxx +4281 3540 1 1 1 1 81 281 281 4281 4281 162 163 RIAAAA EGFAAA AAAAxx +4367 3541 1 3 7 7 67 367 367 4367 4367 134 135 ZLAAAA FGFAAA HHHHxx +9526 3542 0 2 6 6 26 526 1526 4526 9526 52 53 KCAAAA GGFAAA OOOOxx +5880 3543 0 0 0 0 80 880 1880 880 5880 160 161 ESAAAA HGFAAA VVVVxx +8480 3544 0 0 0 0 80 480 480 3480 8480 160 161 EOAAAA IGFAAA AAAAxx +2476 3545 0 0 6 16 76 476 476 2476 2476 152 153 GRAAAA JGFAAA HHHHxx +9074 3546 0 2 4 14 74 74 1074 4074 9074 148 149 ALAAAA KGFAAA OOOOxx +4830 3547 0 2 0 10 30 830 830 4830 4830 60 61 UDAAAA LGFAAA VVVVxx +3207 3548 1 3 7 7 7 207 1207 3207 3207 14 15 JTAAAA MGFAAA AAAAxx +7894 3549 0 2 4 14 94 894 1894 2894 7894 188 189 QRAAAA NGFAAA HHHHxx +3860 3550 0 0 0 0 60 860 1860 3860 3860 120 121 MSAAAA OGFAAA OOOOxx +5293 3551 1 1 3 13 93 293 1293 293 5293 186 187 PVAAAA PGFAAA VVVVxx +6895 3552 1 3 5 15 95 895 895 1895 6895 190 191 FFAAAA QGFAAA AAAAxx +9908 3553 0 0 8 8 8 908 1908 4908 9908 16 17 CRAAAA RGFAAA HHHHxx +9247 3554 1 3 7 7 47 247 1247 4247 9247 94 95 RRAAAA SGFAAA OOOOxx +8110 3555 0 2 0 10 10 110 110 3110 8110 20 21 YZAAAA TGFAAA VVVVxx +4716 3556 0 0 6 16 16 716 716 4716 4716 32 33 KZAAAA UGFAAA AAAAxx +4979 3557 1 3 9 19 79 979 979 4979 4979 158 159 NJAAAA VGFAAA HHHHxx +5280 3558 0 0 0 0 80 280 1280 280 5280 160 161 CVAAAA WGFAAA OOOOxx +8326 3559 0 2 6 6 26 326 326 3326 8326 52 53 GIAAAA XGFAAA VVVVxx +5572 3560 0 0 2 12 72 572 1572 572 5572 144 145 IGAAAA YGFAAA AAAAxx +4665 3561 1 1 5 5 65 665 665 4665 4665 130 131 LXAAAA ZGFAAA HHHHxx +3665 3562 1 1 5 5 65 665 1665 3665 3665 130 131 ZKAAAA AHFAAA OOOOxx +6744 3563 0 0 4 4 44 744 744 1744 6744 88 89 KZAAAA BHFAAA VVVVxx +1897 3564 1 1 7 17 97 897 1897 1897 1897 194 195 ZUAAAA CHFAAA AAAAxx +1220 3565 0 0 0 0 20 220 1220 1220 1220 40 41 YUAAAA DHFAAA HHHHxx +2614 3566 0 2 4 14 14 614 614 2614 2614 28 29 OWAAAA EHFAAA OOOOxx +8509 3567 1 1 9 9 9 509 509 3509 8509 18 19 HPAAAA FHFAAA VVVVxx +8521 3568 1 1 1 1 21 521 521 3521 8521 42 43 TPAAAA GHFAAA AAAAxx +4121 3569 1 1 1 1 21 121 121 4121 4121 42 43 NCAAAA HHFAAA HHHHxx +9663 3570 1 3 3 3 63 663 1663 4663 9663 126 127 RHAAAA IHFAAA OOOOxx +2346 3571 0 2 6 6 46 346 346 2346 2346 92 93 GMAAAA JHFAAA VVVVxx +3370 3572 0 2 0 10 70 370 1370 3370 3370 140 141 QZAAAA KHFAAA AAAAxx +1498 3573 0 2 8 18 98 498 1498 1498 1498 196 197 QFAAAA LHFAAA HHHHxx +7422 3574 0 2 2 2 22 422 1422 2422 7422 44 45 MZAAAA MHFAAA OOOOxx +3472 3575 0 0 2 12 72 472 1472 3472 3472 144 145 ODAAAA NHFAAA VVVVxx +4126 3576 0 2 6 6 26 126 126 4126 4126 52 53 SCAAAA OHFAAA AAAAxx +4494 3577 0 2 4 14 94 494 494 4494 4494 188 189 WQAAAA PHFAAA HHHHxx +6323 3578 1 3 3 3 23 323 323 1323 6323 46 47 FJAAAA QHFAAA OOOOxx +2823 3579 1 3 3 3 23 823 823 2823 2823 46 47 PEAAAA RHFAAA VVVVxx +8596 3580 0 0 6 16 96 596 596 3596 8596 192 193 QSAAAA SHFAAA AAAAxx +6642 3581 0 2 2 2 42 642 642 1642 6642 84 85 MVAAAA THFAAA HHHHxx +9276 3582 0 0 6 16 76 276 1276 4276 9276 152 153 USAAAA UHFAAA OOOOxx +4148 3583 0 0 8 8 48 148 148 4148 4148 96 97 ODAAAA VHFAAA VVVVxx +9770 3584 0 2 0 10 70 770 1770 4770 9770 140 141 ULAAAA WHFAAA AAAAxx +9812 3585 0 0 2 12 12 812 1812 4812 9812 24 25 KNAAAA XHFAAA HHHHxx +4419 3586 1 3 9 19 19 419 419 4419 4419 38 39 ZNAAAA YHFAAA OOOOxx +3802 3587 0 2 2 2 2 802 1802 3802 3802 4 5 GQAAAA ZHFAAA VVVVxx +3210 3588 0 2 0 10 10 210 1210 3210 3210 20 21 MTAAAA AIFAAA AAAAxx +6794 3589 0 2 4 14 94 794 794 1794 6794 188 189 IBAAAA BIFAAA HHHHxx +242 3590 0 2 2 2 42 242 242 242 242 84 85 IJAAAA CIFAAA OOOOxx +962 3591 0 2 2 2 62 962 962 962 962 124 125 ALAAAA DIFAAA VVVVxx +7151 3592 1 3 1 11 51 151 1151 2151 7151 102 103 BPAAAA EIFAAA AAAAxx +9440 3593 0 0 0 0 40 440 1440 4440 9440 80 81 CZAAAA FIFAAA HHHHxx +721 3594 1 1 1 1 21 721 721 721 721 42 43 TBAAAA GIFAAA OOOOxx +2119 3595 1 3 9 19 19 119 119 2119 2119 38 39 NDAAAA HIFAAA VVVVxx +9883 3596 1 3 3 3 83 883 1883 4883 9883 166 167 DQAAAA IIFAAA AAAAxx +5071 3597 1 3 1 11 71 71 1071 71 5071 142 143 BNAAAA JIFAAA HHHHxx +8239 3598 1 3 9 19 39 239 239 3239 8239 78 79 XEAAAA KIFAAA OOOOxx +7451 3599 1 3 1 11 51 451 1451 2451 7451 102 103 PAAAAA LIFAAA VVVVxx +9517 3600 1 1 7 17 17 517 1517 4517 9517 34 35 BCAAAA MIFAAA AAAAxx +9180 3601 0 0 0 0 80 180 1180 4180 9180 160 161 CPAAAA NIFAAA HHHHxx +9327 3602 1 3 7 7 27 327 1327 4327 9327 54 55 TUAAAA OIFAAA OOOOxx +5462 3603 0 2 2 2 62 462 1462 462 5462 124 125 CCAAAA PIFAAA VVVVxx +8306 3604 0 2 6 6 6 306 306 3306 8306 12 13 MHAAAA QIFAAA AAAAxx +6234 3605 0 2 4 14 34 234 234 1234 6234 68 69 UFAAAA RIFAAA HHHHxx +8771 3606 1 3 1 11 71 771 771 3771 8771 142 143 JZAAAA SIFAAA OOOOxx +5853 3607 1 1 3 13 53 853 1853 853 5853 106 107 DRAAAA TIFAAA VVVVxx +8373 3608 1 1 3 13 73 373 373 3373 8373 146 147 BKAAAA UIFAAA AAAAxx +5017 3609 1 1 7 17 17 17 1017 17 5017 34 35 ZKAAAA VIFAAA HHHHxx +8025 3610 1 1 5 5 25 25 25 3025 8025 50 51 RWAAAA WIFAAA OOOOxx +2526 3611 0 2 6 6 26 526 526 2526 2526 52 53 ETAAAA XIFAAA VVVVxx +7419 3612 1 3 9 19 19 419 1419 2419 7419 38 39 JZAAAA YIFAAA AAAAxx +4572 3613 0 0 2 12 72 572 572 4572 4572 144 145 WTAAAA ZIFAAA HHHHxx +7744 3614 0 0 4 4 44 744 1744 2744 7744 88 89 WLAAAA AJFAAA OOOOxx +8825 3615 1 1 5 5 25 825 825 3825 8825 50 51 LBAAAA BJFAAA VVVVxx +6067 3616 1 3 7 7 67 67 67 1067 6067 134 135 JZAAAA CJFAAA AAAAxx +3291 3617 1 3 1 11 91 291 1291 3291 3291 182 183 PWAAAA DJFAAA HHHHxx +7115 3618 1 3 5 15 15 115 1115 2115 7115 30 31 RNAAAA EJFAAA OOOOxx +2626 3619 0 2 6 6 26 626 626 2626 2626 52 53 AXAAAA FJFAAA VVVVxx +4109 3620 1 1 9 9 9 109 109 4109 4109 18 19 BCAAAA GJFAAA AAAAxx +4056 3621 0 0 6 16 56 56 56 4056 4056 112 113 AAAAAA HJFAAA HHHHxx +6811 3622 1 3 1 11 11 811 811 1811 6811 22 23 ZBAAAA IJFAAA OOOOxx +680 3623 0 0 0 0 80 680 680 680 680 160 161 EAAAAA JJFAAA VVVVxx +474 3624 0 2 4 14 74 474 474 474 474 148 149 GSAAAA KJFAAA AAAAxx +9294 3625 0 2 4 14 94 294 1294 4294 9294 188 189 MTAAAA LJFAAA HHHHxx +7555 3626 1 3 5 15 55 555 1555 2555 7555 110 111 PEAAAA MJFAAA OOOOxx +8076 3627 0 0 6 16 76 76 76 3076 8076 152 153 QYAAAA NJFAAA VVVVxx +3840 3628 0 0 0 0 40 840 1840 3840 3840 80 81 SRAAAA OJFAAA AAAAxx +5955 3629 1 3 5 15 55 955 1955 955 5955 110 111 BVAAAA PJFAAA HHHHxx +994 3630 0 2 4 14 94 994 994 994 994 188 189 GMAAAA QJFAAA OOOOxx +2089 3631 1 1 9 9 89 89 89 2089 2089 178 179 JCAAAA RJFAAA VVVVxx +869 3632 1 1 9 9 69 869 869 869 869 138 139 LHAAAA SJFAAA AAAAxx +1223 3633 1 3 3 3 23 223 1223 1223 1223 46 47 BVAAAA TJFAAA HHHHxx +1514 3634 0 2 4 14 14 514 1514 1514 1514 28 29 GGAAAA UJFAAA OOOOxx +4891 3635 1 3 1 11 91 891 891 4891 4891 182 183 DGAAAA VJFAAA VVVVxx +4190 3636 0 2 0 10 90 190 190 4190 4190 180 181 EFAAAA WJFAAA AAAAxx +4377 3637 1 1 7 17 77 377 377 4377 4377 154 155 JMAAAA XJFAAA HHHHxx +9195 3638 1 3 5 15 95 195 1195 4195 9195 190 191 RPAAAA YJFAAA OOOOxx +3827 3639 1 3 7 7 27 827 1827 3827 3827 54 55 FRAAAA ZJFAAA VVVVxx +7386 3640 0 2 6 6 86 386 1386 2386 7386 172 173 CYAAAA AKFAAA AAAAxx +6665 3641 1 1 5 5 65 665 665 1665 6665 130 131 JWAAAA BKFAAA HHHHxx +7514 3642 0 2 4 14 14 514 1514 2514 7514 28 29 ADAAAA CKFAAA OOOOxx +6431 3643 1 3 1 11 31 431 431 1431 6431 62 63 JNAAAA DKFAAA VVVVxx +3251 3644 1 3 1 11 51 251 1251 3251 3251 102 103 BVAAAA EKFAAA AAAAxx +8439 3645 1 3 9 19 39 439 439 3439 8439 78 79 PMAAAA FKFAAA HHHHxx +831 3646 1 3 1 11 31 831 831 831 831 62 63 ZFAAAA GKFAAA OOOOxx +8485 3647 1 1 5 5 85 485 485 3485 8485 170 171 JOAAAA HKFAAA VVVVxx +7314 3648 0 2 4 14 14 314 1314 2314 7314 28 29 IVAAAA IKFAAA AAAAxx +3044 3649 0 0 4 4 44 44 1044 3044 3044 88 89 CNAAAA JKFAAA HHHHxx +4283 3650 1 3 3 3 83 283 283 4283 4283 166 167 TIAAAA KKFAAA OOOOxx +298 3651 0 2 8 18 98 298 298 298 298 196 197 MLAAAA LKFAAA VVVVxx +7114 3652 0 2 4 14 14 114 1114 2114 7114 28 29 QNAAAA MKFAAA AAAAxx +9664 3653 0 0 4 4 64 664 1664 4664 9664 128 129 SHAAAA NKFAAA HHHHxx +5315 3654 1 3 5 15 15 315 1315 315 5315 30 31 LWAAAA OKFAAA OOOOxx +2164 3655 0 0 4 4 64 164 164 2164 2164 128 129 GFAAAA PKFAAA VVVVxx +3390 3656 0 2 0 10 90 390 1390 3390 3390 180 181 KAAAAA QKFAAA AAAAxx +836 3657 0 0 6 16 36 836 836 836 836 72 73 EGAAAA RKFAAA HHHHxx +3316 3658 0 0 6 16 16 316 1316 3316 3316 32 33 OXAAAA SKFAAA OOOOxx +1284 3659 0 0 4 4 84 284 1284 1284 1284 168 169 KXAAAA TKFAAA VVVVxx +2497 3660 1 1 7 17 97 497 497 2497 2497 194 195 BSAAAA UKFAAA AAAAxx +1374 3661 0 2 4 14 74 374 1374 1374 1374 148 149 WAAAAA VKFAAA HHHHxx +9525 3662 1 1 5 5 25 525 1525 4525 9525 50 51 JCAAAA WKFAAA OOOOxx +2911 3663 1 3 1 11 11 911 911 2911 2911 22 23 ZHAAAA XKFAAA VVVVxx +9686 3664 0 2 6 6 86 686 1686 4686 9686 172 173 OIAAAA YKFAAA AAAAxx +584 3665 0 0 4 4 84 584 584 584 584 168 169 MWAAAA ZKFAAA HHHHxx +5653 3666 1 1 3 13 53 653 1653 653 5653 106 107 LJAAAA ALFAAA OOOOxx +4986 3667 0 2 6 6 86 986 986 4986 4986 172 173 UJAAAA BLFAAA VVVVxx +6049 3668 1 1 9 9 49 49 49 1049 6049 98 99 RYAAAA CLFAAA AAAAxx +9891 3669 1 3 1 11 91 891 1891 4891 9891 182 183 LQAAAA DLFAAA HHHHxx +8809 3670 1 1 9 9 9 809 809 3809 8809 18 19 VAAAAA ELFAAA OOOOxx +8598 3671 0 2 8 18 98 598 598 3598 8598 196 197 SSAAAA FLFAAA VVVVxx +2573 3672 1 1 3 13 73 573 573 2573 2573 146 147 ZUAAAA GLFAAA AAAAxx +6864 3673 0 0 4 4 64 864 864 1864 6864 128 129 AEAAAA HLFAAA HHHHxx +7932 3674 0 0 2 12 32 932 1932 2932 7932 64 65 CTAAAA ILFAAA OOOOxx +6605 3675 1 1 5 5 5 605 605 1605 6605 10 11 BUAAAA JLFAAA VVVVxx +9500 3676 0 0 0 0 0 500 1500 4500 9500 0 1 KBAAAA KLFAAA AAAAxx +8742 3677 0 2 2 2 42 742 742 3742 8742 84 85 GYAAAA LLFAAA HHHHxx +9815 3678 1 3 5 15 15 815 1815 4815 9815 30 31 NNAAAA MLFAAA OOOOxx +3319 3679 1 3 9 19 19 319 1319 3319 3319 38 39 RXAAAA NLFAAA VVVVxx +184 3680 0 0 4 4 84 184 184 184 184 168 169 CHAAAA OLFAAA AAAAxx +8886 3681 0 2 6 6 86 886 886 3886 8886 172 173 UDAAAA PLFAAA HHHHxx +7050 3682 0 2 0 10 50 50 1050 2050 7050 100 101 ELAAAA QLFAAA OOOOxx +9781 3683 1 1 1 1 81 781 1781 4781 9781 162 163 FMAAAA RLFAAA VVVVxx +2443 3684 1 3 3 3 43 443 443 2443 2443 86 87 ZPAAAA SLFAAA AAAAxx +1160 3685 0 0 0 0 60 160 1160 1160 1160 120 121 QSAAAA TLFAAA HHHHxx +4600 3686 0 0 0 0 0 600 600 4600 4600 0 1 YUAAAA ULFAAA OOOOxx +813 3687 1 1 3 13 13 813 813 813 813 26 27 HFAAAA VLFAAA VVVVxx +5078 3688 0 2 8 18 78 78 1078 78 5078 156 157 INAAAA WLFAAA AAAAxx +9008 3689 0 0 8 8 8 8 1008 4008 9008 16 17 MIAAAA XLFAAA HHHHxx +9016 3690 0 0 6 16 16 16 1016 4016 9016 32 33 UIAAAA YLFAAA OOOOxx +2747 3691 1 3 7 7 47 747 747 2747 2747 94 95 RBAAAA ZLFAAA VVVVxx +3106 3692 0 2 6 6 6 106 1106 3106 3106 12 13 MPAAAA AMFAAA AAAAxx +8235 3693 1 3 5 15 35 235 235 3235 8235 70 71 TEAAAA BMFAAA HHHHxx +5582 3694 0 2 2 2 82 582 1582 582 5582 164 165 SGAAAA CMFAAA OOOOxx +4334 3695 0 2 4 14 34 334 334 4334 4334 68 69 SKAAAA DMFAAA VVVVxx +1612 3696 0 0 2 12 12 612 1612 1612 1612 24 25 AKAAAA EMFAAA AAAAxx +5650 3697 0 2 0 10 50 650 1650 650 5650 100 101 IJAAAA FMFAAA HHHHxx +6086 3698 0 2 6 6 86 86 86 1086 6086 172 173 CAAAAA GMFAAA OOOOxx +9667 3699 1 3 7 7 67 667 1667 4667 9667 134 135 VHAAAA HMFAAA VVVVxx +4215 3700 1 3 5 15 15 215 215 4215 4215 30 31 DGAAAA IMFAAA AAAAxx +8553 3701 1 1 3 13 53 553 553 3553 8553 106 107 ZQAAAA JMFAAA HHHHxx +9066 3702 0 2 6 6 66 66 1066 4066 9066 132 133 SKAAAA KMFAAA OOOOxx +1092 3703 0 0 2 12 92 92 1092 1092 1092 184 185 AQAAAA LMFAAA VVVVxx +2848 3704 0 0 8 8 48 848 848 2848 2848 96 97 OFAAAA MMFAAA AAAAxx +2765 3705 1 1 5 5 65 765 765 2765 2765 130 131 JCAAAA NMFAAA HHHHxx +6513 3706 1 1 3 13 13 513 513 1513 6513 26 27 NQAAAA OMFAAA OOOOxx +6541 3707 1 1 1 1 41 541 541 1541 6541 82 83 PRAAAA PMFAAA VVVVxx +9617 3708 1 1 7 17 17 617 1617 4617 9617 34 35 XFAAAA QMFAAA AAAAxx +5870 3709 0 2 0 10 70 870 1870 870 5870 140 141 URAAAA RMFAAA HHHHxx +8811 3710 1 3 1 11 11 811 811 3811 8811 22 23 XAAAAA SMFAAA OOOOxx +4529 3711 1 1 9 9 29 529 529 4529 4529 58 59 FSAAAA TMFAAA VVVVxx +161 3712 1 1 1 1 61 161 161 161 161 122 123 FGAAAA UMFAAA AAAAxx +641 3713 1 1 1 1 41 641 641 641 641 82 83 RYAAAA VMFAAA HHHHxx +4767 3714 1 3 7 7 67 767 767 4767 4767 134 135 JBAAAA WMFAAA OOOOxx +6293 3715 1 1 3 13 93 293 293 1293 6293 186 187 BIAAAA XMFAAA VVVVxx +3816 3716 0 0 6 16 16 816 1816 3816 3816 32 33 UQAAAA YMFAAA AAAAxx +4748 3717 0 0 8 8 48 748 748 4748 4748 96 97 QAAAAA ZMFAAA HHHHxx +9924 3718 0 0 4 4 24 924 1924 4924 9924 48 49 SRAAAA ANFAAA OOOOxx +6716 3719 0 0 6 16 16 716 716 1716 6716 32 33 IYAAAA BNFAAA VVVVxx +8828 3720 0 0 8 8 28 828 828 3828 8828 56 57 OBAAAA CNFAAA AAAAxx +4967 3721 1 3 7 7 67 967 967 4967 4967 134 135 BJAAAA DNFAAA HHHHxx +9680 3722 0 0 0 0 80 680 1680 4680 9680 160 161 IIAAAA ENFAAA OOOOxx +2784 3723 0 0 4 4 84 784 784 2784 2784 168 169 CDAAAA FNFAAA VVVVxx +2882 3724 0 2 2 2 82 882 882 2882 2882 164 165 WGAAAA GNFAAA AAAAxx +3641 3725 1 1 1 1 41 641 1641 3641 3641 82 83 BKAAAA HNFAAA HHHHxx +5537 3726 1 1 7 17 37 537 1537 537 5537 74 75 ZEAAAA INFAAA OOOOxx +820 3727 0 0 0 0 20 820 820 820 820 40 41 OFAAAA JNFAAA VVVVxx +5847 3728 1 3 7 7 47 847 1847 847 5847 94 95 XQAAAA KNFAAA AAAAxx +566 3729 0 2 6 6 66 566 566 566 566 132 133 UVAAAA LNFAAA HHHHxx +2246 3730 0 2 6 6 46 246 246 2246 2246 92 93 KIAAAA MNFAAA OOOOxx +6680 3731 0 0 0 0 80 680 680 1680 6680 160 161 YWAAAA NNFAAA VVVVxx +2014 3732 0 2 4 14 14 14 14 2014 2014 28 29 MZAAAA ONFAAA AAAAxx +8355 3733 1 3 5 15 55 355 355 3355 8355 110 111 JJAAAA PNFAAA HHHHxx +1610 3734 0 2 0 10 10 610 1610 1610 1610 20 21 YJAAAA QNFAAA OOOOxx +9719 3735 1 3 9 19 19 719 1719 4719 9719 38 39 VJAAAA RNFAAA VVVVxx +8498 3736 0 2 8 18 98 498 498 3498 8498 196 197 WOAAAA SNFAAA AAAAxx +5883 3737 1 3 3 3 83 883 1883 883 5883 166 167 HSAAAA TNFAAA HHHHxx +7380 3738 0 0 0 0 80 380 1380 2380 7380 160 161 WXAAAA UNFAAA OOOOxx +8865 3739 1 1 5 5 65 865 865 3865 8865 130 131 ZCAAAA VNFAAA VVVVxx +4743 3740 1 3 3 3 43 743 743 4743 4743 86 87 LAAAAA WNFAAA AAAAxx +5086 3741 0 2 6 6 86 86 1086 86 5086 172 173 QNAAAA XNFAAA HHHHxx +2739 3742 1 3 9 19 39 739 739 2739 2739 78 79 JBAAAA YNFAAA OOOOxx +9375 3743 1 3 5 15 75 375 1375 4375 9375 150 151 PWAAAA ZNFAAA VVVVxx +7876 3744 0 0 6 16 76 876 1876 2876 7876 152 153 YQAAAA AOFAAA AAAAxx +453 3745 1 1 3 13 53 453 453 453 453 106 107 LRAAAA BOFAAA HHHHxx +6987 3746 1 3 7 7 87 987 987 1987 6987 174 175 TIAAAA COFAAA OOOOxx +2860 3747 0 0 0 0 60 860 860 2860 2860 120 121 AGAAAA DOFAAA VVVVxx +8372 3748 0 0 2 12 72 372 372 3372 8372 144 145 AKAAAA EOFAAA AAAAxx +2048 3749 0 0 8 8 48 48 48 2048 2048 96 97 UAAAAA FOFAAA HHHHxx +9231 3750 1 3 1 11 31 231 1231 4231 9231 62 63 BRAAAA GOFAAA OOOOxx +634 3751 0 2 4 14 34 634 634 634 634 68 69 KYAAAA HOFAAA VVVVxx +3998 3752 0 2 8 18 98 998 1998 3998 3998 196 197 UXAAAA IOFAAA AAAAxx +4728 3753 0 0 8 8 28 728 728 4728 4728 56 57 WZAAAA JOFAAA HHHHxx +579 3754 1 3 9 19 79 579 579 579 579 158 159 HWAAAA KOFAAA OOOOxx +815 3755 1 3 5 15 15 815 815 815 815 30 31 JFAAAA LOFAAA VVVVxx +1009 3756 1 1 9 9 9 9 1009 1009 1009 18 19 VMAAAA MOFAAA AAAAxx +6596 3757 0 0 6 16 96 596 596 1596 6596 192 193 STAAAA NOFAAA HHHHxx +2793 3758 1 1 3 13 93 793 793 2793 2793 186 187 LDAAAA OOFAAA OOOOxx +9589 3759 1 1 9 9 89 589 1589 4589 9589 178 179 VEAAAA POFAAA VVVVxx +2794 3760 0 2 4 14 94 794 794 2794 2794 188 189 MDAAAA QOFAAA AAAAxx +2551 3761 1 3 1 11 51 551 551 2551 2551 102 103 DUAAAA ROFAAA HHHHxx +1588 3762 0 0 8 8 88 588 1588 1588 1588 176 177 CJAAAA SOFAAA OOOOxx +4443 3763 1 3 3 3 43 443 443 4443 4443 86 87 XOAAAA TOFAAA VVVVxx +5009 3764 1 1 9 9 9 9 1009 9 5009 18 19 RKAAAA UOFAAA AAAAxx +4287 3765 1 3 7 7 87 287 287 4287 4287 174 175 XIAAAA VOFAAA HHHHxx +2167 3766 1 3 7 7 67 167 167 2167 2167 134 135 JFAAAA WOFAAA OOOOxx +2290 3767 0 2 0 10 90 290 290 2290 2290 180 181 CKAAAA XOFAAA VVVVxx +7225 3768 1 1 5 5 25 225 1225 2225 7225 50 51 XRAAAA YOFAAA AAAAxx +8992 3769 0 0 2 12 92 992 992 3992 8992 184 185 WHAAAA ZOFAAA HHHHxx +1540 3770 0 0 0 0 40 540 1540 1540 1540 80 81 GHAAAA APFAAA OOOOxx +2029 3771 1 1 9 9 29 29 29 2029 2029 58 59 BAAAAA BPFAAA VVVVxx +2855 3772 1 3 5 15 55 855 855 2855 2855 110 111 VFAAAA CPFAAA AAAAxx +3534 3773 0 2 4 14 34 534 1534 3534 3534 68 69 YFAAAA DPFAAA HHHHxx +8078 3774 0 2 8 18 78 78 78 3078 8078 156 157 SYAAAA EPFAAA OOOOxx +9778 3775 0 2 8 18 78 778 1778 4778 9778 156 157 CMAAAA FPFAAA VVVVxx +3543 3776 1 3 3 3 43 543 1543 3543 3543 86 87 HGAAAA GPFAAA AAAAxx +4778 3777 0 2 8 18 78 778 778 4778 4778 156 157 UBAAAA HPFAAA HHHHxx +8931 3778 1 3 1 11 31 931 931 3931 8931 62 63 NFAAAA IPFAAA OOOOxx +557 3779 1 1 7 17 57 557 557 557 557 114 115 LVAAAA JPFAAA VVVVxx +5546 3780 0 2 6 6 46 546 1546 546 5546 92 93 IFAAAA KPFAAA AAAAxx +7527 3781 1 3 7 7 27 527 1527 2527 7527 54 55 NDAAAA LPFAAA HHHHxx +5000 3782 0 0 0 0 0 0 1000 0 5000 0 1 IKAAAA MPFAAA OOOOxx +7587 3783 1 3 7 7 87 587 1587 2587 7587 174 175 VFAAAA NPFAAA VVVVxx +3014 3784 0 2 4 14 14 14 1014 3014 3014 28 29 YLAAAA OPFAAA AAAAxx +5276 3785 0 0 6 16 76 276 1276 276 5276 152 153 YUAAAA PPFAAA HHHHxx +6457 3786 1 1 7 17 57 457 457 1457 6457 114 115 JOAAAA QPFAAA OOOOxx +389 3787 1 1 9 9 89 389 389 389 389 178 179 ZOAAAA RPFAAA VVVVxx +7104 3788 0 0 4 4 4 104 1104 2104 7104 8 9 GNAAAA SPFAAA AAAAxx +9995 3789 1 3 5 15 95 995 1995 4995 9995 190 191 LUAAAA TPFAAA HHHHxx +7368 3790 0 0 8 8 68 368 1368 2368 7368 136 137 KXAAAA UPFAAA OOOOxx +3258 3791 0 2 8 18 58 258 1258 3258 3258 116 117 IVAAAA VPFAAA VVVVxx +9208 3792 0 0 8 8 8 208 1208 4208 9208 16 17 EQAAAA WPFAAA AAAAxx +2396 3793 0 0 6 16 96 396 396 2396 2396 192 193 EOAAAA XPFAAA HHHHxx +1715 3794 1 3 5 15 15 715 1715 1715 1715 30 31 ZNAAAA YPFAAA OOOOxx +1240 3795 0 0 0 0 40 240 1240 1240 1240 80 81 SVAAAA ZPFAAA VVVVxx +1952 3796 0 0 2 12 52 952 1952 1952 1952 104 105 CXAAAA AQFAAA AAAAxx +4403 3797 1 3 3 3 3 403 403 4403 4403 6 7 JNAAAA BQFAAA HHHHxx +6333 3798 1 1 3 13 33 333 333 1333 6333 66 67 PJAAAA CQFAAA OOOOxx +2492 3799 0 0 2 12 92 492 492 2492 2492 184 185 WRAAAA DQFAAA VVVVxx +6543 3800 1 3 3 3 43 543 543 1543 6543 86 87 RRAAAA EQFAAA AAAAxx +5548 3801 0 0 8 8 48 548 1548 548 5548 96 97 KFAAAA FQFAAA HHHHxx +3458 3802 0 2 8 18 58 458 1458 3458 3458 116 117 ADAAAA GQFAAA OOOOxx +2588 3803 0 0 8 8 88 588 588 2588 2588 176 177 OVAAAA HQFAAA VVVVxx +1364 3804 0 0 4 4 64 364 1364 1364 1364 128 129 MAAAAA IQFAAA AAAAxx +9856 3805 0 0 6 16 56 856 1856 4856 9856 112 113 CPAAAA JQFAAA HHHHxx +4964 3806 0 0 4 4 64 964 964 4964 4964 128 129 YIAAAA KQFAAA OOOOxx +773 3807 1 1 3 13 73 773 773 773 773 146 147 TDAAAA LQFAAA VVVVxx +6402 3808 0 2 2 2 2 402 402 1402 6402 4 5 GMAAAA MQFAAA AAAAxx +7213 3809 1 1 3 13 13 213 1213 2213 7213 26 27 LRAAAA NQFAAA HHHHxx +3385 3810 1 1 5 5 85 385 1385 3385 3385 170 171 FAAAAA OQFAAA OOOOxx +6005 3811 1 1 5 5 5 5 5 1005 6005 10 11 ZWAAAA PQFAAA VVVVxx +9346 3812 0 2 6 6 46 346 1346 4346 9346 92 93 MVAAAA QQFAAA AAAAxx +1831 3813 1 3 1 11 31 831 1831 1831 1831 62 63 LSAAAA RQFAAA HHHHxx +5406 3814 0 2 6 6 6 406 1406 406 5406 12 13 YZAAAA SQFAAA OOOOxx +2154 3815 0 2 4 14 54 154 154 2154 2154 108 109 WEAAAA TQFAAA VVVVxx +3721 3816 1 1 1 1 21 721 1721 3721 3721 42 43 DNAAAA UQFAAA AAAAxx +2889 3817 1 1 9 9 89 889 889 2889 2889 178 179 DHAAAA VQFAAA HHHHxx +4410 3818 0 2 0 10 10 410 410 4410 4410 20 21 QNAAAA WQFAAA OOOOxx +7102 3819 0 2 2 2 2 102 1102 2102 7102 4 5 ENAAAA XQFAAA VVVVxx +4057 3820 1 1 7 17 57 57 57 4057 4057 114 115 BAAAAA YQFAAA AAAAxx +9780 3821 0 0 0 0 80 780 1780 4780 9780 160 161 EMAAAA ZQFAAA HHHHxx +9481 3822 1 1 1 1 81 481 1481 4481 9481 162 163 RAAAAA ARFAAA OOOOxx +2366 3823 0 2 6 6 66 366 366 2366 2366 132 133 ANAAAA BRFAAA VVVVxx +2708 3824 0 0 8 8 8 708 708 2708 2708 16 17 EAAAAA CRFAAA AAAAxx +7399 3825 1 3 9 19 99 399 1399 2399 7399 198 199 PYAAAA DRFAAA HHHHxx +5234 3826 0 2 4 14 34 234 1234 234 5234 68 69 ITAAAA ERFAAA OOOOxx +1843 3827 1 3 3 3 43 843 1843 1843 1843 86 87 XSAAAA FRFAAA VVVVxx +1006 3828 0 2 6 6 6 6 1006 1006 1006 12 13 SMAAAA GRFAAA AAAAxx +7696 3829 0 0 6 16 96 696 1696 2696 7696 192 193 AKAAAA HRFAAA HHHHxx +6411 3830 1 3 1 11 11 411 411 1411 6411 22 23 PMAAAA IRFAAA OOOOxx +3913 3831 1 1 3 13 13 913 1913 3913 3913 26 27 NUAAAA JRFAAA VVVVxx +2538 3832 0 2 8 18 38 538 538 2538 2538 76 77 QTAAAA KRFAAA AAAAxx +3019 3833 1 3 9 19 19 19 1019 3019 3019 38 39 DMAAAA LRFAAA HHHHxx +107 3834 1 3 7 7 7 107 107 107 107 14 15 DEAAAA MRFAAA OOOOxx +427 3835 1 3 7 7 27 427 427 427 427 54 55 LQAAAA NRFAAA VVVVxx +9849 3836 1 1 9 9 49 849 1849 4849 9849 98 99 VOAAAA ORFAAA AAAAxx +4195 3837 1 3 5 15 95 195 195 4195 4195 190 191 JFAAAA PRFAAA HHHHxx +9215 3838 1 3 5 15 15 215 1215 4215 9215 30 31 LQAAAA QRFAAA OOOOxx +3165 3839 1 1 5 5 65 165 1165 3165 3165 130 131 TRAAAA RRFAAA VVVVxx +3280 3840 0 0 0 0 80 280 1280 3280 3280 160 161 EWAAAA SRFAAA AAAAxx +4477 3841 1 1 7 17 77 477 477 4477 4477 154 155 FQAAAA TRFAAA HHHHxx +5885 3842 1 1 5 5 85 885 1885 885 5885 170 171 JSAAAA URFAAA OOOOxx +3311 3843 1 3 1 11 11 311 1311 3311 3311 22 23 JXAAAA VRFAAA VVVVxx +6453 3844 1 1 3 13 53 453 453 1453 6453 106 107 FOAAAA WRFAAA AAAAxx +8527 3845 1 3 7 7 27 527 527 3527 8527 54 55 ZPAAAA XRFAAA HHHHxx +1921 3846 1 1 1 1 21 921 1921 1921 1921 42 43 XVAAAA YRFAAA OOOOxx +2427 3847 1 3 7 7 27 427 427 2427 2427 54 55 JPAAAA ZRFAAA VVVVxx +3691 3848 1 3 1 11 91 691 1691 3691 3691 182 183 ZLAAAA ASFAAA AAAAxx +3882 3849 0 2 2 2 82 882 1882 3882 3882 164 165 ITAAAA BSFAAA HHHHxx +562 3850 0 2 2 2 62 562 562 562 562 124 125 QVAAAA CSFAAA OOOOxx +377 3851 1 1 7 17 77 377 377 377 377 154 155 NOAAAA DSFAAA VVVVxx +1497 3852 1 1 7 17 97 497 1497 1497 1497 194 195 PFAAAA ESFAAA AAAAxx +4453 3853 1 1 3 13 53 453 453 4453 4453 106 107 HPAAAA FSFAAA HHHHxx +4678 3854 0 2 8 18 78 678 678 4678 4678 156 157 YXAAAA GSFAAA OOOOxx +2234 3855 0 2 4 14 34 234 234 2234 2234 68 69 YHAAAA HSFAAA VVVVxx +1073 3856 1 1 3 13 73 73 1073 1073 1073 146 147 HPAAAA ISFAAA AAAAxx +6479 3857 1 3 9 19 79 479 479 1479 6479 158 159 FPAAAA JSFAAA HHHHxx +5665 3858 1 1 5 5 65 665 1665 665 5665 130 131 XJAAAA KSFAAA OOOOxx +586 3859 0 2 6 6 86 586 586 586 586 172 173 OWAAAA LSFAAA VVVVxx +1584 3860 0 0 4 4 84 584 1584 1584 1584 168 169 YIAAAA MSFAAA AAAAxx +2574 3861 0 2 4 14 74 574 574 2574 2574 148 149 AVAAAA NSFAAA HHHHxx +9833 3862 1 1 3 13 33 833 1833 4833 9833 66 67 FOAAAA OSFAAA OOOOxx +6726 3863 0 2 6 6 26 726 726 1726 6726 52 53 SYAAAA PSFAAA VVVVxx +8497 3864 1 1 7 17 97 497 497 3497 8497 194 195 VOAAAA QSFAAA AAAAxx +2914 3865 0 2 4 14 14 914 914 2914 2914 28 29 CIAAAA RSFAAA HHHHxx +8586 3866 0 2 6 6 86 586 586 3586 8586 172 173 GSAAAA SSFAAA OOOOxx +6973 3867 1 1 3 13 73 973 973 1973 6973 146 147 FIAAAA TSFAAA VVVVxx +1322 3868 0 2 2 2 22 322 1322 1322 1322 44 45 WYAAAA USFAAA AAAAxx +5242 3869 0 2 2 2 42 242 1242 242 5242 84 85 QTAAAA VSFAAA HHHHxx +5581 3870 1 1 1 1 81 581 1581 581 5581 162 163 RGAAAA WSFAAA OOOOxx +1365 3871 1 1 5 5 65 365 1365 1365 1365 130 131 NAAAAA XSFAAA VVVVxx +2818 3872 0 2 8 18 18 818 818 2818 2818 36 37 KEAAAA YSFAAA AAAAxx +3758 3873 0 2 8 18 58 758 1758 3758 3758 116 117 OOAAAA ZSFAAA HHHHxx +2665 3874 1 1 5 5 65 665 665 2665 2665 130 131 NYAAAA ATFAAA OOOOxx +9823 3875 1 3 3 3 23 823 1823 4823 9823 46 47 VNAAAA BTFAAA VVVVxx +7057 3876 1 1 7 17 57 57 1057 2057 7057 114 115 LLAAAA CTFAAA AAAAxx +543 3877 1 3 3 3 43 543 543 543 543 86 87 XUAAAA DTFAAA HHHHxx +4008 3878 0 0 8 8 8 8 8 4008 4008 16 17 EYAAAA ETFAAA OOOOxx +4397 3879 1 1 7 17 97 397 397 4397 4397 194 195 DNAAAA FTFAAA VVVVxx +8533 3880 1 1 3 13 33 533 533 3533 8533 66 67 FQAAAA GTFAAA AAAAxx +9728 3881 0 0 8 8 28 728 1728 4728 9728 56 57 EKAAAA HTFAAA HHHHxx +5198 3882 0 2 8 18 98 198 1198 198 5198 196 197 YRAAAA ITFAAA OOOOxx +5036 3883 0 0 6 16 36 36 1036 36 5036 72 73 SLAAAA JTFAAA VVVVxx +4394 3884 0 2 4 14 94 394 394 4394 4394 188 189 ANAAAA KTFAAA AAAAxx +9633 3885 1 1 3 13 33 633 1633 4633 9633 66 67 NGAAAA LTFAAA HHHHxx +3339 3886 1 3 9 19 39 339 1339 3339 3339 78 79 LYAAAA MTFAAA OOOOxx +9529 3887 1 1 9 9 29 529 1529 4529 9529 58 59 NCAAAA NTFAAA VVVVxx +4780 3888 0 0 0 0 80 780 780 4780 4780 160 161 WBAAAA OTFAAA AAAAxx +4862 3889 0 2 2 2 62 862 862 4862 4862 124 125 AFAAAA PTFAAA HHHHxx +8152 3890 0 0 2 12 52 152 152 3152 8152 104 105 OBAAAA QTFAAA OOOOxx +9330 3891 0 2 0 10 30 330 1330 4330 9330 60 61 WUAAAA RTFAAA VVVVxx +4362 3892 0 2 2 2 62 362 362 4362 4362 124 125 ULAAAA STFAAA AAAAxx +4688 3893 0 0 8 8 88 688 688 4688 4688 176 177 IYAAAA TTFAAA HHHHxx +1903 3894 1 3 3 3 3 903 1903 1903 1903 6 7 FVAAAA UTFAAA OOOOxx +9027 3895 1 3 7 7 27 27 1027 4027 9027 54 55 FJAAAA VTFAAA VVVVxx +5385 3896 1 1 5 5 85 385 1385 385 5385 170 171 DZAAAA WTFAAA AAAAxx +9854 3897 0 2 4 14 54 854 1854 4854 9854 108 109 APAAAA XTFAAA HHHHxx +9033 3898 1 1 3 13 33 33 1033 4033 9033 66 67 LJAAAA YTFAAA OOOOxx +3185 3899 1 1 5 5 85 185 1185 3185 3185 170 171 NSAAAA ZTFAAA VVVVxx +2618 3900 0 2 8 18 18 618 618 2618 2618 36 37 SWAAAA AUFAAA AAAAxx +371 3901 1 3 1 11 71 371 371 371 371 142 143 HOAAAA BUFAAA HHHHxx +3697 3902 1 1 7 17 97 697 1697 3697 3697 194 195 FMAAAA CUFAAA OOOOxx +1682 3903 0 2 2 2 82 682 1682 1682 1682 164 165 SMAAAA DUFAAA VVVVxx +3333 3904 1 1 3 13 33 333 1333 3333 3333 66 67 FYAAAA EUFAAA AAAAxx +1722 3905 0 2 2 2 22 722 1722 1722 1722 44 45 GOAAAA FUFAAA HHHHxx +2009 3906 1 1 9 9 9 9 9 2009 2009 18 19 HZAAAA GUFAAA OOOOxx +3517 3907 1 1 7 17 17 517 1517 3517 3517 34 35 HFAAAA HUFAAA VVVVxx +7640 3908 0 0 0 0 40 640 1640 2640 7640 80 81 WHAAAA IUFAAA AAAAxx +259 3909 1 3 9 19 59 259 259 259 259 118 119 ZJAAAA JUFAAA HHHHxx +1400 3910 0 0 0 0 0 400 1400 1400 1400 0 1 WBAAAA KUFAAA OOOOxx +6663 3911 1 3 3 3 63 663 663 1663 6663 126 127 HWAAAA LUFAAA VVVVxx +1576 3912 0 0 6 16 76 576 1576 1576 1576 152 153 QIAAAA MUFAAA AAAAxx +8843 3913 1 3 3 3 43 843 843 3843 8843 86 87 DCAAAA NUFAAA HHHHxx +9474 3914 0 2 4 14 74 474 1474 4474 9474 148 149 KAAAAA OUFAAA OOOOxx +1597 3915 1 1 7 17 97 597 1597 1597 1597 194 195 LJAAAA PUFAAA VVVVxx +1143 3916 1 3 3 3 43 143 1143 1143 1143 86 87 ZRAAAA QUFAAA AAAAxx +4162 3917 0 2 2 2 62 162 162 4162 4162 124 125 CEAAAA RUFAAA HHHHxx +1301 3918 1 1 1 1 1 301 1301 1301 1301 2 3 BYAAAA SUFAAA OOOOxx +2935 3919 1 3 5 15 35 935 935 2935 2935 70 71 XIAAAA TUFAAA VVVVxx +886 3920 0 2 6 6 86 886 886 886 886 172 173 CIAAAA UUFAAA AAAAxx +1661 3921 1 1 1 1 61 661 1661 1661 1661 122 123 XLAAAA VUFAAA HHHHxx +1026 3922 0 2 6 6 26 26 1026 1026 1026 52 53 MNAAAA WUFAAA OOOOxx +7034 3923 0 2 4 14 34 34 1034 2034 7034 68 69 OKAAAA XUFAAA VVVVxx +2305 3924 1 1 5 5 5 305 305 2305 2305 10 11 RKAAAA YUFAAA AAAAxx +1725 3925 1 1 5 5 25 725 1725 1725 1725 50 51 JOAAAA ZUFAAA HHHHxx +909 3926 1 1 9 9 9 909 909 909 909 18 19 ZIAAAA AVFAAA OOOOxx +9906 3927 0 2 6 6 6 906 1906 4906 9906 12 13 ARAAAA BVFAAA VVVVxx +3309 3928 1 1 9 9 9 309 1309 3309 3309 18 19 HXAAAA CVFAAA AAAAxx +515 3929 1 3 5 15 15 515 515 515 515 30 31 VTAAAA DVFAAA HHHHxx +932 3930 0 0 2 12 32 932 932 932 932 64 65 WJAAAA EVFAAA OOOOxx +8144 3931 0 0 4 4 44 144 144 3144 8144 88 89 GBAAAA FVFAAA VVVVxx +5592 3932 0 0 2 12 92 592 1592 592 5592 184 185 CHAAAA GVFAAA AAAAxx +4003 3933 1 3 3 3 3 3 3 4003 4003 6 7 ZXAAAA HVFAAA HHHHxx +9566 3934 0 2 6 6 66 566 1566 4566 9566 132 133 YDAAAA IVFAAA OOOOxx +4556 3935 0 0 6 16 56 556 556 4556 4556 112 113 GTAAAA JVFAAA VVVVxx +268 3936 0 0 8 8 68 268 268 268 268 136 137 IKAAAA KVFAAA AAAAxx +8107 3937 1 3 7 7 7 107 107 3107 8107 14 15 VZAAAA LVFAAA HHHHxx +5816 3938 0 0 6 16 16 816 1816 816 5816 32 33 SPAAAA MVFAAA OOOOxx +8597 3939 1 1 7 17 97 597 597 3597 8597 194 195 RSAAAA NVFAAA VVVVxx +9611 3940 1 3 1 11 11 611 1611 4611 9611 22 23 RFAAAA OVFAAA AAAAxx +8070 3941 0 2 0 10 70 70 70 3070 8070 140 141 KYAAAA PVFAAA HHHHxx +6040 3942 0 0 0 0 40 40 40 1040 6040 80 81 IYAAAA QVFAAA OOOOxx +3184 3943 0 0 4 4 84 184 1184 3184 3184 168 169 MSAAAA RVFAAA VVVVxx +9656 3944 0 0 6 16 56 656 1656 4656 9656 112 113 KHAAAA SVFAAA AAAAxx +1577 3945 1 1 7 17 77 577 1577 1577 1577 154 155 RIAAAA TVFAAA HHHHxx +1805 3946 1 1 5 5 5 805 1805 1805 1805 10 11 LRAAAA UVFAAA OOOOxx +8268 3947 0 0 8 8 68 268 268 3268 8268 136 137 AGAAAA VVFAAA VVVVxx +3489 3948 1 1 9 9 89 489 1489 3489 3489 178 179 FEAAAA WVFAAA AAAAxx +4564 3949 0 0 4 4 64 564 564 4564 4564 128 129 OTAAAA XVFAAA HHHHxx +4006 3950 0 2 6 6 6 6 6 4006 4006 12 13 CYAAAA YVFAAA OOOOxx +8466 3951 0 2 6 6 66 466 466 3466 8466 132 133 QNAAAA ZVFAAA VVVVxx +938 3952 0 2 8 18 38 938 938 938 938 76 77 CKAAAA AWFAAA AAAAxx +5944 3953 0 0 4 4 44 944 1944 944 5944 88 89 QUAAAA BWFAAA HHHHxx +8363 3954 1 3 3 3 63 363 363 3363 8363 126 127 RJAAAA CWFAAA OOOOxx +5348 3955 0 0 8 8 48 348 1348 348 5348 96 97 SXAAAA DWFAAA VVVVxx +71 3956 1 3 1 11 71 71 71 71 71 142 143 TCAAAA EWFAAA AAAAxx +3620 3957 0 0 0 0 20 620 1620 3620 3620 40 41 GJAAAA FWFAAA HHHHxx +3230 3958 0 2 0 10 30 230 1230 3230 3230 60 61 GUAAAA GWFAAA OOOOxx +6132 3959 0 0 2 12 32 132 132 1132 6132 64 65 WBAAAA HWFAAA VVVVxx +6143 3960 1 3 3 3 43 143 143 1143 6143 86 87 HCAAAA IWFAAA AAAAxx +8781 3961 1 1 1 1 81 781 781 3781 8781 162 163 TZAAAA JWFAAA HHHHxx +5522 3962 0 2 2 2 22 522 1522 522 5522 44 45 KEAAAA KWFAAA OOOOxx +6320 3963 0 0 0 0 20 320 320 1320 6320 40 41 CJAAAA LWFAAA VVVVxx +3923 3964 1 3 3 3 23 923 1923 3923 3923 46 47 XUAAAA MWFAAA AAAAxx +2207 3965 1 3 7 7 7 207 207 2207 2207 14 15 XGAAAA NWFAAA HHHHxx +966 3966 0 2 6 6 66 966 966 966 966 132 133 ELAAAA OWFAAA OOOOxx +9020 3967 0 0 0 0 20 20 1020 4020 9020 40 41 YIAAAA PWFAAA VVVVxx +4616 3968 0 0 6 16 16 616 616 4616 4616 32 33 OVAAAA QWFAAA AAAAxx +8289 3969 1 1 9 9 89 289 289 3289 8289 178 179 VGAAAA RWFAAA HHHHxx +5796 3970 0 0 6 16 96 796 1796 796 5796 192 193 YOAAAA SWFAAA OOOOxx +9259 3971 1 3 9 19 59 259 1259 4259 9259 118 119 DSAAAA TWFAAA VVVVxx +3710 3972 0 2 0 10 10 710 1710 3710 3710 20 21 SMAAAA UWFAAA AAAAxx +251 3973 1 3 1 11 51 251 251 251 251 102 103 RJAAAA VWFAAA HHHHxx +7669 3974 1 1 9 9 69 669 1669 2669 7669 138 139 ZIAAAA WWFAAA OOOOxx +6304 3975 0 0 4 4 4 304 304 1304 6304 8 9 MIAAAA XWFAAA VVVVxx +6454 3976 0 2 4 14 54 454 454 1454 6454 108 109 GOAAAA YWFAAA AAAAxx +1489 3977 1 1 9 9 89 489 1489 1489 1489 178 179 HFAAAA ZWFAAA HHHHxx +715 3978 1 3 5 15 15 715 715 715 715 30 31 NBAAAA AXFAAA OOOOxx +4319 3979 1 3 9 19 19 319 319 4319 4319 38 39 DKAAAA BXFAAA VVVVxx +7112 3980 0 0 2 12 12 112 1112 2112 7112 24 25 ONAAAA CXFAAA AAAAxx +3726 3981 0 2 6 6 26 726 1726 3726 3726 52 53 INAAAA DXFAAA HHHHxx +7727 3982 1 3 7 7 27 727 1727 2727 7727 54 55 FLAAAA EXFAAA OOOOxx +8387 3983 1 3 7 7 87 387 387 3387 8387 174 175 PKAAAA FXFAAA VVVVxx +6555 3984 1 3 5 15 55 555 555 1555 6555 110 111 DSAAAA GXFAAA AAAAxx +1148 3985 0 0 8 8 48 148 1148 1148 1148 96 97 ESAAAA HXFAAA HHHHxx +9000 3986 0 0 0 0 0 0 1000 4000 9000 0 1 EIAAAA IXFAAA OOOOxx +5278 3987 0 2 8 18 78 278 1278 278 5278 156 157 AVAAAA JXFAAA VVVVxx +2388 3988 0 0 8 8 88 388 388 2388 2388 176 177 WNAAAA KXFAAA AAAAxx +7984 3989 0 0 4 4 84 984 1984 2984 7984 168 169 CVAAAA LXFAAA HHHHxx +881 3990 1 1 1 1 81 881 881 881 881 162 163 XHAAAA MXFAAA OOOOxx +6830 3991 0 2 0 10 30 830 830 1830 6830 60 61 SCAAAA NXFAAA VVVVxx +7056 3992 0 0 6 16 56 56 1056 2056 7056 112 113 KLAAAA OXFAAA AAAAxx +7581 3993 1 1 1 1 81 581 1581 2581 7581 162 163 PFAAAA PXFAAA HHHHxx +5214 3994 0 2 4 14 14 214 1214 214 5214 28 29 OSAAAA QXFAAA OOOOxx +2505 3995 1 1 5 5 5 505 505 2505 2505 10 11 JSAAAA RXFAAA VVVVxx +5112 3996 0 0 2 12 12 112 1112 112 5112 24 25 QOAAAA SXFAAA AAAAxx +9884 3997 0 0 4 4 84 884 1884 4884 9884 168 169 EQAAAA TXFAAA HHHHxx +8040 3998 0 0 0 0 40 40 40 3040 8040 80 81 GXAAAA UXFAAA OOOOxx +7033 3999 1 1 3 13 33 33 1033 2033 7033 66 67 NKAAAA VXFAAA VVVVxx +9343 4000 1 3 3 3 43 343 1343 4343 9343 86 87 JVAAAA WXFAAA AAAAxx +2931 4001 1 3 1 11 31 931 931 2931 2931 62 63 TIAAAA XXFAAA HHHHxx +9024 4002 0 0 4 4 24 24 1024 4024 9024 48 49 CJAAAA YXFAAA OOOOxx +6485 4003 1 1 5 5 85 485 485 1485 6485 170 171 LPAAAA ZXFAAA VVVVxx +3465 4004 1 1 5 5 65 465 1465 3465 3465 130 131 HDAAAA AYFAAA AAAAxx +3357 4005 1 1 7 17 57 357 1357 3357 3357 114 115 DZAAAA BYFAAA HHHHxx +2929 4006 1 1 9 9 29 929 929 2929 2929 58 59 RIAAAA CYFAAA OOOOxx +3086 4007 0 2 6 6 86 86 1086 3086 3086 172 173 SOAAAA DYFAAA VVVVxx +8897 4008 1 1 7 17 97 897 897 3897 8897 194 195 FEAAAA EYFAAA AAAAxx +9688 4009 0 0 8 8 88 688 1688 4688 9688 176 177 QIAAAA FYFAAA HHHHxx +6522 4010 0 2 2 2 22 522 522 1522 6522 44 45 WQAAAA GYFAAA OOOOxx +3241 4011 1 1 1 1 41 241 1241 3241 3241 82 83 RUAAAA HYFAAA VVVVxx +8770 4012 0 2 0 10 70 770 770 3770 8770 140 141 IZAAAA IYFAAA AAAAxx +2884 4013 0 0 4 4 84 884 884 2884 2884 168 169 YGAAAA JYFAAA HHHHxx +9579 4014 1 3 9 19 79 579 1579 4579 9579 158 159 LEAAAA KYFAAA OOOOxx +3125 4015 1 1 5 5 25 125 1125 3125 3125 50 51 FQAAAA LYFAAA VVVVxx +4604 4016 0 0 4 4 4 604 604 4604 4604 8 9 CVAAAA MYFAAA AAAAxx +2682 4017 0 2 2 2 82 682 682 2682 2682 164 165 EZAAAA NYFAAA HHHHxx +254 4018 0 2 4 14 54 254 254 254 254 108 109 UJAAAA OYFAAA OOOOxx +6569 4019 1 1 9 9 69 569 569 1569 6569 138 139 RSAAAA PYFAAA VVVVxx +2686 4020 0 2 6 6 86 686 686 2686 2686 172 173 IZAAAA QYFAAA AAAAxx +2123 4021 1 3 3 3 23 123 123 2123 2123 46 47 RDAAAA RYFAAA HHHHxx +1745 4022 1 1 5 5 45 745 1745 1745 1745 90 91 DPAAAA SYFAAA OOOOxx +247 4023 1 3 7 7 47 247 247 247 247 94 95 NJAAAA TYFAAA VVVVxx +5800 4024 0 0 0 0 0 800 1800 800 5800 0 1 CPAAAA UYFAAA AAAAxx +1121 4025 1 1 1 1 21 121 1121 1121 1121 42 43 DRAAAA VYFAAA HHHHxx +8893 4026 1 1 3 13 93 893 893 3893 8893 186 187 BEAAAA WYFAAA OOOOxx +7819 4027 1 3 9 19 19 819 1819 2819 7819 38 39 TOAAAA XYFAAA VVVVxx +1339 4028 1 3 9 19 39 339 1339 1339 1339 78 79 NZAAAA YYFAAA AAAAxx +5680 4029 0 0 0 0 80 680 1680 680 5680 160 161 MKAAAA ZYFAAA HHHHxx +5093 4030 1 1 3 13 93 93 1093 93 5093 186 187 XNAAAA AZFAAA OOOOxx +3508 4031 0 0 8 8 8 508 1508 3508 3508 16 17 YEAAAA BZFAAA VVVVxx +933 4032 1 1 3 13 33 933 933 933 933 66 67 XJAAAA CZFAAA AAAAxx +1106 4033 0 2 6 6 6 106 1106 1106 1106 12 13 OQAAAA DZFAAA HHHHxx +4386 4034 0 2 6 6 86 386 386 4386 4386 172 173 SMAAAA EZFAAA OOOOxx +5895 4035 1 3 5 15 95 895 1895 895 5895 190 191 TSAAAA FZFAAA VVVVxx +2980 4036 0 0 0 0 80 980 980 2980 2980 160 161 QKAAAA GZFAAA AAAAxx +4400 4037 0 0 0 0 0 400 400 4400 4400 0 1 GNAAAA HZFAAA HHHHxx +7433 4038 1 1 3 13 33 433 1433 2433 7433 66 67 XZAAAA IZFAAA OOOOxx +6110 4039 0 2 0 10 10 110 110 1110 6110 20 21 ABAAAA JZFAAA VVVVxx +867 4040 1 3 7 7 67 867 867 867 867 134 135 JHAAAA KZFAAA AAAAxx +5292 4041 0 0 2 12 92 292 1292 292 5292 184 185 OVAAAA LZFAAA HHHHxx +3926 4042 0 2 6 6 26 926 1926 3926 3926 52 53 AVAAAA MZFAAA OOOOxx +1107 4043 1 3 7 7 7 107 1107 1107 1107 14 15 PQAAAA NZFAAA VVVVxx +7355 4044 1 3 5 15 55 355 1355 2355 7355 110 111 XWAAAA OZFAAA AAAAxx +4689 4045 1 1 9 9 89 689 689 4689 4689 178 179 JYAAAA PZFAAA HHHHxx +4872 4046 0 0 2 12 72 872 872 4872 4872 144 145 KFAAAA QZFAAA OOOOxx +7821 4047 1 1 1 1 21 821 1821 2821 7821 42 43 VOAAAA RZFAAA VVVVxx +7277 4048 1 1 7 17 77 277 1277 2277 7277 154 155 XTAAAA SZFAAA AAAAxx +3268 4049 0 0 8 8 68 268 1268 3268 3268 136 137 SVAAAA TZFAAA HHHHxx +8877 4050 1 1 7 17 77 877 877 3877 8877 154 155 LDAAAA UZFAAA OOOOxx +343 4051 1 3 3 3 43 343 343 343 343 86 87 FNAAAA VZFAAA VVVVxx +621 4052 1 1 1 1 21 621 621 621 621 42 43 XXAAAA WZFAAA AAAAxx +5429 4053 1 1 9 9 29 429 1429 429 5429 58 59 VAAAAA XZFAAA HHHHxx +392 4054 0 0 2 12 92 392 392 392 392 184 185 CPAAAA YZFAAA OOOOxx +6004 4055 0 0 4 4 4 4 4 1004 6004 8 9 YWAAAA ZZFAAA VVVVxx +6377 4056 1 1 7 17 77 377 377 1377 6377 154 155 HLAAAA AAGAAA AAAAxx +3037 4057 1 1 7 17 37 37 1037 3037 3037 74 75 VMAAAA BAGAAA HHHHxx +3514 4058 0 2 4 14 14 514 1514 3514 3514 28 29 EFAAAA CAGAAA OOOOxx +8740 4059 0 0 0 0 40 740 740 3740 8740 80 81 EYAAAA DAGAAA VVVVxx +3877 4060 1 1 7 17 77 877 1877 3877 3877 154 155 DTAAAA EAGAAA AAAAxx +5731 4061 1 3 1 11 31 731 1731 731 5731 62 63 LMAAAA FAGAAA HHHHxx +6407 4062 1 3 7 7 7 407 407 1407 6407 14 15 LMAAAA GAGAAA OOOOxx +2044 4063 0 0 4 4 44 44 44 2044 2044 88 89 QAAAAA HAGAAA VVVVxx +7362 4064 0 2 2 2 62 362 1362 2362 7362 124 125 EXAAAA IAGAAA AAAAxx +5458 4065 0 2 8 18 58 458 1458 458 5458 116 117 YBAAAA JAGAAA HHHHxx +6437 4066 1 1 7 17 37 437 437 1437 6437 74 75 PNAAAA KAGAAA OOOOxx +1051 4067 1 3 1 11 51 51 1051 1051 1051 102 103 LOAAAA LAGAAA VVVVxx +1203 4068 1 3 3 3 3 203 1203 1203 1203 6 7 HUAAAA MAGAAA AAAAxx +2176 4069 0 0 6 16 76 176 176 2176 2176 152 153 SFAAAA NAGAAA HHHHxx +8997 4070 1 1 7 17 97 997 997 3997 8997 194 195 BIAAAA OAGAAA OOOOxx +6378 4071 0 2 8 18 78 378 378 1378 6378 156 157 ILAAAA PAGAAA VVVVxx +6006 4072 0 2 6 6 6 6 6 1006 6006 12 13 AXAAAA QAGAAA AAAAxx +2308 4073 0 0 8 8 8 308 308 2308 2308 16 17 UKAAAA RAGAAA HHHHxx +625 4074 1 1 5 5 25 625 625 625 625 50 51 BYAAAA SAGAAA OOOOxx +7298 4075 0 2 8 18 98 298 1298 2298 7298 196 197 SUAAAA TAGAAA VVVVxx +5575 4076 1 3 5 15 75 575 1575 575 5575 150 151 LGAAAA UAGAAA AAAAxx +3565 4077 1 1 5 5 65 565 1565 3565 3565 130 131 DHAAAA VAGAAA HHHHxx +47 4078 1 3 7 7 47 47 47 47 47 94 95 VBAAAA WAGAAA OOOOxx +2413 4079 1 1 3 13 13 413 413 2413 2413 26 27 VOAAAA XAGAAA VVVVxx +2153 4080 1 1 3 13 53 153 153 2153 2153 106 107 VEAAAA YAGAAA AAAAxx +752 4081 0 0 2 12 52 752 752 752 752 104 105 YCAAAA ZAGAAA HHHHxx +4095 4082 1 3 5 15 95 95 95 4095 4095 190 191 NBAAAA ABGAAA OOOOxx +2518 4083 0 2 8 18 18 518 518 2518 2518 36 37 WSAAAA BBGAAA VVVVxx +3681 4084 1 1 1 1 81 681 1681 3681 3681 162 163 PLAAAA CBGAAA AAAAxx +4213 4085 1 1 3 13 13 213 213 4213 4213 26 27 BGAAAA DBGAAA HHHHxx +2615 4086 1 3 5 15 15 615 615 2615 2615 30 31 PWAAAA EBGAAA OOOOxx +1471 4087 1 3 1 11 71 471 1471 1471 1471 142 143 PEAAAA FBGAAA VVVVxx +7315 4088 1 3 5 15 15 315 1315 2315 7315 30 31 JVAAAA GBGAAA AAAAxx +6013 4089 1 1 3 13 13 13 13 1013 6013 26 27 HXAAAA HBGAAA HHHHxx +3077 4090 1 1 7 17 77 77 1077 3077 3077 154 155 JOAAAA IBGAAA OOOOxx +2190 4091 0 2 0 10 90 190 190 2190 2190 180 181 GGAAAA JBGAAA VVVVxx +528 4092 0 0 8 8 28 528 528 528 528 56 57 IUAAAA KBGAAA AAAAxx +9508 4093 0 0 8 8 8 508 1508 4508 9508 16 17 SBAAAA LBGAAA HHHHxx +2473 4094 1 1 3 13 73 473 473 2473 2473 146 147 DRAAAA MBGAAA OOOOxx +167 4095 1 3 7 7 67 167 167 167 167 134 135 LGAAAA NBGAAA VVVVxx +8448 4096 0 0 8 8 48 448 448 3448 8448 96 97 YMAAAA OBGAAA AAAAxx +7538 4097 0 2 8 18 38 538 1538 2538 7538 76 77 YDAAAA PBGAAA HHHHxx +7638 4098 0 2 8 18 38 638 1638 2638 7638 76 77 UHAAAA QBGAAA OOOOxx +4328 4099 0 0 8 8 28 328 328 4328 4328 56 57 MKAAAA RBGAAA VVVVxx +3812 4100 0 0 2 12 12 812 1812 3812 3812 24 25 QQAAAA SBGAAA AAAAxx +2879 4101 1 3 9 19 79 879 879 2879 2879 158 159 TGAAAA TBGAAA HHHHxx +4741 4102 1 1 1 1 41 741 741 4741 4741 82 83 JAAAAA UBGAAA OOOOxx +9155 4103 1 3 5 15 55 155 1155 4155 9155 110 111 DOAAAA VBGAAA VVVVxx +5151 4104 1 3 1 11 51 151 1151 151 5151 102 103 DQAAAA WBGAAA AAAAxx +5591 4105 1 3 1 11 91 591 1591 591 5591 182 183 BHAAAA XBGAAA HHHHxx +1034 4106 0 2 4 14 34 34 1034 1034 1034 68 69 UNAAAA YBGAAA OOOOxx +765 4107 1 1 5 5 65 765 765 765 765 130 131 LDAAAA ZBGAAA VVVVxx +2664 4108 0 0 4 4 64 664 664 2664 2664 128 129 MYAAAA ACGAAA AAAAxx +6854 4109 0 2 4 14 54 854 854 1854 6854 108 109 QDAAAA BCGAAA HHHHxx +8263 4110 1 3 3 3 63 263 263 3263 8263 126 127 VFAAAA CCGAAA OOOOxx +8658 4111 0 2 8 18 58 658 658 3658 8658 116 117 AVAAAA DCGAAA VVVVxx +587 4112 1 3 7 7 87 587 587 587 587 174 175 PWAAAA ECGAAA AAAAxx +4553 4113 1 1 3 13 53 553 553 4553 4553 106 107 DTAAAA FCGAAA HHHHxx +1368 4114 0 0 8 8 68 368 1368 1368 1368 136 137 QAAAAA GCGAAA OOOOxx +1718 4115 0 2 8 18 18 718 1718 1718 1718 36 37 COAAAA HCGAAA VVVVxx +140 4116 0 0 0 0 40 140 140 140 140 80 81 KFAAAA ICGAAA AAAAxx +8341 4117 1 1 1 1 41 341 341 3341 8341 82 83 VIAAAA JCGAAA HHHHxx +72 4118 0 0 2 12 72 72 72 72 72 144 145 UCAAAA KCGAAA OOOOxx +6589 4119 1 1 9 9 89 589 589 1589 6589 178 179 LTAAAA LCGAAA VVVVxx +2024 4120 0 0 4 4 24 24 24 2024 2024 48 49 WZAAAA MCGAAA AAAAxx +8024 4121 0 0 4 4 24 24 24 3024 8024 48 49 QWAAAA NCGAAA HHHHxx +9564 4122 0 0 4 4 64 564 1564 4564 9564 128 129 WDAAAA OCGAAA OOOOxx +8625 4123 1 1 5 5 25 625 625 3625 8625 50 51 TTAAAA PCGAAA VVVVxx +2680 4124 0 0 0 0 80 680 680 2680 2680 160 161 CZAAAA QCGAAA AAAAxx +4323 4125 1 3 3 3 23 323 323 4323 4323 46 47 HKAAAA RCGAAA HHHHxx +8981 4126 1 1 1 1 81 981 981 3981 8981 162 163 LHAAAA SCGAAA OOOOxx +8909 4127 1 1 9 9 9 909 909 3909 8909 18 19 REAAAA TCGAAA VVVVxx +5288 4128 0 0 8 8 88 288 1288 288 5288 176 177 KVAAAA UCGAAA AAAAxx +2057 4129 1 1 7 17 57 57 57 2057 2057 114 115 DBAAAA VCGAAA HHHHxx +5931 4130 1 3 1 11 31 931 1931 931 5931 62 63 DUAAAA WCGAAA OOOOxx +9794 4131 0 2 4 14 94 794 1794 4794 9794 188 189 SMAAAA XCGAAA VVVVxx +1012 4132 0 0 2 12 12 12 1012 1012 1012 24 25 YMAAAA YCGAAA AAAAxx +5496 4133 0 0 6 16 96 496 1496 496 5496 192 193 KDAAAA ZCGAAA HHHHxx +9182 4134 0 2 2 2 82 182 1182 4182 9182 164 165 EPAAAA ADGAAA OOOOxx +5258 4135 0 2 8 18 58 258 1258 258 5258 116 117 GUAAAA BDGAAA VVVVxx +3050 4136 0 2 0 10 50 50 1050 3050 3050 100 101 INAAAA CDGAAA AAAAxx +2083 4137 1 3 3 3 83 83 83 2083 2083 166 167 DCAAAA DDGAAA HHHHxx +3069 4138 1 1 9 9 69 69 1069 3069 3069 138 139 BOAAAA EDGAAA OOOOxx +8459 4139 1 3 9 19 59 459 459 3459 8459 118 119 JNAAAA FDGAAA VVVVxx +169 4140 1 1 9 9 69 169 169 169 169 138 139 NGAAAA GDGAAA AAAAxx +4379 4141 1 3 9 19 79 379 379 4379 4379 158 159 LMAAAA HDGAAA HHHHxx +5126 4142 0 2 6 6 26 126 1126 126 5126 52 53 EPAAAA IDGAAA OOOOxx +1415 4143 1 3 5 15 15 415 1415 1415 1415 30 31 LCAAAA JDGAAA VVVVxx +1163 4144 1 3 3 3 63 163 1163 1163 1163 126 127 TSAAAA KDGAAA AAAAxx +3500 4145 0 0 0 0 0 500 1500 3500 3500 0 1 QEAAAA LDGAAA HHHHxx +7202 4146 0 2 2 2 2 202 1202 2202 7202 4 5 ARAAAA MDGAAA OOOOxx +747 4147 1 3 7 7 47 747 747 747 747 94 95 TCAAAA NDGAAA VVVVxx +9264 4148 0 0 4 4 64 264 1264 4264 9264 128 129 ISAAAA ODGAAA AAAAxx +8548 4149 0 0 8 8 48 548 548 3548 8548 96 97 UQAAAA PDGAAA HHHHxx +4228 4150 0 0 8 8 28 228 228 4228 4228 56 57 QGAAAA QDGAAA OOOOxx +7122 4151 0 2 2 2 22 122 1122 2122 7122 44 45 YNAAAA RDGAAA VVVVxx +3395 4152 1 3 5 15 95 395 1395 3395 3395 190 191 PAAAAA SDGAAA AAAAxx +5674 4153 0 2 4 14 74 674 1674 674 5674 148 149 GKAAAA TDGAAA HHHHxx +7293 4154 1 1 3 13 93 293 1293 2293 7293 186 187 NUAAAA UDGAAA OOOOxx +737 4155 1 1 7 17 37 737 737 737 737 74 75 JCAAAA VDGAAA VVVVxx +9595 4156 1 3 5 15 95 595 1595 4595 9595 190 191 BFAAAA WDGAAA AAAAxx +594 4157 0 2 4 14 94 594 594 594 594 188 189 WWAAAA XDGAAA HHHHxx +5322 4158 0 2 2 2 22 322 1322 322 5322 44 45 SWAAAA YDGAAA OOOOxx +2933 4159 1 1 3 13 33 933 933 2933 2933 66 67 VIAAAA ZDGAAA VVVVxx +4955 4160 1 3 5 15 55 955 955 4955 4955 110 111 PIAAAA AEGAAA AAAAxx +4073 4161 1 1 3 13 73 73 73 4073 4073 146 147 RAAAAA BEGAAA HHHHxx +7249 4162 1 1 9 9 49 249 1249 2249 7249 98 99 VSAAAA CEGAAA OOOOxx +192 4163 0 0 2 12 92 192 192 192 192 184 185 KHAAAA DEGAAA VVVVxx +2617 4164 1 1 7 17 17 617 617 2617 2617 34 35 RWAAAA EEGAAA AAAAxx +7409 4165 1 1 9 9 9 409 1409 2409 7409 18 19 ZYAAAA FEGAAA HHHHxx +4903 4166 1 3 3 3 3 903 903 4903 4903 6 7 PGAAAA GEGAAA OOOOxx +9797 4167 1 1 7 17 97 797 1797 4797 9797 194 195 VMAAAA HEGAAA VVVVxx +9919 4168 1 3 9 19 19 919 1919 4919 9919 38 39 NRAAAA IEGAAA AAAAxx +1878 4169 0 2 8 18 78 878 1878 1878 1878 156 157 GUAAAA JEGAAA HHHHxx +4851 4170 1 3 1 11 51 851 851 4851 4851 102 103 PEAAAA KEGAAA OOOOxx +5514 4171 0 2 4 14 14 514 1514 514 5514 28 29 CEAAAA LEGAAA VVVVxx +2582 4172 0 2 2 2 82 582 582 2582 2582 164 165 IVAAAA MEGAAA AAAAxx +3564 4173 0 0 4 4 64 564 1564 3564 3564 128 129 CHAAAA NEGAAA HHHHxx +7085 4174 1 1 5 5 85 85 1085 2085 7085 170 171 NMAAAA OEGAAA OOOOxx +3619 4175 1 3 9 19 19 619 1619 3619 3619 38 39 FJAAAA PEGAAA VVVVxx +261 4176 1 1 1 1 61 261 261 261 261 122 123 BKAAAA QEGAAA AAAAxx +7338 4177 0 2 8 18 38 338 1338 2338 7338 76 77 GWAAAA REGAAA HHHHxx +4251 4178 1 3 1 11 51 251 251 4251 4251 102 103 NHAAAA SEGAAA OOOOxx +5360 4179 0 0 0 0 60 360 1360 360 5360 120 121 EYAAAA TEGAAA VVVVxx +5678 4180 0 2 8 18 78 678 1678 678 5678 156 157 KKAAAA UEGAAA AAAAxx +9162 4181 0 2 2 2 62 162 1162 4162 9162 124 125 KOAAAA VEGAAA HHHHxx +5920 4182 0 0 0 0 20 920 1920 920 5920 40 41 STAAAA WEGAAA OOOOxx +7156 4183 0 0 6 16 56 156 1156 2156 7156 112 113 GPAAAA XEGAAA VVVVxx +4271 4184 1 3 1 11 71 271 271 4271 4271 142 143 HIAAAA YEGAAA AAAAxx +4698 4185 0 2 8 18 98 698 698 4698 4698 196 197 SYAAAA ZEGAAA HHHHxx +1572 4186 0 0 2 12 72 572 1572 1572 1572 144 145 MIAAAA AFGAAA OOOOxx +6974 4187 0 2 4 14 74 974 974 1974 6974 148 149 GIAAAA BFGAAA VVVVxx +4291 4188 1 3 1 11 91 291 291 4291 4291 182 183 BJAAAA CFGAAA AAAAxx +4036 4189 0 0 6 16 36 36 36 4036 4036 72 73 GZAAAA DFGAAA HHHHxx +7473 4190 1 1 3 13 73 473 1473 2473 7473 146 147 LBAAAA EFGAAA OOOOxx +4786 4191 0 2 6 6 86 786 786 4786 4786 172 173 CCAAAA FFGAAA VVVVxx +2662 4192 0 2 2 2 62 662 662 2662 2662 124 125 KYAAAA GFGAAA AAAAxx +916 4193 0 0 6 16 16 916 916 916 916 32 33 GJAAAA HFGAAA HHHHxx +668 4194 0 0 8 8 68 668 668 668 668 136 137 SZAAAA IFGAAA OOOOxx +4874 4195 0 2 4 14 74 874 874 4874 4874 148 149 MFAAAA JFGAAA VVVVxx +3752 4196 0 0 2 12 52 752 1752 3752 3752 104 105 IOAAAA KFGAAA AAAAxx +4865 4197 1 1 5 5 65 865 865 4865 4865 130 131 DFAAAA LFGAAA HHHHxx +7052 4198 0 0 2 12 52 52 1052 2052 7052 104 105 GLAAAA MFGAAA OOOOxx +5712 4199 0 0 2 12 12 712 1712 712 5712 24 25 SLAAAA NFGAAA VVVVxx +31 4200 1 3 1 11 31 31 31 31 31 62 63 FBAAAA OFGAAA AAAAxx +4944 4201 0 0 4 4 44 944 944 4944 4944 88 89 EIAAAA PFGAAA HHHHxx +1435 4202 1 3 5 15 35 435 1435 1435 1435 70 71 FDAAAA QFGAAA OOOOxx +501 4203 1 1 1 1 1 501 501 501 501 2 3 HTAAAA RFGAAA VVVVxx +9401 4204 1 1 1 1 1 401 1401 4401 9401 2 3 PXAAAA SFGAAA AAAAxx +5014 4205 0 2 4 14 14 14 1014 14 5014 28 29 WKAAAA TFGAAA HHHHxx +9125 4206 1 1 5 5 25 125 1125 4125 9125 50 51 ZMAAAA UFGAAA OOOOxx +6144 4207 0 0 4 4 44 144 144 1144 6144 88 89 ICAAAA VFGAAA VVVVxx +1743 4208 1 3 3 3 43 743 1743 1743 1743 86 87 BPAAAA WFGAAA AAAAxx +4316 4209 0 0 6 16 16 316 316 4316 4316 32 33 AKAAAA XFGAAA HHHHxx +8212 4210 0 0 2 12 12 212 212 3212 8212 24 25 WDAAAA YFGAAA OOOOxx +7344 4211 0 0 4 4 44 344 1344 2344 7344 88 89 MWAAAA ZFGAAA VVVVxx +2051 4212 1 3 1 11 51 51 51 2051 2051 102 103 XAAAAA AGGAAA AAAAxx +8131 4213 1 3 1 11 31 131 131 3131 8131 62 63 TAAAAA BGGAAA HHHHxx +7023 4214 1 3 3 3 23 23 1023 2023 7023 46 47 DKAAAA CGGAAA OOOOxx +9674 4215 0 2 4 14 74 674 1674 4674 9674 148 149 CIAAAA DGGAAA VVVVxx +4984 4216 0 0 4 4 84 984 984 4984 4984 168 169 SJAAAA EGGAAA AAAAxx +111 4217 1 3 1 11 11 111 111 111 111 22 23 HEAAAA FGGAAA HHHHxx +2296 4218 0 0 6 16 96 296 296 2296 2296 192 193 IKAAAA GGGAAA OOOOxx +5025 4219 1 1 5 5 25 25 1025 25 5025 50 51 HLAAAA HGGAAA VVVVxx +1756 4220 0 0 6 16 56 756 1756 1756 1756 112 113 OPAAAA IGGAAA AAAAxx +2885 4221 1 1 5 5 85 885 885 2885 2885 170 171 ZGAAAA JGGAAA HHHHxx +2541 4222 1 1 1 1 41 541 541 2541 2541 82 83 TTAAAA KGGAAA OOOOxx +1919 4223 1 3 9 19 19 919 1919 1919 1919 38 39 VVAAAA LGGAAA VVVVxx +6496 4224 0 0 6 16 96 496 496 1496 6496 192 193 WPAAAA MGGAAA AAAAxx +6103 4225 1 3 3 3 3 103 103 1103 6103 6 7 TAAAAA NGGAAA HHHHxx +98 4226 0 2 8 18 98 98 98 98 98 196 197 UDAAAA OGGAAA OOOOxx +3727 4227 1 3 7 7 27 727 1727 3727 3727 54 55 JNAAAA PGGAAA VVVVxx +689 4228 1 1 9 9 89 689 689 689 689 178 179 NAAAAA QGGAAA AAAAxx +7181 4229 1 1 1 1 81 181 1181 2181 7181 162 163 FQAAAA RGGAAA HHHHxx +8447 4230 1 3 7 7 47 447 447 3447 8447 94 95 XMAAAA SGGAAA OOOOxx +4569 4231 1 1 9 9 69 569 569 4569 4569 138 139 TTAAAA TGGAAA VVVVxx +8844 4232 0 0 4 4 44 844 844 3844 8844 88 89 ECAAAA UGGAAA AAAAxx +2436 4233 0 0 6 16 36 436 436 2436 2436 72 73 SPAAAA VGGAAA HHHHxx +391 4234 1 3 1 11 91 391 391 391 391 182 183 BPAAAA WGGAAA OOOOxx +3035 4235 1 3 5 15 35 35 1035 3035 3035 70 71 TMAAAA XGGAAA VVVVxx +7583 4236 1 3 3 3 83 583 1583 2583 7583 166 167 RFAAAA YGGAAA AAAAxx +1145 4237 1 1 5 5 45 145 1145 1145 1145 90 91 BSAAAA ZGGAAA HHHHxx +93 4238 1 1 3 13 93 93 93 93 93 186 187 PDAAAA AHGAAA OOOOxx +8896 4239 0 0 6 16 96 896 896 3896 8896 192 193 EEAAAA BHGAAA VVVVxx +6719 4240 1 3 9 19 19 719 719 1719 6719 38 39 LYAAAA CHGAAA AAAAxx +7728 4241 0 0 8 8 28 728 1728 2728 7728 56 57 GLAAAA DHGAAA HHHHxx +1349 4242 1 1 9 9 49 349 1349 1349 1349 98 99 XZAAAA EHGAAA OOOOxx +5349 4243 1 1 9 9 49 349 1349 349 5349 98 99 TXAAAA FHGAAA VVVVxx +3040 4244 0 0 0 0 40 40 1040 3040 3040 80 81 YMAAAA GHGAAA AAAAxx +2414 4245 0 2 4 14 14 414 414 2414 2414 28 29 WOAAAA HHGAAA HHHHxx +5122 4246 0 2 2 2 22 122 1122 122 5122 44 45 APAAAA IHGAAA OOOOxx +9553 4247 1 1 3 13 53 553 1553 4553 9553 106 107 LDAAAA JHGAAA VVVVxx +5987 4248 1 3 7 7 87 987 1987 987 5987 174 175 HWAAAA KHGAAA AAAAxx +5939 4249 1 3 9 19 39 939 1939 939 5939 78 79 LUAAAA LHGAAA HHHHxx +3525 4250 1 1 5 5 25 525 1525 3525 3525 50 51 PFAAAA MHGAAA OOOOxx +1371 4251 1 3 1 11 71 371 1371 1371 1371 142 143 TAAAAA NHGAAA VVVVxx +618 4252 0 2 8 18 18 618 618 618 618 36 37 UXAAAA OHGAAA AAAAxx +6529 4253 1 1 9 9 29 529 529 1529 6529 58 59 DRAAAA PHGAAA HHHHxx +4010 4254 0 2 0 10 10 10 10 4010 4010 20 21 GYAAAA QHGAAA OOOOxx +328 4255 0 0 8 8 28 328 328 328 328 56 57 QMAAAA RHGAAA VVVVxx +6121 4256 1 1 1 1 21 121 121 1121 6121 42 43 LBAAAA SHGAAA AAAAxx +3505 4257 1 1 5 5 5 505 1505 3505 3505 10 11 VEAAAA THGAAA HHHHxx +2033 4258 1 1 3 13 33 33 33 2033 2033 66 67 FAAAAA UHGAAA OOOOxx +4724 4259 0 0 4 4 24 724 724 4724 4724 48 49 SZAAAA VHGAAA VVVVxx +8717 4260 1 1 7 17 17 717 717 3717 8717 34 35 HXAAAA WHGAAA AAAAxx +5639 4261 1 3 9 19 39 639 1639 639 5639 78 79 XIAAAA XHGAAA HHHHxx +3448 4262 0 0 8 8 48 448 1448 3448 3448 96 97 QCAAAA YHGAAA OOOOxx +2919 4263 1 3 9 19 19 919 919 2919 2919 38 39 HIAAAA ZHGAAA VVVVxx +3417 4264 1 1 7 17 17 417 1417 3417 3417 34 35 LBAAAA AIGAAA AAAAxx +943 4265 1 3 3 3 43 943 943 943 943 86 87 HKAAAA BIGAAA HHHHxx +775 4266 1 3 5 15 75 775 775 775 775 150 151 VDAAAA CIGAAA OOOOxx +2333 4267 1 1 3 13 33 333 333 2333 2333 66 67 TLAAAA DIGAAA VVVVxx +4801 4268 1 1 1 1 1 801 801 4801 4801 2 3 RCAAAA EIGAAA AAAAxx +7169 4269 1 1 9 9 69 169 1169 2169 7169 138 139 TPAAAA FIGAAA HHHHxx +2840 4270 0 0 0 0 40 840 840 2840 2840 80 81 GFAAAA GIGAAA OOOOxx +9034 4271 0 2 4 14 34 34 1034 4034 9034 68 69 MJAAAA HIGAAA VVVVxx +6154 4272 0 2 4 14 54 154 154 1154 6154 108 109 SCAAAA IIGAAA AAAAxx +1412 4273 0 0 2 12 12 412 1412 1412 1412 24 25 ICAAAA JIGAAA HHHHxx +2263 4274 1 3 3 3 63 263 263 2263 2263 126 127 BJAAAA KIGAAA OOOOxx +7118 4275 0 2 8 18 18 118 1118 2118 7118 36 37 UNAAAA LIGAAA VVVVxx +1526 4276 0 2 6 6 26 526 1526 1526 1526 52 53 SGAAAA MIGAAA AAAAxx +491 4277 1 3 1 11 91 491 491 491 491 182 183 XSAAAA NIGAAA HHHHxx +9732 4278 0 0 2 12 32 732 1732 4732 9732 64 65 IKAAAA OIGAAA OOOOxx +7067 4279 1 3 7 7 67 67 1067 2067 7067 134 135 VLAAAA PIGAAA VVVVxx +212 4280 0 0 2 12 12 212 212 212 212 24 25 EIAAAA QIGAAA AAAAxx +1955 4281 1 3 5 15 55 955 1955 1955 1955 110 111 FXAAAA RIGAAA HHHHxx +3303 4282 1 3 3 3 3 303 1303 3303 3303 6 7 BXAAAA SIGAAA OOOOxx +2715 4283 1 3 5 15 15 715 715 2715 2715 30 31 LAAAAA TIGAAA VVVVxx +8168 4284 0 0 8 8 68 168 168 3168 8168 136 137 ECAAAA UIGAAA AAAAxx +6799 4285 1 3 9 19 99 799 799 1799 6799 198 199 NBAAAA VIGAAA HHHHxx +5080 4286 0 0 0 0 80 80 1080 80 5080 160 161 KNAAAA WIGAAA OOOOxx +4939 4287 1 3 9 19 39 939 939 4939 4939 78 79 ZHAAAA XIGAAA VVVVxx +6604 4288 0 0 4 4 4 604 604 1604 6604 8 9 AUAAAA YIGAAA AAAAxx +6531 4289 1 3 1 11 31 531 531 1531 6531 62 63 FRAAAA ZIGAAA HHHHxx +9948 4290 0 0 8 8 48 948 1948 4948 9948 96 97 QSAAAA AJGAAA OOOOxx +7923 4291 1 3 3 3 23 923 1923 2923 7923 46 47 TSAAAA BJGAAA VVVVxx +9905 4292 1 1 5 5 5 905 1905 4905 9905 10 11 ZQAAAA CJGAAA AAAAxx +340 4293 0 0 0 0 40 340 340 340 340 80 81 CNAAAA DJGAAA HHHHxx +1721 4294 1 1 1 1 21 721 1721 1721 1721 42 43 FOAAAA EJGAAA OOOOxx +9047 4295 1 3 7 7 47 47 1047 4047 9047 94 95 ZJAAAA FJGAAA VVVVxx +4723 4296 1 3 3 3 23 723 723 4723 4723 46 47 RZAAAA GJGAAA AAAAxx +5748 4297 0 0 8 8 48 748 1748 748 5748 96 97 CNAAAA HJGAAA HHHHxx +6845 4298 1 1 5 5 45 845 845 1845 6845 90 91 HDAAAA IJGAAA OOOOxx +1556 4299 0 0 6 16 56 556 1556 1556 1556 112 113 WHAAAA JJGAAA VVVVxx +9505 4300 1 1 5 5 5 505 1505 4505 9505 10 11 PBAAAA KJGAAA AAAAxx +3573 4301 1 1 3 13 73 573 1573 3573 3573 146 147 LHAAAA LJGAAA HHHHxx +3785 4302 1 1 5 5 85 785 1785 3785 3785 170 171 PPAAAA MJGAAA OOOOxx +2772 4303 0 0 2 12 72 772 772 2772 2772 144 145 QCAAAA NJGAAA VVVVxx +7282 4304 0 2 2 2 82 282 1282 2282 7282 164 165 CUAAAA OJGAAA AAAAxx +8106 4305 0 2 6 6 6 106 106 3106 8106 12 13 UZAAAA PJGAAA HHHHxx +2847 4306 1 3 7 7 47 847 847 2847 2847 94 95 NFAAAA QJGAAA OOOOxx +9803 4307 1 3 3 3 3 803 1803 4803 9803 6 7 BNAAAA RJGAAA VVVVxx +7719 4308 1 3 9 19 19 719 1719 2719 7719 38 39 XKAAAA SJGAAA AAAAxx +4649 4309 1 1 9 9 49 649 649 4649 4649 98 99 VWAAAA TJGAAA HHHHxx +6196 4310 0 0 6 16 96 196 196 1196 6196 192 193 IEAAAA UJGAAA OOOOxx +6026 4311 0 2 6 6 26 26 26 1026 6026 52 53 UXAAAA VJGAAA VVVVxx +1646 4312 0 2 6 6 46 646 1646 1646 1646 92 93 ILAAAA WJGAAA AAAAxx +6526 4313 0 2 6 6 26 526 526 1526 6526 52 53 ARAAAA XJGAAA HHHHxx +5110 4314 0 2 0 10 10 110 1110 110 5110 20 21 OOAAAA YJGAAA OOOOxx +3946 4315 0 2 6 6 46 946 1946 3946 3946 92 93 UVAAAA ZJGAAA VVVVxx +445 4316 1 1 5 5 45 445 445 445 445 90 91 DRAAAA AKGAAA AAAAxx +3249 4317 1 1 9 9 49 249 1249 3249 3249 98 99 ZUAAAA BKGAAA HHHHxx +2501 4318 1 1 1 1 1 501 501 2501 2501 2 3 FSAAAA CKGAAA OOOOxx +3243 4319 1 3 3 3 43 243 1243 3243 3243 86 87 TUAAAA DKGAAA VVVVxx +4701 4320 1 1 1 1 1 701 701 4701 4701 2 3 VYAAAA EKGAAA AAAAxx +472 4321 0 0 2 12 72 472 472 472 472 144 145 ESAAAA FKGAAA HHHHxx +3356 4322 0 0 6 16 56 356 1356 3356 3356 112 113 CZAAAA GKGAAA OOOOxx +9967 4323 1 3 7 7 67 967 1967 4967 9967 134 135 JTAAAA HKGAAA VVVVxx +4292 4324 0 0 2 12 92 292 292 4292 4292 184 185 CJAAAA IKGAAA AAAAxx +7005 4325 1 1 5 5 5 5 1005 2005 7005 10 11 LJAAAA JKGAAA HHHHxx +6267 4326 1 3 7 7 67 267 267 1267 6267 134 135 BHAAAA KKGAAA OOOOxx +6678 4327 0 2 8 18 78 678 678 1678 6678 156 157 WWAAAA LKGAAA VVVVxx +6083 4328 1 3 3 3 83 83 83 1083 6083 166 167 ZZAAAA MKGAAA AAAAxx +760 4329 0 0 0 0 60 760 760 760 760 120 121 GDAAAA NKGAAA HHHHxx +7833 4330 1 1 3 13 33 833 1833 2833 7833 66 67 HPAAAA OKGAAA OOOOxx +2877 4331 1 1 7 17 77 877 877 2877 2877 154 155 RGAAAA PKGAAA VVVVxx +8810 4332 0 2 0 10 10 810 810 3810 8810 20 21 WAAAAA QKGAAA AAAAxx +1560 4333 0 0 0 0 60 560 1560 1560 1560 120 121 AIAAAA RKGAAA HHHHxx +1367 4334 1 3 7 7 67 367 1367 1367 1367 134 135 PAAAAA SKGAAA OOOOxx +8756 4335 0 0 6 16 56 756 756 3756 8756 112 113 UYAAAA TKGAAA VVVVxx +1346 4336 0 2 6 6 46 346 1346 1346 1346 92 93 UZAAAA UKGAAA AAAAxx +6449 4337 1 1 9 9 49 449 449 1449 6449 98 99 BOAAAA VKGAAA HHHHxx +6658 4338 0 2 8 18 58 658 658 1658 6658 116 117 CWAAAA WKGAAA OOOOxx +6745 4339 1 1 5 5 45 745 745 1745 6745 90 91 LZAAAA XKGAAA VVVVxx +4866 4340 0 2 6 6 66 866 866 4866 4866 132 133 EFAAAA YKGAAA AAAAxx +14 4341 0 2 4 14 14 14 14 14 14 28 29 OAAAAA ZKGAAA HHHHxx +4506 4342 0 2 6 6 6 506 506 4506 4506 12 13 IRAAAA ALGAAA OOOOxx +1923 4343 1 3 3 3 23 923 1923 1923 1923 46 47 ZVAAAA BLGAAA VVVVxx +8365 4344 1 1 5 5 65 365 365 3365 8365 130 131 TJAAAA CLGAAA AAAAxx +1279 4345 1 3 9 19 79 279 1279 1279 1279 158 159 FXAAAA DLGAAA HHHHxx +7666 4346 0 2 6 6 66 666 1666 2666 7666 132 133 WIAAAA ELGAAA OOOOxx +7404 4347 0 0 4 4 4 404 1404 2404 7404 8 9 UYAAAA FLGAAA VVVVxx +65 4348 1 1 5 5 65 65 65 65 65 130 131 NCAAAA GLGAAA AAAAxx +5820 4349 0 0 0 0 20 820 1820 820 5820 40 41 WPAAAA HLGAAA HHHHxx +459 4350 1 3 9 19 59 459 459 459 459 118 119 RRAAAA ILGAAA OOOOxx +4787 4351 1 3 7 7 87 787 787 4787 4787 174 175 DCAAAA JLGAAA VVVVxx +5631 4352 1 3 1 11 31 631 1631 631 5631 62 63 PIAAAA KLGAAA AAAAxx +9717 4353 1 1 7 17 17 717 1717 4717 9717 34 35 TJAAAA LLGAAA HHHHxx +2560 4354 0 0 0 0 60 560 560 2560 2560 120 121 MUAAAA MLGAAA OOOOxx +8295 4355 1 3 5 15 95 295 295 3295 8295 190 191 BHAAAA NLGAAA VVVVxx +3596 4356 0 0 6 16 96 596 1596 3596 3596 192 193 IIAAAA OLGAAA AAAAxx +2023 4357 1 3 3 3 23 23 23 2023 2023 46 47 VZAAAA PLGAAA HHHHxx +5055 4358 1 3 5 15 55 55 1055 55 5055 110 111 LMAAAA QLGAAA OOOOxx +763 4359 1 3 3 3 63 763 763 763 763 126 127 JDAAAA RLGAAA VVVVxx +6733 4360 1 1 3 13 33 733 733 1733 6733 66 67 ZYAAAA SLGAAA AAAAxx +9266 4361 0 2 6 6 66 266 1266 4266 9266 132 133 KSAAAA TLGAAA HHHHxx +4479 4362 1 3 9 19 79 479 479 4479 4479 158 159 HQAAAA ULGAAA OOOOxx +1816 4363 0 0 6 16 16 816 1816 1816 1816 32 33 WRAAAA VLGAAA VVVVxx +899 4364 1 3 9 19 99 899 899 899 899 198 199 PIAAAA WLGAAA AAAAxx +230 4365 0 2 0 10 30 230 230 230 230 60 61 WIAAAA XLGAAA HHHHxx +5362 4366 0 2 2 2 62 362 1362 362 5362 124 125 GYAAAA YLGAAA OOOOxx +1609 4367 1 1 9 9 9 609 1609 1609 1609 18 19 XJAAAA ZLGAAA VVVVxx +6750 4368 0 2 0 10 50 750 750 1750 6750 100 101 QZAAAA AMGAAA AAAAxx +9704 4369 0 0 4 4 4 704 1704 4704 9704 8 9 GJAAAA BMGAAA HHHHxx +3991 4370 1 3 1 11 91 991 1991 3991 3991 182 183 NXAAAA CMGAAA OOOOxx +3959 4371 1 3 9 19 59 959 1959 3959 3959 118 119 HWAAAA DMGAAA VVVVxx +9021 4372 1 1 1 1 21 21 1021 4021 9021 42 43 ZIAAAA EMGAAA AAAAxx +7585 4373 1 1 5 5 85 585 1585 2585 7585 170 171 TFAAAA FMGAAA HHHHxx +7083 4374 1 3 3 3 83 83 1083 2083 7083 166 167 LMAAAA GMGAAA OOOOxx +7688 4375 0 0 8 8 88 688 1688 2688 7688 176 177 SJAAAA HMGAAA VVVVxx +2673 4376 1 1 3 13 73 673 673 2673 2673 146 147 VYAAAA IMGAAA AAAAxx +3554 4377 0 2 4 14 54 554 1554 3554 3554 108 109 SGAAAA JMGAAA HHHHxx +7416 4378 0 0 6 16 16 416 1416 2416 7416 32 33 GZAAAA KMGAAA OOOOxx +5672 4379 0 0 2 12 72 672 1672 672 5672 144 145 EKAAAA LMGAAA VVVVxx +1355 4380 1 3 5 15 55 355 1355 1355 1355 110 111 DAAAAA MMGAAA AAAAxx +3149 4381 1 1 9 9 49 149 1149 3149 3149 98 99 DRAAAA NMGAAA HHHHxx +5811 4382 1 3 1 11 11 811 1811 811 5811 22 23 NPAAAA OMGAAA OOOOxx +3759 4383 1 3 9 19 59 759 1759 3759 3759 118 119 POAAAA PMGAAA VVVVxx +5634 4384 0 2 4 14 34 634 1634 634 5634 68 69 SIAAAA QMGAAA AAAAxx +8617 4385 1 1 7 17 17 617 617 3617 8617 34 35 LTAAAA RMGAAA HHHHxx +8949 4386 1 1 9 9 49 949 949 3949 8949 98 99 FGAAAA SMGAAA OOOOxx +3964 4387 0 0 4 4 64 964 1964 3964 3964 128 129 MWAAAA TMGAAA VVVVxx +3852 4388 0 0 2 12 52 852 1852 3852 3852 104 105 ESAAAA UMGAAA AAAAxx +1555 4389 1 3 5 15 55 555 1555 1555 1555 110 111 VHAAAA VMGAAA HHHHxx +6536 4390 0 0 6 16 36 536 536 1536 6536 72 73 KRAAAA WMGAAA OOOOxx +4779 4391 1 3 9 19 79 779 779 4779 4779 158 159 VBAAAA XMGAAA VVVVxx +1893 4392 1 1 3 13 93 893 1893 1893 1893 186 187 VUAAAA YMGAAA AAAAxx +9358 4393 0 2 8 18 58 358 1358 4358 9358 116 117 YVAAAA ZMGAAA HHHHxx +7438 4394 0 2 8 18 38 438 1438 2438 7438 76 77 CAAAAA ANGAAA OOOOxx +941 4395 1 1 1 1 41 941 941 941 941 82 83 FKAAAA BNGAAA VVVVxx +4844 4396 0 0 4 4 44 844 844 4844 4844 88 89 IEAAAA CNGAAA AAAAxx +4745 4397 1 1 5 5 45 745 745 4745 4745 90 91 NAAAAA DNGAAA HHHHxx +1017 4398 1 1 7 17 17 17 1017 1017 1017 34 35 DNAAAA ENGAAA OOOOxx +327 4399 1 3 7 7 27 327 327 327 327 54 55 PMAAAA FNGAAA VVVVxx +3152 4400 0 0 2 12 52 152 1152 3152 3152 104 105 GRAAAA GNGAAA AAAAxx +4711 4401 1 3 1 11 11 711 711 4711 4711 22 23 FZAAAA HNGAAA HHHHxx +141 4402 1 1 1 1 41 141 141 141 141 82 83 LFAAAA INGAAA OOOOxx +1303 4403 1 3 3 3 3 303 1303 1303 1303 6 7 DYAAAA JNGAAA VVVVxx +8873 4404 1 1 3 13 73 873 873 3873 8873 146 147 HDAAAA KNGAAA AAAAxx +8481 4405 1 1 1 1 81 481 481 3481 8481 162 163 FOAAAA LNGAAA HHHHxx +5445 4406 1 1 5 5 45 445 1445 445 5445 90 91 LBAAAA MNGAAA OOOOxx +7868 4407 0 0 8 8 68 868 1868 2868 7868 136 137 QQAAAA NNGAAA VVVVxx +6722 4408 0 2 2 2 22 722 722 1722 6722 44 45 OYAAAA ONGAAA AAAAxx +6628 4409 0 0 8 8 28 628 628 1628 6628 56 57 YUAAAA PNGAAA HHHHxx +7738 4410 0 2 8 18 38 738 1738 2738 7738 76 77 QLAAAA QNGAAA OOOOxx +1018 4411 0 2 8 18 18 18 1018 1018 1018 36 37 ENAAAA RNGAAA VVVVxx +3296 4412 0 0 6 16 96 296 1296 3296 3296 192 193 UWAAAA SNGAAA AAAAxx +1946 4413 0 2 6 6 46 946 1946 1946 1946 92 93 WWAAAA TNGAAA HHHHxx +6603 4414 1 3 3 3 3 603 603 1603 6603 6 7 ZTAAAA UNGAAA OOOOxx +3562 4415 0 2 2 2 62 562 1562 3562 3562 124 125 AHAAAA VNGAAA VVVVxx +1147 4416 1 3 7 7 47 147 1147 1147 1147 94 95 DSAAAA WNGAAA AAAAxx +6031 4417 1 3 1 11 31 31 31 1031 6031 62 63 ZXAAAA XNGAAA HHHHxx +6484 4418 0 0 4 4 84 484 484 1484 6484 168 169 KPAAAA YNGAAA OOOOxx +496 4419 0 0 6 16 96 496 496 496 496 192 193 CTAAAA ZNGAAA VVVVxx +4563 4420 1 3 3 3 63 563 563 4563 4563 126 127 NTAAAA AOGAAA AAAAxx +1037 4421 1 1 7 17 37 37 1037 1037 1037 74 75 XNAAAA BOGAAA HHHHxx +9672 4422 0 0 2 12 72 672 1672 4672 9672 144 145 AIAAAA COGAAA OOOOxx +9053 4423 1 1 3 13 53 53 1053 4053 9053 106 107 FKAAAA DOGAAA VVVVxx +2523 4424 1 3 3 3 23 523 523 2523 2523 46 47 BTAAAA EOGAAA AAAAxx +8519 4425 1 3 9 19 19 519 519 3519 8519 38 39 RPAAAA FOGAAA HHHHxx +8190 4426 0 2 0 10 90 190 190 3190 8190 180 181 ADAAAA GOGAAA OOOOxx +2068 4427 0 0 8 8 68 68 68 2068 2068 136 137 OBAAAA HOGAAA VVVVxx +8569 4428 1 1 9 9 69 569 569 3569 8569 138 139 PRAAAA IOGAAA AAAAxx +6535 4429 1 3 5 15 35 535 535 1535 6535 70 71 JRAAAA JOGAAA HHHHxx +1810 4430 0 2 0 10 10 810 1810 1810 1810 20 21 QRAAAA KOGAAA OOOOxx +3099 4431 1 3 9 19 99 99 1099 3099 3099 198 199 FPAAAA LOGAAA VVVVxx +7466 4432 0 2 6 6 66 466 1466 2466 7466 132 133 EBAAAA MOGAAA AAAAxx +4017 4433 1 1 7 17 17 17 17 4017 4017 34 35 NYAAAA NOGAAA HHHHxx +1097 4434 1 1 7 17 97 97 1097 1097 1097 194 195 FQAAAA OOGAAA OOOOxx +7686 4435 0 2 6 6 86 686 1686 2686 7686 172 173 QJAAAA POGAAA VVVVxx +6742 4436 0 2 2 2 42 742 742 1742 6742 84 85 IZAAAA QOGAAA AAAAxx +5966 4437 0 2 6 6 66 966 1966 966 5966 132 133 MVAAAA ROGAAA HHHHxx +3632 4438 0 0 2 12 32 632 1632 3632 3632 64 65 SJAAAA SOGAAA OOOOxx +8837 4439 1 1 7 17 37 837 837 3837 8837 74 75 XBAAAA TOGAAA VVVVxx +1667 4440 1 3 7 7 67 667 1667 1667 1667 134 135 DMAAAA UOGAAA AAAAxx +8833 4441 1 1 3 13 33 833 833 3833 8833 66 67 TBAAAA VOGAAA HHHHxx +9805 4442 1 1 5 5 5 805 1805 4805 9805 10 11 DNAAAA WOGAAA OOOOxx +3650 4443 0 2 0 10 50 650 1650 3650 3650 100 101 KKAAAA XOGAAA VVVVxx +2237 4444 1 1 7 17 37 237 237 2237 2237 74 75 BIAAAA YOGAAA AAAAxx +9980 4445 0 0 0 0 80 980 1980 4980 9980 160 161 WTAAAA ZOGAAA HHHHxx +2861 4446 1 1 1 1 61 861 861 2861 2861 122 123 BGAAAA APGAAA OOOOxx +1334 4447 0 2 4 14 34 334 1334 1334 1334 68 69 IZAAAA BPGAAA VVVVxx +842 4448 0 2 2 2 42 842 842 842 842 84 85 KGAAAA CPGAAA AAAAxx +1116 4449 0 0 6 16 16 116 1116 1116 1116 32 33 YQAAAA DPGAAA HHHHxx +4055 4450 1 3 5 15 55 55 55 4055 4055 110 111 ZZAAAA EPGAAA OOOOxx +3842 4451 0 2 2 2 42 842 1842 3842 3842 84 85 URAAAA FPGAAA VVVVxx +1886 4452 0 2 6 6 86 886 1886 1886 1886 172 173 OUAAAA GPGAAA AAAAxx +8589 4453 1 1 9 9 89 589 589 3589 8589 178 179 JSAAAA HPGAAA HHHHxx +5873 4454 1 1 3 13 73 873 1873 873 5873 146 147 XRAAAA IPGAAA OOOOxx +7711 4455 1 3 1 11 11 711 1711 2711 7711 22 23 PKAAAA JPGAAA VVVVxx +911 4456 1 3 1 11 11 911 911 911 911 22 23 BJAAAA KPGAAA AAAAxx +5837 4457 1 1 7 17 37 837 1837 837 5837 74 75 NQAAAA LPGAAA HHHHxx +897 4458 1 1 7 17 97 897 897 897 897 194 195 NIAAAA MPGAAA OOOOxx +4299 4459 1 3 9 19 99 299 299 4299 4299 198 199 JJAAAA NPGAAA VVVVxx +7774 4460 0 2 4 14 74 774 1774 2774 7774 148 149 ANAAAA OPGAAA AAAAxx +7832 4461 0 0 2 12 32 832 1832 2832 7832 64 65 GPAAAA PPGAAA HHHHxx +9915 4462 1 3 5 15 15 915 1915 4915 9915 30 31 JRAAAA QPGAAA OOOOxx +9 4463 1 1 9 9 9 9 9 9 9 18 19 JAAAAA RPGAAA VVVVxx +9675 4464 1 3 5 15 75 675 1675 4675 9675 150 151 DIAAAA SPGAAA AAAAxx +7953 4465 1 1 3 13 53 953 1953 2953 7953 106 107 XTAAAA TPGAAA HHHHxx +8912 4466 0 0 2 12 12 912 912 3912 8912 24 25 UEAAAA UPGAAA OOOOxx +4188 4467 0 0 8 8 88 188 188 4188 4188 176 177 CFAAAA VPGAAA VVVVxx +8446 4468 0 2 6 6 46 446 446 3446 8446 92 93 WMAAAA WPGAAA AAAAxx +1600 4469 0 0 0 0 0 600 1600 1600 1600 0 1 OJAAAA XPGAAA HHHHxx +43 4470 1 3 3 3 43 43 43 43 43 86 87 RBAAAA YPGAAA OOOOxx +544 4471 0 0 4 4 44 544 544 544 544 88 89 YUAAAA ZPGAAA VVVVxx +6977 4472 1 1 7 17 77 977 977 1977 6977 154 155 JIAAAA AQGAAA AAAAxx +3191 4473 1 3 1 11 91 191 1191 3191 3191 182 183 TSAAAA BQGAAA HHHHxx +418 4474 0 2 8 18 18 418 418 418 418 36 37 CQAAAA CQGAAA OOOOxx +3142 4475 0 2 2 2 42 142 1142 3142 3142 84 85 WQAAAA DQGAAA VVVVxx +5042 4476 0 2 2 2 42 42 1042 42 5042 84 85 YLAAAA EQGAAA AAAAxx +2194 4477 0 2 4 14 94 194 194 2194 2194 188 189 KGAAAA FQGAAA HHHHxx +2397 4478 1 1 7 17 97 397 397 2397 2397 194 195 FOAAAA GQGAAA OOOOxx +4684 4479 0 0 4 4 84 684 684 4684 4684 168 169 EYAAAA HQGAAA VVVVxx +34 4480 0 2 4 14 34 34 34 34 34 68 69 IBAAAA IQGAAA AAAAxx +3844 4481 0 0 4 4 44 844 1844 3844 3844 88 89 WRAAAA JQGAAA HHHHxx +7824 4482 0 0 4 4 24 824 1824 2824 7824 48 49 YOAAAA KQGAAA OOOOxx +6177 4483 1 1 7 17 77 177 177 1177 6177 154 155 PDAAAA LQGAAA VVVVxx +9657 4484 1 1 7 17 57 657 1657 4657 9657 114 115 LHAAAA MQGAAA AAAAxx +4546 4485 0 2 6 6 46 546 546 4546 4546 92 93 WSAAAA NQGAAA HHHHxx +599 4486 1 3 9 19 99 599 599 599 599 198 199 BXAAAA OQGAAA OOOOxx +153 4487 1 1 3 13 53 153 153 153 153 106 107 XFAAAA PQGAAA VVVVxx +6910 4488 0 2 0 10 10 910 910 1910 6910 20 21 UFAAAA QQGAAA AAAAxx +4408 4489 0 0 8 8 8 408 408 4408 4408 16 17 ONAAAA RQGAAA HHHHxx +1164 4490 0 0 4 4 64 164 1164 1164 1164 128 129 USAAAA SQGAAA OOOOxx +6469 4491 1 1 9 9 69 469 469 1469 6469 138 139 VOAAAA TQGAAA VVVVxx +5996 4492 0 0 6 16 96 996 1996 996 5996 192 193 QWAAAA UQGAAA AAAAxx +2639 4493 1 3 9 19 39 639 639 2639 2639 78 79 NXAAAA VQGAAA HHHHxx +2678 4494 0 2 8 18 78 678 678 2678 2678 156 157 AZAAAA WQGAAA OOOOxx +8392 4495 0 0 2 12 92 392 392 3392 8392 184 185 UKAAAA XQGAAA VVVVxx +1386 4496 0 2 6 6 86 386 1386 1386 1386 172 173 IBAAAA YQGAAA AAAAxx +5125 4497 1 1 5 5 25 125 1125 125 5125 50 51 DPAAAA ZQGAAA HHHHxx +8453 4498 1 1 3 13 53 453 453 3453 8453 106 107 DNAAAA ARGAAA OOOOxx +2369 4499 1 1 9 9 69 369 369 2369 2369 138 139 DNAAAA BRGAAA VVVVxx +1608 4500 0 0 8 8 8 608 1608 1608 1608 16 17 WJAAAA CRGAAA AAAAxx +3781 4501 1 1 1 1 81 781 1781 3781 3781 162 163 LPAAAA DRGAAA HHHHxx +903 4502 1 3 3 3 3 903 903 903 903 6 7 TIAAAA ERGAAA OOOOxx +2099 4503 1 3 9 19 99 99 99 2099 2099 198 199 TCAAAA FRGAAA VVVVxx +538 4504 0 2 8 18 38 538 538 538 538 76 77 SUAAAA GRGAAA AAAAxx +9177 4505 1 1 7 17 77 177 1177 4177 9177 154 155 ZOAAAA HRGAAA HHHHxx +420 4506 0 0 0 0 20 420 420 420 420 40 41 EQAAAA IRGAAA OOOOxx +9080 4507 0 0 0 0 80 80 1080 4080 9080 160 161 GLAAAA JRGAAA VVVVxx +2630 4508 0 2 0 10 30 630 630 2630 2630 60 61 EXAAAA KRGAAA AAAAxx +5978 4509 0 2 8 18 78 978 1978 978 5978 156 157 YVAAAA LRGAAA HHHHxx +9239 4510 1 3 9 19 39 239 1239 4239 9239 78 79 JRAAAA MRGAAA OOOOxx +4372 4511 0 0 2 12 72 372 372 4372 4372 144 145 EMAAAA NRGAAA VVVVxx +4357 4512 1 1 7 17 57 357 357 4357 4357 114 115 PLAAAA ORGAAA AAAAxx +9857 4513 1 1 7 17 57 857 1857 4857 9857 114 115 DPAAAA PRGAAA HHHHxx +7933 4514 1 1 3 13 33 933 1933 2933 7933 66 67 DTAAAA QRGAAA OOOOxx +9574 4515 0 2 4 14 74 574 1574 4574 9574 148 149 GEAAAA RRGAAA VVVVxx +8294 4516 0 2 4 14 94 294 294 3294 8294 188 189 AHAAAA SRGAAA AAAAxx +627 4517 1 3 7 7 27 627 627 627 627 54 55 DYAAAA TRGAAA HHHHxx +3229 4518 1 1 9 9 29 229 1229 3229 3229 58 59 FUAAAA URGAAA OOOOxx +3163 4519 1 3 3 3 63 163 1163 3163 3163 126 127 RRAAAA VRGAAA VVVVxx +7349 4520 1 1 9 9 49 349 1349 2349 7349 98 99 RWAAAA WRGAAA AAAAxx +6889 4521 1 1 9 9 89 889 889 1889 6889 178 179 ZEAAAA XRGAAA HHHHxx +2101 4522 1 1 1 1 1 101 101 2101 2101 2 3 VCAAAA YRGAAA OOOOxx +6476 4523 0 0 6 16 76 476 476 1476 6476 152 153 CPAAAA ZRGAAA VVVVxx +6765 4524 1 1 5 5 65 765 765 1765 6765 130 131 FAAAAA ASGAAA AAAAxx +4204 4525 0 0 4 4 4 204 204 4204 4204 8 9 SFAAAA BSGAAA HHHHxx +5915 4526 1 3 5 15 15 915 1915 915 5915 30 31 NTAAAA CSGAAA OOOOxx +2318 4527 0 2 8 18 18 318 318 2318 2318 36 37 ELAAAA DSGAAA VVVVxx +294 4528 0 2 4 14 94 294 294 294 294 188 189 ILAAAA ESGAAA AAAAxx +5245 4529 1 1 5 5 45 245 1245 245 5245 90 91 TTAAAA FSGAAA HHHHxx +4481 4530 1 1 1 1 81 481 481 4481 4481 162 163 JQAAAA GSGAAA OOOOxx +7754 4531 0 2 4 14 54 754 1754 2754 7754 108 109 GMAAAA HSGAAA VVVVxx +8494 4532 0 2 4 14 94 494 494 3494 8494 188 189 SOAAAA ISGAAA AAAAxx +4014 4533 0 2 4 14 14 14 14 4014 4014 28 29 KYAAAA JSGAAA HHHHxx +2197 4534 1 1 7 17 97 197 197 2197 2197 194 195 NGAAAA KSGAAA OOOOxx +1297 4535 1 1 7 17 97 297 1297 1297 1297 194 195 XXAAAA LSGAAA VVVVxx +1066 4536 0 2 6 6 66 66 1066 1066 1066 132 133 APAAAA MSGAAA AAAAxx +5710 4537 0 2 0 10 10 710 1710 710 5710 20 21 QLAAAA NSGAAA HHHHxx +4100 4538 0 0 0 0 0 100 100 4100 4100 0 1 SBAAAA OSGAAA OOOOxx +7356 4539 0 0 6 16 56 356 1356 2356 7356 112 113 YWAAAA PSGAAA VVVVxx +7658 4540 0 2 8 18 58 658 1658 2658 7658 116 117 OIAAAA QSGAAA AAAAxx +3666 4541 0 2 6 6 66 666 1666 3666 3666 132 133 ALAAAA RSGAAA HHHHxx +9713 4542 1 1 3 13 13 713 1713 4713 9713 26 27 PJAAAA SSGAAA OOOOxx +691 4543 1 3 1 11 91 691 691 691 691 182 183 PAAAAA TSGAAA VVVVxx +3112 4544 0 0 2 12 12 112 1112 3112 3112 24 25 SPAAAA USGAAA AAAAxx +6035 4545 1 3 5 15 35 35 35 1035 6035 70 71 DYAAAA VSGAAA HHHHxx +8353 4546 1 1 3 13 53 353 353 3353 8353 106 107 HJAAAA WSGAAA OOOOxx +5679 4547 1 3 9 19 79 679 1679 679 5679 158 159 LKAAAA XSGAAA VVVVxx +2124 4548 0 0 4 4 24 124 124 2124 2124 48 49 SDAAAA YSGAAA AAAAxx +4714 4549 0 2 4 14 14 714 714 4714 4714 28 29 IZAAAA ZSGAAA HHHHxx +9048 4550 0 0 8 8 48 48 1048 4048 9048 96 97 AKAAAA ATGAAA OOOOxx +7692 4551 0 0 2 12 92 692 1692 2692 7692 184 185 WJAAAA BTGAAA VVVVxx +4542 4552 0 2 2 2 42 542 542 4542 4542 84 85 SSAAAA CTGAAA AAAAxx +8737 4553 1 1 7 17 37 737 737 3737 8737 74 75 BYAAAA DTGAAA HHHHxx +4977 4554 1 1 7 17 77 977 977 4977 4977 154 155 LJAAAA ETGAAA OOOOxx +9349 4555 1 1 9 9 49 349 1349 4349 9349 98 99 PVAAAA FTGAAA VVVVxx +731 4556 1 3 1 11 31 731 731 731 731 62 63 DCAAAA GTGAAA AAAAxx +1788 4557 0 0 8 8 88 788 1788 1788 1788 176 177 UQAAAA HTGAAA HHHHxx +7830 4558 0 2 0 10 30 830 1830 2830 7830 60 61 EPAAAA ITGAAA OOOOxx +3977 4559 1 1 7 17 77 977 1977 3977 3977 154 155 ZWAAAA JTGAAA VVVVxx +2421 4560 1 1 1 1 21 421 421 2421 2421 42 43 DPAAAA KTGAAA AAAAxx +5891 4561 1 3 1 11 91 891 1891 891 5891 182 183 PSAAAA LTGAAA HHHHxx +1111 4562 1 3 1 11 11 111 1111 1111 1111 22 23 TQAAAA MTGAAA OOOOxx +9224 4563 0 0 4 4 24 224 1224 4224 9224 48 49 UQAAAA NTGAAA VVVVxx +9872 4564 0 0 2 12 72 872 1872 4872 9872 144 145 SPAAAA OTGAAA AAAAxx +2433 4565 1 1 3 13 33 433 433 2433 2433 66 67 PPAAAA PTGAAA HHHHxx +1491 4566 1 3 1 11 91 491 1491 1491 1491 182 183 JFAAAA QTGAAA OOOOxx +6653 4567 1 1 3 13 53 653 653 1653 6653 106 107 XVAAAA RTGAAA VVVVxx +1907 4568 1 3 7 7 7 907 1907 1907 1907 14 15 JVAAAA STGAAA AAAAxx +889 4569 1 1 9 9 89 889 889 889 889 178 179 FIAAAA TTGAAA HHHHxx +561 4570 1 1 1 1 61 561 561 561 561 122 123 PVAAAA UTGAAA OOOOxx +7415 4571 1 3 5 15 15 415 1415 2415 7415 30 31 FZAAAA VTGAAA VVVVxx +2703 4572 1 3 3 3 3 703 703 2703 2703 6 7 ZZAAAA WTGAAA AAAAxx +2561 4573 1 1 1 1 61 561 561 2561 2561 122 123 NUAAAA XTGAAA HHHHxx +1257 4574 1 1 7 17 57 257 1257 1257 1257 114 115 JWAAAA YTGAAA OOOOxx +2390 4575 0 2 0 10 90 390 390 2390 2390 180 181 YNAAAA ZTGAAA VVVVxx +3915 4576 1 3 5 15 15 915 1915 3915 3915 30 31 PUAAAA AUGAAA AAAAxx +8476 4577 0 0 6 16 76 476 476 3476 8476 152 153 AOAAAA BUGAAA HHHHxx +607 4578 1 3 7 7 7 607 607 607 607 14 15 JXAAAA CUGAAA OOOOxx +3891 4579 1 3 1 11 91 891 1891 3891 3891 182 183 RTAAAA DUGAAA VVVVxx +7269 4580 1 1 9 9 69 269 1269 2269 7269 138 139 PTAAAA EUGAAA AAAAxx +9537 4581 1 1 7 17 37 537 1537 4537 9537 74 75 VCAAAA FUGAAA HHHHxx +8518 4582 0 2 8 18 18 518 518 3518 8518 36 37 QPAAAA GUGAAA OOOOxx +5221 4583 1 1 1 1 21 221 1221 221 5221 42 43 VSAAAA HUGAAA VVVVxx +3274 4584 0 2 4 14 74 274 1274 3274 3274 148 149 YVAAAA IUGAAA AAAAxx +6677 4585 1 1 7 17 77 677 677 1677 6677 154 155 VWAAAA JUGAAA HHHHxx +3114 4586 0 2 4 14 14 114 1114 3114 3114 28 29 UPAAAA KUGAAA OOOOxx +1966 4587 0 2 6 6 66 966 1966 1966 1966 132 133 QXAAAA LUGAAA VVVVxx +5941 4588 1 1 1 1 41 941 1941 941 5941 82 83 NUAAAA MUGAAA AAAAxx +9463 4589 1 3 3 3 63 463 1463 4463 9463 126 127 ZZAAAA NUGAAA HHHHxx +8966 4590 0 2 6 6 66 966 966 3966 8966 132 133 WGAAAA OUGAAA OOOOxx +4402 4591 0 2 2 2 2 402 402 4402 4402 4 5 INAAAA PUGAAA VVVVxx +3364 4592 0 0 4 4 64 364 1364 3364 3364 128 129 KZAAAA QUGAAA AAAAxx +3698 4593 0 2 8 18 98 698 1698 3698 3698 196 197 GMAAAA RUGAAA HHHHxx +4651 4594 1 3 1 11 51 651 651 4651 4651 102 103 XWAAAA SUGAAA OOOOxx +2127 4595 1 3 7 7 27 127 127 2127 2127 54 55 VDAAAA TUGAAA VVVVxx +3614 4596 0 2 4 14 14 614 1614 3614 3614 28 29 AJAAAA UUGAAA AAAAxx +5430 4597 0 2 0 10 30 430 1430 430 5430 60 61 WAAAAA VUGAAA HHHHxx +3361 4598 1 1 1 1 61 361 1361 3361 3361 122 123 HZAAAA WUGAAA OOOOxx +4798 4599 0 2 8 18 98 798 798 4798 4798 196 197 OCAAAA XUGAAA VVVVxx +8269 4600 1 1 9 9 69 269 269 3269 8269 138 139 BGAAAA YUGAAA AAAAxx +6458 4601 0 2 8 18 58 458 458 1458 6458 116 117 KOAAAA ZUGAAA HHHHxx +3358 4602 0 2 8 18 58 358 1358 3358 3358 116 117 EZAAAA AVGAAA OOOOxx +5898 4603 0 2 8 18 98 898 1898 898 5898 196 197 WSAAAA BVGAAA VVVVxx +1880 4604 0 0 0 0 80 880 1880 1880 1880 160 161 IUAAAA CVGAAA AAAAxx +782 4605 0 2 2 2 82 782 782 782 782 164 165 CEAAAA DVGAAA HHHHxx +3102 4606 0 2 2 2 2 102 1102 3102 3102 4 5 IPAAAA EVGAAA OOOOxx +6366 4607 0 2 6 6 66 366 366 1366 6366 132 133 WKAAAA FVGAAA VVVVxx +399 4608 1 3 9 19 99 399 399 399 399 198 199 JPAAAA GVGAAA AAAAxx +6773 4609 1 1 3 13 73 773 773 1773 6773 146 147 NAAAAA HVGAAA HHHHxx +7942 4610 0 2 2 2 42 942 1942 2942 7942 84 85 MTAAAA IVGAAA OOOOxx +6274 4611 0 2 4 14 74 274 274 1274 6274 148 149 IHAAAA JVGAAA VVVVxx +7447 4612 1 3 7 7 47 447 1447 2447 7447 94 95 LAAAAA KVGAAA AAAAxx +7648 4613 0 0 8 8 48 648 1648 2648 7648 96 97 EIAAAA LVGAAA HHHHxx +3997 4614 1 1 7 17 97 997 1997 3997 3997 194 195 TXAAAA MVGAAA OOOOxx +1759 4615 1 3 9 19 59 759 1759 1759 1759 118 119 RPAAAA NVGAAA VVVVxx +1785 4616 1 1 5 5 85 785 1785 1785 1785 170 171 RQAAAA OVGAAA AAAAxx +8930 4617 0 2 0 10 30 930 930 3930 8930 60 61 MFAAAA PVGAAA HHHHxx +7595 4618 1 3 5 15 95 595 1595 2595 7595 190 191 DGAAAA QVGAAA OOOOxx +6752 4619 0 0 2 12 52 752 752 1752 6752 104 105 SZAAAA RVGAAA VVVVxx +5635 4620 1 3 5 15 35 635 1635 635 5635 70 71 TIAAAA SVGAAA AAAAxx +1579 4621 1 3 9 19 79 579 1579 1579 1579 158 159 TIAAAA TVGAAA HHHHxx +7743 4622 1 3 3 3 43 743 1743 2743 7743 86 87 VLAAAA UVGAAA OOOOxx +5856 4623 0 0 6 16 56 856 1856 856 5856 112 113 GRAAAA VVGAAA VVVVxx +7273 4624 1 1 3 13 73 273 1273 2273 7273 146 147 TTAAAA WVGAAA AAAAxx +1399 4625 1 3 9 19 99 399 1399 1399 1399 198 199 VBAAAA XVGAAA HHHHxx +3694 4626 0 2 4 14 94 694 1694 3694 3694 188 189 CMAAAA YVGAAA OOOOxx +2782 4627 0 2 2 2 82 782 782 2782 2782 164 165 ADAAAA ZVGAAA VVVVxx +6951 4628 1 3 1 11 51 951 951 1951 6951 102 103 JHAAAA AWGAAA AAAAxx +6053 4629 1 1 3 13 53 53 53 1053 6053 106 107 VYAAAA BWGAAA HHHHxx +1753 4630 1 1 3 13 53 753 1753 1753 1753 106 107 LPAAAA CWGAAA OOOOxx +3985 4631 1 1 5 5 85 985 1985 3985 3985 170 171 HXAAAA DWGAAA VVVVxx +6159 4632 1 3 9 19 59 159 159 1159 6159 118 119 XCAAAA EWGAAA AAAAxx +6250 4633 0 2 0 10 50 250 250 1250 6250 100 101 KGAAAA FWGAAA HHHHxx +6240 4634 0 0 0 0 40 240 240 1240 6240 80 81 AGAAAA GWGAAA OOOOxx +6571 4635 1 3 1 11 71 571 571 1571 6571 142 143 TSAAAA HWGAAA VVVVxx +8624 4636 0 0 4 4 24 624 624 3624 8624 48 49 STAAAA IWGAAA AAAAxx +9718 4637 0 2 8 18 18 718 1718 4718 9718 36 37 UJAAAA JWGAAA HHHHxx +5529 4638 1 1 9 9 29 529 1529 529 5529 58 59 REAAAA KWGAAA OOOOxx +7089 4639 1 1 9 9 89 89 1089 2089 7089 178 179 RMAAAA LWGAAA VVVVxx +5488 4640 0 0 8 8 88 488 1488 488 5488 176 177 CDAAAA MWGAAA AAAAxx +5444 4641 0 0 4 4 44 444 1444 444 5444 88 89 KBAAAA NWGAAA HHHHxx +4899 4642 1 3 9 19 99 899 899 4899 4899 198 199 LGAAAA OWGAAA OOOOxx +7928 4643 0 0 8 8 28 928 1928 2928 7928 56 57 YSAAAA PWGAAA VVVVxx +4736 4644 0 0 6 16 36 736 736 4736 4736 72 73 EAAAAA QWGAAA AAAAxx +4317 4645 1 1 7 17 17 317 317 4317 4317 34 35 BKAAAA RWGAAA HHHHxx +1174 4646 0 2 4 14 74 174 1174 1174 1174 148 149 ETAAAA SWGAAA OOOOxx +6138 4647 0 2 8 18 38 138 138 1138 6138 76 77 CCAAAA TWGAAA VVVVxx +3943 4648 1 3 3 3 43 943 1943 3943 3943 86 87 RVAAAA UWGAAA AAAAxx +1545 4649 1 1 5 5 45 545 1545 1545 1545 90 91 LHAAAA VWGAAA HHHHxx +6867 4650 1 3 7 7 67 867 867 1867 6867 134 135 DEAAAA WWGAAA OOOOxx +6832 4651 0 0 2 12 32 832 832 1832 6832 64 65 UCAAAA XWGAAA VVVVxx +2987 4652 1 3 7 7 87 987 987 2987 2987 174 175 XKAAAA YWGAAA AAAAxx +5169 4653 1 1 9 9 69 169 1169 169 5169 138 139 VQAAAA ZWGAAA HHHHxx +8998 4654 0 2 8 18 98 998 998 3998 8998 196 197 CIAAAA AXGAAA OOOOxx +9347 4655 1 3 7 7 47 347 1347 4347 9347 94 95 NVAAAA BXGAAA VVVVxx +4800 4656 0 0 0 0 0 800 800 4800 4800 0 1 QCAAAA CXGAAA AAAAxx +4200 4657 0 0 0 0 0 200 200 4200 4200 0 1 OFAAAA DXGAAA HHHHxx +4046 4658 0 2 6 6 46 46 46 4046 4046 92 93 QZAAAA EXGAAA OOOOxx +7142 4659 0 2 2 2 42 142 1142 2142 7142 84 85 SOAAAA FXGAAA VVVVxx +2733 4660 1 1 3 13 33 733 733 2733 2733 66 67 DBAAAA GXGAAA AAAAxx +1568 4661 0 0 8 8 68 568 1568 1568 1568 136 137 IIAAAA HXGAAA HHHHxx +5105 4662 1 1 5 5 5 105 1105 105 5105 10 11 JOAAAA IXGAAA OOOOxx +9115 4663 1 3 5 15 15 115 1115 4115 9115 30 31 PMAAAA JXGAAA VVVVxx +6475 4664 1 3 5 15 75 475 475 1475 6475 150 151 BPAAAA KXGAAA AAAAxx +3796 4665 0 0 6 16 96 796 1796 3796 3796 192 193 AQAAAA LXGAAA HHHHxx +5410 4666 0 2 0 10 10 410 1410 410 5410 20 21 CAAAAA MXGAAA OOOOxx +4023 4667 1 3 3 3 23 23 23 4023 4023 46 47 TYAAAA NXGAAA VVVVxx +8904 4668 0 0 4 4 4 904 904 3904 8904 8 9 MEAAAA OXGAAA AAAAxx +450 4669 0 2 0 10 50 450 450 450 450 100 101 IRAAAA PXGAAA HHHHxx +8087 4670 1 3 7 7 87 87 87 3087 8087 174 175 BZAAAA QXGAAA OOOOxx +6478 4671 0 2 8 18 78 478 478 1478 6478 156 157 EPAAAA RXGAAA VVVVxx +2696 4672 0 0 6 16 96 696 696 2696 2696 192 193 SZAAAA SXGAAA AAAAxx +1792 4673 0 0 2 12 92 792 1792 1792 1792 184 185 YQAAAA TXGAAA HHHHxx +9699 4674 1 3 9 19 99 699 1699 4699 9699 198 199 BJAAAA UXGAAA OOOOxx +9160 4675 0 0 0 0 60 160 1160 4160 9160 120 121 IOAAAA VXGAAA VVVVxx +9989 4676 1 1 9 9 89 989 1989 4989 9989 178 179 FUAAAA WXGAAA AAAAxx +9568 4677 0 0 8 8 68 568 1568 4568 9568 136 137 AEAAAA XXGAAA HHHHxx +487 4678 1 3 7 7 87 487 487 487 487 174 175 TSAAAA YXGAAA OOOOxx +7863 4679 1 3 3 3 63 863 1863 2863 7863 126 127 LQAAAA ZXGAAA VVVVxx +1884 4680 0 0 4 4 84 884 1884 1884 1884 168 169 MUAAAA AYGAAA AAAAxx +2651 4681 1 3 1 11 51 651 651 2651 2651 102 103 ZXAAAA BYGAAA HHHHxx +8285 4682 1 1 5 5 85 285 285 3285 8285 170 171 RGAAAA CYGAAA OOOOxx +3927 4683 1 3 7 7 27 927 1927 3927 3927 54 55 BVAAAA DYGAAA VVVVxx +4076 4684 0 0 6 16 76 76 76 4076 4076 152 153 UAAAAA EYGAAA AAAAxx +6149 4685 1 1 9 9 49 149 149 1149 6149 98 99 NCAAAA FYGAAA HHHHxx +6581 4686 1 1 1 1 81 581 581 1581 6581 162 163 DTAAAA GYGAAA OOOOxx +8293 4687 1 1 3 13 93 293 293 3293 8293 186 187 ZGAAAA HYGAAA VVVVxx +7665 4688 1 1 5 5 65 665 1665 2665 7665 130 131 VIAAAA IYGAAA AAAAxx +4435 4689 1 3 5 15 35 435 435 4435 4435 70 71 POAAAA JYGAAA HHHHxx +1271 4690 1 3 1 11 71 271 1271 1271 1271 142 143 XWAAAA KYGAAA OOOOxx +3928 4691 0 0 8 8 28 928 1928 3928 3928 56 57 CVAAAA LYGAAA VVVVxx +7045 4692 1 1 5 5 45 45 1045 2045 7045 90 91 ZKAAAA MYGAAA AAAAxx +4943 4693 1 3 3 3 43 943 943 4943 4943 86 87 DIAAAA NYGAAA HHHHxx +8473 4694 1 1 3 13 73 473 473 3473 8473 146 147 XNAAAA OYGAAA OOOOxx +1707 4695 1 3 7 7 7 707 1707 1707 1707 14 15 RNAAAA PYGAAA VVVVxx +7509 4696 1 1 9 9 9 509 1509 2509 7509 18 19 VCAAAA QYGAAA AAAAxx +1593 4697 1 1 3 13 93 593 1593 1593 1593 186 187 HJAAAA RYGAAA HHHHxx +9281 4698 1 1 1 1 81 281 1281 4281 9281 162 163 ZSAAAA SYGAAA OOOOxx +8986 4699 0 2 6 6 86 986 986 3986 8986 172 173 QHAAAA TYGAAA VVVVxx +3740 4700 0 0 0 0 40 740 1740 3740 3740 80 81 WNAAAA UYGAAA AAAAxx +9265 4701 1 1 5 5 65 265 1265 4265 9265 130 131 JSAAAA VYGAAA HHHHxx +1510 4702 0 2 0 10 10 510 1510 1510 1510 20 21 CGAAAA WYGAAA OOOOxx +3022 4703 0 2 2 2 22 22 1022 3022 3022 44 45 GMAAAA XYGAAA VVVVxx +9014 4704 0 2 4 14 14 14 1014 4014 9014 28 29 SIAAAA YYGAAA AAAAxx +6816 4705 0 0 6 16 16 816 816 1816 6816 32 33 ECAAAA ZYGAAA HHHHxx +5518 4706 0 2 8 18 18 518 1518 518 5518 36 37 GEAAAA AZGAAA OOOOxx +4451 4707 1 3 1 11 51 451 451 4451 4451 102 103 FPAAAA BZGAAA VVVVxx +8747 4708 1 3 7 7 47 747 747 3747 8747 94 95 LYAAAA CZGAAA AAAAxx +4646 4709 0 2 6 6 46 646 646 4646 4646 92 93 SWAAAA DZGAAA HHHHxx +7296 4710 0 0 6 16 96 296 1296 2296 7296 192 193 QUAAAA EZGAAA OOOOxx +9644 4711 0 0 4 4 44 644 1644 4644 9644 88 89 YGAAAA FZGAAA VVVVxx +5977 4712 1 1 7 17 77 977 1977 977 5977 154 155 XVAAAA GZGAAA AAAAxx +6270 4713 0 2 0 10 70 270 270 1270 6270 140 141 EHAAAA HZGAAA HHHHxx +5578 4714 0 2 8 18 78 578 1578 578 5578 156 157 OGAAAA IZGAAA OOOOxx +2465 4715 1 1 5 5 65 465 465 2465 2465 130 131 VQAAAA JZGAAA VVVVxx +6436 4716 0 0 6 16 36 436 436 1436 6436 72 73 ONAAAA KZGAAA AAAAxx +8089 4717 1 1 9 9 89 89 89 3089 8089 178 179 DZAAAA LZGAAA HHHHxx +2409 4718 1 1 9 9 9 409 409 2409 2409 18 19 ROAAAA MZGAAA OOOOxx +284 4719 0 0 4 4 84 284 284 284 284 168 169 YKAAAA NZGAAA VVVVxx +5576 4720 0 0 6 16 76 576 1576 576 5576 152 153 MGAAAA OZGAAA AAAAxx +6534 4721 0 2 4 14 34 534 534 1534 6534 68 69 IRAAAA PZGAAA HHHHxx +8848 4722 0 0 8 8 48 848 848 3848 8848 96 97 ICAAAA QZGAAA OOOOxx +4305 4723 1 1 5 5 5 305 305 4305 4305 10 11 PJAAAA RZGAAA VVVVxx +5574 4724 0 2 4 14 74 574 1574 574 5574 148 149 KGAAAA SZGAAA AAAAxx +596 4725 0 0 6 16 96 596 596 596 596 192 193 YWAAAA TZGAAA HHHHxx +1253 4726 1 1 3 13 53 253 1253 1253 1253 106 107 FWAAAA UZGAAA OOOOxx +521 4727 1 1 1 1 21 521 521 521 521 42 43 BUAAAA VZGAAA VVVVxx +8739 4728 1 3 9 19 39 739 739 3739 8739 78 79 DYAAAA WZGAAA AAAAxx +908 4729 0 0 8 8 8 908 908 908 908 16 17 YIAAAA XZGAAA HHHHxx +6937 4730 1 1 7 17 37 937 937 1937 6937 74 75 VGAAAA YZGAAA OOOOxx +4515 4731 1 3 5 15 15 515 515 4515 4515 30 31 RRAAAA ZZGAAA VVVVxx +8630 4732 0 2 0 10 30 630 630 3630 8630 60 61 YTAAAA AAHAAA AAAAxx +7518 4733 0 2 8 18 18 518 1518 2518 7518 36 37 EDAAAA BAHAAA HHHHxx +8300 4734 0 0 0 0 0 300 300 3300 8300 0 1 GHAAAA CAHAAA OOOOxx +8434 4735 0 2 4 14 34 434 434 3434 8434 68 69 KMAAAA DAHAAA VVVVxx +6000 4736 0 0 0 0 0 0 0 1000 6000 0 1 UWAAAA EAHAAA AAAAxx +4508 4737 0 0 8 8 8 508 508 4508 4508 16 17 KRAAAA FAHAAA HHHHxx +7861 4738 1 1 1 1 61 861 1861 2861 7861 122 123 JQAAAA GAHAAA OOOOxx +5953 4739 1 1 3 13 53 953 1953 953 5953 106 107 ZUAAAA HAHAAA VVVVxx +5063 4740 1 3 3 3 63 63 1063 63 5063 126 127 TMAAAA IAHAAA AAAAxx +4501 4741 1 1 1 1 1 501 501 4501 4501 2 3 DRAAAA JAHAAA HHHHxx +7092 4742 0 0 2 12 92 92 1092 2092 7092 184 185 UMAAAA KAHAAA OOOOxx +4388 4743 0 0 8 8 88 388 388 4388 4388 176 177 UMAAAA LAHAAA VVVVxx +1826 4744 0 2 6 6 26 826 1826 1826 1826 52 53 GSAAAA MAHAAA AAAAxx +568 4745 0 0 8 8 68 568 568 568 568 136 137 WVAAAA NAHAAA HHHHxx +8184 4746 0 0 4 4 84 184 184 3184 8184 168 169 UCAAAA OAHAAA OOOOxx +4268 4747 0 0 8 8 68 268 268 4268 4268 136 137 EIAAAA PAHAAA VVVVxx +5798 4748 0 2 8 18 98 798 1798 798 5798 196 197 APAAAA QAHAAA AAAAxx +5190 4749 0 2 0 10 90 190 1190 190 5190 180 181 QRAAAA RAHAAA HHHHxx +1298 4750 0 2 8 18 98 298 1298 1298 1298 196 197 YXAAAA SAHAAA OOOOxx +4035 4751 1 3 5 15 35 35 35 4035 4035 70 71 FZAAAA TAHAAA VVVVxx +4504 4752 0 0 4 4 4 504 504 4504 4504 8 9 GRAAAA UAHAAA AAAAxx +5992 4753 0 0 2 12 92 992 1992 992 5992 184 185 MWAAAA VAHAAA HHHHxx +770 4754 0 2 0 10 70 770 770 770 770 140 141 QDAAAA WAHAAA OOOOxx +7502 4755 0 2 2 2 2 502 1502 2502 7502 4 5 OCAAAA XAHAAA VVVVxx +824 4756 0 0 4 4 24 824 824 824 824 48 49 SFAAAA YAHAAA AAAAxx +7716 4757 0 0 6 16 16 716 1716 2716 7716 32 33 UKAAAA ZAHAAA HHHHxx +5749 4758 1 1 9 9 49 749 1749 749 5749 98 99 DNAAAA ABHAAA OOOOxx +9814 4759 0 2 4 14 14 814 1814 4814 9814 28 29 MNAAAA BBHAAA VVVVxx +350 4760 0 2 0 10 50 350 350 350 350 100 101 MNAAAA CBHAAA AAAAxx +1390 4761 0 2 0 10 90 390 1390 1390 1390 180 181 MBAAAA DBHAAA HHHHxx +6994 4762 0 2 4 14 94 994 994 1994 6994 188 189 AJAAAA EBHAAA OOOOxx +3629 4763 1 1 9 9 29 629 1629 3629 3629 58 59 PJAAAA FBHAAA VVVVxx +9937 4764 1 1 7 17 37 937 1937 4937 9937 74 75 FSAAAA GBHAAA AAAAxx +5285 4765 1 1 5 5 85 285 1285 285 5285 170 171 HVAAAA HBHAAA HHHHxx +3157 4766 1 1 7 17 57 157 1157 3157 3157 114 115 LRAAAA IBHAAA OOOOxx +9549 4767 1 1 9 9 49 549 1549 4549 9549 98 99 HDAAAA JBHAAA VVVVxx +4118 4768 0 2 8 18 18 118 118 4118 4118 36 37 KCAAAA KBHAAA AAAAxx +756 4769 0 0 6 16 56 756 756 756 756 112 113 CDAAAA LBHAAA HHHHxx +5964 4770 0 0 4 4 64 964 1964 964 5964 128 129 KVAAAA MBHAAA OOOOxx +7701 4771 1 1 1 1 1 701 1701 2701 7701 2 3 FKAAAA NBHAAA VVVVxx +1242 4772 0 2 2 2 42 242 1242 1242 1242 84 85 UVAAAA OBHAAA AAAAxx +7890 4773 0 2 0 10 90 890 1890 2890 7890 180 181 MRAAAA PBHAAA HHHHxx +1991 4774 1 3 1 11 91 991 1991 1991 1991 182 183 PYAAAA QBHAAA OOOOxx +110 4775 0 2 0 10 10 110 110 110 110 20 21 GEAAAA RBHAAA VVVVxx +9334 4776 0 2 4 14 34 334 1334 4334 9334 68 69 AVAAAA SBHAAA AAAAxx +6231 4777 1 3 1 11 31 231 231 1231 6231 62 63 RFAAAA TBHAAA HHHHxx +9871 4778 1 3 1 11 71 871 1871 4871 9871 142 143 RPAAAA UBHAAA OOOOxx +9471 4779 1 3 1 11 71 471 1471 4471 9471 142 143 HAAAAA VBHAAA VVVVxx +2697 4780 1 1 7 17 97 697 697 2697 2697 194 195 TZAAAA WBHAAA AAAAxx +4761 4781 1 1 1 1 61 761 761 4761 4761 122 123 DBAAAA XBHAAA HHHHxx +8493 4782 1 1 3 13 93 493 493 3493 8493 186 187 ROAAAA YBHAAA OOOOxx +1045 4783 1 1 5 5 45 45 1045 1045 1045 90 91 FOAAAA ZBHAAA VVVVxx +3403 4784 1 3 3 3 3 403 1403 3403 3403 6 7 XAAAAA ACHAAA AAAAxx +9412 4785 0 0 2 12 12 412 1412 4412 9412 24 25 AYAAAA BCHAAA HHHHxx +7652 4786 0 0 2 12 52 652 1652 2652 7652 104 105 IIAAAA CCHAAA OOOOxx +5866 4787 0 2 6 6 66 866 1866 866 5866 132 133 QRAAAA DCHAAA VVVVxx +6942 4788 0 2 2 2 42 942 942 1942 6942 84 85 AHAAAA ECHAAA AAAAxx +9353 4789 1 1 3 13 53 353 1353 4353 9353 106 107 TVAAAA FCHAAA HHHHxx +2600 4790 0 0 0 0 0 600 600 2600 2600 0 1 AWAAAA GCHAAA OOOOxx +6971 4791 1 3 1 11 71 971 971 1971 6971 142 143 DIAAAA HCHAAA VVVVxx +5391 4792 1 3 1 11 91 391 1391 391 5391 182 183 JZAAAA ICHAAA AAAAxx +7654 4793 0 2 4 14 54 654 1654 2654 7654 108 109 KIAAAA JCHAAA HHHHxx +1797 4794 1 1 7 17 97 797 1797 1797 1797 194 195 DRAAAA KCHAAA OOOOxx +4530 4795 0 2 0 10 30 530 530 4530 4530 60 61 GSAAAA LCHAAA VVVVxx +3130 4796 0 2 0 10 30 130 1130 3130 3130 60 61 KQAAAA MCHAAA AAAAxx +9442 4797 0 2 2 2 42 442 1442 4442 9442 84 85 EZAAAA NCHAAA HHHHxx +6659 4798 1 3 9 19 59 659 659 1659 6659 118 119 DWAAAA OCHAAA OOOOxx +9714 4799 0 2 4 14 14 714 1714 4714 9714 28 29 QJAAAA PCHAAA VVVVxx +3660 4800 0 0 0 0 60 660 1660 3660 3660 120 121 UKAAAA QCHAAA AAAAxx +1906 4801 0 2 6 6 6 906 1906 1906 1906 12 13 IVAAAA RCHAAA HHHHxx +7927 4802 1 3 7 7 27 927 1927 2927 7927 54 55 XSAAAA SCHAAA OOOOxx +1767 4803 1 3 7 7 67 767 1767 1767 1767 134 135 ZPAAAA TCHAAA VVVVxx +5523 4804 1 3 3 3 23 523 1523 523 5523 46 47 LEAAAA UCHAAA AAAAxx +9289 4805 1 1 9 9 89 289 1289 4289 9289 178 179 HTAAAA VCHAAA HHHHxx +2717 4806 1 1 7 17 17 717 717 2717 2717 34 35 NAAAAA WCHAAA OOOOxx +4099 4807 1 3 9 19 99 99 99 4099 4099 198 199 RBAAAA XCHAAA VVVVxx +4387 4808 1 3 7 7 87 387 387 4387 4387 174 175 TMAAAA YCHAAA AAAAxx +8864 4809 0 0 4 4 64 864 864 3864 8864 128 129 YCAAAA ZCHAAA HHHHxx +1774 4810 0 2 4 14 74 774 1774 1774 1774 148 149 GQAAAA ADHAAA OOOOxx +6292 4811 0 0 2 12 92 292 292 1292 6292 184 185 AIAAAA BDHAAA VVVVxx +847 4812 1 3 7 7 47 847 847 847 847 94 95 PGAAAA CDHAAA AAAAxx +5954 4813 0 2 4 14 54 954 1954 954 5954 108 109 AVAAAA DDHAAA HHHHxx +8032 4814 0 0 2 12 32 32 32 3032 8032 64 65 YWAAAA EDHAAA OOOOxx +3295 4815 1 3 5 15 95 295 1295 3295 3295 190 191 TWAAAA FDHAAA VVVVxx +8984 4816 0 0 4 4 84 984 984 3984 8984 168 169 OHAAAA GDHAAA AAAAxx +7809 4817 1 1 9 9 9 809 1809 2809 7809 18 19 JOAAAA HDHAAA HHHHxx +1670 4818 0 2 0 10 70 670 1670 1670 1670 140 141 GMAAAA IDHAAA OOOOxx +7733 4819 1 1 3 13 33 733 1733 2733 7733 66 67 LLAAAA JDHAAA VVVVxx +6187 4820 1 3 7 7 87 187 187 1187 6187 174 175 ZDAAAA KDHAAA AAAAxx +9326 4821 0 2 6 6 26 326 1326 4326 9326 52 53 SUAAAA LDHAAA HHHHxx +2493 4822 1 1 3 13 93 493 493 2493 2493 186 187 XRAAAA MDHAAA OOOOxx +9512 4823 0 0 2 12 12 512 1512 4512 9512 24 25 WBAAAA NDHAAA VVVVxx +4342 4824 0 2 2 2 42 342 342 4342 4342 84 85 ALAAAA ODHAAA AAAAxx +5350 4825 0 2 0 10 50 350 1350 350 5350 100 101 UXAAAA PDHAAA HHHHxx +6009 4826 1 1 9 9 9 9 9 1009 6009 18 19 DXAAAA QDHAAA OOOOxx +1208 4827 0 0 8 8 8 208 1208 1208 1208 16 17 MUAAAA RDHAAA VVVVxx +7014 4828 0 2 4 14 14 14 1014 2014 7014 28 29 UJAAAA SDHAAA AAAAxx +2967 4829 1 3 7 7 67 967 967 2967 2967 134 135 DKAAAA TDHAAA HHHHxx +5831 4830 1 3 1 11 31 831 1831 831 5831 62 63 HQAAAA UDHAAA OOOOxx +3097 4831 1 1 7 17 97 97 1097 3097 3097 194 195 DPAAAA VDHAAA VVVVxx +1528 4832 0 0 8 8 28 528 1528 1528 1528 56 57 UGAAAA WDHAAA AAAAxx +6429 4833 1 1 9 9 29 429 429 1429 6429 58 59 HNAAAA XDHAAA HHHHxx +7320 4834 0 0 0 0 20 320 1320 2320 7320 40 41 OVAAAA YDHAAA OOOOxx +844 4835 0 0 4 4 44 844 844 844 844 88 89 MGAAAA ZDHAAA VVVVxx +7054 4836 0 2 4 14 54 54 1054 2054 7054 108 109 ILAAAA AEHAAA AAAAxx +1643 4837 1 3 3 3 43 643 1643 1643 1643 86 87 FLAAAA BEHAAA HHHHxx +7626 4838 0 2 6 6 26 626 1626 2626 7626 52 53 IHAAAA CEHAAA OOOOxx +8728 4839 0 0 8 8 28 728 728 3728 8728 56 57 SXAAAA DEHAAA VVVVxx +8277 4840 1 1 7 17 77 277 277 3277 8277 154 155 JGAAAA EEHAAA AAAAxx +189 4841 1 1 9 9 89 189 189 189 189 178 179 HHAAAA FEHAAA HHHHxx +3717 4842 1 1 7 17 17 717 1717 3717 3717 34 35 ZMAAAA GEHAAA OOOOxx +1020 4843 0 0 0 0 20 20 1020 1020 1020 40 41 GNAAAA HEHAAA VVVVxx +9234 4844 0 2 4 14 34 234 1234 4234 9234 68 69 ERAAAA IEHAAA AAAAxx +9541 4845 1 1 1 1 41 541 1541 4541 9541 82 83 ZCAAAA JEHAAA HHHHxx +380 4846 0 0 0 0 80 380 380 380 380 160 161 QOAAAA KEHAAA OOOOxx +397 4847 1 1 7 17 97 397 397 397 397 194 195 HPAAAA LEHAAA VVVVxx +835 4848 1 3 5 15 35 835 835 835 835 70 71 DGAAAA MEHAAA AAAAxx +347 4849 1 3 7 7 47 347 347 347 347 94 95 JNAAAA NEHAAA HHHHxx +2490 4850 0 2 0 10 90 490 490 2490 2490 180 181 URAAAA OEHAAA OOOOxx +605 4851 1 1 5 5 5 605 605 605 605 10 11 HXAAAA PEHAAA VVVVxx +7960 4852 0 0 0 0 60 960 1960 2960 7960 120 121 EUAAAA QEHAAA AAAAxx +9681 4853 1 1 1 1 81 681 1681 4681 9681 162 163 JIAAAA REHAAA HHHHxx +5753 4854 1 1 3 13 53 753 1753 753 5753 106 107 HNAAAA SEHAAA OOOOxx +1676 4855 0 0 6 16 76 676 1676 1676 1676 152 153 MMAAAA TEHAAA VVVVxx +5533 4856 1 1 3 13 33 533 1533 533 5533 66 67 VEAAAA UEHAAA AAAAxx +8958 4857 0 2 8 18 58 958 958 3958 8958 116 117 OGAAAA VEHAAA HHHHxx +664 4858 0 0 4 4 64 664 664 664 664 128 129 OZAAAA WEHAAA OOOOxx +3005 4859 1 1 5 5 5 5 1005 3005 3005 10 11 PLAAAA XEHAAA VVVVxx +8576 4860 0 0 6 16 76 576 576 3576 8576 152 153 WRAAAA YEHAAA AAAAxx +7304 4861 0 0 4 4 4 304 1304 2304 7304 8 9 YUAAAA ZEHAAA HHHHxx +3375 4862 1 3 5 15 75 375 1375 3375 3375 150 151 VZAAAA AFHAAA OOOOxx +6336 4863 0 0 6 16 36 336 336 1336 6336 72 73 SJAAAA BFHAAA VVVVxx +1392 4864 0 0 2 12 92 392 1392 1392 1392 184 185 OBAAAA CFHAAA AAAAxx +2925 4865 1 1 5 5 25 925 925 2925 2925 50 51 NIAAAA DFHAAA HHHHxx +1217 4866 1 1 7 17 17 217 1217 1217 1217 34 35 VUAAAA EFHAAA OOOOxx +3714 4867 0 2 4 14 14 714 1714 3714 3714 28 29 WMAAAA FFHAAA VVVVxx +2120 4868 0 0 0 0 20 120 120 2120 2120 40 41 ODAAAA GFHAAA AAAAxx +2845 4869 1 1 5 5 45 845 845 2845 2845 90 91 LFAAAA HFHAAA HHHHxx +3865 4870 1 1 5 5 65 865 1865 3865 3865 130 131 RSAAAA IFHAAA OOOOxx +124 4871 0 0 4 4 24 124 124 124 124 48 49 UEAAAA JFHAAA VVVVxx +865 4872 1 1 5 5 65 865 865 865 865 130 131 HHAAAA KFHAAA AAAAxx +9361 4873 1 1 1 1 61 361 1361 4361 9361 122 123 BWAAAA LFHAAA HHHHxx +6338 4874 0 2 8 18 38 338 338 1338 6338 76 77 UJAAAA MFHAAA OOOOxx +7330 4875 0 2 0 10 30 330 1330 2330 7330 60 61 YVAAAA NFHAAA VVVVxx +513 4876 1 1 3 13 13 513 513 513 513 26 27 TTAAAA OFHAAA AAAAxx +5001 4877 1 1 1 1 1 1 1001 1 5001 2 3 JKAAAA PFHAAA HHHHxx +549 4878 1 1 9 9 49 549 549 549 549 98 99 DVAAAA QFHAAA OOOOxx +1808 4879 0 0 8 8 8 808 1808 1808 1808 16 17 ORAAAA RFHAAA VVVVxx +7168 4880 0 0 8 8 68 168 1168 2168 7168 136 137 SPAAAA SFHAAA AAAAxx +9878 4881 0 2 8 18 78 878 1878 4878 9878 156 157 YPAAAA TFHAAA HHHHxx +233 4882 1 1 3 13 33 233 233 233 233 66 67 ZIAAAA UFHAAA OOOOxx +4262 4883 0 2 2 2 62 262 262 4262 4262 124 125 YHAAAA VFHAAA VVVVxx +7998 4884 0 2 8 18 98 998 1998 2998 7998 196 197 QVAAAA WFHAAA AAAAxx +2419 4885 1 3 9 19 19 419 419 2419 2419 38 39 BPAAAA XFHAAA HHHHxx +9960 4886 0 0 0 0 60 960 1960 4960 9960 120 121 CTAAAA YFHAAA OOOOxx +3523 4887 1 3 3 3 23 523 1523 3523 3523 46 47 NFAAAA ZFHAAA VVVVxx +5440 4888 0 0 0 0 40 440 1440 440 5440 80 81 GBAAAA AGHAAA AAAAxx +3030 4889 0 2 0 10 30 30 1030 3030 3030 60 61 OMAAAA BGHAAA HHHHxx +2745 4890 1 1 5 5 45 745 745 2745 2745 90 91 PBAAAA CGHAAA OOOOxx +7175 4891 1 3 5 15 75 175 1175 2175 7175 150 151 ZPAAAA DGHAAA VVVVxx +640 4892 0 0 0 0 40 640 640 640 640 80 81 QYAAAA EGHAAA AAAAxx +1798 4893 0 2 8 18 98 798 1798 1798 1798 196 197 ERAAAA FGHAAA HHHHxx +7499 4894 1 3 9 19 99 499 1499 2499 7499 198 199 LCAAAA GGHAAA OOOOxx +1924 4895 0 0 4 4 24 924 1924 1924 1924 48 49 AWAAAA HGHAAA VVVVxx +1327 4896 1 3 7 7 27 327 1327 1327 1327 54 55 BZAAAA IGHAAA AAAAxx +73 4897 1 1 3 13 73 73 73 73 73 146 147 VCAAAA JGHAAA HHHHxx +9558 4898 0 2 8 18 58 558 1558 4558 9558 116 117 QDAAAA KGHAAA OOOOxx +818 4899 0 2 8 18 18 818 818 818 818 36 37 MFAAAA LGHAAA VVVVxx +9916 4900 0 0 6 16 16 916 1916 4916 9916 32 33 KRAAAA MGHAAA AAAAxx +2978 4901 0 2 8 18 78 978 978 2978 2978 156 157 OKAAAA NGHAAA HHHHxx +8469 4902 1 1 9 9 69 469 469 3469 8469 138 139 TNAAAA OGHAAA OOOOxx +9845 4903 1 1 5 5 45 845 1845 4845 9845 90 91 ROAAAA PGHAAA VVVVxx +2326 4904 0 2 6 6 26 326 326 2326 2326 52 53 MLAAAA QGHAAA AAAAxx +4032 4905 0 0 2 12 32 32 32 4032 4032 64 65 CZAAAA RGHAAA HHHHxx +5604 4906 0 0 4 4 4 604 1604 604 5604 8 9 OHAAAA SGHAAA OOOOxx +9610 4907 0 2 0 10 10 610 1610 4610 9610 20 21 QFAAAA TGHAAA VVVVxx +5101 4908 1 1 1 1 1 101 1101 101 5101 2 3 FOAAAA UGHAAA AAAAxx +7246 4909 0 2 6 6 46 246 1246 2246 7246 92 93 SSAAAA VGHAAA HHHHxx +1292 4910 0 0 2 12 92 292 1292 1292 1292 184 185 SXAAAA WGHAAA OOOOxx +6235 4911 1 3 5 15 35 235 235 1235 6235 70 71 VFAAAA XGHAAA VVVVxx +1733 4912 1 1 3 13 33 733 1733 1733 1733 66 67 ROAAAA YGHAAA AAAAxx +4647 4913 1 3 7 7 47 647 647 4647 4647 94 95 TWAAAA ZGHAAA HHHHxx +258 4914 0 2 8 18 58 258 258 258 258 116 117 YJAAAA AHHAAA OOOOxx +8438 4915 0 2 8 18 38 438 438 3438 8438 76 77 OMAAAA BHHAAA VVVVxx +7869 4916 1 1 9 9 69 869 1869 2869 7869 138 139 RQAAAA CHHAAA AAAAxx +9691 4917 1 3 1 11 91 691 1691 4691 9691 182 183 TIAAAA DHHAAA HHHHxx +5422 4918 0 2 2 2 22 422 1422 422 5422 44 45 OAAAAA EHHAAA OOOOxx +9630 4919 0 2 0 10 30 630 1630 4630 9630 60 61 KGAAAA FHHAAA VVVVxx +4439 4920 1 3 9 19 39 439 439 4439 4439 78 79 TOAAAA GHHAAA AAAAxx +3140 4921 0 0 0 0 40 140 1140 3140 3140 80 81 UQAAAA HHHAAA HHHHxx +9111 4922 1 3 1 11 11 111 1111 4111 9111 22 23 LMAAAA IHHAAA OOOOxx +4606 4923 0 2 6 6 6 606 606 4606 4606 12 13 EVAAAA JHHAAA VVVVxx +8620 4924 0 0 0 0 20 620 620 3620 8620 40 41 OTAAAA KHHAAA AAAAxx +7849 4925 1 1 9 9 49 849 1849 2849 7849 98 99 XPAAAA LHHAAA HHHHxx +346 4926 0 2 6 6 46 346 346 346 346 92 93 INAAAA MHHAAA OOOOxx +9528 4927 0 0 8 8 28 528 1528 4528 9528 56 57 MCAAAA NHHAAA VVVVxx +1811 4928 1 3 1 11 11 811 1811 1811 1811 22 23 RRAAAA OHHAAA AAAAxx +6068 4929 0 0 8 8 68 68 68 1068 6068 136 137 KZAAAA PHHAAA HHHHxx +6260 4930 0 0 0 0 60 260 260 1260 6260 120 121 UGAAAA QHHAAA OOOOxx +5909 4931 1 1 9 9 9 909 1909 909 5909 18 19 HTAAAA RHHAAA VVVVxx +4518 4932 0 2 8 18 18 518 518 4518 4518 36 37 URAAAA SHHAAA AAAAxx +7530 4933 0 2 0 10 30 530 1530 2530 7530 60 61 QDAAAA THHAAA HHHHxx +3900 4934 0 0 0 0 0 900 1900 3900 3900 0 1 AUAAAA UHHAAA OOOOxx +3969 4935 1 1 9 9 69 969 1969 3969 3969 138 139 RWAAAA VHHAAA VVVVxx +8690 4936 0 2 0 10 90 690 690 3690 8690 180 181 GWAAAA WHHAAA AAAAxx +5532 4937 0 0 2 12 32 532 1532 532 5532 64 65 UEAAAA XHHAAA HHHHxx +5989 4938 1 1 9 9 89 989 1989 989 5989 178 179 JWAAAA YHHAAA OOOOxx +1870 4939 0 2 0 10 70 870 1870 1870 1870 140 141 YTAAAA ZHHAAA VVVVxx +1113 4940 1 1 3 13 13 113 1113 1113 1113 26 27 VQAAAA AIHAAA AAAAxx +5155 4941 1 3 5 15 55 155 1155 155 5155 110 111 HQAAAA BIHAAA HHHHxx +7460 4942 0 0 0 0 60 460 1460 2460 7460 120 121 YAAAAA CIHAAA OOOOxx +6217 4943 1 1 7 17 17 217 217 1217 6217 34 35 DFAAAA DIHAAA VVVVxx +8333 4944 1 1 3 13 33 333 333 3333 8333 66 67 NIAAAA EIHAAA AAAAxx +6341 4945 1 1 1 1 41 341 341 1341 6341 82 83 XJAAAA FIHAAA HHHHxx +6230 4946 0 2 0 10 30 230 230 1230 6230 60 61 QFAAAA GIHAAA OOOOxx +6902 4947 0 2 2 2 2 902 902 1902 6902 4 5 MFAAAA HIHAAA VVVVxx +670 4948 0 2 0 10 70 670 670 670 670 140 141 UZAAAA IIHAAA AAAAxx +805 4949 1 1 5 5 5 805 805 805 805 10 11 ZEAAAA JIHAAA HHHHxx +1340 4950 0 0 0 0 40 340 1340 1340 1340 80 81 OZAAAA KIHAAA OOOOxx +8649 4951 1 1 9 9 49 649 649 3649 8649 98 99 RUAAAA LIHAAA VVVVxx +3887 4952 1 3 7 7 87 887 1887 3887 3887 174 175 NTAAAA MIHAAA AAAAxx +5400 4953 0 0 0 0 0 400 1400 400 5400 0 1 SZAAAA NIHAAA HHHHxx +4354 4954 0 2 4 14 54 354 354 4354 4354 108 109 MLAAAA OIHAAA OOOOxx +950 4955 0 2 0 10 50 950 950 950 950 100 101 OKAAAA PIHAAA VVVVxx +1544 4956 0 0 4 4 44 544 1544 1544 1544 88 89 KHAAAA QIHAAA AAAAxx +3898 4957 0 2 8 18 98 898 1898 3898 3898 196 197 YTAAAA RIHAAA HHHHxx +8038 4958 0 2 8 18 38 38 38 3038 8038 76 77 EXAAAA SIHAAA OOOOxx +1095 4959 1 3 5 15 95 95 1095 1095 1095 190 191 DQAAAA TIHAAA VVVVxx +1748 4960 0 0 8 8 48 748 1748 1748 1748 96 97 GPAAAA UIHAAA AAAAxx +9154 4961 0 2 4 14 54 154 1154 4154 9154 108 109 COAAAA VIHAAA HHHHxx +2182 4962 0 2 2 2 82 182 182 2182 2182 164 165 YFAAAA WIHAAA OOOOxx +6797 4963 1 1 7 17 97 797 797 1797 6797 194 195 LBAAAA XIHAAA VVVVxx +9149 4964 1 1 9 9 49 149 1149 4149 9149 98 99 XNAAAA YIHAAA AAAAxx +7351 4965 1 3 1 11 51 351 1351 2351 7351 102 103 TWAAAA ZIHAAA HHHHxx +2820 4966 0 0 0 0 20 820 820 2820 2820 40 41 MEAAAA AJHAAA OOOOxx +9696 4967 0 0 6 16 96 696 1696 4696 9696 192 193 YIAAAA BJHAAA VVVVxx +253 4968 1 1 3 13 53 253 253 253 253 106 107 TJAAAA CJHAAA AAAAxx +3600 4969 0 0 0 0 0 600 1600 3600 3600 0 1 MIAAAA DJHAAA HHHHxx +3892 4970 0 0 2 12 92 892 1892 3892 3892 184 185 STAAAA EJHAAA OOOOxx +231 4971 1 3 1 11 31 231 231 231 231 62 63 XIAAAA FJHAAA VVVVxx +8331 4972 1 3 1 11 31 331 331 3331 8331 62 63 LIAAAA GJHAAA AAAAxx +403 4973 1 3 3 3 3 403 403 403 403 6 7 NPAAAA HJHAAA HHHHxx +8642 4974 0 2 2 2 42 642 642 3642 8642 84 85 KUAAAA IJHAAA OOOOxx +3118 4975 0 2 8 18 18 118 1118 3118 3118 36 37 YPAAAA JJHAAA VVVVxx +3835 4976 1 3 5 15 35 835 1835 3835 3835 70 71 NRAAAA KJHAAA AAAAxx +1117 4977 1 1 7 17 17 117 1117 1117 1117 34 35 ZQAAAA LJHAAA HHHHxx +7024 4978 0 0 4 4 24 24 1024 2024 7024 48 49 EKAAAA MJHAAA OOOOxx +2636 4979 0 0 6 16 36 636 636 2636 2636 72 73 KXAAAA NJHAAA VVVVxx +3778 4980 0 2 8 18 78 778 1778 3778 3778 156 157 IPAAAA OJHAAA AAAAxx +2003 4981 1 3 3 3 3 3 3 2003 2003 6 7 BZAAAA PJHAAA HHHHxx +5717 4982 1 1 7 17 17 717 1717 717 5717 34 35 XLAAAA QJHAAA OOOOxx +4869 4983 1 1 9 9 69 869 869 4869 4869 138 139 HFAAAA RJHAAA VVVVxx +8921 4984 1 1 1 1 21 921 921 3921 8921 42 43 DFAAAA SJHAAA AAAAxx +888 4985 0 0 8 8 88 888 888 888 888 176 177 EIAAAA TJHAAA HHHHxx +7599 4986 1 3 9 19 99 599 1599 2599 7599 198 199 HGAAAA UJHAAA OOOOxx +8621 4987 1 1 1 1 21 621 621 3621 8621 42 43 PTAAAA VJHAAA VVVVxx +811 4988 1 3 1 11 11 811 811 811 811 22 23 FFAAAA WJHAAA AAAAxx +9147 4989 1 3 7 7 47 147 1147 4147 9147 94 95 VNAAAA XJHAAA HHHHxx +1413 4990 1 1 3 13 13 413 1413 1413 1413 26 27 JCAAAA YJHAAA OOOOxx +5232 4991 0 0 2 12 32 232 1232 232 5232 64 65 GTAAAA ZJHAAA VVVVxx +5912 4992 0 0 2 12 12 912 1912 912 5912 24 25 KTAAAA AKHAAA AAAAxx +3418 4993 0 2 8 18 18 418 1418 3418 3418 36 37 MBAAAA BKHAAA HHHHxx +3912 4994 0 0 2 12 12 912 1912 3912 3912 24 25 MUAAAA CKHAAA OOOOxx +9576 4995 0 0 6 16 76 576 1576 4576 9576 152 153 IEAAAA DKHAAA VVVVxx +4225 4996 1 1 5 5 25 225 225 4225 4225 50 51 NGAAAA EKHAAA AAAAxx +8222 4997 0 2 2 2 22 222 222 3222 8222 44 45 GEAAAA FKHAAA HHHHxx +7013 4998 1 1 3 13 13 13 1013 2013 7013 26 27 TJAAAA GKHAAA OOOOxx +7037 4999 1 1 7 17 37 37 1037 2037 7037 74 75 RKAAAA HKHAAA VVVVxx +1205 5000 1 1 5 5 5 205 1205 1205 1205 10 11 JUAAAA IKHAAA AAAAxx +8114 5001 0 2 4 14 14 114 114 3114 8114 28 29 CAAAAA JKHAAA HHHHxx +6585 5002 1 1 5 5 85 585 585 1585 6585 170 171 HTAAAA KKHAAA OOOOxx +155 5003 1 3 5 15 55 155 155 155 155 110 111 ZFAAAA LKHAAA VVVVxx +2841 5004 1 1 1 1 41 841 841 2841 2841 82 83 HFAAAA MKHAAA AAAAxx +1996 5005 0 0 6 16 96 996 1996 1996 1996 192 193 UYAAAA NKHAAA HHHHxx +4948 5006 0 0 8 8 48 948 948 4948 4948 96 97 IIAAAA OKHAAA OOOOxx +3304 5007 0 0 4 4 4 304 1304 3304 3304 8 9 CXAAAA PKHAAA VVVVxx +5684 5008 0 0 4 4 84 684 1684 684 5684 168 169 QKAAAA QKHAAA AAAAxx +6962 5009 0 2 2 2 62 962 962 1962 6962 124 125 UHAAAA RKHAAA HHHHxx +8691 5010 1 3 1 11 91 691 691 3691 8691 182 183 HWAAAA SKHAAA OOOOxx +8501 5011 1 1 1 1 1 501 501 3501 8501 2 3 ZOAAAA TKHAAA VVVVxx +4783 5012 1 3 3 3 83 783 783 4783 4783 166 167 ZBAAAA UKHAAA AAAAxx +3762 5013 0 2 2 2 62 762 1762 3762 3762 124 125 SOAAAA VKHAAA HHHHxx +4534 5014 0 2 4 14 34 534 534 4534 4534 68 69 KSAAAA WKHAAA OOOOxx +4999 5015 1 3 9 19 99 999 999 4999 4999 198 199 HKAAAA XKHAAA VVVVxx +4618 5016 0 2 8 18 18 618 618 4618 4618 36 37 QVAAAA YKHAAA AAAAxx +4220 5017 0 0 0 0 20 220 220 4220 4220 40 41 IGAAAA ZKHAAA HHHHxx +3384 5018 0 0 4 4 84 384 1384 3384 3384 168 169 EAAAAA ALHAAA OOOOxx +3036 5019 0 0 6 16 36 36 1036 3036 3036 72 73 UMAAAA BLHAAA VVVVxx +545 5020 1 1 5 5 45 545 545 545 545 90 91 ZUAAAA CLHAAA AAAAxx +9946 5021 0 2 6 6 46 946 1946 4946 9946 92 93 OSAAAA DLHAAA HHHHxx +1985 5022 1 1 5 5 85 985 1985 1985 1985 170 171 JYAAAA ELHAAA OOOOxx +2310 5023 0 2 0 10 10 310 310 2310 2310 20 21 WKAAAA FLHAAA VVVVxx +6563 5024 1 3 3 3 63 563 563 1563 6563 126 127 LSAAAA GLHAAA AAAAxx +4886 5025 0 2 6 6 86 886 886 4886 4886 172 173 YFAAAA HLHAAA HHHHxx +9359 5026 1 3 9 19 59 359 1359 4359 9359 118 119 ZVAAAA ILHAAA OOOOxx +400 5027 0 0 0 0 0 400 400 400 400 0 1 KPAAAA JLHAAA VVVVxx +9742 5028 0 2 2 2 42 742 1742 4742 9742 84 85 SKAAAA KLHAAA AAAAxx +6736 5029 0 0 6 16 36 736 736 1736 6736 72 73 CZAAAA LLHAAA HHHHxx +8166 5030 0 2 6 6 66 166 166 3166 8166 132 133 CCAAAA MLHAAA OOOOxx +861 5031 1 1 1 1 61 861 861 861 861 122 123 DHAAAA NLHAAA VVVVxx +7492 5032 0 0 2 12 92 492 1492 2492 7492 184 185 ECAAAA OLHAAA AAAAxx +1155 5033 1 3 5 15 55 155 1155 1155 1155 110 111 LSAAAA PLHAAA HHHHxx +9769 5034 1 1 9 9 69 769 1769 4769 9769 138 139 TLAAAA QLHAAA OOOOxx +6843 5035 1 3 3 3 43 843 843 1843 6843 86 87 FDAAAA RLHAAA VVVVxx +5625 5036 1 1 5 5 25 625 1625 625 5625 50 51 JIAAAA SLHAAA AAAAxx +1910 5037 0 2 0 10 10 910 1910 1910 1910 20 21 MVAAAA TLHAAA HHHHxx +9796 5038 0 0 6 16 96 796 1796 4796 9796 192 193 UMAAAA ULHAAA OOOOxx +6950 5039 0 2 0 10 50 950 950 1950 6950 100 101 IHAAAA VLHAAA VVVVxx +3084 5040 0 0 4 4 84 84 1084 3084 3084 168 169 QOAAAA WLHAAA AAAAxx +2959 5041 1 3 9 19 59 959 959 2959 2959 118 119 VJAAAA XLHAAA HHHHxx +2093 5042 1 1 3 13 93 93 93 2093 2093 186 187 NCAAAA YLHAAA OOOOxx +2738 5043 0 2 8 18 38 738 738 2738 2738 76 77 IBAAAA ZLHAAA VVVVxx +6406 5044 0 2 6 6 6 406 406 1406 6406 12 13 KMAAAA AMHAAA AAAAxx +9082 5045 0 2 2 2 82 82 1082 4082 9082 164 165 ILAAAA BMHAAA HHHHxx +8568 5046 0 0 8 8 68 568 568 3568 8568 136 137 ORAAAA CMHAAA OOOOxx +3566 5047 0 2 6 6 66 566 1566 3566 3566 132 133 EHAAAA DMHAAA VVVVxx +3016 5048 0 0 6 16 16 16 1016 3016 3016 32 33 AMAAAA EMHAAA AAAAxx +1207 5049 1 3 7 7 7 207 1207 1207 1207 14 15 LUAAAA FMHAAA HHHHxx +4045 5050 1 1 5 5 45 45 45 4045 4045 90 91 PZAAAA GMHAAA OOOOxx +4173 5051 1 1 3 13 73 173 173 4173 4173 146 147 NEAAAA HMHAAA VVVVxx +3939 5052 1 3 9 19 39 939 1939 3939 3939 78 79 NVAAAA IMHAAA AAAAxx +9683 5053 1 3 3 3 83 683 1683 4683 9683 166 167 LIAAAA JMHAAA HHHHxx +1684 5054 0 0 4 4 84 684 1684 1684 1684 168 169 UMAAAA KMHAAA OOOOxx +9271 5055 1 3 1 11 71 271 1271 4271 9271 142 143 PSAAAA LMHAAA VVVVxx +9317 5056 1 1 7 17 17 317 1317 4317 9317 34 35 JUAAAA MMHAAA AAAAxx +5793 5057 1 1 3 13 93 793 1793 793 5793 186 187 VOAAAA NMHAAA HHHHxx +352 5058 0 0 2 12 52 352 352 352 352 104 105 ONAAAA OMHAAA OOOOxx +7328 5059 0 0 8 8 28 328 1328 2328 7328 56 57 WVAAAA PMHAAA VVVVxx +4582 5060 0 2 2 2 82 582 582 4582 4582 164 165 GUAAAA QMHAAA AAAAxx +7413 5061 1 1 3 13 13 413 1413 2413 7413 26 27 DZAAAA RMHAAA HHHHxx +6772 5062 0 0 2 12 72 772 772 1772 6772 144 145 MAAAAA SMHAAA OOOOxx +4973 5063 1 1 3 13 73 973 973 4973 4973 146 147 HJAAAA TMHAAA VVVVxx +7480 5064 0 0 0 0 80 480 1480 2480 7480 160 161 SBAAAA UMHAAA AAAAxx +5555 5065 1 3 5 15 55 555 1555 555 5555 110 111 RFAAAA VMHAAA HHHHxx +4227 5066 1 3 7 7 27 227 227 4227 4227 54 55 PGAAAA WMHAAA OOOOxx +4153 5067 1 1 3 13 53 153 153 4153 4153 106 107 TDAAAA XMHAAA VVVVxx +4601 5068 1 1 1 1 1 601 601 4601 4601 2 3 ZUAAAA YMHAAA AAAAxx +3782 5069 0 2 2 2 82 782 1782 3782 3782 164 165 MPAAAA ZMHAAA HHHHxx +3872 5070 0 0 2 12 72 872 1872 3872 3872 144 145 YSAAAA ANHAAA OOOOxx +893 5071 1 1 3 13 93 893 893 893 893 186 187 JIAAAA BNHAAA VVVVxx +2430 5072 0 2 0 10 30 430 430 2430 2430 60 61 MPAAAA CNHAAA AAAAxx +2591 5073 1 3 1 11 91 591 591 2591 2591 182 183 RVAAAA DNHAAA HHHHxx +264 5074 0 0 4 4 64 264 264 264 264 128 129 EKAAAA ENHAAA OOOOxx +6238 5075 0 2 8 18 38 238 238 1238 6238 76 77 YFAAAA FNHAAA VVVVxx +633 5076 1 1 3 13 33 633 633 633 633 66 67 JYAAAA GNHAAA AAAAxx +1029 5077 1 1 9 9 29 29 1029 1029 1029 58 59 PNAAAA HNHAAA HHHHxx +5934 5078 0 2 4 14 34 934 1934 934 5934 68 69 GUAAAA INHAAA OOOOxx +8694 5079 0 2 4 14 94 694 694 3694 8694 188 189 KWAAAA JNHAAA VVVVxx +7401 5080 1 1 1 1 1 401 1401 2401 7401 2 3 RYAAAA KNHAAA AAAAxx +1165 5081 1 1 5 5 65 165 1165 1165 1165 130 131 VSAAAA LNHAAA HHHHxx +9438 5082 0 2 8 18 38 438 1438 4438 9438 76 77 AZAAAA MNHAAA OOOOxx +4790 5083 0 2 0 10 90 790 790 4790 4790 180 181 GCAAAA NNHAAA VVVVxx +4531 5084 1 3 1 11 31 531 531 4531 4531 62 63 HSAAAA ONHAAA AAAAxx +6099 5085 1 3 9 19 99 99 99 1099 6099 198 199 PAAAAA PNHAAA HHHHxx +8236 5086 0 0 6 16 36 236 236 3236 8236 72 73 UEAAAA QNHAAA OOOOxx +8551 5087 1 3 1 11 51 551 551 3551 8551 102 103 XQAAAA RNHAAA VVVVxx +3128 5088 0 0 8 8 28 128 1128 3128 3128 56 57 IQAAAA SNHAAA AAAAxx +3504 5089 0 0 4 4 4 504 1504 3504 3504 8 9 UEAAAA TNHAAA HHHHxx +9071 5090 1 3 1 11 71 71 1071 4071 9071 142 143 XKAAAA UNHAAA OOOOxx +5930 5091 0 2 0 10 30 930 1930 930 5930 60 61 CUAAAA VNHAAA VVVVxx +6825 5092 1 1 5 5 25 825 825 1825 6825 50 51 NCAAAA WNHAAA AAAAxx +2218 5093 0 2 8 18 18 218 218 2218 2218 36 37 IHAAAA XNHAAA HHHHxx +3604 5094 0 0 4 4 4 604 1604 3604 3604 8 9 QIAAAA YNHAAA OOOOxx +5761 5095 1 1 1 1 61 761 1761 761 5761 122 123 PNAAAA ZNHAAA VVVVxx +5414 5096 0 2 4 14 14 414 1414 414 5414 28 29 GAAAAA AOHAAA AAAAxx +5892 5097 0 0 2 12 92 892 1892 892 5892 184 185 QSAAAA BOHAAA HHHHxx +4080 5098 0 0 0 0 80 80 80 4080 4080 160 161 YAAAAA COHAAA OOOOxx +8018 5099 0 2 8 18 18 18 18 3018 8018 36 37 KWAAAA DOHAAA VVVVxx +1757 5100 1 1 7 17 57 757 1757 1757 1757 114 115 PPAAAA EOHAAA AAAAxx +5854 5101 0 2 4 14 54 854 1854 854 5854 108 109 ERAAAA FOHAAA HHHHxx +1335 5102 1 3 5 15 35 335 1335 1335 1335 70 71 JZAAAA GOHAAA OOOOxx +3811 5103 1 3 1 11 11 811 1811 3811 3811 22 23 PQAAAA HOHAAA VVVVxx +9917 5104 1 1 7 17 17 917 1917 4917 9917 34 35 LRAAAA IOHAAA AAAAxx +5947 5105 1 3 7 7 47 947 1947 947 5947 94 95 TUAAAA JOHAAA HHHHxx +7263 5106 1 3 3 3 63 263 1263 2263 7263 126 127 JTAAAA KOHAAA OOOOxx +1730 5107 0 2 0 10 30 730 1730 1730 1730 60 61 OOAAAA LOHAAA VVVVxx +5747 5108 1 3 7 7 47 747 1747 747 5747 94 95 BNAAAA MOHAAA AAAAxx +3876 5109 0 0 6 16 76 876 1876 3876 3876 152 153 CTAAAA NOHAAA HHHHxx +2762 5110 0 2 2 2 62 762 762 2762 2762 124 125 GCAAAA OOHAAA OOOOxx +7613 5111 1 1 3 13 13 613 1613 2613 7613 26 27 VGAAAA POHAAA VVVVxx +152 5112 0 0 2 12 52 152 152 152 152 104 105 WFAAAA QOHAAA AAAAxx +3941 5113 1 1 1 1 41 941 1941 3941 3941 82 83 PVAAAA ROHAAA HHHHxx +5614 5114 0 2 4 14 14 614 1614 614 5614 28 29 YHAAAA SOHAAA OOOOxx +9279 5115 1 3 9 19 79 279 1279 4279 9279 158 159 XSAAAA TOHAAA VVVVxx +3048 5116 0 0 8 8 48 48 1048 3048 3048 96 97 GNAAAA UOHAAA AAAAxx +6152 5117 0 0 2 12 52 152 152 1152 6152 104 105 QCAAAA VOHAAA HHHHxx +5481 5118 1 1 1 1 81 481 1481 481 5481 162 163 VCAAAA WOHAAA OOOOxx +4675 5119 1 3 5 15 75 675 675 4675 4675 150 151 VXAAAA XOHAAA VVVVxx +3334 5120 0 2 4 14 34 334 1334 3334 3334 68 69 GYAAAA YOHAAA AAAAxx +4691 5121 1 3 1 11 91 691 691 4691 4691 182 183 LYAAAA ZOHAAA HHHHxx +803 5122 1 3 3 3 3 803 803 803 803 6 7 XEAAAA APHAAA OOOOxx +5409 5123 1 1 9 9 9 409 1409 409 5409 18 19 BAAAAA BPHAAA VVVVxx +1054 5124 0 2 4 14 54 54 1054 1054 1054 108 109 OOAAAA CPHAAA AAAAxx +103 5125 1 3 3 3 3 103 103 103 103 6 7 ZDAAAA DPHAAA HHHHxx +8565 5126 1 1 5 5 65 565 565 3565 8565 130 131 LRAAAA EPHAAA OOOOxx +4666 5127 0 2 6 6 66 666 666 4666 4666 132 133 MXAAAA FPHAAA VVVVxx +6634 5128 0 2 4 14 34 634 634 1634 6634 68 69 EVAAAA GPHAAA AAAAxx +5538 5129 0 2 8 18 38 538 1538 538 5538 76 77 AFAAAA HPHAAA HHHHxx +3789 5130 1 1 9 9 89 789 1789 3789 3789 178 179 TPAAAA IPHAAA OOOOxx +4641 5131 1 1 1 1 41 641 641 4641 4641 82 83 NWAAAA JPHAAA VVVVxx +2458 5132 0 2 8 18 58 458 458 2458 2458 116 117 OQAAAA KPHAAA AAAAxx +5667 5133 1 3 7 7 67 667 1667 667 5667 134 135 ZJAAAA LPHAAA HHHHxx +6524 5134 0 0 4 4 24 524 524 1524 6524 48 49 YQAAAA MPHAAA OOOOxx +9179 5135 1 3 9 19 79 179 1179 4179 9179 158 159 BPAAAA NPHAAA VVVVxx +6358 5136 0 2 8 18 58 358 358 1358 6358 116 117 OKAAAA OPHAAA AAAAxx +6668 5137 0 0 8 8 68 668 668 1668 6668 136 137 MWAAAA PPHAAA HHHHxx +6414 5138 0 2 4 14 14 414 414 1414 6414 28 29 SMAAAA QPHAAA OOOOxx +2813 5139 1 1 3 13 13 813 813 2813 2813 26 27 FEAAAA RPHAAA VVVVxx +8927 5140 1 3 7 7 27 927 927 3927 8927 54 55 JFAAAA SPHAAA AAAAxx +8695 5141 1 3 5 15 95 695 695 3695 8695 190 191 LWAAAA TPHAAA HHHHxx +363 5142 1 3 3 3 63 363 363 363 363 126 127 ZNAAAA UPHAAA OOOOxx +9966 5143 0 2 6 6 66 966 1966 4966 9966 132 133 ITAAAA VPHAAA VVVVxx +1323 5144 1 3 3 3 23 323 1323 1323 1323 46 47 XYAAAA WPHAAA AAAAxx +8211 5145 1 3 1 11 11 211 211 3211 8211 22 23 VDAAAA XPHAAA HHHHxx +4375 5146 1 3 5 15 75 375 375 4375 4375 150 151 HMAAAA YPHAAA OOOOxx +3257 5147 1 1 7 17 57 257 1257 3257 3257 114 115 HVAAAA ZPHAAA VVVVxx +6239 5148 1 3 9 19 39 239 239 1239 6239 78 79 ZFAAAA AQHAAA AAAAxx +3602 5149 0 2 2 2 2 602 1602 3602 3602 4 5 OIAAAA BQHAAA HHHHxx +9830 5150 0 2 0 10 30 830 1830 4830 9830 60 61 COAAAA CQHAAA OOOOxx +7826 5151 0 2 6 6 26 826 1826 2826 7826 52 53 APAAAA DQHAAA VVVVxx +2108 5152 0 0 8 8 8 108 108 2108 2108 16 17 CDAAAA EQHAAA AAAAxx +7245 5153 1 1 5 5 45 245 1245 2245 7245 90 91 RSAAAA FQHAAA HHHHxx +8330 5154 0 2 0 10 30 330 330 3330 8330 60 61 KIAAAA GQHAAA OOOOxx +7441 5155 1 1 1 1 41 441 1441 2441 7441 82 83 FAAAAA HQHAAA VVVVxx +9848 5156 0 0 8 8 48 848 1848 4848 9848 96 97 UOAAAA IQHAAA AAAAxx +1226 5157 0 2 6 6 26 226 1226 1226 1226 52 53 EVAAAA JQHAAA HHHHxx +414 5158 0 2 4 14 14 414 414 414 414 28 29 YPAAAA KQHAAA OOOOxx +1273 5159 1 1 3 13 73 273 1273 1273 1273 146 147 ZWAAAA LQHAAA VVVVxx +9866 5160 0 2 6 6 66 866 1866 4866 9866 132 133 MPAAAA MQHAAA AAAAxx +4633 5161 1 1 3 13 33 633 633 4633 4633 66 67 FWAAAA NQHAAA HHHHxx +8727 5162 1 3 7 7 27 727 727 3727 8727 54 55 RXAAAA OQHAAA OOOOxx +5308 5163 0 0 8 8 8 308 1308 308 5308 16 17 EWAAAA PQHAAA VVVVxx +1395 5164 1 3 5 15 95 395 1395 1395 1395 190 191 RBAAAA QQHAAA AAAAxx +1825 5165 1 1 5 5 25 825 1825 1825 1825 50 51 FSAAAA RQHAAA HHHHxx +7606 5166 0 2 6 6 6 606 1606 2606 7606 12 13 OGAAAA SQHAAA OOOOxx +9390 5167 0 2 0 10 90 390 1390 4390 9390 180 181 EXAAAA TQHAAA VVVVxx +2376 5168 0 0 6 16 76 376 376 2376 2376 152 153 KNAAAA UQHAAA AAAAxx +2377 5169 1 1 7 17 77 377 377 2377 2377 154 155 LNAAAA VQHAAA HHHHxx +5346 5170 0 2 6 6 46 346 1346 346 5346 92 93 QXAAAA WQHAAA OOOOxx +4140 5171 0 0 0 0 40 140 140 4140 4140 80 81 GDAAAA XQHAAA VVVVxx +6032 5172 0 0 2 12 32 32 32 1032 6032 64 65 AYAAAA YQHAAA AAAAxx +9453 5173 1 1 3 13 53 453 1453 4453 9453 106 107 PZAAAA ZQHAAA HHHHxx +9297 5174 1 1 7 17 97 297 1297 4297 9297 194 195 PTAAAA ARHAAA OOOOxx +6455 5175 1 3 5 15 55 455 455 1455 6455 110 111 HOAAAA BRHAAA VVVVxx +4458 5176 0 2 8 18 58 458 458 4458 4458 116 117 MPAAAA CRHAAA AAAAxx +9516 5177 0 0 6 16 16 516 1516 4516 9516 32 33 ACAAAA DRHAAA HHHHxx +6211 5178 1 3 1 11 11 211 211 1211 6211 22 23 XEAAAA ERHAAA OOOOxx +526 5179 0 2 6 6 26 526 526 526 526 52 53 GUAAAA FRHAAA VVVVxx +3570 5180 0 2 0 10 70 570 1570 3570 3570 140 141 IHAAAA GRHAAA AAAAxx +4885 5181 1 1 5 5 85 885 885 4885 4885 170 171 XFAAAA HRHAAA HHHHxx +6390 5182 0 2 0 10 90 390 390 1390 6390 180 181 ULAAAA IRHAAA OOOOxx +1606 5183 0 2 6 6 6 606 1606 1606 1606 12 13 UJAAAA JRHAAA VVVVxx +7850 5184 0 2 0 10 50 850 1850 2850 7850 100 101 YPAAAA KRHAAA AAAAxx +3315 5185 1 3 5 15 15 315 1315 3315 3315 30 31 NXAAAA LRHAAA HHHHxx +8322 5186 0 2 2 2 22 322 322 3322 8322 44 45 CIAAAA MRHAAA OOOOxx +3703 5187 1 3 3 3 3 703 1703 3703 3703 6 7 LMAAAA NRHAAA VVVVxx +9489 5188 1 1 9 9 89 489 1489 4489 9489 178 179 ZAAAAA ORHAAA AAAAxx +6104 5189 0 0 4 4 4 104 104 1104 6104 8 9 UAAAAA PRHAAA HHHHxx +3067 5190 1 3 7 7 67 67 1067 3067 3067 134 135 ZNAAAA QRHAAA OOOOxx +2521 5191 1 1 1 1 21 521 521 2521 2521 42 43 ZSAAAA RRHAAA VVVVxx +2581 5192 1 1 1 1 81 581 581 2581 2581 162 163 HVAAAA SRHAAA AAAAxx +595 5193 1 3 5 15 95 595 595 595 595 190 191 XWAAAA TRHAAA HHHHxx +8291 5194 1 3 1 11 91 291 291 3291 8291 182 183 XGAAAA URHAAA OOOOxx +1727 5195 1 3 7 7 27 727 1727 1727 1727 54 55 LOAAAA VRHAAA VVVVxx +6847 5196 1 3 7 7 47 847 847 1847 6847 94 95 JDAAAA WRHAAA AAAAxx +7494 5197 0 2 4 14 94 494 1494 2494 7494 188 189 GCAAAA XRHAAA HHHHxx +7093 5198 1 1 3 13 93 93 1093 2093 7093 186 187 VMAAAA YRHAAA OOOOxx +7357 5199 1 1 7 17 57 357 1357 2357 7357 114 115 ZWAAAA ZRHAAA VVVVxx +620 5200 0 0 0 0 20 620 620 620 620 40 41 WXAAAA ASHAAA AAAAxx +2460 5201 0 0 0 0 60 460 460 2460 2460 120 121 QQAAAA BSHAAA HHHHxx +1598 5202 0 2 8 18 98 598 1598 1598 1598 196 197 MJAAAA CSHAAA OOOOxx +4112 5203 0 0 2 12 12 112 112 4112 4112 24 25 ECAAAA DSHAAA VVVVxx +2956 5204 0 0 6 16 56 956 956 2956 2956 112 113 SJAAAA ESHAAA AAAAxx +3193 5205 1 1 3 13 93 193 1193 3193 3193 186 187 VSAAAA FSHAAA HHHHxx +6356 5206 0 0 6 16 56 356 356 1356 6356 112 113 MKAAAA GSHAAA OOOOxx +730 5207 0 2 0 10 30 730 730 730 730 60 61 CCAAAA HSHAAA VVVVxx +8826 5208 0 2 6 6 26 826 826 3826 8826 52 53 MBAAAA ISHAAA AAAAxx +9036 5209 0 0 6 16 36 36 1036 4036 9036 72 73 OJAAAA JSHAAA HHHHxx +2085 5210 1 1 5 5 85 85 85 2085 2085 170 171 FCAAAA KSHAAA OOOOxx +9007 5211 1 3 7 7 7 7 1007 4007 9007 14 15 LIAAAA LSHAAA VVVVxx +6047 5212 1 3 7 7 47 47 47 1047 6047 94 95 PYAAAA MSHAAA AAAAxx +3953 5213 1 1 3 13 53 953 1953 3953 3953 106 107 BWAAAA NSHAAA HHHHxx +1214 5214 0 2 4 14 14 214 1214 1214 1214 28 29 SUAAAA OSHAAA OOOOxx +4814 5215 0 2 4 14 14 814 814 4814 4814 28 29 EDAAAA PSHAAA VVVVxx +5738 5216 0 2 8 18 38 738 1738 738 5738 76 77 SMAAAA QSHAAA AAAAxx +7176 5217 0 0 6 16 76 176 1176 2176 7176 152 153 AQAAAA RSHAAA HHHHxx +3609 5218 1 1 9 9 9 609 1609 3609 3609 18 19 VIAAAA SSHAAA OOOOxx +592 5219 0 0 2 12 92 592 592 592 592 184 185 UWAAAA TSHAAA VVVVxx +9391 5220 1 3 1 11 91 391 1391 4391 9391 182 183 FXAAAA USHAAA AAAAxx +5345 5221 1 1 5 5 45 345 1345 345 5345 90 91 PXAAAA VSHAAA HHHHxx +1171 5222 1 3 1 11 71 171 1171 1171 1171 142 143 BTAAAA WSHAAA OOOOxx +7238 5223 0 2 8 18 38 238 1238 2238 7238 76 77 KSAAAA XSHAAA VVVVxx +7561 5224 1 1 1 1 61 561 1561 2561 7561 122 123 VEAAAA YSHAAA AAAAxx +5876 5225 0 0 6 16 76 876 1876 876 5876 152 153 ASAAAA ZSHAAA HHHHxx +6611 5226 1 3 1 11 11 611 611 1611 6611 22 23 HUAAAA ATHAAA OOOOxx +7300 5227 0 0 0 0 0 300 1300 2300 7300 0 1 UUAAAA BTHAAA VVVVxx +1506 5228 0 2 6 6 6 506 1506 1506 1506 12 13 YFAAAA CTHAAA AAAAxx +1153 5229 1 1 3 13 53 153 1153 1153 1153 106 107 JSAAAA DTHAAA HHHHxx +3831 5230 1 3 1 11 31 831 1831 3831 3831 62 63 JRAAAA ETHAAA OOOOxx +9255 5231 1 3 5 15 55 255 1255 4255 9255 110 111 ZRAAAA FTHAAA VVVVxx +1841 5232 1 1 1 1 41 841 1841 1841 1841 82 83 VSAAAA GTHAAA AAAAxx +5075 5233 1 3 5 15 75 75 1075 75 5075 150 151 FNAAAA HTHAAA HHHHxx +101 5234 1 1 1 1 1 101 101 101 101 2 3 XDAAAA ITHAAA OOOOxx +2627 5235 1 3 7 7 27 627 627 2627 2627 54 55 BXAAAA JTHAAA VVVVxx +7078 5236 0 2 8 18 78 78 1078 2078 7078 156 157 GMAAAA KTHAAA AAAAxx +2850 5237 0 2 0 10 50 850 850 2850 2850 100 101 QFAAAA LTHAAA HHHHxx +8703 5238 1 3 3 3 3 703 703 3703 8703 6 7 TWAAAA MTHAAA OOOOxx +4101 5239 1 1 1 1 1 101 101 4101 4101 2 3 TBAAAA NTHAAA VVVVxx +318 5240 0 2 8 18 18 318 318 318 318 36 37 GMAAAA OTHAAA AAAAxx +6452 5241 0 0 2 12 52 452 452 1452 6452 104 105 EOAAAA PTHAAA HHHHxx +5558 5242 0 2 8 18 58 558 1558 558 5558 116 117 UFAAAA QTHAAA OOOOxx +3127 5243 1 3 7 7 27 127 1127 3127 3127 54 55 HQAAAA RTHAAA VVVVxx +535 5244 1 3 5 15 35 535 535 535 535 70 71 PUAAAA STHAAA AAAAxx +270 5245 0 2 0 10 70 270 270 270 270 140 141 KKAAAA TTHAAA HHHHxx +4038 5246 0 2 8 18 38 38 38 4038 4038 76 77 IZAAAA UTHAAA OOOOxx +3404 5247 0 0 4 4 4 404 1404 3404 3404 8 9 YAAAAA VTHAAA VVVVxx +2374 5248 0 2 4 14 74 374 374 2374 2374 148 149 INAAAA WTHAAA AAAAxx +6446 5249 0 2 6 6 46 446 446 1446 6446 92 93 YNAAAA XTHAAA HHHHxx +7758 5250 0 2 8 18 58 758 1758 2758 7758 116 117 KMAAAA YTHAAA OOOOxx +356 5251 0 0 6 16 56 356 356 356 356 112 113 SNAAAA ZTHAAA VVVVxx +9197 5252 1 1 7 17 97 197 1197 4197 9197 194 195 TPAAAA AUHAAA AAAAxx +9765 5253 1 1 5 5 65 765 1765 4765 9765 130 131 PLAAAA BUHAAA HHHHxx +4974 5254 0 2 4 14 74 974 974 4974 4974 148 149 IJAAAA CUHAAA OOOOxx +442 5255 0 2 2 2 42 442 442 442 442 84 85 ARAAAA DUHAAA VVVVxx +4349 5256 1 1 9 9 49 349 349 4349 4349 98 99 HLAAAA EUHAAA AAAAxx +6119 5257 1 3 9 19 19 119 119 1119 6119 38 39 JBAAAA FUHAAA HHHHxx +7574 5258 0 2 4 14 74 574 1574 2574 7574 148 149 IFAAAA GUHAAA OOOOxx +4445 5259 1 1 5 5 45 445 445 4445 4445 90 91 ZOAAAA HUHAAA VVVVxx +940 5260 0 0 0 0 40 940 940 940 940 80 81 EKAAAA IUHAAA AAAAxx +1875 5261 1 3 5 15 75 875 1875 1875 1875 150 151 DUAAAA JUHAAA HHHHxx +5951 5262 1 3 1 11 51 951 1951 951 5951 102 103 XUAAAA KUHAAA OOOOxx +9132 5263 0 0 2 12 32 132 1132 4132 9132 64 65 GNAAAA LUHAAA VVVVxx +6913 5264 1 1 3 13 13 913 913 1913 6913 26 27 XFAAAA MUHAAA AAAAxx +3308 5265 0 0 8 8 8 308 1308 3308 3308 16 17 GXAAAA NUHAAA HHHHxx +7553 5266 1 1 3 13 53 553 1553 2553 7553 106 107 NEAAAA OUHAAA OOOOxx +2138 5267 0 2 8 18 38 138 138 2138 2138 76 77 GEAAAA PUHAAA VVVVxx +6252 5268 0 0 2 12 52 252 252 1252 6252 104 105 MGAAAA QUHAAA AAAAxx +2171 5269 1 3 1 11 71 171 171 2171 2171 142 143 NFAAAA RUHAAA HHHHxx +4159 5270 1 3 9 19 59 159 159 4159 4159 118 119 ZDAAAA SUHAAA OOOOxx +2401 5271 1 1 1 1 1 401 401 2401 2401 2 3 JOAAAA TUHAAA VVVVxx +6553 5272 1 1 3 13 53 553 553 1553 6553 106 107 BSAAAA UUHAAA AAAAxx +5217 5273 1 1 7 17 17 217 1217 217 5217 34 35 RSAAAA VUHAAA HHHHxx +1405 5274 1 1 5 5 5 405 1405 1405 1405 10 11 BCAAAA WUHAAA OOOOxx +1494 5275 0 2 4 14 94 494 1494 1494 1494 188 189 MFAAAA XUHAAA VVVVxx +5553 5276 1 1 3 13 53 553 1553 553 5553 106 107 PFAAAA YUHAAA AAAAxx +8296 5277 0 0 6 16 96 296 296 3296 8296 192 193 CHAAAA ZUHAAA HHHHxx +6565 5278 1 1 5 5 65 565 565 1565 6565 130 131 NSAAAA AVHAAA OOOOxx +817 5279 1 1 7 17 17 817 817 817 817 34 35 LFAAAA BVHAAA VVVVxx +6947 5280 1 3 7 7 47 947 947 1947 6947 94 95 FHAAAA CVHAAA AAAAxx +4184 5281 0 0 4 4 84 184 184 4184 4184 168 169 YEAAAA DVHAAA HHHHxx +6577 5282 1 1 7 17 77 577 577 1577 6577 154 155 ZSAAAA EVHAAA OOOOxx +6424 5283 0 0 4 4 24 424 424 1424 6424 48 49 CNAAAA FVHAAA VVVVxx +2482 5284 0 2 2 2 82 482 482 2482 2482 164 165 MRAAAA GVHAAA AAAAxx +6874 5285 0 2 4 14 74 874 874 1874 6874 148 149 KEAAAA HVHAAA HHHHxx +7601 5286 1 1 1 1 1 601 1601 2601 7601 2 3 JGAAAA IVHAAA OOOOxx +4552 5287 0 0 2 12 52 552 552 4552 4552 104 105 CTAAAA JVHAAA VVVVxx +8406 5288 0 2 6 6 6 406 406 3406 8406 12 13 ILAAAA KVHAAA AAAAxx +2924 5289 0 0 4 4 24 924 924 2924 2924 48 49 MIAAAA LVHAAA HHHHxx +8255 5290 1 3 5 15 55 255 255 3255 8255 110 111 NFAAAA MVHAAA OOOOxx +4920 5291 0 0 0 0 20 920 920 4920 4920 40 41 GHAAAA NVHAAA VVVVxx +228 5292 0 0 8 8 28 228 228 228 228 56 57 UIAAAA OVHAAA AAAAxx +9431 5293 1 3 1 11 31 431 1431 4431 9431 62 63 TYAAAA PVHAAA HHHHxx +4021 5294 1 1 1 1 21 21 21 4021 4021 42 43 RYAAAA QVHAAA OOOOxx +2966 5295 0 2 6 6 66 966 966 2966 2966 132 133 CKAAAA RVHAAA VVVVxx +2862 5296 0 2 2 2 62 862 862 2862 2862 124 125 CGAAAA SVHAAA AAAAxx +4303 5297 1 3 3 3 3 303 303 4303 4303 6 7 NJAAAA TVHAAA HHHHxx +9643 5298 1 3 3 3 43 643 1643 4643 9643 86 87 XGAAAA UVHAAA OOOOxx +3008 5299 0 0 8 8 8 8 1008 3008 3008 16 17 SLAAAA VVHAAA VVVVxx +7476 5300 0 0 6 16 76 476 1476 2476 7476 152 153 OBAAAA WVHAAA AAAAxx +3686 5301 0 2 6 6 86 686 1686 3686 3686 172 173 ULAAAA XVHAAA HHHHxx +9051 5302 1 3 1 11 51 51 1051 4051 9051 102 103 DKAAAA YVHAAA OOOOxx +6592 5303 0 0 2 12 92 592 592 1592 6592 184 185 OTAAAA ZVHAAA VVVVxx +924 5304 0 0 4 4 24 924 924 924 924 48 49 OJAAAA AWHAAA AAAAxx +4406 5305 0 2 6 6 6 406 406 4406 4406 12 13 MNAAAA BWHAAA HHHHxx +5233 5306 1 1 3 13 33 233 1233 233 5233 66 67 HTAAAA CWHAAA OOOOxx +8881 5307 1 1 1 1 81 881 881 3881 8881 162 163 PDAAAA DWHAAA VVVVxx +2212 5308 0 0 2 12 12 212 212 2212 2212 24 25 CHAAAA EWHAAA AAAAxx +5804 5309 0 0 4 4 4 804 1804 804 5804 8 9 GPAAAA FWHAAA HHHHxx +2990 5310 0 2 0 10 90 990 990 2990 2990 180 181 ALAAAA GWHAAA OOOOxx +4069 5311 1 1 9 9 69 69 69 4069 4069 138 139 NAAAAA HWHAAA VVVVxx +5380 5312 0 0 0 0 80 380 1380 380 5380 160 161 YYAAAA IWHAAA AAAAxx +5016 5313 0 0 6 16 16 16 1016 16 5016 32 33 YKAAAA JWHAAA HHHHxx +5056 5314 0 0 6 16 56 56 1056 56 5056 112 113 MMAAAA KWHAAA OOOOxx +3732 5315 0 0 2 12 32 732 1732 3732 3732 64 65 ONAAAA LWHAAA VVVVxx +5527 5316 1 3 7 7 27 527 1527 527 5527 54 55 PEAAAA MWHAAA AAAAxx +1151 5317 1 3 1 11 51 151 1151 1151 1151 102 103 HSAAAA NWHAAA HHHHxx +7900 5318 0 0 0 0 0 900 1900 2900 7900 0 1 WRAAAA OWHAAA OOOOxx +1660 5319 0 0 0 0 60 660 1660 1660 1660 120 121 WLAAAA PWHAAA VVVVxx +8064 5320 0 0 4 4 64 64 64 3064 8064 128 129 EYAAAA QWHAAA AAAAxx +8240 5321 0 0 0 0 40 240 240 3240 8240 80 81 YEAAAA RWHAAA HHHHxx +413 5322 1 1 3 13 13 413 413 413 413 26 27 XPAAAA SWHAAA OOOOxx +8311 5323 1 3 1 11 11 311 311 3311 8311 22 23 RHAAAA TWHAAA VVVVxx +1065 5324 1 1 5 5 65 65 1065 1065 1065 130 131 ZOAAAA UWHAAA AAAAxx +2741 5325 1 1 1 1 41 741 741 2741 2741 82 83 LBAAAA VWHAAA HHHHxx +5306 5326 0 2 6 6 6 306 1306 306 5306 12 13 CWAAAA WWHAAA OOOOxx +5464 5327 0 0 4 4 64 464 1464 464 5464 128 129 ECAAAA XWHAAA VVVVxx +4237 5328 1 1 7 17 37 237 237 4237 4237 74 75 ZGAAAA YWHAAA AAAAxx +3822 5329 0 2 2 2 22 822 1822 3822 3822 44 45 ARAAAA ZWHAAA HHHHxx +2548 5330 0 0 8 8 48 548 548 2548 2548 96 97 AUAAAA AXHAAA OOOOxx +2688 5331 0 0 8 8 88 688 688 2688 2688 176 177 KZAAAA BXHAAA VVVVxx +8061 5332 1 1 1 1 61 61 61 3061 8061 122 123 BYAAAA CXHAAA AAAAxx +9340 5333 0 0 0 0 40 340 1340 4340 9340 80 81 GVAAAA DXHAAA HHHHxx +4031 5334 1 3 1 11 31 31 31 4031 4031 62 63 BZAAAA EXHAAA OOOOxx +2635 5335 1 3 5 15 35 635 635 2635 2635 70 71 JXAAAA FXHAAA VVVVxx +809 5336 1 1 9 9 9 809 809 809 809 18 19 DFAAAA GXHAAA AAAAxx +3209 5337 1 1 9 9 9 209 1209 3209 3209 18 19 LTAAAA HXHAAA HHHHxx +3825 5338 1 1 5 5 25 825 1825 3825 3825 50 51 DRAAAA IXHAAA OOOOxx +1448 5339 0 0 8 8 48 448 1448 1448 1448 96 97 SDAAAA JXHAAA VVVVxx +9077 5340 1 1 7 17 77 77 1077 4077 9077 154 155 DLAAAA KXHAAA AAAAxx +3730 5341 0 2 0 10 30 730 1730 3730 3730 60 61 MNAAAA LXHAAA HHHHxx +9596 5342 0 0 6 16 96 596 1596 4596 9596 192 193 CFAAAA MXHAAA OOOOxx +3563 5343 1 3 3 3 63 563 1563 3563 3563 126 127 BHAAAA NXHAAA VVVVxx +4116 5344 0 0 6 16 16 116 116 4116 4116 32 33 ICAAAA OXHAAA AAAAxx +4825 5345 1 1 5 5 25 825 825 4825 4825 50 51 PDAAAA PXHAAA HHHHxx +8376 5346 0 0 6 16 76 376 376 3376 8376 152 153 EKAAAA QXHAAA OOOOxx +3917 5347 1 1 7 17 17 917 1917 3917 3917 34 35 RUAAAA RXHAAA VVVVxx +4407 5348 1 3 7 7 7 407 407 4407 4407 14 15 NNAAAA SXHAAA AAAAxx +8202 5349 0 2 2 2 2 202 202 3202 8202 4 5 MDAAAA TXHAAA HHHHxx +7675 5350 1 3 5 15 75 675 1675 2675 7675 150 151 FJAAAA UXHAAA OOOOxx +4104 5351 0 0 4 4 4 104 104 4104 4104 8 9 WBAAAA VXHAAA VVVVxx +9225 5352 1 1 5 5 25 225 1225 4225 9225 50 51 VQAAAA WXHAAA AAAAxx +2834 5353 0 2 4 14 34 834 834 2834 2834 68 69 AFAAAA XXHAAA HHHHxx +1227 5354 1 3 7 7 27 227 1227 1227 1227 54 55 FVAAAA YXHAAA OOOOxx +3383 5355 1 3 3 3 83 383 1383 3383 3383 166 167 DAAAAA ZXHAAA VVVVxx +67 5356 1 3 7 7 67 67 67 67 67 134 135 PCAAAA AYHAAA AAAAxx +1751 5357 1 3 1 11 51 751 1751 1751 1751 102 103 JPAAAA BYHAAA HHHHxx +8054 5358 0 2 4 14 54 54 54 3054 8054 108 109 UXAAAA CYHAAA OOOOxx +8571 5359 1 3 1 11 71 571 571 3571 8571 142 143 RRAAAA DYHAAA VVVVxx +2466 5360 0 2 6 6 66 466 466 2466 2466 132 133 WQAAAA EYHAAA AAAAxx +9405 5361 1 1 5 5 5 405 1405 4405 9405 10 11 TXAAAA FYHAAA HHHHxx +6883 5362 1 3 3 3 83 883 883 1883 6883 166 167 TEAAAA GYHAAA OOOOxx +4301 5363 1 1 1 1 1 301 301 4301 4301 2 3 LJAAAA HYHAAA VVVVxx +3705 5364 1 1 5 5 5 705 1705 3705 3705 10 11 NMAAAA IYHAAA AAAAxx +5420 5365 0 0 0 0 20 420 1420 420 5420 40 41 MAAAAA JYHAAA HHHHxx +3692 5366 0 0 2 12 92 692 1692 3692 3692 184 185 AMAAAA KYHAAA OOOOxx +6851 5367 1 3 1 11 51 851 851 1851 6851 102 103 NDAAAA LYHAAA VVVVxx +9363 5368 1 3 3 3 63 363 1363 4363 9363 126 127 DWAAAA MYHAAA AAAAxx +2269 5369 1 1 9 9 69 269 269 2269 2269 138 139 HJAAAA NYHAAA HHHHxx +4918 5370 0 2 8 18 18 918 918 4918 4918 36 37 EHAAAA OYHAAA OOOOxx +4297 5371 1 1 7 17 97 297 297 4297 4297 194 195 HJAAAA PYHAAA VVVVxx +1836 5372 0 0 6 16 36 836 1836 1836 1836 72 73 QSAAAA QYHAAA AAAAxx +237 5373 1 1 7 17 37 237 237 237 237 74 75 DJAAAA RYHAAA HHHHxx +6131 5374 1 3 1 11 31 131 131 1131 6131 62 63 VBAAAA SYHAAA OOOOxx +3174 5375 0 2 4 14 74 174 1174 3174 3174 148 149 CSAAAA TYHAAA VVVVxx +9987 5376 1 3 7 7 87 987 1987 4987 9987 174 175 DUAAAA UYHAAA AAAAxx +3630 5377 0 2 0 10 30 630 1630 3630 3630 60 61 QJAAAA VYHAAA HHHHxx +2899 5378 1 3 9 19 99 899 899 2899 2899 198 199 NHAAAA WYHAAA OOOOxx +4079 5379 1 3 9 19 79 79 79 4079 4079 158 159 XAAAAA XYHAAA VVVVxx +5049 5380 1 1 9 9 49 49 1049 49 5049 98 99 FMAAAA YYHAAA AAAAxx +2963 5381 1 3 3 3 63 963 963 2963 2963 126 127 ZJAAAA ZYHAAA HHHHxx +3962 5382 0 2 2 2 62 962 1962 3962 3962 124 125 KWAAAA AZHAAA OOOOxx +7921 5383 1 1 1 1 21 921 1921 2921 7921 42 43 RSAAAA BZHAAA VVVVxx +3967 5384 1 3 7 7 67 967 1967 3967 3967 134 135 PWAAAA CZHAAA AAAAxx +2752 5385 0 0 2 12 52 752 752 2752 2752 104 105 WBAAAA DZHAAA HHHHxx +7944 5386 0 0 4 4 44 944 1944 2944 7944 88 89 OTAAAA EZHAAA OOOOxx +2205 5387 1 1 5 5 5 205 205 2205 2205 10 11 VGAAAA FZHAAA VVVVxx +5035 5388 1 3 5 15 35 35 1035 35 5035 70 71 RLAAAA GZHAAA AAAAxx +1425 5389 1 1 5 5 25 425 1425 1425 1425 50 51 VCAAAA HZHAAA HHHHxx +832 5390 0 0 2 12 32 832 832 832 832 64 65 AGAAAA IZHAAA OOOOxx +1447 5391 1 3 7 7 47 447 1447 1447 1447 94 95 RDAAAA JZHAAA VVVVxx +6108 5392 0 0 8 8 8 108 108 1108 6108 16 17 YAAAAA KZHAAA AAAAxx +4936 5393 0 0 6 16 36 936 936 4936 4936 72 73 WHAAAA LZHAAA HHHHxx +7704 5394 0 0 4 4 4 704 1704 2704 7704 8 9 IKAAAA MZHAAA OOOOxx +142 5395 0 2 2 2 42 142 142 142 142 84 85 MFAAAA NZHAAA VVVVxx +4272 5396 0 0 2 12 72 272 272 4272 4272 144 145 IIAAAA OZHAAA AAAAxx +7667 5397 1 3 7 7 67 667 1667 2667 7667 134 135 XIAAAA PZHAAA HHHHxx +366 5398 0 2 6 6 66 366 366 366 366 132 133 COAAAA QZHAAA OOOOxx +8866 5399 0 2 6 6 66 866 866 3866 8866 132 133 ADAAAA RZHAAA VVVVxx +7712 5400 0 0 2 12 12 712 1712 2712 7712 24 25 QKAAAA SZHAAA AAAAxx +3880 5401 0 0 0 0 80 880 1880 3880 3880 160 161 GTAAAA TZHAAA HHHHxx +4631 5402 1 3 1 11 31 631 631 4631 4631 62 63 DWAAAA UZHAAA OOOOxx +2789 5403 1 1 9 9 89 789 789 2789 2789 178 179 HDAAAA VZHAAA VVVVxx +7720 5404 0 0 0 0 20 720 1720 2720 7720 40 41 YKAAAA WZHAAA AAAAxx +7618 5405 0 2 8 18 18 618 1618 2618 7618 36 37 AHAAAA XZHAAA HHHHxx +4990 5406 0 2 0 10 90 990 990 4990 4990 180 181 YJAAAA YZHAAA OOOOxx +7918 5407 0 2 8 18 18 918 1918 2918 7918 36 37 OSAAAA ZZHAAA VVVVxx +5067 5408 1 3 7 7 67 67 1067 67 5067 134 135 XMAAAA AAIAAA AAAAxx +6370 5409 0 2 0 10 70 370 370 1370 6370 140 141 ALAAAA BAIAAA HHHHxx +2268 5410 0 0 8 8 68 268 268 2268 2268 136 137 GJAAAA CAIAAA OOOOxx +1949 5411 1 1 9 9 49 949 1949 1949 1949 98 99 ZWAAAA DAIAAA VVVVxx +5503 5412 1 3 3 3 3 503 1503 503 5503 6 7 RDAAAA EAIAAA AAAAxx +9951 5413 1 3 1 11 51 951 1951 4951 9951 102 103 TSAAAA FAIAAA HHHHxx +6823 5414 1 3 3 3 23 823 823 1823 6823 46 47 LCAAAA GAIAAA OOOOxx +6287 5415 1 3 7 7 87 287 287 1287 6287 174 175 VHAAAA HAIAAA VVVVxx +6016 5416 0 0 6 16 16 16 16 1016 6016 32 33 KXAAAA IAIAAA AAAAxx +1977 5417 1 1 7 17 77 977 1977 1977 1977 154 155 BYAAAA JAIAAA HHHHxx +8579 5418 1 3 9 19 79 579 579 3579 8579 158 159 ZRAAAA KAIAAA OOOOxx +6204 5419 0 0 4 4 4 204 204 1204 6204 8 9 QEAAAA LAIAAA VVVVxx +9764 5420 0 0 4 4 64 764 1764 4764 9764 128 129 OLAAAA MAIAAA AAAAxx +2005 5421 1 1 5 5 5 5 5 2005 2005 10 11 DZAAAA NAIAAA HHHHxx +1648 5422 0 0 8 8 48 648 1648 1648 1648 96 97 KLAAAA OAIAAA OOOOxx +2457 5423 1 1 7 17 57 457 457 2457 2457 114 115 NQAAAA PAIAAA VVVVxx +2698 5424 0 2 8 18 98 698 698 2698 2698 196 197 UZAAAA QAIAAA AAAAxx +7730 5425 0 2 0 10 30 730 1730 2730 7730 60 61 ILAAAA RAIAAA HHHHxx +7287 5426 1 3 7 7 87 287 1287 2287 7287 174 175 HUAAAA SAIAAA OOOOxx +2937 5427 1 1 7 17 37 937 937 2937 2937 74 75 ZIAAAA TAIAAA VVVVxx +6824 5428 0 0 4 4 24 824 824 1824 6824 48 49 MCAAAA UAIAAA AAAAxx +9256 5429 0 0 6 16 56 256 1256 4256 9256 112 113 ASAAAA VAIAAA HHHHxx +4810 5430 0 2 0 10 10 810 810 4810 4810 20 21 ADAAAA WAIAAA OOOOxx +3869 5431 1 1 9 9 69 869 1869 3869 3869 138 139 VSAAAA XAIAAA VVVVxx +1993 5432 1 1 3 13 93 993 1993 1993 1993 186 187 RYAAAA YAIAAA AAAAxx +6048 5433 0 0 8 8 48 48 48 1048 6048 96 97 QYAAAA ZAIAAA HHHHxx +6922 5434 0 2 2 2 22 922 922 1922 6922 44 45 GGAAAA ABIAAA OOOOxx +8 5435 0 0 8 8 8 8 8 8 8 16 17 IAAAAA BBIAAA VVVVxx +6706 5436 0 2 6 6 6 706 706 1706 6706 12 13 YXAAAA CBIAAA AAAAxx +9159 5437 1 3 9 19 59 159 1159 4159 9159 118 119 HOAAAA DBIAAA HHHHxx +7020 5438 0 0 0 0 20 20 1020 2020 7020 40 41 AKAAAA EBIAAA OOOOxx +767 5439 1 3 7 7 67 767 767 767 767 134 135 NDAAAA FBIAAA VVVVxx +8602 5440 0 2 2 2 2 602 602 3602 8602 4 5 WSAAAA GBIAAA AAAAxx +4442 5441 0 2 2 2 42 442 442 4442 4442 84 85 WOAAAA HBIAAA HHHHxx +2040 5442 0 0 0 0 40 40 40 2040 2040 80 81 MAAAAA IBIAAA OOOOxx +5493 5443 1 1 3 13 93 493 1493 493 5493 186 187 HDAAAA JBIAAA VVVVxx +275 5444 1 3 5 15 75 275 275 275 275 150 151 PKAAAA KBIAAA AAAAxx +8876 5445 0 0 6 16 76 876 876 3876 8876 152 153 KDAAAA LBIAAA HHHHxx +7381 5446 1 1 1 1 81 381 1381 2381 7381 162 163 XXAAAA MBIAAA OOOOxx +1827 5447 1 3 7 7 27 827 1827 1827 1827 54 55 HSAAAA NBIAAA VVVVxx +3537 5448 1 1 7 17 37 537 1537 3537 3537 74 75 BGAAAA OBIAAA AAAAxx +6978 5449 0 2 8 18 78 978 978 1978 6978 156 157 KIAAAA PBIAAA HHHHxx +6160 5450 0 0 0 0 60 160 160 1160 6160 120 121 YCAAAA QBIAAA OOOOxx +9219 5451 1 3 9 19 19 219 1219 4219 9219 38 39 PQAAAA RBIAAA VVVVxx +5034 5452 0 2 4 14 34 34 1034 34 5034 68 69 QLAAAA SBIAAA AAAAxx +8463 5453 1 3 3 3 63 463 463 3463 8463 126 127 NNAAAA TBIAAA HHHHxx +2038 5454 0 2 8 18 38 38 38 2038 2038 76 77 KAAAAA UBIAAA OOOOxx +9562 5455 0 2 2 2 62 562 1562 4562 9562 124 125 UDAAAA VBIAAA VVVVxx +2687 5456 1 3 7 7 87 687 687 2687 2687 174 175 JZAAAA WBIAAA AAAAxx +5092 5457 0 0 2 12 92 92 1092 92 5092 184 185 WNAAAA XBIAAA HHHHxx +539 5458 1 3 9 19 39 539 539 539 539 78 79 TUAAAA YBIAAA OOOOxx +2139 5459 1 3 9 19 39 139 139 2139 2139 78 79 HEAAAA ZBIAAA VVVVxx +9221 5460 1 1 1 1 21 221 1221 4221 9221 42 43 RQAAAA ACIAAA AAAAxx +965 5461 1 1 5 5 65 965 965 965 965 130 131 DLAAAA BCIAAA HHHHxx +6051 5462 1 3 1 11 51 51 51 1051 6051 102 103 TYAAAA CCIAAA OOOOxx +5822 5463 0 2 2 2 22 822 1822 822 5822 44 45 YPAAAA DCIAAA VVVVxx +6397 5464 1 1 7 17 97 397 397 1397 6397 194 195 BMAAAA ECIAAA AAAAxx +2375 5465 1 3 5 15 75 375 375 2375 2375 150 151 JNAAAA FCIAAA HHHHxx +9415 5466 1 3 5 15 15 415 1415 4415 9415 30 31 DYAAAA GCIAAA OOOOxx +6552 5467 0 0 2 12 52 552 552 1552 6552 104 105 ASAAAA HCIAAA VVVVxx +2248 5468 0 0 8 8 48 248 248 2248 2248 96 97 MIAAAA ICIAAA AAAAxx +2611 5469 1 3 1 11 11 611 611 2611 2611 22 23 LWAAAA JCIAAA HHHHxx +9609 5470 1 1 9 9 9 609 1609 4609 9609 18 19 PFAAAA KCIAAA OOOOxx +2132 5471 0 0 2 12 32 132 132 2132 2132 64 65 AEAAAA LCIAAA VVVVxx +8452 5472 0 0 2 12 52 452 452 3452 8452 104 105 CNAAAA MCIAAA AAAAxx +9407 5473 1 3 7 7 7 407 1407 4407 9407 14 15 VXAAAA NCIAAA HHHHxx +2814 5474 0 2 4 14 14 814 814 2814 2814 28 29 GEAAAA OCIAAA OOOOxx +1889 5475 1 1 9 9 89 889 1889 1889 1889 178 179 RUAAAA PCIAAA VVVVxx +7489 5476 1 1 9 9 89 489 1489 2489 7489 178 179 BCAAAA QCIAAA AAAAxx +2255 5477 1 3 5 15 55 255 255 2255 2255 110 111 TIAAAA RCIAAA HHHHxx +3380 5478 0 0 0 0 80 380 1380 3380 3380 160 161 AAAAAA SCIAAA OOOOxx +1167 5479 1 3 7 7 67 167 1167 1167 1167 134 135 XSAAAA TCIAAA VVVVxx +5369 5480 1 1 9 9 69 369 1369 369 5369 138 139 NYAAAA UCIAAA AAAAxx +2378 5481 0 2 8 18 78 378 378 2378 2378 156 157 MNAAAA VCIAAA HHHHxx +8315 5482 1 3 5 15 15 315 315 3315 8315 30 31 VHAAAA WCIAAA OOOOxx +2934 5483 0 2 4 14 34 934 934 2934 2934 68 69 WIAAAA XCIAAA VVVVxx +7924 5484 0 0 4 4 24 924 1924 2924 7924 48 49 USAAAA YCIAAA AAAAxx +2867 5485 1 3 7 7 67 867 867 2867 2867 134 135 HGAAAA ZCIAAA HHHHxx +9141 5486 1 1 1 1 41 141 1141 4141 9141 82 83 PNAAAA ADIAAA OOOOxx +3613 5487 1 1 3 13 13 613 1613 3613 3613 26 27 ZIAAAA BDIAAA VVVVxx +2461 5488 1 1 1 1 61 461 461 2461 2461 122 123 RQAAAA CDIAAA AAAAxx +4567 5489 1 3 7 7 67 567 567 4567 4567 134 135 RTAAAA DDIAAA HHHHxx +2906 5490 0 2 6 6 6 906 906 2906 2906 12 13 UHAAAA EDIAAA OOOOxx +4848 5491 0 0 8 8 48 848 848 4848 4848 96 97 MEAAAA FDIAAA VVVVxx +6614 5492 0 2 4 14 14 614 614 1614 6614 28 29 KUAAAA GDIAAA AAAAxx +6200 5493 0 0 0 0 0 200 200 1200 6200 0 1 MEAAAA HDIAAA HHHHxx +7895 5494 1 3 5 15 95 895 1895 2895 7895 190 191 RRAAAA IDIAAA OOOOxx +6829 5495 1 1 9 9 29 829 829 1829 6829 58 59 RCAAAA JDIAAA VVVVxx +4087 5496 1 3 7 7 87 87 87 4087 4087 174 175 FBAAAA KDIAAA AAAAxx +8787 5497 1 3 7 7 87 787 787 3787 8787 174 175 ZZAAAA LDIAAA HHHHxx +3322 5498 0 2 2 2 22 322 1322 3322 3322 44 45 UXAAAA MDIAAA OOOOxx +9091 5499 1 3 1 11 91 91 1091 4091 9091 182 183 RLAAAA NDIAAA VVVVxx +5268 5500 0 0 8 8 68 268 1268 268 5268 136 137 QUAAAA ODIAAA AAAAxx +2719 5501 1 3 9 19 19 719 719 2719 2719 38 39 PAAAAA PDIAAA HHHHxx +30 5502 0 2 0 10 30 30 30 30 30 60 61 EBAAAA QDIAAA OOOOxx +1975 5503 1 3 5 15 75 975 1975 1975 1975 150 151 ZXAAAA RDIAAA VVVVxx +2641 5504 1 1 1 1 41 641 641 2641 2641 82 83 PXAAAA SDIAAA AAAAxx +8616 5505 0 0 6 16 16 616 616 3616 8616 32 33 KTAAAA TDIAAA HHHHxx +5980 5506 0 0 0 0 80 980 1980 980 5980 160 161 AWAAAA UDIAAA OOOOxx +5170 5507 0 2 0 10 70 170 1170 170 5170 140 141 WQAAAA VDIAAA VVVVxx +1960 5508 0 0 0 0 60 960 1960 1960 1960 120 121 KXAAAA WDIAAA AAAAxx +8141 5509 1 1 1 1 41 141 141 3141 8141 82 83 DBAAAA XDIAAA HHHHxx +6692 5510 0 0 2 12 92 692 692 1692 6692 184 185 KXAAAA YDIAAA OOOOxx +7621 5511 1 1 1 1 21 621 1621 2621 7621 42 43 DHAAAA ZDIAAA VVVVxx +3890 5512 0 2 0 10 90 890 1890 3890 3890 180 181 QTAAAA AEIAAA AAAAxx +4300 5513 0 0 0 0 0 300 300 4300 4300 0 1 KJAAAA BEIAAA HHHHxx +736 5514 0 0 6 16 36 736 736 736 736 72 73 ICAAAA CEIAAA OOOOxx +6626 5515 0 2 6 6 26 626 626 1626 6626 52 53 WUAAAA DEIAAA VVVVxx +1800 5516 0 0 0 0 0 800 1800 1800 1800 0 1 GRAAAA EEIAAA AAAAxx +3430 5517 0 2 0 10 30 430 1430 3430 3430 60 61 YBAAAA FEIAAA HHHHxx +9519 5518 1 3 9 19 19 519 1519 4519 9519 38 39 DCAAAA GEIAAA OOOOxx +5111 5519 1 3 1 11 11 111 1111 111 5111 22 23 POAAAA HEIAAA VVVVxx +6915 5520 1 3 5 15 15 915 915 1915 6915 30 31 ZFAAAA IEIAAA AAAAxx +9246 5521 0 2 6 6 46 246 1246 4246 9246 92 93 QRAAAA JEIAAA HHHHxx +5141 5522 1 1 1 1 41 141 1141 141 5141 82 83 TPAAAA KEIAAA OOOOxx +5922 5523 0 2 2 2 22 922 1922 922 5922 44 45 UTAAAA LEIAAA VVVVxx +3087 5524 1 3 7 7 87 87 1087 3087 3087 174 175 TOAAAA MEIAAA AAAAxx +1859 5525 1 3 9 19 59 859 1859 1859 1859 118 119 NTAAAA NEIAAA HHHHxx +8482 5526 0 2 2 2 82 482 482 3482 8482 164 165 GOAAAA OEIAAA OOOOxx +8414 5527 0 2 4 14 14 414 414 3414 8414 28 29 QLAAAA PEIAAA VVVVxx +6662 5528 0 2 2 2 62 662 662 1662 6662 124 125 GWAAAA QEIAAA AAAAxx +8614 5529 0 2 4 14 14 614 614 3614 8614 28 29 ITAAAA REIAAA HHHHxx +42 5530 0 2 2 2 42 42 42 42 42 84 85 QBAAAA SEIAAA OOOOxx +7582 5531 0 2 2 2 82 582 1582 2582 7582 164 165 QFAAAA TEIAAA VVVVxx +8183 5532 1 3 3 3 83 183 183 3183 8183 166 167 TCAAAA UEIAAA AAAAxx +1299 5533 1 3 9 19 99 299 1299 1299 1299 198 199 ZXAAAA VEIAAA HHHHxx +7004 5534 0 0 4 4 4 4 1004 2004 7004 8 9 KJAAAA WEIAAA OOOOxx +3298 5535 0 2 8 18 98 298 1298 3298 3298 196 197 WWAAAA XEIAAA VVVVxx +7884 5536 0 0 4 4 84 884 1884 2884 7884 168 169 GRAAAA YEIAAA AAAAxx +4191 5537 1 3 1 11 91 191 191 4191 4191 182 183 FFAAAA ZEIAAA HHHHxx +7346 5538 0 2 6 6 46 346 1346 2346 7346 92 93 OWAAAA AFIAAA OOOOxx +7989 5539 1 1 9 9 89 989 1989 2989 7989 178 179 HVAAAA BFIAAA VVVVxx +5719 5540 1 3 9 19 19 719 1719 719 5719 38 39 ZLAAAA CFIAAA AAAAxx +800 5541 0 0 0 0 0 800 800 800 800 0 1 UEAAAA DFIAAA HHHHxx +6509 5542 1 1 9 9 9 509 509 1509 6509 18 19 JQAAAA EFIAAA OOOOxx +4672 5543 0 0 2 12 72 672 672 4672 4672 144 145 SXAAAA FFIAAA VVVVxx +4434 5544 0 2 4 14 34 434 434 4434 4434 68 69 OOAAAA GFIAAA AAAAxx +8309 5545 1 1 9 9 9 309 309 3309 8309 18 19 PHAAAA HFIAAA HHHHxx +5134 5546 0 2 4 14 34 134 1134 134 5134 68 69 MPAAAA IFIAAA OOOOxx +5153 5547 1 1 3 13 53 153 1153 153 5153 106 107 FQAAAA JFIAAA VVVVxx +1522 5548 0 2 2 2 22 522 1522 1522 1522 44 45 OGAAAA KFIAAA AAAAxx +8629 5549 1 1 9 9 29 629 629 3629 8629 58 59 XTAAAA LFIAAA HHHHxx +4549 5550 1 1 9 9 49 549 549 4549 4549 98 99 ZSAAAA MFIAAA OOOOxx +9506 5551 0 2 6 6 6 506 1506 4506 9506 12 13 QBAAAA NFIAAA VVVVxx +6542 5552 0 2 2 2 42 542 542 1542 6542 84 85 QRAAAA OFIAAA AAAAxx +2579 5553 1 3 9 19 79 579 579 2579 2579 158 159 FVAAAA PFIAAA HHHHxx +4664 5554 0 0 4 4 64 664 664 4664 4664 128 129 KXAAAA QFIAAA OOOOxx +696 5555 0 0 6 16 96 696 696 696 696 192 193 UAAAAA RFIAAA VVVVxx +7950 5556 0 2 0 10 50 950 1950 2950 7950 100 101 UTAAAA SFIAAA AAAAxx +5 5557 1 1 5 5 5 5 5 5 5 10 11 FAAAAA TFIAAA HHHHxx +7806 5558 0 2 6 6 6 806 1806 2806 7806 12 13 GOAAAA UFIAAA OOOOxx +2770 5559 0 2 0 10 70 770 770 2770 2770 140 141 OCAAAA VFIAAA VVVVxx +1344 5560 0 0 4 4 44 344 1344 1344 1344 88 89 SZAAAA WFIAAA AAAAxx +511 5561 1 3 1 11 11 511 511 511 511 22 23 RTAAAA XFIAAA HHHHxx +9070 5562 0 2 0 10 70 70 1070 4070 9070 140 141 WKAAAA YFIAAA OOOOxx +2961 5563 1 1 1 1 61 961 961 2961 2961 122 123 XJAAAA ZFIAAA VVVVxx +8031 5564 1 3 1 11 31 31 31 3031 8031 62 63 XWAAAA AGIAAA AAAAxx +326 5565 0 2 6 6 26 326 326 326 326 52 53 OMAAAA BGIAAA HHHHxx +183 5566 1 3 3 3 83 183 183 183 183 166 167 BHAAAA CGIAAA OOOOxx +5917 5567 1 1 7 17 17 917 1917 917 5917 34 35 PTAAAA DGIAAA VVVVxx +8256 5568 0 0 6 16 56 256 256 3256 8256 112 113 OFAAAA EGIAAA AAAAxx +7889 5569 1 1 9 9 89 889 1889 2889 7889 178 179 LRAAAA FGIAAA HHHHxx +9029 5570 1 1 9 9 29 29 1029 4029 9029 58 59 HJAAAA GGIAAA OOOOxx +1316 5571 0 0 6 16 16 316 1316 1316 1316 32 33 QYAAAA HGIAAA VVVVxx +7442 5572 0 2 2 2 42 442 1442 2442 7442 84 85 GAAAAA IGIAAA AAAAxx +2810 5573 0 2 0 10 10 810 810 2810 2810 20 21 CEAAAA JGIAAA HHHHxx +20 5574 0 0 0 0 20 20 20 20 20 40 41 UAAAAA KGIAAA OOOOxx +2306 5575 0 2 6 6 6 306 306 2306 2306 12 13 SKAAAA LGIAAA VVVVxx +4694 5576 0 2 4 14 94 694 694 4694 4694 188 189 OYAAAA MGIAAA AAAAxx +9710 5577 0 2 0 10 10 710 1710 4710 9710 20 21 MJAAAA NGIAAA HHHHxx +1791 5578 1 3 1 11 91 791 1791 1791 1791 182 183 XQAAAA OGIAAA OOOOxx +6730 5579 0 2 0 10 30 730 730 1730 6730 60 61 WYAAAA PGIAAA VVVVxx +359 5580 1 3 9 19 59 359 359 359 359 118 119 VNAAAA QGIAAA AAAAxx +8097 5581 1 1 7 17 97 97 97 3097 8097 194 195 LZAAAA RGIAAA HHHHxx +6147 5582 1 3 7 7 47 147 147 1147 6147 94 95 LCAAAA SGIAAA OOOOxx +643 5583 1 3 3 3 43 643 643 643 643 86 87 TYAAAA TGIAAA VVVVxx +698 5584 0 2 8 18 98 698 698 698 698 196 197 WAAAAA UGIAAA AAAAxx +3881 5585 1 1 1 1 81 881 1881 3881 3881 162 163 HTAAAA VGIAAA HHHHxx +7600 5586 0 0 0 0 0 600 1600 2600 7600 0 1 IGAAAA WGIAAA OOOOxx +1583 5587 1 3 3 3 83 583 1583 1583 1583 166 167 XIAAAA XGIAAA VVVVxx +9612 5588 0 0 2 12 12 612 1612 4612 9612 24 25 SFAAAA YGIAAA AAAAxx +1032 5589 0 0 2 12 32 32 1032 1032 1032 64 65 SNAAAA ZGIAAA HHHHxx +4834 5590 0 2 4 14 34 834 834 4834 4834 68 69 YDAAAA AHIAAA OOOOxx +5076 5591 0 0 6 16 76 76 1076 76 5076 152 153 GNAAAA BHIAAA VVVVxx +3070 5592 0 2 0 10 70 70 1070 3070 3070 140 141 COAAAA CHIAAA AAAAxx +1421 5593 1 1 1 1 21 421 1421 1421 1421 42 43 RCAAAA DHIAAA HHHHxx +8970 5594 0 2 0 10 70 970 970 3970 8970 140 141 AHAAAA EHIAAA OOOOxx +6271 5595 1 3 1 11 71 271 271 1271 6271 142 143 FHAAAA FHIAAA VVVVxx +8547 5596 1 3 7 7 47 547 547 3547 8547 94 95 TQAAAA GHIAAA AAAAxx +1259 5597 1 3 9 19 59 259 1259 1259 1259 118 119 LWAAAA HHIAAA HHHHxx +8328 5598 0 0 8 8 28 328 328 3328 8328 56 57 IIAAAA IHIAAA OOOOxx +1503 5599 1 3 3 3 3 503 1503 1503 1503 6 7 VFAAAA JHIAAA VVVVxx +2253 5600 1 1 3 13 53 253 253 2253 2253 106 107 RIAAAA KHIAAA AAAAxx +7449 5601 1 1 9 9 49 449 1449 2449 7449 98 99 NAAAAA LHIAAA HHHHxx +3579 5602 1 3 9 19 79 579 1579 3579 3579 158 159 RHAAAA MHIAAA OOOOxx +1585 5603 1 1 5 5 85 585 1585 1585 1585 170 171 ZIAAAA NHIAAA VVVVxx +5543 5604 1 3 3 3 43 543 1543 543 5543 86 87 FFAAAA OHIAAA AAAAxx +8627 5605 1 3 7 7 27 627 627 3627 8627 54 55 VTAAAA PHIAAA HHHHxx +8618 5606 0 2 8 18 18 618 618 3618 8618 36 37 MTAAAA QHIAAA OOOOxx +1911 5607 1 3 1 11 11 911 1911 1911 1911 22 23 NVAAAA RHIAAA VVVVxx +2758 5608 0 2 8 18 58 758 758 2758 2758 116 117 CCAAAA SHIAAA AAAAxx +5744 5609 0 0 4 4 44 744 1744 744 5744 88 89 YMAAAA THIAAA HHHHxx +4976 5610 0 0 6 16 76 976 976 4976 4976 152 153 KJAAAA UHIAAA OOOOxx +6380 5611 0 0 0 0 80 380 380 1380 6380 160 161 KLAAAA VHIAAA VVVVxx +1937 5612 1 1 7 17 37 937 1937 1937 1937 74 75 NWAAAA WHIAAA AAAAxx +9903 5613 1 3 3 3 3 903 1903 4903 9903 6 7 XQAAAA XHIAAA HHHHxx +4409 5614 1 1 9 9 9 409 409 4409 4409 18 19 PNAAAA YHIAAA OOOOxx +4133 5615 1 1 3 13 33 133 133 4133 4133 66 67 ZCAAAA ZHIAAA VVVVxx +5263 5616 1 3 3 3 63 263 1263 263 5263 126 127 LUAAAA AIIAAA AAAAxx +7888 5617 0 0 8 8 88 888 1888 2888 7888 176 177 KRAAAA BIIAAA HHHHxx +6060 5618 0 0 0 0 60 60 60 1060 6060 120 121 CZAAAA CIIAAA OOOOxx +2522 5619 0 2 2 2 22 522 522 2522 2522 44 45 ATAAAA DIIAAA VVVVxx +5550 5620 0 2 0 10 50 550 1550 550 5550 100 101 MFAAAA EIIAAA AAAAxx +9396 5621 0 0 6 16 96 396 1396 4396 9396 192 193 KXAAAA FIIAAA HHHHxx +176 5622 0 0 6 16 76 176 176 176 176 152 153 UGAAAA GIIAAA OOOOxx +5148 5623 0 0 8 8 48 148 1148 148 5148 96 97 AQAAAA HIIAAA VVVVxx +6691 5624 1 3 1 11 91 691 691 1691 6691 182 183 JXAAAA IIIAAA AAAAxx +4652 5625 0 0 2 12 52 652 652 4652 4652 104 105 YWAAAA JIIAAA HHHHxx +5096 5626 0 0 6 16 96 96 1096 96 5096 192 193 AOAAAA KIIAAA OOOOxx +2408 5627 0 0 8 8 8 408 408 2408 2408 16 17 QOAAAA LIIAAA VVVVxx +7322 5628 0 2 2 2 22 322 1322 2322 7322 44 45 QVAAAA MIIAAA AAAAxx +6782 5629 0 2 2 2 82 782 782 1782 6782 164 165 WAAAAA NIIAAA HHHHxx +4642 5630 0 2 2 2 42 642 642 4642 4642 84 85 OWAAAA OIIAAA OOOOxx +5427 5631 1 3 7 7 27 427 1427 427 5427 54 55 TAAAAA PIIAAA VVVVxx +4461 5632 1 1 1 1 61 461 461 4461 4461 122 123 PPAAAA QIIAAA AAAAxx +8416 5633 0 0 6 16 16 416 416 3416 8416 32 33 SLAAAA RIIAAA HHHHxx +2593 5634 1 1 3 13 93 593 593 2593 2593 186 187 TVAAAA SIIAAA OOOOxx +6202 5635 0 2 2 2 2 202 202 1202 6202 4 5 OEAAAA TIIAAA VVVVxx +3826 5636 0 2 6 6 26 826 1826 3826 3826 52 53 ERAAAA UIIAAA AAAAxx +4417 5637 1 1 7 17 17 417 417 4417 4417 34 35 XNAAAA VIIAAA HHHHxx +7871 5638 1 3 1 11 71 871 1871 2871 7871 142 143 TQAAAA WIIAAA OOOOxx +5622 5639 0 2 2 2 22 622 1622 622 5622 44 45 GIAAAA XIIAAA VVVVxx +3010 5640 0 2 0 10 10 10 1010 3010 3010 20 21 ULAAAA YIIAAA AAAAxx +3407 5641 1 3 7 7 7 407 1407 3407 3407 14 15 BBAAAA ZIIAAA HHHHxx +1274 5642 0 2 4 14 74 274 1274 1274 1274 148 149 AXAAAA AJIAAA OOOOxx +2828 5643 0 0 8 8 28 828 828 2828 2828 56 57 UEAAAA BJIAAA VVVVxx +3427 5644 1 3 7 7 27 427 1427 3427 3427 54 55 VBAAAA CJIAAA AAAAxx +612 5645 0 0 2 12 12 612 612 612 612 24 25 OXAAAA DJIAAA HHHHxx +8729 5646 1 1 9 9 29 729 729 3729 8729 58 59 TXAAAA EJIAAA OOOOxx +1239 5647 1 3 9 19 39 239 1239 1239 1239 78 79 RVAAAA FJIAAA VVVVxx +8990 5648 0 2 0 10 90 990 990 3990 8990 180 181 UHAAAA GJIAAA AAAAxx +5609 5649 1 1 9 9 9 609 1609 609 5609 18 19 THAAAA HJIAAA HHHHxx +4441 5650 1 1 1 1 41 441 441 4441 4441 82 83 VOAAAA IJIAAA OOOOxx +9078 5651 0 2 8 18 78 78 1078 4078 9078 156 157 ELAAAA JJIAAA VVVVxx +6699 5652 1 3 9 19 99 699 699 1699 6699 198 199 RXAAAA KJIAAA AAAAxx +8390 5653 0 2 0 10 90 390 390 3390 8390 180 181 SKAAAA LJIAAA HHHHxx +5455 5654 1 3 5 15 55 455 1455 455 5455 110 111 VBAAAA MJIAAA OOOOxx +7537 5655 1 1 7 17 37 537 1537 2537 7537 74 75 XDAAAA NJIAAA VVVVxx +4669 5656 1 1 9 9 69 669 669 4669 4669 138 139 PXAAAA OJIAAA AAAAxx +5534 5657 0 2 4 14 34 534 1534 534 5534 68 69 WEAAAA PJIAAA HHHHxx +1920 5658 0 0 0 0 20 920 1920 1920 1920 40 41 WVAAAA QJIAAA OOOOxx +9465 5659 1 1 5 5 65 465 1465 4465 9465 130 131 BAAAAA RJIAAA VVVVxx +4897 5660 1 1 7 17 97 897 897 4897 4897 194 195 JGAAAA SJIAAA AAAAxx +1990 5661 0 2 0 10 90 990 1990 1990 1990 180 181 OYAAAA TJIAAA HHHHxx +7148 5662 0 0 8 8 48 148 1148 2148 7148 96 97 YOAAAA UJIAAA OOOOxx +533 5663 1 1 3 13 33 533 533 533 533 66 67 NUAAAA VJIAAA VVVVxx +4339 5664 1 3 9 19 39 339 339 4339 4339 78 79 XKAAAA WJIAAA AAAAxx +6450 5665 0 2 0 10 50 450 450 1450 6450 100 101 COAAAA XJIAAA HHHHxx +9627 5666 1 3 7 7 27 627 1627 4627 9627 54 55 HGAAAA YJIAAA OOOOxx +5539 5667 1 3 9 19 39 539 1539 539 5539 78 79 BFAAAA ZJIAAA VVVVxx +6758 5668 0 2 8 18 58 758 758 1758 6758 116 117 YZAAAA AKIAAA AAAAxx +3435 5669 1 3 5 15 35 435 1435 3435 3435 70 71 DCAAAA BKIAAA HHHHxx +4350 5670 0 2 0 10 50 350 350 4350 4350 100 101 ILAAAA CKIAAA OOOOxx +9088 5671 0 0 8 8 88 88 1088 4088 9088 176 177 OLAAAA DKIAAA VVVVxx +6368 5672 0 0 8 8 68 368 368 1368 6368 136 137 YKAAAA EKIAAA AAAAxx +6337 5673 1 1 7 17 37 337 337 1337 6337 74 75 TJAAAA FKIAAA HHHHxx +4361 5674 1 1 1 1 61 361 361 4361 4361 122 123 TLAAAA GKIAAA OOOOxx +1719 5675 1 3 9 19 19 719 1719 1719 1719 38 39 DOAAAA HKIAAA VVVVxx +3109 5676 1 1 9 9 9 109 1109 3109 3109 18 19 PPAAAA IKIAAA AAAAxx +7135 5677 1 3 5 15 35 135 1135 2135 7135 70 71 LOAAAA JKIAAA HHHHxx +1964 5678 0 0 4 4 64 964 1964 1964 1964 128 129 OXAAAA KKIAAA OOOOxx +3 5679 1 3 3 3 3 3 3 3 3 6 7 DAAAAA LKIAAA VVVVxx +1868 5680 0 0 8 8 68 868 1868 1868 1868 136 137 WTAAAA MKIAAA AAAAxx +5182 5681 0 2 2 2 82 182 1182 182 5182 164 165 IRAAAA NKIAAA HHHHxx +7567 5682 1 3 7 7 67 567 1567 2567 7567 134 135 BFAAAA OKIAAA OOOOxx +3676 5683 0 0 6 16 76 676 1676 3676 3676 152 153 KLAAAA PKIAAA VVVVxx +9382 5684 0 2 2 2 82 382 1382 4382 9382 164 165 WWAAAA QKIAAA AAAAxx +8645 5685 1 1 5 5 45 645 645 3645 8645 90 91 NUAAAA RKIAAA HHHHxx +2018 5686 0 2 8 18 18 18 18 2018 2018 36 37 QZAAAA SKIAAA OOOOxx +217 5687 1 1 7 17 17 217 217 217 217 34 35 JIAAAA TKIAAA VVVVxx +6793 5688 1 1 3 13 93 793 793 1793 6793 186 187 HBAAAA UKIAAA AAAAxx +7280 5689 0 0 0 0 80 280 1280 2280 7280 160 161 AUAAAA VKIAAA HHHHxx +2168 5690 0 0 8 8 68 168 168 2168 2168 136 137 KFAAAA WKIAAA OOOOxx +5259 5691 1 3 9 19 59 259 1259 259 5259 118 119 HUAAAA XKIAAA VVVVxx +6019 5692 1 3 9 19 19 19 19 1019 6019 38 39 NXAAAA YKIAAA AAAAxx +877 5693 1 1 7 17 77 877 877 877 877 154 155 THAAAA ZKIAAA HHHHxx +4961 5694 1 1 1 1 61 961 961 4961 4961 122 123 VIAAAA ALIAAA OOOOxx +1873 5695 1 1 3 13 73 873 1873 1873 1873 146 147 BUAAAA BLIAAA VVVVxx +13 5696 1 1 3 13 13 13 13 13 13 26 27 NAAAAA CLIAAA AAAAxx +1537 5697 1 1 7 17 37 537 1537 1537 1537 74 75 DHAAAA DLIAAA HHHHxx +3129 5698 1 1 9 9 29 129 1129 3129 3129 58 59 JQAAAA ELIAAA OOOOxx +6473 5699 1 1 3 13 73 473 473 1473 6473 146 147 ZOAAAA FLIAAA VVVVxx +7865 5700 1 1 5 5 65 865 1865 2865 7865 130 131 NQAAAA GLIAAA AAAAxx +7822 5701 0 2 2 2 22 822 1822 2822 7822 44 45 WOAAAA HLIAAA HHHHxx +239 5702 1 3 9 19 39 239 239 239 239 78 79 FJAAAA ILIAAA OOOOxx +2062 5703 0 2 2 2 62 62 62 2062 2062 124 125 IBAAAA JLIAAA VVVVxx +762 5704 0 2 2 2 62 762 762 762 762 124 125 IDAAAA KLIAAA AAAAxx +3764 5705 0 0 4 4 64 764 1764 3764 3764 128 129 UOAAAA LLIAAA HHHHxx +465 5706 1 1 5 5 65 465 465 465 465 130 131 XRAAAA MLIAAA OOOOxx +2587 5707 1 3 7 7 87 587 587 2587 2587 174 175 NVAAAA NLIAAA VVVVxx +8402 5708 0 2 2 2 2 402 402 3402 8402 4 5 ELAAAA OLIAAA AAAAxx +1055 5709 1 3 5 15 55 55 1055 1055 1055 110 111 POAAAA PLIAAA HHHHxx +3072 5710 0 0 2 12 72 72 1072 3072 3072 144 145 EOAAAA QLIAAA OOOOxx +7359 5711 1 3 9 19 59 359 1359 2359 7359 118 119 BXAAAA RLIAAA VVVVxx +6558 5712 0 2 8 18 58 558 558 1558 6558 116 117 GSAAAA SLIAAA AAAAxx +48 5713 0 0 8 8 48 48 48 48 48 96 97 WBAAAA TLIAAA HHHHxx +5382 5714 0 2 2 2 82 382 1382 382 5382 164 165 AZAAAA ULIAAA OOOOxx +947 5715 1 3 7 7 47 947 947 947 947 94 95 LKAAAA VLIAAA VVVVxx +2644 5716 0 0 4 4 44 644 644 2644 2644 88 89 SXAAAA WLIAAA AAAAxx +7516 5717 0 0 6 16 16 516 1516 2516 7516 32 33 CDAAAA XLIAAA HHHHxx +2362 5718 0 2 2 2 62 362 362 2362 2362 124 125 WMAAAA YLIAAA OOOOxx +839 5719 1 3 9 19 39 839 839 839 839 78 79 HGAAAA ZLIAAA VVVVxx +2216 5720 0 0 6 16 16 216 216 2216 2216 32 33 GHAAAA AMIAAA AAAAxx +7673 5721 1 1 3 13 73 673 1673 2673 7673 146 147 DJAAAA BMIAAA HHHHxx +8173 5722 1 1 3 13 73 173 173 3173 8173 146 147 JCAAAA CMIAAA OOOOxx +1630 5723 0 2 0 10 30 630 1630 1630 1630 60 61 SKAAAA DMIAAA VVVVxx +9057 5724 1 1 7 17 57 57 1057 4057 9057 114 115 JKAAAA EMIAAA AAAAxx +4392 5725 0 0 2 12 92 392 392 4392 4392 184 185 YMAAAA FMIAAA HHHHxx +3695 5726 1 3 5 15 95 695 1695 3695 3695 190 191 DMAAAA GMIAAA OOOOxx +5751 5727 1 3 1 11 51 751 1751 751 5751 102 103 FNAAAA HMIAAA VVVVxx +5745 5728 1 1 5 5 45 745 1745 745 5745 90 91 ZMAAAA IMIAAA AAAAxx +7945 5729 1 1 5 5 45 945 1945 2945 7945 90 91 PTAAAA JMIAAA HHHHxx +5174 5730 0 2 4 14 74 174 1174 174 5174 148 149 ARAAAA KMIAAA OOOOxx +3829 5731 1 1 9 9 29 829 1829 3829 3829 58 59 HRAAAA LMIAAA VVVVxx +3317 5732 1 1 7 17 17 317 1317 3317 3317 34 35 PXAAAA MMIAAA AAAAxx +4253 5733 1 1 3 13 53 253 253 4253 4253 106 107 PHAAAA NMIAAA HHHHxx +1291 5734 1 3 1 11 91 291 1291 1291 1291 182 183 RXAAAA OMIAAA OOOOxx +3266 5735 0 2 6 6 66 266 1266 3266 3266 132 133 QVAAAA PMIAAA VVVVxx +2939 5736 1 3 9 19 39 939 939 2939 2939 78 79 BJAAAA QMIAAA AAAAxx +2755 5737 1 3 5 15 55 755 755 2755 2755 110 111 ZBAAAA RMIAAA HHHHxx +6844 5738 0 0 4 4 44 844 844 1844 6844 88 89 GDAAAA SMIAAA OOOOxx +8594 5739 0 2 4 14 94 594 594 3594 8594 188 189 OSAAAA TMIAAA VVVVxx +704 5740 0 0 4 4 4 704 704 704 704 8 9 CBAAAA UMIAAA AAAAxx +1681 5741 1 1 1 1 81 681 1681 1681 1681 162 163 RMAAAA VMIAAA HHHHxx +364 5742 0 0 4 4 64 364 364 364 364 128 129 AOAAAA WMIAAA OOOOxx +2928 5743 0 0 8 8 28 928 928 2928 2928 56 57 QIAAAA XMIAAA VVVVxx +117 5744 1 1 7 17 17 117 117 117 117 34 35 NEAAAA YMIAAA AAAAxx +96 5745 0 0 6 16 96 96 96 96 96 192 193 SDAAAA ZMIAAA HHHHxx +7796 5746 0 0 6 16 96 796 1796 2796 7796 192 193 WNAAAA ANIAAA OOOOxx +3101 5747 1 1 1 1 1 101 1101 3101 3101 2 3 HPAAAA BNIAAA VVVVxx +3397 5748 1 1 7 17 97 397 1397 3397 3397 194 195 RAAAAA CNIAAA AAAAxx +1605 5749 1 1 5 5 5 605 1605 1605 1605 10 11 TJAAAA DNIAAA HHHHxx +4881 5750 1 1 1 1 81 881 881 4881 4881 162 163 TFAAAA ENIAAA OOOOxx +4521 5751 1 1 1 1 21 521 521 4521 4521 42 43 XRAAAA FNIAAA VVVVxx +6430 5752 0 2 0 10 30 430 430 1430 6430 60 61 INAAAA GNIAAA AAAAxx +282 5753 0 2 2 2 82 282 282 282 282 164 165 WKAAAA HNIAAA HHHHxx +9645 5754 1 1 5 5 45 645 1645 4645 9645 90 91 ZGAAAA INIAAA OOOOxx +8946 5755 0 2 6 6 46 946 946 3946 8946 92 93 CGAAAA JNIAAA VVVVxx +5064 5756 0 0 4 4 64 64 1064 64 5064 128 129 UMAAAA KNIAAA AAAAxx +7470 5757 0 2 0 10 70 470 1470 2470 7470 140 141 IBAAAA LNIAAA HHHHxx +5886 5758 0 2 6 6 86 886 1886 886 5886 172 173 KSAAAA MNIAAA OOOOxx +6280 5759 0 0 0 0 80 280 280 1280 6280 160 161 OHAAAA NNIAAA VVVVxx +5247 5760 1 3 7 7 47 247 1247 247 5247 94 95 VTAAAA ONIAAA AAAAxx +412 5761 0 0 2 12 12 412 412 412 412 24 25 WPAAAA PNIAAA HHHHxx +5342 5762 0 2 2 2 42 342 1342 342 5342 84 85 MXAAAA QNIAAA OOOOxx +2271 5763 1 3 1 11 71 271 271 2271 2271 142 143 JJAAAA RNIAAA VVVVxx +849 5764 1 1 9 9 49 849 849 849 849 98 99 RGAAAA SNIAAA AAAAxx +1885 5765 1 1 5 5 85 885 1885 1885 1885 170 171 NUAAAA TNIAAA HHHHxx +5620 5766 0 0 0 0 20 620 1620 620 5620 40 41 EIAAAA UNIAAA OOOOxx +7079 5767 1 3 9 19 79 79 1079 2079 7079 158 159 HMAAAA VNIAAA VVVVxx +5819 5768 1 3 9 19 19 819 1819 819 5819 38 39 VPAAAA WNIAAA AAAAxx +7497 5769 1 1 7 17 97 497 1497 2497 7497 194 195 JCAAAA XNIAAA HHHHxx +5993 5770 1 1 3 13 93 993 1993 993 5993 186 187 NWAAAA YNIAAA OOOOxx +3739 5771 1 3 9 19 39 739 1739 3739 3739 78 79 VNAAAA ZNIAAA VVVVxx +6296 5772 0 0 6 16 96 296 296 1296 6296 192 193 EIAAAA AOIAAA AAAAxx +2716 5773 0 0 6 16 16 716 716 2716 2716 32 33 MAAAAA BOIAAA HHHHxx +1130 5774 0 2 0 10 30 130 1130 1130 1130 60 61 MRAAAA COIAAA OOOOxx +5593 5775 1 1 3 13 93 593 1593 593 5593 186 187 DHAAAA DOIAAA VVVVxx +6972 5776 0 0 2 12 72 972 972 1972 6972 144 145 EIAAAA EOIAAA AAAAxx +8360 5777 0 0 0 0 60 360 360 3360 8360 120 121 OJAAAA FOIAAA HHHHxx +6448 5778 0 0 8 8 48 448 448 1448 6448 96 97 AOAAAA GOIAAA OOOOxx +3689 5779 1 1 9 9 89 689 1689 3689 3689 178 179 XLAAAA HOIAAA VVVVxx +7951 5780 1 3 1 11 51 951 1951 2951 7951 102 103 VTAAAA IOIAAA AAAAxx +2974 5781 0 2 4 14 74 974 974 2974 2974 148 149 KKAAAA JOIAAA HHHHxx +6600 5782 0 0 0 0 0 600 600 1600 6600 0 1 WTAAAA KOIAAA OOOOxx +4662 5783 0 2 2 2 62 662 662 4662 4662 124 125 IXAAAA LOIAAA VVVVxx +4765 5784 1 1 5 5 65 765 765 4765 4765 130 131 HBAAAA MOIAAA AAAAxx +355 5785 1 3 5 15 55 355 355 355 355 110 111 RNAAAA NOIAAA HHHHxx +6228 5786 0 0 8 8 28 228 228 1228 6228 56 57 OFAAAA OOIAAA OOOOxx +964 5787 0 0 4 4 64 964 964 964 964 128 129 CLAAAA POIAAA VVVVxx +3082 5788 0 2 2 2 82 82 1082 3082 3082 164 165 OOAAAA QOIAAA AAAAxx +7028 5789 0 0 8 8 28 28 1028 2028 7028 56 57 IKAAAA ROIAAA HHHHxx +4505 5790 1 1 5 5 5 505 505 4505 4505 10 11 HRAAAA SOIAAA OOOOxx +8961 5791 1 1 1 1 61 961 961 3961 8961 122 123 RGAAAA TOIAAA VVVVxx +9571 5792 1 3 1 11 71 571 1571 4571 9571 142 143 DEAAAA UOIAAA AAAAxx +9394 5793 0 2 4 14 94 394 1394 4394 9394 188 189 IXAAAA VOIAAA HHHHxx +4245 5794 1 1 5 5 45 245 245 4245 4245 90 91 HHAAAA WOIAAA OOOOxx +7560 5795 0 0 0 0 60 560 1560 2560 7560 120 121 UEAAAA XOIAAA VVVVxx +2907 5796 1 3 7 7 7 907 907 2907 2907 14 15 VHAAAA YOIAAA AAAAxx +7817 5797 1 1 7 17 17 817 1817 2817 7817 34 35 ROAAAA ZOIAAA HHHHxx +5408 5798 0 0 8 8 8 408 1408 408 5408 16 17 AAAAAA APIAAA OOOOxx +8092 5799 0 0 2 12 92 92 92 3092 8092 184 185 GZAAAA BPIAAA VVVVxx +1309 5800 1 1 9 9 9 309 1309 1309 1309 18 19 JYAAAA CPIAAA AAAAxx +6673 5801 1 1 3 13 73 673 673 1673 6673 146 147 RWAAAA DPIAAA HHHHxx +1245 5802 1 1 5 5 45 245 1245 1245 1245 90 91 XVAAAA EPIAAA OOOOxx +6790 5803 0 2 0 10 90 790 790 1790 6790 180 181 EBAAAA FPIAAA VVVVxx +8380 5804 0 0 0 0 80 380 380 3380 8380 160 161 IKAAAA GPIAAA AAAAxx +5786 5805 0 2 6 6 86 786 1786 786 5786 172 173 OOAAAA HPIAAA HHHHxx +9590 5806 0 2 0 10 90 590 1590 4590 9590 180 181 WEAAAA IPIAAA OOOOxx +5763 5807 1 3 3 3 63 763 1763 763 5763 126 127 RNAAAA JPIAAA VVVVxx +1345 5808 1 1 5 5 45 345 1345 1345 1345 90 91 TZAAAA KPIAAA AAAAxx +3480 5809 0 0 0 0 80 480 1480 3480 3480 160 161 WDAAAA LPIAAA HHHHxx +7864 5810 0 0 4 4 64 864 1864 2864 7864 128 129 MQAAAA MPIAAA OOOOxx +4853 5811 1 1 3 13 53 853 853 4853 4853 106 107 REAAAA NPIAAA VVVVxx +1445 5812 1 1 5 5 45 445 1445 1445 1445 90 91 PDAAAA OPIAAA AAAAxx +170 5813 0 2 0 10 70 170 170 170 170 140 141 OGAAAA PPIAAA HHHHxx +7348 5814 0 0 8 8 48 348 1348 2348 7348 96 97 QWAAAA QPIAAA OOOOxx +3920 5815 0 0 0 0 20 920 1920 3920 3920 40 41 UUAAAA RPIAAA VVVVxx +3307 5816 1 3 7 7 7 307 1307 3307 3307 14 15 FXAAAA SPIAAA AAAAxx +4584 5817 0 0 4 4 84 584 584 4584 4584 168 169 IUAAAA TPIAAA HHHHxx +3344 5818 0 0 4 4 44 344 1344 3344 3344 88 89 QYAAAA UPIAAA OOOOxx +4360 5819 0 0 0 0 60 360 360 4360 4360 120 121 SLAAAA VPIAAA VVVVxx +8757 5820 1 1 7 17 57 757 757 3757 8757 114 115 VYAAAA WPIAAA AAAAxx +4315 5821 1 3 5 15 15 315 315 4315 4315 30 31 ZJAAAA XPIAAA HHHHxx +5243 5822 1 3 3 3 43 243 1243 243 5243 86 87 RTAAAA YPIAAA OOOOxx +8550 5823 0 2 0 10 50 550 550 3550 8550 100 101 WQAAAA ZPIAAA VVVVxx +159 5824 1 3 9 19 59 159 159 159 159 118 119 DGAAAA AQIAAA AAAAxx +4710 5825 0 2 0 10 10 710 710 4710 4710 20 21 EZAAAA BQIAAA HHHHxx +7179 5826 1 3 9 19 79 179 1179 2179 7179 158 159 DQAAAA CQIAAA OOOOxx +2509 5827 1 1 9 9 9 509 509 2509 2509 18 19 NSAAAA DQIAAA VVVVxx +6981 5828 1 1 1 1 81 981 981 1981 6981 162 163 NIAAAA EQIAAA AAAAxx +5060 5829 0 0 0 0 60 60 1060 60 5060 120 121 QMAAAA FQIAAA HHHHxx +5601 5830 1 1 1 1 1 601 1601 601 5601 2 3 LHAAAA GQIAAA OOOOxx +703 5831 1 3 3 3 3 703 703 703 703 6 7 BBAAAA HQIAAA VVVVxx +8719 5832 1 3 9 19 19 719 719 3719 8719 38 39 JXAAAA IQIAAA AAAAxx +1570 5833 0 2 0 10 70 570 1570 1570 1570 140 141 KIAAAA JQIAAA HHHHxx +1036 5834 0 0 6 16 36 36 1036 1036 1036 72 73 WNAAAA KQIAAA OOOOxx +6703 5835 1 3 3 3 3 703 703 1703 6703 6 7 VXAAAA LQIAAA VVVVxx +252 5836 0 0 2 12 52 252 252 252 252 104 105 SJAAAA MQIAAA AAAAxx +631 5837 1 3 1 11 31 631 631 631 631 62 63 HYAAAA NQIAAA HHHHxx +5098 5838 0 2 8 18 98 98 1098 98 5098 196 197 COAAAA OQIAAA OOOOxx +8346 5839 0 2 6 6 46 346 346 3346 8346 92 93 AJAAAA PQIAAA VVVVxx +4910 5840 0 2 0 10 10 910 910 4910 4910 20 21 WGAAAA QQIAAA AAAAxx +559 5841 1 3 9 19 59 559 559 559 559 118 119 NVAAAA RQIAAA HHHHxx +1477 5842 1 1 7 17 77 477 1477 1477 1477 154 155 VEAAAA SQIAAA OOOOxx +5115 5843 1 3 5 15 15 115 1115 115 5115 30 31 TOAAAA TQIAAA VVVVxx +8784 5844 0 0 4 4 84 784 784 3784 8784 168 169 WZAAAA UQIAAA AAAAxx +4422 5845 0 2 2 2 22 422 422 4422 4422 44 45 COAAAA VQIAAA HHHHxx +2702 5846 0 2 2 2 2 702 702 2702 2702 4 5 YZAAAA WQIAAA OOOOxx +9599 5847 1 3 9 19 99 599 1599 4599 9599 198 199 FFAAAA XQIAAA VVVVxx +2463 5848 1 3 3 3 63 463 463 2463 2463 126 127 TQAAAA YQIAAA AAAAxx +498 5849 0 2 8 18 98 498 498 498 498 196 197 ETAAAA ZQIAAA HHHHxx +494 5850 0 2 4 14 94 494 494 494 494 188 189 ATAAAA ARIAAA OOOOxx +8632 5851 0 0 2 12 32 632 632 3632 8632 64 65 AUAAAA BRIAAA VVVVxx +3449 5852 1 1 9 9 49 449 1449 3449 3449 98 99 RCAAAA CRIAAA AAAAxx +5888 5853 0 0 8 8 88 888 1888 888 5888 176 177 MSAAAA DRIAAA HHHHxx +2211 5854 1 3 1 11 11 211 211 2211 2211 22 23 BHAAAA ERIAAA OOOOxx +2835 5855 1 3 5 15 35 835 835 2835 2835 70 71 BFAAAA FRIAAA VVVVxx +4196 5856 0 0 6 16 96 196 196 4196 4196 192 193 KFAAAA GRIAAA AAAAxx +2177 5857 1 1 7 17 77 177 177 2177 2177 154 155 TFAAAA HRIAAA HHHHxx +1959 5858 1 3 9 19 59 959 1959 1959 1959 118 119 JXAAAA IRIAAA OOOOxx +5172 5859 0 0 2 12 72 172 1172 172 5172 144 145 YQAAAA JRIAAA VVVVxx +7898 5860 0 2 8 18 98 898 1898 2898 7898 196 197 URAAAA KRIAAA AAAAxx +5729 5861 1 1 9 9 29 729 1729 729 5729 58 59 JMAAAA LRIAAA HHHHxx +469 5862 1 1 9 9 69 469 469 469 469 138 139 BSAAAA MRIAAA OOOOxx +4456 5863 0 0 6 16 56 456 456 4456 4456 112 113 KPAAAA NRIAAA VVVVxx +3578 5864 0 2 8 18 78 578 1578 3578 3578 156 157 QHAAAA ORIAAA AAAAxx +8623 5865 1 3 3 3 23 623 623 3623 8623 46 47 RTAAAA PRIAAA HHHHxx +6749 5866 1 1 9 9 49 749 749 1749 6749 98 99 PZAAAA QRIAAA OOOOxx +6735 5867 1 3 5 15 35 735 735 1735 6735 70 71 BZAAAA RRIAAA VVVVxx +5197 5868 1 1 7 17 97 197 1197 197 5197 194 195 XRAAAA SRIAAA AAAAxx +2067 5869 1 3 7 7 67 67 67 2067 2067 134 135 NBAAAA TRIAAA HHHHxx +5600 5870 0 0 0 0 0 600 1600 600 5600 0 1 KHAAAA URIAAA OOOOxx +7741 5871 1 1 1 1 41 741 1741 2741 7741 82 83 TLAAAA VRIAAA VVVVxx +9925 5872 1 1 5 5 25 925 1925 4925 9925 50 51 TRAAAA WRIAAA AAAAxx +9685 5873 1 1 5 5 85 685 1685 4685 9685 170 171 NIAAAA XRIAAA HHHHxx +7622 5874 0 2 2 2 22 622 1622 2622 7622 44 45 EHAAAA YRIAAA OOOOxx +6859 5875 1 3 9 19 59 859 859 1859 6859 118 119 VDAAAA ZRIAAA VVVVxx +3094 5876 0 2 4 14 94 94 1094 3094 3094 188 189 APAAAA ASIAAA AAAAxx +2628 5877 0 0 8 8 28 628 628 2628 2628 56 57 CXAAAA BSIAAA HHHHxx +40 5878 0 0 0 0 40 40 40 40 40 80 81 OBAAAA CSIAAA OOOOxx +1644 5879 0 0 4 4 44 644 1644 1644 1644 88 89 GLAAAA DSIAAA VVVVxx +588 5880 0 0 8 8 88 588 588 588 588 176 177 QWAAAA ESIAAA AAAAxx +7522 5881 0 2 2 2 22 522 1522 2522 7522 44 45 IDAAAA FSIAAA HHHHxx +162 5882 0 2 2 2 62 162 162 162 162 124 125 GGAAAA GSIAAA OOOOxx +3610 5883 0 2 0 10 10 610 1610 3610 3610 20 21 WIAAAA HSIAAA VVVVxx +3561 5884 1 1 1 1 61 561 1561 3561 3561 122 123 ZGAAAA ISIAAA AAAAxx +8185 5885 1 1 5 5 85 185 185 3185 8185 170 171 VCAAAA JSIAAA HHHHxx +7237 5886 1 1 7 17 37 237 1237 2237 7237 74 75 JSAAAA KSIAAA OOOOxx +4592 5887 0 0 2 12 92 592 592 4592 4592 184 185 QUAAAA LSIAAA VVVVxx +7082 5888 0 2 2 2 82 82 1082 2082 7082 164 165 KMAAAA MSIAAA AAAAxx +4719 5889 1 3 9 19 19 719 719 4719 4719 38 39 NZAAAA NSIAAA HHHHxx +3879 5890 1 3 9 19 79 879 1879 3879 3879 158 159 FTAAAA OSIAAA OOOOxx +1662 5891 0 2 2 2 62 662 1662 1662 1662 124 125 YLAAAA PSIAAA VVVVxx +3995 5892 1 3 5 15 95 995 1995 3995 3995 190 191 RXAAAA QSIAAA AAAAxx +5828 5893 0 0 8 8 28 828 1828 828 5828 56 57 EQAAAA RSIAAA HHHHxx +4197 5894 1 1 7 17 97 197 197 4197 4197 194 195 LFAAAA SSIAAA OOOOxx +5146 5895 0 2 6 6 46 146 1146 146 5146 92 93 YPAAAA TSIAAA VVVVxx +753 5896 1 1 3 13 53 753 753 753 753 106 107 ZCAAAA USIAAA AAAAxx +7064 5897 0 0 4 4 64 64 1064 2064 7064 128 129 SLAAAA VSIAAA HHHHxx +1312 5898 0 0 2 12 12 312 1312 1312 1312 24 25 MYAAAA WSIAAA OOOOxx +5573 5899 1 1 3 13 73 573 1573 573 5573 146 147 JGAAAA XSIAAA VVVVxx +7634 5900 0 2 4 14 34 634 1634 2634 7634 68 69 QHAAAA YSIAAA AAAAxx +2459 5901 1 3 9 19 59 459 459 2459 2459 118 119 PQAAAA ZSIAAA HHHHxx +8636 5902 0 0 6 16 36 636 636 3636 8636 72 73 EUAAAA ATIAAA OOOOxx +5318 5903 0 2 8 18 18 318 1318 318 5318 36 37 OWAAAA BTIAAA VVVVxx +1064 5904 0 0 4 4 64 64 1064 1064 1064 128 129 YOAAAA CTIAAA AAAAxx +9779 5905 1 3 9 19 79 779 1779 4779 9779 158 159 DMAAAA DTIAAA HHHHxx +6512 5906 0 0 2 12 12 512 512 1512 6512 24 25 MQAAAA ETIAAA OOOOxx +3572 5907 0 0 2 12 72 572 1572 3572 3572 144 145 KHAAAA FTIAAA VVVVxx +816 5908 0 0 6 16 16 816 816 816 816 32 33 KFAAAA GTIAAA AAAAxx +3978 5909 0 2 8 18 78 978 1978 3978 3978 156 157 AXAAAA HTIAAA HHHHxx +5390 5910 0 2 0 10 90 390 1390 390 5390 180 181 IZAAAA ITIAAA OOOOxx +4685 5911 1 1 5 5 85 685 685 4685 4685 170 171 FYAAAA JTIAAA VVVVxx +3003 5912 1 3 3 3 3 3 1003 3003 3003 6 7 NLAAAA KTIAAA AAAAxx +2638 5913 0 2 8 18 38 638 638 2638 2638 76 77 MXAAAA LTIAAA HHHHxx +9716 5914 0 0 6 16 16 716 1716 4716 9716 32 33 SJAAAA MTIAAA OOOOxx +9598 5915 0 2 8 18 98 598 1598 4598 9598 196 197 EFAAAA NTIAAA VVVVxx +9501 5916 1 1 1 1 1 501 1501 4501 9501 2 3 LBAAAA OTIAAA AAAAxx +1704 5917 0 0 4 4 4 704 1704 1704 1704 8 9 ONAAAA PTIAAA HHHHxx +8609 5918 1 1 9 9 9 609 609 3609 8609 18 19 DTAAAA QTIAAA OOOOxx +5211 5919 1 3 1 11 11 211 1211 211 5211 22 23 LSAAAA RTIAAA VVVVxx +3605 5920 1 1 5 5 5 605 1605 3605 3605 10 11 RIAAAA STIAAA AAAAxx +8730 5921 0 2 0 10 30 730 730 3730 8730 60 61 UXAAAA TTIAAA HHHHxx +4208 5922 0 0 8 8 8 208 208 4208 4208 16 17 WFAAAA UTIAAA OOOOxx +7784 5923 0 0 4 4 84 784 1784 2784 7784 168 169 KNAAAA VTIAAA VVVVxx +7501 5924 1 1 1 1 1 501 1501 2501 7501 2 3 NCAAAA WTIAAA AAAAxx +7862 5925 0 2 2 2 62 862 1862 2862 7862 124 125 KQAAAA XTIAAA HHHHxx +8922 5926 0 2 2 2 22 922 922 3922 8922 44 45 EFAAAA YTIAAA OOOOxx +3857 5927 1 1 7 17 57 857 1857 3857 3857 114 115 JSAAAA ZTIAAA VVVVxx +6393 5928 1 1 3 13 93 393 393 1393 6393 186 187 XLAAAA AUIAAA AAAAxx +506 5929 0 2 6 6 6 506 506 506 506 12 13 MTAAAA BUIAAA HHHHxx +4232 5930 0 0 2 12 32 232 232 4232 4232 64 65 UGAAAA CUIAAA OOOOxx +8991 5931 1 3 1 11 91 991 991 3991 8991 182 183 VHAAAA DUIAAA VVVVxx +8578 5932 0 2 8 18 78 578 578 3578 8578 156 157 YRAAAA EUIAAA AAAAxx +3235 5933 1 3 5 15 35 235 1235 3235 3235 70 71 LUAAAA FUIAAA HHHHxx +963 5934 1 3 3 3 63 963 963 963 963 126 127 BLAAAA GUIAAA OOOOxx +113 5935 1 1 3 13 13 113 113 113 113 26 27 JEAAAA HUIAAA VVVVxx +8234 5936 0 2 4 14 34 234 234 3234 8234 68 69 SEAAAA IUIAAA AAAAxx +2613 5937 1 1 3 13 13 613 613 2613 2613 26 27 NWAAAA JUIAAA HHHHxx +5540 5938 0 0 0 0 40 540 1540 540 5540 80 81 CFAAAA KUIAAA OOOOxx +9727 5939 1 3 7 7 27 727 1727 4727 9727 54 55 DKAAAA LUIAAA VVVVxx +2229 5940 1 1 9 9 29 229 229 2229 2229 58 59 THAAAA MUIAAA AAAAxx +6242 5941 0 2 2 2 42 242 242 1242 6242 84 85 CGAAAA NUIAAA HHHHxx +2502 5942 0 2 2 2 2 502 502 2502 2502 4 5 GSAAAA OUIAAA OOOOxx +6212 5943 0 0 2 12 12 212 212 1212 6212 24 25 YEAAAA PUIAAA VVVVxx +3495 5944 1 3 5 15 95 495 1495 3495 3495 190 191 LEAAAA QUIAAA AAAAxx +2364 5945 0 0 4 4 64 364 364 2364 2364 128 129 YMAAAA RUIAAA HHHHxx +6777 5946 1 1 7 17 77 777 777 1777 6777 154 155 RAAAAA SUIAAA OOOOxx +9811 5947 1 3 1 11 11 811 1811 4811 9811 22 23 JNAAAA TUIAAA VVVVxx +1450 5948 0 2 0 10 50 450 1450 1450 1450 100 101 UDAAAA UUIAAA AAAAxx +5008 5949 0 0 8 8 8 8 1008 8 5008 16 17 QKAAAA VUIAAA HHHHxx +1318 5950 0 2 8 18 18 318 1318 1318 1318 36 37 SYAAAA WUIAAA OOOOxx +3373 5951 1 1 3 13 73 373 1373 3373 3373 146 147 TZAAAA XUIAAA VVVVxx +398 5952 0 2 8 18 98 398 398 398 398 196 197 IPAAAA YUIAAA AAAAxx +3804 5953 0 0 4 4 4 804 1804 3804 3804 8 9 IQAAAA ZUIAAA HHHHxx +9148 5954 0 0 8 8 48 148 1148 4148 9148 96 97 WNAAAA AVIAAA OOOOxx +4382 5955 0 2 2 2 82 382 382 4382 4382 164 165 OMAAAA BVIAAA VVVVxx +4026 5956 0 2 6 6 26 26 26 4026 4026 52 53 WYAAAA CVIAAA AAAAxx +7804 5957 0 0 4 4 4 804 1804 2804 7804 8 9 EOAAAA DVIAAA HHHHxx +6839 5958 1 3 9 19 39 839 839 1839 6839 78 79 BDAAAA EVIAAA OOOOxx +3756 5959 0 0 6 16 56 756 1756 3756 3756 112 113 MOAAAA FVIAAA VVVVxx +6734 5960 0 2 4 14 34 734 734 1734 6734 68 69 AZAAAA GVIAAA AAAAxx +2228 5961 0 0 8 8 28 228 228 2228 2228 56 57 SHAAAA HVIAAA HHHHxx +3273 5962 1 1 3 13 73 273 1273 3273 3273 146 147 XVAAAA IVIAAA OOOOxx +3708 5963 0 0 8 8 8 708 1708 3708 3708 16 17 QMAAAA JVIAAA VVVVxx +4320 5964 0 0 0 0 20 320 320 4320 4320 40 41 EKAAAA KVIAAA AAAAxx +74 5965 0 2 4 14 74 74 74 74 74 148 149 WCAAAA LVIAAA HHHHxx +2520 5966 0 0 0 0 20 520 520 2520 2520 40 41 YSAAAA MVIAAA OOOOxx +9619 5967 1 3 9 19 19 619 1619 4619 9619 38 39 ZFAAAA NVIAAA VVVVxx +1801 5968 1 1 1 1 1 801 1801 1801 1801 2 3 HRAAAA OVIAAA AAAAxx +6399 5969 1 3 9 19 99 399 399 1399 6399 198 199 DMAAAA PVIAAA HHHHxx +8313 5970 1 1 3 13 13 313 313 3313 8313 26 27 THAAAA QVIAAA OOOOxx +7003 5971 1 3 3 3 3 3 1003 2003 7003 6 7 JJAAAA RVIAAA VVVVxx +329 5972 1 1 9 9 29 329 329 329 329 58 59 RMAAAA SVIAAA AAAAxx +9090 5973 0 2 0 10 90 90 1090 4090 9090 180 181 QLAAAA TVIAAA HHHHxx +2299 5974 1 3 9 19 99 299 299 2299 2299 198 199 LKAAAA UVIAAA OOOOxx +3925 5975 1 1 5 5 25 925 1925 3925 3925 50 51 ZUAAAA VVIAAA VVVVxx +8145 5976 1 1 5 5 45 145 145 3145 8145 90 91 HBAAAA WVIAAA AAAAxx +8561 5977 1 1 1 1 61 561 561 3561 8561 122 123 HRAAAA XVIAAA HHHHxx +2797 5978 1 1 7 17 97 797 797 2797 2797 194 195 PDAAAA YVIAAA OOOOxx +1451 5979 1 3 1 11 51 451 1451 1451 1451 102 103 VDAAAA ZVIAAA VVVVxx +7977 5980 1 1 7 17 77 977 1977 2977 7977 154 155 VUAAAA AWIAAA AAAAxx +112 5981 0 0 2 12 12 112 112 112 112 24 25 IEAAAA BWIAAA HHHHxx +5265 5982 1 1 5 5 65 265 1265 265 5265 130 131 NUAAAA CWIAAA OOOOxx +3819 5983 1 3 9 19 19 819 1819 3819 3819 38 39 XQAAAA DWIAAA VVVVxx +3648 5984 0 0 8 8 48 648 1648 3648 3648 96 97 IKAAAA EWIAAA AAAAxx +6306 5985 0 2 6 6 6 306 306 1306 6306 12 13 OIAAAA FWIAAA HHHHxx +2385 5986 1 1 5 5 85 385 385 2385 2385 170 171 TNAAAA GWIAAA OOOOxx +9084 5987 0 0 4 4 84 84 1084 4084 9084 168 169 KLAAAA HWIAAA VVVVxx +4499 5988 1 3 9 19 99 499 499 4499 4499 198 199 BRAAAA IWIAAA AAAAxx +1154 5989 0 2 4 14 54 154 1154 1154 1154 108 109 KSAAAA JWIAAA HHHHxx +6800 5990 0 0 0 0 0 800 800 1800 6800 0 1 OBAAAA KWIAAA OOOOxx +8049 5991 1 1 9 9 49 49 49 3049 8049 98 99 PXAAAA LWIAAA VVVVxx +3733 5992 1 1 3 13 33 733 1733 3733 3733 66 67 PNAAAA MWIAAA AAAAxx +8496 5993 0 0 6 16 96 496 496 3496 8496 192 193 UOAAAA NWIAAA HHHHxx +9952 5994 0 0 2 12 52 952 1952 4952 9952 104 105 USAAAA OWIAAA OOOOxx +9792 5995 0 0 2 12 92 792 1792 4792 9792 184 185 QMAAAA PWIAAA VVVVxx +5081 5996 1 1 1 1 81 81 1081 81 5081 162 163 LNAAAA QWIAAA AAAAxx +7908 5997 0 0 8 8 8 908 1908 2908 7908 16 17 ESAAAA RWIAAA HHHHxx +5398 5998 0 2 8 18 98 398 1398 398 5398 196 197 QZAAAA SWIAAA OOOOxx +8423 5999 1 3 3 3 23 423 423 3423 8423 46 47 ZLAAAA TWIAAA VVVVxx +3362 6000 0 2 2 2 62 362 1362 3362 3362 124 125 IZAAAA UWIAAA AAAAxx +7767 6001 1 3 7 7 67 767 1767 2767 7767 134 135 TMAAAA VWIAAA HHHHxx +7063 6002 1 3 3 3 63 63 1063 2063 7063 126 127 RLAAAA WWIAAA OOOOxx +8350 6003 0 2 0 10 50 350 350 3350 8350 100 101 EJAAAA XWIAAA VVVVxx +6779 6004 1 3 9 19 79 779 779 1779 6779 158 159 TAAAAA YWIAAA AAAAxx +5742 6005 0 2 2 2 42 742 1742 742 5742 84 85 WMAAAA ZWIAAA HHHHxx +9045 6006 1 1 5 5 45 45 1045 4045 9045 90 91 XJAAAA AXIAAA OOOOxx +8792 6007 0 0 2 12 92 792 792 3792 8792 184 185 EAAAAA BXIAAA VVVVxx +8160 6008 0 0 0 0 60 160 160 3160 8160 120 121 WBAAAA CXIAAA AAAAxx +3061 6009 1 1 1 1 61 61 1061 3061 3061 122 123 TNAAAA DXIAAA HHHHxx +4721 6010 1 1 1 1 21 721 721 4721 4721 42 43 PZAAAA EXIAAA OOOOxx +9817 6011 1 1 7 17 17 817 1817 4817 9817 34 35 PNAAAA FXIAAA VVVVxx +9257 6012 1 1 7 17 57 257 1257 4257 9257 114 115 BSAAAA GXIAAA AAAAxx +7779 6013 1 3 9 19 79 779 1779 2779 7779 158 159 FNAAAA HXIAAA HHHHxx +2663 6014 1 3 3 3 63 663 663 2663 2663 126 127 LYAAAA IXIAAA OOOOxx +3885 6015 1 1 5 5 85 885 1885 3885 3885 170 171 LTAAAA JXIAAA VVVVxx +9469 6016 1 1 9 9 69 469 1469 4469 9469 138 139 FAAAAA KXIAAA AAAAxx +6766 6017 0 2 6 6 66 766 766 1766 6766 132 133 GAAAAA LXIAAA HHHHxx +7173 6018 1 1 3 13 73 173 1173 2173 7173 146 147 XPAAAA MXIAAA OOOOxx +4709 6019 1 1 9 9 9 709 709 4709 4709 18 19 DZAAAA NXIAAA VVVVxx +4210 6020 0 2 0 10 10 210 210 4210 4210 20 21 YFAAAA OXIAAA AAAAxx +3715 6021 1 3 5 15 15 715 1715 3715 3715 30 31 XMAAAA PXIAAA HHHHxx +5089 6022 1 1 9 9 89 89 1089 89 5089 178 179 TNAAAA QXIAAA OOOOxx +1639 6023 1 3 9 19 39 639 1639 1639 1639 78 79 BLAAAA RXIAAA VVVVxx +5757 6024 1 1 7 17 57 757 1757 757 5757 114 115 LNAAAA SXIAAA AAAAxx +3545 6025 1 1 5 5 45 545 1545 3545 3545 90 91 JGAAAA TXIAAA HHHHxx +709 6026 1 1 9 9 9 709 709 709 709 18 19 HBAAAA UXIAAA OOOOxx +6519 6027 1 3 9 19 19 519 519 1519 6519 38 39 TQAAAA VXIAAA VVVVxx +4341 6028 1 1 1 1 41 341 341 4341 4341 82 83 ZKAAAA WXIAAA AAAAxx +2381 6029 1 1 1 1 81 381 381 2381 2381 162 163 PNAAAA XXIAAA HHHHxx +7215 6030 1 3 5 15 15 215 1215 2215 7215 30 31 NRAAAA YXIAAA OOOOxx +9323 6031 1 3 3 3 23 323 1323 4323 9323 46 47 PUAAAA ZXIAAA VVVVxx +3593 6032 1 1 3 13 93 593 1593 3593 3593 186 187 FIAAAA AYIAAA AAAAxx +3123 6033 1 3 3 3 23 123 1123 3123 3123 46 47 DQAAAA BYIAAA HHHHxx +8673 6034 1 1 3 13 73 673 673 3673 8673 146 147 PVAAAA CYIAAA OOOOxx +5094 6035 0 2 4 14 94 94 1094 94 5094 188 189 YNAAAA DYIAAA VVVVxx +6477 6036 1 1 7 17 77 477 477 1477 6477 154 155 DPAAAA EYIAAA AAAAxx +9734 6037 0 2 4 14 34 734 1734 4734 9734 68 69 KKAAAA FYIAAA HHHHxx +2998 6038 0 2 8 18 98 998 998 2998 2998 196 197 ILAAAA GYIAAA OOOOxx +7807 6039 1 3 7 7 7 807 1807 2807 7807 14 15 HOAAAA HYIAAA VVVVxx +5739 6040 1 3 9 19 39 739 1739 739 5739 78 79 TMAAAA IYIAAA AAAAxx +138 6041 0 2 8 18 38 138 138 138 138 76 77 IFAAAA JYIAAA HHHHxx +2403 6042 1 3 3 3 3 403 403 2403 2403 6 7 LOAAAA KYIAAA OOOOxx +2484 6043 0 0 4 4 84 484 484 2484 2484 168 169 ORAAAA LYIAAA VVVVxx +2805 6044 1 1 5 5 5 805 805 2805 2805 10 11 XDAAAA MYIAAA AAAAxx +5189 6045 1 1 9 9 89 189 1189 189 5189 178 179 PRAAAA NYIAAA HHHHxx +8336 6046 0 0 6 16 36 336 336 3336 8336 72 73 QIAAAA OYIAAA OOOOxx +5241 6047 1 1 1 1 41 241 1241 241 5241 82 83 PTAAAA PYIAAA VVVVxx +2612 6048 0 0 2 12 12 612 612 2612 2612 24 25 MWAAAA QYIAAA AAAAxx +2571 6049 1 3 1 11 71 571 571 2571 2571 142 143 XUAAAA RYIAAA HHHHxx +926 6050 0 2 6 6 26 926 926 926 926 52 53 QJAAAA SYIAAA OOOOxx +337 6051 1 1 7 17 37 337 337 337 337 74 75 ZMAAAA TYIAAA VVVVxx +2821 6052 1 1 1 1 21 821 821 2821 2821 42 43 NEAAAA UYIAAA AAAAxx +2658 6053 0 2 8 18 58 658 658 2658 2658 116 117 GYAAAA VYIAAA HHHHxx +9054 6054 0 2 4 14 54 54 1054 4054 9054 108 109 GKAAAA WYIAAA OOOOxx +5492 6055 0 0 2 12 92 492 1492 492 5492 184 185 GDAAAA XYIAAA VVVVxx +7313 6056 1 1 3 13 13 313 1313 2313 7313 26 27 HVAAAA YYIAAA AAAAxx +75 6057 1 3 5 15 75 75 75 75 75 150 151 XCAAAA ZYIAAA HHHHxx +5489 6058 1 1 9 9 89 489 1489 489 5489 178 179 DDAAAA AZIAAA OOOOxx +8413 6059 1 1 3 13 13 413 413 3413 8413 26 27 PLAAAA BZIAAA VVVVxx +3693 6060 1 1 3 13 93 693 1693 3693 3693 186 187 BMAAAA CZIAAA AAAAxx +9820 6061 0 0 0 0 20 820 1820 4820 9820 40 41 SNAAAA DZIAAA HHHHxx +8157 6062 1 1 7 17 57 157 157 3157 8157 114 115 TBAAAA EZIAAA OOOOxx +4161 6063 1 1 1 1 61 161 161 4161 4161 122 123 BEAAAA FZIAAA VVVVxx +8339 6064 1 3 9 19 39 339 339 3339 8339 78 79 TIAAAA GZIAAA AAAAxx +4141 6065 1 1 1 1 41 141 141 4141 4141 82 83 HDAAAA HZIAAA HHHHxx +9001 6066 1 1 1 1 1 1 1001 4001 9001 2 3 FIAAAA IZIAAA OOOOxx +8247 6067 1 3 7 7 47 247 247 3247 8247 94 95 FFAAAA JZIAAA VVVVxx +1182 6068 0 2 2 2 82 182 1182 1182 1182 164 165 MTAAAA KZIAAA AAAAxx +9876 6069 0 0 6 16 76 876 1876 4876 9876 152 153 WPAAAA LZIAAA HHHHxx +4302 6070 0 2 2 2 2 302 302 4302 4302 4 5 MJAAAA MZIAAA OOOOxx +6674 6071 0 2 4 14 74 674 674 1674 6674 148 149 SWAAAA NZIAAA VVVVxx +4214 6072 0 2 4 14 14 214 214 4214 4214 28 29 CGAAAA OZIAAA AAAAxx +5584 6073 0 0 4 4 84 584 1584 584 5584 168 169 UGAAAA PZIAAA HHHHxx +265 6074 1 1 5 5 65 265 265 265 265 130 131 FKAAAA QZIAAA OOOOxx +9207 6075 1 3 7 7 7 207 1207 4207 9207 14 15 DQAAAA RZIAAA VVVVxx +9434 6076 0 2 4 14 34 434 1434 4434 9434 68 69 WYAAAA SZIAAA AAAAxx +2921 6077 1 1 1 1 21 921 921 2921 2921 42 43 JIAAAA TZIAAA HHHHxx +9355 6078 1 3 5 15 55 355 1355 4355 9355 110 111 VVAAAA UZIAAA OOOOxx +8538 6079 0 2 8 18 38 538 538 3538 8538 76 77 KQAAAA VZIAAA VVVVxx +4559 6080 1 3 9 19 59 559 559 4559 4559 118 119 JTAAAA WZIAAA AAAAxx +9175 6081 1 3 5 15 75 175 1175 4175 9175 150 151 XOAAAA XZIAAA HHHHxx +4489 6082 1 1 9 9 89 489 489 4489 4489 178 179 RQAAAA YZIAAA OOOOxx +1485 6083 1 1 5 5 85 485 1485 1485 1485 170 171 DFAAAA ZZIAAA VVVVxx +8853 6084 1 1 3 13 53 853 853 3853 8853 106 107 NCAAAA AAJAAA AAAAxx +9143 6085 1 3 3 3 43 143 1143 4143 9143 86 87 RNAAAA BAJAAA HHHHxx +9551 6086 1 3 1 11 51 551 1551 4551 9551 102 103 JDAAAA CAJAAA OOOOxx +49 6087 1 1 9 9 49 49 49 49 49 98 99 XBAAAA DAJAAA VVVVxx +8351 6088 1 3 1 11 51 351 351 3351 8351 102 103 FJAAAA EAJAAA AAAAxx +9748 6089 0 0 8 8 48 748 1748 4748 9748 96 97 YKAAAA FAJAAA HHHHxx +4536 6090 0 0 6 16 36 536 536 4536 4536 72 73 MSAAAA GAJAAA OOOOxx +930 6091 0 2 0 10 30 930 930 930 930 60 61 UJAAAA HAJAAA VVVVxx +2206 6092 0 2 6 6 6 206 206 2206 2206 12 13 WGAAAA IAJAAA AAAAxx +8004 6093 0 0 4 4 4 4 4 3004 8004 8 9 WVAAAA JAJAAA HHHHxx +219 6094 1 3 9 19 19 219 219 219 219 38 39 LIAAAA KAJAAA OOOOxx +2724 6095 0 0 4 4 24 724 724 2724 2724 48 49 UAAAAA LAJAAA VVVVxx +4868 6096 0 0 8 8 68 868 868 4868 4868 136 137 GFAAAA MAJAAA AAAAxx +5952 6097 0 0 2 12 52 952 1952 952 5952 104 105 YUAAAA NAJAAA HHHHxx +2094 6098 0 2 4 14 94 94 94 2094 2094 188 189 OCAAAA OAJAAA OOOOxx +5707 6099 1 3 7 7 7 707 1707 707 5707 14 15 NLAAAA PAJAAA VVVVxx +5200 6100 0 0 0 0 0 200 1200 200 5200 0 1 ASAAAA QAJAAA AAAAxx +967 6101 1 3 7 7 67 967 967 967 967 134 135 FLAAAA RAJAAA HHHHxx +1982 6102 0 2 2 2 82 982 1982 1982 1982 164 165 GYAAAA SAJAAA OOOOxx +3410 6103 0 2 0 10 10 410 1410 3410 3410 20 21 EBAAAA TAJAAA VVVVxx +174 6104 0 2 4 14 74 174 174 174 174 148 149 SGAAAA UAJAAA AAAAxx +9217 6105 1 1 7 17 17 217 1217 4217 9217 34 35 NQAAAA VAJAAA HHHHxx +9103 6106 1 3 3 3 3 103 1103 4103 9103 6 7 DMAAAA WAJAAA OOOOxx +868 6107 0 0 8 8 68 868 868 868 868 136 137 KHAAAA XAJAAA VVVVxx +8261 6108 1 1 1 1 61 261 261 3261 8261 122 123 TFAAAA YAJAAA AAAAxx +2720 6109 0 0 0 0 20 720 720 2720 2720 40 41 QAAAAA ZAJAAA HHHHxx +2999 6110 1 3 9 19 99 999 999 2999 2999 198 199 JLAAAA ABJAAA OOOOxx +769 6111 1 1 9 9 69 769 769 769 769 138 139 PDAAAA BBJAAA VVVVxx +4533 6112 1 1 3 13 33 533 533 4533 4533 66 67 JSAAAA CBJAAA AAAAxx +2030 6113 0 2 0 10 30 30 30 2030 2030 60 61 CAAAAA DBJAAA HHHHxx +5824 6114 0 0 4 4 24 824 1824 824 5824 48 49 AQAAAA EBJAAA OOOOxx +2328 6115 0 0 8 8 28 328 328 2328 2328 56 57 OLAAAA FBJAAA VVVVxx +9970 6116 0 2 0 10 70 970 1970 4970 9970 140 141 MTAAAA GBJAAA AAAAxx +3192 6117 0 0 2 12 92 192 1192 3192 3192 184 185 USAAAA HBJAAA HHHHxx +3387 6118 1 3 7 7 87 387 1387 3387 3387 174 175 HAAAAA IBJAAA OOOOxx +1936 6119 0 0 6 16 36 936 1936 1936 1936 72 73 MWAAAA JBJAAA VVVVxx +6934 6120 0 2 4 14 34 934 934 1934 6934 68 69 SGAAAA KBJAAA AAAAxx +5615 6121 1 3 5 15 15 615 1615 615 5615 30 31 ZHAAAA LBJAAA HHHHxx +2241 6122 1 1 1 1 41 241 241 2241 2241 82 83 FIAAAA MBJAAA OOOOxx +1842 6123 0 2 2 2 42 842 1842 1842 1842 84 85 WSAAAA NBJAAA VVVVxx +8044 6124 0 0 4 4 44 44 44 3044 8044 88 89 KXAAAA OBJAAA AAAAxx +8902 6125 0 2 2 2 2 902 902 3902 8902 4 5 KEAAAA PBJAAA HHHHxx +4519 6126 1 3 9 19 19 519 519 4519 4519 38 39 VRAAAA QBJAAA OOOOxx +492 6127 0 0 2 12 92 492 492 492 492 184 185 YSAAAA RBJAAA VVVVxx +2694 6128 0 2 4 14 94 694 694 2694 2694 188 189 QZAAAA SBJAAA AAAAxx +5861 6129 1 1 1 1 61 861 1861 861 5861 122 123 LRAAAA TBJAAA HHHHxx +2104 6130 0 0 4 4 4 104 104 2104 2104 8 9 YCAAAA UBJAAA OOOOxx +5376 6131 0 0 6 16 76 376 1376 376 5376 152 153 UYAAAA VBJAAA VVVVxx +3147 6132 1 3 7 7 47 147 1147 3147 3147 94 95 BRAAAA WBJAAA AAAAxx +9880 6133 0 0 0 0 80 880 1880 4880 9880 160 161 AQAAAA XBJAAA HHHHxx +6171 6134 1 3 1 11 71 171 171 1171 6171 142 143 JDAAAA YBJAAA OOOOxx +1850 6135 0 2 0 10 50 850 1850 1850 1850 100 101 ETAAAA ZBJAAA VVVVxx +1775 6136 1 3 5 15 75 775 1775 1775 1775 150 151 HQAAAA ACJAAA AAAAxx +9261 6137 1 1 1 1 61 261 1261 4261 9261 122 123 FSAAAA BCJAAA HHHHxx +9648 6138 0 0 8 8 48 648 1648 4648 9648 96 97 CHAAAA CCJAAA OOOOxx +7846 6139 0 2 6 6 46 846 1846 2846 7846 92 93 UPAAAA DCJAAA VVVVxx +1446 6140 0 2 6 6 46 446 1446 1446 1446 92 93 QDAAAA ECJAAA AAAAxx +3139 6141 1 3 9 19 39 139 1139 3139 3139 78 79 TQAAAA FCJAAA HHHHxx +6142 6142 0 2 2 2 42 142 142 1142 6142 84 85 GCAAAA GCJAAA OOOOxx +5812 6143 0 0 2 12 12 812 1812 812 5812 24 25 OPAAAA HCJAAA VVVVxx +6728 6144 0 0 8 8 28 728 728 1728 6728 56 57 UYAAAA ICJAAA AAAAxx +4428 6145 0 0 8 8 28 428 428 4428 4428 56 57 IOAAAA JCJAAA HHHHxx +502 6146 0 2 2 2 2 502 502 502 502 4 5 ITAAAA KCJAAA OOOOxx +2363 6147 1 3 3 3 63 363 363 2363 2363 126 127 XMAAAA LCJAAA VVVVxx +3808 6148 0 0 8 8 8 808 1808 3808 3808 16 17 MQAAAA MCJAAA AAAAxx +1010 6149 0 2 0 10 10 10 1010 1010 1010 20 21 WMAAAA NCJAAA HHHHxx +9565 6150 1 1 5 5 65 565 1565 4565 9565 130 131 XDAAAA OCJAAA OOOOxx +1587 6151 1 3 7 7 87 587 1587 1587 1587 174 175 BJAAAA PCJAAA VVVVxx +1474 6152 0 2 4 14 74 474 1474 1474 1474 148 149 SEAAAA QCJAAA AAAAxx +6215 6153 1 3 5 15 15 215 215 1215 6215 30 31 BFAAAA RCJAAA HHHHxx +2395 6154 1 3 5 15 95 395 395 2395 2395 190 191 DOAAAA SCJAAA OOOOxx +8753 6155 1 1 3 13 53 753 753 3753 8753 106 107 RYAAAA TCJAAA VVVVxx +2446 6156 0 2 6 6 46 446 446 2446 2446 92 93 CQAAAA UCJAAA AAAAxx +60 6157 0 0 0 0 60 60 60 60 60 120 121 ICAAAA VCJAAA HHHHxx +982 6158 0 2 2 2 82 982 982 982 982 164 165 ULAAAA WCJAAA OOOOxx +6489 6159 1 1 9 9 89 489 489 1489 6489 178 179 PPAAAA XCJAAA VVVVxx +5334 6160 0 2 4 14 34 334 1334 334 5334 68 69 EXAAAA YCJAAA AAAAxx +8540 6161 0 0 0 0 40 540 540 3540 8540 80 81 MQAAAA ZCJAAA HHHHxx +490 6162 0 2 0 10 90 490 490 490 490 180 181 WSAAAA ADJAAA OOOOxx +6763 6163 1 3 3 3 63 763 763 1763 6763 126 127 DAAAAA BDJAAA VVVVxx +8273 6164 1 1 3 13 73 273 273 3273 8273 146 147 FGAAAA CDJAAA AAAAxx +8327 6165 1 3 7 7 27 327 327 3327 8327 54 55 HIAAAA DDJAAA HHHHxx +8541 6166 1 1 1 1 41 541 541 3541 8541 82 83 NQAAAA EDJAAA OOOOxx +3459 6167 1 3 9 19 59 459 1459 3459 3459 118 119 BDAAAA FDJAAA VVVVxx +5557 6168 1 1 7 17 57 557 1557 557 5557 114 115 TFAAAA GDJAAA AAAAxx +158 6169 0 2 8 18 58 158 158 158 158 116 117 CGAAAA HDJAAA HHHHxx +1741 6170 1 1 1 1 41 741 1741 1741 1741 82 83 ZOAAAA IDJAAA OOOOxx +8385 6171 1 1 5 5 85 385 385 3385 8385 170 171 NKAAAA JDJAAA VVVVxx +617 6172 1 1 7 17 17 617 617 617 617 34 35 TXAAAA KDJAAA AAAAxx +3560 6173 0 0 0 0 60 560 1560 3560 3560 120 121 YGAAAA LDJAAA HHHHxx +5216 6174 0 0 6 16 16 216 1216 216 5216 32 33 QSAAAA MDJAAA OOOOxx +8443 6175 1 3 3 3 43 443 443 3443 8443 86 87 TMAAAA NDJAAA VVVVxx +2700 6176 0 0 0 0 0 700 700 2700 2700 0 1 WZAAAA ODJAAA AAAAxx +3661 6177 1 1 1 1 61 661 1661 3661 3661 122 123 VKAAAA PDJAAA HHHHxx +4875 6178 1 3 5 15 75 875 875 4875 4875 150 151 NFAAAA QDJAAA OOOOxx +6721 6179 1 1 1 1 21 721 721 1721 6721 42 43 NYAAAA RDJAAA VVVVxx +3659 6180 1 3 9 19 59 659 1659 3659 3659 118 119 TKAAAA SDJAAA AAAAxx +8944 6181 0 0 4 4 44 944 944 3944 8944 88 89 AGAAAA TDJAAA HHHHxx +9133 6182 1 1 3 13 33 133 1133 4133 9133 66 67 HNAAAA UDJAAA OOOOxx +9882 6183 0 2 2 2 82 882 1882 4882 9882 164 165 CQAAAA VDJAAA VVVVxx +2102 6184 0 2 2 2 2 102 102 2102 2102 4 5 WCAAAA WDJAAA AAAAxx +9445 6185 1 1 5 5 45 445 1445 4445 9445 90 91 HZAAAA XDJAAA HHHHxx +5559 6186 1 3 9 19 59 559 1559 559 5559 118 119 VFAAAA YDJAAA OOOOxx +6096 6187 0 0 6 16 96 96 96 1096 6096 192 193 MAAAAA ZDJAAA VVVVxx +9336 6188 0 0 6 16 36 336 1336 4336 9336 72 73 CVAAAA AEJAAA AAAAxx +2162 6189 0 2 2 2 62 162 162 2162 2162 124 125 EFAAAA BEJAAA HHHHxx +7459 6190 1 3 9 19 59 459 1459 2459 7459 118 119 XAAAAA CEJAAA OOOOxx +3248 6191 0 0 8 8 48 248 1248 3248 3248 96 97 YUAAAA DEJAAA VVVVxx +9539 6192 1 3 9 19 39 539 1539 4539 9539 78 79 XCAAAA EEJAAA AAAAxx +4449 6193 1 1 9 9 49 449 449 4449 4449 98 99 DPAAAA FEJAAA HHHHxx +2809 6194 1 1 9 9 9 809 809 2809 2809 18 19 BEAAAA GEJAAA OOOOxx +7058 6195 0 2 8 18 58 58 1058 2058 7058 116 117 MLAAAA HEJAAA VVVVxx +3512 6196 0 0 2 12 12 512 1512 3512 3512 24 25 CFAAAA IEJAAA AAAAxx +2802 6197 0 2 2 2 2 802 802 2802 2802 4 5 UDAAAA JEJAAA HHHHxx +6289 6198 1 1 9 9 89 289 289 1289 6289 178 179 XHAAAA KEJAAA OOOOxx +1947 6199 1 3 7 7 47 947 1947 1947 1947 94 95 XWAAAA LEJAAA VVVVxx +9572 6200 0 0 2 12 72 572 1572 4572 9572 144 145 EEAAAA MEJAAA AAAAxx +2356 6201 0 0 6 16 56 356 356 2356 2356 112 113 QMAAAA NEJAAA HHHHxx +3039 6202 1 3 9 19 39 39 1039 3039 3039 78 79 XMAAAA OEJAAA OOOOxx +9452 6203 0 0 2 12 52 452 1452 4452 9452 104 105 OZAAAA PEJAAA VVVVxx +6328 6204 0 0 8 8 28 328 328 1328 6328 56 57 KJAAAA QEJAAA AAAAxx +7661 6205 1 1 1 1 61 661 1661 2661 7661 122 123 RIAAAA REJAAA HHHHxx +2566 6206 0 2 6 6 66 566 566 2566 2566 132 133 SUAAAA SEJAAA OOOOxx +6095 6207 1 3 5 15 95 95 95 1095 6095 190 191 LAAAAA TEJAAA VVVVxx +6367 6208 1 3 7 7 67 367 367 1367 6367 134 135 XKAAAA UEJAAA AAAAxx +3368 6209 0 0 8 8 68 368 1368 3368 3368 136 137 OZAAAA VEJAAA HHHHxx +5567 6210 1 3 7 7 67 567 1567 567 5567 134 135 DGAAAA WEJAAA OOOOxx +9834 6211 0 2 4 14 34 834 1834 4834 9834 68 69 GOAAAA XEJAAA VVVVxx +9695 6212 1 3 5 15 95 695 1695 4695 9695 190 191 XIAAAA YEJAAA AAAAxx +7291 6213 1 3 1 11 91 291 1291 2291 7291 182 183 LUAAAA ZEJAAA HHHHxx +4806 6214 0 2 6 6 6 806 806 4806 4806 12 13 WCAAAA AFJAAA OOOOxx +2000 6215 0 0 0 0 0 0 0 2000 2000 0 1 YYAAAA BFJAAA VVVVxx +6817 6216 1 1 7 17 17 817 817 1817 6817 34 35 FCAAAA CFJAAA AAAAxx +8487 6217 1 3 7 7 87 487 487 3487 8487 174 175 LOAAAA DFJAAA HHHHxx +3245 6218 1 1 5 5 45 245 1245 3245 3245 90 91 VUAAAA EFJAAA OOOOxx +632 6219 0 0 2 12 32 632 632 632 632 64 65 IYAAAA FFJAAA VVVVxx +8067 6220 1 3 7 7 67 67 67 3067 8067 134 135 HYAAAA GFJAAA AAAAxx +7140 6221 0 0 0 0 40 140 1140 2140 7140 80 81 QOAAAA HFJAAA HHHHxx +6802 6222 0 2 2 2 2 802 802 1802 6802 4 5 QBAAAA IFJAAA OOOOxx +3980 6223 0 0 0 0 80 980 1980 3980 3980 160 161 CXAAAA JFJAAA VVVVxx +1321 6224 1 1 1 1 21 321 1321 1321 1321 42 43 VYAAAA KFJAAA AAAAxx +2273 6225 1 1 3 13 73 273 273 2273 2273 146 147 LJAAAA LFJAAA HHHHxx +6787 6226 1 3 7 7 87 787 787 1787 6787 174 175 BBAAAA MFJAAA OOOOxx +9480 6227 0 0 0 0 80 480 1480 4480 9480 160 161 QAAAAA NFJAAA VVVVxx +9404 6228 0 0 4 4 4 404 1404 4404 9404 8 9 SXAAAA OFJAAA AAAAxx +3914 6229 0 2 4 14 14 914 1914 3914 3914 28 29 OUAAAA PFJAAA HHHHxx +5507 6230 1 3 7 7 7 507 1507 507 5507 14 15 VDAAAA QFJAAA OOOOxx +1813 6231 1 1 3 13 13 813 1813 1813 1813 26 27 TRAAAA RFJAAA VVVVxx +1999 6232 1 3 9 19 99 999 1999 1999 1999 198 199 XYAAAA SFJAAA AAAAxx +3848 6233 0 0 8 8 48 848 1848 3848 3848 96 97 ASAAAA TFJAAA HHHHxx +9693 6234 1 1 3 13 93 693 1693 4693 9693 186 187 VIAAAA UFJAAA OOOOxx +1353 6235 1 1 3 13 53 353 1353 1353 1353 106 107 BAAAAA VFJAAA VVVVxx +7218 6236 0 2 8 18 18 218 1218 2218 7218 36 37 QRAAAA WFJAAA AAAAxx +8223 6237 1 3 3 3 23 223 223 3223 8223 46 47 HEAAAA XFJAAA HHHHxx +9982 6238 0 2 2 2 82 982 1982 4982 9982 164 165 YTAAAA YFJAAA OOOOxx +8799 6239 1 3 9 19 99 799 799 3799 8799 198 199 LAAAAA ZFJAAA VVVVxx +8929 6240 1 1 9 9 29 929 929 3929 8929 58 59 LFAAAA AGJAAA AAAAxx +4626 6241 0 2 6 6 26 626 626 4626 4626 52 53 YVAAAA BGJAAA HHHHxx +7958 6242 0 2 8 18 58 958 1958 2958 7958 116 117 CUAAAA CGJAAA OOOOxx +3743 6243 1 3 3 3 43 743 1743 3743 3743 86 87 ZNAAAA DGJAAA VVVVxx +8165 6244 1 1 5 5 65 165 165 3165 8165 130 131 BCAAAA EGJAAA AAAAxx +7899 6245 1 3 9 19 99 899 1899 2899 7899 198 199 VRAAAA FGJAAA HHHHxx +8698 6246 0 2 8 18 98 698 698 3698 8698 196 197 OWAAAA GGJAAA OOOOxx +9270 6247 0 2 0 10 70 270 1270 4270 9270 140 141 OSAAAA HGJAAA VVVVxx +6348 6248 0 0 8 8 48 348 348 1348 6348 96 97 EKAAAA IGJAAA AAAAxx +6999 6249 1 3 9 19 99 999 999 1999 6999 198 199 FJAAAA JGJAAA HHHHxx +8467 6250 1 3 7 7 67 467 467 3467 8467 134 135 RNAAAA KGJAAA OOOOxx +3907 6251 1 3 7 7 7 907 1907 3907 3907 14 15 HUAAAA LGJAAA VVVVxx +4738 6252 0 2 8 18 38 738 738 4738 4738 76 77 GAAAAA MGJAAA AAAAxx +248 6253 0 0 8 8 48 248 248 248 248 96 97 OJAAAA NGJAAA HHHHxx +8769 6254 1 1 9 9 69 769 769 3769 8769 138 139 HZAAAA OGJAAA OOOOxx +9922 6255 0 2 2 2 22 922 1922 4922 9922 44 45 QRAAAA PGJAAA VVVVxx +778 6256 0 2 8 18 78 778 778 778 778 156 157 YDAAAA QGJAAA AAAAxx +1233 6257 1 1 3 13 33 233 1233 1233 1233 66 67 LVAAAA RGJAAA HHHHxx +1183 6258 1 3 3 3 83 183 1183 1183 1183 166 167 NTAAAA SGJAAA OOOOxx +2838 6259 0 2 8 18 38 838 838 2838 2838 76 77 EFAAAA TGJAAA VVVVxx +3096 6260 0 0 6 16 96 96 1096 3096 3096 192 193 CPAAAA UGJAAA AAAAxx +8566 6261 0 2 6 6 66 566 566 3566 8566 132 133 MRAAAA VGJAAA HHHHxx +7635 6262 1 3 5 15 35 635 1635 2635 7635 70 71 RHAAAA WGJAAA OOOOxx +5428 6263 0 0 8 8 28 428 1428 428 5428 56 57 UAAAAA XGJAAA VVVVxx +7430 6264 0 2 0 10 30 430 1430 2430 7430 60 61 UZAAAA YGJAAA AAAAxx +7210 6265 0 2 0 10 10 210 1210 2210 7210 20 21 IRAAAA ZGJAAA HHHHxx +4485 6266 1 1 5 5 85 485 485 4485 4485 170 171 NQAAAA AHJAAA OOOOxx +9623 6267 1 3 3 3 23 623 1623 4623 9623 46 47 DGAAAA BHJAAA VVVVxx +3670 6268 0 2 0 10 70 670 1670 3670 3670 140 141 ELAAAA CHJAAA AAAAxx +1575 6269 1 3 5 15 75 575 1575 1575 1575 150 151 PIAAAA DHJAAA HHHHxx +5874 6270 0 2 4 14 74 874 1874 874 5874 148 149 YRAAAA EHJAAA OOOOxx +673 6271 1 1 3 13 73 673 673 673 673 146 147 XZAAAA FHJAAA VVVVxx +9712 6272 0 0 2 12 12 712 1712 4712 9712 24 25 OJAAAA GHJAAA AAAAxx +7729 6273 1 1 9 9 29 729 1729 2729 7729 58 59 HLAAAA HHJAAA HHHHxx +4318 6274 0 2 8 18 18 318 318 4318 4318 36 37 CKAAAA IHJAAA OOOOxx +4143 6275 1 3 3 3 43 143 143 4143 4143 86 87 JDAAAA JHJAAA VVVVxx +4932 6276 0 0 2 12 32 932 932 4932 4932 64 65 SHAAAA KHJAAA AAAAxx +5835 6277 1 3 5 15 35 835 1835 835 5835 70 71 LQAAAA LHJAAA HHHHxx +4966 6278 0 2 6 6 66 966 966 4966 4966 132 133 AJAAAA MHJAAA OOOOxx +6711 6279 1 3 1 11 11 711 711 1711 6711 22 23 DYAAAA NHJAAA VVVVxx +3990 6280 0 2 0 10 90 990 1990 3990 3990 180 181 MXAAAA OHJAAA AAAAxx +990 6281 0 2 0 10 90 990 990 990 990 180 181 CMAAAA PHJAAA HHHHxx +220 6282 0 0 0 0 20 220 220 220 220 40 41 MIAAAA QHJAAA OOOOxx +5693 6283 1 1 3 13 93 693 1693 693 5693 186 187 ZKAAAA RHJAAA VVVVxx +3662 6284 0 2 2 2 62 662 1662 3662 3662 124 125 WKAAAA SHJAAA AAAAxx +7844 6285 0 0 4 4 44 844 1844 2844 7844 88 89 SPAAAA THJAAA HHHHxx +5515 6286 1 3 5 15 15 515 1515 515 5515 30 31 DEAAAA UHJAAA OOOOxx +5551 6287 1 3 1 11 51 551 1551 551 5551 102 103 NFAAAA VHJAAA VVVVxx +2358 6288 0 2 8 18 58 358 358 2358 2358 116 117 SMAAAA WHJAAA AAAAxx +8977 6289 1 1 7 17 77 977 977 3977 8977 154 155 HHAAAA XHJAAA HHHHxx +7040 6290 0 0 0 0 40 40 1040 2040 7040 80 81 UKAAAA YHJAAA OOOOxx +105 6291 1 1 5 5 5 105 105 105 105 10 11 BEAAAA ZHJAAA VVVVxx +4496 6292 0 0 6 16 96 496 496 4496 4496 192 193 YQAAAA AIJAAA AAAAxx +2254 6293 0 2 4 14 54 254 254 2254 2254 108 109 SIAAAA BIJAAA HHHHxx +411 6294 1 3 1 11 11 411 411 411 411 22 23 VPAAAA CIJAAA OOOOxx +2373 6295 1 1 3 13 73 373 373 2373 2373 146 147 HNAAAA DIJAAA VVVVxx +3477 6296 1 1 7 17 77 477 1477 3477 3477 154 155 TDAAAA EIJAAA AAAAxx +8964 6297 0 0 4 4 64 964 964 3964 8964 128 129 UGAAAA FIJAAA HHHHxx +8471 6298 1 3 1 11 71 471 471 3471 8471 142 143 VNAAAA GIJAAA OOOOxx +5776 6299 0 0 6 16 76 776 1776 776 5776 152 153 EOAAAA HIJAAA VVVVxx +9921 6300 1 1 1 1 21 921 1921 4921 9921 42 43 PRAAAA IIJAAA AAAAxx +7816 6301 0 0 6 16 16 816 1816 2816 7816 32 33 QOAAAA JIJAAA HHHHxx +2439 6302 1 3 9 19 39 439 439 2439 2439 78 79 VPAAAA KIJAAA OOOOxx +9298 6303 0 2 8 18 98 298 1298 4298 9298 196 197 QTAAAA LIJAAA VVVVxx +9424 6304 0 0 4 4 24 424 1424 4424 9424 48 49 MYAAAA MIJAAA AAAAxx +3252 6305 0 0 2 12 52 252 1252 3252 3252 104 105 CVAAAA NIJAAA HHHHxx +1401 6306 1 1 1 1 1 401 1401 1401 1401 2 3 XBAAAA OIJAAA OOOOxx +9632 6307 0 0 2 12 32 632 1632 4632 9632 64 65 MGAAAA PIJAAA VVVVxx +370 6308 0 2 0 10 70 370 370 370 370 140 141 GOAAAA QIJAAA AAAAxx +728 6309 0 0 8 8 28 728 728 728 728 56 57 ACAAAA RIJAAA HHHHxx +2888 6310 0 0 8 8 88 888 888 2888 2888 176 177 CHAAAA SIJAAA OOOOxx +1441 6311 1 1 1 1 41 441 1441 1441 1441 82 83 LDAAAA TIJAAA VVVVxx +8308 6312 0 0 8 8 8 308 308 3308 8308 16 17 OHAAAA UIJAAA AAAAxx +2165 6313 1 1 5 5 65 165 165 2165 2165 130 131 HFAAAA VIJAAA HHHHxx +6359 6314 1 3 9 19 59 359 359 1359 6359 118 119 PKAAAA WIJAAA OOOOxx +9637 6315 1 1 7 17 37 637 1637 4637 9637 74 75 RGAAAA XIJAAA VVVVxx +5208 6316 0 0 8 8 8 208 1208 208 5208 16 17 ISAAAA YIJAAA AAAAxx +4705 6317 1 1 5 5 5 705 705 4705 4705 10 11 ZYAAAA ZIJAAA HHHHxx +2341 6318 1 1 1 1 41 341 341 2341 2341 82 83 BMAAAA AJJAAA OOOOxx +8539 6319 1 3 9 19 39 539 539 3539 8539 78 79 LQAAAA BJJAAA VVVVxx +7528 6320 0 0 8 8 28 528 1528 2528 7528 56 57 ODAAAA CJJAAA AAAAxx +7969 6321 1 1 9 9 69 969 1969 2969 7969 138 139 NUAAAA DJJAAA HHHHxx +6381 6322 1 1 1 1 81 381 381 1381 6381 162 163 LLAAAA EJJAAA OOOOxx +4906 6323 0 2 6 6 6 906 906 4906 4906 12 13 SGAAAA FJJAAA VVVVxx +8697 6324 1 1 7 17 97 697 697 3697 8697 194 195 NWAAAA GJJAAA AAAAxx +6301 6325 1 1 1 1 1 301 301 1301 6301 2 3 JIAAAA HJJAAA HHHHxx +7554 6326 0 2 4 14 54 554 1554 2554 7554 108 109 OEAAAA IJJAAA OOOOxx +5107 6327 1 3 7 7 7 107 1107 107 5107 14 15 LOAAAA JJJAAA VVVVxx +5046 6328 0 2 6 6 46 46 1046 46 5046 92 93 CMAAAA KJJAAA AAAAxx +4063 6329 1 3 3 3 63 63 63 4063 4063 126 127 HAAAAA LJJAAA HHHHxx +7580 6330 0 0 0 0 80 580 1580 2580 7580 160 161 OFAAAA MJJAAA OOOOxx +2245 6331 1 1 5 5 45 245 245 2245 2245 90 91 JIAAAA NJJAAA VVVVxx +3711 6332 1 3 1 11 11 711 1711 3711 3711 22 23 TMAAAA OJJAAA AAAAxx +3220 6333 0 0 0 0 20 220 1220 3220 3220 40 41 WTAAAA PJJAAA HHHHxx +6463 6334 1 3 3 3 63 463 463 1463 6463 126 127 POAAAA QJJAAA OOOOxx +8196 6335 0 0 6 16 96 196 196 3196 8196 192 193 GDAAAA RJJAAA VVVVxx +9875 6336 1 3 5 15 75 875 1875 4875 9875 150 151 VPAAAA SJJAAA AAAAxx +1333 6337 1 1 3 13 33 333 1333 1333 1333 66 67 HZAAAA TJJAAA HHHHxx +7880 6338 0 0 0 0 80 880 1880 2880 7880 160 161 CRAAAA UJJAAA OOOOxx +2322 6339 0 2 2 2 22 322 322 2322 2322 44 45 ILAAAA VJJAAA VVVVxx +2163 6340 1 3 3 3 63 163 163 2163 2163 126 127 FFAAAA WJJAAA AAAAxx +421 6341 1 1 1 1 21 421 421 421 421 42 43 FQAAAA XJJAAA HHHHxx +2042 6342 0 2 2 2 42 42 42 2042 2042 84 85 OAAAAA YJJAAA OOOOxx +1424 6343 0 0 4 4 24 424 1424 1424 1424 48 49 UCAAAA ZJJAAA VVVVxx +7870 6344 0 2 0 10 70 870 1870 2870 7870 140 141 SQAAAA AKJAAA AAAAxx +2653 6345 1 1 3 13 53 653 653 2653 2653 106 107 BYAAAA BKJAAA HHHHxx +4216 6346 0 0 6 16 16 216 216 4216 4216 32 33 EGAAAA CKJAAA OOOOxx +1515 6347 1 3 5 15 15 515 1515 1515 1515 30 31 HGAAAA DKJAAA VVVVxx +7860 6348 0 0 0 0 60 860 1860 2860 7860 120 121 IQAAAA EKJAAA AAAAxx +2984 6349 0 0 4 4 84 984 984 2984 2984 168 169 UKAAAA FKJAAA HHHHxx +6269 6350 1 1 9 9 69 269 269 1269 6269 138 139 DHAAAA GKJAAA OOOOxx +2609 6351 1 1 9 9 9 609 609 2609 2609 18 19 JWAAAA HKJAAA VVVVxx +3671 6352 1 3 1 11 71 671 1671 3671 3671 142 143 FLAAAA IKJAAA AAAAxx +4544 6353 0 0 4 4 44 544 544 4544 4544 88 89 USAAAA JKJAAA HHHHxx +4668 6354 0 0 8 8 68 668 668 4668 4668 136 137 OXAAAA KKJAAA OOOOxx +2565 6355 1 1 5 5 65 565 565 2565 2565 130 131 RUAAAA LKJAAA VVVVxx +3126 6356 0 2 6 6 26 126 1126 3126 3126 52 53 GQAAAA MKJAAA AAAAxx +7573 6357 1 1 3 13 73 573 1573 2573 7573 146 147 HFAAAA NKJAAA HHHHxx +1476 6358 0 0 6 16 76 476 1476 1476 1476 152 153 UEAAAA OKJAAA OOOOxx +2146 6359 0 2 6 6 46 146 146 2146 2146 92 93 OEAAAA PKJAAA VVVVxx +9990 6360 0 2 0 10 90 990 1990 4990 9990 180 181 GUAAAA QKJAAA AAAAxx +2530 6361 0 2 0 10 30 530 530 2530 2530 60 61 ITAAAA RKJAAA HHHHxx +9288 6362 0 0 8 8 88 288 1288 4288 9288 176 177 GTAAAA SKJAAA OOOOxx +9755 6363 1 3 5 15 55 755 1755 4755 9755 110 111 FLAAAA TKJAAA VVVVxx +5305 6364 1 1 5 5 5 305 1305 305 5305 10 11 BWAAAA UKJAAA AAAAxx +2495 6365 1 3 5 15 95 495 495 2495 2495 190 191 ZRAAAA VKJAAA HHHHxx +5443 6366 1 3 3 3 43 443 1443 443 5443 86 87 JBAAAA WKJAAA OOOOxx +1930 6367 0 2 0 10 30 930 1930 1930 1930 60 61 GWAAAA XKJAAA VVVVxx +9134 6368 0 2 4 14 34 134 1134 4134 9134 68 69 INAAAA YKJAAA AAAAxx +2844 6369 0 0 4 4 44 844 844 2844 2844 88 89 KFAAAA ZKJAAA HHHHxx +896 6370 0 0 6 16 96 896 896 896 896 192 193 MIAAAA ALJAAA OOOOxx +1330 6371 0 2 0 10 30 330 1330 1330 1330 60 61 EZAAAA BLJAAA VVVVxx +8980 6372 0 0 0 0 80 980 980 3980 8980 160 161 KHAAAA CLJAAA AAAAxx +5940 6373 0 0 0 0 40 940 1940 940 5940 80 81 MUAAAA DLJAAA HHHHxx +6494 6374 0 2 4 14 94 494 494 1494 6494 188 189 UPAAAA ELJAAA OOOOxx +165 6375 1 1 5 5 65 165 165 165 165 130 131 JGAAAA FLJAAA VVVVxx +2510 6376 0 2 0 10 10 510 510 2510 2510 20 21 OSAAAA GLJAAA AAAAxx +9950 6377 0 2 0 10 50 950 1950 4950 9950 100 101 SSAAAA HLJAAA HHHHxx +3854 6378 0 2 4 14 54 854 1854 3854 3854 108 109 GSAAAA ILJAAA OOOOxx +7493 6379 1 1 3 13 93 493 1493 2493 7493 186 187 FCAAAA JLJAAA VVVVxx +4124 6380 0 0 4 4 24 124 124 4124 4124 48 49 QCAAAA KLJAAA AAAAxx +8563 6381 1 3 3 3 63 563 563 3563 8563 126 127 JRAAAA LLJAAA HHHHxx +8735 6382 1 3 5 15 35 735 735 3735 8735 70 71 ZXAAAA MLJAAA OOOOxx +9046 6383 0 2 6 6 46 46 1046 4046 9046 92 93 YJAAAA NLJAAA VVVVxx +1754 6384 0 2 4 14 54 754 1754 1754 1754 108 109 MPAAAA OLJAAA AAAAxx +6954 6385 0 2 4 14 54 954 954 1954 6954 108 109 MHAAAA PLJAAA HHHHxx +4953 6386 1 1 3 13 53 953 953 4953 4953 106 107 NIAAAA QLJAAA OOOOxx +8142 6387 0 2 2 2 42 142 142 3142 8142 84 85 EBAAAA RLJAAA VVVVxx +9661 6388 1 1 1 1 61 661 1661 4661 9661 122 123 PHAAAA SLJAAA AAAAxx +6415 6389 1 3 5 15 15 415 415 1415 6415 30 31 TMAAAA TLJAAA HHHHxx +5782 6390 0 2 2 2 82 782 1782 782 5782 164 165 KOAAAA ULJAAA OOOOxx +7721 6391 1 1 1 1 21 721 1721 2721 7721 42 43 ZKAAAA VLJAAA VVVVxx +580 6392 0 0 0 0 80 580 580 580 580 160 161 IWAAAA WLJAAA AAAAxx +3784 6393 0 0 4 4 84 784 1784 3784 3784 168 169 OPAAAA XLJAAA HHHHxx +9810 6394 0 2 0 10 10 810 1810 4810 9810 20 21 INAAAA YLJAAA OOOOxx +8488 6395 0 0 8 8 88 488 488 3488 8488 176 177 MOAAAA ZLJAAA VVVVxx +6214 6396 0 2 4 14 14 214 214 1214 6214 28 29 AFAAAA AMJAAA AAAAxx +9433 6397 1 1 3 13 33 433 1433 4433 9433 66 67 VYAAAA BMJAAA HHHHxx +9959 6398 1 3 9 19 59 959 1959 4959 9959 118 119 BTAAAA CMJAAA OOOOxx +554 6399 0 2 4 14 54 554 554 554 554 108 109 IVAAAA DMJAAA VVVVxx +6646 6400 0 2 6 6 46 646 646 1646 6646 92 93 QVAAAA EMJAAA AAAAxx +1138 6401 0 2 8 18 38 138 1138 1138 1138 76 77 URAAAA FMJAAA HHHHxx +9331 6402 1 3 1 11 31 331 1331 4331 9331 62 63 XUAAAA GMJAAA OOOOxx +7331 6403 1 3 1 11 31 331 1331 2331 7331 62 63 ZVAAAA HMJAAA VVVVxx +3482 6404 0 2 2 2 82 482 1482 3482 3482 164 165 YDAAAA IMJAAA AAAAxx +3795 6405 1 3 5 15 95 795 1795 3795 3795 190 191 ZPAAAA JMJAAA HHHHxx +2441 6406 1 1 1 1 41 441 441 2441 2441 82 83 XPAAAA KMJAAA OOOOxx +5229 6407 1 1 9 9 29 229 1229 229 5229 58 59 DTAAAA LMJAAA VVVVxx +7012 6408 0 0 2 12 12 12 1012 2012 7012 24 25 SJAAAA MMJAAA AAAAxx +7036 6409 0 0 6 16 36 36 1036 2036 7036 72 73 QKAAAA NMJAAA HHHHxx +8243 6410 1 3 3 3 43 243 243 3243 8243 86 87 BFAAAA OMJAAA OOOOxx +9320 6411 0 0 0 0 20 320 1320 4320 9320 40 41 MUAAAA PMJAAA VVVVxx +4693 6412 1 1 3 13 93 693 693 4693 4693 186 187 NYAAAA QMJAAA AAAAxx +6741 6413 1 1 1 1 41 741 741 1741 6741 82 83 HZAAAA RMJAAA HHHHxx +2997 6414 1 1 7 17 97 997 997 2997 2997 194 195 HLAAAA SMJAAA OOOOxx +4838 6415 0 2 8 18 38 838 838 4838 4838 76 77 CEAAAA TMJAAA VVVVxx +6945 6416 1 1 5 5 45 945 945 1945 6945 90 91 DHAAAA UMJAAA AAAAxx +8253 6417 1 1 3 13 53 253 253 3253 8253 106 107 LFAAAA VMJAAA HHHHxx +8989 6418 1 1 9 9 89 989 989 3989 8989 178 179 THAAAA WMJAAA OOOOxx +2640 6419 0 0 0 0 40 640 640 2640 2640 80 81 OXAAAA XMJAAA VVVVxx +5647 6420 1 3 7 7 47 647 1647 647 5647 94 95 FJAAAA YMJAAA AAAAxx +7186 6421 0 2 6 6 86 186 1186 2186 7186 172 173 KQAAAA ZMJAAA HHHHxx +3278 6422 0 2 8 18 78 278 1278 3278 3278 156 157 CWAAAA ANJAAA OOOOxx +8546 6423 0 2 6 6 46 546 546 3546 8546 92 93 SQAAAA BNJAAA VVVVxx +8297 6424 1 1 7 17 97 297 297 3297 8297 194 195 DHAAAA CNJAAA AAAAxx +9534 6425 0 2 4 14 34 534 1534 4534 9534 68 69 SCAAAA DNJAAA HHHHxx +9618 6426 0 2 8 18 18 618 1618 4618 9618 36 37 YFAAAA ENJAAA OOOOxx +8839 6427 1 3 9 19 39 839 839 3839 8839 78 79 ZBAAAA FNJAAA VVVVxx +7605 6428 1 1 5 5 5 605 1605 2605 7605 10 11 NGAAAA GNJAAA AAAAxx +6421 6429 1 1 1 1 21 421 421 1421 6421 42 43 ZMAAAA HNJAAA HHHHxx +3582 6430 0 2 2 2 82 582 1582 3582 3582 164 165 UHAAAA INJAAA OOOOxx +485 6431 1 1 5 5 85 485 485 485 485 170 171 RSAAAA JNJAAA VVVVxx +1925 6432 1 1 5 5 25 925 1925 1925 1925 50 51 BWAAAA KNJAAA AAAAxx +4296 6433 0 0 6 16 96 296 296 4296 4296 192 193 GJAAAA LNJAAA HHHHxx +8874 6434 0 2 4 14 74 874 874 3874 8874 148 149 IDAAAA MNJAAA OOOOxx +1443 6435 1 3 3 3 43 443 1443 1443 1443 86 87 NDAAAA NNJAAA VVVVxx +4239 6436 1 3 9 19 39 239 239 4239 4239 78 79 BHAAAA ONJAAA AAAAxx +9760 6437 0 0 0 0 60 760 1760 4760 9760 120 121 KLAAAA PNJAAA HHHHxx +136 6438 0 0 6 16 36 136 136 136 136 72 73 GFAAAA QNJAAA OOOOxx +6472 6439 0 0 2 12 72 472 472 1472 6472 144 145 YOAAAA RNJAAA VVVVxx +4896 6440 0 0 6 16 96 896 896 4896 4896 192 193 IGAAAA SNJAAA AAAAxx +9028 6441 0 0 8 8 28 28 1028 4028 9028 56 57 GJAAAA TNJAAA HHHHxx +8354 6442 0 2 4 14 54 354 354 3354 8354 108 109 IJAAAA UNJAAA OOOOxx +8648 6443 0 0 8 8 48 648 648 3648 8648 96 97 QUAAAA VNJAAA VVVVxx +918 6444 0 2 8 18 18 918 918 918 918 36 37 IJAAAA WNJAAA AAAAxx +6606 6445 0 2 6 6 6 606 606 1606 6606 12 13 CUAAAA XNJAAA HHHHxx +2462 6446 0 2 2 2 62 462 462 2462 2462 124 125 SQAAAA YNJAAA OOOOxx +7536 6447 0 0 6 16 36 536 1536 2536 7536 72 73 WDAAAA ZNJAAA VVVVxx +1700 6448 0 0 0 0 0 700 1700 1700 1700 0 1 KNAAAA AOJAAA AAAAxx +6740 6449 0 0 0 0 40 740 740 1740 6740 80 81 GZAAAA BOJAAA HHHHxx +28 6450 0 0 8 8 28 28 28 28 28 56 57 CBAAAA COJAAA OOOOxx +6044 6451 0 0 4 4 44 44 44 1044 6044 88 89 MYAAAA DOJAAA VVVVxx +5053 6452 1 1 3 13 53 53 1053 53 5053 106 107 JMAAAA EOJAAA AAAAxx +4832 6453 0 0 2 12 32 832 832 4832 4832 64 65 WDAAAA FOJAAA HHHHxx +9145 6454 1 1 5 5 45 145 1145 4145 9145 90 91 TNAAAA GOJAAA OOOOxx +5482 6455 0 2 2 2 82 482 1482 482 5482 164 165 WCAAAA HOJAAA VVVVxx +7644 6456 0 0 4 4 44 644 1644 2644 7644 88 89 AIAAAA IOJAAA AAAAxx +2128 6457 0 0 8 8 28 128 128 2128 2128 56 57 WDAAAA JOJAAA HHHHxx +6583 6458 1 3 3 3 83 583 583 1583 6583 166 167 FTAAAA KOJAAA OOOOxx +4224 6459 0 0 4 4 24 224 224 4224 4224 48 49 MGAAAA LOJAAA VVVVxx +5253 6460 1 1 3 13 53 253 1253 253 5253 106 107 BUAAAA MOJAAA AAAAxx +8219 6461 1 3 9 19 19 219 219 3219 8219 38 39 DEAAAA NOJAAA HHHHxx +8113 6462 1 1 3 13 13 113 113 3113 8113 26 27 BAAAAA OOJAAA OOOOxx +3616 6463 0 0 6 16 16 616 1616 3616 3616 32 33 CJAAAA POJAAA VVVVxx +1361 6464 1 1 1 1 61 361 1361 1361 1361 122 123 JAAAAA QOJAAA AAAAxx +949 6465 1 1 9 9 49 949 949 949 949 98 99 NKAAAA ROJAAA HHHHxx +8582 6466 0 2 2 2 82 582 582 3582 8582 164 165 CSAAAA SOJAAA OOOOxx +5104 6467 0 0 4 4 4 104 1104 104 5104 8 9 IOAAAA TOJAAA VVVVxx +6146 6468 0 2 6 6 46 146 146 1146 6146 92 93 KCAAAA UOJAAA AAAAxx +7681 6469 1 1 1 1 81 681 1681 2681 7681 162 163 LJAAAA VOJAAA HHHHxx +1904 6470 0 0 4 4 4 904 1904 1904 1904 8 9 GVAAAA WOJAAA OOOOxx +1989 6471 1 1 9 9 89 989 1989 1989 1989 178 179 NYAAAA XOJAAA VVVVxx +4179 6472 1 3 9 19 79 179 179 4179 4179 158 159 TEAAAA YOJAAA AAAAxx +1739 6473 1 3 9 19 39 739 1739 1739 1739 78 79 XOAAAA ZOJAAA HHHHxx +2447 6474 1 3 7 7 47 447 447 2447 2447 94 95 DQAAAA APJAAA OOOOxx +3029 6475 1 1 9 9 29 29 1029 3029 3029 58 59 NMAAAA BPJAAA VVVVxx +9783 6476 1 3 3 3 83 783 1783 4783 9783 166 167 HMAAAA CPJAAA AAAAxx +8381 6477 1 1 1 1 81 381 381 3381 8381 162 163 JKAAAA DPJAAA HHHHxx +8755 6478 1 3 5 15 55 755 755 3755 8755 110 111 TYAAAA EPJAAA OOOOxx +8384 6479 0 0 4 4 84 384 384 3384 8384 168 169 MKAAAA FPJAAA VVVVxx +7655 6480 1 3 5 15 55 655 1655 2655 7655 110 111 LIAAAA GPJAAA AAAAxx +4766 6481 0 2 6 6 66 766 766 4766 4766 132 133 IBAAAA HPJAAA HHHHxx +3324 6482 0 0 4 4 24 324 1324 3324 3324 48 49 WXAAAA IPJAAA OOOOxx +5022 6483 0 2 2 2 22 22 1022 22 5022 44 45 ELAAAA JPJAAA VVVVxx +2856 6484 0 0 6 16 56 856 856 2856 2856 112 113 WFAAAA KPJAAA AAAAxx +6503 6485 1 3 3 3 3 503 503 1503 6503 6 7 DQAAAA LPJAAA HHHHxx +6872 6486 0 0 2 12 72 872 872 1872 6872 144 145 IEAAAA MPJAAA OOOOxx +1663 6487 1 3 3 3 63 663 1663 1663 1663 126 127 ZLAAAA NPJAAA VVVVxx +6964 6488 0 0 4 4 64 964 964 1964 6964 128 129 WHAAAA OPJAAA AAAAxx +4622 6489 0 2 2 2 22 622 622 4622 4622 44 45 UVAAAA PPJAAA HHHHxx +6089 6490 1 1 9 9 89 89 89 1089 6089 178 179 FAAAAA QPJAAA OOOOxx +8567 6491 1 3 7 7 67 567 567 3567 8567 134 135 NRAAAA RPJAAA VVVVxx +597 6492 1 1 7 17 97 597 597 597 597 194 195 ZWAAAA SPJAAA AAAAxx +4222 6493 0 2 2 2 22 222 222 4222 4222 44 45 KGAAAA TPJAAA HHHHxx +9322 6494 0 2 2 2 22 322 1322 4322 9322 44 45 OUAAAA UPJAAA OOOOxx +624 6495 0 0 4 4 24 624 624 624 624 48 49 AYAAAA VPJAAA VVVVxx +4329 6496 1 1 9 9 29 329 329 4329 4329 58 59 NKAAAA WPJAAA AAAAxx +6781 6497 1 1 1 1 81 781 781 1781 6781 162 163 VAAAAA XPJAAA HHHHxx +1673 6498 1 1 3 13 73 673 1673 1673 1673 146 147 JMAAAA YPJAAA OOOOxx +6633 6499 1 1 3 13 33 633 633 1633 6633 66 67 DVAAAA ZPJAAA VVVVxx +2569 6500 1 1 9 9 69 569 569 2569 2569 138 139 VUAAAA AQJAAA AAAAxx +4995 6501 1 3 5 15 95 995 995 4995 4995 190 191 DKAAAA BQJAAA HHHHxx +2749 6502 1 1 9 9 49 749 749 2749 2749 98 99 TBAAAA CQJAAA OOOOxx +9044 6503 0 0 4 4 44 44 1044 4044 9044 88 89 WJAAAA DQJAAA VVVVxx +5823 6504 1 3 3 3 23 823 1823 823 5823 46 47 ZPAAAA EQJAAA AAAAxx +9366 6505 0 2 6 6 66 366 1366 4366 9366 132 133 GWAAAA FQJAAA HHHHxx +1169 6506 1 1 9 9 69 169 1169 1169 1169 138 139 ZSAAAA GQJAAA OOOOxx +1300 6507 0 0 0 0 0 300 1300 1300 1300 0 1 AYAAAA HQJAAA VVVVxx +9973 6508 1 1 3 13 73 973 1973 4973 9973 146 147 PTAAAA IQJAAA AAAAxx +2092 6509 0 0 2 12 92 92 92 2092 2092 184 185 MCAAAA JQJAAA HHHHxx +9776 6510 0 0 6 16 76 776 1776 4776 9776 152 153 AMAAAA KQJAAA OOOOxx +7612 6511 0 0 2 12 12 612 1612 2612 7612 24 25 UGAAAA LQJAAA VVVVxx +7190 6512 0 2 0 10 90 190 1190 2190 7190 180 181 OQAAAA MQJAAA AAAAxx +5147 6513 1 3 7 7 47 147 1147 147 5147 94 95 ZPAAAA NQJAAA HHHHxx +3722 6514 0 2 2 2 22 722 1722 3722 3722 44 45 ENAAAA OQJAAA OOOOxx +5858 6515 0 2 8 18 58 858 1858 858 5858 116 117 IRAAAA PQJAAA VVVVxx +3204 6516 0 0 4 4 4 204 1204 3204 3204 8 9 GTAAAA QQJAAA AAAAxx +8994 6517 0 2 4 14 94 994 994 3994 8994 188 189 YHAAAA RQJAAA HHHHxx +7478 6518 0 2 8 18 78 478 1478 2478 7478 156 157 QBAAAA SQJAAA OOOOxx +9624 6519 0 0 4 4 24 624 1624 4624 9624 48 49 EGAAAA TQJAAA VVVVxx +6639 6520 1 3 9 19 39 639 639 1639 6639 78 79 JVAAAA UQJAAA AAAAxx +369 6521 1 1 9 9 69 369 369 369 369 138 139 FOAAAA VQJAAA HHHHxx +7766 6522 0 2 6 6 66 766 1766 2766 7766 132 133 SMAAAA WQJAAA OOOOxx +4094 6523 0 2 4 14 94 94 94 4094 4094 188 189 MBAAAA XQJAAA VVVVxx +9556 6524 0 0 6 16 56 556 1556 4556 9556 112 113 ODAAAA YQJAAA AAAAxx +4887 6525 1 3 7 7 87 887 887 4887 4887 174 175 ZFAAAA ZQJAAA HHHHxx +2321 6526 1 1 1 1 21 321 321 2321 2321 42 43 HLAAAA ARJAAA OOOOxx +9201 6527 1 1 1 1 1 201 1201 4201 9201 2 3 XPAAAA BRJAAA VVVVxx +1627 6528 1 3 7 7 27 627 1627 1627 1627 54 55 PKAAAA CRJAAA AAAAxx +150 6529 0 2 0 10 50 150 150 150 150 100 101 UFAAAA DRJAAA HHHHxx +8010 6530 0 2 0 10 10 10 10 3010 8010 20 21 CWAAAA ERJAAA OOOOxx +8026 6531 0 2 6 6 26 26 26 3026 8026 52 53 SWAAAA FRJAAA VVVVxx +5495 6532 1 3 5 15 95 495 1495 495 5495 190 191 JDAAAA GRJAAA AAAAxx +6213 6533 1 1 3 13 13 213 213 1213 6213 26 27 ZEAAAA HRJAAA HHHHxx +6464 6534 0 0 4 4 64 464 464 1464 6464 128 129 QOAAAA IRJAAA OOOOxx +1158 6535 0 2 8 18 58 158 1158 1158 1158 116 117 OSAAAA JRJAAA VVVVxx +8669 6536 1 1 9 9 69 669 669 3669 8669 138 139 LVAAAA KRJAAA AAAAxx +3225 6537 1 1 5 5 25 225 1225 3225 3225 50 51 BUAAAA LRJAAA HHHHxx +1294 6538 0 2 4 14 94 294 1294 1294 1294 188 189 UXAAAA MRJAAA OOOOxx +2166 6539 0 2 6 6 66 166 166 2166 2166 132 133 IFAAAA NRJAAA VVVVxx +9328 6540 0 0 8 8 28 328 1328 4328 9328 56 57 UUAAAA ORJAAA AAAAxx +8431 6541 1 3 1 11 31 431 431 3431 8431 62 63 HMAAAA PRJAAA HHHHxx +7100 6542 0 0 0 0 0 100 1100 2100 7100 0 1 CNAAAA QRJAAA OOOOxx +8126 6543 0 2 6 6 26 126 126 3126 8126 52 53 OAAAAA RRJAAA VVVVxx +2185 6544 1 1 5 5 85 185 185 2185 2185 170 171 BGAAAA SRJAAA AAAAxx +5697 6545 1 1 7 17 97 697 1697 697 5697 194 195 DLAAAA TRJAAA HHHHxx +5531 6546 1 3 1 11 31 531 1531 531 5531 62 63 TEAAAA URJAAA OOOOxx +3020 6547 0 0 0 0 20 20 1020 3020 3020 40 41 EMAAAA VRJAAA VVVVxx +3076 6548 0 0 6 16 76 76 1076 3076 3076 152 153 IOAAAA WRJAAA AAAAxx +9228 6549 0 0 8 8 28 228 1228 4228 9228 56 57 YQAAAA XRJAAA HHHHxx +1734 6550 0 2 4 14 34 734 1734 1734 1734 68 69 SOAAAA YRJAAA OOOOxx +7616 6551 0 0 6 16 16 616 1616 2616 7616 32 33 YGAAAA ZRJAAA VVVVxx +9059 6552 1 3 9 19 59 59 1059 4059 9059 118 119 LKAAAA ASJAAA AAAAxx +323 6553 1 3 3 3 23 323 323 323 323 46 47 LMAAAA BSJAAA HHHHxx +1283 6554 1 3 3 3 83 283 1283 1283 1283 166 167 JXAAAA CSJAAA OOOOxx +9535 6555 1 3 5 15 35 535 1535 4535 9535 70 71 TCAAAA DSJAAA VVVVxx +2580 6556 0 0 0 0 80 580 580 2580 2580 160 161 GVAAAA ESJAAA AAAAxx +7633 6557 1 1 3 13 33 633 1633 2633 7633 66 67 PHAAAA FSJAAA HHHHxx +9497 6558 1 1 7 17 97 497 1497 4497 9497 194 195 HBAAAA GSJAAA OOOOxx +9842 6559 0 2 2 2 42 842 1842 4842 9842 84 85 OOAAAA HSJAAA VVVVxx +3426 6560 0 2 6 6 26 426 1426 3426 3426 52 53 UBAAAA ISJAAA AAAAxx +7650 6561 0 2 0 10 50 650 1650 2650 7650 100 101 GIAAAA JSJAAA HHHHxx +9935 6562 1 3 5 15 35 935 1935 4935 9935 70 71 DSAAAA KSJAAA OOOOxx +9354 6563 0 2 4 14 54 354 1354 4354 9354 108 109 UVAAAA LSJAAA VVVVxx +5569 6564 1 1 9 9 69 569 1569 569 5569 138 139 FGAAAA MSJAAA AAAAxx +5765 6565 1 1 5 5 65 765 1765 765 5765 130 131 TNAAAA NSJAAA HHHHxx +7283 6566 1 3 3 3 83 283 1283 2283 7283 166 167 DUAAAA OSJAAA OOOOxx +1068 6567 0 0 8 8 68 68 1068 1068 1068 136 137 CPAAAA PSJAAA VVVVxx +1641 6568 1 1 1 1 41 641 1641 1641 1641 82 83 DLAAAA QSJAAA AAAAxx +1688 6569 0 0 8 8 88 688 1688 1688 1688 176 177 YMAAAA RSJAAA HHHHxx +1133 6570 1 1 3 13 33 133 1133 1133 1133 66 67 PRAAAA SSJAAA OOOOxx +4493 6571 1 1 3 13 93 493 493 4493 4493 186 187 VQAAAA TSJAAA VVVVxx +3354 6572 0 2 4 14 54 354 1354 3354 3354 108 109 AZAAAA USJAAA AAAAxx +4029 6573 1 1 9 9 29 29 29 4029 4029 58 59 ZYAAAA VSJAAA HHHHxx +6704 6574 0 0 4 4 4 704 704 1704 6704 8 9 WXAAAA WSJAAA OOOOxx +3221 6575 1 1 1 1 21 221 1221 3221 3221 42 43 XTAAAA XSJAAA VVVVxx +9432 6576 0 0 2 12 32 432 1432 4432 9432 64 65 UYAAAA YSJAAA AAAAxx +6990 6577 0 2 0 10 90 990 990 1990 6990 180 181 WIAAAA ZSJAAA HHHHxx +1760 6578 0 0 0 0 60 760 1760 1760 1760 120 121 SPAAAA ATJAAA OOOOxx +4754 6579 0 2 4 14 54 754 754 4754 4754 108 109 WAAAAA BTJAAA VVVVxx +7724 6580 0 0 4 4 24 724 1724 2724 7724 48 49 CLAAAA CTJAAA AAAAxx +9487 6581 1 3 7 7 87 487 1487 4487 9487 174 175 XAAAAA DTJAAA HHHHxx +166 6582 0 2 6 6 66 166 166 166 166 132 133 KGAAAA ETJAAA OOOOxx +5479 6583 1 3 9 19 79 479 1479 479 5479 158 159 TCAAAA FTJAAA VVVVxx +8744 6584 0 0 4 4 44 744 744 3744 8744 88 89 IYAAAA GTJAAA AAAAxx +5746 6585 0 2 6 6 46 746 1746 746 5746 92 93 ANAAAA HTJAAA HHHHxx +907 6586 1 3 7 7 7 907 907 907 907 14 15 XIAAAA ITJAAA OOOOxx +3968 6587 0 0 8 8 68 968 1968 3968 3968 136 137 QWAAAA JTJAAA VVVVxx +5721 6588 1 1 1 1 21 721 1721 721 5721 42 43 BMAAAA KTJAAA AAAAxx +6738 6589 0 2 8 18 38 738 738 1738 6738 76 77 EZAAAA LTJAAA HHHHxx +4097 6590 1 1 7 17 97 97 97 4097 4097 194 195 PBAAAA MTJAAA OOOOxx +8456 6591 0 0 6 16 56 456 456 3456 8456 112 113 GNAAAA NTJAAA VVVVxx +1269 6592 1 1 9 9 69 269 1269 1269 1269 138 139 VWAAAA OTJAAA AAAAxx +7997 6593 1 1 7 17 97 997 1997 2997 7997 194 195 PVAAAA PTJAAA HHHHxx +9457 6594 1 1 7 17 57 457 1457 4457 9457 114 115 TZAAAA QTJAAA OOOOxx +1159 6595 1 3 9 19 59 159 1159 1159 1159 118 119 PSAAAA RTJAAA VVVVxx +1631 6596 1 3 1 11 31 631 1631 1631 1631 62 63 TKAAAA STJAAA AAAAxx +2019 6597 1 3 9 19 19 19 19 2019 2019 38 39 RZAAAA TTJAAA HHHHxx +3186 6598 0 2 6 6 86 186 1186 3186 3186 172 173 OSAAAA UTJAAA OOOOxx +5587 6599 1 3 7 7 87 587 1587 587 5587 174 175 XGAAAA VTJAAA VVVVxx +9172 6600 0 0 2 12 72 172 1172 4172 9172 144 145 UOAAAA WTJAAA AAAAxx +5589 6601 1 1 9 9 89 589 1589 589 5589 178 179 ZGAAAA XTJAAA HHHHxx +5103 6602 1 3 3 3 3 103 1103 103 5103 6 7 HOAAAA YTJAAA OOOOxx +3177 6603 1 1 7 17 77 177 1177 3177 3177 154 155 FSAAAA ZTJAAA VVVVxx +8887 6604 1 3 7 7 87 887 887 3887 8887 174 175 VDAAAA AUJAAA AAAAxx +12 6605 0 0 2 12 12 12 12 12 12 24 25 MAAAAA BUJAAA HHHHxx +8575 6606 1 3 5 15 75 575 575 3575 8575 150 151 VRAAAA CUJAAA OOOOxx +4335 6607 1 3 5 15 35 335 335 4335 4335 70 71 TKAAAA DUJAAA VVVVxx +4581 6608 1 1 1 1 81 581 581 4581 4581 162 163 FUAAAA EUJAAA AAAAxx +4444 6609 0 0 4 4 44 444 444 4444 4444 88 89 YOAAAA FUJAAA HHHHxx +7978 6610 0 2 8 18 78 978 1978 2978 7978 156 157 WUAAAA GUJAAA OOOOxx +3081 6611 1 1 1 1 81 81 1081 3081 3081 162 163 NOAAAA HUJAAA VVVVxx +4059 6612 1 3 9 19 59 59 59 4059 4059 118 119 DAAAAA IUJAAA AAAAxx +5711 6613 1 3 1 11 11 711 1711 711 5711 22 23 RLAAAA JUJAAA HHHHxx +7069 6614 1 1 9 9 69 69 1069 2069 7069 138 139 XLAAAA KUJAAA OOOOxx +6150 6615 0 2 0 10 50 150 150 1150 6150 100 101 OCAAAA LUJAAA VVVVxx +9550 6616 0 2 0 10 50 550 1550 4550 9550 100 101 IDAAAA MUJAAA AAAAxx +7087 6617 1 3 7 7 87 87 1087 2087 7087 174 175 PMAAAA NUJAAA HHHHxx +9557 6618 1 1 7 17 57 557 1557 4557 9557 114 115 PDAAAA OUJAAA OOOOxx +7856 6619 0 0 6 16 56 856 1856 2856 7856 112 113 EQAAAA PUJAAA VVVVxx +1115 6620 1 3 5 15 15 115 1115 1115 1115 30 31 XQAAAA QUJAAA AAAAxx +1086 6621 0 2 6 6 86 86 1086 1086 1086 172 173 UPAAAA RUJAAA HHHHxx +5048 6622 0 0 8 8 48 48 1048 48 5048 96 97 EMAAAA SUJAAA OOOOxx +5168 6623 0 0 8 8 68 168 1168 168 5168 136 137 UQAAAA TUJAAA VVVVxx +6029 6624 1 1 9 9 29 29 29 1029 6029 58 59 XXAAAA UUJAAA AAAAxx +546 6625 0 2 6 6 46 546 546 546 546 92 93 AVAAAA VUJAAA HHHHxx +2908 6626 0 0 8 8 8 908 908 2908 2908 16 17 WHAAAA WUJAAA OOOOxx +779 6627 1 3 9 19 79 779 779 779 779 158 159 ZDAAAA XUJAAA VVVVxx +4202 6628 0 2 2 2 2 202 202 4202 4202 4 5 QFAAAA YUJAAA AAAAxx +9984 6629 0 0 4 4 84 984 1984 4984 9984 168 169 AUAAAA ZUJAAA HHHHxx +4730 6630 0 2 0 10 30 730 730 4730 4730 60 61 YZAAAA AVJAAA OOOOxx +6517 6631 1 1 7 17 17 517 517 1517 6517 34 35 RQAAAA BVJAAA VVVVxx +8410 6632 0 2 0 10 10 410 410 3410 8410 20 21 MLAAAA CVJAAA AAAAxx +4793 6633 1 1 3 13 93 793 793 4793 4793 186 187 JCAAAA DVJAAA HHHHxx +3431 6634 1 3 1 11 31 431 1431 3431 3431 62 63 ZBAAAA EVJAAA OOOOxx +2481 6635 1 1 1 1 81 481 481 2481 2481 162 163 LRAAAA FVJAAA VVVVxx +3905 6636 1 1 5 5 5 905 1905 3905 3905 10 11 FUAAAA GVJAAA AAAAxx +8807 6637 1 3 7 7 7 807 807 3807 8807 14 15 TAAAAA HVJAAA HHHHxx +2660 6638 0 0 0 0 60 660 660 2660 2660 120 121 IYAAAA IVJAAA OOOOxx +4985 6639 1 1 5 5 85 985 985 4985 4985 170 171 TJAAAA JVJAAA VVVVxx +3080 6640 0 0 0 0 80 80 1080 3080 3080 160 161 MOAAAA KVJAAA AAAAxx +1090 6641 0 2 0 10 90 90 1090 1090 1090 180 181 YPAAAA LVJAAA HHHHxx +6917 6642 1 1 7 17 17 917 917 1917 6917 34 35 BGAAAA MVJAAA OOOOxx +5177 6643 1 1 7 17 77 177 1177 177 5177 154 155 DRAAAA NVJAAA VVVVxx +2729 6644 1 1 9 9 29 729 729 2729 2729 58 59 ZAAAAA OVJAAA AAAAxx +9706 6645 0 2 6 6 6 706 1706 4706 9706 12 13 IJAAAA PVJAAA HHHHxx +9929 6646 1 1 9 9 29 929 1929 4929 9929 58 59 XRAAAA QVJAAA OOOOxx +1547 6647 1 3 7 7 47 547 1547 1547 1547 94 95 NHAAAA RVJAAA VVVVxx +2798 6648 0 2 8 18 98 798 798 2798 2798 196 197 QDAAAA SVJAAA AAAAxx +4420 6649 0 0 0 0 20 420 420 4420 4420 40 41 AOAAAA TVJAAA HHHHxx +6771 6650 1 3 1 11 71 771 771 1771 6771 142 143 LAAAAA UVJAAA OOOOxx +2004 6651 0 0 4 4 4 4 4 2004 2004 8 9 CZAAAA VVJAAA VVVVxx +8686 6652 0 2 6 6 86 686 686 3686 8686 172 173 CWAAAA WVJAAA AAAAxx +3663 6653 1 3 3 3 63 663 1663 3663 3663 126 127 XKAAAA XVJAAA HHHHxx +806 6654 0 2 6 6 6 806 806 806 806 12 13 AFAAAA YVJAAA OOOOxx +4309 6655 1 1 9 9 9 309 309 4309 4309 18 19 TJAAAA ZVJAAA VVVVxx +7443 6656 1 3 3 3 43 443 1443 2443 7443 86 87 HAAAAA AWJAAA AAAAxx +5779 6657 1 3 9 19 79 779 1779 779 5779 158 159 HOAAAA BWJAAA HHHHxx +8821 6658 1 1 1 1 21 821 821 3821 8821 42 43 HBAAAA CWJAAA OOOOxx +4198 6659 0 2 8 18 98 198 198 4198 4198 196 197 MFAAAA DWJAAA VVVVxx +8115 6660 1 3 5 15 15 115 115 3115 8115 30 31 DAAAAA EWJAAA AAAAxx +9554 6661 0 2 4 14 54 554 1554 4554 9554 108 109 MDAAAA FWJAAA HHHHxx +8956 6662 0 0 6 16 56 956 956 3956 8956 112 113 MGAAAA GWJAAA OOOOxx +4733 6663 1 1 3 13 33 733 733 4733 4733 66 67 BAAAAA HWJAAA VVVVxx +5417 6664 1 1 7 17 17 417 1417 417 5417 34 35 JAAAAA IWJAAA AAAAxx +4792 6665 0 0 2 12 92 792 792 4792 4792 184 185 ICAAAA JWJAAA HHHHxx +462 6666 0 2 2 2 62 462 462 462 462 124 125 URAAAA KWJAAA OOOOxx +3687 6667 1 3 7 7 87 687 1687 3687 3687 174 175 VLAAAA LWJAAA VVVVxx +2013 6668 1 1 3 13 13 13 13 2013 2013 26 27 LZAAAA MWJAAA AAAAxx +5386 6669 0 2 6 6 86 386 1386 386 5386 172 173 EZAAAA NWJAAA HHHHxx +2816 6670 0 0 6 16 16 816 816 2816 2816 32 33 IEAAAA OWJAAA OOOOxx +7827 6671 1 3 7 7 27 827 1827 2827 7827 54 55 BPAAAA PWJAAA VVVVxx +5077 6672 1 1 7 17 77 77 1077 77 5077 154 155 HNAAAA QWJAAA AAAAxx +6039 6673 1 3 9 19 39 39 39 1039 6039 78 79 HYAAAA RWJAAA HHHHxx +215 6674 1 3 5 15 15 215 215 215 215 30 31 HIAAAA SWJAAA OOOOxx +855 6675 1 3 5 15 55 855 855 855 855 110 111 XGAAAA TWJAAA VVVVxx +9692 6676 0 0 2 12 92 692 1692 4692 9692 184 185 UIAAAA UWJAAA AAAAxx +8391 6677 1 3 1 11 91 391 391 3391 8391 182 183 TKAAAA VWJAAA HHHHxx +8424 6678 0 0 4 4 24 424 424 3424 8424 48 49 AMAAAA WWJAAA OOOOxx +6331 6679 1 3 1 11 31 331 331 1331 6331 62 63 NJAAAA XWJAAA VVVVxx +6561 6680 1 1 1 1 61 561 561 1561 6561 122 123 JSAAAA YWJAAA AAAAxx +8955 6681 1 3 5 15 55 955 955 3955 8955 110 111 LGAAAA ZWJAAA HHHHxx +1764 6682 0 0 4 4 64 764 1764 1764 1764 128 129 WPAAAA AXJAAA OOOOxx +6623 6683 1 3 3 3 23 623 623 1623 6623 46 47 TUAAAA BXJAAA VVVVxx +2900 6684 0 0 0 0 0 900 900 2900 2900 0 1 OHAAAA CXJAAA AAAAxx +7048 6685 0 0 8 8 48 48 1048 2048 7048 96 97 CLAAAA DXJAAA HHHHxx +3843 6686 1 3 3 3 43 843 1843 3843 3843 86 87 VRAAAA EXJAAA OOOOxx +4855 6687 1 3 5 15 55 855 855 4855 4855 110 111 TEAAAA FXJAAA VVVVxx +7383 6688 1 3 3 3 83 383 1383 2383 7383 166 167 ZXAAAA GXJAAA AAAAxx +7765 6689 1 1 5 5 65 765 1765 2765 7765 130 131 RMAAAA HXJAAA HHHHxx +1125 6690 1 1 5 5 25 125 1125 1125 1125 50 51 HRAAAA IXJAAA OOOOxx +755 6691 1 3 5 15 55 755 755 755 755 110 111 BDAAAA JXJAAA VVVVxx +2995 6692 1 3 5 15 95 995 995 2995 2995 190 191 FLAAAA KXJAAA AAAAxx +8907 6693 1 3 7 7 7 907 907 3907 8907 14 15 PEAAAA LXJAAA HHHHxx +9357 6694 1 1 7 17 57 357 1357 4357 9357 114 115 XVAAAA MXJAAA OOOOxx +4469 6695 1 1 9 9 69 469 469 4469 4469 138 139 XPAAAA NXJAAA VVVVxx +2147 6696 1 3 7 7 47 147 147 2147 2147 94 95 PEAAAA OXJAAA AAAAxx +2952 6697 0 0 2 12 52 952 952 2952 2952 104 105 OJAAAA PXJAAA HHHHxx +1324 6698 0 0 4 4 24 324 1324 1324 1324 48 49 YYAAAA QXJAAA OOOOxx +1173 6699 1 1 3 13 73 173 1173 1173 1173 146 147 DTAAAA RXJAAA VVVVxx +3169 6700 1 1 9 9 69 169 1169 3169 3169 138 139 XRAAAA SXJAAA AAAAxx +5149 6701 1 1 9 9 49 149 1149 149 5149 98 99 BQAAAA TXJAAA HHHHxx +9660 6702 0 0 0 0 60 660 1660 4660 9660 120 121 OHAAAA UXJAAA OOOOxx +3446 6703 0 2 6 6 46 446 1446 3446 3446 92 93 OCAAAA VXJAAA VVVVxx +6988 6704 0 0 8 8 88 988 988 1988 6988 176 177 UIAAAA WXJAAA AAAAxx +5829 6705 1 1 9 9 29 829 1829 829 5829 58 59 FQAAAA XXJAAA HHHHxx +7166 6706 0 2 6 6 66 166 1166 2166 7166 132 133 QPAAAA YXJAAA OOOOxx +3940 6707 0 0 0 0 40 940 1940 3940 3940 80 81 OVAAAA ZXJAAA VVVVxx +2645 6708 1 1 5 5 45 645 645 2645 2645 90 91 TXAAAA AYJAAA AAAAxx +478 6709 0 2 8 18 78 478 478 478 478 156 157 KSAAAA BYJAAA HHHHxx +1156 6710 0 0 6 16 56 156 1156 1156 1156 112 113 MSAAAA CYJAAA OOOOxx +2731 6711 1 3 1 11 31 731 731 2731 2731 62 63 BBAAAA DYJAAA VVVVxx +5637 6712 1 1 7 17 37 637 1637 637 5637 74 75 VIAAAA EYJAAA AAAAxx +7517 6713 1 1 7 17 17 517 1517 2517 7517 34 35 DDAAAA FYJAAA HHHHxx +5331 6714 1 3 1 11 31 331 1331 331 5331 62 63 BXAAAA GYJAAA OOOOxx +9640 6715 0 0 0 0 40 640 1640 4640 9640 80 81 UGAAAA HYJAAA VVVVxx +4108 6716 0 0 8 8 8 108 108 4108 4108 16 17 ACAAAA IYJAAA AAAAxx +1087 6717 1 3 7 7 87 87 1087 1087 1087 174 175 VPAAAA JYJAAA HHHHxx +8017 6718 1 1 7 17 17 17 17 3017 8017 34 35 JWAAAA KYJAAA OOOOxx +8795 6719 1 3 5 15 95 795 795 3795 8795 190 191 HAAAAA LYJAAA VVVVxx +7060 6720 0 0 0 0 60 60 1060 2060 7060 120 121 OLAAAA MYJAAA AAAAxx +9450 6721 0 2 0 10 50 450 1450 4450 9450 100 101 MZAAAA NYJAAA HHHHxx +390 6722 0 2 0 10 90 390 390 390 390 180 181 APAAAA OYJAAA OOOOxx +66 6723 0 2 6 6 66 66 66 66 66 132 133 OCAAAA PYJAAA VVVVxx +8789 6724 1 1 9 9 89 789 789 3789 8789 178 179 BAAAAA QYJAAA AAAAxx +9260 6725 0 0 0 0 60 260 1260 4260 9260 120 121 ESAAAA RYJAAA HHHHxx +6679 6726 1 3 9 19 79 679 679 1679 6679 158 159 XWAAAA SYJAAA OOOOxx +9052 6727 0 0 2 12 52 52 1052 4052 9052 104 105 EKAAAA TYJAAA VVVVxx +9561 6728 1 1 1 1 61 561 1561 4561 9561 122 123 TDAAAA UYJAAA AAAAxx +9725 6729 1 1 5 5 25 725 1725 4725 9725 50 51 BKAAAA VYJAAA HHHHxx +6298 6730 0 2 8 18 98 298 298 1298 6298 196 197 GIAAAA WYJAAA OOOOxx +8654 6731 0 2 4 14 54 654 654 3654 8654 108 109 WUAAAA XYJAAA VVVVxx +8725 6732 1 1 5 5 25 725 725 3725 8725 50 51 PXAAAA YYJAAA AAAAxx +9377 6733 1 1 7 17 77 377 1377 4377 9377 154 155 RWAAAA ZYJAAA HHHHxx +3807 6734 1 3 7 7 7 807 1807 3807 3807 14 15 LQAAAA AZJAAA OOOOxx +8048 6735 0 0 8 8 48 48 48 3048 8048 96 97 OXAAAA BZJAAA VVVVxx +764 6736 0 0 4 4 64 764 764 764 764 128 129 KDAAAA CZJAAA AAAAxx +9702 6737 0 2 2 2 2 702 1702 4702 9702 4 5 EJAAAA DZJAAA HHHHxx +8060 6738 0 0 0 0 60 60 60 3060 8060 120 121 AYAAAA EZJAAA OOOOxx +6371 6739 1 3 1 11 71 371 371 1371 6371 142 143 BLAAAA FZJAAA VVVVxx +5237 6740 1 1 7 17 37 237 1237 237 5237 74 75 LTAAAA GZJAAA AAAAxx +743 6741 1 3 3 3 43 743 743 743 743 86 87 PCAAAA HZJAAA HHHHxx +7395 6742 1 3 5 15 95 395 1395 2395 7395 190 191 LYAAAA IZJAAA OOOOxx +3365 6743 1 1 5 5 65 365 1365 3365 3365 130 131 LZAAAA JZJAAA VVVVxx +6667 6744 1 3 7 7 67 667 667 1667 6667 134 135 LWAAAA KZJAAA AAAAxx +3445 6745 1 1 5 5 45 445 1445 3445 3445 90 91 NCAAAA LZJAAA HHHHxx +4019 6746 1 3 9 19 19 19 19 4019 4019 38 39 PYAAAA MZJAAA OOOOxx +7035 6747 1 3 5 15 35 35 1035 2035 7035 70 71 PKAAAA NZJAAA VVVVxx +5274 6748 0 2 4 14 74 274 1274 274 5274 148 149 WUAAAA OZJAAA AAAAxx +519 6749 1 3 9 19 19 519 519 519 519 38 39 ZTAAAA PZJAAA HHHHxx +2801 6750 1 1 1 1 1 801 801 2801 2801 2 3 TDAAAA QZJAAA OOOOxx +3320 6751 0 0 0 0 20 320 1320 3320 3320 40 41 SXAAAA RZJAAA VVVVxx +3153 6752 1 1 3 13 53 153 1153 3153 3153 106 107 HRAAAA SZJAAA AAAAxx +7680 6753 0 0 0 0 80 680 1680 2680 7680 160 161 KJAAAA TZJAAA HHHHxx +8942 6754 0 2 2 2 42 942 942 3942 8942 84 85 YFAAAA UZJAAA OOOOxx +3195 6755 1 3 5 15 95 195 1195 3195 3195 190 191 XSAAAA VZJAAA VVVVxx +2287 6756 1 3 7 7 87 287 287 2287 2287 174 175 ZJAAAA WZJAAA AAAAxx +8325 6757 1 1 5 5 25 325 325 3325 8325 50 51 FIAAAA XZJAAA HHHHxx +2603 6758 1 3 3 3 3 603 603 2603 2603 6 7 DWAAAA YZJAAA OOOOxx +5871 6759 1 3 1 11 71 871 1871 871 5871 142 143 VRAAAA ZZJAAA VVVVxx +1773 6760 1 1 3 13 73 773 1773 1773 1773 146 147 FQAAAA AAKAAA AAAAxx +3323 6761 1 3 3 3 23 323 1323 3323 3323 46 47 VXAAAA BAKAAA HHHHxx +2053 6762 1 1 3 13 53 53 53 2053 2053 106 107 ZAAAAA CAKAAA OOOOxx +4062 6763 0 2 2 2 62 62 62 4062 4062 124 125 GAAAAA DAKAAA VVVVxx +4611 6764 1 3 1 11 11 611 611 4611 4611 22 23 JVAAAA EAKAAA AAAAxx +3451 6765 1 3 1 11 51 451 1451 3451 3451 102 103 TCAAAA FAKAAA HHHHxx +1819 6766 1 3 9 19 19 819 1819 1819 1819 38 39 ZRAAAA GAKAAA OOOOxx +9806 6767 0 2 6 6 6 806 1806 4806 9806 12 13 ENAAAA HAKAAA VVVVxx +6619 6768 1 3 9 19 19 619 619 1619 6619 38 39 PUAAAA IAKAAA AAAAxx +1031 6769 1 3 1 11 31 31 1031 1031 1031 62 63 RNAAAA JAKAAA HHHHxx +1865 6770 1 1 5 5 65 865 1865 1865 1865 130 131 TTAAAA KAKAAA OOOOxx +6282 6771 0 2 2 2 82 282 282 1282 6282 164 165 QHAAAA LAKAAA VVVVxx +1178 6772 0 2 8 18 78 178 1178 1178 1178 156 157 ITAAAA MAKAAA AAAAxx +8007 6773 1 3 7 7 7 7 7 3007 8007 14 15 ZVAAAA NAKAAA HHHHxx +9126 6774 0 2 6 6 26 126 1126 4126 9126 52 53 ANAAAA OAKAAA OOOOxx +9113 6775 1 1 3 13 13 113 1113 4113 9113 26 27 NMAAAA PAKAAA VVVVxx +537 6776 1 1 7 17 37 537 537 537 537 74 75 RUAAAA QAKAAA AAAAxx +6208 6777 0 0 8 8 8 208 208 1208 6208 16 17 UEAAAA RAKAAA HHHHxx +1626 6778 0 2 6 6 26 626 1626 1626 1626 52 53 OKAAAA SAKAAA OOOOxx +7188 6779 0 0 8 8 88 188 1188 2188 7188 176 177 MQAAAA TAKAAA VVVVxx +9216 6780 0 0 6 16 16 216 1216 4216 9216 32 33 MQAAAA UAKAAA AAAAxx +6134 6781 0 2 4 14 34 134 134 1134 6134 68 69 YBAAAA VAKAAA HHHHxx +2074 6782 0 2 4 14 74 74 74 2074 2074 148 149 UBAAAA WAKAAA OOOOxx +6369 6783 1 1 9 9 69 369 369 1369 6369 138 139 ZKAAAA XAKAAA VVVVxx +9306 6784 0 2 6 6 6 306 1306 4306 9306 12 13 YTAAAA YAKAAA AAAAxx +3155 6785 1 3 5 15 55 155 1155 3155 3155 110 111 JRAAAA ZAKAAA HHHHxx +3611 6786 1 3 1 11 11 611 1611 3611 3611 22 23 XIAAAA ABKAAA OOOOxx +6530 6787 0 2 0 10 30 530 530 1530 6530 60 61 ERAAAA BBKAAA VVVVxx +6979 6788 1 3 9 19 79 979 979 1979 6979 158 159 LIAAAA CBKAAA AAAAxx +9129 6789 1 1 9 9 29 129 1129 4129 9129 58 59 DNAAAA DBKAAA HHHHxx +8013 6790 1 1 3 13 13 13 13 3013 8013 26 27 FWAAAA EBKAAA OOOOxx +6926 6791 0 2 6 6 26 926 926 1926 6926 52 53 KGAAAA FBKAAA VVVVxx +1877 6792 1 1 7 17 77 877 1877 1877 1877 154 155 FUAAAA GBKAAA AAAAxx +1882 6793 0 2 2 2 82 882 1882 1882 1882 164 165 KUAAAA HBKAAA HHHHxx +6720 6794 0 0 0 0 20 720 720 1720 6720 40 41 MYAAAA IBKAAA OOOOxx +690 6795 0 2 0 10 90 690 690 690 690 180 181 OAAAAA JBKAAA VVVVxx +143 6796 1 3 3 3 43 143 143 143 143 86 87 NFAAAA KBKAAA AAAAxx +7241 6797 1 1 1 1 41 241 1241 2241 7241 82 83 NSAAAA LBKAAA HHHHxx +6461 6798 1 1 1 1 61 461 461 1461 6461 122 123 NOAAAA MBKAAA OOOOxx +2258 6799 0 2 8 18 58 258 258 2258 2258 116 117 WIAAAA NBKAAA VVVVxx +2280 6800 0 0 0 0 80 280 280 2280 2280 160 161 SJAAAA OBKAAA AAAAxx +7556 6801 0 0 6 16 56 556 1556 2556 7556 112 113 QEAAAA PBKAAA HHHHxx +1038 6802 0 2 8 18 38 38 1038 1038 1038 76 77 YNAAAA QBKAAA OOOOxx +2634 6803 0 2 4 14 34 634 634 2634 2634 68 69 IXAAAA RBKAAA VVVVxx +7847 6804 1 3 7 7 47 847 1847 2847 7847 94 95 VPAAAA SBKAAA AAAAxx +4415 6805 1 3 5 15 15 415 415 4415 4415 30 31 VNAAAA TBKAAA HHHHxx +1933 6806 1 1 3 13 33 933 1933 1933 1933 66 67 JWAAAA UBKAAA OOOOxx +8034 6807 0 2 4 14 34 34 34 3034 8034 68 69 AXAAAA VBKAAA VVVVxx +9233 6808 1 1 3 13 33 233 1233 4233 9233 66 67 DRAAAA WBKAAA AAAAxx +6572 6809 0 0 2 12 72 572 572 1572 6572 144 145 USAAAA XBKAAA HHHHxx +1586 6810 0 2 6 6 86 586 1586 1586 1586 172 173 AJAAAA YBKAAA OOOOxx +8512 6811 0 0 2 12 12 512 512 3512 8512 24 25 KPAAAA ZBKAAA VVVVxx +7421 6812 1 1 1 1 21 421 1421 2421 7421 42 43 LZAAAA ACKAAA AAAAxx +503 6813 1 3 3 3 3 503 503 503 503 6 7 JTAAAA BCKAAA HHHHxx +5332 6814 0 0 2 12 32 332 1332 332 5332 64 65 CXAAAA CCKAAA OOOOxx +2602 6815 0 2 2 2 2 602 602 2602 2602 4 5 CWAAAA DCKAAA VVVVxx +2902 6816 0 2 2 2 2 902 902 2902 2902 4 5 QHAAAA ECKAAA AAAAxx +2979 6817 1 3 9 19 79 979 979 2979 2979 158 159 PKAAAA FCKAAA HHHHxx +1431 6818 1 3 1 11 31 431 1431 1431 1431 62 63 BDAAAA GCKAAA OOOOxx +8639 6819 1 3 9 19 39 639 639 3639 8639 78 79 HUAAAA HCKAAA VVVVxx +4218 6820 0 2 8 18 18 218 218 4218 4218 36 37 GGAAAA ICKAAA AAAAxx +7453 6821 1 1 3 13 53 453 1453 2453 7453 106 107 RAAAAA JCKAAA HHHHxx +5448 6822 0 0 8 8 48 448 1448 448 5448 96 97 OBAAAA KCKAAA OOOOxx +6768 6823 0 0 8 8 68 768 768 1768 6768 136 137 IAAAAA LCKAAA VVVVxx +3104 6824 0 0 4 4 4 104 1104 3104 3104 8 9 KPAAAA MCKAAA AAAAxx +2297 6825 1 1 7 17 97 297 297 2297 2297 194 195 JKAAAA NCKAAA HHHHxx +7994 6826 0 2 4 14 94 994 1994 2994 7994 188 189 MVAAAA OCKAAA OOOOxx +550 6827 0 2 0 10 50 550 550 550 550 100 101 EVAAAA PCKAAA VVVVxx +4777 6828 1 1 7 17 77 777 777 4777 4777 154 155 TBAAAA QCKAAA AAAAxx +5962 6829 0 2 2 2 62 962 1962 962 5962 124 125 IVAAAA RCKAAA HHHHxx +1763 6830 1 3 3 3 63 763 1763 1763 1763 126 127 VPAAAA SCKAAA OOOOxx +3654 6831 0 2 4 14 54 654 1654 3654 3654 108 109 OKAAAA TCKAAA VVVVxx +4106 6832 0 2 6 6 6 106 106 4106 4106 12 13 YBAAAA UCKAAA AAAAxx +5156 6833 0 0 6 16 56 156 1156 156 5156 112 113 IQAAAA VCKAAA HHHHxx +422 6834 0 2 2 2 22 422 422 422 422 44 45 GQAAAA WCKAAA OOOOxx +5011 6835 1 3 1 11 11 11 1011 11 5011 22 23 TKAAAA XCKAAA VVVVxx +218 6836 0 2 8 18 18 218 218 218 218 36 37 KIAAAA YCKAAA AAAAxx +9762 6837 0 2 2 2 62 762 1762 4762 9762 124 125 MLAAAA ZCKAAA HHHHxx +6074 6838 0 2 4 14 74 74 74 1074 6074 148 149 QZAAAA ADKAAA OOOOxx +4060 6839 0 0 0 0 60 60 60 4060 4060 120 121 EAAAAA BDKAAA VVVVxx +8680 6840 0 0 0 0 80 680 680 3680 8680 160 161 WVAAAA CDKAAA AAAAxx +5863 6841 1 3 3 3 63 863 1863 863 5863 126 127 NRAAAA DDKAAA HHHHxx +8042 6842 0 2 2 2 42 42 42 3042 8042 84 85 IXAAAA EDKAAA OOOOxx +2964 6843 0 0 4 4 64 964 964 2964 2964 128 129 AKAAAA FDKAAA VVVVxx +6931 6844 1 3 1 11 31 931 931 1931 6931 62 63 PGAAAA GDKAAA AAAAxx +6715 6845 1 3 5 15 15 715 715 1715 6715 30 31 HYAAAA HDKAAA HHHHxx +5859 6846 1 3 9 19 59 859 1859 859 5859 118 119 JRAAAA IDKAAA OOOOxx +6173 6847 1 1 3 13 73 173 173 1173 6173 146 147 LDAAAA JDKAAA VVVVxx +7788 6848 0 0 8 8 88 788 1788 2788 7788 176 177 ONAAAA KDKAAA AAAAxx +9370 6849 0 2 0 10 70 370 1370 4370 9370 140 141 KWAAAA LDKAAA HHHHxx +3038 6850 0 2 8 18 38 38 1038 3038 3038 76 77 WMAAAA MDKAAA OOOOxx +6483 6851 1 3 3 3 83 483 483 1483 6483 166 167 JPAAAA NDKAAA VVVVxx +7534 6852 0 2 4 14 34 534 1534 2534 7534 68 69 UDAAAA ODKAAA AAAAxx +5769 6853 1 1 9 9 69 769 1769 769 5769 138 139 XNAAAA PDKAAA HHHHxx +9152 6854 0 0 2 12 52 152 1152 4152 9152 104 105 AOAAAA QDKAAA OOOOxx +6251 6855 1 3 1 11 51 251 251 1251 6251 102 103 LGAAAA RDKAAA VVVVxx +9209 6856 1 1 9 9 9 209 1209 4209 9209 18 19 FQAAAA SDKAAA AAAAxx +5365 6857 1 1 5 5 65 365 1365 365 5365 130 131 JYAAAA TDKAAA HHHHxx +509 6858 1 1 9 9 9 509 509 509 509 18 19 PTAAAA UDKAAA OOOOxx +3132 6859 0 0 2 12 32 132 1132 3132 3132 64 65 MQAAAA VDKAAA VVVVxx +5373 6860 1 1 3 13 73 373 1373 373 5373 146 147 RYAAAA WDKAAA AAAAxx +4247 6861 1 3 7 7 47 247 247 4247 4247 94 95 JHAAAA XDKAAA HHHHxx +3491 6862 1 3 1 11 91 491 1491 3491 3491 182 183 HEAAAA YDKAAA OOOOxx +495 6863 1 3 5 15 95 495 495 495 495 190 191 BTAAAA ZDKAAA VVVVxx +1594 6864 0 2 4 14 94 594 1594 1594 1594 188 189 IJAAAA AEKAAA AAAAxx +2243 6865 1 3 3 3 43 243 243 2243 2243 86 87 HIAAAA BEKAAA HHHHxx +7780 6866 0 0 0 0 80 780 1780 2780 7780 160 161 GNAAAA CEKAAA OOOOxx +5632 6867 0 0 2 12 32 632 1632 632 5632 64 65 QIAAAA DEKAAA VVVVxx +2679 6868 1 3 9 19 79 679 679 2679 2679 158 159 BZAAAA EEKAAA AAAAxx +1354 6869 0 2 4 14 54 354 1354 1354 1354 108 109 CAAAAA FEKAAA HHHHxx +180 6870 0 0 0 0 80 180 180 180 180 160 161 YGAAAA GEKAAA OOOOxx +7017 6871 1 1 7 17 17 17 1017 2017 7017 34 35 XJAAAA HEKAAA VVVVxx +1867 6872 1 3 7 7 67 867 1867 1867 1867 134 135 VTAAAA IEKAAA AAAAxx +2213 6873 1 1 3 13 13 213 213 2213 2213 26 27 DHAAAA JEKAAA HHHHxx +8773 6874 1 1 3 13 73 773 773 3773 8773 146 147 LZAAAA KEKAAA OOOOxx +1784 6875 0 0 4 4 84 784 1784 1784 1784 168 169 QQAAAA LEKAAA VVVVxx +5961 6876 1 1 1 1 61 961 1961 961 5961 122 123 HVAAAA MEKAAA AAAAxx +8801 6877 1 1 1 1 1 801 801 3801 8801 2 3 NAAAAA NEKAAA HHHHxx +4860 6878 0 0 0 0 60 860 860 4860 4860 120 121 YEAAAA OEKAAA OOOOxx +2214 6879 0 2 4 14 14 214 214 2214 2214 28 29 EHAAAA PEKAAA VVVVxx +1735 6880 1 3 5 15 35 735 1735 1735 1735 70 71 TOAAAA QEKAAA AAAAxx +578 6881 0 2 8 18 78 578 578 578 578 156 157 GWAAAA REKAAA HHHHxx +7853 6882 1 1 3 13 53 853 1853 2853 7853 106 107 BQAAAA SEKAAA OOOOxx +2215 6883 1 3 5 15 15 215 215 2215 2215 30 31 FHAAAA TEKAAA VVVVxx +4704 6884 0 0 4 4 4 704 704 4704 4704 8 9 YYAAAA UEKAAA AAAAxx +9379 6885 1 3 9 19 79 379 1379 4379 9379 158 159 TWAAAA VEKAAA HHHHxx +9745 6886 1 1 5 5 45 745 1745 4745 9745 90 91 VKAAAA WEKAAA OOOOxx +5636 6887 0 0 6 16 36 636 1636 636 5636 72 73 UIAAAA XEKAAA VVVVxx +4548 6888 0 0 8 8 48 548 548 4548 4548 96 97 YSAAAA YEKAAA AAAAxx +6537 6889 1 1 7 17 37 537 537 1537 6537 74 75 LRAAAA ZEKAAA HHHHxx +7748 6890 0 0 8 8 48 748 1748 2748 7748 96 97 AMAAAA AFKAAA OOOOxx +687 6891 1 3 7 7 87 687 687 687 687 174 175 LAAAAA BFKAAA VVVVxx +1243 6892 1 3 3 3 43 243 1243 1243 1243 86 87 VVAAAA CFKAAA AAAAxx +852 6893 0 0 2 12 52 852 852 852 852 104 105 UGAAAA DFKAAA HHHHxx +785 6894 1 1 5 5 85 785 785 785 785 170 171 FEAAAA EFKAAA OOOOxx +2002 6895 0 2 2 2 2 2 2 2002 2002 4 5 AZAAAA FFKAAA VVVVxx +2748 6896 0 0 8 8 48 748 748 2748 2748 96 97 SBAAAA GFKAAA AAAAxx +6075 6897 1 3 5 15 75 75 75 1075 6075 150 151 RZAAAA HFKAAA HHHHxx +7029 6898 1 1 9 9 29 29 1029 2029 7029 58 59 JKAAAA IFKAAA OOOOxx +7474 6899 0 2 4 14 74 474 1474 2474 7474 148 149 MBAAAA JFKAAA VVVVxx +7755 6900 1 3 5 15 55 755 1755 2755 7755 110 111 HMAAAA KFKAAA AAAAxx +1456 6901 0 0 6 16 56 456 1456 1456 1456 112 113 AEAAAA LFKAAA HHHHxx +2808 6902 0 0 8 8 8 808 808 2808 2808 16 17 AEAAAA MFKAAA OOOOxx +4089 6903 1 1 9 9 89 89 89 4089 4089 178 179 HBAAAA NFKAAA VVVVxx +4718 6904 0 2 8 18 18 718 718 4718 4718 36 37 MZAAAA OFKAAA AAAAxx +910 6905 0 2 0 10 10 910 910 910 910 20 21 AJAAAA PFKAAA HHHHxx +2868 6906 0 0 8 8 68 868 868 2868 2868 136 137 IGAAAA QFKAAA OOOOxx +2103 6907 1 3 3 3 3 103 103 2103 2103 6 7 XCAAAA RFKAAA VVVVxx +2407 6908 1 3 7 7 7 407 407 2407 2407 14 15 POAAAA SFKAAA AAAAxx +4353 6909 1 1 3 13 53 353 353 4353 4353 106 107 LLAAAA TFKAAA HHHHxx +7988 6910 0 0 8 8 88 988 1988 2988 7988 176 177 GVAAAA UFKAAA OOOOxx +2750 6911 0 2 0 10 50 750 750 2750 2750 100 101 UBAAAA VFKAAA VVVVxx +2006 6912 0 2 6 6 6 6 6 2006 2006 12 13 EZAAAA WFKAAA AAAAxx +4617 6913 1 1 7 17 17 617 617 4617 4617 34 35 PVAAAA XFKAAA HHHHxx +1251 6914 1 3 1 11 51 251 1251 1251 1251 102 103 DWAAAA YFKAAA OOOOxx +4590 6915 0 2 0 10 90 590 590 4590 4590 180 181 OUAAAA ZFKAAA VVVVxx +1144 6916 0 0 4 4 44 144 1144 1144 1144 88 89 ASAAAA AGKAAA AAAAxx +7131 6917 1 3 1 11 31 131 1131 2131 7131 62 63 HOAAAA BGKAAA HHHHxx +95 6918 1 3 5 15 95 95 95 95 95 190 191 RDAAAA CGKAAA OOOOxx +4827 6919 1 3 7 7 27 827 827 4827 4827 54 55 RDAAAA DGKAAA VVVVxx +4307 6920 1 3 7 7 7 307 307 4307 4307 14 15 RJAAAA EGKAAA AAAAxx +1505 6921 1 1 5 5 5 505 1505 1505 1505 10 11 XFAAAA FGKAAA HHHHxx +8191 6922 1 3 1 11 91 191 191 3191 8191 182 183 BDAAAA GGKAAA OOOOxx +5037 6923 1 1 7 17 37 37 1037 37 5037 74 75 TLAAAA HGKAAA VVVVxx +7363 6924 1 3 3 3 63 363 1363 2363 7363 126 127 FXAAAA IGKAAA AAAAxx +8427 6925 1 3 7 7 27 427 427 3427 8427 54 55 DMAAAA JGKAAA HHHHxx +5231 6926 1 3 1 11 31 231 1231 231 5231 62 63 FTAAAA KGKAAA OOOOxx +2943 6927 1 3 3 3 43 943 943 2943 2943 86 87 FJAAAA LGKAAA VVVVxx +4624 6928 0 0 4 4 24 624 624 4624 4624 48 49 WVAAAA MGKAAA AAAAxx +2020 6929 0 0 0 0 20 20 20 2020 2020 40 41 SZAAAA NGKAAA HHHHxx +6155 6930 1 3 5 15 55 155 155 1155 6155 110 111 TCAAAA OGKAAA OOOOxx +4381 6931 1 1 1 1 81 381 381 4381 4381 162 163 NMAAAA PGKAAA VVVVxx +1057 6932 1 1 7 17 57 57 1057 1057 1057 114 115 ROAAAA QGKAAA AAAAxx +9010 6933 0 2 0 10 10 10 1010 4010 9010 20 21 OIAAAA RGKAAA HHHHxx +4947 6934 1 3 7 7 47 947 947 4947 4947 94 95 HIAAAA SGKAAA OOOOxx +335 6935 1 3 5 15 35 335 335 335 335 70 71 XMAAAA TGKAAA VVVVxx +6890 6936 0 2 0 10 90 890 890 1890 6890 180 181 AFAAAA UGKAAA AAAAxx +5070 6937 0 2 0 10 70 70 1070 70 5070 140 141 ANAAAA VGKAAA HHHHxx +5270 6938 0 2 0 10 70 270 1270 270 5270 140 141 SUAAAA WGKAAA OOOOxx +8657 6939 1 1 7 17 57 657 657 3657 8657 114 115 ZUAAAA XGKAAA VVVVxx +7625 6940 1 1 5 5 25 625 1625 2625 7625 50 51 HHAAAA YGKAAA AAAAxx +5759 6941 1 3 9 19 59 759 1759 759 5759 118 119 NNAAAA ZGKAAA HHHHxx +9483 6942 1 3 3 3 83 483 1483 4483 9483 166 167 TAAAAA AHKAAA OOOOxx +8304 6943 0 0 4 4 4 304 304 3304 8304 8 9 KHAAAA BHKAAA VVVVxx +296 6944 0 0 6 16 96 296 296 296 296 192 193 KLAAAA CHKAAA AAAAxx +1176 6945 0 0 6 16 76 176 1176 1176 1176 152 153 GTAAAA DHKAAA HHHHxx +2069 6946 1 1 9 9 69 69 69 2069 2069 138 139 PBAAAA EHKAAA OOOOxx +1531 6947 1 3 1 11 31 531 1531 1531 1531 62 63 XGAAAA FHKAAA VVVVxx +5329 6948 1 1 9 9 29 329 1329 329 5329 58 59 ZWAAAA GHKAAA AAAAxx +3702 6949 0 2 2 2 2 702 1702 3702 3702 4 5 KMAAAA HHKAAA HHHHxx +6520 6950 0 0 0 0 20 520 520 1520 6520 40 41 UQAAAA IHKAAA OOOOxx +7310 6951 0 2 0 10 10 310 1310 2310 7310 20 21 EVAAAA JHKAAA VVVVxx +1175 6952 1 3 5 15 75 175 1175 1175 1175 150 151 FTAAAA KHKAAA AAAAxx +9107 6953 1 3 7 7 7 107 1107 4107 9107 14 15 HMAAAA LHKAAA HHHHxx +2737 6954 1 1 7 17 37 737 737 2737 2737 74 75 HBAAAA MHKAAA OOOOxx +3437 6955 1 1 7 17 37 437 1437 3437 3437 74 75 FCAAAA NHKAAA VVVVxx +281 6956 1 1 1 1 81 281 281 281 281 162 163 VKAAAA OHKAAA AAAAxx +6676 6957 0 0 6 16 76 676 676 1676 6676 152 153 UWAAAA PHKAAA HHHHxx +145 6958 1 1 5 5 45 145 145 145 145 90 91 PFAAAA QHKAAA OOOOxx +3172 6959 0 0 2 12 72 172 1172 3172 3172 144 145 ASAAAA RHKAAA VVVVxx +4049 6960 1 1 9 9 49 49 49 4049 4049 98 99 TZAAAA SHKAAA AAAAxx +6042 6961 0 2 2 2 42 42 42 1042 6042 84 85 KYAAAA THKAAA HHHHxx +9122 6962 0 2 2 2 22 122 1122 4122 9122 44 45 WMAAAA UHKAAA OOOOxx +7244 6963 0 0 4 4 44 244 1244 2244 7244 88 89 QSAAAA VHKAAA VVVVxx +5361 6964 1 1 1 1 61 361 1361 361 5361 122 123 FYAAAA WHKAAA AAAAxx +8647 6965 1 3 7 7 47 647 647 3647 8647 94 95 PUAAAA XHKAAA HHHHxx +7956 6966 0 0 6 16 56 956 1956 2956 7956 112 113 AUAAAA YHKAAA OOOOxx +7812 6967 0 0 2 12 12 812 1812 2812 7812 24 25 MOAAAA ZHKAAA VVVVxx +570 6968 0 2 0 10 70 570 570 570 570 140 141 YVAAAA AIKAAA AAAAxx +4115 6969 1 3 5 15 15 115 115 4115 4115 30 31 HCAAAA BIKAAA HHHHxx +1856 6970 0 0 6 16 56 856 1856 1856 1856 112 113 KTAAAA CIKAAA OOOOxx +9582 6971 0 2 2 2 82 582 1582 4582 9582 164 165 OEAAAA DIKAAA VVVVxx +2025 6972 1 1 5 5 25 25 25 2025 2025 50 51 XZAAAA EIKAAA AAAAxx +986 6973 0 2 6 6 86 986 986 986 986 172 173 YLAAAA FIKAAA HHHHxx +8358 6974 0 2 8 18 58 358 358 3358 8358 116 117 MJAAAA GIKAAA OOOOxx +510 6975 0 2 0 10 10 510 510 510 510 20 21 QTAAAA HIKAAA VVVVxx +6101 6976 1 1 1 1 1 101 101 1101 6101 2 3 RAAAAA IIKAAA AAAAxx +4167 6977 1 3 7 7 67 167 167 4167 4167 134 135 HEAAAA JIKAAA HHHHxx +6139 6978 1 3 9 19 39 139 139 1139 6139 78 79 DCAAAA KIKAAA OOOOxx +6912 6979 0 0 2 12 12 912 912 1912 6912 24 25 WFAAAA LIKAAA VVVVxx +339 6980 1 3 9 19 39 339 339 339 339 78 79 BNAAAA MIKAAA AAAAxx +8759 6981 1 3 9 19 59 759 759 3759 8759 118 119 XYAAAA NIKAAA HHHHxx +246 6982 0 2 6 6 46 246 246 246 246 92 93 MJAAAA OIKAAA OOOOxx +2831 6983 1 3 1 11 31 831 831 2831 2831 62 63 XEAAAA PIKAAA VVVVxx +2327 6984 1 3 7 7 27 327 327 2327 2327 54 55 NLAAAA QIKAAA AAAAxx +7001 6985 1 1 1 1 1 1 1001 2001 7001 2 3 HJAAAA RIKAAA HHHHxx +4398 6986 0 2 8 18 98 398 398 4398 4398 196 197 ENAAAA SIKAAA OOOOxx +1495 6987 1 3 5 15 95 495 1495 1495 1495 190 191 NFAAAA TIKAAA VVVVxx +8522 6988 0 2 2 2 22 522 522 3522 8522 44 45 UPAAAA UIKAAA AAAAxx +7090 6989 0 2 0 10 90 90 1090 2090 7090 180 181 SMAAAA VIKAAA HHHHxx +8457 6990 1 1 7 17 57 457 457 3457 8457 114 115 HNAAAA WIKAAA OOOOxx +4238 6991 0 2 8 18 38 238 238 4238 4238 76 77 AHAAAA XIKAAA VVVVxx +6791 6992 1 3 1 11 91 791 791 1791 6791 182 183 FBAAAA YIKAAA AAAAxx +1342 6993 0 2 2 2 42 342 1342 1342 1342 84 85 QZAAAA ZIKAAA HHHHxx +4580 6994 0 0 0 0 80 580 580 4580 4580 160 161 EUAAAA AJKAAA OOOOxx +1475 6995 1 3 5 15 75 475 1475 1475 1475 150 151 TEAAAA BJKAAA VVVVxx +9184 6996 0 0 4 4 84 184 1184 4184 9184 168 169 GPAAAA CJKAAA AAAAxx +1189 6997 1 1 9 9 89 189 1189 1189 1189 178 179 TTAAAA DJKAAA HHHHxx +638 6998 0 2 8 18 38 638 638 638 638 76 77 OYAAAA EJKAAA OOOOxx +5867 6999 1 3 7 7 67 867 1867 867 5867 134 135 RRAAAA FJKAAA VVVVxx +9911 7000 1 3 1 11 11 911 1911 4911 9911 22 23 FRAAAA GJKAAA AAAAxx +8147 7001 1 3 7 7 47 147 147 3147 8147 94 95 JBAAAA HJKAAA HHHHxx +4492 7002 0 0 2 12 92 492 492 4492 4492 184 185 UQAAAA IJKAAA OOOOxx +385 7003 1 1 5 5 85 385 385 385 385 170 171 VOAAAA JJKAAA VVVVxx +5235 7004 1 3 5 15 35 235 1235 235 5235 70 71 JTAAAA KJKAAA AAAAxx +4812 7005 0 0 2 12 12 812 812 4812 4812 24 25 CDAAAA LJKAAA HHHHxx +9807 7006 1 3 7 7 7 807 1807 4807 9807 14 15 FNAAAA MJKAAA OOOOxx +9588 7007 0 0 8 8 88 588 1588 4588 9588 176 177 UEAAAA NJKAAA VVVVxx +9832 7008 0 0 2 12 32 832 1832 4832 9832 64 65 EOAAAA OJKAAA AAAAxx +3757 7009 1 1 7 17 57 757 1757 3757 3757 114 115 NOAAAA PJKAAA HHHHxx +9703 7010 1 3 3 3 3 703 1703 4703 9703 6 7 FJAAAA QJKAAA OOOOxx +1022 7011 0 2 2 2 22 22 1022 1022 1022 44 45 INAAAA RJKAAA VVVVxx +5165 7012 1 1 5 5 65 165 1165 165 5165 130 131 RQAAAA SJKAAA AAAAxx +7129 7013 1 1 9 9 29 129 1129 2129 7129 58 59 FOAAAA TJKAAA HHHHxx +4164 7014 0 0 4 4 64 164 164 4164 4164 128 129 EEAAAA UJKAAA OOOOxx +7239 7015 1 3 9 19 39 239 1239 2239 7239 78 79 LSAAAA VJKAAA VVVVxx +523 7016 1 3 3 3 23 523 523 523 523 46 47 DUAAAA WJKAAA AAAAxx +4670 7017 0 2 0 10 70 670 670 4670 4670 140 141 QXAAAA XJKAAA HHHHxx +8503 7018 1 3 3 3 3 503 503 3503 8503 6 7 BPAAAA YJKAAA OOOOxx +714 7019 0 2 4 14 14 714 714 714 714 28 29 MBAAAA ZJKAAA VVVVxx +1350 7020 0 2 0 10 50 350 1350 1350 1350 100 101 YZAAAA AKKAAA AAAAxx +8318 7021 0 2 8 18 18 318 318 3318 8318 36 37 YHAAAA BKKAAA HHHHxx +1834 7022 0 2 4 14 34 834 1834 1834 1834 68 69 OSAAAA CKKAAA OOOOxx +4306 7023 0 2 6 6 6 306 306 4306 4306 12 13 QJAAAA DKKAAA VVVVxx +8543 7024 1 3 3 3 43 543 543 3543 8543 86 87 PQAAAA EKKAAA AAAAxx +9397 7025 1 1 7 17 97 397 1397 4397 9397 194 195 LXAAAA FKKAAA HHHHxx +3145 7026 1 1 5 5 45 145 1145 3145 3145 90 91 ZQAAAA GKKAAA OOOOxx +3942 7027 0 2 2 2 42 942 1942 3942 3942 84 85 QVAAAA HKKAAA VVVVxx +8583 7028 1 3 3 3 83 583 583 3583 8583 166 167 DSAAAA IKKAAA AAAAxx +8073 7029 1 1 3 13 73 73 73 3073 8073 146 147 NYAAAA JKKAAA HHHHxx +4940 7030 0 0 0 0 40 940 940 4940 4940 80 81 AIAAAA KKKAAA OOOOxx +9573 7031 1 1 3 13 73 573 1573 4573 9573 146 147 FEAAAA LKKAAA VVVVxx +5325 7032 1 1 5 5 25 325 1325 325 5325 50 51 VWAAAA MKKAAA AAAAxx +1833 7033 1 1 3 13 33 833 1833 1833 1833 66 67 NSAAAA NKKAAA HHHHxx +1337 7034 1 1 7 17 37 337 1337 1337 1337 74 75 LZAAAA OKKAAA OOOOxx +9749 7035 1 1 9 9 49 749 1749 4749 9749 98 99 ZKAAAA PKKAAA VVVVxx +7505 7036 1 1 5 5 5 505 1505 2505 7505 10 11 RCAAAA QKKAAA AAAAxx +9731 7037 1 3 1 11 31 731 1731 4731 9731 62 63 HKAAAA RKKAAA HHHHxx +4098 7038 0 2 8 18 98 98 98 4098 4098 196 197 QBAAAA SKKAAA OOOOxx +1418 7039 0 2 8 18 18 418 1418 1418 1418 36 37 OCAAAA TKKAAA VVVVxx +63 7040 1 3 3 3 63 63 63 63 63 126 127 LCAAAA UKKAAA AAAAxx +9889 7041 1 1 9 9 89 889 1889 4889 9889 178 179 JQAAAA VKKAAA HHHHxx +2871 7042 1 3 1 11 71 871 871 2871 2871 142 143 LGAAAA WKKAAA OOOOxx +1003 7043 1 3 3 3 3 3 1003 1003 1003 6 7 PMAAAA XKKAAA VVVVxx +8796 7044 0 0 6 16 96 796 796 3796 8796 192 193 IAAAAA YKKAAA AAAAxx +22 7045 0 2 2 2 22 22 22 22 22 44 45 WAAAAA ZKKAAA HHHHxx +8244 7046 0 0 4 4 44 244 244 3244 8244 88 89 CFAAAA ALKAAA OOOOxx +2282 7047 0 2 2 2 82 282 282 2282 2282 164 165 UJAAAA BLKAAA VVVVxx +3487 7048 1 3 7 7 87 487 1487 3487 3487 174 175 DEAAAA CLKAAA AAAAxx +8633 7049 1 1 3 13 33 633 633 3633 8633 66 67 BUAAAA DLKAAA HHHHxx +6418 7050 0 2 8 18 18 418 418 1418 6418 36 37 WMAAAA ELKAAA OOOOxx +4682 7051 0 2 2 2 82 682 682 4682 4682 164 165 CYAAAA FLKAAA VVVVxx +4103 7052 1 3 3 3 3 103 103 4103 4103 6 7 VBAAAA GLKAAA AAAAxx +6256 7053 0 0 6 16 56 256 256 1256 6256 112 113 QGAAAA HLKAAA HHHHxx +4040 7054 0 0 0 0 40 40 40 4040 4040 80 81 KZAAAA ILKAAA OOOOxx +9342 7055 0 2 2 2 42 342 1342 4342 9342 84 85 IVAAAA JLKAAA VVVVxx +9969 7056 1 1 9 9 69 969 1969 4969 9969 138 139 LTAAAA KLKAAA AAAAxx +223 7057 1 3 3 3 23 223 223 223 223 46 47 PIAAAA LLKAAA HHHHxx +4593 7058 1 1 3 13 93 593 593 4593 4593 186 187 RUAAAA MLKAAA OOOOxx +44 7059 0 0 4 4 44 44 44 44 44 88 89 SBAAAA NLKAAA VVVVxx +3513 7060 1 1 3 13 13 513 1513 3513 3513 26 27 DFAAAA OLKAAA AAAAxx +5771 7061 1 3 1 11 71 771 1771 771 5771 142 143 ZNAAAA PLKAAA HHHHxx +5083 7062 1 3 3 3 83 83 1083 83 5083 166 167 NNAAAA QLKAAA OOOOxx +3839 7063 1 3 9 19 39 839 1839 3839 3839 78 79 RRAAAA RLKAAA VVVVxx +2986 7064 0 2 6 6 86 986 986 2986 2986 172 173 WKAAAA SLKAAA AAAAxx +2200 7065 0 0 0 0 0 200 200 2200 2200 0 1 QGAAAA TLKAAA HHHHxx +197 7066 1 1 7 17 97 197 197 197 197 194 195 PHAAAA ULKAAA OOOOxx +7455 7067 1 3 5 15 55 455 1455 2455 7455 110 111 TAAAAA VLKAAA VVVVxx +1379 7068 1 3 9 19 79 379 1379 1379 1379 158 159 BBAAAA WLKAAA AAAAxx +4356 7069 0 0 6 16 56 356 356 4356 4356 112 113 OLAAAA XLKAAA HHHHxx +6888 7070 0 0 8 8 88 888 888 1888 6888 176 177 YEAAAA YLKAAA OOOOxx +9139 7071 1 3 9 19 39 139 1139 4139 9139 78 79 NNAAAA ZLKAAA VVVVxx +7682 7072 0 2 2 2 82 682 1682 2682 7682 164 165 MJAAAA AMKAAA AAAAxx +4873 7073 1 1 3 13 73 873 873 4873 4873 146 147 LFAAAA BMKAAA HHHHxx +783 7074 1 3 3 3 83 783 783 783 783 166 167 DEAAAA CMKAAA OOOOxx +6071 7075 1 3 1 11 71 71 71 1071 6071 142 143 NZAAAA DMKAAA VVVVxx +5160 7076 0 0 0 0 60 160 1160 160 5160 120 121 MQAAAA EMKAAA AAAAxx +2291 7077 1 3 1 11 91 291 291 2291 2291 182 183 DKAAAA FMKAAA HHHHxx +187 7078 1 3 7 7 87 187 187 187 187 174 175 FHAAAA GMKAAA OOOOxx +7786 7079 0 2 6 6 86 786 1786 2786 7786 172 173 MNAAAA HMKAAA VVVVxx +3432 7080 0 0 2 12 32 432 1432 3432 3432 64 65 ACAAAA IMKAAA AAAAxx +5450 7081 0 2 0 10 50 450 1450 450 5450 100 101 QBAAAA JMKAAA HHHHxx +2699 7082 1 3 9 19 99 699 699 2699 2699 198 199 VZAAAA KMKAAA OOOOxx +692 7083 0 0 2 12 92 692 692 692 692 184 185 QAAAAA LMKAAA VVVVxx +6081 7084 1 1 1 1 81 81 81 1081 6081 162 163 XZAAAA MMKAAA AAAAxx +4829 7085 1 1 9 9 29 829 829 4829 4829 58 59 TDAAAA NMKAAA HHHHxx +238 7086 0 2 8 18 38 238 238 238 238 76 77 EJAAAA OMKAAA OOOOxx +9100 7087 0 0 0 0 0 100 1100 4100 9100 0 1 AMAAAA PMKAAA VVVVxx +1968 7088 0 0 8 8 68 968 1968 1968 1968 136 137 SXAAAA QMKAAA AAAAxx +1872 7089 0 0 2 12 72 872 1872 1872 1872 144 145 AUAAAA RMKAAA HHHHxx +7051 7090 1 3 1 11 51 51 1051 2051 7051 102 103 FLAAAA SMKAAA OOOOxx +2743 7091 1 3 3 3 43 743 743 2743 2743 86 87 NBAAAA TMKAAA VVVVxx +1237 7092 1 1 7 17 37 237 1237 1237 1237 74 75 PVAAAA UMKAAA AAAAxx +3052 7093 0 0 2 12 52 52 1052 3052 3052 104 105 KNAAAA VMKAAA HHHHxx +8021 7094 1 1 1 1 21 21 21 3021 8021 42 43 NWAAAA WMKAAA OOOOxx +657 7095 1 1 7 17 57 657 657 657 657 114 115 HZAAAA XMKAAA VVVVxx +2236 7096 0 0 6 16 36 236 236 2236 2236 72 73 AIAAAA YMKAAA AAAAxx +7011 7097 1 3 1 11 11 11 1011 2011 7011 22 23 RJAAAA ZMKAAA HHHHxx +4067 7098 1 3 7 7 67 67 67 4067 4067 134 135 LAAAAA ANKAAA OOOOxx +9449 7099 1 1 9 9 49 449 1449 4449 9449 98 99 LZAAAA BNKAAA VVVVxx +7428 7100 0 0 8 8 28 428 1428 2428 7428 56 57 SZAAAA CNKAAA AAAAxx +1272 7101 0 0 2 12 72 272 1272 1272 1272 144 145 YWAAAA DNKAAA HHHHxx +6897 7102 1 1 7 17 97 897 897 1897 6897 194 195 HFAAAA ENKAAA OOOOxx +5839 7103 1 3 9 19 39 839 1839 839 5839 78 79 PQAAAA FNKAAA VVVVxx +6835 7104 1 3 5 15 35 835 835 1835 6835 70 71 XCAAAA GNKAAA AAAAxx +1887 7105 1 3 7 7 87 887 1887 1887 1887 174 175 PUAAAA HNKAAA HHHHxx +1551 7106 1 3 1 11 51 551 1551 1551 1551 102 103 RHAAAA INKAAA OOOOxx +4667 7107 1 3 7 7 67 667 667 4667 4667 134 135 NXAAAA JNKAAA VVVVxx +9603 7108 1 3 3 3 3 603 1603 4603 9603 6 7 JFAAAA KNKAAA AAAAxx +4332 7109 0 0 2 12 32 332 332 4332 4332 64 65 QKAAAA LNKAAA HHHHxx +5681 7110 1 1 1 1 81 681 1681 681 5681 162 163 NKAAAA MNKAAA OOOOxx +8062 7111 0 2 2 2 62 62 62 3062 8062 124 125 CYAAAA NNKAAA VVVVxx +2302 7112 0 2 2 2 2 302 302 2302 2302 4 5 OKAAAA ONKAAA AAAAxx +2825 7113 1 1 5 5 25 825 825 2825 2825 50 51 REAAAA PNKAAA HHHHxx +4527 7114 1 3 7 7 27 527 527 4527 4527 54 55 DSAAAA QNKAAA OOOOxx +4230 7115 0 2 0 10 30 230 230 4230 4230 60 61 SGAAAA RNKAAA VVVVxx +3053 7116 1 1 3 13 53 53 1053 3053 3053 106 107 LNAAAA SNKAAA AAAAxx +983 7117 1 3 3 3 83 983 983 983 983 166 167 VLAAAA TNKAAA HHHHxx +9458 7118 0 2 8 18 58 458 1458 4458 9458 116 117 UZAAAA UNKAAA OOOOxx +4128 7119 0 0 8 8 28 128 128 4128 4128 56 57 UCAAAA VNKAAA VVVVxx +425 7120 1 1 5 5 25 425 425 425 425 50 51 JQAAAA WNKAAA AAAAxx +3911 7121 1 3 1 11 11 911 1911 3911 3911 22 23 LUAAAA XNKAAA HHHHxx +6607 7122 1 3 7 7 7 607 607 1607 6607 14 15 DUAAAA YNKAAA OOOOxx +5431 7123 1 3 1 11 31 431 1431 431 5431 62 63 XAAAAA ZNKAAA VVVVxx +6330 7124 0 2 0 10 30 330 330 1330 6330 60 61 MJAAAA AOKAAA AAAAxx +3592 7125 0 0 2 12 92 592 1592 3592 3592 184 185 EIAAAA BOKAAA HHHHxx +154 7126 0 2 4 14 54 154 154 154 154 108 109 YFAAAA COKAAA OOOOxx +9879 7127 1 3 9 19 79 879 1879 4879 9879 158 159 ZPAAAA DOKAAA VVVVxx +3202 7128 0 2 2 2 2 202 1202 3202 3202 4 5 ETAAAA EOKAAA AAAAxx +3056 7129 0 0 6 16 56 56 1056 3056 3056 112 113 ONAAAA FOKAAA HHHHxx +9890 7130 0 2 0 10 90 890 1890 4890 9890 180 181 KQAAAA GOKAAA OOOOxx +5840 7131 0 0 0 0 40 840 1840 840 5840 80 81 QQAAAA HOKAAA VVVVxx +9804 7132 0 0 4 4 4 804 1804 4804 9804 8 9 CNAAAA IOKAAA AAAAxx +681 7133 1 1 1 1 81 681 681 681 681 162 163 FAAAAA JOKAAA HHHHxx +3443 7134 1 3 3 3 43 443 1443 3443 3443 86 87 LCAAAA KOKAAA OOOOxx +8088 7135 0 0 8 8 88 88 88 3088 8088 176 177 CZAAAA LOKAAA VVVVxx +9447 7136 1 3 7 7 47 447 1447 4447 9447 94 95 JZAAAA MOKAAA AAAAxx +1490 7137 0 2 0 10 90 490 1490 1490 1490 180 181 IFAAAA NOKAAA HHHHxx +3684 7138 0 0 4 4 84 684 1684 3684 3684 168 169 SLAAAA OOKAAA OOOOxx +3113 7139 1 1 3 13 13 113 1113 3113 3113 26 27 TPAAAA POKAAA VVVVxx +9004 7140 0 0 4 4 4 4 1004 4004 9004 8 9 IIAAAA QOKAAA AAAAxx +7147 7141 1 3 7 7 47 147 1147 2147 7147 94 95 XOAAAA ROKAAA HHHHxx +7571 7142 1 3 1 11 71 571 1571 2571 7571 142 143 FFAAAA SOKAAA OOOOxx +5545 7143 1 1 5 5 45 545 1545 545 5545 90 91 HFAAAA TOKAAA VVVVxx +4558 7144 0 2 8 18 58 558 558 4558 4558 116 117 ITAAAA UOKAAA AAAAxx +6206 7145 0 2 6 6 6 206 206 1206 6206 12 13 SEAAAA VOKAAA HHHHxx +5695 7146 1 3 5 15 95 695 1695 695 5695 190 191 BLAAAA WOKAAA OOOOxx +9600 7147 0 0 0 0 0 600 1600 4600 9600 0 1 GFAAAA XOKAAA VVVVxx +5432 7148 0 0 2 12 32 432 1432 432 5432 64 65 YAAAAA YOKAAA AAAAxx +9299 7149 1 3 9 19 99 299 1299 4299 9299 198 199 RTAAAA ZOKAAA HHHHxx +2386 7150 0 2 6 6 86 386 386 2386 2386 172 173 UNAAAA APKAAA OOOOxx +2046 7151 0 2 6 6 46 46 46 2046 2046 92 93 SAAAAA BPKAAA VVVVxx +3293 7152 1 1 3 13 93 293 1293 3293 3293 186 187 RWAAAA CPKAAA AAAAxx +3046 7153 0 2 6 6 46 46 1046 3046 3046 92 93 ENAAAA DPKAAA HHHHxx +214 7154 0 2 4 14 14 214 214 214 214 28 29 GIAAAA EPKAAA OOOOxx +7893 7155 1 1 3 13 93 893 1893 2893 7893 186 187 PRAAAA FPKAAA VVVVxx +891 7156 1 3 1 11 91 891 891 891 891 182 183 HIAAAA GPKAAA AAAAxx +6499 7157 1 3 9 19 99 499 499 1499 6499 198 199 ZPAAAA HPKAAA HHHHxx +5003 7158 1 3 3 3 3 3 1003 3 5003 6 7 LKAAAA IPKAAA OOOOxx +6487 7159 1 3 7 7 87 487 487 1487 6487 174 175 NPAAAA JPKAAA VVVVxx +9403 7160 1 3 3 3 3 403 1403 4403 9403 6 7 RXAAAA KPKAAA AAAAxx +945 7161 1 1 5 5 45 945 945 945 945 90 91 JKAAAA LPKAAA HHHHxx +6713 7162 1 1 3 13 13 713 713 1713 6713 26 27 FYAAAA MPKAAA OOOOxx +9928 7163 0 0 8 8 28 928 1928 4928 9928 56 57 WRAAAA NPKAAA VVVVxx +8585 7164 1 1 5 5 85 585 585 3585 8585 170 171 FSAAAA OPKAAA AAAAxx +4004 7165 0 0 4 4 4 4 4 4004 4004 8 9 AYAAAA PPKAAA HHHHxx +2528 7166 0 0 8 8 28 528 528 2528 2528 56 57 GTAAAA QPKAAA OOOOxx +3350 7167 0 2 0 10 50 350 1350 3350 3350 100 101 WYAAAA RPKAAA VVVVxx +2160 7168 0 0 0 0 60 160 160 2160 2160 120 121 CFAAAA SPKAAA AAAAxx +1521 7169 1 1 1 1 21 521 1521 1521 1521 42 43 NGAAAA TPKAAA HHHHxx +5660 7170 0 0 0 0 60 660 1660 660 5660 120 121 SJAAAA UPKAAA OOOOxx +5755 7171 1 3 5 15 55 755 1755 755 5755 110 111 JNAAAA VPKAAA VVVVxx +7614 7172 0 2 4 14 14 614 1614 2614 7614 28 29 WGAAAA WPKAAA AAAAxx +3121 7173 1 1 1 1 21 121 1121 3121 3121 42 43 BQAAAA XPKAAA HHHHxx +2735 7174 1 3 5 15 35 735 735 2735 2735 70 71 FBAAAA YPKAAA OOOOxx +7506 7175 0 2 6 6 6 506 1506 2506 7506 12 13 SCAAAA ZPKAAA VVVVxx +2693 7176 1 1 3 13 93 693 693 2693 2693 186 187 PZAAAA AQKAAA AAAAxx +2892 7177 0 0 2 12 92 892 892 2892 2892 184 185 GHAAAA BQKAAA HHHHxx +3310 7178 0 2 0 10 10 310 1310 3310 3310 20 21 IXAAAA CQKAAA OOOOxx +3484 7179 0 0 4 4 84 484 1484 3484 3484 168 169 AEAAAA DQKAAA VVVVxx +9733 7180 1 1 3 13 33 733 1733 4733 9733 66 67 JKAAAA EQKAAA AAAAxx +29 7181 1 1 9 9 29 29 29 29 29 58 59 DBAAAA FQKAAA HHHHxx +9013 7182 1 1 3 13 13 13 1013 4013 9013 26 27 RIAAAA GQKAAA OOOOxx +3847 7183 1 3 7 7 47 847 1847 3847 3847 94 95 ZRAAAA HQKAAA VVVVxx +6724 7184 0 0 4 4 24 724 724 1724 6724 48 49 QYAAAA IQKAAA AAAAxx +2559 7185 1 3 9 19 59 559 559 2559 2559 118 119 LUAAAA JQKAAA HHHHxx +5326 7186 0 2 6 6 26 326 1326 326 5326 52 53 WWAAAA KQKAAA OOOOxx +4802 7187 0 2 2 2 2 802 802 4802 4802 4 5 SCAAAA LQKAAA VVVVxx +131 7188 1 3 1 11 31 131 131 131 131 62 63 BFAAAA MQKAAA AAAAxx +1634 7189 0 2 4 14 34 634 1634 1634 1634 68 69 WKAAAA NQKAAA HHHHxx +919 7190 1 3 9 19 19 919 919 919 919 38 39 JJAAAA OQKAAA OOOOxx +9575 7191 1 3 5 15 75 575 1575 4575 9575 150 151 HEAAAA PQKAAA VVVVxx +1256 7192 0 0 6 16 56 256 1256 1256 1256 112 113 IWAAAA QQKAAA AAAAxx +9428 7193 0 0 8 8 28 428 1428 4428 9428 56 57 QYAAAA RQKAAA HHHHxx +5121 7194 1 1 1 1 21 121 1121 121 5121 42 43 ZOAAAA SQKAAA OOOOxx +6584 7195 0 0 4 4 84 584 584 1584 6584 168 169 GTAAAA TQKAAA VVVVxx +7193 7196 1 1 3 13 93 193 1193 2193 7193 186 187 RQAAAA UQKAAA AAAAxx +4047 7197 1 3 7 7 47 47 47 4047 4047 94 95 RZAAAA VQKAAA HHHHxx +104 7198 0 0 4 4 4 104 104 104 104 8 9 AEAAAA WQKAAA OOOOxx +1527 7199 1 3 7 7 27 527 1527 1527 1527 54 55 TGAAAA XQKAAA VVVVxx +3460 7200 0 0 0 0 60 460 1460 3460 3460 120 121 CDAAAA YQKAAA AAAAxx +8526 7201 0 2 6 6 26 526 526 3526 8526 52 53 YPAAAA ZQKAAA HHHHxx +8959 7202 1 3 9 19 59 959 959 3959 8959 118 119 PGAAAA ARKAAA OOOOxx +3633 7203 1 1 3 13 33 633 1633 3633 3633 66 67 TJAAAA BRKAAA VVVVxx +1799 7204 1 3 9 19 99 799 1799 1799 1799 198 199 FRAAAA CRKAAA AAAAxx +461 7205 1 1 1 1 61 461 461 461 461 122 123 TRAAAA DRKAAA HHHHxx +718 7206 0 2 8 18 18 718 718 718 718 36 37 QBAAAA ERKAAA OOOOxx +3219 7207 1 3 9 19 19 219 1219 3219 3219 38 39 VTAAAA FRKAAA VVVVxx +3494 7208 0 2 4 14 94 494 1494 3494 3494 188 189 KEAAAA GRKAAA AAAAxx +9402 7209 0 2 2 2 2 402 1402 4402 9402 4 5 QXAAAA HRKAAA HHHHxx +7983 7210 1 3 3 3 83 983 1983 2983 7983 166 167 BVAAAA IRKAAA OOOOxx +7919 7211 1 3 9 19 19 919 1919 2919 7919 38 39 PSAAAA JRKAAA VVVVxx +8036 7212 0 0 6 16 36 36 36 3036 8036 72 73 CXAAAA KRKAAA AAAAxx +5164 7213 0 0 4 4 64 164 1164 164 5164 128 129 QQAAAA LRKAAA HHHHxx +4160 7214 0 0 0 0 60 160 160 4160 4160 120 121 AEAAAA MRKAAA OOOOxx +5370 7215 0 2 0 10 70 370 1370 370 5370 140 141 OYAAAA NRKAAA VVVVxx +5347 7216 1 3 7 7 47 347 1347 347 5347 94 95 RXAAAA ORKAAA AAAAxx +7109 7217 1 1 9 9 9 109 1109 2109 7109 18 19 LNAAAA PRKAAA HHHHxx +4826 7218 0 2 6 6 26 826 826 4826 4826 52 53 QDAAAA QRKAAA OOOOxx +1338 7219 0 2 8 18 38 338 1338 1338 1338 76 77 MZAAAA RRKAAA VVVVxx +2711 7220 1 3 1 11 11 711 711 2711 2711 22 23 HAAAAA SRKAAA AAAAxx +6299 7221 1 3 9 19 99 299 299 1299 6299 198 199 HIAAAA TRKAAA HHHHxx +1616 7222 0 0 6 16 16 616 1616 1616 1616 32 33 EKAAAA URKAAA OOOOxx +7519 7223 1 3 9 19 19 519 1519 2519 7519 38 39 FDAAAA VRKAAA VVVVxx +1262 7224 0 2 2 2 62 262 1262 1262 1262 124 125 OWAAAA WRKAAA AAAAxx +7228 7225 0 0 8 8 28 228 1228 2228 7228 56 57 ASAAAA XRKAAA HHHHxx +7892 7226 0 0 2 12 92 892 1892 2892 7892 184 185 ORAAAA YRKAAA OOOOxx +7929 7227 1 1 9 9 29 929 1929 2929 7929 58 59 ZSAAAA ZRKAAA VVVVxx +7705 7228 1 1 5 5 5 705 1705 2705 7705 10 11 JKAAAA ASKAAA AAAAxx +3111 7229 1 3 1 11 11 111 1111 3111 3111 22 23 RPAAAA BSKAAA HHHHxx +3066 7230 0 2 6 6 66 66 1066 3066 3066 132 133 YNAAAA CSKAAA OOOOxx +9559 7231 1 3 9 19 59 559 1559 4559 9559 118 119 RDAAAA DSKAAA VVVVxx +3787 7232 1 3 7 7 87 787 1787 3787 3787 174 175 RPAAAA ESKAAA AAAAxx +8710 7233 0 2 0 10 10 710 710 3710 8710 20 21 AXAAAA FSKAAA HHHHxx +4870 7234 0 2 0 10 70 870 870 4870 4870 140 141 IFAAAA GSKAAA OOOOxx +1883 7235 1 3 3 3 83 883 1883 1883 1883 166 167 LUAAAA HSKAAA VVVVxx +9689 7236 1 1 9 9 89 689 1689 4689 9689 178 179 RIAAAA ISKAAA AAAAxx +9491 7237 1 3 1 11 91 491 1491 4491 9491 182 183 BBAAAA JSKAAA HHHHxx +2035 7238 1 3 5 15 35 35 35 2035 2035 70 71 HAAAAA KSKAAA OOOOxx +655 7239 1 3 5 15 55 655 655 655 655 110 111 FZAAAA LSKAAA VVVVxx +6305 7240 1 1 5 5 5 305 305 1305 6305 10 11 NIAAAA MSKAAA AAAAxx +9423 7241 1 3 3 3 23 423 1423 4423 9423 46 47 LYAAAA NSKAAA HHHHxx +283 7242 1 3 3 3 83 283 283 283 283 166 167 XKAAAA OSKAAA OOOOxx +2607 7243 1 3 7 7 7 607 607 2607 2607 14 15 HWAAAA PSKAAA VVVVxx +7740 7244 0 0 0 0 40 740 1740 2740 7740 80 81 SLAAAA QSKAAA AAAAxx +6956 7245 0 0 6 16 56 956 956 1956 6956 112 113 OHAAAA RSKAAA HHHHxx +884 7246 0 0 4 4 84 884 884 884 884 168 169 AIAAAA SSKAAA OOOOxx +5730 7247 0 2 0 10 30 730 1730 730 5730 60 61 KMAAAA TSKAAA VVVVxx +3438 7248 0 2 8 18 38 438 1438 3438 3438 76 77 GCAAAA USKAAA AAAAxx +3250 7249 0 2 0 10 50 250 1250 3250 3250 100 101 AVAAAA VSKAAA HHHHxx +5470 7250 0 2 0 10 70 470 1470 470 5470 140 141 KCAAAA WSKAAA OOOOxx +2037 7251 1 1 7 17 37 37 37 2037 2037 74 75 JAAAAA XSKAAA VVVVxx +6593 7252 1 1 3 13 93 593 593 1593 6593 186 187 PTAAAA YSKAAA AAAAxx +3893 7253 1 1 3 13 93 893 1893 3893 3893 186 187 TTAAAA ZSKAAA HHHHxx +3200 7254 0 0 0 0 0 200 1200 3200 3200 0 1 CTAAAA ATKAAA OOOOxx +7125 7255 1 1 5 5 25 125 1125 2125 7125 50 51 BOAAAA BTKAAA VVVVxx +2295 7256 1 3 5 15 95 295 295 2295 2295 190 191 HKAAAA CTKAAA AAAAxx +2056 7257 0 0 6 16 56 56 56 2056 2056 112 113 CBAAAA DTKAAA HHHHxx +2962 7258 0 2 2 2 62 962 962 2962 2962 124 125 YJAAAA ETKAAA OOOOxx +993 7259 1 1 3 13 93 993 993 993 993 186 187 FMAAAA FTKAAA VVVVxx +9127 7260 1 3 7 7 27 127 1127 4127 9127 54 55 BNAAAA GTKAAA AAAAxx +2075 7261 1 3 5 15 75 75 75 2075 2075 150 151 VBAAAA HTKAAA HHHHxx +9338 7262 0 2 8 18 38 338 1338 4338 9338 76 77 EVAAAA ITKAAA OOOOxx +8100 7263 0 0 0 0 0 100 100 3100 8100 0 1 OZAAAA JTKAAA VVVVxx +5047 7264 1 3 7 7 47 47 1047 47 5047 94 95 DMAAAA KTKAAA AAAAxx +7032 7265 0 0 2 12 32 32 1032 2032 7032 64 65 MKAAAA LTKAAA HHHHxx +6374 7266 0 2 4 14 74 374 374 1374 6374 148 149 ELAAAA MTKAAA OOOOxx +4137 7267 1 1 7 17 37 137 137 4137 4137 74 75 DDAAAA NTKAAA VVVVxx +7132 7268 0 0 2 12 32 132 1132 2132 7132 64 65 IOAAAA OTKAAA AAAAxx +3064 7269 0 0 4 4 64 64 1064 3064 3064 128 129 WNAAAA PTKAAA HHHHxx +3621 7270 1 1 1 1 21 621 1621 3621 3621 42 43 HJAAAA QTKAAA OOOOxx +6199 7271 1 3 9 19 99 199 199 1199 6199 198 199 LEAAAA RTKAAA VVVVxx +4926 7272 0 2 6 6 26 926 926 4926 4926 52 53 MHAAAA STKAAA AAAAxx +8035 7273 1 3 5 15 35 35 35 3035 8035 70 71 BXAAAA TTKAAA HHHHxx +2195 7274 1 3 5 15 95 195 195 2195 2195 190 191 LGAAAA UTKAAA OOOOxx +5366 7275 0 2 6 6 66 366 1366 366 5366 132 133 KYAAAA VTKAAA VVVVxx +3478 7276 0 2 8 18 78 478 1478 3478 3478 156 157 UDAAAA WTKAAA AAAAxx +1926 7277 0 2 6 6 26 926 1926 1926 1926 52 53 CWAAAA XTKAAA HHHHxx +7265 7278 1 1 5 5 65 265 1265 2265 7265 130 131 LTAAAA YTKAAA OOOOxx +7668 7279 0 0 8 8 68 668 1668 2668 7668 136 137 YIAAAA ZTKAAA VVVVxx +3335 7280 1 3 5 15 35 335 1335 3335 3335 70 71 HYAAAA AUKAAA AAAAxx +7660 7281 0 0 0 0 60 660 1660 2660 7660 120 121 QIAAAA BUKAAA HHHHxx +9604 7282 0 0 4 4 4 604 1604 4604 9604 8 9 KFAAAA CUKAAA OOOOxx +7301 7283 1 1 1 1 1 301 1301 2301 7301 2 3 VUAAAA DUKAAA VVVVxx +4475 7284 1 3 5 15 75 475 475 4475 4475 150 151 DQAAAA EUKAAA AAAAxx +9954 7285 0 2 4 14 54 954 1954 4954 9954 108 109 WSAAAA FUKAAA HHHHxx +5723 7286 1 3 3 3 23 723 1723 723 5723 46 47 DMAAAA GUKAAA OOOOxx +2669 7287 1 1 9 9 69 669 669 2669 2669 138 139 RYAAAA HUKAAA VVVVxx +1685 7288 1 1 5 5 85 685 1685 1685 1685 170 171 VMAAAA IUKAAA AAAAxx +2233 7289 1 1 3 13 33 233 233 2233 2233 66 67 XHAAAA JUKAAA HHHHxx +8111 7290 1 3 1 11 11 111 111 3111 8111 22 23 ZZAAAA KUKAAA OOOOxx +7685 7291 1 1 5 5 85 685 1685 2685 7685 170 171 PJAAAA LUKAAA VVVVxx +3773 7292 1 1 3 13 73 773 1773 3773 3773 146 147 DPAAAA MUKAAA AAAAxx +7172 7293 0 0 2 12 72 172 1172 2172 7172 144 145 WPAAAA NUKAAA HHHHxx +1740 7294 0 0 0 0 40 740 1740 1740 1740 80 81 YOAAAA OUKAAA OOOOxx +5416 7295 0 0 6 16 16 416 1416 416 5416 32 33 IAAAAA PUKAAA VVVVxx +1823 7296 1 3 3 3 23 823 1823 1823 1823 46 47 DSAAAA QUKAAA AAAAxx +1668 7297 0 0 8 8 68 668 1668 1668 1668 136 137 EMAAAA RUKAAA HHHHxx +1795 7298 1 3 5 15 95 795 1795 1795 1795 190 191 BRAAAA SUKAAA OOOOxx +8599 7299 1 3 9 19 99 599 599 3599 8599 198 199 TSAAAA TUKAAA VVVVxx +5542 7300 0 2 2 2 42 542 1542 542 5542 84 85 EFAAAA UUKAAA AAAAxx +5658 7301 0 2 8 18 58 658 1658 658 5658 116 117 QJAAAA VUKAAA HHHHxx +9824 7302 0 0 4 4 24 824 1824 4824 9824 48 49 WNAAAA WUKAAA OOOOxx +19 7303 1 3 9 19 19 19 19 19 19 38 39 TAAAAA XUKAAA VVVVxx +9344 7304 0 0 4 4 44 344 1344 4344 9344 88 89 KVAAAA YUKAAA AAAAxx +5900 7305 0 0 0 0 0 900 1900 900 5900 0 1 YSAAAA ZUKAAA HHHHxx +7818 7306 0 2 8 18 18 818 1818 2818 7818 36 37 SOAAAA AVKAAA OOOOxx +8377 7307 1 1 7 17 77 377 377 3377 8377 154 155 FKAAAA BVKAAA VVVVxx +6886 7308 0 2 6 6 86 886 886 1886 6886 172 173 WEAAAA CVKAAA AAAAxx +3201 7309 1 1 1 1 1 201 1201 3201 3201 2 3 DTAAAA DVKAAA HHHHxx +87 7310 1 3 7 7 87 87 87 87 87 174 175 JDAAAA EVKAAA OOOOxx +1089 7311 1 1 9 9 89 89 1089 1089 1089 178 179 XPAAAA FVKAAA VVVVxx +3948 7312 0 0 8 8 48 948 1948 3948 3948 96 97 WVAAAA GVKAAA AAAAxx +6383 7313 1 3 3 3 83 383 383 1383 6383 166 167 NLAAAA HVKAAA HHHHxx +837 7314 1 1 7 17 37 837 837 837 837 74 75 FGAAAA IVKAAA OOOOxx +6285 7315 1 1 5 5 85 285 285 1285 6285 170 171 THAAAA JVKAAA VVVVxx +78 7316 0 2 8 18 78 78 78 78 78 156 157 ADAAAA KVKAAA AAAAxx +4389 7317 1 1 9 9 89 389 389 4389 4389 178 179 VMAAAA LVKAAA HHHHxx +4795 7318 1 3 5 15 95 795 795 4795 4795 190 191 LCAAAA MVKAAA OOOOxx +9369 7319 1 1 9 9 69 369 1369 4369 9369 138 139 JWAAAA NVKAAA VVVVxx +69 7320 1 1 9 9 69 69 69 69 69 138 139 RCAAAA OVKAAA AAAAxx +7689 7321 1 1 9 9 89 689 1689 2689 7689 178 179 TJAAAA PVKAAA HHHHxx +5642 7322 0 2 2 2 42 642 1642 642 5642 84 85 AJAAAA QVKAAA OOOOxx +2348 7323 0 0 8 8 48 348 348 2348 2348 96 97 IMAAAA RVKAAA VVVVxx +9308 7324 0 0 8 8 8 308 1308 4308 9308 16 17 AUAAAA SVKAAA AAAAxx +9093 7325 1 1 3 13 93 93 1093 4093 9093 186 187 TLAAAA TVKAAA HHHHxx +1199 7326 1 3 9 19 99 199 1199 1199 1199 198 199 DUAAAA UVKAAA OOOOxx +307 7327 1 3 7 7 7 307 307 307 307 14 15 VLAAAA VVKAAA VVVVxx +3814 7328 0 2 4 14 14 814 1814 3814 3814 28 29 SQAAAA WVKAAA AAAAxx +8817 7329 1 1 7 17 17 817 817 3817 8817 34 35 DBAAAA XVKAAA HHHHxx +2329 7330 1 1 9 9 29 329 329 2329 2329 58 59 PLAAAA YVKAAA OOOOxx +2932 7331 0 0 2 12 32 932 932 2932 2932 64 65 UIAAAA ZVKAAA VVVVxx +1986 7332 0 2 6 6 86 986 1986 1986 1986 172 173 KYAAAA AWKAAA AAAAxx +5279 7333 1 3 9 19 79 279 1279 279 5279 158 159 BVAAAA BWKAAA HHHHxx +5357 7334 1 1 7 17 57 357 1357 357 5357 114 115 BYAAAA CWKAAA OOOOxx +6778 7335 0 2 8 18 78 778 778 1778 6778 156 157 SAAAAA DWKAAA VVVVxx +2773 7336 1 1 3 13 73 773 773 2773 2773 146 147 RCAAAA EWKAAA AAAAxx +244 7337 0 0 4 4 44 244 244 244 244 88 89 KJAAAA FWKAAA HHHHxx +6900 7338 0 0 0 0 0 900 900 1900 6900 0 1 KFAAAA GWKAAA OOOOxx +4739 7339 1 3 9 19 39 739 739 4739 4739 78 79 HAAAAA HWKAAA VVVVxx +3217 7340 1 1 7 17 17 217 1217 3217 3217 34 35 TTAAAA IWKAAA AAAAxx +7563 7341 1 3 3 3 63 563 1563 2563 7563 126 127 XEAAAA JWKAAA HHHHxx +1807 7342 1 3 7 7 7 807 1807 1807 1807 14 15 NRAAAA KWKAAA OOOOxx +4199 7343 1 3 9 19 99 199 199 4199 4199 198 199 NFAAAA LWKAAA VVVVxx +1077 7344 1 1 7 17 77 77 1077 1077 1077 154 155 LPAAAA MWKAAA AAAAxx +8348 7345 0 0 8 8 48 348 348 3348 8348 96 97 CJAAAA NWKAAA HHHHxx +841 7346 1 1 1 1 41 841 841 841 841 82 83 JGAAAA OWKAAA OOOOxx +8154 7347 0 2 4 14 54 154 154 3154 8154 108 109 QBAAAA PWKAAA VVVVxx +5261 7348 1 1 1 1 61 261 1261 261 5261 122 123 JUAAAA QWKAAA AAAAxx +1950 7349 0 2 0 10 50 950 1950 1950 1950 100 101 AXAAAA RWKAAA HHHHxx +8472 7350 0 0 2 12 72 472 472 3472 8472 144 145 WNAAAA SWKAAA OOOOxx +8745 7351 1 1 5 5 45 745 745 3745 8745 90 91 JYAAAA TWKAAA VVVVxx +8715 7352 1 3 5 15 15 715 715 3715 8715 30 31 FXAAAA UWKAAA AAAAxx +9708 7353 0 0 8 8 8 708 1708 4708 9708 16 17 KJAAAA VWKAAA HHHHxx +5860 7354 0 0 0 0 60 860 1860 860 5860 120 121 KRAAAA WWKAAA OOOOxx +9142 7355 0 2 2 2 42 142 1142 4142 9142 84 85 QNAAAA XWKAAA VVVVxx +6582 7356 0 2 2 2 82 582 582 1582 6582 164 165 ETAAAA YWKAAA AAAAxx +1255 7357 1 3 5 15 55 255 1255 1255 1255 110 111 HWAAAA ZWKAAA HHHHxx +6459 7358 1 3 9 19 59 459 459 1459 6459 118 119 LOAAAA AXKAAA OOOOxx +6327 7359 1 3 7 7 27 327 327 1327 6327 54 55 JJAAAA BXKAAA VVVVxx +4692 7360 0 0 2 12 92 692 692 4692 4692 184 185 MYAAAA CXKAAA AAAAxx +3772 7361 0 0 2 12 72 772 1772 3772 3772 144 145 CPAAAA DXKAAA HHHHxx +4203 7362 1 3 3 3 3 203 203 4203 4203 6 7 RFAAAA EXKAAA OOOOxx +2946 7363 0 2 6 6 46 946 946 2946 2946 92 93 IJAAAA FXKAAA VVVVxx +3524 7364 0 0 4 4 24 524 1524 3524 3524 48 49 OFAAAA GXKAAA AAAAxx +8409 7365 1 1 9 9 9 409 409 3409 8409 18 19 LLAAAA HXKAAA HHHHxx +1824 7366 0 0 4 4 24 824 1824 1824 1824 48 49 ESAAAA IXKAAA OOOOxx +4637 7367 1 1 7 17 37 637 637 4637 4637 74 75 JWAAAA JXKAAA VVVVxx +589 7368 1 1 9 9 89 589 589 589 589 178 179 RWAAAA KXKAAA AAAAxx +484 7369 0 0 4 4 84 484 484 484 484 168 169 QSAAAA LXKAAA HHHHxx +8963 7370 1 3 3 3 63 963 963 3963 8963 126 127 TGAAAA MXKAAA OOOOxx +5502 7371 0 2 2 2 2 502 1502 502 5502 4 5 QDAAAA NXKAAA VVVVxx +6982 7372 0 2 2 2 82 982 982 1982 6982 164 165 OIAAAA OXKAAA AAAAxx +8029 7373 1 1 9 9 29 29 29 3029 8029 58 59 VWAAAA PXKAAA HHHHxx +4395 7374 1 3 5 15 95 395 395 4395 4395 190 191 BNAAAA QXKAAA OOOOxx +2595 7375 1 3 5 15 95 595 595 2595 2595 190 191 VVAAAA RXKAAA VVVVxx +2133 7376 1 1 3 13 33 133 133 2133 2133 66 67 BEAAAA SXKAAA AAAAxx +1414 7377 0 2 4 14 14 414 1414 1414 1414 28 29 KCAAAA TXKAAA HHHHxx +8201 7378 1 1 1 1 1 201 201 3201 8201 2 3 LDAAAA UXKAAA OOOOxx +4706 7379 0 2 6 6 6 706 706 4706 4706 12 13 AZAAAA VXKAAA VVVVxx +5310 7380 0 2 0 10 10 310 1310 310 5310 20 21 GWAAAA WXKAAA AAAAxx +7333 7381 1 1 3 13 33 333 1333 2333 7333 66 67 BWAAAA XXKAAA HHHHxx +9420 7382 0 0 0 0 20 420 1420 4420 9420 40 41 IYAAAA YXKAAA OOOOxx +1383 7383 1 3 3 3 83 383 1383 1383 1383 166 167 FBAAAA ZXKAAA VVVVxx +6225 7384 1 1 5 5 25 225 225 1225 6225 50 51 LFAAAA AYKAAA AAAAxx +2064 7385 0 0 4 4 64 64 64 2064 2064 128 129 KBAAAA BYKAAA HHHHxx +6700 7386 0 0 0 0 0 700 700 1700 6700 0 1 SXAAAA CYKAAA OOOOxx +1352 7387 0 0 2 12 52 352 1352 1352 1352 104 105 AAAAAA DYKAAA VVVVxx +4249 7388 1 1 9 9 49 249 249 4249 4249 98 99 LHAAAA EYKAAA AAAAxx +9429 7389 1 1 9 9 29 429 1429 4429 9429 58 59 RYAAAA FYKAAA HHHHxx +8090 7390 0 2 0 10 90 90 90 3090 8090 180 181 EZAAAA GYKAAA OOOOxx +5378 7391 0 2 8 18 78 378 1378 378 5378 156 157 WYAAAA HYKAAA VVVVxx +9085 7392 1 1 5 5 85 85 1085 4085 9085 170 171 LLAAAA IYKAAA AAAAxx +7468 7393 0 0 8 8 68 468 1468 2468 7468 136 137 GBAAAA JYKAAA HHHHxx +9955 7394 1 3 5 15 55 955 1955 4955 9955 110 111 XSAAAA KYKAAA OOOOxx +8692 7395 0 0 2 12 92 692 692 3692 8692 184 185 IWAAAA LYKAAA VVVVxx +1463 7396 1 3 3 3 63 463 1463 1463 1463 126 127 HEAAAA MYKAAA AAAAxx +3577 7397 1 1 7 17 77 577 1577 3577 3577 154 155 PHAAAA NYKAAA HHHHxx +5654 7398 0 2 4 14 54 654 1654 654 5654 108 109 MJAAAA OYKAAA OOOOxx +7955 7399 1 3 5 15 55 955 1955 2955 7955 110 111 ZTAAAA PYKAAA VVVVxx +4843 7400 1 3 3 3 43 843 843 4843 4843 86 87 HEAAAA QYKAAA AAAAxx +1776 7401 0 0 6 16 76 776 1776 1776 1776 152 153 IQAAAA RYKAAA HHHHxx +2223 7402 1 3 3 3 23 223 223 2223 2223 46 47 NHAAAA SYKAAA OOOOxx +8442 7403 0 2 2 2 42 442 442 3442 8442 84 85 SMAAAA TYKAAA VVVVxx +9738 7404 0 2 8 18 38 738 1738 4738 9738 76 77 OKAAAA UYKAAA AAAAxx +4867 7405 1 3 7 7 67 867 867 4867 4867 134 135 FFAAAA VYKAAA HHHHxx +2983 7406 1 3 3 3 83 983 983 2983 2983 166 167 TKAAAA WYKAAA OOOOxx +3300 7407 0 0 0 0 0 300 1300 3300 3300 0 1 YWAAAA XYKAAA VVVVxx +3815 7408 1 3 5 15 15 815 1815 3815 3815 30 31 TQAAAA YYKAAA AAAAxx +1779 7409 1 3 9 19 79 779 1779 1779 1779 158 159 LQAAAA ZYKAAA HHHHxx +1123 7410 1 3 3 3 23 123 1123 1123 1123 46 47 FRAAAA AZKAAA OOOOxx +4824 7411 0 0 4 4 24 824 824 4824 4824 48 49 ODAAAA BZKAAA VVVVxx +5407 7412 1 3 7 7 7 407 1407 407 5407 14 15 ZZAAAA CZKAAA AAAAxx +5123 7413 1 3 3 3 23 123 1123 123 5123 46 47 BPAAAA DZKAAA HHHHxx +2515 7414 1 3 5 15 15 515 515 2515 2515 30 31 TSAAAA EZKAAA OOOOxx +4781 7415 1 1 1 1 81 781 781 4781 4781 162 163 XBAAAA FZKAAA VVVVxx +7831 7416 1 3 1 11 31 831 1831 2831 7831 62 63 FPAAAA GZKAAA AAAAxx +6946 7417 0 2 6 6 46 946 946 1946 6946 92 93 EHAAAA HZKAAA HHHHxx +1215 7418 1 3 5 15 15 215 1215 1215 1215 30 31 TUAAAA IZKAAA OOOOxx +7783 7419 1 3 3 3 83 783 1783 2783 7783 166 167 JNAAAA JZKAAA VVVVxx +4532 7420 0 0 2 12 32 532 532 4532 4532 64 65 ISAAAA KZKAAA AAAAxx +9068 7421 0 0 8 8 68 68 1068 4068 9068 136 137 UKAAAA LZKAAA HHHHxx +7030 7422 0 2 0 10 30 30 1030 2030 7030 60 61 KKAAAA MZKAAA OOOOxx +436 7423 0 0 6 16 36 436 436 436 436 72 73 UQAAAA NZKAAA VVVVxx +6549 7424 1 1 9 9 49 549 549 1549 6549 98 99 XRAAAA OZKAAA AAAAxx +3348 7425 0 0 8 8 48 348 1348 3348 3348 96 97 UYAAAA PZKAAA HHHHxx +6229 7426 1 1 9 9 29 229 229 1229 6229 58 59 PFAAAA QZKAAA OOOOxx +3933 7427 1 1 3 13 33 933 1933 3933 3933 66 67 HVAAAA RZKAAA VVVVxx +1876 7428 0 0 6 16 76 876 1876 1876 1876 152 153 EUAAAA SZKAAA AAAAxx +8920 7429 0 0 0 0 20 920 920 3920 8920 40 41 CFAAAA TZKAAA HHHHxx +7926 7430 0 2 6 6 26 926 1926 2926 7926 52 53 WSAAAA UZKAAA OOOOxx +8805 7431 1 1 5 5 5 805 805 3805 8805 10 11 RAAAAA VZKAAA VVVVxx +6729 7432 1 1 9 9 29 729 729 1729 6729 58 59 VYAAAA WZKAAA AAAAxx +7397 7433 1 1 7 17 97 397 1397 2397 7397 194 195 NYAAAA XZKAAA HHHHxx +9303 7434 1 3 3 3 3 303 1303 4303 9303 6 7 VTAAAA YZKAAA OOOOxx +4255 7435 1 3 5 15 55 255 255 4255 4255 110 111 RHAAAA ZZKAAA VVVVxx +7229 7436 1 1 9 9 29 229 1229 2229 7229 58 59 BSAAAA AALAAA AAAAxx +854 7437 0 2 4 14 54 854 854 854 854 108 109 WGAAAA BALAAA HHHHxx +6723 7438 1 3 3 3 23 723 723 1723 6723 46 47 PYAAAA CALAAA OOOOxx +9597 7439 1 1 7 17 97 597 1597 4597 9597 194 195 DFAAAA DALAAA VVVVxx +6532 7440 0 0 2 12 32 532 532 1532 6532 64 65 GRAAAA EALAAA AAAAxx +2910 7441 0 2 0 10 10 910 910 2910 2910 20 21 YHAAAA FALAAA HHHHxx +6717 7442 1 1 7 17 17 717 717 1717 6717 34 35 JYAAAA GALAAA OOOOxx +1790 7443 0 2 0 10 90 790 1790 1790 1790 180 181 WQAAAA HALAAA VVVVxx +3761 7444 1 1 1 1 61 761 1761 3761 3761 122 123 ROAAAA IALAAA AAAAxx +1565 7445 1 1 5 5 65 565 1565 1565 1565 130 131 FIAAAA JALAAA HHHHxx +6205 7446 1 1 5 5 5 205 205 1205 6205 10 11 REAAAA KALAAA OOOOxx +2726 7447 0 2 6 6 26 726 726 2726 2726 52 53 WAAAAA LALAAA VVVVxx +799 7448 1 3 9 19 99 799 799 799 799 198 199 TEAAAA MALAAA AAAAxx +3540 7449 0 0 0 0 40 540 1540 3540 3540 80 81 EGAAAA NALAAA HHHHxx +5878 7450 0 2 8 18 78 878 1878 878 5878 156 157 CSAAAA OALAAA OOOOxx +2542 7451 0 2 2 2 42 542 542 2542 2542 84 85 UTAAAA PALAAA VVVVxx +4888 7452 0 0 8 8 88 888 888 4888 4888 176 177 AGAAAA QALAAA AAAAxx +5290 7453 0 2 0 10 90 290 1290 290 5290 180 181 MVAAAA RALAAA HHHHxx +7995 7454 1 3 5 15 95 995 1995 2995 7995 190 191 NVAAAA SALAAA OOOOxx +3519 7455 1 3 9 19 19 519 1519 3519 3519 38 39 JFAAAA TALAAA VVVVxx +3571 7456 1 3 1 11 71 571 1571 3571 3571 142 143 JHAAAA UALAAA AAAAxx +7854 7457 0 2 4 14 54 854 1854 2854 7854 108 109 CQAAAA VALAAA HHHHxx +5184 7458 0 0 4 4 84 184 1184 184 5184 168 169 KRAAAA WALAAA OOOOxx +3498 7459 0 2 8 18 98 498 1498 3498 3498 196 197 OEAAAA XALAAA VVVVxx +1264 7460 0 0 4 4 64 264 1264 1264 1264 128 129 QWAAAA YALAAA AAAAxx +3159 7461 1 3 9 19 59 159 1159 3159 3159 118 119 NRAAAA ZALAAA HHHHxx +5480 7462 0 0 0 0 80 480 1480 480 5480 160 161 UCAAAA ABLAAA OOOOxx +1706 7463 0 2 6 6 6 706 1706 1706 1706 12 13 QNAAAA BBLAAA VVVVxx +4540 7464 0 0 0 0 40 540 540 4540 4540 80 81 QSAAAA CBLAAA AAAAxx +2799 7465 1 3 9 19 99 799 799 2799 2799 198 199 RDAAAA DBLAAA HHHHxx +7389 7466 1 1 9 9 89 389 1389 2389 7389 178 179 FYAAAA EBLAAA OOOOxx +5565 7467 1 1 5 5 65 565 1565 565 5565 130 131 BGAAAA FBLAAA VVVVxx +3896 7468 0 0 6 16 96 896 1896 3896 3896 192 193 WTAAAA GBLAAA AAAAxx +2100 7469 0 0 0 0 0 100 100 2100 2100 0 1 UCAAAA HBLAAA HHHHxx +3507 7470 1 3 7 7 7 507 1507 3507 3507 14 15 XEAAAA IBLAAA OOOOxx +7971 7471 1 3 1 11 71 971 1971 2971 7971 142 143 PUAAAA JBLAAA VVVVxx +2312 7472 0 0 2 12 12 312 312 2312 2312 24 25 YKAAAA KBLAAA AAAAxx +2494 7473 0 2 4 14 94 494 494 2494 2494 188 189 YRAAAA LBLAAA HHHHxx +2474 7474 0 2 4 14 74 474 474 2474 2474 148 149 ERAAAA MBLAAA OOOOxx +3136 7475 0 0 6 16 36 136 1136 3136 3136 72 73 QQAAAA NBLAAA VVVVxx +7242 7476 0 2 2 2 42 242 1242 2242 7242 84 85 OSAAAA OBLAAA AAAAxx +9430 7477 0 2 0 10 30 430 1430 4430 9430 60 61 SYAAAA PBLAAA HHHHxx +1052 7478 0 0 2 12 52 52 1052 1052 1052 104 105 MOAAAA QBLAAA OOOOxx +4172 7479 0 0 2 12 72 172 172 4172 4172 144 145 MEAAAA RBLAAA VVVVxx +970 7480 0 2 0 10 70 970 970 970 970 140 141 ILAAAA SBLAAA AAAAxx +882 7481 0 2 2 2 82 882 882 882 882 164 165 YHAAAA TBLAAA HHHHxx +9799 7482 1 3 9 19 99 799 1799 4799 9799 198 199 XMAAAA UBLAAA OOOOxx +5850 7483 0 2 0 10 50 850 1850 850 5850 100 101 ARAAAA VBLAAA VVVVxx +9473 7484 1 1 3 13 73 473 1473 4473 9473 146 147 JAAAAA WBLAAA AAAAxx +8635 7485 1 3 5 15 35 635 635 3635 8635 70 71 DUAAAA XBLAAA HHHHxx +2349 7486 1 1 9 9 49 349 349 2349 2349 98 99 JMAAAA YBLAAA OOOOxx +2270 7487 0 2 0 10 70 270 270 2270 2270 140 141 IJAAAA ZBLAAA VVVVxx +7887 7488 1 3 7 7 87 887 1887 2887 7887 174 175 JRAAAA ACLAAA AAAAxx +3091 7489 1 3 1 11 91 91 1091 3091 3091 182 183 XOAAAA BCLAAA HHHHxx +3728 7490 0 0 8 8 28 728 1728 3728 3728 56 57 KNAAAA CCLAAA OOOOxx +3658 7491 0 2 8 18 58 658 1658 3658 3658 116 117 SKAAAA DCLAAA VVVVxx +5975 7492 1 3 5 15 75 975 1975 975 5975 150 151 VVAAAA ECLAAA AAAAxx +332 7493 0 0 2 12 32 332 332 332 332 64 65 UMAAAA FCLAAA HHHHxx +7990 7494 0 2 0 10 90 990 1990 2990 7990 180 181 IVAAAA GCLAAA OOOOxx +8688 7495 0 0 8 8 88 688 688 3688 8688 176 177 EWAAAA HCLAAA VVVVxx +9601 7496 1 1 1 1 1 601 1601 4601 9601 2 3 HFAAAA ICLAAA AAAAxx +8401 7497 1 1 1 1 1 401 401 3401 8401 2 3 DLAAAA JCLAAA HHHHxx +8093 7498 1 1 3 13 93 93 93 3093 8093 186 187 HZAAAA KCLAAA OOOOxx +4278 7499 0 2 8 18 78 278 278 4278 4278 156 157 OIAAAA LCLAAA VVVVxx +5467 7500 1 3 7 7 67 467 1467 467 5467 134 135 HCAAAA MCLAAA AAAAxx +3137 7501 1 1 7 17 37 137 1137 3137 3137 74 75 RQAAAA NCLAAA HHHHxx +204 7502 0 0 4 4 4 204 204 204 204 8 9 WHAAAA OCLAAA OOOOxx +8224 7503 0 0 4 4 24 224 224 3224 8224 48 49 IEAAAA PCLAAA VVVVxx +2944 7504 0 0 4 4 44 944 944 2944 2944 88 89 GJAAAA QCLAAA AAAAxx +7593 7505 1 1 3 13 93 593 1593 2593 7593 186 187 BGAAAA RCLAAA HHHHxx +814 7506 0 2 4 14 14 814 814 814 814 28 29 IFAAAA SCLAAA OOOOxx +8047 7507 1 3 7 7 47 47 47 3047 8047 94 95 NXAAAA TCLAAA VVVVxx +7802 7508 0 2 2 2 2 802 1802 2802 7802 4 5 COAAAA UCLAAA AAAAxx +901 7509 1 1 1 1 1 901 901 901 901 2 3 RIAAAA VCLAAA HHHHxx +6168 7510 0 0 8 8 68 168 168 1168 6168 136 137 GDAAAA WCLAAA OOOOxx +2950 7511 0 2 0 10 50 950 950 2950 2950 100 101 MJAAAA XCLAAA VVVVxx +5393 7512 1 1 3 13 93 393 1393 393 5393 186 187 LZAAAA YCLAAA AAAAxx +3585 7513 1 1 5 5 85 585 1585 3585 3585 170 171 XHAAAA ZCLAAA HHHHxx +9392 7514 0 0 2 12 92 392 1392 4392 9392 184 185 GXAAAA ADLAAA OOOOxx +8314 7515 0 2 4 14 14 314 314 3314 8314 28 29 UHAAAA BDLAAA VVVVxx +9972 7516 0 0 2 12 72 972 1972 4972 9972 144 145 OTAAAA CDLAAA AAAAxx +9130 7517 0 2 0 10 30 130 1130 4130 9130 60 61 ENAAAA DDLAAA HHHHxx +975 7518 1 3 5 15 75 975 975 975 975 150 151 NLAAAA EDLAAA OOOOxx +5720 7519 0 0 0 0 20 720 1720 720 5720 40 41 AMAAAA FDLAAA VVVVxx +3769 7520 1 1 9 9 69 769 1769 3769 3769 138 139 ZOAAAA GDLAAA AAAAxx +5303 7521 1 3 3 3 3 303 1303 303 5303 6 7 ZVAAAA HDLAAA HHHHxx +6564 7522 0 0 4 4 64 564 564 1564 6564 128 129 MSAAAA IDLAAA OOOOxx +7855 7523 1 3 5 15 55 855 1855 2855 7855 110 111 DQAAAA JDLAAA VVVVxx +8153 7524 1 1 3 13 53 153 153 3153 8153 106 107 PBAAAA KDLAAA AAAAxx +2292 7525 0 0 2 12 92 292 292 2292 2292 184 185 EKAAAA LDLAAA HHHHxx +3156 7526 0 0 6 16 56 156 1156 3156 3156 112 113 KRAAAA MDLAAA OOOOxx +6580 7527 0 0 0 0 80 580 580 1580 6580 160 161 CTAAAA NDLAAA VVVVxx +5324 7528 0 0 4 4 24 324 1324 324 5324 48 49 UWAAAA ODLAAA AAAAxx +8871 7529 1 3 1 11 71 871 871 3871 8871 142 143 FDAAAA PDLAAA HHHHxx +2543 7530 1 3 3 3 43 543 543 2543 2543 86 87 VTAAAA QDLAAA OOOOxx +7857 7531 1 1 7 17 57 857 1857 2857 7857 114 115 FQAAAA RDLAAA VVVVxx +4084 7532 0 0 4 4 84 84 84 4084 4084 168 169 CBAAAA SDLAAA AAAAxx +9887 7533 1 3 7 7 87 887 1887 4887 9887 174 175 HQAAAA TDLAAA HHHHxx +6940 7534 0 0 0 0 40 940 940 1940 6940 80 81 YGAAAA UDLAAA OOOOxx +3415 7535 1 3 5 15 15 415 1415 3415 3415 30 31 JBAAAA VDLAAA VVVVxx +5012 7536 0 0 2 12 12 12 1012 12 5012 24 25 UKAAAA WDLAAA AAAAxx +3187 7537 1 3 7 7 87 187 1187 3187 3187 174 175 PSAAAA XDLAAA HHHHxx +8556 7538 0 0 6 16 56 556 556 3556 8556 112 113 CRAAAA YDLAAA OOOOxx +7966 7539 0 2 6 6 66 966 1966 2966 7966 132 133 KUAAAA ZDLAAA VVVVxx +7481 7540 1 1 1 1 81 481 1481 2481 7481 162 163 TBAAAA AELAAA AAAAxx +8524 7541 0 0 4 4 24 524 524 3524 8524 48 49 WPAAAA BELAAA HHHHxx +3021 7542 1 1 1 1 21 21 1021 3021 3021 42 43 FMAAAA CELAAA OOOOxx +6045 7543 1 1 5 5 45 45 45 1045 6045 90 91 NYAAAA DELAAA VVVVxx +8022 7544 0 2 2 2 22 22 22 3022 8022 44 45 OWAAAA EELAAA AAAAxx +3626 7545 0 2 6 6 26 626 1626 3626 3626 52 53 MJAAAA FELAAA HHHHxx +1030 7546 0 2 0 10 30 30 1030 1030 1030 60 61 QNAAAA GELAAA OOOOxx +8903 7547 1 3 3 3 3 903 903 3903 8903 6 7 LEAAAA HELAAA VVVVxx +7488 7548 0 0 8 8 88 488 1488 2488 7488 176 177 ACAAAA IELAAA AAAAxx +9293 7549 1 1 3 13 93 293 1293 4293 9293 186 187 LTAAAA JELAAA HHHHxx +4586 7550 0 2 6 6 86 586 586 4586 4586 172 173 KUAAAA KELAAA OOOOxx +9282 7551 0 2 2 2 82 282 1282 4282 9282 164 165 ATAAAA LELAAA VVVVxx +1948 7552 0 0 8 8 48 948 1948 1948 1948 96 97 YWAAAA MELAAA AAAAxx +2534 7553 0 2 4 14 34 534 534 2534 2534 68 69 MTAAAA NELAAA HHHHxx +1150 7554 0 2 0 10 50 150 1150 1150 1150 100 101 GSAAAA OELAAA OOOOxx +4931 7555 1 3 1 11 31 931 931 4931 4931 62 63 RHAAAA PELAAA VVVVxx +2866 7556 0 2 6 6 66 866 866 2866 2866 132 133 GGAAAA QELAAA AAAAxx +6172 7557 0 0 2 12 72 172 172 1172 6172 144 145 KDAAAA RELAAA HHHHxx +4819 7558 1 3 9 19 19 819 819 4819 4819 38 39 JDAAAA SELAAA OOOOxx +569 7559 1 1 9 9 69 569 569 569 569 138 139 XVAAAA TELAAA VVVVxx +1146 7560 0 2 6 6 46 146 1146 1146 1146 92 93 CSAAAA UELAAA AAAAxx +3062 7561 0 2 2 2 62 62 1062 3062 3062 124 125 UNAAAA VELAAA HHHHxx +7690 7562 0 2 0 10 90 690 1690 2690 7690 180 181 UJAAAA WELAAA OOOOxx +8611 7563 1 3 1 11 11 611 611 3611 8611 22 23 FTAAAA XELAAA VVVVxx +1142 7564 0 2 2 2 42 142 1142 1142 1142 84 85 YRAAAA YELAAA AAAAxx +1193 7565 1 1 3 13 93 193 1193 1193 1193 186 187 XTAAAA ZELAAA HHHHxx +2507 7566 1 3 7 7 7 507 507 2507 2507 14 15 LSAAAA AFLAAA OOOOxx +1043 7567 1 3 3 3 43 43 1043 1043 1043 86 87 DOAAAA BFLAAA VVVVxx +7472 7568 0 0 2 12 72 472 1472 2472 7472 144 145 KBAAAA CFLAAA AAAAxx +1817 7569 1 1 7 17 17 817 1817 1817 1817 34 35 XRAAAA DFLAAA HHHHxx +3868 7570 0 0 8 8 68 868 1868 3868 3868 136 137 USAAAA EFLAAA OOOOxx +9031 7571 1 3 1 11 31 31 1031 4031 9031 62 63 JJAAAA FFLAAA VVVVxx +7254 7572 0 2 4 14 54 254 1254 2254 7254 108 109 ATAAAA GFLAAA AAAAxx +5030 7573 0 2 0 10 30 30 1030 30 5030 60 61 MLAAAA HFLAAA HHHHxx +6594 7574 0 2 4 14 94 594 594 1594 6594 188 189 QTAAAA IFLAAA OOOOxx +6862 7575 0 2 2 2 62 862 862 1862 6862 124 125 YDAAAA JFLAAA VVVVxx +1994 7576 0 2 4 14 94 994 1994 1994 1994 188 189 SYAAAA KFLAAA AAAAxx +9017 7577 1 1 7 17 17 17 1017 4017 9017 34 35 VIAAAA LFLAAA HHHHxx +5716 7578 0 0 6 16 16 716 1716 716 5716 32 33 WLAAAA MFLAAA OOOOxx +1900 7579 0 0 0 0 0 900 1900 1900 1900 0 1 CVAAAA NFLAAA VVVVxx +120 7580 0 0 0 0 20 120 120 120 120 40 41 QEAAAA OFLAAA AAAAxx +9003 7581 1 3 3 3 3 3 1003 4003 9003 6 7 HIAAAA PFLAAA HHHHxx +4178 7582 0 2 8 18 78 178 178 4178 4178 156 157 SEAAAA QFLAAA OOOOxx +8777 7583 1 1 7 17 77 777 777 3777 8777 154 155 PZAAAA RFLAAA VVVVxx +3653 7584 1 1 3 13 53 653 1653 3653 3653 106 107 NKAAAA SFLAAA AAAAxx +1137 7585 1 1 7 17 37 137 1137 1137 1137 74 75 TRAAAA TFLAAA HHHHxx +6362 7586 0 2 2 2 62 362 362 1362 6362 124 125 SKAAAA UFLAAA OOOOxx +8537 7587 1 1 7 17 37 537 537 3537 8537 74 75 JQAAAA VFLAAA VVVVxx +1590 7588 0 2 0 10 90 590 1590 1590 1590 180 181 EJAAAA WFLAAA AAAAxx +374 7589 0 2 4 14 74 374 374 374 374 148 149 KOAAAA XFLAAA HHHHxx +2597 7590 1 1 7 17 97 597 597 2597 2597 194 195 XVAAAA YFLAAA OOOOxx +8071 7591 1 3 1 11 71 71 71 3071 8071 142 143 LYAAAA ZFLAAA VVVVxx +9009 7592 1 1 9 9 9 9 1009 4009 9009 18 19 NIAAAA AGLAAA AAAAxx +1978 7593 0 2 8 18 78 978 1978 1978 1978 156 157 CYAAAA BGLAAA HHHHxx +1541 7594 1 1 1 1 41 541 1541 1541 1541 82 83 HHAAAA CGLAAA OOOOxx +4998 7595 0 2 8 18 98 998 998 4998 4998 196 197 GKAAAA DGLAAA VVVVxx +1649 7596 1 1 9 9 49 649 1649 1649 1649 98 99 LLAAAA EGLAAA AAAAxx +5426 7597 0 2 6 6 26 426 1426 426 5426 52 53 SAAAAA FGLAAA HHHHxx +1492 7598 0 0 2 12 92 492 1492 1492 1492 184 185 KFAAAA GGLAAA OOOOxx +9622 7599 0 2 2 2 22 622 1622 4622 9622 44 45 CGAAAA HGLAAA VVVVxx +701 7600 1 1 1 1 1 701 701 701 701 2 3 ZAAAAA IGLAAA AAAAxx +2781 7601 1 1 1 1 81 781 781 2781 2781 162 163 ZCAAAA JGLAAA HHHHxx +3982 7602 0 2 2 2 82 982 1982 3982 3982 164 165 EXAAAA KGLAAA OOOOxx +7259 7603 1 3 9 19 59 259 1259 2259 7259 118 119 FTAAAA LGLAAA VVVVxx +9868 7604 0 0 8 8 68 868 1868 4868 9868 136 137 OPAAAA MGLAAA AAAAxx +564 7605 0 0 4 4 64 564 564 564 564 128 129 SVAAAA NGLAAA HHHHxx +6315 7606 1 3 5 15 15 315 315 1315 6315 30 31 XIAAAA OGLAAA OOOOxx +9092 7607 0 0 2 12 92 92 1092 4092 9092 184 185 SLAAAA PGLAAA VVVVxx +8237 7608 1 1 7 17 37 237 237 3237 8237 74 75 VEAAAA QGLAAA AAAAxx +1513 7609 1 1 3 13 13 513 1513 1513 1513 26 27 FGAAAA RGLAAA HHHHxx +1922 7610 0 2 2 2 22 922 1922 1922 1922 44 45 YVAAAA SGLAAA OOOOxx +5396 7611 0 0 6 16 96 396 1396 396 5396 192 193 OZAAAA TGLAAA VVVVxx +2485 7612 1 1 5 5 85 485 485 2485 2485 170 171 PRAAAA UGLAAA AAAAxx +5774 7613 0 2 4 14 74 774 1774 774 5774 148 149 COAAAA VGLAAA HHHHxx +3983 7614 1 3 3 3 83 983 1983 3983 3983 166 167 FXAAAA WGLAAA OOOOxx +221 7615 1 1 1 1 21 221 221 221 221 42 43 NIAAAA XGLAAA VVVVxx +8662 7616 0 2 2 2 62 662 662 3662 8662 124 125 EVAAAA YGLAAA AAAAxx +2456 7617 0 0 6 16 56 456 456 2456 2456 112 113 MQAAAA ZGLAAA HHHHxx +9736 7618 0 0 6 16 36 736 1736 4736 9736 72 73 MKAAAA AHLAAA OOOOxx +8936 7619 0 0 6 16 36 936 936 3936 8936 72 73 SFAAAA BHLAAA VVVVxx +5395 7620 1 3 5 15 95 395 1395 395 5395 190 191 NZAAAA CHLAAA AAAAxx +9523 7621 1 3 3 3 23 523 1523 4523 9523 46 47 HCAAAA DHLAAA HHHHxx +6980 7622 0 0 0 0 80 980 980 1980 6980 160 161 MIAAAA EHLAAA OOOOxx +2091 7623 1 3 1 11 91 91 91 2091 2091 182 183 LCAAAA FHLAAA VVVVxx +6807 7624 1 3 7 7 7 807 807 1807 6807 14 15 VBAAAA GHLAAA AAAAxx +8818 7625 0 2 8 18 18 818 818 3818 8818 36 37 EBAAAA HHLAAA HHHHxx +5298 7626 0 2 8 18 98 298 1298 298 5298 196 197 UVAAAA IHLAAA OOOOxx +1726 7627 0 2 6 6 26 726 1726 1726 1726 52 53 KOAAAA JHLAAA VVVVxx +3878 7628 0 2 8 18 78 878 1878 3878 3878 156 157 ETAAAA KHLAAA AAAAxx +8700 7629 0 0 0 0 0 700 700 3700 8700 0 1 QWAAAA LHLAAA HHHHxx +5201 7630 1 1 1 1 1 201 1201 201 5201 2 3 BSAAAA MHLAAA OOOOxx +3936 7631 0 0 6 16 36 936 1936 3936 3936 72 73 KVAAAA NHLAAA VVVVxx +776 7632 0 0 6 16 76 776 776 776 776 152 153 WDAAAA OHLAAA AAAAxx +5302 7633 0 2 2 2 2 302 1302 302 5302 4 5 YVAAAA PHLAAA HHHHxx +3595 7634 1 3 5 15 95 595 1595 3595 3595 190 191 HIAAAA QHLAAA OOOOxx +9061 7635 1 1 1 1 61 61 1061 4061 9061 122 123 NKAAAA RHLAAA VVVVxx +6261 7636 1 1 1 1 61 261 261 1261 6261 122 123 VGAAAA SHLAAA AAAAxx +8878 7637 0 2 8 18 78 878 878 3878 8878 156 157 MDAAAA THLAAA HHHHxx +3312 7638 0 0 2 12 12 312 1312 3312 3312 24 25 KXAAAA UHLAAA OOOOxx +9422 7639 0 2 2 2 22 422 1422 4422 9422 44 45 KYAAAA VHLAAA VVVVxx +7321 7640 1 1 1 1 21 321 1321 2321 7321 42 43 PVAAAA WHLAAA AAAAxx +3813 7641 1 1 3 13 13 813 1813 3813 3813 26 27 RQAAAA XHLAAA HHHHxx +5848 7642 0 0 8 8 48 848 1848 848 5848 96 97 YQAAAA YHLAAA OOOOxx +3535 7643 1 3 5 15 35 535 1535 3535 3535 70 71 ZFAAAA ZHLAAA VVVVxx +1040 7644 0 0 0 0 40 40 1040 1040 1040 80 81 AOAAAA AILAAA AAAAxx +8572 7645 0 0 2 12 72 572 572 3572 8572 144 145 SRAAAA BILAAA HHHHxx +5435 7646 1 3 5 15 35 435 1435 435 5435 70 71 BBAAAA CILAAA OOOOxx +8199 7647 1 3 9 19 99 199 199 3199 8199 198 199 JDAAAA DILAAA VVVVxx +8775 7648 1 3 5 15 75 775 775 3775 8775 150 151 NZAAAA EILAAA AAAAxx +7722 7649 0 2 2 2 22 722 1722 2722 7722 44 45 ALAAAA FILAAA HHHHxx +3549 7650 1 1 9 9 49 549 1549 3549 3549 98 99 NGAAAA GILAAA OOOOxx +2578 7651 0 2 8 18 78 578 578 2578 2578 156 157 EVAAAA HILAAA VVVVxx +1695 7652 1 3 5 15 95 695 1695 1695 1695 190 191 FNAAAA IILAAA AAAAxx +1902 7653 0 2 2 2 2 902 1902 1902 1902 4 5 EVAAAA JILAAA HHHHxx +6058 7654 0 2 8 18 58 58 58 1058 6058 116 117 AZAAAA KILAAA OOOOxx +6591 7655 1 3 1 11 91 591 591 1591 6591 182 183 NTAAAA LILAAA VVVVxx +7962 7656 0 2 2 2 62 962 1962 2962 7962 124 125 GUAAAA MILAAA AAAAxx +5612 7657 0 0 2 12 12 612 1612 612 5612 24 25 WHAAAA NILAAA HHHHxx +3341 7658 1 1 1 1 41 341 1341 3341 3341 82 83 NYAAAA OILAAA OOOOxx +5460 7659 0 0 0 0 60 460 1460 460 5460 120 121 ACAAAA PILAAA VVVVxx +2368 7660 0 0 8 8 68 368 368 2368 2368 136 137 CNAAAA QILAAA AAAAxx +8646 7661 0 2 6 6 46 646 646 3646 8646 92 93 OUAAAA RILAAA HHHHxx +4987 7662 1 3 7 7 87 987 987 4987 4987 174 175 VJAAAA SILAAA OOOOxx +9018 7663 0 2 8 18 18 18 1018 4018 9018 36 37 WIAAAA TILAAA VVVVxx +8685 7664 1 1 5 5 85 685 685 3685 8685 170 171 BWAAAA UILAAA AAAAxx +694 7665 0 2 4 14 94 694 694 694 694 188 189 SAAAAA VILAAA HHHHxx +2012 7666 0 0 2 12 12 12 12 2012 2012 24 25 KZAAAA WILAAA OOOOxx +2417 7667 1 1 7 17 17 417 417 2417 2417 34 35 ZOAAAA XILAAA VVVVxx +4022 7668 0 2 2 2 22 22 22 4022 4022 44 45 SYAAAA YILAAA AAAAxx +5935 7669 1 3 5 15 35 935 1935 935 5935 70 71 HUAAAA ZILAAA HHHHxx +1656 7670 0 0 6 16 56 656 1656 1656 1656 112 113 SLAAAA AJLAAA OOOOxx +6195 7671 1 3 5 15 95 195 195 1195 6195 190 191 HEAAAA BJLAAA VVVVxx +3057 7672 1 1 7 17 57 57 1057 3057 3057 114 115 PNAAAA CJLAAA AAAAxx +2852 7673 0 0 2 12 52 852 852 2852 2852 104 105 SFAAAA DJLAAA HHHHxx +4634 7674 0 2 4 14 34 634 634 4634 4634 68 69 GWAAAA EJLAAA OOOOxx +1689 7675 1 1 9 9 89 689 1689 1689 1689 178 179 ZMAAAA FJLAAA VVVVxx +4102 7676 0 2 2 2 2 102 102 4102 4102 4 5 UBAAAA GJLAAA AAAAxx +3287 7677 1 3 7 7 87 287 1287 3287 3287 174 175 LWAAAA HJLAAA HHHHxx +5246 7678 0 2 6 6 46 246 1246 246 5246 92 93 UTAAAA IJLAAA OOOOxx +7450 7679 0 2 0 10 50 450 1450 2450 7450 100 101 OAAAAA JJLAAA VVVVxx +6548 7680 0 0 8 8 48 548 548 1548 6548 96 97 WRAAAA KJLAAA AAAAxx +379 7681 1 3 9 19 79 379 379 379 379 158 159 POAAAA LJLAAA HHHHxx +7435 7682 1 3 5 15 35 435 1435 2435 7435 70 71 ZZAAAA MJLAAA OOOOxx +2041 7683 1 1 1 1 41 41 41 2041 2041 82 83 NAAAAA NJLAAA VVVVxx +8462 7684 0 2 2 2 62 462 462 3462 8462 124 125 MNAAAA OJLAAA AAAAxx +9076 7685 0 0 6 16 76 76 1076 4076 9076 152 153 CLAAAA PJLAAA HHHHxx +761 7686 1 1 1 1 61 761 761 761 761 122 123 HDAAAA QJLAAA OOOOxx +795 7687 1 3 5 15 95 795 795 795 795 190 191 PEAAAA RJLAAA VVVVxx +1671 7688 1 3 1 11 71 671 1671 1671 1671 142 143 HMAAAA SJLAAA AAAAxx +695 7689 1 3 5 15 95 695 695 695 695 190 191 TAAAAA TJLAAA HHHHxx +4981 7690 1 1 1 1 81 981 981 4981 4981 162 163 PJAAAA UJLAAA OOOOxx +1211 7691 1 3 1 11 11 211 1211 1211 1211 22 23 PUAAAA VJLAAA VVVVxx +5914 7692 0 2 4 14 14 914 1914 914 5914 28 29 MTAAAA WJLAAA AAAAxx +9356 7693 0 0 6 16 56 356 1356 4356 9356 112 113 WVAAAA XJLAAA HHHHxx +1500 7694 0 0 0 0 0 500 1500 1500 1500 0 1 SFAAAA YJLAAA OOOOxx +3353 7695 1 1 3 13 53 353 1353 3353 3353 106 107 ZYAAAA ZJLAAA VVVVxx +1060 7696 0 0 0 0 60 60 1060 1060 1060 120 121 UOAAAA AKLAAA AAAAxx +7910 7697 0 2 0 10 10 910 1910 2910 7910 20 21 GSAAAA BKLAAA HHHHxx +1329 7698 1 1 9 9 29 329 1329 1329 1329 58 59 DZAAAA CKLAAA OOOOxx +6011 7699 1 3 1 11 11 11 11 1011 6011 22 23 FXAAAA DKLAAA VVVVxx +7146 7700 0 2 6 6 46 146 1146 2146 7146 92 93 WOAAAA EKLAAA AAAAxx +4602 7701 0 2 2 2 2 602 602 4602 4602 4 5 AVAAAA FKLAAA HHHHxx +6751 7702 1 3 1 11 51 751 751 1751 6751 102 103 RZAAAA GKLAAA OOOOxx +2666 7703 0 2 6 6 66 666 666 2666 2666 132 133 OYAAAA HKLAAA VVVVxx +2785 7704 1 1 5 5 85 785 785 2785 2785 170 171 DDAAAA IKLAAA AAAAxx +5851 7705 1 3 1 11 51 851 1851 851 5851 102 103 BRAAAA JKLAAA HHHHxx +2435 7706 1 3 5 15 35 435 435 2435 2435 70 71 RPAAAA KKLAAA OOOOxx +7429 7707 1 1 9 9 29 429 1429 2429 7429 58 59 TZAAAA LKLAAA VVVVxx +4241 7708 1 1 1 1 41 241 241 4241 4241 82 83 DHAAAA MKLAAA AAAAxx +5691 7709 1 3 1 11 91 691 1691 691 5691 182 183 XKAAAA NKLAAA HHHHxx +7731 7710 1 3 1 11 31 731 1731 2731 7731 62 63 JLAAAA OKLAAA OOOOxx +249 7711 1 1 9 9 49 249 249 249 249 98 99 PJAAAA PKLAAA VVVVxx +1731 7712 1 3 1 11 31 731 1731 1731 1731 62 63 POAAAA QKLAAA AAAAxx +8716 7713 0 0 6 16 16 716 716 3716 8716 32 33 GXAAAA RKLAAA HHHHxx +2670 7714 0 2 0 10 70 670 670 2670 2670 140 141 SYAAAA SKLAAA OOOOxx +4654 7715 0 2 4 14 54 654 654 4654 4654 108 109 AXAAAA TKLAAA VVVVxx +1027 7716 1 3 7 7 27 27 1027 1027 1027 54 55 NNAAAA UKLAAA AAAAxx +1099 7717 1 3 9 19 99 99 1099 1099 1099 198 199 HQAAAA VKLAAA HHHHxx +3617 7718 1 1 7 17 17 617 1617 3617 3617 34 35 DJAAAA WKLAAA OOOOxx +4330 7719 0 2 0 10 30 330 330 4330 4330 60 61 OKAAAA XKLAAA VVVVxx +9750 7720 0 2 0 10 50 750 1750 4750 9750 100 101 ALAAAA YKLAAA AAAAxx +467 7721 1 3 7 7 67 467 467 467 467 134 135 ZRAAAA ZKLAAA HHHHxx +8525 7722 1 1 5 5 25 525 525 3525 8525 50 51 XPAAAA ALLAAA OOOOxx +5990 7723 0 2 0 10 90 990 1990 990 5990 180 181 KWAAAA BLLAAA VVVVxx +4839 7724 1 3 9 19 39 839 839 4839 4839 78 79 DEAAAA CLLAAA AAAAxx +9914 7725 0 2 4 14 14 914 1914 4914 9914 28 29 IRAAAA DLLAAA HHHHxx +7047 7726 1 3 7 7 47 47 1047 2047 7047 94 95 BLAAAA ELLAAA OOOOxx +874 7727 0 2 4 14 74 874 874 874 874 148 149 QHAAAA FLLAAA VVVVxx +6061 7728 1 1 1 1 61 61 61 1061 6061 122 123 DZAAAA GLLAAA AAAAxx +5491 7729 1 3 1 11 91 491 1491 491 5491 182 183 FDAAAA HLLAAA HHHHxx +4344 7730 0 0 4 4 44 344 344 4344 4344 88 89 CLAAAA ILLAAA OOOOxx +1281 7731 1 1 1 1 81 281 1281 1281 1281 162 163 HXAAAA JLLAAA VVVVxx +3597 7732 1 1 7 17 97 597 1597 3597 3597 194 195 JIAAAA KLLAAA AAAAxx +4992 7733 0 0 2 12 92 992 992 4992 4992 184 185 AKAAAA LLLAAA HHHHxx +3849 7734 1 1 9 9 49 849 1849 3849 3849 98 99 BSAAAA MLLAAA OOOOxx +2655 7735 1 3 5 15 55 655 655 2655 2655 110 111 DYAAAA NLLAAA VVVVxx +147 7736 1 3 7 7 47 147 147 147 147 94 95 RFAAAA OLLAAA AAAAxx +9110 7737 0 2 0 10 10 110 1110 4110 9110 20 21 KMAAAA PLLAAA HHHHxx +1637 7738 1 1 7 17 37 637 1637 1637 1637 74 75 ZKAAAA QLLAAA OOOOxx +9826 7739 0 2 6 6 26 826 1826 4826 9826 52 53 YNAAAA RLLAAA VVVVxx +5957 7740 1 1 7 17 57 957 1957 957 5957 114 115 DVAAAA SLLAAA AAAAxx +6932 7741 0 0 2 12 32 932 932 1932 6932 64 65 QGAAAA TLLAAA HHHHxx +9684 7742 0 0 4 4 84 684 1684 4684 9684 168 169 MIAAAA ULLAAA OOOOxx +4653 7743 1 1 3 13 53 653 653 4653 4653 106 107 ZWAAAA VLLAAA VVVVxx +8065 7744 1 1 5 5 65 65 65 3065 8065 130 131 FYAAAA WLLAAA AAAAxx +1202 7745 0 2 2 2 2 202 1202 1202 1202 4 5 GUAAAA XLLAAA HHHHxx +9214 7746 0 2 4 14 14 214 1214 4214 9214 28 29 KQAAAA YLLAAA OOOOxx +196 7747 0 0 6 16 96 196 196 196 196 192 193 OHAAAA ZLLAAA VVVVxx +4486 7748 0 2 6 6 86 486 486 4486 4486 172 173 OQAAAA AMLAAA AAAAxx +2585 7749 1 1 5 5 85 585 585 2585 2585 170 171 LVAAAA BMLAAA HHHHxx +2464 7750 0 0 4 4 64 464 464 2464 2464 128 129 UQAAAA CMLAAA OOOOxx +3467 7751 1 3 7 7 67 467 1467 3467 3467 134 135 JDAAAA DMLAAA VVVVxx +9295 7752 1 3 5 15 95 295 1295 4295 9295 190 191 NTAAAA EMLAAA AAAAxx +517 7753 1 1 7 17 17 517 517 517 517 34 35 XTAAAA FMLAAA HHHHxx +6870 7754 0 2 0 10 70 870 870 1870 6870 140 141 GEAAAA GMLAAA OOOOxx +5732 7755 0 0 2 12 32 732 1732 732 5732 64 65 MMAAAA HMLAAA VVVVxx +9376 7756 0 0 6 16 76 376 1376 4376 9376 152 153 QWAAAA IMLAAA AAAAxx +838 7757 0 2 8 18 38 838 838 838 838 76 77 GGAAAA JMLAAA HHHHxx +9254 7758 0 2 4 14 54 254 1254 4254 9254 108 109 YRAAAA KMLAAA OOOOxx +8879 7759 1 3 9 19 79 879 879 3879 8879 158 159 NDAAAA LMLAAA VVVVxx +6281 7760 1 1 1 1 81 281 281 1281 6281 162 163 PHAAAA MMLAAA AAAAxx +8216 7761 0 0 6 16 16 216 216 3216 8216 32 33 AEAAAA NMLAAA HHHHxx +9213 7762 1 1 3 13 13 213 1213 4213 9213 26 27 JQAAAA OMLAAA OOOOxx +7234 7763 0 2 4 14 34 234 1234 2234 7234 68 69 GSAAAA PMLAAA VVVVxx +5692 7764 0 0 2 12 92 692 1692 692 5692 184 185 YKAAAA QMLAAA AAAAxx +693 7765 1 1 3 13 93 693 693 693 693 186 187 RAAAAA RMLAAA HHHHxx +9050 7766 0 2 0 10 50 50 1050 4050 9050 100 101 CKAAAA SMLAAA OOOOxx +3623 7767 1 3 3 3 23 623 1623 3623 3623 46 47 JJAAAA TMLAAA VVVVxx +2130 7768 0 2 0 10 30 130 130 2130 2130 60 61 YDAAAA UMLAAA AAAAxx +2514 7769 0 2 4 14 14 514 514 2514 2514 28 29 SSAAAA VMLAAA HHHHxx +1812 7770 0 0 2 12 12 812 1812 1812 1812 24 25 SRAAAA WMLAAA OOOOxx +9037 7771 1 1 7 17 37 37 1037 4037 9037 74 75 PJAAAA XMLAAA VVVVxx +5054 7772 0 2 4 14 54 54 1054 54 5054 108 109 KMAAAA YMLAAA AAAAxx +7801 7773 1 1 1 1 1 801 1801 2801 7801 2 3 BOAAAA ZMLAAA HHHHxx +7939 7774 1 3 9 19 39 939 1939 2939 7939 78 79 JTAAAA ANLAAA OOOOxx +7374 7775 0 2 4 14 74 374 1374 2374 7374 148 149 QXAAAA BNLAAA VVVVxx +1058 7776 0 2 8 18 58 58 1058 1058 1058 116 117 SOAAAA CNLAAA AAAAxx +1972 7777 0 0 2 12 72 972 1972 1972 1972 144 145 WXAAAA DNLAAA HHHHxx +3741 7778 1 1 1 1 41 741 1741 3741 3741 82 83 XNAAAA ENLAAA OOOOxx +2227 7779 1 3 7 7 27 227 227 2227 2227 54 55 RHAAAA FNLAAA VVVVxx +304 7780 0 0 4 4 4 304 304 304 304 8 9 SLAAAA GNLAAA AAAAxx +4914 7781 0 2 4 14 14 914 914 4914 4914 28 29 AHAAAA HNLAAA HHHHxx +2428 7782 0 0 8 8 28 428 428 2428 2428 56 57 KPAAAA INLAAA OOOOxx +6660 7783 0 0 0 0 60 660 660 1660 6660 120 121 EWAAAA JNLAAA VVVVxx +2676 7784 0 0 6 16 76 676 676 2676 2676 152 153 YYAAAA KNLAAA AAAAxx +2454 7785 0 2 4 14 54 454 454 2454 2454 108 109 KQAAAA LNLAAA HHHHxx +3798 7786 0 2 8 18 98 798 1798 3798 3798 196 197 CQAAAA MNLAAA OOOOxx +1341 7787 1 1 1 1 41 341 1341 1341 1341 82 83 PZAAAA NNLAAA VVVVxx +1611 7788 1 3 1 11 11 611 1611 1611 1611 22 23 ZJAAAA ONLAAA AAAAxx +2681 7789 1 1 1 1 81 681 681 2681 2681 162 163 DZAAAA PNLAAA HHHHxx +7292 7790 0 0 2 12 92 292 1292 2292 7292 184 185 MUAAAA QNLAAA OOOOxx +7775 7791 1 3 5 15 75 775 1775 2775 7775 150 151 BNAAAA RNLAAA VVVVxx +794 7792 0 2 4 14 94 794 794 794 794 188 189 OEAAAA SNLAAA AAAAxx +8709 7793 1 1 9 9 9 709 709 3709 8709 18 19 ZWAAAA TNLAAA HHHHxx +1901 7794 1 1 1 1 1 901 1901 1901 1901 2 3 DVAAAA UNLAAA OOOOxx +3089 7795 1 1 9 9 89 89 1089 3089 3089 178 179 VOAAAA VNLAAA VVVVxx +7797 7796 1 1 7 17 97 797 1797 2797 7797 194 195 XNAAAA WNLAAA AAAAxx +6070 7797 0 2 0 10 70 70 70 1070 6070 140 141 MZAAAA XNLAAA HHHHxx +2191 7798 1 3 1 11 91 191 191 2191 2191 182 183 HGAAAA YNLAAA OOOOxx +3497 7799 1 1 7 17 97 497 1497 3497 3497 194 195 NEAAAA ZNLAAA VVVVxx +8302 7800 0 2 2 2 2 302 302 3302 8302 4 5 IHAAAA AOLAAA AAAAxx +4365 7801 1 1 5 5 65 365 365 4365 4365 130 131 XLAAAA BOLAAA HHHHxx +3588 7802 0 0 8 8 88 588 1588 3588 3588 176 177 AIAAAA COLAAA OOOOxx +8292 7803 0 0 2 12 92 292 292 3292 8292 184 185 YGAAAA DOLAAA VVVVxx +4696 7804 0 0 6 16 96 696 696 4696 4696 192 193 QYAAAA EOLAAA AAAAxx +5641 7805 1 1 1 1 41 641 1641 641 5641 82 83 ZIAAAA FOLAAA HHHHxx +9386 7806 0 2 6 6 86 386 1386 4386 9386 172 173 AXAAAA GOLAAA OOOOxx +507 7807 1 3 7 7 7 507 507 507 507 14 15 NTAAAA HOLAAA VVVVxx +7201 7808 1 1 1 1 1 201 1201 2201 7201 2 3 ZQAAAA IOLAAA AAAAxx +7785 7809 1 1 5 5 85 785 1785 2785 7785 170 171 LNAAAA JOLAAA HHHHxx +463 7810 1 3 3 3 63 463 463 463 463 126 127 VRAAAA KOLAAA OOOOxx +6656 7811 0 0 6 16 56 656 656 1656 6656 112 113 AWAAAA LOLAAA VVVVxx +807 7812 1 3 7 7 7 807 807 807 807 14 15 BFAAAA MOLAAA AAAAxx +7278 7813 0 2 8 18 78 278 1278 2278 7278 156 157 YTAAAA NOLAAA HHHHxx +6237 7814 1 1 7 17 37 237 237 1237 6237 74 75 XFAAAA OOLAAA OOOOxx +7671 7815 1 3 1 11 71 671 1671 2671 7671 142 143 BJAAAA POLAAA VVVVxx +2235 7816 1 3 5 15 35 235 235 2235 2235 70 71 ZHAAAA QOLAAA AAAAxx +4042 7817 0 2 2 2 42 42 42 4042 4042 84 85 MZAAAA ROLAAA HHHHxx +5273 7818 1 1 3 13 73 273 1273 273 5273 146 147 VUAAAA SOLAAA OOOOxx +7557 7819 1 1 7 17 57 557 1557 2557 7557 114 115 REAAAA TOLAAA VVVVxx +4007 7820 1 3 7 7 7 7 7 4007 4007 14 15 DYAAAA UOLAAA AAAAxx +1428 7821 0 0 8 8 28 428 1428 1428 1428 56 57 YCAAAA VOLAAA HHHHxx +9739 7822 1 3 9 19 39 739 1739 4739 9739 78 79 PKAAAA WOLAAA OOOOxx +7836 7823 0 0 6 16 36 836 1836 2836 7836 72 73 KPAAAA XOLAAA VVVVxx +1777 7824 1 1 7 17 77 777 1777 1777 1777 154 155 JQAAAA YOLAAA AAAAxx +5192 7825 0 0 2 12 92 192 1192 192 5192 184 185 SRAAAA ZOLAAA HHHHxx +7236 7826 0 0 6 16 36 236 1236 2236 7236 72 73 ISAAAA APLAAA OOOOxx +1623 7827 1 3 3 3 23 623 1623 1623 1623 46 47 LKAAAA BPLAAA VVVVxx +8288 7828 0 0 8 8 88 288 288 3288 8288 176 177 UGAAAA CPLAAA AAAAxx +2827 7829 1 3 7 7 27 827 827 2827 2827 54 55 TEAAAA DPLAAA HHHHxx +458 7830 0 2 8 18 58 458 458 458 458 116 117 QRAAAA EPLAAA OOOOxx +1818 7831 0 2 8 18 18 818 1818 1818 1818 36 37 YRAAAA FPLAAA VVVVxx +6837 7832 1 1 7 17 37 837 837 1837 6837 74 75 ZCAAAA GPLAAA AAAAxx +7825 7833 1 1 5 5 25 825 1825 2825 7825 50 51 ZOAAAA HPLAAA HHHHxx +9146 7834 0 2 6 6 46 146 1146 4146 9146 92 93 UNAAAA IPLAAA OOOOxx +8451 7835 1 3 1 11 51 451 451 3451 8451 102 103 BNAAAA JPLAAA VVVVxx +6438 7836 0 2 8 18 38 438 438 1438 6438 76 77 QNAAAA KPLAAA AAAAxx +4020 7837 0 0 0 0 20 20 20 4020 4020 40 41 QYAAAA LPLAAA HHHHxx +4068 7838 0 0 8 8 68 68 68 4068 4068 136 137 MAAAAA MPLAAA OOOOxx +2411 7839 1 3 1 11 11 411 411 2411 2411 22 23 TOAAAA NPLAAA VVVVxx +6222 7840 0 2 2 2 22 222 222 1222 6222 44 45 IFAAAA OPLAAA AAAAxx +3164 7841 0 0 4 4 64 164 1164 3164 3164 128 129 SRAAAA PPLAAA HHHHxx +311 7842 1 3 1 11 11 311 311 311 311 22 23 ZLAAAA QPLAAA OOOOxx +5683 7843 1 3 3 3 83 683 1683 683 5683 166 167 PKAAAA RPLAAA VVVVxx +3993 7844 1 1 3 13 93 993 1993 3993 3993 186 187 PXAAAA SPLAAA AAAAxx +9897 7845 1 1 7 17 97 897 1897 4897 9897 194 195 RQAAAA TPLAAA HHHHxx +6609 7846 1 1 9 9 9 609 609 1609 6609 18 19 FUAAAA UPLAAA OOOOxx +1362 7847 0 2 2 2 62 362 1362 1362 1362 124 125 KAAAAA VPLAAA VVVVxx +3918 7848 0 2 8 18 18 918 1918 3918 3918 36 37 SUAAAA WPLAAA AAAAxx +7376 7849 0 0 6 16 76 376 1376 2376 7376 152 153 SXAAAA XPLAAA HHHHxx +6996 7850 0 0 6 16 96 996 996 1996 6996 192 193 CJAAAA YPLAAA OOOOxx +9567 7851 1 3 7 7 67 567 1567 4567 9567 134 135 ZDAAAA ZPLAAA VVVVxx +7525 7852 1 1 5 5 25 525 1525 2525 7525 50 51 LDAAAA AQLAAA AAAAxx +9069 7853 1 1 9 9 69 69 1069 4069 9069 138 139 VKAAAA BQLAAA HHHHxx +9999 7854 1 3 9 19 99 999 1999 4999 9999 198 199 PUAAAA CQLAAA OOOOxx +9237 7855 1 1 7 17 37 237 1237 4237 9237 74 75 HRAAAA DQLAAA VVVVxx +8441 7856 1 1 1 1 41 441 441 3441 8441 82 83 RMAAAA EQLAAA AAAAxx +6769 7857 1 1 9 9 69 769 769 1769 6769 138 139 JAAAAA FQLAAA HHHHxx +6073 7858 1 1 3 13 73 73 73 1073 6073 146 147 PZAAAA GQLAAA OOOOxx +1091 7859 1 3 1 11 91 91 1091 1091 1091 182 183 ZPAAAA HQLAAA VVVVxx +9886 7860 0 2 6 6 86 886 1886 4886 9886 172 173 GQAAAA IQLAAA AAAAxx +3971 7861 1 3 1 11 71 971 1971 3971 3971 142 143 TWAAAA JQLAAA HHHHxx +4621 7862 1 1 1 1 21 621 621 4621 4621 42 43 TVAAAA KQLAAA OOOOxx +3120 7863 0 0 0 0 20 120 1120 3120 3120 40 41 AQAAAA LQLAAA VVVVxx +9773 7864 1 1 3 13 73 773 1773 4773 9773 146 147 XLAAAA MQLAAA AAAAxx +8712 7865 0 0 2 12 12 712 712 3712 8712 24 25 CXAAAA NQLAAA HHHHxx +801 7866 1 1 1 1 1 801 801 801 801 2 3 VEAAAA OQLAAA OOOOxx +9478 7867 0 2 8 18 78 478 1478 4478 9478 156 157 OAAAAA PQLAAA VVVVxx +3466 7868 0 2 6 6 66 466 1466 3466 3466 132 133 IDAAAA QQLAAA AAAAxx +6326 7869 0 2 6 6 26 326 326 1326 6326 52 53 IJAAAA RQLAAA HHHHxx +1723 7870 1 3 3 3 23 723 1723 1723 1723 46 47 HOAAAA SQLAAA OOOOxx +4978 7871 0 2 8 18 78 978 978 4978 4978 156 157 MJAAAA TQLAAA VVVVxx +2311 7872 1 3 1 11 11 311 311 2311 2311 22 23 XKAAAA UQLAAA AAAAxx +9532 7873 0 0 2 12 32 532 1532 4532 9532 64 65 QCAAAA VQLAAA HHHHxx +3680 7874 0 0 0 0 80 680 1680 3680 3680 160 161 OLAAAA WQLAAA OOOOxx +1244 7875 0 0 4 4 44 244 1244 1244 1244 88 89 WVAAAA XQLAAA VVVVxx +3821 7876 1 1 1 1 21 821 1821 3821 3821 42 43 ZQAAAA YQLAAA AAAAxx +9586 7877 0 2 6 6 86 586 1586 4586 9586 172 173 SEAAAA ZQLAAA HHHHxx +3894 7878 0 2 4 14 94 894 1894 3894 3894 188 189 UTAAAA ARLAAA OOOOxx +6169 7879 1 1 9 9 69 169 169 1169 6169 138 139 HDAAAA BRLAAA VVVVxx +5919 7880 1 3 9 19 19 919 1919 919 5919 38 39 RTAAAA CRLAAA AAAAxx +4187 7881 1 3 7 7 87 187 187 4187 4187 174 175 BFAAAA DRLAAA HHHHxx +5477 7882 1 1 7 17 77 477 1477 477 5477 154 155 RCAAAA ERLAAA OOOOxx +2806 7883 0 2 6 6 6 806 806 2806 2806 12 13 YDAAAA FRLAAA VVVVxx +8158 7884 0 2 8 18 58 158 158 3158 8158 116 117 UBAAAA GRLAAA AAAAxx +7130 7885 0 2 0 10 30 130 1130 2130 7130 60 61 GOAAAA HRLAAA HHHHxx +7133 7886 1 1 3 13 33 133 1133 2133 7133 66 67 JOAAAA IRLAAA OOOOxx +6033 7887 1 1 3 13 33 33 33 1033 6033 66 67 BYAAAA JRLAAA VVVVxx +2415 7888 1 3 5 15 15 415 415 2415 2415 30 31 XOAAAA KRLAAA AAAAxx +8091 7889 1 3 1 11 91 91 91 3091 8091 182 183 FZAAAA LRLAAA HHHHxx +8347 7890 1 3 7 7 47 347 347 3347 8347 94 95 BJAAAA MRLAAA OOOOxx +7879 7891 1 3 9 19 79 879 1879 2879 7879 158 159 BRAAAA NRLAAA VVVVxx +9360 7892 0 0 0 0 60 360 1360 4360 9360 120 121 AWAAAA ORLAAA AAAAxx +3369 7893 1 1 9 9 69 369 1369 3369 3369 138 139 PZAAAA PRLAAA HHHHxx +8536 7894 0 0 6 16 36 536 536 3536 8536 72 73 IQAAAA QRLAAA OOOOxx +8628 7895 0 0 8 8 28 628 628 3628 8628 56 57 WTAAAA RRLAAA VVVVxx +1580 7896 0 0 0 0 80 580 1580 1580 1580 160 161 UIAAAA SRLAAA AAAAxx +705 7897 1 1 5 5 5 705 705 705 705 10 11 DBAAAA TRLAAA HHHHxx +4650 7898 0 2 0 10 50 650 650 4650 4650 100 101 WWAAAA URLAAA OOOOxx +9165 7899 1 1 5 5 65 165 1165 4165 9165 130 131 NOAAAA VRLAAA VVVVxx +4820 7900 0 0 0 0 20 820 820 4820 4820 40 41 KDAAAA WRLAAA AAAAxx +3538 7901 0 2 8 18 38 538 1538 3538 3538 76 77 CGAAAA XRLAAA HHHHxx +9947 7902 1 3 7 7 47 947 1947 4947 9947 94 95 PSAAAA YRLAAA OOOOxx +4954 7903 0 2 4 14 54 954 954 4954 4954 108 109 OIAAAA ZRLAAA VVVVxx +1104 7904 0 0 4 4 4 104 1104 1104 1104 8 9 MQAAAA ASLAAA AAAAxx +8455 7905 1 3 5 15 55 455 455 3455 8455 110 111 FNAAAA BSLAAA HHHHxx +8307 7906 1 3 7 7 7 307 307 3307 8307 14 15 NHAAAA CSLAAA OOOOxx +9203 7907 1 3 3 3 3 203 1203 4203 9203 6 7 ZPAAAA DSLAAA VVVVxx +7565 7908 1 1 5 5 65 565 1565 2565 7565 130 131 ZEAAAA ESLAAA AAAAxx +7745 7909 1 1 5 5 45 745 1745 2745 7745 90 91 XLAAAA FSLAAA HHHHxx +1787 7910 1 3 7 7 87 787 1787 1787 1787 174 175 TQAAAA GSLAAA OOOOxx +4861 7911 1 1 1 1 61 861 861 4861 4861 122 123 ZEAAAA HSLAAA VVVVxx +5183 7912 1 3 3 3 83 183 1183 183 5183 166 167 JRAAAA ISLAAA AAAAxx +529 7913 1 1 9 9 29 529 529 529 529 58 59 JUAAAA JSLAAA HHHHxx +2470 7914 0 2 0 10 70 470 470 2470 2470 140 141 ARAAAA KSLAAA OOOOxx +1267 7915 1 3 7 7 67 267 1267 1267 1267 134 135 TWAAAA LSLAAA VVVVxx +2059 7916 1 3 9 19 59 59 59 2059 2059 118 119 FBAAAA MSLAAA AAAAxx +1862 7917 0 2 2 2 62 862 1862 1862 1862 124 125 QTAAAA NSLAAA HHHHxx +7382 7918 0 2 2 2 82 382 1382 2382 7382 164 165 YXAAAA OSLAAA OOOOxx +4796 7919 0 0 6 16 96 796 796 4796 4796 192 193 MCAAAA PSLAAA VVVVxx +2331 7920 1 3 1 11 31 331 331 2331 2331 62 63 RLAAAA QSLAAA AAAAxx +8870 7921 0 2 0 10 70 870 870 3870 8870 140 141 EDAAAA RSLAAA HHHHxx +9581 7922 1 1 1 1 81 581 1581 4581 9581 162 163 NEAAAA SSLAAA OOOOxx +9063 7923 1 3 3 3 63 63 1063 4063 9063 126 127 PKAAAA TSLAAA VVVVxx +2192 7924 0 0 2 12 92 192 192 2192 2192 184 185 IGAAAA USLAAA AAAAxx +6466 7925 0 2 6 6 66 466 466 1466 6466 132 133 SOAAAA VSLAAA HHHHxx +7096 7926 0 0 6 16 96 96 1096 2096 7096 192 193 YMAAAA WSLAAA OOOOxx +6257 7927 1 1 7 17 57 257 257 1257 6257 114 115 RGAAAA XSLAAA VVVVxx +7009 7928 1 1 9 9 9 9 1009 2009 7009 18 19 PJAAAA YSLAAA AAAAxx +8136 7929 0 0 6 16 36 136 136 3136 8136 72 73 YAAAAA ZSLAAA HHHHxx +1854 7930 0 2 4 14 54 854 1854 1854 1854 108 109 ITAAAA ATLAAA OOOOxx +3644 7931 0 0 4 4 44 644 1644 3644 3644 88 89 EKAAAA BTLAAA VVVVxx +4437 7932 1 1 7 17 37 437 437 4437 4437 74 75 ROAAAA CTLAAA AAAAxx +7209 7933 1 1 9 9 9 209 1209 2209 7209 18 19 HRAAAA DTLAAA HHHHxx +1516 7934 0 0 6 16 16 516 1516 1516 1516 32 33 IGAAAA ETLAAA OOOOxx +822 7935 0 2 2 2 22 822 822 822 822 44 45 QFAAAA FTLAAA VVVVxx +1778 7936 0 2 8 18 78 778 1778 1778 1778 156 157 KQAAAA GTLAAA AAAAxx +8161 7937 1 1 1 1 61 161 161 3161 8161 122 123 XBAAAA HTLAAA HHHHxx +6030 7938 0 2 0 10 30 30 30 1030 6030 60 61 YXAAAA ITLAAA OOOOxx +3515 7939 1 3 5 15 15 515 1515 3515 3515 30 31 FFAAAA JTLAAA VVVVxx +1702 7940 0 2 2 2 2 702 1702 1702 1702 4 5 MNAAAA KTLAAA AAAAxx +2671 7941 1 3 1 11 71 671 671 2671 2671 142 143 TYAAAA LTLAAA HHHHxx +7623 7942 1 3 3 3 23 623 1623 2623 7623 46 47 FHAAAA MTLAAA OOOOxx +9828 7943 0 0 8 8 28 828 1828 4828 9828 56 57 AOAAAA NTLAAA VVVVxx +1888 7944 0 0 8 8 88 888 1888 1888 1888 176 177 QUAAAA OTLAAA AAAAxx +4520 7945 0 0 0 0 20 520 520 4520 4520 40 41 WRAAAA PTLAAA HHHHxx +3461 7946 1 1 1 1 61 461 1461 3461 3461 122 123 DDAAAA QTLAAA OOOOxx +1488 7947 0 0 8 8 88 488 1488 1488 1488 176 177 GFAAAA RTLAAA VVVVxx +7753 7948 1 1 3 13 53 753 1753 2753 7753 106 107 FMAAAA STLAAA AAAAxx +5525 7949 1 1 5 5 25 525 1525 525 5525 50 51 NEAAAA TTLAAA HHHHxx +5220 7950 0 0 0 0 20 220 1220 220 5220 40 41 USAAAA UTLAAA OOOOxx +305 7951 1 1 5 5 5 305 305 305 305 10 11 TLAAAA VTLAAA VVVVxx +7883 7952 1 3 3 3 83 883 1883 2883 7883 166 167 FRAAAA WTLAAA AAAAxx +1222 7953 0 2 2 2 22 222 1222 1222 1222 44 45 AVAAAA XTLAAA HHHHxx +8552 7954 0 0 2 12 52 552 552 3552 8552 104 105 YQAAAA YTLAAA OOOOxx +6097 7955 1 1 7 17 97 97 97 1097 6097 194 195 NAAAAA ZTLAAA VVVVxx +2298 7956 0 2 8 18 98 298 298 2298 2298 196 197 KKAAAA AULAAA AAAAxx +956 7957 0 0 6 16 56 956 956 956 956 112 113 UKAAAA BULAAA HHHHxx +9351 7958 1 3 1 11 51 351 1351 4351 9351 102 103 RVAAAA CULAAA OOOOxx +6669 7959 1 1 9 9 69 669 669 1669 6669 138 139 NWAAAA DULAAA VVVVxx +9383 7960 1 3 3 3 83 383 1383 4383 9383 166 167 XWAAAA EULAAA AAAAxx +1607 7961 1 3 7 7 7 607 1607 1607 1607 14 15 VJAAAA FULAAA HHHHxx +812 7962 0 0 2 12 12 812 812 812 812 24 25 GFAAAA GULAAA OOOOxx +2109 7963 1 1 9 9 9 109 109 2109 2109 18 19 DDAAAA HULAAA VVVVxx +207 7964 1 3 7 7 7 207 207 207 207 14 15 ZHAAAA IULAAA AAAAxx +7124 7965 0 0 4 4 24 124 1124 2124 7124 48 49 AOAAAA JULAAA HHHHxx +9333 7966 1 1 3 13 33 333 1333 4333 9333 66 67 ZUAAAA KULAAA OOOOxx +3262 7967 0 2 2 2 62 262 1262 3262 3262 124 125 MVAAAA LULAAA VVVVxx +1070 7968 0 2 0 10 70 70 1070 1070 1070 140 141 EPAAAA MULAAA AAAAxx +7579 7969 1 3 9 19 79 579 1579 2579 7579 158 159 NFAAAA NULAAA HHHHxx +9283 7970 1 3 3 3 83 283 1283 4283 9283 166 167 BTAAAA OULAAA OOOOxx +4917 7971 1 1 7 17 17 917 917 4917 4917 34 35 DHAAAA PULAAA VVVVxx +1328 7972 0 0 8 8 28 328 1328 1328 1328 56 57 CZAAAA QULAAA AAAAxx +3042 7973 0 2 2 2 42 42 1042 3042 3042 84 85 ANAAAA RULAAA HHHHxx +8352 7974 0 0 2 12 52 352 352 3352 8352 104 105 GJAAAA SULAAA OOOOxx +2710 7975 0 2 0 10 10 710 710 2710 2710 20 21 GAAAAA TULAAA VVVVxx +3330 7976 0 2 0 10 30 330 1330 3330 3330 60 61 CYAAAA UULAAA AAAAxx +2822 7977 0 2 2 2 22 822 822 2822 2822 44 45 OEAAAA VULAAA HHHHxx +5627 7978 1 3 7 7 27 627 1627 627 5627 54 55 LIAAAA WULAAA OOOOxx +7848 7979 0 0 8 8 48 848 1848 2848 7848 96 97 WPAAAA XULAAA VVVVxx +7384 7980 0 0 4 4 84 384 1384 2384 7384 168 169 AYAAAA YULAAA AAAAxx +727 7981 1 3 7 7 27 727 727 727 727 54 55 ZBAAAA ZULAAA HHHHxx +9926 7982 0 2 6 6 26 926 1926 4926 9926 52 53 URAAAA AVLAAA OOOOxx +2647 7983 1 3 7 7 47 647 647 2647 2647 94 95 VXAAAA BVLAAA VVVVxx +6416 7984 0 0 6 16 16 416 416 1416 6416 32 33 UMAAAA CVLAAA AAAAxx +8751 7985 1 3 1 11 51 751 751 3751 8751 102 103 PYAAAA DVLAAA HHHHxx +6515 7986 1 3 5 15 15 515 515 1515 6515 30 31 PQAAAA EVLAAA OOOOxx +2472 7987 0 0 2 12 72 472 472 2472 2472 144 145 CRAAAA FVLAAA VVVVxx +7205 7988 1 1 5 5 5 205 1205 2205 7205 10 11 DRAAAA GVLAAA AAAAxx +9654 7989 0 2 4 14 54 654 1654 4654 9654 108 109 IHAAAA HVLAAA HHHHxx +5646 7990 0 2 6 6 46 646 1646 646 5646 92 93 EJAAAA IVLAAA OOOOxx +4217 7991 1 1 7 17 17 217 217 4217 4217 34 35 FGAAAA JVLAAA VVVVxx +4484 7992 0 0 4 4 84 484 484 4484 4484 168 169 MQAAAA KVLAAA AAAAxx +6654 7993 0 2 4 14 54 654 654 1654 6654 108 109 YVAAAA LVLAAA HHHHxx +4876 7994 0 0 6 16 76 876 876 4876 4876 152 153 OFAAAA MVLAAA OOOOxx +9690 7995 0 2 0 10 90 690 1690 4690 9690 180 181 SIAAAA NVLAAA VVVVxx +2453 7996 1 1 3 13 53 453 453 2453 2453 106 107 JQAAAA OVLAAA AAAAxx +829 7997 1 1 9 9 29 829 829 829 829 58 59 XFAAAA PVLAAA HHHHxx +2547 7998 1 3 7 7 47 547 547 2547 2547 94 95 ZTAAAA QVLAAA OOOOxx +9726 7999 0 2 6 6 26 726 1726 4726 9726 52 53 CKAAAA RVLAAA VVVVxx +9267 8000 1 3 7 7 67 267 1267 4267 9267 134 135 LSAAAA SVLAAA AAAAxx +7448 8001 0 0 8 8 48 448 1448 2448 7448 96 97 MAAAAA TVLAAA HHHHxx +610 8002 0 2 0 10 10 610 610 610 610 20 21 MXAAAA UVLAAA OOOOxx +2791 8003 1 3 1 11 91 791 791 2791 2791 182 183 JDAAAA VVLAAA VVVVxx +3651 8004 1 3 1 11 51 651 1651 3651 3651 102 103 LKAAAA WVLAAA AAAAxx +5206 8005 0 2 6 6 6 206 1206 206 5206 12 13 GSAAAA XVLAAA HHHHxx +8774 8006 0 2 4 14 74 774 774 3774 8774 148 149 MZAAAA YVLAAA OOOOxx +4753 8007 1 1 3 13 53 753 753 4753 4753 106 107 VAAAAA ZVLAAA VVVVxx +4755 8008 1 3 5 15 55 755 755 4755 4755 110 111 XAAAAA AWLAAA AAAAxx +686 8009 0 2 6 6 86 686 686 686 686 172 173 KAAAAA BWLAAA HHHHxx +8281 8010 1 1 1 1 81 281 281 3281 8281 162 163 NGAAAA CWLAAA OOOOxx +2058 8011 0 2 8 18 58 58 58 2058 2058 116 117 EBAAAA DWLAAA VVVVxx +8900 8012 0 0 0 0 0 900 900 3900 8900 0 1 IEAAAA EWLAAA AAAAxx +8588 8013 0 0 8 8 88 588 588 3588 8588 176 177 ISAAAA FWLAAA HHHHxx +2904 8014 0 0 4 4 4 904 904 2904 2904 8 9 SHAAAA GWLAAA OOOOxx +8917 8015 1 1 7 17 17 917 917 3917 8917 34 35 ZEAAAA HWLAAA VVVVxx +9026 8016 0 2 6 6 26 26 1026 4026 9026 52 53 EJAAAA IWLAAA AAAAxx +2416 8017 0 0 6 16 16 416 416 2416 2416 32 33 YOAAAA JWLAAA HHHHxx +1053 8018 1 1 3 13 53 53 1053 1053 1053 106 107 NOAAAA KWLAAA OOOOxx +7141 8019 1 1 1 1 41 141 1141 2141 7141 82 83 ROAAAA LWLAAA VVVVxx +9771 8020 1 3 1 11 71 771 1771 4771 9771 142 143 VLAAAA MWLAAA AAAAxx +2774 8021 0 2 4 14 74 774 774 2774 2774 148 149 SCAAAA NWLAAA HHHHxx +3213 8022 1 1 3 13 13 213 1213 3213 3213 26 27 PTAAAA OWLAAA OOOOxx +5694 8023 0 2 4 14 94 694 1694 694 5694 188 189 ALAAAA PWLAAA VVVVxx +6631 8024 1 3 1 11 31 631 631 1631 6631 62 63 BVAAAA QWLAAA AAAAxx +6638 8025 0 2 8 18 38 638 638 1638 6638 76 77 IVAAAA RWLAAA HHHHxx +7407 8026 1 3 7 7 7 407 1407 2407 7407 14 15 XYAAAA SWLAAA OOOOxx +8972 8027 0 0 2 12 72 972 972 3972 8972 144 145 CHAAAA TWLAAA VVVVxx +2202 8028 0 2 2 2 2 202 202 2202 2202 4 5 SGAAAA UWLAAA AAAAxx +6135 8029 1 3 5 15 35 135 135 1135 6135 70 71 ZBAAAA VWLAAA HHHHxx +5043 8030 1 3 3 3 43 43 1043 43 5043 86 87 ZLAAAA WWLAAA OOOOxx +5163 8031 1 3 3 3 63 163 1163 163 5163 126 127 PQAAAA XWLAAA VVVVxx +1191 8032 1 3 1 11 91 191 1191 1191 1191 182 183 VTAAAA YWLAAA AAAAxx +6576 8033 0 0 6 16 76 576 576 1576 6576 152 153 YSAAAA ZWLAAA HHHHxx +3455 8034 1 3 5 15 55 455 1455 3455 3455 110 111 XCAAAA AXLAAA OOOOxx +3688 8035 0 0 8 8 88 688 1688 3688 3688 176 177 WLAAAA BXLAAA VVVVxx +4982 8036 0 2 2 2 82 982 982 4982 4982 164 165 QJAAAA CXLAAA AAAAxx +4180 8037 0 0 0 0 80 180 180 4180 4180 160 161 UEAAAA DXLAAA HHHHxx +4708 8038 0 0 8 8 8 708 708 4708 4708 16 17 CZAAAA EXLAAA OOOOxx +1241 8039 1 1 1 1 41 241 1241 1241 1241 82 83 TVAAAA FXLAAA VVVVxx +4921 8040 1 1 1 1 21 921 921 4921 4921 42 43 HHAAAA GXLAAA AAAAxx +3197 8041 1 1 7 17 97 197 1197 3197 3197 194 195 ZSAAAA HXLAAA HHHHxx +8225 8042 1 1 5 5 25 225 225 3225 8225 50 51 JEAAAA IXLAAA OOOOxx +5913 8043 1 1 3 13 13 913 1913 913 5913 26 27 LTAAAA JXLAAA VVVVxx +6387 8044 1 3 7 7 87 387 387 1387 6387 174 175 RLAAAA KXLAAA AAAAxx +2706 8045 0 2 6 6 6 706 706 2706 2706 12 13 CAAAAA LXLAAA HHHHxx +1461 8046 1 1 1 1 61 461 1461 1461 1461 122 123 FEAAAA MXLAAA OOOOxx +7646 8047 0 2 6 6 46 646 1646 2646 7646 92 93 CIAAAA NXLAAA VVVVxx +8066 8048 0 2 6 6 66 66 66 3066 8066 132 133 GYAAAA OXLAAA AAAAxx +4171 8049 1 3 1 11 71 171 171 4171 4171 142 143 LEAAAA PXLAAA HHHHxx +8008 8050 0 0 8 8 8 8 8 3008 8008 16 17 AWAAAA QXLAAA OOOOxx +2088 8051 0 0 8 8 88 88 88 2088 2088 176 177 ICAAAA RXLAAA VVVVxx +7907 8052 1 3 7 7 7 907 1907 2907 7907 14 15 DSAAAA SXLAAA AAAAxx +2429 8053 1 1 9 9 29 429 429 2429 2429 58 59 LPAAAA TXLAAA HHHHxx +9629 8054 1 1 9 9 29 629 1629 4629 9629 58 59 JGAAAA UXLAAA OOOOxx +1470 8055 0 2 0 10 70 470 1470 1470 1470 140 141 OEAAAA VXLAAA VVVVxx +4346 8056 0 2 6 6 46 346 346 4346 4346 92 93 ELAAAA WXLAAA AAAAxx +7219 8057 1 3 9 19 19 219 1219 2219 7219 38 39 RRAAAA XXLAAA HHHHxx +1185 8058 1 1 5 5 85 185 1185 1185 1185 170 171 PTAAAA YXLAAA OOOOxx +8776 8059 0 0 6 16 76 776 776 3776 8776 152 153 OZAAAA ZXLAAA VVVVxx +684 8060 0 0 4 4 84 684 684 684 684 168 169 IAAAAA AYLAAA AAAAxx +2343 8061 1 3 3 3 43 343 343 2343 2343 86 87 DMAAAA BYLAAA HHHHxx +4470 8062 0 2 0 10 70 470 470 4470 4470 140 141 YPAAAA CYLAAA OOOOxx +5116 8063 0 0 6 16 16 116 1116 116 5116 32 33 UOAAAA DYLAAA VVVVxx +1746 8064 0 2 6 6 46 746 1746 1746 1746 92 93 EPAAAA EYLAAA AAAAxx +3216 8065 0 0 6 16 16 216 1216 3216 3216 32 33 STAAAA FYLAAA HHHHxx +4594 8066 0 2 4 14 94 594 594 4594 4594 188 189 SUAAAA GYLAAA OOOOxx +3013 8067 1 1 3 13 13 13 1013 3013 3013 26 27 XLAAAA HYLAAA VVVVxx +2307 8068 1 3 7 7 7 307 307 2307 2307 14 15 TKAAAA IYLAAA AAAAxx +7663 8069 1 3 3 3 63 663 1663 2663 7663 126 127 TIAAAA JYLAAA HHHHxx +8504 8070 0 0 4 4 4 504 504 3504 8504 8 9 CPAAAA KYLAAA OOOOxx +3683 8071 1 3 3 3 83 683 1683 3683 3683 166 167 RLAAAA LYLAAA VVVVxx +144 8072 0 0 4 4 44 144 144 144 144 88 89 OFAAAA MYLAAA AAAAxx +203 8073 1 3 3 3 3 203 203 203 203 6 7 VHAAAA NYLAAA HHHHxx +5255 8074 1 3 5 15 55 255 1255 255 5255 110 111 DUAAAA OYLAAA OOOOxx +4150 8075 0 2 0 10 50 150 150 4150 4150 100 101 QDAAAA PYLAAA VVVVxx +5701 8076 1 1 1 1 1 701 1701 701 5701 2 3 HLAAAA QYLAAA AAAAxx +7400 8077 0 0 0 0 0 400 1400 2400 7400 0 1 QYAAAA RYLAAA HHHHxx +8203 8078 1 3 3 3 3 203 203 3203 8203 6 7 NDAAAA SYLAAA OOOOxx +637 8079 1 1 7 17 37 637 637 637 637 74 75 NYAAAA TYLAAA VVVVxx +2898 8080 0 2 8 18 98 898 898 2898 2898 196 197 MHAAAA UYLAAA AAAAxx +1110 8081 0 2 0 10 10 110 1110 1110 1110 20 21 SQAAAA VYLAAA HHHHxx +6255 8082 1 3 5 15 55 255 255 1255 6255 110 111 PGAAAA WYLAAA OOOOxx +1071 8083 1 3 1 11 71 71 1071 1071 1071 142 143 FPAAAA XYLAAA VVVVxx +541 8084 1 1 1 1 41 541 541 541 541 82 83 VUAAAA YYLAAA AAAAxx +8077 8085 1 1 7 17 77 77 77 3077 8077 154 155 RYAAAA ZYLAAA HHHHxx +6809 8086 1 1 9 9 9 809 809 1809 6809 18 19 XBAAAA AZLAAA OOOOxx +4749 8087 1 1 9 9 49 749 749 4749 4749 98 99 RAAAAA BZLAAA VVVVxx +2886 8088 0 2 6 6 86 886 886 2886 2886 172 173 AHAAAA CZLAAA AAAAxx +5510 8089 0 2 0 10 10 510 1510 510 5510 20 21 YDAAAA DZLAAA HHHHxx +713 8090 1 1 3 13 13 713 713 713 713 26 27 LBAAAA EZLAAA OOOOxx +8388 8091 0 0 8 8 88 388 388 3388 8388 176 177 QKAAAA FZLAAA VVVVxx +9524 8092 0 0 4 4 24 524 1524 4524 9524 48 49 ICAAAA GZLAAA AAAAxx +9949 8093 1 1 9 9 49 949 1949 4949 9949 98 99 RSAAAA HZLAAA HHHHxx +885 8094 1 1 5 5 85 885 885 885 885 170 171 BIAAAA IZLAAA OOOOxx +8699 8095 1 3 9 19 99 699 699 3699 8699 198 199 PWAAAA JZLAAA VVVVxx +2232 8096 0 0 2 12 32 232 232 2232 2232 64 65 WHAAAA KZLAAA AAAAxx +5142 8097 0 2 2 2 42 142 1142 142 5142 84 85 UPAAAA LZLAAA HHHHxx +8891 8098 1 3 1 11 91 891 891 3891 8891 182 183 ZDAAAA MZLAAA OOOOxx +1881 8099 1 1 1 1 81 881 1881 1881 1881 162 163 JUAAAA NZLAAA VVVVxx +3751 8100 1 3 1 11 51 751 1751 3751 3751 102 103 HOAAAA OZLAAA AAAAxx +1896 8101 0 0 6 16 96 896 1896 1896 1896 192 193 YUAAAA PZLAAA HHHHxx +8258 8102 0 2 8 18 58 258 258 3258 8258 116 117 QFAAAA QZLAAA OOOOxx +3820 8103 0 0 0 0 20 820 1820 3820 3820 40 41 YQAAAA RZLAAA VVVVxx +6617 8104 1 1 7 17 17 617 617 1617 6617 34 35 NUAAAA SZLAAA AAAAxx +5100 8105 0 0 0 0 0 100 1100 100 5100 0 1 EOAAAA TZLAAA HHHHxx +4277 8106 1 1 7 17 77 277 277 4277 4277 154 155 NIAAAA UZLAAA OOOOxx +2498 8107 0 2 8 18 98 498 498 2498 2498 196 197 CSAAAA VZLAAA VVVVxx +4343 8108 1 3 3 3 43 343 343 4343 4343 86 87 BLAAAA WZLAAA AAAAxx +8319 8109 1 3 9 19 19 319 319 3319 8319 38 39 ZHAAAA XZLAAA HHHHxx +4803 8110 1 3 3 3 3 803 803 4803 4803 6 7 TCAAAA YZLAAA OOOOxx +3100 8111 0 0 0 0 0 100 1100 3100 3100 0 1 GPAAAA ZZLAAA VVVVxx +428 8112 0 0 8 8 28 428 428 428 428 56 57 MQAAAA AAMAAA AAAAxx +2811 8113 1 3 1 11 11 811 811 2811 2811 22 23 DEAAAA BAMAAA HHHHxx +2989 8114 1 1 9 9 89 989 989 2989 2989 178 179 ZKAAAA CAMAAA OOOOxx +1100 8115 0 0 0 0 0 100 1100 1100 1100 0 1 IQAAAA DAMAAA VVVVxx +6586 8116 0 2 6 6 86 586 586 1586 6586 172 173 ITAAAA EAMAAA AAAAxx +3124 8117 0 0 4 4 24 124 1124 3124 3124 48 49 EQAAAA FAMAAA HHHHxx +1635 8118 1 3 5 15 35 635 1635 1635 1635 70 71 XKAAAA GAMAAA OOOOxx +3888 8119 0 0 8 8 88 888 1888 3888 3888 176 177 OTAAAA HAMAAA VVVVxx +8369 8120 1 1 9 9 69 369 369 3369 8369 138 139 XJAAAA IAMAAA AAAAxx +3148 8121 0 0 8 8 48 148 1148 3148 3148 96 97 CRAAAA JAMAAA HHHHxx +2842 8122 0 2 2 2 42 842 842 2842 2842 84 85 IFAAAA KAMAAA OOOOxx +4965 8123 1 1 5 5 65 965 965 4965 4965 130 131 ZIAAAA LAMAAA VVVVxx +3742 8124 0 2 2 2 42 742 1742 3742 3742 84 85 YNAAAA MAMAAA AAAAxx +5196 8125 0 0 6 16 96 196 1196 196 5196 192 193 WRAAAA NAMAAA HHHHxx +9105 8126 1 1 5 5 5 105 1105 4105 9105 10 11 FMAAAA OAMAAA OOOOxx +6806 8127 0 2 6 6 6 806 806 1806 6806 12 13 UBAAAA PAMAAA VVVVxx +5849 8128 1 1 9 9 49 849 1849 849 5849 98 99 ZQAAAA QAMAAA AAAAxx +6504 8129 0 0 4 4 4 504 504 1504 6504 8 9 EQAAAA RAMAAA HHHHxx +9841 8130 1 1 1 1 41 841 1841 4841 9841 82 83 NOAAAA SAMAAA OOOOxx +457 8131 1 1 7 17 57 457 457 457 457 114 115 PRAAAA TAMAAA VVVVxx +8856 8132 0 0 6 16 56 856 856 3856 8856 112 113 QCAAAA UAMAAA AAAAxx +8043 8133 1 3 3 3 43 43 43 3043 8043 86 87 JXAAAA VAMAAA HHHHxx +5933 8134 1 1 3 13 33 933 1933 933 5933 66 67 FUAAAA WAMAAA OOOOxx +5725 8135 1 1 5 5 25 725 1725 725 5725 50 51 FMAAAA XAMAAA VVVVxx +8607 8136 1 3 7 7 7 607 607 3607 8607 14 15 BTAAAA YAMAAA AAAAxx +9280 8137 0 0 0 0 80 280 1280 4280 9280 160 161 YSAAAA ZAMAAA HHHHxx +6017 8138 1 1 7 17 17 17 17 1017 6017 34 35 LXAAAA ABMAAA OOOOxx +4946 8139 0 2 6 6 46 946 946 4946 4946 92 93 GIAAAA BBMAAA VVVVxx +7373 8140 1 1 3 13 73 373 1373 2373 7373 146 147 PXAAAA CBMAAA AAAAxx +8096 8141 0 0 6 16 96 96 96 3096 8096 192 193 KZAAAA DBMAAA HHHHxx +3178 8142 0 2 8 18 78 178 1178 3178 3178 156 157 GSAAAA EBMAAA OOOOxx +1849 8143 1 1 9 9 49 849 1849 1849 1849 98 99 DTAAAA FBMAAA VVVVxx +8813 8144 1 1 3 13 13 813 813 3813 8813 26 27 ZAAAAA GBMAAA AAAAxx +460 8145 0 0 0 0 60 460 460 460 460 120 121 SRAAAA HBMAAA HHHHxx +7756 8146 0 0 6 16 56 756 1756 2756 7756 112 113 IMAAAA IBMAAA OOOOxx +4425 8147 1 1 5 5 25 425 425 4425 4425 50 51 FOAAAA JBMAAA VVVVxx +1602 8148 0 2 2 2 2 602 1602 1602 1602 4 5 QJAAAA KBMAAA AAAAxx +5981 8149 1 1 1 1 81 981 1981 981 5981 162 163 BWAAAA LBMAAA HHHHxx +8139 8150 1 3 9 19 39 139 139 3139 8139 78 79 BBAAAA MBMAAA OOOOxx +754 8151 0 2 4 14 54 754 754 754 754 108 109 ADAAAA NBMAAA VVVVxx +26 8152 0 2 6 6 26 26 26 26 26 52 53 ABAAAA OBMAAA AAAAxx +106 8153 0 2 6 6 6 106 106 106 106 12 13 CEAAAA PBMAAA HHHHxx +7465 8154 1 1 5 5 65 465 1465 2465 7465 130 131 DBAAAA QBMAAA OOOOxx +1048 8155 0 0 8 8 48 48 1048 1048 1048 96 97 IOAAAA RBMAAA VVVVxx +2303 8156 1 3 3 3 3 303 303 2303 2303 6 7 PKAAAA SBMAAA AAAAxx +5794 8157 0 2 4 14 94 794 1794 794 5794 188 189 WOAAAA TBMAAA HHHHxx +3321 8158 1 1 1 1 21 321 1321 3321 3321 42 43 TXAAAA UBMAAA OOOOxx +6122 8159 0 2 2 2 22 122 122 1122 6122 44 45 MBAAAA VBMAAA VVVVxx +6474 8160 0 2 4 14 74 474 474 1474 6474 148 149 APAAAA WBMAAA AAAAxx +827 8161 1 3 7 7 27 827 827 827 827 54 55 VFAAAA XBMAAA HHHHxx +6616 8162 0 0 6 16 16 616 616 1616 6616 32 33 MUAAAA YBMAAA OOOOxx +2131 8163 1 3 1 11 31 131 131 2131 2131 62 63 ZDAAAA ZBMAAA VVVVxx +5483 8164 1 3 3 3 83 483 1483 483 5483 166 167 XCAAAA ACMAAA AAAAxx +606 8165 0 2 6 6 6 606 606 606 606 12 13 IXAAAA BCMAAA HHHHxx +922 8166 0 2 2 2 22 922 922 922 922 44 45 MJAAAA CCMAAA OOOOxx +8475 8167 1 3 5 15 75 475 475 3475 8475 150 151 ZNAAAA DCMAAA VVVVxx +7645 8168 1 1 5 5 45 645 1645 2645 7645 90 91 BIAAAA ECMAAA AAAAxx +5097 8169 1 1 7 17 97 97 1097 97 5097 194 195 BOAAAA FCMAAA HHHHxx +5377 8170 1 1 7 17 77 377 1377 377 5377 154 155 VYAAAA GCMAAA OOOOxx +6116 8171 0 0 6 16 16 116 116 1116 6116 32 33 GBAAAA HCMAAA VVVVxx +8674 8172 0 2 4 14 74 674 674 3674 8674 148 149 QVAAAA ICMAAA AAAAxx +8063 8173 1 3 3 3 63 63 63 3063 8063 126 127 DYAAAA JCMAAA HHHHxx +5271 8174 1 3 1 11 71 271 1271 271 5271 142 143 TUAAAA KCMAAA OOOOxx +1619 8175 1 3 9 19 19 619 1619 1619 1619 38 39 HKAAAA LCMAAA VVVVxx +6419 8176 1 3 9 19 19 419 419 1419 6419 38 39 XMAAAA MCMAAA AAAAxx +7651 8177 1 3 1 11 51 651 1651 2651 7651 102 103 HIAAAA NCMAAA HHHHxx +2897 8178 1 1 7 17 97 897 897 2897 2897 194 195 LHAAAA OCMAAA OOOOxx +8148 8179 0 0 8 8 48 148 148 3148 8148 96 97 KBAAAA PCMAAA VVVVxx +7461 8180 1 1 1 1 61 461 1461 2461 7461 122 123 ZAAAAA QCMAAA AAAAxx +9186 8181 0 2 6 6 86 186 1186 4186 9186 172 173 IPAAAA RCMAAA HHHHxx +7127 8182 1 3 7 7 27 127 1127 2127 7127 54 55 DOAAAA SCMAAA OOOOxx +8233 8183 1 1 3 13 33 233 233 3233 8233 66 67 REAAAA TCMAAA VVVVxx +9651 8184 1 3 1 11 51 651 1651 4651 9651 102 103 FHAAAA UCMAAA AAAAxx +6746 8185 0 2 6 6 46 746 746 1746 6746 92 93 MZAAAA VCMAAA HHHHxx +7835 8186 1 3 5 15 35 835 1835 2835 7835 70 71 JPAAAA WCMAAA OOOOxx +8815 8187 1 3 5 15 15 815 815 3815 8815 30 31 BBAAAA XCMAAA VVVVxx +6398 8188 0 2 8 18 98 398 398 1398 6398 196 197 CMAAAA YCMAAA AAAAxx +5344 8189 0 0 4 4 44 344 1344 344 5344 88 89 OXAAAA ZCMAAA HHHHxx +8209 8190 1 1 9 9 9 209 209 3209 8209 18 19 TDAAAA ADMAAA OOOOxx +8444 8191 0 0 4 4 44 444 444 3444 8444 88 89 UMAAAA BDMAAA VVVVxx +5669 8192 1 1 9 9 69 669 1669 669 5669 138 139 BKAAAA CDMAAA AAAAxx +2455 8193 1 3 5 15 55 455 455 2455 2455 110 111 LQAAAA DDMAAA HHHHxx +6767 8194 1 3 7 7 67 767 767 1767 6767 134 135 HAAAAA EDMAAA OOOOxx +135 8195 1 3 5 15 35 135 135 135 135 70 71 FFAAAA FDMAAA VVVVxx +3503 8196 1 3 3 3 3 503 1503 3503 3503 6 7 TEAAAA GDMAAA AAAAxx +6102 8197 0 2 2 2 2 102 102 1102 6102 4 5 SAAAAA HDMAAA HHHHxx +7136 8198 0 0 6 16 36 136 1136 2136 7136 72 73 MOAAAA IDMAAA OOOOxx +4933 8199 1 1 3 13 33 933 933 4933 4933 66 67 THAAAA JDMAAA VVVVxx +8804 8200 0 0 4 4 4 804 804 3804 8804 8 9 QAAAAA KDMAAA AAAAxx +3760 8201 0 0 0 0 60 760 1760 3760 3760 120 121 QOAAAA LDMAAA HHHHxx +8603 8202 1 3 3 3 3 603 603 3603 8603 6 7 XSAAAA MDMAAA OOOOxx +7411 8203 1 3 1 11 11 411 1411 2411 7411 22 23 BZAAAA NDMAAA VVVVxx +834 8204 0 2 4 14 34 834 834 834 834 68 69 CGAAAA ODMAAA AAAAxx +7385 8205 1 1 5 5 85 385 1385 2385 7385 170 171 BYAAAA PDMAAA HHHHxx +3696 8206 0 0 6 16 96 696 1696 3696 3696 192 193 EMAAAA QDMAAA OOOOxx +8720 8207 0 0 0 0 20 720 720 3720 8720 40 41 KXAAAA RDMAAA VVVVxx +4539 8208 1 3 9 19 39 539 539 4539 4539 78 79 PSAAAA SDMAAA AAAAxx +9837 8209 1 1 7 17 37 837 1837 4837 9837 74 75 JOAAAA TDMAAA HHHHxx +8595 8210 1 3 5 15 95 595 595 3595 8595 190 191 PSAAAA UDMAAA OOOOxx +3673 8211 1 1 3 13 73 673 1673 3673 3673 146 147 HLAAAA VDMAAA VVVVxx +475 8212 1 3 5 15 75 475 475 475 475 150 151 HSAAAA WDMAAA AAAAxx +2256 8213 0 0 6 16 56 256 256 2256 2256 112 113 UIAAAA XDMAAA HHHHxx +6349 8214 1 1 9 9 49 349 349 1349 6349 98 99 FKAAAA YDMAAA OOOOxx +9968 8215 0 0 8 8 68 968 1968 4968 9968 136 137 KTAAAA ZDMAAA VVVVxx +7261 8216 1 1 1 1 61 261 1261 2261 7261 122 123 HTAAAA AEMAAA AAAAxx +5799 8217 1 3 9 19 99 799 1799 799 5799 198 199 BPAAAA BEMAAA HHHHxx +8159 8218 1 3 9 19 59 159 159 3159 8159 118 119 VBAAAA CEMAAA OOOOxx +92 8219 0 0 2 12 92 92 92 92 92 184 185 ODAAAA DEMAAA VVVVxx +5927 8220 1 3 7 7 27 927 1927 927 5927 54 55 ZTAAAA EEMAAA AAAAxx +7925 8221 1 1 5 5 25 925 1925 2925 7925 50 51 VSAAAA FEMAAA HHHHxx +5836 8222 0 0 6 16 36 836 1836 836 5836 72 73 MQAAAA GEMAAA OOOOxx +7935 8223 1 3 5 15 35 935 1935 2935 7935 70 71 FTAAAA HEMAAA VVVVxx +5505 8224 1 1 5 5 5 505 1505 505 5505 10 11 TDAAAA IEMAAA AAAAxx +5882 8225 0 2 2 2 82 882 1882 882 5882 164 165 GSAAAA JEMAAA HHHHxx +4411 8226 1 3 1 11 11 411 411 4411 4411 22 23 RNAAAA KEMAAA OOOOxx +64 8227 0 0 4 4 64 64 64 64 64 128 129 MCAAAA LEMAAA VVVVxx +2851 8228 1 3 1 11 51 851 851 2851 2851 102 103 RFAAAA MEMAAA AAAAxx +1665 8229 1 1 5 5 65 665 1665 1665 1665 130 131 BMAAAA NEMAAA HHHHxx +2895 8230 1 3 5 15 95 895 895 2895 2895 190 191 JHAAAA OEMAAA OOOOxx +2210 8231 0 2 0 10 10 210 210 2210 2210 20 21 AHAAAA PEMAAA VVVVxx +9873 8232 1 1 3 13 73 873 1873 4873 9873 146 147 TPAAAA QEMAAA AAAAxx +5402 8233 0 2 2 2 2 402 1402 402 5402 4 5 UZAAAA REMAAA HHHHxx +285 8234 1 1 5 5 85 285 285 285 285 170 171 ZKAAAA SEMAAA OOOOxx +8545 8235 1 1 5 5 45 545 545 3545 8545 90 91 RQAAAA TEMAAA VVVVxx +5328 8236 0 0 8 8 28 328 1328 328 5328 56 57 YWAAAA UEMAAA AAAAxx +733 8237 1 1 3 13 33 733 733 733 733 66 67 FCAAAA VEMAAA HHHHxx +7726 8238 0 2 6 6 26 726 1726 2726 7726 52 53 ELAAAA WEMAAA OOOOxx +5418 8239 0 2 8 18 18 418 1418 418 5418 36 37 KAAAAA XEMAAA VVVVxx +7761 8240 1 1 1 1 61 761 1761 2761 7761 122 123 NMAAAA YEMAAA AAAAxx +9263 8241 1 3 3 3 63 263 1263 4263 9263 126 127 HSAAAA ZEMAAA HHHHxx +5579 8242 1 3 9 19 79 579 1579 579 5579 158 159 PGAAAA AFMAAA OOOOxx +5434 8243 0 2 4 14 34 434 1434 434 5434 68 69 ABAAAA BFMAAA VVVVxx +5230 8244 0 2 0 10 30 230 1230 230 5230 60 61 ETAAAA CFMAAA AAAAxx +9981 8245 1 1 1 1 81 981 1981 4981 9981 162 163 XTAAAA DFMAAA HHHHxx +5830 8246 0 2 0 10 30 830 1830 830 5830 60 61 GQAAAA EFMAAA OOOOxx +128 8247 0 0 8 8 28 128 128 128 128 56 57 YEAAAA FFMAAA VVVVxx +2734 8248 0 2 4 14 34 734 734 2734 2734 68 69 EBAAAA GFMAAA AAAAxx +4537 8249 1 1 7 17 37 537 537 4537 4537 74 75 NSAAAA HFMAAA HHHHxx +3899 8250 1 3 9 19 99 899 1899 3899 3899 198 199 ZTAAAA IFMAAA OOOOxx +1000 8251 0 0 0 0 0 0 1000 1000 1000 0 1 MMAAAA JFMAAA VVVVxx +9896 8252 0 0 6 16 96 896 1896 4896 9896 192 193 QQAAAA KFMAAA AAAAxx +3640 8253 0 0 0 0 40 640 1640 3640 3640 80 81 AKAAAA LFMAAA HHHHxx +2568 8254 0 0 8 8 68 568 568 2568 2568 136 137 UUAAAA MFMAAA OOOOxx +2026 8255 0 2 6 6 26 26 26 2026 2026 52 53 YZAAAA NFMAAA VVVVxx +3955 8256 1 3 5 15 55 955 1955 3955 3955 110 111 DWAAAA OFMAAA AAAAxx +7152 8257 0 0 2 12 52 152 1152 2152 7152 104 105 CPAAAA PFMAAA HHHHxx +2402 8258 0 2 2 2 2 402 402 2402 2402 4 5 KOAAAA QFMAAA OOOOxx +9522 8259 0 2 2 2 22 522 1522 4522 9522 44 45 GCAAAA RFMAAA VVVVxx +4011 8260 1 3 1 11 11 11 11 4011 4011 22 23 HYAAAA SFMAAA AAAAxx +3297 8261 1 1 7 17 97 297 1297 3297 3297 194 195 VWAAAA TFMAAA HHHHxx +4915 8262 1 3 5 15 15 915 915 4915 4915 30 31 BHAAAA UFMAAA OOOOxx +5397 8263 1 1 7 17 97 397 1397 397 5397 194 195 PZAAAA VFMAAA VVVVxx +5454 8264 0 2 4 14 54 454 1454 454 5454 108 109 UBAAAA WFMAAA AAAAxx +4568 8265 0 0 8 8 68 568 568 4568 4568 136 137 STAAAA XFMAAA HHHHxx +5875 8266 1 3 5 15 75 875 1875 875 5875 150 151 ZRAAAA YFMAAA OOOOxx +3642 8267 0 2 2 2 42 642 1642 3642 3642 84 85 CKAAAA ZFMAAA VVVVxx +8506 8268 0 2 6 6 6 506 506 3506 8506 12 13 EPAAAA AGMAAA AAAAxx +9621 8269 1 1 1 1 21 621 1621 4621 9621 42 43 BGAAAA BGMAAA HHHHxx +7739 8270 1 3 9 19 39 739 1739 2739 7739 78 79 RLAAAA CGMAAA OOOOxx +3987 8271 1 3 7 7 87 987 1987 3987 3987 174 175 JXAAAA DGMAAA VVVVxx +2090 8272 0 2 0 10 90 90 90 2090 2090 180 181 KCAAAA EGMAAA AAAAxx +3838 8273 0 2 8 18 38 838 1838 3838 3838 76 77 QRAAAA FGMAAA HHHHxx +17 8274 1 1 7 17 17 17 17 17 17 34 35 RAAAAA GGMAAA OOOOxx +3406 8275 0 2 6 6 6 406 1406 3406 3406 12 13 ABAAAA HGMAAA VVVVxx +8312 8276 0 0 2 12 12 312 312 3312 8312 24 25 SHAAAA IGMAAA AAAAxx +4034 8277 0 2 4 14 34 34 34 4034 4034 68 69 EZAAAA JGMAAA HHHHxx +1535 8278 1 3 5 15 35 535 1535 1535 1535 70 71 BHAAAA KGMAAA OOOOxx +7198 8279 0 2 8 18 98 198 1198 2198 7198 196 197 WQAAAA LGMAAA VVVVxx +8885 8280 1 1 5 5 85 885 885 3885 8885 170 171 TDAAAA MGMAAA AAAAxx +4081 8281 1 1 1 1 81 81 81 4081 4081 162 163 ZAAAAA NGMAAA HHHHxx +980 8282 0 0 0 0 80 980 980 980 980 160 161 SLAAAA OGMAAA OOOOxx +551 8283 1 3 1 11 51 551 551 551 551 102 103 FVAAAA PGMAAA VVVVxx +7746 8284 0 2 6 6 46 746 1746 2746 7746 92 93 YLAAAA QGMAAA AAAAxx +4756 8285 0 0 6 16 56 756 756 4756 4756 112 113 YAAAAA RGMAAA HHHHxx +3655 8286 1 3 5 15 55 655 1655 3655 3655 110 111 PKAAAA SGMAAA OOOOxx +7075 8287 1 3 5 15 75 75 1075 2075 7075 150 151 DMAAAA TGMAAA VVVVxx +3950 8288 0 2 0 10 50 950 1950 3950 3950 100 101 YVAAAA UGMAAA AAAAxx +2314 8289 0 2 4 14 14 314 314 2314 2314 28 29 ALAAAA VGMAAA HHHHxx +8432 8290 0 0 2 12 32 432 432 3432 8432 64 65 IMAAAA WGMAAA OOOOxx +62 8291 0 2 2 2 62 62 62 62 62 124 125 KCAAAA XGMAAA VVVVxx +6920 8292 0 0 0 0 20 920 920 1920 6920 40 41 EGAAAA YGMAAA AAAAxx +4077 8293 1 1 7 17 77 77 77 4077 4077 154 155 VAAAAA ZGMAAA HHHHxx +9118 8294 0 2 8 18 18 118 1118 4118 9118 36 37 SMAAAA AHMAAA OOOOxx +5375 8295 1 3 5 15 75 375 1375 375 5375 150 151 TYAAAA BHMAAA VVVVxx +178 8296 0 2 8 18 78 178 178 178 178 156 157 WGAAAA CHMAAA AAAAxx +1079 8297 1 3 9 19 79 79 1079 1079 1079 158 159 NPAAAA DHMAAA HHHHxx +4279 8298 1 3 9 19 79 279 279 4279 4279 158 159 PIAAAA EHMAAA OOOOxx +8436 8299 0 0 6 16 36 436 436 3436 8436 72 73 MMAAAA FHMAAA VVVVxx +1931 8300 1 3 1 11 31 931 1931 1931 1931 62 63 HWAAAA GHMAAA AAAAxx +2096 8301 0 0 6 16 96 96 96 2096 2096 192 193 QCAAAA HHMAAA HHHHxx +1638 8302 0 2 8 18 38 638 1638 1638 1638 76 77 ALAAAA IHMAAA OOOOxx +2788 8303 0 0 8 8 88 788 788 2788 2788 176 177 GDAAAA JHMAAA VVVVxx +4751 8304 1 3 1 11 51 751 751 4751 4751 102 103 TAAAAA KHMAAA AAAAxx +8824 8305 0 0 4 4 24 824 824 3824 8824 48 49 KBAAAA LHMAAA HHHHxx +3098 8306 0 2 8 18 98 98 1098 3098 3098 196 197 EPAAAA MHMAAA OOOOxx +4497 8307 1 1 7 17 97 497 497 4497 4497 194 195 ZQAAAA NHMAAA VVVVxx +5223 8308 1 3 3 3 23 223 1223 223 5223 46 47 XSAAAA OHMAAA AAAAxx +9212 8309 0 0 2 12 12 212 1212 4212 9212 24 25 IQAAAA PHMAAA HHHHxx +4265 8310 1 1 5 5 65 265 265 4265 4265 130 131 BIAAAA QHMAAA OOOOxx +6898 8311 0 2 8 18 98 898 898 1898 6898 196 197 IFAAAA RHMAAA VVVVxx +8808 8312 0 0 8 8 8 808 808 3808 8808 16 17 UAAAAA SHMAAA AAAAxx +5629 8313 1 1 9 9 29 629 1629 629 5629 58 59 NIAAAA THMAAA HHHHxx +3779 8314 1 3 9 19 79 779 1779 3779 3779 158 159 JPAAAA UHMAAA OOOOxx +4972 8315 0 0 2 12 72 972 972 4972 4972 144 145 GJAAAA VHMAAA VVVVxx +4511 8316 1 3 1 11 11 511 511 4511 4511 22 23 NRAAAA WHMAAA AAAAxx +6761 8317 1 1 1 1 61 761 761 1761 6761 122 123 BAAAAA XHMAAA HHHHxx +2335 8318 1 3 5 15 35 335 335 2335 2335 70 71 VLAAAA YHMAAA OOOOxx +732 8319 0 0 2 12 32 732 732 732 732 64 65 ECAAAA ZHMAAA VVVVxx +4757 8320 1 1 7 17 57 757 757 4757 4757 114 115 ZAAAAA AIMAAA AAAAxx +6624 8321 0 0 4 4 24 624 624 1624 6624 48 49 UUAAAA BIMAAA HHHHxx +5869 8322 1 1 9 9 69 869 1869 869 5869 138 139 TRAAAA CIMAAA OOOOxx +5842 8323 0 2 2 2 42 842 1842 842 5842 84 85 SQAAAA DIMAAA VVVVxx +5735 8324 1 3 5 15 35 735 1735 735 5735 70 71 PMAAAA EIMAAA AAAAxx +8276 8325 0 0 6 16 76 276 276 3276 8276 152 153 IGAAAA FIMAAA HHHHxx +7227 8326 1 3 7 7 27 227 1227 2227 7227 54 55 ZRAAAA GIMAAA OOOOxx +4923 8327 1 3 3 3 23 923 923 4923 4923 46 47 JHAAAA HIMAAA VVVVxx +9135 8328 1 3 5 15 35 135 1135 4135 9135 70 71 JNAAAA IIMAAA AAAAxx +5813 8329 1 1 3 13 13 813 1813 813 5813 26 27 PPAAAA JIMAAA HHHHxx +9697 8330 1 1 7 17 97 697 1697 4697 9697 194 195 ZIAAAA KIMAAA OOOOxx +3222 8331 0 2 2 2 22 222 1222 3222 3222 44 45 YTAAAA LIMAAA VVVVxx +2394 8332 0 2 4 14 94 394 394 2394 2394 188 189 COAAAA MIMAAA AAAAxx +5784 8333 0 0 4 4 84 784 1784 784 5784 168 169 MOAAAA NIMAAA HHHHxx +3652 8334 0 0 2 12 52 652 1652 3652 3652 104 105 MKAAAA OIMAAA OOOOxx +8175 8335 1 3 5 15 75 175 175 3175 8175 150 151 LCAAAA PIMAAA VVVVxx +7568 8336 0 0 8 8 68 568 1568 2568 7568 136 137 CFAAAA QIMAAA AAAAxx +6645 8337 1 1 5 5 45 645 645 1645 6645 90 91 PVAAAA RIMAAA HHHHxx +8176 8338 0 0 6 16 76 176 176 3176 8176 152 153 MCAAAA SIMAAA OOOOxx +530 8339 0 2 0 10 30 530 530 530 530 60 61 KUAAAA TIMAAA VVVVxx +5439 8340 1 3 9 19 39 439 1439 439 5439 78 79 FBAAAA UIMAAA AAAAxx +61 8341 1 1 1 1 61 61 61 61 61 122 123 JCAAAA VIMAAA HHHHxx +3951 8342 1 3 1 11 51 951 1951 3951 3951 102 103 ZVAAAA WIMAAA OOOOxx +5283 8343 1 3 3 3 83 283 1283 283 5283 166 167 FVAAAA XIMAAA VVVVxx +7226 8344 0 2 6 6 26 226 1226 2226 7226 52 53 YRAAAA YIMAAA AAAAxx +1954 8345 0 2 4 14 54 954 1954 1954 1954 108 109 EXAAAA ZIMAAA HHHHxx +334 8346 0 2 4 14 34 334 334 334 334 68 69 WMAAAA AJMAAA OOOOxx +3921 8347 1 1 1 1 21 921 1921 3921 3921 42 43 VUAAAA BJMAAA VVVVxx +6276 8348 0 0 6 16 76 276 276 1276 6276 152 153 KHAAAA CJMAAA AAAAxx +3378 8349 0 2 8 18 78 378 1378 3378 3378 156 157 YZAAAA DJMAAA HHHHxx +5236 8350 0 0 6 16 36 236 1236 236 5236 72 73 KTAAAA EJMAAA OOOOxx +7781 8351 1 1 1 1 81 781 1781 2781 7781 162 163 HNAAAA FJMAAA VVVVxx +8601 8352 1 1 1 1 1 601 601 3601 8601 2 3 VSAAAA GJMAAA AAAAxx +1473 8353 1 1 3 13 73 473 1473 1473 1473 146 147 REAAAA HJMAAA HHHHxx +3246 8354 0 2 6 6 46 246 1246 3246 3246 92 93 WUAAAA IJMAAA OOOOxx +3601 8355 1 1 1 1 1 601 1601 3601 3601 2 3 NIAAAA JJMAAA VVVVxx +6861 8356 1 1 1 1 61 861 861 1861 6861 122 123 XDAAAA KJMAAA AAAAxx +9032 8357 0 0 2 12 32 32 1032 4032 9032 64 65 KJAAAA LJMAAA HHHHxx +216 8358 0 0 6 16 16 216 216 216 216 32 33 IIAAAA MJMAAA OOOOxx +3824 8359 0 0 4 4 24 824 1824 3824 3824 48 49 CRAAAA NJMAAA VVVVxx +8486 8360 0 2 6 6 86 486 486 3486 8486 172 173 KOAAAA OJMAAA AAAAxx +276 8361 0 0 6 16 76 276 276 276 276 152 153 QKAAAA PJMAAA HHHHxx +1838 8362 0 2 8 18 38 838 1838 1838 1838 76 77 SSAAAA QJMAAA OOOOxx +6175 8363 1 3 5 15 75 175 175 1175 6175 150 151 NDAAAA RJMAAA VVVVxx +3719 8364 1 3 9 19 19 719 1719 3719 3719 38 39 BNAAAA SJMAAA AAAAxx +6958 8365 0 2 8 18 58 958 958 1958 6958 116 117 QHAAAA TJMAAA HHHHxx +6822 8366 0 2 2 2 22 822 822 1822 6822 44 45 KCAAAA UJMAAA OOOOxx +3318 8367 0 2 8 18 18 318 1318 3318 3318 36 37 QXAAAA VJMAAA VVVVxx +7222 8368 0 2 2 2 22 222 1222 2222 7222 44 45 URAAAA WJMAAA AAAAxx +85 8369 1 1 5 5 85 85 85 85 85 170 171 HDAAAA XJMAAA HHHHxx +5158 8370 0 2 8 18 58 158 1158 158 5158 116 117 KQAAAA YJMAAA OOOOxx +6360 8371 0 0 0 0 60 360 360 1360 6360 120 121 QKAAAA ZJMAAA VVVVxx +2599 8372 1 3 9 19 99 599 599 2599 2599 198 199 ZVAAAA AKMAAA AAAAxx +4002 8373 0 2 2 2 2 2 2 4002 4002 4 5 YXAAAA BKMAAA HHHHxx +6597 8374 1 1 7 17 97 597 597 1597 6597 194 195 TTAAAA CKMAAA OOOOxx +5762 8375 0 2 2 2 62 762 1762 762 5762 124 125 QNAAAA DKMAAA VVVVxx +8383 8376 1 3 3 3 83 383 383 3383 8383 166 167 LKAAAA EKMAAA AAAAxx +4686 8377 0 2 6 6 86 686 686 4686 4686 172 173 GYAAAA FKMAAA HHHHxx +5972 8378 0 0 2 12 72 972 1972 972 5972 144 145 SVAAAA GKMAAA OOOOxx +1432 8379 0 0 2 12 32 432 1432 1432 1432 64 65 CDAAAA HKMAAA VVVVxx +1601 8380 1 1 1 1 1 601 1601 1601 1601 2 3 PJAAAA IKMAAA AAAAxx +3012 8381 0 0 2 12 12 12 1012 3012 3012 24 25 WLAAAA JKMAAA HHHHxx +9345 8382 1 1 5 5 45 345 1345 4345 9345 90 91 LVAAAA KKMAAA OOOOxx +8869 8383 1 1 9 9 69 869 869 3869 8869 138 139 DDAAAA LKMAAA VVVVxx +6612 8384 0 0 2 12 12 612 612 1612 6612 24 25 IUAAAA MKMAAA AAAAxx +262 8385 0 2 2 2 62 262 262 262 262 124 125 CKAAAA NKMAAA HHHHxx +300 8386 0 0 0 0 0 300 300 300 300 0 1 OLAAAA OKMAAA OOOOxx +3045 8387 1 1 5 5 45 45 1045 3045 3045 90 91 DNAAAA PKMAAA VVVVxx +7252 8388 0 0 2 12 52 252 1252 2252 7252 104 105 YSAAAA QKMAAA AAAAxx +9099 8389 1 3 9 19 99 99 1099 4099 9099 198 199 ZLAAAA RKMAAA HHHHxx +9006 8390 0 2 6 6 6 6 1006 4006 9006 12 13 KIAAAA SKMAAA OOOOxx +3078 8391 0 2 8 18 78 78 1078 3078 3078 156 157 KOAAAA TKMAAA VVVVxx +5159 8392 1 3 9 19 59 159 1159 159 5159 118 119 LQAAAA UKMAAA AAAAxx +9329 8393 1 1 9 9 29 329 1329 4329 9329 58 59 VUAAAA VKMAAA HHHHxx +1393 8394 1 1 3 13 93 393 1393 1393 1393 186 187 PBAAAA WKMAAA OOOOxx +5894 8395 0 2 4 14 94 894 1894 894 5894 188 189 SSAAAA XKMAAA VVVVxx +11 8396 1 3 1 11 11 11 11 11 11 22 23 LAAAAA YKMAAA AAAAxx +5606 8397 0 2 6 6 6 606 1606 606 5606 12 13 QHAAAA ZKMAAA HHHHxx +5541 8398 1 1 1 1 41 541 1541 541 5541 82 83 DFAAAA ALMAAA OOOOxx +2689 8399 1 1 9 9 89 689 689 2689 2689 178 179 LZAAAA BLMAAA VVVVxx +1023 8400 1 3 3 3 23 23 1023 1023 1023 46 47 JNAAAA CLMAAA AAAAxx +8134 8401 0 2 4 14 34 134 134 3134 8134 68 69 WAAAAA DLMAAA HHHHxx +5923 8402 1 3 3 3 23 923 1923 923 5923 46 47 VTAAAA ELMAAA OOOOxx +6056 8403 0 0 6 16 56 56 56 1056 6056 112 113 YYAAAA FLMAAA VVVVxx +653 8404 1 1 3 13 53 653 653 653 653 106 107 DZAAAA GLMAAA AAAAxx +367 8405 1 3 7 7 67 367 367 367 367 134 135 DOAAAA HLMAAA HHHHxx +1828 8406 0 0 8 8 28 828 1828 1828 1828 56 57 ISAAAA ILMAAA OOOOxx +6506 8407 0 2 6 6 6 506 506 1506 6506 12 13 GQAAAA JLMAAA VVVVxx +5772 8408 0 0 2 12 72 772 1772 772 5772 144 145 AOAAAA KLMAAA AAAAxx +8052 8409 0 0 2 12 52 52 52 3052 8052 104 105 SXAAAA LLMAAA HHHHxx +2633 8410 1 1 3 13 33 633 633 2633 2633 66 67 HXAAAA MLMAAA OOOOxx +4878 8411 0 2 8 18 78 878 878 4878 4878 156 157 QFAAAA NLMAAA VVVVxx +5621 8412 1 1 1 1 21 621 1621 621 5621 42 43 FIAAAA OLMAAA AAAAxx +41 8413 1 1 1 1 41 41 41 41 41 82 83 PBAAAA PLMAAA HHHHxx +4613 8414 1 1 3 13 13 613 613 4613 4613 26 27 LVAAAA QLMAAA OOOOxx +9389 8415 1 1 9 9 89 389 1389 4389 9389 178 179 DXAAAA RLMAAA VVVVxx +9414 8416 0 2 4 14 14 414 1414 4414 9414 28 29 CYAAAA SLMAAA AAAAxx +3583 8417 1 3 3 3 83 583 1583 3583 3583 166 167 VHAAAA TLMAAA HHHHxx +3454 8418 0 2 4 14 54 454 1454 3454 3454 108 109 WCAAAA ULMAAA OOOOxx +719 8419 1 3 9 19 19 719 719 719 719 38 39 RBAAAA VLMAAA VVVVxx +6188 8420 0 0 8 8 88 188 188 1188 6188 176 177 AEAAAA WLMAAA AAAAxx +2288 8421 0 0 8 8 88 288 288 2288 2288 176 177 AKAAAA XLMAAA HHHHxx +1287 8422 1 3 7 7 87 287 1287 1287 1287 174 175 NXAAAA YLMAAA OOOOxx +1397 8423 1 1 7 17 97 397 1397 1397 1397 194 195 TBAAAA ZLMAAA VVVVxx +7763 8424 1 3 3 3 63 763 1763 2763 7763 126 127 PMAAAA AMMAAA AAAAxx +5194 8425 0 2 4 14 94 194 1194 194 5194 188 189 URAAAA BMMAAA HHHHxx +3167 8426 1 3 7 7 67 167 1167 3167 3167 134 135 VRAAAA CMMAAA OOOOxx +9218 8427 0 2 8 18 18 218 1218 4218 9218 36 37 OQAAAA DMMAAA VVVVxx +2065 8428 1 1 5 5 65 65 65 2065 2065 130 131 LBAAAA EMMAAA AAAAxx +9669 8429 1 1 9 9 69 669 1669 4669 9669 138 139 XHAAAA FMMAAA HHHHxx +146 8430 0 2 6 6 46 146 146 146 146 92 93 QFAAAA GMMAAA OOOOxx +6141 8431 1 1 1 1 41 141 141 1141 6141 82 83 FCAAAA HMMAAA VVVVxx +2843 8432 1 3 3 3 43 843 843 2843 2843 86 87 JFAAAA IMMAAA AAAAxx +7934 8433 0 2 4 14 34 934 1934 2934 7934 68 69 ETAAAA JMMAAA HHHHxx +2536 8434 0 0 6 16 36 536 536 2536 2536 72 73 OTAAAA KMMAAA OOOOxx +7088 8435 0 0 8 8 88 88 1088 2088 7088 176 177 QMAAAA LMMAAA VVVVxx +2519 8436 1 3 9 19 19 519 519 2519 2519 38 39 XSAAAA MMMAAA AAAAxx +6650 8437 0 2 0 10 50 650 650 1650 6650 100 101 UVAAAA NMMAAA HHHHxx +3007 8438 1 3 7 7 7 7 1007 3007 3007 14 15 RLAAAA OMMAAA OOOOxx +4507 8439 1 3 7 7 7 507 507 4507 4507 14 15 JRAAAA PMMAAA VVVVxx +4892 8440 0 0 2 12 92 892 892 4892 4892 184 185 EGAAAA QMMAAA AAAAxx +7159 8441 1 3 9 19 59 159 1159 2159 7159 118 119 JPAAAA RMMAAA HHHHxx +3171 8442 1 3 1 11 71 171 1171 3171 3171 142 143 ZRAAAA SMMAAA OOOOxx +1080 8443 0 0 0 0 80 80 1080 1080 1080 160 161 OPAAAA TMMAAA VVVVxx +7248 8444 0 0 8 8 48 248 1248 2248 7248 96 97 USAAAA UMMAAA AAAAxx +7230 8445 0 2 0 10 30 230 1230 2230 7230 60 61 CSAAAA VMMAAA HHHHxx +3823 8446 1 3 3 3 23 823 1823 3823 3823 46 47 BRAAAA WMMAAA OOOOxx +5517 8447 1 1 7 17 17 517 1517 517 5517 34 35 FEAAAA XMMAAA VVVVxx +1482 8448 0 2 2 2 82 482 1482 1482 1482 164 165 AFAAAA YMMAAA AAAAxx +9953 8449 1 1 3 13 53 953 1953 4953 9953 106 107 VSAAAA ZMMAAA HHHHxx +2754 8450 0 2 4 14 54 754 754 2754 2754 108 109 YBAAAA ANMAAA OOOOxx +3875 8451 1 3 5 15 75 875 1875 3875 3875 150 151 BTAAAA BNMAAA VVVVxx +9800 8452 0 0 0 0 0 800 1800 4800 9800 0 1 YMAAAA CNMAAA AAAAxx +8819 8453 1 3 9 19 19 819 819 3819 8819 38 39 FBAAAA DNMAAA HHHHxx +8267 8454 1 3 7 7 67 267 267 3267 8267 134 135 ZFAAAA ENMAAA OOOOxx +520 8455 0 0 0 0 20 520 520 520 520 40 41 AUAAAA FNMAAA VVVVxx +5770 8456 0 2 0 10 70 770 1770 770 5770 140 141 YNAAAA GNMAAA AAAAxx +2114 8457 0 2 4 14 14 114 114 2114 2114 28 29 IDAAAA HNMAAA HHHHxx +5045 8458 1 1 5 5 45 45 1045 45 5045 90 91 BMAAAA INMAAA OOOOxx +1094 8459 0 2 4 14 94 94 1094 1094 1094 188 189 CQAAAA JNMAAA VVVVxx +8786 8460 0 2 6 6 86 786 786 3786 8786 172 173 YZAAAA KNMAAA AAAAxx +353 8461 1 1 3 13 53 353 353 353 353 106 107 PNAAAA LNMAAA HHHHxx +290 8462 0 2 0 10 90 290 290 290 290 180 181 ELAAAA MNMAAA OOOOxx +3376 8463 0 0 6 16 76 376 1376 3376 3376 152 153 WZAAAA NNMAAA VVVVxx +9305 8464 1 1 5 5 5 305 1305 4305 9305 10 11 XTAAAA ONMAAA AAAAxx +186 8465 0 2 6 6 86 186 186 186 186 172 173 EHAAAA PNMAAA HHHHxx +4817 8466 1 1 7 17 17 817 817 4817 4817 34 35 HDAAAA QNMAAA OOOOxx +4638 8467 0 2 8 18 38 638 638 4638 4638 76 77 KWAAAA RNMAAA VVVVxx +3558 8468 0 2 8 18 58 558 1558 3558 3558 116 117 WGAAAA SNMAAA AAAAxx +9285 8469 1 1 5 5 85 285 1285 4285 9285 170 171 DTAAAA TNMAAA HHHHxx +848 8470 0 0 8 8 48 848 848 848 848 96 97 QGAAAA UNMAAA OOOOxx +8923 8471 1 3 3 3 23 923 923 3923 8923 46 47 FFAAAA VNMAAA VVVVxx +6826 8472 0 2 6 6 26 826 826 1826 6826 52 53 OCAAAA WNMAAA AAAAxx +5187 8473 1 3 7 7 87 187 1187 187 5187 174 175 NRAAAA XNMAAA HHHHxx +2398 8474 0 2 8 18 98 398 398 2398 2398 196 197 GOAAAA YNMAAA OOOOxx +7653 8475 1 1 3 13 53 653 1653 2653 7653 106 107 JIAAAA ZNMAAA VVVVxx +8835 8476 1 3 5 15 35 835 835 3835 8835 70 71 VBAAAA AOMAAA AAAAxx +5736 8477 0 0 6 16 36 736 1736 736 5736 72 73 QMAAAA BOMAAA HHHHxx +1238 8478 0 2 8 18 38 238 1238 1238 1238 76 77 QVAAAA COMAAA OOOOxx +6021 8479 1 1 1 1 21 21 21 1021 6021 42 43 PXAAAA DOMAAA VVVVxx +6815 8480 1 3 5 15 15 815 815 1815 6815 30 31 DCAAAA EOMAAA AAAAxx +2549 8481 1 1 9 9 49 549 549 2549 2549 98 99 BUAAAA FOMAAA HHHHxx +5657 8482 1 1 7 17 57 657 1657 657 5657 114 115 PJAAAA GOMAAA OOOOxx +6855 8483 1 3 5 15 55 855 855 1855 6855 110 111 RDAAAA HOMAAA VVVVxx +1225 8484 1 1 5 5 25 225 1225 1225 1225 50 51 DVAAAA IOMAAA AAAAxx +7452 8485 0 0 2 12 52 452 1452 2452 7452 104 105 QAAAAA JOMAAA HHHHxx +2479 8486 1 3 9 19 79 479 479 2479 2479 158 159 JRAAAA KOMAAA OOOOxx +7974 8487 0 2 4 14 74 974 1974 2974 7974 148 149 SUAAAA LOMAAA VVVVxx +1212 8488 0 0 2 12 12 212 1212 1212 1212 24 25 QUAAAA MOMAAA AAAAxx +8883 8489 1 3 3 3 83 883 883 3883 8883 166 167 RDAAAA NOMAAA HHHHxx +8150 8490 0 2 0 10 50 150 150 3150 8150 100 101 MBAAAA OOMAAA OOOOxx +3392 8491 0 0 2 12 92 392 1392 3392 3392 184 185 MAAAAA POMAAA VVVVxx +6774 8492 0 2 4 14 74 774 774 1774 6774 148 149 OAAAAA QOMAAA AAAAxx +904 8493 0 0 4 4 4 904 904 904 904 8 9 UIAAAA ROMAAA HHHHxx +5068 8494 0 0 8 8 68 68 1068 68 5068 136 137 YMAAAA SOMAAA OOOOxx +9339 8495 1 3 9 19 39 339 1339 4339 9339 78 79 FVAAAA TOMAAA VVVVxx +1062 8496 0 2 2 2 62 62 1062 1062 1062 124 125 WOAAAA UOMAAA AAAAxx +3841 8497 1 1 1 1 41 841 1841 3841 3841 82 83 TRAAAA VOMAAA HHHHxx +8924 8498 0 0 4 4 24 924 924 3924 8924 48 49 GFAAAA WOMAAA OOOOxx +9795 8499 1 3 5 15 95 795 1795 4795 9795 190 191 TMAAAA XOMAAA VVVVxx +3981 8500 1 1 1 1 81 981 1981 3981 3981 162 163 DXAAAA YOMAAA AAAAxx +4290 8501 0 2 0 10 90 290 290 4290 4290 180 181 AJAAAA ZOMAAA HHHHxx +1067 8502 1 3 7 7 67 67 1067 1067 1067 134 135 BPAAAA APMAAA OOOOxx +8679 8503 1 3 9 19 79 679 679 3679 8679 158 159 VVAAAA BPMAAA VVVVxx +2894 8504 0 2 4 14 94 894 894 2894 2894 188 189 IHAAAA CPMAAA AAAAxx +9248 8505 0 0 8 8 48 248 1248 4248 9248 96 97 SRAAAA DPMAAA HHHHxx +1072 8506 0 0 2 12 72 72 1072 1072 1072 144 145 GPAAAA EPMAAA OOOOxx +3510 8507 0 2 0 10 10 510 1510 3510 3510 20 21 AFAAAA FPMAAA VVVVxx +6871 8508 1 3 1 11 71 871 871 1871 6871 142 143 HEAAAA GPMAAA AAAAxx +8701 8509 1 1 1 1 1 701 701 3701 8701 2 3 RWAAAA HPMAAA HHHHxx +8170 8510 0 2 0 10 70 170 170 3170 8170 140 141 GCAAAA IPMAAA OOOOxx +2730 8511 0 2 0 10 30 730 730 2730 2730 60 61 ABAAAA JPMAAA VVVVxx +2668 8512 0 0 8 8 68 668 668 2668 2668 136 137 QYAAAA KPMAAA AAAAxx +8723 8513 1 3 3 3 23 723 723 3723 8723 46 47 NXAAAA LPMAAA HHHHxx +3439 8514 1 3 9 19 39 439 1439 3439 3439 78 79 HCAAAA MPMAAA OOOOxx +6219 8515 1 3 9 19 19 219 219 1219 6219 38 39 FFAAAA NPMAAA VVVVxx +4264 8516 0 0 4 4 64 264 264 4264 4264 128 129 AIAAAA OPMAAA AAAAxx +3929 8517 1 1 9 9 29 929 1929 3929 3929 58 59 DVAAAA PPMAAA HHHHxx +7 8518 1 3 7 7 7 7 7 7 7 14 15 HAAAAA QPMAAA OOOOxx +3737 8519 1 1 7 17 37 737 1737 3737 3737 74 75 TNAAAA RPMAAA VVVVxx +358 8520 0 2 8 18 58 358 358 358 358 116 117 UNAAAA SPMAAA AAAAxx +5128 8521 0 0 8 8 28 128 1128 128 5128 56 57 GPAAAA TPMAAA HHHHxx +7353 8522 1 1 3 13 53 353 1353 2353 7353 106 107 VWAAAA UPMAAA OOOOxx +8758 8523 0 2 8 18 58 758 758 3758 8758 116 117 WYAAAA VPMAAA VVVVxx +7284 8524 0 0 4 4 84 284 1284 2284 7284 168 169 EUAAAA WPMAAA AAAAxx +4037 8525 1 1 7 17 37 37 37 4037 4037 74 75 HZAAAA XPMAAA HHHHxx +435 8526 1 3 5 15 35 435 435 435 435 70 71 TQAAAA YPMAAA OOOOxx +3580 8527 0 0 0 0 80 580 1580 3580 3580 160 161 SHAAAA ZPMAAA VVVVxx +4554 8528 0 2 4 14 54 554 554 4554 4554 108 109 ETAAAA AQMAAA AAAAxx +4337 8529 1 1 7 17 37 337 337 4337 4337 74 75 VKAAAA BQMAAA HHHHxx +512 8530 0 0 2 12 12 512 512 512 512 24 25 STAAAA CQMAAA OOOOxx +2032 8531 0 0 2 12 32 32 32 2032 2032 64 65 EAAAAA DQMAAA VVVVxx +1755 8532 1 3 5 15 55 755 1755 1755 1755 110 111 NPAAAA EQMAAA AAAAxx +9923 8533 1 3 3 3 23 923 1923 4923 9923 46 47 RRAAAA FQMAAA HHHHxx +3747 8534 1 3 7 7 47 747 1747 3747 3747 94 95 DOAAAA GQMAAA OOOOxx +27 8535 1 3 7 7 27 27 27 27 27 54 55 BBAAAA HQMAAA VVVVxx +3075 8536 1 3 5 15 75 75 1075 3075 3075 150 151 HOAAAA IQMAAA AAAAxx +6259 8537 1 3 9 19 59 259 259 1259 6259 118 119 TGAAAA JQMAAA HHHHxx +2940 8538 0 0 0 0 40 940 940 2940 2940 80 81 CJAAAA KQMAAA OOOOxx +5724 8539 0 0 4 4 24 724 1724 724 5724 48 49 EMAAAA LQMAAA VVVVxx +5638 8540 0 2 8 18 38 638 1638 638 5638 76 77 WIAAAA MQMAAA AAAAxx +479 8541 1 3 9 19 79 479 479 479 479 158 159 LSAAAA NQMAAA HHHHxx +4125 8542 1 1 5 5 25 125 125 4125 4125 50 51 RCAAAA OQMAAA OOOOxx +1525 8543 1 1 5 5 25 525 1525 1525 1525 50 51 RGAAAA PQMAAA VVVVxx +7529 8544 1 1 9 9 29 529 1529 2529 7529 58 59 PDAAAA QQMAAA AAAAxx +931 8545 1 3 1 11 31 931 931 931 931 62 63 VJAAAA RQMAAA HHHHxx +5175 8546 1 3 5 15 75 175 1175 175 5175 150 151 BRAAAA SQMAAA OOOOxx +6798 8547 0 2 8 18 98 798 798 1798 6798 196 197 MBAAAA TQMAAA VVVVxx +2111 8548 1 3 1 11 11 111 111 2111 2111 22 23 FDAAAA UQMAAA AAAAxx +6145 8549 1 1 5 5 45 145 145 1145 6145 90 91 JCAAAA VQMAAA HHHHxx +4712 8550 0 0 2 12 12 712 712 4712 4712 24 25 GZAAAA WQMAAA OOOOxx +3110 8551 0 2 0 10 10 110 1110 3110 3110 20 21 QPAAAA XQMAAA VVVVxx +97 8552 1 1 7 17 97 97 97 97 97 194 195 TDAAAA YQMAAA AAAAxx +758 8553 0 2 8 18 58 758 758 758 758 116 117 EDAAAA ZQMAAA HHHHxx +1895 8554 1 3 5 15 95 895 1895 1895 1895 190 191 XUAAAA ARMAAA OOOOxx +5289 8555 1 1 9 9 89 289 1289 289 5289 178 179 LVAAAA BRMAAA VVVVxx +5026 8556 0 2 6 6 26 26 1026 26 5026 52 53 ILAAAA CRMAAA AAAAxx +4725 8557 1 1 5 5 25 725 725 4725 4725 50 51 TZAAAA DRMAAA HHHHxx +1679 8558 1 3 9 19 79 679 1679 1679 1679 158 159 PMAAAA ERMAAA OOOOxx +4433 8559 1 1 3 13 33 433 433 4433 4433 66 67 NOAAAA FRMAAA VVVVxx +5340 8560 0 0 0 0 40 340 1340 340 5340 80 81 KXAAAA GRMAAA AAAAxx +6340 8561 0 0 0 0 40 340 340 1340 6340 80 81 WJAAAA HRMAAA HHHHxx +3261 8562 1 1 1 1 61 261 1261 3261 3261 122 123 LVAAAA IRMAAA OOOOxx +8108 8563 0 0 8 8 8 108 108 3108 8108 16 17 WZAAAA JRMAAA VVVVxx +8785 8564 1 1 5 5 85 785 785 3785 8785 170 171 XZAAAA KRMAAA AAAAxx +7391 8565 1 3 1 11 91 391 1391 2391 7391 182 183 HYAAAA LRMAAA HHHHxx +1496 8566 0 0 6 16 96 496 1496 1496 1496 192 193 OFAAAA MRMAAA OOOOxx +1484 8567 0 0 4 4 84 484 1484 1484 1484 168 169 CFAAAA NRMAAA VVVVxx +5884 8568 0 0 4 4 84 884 1884 884 5884 168 169 ISAAAA ORMAAA AAAAxx +342 8569 0 2 2 2 42 342 342 342 342 84 85 ENAAAA PRMAAA HHHHxx +7659 8570 1 3 9 19 59 659 1659 2659 7659 118 119 PIAAAA QRMAAA OOOOxx +6635 8571 1 3 5 15 35 635 635 1635 6635 70 71 FVAAAA RRMAAA VVVVxx +8507 8572 1 3 7 7 7 507 507 3507 8507 14 15 FPAAAA SRMAAA AAAAxx +2583 8573 1 3 3 3 83 583 583 2583 2583 166 167 JVAAAA TRMAAA HHHHxx +6533 8574 1 1 3 13 33 533 533 1533 6533 66 67 HRAAAA URMAAA OOOOxx +5879 8575 1 3 9 19 79 879 1879 879 5879 158 159 DSAAAA VRMAAA VVVVxx +5511 8576 1 3 1 11 11 511 1511 511 5511 22 23 ZDAAAA WRMAAA AAAAxx +3682 8577 0 2 2 2 82 682 1682 3682 3682 164 165 QLAAAA XRMAAA HHHHxx +7182 8578 0 2 2 2 82 182 1182 2182 7182 164 165 GQAAAA YRMAAA OOOOxx +1409 8579 1 1 9 9 9 409 1409 1409 1409 18 19 FCAAAA ZRMAAA VVVVxx +3363 8580 1 3 3 3 63 363 1363 3363 3363 126 127 JZAAAA ASMAAA AAAAxx +729 8581 1 1 9 9 29 729 729 729 729 58 59 BCAAAA BSMAAA HHHHxx +5857 8582 1 1 7 17 57 857 1857 857 5857 114 115 HRAAAA CSMAAA OOOOxx +235 8583 1 3 5 15 35 235 235 235 235 70 71 BJAAAA DSMAAA VVVVxx +193 8584 1 1 3 13 93 193 193 193 193 186 187 LHAAAA ESMAAA AAAAxx +5586 8585 0 2 6 6 86 586 1586 586 5586 172 173 WGAAAA FSMAAA HHHHxx +6203 8586 1 3 3 3 3 203 203 1203 6203 6 7 PEAAAA GSMAAA OOOOxx +6795 8587 1 3 5 15 95 795 795 1795 6795 190 191 JBAAAA HSMAAA VVVVxx +3211 8588 1 3 1 11 11 211 1211 3211 3211 22 23 NTAAAA ISMAAA AAAAxx +9763 8589 1 3 3 3 63 763 1763 4763 9763 126 127 NLAAAA JSMAAA HHHHxx +9043 8590 1 3 3 3 43 43 1043 4043 9043 86 87 VJAAAA KSMAAA OOOOxx +2854 8591 0 2 4 14 54 854 854 2854 2854 108 109 UFAAAA LSMAAA VVVVxx +565 8592 1 1 5 5 65 565 565 565 565 130 131 TVAAAA MSMAAA AAAAxx +9284 8593 0 0 4 4 84 284 1284 4284 9284 168 169 CTAAAA NSMAAA HHHHxx +7886 8594 0 2 6 6 86 886 1886 2886 7886 172 173 IRAAAA OSMAAA OOOOxx +122 8595 0 2 2 2 22 122 122 122 122 44 45 SEAAAA PSMAAA VVVVxx +4934 8596 0 2 4 14 34 934 934 4934 4934 68 69 UHAAAA QSMAAA AAAAxx +1766 8597 0 2 6 6 66 766 1766 1766 1766 132 133 YPAAAA RSMAAA HHHHxx +2554 8598 0 2 4 14 54 554 554 2554 2554 108 109 GUAAAA SSMAAA OOOOxx +488 8599 0 0 8 8 88 488 488 488 488 176 177 USAAAA TSMAAA VVVVxx +825 8600 1 1 5 5 25 825 825 825 825 50 51 TFAAAA USMAAA AAAAxx +678 8601 0 2 8 18 78 678 678 678 678 156 157 CAAAAA VSMAAA HHHHxx +4543 8602 1 3 3 3 43 543 543 4543 4543 86 87 TSAAAA WSMAAA OOOOxx +1699 8603 1 3 9 19 99 699 1699 1699 1699 198 199 JNAAAA XSMAAA VVVVxx +3771 8604 1 3 1 11 71 771 1771 3771 3771 142 143 BPAAAA YSMAAA AAAAxx +1234 8605 0 2 4 14 34 234 1234 1234 1234 68 69 MVAAAA ZSMAAA HHHHxx +4152 8606 0 0 2 12 52 152 152 4152 4152 104 105 SDAAAA ATMAAA OOOOxx +1632 8607 0 0 2 12 32 632 1632 1632 1632 64 65 UKAAAA BTMAAA VVVVxx +4988 8608 0 0 8 8 88 988 988 4988 4988 176 177 WJAAAA CTMAAA AAAAxx +1980 8609 0 0 0 0 80 980 1980 1980 1980 160 161 EYAAAA DTMAAA HHHHxx +7479 8610 1 3 9 19 79 479 1479 2479 7479 158 159 RBAAAA ETMAAA OOOOxx +2586 8611 0 2 6 6 86 586 586 2586 2586 172 173 MVAAAA FTMAAA VVVVxx +5433 8612 1 1 3 13 33 433 1433 433 5433 66 67 ZAAAAA GTMAAA AAAAxx +2261 8613 1 1 1 1 61 261 261 2261 2261 122 123 ZIAAAA HTMAAA HHHHxx +1180 8614 0 0 0 0 80 180 1180 1180 1180 160 161 KTAAAA ITMAAA OOOOxx +3938 8615 0 2 8 18 38 938 1938 3938 3938 76 77 MVAAAA JTMAAA VVVVxx +6714 8616 0 2 4 14 14 714 714 1714 6714 28 29 GYAAAA KTMAAA AAAAxx +2890 8617 0 2 0 10 90 890 890 2890 2890 180 181 EHAAAA LTMAAA HHHHxx +7379 8618 1 3 9 19 79 379 1379 2379 7379 158 159 VXAAAA MTMAAA OOOOxx +5896 8619 0 0 6 16 96 896 1896 896 5896 192 193 USAAAA NTMAAA VVVVxx +5949 8620 1 1 9 9 49 949 1949 949 5949 98 99 VUAAAA OTMAAA AAAAxx +3194 8621 0 2 4 14 94 194 1194 3194 3194 188 189 WSAAAA PTMAAA HHHHxx +9325 8622 1 1 5 5 25 325 1325 4325 9325 50 51 RUAAAA QTMAAA OOOOxx +9531 8623 1 3 1 11 31 531 1531 4531 9531 62 63 PCAAAA RTMAAA VVVVxx +711 8624 1 3 1 11 11 711 711 711 711 22 23 JBAAAA STMAAA AAAAxx +2450 8625 0 2 0 10 50 450 450 2450 2450 100 101 GQAAAA TTMAAA HHHHxx +1929 8626 1 1 9 9 29 929 1929 1929 1929 58 59 FWAAAA UTMAAA OOOOxx +6165 8627 1 1 5 5 65 165 165 1165 6165 130 131 DDAAAA VTMAAA VVVVxx +4050 8628 0 2 0 10 50 50 50 4050 4050 100 101 UZAAAA WTMAAA AAAAxx +9011 8629 1 3 1 11 11 11 1011 4011 9011 22 23 PIAAAA XTMAAA HHHHxx +7916 8630 0 0 6 16 16 916 1916 2916 7916 32 33 MSAAAA YTMAAA OOOOxx +9136 8631 0 0 6 16 36 136 1136 4136 9136 72 73 KNAAAA ZTMAAA VVVVxx +8782 8632 0 2 2 2 82 782 782 3782 8782 164 165 UZAAAA AUMAAA AAAAxx +8491 8633 1 3 1 11 91 491 491 3491 8491 182 183 POAAAA BUMAAA HHHHxx +5114 8634 0 2 4 14 14 114 1114 114 5114 28 29 SOAAAA CUMAAA OOOOxx +5815 8635 1 3 5 15 15 815 1815 815 5815 30 31 RPAAAA DUMAAA VVVVxx +5628 8636 0 0 8 8 28 628 1628 628 5628 56 57 MIAAAA EUMAAA AAAAxx +810 8637 0 2 0 10 10 810 810 810 810 20 21 EFAAAA FUMAAA HHHHxx +6178 8638 0 2 8 18 78 178 178 1178 6178 156 157 QDAAAA GUMAAA OOOOxx +2619 8639 1 3 9 19 19 619 619 2619 2619 38 39 TWAAAA HUMAAA VVVVxx +3340 8640 0 0 0 0 40 340 1340 3340 3340 80 81 MYAAAA IUMAAA AAAAxx +2491 8641 1 3 1 11 91 491 491 2491 2491 182 183 VRAAAA JUMAAA HHHHxx +3574 8642 0 2 4 14 74 574 1574 3574 3574 148 149 MHAAAA KUMAAA OOOOxx +6754 8643 0 2 4 14 54 754 754 1754 6754 108 109 UZAAAA LUMAAA VVVVxx +1566 8644 0 2 6 6 66 566 1566 1566 1566 132 133 GIAAAA MUMAAA AAAAxx +9174 8645 0 2 4 14 74 174 1174 4174 9174 148 149 WOAAAA NUMAAA HHHHxx +1520 8646 0 0 0 0 20 520 1520 1520 1520 40 41 MGAAAA OUMAAA OOOOxx +2691 8647 1 3 1 11 91 691 691 2691 2691 182 183 NZAAAA PUMAAA VVVVxx +6961 8648 1 1 1 1 61 961 961 1961 6961 122 123 THAAAA QUMAAA AAAAxx +5722 8649 0 2 2 2 22 722 1722 722 5722 44 45 CMAAAA RUMAAA HHHHxx +9707 8650 1 3 7 7 7 707 1707 4707 9707 14 15 JJAAAA SUMAAA OOOOxx +2891 8651 1 3 1 11 91 891 891 2891 2891 182 183 FHAAAA TUMAAA VVVVxx +341 8652 1 1 1 1 41 341 341 341 341 82 83 DNAAAA UUMAAA AAAAxx +4690 8653 0 2 0 10 90 690 690 4690 4690 180 181 KYAAAA VUMAAA HHHHxx +7841 8654 1 1 1 1 41 841 1841 2841 7841 82 83 PPAAAA WUMAAA OOOOxx +6615 8655 1 3 5 15 15 615 615 1615 6615 30 31 LUAAAA XUMAAA VVVVxx +9169 8656 1 1 9 9 69 169 1169 4169 9169 138 139 ROAAAA YUMAAA AAAAxx +6689 8657 1 1 9 9 89 689 689 1689 6689 178 179 HXAAAA ZUMAAA HHHHxx +8721 8658 1 1 1 1 21 721 721 3721 8721 42 43 LXAAAA AVMAAA OOOOxx +7508 8659 0 0 8 8 8 508 1508 2508 7508 16 17 UCAAAA BVMAAA VVVVxx +8631 8660 1 3 1 11 31 631 631 3631 8631 62 63 ZTAAAA CVMAAA AAAAxx +480 8661 0 0 0 0 80 480 480 480 480 160 161 MSAAAA DVMAAA HHHHxx +7094 8662 0 2 4 14 94 94 1094 2094 7094 188 189 WMAAAA EVMAAA OOOOxx +319 8663 1 3 9 19 19 319 319 319 319 38 39 HMAAAA FVMAAA VVVVxx +9421 8664 1 1 1 1 21 421 1421 4421 9421 42 43 JYAAAA GVMAAA AAAAxx +4352 8665 0 0 2 12 52 352 352 4352 4352 104 105 KLAAAA HVMAAA HHHHxx +5019 8666 1 3 9 19 19 19 1019 19 5019 38 39 BLAAAA IVMAAA OOOOxx +3956 8667 0 0 6 16 56 956 1956 3956 3956 112 113 EWAAAA JVMAAA VVVVxx +114 8668 0 2 4 14 14 114 114 114 114 28 29 KEAAAA KVMAAA AAAAxx +1196 8669 0 0 6 16 96 196 1196 1196 1196 192 193 AUAAAA LVMAAA HHHHxx +1407 8670 1 3 7 7 7 407 1407 1407 1407 14 15 DCAAAA MVMAAA OOOOxx +7432 8671 0 0 2 12 32 432 1432 2432 7432 64 65 WZAAAA NVMAAA VVVVxx +3141 8672 1 1 1 1 41 141 1141 3141 3141 82 83 VQAAAA OVMAAA AAAAxx +2073 8673 1 1 3 13 73 73 73 2073 2073 146 147 TBAAAA PVMAAA HHHHxx +3400 8674 0 0 0 0 0 400 1400 3400 3400 0 1 UAAAAA QVMAAA OOOOxx +505 8675 1 1 5 5 5 505 505 505 505 10 11 LTAAAA RVMAAA VVVVxx +1263 8676 1 3 3 3 63 263 1263 1263 1263 126 127 PWAAAA SVMAAA AAAAxx +190 8677 0 2 0 10 90 190 190 190 190 180 181 IHAAAA TVMAAA HHHHxx +6686 8678 0 2 6 6 86 686 686 1686 6686 172 173 EXAAAA UVMAAA OOOOxx +9821 8679 1 1 1 1 21 821 1821 4821 9821 42 43 TNAAAA VVMAAA VVVVxx +1119 8680 1 3 9 19 19 119 1119 1119 1119 38 39 BRAAAA WVMAAA AAAAxx +2955 8681 1 3 5 15 55 955 955 2955 2955 110 111 RJAAAA XVMAAA HHHHxx +224 8682 0 0 4 4 24 224 224 224 224 48 49 QIAAAA YVMAAA OOOOxx +7562 8683 0 2 2 2 62 562 1562 2562 7562 124 125 WEAAAA ZVMAAA VVVVxx +8845 8684 1 1 5 5 45 845 845 3845 8845 90 91 FCAAAA AWMAAA AAAAxx +5405 8685 1 1 5 5 5 405 1405 405 5405 10 11 XZAAAA BWMAAA HHHHxx +9192 8686 0 0 2 12 92 192 1192 4192 9192 184 185 OPAAAA CWMAAA OOOOxx +4927 8687 1 3 7 7 27 927 927 4927 4927 54 55 NHAAAA DWMAAA VVVVxx +997 8688 1 1 7 17 97 997 997 997 997 194 195 JMAAAA EWMAAA AAAAxx +989 8689 1 1 9 9 89 989 989 989 989 178 179 BMAAAA FWMAAA HHHHxx +7258 8690 0 2 8 18 58 258 1258 2258 7258 116 117 ETAAAA GWMAAA OOOOxx +6899 8691 1 3 9 19 99 899 899 1899 6899 198 199 JFAAAA HWMAAA VVVVxx +1770 8692 0 2 0 10 70 770 1770 1770 1770 140 141 CQAAAA IWMAAA AAAAxx +4423 8693 1 3 3 3 23 423 423 4423 4423 46 47 DOAAAA JWMAAA HHHHxx +5671 8694 1 3 1 11 71 671 1671 671 5671 142 143 DKAAAA KWMAAA OOOOxx +8393 8695 1 1 3 13 93 393 393 3393 8393 186 187 VKAAAA LWMAAA VVVVxx +4355 8696 1 3 5 15 55 355 355 4355 4355 110 111 NLAAAA MWMAAA AAAAxx +3919 8697 1 3 9 19 19 919 1919 3919 3919 38 39 TUAAAA NWMAAA HHHHxx +338 8698 0 2 8 18 38 338 338 338 338 76 77 ANAAAA OWMAAA OOOOxx +5790 8699 0 2 0 10 90 790 1790 790 5790 180 181 SOAAAA PWMAAA VVVVxx +1452 8700 0 0 2 12 52 452 1452 1452 1452 104 105 WDAAAA QWMAAA AAAAxx +939 8701 1 3 9 19 39 939 939 939 939 78 79 DKAAAA RWMAAA HHHHxx +8913 8702 1 1 3 13 13 913 913 3913 8913 26 27 VEAAAA SWMAAA OOOOxx +7157 8703 1 1 7 17 57 157 1157 2157 7157 114 115 HPAAAA TWMAAA VVVVxx +7240 8704 0 0 0 0 40 240 1240 2240 7240 80 81 MSAAAA UWMAAA AAAAxx +3492 8705 0 0 2 12 92 492 1492 3492 3492 184 185 IEAAAA VWMAAA HHHHxx +3464 8706 0 0 4 4 64 464 1464 3464 3464 128 129 GDAAAA WWMAAA OOOOxx +388 8707 0 0 8 8 88 388 388 388 388 176 177 YOAAAA XWMAAA VVVVxx +4135 8708 1 3 5 15 35 135 135 4135 4135 70 71 BDAAAA YWMAAA AAAAxx +1194 8709 0 2 4 14 94 194 1194 1194 1194 188 189 YTAAAA ZWMAAA HHHHxx +5476 8710 0 0 6 16 76 476 1476 476 5476 152 153 QCAAAA AXMAAA OOOOxx +9844 8711 0 0 4 4 44 844 1844 4844 9844 88 89 QOAAAA BXMAAA VVVVxx +9364 8712 0 0 4 4 64 364 1364 4364 9364 128 129 EWAAAA CXMAAA AAAAxx +5238 8713 0 2 8 18 38 238 1238 238 5238 76 77 MTAAAA DXMAAA HHHHxx +3712 8714 0 0 2 12 12 712 1712 3712 3712 24 25 UMAAAA EXMAAA OOOOxx +6189 8715 1 1 9 9 89 189 189 1189 6189 178 179 BEAAAA FXMAAA VVVVxx +5257 8716 1 1 7 17 57 257 1257 257 5257 114 115 FUAAAA GXMAAA AAAAxx +81 8717 1 1 1 1 81 81 81 81 81 162 163 DDAAAA HXMAAA HHHHxx +3289 8718 1 1 9 9 89 289 1289 3289 3289 178 179 NWAAAA IXMAAA OOOOxx +1177 8719 1 1 7 17 77 177 1177 1177 1177 154 155 HTAAAA JXMAAA VVVVxx +5038 8720 0 2 8 18 38 38 1038 38 5038 76 77 ULAAAA KXMAAA AAAAxx +325 8721 1 1 5 5 25 325 325 325 325 50 51 NMAAAA LXMAAA HHHHxx +7221 8722 1 1 1 1 21 221 1221 2221 7221 42 43 TRAAAA MXMAAA OOOOxx +7123 8723 1 3 3 3 23 123 1123 2123 7123 46 47 ZNAAAA NXMAAA VVVVxx +6364 8724 0 0 4 4 64 364 364 1364 6364 128 129 UKAAAA OXMAAA AAAAxx +4468 8725 0 0 8 8 68 468 468 4468 4468 136 137 WPAAAA PXMAAA HHHHxx +9185 8726 1 1 5 5 85 185 1185 4185 9185 170 171 HPAAAA QXMAAA OOOOxx +4158 8727 0 2 8 18 58 158 158 4158 4158 116 117 YDAAAA RXMAAA VVVVxx +9439 8728 1 3 9 19 39 439 1439 4439 9439 78 79 BZAAAA SXMAAA AAAAxx +7759 8729 1 3 9 19 59 759 1759 2759 7759 118 119 LMAAAA TXMAAA HHHHxx +3325 8730 1 1 5 5 25 325 1325 3325 3325 50 51 XXAAAA UXMAAA OOOOxx +7991 8731 1 3 1 11 91 991 1991 2991 7991 182 183 JVAAAA VXMAAA VVVVxx +1650 8732 0 2 0 10 50 650 1650 1650 1650 100 101 MLAAAA WXMAAA AAAAxx +8395 8733 1 3 5 15 95 395 395 3395 8395 190 191 XKAAAA XXMAAA HHHHxx +286 8734 0 2 6 6 86 286 286 286 286 172 173 ALAAAA YXMAAA OOOOxx +1507 8735 1 3 7 7 7 507 1507 1507 1507 14 15 ZFAAAA ZXMAAA VVVVxx +4122 8736 0 2 2 2 22 122 122 4122 4122 44 45 OCAAAA AYMAAA AAAAxx +2625 8737 1 1 5 5 25 625 625 2625 2625 50 51 ZWAAAA BYMAAA HHHHxx +1140 8738 0 0 0 0 40 140 1140 1140 1140 80 81 WRAAAA CYMAAA OOOOxx +5262 8739 0 2 2 2 62 262 1262 262 5262 124 125 KUAAAA DYMAAA VVVVxx +4919 8740 1 3 9 19 19 919 919 4919 4919 38 39 FHAAAA EYMAAA AAAAxx +7266 8741 0 2 6 6 66 266 1266 2266 7266 132 133 MTAAAA FYMAAA HHHHxx +630 8742 0 2 0 10 30 630 630 630 630 60 61 GYAAAA GYMAAA OOOOxx +2129 8743 1 1 9 9 29 129 129 2129 2129 58 59 XDAAAA HYMAAA VVVVxx +9552 8744 0 0 2 12 52 552 1552 4552 9552 104 105 KDAAAA IYMAAA AAAAxx +3018 8745 0 2 8 18 18 18 1018 3018 3018 36 37 CMAAAA JYMAAA HHHHxx +7145 8746 1 1 5 5 45 145 1145 2145 7145 90 91 VOAAAA KYMAAA OOOOxx +1633 8747 1 1 3 13 33 633 1633 1633 1633 66 67 VKAAAA LYMAAA VVVVxx +7957 8748 1 1 7 17 57 957 1957 2957 7957 114 115 BUAAAA MYMAAA AAAAxx +774 8749 0 2 4 14 74 774 774 774 774 148 149 UDAAAA NYMAAA HHHHxx +9371 8750 1 3 1 11 71 371 1371 4371 9371 142 143 LWAAAA OYMAAA OOOOxx +6007 8751 1 3 7 7 7 7 7 1007 6007 14 15 BXAAAA PYMAAA VVVVxx +5277 8752 1 1 7 17 77 277 1277 277 5277 154 155 ZUAAAA QYMAAA AAAAxx +9426 8753 0 2 6 6 26 426 1426 4426 9426 52 53 OYAAAA RYMAAA HHHHxx +9190 8754 0 2 0 10 90 190 1190 4190 9190 180 181 MPAAAA SYMAAA OOOOxx +8996 8755 0 0 6 16 96 996 996 3996 8996 192 193 AIAAAA TYMAAA VVVVxx +3409 8756 1 1 9 9 9 409 1409 3409 3409 18 19 DBAAAA UYMAAA AAAAxx +7212 8757 0 0 2 12 12 212 1212 2212 7212 24 25 KRAAAA VYMAAA HHHHxx +416 8758 0 0 6 16 16 416 416 416 416 32 33 AQAAAA WYMAAA OOOOxx +7211 8759 1 3 1 11 11 211 1211 2211 7211 22 23 JRAAAA XYMAAA VVVVxx +7454 8760 0 2 4 14 54 454 1454 2454 7454 108 109 SAAAAA YYMAAA AAAAxx +8417 8761 1 1 7 17 17 417 417 3417 8417 34 35 TLAAAA ZYMAAA HHHHxx +5562 8762 0 2 2 2 62 562 1562 562 5562 124 125 YFAAAA AZMAAA OOOOxx +4996 8763 0 0 6 16 96 996 996 4996 4996 192 193 EKAAAA BZMAAA VVVVxx +5718 8764 0 2 8 18 18 718 1718 718 5718 36 37 YLAAAA CZMAAA AAAAxx +7838 8765 0 2 8 18 38 838 1838 2838 7838 76 77 MPAAAA DZMAAA HHHHxx +7715 8766 1 3 5 15 15 715 1715 2715 7715 30 31 TKAAAA EZMAAA OOOOxx +2780 8767 0 0 0 0 80 780 780 2780 2780 160 161 YCAAAA FZMAAA VVVVxx +1013 8768 1 1 3 13 13 13 1013 1013 1013 26 27 ZMAAAA GZMAAA AAAAxx +8465 8769 1 1 5 5 65 465 465 3465 8465 130 131 PNAAAA HZMAAA HHHHxx +7976 8770 0 0 6 16 76 976 1976 2976 7976 152 153 UUAAAA IZMAAA OOOOxx +7150 8771 0 2 0 10 50 150 1150 2150 7150 100 101 APAAAA JZMAAA VVVVxx +6471 8772 1 3 1 11 71 471 471 1471 6471 142 143 XOAAAA KZMAAA AAAAxx +1927 8773 1 3 7 7 27 927 1927 1927 1927 54 55 DWAAAA LZMAAA HHHHxx +227 8774 1 3 7 7 27 227 227 227 227 54 55 TIAAAA MZMAAA OOOOxx +6462 8775 0 2 2 2 62 462 462 1462 6462 124 125 OOAAAA NZMAAA VVVVxx +5227 8776 1 3 7 7 27 227 1227 227 5227 54 55 BTAAAA OZMAAA AAAAxx +1074 8777 0 2 4 14 74 74 1074 1074 1074 148 149 IPAAAA PZMAAA HHHHxx +9448 8778 0 0 8 8 48 448 1448 4448 9448 96 97 KZAAAA QZMAAA OOOOxx +4459 8779 1 3 9 19 59 459 459 4459 4459 118 119 NPAAAA RZMAAA VVVVxx +2478 8780 0 2 8 18 78 478 478 2478 2478 156 157 IRAAAA SZMAAA AAAAxx +5005 8781 1 1 5 5 5 5 1005 5 5005 10 11 NKAAAA TZMAAA HHHHxx +2418 8782 0 2 8 18 18 418 418 2418 2418 36 37 APAAAA UZMAAA OOOOxx +6991 8783 1 3 1 11 91 991 991 1991 6991 182 183 XIAAAA VZMAAA VVVVxx +4729 8784 1 1 9 9 29 729 729 4729 4729 58 59 XZAAAA WZMAAA AAAAxx +3548 8785 0 0 8 8 48 548 1548 3548 3548 96 97 MGAAAA XZMAAA HHHHxx +9616 8786 0 0 6 16 16 616 1616 4616 9616 32 33 WFAAAA YZMAAA OOOOxx +2901 8787 1 1 1 1 1 901 901 2901 2901 2 3 PHAAAA ZZMAAA VVVVxx +10 8788 0 2 0 10 10 10 10 10 10 20 21 KAAAAA AANAAA AAAAxx +2637 8789 1 1 7 17 37 637 637 2637 2637 74 75 LXAAAA BANAAA HHHHxx +6747 8790 1 3 7 7 47 747 747 1747 6747 94 95 NZAAAA CANAAA OOOOxx +797 8791 1 1 7 17 97 797 797 797 797 194 195 REAAAA DANAAA VVVVxx +7609 8792 1 1 9 9 9 609 1609 2609 7609 18 19 RGAAAA EANAAA AAAAxx +8290 8793 0 2 0 10 90 290 290 3290 8290 180 181 WGAAAA FANAAA HHHHxx +8765 8794 1 1 5 5 65 765 765 3765 8765 130 131 DZAAAA GANAAA OOOOxx +8053 8795 1 1 3 13 53 53 53 3053 8053 106 107 TXAAAA HANAAA VVVVxx +5602 8796 0 2 2 2 2 602 1602 602 5602 4 5 MHAAAA IANAAA AAAAxx +3672 8797 0 0 2 12 72 672 1672 3672 3672 144 145 GLAAAA JANAAA HHHHxx +7513 8798 1 1 3 13 13 513 1513 2513 7513 26 27 ZCAAAA KANAAA OOOOxx +3462 8799 0 2 2 2 62 462 1462 3462 3462 124 125 EDAAAA LANAAA VVVVxx +4457 8800 1 1 7 17 57 457 457 4457 4457 114 115 LPAAAA MANAAA AAAAxx +6547 8801 1 3 7 7 47 547 547 1547 6547 94 95 VRAAAA NANAAA HHHHxx +7417 8802 1 1 7 17 17 417 1417 2417 7417 34 35 HZAAAA OANAAA OOOOxx +8641 8803 1 1 1 1 41 641 641 3641 8641 82 83 JUAAAA PANAAA VVVVxx +149 8804 1 1 9 9 49 149 149 149 149 98 99 TFAAAA QANAAA AAAAxx +5041 8805 1 1 1 1 41 41 1041 41 5041 82 83 XLAAAA RANAAA HHHHxx +9232 8806 0 0 2 12 32 232 1232 4232 9232 64 65 CRAAAA SANAAA OOOOxx +3603 8807 1 3 3 3 3 603 1603 3603 3603 6 7 PIAAAA TANAAA VVVVxx +2792 8808 0 0 2 12 92 792 792 2792 2792 184 185 KDAAAA UANAAA AAAAxx +6620 8809 0 0 0 0 20 620 620 1620 6620 40 41 QUAAAA VANAAA HHHHxx +4000 8810 0 0 0 0 0 0 0 4000 4000 0 1 WXAAAA WANAAA OOOOxx +659 8811 1 3 9 19 59 659 659 659 659 118 119 JZAAAA XANAAA VVVVxx +8174 8812 0 2 4 14 74 174 174 3174 8174 148 149 KCAAAA YANAAA AAAAxx +4599 8813 1 3 9 19 99 599 599 4599 4599 198 199 XUAAAA ZANAAA HHHHxx +7851 8814 1 3 1 11 51 851 1851 2851 7851 102 103 ZPAAAA ABNAAA OOOOxx +6284 8815 0 0 4 4 84 284 284 1284 6284 168 169 SHAAAA BBNAAA VVVVxx +7116 8816 0 0 6 16 16 116 1116 2116 7116 32 33 SNAAAA CBNAAA AAAAxx +5595 8817 1 3 5 15 95 595 1595 595 5595 190 191 FHAAAA DBNAAA HHHHxx +2903 8818 1 3 3 3 3 903 903 2903 2903 6 7 RHAAAA EBNAAA OOOOxx +5948 8819 0 0 8 8 48 948 1948 948 5948 96 97 UUAAAA FBNAAA VVVVxx +225 8820 1 1 5 5 25 225 225 225 225 50 51 RIAAAA GBNAAA AAAAxx +524 8821 0 0 4 4 24 524 524 524 524 48 49 EUAAAA HBNAAA HHHHxx +7639 8822 1 3 9 19 39 639 1639 2639 7639 78 79 VHAAAA IBNAAA OOOOxx +7297 8823 1 1 7 17 97 297 1297 2297 7297 194 195 RUAAAA JBNAAA VVVVxx +2606 8824 0 2 6 6 6 606 606 2606 2606 12 13 GWAAAA KBNAAA AAAAxx +4771 8825 1 3 1 11 71 771 771 4771 4771 142 143 NBAAAA LBNAAA HHHHxx +8162 8826 0 2 2 2 62 162 162 3162 8162 124 125 YBAAAA MBNAAA OOOOxx +8999 8827 1 3 9 19 99 999 999 3999 8999 198 199 DIAAAA NBNAAA VVVVxx +2309 8828 1 1 9 9 9 309 309 2309 2309 18 19 VKAAAA OBNAAA AAAAxx +3594 8829 0 2 4 14 94 594 1594 3594 3594 188 189 GIAAAA PBNAAA HHHHxx +6092 8830 0 0 2 12 92 92 92 1092 6092 184 185 IAAAAA QBNAAA OOOOxx +7467 8831 1 3 7 7 67 467 1467 2467 7467 134 135 FBAAAA RBNAAA VVVVxx +6986 8832 0 2 6 6 86 986 986 1986 6986 172 173 SIAAAA SBNAAA AAAAxx +9898 8833 0 2 8 18 98 898 1898 4898 9898 196 197 SQAAAA TBNAAA HHHHxx +9578 8834 0 2 8 18 78 578 1578 4578 9578 156 157 KEAAAA UBNAAA OOOOxx +156 8835 0 0 6 16 56 156 156 156 156 112 113 AGAAAA VBNAAA VVVVxx +5810 8836 0 2 0 10 10 810 1810 810 5810 20 21 MPAAAA WBNAAA AAAAxx +790 8837 0 2 0 10 90 790 790 790 790 180 181 KEAAAA XBNAAA HHHHxx +6840 8838 0 0 0 0 40 840 840 1840 6840 80 81 CDAAAA YBNAAA OOOOxx +6725 8839 1 1 5 5 25 725 725 1725 6725 50 51 RYAAAA ZBNAAA VVVVxx +5528 8840 0 0 8 8 28 528 1528 528 5528 56 57 QEAAAA ACNAAA AAAAxx +4120 8841 0 0 0 0 20 120 120 4120 4120 40 41 MCAAAA BCNAAA HHHHxx +6694 8842 0 2 4 14 94 694 694 1694 6694 188 189 MXAAAA CCNAAA OOOOxx +3552 8843 0 0 2 12 52 552 1552 3552 3552 104 105 QGAAAA DCNAAA VVVVxx +1478 8844 0 2 8 18 78 478 1478 1478 1478 156 157 WEAAAA ECNAAA AAAAxx +8084 8845 0 0 4 4 84 84 84 3084 8084 168 169 YYAAAA FCNAAA HHHHxx +7578 8846 0 2 8 18 78 578 1578 2578 7578 156 157 MFAAAA GCNAAA OOOOxx +6314 8847 0 2 4 14 14 314 314 1314 6314 28 29 WIAAAA HCNAAA VVVVxx +6123 8848 1 3 3 3 23 123 123 1123 6123 46 47 NBAAAA ICNAAA AAAAxx +9443 8849 1 3 3 3 43 443 1443 4443 9443 86 87 FZAAAA JCNAAA HHHHxx +9628 8850 0 0 8 8 28 628 1628 4628 9628 56 57 IGAAAA KCNAAA OOOOxx +8508 8851 0 0 8 8 8 508 508 3508 8508 16 17 GPAAAA LCNAAA VVVVxx +5552 8852 0 0 2 12 52 552 1552 552 5552 104 105 OFAAAA MCNAAA AAAAxx +5327 8853 1 3 7 7 27 327 1327 327 5327 54 55 XWAAAA NCNAAA HHHHxx +7771 8854 1 3 1 11 71 771 1771 2771 7771 142 143 XMAAAA OCNAAA OOOOxx +8932 8855 0 0 2 12 32 932 932 3932 8932 64 65 OFAAAA PCNAAA VVVVxx +3526 8856 0 2 6 6 26 526 1526 3526 3526 52 53 QFAAAA QCNAAA AAAAxx +4340 8857 0 0 0 0 40 340 340 4340 4340 80 81 YKAAAA RCNAAA HHHHxx +9419 8858 1 3 9 19 19 419 1419 4419 9419 38 39 HYAAAA SCNAAA OOOOxx +8421 8859 1 1 1 1 21 421 421 3421 8421 42 43 XLAAAA TCNAAA VVVVxx +7431 8860 1 3 1 11 31 431 1431 2431 7431 62 63 VZAAAA UCNAAA AAAAxx +172 8861 0 0 2 12 72 172 172 172 172 144 145 QGAAAA VCNAAA HHHHxx +3279 8862 1 3 9 19 79 279 1279 3279 3279 158 159 DWAAAA WCNAAA OOOOxx +1508 8863 0 0 8 8 8 508 1508 1508 1508 16 17 AGAAAA XCNAAA VVVVxx +7091 8864 1 3 1 11 91 91 1091 2091 7091 182 183 TMAAAA YCNAAA AAAAxx +1419 8865 1 3 9 19 19 419 1419 1419 1419 38 39 PCAAAA ZCNAAA HHHHxx +3032 8866 0 0 2 12 32 32 1032 3032 3032 64 65 QMAAAA ADNAAA OOOOxx +8683 8867 1 3 3 3 83 683 683 3683 8683 166 167 ZVAAAA BDNAAA VVVVxx +4763 8868 1 3 3 3 63 763 763 4763 4763 126 127 FBAAAA CDNAAA AAAAxx +4424 8869 0 0 4 4 24 424 424 4424 4424 48 49 EOAAAA DDNAAA HHHHxx +8640 8870 0 0 0 0 40 640 640 3640 8640 80 81 IUAAAA EDNAAA OOOOxx +7187 8871 1 3 7 7 87 187 1187 2187 7187 174 175 LQAAAA FDNAAA VVVVxx +6247 8872 1 3 7 7 47 247 247 1247 6247 94 95 HGAAAA GDNAAA AAAAxx +7340 8873 0 0 0 0 40 340 1340 2340 7340 80 81 IWAAAA HDNAAA HHHHxx +182 8874 0 2 2 2 82 182 182 182 182 164 165 AHAAAA IDNAAA OOOOxx +2948 8875 0 0 8 8 48 948 948 2948 2948 96 97 KJAAAA JDNAAA VVVVxx +9462 8876 0 2 2 2 62 462 1462 4462 9462 124 125 YZAAAA KDNAAA AAAAxx +5997 8877 1 1 7 17 97 997 1997 997 5997 194 195 RWAAAA LDNAAA HHHHxx +5608 8878 0 0 8 8 8 608 1608 608 5608 16 17 SHAAAA MDNAAA OOOOxx +1472 8879 0 0 2 12 72 472 1472 1472 1472 144 145 QEAAAA NDNAAA VVVVxx +277 8880 1 1 7 17 77 277 277 277 277 154 155 RKAAAA ODNAAA AAAAxx +4807 8881 1 3 7 7 7 807 807 4807 4807 14 15 XCAAAA PDNAAA HHHHxx +4969 8882 1 1 9 9 69 969 969 4969 4969 138 139 DJAAAA QDNAAA OOOOxx +5611 8883 1 3 1 11 11 611 1611 611 5611 22 23 VHAAAA RDNAAA VVVVxx +372 8884 0 0 2 12 72 372 372 372 372 144 145 IOAAAA SDNAAA AAAAxx +6666 8885 0 2 6 6 66 666 666 1666 6666 132 133 KWAAAA TDNAAA HHHHxx +476 8886 0 0 6 16 76 476 476 476 476 152 153 ISAAAA UDNAAA OOOOxx +5225 8887 1 1 5 5 25 225 1225 225 5225 50 51 ZSAAAA VDNAAA VVVVxx +5143 8888 1 3 3 3 43 143 1143 143 5143 86 87 VPAAAA WDNAAA AAAAxx +1853 8889 1 1 3 13 53 853 1853 1853 1853 106 107 HTAAAA XDNAAA HHHHxx +675 8890 1 3 5 15 75 675 675 675 675 150 151 ZZAAAA YDNAAA OOOOxx +5643 8891 1 3 3 3 43 643 1643 643 5643 86 87 BJAAAA ZDNAAA VVVVxx +5317 8892 1 1 7 17 17 317 1317 317 5317 34 35 NWAAAA AENAAA AAAAxx +8102 8893 0 2 2 2 2 102 102 3102 8102 4 5 QZAAAA BENAAA HHHHxx +978 8894 0 2 8 18 78 978 978 978 978 156 157 QLAAAA CENAAA OOOOxx +4620 8895 0 0 0 0 20 620 620 4620 4620 40 41 SVAAAA DENAAA VVVVxx +151 8896 1 3 1 11 51 151 151 151 151 102 103 VFAAAA EENAAA AAAAxx +972 8897 0 0 2 12 72 972 972 972 972 144 145 KLAAAA FENAAA HHHHxx +6820 8898 0 0 0 0 20 820 820 1820 6820 40 41 ICAAAA GENAAA OOOOxx +7387 8899 1 3 7 7 87 387 1387 2387 7387 174 175 DYAAAA HENAAA VVVVxx +9634 8900 0 2 4 14 34 634 1634 4634 9634 68 69 OGAAAA IENAAA AAAAxx +6308 8901 0 0 8 8 8 308 308 1308 6308 16 17 QIAAAA JENAAA HHHHxx +8323 8902 1 3 3 3 23 323 323 3323 8323 46 47 DIAAAA KENAAA OOOOxx +6672 8903 0 0 2 12 72 672 672 1672 6672 144 145 QWAAAA LENAAA VVVVxx +8283 8904 1 3 3 3 83 283 283 3283 8283 166 167 PGAAAA MENAAA AAAAxx +7996 8905 0 0 6 16 96 996 1996 2996 7996 192 193 OVAAAA NENAAA HHHHxx +6488 8906 0 0 8 8 88 488 488 1488 6488 176 177 OPAAAA OENAAA OOOOxx +2365 8907 1 1 5 5 65 365 365 2365 2365 130 131 ZMAAAA PENAAA VVVVxx +9746 8908 0 2 6 6 46 746 1746 4746 9746 92 93 WKAAAA QENAAA AAAAxx +8605 8909 1 1 5 5 5 605 605 3605 8605 10 11 ZSAAAA RENAAA HHHHxx +3342 8910 0 2 2 2 42 342 1342 3342 3342 84 85 OYAAAA SENAAA OOOOxx +8429 8911 1 1 9 9 29 429 429 3429 8429 58 59 FMAAAA TENAAA VVVVxx +1162 8912 0 2 2 2 62 162 1162 1162 1162 124 125 SSAAAA UENAAA AAAAxx +531 8913 1 3 1 11 31 531 531 531 531 62 63 LUAAAA VENAAA HHHHxx +8408 8914 0 0 8 8 8 408 408 3408 8408 16 17 KLAAAA WENAAA OOOOxx +8862 8915 0 2 2 2 62 862 862 3862 8862 124 125 WCAAAA XENAAA VVVVxx +5843 8916 1 3 3 3 43 843 1843 843 5843 86 87 TQAAAA YENAAA AAAAxx +8704 8917 0 0 4 4 4 704 704 3704 8704 8 9 UWAAAA ZENAAA HHHHxx +7070 8918 0 2 0 10 70 70 1070 2070 7070 140 141 YLAAAA AFNAAA OOOOxx +9119 8919 1 3 9 19 19 119 1119 4119 9119 38 39 TMAAAA BFNAAA VVVVxx +8344 8920 0 0 4 4 44 344 344 3344 8344 88 89 YIAAAA CFNAAA AAAAxx +8979 8921 1 3 9 19 79 979 979 3979 8979 158 159 JHAAAA DFNAAA HHHHxx +2971 8922 1 3 1 11 71 971 971 2971 2971 142 143 HKAAAA EFNAAA OOOOxx +7700 8923 0 0 0 0 0 700 1700 2700 7700 0 1 EKAAAA FFNAAA VVVVxx +8280 8924 0 0 0 0 80 280 280 3280 8280 160 161 MGAAAA GFNAAA AAAAxx +9096 8925 0 0 6 16 96 96 1096 4096 9096 192 193 WLAAAA HFNAAA HHHHxx +99 8926 1 3 9 19 99 99 99 99 99 198 199 VDAAAA IFNAAA OOOOxx +6696 8927 0 0 6 16 96 696 696 1696 6696 192 193 OXAAAA JFNAAA VVVVxx +9490 8928 0 2 0 10 90 490 1490 4490 9490 180 181 ABAAAA KFNAAA AAAAxx +9073 8929 1 1 3 13 73 73 1073 4073 9073 146 147 ZKAAAA LFNAAA HHHHxx +1861 8930 1 1 1 1 61 861 1861 1861 1861 122 123 PTAAAA MFNAAA OOOOxx +4413 8931 1 1 3 13 13 413 413 4413 4413 26 27 TNAAAA NFNAAA VVVVxx +6002 8932 0 2 2 2 2 2 2 1002 6002 4 5 WWAAAA OFNAAA AAAAxx +439 8933 1 3 9 19 39 439 439 439 439 78 79 XQAAAA PFNAAA HHHHxx +5449 8934 1 1 9 9 49 449 1449 449 5449 98 99 PBAAAA QFNAAA OOOOxx +9737 8935 1 1 7 17 37 737 1737 4737 9737 74 75 NKAAAA RFNAAA VVVVxx +1898 8936 0 2 8 18 98 898 1898 1898 1898 196 197 AVAAAA SFNAAA AAAAxx +4189 8937 1 1 9 9 89 189 189 4189 4189 178 179 DFAAAA TFNAAA HHHHxx +1408 8938 0 0 8 8 8 408 1408 1408 1408 16 17 ECAAAA UFNAAA OOOOxx +394 8939 0 2 4 14 94 394 394 394 394 188 189 EPAAAA VFNAAA VVVVxx +1935 8940 1 3 5 15 35 935 1935 1935 1935 70 71 LWAAAA WFNAAA AAAAxx +3965 8941 1 1 5 5 65 965 1965 3965 3965 130 131 NWAAAA XFNAAA HHHHxx +6821 8942 1 1 1 1 21 821 821 1821 6821 42 43 JCAAAA YFNAAA OOOOxx +349 8943 1 1 9 9 49 349 349 349 349 98 99 LNAAAA ZFNAAA VVVVxx +8428 8944 0 0 8 8 28 428 428 3428 8428 56 57 EMAAAA AGNAAA AAAAxx +8200 8945 0 0 0 0 0 200 200 3200 8200 0 1 KDAAAA BGNAAA HHHHxx +1737 8946 1 1 7 17 37 737 1737 1737 1737 74 75 VOAAAA CGNAAA OOOOxx +6516 8947 0 0 6 16 16 516 516 1516 6516 32 33 QQAAAA DGNAAA VVVVxx +5441 8948 1 1 1 1 41 441 1441 441 5441 82 83 HBAAAA EGNAAA AAAAxx +5999 8949 1 3 9 19 99 999 1999 999 5999 198 199 TWAAAA FGNAAA HHHHxx +1539 8950 1 3 9 19 39 539 1539 1539 1539 78 79 FHAAAA GGNAAA OOOOxx +9067 8951 1 3 7 7 67 67 1067 4067 9067 134 135 TKAAAA HGNAAA VVVVxx +4061 8952 1 1 1 1 61 61 61 4061 4061 122 123 FAAAAA IGNAAA AAAAxx +1642 8953 0 2 2 2 42 642 1642 1642 1642 84 85 ELAAAA JGNAAA HHHHxx +4657 8954 1 1 7 17 57 657 657 4657 4657 114 115 DXAAAA KGNAAA OOOOxx +9934 8955 0 2 4 14 34 934 1934 4934 9934 68 69 CSAAAA LGNAAA VVVVxx +6385 8956 1 1 5 5 85 385 385 1385 6385 170 171 PLAAAA MGNAAA AAAAxx +6775 8957 1 3 5 15 75 775 775 1775 6775 150 151 PAAAAA NGNAAA HHHHxx +3873 8958 1 1 3 13 73 873 1873 3873 3873 146 147 ZSAAAA OGNAAA OOOOxx +3862 8959 0 2 2 2 62 862 1862 3862 3862 124 125 OSAAAA PGNAAA VVVVxx +1224 8960 0 0 4 4 24 224 1224 1224 1224 48 49 CVAAAA QGNAAA AAAAxx +4483 8961 1 3 3 3 83 483 483 4483 4483 166 167 LQAAAA RGNAAA HHHHxx +3685 8962 1 1 5 5 85 685 1685 3685 3685 170 171 TLAAAA SGNAAA OOOOxx +6082 8963 0 2 2 2 82 82 82 1082 6082 164 165 YZAAAA TGNAAA VVVVxx +7798 8964 0 2 8 18 98 798 1798 2798 7798 196 197 YNAAAA UGNAAA AAAAxx +9039 8965 1 3 9 19 39 39 1039 4039 9039 78 79 RJAAAA VGNAAA HHHHxx +985 8966 1 1 5 5 85 985 985 985 985 170 171 XLAAAA WGNAAA OOOOxx +5389 8967 1 1 9 9 89 389 1389 389 5389 178 179 HZAAAA XGNAAA VVVVxx +1716 8968 0 0 6 16 16 716 1716 1716 1716 32 33 AOAAAA YGNAAA AAAAxx +4209 8969 1 1 9 9 9 209 209 4209 4209 18 19 XFAAAA ZGNAAA HHHHxx +746 8970 0 2 6 6 46 746 746 746 746 92 93 SCAAAA AHNAAA OOOOxx +6295 8971 1 3 5 15 95 295 295 1295 6295 190 191 DIAAAA BHNAAA VVVVxx +9754 8972 0 2 4 14 54 754 1754 4754 9754 108 109 ELAAAA CHNAAA AAAAxx +2336 8973 0 0 6 16 36 336 336 2336 2336 72 73 WLAAAA DHNAAA HHHHxx +3701 8974 1 1 1 1 1 701 1701 3701 3701 2 3 JMAAAA EHNAAA OOOOxx +3551 8975 1 3 1 11 51 551 1551 3551 3551 102 103 PGAAAA FHNAAA VVVVxx +8516 8976 0 0 6 16 16 516 516 3516 8516 32 33 OPAAAA GHNAAA AAAAxx +9290 8977 0 2 0 10 90 290 1290 4290 9290 180 181 ITAAAA HHNAAA HHHHxx +5686 8978 0 2 6 6 86 686 1686 686 5686 172 173 SKAAAA IHNAAA OOOOxx +2893 8979 1 1 3 13 93 893 893 2893 2893 186 187 HHAAAA JHNAAA VVVVxx +6279 8980 1 3 9 19 79 279 279 1279 6279 158 159 NHAAAA KHNAAA AAAAxx +2278 8981 0 2 8 18 78 278 278 2278 2278 156 157 QJAAAA LHNAAA HHHHxx +1618 8982 0 2 8 18 18 618 1618 1618 1618 36 37 GKAAAA MHNAAA OOOOxx +3450 8983 0 2 0 10 50 450 1450 3450 3450 100 101 SCAAAA NHNAAA VVVVxx +8857 8984 1 1 7 17 57 857 857 3857 8857 114 115 RCAAAA OHNAAA AAAAxx +1005 8985 1 1 5 5 5 5 1005 1005 1005 10 11 RMAAAA PHNAAA HHHHxx +4727 8986 1 3 7 7 27 727 727 4727 4727 54 55 VZAAAA QHNAAA OOOOxx +7617 8987 1 1 7 17 17 617 1617 2617 7617 34 35 ZGAAAA RHNAAA VVVVxx +2021 8988 1 1 1 1 21 21 21 2021 2021 42 43 TZAAAA SHNAAA AAAAxx +9124 8989 0 0 4 4 24 124 1124 4124 9124 48 49 YMAAAA THNAAA HHHHxx +3175 8990 1 3 5 15 75 175 1175 3175 3175 150 151 DSAAAA UHNAAA OOOOxx +2949 8991 1 1 9 9 49 949 949 2949 2949 98 99 LJAAAA VHNAAA VVVVxx +2424 8992 0 0 4 4 24 424 424 2424 2424 48 49 GPAAAA WHNAAA AAAAxx +4791 8993 1 3 1 11 91 791 791 4791 4791 182 183 HCAAAA XHNAAA HHHHxx +7500 8994 0 0 0 0 0 500 1500 2500 7500 0 1 MCAAAA YHNAAA OOOOxx +4893 8995 1 1 3 13 93 893 893 4893 4893 186 187 FGAAAA ZHNAAA VVVVxx +121 8996 1 1 1 1 21 121 121 121 121 42 43 REAAAA AINAAA AAAAxx +1965 8997 1 1 5 5 65 965 1965 1965 1965 130 131 PXAAAA BINAAA HHHHxx +2972 8998 0 0 2 12 72 972 972 2972 2972 144 145 IKAAAA CINAAA OOOOxx +662 8999 0 2 2 2 62 662 662 662 662 124 125 MZAAAA DINAAA VVVVxx +7074 9000 0 2 4 14 74 74 1074 2074 7074 148 149 CMAAAA EINAAA AAAAxx +981 9001 1 1 1 1 81 981 981 981 981 162 163 TLAAAA FINAAA HHHHxx +3520 9002 0 0 0 0 20 520 1520 3520 3520 40 41 KFAAAA GINAAA OOOOxx +6540 9003 0 0 0 0 40 540 540 1540 6540 80 81 ORAAAA HINAAA VVVVxx +6648 9004 0 0 8 8 48 648 648 1648 6648 96 97 SVAAAA IINAAA AAAAxx +7076 9005 0 0 6 16 76 76 1076 2076 7076 152 153 EMAAAA JINAAA HHHHxx +6919 9006 1 3 9 19 19 919 919 1919 6919 38 39 DGAAAA KINAAA OOOOxx +1108 9007 0 0 8 8 8 108 1108 1108 1108 16 17 QQAAAA LINAAA VVVVxx +317 9008 1 1 7 17 17 317 317 317 317 34 35 FMAAAA MINAAA AAAAxx +3483 9009 1 3 3 3 83 483 1483 3483 3483 166 167 ZDAAAA NINAAA HHHHxx +6764 9010 0 0 4 4 64 764 764 1764 6764 128 129 EAAAAA OINAAA OOOOxx +1235 9011 1 3 5 15 35 235 1235 1235 1235 70 71 NVAAAA PINAAA VVVVxx +7121 9012 1 1 1 1 21 121 1121 2121 7121 42 43 XNAAAA QINAAA AAAAxx +426 9013 0 2 6 6 26 426 426 426 426 52 53 KQAAAA RINAAA HHHHxx +6880 9014 0 0 0 0 80 880 880 1880 6880 160 161 QEAAAA SINAAA OOOOxx +5401 9015 1 1 1 1 1 401 1401 401 5401 2 3 TZAAAA TINAAA VVVVxx +7323 9016 1 3 3 3 23 323 1323 2323 7323 46 47 RVAAAA UINAAA AAAAxx +9751 9017 1 3 1 11 51 751 1751 4751 9751 102 103 BLAAAA VINAAA HHHHxx +3436 9018 0 0 6 16 36 436 1436 3436 3436 72 73 ECAAAA WINAAA OOOOxx +7319 9019 1 3 9 19 19 319 1319 2319 7319 38 39 NVAAAA XINAAA VVVVxx +7882 9020 0 2 2 2 82 882 1882 2882 7882 164 165 ERAAAA YINAAA AAAAxx +8260 9021 0 0 0 0 60 260 260 3260 8260 120 121 SFAAAA ZINAAA HHHHxx +9758 9022 0 2 8 18 58 758 1758 4758 9758 116 117 ILAAAA AJNAAA OOOOxx +4205 9023 1 1 5 5 5 205 205 4205 4205 10 11 TFAAAA BJNAAA VVVVxx +8884 9024 0 0 4 4 84 884 884 3884 8884 168 169 SDAAAA CJNAAA AAAAxx +1112 9025 0 0 2 12 12 112 1112 1112 1112 24 25 UQAAAA DJNAAA HHHHxx +2186 9026 0 2 6 6 86 186 186 2186 2186 172 173 CGAAAA EJNAAA OOOOxx +8666 9027 0 2 6 6 66 666 666 3666 8666 132 133 IVAAAA FJNAAA VVVVxx +4325 9028 1 1 5 5 25 325 325 4325 4325 50 51 JKAAAA GJNAAA AAAAxx +4912 9029 0 0 2 12 12 912 912 4912 4912 24 25 YGAAAA HJNAAA HHHHxx +6497 9030 1 1 7 17 97 497 497 1497 6497 194 195 XPAAAA IJNAAA OOOOxx +9072 9031 0 0 2 12 72 72 1072 4072 9072 144 145 YKAAAA JJNAAA VVVVxx +8899 9032 1 3 9 19 99 899 899 3899 8899 198 199 HEAAAA KJNAAA AAAAxx +5619 9033 1 3 9 19 19 619 1619 619 5619 38 39 DIAAAA LJNAAA HHHHxx +4110 9034 0 2 0 10 10 110 110 4110 4110 20 21 CCAAAA MJNAAA OOOOxx +7025 9035 1 1 5 5 25 25 1025 2025 7025 50 51 FKAAAA NJNAAA VVVVxx +5605 9036 1 1 5 5 5 605 1605 605 5605 10 11 PHAAAA OJNAAA AAAAxx +2572 9037 0 0 2 12 72 572 572 2572 2572 144 145 YUAAAA PJNAAA HHHHxx +3895 9038 1 3 5 15 95 895 1895 3895 3895 190 191 VTAAAA QJNAAA OOOOxx +9138 9039 0 2 8 18 38 138 1138 4138 9138 76 77 MNAAAA RJNAAA VVVVxx +4713 9040 1 1 3 13 13 713 713 4713 4713 26 27 HZAAAA SJNAAA AAAAxx +6079 9041 1 3 9 19 79 79 79 1079 6079 158 159 VZAAAA TJNAAA HHHHxx +8898 9042 0 2 8 18 98 898 898 3898 8898 196 197 GEAAAA UJNAAA OOOOxx +2650 9043 0 2 0 10 50 650 650 2650 2650 100 101 YXAAAA VJNAAA VVVVxx +5316 9044 0 0 6 16 16 316 1316 316 5316 32 33 MWAAAA WJNAAA AAAAxx +5133 9045 1 1 3 13 33 133 1133 133 5133 66 67 LPAAAA XJNAAA HHHHxx +2184 9046 0 0 4 4 84 184 184 2184 2184 168 169 AGAAAA YJNAAA OOOOxx +2728 9047 0 0 8 8 28 728 728 2728 2728 56 57 YAAAAA ZJNAAA VVVVxx +6737 9048 1 1 7 17 37 737 737 1737 6737 74 75 DZAAAA AKNAAA AAAAxx +1128 9049 0 0 8 8 28 128 1128 1128 1128 56 57 KRAAAA BKNAAA HHHHxx +9662 9050 0 2 2 2 62 662 1662 4662 9662 124 125 QHAAAA CKNAAA OOOOxx +9384 9051 0 0 4 4 84 384 1384 4384 9384 168 169 YWAAAA DKNAAA VVVVxx +4576 9052 0 0 6 16 76 576 576 4576 4576 152 153 AUAAAA EKNAAA AAAAxx +9613 9053 1 1 3 13 13 613 1613 4613 9613 26 27 TFAAAA FKNAAA HHHHxx +4001 9054 1 1 1 1 1 1 1 4001 4001 2 3 XXAAAA GKNAAA OOOOxx +3628 9055 0 0 8 8 28 628 1628 3628 3628 56 57 OJAAAA HKNAAA VVVVxx +6968 9056 0 0 8 8 68 968 968 1968 6968 136 137 AIAAAA IKNAAA AAAAxx +6491 9057 1 3 1 11 91 491 491 1491 6491 182 183 RPAAAA JKNAAA HHHHxx +1265 9058 1 1 5 5 65 265 1265 1265 1265 130 131 RWAAAA KKNAAA OOOOxx +6128 9059 0 0 8 8 28 128 128 1128 6128 56 57 SBAAAA LKNAAA VVVVxx +4274 9060 0 2 4 14 74 274 274 4274 4274 148 149 KIAAAA MKNAAA AAAAxx +3598 9061 0 2 8 18 98 598 1598 3598 3598 196 197 KIAAAA NKNAAA HHHHxx +7961 9062 1 1 1 1 61 961 1961 2961 7961 122 123 FUAAAA OKNAAA OOOOxx +2643 9063 1 3 3 3 43 643 643 2643 2643 86 87 RXAAAA PKNAAA VVVVxx +4547 9064 1 3 7 7 47 547 547 4547 4547 94 95 XSAAAA QKNAAA AAAAxx +3568 9065 0 0 8 8 68 568 1568 3568 3568 136 137 GHAAAA RKNAAA HHHHxx +8954 9066 0 2 4 14 54 954 954 3954 8954 108 109 KGAAAA SKNAAA OOOOxx +8802 9067 0 2 2 2 2 802 802 3802 8802 4 5 OAAAAA TKNAAA VVVVxx +7829 9068 1 1 9 9 29 829 1829 2829 7829 58 59 DPAAAA UKNAAA AAAAxx +1008 9069 0 0 8 8 8 8 1008 1008 1008 16 17 UMAAAA VKNAAA HHHHxx +3627 9070 1 3 7 7 27 627 1627 3627 3627 54 55 NJAAAA WKNAAA OOOOxx +3999 9071 1 3 9 19 99 999 1999 3999 3999 198 199 VXAAAA XKNAAA VVVVxx +7697 9072 1 1 7 17 97 697 1697 2697 7697 194 195 BKAAAA YKNAAA AAAAxx +9380 9073 0 0 0 0 80 380 1380 4380 9380 160 161 UWAAAA ZKNAAA HHHHxx +2707 9074 1 3 7 7 7 707 707 2707 2707 14 15 DAAAAA ALNAAA OOOOxx +4430 9075 0 2 0 10 30 430 430 4430 4430 60 61 KOAAAA BLNAAA VVVVxx +6440 9076 0 0 0 0 40 440 440 1440 6440 80 81 SNAAAA CLNAAA AAAAxx +9958 9077 0 2 8 18 58 958 1958 4958 9958 116 117 ATAAAA DLNAAA HHHHxx +7592 9078 0 0 2 12 92 592 1592 2592 7592 184 185 AGAAAA ELNAAA OOOOxx +7852 9079 0 0 2 12 52 852 1852 2852 7852 104 105 AQAAAA FLNAAA VVVVxx +9253 9080 1 1 3 13 53 253 1253 4253 9253 106 107 XRAAAA GLNAAA AAAAxx +5910 9081 0 2 0 10 10 910 1910 910 5910 20 21 ITAAAA HLNAAA HHHHxx +7487 9082 1 3 7 7 87 487 1487 2487 7487 174 175 ZBAAAA ILNAAA OOOOxx +6324 9083 0 0 4 4 24 324 324 1324 6324 48 49 GJAAAA JLNAAA VVVVxx +5792 9084 0 0 2 12 92 792 1792 792 5792 184 185 UOAAAA KLNAAA AAAAxx +7390 9085 0 2 0 10 90 390 1390 2390 7390 180 181 GYAAAA LLNAAA HHHHxx +8534 9086 0 2 4 14 34 534 534 3534 8534 68 69 GQAAAA MLNAAA OOOOxx +2690 9087 0 2 0 10 90 690 690 2690 2690 180 181 MZAAAA NLNAAA VVVVxx +3992 9088 0 0 2 12 92 992 1992 3992 3992 184 185 OXAAAA OLNAAA AAAAxx +6928 9089 0 0 8 8 28 928 928 1928 6928 56 57 MGAAAA PLNAAA HHHHxx +7815 9090 1 3 5 15 15 815 1815 2815 7815 30 31 POAAAA QLNAAA OOOOxx +9477 9091 1 1 7 17 77 477 1477 4477 9477 154 155 NAAAAA RLNAAA VVVVxx +497 9092 1 1 7 17 97 497 497 497 497 194 195 DTAAAA SLNAAA AAAAxx +7532 9093 0 0 2 12 32 532 1532 2532 7532 64 65 SDAAAA TLNAAA HHHHxx +9838 9094 0 2 8 18 38 838 1838 4838 9838 76 77 KOAAAA ULNAAA OOOOxx +1557 9095 1 1 7 17 57 557 1557 1557 1557 114 115 XHAAAA VLNAAA VVVVxx +2467 9096 1 3 7 7 67 467 467 2467 2467 134 135 XQAAAA WLNAAA AAAAxx +2367 9097 1 3 7 7 67 367 367 2367 2367 134 135 BNAAAA XLNAAA HHHHxx +5677 9098 1 1 7 17 77 677 1677 677 5677 154 155 JKAAAA YLNAAA OOOOxx +6193 9099 1 1 3 13 93 193 193 1193 6193 186 187 FEAAAA ZLNAAA VVVVxx +7126 9100 0 2 6 6 26 126 1126 2126 7126 52 53 COAAAA AMNAAA AAAAxx +5264 9101 0 0 4 4 64 264 1264 264 5264 128 129 MUAAAA BMNAAA HHHHxx +850 9102 0 2 0 10 50 850 850 850 850 100 101 SGAAAA CMNAAA OOOOxx +4854 9103 0 2 4 14 54 854 854 4854 4854 108 109 SEAAAA DMNAAA VVVVxx +4414 9104 0 2 4 14 14 414 414 4414 4414 28 29 UNAAAA EMNAAA AAAAxx +8971 9105 1 3 1 11 71 971 971 3971 8971 142 143 BHAAAA FMNAAA HHHHxx +9240 9106 0 0 0 0 40 240 1240 4240 9240 80 81 KRAAAA GMNAAA OOOOxx +7341 9107 1 1 1 1 41 341 1341 2341 7341 82 83 JWAAAA HMNAAA VVVVxx +3151 9108 1 3 1 11 51 151 1151 3151 3151 102 103 FRAAAA IMNAAA AAAAxx +1742 9109 0 2 2 2 42 742 1742 1742 1742 84 85 APAAAA JMNAAA HHHHxx +1347 9110 1 3 7 7 47 347 1347 1347 1347 94 95 VZAAAA KMNAAA OOOOxx +9418 9111 0 2 8 18 18 418 1418 4418 9418 36 37 GYAAAA LMNAAA VVVVxx +5452 9112 0 0 2 12 52 452 1452 452 5452 104 105 SBAAAA MMNAAA AAAAxx +8637 9113 1 1 7 17 37 637 637 3637 8637 74 75 FUAAAA NMNAAA HHHHxx +8287 9114 1 3 7 7 87 287 287 3287 8287 174 175 TGAAAA OMNAAA OOOOxx +9865 9115 1 1 5 5 65 865 1865 4865 9865 130 131 LPAAAA PMNAAA VVVVxx +1664 9116 0 0 4 4 64 664 1664 1664 1664 128 129 AMAAAA QMNAAA AAAAxx +9933 9117 1 1 3 13 33 933 1933 4933 9933 66 67 BSAAAA RMNAAA HHHHxx +3416 9118 0 0 6 16 16 416 1416 3416 3416 32 33 KBAAAA SMNAAA OOOOxx +7981 9119 1 1 1 1 81 981 1981 2981 7981 162 163 ZUAAAA TMNAAA VVVVxx +1981 9120 1 1 1 1 81 981 1981 1981 1981 162 163 FYAAAA UMNAAA AAAAxx +441 9121 1 1 1 1 41 441 441 441 441 82 83 ZQAAAA VMNAAA HHHHxx +1380 9122 0 0 0 0 80 380 1380 1380 1380 160 161 CBAAAA WMNAAA OOOOxx +7325 9123 1 1 5 5 25 325 1325 2325 7325 50 51 TVAAAA XMNAAA VVVVxx +5682 9124 0 2 2 2 82 682 1682 682 5682 164 165 OKAAAA YMNAAA AAAAxx +1024 9125 0 0 4 4 24 24 1024 1024 1024 48 49 KNAAAA ZMNAAA HHHHxx +1096 9126 0 0 6 16 96 96 1096 1096 1096 192 193 EQAAAA ANNAAA OOOOxx +4717 9127 1 1 7 17 17 717 717 4717 4717 34 35 LZAAAA BNNAAA VVVVxx +7948 9128 0 0 8 8 48 948 1948 2948 7948 96 97 STAAAA CNNAAA AAAAxx +4074 9129 0 2 4 14 74 74 74 4074 4074 148 149 SAAAAA DNNAAA HHHHxx +211 9130 1 3 1 11 11 211 211 211 211 22 23 DIAAAA ENNAAA OOOOxx +8993 9131 1 1 3 13 93 993 993 3993 8993 186 187 XHAAAA FNNAAA VVVVxx +4509 9132 1 1 9 9 9 509 509 4509 4509 18 19 LRAAAA GNNAAA AAAAxx +823 9133 1 3 3 3 23 823 823 823 823 46 47 RFAAAA HNNAAA HHHHxx +4747 9134 1 3 7 7 47 747 747 4747 4747 94 95 PAAAAA INNAAA OOOOxx +6955 9135 1 3 5 15 55 955 955 1955 6955 110 111 NHAAAA JNNAAA VVVVxx +7922 9136 0 2 2 2 22 922 1922 2922 7922 44 45 SSAAAA KNNAAA AAAAxx +6936 9137 0 0 6 16 36 936 936 1936 6936 72 73 UGAAAA LNNAAA HHHHxx +1546 9138 0 2 6 6 46 546 1546 1546 1546 92 93 MHAAAA MNNAAA OOOOxx +9836 9139 0 0 6 16 36 836 1836 4836 9836 72 73 IOAAAA NNNAAA VVVVxx +5626 9140 0 2 6 6 26 626 1626 626 5626 52 53 KIAAAA ONNAAA AAAAxx +4879 9141 1 3 9 19 79 879 879 4879 4879 158 159 RFAAAA PNNAAA HHHHxx +8590 9142 0 2 0 10 90 590 590 3590 8590 180 181 KSAAAA QNNAAA OOOOxx +8842 9143 0 2 2 2 42 842 842 3842 8842 84 85 CCAAAA RNNAAA VVVVxx +6505 9144 1 1 5 5 5 505 505 1505 6505 10 11 FQAAAA SNNAAA AAAAxx +2803 9145 1 3 3 3 3 803 803 2803 2803 6 7 VDAAAA TNNAAA HHHHxx +9258 9146 0 2 8 18 58 258 1258 4258 9258 116 117 CSAAAA UNNAAA OOOOxx +741 9147 1 1 1 1 41 741 741 741 741 82 83 NCAAAA VNNAAA VVVVxx +1457 9148 1 1 7 17 57 457 1457 1457 1457 114 115 BEAAAA WNNAAA AAAAxx +5777 9149 1 1 7 17 77 777 1777 777 5777 154 155 FOAAAA XNNAAA HHHHxx +2883 9150 1 3 3 3 83 883 883 2883 2883 166 167 XGAAAA YNNAAA OOOOxx +6610 9151 0 2 0 10 10 610 610 1610 6610 20 21 GUAAAA ZNNAAA VVVVxx +4331 9152 1 3 1 11 31 331 331 4331 4331 62 63 PKAAAA AONAAA AAAAxx +2712 9153 0 0 2 12 12 712 712 2712 2712 24 25 IAAAAA BONAAA HHHHxx +9268 9154 0 0 8 8 68 268 1268 4268 9268 136 137 MSAAAA CONAAA OOOOxx +410 9155 0 2 0 10 10 410 410 410 410 20 21 UPAAAA DONAAA VVVVxx +9411 9156 1 3 1 11 11 411 1411 4411 9411 22 23 ZXAAAA EONAAA AAAAxx +4683 9157 1 3 3 3 83 683 683 4683 4683 166 167 DYAAAA FONAAA HHHHxx +7072 9158 0 0 2 12 72 72 1072 2072 7072 144 145 AMAAAA GONAAA OOOOxx +5050 9159 0 2 0 10 50 50 1050 50 5050 100 101 GMAAAA HONAAA VVVVxx +5932 9160 0 0 2 12 32 932 1932 932 5932 64 65 EUAAAA IONAAA AAAAxx +2756 9161 0 0 6 16 56 756 756 2756 2756 112 113 ACAAAA JONAAA HHHHxx +9813 9162 1 1 3 13 13 813 1813 4813 9813 26 27 LNAAAA KONAAA OOOOxx +7388 9163 0 0 8 8 88 388 1388 2388 7388 176 177 EYAAAA LONAAA VVVVxx +2596 9164 0 0 6 16 96 596 596 2596 2596 192 193 WVAAAA MONAAA AAAAxx +5102 9165 0 2 2 2 2 102 1102 102 5102 4 5 GOAAAA NONAAA HHHHxx +208 9166 0 0 8 8 8 208 208 208 208 16 17 AIAAAA OONAAA OOOOxx +86 9167 0 2 6 6 86 86 86 86 86 172 173 IDAAAA PONAAA VVVVxx +8127 9168 1 3 7 7 27 127 127 3127 8127 54 55 PAAAAA QONAAA AAAAxx +5154 9169 0 2 4 14 54 154 1154 154 5154 108 109 GQAAAA RONAAA HHHHxx +4491 9170 1 3 1 11 91 491 491 4491 4491 182 183 TQAAAA SONAAA OOOOxx +7423 9171 1 3 3 3 23 423 1423 2423 7423 46 47 NZAAAA TONAAA VVVVxx +6441 9172 1 1 1 1 41 441 441 1441 6441 82 83 TNAAAA UONAAA AAAAxx +2920 9173 0 0 0 0 20 920 920 2920 2920 40 41 IIAAAA VONAAA HHHHxx +6386 9174 0 2 6 6 86 386 386 1386 6386 172 173 QLAAAA WONAAA OOOOxx +9744 9175 0 0 4 4 44 744 1744 4744 9744 88 89 UKAAAA XONAAA VVVVxx +2667 9176 1 3 7 7 67 667 667 2667 2667 134 135 PYAAAA YONAAA AAAAxx +5754 9177 0 2 4 14 54 754 1754 754 5754 108 109 INAAAA ZONAAA HHHHxx +4645 9178 1 1 5 5 45 645 645 4645 4645 90 91 RWAAAA APNAAA OOOOxx +4327 9179 1 3 7 7 27 327 327 4327 4327 54 55 LKAAAA BPNAAA VVVVxx +843 9180 1 3 3 3 43 843 843 843 843 86 87 LGAAAA CPNAAA AAAAxx +4085 9181 1 1 5 5 85 85 85 4085 4085 170 171 DBAAAA DPNAAA HHHHxx +2849 9182 1 1 9 9 49 849 849 2849 2849 98 99 PFAAAA EPNAAA OOOOxx +5734 9183 0 2 4 14 34 734 1734 734 5734 68 69 OMAAAA FPNAAA VVVVxx +5307 9184 1 3 7 7 7 307 1307 307 5307 14 15 DWAAAA GPNAAA AAAAxx +8433 9185 1 1 3 13 33 433 433 3433 8433 66 67 JMAAAA HPNAAA HHHHxx +3031 9186 1 3 1 11 31 31 1031 3031 3031 62 63 PMAAAA IPNAAA OOOOxx +5714 9187 0 2 4 14 14 714 1714 714 5714 28 29 ULAAAA JPNAAA VVVVxx +5969 9188 1 1 9 9 69 969 1969 969 5969 138 139 PVAAAA KPNAAA AAAAxx +2532 9189 0 0 2 12 32 532 532 2532 2532 64 65 KTAAAA LPNAAA HHHHxx +5219 9190 1 3 9 19 19 219 1219 219 5219 38 39 TSAAAA MPNAAA OOOOxx +7343 9191 1 3 3 3 43 343 1343 2343 7343 86 87 LWAAAA NPNAAA VVVVxx +9089 9192 1 1 9 9 89 89 1089 4089 9089 178 179 PLAAAA OPNAAA AAAAxx +9337 9193 1 1 7 17 37 337 1337 4337 9337 74 75 DVAAAA PPNAAA HHHHxx +5131 9194 1 3 1 11 31 131 1131 131 5131 62 63 JPAAAA QPNAAA OOOOxx +6253 9195 1 1 3 13 53 253 253 1253 6253 106 107 NGAAAA RPNAAA VVVVxx +5140 9196 0 0 0 0 40 140 1140 140 5140 80 81 SPAAAA SPNAAA AAAAxx +2953 9197 1 1 3 13 53 953 953 2953 2953 106 107 PJAAAA TPNAAA HHHHxx +4293 9198 1 1 3 13 93 293 293 4293 4293 186 187 DJAAAA UPNAAA OOOOxx +9974 9199 0 2 4 14 74 974 1974 4974 9974 148 149 QTAAAA VPNAAA VVVVxx +5061 9200 1 1 1 1 61 61 1061 61 5061 122 123 RMAAAA WPNAAA AAAAxx +8570 9201 0 2 0 10 70 570 570 3570 8570 140 141 QRAAAA XPNAAA HHHHxx +9504 9202 0 0 4 4 4 504 1504 4504 9504 8 9 OBAAAA YPNAAA OOOOxx +604 9203 0 0 4 4 4 604 604 604 604 8 9 GXAAAA ZPNAAA VVVVxx +4991 9204 1 3 1 11 91 991 991 4991 4991 182 183 ZJAAAA AQNAAA AAAAxx +880 9205 0 0 0 0 80 880 880 880 880 160 161 WHAAAA BQNAAA HHHHxx +3861 9206 1 1 1 1 61 861 1861 3861 3861 122 123 NSAAAA CQNAAA OOOOxx +8262 9207 0 2 2 2 62 262 262 3262 8262 124 125 UFAAAA DQNAAA VVVVxx +5689 9208 1 1 9 9 89 689 1689 689 5689 178 179 VKAAAA EQNAAA AAAAxx +1793 9209 1 1 3 13 93 793 1793 1793 1793 186 187 ZQAAAA FQNAAA HHHHxx +2661 9210 1 1 1 1 61 661 661 2661 2661 122 123 JYAAAA GQNAAA OOOOxx +7954 9211 0 2 4 14 54 954 1954 2954 7954 108 109 YTAAAA HQNAAA VVVVxx +1874 9212 0 2 4 14 74 874 1874 1874 1874 148 149 CUAAAA IQNAAA AAAAxx +2982 9213 0 2 2 2 82 982 982 2982 2982 164 165 SKAAAA JQNAAA HHHHxx +331 9214 1 3 1 11 31 331 331 331 331 62 63 TMAAAA KQNAAA OOOOxx +5021 9215 1 1 1 1 21 21 1021 21 5021 42 43 DLAAAA LQNAAA VVVVxx +9894 9216 0 2 4 14 94 894 1894 4894 9894 188 189 OQAAAA MQNAAA AAAAxx +7709 9217 1 1 9 9 9 709 1709 2709 7709 18 19 NKAAAA NQNAAA HHHHxx +4980 9218 0 0 0 0 80 980 980 4980 4980 160 161 OJAAAA OQNAAA OOOOxx +8249 9219 1 1 9 9 49 249 249 3249 8249 98 99 HFAAAA PQNAAA VVVVxx +7120 9220 0 0 0 0 20 120 1120 2120 7120 40 41 WNAAAA QQNAAA AAAAxx +7464 9221 0 0 4 4 64 464 1464 2464 7464 128 129 CBAAAA RQNAAA HHHHxx +8086 9222 0 2 6 6 86 86 86 3086 8086 172 173 AZAAAA SQNAAA OOOOxx +3509 9223 1 1 9 9 9 509 1509 3509 3509 18 19 ZEAAAA TQNAAA VVVVxx +3902 9224 0 2 2 2 2 902 1902 3902 3902 4 5 CUAAAA UQNAAA AAAAxx +9907 9225 1 3 7 7 7 907 1907 4907 9907 14 15 BRAAAA VQNAAA HHHHxx +6278 9226 0 2 8 18 78 278 278 1278 6278 156 157 MHAAAA WQNAAA OOOOxx +9316 9227 0 0 6 16 16 316 1316 4316 9316 32 33 IUAAAA XQNAAA VVVVxx +2824 9228 0 0 4 4 24 824 824 2824 2824 48 49 QEAAAA YQNAAA AAAAxx +1558 9229 0 2 8 18 58 558 1558 1558 1558 116 117 YHAAAA ZQNAAA HHHHxx +5436 9230 0 0 6 16 36 436 1436 436 5436 72 73 CBAAAA ARNAAA OOOOxx +1161 9231 1 1 1 1 61 161 1161 1161 1161 122 123 RSAAAA BRNAAA VVVVxx +7569 9232 1 1 9 9 69 569 1569 2569 7569 138 139 DFAAAA CRNAAA AAAAxx +9614 9233 0 2 4 14 14 614 1614 4614 9614 28 29 UFAAAA DRNAAA HHHHxx +6970 9234 0 2 0 10 70 970 970 1970 6970 140 141 CIAAAA ERNAAA OOOOxx +2422 9235 0 2 2 2 22 422 422 2422 2422 44 45 EPAAAA FRNAAA VVVVxx +8860 9236 0 0 0 0 60 860 860 3860 8860 120 121 UCAAAA GRNAAA AAAAxx +9912 9237 0 0 2 12 12 912 1912 4912 9912 24 25 GRAAAA HRNAAA HHHHxx +1109 9238 1 1 9 9 9 109 1109 1109 1109 18 19 RQAAAA IRNAAA OOOOxx +3286 9239 0 2 6 6 86 286 1286 3286 3286 172 173 KWAAAA JRNAAA VVVVxx +2277 9240 1 1 7 17 77 277 277 2277 2277 154 155 PJAAAA KRNAAA AAAAxx +8656 9241 0 0 6 16 56 656 656 3656 8656 112 113 YUAAAA LRNAAA HHHHxx +4656 9242 0 0 6 16 56 656 656 4656 4656 112 113 CXAAAA MRNAAA OOOOxx +6965 9243 1 1 5 5 65 965 965 1965 6965 130 131 XHAAAA NRNAAA VVVVxx +7591 9244 1 3 1 11 91 591 1591 2591 7591 182 183 ZFAAAA ORNAAA AAAAxx +4883 9245 1 3 3 3 83 883 883 4883 4883 166 167 VFAAAA PRNAAA HHHHxx +452 9246 0 0 2 12 52 452 452 452 452 104 105 KRAAAA QRNAAA OOOOxx +4018 9247 0 2 8 18 18 18 18 4018 4018 36 37 OYAAAA RRNAAA VVVVxx +4066 9248 0 2 6 6 66 66 66 4066 4066 132 133 KAAAAA SRNAAA AAAAxx +6480 9249 0 0 0 0 80 480 480 1480 6480 160 161 GPAAAA TRNAAA HHHHxx +8634 9250 0 2 4 14 34 634 634 3634 8634 68 69 CUAAAA URNAAA OOOOxx +9387 9251 1 3 7 7 87 387 1387 4387 9387 174 175 BXAAAA VRNAAA VVVVxx +3476 9252 0 0 6 16 76 476 1476 3476 3476 152 153 SDAAAA WRNAAA AAAAxx +5995 9253 1 3 5 15 95 995 1995 995 5995 190 191 PWAAAA XRNAAA HHHHxx +9677 9254 1 1 7 17 77 677 1677 4677 9677 154 155 FIAAAA YRNAAA OOOOxx +3884 9255 0 0 4 4 84 884 1884 3884 3884 168 169 KTAAAA ZRNAAA VVVVxx +6500 9256 0 0 0 0 0 500 500 1500 6500 0 1 AQAAAA ASNAAA AAAAxx +7972 9257 0 0 2 12 72 972 1972 2972 7972 144 145 QUAAAA BSNAAA HHHHxx +5281 9258 1 1 1 1 81 281 1281 281 5281 162 163 DVAAAA CSNAAA OOOOxx +1288 9259 0 0 8 8 88 288 1288 1288 1288 176 177 OXAAAA DSNAAA VVVVxx +4366 9260 0 2 6 6 66 366 366 4366 4366 132 133 YLAAAA ESNAAA AAAAxx +6557 9261 1 1 7 17 57 557 557 1557 6557 114 115 FSAAAA FSNAAA HHHHxx +7086 9262 0 2 6 6 86 86 1086 2086 7086 172 173 OMAAAA GSNAAA OOOOxx +6588 9263 0 0 8 8 88 588 588 1588 6588 176 177 KTAAAA HSNAAA VVVVxx +9062 9264 0 2 2 2 62 62 1062 4062 9062 124 125 OKAAAA ISNAAA AAAAxx +9230 9265 0 2 0 10 30 230 1230 4230 9230 60 61 ARAAAA JSNAAA HHHHxx +7672 9266 0 0 2 12 72 672 1672 2672 7672 144 145 CJAAAA KSNAAA OOOOxx +5204 9267 0 0 4 4 4 204 1204 204 5204 8 9 ESAAAA LSNAAA VVVVxx +2836 9268 0 0 6 16 36 836 836 2836 2836 72 73 CFAAAA MSNAAA AAAAxx +7165 9269 1 1 5 5 65 165 1165 2165 7165 130 131 PPAAAA NSNAAA HHHHxx +971 9270 1 3 1 11 71 971 971 971 971 142 143 JLAAAA OSNAAA OOOOxx +3851 9271 1 3 1 11 51 851 1851 3851 3851 102 103 DSAAAA PSNAAA VVVVxx +8593 9272 1 1 3 13 93 593 593 3593 8593 186 187 NSAAAA QSNAAA AAAAxx +7742 9273 0 2 2 2 42 742 1742 2742 7742 84 85 ULAAAA RSNAAA HHHHxx +2887 9274 1 3 7 7 87 887 887 2887 2887 174 175 BHAAAA SSNAAA OOOOxx +8479 9275 1 3 9 19 79 479 479 3479 8479 158 159 DOAAAA TSNAAA VVVVxx +9514 9276 0 2 4 14 14 514 1514 4514 9514 28 29 YBAAAA USNAAA AAAAxx +273 9277 1 1 3 13 73 273 273 273 273 146 147 NKAAAA VSNAAA HHHHxx +2938 9278 0 2 8 18 38 938 938 2938 2938 76 77 AJAAAA WSNAAA OOOOxx +9793 9279 1 1 3 13 93 793 1793 4793 9793 186 187 RMAAAA XSNAAA VVVVxx +8050 9280 0 2 0 10 50 50 50 3050 8050 100 101 QXAAAA YSNAAA AAAAxx +6702 9281 0 2 2 2 2 702 702 1702 6702 4 5 UXAAAA ZSNAAA HHHHxx +7290 9282 0 2 0 10 90 290 1290 2290 7290 180 181 KUAAAA ATNAAA OOOOxx +1837 9283 1 1 7 17 37 837 1837 1837 1837 74 75 RSAAAA BTNAAA VVVVxx +3206 9284 0 2 6 6 6 206 1206 3206 3206 12 13 ITAAAA CTNAAA AAAAxx +4925 9285 1 1 5 5 25 925 925 4925 4925 50 51 LHAAAA DTNAAA HHHHxx +5066 9286 0 2 6 6 66 66 1066 66 5066 132 133 WMAAAA ETNAAA OOOOxx +3401 9287 1 1 1 1 1 401 1401 3401 3401 2 3 VAAAAA FTNAAA VVVVxx +3474 9288 0 2 4 14 74 474 1474 3474 3474 148 149 QDAAAA GTNAAA AAAAxx +57 9289 1 1 7 17 57 57 57 57 57 114 115 FCAAAA HTNAAA HHHHxx +2082 9290 0 2 2 2 82 82 82 2082 2082 164 165 CCAAAA ITNAAA OOOOxx +100 9291 0 0 0 0 0 100 100 100 100 0 1 WDAAAA JTNAAA VVVVxx +9665 9292 1 1 5 5 65 665 1665 4665 9665 130 131 THAAAA KTNAAA AAAAxx +8284 9293 0 0 4 4 84 284 284 3284 8284 168 169 QGAAAA LTNAAA HHHHxx +958 9294 0 2 8 18 58 958 958 958 958 116 117 WKAAAA MTNAAA OOOOxx +5282 9295 0 2 2 2 82 282 1282 282 5282 164 165 EVAAAA NTNAAA VVVVxx +4257 9296 1 1 7 17 57 257 257 4257 4257 114 115 THAAAA OTNAAA AAAAxx +3160 9297 0 0 0 0 60 160 1160 3160 3160 120 121 ORAAAA PTNAAA HHHHxx +8449 9298 1 1 9 9 49 449 449 3449 8449 98 99 ZMAAAA QTNAAA OOOOxx +500 9299 0 0 0 0 0 500 500 500 500 0 1 GTAAAA RTNAAA VVVVxx +6432 9300 0 0 2 12 32 432 432 1432 6432 64 65 KNAAAA STNAAA AAAAxx +6220 9301 0 0 0 0 20 220 220 1220 6220 40 41 GFAAAA TTNAAA HHHHxx +7233 9302 1 1 3 13 33 233 1233 2233 7233 66 67 FSAAAA UTNAAA OOOOxx +2723 9303 1 3 3 3 23 723 723 2723 2723 46 47 TAAAAA VTNAAA VVVVxx +1899 9304 1 3 9 19 99 899 1899 1899 1899 198 199 BVAAAA WTNAAA AAAAxx +7158 9305 0 2 8 18 58 158 1158 2158 7158 116 117 IPAAAA XTNAAA HHHHxx +202 9306 0 2 2 2 2 202 202 202 202 4 5 UHAAAA YTNAAA OOOOxx +2286 9307 0 2 6 6 86 286 286 2286 2286 172 173 YJAAAA ZTNAAA VVVVxx +5356 9308 0 0 6 16 56 356 1356 356 5356 112 113 AYAAAA AUNAAA AAAAxx +3809 9309 1 1 9 9 9 809 1809 3809 3809 18 19 NQAAAA BUNAAA HHHHxx +3979 9310 1 3 9 19 79 979 1979 3979 3979 158 159 BXAAAA CUNAAA OOOOxx +8359 9311 1 3 9 19 59 359 359 3359 8359 118 119 NJAAAA DUNAAA VVVVxx +3479 9312 1 3 9 19 79 479 1479 3479 3479 158 159 VDAAAA EUNAAA AAAAxx +4895 9313 1 3 5 15 95 895 895 4895 4895 190 191 HGAAAA FUNAAA HHHHxx +6059 9314 1 3 9 19 59 59 59 1059 6059 118 119 BZAAAA GUNAAA OOOOxx +9560 9315 0 0 0 0 60 560 1560 4560 9560 120 121 SDAAAA HUNAAA VVVVxx +6756 9316 0 0 6 16 56 756 756 1756 6756 112 113 WZAAAA IUNAAA AAAAxx +7504 9317 0 0 4 4 4 504 1504 2504 7504 8 9 QCAAAA JUNAAA HHHHxx +6762 9318 0 2 2 2 62 762 762 1762 6762 124 125 CAAAAA KUNAAA OOOOxx +5304 9319 0 0 4 4 4 304 1304 304 5304 8 9 AWAAAA LUNAAA VVVVxx +9533 9320 1 1 3 13 33 533 1533 4533 9533 66 67 RCAAAA MUNAAA AAAAxx +6649 9321 1 1 9 9 49 649 649 1649 6649 98 99 TVAAAA NUNAAA HHHHxx +38 9322 0 2 8 18 38 38 38 38 38 76 77 MBAAAA OUNAAA OOOOxx +5713 9323 1 1 3 13 13 713 1713 713 5713 26 27 TLAAAA PUNAAA VVVVxx +3000 9324 0 0 0 0 0 0 1000 3000 3000 0 1 KLAAAA QUNAAA AAAAxx +3738 9325 0 2 8 18 38 738 1738 3738 3738 76 77 UNAAAA RUNAAA HHHHxx +3327 9326 1 3 7 7 27 327 1327 3327 3327 54 55 ZXAAAA SUNAAA OOOOxx +3922 9327 0 2 2 2 22 922 1922 3922 3922 44 45 WUAAAA TUNAAA VVVVxx +9245 9328 1 1 5 5 45 245 1245 4245 9245 90 91 PRAAAA UUNAAA AAAAxx +2172 9329 0 0 2 12 72 172 172 2172 2172 144 145 OFAAAA VUNAAA HHHHxx +7128 9330 0 0 8 8 28 128 1128 2128 7128 56 57 EOAAAA WUNAAA OOOOxx +1195 9331 1 3 5 15 95 195 1195 1195 1195 190 191 ZTAAAA XUNAAA VVVVxx +8445 9332 1 1 5 5 45 445 445 3445 8445 90 91 VMAAAA YUNAAA AAAAxx +8638 9333 0 2 8 18 38 638 638 3638 8638 76 77 GUAAAA ZUNAAA HHHHxx +1249 9334 1 1 9 9 49 249 1249 1249 1249 98 99 BWAAAA AVNAAA OOOOxx +8659 9335 1 3 9 19 59 659 659 3659 8659 118 119 BVAAAA BVNAAA VVVVxx +3556 9336 0 0 6 16 56 556 1556 3556 3556 112 113 UGAAAA CVNAAA AAAAxx +3347 9337 1 3 7 7 47 347 1347 3347 3347 94 95 TYAAAA DVNAAA HHHHxx +3260 9338 0 0 0 0 60 260 1260 3260 3260 120 121 KVAAAA EVNAAA OOOOxx +5139 9339 1 3 9 19 39 139 1139 139 5139 78 79 RPAAAA FVNAAA VVVVxx +9991 9340 1 3 1 11 91 991 1991 4991 9991 182 183 HUAAAA GVNAAA AAAAxx +5499 9341 1 3 9 19 99 499 1499 499 5499 198 199 NDAAAA HVNAAA HHHHxx +8082 9342 0 2 2 2 82 82 82 3082 8082 164 165 WYAAAA IVNAAA OOOOxx +1640 9343 0 0 0 0 40 640 1640 1640 1640 80 81 CLAAAA JVNAAA VVVVxx +8726 9344 0 2 6 6 26 726 726 3726 8726 52 53 QXAAAA KVNAAA AAAAxx +2339 9345 1 3 9 19 39 339 339 2339 2339 78 79 ZLAAAA LVNAAA HHHHxx +2601 9346 1 1 1 1 1 601 601 2601 2601 2 3 BWAAAA MVNAAA OOOOxx +9940 9347 0 0 0 0 40 940 1940 4940 9940 80 81 ISAAAA NVNAAA VVVVxx +4185 9348 1 1 5 5 85 185 185 4185 4185 170 171 ZEAAAA OVNAAA AAAAxx +9546 9349 0 2 6 6 46 546 1546 4546 9546 92 93 EDAAAA PVNAAA HHHHxx +5218 9350 0 2 8 18 18 218 1218 218 5218 36 37 SSAAAA QVNAAA OOOOxx +4374 9351 0 2 4 14 74 374 374 4374 4374 148 149 GMAAAA RVNAAA VVVVxx +288 9352 0 0 8 8 88 288 288 288 288 176 177 CLAAAA SVNAAA AAAAxx +7445 9353 1 1 5 5 45 445 1445 2445 7445 90 91 JAAAAA TVNAAA HHHHxx +1710 9354 0 2 0 10 10 710 1710 1710 1710 20 21 UNAAAA UVNAAA OOOOxx +6409 9355 1 1 9 9 9 409 409 1409 6409 18 19 NMAAAA VVNAAA VVVVxx +7982 9356 0 2 2 2 82 982 1982 2982 7982 164 165 AVAAAA WVNAAA AAAAxx +4950 9357 0 2 0 10 50 950 950 4950 4950 100 101 KIAAAA XVNAAA HHHHxx +9242 9358 0 2 2 2 42 242 1242 4242 9242 84 85 MRAAAA YVNAAA OOOOxx +3272 9359 0 0 2 12 72 272 1272 3272 3272 144 145 WVAAAA ZVNAAA VVVVxx +739 9360 1 3 9 19 39 739 739 739 739 78 79 LCAAAA AWNAAA AAAAxx +5526 9361 0 2 6 6 26 526 1526 526 5526 52 53 OEAAAA BWNAAA HHHHxx +8189 9362 1 1 9 9 89 189 189 3189 8189 178 179 ZCAAAA CWNAAA OOOOxx +9106 9363 0 2 6 6 6 106 1106 4106 9106 12 13 GMAAAA DWNAAA VVVVxx +9775 9364 1 3 5 15 75 775 1775 4775 9775 150 151 ZLAAAA EWNAAA AAAAxx +4643 9365 1 3 3 3 43 643 643 4643 4643 86 87 PWAAAA FWNAAA HHHHxx +8396 9366 0 0 6 16 96 396 396 3396 8396 192 193 YKAAAA GWNAAA OOOOxx +3255 9367 1 3 5 15 55 255 1255 3255 3255 110 111 FVAAAA HWNAAA VVVVxx +301 9368 1 1 1 1 1 301 301 301 301 2 3 PLAAAA IWNAAA AAAAxx +6014 9369 0 2 4 14 14 14 14 1014 6014 28 29 IXAAAA JWNAAA HHHHxx +6046 9370 0 2 6 6 46 46 46 1046 6046 92 93 OYAAAA KWNAAA OOOOxx +984 9371 0 0 4 4 84 984 984 984 984 168 169 WLAAAA LWNAAA VVVVxx +2420 9372 0 0 0 0 20 420 420 2420 2420 40 41 CPAAAA MWNAAA AAAAxx +2922 9373 0 2 2 2 22 922 922 2922 2922 44 45 KIAAAA NWNAAA HHHHxx +2317 9374 1 1 7 17 17 317 317 2317 2317 34 35 DLAAAA OWNAAA OOOOxx +7332 9375 0 0 2 12 32 332 1332 2332 7332 64 65 AWAAAA PWNAAA VVVVxx +6451 9376 1 3 1 11 51 451 451 1451 6451 102 103 DOAAAA QWNAAA AAAAxx +2589 9377 1 1 9 9 89 589 589 2589 2589 178 179 PVAAAA RWNAAA HHHHxx +4333 9378 1 1 3 13 33 333 333 4333 4333 66 67 RKAAAA SWNAAA OOOOxx +8650 9379 0 2 0 10 50 650 650 3650 8650 100 101 SUAAAA TWNAAA VVVVxx +6856 9380 0 0 6 16 56 856 856 1856 6856 112 113 SDAAAA UWNAAA AAAAxx +4194 9381 0 2 4 14 94 194 194 4194 4194 188 189 IFAAAA VWNAAA HHHHxx +6246 9382 0 2 6 6 46 246 246 1246 6246 92 93 GGAAAA WWNAAA OOOOxx +4371 9383 1 3 1 11 71 371 371 4371 4371 142 143 DMAAAA XWNAAA VVVVxx +1388 9384 0 0 8 8 88 388 1388 1388 1388 176 177 KBAAAA YWNAAA AAAAxx +1056 9385 0 0 6 16 56 56 1056 1056 1056 112 113 QOAAAA ZWNAAA HHHHxx +6041 9386 1 1 1 1 41 41 41 1041 6041 82 83 JYAAAA AXNAAA OOOOxx +6153 9387 1 1 3 13 53 153 153 1153 6153 106 107 RCAAAA BXNAAA VVVVxx +8450 9388 0 2 0 10 50 450 450 3450 8450 100 101 ANAAAA CXNAAA AAAAxx +3469 9389 1 1 9 9 69 469 1469 3469 3469 138 139 LDAAAA DXNAAA HHHHxx +5226 9390 0 2 6 6 26 226 1226 226 5226 52 53 ATAAAA EXNAAA OOOOxx +8112 9391 0 0 2 12 12 112 112 3112 8112 24 25 AAAAAA FXNAAA VVVVxx +647 9392 1 3 7 7 47 647 647 647 647 94 95 XYAAAA GXNAAA AAAAxx +2567 9393 1 3 7 7 67 567 567 2567 2567 134 135 TUAAAA HXNAAA HHHHxx +9064 9394 0 0 4 4 64 64 1064 4064 9064 128 129 QKAAAA IXNAAA OOOOxx +5161 9395 1 1 1 1 61 161 1161 161 5161 122 123 NQAAAA JXNAAA VVVVxx +5260 9396 0 0 0 0 60 260 1260 260 5260 120 121 IUAAAA KXNAAA AAAAxx +8988 9397 0 0 8 8 88 988 988 3988 8988 176 177 SHAAAA LXNAAA HHHHxx +9678 9398 0 2 8 18 78 678 1678 4678 9678 156 157 GIAAAA MXNAAA OOOOxx +6853 9399 1 1 3 13 53 853 853 1853 6853 106 107 PDAAAA NXNAAA VVVVxx +5294 9400 0 2 4 14 94 294 1294 294 5294 188 189 QVAAAA OXNAAA AAAAxx +9864 9401 0 0 4 4 64 864 1864 4864 9864 128 129 KPAAAA PXNAAA HHHHxx +8702 9402 0 2 2 2 2 702 702 3702 8702 4 5 SWAAAA QXNAAA OOOOxx +1132 9403 0 0 2 12 32 132 1132 1132 1132 64 65 ORAAAA RXNAAA VVVVxx +1524 9404 0 0 4 4 24 524 1524 1524 1524 48 49 QGAAAA SXNAAA AAAAxx +4560 9405 0 0 0 0 60 560 560 4560 4560 120 121 KTAAAA TXNAAA HHHHxx +2137 9406 1 1 7 17 37 137 137 2137 2137 74 75 FEAAAA UXNAAA OOOOxx +3283 9407 1 3 3 3 83 283 1283 3283 3283 166 167 HWAAAA VXNAAA VVVVxx +3377 9408 1 1 7 17 77 377 1377 3377 3377 154 155 XZAAAA WXNAAA AAAAxx +2267 9409 1 3 7 7 67 267 267 2267 2267 134 135 FJAAAA XXNAAA HHHHxx +8987 9410 1 3 7 7 87 987 987 3987 8987 174 175 RHAAAA YXNAAA OOOOxx +6709 9411 1 1 9 9 9 709 709 1709 6709 18 19 BYAAAA ZXNAAA VVVVxx +8059 9412 1 3 9 19 59 59 59 3059 8059 118 119 ZXAAAA AYNAAA AAAAxx +3402 9413 0 2 2 2 2 402 1402 3402 3402 4 5 WAAAAA BYNAAA HHHHxx +6443 9414 1 3 3 3 43 443 443 1443 6443 86 87 VNAAAA CYNAAA OOOOxx +8858 9415 0 2 8 18 58 858 858 3858 8858 116 117 SCAAAA DYNAAA VVVVxx +3974 9416 0 2 4 14 74 974 1974 3974 3974 148 149 WWAAAA EYNAAA AAAAxx +3521 9417 1 1 1 1 21 521 1521 3521 3521 42 43 LFAAAA FYNAAA HHHHxx +9509 9418 1 1 9 9 9 509 1509 4509 9509 18 19 TBAAAA GYNAAA OOOOxx +5442 9419 0 2 2 2 42 442 1442 442 5442 84 85 IBAAAA HYNAAA VVVVxx +8968 9420 0 0 8 8 68 968 968 3968 8968 136 137 YGAAAA IYNAAA AAAAxx +333 9421 1 1 3 13 33 333 333 333 333 66 67 VMAAAA JYNAAA HHHHxx +952 9422 0 0 2 12 52 952 952 952 952 104 105 QKAAAA KYNAAA OOOOxx +7482 9423 0 2 2 2 82 482 1482 2482 7482 164 165 UBAAAA LYNAAA VVVVxx +1486 9424 0 2 6 6 86 486 1486 1486 1486 172 173 EFAAAA MYNAAA AAAAxx +1815 9425 1 3 5 15 15 815 1815 1815 1815 30 31 VRAAAA NYNAAA HHHHxx +7937 9426 1 1 7 17 37 937 1937 2937 7937 74 75 HTAAAA OYNAAA OOOOxx +1436 9427 0 0 6 16 36 436 1436 1436 1436 72 73 GDAAAA PYNAAA VVVVxx +3470 9428 0 2 0 10 70 470 1470 3470 3470 140 141 MDAAAA QYNAAA AAAAxx +8195 9429 1 3 5 15 95 195 195 3195 8195 190 191 FDAAAA RYNAAA HHHHxx +6906 9430 0 2 6 6 6 906 906 1906 6906 12 13 QFAAAA SYNAAA OOOOxx +2539 9431 1 3 9 19 39 539 539 2539 2539 78 79 RTAAAA TYNAAA VVVVxx +5988 9432 0 0 8 8 88 988 1988 988 5988 176 177 IWAAAA UYNAAA AAAAxx +8908 9433 0 0 8 8 8 908 908 3908 8908 16 17 QEAAAA VYNAAA HHHHxx +2319 9434 1 3 9 19 19 319 319 2319 2319 38 39 FLAAAA WYNAAA OOOOxx +3263 9435 1 3 3 3 63 263 1263 3263 3263 126 127 NVAAAA XYNAAA VVVVxx +4039 9436 1 3 9 19 39 39 39 4039 4039 78 79 JZAAAA YYNAAA AAAAxx +6373 9437 1 1 3 13 73 373 373 1373 6373 146 147 DLAAAA ZYNAAA HHHHxx +1168 9438 0 0 8 8 68 168 1168 1168 1168 136 137 YSAAAA AZNAAA OOOOxx +8338 9439 0 2 8 18 38 338 338 3338 8338 76 77 SIAAAA BZNAAA VVVVxx +1172 9440 0 0 2 12 72 172 1172 1172 1172 144 145 CTAAAA CZNAAA AAAAxx +200 9441 0 0 0 0 0 200 200 200 200 0 1 SHAAAA DZNAAA HHHHxx +6355 9442 1 3 5 15 55 355 355 1355 6355 110 111 LKAAAA EZNAAA OOOOxx +7768 9443 0 0 8 8 68 768 1768 2768 7768 136 137 UMAAAA FZNAAA VVVVxx +25 9444 1 1 5 5 25 25 25 25 25 50 51 ZAAAAA GZNAAA AAAAxx +7144 9445 0 0 4 4 44 144 1144 2144 7144 88 89 UOAAAA HZNAAA HHHHxx +8671 9446 1 3 1 11 71 671 671 3671 8671 142 143 NVAAAA IZNAAA OOOOxx +9163 9447 1 3 3 3 63 163 1163 4163 9163 126 127 LOAAAA JZNAAA VVVVxx +8889 9448 1 1 9 9 89 889 889 3889 8889 178 179 XDAAAA KZNAAA AAAAxx +5950 9449 0 2 0 10 50 950 1950 950 5950 100 101 WUAAAA LZNAAA HHHHxx +6163 9450 1 3 3 3 63 163 163 1163 6163 126 127 BDAAAA MZNAAA OOOOxx +8119 9451 1 3 9 19 19 119 119 3119 8119 38 39 HAAAAA NZNAAA VVVVxx +1416 9452 0 0 6 16 16 416 1416 1416 1416 32 33 MCAAAA OZNAAA AAAAxx +4132 9453 0 0 2 12 32 132 132 4132 4132 64 65 YCAAAA PZNAAA HHHHxx +2294 9454 0 2 4 14 94 294 294 2294 2294 188 189 GKAAAA QZNAAA OOOOxx +9094 9455 0 2 4 14 94 94 1094 4094 9094 188 189 ULAAAA RZNAAA VVVVxx +4168 9456 0 0 8 8 68 168 168 4168 4168 136 137 IEAAAA SZNAAA AAAAxx +9108 9457 0 0 8 8 8 108 1108 4108 9108 16 17 IMAAAA TZNAAA HHHHxx +5706 9458 0 2 6 6 6 706 1706 706 5706 12 13 MLAAAA UZNAAA OOOOxx +2231 9459 1 3 1 11 31 231 231 2231 2231 62 63 VHAAAA VZNAAA VVVVxx +2173 9460 1 1 3 13 73 173 173 2173 2173 146 147 PFAAAA WZNAAA AAAAxx +90 9461 0 2 0 10 90 90 90 90 90 180 181 MDAAAA XZNAAA HHHHxx +9996 9462 0 0 6 16 96 996 1996 4996 9996 192 193 MUAAAA YZNAAA OOOOxx +330 9463 0 2 0 10 30 330 330 330 330 60 61 SMAAAA ZZNAAA VVVVxx +2052 9464 0 0 2 12 52 52 52 2052 2052 104 105 YAAAAA AAOAAA AAAAxx +1093 9465 1 1 3 13 93 93 1093 1093 1093 186 187 BQAAAA BAOAAA HHHHxx +5817 9466 1 1 7 17 17 817 1817 817 5817 34 35 TPAAAA CAOAAA OOOOxx +1559 9467 1 3 9 19 59 559 1559 1559 1559 118 119 ZHAAAA DAOAAA VVVVxx +8405 9468 1 1 5 5 5 405 405 3405 8405 10 11 HLAAAA EAOAAA AAAAxx +9962 9469 0 2 2 2 62 962 1962 4962 9962 124 125 ETAAAA FAOAAA HHHHxx +9461 9470 1 1 1 1 61 461 1461 4461 9461 122 123 XZAAAA GAOAAA OOOOxx +3028 9471 0 0 8 8 28 28 1028 3028 3028 56 57 MMAAAA HAOAAA VVVVxx +6814 9472 0 2 4 14 14 814 814 1814 6814 28 29 CCAAAA IAOAAA AAAAxx +9587 9473 1 3 7 7 87 587 1587 4587 9587 174 175 TEAAAA JAOAAA HHHHxx +6863 9474 1 3 3 3 63 863 863 1863 6863 126 127 ZDAAAA KAOAAA OOOOxx +4963 9475 1 3 3 3 63 963 963 4963 4963 126 127 XIAAAA LAOAAA VVVVxx +7811 9476 1 3 1 11 11 811 1811 2811 7811 22 23 LOAAAA MAOAAA AAAAxx +7608 9477 0 0 8 8 8 608 1608 2608 7608 16 17 QGAAAA NAOAAA HHHHxx +5321 9478 1 1 1 1 21 321 1321 321 5321 42 43 RWAAAA OAOAAA OOOOxx +9971 9479 1 3 1 11 71 971 1971 4971 9971 142 143 NTAAAA PAOAAA VVVVxx +6161 9480 1 1 1 1 61 161 161 1161 6161 122 123 ZCAAAA QAOAAA AAAAxx +2181 9481 1 1 1 1 81 181 181 2181 2181 162 163 XFAAAA RAOAAA HHHHxx +3828 9482 0 0 8 8 28 828 1828 3828 3828 56 57 GRAAAA SAOAAA OOOOxx +348 9483 0 0 8 8 48 348 348 348 348 96 97 KNAAAA TAOAAA VVVVxx +5459 9484 1 3 9 19 59 459 1459 459 5459 118 119 ZBAAAA UAOAAA AAAAxx +9406 9485 0 2 6 6 6 406 1406 4406 9406 12 13 UXAAAA VAOAAA HHHHxx +9852 9486 0 0 2 12 52 852 1852 4852 9852 104 105 YOAAAA WAOAAA OOOOxx +3095 9487 1 3 5 15 95 95 1095 3095 3095 190 191 BPAAAA XAOAAA VVVVxx +5597 9488 1 1 7 17 97 597 1597 597 5597 194 195 HHAAAA YAOAAA AAAAxx +8841 9489 1 1 1 1 41 841 841 3841 8841 82 83 BCAAAA ZAOAAA HHHHxx +3536 9490 0 0 6 16 36 536 1536 3536 3536 72 73 AGAAAA ABOAAA OOOOxx +4009 9491 1 1 9 9 9 9 9 4009 4009 18 19 FYAAAA BBOAAA VVVVxx +7366 9492 0 2 6 6 66 366 1366 2366 7366 132 133 IXAAAA CBOAAA AAAAxx +7327 9493 1 3 7 7 27 327 1327 2327 7327 54 55 VVAAAA DBOAAA HHHHxx +1613 9494 1 1 3 13 13 613 1613 1613 1613 26 27 BKAAAA EBOAAA OOOOxx +8619 9495 1 3 9 19 19 619 619 3619 8619 38 39 NTAAAA FBOAAA VVVVxx +4880 9496 0 0 0 0 80 880 880 4880 4880 160 161 SFAAAA GBOAAA AAAAxx +1552 9497 0 0 2 12 52 552 1552 1552 1552 104 105 SHAAAA HBOAAA HHHHxx +7636 9498 0 0 6 16 36 636 1636 2636 7636 72 73 SHAAAA IBOAAA OOOOxx +8397 9499 1 1 7 17 97 397 397 3397 8397 194 195 ZKAAAA JBOAAA VVVVxx +6224 9500 0 0 4 4 24 224 224 1224 6224 48 49 KFAAAA KBOAAA AAAAxx +9102 9501 0 2 2 2 2 102 1102 4102 9102 4 5 CMAAAA LBOAAA HHHHxx +7906 9502 0 2 6 6 6 906 1906 2906 7906 12 13 CSAAAA MBOAAA OOOOxx +9467 9503 1 3 7 7 67 467 1467 4467 9467 134 135 DAAAAA NBOAAA VVVVxx +828 9504 0 0 8 8 28 828 828 828 828 56 57 WFAAAA OBOAAA AAAAxx +9585 9505 1 1 5 5 85 585 1585 4585 9585 170 171 REAAAA PBOAAA HHHHxx +925 9506 1 1 5 5 25 925 925 925 925 50 51 PJAAAA QBOAAA OOOOxx +7375 9507 1 3 5 15 75 375 1375 2375 7375 150 151 RXAAAA RBOAAA VVVVxx +4027 9508 1 3 7 7 27 27 27 4027 4027 54 55 XYAAAA SBOAAA AAAAxx +766 9509 0 2 6 6 66 766 766 766 766 132 133 MDAAAA TBOAAA HHHHxx +5633 9510 1 1 3 13 33 633 1633 633 5633 66 67 RIAAAA UBOAAA OOOOxx +5648 9511 0 0 8 8 48 648 1648 648 5648 96 97 GJAAAA VBOAAA VVVVxx +148 9512 0 0 8 8 48 148 148 148 148 96 97 SFAAAA WBOAAA AAAAxx +2072 9513 0 0 2 12 72 72 72 2072 2072 144 145 SBAAAA XBOAAA HHHHxx +431 9514 1 3 1 11 31 431 431 431 431 62 63 PQAAAA YBOAAA OOOOxx +1711 9515 1 3 1 11 11 711 1711 1711 1711 22 23 VNAAAA ZBOAAA VVVVxx +9378 9516 0 2 8 18 78 378 1378 4378 9378 156 157 SWAAAA ACOAAA AAAAxx +6776 9517 0 0 6 16 76 776 776 1776 6776 152 153 QAAAAA BCOAAA HHHHxx +6842 9518 0 2 2 2 42 842 842 1842 6842 84 85 EDAAAA CCOAAA OOOOxx +2656 9519 0 0 6 16 56 656 656 2656 2656 112 113 EYAAAA DCOAAA VVVVxx +3116 9520 0 0 6 16 16 116 1116 3116 3116 32 33 WPAAAA ECOAAA AAAAxx +7904 9521 0 0 4 4 4 904 1904 2904 7904 8 9 ASAAAA FCOAAA HHHHxx +3529 9522 1 1 9 9 29 529 1529 3529 3529 58 59 TFAAAA GCOAAA OOOOxx +3240 9523 0 0 0 0 40 240 1240 3240 3240 80 81 QUAAAA HCOAAA VVVVxx +5801 9524 1 1 1 1 1 801 1801 801 5801 2 3 DPAAAA ICOAAA AAAAxx +4090 9525 0 2 0 10 90 90 90 4090 4090 180 181 IBAAAA JCOAAA HHHHxx +7687 9526 1 3 7 7 87 687 1687 2687 7687 174 175 RJAAAA KCOAAA OOOOxx +9711 9527 1 3 1 11 11 711 1711 4711 9711 22 23 NJAAAA LCOAAA VVVVxx +4760 9528 0 0 0 0 60 760 760 4760 4760 120 121 CBAAAA MCOAAA AAAAxx +5524 9529 0 0 4 4 24 524 1524 524 5524 48 49 MEAAAA NCOAAA HHHHxx +2251 9530 1 3 1 11 51 251 251 2251 2251 102 103 PIAAAA OCOAAA OOOOxx +1511 9531 1 3 1 11 11 511 1511 1511 1511 22 23 DGAAAA PCOAAA VVVVxx +5991 9532 1 3 1 11 91 991 1991 991 5991 182 183 LWAAAA QCOAAA AAAAxx +7808 9533 0 0 8 8 8 808 1808 2808 7808 16 17 IOAAAA RCOAAA HHHHxx +8708 9534 0 0 8 8 8 708 708 3708 8708 16 17 YWAAAA SCOAAA OOOOxx +8939 9535 1 3 9 19 39 939 939 3939 8939 78 79 VFAAAA TCOAAA VVVVxx +4295 9536 1 3 5 15 95 295 295 4295 4295 190 191 FJAAAA UCOAAA AAAAxx +5905 9537 1 1 5 5 5 905 1905 905 5905 10 11 DTAAAA VCOAAA HHHHxx +2649 9538 1 1 9 9 49 649 649 2649 2649 98 99 XXAAAA WCOAAA OOOOxx +2347 9539 1 3 7 7 47 347 347 2347 2347 94 95 HMAAAA XCOAAA VVVVxx +6339 9540 1 3 9 19 39 339 339 1339 6339 78 79 VJAAAA YCOAAA AAAAxx +292 9541 0 0 2 12 92 292 292 292 292 184 185 GLAAAA ZCOAAA HHHHxx +9314 9542 0 2 4 14 14 314 1314 4314 9314 28 29 GUAAAA ADOAAA OOOOxx +6893 9543 1 1 3 13 93 893 893 1893 6893 186 187 DFAAAA BDOAAA VVVVxx +3970 9544 0 2 0 10 70 970 1970 3970 3970 140 141 SWAAAA CDOAAA AAAAxx +1652 9545 0 0 2 12 52 652 1652 1652 1652 104 105 OLAAAA DDOAAA HHHHxx +4326 9546 0 2 6 6 26 326 326 4326 4326 52 53 KKAAAA EDOAAA OOOOxx +7881 9547 1 1 1 1 81 881 1881 2881 7881 162 163 DRAAAA FDOAAA VVVVxx +5291 9548 1 3 1 11 91 291 1291 291 5291 182 183 NVAAAA GDOAAA AAAAxx +957 9549 1 1 7 17 57 957 957 957 957 114 115 VKAAAA HDOAAA HHHHxx +2313 9550 1 1 3 13 13 313 313 2313 2313 26 27 ZKAAAA IDOAAA OOOOxx +5463 9551 1 3 3 3 63 463 1463 463 5463 126 127 DCAAAA JDOAAA VVVVxx +1268 9552 0 0 8 8 68 268 1268 1268 1268 136 137 UWAAAA KDOAAA AAAAxx +5028 9553 0 0 8 8 28 28 1028 28 5028 56 57 KLAAAA LDOAAA HHHHxx +656 9554 0 0 6 16 56 656 656 656 656 112 113 GZAAAA MDOAAA OOOOxx +9274 9555 0 2 4 14 74 274 1274 4274 9274 148 149 SSAAAA NDOAAA VVVVxx +8217 9556 1 1 7 17 17 217 217 3217 8217 34 35 BEAAAA ODOAAA AAAAxx +2175 9557 1 3 5 15 75 175 175 2175 2175 150 151 RFAAAA PDOAAA HHHHxx +6028 9558 0 0 8 8 28 28 28 1028 6028 56 57 WXAAAA QDOAAA OOOOxx +7584 9559 0 0 4 4 84 584 1584 2584 7584 168 169 SFAAAA RDOAAA VVVVxx +4114 9560 0 2 4 14 14 114 114 4114 4114 28 29 GCAAAA SDOAAA AAAAxx +8894 9561 0 2 4 14 94 894 894 3894 8894 188 189 CEAAAA TDOAAA HHHHxx +781 9562 1 1 1 1 81 781 781 781 781 162 163 BEAAAA UDOAAA OOOOxx +133 9563 1 1 3 13 33 133 133 133 133 66 67 DFAAAA VDOAAA VVVVxx +7572 9564 0 0 2 12 72 572 1572 2572 7572 144 145 GFAAAA WDOAAA AAAAxx +8514 9565 0 2 4 14 14 514 514 3514 8514 28 29 MPAAAA XDOAAA HHHHxx +3352 9566 0 0 2 12 52 352 1352 3352 3352 104 105 YYAAAA YDOAAA OOOOxx +8098 9567 0 2 8 18 98 98 98 3098 8098 196 197 MZAAAA ZDOAAA VVVVxx +9116 9568 0 0 6 16 16 116 1116 4116 9116 32 33 QMAAAA AEOAAA AAAAxx +9444 9569 0 0 4 4 44 444 1444 4444 9444 88 89 GZAAAA BEOAAA HHHHxx +2590 9570 0 2 0 10 90 590 590 2590 2590 180 181 QVAAAA CEOAAA OOOOxx +7302 9571 0 2 2 2 2 302 1302 2302 7302 4 5 WUAAAA DEOAAA VVVVxx +7444 9572 0 0 4 4 44 444 1444 2444 7444 88 89 IAAAAA EEOAAA AAAAxx +8748 9573 0 0 8 8 48 748 748 3748 8748 96 97 MYAAAA FEOAAA HHHHxx +7615 9574 1 3 5 15 15 615 1615 2615 7615 30 31 XGAAAA GEOAAA OOOOxx +6090 9575 0 2 0 10 90 90 90 1090 6090 180 181 GAAAAA HEOAAA VVVVxx +1529 9576 1 1 9 9 29 529 1529 1529 1529 58 59 VGAAAA IEOAAA AAAAxx +9398 9577 0 2 8 18 98 398 1398 4398 9398 196 197 MXAAAA JEOAAA HHHHxx +6114 9578 0 2 4 14 14 114 114 1114 6114 28 29 EBAAAA KEOAAA OOOOxx +2736 9579 0 0 6 16 36 736 736 2736 2736 72 73 GBAAAA LEOAAA VVVVxx +468 9580 0 0 8 8 68 468 468 468 468 136 137 ASAAAA MEOAAA AAAAxx +1487 9581 1 3 7 7 87 487 1487 1487 1487 174 175 FFAAAA NEOAAA HHHHxx +4784 9582 0 0 4 4 84 784 784 4784 4784 168 169 ACAAAA OEOAAA OOOOxx +6731 9583 1 3 1 11 31 731 731 1731 6731 62 63 XYAAAA PEOAAA VVVVxx +3328 9584 0 0 8 8 28 328 1328 3328 3328 56 57 AYAAAA QEOAAA AAAAxx +6891 9585 1 3 1 11 91 891 891 1891 6891 182 183 BFAAAA REOAAA HHHHxx +8039 9586 1 3 9 19 39 39 39 3039 8039 78 79 FXAAAA SEOAAA OOOOxx +4064 9587 0 0 4 4 64 64 64 4064 4064 128 129 IAAAAA TEOAAA VVVVxx +542 9588 0 2 2 2 42 542 542 542 542 84 85 WUAAAA UEOAAA AAAAxx +1039 9589 1 3 9 19 39 39 1039 1039 1039 78 79 ZNAAAA VEOAAA HHHHxx +5603 9590 1 3 3 3 3 603 1603 603 5603 6 7 NHAAAA WEOAAA OOOOxx +6641 9591 1 1 1 1 41 641 641 1641 6641 82 83 LVAAAA XEOAAA VVVVxx +6307 9592 1 3 7 7 7 307 307 1307 6307 14 15 PIAAAA YEOAAA AAAAxx +5354 9593 0 2 4 14 54 354 1354 354 5354 108 109 YXAAAA ZEOAAA HHHHxx +7878 9594 0 2 8 18 78 878 1878 2878 7878 156 157 ARAAAA AFOAAA OOOOxx +6391 9595 1 3 1 11 91 391 391 1391 6391 182 183 VLAAAA BFOAAA VVVVxx +4575 9596 1 3 5 15 75 575 575 4575 4575 150 151 ZTAAAA CFOAAA AAAAxx +6644 9597 0 0 4 4 44 644 644 1644 6644 88 89 OVAAAA DFOAAA HHHHxx +5207 9598 1 3 7 7 7 207 1207 207 5207 14 15 HSAAAA EFOAAA OOOOxx +1736 9599 0 0 6 16 36 736 1736 1736 1736 72 73 UOAAAA FFOAAA VVVVxx +3547 9600 1 3 7 7 47 547 1547 3547 3547 94 95 LGAAAA GFOAAA AAAAxx +6647 9601 1 3 7 7 47 647 647 1647 6647 94 95 RVAAAA HFOAAA HHHHxx +4107 9602 1 3 7 7 7 107 107 4107 4107 14 15 ZBAAAA IFOAAA OOOOxx +8125 9603 1 1 5 5 25 125 125 3125 8125 50 51 NAAAAA JFOAAA VVVVxx +9223 9604 1 3 3 3 23 223 1223 4223 9223 46 47 TQAAAA KFOAAA AAAAxx +6903 9605 1 3 3 3 3 903 903 1903 6903 6 7 NFAAAA LFOAAA HHHHxx +3639 9606 1 3 9 19 39 639 1639 3639 3639 78 79 ZJAAAA MFOAAA OOOOxx +9606 9607 0 2 6 6 6 606 1606 4606 9606 12 13 MFAAAA NFOAAA VVVVxx +3232 9608 0 0 2 12 32 232 1232 3232 3232 64 65 IUAAAA OFOAAA AAAAxx +2063 9609 1 3 3 3 63 63 63 2063 2063 126 127 JBAAAA PFOAAA HHHHxx +3731 9610 1 3 1 11 31 731 1731 3731 3731 62 63 NNAAAA QFOAAA OOOOxx +2558 9611 0 2 8 18 58 558 558 2558 2558 116 117 KUAAAA RFOAAA VVVVxx +2357 9612 1 1 7 17 57 357 357 2357 2357 114 115 RMAAAA SFOAAA AAAAxx +6008 9613 0 0 8 8 8 8 8 1008 6008 16 17 CXAAAA TFOAAA HHHHxx +8246 9614 0 2 6 6 46 246 246 3246 8246 92 93 EFAAAA UFOAAA OOOOxx +8220 9615 0 0 0 0 20 220 220 3220 8220 40 41 EEAAAA VFOAAA VVVVxx +1075 9616 1 3 5 15 75 75 1075 1075 1075 150 151 JPAAAA WFOAAA AAAAxx +2410 9617 0 2 0 10 10 410 410 2410 2410 20 21 SOAAAA XFOAAA HHHHxx +3253 9618 1 1 3 13 53 253 1253 3253 3253 106 107 DVAAAA YFOAAA OOOOxx +4370 9619 0 2 0 10 70 370 370 4370 4370 140 141 CMAAAA ZFOAAA VVVVxx +8426 9620 0 2 6 6 26 426 426 3426 8426 52 53 CMAAAA AGOAAA AAAAxx +2262 9621 0 2 2 2 62 262 262 2262 2262 124 125 AJAAAA BGOAAA HHHHxx +4149 9622 1 1 9 9 49 149 149 4149 4149 98 99 PDAAAA CGOAAA OOOOxx +2732 9623 0 0 2 12 32 732 732 2732 2732 64 65 CBAAAA DGOAAA VVVVxx +8606 9624 0 2 6 6 6 606 606 3606 8606 12 13 ATAAAA EGOAAA AAAAxx +6311 9625 1 3 1 11 11 311 311 1311 6311 22 23 TIAAAA FGOAAA HHHHxx +7223 9626 1 3 3 3 23 223 1223 2223 7223 46 47 VRAAAA GGOAAA OOOOxx +3054 9627 0 2 4 14 54 54 1054 3054 3054 108 109 MNAAAA HGOAAA VVVVxx +3952 9628 0 0 2 12 52 952 1952 3952 3952 104 105 AWAAAA IGOAAA AAAAxx +8252 9629 0 0 2 12 52 252 252 3252 8252 104 105 KFAAAA JGOAAA HHHHxx +6020 9630 0 0 0 0 20 20 20 1020 6020 40 41 OXAAAA KGOAAA OOOOxx +3846 9631 0 2 6 6 46 846 1846 3846 3846 92 93 YRAAAA LGOAAA VVVVxx +3755 9632 1 3 5 15 55 755 1755 3755 3755 110 111 LOAAAA MGOAAA AAAAxx +3765 9633 1 1 5 5 65 765 1765 3765 3765 130 131 VOAAAA NGOAAA HHHHxx +3434 9634 0 2 4 14 34 434 1434 3434 3434 68 69 CCAAAA OGOAAA OOOOxx +1381 9635 1 1 1 1 81 381 1381 1381 1381 162 163 DBAAAA PGOAAA VVVVxx +287 9636 1 3 7 7 87 287 287 287 287 174 175 BLAAAA QGOAAA AAAAxx +4476 9637 0 0 6 16 76 476 476 4476 4476 152 153 EQAAAA RGOAAA HHHHxx +2916 9638 0 0 6 16 16 916 916 2916 2916 32 33 EIAAAA SGOAAA OOOOxx +4517 9639 1 1 7 17 17 517 517 4517 4517 34 35 TRAAAA TGOAAA VVVVxx +4561 9640 1 1 1 1 61 561 561 4561 4561 122 123 LTAAAA UGOAAA AAAAxx +5106 9641 0 2 6 6 6 106 1106 106 5106 12 13 KOAAAA VGOAAA HHHHxx +2077 9642 1 1 7 17 77 77 77 2077 2077 154 155 XBAAAA WGOAAA OOOOxx +5269 9643 1 1 9 9 69 269 1269 269 5269 138 139 RUAAAA XGOAAA VVVVxx +5688 9644 0 0 8 8 88 688 1688 688 5688 176 177 UKAAAA YGOAAA AAAAxx +8831 9645 1 3 1 11 31 831 831 3831 8831 62 63 RBAAAA ZGOAAA HHHHxx +3867 9646 1 3 7 7 67 867 1867 3867 3867 134 135 TSAAAA AHOAAA OOOOxx +6062 9647 0 2 2 2 62 62 62 1062 6062 124 125 EZAAAA BHOAAA VVVVxx +8460 9648 0 0 0 0 60 460 460 3460 8460 120 121 KNAAAA CHOAAA AAAAxx +3138 9649 0 2 8 18 38 138 1138 3138 3138 76 77 SQAAAA DHOAAA HHHHxx +3173 9650 1 1 3 13 73 173 1173 3173 3173 146 147 BSAAAA EHOAAA OOOOxx +7018 9651 0 2 8 18 18 18 1018 2018 7018 36 37 YJAAAA FHOAAA VVVVxx +4836 9652 0 0 6 16 36 836 836 4836 4836 72 73 AEAAAA GHOAAA AAAAxx +1007 9653 1 3 7 7 7 7 1007 1007 1007 14 15 TMAAAA HHOAAA HHHHxx +658 9654 0 2 8 18 58 658 658 658 658 116 117 IZAAAA IHOAAA OOOOxx +5205 9655 1 1 5 5 5 205 1205 205 5205 10 11 FSAAAA JHOAAA VVVVxx +5805 9656 1 1 5 5 5 805 1805 805 5805 10 11 HPAAAA KHOAAA AAAAxx +5959 9657 1 3 9 19 59 959 1959 959 5959 118 119 FVAAAA LHOAAA HHHHxx +2863 9658 1 3 3 3 63 863 863 2863 2863 126 127 DGAAAA MHOAAA OOOOxx +7272 9659 0 0 2 12 72 272 1272 2272 7272 144 145 STAAAA NHOAAA VVVVxx +8437 9660 1 1 7 17 37 437 437 3437 8437 74 75 NMAAAA OHOAAA AAAAxx +4900 9661 0 0 0 0 0 900 900 4900 4900 0 1 MGAAAA PHOAAA HHHHxx +890 9662 0 2 0 10 90 890 890 890 890 180 181 GIAAAA QHOAAA OOOOxx +3530 9663 0 2 0 10 30 530 1530 3530 3530 60 61 UFAAAA RHOAAA VVVVxx +6209 9664 1 1 9 9 9 209 209 1209 6209 18 19 VEAAAA SHOAAA AAAAxx +4595 9665 1 3 5 15 95 595 595 4595 4595 190 191 TUAAAA THOAAA HHHHxx +5982 9666 0 2 2 2 82 982 1982 982 5982 164 165 CWAAAA UHOAAA OOOOxx +1101 9667 1 1 1 1 1 101 1101 1101 1101 2 3 JQAAAA VHOAAA VVVVxx +9555 9668 1 3 5 15 55 555 1555 4555 9555 110 111 NDAAAA WHOAAA AAAAxx +1918 9669 0 2 8 18 18 918 1918 1918 1918 36 37 UVAAAA XHOAAA HHHHxx +3527 9670 1 3 7 7 27 527 1527 3527 3527 54 55 RFAAAA YHOAAA OOOOxx +7309 9671 1 1 9 9 9 309 1309 2309 7309 18 19 DVAAAA ZHOAAA VVVVxx +8213 9672 1 1 3 13 13 213 213 3213 8213 26 27 XDAAAA AIOAAA AAAAxx +306 9673 0 2 6 6 6 306 306 306 306 12 13 ULAAAA BIOAAA HHHHxx +845 9674 1 1 5 5 45 845 845 845 845 90 91 NGAAAA CIOAAA OOOOxx +16 9675 0 0 6 16 16 16 16 16 16 32 33 QAAAAA DIOAAA VVVVxx +437 9676 1 1 7 17 37 437 437 437 437 74 75 VQAAAA EIOAAA AAAAxx +9518 9677 0 2 8 18 18 518 1518 4518 9518 36 37 CCAAAA FIOAAA HHHHxx +2142 9678 0 2 2 2 42 142 142 2142 2142 84 85 KEAAAA GIOAAA OOOOxx +8121 9679 1 1 1 1 21 121 121 3121 8121 42 43 JAAAAA HIOAAA VVVVxx +7354 9680 0 2 4 14 54 354 1354 2354 7354 108 109 WWAAAA IIOAAA AAAAxx +1720 9681 0 0 0 0 20 720 1720 1720 1720 40 41 EOAAAA JIOAAA HHHHxx +6078 9682 0 2 8 18 78 78 78 1078 6078 156 157 UZAAAA KIOAAA OOOOxx +5929 9683 1 1 9 9 29 929 1929 929 5929 58 59 BUAAAA LIOAAA VVVVxx +3856 9684 0 0 6 16 56 856 1856 3856 3856 112 113 ISAAAA MIOAAA AAAAxx +3424 9685 0 0 4 4 24 424 1424 3424 3424 48 49 SBAAAA NIOAAA HHHHxx +1712 9686 0 0 2 12 12 712 1712 1712 1712 24 25 WNAAAA OIOAAA OOOOxx +2340 9687 0 0 0 0 40 340 340 2340 2340 80 81 AMAAAA PIOAAA VVVVxx +5570 9688 0 2 0 10 70 570 1570 570 5570 140 141 GGAAAA QIOAAA AAAAxx +8734 9689 0 2 4 14 34 734 734 3734 8734 68 69 YXAAAA RIOAAA HHHHxx +6077 9690 1 1 7 17 77 77 77 1077 6077 154 155 TZAAAA SIOAAA OOOOxx +2960 9691 0 0 0 0 60 960 960 2960 2960 120 121 WJAAAA TIOAAA VVVVxx +5062 9692 0 2 2 2 62 62 1062 62 5062 124 125 SMAAAA UIOAAA AAAAxx +1532 9693 0 0 2 12 32 532 1532 1532 1532 64 65 YGAAAA VIOAAA HHHHxx +8298 9694 0 2 8 18 98 298 298 3298 8298 196 197 EHAAAA WIOAAA OOOOxx +2496 9695 0 0 6 16 96 496 496 2496 2496 192 193 ASAAAA XIOAAA VVVVxx +8412 9696 0 0 2 12 12 412 412 3412 8412 24 25 OLAAAA YIOAAA AAAAxx +724 9697 0 0 4 4 24 724 724 724 724 48 49 WBAAAA ZIOAAA HHHHxx +1019 9698 1 3 9 19 19 19 1019 1019 1019 38 39 FNAAAA AJOAAA OOOOxx +6265 9699 1 1 5 5 65 265 265 1265 6265 130 131 ZGAAAA BJOAAA VVVVxx +740 9700 0 0 0 0 40 740 740 740 740 80 81 MCAAAA CJOAAA AAAAxx +8495 9701 1 3 5 15 95 495 495 3495 8495 190 191 TOAAAA DJOAAA HHHHxx +6983 9702 1 3 3 3 83 983 983 1983 6983 166 167 PIAAAA EJOAAA OOOOxx +991 9703 1 3 1 11 91 991 991 991 991 182 183 DMAAAA FJOAAA VVVVxx +3189 9704 1 1 9 9 89 189 1189 3189 3189 178 179 RSAAAA GJOAAA AAAAxx +4487 9705 1 3 7 7 87 487 487 4487 4487 174 175 PQAAAA HJOAAA HHHHxx +5554 9706 0 2 4 14 54 554 1554 554 5554 108 109 QFAAAA IJOAAA OOOOxx +1258 9707 0 2 8 18 58 258 1258 1258 1258 116 117 KWAAAA JJOAAA VVVVxx +5359 9708 1 3 9 19 59 359 1359 359 5359 118 119 DYAAAA KJOAAA AAAAxx +2709 9709 1 1 9 9 9 709 709 2709 2709 18 19 FAAAAA LJOAAA HHHHxx +361 9710 1 1 1 1 61 361 361 361 361 122 123 XNAAAA MJOAAA OOOOxx +4028 9711 0 0 8 8 28 28 28 4028 4028 56 57 YYAAAA NJOAAA VVVVxx +3735 9712 1 3 5 15 35 735 1735 3735 3735 70 71 RNAAAA OJOAAA AAAAxx +4427 9713 1 3 7 7 27 427 427 4427 4427 54 55 HOAAAA PJOAAA HHHHxx +7540 9714 0 0 0 0 40 540 1540 2540 7540 80 81 AEAAAA QJOAAA OOOOxx +3569 9715 1 1 9 9 69 569 1569 3569 3569 138 139 HHAAAA RJOAAA VVVVxx +1916 9716 0 0 6 16 16 916 1916 1916 1916 32 33 SVAAAA SJOAAA AAAAxx +7596 9717 0 0 6 16 96 596 1596 2596 7596 192 193 EGAAAA TJOAAA HHHHxx +9721 9718 1 1 1 1 21 721 1721 4721 9721 42 43 XJAAAA UJOAAA OOOOxx +4429 9719 1 1 9 9 29 429 429 4429 4429 58 59 JOAAAA VJOAAA VVVVxx +3471 9720 1 3 1 11 71 471 1471 3471 3471 142 143 NDAAAA WJOAAA AAAAxx +1157 9721 1 1 7 17 57 157 1157 1157 1157 114 115 NSAAAA XJOAAA HHHHxx +5700 9722 0 0 0 0 0 700 1700 700 5700 0 1 GLAAAA YJOAAA OOOOxx +4431 9723 1 3 1 11 31 431 431 4431 4431 62 63 LOAAAA ZJOAAA VVVVxx +9409 9724 1 1 9 9 9 409 1409 4409 9409 18 19 XXAAAA AKOAAA AAAAxx +8752 9725 0 0 2 12 52 752 752 3752 8752 104 105 QYAAAA BKOAAA HHHHxx +9484 9726 0 0 4 4 84 484 1484 4484 9484 168 169 UAAAAA CKOAAA OOOOxx +1266 9727 0 2 6 6 66 266 1266 1266 1266 132 133 SWAAAA DKOAAA VVVVxx +9097 9728 1 1 7 17 97 97 1097 4097 9097 194 195 XLAAAA EKOAAA AAAAxx +3068 9729 0 0 8 8 68 68 1068 3068 3068 136 137 AOAAAA FKOAAA HHHHxx +5490 9730 0 2 0 10 90 490 1490 490 5490 180 181 EDAAAA GKOAAA OOOOxx +1375 9731 1 3 5 15 75 375 1375 1375 1375 150 151 XAAAAA HKOAAA VVVVxx +2487 9732 1 3 7 7 87 487 487 2487 2487 174 175 RRAAAA IKOAAA AAAAxx +1705 9733 1 1 5 5 5 705 1705 1705 1705 10 11 PNAAAA JKOAAA HHHHxx +1571 9734 1 3 1 11 71 571 1571 1571 1571 142 143 LIAAAA KKOAAA OOOOxx +4005 9735 1 1 5 5 5 5 5 4005 4005 10 11 BYAAAA LKOAAA VVVVxx +5497 9736 1 1 7 17 97 497 1497 497 5497 194 195 LDAAAA MKOAAA AAAAxx +2144 9737 0 0 4 4 44 144 144 2144 2144 88 89 MEAAAA NKOAAA HHHHxx +4052 9738 0 0 2 12 52 52 52 4052 4052 104 105 WZAAAA OKOAAA OOOOxx +4942 9739 0 2 2 2 42 942 942 4942 4942 84 85 CIAAAA PKOAAA VVVVxx +5504 9740 0 0 4 4 4 504 1504 504 5504 8 9 SDAAAA QKOAAA AAAAxx +2913 9741 1 1 3 13 13 913 913 2913 2913 26 27 BIAAAA RKOAAA HHHHxx +5617 9742 1 1 7 17 17 617 1617 617 5617 34 35 BIAAAA SKOAAA OOOOxx +8179 9743 1 3 9 19 79 179 179 3179 8179 158 159 PCAAAA TKOAAA VVVVxx +9437 9744 1 1 7 17 37 437 1437 4437 9437 74 75 ZYAAAA UKOAAA AAAAxx +1821 9745 1 1 1 1 21 821 1821 1821 1821 42 43 BSAAAA VKOAAA HHHHxx +5737 9746 1 1 7 17 37 737 1737 737 5737 74 75 RMAAAA WKOAAA OOOOxx +4207 9747 1 3 7 7 7 207 207 4207 4207 14 15 VFAAAA XKOAAA VVVVxx +4815 9748 1 3 5 15 15 815 815 4815 4815 30 31 FDAAAA YKOAAA AAAAxx +8707 9749 1 3 7 7 7 707 707 3707 8707 14 15 XWAAAA ZKOAAA HHHHxx +5970 9750 0 2 0 10 70 970 1970 970 5970 140 141 QVAAAA ALOAAA OOOOxx +5501 9751 1 1 1 1 1 501 1501 501 5501 2 3 PDAAAA BLOAAA VVVVxx +4013 9752 1 1 3 13 13 13 13 4013 4013 26 27 JYAAAA CLOAAA AAAAxx +9235 9753 1 3 5 15 35 235 1235 4235 9235 70 71 FRAAAA DLOAAA HHHHxx +2503 9754 1 3 3 3 3 503 503 2503 2503 6 7 HSAAAA ELOAAA OOOOxx +9181 9755 1 1 1 1 81 181 1181 4181 9181 162 163 DPAAAA FLOAAA VVVVxx +2289 9756 1 1 9 9 89 289 289 2289 2289 178 179 BKAAAA GLOAAA AAAAxx +4256 9757 0 0 6 16 56 256 256 4256 4256 112 113 SHAAAA HLOAAA HHHHxx +191 9758 1 3 1 11 91 191 191 191 191 182 183 JHAAAA ILOAAA OOOOxx +9655 9759 1 3 5 15 55 655 1655 4655 9655 110 111 JHAAAA JLOAAA VVVVxx +8615 9760 1 3 5 15 15 615 615 3615 8615 30 31 JTAAAA KLOAAA AAAAxx +3011 9761 1 3 1 11 11 11 1011 3011 3011 22 23 VLAAAA LLOAAA HHHHxx +6376 9762 0 0 6 16 76 376 376 1376 6376 152 153 GLAAAA MLOAAA OOOOxx +68 9763 0 0 8 8 68 68 68 68 68 136 137 QCAAAA NLOAAA VVVVxx +4720 9764 0 0 0 0 20 720 720 4720 4720 40 41 OZAAAA OLOAAA AAAAxx +6848 9765 0 0 8 8 48 848 848 1848 6848 96 97 KDAAAA PLOAAA HHHHxx +456 9766 0 0 6 16 56 456 456 456 456 112 113 ORAAAA QLOAAA OOOOxx +5887 9767 1 3 7 7 87 887 1887 887 5887 174 175 LSAAAA RLOAAA VVVVxx +9249 9768 1 1 9 9 49 249 1249 4249 9249 98 99 TRAAAA SLOAAA AAAAxx +4041 9769 1 1 1 1 41 41 41 4041 4041 82 83 LZAAAA TLOAAA HHHHxx +2304 9770 0 0 4 4 4 304 304 2304 2304 8 9 QKAAAA ULOAAA OOOOxx +8763 9771 1 3 3 3 63 763 763 3763 8763 126 127 BZAAAA VLOAAA VVVVxx +2115 9772 1 3 5 15 15 115 115 2115 2115 30 31 JDAAAA WLOAAA AAAAxx +8014 9773 0 2 4 14 14 14 14 3014 8014 28 29 GWAAAA XLOAAA HHHHxx +9895 9774 1 3 5 15 95 895 1895 4895 9895 190 191 PQAAAA YLOAAA OOOOxx +671 9775 1 3 1 11 71 671 671 671 671 142 143 VZAAAA ZLOAAA VVVVxx +3774 9776 0 2 4 14 74 774 1774 3774 3774 148 149 EPAAAA AMOAAA AAAAxx +134 9777 0 2 4 14 34 134 134 134 134 68 69 EFAAAA BMOAAA HHHHxx +534 9778 0 2 4 14 34 534 534 534 534 68 69 OUAAAA CMOAAA OOOOxx +7308 9779 0 0 8 8 8 308 1308 2308 7308 16 17 CVAAAA DMOAAA VVVVxx +5244 9780 0 0 4 4 44 244 1244 244 5244 88 89 STAAAA EMOAAA AAAAxx +1512 9781 0 0 2 12 12 512 1512 1512 1512 24 25 EGAAAA FMOAAA HHHHxx +8960 9782 0 0 0 0 60 960 960 3960 8960 120 121 QGAAAA GMOAAA OOOOxx +6602 9783 0 2 2 2 2 602 602 1602 6602 4 5 YTAAAA HMOAAA VVVVxx +593 9784 1 1 3 13 93 593 593 593 593 186 187 VWAAAA IMOAAA AAAAxx +2353 9785 1 1 3 13 53 353 353 2353 2353 106 107 NMAAAA JMOAAA HHHHxx +4139 9786 1 3 9 19 39 139 139 4139 4139 78 79 FDAAAA KMOAAA OOOOxx +3063 9787 1 3 3 3 63 63 1063 3063 3063 126 127 VNAAAA LMOAAA VVVVxx +652 9788 0 0 2 12 52 652 652 652 652 104 105 CZAAAA MMOAAA AAAAxx +7405 9789 1 1 5 5 5 405 1405 2405 7405 10 11 VYAAAA NMOAAA HHHHxx +3034 9790 0 2 4 14 34 34 1034 3034 3034 68 69 SMAAAA OMOAAA OOOOxx +4614 9791 0 2 4 14 14 614 614 4614 4614 28 29 MVAAAA PMOAAA VVVVxx +2351 9792 1 3 1 11 51 351 351 2351 2351 102 103 LMAAAA QMOAAA AAAAxx +8208 9793 0 0 8 8 8 208 208 3208 8208 16 17 SDAAAA RMOAAA HHHHxx +5475 9794 1 3 5 15 75 475 1475 475 5475 150 151 PCAAAA SMOAAA OOOOxx +6875 9795 1 3 5 15 75 875 875 1875 6875 150 151 LEAAAA TMOAAA VVVVxx +563 9796 1 3 3 3 63 563 563 563 563 126 127 RVAAAA UMOAAA AAAAxx +3346 9797 0 2 6 6 46 346 1346 3346 3346 92 93 SYAAAA VMOAAA HHHHxx +291 9798 1 3 1 11 91 291 291 291 291 182 183 FLAAAA WMOAAA OOOOxx +6345 9799 1 1 5 5 45 345 345 1345 6345 90 91 BKAAAA XMOAAA VVVVxx +8099 9800 1 3 9 19 99 99 99 3099 8099 198 199 NZAAAA YMOAAA AAAAxx +2078 9801 0 2 8 18 78 78 78 2078 2078 156 157 YBAAAA ZMOAAA HHHHxx +8238 9802 0 2 8 18 38 238 238 3238 8238 76 77 WEAAAA ANOAAA OOOOxx +4482 9803 0 2 2 2 82 482 482 4482 4482 164 165 KQAAAA BNOAAA VVVVxx +716 9804 0 0 6 16 16 716 716 716 716 32 33 OBAAAA CNOAAA AAAAxx +7288 9805 0 0 8 8 88 288 1288 2288 7288 176 177 IUAAAA DNOAAA HHHHxx +5906 9806 0 2 6 6 6 906 1906 906 5906 12 13 ETAAAA ENOAAA OOOOxx +5618 9807 0 2 8 18 18 618 1618 618 5618 36 37 CIAAAA FNOAAA VVVVxx +1141 9808 1 1 1 1 41 141 1141 1141 1141 82 83 XRAAAA GNOAAA AAAAxx +8231 9809 1 3 1 11 31 231 231 3231 8231 62 63 PEAAAA HNOAAA HHHHxx +3713 9810 1 1 3 13 13 713 1713 3713 3713 26 27 VMAAAA INOAAA OOOOxx +9158 9811 0 2 8 18 58 158 1158 4158 9158 116 117 GOAAAA JNOAAA VVVVxx +4051 9812 1 3 1 11 51 51 51 4051 4051 102 103 VZAAAA KNOAAA AAAAxx +1973 9813 1 1 3 13 73 973 1973 1973 1973 146 147 XXAAAA LNOAAA HHHHxx +6710 9814 0 2 0 10 10 710 710 1710 6710 20 21 CYAAAA MNOAAA OOOOxx +1021 9815 1 1 1 1 21 21 1021 1021 1021 42 43 HNAAAA NNOAAA VVVVxx +2196 9816 0 0 6 16 96 196 196 2196 2196 192 193 MGAAAA ONOAAA AAAAxx +8335 9817 1 3 5 15 35 335 335 3335 8335 70 71 PIAAAA PNOAAA HHHHxx +2272 9818 0 0 2 12 72 272 272 2272 2272 144 145 KJAAAA QNOAAA OOOOxx +3818 9819 0 2 8 18 18 818 1818 3818 3818 36 37 WQAAAA RNOAAA VVVVxx +679 9820 1 3 9 19 79 679 679 679 679 158 159 DAAAAA SNOAAA AAAAxx +7512 9821 0 0 2 12 12 512 1512 2512 7512 24 25 YCAAAA TNOAAA HHHHxx +493 9822 1 1 3 13 93 493 493 493 493 186 187 ZSAAAA UNOAAA OOOOxx +5663 9823 1 3 3 3 63 663 1663 663 5663 126 127 VJAAAA VNOAAA VVVVxx +4655 9824 1 3 5 15 55 655 655 4655 4655 110 111 BXAAAA WNOAAA AAAAxx +3996 9825 0 0 6 16 96 996 1996 3996 3996 192 193 SXAAAA XNOAAA HHHHxx +8797 9826 1 1 7 17 97 797 797 3797 8797 194 195 JAAAAA YNOAAA OOOOxx +2991 9827 1 3 1 11 91 991 991 2991 2991 182 183 BLAAAA ZNOAAA VVVVxx +7038 9828 0 2 8 18 38 38 1038 2038 7038 76 77 SKAAAA AOOAAA AAAAxx +4174 9829 0 2 4 14 74 174 174 4174 4174 148 149 OEAAAA BOOAAA HHHHxx +6908 9830 0 0 8 8 8 908 908 1908 6908 16 17 SFAAAA COOAAA OOOOxx +8477 9831 1 1 7 17 77 477 477 3477 8477 154 155 BOAAAA DOOAAA VVVVxx +3576 9832 0 0 6 16 76 576 1576 3576 3576 152 153 OHAAAA EOOAAA AAAAxx +2685 9833 1 1 5 5 85 685 685 2685 2685 170 171 HZAAAA FOOAAA HHHHxx +9161 9834 1 1 1 1 61 161 1161 4161 9161 122 123 JOAAAA GOOAAA OOOOxx +2951 9835 1 3 1 11 51 951 951 2951 2951 102 103 NJAAAA HOOAAA VVVVxx +8362 9836 0 2 2 2 62 362 362 3362 8362 124 125 QJAAAA IOOAAA AAAAxx +2379 9837 1 3 9 19 79 379 379 2379 2379 158 159 NNAAAA JOOAAA HHHHxx +1277 9838 1 1 7 17 77 277 1277 1277 1277 154 155 DXAAAA KOOAAA OOOOxx +1728 9839 0 0 8 8 28 728 1728 1728 1728 56 57 MOAAAA LOOAAA VVVVxx +9816 9840 0 0 6 16 16 816 1816 4816 9816 32 33 ONAAAA MOOAAA AAAAxx +6288 9841 0 0 8 8 88 288 288 1288 6288 176 177 WHAAAA NOOAAA HHHHxx +8985 9842 1 1 5 5 85 985 985 3985 8985 170 171 PHAAAA OOOAAA OOOOxx +771 9843 1 3 1 11 71 771 771 771 771 142 143 RDAAAA POOAAA VVVVxx +464 9844 0 0 4 4 64 464 464 464 464 128 129 WRAAAA QOOAAA AAAAxx +9625 9845 1 1 5 5 25 625 1625 4625 9625 50 51 FGAAAA ROOAAA HHHHxx +9608 9846 0 0 8 8 8 608 1608 4608 9608 16 17 OFAAAA SOOAAA OOOOxx +9170 9847 0 2 0 10 70 170 1170 4170 9170 140 141 SOAAAA TOOAAA VVVVxx +9658 9848 0 2 8 18 58 658 1658 4658 9658 116 117 MHAAAA UOOAAA AAAAxx +7515 9849 1 3 5 15 15 515 1515 2515 7515 30 31 BDAAAA VOOAAA HHHHxx +9400 9850 0 0 0 0 0 400 1400 4400 9400 0 1 OXAAAA WOOAAA OOOOxx +2045 9851 1 1 5 5 45 45 45 2045 2045 90 91 RAAAAA XOOAAA VVVVxx +324 9852 0 0 4 4 24 324 324 324 324 48 49 MMAAAA YOOAAA AAAAxx +4252 9853 0 0 2 12 52 252 252 4252 4252 104 105 OHAAAA ZOOAAA HHHHxx +8329 9854 1 1 9 9 29 329 329 3329 8329 58 59 JIAAAA APOAAA OOOOxx +4472 9855 0 0 2 12 72 472 472 4472 4472 144 145 AQAAAA BPOAAA VVVVxx +1047 9856 1 3 7 7 47 47 1047 1047 1047 94 95 HOAAAA CPOAAA AAAAxx +9341 9857 1 1 1 1 41 341 1341 4341 9341 82 83 HVAAAA DPOAAA HHHHxx +7000 9858 0 0 0 0 0 0 1000 2000 7000 0 1 GJAAAA EPOAAA OOOOxx +1429 9859 1 1 9 9 29 429 1429 1429 1429 58 59 ZCAAAA FPOAAA VVVVxx +2701 9860 1 1 1 1 1 701 701 2701 2701 2 3 XZAAAA GPOAAA AAAAxx +6630 9861 0 2 0 10 30 630 630 1630 6630 60 61 AVAAAA HPOAAA HHHHxx +3669 9862 1 1 9 9 69 669 1669 3669 3669 138 139 DLAAAA IPOAAA OOOOxx +8613 9863 1 1 3 13 13 613 613 3613 8613 26 27 HTAAAA JPOAAA VVVVxx +7080 9864 0 0 0 0 80 80 1080 2080 7080 160 161 IMAAAA KPOAAA AAAAxx +8788 9865 0 0 8 8 88 788 788 3788 8788 176 177 AAAAAA LPOAAA HHHHxx +6291 9866 1 3 1 11 91 291 291 1291 6291 182 183 ZHAAAA MPOAAA OOOOxx +7885 9867 1 1 5 5 85 885 1885 2885 7885 170 171 HRAAAA NPOAAA VVVVxx +7160 9868 0 0 0 0 60 160 1160 2160 7160 120 121 KPAAAA OPOAAA AAAAxx +6140 9869 0 0 0 0 40 140 140 1140 6140 80 81 ECAAAA PPOAAA HHHHxx +9881 9870 1 1 1 1 81 881 1881 4881 9881 162 163 BQAAAA QPOAAA OOOOxx +9140 9871 0 0 0 0 40 140 1140 4140 9140 80 81 ONAAAA RPOAAA VVVVxx +644 9872 0 0 4 4 44 644 644 644 644 88 89 UYAAAA SPOAAA AAAAxx +3667 9873 1 3 7 7 67 667 1667 3667 3667 134 135 BLAAAA TPOAAA HHHHxx +2675 9874 1 3 5 15 75 675 675 2675 2675 150 151 XYAAAA UPOAAA OOOOxx +9492 9875 0 0 2 12 92 492 1492 4492 9492 184 185 CBAAAA VPOAAA VVVVxx +5004 9876 0 0 4 4 4 4 1004 4 5004 8 9 MKAAAA WPOAAA AAAAxx +9456 9877 0 0 6 16 56 456 1456 4456 9456 112 113 SZAAAA XPOAAA HHHHxx +8197 9878 1 1 7 17 97 197 197 3197 8197 194 195 HDAAAA YPOAAA OOOOxx +2837 9879 1 1 7 17 37 837 837 2837 2837 74 75 DFAAAA ZPOAAA VVVVxx +127 9880 1 3 7 7 27 127 127 127 127 54 55 XEAAAA AQOAAA AAAAxx +9772 9881 0 0 2 12 72 772 1772 4772 9772 144 145 WLAAAA BQOAAA HHHHxx +5743 9882 1 3 3 3 43 743 1743 743 5743 86 87 XMAAAA CQOAAA OOOOxx +2007 9883 1 3 7 7 7 7 7 2007 2007 14 15 FZAAAA DQOAAA VVVVxx +7586 9884 0 2 6 6 86 586 1586 2586 7586 172 173 UFAAAA EQOAAA AAAAxx +45 9885 1 1 5 5 45 45 45 45 45 90 91 TBAAAA FQOAAA HHHHxx +6482 9886 0 2 2 2 82 482 482 1482 6482 164 165 IPAAAA GQOAAA OOOOxx +4565 9887 1 1 5 5 65 565 565 4565 4565 130 131 PTAAAA HQOAAA VVVVxx +6975 9888 1 3 5 15 75 975 975 1975 6975 150 151 HIAAAA IQOAAA AAAAxx +7260 9889 0 0 0 0 60 260 1260 2260 7260 120 121 GTAAAA JQOAAA HHHHxx +2830 9890 0 2 0 10 30 830 830 2830 2830 60 61 WEAAAA KQOAAA OOOOxx +9365 9891 1 1 5 5 65 365 1365 4365 9365 130 131 FWAAAA LQOAAA VVVVxx +8207 9892 1 3 7 7 7 207 207 3207 8207 14 15 RDAAAA MQOAAA AAAAxx +2506 9893 0 2 6 6 6 506 506 2506 2506 12 13 KSAAAA NQOAAA HHHHxx +8081 9894 1 1 1 1 81 81 81 3081 8081 162 163 VYAAAA OQOAAA OOOOxx +8678 9895 0 2 8 18 78 678 678 3678 8678 156 157 UVAAAA PQOAAA VVVVxx +9932 9896 0 0 2 12 32 932 1932 4932 9932 64 65 ASAAAA QQOAAA AAAAxx +447 9897 1 3 7 7 47 447 447 447 447 94 95 FRAAAA RQOAAA HHHHxx +9187 9898 1 3 7 7 87 187 1187 4187 9187 174 175 JPAAAA SQOAAA OOOOxx +89 9899 1 1 9 9 89 89 89 89 89 178 179 LDAAAA TQOAAA VVVVxx +7027 9900 1 3 7 7 27 27 1027 2027 7027 54 55 HKAAAA UQOAAA AAAAxx +1536 9901 0 0 6 16 36 536 1536 1536 1536 72 73 CHAAAA VQOAAA HHHHxx +160 9902 0 0 0 0 60 160 160 160 160 120 121 EGAAAA WQOAAA OOOOxx +7679 9903 1 3 9 19 79 679 1679 2679 7679 158 159 JJAAAA XQOAAA VVVVxx +5973 9904 1 1 3 13 73 973 1973 973 5973 146 147 TVAAAA YQOAAA AAAAxx +4401 9905 1 1 1 1 1 401 401 4401 4401 2 3 HNAAAA ZQOAAA HHHHxx +395 9906 1 3 5 15 95 395 395 395 395 190 191 FPAAAA AROAAA OOOOxx +4904 9907 0 0 4 4 4 904 904 4904 4904 8 9 QGAAAA BROAAA VVVVxx +2759 9908 1 3 9 19 59 759 759 2759 2759 118 119 DCAAAA CROAAA AAAAxx +8713 9909 1 1 3 13 13 713 713 3713 8713 26 27 DXAAAA DROAAA HHHHxx +3770 9910 0 2 0 10 70 770 1770 3770 3770 140 141 APAAAA EROAAA OOOOxx +8272 9911 0 0 2 12 72 272 272 3272 8272 144 145 EGAAAA FROAAA VVVVxx +5358 9912 0 2 8 18 58 358 1358 358 5358 116 117 CYAAAA GROAAA AAAAxx +9747 9913 1 3 7 7 47 747 1747 4747 9747 94 95 XKAAAA HROAAA HHHHxx +1567 9914 1 3 7 7 67 567 1567 1567 1567 134 135 HIAAAA IROAAA OOOOxx +2136 9915 0 0 6 16 36 136 136 2136 2136 72 73 EEAAAA JROAAA VVVVxx +314 9916 0 2 4 14 14 314 314 314 314 28 29 CMAAAA KROAAA AAAAxx +4583 9917 1 3 3 3 83 583 583 4583 4583 166 167 HUAAAA LROAAA HHHHxx +375 9918 1 3 5 15 75 375 375 375 375 150 151 LOAAAA MROAAA OOOOxx +5566 9919 0 2 6 6 66 566 1566 566 5566 132 133 CGAAAA NROAAA VVVVxx +6865 9920 1 1 5 5 65 865 865 1865 6865 130 131 BEAAAA OROAAA AAAAxx +894 9921 0 2 4 14 94 894 894 894 894 188 189 KIAAAA PROAAA HHHHxx +5399 9922 1 3 9 19 99 399 1399 399 5399 198 199 RZAAAA QROAAA OOOOxx +1385 9923 1 1 5 5 85 385 1385 1385 1385 170 171 HBAAAA RROAAA VVVVxx +2156 9924 0 0 6 16 56 156 156 2156 2156 112 113 YEAAAA SROAAA AAAAxx +9659 9925 1 3 9 19 59 659 1659 4659 9659 118 119 NHAAAA TROAAA HHHHxx +477 9926 1 1 7 17 77 477 477 477 477 154 155 JSAAAA UROAAA OOOOxx +8194 9927 0 2 4 14 94 194 194 3194 8194 188 189 EDAAAA VROAAA VVVVxx +3937 9928 1 1 7 17 37 937 1937 3937 3937 74 75 LVAAAA WROAAA AAAAxx +3745 9929 1 1 5 5 45 745 1745 3745 3745 90 91 BOAAAA XROAAA HHHHxx +4096 9930 0 0 6 16 96 96 96 4096 4096 192 193 OBAAAA YROAAA OOOOxx +5487 9931 1 3 7 7 87 487 1487 487 5487 174 175 BDAAAA ZROAAA VVVVxx +2475 9932 1 3 5 15 75 475 475 2475 2475 150 151 FRAAAA ASOAAA AAAAxx +6105 9933 1 1 5 5 5 105 105 1105 6105 10 11 VAAAAA BSOAAA HHHHxx +6036 9934 0 0 6 16 36 36 36 1036 6036 72 73 EYAAAA CSOAAA OOOOxx +1315 9935 1 3 5 15 15 315 1315 1315 1315 30 31 PYAAAA DSOAAA VVVVxx +4473 9936 1 1 3 13 73 473 473 4473 4473 146 147 BQAAAA ESOAAA AAAAxx +4016 9937 0 0 6 16 16 16 16 4016 4016 32 33 MYAAAA FSOAAA HHHHxx +8135 9938 1 3 5 15 35 135 135 3135 8135 70 71 XAAAAA GSOAAA OOOOxx +8892 9939 0 0 2 12 92 892 892 3892 8892 184 185 AEAAAA HSOAAA VVVVxx +4850 9940 0 2 0 10 50 850 850 4850 4850 100 101 OEAAAA ISOAAA AAAAxx +2545 9941 1 1 5 5 45 545 545 2545 2545 90 91 XTAAAA JSOAAA HHHHxx +3788 9942 0 0 8 8 88 788 1788 3788 3788 176 177 SPAAAA KSOAAA OOOOxx +1672 9943 0 0 2 12 72 672 1672 1672 1672 144 145 IMAAAA LSOAAA VVVVxx +3664 9944 0 0 4 4 64 664 1664 3664 3664 128 129 YKAAAA MSOAAA AAAAxx +3775 9945 1 3 5 15 75 775 1775 3775 3775 150 151 FPAAAA NSOAAA HHHHxx +3103 9946 1 3 3 3 3 103 1103 3103 3103 6 7 JPAAAA OSOAAA OOOOxx +9335 9947 1 3 5 15 35 335 1335 4335 9335 70 71 BVAAAA PSOAAA VVVVxx +9200 9948 0 0 0 0 0 200 1200 4200 9200 0 1 WPAAAA QSOAAA AAAAxx +8665 9949 1 1 5 5 65 665 665 3665 8665 130 131 HVAAAA RSOAAA HHHHxx +1356 9950 0 0 6 16 56 356 1356 1356 1356 112 113 EAAAAA SSOAAA OOOOxx +6118 9951 0 2 8 18 18 118 118 1118 6118 36 37 IBAAAA TSOAAA VVVVxx +4605 9952 1 1 5 5 5 605 605 4605 4605 10 11 DVAAAA USOAAA AAAAxx +5651 9953 1 3 1 11 51 651 1651 651 5651 102 103 JJAAAA VSOAAA HHHHxx +9055 9954 1 3 5 15 55 55 1055 4055 9055 110 111 HKAAAA WSOAAA OOOOxx +8461 9955 1 1 1 1 61 461 461 3461 8461 122 123 LNAAAA XSOAAA VVVVxx +6107 9956 1 3 7 7 7 107 107 1107 6107 14 15 XAAAAA YSOAAA AAAAxx +1967 9957 1 3 7 7 67 967 1967 1967 1967 134 135 RXAAAA ZSOAAA HHHHxx +8910 9958 0 2 0 10 10 910 910 3910 8910 20 21 SEAAAA ATOAAA OOOOxx +8257 9959 1 1 7 17 57 257 257 3257 8257 114 115 PFAAAA BTOAAA VVVVxx +851 9960 1 3 1 11 51 851 851 851 851 102 103 TGAAAA CTOAAA AAAAxx +7823 9961 1 3 3 3 23 823 1823 2823 7823 46 47 XOAAAA DTOAAA HHHHxx +3208 9962 0 0 8 8 8 208 1208 3208 3208 16 17 KTAAAA ETOAAA OOOOxx +856 9963 0 0 6 16 56 856 856 856 856 112 113 YGAAAA FTOAAA VVVVxx +2654 9964 0 2 4 14 54 654 654 2654 2654 108 109 CYAAAA GTOAAA AAAAxx +7185 9965 1 1 5 5 85 185 1185 2185 7185 170 171 JQAAAA HTOAAA HHHHxx +309 9966 1 1 9 9 9 309 309 309 309 18 19 XLAAAA ITOAAA OOOOxx +9752 9967 0 0 2 12 52 752 1752 4752 9752 104 105 CLAAAA JTOAAA VVVVxx +6405 9968 1 1 5 5 5 405 405 1405 6405 10 11 JMAAAA KTOAAA AAAAxx +6113 9969 1 1 3 13 13 113 113 1113 6113 26 27 DBAAAA LTOAAA HHHHxx +9774 9970 0 2 4 14 74 774 1774 4774 9774 148 149 YLAAAA MTOAAA OOOOxx +1674 9971 0 2 4 14 74 674 1674 1674 1674 148 149 KMAAAA NTOAAA VVVVxx +9602 9972 0 2 2 2 2 602 1602 4602 9602 4 5 IFAAAA OTOAAA AAAAxx +1363 9973 1 3 3 3 63 363 1363 1363 1363 126 127 LAAAAA PTOAAA HHHHxx +6887 9974 1 3 7 7 87 887 887 1887 6887 174 175 XEAAAA QTOAAA OOOOxx +6170 9975 0 2 0 10 70 170 170 1170 6170 140 141 IDAAAA RTOAAA VVVVxx +8888 9976 0 0 8 8 88 888 888 3888 8888 176 177 WDAAAA STOAAA AAAAxx +2981 9977 1 1 1 1 81 981 981 2981 2981 162 163 RKAAAA TTOAAA HHHHxx +7369 9978 1 1 9 9 69 369 1369 2369 7369 138 139 LXAAAA UTOAAA OOOOxx +6227 9979 1 3 7 7 27 227 227 1227 6227 54 55 NFAAAA VTOAAA VVVVxx +8002 9980 0 2 2 2 2 2 2 3002 8002 4 5 UVAAAA WTOAAA AAAAxx +4288 9981 0 0 8 8 88 288 288 4288 4288 176 177 YIAAAA XTOAAA HHHHxx +5136 9982 0 0 6 16 36 136 1136 136 5136 72 73 OPAAAA YTOAAA OOOOxx +1084 9983 0 0 4 4 84 84 1084 1084 1084 168 169 SPAAAA ZTOAAA VVVVxx +9117 9984 1 1 7 17 17 117 1117 4117 9117 34 35 RMAAAA AUOAAA AAAAxx +2406 9985 0 2 6 6 6 406 406 2406 2406 12 13 OOAAAA BUOAAA HHHHxx +1384 9986 0 0 4 4 84 384 1384 1384 1384 168 169 GBAAAA CUOAAA OOOOxx +9194 9987 0 2 4 14 94 194 1194 4194 9194 188 189 QPAAAA DUOAAA VVVVxx +858 9988 0 2 8 18 58 858 858 858 858 116 117 AHAAAA EUOAAA AAAAxx +8592 9989 0 0 2 12 92 592 592 3592 8592 184 185 MSAAAA FUOAAA HHHHxx +4773 9990 1 1 3 13 73 773 773 4773 4773 146 147 PBAAAA GUOAAA OOOOxx +4093 9991 1 1 3 13 93 93 93 4093 4093 186 187 LBAAAA HUOAAA VVVVxx +6587 9992 1 3 7 7 87 587 587 1587 6587 174 175 JTAAAA IUOAAA AAAAxx +6093 9993 1 1 3 13 93 93 93 1093 6093 186 187 JAAAAA JUOAAA HHHHxx +429 9994 1 1 9 9 29 429 429 429 429 58 59 NQAAAA KUOAAA OOOOxx +5780 9995 0 0 0 0 80 780 1780 780 5780 160 161 IOAAAA LUOAAA VVVVxx +1783 9996 1 3 3 3 83 783 1783 1783 1783 166 167 PQAAAA MUOAAA AAAAxx +2992 9997 0 0 2 12 92 992 992 2992 2992 184 185 CLAAAA NUOAAA HHHHxx +0 9998 0 0 0 0 0 0 0 0 0 0 1 AAAAAA OUOAAA OOOOxx +2968 9999 0 0 8 8 68 968 968 2968 2968 136 137 EKAAAA PUOAAA VVVVxx diff --git a/sql/core/src/test/resources/test-data/with-array-fields.json b/sql/core/src/test/resources/test-data/with-array-fields.json new file mode 100644 index 0000000000000..09022ec028954 --- /dev/null +++ b/sql/core/src/test/resources/test-data/with-array-fields.json @@ -0,0 +1,3 @@ +{ "id": 1, "intervals": [{ "startTime": 111, "endTime": 211 }, { "startTime": 121, "endTime": 221 }], "ints": [11, 12, 13, 14]} +{ "id": 2, "intervals": [{ "startTime": 112, "endTime": 212 }, { "startTime": 122, "endTime": 222 }], "ints": [21, 22, 23, 24]} +{ "id": 3, "intervals": [{ "startTime": 113, "endTime": 213 }, { "startTime": 123, "endTime": 223 }], "ints": [31, 32, 33, 34]} \ No newline at end of file diff --git a/sql/core/src/test/resources/test-data/with-map-fields.json b/sql/core/src/test/resources/test-data/with-map-fields.json new file mode 100644 index 0000000000000..576fbb9b8758b --- /dev/null +++ b/sql/core/src/test/resources/test-data/with-map-fields.json @@ -0,0 +1,5 @@ +{ "id": 1, "intervals": { "a": { "startTime": 111, "endTime": 211 }, "b": { "startTime": 121, "endTime": 221 }}} +{ "id": 2, "intervals": { "a": { "startTime": 112, "endTime": 212 }, "b": { "startTime": 122, "endTime": 222 }}} +{ "id": 3, "intervals": { "a": { "startTime": 113, "endTime": 213 }, "b": { "startTime": 123, "endTime": 223 }}} +{ "id": 4, "intervals": { }} +{ "id": 5 } \ No newline at end of file diff --git a/sql/core/src/test/resources/tpcds/q32.sql b/sql/core/src/test/resources/tpcds/q32.sql index 1a907961e74bb..a6f59ecb87366 100755 --- a/sql/core/src/test/resources/tpcds/q32.sql +++ b/sql/core/src/test/resources/tpcds/q32.sql @@ -10,6 +10,6 @@ WHERE SELECT 1.3 * avg(cs_ext_discount_amt) FROM catalog_sales, date_dim WHERE cs_item_sk = i_item_sk - AND d_date BETWEEN '2000-01-27]' AND (cast('2000-01-27' AS DATE) + interval 90 days) + AND d_date BETWEEN '2000-01-27' AND (cast('2000-01-27' AS DATE) + interval 90 days) AND d_date_sk = cs_sold_date_sk) LIMIT 100 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala index 938d76c9f0837..b253c4a70bbf9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/AggregateHashMapSuite.scala @@ -20,33 +20,34 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf +import org.apache.spark.sql.internal.SQLConf class SingleLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "false") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "false") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "false", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "false", "configuration parameter changed in test body") } } class TwoLevelAggregateHashMapSuite extends DataFrameAggregateSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") } } @@ -56,18 +57,18 @@ class TwoLevelAggregateHashMapWithVectorizedMapSuite with BeforeAndAfter { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.sql.codegen.fallback", "false") - .set("spark.sql.codegen.aggregate.map.twolevel.enabled", "true") - .set("spark.sql.codegen.aggregate.map.vectorized.enable", "true") + .set(SQLConf.CODEGEN_FALLBACK.key, "false") + .set(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key, "true") + .set(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key, "true") // adding some checking after each test is run, assuring that the configs are not changed // in test code after { - assert(sparkConf.get("spark.sql.codegen.fallback") == "false", + assert(sparkConf.get(SQLConf.CODEGEN_FALLBACK.key) == "false", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.twolevel.enabled") == "true", + assert(sparkConf.get(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key) == "true", "configuration parameter changed in test body") - assert(sparkConf.get("spark.sql.codegen.aggregate.map.vectorized.enable") == "true", + assert(sparkConf.get(SQLConf.ENABLE_VECTORIZED_HASH_MAP.key) == "true", "configuration parameter changed in test body") } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala index d635912cf7205..52708f5fe4108 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ApproximatePercentileQuerySuite.scala @@ -208,7 +208,7 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { test("percentile_approx(col, ...), input rows contains null, with out group by") { withTempView(table) { - (1 to 1000).map(new Integer(_)).flatMap(Seq(null: Integer, _)).toDF("col") + (1 to 1000).map(Integer.valueOf(_)).flatMap(Seq(null: Integer, _)).toDF("col") .createOrReplaceTempView(table) checkAnswer( spark.sql( @@ -226,8 +226,8 @@ class ApproximatePercentileQuerySuite extends QueryTest with SharedSQLContext { withTempView(table) { val rand = new java.util.Random() (1 to 1000) - .map(new Integer(_)) - .map(v => (new Integer(v % 2), v)) + .map(Integer.valueOf(_)) + .map(v => (Integer.valueOf(v % 2), v)) // Add some nulls .flatMap(Seq(_, (null: Integer, null: Integer))) .toDF("key", "value").createOrReplaceTempView(table) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala index d95794d624033..c37d663941d8d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/BenchmarkQueryTest.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterAll +import org.apache.spark.internal.config.Tests.IS_TESTING import org.apache.spark.sql.catalyst.expressions.codegen.{CodeFormatter, CodeGenerator} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} @@ -29,7 +30,7 @@ abstract class BenchmarkQueryTest extends QueryTest with SharedSQLContext with B // When Utils.isTesting is true, the RuleExecutor will issue an exception when hitting // the max iteration of analyzer/optimizer batches. - assert(Utils.isTesting, "spark.testing is not set to true") + assert(Utils.isTesting, s"${IS_TESTING.key} is not set to true") /** * Drop all the tables 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 60c73df88896b..267f255a11e86 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 @@ -19,18 +19,23 @@ package org.apache.spark.sql import scala.collection.mutable.HashSet import scala.concurrent.duration._ -import scala.language.postfixOps import org.apache.spark.CleanerListener +import org.apache.spark.executor.DataReadMethod._ +import org.apache.spark.executor.DataReadMethod.DataReadMethod import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.SubqueryExpression +import org.apache.spark.sql.catalyst.plans.logical.{BROADCAST, Join, JoinStrategyHint, SHUFFLE_HASH} import org.apache.spark.sql.execution.{RDDScanExec, SparkPlan} import org.apache.spark.sql.execution.columnar._ import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types.{StringType, StructField, StructType} import org.apache.spark.storage.{RDDBlockId, StorageLevel} +import org.apache.spark.storage.StorageLevel.{MEMORY_AND_DISK_2, MEMORY_ONLY} import org.apache.spark.util.{AccumulatorContext, Utils} private case class BigData(s: String) @@ -64,6 +69,13 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext maybeBlock.nonEmpty } + def isExpectStorageLevel(rddId: Int, level: DataReadMethod): Boolean = { + val maybeBlock = sparkContext.env.blockManager.get(RDDBlockId(rddId, 0)) + val isExpectLevel = maybeBlock.forall(_.readMethod === level) + maybeBlock.foreach(_ => sparkContext.env.blockManager.releaseLock(RDDBlockId(rddId, 0))) + maybeBlock.nonEmpty && isExpectLevel + } + private def getNumInMemoryRelations(ds: Dataset[_]): Int = { val plan = ds.queryExecution.withCachedData var sum = plan.collect { case _: InMemoryRelation => 1 }.sum @@ -88,7 +100,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext assertCached(sql("SELECT COUNT(*) FROM tempTable"), 0) spark.catalog.cacheTable("tempTable") assertCached(sql("SELECT COUNT(*) FROM tempTable")) - spark.catalog.uncacheTable("tempTable") + uncacheTable("tempTable") } } @@ -110,7 +122,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext withTempView("tempTable") { sql("CACHE TABLE tempTable AS SELECT key FROM testData") assertCached(sql("SELECT COUNT(*) FROM tempTable")) - spark.catalog.uncacheTable("tempTable") + uncacheTable("tempTable") } } @@ -123,7 +135,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext assertCached(sql("SELECT COUNT(*) FROM tempTable2")) // Is this valid? - spark.catalog.uncacheTable("tempTable2") + uncacheTable("tempTable2") // Should this be cached? assertCached(sql("SELECT COUNT(*) FROM tempTable1"), 0) @@ -160,7 +172,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext case _ => false }) - spark.catalog.uncacheTable("testData") + uncacheTable("testData") assert(!spark.catalog.isCached("testData")) assert(spark.table("testData").queryExecution.withCachedData match { case _: InMemoryRelation => false @@ -185,7 +197,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext }.size } - spark.catalog.uncacheTable("testData") + uncacheTable("testData") } test("read from cached table and uncache") { @@ -193,7 +205,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer(spark.table("testData"), testData.collect().toSeq) assertCached(spark.table("testData")) - spark.catalog.uncacheTable("testData") + uncacheTable("testData") checkAnswer(spark.table("testData"), testData.collect().toSeq) assertCached(spark.table("testData"), 0) } @@ -204,7 +216,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( sql("SELECT * FROM selectStar WHERE key = 1"), Seq(Row(1, "1"))) - spark.catalog.uncacheTable("selectStar") + uncacheTable("selectStar") } test("Self-join cached") { @@ -214,7 +226,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key"), unCachedAnswer.toSeq) - spark.catalog.uncacheTable("testData") + uncacheTable("testData") } test("'CACHE TABLE' and 'UNCACHE TABLE' SQL statement") { @@ -229,7 +241,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext sql("UNCACHE TABLE testData") assert(!spark.catalog.isCached("testData"), "Table 'testData' should not be cached") - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } @@ -244,8 +256,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext isMaterialized(rddId), "Eagerly cached in-memory table should have already been materialized") - spark.catalog.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { + uncacheTable("testCacheTable") + eventually(timeout(10.seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } @@ -261,8 +273,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext isMaterialized(rddId), "Eagerly cached in-memory table should have already been materialized") - spark.catalog.uncacheTable("testCacheTable") - eventually(timeout(10 seconds)) { + uncacheTable("testCacheTable") + eventually(timeout(10.seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } @@ -282,12 +294,63 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext isMaterialized(rddId), "Lazily cached in-memory table should have been materialized") - spark.catalog.uncacheTable("testData") - eventually(timeout(10 seconds)) { + uncacheTable("testData") + eventually(timeout(10.seconds)) { assert(!isMaterialized(rddId), "Uncached in-memory table should have been unpersisted") } } + private def assertStorageLevel(cacheOptions: String, level: DataReadMethod): Unit = { + sql(s"CACHE TABLE testData OPTIONS$cacheOptions") + assertCached(spark.table("testData")) + val rddId = rddIdOf("testData") + assert(isExpectStorageLevel(rddId, level)) + } + + test("SQL interface support storageLevel(DISK_ONLY)") { + assertStorageLevel("('storageLevel' 'DISK_ONLY')", Disk) + } + + test("SQL interface support storageLevel(DISK_ONLY) with invalid options") { + assertStorageLevel("('storageLevel' 'DISK_ONLY', 'a' '1', 'b' '2')", Disk) + } + + test("SQL interface support storageLevel(MEMORY_ONLY)") { + assertStorageLevel("('storageLevel' 'MEMORY_ONLY')", Memory) + } + + test("SQL interface cache SELECT ... support storageLevel(DISK_ONLY)") { + withTempView("testCacheSelect") { + sql("CACHE TABLE testCacheSelect OPTIONS('storageLevel' 'DISK_ONLY') SELECT * FROM testData") + assertCached(spark.table("testCacheSelect")) + val rddId = rddIdOf("testCacheSelect") + assert(isExpectStorageLevel(rddId, Disk)) + } + } + + test("SQL interface support storageLevel(Invalid StorageLevel)") { + val message = intercept[IllegalArgumentException] { + sql("CACHE TABLE testData OPTIONS('storageLevel' 'invalid_storage_level')") + }.getMessage + assert(message.contains("Invalid StorageLevel: INVALID_STORAGE_LEVEL")) + } + + test("SQL interface support storageLevel(with LAZY)") { + sql("CACHE LAZY TABLE testData OPTIONS('storageLevel' 'disk_only')") + assertCached(spark.table("testData")) + + val rddId = rddIdOf("testData") + assert( + !isMaterialized(rddId), + "Lazily cached in-memory table shouldn't be materialized eagerly") + + sql("SELECT COUNT(*) FROM testData").collect() + assert( + isMaterialized(rddId), + "Lazily cached in-memory table should have been materialized") + assert(isExpectStorageLevel(rddId, Disk)) + } + test("InMemoryRelation statistics") { sql("CACHE TABLE testData") spark.table("testData").queryExecution.withCachedData.collect { @@ -368,12 +431,12 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } spark.sparkContext.cleaner.get.attachListener(cleanerListener) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") System.gc() - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { assert(toBeCleanedAccIds.synchronized { toBeCleanedAccIds.isEmpty }, "batchStats accumulators should be cleared after GC when uncacheTable") } @@ -416,7 +479,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( sql("SELECT key, count(*) FROM orderedTable GROUP BY key ORDER BY key"), sql("SELECT key, count(*) FROM testData3x GROUP BY key ORDER BY key").collect()) - spark.catalog.uncacheTable("orderedTable") + uncacheTable("orderedTable") spark.catalog.dropTempView("orderedTable") // Set up two tables distributed in the same way. Try this with the data distributed into @@ -438,8 +501,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer(sql("SELECT count(*) FROM t1 GROUP BY key"), sql("SELECT count(*) FROM testData GROUP BY key")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } } @@ -456,8 +519,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } // One side of join is not partitioned in the desired way. Need to shuffle one side. @@ -473,8 +536,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } withTempView("t1", "t2") { @@ -489,8 +552,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } // One side of join is not partitioned in the desired way. Since the number of partitions of @@ -507,8 +570,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, testData.join(testData2, $"key" === $"a").select($"key", $"value", $"a", $"b")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } // repartition's column ordering is different from group by column ordering. @@ -522,7 +585,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, testData.distinct().select($"value", $"key")) - spark.catalog.uncacheTable("t1") + uncacheTable("t1") } // repartition's column ordering is different from join condition's column ordering. @@ -544,8 +607,8 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext checkAnswer( query, df1.join(df2, $"key" === $"a" && $"value" === $"b").select($"key", $"value", $"a", $"b")) - spark.catalog.uncacheTable("t1") - spark.catalog.uncacheTable("t2") + uncacheTable("t1") + uncacheTable("t2") } } @@ -558,7 +621,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext selectStar, Seq(Row(1, "1"))) - spark.catalog.uncacheTable("selectStar") + uncacheTable("selectStar") checkAnswer( selectStar, Seq(Row(1, "1"))) @@ -636,7 +699,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext Utils.deleteRecursively(path) spark.sessionState.catalog.refreshTable(TableIdentifier("t")) - spark.catalog.uncacheTable("t") + uncacheTable("t") assert(spark.table("t").select($"i").count() == 0) assert(getNumInMemoryRelations(spark.table("t").select($"i")) == 0) } @@ -769,7 +832,7 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext val df = spark.range(10).cache() df.queryExecution.executedPlan.foreach { case i: InMemoryTableScanExec => - assert(i.supportsBatch == vectorized && i.supportCodegen == vectorized) + assert(i.supportsColumnar == vectorized) case _ => } } @@ -865,4 +928,170 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with SharedSQLContext } } } + + test("Cache should respect the hint") { + def testHint(df: Dataset[_], expectedHint: JoinStrategyHint): Unit = { + val df2 = spark.range(2000).cache() + df2.count() + + def checkHintExists(): Unit = { + // Test the broadcast hint. + val joinPlan = df.join(df2, "id").queryExecution.optimizedPlan + val joinHints = joinPlan.collect { + case Join(_, _, _, _, hint) => hint + } + assert(joinHints.size == 1) + assert(joinHints(0).leftHint.get.strategy.contains(expectedHint)) + assert(joinHints(0).rightHint.isEmpty) + } + + // Make sure the hint does exist when `df` is not cached. + checkHintExists() + + df.cache() + try { + df.count() + // Make sure the hint still exists when `df` is cached. + checkHintExists() + } finally { + // Clean-up + df.unpersist() + } + } + + // The hint is the root node + testHint(broadcast(spark.range(1000)), BROADCAST) + // The hint is under subquery alias + testHint(broadcast(spark.range(1000)).as("df"), BROADCAST) + // The hint is under filter + testHint(broadcast(spark.range(1000)).filter($"id" > 100), BROADCAST) + // If there are 2 adjacent hints, the top one takes effect. + testHint( + spark.range(1000) + .hint("SHUFFLE_MERGE") + .hint("SHUFFLE_HASH") + .as("df"), + SHUFFLE_HASH) + } + + test("analyzes column statistics in cached query") { + def query(): DataFrame = { + spark.range(100) + .selectExpr("id % 3 AS c0", "id % 5 AS c1", "2 AS c2") + .groupBy("c0") + .agg(avg("c1").as("v1"), sum("c2").as("v2")) + } + // First, checks if there is no column statistic in cached query + val queryStats1 = query().cache.queryExecution.optimizedPlan.stats.attributeStats + assert(queryStats1.map(_._1.name).isEmpty) + + val cacheManager = spark.sharedState.cacheManager + val cachedData = cacheManager.lookupCachedData(query().logicalPlan) + assert(cachedData.isDefined) + val queryAttrs = cachedData.get.plan.output + assert(queryAttrs.size === 3) + val (c0, v1, v2) = (queryAttrs(0), queryAttrs(1), queryAttrs(2)) + + // Analyzes one column in the query output + cacheManager.analyzeColumnCacheQuery(spark, cachedData.get, v1 :: Nil) + val queryStats2 = query().queryExecution.optimizedPlan.stats.attributeStats + assert(queryStats2.map(_._1.name).toSet === Set("v1")) + + // Analyzes two more columns + cacheManager.analyzeColumnCacheQuery(spark, cachedData.get, c0 :: v2 :: Nil) + val queryStats3 = query().queryExecution.optimizedPlan.stats.attributeStats + assert(queryStats3.map(_._1.name).toSet === Set("c0", "v1", "v2")) + } + + test("SPARK-27248 refreshTable should recreate cache with same cache name and storage level") { + // This section tests when a table is cached with its qualified name but it is refreshed with + // its unqualified name. + withTempDatabase { db => + withTempPath { path => + withTable(s"$db.cachedTable") { + // Create table 'cachedTable' in temp db for testing purpose. + spark.catalog.createTable( + s"$db.cachedTable", + "PARQUET", + StructType(Array(StructField("key", StringType))), + Map("LOCATION" -> path.toURI.toString)) + + withCache(s"$db.cachedTable") { + // Cache the table 'cachedTable' in temp db with qualified table name with storage level + // MEMORY_ONLY, and then check whether the table is cached with expected name and + // storage level. + spark.catalog.cacheTable(s"$db.cachedTable", MEMORY_ONLY) + assertCached(spark.table(s"$db.cachedTable"), s"$db.cachedTable", MEMORY_ONLY) + assert(spark.catalog.isCached(s"$db.cachedTable"), + s"Table '$db.cachedTable' should be cached.") + + // Refresh the table 'cachedTable' in temp db with qualified table name, and then check + // whether the table is still cached with the same name and storage level. + // Without bug fix 'SPARK-27248', the recreated cache storage level will be default + // storage level 'MEMORY_AND_DISK', instead of 'MEMORY_ONLY'. + spark.catalog.refreshTable(s"$db.cachedTable") + assertCached(spark.table(s"$db.cachedTable"), s"$db.cachedTable", MEMORY_ONLY) + assert(spark.catalog.isCached(s"$db.cachedTable"), + s"Table '$db.cachedTable' should be cached after refreshing with its qualified name.") + + // Change the active database to the temp db and refresh the table with unqualified + // table name, and then check whether the table is still cached with the same name and + // storage level. + // Without bug fix 'SPARK-27248', the recreated cache name will be changed to + // 'cachedTable', instead of '$db.cachedTable' + activateDatabase(db) { + spark.catalog.refreshTable("cachedTable") + assertCached(spark.table("cachedTable"), s"$db.cachedTable", MEMORY_ONLY) + assert(spark.catalog.isCached("cachedTable"), + s"Table '$db.cachedTable' should be cached after refreshing with its " + + "unqualified name.") + } + } + } + } + + // This section tests when a table is cached with its unqualified name but it is refreshed + // with its qualified name. + withTempPath { path => + withTable("cachedTable") { + // Create table 'cachedTable' in default db for testing purpose. + spark.catalog.createTable( + "cachedTable", + "PARQUET", + StructType(Array(StructField("key", StringType))), + Map("LOCATION" -> path.toURI.toString)) + withCache("cachedTable") { + // Cache the table 'cachedTable' in default db without qualified table name with storage + // level 'MEMORY_AND_DISK2', and then check whether the table is cached with expected + // name and storage level. + spark.catalog.cacheTable("cachedTable", MEMORY_AND_DISK_2) + assertCached(spark.table("cachedTable"), "cachedTable", MEMORY_AND_DISK_2) + assert(spark.catalog.isCached("cachedTable"), + "Table 'cachedTable' should be cached.") + + // Refresh the table 'cachedTable' in default db with unqualified table name, and then + // check whether the table is still cached with the same name and storage level. + // Without bug fix 'SPARK-27248', the recreated cache storage level will be default + // storage level 'MEMORY_AND_DISK', instead of 'MEMORY_AND_DISK2'. + spark.catalog.refreshTable("cachedTable") + assertCached(spark.table("cachedTable"), "cachedTable", MEMORY_AND_DISK_2) + assert(spark.catalog.isCached("cachedTable"), + "Table 'cachedTable' should be cached after refreshing with its unqualified name.") + + // Change the active database to the temp db and refresh the table with qualified + // table name, and then check whether the table is still cached with the same name and + // storage level. + // Without bug fix 'SPARK-27248', the recreated cache name will be changed to + // 'default.cachedTable', instead of 'cachedTable' + activateDatabase(db) { + spark.catalog.refreshTable("default.cachedTable") + assertCached(spark.table("default.cachedTable"), "cachedTable", MEMORY_AND_DISK_2) + assert(spark.catalog.isCached("default.cachedTable"), + "Table 'cachedTable' should be cached after refreshing with its qualified name.") + } + } + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala index f984a1b722e36..ee258f8ac3661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ColumnExpressionSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql +import java.sql.Date import java.util.Locale import scala.collection.JavaConverters._ @@ -28,6 +29,7 @@ import org.scalatest.Matchers._ import org.apache.spark.sql.catalyst.expressions.NamedExpression import org.apache.spark.sql.execution.ProjectExec import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -413,6 +415,43 @@ class ColumnExpressionSuite extends QueryTest with SharedSQLContext { } } + test("IN/INSET with bytes, shorts, ints, dates") { + def check(): Unit = { + val values = Seq( + (Byte.MinValue, Some(Short.MinValue), Int.MinValue, Date.valueOf("2017-01-01")), + (Byte.MaxValue, None, Int.MaxValue, null)) + val df = values.toDF("b", "s", "i", "d") + checkAnswer(df.select($"b".isin(Byte.MinValue, Byte.MaxValue)), Seq(Row(true), Row(true))) + checkAnswer(df.select($"b".isin(-1.toByte, 2.toByte)), Seq(Row(false), Row(false))) + checkAnswer(df.select($"s".isin(Short.MinValue, 1.toShort)), Seq(Row(true), Row(null))) + checkAnswer(df.select($"s".isin(0.toShort, null)), Seq(Row(null), Row(null))) + checkAnswer(df.select($"i".isin(0, Int.MinValue)), Seq(Row(true), Row(false))) + checkAnswer(df.select($"i".isin(null, Int.MinValue)), Seq(Row(true), Row(null))) + checkAnswer( + df.select($"d".isin(Date.valueOf("1950-01-01"), Date.valueOf("2017-01-01"))), + Seq(Row(true), Row(null))) + checkAnswer( + df.select($"d".isin(Date.valueOf("1950-01-01"), null)), + Seq(Row(null), Row(null))) + } + + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "10") { + check() + } + + withSQLConf( + SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "0", + SQLConf.OPTIMIZER_INSET_SWITCH_THRESHOLD.key -> "0") { + check() + } + + withSQLConf( + SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "0", + SQLConf.OPTIMIZER_INSET_SWITCH_THRESHOLD.key -> "20") { + check() + } + } + test("isInCollection: Scala Collection") { val df = Seq((1, "x"), (2, "y"), (3, "z")).toDF("a", "b") // Test with different types of collections 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 new file mode 100644 index 0000000000000..6b67fccf86b9f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/CsvFunctionsSuite.scala @@ -0,0 +1,184 @@ +/* + * 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 java.text.SimpleDateFormat +import java.util.Locale + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkException +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +class CsvFunctionsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("from_csv with empty options") { + val df = Seq("1").toDS() + val schema = "a int" + + checkAnswer( + df.select(from_csv($"value", lit(schema), Map[String, String]().asJava)), + Row(Row(1)) :: Nil) + } + + test("from_csv with option") { + val df = Seq("26/08/2015 18:00").toDS() + val schema = new StructType().add("time", TimestampType) + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm") + + checkAnswer( + df.select(from_csv($"value", schema, options)), + Row(Row(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))) + } + + test("checking the columnNameOfCorruptRecord option") { + val columnNameOfCorruptRecord = "_unparsed" + val df = Seq("0,2013-111-11 12:13:14", "1,1983-08-04").toDS() + val schema = new StructType().add("a", IntegerType).add("b", DateType) + val schemaWithCorrField1 = schema.add(columnNameOfCorruptRecord, StringType) + val df2 = df + .select(from_csv($"value", schemaWithCorrField1, Map( + "mode" -> "Permissive", "columnNameOfCorruptRecord" -> columnNameOfCorruptRecord))) + + checkAnswer(df2, Seq( + Row(Row(0, null, "0,2013-111-11 12:13:14")), + Row(Row(1, java.sql.Date.valueOf("1983-08-04"), null)))) + } + + test("schema_of_csv - infers schemas") { + checkAnswer( + spark.range(1).select(schema_of_csv(lit("0.1,1"))), + Seq(Row("struct<_c0:double,_c1:int>"))) + checkAnswer( + spark.range(1).select(schema_of_csv("0.1,1")), + Seq(Row("struct<_c0:double,_c1:int>"))) + } + + test("schema_of_csv - infers schemas using options") { + val df = spark.range(1) + .select(schema_of_csv(lit("0.1 1"), Map("sep" -> " ").asJava)) + checkAnswer(df, Seq(Row("struct<_c0:double,_c1:int>"))) + } + + test("to_csv - struct") { + val df = Seq(Tuple1(Tuple1(1))).toDF("a") + + checkAnswer(df.select(to_csv($"a")), Row("1") :: Nil) + } + + test("to_csv with option") { + val df = Seq(Tuple1(Tuple1(java.sql.Timestamp.valueOf("2015-08-26 18:00:00.0")))).toDF("a") + val options = Map("timestampFormat" -> "dd/MM/yyyy HH:mm").asJava + + checkAnswer(df.select(to_csv($"a", options)), Row("26/08/2015 18:00") :: Nil) + } + + test("from_csv invalid csv - check modes") { + withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { + val schema = new StructType() + .add("a", IntegerType) + .add("b", IntegerType) + .add("_unparsed", StringType) + val badRec = "\"" + val df = Seq(badRec, "2,12").toDS() + + checkAnswer( + df.select(from_csv($"value", schema, Map("mode" -> "PERMISSIVE"))), + Row(Row(null, null, badRec)) :: Row(Row(2, 12, null)) :: Nil) + + val exception1 = intercept[SparkException] { + df.select(from_csv($"value", schema, Map("mode" -> "FAILFAST"))).collect() + }.getMessage + assert(exception1.contains( + "Malformed records are detected in record parsing. Parse Mode: FAILFAST.")) + + val exception2 = intercept[SparkException] { + df.select(from_csv($"value", schema, Map("mode" -> "DROPMALFORMED"))) + .collect() + }.getMessage + assert(exception2.contains( + "from_csv() doesn't support the DROPMALFORMED mode. " + + "Acceptable modes are PERMISSIVE and FAILFAST.")) + } + } + + test("from_csv uses DDL strings for defining a schema - java") { + val df = Seq("""1,"haa"""").toDS() + checkAnswer( + df.select( + from_csv($"value", lit("a INT, b STRING"), new java.util.HashMap[String, String]())), + Row(Row(1, "haa")) :: Nil) + } + + test("roundtrip to_csv -> from_csv") { + val df = Seq(Tuple1(Tuple1(1)), Tuple1(null)).toDF("struct") + val schema = df.schema(0).dataType.asInstanceOf[StructType] + val options = Map.empty[String, String] + val readback = df.select(to_csv($"struct").as("csv")) + .select(from_csv($"csv", schema, options).as("struct")) + + checkAnswer(df, readback) + } + + test("roundtrip from_csv -> to_csv") { + val df = Seq(Some("1"), None).toDF("csv") + val schema = new StructType().add("a", IntegerType) + val options = Map.empty[String, String] + val readback = df.select(from_csv($"csv", schema, options).as("struct")) + .select(to_csv($"struct").as("csv")) + + checkAnswer(df, readback) + } + + test("infers schemas of a CSV string and pass to to from_csv") { + val in = Seq("""0.123456789,987654321,"San Francisco"""").toDS() + val options = Map.empty[String, String].asJava + val out = in.select(from_csv('value, schema_of_csv("0.1,1,a"), options) as "parsed") + val expected = StructType(Seq(StructField( + "parsed", + StructType(Seq( + StructField("_c0", DoubleType, true), + StructField("_c1", IntegerType, true), + StructField("_c2", StringType, true)))))) + + assert(out.schema == expected) + } + + test("Support to_csv in SQL") { + val df1 = Seq(Tuple1(Tuple1(1))).toDF("a") + checkAnswer(df1.selectExpr("to_csv(a)"), Row("1") :: Nil) + } + + test("parse timestamps with locale") { + Seq("en-US", "ko-KR", "zh-CN", "ru-RU").foreach { langTag => + val locale = Locale.forLanguageTag(langTag) + val ts = new SimpleDateFormat("dd/MM/yyyy HH:mm").parse("06/11/2018 18:00") + val timestampFormat = "dd MMM yyyy HH:mm" + val sdf = new SimpleDateFormat(timestampFormat, locale) + val input = Seq(s"""${sdf.format(ts)}""").toDS() + val options = Map("timestampFormat" -> timestampFormat, "locale" -> langTag) + val df = input.select(from_csv($"value", lit("time timestamp"), options.asJava)) + + checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala index d0106c44b7db2..e49ef012f5ebb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameAggregateSuite.scala @@ -669,23 +669,19 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { } } - Seq(true, false).foreach { codegen => - test("SPARK-22951: dropDuplicates on empty dataFrames should produce correct aggregate " + - s"results when codegen is enabled: $codegen") { - withSQLConf((SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, codegen.toString)) { - // explicit global aggregations - val emptyAgg = Map.empty[String, String] - checkAnswer(spark.emptyDataFrame.agg(emptyAgg), Seq(Row())) - checkAnswer(spark.emptyDataFrame.groupBy().agg(emptyAgg), Seq(Row())) - checkAnswer(spark.emptyDataFrame.groupBy().agg(count("*")), Seq(Row(0))) - checkAnswer(spark.emptyDataFrame.dropDuplicates().agg(emptyAgg), Seq(Row())) - checkAnswer(spark.emptyDataFrame.dropDuplicates().groupBy().agg(emptyAgg), Seq(Row())) - checkAnswer(spark.emptyDataFrame.dropDuplicates().groupBy().agg(count("*")), Seq(Row(0))) - - // global aggregation is converted to grouping aggregation: - assert(spark.emptyDataFrame.dropDuplicates().count() == 0) - } - } + testWithWholeStageCodegenOnAndOff("SPARK-22951: dropDuplicates on empty dataFrames " + + "should produce correct aggregate") { _ => + // explicit global aggregations + val emptyAgg = Map.empty[String, String] + checkAnswer(spark.emptyDataFrame.agg(emptyAgg), Seq(Row())) + checkAnswer(spark.emptyDataFrame.groupBy().agg(emptyAgg), Seq(Row())) + checkAnswer(spark.emptyDataFrame.groupBy().agg(count("*")), Seq(Row(0))) + checkAnswer(spark.emptyDataFrame.dropDuplicates().agg(emptyAgg), Seq(Row())) + checkAnswer(spark.emptyDataFrame.dropDuplicates().groupBy().agg(emptyAgg), Seq(Row())) + checkAnswer(spark.emptyDataFrame.dropDuplicates().groupBy().agg(count("*")), Seq(Row(0))) + + // global aggregation is converted to grouping aggregation: + assert(spark.emptyDataFrame.dropDuplicates().count() == 0) } test("SPARK-21896: Window functions inside aggregate functions") { @@ -727,4 +723,208 @@ class DataFrameAggregateSuite extends QueryTest with SharedSQLContext { "grouping expressions: [current_date(None)], value: [key: int, value: string], " + "type: GroupBy]")) } + + test("SPARK-26021: NaN and -0.0 in grouping expressions") { + checkAnswer( + Seq(0.0f, -0.0f, 0.0f/0.0f, Float.NaN).toDF("f").groupBy("f").count(), + Row(0.0f, 2) :: Row(Float.NaN, 2) :: Nil) + checkAnswer( + Seq(0.0d, -0.0d, 0.0d/0.0d, Double.NaN).toDF("d").groupBy("d").count(), + Row(0.0d, 2) :: Row(Double.NaN, 2) :: Nil) + + // test with complicated type grouping expressions + checkAnswer( + Seq(0.0f, -0.0f, 0.0f/0.0f, Float.NaN).toDF("f") + .groupBy(array("f"), struct("f")).count(), + Row(Seq(0.0f), Row(0.0f), 2) :: + Row(Seq(Float.NaN), Row(Float.NaN), 2) :: Nil) + checkAnswer( + Seq(0.0d, -0.0d, 0.0d/0.0d, Double.NaN).toDF("d") + .groupBy(array("d"), struct("d")).count(), + Row(Seq(0.0d), Row(0.0d), 2) :: + Row(Seq(Double.NaN), Row(Double.NaN), 2) :: Nil) + + checkAnswer( + Seq(0.0f, -0.0f, 0.0f/0.0f, Float.NaN).toDF("f") + .groupBy(array(struct("f")), struct(array("f"))).count(), + Row(Seq(Row(0.0f)), Row(Seq(0.0f)), 2) :: + Row(Seq(Row(Float.NaN)), Row(Seq(Float.NaN)), 2) :: Nil) + checkAnswer( + Seq(0.0d, -0.0d, 0.0d/0.0d, Double.NaN).toDF("d") + .groupBy(array(struct("d")), struct(array("d"))).count(), + Row(Seq(Row(0.0d)), Row(Seq(0.0d)), 2) :: + Row(Seq(Row(Double.NaN)), Row(Seq(Double.NaN)), 2) :: Nil) + + // test with complicated type grouping columns + val df = Seq( + (Array(-0.0f, 0.0f), Tuple2(-0.0d, Double.NaN), Seq(Tuple2(-0.0d, Double.NaN))), + (Array(0.0f, -0.0f), Tuple2(0.0d, Double.NaN), Seq(Tuple2(0.0d, 0.0/0.0))) + ).toDF("arr", "stru", "arrOfStru") + checkAnswer( + df.groupBy("arr", "stru", "arrOfStru").count(), + Row(Seq(0.0f, 0.0f), Row(0.0d, Double.NaN), Seq(Row(0.0d, Double.NaN)), 2) + ) + } + + test("SPARK-27581: DataFrame countDistinct(\"*\") shouldn't fail with AnalysisException") { + val df = sql("select id % 100 from range(100000)") + val distinctCount1 = df.select(expr("count(distinct(*))")) + val distinctCount2 = df.select(countDistinct("*")) + checkAnswer(distinctCount1, distinctCount2) + + val countAndDistinct = df.select(count("*"), countDistinct("*")) + checkAnswer(countAndDistinct, Row(100000, 100)) + } + + test("max_by") { + val yearOfMaxEarnings = + sql("SELECT course, max_by(year, earnings) FROM courseSales GROUP BY course") + checkAnswer(yearOfMaxEarnings, Row("dotNET", 2013) :: Row("Java", 2013) :: Nil) + + checkAnswer( + sql("SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y)"), + Row("b") :: Nil + ) + + checkAnswer( + sql("SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', null)), (('c', 20)) AS tab(x, y)"), + Row("c") :: Nil + ) + + checkAnswer( + sql("SELECT max_by(x, y) FROM VALUES (('a', null)), (('b', null)), (('c', 20)) AS tab(x, y)"), + Row("c") :: Nil + ) + + checkAnswer( + sql("SELECT max_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', null)) AS tab(x, y)"), + Row("b") :: Nil + ) + + checkAnswer( + sql("SELECT max_by(x, y) FROM VALUES (('a', null)), (('b', null)) AS tab(x, y)"), + Row(null) :: Nil + ) + + // structs as ordering value. + checkAnswer( + sql("select max_by(x, y) FROM VALUES (('a', (10, 20))), (('b', (10, 50))), " + + "(('c', (10, 60))) AS tab(x, y)"), + Row("c") :: Nil + ) + + checkAnswer( + sql("select max_by(x, y) FROM VALUES (('a', (10, 20))), (('b', (10, 50))), " + + "(('c', null)) AS tab(x, y)"), + Row("b") :: Nil + ) + + withTempView("tempView") { + val dfWithMap = Seq((0, "a"), (1, "b"), (2, "c")) + .toDF("x", "y") + .select($"x", map($"x", $"y").as("y")) + .createOrReplaceTempView("tempView") + val error = intercept[AnalysisException] { + sql("SELECT max_by(x, y) FROM tempView").show + } + assert( + error.message.contains("function max_by does not support ordering on type map")) + } + } + + test("min_by") { + val yearOfMinEarnings = + sql("SELECT course, min_by(year, earnings) FROM courseSales GROUP BY course") + checkAnswer(yearOfMinEarnings, Row("dotNET", 2012) :: Row("Java", 2012) :: Nil) + + checkAnswer( + sql("SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', 20)) AS tab(x, y)"), + Row("a") :: Nil + ) + + checkAnswer( + sql("SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', null)), (('c', 20)) AS tab(x, y)"), + Row("a") :: Nil + ) + + checkAnswer( + sql("SELECT min_by(x, y) FROM VALUES (('a', null)), (('b', null)), (('c', 20)) AS tab(x, y)"), + Row("c") :: Nil + ) + + checkAnswer( + sql("SELECT min_by(x, y) FROM VALUES (('a', 10)), (('b', 50)), (('c', null)) AS tab(x, y)"), + Row("a") :: Nil + ) + + checkAnswer( + sql("SELECT min_by(x, y) FROM VALUES (('a', null)), (('b', null)) AS tab(x, y)"), + Row(null) :: Nil + ) + + // structs as ordering value. + checkAnswer( + sql("select min_by(x, y) FROM VALUES (('a', (10, 20))), (('b', (10, 50))), " + + "(('c', (10, 60))) AS tab(x, y)"), + Row("a") :: Nil + ) + + checkAnswer( + sql("select min_by(x, y) FROM VALUES (('a', null)), (('b', (10, 50))), " + + "(('c', (10, 60))) AS tab(x, y)"), + Row("b") :: Nil + ) + + withTempView("tempView") { + val dfWithMap = Seq((0, "a"), (1, "b"), (2, "c")) + .toDF("x", "y") + .select($"x", map($"x", $"y").as("y")) + .createOrReplaceTempView("tempView") + val error = intercept[AnalysisException] { + sql("SELECT min_by(x, y) FROM tempView").show + } + assert( + error.message.contains("function min_by does not support ordering on type map")) + } + } + + test("count_if") { + withTempView("tempView") { + Seq(("a", None), ("a", Some(1)), ("a", Some(2)), ("a", Some(3)), + ("b", None), ("b", Some(4)), ("b", Some(5)), ("b", Some(6))) + .toDF("x", "y") + .createOrReplaceTempView("tempView") + + checkAnswer( + sql("SELECT COUNT_IF(NULL), COUNT_IF(y % 2 = 0), COUNT_IF(y % 2 <> 0), " + + "COUNT_IF(y IS NULL) FROM tempView"), + Row(0L, 3L, 3L, 2L)) + + checkAnswer( + sql("SELECT x, COUNT_IF(NULL), COUNT_IF(y % 2 = 0), COUNT_IF(y % 2 <> 0), " + + "COUNT_IF(y IS NULL) FROM tempView GROUP BY x"), + Row("a", 0L, 1L, 2L, 1L) :: Row("b", 0L, 2L, 1L, 1L) :: Nil) + + checkAnswer( + sql("SELECT x FROM tempView GROUP BY x HAVING COUNT_IF(y % 2 = 0) = 1"), + Row("a")) + + checkAnswer( + sql("SELECT x FROM tempView GROUP BY x HAVING COUNT_IF(y % 2 = 0) = 2"), + Row("b")) + + checkAnswer( + sql("SELECT x FROM tempView GROUP BY x HAVING COUNT_IF(y IS NULL) > 0"), + Row("a") :: Row("b") :: Nil) + + checkAnswer( + sql("SELECT x FROM tempView GROUP BY x HAVING COUNT_IF(NULL) > 0"), + Nil) + + val error = intercept[AnalysisException] { + sql("SELECT COUNT_IF(x) FROM tempView") + } + assert(error.message.contains("function count_if requires boolean type")) + } + } } 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 60ebc5e6cc09b..3f16f64f4b900 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 @@ -89,13 +89,13 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val msg1 = intercept[Exception] { df5.select(map_from_arrays($"k", $"v")).collect }.getMessage - assert(msg1.contains("Cannot use null as map key!")) + assert(msg1.contains("Cannot use null as map key")) val df6 = Seq((Seq(1, 2), Seq("a"))).toDF("k", "v") val msg2 = intercept[Exception] { df6.select(map_from_arrays($"k", $"v")).collect }.getMessage - assert(msg2.contains("The given two arrays should have the same length")) + assert(msg2.contains("The key array and value array of MapData must have the same length")) } test("struct with column name") { @@ -458,15 +458,12 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(df8.selectExpr("arrays_zip(v1, v2)"), expectedValue8) } - test("SPARK-24633: arrays_zip splits input processing correctly") { - Seq("true", "false").foreach { wholestageCodegenEnabled => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholestageCodegenEnabled) { - val df = spark.range(1) - val exprs = (0 to 5).map(x => array($"id" + lit(x))) - checkAnswer(df.select(arrays_zip(exprs: _*)), - Row(Seq(Row(0, 1, 2, 3, 4, 5)))) - } - } + testWithWholeStageCodegenOnAndOff("SPARK-24633: arrays_zip splits input " + + "processing correctly") { _ => + val df = spark.range(1) + val exprs = (0 to 5).map(x => array($"id" + lit(x))) + checkAnswer(df.select(arrays_zip(exprs: _*)), + Row(Seq(Row(0, 1, 2, 3, 4, 5)))) } def testSizeOfMap(sizeOfNull: Any): Unit = { @@ -1398,7 +1395,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } // Test with local relation, the Project will be evaluated without codegen - df.unpersist() + df.unpersist(blocking = true) nullTest() // Test with cached relation, the Project will be evaluated with codegen df.cache() @@ -2249,6 +2246,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { test("exists function - array for primitive type containing null") { val df = Seq[Seq[Integer]]( Seq(1, 9, 8, null, 7), + Seq(1, 3, 5), Seq(5, null, null, 9, 7, null), Seq.empty, null @@ -2259,6 +2257,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { Seq( Row(true), Row(false), + Row(null), Row(false), Row(null))) } @@ -2591,7 +2590,7 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { val ex3 = intercept[Exception] { dfExample1.selectExpr("transform_keys(i, (k, v) -> v)").show() } - assert(ex3.getMessage.contains("Cannot use null as map key!")) + assert(ex3.getMessage.contains("Cannot use null as map key")) val ex4 = intercept[AnalysisException] { dfExample2.selectExpr("transform_keys(j, (k, v) -> k + 1)") @@ -2887,7 +2886,9 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { ("coalesce", (df: DataFrame) => df.select(coalesce())) :: ("coalesce", (df: DataFrame) => df.selectExpr("coalesce()")) :: ("hash", (df: DataFrame) => df.select(hash())) :: - ("hash", (df: DataFrame) => df.selectExpr("hash()")) :: Nil + ("hash", (df: DataFrame) => df.selectExpr("hash()")) :: + ("xxhash64", (df: DataFrame) => df.select(xxhash64())) :: + ("xxhash64", (df: DataFrame) => df.selectExpr("xxhash64()")) :: Nil funcsMustHaveAtLeastOneArg.foreach { case (name, func) => val errMsg = intercept[AnalysisException] { func(df) }.getMessage assert(errMsg.contains(s"input to function $name requires at least one argument")) @@ -2911,6 +2912,26 @@ class DataFrameFunctionsSuite extends QueryTest with SharedSQLContext { } assert(ex.getMessage.contains("Cannot use null as map key")) } + + test("SPARK-26370: Fix resolution of higher-order function for the same identifier") { + val df = Seq( + (Seq(1, 9, 8, 7), 1, 2), + (Seq(5, 9, 7), 2, 2), + (Seq.empty, 3, 2), + (null, 4, 2) + ).toDF("i", "x", "d") + + checkAnswer(df.selectExpr("x", "exists(i, x -> x % d == 0)"), + Seq( + Row(1, true), + Row(2, false), + Row(3, false), + Row(4, null))) + checkAnswer(df.filter("exists(i, x -> x % d == 0)"), + Seq(Row(Seq(1, 9, 8, 7), 1, 2))) + checkAnswer(df.select("x").filter("exists(i, x -> x % d == 0)"), + Seq(Row(1))) + } } object DataFrameFunctionsSuite { 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 e6b30f9956daf..ba120dca712d8 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 @@ -84,6 +84,18 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { checkAnswer( df.join(df2, Seq("int", "str"), "left_semi"), Row(1, "1", 2) :: Nil) + + checkAnswer( + df.join(df2, Seq("int", "str"), "semi"), + Row(1, "1", 2) :: Nil) + + checkAnswer( + df.join(df2, Seq("int", "str"), "left_anti"), + Row(3, "3", 4) :: Nil) + + checkAnswer( + df.join(df2, Seq("int", "str"), "anti"), + Row(3, "3", 4) :: Nil) } test("join - join using self join") { @@ -198,7 +210,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // outer -> left val outerJoin2Left = df.join(df2, $"a.int" === $"b.int", "outer").where($"a.int" >= 3) assert(outerJoin2Left.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, LeftOuter, _) => j }.size === 1) + case j @ Join(_, _, LeftOuter, _, _) => j }.size === 1) checkAnswer( outerJoin2Left, Row(3, 4, "3", null, null, null) :: Nil) @@ -206,7 +218,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // outer -> right val outerJoin2Right = df.join(df2, $"a.int" === $"b.int", "outer").where($"b.int" >= 3) assert(outerJoin2Right.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, RightOuter, _) => j }.size === 1) + case j @ Join(_, _, RightOuter, _, _) => j }.size === 1) checkAnswer( outerJoin2Right, Row(null, null, null, 5, 6, "5") :: Nil) @@ -215,7 +227,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { val outerJoin2Inner = df.join(df2, $"a.int" === $"b.int", "outer"). where($"a.int" === 1 && $"b.int2" === 3) assert(outerJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( outerJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) @@ -223,7 +235,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // right -> inner val rightJoin2Inner = df.join(df2, $"a.int" === $"b.int", "right").where($"a.int" > 0) assert(rightJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( rightJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) @@ -231,7 +243,7 @@ class DataFrameJoinSuite extends QueryTest with SharedSQLContext { // left -> inner val leftJoin2Inner = df.join(df2, $"a.int" === $"b.int", "left").where($"b.int2" > 0) assert(leftJoin2Inner.queryExecution.optimizedPlan.collect { - case j @ Join(_, _, Inner, _) => j }.size === 1) + case j @ Join(_, _, Inner, _, _) => j }.size === 1) checkAnswer( leftJoin2Inner, Row(1, 2, "1", 1, 3, "1") :: Nil) 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 b52ca58c07d27..8c2c11be9b6fd 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 @@ -333,4 +333,15 @@ class DataFramePivotSuite extends QueryTest with SharedSQLContext { } assert(exception.getMessage.contains("Unsupported literal type")) } + + test("SPARK-26403: pivoting by array column") { + val df = Seq( + (2, Seq.empty[String]), + (2, Seq("a", "x")), + (3, Seq.empty[String]), + (3, Seq("a", "x"))).toDF("x", "s") + val expected = Seq((3, 1, 1), (2, 1, 1)).toDF + val actual = df.groupBy("x").pivot("s").count() + checkAnswer(actual, expected) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala index b0b46640ff317..8cc7020579431 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameRangeSuite.scala @@ -31,7 +31,6 @@ import org.apache.spark.sql.test.SharedSQLContext class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventually { - import testImplicits._ test("SPARK-7150 range api") { // numSlice is greater than length @@ -107,7 +106,7 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall assert(res17.collect === (1 to 10).map(i => Row(i)).toArray) } - test("Range with randomized parameters") { + testWithWholeStageCodegenOnAndOff("Range with randomized parameters") { codegenEnabled => val MAX_NUM_STEPS = 10L * 1000 val seed = System.currentTimeMillis() @@ -133,25 +132,21 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall val expCount = (start until end by step).size val expSum = (start until end by step).sum - for (codegen <- List(false, true)) { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) { - val res = spark.range(start, end, step, partitions).toDF("id"). - agg(count("id"), sum("id")).collect() - - withClue(s"seed = $seed start = $start end = $end step = $step partitions = " + - s"$partitions codegen = $codegen") { - assert(!res.isEmpty) - assert(res.head.getLong(0) == expCount) - if (expCount > 0) { - assert(res.head.getLong(1) == expSum) - } - } + val res = spark.range(start, end, step, partitions).toDF("id"). + agg(count("id"), sum("id")).collect() + + withClue(s"seed = $seed start = $start end = $end step = $step partitions = " + + s"$partitions codegen = $codegenEnabled") { + assert(!res.isEmpty) + assert(res.head.getLong(0) == expCount) + if (expCount > 0) { + assert(res.head.getLong(1) == expSum) } } } } - test("Cancelling stage in a query with Range.") { + testWithWholeStageCodegenOnAndOff("Cancelling stage in a query with Range.") { _ => val listener = new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart): Unit = { sparkContext.cancelStage(taskStart.stageId) @@ -159,27 +154,25 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall } sparkContext.addSparkListener(listener) - for (codegen <- Seq(true, false)) { - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegen.toString()) { - val ex = intercept[SparkException] { - spark.range(0, 100000000000L, 1, 1) - .toDF("id").agg(sum("id")).collect() - } - ex.getCause() match { - case null => - assert(ex.getMessage().contains("cancelled")) - case cause: SparkException => - assert(cause.getMessage().contains("cancelled")) - case cause: Throwable => - fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.") - } - } - // Wait until all ListenerBus events consumed to make sure cancelStage called for all stages - sparkContext.listenerBus.waitUntilEmpty(20.seconds.toMillis) - eventually(timeout(20.seconds)) { - assert(sparkContext.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) - } + val ex = intercept[SparkException] { + spark.range(0, 100000000000L, 1, 1) + .toDF("id").agg(sum("id")).collect() + } + ex.getCause() match { + case null => + assert(ex.getMessage().contains("cancelled")) + case cause: SparkException => + assert(cause.getMessage().contains("cancelled")) + case cause: Throwable => + fail("Expected the cause to be SparkException, got " + cause.toString() + " instead.") } + + // Wait until all ListenerBus events consumed to make sure cancelStage called for all stages + sparkContext.listenerBus.waitUntilEmpty(20.seconds.toMillis) + eventually(timeout(20.seconds)) { + assert(sparkContext.statusTracker.getExecutorInfos.map(_.numRunningTasks()).sum == 0) + } + sparkContext.removeSparkListener(listener) } @@ -189,14 +182,11 @@ class DataFrameRangeSuite extends QueryTest with SharedSQLContext with Eventuall } } - test("SPARK-21041 SparkSession.range()'s behavior is inconsistent with SparkContext.range()") { + testWithWholeStageCodegenOnAndOff("SPARK-21041 SparkSession.range()'s behavior is " + + "inconsistent with SparkContext.range()") { _ => val start = java.lang.Long.MAX_VALUE - 3 val end = java.lang.Long.MIN_VALUE + 2 - Seq("false", "true").foreach { value => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> value) { - assert(spark.range(start, end, 1).collect.length == 0) - assert(spark.range(start, start, 1).collect.length == 0) - } - } + assert(spark.range(start, end, 1).collect.length == 0) + assert(spark.range(start, start, 1).collect.length == 0) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala new file mode 100644 index 0000000000000..30452af1fad64 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameSetOperationsSuite.scala @@ -0,0 +1,509 @@ +/* + * 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 java.sql.{Date, Timestamp} + +import org.apache.spark.sql.catalyst.plans.logical.Union +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} +import org.apache.spark.sql.test.SQLTestData.NullStrings +import org.apache.spark.sql.types._ + +class DataFrameSetOperationsSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("except") { + checkAnswer( + lowerCaseData.except(upperCaseData), + Row(1, "a") :: + Row(2, "b") :: + Row(3, "c") :: + Row(4, "d") :: Nil) + checkAnswer(lowerCaseData.except(lowerCaseData), Nil) + checkAnswer(upperCaseData.except(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.except(nullInts.filter("0 = 1")), + nullInts) + checkAnswer( + nullInts.except(nullInts), + Nil) + + // check if values are de-duplicated + checkAnswer( + allNulls.except(allNulls.filter("0 = 1")), + Row(null) :: Nil) + checkAnswer( + allNulls.except(allNulls), + Nil) + + // check if values are de-duplicated + val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") + checkAnswer( + df.except(df.filter("0 = 1")), + Row("id1", 1) :: + Row("id", 1) :: + Row("id1", 2) :: Nil) + + // check if the empty set on the left side works + checkAnswer( + allNulls.filter("0 = 1").except(allNulls), + Nil) + } + + test("SPARK-23274: except between two projects without references used in filter") { + val df = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") + val df1 = df.filter($"a" === 1) + val df2 = df.filter($"a" === 2) + checkAnswer(df1.select("b").except(df2.select("b")), Row(3) :: Nil) + checkAnswer(df1.select("b").except(df2.select("c")), Row(2) :: Nil) + } + + test("except distinct - SQL compliance") { + val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id") + val df_right = Seq(1, 3).toDF("id") + + checkAnswer( + df_left.except(df_right), + Row(2) :: Row(4) :: Nil + ) + } + + test("except - nullability") { + val nonNullableInts = Seq(Tuple1(11), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.except(nullInts) + checkAnswer(df1, Row(11) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.except(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(2) :: Row(null) :: Nil) + assert(df2.schema.forall(_.nullable)) + + val df3 = nullInts.except(nullInts) + checkAnswer(df3, Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.except(nonNullableInts) + checkAnswer(df4, Nil) + assert(df4.schema.forall(!_.nullable)) + } + + test("except all") { + checkAnswer( + lowerCaseData.exceptAll(upperCaseData), + Row(1, "a") :: + Row(2, "b") :: + Row(3, "c") :: + Row(4, "d") :: Nil) + checkAnswer(lowerCaseData.exceptAll(lowerCaseData), Nil) + checkAnswer(upperCaseData.exceptAll(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.exceptAll(nullInts.filter("0 = 1")), + nullInts) + checkAnswer( + nullInts.exceptAll(nullInts), + Nil) + + // check that duplicate values are preserved + checkAnswer( + allNulls.exceptAll(allNulls.filter("0 = 1")), + Row(null) :: Row(null) :: Row(null) :: Row(null) :: Nil) + checkAnswer( + allNulls.exceptAll(allNulls.limit(2)), + Row(null) :: Row(null) :: Nil) + + // check that duplicates are retained. + val df = spark.sparkContext.parallelize( + NullStrings(1, "id1") :: + NullStrings(1, "id1") :: + NullStrings(2, "id1") :: + NullStrings(3, null) :: Nil).toDF("id", "value") + + checkAnswer( + df.exceptAll(df.filter("0 = 1")), + Row(1, "id1") :: + Row(1, "id1") :: + Row(2, "id1") :: + Row(3, null) :: Nil) + + // check if the empty set on the left side works + checkAnswer( + allNulls.filter("0 = 1").exceptAll(allNulls), + Nil) + + } + + test("exceptAll - nullability") { + val nonNullableInts = Seq(Tuple1(11), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.exceptAll(nullInts) + checkAnswer(df1, Row(11) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.exceptAll(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(2) :: Row(null) :: Nil) + assert(df2.schema.forall(_.nullable)) + + val df3 = nullInts.exceptAll(nullInts) + checkAnswer(df3, Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.exceptAll(nonNullableInts) + checkAnswer(df4, Nil) + assert(df4.schema.forall(!_.nullable)) + } + + test("intersect") { + checkAnswer( + lowerCaseData.intersect(lowerCaseData), + Row(1, "a") :: + Row(2, "b") :: + Row(3, "c") :: + Row(4, "d") :: Nil) + checkAnswer(lowerCaseData.intersect(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.intersect(nullInts), + Row(1) :: + Row(2) :: + Row(3) :: + Row(null) :: Nil) + + // check if values are de-duplicated + checkAnswer( + allNulls.intersect(allNulls), + Row(null) :: Nil) + + // check if values are de-duplicated + val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") + checkAnswer( + df.intersect(df), + Row("id1", 1) :: + Row("id", 1) :: + Row("id1", 2) :: Nil) + } + + test("intersect - nullability") { + val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.intersect(nullInts) + checkAnswer(df1, Row(1) :: Row(3) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.intersect(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(3) :: Nil) + assert(df2.schema.forall(!_.nullable)) + + val df3 = nullInts.intersect(nullInts) + checkAnswer(df3, Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.intersect(nonNullableInts) + checkAnswer(df4, Row(1) :: Row(3) :: Nil) + assert(df4.schema.forall(!_.nullable)) + } + + test("intersectAll") { + checkAnswer( + lowerCaseDataWithDuplicates.intersectAll(lowerCaseDataWithDuplicates), + Row(1, "a") :: + Row(2, "b") :: + Row(2, "b") :: + Row(3, "c") :: + Row(3, "c") :: + Row(3, "c") :: + Row(4, "d") :: Nil) + checkAnswer(lowerCaseData.intersectAll(upperCaseData), Nil) + + // check null equality + checkAnswer( + nullInts.intersectAll(nullInts), + Row(1) :: + Row(2) :: + Row(3) :: + Row(null) :: Nil) + + // Duplicate nulls are preserved. + checkAnswer( + allNulls.intersectAll(allNulls), + Row(null) :: Row(null) :: Row(null) :: Row(null) :: Nil) + + val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id") + val df_right = Seq(1, 2, 2, 3).toDF("id") + + checkAnswer( + df_left.intersectAll(df_right), + Row(1) :: Row(2) :: Row(2) :: Row(3) :: Nil) + } + + test("intersectAll - nullability") { + val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF() + assert(nonNullableInts.schema.forall(!_.nullable)) + + val df1 = nonNullableInts.intersectAll(nullInts) + checkAnswer(df1, Row(1) :: Row(3) :: Nil) + assert(df1.schema.forall(!_.nullable)) + + val df2 = nullInts.intersectAll(nonNullableInts) + checkAnswer(df2, Row(1) :: Row(3) :: Nil) + assert(df2.schema.forall(!_.nullable)) + + val df3 = nullInts.intersectAll(nullInts) + checkAnswer(df3, Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) + assert(df3.schema.forall(_.nullable)) + + val df4 = nonNullableInts.intersectAll(nonNullableInts) + checkAnswer(df4, Row(1) :: Row(3) :: Nil) + assert(df4.schema.forall(!_.nullable)) + } + + test("SPARK-10539: Project should not be pushed down through Intersect or Except") { + val df1 = (1 to 100).map(Tuple1.apply).toDF("i") + val df2 = (1 to 30).map(Tuple1.apply).toDF("i") + val intersect = df1.intersect(df2) + val except = df1.except(df2) + assert(intersect.count() === 30) + assert(except.count() === 70) + } + + test("SPARK-10740: handle nondeterministic expressions correctly for set operations") { + val df1 = (1 to 20).map(Tuple1.apply).toDF("i") + val df2 = (1 to 10).map(Tuple1.apply).toDF("i") + + // When generating expected results at here, we need to follow the implementation of + // Rand expression. + def expected(df: DataFrame): Seq[Row] = { + df.rdd.collectPartitions().zipWithIndex.flatMap { + case (data, index) => + val rng = new org.apache.spark.util.random.XORShiftRandom(7 + index) + data.filter(_.getInt(0) < rng.nextDouble() * 10) + } + } + + val union = df1.union(df2) + checkAnswer( + union.filter('i < rand(7) * 10), + expected(union) + ) + checkAnswer( + union.select(rand(7)), + union.rdd.collectPartitions().zipWithIndex.flatMap { + case (data, index) => + val rng = new org.apache.spark.util.random.XORShiftRandom(7 + index) + data.map(_ => rng.nextDouble()).map(i => Row(i)) + } + ) + + val intersect = df1.intersect(df2) + checkAnswer( + intersect.filter('i < rand(7) * 10), + expected(intersect) + ) + + val except = df1.except(df2) + checkAnswer( + except.filter('i < rand(7) * 10), + expected(except) + ) + } + + test("SPARK-17123: Performing set operations that combine non-scala native types") { + val dates = Seq( + (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), + (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) + ).toDF("date", "timestamp", "decimal") + + val widenTypedRows = Seq( + (new Timestamp(2), 10.5D, "string") + ).toDF("date", "timestamp", "decimal") + + dates.union(widenTypedRows).collect() + dates.except(widenTypedRows).collect() + dates.intersect(widenTypedRows).collect() + } + + test("SPARK-19893: cannot run set operations with map type") { + val df = spark.range(1).select(map(lit("key"), $"id").as("m")) + val e = intercept[AnalysisException](df.intersect(df)) + assert(e.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + val e2 = intercept[AnalysisException](df.except(df)) + assert(e2.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + val e3 = intercept[AnalysisException](df.distinct()) + assert(e3.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + withTempView("v") { + df.createOrReplaceTempView("v") + val e4 = intercept[AnalysisException](sql("SELECT DISTINCT m FROM v")) + assert(e4.message.contains( + "Cannot have map type columns in DataFrame which calls set operations")) + } + } + + test("union all") { + val unionDF = testData.union(testData).union(testData) + .union(testData).union(testData) + + // Before optimizer, Union should be combined. + assert(unionDF.queryExecution.analyzed.collect { + case j: Union if j.children.size == 5 => j }.size === 1) + + checkAnswer( + unionDF.agg(avg('key), max('key), min('key), sum('key)), + Row(50.5, 100, 1, 25250) :: Nil + ) + + // unionAll is an alias of union + val unionAllDF = testData.unionAll(testData).unionAll(testData) + .unionAll(testData).unionAll(testData) + + checkAnswer(unionDF, unionAllDF) + } + + test("union should union DataFrames with UDTs (SPARK-13410)") { + val rowRDD1 = sparkContext.parallelize(Seq(Row(1, new ExamplePoint(1.0, 2.0)))) + val schema1 = StructType(Array(StructField("label", IntegerType, false), + StructField("point", new ExamplePointUDT(), false))) + val rowRDD2 = sparkContext.parallelize(Seq(Row(2, new ExamplePoint(3.0, 4.0)))) + val schema2 = StructType(Array(StructField("label", IntegerType, false), + StructField("point", new ExamplePointUDT(), false))) + val df1 = spark.createDataFrame(rowRDD1, schema1) + val df2 = spark.createDataFrame(rowRDD2, schema2) + + checkAnswer( + df1.union(df2).orderBy("label"), + Seq(Row(1, new ExamplePoint(1.0, 2.0)), Row(2, new ExamplePoint(3.0, 4.0))) + ) + } + + test("union by name") { + var df1 = Seq((1, 2, 3)).toDF("a", "b", "c") + var df2 = Seq((3, 1, 2)).toDF("c", "a", "b") + val df3 = Seq((2, 3, 1)).toDF("b", "c", "a") + val unionDf = df1.unionByName(df2.unionByName(df3)) + checkAnswer(unionDf, + Row(1, 2, 3) :: Row(1, 2, 3) :: Row(1, 2, 3) :: Nil + ) + + // Check if adjacent unions are combined into a single one + assert(unionDf.queryExecution.optimizedPlan.collect { case u: Union => true }.size == 1) + + // Check failure cases + df1 = Seq((1, 2)).toDF("a", "c") + df2 = Seq((3, 4, 5)).toDF("a", "b", "c") + var errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains( + "Union can only be performed on tables with the same number of columns, " + + "but the first table has 2 columns and the second table has 3 columns")) + + df1 = Seq((1, 2, 3)).toDF("a", "b", "c") + df2 = Seq((4, 5, 6)).toDF("a", "c", "d") + errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("""Cannot resolve column name "b" among (a, c, d)""")) + } + + test("union by name - type coercion") { + var df1 = Seq((1, "a")).toDF("c0", "c1") + var df2 = Seq((3, 1L)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(1L, "a") :: Row(1L, "3") :: Nil) + + df1 = Seq((1, 1.0)).toDF("c0", "c1") + df2 = Seq((8L, 3.0)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(1.0, 1.0) :: Row(3.0, 8.0) :: Nil) + + df1 = Seq((2.0f, 7.4)).toDF("c0", "c1") + df2 = Seq(("a", 4.0)).toDF("c1", "c0") + checkAnswer(df1.unionByName(df2), Row(2.0, "7.4") :: Row(4.0, "a") :: Nil) + + df1 = Seq((1, "a", 3.0)).toDF("c0", "c1", "c2") + df2 = Seq((1.2, 2, "bc")).toDF("c2", "c0", "c1") + val df3 = Seq(("def", 1.2, 3)).toDF("c1", "c2", "c0") + checkAnswer(df1.unionByName(df2.unionByName(df3)), + Row(1, "a", 3.0) :: Row(2, "bc", 1.2) :: Row(3, "def", 1.2) :: Nil + ) + } + + test("union by name - check case sensitivity") { + def checkCaseSensitiveTest(): Unit = { + val df1 = Seq((1, 2, 3)).toDF("ab", "cd", "ef") + val df2 = Seq((4, 5, 6)).toDF("cd", "ef", "AB") + checkAnswer(df1.unionByName(df2), Row(1, 2, 3) :: Row(6, 4, 5) :: Nil) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val errMsg2 = intercept[AnalysisException] { + checkCaseSensitiveTest() + }.getMessage + assert(errMsg2.contains("""Cannot resolve column name "ab" among (cd, ef, AB)""")) + } + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + checkCaseSensitiveTest() + } + } + + test("union by name - check name duplication") { + Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => + withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { + var df1 = Seq((1, 1)).toDF(c0, c1) + var df2 = Seq((1, 1)).toDF("c0", "c1") + var errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the left attributes:")) + df1 = Seq((1, 1)).toDF("c0", "c1") + df2 = Seq((1, 1)).toDF(c0, c1) + errMsg = intercept[AnalysisException] { + df1.unionByName(df2) + }.getMessage + assert(errMsg.contains("Found duplicate column(s) in the right attributes:")) + } + } + } + + test("SPARK-25368 Incorrect predicate pushdown returns wrong result") { + def check(newCol: Column, filter: Column, result: Seq[Row]): Unit = { + val df1 = spark.createDataFrame(Seq( + (1, 1) + )).toDF("a", "b").withColumn("c", newCol) + + val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter(filter) + checkAnswer(df2, result) + } + + check(lit(null).cast("int"), $"c".isNull, Seq(Row(1, 1, null, 0), Row(1, 1, null, 1))) + check(lit(null).cast("int"), $"c".isNotNull, Seq()) + check(lit(2).cast("int"), $"c".isNull, Seq()) + check(lit(2).cast("int"), $"c".isNotNull, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) + check(lit(2).cast("int"), $"c" === 2, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) + check(lit(2).cast("int"), $"c" =!= 2, Seq()) + } +} 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 589873b9c3ea4..2a74bfe4d3789 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 @@ -47,7 +47,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { val data = sparkContext.parallelize(1 to n, 2).toDF("id") checkAnswer( data.sample(withReplacement = false, 0.05, seed = 13), - Seq(3, 17, 27, 58, 62).map(Row(_)) + Seq(37, 8, 90).map(Row(_)) ) } @@ -371,7 +371,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { val sampled = df.stat.sampleBy("key", Map(0 -> 0.1, 1 -> 0.2), 0L) checkAnswer( sampled.groupBy("key").count().orderBy("key"), - Seq(Row(0, 6), Row(1, 11))) + Seq(Row(0, 1), Row(1, 6))) } test("sampleBy one column") { @@ -379,7 +379,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { val sampled = df.stat.sampleBy($"key", Map(0 -> 0.1, 1 -> 0.2), 0L) checkAnswer( sampled.groupBy("key").count().orderBy("key"), - Seq(Row(0, 6), Row(1, 11))) + Seq(Row(0, 1), Row(1, 6))) } test("sampleBy multiple columns") { @@ -389,7 +389,7 @@ class DataFrameStatSuite extends QueryTest with SharedSQLContext { struct($"name", $"key"), Map(Row("Foo", 0) -> 0.1, Row("Foo", 1) -> 0.2), 0L) checkAnswer( sampled.groupBy("key").count().orderBy("key"), - Seq(Row(0, 6), Row(1, 11))) + Seq(Row(0, 1), Row(1, 6))) } // This test case only verifies that `DataFrame.countMinSketch()` methods do return 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 c0b277f76ae68..e8ddd4e1fd974 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql -import java.io.File +import java.io.{ByteArrayOutputStream, File} import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util.UUID @@ -30,14 +30,15 @@ import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerJobEnd} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.Uuid -import org.apache.spark.sql.catalyst.plans.logical.{Filter, OneRowRelation, Union} +import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation +import org.apache.spark.sql.catalyst.plans.logical.{OneRowRelation, Union} import org.apache.spark.sql.execution.{FilterExec, QueryExecution, WholeStageCodegenExec} import org.apache.spark.sql.execution.aggregate.HashAggregateExec import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ReusedExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} -import org.apache.spark.sql.test.SQLTestData.{NullInts, NullStrings, TestData2} +import org.apache.spark.sql.test.SQLTestData.{NullStrings, TestData2} import org.apache.spark.sql.types._ import org.apache.spark.util.Utils import org.apache.spark.util.random.XORShiftRandom @@ -84,123 +85,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { testData.collect().toSeq) } - test("union all") { - val unionDF = testData.union(testData).union(testData) - .union(testData).union(testData) - - // Before optimizer, Union should be combined. - assert(unionDF.queryExecution.analyzed.collect { - case j: Union if j.children.size == 5 => j }.size === 1) - - checkAnswer( - unionDF.agg(avg('key), max('key), min('key), sum('key)), - Row(50.5, 100, 1, 25250) :: Nil - ) - } - - test("union should union DataFrames with UDTs (SPARK-13410)") { - val rowRDD1 = sparkContext.parallelize(Seq(Row(1, new ExamplePoint(1.0, 2.0)))) - val schema1 = StructType(Array(StructField("label", IntegerType, false), - StructField("point", new ExamplePointUDT(), false))) - val rowRDD2 = sparkContext.parallelize(Seq(Row(2, new ExamplePoint(3.0, 4.0)))) - val schema2 = StructType(Array(StructField("label", IntegerType, false), - StructField("point", new ExamplePointUDT(), false))) - val df1 = spark.createDataFrame(rowRDD1, schema1) - val df2 = spark.createDataFrame(rowRDD2, schema2) - - checkAnswer( - df1.union(df2).orderBy("label"), - Seq(Row(1, new ExamplePoint(1.0, 2.0)), Row(2, new ExamplePoint(3.0, 4.0))) - ) - } - - test("union by name") { - var df1 = Seq((1, 2, 3)).toDF("a", "b", "c") - var df2 = Seq((3, 1, 2)).toDF("c", "a", "b") - val df3 = Seq((2, 3, 1)).toDF("b", "c", "a") - val unionDf = df1.unionByName(df2.unionByName(df3)) - checkAnswer(unionDf, - Row(1, 2, 3) :: Row(1, 2, 3) :: Row(1, 2, 3) :: Nil - ) - - // Check if adjacent unions are combined into a single one - assert(unionDf.queryExecution.optimizedPlan.collect { case u: Union => true }.size == 1) - - // Check failure cases - df1 = Seq((1, 2)).toDF("a", "c") - df2 = Seq((3, 4, 5)).toDF("a", "b", "c") - var errMsg = intercept[AnalysisException] { - df1.unionByName(df2) - }.getMessage - assert(errMsg.contains( - "Union can only be performed on tables with the same number of columns, " + - "but the first table has 2 columns and the second table has 3 columns")) - - df1 = Seq((1, 2, 3)).toDF("a", "b", "c") - df2 = Seq((4, 5, 6)).toDF("a", "c", "d") - errMsg = intercept[AnalysisException] { - df1.unionByName(df2) - }.getMessage - assert(errMsg.contains("""Cannot resolve column name "b" among (a, c, d)""")) - } - - test("union by name - type coercion") { - var df1 = Seq((1, "a")).toDF("c0", "c1") - var df2 = Seq((3, 1L)).toDF("c1", "c0") - checkAnswer(df1.unionByName(df2), Row(1L, "a") :: Row(1L, "3") :: Nil) - - df1 = Seq((1, 1.0)).toDF("c0", "c1") - df2 = Seq((8L, 3.0)).toDF("c1", "c0") - checkAnswer(df1.unionByName(df2), Row(1.0, 1.0) :: Row(3.0, 8.0) :: Nil) - - df1 = Seq((2.0f, 7.4)).toDF("c0", "c1") - df2 = Seq(("a", 4.0)).toDF("c1", "c0") - checkAnswer(df1.unionByName(df2), Row(2.0, "7.4") :: Row(4.0, "a") :: Nil) - - df1 = Seq((1, "a", 3.0)).toDF("c0", "c1", "c2") - df2 = Seq((1.2, 2, "bc")).toDF("c2", "c0", "c1") - val df3 = Seq(("def", 1.2, 3)).toDF("c1", "c2", "c0") - checkAnswer(df1.unionByName(df2.unionByName(df3)), - Row(1, "a", 3.0) :: Row(2, "bc", 1.2) :: Row(3, "def", 1.2) :: Nil - ) - } - - test("union by name - check case sensitivity") { - def checkCaseSensitiveTest(): Unit = { - val df1 = Seq((1, 2, 3)).toDF("ab", "cd", "ef") - val df2 = Seq((4, 5, 6)).toDF("cd", "ef", "AB") - checkAnswer(df1.unionByName(df2), Row(1, 2, 3) :: Row(6, 4, 5) :: Nil) - } - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { - val errMsg2 = intercept[AnalysisException] { - checkCaseSensitiveTest() - }.getMessage - assert(errMsg2.contains("""Cannot resolve column name "ab" among (cd, ef, AB)""")) - } - withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { - checkCaseSensitiveTest() - } - } - - test("union by name - check name duplication") { - Seq((true, ("a", "a")), (false, ("aA", "Aa"))).foreach { case (caseSensitive, (c0, c1)) => - withSQLConf(SQLConf.CASE_SENSITIVE.key -> caseSensitive.toString) { - var df1 = Seq((1, 1)).toDF(c0, c1) - var df2 = Seq((1, 1)).toDF("c0", "c1") - var errMsg = intercept[AnalysisException] { - df1.unionByName(df2) - }.getMessage - assert(errMsg.contains("Found duplicate column(s) in the left attributes:")) - df1 = Seq((1, 1)).toDF("c0", "c1") - df2 = Seq((1, 1)).toDF(c0, c1) - errMsg = intercept[AnalysisException] { - df1.unionByName(df2) - }.getMessage - assert(errMsg.contains("Found duplicate column(s) in the right attributes:")) - } - } - } - test("empty data frame") { assert(spark.emptyDataFrame.columns.toSeq === Seq.empty[String]) assert(spark.emptyDataFrame.count() === 0) @@ -219,31 +103,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { 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 @@ -278,25 +137,38 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { structDf.select(hash($"a", $"record.*"))) } + test("Star Expansion - xxhash64") { + val structDf = testData2.select("a", "b").as("record") + checkAnswer( + structDf.groupBy($"a", $"b").agg(min(xxhash64($"a", $"*"))), + structDf.groupBy($"a", $"b").agg(min(xxhash64($"a", $"a", $"b")))) + + checkAnswer( + structDf.groupBy($"a", $"b").agg(xxhash64($"a", $"*")), + structDf.groupBy($"a", $"b").agg(xxhash64($"a", $"a", $"b"))) + + checkAnswer( + structDf.select(xxhash64($"*")), + structDf.select(xxhash64($"record.*"))) + + checkAnswer( + structDf.select(xxhash64($"a", $"*")), + structDf.select(xxhash64($"a", $"record.*"))) + } + test("Star Expansion - 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 df = Seq(("1,2"), ("4"), ("7,8,9")).toDF("csv") val e = intercept[AnalysisException] { - df.explode($"*") { case Row(prefix: String, csv: String) => - csv.split(",").map(v => Tuple1(prefix + ":" + v)).toSeq - }.queryExecution.assertAnalyzed() + df.select(explode($"*")) } - assert(e.getMessage.contains("Invalid usage of '*' in explode/json_tuple/UDTF")) + assert(e.getMessage.contains("Invalid usage of '*' in expression 'explode'")) + } + test("explode on output of array-valued function") { + val df = Seq(("1,2"), ("4"), ("7,8,9")).toDF("csv") 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) + df.select(explode(split($"csv", ","))), + Row("1") :: Row("2") :: Row("4") :: Row("7") :: Row("8") :: Row("9") :: Nil) } test("Star Expansion - explode alias and star") { @@ -552,259 +424,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { ) } - test("except") { - checkAnswer( - lowerCaseData.except(upperCaseData), - Row(1, "a") :: - Row(2, "b") :: - Row(3, "c") :: - Row(4, "d") :: Nil) - checkAnswer(lowerCaseData.except(lowerCaseData), Nil) - checkAnswer(upperCaseData.except(upperCaseData), Nil) - - // check null equality - checkAnswer( - nullInts.except(nullInts.filter("0 = 1")), - nullInts) - checkAnswer( - nullInts.except(nullInts), - Nil) - - // check if values are de-duplicated - checkAnswer( - allNulls.except(allNulls.filter("0 = 1")), - Row(null) :: Nil) - checkAnswer( - allNulls.except(allNulls), - Nil) - - // check if values are de-duplicated - val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") - checkAnswer( - df.except(df.filter("0 = 1")), - Row("id1", 1) :: - Row("id", 1) :: - Row("id1", 2) :: Nil) - - // check if the empty set on the left side works - checkAnswer( - allNulls.filter("0 = 1").except(allNulls), - Nil) - } - - test("SPARK-23274: except between two projects without references used in filter") { - val df = Seq((1, 2, 4), (1, 3, 5), (2, 2, 3), (2, 4, 5)).toDF("a", "b", "c") - val df1 = df.filter($"a" === 1) - val df2 = df.filter($"a" === 2) - checkAnswer(df1.select("b").except(df2.select("b")), Row(3) :: Nil) - checkAnswer(df1.select("b").except(df2.select("c")), Row(2) :: Nil) - } - - test("except distinct - SQL compliance") { - val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id") - val df_right = Seq(1, 3).toDF("id") - - checkAnswer( - df_left.except(df_right), - Row(2) :: Row(4) :: Nil - ) - } - - test("except - nullability") { - val nonNullableInts = Seq(Tuple1(11), Tuple1(3)).toDF() - assert(nonNullableInts.schema.forall(!_.nullable)) - - val df1 = nonNullableInts.except(nullInts) - checkAnswer(df1, Row(11) :: Nil) - assert(df1.schema.forall(!_.nullable)) - - val df2 = nullInts.except(nonNullableInts) - checkAnswer(df2, Row(1) :: Row(2) :: Row(null) :: Nil) - assert(df2.schema.forall(_.nullable)) - - val df3 = nullInts.except(nullInts) - checkAnswer(df3, Nil) - assert(df3.schema.forall(_.nullable)) - - val df4 = nonNullableInts.except(nonNullableInts) - checkAnswer(df4, Nil) - assert(df4.schema.forall(!_.nullable)) - } - - test("except all") { - checkAnswer( - lowerCaseData.exceptAll(upperCaseData), - Row(1, "a") :: - Row(2, "b") :: - Row(3, "c") :: - Row(4, "d") :: Nil) - checkAnswer(lowerCaseData.exceptAll(lowerCaseData), Nil) - checkAnswer(upperCaseData.exceptAll(upperCaseData), Nil) - - // check null equality - checkAnswer( - nullInts.exceptAll(nullInts.filter("0 = 1")), - nullInts) - checkAnswer( - nullInts.exceptAll(nullInts), - Nil) - - // check that duplicate values are preserved - checkAnswer( - allNulls.exceptAll(allNulls.filter("0 = 1")), - Row(null) :: Row(null) :: Row(null) :: Row(null) :: Nil) - checkAnswer( - allNulls.exceptAll(allNulls.limit(2)), - Row(null) :: Row(null) :: Nil) - - // check that duplicates are retained. - val df = spark.sparkContext.parallelize( - NullStrings(1, "id1") :: - NullStrings(1, "id1") :: - NullStrings(2, "id1") :: - NullStrings(3, null) :: Nil).toDF("id", "value") - - checkAnswer( - df.exceptAll(df.filter("0 = 1")), - Row(1, "id1") :: - Row(1, "id1") :: - Row(2, "id1") :: - Row(3, null) :: Nil) - - // check if the empty set on the left side works - checkAnswer( - allNulls.filter("0 = 1").exceptAll(allNulls), - Nil) - - } - - test("exceptAll - nullability") { - val nonNullableInts = Seq(Tuple1(11), Tuple1(3)).toDF() - assert(nonNullableInts.schema.forall(!_.nullable)) - - val df1 = nonNullableInts.exceptAll(nullInts) - checkAnswer(df1, Row(11) :: Nil) - assert(df1.schema.forall(!_.nullable)) - - val df2 = nullInts.exceptAll(nonNullableInts) - checkAnswer(df2, Row(1) :: Row(2) :: Row(null) :: Nil) - assert(df2.schema.forall(_.nullable)) - - val df3 = nullInts.exceptAll(nullInts) - checkAnswer(df3, Nil) - assert(df3.schema.forall(_.nullable)) - - val df4 = nonNullableInts.exceptAll(nonNullableInts) - checkAnswer(df4, Nil) - assert(df4.schema.forall(!_.nullable)) - } - - test("intersect") { - checkAnswer( - lowerCaseData.intersect(lowerCaseData), - Row(1, "a") :: - Row(2, "b") :: - Row(3, "c") :: - Row(4, "d") :: Nil) - checkAnswer(lowerCaseData.intersect(upperCaseData), Nil) - - // check null equality - checkAnswer( - nullInts.intersect(nullInts), - Row(1) :: - Row(2) :: - Row(3) :: - Row(null) :: Nil) - - // check if values are de-duplicated - checkAnswer( - allNulls.intersect(allNulls), - Row(null) :: Nil) - - // check if values are de-duplicated - val df = Seq(("id1", 1), ("id1", 1), ("id", 1), ("id1", 2)).toDF("id", "value") - checkAnswer( - df.intersect(df), - Row("id1", 1) :: - Row("id", 1) :: - Row("id1", 2) :: Nil) - } - - test("intersect - nullability") { - val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF() - assert(nonNullableInts.schema.forall(!_.nullable)) - - val df1 = nonNullableInts.intersect(nullInts) - checkAnswer(df1, Row(1) :: Row(3) :: Nil) - assert(df1.schema.forall(!_.nullable)) - - val df2 = nullInts.intersect(nonNullableInts) - checkAnswer(df2, Row(1) :: Row(3) :: Nil) - assert(df2.schema.forall(!_.nullable)) - - val df3 = nullInts.intersect(nullInts) - checkAnswer(df3, Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) - assert(df3.schema.forall(_.nullable)) - - val df4 = nonNullableInts.intersect(nonNullableInts) - checkAnswer(df4, Row(1) :: Row(3) :: Nil) - assert(df4.schema.forall(!_.nullable)) - } - - test("intersectAll") { - checkAnswer( - lowerCaseDataWithDuplicates.intersectAll(lowerCaseDataWithDuplicates), - Row(1, "a") :: - Row(2, "b") :: - Row(2, "b") :: - Row(3, "c") :: - Row(3, "c") :: - Row(3, "c") :: - Row(4, "d") :: Nil) - checkAnswer(lowerCaseData.intersectAll(upperCaseData), Nil) - - // check null equality - checkAnswer( - nullInts.intersectAll(nullInts), - Row(1) :: - Row(2) :: - Row(3) :: - Row(null) :: Nil) - - // Duplicate nulls are preserved. - checkAnswer( - allNulls.intersectAll(allNulls), - Row(null) :: Row(null) :: Row(null) :: Row(null) :: Nil) - - val df_left = Seq(1, 2, 2, 3, 3, 4).toDF("id") - val df_right = Seq(1, 2, 2, 3).toDF("id") - - checkAnswer( - df_left.intersectAll(df_right), - Row(1) :: Row(2) :: Row(2) :: Row(3) :: Nil) - } - - test("intersectAll - nullability") { - val nonNullableInts = Seq(Tuple1(1), Tuple1(3)).toDF() - assert(nonNullableInts.schema.forall(!_.nullable)) - - val df1 = nonNullableInts.intersectAll(nullInts) - checkAnswer(df1, Row(1) :: Row(3) :: Nil) - assert(df1.schema.forall(!_.nullable)) - - val df2 = nullInts.intersectAll(nonNullableInts) - checkAnswer(df2, Row(1) :: Row(3) :: Nil) - assert(df2.schema.forall(!_.nullable)) - - val df3 = nullInts.intersectAll(nullInts) - checkAnswer(df3, Row(1) :: Row(2) :: Row(3) :: Row(null) :: Nil) - assert(df3.schema.forall(_.nullable)) - - val df4 = nonNullableInts.intersectAll(nonNullableInts) - checkAnswer(df4, Row(1) :: Row(3) :: Nil) - assert(df4.schema.forall(!_.nullable)) - } - test("udf") { val foo = udf((a: Int, b: String) => a.toString + b) @@ -953,6 +572,29 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { assert(df.schema.map(_.name) === Seq("value")) } + test("SPARK-28189 drop column using drop with column reference with case-insensitive names") { + // With SQL config caseSensitive OFF, case insensitive column name should work + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val col1 = testData("KEY") + val df1 = testData.drop(col1) + checkAnswer(df1, testData.selectExpr("value")) + assert(df1.schema.map(_.name) === Seq("value")) + + val col2 = testData("Key") + val df2 = testData.drop(col2) + checkAnswer(df2, testData.selectExpr("value")) + assert(df2.schema.map(_.name) === Seq("value")) + } + + // With SQL config caseSensitive ON, AnalysisException should be thrown + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val e = intercept[AnalysisException] { + testData("KEY") + }.getMessage + assert(e.contains("Cannot resolve column name")) + } + } + test("drop unknown column (no-op) with column reference") { val col = Column("random") val df = testData.drop(col) @@ -1143,22 +785,26 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("inputFiles") { - withTempDir { dir => - val df = Seq((1, 22)).toDF("a", "b") - - val parquetDir = new File(dir, "parquet").getCanonicalPath - df.write.parquet(parquetDir) - val parquetDF = spark.read.parquet(parquetDir) - assert(parquetDF.inputFiles.nonEmpty) - - val jsonDir = new File(dir, "json").getCanonicalPath - df.write.json(jsonDir) - val jsonDF = spark.read.json(jsonDir) - assert(parquetDF.inputFiles.nonEmpty) - - val unioned = jsonDF.union(parquetDF).inputFiles.sorted - val allFiles = (jsonDF.inputFiles ++ parquetDF.inputFiles).distinct.sorted - assert(unioned === allFiles) + Seq("csv", "").foreach { useV1List => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1List) { + withTempDir { dir => + val df = Seq((1, 22)).toDF("a", "b") + + val parquetDir = new File(dir, "parquet").getCanonicalPath + df.write.parquet(parquetDir) + val parquetDF = spark.read.parquet(parquetDir) + assert(parquetDF.inputFiles.nonEmpty) + + val csvDir = new File(dir, "csv").getCanonicalPath + df.write.json(csvDir) + val csvDF = spark.read.json(csvDir) + assert(csvDF.inputFiles.nonEmpty) + + val unioned = csvDF.union(parquetDF).inputFiles.sorted + val allFiles = (csvDF.inputFiles ++ parquetDF.inputFiles).distinct.sorted + assert(unioned === allFiles) + } + } } } @@ -1798,62 +1444,15 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("SPARK-10316: respect non-deterministic expressions in PhysicalOperation") { - val input = spark.read.json((1 to 10).map(i => s"""{"id": $i}""").toDS()) - - val df = input.select($"id", rand(0).as('r)) - df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => - assert(row.getDouble(1) - row.getDouble(3) === 0.0 +- 0.001) - } - } - - test("SPARK-10539: Project should not be pushed down through Intersect or Except") { - val df1 = (1 to 100).map(Tuple1.apply).toDF("i") - val df2 = (1 to 30).map(Tuple1.apply).toDF("i") - val intersect = df1.intersect(df2) - val except = df1.except(df2) - assert(intersect.count() === 30) - assert(except.count() === 70) - } - - test("SPARK-10740: handle nondeterministic expressions correctly for set operations") { - val df1 = (1 to 20).map(Tuple1.apply).toDF("i") - val df2 = (1 to 10).map(Tuple1.apply).toDF("i") + withTempDir { dir => + (1 to 10).toDF("id").write.mode(SaveMode.Overwrite).json(dir.getCanonicalPath) + val input = spark.read.json(dir.getCanonicalPath) - // When generating expected results at here, we need to follow the implementation of - // Rand expression. - def expected(df: DataFrame): Seq[Row] = { - df.rdd.collectPartitions().zipWithIndex.flatMap { - case (data, index) => - val rng = new org.apache.spark.util.random.XORShiftRandom(7 + index) - data.filter(_.getInt(0) < rng.nextDouble() * 10) + val df = input.select($"id", rand(0).as('r)) + df.as("a").join(df.filter($"r" < 0.5).as("b"), $"a.id" === $"b.id").collect().foreach { row => + assert(row.getDouble(1) - row.getDouble(3) === 0.0 +- 0.001) } } - - val union = df1.union(df2) - checkAnswer( - union.filter('i < rand(7) * 10), - expected(union) - ) - checkAnswer( - union.select(rand(7)), - union.rdd.collectPartitions().zipWithIndex.flatMap { - case (data, index) => - val rng = new org.apache.spark.util.random.XORShiftRandom(7 + index) - data.map(_ => rng.nextDouble()).map(i => Row(i)) - } - ) - - val intersect = df1.intersect(df2) - checkAnswer( - intersect.filter('i < rand(7) * 10), - expected(intersect) - ) - - val except = df1.except(df2) - checkAnswer( - except.filter('i < rand(7) * 10), - expected(except) - ) } test("SPARK-10743: keep the name of expression if possible when do cast") { @@ -2000,8 +1599,8 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val rdd = sparkContext.makeRDD(Seq(Row(1, 3), Row(2, 1))) val df = spark.createDataFrame( rdd, - new StructType().add("f1", IntegerType).add("f2", IntegerType), - needsConversion = false).select($"F1", $"f2".as("f2")) + new StructType().add("f1", IntegerType).add("f2", IntegerType)) + .select($"F1", $"f2".as("f2")) val df1 = df.as("a") val df2 = df.as("b") checkAnswer(df1.join(df2, $"a.f2" === $"b.f2"), Row(1, 3, 1, 3) :: Row(2, 1, 2, 1) :: Nil) @@ -2016,7 +1615,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-11725: correctly handle null inputs for ScalaUDF") { val df = sparkContext.parallelize(Seq( - new java.lang.Integer(22) -> "John", + java.lang.Integer.valueOf(22) -> "John", null.asInstanceOf[java.lang.Integer] -> "Lucy")).toDF("age", "name") // passing null into the UDF that could handle it @@ -2073,7 +1672,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } test("reuse exchange") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "2") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "2") { val df = spark.range(100).toDF() val join = df.join(df, "id") val plan = join.queryExecution.executedPlan @@ -2202,7 +1801,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val size = 201L val rdd = sparkContext.makeRDD(Seq(Row.fromSeq(Seq.range(0, size)))) val schemas = List.range(0, size).map(a => StructField("name" + a, LongType, true)) - val df = spark.createDataFrame(rdd, StructType(schemas), false) + val df = spark.createDataFrame(rdd, StructType(schemas)) assert(df.persist.take(1).apply(0).toSeq(100).asInstanceOf[Long] == 100) } @@ -2249,9 +1848,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-17957: no change on nullability in FilterExec output") { val df = sparkContext.parallelize(Seq( - null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), - new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], - new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + null.asInstanceOf[java.lang.Integer] -> java.lang.Integer.valueOf(3), + java.lang.Integer.valueOf(1) -> null.asInstanceOf[java.lang.Integer], + java.lang.Integer.valueOf(2) -> java.lang.Integer.valueOf(4))).toDF() verifyNullabilityInFilterExec(df, expr = "Rand()", expectedNonNullableColumns = Seq.empty[String]) @@ -2266,9 +1865,9 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { test("SPARK-17957: set nullability to false in FilterExec output") { val df = sparkContext.parallelize(Seq( - null.asInstanceOf[java.lang.Integer] -> new java.lang.Integer(3), - new java.lang.Integer(1) -> null.asInstanceOf[java.lang.Integer], - new java.lang.Integer(2) -> new java.lang.Integer(4))).toDF() + null.asInstanceOf[java.lang.Integer] -> java.lang.Integer.valueOf(3), + java.lang.Integer.valueOf(1) -> null.asInstanceOf[java.lang.Integer], + java.lang.Integer.valueOf(2) -> java.lang.Integer.valueOf(4))).toDF() verifyNullabilityInFilterExec(df, expr = "_1 + _2 * 3", expectedNonNullableColumns = Seq("_1", "_2")) @@ -2304,21 +1903,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { } } - test("SPARK-17123: Performing set operations that combine non-scala native types") { - val dates = Seq( - (new Date(0), BigDecimal.valueOf(1), new Timestamp(2)), - (new Date(3), BigDecimal.valueOf(4), new Timestamp(5)) - ).toDF("date", "timestamp", "decimal") - - val widenTypedRows = Seq( - (new Timestamp(2), 10.5D, "string") - ).toDF("date", "timestamp", "decimal") - - dates.union(widenTypedRows).collect() - dates.except(widenTypedRows).collect() - dates.intersect(widenTypedRows).collect() - } - test("SPARK-18070 binary operator should not consider nullability when comparing input types") { val rows = Seq(Row(Seq(1), Seq(1))) val schema = new StructType() @@ -2338,25 +1922,6 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Row(BigDecimal(0)) :: Nil) } - test("SPARK-19893: cannot run set operations with map type") { - val df = spark.range(1).select(map(lit("key"), $"id").as("m")) - val e = intercept[AnalysisException](df.intersect(df)) - assert(e.message.contains( - "Cannot have map type columns in DataFrame which calls set operations")) - val e2 = intercept[AnalysisException](df.except(df)) - assert(e2.message.contains( - "Cannot have map type columns in DataFrame which calls set operations")) - val e3 = intercept[AnalysisException](df.distinct()) - assert(e3.message.contains( - "Cannot have map type columns in DataFrame which calls set operations")) - withTempView("v") { - df.createOrReplaceTempView("v") - val e4 = intercept[AnalysisException](sql("SELECT DISTINCT m FROM v")) - assert(e4.message.contains( - "Cannot have map type columns in DataFrame which calls set operations")) - } - } - test("SPARK-20359: catalyst outer join optimization should not throw npe") { val df1 = Seq("a", "b", "c").toDF("x") .withColumn("y", udf{ (x: String) => x.substring(0, 1) + "!" }.apply($"x")) @@ -2388,7 +1953,7 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val e = intercept[SparkException] { df.filter(filter).count() }.getMessage - assert(e.contains("grows beyond 64 KB")) + assert(e.contains("grows beyond 64 KiB")) } } @@ -2536,28 +2101,11 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { val df = spark.read.json(path.getCanonicalPath) assert(df.columns === Array("i", "p")) + spark.sparkContext.listenerBus.waitUntilEmpty(10000) assert(numJobs == 1) } } - test("SPARK-25368 Incorrect predicate pushdown returns wrong result") { - def check(newCol: Column, filter: Column, result: Seq[Row]): Unit = { - val df1 = spark.createDataFrame(Seq( - (1, 1) - )).toDF("a", "b").withColumn("c", newCol) - - val df2 = df1.union(df1).withColumn("d", spark_partition_id).filter(filter) - checkAnswer(df2, result) - } - - check(lit(null).cast("int"), $"c".isNull, Seq(Row(1, 1, null, 0), Row(1, 1, null, 1))) - check(lit(null).cast("int"), $"c".isNotNull, Seq()) - check(lit(2).cast("int"), $"c".isNull, Seq()) - check(lit(2).cast("int"), $"c".isNotNull, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) - check(lit(2).cast("int"), $"c" === 2, Seq(Row(1, 1, 2, 0), Row(1, 1, 2, 1))) - check(lit(2).cast("int"), $"c" =!= 2, Seq()) - } - test("SPARK-25402 Null handling in BooleanSimplification") { val schema = StructType.fromDDL("a boolean, b int") val rows = Seq(Row(null, 1)) @@ -2567,4 +2115,78 @@ class DataFrameSuite extends QueryTest with SharedSQLContext { checkAnswer(df.where("(NOT a) OR a"), Seq.empty) } + + test("SPARK-25714 Null handling in BooleanSimplification") { + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> ConvertToLocalRelation.ruleName) { + val df = Seq(("abc", 1), (null, 3)).toDF("col1", "col2") + checkAnswer( + df.filter("col1 = 'abc' OR (col1 != 'abc' AND col2 == 3)"), + Row ("abc", 1)) + } + } + + test("SPARK-25816 ResolveReferences works with nested extractors") { + val df = Seq((1, Map(1 -> "a")), (2, Map(2 -> "b"))).toDF("key", "map") + val swappedDf = df.select($"key".as("map"), $"map".as("key")) + + checkAnswer(swappedDf.filter($"key"($"map") > "a"), Row(2, Map(2 -> "b"))) + } + + test("SPARK-26057: attribute deduplication on already analyzed plans") { + withTempView("a", "b", "v") { + val df1 = Seq(("1-1", 6)).toDF("id", "n") + df1.createOrReplaceTempView("a") + val df3 = Seq("1-1").toDF("id") + df3.createOrReplaceTempView("b") + spark.sql( + """ + |SELECT a.id, n as m + |FROM a + |WHERE EXISTS( + | SELECT 1 + | FROM b + | WHERE b.id = a.id) + """.stripMargin).createOrReplaceTempView("v") + val res = spark.sql( + """ + |SELECT a.id, n, m + | FROM a + | LEFT OUTER JOIN v ON v.id = a.id + """.stripMargin) + checkAnswer(res, Row("1-1", 6, 6)) + } + } + + test("SPARK-27671: Fix analysis exception when casting null in nested field in struct") { + val df = sql("SELECT * FROM VALUES (('a', (10, null))), (('b', (10, 50))), " + + "(('c', null)) AS tab(x, y)") + checkAnswer(df, Row("a", Row(10, null)) :: Row("b", Row(10, 50)) :: Row("c", null) :: Nil) + + val cast = sql("SELECT cast(struct(1, null) AS struct)") + checkAnswer(cast, Row(Row(1, null)) :: Nil) + } + + test("SPARK-27439: Explain result should match collected result after view change") { + withTempView("test", "test2", "tmp") { + spark.range(10).createOrReplaceTempView("test") + spark.range(5).createOrReplaceTempView("test2") + spark.sql("select * from test").createOrReplaceTempView("tmp") + val df = spark.sql("select * from tmp") + spark.sql("select * from test2").createOrReplaceTempView("tmp") + + val captured = new ByteArrayOutputStream() + Console.withOut(captured) { + df.explain(extended = true) + } + checkAnswer(df, spark.range(10).toDF) + val output = captured.toString + assert(output.contains( + """== Parsed Logical Plan == + |'Project [*] + |+- 'UnresolvedRelation [tmp]""".stripMargin)) + assert(output.contains( + """== Physical Plan == + |*(1) Range (0, 10, step=1, splits=2)""".stripMargin)) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala index 2a0b2b85e10a9..002c17f4cce4f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFramesSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql -import java.sql.{Date, Timestamp} +import java.sql.Date import org.apache.spark.sql.expressions.Window import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.unsafe.types.CalendarInterval /** * Window frame testing for DataFrame API. @@ -219,52 +218,6 @@ class DataFrameWindowFramesSuite extends QueryTest with SharedSQLContext { Window.partitionBy($"value").orderBy($"key").rangeBetween(-2147483649L, 0))), Seq(Row(1, 2), Row(1, 2), Row(2, 3), Row(2147483650L, 2), Row(2147483650L, 4), Row(3, 1)) ) - - def dt(date: String): Date = Date.valueOf(date) - - val df2 = Seq((dt("2017-08-01"), "1"), (dt("2017-08-01"), "1"), (dt("2020-12-31"), "1"), - (dt("2017-08-03"), "2"), (dt("2017-08-02"), "1"), (dt("2020-12-31"), "2")) - .toDF("key", "value") - val window = Window.partitionBy($"value").orderBy($"key").rangeBetween(lit(0), lit(2)) - - checkAnswer( - df2.select( - $"key", - count("key").over(window)), - Seq(Row(dt("2017-08-01"), 3), Row(dt("2017-08-01"), 3), Row(dt("2020-12-31"), 1), - Row(dt("2017-08-03"), 1), Row(dt("2017-08-02"), 1), Row(dt("2020-12-31"), 1)) - ) - } - - test("range between should accept double values as boundary") { - val df = Seq((1.0D, "1"), (1.0D, "1"), (100.001D, "1"), (3.3D, "2"), (2.02D, "1"), - (100.001D, "2")).toDF("key", "value") - val window = Window.partitionBy($"value").orderBy($"key").rangeBetween(currentRow, lit(2.5D)) - - checkAnswer( - df.select( - $"key", - count("key").over(window)), - Seq(Row(1.0, 3), Row(1.0, 3), Row(100.001, 1), Row(3.3, 1), Row(2.02, 1), Row(100.001, 1)) - ) - } - - test("range between should accept interval values as boundary") { - def ts(timestamp: Long): Timestamp = new Timestamp(timestamp * 1000) - - val df = Seq((ts(1501545600), "1"), (ts(1501545600), "1"), (ts(1609372800), "1"), - (ts(1503000000), "2"), (ts(1502000000), "1"), (ts(1609372800), "2")) - .toDF("key", "value") - val window = Window.partitionBy($"value").orderBy($"key") - .rangeBetween(currentRow, lit(CalendarInterval.fromString("interval 23 days 4 hours"))) - - checkAnswer( - df.select( - $"key", - count("key").over(window)), - Seq(Row(ts(1501545600), 3), Row(ts(1501545600), 3), Row(ts(1609372800), 1), - Row(ts(1503000000), 1), Row(ts(1502000000), 1), Row(ts(1609372800), 1)) - ) } test("unbounded rows/range between with aggregation") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala index 78277d7dcf757..a07553008fd78 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameWindowFunctionsSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.sql import org.scalatest.Matchers.the import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} +import org.apache.spark.sql.catalyst.optimizer.TransposeWindow +import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.expressions.{MutableAggregationBuffer, UserDefinedAggregateFunction, Window} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -668,17 +670,76 @@ class DataFrameWindowFunctionsSuite extends QueryTest with SharedSQLContext { ("S2", "P2", 300) ).toDF("sno", "pno", "qty") - val w1 = Window.partitionBy("sno") - val w2 = Window.partitionBy("sno", "pno") + Seq(true, false).foreach { transposeWindowEnabled => + val excludedRules = if (transposeWindowEnabled) "" else TransposeWindow.ruleName + withSQLConf(SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> excludedRules) { + val w1 = Window.partitionBy("sno") + val w2 = Window.partitionBy("sno", "pno") + + val select = df.select($"sno", $"pno", $"qty", sum($"qty").over(w2).alias("sum_qty_2")) + .select($"sno", $"pno", $"qty", col("sum_qty_2"), sum("qty").over(w1).alias("sum_qty_1")) + + val expectedNumExchanges = if (transposeWindowEnabled) 1 else 2 + val actualNumExchanges = select.queryExecution.executedPlan.collect { + case e: Exchange => e + }.length + assert(actualNumExchanges == expectedNumExchanges) + + checkAnswer( + select, + Seq( + Row("S1", "P1", 100, 800, 800), + Row("S1", "P1", 700, 800, 800), + Row("S2", "P1", 200, 200, 500), + Row("S2", "P2", 300, 300, 500))) + } + } + } + + test("NaN and -0.0 in window partition keys") { + val df = Seq( + (Float.NaN, Double.NaN), + (0.0f/0.0f, 0.0/0.0), + (0.0f, 0.0), + (-0.0f, -0.0)).toDF("f", "d") + + checkAnswer( + df.select($"f", count(lit(1)).over(Window.partitionBy("f", "d"))), + Seq( + Row(Float.NaN, 2), + Row(0.0f/0.0f, 2), + Row(0.0f, 2), + Row(-0.0f, 2))) + // test with complicated window partition keys. + val windowSpec1 = Window.partitionBy(array("f"), struct("d")) checkAnswer( - df.select($"sno", $"pno", $"qty", sum($"qty").over(w2).alias("sum_qty_2")) - .select($"sno", $"pno", $"qty", col("sum_qty_2"), sum("qty").over(w1).alias("sum_qty_1")), + df.select($"f", count(lit(1)).over(windowSpec1)), Seq( - Row("S1", "P1", 100, 800, 800), - Row("S1", "P1", 700, 800, 800), - Row("S2", "P1", 200, 200, 500), - Row("S2", "P2", 300, 300, 500))) + Row(Float.NaN, 2), + Row(0.0f/0.0f, 2), + Row(0.0f, 2), + Row(-0.0f, 2))) + val windowSpec2 = Window.partitionBy(array(struct("f")), struct(array("d"))) + checkAnswer( + df.select($"f", count(lit(1)).over(windowSpec2)), + Seq( + Row(Float.NaN, 2), + Row(0.0f/0.0f, 2), + Row(0.0f, 2), + Row(-0.0f, 2))) + + // test with df with complicated-type columns. + val df2 = Seq( + (Array(-0.0f, 0.0f), Tuple2(-0.0d, Double.NaN), Seq(Tuple2(-0.0d, Double.NaN))), + (Array(0.0f, -0.0f), Tuple2(0.0d, Double.NaN), Seq(Tuple2(0.0d, 0.0/0.0))) + ).toDF("arr", "stru", "arrOfStru") + val windowSpec3 = Window.partitionBy("arr", "stru", "arrOfStru") + checkAnswer( + df2.select($"arr", $"stru", $"arrOfStru", count(lit(1)).over(windowSpec3)), + Seq( + Row(Seq(-0.0f, 0.0f), Row(-0.0d, Double.NaN), Seq(Row(-0.0d, Double.NaN)), 2), + Row(Seq(0.0f, -0.0f), Row(0.0d, Double.NaN), Seq(Row(0.0d, 0.0/0.0)), 2))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala index 538ea3c66c40e..e581211e4e76f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetAggregatorSuite.scala @@ -22,7 +22,7 @@ import org.apache.spark.sql.expressions.Aggregator import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.StringType +import org.apache.spark.sql.types.{BooleanType, IntegerType, StringType, StructType} object ComplexResultAgg extends Aggregator[(String, Int), (Long, Long), (Long, Long)] { @@ -149,6 +149,7 @@ object VeryComplexResultAgg extends Aggregator[Row, String, ComplexAggData] { case class OptionBooleanData(name: String, isGood: Option[Boolean]) +case class OptionBooleanIntData(name: String, isGood: Option[(Boolean, Int)]) case class OptionBooleanAggregator(colName: String) extends Aggregator[Row, Option[Boolean], Option[Boolean]] { @@ -183,6 +184,43 @@ case class OptionBooleanAggregator(colName: String) def OptionalBoolEncoder: Encoder[Option[Boolean]] = ExpressionEncoder() } +case class OptionBooleanIntAggregator(colName: String) + extends Aggregator[Row, Option[(Boolean, Int)], Option[(Boolean, Int)]] { + + override def zero: Option[(Boolean, Int)] = None + + override def reduce(buffer: Option[(Boolean, Int)], row: Row): Option[(Boolean, Int)] = { + val index = row.fieldIndex(colName) + val value = if (row.isNullAt(index)) { + Option.empty[(Boolean, Int)] + } else { + val nestedRow = row.getStruct(index) + Some((nestedRow.getBoolean(0), nestedRow.getInt(1))) + } + merge(buffer, value) + } + + override def merge( + b1: Option[(Boolean, Int)], + b2: Option[(Boolean, Int)]): Option[(Boolean, Int)] = { + if ((b1.isDefined && b1.get._1) || (b2.isDefined && b2.get._1)) { + val newInt = b1.map(_._2).getOrElse(0) + b2.map(_._2).getOrElse(0) + Some((true, newInt)) + } else if (b1.isDefined) { + b1 + } else { + b2 + } + } + + override def finish(reduction: Option[(Boolean, Int)]): Option[(Boolean, Int)] = reduction + + override def bufferEncoder: Encoder[Option[(Boolean, Int)]] = OptionalBoolIntEncoder + override def outputEncoder: Encoder[Option[(Boolean, Int)]] = OptionalBoolIntEncoder + + def OptionalBoolIntEncoder: Encoder[Option[(Boolean, Int)]] = ExpressionEncoder() +} + class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { import testImplicits._ @@ -356,9 +394,9 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { val ds1 = Seq(1, 3, 2, 5).toDS() assert(ds1.select(typed.sum((i: Int) => i)).schema.head.nullable === false) val ds2 = Seq(AggData(1, "a"), AggData(2, "a")).toDS() - assert(ds2.select(SeqAgg.toColumn).schema.head.nullable === true) + assert(ds2.select(SeqAgg.toColumn).schema.head.nullable) val ds3 = sql("SELECT 'Some String' AS b, 1279869254 AS a").as[AggData] - assert(ds3.select(NameAgg.toColumn).schema.head.nullable === true) + assert(ds3.select(NameAgg.toColumn).schema.head.nullable) } test("SPARK-18147: very complex aggregator result type") { @@ -393,4 +431,28 @@ class DatasetAggregatorSuite extends QueryTest with SharedSQLContext { assert(grouped.schema == df.schema) checkDataset(grouped.as[OptionBooleanData], OptionBooleanData("bob", Some(true))) } + + test("SPARK-24762: Aggregator should be able to use Option of Product encoder") { + val df = Seq( + OptionBooleanIntData("bob", Some((true, 1))), + OptionBooleanIntData("bob", Some((false, 2))), + OptionBooleanIntData("bob", None)).toDF() + + val group = df + .groupBy("name") + .agg(OptionBooleanIntAggregator("isGood").toColumn.alias("isGood")) + + val expectedSchema = new StructType() + .add("name", StringType, nullable = true) + .add("isGood", + new StructType() + .add("_1", BooleanType, nullable = false) + .add("_2", IntegerType, nullable = false), + nullable = true) + + assert(df.schema == expectedSchema) + assert(group.schema == expectedSchema) + checkAnswer(group, Row("bob", Row(true, 3)) :: Nil) + checkDataset(group.as[OptionBooleanIntData], OptionBooleanIntData("bob", Some((true, 3)))) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala index e3df449b41f0a..dba906f63aed4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetBenchmark.scala @@ -256,7 +256,7 @@ object DatasetBenchmark extends SqlBasedBenchmark { .getOrCreate() } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { val numRows = 100000000 val numChains = 10 runBenchmark("Dataset Benchmark") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala index fef6ddd0b93c5..b828b23025151 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetCacheSuite.scala @@ -49,7 +49,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits assert(ds1.storageLevel == StorageLevel.MEMORY_AND_DISK) assert(ds2.storageLevel == StorageLevel.MEMORY_AND_DISK) // unpersist - ds1.unpersist() + ds1.unpersist(blocking = true) assert(ds1.storageLevel == StorageLevel.NONE) // non-default storage level ds1.persist(StorageLevel.MEMORY_ONLY_2) @@ -71,7 +71,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits cached, 2, 3, 4) // Drop the cache. - cached.unpersist() + cached.unpersist(blocking = true) assert(cached.storageLevel == StorageLevel.NONE, "The Dataset should not be cached.") } @@ -88,9 +88,9 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits checkDataset(joined, ("2", 2)) assertCached(joined, 2) - ds1.unpersist() + ds1.unpersist(blocking = true) assert(ds1.storageLevel == StorageLevel.NONE, "The Dataset ds1 should not be cached.") - ds2.unpersist() + ds2.unpersist(blocking = true) assert(ds2.storageLevel == StorageLevel.NONE, "The Dataset ds2 should not be cached.") } @@ -105,9 +105,9 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits ("b", 3)) assertCached(agged.filter(_._1 == "b")) - ds.unpersist() + ds.unpersist(blocking = true) assert(ds.storageLevel == StorageLevel.NONE, "The Dataset ds should not be cached.") - agged.unpersist() + agged.unpersist(blocking = true) assert(agged.storageLevel == StorageLevel.NONE, "The Dataset agged should not be cached.") } @@ -122,7 +122,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits df.count() assertCached(df2) - df.unpersist() + df.unpersist(blocking = true) assert(df.storageLevel == StorageLevel.NONE) } @@ -136,11 +136,11 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits assertCached(df2) // udf has been evaluated during caching, and thus should not be re-evaluated here - failAfter(2 seconds) { + failAfter(2.seconds) { df2.collect() } - df.unpersist() + df.unpersist(blocking = true) assert(df.storageLevel == StorageLevel.NONE) } @@ -166,7 +166,7 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits df.count() df3.cache() - df.unpersist() + df.unpersist(blocking = true) // df un-cached; df2 and df3's cache plan re-compiled assert(df.storageLevel == StorageLevel.NONE) @@ -190,14 +190,14 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits df1.unpersist(blocking = true) - // df1 un-cached; df2's cache plan re-compiled + // df1 un-cached; df2's cache plan stays the same assert(df1.storageLevel == StorageLevel.NONE) - assertCacheDependency(df1.groupBy('a).agg(sum('b)), 0) + assertCacheDependency(df1.groupBy('a).agg(sum('b))) val df4 = df1.groupBy('a).agg(sum('b)).agg(sum("sum(b)")) assertCached(df4) // reuse loaded cache - failAfter(3 seconds) { + failAfter(3.seconds) { checkDataset(df4, Row(10)) } @@ -206,4 +206,44 @@ class DatasetCacheSuite extends QueryTest with SharedSQLContext with TimeLimits // first time use, load cache checkDataset(df5, Row(10)) } + + test("SPARK-26708 Cache data and cached plan should stay consistent") { + val df = spark.range(0, 5).toDF("a") + val df1 = df.withColumn("b", 'a + 1) + val df2 = df.filter('a > 1) + + df.cache() + // Add df1 to the CacheManager; the buffer is currently empty. + df1.cache() + // After calling collect(), df1's buffer has been loaded. + df1.collect() + // Add df2 to the CacheManager; the buffer is currently empty. + df2.cache() + + // Verify that df1 is a InMemoryRelation plan with dependency on another cached plan. + assertCacheDependency(df1) + val df1InnerPlan = df1.queryExecution.withCachedData + .asInstanceOf[InMemoryRelation].cacheBuilder.cachedPlan + // Verify that df2 is a InMemoryRelation plan with dependency on another cached plan. + assertCacheDependency(df2) + + df.unpersist(blocking = true) + + // Verify that df1's cache has stayed the same, since df1's cache already has data + // before df.unpersist(). + val df1Limit = df1.limit(2) + val df1LimitInnerPlan = df1Limit.queryExecution.withCachedData.collectFirst { + case i: InMemoryRelation => i.cacheBuilder.cachedPlan + } + assert(df1LimitInnerPlan.isDefined && df1LimitInnerPlan.get == df1InnerPlan) + + // Verify that df2's cache has been re-cached, with a new physical plan rid of dependency + // on df, since df2's cache had not been loaded before df.unpersist(). + val df2Limit = df2.limit(2) + val df2LimitInnerPlan = df2Limit.queryExecution.withCachedData.collectFirst { + case i: InMemoryRelation => i.cacheBuilder.cachedPlan + } + assert(df2LimitInnerPlan.isDefined && + df2LimitInnerPlan.get.find(_.isInstanceOf[InMemoryTableScanExec]).isEmpty) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala new file mode 100644 index 0000000000000..ae051e43fbcb5 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetOptimizationSuite.scala @@ -0,0 +1,207 @@ +/* + * 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.metrics.source.CodegenMetrics +import org.apache.spark.sql.catalyst.expressions.{CreateNamedStruct, Expression} +import org.apache.spark.sql.catalyst.expressions.objects.ExternalMapToCatalyst +import org.apache.spark.sql.catalyst.plans.logical.SerializeFromObject +import org.apache.spark.sql.functions.expr +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class DatasetOptimizationSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("SPARK-26619: Prune the unused serializers from SerializeFromObject") { + val data = Seq(("a", 1), ("b", 2), ("c", 3)) + val ds = data.toDS().map(t => (t._1, t._2 + 1)).select("_1") + val serializer = ds.queryExecution.optimizedPlan.collect { + case s: SerializeFromObject => s + }.head + assert(serializer.serializer.size == 1) + checkAnswer(ds, Seq(Row("a"), Row("b"), Row("c"))) + } + + + // This methods checks if the given DataFrame has specified struct fields in object + // serializer. The varargs parameter `structFields` is the struct fields for object + // serializers. The first `structFields` is aligned with first serializer and ditto + // for other `structFields`. + private def testSerializer(df: DataFrame, structFields: Seq[Seq[String]]*): Unit = { + val serializer = df.queryExecution.optimizedPlan.collect { + case s: SerializeFromObject => s + }.head + + def collectNamedStruct: PartialFunction[Expression, Seq[CreateNamedStruct]] = { + case c: CreateNamedStruct => Seq(c) + } + + serializer.serializer.zip(structFields).foreach { case (ser, fields) => + val structs: Seq[CreateNamedStruct] = ser.collect(collectNamedStruct).flatten + assert(structs.size == fields.size) + structs.zip(fields).foreach { case (struct, fieldNames) => + assert(struct.names.map(_.toString) == fieldNames) + } + } + } + + test("Prune nested serializers: struct") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val data = Seq((("a", 1, ("aa", 1.0)), 1), (("b", 2, ("bb", 2.0)), 2), + (("c", 3, ("cc", 3.0)), 3)) + val ds = data.toDS().map(t => (t._1, t._2 + 1)) + + val df1 = ds.select("_1._1") + testSerializer(df1, Seq(Seq("_1"))) + checkAnswer(df1, Seq(Row("a"), Row("b"), Row("c"))) + + val df2 = ds.select("_1._2") + testSerializer(df2, Seq(Seq("_2"))) + checkAnswer(df2, Seq(Row(1), Row(2), Row(3))) + + val df3 = ds.select("_1._3._1") + testSerializer(df3, Seq(Seq("_3"), Seq("_1"))) + checkAnswer(df3, Seq(Row("aa"), Row("bb"), Row("cc"))) + + val df4 = ds.select("_1._3._1", "_1._2") + testSerializer(df4, Seq(Seq("_2", "_3"), Seq("_1"))) + checkAnswer(df4, Seq(Row("aa", 1), Row("bb", 2), Row("cc", 3))) + } + } + + test("Prune nested serializers: array of struct") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val arrayData = Seq((Seq(("a", 1, ("a_1", 11)), ("b", 2, ("b_1", 22))), 1, ("aa", 1.0)), + (Seq(("c", 3, ("c_1", 33)), ("d", 4, ("d_1", 44))), 2, ("bb", 2.0))) + val arrayDs = arrayData.toDS().map(t => (t._1, t._2 + 1, t._3)) + val df1 = arrayDs.select("_1._1") + // The serializer creates array of struct of one field "_1". + testSerializer(df1, Seq(Seq("_1"))) + checkAnswer(df1, Seq(Row(Seq("a", "b")), Row(Seq("c", "d")))) + + val df2 = arrayDs.select("_3._2") + testSerializer(df2, Seq(Seq("_2"))) + checkAnswer(df2, Seq(Row(1.0), Row(2.0))) + + // This is a more complex case. We select two root fields "_1" and "_3". + // The first serializer creates array of struct of two fields ("_1", "_3") and + // the field "_3" is a struct of one field "_2". + // The second serializer creates a struct of just one field "_1". + val df3 = arrayDs.select("_1._1", "_1._3._2", "_3._1") + testSerializer(df3, Seq(Seq("_1", "_3"), Seq("_2")), Seq(Seq("_1"))) + checkAnswer(df3, Seq(Row(Seq("a", "b"), Seq(11, 22), "aa"), + Row(Seq("c", "d"), Seq(33, 44), "bb"))) + } + } + + test("Prune nested serializers: map of struct") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val mapData = Seq((Map(("k", ("a_1", 11))), 1), (Map(("k", ("b_1", 22))), 2), + (Map(("k", ("c_1", 33))), 3)) + val mapDs = mapData.toDS().map(t => (t._1, t._2 + 1)) + val df1 = mapDs.select("_1.k._1") + testSerializer(df1, Seq(Seq("_1"))) + checkAnswer(df1, Seq(Row("a_1"), Row("b_1"), Row("c_1"))) + + val df2 = mapDs.select("_1.k._2") + testSerializer(df2, Seq(Seq("_2"))) + checkAnswer(df2, Seq(Row(11), Row(22), Row(33))) + + val df3 = mapDs.select(expr("map_values(_1)._2[0]")) + testSerializer(df3, Seq(Seq("_2"))) + checkAnswer(df3, Seq(Row(11), Row(22), Row(33))) + } + } + + test("Pruned nested serializers: map of complex key") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val mapData = Seq((Map((("1", 1), "a_1")), 1), (Map((("2", 2), "b_1")), 2), + (Map((("3", 3), "c_1")), 3)) + val mapDs = mapData.toDS().map(t => (t._1, t._2 + 1)) + val df1 = mapDs.select(expr("map_keys(_1)._1[0]")) + testSerializer(df1, Seq(Seq("_1"))) + checkAnswer(df1, Seq(Row("1"), Row("2"), Row("3"))) + } + } + + test("Pruned nested serializers: map of map value") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val mapData = Seq( + (Map(("k", Map(("k2", ("a_1", 11))))), 1), + (Map(("k", Map(("k2", ("b_1", 22))))), 2), + (Map(("k", Map(("k2", ("c_1", 33))))), 3)) + val mapDs = mapData.toDS().map(t => (t._1, t._2 + 1)) + val df = mapDs.select("_1.k.k2._1") + testSerializer(df, Seq(Seq("_1"))) + } + } + + test("Pruned nested serializers: map of map key") { + withSQLConf(SQLConf.SERIALIZER_NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + val mapData = Seq( + (Map((Map((("1", 1), "val1")), "a_1")), 1), + (Map((Map((("2", 2), "val2")), "b_1")), 2), + (Map((Map((("3", 3), "val3")), "c_1")), 3)) + val mapDs = mapData.toDS().map(t => (t._1, t._2 + 1)) + val df = mapDs.select(expr("map_keys(map_keys(_1)[0])._1[0]")) + testSerializer(df, Seq(Seq("_1"))) + checkAnswer(df, Seq(Row("1"), Row("2"), Row("3"))) + } + } + + test("SPARK-27871: Dataset encoder should benefit from codegen cache") { + def checkCodegenCache(createDataset: () => Dataset[_]): Unit = { + def getCodegenCount(): Long = CodegenMetrics.METRIC_COMPILATION_TIME.getCount() + + val count1 = getCodegenCount() + // trigger codegen for Dataset + createDataset().collect() + val count2 = getCodegenCount() + // codegen happens + assert(count2 > count1) + + // trigger codegen for another Dataset of same type + createDataset().collect() + // codegen cache should work for Datasets of same type. + val count3 = getCodegenCount() + assert(count3 == count2) + + withSQLConf(SQLConf.OPTIMIZER_REASSIGN_LAMBDA_VARIABLE_ID.key -> "false") { + // trigger codegen for another Dataset of same type + createDataset().collect() + // with the rule disabled, codegen happens again for encoder serializer and encoder + // deserializer + val count4 = getCodegenCount() + assert(count4 == (count3 + 2)) + } + } + + withClue("array type") { + checkCodegenCache(() => Seq(Seq("abc")).toDS()) + } + + withClue("map type") { + checkCodegenCache(() => Seq(Map("abc" -> 1)).toDS()) + } + + withClue("array of map") { + checkCodegenCache(() => Seq(Seq(Map("abc" -> 1))).toDS()) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala index edcdd77908d3a..4d7037f36b1fd 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetPrimitiveSuite.scala @@ -295,7 +295,7 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { checkDataset(Seq(LHMap("test" -> 2.toLong)).toDS(), LHMap("test" -> 2.toLong)) } - ignore("SPARK-19104: map and product combinations") { + test("SPARK-25817: map and product combinations") { // Case classes checkDataset(Seq(MapClass(Map(1 -> 2))).toDS(), MapClass(Map(1 -> 2))) checkDataset(Seq(Map(1 -> MapClass(Map(2 -> 3)))).toDS(), Map(1 -> MapClass(Map(2 -> 3)))) @@ -393,4 +393,54 @@ class DatasetPrimitiveSuite extends QueryTest with SharedSQLContext { val ds = spark.createDataset(data) checkDataset(ds, data: _*) } + + test("special floating point values") { + import org.scalatest.exceptions.TestFailedException + + // Spark distinguishes -0.0 and 0.0 + intercept[TestFailedException] { + checkDataset(Seq(-0.0d).toDS(), 0.0d) + } + intercept[TestFailedException] { + checkAnswer(Seq(-0.0d).toDF(), Row(0.0d)) + } + intercept[TestFailedException] { + checkDataset(Seq(-0.0f).toDS(), 0.0f) + } + intercept[TestFailedException] { + checkAnswer(Seq(-0.0f).toDF(), Row(0.0f)) + } + intercept[TestFailedException] { + checkDataset(Seq(Tuple1(-0.0)).toDS(), Tuple1(0.0)) + } + intercept[TestFailedException] { + checkAnswer(Seq(Tuple1(-0.0)).toDF(), Row(Row(0.0))) + } + intercept[TestFailedException] { + checkDataset(Seq(Seq(-0.0)).toDS(), Seq(0.0)) + } + intercept[TestFailedException] { + checkAnswer(Seq(Seq(-0.0)).toDF(), Row(Seq(0.0))) + } + + val floats = Seq[Float](-0.0f, 0.0f, Float.NaN) + checkDataset(floats.toDS(), floats: _*) + + val arrayOfFloats = Seq[Array[Float]](Array(0.0f, -0.0f), Array(-0.0f, Float.NaN)) + checkDataset(arrayOfFloats.toDS(), arrayOfFloats: _*) + + val doubles = Seq[Double](-0.0d, 0.0d, Double.NaN) + checkDataset(doubles.toDS(), doubles: _*) + + val arrayOfDoubles = Seq[Array[Double]](Array(0.0d, -0.0d), Array(-0.0d, Double.NaN)) + checkDataset(arrayOfDoubles.toDS(), arrayOfDoubles: _*) + + val tuples = Seq[(Float, Float, Double, Double)]( + (0.0f, -0.0f, 0.0d, -0.0d), + (-0.0f, Float.NaN, -0.0d, Double.NaN)) + checkDataset(tuples.toDS(), tuples: _*) + + val complex = Map(Array(Seq(Tuple1(Double.NaN))) -> Map(Tuple2(Float.NaN, null))) + checkDataset(Seq(complex).toDS(), complex) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala index 68f7de047b392..69728efcd1d9a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSerializerRegistratorSuite.scala @@ -21,6 +21,7 @@ import com.esotericsoftware.kryo.{Kryo, Serializer} import com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.SparkConf +import org.apache.spark.internal.config.Kryo._ import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.sql.test.SharedSQLContext @@ -33,7 +34,7 @@ class DatasetSerializerRegistratorSuite extends QueryTest with SharedSQLContext override protected def sparkConf: SparkConf = { // Make sure we use the KryoRegistrator - super.sparkConf.set("spark.kryo.registrator", TestRegistrator().getClass.getCanonicalName) + super.sparkConf.set(KRYO_USER_REGISTRATORS, TestRegistrator().getClass.getCanonicalName) } test("Kryo registrator") { 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 4e593ff046a53..ff6143162ff2f 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 @@ -20,13 +20,17 @@ package org.apache.spark.sql import java.io.{Externalizable, ObjectInput, ObjectOutput} import java.sql.{Date, Timestamp} -import org.apache.spark.SparkException +import org.scalatest.exceptions.TestFailedException + +import org.apache.spark.{SparkException, TaskContext} +import org.apache.spark.sql.catalyst.ScroogeLikeExample import org.apache.spark.sql.catalyst.encoders.{OuterScopes, RowEncoder} import org.apache.spark.sql.catalyst.plans.{LeftAnti, LeftSemi} import org.apache.spark.sql.catalyst.util.sideBySide -import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec} +import org.apache.spark.sql.execution.{LogicalRDD, RDDScanExec, SQLExecution} import org.apache.spark.sql.execution.exchange.{BroadcastExchangeExec, ShuffleExchangeExec} import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.expressions.UserDefinedFunction import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -64,6 +68,41 @@ class DatasetSuite extends QueryTest with SharedSQLContext { data: _*) } + test("toDS should compare map with byte array keys correctly") { + // Choose the order of arrays in such way, that sorting keys of different maps by _.toString + // will not incidentally put equal keys together. + val arrays = (1 to 5).map(_ => Array[Byte](0.toByte, 0.toByte)).sortBy(_.toString).toArray + arrays(0)(1) = 1.toByte + arrays(1)(1) = 2.toByte + arrays(2)(1) = 2.toByte + arrays(3)(1) = 1.toByte + + val mapA = Map(arrays(0) -> "one", arrays(2) -> "two") + val subsetOfA = Map(arrays(0) -> "one") + val equalToA = Map(arrays(1) -> "two", arrays(3) -> "one") + val notEqualToA1 = Map(arrays(1) -> "two", arrays(3) -> "not one") + val notEqualToA2 = Map(arrays(1) -> "two", arrays(4) -> "one") + + // Comparing map with itself + checkDataset(Seq(mapA).toDS(), mapA) + + // Comparing map with equivalent map + checkDataset(Seq(equalToA).toDS(), mapA) + checkDataset(Seq(mapA).toDS(), equalToA) + + // Comparing map with it's subset + intercept[TestFailedException](checkDataset(Seq(subsetOfA).toDS(), mapA)) + intercept[TestFailedException](checkDataset(Seq(mapA).toDS(), subsetOfA)) + + // Comparing map with another map differing by single value + intercept[TestFailedException](checkDataset(Seq(notEqualToA1).toDS(), mapA)) + intercept[TestFailedException](checkDataset(Seq(mapA).toDS(), notEqualToA1)) + + // Comparing map with another map differing by single key + intercept[TestFailedException](checkDataset(Seq(notEqualToA2).toDS(), mapA)) + intercept[TestFailedException](checkDataset(Seq(mapA).toDS(), notEqualToA2)) + } + test("toDS with RDD") { val ds = sparkContext.makeRDD(Seq("a", "b", "c"), 3).toDS() checkDataset( @@ -164,6 +203,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { Seq(ClassData("a", 2)))) } + test("as map of case class - reorder fields by name") { + val df = spark.range(3).select(map(lit(1), struct($"id".cast("int").as("b"), lit("a").as("a")))) + val ds = df.as[Map[Int, ClassData]] + assert(ds.collect() === Array( + Map(1 -> ClassData("a", 0)), + Map(1 -> ClassData("a", 1)), + Map(1 -> ClassData("a", 2)))) + } + test("map") { val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() checkDataset( @@ -378,8 +426,17 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds1 = Seq(1, 2, 3).toDS().as("a") val ds2 = Seq(1, 2).toDS().as("b") + val joined = ds1.joinWith(ds2, $"a.value" === $"b.value", "inner") + + val expectedSchema = StructType(Seq( + StructField("_1", IntegerType, nullable = false), + StructField("_2", IntegerType, nullable = false) + )) + + assert(joined.schema === expectedSchema) + checkDataset( - ds1.joinWith(ds2, $"a.value" === $"b.value", "inner"), + joined, (1, 1), (2, 2)) } @@ -387,8 +444,21 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds1 = Seq(1, 1, 2).toDS() val ds2 = Seq(("a", 1), ("b", 2)).toDS() + val joined = ds1.joinWith(ds2, $"value" === $"_2") + + // This is an inner join, so both outputs fields are non-nullable + val expectedSchema = StructType(Seq( + StructField("_1", IntegerType, nullable = false), + StructField("_2", + StructType(Seq( + StructField("_1", StringType), + StructField("_2", IntegerType, nullable = false) + )), nullable = false) + )) + assert(joined.schema === expectedSchema) + checkDataset( - ds1.joinWith(ds2, $"value" === $"_2"), + joined, (1, ("a", 1)), (1, ("a", 1)), (2, ("b", 2))) } @@ -422,9 +492,19 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(e1.contains("Invalid join type in joinWith: " + LeftSemi.sql)) val e2 = intercept[AnalysisException] { + ds1.joinWith(ds2, $"a.value" === $"b.value", "semi") + }.getMessage + assert(e2.contains("Invalid join type in joinWith: " + LeftSemi.sql)) + + val e3 = intercept[AnalysisException] { ds1.joinWith(ds2, $"a.value" === $"b.value", "left_anti") }.getMessage - assert(e2.contains("Invalid join type in joinWith: " + LeftAnti.sql)) + assert(e3.contains("Invalid join type in joinWith: " + LeftAnti.sql)) + + val e4 = intercept[AnalysisException] { + ds1.joinWith(ds2, $"a.value" === $"b.value", "anti") + }.getMessage + assert(e4.contains("Invalid join type in joinWith: " + LeftAnti.sql)) } test("groupBy function, keys") { @@ -465,7 +545,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val keyValue1 = ds.groupByKey(t => (t._1, "key")).mapValues(t => (t._2, "value")) val agged1 = keyValue1.mapGroups { case (g, iter) => (g._1, iter.map(_._1).sum) } - checkDataset(agged, ("a", 30), ("b", 3), ("c", 1)) + checkDataset(agged1, ("a", 30), ("b", 3), ("c", 1)) } test("groupBy function, reduce") { @@ -523,6 +603,70 @@ class DatasetSuite extends QueryTest with SharedSQLContext { ("a", 30L, 32L, 2L, 15.0), ("b", 3L, 5L, 2L, 1.5), ("c", 1L, 2L, 1L, 1.0)) } + test("typed aggregation: expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L), ("b", 3L, 5L, 2L, 1.5, 2L), ("c", 1L, 2L, 1L, 1.0, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L)) + } + + test("typed aggregation: expr, expr, expr, expr, expr, expr, expr, expr") { + val ds = Seq(("a", 10), ("a", 20), ("b", 1), ("b", 2), ("c", 1)).toDS() + + checkDatasetUnorderly( + ds.groupByKey(_._1).agg( + sum("_2").as[Long], + sum($"_2" + 1).as[Long], + count("*").as[Long], + avg("_2").as[Double], + countDistinct("*").as[Long], + max("_2").as[Long], + min("_2").as[Long], + mean("_2").as[Double]), + ("a", 30L, 32L, 2L, 15.0, 2L, 20L, 10L, 15.0), + ("b", 3L, 5L, 2L, 1.5, 2L, 2L, 1L, 1.5), + ("c", 1L, 2L, 1L, 1.0, 1L, 1L, 1L, 1.0)) + } + test("cogroup") { val ds1 = Seq(1 -> "a", 3 -> "abc", 5 -> "hello", 3 -> "foo").toDS() val ds2 = Seq(2 -> "q", 3 -> "w", 5 -> "e", 5 -> "r").toDS() @@ -560,7 +704,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val data = sparkContext.parallelize(1 to n, 2).toDS() checkDataset( data.sample(withReplacement = false, 0.05, seed = 13), - 3, 17, 27, 58, 62) + 8, 37, 90) } test("sample fraction should not be negative with replacement") { @@ -592,9 +736,10 @@ class DatasetSuite extends QueryTest with SharedSQLContext { } test("SPARK-16686: Dataset.sample with seed results shouldn't depend on downstream usage") { + val a = 7 val simpleUdf = udf((n: Int) => { - require(n != 1, "simpleUdf shouldn't see id=1!") - 1 + require(n != a, s"simpleUdf shouldn't see id=$a!") + a }) val df = Seq( @@ -610,10 +755,10 @@ class DatasetSuite extends QueryTest with SharedSQLContext { (9, "string9") ).toDF("id", "stringData") val sampleDF = df.sample(false, 0.7, 50) - // After sampling, sampleDF doesn't contain id=1. - assert(!sampleDF.select("id").as[Int].collect.contains(1)) - // simpleUdf should not encounter id=1. - checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(1))) + // After sampling, sampleDF doesn't contain id=a. + assert(!sampleDF.select("id").as[Int].collect.contains(a)) + // simpleUdf should not encounter id=a. + checkAnswer(sampleDF.select(simpleUdf($"id")), List.fill(sampleDF.count.toInt)(Row(a))) } test("SPARK-11436: we should rebind right encoder when join 2 datasets") { @@ -656,11 +801,11 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("Kryo encoder: check the schema mismatch when converting DataFrame to Dataset") { implicit val kryoEncoder = Encoders.kryo[KryoData] - val df = Seq((1)).toDF("a") + val df = Seq((1.0)).toDF("a") val e = intercept[AnalysisException] { df.as[KryoData] }.message - assert(e.contains("cannot cast int to binary")) + assert(e.contains("cannot cast double to binary")) } test("Java encoder") { @@ -688,15 +833,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("SPARK-11894: Incorrect results are returned when using null") { val nullInt = null.asInstanceOf[java.lang.Integer] - val ds1 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() - val ds2 = Seq((nullInt, "1"), (new java.lang.Integer(22), "2")).toDS() + val ds1 = Seq((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")).toDS() + val ds2 = Seq((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")).toDS() checkDataset( ds1.joinWith(ds2, lit(true), "cross"), ((nullInt, "1"), (nullInt, "1")), - ((nullInt, "1"), (new java.lang.Integer(22), "2")), - ((new java.lang.Integer(22), "2"), (nullInt, "1")), - ((new java.lang.Integer(22), "2"), (new java.lang.Integer(22), "2"))) + ((nullInt, "1"), (java.lang.Integer.valueOf(22), "2")), + ((java.lang.Integer.valueOf(22), "2"), (nullInt, "1")), + ((java.lang.Integer.valueOf(22), "2"), (java.lang.Integer.valueOf(22), "2"))) } test("change encoder with compatible schema") { @@ -872,7 +1017,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { assert(ds.rdd.map(r => r.id).count === 2) assert(ds2.rdd.map(r => r.id).count === 2) - val ds3 = ds.map(g => new java.lang.Long(g.id)) + val ds3 = ds.map(g => java.lang.Long.valueOf(g.id)) assert(ds3.rdd.map(r => r).count === 2) } @@ -1046,6 +1191,23 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val left = Seq(ClassData("a", 1), ClassData("b", 2)).toDS().as("left") val right = Seq(ClassData("x", 2), ClassData("y", 3)).toDS().as("right") val joined = left.joinWith(right, $"left.b" === $"right.b", "left") + + val expectedSchema = StructType(Seq( + StructField("_1", + StructType(Seq( + StructField("a", StringType), + StructField("b", IntegerType, nullable = false) + )), + nullable = false), + // This is a left join, so the right output is nullable: + StructField("_2", + StructType(Seq( + StructField("a", StringType), + StructField("b", IntegerType, nullable = false) + ))) + )) + assert(joined.schema === expectedSchema) + val result = joined.collect().toSet assert(result == Set(ClassData("a", 1) -> null, ClassData("b", 2) -> ClassData("x", 2))) } @@ -1065,7 +1227,7 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("Dataset should throw RuntimeException if top-level product input object is null") { val e = intercept[RuntimeException](Seq(ClassData("a", 1), null).toDS()) assert(e.getMessage.contains("Null value appeared in non-nullable field")) - assert(e.getMessage.contains("top level Product input object")) + assert(e.getMessage.contains("top level Product or row object")) } test("dropDuplicates") { @@ -1136,11 +1298,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { GroupedRoutes("a", "c", Seq(Route("a", "c", 2))) ) - implicit def ordering[GroupedRoutes]: Ordering[GroupedRoutes] = new Ordering[GroupedRoutes] { - override def compare(x: GroupedRoutes, y: GroupedRoutes): Int = { - x.toString.compareTo(y.toString) - } - } + implicit def ordering[GroupedRoutes]: Ordering[GroupedRoutes] = + (x: GroupedRoutes, y: GroupedRoutes) => x.toString.compareTo(y.toString) checkDatasetUnorderly(grped, expected: _*) } @@ -1161,14 +1320,14 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val df1 = Seq(1, 2, 3, 4).toDF assert(df1.schema(0).nullable == false) val df2 = Seq(Integer.valueOf(1), Integer.valueOf(2)).toDF - assert(df2.schema(0).nullable == true) + assert(df2.schema(0).nullable) val df3 = Seq(Seq(1, 2), Seq(3, 4)).toDF - assert(df3.schema(0).nullable == true) + assert(df3.schema(0).nullable) assert(df3.schema(0).dataType.asInstanceOf[ArrayType].containsNull == false) val df4 = Seq(Seq("a", "b"), Seq("c", "d")).toDF - assert(df4.schema(0).nullable == true) - assert(df4.schema(0).dataType.asInstanceOf[ArrayType].containsNull == true) + assert(df4.schema(0).nullable) + assert(df4.schema(0).dataType.asInstanceOf[ArrayType].containsNull) val df5 = Seq((0, 1.0), (2, 2.0)).toDF("id", "v") assert(df5.schema(0).nullable == false) @@ -1176,32 +1335,32 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val df6 = Seq((0, 1.0, "a"), (2, 2.0, "b")).toDF("id", "v1", "v2") assert(df6.schema(0).nullable == false) assert(df6.schema(1).nullable == false) - assert(df6.schema(2).nullable == true) + assert(df6.schema(2).nullable) val df7 = (Tuple1(Array(1, 2, 3)) :: Nil).toDF("a") - assert(df7.schema(0).nullable == true) + assert(df7.schema(0).nullable) assert(df7.schema(0).dataType.asInstanceOf[ArrayType].containsNull == false) val df8 = (Tuple1(Array((null: Integer), (null: Integer))) :: Nil).toDF("a") - assert(df8.schema(0).nullable == true) - assert(df8.schema(0).dataType.asInstanceOf[ArrayType].containsNull == true) + assert(df8.schema(0).nullable) + assert(df8.schema(0).dataType.asInstanceOf[ArrayType].containsNull) val df9 = (Tuple1(Map(2 -> 3)) :: Nil).toDF("m") - assert(df9.schema(0).nullable == true) + assert(df9.schema(0).nullable) assert(df9.schema(0).dataType.asInstanceOf[MapType].valueContainsNull == false) val df10 = (Tuple1(Map(1 -> (null: Integer))) :: Nil).toDF("m") - assert(df10.schema(0).nullable == true) - assert(df10.schema(0).dataType.asInstanceOf[MapType].valueContainsNull == true) + assert(df10.schema(0).nullable) + assert(df10.schema(0).dataType.asInstanceOf[MapType].valueContainsNull) val df11 = Seq(TestDataPoint(1, 2.2, "a", null), TestDataPoint(3, 4.4, "null", (TestDataPoint2(33, "b")))).toDF assert(df11.schema(0).nullable == false) assert(df11.schema(1).nullable == false) - assert(df11.schema(2).nullable == true) - assert(df11.schema(3).nullable == true) + assert(df11.schema(2).nullable) + assert(df11.schema(3).nullable) assert(df11.schema(3).dataType.asInstanceOf[StructType].fields(0).nullable == false) - assert(df11.schema(3).dataType.asInstanceOf[StructType].fields(1).nullable == true) + assert(df11.schema(3).dataType.asInstanceOf[StructType].fields(1).nullable) } Seq(true, false).foreach { eager => @@ -1302,15 +1461,6 @@ class DatasetSuite extends QueryTest with SharedSQLContext { checkDataset(dsString, arrayString) } - test("SPARK-18251: the type of Dataset can't be Option of Product type") { - checkDataset(Seq(Some(1), None).toDS(), Some(1), None) - - val e = intercept[UnsupportedOperationException] { - Seq(Some(1 -> "a"), None).toDS() - } - assert(e.getMessage.contains("Cannot create encoder for Option of Product type")) - } - test ("SPARK-17460: the sizeInBytes in Statistics shouldn't overflow to a negative number") { // Since the sizeInBytes in Statistics could exceed the limit of an Int, we should use BigInt // instead of Int for avoiding possible overflow. @@ -1478,24 +1628,26 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val ds1 = spark.emptyDataset[Int] val ds2 = Seq(1, 2, 3).toDS() - assert(ds1.isEmpty == true) + assert(ds1.isEmpty) assert(ds2.isEmpty == false) } test("SPARK-22472: add null check for top-level primitive values") { // If the primitive values are from Option, we need to do runtime null check. val ds = Seq(Some(1), None).toDS().as[Int] - intercept[NullPointerException](ds.collect()) - val e = intercept[SparkException](ds.map(_ * 2).collect()) - assert(e.getCause.isInstanceOf[NullPointerException]) + val e1 = intercept[RuntimeException](ds.collect()) + assert(e1.getCause.isInstanceOf[NullPointerException]) + val e2 = intercept[SparkException](ds.map(_ * 2).collect()) + assert(e2.getCause.isInstanceOf[NullPointerException]) withTempPath { path => - Seq(new Integer(1), null).toDF("i").write.parquet(path.getCanonicalPath) + Seq(Integer.valueOf(1), null).toDF("i").write.parquet(path.getCanonicalPath) // If the primitive values are from files, we need to do runtime null check. val ds = spark.read.parquet(path.getCanonicalPath).as[Int] - intercept[NullPointerException](ds.collect()) - val e = intercept[SparkException](ds.map(_ * 2).collect()) - assert(e.getCause.isInstanceOf[NullPointerException]) + val e1 = intercept[RuntimeException](ds.collect()) + assert(e1.getCause.isInstanceOf[NullPointerException]) + val e2 = intercept[SparkException](ds.map(_ * 2).collect()) + assert(e2.getCause.isInstanceOf[NullPointerException]) } } @@ -1513,7 +1665,8 @@ class DatasetSuite extends QueryTest with SharedSQLContext { test("SPARK-23835: null primitive data type should throw NullPointerException") { val ds = Seq[(Option[Int], Option[Int])]((Some(1), None)).toDS() - intercept[NullPointerException](ds.as[(Int, Int)].collect()) + val e = intercept[RuntimeException](ds.as[(Int, Int)].collect()) + assert(e.getCause.isInstanceOf[NullPointerException]) } test("SPARK-24569: Option of primitive types are mistakenly mapped to struct type") { @@ -1544,9 +1697,157 @@ class DatasetSuite extends QueryTest with SharedSQLContext { val df = Seq("Amsterdam", "San Francisco", "X").toDF("city") checkAnswer(df.where('city === 'X'), Seq(Row("X"))) checkAnswer( - df.where($"city".contains(new java.lang.Character('A'))), + df.where($"city".contains(java.lang.Character.valueOf('A'))), Seq(Row("Amsterdam"))) } + + test("SPARK-24762: Enable top-level Option of Product encoders") { + val data = Seq(Some((1, "a")), Some((2, "b")), None) + val ds = data.toDS() + + checkDataset( + ds, + data: _*) + + val schema = new StructType().add( + "value", + new StructType() + .add("_1", IntegerType, nullable = false) + .add("_2", StringType, nullable = true), + nullable = true) + + assert(ds.schema == schema) + + val nestedOptData = Seq(Some((Some((1, "a")), 2.0)), Some((Some((2, "b")), 3.0))) + val nestedDs = nestedOptData.toDS() + + checkDataset( + nestedDs, + nestedOptData: _*) + + val nestedSchema = StructType(Seq( + StructField("value", StructType(Seq( + StructField("_1", StructType(Seq( + StructField("_1", IntegerType, nullable = false), + StructField("_2", StringType, nullable = true)))), + StructField("_2", DoubleType, nullable = false) + )), nullable = true) + )) + assert(nestedDs.schema == nestedSchema) + } + + test("SPARK-24762: Resolving Option[Product] field") { + val ds = Seq((1, ("a", 1.0)), (2, ("b", 2.0)), (3, null)).toDS() + .as[(Int, Option[(String, Double)])] + checkDataset(ds, + (1, Some(("a", 1.0))), (2, Some(("b", 2.0))), (3, None)) + } + + test("SPARK-24762: select Option[Product] field") { + val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS() + val ds1 = ds.select(expr("struct(_2, _2 + 1)").as[Option[(Int, Int)]]) + checkDataset(ds1, + Some((1, 2)), Some((2, 3)), Some((3, 4))) + + val ds2 = ds.select(expr("if(_2 > 2, struct(_2, _2 + 1), null)").as[Option[(Int, Int)]]) + checkDataset(ds2, + None, None, Some((3, 4))) + } + + test("SPARK-24762: joinWith on Option[Product]") { + val ds1 = Seq(Some((1, 2)), Some((2, 3)), None).toDS().as("a") + val ds2 = Seq(Some((1, 2)), Some((2, 3)), None).toDS().as("b") + val joined = ds1.joinWith(ds2, $"a.value._1" === $"b.value._2", "inner") + checkDataset(joined, (Some((2, 3)), Some((1, 2)))) + } + + test("SPARK-24762: typed agg on Option[Product] type") { + val ds = Seq(Some((1, 2)), Some((2, 3)), Some((1, 3))).toDS() + assert(ds.groupByKey(_.get._1).count().collect() === Seq((1, 2), (2, 1))) + + assert(ds.groupByKey(x => x).count().collect() === + Seq((Some((1, 2)), 1), (Some((2, 3)), 1), (Some((1, 3)), 1))) + } + + test("SPARK-25942: typed aggregation on primitive type") { + val ds = Seq(1, 2, 3).toDS() + + val agg = ds.groupByKey(_ >= 2) + .agg(sum("value").as[Long], sum($"value" + 1).as[Long]) + checkDatasetUnorderly(agg, (false, 1L, 2L), (true, 5L, 7L)) + } + + test("SPARK-25942: typed aggregation on product type") { + val ds = Seq((1, 2), (2, 3), (3, 4)).toDS() + val agg = ds.groupByKey(x => x).agg(sum("_1").as[Long], sum($"_2" + 1).as[Long]) + checkDatasetUnorderly(agg, ((1, 2), 1L, 3L), ((2, 3), 2L, 4L), ((3, 4), 3L, 5L)) + } + + test("SPARK-26085: fix key attribute name for atomic type for typed aggregation") { + val ds = Seq(1, 2, 3).toDS() + assert(ds.groupByKey(x => x).count().schema.head.name == "key") + + // Enable legacy flag to follow previous Spark behavior + withSQLConf(SQLConf.NAME_NON_STRUCT_GROUPING_KEY_AS_VALUE.key -> "true") { + assert(ds.groupByKey(x => x).count().schema.head.name == "value") + } + } + + test("SPARK-8288: class with only a companion object constructor") { + val data = Seq(ScroogeLikeExample(1), ScroogeLikeExample(2)) + val ds = data.toDS + checkDataset(ds, data: _*) + checkAnswer(ds.select("x"), Seq(Row(1), Row(2))) + } + + test("SPARK-26233: serializer should enforce decimal precision and scale") { + val s = StructType(Seq(StructField("a", StringType), StructField("b", DecimalType(38, 8)))) + val encoder = RowEncoder(s) + implicit val uEnc = encoder + val df = spark.range(2).map(l => Row(l.toString, BigDecimal.valueOf(l + 0.1111))) + checkAnswer(df.groupBy(col("a")).agg(first(col("b"))), + Seq(Row("0", BigDecimal.valueOf(0.1111)), Row("1", BigDecimal.valueOf(1.1111)))) + } + + test("SPARK-26366: return nulls which are not filtered in except") { + val inputDF = sqlContext.createDataFrame( + sparkContext.parallelize(Seq(Row("0", "a"), Row("1", null))), + StructType(Seq( + StructField("a", StringType, nullable = true), + StructField("b", StringType, nullable = true)))) + + val exceptDF = inputDF.filter(col("a").isin("0") or col("b") > "c") + checkAnswer(inputDF.except(exceptDF), Seq(Row("1", null))) + } + + test("SPARK-26706: Fix Cast.mayTruncate for bytes") { + val thrownException = intercept[AnalysisException] { + spark.range(Long.MaxValue - 10, Long.MaxValue).as[Byte] + .map(b => b - 1) + .collect() + } + assert(thrownException.message.contains("Cannot up cast `id` from bigint to tinyint")) + } + + test("SPARK-26690: checkpoints should be executed with an execution id") { + def assertExecutionId: UserDefinedFunction = udf(AssertExecutionId.apply _) + spark.range(10).select(assertExecutionId($"id")).localCheckpoint(true) + } + + test("implicit encoder for LocalDate and Instant") { + val localDate = java.time.LocalDate.of(2019, 3, 30) + assert(spark.range(1).map { _ => localDate }.head === localDate) + + val instant = java.time.Instant.parse("2019-03-30T09:54:00Z") + assert(spark.range(1).map { _ => instant }.head === instant) + } +} + +object AssertExecutionId { + def apply(id: Long): Long = { + assert(TaskContext.get().getLocalProperty(SQLExecution.EXECUTION_ID_KEY) != null) + id + } } case class TestDataUnion(x: Int, y: Int, z: Int) 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 c4ec7150c4075..ff48ac8d7a6c4 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 @@ -19,10 +19,13 @@ package org.apache.spark.sql import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat +import java.time.Instant import java.util.Locale +import java.util.concurrent.TimeUnit import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.unsafe.types.CalendarInterval @@ -298,11 +301,11 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, t1, d1), (3, t2, d2)).toDF("n", "t", "d") checkAnswer( df.selectExpr(s"d - $i"), - Seq(Row(Date.valueOf("2015-07-30")), Row(Date.valueOf("2015-12-30")))) + Seq(Row(Date.valueOf("2015-07-29")), Row(Date.valueOf("2015-12-28")))) checkAnswer( df.selectExpr(s"t - $i"), Seq(Row(Timestamp.valueOf("2015-07-31 23:59:59")), - Row(Timestamp.valueOf("2015-12-31 00:00:00")))) + Row(Timestamp.valueOf("2015-12-29 00:00:00")))) } test("function add_months") { @@ -311,10 +314,10 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df = Seq((1, d1), (2, d2)).toDF("n", "d") checkAnswer( df.select(add_months(col("d"), 1)), - Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-31")))) + Seq(Row(Date.valueOf("2015-09-30")), Row(Date.valueOf("2015-03-28")))) checkAnswer( df.selectExpr("add_months(d, -1)"), - Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-31")))) + Seq(Row(Date.valueOf("2015-07-31")), Row(Date.valueOf("2015-01-28")))) } test("function months_between") { @@ -405,7 +408,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { Row(Date.valueOf("2014-12-31")))) checkAnswer( df.select(to_date(col("s"), "yyyy-MM-dd")), - Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")), Row(null))) + Seq(Row(null), Row(Date.valueOf("2014-12-31")), Row(null))) // now switch format checkAnswer( @@ -515,6 +518,8 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { Seq(Row(sdf3.format(new Timestamp(1000000))), Row(sdf3.format(new Timestamp(-1000000))))) } + private def secs(millis: Long): Long = TimeUnit.MILLISECONDS.toSeconds(millis) + test("unix_timestamp") { val date1 = Date.valueOf("2015-07-24") val date2 = Date.valueOf("2015-07-25") @@ -527,21 +532,21 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val fmt = "yyyy/MM/dd HH:mm:ss.S" val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") checkAnswer(df.select(unix_timestamp(col("ts"))), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.select(unix_timestamp(col("ss"))), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.select(unix_timestamp(col("d"), fmt)), Seq( - Row(date1.getTime / 1000L), Row(date2.getTime / 1000L))) + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) checkAnswer(df.select(unix_timestamp(col("s"), fmt)), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.selectExpr("unix_timestamp(ts)"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.selectExpr("unix_timestamp(ss)"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.selectExpr(s"unix_timestamp(d, '$fmt')"), Seq( - Row(date1.getTime / 1000L), Row(date2.getTime / 1000L))) + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) checkAnswer(df.selectExpr(s"unix_timestamp(s, '$fmt')"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) val x1 = "2015-07-24 10:00:00" val x2 = "2015-25-07 02:02:02" @@ -552,13 +557,13 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df1 = Seq(x1, x2, x3, x4).toDF("x") checkAnswer(df1.select(unix_timestamp(col("x"))), Seq( - Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) checkAnswer(df1.selectExpr("unix_timestamp(x)"), Seq( - Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) checkAnswer(df1.select(unix_timestamp(col("x"), "yyyy-dd-MM HH:mm:ss")), Seq( - Row(null), Row(ts2.getTime / 1000L), Row(null), Row(null))) + Row(null), Row(secs(ts2.getTime)), Row(null), Row(null))) checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( - Row(ts4.getTime / 1000L), Row(null), Row(ts3.getTime / 1000L), Row(null))) + Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) // invalid format checkAnswer(df1.selectExpr(s"unix_timestamp(x, 'yyyy-MM-dd aa:HH:ss')"), Seq( @@ -570,10 +575,12 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") val df2 = Seq(y1, y2).toDF("y") checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( - Row(ts5.getTime / 1000L), Row(null))) + Row(secs(ts5.getTime)), Row(null))) val now = sql("select unix_timestamp()").collect().head.getLong(0) - checkAnswer(sql(s"select cast ($now as timestamp)"), Row(new java.util.Date(now * 1000))) + checkAnswer( + sql(s"select cast ($now as timestamp)"), + Row(new java.util.Date(TimeUnit.SECONDS.toMillis(now)))) } test("to_unix_timestamp") { @@ -588,13 +595,13 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val fmt = "yyyy/MM/dd HH:mm:ss.S" val df = Seq((date1, ts1, s1, ss1), (date2, ts2, s2, ss2)).toDF("d", "ts", "s", "ss") checkAnswer(df.selectExpr("to_unix_timestamp(ts)"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.selectExpr("to_unix_timestamp(ss)"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) checkAnswer(df.selectExpr(s"to_unix_timestamp(d, '$fmt')"), Seq( - Row(date1.getTime / 1000L), Row(date2.getTime / 1000L))) + Row(secs(date1.getTime)), Row(secs(date2.getTime)))) checkAnswer(df.selectExpr(s"to_unix_timestamp(s, '$fmt')"), Seq( - Row(ts1.getTime / 1000L), Row(ts2.getTime / 1000L))) + Row(secs(ts1.getTime)), Row(secs(ts2.getTime)))) val x1 = "2015-07-24 10:00:00" val x2 = "2015-25-07 02:02:02" @@ -605,9 +612,9 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val df1 = Seq(x1, x2, x3, x4).toDF("x") checkAnswer(df1.selectExpr("to_unix_timestamp(x)"), Seq( - Row(ts1.getTime / 1000L), Row(null), Row(null), Row(null))) + Row(secs(ts1.getTime)), Row(null), Row(null), Row(null))) checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd mm:HH:ss')"), Seq( - Row(ts4.getTime / 1000L), Row(null), Row(ts3.getTime / 1000L), Row(null))) + Row(secs(ts4.getTime)), Row(null), Row(secs(ts3.getTime)), Row(null))) // february val y1 = "2016-02-29" @@ -615,7 +622,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val ts5 = Timestamp.valueOf("2016-02-29 00:00:00") val df2 = Seq(y1, y2).toDF("y") checkAnswer(df2.select(unix_timestamp(col("y"), "yyyy-MM-dd")), Seq( - Row(ts5.getTime / 1000L), Row(null))) + Row(secs(ts5.getTime)), Row(null))) // invalid format checkAnswer(df1.selectExpr(s"to_unix_timestamp(x, 'yyyy-MM-dd bb:HH:ss')"), Seq( @@ -632,6 +639,8 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { val ts2 = Timestamp.valueOf("2015-07-25 02:02:02") val s1 = "2015/07/24 10:00:00.5" val s2 = "2015/07/25 02:02:02.6" + val ts1m = Timestamp.valueOf("2015-07-24 10:00:00.5") + val ts2m = Timestamp.valueOf("2015-07-25 02:02:02.6") val ss1 = "2015-07-24 10:00:00" val ss2 = "2015-07-25 02:02:02" val fmt = "yyyy/MM/dd HH:mm:ss.S" @@ -642,7 +651,7 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer(df.select(to_timestamp(col("ss"))), Seq( Row(ts1), Row(ts2))) checkAnswer(df.select(to_timestamp(col("s"), fmt)), Seq( - Row(ts1), Row(ts2))) + Row(ts1m), Row(ts2m))) checkAnswer(df.select(to_timestamp(col("ts"), fmt)), Seq( Row(ts1), Row(ts2))) checkAnswer(df.select(to_timestamp(col("d"), "yyyy-MM-dd")), Seq( @@ -667,33 +676,41 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), (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")), - 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")), - Seq( - Row(Timestamp.valueOf("2015-07-23 17:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + checkAnswer( + df.select(from_utc_timestamp(col("a"), "PST")), + 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")), + Seq( + Row(Timestamp.valueOf("2015-07-23 17:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } + val msg = intercept[AnalysisException] { + df.select(from_utc_timestamp(col("a"), "PST")).collect() + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) } 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") - ).toDF("a", "b", "c") - checkAnswer( - df.select(from_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) - checkAnswer( - df.select(from_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 02:00:00")), - Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + 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") + ).toDF("a", "b", "c") + checkAnswer( + df.select(from_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + checkAnswer( + df.select(from_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 02:00:00")), + Row(Timestamp.valueOf("2015-07-24 17:00:00")))) + } } test("to_utc_timestamp with literal zone") { @@ -701,32 +718,50 @@ class DateFunctionsSuite extends QueryTest with SharedSQLContext { (Timestamp.valueOf("2015-07-24 00:00:00"), "2015-07-24 00:00:00"), (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")), - 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")), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + checkAnswer( + df.select(to_utc_timestamp(col("a"), "PST")), + 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")), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-25 07:00:00")))) + } + val msg = intercept[AnalysisException] { + df.select(to_utc_timestamp(col("a"), "PST")).collect() + }.getMessage + assert(msg.contains(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key)) } 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") - ).toDF("a", "b", "c") - checkAnswer( - df.select(to_utc_timestamp(col("a"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) - checkAnswer( - df.select(to_utc_timestamp(col("b"), col("c"))), - Seq( - Row(Timestamp.valueOf("2015-07-24 07:00:00")), - Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + 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") + ).toDF("a", "b", "c") + checkAnswer( + df.select(to_utc_timestamp(col("a"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + checkAnswer( + df.select(to_utc_timestamp(col("b"), col("c"))), + Seq( + Row(Timestamp.valueOf("2015-07-24 07:00:00")), + Row(Timestamp.valueOf("2015-07-24 22:00:00")))) + } + } + + + test("to_timestamp with microseconds precision") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val timestamp = "1970-01-01T00:00:00.123456Z" + val df = Seq(timestamp).toDF("t") + checkAnswer(df.select(to_timestamp($"t", "yyyy-MM-dd'T'HH:mm:ss.SSSSSSX")), + Seq(Row(Instant.parse(timestamp)))) + } } } 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 56d300e30a58e..ec688282d5740 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -24,15 +25,25 @@ class ExplainSuite extends QueryTest with SharedSQLContext { import testImplicits._ /** - * Runs the plan and makes sure the plans contains all of the keywords. + * Get the explain from a DataFrame and run the specified action on it. */ - private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { + private def withNormalizedExplain(df: DataFrame, extended: Boolean)(f: String => Unit) = { val output = new java.io.ByteArrayOutputStream() Console.withOut(output) { - df.explain(extended = false) + df.explain(extended = extended) } - for (key <- keywords) { - assert(output.toString.contains(key)) + val normalizedOutput = output.toString.replaceAll("#\\d+", "#x") + f(normalizedOutput) + } + + /** + * Runs the plan and makes sure the plans contains all of the keywords. + */ + private def checkKeywordsExistsInExplain(df: DataFrame, keywords: String*): Unit = { + withNormalizedExplain(df, extended = true) { normalizedOutput => + for (key <- keywords) { + assert(normalizedOutput.contains(key)) + } } } @@ -53,6 +64,142 @@ class ExplainSuite extends QueryTest with SharedSQLContext { checkKeywordsExistsInExplain(df, keywords = "InMemoryRelation", "StorageLevel(disk, memory, deserialized, 1 replicas)") } + + test("optimized plan should show the rewritten aggregate expression") { + withTempView("test_agg") { + sql( + """ + |CREATE TEMPORARY VIEW test_agg AS SELECT * FROM VALUES + | (1, true), (1, false), + | (2, true), + | (3, false), (3, null), + | (4, null), (4, null), + | (5, null), (5, true), (5, false) AS test_agg(k, v) + """.stripMargin) + + // simple explain of queries having every/some/any aggregates. Optimized + // 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]") + } + } + + test("explain inline tables cross-joins") { + val df = sql( + """ + |SELECT * FROM VALUES ('one', 1), ('three', null) + | CROSS JOIN VALUES ('one', 1), ('three', null) + """.stripMargin) + checkKeywordsExistsInExplain(df, + "Join Cross", + ":- LocalRelation [col1#x, col2#x]", + "+- LocalRelation [col1#x, col2#x]") + } + + test("explain table valued functions") { + checkKeywordsExistsInExplain(sql("select * from RaNgE(2)"), "Range (0, 2, step=1, splits=None)") + checkKeywordsExistsInExplain(sql("SELECT * FROM range(3) CROSS JOIN range(3)"), + "Join Cross", + ":- Range (0, 3, step=1, splits=None)", + "+- Range (0, 3, step=1, splits=None)") + } + + test("explain string functions") { + // Check if catalyst combine nested `Concat`s + val df1 = sql( + """ + |SELECT (col1 || col2 || col3 || col4) col + | FROM (SELECT id col1, id col2, id col3, id col4 FROM range(10)) + """.stripMargin) + checkKeywordsExistsInExplain(df1, + "Project [concat(cast(id#xL as string), cast(id#xL as string), cast(id#xL as string)" + + ", cast(id#xL as string)) AS col#x]") + + // Check if catalyst combine nested `Concat`s if concatBinaryAsString=false + withSQLConf(SQLConf.CONCAT_BINARY_AS_STRING.key -> "false") { + val df2 = sql( + """ + |SELECT ((col1 || col2) || (col3 || col4)) col + |FROM ( + | SELECT + | string(id) col1, + | string(id + 1) col2, + | encode(string(id + 2), 'utf-8') col3, + | encode(string(id + 3), 'utf-8') col4 + | FROM range(10) + |) + """.stripMargin) + checkKeywordsExistsInExplain(df2, + "Project [concat(cast(id#xL as string), cast((id#xL + 1) as string), " + + "cast(encode(cast((id#xL + 2) as string), utf-8) as string), " + + "cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x]") + + val df3 = sql( + """ + |SELECT (col1 || (col3 || col4)) col + |FROM ( + | SELECT + | string(id) col1, + | encode(string(id + 2), 'utf-8') col3, + | encode(string(id + 3), 'utf-8') col4 + | FROM range(10) + |) + """.stripMargin) + checkKeywordsExistsInExplain(df3, + "Project [concat(cast(id#xL as string), " + + "cast(encode(cast((id#xL + 2) as string), utf-8) as string), " + + "cast(encode(cast((id#xL + 3) as string), utf-8) as string)) AS col#x]") + } + } + + test("check operator precedence") { + // We follow Oracle operator precedence in the table below that lists the levels + // of precedence among SQL operators from high to low: + // --------------------------------------------------------------------------------------- + // Operator Operation + // --------------------------------------------------------------------------------------- + // +, - identity, negation + // *, / multiplication, division + // +, -, || addition, subtraction, concatenation + // =, !=, <, >, <=, >=, IS NULL, LIKE, BETWEEN, IN comparison + // NOT exponentiation, logical negation + // AND conjunction + // OR disjunction + // --------------------------------------------------------------------------------------- + checkKeywordsExistsInExplain(sql("select 'a' || 1 + 2"), + "Project [null AS (CAST(concat(a, CAST(1 AS STRING)) AS DOUBLE) + CAST(2 AS DOUBLE))#x]") + checkKeywordsExistsInExplain(sql("select 1 - 2 || 'b'"), + "Project [-1b AS concat(CAST((1 - 2) AS STRING), b)#x]") + checkKeywordsExistsInExplain(sql("select 2 * 4 + 3 || 'b'"), + "Project [11b AS concat(CAST(((2 * 4) + 3) AS STRING), b)#x]") + checkKeywordsExistsInExplain(sql("select 3 + 1 || 'a' || 4 / 2"), + "Project [4a2.0 AS concat(concat(CAST((3 + 1) AS STRING), a), " + + "CAST((CAST(4 AS DOUBLE) / CAST(2 AS DOUBLE)) AS STRING))#x]") + checkKeywordsExistsInExplain(sql("select 1 == 1 OR 'a' || 'b' == 'ab'"), + "Project [true AS ((1 = 1) OR (concat(a, b) = ab))#x]") + checkKeywordsExistsInExplain(sql("select 'a' || 'c' == 'ac' AND 2 == 3"), + "Project [false AS ((concat(a, c) = ac) AND (2 = 3))#x]") + } + + test("explain for these functions; use range to avoid constant folding") { + val df = sql("select ifnull(id, 'x'), nullif(id, 'x'), nvl(id, 'x'), nvl2(id, 'x', 'y') " + + "from range(2)") + checkKeywordsExistsInExplain(df, + "Project [coalesce(cast(id#xL as string), x) AS ifnull(`id`, 'x')#x, " + + "id#xL AS nullif(`id`, 'x')#xL, coalesce(cast(id#xL as string), x) AS nvl(`id`, 'x')#x, " + + "x AS nvl2(`id`, 'x', 'y')#x]") + } + + test("SPARK-26659: explain of DataWritingCommandExec should not contain duplicate cmd.nodeName") { + withTable("temptable") { + val df = sql("create table temptable using parquet as select * from range(2)") + withNormalizedExplain(df, extended = false) { normalizedOutput => + assert("Create\\w*?TableAsSelectCommand".r.findAllMatchIn(normalizedOutput).length == 1) + } + } + } } case class ExplainSingleData(id: Int) 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 94f163708832c..89195284a5b5f 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 @@ -17,7 +17,8 @@ package org.apache.spark.sql -import java.io.{File, FileNotFoundException} +import java.io.{File, FilenameFilter, FileNotFoundException} +import java.nio.file.{Files, StandardOpenOption} import java.util.Locale import scala.collection.mutable @@ -28,6 +29,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkException import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT, NullData, NullUDT} +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -312,13 +314,13 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo assert(msg.contains("CSV data source does not support array data type")) msg = intercept[AnalysisException] { - Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") + Seq((1, new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25)))).toDF("id", "vectors") .write.mode("overwrite").csv(csvDir) }.getMessage assert(msg.contains("CSV data source does not support array data type")) msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", new UDT.MyDenseVectorUDT(), true) :: Nil) + val schema = StructType(StructField("a", new TestUDT.MyDenseVectorUDT(), true) :: Nil) spark.range(1).write.mode("overwrite").csv(csvDir) spark.read.schema(schema).csv(csvDir).collect() }.getMessage @@ -329,107 +331,99 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo test("SPARK-24204 error handling for unsupported Interval data types - csv, json, parquet, orc") { withTempDir { dir => val tempDir = new File(dir, "files").getCanonicalPath + // TODO: test file source V2 after write path is fixed. + Seq(true).foreach { useV1 => + val useV1List = if (useV1) { + "csv,json,orc,parquet" + } else { + "" + } + def validateErrorMessage(msg: String): Unit = { + val msg1 = "cannot save interval data type into external storage." + val msg2 = "data source does not support calendarinterval data type." + assert(msg.toLowerCase(Locale.ROOT).contains(msg1) || + msg.toLowerCase(Locale.ROOT).contains(msg2)) + } - // write path - Seq("csv", "json", "parquet", "orc").foreach { format => - var msg = intercept[AnalysisException] { - sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.contains("Cannot save interval data type into external storage.")) - - msg = intercept[AnalysisException] { - spark.udf.register("testType", () => new IntervalData()) - sql("select testType()").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support calendarinterval data type.")) - } + withSQLConf(SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> useV1List) { + // write path + Seq("csv", "json", "parquet", "orc").foreach { format => + val msg = intercept[AnalysisException] { + sql("select interval 1 days").write.format(format).mode("overwrite").save(tempDir) + }.getMessage + validateErrorMessage(msg) + } - // read path - Seq("parquet", "csv").foreach { format => - var msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", CalendarIntervalType, true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support calendarinterval data type.")) - - msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", new IntervalUDT(), true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support calendarinterval data type.")) + // read path + Seq("parquet", "csv").foreach { format => + var msg = intercept[AnalysisException] { + val schema = StructType(StructField("a", CalendarIntervalType, true) :: Nil) + spark.range(1).write.format(format).mode("overwrite").save(tempDir) + spark.read.schema(schema).format(format).load(tempDir).collect() + }.getMessage + validateErrorMessage(msg) + + msg = intercept[AnalysisException] { + val schema = StructType(StructField("a", new IntervalUDT(), true) :: Nil) + spark.range(1).write.format(format).mode("overwrite").save(tempDir) + spark.read.schema(schema).format(format).load(tempDir).collect() + }.getMessage + validateErrorMessage(msg) + } + } } } } test("SPARK-24204 error handling for unsupported Null data types - csv, parquet, orc") { - withTempDir { dir => - val tempDir = new File(dir, "files").getCanonicalPath - - Seq("orc").foreach { format => - // write path - var msg = intercept[AnalysisException] { - sql("select null").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) - - msg = intercept[AnalysisException] { - spark.udf.register("testType", () => new NullData()) - sql("select testType()").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) - - // read path - // We expect the types below should be passed for backward-compatibility - - // Null type - var schema = StructType(StructField("a", NullType, true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() - - // UDT having null data - schema = StructType(StructField("a", new NullUDT(), true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() + // TODO: test file source V2 after write path is fixed. + Seq(true).foreach { useV1 => + val useV1List = if (useV1) { + "csv,orc,parquet" + } else { + "" } - - Seq("parquet", "csv").foreach { format => - // write path - var msg = intercept[AnalysisException] { - sql("select null").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) - - msg = intercept[AnalysisException] { - spark.udf.register("testType", () => new NullData()) - sql("select testType()").write.format(format).mode("overwrite").save(tempDir) - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) - - // read path - msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", NullType, true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) - - msg = intercept[AnalysisException] { - val schema = StructType(StructField("a", new NullUDT(), true) :: Nil) - spark.range(1).write.format(format).mode("overwrite").save(tempDir) - spark.read.schema(schema).format(format).load(tempDir).collect() - }.getMessage - assert(msg.toLowerCase(Locale.ROOT) - .contains(s"$format data source does not support null data type.")) + def errorMessage(format: String): String = { + s"$format data source does not support null data type." + } + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1List, + SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> useV1List) { + withTempDir { dir => + val tempDir = new File(dir, "files").getCanonicalPath + + Seq("parquet", "csv", "orc").foreach { format => + // write path + var msg = intercept[AnalysisException] { + sql("select null").write.format(format).mode("overwrite").save(tempDir) + }.getMessage + assert(msg.toLowerCase(Locale.ROOT) + .contains(errorMessage(format))) + + msg = intercept[AnalysisException] { + spark.udf.register("testType", () => new NullData()) + sql("select testType()").write.format(format).mode("overwrite").save(tempDir) + }.getMessage + assert(msg.toLowerCase(Locale.ROOT) + .contains(errorMessage(format))) + + // read path + msg = intercept[AnalysisException] { + val schema = StructType(StructField("a", NullType, true) :: Nil) + spark.range(1).write.format(format).mode("overwrite").save(tempDir) + spark.read.schema(schema).format(format).load(tempDir).collect() + }.getMessage + assert(msg.toLowerCase(Locale.ROOT) + .contains(errorMessage(format))) + + msg = intercept[AnalysisException] { + val schema = StructType(StructField("a", new NullUDT(), true) :: Nil) + spark.range(1).write.format(format).mode("overwrite").save(tempDir) + spark.read.schema(schema).format(format).load(tempDir).collect() + }.getMessage + assert(msg.toLowerCase(Locale.ROOT) + .contains(errorMessage(format))) + } + } } } } @@ -479,22 +473,237 @@ class FileBasedDataSourceSuite extends QueryTest with SharedSQLContext with Befo } test("SPARK-25237 compute correct input metrics in FileScanRDD") { - withTempPath { p => - val path = p.getAbsolutePath - spark.range(1000).repartition(1).write.csv(path) - val bytesReads = new mutable.ArrayBuffer[Long]() - val bytesReadListener = new SparkListener() { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + // TODO: Test CSV V2 as well after it implements [[SupportsReportStatistics]]. + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "csv") { + withTempPath { p => + val path = p.getAbsolutePath + spark.range(1000).repartition(1).write.csv(path) + val bytesReads = new mutable.ArrayBuffer[Long]() + val bytesReadListener = new SparkListener() { + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + bytesReads += taskEnd.taskMetrics.inputMetrics.bytesRead + } + } + sparkContext.addSparkListener(bytesReadListener) + try { + spark.read.csv(path).limit(1).collect() + sparkContext.listenerBus.waitUntilEmpty(1000L) + assert(bytesReads.sum === 7860) + } finally { + sparkContext.removeSparkListener(bytesReadListener) + } + } + } + } + + test("Do not use cache on overwrite") { + Seq("", "orc").foreach { useV1SourceReaderList => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReaderList) { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("overwrite").orc(path) + val df = spark.read.orc(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("overwrite").orc(path) + assert(df.count() == 10) + assert(spark.read.orc(path).count() == 10) + } + } + } + } + + test("Do not use cache on append") { + Seq("", "orc").foreach { useV1SourceReaderList => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReaderList) { + withTempDir { dir => + val path = dir.toString + spark.range(1000).write.mode("append").orc(path) + val df = spark.read.orc(path).cache() + assert(df.count() == 1000) + spark.range(10).write.mode("append").orc(path) + assert(df.count() == 1010) + assert(spark.read.orc(path).count() == 1010) + } + } + } + } + + test("UDF input_file_name()") { + Seq("", "orc").foreach { useV1SourceReaderList => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReaderList) { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(10).write.orc(path) + val row = spark.read.orc(path).select(input_file_name).first() + assert(row.getString(0).contains(path)) } } - sparkContext.addSparkListener(bytesReadListener) - try { - spark.read.csv(path).limit(1).collect() - sparkContext.listenerBus.waitUntilEmpty(1000L) - assert(bytesReads.sum === 7860) - } finally { - sparkContext.removeSparkListener(bytesReadListener) + } + } + + test("Option pathGlobFilter: filter files correctly") { + withTempPath { path => + val dataDir = path.getCanonicalPath + Seq("foo").toDS().write.text(dataDir) + Seq("bar").toDS().write.mode("append").orc(dataDir) + val df = spark.read.option("pathGlobFilter", "*.txt").text(dataDir) + checkAnswer(df, Row("foo")) + + // Both glob pattern in option and path should be effective to filter files. + val df2 = spark.read.option("pathGlobFilter", "*.txt").text(dataDir + "/*.orc") + checkAnswer(df2, Seq.empty) + + val df3 = spark.read.option("pathGlobFilter", "*.txt").text(dataDir + "/*xt") + checkAnswer(df3, Row("foo")) + } + } + + test("Option pathGlobFilter: simple extension filtering should contains partition info") { + withTempPath { path => + val input = Seq(("foo", 1), ("oof", 2)).toDF("a", "b") + input.write.partitionBy("b").text(path.getCanonicalPath) + Seq("bar").toDS().write.mode("append").orc(path.getCanonicalPath + "/b=1") + + // If we use glob pattern in the path, the partition column won't be shown in the result. + val df = spark.read.text(path.getCanonicalPath + "/*/*.txt") + checkAnswer(df, input.select("a")) + + val df2 = spark.read.option("pathGlobFilter", "*.txt").text(path.getCanonicalPath) + checkAnswer(df2, input) + } + } + + test("Option recursiveFileLookup: recursive loading correctly") { + + val expectedFileList = mutable.ListBuffer[String]() + + def createFile(dir: File, fileName: String, format: String): Unit = { + val path = new File(dir, s"${fileName}.${format}") + Files.write( + path.toPath, + s"content of ${path.toString}".getBytes, + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + val fsPath = new Path(path.getAbsoluteFile.toURI).toString + expectedFileList.append(fsPath) + } + + def createDir(path: File, dirName: String, level: Int): Unit = { + val dir = new File(path, s"dir${dirName}-${level}") + dir.mkdir() + createFile(dir, s"file${level}", "bin") + createFile(dir, s"file${level}", "text") + + if (level < 4) { + // create sub-dir + createDir(dir, "sub0", level + 1) + createDir(dir, "sub1", level + 1) + } + } + + withTempPath { path => + path.mkdir() + createDir(path, "root", 0) + + val dataPath = new File(path, "dirroot-0").getAbsolutePath + val fileList = spark.read.format("binaryFile") + .option("recursiveFileLookup", true) + .load(dataPath) + .select("path").collect().map(_.getString(0)) + + assert(fileList.toSet === expectedFileList.toSet) + + val fileList2 = spark.read.format("binaryFile") + .option("recursiveFileLookup", true) + .option("pathGlobFilter", "*.bin") + .load(dataPath) + .select("path").collect().map(_.getString(0)) + + assert(fileList2.toSet === expectedFileList.filter(_.endsWith(".bin")).toSet) + } + } + + test("Option recursiveFileLookup: disable partition inferring") { + val dataPath = Thread.currentThread().getContextClassLoader + .getResource("test-data/text-partitioned").toString + + val df = spark.read.format("binaryFile") + .option("recursiveFileLookup", true) + .load(dataPath) + + assert(!df.columns.contains("year"), "Expect partition inferring disabled") + val fileList = df.select("path").collect().map(_.getString(0)) + + val expectedFileList = Array( + dataPath + "/year=2014/data.txt", + dataPath + "/year=2015/data.txt" + ).map(path => new Path(path).toString) + + assert(fileList.toSet === expectedFileList.toSet) + } + + test("Return correct results when data columns overlap with partition columns") { + Seq("parquet", "orc", "json").foreach { format => + withTempPath { path => + val tablePath = new File(s"${path.getCanonicalPath}/cOl3=c/cOl1=a/cOl5=e") + Seq((1, 2, 3, 4, 5)).toDF("cOl1", "cOl2", "cOl3", "cOl4", "cOl5") + .write.format(format).save(tablePath.getCanonicalPath) + + val df = spark.read.format(format).load(path.getCanonicalPath) + .select("CoL1", "Col2", "CoL5", "CoL3") + checkAnswer(df, Row("a", 2, "e", "c")) + } + } + } + + test("sizeInBytes should be the total size of all files") { + Seq("orc", "").foreach { useV1SourceReaderList => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReaderList) { + withTempDir { dir => + dir.delete() + spark.range(1000).write.orc(dir.toString) + val df = spark.read.orc(dir.toString) + assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(getLocalDirSize(dir))) + } + } + } + } + + test("SPARK-22790,SPARK-27668: spark.sql.sources.compressionFactor takes effect") { + Seq(1.0, 0.5).foreach { compressionFactor => + withSQLConf(SQLConf.FILE_COMRESSION_FACTOR.key -> compressionFactor.toString, + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "250") { + withTempPath { workDir => + // the file size is 486 bytes + val workDirPath = workDir.getAbsolutePath + val data1 = Seq(100, 200, 300, 400).toDF("count") + data1.write.orc(workDirPath + "/data1") + val df1FromFile = spark.read.orc(workDirPath + "/data1") + val data2 = Seq(100, 200, 300, 400).toDF("count") + data2.write.orc(workDirPath + "/data2") + val df2FromFile = spark.read.orc(workDirPath + "/data2") + val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) + if (compressionFactor == 0.5) { + val bJoinExec = joinedDF.queryExecution.executedPlan.collect { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.nonEmpty) + val smJoinExec = joinedDF.queryExecution.executedPlan.collect { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.isEmpty) + } else { + // compressionFactor is 1.0 + val bJoinExec = joinedDF.queryExecution.executedPlan.collect { + case bJoin: BroadcastHashJoinExec => bJoin + } + assert(bJoinExec.isEmpty) + val smJoinExec = joinedDF.queryExecution.executedPlan.collect { + case smJoin: SortMergeJoinExec => smJoin + } + assert(smJoinExec.nonEmpty) + } + } } } } @@ -509,9 +718,9 @@ object TestingUDT { override def sqlType: DataType = CalendarIntervalType override def serialize(obj: IntervalData): Any = - throw new NotImplementedError("Not implemented") + throw new UnsupportedOperationException("Not implemented") override def deserialize(datum: Any): IntervalData = - throw new NotImplementedError("Not implemented") + throw new UnsupportedOperationException("Not implemented") override def userClass: Class[IntervalData] = classOf[IntervalData] } @@ -521,9 +730,10 @@ object TestingUDT { private[sql] class NullUDT extends UserDefinedType[NullData] { override def sqlType: DataType = NullType - override def serialize(obj: NullData): Any = throw new NotImplementedError("Not implemented") + override def serialize(obj: NullData): Any = + throw new UnsupportedOperationException("Not implemented") override def deserialize(datum: Any): NullData = - throw new NotImplementedError("Not implemented") + throw new UnsupportedOperationException("Not implemented") override def userClass: Class[NullData] = classOf[NullData] } } 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 new file mode 100644 index 0000000000000..d62fe961117a9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/IntegratedUDFTestUtils.scala @@ -0,0 +1,346 @@ +/* + * 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 java.nio.file.{Files, Paths} + +import scala.collection.JavaConverters._ +import scala.util.Try + +import org.apache.spark.TestUtils +import org.apache.spark.api.python.{PythonBroadcast, PythonEvalType, PythonFunction, PythonUtils} +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.internal.config.Tests +import org.apache.spark.sql.catalyst.expressions.{Cast, Expression} +import org.apache.spark.sql.catalyst.plans.SQLHelper +import org.apache.spark.sql.execution.python.UserDefinedPythonFunction +import org.apache.spark.sql.expressions.SparkUserDefinedFunction +import org.apache.spark.sql.types.StringType + +/** + * This object targets to integrate various UDF test cases so that Scalar UDF, Python UDF and + * Scalar Pandas UDFs can be tested in SBT & Maven tests. + * + * The available UDFs are special. It defines an UDF wrapped by cast. So, the input column is + * casted into string, UDF returns strings as are, and then output column is casted back to + * the input column. In this way, UDF is virtually no-op. + * + * Note that, due to this implementation limitation, complex types such as map, array and struct + * types do not work with this UDFs because they cannot be same after the cast roundtrip. + * + * To register Scala UDF in SQL: + * {{{ + * val scalaTestUDF = TestScalaUDF(name = "udf_name") + * registerTestUDF(scalaTestUDF, spark) + * }}} + * + * To register Python UDF in SQL: + * {{{ + * val pythonTestUDF = TestPythonUDF(name = "udf_name") + * registerTestUDF(pythonTestUDF, spark) + * }}} + * + * To register Scalar Pandas UDF in SQL: + * {{{ + * val pandasTestUDF = TestScalarPandasUDF(name = "udf_name") + * registerTestUDF(pandasTestUDF, spark) + * }}} + * + * To use it in Scala API and SQL: + * {{{ + * sql("SELECT udf_name(1)") + * val df = spark.range(10) + * df.select(expr("udf_name(id)") + * df.select(pandasTestUDF(df("id"))) + * }}} + */ +object IntegratedUDFTestUtils extends SQLHelper { + import scala.sys.process._ + + private lazy val pythonPath = sys.env.getOrElse("PYTHONPATH", "") + private lazy val sparkHome = if (sys.props.contains(Tests.IS_TESTING.key)) { + assert(sys.props.contains("spark.test.home") || + sys.env.contains("SPARK_HOME"), "spark.test.home or SPARK_HOME is not set.") + sys.props.getOrElse("spark.test.home", sys.env("SPARK_HOME")) + } else { + assert(sys.env.contains("SPARK_HOME"), "SPARK_HOME is not set.") + sys.env("SPARK_HOME") + } + // Note that we will directly refer pyspark's source, not the zip from a regular build. + // It is possible the test is being ran without the build. + private lazy val sourcePath = Paths.get(sparkHome, "python").toAbsolutePath + private lazy val py4jPath = Paths.get( + sparkHome, "python", "lib", PythonUtils.PY4J_ZIP_NAME).toAbsolutePath + private lazy val pysparkPythonPath = s"$py4jPath:$sourcePath" + + private lazy val isPythonAvailable: Boolean = TestUtils.testCommandAvailable(pythonExec) + + private lazy val isPySparkAvailable: Boolean = isPythonAvailable && Try { + Process( + Seq(pythonExec, "-c", "import pyspark"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! + true + }.getOrElse(false) + + private lazy val isPandasAvailable: Boolean = isPythonAvailable && isPySparkAvailable && Try { + Process( + Seq( + pythonExec, + "-c", + "from pyspark.sql.utils import require_minimum_pandas_version;" + + "require_minimum_pandas_version()"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! + true + }.getOrElse(false) + + private lazy val isPyArrowAvailable: Boolean = isPythonAvailable && isPySparkAvailable && Try { + Process( + Seq( + pythonExec, + "-c", + "from pyspark.sql.utils import require_minimum_pyarrow_version;" + + "require_minimum_pyarrow_version()"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! + true + }.getOrElse(false) + + private lazy val pythonVer = if (isPythonAvailable) { + Process( + Seq(pythonExec, "-c", "import sys; print('%d.%d' % sys.version_info[:2])"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!!.trim() + } else { + throw new RuntimeException(s"Python executable [$pythonExec] is unavailable.") + } + + // Dynamically pickles and reads the Python instance into JVM side in order to mimic + // Python native function within Python UDF. + private lazy val pythonFunc: Array[Byte] = if (shouldTestPythonUDFs) { + var binaryPythonFunc: Array[Byte] = null + withTempPath { path => + Process( + Seq( + pythonExec, + "-c", + "from pyspark.sql.types import StringType; " + + "from pyspark.serializers import CloudPickleSerializer; " + + s"f = open('$path', 'wb');" + + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: None if x is None else str(x), StringType())))"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! + binaryPythonFunc = Files.readAllBytes(path.toPath) + } + assert(binaryPythonFunc != null) + binaryPythonFunc + } else { + throw new RuntimeException(s"Python executable [$pythonExec] and/or pyspark are unavailable.") + } + + private lazy val pandasFunc: Array[Byte] = if (shouldTestScalarPandasUDFs) { + var binaryPandasFunc: Array[Byte] = null + withTempPath { path => + Process( + Seq( + pythonExec, + "-c", + "from pyspark.sql.types import StringType; " + + "from pyspark.serializers import CloudPickleSerializer; " + + s"f = open('$path', 'wb');" + + "f.write(CloudPickleSerializer().dumps((" + + "lambda x: x.apply(" + + "lambda v: None if v is None else str(v)), StringType())))"), + None, + "PYTHONPATH" -> s"$pysparkPythonPath:$pythonPath").!! + binaryPandasFunc = Files.readAllBytes(path.toPath) + } + assert(binaryPandasFunc != null) + binaryPandasFunc + } else { + throw new RuntimeException(s"Python executable [$pythonExec] and/or pyspark are unavailable.") + } + + // Make sure this map stays mutable - this map gets updated later in Python runners. + private val workerEnv = new java.util.HashMap[String, String]() + workerEnv.put("PYTHONPATH", s"$pysparkPythonPath:$pythonPath") + + lazy val pythonExec: String = { + val pythonExec = sys.env.getOrElse( + "PYSPARK_DRIVER_PYTHON", sys.env.getOrElse("PYSPARK_PYTHON", "python3.6")) + if (TestUtils.testCommandAvailable(pythonExec)) { + pythonExec + } else { + "python" + } + } + + lazy val shouldTestPythonUDFs: Boolean = isPythonAvailable && isPySparkAvailable + + lazy val shouldTestScalarPandasUDFs: Boolean = + isPythonAvailable && isPandasAvailable && isPyArrowAvailable + + /** + * A base trait for various UDFs defined in this object. + */ + sealed trait TestUDF { + def apply(exprs: Column*): Column + + val prettyName: String + } + + /** + * A Python UDF that takes one column, casts into string, executes the Python native function, + * and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import udf + * + * df = spark.range(3).toDF("col") + * python_udf = udf(lambda x: str(x), "string") + * casted_col = python_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} + */ + case class TestPythonUDF(name: String) extends TestUDF { + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( + name = name, + func = PythonFunction( + command = pythonFunc, + envVars = workerEnv.clone().asInstanceOf[java.util.Map[String, String]], + pythonIncludes = List.empty[String].asJava, + pythonExec = pythonExec, + pythonVer = pythonVer, + broadcastVars = List.empty[Broadcast[PythonBroadcast]].asJava, + accumulator = null), + dataType = StringType, + pythonEvalType = PythonEvalType.SQL_BATCHED_UDF, + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } + + def apply(exprs: Column*): Column = udf(exprs: _*) + + val prettyName: String = "Regular Python UDF" + } + + /** + * A Scalar Pandas UDF that takes one column, casts into string, executes the + * Python native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * from pyspark.sql.functions import pandas_udf + * + * df = spark.range(3).toDF("col") + * scalar_udf = pandas_udf(lambda x: x.apply(lambda v: str(v)), "string") + * casted_col = scalar_udf(df.col.cast("string")) + * casted_col.cast(df.schema["col"].dataType) + * }}} + */ + case class TestScalarPandasUDF(name: String) extends TestUDF { + private[IntegratedUDFTestUtils] lazy val udf = new UserDefinedPythonFunction( + name = name, + func = PythonFunction( + command = pandasFunc, + envVars = workerEnv.clone().asInstanceOf[java.util.Map[String, String]], + pythonIncludes = List.empty[String].asJava, + pythonExec = pythonExec, + pythonVer = pythonVer, + broadcastVars = List.empty[Broadcast[PythonBroadcast]].asJava, + accumulator = null), + dataType = StringType, + pythonEvalType = PythonEvalType.SQL_SCALAR_PANDAS_UDF, + udfDeterministic = true) { + + override def builder(e: Seq[Expression]): Expression = { + assert(e.length == 1, "Defined UDF only has one column") + val expr = e.head + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Cast(super.builder(Cast(expr, StringType) :: Nil), expr.dataType) + } + } + + def apply(exprs: Column*): Column = udf(exprs: _*) + + val prettyName: String = "Scalar Pandas UDF" + } + + /** + * A Scala UDF that takes one column, casts into string, executes the + * Scala native function, and casts back to the type of input column. + * + * Virtually equivalent to: + * + * {{{ + * import org.apache.spark.sql.functions.udf + * + * val df = spark.range(3).toDF("col") + * val scala_udf = udf((input: Any) => input.toString) + * val casted_col = scala_udf(df.col("col").cast("string")) + * casted_col.cast(df.schema("col").dataType) + * }}} + */ + case class TestScalaUDF(name: String) extends TestUDF { + private[IntegratedUDFTestUtils] lazy val udf = new SparkUserDefinedFunction( + (input: Any) => if (input == null) { + null + } else { + input.toString + }, + StringType, + inputSchemas = Seq.fill(1)(None), + name = Some(name)) { + + override def apply(exprs: Column*): Column = { + assert(exprs.length == 1, "Defined UDF only has one column") + val expr = exprs.head.expr + assert(expr.resolved, "column should be resolved to use the same type " + + "as input. Try df(name) or df.col(name)") + Column(Cast(createScalaUDF(Cast(expr, StringType) :: Nil), expr.dataType)) + } + } + + def apply(exprs: Column*): Column = udf(exprs: _*) + + val prettyName: String = "Scala UDF" + } + + /** + * Register UDFs used in this test case. + */ + def registerTestUDF(testUDF: TestUDF, session: SparkSession): Unit = testUDF match { + case udf: TestPythonUDF => session.udf.registerPython(udf.name, udf.udf) + case udf: TestScalarPandasUDF => session.udf.registerPython(udf.name, udf.udf) + case udf: TestScalaUDF => session.udf.register(udf.name, udf.udf) + case other => throw new RuntimeException(s"Unknown UDF class [${other.getClass}]") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala new file mode 100644 index 0000000000000..6bda1afdfdd42 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinHintSuite.scala @@ -0,0 +1,583 @@ +/* + * 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 scala.collection.mutable.ArrayBuffer + +import org.apache.log4j.{AppenderSkeleton, Level} +import org.apache.log4j.spi.LoggingEvent + +import org.apache.spark.sql.catalyst.optimizer.EliminateResolvedHint +import org.apache.spark.sql.catalyst.plans.PlanTest +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class JoinHintSuite extends PlanTest with SharedSQLContext { + import testImplicits._ + + lazy val df = spark.range(10) + lazy val df1 = df.selectExpr("id as a1", "id as a2") + lazy val df2 = df.selectExpr("id as b1", "id as b2") + lazy val df3 = df.selectExpr("id as c1", "id as c2") + + class MockAppender extends AppenderSkeleton { + val loggingEvents = new ArrayBuffer[LoggingEvent]() + + override def append(loggingEvent: LoggingEvent): Unit = loggingEvents.append(loggingEvent) + override def close(): Unit = {} + override def requiresLayout(): Boolean = false + } + + def msgNoHintRelationFound(relation: String, hint: String): String = + s"Count not find relation '$relation' specified in hint '$hint'." + + def msgNoJoinForJoinHint(strategy: String): String = + s"A join hint (strategy=$strategy) is specified but it is not part of a join relation." + + def msgJoinHintOverridden(strategy: String): String = + s"Hint (strategy=$strategy) is overridden by another hint and will not take effect." + + def verifyJoinHintWithWarnings( + df: => DataFrame, + expectedHints: Seq[JoinHint], + warnings: Seq[String]): Unit = { + val logAppender = new MockAppender() + withLogAppender(logAppender) { + verifyJoinHint(df, expectedHints) + } + val warningMessages = logAppender.loggingEvents + .filter(_.getLevel == Level.WARN) + .map(_.getRenderedMessage) + .filter(_.contains("hint")) + assert(warningMessages.size == warnings.size) + warnings.foreach { w => + assert(warningMessages.contains(w)) + } + } + + def verifyJoinHint(df: DataFrame, expectedHints: Seq[JoinHint]): Unit = { + val optimized = df.queryExecution.optimizedPlan + val joinHints = optimized collect { + case Join(_, _, _, _, hint) => hint + case _: ResolvedHint => fail("ResolvedHint should not appear after optimize.") + } + assert(joinHints == expectedHints) + } + + test("single join") { + verifyJoinHint( + df.hint("broadcast").join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + verifyJoinHint( + df.join(df.hint("broadcast"), "id"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil + ) + } + + test("multiple joins") { + verifyJoinHint( + df1.join(df2.hint("broadcast").join(df3, 'b1 === 'c1).hint("broadcast"), 'a1 === 'c1), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + verifyJoinHint( + df1.hint("broadcast").join(df2, 'a1 === 'b1).hint("broadcast").join(df3, 'a1 === 'c1), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + } + + test("hint scope") { + withTempView("a", "b", "c") { + df1.createOrReplaceTempView("a") + df2.createOrReplaceTempView("b") + verifyJoinHint( + sql( + """ + |select /*+ broadcast(a, b)*/ * from ( + | select /*+ broadcast(b)*/ * from a join b on a.a1 = b.b1 + |) a join ( + | select /*+ broadcast(a)*/ * from a join b on a.a1 = b.b1 + |) b on a.a1 = b.b1 + """.stripMargin), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + } + } + + test("hints prevent join reorder") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true", SQLConf.JOIN_REORDER_ENABLED.key -> "true") { + withTempView("a", "b", "c") { + df1.createOrReplaceTempView("a") + df2.createOrReplaceTempView("b") + df3.createOrReplaceTempView("c") + verifyJoinHint( + sql("select /*+ broadcast(a, c)*/ * from a, b, c " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + verifyJoinHint( + sql("select /*+ broadcast(a, c)*/ * from a, c, b " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint.NONE :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil + ) + verifyJoinHint( + sql("select /*+ broadcast(b, c)*/ * from a, c, b " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil + ) + + verifyJoinHint( + df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") + .join(df3, 'b1 === 'c1 && 'a1 < 10), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: + JoinHint.NONE :: Nil + ) + + verifyJoinHint( + df1.join(df2, 'a1 === 'b1 && 'a1 > 5).hint("broadcast") + .join(df3, 'b1 === 'c1 && 'a1 < 10) + .join(df, 'b1 === 'id), + JoinHint.NONE :: + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: + JoinHint.NONE :: Nil + ) + } + } + } + + test("intersect/except") { + val dfSub = spark.range(2) + verifyJoinHint( + df.hint("broadcast").except(dfSub).join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: + JoinHint.NONE :: Nil + ) + verifyJoinHint( + df.join(df.hint("broadcast").intersect(dfSub), "id"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint.NONE :: Nil + ) + } + + test("hint merge") { + verifyJoinHintWithWarnings( + df.hint("broadcast").filter('id > 2).hint("broadcast").join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil, + Nil + ) + verifyJoinHintWithWarnings( + df.join(df.hint("broadcast").limit(2).hint("broadcast"), "id"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil, + Nil + ) + verifyJoinHintWithWarnings( + df.hint("merge").filter('id > 2).hint("shuffle_hash").join(df, "id").hint("broadcast"), + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_HASH))), + None) :: Nil, + msgJoinHintOverridden("merge") :: + msgNoJoinForJoinHint("broadcast") :: Nil + ) + verifyJoinHintWithWarnings( + df.join(df.hint("broadcast").limit(2).hint("merge"), "id") + .hint("shuffle_hash") + .hint("shuffle_replicate_nl") + .join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_REPLICATE_NL))), + None) :: + JoinHint( + None, + Some(HintInfo(strategy = Some(SHUFFLE_MERGE)))) :: Nil, + msgJoinHintOverridden("broadcast") :: + msgJoinHintOverridden("shuffle_hash") :: Nil + ) + } + + test("hint merge - SQL") { + withTempView("a", "b", "c") { + df1.createOrReplaceTempView("a") + df2.createOrReplaceTempView("b") + df3.createOrReplaceTempView("c") + verifyJoinHintWithWarnings( + sql("select /*+ shuffle_hash merge(a, c) broadcast(a, b)*/ * from a, b, c " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint( + None, + Some(HintInfo(strategy = Some(SHUFFLE_MERGE)))) :: + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_MERGE))), + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil, + msgNoJoinForJoinHint("shuffle_hash") :: + msgJoinHintOverridden("broadcast") :: Nil + ) + verifyJoinHintWithWarnings( + sql("select /*+ shuffle_hash(a, b) merge(b, d) broadcast(b)*/ * from a, b, c " + + "where a.a1 = b.b1 and b.b1 = c.c1"), + JoinHint.NONE :: + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_HASH))), + Some(HintInfo(strategy = Some(SHUFFLE_HASH)))) :: Nil, + msgNoHintRelationFound("d", "merge(b, d)") :: + msgJoinHintOverridden("broadcast") :: + msgJoinHintOverridden("merge") :: Nil + ) + verifyJoinHintWithWarnings( + sql( + """ + |select /*+ broadcast(a, c) merge(a, d)*/ * from a + |join ( + | select /*+ shuffle_hash(c) shuffle_replicate_nl(b, c)*/ * from b + | join c on b.b1 = c.c1 + |) as d + |on a.a2 = d.b2 + """.stripMargin), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + Some(HintInfo(strategy = Some(SHUFFLE_MERGE)))) :: + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_REPLICATE_NL))), + Some(HintInfo(strategy = Some(SHUFFLE_HASH)))) :: Nil, + msgNoHintRelationFound("c", "broadcast(a, c)") :: + msgJoinHintOverridden("merge") :: + msgJoinHintOverridden("shuffle_replicate_nl") :: Nil + ) + } + } + + test("nested hint") { + verifyJoinHint( + df.hint("broadcast").hint("broadcast").filter('id > 2).join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(BROADCAST))), + None) :: Nil + ) + verifyJoinHint( + df.hint("shuffle_hash").hint("broadcast").hint("merge").filter('id > 2).join(df, "id"), + JoinHint( + Some(HintInfo(strategy = Some(SHUFFLE_MERGE))), + None) :: Nil + ) + } + + test("hints prevent cost-based join reorder") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true", SQLConf.JOIN_REORDER_ENABLED.key -> "true") { + val join = df.join(df, "id") + val broadcasted = join.hint("broadcast") + verifyJoinHint( + join.join(broadcasted, "id").join(broadcasted, "id"), + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: + JoinHint.NONE :: JoinHint.NONE :: JoinHint.NONE :: Nil + ) + } + } + + def equiJoinQueryWithHint(hints: Seq[String], joinType: String = "INNER"): String = + hints.map("/*+ " + _ + " */").mkString( + "SELECT ", " ", s" * FROM t1 $joinType JOIN t2 ON t1.key = t2.key") + + def nonEquiJoinQueryWithHint(hints: Seq[String], joinType: String = "INNER"): String = + hints.map("/*+ " + _ + " */").mkString( + "SELECT ", " ", s" * FROM t1 $joinType JOIN t2 ON t1.key > t2.key") + + private def assertBroadcastHashJoin(df: DataFrame, buildSide: BuildSide): Unit = { + val executedPlan = df.queryExecution.executedPlan + val broadcastHashJoins = executedPlan.collect { + case b: BroadcastHashJoinExec => b + } + assert(broadcastHashJoins.size == 1) + assert(broadcastHashJoins.head.buildSide == buildSide) + } + + private def assertBroadcastNLJoin(df: DataFrame, buildSide: BuildSide): Unit = { + val executedPlan = df.queryExecution.executedPlan + val broadcastNLJoins = executedPlan.collect { + case b: BroadcastNestedLoopJoinExec => b + } + assert(broadcastNLJoins.size == 1) + assert(broadcastNLJoins.head.buildSide == buildSide) + } + + private def assertShuffleHashJoin(df: DataFrame, buildSide: BuildSide): Unit = { + val executedPlan = df.queryExecution.executedPlan + val shuffleHashJoins = executedPlan.collect { + case s: ShuffledHashJoinExec => s + } + assert(shuffleHashJoins.size == 1) + assert(shuffleHashJoins.head.buildSide == buildSide) + } + + private def assertShuffleMergeJoin(df: DataFrame): Unit = { + val executedPlan = df.queryExecution.executedPlan + val shuffleMergeJoins = executedPlan.collect { + case s: SortMergeJoinExec => s + } + assert(shuffleMergeJoins.size == 1) + } + + private def assertShuffleReplicateNLJoin(df: DataFrame): Unit = { + val executedPlan = df.queryExecution.executedPlan + val shuffleReplicateNLJoins = executedPlan.collect { + case c: CartesianProductExec => c + } + assert(shuffleReplicateNLJoins.size == 1) + } + + test("join strategy hint - broadcast") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + + val t1Size = spark.table("t1").queryExecution.analyzed.children.head.stats.sizeInBytes + val t2Size = spark.table("t2").queryExecution.analyzed.children.head.stats.sizeInBytes + assert(t1Size < t2Size) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + // Broadcast hint specified on one side + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: Nil)), BuildLeft) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("BROADCAST(t2)" :: Nil)), BuildRight) + + // Determine build side based on the join type and child relation sizes + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1, t2)" :: Nil)), BuildLeft) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("BROADCAST(t1, t2)" :: Nil, "left")), BuildRight) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("BROADCAST(t1, t2)" :: Nil, "right")), BuildLeft) + + // Use broadcast-hash join if hinted "broadcast" and equi-join + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t2)" :: "SHUFFLE_HASH(t1)" :: Nil)), BuildRight) + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: "MERGE(t1, t2)" :: Nil)), BuildLeft) + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: "SHUFFLE_REPLICATE_NL(t2)" :: Nil)), + BuildLeft) + + // Use broadcast-nl join if hinted "broadcast" and non-equi-join + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("SHUFFLE_HASH(t2)" :: "BROADCAST(t1)" :: Nil)), BuildLeft) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("MERGE(t1)" :: "BROADCAST(t2)" :: Nil)), BuildRight) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t1)" :: "BROADCAST(t2)" :: Nil)), + BuildRight) + + // Broadcast hint specified but not doable + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: Nil, "left"))) + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("BROADCAST(t2)" :: Nil, "right"))) + } + } + } + + test("join strategy hint - shuffle-merge") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Int.MaxValue.toString) { + // Shuffle-merge hint specified on one side + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("SHUFFLE_MERGE(t1)" :: Nil))) + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("MERGEJOIN(t2)" :: Nil))) + + // Shuffle-merge hint specified on both sides + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("MERGE(t1, t2)" :: Nil))) + + // Shuffle-merge hint prioritized over shuffle-hash hint and shuffle-replicate-nl hint + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t2)" :: "MERGE(t1)" :: Nil, "left"))) + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("MERGE(t2)" :: "SHUFFLE_HASH(t1)" :: Nil, "right"))) + + // Broadcast hint prioritized over shuffle-merge hint, but broadcast hint is not applicable + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: "MERGE(t2)" :: Nil, "left"))) + assertShuffleMergeJoin( + sql(equiJoinQueryWithHint("BROADCAST(t2)" :: "MERGE(t1)" :: Nil, "right"))) + + // Shuffle-merge hint specified but not doable + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("MERGE(t1, t2)" :: Nil, "left")), BuildRight) + } + } + } + + test("join strategy hint - shuffle-hash") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + + val t1Size = spark.table("t1").queryExecution.analyzed.children.head.stats.sizeInBytes + val t2Size = spark.table("t2").queryExecution.analyzed.children.head.stats.sizeInBytes + assert(t1Size < t2Size) + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Int.MaxValue.toString) { + // Shuffle-hash hint specified on one side + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t1)" :: Nil)), BuildLeft) + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t2)" :: Nil)), BuildRight) + + // Determine build side based on the join type and child relation sizes + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t1, t2)" :: Nil)), BuildLeft) + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t1, t2)" :: Nil, "left")), BuildRight) + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t1, t2)" :: Nil, "right")), BuildLeft) + + // Shuffle-hash hint prioritized over shuffle-replicate-nl hint + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t2)" :: "SHUFFLE_HASH(t1)" :: Nil)), + BuildLeft) + + // Broadcast hint prioritized over shuffle-hash hint, but broadcast hint is not applicable + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t1)" :: "SHUFFLE_HASH(t2)" :: Nil, "left")), + BuildRight) + assertShuffleHashJoin( + sql(equiJoinQueryWithHint("BROADCAST(t2)" :: "SHUFFLE_HASH(t1)" :: Nil, "right")), + BuildLeft) + + // Shuffle-hash hint specified but not doable + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_HASH(t1)" :: Nil, "left")), BuildRight) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("SHUFFLE_HASH(t1)" :: Nil)), BuildLeft) + } + } + } + + test("join strategy hint - shuffle-replicate-nl") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Int.MaxValue.toString) { + // Shuffle-replicate-nl hint specified on one side + assertShuffleReplicateNLJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t1)" :: Nil))) + assertShuffleReplicateNLJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t2)" :: Nil))) + + // Shuffle-replicate-nl hint specified on both sides + assertShuffleReplicateNLJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t1, t2)" :: Nil))) + + // Shuffle-merge hint prioritized over shuffle-replicate-nl hint, but shuffle-merge hint + // is not applicable + assertShuffleReplicateNLJoin( + sql(nonEquiJoinQueryWithHint("MERGE(t1)" :: "SHUFFLE_REPLICATE_NL(t2)" :: Nil))) + + // Shuffle-hash hint prioritized over shuffle-replicate-nl hint, but shuffle-hash hint is + // not applicable + assertShuffleReplicateNLJoin( + sql(nonEquiJoinQueryWithHint("SHUFFLE_HASH(t2)" :: "SHUFFLE_REPLICATE_NL(t1)" :: Nil))) + + // Shuffle-replicate-nl hint specified but not doable + assertBroadcastHashJoin( + sql(equiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t1, t2)" :: Nil, "left")), BuildRight) + assertBroadcastNLJoin( + sql(nonEquiJoinQueryWithHint("SHUFFLE_REPLICATE_NL(t1, t2)" :: Nil, "right")), BuildLeft) + } + } + } + + test("Verify that the EliminatedResolvedHint rule is idempotent") { + withTempView("t1", "t2") { + Seq((1, "4"), (2, "2")).toDF("key", "value").createTempView("t1") + Seq((1, "1"), (2, "12.3"), (2, "123")).toDF("key", "value").createTempView("t2") + val df = sql("SELECT /*+ broadcast(t2) */ * from t1 join t2 ON t1.key = t2.key") + val optimize = new RuleExecutor[LogicalPlan] { + val batches = Batch("EliminateResolvedHint", FixedPoint(10), EliminateResolvedHint) :: Nil + } + val optimized = optimize.execute(df.logicalPlan) + val expectedHints = + JoinHint( + None, + Some(HintInfo(strategy = Some(BROADCAST)))) :: Nil + val joinHints = optimized collect { + case Join(_, _, _, _, hint) => hint + case _: ResolvedHint => fail("ResolvedHint should not appear after optimize.") + } + assert(joinHints == expectedHints) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala index aa2162c9d2cda..059dbf892c653 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/JoinSuite.scala @@ -21,14 +21,15 @@ import java.util.Locale import scala.collection.JavaConverters._ import scala.collection.mutable.ListBuffer -import scala.language.existentials import org.apache.spark.TestUtils.{assertNotSpilled, assertSpilled} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.{Ascending, SortOrder} -import org.apache.spark.sql.execution.{BinaryExecNode, SortExec} +import org.apache.spark.sql.catalyst.plans.logical.Filter +import org.apache.spark.sql.execution.{BinaryExecNode, FilterExec, SortExec} import org.apache.spark.sql.execution.joins._ +import org.apache.spark.sql.execution.python.BatchEvalPythonExec import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -50,8 +51,9 @@ class JoinSuite extends QueryTest with SharedSQLContext { assert(planned.size === 1) } - def assertJoin(pair: (String, Class[_])): Any = { - val (sqlString, c) = pair + def assertJoin(pair: (String, Class[_ <: BinaryExecNode])): Any = { + val sqlString = pair._1 + val c = pair._2 val df = sql(sqlString) val physical = df.queryExecution.sparkPlan val operators = physical.collect { @@ -71,7 +73,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("join operator selection") { spark.sharedState.cacheManager.clearCache() - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "0", + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0", SQLConf.CROSS_JOINS_ENABLED.key -> "true") { Seq( ("SELECT * FROM testData LEFT SEMI JOIN testData2 ON key = a", @@ -650,7 +652,7 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (without spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> Int.MaxValue.toString) { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> Int.MaxValue.toString) { assertNotSpilled(sparkContext, "inner join") { checkAnswer( @@ -707,8 +709,8 @@ class JoinSuite extends QueryTest with SharedSQLContext { test("test SortMergeJoin (with spill)") { withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "1", - "spark.sql.sortMergeJoinExec.buffer.in.memory.threshold" -> "0", - "spark.sql.sortMergeJoinExec.buffer.spill.threshold" -> "1") { + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "0", + SQLConf.SORT_MERGE_JOIN_EXEC_BUFFER_SPILL_THRESHOLD.key -> "1") { assertSpilled(sparkContext, "inner join") { checkAnswer( @@ -895,4 +897,144 @@ class JoinSuite extends QueryTest with SharedSQLContext { checkAnswer(res, Row(0, 0, 0)) } } + + test("SPARK-27485: EnsureRequirements should not fail join with duplicate keys") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + val tbl_a = spark.range(40) + .select($"id" as "x", $"id" % 10 as "y") + .repartition(2, $"x", $"y", $"x") + .as("tbl_a") + + val tbl_b = spark.range(20) + .select($"id" as "x", $"id" % 2 as "y1", $"id" % 20 as "y2") + .as("tbl_b") + + val res = tbl_a + .join(tbl_b, + $"tbl_a.x" === $"tbl_b.x" && $"tbl_a.y" === $"tbl_b.y1" && $"tbl_a.y" === $"tbl_b.y2") + .select($"tbl_a.x") + checkAnswer(res, Row(0L) :: Row(1L) :: Nil) + } + } + + test("SPARK-26352: join reordering should not change the order of columns") { + withTable("tab1", "tab2", "tab3") { + spark.sql("select 1 as x, 100 as y").write.saveAsTable("tab1") + spark.sql("select 42 as i, 200 as j").write.saveAsTable("tab2") + spark.sql("select 1 as a, 42 as b").write.saveAsTable("tab3") + + val df = spark.sql(""" + with tmp as (select * from tab1 cross join tab2) + select * from tmp join tab3 on a = x and b = i + """) + checkAnswer(df, Row(1, 100, 42, 200, 1, 42)) + } + } + + test("NaN and -0.0 in join keys") { + withTempView("v1", "v2", "v3", "v4") { + Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d").createTempView("v1") + Seq(Float.NaN -> Double.NaN, 0.0f -> 0.0, -0.0f -> -0.0).toDF("f", "d").createTempView("v2") + + checkAnswer( + sql( + """ + |SELECT v1.f, v1.d, v2.f, v2.d + |FROM v1 JOIN v2 + |ON v1.f = v2.f AND v1.d = v2.d + """.stripMargin), + Seq( + Row(Float.NaN, Double.NaN, Float.NaN, Double.NaN), + Row(0.0f, 0.0, 0.0f, 0.0), + Row(0.0f, 0.0, -0.0f, -0.0), + Row(-0.0f, -0.0, 0.0f, 0.0), + Row(-0.0f, -0.0, -0.0f, -0.0))) + + // test with complicated join keys. + checkAnswer( + sql( + """ + |SELECT v1.f, v1.d, v2.f, v2.d + |FROM v1 JOIN v2 + |ON + | array(v1.f) = array(v2.f) AND + | struct(v1.d) = struct(v2.d) AND + | array(struct(v1.f, v1.d)) = array(struct(v2.f, v2.d)) AND + | struct(array(v1.f), array(v1.d)) = struct(array(v2.f), array(v2.d)) + """.stripMargin), + Seq( + Row(Float.NaN, Double.NaN, Float.NaN, Double.NaN), + Row(0.0f, 0.0, 0.0f, 0.0), + Row(0.0f, 0.0, -0.0f, -0.0), + Row(-0.0f, -0.0, 0.0f, 0.0), + Row(-0.0f, -0.0, -0.0f, -0.0))) + + // test with tables with complicated-type columns. + Seq((Array(-0.0f, 0.0f), Tuple2(-0.0d, Double.NaN), Seq(Tuple2(-0.0d, Double.NaN)))) + .toDF("arr", "stru", "arrOfStru").createTempView("v3") + Seq((Array(0.0f, -0.0f), Tuple2(0.0d, 0.0/0.0), Seq(Tuple2(0.0d, 0.0/0.0)))) + .toDF("arr", "stru", "arrOfStru").createTempView("v4") + checkAnswer( + sql( + """ + |SELECT v3.arr, v3.stru, v3.arrOfStru, v4.arr, v4.stru, v4.arrOfStru + |FROM v3 JOIN v4 + |ON v3.arr = v4.arr AND v3.stru = v4.stru AND v3.arrOfStru = v4.arrOfStru + """.stripMargin), + Seq(Row( + Seq(-0.0f, 0.0f), + Row(-0.0d, Double.NaN), + Seq(Row(-0.0d, Double.NaN)), + Seq(0.0f, -0.0f), + Row(0.0d, 0.0/0.0), + Seq(Row(0.0d, 0.0/0.0))))) + } + } + + test("SPARK-28323: PythonUDF should be able to use in join condition") { + import IntegratedUDFTestUtils._ + + assume(shouldTestPythonUDFs) + + val pythonTestUDF = TestPythonUDF(name = "udf") + + val left = Seq((1, 2), (2, 3)).toDF("a", "b") + val right = Seq((1, 2), (3, 4)).toDF("c", "d") + val df = left.join(right, pythonTestUDF(left("a")) === pythonTestUDF(right.col("c"))) + + val joinNode = df.queryExecution.executedPlan.find(_.isInstanceOf[BroadcastHashJoinExec]) + assert(joinNode.isDefined) + + // There are two PythonUDFs which use attribute from left and right of join, individually. + // So two PythonUDFs should be evaluated before the join operator, at left and right side. + val pythonEvals = joinNode.get.collect { + case p: BatchEvalPythonExec => p + } + assert(pythonEvals.size == 2) + + checkAnswer(df, Row(1, 2, 1, 2) :: Nil) + } + + test("SPARK-28345: PythonUDF predicate should be able to pushdown to join") { + import IntegratedUDFTestUtils._ + + assume(shouldTestPythonUDFs) + + val pythonTestUDF = TestPythonUDF(name = "udf") + + val left = Seq((1, 2), (2, 3)).toDF("a", "b") + val right = Seq((1, 2), (3, 4)).toDF("c", "d") + val df = left.crossJoin(right).where(pythonTestUDF(left("a")) === right.col("c")) + + // Before optimization, there is a logical Filter operator. + val filterInAnalysis = df.queryExecution.analyzed.find(_.isInstanceOf[Filter]) + assert(filterInAnalysis.isDefined) + + // Filter predicate was pushdown as join condition. So there is no Filter exec operator. + val filterExec = df.queryExecution.executedPlan.find(_.isInstanceOf[FilterExec]) + assert(filterExec.isEmpty) + + checkAnswer(df, Row(1, 2, 1, 2) :: Nil) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/JsonFunctionsSuite.scala index 5cbf10129a4da..24e7564259c83 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 @@ -17,9 +17,14 @@ package org.apache.spark.sql +import java.text.SimpleDateFormat +import java.util.Locale + import collection.JavaConverters._ +import org.apache.spark.SparkException import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -132,7 +137,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { checkAnswer( df.select(from_json($"value", schema)), - Row(null) :: Nil) + Row(Row(null)) :: Nil) } test("from_json - json doesn't conform to the array type") { @@ -393,7 +398,7 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { test("SPARK-24709: infers schemas of json strings and pass them to from_json") { val in = Seq("""{"a": [1, 2, 3]}""").toDS() - val out = in.select(from_json('value, schema_of_json(lit("""{"a": [1]}"""))) as "parsed") + val out = in.select(from_json('value, schema_of_json("""{"a": [1]}""")) as "parsed") val expected = StructType(StructField( "parsed", StructType(StructField( @@ -547,4 +552,60 @@ class JsonFunctionsSuite extends QueryTest with SharedSQLContext { Map("pretty" -> "true"))), Seq(Row(expected))) } + + test("from_json invalid json - check modes") { + withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { + val schema = new StructType() + .add("a", IntegerType) + .add("b", IntegerType) + .add("_unparsed", StringType) + val badRec = """{"a" 1, "b": 11}""" + val df = Seq(badRec, """{"a": 2, "b": 12}""").toDS() + + checkAnswer( + df.select(from_json($"value", schema, Map("mode" -> "PERMISSIVE"))), + Row(Row(null, null, badRec)) :: Row(Row(2, 12, null)) :: Nil) + + val exception1 = intercept[SparkException] { + df.select(from_json($"value", schema, Map("mode" -> "FAILFAST"))).collect() + }.getMessage + assert(exception1.contains( + "Malformed records are detected in record parsing. Parse Mode: FAILFAST.")) + + val exception2 = intercept[SparkException] { + df.select(from_json($"value", schema, Map("mode" -> "DROPMALFORMED"))) + .collect() + }.getMessage + assert(exception2.contains( + "from_json() doesn't support the DROPMALFORMED mode. " + + "Acceptable modes are PERMISSIVE and FAILFAST.")) + } + } + + test("corrupt record column in the middle") { + val schema = new StructType() + .add("a", IntegerType) + .add("_unparsed", StringType) + .add("b", IntegerType) + val badRec = """{"a" 1, "b": 11}""" + val df = Seq(badRec, """{"a": 2, "b": 12}""").toDS() + + checkAnswer( + df.select(from_json($"value", schema, Map("columnNameOfCorruptRecord" -> "_unparsed"))), + Row(Row(null, badRec, null)) :: Row(Row(2, null, 12)) :: Nil) + } + + test("parse timestamps with locale") { + Seq("en-US", "ko-KR", "zh-CN", "ru-RU").foreach { langTag => + val locale = Locale.forLanguageTag(langTag) + val ts = new SimpleDateFormat("dd/MM/yyyy HH:mm").parse("06/11/2018 18:00") + val timestampFormat = "dd MMM yyyy HH:mm" + val sdf = new SimpleDateFormat(timestampFormat, locale) + val input = Seq(s"""{"time": "${sdf.format(ts)}"}""").toDS() + val options = Map("timestampFormat" -> timestampFormat, "locale" -> langTag) + val df = input.select(from_json($"value", "time timestamp", options)) + + checkAnswer(df, Row(Row(java.sql.Timestamp.valueOf("2018-11-06 18:00:00.0")))) + } + } } 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 98aa447fc0560..602951bc48a1e 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 @@ -19,17 +19,17 @@ package org.apache.spark.sql import java.io.File -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext /** * Test suite to handle metadata cache related. */ -class MetadataCacheSuite extends QueryTest with SharedSQLContext { +abstract class MetadataCacheSuite extends QueryTest with SharedSQLContext { /** Removes one data file in the given directory. */ - private def deleteOneFileInDirectory(dir: File): Unit = { + protected def deleteOneFileInDirectory(dir: File): Unit = { assert(dir.isDirectory) val oneFile = dir.listFiles().find { file => !file.getName.startsWith("_") && !file.getName.startsWith(".") @@ -38,14 +38,14 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { oneFile.foreach(_.delete()) } - test("SPARK-16336 Suggest doing table refresh when encountering FileNotFoundException") { + test("SPARK-16336,SPARK-27961 Suggest fixing FileNotFoundException") { withTempPath { (location: File) => - // Create a Parquet directory + // Create an ORC directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) - .write.parquet(location.getAbsolutePath) + .write.orc(location.getAbsolutePath) // Read the directory in - val df = spark.read.parquet(location.getAbsolutePath) + val df = spark.read.orc(location.getAbsolutePath) assert(df.count() == 100) // Delete a file @@ -56,18 +56,25 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { df.count() } assert(e.getMessage.contains("FileNotFoundException")) - assert(e.getMessage.contains("REFRESH")) + assert(e.getMessage.contains("recreating the Dataset/DataFrame involved")) } } +} + +class MetadataCacheV1Suite extends MetadataCacheSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") test("SPARK-16337 temporary view refresh") { withTempView("view_refresh") { withTempPath { (location: File) => - // Create a Parquet directory + // Create an ORC directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) - .write.parquet(location.getAbsolutePath) + .write.orc(location.getAbsolutePath) // Read the directory in - spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + spark.read.orc(location.getAbsolutePath).createOrReplaceTempView("view_refresh") assert(sql("select count(*) from view_refresh").first().getLong(0) == 100) // Delete a file @@ -93,10 +100,10 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { withTempPath { (location: File) => // Create a Parquet directory spark.range(start = 0, end = 100, step = 1, numPartitions = 3) - .write.parquet(location.getAbsolutePath) + .write.orc(location.getAbsolutePath) // Read the directory in - spark.read.parquet(location.getAbsolutePath).createOrReplaceTempView("view_refresh") + spark.read.orc(location.getAbsolutePath).createOrReplaceTempView("view_refresh") // Delete a file deleteOneFileInDirectory(location) @@ -111,3 +118,10 @@ class MetadataCacheSuite extends QueryTest with SharedSQLContext { } } } + +class MetadataCacheV2Suite extends MetadataCacheSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala index 623a1b6f854cf..e33870d4e1afc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ProcessingTimeSuite.scala @@ -22,12 +22,15 @@ import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.{ProcessingTime, Trigger} +import org.apache.spark.sql.execution.streaming.ProcessingTimeTrigger +import org.apache.spark.sql.streaming.Trigger class ProcessingTimeSuite extends SparkFunSuite { test("create") { - def getIntervalMs(trigger: Trigger): Long = trigger.asInstanceOf[ProcessingTime].intervalMs + def getIntervalMs(trigger: Trigger): Long = { + trigger.asInstanceOf[ProcessingTimeTrigger].intervalMs + } assert(getIntervalMs(Trigger.ProcessingTime(10.seconds)) === 10 * 1000) assert(getIntervalMs(Trigger.ProcessingTime(10, TimeUnit.SECONDS)) === 10 * 1000) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index baca9c1cfb9a0..d034e10e0d077 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -23,7 +23,9 @@ import scala.collection.JavaConverters._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.columnar.InMemoryRelation +import org.apache.spark.storage.StorageLevel abstract class QueryTest extends PlanTest { @@ -64,7 +66,7 @@ abstract class QueryTest extends PlanTest { expectedAnswer: T*): Unit = { val result = getResult(ds) - if (!compare(result.toSeq, expectedAnswer)) { + if (!QueryTest.compare(result.toSeq, expectedAnswer)) { fail( s""" |Decoded objects do not match expected objects: @@ -84,7 +86,7 @@ abstract class QueryTest extends PlanTest { expectedAnswer: T*): Unit = { val result = getResult(ds) - if (!compare(result.toSeq.sorted, expectedAnswer.sorted)) { + if (!QueryTest.compare(result.toSeq.sorted, expectedAnswer.sorted)) { fail( s""" |Decoded objects do not match expected objects: @@ -124,17 +126,6 @@ abstract class QueryTest extends PlanTest { } } - private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { - case (null, null) => true - case (null, _) => false - case (_, null) => false - case (a: Array[_], b: Array[_]) => - a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} - case (a: Iterable[_], b: Iterable[_]) => - a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} - case (a, b) => a == b - } - /** * Runs the plan and makes sure the answer matches the expected result. * @@ -215,6 +206,22 @@ abstract class QueryTest extends PlanTest { planWithCaching) } + /** + * Asserts that a given [[Dataset]] will be executed using the cache with the given name and + * storage level. + */ + def assertCached(query: Dataset[_], cachedName: String, storageLevel: StorageLevel): Unit = { + val planWithCaching = query.queryExecution.withCachedData + val matched = planWithCaching.collectFirst { case cached: InMemoryRelation => + val cacheBuilder = cached.asInstanceOf[InMemoryRelation].cacheBuilder + cachedName == cacheBuilder.tableName.get && + (storageLevel == cacheBuilder.storageLevel) + }.getOrElse(false) + + assert(matched, s"Expected query plan to hit cache $cachedName with storage " + + s"level $storageLevel, but it doesn't.") + } + /** * Asserts that a given [[Dataset]] does not have missing inputs in all the analyzed plans. */ @@ -245,7 +252,9 @@ object QueryTest { checkToRDD: Boolean = true): Option[String] = { val isSorted = df.logicalPlan.collect { case s: logical.Sort => s }.nonEmpty if (checkToRDD) { - df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + SQLExecution.withSQLConfPropagated(df.sparkSession) { + df.rdd.count() // Also attempt to deserialize as an RDD [SPARK-15791] + } } val sparkAnswer = try df.collect().toSeq catch { @@ -289,7 +298,7 @@ object QueryTest { def prepareRow(row: Row): Row = { Row.fromSeq(row.toSeq.map { case null => null - case d: java.math.BigDecimal => BigDecimal(d) + case bd: java.math.BigDecimal => BigDecimal(bd) // Equality of WrappedArray differs for AnyVal and AnyRef in Scala 2.12.2+ case seq: Seq[_] => seq.map { case b: java.lang.Byte => b.byteValue @@ -342,11 +351,35 @@ object QueryTest { None } + private def compare(obj1: Any, obj2: Any): Boolean = (obj1, obj2) match { + case (null, null) => true + case (null, _) => false + case (_, null) => false + case (a: Array[_], b: Array[_]) => + a.length == b.length && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Map[_, _], b: Map[_, _]) => + a.size == b.size && a.keys.forall { aKey => + b.keys.find(bKey => compare(aKey, bKey)).exists(bKey => compare(a(aKey), b(bKey))) + } + case (a: Iterable[_], b: Iterable[_]) => + a.size == b.size && a.zip(b).forall { case (l, r) => compare(l, r)} + case (a: Product, b: Product) => + compare(a.productIterator.toSeq, b.productIterator.toSeq) + case (a: Row, b: Row) => + compare(a.toSeq, b.toSeq) + // 0.0 == -0.0, turn float/double to bits before comparison, to distinguish 0.0 and -0.0. + case (a: Double, b: Double) => + java.lang.Double.doubleToRawLongBits(a) == java.lang.Double.doubleToRawLongBits(b) + case (a: Float, b: Float) => + java.lang.Float.floatToRawIntBits(a) == java.lang.Float.floatToRawIntBits(b) + case (a, b) => a == b + } + def sameRows( expectedAnswer: Seq[Row], sparkAnswer: Seq[Row], isSorted: Boolean = false): Option[String] = { - if (prepareAnswer(expectedAnswer, isSorted) != prepareAnswer(sparkAnswer, isSorted)) { + if (!compare(prepareAnswer(expectedAnswer, isSorted), prepareAnswer(sparkAnswer, isSorted))) { return Some(genError(expectedAnswer, sparkAnswer, isSorted)) } None diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala new file mode 100644 index 0000000000000..1729c3c0c557a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ReplaceNullWithFalseInPredicateEndToEndSuite.scala @@ -0,0 +1,115 @@ +/* + * 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.catalyst.expressions.{CaseWhen, If, Literal} +import org.apache.spark.sql.execution.LocalTableScanExec +import org.apache.spark.sql.functions.{lit, when} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.BooleanType + +class ReplaceNullWithFalseInPredicateEndToEndSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + test("SPARK-25860: Replace Literal(null, _) with FalseLiteral whenever possible") { + withTable("t1", "t2") { + Seq((1, true), (2, false)).toDF("l", "b").write.saveAsTable("t1") + Seq(2, 3).toDF("l").write.saveAsTable("t2") + val df1 = spark.table("t1") + val df2 = spark.table("t2") + + val q1 = df1.where("IF(l > 10, false, b AND null)") + checkAnswer(q1, Seq.empty) + checkPlanIsEmptyLocalScan(q1) + + val q2 = df1.where("CASE WHEN l < 10 THEN null WHEN l > 40 THEN false ELSE null END") + checkAnswer(q2, Seq.empty) + checkPlanIsEmptyLocalScan(q2) + + val q3 = df1.join(df2, when(df1("l") > df2("l"), lit(null)).otherwise(df1("b") && lit(null))) + checkAnswer(q3, Seq.empty) + checkPlanIsEmptyLocalScan(q3) + + val q4 = df1.where("IF(IF(b, null, false), true, null)") + checkAnswer(q4, Seq.empty) + checkPlanIsEmptyLocalScan(q4) + + val q5 = df1.selectExpr("IF(l > 1 AND null, 5, 1) AS out") + checkAnswer(q5, Row(1) :: Row(1) :: Nil) + q5.queryExecution.executedPlan.foreach { p => + assert(p.expressions.forall(e => e.find(_.isInstanceOf[If]).isEmpty)) + } + + val q6 = df1.selectExpr("CASE WHEN (l > 2 AND null) THEN 3 ELSE 2 END") + checkAnswer(q6, Row(2) :: Row(2) :: Nil) + q6.queryExecution.executedPlan.foreach { p => + assert(p.expressions.forall(e => e.find(_.isInstanceOf[CaseWhen]).isEmpty)) + } + + checkAnswer(df1.where("IF(l > 10, false, b OR null)"), Row(1, true)) + } + + def checkPlanIsEmptyLocalScan(df: DataFrame): Unit = df.queryExecution.executedPlan match { + case s: LocalTableScanExec => assert(s.rows.isEmpty) + case p => fail(s"$p is not LocalTableScanExec") + } + } + + test("SPARK-26107: Replace Literal(null, _) with FalseLiteral in higher-order functions") { + def assertNoLiteralNullInPlan(df: DataFrame): Unit = { + df.queryExecution.executedPlan.foreach { p => + assert(p.expressions.forall(_.find { + case Literal(null, BooleanType) => true + case _ => false + }.isEmpty)) + } + } + + withTable("t1", "t2") { + // to test ArrayFilter and ArrayExists + spark.sql("select array(null, 1, null, 3) as a") + .write.saveAsTable("t1") + // to test MapFilter + spark.sql(""" + select map_from_entries(arrays_zip(a, transform(a, e -> if(mod(e, 2) = 0, null, e)))) as m + from (select array(0, 1, 2, 3) as a) + """).write.saveAsTable("t2") + + val df1 = spark.table("t1") + val df2 = spark.table("t2") + + // ArrayExists + withSQLConf(SQLConf.LEGACY_ARRAY_EXISTS_FOLLOWS_THREE_VALUED_LOGIC.key -> "false") { + val q1 = df1.selectExpr("EXISTS(a, e -> IF(e is null, null, true))") + checkAnswer(q1, Row(true) :: Nil) + assertNoLiteralNullInPlan(q1) + } + + // ArrayFilter + val q2 = df1.selectExpr("FILTER(a, e -> IF(e is null, null, true))") + checkAnswer(q2, Row(Seq[Any](1, 3)) :: Nil) + assertNoLiteralNullInPlan(q2) + + // MapFilter + val q3 = df2.selectExpr("MAP_FILTER(m, (k, v) -> IF(v is null, null, true))") + checkAnswer(q3, Row(Map[Any, Any](1 -> 1, 3 -> 3))) + assertNoLiteralNullInPlan(q3) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala index 57b5f5e4ab99f..c53fd5bd7a158 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RowSuite.scala @@ -84,4 +84,25 @@ class RowSuite extends SparkFunSuite with SharedSQLContext { val r3 = Row("World") assert(r3.hashCode() != r1.hashCode()) } + + test("toString") { + val r1 = Row(2147483647, 21474.8364, (-5).toShort, "this is a string", true, null) + assert(r1.toString == "[2147483647,21474.8364,-5,this is a string,true,null]") + val r2 = Row(null, Int.MinValue, Double.NaN, Short.MaxValue, "", false) + assert(r2.toString == "[null,-2147483648,NaN,32767,,false]") + val tsString = "2019-05-01 17:30:12.0" + val dtString = "2019-05-01" + val r3 = Row( + r1, + Seq(1, 2, 3), + Map(1 -> "a", 2 -> "b"), + java.sql.Timestamp.valueOf(tsString), + java.sql.Date.valueOf(dtString), + BigDecimal("1234567890.1234567890"), + (-1).toByte) + assert(r3.toString == "[[2147483647,21474.8364,-5,this is a string,true,null],List(1, 2, 3)," + + s"Map(1 -> a, 2 -> b),$tsString,$dtString,1234567890.1234567890,-1]") + val empty = Row() + assert(empty.toString == "[]") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala index cdcea09ad9758..720d570ca8384 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/RuntimeConfigSuite.scala @@ -18,6 +18,9 @@ package org.apache.spark.sql import org.apache.spark.SparkFunSuite +import org.apache.spark.internal.config +import org.apache.spark.sql.internal.SQLConf.CHECKPOINT_LOCATION +import org.apache.spark.sql.internal.StaticSQLConf.SCHEMA_STRING_LENGTH_THRESHOLD class RuntimeConfigSuite extends SparkFunSuite { @@ -59,13 +62,22 @@ class RuntimeConfigSuite extends SparkFunSuite { val conf = newConf() // SQL configs - assert(!conf.isModifiable("spark.sql.sources.schemaStringLengthThreshold")) - assert(conf.isModifiable("spark.sql.streaming.checkpointLocation")) + assert(!conf.isModifiable(SCHEMA_STRING_LENGTH_THRESHOLD.key)) + assert(conf.isModifiable(CHECKPOINT_LOCATION.key)) // Core configs - assert(!conf.isModifiable("spark.task.cpus")) + assert(!conf.isModifiable(config.CPUS_PER_TASK.key)) assert(!conf.isModifiable("spark.executor.cores")) // Invalid config parameters assert(!conf.isModifiable("")) assert(!conf.isModifiable("invalid config parameter")) } + + test("reject SparkConf entries") { + val conf = newConf() + + val ex = intercept[AnalysisException] { + conf.set(config.CPUS_PER_TASK.key, 4) + } + assert(ex.getMessage.contains("Spark config")) + } } 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 a1799829932b8..aab2ae4afc7f5 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,32 +24,14 @@ 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 = SQLContext.getOrCreate(sc) + val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext val session1 = sqlContext.newSession() val session2 = sqlContext.newSession() @@ -77,13 +59,13 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("Catalyst optimization passes are modifiable at runtime") { - val sqlContext = SQLContext.getOrCreate(sc) + val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext sqlContext.experimental.extraOptimizations = Seq(DummyRule) assert(sqlContext.sessionState.optimizer.batches.flatMap(_.rules).contains(DummyRule)) } test("get all tables") { - val sqlContext = SQLContext.getOrCreate(sc) + val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") assert( @@ -100,7 +82,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("getting all tables with a database name has no impact on returned table names") { - val sqlContext = SQLContext.getOrCreate(sc) + val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") assert( @@ -117,7 +99,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { } test("query the returned DataFrame of tables") { - val sqlContext = SQLContext.getOrCreate(sc) + val sqlContext = SparkSession.builder().sparkContext(sc).getOrCreate().sqlContext val df = sqlContext.range(10) df.createOrReplaceTempView("listtablessuitetable") @@ -127,7 +109,7 @@ class SQLContextSuite extends SparkFunSuite with SharedSparkContext { StructField("isTemporary", BooleanType, false) :: Nil) Seq(sqlContext.tables(), sqlContext.sql("SHOW TABLes")).foreach { - case tableDF => + 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 631ab1b7ece7f..972950669198c 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 @@ -19,15 +19,17 @@ package org.apache.spark.sql import java.io.File import java.net.{MalformedURLException, URL} -import java.sql.Timestamp +import java.sql.{Date, Timestamp} import java.util.concurrent.atomic.AtomicBoolean import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql.catalyst.util.StringUtils -import org.apache.spark.sql.execution.aggregate import org.apache.spark.sql.execution.aggregate.{HashAggregateExec, SortAggregateExec} -import org.apache.spark.sql.execution.datasources.FilePartition +import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, CartesianProductExec, SortMergeJoinExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -261,7 +263,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { val df = sql(sqlText) // First, check if we have GeneratedAggregate. val hasGeneratedAgg = df.queryExecution.sparkPlan - .collect { case _: aggregate.HashAggregateExec => true } + .collect { case _: HashAggregateExec => true } .nonEmpty if (!hasGeneratedAgg) { fail( @@ -413,32 +415,32 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.0'"), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00"))) + Row(Timestamp.valueOf("1969-12-31 16:00:00"))) checkAnswer(sql( "SELECT time FROM timestamps WHERE time=CAST('1969-12-31 16:00:00.001' AS TIMESTAMP)"), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) + Row(Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( "SELECT time FROM timestamps WHERE time='1969-12-31 16:00:00.001'"), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) + Row(Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( "SELECT time FROM timestamps WHERE '1969-12-31 16:00:00.001'=time"), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001"))) + Row(Timestamp.valueOf("1969-12-31 16:00:00.001"))) checkAnswer(sql( """SELECT time FROM timestamps WHERE time<'1969-12-31 16:00:00.003' AND time>'1969-12-31 16:00:00.001'"""), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002"))) + Row(Timestamp.valueOf("1969-12-31 16:00:00.002"))) checkAnswer(sql( """ |SELECT time FROM timestamps |WHERE time IN ('1969-12-31 16:00:00.001','1969-12-31 16:00:00.002') """.stripMargin), - Seq(Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.001")), - Row(java.sql.Timestamp.valueOf("1969-12-31 16:00:00.002")))) + Seq(Row(Timestamp.valueOf("1969-12-31 16:00:00.001")), + Row(Timestamp.valueOf("1969-12-31 16:00:00.002")))) checkAnswer(sql( "SELECT time FROM timestamps WHERE time='123'"), @@ -548,7 +550,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("date row") { checkAnswer(sql( """select cast("2015-01-28" as date) from testData limit 1"""), - Row(java.sql.Date.valueOf("2015-01-28")) + Row(Date.valueOf("2015-01-28")) ) } @@ -1484,11 +1486,12 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql("select interval") } assert(e1.message.contains("at least one time unit should be given for interval literal")) + // Currently we don't yet support nanosecond val e2 = intercept[AnalysisException] { sql("select interval 23 nanosecond") } - assert(e2.message.contains("No interval can be constructed")) + assert(e2.message.contains("no viable alternative at input 'interval 23 nanosecond'")) } test("SPARK-8945: add and subtract expressions for interval type") { @@ -1694,7 +1697,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { sql(s"select id from `org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`") } assert(e.message.contains("Table or view not found: " + - "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.`file_path`")) + "`org.apache.spark.sql.sources.HadoopFsRelationProvider`.file_path")) e = intercept[AnalysisException] { sql(s"select id from `Jdbc`.`file_path`") @@ -1893,7 +1896,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } test("Star Expansion - group by") { - withSQLConf("spark.sql.retainGroupColumns" -> "false") { + withSQLConf(SQLConf.DATAFRAME_RETAIN_GROUP_COLUMNS.key -> "false") { checkAnswer( testData2.groupBy($"a", $"b").agg($"*"), sql("SELECT * FROM testData2 group by a, b")) @@ -1933,7 +1936,7 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { test("Common subexpression elimination") { // TODO: support subexpression elimination in whole stage codegen - withSQLConf("spark.sql.codegen.wholeStage" -> "false") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "false") { // select from a table to prevent constant folding. val df = sql("SELECT a, b from testData2 limit 1") checkAnswer(df, Row(1, 1)) @@ -1982,9 +1985,9 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { df.selectExpr("testUdf(a + 1) + testUdf(1 + a)", "testUdf(a + 1)"), Row(4, 2), 1) // Try disabling it via configuration. - spark.conf.set("spark.sql.subexpressionElimination.enabled", "false") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "false") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 2) - spark.conf.set("spark.sql.subexpressionElimination.enabled", "true") + spark.conf.set(SQLConf.SUBEXPRESSION_ELIMINATION_ENABLED.key, "true") verifyCallCount(df.selectExpr("testUdf(a)", "testUdf(a)"), Row(1, 1), 1) } } @@ -2046,6 +2049,17 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { } } + test("xxhash64 function") { + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j") + withTempView("tbl") { + df.createOrReplaceTempView("tbl") + checkAnswer( + df.select(xxhash64($"i", $"j")), + sql("SELECT xxhash64(i, j) from tbl") + ) + } + } + test("join with using clause") { val df1 = Seq(("r1c1", "r1c2", "t1r1c3"), ("r2c1", "r2c2", "t1r2c3"), ("r3c1x", "r3c2", "t1r3c3")).toDF("c1", "c2", "c3") @@ -2856,6 +2870,316 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext { checkAnswer(sql("select 26393499451 / (1e6 * 1000)"), Row(BigDecimal("26.3934994510000"))) } } + + test("SPARK-25988: self join with aliases on partitioned tables #1") { + withTempView("tmpView1", "tmpView2") { + withTable("tab1", "tab2") { + sql( + """ + |CREATE TABLE `tab1` (`col1` INT, `TDATE` DATE) + |USING CSV + |PARTITIONED BY (TDATE) + """.stripMargin) + spark.table("tab1").where("TDATE >= '2017-08-15'").createOrReplaceTempView("tmpView1") + sql("CREATE TABLE `tab2` (`TDATE` DATE) USING parquet") + sql( + """ + |CREATE OR REPLACE TEMPORARY VIEW tmpView2 AS + |SELECT N.tdate, col1 AS aliasCol1 + |FROM tmpView1 N + |JOIN tab2 Z + |ON N.tdate = Z.tdate + """.stripMargin) + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "0") { + sql("SELECT * FROM tmpView2 x JOIN tmpView2 y ON x.tdate = y.tdate").collect() + } + } + } + } + + test("SPARK-25988: self join with aliases on partitioned tables #2") { + withTempView("tmp") { + withTable("tab1", "tab2") { + sql( + """ + |CREATE TABLE `tab1` (`EX` STRING, `TDATE` DATE) + |USING parquet + |PARTITIONED BY (tdate) + """.stripMargin) + sql("CREATE TABLE `tab2` (`TDATE` DATE) USING parquet") + sql( + """ + |CREATE OR REPLACE TEMPORARY VIEW TMP as + |SELECT N.tdate, EX AS new_ex + |FROM tab1 N + |JOIN tab2 Z + |ON N.tdate = Z.tdate + """.stripMargin) + sql( + """ + |SELECT * FROM TMP x JOIN TMP y + |ON x.tdate = y.tdate + """.stripMargin).queryExecution.executedPlan + } + } + } + + test("SPARK-26366: verify ReplaceExceptWithFilter") { + Seq(true, false).foreach { enabled => + withSQLConf(SQLConf.REPLACE_EXCEPT_WITH_FILTER.key -> enabled.toString) { + val df = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(0, 3, 5), + Row(0, 3, null), + Row(null, 3, 5), + Row(0, null, 5), + Row(0, null, null), + Row(null, null, 5), + Row(null, 3, null), + Row(null, null, null))), + StructType(Seq(StructField("c1", IntegerType), + StructField("c2", IntegerType), + StructField("c3", IntegerType)))) + val where = "c2 >= 3 OR c1 >= 0" + val whereNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + |OR (c1 IS NOT NULL AND c1 >= 0) + """.stripMargin + + val df_a = df.filter(where) + val df_b = df.filter(whereNullSafe) + checkAnswer(df.except(df_a), df.except(df_b)) + + val whereWithIn = "c2 >= 3 OR c1 in (2)" + val whereWithInNullSafe = + """ + |(c2 IS NOT NULL AND c2 >= 3) + """.stripMargin + val dfIn_a = df.filter(whereWithIn) + val dfIn_b = df.filter(whereWithInNullSafe) + checkAnswer(df.except(dfIn_a), df.except(dfIn_b)) + } + } + } + + test("SPARK-26402: accessing nested fields with different cases in case insensitive mode") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AnalysisException] { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + }.message + assert(msg.contains("No such struct field I in i")) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withTable("t") { + sql("create table t (s struct) using json") + checkAnswer(sql("select s.I from t group by s.i"), Nil) + } + } + } + + test("SPARK-27699 Validate pushed down filters") { + def checkPushedFilters(format: String, df: DataFrame, filters: Array[sources.Filter]): Unit = { + val scan = df.queryExecution.sparkPlan + .find(_.isInstanceOf[BatchScanExec]).get.asInstanceOf[BatchScanExec] + .scan + format match { + case "orc" => + assert(scan.isInstanceOf[OrcScan]) + assert(scan.asInstanceOf[OrcScan].pushedFilters === filters) + case "parquet" => + assert(scan.isInstanceOf[ParquetScan]) + assert(scan.asInstanceOf[ParquetScan].pushedFilters === filters) + case _ => + fail(s"unknow format $format") + } + } + + Seq("orc", "parquet").foreach { format => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + withTempPath { dir => + spark.range(10).map(i => (i, i.toString)).toDF("id", "s") + .write + .format(format) + .save(dir.getCanonicalPath) + val df = spark.read.format(format).load(dir.getCanonicalPath) + checkPushedFilters( + format, + df.where(('id < 2 and 's.contains("foo")) or ('id > 10 and 's.contains("bar"))), + Array(sources.Or(sources.LessThan("id", 2), sources.GreaterThan("id", 10)))) + checkPushedFilters( + format, + df.where('s.contains("foo") or ('id > 10 and 's.contains("bar"))), + Array.empty) + checkPushedFilters( + format, + df.where('id < 2 and not('id > 10 and 's.contains("bar"))), + Array(sources.IsNotNull("id"), sources.LessThan("id", 2))) + } + } + } + } + + test("SPARK-26709: OptimizeMetadataOnlyQuery does not handle empty records correctly") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + Seq(true, false).foreach { enableOptimizeMetadataOnlyQuery => + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> + enableOptimizeMetadataOnlyQuery.toString) { + withTable("t") { + sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)") + sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)") + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(5)) + } else { + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(null)) + } + checkAnswer(sql("SELECT MAX(col1) FROM t"), Row(null)) + } + + withTempPath { path => + val tabLocation = path.getCanonicalPath + val partLocation1 = tabLocation + "/p=3" + val partLocation2 = tabLocation + "/p=1" + // SPARK-23271 empty RDD when saved should write a metadata only file + val df = spark.emptyDataFrame.select(lit(1).as("col")) + df.write.parquet(partLocation1) + val df2 = spark.range(10).toDF("col") + df2.write.parquet(partLocation2) + val readDF = spark.read.parquet(tabLocation) + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(readDF.selectExpr("max(p)"), Row(3)) + } else { + checkAnswer(readDF.selectExpr("max(p)"), Row(1)) + } + checkAnswer(readDF.selectExpr("max(col)"), Row(9)) + } + } + } + } + } + + test("reset command should not fail with cache") { + withTable("tbl") { + val provider = spark.sessionState.conf.defaultDataSourceName + sql(s"CREATE TABLE tbl(i INT, j STRING) USING $provider") + sql("reset") + sql("cache table tbl") + sql("reset") + } + } + + test("string date comparison") { + spark.range(1).selectExpr("date '2000-01-01' as d").createOrReplaceTempView("t1") + val result = Date.valueOf("2000-01-01") + checkAnswer(sql("select * from t1 where d < '2000'"), Nil) + checkAnswer(sql("select * from t1 where d < '2001'"), Row(result)) + checkAnswer(sql("select * from t1 where d < '2000-01'"), Nil) + checkAnswer(sql("select * from t1 where d < '2000-01-01'"), Nil) + checkAnswer(sql("select * from t1 where d < '2000-1-1'"), Nil) + checkAnswer(sql("select * from t1 where d <= '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '1999-12-30'"), Nil) + checkAnswer(sql("select * from t1 where d = '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d = '2000-01-01'"), Row(result)) + checkAnswer(sql("select * from t1 where d = '2000-1-02'"), Nil) + checkAnswer(sql("select * from t1 where d > '2000-01-01'"), Nil) + checkAnswer(sql("select * from t1 where d > '1999'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-1-01'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-01'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-02'"), Nil) + checkAnswer(sql("select * from t1 where '2000' >= d"), Row(result)) + checkAnswer(sql("select * from t1 where d > '2000-13'"), Nil) + + withSQLConf(SQLConf.LEGACY_CAST_DATETIME_TO_STRING.key -> "true") { + checkAnswer(sql("select * from t1 where d < '2000'"), Nil) + checkAnswer(sql("select * from t1 where d < '2001'"), Row(result)) + checkAnswer(sql("select * from t1 where d < '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '1999'"), Nil) + checkAnswer(sql("select * from t1 where d >= '2000'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '1999-13'"), Row(result)) + checkAnswer(sql("select to_date('2000-01-01') > '1'"), Row(true)) + } + } + + test("string timestamp comparison") { + spark.range(1) + .selectExpr("timestamp '2000-01-01 01:10:00.000' as d") + .createOrReplaceTempView("t1") + val result = Timestamp.valueOf("2000-01-01 01:10:00") + checkAnswer(sql("select * from t1 where d < '2000'"), Nil) + checkAnswer(sql("select * from t1 where d < '2001'"), Row(result)) + checkAnswer(sql("select * from t1 where d < '2000-01'"), Nil) + checkAnswer(sql("select * from t1 where d < '2000-1-1'"), Nil) + checkAnswer(sql("select * from t1 where d < '2000-01-01 01:10:00.000'"), Nil) + checkAnswer(sql("select * from t1 where d < '2000-01-01 02:10:00.000'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '2000-1-1 01:10:00'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '2000-1-1 01:00:00'"), Nil) + checkAnswer(sql("select * from t1 where d = '2000-1-1 01:10:00.000'"), Row(result)) + checkAnswer(sql("select * from t1 where d = '2000-01-01 01:10:00.000'"), Row(result)) + checkAnswer(sql("select * from t1 where d = '2000-1-02 01:10:00.000'"), Nil) + checkAnswer(sql("select * from t1 where d > '2000'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '2000-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '2000-1-1 01:00:00.000'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '2001'"), Nil) + checkAnswer(sql("select * from t1 where d > '2000-01-02'"), Nil) + checkAnswer(sql("select * from t1 where d >= '2000-1-01'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-01'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-01 01:10:00.000'"), Row(result)) + checkAnswer(sql("select * from t1 where d >= '2000-01-02 01:10:00.000'"), Nil) + checkAnswer(sql("select * from t1 where '2000' >= d"), Nil) + checkAnswer(sql("select * from t1 where d > '2000-13'"), Nil) + + withSQLConf(SQLConf.LEGACY_CAST_DATETIME_TO_STRING.key -> "true") { + checkAnswer(sql("select * from t1 where d < '2000'"), Nil) + checkAnswer(sql("select * from t1 where d < '2001'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '2000-1-1'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '2000-01-02'"), Row(result)) + checkAnswer(sql("select * from t1 where d <= '1999'"), Nil) + checkAnswer(sql("select * from t1 where d >= '2000'"), Row(result)) + checkAnswer(sql("select * from t1 where d > '1999-13'"), Row(result)) + checkAnswer(sql("select to_timestamp('2000-01-01 01:10:00') > '1'"), Row(true)) + } + } + + test("SPARK-28156: self-join should not miss cached view") { + withTable("table1") { + withView("table1_vw") { + val df = Seq.tabulate(5) { x => (x, x + 1, x + 2, x + 3) }.toDF("a", "b", "c", "d") + df.write.mode("overwrite").format("orc").saveAsTable("table1") + sql("drop view if exists table1_vw") + sql("create view table1_vw as select * from table1") + + val cachedView = sql("select a, b, c, d from table1_vw") + + cachedView.createOrReplaceTempView("cachedview") + cachedView.persist() + + val queryDf = sql( + s"""select leftside.a, leftside.b + |from cachedview leftside + |join cachedview rightside + |on leftside.a = rightside.a + """.stripMargin) + + val inMemoryTableScan = queryDf.queryExecution.executedPlan.collect { + case i: InMemoryTableScanExec => i + } + assert(inMemoryTableScan.size == 2) + checkAnswer(queryDf, Row(0, 1) :: Row(1, 2) :: Row(2, 3) :: Row(3, 4) :: Row(4, 5) :: Nil) + } + } + + } } 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 826408c7161e9..e4052b7ed3fff 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 @@ -26,7 +26,9 @@ 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.execution.command.{DescribeColumnCommand, DescribeTableCommand} +import org.apache.spark.sql.execution.HiveResult.hiveResultString +import org.apache.spark.sql.execution.command.{DescribeColumnCommand, DescribeCommandBase} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.StructType @@ -46,11 +48,16 @@ import org.apache.spark.sql.types.StructType * build/sbt "~sql/test-only *SQLQueryTestSuite -- -z inline-table.sql" * }}} * - * To re-generate golden files, run: + * To re-generate golden files for entire suite, run: * {{{ * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite" * }}} * + * To re-generate golden file for a single test, run: + * {{{ + * SPARK_GENERATE_GOLDEN_FILES=1 build/sbt "sql/test-only *SQLQueryTestSuite -- -z describe.sql" + * }}} + * * The format for input files is simple: * 1. A list of SQL queries separated by semicolon. * 2. Lines starting with -- are treated as comments and ignored. @@ -79,9 +86,25 @@ import org.apache.spark.sql.types.StructType * -- !query 1 * ... * }}} + * + * Note that UDF tests work differently. After the test files under 'inputs/udf' directory are + * detected, it creates three test cases: + * + * - Scala UDF test case with a Scalar UDF registered as the name 'udf'. + * + * - Python UDF test case with a Python UDF registered as the name 'udf' + * iff Python executable and pyspark are available. + * + * - Scalar Pandas UDF test case with a Scalar Pandas UDF registered as the name 'udf' + * iff Python executable, pyspark, pandas and pyarrow are available. + * + * Therefore, UDF test cases should have single input and output files but executed by three + * different types of UDFs. See 'udf/udf-inner-join.sql' as an example. */ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { + import IntegratedUDFTestUtils._ + private val regenerateGoldenFiles: Boolean = System.getenv("SPARK_GENERATE_GOLDEN_FILES") == "1" private val baseResourcePath = { @@ -98,19 +121,16 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { private val inputFilePath = new File(baseResourcePath, "inputs").getAbsolutePath private val goldenFilePath = new File(baseResourcePath, "results").getAbsolutePath + private val validFileExtensions = ".sql" + /** List of test cases to ignore, in lower cases. */ private val blackList = Set( - "blacklist.sql", // Do NOT remove this one. It is here to test the blacklist functionality. - ".DS_Store" // A meta-file that may be created on Mac by Finder App. - // We should ignore this file from processing. + "blacklist.sql" // Do NOT remove this one. It is here to test the blacklist functionality. ) // Create all the test cases. listTestCases().foreach(createScalaTestCase) - /** A test case. */ - private case class TestCase(name: String, inputFile: String, resultFile: String) - /** A single SQL query's output. */ private case class QueryOutput(sql: String, schema: String, output: String) { def toString(queryIndex: Int): String = { @@ -119,51 +139,123 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { sql + "\n" + s"-- !query $queryIndex schema\n" + schema + "\n" + - s"-- !query $queryIndex output\n" + + s"-- !query $queryIndex output\n" + output } } + /** A test case. */ + private trait TestCase { + val name: String + val inputFile: String + val resultFile: String + } + + /** + * traits that indicate UDF or PgSQL to trigger the code path specific to each. For instance, + * PgSQL tests require to register some UDF functions. + */ + private trait PgSQLTest + + private trait UDFTest { + val udf: TestUDF + } + + /** A regular test case. */ + private case class RegularTestCase( + name: String, inputFile: String, resultFile: String) extends TestCase + + /** A PostgreSQL test case. */ + private case class PgSQLTestCase( + name: String, inputFile: String, resultFile: String) extends TestCase with PgSQLTest + + /** A UDF test case. */ + private case class UDFTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest + + /** A UDF PostgreSQL test case. */ + private case class UDFPgSQLTestCase( + name: String, + inputFile: String, + resultFile: String, + udf: TestUDF) extends TestCase with UDFTest with PgSQLTest + private def createScalaTestCase(testCase: TestCase): Unit = { if (blackList.exists(t => testCase.name.toLowerCase(Locale.ROOT).contains(t.toLowerCase(Locale.ROOT)))) { // Create a test case to ignore this case. ignore(testCase.name) { /* Do nothing */ } - } else { - // Create a test case to run this case. - test(testCase.name) { runTest(testCase) } + } else testCase match { + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestPythonUDF] && !shouldTestPythonUDFs => + ignore(s"${testCase.name} is skipped because " + + s"[$pythonExec] and/or pyspark were not available.") { + /* Do nothing */ + } + case udfTestCase: UDFTest + if udfTestCase.udf.isInstanceOf[TestScalarPandasUDF] && !shouldTestScalarPandasUDFs => + ignore(s"${testCase.name} is skipped because pyspark," + + s"pandas and/or pyarrow were not available in [$pythonExec].") { + /* Do nothing */ + } + case _ => + // Create a test case to run this case. + test(testCase.name) { + runTest(testCase) + } } } + // For better test coverage, runs the tests on mixed config sets: WHOLESTAGE_CODEGEN_ENABLED + // and CODEGEN_FACTORY_MODE. + private lazy val codegenConfigSets = Array( + ("true", "CODEGEN_ONLY"), + ("false", "CODEGEN_ONLY"), + ("false", "NO_CODEGEN") + ).map { case (wholeStageCodegenEnabled, codegenFactoryMode) => + Array(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> wholeStageCodegenEnabled, + SQLConf.CODEGEN_FACTORY_MODE.key -> codegenFactoryMode) + } + /** Run a test case. */ private def runTest(testCase: TestCase): Unit = { val input = fileToString(new File(testCase.inputFile)) val (comments, code) = input.split("\n").partition(_.startsWith("--")) - val configSets = { - val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) - val configs = configLines.map(_.split(",").map { confAndValue => - val (conf, value) = confAndValue.span(_ != '=') - conf.trim -> value.substring(1).trim - }) - // When we are regenerating the golden files we don't need to run all the configs as they - // all need to return the same result - if (regenerateGoldenFiles && configs.nonEmpty) { - configs.take(1) - } else { - configs - } - } + // List of SQL queries to run // note: this is not a robust way to split queries using semicolon, but works for now. val queries = code.mkString("\n").split("(?<=[^\\\\]);").map(_.trim).filter(_ != "").toSeq - if (configSets.isEmpty) { - runQueries(queries, testCase.resultFile, None) + // When we are regenerating the golden files, we don't need to set any config as they + // all need to return the same result + if (regenerateGoldenFiles) { + runQueries(queries, testCase, None) } else { + val configSets = { + val configLines = comments.filter(_.startsWith("--SET")).map(_.substring(5)) + val configs = configLines.map(_.split(",").map { confAndValue => + val (conf, value) = confAndValue.span(_ != '=') + conf.trim -> value.substring(1).trim + }) + + if (configs.nonEmpty) { + codegenConfigSets.flatMap { codegenConfig => + configs.map { config => + config ++ codegenConfig + } + } + } else { + codegenConfigSets + } + } + configSets.foreach { configSet => try { - runQueries(queries, testCase.resultFile, Some(configSet)) + runQueries(queries, testCase, Some(configSet)) } catch { case e: Throwable => val configs = configSet.map { @@ -178,13 +270,33 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { private def runQueries( queries: Seq[String], - resultFileName: String, + testCase: TestCase, configSet: Option[Seq[(String, String)]]): Unit = { // 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 => + registerTestUDF(udfTestCase.udf, localSparkSession) + case _ => + } + + testCase match { + case _: PgSQLTest => + // booleq/boolne used by boolean.sql + localSparkSession.udf.register("booleq", (b1: Boolean, b2: Boolean) => b1 == b2) + localSparkSession.udf.register("boolne", (b1: Boolean, b2: Boolean) => b1 != b2) + // vol used by boolean.sql and case.sql. + localSparkSession.udf.register("vol", (s: String) => s) + // PostgreSQL enabled cartesian product by default. + localSparkSession.conf.set(SQLConf.CROSS_JOINS_ENABLED.key, true) + localSparkSession.conf.set(SQLConf.ANSI_SQL_PARSER.key, true) + localSparkSession.conf.set(SQLConf.PREFER_INTEGRAL_DIVISION.key, true) + case _ => + } + if (configSet.isDefined) { // Execute the list of set operation in order to add the desired configs val setOperations = configSet.get.map { case (key, value) => s"set $key=$value" } @@ -208,7 +320,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { s"-- Number of queries: ${outputs.size}\n\n\n" + outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n" } - val resultFile = new File(resultFileName) + val resultFile = new File(testCase.resultFile) val parent = resultFile.getParentFile if (!parent.exists()) { assert(parent.mkdirs(), "Could not create directory: " + parent) @@ -218,7 +330,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Read back the golden file. val expectedOutputs: Seq[QueryOutput] = { - val goldenOutput = fileToString(new File(resultFileName)) + val goldenOutput = fileToString(new File(testCase.resultFile)) val segments = goldenOutput.split("-- !query.+\n") // each query has 3 segments, plus the header @@ -243,7 +355,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { assertResult(expected.sql, s"SQL query did not match for query #$i\n${expected.sql}") { output.sql } - assertResult(expected.schema, s"Schema did not match for query #$i\n${expected.sql}") { + assertResult(expected.schema, + s"Schema did not match for query #$i\n${expected.sql}: $output") { output.schema } assertResult(expected.output, s"Result did not match for query #$i\n${expected.sql}") { @@ -257,7 +370,7 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { // Returns true if the plan is supposed to be sorted. def isSorted(plan: LogicalPlan): Boolean = plan match { case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false - case _: DescribeTableCommand | _: DescribeColumnCommand => true + case _: DescribeCommandBase | _: DescribeColumnCommand => true case PhysicalOperation(_, _, Sort(_, true, _)) => true case _ => plan.children.iterator.exists(isSorted) } @@ -266,12 +379,17 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { val df = session.sql(sql) val schema = df.schema val notIncludedMsg = "[not included in comparison]" + val clsName = this.getClass.getCanonicalName // Get answer, but also get rid of the #1234 expression ids that show up in explain plans - val answer = df.queryExecution.hiveResultString().map(_.replaceAll("#\\d+", "#x") - .replaceAll("Location.*/sql/core/", s"Location ${notIncludedMsg}sql/core/") + val answer = hiveResultString(df.queryExecution.executedPlan) + .map(_.replaceAll("#\\d+", "#x") + .replaceAll( + s"Location.*/sql/core/spark-warehouse/$clsName/", + s"Location ${notIncludedMsg}sql/core/spark-warehouse/") .replaceAll("Created By.*", s"Created By $notIncludedMsg") .replaceAll("Created Time.*", s"Created Time $notIncludedMsg") .replaceAll("Last Access.*", s"Last Access $notIncludedMsg") + .replaceAll("Partition Statistics\t\\d+", s"Partition Statistics\t$notIncludedMsg") .replaceAll("\\*\\(\\d+\\) ", "*")) // remove the WholeStageCodegen codegenStageIds // If the output is not pre-sorted, sort it. @@ -291,18 +409,37 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { } private def listTestCases(): Seq[TestCase] = { - listFilesRecursively(new File(inputFilePath)).map { file => + listFilesRecursively(new File(inputFilePath)).flatMap { file => val resultFile = file.getAbsolutePath.replace(inputFilePath, goldenFilePath) + ".out" val absPath = file.getAbsolutePath val testCaseName = absPath.stripPrefix(inputFilePath).stripPrefix(File.separator) - TestCase(testCaseName, absPath, resultFile) + + if (file.getAbsolutePath.startsWith( + s"$inputFilePath${File.separator}udf${File.separator}pgSQL")) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => + UDFPgSQLTestCase( + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}udf")) { + Seq(TestScalaUDF("udf"), TestPythonUDF("udf"), TestScalarPandasUDF("udf")).map { udf => + UDFTestCase( + s"$testCaseName - ${udf.prettyName}", absPath, resultFile, udf) + } + } else if (file.getAbsolutePath.startsWith(s"$inputFilePath${File.separator}pgSQL")) { + PgSQLTestCase(testCaseName, absPath, resultFile) :: Nil + } else { + RegularTestCase(testCaseName, absPath, resultFile) :: Nil + } } } /** Returns all the files (not directories) in a directory, recursively. */ private def listFilesRecursively(path: File): Seq[File] = { val (dirs, files) = path.listFiles().partition(_.isDirectory) - files ++ dirs.flatMap(listFilesRecursively) + // Filter out test files with invalid extensions such as temp files created + // by vi (.swp), Mac (.DS_Store) etc. + val filteredFiles = files.filter(_.getName.endsWith(validFileExtensions)) + filteredFiles ++ dirs.flatMap(listFilesRecursively) } /** Load built-in test tables into the SparkSession. */ @@ -322,6 +459,66 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext { Tuple1(Map(1 -> "a5")) :: Nil) .toDF("mapcol") .createOrReplaceTempView("mapdata") + + session + .read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema("a int, b float") + .load(testFile("test-data/postgresql/agg.data")) + .createOrReplaceTempView("aggtest") + + session + .read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema( + """ + |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 + """.stripMargin) + .load(testFile("test-data/postgresql/onek.data")) + .createOrReplaceTempView("onek") + + session + .read + .format("csv") + .options(Map("delimiter" -> "\t", "header" -> "false")) + .schema( + """ + |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 + """.stripMargin) + .load(testFile("test-data/postgresql/tenk.data")) + .createOrReplaceTempView("tenk1") } private val originalTimeZone = TimeZone.getDefault diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala index cd6b2647e0be6..1a1c956aed3d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SerializationSuite.scala @@ -27,4 +27,9 @@ class SerializationSuite extends SparkFunSuite with SharedSQLContext { val spark = SparkSession.builder.getOrCreate() new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sqlContext) } + + test("[SPARK-26409] SQLConf should be serializable") { + val spark = SparkSession.builder.getOrCreate() + new JavaSerializer(new SparkConf()).newInstance().serialize(spark.sessionState.conf) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala index e1b5eba53f06a..6fe58b780eaef 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SessionStateSuite.scala @@ -37,7 +37,10 @@ class SessionStateSuite extends SparkFunSuite { override def beforeAll(): Unit = { super.beforeAll() - activeSession = SparkSession.builder().master("local").getOrCreate() + activeSession = SparkSession.builder() + .master("local") + .config("default-config", "default") + .getOrCreate() } override def afterAll(): Unit = { @@ -155,6 +158,7 @@ class SessionStateSuite extends SparkFunSuite { assert(forkedSession ne activeSession) assert(forkedSession.listenerManager ne activeSession.listenerManager) runCollectQueryOn(forkedSession) + activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) assert(collectorA.commands.length == 1) // forked should callback to A assert(collectorA.commands(0) == "collect") @@ -162,12 +166,14 @@ class SessionStateSuite extends SparkFunSuite { // => changes to forked do not affect original forkedSession.listenerManager.register(collectorB) runCollectQueryOn(activeSession) + activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) assert(collectorB.commands.isEmpty) // original should not callback to B assert(collectorA.commands.length == 2) // original should still callback to A assert(collectorA.commands(1) == "collect") // <= changes to original do not affect forked activeSession.listenerManager.register(collectorC) runCollectQueryOn(forkedSession) + activeSession.sparkContext.listenerBus.waitUntilEmpty(1000) assert(collectorC.commands.isEmpty) // forked should not callback to C assert(collectorA.commands.length == 3) // forked should still callback to A assert(collectorB.commands.length == 1) // forked should still callback to B @@ -216,4 +222,21 @@ class SessionStateSuite extends SparkFunSuite { val forkedSession = activeSession.cloneSession() assert(activeSession.sharedState eq forkedSession.sharedState) } + + test("SPARK-27253: forked new session should not discard SQLConf overrides") { + val key = "default-config" + try { + // override default config + activeSession.conf.set(key, "active") + + val forkedSession = activeSession.cloneSession() + assert(forkedSession ne activeSession) + assert(forkedSession.conf ne activeSession.conf) + + // forked new session should not discard SQLConf overrides + assert(forkedSession.conf.get(key) == "active") + } finally { + activeSession.conf.unset(key) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala similarity index 56% rename from sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala index 34ca790299859..5c347d2677d5e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ShowCreateTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ShowCreateTableSuite.scala @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql -import org.apache.spark.sql.{AnalysisException, QueryTest} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} import org.apache.spark.util.Utils -class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { +class SimpleShowCreateTableSuite extends ShowCreateTableSuite with SharedSQLContext + +abstract class ShowCreateTableSuite extends QueryTest with SQLTestUtils { import testImplicits._ test("data source table with user specified schema") { @@ -105,193 +105,67 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing } } - test("data source table using Dataset API") { + test("data source table with a comment") { withTable("ddl_test") { - spark - .range(3) - .select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd, 'id as 'e) - .write - .mode("overwrite") - .partitionBy("a", "b") - .bucketBy(2, "c", "d") - .saveAsTable("ddl_test") - - checkCreateTable("ddl_test") - } - } - - test("simple hive table") { - withTable("t1") { - sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |TBLPROPERTIES ( - | 'prop1' = 'value1', - | 'prop2' = 'value2' - |) - """.stripMargin - ) - - checkCreateTable("t1") - } - } - - test("simple external hive table") { - withTempDir { dir => - withTable("t1") { - sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |LOCATION '${dir.toURI}' - |TBLPROPERTIES ( - | 'prop1' = 'value1', - | 'prop2' = 'value2' - |) - """.stripMargin - ) - - checkCreateTable("t1") - } - } - } - - test("partitioned hive table") { - withTable("t1") { sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |COMMENT 'bla' - |PARTITIONED BY ( - | p1 BIGINT COMMENT 'bla', - | p2 STRING - |) - """.stripMargin - ) - - checkCreateTable("t1") - } - } - - test("hive table with explicit storage info") { - withTable("t1") { - sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' - |COLLECTION ITEMS TERMINATED BY '@' - |MAP KEYS TERMINATED BY '#' - |NULL DEFINED AS 'NaN' + s"""CREATE TABLE ddl_test + |USING json + |COMMENT 'This is a comment' + |AS SELECT 1 AS a, "foo" AS b, 2.5 AS c """.stripMargin ) - checkCreateTable("t1") + checkCreateTable("ddl_test") } } - test("hive table with STORED AS clause") { - withTable("t1") { + test("data source table with table properties") { + withTable("ddl_test") { sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |STORED AS PARQUET + s"""CREATE TABLE ddl_test + |USING json + |TBLPROPERTIES ('a' = '1') + |AS SELECT 1 AS a, "foo" AS b, 2.5 AS c """.stripMargin ) - checkCreateTable("t1") + checkCreateTable("ddl_test") } } - test("hive table with serde info") { - withTable("t1") { - sql( - s"""CREATE TABLE t1 ( - | c1 INT COMMENT 'bla', - | c2 STRING - |) - |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' - |WITH SERDEPROPERTIES ( - | 'mapkey.delim' = ',', - | 'field.delim' = ',' - |) - |STORED AS - | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' - | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' - """.stripMargin - ) + test("data source table using Dataset API") { + withTable("ddl_test") { + spark + .range(3) + .select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd, 'id as 'e) + .write + .mode("overwrite") + .partitionBy("a", "b") + .bucketBy(2, "c", "d") + .saveAsTable("ddl_test") - checkCreateTable("t1") + checkCreateTable("ddl_test") } } - test("hive view") { + test("view") { withView("v1") { sql("CREATE VIEW v1 AS SELECT 1 AS a") checkCreateView("v1") } } - test("hive view with output columns") { + test("view with output columns") { withView("v1") { sql("CREATE VIEW v1 (b) AS SELECT 1 AS a") checkCreateView("v1") } } - test("hive bucketing is supported") { - withTable("t1") { - sql( - s"""CREATE TABLE t1 (a INT, b STRING) - |CLUSTERED BY (a) - |SORTED BY (b) - |INTO 2 BUCKETS - """.stripMargin - ) - checkCreateTable("t1") - } - } - - test("hive partitioned view is not supported") { - withTable("t1") { - withView("v1") { - sql( - s""" - |CREATE TABLE t1 (c1 INT, c2 STRING) - |PARTITIONED BY ( - | p1 BIGINT COMMENT 'bla', - | p2 STRING ) - """.stripMargin) - - createRawHiveTable( - s""" - |CREATE VIEW v1 - |PARTITIONED ON (p1, p2) - |AS SELECT * from t1 - """.stripMargin - ) - - val cause = intercept[AnalysisException] { - sql("SHOW CREATE TABLE v1") - } - - assert(cause.getMessage.contains(" - partitioned view")) - } - } - } - test("SPARK-24911: keep quotes for nested fields") { withTable("t1") { - val createTable = "CREATE TABLE `t1`(`a` STRUCT<`b`: STRING>)" - sql(createTable) + val createTable = "CREATE TABLE `t1` (`a` STRUCT<`b`: STRING>)" + sql(s"$createTable USING json") val shownDDL = sql(s"SHOW CREATE TABLE t1") .head() .getString(0) @@ -303,16 +177,11 @@ class ShowCreateTableSuite extends QueryTest with SQLTestUtils with TestHiveSing } } - private def createRawHiveTable(ddl: String): Unit = { - hiveContext.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog] - .client.runSqlHive(ddl) - } - - private def checkCreateTable(table: String): Unit = { + protected def checkCreateTable(table: String): Unit = { checkCreateTableOrView(TableIdentifier(table, Some("default")), "TABLE") } - private def checkCreateView(table: String): Unit = { + protected def checkCreateView(table: String): Unit = { checkCreateTableOrView(TableIdentifier(table, Some("default")), "VIEW") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala index 44bf8624a6bcd..10b17571d2aaa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SparkSessionBuilderSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql import org.scalatest.BeforeAndAfterEach import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.internal.SQLConf /** @@ -38,7 +39,7 @@ class SparkSessionBuilderSuite extends SparkFunSuite with BeforeAndAfterEach { test("create with config options and propagate them to SparkContext and SparkSession") { val session = SparkSession.builder() .master("local") - .config("spark.ui.enabled", value = false) + .config(UI_ENABLED.key, value = false) .config("some-config", "v2") .getOrCreate() assert(session.sparkContext.conf.get("some-config") == "v2") 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 43db79663322a..74341f93dd5ba 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 @@ -16,21 +16,27 @@ */ package org.apache.spark.sql -import org.apache.spark.SparkFunSuite +import org.apache.spark.{SparkFunSuite, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.{SparkPlan, SparkStrategy} -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.execution._ +import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector +import org.apache.spark.sql.internal.StaticSQLConf.SPARK_SESSION_EXTENSIONS +import org.apache.spark.sql.types.{DataType, Decimal, IntegerType, LongType, Metadata, StructType} +import org.apache.spark.sql.vectorized.{ColumnarArray, ColumnarBatch, ColumnarMap, ColumnVector} +import org.apache.spark.unsafe.types.UTF8String /** * Test cases for the [[SparkSessionExtensions]]. */ class SparkSessionExtensionSuite extends SparkFunSuite { type ExtensionsBuilder = SparkSessionExtensions => Unit - private def create(builder: ExtensionsBuilder): ExtensionsBuilder = builder + private def create(builder: ExtensionsBuilder): Seq[ExtensionsBuilder] = Seq(builder) private def stop(spark: SparkSession): Unit = { spark.stop() @@ -38,66 +44,201 @@ class SparkSessionExtensionSuite extends SparkFunSuite { SparkSession.clearDefaultSession() } - private def withSession(builder: ExtensionsBuilder)(f: SparkSession => Unit): Unit = { - val spark = SparkSession.builder().master("local[1]").withExtensions(builder).getOrCreate() + private def withSession(builders: Seq[ExtensionsBuilder])(f: SparkSession => Unit): Unit = { + val builder = SparkSession.builder().master("local[1]") + builders.foreach(builder.withExtensions) + val spark = builder.getOrCreate() try f(spark) finally { stop(spark) } } test("inject analyzer rule") { - withSession(_.injectResolutionRule(MyRule)) { session => + withSession(Seq(_.injectResolutionRule(MyRule))) { session => assert(session.sessionState.analyzer.extendedResolutionRules.contains(MyRule(session))) } } + test("inject post hoc resolution analyzer rule") { + withSession(Seq(_.injectPostHocResolutionRule(MyRule))) { session => + assert(session.sessionState.analyzer.postHocResolutionRules.contains(MyRule(session))) + } + } + test("inject check analysis rule") { - withSession(_.injectCheckRule(MyCheckRule)) { session => + withSession(Seq(_.injectCheckRule(MyCheckRule))) { session => assert(session.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(session))) } } test("inject optimizer rule") { - withSession(_.injectOptimizerRule(MyRule)) { session => + withSession(Seq(_.injectOptimizerRule(MyRule))) { session => assert(session.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(session))) } } test("inject spark planner strategy") { - withSession(_.injectPlannerStrategy(MySparkStrategy)) { session => + withSession(Seq(_.injectPlannerStrategy(MySparkStrategy))) { session => assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) } } test("inject parser") { val extension = create { extensions => - extensions.injectParser((_, _) => CatalystSqlParser) + extensions.injectParser((_: SparkSession, _: ParserInterface) => CatalystSqlParser) } withSession(extension) { session => - assert(session.sessionState.sqlParser == CatalystSqlParser) + assert(session.sessionState.sqlParser === CatalystSqlParser) + } + } + + test("inject multiple rules") { + withSession(Seq(_.injectOptimizerRule(MyRule), + _.injectPlannerStrategy(MySparkStrategy))) { session => + assert(session.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(session))) + assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) } } test("inject stacked parsers") { val extension = create { extensions => - extensions.injectParser((_, _) => CatalystSqlParser) + extensions.injectParser((_: SparkSession, _: ParserInterface) => CatalystSqlParser) extensions.injectParser(MyParser) extensions.injectParser(MyParser) } withSession(extension) { session => val parser = MyParser(session, MyParser(session, CatalystSqlParser)) - assert(session.sessionState.sqlParser == parser) + assert(session.sessionState.sqlParser === parser) + } + } + + test("inject function") { + val extensions = create { extensions => + extensions.injectFunction(MyExtensions.myFunction) + } + withSession(extensions) { session => + assert(session.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1).isDefined) + } + } + + test("inject columnar") { + val extensions = create { extensions => + extensions.injectColumnar(session => + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) + } + withSession(extensions) { session => + assert(session.sessionState.columnarRules.contains( + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + import session.sqlContext.implicits._ + // repartitioning avoids having the add operation pushed up into the LocalTableScan + val data = Seq((100L), (200L), (300L)).toDF("vals").repartition(1) + val df = data.selectExpr("vals + 1") + // Verify that both pre and post processing of the plan worked. + val found = df.queryExecution.executedPlan.collect { + case rep: ReplacedRowToColumnarExec => 1 + case proj: ColumnarProjectExec => 10 + case c2r: ColumnarToRowExec => 100 + }.sum + assert(found == 111) + + // Verify that we get back the expected, wrong, result + val result = df.collect() + assert(result(0).getLong(0) == 102L) // Check that broken columnar Add was used. + assert(result(1).getLong(0) == 202L) + assert(result(2).getLong(0) == 302L) } } test("use custom class for extensions") { val session = SparkSession.builder() .master("local[1]") - .config("spark.sql.extensions", classOf[MyExtensions].getCanonicalName) + .config(SPARK_SESSION_EXTENSIONS.key, classOf[MyExtensions].getCanonicalName) .getOrCreate() try { assert(session.sessionState.planner.strategies.contains(MySparkStrategy(session))) assert(session.sessionState.analyzer.extendedResolutionRules.contains(MyRule(session))) + assert(session.sessionState.analyzer.postHocResolutionRules.contains(MyRule(session))) + assert(session.sessionState.analyzer.extendedCheckRules.contains(MyCheckRule(session))) + assert(session.sessionState.optimizer.batches.flatMap(_.rules).contains(MyRule(session))) + assert(session.sessionState.sqlParser.isInstanceOf[MyParser]) + assert(session.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1).isDefined) + assert(session.sessionState.columnarRules.contains( + MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule()))) + } finally { + stop(session) + } + } + + test("use multiple custom class for extensions in the specified order") { + val session = SparkSession.builder() + .master("local[1]") + .config(SPARK_SESSION_EXTENSIONS.key, Seq( + classOf[MyExtensions2].getCanonicalName, + classOf[MyExtensions].getCanonicalName).mkString(",")) + .getOrCreate() + try { + assert(session.sessionState.planner.strategies.containsSlice( + Seq(MySparkStrategy2(session), MySparkStrategy(session)))) + val orderedRules = Seq(MyRule2(session), MyRule(session)) + val orderedCheckRules = Seq(MyCheckRule2(session), MyCheckRule(session)) + val parser = MyParser(session, CatalystSqlParser) + assert(session.sessionState.analyzer.extendedResolutionRules.containsSlice(orderedRules)) + assert(session.sessionState.analyzer.postHocResolutionRules.containsSlice(orderedRules)) + assert(session.sessionState.analyzer.extendedCheckRules.containsSlice(orderedCheckRules)) + assert(session.sessionState.optimizer.batches.flatMap(_.rules).filter(orderedRules.contains) + .containsSlice(orderedRules ++ orderedRules)) // The optimizer rules are duplicated + assert(session.sessionState.sqlParser === parser) + assert(session.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1).isDefined) + assert(session.sessionState.functionRegistry + .lookupFunction(MyExtensions2.myFunction._1).isDefined) + } finally { + stop(session) + } + } + + test("allow an extension to be duplicated") { + val session = SparkSession.builder() + .master("local[1]") + .config(SPARK_SESSION_EXTENSIONS.key, Seq( + classOf[MyExtensions].getCanonicalName, + classOf[MyExtensions].getCanonicalName).mkString(",")) + .getOrCreate() + try { + assert(session.sessionState.planner.strategies.count(_ === MySparkStrategy(session)) === 2) + assert(session.sessionState.analyzer.extendedResolutionRules.count(_ === MyRule(session)) === + 2) + assert(session.sessionState.analyzer.postHocResolutionRules.count(_ === MyRule(session)) === + 2) + assert(session.sessionState.analyzer.extendedCheckRules.count(_ === MyCheckRule(session)) === + 2) + assert(session.sessionState.optimizer.batches.flatMap(_.rules) + .count(_ === MyRule(session)) === 4) // The optimizer rules are duplicated + val outerParser = session.sessionState.sqlParser + assert(outerParser.isInstanceOf[MyParser]) + assert(outerParser.asInstanceOf[MyParser].delegate.isInstanceOf[MyParser]) + assert(session.sessionState.functionRegistry + .lookupFunction(MyExtensions.myFunction._1).isDefined) + } finally { + stop(session) + } + } + + test("use the last registered function name when there are duplicates") { + val session = SparkSession.builder() + .master("local[1]") + .config(SPARK_SESSION_EXTENSIONS.key, Seq( + classOf[MyExtensions2].getCanonicalName, + classOf[MyExtensions2Duplicate].getCanonicalName).mkString(",")) + .getOrCreate() + try { + val lastRegistered = session.sessionState.functionRegistry + .lookupFunction(FunctionIdentifier("myFunction2")) + assert(lastRegistered.isDefined) + assert(lastRegistered.get !== MyExtensions2.myFunction._2) + assert(lastRegistered.get === MyExtensions2Duplicate.myFunction._2) } finally { stop(session) } @@ -129,6 +270,9 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars override def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = delegate.parseFunctionIdentifier(sqlText) + override def parseMultipartIdentifier(sqlText: String): Seq[String] = + delegate.parseMultipartIdentifier(sqlText) + override def parseTableSchema(sqlText: String): StructType = delegate.parseTableSchema(sqlText) @@ -136,9 +280,431 @@ case class MyParser(spark: SparkSession, delegate: ParserInterface) extends Pars delegate.parseDataType(sqlText) } +object MyExtensions { + + val myFunction = (FunctionIdentifier("myFunction"), + new ExpressionInfo("noClass", "myDb", "myFunction", "usage", "extended usage"), + (_: Seq[Expression]) => Literal(5, IntegerType)) +} + +case class CloseableColumnBatchIterator(itr: Iterator[ColumnarBatch], + f: ColumnarBatch => ColumnarBatch) extends Iterator[ColumnarBatch] { + var cb: ColumnarBatch = null + + private def closeCurrentBatch(): Unit = { + if (cb != null) { + cb.close + cb = null + } + } + + TaskContext.get().addTaskCompletionListener[Unit]((tc: TaskContext) => { + closeCurrentBatch() + }) + + override def hasNext: Boolean = { + closeCurrentBatch() + itr.hasNext + } + + override def next(): ColumnarBatch = { + closeCurrentBatch() + cb = f(itr.next()) + cb + } +} + +object NoCloseColumnVector extends Logging { + def wrapIfNeeded(cv: ColumnVector): NoCloseColumnVector = cv match { + case ref: NoCloseColumnVector => + ref + case vec => NoCloseColumnVector(vec) + } +} + +/** + * Provide a ColumnVector so ColumnarExpression can close temporary values without + * having to guess what type it really is. + */ +case class NoCloseColumnVector(wrapped: ColumnVector) extends ColumnVector(wrapped.dataType) { + private var refCount = 1 + + /** + * Don't actually close the ColumnVector this wraps. The producer of the vector will take + * care of that. + */ + override def close(): Unit = { + // Empty + } + + override def hasNull: Boolean = wrapped.hasNull + + override def numNulls(): Int = wrapped.numNulls + + override def isNullAt(rowId: Int): Boolean = wrapped.isNullAt(rowId) + + override def getBoolean(rowId: Int): Boolean = wrapped.getBoolean(rowId) + + override def getByte(rowId: Int): Byte = wrapped.getByte(rowId) + + override def getShort(rowId: Int): Short = wrapped.getShort(rowId) + + override def getInt(rowId: Int): Int = wrapped.getInt(rowId) + + override def getLong(rowId: Int): Long = wrapped.getLong(rowId) + + override def getFloat(rowId: Int): Float = wrapped.getFloat(rowId) + + override def getDouble(rowId: Int): Double = wrapped.getDouble(rowId) + + override def getArray(rowId: Int): ColumnarArray = wrapped.getArray(rowId) + + override def getMap(ordinal: Int): ColumnarMap = wrapped.getMap(ordinal) + + override def getDecimal(rowId: Int, precision: Int, scale: Int): Decimal = + wrapped.getDecimal(rowId, precision, scale) + + override def getUTF8String(rowId: Int): UTF8String = wrapped.getUTF8String(rowId) + + override def getBinary(rowId: Int): Array[Byte] = wrapped.getBinary(rowId) + + override protected def getChild(ordinal: Int): ColumnVector = wrapped.getChild(ordinal) +} + +trait ColumnarExpression extends Expression with Serializable { + /** + * Returns true if this expression supports columnar processing through [[columnarEval]]. + */ + def supportsColumnar: Boolean = true + + /** + * Returns the result of evaluating this expression on the entire + * [[org.apache.spark.sql.vectorized.ColumnarBatch]]. The result of + * calling this may be a single [[org.apache.spark.sql.vectorized.ColumnVector]] or a scalar + * value. Scalar values typically happen if they are a part of the expression i.e. col("a") + 100. + * In this case the 100 is a [[org.apache.spark.sql.catalyst.expressions.Literal]] that + * [[org.apache.spark.sql.catalyst.expressions.Add]] would have to be able to handle. + * + * By convention any [[org.apache.spark.sql.vectorized.ColumnVector]] returned by [[columnarEval]] + * is owned by the caller and will need to be closed by them. This can happen by putting it into + * a [[org.apache.spark.sql.vectorized.ColumnarBatch]] and closing the batch or by closing the + * vector directly if it is a temporary value. + */ + def columnarEval(batch: ColumnarBatch): Any = { + throw new IllegalStateException(s"Internal Error ${this.getClass} has column support mismatch") + } + + // We need to override equals because we are subclassing a case class + override def equals(other: Any): Boolean = { + if (!super.equals(other)) { + return false + } + return other.isInstanceOf[ColumnarExpression] + } + + override def hashCode(): Int = super.hashCode() +} + +object ColumnarBindReferences extends Logging { + + // Mostly copied from BoundAttribute.scala so we can do columnar processing + def bindReference[A <: ColumnarExpression]( + expression: A, + input: AttributeSeq, + allowFailures: Boolean = false): A = { + expression.transform { case a: AttributeReference => + val ordinal = input.indexOf(a.exprId) + if (ordinal == -1) { + if (allowFailures) { + a + } else { + sys.error(s"Couldn't find $a in ${input.attrs.mkString("[", ",", "]")}") + } + } else { + new ColumnarBoundReference(ordinal, a.dataType, input(ordinal).nullable) + } + }.asInstanceOf[A] + } + + /** + * A helper function to bind given expressions to an input schema. + */ + def bindReferences[A <: ColumnarExpression]( + expressions: Seq[A], + input: AttributeSeq): Seq[A] = { + expressions.map(ColumnarBindReferences.bindReference(_, input)) + } +} + +class ColumnarBoundReference(ordinal: Int, dataType: DataType, nullable: Boolean) + extends BoundReference(ordinal, dataType, nullable) with ColumnarExpression { + + override def columnarEval(batch: ColumnarBatch): Any = { + // Because of the convention that the returned ColumnVector must be closed by the + // caller we wrap this column vector so a close is a NOOP, and let the original source + // of the vector close it. + NoCloseColumnVector.wrapIfNeeded(batch.column(ordinal)) + } +} + +class ColumnarAlias(child: ColumnarExpression, name: String)( + override val exprId: ExprId = NamedExpression.newExprId, + override val qualifier: Seq[String] = Seq.empty, + override val explicitMetadata: Option[Metadata] = None) + extends Alias(child, name)(exprId, qualifier, explicitMetadata) + with ColumnarExpression { + + override def columnarEval(batch: ColumnarBatch): Any = child.columnarEval(batch) +} + +class ColumnarAttributeReference( + name: String, + dataType: DataType, + nullable: Boolean = true, + override val metadata: Metadata = Metadata.empty)( + override val exprId: ExprId = NamedExpression.newExprId, + override val qualifier: Seq[String] = Seq.empty[String]) + extends AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier) + with ColumnarExpression { + + // No columnar eval is needed because this must be bound before it is evaluated +} + +class ColumnarLiteral (value: Any, dataType: DataType) extends Literal(value, dataType) + with ColumnarExpression { + override def columnarEval(batch: ColumnarBatch): Any = value +} + +/** + * A version of ProjectExec that adds in columnar support. + */ +class ColumnarProjectExec(projectList: Seq[NamedExpression], child: SparkPlan) + extends ProjectExec(projectList, child) { + + override def supportsColumnar: Boolean = + projectList.forall(_.asInstanceOf[ColumnarExpression].supportsColumnar) + + // Disable code generation + override def supportCodegen: Boolean = false + + override def doExecuteColumnar() : RDD[ColumnarBatch] = { + val boundProjectList: Seq[Any] = + ColumnarBindReferences.bindReferences( + projectList.asInstanceOf[Seq[ColumnarExpression]], child.output) + val rdd = child.executeColumnar() + rdd.mapPartitions((itr) => CloseableColumnBatchIterator(itr, + (cb) => { + val newColumns = boundProjectList.map( + expr => expr.asInstanceOf[ColumnarExpression].columnarEval(cb).asInstanceOf[ColumnVector] + ).toArray + new ColumnarBatch(newColumns, cb.numRows()) + }) + ) + } + + // We have to override equals because subclassing a case class like ProjectExec is not that clean + // One of the issues is that the generated equals will see ColumnarProjectExec and ProjectExec + // as being equal and this can result in the withNewChildren method not actually replacing + // anything + override def equals(other: Any): Boolean = { + if (!super.equals(other)) { + return false + } + return other.isInstanceOf[ColumnarProjectExec] + } + + override def hashCode(): Int = super.hashCode() +} + +/** + * A version of add that supports columnar processing for longs. This version is broken + * on purpose so it adds the numbers plus 1 so that the tests can show that it was replaced. + */ +class BrokenColumnarAdd(left: ColumnarExpression, right: ColumnarExpression) + extends Add(left, right) with ColumnarExpression { + + override def supportsColumnar(): Boolean = left.supportsColumnar && right.supportsColumnar + + override def columnarEval(batch: ColumnarBatch): Any = { + var lhs: Any = null + var rhs: Any = null + var ret: Any = null + try { + lhs = left.columnarEval(batch) + rhs = right.columnarEval(batch) + + if (lhs == null || rhs == null) { + ret = null + } else if (lhs.isInstanceOf[ColumnVector] && rhs.isInstanceOf[ColumnVector]) { + val l = lhs.asInstanceOf[ColumnVector] + val r = rhs.asInstanceOf[ColumnVector] + val result = new OnHeapColumnVector(batch.numRows(), dataType) + ret = result + + for (i <- 0 until batch.numRows()) { + result.appendLong(l.getLong(i) + r.getLong(i) + 1) // BUG to show we replaced Add + } + } else if (rhs.isInstanceOf[ColumnVector]) { + val l = lhs.asInstanceOf[Long] + val r = rhs.asInstanceOf[ColumnVector] + val result = new OnHeapColumnVector(batch.numRows(), dataType) + ret = result + + for (i <- 0 until batch.numRows()) { + result.appendLong(l + r.getLong(i) + 1) // BUG to show we replaced Add + } + } else if (lhs.isInstanceOf[ColumnVector]) { + val l = lhs.asInstanceOf[ColumnVector] + val r = rhs.asInstanceOf[Long] + val result = new OnHeapColumnVector(batch.numRows(), dataType) + ret = result + + for (i <- 0 until batch.numRows()) { + result.appendLong(l.getLong(i) + r + 1) // BUG to show we replaced Add + } + } else { + ret = nullSafeEval(lhs, rhs) + } + } finally { + if (lhs != null && lhs.isInstanceOf[ColumnVector]) { + lhs.asInstanceOf[ColumnVector].close() + } + if (rhs != null && rhs.isInstanceOf[ColumnVector]) { + rhs.asInstanceOf[ColumnVector].close() + } + } + ret + } +} + +class CannotReplaceException(str: String) extends RuntimeException(str) { + +} + +case class PreRuleReplaceAddWithBrokenVersion() extends Rule[SparkPlan] { + def replaceWithColumnarExpression(exp: Expression): ColumnarExpression = exp match { + case a: Alias => + new ColumnarAlias(replaceWithColumnarExpression(a.child), + a.name)(a.exprId, a.qualifier, a.explicitMetadata) + case att: AttributeReference => + new ColumnarAttributeReference(att.name, att.dataType, att.nullable, + att.metadata)(att.exprId, att.qualifier) + case lit: Literal => + new ColumnarLiteral(lit.value, lit.dataType) + case add: Add if (add.dataType == LongType) && + (add.left.dataType == LongType) && + (add.right.dataType == LongType) => + // Add only supports Longs for now. + new BrokenColumnarAdd(replaceWithColumnarExpression(add.left), + replaceWithColumnarExpression(add.right)) + case exp => + throw new CannotReplaceException(s"expression " + + s"${exp.getClass} ${exp} is not currently supported.") + } + + def replaceWithColumnarPlan(plan: SparkPlan): SparkPlan = + try { + plan match { + case plan: ProjectExec => + new ColumnarProjectExec(plan.projectList.map((exp) => + replaceWithColumnarExpression(exp).asInstanceOf[NamedExpression]), + replaceWithColumnarPlan(plan.child)) + case p => + logWarning(s"Columnar processing for ${p.getClass} is not currently supported.") + p.withNewChildren(p.children.map(replaceWithColumnarPlan)) + } + } catch { + case exp: CannotReplaceException => + logWarning(s"Columnar processing for ${plan.getClass} is not currently supported" + + s"because ${exp.getMessage}") + plan + } + + override def apply(plan: SparkPlan): SparkPlan = replaceWithColumnarPlan(plan) +} + +class ReplacedRowToColumnarExec(override val child: SparkPlan) + extends RowToColumnarExec(child) { + + // We have to override equals because subclassing a case class like ProjectExec is not that clean + // One of the issues is that the generated equals will see ColumnarProjectExec and ProjectExec + // as being equal and this can result in the withNewChildren method not actually replacing + // anything + override def equals(other: Any): Boolean = { + if (!super.equals(other)) { + return false + } + return other.isInstanceOf[ReplacedRowToColumnarExec] + } + + override def hashCode(): Int = super.hashCode() +} + +case class MyPostRule() extends Rule[SparkPlan] { + override def apply(plan: SparkPlan): SparkPlan = plan match { + case rc: RowToColumnarExec => new ReplacedRowToColumnarExec(rc.child) + case plan => plan.withNewChildren(plan.children.map(apply)) + } +} + +case class MyColumarRule(pre: Rule[SparkPlan], post: Rule[SparkPlan]) extends ColumnarRule { + override def preColumnarTransitions: Rule[SparkPlan] = pre + override def postColumnarTransitions: Rule[SparkPlan] = post +} + class MyExtensions extends (SparkSessionExtensions => Unit) { def apply(e: SparkSessionExtensions): Unit = { e.injectPlannerStrategy(MySparkStrategy) e.injectResolutionRule(MyRule) + e.injectPostHocResolutionRule(MyRule) + e.injectCheckRule(MyCheckRule) + e.injectOptimizerRule(MyRule) + e.injectParser(MyParser) + e.injectFunction(MyExtensions.myFunction) + e.injectColumnar(session => MyColumarRule(PreRuleReplaceAddWithBrokenVersion(), MyPostRule())) + } +} + +case class MyRule2(spark: SparkSession) extends Rule[LogicalPlan] { + override def apply(plan: LogicalPlan): LogicalPlan = plan +} + +case class MyCheckRule2(spark: SparkSession) extends (LogicalPlan => Unit) { + override def apply(plan: LogicalPlan): Unit = { } +} + +case class MySparkStrategy2(spark: SparkSession) extends SparkStrategy { + override def apply(plan: LogicalPlan): Seq[SparkPlan] = Seq.empty +} + +object MyExtensions2 { + + val myFunction = (FunctionIdentifier("myFunction2"), + new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), + (_: Seq[Expression]) => Literal(5, IntegerType)) +} + +class MyExtensions2 extends (SparkSessionExtensions => Unit) { + def apply(e: SparkSessionExtensions): Unit = { + e.injectPlannerStrategy(MySparkStrategy2) + e.injectResolutionRule(MyRule2) + e.injectPostHocResolutionRule(MyRule2) + e.injectCheckRule(MyCheckRule2) + e.injectOptimizerRule(MyRule2) + e.injectParser((_: SparkSession, _: ParserInterface) => CatalystSqlParser) + e.injectFunction(MyExtensions2.myFunction) + } +} + +object MyExtensions2Duplicate { + + val myFunction = (FunctionIdentifier("myFunction2"), + new ExpressionInfo("noClass", "myDb", "myFunction2", "usage", "extended usage"), + (_: Seq[Expression]) => Literal(5, IntegerType)) +} + +class MyExtensions2Duplicate extends (SparkSessionExtensions => Unit) { + def apply(e: SparkSessionExtensions): Unit = { + e.injectFunction(MyExtensions2Duplicate.myFunction) } } 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 cb562d65b6147..4c78f85a3ac69 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 @@ -17,13 +17,18 @@ package org.apache.spark.sql -import java.io.File +import java.io.{File, PrintWriter} +import java.net.URI +import java.util.TimeZone +import java.util.concurrent.TimeUnit import scala.collection.mutable import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.CatalogColumnStat import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.util.{DateTimeTestUtils, DateTimeUtils} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData.ArrayData @@ -227,18 +232,17 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared BigInt(0) -> (("0.0 B", "0")), BigInt(100) -> (("100.0 B", "100")), BigInt(2047) -> (("2047.0 B", "2.05E+3")), - BigInt(2048) -> (("2.0 KB", "2.05E+3")), - BigInt(3333333) -> (("3.2 MB", "3.33E+6")), - BigInt(4444444444L) -> (("4.1 GB", "4.44E+9")), - BigInt(5555555555555L) -> (("5.1 TB", "5.56E+12")), - BigInt(6666666666666666L) -> (("5.9 PB", "6.67E+15")), - BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EB", "1.18E+21")), + BigInt(2048) -> (("2.0 KiB", "2.05E+3")), + BigInt(3333333) -> (("3.2 MiB", "3.33E+6")), + BigInt(4444444444L) -> (("4.1 GiB", "4.44E+9")), + BigInt(5555555555555L) -> (("5.1 TiB", "5.56E+12")), + BigInt(6666666666666666L) -> (("5.9 PiB", "6.67E+15")), + BigInt(1L << 10 ) * (1L << 60) -> (("1024.0 EiB", "1.18E+21")), BigInt(1L << 11) * (1L << 60) -> (("2.36E+21 B", "2.36E+21")) ) numbers.foreach { case (input, (expectedSize, expectedRows)) => val stats = Statistics(sizeInBytes = input, rowCount = Some(input)) - val expectedString = s"sizeInBytes=$expectedSize, rowCount=$expectedRows," + - s" hints=none" + val expectedString = s"sizeInBytes=$expectedSize, rowCount=$expectedRows" assert(stats.simpleString == expectedString) } } @@ -334,6 +338,26 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } + test("auto gather stats after insert command") { + val table = "change_stats_insert_datasource_table" + Seq(false, true).foreach { autoUpdate => + withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> autoUpdate.toString) { + withTable(table) { + sql(s"CREATE TABLE $table (i int, j string) USING PARQUET") + // insert into command + sql(s"INSERT INTO TABLE $table SELECT 1, 'abc'") + val stats = getCatalogTable(table).stats + if (autoUpdate) { + assert(stats.isDefined) + assert(stats.get.sizeInBytes >= 0) + } else { + assert(stats.isEmpty) + } + } + } + } + } + test("invalidation of tableRelationCache after inserts") { val table = "invalidate_catalog_cache_table" Seq(false, true).foreach { autoUpdate => @@ -428,4 +452,202 @@ class StatisticsCollectionSuite extends StatisticsCollectionTestBase with Shared } } } + + test("store and retrieve column stats in different time zones") { + val (start, end) = (0, TimeUnit.DAYS.toSeconds(2)) + + def checkTimestampStats( + t: DataType, + srcTimeZone: TimeZone, + dstTimeZone: TimeZone)(checker: ColumnStat => Unit): Unit = { + val table = "time_table" + val column = "T" + val original = TimeZone.getDefault + try { + withTable(table) { + TimeZone.setDefault(srcTimeZone) + spark.range(start, end) + .select('id.cast(TimestampType).cast(t).as(column)) + .write.saveAsTable(table) + sql(s"ANALYZE TABLE $table COMPUTE STATISTICS FOR COLUMNS $column") + + TimeZone.setDefault(dstTimeZone) + val stats = getCatalogTable(table) + .stats.get.colStats(column).toPlanStat(column, t) + checker(stats) + } + } finally { + TimeZone.setDefault(original) + } + } + + DateTimeTestUtils.outstandingTimezones.foreach { timeZone => + 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)) + } + checkTimestampStats(TimestampType, DateTimeUtils.TimeZoneUTC, timeZone) { stats => + assert(stats.min.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(start)) + assert(stats.max.get.asInstanceOf[Long] == TimeUnit.SECONDS.toMicros(end - 1)) + } + } + } + + def getStatAttrNames(tableName: String): Set[String] = { + val queryStats = spark.table(tableName).queryExecution.optimizedPlan.stats.attributeStats + queryStats.map(_._1.name).toSet + } + + test("analyzes column statistics in cached query") { + withTempView("cachedQuery") { + sql( + """CACHE TABLE cachedQuery AS + | SELECT c0, avg(c1) AS v1, avg(c2) AS v2 + | FROM (SELECT id % 3 AS c0, id % 5 AS c1, 2 AS c2 FROM range(1, 30)) + | GROUP BY c0 + """.stripMargin) + + // Analyzes one column in the cached logical plan + sql("ANALYZE TABLE cachedQuery COMPUTE STATISTICS FOR COLUMNS v1") + assert(getStatAttrNames("cachedQuery") === Set("v1")) + + // Analyzes two more columns + sql("ANALYZE TABLE cachedQuery COMPUTE STATISTICS FOR COLUMNS c0, v2") + assert(getStatAttrNames("cachedQuery") === Set("c0", "v1", "v2")) + } + } + + test("analyzes column statistics in cached local temporary view") { + withTempView("tempView") { + // Analyzes in a temporary view + sql("CREATE TEMPORARY VIEW tempView AS SELECT * FROM range(1, 30)") + val errMsg = intercept[AnalysisException] { + sql("ANALYZE TABLE tempView COMPUTE STATISTICS FOR COLUMNS id") + }.getMessage + assert(errMsg.contains(s"Table or view 'tempView' not found in database 'default'")) + + // Cache the view then analyze it + sql("CACHE TABLE tempView") + assert(getStatAttrNames("tempView") !== Set("id")) + sql("ANALYZE TABLE tempView COMPUTE STATISTICS FOR COLUMNS id") + assert(getStatAttrNames("tempView") === Set("id")) + } + } + + test("analyzes column statistics in cached global temporary view") { + withGlobalTempView("gTempView") { + val globalTempDB = spark.sharedState.globalTempViewManager.database + val errMsg1 = intercept[NoSuchTableException] { + sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") + }.getMessage + assert(errMsg1.contains(s"Table or view 'gTempView' not found in database '$globalTempDB'")) + // Analyzes in a global temporary view + sql("CREATE GLOBAL TEMP VIEW gTempView AS SELECT * FROM range(1, 30)") + val errMsg2 = intercept[AnalysisException] { + sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") + }.getMessage + assert(errMsg2.contains(s"Table or view 'gTempView' not found in database '$globalTempDB'")) + + // Cache the view then analyze it + sql(s"CACHE TABLE $globalTempDB.gTempView") + assert(getStatAttrNames(s"$globalTempDB.gTempView") !== Set("id")) + sql(s"ANALYZE TABLE $globalTempDB.gTempView COMPUTE STATISTICS FOR COLUMNS id") + assert(getStatAttrNames(s"$globalTempDB.gTempView") === Set("id")) + } + } + + test("analyzes column statistics in cached catalog view") { + withTempDatabase { database => + sql(s"CREATE VIEW $database.v AS SELECT 1 c") + sql(s"CACHE TABLE $database.v") + assert(getStatAttrNames(s"$database.v") !== Set("c")) + sql(s"ANALYZE TABLE $database.v COMPUTE STATISTICS FOR COLUMNS c") + assert(getStatAttrNames(s"$database.v") === Set("c")) + } + } + + test("analyzes table statistics in cached catalog view") { + def getTableStats(tableName: String): Statistics = { + spark.table(tableName).queryExecution.optimizedPlan.stats + } + + withTempDatabase { database => + sql(s"CREATE VIEW $database.v AS SELECT 1 c") + // Cache data eagerly by default, so this operation collects table stats + sql(s"CACHE TABLE $database.v") + val stats1 = getTableStats(s"$database.v") + assert(stats1.sizeInBytes > 0) + assert(stats1.rowCount === Some(1)) + sql(s"UNCACHE TABLE $database.v") + + // Cache data lazily, then analyze table stats + sql(s"CACHE LAZY TABLE $database.v") + val stats2 = getTableStats(s"$database.v") + assert(stats2.sizeInBytes === OneRowRelation().computeStats().sizeInBytes) + assert(stats2.rowCount === None) + + sql(s"ANALYZE TABLE $database.v COMPUTE STATISTICS NOSCAN") + val stats3 = getTableStats(s"$database.v") + assert(stats3.sizeInBytes === OneRowRelation().computeStats().sizeInBytes) + assert(stats3.rowCount === None) + + sql(s"ANALYZE TABLE $database.v COMPUTE STATISTICS") + val stats4 = getTableStats(s"$database.v") + assert(stats4.sizeInBytes === stats1.sizeInBytes) + assert(stats4.rowCount === Some(1)) + } + } + + test(s"CTAS should update statistics if ${SQLConf.AUTO_SIZE_UPDATE_ENABLED.key} is enabled") { + val tableName = "spark_27694" + Seq(false, true).foreach { updateEnabled => + withSQLConf(SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> updateEnabled.toString) { + withTable(tableName) { + // Create a data source table using the result of a query. + sql(s"CREATE TABLE $tableName USING parquet AS SELECT 'a', 'b'") + val catalogTable = getCatalogTable(tableName) + if (updateEnabled) { + assert(catalogTable.stats.nonEmpty) + } else { + assert(catalogTable.stats.isEmpty) + } + } + } + } + } + + test("Metadata files and temporary files should not be counted as data files") { + withTempDir { tempDir => + val tableName = "t1" + val stagingDirName = ".test-staging-dir" + val tableLocation = s"${tempDir.toURI}/$tableName" + withSQLConf( + SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> "true", + "hive.exec.stagingdir" -> stagingDirName) { + withTable("t1") { + sql(s"CREATE TABLE $tableName(c1 BIGINT) USING PARQUET LOCATION '$tableLocation'") + sql(s"INSERT INTO TABLE $tableName VALUES(1)") + + val staging = new File(new URI(s"$tableLocation/$stagingDirName")) + Utils.tryWithResource(new PrintWriter(staging)) { stagingWriter => + stagingWriter.write("12") + } + + val metadata = new File(new URI(s"$tableLocation/_metadata")) + Utils.tryWithResource(new PrintWriter(metadata)) { metadataWriter => + metadataWriter.write("1234") + } + + sql(s"INSERT INTO TABLE $tableName VALUES(1)") + + val stagingFileSize = staging.length() + val metadataFileSize = metadata.length() + val tableLocationSize = getDataSize(new File(new URI(tableLocation))) + + val stats = checkTableStats(tableName, hasSizeInBytes = true, expectedRowCounts = None) + assert(stats.get.sizeInBytes === tableLocationSize - stagingFileSize - metadataFileSize) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala index bf4abb6e625c8..0a2abdf4233ed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StatisticsCollectionTestBase.scala @@ -18,7 +18,9 @@ package org.apache.spark.sql import java.{lang => jl} +import java.io.File import java.sql.{Date, Timestamp} +import java.util.concurrent.TimeUnit import scala.collection.mutable import scala.util.Random @@ -26,12 +28,10 @@ import scala.util.Random import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog.{CatalogColumnStat, CatalogStatistics, CatalogTable, HiveTableRelation} import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Histogram, HistogramBin, HistogramSerializer, LogicalPlan} -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.DateTimeTestUtils._ import org.apache.spark.sql.execution.datasources.LogicalRelation import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.types.Decimal - /** * The base for statistics test cases that we want to include in both the hive module (for @@ -42,14 +42,18 @@ abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils private val dec1 = new java.math.BigDecimal("1.000000000000000000") private val dec2 = new java.math.BigDecimal("8.000000000000000000") - private val d1 = Date.valueOf("2016-05-08") - private val d2 = Date.valueOf("2016-05-09") - private val t1 = Timestamp.valueOf("2016-05-08 00:00:01") - private val t2 = Timestamp.valueOf("2016-05-09 00:00:02") - private val d1Internal = DateTimeUtils.fromJavaDate(d1) - private val d2Internal = DateTimeUtils.fromJavaDate(d2) - private val t1Internal = DateTimeUtils.fromJavaTimestamp(t1) - private val t2Internal = DateTimeUtils.fromJavaTimestamp(t2) + private val d1Str = "2016-05-08" + private val d1Internal = days(2016, 5, 8) + private val d1 = Date.valueOf(d1Str) + private val d2Str = "2016-05-09" + private val d2Internal = days(2016, 5, 9) + private val d2 = Date.valueOf(d2Str) + private val t1Str = "2016-05-08 00:00:01.000000" + private val t1Internal = date(2016, 5, 8, 0, 0, 1) + private val t1 = new Timestamp(TimeUnit.MICROSECONDS.toMillis(t1Internal)) + private val t2Str = "2016-05-09 00:00:02.000000" + private val t2Internal = date(2016, 5, 9, 0, 0, 2) + private val t2 = new Timestamp(TimeUnit.MICROSECONDS.toMillis(t2Internal)) /** * Define a very simple 3 row table used for testing column serialization. @@ -78,10 +82,10 @@ abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils Some(16), Some(16)), "cstring" -> CatalogColumnStat(Some(2), None, None, Some(1), Some(3), Some(3)), "cbinary" -> CatalogColumnStat(Some(2), None, None, Some(1), Some(3), Some(3)), - "cdate" -> CatalogColumnStat(Some(2), Some(d1.toString), Some(d2.toString), Some(1), Some(4), - Some(4)), - "ctimestamp" -> CatalogColumnStat(Some(2), Some(t1.toString), Some(t2.toString), Some(1), - Some(8), Some(8)) + "cdate" -> CatalogColumnStat(Some(2), Some(d1Str), Some(d2Str), + Some(1), Some(4), Some(4)), + "ctimestamp" -> CatalogColumnStat(Some(2), Some(t1Str), + Some(t2Str), Some(1), Some(8), Some(8)) ) /** @@ -113,87 +117,88 @@ abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils colStats } + private val strVersion = CatalogColumnStat.VERSION.toString val expectedSerializedColStats = Map( "spark.sql.statistics.colStats.cbinary.avgLen" -> "3", "spark.sql.statistics.colStats.cbinary.distinctCount" -> "2", "spark.sql.statistics.colStats.cbinary.maxLen" -> "3", "spark.sql.statistics.colStats.cbinary.nullCount" -> "1", - "spark.sql.statistics.colStats.cbinary.version" -> "1", + "spark.sql.statistics.colStats.cbinary.version" -> strVersion, "spark.sql.statistics.colStats.cbool.avgLen" -> "1", "spark.sql.statistics.colStats.cbool.distinctCount" -> "2", "spark.sql.statistics.colStats.cbool.max" -> "true", "spark.sql.statistics.colStats.cbool.maxLen" -> "1", "spark.sql.statistics.colStats.cbool.min" -> "false", "spark.sql.statistics.colStats.cbool.nullCount" -> "1", - "spark.sql.statistics.colStats.cbool.version" -> "1", + "spark.sql.statistics.colStats.cbool.version" -> strVersion, "spark.sql.statistics.colStats.cbyte.avgLen" -> "1", "spark.sql.statistics.colStats.cbyte.distinctCount" -> "2", "spark.sql.statistics.colStats.cbyte.max" -> "2", "spark.sql.statistics.colStats.cbyte.maxLen" -> "1", "spark.sql.statistics.colStats.cbyte.min" -> "1", "spark.sql.statistics.colStats.cbyte.nullCount" -> "1", - "spark.sql.statistics.colStats.cbyte.version" -> "1", + "spark.sql.statistics.colStats.cbyte.version" -> strVersion, "spark.sql.statistics.colStats.cdate.avgLen" -> "4", "spark.sql.statistics.colStats.cdate.distinctCount" -> "2", "spark.sql.statistics.colStats.cdate.max" -> "2016-05-09", "spark.sql.statistics.colStats.cdate.maxLen" -> "4", "spark.sql.statistics.colStats.cdate.min" -> "2016-05-08", "spark.sql.statistics.colStats.cdate.nullCount" -> "1", - "spark.sql.statistics.colStats.cdate.version" -> "1", + "spark.sql.statistics.colStats.cdate.version" -> strVersion, "spark.sql.statistics.colStats.cdecimal.avgLen" -> "16", "spark.sql.statistics.colStats.cdecimal.distinctCount" -> "2", "spark.sql.statistics.colStats.cdecimal.max" -> "8.000000000000000000", "spark.sql.statistics.colStats.cdecimal.maxLen" -> "16", "spark.sql.statistics.colStats.cdecimal.min" -> "1.000000000000000000", "spark.sql.statistics.colStats.cdecimal.nullCount" -> "1", - "spark.sql.statistics.colStats.cdecimal.version" -> "1", + "spark.sql.statistics.colStats.cdecimal.version" -> strVersion, "spark.sql.statistics.colStats.cdouble.avgLen" -> "8", "spark.sql.statistics.colStats.cdouble.distinctCount" -> "2", "spark.sql.statistics.colStats.cdouble.max" -> "6.0", "spark.sql.statistics.colStats.cdouble.maxLen" -> "8", "spark.sql.statistics.colStats.cdouble.min" -> "1.0", "spark.sql.statistics.colStats.cdouble.nullCount" -> "1", - "spark.sql.statistics.colStats.cdouble.version" -> "1", + "spark.sql.statistics.colStats.cdouble.version" -> strVersion, "spark.sql.statistics.colStats.cfloat.avgLen" -> "4", "spark.sql.statistics.colStats.cfloat.distinctCount" -> "2", "spark.sql.statistics.colStats.cfloat.max" -> "7.0", "spark.sql.statistics.colStats.cfloat.maxLen" -> "4", "spark.sql.statistics.colStats.cfloat.min" -> "1.0", "spark.sql.statistics.colStats.cfloat.nullCount" -> "1", - "spark.sql.statistics.colStats.cfloat.version" -> "1", + "spark.sql.statistics.colStats.cfloat.version" -> strVersion, "spark.sql.statistics.colStats.cint.avgLen" -> "4", "spark.sql.statistics.colStats.cint.distinctCount" -> "2", "spark.sql.statistics.colStats.cint.max" -> "4", "spark.sql.statistics.colStats.cint.maxLen" -> "4", "spark.sql.statistics.colStats.cint.min" -> "1", "spark.sql.statistics.colStats.cint.nullCount" -> "1", - "spark.sql.statistics.colStats.cint.version" -> "1", + "spark.sql.statistics.colStats.cint.version" -> strVersion, "spark.sql.statistics.colStats.clong.avgLen" -> "8", "spark.sql.statistics.colStats.clong.distinctCount" -> "2", "spark.sql.statistics.colStats.clong.max" -> "5", "spark.sql.statistics.colStats.clong.maxLen" -> "8", "spark.sql.statistics.colStats.clong.min" -> "1", "spark.sql.statistics.colStats.clong.nullCount" -> "1", - "spark.sql.statistics.colStats.clong.version" -> "1", + "spark.sql.statistics.colStats.clong.version" -> strVersion, "spark.sql.statistics.colStats.cshort.avgLen" -> "2", "spark.sql.statistics.colStats.cshort.distinctCount" -> "2", "spark.sql.statistics.colStats.cshort.max" -> "3", "spark.sql.statistics.colStats.cshort.maxLen" -> "2", "spark.sql.statistics.colStats.cshort.min" -> "1", "spark.sql.statistics.colStats.cshort.nullCount" -> "1", - "spark.sql.statistics.colStats.cshort.version" -> "1", + "spark.sql.statistics.colStats.cshort.version" -> strVersion, "spark.sql.statistics.colStats.cstring.avgLen" -> "3", "spark.sql.statistics.colStats.cstring.distinctCount" -> "2", "spark.sql.statistics.colStats.cstring.maxLen" -> "3", "spark.sql.statistics.colStats.cstring.nullCount" -> "1", - "spark.sql.statistics.colStats.cstring.version" -> "1", + "spark.sql.statistics.colStats.cstring.version" -> strVersion, "spark.sql.statistics.colStats.ctimestamp.avgLen" -> "8", "spark.sql.statistics.colStats.ctimestamp.distinctCount" -> "2", - "spark.sql.statistics.colStats.ctimestamp.max" -> "2016-05-09 00:00:02.0", + "spark.sql.statistics.colStats.ctimestamp.max" -> "2016-05-09 00:00:02.000000", "spark.sql.statistics.colStats.ctimestamp.maxLen" -> "8", - "spark.sql.statistics.colStats.ctimestamp.min" -> "2016-05-08 00:00:01.0", + "spark.sql.statistics.colStats.ctimestamp.min" -> "2016-05-08 00:00:01.000000", "spark.sql.statistics.colStats.ctimestamp.nullCount" -> "1", - "spark.sql.statistics.colStats.ctimestamp.version" -> "1" + "spark.sql.statistics.colStats.ctimestamp.version" -> strVersion ) val expectedSerializedHistograms = Map( @@ -290,6 +295,9 @@ abstract class StatisticsCollectionTestBase extends QueryTest with SQLTestUtils } } + def getDataSize(file: File): Long = + file.listFiles.filter(!_.getName.endsWith(".crc")).map(_.length).sum + // This test will be run twice: with and without Hive support test("SPARK-18856: non-empty partitioned table should not report zero size") { withTable("ds_tbl", "hive_tbl") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala index bb19fde2b2b5f..cc5af327cf599 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/StringFunctionsSuite.scala @@ -129,6 +129,18 @@ class StringFunctionsSuite extends QueryTest with SharedSQLContext { Row("AQIDBA==", bytes)) } + test("overlay function") { + // scalastyle:off + // non ascii characters are not allowed in the code, so we disable the scalastyle here. + val df = Seq(("Spark SQL", "Spark的SQL")).toDF("a", "b") + checkAnswer(df.select(overlay($"a", "_", 6)), Row("Spark_SQL")) + checkAnswer(df.select(overlay($"a", "CORE", 7)), Row("Spark CORE")) + checkAnswer(df.select(overlay($"a", "ANSI ", 7, 0)), Row("Spark ANSI SQL")) + checkAnswer(df.select(overlay($"a", "tructured", 2, 4)), Row("Structured SQL")) + checkAnswer(df.select(overlay($"b", "_", 6)), Row("Spark_SQL")) + // scalastyle:on + } + test("string / binary substring function") { // scalastyle:off // non ascii characters are not allowed in the code, so we disable the scalastyle here. 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 cbffed994bb4f..b2c38684071dc 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 @@ -21,6 +21,9 @@ 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.datasources.FileScanRDD +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class SubquerySuite extends QueryTest with SharedSQLContext { @@ -1186,7 +1189,7 @@ class SubquerySuite extends QueryTest with SharedSQLContext { } } - test("SPARK-23957 Remove redundant sort from subquery plan(scalar subquery)") { + ignore("SPARK-23957 Remove redundant sort from subquery plan(scalar subquery)") { withTempView("t1", "t2", "t3") { Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t1") Seq((1, 1), (2, 2)).toDF("c1", "c2").createOrReplaceTempView("t2") @@ -1268,4 +1271,117 @@ class SubquerySuite extends QueryTest with SharedSQLContext { assert(getNumSortsInQuery(query5) == 1) } } + + test("SPARK-25482: Forbid pushdown to datasources of filters containing subqueries") { + withTempView("t1", "t2") { + sql("create temporary view t1(a int) using parquet") + sql("create temporary view t2(b int) using parquet") + val plan = sql("select * from t2 where b > (select max(a) from t1)") + val subqueries = plan.queryExecution.executedPlan.collect { + case p => p.subqueries + }.flatten + assert(subqueries.length == 1) + } + } + + test("SPARK-26893: Allow pushdown of partition pruning subquery filters to file source") { + withTable("a", "b") { + spark.range(4).selectExpr("id", "id % 2 AS p").write.partitionBy("p").saveAsTable("a") + spark.range(2).write.saveAsTable("b") + + val df = sql("SELECT * FROM a WHERE p <= (SELECT MIN(id) FROM b)") + checkAnswer(df, Seq(Row(0, 0), Row(2, 0))) + // need to execute the query before we can examine fs.inputRDDs() + assert(df.queryExecution.executedPlan match { + case WholeStageCodegenExec(ColumnarToRowExec(InputAdapter( + fs @ FileSourceScanExec(_, _, _, partitionFilters, _, _, _), _))) => + partitionFilters.exists(ExecSubqueryExpression.hasSubquery) && + fs.inputRDDs().forall( + _.asInstanceOf[FileScanRDD].filePartitions.forall( + _.files.forall(_.filePath.contains("p=0")))) + case _ => false + }) + } + } + + test("SPARK-26078: deduplicate fake self joins for IN subqueries") { + withTempView("a", "b") { + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("a") + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("b") + + val df1 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df1, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + val df2 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id NOT IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + checkAnswer(df2, Seq(Row("b", 1, "a"), Row("b", 1, "b"))) + val df3 = spark.sql( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) OR + |c.id IN (SELECT id FROM b WHERE num = 3) + """.stripMargin) + checkAnswer(df3, Seq(Row("a", 2, "a"), Row("a", 2, "b"))) + } + } + + test("SPARK-27279: Reuse Subquery") { + Seq(true, false).foreach { reuse => + withSQLConf(SQLConf.SUBQUERY_REUSE_ENABLED.key -> reuse.toString) { + val df = sql( + """ + |SELECT (SELECT avg(key) FROM testData) + (SELECT avg(key) FROM testData) + |FROM testData + |LIMIT 1 + """.stripMargin) + + var countSubqueryExec = 0 + var countReuseSubqueryExec = 0 + df.queryExecution.executedPlan.transformAllExpressions { + case s @ ScalarSubquery(_: SubqueryExec, _) => + countSubqueryExec = countSubqueryExec + 1 + s + case s @ ScalarSubquery(_: ReusedSubqueryExec, _) => + countReuseSubqueryExec = countReuseSubqueryExec + 1 + s + } + + if (reuse) { + assert(countSubqueryExec == 1, "Subquery reusing not working correctly") + assert(countReuseSubqueryExec == 1, "Subquery reusing not working correctly") + } else { + assert(countSubqueryExec == 2, "expect 2 SubqueryExec when not reusing") + assert(countReuseSubqueryExec == 0, + "expect 0 ReusedSubqueryExec when not reusing") + } + } + } + } + + test("Scalar subquery name should start with scalar-subquery#") { + val df = sql("SELECT a FROM l WHERE a = (SELECT max(c) FROM r WHERE c = 1)".stripMargin) + var subqueryExecs: ArrayBuffer[SubqueryExec] = ArrayBuffer.empty + df.queryExecution.executedPlan.transformAllExpressions { + case s @ ScalarSubquery(p: SubqueryExec, _) => + subqueryExecs += p + s + } + assert(subqueryExecs.forall(_.name.startsWith("scalar-subquery#")), + "SubqueryExec name should start with scalar-subquery#") + } } 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 30dca9497ddde..058c5ba7e50b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UDFSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.math.BigDecimal + import org.apache.spark.sql.api.java._ import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.QueryExecution @@ -24,9 +26,10 @@ import org.apache.spark.sql.execution.columnar.InMemoryRelation import org.apache.spark.sql.execution.command.{CreateDataSourceTableAsSelectCommand, ExplainCommand} import org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelationCommand import org.apache.spark.sql.functions.{lit, udf} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.test.SQLTestData._ -import org.apache.spark.sql.types.{DataTypes, DoubleType} +import org.apache.spark.sql.types._ import org.apache.spark.sql.util.QueryExecutionListener @@ -313,9 +316,9 @@ class UDFSuite extends QueryTest with SharedSQLContext { val udf2Name = "myUdf2" val udf1 = spark.udf.register(udf1Name, (n: Int) => n + 1) val udf2 = spark.udf.register(udf2Name, (n: Int) => n * 1) - assert(explainStr(sql("SELECT myUdf1(myUdf2(1))")).contains(s"UDF:$udf1Name(UDF:$udf2Name(1))")) + assert(explainStr(sql("SELECT myUdf1(myUdf2(1))")).contains(s"$udf1Name($udf2Name(1))")) assert(explainStr(spark.range(1).select(udf1(udf2(functions.lit(1))))) - .contains(s"UDF:$udf1Name(UDF:$udf2Name(1))")) + .contains(s"$udf1Name($udf2Name(1))")) } test("SPARK-23666 Do not display exprId in argument names") { @@ -326,7 +329,7 @@ class UDFSuite extends QueryTest with SharedSQLContext { Console.withOut(outputStream) { spark.sql("SELECT f(a._1) FROM x").show } - assert(outputStream.toString.contains("UDF:f(a._1 AS `_1`)")) + assert(outputStream.toString.contains("f(a._1 AS `_1`)")) } } @@ -356,10 +359,13 @@ class UDFSuite extends QueryTest with SharedSQLContext { .withColumn("b", udf1($"a", lit(10))) df.cache() df.write.saveAsTable("t") + sparkContext.listenerBus.waitUntilEmpty(1000) assert(numTotalCachedHit == 1, "expected to be cached in saveAsTable") df.write.insertInto("t") + sparkContext.listenerBus.waitUntilEmpty(1000) assert(numTotalCachedHit == 2, "expected to be cached in insertInto") df.write.save(path.getCanonicalPath) + sparkContext.listenerBus.waitUntilEmpty(1000) assert(numTotalCachedHit == 3, "expected to be cached in save for native") } } @@ -393,4 +399,128 @@ class UDFSuite extends QueryTest with SharedSQLContext { checkAnswer(df, Seq(Row("12"), Row("24"), Row("3null"), Row(null))) } } + + test("SPARK-25044 Verify null input handling for primitive types - with udf()") { + val input = Seq( + (null, Integer.valueOf(1), "x"), + ("M", null, "y"), + ("N", Integer.valueOf(3), null)).toDF("a", "b", "c") + + val udf1 = udf((a: String, b: Int, c: Any) => a + b + c) + val df = input.select(udf1('a, 'b, 'c)) + checkAnswer(df, Seq(Row("null1x"), Row(null), Row("N3null"))) + + // test Java UDF. Java UDF can't have primitive inputs, as it's generic typed. + val udf2 = udf(new UDF3[String, Integer, Object, String] { + override def call(t1: String, t2: Integer, t3: Object): String = { + t1 + t2 + t3 + } + }, StringType) + val df2 = input.select(udf2('a, 'b, 'c)) + checkAnswer(df2, Seq(Row("null1x"), Row("Mnully"), Row("N3null"))) + } + + test("SPARK-25044 Verify null input handling for primitive types - with udf.register") { + withTable("t") { + Seq((null, Integer.valueOf(1), "x"), ("M", null, "y"), ("N", Integer.valueOf(3), null)) + .toDF("a", "b", "c").write.format("json").saveAsTable("t") + spark.udf.register("f", (a: String, b: Int, c: Any) => a + b + c) + val df = spark.sql("SELECT f(a, b, c) FROM t") + checkAnswer(df, Seq(Row("null1x"), Row(null), Row("N3null"))) + + // test Java UDF. Java UDF can't have primitive inputs, as it's generic typed. + spark.udf.register("f2", new UDF3[String, Integer, Object, String] { + override def call(t1: String, t2: Integer, t3: Object): String = { + t1 + t2 + t3 + } + }, StringType) + val df2 = spark.sql("SELECT f2(a, b, c) FROM t") + checkAnswer(df2, Seq(Row("null1x"), Row("Mnully"), Row("N3null"))) + } + } + + test("SPARK-25044 Verify null input handling for primitive types - with udf(Any, DataType)") { + val f = udf((x: Int) => x, IntegerType) + checkAnswer( + Seq(new Integer(1), null).toDF("x").select(f($"x")), + Row(1) :: Row(0) :: Nil) + + val f2 = udf((x: Double) => x, DoubleType) + checkAnswer( + Seq(new java.lang.Double(1.1), null).toDF("x").select(f2($"x")), + Row(1.1) :: Row(0.0) :: Nil) + + } + + test("SPARK-26308: udf with decimal") { + val df1 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(new BigDecimal("2011000000000002456556")))), + StructType(Seq(StructField("col1", DecimalType(30, 0))))) + val udf1 = org.apache.spark.sql.functions.udf((value: BigDecimal) => { + if (value == null) null else value.toBigInteger.toString + }) + checkAnswer(df1.select(udf1(df1.col("col1"))), Seq(Row("2011000000000002456556"))) + } + + test("SPARK-26308: udf with complex types of decimal") { + val df1 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(Array(new BigDecimal("2011000000000002456556"))))), + StructType(Seq(StructField("col1", ArrayType(DecimalType(30, 0)))))) + val udf1 = org.apache.spark.sql.functions.udf((arr: Seq[BigDecimal]) => { + arr.map(value => if (value == null) null else value.toBigInteger.toString) + }) + checkAnswer(df1.select(udf1($"col1")), Seq(Row(Array("2011000000000002456556")))) + + val df2 = spark.createDataFrame( + sparkContext.parallelize(Seq(Row(Map("a" -> new BigDecimal("2011000000000002456556"))))), + StructType(Seq(StructField("col1", MapType(StringType, DecimalType(30, 0)))))) + val udf2 = org.apache.spark.sql.functions.udf((map: Map[String, BigDecimal]) => { + map.mapValues(value => if (value == null) null else value.toBigInteger.toString) + }) + checkAnswer(df2.select(udf2($"col1")), Seq(Row(Map("a" -> "2011000000000002456556")))) + } + + test("SPARK-26323 Verify input type check - with udf()") { + val f = udf((x: Long, y: Any) => x) + val df = Seq(1 -> "a", 2 -> "b").toDF("i", "j").select(f($"i", $"j")) + checkAnswer(df, Seq(Row(1L), Row(2L))) + } + + test("SPARK-26323 Verify input type check - with udf.register") { + withTable("t") { + Seq(1 -> "a", 2 -> "b").toDF("i", "j").write.format("json").saveAsTable("t") + spark.udf.register("f", (x: Long, y: Any) => x) + val df = spark.sql("SELECT f(i, j) FROM t") + checkAnswer(df, Seq(Row(1L), Row(2L))) + } + } + + test("Using java.time.Instant in UDF") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val expected = java.time.Instant.parse("2019-02-27T00:00:00Z") + val plusSec = udf((i: java.time.Instant) => i.plusSeconds(1)) + val df = spark.sql("SELECT TIMESTAMP '2019-02-26 23:59:59Z' as t") + .select(plusSec('t)) + assert(df.collect().toSeq === Seq(Row(expected))) + } + } + + test("Using java.time.LocalDate in UDF") { + withSQLConf(SQLConf.DATETIME_JAVA8API_ENABLED.key -> "true") { + val expected = java.time.LocalDate.parse("2019-02-27") + val plusDay = udf((i: java.time.LocalDate) => i.plusDays(1)) + val df = spark.sql("SELECT DATE '2019-02-26' as d") + .select(plusDay('d)) + assert(df.collect().toSeq === Seq(Row(expected))) + } + } + + test("SPARK-28321 0-args Java UDF should not be called only once") { + val nonDeterministicJavaUDF = udf( + new UDF0[Int] { + override def call(): Int = scala.util.Random.nextInt() + }, IntegerType).asNondeterministic() + + assert(spark.range(2).select(nonDeterministicJavaUDF()).distinct().count() == 2) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala index cc8b600efa46a..6628d36ffc702 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/UserDefinedTypeSuite.scala @@ -17,61 +17,17 @@ package org.apache.spark.sql -import scala.beans.{BeanInfo, BeanProperty} - import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{Cast, ExpressionEvalHelper, GenericInternalRow, Literal} -import org.apache.spark.sql.catalyst.util.{ArrayData, GenericArrayData} import org.apache.spark.sql.execution.datasources.parquet.ParquetTest import org.apache.spark.sql.functions._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ -@BeanInfo -private[sql] case class MyLabeledPoint( - @BeanProperty label: Double, - @BeanProperty features: UDT.MyDenseVector) - -// Wrapped in an object to check Scala compatibility. See SPARK-13929 -object UDT { - - @SQLUserDefinedType(udt = classOf[MyDenseVectorUDT]) - private[sql] class MyDenseVector(val data: Array[Double]) extends Serializable { - override def hashCode(): Int = java.util.Arrays.hashCode(data) - - override def equals(other: Any): Boolean = other match { - case v: MyDenseVector => java.util.Arrays.equals(this.data, v.data) - case _ => false - } - - override def toString: String = data.mkString("(", ", ", ")") - } - - private[sql] class MyDenseVectorUDT extends UserDefinedType[MyDenseVector] { - - override def sqlType: DataType = ArrayType(DoubleType, containsNull = false) - - override def serialize(features: MyDenseVector): ArrayData = { - new GenericArrayData(features.data.map(_.asInstanceOf[Any])) - } - - override def deserialize(datum: Any): MyDenseVector = { - datum match { - case data: ArrayData => - new MyDenseVector(data.toDoubleArray()) - } - } - - override def userClass: Class[MyDenseVector] = classOf[MyDenseVector] - - private[spark] override def asNullable: MyDenseVectorUDT = this - - override def hashCode(): Int = getClass.hashCode() - - override def equals(other: Any): Boolean = other.isInstanceOf[MyDenseVectorUDT] - } - +private[sql] case class MyLabeledPoint(label: Double, features: TestUDT.MyDenseVector) { + def getLabel: Double = label + def getFeatures: TestUDT.MyDenseVector = features } // object and classes to test SPARK-19311 @@ -150,12 +106,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT import testImplicits._ private lazy val pointsRDD = Seq( - MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.2, 2.0)))).toDF() + MyLabeledPoint(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0)))).toDF() private lazy val pointsRDD2 = Seq( - MyLabeledPoint(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - MyLabeledPoint(0.0, new UDT.MyDenseVector(Array(0.3, 3.0)))).toDF() + MyLabeledPoint(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + MyLabeledPoint(0.0, new TestUDT.MyDenseVector(Array(0.3, 3.0)))).toDF() test("register user type: MyDenseVector for MyLabeledPoint") { val labels: RDD[Double] = pointsRDD.select('label).rdd.map { case Row(v: Double) => v } @@ -164,16 +120,17 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT assert(labelsArrays.contains(1.0)) assert(labelsArrays.contains(0.0)) - val features: RDD[UDT.MyDenseVector] = - pointsRDD.select('features).rdd.map { case Row(v: UDT.MyDenseVector) => v } - val featuresArrays: Array[UDT.MyDenseVector] = features.collect() + val features: RDD[TestUDT.MyDenseVector] = + pointsRDD.select('features).rdd.map { case Row(v: TestUDT.MyDenseVector) => v } + val featuresArrays: Array[TestUDT.MyDenseVector] = features.collect() assert(featuresArrays.size === 2) - assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.1, 1.0)))) - assert(featuresArrays.contains(new UDT.MyDenseVector(Array(0.2, 2.0)))) + assert(featuresArrays.contains(new TestUDT.MyDenseVector(Array(0.1, 1.0)))) + assert(featuresArrays.contains(new TestUDT.MyDenseVector(Array(0.2, 2.0)))) } test("UDTs and UDFs") { - spark.udf.register("testType", (d: UDT.MyDenseVector) => d.isInstanceOf[UDT.MyDenseVector]) + spark.udf.register("testType", + (d: TestUDT.MyDenseVector) => d.isInstanceOf[TestUDT.MyDenseVector]) pointsRDD.createOrReplaceTempView("points") checkAnswer( sql("SELECT testType(features) from points"), @@ -187,8 +144,8 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( spark.read.parquet(path), Seq( - Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } } @@ -199,17 +156,17 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT checkAnswer( spark.read.parquet(path), Seq( - Row(1.0, new UDT.MyDenseVector(Array(0.1, 1.0))), - Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Row(1.0, new TestUDT.MyDenseVector(Array(0.1, 1.0))), + Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } } // Tests to make sure that all operators correctly convert types on the way out. test("Local UDTs") { - val vec = new UDT.MyDenseVector(Array(0.1, 1.0)) + val vec = new TestUDT.MyDenseVector(Array(0.1, 1.0)) val df = Seq((1, vec)).toDF("int", "vec") - assert(vec === df.collect()(0).getAs[UDT.MyDenseVector](1)) - assert(vec === df.take(1)(0).getAs[UDT.MyDenseVector](1)) + assert(vec === df.collect()(0).getAs[TestUDT.MyDenseVector](1)) + assert(vec === df.take(1)(0).getAs[TestUDT.MyDenseVector](1)) checkAnswer(df.limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) checkAnswer(df.orderBy('int).limit(1).groupBy('int).agg(first('vec)), Row(1, vec)) } @@ -221,14 +178,14 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT ) val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new UDT.MyDenseVectorUDT, false) + StructField("vec", new TestUDT.MyDenseVectorUDT, false) )) val jsonRDD = spark.read.schema(schema).json(data.toDS()) checkAnswer( jsonRDD, - Row(1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: - Row(2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: + Row(1, new TestUDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))) :: + Row(2, new TestUDT.MyDenseVector(Array(2.25, 4.5, 8.75))) :: Nil ) } @@ -241,25 +198,25 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT val schema = StructType(Seq( StructField("id", IntegerType, false), - StructField("vec", new UDT.MyDenseVectorUDT, false) + StructField("vec", new TestUDT.MyDenseVectorUDT, false) )) val jsonDataset = spark.read.schema(schema).json(data.toDS()) - .as[(Int, UDT.MyDenseVector)] + .as[(Int, TestUDT.MyDenseVector)] checkDataset( jsonDataset, - (1, new UDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), - (2, new UDT.MyDenseVector(Array(2.25, 4.5, 8.75))) + (1, new TestUDT.MyDenseVector(Array(1.1, 2.2, 3.3, 4.4))), + (2, new TestUDT.MyDenseVector(Array(2.25, 4.5, 8.75))) ) } test("SPARK-10472 UserDefinedType.typeName") { assert(IntegerType.typeName === "integer") - assert(new UDT.MyDenseVectorUDT().typeName === "mydensevector") + assert(new TestUDT.MyDenseVectorUDT().typeName === "mydensevector") } test("Catalyst type converter null handling for UDTs") { - val udt = new UDT.MyDenseVectorUDT() + val udt = new TestUDT.MyDenseVectorUDT() val toScalaConverter = CatalystTypeConverters.createToScalaConverter(udt) assert(toScalaConverter(null) === null) @@ -305,12 +262,12 @@ class UserDefinedTypeSuite extends QueryTest with SharedSQLContext with ParquetT test("except on UDT") { checkAnswer( pointsRDD.except(pointsRDD2), - Seq(Row(0.0, new UDT.MyDenseVector(Array(0.2, 2.0))))) + Seq(Row(0.0, new TestUDT.MyDenseVector(Array(0.2, 2.0))))) } test("SPARK-23054 Cast UserDefinedType to string") { - val udt = new UDT.MyDenseVectorUDT() - val vector = new UDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) + val udt = new TestUDT.MyDenseVectorUDT() + val vector = new TestUDT.MyDenseVector(Array(1.0, 3.0, 5.0, 7.0, 9.0)) val data = udt.serialize(vector) val ret = Cast(Literal(data, udt), StringType, None) checkEvaluation(ret, "(1.0, 3.0, 5.0, 7.0, 9.0)") diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.scala new file mode 100644 index 0000000000000..4e39df928603f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/BroadcastExchangeSuite.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 + +import java.util.concurrent.{CountDownLatch, TimeUnit} + +import org.apache.spark.SparkException +import org.apache.spark.scheduler._ +import org.apache.spark.sql.execution.exchange.BroadcastExchangeExec +import org.apache.spark.sql.execution.joins.HashedRelation +import org.apache.spark.sql.functions.broadcast +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class BroadcastExchangeSuite extends SparkPlanTest with SharedSQLContext { + + import testImplicits._ + + test("BroadcastExchange should cancel the job group if timeout") { + val startLatch = new CountDownLatch(1) + val endLatch = new CountDownLatch(1) + var jobEvents: Seq[SparkListenerEvent] = Seq.empty[SparkListenerEvent] + spark.sparkContext.addSparkListener(new SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd): Unit = { + jobEvents :+= jobEnd + endLatch.countDown() + } + override def onJobStart(jobStart: SparkListenerJobStart): Unit = { + jobEvents :+= jobStart + startLatch.countDown() + } + }) + + withSQLConf(SQLConf.BROADCAST_TIMEOUT.key -> "0") { + val df = spark.range(100).join(spark.range(15).as[Long].map { x => + Thread.sleep(5000) + x + }).where("id = value") + + // get the exchange physical plan + val hashExchange = df.queryExecution.executedPlan + .collect { case p: BroadcastExchangeExec => p }.head + + // materialize the future and wait for the job being scheduled + hashExchange.prepare() + startLatch.await(5, TimeUnit.SECONDS) + + // check timeout exception is captured by just executing the exchange + val hashEx = intercept[SparkException] { + hashExchange.executeBroadcast[HashedRelation]() + } + assert(hashEx.getMessage.contains("Could not execute broadcast")) + + // wait for cancel is posted and then check the results. + endLatch.await(5, TimeUnit.SECONDS) + assert(jobCancelled()) + } + + def jobCancelled(): Boolean = { + val events = jobEvents.toArray + val hasStart = events(0).isInstanceOf[SparkListenerJobStart] + val hasCancelled = events(1).asInstanceOf[SparkListenerJobEnd].jobResult + .asInstanceOf[JobFailed].exception.getMessage.contains("cancelled job group") + events.length == 2 && hasStart && hasCancelled + } + } + + test("set broadcastTimeout to -1") { + withSQLConf(SQLConf.BROADCAST_TIMEOUT.key -> "-1") { + val df = spark.range(1).toDF() + val joinDF = df.join(broadcast(df), "id") + val broadcastExchangeExec = joinDF.queryExecution.executedPlan + .collect { case p: BroadcastExchangeExec => p } + assert(broadcastExchangeExec.size == 1, "one and only BroadcastExchangeExec") + assert(joinDF.collect().length == 1) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala index 11a1c9a1f9b9c..20e864ab24b71 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/DataSourceScanExecRedactionSuite.scala @@ -19,26 +19,35 @@ package org.apache.spark.sql.execution import org.apache.hadoop.fs.Path import org.apache.spark.SparkConf -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{DataFrame, QueryTest} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext /** - * Suite that tests the redaction of DataSourceScanExec + * Test suite base for testing the redaction of DataSourceScanExec/BatchScanExec. */ -class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext { +abstract class DataSourceScanRedactionTest extends QueryTest with SharedSQLContext { override protected def sparkConf: SparkConf = super.sparkConf - .set("spark.redaction.string.regex", "file:/[\\w_]+") + .set("spark.redaction.string.regex", "file:/[^\\]\\s]+") + + final protected def isIncluded(queryExecution: QueryExecution, msg: String): Boolean = { + queryExecution.toString.contains(msg) || + queryExecution.simpleString.contains(msg) || + queryExecution.stringWithStats.contains(msg) + } + + protected def getRootPath(df: DataFrame): Path test("treeString is redacted") { withTempDir { dir => val basePath = dir.getCanonicalPath - spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) - val df = spark.read.parquet(basePath) + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + val df = spark.read.orc(basePath) - val rootPath = df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - .asInstanceOf[FileSourceScanExec].relation.location.rootPaths.head + val rootPath = getRootPath(df) assert(rootPath.toString.contains(dir.toURI.getPath.stripSuffix("/"))) assert(!df.queryExecution.sparkPlan.treeString(verbose = true).contains(rootPath.getName)) @@ -53,18 +62,24 @@ class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext { assert(df.queryExecution.simpleString.contains(replacement)) } } +} - private def isIncluded(queryExecution: QueryExecution, msg: String): Boolean = { - queryExecution.toString.contains(msg) || - queryExecution.simpleString.contains(msg) || - queryExecution.stringWithStats.contains(msg) - } +/** + * Suite that tests the redaction of DataSourceScanExec + */ +class DataSourceScanExecRedactionSuite extends DataSourceScanRedactionTest { + override protected def sparkConf: SparkConf = super.sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST.key, "orc") + + override protected def getRootPath(df: DataFrame): Path = + df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + .asInstanceOf[FileSourceScanExec].relation.location.rootPaths.head test("explain is redacted using SQLConf") { withTempDir { dir => val basePath = dir.getCanonicalPath - spark.range(0, 10).toDF("a").write.parquet(new Path(basePath, "foo=1").toString) - val df = spark.read.parquet(basePath) + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + val df = spark.read.orc(basePath) val replacement = "*********" // Respect SparkConf and replace file:/ @@ -86,8 +101,8 @@ class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext { test("FileSourceScanExec metadata") { withTempPath { path => val dir = path.getCanonicalPath - spark.range(0, 10).write.parquet(dir) - val df = spark.read.parquet(dir) + spark.range(0, 10).write.orc(dir) + val df = spark.read.orc(dir) assert(isIncluded(df.queryExecution, "Format")) assert(isIncluded(df.queryExecution, "ReadSchema")) @@ -98,5 +113,52 @@ class DataSourceScanExecRedactionSuite extends QueryTest with SharedSQLContext { assert(isIncluded(df.queryExecution, "Location")) } } +} + +/** + * Suite that tests the redaction of BatchScanExec. + */ +class DataSourceV2ScanExecRedactionSuite extends DataSourceScanRedactionTest { + override protected def sparkConf: SparkConf = super.sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST.key, "") + + override protected def getRootPath(df: DataFrame): Path = + df.queryExecution.sparkPlan.find(_.isInstanceOf[BatchScanExec]).get + .asInstanceOf[BatchScanExec].scan.asInstanceOf[OrcScan].fileIndex.rootPaths.head + + test("explain is redacted using SQLConf") { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + val df = spark.read.orc(basePath) + val replacement = "*********" + + // Respect SparkConf and replace file:/ + assert(isIncluded(df.queryExecution, replacement)) + assert(isIncluded(df.queryExecution, "BatchScan")) + assert(!isIncluded(df.queryExecution, "file:/")) + + withSQLConf(SQLConf.SQL_STRING_REDACTION_PATTERN.key -> "(?i)BatchScan") { + // Respect SQLConf and replace FileScan + assert(isIncluded(df.queryExecution, replacement)) + + assert(!isIncluded(df.queryExecution, "BatchScan")) + assert(isIncluded(df.queryExecution, "file:/")) + } + } + } + + test("FileScan description") { + withTempPath { path => + val dir = path.getCanonicalPath + spark.range(0, 10).write.orc(dir) + val df = spark.read.orc(dir) + + assert(isIncluded(df.queryExecution, "ReadSchema")) + assert(isIncluded(df.queryExecution, "BatchScan")) + assert(isIncluded(df.queryExecution, "PushedFilters")) + assert(isIncluded(df.queryExecution, "Location")) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala index 611b2fc037f3d..0869e25674e69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ExternalAppendOnlyUnsafeRowArrayBenchmark.scala @@ -20,24 +20,57 @@ package org.apache.spark.sql.execution import scala.collection.mutable.ArrayBuffer import org.apache.spark.{SparkConf, SparkContext, SparkEnv, TaskContext} -import org.apache.spark.benchmark.Benchmark +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.internal.config import org.apache.spark.memory.MemoryTestingUtils import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.util.collection.unsafe.sort.UnsafeExternalSorter -object ExternalAppendOnlyUnsafeRowArrayBenchmark { +/** + * Benchmark ExternalAppendOnlyUnsafeRowArray. + * To run this benchmark: + * {{{ + * 1. without sbt: + * bin/spark-submit --class --jars + * 2. build/sbt build/sbt ";project sql;set javaOptions + * in Test += \"-Dspark.memory.debugFill=false\";test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt ";project sql;set javaOptions + * in Test += \"-Dspark.memory.debugFill=false\";test:runMain " + * Results will be written to + * "benchmarks/ExternalAppendOnlyUnsafeRowArrayBenchmark-results.txt". + * }}} + */ +object ExternalAppendOnlyUnsafeRowArrayBenchmark extends BenchmarkBase { - def testAgainstRawArrayBuffer(numSpillThreshold: Int, numRows: Int, iterations: Int): Unit = { + private val conf = new SparkConf(false) + // Make the Java serializer write a reset instruction (TC_RESET) after each object to test + // for a bug we had with bytes written past the last object in a batch (SPARK-2792) + .set(config.SERIALIZER_OBJECT_STREAM_RESET, 1) + .set(config.SERIALIZER, "org.apache.spark.serializer.JavaSerializer") + + private def withFakeTaskContext(f: => Unit): Unit = { + val sc = new SparkContext("local", "test", conf) + val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) + TaskContext.setTaskContext(taskContext) + f + sc.stop() + } + + private def testRows(numRows: Int): Seq[UnsafeRow] = { val random = new java.util.Random() - val rows = (1 to numRows).map(_ => { + (1 to numRows).map(_ => { val row = new UnsafeRow(1) row.pointTo(new Array[Byte](64), 16) row.setLong(0, random.nextLong()) row }) + } - val benchmark = new Benchmark(s"Array with $numRows rows", iterations * numRows) + def testAgainstRawArrayBuffer(numSpillThreshold: Int, numRows: Int, iterations: Int): Unit = { + val rows = testRows(numRows) + + val benchmark = new Benchmark(s"Array with $numRows rows", iterations * numRows, + output = output) // Internally, `ExternalAppendOnlyUnsafeRowArray` will create an // in-memory buffer of size `numSpillThreshold`. This will mimic that @@ -82,33 +115,19 @@ object ExternalAppendOnlyUnsafeRowArrayBenchmark { } } - val conf = new SparkConf(false) - // Make the Java serializer write a reset instruction (TC_RESET) after each object to test - // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") - - val sc = new SparkContext("local", "test", conf) - val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) - TaskContext.setTaskContext(taskContext) - benchmark.run() - sc.stop() + withFakeTaskContext { + benchmark.run() + } } def testAgainstRawUnsafeExternalSorter( numSpillThreshold: Int, numRows: Int, iterations: Int): Unit = { + val rows = testRows(numRows) - val random = new java.util.Random() - val rows = (1 to numRows).map(_ => { - val row = new UnsafeRow(1) - row.pointTo(new Array[Byte](64), 16) - row.setLong(0, random.nextLong()) - row - }) - - val benchmark = new Benchmark(s"Spilling with $numRows rows", iterations * numRows) + val benchmark = new Benchmark(s"Spilling with $numRows rows", iterations * numRows, + output = output) benchmark.addCase("UnsafeExternalSorter") { _: Int => var sum = 0L @@ -158,80 +177,23 @@ object ExternalAppendOnlyUnsafeRowArrayBenchmark { } } - val conf = new SparkConf(false) - // Make the Java serializer write a reset instruction (TC_RESET) after each object to test - // for a bug we had with bytes written past the last object in a batch (SPARK-2792) - conf.set("spark.serializer.objectStreamReset", "1") - conf.set("spark.serializer", "org.apache.spark.serializer.JavaSerializer") - - val sc = new SparkContext("local", "test", conf) - val taskContext = MemoryTestingUtils.fakeTaskContext(SparkEnv.get) - TaskContext.setTaskContext(taskContext) - benchmark.run() - sc.stop() + withFakeTaskContext { + benchmark.run() + } } - def main(args: Array[String]): Unit = { - - // ========================================================================================= // - // WITHOUT SPILL - // ========================================================================================= // - - val spillThreshold = 100 * 1000 - - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - - Array with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - ArrayBuffer 7821 / 7941 33.5 29.8 1.0X - ExternalAppendOnlyUnsafeRowArray 8798 / 8819 29.8 33.6 0.9X - */ - testAgainstRawArrayBuffer(spillThreshold, 1000, 1 << 18) - - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - - Array with 30000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - ArrayBuffer 19200 / 19206 25.6 39.1 1.0X - ExternalAppendOnlyUnsafeRowArray 19558 / 19562 25.1 39.8 1.0X - */ - testAgainstRawArrayBuffer(spillThreshold, 30 * 1000, 1 << 14) - - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - - Array with 100000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - ArrayBuffer 5949 / 6028 17.2 58.1 1.0X - ExternalAppendOnlyUnsafeRowArray 6078 / 6138 16.8 59.4 1.0X - */ - testAgainstRawArrayBuffer(spillThreshold, 100 * 1000, 1 << 10) - - // ========================================================================================= // - // WITH SPILL - // ========================================================================================= // - - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - - Spilling with 1000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - UnsafeExternalSorter 9239 / 9470 28.4 35.2 1.0X - ExternalAppendOnlyUnsafeRowArray 8857 / 8909 29.6 33.8 1.0X - */ - testAgainstRawUnsafeExternalSorter(100 * 1000, 1000, 1 << 18) - - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - - Spilling with 10000 rows: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - UnsafeExternalSorter 4 / 5 39.3 25.5 1.0X - ExternalAppendOnlyUnsafeRowArray 5 / 6 29.8 33.5 0.8X - */ - testAgainstRawUnsafeExternalSorter( - config.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get, 10 * 1000, 1 << 4) + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("WITHOUT SPILL") { + val spillThreshold = 100 * 1000 + testAgainstRawArrayBuffer(spillThreshold, 100 * 1000, 1 << 10) + testAgainstRawArrayBuffer(spillThreshold, 1000, 1 << 18) + testAgainstRawArrayBuffer(spillThreshold, 30 * 1000, 1 << 14) + } + + runBenchmark("WITH SPILL") { + testAgainstRawUnsafeExternalSorter(100 * 1000, 1000, 1 << 18) + testAgainstRawUnsafeExternalSorter( + config.SHUFFLE_SPILL_NUM_ELEMENTS_FORCE_SPILL_THRESHOLD.defaultValue.get, 10 * 1000, 1 << 4) + } } } 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 new file mode 100644 index 0000000000000..bbce4705871df --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/HiveResultSuite.scala @@ -0,0 +1,47 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import java.sql.{Date, Timestamp} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SharedSQLContext} + +class HiveResultSuite extends SparkFunSuite with SharedSQLContext { + import testImplicits._ + + test("date formatting in hive result") { + val date = "2018-12-28" + val executedPlan = Seq(Date.valueOf(date)).toDS().queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head == date) + } + + test("timestamp formatting in hive result") { + val timestamp = "2018-12-28 01:02:03" + val executedPlan = Seq(Timestamp.valueOf(timestamp)).toDS().queryExecution.executedPlan + val result = HiveResult.hiveResultString(executedPlan) + assert(result.head == timestamp) + } + + test("toHiveString correctly handles UDTs") { + val point = new ExamplePoint(50.0, 50.0) + val tpe = new ExamplePointUDT() + assert(HiveResult.toHiveString((point, tpe)) === "(50.0, 50.0)") + } +} 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 new file mode 100644 index 0000000000000..b1143484a85e8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/LogicalPlanTagInSparkPlanSuite.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.sql.execution + +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.aggregate.{HashAggregateExec, ObjectHashAggregateExec, SortAggregateExec} +import org.apache.spark.sql.execution.columnar.{InMemoryRelation, InMemoryTableScanExec} +import org.apache.spark.sql.execution.datasources.LogicalRelation +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} +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 + +class LogicalPlanTagInSparkPlanSuite extends TPCDSQuerySuite { + + override protected def checkGeneratedCode(plan: SparkPlan): Unit = { + super.checkGeneratedCode(plan) + checkLogicalPlanTag(plan) + } + + private def isFinalAgg(aggExprs: Seq[AggregateExpression]): Boolean = { + // TODO: aggregate node without aggregate expressions can also be a final aggregate, but + // currently the aggregate node doesn't have a final/partial flag. + aggExprs.nonEmpty && aggExprs.forall(ae => ae.mode == Complete || ae.mode == Final) + } + + // A scan plan tree is a plan tree that has a leaf node under zero or more Project/Filter nodes. + // Because of how codegen and columnar to row transitions work, we may have InputAdaptors + // and ColumnarToRow transformations in the middle of it, but they will not have the tag + // we want, so skip them if they are the first thing we see + private def isScanPlanTree(plan: SparkPlan, first: Boolean): Boolean = plan match { + case i: InputAdapter if !first => isScanPlanTree(i.child, false) + case c: ColumnarToRowExec if !first => isScanPlanTree(c.child, false) + case p: ProjectExec => isScanPlanTree(p.child, false) + case f: FilterExec => isScanPlanTree(f.child, false) + case _: LeafExecNode => true + case _ => false + } + + private def checkLogicalPlanTag(plan: SparkPlan): Unit = { + plan match { + case _: HashJoin | _: BroadcastNestedLoopJoinExec | _: CartesianProductExec + | _: ShuffledHashJoinExec | _: SortMergeJoinExec => + assertLogicalPlanType[Join](plan) + + // There is no corresponding logical plan for the physical partial aggregate. + case agg: HashAggregateExec if isFinalAgg(agg.aggregateExpressions) => + assertLogicalPlanType[Aggregate](plan) + case agg: ObjectHashAggregateExec if isFinalAgg(agg.aggregateExpressions) => + assertLogicalPlanType[Aggregate](plan) + case agg: SortAggregateExec if isFinalAgg(agg.aggregateExpressions) => + assertLogicalPlanType[Aggregate](plan) + + case _: WindowExec => + assertLogicalPlanType[Window](plan) + + case _: UnionExec => + assertLogicalPlanType[Union](plan) + + case _: SampleExec => + assertLogicalPlanType[Sample](plan) + + case _: GenerateExec => + assertLogicalPlanType[Generate](plan) + + // The exchange related nodes are created after the planning, they don't have corresponding + // logical plan. + case _: ShuffleExchangeExec | _: BroadcastExchangeExec | _: ReusedExchangeExec => + assert(plan.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isEmpty) + + // The subquery exec nodes are just wrappers of the actual nodes, they don't have + // corresponding logical plan. + case _: SubqueryExec | _: ReusedSubqueryExec => + assert(plan.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).isEmpty) + + case _ if isScanPlanTree(plan, true) => + // The strategies for planning scan can remove or add FilterExec/ProjectExec nodes, + // so it's not simple to check. Instead, we only check that the origin LogicalPlan + // contains the corresponding leaf node of the SparkPlan. + // a strategy might remove the filter if it's totally pushed down, e.g.: + // logical = Project(Filter(Scan A)) + // physical = ProjectExec(ScanExec A) + // we only check that leaf modes match between logical and physical plan. + val logicalLeaves = getLogicalPlan(plan).collectLeaves() + val physicalLeaves = plan.collectLeaves() + assert(logicalLeaves.length == 1) + assert(physicalLeaves.length == 1) + physicalLeaves.head match { + case _: RangeExec => logicalLeaves.head.isInstanceOf[Range] + case _: DataSourceScanExec => logicalLeaves.head.isInstanceOf[LogicalRelation] + case _: InMemoryTableScanExec => logicalLeaves.head.isInstanceOf[InMemoryRelation] + case _: LocalTableScanExec => logicalLeaves.head.isInstanceOf[LocalRelation] + case _: ExternalRDDScanExec[_] => logicalLeaves.head.isInstanceOf[ExternalRDD[_]] + case _: BatchScanExec => logicalLeaves.head.isInstanceOf[DataSourceV2Relation] + case _ => + } + // Do not need to check the children recursively. + return + + case _ => + } + + plan.children.foreach(checkLogicalPlanTag) + plan.subqueries.foreach(checkLogicalPlanTag) + } + + private def getLogicalPlan(node: SparkPlan): LogicalPlan = { + node.getTagValue(SparkPlan.LOGICAL_PLAN_TAG).getOrElse { + fail(node.getClass.getSimpleName + " does not have a logical plan link") + } + } + + private def assertLogicalPlanType[T <: LogicalPlan : ClassTag](node: SparkPlan): Unit = { + val logicalPlan = getLogicalPlan(node) + val expectedCls = implicitly[ClassTag[T]].runtimeClass + assert(expectedCls == logicalPlan.getClass) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala index a543eb8351656..e20a82ba9bc48 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala @@ -130,7 +130,10 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest with SharedSQLContext { } test("Incorrect result caused by the rule OptimizeMetadataOnlyQuery") { - withSQLConf(OPTIMIZER_METADATA_ONLY.key -> "true") { + // This test case is only for file source V1. As the rule OptimizeMetadataOnlyQuery is disabled + // by default, we can skip testing file source v2 in current stage. + withSQLConf(OPTIMIZER_METADATA_ONLY.key -> "true", + SQLConf.USE_V1_SOURCE_READER_LIST.key -> "json") { withTempPath { path => val tablePath = new File(s"${path.getCanonicalPath}/cOl3=c/cOl1=a/cOl5=e") Seq(("a", "b", "c", "d", "e")).toDF("cOl1", "cOl2", "cOl3", "cOl4", "cOl5") 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 e4e224df7607f..e30fb13d10df1 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 @@ -22,7 +22,7 @@ import org.apache.spark.sql.{execution, DataFrame, Row} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range, Repartition, Sort, Union} +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.columnar.{InMemoryRelation, InMemoryTableScanExec} import org.apache.spark.sql.execution.exchange.{EnsureRequirements, ReusedExchangeExec, ReuseExchange, ShuffleExchangeExec} @@ -172,15 +172,17 @@ class PlannerSuite extends SharedSQLContext { } test("SPARK-11390 explain should print PushedFilters of PhysicalRDD") { - withTempPath { file => - val path = file.getCanonicalPath - testData.write.parquet(path) - val df = spark.read.parquet(path) - df.createOrReplaceTempView("testPushed") - - withTempView("testPushed") { - val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan - assert(exp.toString.contains("PushedFilters: [IsNotNull(key), EqualTo(key,15)]")) + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempPath { file => + val path = file.getCanonicalPath + testData.write.parquet(path) + val df = spark.read.parquet(path) + df.createOrReplaceTempView("testPushed") + + withTempView("testPushed") { + val exp = sql("select * from testPushed where key = 15").queryExecution.sparkPlan + assert(exp.toString.contains("PushedFilters: [IsNotNull(key), EqualTo(key,15)]")) + } } } } @@ -293,7 +295,7 @@ class PlannerSuite extends SharedSQLContext { case Repartition (numPartitions, shuffle, Repartition(_, shuffleChild, _)) => assert(numPartitions === 5) assert(shuffle === false) - assert(shuffleChild === true) + assert(shuffleChild) } } @@ -411,8 +413,7 @@ class PlannerSuite extends SharedSQLContext { val inputPlan = ShuffleExchangeExec( partitioning, - DummySparkPlan(outputPartitioning = partitioning), - None) + DummySparkPlan(outputPartitioning = partitioning)) val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) if (outputPlan.collect { case e: ShuffleExchangeExec => true }.size == 2) { @@ -427,8 +428,7 @@ class PlannerSuite extends SharedSQLContext { val inputPlan = ShuffleExchangeExec( partitioning, - DummySparkPlan(outputPartitioning = partitioning), - None) + DummySparkPlan(outputPartitioning = partitioning)) val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) assertDistributionRequirementsAreSatisfied(outputPlan) if (outputPlan.collect { case e: ShuffleExchangeExec => true }.size == 1) { @@ -450,7 +450,7 @@ class PlannerSuite extends SharedSQLContext { val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(inputPlan) val shuffle = outputPlan.collect { case e: ShuffleExchangeExec => e } assert(shuffle.size === 1) - assert(shuffle.head.newPartitioning === finalPartitioning) + assert(shuffle.head.outputPartitioning === finalPartitioning) } test("Reuse exchanges") { @@ -462,8 +462,7 @@ class PlannerSuite extends SharedSQLContext { DummySparkPlan( children = DummySparkPlan(outputPartitioning = childPartitioning) :: Nil, requiredChildDistribution = Seq(distribution), - requiredChildOrdering = Seq(Seq.empty)), - None) + requiredChildOrdering = Seq(Seq.empty))) val inputPlan = SortMergeJoinExec( Literal(1) :: Nil, @@ -697,6 +696,32 @@ class PlannerSuite extends SharedSQLContext { } } + test("SPARK-27485: EnsureRequirements.reorder should handle duplicate expressions") { + val plan1 = DummySparkPlan( + outputPartitioning = HashPartitioning(exprA :: exprB :: exprA :: Nil, 5)) + val plan2 = DummySparkPlan() + val smjExec = SortMergeJoinExec( + leftKeys = exprA :: exprB :: exprB :: Nil, + rightKeys = exprA :: exprC :: exprC :: Nil, + joinType = Inner, + condition = None, + left = plan1, + right = plan2) + val outputPlan = EnsureRequirements(spark.sessionState.conf).apply(smjExec) + outputPlan match { + case SortMergeJoinExec(leftKeys, rightKeys, _, _, + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(leftPartitioningExpressions, _), _, _), _), + SortExec(_, _, + ShuffleExchangeExec(HashPartitioning(rightPartitioningExpressions, _), _, _), _)) => + assert(leftKeys === smjExec.leftKeys) + assert(rightKeys === smjExec.rightKeys) + assert(leftKeys === leftPartitioningExpressions) + assert(rightKeys === rightPartitioningExpressions) + case _ => fail(outputPlan.toString) + } + } + test("SPARK-24500: create union with stream of children") { val df = Union(Stream( Range(1, 1, 1, 1), @@ -780,6 +805,57 @@ class PlannerSuite extends SharedSQLContext { classOf[PartitioningCollection]) } } + + test("SPARK-26812: wrong nullability for complex datatypes in union") { + def testUnionOutputType(input1: DataType, input2: DataType, output: DataType): Unit = { + val query = Union( + LocalRelation(StructField("a", input1)), LocalRelation(StructField("a", input2))) + assert(query.output.head.dataType == output) + } + + // Map + testUnionOutputType( + MapType(StringType, StringType, valueContainsNull = false), + MapType(StringType, StringType, valueContainsNull = true), + MapType(StringType, StringType, valueContainsNull = true)) + testUnionOutputType( + MapType(StringType, StringType, valueContainsNull = true), + MapType(StringType, StringType, valueContainsNull = false), + MapType(StringType, StringType, valueContainsNull = true)) + testUnionOutputType( + MapType(StringType, StringType, valueContainsNull = false), + MapType(StringType, StringType, valueContainsNull = false), + MapType(StringType, StringType, valueContainsNull = false)) + + // Array + testUnionOutputType( + ArrayType(StringType, containsNull = false), + ArrayType(StringType, containsNull = true), + ArrayType(StringType, containsNull = true)) + testUnionOutputType( + ArrayType(StringType, containsNull = true), + ArrayType(StringType, containsNull = false), + ArrayType(StringType, containsNull = true)) + testUnionOutputType( + ArrayType(StringType, containsNull = false), + ArrayType(StringType, containsNull = false), + ArrayType(StringType, containsNull = false)) + + // Struct + testUnionOutputType( + StructType(Seq( + StructField("f1", StringType, nullable = false), + StructField("f2", StringType, nullable = true), + StructField("f3", StringType, nullable = false))), + StructType(Seq( + StructField("f1", StringType, nullable = true), + StructField("f2", StringType, nullable = false), + StructField("f3", StringType, nullable = false))), + StructType(Seq( + StructField("f1", StringType, nullable = true), + StructField("f2", StringType, nullable = true), + StructField("f3", StringType, nullable = false)))) + } } // Used for unit-testing EnsureRequirements @@ -790,6 +866,6 @@ private case class DummySparkPlan( override val requiredChildDistribution: Seq[Distribution] = Nil, override val requiredChildOrdering: Seq[Seq[SortOrder]] = Nil ) extends SparkPlan { - override protected def doExecute(): RDD[InternalRow] = throw new NotImplementedError + override protected def doExecute(): RDD[InternalRow] = throw new UnsupportedOperationException override def output: Seq[Attribute] = Seq.empty } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala index 964440346deb0..7de5e826f6676 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryExecutionSuite.scala @@ -16,16 +16,111 @@ */ package org.apache.spark.sql.execution +import scala.io.Source + import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, OneRowRelation} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +case class QueryExecutionTestRecord( + c0: Int, c1: Int, c2: Int, c3: Int, c4: Int, + c5: Int, c6: Int, c7: Int, c8: Int, c9: Int, + c10: Int, c11: Int, c12: Int, c13: Int, c14: Int, + c15: Int, c16: Int, c17: Int, c18: Int, c19: Int, + c20: Int, c21: Int, c22: Int, c23: Int, c24: Int, + c25: Int, c26: Int) + class QueryExecutionSuite extends SharedSQLContext { + import testImplicits._ + + def checkDumpedPlans(path: String, expected: Int): Unit = { + assert(Source.fromFile(path).getLines.toList + .takeWhile(_ != "== Whole Stage Codegen ==") == List( + "== Parsed Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Analyzed Logical Plan ==", + "id: bigint", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Optimized Logical Plan ==", + s"Range (0, $expected, step=1, splits=Some(2))", + "", + "== Physical Plan ==", + s"*(1) Range (0, $expected, step=1, splits=2)", + "")) + } + test("dumping query execution info to a file") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path) + + checkDumpedPlans(path, expected = 10) + } + } + + test("dumping query execution info to an existing file") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val df = spark.range(0, 10) + df.queryExecution.debug.toFile(path) + + val df2 = spark.range(0, 1) + df2.queryExecution.debug.toFile(path) + checkDumpedPlans(path, expected = 1) + } + } + + test("dumping query execution info to non-existing folder") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/newfolder/plans.txt" + val df = spark.range(0, 100) + df.queryExecution.debug.toFile(path) + checkDumpedPlans(path, expected = 100) + } + } + + test("dumping query execution info by invalid path") { + val path = "1234567890://plans.txt" + val exception = intercept[IllegalArgumentException] { + spark.range(0, 100).queryExecution.debug.toFile(path) + } + + assert(exception.getMessage.contains("Illegal character in scheme name")) + } + + test("limit number of fields by sql config") { + def relationPlans: String = { + val ds = spark.createDataset(Seq(QueryExecutionTestRecord( + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, + 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))) + ds.queryExecution.toString + } + withSQLConf(SQLConf.MAX_TO_STRING_FIELDS.key -> "26") { + assert(relationPlans.contains("more fields")) + } + withSQLConf(SQLConf.MAX_TO_STRING_FIELDS.key -> "27") { + assert(!relationPlans.contains("more fields")) + } + } + + test("check maximum fields restriction") { + withTempDir { dir => + val path = dir.getCanonicalPath + "/plans.txt" + val ds = spark.createDataset(Seq(QueryExecutionTestRecord( + 0, 1, 2, 3, 4, 5, 6, 7, 8, 9, 10, 11, 12, 13, 14, 15, + 16, 17, 18, 19, 20, 21, 22, 23, 24, 25, 26))) + ds.queryExecution.debug.toFile(path) + val localRelations = Source.fromFile(path).getLines().filter(_.contains("LocalRelation")) + + assert(!localRelations.exists(_.contains("more fields"))) + } + } + test("toString() exception/error handling") { - spark.experimental.extraStrategies = Seq( - new SparkStrategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = Nil - }) + spark.experimental.extraStrategies = Seq[SparkStrategy]((_: LogicalPlan) => Nil) def qe: QueryExecution = new QueryExecution(spark, OneRowRelation()) @@ -33,19 +128,13 @@ class QueryExecutionSuite extends SharedSQLContext { assert(qe.toString.contains("OneRowRelation")) // Throw an AnalysisException - this should be captured. - spark.experimental.extraStrategies = Seq( - new SparkStrategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = - throw new AnalysisException("exception") - }) + spark.experimental.extraStrategies = Seq[SparkStrategy]( + (_: LogicalPlan) => throw new AnalysisException("exception")) assert(qe.toString.contains("org.apache.spark.sql.AnalysisException")) // Throw an Error - this should not be captured. - spark.experimental.extraStrategies = Seq( - new SparkStrategy { - override def apply(plan: LogicalPlan): Seq[SparkPlan] = - throw new Error("error") - }) + spark.experimental.extraStrategies = Seq[SparkStrategy]( + (_: LogicalPlan) => throw new Error("error")) val error = intercept[Error](qe.toString) assert(error.getMessage.contains("error")) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala new file mode 100644 index 0000000000000..e42177c156ee9 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/QueryPlanningTrackerEndToEndSuite.scala @@ -0,0 +1,41 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution + +import org.apache.spark.sql.test.SharedSQLContext + +class QueryPlanningTrackerEndToEndSuite extends SharedSQLContext { + + test("programmatic API") { + val df = spark.range(1000).selectExpr("count(*)") + df.collect() + val tracker = df.queryExecution.tracker + assert(tracker.phases.keySet == Set("analysis", "optimization", "planning")) + assert(tracker.rules.nonEmpty) + } + + test("sql") { + val df = spark.sql("select * from range(1)") + df.collect() + + val tracker = df.queryExecution.tracker + assert(tracker.phases.keySet == Set("parsing", "analysis", "optimization", "planning")) + assert(tracker.rules.nonEmpty) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala similarity index 64% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala index 6ad025f37e440..35c33a7157d38 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/ExchangeCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ReduceNumShufflePartitionsSuite.scala @@ -20,12 +20,14 @@ package org.apache.spark.sql.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{MapOutputStatistics, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql._ -import org.apache.spark.sql.execution.exchange.{ExchangeCoordinator, ReusedExchangeExec, ShuffleExchangeExec} +import org.apache.spark.sql.execution.adaptive._ +import org.apache.spark.sql.execution.adaptive.rule.{CoalescedShuffleReaderExec, ReduceNumShufflePartitions} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { +class ReduceNumShufflePartitionsSuite extends SparkFunSuite with BeforeAndAfterAll { private var originalActiveSparkSession: Option[SparkSession] = _ private var originalInstantiatedSparkSession: Option[SparkSession] = _ @@ -50,7 +52,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { } private def checkEstimation( - coordinator: ExchangeCoordinator, + rule: ReduceNumShufflePartitions, bytesByPartitionIdArray: Array[Array[Long]], expectedPartitionStartIndices: Array[Int]): Unit = { val mapOutputStatistics = bytesByPartitionIdArray.zipWithIndex.map { @@ -58,18 +60,27 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { new MapOutputStatistics(index, bytesByPartitionId) } val estimatedPartitionStartIndices = - coordinator.estimatePartitionStartIndices(mapOutputStatistics) + rule.estimatePartitionStartIndices(mapOutputStatistics) assert(estimatedPartitionStartIndices === expectedPartitionStartIndices) } + private def createReduceNumShufflePartitionsRule( + advisoryTargetPostShuffleInputSize: Long, + minNumPostShufflePartitions: Int = 1): ReduceNumShufflePartitions = { + val conf = new SQLConf().copy( + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE -> advisoryTargetPostShuffleInputSize, + SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS -> minNumPostShufflePartitions) + ReduceNumShufflePartitions(conf) + } + test("test estimatePartitionStartIndices - 1 Exchange") { - val coordinator = new ExchangeCoordinator(100L) + val rule = createReduceNumShufflePartitionsRule(100L) { // All bytes per partition are 0. val bytesByPartitionId = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { @@ -77,40 +88,40 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // 1 post-shuffle partition is needed. val bytesByPartitionId = Array[Long](10, 0, 20, 0, 0) val expectedPartitionStartIndices = Array[Int](0) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // 2 post-shuffle partitions are needed. val bytesByPartitionId = Array[Long](10, 0, 90, 20, 0) val expectedPartitionStartIndices = Array[Int](0, 3) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // There are a few large pre-shuffle partitions. val bytesByPartitionId = Array[Long](110, 10, 100, 110, 0) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // All pre-shuffle partitions are larger than the targeted size. val bytesByPartitionId = Array[Long](100, 110, 100, 110, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } { // The last pre-shuffle partition is in a single post-shuffle partition. val bytesByPartitionId = Array[Long](30, 30, 0, 40, 110) val expectedPartitionStartIndices = Array[Int](0, 4) - checkEstimation(coordinator, Array(bytesByPartitionId), expectedPartitionStartIndices) + checkEstimation(rule, Array(bytesByPartitionId), expectedPartitionStartIndices) } } test("test estimatePartitionStartIndices - 2 Exchanges") { - val coordinator = new ExchangeCoordinator(100L) + val rule = createReduceNumShufflePartitionsRule(100L) { // If there are multiple values of the number of pre-shuffle partitions, @@ -121,7 +132,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { Array( new MapOutputStatistics(0, bytesByPartitionId1), new MapOutputStatistics(1, bytesByPartitionId2)) - intercept[AssertionError](coordinator.estimatePartitionStartIndices(mapOutputStatistics)) + intercept[AssertionError](rule.estimatePartitionStartIndices(mapOutputStatistics)) } { @@ -130,7 +141,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -142,7 +153,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 20, 0, 20) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -153,7 +164,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -164,7 +175,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -175,7 +186,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 70, 0, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -186,7 +197,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 60, 0, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -197,14 +208,14 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](30, 0, 60, 70, 110) val expectedPartitionStartIndices = Array[Int](0, 1, 2, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } } test("test estimatePartitionStartIndices and enforce minimal number of reducers") { - val coordinator = new ExchangeCoordinator(100L, Some(2)) + val rule = createReduceNumShufflePartitionsRule(100L, 2) { // The minimal number of post-shuffle partitions is not enforced because @@ -213,7 +224,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](0, 0, 0, 0, 0) val expectedPartitionStartIndices = Array[Int](0) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -224,7 +235,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](5, 10, 0, 10, 5) val expectedPartitionStartIndices = Array[Int](0, 3) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -235,7 +246,7 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { val bytesByPartitionId2 = Array[Long](40, 10, 0, 10, 30) val expectedPartitionStartIndices = Array[Int](0, 1, 3, 4) checkEstimation( - coordinator, + rule, Array(bytesByPartitionId1, bytesByPartitionId2), expectedPartitionStartIndices) } @@ -256,25 +267,24 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { def withSparkSession( f: SparkSession => Unit, - targetNumPostShufflePartitions: Int, + targetPostShuffleInputSize: Int, minNumPostShufflePartitions: Option[Int]): Unit = { val sparkConf = new SparkConf(false) .setMaster("local[*]") .setAppName("test") - .set("spark.ui.enabled", "false") - .set("spark.driver.allowMultipleContexts", "true") - .set(SQLConf.SHUFFLE_PARTITIONS.key, "5") + .set(UI_ENABLED, false) + .set(SQLConf.SHUFFLE_MAX_NUM_POSTSHUFFLE_PARTITIONS.key, "5") .set(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key, "true") .set(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key, "-1") .set( SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key, - targetNumPostShufflePartitions.toString) + targetPostShuffleInputSize.toString) minNumPostShufflePartitions match { case Some(numPartitions) => sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, numPartitions.toString) case None => - sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, "-1") + sparkConf.set(SQLConf.SHUFFLE_MIN_NUM_POSTSHUFFLE_PARTITIONS.key, "1") } val spark = SparkSession.builder() @@ -304,25 +314,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = agg.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val finalPlan = agg.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(exchanges.length === 1) + assert(shuffleReaders.length === 1) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 3) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 3) } } } @@ -355,25 +361,21 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(exchanges.length === 2) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 2) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 2) } } } @@ -411,26 +413,26 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(exchanges.length === 4) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(2, 3)) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 2) + } } } - withSparkSession(test, 6644, minNumPostShufflePartitions) + withSparkSession(test, 16384, minNumPostShufflePartitions) } test(s"determining the number of reducers: complex query 2$testNameNote") { @@ -463,39 +465,126 @@ class ExchangeCoordinatorSuite extends SparkFunSuite with BeforeAndAfterAll { // Then, let's look at the number of post-shuffle partitions estimated // by the ExchangeCoordinator. - val exchanges = join.queryExecution.executedPlan.collect { - case e: ShuffleExchangeExec => e + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader } - assert(exchanges.length === 3) + assert(shuffleReaders.length === 2) minNumPostShufflePartitions match { case Some(numPartitions) => - exchanges.foreach { - case e: ShuffleExchangeExec => - assert(e.coordinator.isDefined) - assert(e.outputPartitioning.numPartitions === 5) - case o => + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === numPartitions) } case None => - assert(exchanges.forall(_.coordinator.isDefined)) - assert(exchanges.map(_.outputPartitioning.numPartitions).toSet === Set(5, 3)) + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 3) + } } } - withSparkSession(test, 6144, minNumPostShufflePartitions) + withSparkSession(test, 12000, minNumPostShufflePartitions) + } + + test(s"determining the number of reducers: plan already partitioned$testNameNote") { + val test: SparkSession => Unit = { spark: SparkSession => + try { + spark.range(1000).write.bucketBy(30, "id").saveAsTable("t") + // `df1` is hash partitioned by `id`. + val df1 = spark.read.table("t") + val df2 = + spark + .range(0, 1000, 1, numInputPartitions) + .selectExpr("id % 500 as key2", "id as value2") + + val join = df1.join(df2, col("id") === col("key2")).select(col("id"), col("value2")) + + // Check the answer first. + val expectedAnswer = spark.range(0, 500).selectExpr("id % 500", "id as value") + .union(spark.range(500, 1000).selectExpr("id % 500", "id as value")) + checkAnswer( + join, + expectedAnswer.collect()) + + // Then, let's make sure we do not reduce number of ppst shuffle partitions. + val finalPlan = join.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val shuffleReaders = finalPlan.collect { + case reader: CoalescedShuffleReaderExec => reader + } + assert(shuffleReaders.length === 0) + } finally { + spark.sql("drop table t") + } + } + withSparkSession(test, 12000, minNumPostShufflePartitions) } } test("SPARK-24705 adaptive query execution works correctly when exchange reuse enabled") { - val test = { spark: SparkSession => + val test: SparkSession => Unit = { spark: SparkSession => spark.sql("SET spark.sql.exchange.reuse=true") val df = spark.range(1).selectExpr("id AS key", "id AS value") + + // test case 1: a query stage has 3 child stages but they are the same stage. + // Final Stage 1 + // ShuffleQueryStage 0 + // ReusedQueryStage 0 + // ReusedQueryStage 0 val resultDf = df.join(df, "key").join(df, "key") - val sparkPlan = resultDf.queryExecution.executedPlan - assert(sparkPlan.collect { case p: ReusedExchangeExec => p }.length == 1) - assert(sparkPlan.collect { case p @ ShuffleExchangeExec(_, _, Some(c)) => p }.length == 3) checkAnswer(resultDf, Row(0, 0, 0, 0) :: Nil) + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert(finalPlan.collect { case p: ReusedQueryStageExec => p }.length == 2) + assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 3) + + + // test case 2: a query stage has 2 parent stages. + // Final Stage 3 + // ShuffleQueryStage 1 + // ShuffleQueryStage 0 + // ShuffleQueryStage 2 + // ReusedQueryStage 0 + val grouped = df.groupBy("key").agg(max("value").as("value")) + val resultDf2 = grouped.groupBy(col("key") + 1).max("value") + .union(grouped.groupBy(col("key") + 2).max("value")) + checkAnswer(resultDf2, Row(1, 0) :: Row(2, 0) :: Nil) + + val finalPlan2 = resultDf2.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + + // The result stage has 2 children + val level1Stages = finalPlan2.collect { case q: QueryStageExec => q } + assert(level1Stages.length == 2) + + val leafStages = level1Stages.flatMap { stage => + // All of the child stages of result stage have only one child stage. + val children = stage.plan.collect { case q: QueryStageExec => q } + assert(children.length == 1) + children + } + assert(leafStages.length == 2) + + val reusedStages = level1Stages.flatMap { stage => + stage.plan.collect { case r: ReusedQueryStageExec => r } + } + assert(reusedStages.length == 1) } withSparkSession(test, 4, None) } + + test("Do not reduce the number of shuffle partition for repartition") { + val test: SparkSession => Unit = { spark: SparkSession => + val ds = spark.range(3) + val resultDf = ds.repartition(2, ds.col("id")).toDF() + + checkAnswer(resultDf, + Seq(0, 1, 2).map(i => Row(i))) + val finalPlan = resultDf.queryExecution.executedPlan + .asInstanceOf[AdaptiveSparkPlanExec].executedPlan + assert(finalPlan.collect { case p: CoalescedShuffleReaderExec => p }.length == 0) + } + withSparkSession(test, 200, None) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala index 08e40e28d3d57..08789e63fa7f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLJsonProtocolSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution -import org.json4s.jackson.JsonMethods.parse +import org.json4s.jackson.JsonMethods._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionStart +import org.apache.spark.sql.LocalSparkSession +import org.apache.spark.sql.execution.ui.{SparkListenerSQLExecutionEnd, SparkListenerSQLExecutionStart} +import org.apache.spark.sql.test.TestSparkSession import org.apache.spark.util.JsonProtocol -class SQLJsonProtocolSuite extends SparkFunSuite { +class SQLJsonProtocolSuite extends SparkFunSuite with LocalSparkSession { test("SparkPlanGraph backward compatibility: metadata") { val SQLExecutionStartJsonString = @@ -49,4 +51,29 @@ class SQLJsonProtocolSuite extends SparkFunSuite { new SparkPlanInfo("TestNode", "test string", Nil, Map(), Nil), 0) assert(reconstructedEvent == expectedEvent) } + + test("SparkListenerSQLExecutionEnd backward compatibility") { + spark = new TestSparkSession() + val qe = spark.sql("select 1").queryExecution + val event = SparkListenerSQLExecutionEnd(1, 10) + event.duration = 1000 + event.executionName = Some("test") + event.qe = qe + event.executionFailure = Some(new RuntimeException("test")) + val json = JsonProtocol.sparkEventToJson(event) + assert(json == parse( + """ + |{ + | "Event" : "org.apache.spark.sql.execution.ui.SparkListenerSQLExecutionEnd", + | "executionId" : 1, + | "time" : 10 + |} + """.stripMargin)) + val readBack = JsonProtocol.sparkEventFromJson(json) + event.duration = 0 + event.executionName = None + event.qe = null + event.executionFailure = None + assert(readBack == event) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala index 8269d4d3a285d..64e305cd5c371 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLViewSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException +import org.apache.spark.sql.internal.SQLConf.MAX_NESTED_VIEW_DEPTH import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} class SimpleSQLViewSuite extends SQLViewSuite with SharedSQLContext @@ -665,17 +666,17 @@ abstract class SQLViewSuite extends QueryTest with SQLTestUtils { sql(s"CREATE VIEW view${idx + 1} AS SELECT * FROM view$idx") } - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "10") { + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "10") { val e = intercept[AnalysisException] { sql("SELECT * FROM view10") }.getMessage assert(e.contains("The depth of view `default`.`view0` exceeds the maximum view " + "resolution depth (10). Analysis is aborted to avoid errors. Increase the value " + - "of spark.sql.view.maxNestedViewDepth to work around this.")) + s"of ${MAX_NESTED_VIEW_DEPTH.key} to work around this.")) } val e = intercept[IllegalArgumentException] { - withSQLConf("spark.sql.view.maxNestedViewDepth" -> "0") {} + withSQLConf(MAX_NESTED_VIEW_DEPTH.key -> "0") {} }.getMessage assert(e.contains("The maximum depth of a view reference in a nested view must be " + "positive.")) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala index 1c6fc3530cbe1..971fd842f046a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SQLWindowFunctionSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.TestUtils.assertSpilled import org.apache.spark.sql.{AnalysisException, QueryTest, Row} +import org.apache.spark.sql.internal.SQLConf.{WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD, WINDOW_EXEC_BUFFER_SPILL_THRESHOLD} import org.apache.spark.sql.test.SharedSQLContext case class WindowData(month: Int, area: String, product: Int) @@ -477,8 +478,8 @@ class SQLWindowFunctionSuite extends QueryTest with SharedSQLContext { |WINDOW w1 AS (ORDER BY x ROWS BETWEEN UNBOUNDED PRECEDiNG AND CURRENT RoW) """.stripMargin) - withSQLConf("spark.sql.windowExec.buffer.in.memory.threshold" -> "1", - "spark.sql.windowExec.buffer.spill.threshold" -> "2") { + withSQLConf(WINDOW_EXEC_BUFFER_IN_MEMORY_THRESHOLD.key -> "1", + WINDOW_EXEC_BUFFER_SPILL_THRESHOLD.key -> "2") { assertSpilled(sparkContext, "test with low buffer spill threshold") { checkAnswer(actual, expected) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala index d088e24e53bfe..b252100d890e3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SameResultSuite.scala @@ -20,7 +20,9 @@ package org.apache.spark.sql.execution import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, Project} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, FileScan} import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.IntegerType @@ -31,22 +33,83 @@ class SameResultSuite extends QueryTest with SharedSQLContext { import testImplicits._ test("FileSourceScanExec: different orders of data filters and partition filters") { - withTempPath { path => - val tmpDir = path.getCanonicalPath - spark.range(10) - .selectExpr("id as a", "id + 1 as b", "id + 2 as c", "id + 3 as d") - .write - .partitionBy("a", "b") - .parquet(tmpDir) - val df = spark.read.parquet(tmpDir) - // partition filters: a > 1 AND b < 9 - // data filters: c > 1 AND d < 9 - val plan1 = getFileSourceScanExec(df.where("a > 1 AND b < 9 AND c > 1 AND d < 9")) - val plan2 = getFileSourceScanExec(df.where("b < 9 AND a > 1 AND d < 9 AND c > 1")) - assert(plan1.sameResult(plan2)) + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempPath { path => + val tmpDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id as a", "id + 1 as b", "id + 2 as c", "id + 3 as d") + .write + .partitionBy("a", "b") + .parquet(tmpDir) + val df = spark.read.parquet(tmpDir) + // partition filters: a > 1 AND b < 9 + // data filters: c > 1 AND d < 9 + val plan1 = getFileSourceScanExec(df.where("a > 1 AND b < 9 AND c > 1 AND d < 9")) + val plan2 = getFileSourceScanExec(df.where("b < 9 AND a > 1 AND d < 9 AND c > 1")) + assert(plan1.sameResult(plan2)) + } } } + test("FileScan: different orders of data filters and partition filters") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + Seq("orc", "json", "csv", "parquet").foreach { format => + withTempPath { path => + val tmpDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id as a", "id + 1 as b", "id + 2 as c", "id + 3 as d") + .write + .partitionBy("a", "b") + .format(format) + .option("header", true) + .save(tmpDir) + val df = spark.read.format(format).option("header", true).load(tmpDir) + // partition filters: a > 1 AND b < 9 + // data filters: c > 1 AND d < 9 + val plan1 = df.where("a > 1 AND b < 9 AND c > 1 AND d < 9").queryExecution.sparkPlan + val plan2 = df.where("b < 9 AND a > 1 AND d < 9 AND c > 1").queryExecution.sparkPlan + assert(plan1.sameResult(plan2)) + val scan1 = getBatchScanExec(plan1) + val scan2 = getBatchScanExec(plan2) + assert(scan1.sameResult(scan2)) + val plan3 = df.where("b < 9 AND a > 1 AND d < 8 AND c > 1").queryExecution.sparkPlan + assert(!plan1.sameResult(plan3)) + // The [[FileScan]]s should have different results if they support filter pushdown. + if (format == "orc" || format == "parquet") { + val scan3 = getBatchScanExec(plan3) + assert(!scan1.sameResult(scan3)) + } + } + } + } + } + + test("TextScan") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + withTempPath { path => + val tmpDir = path.getCanonicalPath + spark.range(10) + .selectExpr("id as a", "id + 1 as b", "cast(id as string) value") + .write + .partitionBy("a", "b") + .text(tmpDir) + val df = spark.read.text(tmpDir) + // partition filters: a > 1 AND b < 9 + // data filters: c > 1 AND d < 9 + val plan1 = df.where("a > 1 AND b < 9 AND value == '3'").queryExecution.sparkPlan + val plan2 = df.where("value == '3' AND a > 1 AND b < 9").queryExecution.sparkPlan + assert(plan1.sameResult(plan2)) + val scan1 = getBatchScanExec(plan1) + val scan2 = getBatchScanExec(plan2) + assert(scan1.sameResult(scan2)) + } + } + } + + private def getBatchScanExec(plan: SparkPlan): BatchScanExec = { + plan.find(_.isInstanceOf[BatchScanExec]).get.asInstanceOf[BatchScanExec] + } + private def getFileSourceScanExec(df: DataFrame): FileSourceScanExec = { df.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get .asInstanceOf[FileSourceScanExec] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala index 47ff372992b91..e9d0556ebb513 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkPlanSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.execution import org.apache.spark.SparkEnv import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class SparkPlanSuite extends QueryTest with SharedSQLContext { @@ -35,27 +37,50 @@ class SparkPlanSuite extends QueryTest with SharedSQLContext { } test("SPARK-23731 plans should be canonicalizable after being (de)serialized") { - withTempPath { path => - spark.range(1).write.parquet(path.getAbsolutePath) - val df = spark.read.parquet(path.getAbsolutePath) - val fileSourceScanExec = - df.queryExecution.sparkPlan.collectFirst { case p: FileSourceScanExec => p }.get - val serializer = SparkEnv.get.serializer.newInstance() - val readback = - serializer.deserialize[FileSourceScanExec](serializer.serialize(fileSourceScanExec)) - try { - readback.canonicalized - } catch { - case e: Throwable => fail("FileSourceScanExec was not canonicalizable", e) + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempPath { path => + spark.range(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + val fileSourceScanExec = + df.queryExecution.sparkPlan.collectFirst { case p: FileSourceScanExec => p }.get + val serializer = SparkEnv.get.serializer.newInstance() + val readback = + serializer.deserialize[FileSourceScanExec](serializer.serialize(fileSourceScanExec)) + try { + readback.canonicalized + } catch { + case e: Throwable => fail("FileSourceScanExec was not canonicalizable", e) + } + } + } + } + + test("SPARK-27418 BatchScanExec should be canonicalizable after being (de)serialized") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + withTempPath { path => + spark.range(1).write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + val batchScanExec = + df.queryExecution.sparkPlan.collectFirst { case p: BatchScanExec => p }.get + val serializer = SparkEnv.get.serializer.newInstance() + val readback = + serializer.deserialize[BatchScanExec](serializer.serialize(batchScanExec)) + try { + readback.canonicalized + } catch { + case e: Throwable => fail("BatchScanExec was not canonicalizable", e) + } } } } test("SPARK-25357 SparkPlanInfo of FileScan contains nonEmpty metadata") { - withTempPath { path => - spark.range(5).write.parquet(path.getAbsolutePath) - val f = spark.read.parquet(path.getAbsolutePath) - assert(SparkPlanInfo.fromSparkPlan(f.queryExecution.sparkPlan).metadata.nonEmpty) + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempPath { path => + spark.range(5).write.parquet(path.getAbsolutePath) + val f = spark.read.parquet(path.getAbsolutePath) + assert(SparkPlanInfo.fromSparkPlan(f.queryExecution.sparkPlan).metadata.nonEmpty) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala index 31b9bcdafbab8..df80311c0ce25 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala @@ -22,7 +22,6 @@ import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAlias, UnresolvedAttribute, UnresolvedRelation, UnresolvedStar} import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} import org.apache.spark.sql.catalyst.expressions.{Ascending, Concat, SortOrder} -import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, RepartitionByExpression, Sort} import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.datasources.{CreateTable, RefreshResource} @@ -59,12 +58,8 @@ class SparkSqlParserSuite extends AnalysisTest { comparePlans(normalized1, normalized2) } - private def intercept(sqlCommand: String, messages: String*): Unit = { - val e = intercept[ParseException](parser.parsePlan(sqlCommand)) - messages.foreach { message => - assert(e.message.contains(message)) - } - } + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parser.parsePlan)(sqlCommand, messages: _*) test("refresh resource") { assertEqual("REFRESH prefix_path", RefreshResource("prefix_path")) @@ -215,40 +210,21 @@ class SparkSqlParserSuite extends AnalysisTest { "no viable alternative at input") } - test("create table using - schema") { - assertEqual("CREATE TABLE my_tab(a INT COMMENT 'test', b STRING) USING parquet", - createTableUsing( - table = "my_tab", - schema = (new StructType) - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType) - ) - ) - intercept("CREATE TABLE my_tab(a: INT COMMENT 'test', b: STRING) USING parquet", - "no viable alternative at input") - } - - test("create view as insert into table") { - // Single insert query - intercept("CREATE VIEW testView AS INSERT INTO jt VALUES(1, 1)", - "Operation not allowed: CREATE VIEW ... AS INSERT INTO") - - // Multi insert query - intercept("CREATE VIEW testView AS FROM jt INSERT INTO tbl1 SELECT * WHERE jt.id < 5 " + - "INSERT INTO tbl2 SELECT * WHERE jt.id > 4", - "Operation not allowed: CREATE VIEW ... AS FROM ... [INSERT INTO ...]+") - } - test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") { + assertEqual("describe t", + DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) assertEqual("describe table t", - DescribeTableCommand( - TableIdentifier("t"), Map.empty, isExtended = false)) + DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = false)) assertEqual("describe table extended t", - DescribeTableCommand( - TableIdentifier("t"), Map.empty, isExtended = true)) + DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) assertEqual("describe table formatted t", - DescribeTableCommand( - TableIdentifier("t"), Map.empty, isExtended = true)) + DescribeTableCommand(TableIdentifier("t"), Map.empty, isExtended = true)) + } + + test("describe query") { + val query = "SELECT * FROM t" + assertEqual("DESCRIBE QUERY " + query, DescribeQueryCommand(query, parser.parsePlan(query))) + assertEqual("DESCRIBE " + query, DescribeQueryCommand(query, parser.parsePlan(query))) } test("describe table column") { @@ -377,14 +353,11 @@ class SparkSqlParserSuite extends AnalysisTest { Project(UnresolvedAlias(concat) :: Nil, UnresolvedRelation(TableIdentifier("t")))) } - test("SPARK-25046 Fix Alter View ... As Insert Into Table") { - // Single insert query - intercept("ALTER VIEW testView AS INSERT INTO jt VALUES(1, 1)", - "Operation not allowed: ALTER VIEW ... AS INSERT INTO") - - // Multi insert query - intercept("ALTER VIEW testView AS FROM jt INSERT INTO tbl1 SELECT * WHERE jt.id < 5 " + - "INSERT INTO tbl2 SELECT * WHERE jt.id > 4", - "Operation not allowed: ALTER VIEW ... AS FROM ... [INSERT INTO ...]+") + test("database and schema tokens are interchangeable") { + assertEqual("CREATE DATABASE foo", parser.parsePlan("CREATE SCHEMA foo")) + assertEqual("DROP DATABASE foo", parser.parsePlan("DROP SCHEMA foo")) + assertEqual("ALTER DATABASE foo SET DBPROPERTIES ('x' = 'y')", + parser.parsePlan("ALTER SCHEMA foo SET DBPROPERTIES ('x' = 'y')")) + assertEqual("DESC DATABASE foo", parser.parsePlan("DESC SCHEMA foo")) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala index d305ce3e698ae..1640a9611ec35 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/UnsafeRowSerializerSuite.scala @@ -21,11 +21,14 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, File} import java.util.Properties import org.apache.spark._ +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Tests.TEST_MEMORY import org.apache.spark.memory.TaskMemoryManager import org.apache.spark.rdd.RDD import org.apache.spark.sql.{LocalSparkSession, Row, SparkSession} import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow} import org.apache.spark.sql.catalyst.expressions.{UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.execution.metric.SQLShuffleReadMetricsReporter import org.apache.spark.sql.types._ import org.apache.spark.storage.ShuffleBlockId import org.apache.spark.util.collection.ExternalSorter @@ -96,9 +99,10 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { test("SPARK-10466: external sorter spilling with unsafe row serializer") { val conf = new SparkConf() - .set("spark.shuffle.spill.initialMemoryThreshold", "1") - .set("spark.shuffle.sort.bypassMergeThreshold", "0") - .set("spark.testing.memory", "80000") + .set(SHUFFLE_SPILL_INITIAL_MEM_THRESHOLD, 1L) + .set(SHUFFLE_SORT_BYPASS_MERGE_THRESHOLD, 0) + .set(TEST_MEMORY, 80000L) + spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() val outputFile = File.createTempFile("test-unsafe-row-serializer-spill", "") outputFile.deleteOnExit() @@ -125,7 +129,7 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { } test("SPARK-10403: unsafe row serializer with SortShuffleManager") { - val conf = new SparkConf().set("spark.shuffle.manager", "sort") + val conf = new SparkConf().set(SHUFFLE_MANAGER, "sort") spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() val row = Row("Hello", 123) val unsafeRow = toUnsafeRow(row, Array(StringType, IntegerType)) @@ -137,7 +141,9 @@ class UnsafeRowSerializerSuite extends SparkFunSuite with LocalSparkSession { rowsRDD, new PartitionIdPassthrough(2), new UnsafeRowSerializer(2)) - val shuffled = new ShuffledRowRDD(dependency) + val shuffled = new ShuffledRowRDD( + dependency, + SQLShuffleReadMetricsReporter.createShuffleReadMetrics(spark.sparkContext)) shuffled.count() } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.scala new file mode 100644 index 0000000000000..f985386eee292 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/WholeStageCodegenSparkSubmitSuite.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 + +import org.scalatest.{Assertions, BeforeAndAfterEach, Matchers} +import org.scalatest.concurrent.TimeLimits + +import org.apache.spark.{SparkFunSuite, TestUtils} +import org.apache.spark.deploy.SparkSubmitSuite +import org.apache.spark.internal.Logging +import org.apache.spark.sql.{LocalSparkSession, QueryTest, Row, SparkSession} +import org.apache.spark.sql.functions.{array, col, count, lit} +import org.apache.spark.sql.types.IntegerType +import org.apache.spark.unsafe.Platform +import org.apache.spark.util.ResetSystemProperties + +// Due to the need to set driver's extraJavaOptions, this test needs to use actual SparkSubmit. +class WholeStageCodegenSparkSubmitSuite extends SparkFunSuite + with Matchers + with BeforeAndAfterEach + with ResetSystemProperties { + + test("Generated code on driver should not embed platform-specific constant") { + val unusedJar = TestUtils.createJarWithClasses(Seq.empty) + + // HotSpot JVM specific: Set up a local cluster with the driver/executor using mismatched + // settings of UseCompressedOops JVM option. + val argsForSparkSubmit = Seq( + "--class", WholeStageCodegenSparkSubmitSuite.getClass.getName.stripSuffix("$"), + "--master", "local-cluster[1,1,1024]", + "--driver-memory", "1g", + "--conf", "spark.ui.enabled=false", + "--conf", "spark.master.rest.enabled=false", + "--conf", "spark.driver.extraJavaOptions=-XX:-UseCompressedOops", + "--conf", "spark.executor.extraJavaOptions=-XX:+UseCompressedOops", + unusedJar.toString) + SparkSubmitSuite.runSparkSubmit(argsForSparkSubmit, "../..") + } +} + +object WholeStageCodegenSparkSubmitSuite extends Assertions with Logging { + + var spark: SparkSession = _ + + def main(args: Array[String]): Unit = { + TestUtils.configTestLog4j("INFO") + + spark = SparkSession.builder().getOrCreate() + + // Make sure the test is run where the driver and the executors uses different object layouts + val driverArrayHeaderSize = Platform.BYTE_ARRAY_OFFSET + val executorArrayHeaderSize = + spark.sparkContext.range(0, 1).map(_ => Platform.BYTE_ARRAY_OFFSET).collect.head.toInt + assert(driverArrayHeaderSize > executorArrayHeaderSize) + + val df = spark.range(71773).select((col("id") % lit(10)).cast(IntegerType) as "v") + .groupBy(array(col("v"))).agg(count(col("*"))) + val plan = df.queryExecution.executedPlan + assert(plan.find(_.isInstanceOf[WholeStageCodegenExec]).isDefined) + + val expectedAnswer = + Row(Array(0), 7178) :: + Row(Array(1), 7178) :: + Row(Array(2), 7178) :: + Row(Array(3), 7177) :: + Row(Array(4), 7177) :: + Row(Array(5), 7177) :: + Row(Array(6), 7177) :: + Row(Array(7), 7177) :: + Row(Array(8), 7177) :: + Row(Array(9), 7177) :: Nil + val result = df.collect + QueryTest.sameRows(result.toSeq, expectedAnswer) match { + case Some(errMsg) => fail(errMsg) + case _ => + } + } +} 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 b714dcd5269fc..483a046103383 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 @@ -17,15 +17,13 @@ package org.apache.spark.sql.execution -import org.apache.spark.metrics.source.CodegenMetrics -import org.apache.spark.sql.{QueryTest, Row, SaveMode} +import org.apache.spark.sql.{Dataset, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.expressions.codegen.{CodeAndComment, CodeGenerator} import org.apache.spark.sql.execution.aggregate.HashAggregateExec -import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec import org.apache.spark.sql.execution.joins.SortMergeJoinExec import org.apache.spark.sql.expressions.scalalang.typed -import org.apache.spark.sql.functions.{avg, broadcast, col, lit, max} +import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StringType, StructType} @@ -121,29 +119,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(ds.collect() === Array(("a", 10.0), ("b", 3.0), ("c", 1.0))) } - test("cache for primitive type should be in WholeStageCodegen with InMemoryTableScanExec") { - import testImplicits._ - - val dsInt = spark.range(3).cache() - dsInt.count() - val dsIntFilter = dsInt.filter(_ > 0) - val planInt = dsIntFilter.queryExecution.executedPlan - assert(planInt.collect { - case WholeStageCodegenExec(FilterExec(_, i: InMemoryTableScanExec)) if i.supportsBatch => () - }.length == 1) - assert(dsIntFilter.collect() === Array(1, 2)) - - // cache for string type is not supported for InMemoryTableScanExec - val dsString = spark.range(3).map(_.toString).cache() - dsString.count() - val dsStringFilter = dsString.filter(_ == "1") - val planString = dsStringFilter.queryExecution.executedPlan - assert(planString.collect { - case i: InMemoryTableScanExec if !i.supportsBatch => () - }.length == 1) - assert(dsStringFilter.collect() === Array("1")) - } - test("SPARK-19512 codegen for comparing structs is incorrect") { // this would raise CompileException before the fix spark.range(10) @@ -168,10 +143,10 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { .select("int") val plan = df.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && p.asInstanceOf[WholeStageCodegenExec].child.children(0) - .isInstanceOf[SortMergeJoinExec]).isDefined) + .isInstanceOf[SortMergeJoinExec]).isEmpty) assert(df.collect() === Array(Row(1), Row(2))) } } @@ -204,6 +179,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { wholeStageCodeGenExec.get.asInstanceOf[WholeStageCodegenExec].doCodeGen()._2 } + def genCode(ds: Dataset[_]): Seq[CodeAndComment] = { + val plan = ds.queryExecution.executedPlan + val wholeStageCodeGenExecs = plan.collect { case p: WholeStageCodegenExec => p } + assert(wholeStageCodeGenExecs.nonEmpty, "WholeStageCodegenExec is expected") + wholeStageCodeGenExecs.map(_.doCodeGen()._2) + } + ignore("SPARK-21871 check if we can get large code size when compiling too long functions") { val codeWithShortFunctions = genGroupByCode(3) val (_, maxCodeSize1) = CodeGenerator.compile(codeWithShortFunctions) @@ -213,25 +195,6 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(maxCodeSize2 > SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.defaultValue.get) } - ignore("bytecode of batch file scan exceeds the limit of WHOLESTAGE_HUGE_METHOD_LIMIT") { - import testImplicits._ - withTempPath { dir => - val path = dir.getCanonicalPath - val df = spark.range(10).select(Seq.tabulate(201) {i => ('id + i).as(s"c$i")} : _*) - df.write.mode(SaveMode.Overwrite).parquet(path) - - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "202", - SQLConf.WHOLESTAGE_HUGE_METHOD_LIMIT.key -> "2000") { - // wide table batch scan causes the byte code of codegen exceeds the limit of - // WHOLESTAGE_HUGE_METHOD_LIMIT - val df2 = spark.read.parquet(path) - val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(fileScan2.asInstanceOf[FileSourceScanExec].supportsBatch) - checkAnswer(df2, df) - } - } - } - test("Control splitting consume function by operators with config") { import testImplicits._ val df = spark.range(10).select(Seq.tabulate(2) {i => ('id + i).as(s"c$i")} : _*) @@ -283,9 +246,9 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { val df = spark.range(100) val join = df.join(df, "id") val plan = join.queryExecution.executedPlan - assert(!plan.find(p => + assert(plan.find(p => p.isInstanceOf[WholeStageCodegenExec] && - p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isDefined, + p.asInstanceOf[WholeStageCodegenExec].codegenStageId == 0).isEmpty, "codegen stage IDs should be preserved through ReuseExchange") checkAnswer(join, df.toDF) } @@ -295,18 +258,13 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { import testImplicits._ withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_USE_ID_IN_CLASS_NAME.key -> "true") { - val bytecodeSizeHisto = CodegenMetrics.METRIC_GENERATED_METHOD_BYTECODE_SIZE - - // the same query run twice should hit the codegen cache - spark.range(3).select('id + 2).collect - val after1 = bytecodeSizeHisto.getCount - spark.range(3).select('id + 2).collect - val after2 = bytecodeSizeHisto.getCount // same query shape as above, deliberately - // bytecodeSizeHisto's count is always monotonically increasing if new compilation to - // bytecode had occurred. If the count stayed the same that means we've got a cache hit. - assert(after1 == after2, "Should hit codegen cache. No new compilation to bytecode expected") - - // a different query can result in codegen cache miss, that's by design + // the same query run twice should produce identical code, which would imply a hit in + // the generated code cache. + val ds1 = spark.range(3).select('id + 2) + val code1 = genCode(ds1) + val ds2 = spark.range(3).select('id + 2) + val code2 = genCode(ds2) // same query shape as above, deliberately + assert(code1 == code2, "Should produce same code") } } @@ -319,4 +277,52 @@ class WholeStageCodegenSuite extends QueryTest with SharedSQLContext { assert(df.limit(1).collect() === Array(Row("bat", 8.0))) } } + + test("SPARK-25767: Lazy evaluated stream of expressions handled correctly") { + val a = Seq(1).toDF("key") + val b = Seq((1, "a")).toDF("key", "value") + val c = Seq(1).toDF("key") + + val ab = a.join(b, Stream("key"), "left") + val abc = ab.join(c, Seq("key"), "left") + + checkAnswer(abc, Row(1, "a")) + } + + test("SPARK-26680: Stream in groupBy does not cause StackOverflowError") { + val groupByCols = Stream(col("key")) + val df = Seq((1, 2), (2, 3), (1, 3)).toDF("key", "value") + .groupBy(groupByCols: _*) + .max("value") + + checkAnswer(df, Seq(Row(1, 3), Row(2, 3))) + } + + test("SPARK-26572: evaluate non-deterministic expressions for aggregate results") { + withSQLConf( + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> Long.MaxValue.toString, + SQLConf.SHUFFLE_PARTITIONS.key -> "1") { + val baseTable = Seq(1, 1).toDF("idx") + + // BroadcastHashJoinExec with a HashAggregateExec child containing no aggregate expressions + val distinctWithId = baseTable.distinct().withColumn("id", monotonically_increasing_id()) + .join(baseTable, "idx") + assert(distinctWithId.queryExecution.executedPlan.collectFirst { + case WholeStageCodegenExec( + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + }.isDefined) + checkAnswer(distinctWithId, Seq(Row(1, 0), Row(1, 0))) + + // BroadcastHashJoinExec with a HashAggregateExec child containing a Final mode aggregate + // expression + val groupByWithId = + baseTable.groupBy("idx").sum().withColumn("id", monotonically_increasing_id()) + .join(baseTable, "idx") + assert(groupByWithId.queryExecution.executedPlan.collectFirst { + case WholeStageCodegenExec( + ProjectExec(_, BroadcastHashJoinExec(_, _, _, _, _, _: HashAggregateExec, _))) => true + }.isDefined) + checkAnswer(groupByWithId, Seq(Row(1, 2, 0), Row(1, 2, 0))) + } + } } 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 new file mode 100644 index 0000000000000..d8efca323d519 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/adaptive/AdaptiveQueryExecSuite.scala @@ -0,0 +1,341 @@ +/* + * 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 org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.{ReusedSubqueryExec, SparkPlan} +import org.apache.spark.sql.execution.adaptive.rule.CoalescedShuffleReaderExec +import org.apache.spark.sql.execution.exchange.Exchange +import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext + +class AdaptiveQueryExecSuite extends QueryTest with SharedSQLContext { + import testImplicits._ + + setupTestData() + + private def runAdaptiveAndVerifyResult(query: String): (SparkPlan, SparkPlan) = { + val dfAdaptive = sql(query) + val planBefore = dfAdaptive.queryExecution.executedPlan + assert(planBefore.toString.startsWith("AdaptiveSparkPlan(isFinalPlan=false)")) + val result = dfAdaptive.collect() + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "false") { + val df = sql(query) + QueryTest.sameRows(result.toSeq, df.collect().toSeq) + } + val planAfter = dfAdaptive.queryExecution.executedPlan + assert(planAfter.toString.startsWith("AdaptiveSparkPlan(isFinalPlan=true)")) + val adaptivePlan = planAfter.asInstanceOf[AdaptiveSparkPlanExec].executedPlan + val exchanges = adaptivePlan.collect { + case e: Exchange => e + } + assert(exchanges.isEmpty, "The final plan should not contain any Exchange node.") + (dfAdaptive.queryExecution.sparkPlan, adaptivePlan) + } + + private def findTopLevelBroadcastHashJoin(plan: SparkPlan): Seq[BroadcastHashJoinExec] = { + plan.collect { + case j: BroadcastHashJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelBroadcastHashJoin(s.plan) + }.flatten + } + + private def findTopLevelSortMergeJoin(plan: SparkPlan): Seq[SortMergeJoinExec] = { + plan.collect { + case j: SortMergeJoinExec => Seq(j) + case s: QueryStageExec => findTopLevelSortMergeJoin(s.plan) + }.flatten + } + + private def findReusedExchange(plan: SparkPlan): Seq[ReusedQueryStageExec] = { + plan.collect { + case e: ReusedQueryStageExec => Seq(e) + case a: AdaptiveSparkPlanExec => findReusedExchange(a.executedPlan) + case s: QueryStageExec => findReusedExchange(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedExchange) + }.flatten + } + + private def findReusedSubquery(plan: SparkPlan): Seq[ReusedSubqueryExec] = { + plan.collect { + case e: ReusedSubqueryExec => Seq(e) + case s: QueryStageExec => findReusedSubquery(s.plan) + case p: SparkPlan => p.subqueries.flatMap(findReusedSubquery) + }.flatten + } + + test("Change merge join to broadcast join") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + } + } + + test("Change merge join to broadcast join and reduce number of shuffle partitions") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.REDUCE_POST_SHUFFLE_PARTITIONS_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.SHUFFLE_TARGET_POSTSHUFFLE_INPUT_SIZE.key -> "150") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a where value = '1'") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + + val shuffleReaders = adaptivePlan.collect { + case reader: CoalescedShuffleReaderExec => reader + } + assert(shuffleReaders.length === 1) + // The pre-shuffle partition size is [0, 72, 0, 72, 126] + shuffleReaders.foreach { reader => + assert(reader.outputPartitioning.numPartitions === 2) + } + } + } + + test("Scalar subquery") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData3)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + } + } + + test("Scalar subquery in later stages") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT * FROM testData join testData2 ON key = a " + + "where (value + a) = (SELECT max(a) from testData3)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + } + } + + test("multiple joins") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN testData3 t3 ON t2.n = t3.a + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON key = t4.a + |WHERE value = 1 + """.stripMargin) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 3) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 2) + } + } + + test("multiple joins with aggregate") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, sum(b) from testData3 group by a + | ) t3 ON t2.n = t3.a + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON key = t4.a + |WHERE value = 1 + """.stripMargin) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 3) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 2) + } + } + + test("multiple joins with aggregate 2") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "500") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + """ + |WITH t4 AS ( + | SELECT * FROM lowercaseData t2 JOIN ( + | select a, max(b) b from testData2 group by a + | ) t3 ON t2.n = t3.b + |) + |SELECT * FROM testData + |JOIN testData2 t2 ON key = t2.a + |JOIN t4 ON value = t4.a + |WHERE value = 1 + """.stripMargin) + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 3) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 3) + } + } + + test("Exchange reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT value FROM testData join testData2 ON key = a " + + "join (SELECT value v from testData join testData3 ON key = a) on value = v") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 3) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 2) + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + test("Exchange reuse with subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value = (SELECT max(a) from testData join testData2 ON key = a)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val ex = findReusedExchange(adaptivePlan) + assert(ex.size == 1) + } + } + + test("Exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + test("Subquery reuse") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "80") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (SELECT max(a) from testData join testData2 ON key = a) " + + "and a <= (SELECT max(a) from testData join testData2 ON key = a)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val ex = findReusedExchange(adaptivePlan) + assert(ex.isEmpty) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.nonEmpty) + } + } + + test("Broadcast exchange reuse across subqueries") { + withSQLConf( + SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "20000000", + SQLConf.SUBQUERY_REUSE_ENABLED.key -> "false") { + val (plan, adaptivePlan) = runAdaptiveAndVerifyResult( + "SELECT a FROM testData join testData2 ON key = a " + + "where value >= (" + + "SELECT /*+ broadcast(testData2) */ max(key) from testData join testData2 ON key = a) " + + "and a <= (" + + "SELECT /*+ broadcast(testData2) */ max(value) from testData join testData2 ON key = a)") + val smj = findTopLevelSortMergeJoin(plan) + assert(smj.size == 1) + val bhj = findTopLevelBroadcastHashJoin(adaptivePlan) + assert(bhj.size == 1) + val ex = findReusedExchange(adaptivePlan) + assert(ex.nonEmpty) + assert(ex.head.plan.isInstanceOf[BroadcastQueryStageExec]) + val sub = findReusedSubquery(adaptivePlan) + assert(sub.isEmpty) + } + } + + test("Union/Except/Intersect queries") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + runAdaptiveAndVerifyResult( + """ + |SELECT * FROM testData + |EXCEPT + |SELECT * FROM testData2 + |UNION ALL + |SELECT * FROM testData + |INTERSECT ALL + |SELECT * FROM testData2 + """.stripMargin) + } + } + + test("Subquery de-correlation in Union queries") { + withSQLConf(SQLConf.ADAPTIVE_EXECUTION_ENABLED.key -> "true") { + withTempView("a", "b") { + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("a") + Seq("a" -> 2, "b" -> 1).toDF("id", "num").createTempView("b") + + runAdaptiveAndVerifyResult( + """ + |SELECT id,num,source FROM ( + | SELECT id, num, 'a' as source FROM a + | UNION ALL + | SELECT id, num, 'b' as source FROM b + |) AS c WHERE c.id IN (SELECT id FROM b WHERE num = 2) + """.stripMargin) + } + } + } +} 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 c36872a6a5289..67c3fa0d3bf59 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 @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{BinaryType, Decimal, IntegerType, StructField, StructType} +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.util.Utils @@ -1190,7 +1191,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { test("max records in batch conf") { val totalRecords = 10 val maxRecordsPerBatch = 3 - spark.conf.set("spark.sql.execution.arrow.maxRecordsPerBatch", maxRecordsPerBatch) + spark.conf.set(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key, maxRecordsPerBatch) val df = spark.sparkContext.parallelize(1 to totalRecords, 2).toDF("i") val arrowBatches = df.toArrowBatchRdd.collect() assert(arrowBatches.length >= 4) @@ -1205,7 +1206,7 @@ class ArrowConvertersSuite extends SharedSQLContext with BeforeAndAfterAll { } assert(recordCount == totalRecords) allocator.close() - spark.conf.unset("spark.sql.execution.arrow.maxRecordsPerBatch") + spark.conf.unset(SQLConf.ARROW_EXECUTION_MAX_RECORDS_PER_BATCH.key) } testQuietly("unsupported types") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala index 86e0df2fea350..2776bc310fefe 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/AggregateBenchmark.scala @@ -22,7 +22,7 @@ import java.util.HashMap import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark import org.apache.spark.internal.config._ -import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.sql.catalyst.expressions.UnsafeRow import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap import org.apache.spark.sql.execution.vectorized.AggregateHashMap @@ -44,7 +44,7 @@ import org.apache.spark.unsafe.map.BytesToBytesMap */ object AggregateBenchmark extends SqlBasedBenchmark { - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("aggregate without grouping") { val N = 500L << 22 codegenBenchmark("agg w/o group", N) { @@ -83,7 +83,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -92,7 +92,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -119,7 +119,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -128,7 +128,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -154,7 +154,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -163,7 +163,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -189,7 +189,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -198,7 +198,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -234,7 +234,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "false", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "false") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "false") { f() } } @@ -243,7 +243,7 @@ object AggregateBenchmark extends SqlBasedBenchmark { withSQLConf( SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> "true", SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> "true", - "spark.sql.codegen.aggregate.map.vectorized.enable" -> "true") { + SQLConf.ENABLE_VECTORIZED_HASH_MAP.key -> "true") { f() } } @@ -473,10 +473,10 @@ object AggregateBenchmark extends SqlBasedBenchmark { value.pointTo(valueBytes, Platform.BYTE_ARRAY_OFFSET, 16) value.setInt(0, 555) val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val map = new LongToUnsafeRowMap(taskMemoryManager, 64) @@ -504,11 +504,11 @@ object AggregateBenchmark extends SqlBasedBenchmark { Seq("off", "on").foreach { heap => benchmark.addCase(s"BytesToBytesMap ($heap Heap)") { _ => val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, s"${heap == "off"}") .set(MEMORY_OFFHEAP_SIZE.key, "102400000"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val map = new BytesToBytesMap(taskMemoryManager, 1024, 64L << 20) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala deleted file mode 100644 index 76367cbbe5342..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWideTable.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.benchmark - -import org.apache.spark.benchmark.Benchmark - -/** - * Benchmark to measure performance for wide table. - * To run this: - * build/sbt "sql/test-only *benchmark.BenchmarkWideTable" - * - * Benchmarks in this file are skipped in normal builds. - */ -class BenchmarkWideTable extends BenchmarkWithCodegen { - - ignore("project on wide table") { - val N = 1 << 20 - val df = sparkSession.range(N) - val columns = (0 until 400).map{ i => s"id as id$i"} - val benchmark = new Benchmark("projection on wide table", N) - benchmark.addCase("wide table", numIters = 5) { iter => - df.selectExpr(columns : _*).queryExecution.toRdd.count() - } - benchmark.run() - - /** - * Here are some numbers with different split threshold: - * - * Split threshold methods Rate(M/s) Per Row(ns) - * 10 400 0.4 2279 - * 100 200 0.6 1554 - * 1k 37 0.9 1116 - * 8k 5 0.5 2025 - * 64k 1 0.0 21649 - */ - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWithCodegen.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWithCodegen.scala deleted file mode 100644 index 51331500479a3..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BenchmarkWithCodegen.scala +++ /dev/null @@ -1,54 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.benchmark - -import org.apache.spark.SparkFunSuite -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SparkSession - -/** - * Common base trait for micro benchmarks that are supposed to run standalone (i.e. not together - * with other test suites). - */ -private[benchmark] trait BenchmarkWithCodegen extends SparkFunSuite { - - lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName("microbenchmark") - .config("spark.sql.shuffle.partitions", 1) - .config("spark.sql.autoBroadcastJoinThreshold", 1) - .getOrCreate() - - /** Runs function `f` with whole stage codegen on and off. */ - def runBenchmark(name: String, cardinality: Long)(f: => Unit): Unit = { - val benchmark = new Benchmark(name, cardinality) - - benchmark.addCase(s"$name wholestage off", numIters = 2) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = false) - f - } - - benchmark.addCase(s"$name wholestage on", numIters = 5) { iter => - sparkSession.conf.set("spark.sql.codegen.wholeStage", value = true) - f - } - - benchmark.run() - } - -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala index 2f3caca849cdf..f727ebcf3fd1e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BloomFilterBenchmark.scala @@ -80,7 +80,7 @@ object BloomFilterBenchmark extends SqlBasedBenchmark { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { writeBenchmark() readBenchmark() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala index 2de516c19da9e..6925bdd72674f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/BuiltInDataSourceWriteBenchmark.scala @@ -16,64 +16,44 @@ */ package org.apache.spark.sql.execution.benchmark +import org.apache.spark.sql.internal.SQLConf + /** * Benchmark to measure built-in data sources write performance. - * By default it measures 4 data source format: Parquet, ORC, JSON, CSV. Run it with spark-submit: - * spark-submit --class - * Or with sbt: - * build/sbt "sql/test:runMain " + * To run this benchmark: + * {{{ + * By default it measures 4 data source format: Parquet, ORC, JSON, CSV. + * 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/BuiltInDataSourceWriteBenchmark-results.txt". + * + * To measure specified formats, run it with arguments. + * 1. without sbt: + * bin/spark-submit --class format1 [format2] [...] + * 2. build/sbt "sql/test:runMain format1 [format2] [...]" + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt + * "sql/test:runMain format1 [format2] [...]" + * Results will be written to "benchmarks/BuiltInDataSourceWriteBenchmark-results.txt". + * }}} * - * To measure specified formats, run it with arguments: - * spark-submit --class format1 [format2] [...] - * Or with sbt: - * build/sbt "sql/test:runMain format1 [format2] [...]" */ object BuiltInDataSourceWriteBenchmark extends DataSourceWriteBenchmark { - def main(args: Array[String]): Unit = { - val formats: Seq[String] = if (args.isEmpty) { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val formats: Seq[String] = if (mainArgs.isEmpty) { Seq("Parquet", "ORC", "JSON", "CSV") } else { - args + mainArgs } - spark.conf.set("spark.sql.parquet.compression.codec", "snappy") - spark.conf.set("spark.sql.orc.compression.codec", "snappy") - /* - Intel(R) Core(TM) i7-6920HQ CPU @ 2.90GHz - Parquet writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1815 / 1932 8.7 115.4 1.0X - Output Single Double Column 1877 / 1878 8.4 119.3 1.0X - Output Int and String Column 6265 / 6543 2.5 398.3 0.3X - Output Partitions 4067 / 4457 3.9 258.6 0.4X - Output Buckets 5608 / 5820 2.8 356.6 0.3X - - ORC writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1201 / 1239 13.1 76.3 1.0X - Output Single Double Column 1542 / 1600 10.2 98.0 0.8X - Output Int and String Column 6495 / 6580 2.4 412.9 0.2X - Output Partitions 3648 / 3842 4.3 231.9 0.3X - Output Buckets 5022 / 5145 3.1 319.3 0.2X - - JSON writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 1988 / 2093 7.9 126.4 1.0X - Output Single Double Column 2854 / 2911 5.5 181.4 0.7X - Output Int and String Column 6467 / 6653 2.4 411.1 0.3X - Output Partitions 4548 / 5055 3.5 289.1 0.4X - Output Buckets 5664 / 5765 2.8 360.1 0.4X + spark.conf.set(SQLConf.PARQUET_COMPRESSION.key, "snappy") + spark.conf.set(SQLConf.ORC_COMPRESSION.key, "snappy") - CSV writer benchmark: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - Output Single Int Column 3025 / 3190 5.2 192.3 1.0X - Output Single Double Column 3575 / 3634 4.4 227.3 0.8X - Output Int and String Column 7313 / 7399 2.2 464.9 0.4X - Output Partitions 5105 / 5190 3.1 324.6 0.6X - Output Buckets 6986 / 6992 2.3 444.1 0.4X - */ formats.foreach { format => - runBenchmark(format) + runBenchmark(s"$format writer benchmark") { + runDataSourceBenchmark(format) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala index a1e7f9e36f4b0..bd2470ee20660 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceReadBenchmark.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, DataFrameWriter, Row, SparkSession} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.plans.SQLHelper @@ -50,13 +51,12 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { .set("spark.master", "local[1]") .setIfMissing("spark.driver.memory", "3g") .setIfMissing("spark.executor.memory", "3g") - .setIfMissing("spark.ui.enabled", "false") + .setIfMissing(UI_ENABLED, false) val spark = SparkSession.builder.config(conf).getOrCreate() // Set default configs. Individual cases will change them if necessary. spark.conf.set(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key, "true") - spark.conf.set(SQLConf.ORC_COPY_BATCH_TO_SPARK.key, "false") spark.conf.set(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key, "true") spark.conf.set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") @@ -139,12 +139,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(id) FROM orcTable").collect() } - sqlBenchmark.addCase("SQL ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(id) FROM orcTable").collect() - } - } - sqlBenchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(id) FROM orcTable").collect() @@ -261,12 +255,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").collect() } - benchmark.addCase("SQL ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").collect() - } - } - benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(c1), sum(length(c2)) FROM orcTable").collect() @@ -312,12 +300,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("select sum(length(c1)) from orcTable").collect() } - benchmark.addCase("SQL ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("select sum(length(c1)) from orcTable").collect() - } - } - benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("select sum(length(c1)) from orcTable").collect() @@ -361,12 +343,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(id) FROM orcTable").collect() } - benchmark.addCase("Data column - ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(id) FROM orcTable").collect() - } - } - benchmark.addCase("Data column - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(id) FROM orcTable").collect() @@ -395,12 +371,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(p) FROM orcTable").collect() } - benchmark.addCase("Partition column - ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(p) FROM orcTable").collect() - } - } - benchmark.addCase("Partition column - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(p) FROM orcTable").collect() @@ -429,12 +399,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(p), sum(id) FROM orcTable").collect() } - benchmark.addCase("Both column - ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(p), sum(id) FROM orcTable").collect() - } - } - benchmark.addCase("Both columns - ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT sum(p), sum(id) FROM orcTable").collect() @@ -447,7 +411,9 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { } def stringWithNullsScanBenchmark(values: Int, fractionOfNulls: Double): Unit = { - val benchmark = new Benchmark("String with Nulls Scan", values, output = output) + val percentageOfNulls = fractionOfNulls * 100 + val benchmark = + new Benchmark(s"String with Nulls Scan ($percentageOfNulls%)", values, output = output) withTempPath { dir => withTempTable("t1", "csvTable", "jsonTable", "parquetTable", "orcTable") { @@ -511,13 +477,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } - benchmark.addCase("SQL ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT SUM(LENGTH(c2)) FROM orcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() - } - } - benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql("SELECT SUM(LENGTH(c2)) FROM orcTable " + @@ -568,12 +527,6 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql(s"SELECT sum(c$middle) FROM orcTable").collect() } - benchmark.addCase("SQL ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql(s"SELECT sum(c$middle) FROM orcTable").collect() - } - } - benchmark.addCase("SQL ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { spark.sql(s"SELECT sum(c$middle) FROM orcTable").collect() @@ -585,7 +538,7 @@ object DataSourceReadBenchmark extends BenchmarkBase with SQLHelper { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("SQL Single Numeric Column Scan") { Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => numericScanBenchmark(1024 * 1024 * 15, dataType) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala index 994d6b5b7d334..405d60794ede0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DataSourceWriteBenchmark.scala @@ -16,18 +16,9 @@ */ package org.apache.spark.sql.execution.benchmark -import org.apache.spark.SparkConf import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.SparkSession -import org.apache.spark.sql.internal.SQLConf -trait DataSourceWriteBenchmark { - val conf = new SparkConf() - .setAppName("DataSourceWriteBenchmark") - .setIfMissing("spark.master", "local[1]") - .set(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key, "true") - - val spark = SparkSession.builder.config(conf).getOrCreate() +trait DataSourceWriteBenchmark extends SqlBasedBenchmark { val tempTable = "temp" val numRows = 1024 * 1024 * 15 @@ -75,7 +66,7 @@ trait DataSourceWriteBenchmark { } } - def runBenchmark(format: String): Unit = { + def runDataSourceBenchmark(format: String): Unit = { val tableInt = "tableInt" val tableDouble = "tableDouble" val tableIntString = "tableIntString" @@ -84,7 +75,7 @@ trait DataSourceWriteBenchmark { withTempTable(tempTable) { spark.range(numRows).createOrReplaceTempView(tempTable) withTable(tableInt, tableDouble, tableIntString, tablePartition, tableBucket) { - val benchmark = new Benchmark(s"$format writer benchmark", numRows) + val benchmark = new Benchmark(s"$format writer benchmark", numRows, output = output) writeNumeric(tableInt, format, benchmark, "Int") writeNumeric(tableDouble, format, benchmark, "Double") writeIntString(tableIntString, format, benchmark) 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 new file mode 100644 index 0000000000000..df0f87e483cdc --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/DateTimeBenchmark.scala @@ -0,0 +1,149 @@ +/* + * 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.sql.Timestamp + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for date and timestamp functions. + * 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/DateTimeBenchmark-results.txt". + * }}} + */ +object DateTimeBenchmark extends SqlBasedBenchmark { + private def doBenchmark(cardinality: Int, exprs: String*): Unit = { + spark.range(cardinality).selectExpr(exprs: _*).write.format("noop").save() + } + + private def run(cardinality: Int, name: String, exprs: String*): Unit = { + codegenBenchmark(name, cardinality) { + doBenchmark(cardinality, exprs: _*) + } + } + + private def run(cardinality: Int, func: String): Unit = { + codegenBenchmark(s"$func of timestamp", cardinality) { + doBenchmark(cardinality, s"$func(cast(id as timestamp))") + } + } + + 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") { + withSQLConf(SQLConf.UTC_TIMESTAMP_FUNC_ENABLED.key -> "true") { + 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-', cast(mod(id, 25) as string))" + 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)) + .write.format("noop").save() + } + 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() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala index cf05ca3361711..b040243717137 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/FilterPushdownBenchmark.scala @@ -23,6 +23,7 @@ import scala.util.Random import org.apache.spark.SparkConf import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.internal.config.UI._ import org.apache.spark.sql.{DataFrame, SparkSession} import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.functions.monotonically_increasing_id @@ -48,7 +49,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { .set("spark.master", "local[1]") .setIfMissing("spark.driver.memory", "3g") .setIfMissing("spark.executor.memory", "3g") - .setIfMissing("spark.ui.enabled", "false") + .setIfMissing(UI_ENABLED, false) .setIfMissing("orc.compression", "snappy") .setIfMissing("spark.sql.parquet.compression.codec", "snappy") @@ -198,7 +199,7 @@ object FilterPushdownBenchmark extends BenchmarkBase with SQLHelper { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Pushdown for many distinct value case") { withTempPath { dir => withTempTable("orcTable", "parquetTable") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala new file mode 100644 index 0000000000000..ebe278bff7d86 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/HashedRelationMetricsBenchmark.scala @@ -0,0 +1,84 @@ +/* + * 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 org.apache.spark.SparkConf +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED +import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{BoundReference, UnsafeProjection} +import org.apache.spark.sql.execution.joins.LongToUnsafeRowMap +import org.apache.spark.sql.types.LongType + +/** + * Benchmark to measure metrics performance at HashedRelation. + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * 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/HashedRelationMetricsBenchmark-results.txt". + * }}} + */ +object HashedRelationMetricsBenchmark extends SqlBasedBenchmark { + + def benchmarkLongToUnsafeRowMapMetrics(numRows: Int): Unit = { + runBenchmark("LongToUnsafeRowMap metrics") { + val benchmark = new Benchmark("LongToUnsafeRowMap metrics", numRows, output = output) + benchmark.addCase("LongToUnsafeRowMap") { iter => + val taskMemoryManager = new TaskMemoryManager( + new UnifiedMemoryManager( + new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), + Long.MaxValue, + Long.MaxValue / 2, + 1), + 0) + val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) + + val keys = Range.Long(0, numRows, 1) + val map = new LongToUnsafeRowMap(taskMemoryManager, 1) + keys.foreach { k => + map.append(k, unsafeProj(InternalRow(k))) + } + map.optimize() + + val threads = (0 to 100).map { _ => + val thread = new Thread { + override def run: Unit = { + val row = unsafeProj(InternalRow(0L)).copy() + keys.foreach { k => + assert(map.getValue(k, row) eq row) + assert(row.getLong(0) == k) + } + } + } + thread.start() + thread + } + threads.map(_.join()) + map.free() + } + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + benchmarkLongToUnsafeRowMapMetrics(500000) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala new file mode 100644 index 0000000000000..611f582b66605 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/InExpressionBenchmark.scala @@ -0,0 +1,252 @@ +/* + * 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 org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.functions.{array, struct} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.types._ + +/** + * A benchmark that compares the performance of different ways to evaluate SQL IN expressions. + * + * Specifically, this class compares the if-based approach, which might iterate through all items + * inside the IN value list, to other options with better worst-case time complexities (e.g., sets). + * + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * 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/InExpressionBenchmark-results.txt". + * }}} + */ +object InExpressionBenchmark extends SqlBasedBenchmark { + + import spark.implicits._ + + private def runByteBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems bytes" + val values = (Byte.MinValue until Byte.MinValue + numItems).map(v => s"${v}Y") + val df = spark.range(0, numRows).select($"id".cast(ByteType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runShortBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems shorts" + val values = (1 to numItems).map(v => s"${v}S") + val df = spark.range(0, numRows).select($"id".cast(ShortType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runNonCompactShortBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val step = (Short.MaxValue.toInt - Short.MinValue.toInt) / numItems + val maxValue = Short.MinValue + numItems * step + val rangeSize = maxValue - Short.MinValue + require(isLookupSwitch(rangeSize, numItems)) + val name = s"$numItems shorts (non-compact)" + val values = (Short.MinValue until maxValue by step).map(v => s"${v}S") + val df = spark.range(0, numRows).select($"id".cast(ShortType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runIntBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems ints" + val values = 1 to numItems + val df = spark.range(0, numRows).select($"id".cast(IntegerType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runNonCompactIntBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val step = (Int.MaxValue.toLong - Int.MinValue.toLong) / numItems + val maxValue = Int.MinValue + numItems * step + val rangeSize = maxValue - Int.MinValue + require(isLookupSwitch(rangeSize, numItems)) + val name = s"$numItems ints (non-compact)" + val values = Int.MinValue until maxValue.toInt by step.toInt + val df = spark.range(0, numRows).select($"id".cast(IntegerType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runLongBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems longs" + val values = (1 to numItems).map(v => s"${v}L") + val df = spark.range(0, numRows).toDF("id") + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runFloatBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems floats" + val values = (1 to numItems).map(v => s"CAST($v AS float)") + val df = spark.range(0, numRows).select($"id".cast(FloatType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runDoubleBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems doubles" + val values = (1 to numItems).map(v => s"$v.0D") + val df = spark.range(0, numRows).select($"id".cast(DoubleType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runSmallDecimalBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems small decimals" + val values = (1 to numItems).map(v => s"CAST($v AS decimal(12, 1))") + val df = spark.range(0, numRows).select($"id".cast(DecimalType(12, 1))) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runLargeDecimalBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems large decimals" + val values = (1 to numItems).map(v => s"9223372036854775812.10539$v") + val df = spark.range(0, numRows).select($"id".cast(DecimalType(30, 7))) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runStringBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems strings" + val values = (1 to numItems).map(n => s"'$n'") + val df = spark.range(0, numRows).select($"id".cast(StringType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runTimestampBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems timestamps" + val values = (1 to numItems).map(m => s"CAST('1970-01-01 01:00:00.$m' AS timestamp)") + val df = spark.range(0, numRows).select($"id".cast(TimestampType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runDateBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems dates" + val values = (1 to numItems).map(n => 1970 + n).map(y => s"CAST('$y-01-01' AS date)") + val df = spark.range(0, numRows).select($"id".cast(TimestampType).cast(DateType)) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runArrayBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems arrays" + val values = (1 to numItems).map(i => s"array($i)") + val df = spark.range(0, numRows).select(array($"id").as("id")) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runStructBenchmark(numItems: Int, numRows: Long, minNumIters: Int): Unit = { + val name = s"$numItems structs" + val values = (1 to numItems).map(i => s"struct($i)") + val df = spark.range(0, numRows).select(struct($"id".as("col1")).as("id")) + runBenchmark(name, df, values, numRows, minNumIters) + } + + private def runBenchmark( + name: String, + df: DataFrame, + values: Seq[Any], + numRows: Long, + minNumIters: Int): Unit = { + + val benchmark = new Benchmark(name, numRows, minNumIters, output = output) + + df.createOrReplaceTempView("t") + + def testClosure(): Unit = { + val df = spark.sql(s"SELECT * FROM t WHERE id IN (${values.mkString(",")})") + df.queryExecution.toRdd.foreach(_ => Unit) + } + + benchmark.addCase("In expression") { _ => + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> values.size.toString) { + testClosure() + } + } + + benchmark.addCase("InSet expression") { _ => + withSQLConf(SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD.key -> "1") { + testClosure() + } + } + + benchmark.run() + } + + // this logic is derived from visitSwitch in com.sun.tools.javac.jvm.Gen + private def isLookupSwitch(rangeSize: Long, numLabels: Int): Boolean = { + val tableSpaceCost = 4 + rangeSize + val tableTimeCost = 3 + val lookupSpaceCost = 3 + 2 * numLabels + val lookupTimeCost = numLabels + lookupSpaceCost + 3 * lookupTimeCost < tableSpaceCost + 3 * tableTimeCost + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val smallNumItemsSeq = Seq(5, 10, 25, 50, 100, 200) + val largeNumItemsSeq = Seq(300, 400, 500) + val largeNumRows = 10000000 + val smallNumRows = 1000000 + val minNumIters = 5 + + runBenchmark("In Expression Benchmark") { + smallNumItemsSeq.foreach { numItems => + runByteBenchmark(numItems, largeNumRows, minNumIters) + } + (smallNumItemsSeq ++ largeNumItemsSeq).foreach { numItems => + runShortBenchmark(numItems, largeNumRows, minNumIters) + } + (smallNumItemsSeq ++ largeNumItemsSeq).foreach { numItems => + runNonCompactShortBenchmark(numItems, largeNumRows, minNumIters) + } + (smallNumItemsSeq ++ largeNumItemsSeq).foreach { numItems => + runIntBenchmark(numItems, largeNumRows, minNumIters) + } + (smallNumItemsSeq ++ largeNumItemsSeq).foreach { numItems => + runNonCompactIntBenchmark(numItems, largeNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runLongBenchmark(numItems, largeNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runFloatBenchmark(numItems, largeNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runDoubleBenchmark(numItems, largeNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runSmallDecimalBenchmark(numItems, smallNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runLargeDecimalBenchmark(numItems, smallNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runStringBenchmark(numItems, smallNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runTimestampBenchmark(numItems, largeNumRows, minNumIters) + } + (smallNumItemsSeq ++ largeNumItemsSeq).foreach { numItems => + runDateBenchmark(numItems, largeNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runArrayBenchmark(numItems, smallNumRows, minNumIters) + } + smallNumItemsSeq.foreach { numItems => + runStructBenchmark(numItems, smallNumRows, minNumIters) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala index 37744dccc06f8..ad81711a13947 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/JoinBenchmark.scala @@ -19,229 +19,163 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.sql.execution.joins._ import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.IntegerType /** - * Benchmark to measure performance for aggregate primitives. - * To run this: - * build/sbt "sql/test-only *benchmark.JoinBenchmark" - * - * Benchmarks in this file are skipped in normal builds. + * Benchmark to measure performance for joins. + * 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/JoinBenchmark-results.txt". + * }}} */ -class JoinBenchmark extends BenchmarkWithCodegen { +object JoinBenchmark extends SqlBasedBenchmark { - ignore("broadcast hash join, long key") { + def broadcastHashJoinLongKey(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) - runBenchmark("Join w long", N) { - val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k")) + val dim = broadcast(spark.range(M).selectExpr("id as k", "cast(id as string) as v")) + codegenBenchmark("Join w long", N) { + val df = spark.range(N).join(dim, (col("id") % M) === col("k")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - Join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------- - Join w long codegen=false 3002 / 3262 7.0 143.2 1.0X - Join w long codegen=true 321 / 371 65.3 15.3 9.3X - */ } - ignore("broadcast hash join, long key with duplicates") { + def broadcastHashJoinLongKeyWithDuplicates(): Unit = { val N = 20 << 20 val M = 1 << 16 - - val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) - runBenchmark("Join w long duplicated", N) { - val dim = broadcast(sparkSession.range(M).selectExpr("cast(id/10 as long) as k")) - val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k")) + val dim = broadcast(spark.range(M).selectExpr("cast(id/10 as long) as k")) + codegenBenchmark("Join w long duplicated", N) { + val df = spark.range(N).join(dim, (col("id") % M) === col("k")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *Join w long duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *Join w long duplicated codegen=false 3446 / 3478 6.1 164.3 1.0X - *Join w long duplicated codegen=true 322 / 351 65.2 15.3 10.7X - */ } - ignore("broadcast hash join, two int key") { + def broadcastHashJoinTwoIntKey(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim2 = broadcast(sparkSession.range(M) + val dim2 = broadcast(spark.range(M) .selectExpr("cast(id as int) as k1", "cast(id as int) as k2", "cast(id as string) as v")) - runBenchmark("Join w 2 ints", N) { - val df = sparkSession.range(N).join(dim2, + codegenBenchmark("Join w 2 ints", N) { + val df = spark.range(N).join(dim2, (col("id") % M).cast(IntegerType) === col("k1") && (col("id") % M).cast(IntegerType) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *Join w 2 ints: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *Join w 2 ints codegen=false 4426 / 4501 4.7 211.1 1.0X - *Join w 2 ints codegen=true 791 / 818 26.5 37.7 5.6X - */ } - ignore("broadcast hash join, two long key") { + def broadcastHashJoinTwoLongKey(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim3 = broadcast(sparkSession.range(M) + val dim3 = broadcast(spark.range(M) .selectExpr("id as k1", "id as k2", "cast(id as string) as v")) - runBenchmark("Join w 2 longs", N) { - val df = sparkSession.range(N).join(dim3, + codegenBenchmark("Join w 2 longs", N) { + val df = spark.range(N).join(dim3, (col("id") % M) === col("k1") && (col("id") % M) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *Join w 2 longs: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *Join w 2 longs codegen=false 5905 / 6123 3.6 281.6 1.0X - *Join w 2 longs codegen=true 2230 / 2529 9.4 106.3 2.6X - */ } - ignore("broadcast hash join, two long key with duplicates") { + def broadcastHashJoinTwoLongKeyWithDuplicates(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim4 = broadcast(sparkSession.range(M) + val dim4 = broadcast(spark.range(M) .selectExpr("cast(id/10 as long) as k1", "cast(id/10 as long) as k2")) - runBenchmark("Join w 2 longs duplicated", N) { - val df = sparkSession.range(N).join(dim4, + codegenBenchmark("Join w 2 longs duplicated", N) { + val df = spark.range(N).join(dim4, (col("id") bitwiseAND M) === col("k1") && (col("id") bitwiseAND M) === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *Join w 2 longs duplicated: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *Join w 2 longs duplicated codegen=false 6420 / 6587 3.3 306.1 1.0X - *Join w 2 longs duplicated codegen=true 2080 / 2139 10.1 99.2 3.1X - */ } - ignore("broadcast hash join, outer join long key") { + def broadcastHashJoinOuterJoinLongKey(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) - runBenchmark("outer join w long", N) { - val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "left") + val dim = broadcast(spark.range(M).selectExpr("id as k", "cast(id as string) as v")) + codegenBenchmark("outer join w long", N) { + val df = spark.range(N).join(dim, (col("id") % M) === col("k"), "left") assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *outer join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *outer join w long codegen=false 3055 / 3189 6.9 145.7 1.0X - *outer join w long codegen=true 261 / 276 80.5 12.4 11.7X - */ } - ignore("broadcast hash join, semi join long key") { + def broadcastHashJoinSemiJoinLongKey(): Unit = { val N = 20 << 20 val M = 1 << 16 - val dim = broadcast(sparkSession.range(M).selectExpr("id as k", "cast(id as string) as v")) - runBenchmark("semi join w long", N) { - val df = sparkSession.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi") + val dim = broadcast(spark.range(M).selectExpr("id as k", "cast(id as string) as v")) + codegenBenchmark("semi join w long", N) { + val df = spark.range(N).join(dim, (col("id") % M) === col("k"), "leftsemi") assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[BroadcastHashJoinExec]).isDefined) df.count() } - - /* - *Java HotSpot(TM) 64-Bit Server VM 1.7.0_60-b19 on Mac OS X 10.9.5 - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *semi join w long: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *semi join w long codegen=false 1912 / 1990 11.0 91.2 1.0X - *semi join w long codegen=true 237 / 244 88.3 11.3 8.1X - */ } - ignore("sort merge join") { + def sortMergeJoin(): Unit = { val N = 2 << 20 - runBenchmark("merge join", N) { - val df1 = sparkSession.range(N).selectExpr(s"id * 2 as k1") - val df2 = sparkSession.range(N).selectExpr(s"id * 3 as k2") + codegenBenchmark("sort merge join", N) { + val df1 = spark.range(N).selectExpr(s"id * 2 as k1") + val df2 = spark.range(N).selectExpr(s"id * 3 as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) df.count() } - - /* - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *merge join codegen=false 1588 / 1880 1.3 757.1 1.0X - *merge join codegen=true 1477 / 1531 1.4 704.2 1.1X - */ } - ignore("sort merge join with duplicates") { + def sortMergeJoinWithDuplicates(): Unit = { val N = 2 << 20 - runBenchmark("sort merge join", N) { - val df1 = sparkSession.range(N) + codegenBenchmark("sort merge join with duplicates", N) { + val df1 = spark.range(N) .selectExpr(s"(id * 15485863) % ${N*10} as k1") - val df2 = sparkSession.range(N) + val df2 = spark.range(N) .selectExpr(s"(id * 15485867) % ${N*10} as k2") val df = df1.join(df2, col("k1") === col("k2")) assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[SortMergeJoinExec]).isDefined) df.count() } - - /* - *Intel(R) Core(TM) i7-4558U CPU @ 2.80GHz - *sort merge join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *sort merge join codegen=false 3626 / 3667 0.6 1728.9 1.0X - *sort merge join codegen=true 3405 / 3438 0.6 1623.8 1.1X - */ } - ignore("shuffle hash join") { - val N = 4 << 20 - sparkSession.conf.set("spark.sql.shuffle.partitions", "2") - sparkSession.conf.set("spark.sql.autoBroadcastJoinThreshold", "10000000") - sparkSession.conf.set("spark.sql.join.preferSortMergeJoin", "false") - runBenchmark("shuffle hash join", N) { - val df1 = sparkSession.range(N).selectExpr(s"id as k1") - val df2 = sparkSession.range(N / 3).selectExpr(s"id * 3 as k2") - val df = df1.join(df2, col("k1") === col("k2")) - assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined) - df.count() + def shuffleHashJoin(): Unit = { + val N: Long = 4 << 20 + withSQLConf( + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "10000000", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { + codegenBenchmark("shuffle hash join", N) { + val df1 = spark.range(N).selectExpr(s"id as k1") + val df2 = spark.range(N / 3).selectExpr(s"id * 3 as k2") + val df = df1.join(df2, col("k1") === col("k2")) + assert(df.queryExecution.sparkPlan.find(_.isInstanceOf[ShuffledHashJoinExec]).isDefined) + df.count() + } } + } - /* - *Java HotSpot(TM) 64-Bit Server VM 1.8.0_60-b27 on Windows 7 6.1 - *Intel64 Family 6 Model 94 Stepping 3, GenuineIntel - *shuffle hash join: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - *------------------------------------------------------------------------------------------- - *shuffle hash join codegen=false 2005 / 2010 2.1 478.0 1.0X - *shuffle hash join codegen=true 1773 / 1792 2.4 422.7 1.1X - */ + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Join Benchmark") { + broadcastHashJoinLongKey() + broadcastHashJoinLongKeyWithDuplicates() + broadcastHashJoinTwoIntKey() + broadcastHashJoinTwoLongKey() + broadcastHashJoinTwoLongKeyWithDuplicates() + broadcastHashJoinOuterJoinLongKey() + broadcastHashJoinSemiJoinLongKey() + sortMergeJoin() + sortMergeJoinWithDuplicates() + shuffleHashJoin() + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala index 43380869fefe4..bafc0337bdc0e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/MiscBenchmark.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.execution.benchmark import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf /** * Benchmark to measure whole stage codegen performance. @@ -130,6 +131,20 @@ object MiscBenchmark extends SqlBasedBenchmark { df.selectExpr("*", "explode(arr) as arr_col") .select("col", "arr_col.*").count } + + withSQLConf(SQLConf.NESTED_PRUNING_ON_EXPRESSIONS.key -> "true") { + codegenBenchmark("generate big nested struct array", M) { + import spark.implicits._ + val df = spark.sparkContext.parallelize(Seq(("1", + Array.fill(M)({ + val i = math.random + (i.toString, (i + 1).toString, (i + 2).toString, (i + 3).toString) + })))).toDF("col", "arr") + .selectExpr("col", "struct(col, arr) as st") + .selectExpr("col", "st.col as col1", "explode(st.arr) as arr_col") + df.collect() + } + } } } @@ -148,7 +163,7 @@ object MiscBenchmark extends SqlBasedBenchmark { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { filterAndAggregateWithoutGroup(500L << 22) limitAndAggregateWithoutGroup(500L << 20) sample(500 << 18) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala new file mode 100644 index 0000000000000..96f90f29707d2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/NestedSchemaPruningBenchmark.scala @@ -0,0 +1,192 @@ +/* + * 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 org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf + +/** + * The base class for synthetic benchmark for nested schema pruning performance. + */ +abstract class NestedSchemaPruningBenchmark extends SqlBasedBenchmark { + + import spark.implicits._ + + val dataSourceName: String + val benchmarkName: String + + protected val N = 1000000 + protected val numIters = 10 + + // We use `col1 BIGINT, col2 STRUCT<_1: BIGINT, _2: STRING>, + // col3 ARRAY>` as a test schema. + // col1, col2._1 and col3._1 are used for comparision. col2._2 and col3._2 mimics the burden + // for the other columns + private val df = spark + .range(N * 10) + .sample(false, 0.1) + .map { x => + val col3 = (0 until 5).map(i => (x + i, s"$x" * 5)) + (x, (x, s"$x" * 100), col3) + }.toDF("col1", "col2", "col3") + + private def addCase(benchmark: Benchmark, name: String, sql: String): Unit = { + benchmark.addCase(name) { _ => + spark.sql(sql).write.format("noop").save() + } + } + + protected def selectBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Selection", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", "SELECT col1 FROM (SELECT col1 FROM t1)") + addCase(benchmark, "Nested column", "SELECT col2._1 FROM (SELECT col2 FROM t2)") + addCase(benchmark, "Nested column in array", "SELECT col3._1 FROM (SELECT col3 FROM t3)") + + benchmark.run() + } + } + + protected def limitBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Limiting", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", + s"SELECT col1 FROM (SELECT col1 FROM t1 LIMIT ${Int.MaxValue})") + addCase(benchmark, "Nested column", + s"SELECT col2._1 FROM (SELECT col2 FROM t2 LIMIT ${Int.MaxValue})") + addCase(benchmark, "Nested column in array", + s"SELECT col3._1 FROM (SELECT col3 FROM t3 LIMIT ${Int.MaxValue})") + + benchmark.run() + } + } + + protected def repartitionBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Repartitioning", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", + s"SELECT col1 FROM (SELECT /*+ REPARTITION(1) */ col1 FROM t1)") + addCase(benchmark, "Nested column", + s"SELECT col2._1 FROM (SELECT /*+ REPARTITION(1) */ col2 FROM t2)") + addCase(benchmark, "Nested column in array", + s"SELECT col3._1 FROM (SELECT /*+ REPARTITION(1) */ col3 FROM t3)") + + benchmark.run() + } + } + + protected def repartitionByExprBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Repartitioning by exprs", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", + s"SELECT col1 FROM (SELECT col1 FROM t1 DISTRIBUTE BY col1)") + addCase(benchmark, "Nested column", + s"SELECT col2._1 FROM (SELECT col2 FROM t2 DISTRIBUTE BY col2._1)") + addCase(benchmark, "Nested column in array", + s"SELECT col3._1 FROM (SELECT col3 FROM t3 DISTRIBUTE BY col3._1)") + + benchmark.run() + } + } + + protected def sampleBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Sample", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", + s"SELECT col1 FROM (SELECT col1 FROM t1 TABLESAMPLE(100 percent))") + addCase(benchmark, "Nested column", + s"SELECT col2._1 FROM (SELECT col2 FROM t2 TABLESAMPLE(100 percent))") + addCase(benchmark, "Nested column in array", + s"SELECT col3._1 FROM (SELECT col3 FROM t3 TABLESAMPLE(100 percent))") + + benchmark.run() + } + } + + protected def sortBenchmark(numRows: Int, numIters: Int): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + Seq(1, 2, 3).foreach { i => + df.write.format(dataSourceName).save(path + s"/$i") + spark.read.format(dataSourceName).load(path + s"/$i").createOrReplaceTempView(s"t$i") + } + + val benchmark = new Benchmark(s"Sorting", numRows, numIters, output = output) + + addCase(benchmark, "Top-level column", "SELECT col1 FROM t1 ORDER BY col1") + addCase(benchmark, "Nested column", "SELECT col2._1 FROM t2 ORDER BY col2._1") + addCase(benchmark, "Nested column in array", "SELECT col3._1 FROM t3 ORDER BY col3._1") + + benchmark.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark(benchmarkName) { + withSQLConf(SQLConf.NESTED_SCHEMA_PRUNING_ENABLED.key -> "true") { + selectBenchmark(N, numIters) + limitBenchmark(N, numIters) + repartitionBenchmark(N, numIters) + repartitionByExprBenchmark(N, numIters) + sampleBenchmark(N, numIters) + sortBenchmark(N, numIters) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala new file mode 100644 index 0000000000000..947fc67bdeae7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcNestedSchemaPruningBenchmark.scala @@ -0,0 +1,44 @@ +/* + * 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 org.apache.spark.sql.internal.SQLConf + +/** + * Synthetic benchmark for nested schema pruning performance for ORC V1 datasource. + * 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/OrcNestedSchemaPruningBenchmark-results.txt". + * }}} + */ +object OrcNestedSchemaPruningBenchmark extends NestedSchemaPruningBenchmark { + override val dataSourceName: String = "orc" + override val benchmarkName: String = "Nested Schema Pruning Benchmark For ORC v1" + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "orc", + SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> "orc") { + super.runBenchmarkSuite(mainArgs) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.scala new file mode 100644 index 0000000000000..e735d1cf257d0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/OrcV2NestedSchemaPruningBenchmark.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.sql.execution.benchmark + +/** + * Synthetic benchmark for nested schema pruning performance for ORC V2 datasource. + * 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/OrcV2NestedSchemaPruningBenchmark-results.txt". + * }}} + */ +object OrcV2NestedSchemaPruningBenchmark extends NestedSchemaPruningBenchmark { + override val dataSourceName: String = "orc" + override val benchmarkName: String = "Nested Schema Pruning Benchmark For ORC v2" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.scala new file mode 100644 index 0000000000000..1c9cc2c371176 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/ParquetNestedSchemaPruningBenchmark.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.sql.execution.benchmark + +/** + * Synthetic benchmark for nested schema pruning performance for Parquet datasource. + * 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/ParquetNestedSchemaPruningBenchmark-results.txt". + * }}} + */ +object ParquetNestedSchemaPruningBenchmark extends NestedSchemaPruningBenchmark { + override val dataSourceName: String = "parquet" + override val benchmarkName: String = "Nested Schema Pruning Benchmark For Parquet" +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala index 83edf73abfae5..8b1c422e63a3f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/PrimitiveArrayBenchmark.scala @@ -36,7 +36,7 @@ object PrimitiveArrayBenchmark extends BenchmarkBase { .config("spark.sql.autoBroadcastJoinThreshold", 1) .getOrCreate() - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Write primitive arrays in dataset") { writeDatasetArray(4) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala new file mode 100644 index 0000000000000..a9f873f9094ba --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/RangeBenchmark.scala @@ -0,0 +1,65 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.benchmark.Benchmark + +/** + * Benchmark to measure performance for range operator. + * 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/RangeBenchmark-results.txt". + * }}} + */ +object RangeBenchmark extends SqlBasedBenchmark { + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + import spark.implicits._ + + runBenchmark("range") { + val N = 500L << 20 + val benchmark = new Benchmark("range", N, output = output) + + benchmark.addCase("full scan", numIters = 4) { _ => + spark.range(N).queryExecution.toRdd.foreach(_ => ()) + } + + benchmark.addCase("limit after range", numIters = 4) { _ => + spark.range(N).limit(100).queryExecution.toRdd.foreach(_ => ()) + } + + benchmark.addCase("filter after range", numIters = 4) { _ => + spark.range(N).filter('id % 100 === 0).queryExecution.toRdd.foreach(_ => ()) + } + + benchmark.addCase("count after range", numIters = 4) { _ => + spark.range(N).count() + } + + benchmark.addCase("count after limit after range", numIters = 4) { _ => + spark.range(N).limit(100).count() + } + + benchmark.run() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala index 9a54e2320b80f..3760539c16841 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/SortBenchmark.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution.benchmark -import java.util.{Arrays, Comparator} +import java.util.Arrays import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} import org.apache.spark.unsafe.array.LongArray @@ -40,14 +40,9 @@ object SortBenchmark extends BenchmarkBase { private def referenceKeyPrefixSort(buf: LongArray, lo: Int, hi: Int, refCmp: PrefixComparator) { val sortBuffer = new LongArray(MemoryBlock.fromLongArray(new Array[Long](buf.size().toInt))) - new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort( - buf, lo, hi, new Comparator[RecordPointerAndKeyPrefix] { - override def compare( - r1: RecordPointerAndKeyPrefix, - r2: RecordPointerAndKeyPrefix): Int = { - refCmp.compare(r1.keyPrefix, r2.keyPrefix) - } - }) + new Sorter(new UnsafeSortDataFormat(sortBuffer)).sort(buf, lo, hi, + (r1: RecordPointerAndKeyPrefix, r2: RecordPointerAndKeyPrefix) => + refCmp.compare(r1.keyPrefix, r2.keyPrefix)) } private def generateKeyPrefixTestData(size: Int, rand: => Long): (LongArray, LongArray) = { @@ -119,7 +114,7 @@ object SortBenchmark extends BenchmarkBase { benchmark.run() } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("radix sort") { sortBenchmark() } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala new file mode 100644 index 0000000000000..9cbd6423f667f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UDFBenchmark.scala @@ -0,0 +1,122 @@ +/* + * 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 org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.catalyst.expressions.Literal +import org.apache.spark.sql.expressions.UserDefinedFunction +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types.{IntegerType, StringType} + +/** + * Synthetic benchmark for Scala User Defined Functions. + * 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/UDFBenchmark-results.txt". + * }}} + */ +object UDFBenchmark extends SqlBasedBenchmark { + + private def doRunBenchmarkWithMixedTypes(udf: UserDefinedFunction, cardinality: Int): Unit = { + val idCol = col("id") + val nullableIntCol = when( + idCol % 2 === 0, idCol.cast(IntegerType)).otherwise(Literal(null, IntegerType)) + val stringCol = idCol.cast(StringType) + spark.range(cardinality).select( + udf(idCol, nullableIntCol, stringCol)).write.format("noop").save() + } + + private def doRunBenchmarkWithPrimitiveTypes( + udf: UserDefinedFunction, cardinality: Int): Unit = { + val idCol = col("id") + val nullableIntCol = when( + idCol % 2 === 0, idCol.cast(IntegerType)).otherwise(Literal(null, IntegerType)) + spark.range(cardinality).select(udf(idCol, nullableIntCol)).write.format("noop").save() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val cardinality = 100000 + runBenchmark("UDF with mixed input types") { + codegenBenchmark("long/nullable int/string to string", cardinality) { + val sampleUDF = udf {(a: Long, b: java.lang.Integer, c: String) => + s"$a,$b,$c" + } + doRunBenchmarkWithMixedTypes(sampleUDF, cardinality) + } + + codegenBenchmark("long/nullable int/string to option", cardinality) { + val sampleUDF = udf {(_: Long, b: java.lang.Integer, _: String) => + Option(b) + } + doRunBenchmarkWithMixedTypes(sampleUDF, cardinality) + } + + codegenBenchmark("long/nullable int/string to primitive", cardinality) { + val sampleUDF = udf {(a: Long, b: java.lang.Integer, _: String) => + Option(b).map(_.longValue()).getOrElse(a) + } + doRunBenchmarkWithMixedTypes(sampleUDF, cardinality) + } + } + + runBenchmark("UDF with primitive types") { + codegenBenchmark("long/nullable int to string", cardinality) { + val sampleUDF = udf {(a: Long, b: java.lang.Integer) => + s"$a,$b" + } + doRunBenchmarkWithPrimitiveTypes(sampleUDF, cardinality) + } + + codegenBenchmark("long/nullable int to option", cardinality) { + val sampleUDF = udf {(_: Long, b: java.lang.Integer) => + Option(b) + } + doRunBenchmarkWithPrimitiveTypes(sampleUDF, cardinality) + } + + codegenBenchmark("long/nullable int to primitive", cardinality) { + val sampleUDF = udf {(a: Long, b: java.lang.Integer) => + Option(b).map(_.longValue()).getOrElse(a) + } + doRunBenchmarkWithPrimitiveTypes(sampleUDF, cardinality) + } + + val benchmark = new Benchmark("UDF identity overhead", cardinality, output = output) + + benchmark.addCase(s"Baseline", numIters = 5) { _ => + spark.range(cardinality).select( + col("id"), col("id") * 2, col("id") * 3).write.format("noop").save() + } + + val identityUDF = udf { x: Long => x } + benchmark.addCase(s"With identity UDF", numIters = 5) { _ => + spark.range(cardinality).select( + identityUDF(col("id")), + identityUDF(col("id") * 2), + identityUDF(col("id") * 3)).write.format("noop").save() + } + + benchmark.run() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala index 79eaeab9c399f..f582d844cdc47 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/UnsafeArrayDataBenchmark.scala @@ -194,7 +194,7 @@ object UnsafeArrayDataBenchmark extends BenchmarkBase { benchmark.run } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Benchmark UnsafeArrayData") { readUnsafeArray(10) writeUnsafeArray(10) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala index 81017a6d244f0..f4642e7d353e6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideSchemaBenchmark.scala @@ -15,59 +15,36 @@ * limitations under the License. */ -package org.apache.spark.sql +package org.apache.spark.sql.execution.benchmark -import java.io.{File, FileOutputStream, OutputStream} +import java.io.File -import org.scalatest.BeforeAndAfterEach - -import org.apache.spark.SparkFunSuite import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.functions._ +import org.apache.spark.sql.DataFrame import org.apache.spark.util.Utils /** * Benchmark for performance with very wide and nested DataFrames. - * To run this: - * build/sbt "sql/test-only *WideSchemaBenchmark" - * - * Results will be written to "sql/core/benchmarks/WideSchemaBenchmark-results.txt". + * 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/WideSchemaBenchmark-results.txt". + * }}} */ -class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { +object WideSchemaBenchmark extends SqlBasedBenchmark { private val scaleFactor = 100000 private val widthsToTest = Seq(1, 100, 2500) private val depthsToTest = Seq(1, 100, 250) assert(scaleFactor > widthsToTest.max) - private lazy val sparkSession = SparkSession.builder - .master("local[1]") - .appName("microbenchmark") - .getOrCreate() - - import sparkSession.implicits._ + import spark.implicits._ private var tmpFiles: List[File] = Nil - private var out: OutputStream = null - - override def beforeAll() { - super.beforeAll() - out = new FileOutputStream(new File("benchmarks/WideSchemaBenchmark-results.txt")) - } - override def afterAll() { - try { - out.close() - } finally { - super.afterAll() - } - } - - override def afterEach() { - super.afterEach() - for (tmpFile <- tmpFiles) { - Utils.deleteRecursively(tmpFile) - } - } + private def deleteTmpFiles(): Unit = tmpFiles.foreach(Utils.deleteRecursively) /** * Writes the given DataFrame to parquet at a temporary location, and returns a DataFrame @@ -79,7 +56,7 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { tmpFile.delete() df.write.parquet(tmpFile.getAbsolutePath) assert(tmpFile.isDirectory()) - sparkSession.read.parquet(tmpFile.getAbsolutePath) + spark.read.parquet(tmpFile.getAbsolutePath) } /** @@ -105,33 +82,33 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } } - ignore("parsing large select expressions") { - val benchmark = new Benchmark("parsing large select", 1, output = Some(out)) + def parsingLargeSelectExpressions(): Unit = { + val benchmark = new Benchmark("parsing large select", 1, output = output) for (width <- widthsToTest) { val selectExpr = (1 to width).map(i => s"id as a_$i") benchmark.addCase(s"$width select expressions") { iter => - sparkSession.range(1).toDF.selectExpr(selectExpr: _*) + spark.range(1).toDF.selectExpr(selectExpr: _*) } } benchmark.run() } - ignore("many column field read and write") { - val benchmark = new Benchmark("many column field r/w", scaleFactor, output = Some(out)) + def manyColumnFieldReadAndWrite(): Unit = { + val benchmark = new Benchmark("many column field r/w", scaleFactor, output = output) for (width <- widthsToTest) { // normalize by width to keep constant data size val numRows = scaleFactor / width val selectExpr = (1 to width).map(i => s"id as a_$i") - val df = sparkSession.range(numRows).toDF.selectExpr(selectExpr: _*).cache() + val df = spark.range(numRows).toDF.selectExpr(selectExpr: _*).cache() df.count() // force caching addCases(benchmark, df, s"$width cols x $numRows rows", "a_1") } benchmark.run() } - ignore("wide shallowly nested struct field read and write") { + def wideShallowlyNestedStructFieldReadAndWrite(): Unit = { val benchmark = new Benchmark( - "wide shallowly nested struct field r/w", scaleFactor, output = Some(out)) + "wide shallowly nested struct field r/w", scaleFactor, output = output) for (width <- widthsToTest) { val numRows = scaleFactor / width var datum: String = "{" @@ -144,15 +121,15 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } datum += "}" datum = s"""{"a": {"b": {"c": $datum, "d": $datum}, "e": $datum}}""" - val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + val df = spark.read.json(spark.range(numRows).map(_ => datum)).cache() df.count() // force caching addCases(benchmark, df, s"$width wide x $numRows rows", "a.b.c.value_1") } benchmark.run() } - ignore("deeply nested struct field read and write") { - val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor, output = Some(out)) + def deeplyNestedStructFieldReadAndWrite(): Unit = { + val benchmark = new Benchmark("deeply nested struct field r/w", scaleFactor, output = output) for (depth <- depthsToTest) { val numRows = scaleFactor / depth var datum: String = "{\"value\": 1}" @@ -161,15 +138,15 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { datum = "{\"value\": " + datum + "}" selector = selector + ".value" } - val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + val df = spark.read.json(spark.range(numRows).map(_ => datum)).cache() df.count() // force caching addCases(benchmark, df, s"$depth deep x $numRows rows", selector) } benchmark.run() } - ignore("bushy struct field read and write") { - val benchmark = new Benchmark("bushy struct field r/w", scaleFactor, output = Some(out)) + def bushyStructFieldReadAndWrite(): Unit = { + val benchmark = new Benchmark("bushy struct field r/w", scaleFactor, output = output) for (width <- Seq(1, 100, 1000)) { val numRows = scaleFactor / width var numNodes = 1 @@ -184,15 +161,16 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } // TODO(ekl) seems like the json parsing is actually the majority of the time, perhaps // we should benchmark that too separately. - val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + val df = spark.read.json(spark.range(numRows).map(_ => datum)).cache() df.count() // force caching addCases(benchmark, df, s"$numNodes x $depth deep x $numRows rows", selector) } benchmark.run() } - ignore("wide array field read and write") { - val benchmark = new Benchmark("wide array field r/w", scaleFactor, output = Some(out)) + + def wideArrayFieldReadAndWrite(): Unit = { + val benchmark = new Benchmark("wide array field r/w", scaleFactor, output = output) for (width <- widthsToTest) { val numRows = scaleFactor / width var datum: String = "{\"value\": [" @@ -204,22 +182,60 @@ class WideSchemaBenchmark extends SparkFunSuite with BeforeAndAfterEach { } } datum += "]}" - val df = sparkSession.read.json(sparkSession.range(numRows).map(_ => datum)).cache() + val df = spark.read.json(spark.range(numRows).map(_ => datum)).cache() df.count() // force caching addCases(benchmark, df, s"$width wide x $numRows rows", "value[0]") } benchmark.run() } - ignore("wide map field read and write") { - val benchmark = new Benchmark("wide map field r/w", scaleFactor, output = Some(out)) + def wideMapFieldReadAndWrite(): Unit = { + val benchmark = new Benchmark("wide map field r/w", scaleFactor, output = output) for (width <- widthsToTest) { val numRows = scaleFactor / width val datum = Tuple1((1 to width).map(i => ("value_" + i -> 1)).toMap) - val df = sparkSession.range(numRows).map(_ => datum).toDF.cache() + val df = spark.range(numRows).map(_ => datum).toDF.cache() df.count() // force caching addCases(benchmark, df, s"$width wide x $numRows rows", "_1[\"value_1\"]") } benchmark.run() } + + def runBenchmarkWithDeleteTmpFiles(benchmarkName: String)(func: => Any): Unit = { + runBenchmark(benchmarkName) { + func + } + deleteTmpFiles() + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + + runBenchmarkWithDeleteTmpFiles("parsing large select expressions") { + parsingLargeSelectExpressions() + } + + runBenchmarkWithDeleteTmpFiles("many column field read and write") { + manyColumnFieldReadAndWrite() + } + + runBenchmarkWithDeleteTmpFiles("wide shallowly nested struct field read and write") { + wideShallowlyNestedStructFieldReadAndWrite() + } + + runBenchmarkWithDeleteTmpFiles("deeply nested struct field read and write") { + deeplyNestedStructFieldReadAndWrite() + } + + runBenchmarkWithDeleteTmpFiles("bushy struct field read and write") { + bushyStructFieldReadAndWrite() + } + + runBenchmarkWithDeleteTmpFiles("wide array field read and write") { + wideArrayFieldReadAndWrite() + } + + runBenchmarkWithDeleteTmpFiles("wide map field read and write") { + wideMapFieldReadAndWrite() + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala new file mode 100644 index 0000000000000..52426d81bd1a7 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/benchmark/WideTableBenchmark.scala @@ -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. + */ + +package org.apache.spark.sql.execution.benchmark + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.internal.SQLConf + +/** + * Benchmark to measure performance for wide table. + * {{{ + * 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/WideTableBenchmark-results.txt". + * }}} + */ +object WideTableBenchmark extends SqlBasedBenchmark { + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("projection on wide table") { + val N = 1 << 20 + val df = spark.range(N) + val columns = (0 until 400).map{ i => s"id as id$i"} + val benchmark = new Benchmark("projection on wide table", N, output = output) + Seq("10", "100", "1024", "2048", "4096", "8192", "65536").foreach { n => + benchmark.addCase(s"split threshold $n", numIters = 5) { iter => + withSQLConf(SQLConf.CODEGEN_METHOD_SPLIT_THRESHOLD.key -> n) { + df.selectExpr(columns: _*).foreach(_ => ()) + } + } + } + benchmark.run() + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala index d4e7e362c6c8c..3121b7e99c99d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/ColumnStatsSuite.scala @@ -39,7 +39,7 @@ class ColumnStatsSuite extends SparkFunSuite { val columnStatsName = columnStatsClass.getSimpleName test(s"$columnStatsName: empty") { - val columnStats = columnStatsClass.newInstance() + val columnStats = columnStatsClass.getConstructor().newInstance() columnStats.collectedStatistics.zip(initialStatistics).foreach { case (actual, expected) => assert(actual === expected) } @@ -48,7 +48,7 @@ class ColumnStatsSuite extends SparkFunSuite { test(s"$columnStatsName: non-empty") { import org.apache.spark.sql.execution.columnar.ColumnarTestUtils._ - val columnStats = columnStatsClass.newInstance() + val columnStats = columnStatsClass.getConstructor().newInstance() val rows = Seq.fill(10)(makeRandomRow(columnType)) ++ Seq.fill(10)(makeNullRow(1)) rows.foreach(columnStats.gatherStats(_, 0)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala index efc2f20a907f1..711ecf1738ab1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/InMemoryColumnarQuerySuite.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.{DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, AttributeSet, In} import org.apache.spark.sql.catalyst.plans.logical.LocalRelation import org.apache.spark.sql.catalyst.plans.physical.HashPartitioning -import org.apache.spark.sql.execution.{FilterExec, LocalTableScanExec, WholeStageCodegenExec} +import org.apache.spark.sql.execution.{ColumnarToRowExec, FilterExec, InputAdapter, LocalTableScanExec, WholeStageCodegenExec} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext @@ -340,6 +340,19 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(cached.cacheBuilder.sizeInBytesStats.value === expectedAnswer.size * INT.defaultSize) } + test("cached row count should be calculated") { + val data = spark.range(6).toDF + val plan = spark.sessionState.executePlan(data.logicalPlan).sparkPlan + val cached = InMemoryRelation(true, 5, MEMORY_ONLY, plan, None, data.logicalPlan) + + // Materialize the data. + val expectedAnswer = data.collect() + checkAnswer(cached, expectedAnswer) + + // Check that the right row count was calculated. + assert(cached.cacheBuilder.rowCountStats.value === 6) + } + test("access primitive-type columns in CachedBatch without whole stage codegen") { // whole stage codegen is not applied to a row with more than WHOLESTAGE_MAX_NUM_FIELDS fields withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "2") { @@ -424,12 +437,12 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { } test("SPARK-20356: pruned InMemoryTableScanExec should have correct ordering and partitioning") { - withSQLConf("spark.sql.shuffle.partitions" -> "200") { + withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "200") { val df1 = Seq(("a", 1), ("b", 1), ("c", 2)).toDF("item", "group") val df2 = Seq(("a", 1), ("b", 2), ("c", 3)).toDF("item", "id") val df3 = df1.join(df2, Seq("item")).select($"id", $"group".as("item")).distinct() - df3.unpersist() + df3.unpersist(blocking = true) val agg_without_cache = df3.groupBy($"item").count() df3.cache() @@ -445,7 +458,7 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { // with a non-empty list assert(df.filter($"id".isin(2)).count() == 1) assert(df.filter($"id".isin(2, 3)).count() == 2) - df.unpersist() + df.unpersist(blocking = true) val dfNulls = spark.range(10).selectExpr("null as id").cache() // with null as value for the attribute assert(dfNulls.filter($"id".isin()).count() == 0) @@ -463,68 +476,71 @@ class InMemoryColumnarQuerySuite extends QueryTest with SharedSQLContext { assert(tableScanExec.partitionFilters.isEmpty) } - test("SPARK-22348: table cache should do partition batch pruning") { - Seq("true", "false").foreach { enabled => - withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> enabled) { - val df1 = Seq((1, 1), (1, 1), (2, 2)).toDF("x", "y") - df1.unpersist() - df1.cache() - - // Push predicate to the cached table. - val df2 = df1.where("y = 3") + testWithWholeStageCodegenOnAndOff("SPARK-22348: table cache " + + "should do partition batch pruning") { codegenEnabled => + val df1 = Seq((1, 1), (1, 1), (2, 2)).toDF("x", "y") + df1.unpersist(blocking = true) + df1.cache() - val planBeforeFilter = df2.queryExecution.executedPlan.collect { - case f: FilterExec => f.child - } - assert(planBeforeFilter.head.isInstanceOf[InMemoryTableScanExec]) + // Push predicate to the cached table. + val df2 = df1.where("y = 3") - val execPlan = if (enabled == "true") { - WholeStageCodegenExec(planBeforeFilter.head)(codegenStageId = 0) - } else { - planBeforeFilter.head - } - assert(execPlan.executeCollectPublic().length == 0) - } + val planBeforeFilter = df2.queryExecution.executedPlan.collect { + case FilterExec(_, c: ColumnarToRowExec) => c.child + case WholeStageCodegenExec(FilterExec(_, ColumnarToRowExec(i: InputAdapter))) => i.child } + assert(planBeforeFilter.head.isInstanceOf[InMemoryTableScanExec]) + + val execPlan = planBeforeFilter.head + assert(execPlan.executeCollectPublic().length == 0) + } + + test("SPARK-25727 - otherCopyArgs in InMemoryRelation does not include outputOrdering") { + val data = Seq(100).toDF("count").cache() + val json = data.queryExecution.optimizedPlan.toJSON + assert(json.contains("outputOrdering")) } test("SPARK-22673: InMemoryRelation should utilize existing stats of the plan to be cached") { - // This test case depends on the size of parquet in statistics. - withSQLConf( + Seq("orc", "").foreach { useV1SourceReaderList => + // This test case depends on the size of ORC in statistics. + withSQLConf( SQLConf.CBO_ENABLED.key -> "true", - SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "parquet") { - withTempPath { workDir => - withTable("table1") { - val workDirPath = workDir.getAbsolutePath - val data = Seq(100, 200, 300, 400).toDF("count") - data.write.parquet(workDirPath) - val dfFromFile = spark.read.parquet(workDirPath).cache() - val inMemoryRelation = dfFromFile.queryExecution.optimizedPlan.collect { - case plan: InMemoryRelation => plan - }.head - // InMemoryRelation's stats is file size before the underlying RDD is materialized - assert(inMemoryRelation.computeStats().sizeInBytes === 800) - - // InMemoryRelation's stats is updated after materializing RDD - dfFromFile.collect() - assert(inMemoryRelation.computeStats().sizeInBytes === 16) - - // test of catalog table - val dfFromTable = spark.catalog.createTable("table1", workDirPath).cache() - val inMemoryRelation2 = dfFromTable.queryExecution.optimizedPlan. - collect { case plan: InMemoryRelation => plan }.head - - // Even CBO enabled, InMemoryRelation's stats keeps as the file size before table's stats - // is calculated - assert(inMemoryRelation2.computeStats().sizeInBytes === 800) - - // InMemoryRelation's stats should be updated after calculating stats of the table - // clear cache to simulate a fresh environment - dfFromTable.unpersist(blocking = true) - spark.sql("ANALYZE TABLE table1 COMPUTE STATISTICS") - val inMemoryRelation3 = spark.read.table("table1").cache().queryExecution.optimizedPlan. - collect { case plan: InMemoryRelation => plan }.head - assert(inMemoryRelation3.computeStats().sizeInBytes === 48) + SQLConf.DEFAULT_DATA_SOURCE_NAME.key -> "orc", + SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReaderList) { + withTempPath { workDir => + withTable("table1") { + val workDirPath = workDir.getAbsolutePath + val data = Seq(100, 200, 300, 400).toDF("count") + data.write.orc(workDirPath) + val dfFromFile = spark.read.orc(workDirPath).cache() + val inMemoryRelation = dfFromFile.queryExecution.optimizedPlan.collect { + case plan: InMemoryRelation => plan + }.head + // InMemoryRelation's stats is file size before the underlying RDD is materialized + assert(inMemoryRelation.computeStats().sizeInBytes === getLocalDirSize(workDir)) + + // InMemoryRelation's stats is updated after materializing RDD + dfFromFile.collect() + assert(inMemoryRelation.computeStats().sizeInBytes === 16) + + // test of catalog table + val dfFromTable = spark.catalog.createTable("table1", workDirPath).cache() + val inMemoryRelation2 = dfFromTable.queryExecution.optimizedPlan. + collect { case plan: InMemoryRelation => plan }.head + + // Even CBO enabled, InMemoryRelation's stats keeps as the file size before table's + // stats is calculated + assert(inMemoryRelation2.computeStats().sizeInBytes === getLocalDirSize(workDir)) + + // InMemoryRelation's stats should be updated after calculating stats of the table + // clear cache to simulate a fresh environment + dfFromTable.unpersist(blocking = true) + spark.sql("ANALYZE TABLE table1 COMPUTE STATISTICS") + val inMemoryRelation3 = spark.read.table("table1").cache().queryExecution.optimizedPlan. + collect { case plan: InMemoryRelation => plan }.head + assert(inMemoryRelation3.computeStats().sizeInBytes === 48) + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala index af493e93b5192..e74099202a1df 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/PartitionBatchPruningSuite.scala @@ -50,7 +50,7 @@ class PartitionBatchPruningSuite // Enable in-memory partition pruning spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, true) // Enable in-memory table scan accumulators - spark.conf.set("spark.sql.inMemoryTableScanStatistics.enable", "true") + spark.conf.set(SQLConf.IN_MEMORY_TABLE_SCAN_STATISTICS_ENABLED.key, "true") } override protected def afterAll(): Unit = { @@ -170,6 +170,15 @@ class PartitionBatchPruningSuite } } + // Support `StartsWith` predicate + checkBatchPruning("SELECT CAST(s AS INT) FROM pruningStringData WHERE s like '18%'", 1, 1)( + 180 to 189 + ) + checkBatchPruning("SELECT CAST(s AS INT) FROM pruningStringData WHERE s like '%'", 5, 11)( + 100 to 200 + ) + checkBatchPruning("SELECT CAST(s AS INT) FROM pruningStringData WHERE '18%' like s", 5, 11)(Seq()) + // With disable IN_MEMORY_PARTITION_PRUNING option test("disable IN_MEMORY_PARTITION_PRUNING") { spark.conf.set(SQLConf.IN_MEMORY_PARTITION_PRUNING.key, false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala index 0f9079744a220..8ea20f28a37b2 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/CompressionSchemeBenchmark.scala @@ -233,7 +233,7 @@ object CompressionSchemeBenchmark extends BenchmarkBase with AllCompressionSchem runDecodeBenchmark("STRING Decode", iters, count, STRING, testData) } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Compression Scheme Benchmark") { bitEncodingBenchmark(1024) shortEncodingBenchmark(1024) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala index 0d9f1fb0c02c9..fb3388452e4e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/columnar/compression/IntegralDeltaSuite.scala @@ -46,6 +46,7 @@ class IntegralDeltaSuite extends SparkFunSuite { (input.tail, input.init).zipped.map { case (x: Int, y: Int) => (x - y).toLong case (x: Long, y: Long) => x - y + case other => fail(s"Unexpected input $other") } } @@ -116,7 +117,7 @@ class IntegralDeltaSuite extends SparkFunSuite { val row = new GenericInternalRow(1) val nullRow = new GenericInternalRow(1) nullRow.setNullAt(0) - input.map { value => + input.foreach { value => if (value == nullValue) { builder.appendFrom(nullRow, 0) } else { 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 d5cbc84d83130..5e71c9cc455df 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 @@ -24,24 +24,21 @@ import scala.reflect.{classTag, ClassTag} import org.apache.spark.sql.{AnalysisException, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, UnresolvedAttribute} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.dsl.plans import org.apache.spark.sql.catalyst.dsl.plans.DslLogicalPlan import org.apache.spark.sql.catalyst.expressions.{Expression, JsonTuple, PartitioningAttribute} import org.apache.spark.sql.catalyst.parser.ParseException -import org.apache.spark.sql.catalyst.plans.PlanTest -import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan} -import org.apache.spark.sql.catalyst.plans.logical.{Project, ScriptTransformation} +import org.apache.spark.sql.catalyst.plans.logical.{Generate, InsertIntoDir, LogicalPlan, Project, ScriptTransformation} import org.apache.spark.sql.execution.SparkSqlParser import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.sql.types.{IntegerType, StructField, StructType} - -class DDLParserSuite extends PlanTest with SharedSQLContext { +class DDLParserSuite extends AnalysisTest with SharedSQLContext { private lazy val parser = new SparkSqlParser(new SQLConf) private def assertUnsupported(sql: String, containsThesePhrases: Seq[String] = Seq()): Unit = { @@ -54,12 +51,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { } } - private def intercept(sqlCommand: String, messages: String*): Unit = { - val e = intercept[ParseException](parser.parsePlan(sqlCommand)).getMessage - messages.foreach { message => - assert(e.contains(message)) - } - } + private def intercept(sqlCommand: String, messages: String*): Unit = + interceptParseException(parser.parsePlan)(sqlCommand, messages: _*) private def parseAs[T: ClassTag](query: String): T = { parser.parsePlan(query) match { @@ -85,8 +78,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { val sql = """ |CREATE DATABASE IF NOT EXISTS database_name - |COMMENT 'database_comment' LOCATION '/home/user/db' |WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c') + |COMMENT 'database_comment' LOCATION '/home/user/db' """.stripMargin val parsed = parser.parsePlan(sql) val expected = CreateDatabaseCommand( @@ -98,6 +91,23 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { comparePlans(parsed, expected) } + test("create database -- check duplicates") { + def createDatabase(duplicateClause: String): String = { + s""" + |CREATE DATABASE IF NOT EXISTS database_name + |$duplicateClause + |$duplicateClause + """.stripMargin + } + val sql1 = createDatabase("COMMENT 'database_comment'") + val sql2 = createDatabase("LOCATION '/home/user/db'") + val sql3 = createDatabase("WITH DBPROPERTIES ('a'='a', 'b'='b', 'c'='c')") + + intercept(sql1, "Found duplicate clauses: COMMENT") + intercept(sql2, "Found duplicate clauses: LOCATION") + intercept(sql3, "Found duplicate clauses: WITH DBPROPERTIES") + } + test("create database - property values must be set") { assertUnsupported( sql = "CREATE DATABASE my_db WITH DBPROPERTIES('key_without_value', 'key_with_value'='x')", @@ -415,173 +425,28 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { assert(ct.tableDesc.storage.locationUri == Some(new URI("/something/anything"))) } - test("create table - with partitioned by") { - val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + - "USING parquet PARTITIONED BY (a)" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("my_tab"), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType() - .add("a", IntegerType, nullable = true, "test") - .add("b", StringType), - provider = Some("parquet"), - partitionColumnNames = Seq("a") - ) - - parser.parsePlan(query) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } - } - - test("create table - with bucket") { - val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + - "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("my_tab"), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType().add("a", IntegerType).add("b", StringType), - provider = Some("parquet"), - bucketSpec = Some(BucketSpec(5, Seq("a"), Seq("b"))) - ) - - parser.parsePlan(query) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $query") - } - } - - test("create table - with comment") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("my_tab"), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType().add("a", IntegerType).add("b", StringType), - provider = Some("parquet"), - comment = Some("abc")) - - parser.parsePlan(sql) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } - } - - test("create table - with table properties") { - val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("my_tab"), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType().add("a", IntegerType).add("b", StringType), - provider = Some("parquet"), - properties = Map("test" -> "test")) - - parser.parsePlan(sql) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } - } - - test("Duplicate clauses - create table") { - def createTableHeader(duplicateClause: String, isNative: Boolean): String = { - val fileFormat = if (isNative) "USING parquet" else "STORED AS parquet" - s"CREATE TABLE my_tab(a INT, b STRING) $fileFormat $duplicateClause $duplicateClause" - } - - Seq(true, false).foreach { isNative => - intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')", isNative), - "Found duplicate clauses: TBLPROPERTIES") - intercept(createTableHeader("LOCATION '/tmp/file'", isNative), - "Found duplicate clauses: LOCATION") - intercept(createTableHeader("COMMENT 'a table'", isNative), - "Found duplicate clauses: COMMENT") - intercept(createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS", isNative), - "Found duplicate clauses: CLUSTERED BY") + test("Duplicate clauses - create hive table") { + def createTableHeader(duplicateClause: String): String = { + s"CREATE TABLE my_tab(a INT, b STRING) STORED AS parquet $duplicateClause $duplicateClause" } - // Only for native data source tables - intercept(createTableHeader("PARTITIONED BY (b)", isNative = true), - "Found duplicate clauses: PARTITIONED BY") - - // Only for Hive serde tables - intercept(createTableHeader("PARTITIONED BY (k int)", isNative = false), + intercept(createTableHeader("TBLPROPERTIES('test' = 'test2')"), + "Found duplicate clauses: TBLPROPERTIES") + intercept(createTableHeader("LOCATION '/tmp/file'"), + "Found duplicate clauses: LOCATION") + intercept(createTableHeader("COMMENT 'a table'"), + "Found duplicate clauses: COMMENT") + intercept(createTableHeader("CLUSTERED BY(b) INTO 256 BUCKETS"), + "Found duplicate clauses: CLUSTERED BY") + intercept(createTableHeader("PARTITIONED BY (k int)"), "Found duplicate clauses: PARTITIONED BY") - intercept(createTableHeader("STORED AS parquet", isNative = false), + intercept(createTableHeader("STORED AS parquet"), "Found duplicate clauses: STORED AS/BY") intercept( - createTableHeader("ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe'", isNative = false), + createTableHeader("ROW FORMAT SERDE 'parquet.hive.serde.ParquetHiveSerDe'"), "Found duplicate clauses: ROW FORMAT") } - test("create table - with location") { - val v1 = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("my_tab"), - tableType = CatalogTableType.EXTERNAL, - storage = CatalogStorageFormat.empty.copy(locationUri = Some(new URI("/tmp/file"))), - schema = new StructType().add("a", IntegerType).add("b", StringType), - provider = Some("parquet")) - - parser.parsePlan(v1) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $v1") - } - - val v2 = - """ - |CREATE TABLE my_tab(a INT, b STRING) - |USING parquet - |OPTIONS (path '/tmp/file') - |LOCATION '/tmp/file' - """.stripMargin - val e = intercept[ParseException] { - parser.parsePlan(v2) - } - assert(e.message.contains("you can only specify one of them.")) - } - - test("create table - byte length literal table name") { - val sql = "CREATE TABLE 1m.2g(a INT) USING parquet" - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("2g", Some("1m")), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty, - schema = new StructType().add("a", IntegerType), - provider = Some("parquet")) - - parser.parsePlan(sql) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } - } - test("insert overwrite directory") { val v1 = "INSERT OVERWRITE DIRECTORY '/tmp/file' USING parquet SELECT 1 as a" parser.parsePlan(v1) match { @@ -657,45 +522,6 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { assert(plan.newName == TableIdentifier("tbl2", Some("db1"))) } - // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); - // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); - // ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); - // ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); - test("alter table/view: alter table/view properties") { - val sql1_table = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + - "'comment' = 'new_comment')" - val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" - val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" - val sql1_view = sql1_table.replace("TABLE", "VIEW") - val sql2_view = sql2_table.replace("TABLE", "VIEW") - val sql3_view = sql3_table.replace("TABLE", "VIEW") - - val parsed1_table = parser.parsePlan(sql1_table) - val parsed2_table = parser.parsePlan(sql2_table) - val parsed3_table = parser.parsePlan(sql3_table) - val parsed1_view = parser.parsePlan(sql1_view) - val parsed2_view = parser.parsePlan(sql2_view) - val parsed3_view = parser.parsePlan(sql3_view) - - val tableIdent = TableIdentifier("table_name", None) - val expected1_table = AlterTableSetPropertiesCommand( - tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) - val expected2_table = AlterTableUnsetPropertiesCommand( - tableIdent, Seq("comment", "test"), ifExists = false, isView = false) - val expected3_table = AlterTableUnsetPropertiesCommand( - tableIdent, Seq("comment", "test"), ifExists = true, isView = false) - val expected1_view = expected1_table.copy(isView = true) - val expected2_view = expected2_table.copy(isView = true) - val expected3_view = expected3_table.copy(isView = true) - - comparePlans(parsed1_table, expected1_table) - comparePlans(parsed2_table, expected2_table) - comparePlans(parsed3_table, expected3_table) - comparePlans(parsed1_view, expected1_view) - comparePlans(parsed2_view, expected2_view) - comparePlans(parsed3_view, expected3_view) - } - test("alter table - property values must be set") { assertUnsupported( sql = "ALTER TABLE my_tab SET TBLPROPERTIES('key_without_value', 'key_with_value'='x')", @@ -921,22 +747,15 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { "SET FILEFORMAT PARQUET") } - test("alter table: set location") { - val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + test("alter table: set partition location") { val sql2 = "ALTER TABLE table_name PARTITION (dt='2008-08-08', country='us') " + "SET LOCATION 'new location'" - val parsed1 = parser.parsePlan(sql1) val parsed2 = parser.parsePlan(sql2) val tableIdent = TableIdentifier("table_name", None) - val expected1 = AlterTableSetLocationCommand( - tableIdent, - None, - "new location") val expected2 = AlterTableSetLocationCommand( tableIdent, Some(Map("dt" -> "2008-08-08", "country" -> "us")), "new location") - comparePlans(parsed1, expected1) comparePlans(parsed2, expected2) } @@ -1060,64 +879,6 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { assert(e.contains("Found an empty partition key 'b'")) } - test("drop table") { - val tableName1 = "db.tab" - val tableName2 = "tab" - - val parsed = Seq( - s"DROP TABLE $tableName1", - s"DROP TABLE IF EXISTS $tableName1", - s"DROP TABLE $tableName2", - s"DROP TABLE IF EXISTS $tableName2", - s"DROP TABLE $tableName2 PURGE", - s"DROP TABLE IF EXISTS $tableName2 PURGE" - ).map(parser.parsePlan) - - val expected = Seq( - DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = false, isView = false, - purge = false), - DropTableCommand(TableIdentifier("tab", Option("db")), ifExists = true, isView = false, - purge = false), - DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false, - purge = false), - DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false, - purge = false), - DropTableCommand(TableIdentifier("tab", None), ifExists = false, isView = false, - purge = true), - DropTableCommand(TableIdentifier("tab", None), ifExists = true, isView = false, - purge = true)) - - parsed.zip(expected).foreach { case (p, e) => comparePlans(p, e) } - } - - test("drop view") { - val viewName1 = "db.view" - val viewName2 = "view" - - val parsed1 = parser.parsePlan(s"DROP VIEW $viewName1") - val parsed2 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName1") - val parsed3 = parser.parsePlan(s"DROP VIEW $viewName2") - val parsed4 = parser.parsePlan(s"DROP VIEW IF EXISTS $viewName2") - - val expected1 = - DropTableCommand(TableIdentifier("view", Option("db")), ifExists = false, isView = true, - purge = false) - val expected2 = - DropTableCommand(TableIdentifier("view", Option("db")), ifExists = true, isView = true, - purge = false) - val expected3 = - DropTableCommand(TableIdentifier("view", None), ifExists = false, isView = true, - purge = false) - val expected4 = - DropTableCommand(TableIdentifier("view", None), ifExists = true, isView = true, - purge = false) - - comparePlans(parsed1, expected1) - comparePlans(parsed2, expected2) - comparePlans(parsed3, expected3) - comparePlans(parsed4, expected4) - } - test("show columns") { val sql1 = "SHOW COLUMNS FROM t1" val sql2 = "SHOW COLUMNS IN db1.t1" @@ -1178,99 +939,6 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { comparePlans(parsed, expected) } - test("support for other types in TBLPROPERTIES") { - val sql = - """ - |ALTER TABLE table_name - |SET TBLPROPERTIES ('a' = 1, 'b' = 0.1, 'c' = TRUE) - """.stripMargin - val parsed = parser.parsePlan(sql) - val expected = AlterTableSetPropertiesCommand( - TableIdentifier("table_name"), - Map("a" -> "1", "b" -> "0.1", "c" -> "true"), - isView = false) - - comparePlans(parsed, expected) - } - - test("support for other types in OPTIONS") { - val sql = - """ - |CREATE TABLE table_name USING json - |OPTIONS (a 1, b 0.1, c TRUE) - """.stripMargin - - val expectedTableDesc = CatalogTable( - identifier = TableIdentifier("table_name"), - tableType = CatalogTableType.MANAGED, - storage = CatalogStorageFormat.empty.copy( - properties = Map("a" -> "1", "b" -> "0.1", "c" -> "true") - ), - schema = new StructType, - provider = Some("json") - ) - - parser.parsePlan(sql) match { - case CreateTable(tableDesc, _, None) => - assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) - case other => - fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + - s"got ${other.getClass.getName}: $sql") - } - } - - test("Test CTAS against data source tables") { - val s1 = - """ - |CREATE TABLE IF NOT EXISTS mydb.page_view - |USING parquet - |COMMENT 'This is the staging page view table' - |LOCATION '/user/external/page_view' - |TBLPROPERTIES ('p1'='v1', 'p2'='v2') - |AS SELECT * FROM src - """.stripMargin - - val s2 = - """ - |CREATE TABLE IF NOT EXISTS mydb.page_view - |USING parquet - |LOCATION '/user/external/page_view' - |COMMENT 'This is the staging page view table' - |TBLPROPERTIES ('p1'='v1', 'p2'='v2') - |AS SELECT * FROM src - """.stripMargin - - val s3 = - """ - |CREATE TABLE IF NOT EXISTS mydb.page_view - |USING parquet - |COMMENT 'This is the staging page view table' - |LOCATION '/user/external/page_view' - |TBLPROPERTIES ('p1'='v1', 'p2'='v2') - |AS SELECT * FROM src - """.stripMargin - - checkParsing(s1) - checkParsing(s2) - checkParsing(s3) - - def checkParsing(sql: String): Unit = { - val (desc, exists) = extractTableDesc(sql) - assert(exists) - assert(desc.identifier.database == Some("mydb")) - assert(desc.identifier.table == "page_view") - assert(desc.storage.locationUri == Some(new URI("/user/external/page_view"))) - assert(desc.schema.isEmpty) // will be populated later when the table is actually created - assert(desc.comment == Some("This is the staging page view table")) - assert(desc.viewText.isEmpty) - assert(desc.viewDefaultDatabase.isEmpty) - assert(desc.viewQueryColumnNames.isEmpty) - assert(desc.partitionColumnNames.isEmpty) - assert(desc.provider == Some("parquet")) - assert(desc.properties == Map("p1" -> "v1", "p2" -> "v2")) - } - } - test("Test CTAS #1") { val s1 = """ @@ -1496,6 +1164,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { "month 32 outside range [0, 11]") assertError("select interval '5 49:12:15' day to second", "hour 49 outside range [0, 23]") + assertError("select interval '23:61:15' hour to second", + "minute 61 outside range [0, 59]") assertError("select interval '.1111111111' second", "nanosecond 1111111111 outside range") } @@ -1768,8 +1438,8 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { """ |CREATE OR REPLACE VIEW view1 |(col1, col3 COMMENT 'hello') - |COMMENT 'BLABLA' |TBLPROPERTIES('prop1Key'="prop1Val") + |COMMENT 'BLABLA' |AS SELECT * FROM tab1 """.stripMargin val command = parser.parsePlan(v1).asInstanceOf[CreateViewCommand] @@ -1788,6 +1458,22 @@ class DDLParserSuite extends PlanTest with SharedSQLContext { } } + test("create view - duplicate clauses") { + def createViewStatement(duplicateClause: String): String = { + s""" + |CREATE OR REPLACE VIEW view1 + |(col1, col3 COMMENT 'hello') + |$duplicateClause + |$duplicateClause + |AS SELECT * FROM tab1 + """.stripMargin + } + val sql1 = createViewStatement("COMMENT 'BLABLA'") + val sql2 = createViewStatement("TBLPROPERTIES('prop1Key'=\"prop1Val\")") + intercept(sql1, "Found duplicate clauses: COMMENT") + intercept(sql2, "Found duplicate clauses: TBLPROPERTIES") + } + test("MSCK REPAIR table") { val sql = "MSCK REPAIR TABLE tab1" val parsed = parser.parsePlan(sql) 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 f8d98dead2d42..b777db750a1bb 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 @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution.command -import java.io.File +import java.io.{File, PrintWriter} import java.net.URI import java.util.Locale import org.apache.hadoop.fs.Path import org.scalatest.BeforeAndAfterEach +import org.apache.spark.internal.config +import org.apache.spark.internal.config.RDD_PARALLEL_LISTING_THRESHOLD import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{FunctionRegistry, NoSuchPartitionException, NoSuchTableException, TempTableAlreadyExistsException} @@ -376,41 +378,41 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - test("CTAS a managed table with the existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) + private def withEmptyDirInTablePath(dirName: String)(f : File => Unit): Unit = { + val tableLoc = + new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier(dirName))) try { tableLoc.mkdir() + f(tableLoc) + } finally { + waitForTasksToFinish() + Utils.deleteRecursively(tableLoc) + } + } + + + test("CTAS a managed table with the existing empty directory") { + withEmptyDirInTablePath("tab1") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 USING ${dataSource} AS SELECT 1, 'a'") checkAnswer(spark.table("tab1"), Row(1, "a")) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } test("create a managed table with the existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) - try { - tableLoc.mkdir() + withEmptyDirInTablePath("tab1") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 (col1 int, col2 string) USING ${dataSource}") sql("INSERT INTO tab1 VALUES (1, 'a')") checkAnswer(spark.table("tab1"), Row(1, "a")) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } test("create a managed table with the existing non-empty directory") { withTable("tab1") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab1"))) - try { - // create an empty hidden file - tableLoc.mkdir() + withEmptyDirInTablePath("tab1") { tableLoc => val hiddenGarbageFile = new File(tableLoc.getCanonicalPath, ".garbage") hiddenGarbageFile.createNewFile() val exMsg = "Can not create the managed table('`tab1`'). The associated location" @@ -438,28 +440,20 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { }.getMessage assert(ex.contains(exMsgWithDefaultDB)) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } } test("rename a managed table with existing empty directory") { - val tableLoc = new File(spark.sessionState.catalog.defaultTablePath(TableIdentifier("tab2"))) - try { + withEmptyDirInTablePath("tab2") { tableLoc => withTable("tab1") { sql(s"CREATE TABLE tab1 USING $dataSource AS SELECT 1, 'a'") - tableLoc.mkdir() val ex = intercept[AnalysisException] { sql("ALTER TABLE tab1 RENAME TO tab2") }.getMessage val expectedMsg = "Can not rename the managed table('`tab1`'). The associated location" assert(ex.contains(expectedMsg)) } - } finally { - waitForTasksToFinish() - Utils.deleteRecursively(tableLoc) } } @@ -1129,13 +1123,13 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } test("alter table: recover partitions (sequential)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "10") { + withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "10") { testRecoverPartitions() } } test("alter table: recover partition (parallel)") { - withSQLConf("spark.rdd.parallelListingThreshold" -> "0") { + withSQLConf(RDD_PARALLEL_LISTING_THRESHOLD.key -> "0") { testRecoverPartitions() } } @@ -1378,7 +1372,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { // if (isUsingHiveMetastore) { // assert(storageFormat.properties.get("path") === expected) // } - assert(storageFormat.locationUri === Some(expected)) + assert(storageFormat.locationUri.map(_.getPath) === Some(expected.getPath)) } // set table location sql("ALTER TABLE dbx.tab1 SET LOCATION '/path/to/your/lovely/heart'") @@ -1784,7 +1778,7 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { """Extended Usage: | Examples: | > SELECT 3 ^ 5; - | 2 + | 6 | """.stripMargin) :: Row("Function: ^") :: Row("Usage: expr1 ^ expr2 - Returns the result of " + @@ -2572,7 +2566,10 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } - val supportedNativeFileFormatsForAlterTableAddColumns = Seq("parquet", "json", "csv") + val supportedNativeFileFormatsForAlterTableAddColumns = Seq("csv", "json", "parquet", + "org.apache.spark.sql.execution.datasources.csv.CSVFileFormat", + "org.apache.spark.sql.execution.datasources.json.JsonFileFormat", + "org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat") supportedNativeFileFormatsForAlterTableAddColumns.foreach { provider => test(s"alter datasource table add columns - $provider") { @@ -2715,4 +2712,47 @@ abstract class DDLSuite extends QueryTest with SQLTestUtils { } } } + + test("set command rejects SparkConf entries") { + val ex = intercept[AnalysisException] { + sql(s"SET ${config.CPUS_PER_TASK.key} = 4") + } + assert(ex.getMessage.contains("Spark config")) + } + + test("Refresh table before drop database cascade") { + withTempDir { tempDir => + val file1 = new File(tempDir + "/first.csv") + Utils.tryWithResource(new PrintWriter(file1)) { writer => + writer.write("first") + } + + val file2 = new File(tempDir + "/second.csv") + Utils.tryWithResource(new PrintWriter(file2)) { writer => + writer.write("second") + } + + withDatabase("foo") { + withTable("foo.first") { + sql("CREATE DATABASE foo") + sql( + s"""CREATE TABLE foo.first (id STRING) + |USING csv OPTIONS (path='${file1.toURI}') + """.stripMargin) + sql("SELECT * FROM foo.first") + checkAnswer(spark.table("foo.first"), Row("first")) + + // Dropping the database and again creating same table with different path + sql("DROP DATABASE foo CASCADE") + sql("CREATE DATABASE foo") + sql( + s"""CREATE TABLE foo.first (id STRING) + |USING csv OPTIONS (path='${file2.toURI}') + """.stripMargin) + sql("SELECT * FROM foo.first") + checkAnswer(spark.table("foo.first"), Row("second")) + } + } + } + } } 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 new file mode 100644 index 0000000000000..ce209666024d0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/PlanResolutionSuite.scala @@ -0,0 +1,680 @@ +/* + * 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.command + +import java.net.URI +import java.util.Locale + +import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.catalog.v2.{CatalogNotFoundException, CatalogPlugin, Identifier, LookupCatalog, TableCatalog, TestTableCatalog} +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.analysis.AnalysisTest +import org.apache.spark.sql.catalyst.catalog.{BucketSpec, CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.catalyst.plans.logical.{CreateTableAsSelect, CreateV2Table, DropTable, LogicalPlan} +import org.apache.spark.sql.execution.datasources.{CreateTable, DataSourceResolution} +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.internal.SQLConf.DEFAULT_V2_CATALOG +import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class PlanResolutionSuite extends AnalysisTest { + import CatalystSqlParser._ + + private val orc2 = classOf[OrcDataSourceV2].getName + + private val testCat: TableCatalog = { + val newCatalog = new TestTableCatalog + newCatalog.initialize("testcat", CaseInsensitiveStringMap.empty()) + newCatalog + } + + private val v2SessionCatalog = { + val newCatalog = new TestTableCatalog + newCatalog.initialize("session", CaseInsensitiveStringMap.empty()) + newCatalog + } + + private val lookupWithDefault: LookupCatalog = new LookupCatalog { + override protected def defaultCatalogName: Option[String] = Some("testcat") + + override protected def lookupCatalog(name: String): CatalogPlugin = name match { + case "testcat" => + testCat + case "session" => + v2SessionCatalog + case _ => + throw new CatalogNotFoundException(s"No such catalog: $name") + } + } + + private val lookupWithoutDefault: LookupCatalog = new LookupCatalog { + override protected def defaultCatalogName: Option[String] = None + + override protected def lookupCatalog(name: String): CatalogPlugin = name match { + case "testcat" => + testCat + case "session" => + v2SessionCatalog + case _ => + throw new CatalogNotFoundException(s"No such catalog: $name") + } + } + + def parseAndResolve(query: String, withDefault: Boolean = false): LogicalPlan = { + val newConf = conf.copy() + newConf.setConfString(DEFAULT_V2_CATALOG.key, "testcat") + DataSourceResolution(newConf, if (withDefault) lookupWithDefault else lookupWithoutDefault) + .apply(parsePlan(query)) + } + + private def parseResolveCompare(query: String, expected: LogicalPlan): Unit = + comparePlans(parseAndResolve(query), expected, checkAnalysis = true) + + private def extractTableDesc(sql: String): (CatalogTable, Boolean) = { + parseAndResolve(sql).collect { + case CreateTable(tableDesc, mode, _) => (tableDesc, mode == SaveMode.Ignore) + }.head + } + + test("create table - with partitioned by") { + val query = "CREATE TABLE my_tab(a INT comment 'test', b STRING) " + + "USING parquet PARTITIONED BY (a)" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType() + .add("a", IntegerType, nullable = true, "test") + .add("b", StringType), + provider = Some("parquet"), + partitionColumnNames = Seq("a") + ) + + parseAndResolve(query) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - partitioned by transforms") { + val transforms = Seq( + "bucket(16, b)", "years(ts)", "months(ts)", "days(ts)", "hours(ts)", "foo(a, 'bar', 34)", + "bucket(32, b), days(ts)") + transforms.foreach { transform => + val query = + s""" + |CREATE TABLE my_tab(a INT, b STRING) USING parquet + |PARTITIONED BY ($transform) + """.stripMargin + + val ae = intercept[AnalysisException] { + parseAndResolve(query) + } + + assert(ae.message + .contains(s"Transforms cannot be converted to partition columns: $transform")) + } + } + + test("create table - with bucket") { + val query = "CREATE TABLE my_tab(a INT, b STRING) USING parquet " + + "CLUSTERED BY (a) SORTED BY (b) INTO 5 BUCKETS" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + bucketSpec = Some(BucketSpec(5, Seq("a"), Seq("b"))) + ) + + parseAndResolve(query) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $query") + } + } + + test("create table - with comment") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet COMMENT 'abc'" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + comment = Some("abc")) + + parseAndResolve(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with table properties") { + val sql = "CREATE TABLE my_tab(a INT, b STRING) USING parquet TBLPROPERTIES('test' = 'test')" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet"), + properties = Map("test" -> "test")) + + parseAndResolve(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("create table - with location") { + val v1 = "CREATE TABLE my_tab(a INT, b STRING) USING parquet LOCATION '/tmp/file'" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("my_tab"), + tableType = CatalogTableType.EXTERNAL, + storage = CatalogStorageFormat.empty.copy(locationUri = Some(new URI("/tmp/file"))), + schema = new StructType().add("a", IntegerType).add("b", StringType), + provider = Some("parquet")) + + parseAndResolve(v1) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $v1") + } + + val v2 = + """ + |CREATE TABLE my_tab(a INT, b STRING) + |USING parquet + |OPTIONS (path '/tmp/file') + |LOCATION '/tmp/file' + """.stripMargin + val e = intercept[AnalysisException] { + parseAndResolve(v2) + } + assert(e.message.contains("you can only specify one of them.")) + } + + test("create table - byte length literal table name") { + val sql = "CREATE TABLE 1m.2g(a INT) USING parquet" + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("2g", Some("1m")), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty, + schema = new StructType().add("a", IntegerType), + provider = Some("parquet")) + + parseAndResolve(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("support for other types in OPTIONS") { + val sql = + """ + |CREATE TABLE table_name USING json + |OPTIONS (a 1, b 0.1, c TRUE) + """.stripMargin + + val expectedTableDesc = CatalogTable( + identifier = TableIdentifier("table_name"), + tableType = CatalogTableType.MANAGED, + storage = CatalogStorageFormat.empty.copy( + properties = Map("a" -> "1", "b" -> "0.1", "c" -> "true") + ), + schema = new StructType, + provider = Some("json") + ) + + parseAndResolve(sql) match { + case CreateTable(tableDesc, _, None) => + assert(tableDesc == expectedTableDesc.copy(createTime = tableDesc.createTime)) + case other => + fail(s"Expected to parse ${classOf[CreateTableCommand].getClass.getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test CTAS against data source tables") { + val s1 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + val s2 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |LOCATION '/user/external/page_view' + |COMMENT 'This is the staging page view table' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + val s3 = + """ + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING parquet + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + checkParsing(s1) + checkParsing(s2) + checkParsing(s3) + + def checkParsing(sql: String): Unit = { + val (desc, exists) = extractTableDesc(sql) + assert(exists) + assert(desc.identifier.database.contains("mydb")) + assert(desc.identifier.table == "page_view") + assert(desc.storage.locationUri.contains(new URI("/user/external/page_view"))) + assert(desc.schema.isEmpty) // will be populated later when the table is actually created + assert(desc.comment.contains("This is the staging page view table")) + assert(desc.viewText.isEmpty) + assert(desc.viewDefaultDatabase.isEmpty) + assert(desc.viewQueryColumnNames.isEmpty) + assert(desc.partitionColumnNames.isEmpty) + assert(desc.provider.contains("parquet")) + assert(desc.properties == Map("p1" -> "v1", "p2" -> "v2")) + } + } + + test("Test v2 CreateTable with known catalog in identifier") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS testcat.mydb.table_name ( + | id bigint, + | description string, + | point struct) + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql) match { + case create: CreateV2Table => + assert(create.catalog.name == "testcat") + assert(create.tableName == Identifier.of(Array("mydb"), "table_name")) + assert(create.tableSchema == new StructType() + .add("id", LongType) + .add("description", StringType) + .add("point", new StructType().add("x", DoubleType).add("y", DoubleType))) + assert(create.partitioning.isEmpty) + assert(create.properties == expectedProperties) + assert(create.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateV2Table].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CreateTable with default catalog") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.table_name ( + | id bigint, + | description string, + | point struct) + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql, withDefault = true) match { + case create: CreateV2Table => + assert(create.catalog.name == "testcat") + assert(create.tableName == Identifier.of(Array("mydb"), "table_name")) + assert(create.tableSchema == new StructType() + .add("id", LongType) + .add("description", StringType) + .add("point", new StructType().add("x", DoubleType).add("y", DoubleType))) + assert(create.partitioning.isEmpty) + assert(create.properties == expectedProperties) + assert(create.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateV2Table].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CreateTable with data source v2 provider and no default") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.page_view ( + | id bigint, + | description string, + | point struct) + |USING $orc2 + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "provider" -> orc2, + "location" -> "/user/external/page_view", + "comment" -> "This is the staging page view table") + + parseAndResolve(sql) match { + case create: CreateV2Table => + assert(create.catalog.name == "session") + assert(create.tableName == Identifier.of(Array("mydb"), "page_view")) + assert(create.tableSchema == new StructType() + .add("id", LongType) + .add("description", StringType) + .add("point", new StructType().add("x", DoubleType).add("y", DoubleType))) + assert(create.partitioning.isEmpty) + assert(create.properties == expectedProperties) + assert(create.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateV2Table].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CTAS with known catalog in identifier") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS testcat.mydb.table_name + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + |AS SELECT * FROM src + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql) match { + case ctas: CreateTableAsSelect => + 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.partitioning.isEmpty) + assert(ctas.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableAsSelect].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CTAS with default catalog") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.table_name + |USING parquet + |COMMENT 'table comment' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |OPTIONS (path 's3://bucket/path/to/data', other 20) + |AS SELECT * FROM src + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "other" -> "20", + "provider" -> "parquet", + "location" -> "s3://bucket/path/to/data", + "comment" -> "table comment") + + parseAndResolve(sql, withDefault = true) match { + case ctas: CreateTableAsSelect => + 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.partitioning.isEmpty) + assert(ctas.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableAsSelect].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("Test v2 CTAS with data source v2 provider and no default") { + val sql = + s""" + |CREATE TABLE IF NOT EXISTS mydb.page_view + |USING $orc2 + |COMMENT 'This is the staging page view table' + |LOCATION '/user/external/page_view' + |TBLPROPERTIES ('p1'='v1', 'p2'='v2') + |AS SELECT * FROM src + """.stripMargin + + val expectedProperties = Map( + "p1" -> "v1", + "p2" -> "v2", + "provider" -> orc2, + "location" -> "/user/external/page_view", + "comment" -> "This is the staging page view table") + + parseAndResolve(sql) match { + case ctas: CreateTableAsSelect => + assert(ctas.catalog.name == "session") + assert(ctas.tableName == Identifier.of(Array("mydb"), "page_view")) + assert(ctas.properties == expectedProperties) + assert(ctas.writeOptions.isEmpty) + assert(ctas.partitioning.isEmpty) + assert(ctas.ignoreIfExists) + + case other => + fail(s"Expected to parse ${classOf[CreateTableAsSelect].getName} from query," + + s"got ${other.getClass.getName}: $sql") + } + } + + test("drop table") { + val tableName1 = "db.tab" + val tableIdent1 = TableIdentifier("tab", Option("db")) + val tableName2 = "tab" + val tableIdent2 = TableIdentifier("tab", None) + + parseResolveCompare(s"DROP TABLE $tableName1", + DropTableCommand(tableIdent1, ifExists = false, isView = false, purge = false)) + parseResolveCompare(s"DROP TABLE IF EXISTS $tableName1", + DropTableCommand(tableIdent1, ifExists = true, isView = false, purge = false)) + parseResolveCompare(s"DROP TABLE $tableName2", + DropTableCommand(tableIdent2, ifExists = false, isView = false, purge = false)) + parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2", + DropTableCommand(tableIdent2, ifExists = true, isView = false, purge = false)) + parseResolveCompare(s"DROP TABLE $tableName2 PURGE", + DropTableCommand(tableIdent2, ifExists = false, isView = false, purge = true)) + parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2 PURGE", + DropTableCommand(tableIdent2, ifExists = true, isView = false, purge = true)) + } + + test("drop table in v2 catalog") { + val tableName1 = "testcat.db.tab" + val tableIdent1 = Identifier.of(Array("db"), "tab") + val tableName2 = "testcat.tab" + val tableIdent2 = Identifier.of(Array.empty, "tab") + + parseResolveCompare(s"DROP TABLE $tableName1", + DropTable(testCat, tableIdent1, ifExists = false)) + parseResolveCompare(s"DROP TABLE IF EXISTS $tableName1", + DropTable(testCat, tableIdent1, ifExists = true)) + parseResolveCompare(s"DROP TABLE $tableName2", + DropTable(testCat, tableIdent2, ifExists = false)) + parseResolveCompare(s"DROP TABLE IF EXISTS $tableName2", + DropTable(testCat, tableIdent2, ifExists = true)) + } + + test("drop view") { + val viewName1 = "db.view" + val viewIdent1 = TableIdentifier("view", Option("db")) + val viewName2 = "view" + val viewIdent2 = TableIdentifier("view") + + parseResolveCompare(s"DROP VIEW $viewName1", + DropTableCommand(viewIdent1, ifExists = false, isView = true, purge = false)) + parseResolveCompare(s"DROP VIEW IF EXISTS $viewName1", + DropTableCommand(viewIdent1, ifExists = true, isView = true, purge = false)) + parseResolveCompare(s"DROP VIEW $viewName2", + DropTableCommand(viewIdent2, ifExists = false, isView = true, purge = false)) + parseResolveCompare(s"DROP VIEW IF EXISTS $viewName2", + DropTableCommand(viewIdent2, ifExists = true, isView = true, purge = false)) + } + + test("drop view in v2 catalog") { + intercept[AnalysisException] { + parseAndResolve("DROP VIEW testcat.db.view") + }.getMessage.toLowerCase(Locale.ROOT).contains( + "view support in catalog has not been implemented") + } + + // ALTER VIEW view_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER VIEW view_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter view: alter view properties") { + val sql1_view = "ALTER VIEW table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2_view = "ALTER VIEW table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3_view = "ALTER VIEW table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + val parsed1_view = parseAndResolve(sql1_view) + val parsed2_view = parseAndResolve(sql2_view) + val parsed3_view = parseAndResolve(sql3_view) + + val tableIdent = TableIdentifier("table_name", None) + val expected1_view = AlterTableSetPropertiesCommand( + tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = true) + val expected2_view = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = false, isView = true) + val expected3_view = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = true, isView = true) + + comparePlans(parsed1_view, expected1_view) + comparePlans(parsed2_view, expected2_view) + comparePlans(parsed3_view, expected3_view) + } + + // ALTER TABLE table_name SET TBLPROPERTIES ('comment' = new_comment); + // ALTER TABLE table_name UNSET TBLPROPERTIES [IF EXISTS] ('comment', 'key'); + test("alter table: alter table properties") { + val sql1_table = "ALTER TABLE table_name SET TBLPROPERTIES ('test' = 'test', " + + "'comment' = 'new_comment')" + val sql2_table = "ALTER TABLE table_name UNSET TBLPROPERTIES ('comment', 'test')" + val sql3_table = "ALTER TABLE table_name UNSET TBLPROPERTIES IF EXISTS ('comment', 'test')" + + val parsed1_table = parseAndResolve(sql1_table) + val parsed2_table = parseAndResolve(sql2_table) + val parsed3_table = parseAndResolve(sql3_table) + + val tableIdent = TableIdentifier("table_name", None) + val expected1_table = AlterTableSetPropertiesCommand( + tableIdent, Map("test" -> "test", "comment" -> "new_comment"), isView = false) + val expected2_table = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = false, isView = false) + val expected3_table = AlterTableUnsetPropertiesCommand( + tableIdent, Seq("comment", "test"), ifExists = true, isView = false) + + comparePlans(parsed1_table, expected1_table) + comparePlans(parsed2_table, expected2_table) + comparePlans(parsed3_table, expected3_table) + } + + test("support for other types in TBLPROPERTIES") { + val sql = + """ + |ALTER TABLE table_name + |SET TBLPROPERTIES ('a' = 1, 'b' = 0.1, 'c' = TRUE) + """.stripMargin + val parsed = parseAndResolve(sql) + val expected = AlterTableSetPropertiesCommand( + TableIdentifier("table_name"), + Map("a" -> "1", "b" -> "0.1", "c" -> "true"), + isView = false) + + comparePlans(parsed, expected) + } + + test("alter table: set location") { + val sql1 = "ALTER TABLE table_name SET LOCATION 'new location'" + val parsed1 = parseAndResolve(sql1) + val tableIdent = TableIdentifier("table_name", None) + val expected1 = AlterTableSetLocationCommand( + tableIdent, + None, + "new location") + comparePlans(parsed1, expected1) + } +} 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 f20aded169e44..2f5d5551c5df0 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 @@ -219,6 +219,13 @@ class DataSourceStrategySuite extends PlanTest with SharedSQLContext { IsNotNull(attrInt))), None) } + test("SPARK-26865 DataSourceV2Strategy should push normalized filters") { + val attrInt = 'cint.int + assertResult(Seq(IsNotNull(attrInt))) { + DataSourceStrategy.normalizeFilters(Seq(IsNotNull(attrInt.withName("CiNt"))), Seq(attrInt)) + } + } + /** * Translate the given Catalyst [[Expression]] into data source [[sources.Filter]] * then verify against the given [[sources.Filter]]. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala new file mode 100644 index 0000000000000..bdb161d59a33e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileBasedDataSourceTest.scala @@ -0,0 +1,106 @@ +/* + * 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.File + +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.test.SQLTestUtils + +/** + * A helper trait that provides convenient facilities for file-based data source testing. + * Specifically, it is used for Parquet and Orc testing. It can be used to write tests + * that are shared between Parquet and Orc. + */ +private[sql] trait FileBasedDataSourceTest extends SQLTestUtils { + + // Defines the data source name to run the test. + protected val dataSourceName: String + // The SQL config key for enabling vectorized reader. + protected val vectorizedReaderEnabledKey: String + + /** + * Reads data source file from given `path` as `DataFrame` and passes it to given function. + * + * @param path The path to file + * @param testVectorized Whether to read the file with vectorized reader. + * @param f The given function that takes a `DataFrame` as input. + */ + protected def readFile(path: String, testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = { + withSQLConf(vectorizedReaderEnabledKey -> "false") { + f(spark.read.format(dataSourceName).load(path.toString)) + } + if (testVectorized) { + withSQLConf(vectorizedReaderEnabledKey -> "true") { + f(spark.read.format(dataSourceName).load(path.toString)) + } + } + } + + /** + * Writes `data` to a data source file, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected def withDataSourceFile[T <: Product : ClassTag : TypeTag] + (data: Seq[T]) + (f: String => Unit): Unit = { + withTempPath { file => + spark.createDataFrame(data).write.format(dataSourceName).save(file.getCanonicalPath) + f(file.getCanonicalPath) + } + } + + /** + * Writes `data` to a data source file and reads it back as a [[DataFrame]], + * which is then passed to `f`. The file will be deleted after `f` returns. + */ + protected def withDataSourceDataFrame[T <: Product : ClassTag : TypeTag] + (data: Seq[T], testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = { + withDataSourceFile(data)(path => readFile(path.toString, testVectorized)(f)) + } + + /** + * Writes `data` to a data source file, reads it back as a [[DataFrame]] and registers it as a + * temporary table named `tableName`, then call `f`. The temporary table together with the + * data file will be dropped/deleted after `f` returns. + */ + protected def withDataSourceTable[T <: Product : ClassTag : TypeTag] + (data: Seq[T], tableName: String, testVectorized: Boolean = true) + (f: => Unit): Unit = { + withDataSourceDataFrame(data, testVectorized) { df => + df.createOrReplaceTempView(tableName) + withTempView(tableName)(f) + } + } + + protected def makeDataSourceFile[T <: Product : ClassTag : TypeTag]( + data: Seq[T], path: File): Unit = { + spark.createDataFrame(data).write.mode(SaveMode.Overwrite).format(dataSourceName) + .save(path.getCanonicalPath) + } + + protected def makeDataSourceFile[T <: Product : ClassTag : TypeTag]( + df: DataFrame, path: File): Unit = { + df.write.mode(SaveMode.Overwrite).format(dataSourceName).save(path.getCanonicalPath) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala index 13f0e0bca86c7..e09ec0d7bbb49 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileFormatWriterSuite.scala @@ -18,9 +18,14 @@ package org.apache.spark.sql.execution.datasources import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.catalyst.plans.CodegenInterpretedPlanTest import org.apache.spark.sql.test.SharedSQLContext -class FileFormatWriterSuite extends QueryTest with SharedSQLContext { +class FileFormatWriterSuite + extends QueryTest + with SharedSQLContext + with CodegenInterpretedPlanTest{ + import testImplicits._ test("empty file should be skipped while write to file") { @@ -44,4 +49,16 @@ class FileFormatWriterSuite extends QueryTest with SharedSQLContext { checkAnswer(spark.table("t4"), Row(0, 0)) } } + + test("Null and '' values should not cause dynamic partition failure of string types") { + withTable("t1", "t2") { + Seq((0, None), (1, Some("")), (2, None)).toDF("id", "p") + .write.partitionBy("p").saveAsTable("t1") + checkAnswer(spark.table("t1").sort("id"), Seq(Row(0, null), Row(1, null), Row(2, null))) + + sql("create table t2(id long, p string) using parquet partitioned by (p)") + sql("insert overwrite table t2 partition(p) select id, p from t1") + checkAnswer(spark.table("t2").sort("id"), Seq(Row(0, null), Row(1, null), Row(2, null))) + } + } } 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 49e7af4a9896b..2a5c5a2dd0ff8 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 @@ -17,38 +17,132 @@ package org.apache.spark.sql.execution.datasources -import java.io.File +import java.io.{File, FileNotFoundException} import java.net.URI import scala.collection.mutable -import scala.language.reflectiveCalls import org.apache.hadoop.fs.{BlockLocation, FileStatus, LocatedFileStatus, Path, RawLocalFileSystem} +import org.apache.spark.SparkException import org.apache.spark.metrics.source.HiveCatalogMetrics +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.functions.col import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.util.{KnownSizeEstimation, SizeEstimator} +import org.apache.spark.sql.types.{IntegerType, StringType, StructField, StructType} +import org.apache.spark.util.KnownSizeEstimation class FileIndexSuite extends SharedSQLContext { + private class TestInMemoryFileIndex( + spark: SparkSession, + path: Path, + fileStatusCache: FileStatusCache = NoopCache) + extends InMemoryFileIndex(spark, Seq(path), Map.empty, None, fileStatusCache) { + def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq + def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq + def leafFileStatuses: Iterable[FileStatus] = leafFiles.values + } + test("InMemoryFileIndex: leaf files are qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") stringToFile(file, "text") val path = new Path(file.getCanonicalPath) - val catalog = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) { - def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq - def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq - } + val catalog = new TestInMemoryFileIndex(spark, path) assert(catalog.leafFilePaths.forall(p => p.toString.startsWith("file:/"))) assert(catalog.leafDirPaths.forall(p => p.toString.startsWith("file:/"))) } } + test("SPARK-26188: don't infer data types of partition columns if user specifies schema") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=4d") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("a", StringType, false))) + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 1 && partitionValues(0).numFields == 1 && + partitionValues(0).getString(0) == "4d") + } + } + + test("SPARK-26990: use user specified field names if possible") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=foo") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("A", StringType, false))) + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + assert(fileIndex.partitionSchema.length == 1 && fileIndex.partitionSchema.head.name == "A") + } + } + } + + test("SPARK-26230: if case sensitive, validate partitions with original column names") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=1") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val partitionDirectory2 = new File(dir, "A=2") + partitionDirectory2.mkdir() + val file2 = new File(partitionDirectory2, "text.txt") + stringToFile(file2, "text") + val path = new Path(dir.getCanonicalPath) + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 2) + } + + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + val msg = intercept[AssertionError] { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, None) + fileIndex.partitionSpec() + }.getMessage + assert(msg.contains("Conflicting partition column names detected")) + assert("Partition column name list #[0-1]: A".r.findFirstIn(msg).isDefined) + assert("Partition column name list #[0-1]: a".r.findFirstIn(msg).isDefined) + } + } + } + + test("SPARK-26263: Throw exception when partition value can't be casted to user-specified type") { + withTempDir { dir => + val partitionDirectory = new File(dir, "a=foo") + partitionDirectory.mkdir() + val file = new File(partitionDirectory, "text.txt") + stringToFile(file, "text") + val path = new Path(dir.getCanonicalPath) + val schema = StructType(Seq(StructField("a", IntegerType, false))) + withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "true") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val msg = intercept[RuntimeException] { + fileIndex.partitionSpec() + }.getMessage + assert(msg == "Failed to cast value `foo` to `IntegerType` for partition column `a`") + } + + withSQLConf(SQLConf.VALIDATE_PARTITION_COLUMNS.key -> "false") { + val fileIndex = new InMemoryFileIndex(spark, Seq(path), Map.empty, Some(schema)) + val partitionValues = fileIndex.partitionSpec().partitions.map(_.values) + assert(partitionValues.length == 1 && partitionValues(0).numFields == 1 && + partitionValues(0).isNullAt(0)) + } + } + } + test("InMemoryFileIndex: input paths are converted to qualified paths") { withTempDir { dir => val file = new File(dir, "text.txt") @@ -74,7 +168,7 @@ class FileIndexSuite extends SharedSQLContext { } } - test("InMemoryFileIndex: folders that don't exist don't throw exceptions") { + test("InMemoryFileIndex: root folders that don't exist don't throw exceptions") { withTempDir { dir => val deletedFolder = new File(dir, "deleted") assert(!deletedFolder.exists()) @@ -85,6 +179,67 @@ class FileIndexSuite extends SharedSQLContext { } } + test("SPARK-27676: InMemoryFileIndex respects ignoreMissingFiles config for non-root paths") { + import DeletionRaceFileSystem._ + for ( + raceCondition <- Seq( + classOf[SubdirectoryDeletionRaceFileSystem], + classOf[FileDeletionRaceFileSystem] + ); + ignoreMissingFiles <- Seq(true, false); + parDiscoveryThreshold <- Seq(0, 100) + ) { + withClue(s"raceCondition=$raceCondition, ignoreMissingFiles=$ignoreMissingFiles, " + + s"parDiscoveryThreshold=$parDiscoveryThreshold" + ) { + withSQLConf( + SQLConf.IGNORE_MISSING_FILES.key -> ignoreMissingFiles.toString, + SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> parDiscoveryThreshold.toString, + "fs.mockFs.impl" -> raceCondition.getName, + "fs.mockFs.impl.disable.cache" -> "true" + ) { + def makeCatalog(): InMemoryFileIndex = new InMemoryFileIndex( + spark, Seq(rootDirPath), Map.empty, None) + if (ignoreMissingFiles) { + // We're ignoring missing files, so catalog construction should succeed + val catalog = makeCatalog() + val leafFiles = catalog.listLeafFiles(catalog.rootPaths) + if (raceCondition == classOf[SubdirectoryDeletionRaceFileSystem]) { + // The only subdirectory was missing, so there should be no leaf files: + assert(leafFiles.isEmpty) + } else { + assert(raceCondition == classOf[FileDeletionRaceFileSystem]) + // One of the two leaf files was missing, but we should still list the other: + assert(leafFiles.size == 1) + assert(leafFiles.head.getPath == nonDeletedLeafFilePath) + } + } else { + // We're NOT ignoring missing files, so catalog construction should fail + val e = intercept[Exception] { + makeCatalog() + } + // The exact exception depends on whether we're using parallel listing + if (parDiscoveryThreshold == 0) { + // The FileNotFoundException occurs in a Spark executor (as part of a job) + assert(e.isInstanceOf[SparkException]) + assert(e.getMessage.contains("FileNotFoundException")) + } else { + // The FileNotFoundException occurs directly on the driver + assert(e.isInstanceOf[FileNotFoundException]) + // Test that the FileNotFoundException is triggered for the expected reason: + if (raceCondition == classOf[SubdirectoryDeletionRaceFileSystem]) { + assert(e.getMessage.contains(subDirPath.toString)) + } else { + assert(raceCondition == classOf[FileDeletionRaceFileSystem]) + assert(e.getMessage.contains(leafFilePath.toString)) + } + } + } + } + } + } + } + test("PartitioningAwareFileIndex listing parallelized with many top level dirs") { for ((scale, expectedNumPar) <- Seq((10, 0), (50, 1))) { withTempDir { dir => @@ -202,11 +357,7 @@ class FileIndexSuite extends SharedSQLContext { val fileStatusCache = FileStatusCache.getOrCreate(spark) val dirPath = new Path(dir.getAbsolutePath) val fs = dirPath.getFileSystem(spark.sessionState.newHadoopConf()) - val catalog = - new InMemoryFileIndex(spark, Seq(dirPath), Map.empty, None, fileStatusCache) { - def leafFilePaths: Seq[Path] = leafFiles.keys.toSeq - def leafDirPaths: Seq[Path] = leafDirToChildrenFiles.keys.toSeq - } + val catalog = new TestInMemoryFileIndex(spark, dirPath, fileStatusCache) val file = new File(dir, "text.txt") stringToFile(file, "text") @@ -256,10 +407,7 @@ class FileIndexSuite extends SharedSQLContext { val file = new File(dir, "text.txt") stringToFile(file, "text") - val inMemoryFileIndex = new InMemoryFileIndex( - spark, Seq(new Path(file.getCanonicalPath)), Map.empty, None) { - def leafFileStatuses = leafFiles.values - } + val inMemoryFileIndex = new TestInMemoryFileIndex(spark, new Path(file.getCanonicalPath)) val blockLocations = inMemoryFileIndex.leafFileStatuses.flatMap( _.asInstanceOf[LocatedFileStatus].getBlockLocations) @@ -270,6 +418,66 @@ class FileIndexSuite extends SharedSQLContext { } +object DeletionRaceFileSystem { + val rootDirPath: Path = new Path("mockFs:///rootDir/") + val subDirPath: Path = new Path(rootDirPath, "subDir") + val leafFilePath: Path = new Path(subDirPath, "leafFile") + val nonDeletedLeafFilePath: Path = new Path(subDirPath, "nonDeletedLeafFile") + val rootListing: Array[FileStatus] = + Array(new FileStatus(0, true, 0, 0, 0, subDirPath)) + val subFolderListing: Array[FileStatus] = + Array( + new FileStatus(0, false, 0, 100, 0, leafFilePath), + new FileStatus(0, false, 0, 100, 0, nonDeletedLeafFilePath)) +} + +// Used in SPARK-27676 test to simulate a race where a subdirectory is deleted +// between back-to-back listing calls. +class SubdirectoryDeletionRaceFileSystem extends RawLocalFileSystem { + import DeletionRaceFileSystem._ + + override def getScheme: String = "mockFs" + + override def listStatus(path: Path): Array[FileStatus] = { + if (path == rootDirPath) { + rootListing + } else if (path == subDirPath) { + throw new FileNotFoundException(subDirPath.toString) + } else { + throw new IllegalArgumentException() + } + } +} + +// Used in SPARK-27676 test to simulate a race where a file is deleted between +// being listed and having its size / file status checked. +class FileDeletionRaceFileSystem extends RawLocalFileSystem { + import DeletionRaceFileSystem._ + + override def getScheme: String = "mockFs" + + override def listStatus(path: Path): Array[FileStatus] = { + if (path == rootDirPath) { + rootListing + } else if (path == subDirPath) { + subFolderListing + } else { + throw new IllegalArgumentException() + } + } + + override def getFileBlockLocations( + file: FileStatus, + start: Long, + len: Long): Array[BlockLocation] = { + if (file.getPath == leafFilePath) { + throw new FileNotFoundException(leafFilePath.toString) + } else { + Array.empty + } + } +} + class FakeParentPathFileSystem extends RawLocalFileSystem { override def getScheme: String = "mockFs" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala index bceaf1a9ec061..eaff5a2352a0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/FileSourceStrategySuite.scala @@ -201,7 +201,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("partitioned table - case insensitive") { - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { val table = createTable( files = Seq( @@ -279,7 +279,7 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("Locality support for FileScanRDD") { - val partition = FilePartition(0, Seq( + val partition = FilePartition(0, Array( PartitionedFile(InternalRow.empty, "fakePath0", 0, 10, Array("host0", "host1")), PartitionedFile(InternalRow.empty, "fakePath0", 10, 20, Array("host1", "host2")), PartitionedFile(InternalRow.empty, "fakePath1", 0, 5, Array("host3")), @@ -414,24 +414,30 @@ class FileSourceStrategySuite extends QueryTest with SharedSQLContext with Predi } test("[SPARK-16818] partition pruned file scans implement sameResult correctly") { - withTempPath { path => - val tempDir = path.getCanonicalPath - spark.range(100) - .selectExpr("id", "id as b") - .write - .partitionBy("id") - .parquet(tempDir) - val df = spark.read.parquet(tempDir) - def getPlan(df: DataFrame): SparkPlan = { - df.queryExecution.executedPlan + Seq("orc", "").foreach { useV1ReaderList => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1ReaderList) { + withTempPath { path => + val tempDir = path.getCanonicalPath + spark.range(100) + .selectExpr("id", "id as b") + .write + .partitionBy("id") + .orc(tempDir) + val df = spark.read.orc(tempDir) + + def getPlan(df: DataFrame): SparkPlan = { + df.queryExecution.executedPlan + } + + assert(getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 2")))) + assert(!getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 3")))) + } } - assert(getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 2")))) - assert(!getPlan(df.where("id = 2")).sameResult(getPlan(df.where("id = 3")))) } } test("[SPARK-16818] exchange reuse respects differences in partition pruning") { - spark.conf.set("spark.sql.exchange.reuse", true) + spark.conf.set(SQLConf.EXCHANGE_REUSE_ENABLED.key, true) withTempPath { path => val tempDir = path.getCanonicalPath spark.range(10) @@ -614,7 +620,7 @@ class TestFileFormat extends TextBasedFileFormat { job: Job, options: Map[String, String], dataSchema: StructType): OutputWriterFactory = { - throw new NotImplementedError("JUST FOR TESTING") + throw new UnsupportedOperationException("JUST FOR TESTING") } override def buildReader( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala deleted file mode 100644 index c1f2c18d1417d..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/HadoopFsRelationSuite.scala +++ /dev/null @@ -1,83 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources - -import java.io.{File, FilenameFilter} - -import org.apache.spark.sql.QueryTest -import org.apache.spark.sql.execution.joins.{BroadcastHashJoinExec, SortMergeJoinExec} -import org.apache.spark.sql.test.SharedSQLContext - -class HadoopFsRelationSuite extends QueryTest with SharedSQLContext { - - test("sizeInBytes should be the total size of all files") { - withTempDir{ dir => - dir.delete() - spark.range(1000).write.parquet(dir.toString) - // ignore hidden files - val allFiles = dir.listFiles(new FilenameFilter { - override def accept(dir: File, name: String): Boolean = { - !name.startsWith(".") && !name.startsWith("_") - } - }) - val totalSize = allFiles.map(_.length()).sum - val df = spark.read.parquet(dir.toString) - assert(df.queryExecution.logical.stats.sizeInBytes === BigInt(totalSize)) - } - } - - test("SPARK-22790: spark.sql.sources.compressionFactor takes effect") { - import testImplicits._ - Seq(1.0, 0.5).foreach { compressionFactor => - withSQLConf("spark.sql.sources.fileCompressionFactor" -> compressionFactor.toString, - "spark.sql.autoBroadcastJoinThreshold" -> "400") { - withTempPath { workDir => - // the file size is 740 bytes - val workDirPath = workDir.getAbsolutePath - val data1 = Seq(100, 200, 300, 400).toDF("count") - data1.write.parquet(workDirPath + "/data1") - val df1FromFile = spark.read.parquet(workDirPath + "/data1") - val data2 = Seq(100, 200, 300, 400).toDF("count") - data2.write.parquet(workDirPath + "/data2") - val df2FromFile = spark.read.parquet(workDirPath + "/data2") - val joinedDF = df1FromFile.join(df2FromFile, Seq("count")) - if (compressionFactor == 0.5) { - val bJoinExec = joinedDF.queryExecution.executedPlan.collect { - case bJoin: BroadcastHashJoinExec => bJoin - } - assert(bJoinExec.nonEmpty) - val smJoinExec = joinedDF.queryExecution.executedPlan.collect { - case smJoin: SortMergeJoinExec => smJoin - } - assert(smJoinExec.isEmpty) - } else { - // compressionFactor is 1.0 - val bJoinExec = joinedDF.queryExecution.executedPlan.collect { - case bJoin: BroadcastHashJoinExec => bJoin - } - assert(bJoinExec.isEmpty) - val smJoinExec = joinedDF.queryExecution.executedPlan.collect { - case smJoin: SortMergeJoinExec => smJoin - } - assert(smJoinExec.nonEmpty) - } - } - } - } - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadNestedSchemaTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadNestedSchemaTest.scala new file mode 100644 index 0000000000000..2b0fdfe277810 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadNestedSchemaTest.scala @@ -0,0 +1,158 @@ +/* + * 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.File + +import org.apache.spark.sql.{DataFrame, Row} + + +/** + * Add a nested column. + */ +trait AddNestedColumnTest extends ReadSchemaTest { + + private def testAdd(df1: DataFrame, df2: DataFrame, expectedRows: Seq[Row]): Unit = { + withTempPath { dir => + val path = dir.getCanonicalPath + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(df2.schema) + .format(format) + .options(options) + .load(path) + + checkAnswer(df, expectedRows) + } + } + + test("add a nested column at the end of the leaf struct column") { + testAdd( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4, 'c7', 5)) c2"), + Seq( + Row(1, Row(2, Row(3, 4, null)), "one"), + Row(1, Row(2, Row(3, 4, 5)), "two"))) + } + + test("add a nested column in the middle of the leaf struct column") { + testAdd( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c7', 5, 'c6', 4)) c2"), + Seq( + Row(1, Row(2, Row(3, null, 4)), "one"), + Row(1, Row(2, Row(3, 5, 4)), "two"))) + } + + test("add a nested column at the end of the middle struct column") { + testAdd( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4), 'c7', 5) c2"), + Seq( + Row(1, Row(2, Row(3, 4), null), "one"), + Row(1, Row(2, Row(3, 4), 5), "two"))) + } + + test("add a nested column in the middle of the middle struct column") { + testAdd( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 1 c1, named_struct('c3', 2, 'c7', 5, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + Seq( + Row(1, Row(2, null, Row(3, 4)), "one"), + Row(1, Row(2, 5, Row(3, 4)), "two"))) + } +} + +/** + * Hide a nested column. + */ +trait HideNestedColumnTest extends ReadSchemaTest { + + private def testHide(df1: DataFrame, df2: DataFrame, df3: DataFrame, expectedRows: Seq[Row]) = { + withTempPath { dir => + val path = dir.getCanonicalPath + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + val dir3 = s"$path${File.separator}part=three" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + df3.write.format(format).options(options).save(dir3) + + val df = spark.read + .schema(df1.schema) + .format(format) + .options(options) + .load(path) + + checkAnswer(df, expectedRows) + } + } + + test("hide a nested column at the end of the leaf struct column") { + testHide( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 5 c1, named_struct('c3', 6, 'c4', named_struct('c5', 7, 'c6', 8, 'c7', 9)) c2"), + sql("SELECT 0 c1, named_struct('c3', 1, 'c4', named_struct('c5', 2, 'c6', 3, 'c8', 4)) c2"), + Seq( + Row(1, Row(2, Row(3, 4)), "one"), + Row(5, Row(6, Row(7, 8)), "two"), + Row(0, Row(1, Row(2, 3)), "three"))) + } + + test("hide a nested column in the middle of the leaf struct column") { + testHide( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 5 c1, named_struct('c3', 6, 'c4', named_struct('c5', 7, 'c7', 8, 'c6', 9)) c2"), + sql("SELECT 0 c1, named_struct('c3', 1, 'c4', named_struct('c7', 2, 'c5', 3, 'c6', 4)) c2"), + Seq( + Row(1, Row(2, Row(3, 4)), "one"), + Row(5, Row(6, Row(7, 9)), "two"), + Row(0, Row(1, Row(3, 4)), "three"))) + } + + test("hide a nested column at the end of the middle struct column") { + testHide( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 5 c1, named_struct('c3', 6, 'c4', named_struct('c5', 7, 'c6', 8), 'c7', 9) c2"), + sql("SELECT 0 c1, named_struct('c3', 1, 'c4', named_struct('c5', 2, 'c6', 3), 'c8', 4) c2"), + Seq( + Row(1, Row(2, Row(3, 4)), "one"), + Row(5, Row(6, Row(7, 8)), "two"), + Row(0, Row(1, Row(2, 3)), "three"))) + } + + test("hide a nested column in the middle of the middle struct column") { + testHide( + sql("SELECT 1 c1, named_struct('c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + sql("SELECT 5 c1, named_struct('c3', 6, 'c7', 7, 'c4', named_struct('c5', 8, 'c6', 9)) c2"), + sql("SELECT 0 c1, named_struct('c7', 1, 'c3', 2, 'c4', named_struct('c5', 3, 'c6', 4)) c2"), + Seq( + Row(1, Row(2, Row(3, 4)), "one"), + Row(5, Row(6, Row(8, 9)), "two"), + Row(0, Row(2, Row(3, 4)), "three"))) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala index 23c58e175fe5e..d5502ba5737c0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources +import org.apache.spark.SparkConf import org.apache.spark.sql.internal.SQLConf /** @@ -32,10 +33,13 @@ import org.apache.spark.sql.internal.SQLConf * * -> OrcReadSchemaSuite * -> VectorizedOrcReadSchemaSuite + * -> MergedOrcReadSchemaSuite * * -> ParquetReadSchemaSuite * -> VectorizedParquetReadSchemaSuite * -> MergedParquetReadSchemaSuite + * + * -> AvroReadSchemaSuite */ /** @@ -72,7 +76,10 @@ class HeaderCSVReadSchemaSuite class JsonReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest with IntegralTypeTest with ToDoubleTypeTest @@ -84,7 +91,10 @@ class JsonReadSchemaSuite class OrcReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest { override val format: String = "orc" @@ -103,7 +113,10 @@ class OrcReadSchemaSuite class VectorizedOrcReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest with BooleanTypeTest with IntegralTypeTest @@ -123,9 +136,31 @@ class VectorizedOrcReadSchemaSuite } } +class MergedOrcReadSchemaSuite + extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest + with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest + with ChangePositionTest + with BooleanTypeTest + with IntegralTypeTest + with ToDoubleTypeTest { + + override val format: String = "orc" + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key, "true") +} + class ParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest { override val format: String = "parquet" @@ -144,7 +179,10 @@ class ParquetReadSchemaSuite class VectorizedParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest { override val format: String = "parquet" @@ -163,7 +201,10 @@ class VectorizedParquetReadSchemaSuite class MergedParquetReadSchemaSuite extends ReadSchemaSuite + with AddColumnIntoTheMiddleTest with HideColumnInTheMiddleTest + with AddNestedColumnTest + with HideNestedColumnTest with ChangePositionTest { override val format: String = "parquet" diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala index 2a5457e00b4ef..d42809529ceed 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/ReadSchemaTest.scala @@ -46,6 +46,7 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} * | JSON | 1, 2, 3, 4 | | * | ORC | 1, 2, 3, 4 | Native vectorized ORC reader has the widest coverage. | * | PARQUET | 1, 2, 3 | | + * | AVRO | 1, 2, 3 | | * * This aims to provide an explicit test coverage for reader schema change on file-based data * sources. Since a file format has its own coverage, we need a test suite for each file-based @@ -55,9 +56,12 @@ import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} * * ReadSchemaTest * -> AddColumnTest - * -> HideColumnTest + * -> AddColumnIntoTheMiddleTest + * -> HideColumnAtTheEndTest + * -> HideColumnInTheMiddleTest * -> ChangePositionTest * -> BooleanTypeTest + * -> ToStringTypeTest * -> IntegralTypeTest * -> ToDoubleTypeTest * -> ToDecimalTypeTest @@ -69,7 +73,7 @@ trait ReadSchemaTest extends QueryTest with SQLTestUtils with SharedSQLContext { } /** - * Add column (Case 1). + * Add column (Case 1-1). * This test suite assumes that the missing column should be `null`. */ trait AddColumnTest extends ReadSchemaTest { @@ -108,6 +112,43 @@ trait AddColumnTest extends ReadSchemaTest { } } +/** + * Add column into the middle (Case 1-2). + */ +trait AddColumnIntoTheMiddleTest extends ReadSchemaTest { + import testImplicits._ + + test("append column into middle") { + withTempPath { dir => + val path = dir.getCanonicalPath + + val df1 = Seq((1, 2, "abc"), (4, 5, "def"), (8, 9, null)).toDF("col1", "col2", "col3") + val df2 = Seq((10, null, 20, null), (40, "uvw", 50, "xyz"), (80, null, 90, null)) + .toDF("col1", "col4", "col2", "col3") + + val dir1 = s"$path${File.separator}part=one" + val dir2 = s"$path${File.separator}part=two" + + df1.write.format(format).options(options).save(dir1) + df2.write.format(format).options(options).save(dir2) + + val df = spark.read + .schema(df2.schema) + .format(format) + .options(options) + .load(path) + + checkAnswer(df, Seq( + Row(1, null, 2, "abc", "one"), + Row(4, null, 5, "def", "one"), + Row(8, null, 9, null, "one"), + Row(10, null, 20, null, "two"), + Row(40, "uvw", 50, "xyz", "two"), + Row(80, null, 90, null, "two"))) + } + } +} + /** * Hide column (Case 2-1). */ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala index a1da3ec43eae3..8b06b175a2103 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SaveIntoDataSourceCommandSuite.scala @@ -25,7 +25,7 @@ class SaveIntoDataSourceCommandSuite extends SharedSQLContext { test("simpleString is redacted") { val URL = "connection.url" - val PASS = "123" + val PASS = "mypassword" val DRIVER = "mydriver" val dataSource = DataSource( 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 new file mode 100644 index 0000000000000..09ca42851836b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/SchemaPruningSuite.scala @@ -0,0 +1,440 @@ +/* + * 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.File + +import org.scalactic.Equality + +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.SchemaPruningTest +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType + +abstract class SchemaPruningSuite + extends QueryTest + with FileBasedDataSourceTest + with SchemaPruningTest + with SharedSQLContext { + case class FullName(first: String, middle: String, last: String) + case class Company(name: String, address: String) + case class Employer(id: Int, company: Company) + case class Contact( + id: Int, + name: FullName, + address: String, + pets: Int, + friends: Array[FullName] = Array.empty, + relatives: Map[String, FullName] = Map.empty, + employer: Employer = null, + relations: Map[FullName, String] = Map.empty) + + val janeDoe = FullName("Jane", "X.", "Doe") + val johnDoe = FullName("John", "Y.", "Doe") + val susanSmith = FullName("Susan", "Z.", "Smith") + + val employer = Employer(0, Company("abc", "123 Business Street")) + val employerWithNullCompany = Employer(1, null) + + val contacts = + Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), + relatives = Map("brother" -> johnDoe), employer = employer, + relations = Map(johnDoe -> "brother")) :: + Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe), + employer = employerWithNullCompany, relations = Map(janeDoe -> "sister")) :: Nil + + case class Name(first: String, last: String) + case class BriefContact(id: Int, name: Name, address: String) + + private val briefContacts = + BriefContact(2, Name("Janet", "Jones"), "567 Maple Drive") :: + BriefContact(3, Name("Jim", "Jones"), "6242 Ash Street") :: Nil + + case class ContactWithDataPartitionColumn( + id: Int, + name: FullName, + address: String, + pets: Int, + friends: Array[FullName] = Array(), + relatives: Map[String, FullName] = Map(), + employer: Employer = null, + relations: Map[FullName, String] = Map(), + p: Int) + + case class BriefContactWithDataPartitionColumn(id: Int, name: Name, address: String, p: Int) + + val contactsWithDataPartitionColumn = + contacts.map {case Contact(id, name, address, pets, friends, relatives, employer, relations) => + ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, employer, + relations, 1) } + val briefContactsWithDataPartitionColumn = + briefContacts.map { case BriefContact(id, name, address) => + BriefContactWithDataPartitionColumn(id, name, address, 2) } + + testSchemaPruning("select a single complex field") { + val query = sql("select name.middle from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) + } + + testSchemaPruning("select a single complex field and its parent struct") { + val query = sql("select name.middle, name from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("X.", Row("Jane", "X.", "Doe")) :: + Row("Y.", Row("John", "Y.", "Doe")) :: + Row(null, Row("Janet", null, "Jones")) :: + Row(null, Row("Jim", null, "Jones")) :: + Nil) + } + + testSchemaPruning("select a single complex field array and its parent struct array") { + val query = sql("select friends.middle, friends from contacts where p=1") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row(Array("Z."), Array(Row("Susan", "Z.", "Smith"))) :: + Row(Array.empty[String], Array.empty[Row]) :: + Nil) + } + + testSchemaPruning("select a single complex field from a map entry and its parent map entry") { + val query = + sql("select relatives[\"brother\"].middle, relatives[\"brother\"] from contacts where p=1") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row("Y.", Row("John", "Y.", "Doe")) :: + Row(null, null) :: + Nil) + } + + testSchemaPruning("select a single complex field and the partition column") { + val query = sql("select name.middle, p from contacts") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) + } + + testSchemaPruning("partial schema intersection - select missing subfield") { + val query = sql("select name.middle, address from contacts where p=2") + checkScan(query, "struct,address:string>") + checkAnswer(query.orderBy("id"), + Row(null, "567 Maple Drive") :: + Row(null, "6242 Ash Street") :: Nil) + } + + testSchemaPruning("no unnecessary schema pruning") { + val query = + sql("select id, name.last, name.middle, name.first, relatives[''].last, " + + "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + + "friends[0].first, pets, address from contacts where p=2") + // We've selected every field in the schema. Therefore, no schema pruning should be performed. + // We check this by asserting that the scanned schema of the query is identical to the schema + // of the contacts relation, even though the fields are selected in different orders. + checkScan(query, + "struct,address:string,pets:int," + + "friends:array>," + + "relatives:map>>") + checkAnswer(query.orderBy("id"), + Row(2, "Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: + Row(3, "Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: + Nil) + } + + testSchemaPruning("empty schema intersection") { + val query = sql("select name.middle from contacts where p=2") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row(null) :: Row(null) :: Nil) + } + + testSchemaPruning("select a single complex field and in where clause") { + val query1 = sql("select name.first from contacts where name.first = 'Jane'") + checkScan(query1, "struct>") + checkAnswer(query1, Row("Jane") :: Nil) + + val query2 = sql("select name.first, name.last from contacts where name.first = 'Jane'") + checkScan(query2, "struct>") + checkAnswer(query2, Row("Jane", "Doe") :: Nil) + + val query3 = sql("select name.first from contacts " + + "where employer.company.name = 'abc' and p = 1") + checkScan(query3, "struct," + + "employer:struct>>") + checkAnswer(query3, Row("Jane") :: Nil) + + val query4 = sql("select name.first, employer.company.name from contacts " + + "where employer.company is not null and p = 1") + checkScan(query4, "struct," + + "employer:struct>>") + checkAnswer(query4, Row("Jane", "abc") :: Nil) + } + + testSchemaPruning("select nullable complex field and having is not null predicate") { + val query = sql("select employer.company from contacts " + + "where employer is not null and p = 1") + checkScan(query, "struct>>") + checkAnswer(query, Row(Row("abc", "123 Business Street")) :: Row(null) :: Nil) + } + + testSchemaPruning("select a single complex field and is null expression in project") { + val query = sql("select name.first, address is not null from contacts") + checkScan(query, "struct,address:string>") + checkAnswer(query.orderBy("id"), + Row("Jane", true) :: Row("John", true) :: Row("Janet", true) :: Row("Jim", true) :: Nil) + } + + testSchemaPruning("select a single complex field array and in clause") { + val query = sql("select friends.middle from contacts where friends.first[0] = 'Susan'") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row(Array("Z.")) :: Nil) + } + + testSchemaPruning("select a single complex field from a map entry and in clause") { + val query = + sql("select relatives[\"brother\"].middle from contacts " + + "where relatives[\"brother\"].first = 'John'") + checkScan(query, + "struct>>") + checkAnswer(query.orderBy("id"), + Row("Y.") :: Nil) + } + + testSchemaPruning("select one complex field and having is null predicate on another " + + "complex field") { + val query = sql("select * from contacts") + .where("name.middle is not null") + .select( + "id", + "name.first", + "name.middle", + "name.last" + ) + .where("last = 'Jones'") + .select(count("id")).toDF() + checkScan(query, + "struct>") + checkAnswer(query, Row(0) :: Nil) + } + + testSchemaPruning("select one deep nested complex field and having is null predicate on " + + "another deep nested complex field") { + val query = sql("select * from contacts") + .where("employer.company.address is not null") + .selectExpr( + "id", + "name.first", + "name.middle", + "name.last", + "employer.id as employer_id" + ) + .where("employer_id = 0") + .select(count("id")).toDF() + checkScan(query, + "struct>>") + checkAnswer(query, Row(1) :: Nil) + } + + testSchemaPruning("select nested field from a complex map key using map_keys") { + val query = sql("select map_keys(relations).middle[0], p from contacts") + checkScan(query, "struct,string>>") + checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) + } + + testSchemaPruning("select nested field from a complex map value using map_values") { + val query = sql("select map_values(relatives).middle[0], p from contacts") + checkScan(query, "struct>>") + checkAnswer(query, Row("Y.", 1) :: Row("X.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) + } + + protected def testSchemaPruning(testName: String)(testThunk: => Unit) { + test(s"Spark vectorized reader - without partition data column - $testName") { + withSQLConf(vectorizedReaderEnabledKey -> "true") { + withContacts(testThunk) + } + } + test(s"Spark vectorized reader - with partition data column - $testName") { + withSQLConf(vectorizedReaderEnabledKey -> "true") { + withContactsWithDataPartitionColumn(testThunk) + } + } + + test(s"Non-vectorized reader - without partition data column - $testName") { + withSQLConf(vectorizedReaderEnabledKey -> "false") { + withContacts(testThunk) + } + } + test(s"Non-vectorized reader - with partition data column - $testName") { + withSQLConf(vectorizedReaderEnabledKey-> "false") { + withContactsWithDataPartitionColumn(testThunk) + } + } + } + + private def withContacts(testThunk: => Unit) { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeDataSourceFile(contacts, new File(path + "/contacts/p=1")) + makeDataSourceFile(briefContacts, new File(path + "/contacts/p=2")) + + // Providing user specified schema. Inferred schema from different data sources might + // be different. + val schema = "`id` INT,`name` STRUCT<`first`: STRING, `middle`: STRING, `last`: STRING>, " + + "`address` STRING,`pets` INT,`friends` ARRAY>,`relatives` MAP>,`employer` STRUCT<`id`: INT, `company`: STRUCT<`name`: STRING, " + + "`address`: STRING>>,`relations` MAP,STRING>,`p` INT" + spark.read.format(dataSourceName).schema(schema).load(path + "/contacts") + .createOrReplaceTempView("contacts") + + testThunk + } + } + + private def withContactsWithDataPartitionColumn(testThunk: => Unit) { + withTempPath { dir => + val path = dir.getCanonicalPath + + makeDataSourceFile(contactsWithDataPartitionColumn, new File(path + "/contacts/p=1")) + makeDataSourceFile(briefContactsWithDataPartitionColumn, new File(path + "/contacts/p=2")) + + // Providing user specified schema. Inferred schema from different data sources might + // be different. + val schema = "`id` INT,`name` STRUCT<`first`: STRING, `middle`: STRING, `last`: STRING>, " + + "`address` STRING,`pets` INT,`friends` ARRAY>,`relatives` MAP>,`employer` STRUCT<`id`: INT, `company`: STRUCT<`name`: STRING, " + + "`address`: STRING>>,`relations` MAP,STRING>,`p` INT" + spark.read.format(dataSourceName).schema(schema).load(path + "/contacts") + .createOrReplaceTempView("contacts") + + testThunk + } + } + + case class MixedCaseColumn(a: String, B: Int) + case class MixedCase(id: Int, CoL1: String, coL2: MixedCaseColumn) + + private val mixedCaseData = + MixedCase(0, "r0c1", MixedCaseColumn("abc", 1)) :: + MixedCase(1, "r1c1", MixedCaseColumn("123", 2)) :: + Nil + + testExactCaseQueryPruning("select with exact column names") { + val query = sql("select CoL1, coL2.B from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCaseQueryPruning("select with lowercase column names") { + val query = sql("select col1, col2.b from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCaseQueryPruning("select with different-case column names") { + val query = sql("select cOL1, cOl2.b from mixedcase") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), + Row("r0c1", 1) :: + Row("r1c1", 2) :: + Nil) + } + + testMixedCaseQueryPruning("filter with different-case column names") { + val query = sql("select id from mixedcase where Col2.b = 2") + checkScan(query, "struct>") + checkAnswer(query.orderBy("id"), Row(1) :: Nil) + } + + // Tests schema pruning for a query whose column and field names are exactly the same as the table + // schema's column and field names. N.B. this implies that `testThunk` should pass using either a + // case-sensitive or case-insensitive query parser + private def testExactCaseQueryPruning(testName: String)(testThunk: => Unit) { + test(s"Case-sensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "true") { + withMixedCaseData(testThunk) + } + } + testMixedCaseQueryPruning(testName)(testThunk) + } + + // Tests schema pruning for a query whose column and field names may differ in case from the table + // schema's column and field names + private def testMixedCaseQueryPruning(testName: String)(testThunk: => Unit) { + test(s"Case-insensitive parser - mixed-case schema - $testName") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { + withMixedCaseData(testThunk) + } + } + } + + // Tests given test function with Spark vectorized reader and non-vectorized reader. + private def withMixedCaseData(testThunk: => Unit) { + withDataSourceTable(mixedCaseData, "mixedcase") { + testThunk + } + } + + protected val schemaEquality = new Equality[StructType] { + override def areEqual(a: StructType, b: Any): Boolean = + b match { + case otherType: StructType => a.sameType(otherType) + case _ => false + } + } + + protected def checkScan(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + checkScanSchemata(df, expectedSchemaCatalogStrings: _*) + // We check here that we can execute the query without throwing an exception. The results + // themselves are irrelevant, and should be checked elsewhere as needed + df.collect() + } + + protected def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + df.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan.requiredSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings") + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala new file mode 100644 index 0000000000000..a66b34fe367fa --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/binaryfile/BinaryFileFormatSuite.scala @@ -0,0 +1,383 @@ +/* + * 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.binaryfile + +import java.io.{File, IOException} +import java.nio.file.{Files, StandardOpenOption} +import java.sql.Timestamp + +import scala.collection.JavaConverters._ + +import com.google.common.io.{ByteStreams, Closeables} +import org.apache.hadoop.fs.{FileStatus, FileSystem, GlobFilter, Path} +import org.mockito.Mockito.{mock, when} + +import org.apache.spark.SparkException +import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.catalyst.encoders.RowEncoder +import org.apache.spark.sql.execution.datasources.PartitionedFile +import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf.SOURCES_BINARY_FILE_MAX_LENGTH +import org.apache.spark.sql.sources._ +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils + +class BinaryFileFormatSuite extends QueryTest with SharedSQLContext with SQLTestUtils { + import BinaryFileFormat._ + + private var testDir: String = _ + + private var fsTestDir: Path = _ + + private var fs: FileSystem = _ + + private var file1Status: FileStatus = _ + + override def beforeAll(): Unit = { + super.beforeAll() + + testDir = Utils.createTempDir().getAbsolutePath + fsTestDir = new Path(testDir) + fs = fsTestDir.getFileSystem(sparkContext.hadoopConfiguration) + + val year2014Dir = new File(testDir, "year=2014") + year2014Dir.mkdir() + val year2015Dir = new File(testDir, "year=2015") + year2015Dir.mkdir() + + val file1 = new File(year2014Dir, "data.txt") + Files.write( + file1.toPath, + Seq("2014-test").asJava, + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + file1Status = fs.getFileStatus(new Path(file1.getPath)) + + val file2 = new File(year2014Dir, "data2.bin") + Files.write( + file2.toPath, + "2014-test-bin".getBytes, + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + + val file3 = new File(year2015Dir, "bool.csv") + Files.write( + file3.toPath, + Seq("bool", "True", "False", "true").asJava, + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + + val file4 = new File(year2015Dir, "data.bin") + Files.write( + file4.toPath, + "2015-test".getBytes, + StandardOpenOption.CREATE, StandardOpenOption.WRITE + ) + } + + test("BinaryFileFormat methods") { + val format = new BinaryFileFormat + assert(format.shortName() === "binaryFile") + assert(format.isSplitable(spark, Map.empty, new Path("any")) === false) + assert(format.inferSchema(spark, Map.empty, Seq.empty) === Some(BinaryFileFormat.schema)) + assert(BinaryFileFormat.schema === StructType(Seq( + StructField("path", StringType, false), + StructField("modificationTime", TimestampType, false), + StructField("length", LongType, false), + StructField("content", BinaryType, true)))) + } + + def testBinaryFileDataSource(pathGlobFilter: String): Unit = { + val dfReader = spark.read.format(BINARY_FILE) + if (pathGlobFilter != null) { + dfReader.option("pathGlobFilter", pathGlobFilter) + } + val resultDF = dfReader.load(testDir).select( + col(PATH), + col(MODIFICATION_TIME), + col(LENGTH), + col(CONTENT), + col("year") // this is a partition column + ) + + val expectedRowSet = new collection.mutable.HashSet[Row]() + + val globFilter = if (pathGlobFilter == null) null else new GlobFilter(pathGlobFilter) + for (partitionDirStatus <- fs.listStatus(fsTestDir)) { + val dirPath = partitionDirStatus.getPath + + val partitionName = dirPath.getName.split("=")(1) + val year = partitionName.toInt // partition column "year" value which is `Int` type + + for (fileStatus <- fs.listStatus(dirPath)) { + if (globFilter == null || globFilter.accept(fileStatus.getPath)) { + val fpath = fileStatus.getPath.toString + val flen = fileStatus.getLen + val modificationTime = new Timestamp(fileStatus.getModificationTime) + + val fcontent = { + val stream = fs.open(fileStatus.getPath) + val content = try { + ByteStreams.toByteArray(stream) + } finally { + Closeables.close(stream, true) + } + content + } + + val row = Row(fpath, modificationTime, flen, fcontent, year) + expectedRowSet.add(row) + } + } + } + + checkAnswer(resultDF, expectedRowSet.toSeq) + } + + test("binary file data source test") { + testBinaryFileDataSource(null) + testBinaryFileDataSource("*.*") + testBinaryFileDataSource("*.bin") + testBinaryFileDataSource("*.txt") + testBinaryFileDataSource("*.{txt,csv}") + testBinaryFileDataSource("*.json") + } + + test("binary file data source do not support write operation") { + val df = spark.read.format(BINARY_FILE).load(testDir) + withTempDir { tmpDir => + val thrown = intercept[UnsupportedOperationException] { + df.write + .format(BINARY_FILE) + .save(tmpDir + "/test_save") + } + assert(thrown.getMessage.contains("Write is not supported for binary file data source")) + } + } + + def mockFileStatus(length: Long, modificationTime: Long): FileStatus = { + val status = mock(classOf[FileStatus]) + when(status.getLen).thenReturn(length) + when(status.getModificationTime).thenReturn(modificationTime) + when(status.toString).thenReturn( + s"FileStatus($LENGTH=$length, $MODIFICATION_TIME=$modificationTime)") + status + } + + def testCreateFilterFunction( + filters: Seq[Filter], + testCases: Seq[(FileStatus, Boolean)]): Unit = { + val funcs = filters.map(BinaryFileFormat.createFilterFunction) + testCases.foreach { case (status, expected) => + assert(funcs.forall(f => f(status)) === expected, + s"$filters applied to $status should be $expected.") + } + } + + test("createFilterFunction") { + // test filter applied on `length` column + val l1 = mockFileStatus(1L, 0L) + val l2 = mockFileStatus(2L, 0L) + val l3 = mockFileStatus(3L, 0L) + testCreateFilterFunction( + Seq(LessThan(LENGTH, 2L)), + Seq((l1, true), (l2, false), (l3, false))) + testCreateFilterFunction( + Seq(LessThanOrEqual(LENGTH, 2L)), + Seq((l1, true), (l2, true), (l3, false))) + testCreateFilterFunction( + Seq(GreaterThan(LENGTH, 2L)), + Seq((l1, false), (l2, false), (l3, true))) + testCreateFilterFunction( + Seq(GreaterThanOrEqual(LENGTH, 2L)), + Seq((l1, false), (l2, true), (l3, true))) + testCreateFilterFunction( + Seq(EqualTo(LENGTH, 2L)), + Seq((l1, false), (l2, true), (l3, false))) + testCreateFilterFunction( + Seq(Not(EqualTo(LENGTH, 2L))), + Seq((l1, true), (l2, false), (l3, true))) + testCreateFilterFunction( + Seq(And(GreaterThan(LENGTH, 1L), LessThan(LENGTH, 3L))), + Seq((l1, false), (l2, true), (l3, false))) + testCreateFilterFunction( + Seq(Or(LessThanOrEqual(LENGTH, 1L), GreaterThanOrEqual(LENGTH, 3L))), + Seq((l1, true), (l2, false), (l3, true))) + + // test filter applied on `modificationTime` column + val t1 = mockFileStatus(0L, 1L) + val t2 = mockFileStatus(0L, 2L) + val t3 = mockFileStatus(0L, 3L) + testCreateFilterFunction( + Seq(LessThan(MODIFICATION_TIME, new Timestamp(2L))), + Seq((t1, true), (t2, false), (t3, false))) + testCreateFilterFunction( + Seq(LessThanOrEqual(MODIFICATION_TIME, new Timestamp(2L))), + Seq((t1, true), (t2, true), (t3, false))) + testCreateFilterFunction( + Seq(GreaterThan(MODIFICATION_TIME, new Timestamp(2L))), + Seq((t1, false), (t2, false), (t3, true))) + testCreateFilterFunction( + Seq(GreaterThanOrEqual(MODIFICATION_TIME, new Timestamp(2L))), + Seq((t1, false), (t2, true), (t3, true))) + testCreateFilterFunction( + Seq(EqualTo(MODIFICATION_TIME, new Timestamp(2L))), + Seq((t1, false), (t2, true), (t3, false))) + testCreateFilterFunction( + Seq(Not(EqualTo(MODIFICATION_TIME, new Timestamp(2L)))), + Seq((t1, true), (t2, false), (t3, true))) + testCreateFilterFunction( + Seq(And(GreaterThan(MODIFICATION_TIME, new Timestamp(1L)), + LessThan(MODIFICATION_TIME, new Timestamp(3L)))), + Seq((t1, false), (t2, true), (t3, false))) + testCreateFilterFunction( + Seq(Or(LessThanOrEqual(MODIFICATION_TIME, new Timestamp(1L)), + GreaterThanOrEqual(MODIFICATION_TIME, new Timestamp(3L)))), + Seq((t1, true), (t2, false), (t3, true))) + + // test filters applied on both columns + testCreateFilterFunction( + Seq(And(GreaterThan(LENGTH, 2L), LessThan(MODIFICATION_TIME, new Timestamp(2L)))), + Seq((l1, false), (l2, false), (l3, true), (t1, false), (t2, false), (t3, false))) + + // test nested filters + testCreateFilterFunction( + // NOT (length > 2 OR modificationTime < 2) + Seq(Not(Or(GreaterThan(LENGTH, 2L), LessThan(MODIFICATION_TIME, new Timestamp(2L))))), + Seq((l1, false), (l2, false), (l3, false), (t1, false), (t2, true), (t3, true))) + } + + test("buildReader") { + def testBuildReader(fileStatus: FileStatus, filters: Seq[Filter], expected: Boolean): Unit = { + val format = new BinaryFileFormat + val reader = format.buildReaderWithPartitionValues( + sparkSession = spark, + dataSchema = schema, + partitionSchema = StructType(Nil), + requiredSchema = schema, + filters = filters, + options = Map.empty, + hadoopConf = spark.sessionState.newHadoopConf()) + val partitionedFile = mock(classOf[PartitionedFile]) + when(partitionedFile.filePath).thenReturn(fileStatus.getPath.toString) + assert(reader(partitionedFile).nonEmpty === expected, + s"Filters $filters applied to $fileStatus should be $expected.") + } + testBuildReader(file1Status, Seq.empty, true) + testBuildReader(file1Status, Seq(LessThan(LENGTH, file1Status.getLen)), false) + testBuildReader(file1Status, Seq( + LessThan(MODIFICATION_TIME, new Timestamp(file1Status.getModificationTime)) + ), false) + testBuildReader(file1Status, Seq( + EqualTo(LENGTH, file1Status.getLen), + EqualTo(MODIFICATION_TIME, file1Status.getModificationTime) + ), true) + } + + private def readBinaryFile(file: File, requiredSchema: StructType): Row = { + val format = new BinaryFileFormat + val reader = format.buildReaderWithPartitionValues( + sparkSession = spark, + dataSchema = schema, + partitionSchema = StructType(Nil), + requiredSchema = requiredSchema, + filters = Seq.empty, + options = Map.empty, + hadoopConf = spark.sessionState.newHadoopConf() + ) + val partitionedFile = mock(classOf[PartitionedFile]) + when(partitionedFile.filePath).thenReturn(file.getPath) + val encoder = RowEncoder(requiredSchema).resolveAndBind() + encoder.fromRow(reader(partitionedFile).next()) + } + + test("column pruning") { + withTempPath { file => + val content = "123".getBytes + Files.write(file.toPath, content, StandardOpenOption.CREATE, StandardOpenOption.WRITE) + + val actual = readBinaryFile(file, StructType(schema.takeRight(3))) + val expected = Row(new Timestamp(file.lastModified()), content.length, content) + + assert(actual === expected) + } + } + + test("column pruning - non-readable file") { + withTempPath { file => + val content = "abc".getBytes + Files.write(file.toPath, content, StandardOpenOption.CREATE, StandardOpenOption.WRITE) + file.setReadable(false) + + // If content is selected, it throws an exception because it's not readable. + intercept[IOException] { + readBinaryFile(file, StructType(schema(CONTENT) :: Nil)) + } + + // Otherwise, it should be able to read. + assert( + readBinaryFile(file, StructType(schema(LENGTH) :: Nil)) === Row(content.length), + "Get length should not read content.") + assert( + spark.read.format(BINARY_FILE).load(file.getPath).count() === 1, + "Count should not read content.") + } + } + + test("fail fast and do not attempt to read if a file is too big") { + assert(spark.conf.get(SOURCES_BINARY_FILE_MAX_LENGTH) === Int.MaxValue) + withTempPath { file => + val path = file.getPath + val content = "123".getBytes + Files.write(file.toPath, content, StandardOpenOption.CREATE, StandardOpenOption.WRITE) + def readContent(): DataFrame = { + spark.read.format(BINARY_FILE) + .load(path) + .select(CONTENT) + } + val expected = Seq(Row(content)) + QueryTest.checkAnswer(readContent(), expected) + withSQLConf(SOURCES_BINARY_FILE_MAX_LENGTH.key -> content.length.toString) { + QueryTest.checkAnswer(readContent(), expected) + } + // Disable read. If the implementation attempts to read, the exception would be different. + file.setReadable(false) + val caught = intercept[SparkException] { + withSQLConf(SOURCES_BINARY_FILE_MAX_LENGTH.key -> (content.length - 1).toString) { + QueryTest.checkAnswer(readContent(), expected) + } + } + assert(caught.getMessage.contains("exceeds the max length allowed")) + } + } + + test("SPARK-28030: support chars in file names that require URL encoding") { + withTempDir { dir => + val file = new File(dir, "test space.txt") + val content = "123".getBytes + Files.write(file.toPath, content, StandardOpenOption.CREATE, StandardOpenOption.WRITE) + val df = spark.read.format(BINARY_FILE).load(dir.getPath) + df.select(col(PATH), col(CONTENT)).first() match { + case Row(p: String, c: Array[Byte]) => + assert(p.endsWith(file.getAbsolutePath), "should support space in file name") + assert(c === content, "should read file with space in file name") + } + } + } +} 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 new file mode 100644 index 0000000000000..e41e81af508f1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmark.scala @@ -0,0 +1,307 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.spark.sql.execution.datasources.csv + +import java.io.File +import java.time.{Instant, LocalDate} + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.{Column, Dataset, Row} +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * Benchmark to measure CSV read/write performance. + * 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/CSVBenchmark-results.txt". + * }}} + */ + +object CSVBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + + private def toNoop(ds: Dataset[_]): Unit = ds.write.format("noop").save() + + private def quotedValuesBenchmark(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark(s"Parsing quoted values", rowsNum, output = output) + + withTempPath { path => + val str = (0 until 10000).map(i => s""""$i"""").mkString(",") + + spark.range(rowsNum) + .map(_ => str) + .write.option("header", true) + .csv(path.getAbsolutePath) + + val schema = new StructType().add("value", StringType) + val ds = spark.read.option("header", true).schema(schema).csv(path.getAbsolutePath) + + benchmark.addCase(s"One quoted string", numIters) { _ => + toNoop(ds) + } + + benchmark.run() + } + } + + private def multiColumnsBenchmark(rowsNum: Int, numIters: Int): Unit = { + val colsNum = 1000 + val benchmark = new Benchmark(s"Wide rows with $colsNum columns", rowsNum, output = output) + + withTempPath { path => + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) + val schema = StructType(fields) + val values = (0 until colsNum).map(i => i.toString).mkString(",") + val columnNames = schema.fieldNames + + spark.range(rowsNum) + .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) + .write.option("header", true) + .csv(path.getAbsolutePath) + + val ds = spark.read.schema(schema).csv(path.getAbsolutePath) + + benchmark.addCase(s"Select $colsNum columns", numIters) { _ => + toNoop(ds.select("*")) + } + val cols100 = columnNames.take(100).map(Column(_)) + benchmark.addCase(s"Select 100 columns", numIters) { _ => + toNoop(ds.select(cols100: _*)) + } + benchmark.addCase(s"Select one column", numIters) { _ => + toNoop(ds.select($"col1")) + } + benchmark.addCase(s"count()", numIters) { _ => + ds.count() + } + + val schemaErr1 = StructType(StructField("col0", DateType) +: + (1 until colsNum).map(i => StructField(s"col$i", IntegerType))) + val dsErr1 = spark.read.schema(schemaErr1).csv(path.getAbsolutePath) + benchmark.addCase(s"Select 100 columns, one bad input field", numIters) { _ => + toNoop(dsErr1.select(cols100: _*)) + } + + val badRecColName = "badRecord" + val schemaErr2 = schemaErr1.add(StructField(badRecColName, StringType)) + val dsErr2 = spark.read.schema(schemaErr2) + .option("columnNameOfCorruptRecord", badRecColName) + .csv(path.getAbsolutePath) + benchmark.addCase(s"Select 100 columns, corrupt record field", numIters) { _ => + toNoop(dsErr2.select((Column(badRecColName) +: cols100): _*)) + } + + benchmark.run() + } + } + + private def countBenchmark(rowsNum: Int, numIters: Int): Unit = { + val colsNum = 10 + val benchmark = + new Benchmark(s"Count a dataset with $colsNum columns", rowsNum, output = output) + + withTempPath { path => + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) + val schema = StructType(fields) + + spark.range(rowsNum) + .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) + .write + .csv(path.getAbsolutePath) + + val ds = spark.read.schema(schema).csv(path.getAbsolutePath) + + benchmark.addCase(s"Select $colsNum columns + count()", numIters) { _ => + ds.select("*").filter((_: Row) => true).count() + } + benchmark.addCase(s"Select 1 column + count()", numIters) { _ => + ds.select($"col1").filter((_: Row) => true).count() + } + benchmark.addCase(s"count()", numIters) { _ => + ds.count() + } + + benchmark.run() + } + } + + private def datetimeBenchmark(rowsNum: Int, numIters: Int): Unit = { + def timestamps = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(Instant.ofEpochSecond(_)) + }.select($"value".as("timestamp")) + } + + def dates = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(d => LocalDate.ofEpochDay(d % (100 * 365))) + }.select($"value".as("date")) + } + + withTempPath { path => + + val timestampDir = new File(path, "timestamp").getAbsolutePath + val dateDir = new File(path, "date").getAbsolutePath + + val writeBench = new Benchmark("Write dates and timestamps", rowsNum, output = output) + writeBench.addCase(s"Create a dataset of timestamps", numIters) { _ => + toNoop(timestamps) + } + + writeBench.addCase("to_csv(timestamp)", numIters) { _ => + toNoop(timestamps.select(to_csv(struct($"timestamp")))) + } + + writeBench.addCase("write timestamps to files", numIters) { _ => + timestamps.write.option("header", true).mode("overwrite").csv(timestampDir) + } + + writeBench.addCase("Create a dataset of dates", numIters) { _ => + toNoop(dates) + } + + writeBench.addCase("to_csv(date)", numIters) { _ => + toNoop(dates.select(to_csv(struct($"date")))) + } + + writeBench.addCase("write dates to files", numIters) { _ => + dates.write.option("header", true).mode("overwrite").csv(dateDir) + } + + writeBench.run() + + val readBench = new Benchmark("Read dates and timestamps", rowsNum, output = output) + val tsSchema = new StructType().add("timestamp", TimestampType) + + readBench.addCase("read timestamp text from files", numIters) { _ => + toNoop(spark.read.text(timestampDir)) + } + + readBench.addCase("read timestamps from files", numIters) { _ => + val ds = spark.read + .option("header", true) + .schema(tsSchema) + .csv(timestampDir) + toNoop(ds) + } + + readBench.addCase("infer timestamps from files", numIters) { _ => + val ds = spark.read + .option("header", true) + .option("inferSchema", true) + .csv(timestampDir) + toNoop(ds) + } + + val dateSchema = new StructType().add("date", DateType) + + readBench.addCase("read date text from files", numIters) { _ => + toNoop(spark.read.text(dateDir)) + } + + readBench.addCase("read date from files", numIters) { _ => + val ds = spark.read + .option("header", true) + .schema(dateSchema) + .csv(dateDir) + toNoop(ds) + } + + readBench.addCase("infer date from files", numIters) { _ => + val ds = spark.read + .option("header", true) + .option("inferSchema", true) + .csv(dateDir) + toNoop(ds) + } + + 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") + }.select($"value".as("timestamp")).as[String] + } + + readBench.addCase("timestamp strings", numIters) { _ => + toNoop(timestampStr) + } + + readBench.addCase("parse timestamps from Dataset[String]", numIters) { _ => + val ds = spark.read + .option("header", false) + .schema(tsSchema) + .csv(timestampStr) + toNoop(ds) + } + + readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ => + val ds = spark.read + .option("header", false) + .option("inferSchema", true) + .csv(timestampStr) + toNoop(ds) + } + + def dateStr: Dataset[String] = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(i => LocalDate.ofEpochDay(i % 1000 * 365).toString) + }.select($"value".as("date")).as[String] + } + + readBench.addCase("date strings", numIters) { _ => + toNoop(dateStr) + } + + readBench.addCase("parse dates from Dataset[String]", numIters) { _ => + val ds = spark.read + .option("header", false) + .schema(dateSchema) + .csv(dateStr) + toNoop(ds) + } + + readBench.addCase("from_csv(timestamp)", numIters) { _ => + val ds = timestampStr.select(from_csv($"timestamp", tsSchema, Map.empty[String, String])) + toNoop(ds) + } + + readBench.addCase("from_csv(date)", numIters) { _ => + val ds = dateStr.select(from_csv($"date", dateSchema, Map.empty[String, String])) + toNoop(ds) + } + + readBench.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Benchmark to measure CSV read/write performance") { + val numIters = 3 + quotedValuesBenchmark(rowsNum = 50 * 1000, numIters) + multiColumnsBenchmark(rowsNum = 1000 * 1000, numIters) + countBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmarks.scala deleted file mode 100644 index 5d1a874999c09..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVBenchmarks.scala +++ /dev/null @@ -1,158 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.datasources.csv - -import org.apache.spark.SparkConf -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.{Column, Row, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.types._ - -/** - * Benchmark to measure CSV read/write performance. - * To run this: - * spark-submit --class --jars - */ -object CSVBenchmarks extends SQLHelper { - val conf = new SparkConf() - - val spark = SparkSession.builder - .master("local[1]") - .appName("benchmark-csv-datasource") - .config(conf) - .getOrCreate() - import spark.implicits._ - - def quotedValuesBenchmark(rowsNum: Int, numIters: Int): Unit = { - val benchmark = new Benchmark(s"Parsing quoted values", rowsNum) - - withTempPath { path => - val str = (0 until 10000).map(i => s""""$i"""").mkString(",") - - spark.range(rowsNum) - .map(_ => str) - .write.option("header", true) - .csv(path.getAbsolutePath) - - val schema = new StructType().add("value", StringType) - val ds = spark.read.option("header", true).schema(schema).csv(path.getAbsolutePath) - - benchmark.addCase(s"One quoted string", numIters) { _ => - ds.filter((_: Row) => true).count() - } - - /* - Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz - - Parsing quoted values: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - One quoted string 30273 / 30549 0.0 605451.2 1.0X - */ - benchmark.run() - } - } - - def multiColumnsBenchmark(rowsNum: Int): Unit = { - val colsNum = 1000 - val benchmark = new Benchmark(s"Wide rows with $colsNum columns", rowsNum) - - withTempPath { path => - val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) - val schema = StructType(fields) - val values = (0 until colsNum).map(i => i.toString).mkString(",") - val columnNames = schema.fieldNames - - spark.range(rowsNum) - .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) - .write.option("header", true) - .csv(path.getAbsolutePath) - - val ds = spark.read.schema(schema).csv(path.getAbsolutePath) - - benchmark.addCase(s"Select $colsNum columns", 3) { _ => - ds.select("*").filter((row: Row) => true).count() - } - val cols100 = columnNames.take(100).map(Column(_)) - benchmark.addCase(s"Select 100 columns", 3) { _ => - ds.select(cols100: _*).filter((row: Row) => true).count() - } - benchmark.addCase(s"Select one column", 3) { _ => - ds.select($"col1").filter((row: Row) => true).count() - } - benchmark.addCase(s"count()", 3) { _ => - ds.count() - } - - /* - Intel(R) Core(TM) i7-7920HQ CPU @ 3.10GHz - - Wide rows with 1000 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - -------------------------------------------------------------------------------------------- - Select 1000 columns 81091 / 81692 0.0 81090.7 1.0X - Select 100 columns 30003 / 34448 0.0 30003.0 2.7X - Select one column 24792 / 24855 0.0 24792.0 3.3X - count() 24344 / 24642 0.0 24343.8 3.3X - */ - benchmark.run() - } - } - - def countBenchmark(rowsNum: Int): Unit = { - val colsNum = 10 - val benchmark = new Benchmark(s"Count a dataset with $colsNum columns", rowsNum) - - withTempPath { path => - val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) - val schema = StructType(fields) - - spark.range(rowsNum) - .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) - .write - .csv(path.getAbsolutePath) - - val ds = spark.read.schema(schema).csv(path.getAbsolutePath) - - benchmark.addCase(s"Select $colsNum columns + count()", 3) { _ => - ds.select("*").filter((_: Row) => true).count() - } - benchmark.addCase(s"Select 1 column + count()", 3) { _ => - ds.select($"col1").filter((_: Row) => true).count() - } - benchmark.addCase(s"count()", 3) { _ => - ds.count() - } - - /* - Intel(R) Core(TM) i7-7700HQ CPU @ 2.80GHz - - Count a dataset with 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - --------------------------------------------------------------------------------------------- - Select 10 columns + count() 12598 / 12740 0.8 1259.8 1.0X - Select 1 column + count() 7960 / 8175 1.3 796.0 1.6X - count() 2332 / 2386 4.3 233.2 5.4X - */ - benchmark.run() - } - } - - def main(args: Array[String]): Unit = { - quotedValuesBenchmark(rowsNum = 50 * 1000, numIters = 3) - multiColumnsBenchmark(rowsNum = 1000 * 1000) - countBenchmark(10 * 1000 * 1000) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala deleted file mode 100644 index 57e36e082653c..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchemaSuite.scala +++ /dev/null @@ -1,142 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.csv - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.types._ - -class CSVInferSchemaSuite extends SparkFunSuite { - - test("String fields types are inferred correctly from null types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(NullType, "", options) == NullType) - assert(CSVInferSchema.inferField(NullType, null, options) == NullType) - assert(CSVInferSchema.inferField(NullType, "100000000000", options) == LongType) - assert(CSVInferSchema.inferField(NullType, "60", options) == IntegerType) - assert(CSVInferSchema.inferField(NullType, "3.5", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "test", options) == StringType) - assert(CSVInferSchema.inferField(NullType, "2015-08-20 15:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(NullType, "True", options) == BooleanType) - assert(CSVInferSchema.inferField(NullType, "FAlSE", options) == BooleanType) - - val textValueOne = Long.MaxValue.toString + "0" - val decimalValueOne = new java.math.BigDecimal(textValueOne) - val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) - assert(CSVInferSchema.inferField(NullType, textValueOne, options) == expectedTypeOne) - } - - test("String fields types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(LongType, "1.0", options) == DoubleType) - assert(CSVInferSchema.inferField(LongType, "test", options) == StringType) - assert(CSVInferSchema.inferField(IntegerType, "1.0", options) == DoubleType) - assert(CSVInferSchema.inferField(DoubleType, null, options) == DoubleType) - assert(CSVInferSchema.inferField(DoubleType, "test", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "2015-08-20 14:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 15:57:00", options) == TimestampType) - assert(CSVInferSchema.inferField(LongType, "True", options) == BooleanType) - assert(CSVInferSchema.inferField(IntegerType, "FALSE", options) == BooleanType) - assert(CSVInferSchema.inferField(TimestampType, "FALSE", options) == BooleanType) - - val textValueOne = Long.MaxValue.toString + "0" - val decimalValueOne = new java.math.BigDecimal(textValueOne) - val expectedTypeOne = DecimalType(decimalValueOne.precision, decimalValueOne.scale) - assert(CSVInferSchema.inferField(IntegerType, textValueOne, options) == expectedTypeOne) - } - - test("Timestamp field types are inferred correctly via custom data format") { - var options = new CSVOptions(Map("timestampFormat" -> "yyyy-mm"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) - options = new CSVOptions(Map("timestampFormat" -> "yyyy"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015", options) == TimestampType) - } - - test("Timestamp field types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(IntegerType, "2015-08-20 14", options) == StringType) - assert(CSVInferSchema.inferField(DoubleType, "2015-08-20 14:10", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "2015-08 14:49:00", options) == StringType) - } - - test("Boolean fields types are inferred correctly from other types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(CSVInferSchema.inferField(LongType, "Fale", options) == StringType) - assert(CSVInferSchema.inferField(DoubleType, "TRUEe", options) == StringType) - } - - test("Type arrays are merged to highest common type") { - assert( - CSVInferSchema.mergeRowTypes(Array(StringType), - Array(DoubleType)).deep == Array(StringType).deep) - assert( - CSVInferSchema.mergeRowTypes(Array(IntegerType), - Array(LongType)).deep == Array(LongType).deep) - assert( - CSVInferSchema.mergeRowTypes(Array(DoubleType), - Array(LongType)).deep == Array(DoubleType).deep) - } - - test("Null fields are handled properly when a nullValue is specified") { - var options = new CSVOptions(Map("nullValue" -> "null"), false, "GMT") - assert(CSVInferSchema.inferField(NullType, "null", options) == NullType) - assert(CSVInferSchema.inferField(StringType, "null", options) == StringType) - assert(CSVInferSchema.inferField(LongType, "null", options) == LongType) - - options = new CSVOptions(Map("nullValue" -> "\\N"), false, "GMT") - assert(CSVInferSchema.inferField(IntegerType, "\\N", options) == IntegerType) - assert(CSVInferSchema.inferField(DoubleType, "\\N", options) == DoubleType) - assert(CSVInferSchema.inferField(TimestampType, "\\N", options) == TimestampType) - assert(CSVInferSchema.inferField(BooleanType, "\\N", options) == BooleanType) - assert(CSVInferSchema.inferField(DecimalType(1, 1), "\\N", options) == DecimalType(1, 1)) - } - - test("Merging Nulltypes should yield Nulltype.") { - val mergedNullTypes = CSVInferSchema.mergeRowTypes(Array(NullType), Array(NullType)) - assert(mergedNullTypes.deep == Array(NullType).deep) - } - - test("SPARK-18433: Improve DataSource option keys to be more case-insensitive") { - val options = new CSVOptions(Map("TiMeStampFormat" -> "yyyy-mm"), false, "GMT") - assert(CSVInferSchema.inferField(TimestampType, "2015-08", options) == TimestampType) - } - - test("SPARK-18877: `inferField` on DecimalType should find a common type with `typeSoFar`") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - - // 9.03E+12 is Decimal(3, -10) and 1.19E+11 is Decimal(3, -9). - assert(CSVInferSchema.inferField(DecimalType(3, -10), "1.19E+11", options) == - DecimalType(4, -9)) - - // BigDecimal("12345678901234567890.01234567890123456789") is precision 40 and scale 20. - val value = "12345678901234567890.01234567890123456789" - assert(CSVInferSchema.inferField(DecimalType(3, -10), value, options) == DoubleType) - - // Seq(s"${Long.MaxValue}1", "2015-12-01 00:00:00") should be StringType - assert(CSVInferSchema.inferField(NullType, s"${Long.MaxValue}1", options) == DecimalType(20, 0)) - assert(CSVInferSchema.inferField(DecimalType(20, 0), "2015-12-01 00:00:00", options) - == StringType) - } - - 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") - assert(CSVInferSchema.inferField(NullType, "nan", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "inf", options) == DoubleType) - assert(CSVInferSchema.inferField(NullType, "-inf", options) == DoubleType) - } -} 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 5d4746cf90b3f..fdb50a6dd929c 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 @@ -17,12 +17,13 @@ package org.apache.spark.sql.execution.datasources.csv -import java.io.File -import java.nio.charset.{Charset, UnsupportedCharsetException} +import java.io.{ByteArrayOutputStream, EOFException, File, FileOutputStream} +import java.nio.charset.{Charset, StandardCharsets, UnsupportedCharsetException} import java.nio.file.Files import java.sql.{Date, Timestamp} import java.text.SimpleDateFormat import java.util.Locale +import java.util.zip.GZIPOutputStream import scala.collection.JavaConverters._ import scala.util.Properties @@ -33,7 +34,7 @@ import org.apache.hadoop.io.compress.GzipCodec import org.apache.log4j.{AppenderSkeleton, LogManager} import org.apache.log4j.spi.LoggingEvent -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.internal.SQLConf @@ -52,6 +53,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val carsNullFile = "test-data/cars-null.csv" private val carsEmptyValueFile = "test-data/cars-empty-value.csv" private val carsBlankColName = "test-data/cars-blank-column-name.csv" + private val carsCrlf = "test-data/cars-crlf.csv" private val emptyFile = "test-data/empty.csv" private val commentsFile = "test-data/comments.csv" private val disableCommentsFile = "test-data/disable_comments.csv" @@ -62,6 +64,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te private val datesFile = "test-data/dates.csv" private val unescapedQuotesFile = "test-data/unescaped-quotes.csv" private val valueMalformedFile = "test-data/value-malformed.csv" + private val badAfterGoodFile = "test-data/bad_after_good.csv" /** Verifies data and schema. */ private def verifyCars( @@ -220,6 +223,17 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } } + test("crlf line separators in multiline mode") { + val cars = spark + .read + .format("csv") + .option("multiLine", "true") + .option("header", "true") + .load(testFile(carsCrlf)) + + verifyCars(cars, withHeader = true) + } + test("test aliases sep and encoding for delimiter and charset") { // scalastyle:off val cars = spark @@ -574,6 +588,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val results = spark.read .format("csv") .options(Map("comment" -> "~", "header" -> "false", "inferSchema" -> "true")) + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") .load(testFile(commentsFile)) .collect() @@ -610,10 +625,11 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val options = Map( "header" -> "true", "inferSchema" -> "false", - "dateFormat" -> "dd/MM/yyyy hh:mm") + "dateFormat" -> "dd/MM/yyyy HH:mm") val results = spark.read .format("csv") .options(options) + .option("timeZone", "UTC") .schema(customSchema) .load(testFile(datesFile)) .select("date") @@ -881,36 +897,38 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } test("Write dates correctly in ISO8601 format by default") { - withTempDir { dir => - val customSchema = new StructType(Array(StructField("date", DateType, true))) - val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" - val dates = spark.read - .format("csv") - .schema(customSchema) - .option("header", "true") - .option("inferSchema", "false") - .option("dateFormat", "dd/MM/yyyy HH:mm") - .load(testFile(datesFile)) - dates.write - .format("csv") - .option("header", "true") - .save(iso8601datesPath) + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> "UTC") { + withTempDir { dir => + val customSchema = new StructType(Array(StructField("date", DateType, true))) + val iso8601datesPath = s"${dir.getCanonicalPath}/iso8601dates.csv" + val dates = spark.read + .format("csv") + .schema(customSchema) + .option("header", "true") + .option("inferSchema", "false") + .option("dateFormat", "dd/MM/yyyy HH:mm") + .load(testFile(datesFile)) + dates.write + .format("csv") + .option("header", "true") + .save(iso8601datesPath) - // This will load back the dates as string. - val stringSchema = StructType(StructField("date", StringType, true) :: Nil) - val iso8601dates = spark.read - .format("csv") - .schema(stringSchema) - .option("header", "true") - .load(iso8601datesPath) + // This will load back the dates as string. + val stringSchema = StructType(StructField("date", StringType, true) :: Nil) + val iso8601dates = spark.read + .format("csv") + .schema(stringSchema) + .option("header", "true") + .load(iso8601datesPath) + + val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) + val expectedDates = dates.collect().map { r => + // This should be ISO8601 formatted string. + Row(iso8501.format(r.toSeq.head)) + } - val iso8501 = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US) - val expectedDates = dates.collect().map { r => - // This should be ISO8601 formatted string. - Row(iso8501.format(r.toSeq.head)) + checkAnswer(iso8601dates, expectedDates) } - - checkAnswer(iso8601dates, expectedDates) } } @@ -1095,7 +1113,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te test("SPARK-18699 put malformed records in a `columnNameOfCorruptRecord` field") { Seq(false, true).foreach { multiLine => - val schema = new StructType().add("a", IntegerType).add("b", TimestampType) + val schema = new StructType().add("a", IntegerType).add("b", DateType) // We use `PERMISSIVE` mode by default if invalid string is given. val df1 = spark .read @@ -1104,7 +1122,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schema) .csv(testFile(valueMalformedFile)) checkAnswer(df1, - Row(null, null) :: + Row(0, null) :: Row(1, java.sql.Date.valueOf("1983-08-04")) :: Nil) @@ -1119,7 +1137,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schemaWithCorrField1) .csv(testFile(valueMalformedFile)) checkAnswer(df2, - Row(null, null, "0,2013-111-11 12:13:14") :: + Row(0, null, "0,2013-111-11 12:13:14") :: Row(1, java.sql.Date.valueOf("1983-08-04"), null) :: Nil) @@ -1127,7 +1145,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val schemaWithCorrField2 = new StructType() .add("a", IntegerType) .add(columnNameOfCorruptRecord, StringType) - .add("b", TimestampType) + .add("b", DateType) val df3 = spark .read .option("mode", "permissive") @@ -1136,7 +1154,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .schema(schemaWithCorrField2) .csv(testFile(valueMalformedFile)) checkAnswer(df3, - Row(null, "0,2013-111-11 12:13:14", null) :: + Row(0, "0,2013-111-11 12:13:14", null) :: Row(1, null, java.sql.Date.valueOf("1983-08-04")) :: Nil) @@ -1154,6 +1172,40 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } } + test("Enabling/disabling ignoreCorruptFiles") { + val inputFile = File.createTempFile("input-", ".gz") + try { + // Create a corrupt gzip file + val byteOutput = new ByteArrayOutputStream() + val gzip = new GZIPOutputStream(byteOutput) + try { + gzip.write(Array[Byte](1, 2, 3, 4)) + } finally { + gzip.close() + } + val bytes = byteOutput.toByteArray + val o = new FileOutputStream(inputFile) + try { + // It's corrupt since we only write half of bytes into the file. + o.write(bytes.take(bytes.length / 2)) + } finally { + o.close() + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val e = intercept[SparkException] { + spark.read.csv(inputFile.toURI.toString).collect() + } + assert(e.getCause.isInstanceOf[EOFException]) + assert(e.getCause.getMessage === "Unexpected end of input stream") + } + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { + assert(spark.read.csv(inputFile.toURI.toString).collect().isEmpty) + } + } finally { + inputFile.delete() + } + } + test("SPARK-19610: Parse normal multi-line CSV files") { val primitiveFieldAndType = Seq( """" @@ -1313,7 +1365,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val columnNameOfCorruptRecord = "_corrupt_record" val schema = new StructType() .add("a", IntegerType) - .add("b", TimestampType) + .add("b", DateType) .add(columnNameOfCorruptRecord, StringType) // negative cases val msg = intercept[AnalysisException] { @@ -1326,15 +1378,7 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .collect() }.getMessage assert(msg.contains("only include the internal corrupt record column")) - intercept[org.apache.spark.sql.catalyst.errors.TreeNodeException[_]] { - spark - .read - .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) - .schema(schema) - .csv(testFile(valueMalformedFile)) - .filter($"_corrupt_record".isNotNull) - .count() - } + // workaround val df = spark .read @@ -1355,8 +1399,8 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) @@ -1680,21 +1724,17 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te } val testAppender1 = new TestAppender - LogManager.getRootLogger.addAppender(testAppender1) - try { + withLogAppender(testAppender1) { val ds = Seq("columnA,columnB", "1.0,1000.0").toDS() val ischema = new StructType().add("columnB", DoubleType).add("columnA", DoubleType) spark.read.schema(ischema).option("header", true).option("enforceSchema", true).csv(ds) - } finally { - LogManager.getRootLogger.removeAppender(testAppender1) } assert(testAppender1.events.asScala .exists(msg => msg.getRenderedMessage.contains("CSV header does not conform to the schema"))) val testAppender2 = new TestAppender - LogManager.getRootLogger.addAppender(testAppender2) - try { + withLogAppender(testAppender2) { withTempPath { path => val oschema = new StructType().add("f1", DoubleType).add("f2", DoubleType) val odf = spark.createDataFrame(List(Row(1.0, 1234.5)).asJava, oschema) @@ -1707,8 +1747,6 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te .csv(path.getCanonicalPath) .collect() } - } finally { - LogManager.getRootLogger.removeAppender(testAppender2) } assert(testAppender2.events.asScala .exists(msg => msg.getRenderedMessage.contains("CSV header does not conform to the schema"))) @@ -1826,4 +1864,225 @@ class CSVSuite extends QueryTest with SharedSQLContext with SQLTestUtils with Te val df = spark.read.option("enforceSchema", false).csv(input) checkAnswer(df, Row("1", "2")) } + + test("using the backward slash as the delimiter") { + val input = Seq("""abc\1""").toDS() + val delimiter = """\\""" + checkAnswer(spark.read.option("delimiter", delimiter).csv(input), Row("abc", "1")) + checkAnswer(spark.read.option("inferSchema", true).option("delimiter", delimiter).csv(input), + Row("abc", 1)) + val schema = new StructType().add("a", StringType).add("b", IntegerType) + checkAnswer(spark.read.schema(schema).option("delimiter", delimiter).csv(input), Row("abc", 1)) + } + + test("using spark.sql.columnNameOfCorruptRecord") { + withSQLConf(SQLConf.COLUMN_NAME_OF_CORRUPT_RECORD.key -> "_unparsed") { + val csv = "\"" + val df = spark.read + .schema("a int, _unparsed string") + .csv(Seq(csv).toDS()) + + checkAnswer(df, Row(null, csv)) + } + } + + test("encoding in multiLine mode") { + val df = spark.range(3).toDF() + Seq("UTF-8", "ISO-8859-1", "CP1251", "US-ASCII", "UTF-16BE", "UTF-32LE").foreach { encoding => + Seq(true, false).foreach { header => + withTempPath { path => + df.write + .option("encoding", encoding) + .option("header", header) + .csv(path.getCanonicalPath) + val readback = spark.read + .option("multiLine", true) + .option("encoding", encoding) + .option("inferSchema", true) + .option("header", header) + .csv(path.getCanonicalPath) + checkAnswer(readback, df) + } + } + } + } + + test("""Support line separator - default value \r, \r\n and \n""") { + val data = "\"a\",1\r\"c\",2\r\n\"d\",3\n" + + withTempPath { path => + Files.write(path.toPath, data.getBytes(StandardCharsets.UTF_8)) + val df = spark.read.option("inferSchema", true).csv(path.getAbsolutePath) + val expectedSchema = + StructType(StructField("_c0", StringType) :: StructField("_c1", IntegerType) :: Nil) + checkAnswer(df, Seq(("a", 1), ("c", 2), ("d", 3)).toDF()) + assert(df.schema === expectedSchema) + } + } + + def testLineSeparator(lineSep: String, encoding: String, inferSchema: Boolean, id: Int): Unit = { + test(s"Support line separator in ${encoding} #${id}") { + // Read + val data = + s""""a",1$lineSep + |c,2$lineSep" + |d",3""".stripMargin + val dataWithTrailingLineSep = s"$data$lineSep" + + Seq(data, dataWithTrailingLineSep).foreach { lines => + withTempPath { path => + Files.write(path.toPath, lines.getBytes(encoding)) + val schema = StructType(StructField("_c0", StringType) + :: StructField("_c1", LongType) :: Nil) + + val expected = Seq(("a", 1), ("\nc", 2), ("\nd", 3)) + .toDF("_c0", "_c1") + Seq(false, true).foreach { multiLine => + val reader = spark + .read + .option("lineSep", lineSep) + .option("multiLine", multiLine) + .option("encoding", encoding) + val df = if (inferSchema) { + reader.option("inferSchema", true).csv(path.getAbsolutePath) + } else { + reader.schema(schema).csv(path.getAbsolutePath) + } + checkAnswer(df, expected) + } + } + } + + // Write + withTempPath { path => + Seq("a", "b", "c").toDF("value").coalesce(1) + .write + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + val partFile = TestUtils.recursiveList(path).filter(f => f.getName.startsWith("part-")).head + val readBack = new String(Files.readAllBytes(partFile.toPath), encoding) + assert( + readBack === s"a${lineSep}b${lineSep}c${lineSep}") + } + + // Roundtrip + withTempPath { path => + val df = Seq("a", "b", "c").toDF() + df.write + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + val readBack = spark + .read + .option("lineSep", lineSep) + .option("encoding", encoding) + .csv(path.getAbsolutePath) + checkAnswer(df, readBack) + } + } + } + + // scalastyle:off nonascii + List( + (0, "|", "UTF-8", false), + (1, "^", "UTF-16BE", true), + (2, ":", "ISO-8859-1", true), + (3, "!", "UTF-32LE", false), + (4, 0x1E.toChar.toString, "UTF-8", true), + (5, "아", "UTF-32BE", false), + (6, "у", "CP1251", true), + (8, "\r", "UTF-16LE", true), + (9, "\u000d", "UTF-32BE", false), + (10, "=", "US-ASCII", false), + (11, "$", "utf-32le", true) + ).foreach { case (testNum, sep, encoding, inferSchema) => + testLineSeparator(sep, encoding, inferSchema, testNum) + } + // scalastyle:on nonascii + + test("lineSep restrictions") { + val errMsg1 = intercept[IllegalArgumentException] { + spark.read.option("lineSep", "").csv(testFile(carsFile)).collect + }.getMessage + assert(errMsg1.contains("'lineSep' cannot be an empty string")) + + val errMsg2 = intercept[IllegalArgumentException] { + spark.read.option("lineSep", "123").csv(testFile(carsFile)).collect + }.getMessage + assert(errMsg2.contains("'lineSep' can contain only 1 character")) + } + + test("SPARK-26208: write and read empty data to csv file with headers") { + withTempPath { path => + val df1 = spark.range(10).repartition(2).filter(_ < 0).map(_.toString).toDF + // we have 2 partitions but they are both empty and will be filtered out upon writing + // thanks to SPARK-23271 one new empty partition will be inserted + df1.write.format("csv").option("header", true).save(path.getAbsolutePath) + val df2 = spark.read.format("csv").option("header", true).option("inferSchema", false) + .load(path.getAbsolutePath) + assert(df1.schema === df2.schema) + checkAnswer(df1, df2) + } + } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.csv(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("csv"))) + } + } + + test("Do not reuse last good value for bad input field") { + val schema = StructType( + StructField("col1", StringType) :: + StructField("col2", DateType) :: + Nil + ) + val rows = spark.read + .schema(schema) + .format("csv") + .load(testFile(badAfterGoodFile)) + + val expectedRows = Seq( + Row("good record", java.sql.Date.valueOf("1999-08-01")), + Row("bad record", null)) + + checkAnswer(rows, expectedRows) + } + + test("SPARK-27512: Decimal type inference should not handle ',' for backward compatibility") { + assert(spark.read + .option("delimiter", "|") + .option("inferSchema", "true") + .csv(Seq("1,2").toDS).schema.head.dataType === StringType) + } + + test("SPARK-27873: disabling enforceSchema should not fail columnNameOfCorruptRecord") { + Seq("csv", "").foreach { reader => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> reader) { + withTempPath { path => + val df = Seq(("0", "2013-111-11")).toDF("a", "b") + df.write + .option("header", "true") + .csv(path.getAbsolutePath) + + val schema = StructType.fromDDL("a int, b date") + val columnNameOfCorruptRecord = "_unparsed" + val schemaWithCorrField = schema.add(columnNameOfCorruptRecord, StringType) + val readDF = spark + .read + .option("mode", "Permissive") + .option("header", "true") + .option("enforceSchema", false) + .option("columnNameOfCorruptRecord", columnNameOfCorruptRecord) + .schema(schemaWithCorrField) + .csv(path.getAbsoluteFile.toString) + checkAnswer(readDF, Row(0, null, "0,2013-111-11") :: Nil) + } + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/TestCsvData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/TestCsvData.scala index 3e20cc47dca2c..79993313a86ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/TestCsvData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/TestCsvData.scala @@ -24,8 +24,7 @@ private[csv] trait TestCsvData { def sampledTestData: Dataset[String] = { spark.range(0, 100, 1).map { index => - val predefinedSample = Set[Long](2, 8, 15, 27, 30, 34, 35, 37, 44, 46, - 57, 62, 68, 72) + val predefinedSample = Set[Long](3, 18, 20, 24, 50, 60, 87, 99) if (predefinedSample.contains(index)) { index.toString } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala deleted file mode 100644 index 458edb253fb33..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/csv/UnivocityParserSuite.scala +++ /dev/null @@ -1,200 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.datasources.csv - -import java.math.BigDecimal -import java.util.Locale - -import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.catalyst.util.DateTimeUtils -import org.apache.spark.sql.types._ -import org.apache.spark.unsafe.types.UTF8String - -class UnivocityParserSuite extends SparkFunSuite { - private val parser = new UnivocityParser( - StructType(Seq.empty), - new CSVOptions(Map.empty[String, String], false, "GMT")) - - private def assertNull(v: Any) = assert(v == null) - - test("Can parse decimal type values") { - val stringValues = Seq("10.05", "1,000.01", "158,058,049.001") - val decimalValues = Seq(10.05, 1000.01, 158058049.001) - val decimalType = new DecimalType() - - stringValues.zip(decimalValues).foreach { case (strVal, decimalVal) => - val decimalValue = new BigDecimal(decimalVal.toString) - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(parser.makeConverter("_1", decimalType, options = options).apply(strVal) === - Decimal(decimalValue, decimalType.precision, decimalType.scale)) - } - } - - test("Nullable types are handled") { - val types = Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType, - BooleanType, DecimalType.DoubleDecimal, TimestampType, DateType, StringType) - - // Nullable field with nullValue option. - types.foreach { t => - // Tests that a custom nullValue. - val nullValueOptions = new CSVOptions(Map("nullValue" -> "-"), false, "GMT") - val converter = - parser.makeConverter("_1", t, nullable = true, options = nullValueOptions) - 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") - assertNull(parser.makeConverter("_1", t, nullable = true, options = options).apply("")) - } - - // 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 converter = - parser.makeConverter("_1", t, nullable = false, options = options) - var message = intercept[RuntimeException] { - converter.apply("-") - }.getMessage - assert(message.contains("null value found but field _1 is not nullable.")) - message = intercept[RuntimeException] { - converter.apply(null) - }.getMessage - assert(message.contains("null value found but field _1 is not nullable.")) - } - - // 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 converter = - parser.makeConverter("_1", StringType, nullable = b, options = options) - assert(converter.apply("") == UTF8String.fromString("")) - } - } - - test("Throws exception for empty string with non null type") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - val exception = intercept[RuntimeException]{ - parser.makeConverter("_1", IntegerType, nullable = false, options = options).apply("") - } - assert(exception.getMessage.contains("null value found but field _1 is not nullable.")) - } - - test("Types are cast correctly") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - assert(parser.makeConverter("_1", ByteType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", ShortType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", IntegerType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", LongType, options = options).apply("10") == 10) - assert(parser.makeConverter("_1", FloatType, options = options).apply("1.00") == 1.0) - assert(parser.makeConverter("_1", DoubleType, options = options).apply("1.00") == 1.0) - assert(parser.makeConverter("_1", BooleanType, options = options).apply("true") == true) - - val timestampsOptions = - new CSVOptions(Map("timestampFormat" -> "dd/MM/yyyy hh:mm"), false, "GMT") - val customTimestamp = "31/01/2015 00:00" - val expectedTime = timestampsOptions.timestampFormat.parse(customTimestamp).getTime - val castedTimestamp = - parser.makeConverter("_1", TimestampType, nullable = true, options = timestampsOptions) - .apply(customTimestamp) - assert(castedTimestamp == expectedTime * 1000L) - - val customDate = "31/01/2015" - val dateOptions = new CSVOptions(Map("dateFormat" -> "dd/MM/yyyy"), false, "GMT") - val expectedDate = dateOptions.dateFormat.parse(customDate).getTime - val castedDate = - parser.makeConverter("_1", DateType, nullable = true, options = dateOptions) - .apply(customTimestamp) - assert(castedDate == DateTimeUtils.millisToDays(expectedDate)) - - val timestamp = "2015-01-01 00:00:00" - assert(parser.makeConverter("_1", TimestampType, options = options).apply(timestamp) == - DateTimeUtils.stringToTime(timestamp).getTime * 1000L) - assert(parser.makeConverter("_1", DateType, options = options).apply("2015-01-01") == - DateTimeUtils.millisToDays(DateTimeUtils.stringToTime("2015-01-01").getTime)) - } - - test("Throws exception for casting an invalid string to Float and Double Types") { - val options = new CSVOptions(Map.empty[String, String], false, "GMT") - val types = Seq(DoubleType, FloatType) - val input = Seq("10u000", "abc", "1 2/3") - types.foreach { dt => - input.foreach { v => - val message = intercept[NumberFormatException] { - parser.makeConverter("_1", dt, options = options).apply(v) - }.getMessage - assert(message.contains(v)) - } - } - } - - test("Float NaN values are parsed correctly") { - val options = new CSVOptions(Map("nanValue" -> "nn"), false, "GMT") - val floatVal: Float = parser.makeConverter( - "_1", FloatType, nullable = true, options = options - ).apply("nn").asInstanceOf[Float] - - // Java implements the IEEE-754 floating point standard which guarantees that any comparison - // against NaN will return false (except != which returns true) - assert(floatVal != floatVal) - } - - test("Double NaN values are parsed correctly") { - val options = new CSVOptions(Map("nanValue" -> "-"), false, "GMT") - val doubleVal: Double = parser.makeConverter( - "_1", DoubleType, nullable = true, options = options - ).apply("-").asInstanceOf[Double] - - assert(doubleVal.isNaN) - } - - test("Float infinite values can be parsed") { - val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") - val floatVal1 = parser.makeConverter( - "_1", FloatType, nullable = true, options = negativeInfOptions - ).apply("max").asInstanceOf[Float] - - assert(floatVal1 == Float.NegativeInfinity) - - val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") - val floatVal2 = parser.makeConverter( - "_1", FloatType, nullable = true, options = positiveInfOptions - ).apply("max").asInstanceOf[Float] - - assert(floatVal2 == Float.PositiveInfinity) - } - - test("Double infinite values can be parsed") { - val negativeInfOptions = new CSVOptions(Map("negativeInf" -> "max"), false, "GMT") - val doubleVal1 = parser.makeConverter( - "_1", DoubleType, nullable = true, options = negativeInfOptions - ).apply("max").asInstanceOf[Double] - - assert(doubleVal1 == Double.NegativeInfinity) - - val positiveInfOptions = new CSVOptions(Map("positiveInf" -> "max"), false, "GMT") - val doubleVal2 = parser.makeConverter( - "_1", DoubleType, nullable = true, options = positiveInfOptions - ).apply("max").asInstanceOf[Double] - - assert(doubleVal2 == Double.PositiveInfinity) - } - -} 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 new file mode 100644 index 0000000000000..f486e603e2552 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmark.scala @@ -0,0 +1,515 @@ +/* + * 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.json + +import java.io.File +import java.time.{Instant, LocalDate} + +import org.apache.spark.benchmark.Benchmark +import org.apache.spark.sql.{Dataset, Row} +import org.apache.spark.sql.execution.benchmark.SqlBasedBenchmark +import org.apache.spark.sql.functions._ +import org.apache.spark.sql.types._ + +/** + * The benchmarks aims to measure performance of JSON parsing when encoding is set and isn't. + * 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/JSONBenchmark-results.txt". + * }}} + */ + +object JSONBenchmark extends SqlBasedBenchmark { + import spark.implicits._ + + private def prepareDataInfo(benchmark: Benchmark): Unit = { + // scalastyle:off println + benchmark.out.println("Preparing data for benchmarking ...") + // scalastyle:on println + } + + private def run(ds: Dataset[_]): Unit = { + ds.write.format("noop").save() + } + + def schemaInferring(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("JSON schema inferring", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + + spark.sparkContext.range(0, rowsNum, 1) + .map(_ => "a") + .toDF("fieldA") + .write + .option("encoding", "UTF-8") + .json(path.getAbsolutePath) + + benchmark.addCase("No encoding", numIters) { _ => + spark.read + .option("inferTimestamp", false) + .json(path.getAbsolutePath) + } + + benchmark.addCase("UTF-8 is set", numIters) { _ => + spark.read + .option("encoding", "UTF-8") + .option("inferTimestamp", false) + .json(path.getAbsolutePath) + } + + benchmark.run() + } + } + + def writeShortColumn(path: String, rowsNum: Int): StructType = { + spark.sparkContext.range(0, rowsNum, 1) + .map(_ => "a") + .toDF("fieldA") + .write.json(path) + new StructType().add("fieldA", StringType) + } + + def countShortColumn(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("count a short column", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + val schema = writeShortColumn(path.getAbsolutePath, rowsNum) + + benchmark.addCase("No encoding", numIters) { _ => + spark.read + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.addCase("UTF-8 is set", numIters) { _ => + spark.read + .option("encoding", "UTF-8") + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.run() + } + } + + def writeWideColumn(path: String, rowsNum: Int): StructType = { + spark.sparkContext.range(0, rowsNum, 1) + .map { i => + val s = "abcdef0123456789ABCDEF" * 20 + s"""{"a":"$s","b": $i,"c":"$s","d":$i,"e":"$s","f":$i,"x":"$s","y":$i,"z":"$s"}""" + } + .toDF().write.text(path) + new StructType() + .add("a", StringType).add("b", LongType) + .add("c", StringType).add("d", LongType) + .add("e", StringType).add("f", LongType) + .add("x", StringType).add("y", LongType) + .add("z", StringType) + } + + def writeWideRow(path: String, rowsNum: Int): StructType = { + val colsNum = 1000 + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) + val schema = StructType(fields) + + spark.range(rowsNum) + .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) + .write.json(path) + + schema + } + + def countWideColumn(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("count a wide column", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + val schema = writeWideColumn(path.getAbsolutePath, rowsNum) + + benchmark.addCase("No encoding", numIters) { _ => + spark.read + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.addCase("UTF-8 is set", numIters) { _ => + spark.read + .option("encoding", "UTF-8") + .schema(schema) + .json(path.getAbsolutePath) + .count() + } + + benchmark.run() + } + } + + def countWideRow(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("select wide row", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + val schema = writeWideRow(path.getAbsolutePath, rowsNum) + + benchmark.addCase("No encoding", numIters) { _ => + spark.read + .schema(schema) + .json(path.getAbsolutePath) + .select("*") + .filter((row: Row) => true) + .count() + } + + benchmark.addCase("UTF-8 is set", numIters) { _ => + spark.read + .option("encoding", "UTF-8") + .schema(schema) + .json(path.getAbsolutePath) + .select("*") + .filter((row: Row) => true) + .count() + } + + benchmark.run() + } + } + + def selectSubsetOfColumns(rowsNum: Int, numIters: Int): Unit = { + val colsNum = 10 + val benchmark = + new Benchmark(s"Select a subset of $colsNum columns", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + + val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) + val schema = StructType(fields) + + spark.range(rowsNum) + .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) + .write + .json(path.getAbsolutePath) + + val in = spark.read.schema(schema).json(path.getAbsolutePath) + + benchmark.addCase(s"Select $colsNum columns", numIters) { _ => + val ds = in.select("*") + run(ds) + } + benchmark.addCase(s"Select 1 column", numIters) { _ => + val ds = in.select($"col1") + run(ds) + } + + benchmark.run() + } + } + + def jsonParserCreation(rowsNum: Int, numIters: Int): Unit = { + val benchmark = new Benchmark("creation of JSON parser per line", rowsNum, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + + val shortColumnPath = path.getAbsolutePath + "/short" + val shortSchema = writeShortColumn(shortColumnPath, rowsNum) + + val wideColumnPath = path.getAbsolutePath + "/wide" + val wideSchema = writeWideColumn(wideColumnPath, rowsNum) + + benchmark.addCase("Short column without encoding", numIters) { _ => + val ds = spark.read.schema(shortSchema).json(shortColumnPath) + run(ds) + } + + benchmark.addCase("Short column with UTF-8", numIters) { _ => + val ds = spark.read + .option("encoding", "UTF-8") + .schema(shortSchema) + .json(shortColumnPath) + run(ds) + } + + benchmark.addCase("Wide column without encoding", numIters) { _ => + val ds = spark.read.schema(wideSchema).json(wideColumnPath) + run(ds) + } + + benchmark.addCase("Wide column with UTF-8", numIters) { _ => + val ds = spark.read + .option("encoding", "UTF-8") + .schema(wideSchema) + .json(wideColumnPath) + run(ds) + } + + benchmark.run() + } + } + + def jsonFunctions(rows: Int, iters: Int): Unit = { + val benchmark = new Benchmark("JSON functions", rows, output = output) + + prepareDataInfo(benchmark) + + val in = spark.range(0, rows, 1, 1).map(_ => """{"a":1}""") + + benchmark.addCase("Text read", iters) { _ => + run(in) + } + + benchmark.addCase("from_json", iters) { _ => + val schema = new StructType().add("a", IntegerType) + val from_json_ds = in.select(from_json('value, schema)) + run(from_json_ds) + } + + benchmark.addCase("json_tuple", iters) { _ => + val json_tuple_ds = in.select(json_tuple($"value", "a")) + run(json_tuple_ds) + } + + benchmark.addCase("get_json_object", iters) { _ => + val get_json_object_ds = in.select(get_json_object($"value", "$.a")) + run(get_json_object_ds) + } + + benchmark.run() + } + + def jsonInDS(rows: Int, iters: Int): Unit = { + val benchmark = new Benchmark("Dataset of json strings", rows, output = output) + + prepareDataInfo(benchmark) + + val in = spark.range(0, rows, 1, 1).map(_ => """{"a":1}""") + + benchmark.addCase("Text read", iters) { _ => + run(in) + } + + benchmark.addCase("schema inferring", iters) { _ => + spark.read.json(in).schema + } + + benchmark.addCase("parsing", iters) { _ => + val schema = new StructType().add("a", IntegerType) + val ds = spark.read + .schema(schema) + .json(in) + run(ds) + } + + benchmark.run() + } + + def jsonInFile(rows: Int, iters: Int): Unit = { + val benchmark = new Benchmark("Json files in the per-line mode", rows, output = output) + + withTempPath { path => + prepareDataInfo(benchmark) + + spark.sparkContext.range(0, rows, 1, 1) + .toDF("a") + .write + .json(path.getAbsolutePath) + + benchmark.addCase("Text read", iters) { _ => + val ds = spark.read + .format("text") + .load(path.getAbsolutePath) + run(ds) + } + + benchmark.addCase("Schema inferring", iters) { _ => + val ds = spark.read + .option("multiLine", false) + .json(path.getAbsolutePath) + ds.schema + } + + val schema = new StructType().add("a", LongType) + + benchmark.addCase("Parsing without charset", iters) { _ => + val ds = spark.read + .schema(schema) + .option("multiLine", false) + .json(path.getAbsolutePath) + run(ds) + } + + benchmark.addCase("Parsing with UTF-8", iters) { _ => + val ds = spark.read + .schema(schema) + .option("multiLine", false) + .option("charset", "UTF-8") + .json(path.getAbsolutePath) + + run(ds) + } + + benchmark.run() + } + } + + private def datetimeBenchmark(rowsNum: Int, numIters: Int): Unit = { + def timestamps = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(Instant.ofEpochSecond(_)) + }.select($"value".as("timestamp")) + } + + def dates = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(d => LocalDate.ofEpochDay(d % (100 * 365))) + }.select($"value".as("date")) + } + + withTempPath { path => + + val timestampDir = new File(path, "timestamp").getAbsolutePath + val dateDir = new File(path, "date").getAbsolutePath + + val writeBench = new Benchmark("Write dates and timestamps", rowsNum, output = output) + writeBench.addCase(s"Create a dataset of timestamps", numIters) { _ => + run(timestamps) + } + + writeBench.addCase("to_json(timestamp)", numIters) { _ => + run(timestamps.select(to_json(struct($"timestamp")))) + } + + writeBench.addCase("write timestamps to files", numIters) { _ => + timestamps.write.option("header", true).mode("overwrite").json(timestampDir) + } + + writeBench.addCase("Create a dataset of dates", numIters) { _ => + run(dates) + } + + writeBench.addCase("to_json(date)", numIters) { _ => + run(dates.select(to_json(struct($"date")))) + } + + writeBench.addCase("write dates to files", numIters) { _ => + dates.write.option("header", true).mode("overwrite").json(dateDir) + } + + writeBench.run() + + val readBench = new Benchmark("Read dates and timestamps", rowsNum, output = output) + val tsSchema = new StructType().add("timestamp", TimestampType) + + readBench.addCase("read timestamp text from files", numIters) { _ => + run(spark.read.text(timestampDir)) + } + + readBench.addCase("read timestamps from files", numIters) { _ => + run(spark.read.schema(tsSchema).json(timestampDir)) + } + + readBench.addCase("infer timestamps from files", numIters) { _ => + run(spark.read.json(timestampDir)) + } + + val dateSchema = new StructType().add("date", DateType) + + readBench.addCase("read date text from files", numIters) { _ => + run(spark.read.text(dateDir)) + } + + readBench.addCase("read date from files", numIters) { _ => + run(spark.read.schema(dateSchema).json(dateDir)) + } + + 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"}""") + }.select($"value".as("timestamp")).as[String] + } + + readBench.addCase("timestamp strings", numIters) { _ => + run(timestampStr) + } + + readBench.addCase("parse timestamps from Dataset[String]", numIters) { _ => + run(spark.read.schema(tsSchema).json(timestampStr)) + } + + readBench.addCase("infer timestamps from Dataset[String]", numIters) { _ => + run(spark.read.json(timestampStr)) + } + + def dateStr: Dataset[String] = { + spark.range(0, rowsNum, 1, 1).mapPartitions { iter => + iter.map(i => s"""{"date":"${LocalDate.ofEpochDay(i % 1000 * 365).toString}"}""") + }.select($"value".as("date")).as[String] + } + + readBench.addCase("date strings", numIters) { _ => + run(dateStr) + } + + readBench.addCase("parse dates from Dataset[String]", numIters) { _ => + val ds = spark.read + .option("header", false) + .schema(dateSchema) + .json(dateStr) + run(ds) + } + + readBench.addCase("from_json(timestamp)", numIters) { _ => + val ds = timestampStr.select(from_json($"timestamp", tsSchema, Map.empty[String, String])) + run(ds) + } + + readBench.addCase("from_json(date)", numIters) { _ => + val ds = dateStr.select(from_json($"date", dateSchema, Map.empty[String, String])) + run(ds) + } + + readBench.run() + } + } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + val numIters = 3 + runBenchmark("Benchmark for performance of JSON parsing") { + schemaInferring(100 * 1000 * 1000, numIters) + countShortColumn(100 * 1000 * 1000, numIters) + countWideColumn(10 * 1000 * 1000, numIters) + countWideRow(500 * 1000, numIters) + selectSubsetOfColumns(10 * 1000 * 1000, numIters) + jsonParserCreation(10 * 1000 * 1000, numIters) + jsonFunctions(10 * 1000 * 1000, numIters) + jsonInDS(50 * 1000 * 1000, numIters) + jsonInFile(50 * 1000 * 1000, numIters) + datetimeBenchmark(rowsNum = 10 * 1000 * 1000, numIters) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala deleted file mode 100644 index 3c4a5ab32724b..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/JsonBenchmarks.scala +++ /dev/null @@ -1,217 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.spark.sql.execution.datasources.json - -import java.io.File - -import org.apache.spark.SparkConf -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.{Row, SparkSession} -import org.apache.spark.sql.catalyst.plans.SQLHelper -import org.apache.spark.sql.functions.lit -import org.apache.spark.sql.types._ - -/** - * The benchmarks aims to measure performance of JSON parsing when encoding is set and isn't. - * To run this: - * spark-submit --class --jars - */ -object JSONBenchmarks extends SQLHelper { - val conf = new SparkConf() - - val spark = SparkSession.builder - .master("local[1]") - .appName("benchmark-json-datasource") - .config(conf) - .getOrCreate() - import spark.implicits._ - - def schemaInferring(rowsNum: Int): Unit = { - val benchmark = new Benchmark("JSON schema inferring", rowsNum) - - withTempPath { path => - // scalastyle:off println - benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on println - - spark.sparkContext.range(0, rowsNum, 1) - .map(_ => "a") - .toDF("fieldA") - .write - .option("encoding", "UTF-8") - .json(path.getAbsolutePath) - - benchmark.addCase("No encoding", 3) { _ => - spark.read.json(path.getAbsolutePath) - } - - benchmark.addCase("UTF-8 is set", 3) { _ => - spark.read - .option("encoding", "UTF-8") - .json(path.getAbsolutePath) - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_172-b11 on Mac OS X 10.13.5 - Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - - JSON schema inferring: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - --------------------------------------------------------------------------------------------- - No encoding 45908 / 46480 2.2 459.1 1.0X - UTF-8 is set 68469 / 69762 1.5 684.7 0.7X - */ - benchmark.run() - } - } - - def perlineParsing(rowsNum: Int): Unit = { - val benchmark = new Benchmark("JSON per-line parsing", rowsNum) - - withTempPath { path => - // scalastyle:off println - benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on println - - spark.sparkContext.range(0, rowsNum, 1) - .map(_ => "a") - .toDF("fieldA") - .write.json(path.getAbsolutePath) - val schema = new StructType().add("fieldA", StringType) - - benchmark.addCase("No encoding", 3) { _ => - spark.read - .schema(schema) - .json(path.getAbsolutePath) - .count() - } - - benchmark.addCase("UTF-8 is set", 3) { _ => - spark.read - .option("encoding", "UTF-8") - .schema(schema) - .json(path.getAbsolutePath) - .count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_172-b11 on Mac OS X 10.13.5 - Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - - JSON per-line parsing: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - --------------------------------------------------------------------------------------------- - No encoding 9982 / 10237 10.0 99.8 1.0X - UTF-8 is set 16373 / 16806 6.1 163.7 0.6X - */ - benchmark.run() - } - } - - def perlineParsingOfWideColumn(rowsNum: Int): Unit = { - val benchmark = new Benchmark("JSON parsing of wide lines", rowsNum) - - withTempPath { path => - // scalastyle:off println - benchmark.out.println("Preparing data for benchmarking ...") - // scalastyle:on println - - spark.sparkContext.range(0, rowsNum, 1) - .map { i => - val s = "abcdef0123456789ABCDEF" * 20 - s"""{"a":"$s","b": $i,"c":"$s","d":$i,"e":"$s","f":$i,"x":"$s","y":$i,"z":"$s"}""" - } - .toDF().write.text(path.getAbsolutePath) - val schema = new StructType() - .add("a", StringType).add("b", LongType) - .add("c", StringType).add("d", LongType) - .add("e", StringType).add("f", LongType) - .add("x", StringType).add("y", LongType) - .add("z", StringType) - - benchmark.addCase("No encoding", 3) { _ => - spark.read - .schema(schema) - .json(path.getAbsolutePath) - .count() - } - - benchmark.addCase("UTF-8 is set", 3) { _ => - spark.read - .option("encoding", "UTF-8") - .schema(schema) - .json(path.getAbsolutePath) - .count() - } - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_172-b11 on Mac OS X 10.13.5 - Intel(R) Core(TM) i7-7820HQ CPU @ 2.90GHz - - JSON parsing of wide lines: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - --------------------------------------------------------------------------------------------- - No encoding 26038 / 26386 0.4 2603.8 1.0X - UTF-8 is set 28343 / 28557 0.4 2834.3 0.9X - */ - benchmark.run() - } - } - - def countBenchmark(rowsNum: Int): Unit = { - val colsNum = 10 - val benchmark = new Benchmark(s"Count a dataset with $colsNum columns", rowsNum) - - withTempPath { path => - val fields = Seq.tabulate(colsNum)(i => StructField(s"col$i", IntegerType)) - val schema = StructType(fields) - val columnNames = schema.fieldNames - - spark.range(rowsNum) - .select(Seq.tabulate(colsNum)(i => lit(i).as(s"col$i")): _*) - .write - .json(path.getAbsolutePath) - - val ds = spark.read.schema(schema).json(path.getAbsolutePath) - - benchmark.addCase(s"Select $colsNum columns + count()", 3) { _ => - ds.select("*").filter((_: Row) => true).count() - } - benchmark.addCase(s"Select 1 column + count()", 3) { _ => - ds.select($"col1").filter((_: Row) => true).count() - } - benchmark.addCase(s"count()", 3) { _ => - ds.count() - } - - /* - Intel(R) Core(TM) i7-7700HQ CPU @ 2.80GHz - - Count a dataset with 10 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - --------------------------------------------------------------------------------------------- - Select 10 columns + count() 9961 / 10006 1.0 996.1 1.0X - Select 1 column + count() 8355 / 8470 1.2 835.5 1.2X - count() 2104 / 2156 4.8 210.4 4.7X - */ - benchmark.run() - } - } - - def main(args: Array[String]): Unit = { - schemaInferring(100 * 1000 * 1000) - perlineParsing(100 * 1000 * 1000) - perlineParsingOfWideColumn(10 * 1000 * 1000) - countBenchmark(10 * 1000 * 1000) - } -} 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 3e4cc8f166279..34b44be576897 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 @@ -31,14 +31,14 @@ import org.apache.hadoop.io.compress.GzipCodec import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.rdd.RDD import org.apache.spark.sql.{functions => F, _} -import org.apache.spark.sql.catalyst.json.{CreateJacksonParser, JacksonParser, JsonInferSchema, JSONOptions} -import org.apache.spark.sql.catalyst.json.JsonInferSchema.compatibleType +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.datasources.DataSource import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.sql.types.StructType.fromDDL import org.apache.spark.util.Utils class TestFileFilter extends PathFilter { @@ -58,16 +58,19 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } val factory = new JsonFactory() - def enforceCorrectType(value: Any, dataType: DataType): Any = { + def enforceCorrectType( + value: Any, + dataType: DataType, + options: Map[String, String] = Map.empty): Any = { val writer = new StringWriter() Utils.tryWithResource(factory.createGenerator(writer)) { generator => generator.writeObject(value) generator.flush() } - val dummyOption = new JSONOptions(Map.empty[String, String], "GMT") + val dummyOption = new JSONOptions(options, SQLConf.get.sessionLocalTimeZone) val dummySchema = StructType(Seq.empty) - val parser = new JacksonParser(dummySchema, dummyOption) + val parser = new JacksonParser(dummySchema, dummyOption, allowArrayAsStructs = true) Utils.tryWithResource(factory.createParser(writer.toString)) { jsonParser => jsonParser.nextToken() @@ -97,19 +100,27 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(intNumber.toLong * 1000L)), enforceCorrectType(intNumber.toLong, TimestampType)) val strTime = "2014-09-30 12:34:56" - checkTypePromotion(DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), - enforceCorrectType(strTime, TimestampType)) + checkTypePromotion( + expected = DateTimeUtils.fromJavaTimestamp(Timestamp.valueOf(strTime)), + enforceCorrectType(strTime, TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd HH:mm:ss"))) val strDate = "2014-10-15" checkTypePromotion( DateTimeUtils.fromJavaDate(Date.valueOf(strDate)), enforceCorrectType(strDate, DateType)) val ISO8601Time1 = "1970-01-01T01:00:01.0Z" - val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(3601000)), - enforceCorrectType(ISO8601Time1, TimestampType)) + enforceCorrectType( + ISO8601Time1, + TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SX"))) + val ISO8601Time2 = "1970-01-01T02:00:01-01:00" checkTypePromotion(DateTimeUtils.fromJavaTimestamp(new Timestamp(10801000)), - enforceCorrectType(ISO8601Time2, TimestampType)) + enforceCorrectType( + ISO8601Time2, + TimestampType, + Map("timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ssXXX"))) val ISO8601Date = "1970-01-01" checkTypePromotion(DateTimeUtils.millisToDays(32400000), @@ -118,10 +129,10 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("Get compatible type") { def checkDataType(t1: DataType, t2: DataType, expected: DataType) { - var actual = compatibleType(t1, t2) + var actual = JsonInferSchema.compatibleType(t1, t2) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") - actual = compatibleType(t2, t1) + actual = JsonInferSchema.compatibleType(t2, t1) assert(actual == expected, s"Expected $expected as the most general data type for $t1 and $t2, found $actual") } @@ -1369,9 +1380,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { test("SPARK-6245 JsonInferSchema.infer on empty RDD") { // This is really a test that it doesn't throw an exception - val emptySchema = JsonInferSchema.infer( + val options = new JSONOptions(Map.empty[String, String], "GMT") + val emptySchema = new JsonInferSchema(options).infer( empty.rdd, - new JSONOptions(Map.empty[String, String], "GMT"), CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } @@ -1396,9 +1407,9 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { } test("SPARK-8093 Erase empty structs") { - val emptySchema = JsonInferSchema.infer( + val options = new JSONOptions(Map.empty[String, String], "GMT") + val emptySchema = new JsonInferSchema(options).infer( emptyRecords.rdd, - new JSONOptions(Map.empty[String, String], "GMT"), CreateJacksonParser.string) assert(StructType(Seq()) === emptySchema) } @@ -1436,107 +1447,6 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { }) } - test("backward compatibility") { - // This test we make sure our JSON support can read JSON data generated by previous version - // of Spark generated through toJSON method and JSON data source. - // The data is generated by the following program. - // Here are a few notes: - // - Spark 1.5.0 cannot save timestamp data. So, we manually added timestamp field (col13) - // in the JSON object. - // - For Spark before 1.5.1, we do not generate UDTs. So, we manually added the UDT value to - // JSON objects generated by those Spark versions (col17). - // - If the type is NullType, we do not write data out. - - // Create the schema. - val struct = - StructType( - StructField("f1", FloatType, true) :: - StructField("f2", ArrayType(BooleanType), true) :: Nil) - - val dataTypes = - Seq( - StringType, BinaryType, NullType, BooleanType, - ByteType, ShortType, IntegerType, LongType, - FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), - DateType, TimestampType, - ArrayType(IntegerType), MapType(StringType, LongType), struct, - new UDT.MyDenseVectorUDT()) - val fields = dataTypes.zipWithIndex.map { case (dataType, index) => - StructField(s"col$index", dataType, nullable = true) - } - val schema = StructType(fields) - - val constantValues = - Seq( - "a string in binary".getBytes(StandardCharsets.UTF_8), - null, - true, - 1.toByte, - 2.toShort, - 3, - Long.MaxValue, - 0.25.toFloat, - 0.75, - new java.math.BigDecimal(s"1234.23456"), - new java.math.BigDecimal(s"1.23456"), - java.sql.Date.valueOf("2015-01-01"), - java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"), - Seq(2, 3, 4), - Map("a string" -> 2000L), - Row(4.75.toFloat, Seq(false, true)), - new UDT.MyDenseVector(Array(0.25, 2.25, 4.25))) - val data = - Row.fromSeq(Seq("Spark " + spark.sparkContext.version) ++ constantValues) :: Nil - - // Data generated by previous versions. - // scalastyle:off - val existingJSONData = - """{"col0":"Spark 1.2.2","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.3.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.4.1","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"2015-01-01","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: - """{"col0":"Spark 1.5.0","col1":"YSBzdHJpbmcgaW4gYmluYXJ5","col3":true,"col4":1,"col5":2,"col6":3,"col7":9223372036854775807,"col8":0.25,"col9":0.75,"col10":1234.23456,"col11":1.23456,"col12":"16436","col13":"2015-01-01 23:50:59.123","col14":[2,3,4],"col15":{"a string":2000},"col16":{"f1":4.75,"f2":[false,true]},"col17":[0.25,2.25,4.25]}""" :: Nil - // scalastyle:on - - // Generate data for the current version. - val df = spark.createDataFrame(spark.sparkContext.parallelize(data, 1), schema) - withTempPath { path => - df.write.format("json").mode("overwrite").save(path.getCanonicalPath) - - // df.toJSON will convert internal rows to external rows first and then generate - // JSON objects. While, df.write.format("json") will write internal rows directly. - val allJSON = - existingJSONData ++ - df.toJSON.collect() ++ - sparkContext.textFile(path.getCanonicalPath).collect() - - Utils.deleteRecursively(path) - sparkContext.parallelize(allJSON, 1).saveAsTextFile(path.getCanonicalPath) - - // Read data back with the schema specified. - val col0Values = - Seq( - "Spark 1.2.2", - "Spark 1.3.1", - "Spark 1.3.1", - "Spark 1.4.1", - "Spark 1.4.1", - "Spark 1.5.0", - "Spark 1.5.0", - "Spark " + spark.sparkContext.version, - "Spark " + spark.sparkContext.version) - val expectedResult = col0Values.map { v => - Row.fromSeq(Seq(v) ++ constantValues) - } - checkAnswer( - spark.read.format("json").schema(schema).load(path.getCanonicalPath), - expectedResult - ) - } - } - test("SPARK-11544 test pathfilter") { withTempPath { dir => val path = dir.getCanonicalPath @@ -2051,9 +1961,7 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { spark.read.schema(schema).json(path).select("_corrupt_record").collect() }.getMessage assert(msg.contains("only include the internal corrupt record column")) - intercept[catalyst.errors.TreeNodeException[_]] { - spark.read.schema(schema).json(path).filter($"_corrupt_record".isNotNull).count() - } + // workaround val df = spark.read.schema(schema).json(path).cache() assert(df.filter($"_corrupt_record".isNotNull).count() == 1) @@ -2133,8 +2041,8 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { // that whole test file is mapped to only one partition. This will guarantee // reliable sampling of the input file. withSQLConf( - "spark.sql.files.maxPartitionBytes" -> (128 * 1024 * 1024).toString, - "spark.sql.files.openCostInBytes" -> (4 * 1024 * 1024).toString + SQLConf.FILES_MAX_PARTITION_BYTES.key -> (128 * 1024 * 1024).toString, + SQLConf.FILES_OPEN_COST_IN_BYTES.key -> (4 * 1024 * 1024).toString )(withTempPath { path => val ds = sampledTestData.coalesce(1) ds.write.text(path.getAbsolutePath) @@ -2515,4 +2423,116 @@ class JsonSuite extends QueryTest with SharedSQLContext with TestJsonData { checkCount(2) countForMalformedJSON(0, Seq("")) } + + test("SPARK-26745: count() for non-multiline input with empty lines") { + withTempPath { tempPath => + val path = tempPath.getCanonicalPath + Seq("""{ "a" : 1 }""", "", """ { "a" : 2 }""", " \t ") + .toDS() + .repartition(1) + .write + .text(path) + assert(spark.read.json(path).count() === 2) + } + } + + test("SPARK-25040: empty strings should be disallowed") { + def failedOnEmptyString(dataType: DataType): Unit = { + val df = spark.read.schema(s"a ${dataType.catalogString}") + .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) + val errMessage = intercept[SparkException] { + df.collect() + }.getMessage + assert(errMessage.contains( + s"Failed to parse an empty string for data type ${dataType.catalogString}")) + } + + def emptyString(dataType: DataType, expected: Any): Unit = { + val df = spark.read.schema(s"a ${dataType.catalogString}") + .option("mode", "FAILFAST").json(Seq("""{"a":""}""").toDS) + checkAnswer(df, Row(expected) :: Nil) + } + + failedOnEmptyString(BooleanType) + failedOnEmptyString(ByteType) + failedOnEmptyString(ShortType) + failedOnEmptyString(IntegerType) + failedOnEmptyString(LongType) + failedOnEmptyString(FloatType) + failedOnEmptyString(DoubleType) + failedOnEmptyString(DecimalType.SYSTEM_DEFAULT) + failedOnEmptyString(TimestampType) + failedOnEmptyString(DateType) + failedOnEmptyString(ArrayType(IntegerType)) + failedOnEmptyString(MapType(StringType, IntegerType, true)) + failedOnEmptyString(StructType(StructField("f1", IntegerType, true) :: Nil)) + + emptyString(StringType, "") + emptyString(BinaryType, "".getBytes(StandardCharsets.UTF_8)) + } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.json(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("json"))) + } + } + + test("return partial result for bad records") { + val schema = "a double, b array, c string, _corrupt_record string" + val badRecords = Seq( + """{"a":"-","b":[0, 1, 2],"c":"abc"}""", + """{"a":0.1,"b":{},"c":"def"}""").toDS() + val df = spark.read.schema(schema).json(badRecords) + + checkAnswer( + df, + Row(null, Array(0, 1, 2), "abc", """{"a":"-","b":[0, 1, 2],"c":"abc"}""") :: + Row(0.1, null, "def", """{"a":0.1,"b":{},"c":"def"}""") :: Nil) + } + + test("inferring timestamp type") { + def schemaOf(jsons: String*): StructType = spark.read.json(jsons.toDS).schema + + assert(schemaOf( + """{"a":"2018-12-17T10:11:12.123-01:00"}""", + """{"a":"2018-12-16T22:23:24.123-02:00"}""") === fromDDL("a timestamp")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":1}""") + === fromDDL("a string")) + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":"123"}""") + === fromDDL("a string")) + + assert(schemaOf("""{"a":"2018-12-17T10:11:12.123-01:00"}""", """{"a":null}""") + === fromDDL("a timestamp")) + assert(schemaOf("""{"a":null}""", """{"a":"2018-12-17T10:11:12.123-01:00"}""") + === fromDDL("a timestamp")) + } + + test("roundtrip for timestamp type inferring") { + val customSchema = new StructType().add("date", TimestampType) + withTempDir { dir => + val timestampsWithFormatPath = s"${dir.getCanonicalPath}/timestampsWithFormat.json" + val timestampsWithFormat = spark.read + .option("timestampFormat", "dd/MM/yyyy HH:mm") + .json(datesRecords) + assert(timestampsWithFormat.schema === customSchema) + + timestampsWithFormat.write + .format("json") + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .save(timestampsWithFormatPath) + + val readBack = spark.read + .option("timestampFormat", "yyyy-MM-dd HH:mm:ss") + .option(DateTimeUtils.TIMEZONE_OPTION, "UTC") + .json(timestampsWithFormatPath) + + assert(readBack.schema === customSchema) + checkAnswer(readBack, timestampsWithFormat) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala index 6e9559edf8ec2..17503330bfd5c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/json/TestJsonData.scala @@ -236,8 +236,7 @@ private[json] trait TestJsonData { def sampledTestData: Dataset[String] = { spark.range(0, 100, 1).map { index => - val predefinedSample = Set[Long](2, 8, 15, 27, 30, 34, 35, 37, 44, 46, - 57, 62, 68, 72) + val predefinedSample = Set[Long](3, 18, 20, 24, 50, 60, 87, 99) if (predefinedSample.contains(index)) { s"""{"f1":${index.toString}}""" } else { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala new file mode 100644 index 0000000000000..3cb8287f09b26 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopStreamSuite.scala @@ -0,0 +1,96 @@ +/* + * 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.noop + +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.streaming.MemoryStream +import org.apache.spark.sql.streaming.{StreamingQuery, StreamTest, Trigger} + +class NoopStreamSuite extends StreamTest { + import testImplicits._ + + test("microbatch") { + val input = MemoryStream[Int] + val query = input.toDF().writeStream.format("noop").start() + testMicroBatchQuery(query, input) + } + + test("microbatch restart with checkpoint") { + val input = MemoryStream[Int] + withTempDir { checkpointDir => + def testWithCheckpoint(): Unit = { + val query = input.toDF().writeStream + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .format("noop") + .start() + testMicroBatchQuery(query, input) + } + testWithCheckpoint() + testWithCheckpoint() + } + } + + private def testMicroBatchQuery( + query: StreamingQuery, + input: MemoryStream[Int], + data: Int*): Unit = { + assert(query.isActive) + try { + input.addData(1, 2, 3) + eventually(timeout(streamingTimeout)) { + assert(query.recentProgress.map(_.numInputRows).sum == 3) + } + } finally { + query.stop() + } + } + + test("continuous") { + val input = getRateDataFrame() + val query = input.writeStream.format("noop").trigger(Trigger.Continuous(200)).start() + assert(query.isActive) + query.stop() + } + + test("continuous restart with checkpoint") { + withTempDir { checkpointDir => + def testWithCheckpoint(): Unit = { + val input = getRateDataFrame() + val query = input.writeStream + .option("checkpointLocation", checkpointDir.getAbsolutePath) + .format("noop") + .trigger(Trigger.Continuous(200)) + .start() + assert(query.isActive) + query.stop() + } + testWithCheckpoint() + testWithCheckpoint() + } + } + + private def getRateDataFrame(): DataFrame = { + spark.readStream + .format("rate") + .option("numPartitions", "1") + .option("rowsPerSecond", "5") + .load() + .select('value) + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala new file mode 100644 index 0000000000000..59de28688ec1d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/noop/NoopSuite.scala @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.execution.datasources.noop + +import org.apache.spark.sql.test.SharedSQLContext + +class NoopSuite extends SharedSQLContext { + import testImplicits._ + + test("materialisation of all rows") { + val numElems = 10 + val accum = spark.sparkContext.longAccumulator + spark.range(numElems) + .map { x => + accum.add(1) + x + } + .write + .format("noop") + .save() + assert(accum.value == numElems) + } + + test("read partitioned data") { + val numElems = 100 + withTempPath { dir => + val path = dir.getCanonicalPath + spark.range(numElems) + .select('id mod 10 as "key", 'id as "value") + .write + .partitionBy("key") + .parquet(path) + + val accum = spark.sparkContext.longAccumulator + spark.read + .parquet(path) + .as[(Long, Long)] + .map { x => + accum.add(1) + x + } + .write.format("noop").save() + assert(accum.value == numElems) + } + } +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.scala new file mode 100644 index 0000000000000..c16fcc67f8dd1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcColumnarBatchReaderSuite.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.sql.execution.datasources.orc + +import org.apache.orc.TypeDescription + +import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.vectorized.{OnHeapColumnVector, WritableColumnVector} +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.unsafe.types.UTF8String.fromString + +class OrcColumnarBatchReaderSuite extends QueryTest with SQLTestUtils with SharedSQLContext { + private val dataSchema = StructType.fromDDL("col1 int, col2 int") + private val partitionSchema = StructType.fromDDL("p1 string, p2 string") + private val partitionValues = InternalRow(fromString("partValue1"), fromString("partValue2")) + private val orcFileSchemaList = Seq( + "struct", "struct", + "struct", "struct") + orcFileSchemaList.foreach { case schema => + val orcFileSchema = TypeDescription.fromString(schema) + + val isConstant = classOf[WritableColumnVector].getDeclaredField("isConstant") + isConstant.setAccessible(true) + + def getReader( + requestedDataColIds: Array[Int], + requestedPartitionColIds: Array[Int], + resultFields: Array[StructField]): OrcColumnarBatchReader = { + val reader = new OrcColumnarBatchReader(4096) + reader.initBatch( + orcFileSchema, + resultFields, + requestedDataColIds, + requestedPartitionColIds, + partitionValues) + reader + } + + test(s"all partitions are requested: $schema") { + val requestedDataColIds = Array(0, 1, 0, 0) + val requestedPartitionColIds = Array(-1, -1, 0, 1) + val reader = getReader(requestedDataColIds, requestedPartitionColIds, + dataSchema.fields ++ partitionSchema.fields) + assert(reader.requestedDataColIds === Array(0, 1, -1, -1)) + } + + test(s"initBatch should initialize requested partition columns only: $schema") { + val requestedDataColIds = Array(0, -1) // only `col1` is requested, `col2` doesn't exist + val requestedPartitionColIds = Array(-1, 0) // only `p1` is requested + val reader = getReader(requestedDataColIds, requestedPartitionColIds, + Array(dataSchema.fields(0), partitionSchema.fields(0))) + val batch = reader.columnarBatch + assert(batch.numCols() === 2) + + assert(batch.column(0).isInstanceOf[OrcColumnVector]) + assert(batch.column(1).isInstanceOf[OnHeapColumnVector]) + + val p1 = batch.column(1).asInstanceOf[OnHeapColumnVector] + assert(isConstant.get(p1).asInstanceOf[Boolean]) // Partition column is constant. + assert(p1.getUTF8String(0) === partitionValues.getUTF8String(0)) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala index d1911ea7f32a9..143e3f0997201 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcPartitionDiscoverySuite.scala @@ -19,7 +19,11 @@ package org.apache.spark.sql.execution.datasources.orc import java.io.File +import org.apache.hadoop.fs.{Path, PathFilter} + +import org.apache.spark.SparkConf import org.apache.spark.sql._ +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext // The data where the partitioning key exists only in the directory structure. @@ -28,6 +32,10 @@ case class OrcParData(intField: Int, stringField: String) // The data that also includes the partitioning key case class OrcParDataWithKey(intField: Int, pi: Int, stringField: String, ps: String) +class TestFileFilter extends PathFilter { + override def accept(path: Path): Boolean = path.getParent.getName != "p=2" +} + abstract class OrcPartitionDiscoveryTest extends OrcTest { val defaultPartitionName = "__HIVE_DEFAULT_PARTITION__" @@ -99,6 +107,68 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } } + test("read partitioned table - with nulls") { + withTempDir { base => + for { + // Must be `Integer` rather than `Int` here. `null.asInstanceOf[Int]` results in a zero... + pi <- Seq(1, null.asInstanceOf[Integer]) + ps <- Seq("foo", null.asInstanceOf[String]) + } { + makeOrcFile( + (1 to 10).map(i => OrcParData(i, i.toString)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + spark.read + .option("hive.exec.default.partition.name", defaultPartitionName) + .orc(base.getCanonicalPath) + .createOrReplaceTempView("t") + + withTempTable("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, null.asInstanceOf[Integer]) + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi IS NULL"), + for { + i <- 1 to 10 + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, i.toString, null, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps IS NULL"), + for { + i <- 1 to 10 + pi <- Seq(1, null.asInstanceOf[Integer]) + } yield Row(i, i.toString, pi, null)) + } + } + } + + test("SPARK-27162: handle pathfilter configuration correctly") { + withTempPath { dir => + val path = dir.getCanonicalPath + + val df = spark.range(2) + df.write.orc(path + "/p=1") + df.write.orc(path + "/p=2") + assert(spark.read.orc(path).count() === 4) + + val extraOptions = Map( + "mapred.input.pathFilter.class" -> classOf[TestFileFilter].getName, + "mapreduce.input.pathFilter.class" -> classOf[TestFileFilter].getName + ) + assert(spark.read.options(extraOptions).orc(path).count() === 2) + } + } +} + +class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { test("read partitioned table - partition key included in orc file") { withTempDir { base => for { @@ -119,7 +189,7 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { i <- 1 to 10 pi <- Seq(1, 2) ps <- Seq("foo", "bar") - } yield Row(i, pi, i.toString, ps)) + } yield Row(i, i.toString, pi, ps)) checkAnswer( sql("SELECT intField, pi FROM t"), @@ -134,28 +204,26 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { for { i <- 1 to 10 ps <- Seq("foo", "bar") - } yield Row(i, 1, i.toString, ps)) + } yield Row(i, i.toString, 1, ps)) checkAnswer( sql("SELECT * FROM t WHERE ps = 'foo'"), for { i <- 1 to 10 pi <- Seq(1, 2) - } yield Row(i, pi, i.toString, "foo")) + } yield Row(i, i.toString, pi, "foo")) } } } - - test("read partitioned table - with nulls") { + test("read partitioned table - with nulls and partition keys are included in Orc file") { withTempDir { base => for { - // Must be `Integer` rather than `Int` here. `null.asInstanceOf[Int]` results in a zero... - pi <- Seq(1, null.asInstanceOf[Integer]) + pi <- Seq(1, 2) ps <- Seq("foo", null.asInstanceOf[String]) } { makeOrcFile( - (1 to 10).map(i => OrcParData(i, i.toString)), + (1 to 10).map(i => OrcParDataWithKey(i, pi, i.toString, ps)), makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) } @@ -169,23 +237,71 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { sql("SELECT * FROM t"), for { i <- 1 to 10 - pi <- Seq(1, null.asInstanceOf[Integer]) + pi <- Seq(1, 2) ps <- Seq("foo", null.asInstanceOf[String]) } yield Row(i, i.toString, pi, ps)) checkAnswer( - sql("SELECT * FROM t WHERE pi IS NULL"), + sql("SELECT * FROM t WHERE ps IS NULL"), for { i <- 1 to 10 - ps <- Seq("foo", null.asInstanceOf[String]) - } yield Row(i, i.toString, null, ps)) + pi <- Seq(1, 2) + } yield Row(i, i.toString, pi, null)) + } + } + } +} +class OrcV1PartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "orc") + + test("read partitioned table - partition key included in orc file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } { + makeOrcFile( + (1 to 10).map(i => OrcParDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + spark.read.orc(base.getCanonicalPath).createOrReplaceTempView("t") + + withTempTable("t") { checkAnswer( - sql("SELECT * FROM t WHERE ps IS NULL"), + sql("SELECT * FROM t"), for { i <- 1 to 10 - pi <- Seq(1, null.asInstanceOf[Integer]) - } yield Row(i, i.toString, pi, null)) + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, pi, i.toString, ps)) + + checkAnswer( + sql("SELECT intField, pi FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + _ <- Seq("foo", "bar") + } yield Row(i, pi)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, 1, i.toString, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, pi, i.toString, "foo")) } } } @@ -225,5 +341,3 @@ abstract class OrcPartitionDiscoveryTest extends OrcTest { } } } - -class OrcPartitionDiscoverySuite extends OrcPartitionDiscoveryTest with SharedSQLContext 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 e9dccbf2e261c..18ec3e3ebed0c 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 @@ -31,13 +31,13 @@ import org.apache.orc.OrcConf.COMPRESS import org.apache.orc.mapred.OrcStruct import org.apache.orc.mapreduce.OrcInputFormat -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation, RecordReaderIterator} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.types._ import org.apache.spark.util.Utils case class AllDataTypesWithNonPrimitiveType( @@ -103,7 +103,7 @@ abstract class OrcQueryTest extends OrcTest { test("Read/write UserDefinedType") { withTempPath { path => - val data = Seq((1, new UDT.MyDenseVector(Array(0.25, 2.25, 4.25)))) + val data = Seq((1, new TestUDT.MyDenseVector(Array(0.25, 2.25, 4.25)))) val udtDF = data.toDF("id", "vectors") udtDF.write.orc(path.getAbsolutePath) val readBack = spark.read.schema(udtDF.schema).orc(path.getAbsolutePath) @@ -270,10 +270,16 @@ abstract class OrcQueryTest extends OrcTest { test("appending") { val data = (0 until 10).map(i => (i, i.toString)) spark.createDataFrame(data).toDF("c1", "c2").createOrReplaceTempView("tmp") - withOrcTable(data, "t") { - sql("INSERT INTO TABLE t SELECT * FROM tmp") - checkAnswer(spark.table("t"), (data ++ data).map(Row.fromTuple)) + + withOrcFile(data) { file => + withTempView("t") { + spark.read.orc(file).createOrReplaceTempView("t") + checkAnswer(spark.table("t"), data.map(Row.fromTuple)) + sql("INSERT INTO TABLE t SELECT * FROM tmp") + checkAnswer(spark.table("t"), (data ++ data).map(Row.fromTuple)) + } } + spark.sessionState.catalog.dropTable( TableIdentifier("tmp"), ignoreIfNotExists = true, @@ -445,16 +451,7 @@ abstract class OrcQueryTest extends OrcTest { test("Support for pushing down filters for decimal types") { withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { val data = (0 until 10).map(i => Tuple1(BigDecimal.valueOf(i))) - withTempPath { file => - // It needs to repartition data so that we can have several ORC files - // in order to skip stripes in ORC. - spark.createDataFrame(data).toDF("a").repartition(10) - .write.orc(file.getCanonicalPath) - val df = spark.read.orc(file.getCanonicalPath).where("a == 2") - val actual = stripSparkFilter(df).count() - - assert(actual < 10) - } + checkPredicatePushDown(spark.createDataFrame(data).toDF("a"), 10, "a == 2") } } @@ -465,16 +462,7 @@ abstract class OrcQueryTest extends OrcTest { val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 Tuple1(new Timestamp(milliseconds)) } - withTempPath { file => - // It needs to repartition data so that we can have several ORC files - // in order to skip stripes in ORC. - spark.createDataFrame(data).toDF("a").repartition(10) - .write.orc(file.getCanonicalPath) - val df = spark.read.orc(file.getCanonicalPath).where(s"a == '$timeString'") - val actual = stripSparkFilter(df).count() - - assert(actual < 10) - } + checkPredicatePushDown(spark.createDataFrame(data).toDF("a"), 10, s"a == '$timeString'") } } @@ -615,6 +603,22 @@ abstract class OrcQueryTest extends OrcTest { assert(m4.contains("Malformed ORC file")) } } + + test("SPARK-27160 Predicate pushdown correctness on DecimalType for ORC") { + withTempPath { dir => + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + val path = dir.getCanonicalPath + Seq(BigDecimal(0.1), BigDecimal(0.2), BigDecimal(-0.3)) + .toDF("x").write.orc(path) + val df = spark.read.orc(path) + checkAnswer(df.filter("x >= 0.1"), Seq(Row(0.1), Row(0.2))) + checkAnswer(df.filter("x > 0.1"), Seq(Row(0.2))) + checkAnswer(df.filter("x <= 0.15"), Seq(Row(0.1), Row(-0.3))) + checkAnswer(df.filter("x < 0.1"), Seq(Row(-0.3))) + checkAnswer(df.filter("x == 0.2"), Seq(Row(0.2))) + } + } + } } class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { @@ -674,6 +678,12 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } + test("SPARK-25579 ORC PPD should support column names with dot") { + withSQLConf(SQLConf.ORC_FILTER_PUSHDOWN_ENABLED.key -> "true") { + checkPredicatePushDown(spark.range(10).toDF("col.dot"), 10, "`col.dot` == 2") + } + } + test("SPARK-20728 Make ORCFileFormat configurable between sql/hive and sql/core") { withSQLConf(SQLConf.ORC_IMPLEMENTATION.key -> "hive") { val e = intercept[AnalysisException] { @@ -693,3 +703,11 @@ class OrcQuerySuite extends OrcQueryTest with SharedSQLContext { } } } + +class OrcV1QuerySuite extends OrcQuerySuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "orc") +} 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 dc81c0585bf18..c9f5d9cb23e6b 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 @@ -18,11 +18,12 @@ 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.util.Locale import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, FileSystem, Path} import org.apache.orc.OrcConf.COMPRESS import org.apache.orc.OrcFile import org.apache.orc.OrcProto.ColumnEncoding.Kind.{DICTIONARY_V2, DIRECT, DIRECT_V2} @@ -30,9 +31,12 @@ import org.apache.orc.OrcProto.Stream.Kind import org.apache.orc.impl.RecordReaderImpl import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.Row +import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} +import org.apache.spark.sql.{Row, SPARK_VERSION_METADATA_KEY} +import org.apache.spark.sql.execution.datasources.SchemaMergeUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{LongType, StructField, StructType} import org.apache.spark.util.Utils case class OrcData(intField: Int, stringField: String) @@ -116,7 +120,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } - protected def testSelectiveDictionaryEncoding(isSelective: Boolean) { + protected def testSelectiveDictionaryEncoding(isSelective: Boolean, isHive23: Boolean = false) { val tableName = "orcTable" withTempDir { dir => @@ -169,7 +173,7 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { // Hive 0.11 and RLE v2 is introduced in Hive 0.12 ORC with more improvements. // For more details, see https://orc.apache.org/specification/ assert(stripe.getColumns(1).getKind === DICTIONARY_V2) - if (isSelective) { + if (isSelective || isHive23) { assert(stripe.getColumns(2).getKind === DIRECT_V2) } else { assert(stripe.getColumns(2).getKind === DICTIONARY_V2) @@ -186,6 +190,49 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { } } + protected def testMergeSchemasInParallel( + ignoreCorruptFiles: Boolean, + schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]): Unit = { + withSQLConf( + SQLConf.IGNORE_CORRUPT_FILES.key -> ignoreCorruptFiles.toString, + SQLConf.ORC_IMPLEMENTATION.key -> orcImp) { + withTempDir { dir => + val fs = FileSystem.get(spark.sessionState.newHadoopConf()) + val basePath = dir.getCanonicalPath + + val path1 = new Path(basePath, "first") + val path2 = new Path(basePath, "second") + val path3 = new Path(basePath, "third") + + spark.range(1).toDF("a").coalesce(1).write.orc(path1.toString) + spark.range(1, 2).toDF("b").coalesce(1).write.orc(path2.toString) + spark.range(2, 3).toDF("a").coalesce(1).write.json(path3.toString) + + val fileStatuses = + Seq(fs.listStatus(path1), fs.listStatus(path2), fs.listStatus(path3)).flatten + + val schema = SchemaMergeUtils.mergeSchemasInParallel( + spark, + fileStatuses, + schemaReader) + + assert(schema.isDefined) + assert(schema.get == StructType(Seq( + StructField("a", LongType, true), + StructField("b", LongType, true)))) + } + } + } + + protected def testMergeSchemasInParallel( + schemaReader: (Seq[FileStatus], Configuration, Boolean) => Seq[StructType]): Unit = { + testMergeSchemasInParallel(true, schemaReader) + val exception = intercept[SparkException] { + testMergeSchemasInParallel(false, schemaReader) + }.getCause + assert(exception.getCause.getMessage.contains("Could not read footer for file")) + } + test("create temporary orc table") { checkAnswer(sql("SELECT COUNT(*) FROM normal_orc_source"), Row(10)) @@ -314,6 +361,123 @@ abstract class OrcSuite extends OrcTest with BeforeAndAfterAll { checkAnswer(spark.read.orc(path.getCanonicalPath), Row(ts)) } } + + test("Write Spark version into ORC file metadata") { + withTempPath { path => + spark.range(1).repartition(1).write.orc(path.getCanonicalPath) + + val partFiles = path.listFiles() + .filter(f => f.isFile && !f.getName.startsWith(".") && !f.getName.startsWith("_")) + assert(partFiles.length === 1) + + val orcFilePath = new Path(partFiles.head.getAbsolutePath) + val readerOptions = OrcFile.readerOptions(new Configuration()) + val reader = OrcFile.createReader(orcFilePath, readerOptions) + val version = UTF_8.decode(reader.getMetadataValue(SPARK_VERSION_METADATA_KEY)).toString + assert(version === SPARK_VERSION_SHORT) + } + } + + test("SPARK-11412 test orc merge schema option") { + val conf = spark.sessionState.conf + // Test if the default of spark.sql.orc.mergeSchema is false + assert(new OrcOptions(Map.empty[String, String], conf).mergeSchema == false) + + // OrcOptions's parameters have a higher priority than SQL configuration. + // `mergeSchema` -> `spark.sql.orc.mergeSchema` + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "true") { + val map1 = Map(OrcOptions.MERGE_SCHEMA -> "true") + val map2 = Map(OrcOptions.MERGE_SCHEMA -> "false") + assert(new OrcOptions(map1, conf).mergeSchema == true) + assert(new OrcOptions(map2, conf).mergeSchema == false) + } + + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "false") { + val map1 = Map(OrcOptions.MERGE_SCHEMA -> "true") + val map2 = Map(OrcOptions.MERGE_SCHEMA -> "false") + assert(new OrcOptions(map1, conf).mergeSchema == true) + assert(new OrcOptions(map2, conf).mergeSchema == false) + } + } + + test("SPARK-11412 test enabling/disabling schema merging") { + def testSchemaMerging(expectedColumnNumber: Int): Unit = { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + spark.range(0, 10).toDF("b").write.orc(new Path(basePath, "foo=2").toString) + assert(spark.read.orc(basePath).columns.length === expectedColumnNumber) + + // OrcOptions.MERGE_SCHEMA has higher priority + assert(spark.read.option(OrcOptions.MERGE_SCHEMA, true) + .orc(basePath).columns.length === 3) + assert(spark.read.option(OrcOptions.MERGE_SCHEMA, false) + .orc(basePath).columns.length === 2) + } + } + + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "true") { + testSchemaMerging(3) + } + + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "false") { + testSchemaMerging(2) + } + } + + test("SPARK-11412 test enabling/disabling schema merging with data type conflicts") { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + spark.range(0, 10).map(s => s"value_$s").toDF("a") + .write.orc(new Path(basePath, "foo=2").toString) + + // with schema merging, there should throw exception + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "true") { + val exception = intercept[SparkException] { + spark.read.orc(basePath).columns.length + }.getCause + + val innerMessage = orcImp match { + case "native" => exception.getMessage + case "hive" => exception.getCause.getMessage + case impl => + throw new UnsupportedOperationException(s"Unknown ORC implementation: $impl") + } + + assert(innerMessage.contains("Failed to merge incompatible data types")) + } + + // it is ok if no schema merging + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "false") { + assert(spark.read.orc(basePath).columns.length === 2) + } + } + } + + test("SPARK-11412 test schema merging with corrupt files") { + withSQLConf(SQLConf.ORC_SCHEMA_MERGING_ENABLED.key -> "true") { + withTempDir { dir => + val basePath = dir.getCanonicalPath + spark.range(0, 10).toDF("a").write.orc(new Path(basePath, "foo=1").toString) + spark.range(0, 10).toDF("b").write.orc(new Path(basePath, "foo=2").toString) + spark.range(0, 10).toDF("c").write.json(new Path(basePath, "foo=3").toString) + + // ignore corrupt files + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "true") { + assert(spark.read.orc(basePath).columns.length === 3) + } + + // don't ignore corrupt files + withSQLConf(SQLConf.IGNORE_CORRUPT_FILES.key -> "false") { + val exception = intercept[SparkException] { + spark.read.orc(basePath).columns.length + }.getCause + assert(exception.getCause.getMessage.contains("Could not read footer for file")) + } + } + } + } } class OrcSourceSuite extends OrcSuite with SharedSQLContext { @@ -359,4 +523,8 @@ class OrcSourceSuite extends OrcSuite with SharedSQLContext { test("Enforce direct encoding column-wise selectively") { testSelectiveDictionaryEncoding(isSelective = true) } + + test("SPARK-11412 read and merge orc schemas in parallel") { + testMergeSchemasInParallel(OrcUtils.readOrcSchemasInParallel) + } } 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 38b34a03e3e4c..adbd93dcb4fe8 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 @@ -25,8 +25,13 @@ import scala.reflect.runtime.universe.TypeTag import org.scalatest.BeforeAndAfterAll import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FileBasedDataSourceTest} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION -import org.apache.spark.sql.test.SQLTestUtils /** * OrcTest @@ -42,13 +47,16 @@ import org.apache.spark.sql.test.SQLTestUtils * -> OrcFilterSuite * -> HiveOrcFilterSuite */ -abstract class OrcTest extends QueryTest with SQLTestUtils with BeforeAndAfterAll { - import testImplicits._ +abstract class OrcTest extends QueryTest with FileBasedDataSourceTest with BeforeAndAfterAll { val orcImp: String = "native" private var originalConfORCImplementation = "native" + override protected val dataSourceName: String = "orc" + override protected val vectorizedReaderEnabledKey: String = + SQLConf.ORC_VECTORIZED_READER_ENABLED.key + protected override def beforeAll(): Unit = { super.beforeAll() originalConfORCImplementation = spark.conf.get(ORC_IMPLEMENTATION) @@ -66,22 +74,15 @@ abstract class OrcTest extends QueryTest with SQLTestUtils with BeforeAndAfterAl */ protected def withOrcFile[T <: Product: ClassTag: TypeTag] (data: Seq[T]) - (f: String => Unit): Unit = { - withTempPath { file => - sparkContext.parallelize(data).toDF().write.orc(file.getCanonicalPath) - f(file.getCanonicalPath) - } - } + (f: String => Unit): Unit = withDataSourceFile(data)(f) /** * Writes `data` to a Orc file and reads it back as a `DataFrame`, * which is then passed to `f`. The Orc file will be deleted after `f` returns. */ protected def withOrcDataFrame[T <: Product: ClassTag: TypeTag] - (data: Seq[T]) - (f: DataFrame => Unit): Unit = { - withOrcFile(data)(path => f(spark.read.orc(path))) - } + (data: Seq[T], testVectorized: Boolean = true) + (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) /** * Writes `data` to a Orc file, reads it back as a `DataFrame` and registers it as a @@ -89,21 +90,50 @@ abstract class OrcTest extends QueryTest with SQLTestUtils with BeforeAndAfterAl * Orc file will be dropped/deleted after `f` returns. */ protected def withOrcTable[T <: Product: ClassTag: TypeTag] - (data: Seq[T], tableName: String) - (f: => Unit): Unit = { - withOrcDataFrame(data) { df => - df.createOrReplaceTempView(tableName) - withTempView(tableName)(f) - } - } + (data: Seq[T], tableName: String, testVectorized: Boolean = true) + (f: => Unit): Unit = withDataSourceTable(data, tableName, testVectorized)(f) protected def makeOrcFile[T <: Product: ClassTag: TypeTag]( - data: Seq[T], path: File): Unit = { - data.toDF().write.mode(SaveMode.Overwrite).orc(path.getCanonicalPath) - } + data: Seq[T], path: File): Unit = makeDataSourceFile(data, path) protected def makeOrcFile[T <: Product: ClassTag: TypeTag]( - df: DataFrame, path: File): Unit = { - df.write.mode(SaveMode.Overwrite).orc(path.getCanonicalPath) + df: DataFrame, path: File): Unit = makeDataSourceFile(df, path) + + protected def checkPredicatePushDown(df: DataFrame, numRows: Int, predicate: String): Unit = { + withTempPath { file => + // It needs to repartition data so that we can have several ORC files + // in order to skip stripes in ORC. + df.repartition(numRows).write.orc(file.getCanonicalPath) + val actual = stripSparkFilter(spark.read.orc(file.getCanonicalPath).where(predicate)).count() + assert(actual < numRows) + } + } + + protected def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcTable: OrcTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(options) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + if (noneSupported) { + assert(pushedFilters.isEmpty, "Unsupported filters should not show in pushed filters") + } else { + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isEmpty, s"Couldn't generate filter predicate for $pushedFilters") + } + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala new file mode 100644 index 0000000000000..4452780ff73f2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1FilterSuite.scala @@ -0,0 +1,106 @@ +/* + * 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 scala.collection.JavaConverters._ + +import org.apache.spark.SparkConf +import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Predicate} +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.orc.OrcShimUtils.{Operator, SearchArgument} +import org.apache.spark.sql.internal.SQLConf + +class OrcV1FilterSuite extends OrcFilterSuite { + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "orc") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") + checker(maybeFilter.get) + } + + override def checkFilterPredicate + (predicate: Predicate, filterOperator: Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + override def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + override def checkNoFilterPredicate + (predicate: Predicate, noneSupported: Boolean = false) + (implicit df: DataFrame): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(orcRelation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + + val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) + assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1SchemaPruningSuite.scala new file mode 100644 index 0000000000000..832da59a78ed8 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV1SchemaPruningSuite.scala @@ -0,0 +1,34 @@ +/* + * 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 org.apache.spark.SparkConf +import org.apache.spark.sql.execution.datasources.SchemaPruningSuite +import org.apache.spark.sql.internal.SQLConf + +class OrcV1SchemaPruningSuite extends SchemaPruningSuite { + override protected val dataSourceName: String = "orc" + override protected val vectorizedReaderEnabledKey: String = + SQLConf.ORC_VECTORIZED_READER_ENABLED.key + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "orc") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "orc") +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala new file mode 100644 index 0000000000000..b042f7f9d7160 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcV2SchemaPruningSuite.scala @@ -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. + */ +package org.apache.spark.sql.execution.datasources.orc + +import org.apache.spark.SparkConf +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.execution.datasources.SchemaPruningSuite +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.orc.OrcScan +import org.apache.spark.sql.internal.SQLConf + +class OrcV2SchemaPruningSuite extends SchemaPruningSuite { + override protected val dataSourceName: String = "orc" + override protected val vectorizedReaderEnabledKey: String = + SQLConf.ORC_VECTORIZED_READER_ENABLED.key + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + val fileSourceScanSchemata = + df.queryExecution.executedPlan.collect { + case BatchScanExec(_, scan: OrcScan) => scan.readDataSchema + } + assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, + s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + + s"but expected $expectedSchemaCatalogStrings") + fileSourceScanSchemata.zip(expectedSchemaCatalogStrings).foreach { + case (scanSchema, expectedScanSchemaCatalogString) => + val expectedScanSchema = CatalystSqlParser.parseDataType(expectedScanSchemaCatalogString) + implicit val equality = schemaEquality + assert(scanSchema === expectedScanSchema) + } + } +} 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 01e41b3c5df36..577d1bc8d6a4f 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 @@ -24,13 +24,18 @@ import java.sql.{Date, Timestamp} import org.apache.parquet.filter2.predicate.{FilterApi, FilterPredicate, Operators} import org.apache.parquet.filter2.predicate.FilterApi._ import org.apache.parquet.filter2.predicate.Operators.{Column => _, _} +import org.apache.parquet.schema.MessageType -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.orc.OrcFilters +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetTable import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.ParquetOutputTimestampType @@ -56,12 +61,15 @@ import org.apache.spark.util.{AccumulatorContext, AccumulatorV2} * dependent on this configuration, don't forget you better explicitly set this configuration * within the test. */ -class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext { +abstract class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContext { - private lazy val parquetFilters = - new ParquetFilters(conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, + protected def createParquetFilters( + schema: MessageType, + caseSensitive: Option[Boolean] = None): ParquetFilters = + new ParquetFilters(schema, conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, conf.parquetFilterPushDownDecimal, conf.parquetFilterPushDownStringStartWith, - conf.parquetFilterPushDownInFilterThreshold, conf.caseSensitiveAnalysis) + conf.parquetFilterPushDownInFilterThreshold, + caseSensitive.getOrElse(conf.caseSensitiveAnalysis)) override def beforeEach(): Unit = { super.beforeEach() @@ -77,48 +85,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } - private def checkFilterPredicate( + def checkFilterPredicate( df: DataFrame, predicate: Predicate, filterClass: Class[_ <: FilterPredicate], checker: (DataFrame, Seq[Row]) => Unit, - expected: Seq[Row]): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - - withSQLConf( - SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", - SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", - SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", - SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", - SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", - SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, - LogicalRelation(relation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(relation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - selectedFilters.foreach { pred => - val maybeFilter = parquetFilters.createFilter( - new SparkToParquetSchemaConverter(conf).convert(df.schema), pred) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") - // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) - maybeFilter.exists(_.getClass === filterClass) - } - checker(stripSparkFilter(query), expected) - } - } + expected: Seq[Row]): Unit private def checkFilterPredicate (predicate: Predicate, filterClass: Class[_ <: FilterPredicate], expected: Seq[Row]) @@ -515,50 +487,49 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> ParquetOutputTimestampType.INT96.toString) { withParquetDataFrame(millisData.map(i => Tuple1(i))) { implicit df => + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) assertResult(None) { - parquetFilters.createFilter( - new SparkToParquetSchemaConverter(conf).convert(df.schema), sources.IsNull("_1")) + createParquetFilters(schema).createFilter(sources.IsNull("_1")) } } } } test("filter pushdown - decimal") { - Seq(true, false).foreach { legacyFormat => + Seq( + (false, Decimal.MAX_INT_DIGITS), // int32Writer + (false, Decimal.MAX_LONG_DIGITS), // int64Writer + (true, Decimal.MAX_LONG_DIGITS), // binaryWriterUsingUnscaledLong + (false, DecimalType.MAX_PRECISION) // binaryWriterUsingUnscaledBytes + ).foreach { case (legacyFormat, precision) => withSQLConf(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key -> legacyFormat.toString) { - Seq( - s"a decimal(${Decimal.MAX_INT_DIGITS}, 2)", // 32BitDecimalType - s"a decimal(${Decimal.MAX_LONG_DIGITS}, 2)", // 64BitDecimalType - "a decimal(38, 18)" // ByteArrayDecimalType - ).foreach { schemaDDL => - val schema = StructType.fromDDL(schemaDDL) - 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 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))) } } } @@ -580,24 +551,24 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex assert(decimal.scale() === scale) assert(decimal1.scale() === scale + 1) + val parquetFilters = createParquetFilters(parquetSchema) assertResult(Some(lt(intColumn("cdecimal1"), 1000: Integer))) { - parquetFilters.createFilter(parquetSchema, sources.LessThan("cdecimal1", decimal)) + parquetFilters.createFilter(sources.LessThan("cdecimal1", decimal)) } assertResult(None) { - parquetFilters.createFilter(parquetSchema, sources.LessThan("cdecimal1", decimal1)) + parquetFilters.createFilter(sources.LessThan("cdecimal1", decimal1)) } assertResult(Some(lt(longColumn("cdecimal2"), 1000L: java.lang.Long))) { - parquetFilters.createFilter(parquetSchema, sources.LessThan("cdecimal2", decimal)) + parquetFilters.createFilter(sources.LessThan("cdecimal2", decimal)) } assertResult(None) { - parquetFilters.createFilter(parquetSchema, sources.LessThan("cdecimal2", decimal1)) + parquetFilters.createFilter(sources.LessThan("cdecimal2", decimal1)) } - assert(parquetFilters.createFilter( - parquetSchema, sources.LessThan("cdecimal3", decimal)).isDefined) + assert(parquetFilters.createFilter(sources.LessThan("cdecimal3", decimal)).isDefined) assertResult(None) { - parquetFilters.createFilter(parquetSchema, sources.LessThan("cdecimal3", decimal1)) + parquetFilters.createFilter(sources.LessThan("cdecimal3", decimal1)) } } @@ -758,13 +729,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex )) val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) - + val parquetFilters = createParquetFilters(parquetSchema) assertResult(Some(and( lt(intColumn("a"), 10: Integer), gt(doubleColumn("c"), 1.5: java.lang.Double))) ) { parquetFilters.createFilter( - parquetSchema, sources.And( sources.LessThan("a", 10), sources.GreaterThan("c", 1.5D))) @@ -776,7 +746,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // case (Some(lhsFilter), None) if canRemoveOneSideInAnd => Some(lhsFilter) assertResult(Some(lt(intColumn("a"), 10: Integer))) { parquetFilters.createFilter( - parquetSchema, sources.And( sources.LessThan("a", 10), sources.StringContains("b", "prefix"))) @@ -788,7 +757,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // case (None, Some(rhsFilter)) if canRemoveOneSideInAnd => Some(rhsFilter) assertResult(Some(lt(intColumn("a"), 10: Integer))) { parquetFilters.createFilter( - parquetSchema, sources.And( sources.StringContains("b", "prefix"), sources.LessThan("a", 10))) @@ -798,7 +766,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex assertResult(Some( FilterApi.and(lt(intColumn("a"), 10: Integer), gt(intColumn("a"), 5: Integer)))) { parquetFilters.createFilter( - parquetSchema, sources.And( sources.And( sources.LessThan("a", 10), @@ -811,7 +778,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex assertResult(Some( FilterApi.and(gt(intColumn("a"), 5: Integer), lt(intColumn("a"), 10: Integer)))) { parquetFilters.createFilter( - parquetSchema, sources.And( sources.GreaterThan("a", 5), sources.And( @@ -820,34 +786,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex ))) } - // Testing - // case sources.Or(lhs, rhs) => - // ... - // lhsFilter <- createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd = false) - assertResult(None) { - parquetFilters.createFilter( - parquetSchema, - sources.Or( - sources.And( - sources.GreaterThan("a", 1), - sources.StringContains("b", "prefix")), - sources.GreaterThan("a", 2))) - } - - // Testing - // case sources.Or(lhs, rhs) => - // ... - // rhsFilter <- createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd = false) - assertResult(None) { - parquetFilters.createFilter( - parquetSchema, - sources.Or( - sources.GreaterThan("a", 2), - sources.And( - sources.GreaterThan("a", 1), - sources.StringContains("b", "prefix")))) - } - // Testing // case sources.Not(pred) => // createFilterHelper(nameToParquetField, pred, canRemoveOneSideInAnd = false) @@ -861,7 +799,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // case (Some(lhsFilter), None) if canRemoveOneSideInAnd => Some(lhsFilter) assertResult(None) { parquetFilters.createFilter( - parquetSchema, sources.Not( sources.And( sources.GreaterThan("a", 1), @@ -881,7 +818,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // case (None, Some(rhsFilter)) if canRemoveOneSideInAnd => Some(rhsFilter) assertResult(None) { parquetFilters.createFilter( - parquetSchema, sources.Not( sources.And( sources.StringContains("b", "prefix"), @@ -900,7 +836,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // val lhsFilterOption = createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd) assertResult(None) { parquetFilters.createFilter( - parquetSchema, sources.Not( sources.And( sources.And( @@ -921,7 +856,6 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex // val rhsFilterOption = createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd) assertResult(None) { parquetFilters.createFilter( - parquetSchema, sources.Not( sources.And( sources.GreaterThan("a", 2), @@ -931,6 +865,183 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } + test("SPARK-27699 Converting disjunctions into Parquet filter predicates") { + val schema = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = true), + StructField("c", DoubleType, nullable = true) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + // Testing + // case sources.Or(lhs, rhs) => + // ... + // lhsFilter <- createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd = true) + assertResult(Some( + FilterApi.or(gt(intColumn("a"), 1: Integer), gt(intColumn("a"), 2: Integer)))) { + parquetFilters.createFilter( + sources.Or( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")), + sources.GreaterThan("a", 2))) + } + + // Testing + // case sources.Or(lhs, rhs) => + // ... + // rhsFilter <- createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd = true) + assertResult(Some( + FilterApi.or(gt(intColumn("a"), 2: Integer), gt(intColumn("a"), 1: Integer)))) { + parquetFilters.createFilter( + sources.Or( + sources.GreaterThan("a", 2), + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")))) + } + + // Testing + // case sources.Or(lhs, rhs) => + // ... + // lhsFilter <- createFilterHelper(nameToParquetField, lhs, canRemoveOneSideInAnd = true) + // rhsFilter <- createFilterHelper(nameToParquetField, rhs, canRemoveOneSideInAnd = true) + assertResult(Some( + FilterApi.or(gt(intColumn("a"), 1: Integer), lt(intColumn("a"), 0: Integer)))) { + parquetFilters.createFilter( + sources.Or( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")), + sources.And( + sources.LessThan("a", 0), + sources.StringContains("b", "foobar")))) + } + } + + test("SPARK-27698 Convertible Parquet filter predicates") { + val schema = StructType(Seq( + StructField("a", IntegerType, nullable = false), + StructField("b", StringType, nullable = true), + StructField("c", DoubleType, nullable = true) + )) + + val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) + val parquetFilters = createParquetFilters(parquetSchema) + assertResult(Seq(sources.And(sources.LessThan("a", 10), sources.GreaterThan("c", 1.5D)))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.LessThan("a", 10), + sources.GreaterThan("c", 1.5D)))) + } + + assertResult(Seq(sources.LessThan("a", 10))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.LessThan("a", 10), + sources.StringContains("b", "prefix")))) + } + + assertResult(Seq(sources.LessThan("a", 10))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.StringContains("b", "prefix"), + sources.LessThan("a", 10)))) + } + + // Testing complex And conditions + assertResult(Seq(sources.And(sources.LessThan("a", 10), sources.GreaterThan("a", 5)))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.And( + sources.LessThan("a", 10), + sources.StringContains("b", "prefix") + ), + sources.GreaterThan("a", 5)))) + } + + // Testing complex And conditions + assertResult(Seq(sources.And(sources.GreaterThan("a", 5), sources.LessThan("a", 10)))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.GreaterThan("a", 5), + sources.And( + sources.StringContains("b", "prefix"), + sources.LessThan("a", 10) + )))) + } + + // Testing complex And conditions + assertResult(Seq(sources.Or(sources.GreaterThan("a", 1), sources.GreaterThan("a", 2)))) { + parquetFilters.convertibleFilters( + Seq(sources.Or( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")), + sources.GreaterThan("a", 2)))) + } + + // Testing complex And/Or conditions, the And condition under Or condition can't be pushed down. + assertResult(Seq(sources.And(sources.LessThan("a", 10), + sources.Or(sources.GreaterThan("a", 1), sources.GreaterThan("a", 2))))) { + parquetFilters.convertibleFilters( + Seq(sources.And( + sources.LessThan("a", 10), + sources.Or( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")), + sources.GreaterThan("a", 2))))) + } + + assertResult(Seq(sources.Or(sources.GreaterThan("a", 2), sources.GreaterThan("c", 1.1)))) { + parquetFilters.convertibleFilters( + Seq(sources.Or( + sources.GreaterThan("a", 2), + sources.And( + sources.GreaterThan("c", 1.1), + sources.StringContains("b", "prefix"))))) + } + + // Testing complex Not conditions. + assertResult(Seq.empty) { + parquetFilters.convertibleFilters( + Seq(sources.Not( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix"))))) + } + + assertResult(Seq.empty) { + parquetFilters.convertibleFilters( + Seq(sources.Not( + sources.And( + sources.StringContains("b", "prefix"), + sources.GreaterThan("a", 1))))) + } + + assertResult(Seq.empty) { + parquetFilters.convertibleFilters( + Seq(sources.Not( + sources.And( + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")), + sources.GreaterThan("a", 2))))) + } + + assertResult(Seq.empty) { + parquetFilters.convertibleFilters( + Seq(sources.Not( + sources.And( + sources.GreaterThan("a", 2), + sources.And( + sources.GreaterThan("a", 1), + sources.StringContains("b", "prefix")))))) + } + } + 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 => // Here the schema becomes as below: @@ -1074,30 +1185,37 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex checkFilterPredicate( !'_1.startsWith("").asInstanceOf[Predicate], - classOf[UserDefinedByInstance[_, _]], + classOf[Operators.Not], Seq().map(Row(_))) Seq("2", "2s", "2st", "2str", "2str2").foreach { prefix => checkFilterPredicate( !'_1.startsWith(prefix).asInstanceOf[Predicate], - classOf[UserDefinedByInstance[_, _]], + classOf[Operators.Not], Seq("1str1", "3str3", "4str4").map(Row(_))) } Seq("2S", "null", "2str22").foreach { prefix => checkFilterPredicate( !'_1.startsWith(prefix).asInstanceOf[Predicate], - classOf[UserDefinedByInstance[_, _]], + classOf[Operators.Not], Seq("1str1", "2str2", "3str3", "4str4").map(Row(_))) } + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) assertResult(None) { - parquetFilters.createFilter( - new SparkToParquetSchemaConverter(conf).convert(df.schema), - sources.StringStartsWith("_1", null)) + createParquetFilters(schema).createFilter(sources.StringStartsWith("_1", null)) } } + // SPARK-28371: make sure filter is null-safe. + withParquetDataFrame(Seq(Tuple1[String](null))) { implicit df => + checkFilterPredicate( + '_1.startsWith("blah").asInstanceOf[Predicate], + classOf[UserDefinedByInstance[_, _]], + Seq.empty[Row]) + } + import testImplicits._ // Test canDrop() has taken effect testStringStartsWith(spark.range(1024).map(_.toString).toDF(), "value like 'a%'") @@ -1111,18 +1229,18 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex )) val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) - + val parquetFilters = createParquetFilters(parquetSchema) assertResult(Some(FilterApi.eq(intColumn("a"), null: Integer))) { - parquetFilters.createFilter(parquetSchema, sources.In("a", Array(null))) + parquetFilters.createFilter(sources.In("a", Array(null))) } assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { - parquetFilters.createFilter(parquetSchema, sources.In("a", Array(10))) + parquetFilters.createFilter(sources.In("a", Array(10))) } // Remove duplicates assertResult(Some(FilterApi.eq(intColumn("a"), 10: Integer))) { - parquetFilters.createFilter(parquetSchema, sources.In("a", Array(10, 10))) + parquetFilters.createFilter(sources.In("a", Array(10, 10))) } assertResult(Some(or(or( @@ -1130,12 +1248,12 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex FilterApi.eq(intColumn("a"), 20: Integer)), FilterApi.eq(intColumn("a"), 30: Integer))) ) { - parquetFilters.createFilter(parquetSchema, sources.In("a", Array(10, 20, 30))) + parquetFilters.createFilter(sources.In("a", Array(10, 20, 30))) } - assert(parquetFilters.createFilter(parquetSchema, sources.In("a", + assert(parquetFilters.createFilter(sources.In("a", Range(0, conf.parquetFilterPushDownInFilterThreshold).toArray)).isDefined) - assert(parquetFilters.createFilter(parquetSchema, sources.In("a", + assert(parquetFilters.createFilter(sources.In("a", Range(0, conf.parquetFilterPushDownInFilterThreshold + 1).toArray)).isEmpty) import testImplicits._ @@ -1167,25 +1285,20 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } test("SPARK-25207: Case-insensitive field resolution for pushdown when reading parquet") { - def createParquetFilter(caseSensitive: Boolean): ParquetFilters = { - new ParquetFilters(conf.parquetFilterPushDownDate, conf.parquetFilterPushDownTimestamp, - conf.parquetFilterPushDownDecimal, conf.parquetFilterPushDownStringStartWith, - conf.parquetFilterPushDownInFilterThreshold, caseSensitive) - } - val caseSensitiveParquetFilters = createParquetFilter(caseSensitive = true) - val caseInsensitiveParquetFilters = createParquetFilter(caseSensitive = false) - def testCaseInsensitiveResolution( schema: StructType, expected: FilterPredicate, filter: sources.Filter): Unit = { val parquetSchema = new SparkToParquetSchemaConverter(conf).convert(schema) - + val caseSensitiveParquetFilters = + createParquetFilters(parquetSchema, caseSensitive = Some(true)) + val caseInsensitiveParquetFilters = + createParquetFilters(parquetSchema, caseSensitive = Some(false)) assertResult(Some(expected)) { - caseInsensitiveParquetFilters.createFilter(parquetSchema, filter) + caseInsensitiveParquetFilters.createFilter(filter) } assertResult(None) { - caseSensitiveParquetFilters.createFilter(parquetSchema, filter) + caseSensitiveParquetFilters.createFilter(filter) } } @@ -1242,9 +1355,10 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex val dupFieldSchema = StructType( Seq(StructField("cint", IntegerType), StructField("cINT", IntegerType))) val dupParquetSchema = new SparkToParquetSchemaConverter(conf).convert(dupFieldSchema) + val dupCaseInsensitiveParquetFilters = + createParquetFilters(dupParquetSchema, caseSensitive = Some(false)) assertResult(None) { - caseInsensitiveParquetFilters.createFilter( - dupParquetSchema, sources.EqualTo("CINT", 1000)) + dupCaseInsensitiveParquetFilters.createFilter(sources.EqualTo("CINT", 1000)) } } @@ -1279,6 +1393,127 @@ class ParquetFilterSuite extends QueryTest with ParquetTest with SharedSQLContex } } +class ParquetV1FilterSuite extends ParquetFilterSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "parquet") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "parquet") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + var maybeRelation: Option[HadoopFsRelation] = None + val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { + case PhysicalOperation(_, filters, + LogicalRelation(relation: HadoopFsRelation, _, _, _)) => + maybeRelation = Some(relation) + filters + }.flatten.reduceLeftOption(_ && _) + assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") + + val (_, selectedFilters, _) = + DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) + assert(selectedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(selectedFilters) === selectedFilters) + val pushedParquetFilters = selectedFilters.map { pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert(pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") + + checker(stripSparkFilter(query), expected) + } + } +} + +class ParquetV2FilterSuite extends ParquetFilterSuite { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + + override def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + filterClass: Class[_ <: FilterPredicate], + checker: (DataFrame, Seq[Row]) => Unit, + expected: Seq[Row]): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + + withSQLConf( + SQLConf.PARQUET_FILTER_PUSHDOWN_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DATE_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_TIMESTAMP_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_DECIMAL_ENABLED.key -> "true", + SQLConf.PARQUET_FILTER_PUSHDOWN_STRING_STARTSWITH_ENABLED.key -> "true", + // Disable adding filters from constraints because it adds, for instance, + // is-not-null to pushed filters, which makes it hard to test if the pushed + // filter is expected or not (this had to be fixed with SPARK-13495). + SQLConf.OPTIMIZER_EXCLUDED_RULES.key -> InferFiltersFromConstraints.ruleName, + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan.collectFirst { + case PhysicalOperation(_, filters, + DataSourceV2Relation(parquetTable: ParquetTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = parquetTable.newScanBuilder(options) + val sourceFilters = filters.flatMap(DataSourceStrategy.translateFilter).toArray + scanBuilder.pushFilters(sourceFilters) + val pushedFilters = scanBuilder.pushedFilters() + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val schema = new SparkToParquetSchemaConverter(conf).convert(df.schema) + val parquetFilters = createParquetFilters(schema) + // In this test suite, all the simple predicates are convertible here. + assert(parquetFilters.convertibleFilters(sourceFilters) === pushedFilters) + val pushedParquetFilters = pushedFilters.map { pred => + val maybeFilter = parquetFilters.createFilter(pred) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pred") + maybeFilter.get + } + // Doesn't bother checking type parameters here (e.g. `Eq[Integer]`) + assert(pushedParquetFilters.exists(_.getClass === filterClass), + s"${pushedParquetFilters.map(_.getClass).toList} did not contain ${filterClass}.") + + checker(stripSparkFilter(query), expected) + + case _ => + throw new AnalysisException("Can not match ParquetTable in the query.") + } + } + } +} + class NumRowGroupsAcc extends AccumulatorV2[Integer, Integer] { private var _sum = 0 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 002c42f23bd64..6f2218ba82dc8 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 @@ -27,6 +27,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.mapreduce.{JobContext, TaskAttemptContext} +import org.apache.parquet.HadoopReadOptions import org.apache.parquet.column.{Encoding, ParquetProperties} import org.apache.parquet.example.data.{Group, GroupWriter} import org.apache.parquet.example.data.simple.SimpleGroup @@ -34,10 +35,11 @@ import org.apache.parquet.hadoop._ import org.apache.parquet.hadoop.api.WriteSupport import org.apache.parquet.hadoop.api.WriteSupport.WriteContext import org.apache.parquet.hadoop.metadata.CompressionCodecName +import org.apache.parquet.hadoop.util.HadoopInputFile import org.apache.parquet.io.api.RecordConsumer import org.apache.parquet.schema.{MessageType, MessageTypeParser} -import org.apache.spark.SparkException +import org.apache.spark.{SPARK_VERSION_SHORT, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, ScalaReflection} import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeRow} @@ -473,7 +475,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { val extraOptions = Map( SQLConf.OUTPUT_COMMITTER_CLASS.key -> classOf[ParquetOutputCommitter].getCanonicalName, - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[JobCommitFailureParquetOutputCommitter].getCanonicalName ) withTempPath { dir => @@ -503,7 +505,7 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { // Using a output committer that always fail when committing a task, so that both // `commitTask()` and `abortTask()` are invoked. val extraOptions = Map[String, String]( - "spark.sql.parquet.output.committer.class" -> + SQLConf.PARQUET_OUTPUT_COMMITTER_CLASS.key -> classOf[TaskCommitFailureParquetOutputCommitter].getCanonicalName ) @@ -799,6 +801,23 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext { checkAnswer(spark.read.parquet(file.getAbsolutePath), Seq(Row(Row(1, null, "foo")))) } } + + test("Write Spark version into Parquet metadata") { + withTempPath { dir => + val path = dir.getAbsolutePath + spark.range(1).repartition(1).write.parquet(path) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + val conf = new Configuration() + val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), conf) + val parquetReadOptions = HadoopReadOptions.builder(conf).build() + val m = ParquetFileReader.open(hadoopInputFile, parquetReadOptions) + val metaData = m.getFileMetaData.getKeyValueMetaData + m.close() + + assert(metaData.get(SPARK_VERSION_METADATA_KEY) === SPARK_VERSION_SHORT) + } + } } class JobCommitFailureParquetOutputCommitter(outputPath: Path, context: TaskAttemptContext) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala index f06e1867151e5..edbc2493ac26d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetInteroperabilitySuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File -import scala.language.existentials - import org.apache.commons.io.FileUtils import org.apache.hadoop.fs.{FileSystem, Path, PathFilter} import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER @@ -120,8 +118,12 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS ).map { s => java.sql.Timestamp.valueOf(s) } import testImplicits._ // match the column names of the file from impala - val df = spark.createDataset(ts).toDF().repartition(1).withColumnRenamed("value", "ts") - df.write.parquet(tableDir.getAbsolutePath) + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> + SQLConf.ParquetOutputTimestampType.INT96.toString) { + val df = spark.createDataset(ts).toDF().repartition(1) + .withColumnRenamed("value", "ts") + df.write.parquet(tableDir.getAbsolutePath) + } FileUtils.copyFile(new File(impalaPath), new File(tableDir, "part-00001.parq")) Seq(false, true).foreach { int96TimestampConversion => @@ -182,12 +184,11 @@ class ParquetInteroperabilitySuite extends ParquetCompatibilityTest with SharedS assert(typeName === PrimitiveTypeName.INT96) val oneBlockMeta = oneFooter.getBlocks().get(0) val oneBlockColumnMeta = oneBlockMeta.getColumns().get(0) - val columnStats = oneBlockColumnMeta.getStatistics // This is the important assert. Column stats are written, but they are ignored // when the data is read back as mentioned above, b/c int96 is unsigned. This // assert makes sure this holds even if we change parquet versions (if eg. there // were ever statistics even on unsigned columns). - assert(!columnStats.hasNonNullValue) + assert(!oneBlockColumnMeta.getStatistics.hasNonNullValue) } // These queries should return the entire dataset with the conversion applied, 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 9966ed94a8392..04ace0a236e6d 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File import java.math.BigInteger import java.sql.{Date, Timestamp} +import java.time.{ZoneId, ZoneOffset} import java.util.{Calendar, Locale, TimeZone} import scala.collection.mutable.ArrayBuffer @@ -28,15 +29,16 @@ import com.google.common.io.Files import org.apache.hadoop.fs.Path import org.apache.parquet.hadoop.ParquetOutputFormat +import org.apache.spark.SparkConf import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.ExternalCatalogUtils import org.apache.spark.sql.catalyst.expressions.Literal -import org.apache.spark.sql.catalyst.util.DateTimeUtils +import org.apache.spark.sql.catalyst.util.{DateFormatter, DateTimeUtils, TimestampFormatter} import org.apache.spark.sql.execution.datasources._ import org.apache.spark.sql.execution.datasources.{PartitionPath => Partition} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, FileTable} import org.apache.spark.sql.execution.streaming.MemoryStream -import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -48,14 +50,16 @@ case class ParquetData(intField: Int, stringField: String) // The data that also includes the partitioning key case class ParquetDataWithKey(intField: Int, pi: Int, stringField: String, ps: String) -class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with SharedSQLContext { +abstract class ParquetPartitionDiscoverySuite + extends QueryTest with ParquetTest with SharedSQLContext { import PartitioningUtils._ import testImplicits._ val defaultPartitionName = ExternalCatalogUtils.DEFAULT_PARTITION_NAME - val timeZone = TimeZone.getDefault() - val timeZoneId = timeZone.getID + val timeZoneId = ZoneId.systemDefault() + val df = DateFormatter() + val tf = TimestampFormatter(timestampPartitionPattern, timeZoneId) protected override def beforeAll(): Unit = { super.beforeAll() @@ -68,8 +72,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } test("column type inference") { - def check(raw: String, literal: Literal, timeZone: TimeZone = timeZone): Unit = { - assert(inferPartitionColumnValue(raw, true, timeZone) === literal) + def check(raw: String, literal: Literal, zoneId: ZoneId = timeZoneId): Unit = { + assert(inferPartitionColumnValue(raw, true, zoneId, df, tf) === literal) } check("10", Literal.create(10, IntegerType)) @@ -88,7 +92,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha c.set(Calendar.MILLISECOND, 0) check("1990-02-24 12:00:30", Literal.create(new Timestamp(c.getTimeInMillis), TimestampType), - TimeZone.getTimeZone("GMT")) + ZoneOffset.UTC) check(defaultPartitionName, Literal.create(null, NullType)) } @@ -101,7 +105,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha "hdfs://host:9000/path/a=10.5/b=hello") var exception = intercept[AssertionError] { - parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), true, Set.empty[Path], None, true, true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -115,6 +119,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, + true, timeZoneId) // Valid @@ -128,6 +135,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/something=true/table")), + None, + true, + true, timeZoneId) // Valid @@ -141,6 +151,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/table=true")), + None, + true, + true, timeZoneId) // Invalid @@ -154,6 +167,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/path/")), + None, + true, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -174,6 +190,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, Set(new Path("hdfs://host:9000/tmp/tables/")), + None, + true, + true, timeZoneId) } assert(exception.getMessage().contains("Conflicting directory structures detected")) @@ -181,13 +200,14 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partition") { def check(path: String, expected: Option[PartitionValues]): Unit = { - val actual = parsePartition(new Path(path), true, Set.empty[Path], timeZone)._1 + val actual = parsePartition(new Path(path), true, Set.empty[Path], + Map.empty, true, timeZoneId, df, tf)._1 assert(expected === actual) } def checkThrows[T <: Throwable: Manifest](path: String, expected: String): Unit = { val message = intercept[T] { - parsePartition(new Path(path), true, Set.empty[Path], timeZone) + parsePartition(new Path(path), true, Set.empty[Path], Map.empty, true, timeZoneId, df, tf) }.getMessage assert(message.contains(expected)) @@ -231,7 +251,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path/a=10")), - timeZone = timeZone)._1 + Map.empty, + true, + zoneId = timeZoneId, + df, + tf)._1 assert(partitionSpec1.isEmpty) @@ -240,7 +264,11 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha path = new Path("file://path/a=10"), typeInference = true, basePaths = Set(new Path("file://path")), - timeZone = timeZone)._1 + Map.empty, + true, + zoneId = timeZoneId, + df, + tf)._1 assert(partitionSpec2 == Option(PartitionValues( @@ -258,6 +286,9 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha paths.map(new Path(_)), true, rootPaths, + None, + true, + true, timeZoneId) assert(actualSpec.partitionColumns === spec.partitionColumns) assert(actualSpec.partitions.length === spec.partitions.length) @@ -370,7 +401,8 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha test("parse partitions with type inference disabled") { def check(paths: Seq[String], spec: PartitionSpec): Unit = { val actualSpec = - parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], timeZoneId) + parsePartitions(paths.map(new Path(_)), false, Set.empty[Path], None, + true, true, timeZoneId) assert(actualSpec === spec) } @@ -540,53 +572,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } - test("read partitioned table - partition key included in Parquet file") { - withTempDir { base => - for { - pi <- Seq(1, 2) - ps <- Seq("foo", "bar") - } { - makeParquetFile( - (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), - makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) - } - - spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") - - withTempView("t") { - checkAnswer( - sql("SELECT * FROM t"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - ps <- Seq("foo", "bar") - } yield Row(i, pi, i.toString, ps)) - - checkAnswer( - sql("SELECT intField, pi FROM t"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - _ <- Seq("foo", "bar") - } yield Row(i, pi)) - - checkAnswer( - sql("SELECT * FROM t WHERE pi = 1"), - for { - i <- 1 to 10 - ps <- Seq("foo", "bar") - } yield Row(i, 1, i.toString, ps)) - - checkAnswer( - sql("SELECT * FROM t WHERE ps = 'foo'"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - } yield Row(i, pi, i.toString, "foo")) - } - } - } - test("read partitioned table - with nulls") { withTempDir { base => for { @@ -628,39 +613,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } - test("read partitioned table - with nulls and partition keys are included in Parquet file") { - withTempDir { base => - for { - pi <- Seq(1, 2) - ps <- Seq("foo", null.asInstanceOf[String]) - } { - makeParquetFile( - (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), - makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) - } - - val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) - parquetRelation.createOrReplaceTempView("t") - - withTempView("t") { - checkAnswer( - sql("SELECT * FROM t"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - ps <- Seq("foo", null.asInstanceOf[String]) - } yield Row(i, pi, i.toString, ps)) - - checkAnswer( - sql("SELECT * FROM t WHERE ps IS NULL"), - for { - i <- 1 to 10 - pi <- Seq(1, 2) - } yield Row(i, pi, i.toString, null)) - } - } - } - test("read partitioned table - merging compatible schemas") { withTempDir { base => makeParquetFile( @@ -686,20 +638,6 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } - test("SPARK-7749 Non-partitioned table should have empty partition spec") { - withTempPath { dir => - (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) - val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution - queryExecution.analyzed.collectFirst { - case LogicalRelation( - HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _, _) => - assert(location.partitionSpec() === PartitionSpec.emptySpec) - }.getOrElse { - fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") - } - } - } - test("SPARK-7847: Dynamic partition directory path escaping and unescaping") { withTempPath { dir => val df = Seq("/", "[]", "?").zipWithIndex.map(_.swap).toDF("i", "s") @@ -1015,7 +953,7 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha withSQLConf( ParquetOutputFormat.JOB_SUMMARY_LEVEL -> "ALL", - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[SQLHadoopMapReduceCommitProtocol].getCanonicalName) { spark.range(3).write.parquet(s"$path/p0=0/p1=0") } @@ -1054,6 +992,158 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } + test("SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") { + val df = Seq( + (1, "2015-01-01 00:00:00"), + (2, "2014-01-01 00:00:00"), + (3, "blah")).toDF("i", "str") + + withTempPath { path => + df.write.format("parquet").partitionBy("str").save(path.getAbsolutePath) + checkAnswer(spark.read.load(path.getAbsolutePath), df) + } + } + + test("Resolve type conflicts - decimals, dates and timestamps in partition column") { + withTempPath { path => + val df = Seq((1, "2014-01-01"), (2, "2016-01-01"), (3, "2015-01-01 00:01:00")).toDF("i", "ts") + df.write.format("parquet").partitionBy("ts").save(path.getAbsolutePath) + checkAnswer( + spark.read.load(path.getAbsolutePath), + Row(1, Timestamp.valueOf("2014-01-01 00:00:00")) :: + Row(2, Timestamp.valueOf("2016-01-01 00:00:00")) :: + Row(3, Timestamp.valueOf("2015-01-01 00:01:00")) :: Nil) + } + + withTempPath { path => + val df = Seq((1, "1"), (2, "3"), (3, "2" * 30)).toDF("i", "decimal") + df.write.format("parquet").partitionBy("decimal").save(path.getAbsolutePath) + checkAnswer( + spark.read.load(path.getAbsolutePath), + Row(1, BigDecimal("1")) :: + Row(2, BigDecimal("3")) :: + Row(3, BigDecimal("2" * 30)) :: Nil) + } + } + + test("SPARK-23436: invalid Dates should be inferred as String in partition inference") { + withTempPath { path => + val data = Seq(("1", "2018-01", "2018-01-01-04", "test")) + .toDF("id", "date_month", "date_hour", "data") + + data.write.partitionBy("date_month", "date_hour").parquet(path.getAbsolutePath) + val input = spark.read.parquet(path.getAbsolutePath).select("id", + "date_month", "date_hour", "data") + + assert(input.schema.sameType(input.schema)) + checkAnswer(input, data) + } + } +} + +class ParquetV1PartitionDiscoverySuite extends ParquetPartitionDiscoverySuite { + import testImplicits._ + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "parquet") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "parquet") + + test("read partitioned table - partition key included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") + + withTempView("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, pi, i.toString, ps)) + + checkAnswer( + sql("SELECT intField, pi FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + _ <- Seq("foo", "bar") + } yield Row(i, pi)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, 1, i.toString, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, pi, i.toString, "foo")) + } + } + } + + test("read partitioned table - with nulls and partition keys are included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) + parquetRelation.createOrReplaceTempView("t") + + withTempView("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, pi, i.toString, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps IS NULL"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, pi, i.toString, null)) + } + } + } + + test("SPARK-7749 Non-partitioned table should have empty partition spec") { + withTempPath { dir => + (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) + val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution + queryExecution.analyzed.collectFirst { + case LogicalRelation( + HadoopFsRelation(location: PartitioningAwareFileIndex, _, _, _, _, _), _, _, _) => + assert(location.partitionSpec() === PartitionSpec.emptySpec) + }.getOrElse { + fail(s"Expecting a matching HadoopFsRelation, but got:\n$queryExecution") + } + } + } + test("SPARK-18108 Parquet reader fails when data column types conflict with partition ones") { withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { withTempPath { dir => @@ -1096,52 +1186,150 @@ class ParquetPartitionDiscoverySuite extends QueryTest with ParquetTest with Sha } } } +} - test("SPARK-22109: Resolve type conflicts between strings and timestamps in partition column") { - val df = Seq( - (1, "2015-01-01 00:00:00"), - (2, "2014-01-01 00:00:00"), - (3, "blah")).toDF("i", "str") +class ParquetV2PartitionDiscoverySuite extends ParquetPartitionDiscoverySuite { + import testImplicits._ - withTempPath { path => - df.write.format("parquet").partitionBy("str").save(path.getAbsolutePath) - checkAnswer(spark.read.load(path.getAbsolutePath), df) + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + + test("read partitioned table - partition key included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + spark.read.parquet(base.getCanonicalPath).createOrReplaceTempView("t") + + withTempView("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( + sql("SELECT intField, pi FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + _ <- Seq("foo", "bar") + } yield Row(i, pi)) + + checkAnswer( + sql("SELECT * FROM t WHERE pi = 1"), + for { + i <- 1 to 10 + ps <- Seq("foo", "bar") + } yield Row(i, i.toString, 1, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps = 'foo'"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, i.toString, pi, "foo")) + } } } - test("Resolve type conflicts - decimals, dates and timestamps in partition column") { - withTempPath { path => - val df = Seq((1, "2014-01-01"), (2, "2016-01-01"), (3, "2015-01-01 00:01:00")).toDF("i", "ts") - df.write.format("parquet").partitionBy("ts").save(path.getAbsolutePath) - checkAnswer( - spark.read.load(path.getAbsolutePath), - Row(1, Timestamp.valueOf("2014-01-01 00:00:00")) :: - Row(2, Timestamp.valueOf("2016-01-01 00:00:00")) :: - Row(3, Timestamp.valueOf("2015-01-01 00:01:00")) :: Nil) + test("read partitioned table - with nulls and partition keys are included in Parquet file") { + withTempDir { base => + for { + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } { + makeParquetFile( + (1 to 10).map(i => ParquetDataWithKey(i, pi, i.toString, ps)), + makePartitionDir(base, defaultPartitionName, "pi" -> pi, "ps" -> ps)) + } + + val parquetRelation = spark.read.format("parquet").load(base.getCanonicalPath) + parquetRelation.createOrReplaceTempView("t") + + withTempView("t") { + checkAnswer( + sql("SELECT * FROM t"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + ps <- Seq("foo", null.asInstanceOf[String]) + } yield Row(i, i.toString, pi, ps)) + + checkAnswer( + sql("SELECT * FROM t WHERE ps IS NULL"), + for { + i <- 1 to 10 + pi <- Seq(1, 2) + } yield Row(i, i.toString, pi, null)) + } } + } - withTempPath { path => - val df = Seq((1, "1"), (2, "3"), (3, "2" * 30)).toDF("i", "decimal") - df.write.format("parquet").partitionBy("decimal").save(path.getAbsolutePath) - checkAnswer( - spark.read.load(path.getAbsolutePath), - Row(1, BigDecimal("1")) :: - Row(2, BigDecimal("3")) :: - Row(3, BigDecimal("2" * 30)) :: Nil) + test("SPARK-7749 Non-partitioned table should have empty partition spec") { + withTempPath { dir => + (1 to 10).map(i => (i, i.toString)).toDF("a", "b").write.parquet(dir.getCanonicalPath) + val queryExecution = spark.read.parquet(dir.getCanonicalPath).queryExecution + queryExecution.analyzed.collectFirst { + case DataSourceV2Relation(fileTable: FileTable, _, _) => + assert(fileTable.fileIndex.partitionSpec() === PartitionSpec.emptySpec) + }.getOrElse { + fail(s"Expecting a matching DataSourceV2Relation, but got:\n$queryExecution") + } } } - test("SPARK-23436: invalid Dates should be inferred as String in partition inference") { - withTempPath { path => - val data = Seq(("1", "2018-01", "2018-01-01-04", "test")) - .toDF("id", "date_month", "date_hour", "data") + test("SPARK-18108 Parquet reader fails when data column types conflict with partition ones") { + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = Seq((1L, 2.0)).toDF("a", "b") + df.write.parquet(s"$path/a=1") + checkAnswer(spark.read.parquet(s"$path"), Seq(Row(2.0, 1))) + } + } + } - data.write.partitionBy("date_month", "date_hour").parquet(path.getAbsolutePath) - val input = spark.read.parquet(path.getAbsolutePath).select("id", - "date_month", "date_hour", "data") + test("SPARK-21463: MetadataLogFileIndex should respect userSpecifiedSchema for partition cols") { + withTempDir { tempDir => + val output = new File(tempDir, "output").toString + val checkpoint = new File(tempDir, "chkpoint").toString + try { + val stream = MemoryStream[(String, Int)] + val df = stream.toDS().toDF("time", "value") + val sq = df.writeStream + .option("checkpointLocation", checkpoint) + .format("parquet") + .partitionBy("time") + .start(output) - assert(input.schema.sameType(input.schema)) - checkAnswer(input, data) + stream.addData(("2017-01-01-00", 1), ("2017-01-01-01", 2)) + sq.processAllAvailable() + + val schema = new StructType() + .add("time", StringType) + .add("value", IntegerType) + val readBack = spark.read.schema(schema).parquet(output) + assert(readBack.schema.toSet === schema.toSet) + + checkAnswer( + readBack, + Seq(Row(1, "2017-01-01-00"), Row(2, "2017-01-01-01")) + ) + } finally { + spark.streams.active.foreach(_.stop()) + } } } -} +} \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala index 54c77dddc3525..a6429bfc52924 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala @@ -18,18 +18,20 @@ package org.apache.spark.sql.execution.datasources.parquet import java.io.File -import java.sql.Timestamp +import java.util.concurrent.TimeUnit import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.parquet.hadoop.ParquetOutputFormat -import org.apache.spark.{DebugFilesystem, SparkException} +import org.apache.spark.{DebugFilesystem, SparkConf, SparkException} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.SpecificInternalRow import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.SQLHadoopMapReduceCommitProtocol import org.apache.spark.sql.execution.datasources.parquet.TestingUDT.{NestedStruct, NestedStructUDT, SingleElement} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -38,7 +40,7 @@ import org.apache.spark.util.Utils /** * A test suite that tests various Parquet queries. */ -class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext { +abstract class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext { import testImplicits._ test("simple select queries") { @@ -71,30 +73,6 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext TableIdentifier("tmp"), ignoreIfNotExists = true, purge = false) } - test("SPARK-15678: not use cache on overwrite") { - withTempDir { dir => - val path = dir.toString - spark.range(1000).write.mode("overwrite").parquet(path) - val df = spark.read.parquet(path).cache() - assert(df.count() == 1000) - spark.range(10).write.mode("overwrite").parquet(path) - assert(df.count() == 10) - assert(spark.read.parquet(path).count() == 10) - } - } - - test("SPARK-15678: not use cache on append") { - withTempDir { dir => - val path = dir.toString - spark.range(1000).write.mode("append").parquet(path) - val df = spark.read.parquet(path).cache() - assert(df.count() == 1000) - spark.range(10).write.mode("append").parquet(path) - assert(df.count() == 1010) - assert(spark.read.parquet(path).count() == 1010) - } - } - test("self-join") { // 4 rows, cells of column 1 of row 2 and row 4 are null val data = (1 to 4).map { i => @@ -187,12 +165,12 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("insert into ts values (1, '2016-01-01 10:11:12.123456')") sql("insert into ts values (2, null)") sql("insert into ts values (3, '1965-01-01 10:11:12.123456')") - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123456")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.123456")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123456"), + (2, null), + (3, "1965-01-01 10:11:12.123456")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) } // The microsecond portion is truncated when written as TIMESTAMP_MILLIS. @@ -206,30 +184,30 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext sql("insert into ts values (5, '1965-01-01 10:11:12.1')") sql("insert into ts values (6, '1965-01-01 10:11:12.123456789')") sql("insert into ts values (7, '0001-01-01 00:00:00.000000')") - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), - Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), - Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123"), + (2, null), + (3, "1965-01-01 10:11:12.125"), + (4, "1965-01-01 10:11:12.125"), + (5, "1965-01-01 10:11:12.1"), + (6, "1965-01-01 10:11:12.123"), + (7, "0001-01-01 00:00:00.000")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) // Read timestamps that were encoded as TIMESTAMP_MILLIS annotated as INT64 // with PARQUET_INT64_AS_TIMESTAMP_MILLIS set to false. withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "false") { - checkAnswer( - sql("select * from ts"), - Seq( - Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")), - Row(2, null), - Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")), - Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")), - Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")), - Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000")))) + val expected = Seq( + (1, "2016-01-01 10:11:12.123"), + (2, null), + (3, "1965-01-01 10:11:12.125"), + (4, "1965-01-01 10:11:12.125"), + (5, "1965-01-01 10:11:12.1"), + (6, "1965-01-01 10:11:12.123"), + (7, "0001-01-01 00:00:00.000")) + .toDS().select('_1, $"_2".cast("timestamp")) + checkAnswer(sql("select * from ts"), expected) } } } @@ -791,29 +769,6 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext assert(ParquetReadSupport.expandUDT(schema) === expected) } - test("returning batch for wide table") { - withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { - withTempPath { dir => - val path = dir.getCanonicalPath - val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) - df.write.mode(SaveMode.Overwrite).parquet(path) - - // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) - val df2 = spark.read.parquet(path) - val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(!fileScan2.asInstanceOf[FileSourceScanExec].supportsBatch) - checkAnswer(df2, df) - - // return batch - val columns = Seq.tabulate(9) {i => s"c$i"} - val df3 = df2.selectExpr(columns : _*) - val fileScan3 = df3.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get - assert(fileScan3.asInstanceOf[FileSourceScanExec].supportsBatch) - checkAnswer(df3, df.selectExpr(columns : _*)) - } - } - } - test("SPARK-15719: disable writing summary files by default") { withTempPath { dir => val path = dir.getCanonicalPath @@ -891,6 +846,133 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext } } } + + test("SPARK-26677: negated null-safe equality comparison should not filter matched row groups") { + (true :: false :: Nil).foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { + withTempPath { path => + // Repeated values for dictionary encoding. + Seq(Some("A"), Some("A"), None).toDF.repartition(1) + .write.parquet(path.getAbsolutePath) + val df = spark.read.parquet(path.getAbsolutePath) + checkAnswer(stripSparkFilter(df.where("NOT (value <=> 'A')")), df) + } + } + } + } + + test("Migration from INT96 to TIMESTAMP_MICROS timestamp type") { + def testMigration(fromTsType: String, toTsType: String): Unit = { + def checkAppend(write: DataFrameWriter[_] => Unit, readback: => DataFrame): Unit = { + def data(start: Int, end: Int): Seq[Row] = (start to end).map { i => + val ts = new java.sql.Timestamp(TimeUnit.SECONDS.toMillis(i)) + ts.setNanos(123456000) + Row(ts) + } + val schema = new StructType().add("time", TimestampType) + val df1 = spark.createDataFrame(sparkContext.parallelize(data(0, 1)), schema) + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> fromTsType) { + write(df1.write) + } + val df2 = spark.createDataFrame(sparkContext.parallelize(data(2, 10)), schema) + withSQLConf(SQLConf.PARQUET_OUTPUT_TIMESTAMP_TYPE.key -> toTsType) { + write(df2.write.mode(SaveMode.Append)) + } + Seq("true", "false").foreach { vectorized => + withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) { + checkAnswer(readback, df1.unionAll(df2)) + } + } + } + + Seq(false, true).foreach { mergeSchema => + withTempPath { file => + checkAppend(_.parquet(file.getCanonicalPath), + spark.read.option("mergeSchema", mergeSchema).parquet(file.getCanonicalPath)) + } + + withSQLConf(SQLConf.PARQUET_SCHEMA_MERGING_ENABLED.key -> mergeSchema.toString) { + val tableName = "parquet_timestamp_migration" + withTable(tableName) { + checkAppend(_.saveAsTable(tableName), spark.table(tableName)) + } + } + } + } + + testMigration(fromTsType = "INT96", toTsType = "TIMESTAMP_MICROS") + testMigration(fromTsType = "TIMESTAMP_MICROS", toTsType = "INT96") + } +} + +class ParquetV1QuerySuite extends ParquetQuerySuite { + import testImplicits._ + + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "parquet") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "parquet") + + test("returning batch for wide table") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) + df.write.mode(SaveMode.Overwrite).parquet(path) + + // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) + val df2 = spark.read.parquet(path) + val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + assert(!fileScan2.asInstanceOf[FileSourceScanExec].supportsColumnar) + checkAnswer(df2, df) + + // return batch + val columns = Seq.tabulate(9) {i => s"c$i"} + val df3 = df2.selectExpr(columns : _*) + val fileScan3 = df3.queryExecution.sparkPlan.find(_.isInstanceOf[FileSourceScanExec]).get + assert(fileScan3.asInstanceOf[FileSourceScanExec].supportsColumnar) + checkAnswer(df3, df.selectExpr(columns : _*)) + } + } + } +} + +class ParquetV2QuerySuite extends ParquetQuerySuite { + import testImplicits._ + + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + + test("returning batch for wide table") { + withSQLConf(SQLConf.WHOLESTAGE_MAX_NUM_FIELDS.key -> "10") { + withTempPath { dir => + val path = dir.getCanonicalPath + val df = spark.range(10).select(Seq.tabulate(11) {i => ('id + i).as(s"c$i")} : _*) + df.write.mode(SaveMode.Overwrite).parquet(path) + + // donot return batch, because whole stage codegen is disabled for wide table (>200 columns) + val df2 = spark.read.parquet(path) + val fileScan2 = df2.queryExecution.sparkPlan.find(_.isInstanceOf[BatchScanExec]).get + val parquetScan2 = fileScan2.asInstanceOf[BatchScanExec].scan.asInstanceOf[ParquetScan] + // The method `supportColumnarReads` in Parquet doesn't depends on the input partition. + // Here we can pass null input partition to the method for testing propose. + assert(!parquetScan2.createReaderFactory().supportColumnarReads(null)) + checkAnswer(df2, df) + + // return batch + val columns = Seq.tabulate(9) {i => s"c$i"} + val df3 = df2.selectExpr(columns : _*) + val fileScan3 = df3.queryExecution.sparkPlan.find(_.isInstanceOf[BatchScanExec]).get + val parquetScan3 = fileScan3.asInstanceOf[BatchScanExec].scan.asInstanceOf[ParquetScan] + assert(parquetScan3.createReaderFactory().supportColumnarReads(null)) + checkAnswer(df3, df.selectExpr(columns : _*)) + } + } + } } object TestingUDT { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala index 434c4414edeba..70184b609f2d4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaPruningSuite.scala @@ -17,358 +17,40 @@ package org.apache.spark.sql.execution.datasources.parquet -import java.io.File - -import org.scalactic.Equality - -import org.apache.spark.sql.{DataFrame, QueryTest, Row} -import org.apache.spark.sql.catalyst.SchemaPruningTest +import org.apache.spark.SparkConf +import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.execution.FileSourceScanExec +import org.apache.spark.sql.execution.datasources.SchemaPruningSuite +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SharedSQLContext -import org.apache.spark.sql.types.StructType - -class ParquetSchemaPruningSuite - extends QueryTest - with ParquetTest - with SchemaPruningTest - with SharedSQLContext { - case class FullName(first: String, middle: String, last: String) - case class Company(name: String, address: String) - case class Employer(id: Int, company: Company) - case class Contact( - id: Int, - name: FullName, - address: String, - pets: Int, - friends: Array[FullName] = Array.empty, - relatives: Map[String, FullName] = Map.empty, - employer: Employer = null) - - val janeDoe = FullName("Jane", "X.", "Doe") - val johnDoe = FullName("John", "Y.", "Doe") - val susanSmith = FullName("Susan", "Z.", "Smith") - - val employer = Employer(0, Company("abc", "123 Business Street")) - val employerWithNullCompany = Employer(1, null) - - private val contacts = - Contact(0, janeDoe, "123 Main Street", 1, friends = Array(susanSmith), - relatives = Map("brother" -> johnDoe), employer = employer) :: - Contact(1, johnDoe, "321 Wall Street", 3, relatives = Map("sister" -> janeDoe), - employer = employerWithNullCompany) :: Nil - - case class Name(first: String, last: String) - case class BriefContact(id: Int, name: Name, address: String) - - private val briefContacts = - BriefContact(2, Name("Janet", "Jones"), "567 Maple Drive") :: - BriefContact(3, Name("Jim", "Jones"), "6242 Ash Street") :: Nil - - case class ContactWithDataPartitionColumn( - id: Int, - name: FullName, - address: String, - pets: Int, - friends: Array[FullName] = Array(), - relatives: Map[String, FullName] = Map(), - employer: Employer = null, - p: Int) - - case class BriefContactWithDataPartitionColumn(id: Int, name: Name, address: String, p: Int) - - private val contactsWithDataPartitionColumn = - contacts.map { case Contact(id, name, address, pets, friends, relatives, employer) => - ContactWithDataPartitionColumn(id, name, address, pets, friends, relatives, employer, 1) } - private val briefContactsWithDataPartitionColumn = - briefContacts.map { case BriefContact(id, name, address) => - BriefContactWithDataPartitionColumn(id, name, address, 2) } - - testSchemaPruning("select a single complex field") { - val query = sql("select name.middle from contacts") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), Row("X.") :: Row("Y.") :: Row(null) :: Row(null) :: Nil) - } - - testSchemaPruning("select a single complex field and its parent struct") { - val query = sql("select name.middle, name from contacts") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row("X.", Row("Jane", "X.", "Doe")) :: - Row("Y.", Row("John", "Y.", "Doe")) :: - Row(null, Row("Janet", null, "Jones")) :: - Row(null, Row("Jim", null, "Jones")) :: - Nil) - } - - testSchemaPruning("select a single complex field array and its parent struct array") { - val query = sql("select friends.middle, friends from contacts where p=1") - checkScan(query, - "struct>>") - checkAnswer(query.orderBy("id"), - Row(Array("Z."), Array(Row("Susan", "Z.", "Smith"))) :: - Row(Array.empty[String], Array.empty[Row]) :: - Nil) - } - - testSchemaPruning("select a single complex field from a map entry and its parent map entry") { - val query = - sql("select relatives[\"brother\"].middle, relatives[\"brother\"] from contacts where p=1") - checkScan(query, - "struct>>") - checkAnswer(query.orderBy("id"), - Row("Y.", Row("John", "Y.", "Doe")) :: - Row(null, null) :: - Nil) - } - - testSchemaPruning("select a single complex field and the partition column") { - val query = sql("select name.middle, p from contacts") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row("X.", 1) :: Row("Y.", 1) :: Row(null, 2) :: Row(null, 2) :: Nil) - } - - ignore("partial schema intersection - select missing subfield") { - val query = sql("select name.middle, address from contacts where p=2") - checkScan(query, "struct,address:string>") - checkAnswer(query.orderBy("id"), - Row(null, "567 Maple Drive") :: - Row(null, "6242 Ash Street") :: Nil) - } - testSchemaPruning("no unnecessary schema pruning") { - val query = - sql("select id, name.last, name.middle, name.first, relatives[''].last, " + - "relatives[''].middle, relatives[''].first, friends[0].last, friends[0].middle, " + - "friends[0].first, pets, address from contacts where p=2") - // We've selected every field in the schema. Therefore, no schema pruning should be performed. - // We check this by asserting that the scanned schema of the query is identical to the schema - // of the contacts relation, even though the fields are selected in different orders. - checkScan(query, - "struct,address:string,pets:int," + - "friends:array>," + - "relatives:map>>") - checkAnswer(query.orderBy("id"), - Row(2, "Jones", null, "Janet", null, null, null, null, null, null, null, "567 Maple Drive") :: - Row(3, "Jones", null, "Jim", null, null, null, null, null, null, null, "6242 Ash Street") :: - Nil) - } - - testSchemaPruning("empty schema intersection") { - val query = sql("select name.middle from contacts where p=2") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row(null) :: Row(null) :: Nil) - } - - testSchemaPruning("select a single complex field and in where clause") { - val query1 = sql("select name.first from contacts where name.first = 'Jane'") - checkScan(query1, "struct>") - checkAnswer(query1, Row("Jane") :: Nil) - - val query2 = sql("select name.first, name.last from contacts where name.first = 'Jane'") - checkScan(query2, "struct>") - checkAnswer(query2, Row("Jane", "Doe") :: Nil) - - val query3 = sql("select name.first from contacts " + - "where employer.company.name = 'abc' and p = 1") - checkScan(query3, "struct," + - "employer:struct>>") - checkAnswer(query3, Row("Jane") :: Nil) - - val query4 = sql("select name.first, employer.company.name from contacts " + - "where employer.company is not null and p = 1") - checkScan(query4, "struct," + - "employer:struct>>") - checkAnswer(query4, Row("Jane", "abc") :: Nil) - } - - testSchemaPruning("select nullable complex field and having is not null predicate") { - val query = sql("select employer.company from contacts " + - "where employer is not null and p = 1") - checkScan(query, "struct>>") - checkAnswer(query, Row(Row("abc", "123 Business Street")) :: Row(null) :: Nil) - } - - testSchemaPruning("select a single complex field and is null expression in project") { - val query = sql("select name.first, address is not null from contacts") - checkScan(query, "struct,address:string>") - checkAnswer(query.orderBy("id"), - Row("Jane", true) :: Row("John", true) :: Row("Janet", true) :: Row("Jim", true) :: Nil) - } +abstract class ParquetSchemaPruningSuite extends SchemaPruningSuite { + override protected val dataSourceName: String = "parquet" + override protected val vectorizedReaderEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key - testSchemaPruning("select a single complex field array and in clause") { - val query = sql("select friends.middle from contacts where friends.first[0] = 'Susan'") - checkScan(query, - "struct>>") - checkAnswer(query.orderBy("id"), - Row(Array("Z.")) :: Nil) - } - - testSchemaPruning("select a single complex field from a map entry and in clause") { - val query = - sql("select relatives[\"brother\"].middle from contacts " + - "where relatives[\"brother\"].first = 'John'") - checkScan(query, - "struct>>") - checkAnswer(query.orderBy("id"), - Row("Y.") :: Nil) - } - - private def testSchemaPruning(testName: String)(testThunk: => Unit) { - test(s"Spark vectorized reader - without partition data column - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - withContacts(testThunk) - } - } - test(s"Spark vectorized reader - with partition data column - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true") { - withContactsWithDataPartitionColumn(testThunk) - } - } - - test(s"Parquet-mr reader - without partition data column - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - withContacts(testThunk) - } - } - test(s"Parquet-mr reader - with partition data column - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false") { - withContactsWithDataPartitionColumn(testThunk) - } - } - } - - private def withContacts(testThunk: => Unit) { - withTempPath { dir => - val path = dir.getCanonicalPath - - makeParquetFile(contacts, new File(path + "/contacts/p=1")) - makeParquetFile(briefContacts, new File(path + "/contacts/p=2")) - - spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") - - testThunk - } - } - - private def withContactsWithDataPartitionColumn(testThunk: => Unit) { - withTempPath { dir => - val path = dir.getCanonicalPath - - makeParquetFile(contactsWithDataPartitionColumn, new File(path + "/contacts/p=1")) - makeParquetFile(briefContactsWithDataPartitionColumn, new File(path + "/contacts/p=2")) - - spark.read.parquet(path + "/contacts").createOrReplaceTempView("contacts") - - testThunk - } - } - - case class MixedCaseColumn(a: String, B: Int) - case class MixedCase(id: Int, CoL1: String, coL2: MixedCaseColumn) - - private val mixedCaseData = - MixedCase(0, "r0c1", MixedCaseColumn("abc", 1)) :: - MixedCase(1, "r1c1", MixedCaseColumn("123", 2)) :: - Nil - - testExactCaseQueryPruning("select with exact column names") { - val query = sql("select CoL1, coL2.B from mixedcase") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row("r0c1", 1) :: - Row("r1c1", 2) :: - Nil) - } - - testMixedCaseQueryPruning("select with lowercase column names") { - val query = sql("select col1, col2.b from mixedcase") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row("r0c1", 1) :: - Row("r1c1", 2) :: - Nil) - } - - testMixedCaseQueryPruning("select with different-case column names") { - val query = sql("select cOL1, cOl2.b from mixedcase") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), - Row("r0c1", 1) :: - Row("r1c1", 2) :: - Nil) - } - - testMixedCaseQueryPruning("filter with different-case column names") { - val query = sql("select id from mixedcase where Col2.b = 2") - checkScan(query, "struct>") - checkAnswer(query.orderBy("id"), Row(1) :: Nil) - } - - // Tests schema pruning for a query whose column and field names are exactly the same as the table - // schema's column and field names. N.B. this implies that `testThunk` should pass using either a - // case-sensitive or case-insensitive query parser - private def testExactCaseQueryPruning(testName: String)(testThunk: => Unit) { - test(s"Spark vectorized reader - case-sensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "true") { - withMixedCaseData(testThunk) - } - } - test(s"Parquet-mr reader - case-sensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "true") { - withMixedCaseData(testThunk) - } - } - testMixedCaseQueryPruning(testName)(testThunk) - } - - // Tests schema pruning for a query whose column and field names may differ in case from the table - // schema's column and field names - private def testMixedCaseQueryPruning(testName: String)(testThunk: => Unit) { - test(s"Spark vectorized reader - case-insensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "true", - SQLConf.CASE_SENSITIVE.key -> "false") { - withMixedCaseData(testThunk) - } - } - test(s"Parquet-mr reader - case-insensitive parser - mixed-case schema - $testName") { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> "false", - SQLConf.CASE_SENSITIVE.key -> "false") { - withMixedCaseData(testThunk) - } - } - } - - private def withMixedCaseData(testThunk: => Unit) { - withParquetTable(mixedCaseData, "mixedcase") { - testThunk - } - } +} - private val schemaEquality = new Equality[StructType] { - override def areEqual(a: StructType, b: Any): Boolean = - b match { - case otherType: StructType => a.sameType(otherType) - case _ => false - } - } +class ParquetV1SchemaPruningSuite extends ParquetSchemaPruningSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "parquet") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "parquet") +} - protected def checkScan(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { - checkScanSchemata(df, expectedSchemaCatalogStrings: _*) - // We check here that we can execute the query without throwing an exception. The results - // themselves are irrelevant, and should be checked elsewhere as needed - df.collect() - } +class ParquetV2SchemaPruningSuite extends ParquetSchemaPruningSuite { + // TODO: enable Parquet V2 write path after file source V2 writers are workable. + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") - private def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { + override def checkScanSchemata(df: DataFrame, expectedSchemaCatalogStrings: String*): Unit = { val fileSourceScanSchemata = df.queryExecution.executedPlan.collect { - case scan: FileSourceScanExec => scan.requiredSchema + case scan: BatchScanExec => scan.scan.asInstanceOf[ParquetScan].readDataSchema } assert(fileSourceScanSchemata.size === expectedSchemaCatalogStrings.size, s"Found ${fileSourceScanSchemata.size} file sources in dataframe, " + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala index 528a4d0ca8004..f8e4822af11f1 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala @@ -23,7 +23,7 @@ import scala.reflect.runtime.universe.TypeTag import org.apache.parquet.io.ParquetDecodingException import org.apache.parquet.schema.{MessageType, MessageTypeParser} -import org.apache.spark.SparkException +import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.datasources.SchemaColumnConvertNotSupportedException 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 f05f5722af51a..828ba6aee026b 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 @@ -30,9 +30,9 @@ import org.apache.parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter} import org.apache.parquet.hadoop.metadata.{BlockMetaData, FileMetaData, ParquetMetadata} import org.apache.parquet.schema.MessageType -import org.apache.spark.sql.{DataFrame, SaveMode} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.datasources.FileBasedDataSourceTest import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType /** @@ -42,21 +42,17 @@ import org.apache.spark.sql.types.StructType * convenient to use tuples rather than special case classes when writing test cases/suites. * Especially, `Tuple1.apply` can be used to easily wrap a single type/value. */ -private[sql] trait ParquetTest extends SQLTestUtils { +private[sql] trait ParquetTest extends FileBasedDataSourceTest { + + override protected val dataSourceName: String = "parquet" + override protected val vectorizedReaderEnabledKey: String = + SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key /** * Reads the parquet file at `path` */ protected def readParquetFile(path: String, testVectorized: Boolean = true) - (f: DataFrame => Unit) = { - (true :: false :: Nil).foreach { vectorized => - if (!vectorized || testVectorized) { - withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized.toString) { - f(spark.read.parquet(path.toString)) - } - } - } - } + (f: DataFrame => Unit) = readFile(path, testVectorized)(f) /** * Writes `data` to a Parquet file, which is then passed to `f` and will be deleted after `f` @@ -64,12 +60,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { */ protected def withParquetFile[T <: Product: ClassTag: TypeTag] (data: Seq[T]) - (f: String => Unit): Unit = { - withTempPath { file => - spark.createDataFrame(data).write.parquet(file.getCanonicalPath) - f(file.getCanonicalPath) - } - } + (f: String => Unit): Unit = withDataSourceFile(data)(f) /** * Writes `data` to a Parquet file and reads it back as a [[DataFrame]], @@ -77,9 +68,7 @@ private[sql] trait ParquetTest extends SQLTestUtils { */ protected def withParquetDataFrame[T <: Product: ClassTag: TypeTag] (data: Seq[T], testVectorized: Boolean = true) - (f: DataFrame => Unit): Unit = { - withParquetFile(data)(path => readParquetFile(path.toString, testVectorized)(f)) - } + (f: DataFrame => Unit): Unit = withDataSourceDataFrame(data, testVectorized)(f) /** * Writes `data` to a Parquet file, reads it back as a [[DataFrame]] and registers it as a @@ -88,22 +77,13 @@ private[sql] trait ParquetTest extends SQLTestUtils { */ protected def withParquetTable[T <: Product: ClassTag: TypeTag] (data: Seq[T], tableName: String, testVectorized: Boolean = true) - (f: => Unit): Unit = { - withParquetDataFrame(data, testVectorized) { df => - df.createOrReplaceTempView(tableName) - withTempView(tableName)(f) - } - } + (f: => Unit): Unit = withDataSourceTable(data, tableName, testVectorized)(f) protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( - data: Seq[T], path: File): Unit = { - spark.createDataFrame(data).write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) - } + data: Seq[T], path: File): Unit = makeDataSourceFile(data, path) protected def makeParquetFile[T <: Product: ClassTag: TypeTag]( - df: DataFrame, path: File): Unit = { - df.write.mode(SaveMode.Overwrite).parquet(path.getCanonicalPath) - } + df: DataFrame, path: File): Unit = makeDataSourceFile(df, path) protected def makePartitionDir( basePath: File, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala index 0e7f3afa9c3ab..a86d5ee37f3db 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/text/TextSuite.scala @@ -233,4 +233,13 @@ class TextSuite extends QueryTest with SharedSQLContext { assert(data(3) == Row("\"doh\"")) assert(data.length == 4) } + + test("do not produce empty files for empty partitions") { + withTempPath { dir => + val path = dir.getCanonicalPath + spark.emptyDataset[String].write.text(path) + val files = new File(path).listFiles() + assert(!files.exists(_.getName.endsWith("txt"))) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala new file mode 100644 index 0000000000000..ac1d5672af68c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/FileTableSuite.scala @@ -0,0 +1,88 @@ +/* + * 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.v2 + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.fs.FileStatus + +import org.apache.spark.sql.{QueryTest, SparkSession} +import org.apache.spark.sql.execution.datasources.FileFormat +import org.apache.spark.sql.execution.datasources.text.TextFileFormat +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.test.{SharedSQLContext, SQLTestUtils} +import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class DummyFileTable( + sparkSession: SparkSession, + options: CaseInsensitiveStringMap, + paths: Seq[String], + expectedDataSchema: StructType, + userSpecifiedSchema: Option[StructType]) + extends FileTable(sparkSession, options, paths, userSpecifiedSchema) { + override def inferSchema(files: Seq[FileStatus]): Option[StructType] = Some(expectedDataSchema) + + override def name(): String = "Dummy" + + override def formatName: String = "Dummy" + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = null + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = null + + override def supportsDataType(dataType: DataType): Boolean = dataType == StringType + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[TextFileFormat] +} + +class FileTableSuite extends QueryTest with SharedSQLContext with SQLTestUtils { + + test("Data type validation should check data schema only") { + withTempPath { dir => + val df = spark.createDataFrame(Seq(("a", 1), ("b", 2))).toDF("v", "p") + val pathName = dir.getCanonicalPath + df.write.partitionBy("p").text(pathName) + val options = new CaseInsensitiveStringMap(Map("path" -> pathName).asJava) + val expectedDataSchema = StructType(Seq(StructField("v", StringType, true))) + // DummyFileTable doesn't support Integer data type. + // However, the partition schema is handled by Spark, so it is allowed to contain + // Integer data type here. + val table = new DummyFileTable(spark, options, Seq(pathName), expectedDataSchema, None) + assert(table.dataSchema == expectedDataSchema) + val expectedPartitionSchema = StructType(Seq(StructField("p", IntegerType, true))) + assert(table.fileIndex.partitionSchema == expectedPartitionSchema) + } + } + + test("Returns correct data schema when user specified schema contains partition schema") { + withTempPath { dir => + val df = spark.createDataFrame(Seq(("a", 1), ("b", 2))).toDF("v", "p") + val pathName = dir.getCanonicalPath + df.write.partitionBy("p").text(pathName) + val options = new CaseInsensitiveStringMap(Map("path" -> pathName).asJava) + val userSpecifiedSchema = Some(StructType(Seq( + StructField("v", StringType, true), + StructField("p", IntegerType, true)))) + val expectedDataSchema = StructType(Seq(StructField("v", StringType, true))) + val table = + new DummyFileTable(spark, options, Seq(pathName), expectedDataSchema, userSpecifiedSchema) + assert(table.dataSchema == expectedDataSchema) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala new file mode 100644 index 0000000000000..3822882cc91cb --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2SessionCatalogSuite.scala @@ -0,0 +1,683 @@ +/* + * 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.v2 + +import java.util +import java.util.Collections + +import scala.collection.JavaConverters._ + +import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll} + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.catalog.v2.{Catalogs, Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.catalyst.parser.CatalystSqlParser +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{DoubleType, IntegerType, LongType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class V2SessionCatalogSuite + extends SparkFunSuite with SharedSQLContext with BeforeAndAfter with BeforeAndAfterAll { + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + private val emptyProps: util.Map[String, String] = Collections.emptyMap[String, String] + private val schema: StructType = new StructType() + .add("id", IntegerType) + .add("data", StringType) + + override protected def beforeAll(): Unit = { + super.beforeAll() + spark.sql("""CREATE DATABASE IF NOT EXISTS db""") + spark.sql("""CREATE DATABASE IF NOT EXISTS ns""") + spark.sql("""CREATE DATABASE IF NOT EXISTS ns2""") + } + + override protected def afterAll(): Unit = { + spark.sql("""DROP TABLE IF EXISTS db.test_table""") + spark.sql("""DROP DATABASE IF EXISTS db""") + spark.sql("""DROP DATABASE IF EXISTS ns""") + spark.sql("""DROP DATABASE IF EXISTS ns2""") + super.afterAll() + } + + after { + newCatalog().dropTable(testIdent) + } + + private def newCatalog(): TableCatalog = { + val newCatalog = new V2SessionCatalog(spark.sessionState) + newCatalog.initialize("test", CaseInsensitiveStringMap.empty()) + newCatalog + } + + private val testIdent = Identifier.of(Array("db"), "test_table") + + test("Catalogs can load the catalog") { + val catalog = newCatalog() + + val conf = new SQLConf + conf.setConfString("spark.sql.catalog.test", catalog.getClass.getName) + + val loaded = Catalogs.load("test", conf) + assert(loaded.getClass == catalog.getClass) + } + + test("listTables") { + val catalog = newCatalog() + val ident1 = Identifier.of(Array("ns"), "test_table_1") + val ident2 = Identifier.of(Array("ns"), "test_table_2") + val ident3 = Identifier.of(Array("ns2"), "test_table_1") + + assert(catalog.listTables(Array("ns")).isEmpty) + + catalog.createTable(ident1, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1)) + assert(catalog.listTables(Array("ns2")).isEmpty) + + catalog.createTable(ident3, schema, Array.empty, emptyProps) + catalog.createTable(ident2, schema, Array.empty, emptyProps) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident1, ident2)) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident1) + + assert(catalog.listTables(Array("ns")).toSet == Set(ident2)) + + catalog.dropTable(ident2) + + assert(catalog.listTables(Array("ns")).isEmpty) + assert(catalog.listTables(Array("ns2")).toSet == Set(ident3)) + + catalog.dropTable(ident3) + } + + test("createTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("db", "test_table")) + assert(table.schema == schema) + assert(table.properties.asScala == Map()) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: with properties") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("property", "value") + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + val parsed = CatalystSqlParser.parseMultipartIdentifier(table.name) + assert(parsed == Seq("db", "test_table")) + assert(table.schema == schema) + assert(table.properties == properties) + + assert(catalog.tableExists(testIdent)) + } + + test("createTable: table already exists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + val exc = intercept[TableAlreadyExistsException] { + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + } + + assert(exc.message.contains(table.name())) + assert(exc.message.contains("already exists")) + + assert(catalog.tableExists(testIdent)) + } + + test("tableExists") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + catalog.dropTable(testIdent) + + assert(!catalog.tableExists(testIdent)) + } + + test("loadTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("loadTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.loadTable(testIdent) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("invalidateTable") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + catalog.invalidateTable(testIdent) + + val loaded = catalog.loadTable(testIdent) + + assert(table.name == loaded.name) + assert(table.schema == loaded.schema) + assert(table.properties == loaded.properties) + } + + test("invalidateTable: table does not exist") { + val catalog = newCatalog() + + assert(catalog.tableExists(testIdent) === false) + + catalog.invalidateTable(testIdent) + } + + test("alterTable: add property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-1", "1")) + assert(updated.properties.asScala == Map("prop-1" -> "1")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1")) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add property to existing") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.setProperty("prop-2", "2")) + assert(updated.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map("prop-1" -> "1", "prop-2" -> "2")) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove existing property") { + val catalog = newCatalog() + + val properties = new util.HashMap[String, String]() + properties.put("prop-1", "1") + + val table = catalog.createTable(testIdent, schema, Array.empty, properties) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map("prop-1" -> "1")) + } + + test("alterTable: remove missing property") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.properties.asScala == Map()) + + val updated = catalog.alterTable(testIdent, TableChange.removeProperty("prop-1")) + assert(updated.properties.asScala == Map()) + + val loaded = catalog.loadTable(testIdent) + assert(loaded.properties.asScala == Map()) + + assert(table.properties.asScala == Map()) + } + + test("alterTable: add top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.addColumn(Array("ts"), TimestampType)) + + assert(updated.schema == schema.add("ts", TimestampType)) + } + + test("alterTable: add required column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false)) + + assert(updated.schema == schema.add("ts", TimestampType, nullable = false)) + } + + test("alterTable: add column with comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("ts"), TimestampType, false, "comment text")) + + val field = StructField("ts", TimestampType, nullable = false).withComment("comment text") + assert(updated.schema == schema.add(field)) + } + + test("alterTable: add nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.addColumn(Array("point", "z"), DoubleType)) + + val expectedSchema = schema.add("point", pointStruct.add("z", DoubleType)) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add column to primitive field fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.addColumn(Array("data", "ts"), TimestampType)) + } + + assert(exc.getMessage.contains("Not a struct")) + assert(exc.getMessage.contains("data")) + + // the table has not changed + assert(catalog.loadTable(testIdent).schema == schema) + } + + test("alterTable: add field to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.addColumn(Array("missing_col", "new_field"), StringType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: update column data type") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update column data type and nullability") { + val catalog = newCatalog() + + val originalSchema = new StructType() + .add("id", IntegerType, nullable = false) + .add("data", StringType) + val table = catalog.createTable(testIdent, originalSchema, Array.empty, emptyProps) + + assert(table.schema == originalSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("id"), LongType, true)) + + val expectedSchema = new StructType().add("id", LongType).add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: update optional column to required fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.updateColumnType(Array("id"), LongType, false)) + } + + assert(exc.getMessage.contains("Cannot change optional column to required")) + assert(exc.getMessage.contains("id")) + } + + test("alterTable: update missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnType(Array("missing_col"), LongType)) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: add comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "comment text") + .add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: replace comment") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + catalog.alterTable(testIdent, TableChange.updateColumnComment(Array("id"), "comment text")) + + val expectedSchema = new StructType() + .add("id", IntegerType, nullable = true, "replacement comment") + .add("data", StringType) + + val updated = catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("id"), "replacement comment")) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: add comment to missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.updateColumnComment(Array("missing_col"), "comment")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: rename top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, TableChange.renameColumn(Array("id"), "some_id")) + + val expectedSchema = new StructType().add("some_id", IntegerType).add("data", StringType) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first")) + + val newPointStruct = new StructType().add("first", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename struct column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point"), "p")) + + val newPointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val expectedSchema = schema.add("p", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: rename missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, + TableChange.renameColumn(Array("missing_col"), "new_name")) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: multiple changes") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.renameColumn(Array("point", "x"), "first"), + TableChange.renameColumn(Array("point", "y"), "second")) + + val newPointStruct = new StructType().add("first", DoubleType).add("second", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete top-level column") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("id"))) + + val expectedSchema = new StructType().add("data", StringType) + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete nested column") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val updated = catalog.alterTable(testIdent, + TableChange.deleteColumn(Array("point", "y"))) + + val newPointStruct = new StructType().add("x", DoubleType) + val expectedSchema = schema.add("point", newPointStruct) + + assert(updated.schema == expectedSchema) + } + + test("alterTable: delete missing column fails") { + val catalog = newCatalog() + + val table = catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(table.schema == schema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("missing_col"))) + } + + assert(exc.getMessage.contains("missing_col")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: delete missing nested column fails") { + val catalog = newCatalog() + + val pointStruct = new StructType().add("x", DoubleType).add("y", DoubleType) + val tableSchema = schema.add("point", pointStruct) + + val table = catalog.createTable(testIdent, tableSchema, Array.empty, emptyProps) + + assert(table.schema == tableSchema) + + val exc = intercept[IllegalArgumentException] { + catalog.alterTable(testIdent, TableChange.deleteColumn(Array("point", "z"))) + } + + assert(exc.getMessage.contains("z")) + assert(exc.getMessage.contains("Cannot find")) + } + + test("alterTable: table does not exist") { + val catalog = newCatalog() + + val exc = intercept[NoSuchTableException] { + catalog.alterTable(testIdent, TableChange.setProperty("prop", "val")) + } + + assert(exc.message.contains(testIdent.quoted)) + assert(exc.message.contains("not found")) + } + + test("dropTable") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + catalog.createTable(testIdent, schema, Array.empty, emptyProps) + + assert(catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(wasDropped) + assert(!catalog.tableExists(testIdent)) + } + + test("dropTable: table does not exist") { + val catalog = newCatalog() + + assert(!catalog.tableExists(testIdent)) + + val wasDropped = catalog.dropTable(testIdent) + + assert(!wasDropped) + assert(!catalog.tableExists(testIdent)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala new file mode 100644 index 0000000000000..8a0450fce76a1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/v2/V2StreamingScanSupportCheckSuite.scala @@ -0,0 +1,130 @@ +/* + * 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.v2 + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.SparkFunSuite +import org.apache.spark.sql.{AnalysisException, DataFrame, SQLContext} +import org.apache.spark.sql.catalyst.plans.logical.Union +import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.streaming.{Offset, Source, StreamingRelation, StreamingRelationV2} +import org.apache.spark.sql.sources.StreamSourceProvider +import org.apache.spark.sql.sources.v2.{Table, TableCapability, TableProvider} +import org.apache.spark.sql.test.SharedSparkSession +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class V2StreamingScanSupportCheckSuite extends SparkFunSuite with SharedSparkSession { + import TableCapability._ + + private def createStreamingRelation(table: Table, v1Relation: Option[StreamingRelation]) = { + StreamingRelationV2(FakeTableProvider, "fake", table, CaseInsensitiveStringMap.empty(), + FakeTableProvider.schema.toAttributes, v1Relation)(spark) + } + + private def createStreamingRelationV1() = { + StreamingRelation(DataSource(spark, classOf[FakeStreamSourceProvider].getName)) + } + + test("check correct plan") { + val plan1 = createStreamingRelation(CapabilityTable(MICRO_BATCH_READ), None) + val plan2 = createStreamingRelation(CapabilityTable(CONTINUOUS_READ), None) + val plan3 = createStreamingRelation(CapabilityTable(MICRO_BATCH_READ, CONTINUOUS_READ), None) + val plan4 = createStreamingRelationV1() + + V2StreamingScanSupportCheck(Union(plan1, plan1)) + V2StreamingScanSupportCheck(Union(plan2, plan2)) + V2StreamingScanSupportCheck(Union(plan1, plan3)) + V2StreamingScanSupportCheck(Union(plan2, plan3)) + V2StreamingScanSupportCheck(Union(plan1, plan4)) + V2StreamingScanSupportCheck(Union(plan3, plan4)) + } + + test("table without scan capability") { + val e = intercept[AnalysisException] { + V2StreamingScanSupportCheck(createStreamingRelation(CapabilityTable(), None)) + } + assert(e.message.contains("does not support either micro-batch or continuous scan")) + } + + test("mix micro-batch only and continuous only") { + val plan1 = createStreamingRelation(CapabilityTable(MICRO_BATCH_READ), None) + val plan2 = createStreamingRelation(CapabilityTable(CONTINUOUS_READ), None) + + val e = intercept[AnalysisException] { + V2StreamingScanSupportCheck(Union(plan1, plan2)) + } + assert(e.message.contains( + "The streaming sources in a query do not have a common supported execution mode")) + } + + test("mix continuous only and v1 relation") { + val plan1 = createStreamingRelation(CapabilityTable(CONTINUOUS_READ), None) + val plan2 = createStreamingRelationV1() + val e = intercept[AnalysisException] { + V2StreamingScanSupportCheck(Union(plan1, plan2)) + } + assert(e.message.contains( + "The streaming sources in a query do not have a common supported execution mode")) + } +} + +private object FakeTableProvider extends TableProvider { + val schema = new StructType().add("i", "int") + + override def getTable(options: CaseInsensitiveStringMap): Table = { + throw new UnsupportedOperationException + } +} + +private case class CapabilityTable(_capabilities: TableCapability*) extends Table { + override def name(): String = "capability_test_table" + override def schema(): StructType = FakeTableProvider.schema + override def capabilities(): util.Set[TableCapability] = _capabilities.toSet.asJava +} + +private class FakeStreamSourceProvider extends StreamSourceProvider { + override def sourceSchema( + sqlContext: SQLContext, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): (String, StructType) = { + "fake" -> FakeTableProvider.schema + } + + override def createSource( + sqlContext: SQLContext, + metadataPath: String, + schema: Option[StructType], + providerName: String, + parameters: Map[String, String]): Source = { + new Source { + override def schema: StructType = FakeTableProvider.schema + override def getOffset: Option[Offset] = { + throw new UnsupportedOperationException + } + override def getBatch(start: Option[Offset], end: Offset): DataFrame = { + throw new UnsupportedOperationException + } + override def stop(): Unit = {} + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala index b4ad1db20a9ea..91cb919479bfa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/BroadcastJoinSuite.scala @@ -22,6 +22,7 @@ import scala.reflect.ClassTag import org.apache.spark.AccumulatorSuite import org.apache.spark.sql.{Dataset, QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.expressions.{BitwiseAnd, BitwiseOr, Cast, Literal, ShiftLeft} +import org.apache.spark.sql.catalyst.plans.logical.BROADCAST import org.apache.spark.sql.execution.{SparkPlan, WholeStageCodegenExec} import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.exchange.EnsureRequirements @@ -203,7 +204,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { } test("broadcast hint in SQL") { - import org.apache.spark.sql.catalyst.plans.logical.{ResolvedHint, Join} + import org.apache.spark.sql.catalyst.plans.logical.Join spark.range(10).createOrReplaceTempView("t") spark.range(10).createOrReplaceTempView("u") @@ -216,19 +217,19 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { val plan3 = sql(s"SELECT /*+ $name(v) */ * FROM t JOIN u ON t.id = u.id").queryExecution .optimizedPlan - assert(plan1.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(!plan1.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) - assert(!plan2.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(plan2.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) - assert(!plan3.asInstanceOf[Join].left.isInstanceOf[ResolvedHint]) - assert(!plan3.asInstanceOf[Join].right.isInstanceOf[ResolvedHint]) + assert(plan1.asInstanceOf[Join].hint.leftHint.get.strategy.contains(BROADCAST)) + assert(plan1.asInstanceOf[Join].hint.rightHint.isEmpty) + assert(plan2.asInstanceOf[Join].hint.leftHint.isEmpty) + assert(plan2.asInstanceOf[Join].hint.rightHint.get.strategy.contains(BROADCAST)) + assert(plan3.asInstanceOf[Join].hint.leftHint.isEmpty) + assert(plan3.asInstanceOf[Join].hint.rightHint.isEmpty) } } test("join key rewritten") { val l = Literal(1L) val i = Literal(2) - val s = Literal.create(3, ShortType) + val s = Literal.create(3.toShort, ShortType) val ss = Literal("hello") assert(HashJoin.rewriteKeyExpr(l :: Nil) === l :: Nil) @@ -280,13 +281,16 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { val t2Size = spark.table("t2").queryExecution.analyzed.children.head.stats.sizeInBytes assert(t1Size < t2Size) + /* ######## test cases for equal join ######### */ // INNER JOIN && t1Size < t2Size => BuildLeft assertJoinBuildSide( "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildLeft) // LEFT JOIN => BuildRight + // broadcast hash join can not build left side for left join. assertJoinBuildSide( "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 LEFT JOIN t2 ON t1.key = t2.key", bh, BuildRight) // RIGHT JOIN => BuildLeft + // broadcast hash join can not build right side for right join. assertJoinBuildSide( "SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key", bh, BuildLeft) // INNER JOIN && broadcast(t1) => BuildLeft @@ -296,16 +300,20 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { assertJoinBuildSide( "SELECT /*+ MAPJOIN(t2) */ * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildRight) - + /* ######## test cases for non-equal join ######### */ withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { // INNER JOIN && t1Size < t2Size => BuildLeft assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 JOIN t2", bl, BuildLeft) // FULL JOIN && t1Size < t2Size => BuildLeft assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 FULL JOIN t2", bl, BuildLeft) + // FULL OUTER && t1Size < t2Size => BuildLeft + assertJoinBuildSide("SELECT * FROM t1 FULL OUTER JOIN t2", bl, BuildLeft) // LEFT JOIN => BuildRight assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 LEFT JOIN t2", bl, BuildRight) // RIGHT JOIN => BuildLeft assertJoinBuildSide("SELECT /*+ MAPJOIN(t1, t2) */ * FROM t1 RIGHT JOIN t2", bl, BuildLeft) + + /* #### test with broadcast hint #### */ // INNER JOIN && broadcast(t1) => BuildLeft assertJoinBuildSide("SELECT /*+ MAPJOIN(t1) */ * FROM t1 JOIN t2", bl, BuildLeft) // INNER JOIN && broadcast(t2) => BuildRight @@ -315,8 +323,10 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { // FULL OUTER && broadcast(t2) => BuildRight assertJoinBuildSide( "SELECT /*+ MAPJOIN(t2) */ * FROM t1 FULL OUTER JOIN t2", bl, BuildRight) - // FULL OUTER && t1Size < t2Size => BuildLeft - assertJoinBuildSide("SELECT * FROM t1 FULL OUTER JOIN t2", bl, BuildLeft) + // LEFT JOIN && broadcast(t1) => BuildLeft + assertJoinBuildSide("SELECT /*+ MAPJOIN(t1) */ * FROM t1 LEFT JOIN t2", bl, BuildLeft) + // RIGHT JOIN && broadcast(t2) => BuildRight + assertJoinBuildSide("SELECT /*+ MAPJOIN(t2) */ * FROM t1 RIGHT JOIN t2", bl, BuildRight) } } } @@ -331,6 +341,7 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { val t2Size = spark.table("t2").queryExecution.analyzed.children.head.stats.sizeInBytes assert(t1Size < t2Size) + /* ######## test cases for equal join ######### */ assertJoinBuildSide("SELECT * FROM t1 JOIN t2 ON t1.key = t2.key", bh, BuildLeft) assertJoinBuildSide("SELECT * FROM t2 JOIN t1 ON t1.key = t2.key", bh, BuildRight) @@ -340,13 +351,23 @@ class BroadcastJoinSuite extends QueryTest with SQLTestUtils { assertJoinBuildSide("SELECT * FROM t1 RIGHT JOIN t2 ON t1.key = t2.key", bh, BuildLeft) assertJoinBuildSide("SELECT * FROM t2 RIGHT JOIN t1 ON t1.key = t2.key", bh, BuildLeft) + /* ######## test cases for non-equal join ######### */ withSQLConf(SQLConf.CROSS_JOINS_ENABLED.key -> "true") { + // For full outer join, prefer to broadcast the smaller side. assertJoinBuildSide("SELECT * FROM t1 FULL OUTER JOIN t2", bl, BuildLeft) assertJoinBuildSide("SELECT * FROM t2 FULL OUTER JOIN t1", bl, BuildRight) + // For inner join, prefer to broadcast the smaller side, if broadcast-able. + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> (t2Size + 1).toString()) { + assertJoinBuildSide("SELECT * FROM t1 JOIN t2", bl, BuildLeft) + assertJoinBuildSide("SELECT * FROM t2 JOIN t1", bl, BuildRight) + } + + // For left join, prefer to broadcast the right side. assertJoinBuildSide("SELECT * FROM t1 LEFT JOIN t2", bl, BuildRight) assertJoinBuildSide("SELECT * FROM t2 LEFT JOIN t1", bl, BuildRight) + // For right join, prefer to broadcast the left side. assertJoinBuildSide("SELECT * FROM t1 RIGHT JOIN t2", bl, BuildLeft) assertJoinBuildSide("SELECT * FROM t2 RIGHT JOIN t1", bl, BuildLeft) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala index 38377164c10e6..771a9730247af 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/ExistenceJoinSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{FilterExec, ProjectExec, SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -85,7 +85,8 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Row]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } @@ -102,7 +103,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( @@ -120,8 +121,8 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } } - test(s"$testName using BroadcastHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin") { _ => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( @@ -140,7 +141,7 @@ class ExistenceJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using SortMergeJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(left.sqlContext.sessionState.conf).apply( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala index d9b34dcd16476..02e474ce83372 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/HashedRelationSuite.scala @@ -22,8 +22,9 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream, ObjectInputStream, import scala.util.Random import org.apache.spark.{SparkConf, SparkFunSuite} -import org.apache.spark.internal.config.MEMORY_OFFHEAP_ENABLED -import org.apache.spark.memory.{StaticMemoryManager, TaskMemoryManager} +import org.apache.spark.internal.config._ +import org.apache.spark.internal.config.Kryo._ +import org.apache.spark.memory.{TaskMemoryManager, UnifiedMemoryManager} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -36,10 +37,10 @@ import org.apache.spark.util.collection.CompactBuffer class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { val mm = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) @@ -85,10 +86,10 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("test serialization empty hash map") { val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val binaryMap = new BytesToBytesMap(taskMemoryManager, 1, 1) @@ -157,10 +158,10 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("LongToUnsafeRowMap with very wide range") { val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) @@ -202,10 +203,10 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("LongToUnsafeRowMap with random keys") { val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val unsafeProj = UnsafeProjection.create(Seq(BoundReference(0, LongType, false))) @@ -256,10 +257,10 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("SPARK-24257: insert big values into LongToUnsafeRowMap") { val taskMemoryManager = new TaskMemoryManager( - new StaticMemoryManager( + new UnifiedMemoryManager( new SparkConf().set(MEMORY_OFFHEAP_ENABLED.key, "false"), Long.MaxValue, - Long.MaxValue, + Long.MaxValue / 2, 1), 0) val unsafeProj = UnsafeProjection.create(Array[DataType](StringType)) @@ -309,7 +310,7 @@ class HashedRelationSuite extends SparkFunSuite with SharedSQLContext { test("Spark-14521") { val ser = new KryoSerializer( - (new SparkConf).set("spark.kryo.referenceTracking", "false")).newInstance() + (new SparkConf).set(KRYO_REFERENCE_TRACKING, false)).newInstance() val key = Seq(BoundReference(0, LongType, false)) // Testing Kryo serialization of HashedRelation diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala index 4408ece112258..f99a278bb2427 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/InnerJoinSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.Expression import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans.Inner -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -80,7 +80,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Product]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition())) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition()), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } @@ -127,8 +128,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { EnsureRequirements(spark.sessionState.conf).apply(sortMergeJoin) } - test(s"$testName using BroadcastHashJoin (build=left)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin (build=left)") { _ => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeBroadcastHashJoin( @@ -139,8 +140,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } } - test(s"$testName using BroadcastHashJoin (build=right)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin (build=right)") { _ => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeBroadcastHashJoin( @@ -152,7 +153,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin (build=left)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeShuffledHashJoin( @@ -164,7 +165,7 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using ShuffledHashJoin (build=right)") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeShuffledHashJoin( @@ -175,8 +176,8 @@ class InnerJoinSuite extends SparkPlanTest with SharedSQLContext { } } - test(s"$testName using SortMergeJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + testWithWholeStageCodegenOnAndOff(s"$testName using SortMergeJoin") { _ => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (leftPlan: SparkPlan, rightPlan: SparkPlan) => makeSortMergeJoin(leftKeys, rightKeys, boundCondition, leftPlan, rightPlan), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala index 001feb0f2b399..1f04fcf6ca451 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/joins/OuterJoinSuite.scala @@ -21,7 +21,7 @@ import org.apache.spark.sql.{DataFrame, Row} import org.apache.spark.sql.catalyst.expressions.{And, Expression, LessThan} import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.Join +import org.apache.spark.sql.catalyst.plans.logical.{Join, JoinHint} import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} import org.apache.spark.sql.execution.exchange.EnsureRequirements import org.apache.spark.sql.internal.SQLConf @@ -72,13 +72,14 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { expectedAnswer: Seq[Product]): Unit = { def extractJoinParts(): Option[ExtractEquiJoinKeys.ReturnType] = { - val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, Inner, Some(condition)) + val join = Join(leftRows.logicalPlan, rightRows.logicalPlan, + Inner, Some(condition), JoinHint.NONE) ExtractEquiJoinKeys.unapply(join) } if (joinType != FullOuter) { test(s"$testName using ShuffledHashJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { val buildSide = if (joinType == LeftOuter) BuildRight else BuildLeft checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => @@ -93,13 +94,13 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { } if (joinType != FullOuter) { - test(s"$testName using BroadcastHashJoin") { + testWithWholeStageCodegenOnAndOff(s"$testName using BroadcastHashJoin") { _ => val buildSide = joinType match { case LeftOuter => BuildRight case RightOuter => BuildLeft case _ => fail(s"Unsupported join type $joinType") } - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => BroadcastHashJoinExec( @@ -112,7 +113,7 @@ class OuterJoinSuite extends SparkPlanTest with SharedSQLContext { } test(s"$testName using SortMergeJoin") { - extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _) => + extractJoinParts().foreach { case (_, leftKeys, rightKeys, boundCondition, _, _, _) => withSQLConf(SQLConf.SHUFFLE_PARTITIONS.key -> "1") { checkAnswer2(leftRows, rightRows, (left: SparkPlan, right: SparkPlan) => EnsureRequirements(spark.sessionState.conf).apply( 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 81db3e137964d..dc4a2998a908f 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 @@ -77,11 +77,16 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } test("WholeStageCodegen metrics") { - // Assume the execution plan is - // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Filter(nodeId = 1)) + // Assume the execution plan with node id is + // WholeStageCodegen(nodeId = 0) + // Filter(nodeId = 1) + // Range(nodeId = 2) // TODO: update metrics in generated operators val ds = spark.range(10).filter('id < 5) - testSparkPlanMetrics(ds.toDF(), 1, Map.empty) + testSparkPlanMetricsWithPredicates(ds.toDF(), 1, Map( + 0L -> (("WholeStageCodegen", Map( + "duration total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}))) + ), true) } test("Aggregate metrics") { @@ -91,11 +96,17 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df = testData2.groupBy().count() // 2 partitions val expected1 = Seq( Map("number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 1L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) + val shuffleExpected1 = Map( + "records read" -> 2L, + "local blocks read" -> 2L, + "remote blocks read" -> 0L, + "shuffle records written" -> 2L) testSparkPlanMetrics(df, 1, Map( 2L -> (("HashAggregate", expected1(0))), + 1L -> (("Exchange", shuffleExpected1)), 0L -> (("HashAggregate", expected1(1)))) ) @@ -103,11 +114,17 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df2 = testData2.groupBy('a).count() val expected2 = Seq( Map("number of output rows" -> 4L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)"), + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)"), Map("number of output rows" -> 3L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")) + "avg hash probe bucket list iters (min, med, max)" -> "\n(1, 1, 1)")) + val shuffleExpected2 = Map( + "records read" -> 4L, + "local blocks read" -> 4L, + "remote blocks read" -> 0L, + "shuffle records written" -> 4L) testSparkPlanMetrics(df2, 1, Map( 2L -> (("HashAggregate", expected2(0))), + 1L -> (("Exchange", shuffleExpected2)), 0L -> (("HashAggregate", expected2(1)))) ) } @@ -145,7 +162,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared } val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") + val probes = metrics(nodeId)._2("avg hash probe bucket list iters (min, med, max)") probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => assert(probe.toDouble > 1.0) } @@ -160,6 +177,11 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df = testData2.groupBy().agg(collect_set('a)) // 2 partitions testSparkPlanMetrics(df, 1, Map( 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 2L))), + 1L -> (("Exchange", Map( + "shuffle records written" -> 2L, + "records read" -> 2L, + "local blocks read" -> 2L, + "remote blocks read" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 1L)))) ) @@ -167,15 +189,30 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df2 = testData2.groupBy('a).agg(collect_set('a)) testSparkPlanMetrics(df2, 1, Map( 2L -> (("ObjectHashAggregate", Map("number of output rows" -> 4L))), + 1L -> (("Exchange", Map( + "shuffle records written" -> 4L, + "records read" -> 4L, + "local blocks read" -> 4L, + "remote blocks read" -> 0L))), 0L -> (("ObjectHashAggregate", Map("number of output rows" -> 3L)))) ) } test("Sort metrics") { - // Assume the execution plan is - // WholeStageCodegen(nodeId = 0, Range(nodeId = 2) -> Sort(nodeId = 1)) - val ds = spark.range(10).sort('id) - testSparkPlanMetrics(ds.toDF(), 2, Map.empty) + // Assume the execution plan with node id is + // Sort(nodeId = 0) + // Exchange(nodeId = 1) + // Project(nodeId = 2) + // LocalTableScan(nodeId = 3) + // Because of SPARK-25267, ConvertToLocalRelation is disabled in the test cases of sql/core, + // so Project here is not collapsed into LocalTableScan. + val df = Seq(1, 3, 2).toDF("id").sort('id) + testSparkPlanMetricsWithPredicates(df, 2, Map( + 0L -> (("Sort", Map( + "sort time total (min, med, max)" -> {_.toString.matches(timingMetricPattern)}, + "peak memory total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}, + "spill size total (min, med, max)" -> {_.toString.matches(sizeMetricPattern)}))) + )) } test("SortMergeJoin metrics") { @@ -191,7 +228,12 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared testSparkPlanMetrics(df, 1, Map( 0L -> (("SortMergeJoin", Map( // It's 4 because we only read 3 rows in the first partition and 1 row in the second one - "number of output rows" -> 4L)))) + "number of output rows" -> 4L))), + 2L -> (("Exchange", Map( + "records read" -> 4L, + "local blocks read" -> 2L, + "remote blocks read" -> 0L, + "shuffle records written" -> 2L)))) ) } } @@ -208,7 +250,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "SELECT * FROM testData2 left JOIN testDataForJoin ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df, 1, Map( 0L -> (("SortMergeJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + // It's 8 because we read 6 rows in the left and 2 row in the right one "number of output rows" -> 8L)))) ) @@ -216,7 +258,7 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared "SELECT * FROM testDataForJoin right JOIN testData2 ON testData2.a = testDataForJoin.a") testSparkPlanMetrics(df2, 1, Map( 0L -> (("SortMergeJoin", Map( - // It's 4 because we only read 3 rows in the first partition and 1 row in the second one + // It's 8 because we read 6 rows in the left and 2 row in the right one "number of output rows" -> 8L)))) ) } @@ -234,115 +276,35 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared ) } - test("BroadcastHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#210, b#211, b#221] - // +- BroadcastHashJoin [a#210], [a#220], Inner, BuildRight - // :- Project [_1#207 AS a#210, _2#208 AS b#211] - // : +- Filter isnotnull(_1#207) - // : +- LocalTableScan [_1#207, _2#208] - // +- BroadcastExchange HashedRelationBroadcastMode(List(input[0, binary, true])) - // +- Project [_1#217 AS a#220, _2#218 AS b#221] - // +- Filter isnotnull(_1#217) - // +- LocalTableScan [_1#217, _2#218] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // BroadcastHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // BroadcastHashJoin(nodeId = 2) - // Project(nodeId = 3) - // Filter(nodeId = 4) - // ...(ignored) - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF() - val df2 = generateRandomBytesDF() - val df = df1.join(broadcast(df2), "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 2, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - test("ShuffledHashJoin metrics") { - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "40", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "40", + SQLConf.SHUFFLE_PARTITIONS.key -> "2", + SQLConf.PREFER_SORTMERGEJOIN.key -> "false") { val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = (1 to 10).map(i => (i, i.toString)).toSeq.toDF("key", "value") // Assume the execution plan is - // ... -> ShuffledHashJoin(nodeId = 1) -> Project(nodeId = 0) + // Project(nodeId = 0) + // +- ShuffledHashJoin(nodeId = 1) + // :- Exchange(nodeId = 2) + // : +- Project(nodeId = 3) + // : +- LocalTableScan(nodeId = 4) + // +- Exchange(nodeId = 5) + // +- Project(nodeId = 6) + // +- LocalTableScan(nodeId = 7) val df = df1.join(df2, "key") - val metrics = getSparkPlanMetrics(df, 1, Set(1L)) testSparkPlanMetrics(df, 1, Map( 1L -> (("ShuffledHashJoin", Map( - "number of output rows" -> 2L, - "avg hash probe (min, med, max)" -> "\n(1, 1, 1)")))) + "number of output rows" -> 2L))), + 2L -> (("Exchange", Map( + "shuffle records written" -> 2L, + "records read" -> 2L))), + 5L -> (("Exchange", Map( + "shuffle records written" -> 10L, + "records read" -> 10L)))) ) } } - test("ShuffledHashJoin metrics: track avg probe") { - // The executed plan looks like: - // Project [a#308, b#309, b#319] - // +- ShuffledHashJoin [a#308], [a#318], Inner, BuildRight - // :- Exchange hashpartitioning(a#308, 2) - // : +- Project [_1#305 AS a#308, _2#306 AS b#309] - // : +- Filter isnotnull(_1#305) - // : +- LocalTableScan [_1#305, _2#306] - // +- Exchange hashpartitioning(a#318, 2) - // +- Project [_1#315 AS a#318, _2#316 AS b#319] - // +- Filter isnotnull(_1#315) - // +- LocalTableScan [_1#315, _2#316] - // - // Assume the execution plan with node id is - // WholeStageCodegen disabled: - // Project(nodeId = 0) - // ShuffledHashJoin(nodeId = 1) - // ...(ignored) - // - // WholeStageCodegen enabled: - // WholeStageCodegen(nodeId = 0) - // Project(nodeId = 1) - // ShuffledHashJoin(nodeId = 2) - // ...(ignored) - withSQLConf("spark.sql.autoBroadcastJoinThreshold" -> "5000000", - "spark.sql.shuffle.partitions" -> "2", - "spark.sql.join.preferSortMergeJoin" -> "false") { - Seq(true, false).foreach { enableWholeStage => - val df1 = generateRandomBytesDF(65535 * 5) - val df2 = generateRandomBytesDF(65535) - val df = df1.join(df2, "a") - val nodeIds = if (enableWholeStage) { - Set(2L) - } else { - Set(1L) - } - val metrics = getSparkPlanMetrics(df, 1, nodeIds, enableWholeStage).get - nodeIds.foreach { nodeId => - val probes = metrics(nodeId)._2("avg hash probe (min, med, max)") - probes.toString.stripPrefix("\n(").stripSuffix(")").split(", ").foreach { probe => - assert(probe.toDouble > 1.0) - } - } - } - } - } - test("BroadcastHashJoin(outer) metrics") { val df1 = Seq((1, "a"), (1, "b"), (4, "c")).toDF("key", "value") val df2 = Seq((1, "a"), (1, "b"), (2, "c"), (3, "d")).toDF("key2", "value") @@ -383,10 +345,10 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared val df1 = Seq((1, "1"), (2, "2")).toDF("key", "value") val df2 = Seq((1, "1"), (2, "2"), (3, "3"), (4, "4")).toDF("key2", "value") // Assume the execution plan is - // ... -> BroadcastHashJoin(nodeId = 0) + // ... -> BroadcastHashJoin(nodeId = 1) val df = df1.join(broadcast(df2), $"key" === $"key2", "leftsemi") testSparkPlanMetrics(df, 2, Map( - 0L -> (("BroadcastHashJoin", Map( + 1L -> (("BroadcastHashJoin", Map( "number of output rows" -> 2L)))) ) } @@ -475,28 +437,31 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared ) assert(res2 === (150L, 0L, 150L) :: (0L, 150L, 10L) :: Nil) - withTempDir { tempDir => - val dir = new File(tempDir, "pqS").getCanonicalPath - - spark.range(10).write.parquet(dir) - spark.read.parquet(dir).createOrReplaceTempView("pqS") - - // The executed plan looks like: - // Exchange RoundRobinPartitioning(2) - // +- BroadcastNestedLoopJoin BuildLeft, Cross - // :- BroadcastExchange IdentityBroadcastMode - // : +- Exchange RoundRobinPartitioning(3) - // : +- *Range (0, 30, step=1, splits=2) - // +- *FileScan parquet [id#465L] Batched: true, Format: Parquet, Location: ...(ignored) - val res3 = InputOutputMetricsHelper.run( - spark.range(30).repartition(3).crossJoin(sql("select * from pqS")).repartition(2).toDF() - ) - // The query above is executed in the following stages: - // 1. range(30) => (30, 0, 30) - // 2. sql("select * from pqS") => (0, 30, 0) - // 3. crossJoin(...) of 1. and 2. => (10, 0, 300) - // 4. shuffle & return results => (0, 300, 0) - assert(res3 === (30L, 0L, 30L) :: (0L, 30L, 0L) :: (10L, 0L, 300L) :: (0L, 300L, 0L) :: Nil) + // TODO: test file source V2 as well when its statistics is correctly computed. + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempDir { tempDir => + val dir = new File(tempDir, "pqS").getCanonicalPath + + spark.range(10).write.parquet(dir) + spark.read.parquet(dir).createOrReplaceTempView("pqS") + + // The executed plan looks like: + // Exchange RoundRobinPartitioning(2) + // +- BroadcastNestedLoopJoin BuildLeft, Cross + // :- BroadcastExchange IdentityBroadcastMode + // : +- Exchange RoundRobinPartitioning(3) + // : +- *Range (0, 30, step=1, splits=2) + // +- *FileScan parquet [id#465L] Batched: true, Format: Parquet, Location: ...(ignored) + val res3 = InputOutputMetricsHelper.run( + spark.range(30).repartition(3).crossJoin(sql("select * from pqS")).repartition(2).toDF() + ) + // The query above is executed in the following stages: + // 1. range(30) => (30, 0, 30) + // 2. sql("select * from pqS") => (0, 30, 0) + // 3. crossJoin(...) of 1. and 2. => (10, 0, 300) + // 4. shuffle & return results => (0, 300, 0) + assert(res3 === (30L, 0L, 30L) :: (0L, 30L, 0L) :: (10L, 0L, 300L) :: (0L, 300L, 0L) :: Nil) + } } } @@ -556,16 +521,16 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared df.queryExecution.executedPlan.foreach(_.resetMetrics()) // For each partition, we get 2 rows. Then the Filter should produce 2 rows per-partition, - // and Range should produce 1000 rows (one batch) per-partition. Totally Filter produces - // 4 rows, and Range produces 2000 rows. + // and Range should produce 4 rows per-partition ([0, 1, 2, 3] and [15, 16, 17, 18]). Totally + // Filter produces 4 rows, and Range produces 8 rows. df.queryExecution.toRdd.mapPartitions(_.take(2)).collect() - checkFilterAndRangeMetrics(df, filterNumOutputs = 4, rangeNumOutputs = 2000) + checkFilterAndRangeMetrics(df, filterNumOutputs = 4, rangeNumOutputs = 8) // Top-most limit will call `CollectLimitExec.executeCollect`, which will only run the first - // task, so totally the Filter produces 2 rows, and Range produces 1000 rows (one batch). + // task, so totally the Filter produces 2 rows, and Range produces 4 rows ([0, 1, 2, 3]). val df2 = df.limit(2) df2.collect() - checkFilterAndRangeMetrics(df2, filterNumOutputs = 2, rangeNumOutputs = 1000) + checkFilterAndRangeMetrics(df2, filterNumOutputs = 2, rangeNumOutputs = 4) } } @@ -597,4 +562,41 @@ class SQLMetricsSuite extends SparkFunSuite with SQLMetricsTestUtils with Shared assert(filters.head.metrics("numOutputRows").value == 1) } } + + test("SPARK-26327: FileSourceScanExec metrics") { + withTable("testDataForScan") { + spark.range(10).selectExpr("id", "id % 3 as p") + .write.partitionBy("p").saveAsTable("testDataForScan") + // The execution plan only has 1 FileScan node. + val df = spark.sql( + "SELECT * FROM testDataForScan WHERE p = 1") + testSparkPlanMetrics(df, 1, Map( + 0L -> (("Scan parquet default.testdataforscan", Map( + "number of output rows" -> 3L, + "number of files read" -> 2L)))) + ) + } + } + + test("InMemoryTableScan shows the table name on UI if possible") { + // Show table name on UI + withView("inMemoryTable", "```a``b```") { + sql("CREATE TEMPORARY VIEW inMemoryTable AS SELECT 1 AS c1") + sql("CACHE TABLE inMemoryTable") + testSparkPlanMetrics(spark.table("inMemoryTable"), 1, + Map(1L -> (("Scan In-memory table `inMemoryTable`", Map.empty))) + ) + + sql("CREATE TEMPORARY VIEW ```a``b``` AS SELECT 2 AS c1") + sql("CACHE TABLE ```a``b```") + testSparkPlanMetrics(spark.table("```a``b```"), 1, + Map(1L -> (("Scan In-memory table ```a``b```", Map.empty))) + ) + } + + // Show InMemoryTableScan on UI + testSparkPlanMetrics(spark.range(1).cache().select("id"), 1, + Map(1L -> (("InMemoryTableScan", Map.empty))) + ) + } } 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 dcc540fc4f109..8f26c04307adc 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 @@ -27,6 +27,7 @@ import org.apache.spark.sql.DataFrame import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.SparkPlanInfo import org.apache.spark.sql.execution.ui.{SparkPlanGraph, SQLAppStatusStore} +import org.apache.spark.sql.internal.SQLConf.WHOLESTAGE_CODEGEN_ENABLED import org.apache.spark.sql.test.SQLTestUtils @@ -40,6 +41,18 @@ 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)" + protected val sizeMetricPattern = { + val bytes = "([0-9]+(\\.[0-9]+)?) (EiB|PiB|TiB|GiB|MiB|KiB|B)" + s"\\n$bytes \\($bytes, $bytes, $bytes\\)" + } + + // Pattern of timing SQLMetric value, e.g. "\n2.0 ms (1.0 ms, 1.0 ms, 1.0 ms)" + protected val timingMetricPattern = { + val duration = "([0-9]+(\\.[0-9]+)?) (ms|s|m|h)" + s"\\n$duration \\($duration, $duration, $duration\\)" + } + /** * Get execution metrics for the SQL execution and verify metrics values. * @@ -72,8 +85,10 @@ trait SQLMetricsTestUtils extends SQLTestUtils { assert(metricValue == expected) } - val totalNumBytesMetric = executedNode.metrics.find(_.name == "bytes of written output").get - val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "").toInt + val totalNumBytesMetric = executedNode.metrics.find( + _.name == "written output total (min, med, max)").get + val totalNumBytes = metrics(totalNumBytesMetric.accumulatorId).replaceAll(",", "") + .split(" ").head.trim.toDouble assert(totalNumBytes > 0) } @@ -132,6 +147,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { * @param df `DataFrame` to run * @param expectedNumOfJobs number of jobs that will run * @param expectedNodeIds the node ids of the metrics to collect from execution data. + * @param enableWholeStage enable whole-stage code generation or not. */ protected def getSparkPlanMetrics( df: DataFrame, @@ -139,7 +155,7 @@ trait SQLMetricsTestUtils extends SQLTestUtils { expectedNodeIds: Set[Long], enableWholeStage: Boolean = false): Option[Map[Long, (String, Map[String, Any])]] = { val previousExecutionIds = currentExecutionIds() - withSQLConf("spark.sql.codegen.wholeStage" -> enableWholeStage.toString) { + withSQLConf(WHOLESTAGE_CODEGEN_ENABLED.key -> enableWholeStage.toString) { df.collect() } sparkContext.listenerBus.waitUntilEmpty(10000) @@ -185,15 +201,36 @@ trait SQLMetricsTestUtils extends SQLTestUtils { df: DataFrame, expectedNumOfJobs: Int, expectedMetrics: Map[Long, (String, Map[String, Any])]): Unit = { - val optActualMetrics = getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetrics.keySet) + val expectedMetricsPredicates = expectedMetrics.mapValues { case (nodeName, nodeMetrics) => + (nodeName, nodeMetrics.mapValues(expectedMetricValue => + (actualMetricValue: Any) => expectedMetricValue.toString === actualMetricValue)) + } + testSparkPlanMetricsWithPredicates(df, expectedNumOfJobs, expectedMetricsPredicates) + } + + /** + * Call `df.collect()` and verify if the collected metrics satisfy the specified predicates. + * @param df `DataFrame` to run + * @param expectedNumOfJobs number of jobs that will run + * @param expectedMetricsPredicates the expected metrics predicates. The format is + * `nodeId -> (operatorName, metric name -> metric predicate)`. + * @param enableWholeStage enable whole-stage code generation or not. + */ + protected def testSparkPlanMetricsWithPredicates( + df: DataFrame, + expectedNumOfJobs: Int, + expectedMetricsPredicates: Map[Long, (String, Map[String, Any => Boolean])], + enableWholeStage: Boolean = false): Unit = { + val optActualMetrics = + getSparkPlanMetrics(df, expectedNumOfJobs, expectedMetricsPredicates.keySet, enableWholeStage) optActualMetrics.foreach { actualMetrics => - assert(expectedMetrics.keySet === actualMetrics.keySet) - for (nodeId <- expectedMetrics.keySet) { - val (expectedNodeName, expectedMetricsMap) = expectedMetrics(nodeId) + assert(expectedMetricsPredicates.keySet === actualMetrics.keySet) + for ((nodeId, (expectedNodeName, expectedMetricsPredicatesMap)) + <- expectedMetricsPredicates) { val (actualNodeName, actualMetricsMap) = actualMetrics(nodeId) assert(expectedNodeName === actualNodeName) - for (metricName <- expectedMetricsMap.keySet) { - assert(expectedMetricsMap(metricName).toString === actualMetricsMap(metricName)) + for ((metricName, metricPredicate) <- expectedMetricsPredicatesMap) { + assert(metricPredicate(actualMetricsMap(metricName))) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala index 289cc667a1c66..8a18a1ab5406f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/BatchEvalPythonExecSuite.scala @@ -50,7 +50,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { case f @ FilterExec( And(_: AttributeReference, _: AttributeReference), - InputAdapter(_: BatchEvalPythonExec)) => f + InputAdapter(_: BatchEvalPythonExec, _)) => f case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(FilterExec(_: In, _))) => b } assert(qualifiedPlanNodes.size == 2) @@ -60,7 +60,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { val df = Seq(("Hello", 4)).toDF("a", "b") .where("dummyPythonUDF(a, dummyPythonUDF(a, b)) and a in (3, 4)") val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { - case f @ FilterExec(_: AttributeReference, InputAdapter(_: BatchEvalPythonExec)) => f + case f @ FilterExec(_: AttributeReference, InputAdapter(_: BatchEvalPythonExec, _)) => f case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(FilterExec(_: In, _))) => b } assert(qualifiedPlanNodes.size == 2) @@ -72,7 +72,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { case f @ FilterExec( And(_: AttributeReference, _: GreaterThan), - InputAdapter(_: BatchEvalPythonExec)) => f + InputAdapter(_: BatchEvalPythonExec, _)) => f case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(_: FilterExec)) => b } assert(qualifiedPlanNodes.size == 2) @@ -85,7 +85,7 @@ class BatchEvalPythonExecSuite extends SparkPlanTest with SharedSQLContext { val qualifiedPlanNodes = df.queryExecution.executedPlan.collect { case f @ FilterExec( And(_: AttributeReference, _: GreaterThan), - InputAdapter(_: BatchEvalPythonExec)) => f + InputAdapter(_: BatchEvalPythonExec, _)) => f case b @ BatchEvalPythonExec(_, _, WholeStageCodegenExec(_: FilterExec)) => b } assert(qualifiedPlanNodes.size == 2) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala index 76b609d111acd..311a8ef3257da 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/python/ExtractPythonUDFsSuite.scala @@ -17,13 +17,15 @@ package org.apache.spark.sql.execution.python -import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.{FileSourceScanExec, SparkPlan, SparkPlanTest} +import org.apache.spark.sql.execution.datasources.v2.BatchScanExec +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetScan import org.apache.spark.sql.functions.col +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SharedSQLContext class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSQLContext { - import testImplicits.newProductEncoder - import testImplicits.localSeqToDatasetHolder + import testImplicits._ val batchedPythonUDF = new MyDummyPythonUDF val scalarPandasUDF = new MyDummyScalarPandasUDF @@ -88,5 +90,81 @@ class ExtractPythonUDFsSuite extends SparkPlanTest with SharedSQLContext { assert(pythonEvalNodes.size == 2) assert(arrowEvalNodes.size == 2) } + + test("Python UDF should not break column pruning/filter pushdown -- Parquet V1") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTempPath { f => + spark.range(10).select($"id".as("a"), $"id".as("b")) + .write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: FileSourceScanExec => scan + } + assert(scanNodes.length == 1) + // 'a is not null and 'a > 1 + assert(scanNodes.head.dataFilters.length == 2) + assert(scanNodes.head.dataFilters.flatMap(_.references.map(_.name)).distinct == Seq("a")) + } + } + } + } + + test("Python UDF should not break column pruning/filter pushdown -- Parquet V2") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "") { + withTempPath { f => + spark.range(10).select($"id".as("a"), $"id".as("b")) + .write.parquet(f.getCanonicalPath) + val df = spark.read.parquet(f.getCanonicalPath) + + withClue("column pruning") { + val query = df.filter(batchedPythonUDF($"a")).select($"a") + + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExec => scan + } + assert(scanNodes.length == 1) + assert(scanNodes.head.output.map(_.name) == Seq("a")) + } + + withClue("filter pushdown") { + val query = df.filter($"a" > 1 && batchedPythonUDF($"a")) + val pythonEvalNodes = collectBatchExec(query.queryExecution.executedPlan) + assert(pythonEvalNodes.length == 1) + + val scanNodes = query.queryExecution.executedPlan.collect { + case scan: BatchScanExec => scan + } + assert(scanNodes.length == 1) + // 'a is not null and 'a > 1 + val filters = scanNodes.head.scan.asInstanceOf[ParquetScan].pushedFilters + assert(filters.length == 2) + assert(filters.flatMap(_.references).distinct === Array("a")) + } + } + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala index ec961a9ecb592..71dc3776bcaf6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/CompactibleFileStreamLogSuite.scala @@ -232,6 +232,29 @@ class CompactibleFileStreamLogSuite extends SparkFunSuite with SharedSQLContext }) } + test("prevent removing metadata files via method purge") { + withFakeCompactibleFileStreamLog( + fileCleanupDelayMs = 10000, + defaultCompactInterval = 2, + defaultMinBatchesToRetain = 3, + compactibleLog => { + // compaction batches: 1 + compactibleLog.add(0, Array("some_path_0")) + compactibleLog.add(1, Array("some_path_1")) + compactibleLog.add(2, Array("some_path_2")) + + val exc = intercept[UnsupportedOperationException] { + compactibleLog.purge(2) + } + assert(exc.getMessage.contains("Cannot purge as it might break internal state")) + + // Below line would fail with IllegalStateException if we don't prevent purge: + // - purge(2) would delete batch 0 and 1 which batch 1 is compaction batch + // - allFiles() would read batch 1 (latest compaction) and 2 which batch 1 is deleted + compactibleLog.allFiles() + }) + } + private def withFakeCompactibleFileStreamLog( fileCleanupDelayMs: Long, defaultCompactInterval: Int, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala index 9268306ce4275..04fe1e787909d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/HDFSMetadataLogSuite.scala @@ -91,17 +91,17 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { val e = intercept[IllegalStateException] { func } assert(e.getMessage.contains(s"Log file was malformed: failed to read correct log version")) } - assertLogFileMalformed { metadataLog.parseVersion("", 100) } - assertLogFileMalformed { metadataLog.parseVersion("xyz", 100) } - assertLogFileMalformed { metadataLog.parseVersion("v10.x", 100) } - assertLogFileMalformed { metadataLog.parseVersion("10", 100) } - assertLogFileMalformed { metadataLog.parseVersion("v0", 100) } - assertLogFileMalformed { metadataLog.parseVersion("v-10", 100) } + assertLogFileMalformed { metadataLog.validateVersion("", 100) } + assertLogFileMalformed { metadataLog.validateVersion("xyz", 100) } + assertLogFileMalformed { metadataLog.validateVersion("v10.x", 100) } + assertLogFileMalformed { metadataLog.validateVersion("10", 100) } + assertLogFileMalformed { metadataLog.validateVersion("v0", 100) } + assertLogFileMalformed { metadataLog.validateVersion("v-10", 100) } - assert(metadataLog.parseVersion("v10", 10) === 10) - assert(metadataLog.parseVersion("v10", 100) === 10) + assert(metadataLog.validateVersion("v10", 10) === 10) + assert(metadataLog.validateVersion("v10", 100) === 10) - val e = intercept[IllegalStateException] { metadataLog.parseVersion("v200", 100) } + val e = intercept[IllegalStateException] { metadataLog.validateVersion("v200", 100) } Seq( "maximum supported log version is v100, but encountered v200", "produced by a newer version of Spark and cannot be read by this version" @@ -131,9 +131,10 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { testQuietly("HDFSMetadataLog: metadata directory collision") { withTempDir { temp => - val waiter = new Waiter - val maxBatchId = 100 - for (id <- 0 until 10) { + val waiter = new Waiter() + val maxBatchId = 10 + val numThreads = 5 + for (id <- 0 until numThreads) { new UninterruptibleThread(s"HDFSMetadataLog: metadata directory collision - thread $id") { override def run(): Unit = waiter { val metadataLog = @@ -146,7 +147,7 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { nextBatchId += 1 } } catch { - case e: ConcurrentModificationException => + case _: ConcurrentModificationException => // This is expected since there are multiple writers } finally { waiter.dismiss() @@ -155,7 +156,7 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { }.start() } - waiter.await(timeout(10.seconds), dismissals(10)) + waiter.await(timeout(10.seconds), dismissals(numThreads)) val metadataLog = new HDFSMetadataLog[String](spark, temp.getAbsolutePath) assert(metadataLog.getLatest() === Some(maxBatchId -> maxBatchId.toString)) assert( @@ -178,5 +179,11 @@ class HDFSMetadataLogSuite extends SparkFunSuite with SharedSQLContext { intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), None, Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(2, 3, 4), Some(1L), Some(5L))) intercept[IllegalStateException](verifyBatchIds(Seq(1, 2, 4, 5), Some(1L), Some(5L))) + + // Related to SPARK-26629, this capatures the behavior for verifyBatchIds when startId > endId + intercept[IllegalStateException](verifyBatchIds(Seq(), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(2), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(1), Some(2L), Some(1L))) + intercept[AssertionError](verifyBatchIds(Seq(0), Some(2L), Some(1L))) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala index 3bc36ce55d902..3ead91fcf712a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkSuite.scala @@ -22,6 +22,8 @@ import scala.language.implicitConversions import org.scalatest.BeforeAndAfter import org.apache.spark.sql._ +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.execution.streaming.sources._ import org.apache.spark.sql.streaming.{OutputMode, StreamTest} import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils @@ -36,7 +38,8 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("directly add data in Append output mode") { implicit val schema = new StructType().add(new StructField("value", IntegerType)) - val sink = new MemorySink(schema, OutputMode.Append) + val sink = new MemorySink + val addBatch = addBatchFunc(sink, false) _ // Before adding data, check output assert(sink.latestBatchId === None) @@ -44,25 +47,25 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { checkAnswer(sink.allData, Seq.empty) // Add batch 0 and check outputs - sink.addBatch(0, 1 to 3) + addBatch(0, 1 to 3) assert(sink.latestBatchId === Some(0)) checkAnswer(sink.latestBatchData, 1 to 3) checkAnswer(sink.allData, 1 to 3) // Add batch 1 and check outputs - sink.addBatch(1, 4 to 6) + addBatch(1, 4 to 6) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data // Re-add batch 1 with different data, should not be added and outputs should not be changed - sink.addBatch(1, 7 to 9) + addBatch(1, 7 to 9) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 1 to 6) // Add batch 2 and check outputs - sink.addBatch(2, 7 to 9) + addBatch(2, 7 to 9) assert(sink.latestBatchId === Some(2)) checkAnswer(sink.latestBatchData, 7 to 9) checkAnswer(sink.allData, 1 to 9) @@ -70,7 +73,8 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("directly add data in Update output mode") { implicit val schema = new StructType().add(new StructField("value", IntegerType)) - val sink = new MemorySink(schema, OutputMode.Update) + val sink = new MemorySink + val addBatch = addBatchFunc(sink, false) _ // Before adding data, check output assert(sink.latestBatchId === None) @@ -78,25 +82,25 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { checkAnswer(sink.allData, Seq.empty) // Add batch 0 and check outputs - sink.addBatch(0, 1 to 3) + addBatch(0, 1 to 3) assert(sink.latestBatchId === Some(0)) checkAnswer(sink.latestBatchData, 1 to 3) checkAnswer(sink.allData, 1 to 3) // Add batch 1 and check outputs - sink.addBatch(1, 4 to 6) + addBatch(1, 4 to 6) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 1 to 6) // new data should get appended to old data // Re-add batch 1 with different data, should not be added and outputs should not be changed - sink.addBatch(1, 7 to 9) + addBatch(1, 7 to 9) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 1 to 6) // Add batch 2 and check outputs - sink.addBatch(2, 7 to 9) + addBatch(2, 7 to 9) assert(sink.latestBatchId === Some(2)) checkAnswer(sink.latestBatchData, 7 to 9) checkAnswer(sink.allData, 1 to 9) @@ -104,7 +108,8 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("directly add data in Complete output mode") { implicit val schema = new StructType().add(new StructField("value", IntegerType)) - val sink = new MemorySink(schema, OutputMode.Complete) + val sink = new MemorySink + val addBatch = addBatchFunc(sink, true) _ // Before adding data, check output assert(sink.latestBatchId === None) @@ -112,25 +117,25 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { checkAnswer(sink.allData, Seq.empty) // Add batch 0 and check outputs - sink.addBatch(0, 1 to 3) + addBatch(0, 1 to 3) assert(sink.latestBatchId === Some(0)) checkAnswer(sink.latestBatchData, 1 to 3) checkAnswer(sink.allData, 1 to 3) // Add batch 1 and check outputs - sink.addBatch(1, 4 to 6) + addBatch(1, 4 to 6) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 4 to 6) // new data should replace old data // Re-add batch 1 with different data, should not be added and outputs should not be changed - sink.addBatch(1, 7 to 9) + addBatch(1, 7 to 9) assert(sink.latestBatchId === Some(1)) checkAnswer(sink.latestBatchData, 4 to 6) checkAnswer(sink.allData, 4 to 6) // Add batch 2 and check outputs - sink.addBatch(2, 7 to 9) + addBatch(2, 7 to 9) assert(sink.latestBatchId === Some(2)) checkAnswer(sink.latestBatchData, 7 to 9) checkAnswer(sink.allData, 7 to 9) @@ -211,18 +216,19 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { test("MemoryPlan statistics") { implicit val schema = new StructType().add(new StructField("value", IntegerType)) - val sink = new MemorySink(schema, OutputMode.Append) - val plan = new MemoryPlan(sink) + val sink = new MemorySink + val addBatch = addBatchFunc(sink, false) _ + val plan = new MemoryPlan(sink, schema.toAttributes) // Before adding data, check output checkAnswer(sink.allData, Seq.empty) assert(plan.stats.sizeInBytes === 0) - sink.addBatch(0, 1 to 3) + addBatch(0, 1 to 3) plan.invalidateStatsCache() assert(plan.stats.sizeInBytes === 36) - sink.addBatch(1, 4 to 6) + addBatch(1, 4 to 6) plan.invalidateStatsCache() assert(plan.stats.sizeInBytes === 72) } @@ -285,6 +291,50 @@ class MemorySinkSuite extends StreamTest with BeforeAndAfter { } } + test("data writer") { + val partition = 1234 + val writer = new MemoryDataWriter( + partition, new StructType().add("i", "int")) + writer.write(InternalRow(1)) + writer.write(InternalRow(2)) + writer.write(InternalRow(44)) + val msg = writer.commit() + assert(msg.data.map(_.getInt(0)) == Seq(1, 2, 44)) + assert(msg.partition == partition) + + // Buffer should be cleared, so repeated commits should give empty. + assert(writer.commit().data.isEmpty) + } + + test("streaming writer") { + val sink = new MemorySink + val write = new MemoryStreamingWrite( + sink, new StructType().add("i", "int"), needTruncate = false) + write.commit(0, + Array( + MemoryWriterCommitMessage(0, Seq(Row(1), Row(2))), + MemoryWriterCommitMessage(1, Seq(Row(3), Row(4))), + MemoryWriterCommitMessage(2, Seq(Row(6), Row(7))) + )) + assert(sink.latestBatchId.contains(0)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7)) + write.commit(19, + Array( + MemoryWriterCommitMessage(3, Seq(Row(11), Row(22))), + MemoryWriterCommitMessage(0, Seq(Row(33))) + )) + assert(sink.latestBatchId.contains(19)) + assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(11, 22, 33)) + + assert(sink.allData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7, 11, 22, 33)) + } + + private def addBatchFunc(sink: MemorySink, needTruncate: Boolean)( + batchId: Long, + vals: Seq[Int]): Unit = { + sink.write(batchId, needTruncate, vals.map(Row(_)).toArray) + } + private def checkAnswer(rows: Seq[Row], expected: Seq[Int])(implicit schema: StructType): Unit = { checkAnswer( sqlContext.createDataFrame(sparkContext.makeRDD(rows), schema), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala deleted file mode 100644 index 61857365ac989..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/MemorySinkV2Suite.scala +++ /dev/null @@ -1,66 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.execution.streaming - -import org.scalatest.BeforeAndAfter - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.streaming.sources._ -import org.apache.spark.sql.streaming.{OutputMode, StreamTest} -import org.apache.spark.sql.types.StructType - -class MemorySinkV2Suite extends StreamTest with BeforeAndAfter { - test("data writer") { - val partition = 1234 - val writer = new MemoryDataWriter( - partition, OutputMode.Append(), new StructType().add("i", "int")) - writer.write(InternalRow(1)) - writer.write(InternalRow(2)) - writer.write(InternalRow(44)) - val msg = writer.commit() - assert(msg.data.map(_.getInt(0)) == Seq(1, 2, 44)) - assert(msg.partition == partition) - - // Buffer should be cleared, so repeated commits should give empty. - assert(writer.commit().data.isEmpty) - } - - test("streaming writer") { - val sink = new MemorySinkV2 - val writeSupport = new MemoryStreamingWriteSupport( - sink, OutputMode.Append(), new StructType().add("i", "int")) - writeSupport.commit(0, - Array( - MemoryWriterCommitMessage(0, Seq(Row(1), Row(2))), - MemoryWriterCommitMessage(1, Seq(Row(3), Row(4))), - MemoryWriterCommitMessage(2, Seq(Row(6), Row(7))) - )) - assert(sink.latestBatchId.contains(0)) - assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7)) - writeSupport.commit(19, - Array( - MemoryWriterCommitMessage(3, Seq(Row(11), Row(22))), - MemoryWriterCommitMessage(0, Seq(Row(33))) - )) - assert(sink.latestBatchId.contains(19)) - assert(sink.latestBatchData.map(_.getInt(0)).sorted == Seq(11, 22, 33)) - - assert(sink.allData.map(_.getInt(0)).sorted == Seq(1, 2, 3, 4, 6, 7, 11, 22, 33)) - } -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala index 80c76915e4c23..c0fd3fe3ef7b5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/ProcessingTimeExecutorSuite.scala @@ -19,15 +19,11 @@ package org.apache.spark.sql.execution.streaming import java.util.concurrent.ConcurrentHashMap -import scala.collection.mutable - -import org.eclipse.jetty.util.ConcurrentHashSet import org.scalatest.concurrent.{Eventually, Signaler, ThreadSignaler, TimeLimits} import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.streaming.ProcessingTime import org.apache.spark.sql.streaming.util.StreamManualClock class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @@ -38,7 +34,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { val timeout = 10.seconds test("nextBatchTime") { - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(100)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(100)) assert(processingTimeExecutor.nextBatchTime(0) === 100) assert(processingTimeExecutor.nextBatchTime(1) === 100) assert(processingTimeExecutor.nextBatchTime(99) === 100) @@ -48,11 +44,11 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { } test("trigger timing") { - val triggerTimes = new ConcurrentHashSet[Int] + val triggerTimes = ConcurrentHashMap.newKeySet[Int]() val clock = new StreamManualClock() @volatile var continueExecuting = true @volatile var clockIncrementInTrigger = 0L - val executor = ProcessingTimeExecutor(ProcessingTime("1000 milliseconds"), clock) + val executor = ProcessingTimeExecutor(ProcessingTimeTrigger("1000 milliseconds"), clock) val executorThread = new Thread() { override def run(): Unit = { executor.execute(() => { @@ -100,7 +96,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { test("calling nextBatchTime with the result of a previous call should return the next interval") { val intervalMS = 100 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMS)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMS)) val ITERATION = 10 var nextBatchTime: Long = 0 @@ -114,7 +110,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { private def testBatchTermination(intervalMs: Long): Unit = { var batchCounts = 0 - val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTime(intervalMs)) + val processingTimeExecutor = ProcessingTimeExecutor(ProcessingTimeTrigger(intervalMs)) processingTimeExecutor.execute(() => { batchCounts += 1 // If the batch termination works correctly, batchCounts should be 3 after `execute` @@ -133,7 +129,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { @volatile var batchFallingBehindCalled = false val t = new Thread() { override def run(): Unit = { - val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTime(100), clock) { + val processingTimeExecutor = new ProcessingTimeExecutor(ProcessingTimeTrigger(100), clock) { override def notifyBatchFallingBehind(realElapsedTimeMs: Long): Unit = { batchFallingBehindCalled = true } @@ -149,7 +145,7 @@ class ProcessingTimeExecutorSuite extends SparkFunSuite with TimeLimits { eventually { assert(clock.isStreamWaitingFor(200)) } clock.advance(200) waitForThreadJoin(t) - assert(batchFallingBehindCalled === true) + assert(batchFallingBehindCalled) } private def eventually(body: => Unit): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala similarity index 99% rename from sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala index b42f8267916b4..54ec4a8352c1b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/shuffle/ContinuousShuffleSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/continuous/shuffle/ContinuousShuffleSuite.scala @@ -21,9 +21,9 @@ import java.util.UUID import scala.language.implicitConversions -import org.apache.spark.{HashPartitioner, Partition, TaskContext, TaskContextImpl} +import org.apache.spark.{HashPartitioner, TaskContext, TaskContextImpl} import org.apache.spark.rpc.RpcEndpointRef -import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} +import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection} import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{DataType, IntegerType, StringType} import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala index dd74af873c2e5..ef88598fcb11b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/RateStreamProviderSuite.scala @@ -25,23 +25,26 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.functions._ -import org.apache.spark.sql.sources.v2.{ContinuousReadSupportProvider, DataSourceOptions, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.Offset +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.StreamTest +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.ManualClock -class RateSourceSuite extends StreamTest { +class RateStreamProviderSuite extends StreamTest { import testImplicits._ case class AdvanceRateManualClock(seconds: Long) extends AddData { - override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { + override def addData(query: Option[StreamExecution]): (SparkDataStream, Offset) = { assert(query.nonEmpty) val rateSource = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source: RateStreamMicroBatchReadSupport, _) => source + case r: StreamingDataSourceV2Relation + if r.stream.isInstanceOf[RateStreamMicroBatchStream] => + r.stream.asInstanceOf[RateStreamMicroBatchStream] }.head rateSource.clock.asInstanceOf[ManualClock].advance(TimeUnit.SECONDS.toMillis(seconds)) @@ -51,27 +54,16 @@ class RateSourceSuite extends StreamTest { } } - test("microbatch in registry") { - withTempDir { temp => - DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => - val readSupport = ds.createMicroBatchReadSupport( - temp.getCanonicalPath, DataSourceOptions.empty()) - assert(readSupport.isInstanceOf[RateStreamMicroBatchReadSupport]) - case _ => - throw new IllegalStateException("Could not find read support for rate") - } - } + test("RateStreamProvider in registry") { + val ds = DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() + assert(ds.isInstanceOf[RateStreamProvider], "Could not find rate source") } test("compatible with old path in registry") { - DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.RateSourceProvider", - spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => - assert(ds.isInstanceOf[RateStreamProvider]) - case _ => - throw new IllegalStateException("Could not find read support for rate") - } + val ds = DataSource.lookupDataSource( + "org.apache.spark.sql.execution.streaming.RateSourceProvider", + spark.sqlContext.conf).newInstance() + assert(ds.isInstanceOf[RateStreamProvider], "Could not find rate source") } test("microbatch - basic") { @@ -141,17 +133,17 @@ class RateSourceSuite extends StreamTest { test("microbatch - infer offsets") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( - new DataSourceOptions( - Map("numPartitions" -> "1", "rowsPerSecond" -> "100", "useManualClock" -> "true").asJava), - temp.getCanonicalPath) - readSupport.clock.asInstanceOf[ManualClock].advance(100000) - val startOffset = readSupport.initialOffset() + val stream = new RateStreamMicroBatchStream( + rowsPerSecond = 100, + options = new CaseInsensitiveStringMap(Map("useManualClock" -> "true").asJava), + checkpointLocation = temp.getCanonicalPath) + stream.clock.asInstanceOf[ManualClock].advance(100000) + val startOffset = stream.initialOffset() startOffset match { case r: LongOffset => assert(r.offset === 0L) case _ => throw new IllegalStateException("unexpected offset type") } - readSupport.latestOffset() match { + stream.latestOffset() match { case r: LongOffset => assert(r.offset >= 100) case _ => throw new IllegalStateException("unexpected offset type") } @@ -160,16 +152,14 @@ class RateSourceSuite extends StreamTest { test("microbatch - predetermined batch size") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( - new DataSourceOptions(Map("numPartitions" -> "1", "rowsPerSecond" -> "20").asJava), - temp.getCanonicalPath) - val startOffset = LongOffset(0L) - val endOffset = LongOffset(1L) - val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createReaderFactory(config) - assert(tasks.size == 1) - val dataReader = readerFactory.createReader(tasks(0)) + val stream = new RateStreamMicroBatchStream( + rowsPerSecond = 20, + options = CaseInsensitiveStringMap.empty(), + checkpointLocation = temp.getCanonicalPath) + val partitions = stream.planInputPartitions(LongOffset(0L), LongOffset(1L)) + val readerFactory = stream.createReaderFactory() + assert(partitions.size == 1) + val dataReader = readerFactory.createReader(partitions(0)) val data = ArrayBuffer[InternalRow]() while (dataReader.next()) { data.append(dataReader.get()) @@ -180,17 +170,16 @@ class RateSourceSuite extends StreamTest { test("microbatch - data read") { withTempDir { temp => - val readSupport = new RateStreamMicroBatchReadSupport( - new DataSourceOptions(Map("numPartitions" -> "11", "rowsPerSecond" -> "33").asJava), - temp.getCanonicalPath) - val startOffset = LongOffset(0L) - val endOffset = LongOffset(1L) - val config = readSupport.newScanConfigBuilder(startOffset, endOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createReaderFactory(config) - assert(tasks.size == 11) - - val readData = tasks + val stream = new RateStreamMicroBatchStream( + rowsPerSecond = 33, + numPartitions = 11, + options = CaseInsensitiveStringMap.empty(), + checkpointLocation = temp.getCanonicalPath) + val partitions = stream.planInputPartitions(LongOffset(0L), LongOffset(1L)) + val readerFactory = stream.createReaderFactory() + assert(partitions.size == 11) + + val readData = partitions .map(readerFactory.createReader) .flatMap { reader => val buf = scala.collection.mutable.ListBuffer[InternalRow]() @@ -316,32 +305,19 @@ class RateSourceSuite extends StreamTest { .load() } assert(exception.getMessage.contains( - "rate source does not support user-specified schema")) - } - - test("continuous in registry") { - DataSource.lookupDataSource("rate", spark.sqlContext.conf).newInstance() match { - case ds: ContinuousReadSupportProvider => - val readSupport = ds.createContinuousReadSupport( - "", DataSourceOptions.empty()) - assert(readSupport.isInstanceOf[RateStreamContinuousReadSupport]) - case _ => - throw new IllegalStateException("Could not find read support for continuous rate") - } + "RateStreamProvider source does not support user-specified schema")) } test("continuous data") { - val readSupport = new RateStreamContinuousReadSupport( - new DataSourceOptions(Map("numPartitions" -> "2", "rowsPerSecond" -> "20").asJava)) - val config = readSupport.newScanConfigBuilder(readSupport.initialOffset).build() - val tasks = readSupport.planInputPartitions(config) - val readerFactory = readSupport.createContinuousReaderFactory(config) - assert(tasks.size == 2) + val stream = new RateStreamContinuousStream(rowsPerSecond = 20, numPartitions = 2) + val partitions = stream.planInputPartitions(stream.initialOffset) + val readerFactory = stream.createContinuousReaderFactory() + assert(partitions.size == 2) val data = scala.collection.mutable.ListBuffer[InternalRow]() - tasks.foreach { + partitions.foreach { case t: RateStreamContinuousInputPartition => - val startTimeMs = readSupport.initialOffset() + val startTimeMs = stream.initialOffset() .asInstanceOf[RateStreamOffset] .partitionToValueAndRunTimeMs(t.partitionIndex) .runTimeMs diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala index 409156e5ebc70..2b7df40abcb64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/sources/TextSocketStreamSuite.scala @@ -22,6 +22,7 @@ import java.nio.ByteBuffer import java.nio.channels.ServerSocketChannel import java.sql.Timestamp import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.TimeUnit._ import scala.collection.JavaConverters._ @@ -30,14 +31,15 @@ import org.scalatest.BeforeAndAfterEach import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.datasources.DataSource +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.{DataSourceOptions, MicroBatchReadSupportProvider} -import org.apache.spark.sql.sources.v2.reader.streaming.Offset +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset, SparkDataStream} import org.apache.spark.sql.streaming.{StreamingQueryException, StreamTest} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.CaseInsensitiveStringMap class TextSocketStreamSuite extends StreamTest with SharedSQLContext with BeforeAndAfterEach { @@ -53,13 +55,15 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before private var serverThread: ServerThread = null case class AddSocketData(data: String*) extends AddData { - override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { + override def addData(query: Option[StreamExecution]): (SparkDataStream, Offset) = { require( query.nonEmpty, "Cannot add data when there is no query for finding the active socket source") val sources = query.get.logicalPlan.collect { - case StreamingExecutionRelation(source: TextSocketMicroBatchReadSupport, _) => source + case r: StreamingDataSourceV2Relation + if r.stream.isInstanceOf[TextSocketMicroBatchStream] => + r.stream.asInstanceOf[TextSocketMicroBatchStream] } if (sources.isEmpty) { throw new Exception( @@ -83,13 +87,10 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before } test("backward compatibility with old path") { - DataSource.lookupDataSource("org.apache.spark.sql.execution.streaming.TextSocketSourceProvider", - spark.sqlContext.conf).newInstance() match { - case ds: MicroBatchReadSupportProvider => - assert(ds.isInstanceOf[TextSocketSourceProvider]) - case _ => - throw new IllegalStateException("Could not find socket source") - } + val ds = DataSource.lookupDataSource( + "org.apache.spark.sql.execution.streaming.TextSocketSourceProvider", + spark.sqlContext.conf).newInstance() + assert(ds.isInstanceOf[TextSocketSourceProvider], "Could not find socket source") } test("basic usage") { @@ -167,7 +168,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before // Timestamp for rate stream is round to second which leads to milliseconds lost, that will // make batch1stamp smaller than current timestamp if both of them are in the same second. // Comparing by second to make sure the correct behavior. - assert(batch1Stamp.getTime >= curr / 1000 * 1000) + assert(batch1Stamp.getTime >= SECONDS.toMillis(MILLISECONDS.toSeconds(curr))) assert(!batch2Stamp.before(batch1Stamp)) } } @@ -175,16 +176,13 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before test("params not given") { val provider = new TextSocketSourceProvider intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map.empty[String, String].asJava)) + provider.getTable(CaseInsensitiveStringMap.empty()) } intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map("host" -> "localhost").asJava)) + provider.getTable(new CaseInsensitiveStringMap(Map("host" -> "localhost").asJava)) } intercept[AnalysisException] { - provider.createMicroBatchReadSupport( - "", new DataSourceOptions(Map("port" -> "1234").asJava)) + provider.getTable(new CaseInsensitiveStringMap(Map("port" -> "1234").asJava)) } } @@ -192,8 +190,7 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before val provider = new TextSocketSourceProvider val params = Map("host" -> "localhost", "port" -> "1234", "includeTimestamp" -> "fasle") intercept[AnalysisException] { - val a = new DataSourceOptions(params.asJava) - provider.createMicroBatchReadSupport("", a) + provider.getTable(new CaseInsensitiveStringMap(params.asJava)) } } @@ -204,11 +201,10 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before StructField("area", StringType) :: Nil) val params = Map("host" -> "localhost", "port" -> "1234") val exception = intercept[UnsupportedOperationException] { - provider.createMicroBatchReadSupport( - userSpecifiedSchema, "", new DataSourceOptions(params.asJava)) + provider.getTable(new CaseInsensitiveStringMap(params.asJava), userSpecifiedSchema) } assert(exception.getMessage.contains( - "socket source does not support user-specified schema")) + "TextSocketSourceProvider source does not support user-specified schema")) } test("input row metrics") { @@ -299,25 +295,25 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( - new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", - "port" -> serverThread.port.toString).asJava)) - - val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build() - val tasks = readSupport.planInputPartitions(scanConfig) - assert(tasks.size == 2) + val stream = new TextSocketContinuousStream( + host = "localhost", + port = serverThread.port, + numPartitions = 2, + options = CaseInsensitiveStringMap.empty()) + val partitions = stream.planInputPartitions(stream.initialOffset()) + assert(partitions.length == 2) val numRecords = 10 val data = scala.collection.mutable.ListBuffer[Int]() val offsets = scala.collection.mutable.ListBuffer[Int]() - val readerFactory = readSupport.createContinuousReaderFactory(scanConfig) + val readerFactory = stream.createContinuousReaderFactory() import org.scalatest.time.SpanSugar._ - failAfter(5 seconds) { + failAfter(5.seconds) { // inject rows, read and check the data and offsets for (i <- 0 until numRecords) { serverThread.enqueue(i.toString) } - tasks.foreach { + partitions.foreach { case t: TextSocketContinuousInputPartition => val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader] for (i <- 0 until numRecords / 2) { @@ -335,15 +331,15 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before data.clear() case _ => throw new IllegalStateException("Unexpected task type") } - assert(readSupport.startOffset.offsets == List(3, 3)) - readSupport.commit(TextSocketOffset(List(5, 5))) - assert(readSupport.startOffset.offsets == List(5, 5)) + assert(stream.startOffset.offsets == List(3, 3)) + stream.commit(TextSocketOffset(List(5, 5))) + assert(stream.startOffset.offsets == List(5, 5)) } def commitOffset(partition: Int, offset: Int): Unit = { - val offsetsToCommit = readSupport.startOffset.offsets.updated(partition, offset) - readSupport.commit(TextSocketOffset(offsetsToCommit)) - assert(readSupport.startOffset.offsets == offsetsToCommit) + val offsetsToCommit = stream.startOffset.offsets.updated(partition, offset) + stream.commit(TextSocketOffset(offsetsToCommit)) + assert(stream.startOffset.offsets == offsetsToCommit) } } @@ -351,13 +347,15 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( - new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", - "port" -> serverThread.port.toString).asJava)) + val stream = new TextSocketContinuousStream( + host = "localhost", + port = serverThread.port, + numPartitions = 2, + options = CaseInsensitiveStringMap.empty()) - readSupport.startOffset = TextSocketOffset(List(5, 5)) + stream.startOffset = TextSocketOffset(List(5, 5)) assertThrows[IllegalStateException] { - readSupport.commit(TextSocketOffset(List(6, 6))) + stream.commit(TextSocketOffset(List(6, 6))) } } @@ -365,27 +363,26 @@ class TextSocketStreamSuite extends StreamTest with SharedSQLContext with Before serverThread = new ServerThread() serverThread.start() - val readSupport = new TextSocketContinuousReadSupport( - new DataSourceOptions(Map("numPartitions" -> "2", "host" -> "localhost", - "includeTimestamp" -> "true", - "port" -> serverThread.port.toString).asJava)) - val scanConfig = readSupport.newScanConfigBuilder(readSupport.initialOffset()).build() - val tasks = readSupport.planInputPartitions(scanConfig) - assert(tasks.size == 2) + val stream = new TextSocketContinuousStream( + host = "localhost", + port = serverThread.port, + numPartitions = 2, + options = new CaseInsensitiveStringMap(Map("includeTimestamp" -> "true").asJava)) + val partitions = stream.planInputPartitions(stream.initialOffset()) + assert(partitions.size == 2) val numRecords = 4 // inject rows, read and check the data and offsets for (i <- 0 until numRecords) { serverThread.enqueue(i.toString) } - val readerFactory = readSupport.createContinuousReaderFactory(scanConfig) - tasks.foreach { + val readerFactory = stream.createContinuousReaderFactory() + partitions.foreach { case t: TextSocketContinuousInputPartition => val r = readerFactory.createReader(t).asInstanceOf[TextSocketContinuousPartitionReader] - for (i <- 0 until numRecords / 2) { + for (_ <- 0 until numRecords / 2) { r.next() - assert(r.get().get(0, TextSocketReader.SCHEMA_TIMESTAMP) - .isInstanceOf[(String, Timestamp)]) + assert(r.get().get(0, TextSocketReader.SCHEMA_TIMESTAMP).isInstanceOf[(_, _)]) } case _ => throw new IllegalStateException("Unexpected task type") } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala index 9a7595eee7bd0..7bca225dfdd8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreCoordinatorSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.scheduler.ExecutorCacheTaskLocation import org.apache.spark.sql.SparkSession import org.apache.spark.sql.execution.streaming.{MemoryStream, StreamingQueryWrapper} import org.apache.spark.sql.functions.count +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS import org.apache.spark.util.Utils class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { @@ -41,8 +42,8 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { assert(coordinatorRef.getLocation(id) === None) coordinatorRef.reportActiveInstance(id, "hostX", "exec1") - eventually(timeout(5 seconds)) { - assert(coordinatorRef.verifyIfInstanceActive(id, "exec1") === true) + eventually(timeout(5.seconds)) { + assert(coordinatorRef.verifyIfInstanceActive(id, "exec1")) assert( coordinatorRef.getLocation(id) === Some(ExecutorCacheTaskLocation("hostX", "exec1").toString)) @@ -50,9 +51,9 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { coordinatorRef.reportActiveInstance(id, "hostX", "exec2") - eventually(timeout(5 seconds)) { + eventually(timeout(5.seconds)) { assert(coordinatorRef.verifyIfInstanceActive(id, "exec1") === false) - assert(coordinatorRef.verifyIfInstanceActive(id, "exec2") === true) + assert(coordinatorRef.verifyIfInstanceActive(id, "exec2")) assert( coordinatorRef.getLocation(id) === @@ -75,16 +76,16 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { coordinatorRef.reportActiveInstance(id2, host, exec) coordinatorRef.reportActiveInstance(id3, host, exec) - eventually(timeout(5 seconds)) { - assert(coordinatorRef.verifyIfInstanceActive(id1, exec) === true) - assert(coordinatorRef.verifyIfInstanceActive(id2, exec) === true) - assert(coordinatorRef.verifyIfInstanceActive(id3, exec) === true) + eventually(timeout(5.seconds)) { + assert(coordinatorRef.verifyIfInstanceActive(id1, exec)) + assert(coordinatorRef.verifyIfInstanceActive(id2, exec)) + assert(coordinatorRef.verifyIfInstanceActive(id3, exec)) } coordinatorRef.deactivateInstances(runId1) assert(coordinatorRef.verifyIfInstanceActive(id1, exec) === false) - assert(coordinatorRef.verifyIfInstanceActive(id2, exec) === true) + assert(coordinatorRef.verifyIfInstanceActive(id2, exec)) assert(coordinatorRef.verifyIfInstanceActive(id3, exec) === false) assert(coordinatorRef.getLocation(id1) === None) @@ -107,8 +108,8 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { coordRef1.reportActiveInstance(id, "hostX", "exec1") - eventually(timeout(5 seconds)) { - assert(coordRef2.verifyIfInstanceActive(id, "exec1") === true) + eventually(timeout(5.seconds)) { + assert(coordRef2.verifyIfInstanceActive(id, "exec1")) assert( coordRef2.getLocation(id) === Some(ExecutorCacheTaskLocation("hostX", "exec1").toString)) @@ -124,7 +125,7 @@ class StateStoreCoordinatorSuite extends SparkFunSuite with SharedSparkContext { import spark.implicits._ coordRef = spark.streams.stateStoreCoordinator implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SHUFFLE_PARTITIONS.key, "1") // Start a query and run a batch to load state stores val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala index 5e973145b0a37..a84d107f2cbc0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/streaming/state/StateStoreSuite.scala @@ -35,6 +35,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.LocalSparkContext._ +import org.apache.spark.internal.config.Network.RPC_NUM_RETRIES import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.catalyst.util.quietly @@ -393,7 +394,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] .set(StateStore.MAINTENANCE_INTERVAL_CONFIG, "10ms") // Make sure that when SparkContext stops, the StateStore maintenance thread 'quickly' // fails to talk to the StateStoreCoordinator and unloads all the StateStores - .set("spark.rpc.numRetries", "1") + .set(RPC_NUM_RETRIES, 1) val opId = 0 val dir = newDir() val storeProviderId = StateStoreProviderId(StateStoreId(dir, opId, 0), UUID.randomUUID) @@ -415,7 +416,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] } } - val timeoutDuration = 60 seconds + val timeoutDuration = 1.minute quietly { withSpark(new SparkContext(conf)) { sc => @@ -568,7 +569,7 @@ class StateStoreSuite extends StateStoreSuiteBase[HDFSBackedStateStoreProvider] val spark = SparkSession.builder().master("local[2]").getOrCreate() SparkSession.setActiveSession(spark) implicit val sqlContext = spark.sqlContext - spark.conf.set("spark.sql.shuffle.partitions", "1") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "1") import spark.implicits._ val inputData = MemoryStream[Int] diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.scala new file mode 100644 index 0000000000000..95a6af3720d85 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/ui/AllExecutionsPageSuite.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.sql.execution.ui + +import java.util +import java.util.{Locale, Properties} +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.mockito.Mockito.{mock, when, RETURNS_SMART_NULLS} +import org.scalatest.BeforeAndAfter + +import org.apache.spark.scheduler.{JobFailed, SparkListenerJobEnd, SparkListenerJobStart} +import org.apache.spark.sql.DataFrame +import org.apache.spark.sql.execution.{SparkPlanInfo, SQLExecution} +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.status.ElementTrackingStore +import org.apache.spark.util.kvstore.InMemoryStore + +class AllExecutionsPageSuite extends SharedSQLContext with BeforeAndAfter { + + import testImplicits._ + + var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } + + test("SPARK-27019: correctly display SQL page when event reordering happens") { + val statusStore = createStatusStore + val tab = mock(classOf[SQLTab], RETURNS_SMART_NULLS) + when(tab.sqlStore).thenReturn(statusStore) + + val request = mock(classOf[HttpServletRequest]) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + + val html = renderSQLPage(request, tab, statusStore).toString().toLowerCase(Locale.ROOT) + assert(html.contains("failed queries")) + assert(!html.contains("1970")) + } + + test("sorting should be successful") { + val statusStore = createStatusStore + val tab = mock(classOf[SQLTab], RETURNS_SMART_NULLS) + val request = mock(classOf[HttpServletRequest]) + + when(tab.sqlStore).thenReturn(statusStore) + when(tab.appName).thenReturn("testing") + when(tab.headerTabs).thenReturn(Seq.empty) + when(request.getParameter("failed.sort")).thenReturn("Duration") + val map = new util.HashMap[String, Array[String]]() + map.put("failed.sort", Array("duration")) + when(request.getParameterMap()).thenReturn(map) + val html = renderSQLPage(request, tab, statusStore).toString().toLowerCase(Locale.ROOT) + assert(!html.contains("IllegalArgumentException")) + assert(html.contains("duration")) + } + + + private def createStatusStore: SQLAppStatusStore = { + val conf = sparkContext.conf + kvstore = new ElementTrackingStore(new InMemoryStore, conf) + val listener = new SQLAppStatusListener(conf, kvstore, live = true) + new SQLAppStatusStore(kvstore, Some(listener)) + } + + private def createTestDataFrame: DataFrame = { + Seq( + (1, 1), + (2, 2) + ).toDF().filter("_1 > 1") + } + + /** + * Render a stage page started with the given conf and return the HTML. + * This also runs a dummy execution page to populate the page with useful content. + */ + private def renderSQLPage( + request: HttpServletRequest, + tab: SQLTab, + statusStore: SQLAppStatusStore): Seq[Node] = { + + val listener = statusStore.listener.get + + val page = new AllExecutionsPage(tab) + Seq(0, 1).foreach { executionId => + val df = createTestDataFrame + listener.onOtherEvent(SparkListenerSQLExecutionStart( + executionId, + "test", + "test", + df.queryExecution.toString, + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) + listener.onJobStart(SparkListenerJobStart( + jobId = 0, + time = System.currentTimeMillis(), + stageInfos = Nil, + createProperties(executionId))) + listener.onJobEnd(SparkListenerJobEnd( + jobId = 0, + time = System.currentTimeMillis(), + JobFailed(new RuntimeException("Oops")))) + } + page.render(request) + } + + private def createProperties(executionId: Long): Properties = { + val properties = new Properties() + properties.setProperty(SQLExecution.EXECUTION_ID_KEY, executionId.toString) + properties + } +} + 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 02df45d1b7989..8edbb87706716 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 @@ -22,10 +22,12 @@ import java.util.Properties import scala.collection.mutable.ListBuffer import org.json4s.jackson.JsonMethods._ +import org.scalatest.BeforeAndAfter import org.apache.spark._ import org.apache.spark.LocalSparkContext._ import org.apache.spark.internal.config +import org.apache.spark.internal.config.Status._ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler._ import org.apache.spark.sql.{DataFrame, SparkSession} @@ -38,18 +40,28 @@ import org.apache.spark.sql.execution.metric.{SQLMetric, SQLMetrics} import org.apache.spark.sql.internal.StaticSQLConf.UI_RETAINED_EXECUTIONS import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.status.ElementTrackingStore -import org.apache.spark.status.config._ import org.apache.spark.util.{AccumulatorMetadata, JsonProtocol, LongAccumulator} import org.apache.spark.util.kvstore.InMemoryStore -class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils { +class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with JsonTestUtils + with BeforeAndAfter { + import testImplicits._ override protected def sparkConf = { super.sparkConf.set(LIVE_ENTITY_UPDATE_PERIOD, 0L).set(ASYNC_TRACKING_ENABLED, false) } + private var kvstore: ElementTrackingStore = _ + + after { + if (kvstore != null) { + kvstore.close() + kvstore = null + } + } + private def createTestDataFrame: DataFrame = { Seq( (1, 1), @@ -126,9 +138,9 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with private def createStatusStore(): SQLAppStatusStore = { val conf = sparkContext.conf - val store = new ElementTrackingStore(new InMemoryStore, conf) - val listener = new SQLAppStatusListener(conf, store, live = true) - new SQLAppStatusStore(store, Some(listener)) + kvstore = new ElementTrackingStore(new InMemoryStore, conf) + val listener = new SQLAppStatusListener(conf, kvstore, live = true) + new SQLAppStatusStore(kvstore, Some(listener)) } test("basic") { @@ -140,7 +152,7 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with // 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.get(id).get.trim + val v = actual(id).trim assert(v.startsWith(value.toString), s"Wrong value for accumulator $id") } } @@ -384,6 +396,36 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with assertJobs(statusStore.execution(executionId), failed = Seq(0)) } + test("onJobStart happens after onExecutionEnd shouldn't overwrite kvstore") { + val statusStore = createStatusStore() + val listener = statusStore.listener.get + + val executionId = 0 + val df = createTestDataFrame + listener.onOtherEvent(SparkListenerSQLExecutionStart( + executionId, + "test", + "test", + df.queryExecution.toString, + SparkPlanInfo.fromSparkPlan(df.queryExecution.executedPlan), + System.currentTimeMillis())) + listener.onOtherEvent(SparkListenerSQLExecutionEnd( + executionId, System.currentTimeMillis())) + listener.onJobStart(SparkListenerJobStart( + jobId = 0, + time = System.currentTimeMillis(), + stageInfos = Seq(createStageInfo(0, 0)), + createProperties(executionId))) + listener.onStageSubmitted(SparkListenerStageSubmitted(createStageInfo(0, 0))) + listener.onJobEnd(SparkListenerJobEnd( + jobId = 0, + time = System.currentTimeMillis(), + JobFailed(new RuntimeException("Oops")))) + + assert(listener.noLiveData()) + assert(statusStore.execution(executionId).get.completionTime.nonEmpty) + } + test("handle one execution with multiple jobs") { val statusStore = createStatusStore() val listener = statusStore.listener.get @@ -460,7 +502,8 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with } // Wait for listener to finish computing the metrics for the execution. - while (statusStore.executionsList().last.metricValues == null) { + while (statusStore.executionsList().isEmpty || + statusStore.executionsList().last.metricValues == null) { Thread.sleep(100) } @@ -518,9 +561,9 @@ class SQLAppStatusListenerSuite extends SparkFunSuite with SharedSQLContext with test("eviction should respect execution completion time") { val conf = sparkContext.conf.clone().set(UI_RETAINED_EXECUTIONS.key, "2") - val store = new ElementTrackingStore(new InMemoryStore, conf) - val listener = new SQLAppStatusListener(conf, store, live = true) - val statusStore = new SQLAppStatusStore(store, Some(listener)) + kvstore = new ElementTrackingStore(new InMemoryStore, conf) + val listener = new SQLAppStatusListener(conf, kvstore, live = true) + val statusStore = new SQLAppStatusStore(kvstore, Some(listener)) var time = 0 val df = createTestDataFrame @@ -603,8 +646,8 @@ class SQLAppStatusListenerMemoryLeakSuite extends SparkFunSuite { val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set(config.MAX_TASK_FAILURES, 1) // Don't retry the tasks to run this test quickly - .set("spark.sql.ui.retainedExecutions", "50") // Set it to 50 to run this test quickly + .set(config.TASK_MAX_FAILURES, 1) // Don't retry the tasks to run this test quickly + .set(UI_RETAINED_EXECUTIONS.key, "50") // Set it to 50 to run this test quickly .set(ASYNC_TRACKING_ENABLED, false) withSpark(new SparkContext(conf)) { sc => quietly { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala index 4592a1663faed..60f1b32a41f05 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ArrowColumnVectorSuite.scala @@ -21,8 +21,8 @@ import org.apache.arrow.vector._ import org.apache.arrow.vector.complex._ import org.apache.spark.SparkFunSuite -import org.apache.spark.sql.execution.arrow.ArrowUtils import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.ArrowUtils import org.apache.spark.sql.vectorized.ArrowColumnVector import org.apache.spark.unsafe.types.UTF8String diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala index 2d1ad4b456783..247efd5554a8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnVectorSuite.scala @@ -58,9 +58,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, BooleanType) === (i % 2 == 0)) + assert(arrayCopy.get(i, BooleanType) === (i % 2 == 0)) } } @@ -70,9 +72,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, ByteType) === i.toByte) + assert(arrayCopy.get(i, ByteType) === i.toByte) } } @@ -82,9 +86,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, ShortType) === i.toShort) + assert(arrayCopy.get(i, ShortType) === i.toShort) } } @@ -94,9 +100,25 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, IntegerType) === i) + assert(arrayCopy.get(i, IntegerType) === i) + } + } + + testVectors("date", 10, DateType) { testVector => + (0 until 10).foreach { i => + testVector.appendInt(i) + } + + val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() + + (0 until 10).foreach { i => + assert(array.get(i, DateType) === i) + assert(arrayCopy.get(i, DateType) === i) } } @@ -106,9 +128,25 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, LongType) === i) + assert(arrayCopy.get(i, LongType) === i) + } + } + + testVectors("timestamp", 10, TimestampType) { testVector => + (0 until 10).foreach { i => + testVector.appendLong(i) + } + + val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() + + (0 until 10).foreach { i => + assert(array.get(i, TimestampType) === i) + assert(arrayCopy.get(i, TimestampType) === i) } } @@ -118,9 +156,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, FloatType) === i.toFloat) + assert(arrayCopy.get(i, FloatType) === i.toFloat) } } @@ -130,9 +170,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, DoubleType) === i.toDouble) + assert(arrayCopy.get(i, DoubleType) === i.toDouble) } } @@ -143,9 +185,11 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => assert(array.get(i, StringType) === UTF8String.fromString(s"str$i")) + assert(arrayCopy.get(i, StringType) === UTF8String.fromString(s"str$i")) } } @@ -156,10 +200,12 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { } val array = new ColumnarArray(testVector, 0, 10) + val arrayCopy = array.copy() (0 until 10).foreach { i => val utf8 = s"str$i".getBytes("utf8") assert(array.get(i, BinaryType) === utf8) + assert(arrayCopy.get(i, BinaryType) === utf8) } } @@ -252,7 +298,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getBoolean(i) == (i % 2 == 0)) @@ -276,7 +322,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getByte(i) == i) @@ -300,7 +346,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getShort(i) == i) @@ -324,7 +370,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getInt(i) == i) @@ -348,7 +394,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getLong(i) == i.toLong) @@ -372,7 +418,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getFloat(i) == i.toFloat) @@ -396,7 +442,7 @@ class ColumnVectorSuite extends SparkFunSuite with BeforeAndAfterEach { val columnAccessor = ColumnAccessor(dataType, columnBuilder.build) ColumnAccessor.decompress(columnAccessor, testVector, 16) - assert(testVector.isNullAt(0) == true) + assert(testVector.isNullAt(0)) for (i <- 1 until 16) { assert(testVector.isNullAt(i) == false) assert(testVector.getDouble(i) == i.toDouble) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala index f311465e582ac..953b3a67d976f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchBenchmark.scala @@ -443,7 +443,7 @@ object ColumnarBatchBenchmark extends BenchmarkBase { benchmark.run } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("Int Read/Write") { intAccess(1024 * 40) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala index f57f07b498261..758780c80b284 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/vectorized/ColumnarBatchSuite.scala @@ -31,11 +31,14 @@ import org.apache.spark.SparkFunSuite import org.apache.spark.memory.MemoryMode import org.apache.spark.sql.{RandomDataGenerator, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.arrow.ArrowUtils +import org.apache.spark.sql.catalyst.expressions.GenericInternalRow +import org.apache.spark.sql.catalyst.util.{ArrayBasedMapBuilder, DateTimeUtils, GenericArrayData} +import org.apache.spark.sql.execution.RowToColumnConverter import org.apache.spark.sql.types._ -import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch, ColumnVector} +import org.apache.spark.sql.util.ArrowUtils +import org.apache.spark.sql.vectorized.{ArrowColumnVector, ColumnarBatch} import org.apache.spark.unsafe.Platform -import org.apache.spark.unsafe.types.CalendarInterval +import org.apache.spark.unsafe.types.{CalendarInterval, UTF8String} class ColumnarBatchSuite extends SparkFunSuite { @@ -988,14 +991,14 @@ class ColumnarBatchSuite extends SparkFunSuite { // Verify the results of the row. assert(batch.numCols() == 4) assert(batch.numRows() == 1) - assert(batch.rowIterator().hasNext == true) - assert(batch.rowIterator().hasNext == true) + assert(batch.rowIterator().hasNext) + assert(batch.rowIterator().hasNext) assert(columns(0).getInt(0) == 1) assert(columns(0).isNullAt(0) == false) assert(columns(1).getDouble(0) == 1.1) assert(columns(1).isNullAt(0) == false) - assert(columns(2).isNullAt(0) == true) + assert(columns(2).isNullAt(0)) assert(columns(3).getUTF8String(0).toString == "Hello") // Verify the iterator works correctly. @@ -1006,7 +1009,7 @@ class ColumnarBatchSuite extends SparkFunSuite { assert(row.isNullAt(0) == false) assert(row.getDouble(1) == 1.1) assert(row.isNullAt(1) == false) - assert(row.isNullAt(2) == true) + assert(row.isNullAt(2)) assert(columns(3).getUTF8String(0).toString == "Hello") assert(it.hasNext == false) assert(it.hasNext == false) @@ -1123,7 +1126,7 @@ class ColumnarBatchSuite extends SparkFunSuite { compareStruct(childFields, r1.getStruct(ordinal, fields.length), r2.getStruct(ordinal), seed) case _ => - throw new NotImplementedError("Not implemented " + field.dataType) + throw new UnsupportedOperationException("Not implemented " + field.dataType) } } } @@ -1270,6 +1273,211 @@ class ColumnarBatchSuite extends SparkFunSuite { allocator.close() } + test("RowToColumnConverter") { + val schema = StructType( + StructField("str", StringType) :: + StructField("bool", BooleanType) :: + StructField("byte", ByteType) :: + StructField("short", ShortType) :: + StructField("int", IntegerType) :: + StructField("long", LongType) :: + StructField("float", FloatType) :: + StructField("double", DoubleType) :: + StructField("decimal", DecimalType(25, 5)) :: + StructField("date", DateType) :: + StructField("ts", TimestampType) :: + StructField("cal", CalendarIntervalType) :: + StructField("arr_of_int", ArrayType(IntegerType)) :: + StructField("int_and_int", StructType( + StructField("int1", IntegerType, false) :: + StructField("int2", IntegerType) :: + Nil + )) :: + StructField("int_to_int", MapType(IntegerType, IntegerType)) :: + Nil) + var mapBuilder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + mapBuilder.put(1, 10) + mapBuilder.put(20, null) + val row1 = new GenericInternalRow(Array[Any]( + UTF8String.fromString("a string"), + true, + 1.toByte, + 2.toShort, + 3, + Long.MaxValue, + 0.25.toFloat, + 0.75D, + Decimal("1234.23456"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123")), + new CalendarInterval(1, 0), + new GenericArrayData(Array(1, 2, 3, 4, null)), + new GenericInternalRow(Array[Any](5.asInstanceOf[Any], 10)), + mapBuilder.build() + )) + + mapBuilder = new ArrayBasedMapBuilder(IntegerType, IntegerType) + mapBuilder.put(30, null) + mapBuilder.put(40, 50) + val row2 = new GenericInternalRow(Array[Any]( + UTF8String.fromString("second string"), + false, + -1.toByte, + 17.toShort, + Int.MinValue, + 987654321L, + Float.NaN, + Double.PositiveInfinity, + Decimal("0.01000"), + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("1875-12-12")), + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("1880-01-05 12:45:21.321")), + new CalendarInterval(-10, -100), + new GenericArrayData(Array(5, 10, -100)), + new GenericInternalRow(Array[Any](20.asInstanceOf[Any], null)), + mapBuilder.build() + )) + + val row3 = new GenericInternalRow(Array[Any]( + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null, + null + )) + + val converter = new RowToColumnConverter(schema) + val columns = OnHeapColumnVector.allocateColumns(3, schema) + val batch = new ColumnarBatch(columns.toArray, 3) + try { + converter.convert(row1, columns.toArray) + converter.convert(row2, columns.toArray) + converter.convert(row3, columns.toArray) + + assert(columns(0).dataType() == StringType) + assert(columns(0).getUTF8String(0).toString == "a string") + assert(columns(0).getUTF8String(1).toString == "second string") + assert(columns(0).isNullAt(2)) + + assert(columns(1).dataType() == BooleanType) + assert(columns(1).getBoolean(0) == true) + assert(columns(1).getBoolean(1) == false) + assert(columns(1).isNullAt(2)) + + assert(columns(2).dataType() == ByteType) + assert(columns(2).getByte(0) == 1.toByte) + assert(columns(2).getByte(1) == -1.toByte) + assert(columns(2).isNullAt(2)) + + assert(columns(3).dataType() == ShortType) + assert(columns(3).getShort(0) == 2.toShort) + assert(columns(3).getShort(1) == 17.toShort) + assert(columns(3).isNullAt(2)) + + assert(columns(4).dataType() == IntegerType) + assert(columns(4).getInt(0) == 3) + assert(columns(4).getInt(1) == Int.MinValue) + assert(columns(4).isNullAt(2)) + + assert(columns(5).dataType() == LongType) + assert(columns(5).getLong(0) == Long.MaxValue) + assert(columns(5).getLong(1) == 987654321L) + assert(columns(5).isNullAt(2)) + + assert(columns(6).dataType() == FloatType) + assert(columns(6).getFloat(0) == 0.25.toFloat) + assert(columns(6).getFloat(1).isNaN) + assert(columns(6).isNullAt(2)) + + assert(columns(7).dataType() == DoubleType) + assert(columns(7).getDouble(0) == 0.75D) + assert(columns(7).getDouble(1) == Double.PositiveInfinity) + assert(columns(7).isNullAt(2)) + + assert(columns(8).dataType() == DecimalType(25, 5)) + assert(columns(8).getDecimal(0, 25, 5) == Decimal("1234.23456")) + assert(columns(8).getDecimal(1, 25, 5) == Decimal("0.01000")) + assert(columns(8).isNullAt(2)) + + assert(columns(9).dataType() == DateType) + assert(columns(9).getInt(0) == + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("2015-01-01"))) + assert(columns(9).getInt(1) == + DateTimeUtils.fromJavaDate(java.sql.Date.valueOf("1875-12-12"))) + assert(columns(9).isNullAt(2)) + + assert(columns(10).dataType() == TimestampType) + assert(columns(10).getLong(0) == + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("2015-01-01 23:50:59.123"))) + assert(columns(10).getLong(1) == + DateTimeUtils.fromJavaTimestamp(java.sql.Timestamp.valueOf("1880-01-05 12:45:21.321"))) + assert(columns(10).isNullAt(2)) + + assert(columns(11).dataType() == CalendarIntervalType) + assert(columns(11).getInterval(0) == new CalendarInterval(1, 0)) + assert(columns(11).getInterval(1) == new CalendarInterval(-10, -100)) + assert(columns(11).isNullAt(2)) + + assert(columns(12).dataType() == ArrayType(IntegerType)) + val arr1 = columns(12).getArray(0) + assert(arr1.numElements() == 5) + assert(arr1.getInt(0) == 1) + assert(arr1.getInt(1) == 2) + assert(arr1.getInt(2) == 3) + assert(arr1.getInt(3) == 4) + assert(arr1.isNullAt(4)) + + val arr2 = columns(12).getArray(1) + assert(arr2.numElements() == 3) + assert(arr2.getInt(0) == 5) + assert(arr2.getInt(1) == 10) + assert(arr2.getInt(2) == -100) + + assert(columns(12).isNullAt(2)) + + assert(columns(13).dataType() == StructType( + StructField("int1", IntegerType, false) :: + StructField("int2", IntegerType) :: + Nil + )) + val struct1 = columns(13).getStruct(0) + assert(struct1.getInt(0) == 5) + assert(struct1.getInt(1) == 10) + val struct2 = columns(13).getStruct(1) + assert(struct2.getInt(0) == 20) + assert(struct2.isNullAt(1)) + assert(columns(13).isNullAt(2)) + + assert(columns(14).dataType() == MapType(IntegerType, IntegerType)) + val map1 = columns(14).getMap(0) + assert(map1.numElements() == 2) + assert(map1.keyArray().getInt(0) == 1) + assert(map1.valueArray().getInt(0) == 10) + assert(map1.keyArray().getInt(1) == 20) + assert(map1.valueArray().isNullAt(1)) + + val map2 = columns(14).getMap(1) + assert(map2.numElements() == 2) + assert(map2.keyArray().getInt(0) == 30) + assert(map2.valueArray().isNullAt(0)) + assert(map2.keyArray().getInt(1) == 40) + assert(map2.valueArray().getInt(1) == 50) + + assert(columns(14).isNullAt(2)) + } finally { + batch.close() + } + } + testVector("Decimal API", 4, DecimalType.IntDecimal) { column => diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala index 135370bd1d677..71e0d867ab661 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala @@ -71,15 +71,15 @@ class SQLConfEntrySuite extends SparkFunSuite { assert(conf.getConf(confEntry, false) === false) conf.setConf(confEntry, true) - assert(conf.getConf(confEntry, false) === true) + assert(conf.getConf(confEntry, false)) conf.setConfString(key, "true") assert(conf.getConfString(key, "false") === "true") assert(conf.getConfString(key) === "true") - assert(conf.getConf(confEntry, false) === true) + assert(conf.getConf(confEntry, false)) conf.setConfString(key, " true ") - assert(conf.getConf(confEntry, false) === true) + assert(conf.getConf(confEntry, false)) val e = intercept[IllegalArgumentException] { conf.setConfString(key, "abc") } 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 c9a6975da6be8..829dea48a22be 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.internal import org.apache.hadoop.fs.Path import org.apache.spark.sql._ -import org.apache.spark.sql.execution.WholeStageCodegenExec import org.apache.spark.sql.internal.StaticSQLConf._ import org.apache.spark.sql.test.{SharedSQLContext, TestSQLContext} import org.apache.spark.util.Utils @@ -118,12 +117,12 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { spark.sessionState.conf.clear() val original = spark.conf.get(SQLConf.GROUP_BY_ORDINAL) try { - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL)) sql(s"set ${SQLConf.GROUP_BY_ORDINAL.key}=false") assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === false) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 1) sql(s"reset") - assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL) === true) + assert(spark.conf.get(SQLConf.GROUP_BY_ORDINAL)) assert(sql(s"set").where(s"key = '${SQLConf.GROUP_BY_ORDINAL.key}'").count() == 0) } finally { sql(s"set ${SQLConf.GROUP_BY_ORDINAL}=$original") @@ -258,7 +257,7 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { // check default value assert(spark.sessionState.conf.parquetOutputTimestampType == - SQLConf.ParquetOutputTimestampType.INT96) + SQLConf.ParquetOutputTimestampType.TIMESTAMP_MICROS) // PARQUET_INT64_AS_TIMESTAMP_MILLIS should be respected. spark.sessionState.conf.setConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS, true) @@ -310,4 +309,15 @@ class SQLConfSuite extends QueryTest with SharedSQLContext { SQLConf.unregister(fallback) } + test("SPARK-24783: spark.sql.shuffle.partitions=0 should throw exception ") { + val e = intercept[IllegalArgumentException] { + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, 0) + } + assert(e.getMessage.contains("spark.sql.shuffle.partitions")) + val e2 = intercept[IllegalArgumentException] { + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, -1) + } + assert(e2.getMessage.contains("spark.sql.shuffle.partitions")) + } + } 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 7fa0e7fc162ca..89eaac8e5927c 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 @@ -27,7 +27,7 @@ import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkException import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.parser.CatalystSqlParser -import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap +import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeTestUtils} import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.datasources.LogicalRelation @@ -56,6 +56,20 @@ class JDBCSuite extends QueryTest Some(StringType) } + val testH2DialectTinyInt = new JdbcDialect { + override def canHandle(url: String): Boolean = url.startsWith("jdbc:h2") + override def getCatalystType( + sqlType: Int, + typeName: String, + size: Int, + md: MetadataBuilder): Option[DataType] = { + sqlType match { + case java.sql.Types.TINYINT => Some(ByteType) + case _ => None + } + } + } + before { Utils.classForName("org.h2.Driver") // Extra properties that will be specified for our database. We need these to test @@ -693,6 +707,17 @@ class JDBCSuite extends QueryTest JdbcDialects.unregisterDialect(testH2Dialect) } + test("Map TINYINT to ByteType via JdbcDialects") { + JdbcDialects.registerDialect(testH2DialectTinyInt) + val df = spark.read.jdbc(urlWithUserAndPass, "test.inttypes", new Properties()) + val rows = df.collect() + assert(rows.length === 2) + assert(rows(0).get(2).isInstanceOf[Byte]) + assert(rows(0).getByte(2) === 3) + assert(rows(1).isNullAt(2)) + JdbcDialects.unregisterDialect(testH2DialectTinyInt) + } + test("Default jdbc dialect registration") { assert(JdbcDialects.get("jdbc:mysql://127.0.0.1/db") == MySQLDialect) assert(JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") == PostgresDialect) @@ -825,14 +850,14 @@ class JDBCSuite extends QueryTest test("PostgresDialect type mapping") { val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") + val md = new MetadataBuilder().putLong("scale", 0) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "json", 1, null) === Some(StringType)) assert(Postgres.getCatalystType(java.sql.Types.OTHER, "jsonb", 1, null) === Some(StringType)) + assert(Postgres.getCatalystType(java.sql.Types.ARRAY, "_numeric", 0, md) == + Some(ArrayType(DecimalType.SYSTEM_DEFAULT))) assert(Postgres.getJDBCType(FloatType).map(_.databaseTypeDefinition).get == "FLOAT4") assert(Postgres.getJDBCType(DoubleType).map(_.databaseTypeDefinition).get == "FLOAT8") - val errMsg = intercept[IllegalArgumentException] { - Postgres.getJDBCType(ByteType) - } - assert(errMsg.getMessage contains "Unsupported type in postgresql: ByteType") + assert(Postgres.getJDBCType(ByteType).map(_.databaseTypeDefinition).get == "SMALLINT") } test("DerbyDialect jdbc type mapping") { @@ -857,6 +882,29 @@ class JDBCSuite extends QueryTest Some(TimestampType)) } + test("MsSqlServerDialect jdbc type mapping") { + val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver") + assert(msSqlServerDialect.getJDBCType(TimestampType).map(_.databaseTypeDefinition).get == + "DATETIME") + assert(msSqlServerDialect.getJDBCType(StringType).map(_.databaseTypeDefinition).get == + "NVARCHAR(MAX)") + assert(msSqlServerDialect.getJDBCType(BooleanType).map(_.databaseTypeDefinition).get == + "BIT") + assert(msSqlServerDialect.getJDBCType(BinaryType).map(_.databaseTypeDefinition).get == + "VARBINARY(MAX)") + assert(msSqlServerDialect.getJDBCType(ShortType).map(_.databaseTypeDefinition).get == + "SMALLINT") + } + + test("SPARK-28152 MsSqlServerDialect catalyst type mapping") { + val msSqlServerDialect = JdbcDialects.get("jdbc:sqlserver") + val metadata = new MetadataBuilder().putLong("scale", 1) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.SMALLINT, "SMALLINT", 1, + metadata).get == ShortType) + assert(msSqlServerDialect.getCatalystType(java.sql.Types.REAL, "REAL", 1, + metadata).get == FloatType) + } + test("table exists query by jdbc dialect") { val MySQL = JdbcDialects.get("jdbc:mysql://127.0.0.1/db") val Postgres = JdbcDialects.get("jdbc:postgresql://127.0.0.1/db") @@ -1282,7 +1330,7 @@ class JDBCSuite extends QueryTest testJdbcParitionColumn("THEID", "THEID") testJdbcParitionColumn("\"THEID\"", "THEID") - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { testJdbcParitionColumn("ThEiD", "THEID") } testJdbcParitionColumn("THE ID", "THE ID") @@ -1348,9 +1396,13 @@ class JDBCSuite extends QueryTest |the partition columns using the supplied subquery alias to resolve any ambiguity. |Example : |spark.read.format("jdbc") - | .option("dbtable", "(select c1, c2 from t1) as subq") - | .option("partitionColumn", "subq.c1" - | .load() + | .option("url", jdbcUrl) + | .option("dbtable", "(select c1, c2 from t1) as subq") + | .option("partitionColumn", "c1") + | .option("lowerBound", "1") + | .option("upperBound", "100") + | .option("numPartitions", "3") + | .load() """.stripMargin val e5 = intercept[RuntimeException] { sql( @@ -1478,4 +1530,49 @@ class JDBCSuite extends QueryTest checkNotPushdown(sql("SELECT name, theid FROM predicateOption WHERE theid = 1")), Row("fred", 1) :: Nil) } + + test("SPARK-26383 throw IllegalArgumentException if wrong kind of driver to the given url") { + val e = intercept[IllegalArgumentException] { + val opts = Map( + "url" -> "jdbc:mysql://localhost/db", + "dbtable" -> "table", + "driver" -> "org.postgresql.Driver" + ) + spark.read.format("jdbc").options(opts).load + }.getMessage + assert(e.contains("The driver could not open a JDBC connection. " + + "Check the URL: jdbc:mysql://localhost/db")) + } + + test("support casting patterns for lower/upper bounds of TimestampType") { + DateTimeTestUtils.outstandingTimezonesIds.foreach { timeZone => + withSQLConf(SQLConf.SESSION_LOCAL_TIMEZONE.key -> timeZone) { + Seq( + ("1972-07-04 03:30:00", "1972-07-15 20:50:32.5", "1972-07-27 14:11:05"), + ("2019-01-20 12:00:00.502", "2019-01-20 12:00:00.751", "2019-01-20 12:00:01.000"), + ("2019-01-20T00:00:00.123456", "2019-01-20 00:05:00.123456", + "2019-01-20T00:10:00.123456"), + ("1500-01-20T00:00:00.123456", "1500-01-20 00:05:00.123456", "1500-01-20T00:10:00.123456") + ).foreach { case (lower, middle, upper) => + val df = spark.read.format("jdbc") + .option("url", urlWithUserAndPass) + .option("dbtable", "TEST.DATETIME") + .option("partitionColumn", "t") + .option("lowerBound", lower) + .option("upperBound", upper) + .option("numPartitions", 2) + .load() + + df.logicalPlan match { + case lr: LogicalRelation if lr.relation.isInstanceOf[JDBCRelation] => + val jdbcRelation = lr.relation.asInstanceOf[JDBCRelation] + val whereClauses = jdbcRelation.parts.map(_.asInstanceOf[JDBCPartition].whereClause) + assert(whereClauses.toSet === Set( + s""""T" < '$middle' or "T" is null""", + s""""T" >= '$middle'""")) + } + } + } + } + } } 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 a2bc651bb2bd5..d52a78033e6cd 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 @@ -735,4 +735,54 @@ abstract class BucketedReadSuite extends QueryTest with SQLTestUtils { df1.groupBy("j").agg(max("k"))) } } + + // A test with a partition where the number of files in the partition is + // large. tests for the condition where the serialization of such a task may result in a stack + // overflow if the files list is stored in a recursive data structure + // This test is ignored because it takes long to run (~3 min) + ignore("SPARK-27100 stack overflow: read data with large partitions") { + val nCount = 20000 + // reshuffle data so that many small files are created + val nShufflePartitions = 10000 + // and with one table partition, should result in 10000 files in one partition + val nPartitions = 1 + val nBuckets = 2 + val dfPartitioned = (0 until nCount) + .map(i => (i % nPartitions, i % nBuckets, i.toString)).toDF("i", "j", "k") + + // non-bucketed tables. This part succeeds without the fix for SPARK-27100 + try { + withTable("non_bucketed_table") { + dfPartitioned.repartition(nShufflePartitions) + .write + .format("parquet") + .partitionBy("i") + .saveAsTable("non_bucketed_table") + + val table = spark.table("non_bucketed_table") + val nValues = table.select("j", "k").count() + assert(nValues == nCount) + } + } catch { + case e: Exception => fail("Failed due to exception: " + e) + } + // bucketed tables. This fails without the fix for SPARK-27100 + try { + withTable("bucketed_table") { + dfPartitioned.repartition(nShufflePartitions) + .write + .format("parquet") + .partitionBy("i") + .bucketBy(nBuckets, "j") + .saveAsTable("bucketed_table") + + val table = spark.table("bucketed_table") + val nValues = table.select("j", "k").count() + assert(nValues == nCount) + } + } catch { + case e: Exception => fail("Failed due to exception: " + e) + } + } + } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala index fc61050dc7458..75f68dea96bf0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/BucketedWriteSuite.scala @@ -63,7 +63,7 @@ abstract class BucketedWriteSuite extends QueryTest with SQLTestUtils { val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(SQLConf.BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { withTable("t") { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala index d46029e84433c..5f9856656ac3b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/CreateTableAsSelectSuite.scala @@ -26,6 +26,7 @@ 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.ParseException +import org.apache.spark.sql.internal.SQLConf.BUCKETING_MAX_BUCKETS import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.Utils @@ -252,7 +253,7 @@ class CreateTableAsSelectSuite val maxNrBuckets: Int = 200000 val catalog = spark.sessionState.catalog - withSQLConf("spark.sql.sources.bucketing.maxBuckets" -> maxNrBuckets.toString) { + withSQLConf(BUCKETING_MAX_BUCKETS.key -> maxNrBuckets.toString) { // Within the new limit Seq(100001, maxNrBuckets).foreach(numBuckets => { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala index a538b9458177e..daac207caf473 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/FilteredScanSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.sql.sources import java.util.Locale -import scala.language.existentials - import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.expressions.PredicateHelper 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 0b6d93975daef..4f1ae069d4b89 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 @@ -170,7 +170,7 @@ class InsertSuite extends DataSourceTest with SharedSQLContext { // Writing the table to more part files. val rdd2 = sparkContext.parallelize((1 to 10).map(i => s"""{"a":$i, "b":"str$i"}"""), 10) - spark.read.json(rdd1.toDS()).createOrReplaceTempView("jt2") + spark.read.json(rdd2.toDS()).createOrReplaceTempView("jt2") sql( s""" |INSERT OVERWRITE TABLE jsonTable SELECT a, b FROM jt2 diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala index c1eaf948a4b99..309591dd90f0f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/PrunedScanSuite.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.sources -import scala.language.existentials - import org.apache.spark.rdd.RDD import org.apache.spark.sql._ import org.apache.spark.sql.internal.SQLConf diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala index 12779b46bfe8c..7680f61b8b6c7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/SaveLoadSuite.scala @@ -18,6 +18,8 @@ package org.apache.spark.sql.sources import java.io.File +import java.nio.charset.StandardCharsets +import java.nio.file.{Files, Paths} import org.scalatest.BeforeAndAfter @@ -142,4 +144,17 @@ class SaveLoadSuite extends DataSourceTest with SharedSQLContext with BeforeAndA assert(e.contains(s"Partition column `$unknown` not found in schema $schemaCatalog")) } } + + test("skip empty files in non bucketed read") { + Seq("csv", "text").foreach { format => + withTempDir { dir => + val path = dir.getCanonicalPath + Files.write(Paths.get(path, "empty"), Array.empty[Byte]) + Files.write(Paths.get(path, "notEmpty"), "a".getBytes(StandardCharsets.UTF_8)) + val readBack = spark.read.option("wholetext", true).format(format).load(path) + + assert(readBack.rdd.getNumPartitions === 1) + } + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala new file mode 100644 index 0000000000000..c90090aca3d4e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2SQLSuite.scala @@ -0,0 +1,1201 @@ +/* + * 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.sources.v2 + +import scala.collection.JavaConverters._ + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.SparkException +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalog.v2.Identifier +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.execution.datasources.v2.V2SessionCatalog +import org.apache.spark.sql.execution.datasources.v2.orc.OrcDataSourceV2 +import org.apache.spark.sql.internal.SQLConf.V2_SESSION_CATALOG +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.{ArrayType, DoubleType, IntegerType, LongType, MapType, StringType, StructField, StructType, TimestampType} + +class DataSourceV2SQLSuite extends QueryTest with SharedSQLContext with BeforeAndAfter { + + import org.apache.spark.sql.catalog.v2.CatalogV2Implicits._ + + private val orc2 = classOf[OrcDataSourceV2].getName + + before { + spark.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) + spark.conf.set("spark.sql.catalog.testcat2", classOf[TestInMemoryTableCatalog].getName) + spark.conf.set(V2_SESSION_CATALOG.key, classOf[TestInMemoryTableCatalog].getName) + + val df = spark.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") + val df2 = spark.createDataFrame(Seq((4L, "d"), (5L, "e"), (6L, "f"))).toDF("id", "data") + df2.createOrReplaceTempView("source2") + } + + after { + spark.catalog("testcat").asInstanceOf[TestInMemoryTableCatalog].clearTables() + spark.catalog("session").asInstanceOf[TestInMemoryTableCatalog].clearTables() + } + + test("CreateTable: use v2 plan because catalog is set") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == 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), Seq.empty) + } + + test("CreateTable: use v2 plan and session catalog when provider is v2") { + spark.sql(s"CREATE TABLE table_name (id bigint, data string) USING $orc2") + + val testCatalog = spark.catalog("session").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "session.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> orc2).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), Seq.empty) + } + + test("CreateTable: fail if table exists") { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string) USING foo") + + val testCatalog = spark.catalog("testcat").asTableCatalog + + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) + + // run a second create query that should fail + val exc = intercept[TableAlreadyExistsException] { + spark.sql("CREATE TABLE testcat.table_name (id bigint, data string, id2 bigint) USING bar") + } + + assert(exc.getMessage.contains("table_name")) + + // table should not have changed + val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table2.name == "testcat.table_name") + assert(table2.partitioning.isEmpty) + assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.schema == new StructType().add("id", LongType).add("data", StringType)) + + // check that the table is still empty + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) + } + + test("CreateTable: if not exists") { + spark.sql( + "CREATE TABLE IF NOT EXISTS testcat.table_name (id bigint, data string) USING foo") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) + + spark.sql("CREATE TABLE IF NOT EXISTS testcat.table_name (id bigint, data string) USING bar") + + // table should not have changed + val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table2.name == "testcat.table_name") + assert(table2.partitioning.isEmpty) + assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.schema == new StructType().add("id", LongType).add("data", StringType)) + + // check that the table is still empty + val rdd2 = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd2, table.schema), Seq.empty) + } + + test("CreateTable: use default catalog for v2 sources when default catalog is set") { + val sparkSession = spark.newSession() + sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) + sparkSession.conf.set("spark.sql.default.catalog", "testcat") + sparkSession.sql(s"CREATE TABLE table_name (id bigint, data string) USING foo") + + val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType().add("id", LongType).add("data", StringType)) + + // check that the table is empty + val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), Seq.empty) + } + + test("CreateTableAsSelect: use v2 plan because catalog is set") { + spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + } + + test("CreateTableAsSelect: use v2 plan and session catalog when provider is v2") { + spark.sql(s"CREATE TABLE table_name USING $orc2 AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("session").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "session.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> orc2).asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + } + + test("CreateTableAsSelect: fail if table exists") { + spark.sql("CREATE TABLE testcat.table_name USING foo AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + + // run a second CTAS query that should fail + val exc = intercept[TableAlreadyExistsException] { + spark.sql( + "CREATE TABLE testcat.table_name USING bar AS SELECT id, data, id as id2 FROM source2") + } + + assert(exc.getMessage.contains("table_name")) + + // table should not have changed + val table2 = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + assert(table2.name == "testcat.table_name") + assert(table2.partitioning.isEmpty) + assert(table2.properties == Map("provider" -> "foo").asJava) + assert(table2.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd2 = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd2, table.schema), spark.table("source")) + } + + test("CreateTableAsSelect: if not exists") { + spark.sql( + "CREATE TABLE IF NOT EXISTS testcat.table_name USING foo AS SELECT id, data FROM source") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), spark.table("source")) + + spark.sql( + "CREATE TABLE IF NOT EXISTS testcat.table_name USING foo AS SELECT id, data FROM source2") + + // check that the table contains data from just the first CTAS + val rdd2 = spark.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd2, table.schema), spark.table("source")) + } + + test("CreateTableAsSelect: use default catalog for v2 sources when default catalog is set") { + val sparkSession = spark.newSession() + sparkSession.conf.set("spark.sql.catalog.testcat", classOf[TestInMemoryTableCatalog].getName) + sparkSession.conf.set("spark.sql.default.catalog", "testcat") + + val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") + + // setting the default catalog breaks the reference to source because the default catalog is + // used and AsTableIdentifier no longer matches + sparkSession.sql(s"CREATE TABLE table_name USING foo AS SELECT id, data FROM source") + + val testCatalog = sparkSession.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array(), "table_name")) + + assert(table.name == "testcat.table_name") + assert(table.partitioning.isEmpty) + assert(table.properties == Map("provider" -> "foo").asJava) + assert(table.schema == new StructType() + .add("id", LongType, nullable = false) + .add("data", StringType)) + + val rdd = sparkSession.sparkContext.parallelize(table.asInstanceOf[InMemoryTable].rows) + checkAnswer(spark.internalCreateDataFrame(rdd, table.schema), sparkSession.table("source")) + } + + test("CreateTableAsSelect: v2 session catalog can load v1 source table") { + val sparkSession = spark.newSession() + sparkSession.conf.set(V2_SESSION_CATALOG.key, classOf[V2SessionCatalog].getName) + + val df = sparkSession.createDataFrame(Seq((1L, "a"), (2L, "b"), (3L, "c"))).toDF("id", "data") + df.createOrReplaceTempView("source") + + sparkSession.sql(s"CREATE TABLE table_name USING parquet AS SELECT id, data FROM source") + + // use the catalog name to force loading with the v2 catalog + checkAnswer(sparkSession.sql(s"TABLE session.table_name"), sparkSession.table("source")) + } + + test("DropTable: basic") { + val tableName = "testcat.ns1.ns2.tbl" + val ident = Identifier.of(Array("ns1", "ns2"), "tbl") + sql(s"CREATE TABLE $tableName USING foo AS SELECT id, data FROM source") + assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === true) + sql(s"DROP TABLE $tableName") + assert(spark.catalog("testcat").asTableCatalog.tableExists(ident) === false) + } + + test("DropTable: if exists") { + intercept[NoSuchTableException] { + sql(s"DROP TABLE testcat.db.notbl") + } + sql(s"DROP TABLE IF EXISTS testcat.db.notbl") + } + + test("Relation: basic") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") + checkAnswer(sql(s"TABLE $t1"), spark.table("source")) + checkAnswer(sql(s"SELECT * FROM $t1"), spark.table("source")) + } + } + + test("Relation: SparkSession.table()") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") + checkAnswer(spark.table(s"$t1"), spark.table("source")) + } + } + + test("Relation: CTE") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") + checkAnswer( + sql(s""" + |WITH cte AS (SELECT * FROM $t1) + |SELECT * FROM cte + """.stripMargin), + spark.table("source")) + } + } + + test("Relation: view text") { + val t1 = "testcat.ns1.ns2.tbl" + withTable(t1) { + withView("view1") { v1: String => + sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") + sql(s"CREATE VIEW $v1 AS SELECT * from $t1") + checkAnswer(sql(s"TABLE $v1"), spark.table("source")) + } + } + } + + test("Relation: join tables in 2 catalogs") { + val t1 = "testcat.ns1.ns2.tbl" + val t2 = "testcat2.v2tbl" + withTable(t1, t2) { + sql(s"CREATE TABLE $t1 USING foo AS SELECT id, data FROM source") + sql(s"CREATE TABLE $t2 USING foo AS SELECT id, data FROM source2") + val df1 = spark.table("source") + val df2 = spark.table("source2") + val df_joined = df1.join(df2).where(df1("id") + 1 === df2("id")) + checkAnswer( + sql(s""" + |SELECT * + |FROM $t1 t1, $t2 t2 + |WHERE t1.id + 1 = t2.id + """.stripMargin), + df_joined) + } + } + + test("AlterTable: table does not exist") { + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE testcat.ns1.table_name DROP COLUMN id") + } + + assert(exc.getMessage.contains("testcat.ns1.table_name")) + assert(exc.getMessage.contains("Table or view not found")) + } + + test("AlterTable: change rejected by implementation") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[SparkException] { + sql(s"ALTER TABLE $t DROP COLUMN id") + } + + assert(exc.getMessage.contains("Unsupported table change")) + assert(exc.getMessage.contains("Cannot drop all fields")) // from the implementation + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: add top-level column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN data string") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType).add("data", StringType)) + } + } + + test("AlterTable: add column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN data string COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc")))) + } + } + + test("AlterTable: add multiple columns") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMNS data string COMMENT 'doc', ts timestamp") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq( + StructField("id", IntegerType), + StructField("data", StringType).withComment("doc"), + StructField("ts", TimestampType)))) + } + } + + test("AlterTable: add nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN point.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))))) + } + } + + test("AlterTable: add nested column to map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.key.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType))), LongType))) + } + } + + test("AlterTable: add nested column to map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.value.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add nested column to array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.element.z double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType)))))) + } + } + + test("AlterTable: add complex column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points array>") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: add nested column with comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ADD COLUMN points.element.z double COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType), + StructField("z", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: add nested column parent must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ADD COLUMN point.z double") + } + + assert(exc.getMessage.contains("point")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type int -> long") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", LongType)) + } + } + + test("AlterTable: update nested type float -> double") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with struct type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point TYPE struct") + } + + assert(exc.getMessage.contains("point")) + assert(exc.getMessage.contains("update a struct by adding, deleting, or updating its fields")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: update column with array type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN points TYPE array") + } + + assert(exc.getMessage.contains("update the element by updating points.element")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(IntegerType))) + } + } + + test("AlterTable: update column array element type") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element TYPE long") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(LongType))) + } + } + + test("AlterTable: update column with map type fails") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, m map) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN m TYPE map") + } + + assert(exc.getMessage.contains("update a map by updating m.key or m.value")) + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, IntegerType))) + } + } + + test("AlterTable: update column map value type") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, m map) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN m.value TYPE long") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("m", MapType(StringType, LongType))) + } + } + + test("AlterTable: update nested type in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.key.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))), LongType))) + } + } + + test("AlterTable: update nested type in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.value.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update nested type in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element.x TYPE double") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType)))))) + } + } + + test("AlterTable: update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data TYPE string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x TYPE double") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: update column type must be compatible") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE boolean") + } + + assert(exc.getMessage.contains("id")) + assert(exc.getMessage.contains("int cannot be cast to boolean")) + } + } + + test("AlterTable: update column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", IntegerType).withComment("doc")))) + } + } + + test("AlterTable: update column type and comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN id TYPE bigint COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == StructType(Seq(StructField("id", LongType).withComment("doc")))) + } + } + + test("AlterTable: update nested column comment") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN point.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))))) + } + } + + test("AlterTable: update nested column comment in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map, bigint>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.key.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc"))), LongType))) + } + } + + test("AlterTable: update nested column comment in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.value.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: update nested column comment in array") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t ALTER COLUMN points.element.y COMMENT 'doc'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType).withComment("doc")))))) + } + } + + test("AlterTable: comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN data COMMENT 'doc'") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested comment update column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t ALTER COLUMN point.x COMMENT 'doc'") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: rename column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN id TO user_id") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("user_id", IntegerType)) + } + } + + test("AlterTable: rename nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN point.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))))) + } + } + + test("AlterTable: rename nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN point.key.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType))), LongType))) + } + } + + test("AlterTable: rename nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN points.value.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t RENAME COLUMN points.element.y TO t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType), + StructField("t", DoubleType)))))) + } + } + + test("AlterTable: rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN data TO some_string") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested rename column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t RENAME COLUMN point.x TO z") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: drop column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, data string) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN data") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType().add("id", IntegerType)) + } + } + + test("AlterTable: drop nested column") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point struct) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN point.t") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", StructType(Seq( + StructField("x", DoubleType), + StructField("y", DoubleType))))) + } + } + + test("AlterTable: drop nested column in map key") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, point map, bigint>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN point.key.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("point", MapType(StructType(Seq( + StructField("x", DoubleType))), LongType))) + } + } + + test("AlterTable: drop nested column in map value") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points map>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN points.value.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", MapType(StringType, StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop nested column in array element") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int, points array>) USING foo") + sql(s"ALTER TABLE $t DROP COLUMN points.element.y") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.schema == new StructType() + .add("id", IntegerType) + .add("points", ArrayType(StructType(Seq( + StructField("x", DoubleType)))))) + } + } + + test("AlterTable: drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN data") + } + + assert(exc.getMessage.contains("data")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: nested drop column must exist") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + + val exc = intercept[AnalysisException] { + sql(s"ALTER TABLE $t DROP COLUMN point.x") + } + + assert(exc.getMessage.contains("point.x")) + assert(exc.getMessage.contains("missing field")) + } + } + + test("AlterTable: set location") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t SET LOCATION 's3://bucket/path'") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "location" -> "s3://bucket/path").asJava) + } + } + + test("AlterTable: set table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo") + sql(s"ALTER TABLE $t SET TBLPROPERTIES ('test'='34')") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + } + } + + test("AlterTable: remove table property") { + val t = "testcat.ns1.table_name" + withTable(t) { + sql(s"CREATE TABLE $t (id int) USING foo TBLPROPERTIES('test' = '34')") + + val testCatalog = spark.catalog("testcat").asTableCatalog + val table = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(table.name == "testcat.ns1.table_name") + assert(table.properties == Map("provider" -> "foo", "test" -> "34").asJava) + + sql(s"ALTER TABLE $t UNSET TBLPROPERTIES ('test')") + + val updated = testCatalog.loadTable(Identifier.of(Array("ns1"), "table_name")) + + assert(updated.name == "testcat.ns1.table_name") + assert(updated.properties == Map("provider" -> "foo").asJava) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala index e8f291af13baf..379c9c4303cd6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2Suite.scala @@ -18,38 +18,43 @@ package org.apache.spark.sql.sources.v2 import java.io.File +import java.util +import java.util.OptionalLong + +import scala.collection.JavaConverters._ import test.org.apache.spark.sql.sources.v2._ import org.apache.spark.SparkException -import org.apache.spark.sql.{DataFrame, QueryTest, Row} +import org.apache.spark.sql.{AnalysisException, DataFrame, QueryTest, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, DataSourceV2ScanExec} +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation} import org.apache.spark.sql.execution.exchange.{Exchange, ShuffleExchangeExec} import org.apache.spark.sql.execution.vectorized.OnHeapColumnVector import org.apache.spark.sql.functions._ import org.apache.spark.sql.sources.{Filter, GreaterThan} +import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.partitioning.{ClusteredDistribution, Distribution, Partitioning} import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types.{IntegerType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.sql.vectorized.ColumnarBatch class DataSourceV2Suite extends QueryTest with SharedSQLContext { import testImplicits._ - private def getScanConfig(query: DataFrame): AdvancedScanConfigBuilder = { + private def getBatch(query: DataFrame): AdvancedBatch = { query.queryExecution.executedPlan.collect { - case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[AdvancedScanConfigBuilder] + case d: BatchScanExec => + d.batch.asInstanceOf[AdvancedBatch] }.head } - private def getJavaScanConfig( - query: DataFrame): JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder = { + private def getJavaBatch(query: DataFrame): JavaAdvancedDataSourceV2.AdvancedBatch = { query.queryExecution.executedPlan.collect { - case d: DataSourceV2ScanExec => - d.scanConfig.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedScanConfigBuilder] + case d: BatchScanExec => + d.batch.asInstanceOf[JavaAdvancedDataSourceV2.AdvancedBatch] }.head } @@ -73,51 +78,51 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val q1 = df.select('j) checkAnswer(q1, (0 until 10).map(i => Row(-i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val batch = getBatch(q1) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q1) - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + val batch = getJavaBatch(q1) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } val q2 = df.filter('i > 3) checkAnswer(q2, (4 until 10).map(i => Row(i, -i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val batch = getBatch(q2) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i", "j")) } else { - val config = getJavaScanConfig(q2) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i", "j")) + val batch = getJavaBatch(q2) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i", "j")) } val q3 = df.select('i).filter('i > 6) checkAnswer(q3, (7 until 10).map(i => Row(i))) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val batch = getBatch(q3) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i")) } else { - val config = getJavaScanConfig(q3) - assert(config.filters.flatMap(_.references).toSet == Set("i")) - assert(config.requiredSchema.fieldNames === Seq("i")) + val batch = getJavaBatch(q3) + assert(batch.filters.flatMap(_.references).toSet == Set("i")) + assert(batch.requiredSchema.fieldNames === Seq("i")) } val q4 = df.select('j).filter('j < -10) checkAnswer(q4, Nil) if (cls == classOf[AdvancedDataSourceV2]) { - val config = getScanConfig(q4) + val batch = getBatch(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } else { - val config = getJavaScanConfig(q4) + val batch = getJavaBatch(q4) // 'j < 10 is not supported by the testing data source. - assert(config.filters.isEmpty) - assert(config.requiredSchema.fieldNames === Seq("j")) + assert(batch.filters.isEmpty) + assert(batch.requiredSchema.fieldNames === Seq("j")) } } } @@ -183,6 +188,24 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } } + test ("statistics report data source") { + Seq(classOf[ReportStatisticsDataSource], classOf[JavaReportStatisticsDataSource]).foreach { + cls => + withClue(cls.getName) { + val df = spark.read.format(cls.getName).load() + val logical = df.queryExecution.optimizedPlan.collect { + case d: DataSourceV2Relation => d + }.head + + val statics = logical.computeStats() + assert(statics.rowCount.isDefined && statics.rowCount.get === 10, + "Row count statics should be reported by data source") + assert(statics.sizeInBytes === 80, + "Size in bytes statics should be reported by data source") + } + } + } + test("SPARK-23574: no shuffle exchange with single partition") { val df = spark.read.format(classOf[SimpleSinglePartitionSource].getName).load().agg(count("*")) assert(df.queryExecution.executedPlan.collect { case e: Exchange => e }.isEmpty) @@ -196,14 +219,14 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { assert(spark.read.format(cls.getName).option("path", path).load().collect().isEmpty) spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName) - .option("path", path).save() + .option("path", path).mode("append").save() checkAnswer( spark.read.format(cls.getName).option("path", path).load(), spark.range(10).select('id, -'id)) - // test with different save modes + // default save mode is append spark.range(10).select('id as 'i, -'id as 'j).write.format(cls.getName) - .option("path", path).mode("append").save() + .option("path", path).save() checkAnswer( spark.read.format(cls.getName).option("path", path).load(), spark.range(10).union(spark.range(10)).select('id, -'id)) @@ -214,17 +237,17 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { spark.read.format(cls.getName).option("path", path).load(), spark.range(5).select('id, -'id)) - spark.range(5).select('id as 'i, -'id as 'j).write.format(cls.getName) - .option("path", path).mode("ignore").save() - checkAnswer( - spark.read.format(cls.getName).option("path", path).load(), - spark.range(5).select('id, -'id)) + val e = intercept[AnalysisException] { + spark.range(5).select('id as 'i, -'id as 'j).write.format(cls.getName) + .option("path", path).mode("ignore").save() + } + assert(e.message.contains("please use Append or Overwrite modes instead")) - val e = intercept[Exception] { + val e2 = intercept[AnalysisException] { spark.range(5).select('id as 'i, -'id as 'j).write.format(cls.getName) .option("path", path).mode("error").save() } - assert(e.getMessage.contains("data already exists")) + assert(e2.getMessage.contains("please use Append or Overwrite modes instead")) // test transaction val failingUdf = org.apache.spark.sql.functions.udf { @@ -239,10 +262,10 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } // this input data will fail to read middle way. val input = spark.range(10).select(failingUdf('id).as('i)).select('i, -'i as 'j) - val e2 = intercept[SparkException] { + val e3 = intercept[SparkException] { input.write.format(cls.getName).option("path", path).mode("overwrite").save() } - assert(e2.getMessage.contains("Writing job aborted")) + assert(e3.getMessage.contains("Writing job aborted")) // make sure we don't have partial data. assert(spark.read.format(cls.getName).option("path", path).load().collect().isEmpty) } @@ -279,26 +302,26 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { val q1 = df.select('i + 1) checkAnswer(q1, (1 until 11).map(i => Row(i))) - val config1 = getScanConfig(q1) - assert(config1.requiredSchema.fieldNames === Seq("i")) + val batch1 = getBatch(q1) + assert(batch1.requiredSchema.fieldNames === Seq("i")) val q2 = df.select(lit(1)) checkAnswer(q2, (0 until 10).map(i => Row(1))) - val config2 = getScanConfig(q2) - assert(config2.requiredSchema.isEmpty) + val batch2 = getBatch(q2) + assert(batch2.requiredSchema.isEmpty) // 'j === 1 can't be pushed down, but we should still be able do column pruning val q3 = df.filter('j === -1).select('j * 2) checkAnswer(q3, Row(-2)) - val config3 = getScanConfig(q3) - assert(config3.filters.isEmpty) - assert(config3.requiredSchema.fieldNames === Seq("j")) + val batch3 = getBatch(q3) + assert(batch3.filters.isEmpty) + assert(batch3.requiredSchema.fieldNames === Seq("j")) // column pruning should work with other operators. val q4 = df.sort('i).limit(1).select('i + 1) checkAnswer(q4, Row(1)) - val config4 = getScanConfig(q4) - assert(config4.requiredSchema.fieldNames === Seq("i")) + val batch4 = getBatch(q4) + assert(batch4.requiredSchema.fieldNames === Seq("i")) } test("SPARK-23315: get output from canonicalized data source v2 related plans") { @@ -310,7 +333,7 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { assert(logical.canonicalized.output.length == logicalNumOutput) val physical = df.queryExecution.executedPlan.collect { - case d: DataSourceV2ScanExec => d + case d: BatchScanExec => d }.head assert(physical.canonicalized.output.length == physicalNumOutput) } @@ -330,8 +353,8 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { .format(classOf[DataSourceV2WithSessionConfig].getName).load() val options = df.queryExecution.optimizedPlan.collectFirst { case d: DataSourceV2Relation => d.options - } - assert(options.get.get(optionName) == Some("false")) + }.get + assert(options.get(optionName) === "false") } } @@ -352,21 +375,16 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { } } - test("SPARK-25700: do not read schema when writing in other modes except append mode") { - withTempPath { file => - val cls = classOf[SimpleWriteOnlyDataSource] - val path = file.getCanonicalPath - val df = spark.range(5).select('id as 'i, -'id as 'j) - try { - df.write.format(cls.getName).option("path", path).mode("error").save() - df.write.format(cls.getName).option("path", path).mode("overwrite").save() - df.write.format(cls.getName).option("path", path).mode("ignore").save() - } catch { - case e: SchemaReadAttemptException => fail("Schema read was attempted.", e) - } - intercept[SchemaReadAttemptException] { - df.write.format(cls.getName).option("path", path).mode("append").save() - } + test("SPARK-27411: DataSourceV2Strategy should not eliminate subquery") { + withTempView("t1") { + val t2 = spark.read.format(classOf[SimpleDataSourceV2].getName).load() + Seq(2, 3).toDF("a").createTempView("t1") + val df = t2.where("i < (select max(a) from t1)").select('i) + val subqueries = df.queryExecution.executedPlan.collect { + case p => p.subqueries + }.flatten + assert(subqueries.length == 1) + checkAnswer(df, (0 until 3).map(i => Row(i))) } } } @@ -374,10 +392,6 @@ class DataSourceV2Suite extends QueryTest with SharedSQLContext { case class RangeInputPartition(start: Int, end: Int) extends InputPartition -case class NoopScanConfigBuilder(readSchema: StructType) extends ScanConfigBuilder with ScanConfig { - override def build(): ScanConfig = this -} - object SimpleReaderFactory extends PartitionReaderFactory { override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { val RangeInputPartition(start, end) = partition @@ -396,87 +410,70 @@ object SimpleReaderFactory extends PartitionReaderFactory { } } -abstract class SimpleReadSupport extends BatchReadSupport { - override def fullSchema(): StructType = new StructType().add("i", "int").add("j", "int") +abstract class SimpleBatchTable extends Table with SupportsRead { - override def newScanConfigBuilder(): ScanConfigBuilder = { - NoopScanConfigBuilder(fullSchema()) - } + override def schema(): StructType = new StructType().add("i", "int").add("j", "int") - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - SimpleReaderFactory - } + override def name(): String = this.getClass.toString + + override def capabilities(): util.Set[TableCapability] = Set(BATCH_READ).asJava } +abstract class SimpleScanBuilder extends ScanBuilder + with Batch with Scan { + + override def build(): Scan = this + + override def toBatch: Batch = this + + override def readSchema(): StructType = new StructType().add("i", "int").add("j", "int") -class SimpleSinglePartitionSource extends DataSourceV2 with BatchReadSupportProvider { + override def createReaderFactory(): PartitionReaderFactory = SimpleReaderFactory +} + +class SimpleSinglePartitionSource extends TableProvider { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 5)) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder() + } } } // This class is used by pyspark tests. If this class is modified/moved, make sure pyspark // tests still pass. -class SimpleDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class SimpleDataSourceV2 extends TableProvider { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10)) } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder() + } } } -class AdvancedDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { - - class ReadSupport extends SimpleReadSupport { - override def newScanConfigBuilder(): ScanConfigBuilder = new AdvancedScanConfigBuilder() +class AdvancedDataSourceV2 extends TableProvider { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { - val filters = config.asInstanceOf[AdvancedScanConfigBuilder].filters - - val lowerBound = filters.collectFirst { - case GreaterThan("i", v: Int) => v - } - - val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition] - - if (lowerBound.isEmpty) { - res.append(RangeInputPartition(0, 5)) - res.append(RangeInputPartition(5, 10)) - } else if (lowerBound.get < 4) { - res.append(RangeInputPartition(lowerBound.get + 1, 5)) - res.append(RangeInputPartition(5, 10)) - } else if (lowerBound.get < 9) { - res.append(RangeInputPartition(lowerBound.get + 1, 10)) - } - - res.toArray - } - - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { - val requiredSchema = config.asInstanceOf[AdvancedScanConfigBuilder].requiredSchema - new AdvancedReaderFactory(requiredSchema) + override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new AdvancedScanBuilder() } } - - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } } -class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig - with SupportsPushDownRequiredColumns with SupportsPushDownFilters { +class AdvancedScanBuilder extends ScanBuilder + with Scan with SupportsPushDownFilters with SupportsPushDownRequiredColumns { var requiredSchema = new StructType().add("i", "int").add("j", "int") var filters = Array.empty[Filter] @@ -498,10 +495,40 @@ class AdvancedScanConfigBuilder extends ScanConfigBuilder with ScanConfig override def pushedFilters(): Array[Filter] = filters - override def build(): ScanConfig = this + override def build(): Scan = this + + override def toBatch: Batch = new AdvancedBatch(filters, requiredSchema) +} + +class AdvancedBatch(val filters: Array[Filter], val requiredSchema: StructType) extends Batch { + + override def planInputPartitions(): Array[InputPartition] = { + val lowerBound = filters.collectFirst { + case GreaterThan("i", v: Int) => v + } + + val res = scala.collection.mutable.ArrayBuffer.empty[InputPartition] + + if (lowerBound.isEmpty) { + res.append(RangeInputPartition(0, 5)) + res.append(RangeInputPartition(5, 10)) + } else if (lowerBound.get < 4) { + res.append(RangeInputPartition(lowerBound.get + 1, 5)) + res.append(RangeInputPartition(5, 10)) + } else if (lowerBound.get < 9) { + res.append(RangeInputPartition(lowerBound.get + 1, 10)) + } + + res.toArray + } + + override def createReaderFactory(): PartitionReaderFactory = { + new AdvancedReaderFactory(requiredSchema) + } } class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { val RangeInputPartition(start, end) = partition new PartitionReader[InternalRow] { @@ -526,39 +553,47 @@ class AdvancedReaderFactory(requiredSchema: StructType) extends PartitionReaderF } -class SchemaRequiredDataSource extends DataSourceV2 with BatchReadSupportProvider { +class SchemaRequiredDataSource extends TableProvider { - class ReadSupport(val schema: StructType) extends SimpleReadSupport { - override def fullSchema(): StructType = schema + class MyScanBuilder(schema: StructType) extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = Array.empty - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = - Array.empty + override def readSchema(): StructType = schema } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { + override def getTable(options: CaseInsensitiveStringMap): Table = { throw new IllegalArgumentException("requires a user-supplied schema") } - override def createBatchReadSupport( - schema: StructType, options: DataSourceOptions): BatchReadSupport = { - new ReadSupport(schema) + override def getTable(options: CaseInsensitiveStringMap, schema: StructType): Table = { + val userGivenSchema = schema + new SimpleBatchTable { + override def schema(): StructType = userGivenSchema + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder(userGivenSchema) + } + } } } -class ColumnarDataSourceV2 extends DataSourceV2 with BatchReadSupportProvider { +class ColumnarDataSourceV2 extends TableProvider { + + class MyScanBuilder extends SimpleScanBuilder { - class ReadSupport extends SimpleReadSupport { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): Array[InputPartition] = { Array(RangeInputPartition(0, 50), RangeInputPartition(50, 90)) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { ColumnarReaderFactory } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport + override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder() + } } } @@ -607,22 +642,29 @@ object ColumnarReaderFactory extends PartitionReaderFactory { } } +class PartitionAwareDataSource extends TableProvider { -class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvider { + class MyScanBuilder extends SimpleScanBuilder + with SupportsReportPartitioning{ - class ReadSupport extends SimpleReadSupport with SupportsReportPartitioning { - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(): Array[InputPartition] = { // Note that we don't have same value of column `a` across partitions. Array( SpecificInputPartition(Array(1, 1, 3), Array(4, 4, 6)), SpecificInputPartition(Array(2, 4, 4), Array(6, 2, 2))) } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { SpecificReaderFactory } - override def outputPartitioning(config: ScanConfig): Partitioning = new MyPartitioning + override def outputPartitioning(): Partitioning = new MyPartitioning + } + + override def getTable(options: CaseInsensitiveStringMap): Table = new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder() + } } class MyPartitioning extends Partitioning { @@ -633,10 +675,6 @@ class PartitionAwareDataSource extends DataSourceV2 with BatchReadSupportProvide case _ => false } } - - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - new ReadSupport - } } case class SpecificInputPartition(i: Array[Int], j: Array[Int]) extends InputPartition @@ -662,10 +700,38 @@ object SpecificReaderFactory extends PartitionReaderFactory { class SchemaReadAttemptException(m: String) extends RuntimeException(m) class SimpleWriteOnlyDataSource extends SimpleWritableDataSource { - override def fullSchema(): StructType = { - // This is a bit hacky since this source implements read support but throws - // during schema retrieval. Might have to rewrite but it's done - // such so for minimised changes. - throw new SchemaReadAttemptException("read is not supported") + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new MyTable(options) { + override def schema(): StructType = { + throw new SchemaReadAttemptException("schema should not be read.") + } + } + } +} + +class ReportStatisticsDataSource extends TableProvider { + + class MyScanBuilder extends SimpleScanBuilder + with SupportsReportStatistics { + override def estimateStatistics(): Statistics = { + new Statistics { + override def sizeInBytes(): OptionalLong = OptionalLong.of(80) + + override def numRows(): OptionalLong = OptionalLong.of(10) + } + } + + override def planInputPartitions(): Array[InputPartition] = { + Array(RangeInputPartition(0, 5), RangeInputPartition(5, 10)) + } + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new SimpleBatchTable { + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder + } + } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala index 4911e3225552d..0b1e3b5fb076d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/DataSourceV2UtilsSuite.scala @@ -33,8 +33,8 @@ class DataSourceV2UtilsSuite extends SparkFunSuite { conf.setConfString(s"spark.sql.$keyPrefix.config.name", "false") conf.setConfString("spark.datasource.another.config.name", "123") conf.setConfString(s"spark.datasource.$keyPrefix.", "123") - val cs = classOf[DataSourceV2WithSessionConfig].newInstance() - val confs = DataSourceV2Utils.extractSessionConfigs(cs.asInstanceOf[DataSourceV2], conf) + val source = new DataSourceV2WithSessionConfig + val confs = DataSourceV2Utils.extractSessionConfigs(source, conf) assert(confs.size == 2) assert(confs.keySet.filter(_.startsWith("spark.datasource")).size == 0) assert(confs.keySet.filter(_.startsWith("not.exist.prefix")).size == 0) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala new file mode 100644 index 0000000000000..f79cecc76397e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/FileDataSourceV2FallBackSuite.scala @@ -0,0 +1,219 @@ +/* + * 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.sources.v2 + +import scala.collection.JavaConverters._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.sql.{AnalysisException, QueryTest} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.execution.{FileSourceScanExec, QueryExecution} +import org.apache.spark.sql.execution.datasources.{FileFormat, InsertIntoHadoopFsRelationCommand} +import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.datasources.v2.FileDataSourceV2 +import org.apache.spark.sql.execution.datasources.v2.parquet.ParquetDataSourceV2 +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.ScanBuilder +import org.apache.spark.sql.sources.v2.writer.WriteBuilder +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.{CaseInsensitiveStringMap, QueryExecutionListener} + +class DummyReadOnlyFileDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new DummyReadOnlyFileTable + } +} + +class DummyReadOnlyFileTable extends Table with SupportsRead { + override def name(): String = "dummy" + + override def schema(): StructType = StructType(Nil) + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + throw new AnalysisException("Dummy file reader") + } + + override def capabilities(): java.util.Set[TableCapability] = + Set(TableCapability.BATCH_READ, TableCapability.ACCEPT_ANY_SCHEMA).asJava +} + +class DummyWriteOnlyFileDataSourceV2 extends FileDataSourceV2 { + + override def fallbackFileFormat: Class[_ <: FileFormat] = classOf[ParquetFileFormat] + + override def shortName(): String = "parquet" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new DummyWriteOnlyFileTable + } +} + +class DummyWriteOnlyFileTable extends Table with SupportsWrite { + override def name(): String = "dummy" + + override def schema(): StructType = StructType(Nil) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = + throw new AnalysisException("Dummy file writer") + + override def capabilities(): java.util.Set[TableCapability] = + Set(TableCapability.BATCH_WRITE, TableCapability.ACCEPT_ANY_SCHEMA).asJava +} + +class FileDataSourceV2FallBackSuite extends QueryTest with SharedSQLContext { + + private val dummyParquetReaderV2 = classOf[DummyReadOnlyFileDataSourceV2].getName + private val dummyParquetWriterV2 = classOf[DummyWriteOnlyFileDataSourceV2].getName + + test("Fall back to v1 when writing to file with read only FileDataSourceV2") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + // Writing file should fall back to v1 and succeed. + df.write.format(dummyParquetReaderV2).save(path) + + // Validate write result with [[ParquetFileFormat]]. + checkAnswer(spark.read.parquet(path), df) + + // Dummy File reader should fail as expected. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path).collect() + } + assert(exception.message.equals("Dummy file reader")) + } + } + + test("Fall back read path to v1 with configuration USE_V1_SOURCE_READER_LIST") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + df.write.parquet(path) + Seq( + "foo,parquet,bar", + "ParQuet,bar,foo", + s"foobar,$dummyParquetReaderV2" + ).foreach { fallbackReaders => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> fallbackReaders) { + // Reading file should fall back to v1 and succeed. + checkAnswer(spark.read.format(dummyParquetReaderV2).load(path), df) + checkAnswer(sql(s"SELECT * FROM parquet.`$path`"), df) + } + } + + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> "foo,bar") { + // Dummy File reader should fail as DISABLED_V2_FILE_DATA_SOURCE_READERS doesn't include it. + val exception = intercept[AnalysisException] { + spark.read.format(dummyParquetReaderV2).load(path).collect() + } + assert(exception.message.equals("Dummy file reader")) + } + } + } + + test("Fall back to v1 when reading file with write only FileDataSourceV2") { + val df = spark.range(10).toDF() + withTempPath { file => + val path = file.getCanonicalPath + // Dummy File writer should fail as expected. + withSQLConf(SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> "") { + val exception = intercept[AnalysisException] { + df.write.format(dummyParquetWriterV2).save(path) + } + assert(exception.message.equals("Dummy file writer")) + } + df.write.parquet(path) + // Fallback reads to V1 + checkAnswer(spark.read.format(dummyParquetWriterV2).load(path), df) + } + } + + test("Fall back write path to v1 with configuration USE_V1_SOURCE_WRITER_LIST") { + val df = spark.range(10).toDF() + Seq( + "foo,parquet,bar", + "ParQuet,bar,foo", + s"foobar,$dummyParquetWriterV2" + ).foreach { fallbackWriters => + withSQLConf(SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> fallbackWriters) { + withTempPath { file => + val path = file.getCanonicalPath + // Writes should fall back to v1 and succeed. + df.write.format(dummyParquetWriterV2).save(path) + checkAnswer(spark.read.parquet(path), df) + } + } + } + withSQLConf(SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> "foo,bar") { + withTempPath { file => + val path = file.getCanonicalPath + // Dummy File reader should fail as USE_V1_SOURCE_READER_LIST doesn't include it. + val exception = intercept[AnalysisException] { + df.write.format(dummyParquetWriterV2).save(path) + } + assert(exception.message.equals("Dummy file writer")) + } + } + } + + test("Fallback Parquet V2 to V1") { + Seq("parquet", classOf[ParquetDataSourceV2].getCanonicalName).foreach { format => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> format, + SQLConf.USE_V1_SOURCE_WRITER_LIST.key -> format) { + val commands = ArrayBuffer.empty[(String, LogicalPlan)] + val exceptions = ArrayBuffer.empty[(String, Exception)] + val listener = new QueryExecutionListener { + override def onFailure( + funcName: String, + qe: QueryExecution, + exception: Exception): Unit = { + exceptions += funcName -> exception + } + + override def onSuccess(funcName: String, qe: QueryExecution, duration: Long): Unit = { + commands += funcName -> qe.logical + } + } + spark.listenerManager.register(listener) + + try { + withTempPath { path => + val inputData = spark.range(10) + inputData.write.format(format).save(path.getCanonicalPath) + sparkContext.listenerBus.waitUntilEmpty(1000) + assert(commands.length == 1) + assert(commands.head._1 == "save") + assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) + assert(commands.head._2.asInstanceOf[InsertIntoHadoopFsRelationCommand] + .fileFormat.isInstanceOf[ParquetFileFormat]) + val df = spark.read.format(format).load(path.getCanonicalPath) + checkAnswer(df, inputData.toDF()) + assert( + df.queryExecution.executedPlan.find(_.isInstanceOf[FileSourceScanExec]).isDefined) + } + } finally { + spark.listenerManager.unregister(listener) + } + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala index a7dfc2d1deacc..c9d2f1eef24bb 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/SimpleWritableDataSource.scala @@ -18,7 +18,7 @@ package org.apache.spark.sql.sources.v2 import java.io.{BufferedReader, InputStreamReader, IOException} -import java.util.Optional +import java.util import scala.collection.JavaConverters._ @@ -26,32 +26,27 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkContext -import org.apache.spark.sql.SaveMode import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.sources.v2.TableCapability._ import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.writer._ -import org.apache.spark.sql.types.{DataType, StructType} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.SerializableConfiguration /** * A HDFS based transactional writable data source. - * Each task writes data to `target/_temporary/queryId/$jobId-$partitionId-$attemptNumber`. - * Each job moves files from `target/_temporary/queryId/` to `target`. + * Each task writes data to `target/_temporary/uniqueId/$jobId-$partitionId-$attemptNumber`. + * Each job moves files from `target/_temporary/uniqueId/` to `target`. */ -class SimpleWritableDataSource extends DataSourceV2 - with BatchReadSupportProvider - with BatchWriteSupportProvider - with SessionConfigSupport { +class SimpleWritableDataSource extends TableProvider with SessionConfigSupport { - protected def fullSchema(): StructType = new StructType().add("i", "long").add("j", "long") + private val tableSchema = new StructType().add("i", "long").add("j", "long") override def keyPrefix: String = "simpleWritableDataSource" - class ReadSupport(path: String, conf: Configuration) extends SimpleReadSupport { - - override def fullSchema(): StructType = SimpleWritableDataSource.this.fullSchema() - - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + class MyScanBuilder(path: String, conf: Configuration) extends SimpleScanBuilder { + override def planInputPartitions(): Array[InputPartition] = { val dataPath = new Path(path) val fs = dataPath.getFileSystem(conf) if (fs.exists(dataPath)) { @@ -66,13 +61,43 @@ class SimpleWritableDataSource extends DataSourceV2 } } - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def createReaderFactory(): PartitionReaderFactory = { val serializableConf = new SerializableConfiguration(conf) new CSVReaderFactory(serializableConf) } + + override def readSchema(): StructType = tableSchema } - class WritSupport(queryId: String, path: String, conf: Configuration) extends BatchWriteSupport { + class MyWriteBuilder(path: String) extends WriteBuilder with SupportsTruncate { + private var queryId: String = _ + private var needTruncate = false + + override def withQueryId(queryId: String): WriteBuilder = { + this.queryId = queryId + this + } + + override def truncate(): WriteBuilder = { + this.needTruncate = true + this + } + + override def buildForBatch(): BatchWrite = { + val hadoopPath = new Path(path) + val hadoopConf = SparkContext.getActive.get.hadoopConfiguration + val fs = hadoopPath.getFileSystem(hadoopConf) + + if (needTruncate) { + fs.delete(hadoopPath, true) + } + + val pathStr = hadoopPath.toUri.toString + new MyBatchWrite(queryId, pathStr, hadoopConf) + } + } + + class MyBatchWrite(queryId: String, path: String, conf: Configuration) extends BatchWrite { override def createBatchWriterFactory(): DataWriterFactory = { SimpleCounter.resetCounter new CSVDataWriterFactory(path, queryId, new SerializableConfiguration(conf)) @@ -105,39 +130,28 @@ class SimpleWritableDataSource extends DataSourceV2 } } - override def createBatchReadSupport(options: DataSourceOptions): BatchReadSupport = { - val path = new Path(options.get("path").get()) - val conf = SparkContext.getActive.get.hadoopConfiguration - new ReadSupport(path.toUri.toString, conf) - } + class MyTable(options: CaseInsensitiveStringMap) + extends SimpleBatchTable with SupportsWrite { - override def createBatchWriteSupport( - queryId: String, - schema: StructType, - mode: SaveMode, - options: DataSourceOptions): Optional[BatchWriteSupport] = { - assert(!SparkContext.getActive.get.conf.getBoolean("spark.speculation", false)) + private val path = options.get("path") + private val conf = SparkContext.getActive.get.hadoopConfiguration - val path = new Path(options.get("path").get()) - val conf = SparkContext.getActive.get.hadoopConfiguration - val fs = path.getFileSystem(conf) + override def schema(): StructType = tableSchema - if (mode == SaveMode.ErrorIfExists) { - if (fs.exists(path)) { - throw new RuntimeException("data already exists.") - } + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new MyScanBuilder(new Path(path).toUri.toString, conf) } - if (mode == SaveMode.Ignore) { - if (fs.exists(path)) { - return Optional.empty() - } - } - if (mode == SaveMode.Overwrite) { - fs.delete(path, true) + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new MyWriteBuilder(path) } - val pathStr = path.toUri.toString - Optional.of(new WritSupport(queryId, pathStr, conf)) + override def capabilities(): util.Set[TableCapability] = + Set(BATCH_READ, BATCH_WRITE, TRUNCATE).asJava + } + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new MyTable(options) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala new file mode 100644 index 0000000000000..380df7a365967 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/TestInMemoryTableCatalog.scala @@ -0,0 +1,236 @@ +/* + * 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.sources.v2 + +import java.util +import java.util.concurrent.ConcurrentHashMap + +import scala.collection.JavaConverters._ +import scala.collection.mutable + +import org.apache.spark.sql.catalog.v2.{CatalogV2Implicits, Identifier, TableCatalog, TableChange} +import org.apache.spark.sql.catalog.v2.expressions.Transform +import org.apache.spark.sql.catalog.v2.utils.CatalogV2Util +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.analysis.{NoSuchTableException, TableAlreadyExistsException} +import org.apache.spark.sql.sources.v2.reader.{Batch, InputPartition, PartitionReader, PartitionReaderFactory, Scan, ScanBuilder} +import org.apache.spark.sql.sources.v2.writer.{BatchWrite, DataWriter, DataWriterFactory, SupportsTruncate, WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +// this is currently in the spark-sql module because the read and write API is not in catalyst +// TODO(rdblue): when the v2 source API is in catalyst, merge with TestTableCatalog/InMemoryTable +class TestInMemoryTableCatalog extends TableCatalog { + import CatalogV2Implicits._ + + private val tables: util.Map[Identifier, InMemoryTable] = + new ConcurrentHashMap[Identifier, InMemoryTable]() + private var _name: Option[String] = None + + override def initialize(name: String, options: CaseInsensitiveStringMap): Unit = { + _name = Some(name) + } + + override def name: String = _name.get + + override def listTables(namespace: Array[String]): Array[Identifier] = { + tables.keySet.asScala.filter(_.namespace.sameElements(namespace)).toArray + } + + override def loadTable(ident: Identifier): Table = { + Option(tables.get(ident)) match { + case Some(table) => + table + case _ => + throw new NoSuchTableException(ident) + } + } + + override def createTable( + ident: Identifier, + schema: StructType, + partitions: Array[Transform], + properties: util.Map[String, String]): Table = { + + if (tables.containsKey(ident)) { + throw new TableAlreadyExistsException(ident) + } + + if (partitions.nonEmpty) { + throw new UnsupportedOperationException( + s"Catalog $name: Partitioned tables are not supported") + } + + val table = new InMemoryTable(s"$name.${ident.quoted}", schema, properties) + + tables.put(ident, table) + + table + } + + override def alterTable(ident: Identifier, changes: TableChange*): Table = { + Option(tables.get(ident)) match { + case Some(table) => + val properties = CatalogV2Util.applyPropertiesChanges(table.properties, changes) + val schema = CatalogV2Util.applySchemaChanges(table.schema, changes) + + // fail if the last column in the schema was dropped + if (schema.fields.isEmpty) { + throw new IllegalArgumentException(s"Cannot drop all fields") + } + + val newTable = new InMemoryTable(table.name, schema, properties, table.data) + + tables.put(ident, newTable) + + newTable + case _ => + throw new NoSuchTableException(ident) + } + } + + override def dropTable(ident: Identifier): Boolean = Option(tables.remove(ident)).isDefined + + def clearTables(): Unit = { + tables.clear() + } +} + +/** + * A simple in-memory table. Rows are stored as a buffered group produced by each output task. + */ +private class InMemoryTable( + val name: String, + val schema: StructType, + override val properties: util.Map[String, String]) + extends Table with SupportsRead with SupportsWrite { + + def this( + name: String, + schema: StructType, + properties: util.Map[String, String], + data: Array[BufferedRows]) = { + this(name, schema, properties) + replaceData(data) + } + + def rows: Seq[InternalRow] = data.flatMap(_.rows) + + @volatile var data: Array[BufferedRows] = Array.empty + + def replaceData(buffers: Array[BufferedRows]): Unit = synchronized { + data = buffers + } + + override def capabilities: util.Set[TableCapability] = Set( + TableCapability.BATCH_READ, TableCapability.BATCH_WRITE, TableCapability.TRUNCATE).asJava + + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + () => new InMemoryBatchScan(data.map(_.asInstanceOf[InputPartition])) + } + + class InMemoryBatchScan(data: Array[InputPartition]) extends Scan with Batch { + override def readSchema(): StructType = schema + + override def toBatch: Batch = this + + override def planInputPartitions(): Array[InputPartition] = data + + override def createReaderFactory(): PartitionReaderFactory = BufferedRowsReaderFactory + } + + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new WriteBuilder with SupportsTruncate { + private var shouldTruncate: Boolean = false + + override def truncate(): WriteBuilder = { + shouldTruncate = true + this + } + + override def buildForBatch(): BatchWrite = { + if (shouldTruncate) TruncateAndAppend else Append + } + } + } + + private object TruncateAndAppend extends BatchWrite { + override def createBatchWriterFactory(): DataWriterFactory = { + BufferedRowsWriterFactory + } + + override def commit(messages: Array[WriterCommitMessage]): Unit = { + replaceData(messages.map(_.asInstanceOf[BufferedRows])) + } + + override def abort(messages: Array[WriterCommitMessage]): Unit = { + } + } + + private object Append extends BatchWrite { + override def createBatchWriterFactory(): DataWriterFactory = { + BufferedRowsWriterFactory + } + + override def commit(messages: Array[WriterCommitMessage]): Unit = { + replaceData(data ++ messages.map(_.asInstanceOf[BufferedRows])) + } + + override def abort(messages: Array[WriterCommitMessage]): Unit = { + } + } +} + +private class BufferedRows extends WriterCommitMessage with InputPartition with Serializable { + val rows = new mutable.ArrayBuffer[InternalRow]() +} + +private object BufferedRowsReaderFactory extends PartitionReaderFactory { + override def createReader(partition: InputPartition): PartitionReader[InternalRow] = { + new BufferedRowsReader(partition.asInstanceOf[BufferedRows]) + } +} + +private class BufferedRowsReader(partition: BufferedRows) extends PartitionReader[InternalRow] { + private var index: Int = -1 + + override def next(): Boolean = { + index += 1 + index < partition.rows.length + } + + override def get(): InternalRow = partition.rows(index) + + override def close(): Unit = {} +} + +private object BufferedRowsWriterFactory extends DataWriterFactory { + override def createWriter(partitionId: Int, taskId: Long): DataWriter[InternalRow] = { + new BufferWriter + } +} + +private class BufferWriter extends DataWriter[InternalRow] { + private val buffer = new BufferedRows + + override def write(row: InternalRow): Unit = buffer.rows.append(row.copy()) + + override def commit(): WriterCommitMessage = buffer + + override def abort(): Unit = {} +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V2WriteSupportCheckSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V2WriteSupportCheckSuite.scala new file mode 100644 index 0000000000000..1d76ee34a0e0b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/v2/V2WriteSupportCheckSuite.scala @@ -0,0 +1,149 @@ +/* + * 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.sources.v2 + +import java.util + +import scala.collection.JavaConverters._ + +import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.catalyst.analysis.{AnalysisTest, NamedRelation} +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, EqualTo, Literal} +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LeafNode, OverwriteByExpression, OverwritePartitionsDynamic} +import org.apache.spark.sql.execution.datasources.v2.{DataSourceV2Relation, V2WriteSupportCheck} +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.types.{LongType, StringType, StructType} +import org.apache.spark.sql.util.CaseInsensitiveStringMap + +class V2WriteSupportCheckSuite extends AnalysisTest { + + test("AppendData: check missing capabilities") { + val plan = AppendData.byName( + DataSourceV2Relation.create(CapabilityTable(), CaseInsensitiveStringMap.empty), TestRelation) + + val exc = intercept[AnalysisException]{ + V2WriteSupportCheck.apply(plan) + } + + assert(exc.getMessage.contains("does not support append in batch mode")) + } + + test("AppendData: check correct capabilities") { + val plan = AppendData.byName( + DataSourceV2Relation.create(CapabilityTable(BATCH_WRITE), CaseInsensitiveStringMap.empty), + TestRelation) + + V2WriteSupportCheck.apply(plan) + } + + test("Truncate: check missing capabilities") { + Seq(CapabilityTable(), + CapabilityTable(BATCH_WRITE), + CapabilityTable(TRUNCATE), + CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => + + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + Literal(true)) + + val exc = intercept[AnalysisException]{ + V2WriteSupportCheck.apply(plan) + } + + assert(exc.getMessage.contains("does not support truncate in batch mode")) + } + } + + test("Truncate: check correct capabilities") { + Seq(CapabilityTable(BATCH_WRITE, TRUNCATE), + CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER)).foreach { table => + + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + Literal(true)) + + V2WriteSupportCheck.apply(plan) + } + } + + test("OverwriteByExpression: check missing capabilities") { + Seq(CapabilityTable(), + CapabilityTable(BATCH_WRITE), + CapabilityTable(OVERWRITE_BY_FILTER)).foreach { table => + + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + EqualTo(AttributeReference("x", LongType)(), Literal(5))) + + val exc = intercept[AnalysisException]{ + V2WriteSupportCheck.apply(plan) + } + + assert(exc.getMessage.contains( + "does not support overwrite expression (`x` = 5) in batch mode")) + } + } + + test("OverwriteByExpression: check correct capabilities") { + val table = CapabilityTable(BATCH_WRITE, OVERWRITE_BY_FILTER) + val plan = OverwriteByExpression.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation, + EqualTo(AttributeReference("x", LongType)(), Literal(5))) + + V2WriteSupportCheck.apply(plan) + } + + test("OverwritePartitionsDynamic: check missing capabilities") { + Seq(CapabilityTable(), + CapabilityTable(BATCH_WRITE), + CapabilityTable(OVERWRITE_DYNAMIC)).foreach { table => + + val plan = OverwritePartitionsDynamic.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + + val exc = intercept[AnalysisException] { + V2WriteSupportCheck.apply(plan) + } + + assert(exc.getMessage.contains("does not support dynamic overwrite in batch mode")) + } + } + + test("OverwritePartitionsDynamic: check correct capabilities") { + val table = CapabilityTable(BATCH_WRITE, OVERWRITE_DYNAMIC) + val plan = OverwritePartitionsDynamic.byName( + DataSourceV2Relation.create(table, CaseInsensitiveStringMap.empty), TestRelation) + + V2WriteSupportCheck.apply(plan) + } +} + +private object V2WriteSupportCheckSuite { + val schema: StructType = new StructType().add("id", LongType).add("data", StringType) +} + +private case object TestRelation extends LeafNode with NamedRelation { + override def name: String = "source_relation" + override def output: Seq[AttributeReference] = V2WriteSupportCheckSuite.schema.toAttributes +} + +private case class CapabilityTable(_capabilities: TableCapability*) extends Table { + override def name(): String = "capability_test_table" + override def schema(): StructType = V2WriteSupportCheckSuite.schema + override def capabilities(): util.Set[TableCapability] = _capabilities.toSet.asJava +} 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 c696204cecc2c..92ec2a0c172ef 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 @@ -21,6 +21,7 @@ import java.{util => ju} import java.io.File import java.text.SimpleDateFormat import java.util.{Calendar, Date, Locale} +import java.util.concurrent.TimeUnit._ import org.apache.commons.io.FileUtils import org.scalatest.{BeforeAndAfter, Matchers} @@ -28,8 +29,8 @@ 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.DateTimeUtils import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.functions.{count, window} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.OutputMode._ @@ -43,9 +44,9 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche sqlContext.streams.active.foreach(_.stop()) } - test("EventTimeStats") { - val epsilon = 10E-6 + private val epsilon = 10E-6 + test("EventTimeStats") { val stats = EventTimeStats(max = 100, min = 10, avg = 20.0, count = 5) stats.add(80L) stats.max should be (100) @@ -62,7 +63,6 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } test("EventTimeStats: avg on large values") { - val epsilon = 10E-6 val largeValue = 10000000000L // 10B // Make sure `largeValue` will cause overflow if we use a Long sum to calc avg. assert(largeValue * largeValue != BigInt(largeValue) * BigInt(largeValue)) @@ -80,6 +80,33 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche stats.avg should be ((largeValue + 0.5) +- epsilon) } + test("EventTimeStats: zero merge zero") { + val stats = EventTimeStats.zero + val stats2 = EventTimeStats.zero + stats.merge(stats2) + stats should be (EventTimeStats.zero) + } + + test("EventTimeStats: non-zero merge zero") { + val stats = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3) + val stats2 = EventTimeStats.zero + stats.merge(stats2) + stats.max should be (10L) + stats.min should be (1L) + stats.avg should be (5.0 +- epsilon) + stats.count should be (3L) + } + + test("EventTimeStats: zero merge non-zero") { + val stats = EventTimeStats.zero + val stats2 = EventTimeStats(max = 10, min = 1, avg = 5.0, count = 3) + stats.merge(stats2) + stats.max should be (10L) + stats.min should be (1L) + stats.avg should be (5.0 +- epsilon) + stats.count should be (3L) + } + test("error on bad column") { val inputData = MemoryStream[Int].toDF() val e = intercept[AnalysisException] { @@ -278,7 +305,7 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche test("update mode") { val inputData = MemoryStream[Int] - spark.conf.set("spark.sql.shuffle.partitions", "10") + spark.conf.set(SQLConf.SHUFFLE_PARTITIONS.key, "10") val windowedAggregation = inputData.toDF() .withColumn("eventTime", $"value".cast("timestamp")) @@ -321,12 +348,13 @@ class EventTimeWatermarkSuite extends StreamTest with BeforeAndAfter with Matche } testStream(aggWithWatermark)( - AddData(input, currentTimeMs / 1000), + AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), CheckAnswer(), - AddData(input, currentTimeMs / 1000), + AddData(input, MILLISECONDS.toSeconds(currentTimeMs)), CheckAnswer(), assertEventStats { e => - assert(timestampFormat.parse(e.get("max")).getTime === (currentTimeMs / 1000) * 1000) + assert(timestampFormat.parse(e.get("max")).getTime === + SECONDS.toMillis(MILLISECONDS.toSeconds((currentTimeMs)))) val watermarkTime = timestampFormat.parse(e.get("watermark")) val monthDiff = monthsSinceEpoch(currentTime) - monthsSinceEpoch(watermarkTime) // monthsSinceEpoch is like `math.floor(num)`, so monthDiff has two possible values. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala index ed53def556cb8..d89caaaca324b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSinkSuite.scala @@ -17,21 +17,27 @@ package org.apache.spark.sql.streaming +import java.io.File +import java.nio.file.Files import java.util.Locale +import scala.collection.JavaConverters._ + import org.apache.hadoop.fs.Path +import org.apache.spark.SparkConf import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd} import org.apache.spark.sql.{AnalysisException, DataFrame} import org.apache.spark.sql.execution.DataSourceScanExec import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.v2.{BatchScanExec, DataSourceV2Relation, FileScan, FileTable} import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{IntegerType, StructField, StructType} import org.apache.spark.util.Utils -class FileStreamSinkSuite extends StreamTest { +abstract class FileStreamSinkSuite extends StreamTest { import testImplicits._ override def beforeAll(): Unit = { @@ -47,6 +53,8 @@ class FileStreamSinkSuite extends StreamTest { } } + protected def checkQueryExecution(df: DataFrame): Unit + test("unpartitioned writing and batch reading") { val inputData = MemoryStream[Int] val df = inputData.toDF() @@ -138,51 +146,12 @@ class FileStreamSinkSuite extends StreamTest { .add(StructField("id", IntegerType)) assert(outputDf.schema === expectedSchema) - // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has - // been inferred - val hadoopdFsRelations = outputDf.queryExecution.analyzed.collect { - case LogicalRelation(baseRelation: HadoopFsRelation, _, _, _) => baseRelation - } - assert(hadoopdFsRelations.size === 1) - assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex]) - assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) - assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) - // Verify the data is correctly read checkDatasetUnorderly( outputDf.as[(Int, Int)], (1000, 1), (2000, 2), (3000, 3)) - /** Check some condition on the partitions of the FileScanRDD generated by a DF */ - def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { - val getFileScanRDD = df.queryExecution.executedPlan.collect { - case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] => - scan.inputRDDs().head.asInstanceOf[FileScanRDD] - }.headOption.getOrElse { - fail(s"No FileScan in query\n${df.queryExecution}") - } - func(getFileScanRDD.filePartitions) - } - - // Read without pruning - checkFileScanPartitions(outputDf) { partitions => - // There should be as many distinct partition values as there are distinct ids - assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) - } - - // Read with pruning, should read only files in partition dir id=1 - checkFileScanPartitions(outputDf.filter("id = 1")) { partitions => - val filesToBeRead = partitions.flatMap(_.files) - assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/"))) - assert(filesToBeRead.map(_.partitionValues).distinct.size === 1) - } - - // Read with pruning, should read only files in partition dir id=1 and id=2 - checkFileScanPartitions(outputDf.filter("id in (1,2)")) { partitions => - val filesToBeRead = partitions.flatMap(_.files) - assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/"))) - assert(filesToBeRead.map(_.partitionValues).distinct.size === 2) - } + checkQueryExecution(outputDf) } finally { if (query != null) { query.stop() @@ -270,6 +239,7 @@ class FileStreamSinkSuite extends StreamTest { "CAST(start as BIGINT) AS start", "CAST(end as BIGINT) AS end", "count") + .orderBy("start") // sort the DataFrame in order to compare with the expected one. checkDataset( outputDf.as[(Long, Long, Long)], expectedResult.map(x => (x._1._1, x._1._2, x._2)): _*) @@ -454,4 +424,142 @@ class FileStreamSinkSuite extends StreamTest { } } } + + test("special characters in output path") { + withTempDir { tempDir => + val checkpointDir = new File(tempDir, "chk") + val outputDir = new File(tempDir, "output @#output") + val inputData = MemoryStream[Int] + inputData.addData(1, 2, 3) + val q = inputData.toDF() + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("parquet") + .start(outputDir.getCanonicalPath) + try { + q.processAllAvailable() + } finally { + q.stop() + } + // The "_spark_metadata" directory should be in "outputDir" + assert(outputDir.listFiles.map(_.getName).contains(FileStreamSink.metadataDir)) + val outputDf = spark.read.parquet(outputDir.getCanonicalPath).as[Int] + checkDatasetUnorderly(outputDf, 1, 2, 3) + } + } + + testQuietly("cleanup incomplete output for aborted task") { + withTempDir { tempDir => + val checkpointDir = new File(tempDir, "chk") + val outputDir = new File(tempDir, "output") + val inputData = MemoryStream[Int] + inputData.addData(1, 2, 3) + val q = inputData.toDS().map(_ / 0) + .writeStream + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .format("parquet") + .start(outputDir.getCanonicalPath) + + intercept[StreamingQueryException] { + try { + q.processAllAvailable() + } finally { + q.stop() + } + } + + val outputFiles = Files.walk(outputDir.toPath).iterator().asScala + .filter(_.toString.endsWith(".parquet")) + assert(outputFiles.toList.isEmpty, "Incomplete files should be cleaned up.") + } + } +} + +class FileStreamSinkV1Suite extends FileStreamSinkSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "csv,json,orc,text,parquet") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "csv,json,orc,text,parquet") + + override def checkQueryExecution(df: DataFrame): Unit = { + // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has + // been inferred + val hadoopdFsRelations = df.queryExecution.analyzed.collect { + case LogicalRelation(baseRelation: HadoopFsRelation, _, _, _) => baseRelation + } + assert(hadoopdFsRelations.size === 1) + assert(hadoopdFsRelations.head.location.isInstanceOf[MetadataLogFileIndex]) + assert(hadoopdFsRelations.head.partitionSchema.exists(_.name == "id")) + assert(hadoopdFsRelations.head.dataSchema.exists(_.name == "value")) + + /** Check some condition on the partitions of the FileScanRDD generated by a DF */ + def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + val getFileScanRDD = df.queryExecution.executedPlan.collect { + case scan: DataSourceScanExec if scan.inputRDDs().head.isInstanceOf[FileScanRDD] => + scan.inputRDDs().head.asInstanceOf[FileScanRDD] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + func(getFileScanRDD.filePartitions) + } + + // Read without pruning + checkFileScanPartitions(df) { partitions => + // There should be as many distinct partition values as there are distinct ids + assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) + } + + // Read with pruning, should read only files in partition dir id=1 + checkFileScanPartitions(df.filter("id = 1")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(filesToBeRead.map(_.filePath).forall(_.contains("/id=1/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 1) + } + + // Read with pruning, should read only files in partition dir id=1 and id=2 + checkFileScanPartitions(df.filter("id in (1,2)")) { partitions => + val filesToBeRead = partitions.flatMap(_.files) + assert(!filesToBeRead.map(_.filePath).exists(_.contains("/id=3/"))) + assert(filesToBeRead.map(_.partitionValues).distinct.size === 2) + } + } +} + +class FileStreamSinkV2Suite extends FileStreamSinkSuite { + override protected def sparkConf: SparkConf = + super + .sparkConf + .set(SQLConf.USE_V1_SOURCE_READER_LIST, "") + .set(SQLConf.USE_V1_SOURCE_WRITER_LIST, "") + + override def checkQueryExecution(df: DataFrame): Unit = { + // Verify that MetadataLogFileIndex is being used and the correct partitioning schema has + // been inferred + val table = df.queryExecution.analyzed.collect { + case DataSourceV2Relation(table: FileTable, _, _) => table + } + assert(table.size === 1) + assert(table.head.fileIndex.isInstanceOf[MetadataLogFileIndex]) + assert(table.head.fileIndex.partitionSchema.exists(_.name == "id")) + assert(table.head.dataSchema.exists(_.name == "value")) + + /** Check some condition on the partitions of the FileScanRDD generated by a DF */ + def checkFileScanPartitions(df: DataFrame)(func: Seq[FilePartition] => Unit): Unit = { + val fileScan = df.queryExecution.executedPlan.collect { + case batch: BatchScanExec if batch.scan.isInstanceOf[FileScan] => + batch.scan.asInstanceOf[FileScan] + }.headOption.getOrElse { + fail(s"No FileScan in query\n${df.queryExecution}") + } + func(fileScan.planInputPartitions().map(_.asInstanceOf[FilePartition])) + } + + // Read without pruning + checkFileScanPartitions(df) { partitions => + // There should be as many distinct partition values as there are distinct ids + assert(partitions.flatMap(_.files.map(_.partitionValues)).distinct.size === 3) + } + // TODO: test partition pruning when file source V2 supports it. + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala index d4bd9c7987f2d..72f893845172d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/FileStreamSourceSuite.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.FileStreamSource.{FileEntry, SeenFilesMap} +import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.ExistsThrowsExceptionFileSystem._ import org.apache.spark.sql.streaming.util.StreamManualClock @@ -48,21 +49,33 @@ abstract class FileStreamSourceTest * `FileStreamSource` actually being used in the execution. */ abstract class AddFileData extends AddData { + private val _qualifiedBasePath = PrivateMethod[Path]('qualifiedBasePath) + + private def isSamePath(fileSource: FileStreamSource, srcPath: File): Boolean = { + val path = (fileSource invokePrivate _qualifiedBasePath()).toString.stripPrefix("file:") + path == srcPath.getCanonicalPath + } + override def addData(query: Option[StreamExecution]): (Source, Offset) = { require( query.nonEmpty, "Cannot add data when there is no query for finding the active file stream source") val sources = getSourcesFromStreamingQuery(query.get) - if (sources.isEmpty) { + val source = if (sources.isEmpty) { throw new Exception( "Could not find file source in the StreamExecution logical plan to add data to") - } else if (sources.size > 1) { - throw new Exception( - "Could not select the file source in the StreamExecution logical plan as there" + - "are multiple file sources:\n\t" + sources.mkString("\n\t")) + } else if (sources.size == 1) { + sources.head + } else { + val matchedSources = sources.filter(isSamePath(_, src)) + if (matchedSources.size != 1) { + throw new Exception( + "Could not select the file source in StreamExecution as there are multiple" + + s" file sources and none / more than one matches $src:\n" + sources.mkString("\n")) + } + matchedSources.head } - val source = sources.head val newOffset = source.withBatchingLocked { addData(source) new FileStreamSourceOffset(source.currentLogOffset + 1) @@ -71,6 +84,9 @@ abstract class FileStreamSourceTest (source, newOffset) } + /** Source directory to add file data to */ + protected def src: File + protected def addData(source: FileStreamSource): Unit } @@ -180,7 +196,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { import testImplicits._ - override val streamingTimeout = 20.seconds + override val streamingTimeout = 80.seconds /** Use `format` and `path` to create FileStreamSource via DataFrameReader */ private def createFileStreamSource( @@ -467,6 +483,25 @@ class FileStreamSourceSuite extends FileStreamSourceTest { } } + test("Option pathGlobFilter") { + val testTableName = "FileStreamSourceTest" + withTable(testTableName) { + withTempPath { output => + Seq("foo").toDS().write.text(output.getCanonicalPath) + Seq("bar").toDS().write.mode("append").orc(output.getCanonicalPath) + val df = spark.readStream.option("pathGlobFilter", "*.txt") + .format("text").load(output.getCanonicalPath) + val query = df.writeStream.format("memory").queryName(testTableName).start() + try { + query.processAllAvailable() + checkDatasetUnorderly(spark.table(testTableName).as[String], "foo") + } finally { + query.stop() + } + } + } + } + test("read from textfile") { withTempDirs { case (src, tmp) => val textStream = spark.readStream.textFile(src.getCanonicalPath) @@ -1275,7 +1310,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { val start = startId.map(new FileStreamSourceOffset(_)) val end = FileStreamSourceOffset(endId) - withSQLConf("spark.sql.streaming.unsupportedOperationCheck" -> "false") { + withSQLConf(SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED.key -> "false") { assert(fileSource.getBatch(start, end).as[String].collect().toSeq === expected) } } @@ -1360,7 +1395,7 @@ class FileStreamSourceSuite extends FileStreamSourceTest { options = srcOptions) val clock = new StreamManualClock() testStream(fileStream)( - StartStream(trigger = ProcessingTime(10), triggerClock = clock), + StartStream(trigger = Trigger.ProcessingTime(10), triggerClock = clock), AssertOnQuery { _ => // Block until the first batch finishes. eventually(timeout(streamingTimeout)) { @@ -1494,6 +1529,54 @@ class FileStreamSourceSuite extends FileStreamSourceTest { newSource.getBatch(None, FileStreamSourceOffset(1)) } } + + test("SPARK-26629: multiple file sources work with restarts when a source does not have data") { + withTempDirs { case (dir, tmp) => + val sourceDir1 = new File(dir, "source1") + val sourceDir2 = new File(dir, "source2") + sourceDir1.mkdirs() + sourceDir2.mkdirs() + + val source1 = createFileStream("text", s"${sourceDir1.getCanonicalPath}") + val source2 = createFileStream("text", s"${sourceDir2.getCanonicalPath}") + val unioned = source1.union(source2) + + def addMultiTextFileData( + source1Content: String, + source2Content: String): StreamAction = { + val actions = Seq( + AddTextFileData(source1Content, sourceDir1, tmp), + AddTextFileData(source2Content, sourceDir2, tmp) + ).filter(_.content != null) // don't write to a source dir if no content specified + StreamProgressLockedActions(actions, desc = actions.mkString("[ ", " | ", " ]")) + } + + testStream(unioned)( + StartStream(), + addMultiTextFileData(source1Content = "source1_0", source2Content = "source2_0"), + CheckNewAnswer("source1_0", "source2_0"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_1", source2Content = null), + CheckNewAnswer("source1_1"), + StopStream, + + // Restart after a batch with one file source having no new data. + // This restart is needed to hit the issue in SPARK-26629. + + StartStream(), + addMultiTextFileData(source1Content = null, source2Content = "source2_2"), + CheckNewAnswer("source2_2"), + StopStream, + + StartStream(), + addMultiTextFileData(source1Content = "source1_3", source2Content = "source2_3"), + CheckNewAnswer("source1_3", "source2_3"), + StopStream + ) + } + } } class FileStreamSourceStressTestSuite extends FileStreamSourceTest { 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 43463a84093ce..df7e9217f9140 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 @@ -267,7 +267,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { val state2 = GroupStateImpl.createForStreaming( initState, 1000, 1000, timeoutConf, hasTimedOut = true, watermarkPresent = false) - assert(state2.hasTimedOut === true) + assert(state2.hasTimedOut) } // for batch queries @@ -1162,7 +1162,7 @@ class FlatMapGroupsWithStateSuite extends StateStoreMetricsTest { test(s"InputProcessor - process timed out state - $testName") { val mapGroupsFunc = (key: Int, values: Iterator[Int], state: GroupState[Int]) => { - assert(state.hasTimedOut === true, "hasTimedOut not true") + assert(state.hasTimedOut, "hasTimedOut not true") assert(values.isEmpty, "values not empty") stateUpdates(state) Iterator.empty 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 f55ddb5419d20..1ed2599444c5d 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 @@ -29,15 +29,15 @@ import org.apache.commons.io.FileUtils import org.apache.hadoop.conf.Configuration import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkConf, SparkContext, TaskContext} +import org.apache.spark.{SparkConf, SparkContext, TaskContext, TestUtils} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.plans.logical.Range import org.apache.spark.sql.catalyst.streaming.InternalOutputModes +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.command.ExplainCommand import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.continuous.ContinuousExecution -import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream +import org.apache.spark.sql.execution.streaming.sources.{ContinuousMemoryStream, MemorySink} import org.apache.spark.sql.execution.streaming.state.{StateStore, StateStoreConf, StateStoreId, StateStoreProvider} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -96,18 +96,16 @@ class StreamSuite extends StreamTest { val streamingRelation = spark.readStream.format("rate").load().logicalPlan collect { case s: StreamingRelationV2 => s } - assert(streamingRelation.nonEmpty, "cannot find StreamingExecutionRelation") + assert(streamingRelation.nonEmpty, "cannot find StreamingRelationV2") assert( streamingRelation.head.computeStats.sizeInBytes == spark.sessionState.conf.defaultSizeInBytes) } test("StreamingExecutionRelation.computeStats") { - val streamingExecutionRelation = MemoryStream[Int].toDF.logicalPlan collect { - case s: StreamingExecutionRelation => s - } - assert(streamingExecutionRelation.nonEmpty, "cannot find StreamingExecutionRelation") - assert(streamingExecutionRelation.head.computeStats.sizeInBytes - == spark.sessionState.conf.defaultSizeInBytes) + val memoryStream = MemoryStream[Int] + val executionRelation = StreamingExecutionRelation( + memoryStream, memoryStream.encoder.schema.toAttributes)(memoryStream.sqlContext.sparkSession) + assert(executionRelation.computeStats.sizeInBytes == spark.sessionState.conf.defaultSizeInBytes) } test("explain join with a normal source") { @@ -221,8 +219,12 @@ class StreamSuite extends StreamTest { } val df = spark.readStream.format(classOf[FakeDefaultSource].getName).load() - assertDF(df) - assertDF(df) + Seq("", "parquet").foreach { useV1SourceReader => + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> useV1SourceReader) { + assertDF(df) + assertDF(df) + } + } } test("Within the same streaming query, one StreamingRelation should only be transformed to one " + @@ -312,7 +314,7 @@ class StreamSuite extends StreamTest { val inputData = MemoryStream[Int] testStream(inputData.toDS())( - StartStream(ProcessingTime("10 seconds"), new StreamManualClock), + StartStream(Trigger.ProcessingTime("10 seconds"), new StreamManualClock), /* -- batch 0 ----------------------- */ // Add some data in batch 0 @@ -353,7 +355,7 @@ class StreamSuite extends StreamTest { /* Stop then restart the Stream */ StopStream, - StartStream(ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), + StartStream(Trigger.ProcessingTime("10 seconds"), new StreamManualClock(60 * 1000)), /* -- batch 1 no rerun ----------------- */ // batch 1 would not re-run because the latest batch id logged in commit log is 1 @@ -495,9 +497,9 @@ class StreamSuite extends StreamTest { val explainWithoutExtended = q.explainInternal(false) // `extended = false` only displays the physical plan. - assert("Streaming RelationV2 MemoryStreamDataSource".r + assert("StreamingDataSourceV2Relation".r .findAllMatchIn(explainWithoutExtended).size === 0) - assert("ScanV2 MemoryStreamDataSource".r + assert("BatchScan".r .findAllMatchIn(explainWithoutExtended).size === 1) // Use "StateStoreRestore" to verify that it does output a streaming physical plan assert(explainWithoutExtended.contains("StateStoreRestore")) @@ -505,9 +507,9 @@ class StreamSuite extends StreamTest { val explainWithExtended = q.explainInternal(true) // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical // plan. - assert("Streaming RelationV2 MemoryStreamDataSource".r + assert("StreamingDataSourceV2Relation".r .findAllMatchIn(explainWithExtended).size === 3) - assert("ScanV2 MemoryStreamDataSource".r + assert("BatchScan".r .findAllMatchIn(explainWithExtended).size === 1) // Use "StateStoreRestore" to verify that it does output a streaming physical plan assert(explainWithExtended.contains("StateStoreRestore")) @@ -550,17 +552,17 @@ class StreamSuite extends StreamTest { val explainWithoutExtended = q.explainInternal(false) // `extended = false` only displays the physical plan. - assert("Streaming RelationV2 ContinuousMemoryStream".r + assert("StreamingDataSourceV2Relation".r .findAllMatchIn(explainWithoutExtended).size === 0) - assert("ScanV2 ContinuousMemoryStream".r + assert("ContinuousScan".r .findAllMatchIn(explainWithoutExtended).size === 1) val explainWithExtended = q.explainInternal(true) // `extended = true` displays 3 logical plans (Parsed/Optimized/Optimized) and 1 physical // plan. - assert("Streaming RelationV2 ContinuousMemoryStream".r + assert("StreamingDataSourceV2Relation".r .findAllMatchIn(explainWithExtended).size === 3) - assert("ScanV2 ContinuousMemoryStream".r + assert("ContinuousScan".r .findAllMatchIn(explainWithExtended).size === 1) } finally { q.stop() @@ -869,7 +871,7 @@ class StreamSuite extends StreamTest { testQuietly("specify custom state store provider") { val providerClassName = classOf[TestStateStoreProvider].getCanonicalName - withSQLConf("spark.sql.streaming.stateStore.providerClass" -> providerClassName) { + withSQLConf(SQLConf.STATE_STORE_PROVIDER_CLASS.key -> providerClassName) { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() val query = df.writeStream.outputMode("complete").format("memory").queryName("name").start() @@ -878,17 +880,17 @@ class StreamSuite extends StreamTest { query.awaitTermination() } - assert(e.getMessage.contains(providerClassName)) - assert(e.getMessage.contains("instantiated")) + TestUtils.assertExceptionMsg(e, providerClassName) + TestUtils.assertExceptionMsg(e, "instantiated") } } testQuietly("custom state store provider read from offset log") { val input = MemoryStream[Int] val df = input.toDS().groupBy().count() - val providerConf1 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf1 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> "org.apache.spark.sql.execution.streaming.state.HDFSBackedStateStoreProvider" - val providerConf2 = "spark.sql.streaming.stateStore.providerClass" -> + val providerConf2 = SQLConf.STATE_STORE_PROVIDER_CLASS.key -> classOf[TestStateStoreProvider].getCanonicalName def runQuery(queryName: String, checkpointLoc: String): Unit = { @@ -1082,6 +1084,47 @@ class StreamSuite extends StreamTest { assert(query.exception.isEmpty) } } + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " + + " to Dataset - use v2 sink") { + testCurrentTimestampOnStreamingQuery() + } + + test("SPARK-26379 Structured Streaming - Exception on adding current_timestamp " + + " to Dataset - use v1 sink") { + testCurrentTimestampOnStreamingQuery() + } + + private def testCurrentTimestampOnStreamingQuery(): Unit = { + val input = MemoryStream[Int] + val df = input.toDS().withColumn("cur_timestamp", lit(current_timestamp())) + + def assertBatchOutputAndUpdateLastTimestamp( + rows: Seq[Row], + curTimestamp: Long, + curDate: Int, + expectedValue: Int): Long = { + assert(rows.size === 1) + val row = rows.head + assert(row.getInt(0) === expectedValue) + assert(row.getTimestamp(1).getTime >= curTimestamp) + row.getTimestamp(1).getTime + } + + var lastTimestamp = System.currentTimeMillis() + val currentDate = DateTimeUtils.millisToDays(lastTimestamp) + testStream(df) ( + AddData(input, 1), + CheckLastBatch { rows: Seq[Row] => + lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 1) + }, + Execute { _ => Thread.sleep(1000) }, + AddData(input, 2), + CheckLastBatch { rows: Seq[Row] => + lastTimestamp = assertBatchOutputAndUpdateLastTimestamp(rows, lastTimestamp, currentDate, 2) + } + ) + } } abstract class FakeSource extends StreamSourceProvider { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala index d878c345c2988..3a4414f6e6ecf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamTest.scala @@ -17,8 +17,6 @@ package org.apache.spark.sql.streaming -import java.lang.Thread.UncaughtExceptionHandler - import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.language.experimental.macros @@ -42,9 +40,9 @@ import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous.{ContinuousExecution, EpochCoordinatorRef, IncrementAndGetEpoch} -import org.apache.spark.sql.execution.streaming.sources.MemorySinkV2 +import org.apache.spark.sql.execution.streaming.sources.MemorySink import org.apache.spark.sql.execution.streaming.state.StateStore -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2, SparkDataStream} import org.apache.spark.sql.streaming.StreamingQueryListener._ import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.util.{Clock, SystemClock, Utils} @@ -87,10 +85,9 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be } protected val defaultTrigger = Trigger.ProcessingTime(0) - protected val defaultUseV2Sink = false /** How long to wait for an active stream to catch up when checking a result. */ - val streamingTimeout = 10.seconds + val streamingTimeout = 60.seconds /** A trait for actions that can be performed while testing a streaming DataFrame. */ trait StreamAction @@ -127,7 +124,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be * the active query, and then return the source object the data was added, as well as the * offset of added data. */ - def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) + def addData(query: Option[StreamExecution]): (SparkDataStream, OffsetV2) } /** A trait that can be extended when testing a source. */ @@ -138,7 +135,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be case class AddDataMemory[A](source: MemoryStreamBase[A], data: Seq[A]) extends AddData { override def toString: String = s"AddData to $source: ${data.mkString(",")}" - override def addData(query: Option[StreamExecution]): (BaseStreamingSource, Offset) = { + override def addData(query: Option[StreamExecution]): (SparkDataStream, OffsetV2) = { (source, source.addData(data)) } } @@ -295,7 +292,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be /** Execute arbitrary code */ object Execute { def apply(name: String)(func: StreamExecution => Any): AssertOnQuery = - AssertOnQuery(query => { func(query); true }, "name") + AssertOnQuery(query => { func(query); true }, name) def apply(func: StreamExecution => Any): AssertOnQuery = apply("Execute")(func) } @@ -328,8 +325,7 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be */ def testStream( _stream: Dataset[_], - outputMode: OutputMode = OutputMode.Append, - useV2Sink: Boolean = defaultUseV2Sink)(actions: StreamAction*): Unit = synchronized { + outputMode: OutputMode = OutputMode.Append)(actions: StreamAction*): Unit = synchronized { import org.apache.spark.sql.streaming.util.StreamManualClock // `synchronized` is added to prevent the user from calling multiple `testStream`s concurrently @@ -341,8 +337,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be var pos = 0 var currentStream: StreamExecution = null var lastStream: StreamExecution = null - val awaiting = new mutable.HashMap[Int, Offset]() // source index -> offset to wait for - val sink = if (useV2Sink) new MemorySinkV2 else new MemorySink(stream.schema, outputMode) + val awaiting = new mutable.HashMap[Int, OffsetV2]() // source index -> offset to wait for + val sink = new MemorySink val resetConfValues = mutable.Map[String, Option[String]]() val defaultCheckpointLocation = Utils.createTempDir(namePrefix = "streaming.metadata").getCanonicalPath @@ -355,11 +351,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be val listener = new StreamingQueryListener { override def onQueryStarted(event: QueryStartedEvent): Unit = { // Note: this assumes there is only one query active in the `testStream` method. - Thread.currentThread.setUncaughtExceptionHandler(new UncaughtExceptionHandler { - override def uncaughtException(t: Thread, e: Throwable): Unit = { - streamThreadDeathCause = e - } - }) + Thread.currentThread.setUncaughtExceptionHandler( + (_: Thread, e: Throwable) => streamThreadDeathCause = e) } override def onQueryProgress(event: QueryProgressEvent): Unit = {} @@ -395,10 +388,8 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be } def testState = { - val sinkDebugString = sink match { - case s: MemorySink => s.toDebugString - case s: MemorySinkV2 => s.toDebugString - } + val sinkDebugString = sink.toDebugString + s""" |== Progress == |$testActions @@ -688,8 +679,14 @@ trait StreamTest extends QueryTest with SharedSQLContext with TimeLimits with Be def findSourceIndex(plan: LogicalPlan): Option[Int] = { plan .collect { + // v1 source case r: StreamingExecutionRelation => r.source - case r: StreamingDataSourceV2Relation => r.readSupport + // v2 source + case r: StreamingDataSourceV2Relation => r.stream + // We can add data to memory stream before starting it. Then the input plan has + // not been processed by the streaming engine and contains `StreamingRelationV2`. + case r: StreamingRelationV2 if r.sourceName == "memory" => + r.table.asInstanceOf[MemoryStreamTable].stream } .zipWithIndex .find(_._1 == source) 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 97dbb9b0360ec..134e61ed12a21 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 @@ -21,7 +21,7 @@ import java.io.File import java.util.{Locale, TimeZone} import org.apache.commons.io.FileUtils -import org.scalatest.{Assertions, BeforeAndAfterAll} +import org.scalatest.Assertions import org.apache.spark.{SparkEnv, SparkException} import org.apache.spark.rdd.BlockRDD @@ -32,7 +32,8 @@ import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.execution.{SparkPlan, UnaryExecNode} import org.apache.spark.sql.execution.exchange.Exchange import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.state.{StateStore, StreamingAggregationStateManager} +import org.apache.spark.sql.execution.streaming.sources.MemorySink +import org.apache.spark.sql.execution.streaming.state.StreamingAggregationStateManager import org.apache.spark.sql.expressions.scalalang.typed import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf @@ -207,15 +208,15 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { AddData(inputData, 1), CheckLastBatch((1, 1), (2, 1)), AssertOnQuery { _.stateNodes.size === 1 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numUpdatedStateRows").value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numTotalStateRows").value === 2 }, AddData(inputData, 2, 3), CheckLastBatch((2, 2), (3, 2), (4, 1)), AssertOnQuery { _.stateNodes.size === 1 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 3 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics("numUpdatedStateRows").value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics("numTotalStateRows").value === 4 } ) // Test with Complete mode @@ -224,15 +225,15 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { AddData(inputData, 1), CheckLastBatch((1, 1), (2, 1)), AssertOnQuery { _.stateNodes.size === 1 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 2 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 2 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numUpdatedStateRows").value === 2 }, + AssertOnQuery { _.stateNodes.head.metrics("numTotalStateRows").value === 2 }, AddData(inputData, 2, 3), CheckLastBatch((1, 1), (2, 2), (3, 2), (4, 1)), AssertOnQuery { _.stateNodes.size === 1 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numOutputRows").get.value === 4 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numUpdatedStateRows").get.value === 3 }, - AssertOnQuery { _.stateNodes.head.metrics.get("numTotalStateRows").get.value === 4 } + AssertOnQuery { _.stateNodes.head.metrics("numOutputRows").value === 4 }, + AssertOnQuery { _.stateNodes.head.metrics("numUpdatedStateRows").value === 3 }, + AssertOnQuery { _.stateNodes.head.metrics("numTotalStateRows").value === 4 } ) } @@ -344,16 +345,15 @@ class StreamingAggregationSuite extends StateStoreMetricsTest with Assertions { testWithAllStateVersions("prune results by current_date, complete mode") { import testImplicits._ val clock = new StreamManualClock - val tz = TimeZone.getDefault.getID val inputData = MemoryStream[Long] val aggregated = inputData.toDF() - .select(to_utc_timestamp(from_unixtime('value * DateTimeUtils.SECONDS_PER_DAY), tz)) - .toDF("value") + .select(($"value" * DateTimeUtils.SECONDS_PER_DAY).cast("timestamp").as("value")) .groupBy($"value") .agg(count("*")) - .where($"value".cast("date") >= date_sub(current_date(), 10)) - .select(($"value".cast("long") / DateTimeUtils.SECONDS_PER_DAY).cast("long"), $"count(1)") + .where($"value".cast("date") >= date_sub(current_timestamp().cast("date"), 10)) + .select( + ($"value".cast("long") / DateTimeUtils.SECONDS_PER_DAY).cast("long"), $"count(1)") testStream(aggregated, Complete)( StartStream(Trigger.ProcessingTime("10 day"), triggerClock = clock), // advance clock to 10 days, should retain all keys diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala index c5cc8df4356a8..42fe9f34ee3ec 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingJoinSuite.scala @@ -350,7 +350,7 @@ class StreamingInnerJoinSuite extends StreamTest with StateStoreMetricsTest with withTempDir { tempDir => val queryId = UUID.randomUUID val opId = 0 - val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextString(10)).toString + val path = Utils.createDirectory(tempDir.getAbsolutePath, Random.nextFloat.toString).toString val stateInfo = StatefulOperatorStateInfo(path, queryId, opId, 0L, 5) implicit val sqlContext = spark.sqlContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala index fe77a1b4469c5..422223b9522d9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenerSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.streaming import java.util.UUID import scala.collection.mutable -import scala.language.reflectiveCalls import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter @@ -82,7 +81,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { testStream(df, OutputMode.Append)( // Start event generated when query started - StartStream(ProcessingTime(100), triggerClock = clock), + StartStream(Trigger.ProcessingTime(100), triggerClock = clock), AssertOnQuery { query => assert(listener.startEvent !== null) assert(listener.startEvent.id === query.id) @@ -124,7 +123,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { }, // Termination event generated with exception message when stopped with error - StartStream(ProcessingTime(100), triggerClock = clock), + StartStream(Trigger.ProcessingTime(100), triggerClock = clock), AssertStreamExecThreadToWaitForClock(), AddData(inputData, 0), AdvanceManualClock(100), // process bad data @@ -180,7 +179,7 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val listeners = (1 to 5).map(_ => new EventCollector) try { listeners.foreach(listener => spark.streams.addListener(listener)) - testStream(df, OutputMode.Append, useV2Sink = true)( + testStream(df, OutputMode.Append)( StartStream(Trigger.Continuous(1000)), StopStream, AssertOnQuery { query => @@ -215,14 +214,14 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { val listener2 = new EventCollector spark.streams.addListener(listener1) - assert(isListenerActive(listener1) === true) + assert(isListenerActive(listener1)) assert(isListenerActive(listener2) === false) spark.streams.addListener(listener2) - assert(isListenerActive(listener1) === true) - assert(isListenerActive(listener2) === true) + assert(isListenerActive(listener1)) + assert(isListenerActive(listener2)) spark.streams.removeListener(listener1) assert(isListenerActive(listener1) === false) - assert(isListenerActive(listener2) === true) + assert(isListenerActive(listener2)) } finally { addedListeners().foreach(spark.streams.removeListener) } @@ -297,8 +296,8 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } spark.streams.addListener(listener) try { + var numTriggers = 0 val input = new MemoryStream[Int](0, sqlContext) { - @volatile var numTriggers = 0 override def latestOffset(): OffsetV2 = { numTriggers += 1 super.latestOffset() @@ -306,13 +305,13 @@ class StreamingQueryListenerSuite extends StreamTest with BeforeAndAfter { } val clock = new StreamManualClock() val actions = mutable.ArrayBuffer[StreamAction]() - actions += StartStream(trigger = ProcessingTime(10), triggerClock = clock) + actions += StartStream(trigger = Trigger.ProcessingTime(10), triggerClock = clock) for (_ <- 1 to 100) { actions += AdvanceManualClock(10) } actions += AssertOnQuery { _ => eventually(timeout(streamingTimeout)) { - assert(input.numTriggers > 100) // at least 100 triggers have occurred + assert(numTriggers > 100) // at least 100 triggers have occurred } true } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala index 1aaf8a9aa2d55..da2f221aaf101 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryListenersConfSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.streaming -import scala.language.reflectiveCalls - import org.scalatest.BeforeAndAfter import org.apache.spark.SparkConf import org.apache.spark.sql.execution.streaming._ +import org.apache.spark.sql.internal.StaticSQLConf.STREAMING_QUERY_LISTENERS import org.apache.spark.sql.streaming.StreamingQueryListener._ @@ -30,9 +29,8 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { import testImplicits._ - override protected def sparkConf: SparkConf = - super.sparkConf.set("spark.sql.streaming.streamingQueryListeners", + super.sparkConf.set(STREAMING_QUERY_LISTENERS.key, "org.apache.spark.sql.streaming.TestListener") test("test if the configured query lister is loaded") { @@ -41,6 +39,8 @@ class StreamingQueryListenersConfSuite extends StreamTest with BeforeAndAfter { StopStream ) + spark.sparkContext.listenerBus.waitUntilEmpty(5000) + assert(TestListener.queryStartedEvent != null) assert(TestListener.queryTerminatedEvent != null) } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala index 46eec736d4027..b26d2556b2e36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQueryManagerSuite.scala @@ -23,34 +23,37 @@ import scala.concurrent.Future import scala.util.Random import scala.util.control.NonFatal -import org.scalatest.BeforeAndAfter -import org.scalatest.concurrent.Eventually._ import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.time.Span import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkException -import org.apache.spark.sql.{AnalysisException, Dataset} +import org.apache.spark.sql.Dataset +import org.apache.spark.sql.execution.datasources.v2.StreamingDataSourceV2Relation import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.streaming.util.BlockingSource import org.apache.spark.util.Utils -class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { +class StreamingQueryManagerSuite extends StreamTest { import AwaitTerminationTester._ import testImplicits._ override val streamingTimeout = 20.seconds - before { + override def beforeEach(): Unit = { + super.beforeEach() assert(spark.streams.active.isEmpty) spark.streams.resetTerminated() } - after { - assert(spark.streams.active.isEmpty) - spark.streams.resetTerminated() + override def afterEach(): Unit = { + try { + assert(spark.streams.active.isEmpty) + spark.streams.resetTerminated() + } finally { + super.afterEach() + } } testQuietly("listing") { @@ -84,7 +87,7 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { } } - testQuietly("awaitAnyTermination without timeout and resetTerminated") { + testRetry("awaitAnyTermination without timeout and resetTerminated") { val datasets = Seq.fill(5)(makeDataset._2) withQueriesOn(datasets: _*) { queries => require(queries.size === datasets.size) @@ -94,7 +97,7 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { testAwaitAnyTermination(ExpectBlocked) // Stop a query asynchronously and see if it is reported through awaitAnyTermination - val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false) + val q1 = stopRandomQueryAsync(stopAfter = 100.milliseconds, withError = false) testAwaitAnyTermination(ExpectNotBlocked) require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned @@ -107,7 +110,7 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { // Terminate a query asynchronously with exception and see awaitAnyTermination throws // the exception - val q2 = stopRandomQueryAsync(100 milliseconds, withError = true) + val q2 = stopRandomQueryAsync(100.milliseconds, withError = true) testAwaitAnyTermination(ExpectException[SparkException]) require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned @@ -120,10 +123,10 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws // the exception - val q3 = stopRandomQueryAsync(10 milliseconds, withError = false) + val q3 = stopRandomQueryAsync(10.milliseconds, withError = false) testAwaitAnyTermination(ExpectNotBlocked) require(!q3.isActive) - val q4 = stopRandomQueryAsync(10 milliseconds, withError = true) + val q4 = stopRandomQueryAsync(10.milliseconds, withError = true) eventually(Timeout(streamingTimeout)) { require(!q4.isActive) } // After q4 terminates with exception, awaitAnyTerm should start throwing exception testAwaitAnyTermination(ExpectException[SparkException]) @@ -139,81 +142,81 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { // awaitAnyTermination should be blocking or non-blocking depending on timeout values testAwaitAnyTermination( ExpectBlocked, - awaitTimeout = 4 seconds, + awaitTimeout = 4.seconds, expectedReturnedValue = false, - testBehaviorFor = 2 seconds) + testBehaviorFor = 2.seconds) testAwaitAnyTermination( ExpectNotBlocked, - awaitTimeout = 50 milliseconds, + awaitTimeout = 50.milliseconds, expectedReturnedValue = false, - testBehaviorFor = 1 second) + testBehaviorFor = 1.second) // Stop a query asynchronously within timeout and awaitAnyTerm should be unblocked - val q1 = stopRandomQueryAsync(stopAfter = 100 milliseconds, withError = false) + val q1 = stopRandomQueryAsync(stopAfter = 100.milliseconds, withError = false) testAwaitAnyTermination( ExpectNotBlocked, - awaitTimeout = 2 seconds, + awaitTimeout = 2.seconds, expectedReturnedValue = true, - testBehaviorFor = 4 seconds) + testBehaviorFor = 4.seconds) require(!q1.isActive) // should be inactive by the time the prev awaitAnyTerm returned // All subsequent calls to awaitAnyTermination should be non-blocking even if timeout is high testAwaitAnyTermination( - ExpectNotBlocked, awaitTimeout = 4 seconds, expectedReturnedValue = true) + ExpectNotBlocked, awaitTimeout = 4.seconds, expectedReturnedValue = true) // Resetting termination should make awaitAnyTermination() blocking again spark.streams.resetTerminated() testAwaitAnyTermination( ExpectBlocked, - awaitTimeout = 4 seconds, + awaitTimeout = 4.seconds, expectedReturnedValue = false, - testBehaviorFor = 1 second) + testBehaviorFor = 1.second) // Terminate a query asynchronously with exception within timeout, awaitAnyTermination should // throws the exception - val q2 = stopRandomQueryAsync(100 milliseconds, withError = true) + val q2 = stopRandomQueryAsync(100.milliseconds, withError = true) testAwaitAnyTermination( ExpectException[SparkException], - awaitTimeout = 4 seconds, - testBehaviorFor = 6 seconds) + awaitTimeout = 4.seconds, + testBehaviorFor = 6.seconds) require(!q2.isActive) // should be inactive by the time the prev awaitAnyTerm returned // All subsequent calls to awaitAnyTermination should throw the exception testAwaitAnyTermination( ExpectException[SparkException], - awaitTimeout = 2 seconds, - testBehaviorFor = 4 seconds) + awaitTimeout = 2.seconds, + testBehaviorFor = 4.seconds) // Terminate a query asynchronously outside the timeout, awaitAnyTerm should be blocked spark.streams.resetTerminated() - val q3 = stopRandomQueryAsync(2 seconds, withError = true) + val q3 = stopRandomQueryAsync(2.seconds, withError = true) testAwaitAnyTermination( ExpectNotBlocked, - awaitTimeout = 100 milliseconds, + awaitTimeout = 100.milliseconds, expectedReturnedValue = false, - testBehaviorFor = 4 seconds) + testBehaviorFor = 4.seconds) // After that query is stopped, awaitAnyTerm should throw exception eventually(Timeout(streamingTimeout)) { require(!q3.isActive) } // wait for query to stop testAwaitAnyTermination( ExpectException[SparkException], - awaitTimeout = 100 milliseconds, - testBehaviorFor = 4 seconds) + awaitTimeout = 100.milliseconds, + testBehaviorFor = 4.seconds) // Terminate multiple queries, one with failure and see whether awaitAnyTermination throws // the exception spark.streams.resetTerminated() - val q4 = stopRandomQueryAsync(10 milliseconds, withError = false) + val q4 = stopRandomQueryAsync(10.milliseconds, withError = false) testAwaitAnyTermination( - ExpectNotBlocked, awaitTimeout = 2 seconds, expectedReturnedValue = true) + ExpectNotBlocked, awaitTimeout = 2.seconds, expectedReturnedValue = true) require(!q4.isActive) - val q5 = stopRandomQueryAsync(10 milliseconds, withError = true) + val q5 = stopRandomQueryAsync(10.milliseconds, withError = true) eventually(Timeout(streamingTimeout)) { require(!q5.isActive) } // After q5 terminates with exception, awaitAnyTerm should start throwing exception - testAwaitAnyTermination(ExpectException[SparkException], awaitTimeout = 2 seconds) + testAwaitAnyTermination(ExpectException[SparkException], awaitTimeout = 2.seconds) } } @@ -277,7 +280,7 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { expectedBehavior: ExpectedBehavior, expectedReturnedValue: Boolean = false, awaitTimeout: Span = null, - testBehaviorFor: Span = 4 seconds + testBehaviorFor: Span = 4.seconds ): Unit = { def awaitTermFunc(): Unit = { @@ -304,8 +307,8 @@ class StreamingQueryManagerSuite extends StreamTest with BeforeAndAfter { if (withError) { logDebug(s"Terminating query ${queryToStop.name} with error") queryToStop.asInstanceOf[StreamingQueryWrapper].streamingQuery.logicalPlan.collect { - case StreamingExecutionRelation(source, _) => - source.asInstanceOf[MemoryStream[Int]].addData(0) + case r: StreamingDataSourceV2Relation => + r.stream.asInstanceOf[MemoryStream[Int]].addData(0) } } else { logDebug(s"Stopping query ${queryToStop.name}") 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 7bef687e7e43b..e784d318b4ffa 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 @@ -20,7 +20,6 @@ package org.apache.spark.sql.streaming import java.util.UUID import scala.collection.JavaConverters._ -import scala.language.postfixOps import org.json4s._ import org.json4s.jackson.JsonMethods._ @@ -73,7 +72,8 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "inputRowsPerSecond" : 10.0 | } ], | "sink" : { - | "description" : "sink" + | "description" : "sink", + | "numOutputRows" : -1 | } |} """.stripMargin.trim) @@ -105,7 +105,8 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { | "numInputRows" : 678 | } ], | "sink" : { - | "description" : "sink" + | "description" : "sink", + | "numOutputRows" : -1 | } |} """.stripMargin.trim) @@ -201,7 +202,7 @@ class StreamingQueryStatusAndProgressSuite extends StreamTest with Eventually { val progress = query.lastProgress assert(progress.stateOperators.length > 0) // Should emit new progresses every 10 ms, but we could be facing a slow Jenkins - eventually(timeout(1 minute)) { + eventually(timeout(1.minute)) { val nextProgress = query.lastProgress assert(nextProgress.timestamp !== progress.timestamp) assert(nextProgress.numInputRows === 0) @@ -250,7 +251,7 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.PositiveInfinity // should not be present in the json ) ), - sink = new SinkProgress("sink") + sink = SinkProgress("sink", None) ) val testProgress2 = new StreamingQueryProgress( @@ -274,7 +275,7 @@ object StreamingQueryStatusAndProgressSuite { processedRowsPerSecond = Double.NegativeInfinity // should not be present in the json ) ), - sink = new SinkProgress("sink") + sink = SinkProgress("sink", None) ) val testStatus = new StreamingQueryStatus("active", true, false) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala index c170641372d61..e6b56e5f46f89 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/StreamingQuerySuite.scala @@ -17,26 +17,29 @@ package org.apache.spark.sql.streaming +import java.io.File import java.util.concurrent.CountDownLatch import scala.collection.mutable +import org.apache.commons.io.FileUtils import org.apache.commons.lang3.RandomStringUtils +import org.apache.hadoop.fs.Path import org.scalactic.TolerantNumerics import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.PatienceConfiguration.Timeout import org.scalatest.mockito.MockitoSugar -import org.apache.spark.SparkException +import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.internal.Logging import org.apache.spark.sql.{Column, DataFrame, Dataset, Row} import org.apache.spark.sql.catalyst.expressions.{Literal, Rand, Randn, Shuffle, Uuid} import org.apache.spark.sql.execution.exchange.ReusedExchangeExec import org.apache.spark.sql.execution.streaming._ -import org.apache.spark.sql.execution.streaming.sources.TestForeachWriter +import org.apache.spark.sql.execution.streaming.sources.{MemorySink, TestForeachWriter} import org.apache.spark.sql.functions._ import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.sources.v2.reader.{InputPartition, ScanConfig} +import org.apache.spark.sql.sources.v2.reader.InputPartition import org.apache.spark.sql.sources.v2.reader.streaming.{Offset => OffsetV2} import org.apache.spark.sql.streaming.util.{BlockingSource, MockSourceProvider, StreamManualClock} import org.apache.spark.sql.types.StructType @@ -131,7 +134,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val mapped = inputData.toDS().map { 6 / _} testStream(mapped)( - AssertOnQuery(_.isActive === true), + AssertOnQuery(_.isActive), AssertOnQuery(_.exception.isEmpty), AddData(inputData, 1, 2), CheckAnswer(6, 3), @@ -145,7 +148,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi TestAwaitTermination(ExpectNotBlocked, timeoutMs = 2000, expectedReturnValue = true), TestAwaitTermination(ExpectNotBlocked, timeoutMs = 10, expectedReturnValue = true), StartStream(), - AssertOnQuery(_.isActive === true), + AssertOnQuery(_.isActive), AddData(inputData, 0), ExpectFailure[SparkException](), AssertOnQuery(_.isActive === false), @@ -167,7 +170,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val mapped = inputData.toDS().map { 6 / _} testStream(mapped)( - AssertOnQuery(_.isActive === true), + AssertOnQuery(_.isActive), StopStream, AddData(inputData, 1, 2), StartStream(trigger = Once), @@ -220,10 +223,10 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } // getBatch should take 100 ms the first time it is called - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def planInputPartitions(start: OffsetV2, end: OffsetV2): Array[InputPartition] = { synchronized { clock.waitTillTime(1150) - super.planInputPartitions(config) + super.planInputPartitions(start, end) } } } @@ -257,7 +260,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi var lastProgressBeforeStop: StreamingQueryProgress = null testStream(mapped, OutputMode.Complete)( - StartStream(ProcessingTime(1000), triggerClock = clock), + StartStream(Trigger.ProcessingTime(1000), triggerClock = clock), AssertStreamExecThreadIsWaitingForTime(1000), AssertOnQuery(_.status.isDataAvailable === false), AssertOnQuery(_.status.isTriggerActive === false), @@ -269,7 +272,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AdvanceManualClock(1000), // time = 1000 to start new trigger, will block on `latestOffset` AssertStreamExecThreadIsWaitingForTime(1050), AssertOnQuery(_.status.isDataAvailable === false), - AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.isTriggerActive), AssertOnQuery(_.status.message.startsWith("Getting offsets from")), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), @@ -277,16 +280,16 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertClockTime(1050), // will block on `planInputPartitions` that needs 1350 AssertStreamExecThreadIsWaitingForTime(1150), - AssertOnQuery(_.status.isDataAvailable === true), - AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.isDataAvailable), + AssertOnQuery(_.status.isTriggerActive), AssertOnQuery(_.status.message === "Processing new data"), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), AdvanceManualClock(100), // time = 1150 to unblock `planInputPartitions` AssertClockTime(1150), AssertStreamExecThreadIsWaitingForTime(1500), // will block on map task that needs 1500 - AssertOnQuery(_.status.isDataAvailable === true), - AssertOnQuery(_.status.isTriggerActive === true), + AssertOnQuery(_.status.isDataAvailable), + AssertOnQuery(_.status.isTriggerActive), AssertOnQuery(_.status.message === "Processing new data"), AssertOnQuery(_.recentProgress.count(_.numInputRows > 0) === 0), @@ -295,7 +298,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertClockTime(1500), CheckAnswer(2), AssertStreamExecThreadIsWaitingForTime(2000), // will block until the next trigger - AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isDataAvailable), AssertOnQuery(_.status.isTriggerActive === false), AssertOnQuery(_.status.message === "Waiting for next trigger"), AssertOnQuery { query => @@ -338,7 +341,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertClockTime(2000), AssertStreamExecThreadIsWaitingForTime(3000), // will block waiting for next trigger time CheckAnswer(4), - AssertOnQuery(_.status.isDataAvailable === true), + AssertOnQuery(_.status.isDataAvailable), AssertOnQuery(_.status.isTriggerActive === false), AssertOnQuery(_.status.message === "Waiting for next trigger"), AssertOnQuery { query => @@ -370,7 +373,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi AssertOnQuery(_.status.message === "Stopped"), // Test status and progress after query terminated with error - StartStream(ProcessingTime(1000), triggerClock = clock), + StartStream(Trigger.ProcessingTime(1000), triggerClock = clock), AdvanceManualClock(1000), // ensure initial trigger completes before AddData AddData(inputData, 0), AdvanceManualClock(1000), // allow another trigger @@ -410,9 +413,9 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi sources.nonEmpty } // Disabled by default - assert(spark.conf.get("spark.sql.streaming.metricsEnabled").toBoolean === false) + assert(spark.conf.get(SQLConf.STREAMING_METRICS_ENABLED.key).toBoolean === false) - withSQLConf("spark.sql.streaming.metricsEnabled" -> "false") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "false") { testStream(inputData.toDF)( AssertOnQuery { q => !isMetricsRegistered(q) }, StopStream, @@ -421,7 +424,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } // Registered when enabled - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { testStream(inputData.toDF)( AssertOnQuery { q => isMetricsRegistered(q) }, StopStream, @@ -431,7 +434,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi } test("SPARK-22975: MetricsReporter defaults when there was no progress reported") { - withSQLConf("spark.sql.streaming.metricsEnabled" -> "true") { + withSQLConf(SQLConf.STREAMING_METRICS_ENABLED.key -> "true") { BlockingSource.latch = new CountDownLatch(1) withTempDir { tempDir => val sq = spark.readStream @@ -495,7 +498,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi test("input row calculation with same V2 source used twice in self-union") { val streamInput = MemoryStream[Int] - testStream(streamInput.toDF().union(streamInput.toDF()), useV2Sink = true)( + testStream(streamInput.toDF().union(streamInput.toDF()))( AddData(streamInput, 1, 2, 3), CheckAnswer(1, 1, 2, 2, 3, 3), AssertOnQuery { q => @@ -516,7 +519,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi // relation, which breaks exchange reuse, as the optimizer will remove Project from one side. // Here we manually add a useful Project, to trigger exchange reuse. val streamDF = memoryStream.toDF().select('value + 0 as "v") - testStream(streamDF.join(streamDF, "v"), useV2Sink = true)( + testStream(streamDF.join(streamDF, "v"))( AddData(memoryStream, 1, 2, 3), CheckAnswer(1, 2, 3), check @@ -553,7 +556,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val streamInput1 = MemoryStream[Int] val streamInput2 = MemoryStream[Int] - testStream(streamInput1.toDF().union(streamInput2.toDF()), useV2Sink = true)( + testStream(streamInput1.toDF().union(streamInput2.toDF()))( AddData(streamInput1, 1, 2, 3), CheckLastBatch(1, 2, 3), AssertOnQuery { q => @@ -584,7 +587,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val streamInput = MemoryStream[Int] val staticInputDF = spark.createDataFrame(Seq(1 -> "1", 2 -> "2")).toDF("value", "anotherValue") - testStream(streamInput.toDF().join(staticInputDF, "value"), useV2Sink = true)( + testStream(streamInput.toDF().join(staticInputDF, "value"))( AddData(streamInput, 1, 2, 3), AssertOnQuery { q => q.processAllAvailable() @@ -606,7 +609,7 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi val streamInput2 = MemoryStream[Int] val staticInputDF2 = staticInputDF.union(staticInputDF).cache() - testStream(streamInput2.toDF().join(staticInputDF2, "value"), useV2Sink = true)( + testStream(streamInput2.toDF().join(staticInputDF2, "value"))( AddData(streamInput2, 1, 2, 3), AssertOnQuery { q => q.processAllAvailable() @@ -714,8 +717,8 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi q3.processAllAvailable() } assert(e.getCause.isInstanceOf[SparkException]) - assert(e.getCause.getCause.isInstanceOf[IllegalStateException]) - assert(e.getMessage.contains("StreamingQuery cannot be used in executors")) + assert(e.getCause.getCause.getCause.isInstanceOf[IllegalStateException]) + TestUtils.assertExceptionMsg(e, "StreamingQuery cannot be used in executors") } finally { q1.stop() q2.stop() @@ -906,15 +909,199 @@ class StreamingQuerySuite extends StreamTest with BeforeAndAfter with Logging wi assert(df.logicalPlan.toJSON.contains("StreamingRelationV2")) testStream(df)( - AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingExecutionRelation")) + AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingDataSourceV2Relation")) ) - testStream(df, useV2Sink = true)( + testStream(df)( StartStream(trigger = Trigger.Continuous(100)), - AssertOnQuery(_.logicalPlan.toJSON.contains("ContinuousExecutionRelation")) + AssertOnQuery(_.logicalPlan.toJSON.contains("StreamingDataSourceV2Relation")) ) } + test("special characters in checkpoint path") { + withTempDir { tempDir => + val checkpointDir = new File(tempDir, "chk @#chk") + val inputData = MemoryStream[Int] + inputData.addData(1) + val q = inputData.toDF() + .writeStream + .format("noop") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start() + try { + q.processAllAvailable() + assert(checkpointDir.listFiles().toList.nonEmpty) + } finally { + q.stop() + } + } + } + + /** + * Copy the checkpoint generated by Spark 2.4.0 from test resource to `dir` to set up a legacy + * streaming checkpoint. + */ + private def setUp2dot4dot0Checkpoint(dir: File): Unit = { + val input = getClass.getResource("/structured-streaming/escaped-path-2.4.0") + assert(input != null, "cannot find test resource '/structured-streaming/escaped-path-2.4.0'") + val inputDir = new File(input.toURI) + + // Copy test files to tempDir so that we won't modify the original data. + FileUtils.copyDirectory(inputDir, dir) + + // Spark 2.4 and earlier escaped the _spark_metadata path once + val legacySparkMetadataDir = new File( + dir, + new Path("output %@#output/_spark_metadata").toUri.toString) + + // Migrate from legacy _spark_metadata directory to the new _spark_metadata directory. + // Ideally we should copy "_spark_metadata" directly like what the user is supposed to do to + // migrate to new version. However, in our test, "tempDir" will be different in each run and + // we need to fix the absolute path in the metadata to match "tempDir". + val sparkMetadata = FileUtils.readFileToString(new File(legacySparkMetadataDir, "0"), "UTF-8") + FileUtils.write( + new File(legacySparkMetadataDir, "0"), + sparkMetadata.replaceAll("TEMPDIR", dir.getCanonicalPath), + "UTF-8") + } + + test("detect escaped path and report the migration guide") { + // Assert that the error message contains the migration conf, path and the legacy path. + def assertMigrationError(errorMessage: String, path: File, legacyPath: File): Unit = { + Seq(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED.key, + path.getCanonicalPath, + legacyPath.getCanonicalPath).foreach { msg => + assert(errorMessage.contains(msg)) + } + } + + withTempDir { tempDir => + setUp2dot4dot0Checkpoint(tempDir) + + // Here are the paths we will use to create the query + val outputDir = new File(tempDir, "output %@#output") + val checkpointDir = new File(tempDir, "chk %@#chk") + val sparkMetadataDir = new File(tempDir, "output %@#output/_spark_metadata") + + // The escaped paths used by Spark 2.4 and earlier. + // Spark 2.4 and earlier escaped the checkpoint path three times + val legacyCheckpointDir = new File( + tempDir, + new Path(new Path(new Path("chk %@#chk").toUri.toString).toUri.toString).toUri.toString) + // Spark 2.4 and earlier escaped the _spark_metadata path once + val legacySparkMetadataDir = new File( + tempDir, + new Path("output %@#output/_spark_metadata").toUri.toString) + + // Reading a file sink output in a batch query should detect the legacy _spark_metadata + // directory and throw an error + val e = intercept[SparkException] { + spark.read.load(outputDir.getCanonicalPath).as[Int] + } + assertMigrationError(e.getMessage, sparkMetadataDir, legacySparkMetadataDir) + + // Restarting the streaming query should detect the legacy _spark_metadata directory and + // throw an error + val inputData = MemoryStream[Int] + val e2 = intercept[SparkException] { + inputData.toDF() + .writeStream + .format("parquet") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start(outputDir.getCanonicalPath) + } + assertMigrationError(e2.getMessage, sparkMetadataDir, legacySparkMetadataDir) + + // Move "_spark_metadata" to fix the file sink and test the checkpoint path. + FileUtils.moveDirectory(legacySparkMetadataDir, sparkMetadataDir) + + // Restarting the streaming query should detect the legacy + // checkpoint path and throw an error. + val e3 = intercept[SparkException] { + inputData.toDF() + .writeStream + .format("parquet") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start(outputDir.getCanonicalPath) + } + assertMigrationError(e3.getMessage, checkpointDir, legacyCheckpointDir) + + // Fix the checkpoint path and verify that the user can migrate the issue by moving files. + FileUtils.moveDirectory(legacyCheckpointDir, checkpointDir) + + val q = inputData.toDF() + .writeStream + .format("parquet") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start(outputDir.getCanonicalPath) + try { + q.processAllAvailable() + // Check the query id to make sure it did use checkpoint + assert(q.id.toString == "09be7fb3-49d8-48a6-840d-e9c2ad92a898") + + // Verify that the batch query can read "_spark_metadata" correctly after migration. + val df = spark.read.load(outputDir.getCanonicalPath) + assert(df.queryExecution.executedPlan.toString contains "MetadataLogFileIndex") + checkDatasetUnorderly(df.as[Int], 1, 2, 3) + } finally { + q.stop() + } + } + } + + test("ignore the escaped path check when the flag is off") { + withTempDir { tempDir => + setUp2dot4dot0Checkpoint(tempDir) + val outputDir = new File(tempDir, "output %@#output") + val checkpointDir = new File(tempDir, "chk %@#chk") + + withSQLConf(SQLConf.STREAMING_CHECKPOINT_ESCAPED_PATH_CHECK_ENABLED.key -> "false") { + // Verify that the batch query ignores the legacy "_spark_metadata" + val df = spark.read.load(outputDir.getCanonicalPath) + assert(!(df.queryExecution.executedPlan.toString contains "MetadataLogFileIndex")) + checkDatasetUnorderly(df.as[Int], 1, 2, 3) + + val inputData = MemoryStream[Int] + val q = inputData.toDF() + .writeStream + .format("parquet") + .option("checkpointLocation", checkpointDir.getCanonicalPath) + .start(outputDir.getCanonicalPath) + try { + q.processAllAvailable() + // Check the query id to make sure it ignores the legacy checkpoint + assert(q.id.toString != "09be7fb3-49d8-48a6-840d-e9c2ad92a898") + } finally { + q.stop() + } + } + } + } + + test("containsSpecialCharsInPath") { + Seq("foo/b ar", + "/foo/b ar", + "file:/foo/b ar", + "file://foo/b ar", + "file:///foo/b ar", + "file://foo:bar@bar/foo/b ar").foreach { p => + assert(StreamExecution.containsSpecialCharsInPath(new Path(p)), s"failed to check $p") + } + Seq("foo/bar", + "/foo/bar", + "file:/foo/bar", + "file://foo/bar", + "file:///foo/bar", + "file://foo:bar@bar/foo/bar", + // Special chars not in a path should not be considered as such urls won't hit the escaped + // path issue. + "file://foo:b ar@bar/foo/bar", + "file://foo:bar@b ar/foo/bar", + "file://f oo:bar@bar/foo/bar").foreach { p => + assert(!StreamExecution.containsSpecialCharsInPath(new Path(p)), s"failed to check $p") + } + } + /** Create a streaming DF that only execute one batch in which it returns the given static DF */ private def createSingleTriggerStreamingDF(triggerDF: DataFrame): DataFrame = { require(!triggerDF.isStreaming) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala index c5b95fa9b64a9..3ec4750c59fc5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousAggregationSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.streaming.continuous import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf.UNSUPPORTED_OPERATION_CHECK_ENABLED import org.apache.spark.sql.streaming.OutputMode class ContinuousAggregationSuite extends ContinuousSuiteBase { @@ -36,7 +37,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("basic") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( @@ -112,7 +113,7 @@ class ContinuousAggregationSuite extends ContinuousSuiteBase { } test("repeated restart") { - withSQLConf(("spark.sql.streaming.unsupportedOperationCheck", "false")) { + withSQLConf((UNSUPPORTED_OPERATION_CHECK_ENABLED.key, "false")) { val input = ContinuousMemoryStream.singlePartition[Int] testStream(input.toDF().agg(max('value)), OutputMode.Complete)( diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala new file mode 100644 index 0000000000000..59d6ac0af52a3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueryStatusAndProgressSuite.scala @@ -0,0 +1,55 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.streaming.continuous + +import org.apache.spark.sql.execution.streaming.StreamExecution +import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream +import org.apache.spark.sql.streaming.Trigger + +class ContinuousQueryStatusAndProgressSuite extends ContinuousSuiteBase { + test("StreamingQueryStatus - ContinuousExecution isDataAvailable and isTriggerActive " + + "should be false") { + import testImplicits._ + + val input = ContinuousMemoryStream[Int] + + def assertStatus(stream: StreamExecution): Unit = { + assert(stream.status.isDataAvailable === false) + assert(stream.status.isTriggerActive === false) + } + + val trigger = Trigger.Continuous(100) + testStream(input.toDF())( + StartStream(trigger), + Execute(assertStatus), + AddData(input, 0, 1, 2), + Execute(assertStatus), + CheckAnswer(0, 1, 2), + Execute(assertStatus), + StopStream, + Execute(assertStatus), + AddData(input, 3, 4, 5), + Execute(assertStatus), + StartStream(trigger), + Execute(assertStatus), + CheckAnswer(0, 1, 2, 3, 4, 5), + Execute(assertStatus), + StopStream, + Execute(assertStatus)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala index d6819eacd07ca..bad22590807a7 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousQueuedDataReaderSuite.scala @@ -27,8 +27,8 @@ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, UnsafeProjection, UnsafeRow} import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousReadSupport, PartitionOffset} -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousPartitionReader, ContinuousStream, PartitionOffset} +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.streaming.StreamTest import org.apache.spark.sql.types.{DataType, IntegerType, StructType} @@ -43,8 +43,8 @@ class ContinuousQueuedDataReaderSuite extends StreamTest with MockitoSugar { override def beforeEach(): Unit = { super.beforeEach() epochEndpoint = EpochCoordinatorRef.create( - mock[StreamingWriteSupport], - mock[ContinuousReadSupport], + mock[StreamingWrite], + mock[ContinuousStream], mock[ContinuousExecution], coordinatorId, startEpoch, diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala index 3d21bc63e0cc1..c6921010a002f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/ContinuousSuite.scala @@ -20,11 +20,12 @@ package org.apache.spark.sql.streaming.continuous import org.apache.spark.{SparkContext, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} import org.apache.spark.sql._ -import org.apache.spark.sql.execution.datasources.v2.DataSourceV2ScanExec +import org.apache.spark.sql.execution.datasources.v2.ContinuousScanExec import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.execution.streaming.continuous._ import org.apache.spark.sql.execution.streaming.sources.ContinuousMemoryStream import org.apache.spark.sql.functions._ +import org.apache.spark.sql.internal.SQLConf.{CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE, MIN_BATCHES_TO_RETAIN} import org.apache.spark.sql.streaming.{StreamTest, Trigger} import org.apache.spark.sql.test.TestSparkSession @@ -36,18 +37,43 @@ class ContinuousSuiteBase extends StreamTest { "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true"))) - protected def waitForRateSourceTriggers(query: StreamExecution, numTriggers: Int): Unit = { - query match { - case s: ContinuousExecution => - assert(numTriggers >= 2, "must wait for at least 2 triggers to ensure query is initialized") - val reader = s.lastExecution.executedPlan.collectFirst { - case DataSourceV2ScanExec(_, _, _, _, r: RateStreamContinuousReadSupport, _) => r - }.get - - val deltaMs = numTriggers * 1000 + 300 - while (System.currentTimeMillis < reader.creationTime + deltaMs) { - Thread.sleep(reader.creationTime + deltaMs - System.currentTimeMillis) + protected def waitForRateSourceTriggers(query: ContinuousExecution, numTriggers: Int): Unit = { + query.awaitEpoch(0) + + // This is called after waiting first epoch to be committed, so we can just treat + // it as partition readers for rate source are already initialized. + val firstCommittedTime = System.nanoTime() + val deltaNs = (numTriggers * 1000 + 300) * 1000000L + var toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + while (toWaitNs > 0) { + Thread.sleep(toWaitNs / 1000000) + toWaitNs = firstCommittedTime + deltaNs - System.nanoTime() + } + } + + protected def waitForRateSourceCommittedValue( + query: ContinuousExecution, + desiredValue: Long, + maxWaitTimeMs: Long): Unit = { + def readHighestCommittedValue(c: ContinuousExecution): Option[Long] = { + c.committedOffsets.lastOption.map { case (_, offset) => + offset match { + case o: RateStreamOffset => + o.partitionToValueAndRunTimeMs.map { + case (_, ValueRunTimeMsPair(value, _)) => value + }.max } + } + } + + val maxWait = System.currentTimeMillis() + maxWaitTimeMs + while (System.currentTimeMillis() < maxWait && + readHighestCommittedValue(query).getOrElse(Long.MinValue) < desiredValue) { + Thread.sleep(100) + } + if (System.currentTimeMillis() > maxWait) { + logWarning(s"Couldn't reach desired value in $maxWaitTimeMs milliseconds!" + + s"Current highest committed value is ${readHighestCommittedValue(query)}") } } @@ -56,7 +82,6 @@ class ContinuousSuiteBase extends StreamTest { protected val longContinuousTrigger = Trigger.Continuous("1 hour") override protected val defaultTrigger = Trigger.Continuous(100) - override protected val defaultUseV2Sink = true } class ContinuousSuite extends ContinuousSuiteBase { @@ -216,14 +241,16 @@ class ContinuousSuite extends ContinuousSuiteBase { .queryName("noharness") .trigger(Trigger.Continuous(100)) .start() + + val expected = Set(0, 1, 2, 3) val continuousExecution = query.asInstanceOf[StreamingQueryWrapper].streamingQuery.asInstanceOf[ContinuousExecution] - continuousExecution.awaitEpoch(0) - waitForRateSourceTriggers(continuousExecution, 2) + waitForRateSourceCommittedValue(continuousExecution, expected.max, 20 * 1000) query.stop() val results = spark.read.table("noharness").collect() - assert(Set(0, 1, 2, 3).map(Row(_)).subsetOf(results.toSet)) + assert(expected.map(Row(_)).subsetOf(results.toSet), + s"Result set ${results.toSet} are not a superset of $expected!") } } @@ -238,13 +265,15 @@ class ContinuousStressSuite extends ContinuousSuiteBase { .load() .select('value) - testStream(df, useV2Sink = true)( + testStream(df)( StartStream(longContinuousTrigger), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 201)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, - CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_))) + CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_))) ) } @@ -256,13 +285,15 @@ class ContinuousStressSuite extends ContinuousSuiteBase { .load() .select('value) - testStream(df, useV2Sink = true)( + testStream(df)( StartStream(Trigger.Continuous(2012)), AwaitEpoch(0), - Execute(waitForRateSourceTriggers(_, 201)), + Execute { exec => + waitForRateSourceTriggers(exec.asInstanceOf[ContinuousExecution], 5) + }, IncrementEpoch(), StopStream, - CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_)))) + CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_)))) } test("restarts") { @@ -273,28 +304,28 @@ class ContinuousStressSuite extends ContinuousSuiteBase { .load() .select('value) - testStream(df, useV2Sink = true)( - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(10), + testStream(df)( + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(2), StopStream, - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(20), + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(4), StopStream, - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(21), + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(5), StopStream, - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(22), + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(6), StopStream, - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(25), + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(8), StopStream, - StartStream(Trigger.Continuous(2012)), + StartStream(Trigger.Continuous(1012)), StopStream, - StartStream(Trigger.Continuous(2012)), - AwaitEpoch(50), + StartStream(Trigger.Continuous(1012)), + AwaitEpoch(15), StopStream, - CheckAnswerRowsContains(scala.Range(0, 25000).map(Row(_)))) + CheckAnswerRowsContains(scala.Range(0, 2500).map(Row(_)))) } } @@ -307,7 +338,7 @@ class ContinuousMetaSuite extends ContinuousSuiteBase { "local[10]", "continuous-stream-test-sql-context", sparkConf.set("spark.sql.testkey", "true") - .set("spark.sql.streaming.minBatchesToRetain", "2"))) + .set(MIN_BATCHES_TO_RETAIN.key, "2"))) test("SPARK-24351: check offsetLog/commitLog retained in the checkpoint directory") { withTempDir { checkpointDir => @@ -343,3 +374,33 @@ class ContinuousMetaSuite extends ContinuousSuiteBase { } } } + +class ContinuousEpochBacklogSuite extends ContinuousSuiteBase { + import testImplicits._ + + override protected def createSparkSession = new TestSparkSession( + new SparkContext( + "local[1]", + "continuous-stream-test-sql-context", + sparkConf.set("spark.sql.testkey", "true"))) + + // This test forces the backlog to overflow by not standing up enough executors for the query + // to make progress. + test("epoch backlog overflow") { + withSQLConf((CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE.key, "10")) { + val df = spark.readStream + .format("rate") + .option("numPartitions", "2") + .option("rowsPerSecond", "500") + .load() + .select('value) + + testStream(df)( + StartStream(Trigger.Continuous(1)), + ExpectFailure[IllegalStateException] { e => + e.getMessage.contains("queue has exceeded its maximum") + } + ) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala index 3c973d8ebc704..e3498db4194e8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/continuous/EpochCoordinatorSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.streaming.continuous -import org.mockito.InOrder -import org.mockito.Matchers.{any, eq => eqTo} +import org.mockito.{ArgumentCaptor, InOrder} +import org.mockito.ArgumentMatchers.{any, eq => eqTo} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfterEach import org.scalatest.mockito.MockitoSugar @@ -27,9 +27,10 @@ import org.apache.spark._ import org.apache.spark.rpc.RpcEndpointRef import org.apache.spark.sql.LocalSparkSession import org.apache.spark.sql.execution.streaming.continuous._ -import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousReadSupport, PartitionOffset} +import org.apache.spark.sql.internal.SQLConf.CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE +import org.apache.spark.sql.sources.v2.reader.streaming.{ContinuousStream, PartitionOffset} import org.apache.spark.sql.sources.v2.writer.WriterCommitMessage -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWrite import org.apache.spark.sql.test.TestSparkSession class EpochCoordinatorSuite @@ -40,20 +41,25 @@ class EpochCoordinatorSuite private var epochCoordinator: RpcEndpointRef = _ - private var writeSupport: StreamingWriteSupport = _ + private var writeSupport: StreamingWrite = _ private var query: ContinuousExecution = _ private var orderVerifier: InOrder = _ + private val epochBacklogQueueSize = 10 override def beforeEach(): Unit = { - val reader = mock[ContinuousReadSupport] - writeSupport = mock[StreamingWriteSupport] + val stream = mock[ContinuousStream] + writeSupport = mock[StreamingWrite] query = mock[ContinuousExecution] orderVerifier = inOrder(writeSupport, query) - spark = new TestSparkSession() + spark = new TestSparkSession( + new SparkContext( + "local[2]", "test-sql-context", + new SparkConf().set("spark.sql.testkey", "true") + .set(CONTINUOUS_STREAMING_EPOCH_BACKLOG_QUEUE_SIZE, epochBacklogQueueSize))) epochCoordinator - = EpochCoordinatorRef.create(writeSupport, reader, query, "test", 1, spark, SparkEnv.get) + = EpochCoordinatorRef.create(writeSupport, stream, query, "test", 1, spark, SparkEnv.get) } test("single epoch") { @@ -186,6 +192,66 @@ class EpochCoordinatorSuite verifyCommitsInOrderOf(List(1, 2, 3, 4, 5)) } + test("several epochs, max epoch backlog reached by partitionOffsets") { + setWriterPartitions(1) + setReaderPartitions(1) + + reportPartitionOffset(0, 1) + // Commit messages not arriving + for (i <- 2 to epochBacklogQueueSize + 1) { + reportPartitionOffset(0, i) + } + + makeSynchronousCall() + + for (i <- 1 to epochBacklogQueueSize + 1) { + verifyNoCommitFor(i) + } + verifyStoppedWithException("Size of the partition offset queue has exceeded its maximum") + } + + test("several epochs, max epoch backlog reached by partitionCommits") { + setWriterPartitions(1) + setReaderPartitions(1) + + commitPartitionEpoch(0, 1) + // Offset messages not arriving + for (i <- 2 to epochBacklogQueueSize + 1) { + commitPartitionEpoch(0, i) + } + + makeSynchronousCall() + + for (i <- 1 to epochBacklogQueueSize + 1) { + verifyNoCommitFor(i) + } + verifyStoppedWithException("Size of the partition commit queue has exceeded its maximum") + } + + test("several epochs, max epoch backlog reached by epochsWaitingToBeCommitted") { + setWriterPartitions(2) + setReaderPartitions(2) + + commitPartitionEpoch(0, 1) + reportPartitionOffset(0, 1) + + // For partition 2 epoch 1 messages never arriving + // +2 because the first epoch not yet arrived + for (i <- 2 to epochBacklogQueueSize + 2) { + commitPartitionEpoch(0, i) + reportPartitionOffset(0, i) + commitPartitionEpoch(1, i) + reportPartitionOffset(1, i) + } + + makeSynchronousCall() + + for (i <- 1 to epochBacklogQueueSize + 2) { + verifyNoCommitFor(i) + } + verifyStoppedWithException("Size of the epoch queue has exceeded its maximum") + } + private def setWriterPartitions(numPartitions: Int): Unit = { epochCoordinator.askSync[Unit](SetWriterPartitions(numPartitions)) } @@ -221,4 +287,13 @@ class EpochCoordinatorSuite private def verifyCommitsInOrderOf(epochs: Seq[Long]): Unit = { epochs.foreach(verifyCommit) } + + private def verifyStoppedWithException(msg: String): Unit = { + val exceptionCaptor = ArgumentCaptor.forClass(classOf[Throwable]); + verify(query, atLeastOnce()).stopInNewThread(exceptionCaptor.capture()) + + import scala.collection.JavaConverters._ + val throwable = exceptionCaptor.getAllValues.asScala.find(_.getMessage === msg) + assert(throwable != null, "Stream stopped with an exception but expected message is missing") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala index 3a0e780a73915..4db605ee1b238 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/sources/StreamingDataSourceV2Suite.scala @@ -17,112 +17,182 @@ package org.apache.spark.sql.streaming.sources +import java.util +import java.util.Collections + +import scala.collection.JavaConverters._ + import org.apache.spark.sql.{DataFrame, SQLContext} import org.apache.spark.sql.execution.datasources.DataSource -import org.apache.spark.sql.execution.streaming.{RateStreamOffset, Sink, StreamingQueryWrapper} -import org.apache.spark.sql.execution.streaming.continuous.ContinuousTrigger +import org.apache.spark.sql.execution.streaming.{ContinuousTrigger, RateStreamOffset, Sink, StreamingQueryWrapper} import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{DataSourceRegister, StreamSinkProvider} import org.apache.spark.sql.sources.v2._ -import org.apache.spark.sql.sources.v2.reader.{InputPartition, PartitionReaderFactory, ScanConfig, ScanConfigBuilder} +import org.apache.spark.sql.sources.v2.TableCapability._ +import org.apache.spark.sql.sources.v2.reader._ import org.apache.spark.sql.sources.v2.reader.streaming._ -import org.apache.spark.sql.sources.v2.writer.streaming.StreamingWriteSupport +import org.apache.spark.sql.sources.v2.writer.{WriteBuilder, WriterCommitMessage} +import org.apache.spark.sql.sources.v2.writer.streaming.{StreamingDataWriterFactory, StreamingWrite} import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamTest, Trigger} import org.apache.spark.sql.types.StructType +import org.apache.spark.sql.util.CaseInsensitiveStringMap import org.apache.spark.util.Utils -case class FakeReadSupport() extends MicroBatchReadSupport with ContinuousReadSupport { +class FakeDataStream extends MicroBatchStream with ContinuousStream { override def deserializeOffset(json: String): Offset = RateStreamOffset(Map()) override def commit(end: Offset): Unit = {} override def stop(): Unit = {} - override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map()) - override def fullSchema(): StructType = StructType(Seq()) - override def newScanConfigBuilder(start: Offset, end: Offset): ScanConfigBuilder = null override def initialOffset(): Offset = RateStreamOffset(Map()) override def latestOffset(): Offset = RateStreamOffset(Map()) - override def newScanConfigBuilder(start: Offset): ScanConfigBuilder = null - override def createReaderFactory(config: ScanConfig): PartitionReaderFactory = { + override def mergeOffsets(offsets: Array[PartitionOffset]): Offset = RateStreamOffset(Map()) + override def planInputPartitions(start: Offset, end: Offset): Array[InputPartition] = { throw new IllegalStateException("fake source - cannot actually read") } - override def createContinuousReaderFactory( - config: ScanConfig): ContinuousPartitionReaderFactory = { + override def planInputPartitions(start: Offset): Array[InputPartition] = { throw new IllegalStateException("fake source - cannot actually read") } - override def planInputPartitions(config: ScanConfig): Array[InputPartition] = { + override def createReaderFactory(): PartitionReaderFactory = { + throw new IllegalStateException("fake source - cannot actually read") + } + override def createContinuousReaderFactory(): ContinuousPartitionReaderFactory = { throw new IllegalStateException("fake source - cannot actually read") } } -trait FakeMicroBatchReadSupportProvider extends MicroBatchReadSupportProvider { - override def createMicroBatchReadSupport( - checkpointLocation: String, - options: DataSourceOptions): MicroBatchReadSupport = { - LastReadOptions.options = options - FakeReadSupport() - } +class FakeScanBuilder extends ScanBuilder with Scan { + override def build(): Scan = this + override def readSchema(): StructType = StructType(Seq()) + override def toMicroBatchStream(checkpointLocation: String): MicroBatchStream = new FakeDataStream + override def toContinuousStream(checkpointLocation: String): ContinuousStream = new FakeDataStream } -trait FakeContinuousReadSupportProvider extends ContinuousReadSupportProvider { - override def createContinuousReadSupport( - checkpointLocation: String, - options: DataSourceOptions): ContinuousReadSupport = { - LastReadOptions.options = options - FakeReadSupport() +class FakeWriteBuilder extends WriteBuilder with StreamingWrite { + override def buildForStreaming(): StreamingWrite = this + override def createStreamingWriterFactory(): StreamingDataWriterFactory = { + throw new IllegalStateException("fake sink - cannot actually write") + } + override def commit(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { + throw new IllegalStateException("fake sink - cannot actually write") + } + override def abort(epochId: Long, messages: Array[WriterCommitMessage]): Unit = { + throw new IllegalStateException("fake sink - cannot actually write") } } -trait FakeStreamingWriteSupportProvider extends StreamingWriteSupportProvider { - override def createStreamingWriteSupport( - queryId: String, - schema: StructType, - mode: OutputMode, - options: DataSourceOptions): StreamingWriteSupport = { - LastWriteOptions.options = options - throw new IllegalStateException("fake sink - cannot actually write") +trait FakeStreamingWriteTable extends Table with SupportsWrite { + override def name(): String = "fake" + override def schema(): StructType = StructType(Seq()) + override def capabilities(): util.Set[TableCapability] = { + Set(STREAMING_WRITE).asJava + } + override def newWriteBuilder(options: CaseInsensitiveStringMap): WriteBuilder = { + new FakeWriteBuilder } } class FakeReadMicroBatchOnly extends DataSourceRegister - with FakeMicroBatchReadSupportProvider + with TableProvider with SessionConfigSupport { override def shortName(): String = "fake-read-microbatch-only" override def keyPrefix: String = shortName() + + override def getTable(options: CaseInsensitiveStringMap): Table = { + LastReadOptions.options = options + new Table with SupportsRead { + override def name(): String = "fake" + override def schema(): StructType = StructType(Seq()) + override def capabilities(): util.Set[TableCapability] = { + Set(MICRO_BATCH_READ).asJava + } + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new FakeScanBuilder + } + } + } } class FakeReadContinuousOnly extends DataSourceRegister - with FakeContinuousReadSupportProvider + with TableProvider with SessionConfigSupport { override def shortName(): String = "fake-read-continuous-only" override def keyPrefix: String = shortName() + + override def getTable(options: CaseInsensitiveStringMap): Table = { + LastReadOptions.options = options + new Table with SupportsRead { + override def name(): String = "fake" + override def schema(): StructType = StructType(Seq()) + override def capabilities(): util.Set[TableCapability] = { + Set(CONTINUOUS_READ).asJava + } + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new FakeScanBuilder + } + } + } } -class FakeReadBothModes extends DataSourceRegister - with FakeMicroBatchReadSupportProvider with FakeContinuousReadSupportProvider { +class FakeReadBothModes extends DataSourceRegister with TableProvider { override def shortName(): String = "fake-read-microbatch-continuous" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new Table with SupportsRead { + override def name(): String = "fake" + override def schema(): StructType = StructType(Seq()) + override def capabilities(): util.Set[TableCapability] = { + Set(MICRO_BATCH_READ, CONTINUOUS_READ).asJava + } + override def newScanBuilder(options: CaseInsensitiveStringMap): ScanBuilder = { + new FakeScanBuilder + } + } + } } -class FakeReadNeitherMode extends DataSourceRegister { +class FakeReadNeitherMode extends DataSourceRegister with TableProvider { override def shortName(): String = "fake-read-neither-mode" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new Table { + override def name(): String = "fake" + override def schema(): StructType = StructType(Nil) + override def capabilities(): util.Set[TableCapability] = Collections.emptySet() + } + } } -class FakeWriteSupportProvider +class FakeWriteOnly extends DataSourceRegister - with FakeStreamingWriteSupportProvider + with TableProvider with SessionConfigSupport { override def shortName(): String = "fake-write-microbatch-continuous" override def keyPrefix: String = shortName() + + override def getTable(options: CaseInsensitiveStringMap): Table = { + LastWriteOptions.options = options + new Table with FakeStreamingWriteTable { + override def name(): String = "fake" + override def schema(): StructType = StructType(Nil) + } + } } -class FakeNoWrite extends DataSourceRegister { +class FakeNoWrite extends DataSourceRegister with TableProvider { override def shortName(): String = "fake-write-neither-mode" + override def getTable(options: CaseInsensitiveStringMap): Table = { + new Table { + override def name(): String = "fake" + override def schema(): StructType = StructType(Nil) + override def capabilities(): util.Set[TableCapability] = Collections.emptySet() + } + } } - case class FakeWriteV1FallbackException() extends Exception class FakeSink extends Sink { @@ -130,21 +200,28 @@ class FakeSink extends Sink { } class FakeWriteSupportProviderV1Fallback extends DataSourceRegister - with FakeStreamingWriteSupportProvider with StreamSinkProvider { + with TableProvider with StreamSinkProvider { override def createSink( - sqlContext: SQLContext, - parameters: Map[String, String], - partitionColumns: Seq[String], - outputMode: OutputMode): Sink = { + sqlContext: SQLContext, + parameters: Map[String, String], + partitionColumns: Seq[String], + outputMode: OutputMode): Sink = { new FakeSink() } override def shortName(): String = "fake-write-v1-fallback" + + override def getTable(options: CaseInsensitiveStringMap): Table = { + new Table with FakeStreamingWriteTable { + override def name(): String = "fake" + override def schema(): StructType = StructType(Nil) + } + } } object LastReadOptions { - var options: DataSourceOptions = _ + var options: CaseInsensitiveStringMap = _ def clear(): Unit = { options = null @@ -152,7 +229,7 @@ object LastReadOptions { } object LastWriteOptions { - var options: DataSourceOptions = _ + var options: CaseInsensitiveStringMap = _ def clear(): Unit = { options = null @@ -164,7 +241,7 @@ class StreamingDataSourceV2Suite extends StreamTest { override def beforeAll(): Unit = { super.beforeAll() val fakeCheckpoint = Utils.createTempDir() - spark.conf.set("spark.sql.streaming.checkpointLocation", fakeCheckpoint.getCanonicalPath) + spark.conf.set(SQLConf.CHECKPOINT_LOCATION.key, fakeCheckpoint.getCanonicalPath) } override def afterEach(): Unit = { @@ -240,7 +317,7 @@ class StreamingDataSourceV2Suite extends StreamTest { testPositiveCaseWithQuery( "fake-read-microbatch-continuous", "fake-write-v1-fallback", Trigger.Once()) { v2Query => assert(v2Query.asInstanceOf[StreamingQueryWrapper].streamingQuery.sink - .isInstanceOf[FakeWriteSupportProviderV1Fallback]) + .isInstanceOf[Table]) } // Ensure we create a V1 sink with the config. Note the config is a comma separated @@ -261,7 +338,7 @@ class StreamingDataSourceV2Suite extends StreamTest { ).foreach { case (source, trigger) => test(s"SPARK-25460: session options are respected in structured streaming sources - $source") { // `keyPrefix` and `shortName` are the same in this test case - val readSource = source.newInstance().shortName() + val readSource = source.getConstructor().newInstance().shortName() val writeSource = "fake-write-microbatch-continuous" val readOptionName = "optionA" @@ -269,8 +346,8 @@ class StreamingDataSourceV2Suite extends StreamTest { testPositiveCaseWithQuery(readSource, writeSource, trigger) { _ => eventually(timeout(streamingTimeout)) { // Write options should not be set. - assert(LastWriteOptions.options.getBoolean(readOptionName, false) == false) - assert(LastReadOptions.options.getBoolean(readOptionName, false) == true) + assert(!LastWriteOptions.options.containsKey(readOptionName)) + assert(LastReadOptions.options.getBoolean(readOptionName, false)) } } } @@ -280,8 +357,8 @@ class StreamingDataSourceV2Suite extends StreamTest { testPositiveCaseWithQuery(readSource, writeSource, trigger) { _ => eventually(timeout(streamingTimeout)) { // Read options should not be set. - assert(LastReadOptions.options.getBoolean(writeOptionName, false) == false) - assert(LastWriteOptions.options.getBoolean(writeOptionName, false) == true) + assert(!LastReadOptions.options.containsKey(writeOptionName)) + assert(LastWriteOptions.options.getBoolean(writeOptionName, false)) } } } @@ -299,43 +376,43 @@ class StreamingDataSourceV2Suite extends StreamTest { for ((read, write, trigger) <- cases) { testQuietly(s"stream with read format $read, write format $write, trigger $trigger") { - val readSource = DataSource.lookupDataSource(read, spark.sqlContext.conf).newInstance() - val writeSource = DataSource.lookupDataSource(write, spark.sqlContext.conf).newInstance() - (readSource, writeSource, trigger) match { - // Valid microbatch queries. - case (_: MicroBatchReadSupportProvider, _: StreamingWriteSupportProvider, t) - if !t.isInstanceOf[ContinuousTrigger] => - testPositiveCase(read, write, trigger) - - // Valid continuous queries. - case (_: ContinuousReadSupportProvider, _: StreamingWriteSupportProvider, - _: ContinuousTrigger) => - testPositiveCase(read, write, trigger) + val sourceTable = DataSource.lookupDataSource(read, spark.sqlContext.conf).getConstructor() + .newInstance().asInstanceOf[TableProvider].getTable(CaseInsensitiveStringMap.empty()) + + val sinkTable = DataSource.lookupDataSource(write, spark.sqlContext.conf).getConstructor() + .newInstance().asInstanceOf[TableProvider].getTable(CaseInsensitiveStringMap.empty()) + import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Implicits._ + trigger match { // Invalid - can't read at all - case (r, _, _) - if !r.isInstanceOf[MicroBatchReadSupportProvider] - && !r.isInstanceOf[ContinuousReadSupportProvider] => + case _ if !sourceTable.supportsAny(MICRO_BATCH_READ, CONTINUOUS_READ) => testNegativeCase(read, write, trigger, s"Data source $read does not support streamed reading") // Invalid - can't write - case (_, w, _) if !w.isInstanceOf[StreamingWriteSupportProvider] => + case _ if !sinkTable.supports(STREAMING_WRITE) => testNegativeCase(read, write, trigger, s"Data source $write does not support streamed writing") - // Invalid - trigger is continuous but reader is not - case (r, _: StreamingWriteSupportProvider, _: ContinuousTrigger) - if !r.isInstanceOf[ContinuousReadSupportProvider] => - testNegativeCase(read, write, trigger, - s"Data source $read does not support continuous processing") - - // Invalid - trigger is microbatch but reader is not - case (r, _, t) - if !r.isInstanceOf[MicroBatchReadSupportProvider] && - !t.isInstanceOf[ContinuousTrigger] => - testPostCreationNegativeCase(read, write, trigger, - s"Data source $read does not support microbatch processing") + case _: ContinuousTrigger => + if (sourceTable.supports(CONTINUOUS_READ)) { + // Valid microbatch queries. + testPositiveCase(read, write, trigger) + } else { + // Invalid - trigger is continuous but reader is not + testNegativeCase( + read, write, trigger, s"Data source $read does not support continuous processing") + } + + case microBatchTrigger => + if (sourceTable.supports(MICRO_BATCH_READ)) { + // Valid continuous queries. + testPositiveCase(read, write, trigger) + } else { + // Invalid - trigger is microbatch but reader is not + testPostCreationNegativeCase(read, write, trigger, + s"Data source $read does not support microbatch processing") + } } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala index 8212fb912ec57..c630f1497a17e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/test/DataStreamReaderWriterSuite.scala @@ -18,13 +18,14 @@ package org.apache.spark.sql.streaming.test import java.io.File +import java.util.ConcurrentModificationException import java.util.Locale import java.util.concurrent.TimeUnit import scala.concurrent.duration._ import org.apache.hadoop.fs.Path -import org.mockito.Matchers.{any, eq => meq} +import org.mockito.ArgumentMatchers.{any, eq => meq} import org.mockito.Mockito._ import org.scalatest.BeforeAndAfter @@ -32,7 +33,7 @@ import org.apache.spark.sql._ import org.apache.spark.sql.execution.streaming._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources.{StreamSinkProvider, StreamSourceProvider} -import org.apache.spark.sql.streaming.{ProcessingTime => DeprecatedProcessingTime, _} +import org.apache.spark.sql.streaming.{OutputMode, StreamingQuery, StreamingQueryException, StreamTest} import org.apache.spark.sql.streaming.Trigger._ import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -103,9 +104,7 @@ class DefaultSource extends StreamSourceProvider with StreamSinkProvider { LastOptions.parameters = parameters LastOptions.partitionColumns = partitionColumns LastOptions.mockStreamSinkProvider.createSink(spark, parameters, partitionColumns, outputMode) - new Sink { - override def addBatch(batchId: Long, data: DataFrame): Unit = {} - } + (_: Long, _: DataFrame) => {} } } @@ -204,7 +203,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { .stop() assert(LastOptions.partitionColumns == Seq("a")) - withSQLConf("spark.sql.caseSensitive" -> "false") { + withSQLConf(SQLConf.CASE_SENSITIVE.key -> "false") { df.writeStream .format("org.apache.spark.sql.streaming.test") .option("checkpointLocation", newMetadataDir) @@ -358,7 +357,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { test("source metadataPath") { LastOptions.clear() - val checkpointLocationURI = new Path(newMetadataDir).toUri + val checkpointLocation = new Path(newMetadataDir) val df1 = spark.readStream .format("org.apache.spark.sql.streaming.test") @@ -370,7 +369,7 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { val q = df1.union(df2).writeStream .format("org.apache.spark.sql.streaming.test") - .option("checkpointLocation", checkpointLocationURI.toString) + .option("checkpointLocation", checkpointLocation.toString) .trigger(ProcessingTime(10.seconds)) .start() q.processAllAvailable() @@ -378,14 +377,14 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { verify(LastOptions.mockStreamSourceProvider).createSource( any(), - meq(s"${makeQualifiedPath(checkpointLocationURI.toString)}/sources/0"), + meq(s"${new Path(makeQualifiedPath(checkpointLocation.toString)).toString}/sources/0"), meq(None), meq("org.apache.spark.sql.streaming.test"), meq(Map.empty)) verify(LastOptions.mockStreamSourceProvider).createSource( any(), - meq(s"${makeQualifiedPath(checkpointLocationURI.toString)}/sources/1"), + meq(s"${new Path(makeQualifiedPath(checkpointLocation.toString)).toString}/sources/1"), meq(None), meq("org.apache.spark.sql.streaming.test"), meq(Map.empty)) @@ -613,6 +612,21 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { } } + test("configured checkpoint dir should not be deleted if a query is stopped without errors and" + + " force temp checkpoint deletion enabled") { + import testImplicits._ + withTempDir { checkpointPath => + withSQLConf(SQLConf.CHECKPOINT_LOCATION.key -> checkpointPath.getAbsolutePath, + SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key -> "true") { + val ds = MemoryStream[Int].toDS + val query = ds.writeStream.format("console").start() + assert(checkpointPath.exists()) + query.stop() + assert(checkpointPath.exists()) + } + } + } + test("temp checkpoint dir should be deleted if a query is stopped without errors") { import testImplicits._ val query = MemoryStream[Int].toDS.writeStream.format("console").start() @@ -626,6 +640,17 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { } testQuietly("temp checkpoint dir should not be deleted if a query is stopped with an error") { + testTempCheckpointWithFailedQuery(false) + } + + testQuietly("temp checkpoint should be deleted if a query is stopped with an error and force" + + " temp checkpoint deletion enabled") { + withSQLConf(SQLConf.FORCE_DELETE_TEMP_CHECKPOINT_LOCATION.key -> "true") { + testTempCheckpointWithFailedQuery(true) + } + } + + private def testTempCheckpointWithFailedQuery(checkpointMustBeDeleted: Boolean): Unit = { import testImplicits._ val input = MemoryStream[Int] val query = input.toDS.map(_ / 0).writeStream.format("console").start() @@ -637,7 +662,11 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { intercept[StreamingQueryException] { query.awaitTermination() } - assert(fs.exists(checkpointDir)) + if (!checkpointMustBeDeleted) { + assert(fs.exists(checkpointDir)) + } else { + assert(!fs.exists(checkpointDir)) + } } test("SPARK-20431: Specify a schema by using a DDL-formatted string") { @@ -651,4 +680,27 @@ class DataStreamReaderWriterSuite extends StreamTest with BeforeAndAfter { LastOptions.clear() } + + test("SPARK-26586: Streams should have isolated confs") { + import testImplicits._ + val input = MemoryStream[Int] + input.addData(1 to 10) + spark.conf.set("testKey1", 0) + val queries = (1 to 10).map { i => + spark.conf.set("testKey1", i) + input.toDF().writeStream + .foreachBatch { (df: Dataset[Row], id: Long) => + val v = df.sparkSession.conf.get("testKey1").toInt + if (i != v) { + throw new ConcurrentModificationException(s"Stream $i has the wrong conf value $v") + } + } + .start() + } + try { + queries.foreach(_.processAllAvailable()) + } finally { + queries.foreach(_.stop()) + } + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala index 19ab2ff13e14e..67158fb99d13d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/streaming/util/BlockingSource.scala @@ -60,11 +60,7 @@ class BlockingSource extends StreamSourceProvider with StreamSinkProvider { spark: SQLContext, parameters: Map[String, String], partitionColumns: Seq[String], - outputMode: OutputMode): Sink = { - new Sink { - override def addBatch(batchId: Long, data: DataFrame): Unit = {} - } - } + outputMode: OutputMode): Sink = (_: Long, _: DataFrame) => {} } object BlockingSource { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala index 237872585e11d..126e23e6e5926 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/DataFrameReaderWriterSuite.scala @@ -23,6 +23,13 @@ import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.JavaConverters._ +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.parquet.hadoop.ParquetFileReader +import org.apache.parquet.hadoop.util.HadoopInputFile +import org.apache.parquet.schema.PrimitiveType +import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName +import org.apache.parquet.schema.Type.Repetition import org.scalatest.BeforeAndAfter import org.apache.spark.SparkContext @@ -31,9 +38,15 @@ import org.apache.spark.internal.io.HadoopMapReduceCommitProtocol import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.plans.logical.{AppendData, LogicalPlan, OverwriteByExpression} +import org.apache.spark.sql.execution.QueryExecution +import org.apache.spark.sql.execution.datasources.DataSourceUtils +import org.apache.spark.sql.execution.datasources.noop.NoopDataSource +import org.apache.spark.sql.execution.datasources.parquet.SpecificParquetRecordReaderBase import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ +import org.apache.spark.sql.util.QueryExecutionListener import org.apache.spark.util.Utils @@ -211,16 +224,86 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be assert(LastOptions.parameters("opt3") == "3") } - test("save mode") { - val df = spark.read + test("pass partitionBy as options") { + Seq(1).toDF.write .format("org.apache.spark.sql.test") - .load() + .partitionBy("col1", "col2") + .save() - df.write + val partColumns = LastOptions.parameters(DataSourceUtils.PARTITIONING_COLUMNS_KEY) + assert(DataSourceUtils.decodePartitioningColumns(partColumns) === Seq("col1", "col2")) + } + + test("save mode") { + spark.range(10).write .format("org.apache.spark.sql.test") .mode(SaveMode.ErrorIfExists) .save() assert(LastOptions.saveMode === SaveMode.ErrorIfExists) + + spark.range(10).write + .format("org.apache.spark.sql.test") + .mode(SaveMode.Append) + .save() + assert(LastOptions.saveMode === SaveMode.Append) + + // By default the save mode is `ErrorIfExists` for data source v1. + spark.range(10).write + .format("org.apache.spark.sql.test") + .save() + assert(LastOptions.saveMode === SaveMode.ErrorIfExists) + + spark.range(10).write + .format("org.apache.spark.sql.test") + .mode("default") + .save() + assert(LastOptions.saveMode === SaveMode.ErrorIfExists) + } + + test("save mode for data source v2") { + var plan: LogicalPlan = null + val listener = new QueryExecutionListener { + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = { + plan = qe.analyzed + + } + override def onFailure(funcName: String, qe: QueryExecution, exception: Exception): Unit = {} + } + + spark.listenerManager.register(listener) + try { + // append mode creates `AppendData` + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .mode(SaveMode.Append) + .save() + sparkContext.listenerBus.waitUntilEmpty(1000) + assert(plan.isInstanceOf[AppendData]) + + // overwrite mode creates `OverwriteByExpression` + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .mode(SaveMode.Overwrite) + .save() + sparkContext.listenerBus.waitUntilEmpty(1000) + assert(plan.isInstanceOf[OverwriteByExpression]) + + // By default the save mode is `ErrorIfExists` for data source v2. + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .save() + sparkContext.listenerBus.waitUntilEmpty(1000) + assert(plan.isInstanceOf[AppendData]) + + spark.range(10).write + .format(classOf[NoopDataSource].getName) + .mode("default") + .save() + sparkContext.listenerBus.waitUntilEmpty(1000) + assert(plan.isInstanceOf[AppendData]) + } finally { + spark.listenerManager.unregister(listener) + } } test("test path option in load") { @@ -326,7 +409,7 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be test("write path implements onTaskCommit API correctly") { withSQLConf( - "spark.sql.sources.commitProtocolClass" -> + SQLConf.FILE_COMMIT_PROTOCOL_CLASS.key -> classOf[MessageCapturingCommitProtocol].getCanonicalName) { withTempDir { dir => val path = dir.getCanonicalPath @@ -522,11 +605,12 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Seq("json", "orc", "parquet", "csv").foreach { format => val schema = StructType( StructField("cl1", IntegerType, nullable = false).withComment("test") :: - StructField("cl2", IntegerType, nullable = true) :: - StructField("cl3", IntegerType, nullable = true) :: Nil) + StructField("cl2", IntegerType, nullable = true) :: + StructField("cl3", IntegerType, nullable = true) :: Nil) val row = Row(3, null, 4) val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + // if we write and then read, the read will enforce schema to be nullable val tableName = "tab" withTable(tableName) { df.write.format(format).mode("overwrite").saveAsTable(tableName) @@ -536,12 +620,41 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be Row("cl1", "test") :: Nil) // Verify the schema val expectedFields = schema.fields.map(f => f.copy(nullable = true)) - assert(spark.table(tableName).schema == schema.copy(fields = expectedFields)) + assert(spark.table(tableName).schema === schema.copy(fields = expectedFields)) } } } } + test("parquet - column nullability -- write only") { + val schema = StructType( + StructField("cl1", IntegerType, nullable = false) :: + StructField("cl2", IntegerType, nullable = true) :: Nil) + val row = Row(3, 4) + val df = spark.createDataFrame(sparkContext.parallelize(row :: Nil), schema) + + withTempPath { dir => + val path = dir.getAbsolutePath + df.write.mode("overwrite").parquet(path) + val file = SpecificParquetRecordReaderBase.listDirectory(dir).get(0) + + val hadoopInputFile = HadoopInputFile.fromPath(new Path(file), new Configuration()) + val f = ParquetFileReader.open(hadoopInputFile) + val parquetSchema = f.getFileMetaData.getSchema.getColumns.asScala + .map(_.getPrimitiveType) + f.close() + + // the write keeps nullable info from the schema + val expectedParquetSchema = Seq( + new PrimitiveType(Repetition.REQUIRED, PrimitiveTypeName.INT32, "cl1"), + new PrimitiveType(Repetition.OPTIONAL, PrimitiveTypeName.INT32, "cl2") + ) + + assert (expectedParquetSchema === parquetSchema) + } + + } + test("SPARK-17230: write out results of decimal calculation") { val df = spark.range(99, 101) .selectExpr("id", "cast(id as long) * cast('1.0' as decimal(38, 18)) as num") @@ -800,6 +913,12 @@ class DataFrameReaderWriterSuite extends QueryTest with SharedSQLContext with Be checkReadUserSpecifiedDataColumnDuplication( Seq((1, 1)).toDF("c0", "c1"), "parquet", c0, c1, src) checkReadPartitionColumnDuplication("parquet", c0, c1, src) + + // Check ORC format + checkWriteDataColumnDuplication("orc", c0, c1, src) + checkReadUserSpecifiedDataColumnDuplication( + Seq((1, 1)).toDF("c0", "c1"), "orc", c0, c1, src) + checkReadPartitionColumnDuplication("orc", c0, c1, src) } } } 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 6b03d1e5b7662..115536da8949e 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 @@ -40,6 +40,8 @@ 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.datasources.DataSourceUtils +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.util.UninterruptibleThread import org.apache.spark.util.Utils @@ -65,6 +67,31 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with } } + /** + * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` + * returns. + */ + protected override def withTempDir(f: File => Unit): Unit = { + super.withTempDir { dir => + f(dir) + waitForTasksToFinish() + } + } + + /** + * A helper function for turning off/on codegen. + */ + protected def testWithWholeStageCodegenOnAndOff(testName: String)(f: String => Unit): Unit = { + Seq("false", "true").foreach { codegenEnabled => + val isTurnOn = if (codegenEnabled == "true") "on" else "off" + test(s"$testName (whole-stage-codegen ${isTurnOn})") { + withSQLConf(SQLConf.WHOLESTAGE_CODEGEN_ENABLED.key -> codegenEnabled) { + f(codegenEnabled) + } + } + } + } + /** * Materialize the test data immediately after the `SQLContext` is set up. * This is necessary if the data is accessed by name but not through direct reference. @@ -128,43 +155,6 @@ private[sql] trait SQLTestUtils extends SparkFunSuite with SQLTestUtilsBase with test(name) { runOnThread() } } } -} - -/** - * Helper trait that can be extended by all external SQL test suites. - * - * This allows subclasses to plugin a custom `SQLContext`. - * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`. - * - * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is - * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. - */ -private[sql] trait SQLTestUtilsBase - extends Eventually - with BeforeAndAfterAll - with SQLTestData - with PlanTestBase { self: Suite => - - protected def sparkContext = spark.sparkContext - - // Shorthand for running a query using our SQLContext - protected lazy val sql = spark.sql _ - - /** - * A helper object for importing SQL implicits. - * - * Note that the alternative of importing `spark.implicits._` is not possible here. - * This is because we create the `SQLContext` immediately before the first test is run, - * but the implicits import is needed in the constructor. - */ - protected object testImplicits extends SQLImplicits { - protected override def _sqlContext: SQLContext = self.spark.sqlContext - } - - protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { - SparkSession.setActiveSession(spark) - super.withSQLConf(pairs: _*)(f) - } /** * Copy file in jar's resource to a temp file, then pass it to `f`. @@ -191,21 +181,6 @@ private[sql] trait SQLTestUtilsBase } } - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - * - * @todo Probably this method should be moved to a more general place - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally { - // wait for all tasks to finish before deleting files - waitForTasksToFinish() - Utils.deleteRecursively(dir) - } - } - /** * Creates the specified number of temporary directories, which is then passed to `f` and will be * deleted after `f` returns. @@ -218,6 +193,43 @@ private[sql] trait SQLTestUtilsBase files.foreach(Utils.deleteRecursively) } } +} + +/** + * Helper trait that can be extended by all external SQL test suites. + * + * This allows subclasses to plugin a custom `SQLContext`. + * To use implicit methods, import `testImplicits._` instead of through the `SQLContext`. + * + * Subclasses should *not* create `SQLContext`s in the test suite constructor, which is + * prone to leaving multiple overlapping [[org.apache.spark.SparkContext]]s in the same JVM. + */ +private[sql] trait SQLTestUtilsBase + extends Eventually + with BeforeAndAfterAll + with SQLTestData + with PlanTestBase { self: Suite => + + protected def sparkContext = spark.sparkContext + + // Shorthand for running a query using our SQLContext + protected lazy val sql = spark.sql _ + + /** + * A helper object for importing SQL implicits. + * + * Note that the alternative of importing `spark.implicits._` is not possible here. + * This is because we create the `SQLContext` immediately before the first test is run, + * but the implicits import is needed in the constructor. + */ + protected object testImplicits extends SQLImplicits { + protected override def _sqlContext: SQLContext = self.spark.sqlContext + } + + protected override def withSQLConf(pairs: (String, String)*)(f: => Unit): Unit = { + SparkSession.setActiveSession(spark) + super.withSQLConf(pairs: _*)(f) + } /** * Drops functions after calling `f`. A function is represented by (functionName, isTemporary). @@ -244,11 +256,13 @@ private[sql] trait SQLTestUtilsBase * Drops temporary view `viewNames` after calling `f`. */ protected def withTempView(viewNames: String*)(f: => Unit): Unit = { - try f finally { - // If the test failed part way, we don't want to mask the failure by failing to remove - // temp views that never got created. - try viewNames.foreach(spark.catalog.dropTempView) catch { - case _: NoSuchTableException => + Utils.tryWithSafeFinally(f) { + viewNames.foreach { viewName => + try spark.catalog.dropTempView(viewName) catch { + // If the test failed part way, we don't want to mask the failure by failing to remove + // temp views that never got created. + case _: NoSuchTableException => + } } } } @@ -257,11 +271,13 @@ private[sql] trait SQLTestUtilsBase * Drops global temporary view `viewNames` after calling `f`. */ protected def withGlobalTempView(viewNames: String*)(f: => Unit): Unit = { - try f finally { - // If the test failed part way, we don't want to mask the failure by failing to remove - // global temp views that never got created. - try viewNames.foreach(spark.catalog.dropGlobalTempView) catch { - case _: NoSuchTableException => + Utils.tryWithSafeFinally(f) { + viewNames.foreach { viewName => + try spark.catalog.dropGlobalTempView(viewName) catch { + // If the test failed part way, we don't want to mask the failure by failing to remove + // global temp views that never got created. + case _: NoSuchTableException => + } } } } @@ -270,7 +286,7 @@ private[sql] trait SQLTestUtilsBase * Drops table `tableName` after calling `f`. */ protected def withTable(tableNames: String*)(f: => Unit): Unit = { - try f finally { + Utils.tryWithSafeFinally(f) { tableNames.foreach { name => spark.sql(s"DROP TABLE IF EXISTS $name") } @@ -281,13 +297,37 @@ private[sql] trait SQLTestUtilsBase * Drops view `viewName` after calling `f`. */ protected def withView(viewNames: String*)(f: => Unit): Unit = { - try f finally { + Utils.tryWithSafeFinally(f)( viewNames.foreach { name => spark.sql(s"DROP VIEW IF EXISTS $name") } + ) + } + + /** + * Drops cache `cacheName` after calling `f`. + */ + protected def withCache(cacheNames: String*)(f: => Unit): Unit = { + Utils.tryWithSafeFinally(f) { + cacheNames.foreach { cacheName => + try uncacheTable(cacheName) catch { + case _: AnalysisException => + } + } } } + // Blocking uncache table for tests + protected def uncacheTable(tableName: String): Unit = { + val tableIdent = spark.sessionState.sqlParser.parseTableIdentifier(tableName) + val cascade = !spark.sessionState.catalog.isTemporaryTable(tableIdent) + spark.sharedState.cacheManager.uncacheQuery( + spark, + spark.table(tableName).logicalPlan, + cascade = cascade, + blocking = true) + } + /** * Creates a temporary database and switches current database to it before executing `f`. This * database is dropped after `f` returns. @@ -315,7 +355,7 @@ private[sql] trait SQLTestUtilsBase * Drops database `dbName` after calling `f`. */ protected def withDatabase(dbNames: String*)(f: => Unit): Unit = { - try f finally { + Utils.tryWithSafeFinally(f) { dbNames.foreach { name => spark.sql(s"DROP DATABASE IF EXISTS $name CASCADE") } @@ -344,7 +384,7 @@ private[sql] trait SQLTestUtilsBase */ protected def activateDatabase(db: String)(f: => Unit): Unit = { spark.sessionState.catalog.setCurrentDatabase(db) - try f finally spark.sessionState.catalog.setCurrentDatabase("default") + Utils.tryWithSafeFinally(f)(spark.sessionState.catalog.setCurrentDatabase("default")) } /** @@ -352,7 +392,7 @@ private[sql] trait SQLTestUtilsBase */ protected def stripSparkFilter(df: DataFrame): DataFrame = { val schema = df.schema - val withoutFilters = df.queryExecution.sparkPlan.transform { + val withoutFilters = df.queryExecution.executedPlan.transform { case FilterExec(_, child) => child } @@ -385,6 +425,14 @@ private[sql] trait SQLTestUtilsBase protected def testFile(fileName: String): String = { Thread.currentThread().getContextClassLoader.getResource(fileName).toString } + + /** + * Returns the size of the local directory except the metadata file and the temporary file. + */ + def getLocalDirSize(file: File): Long = { + assert(file.isDirectory) + file.listFiles.filter(f => DataSourceUtils.isDataFile(f.getName)).map(_.length).sum + } } private[sql] object SQLTestUtils { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala index e7e0ce64963a3..ff6211b95042f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SharedSparkSession.scala @@ -23,9 +23,10 @@ import org.scalatest.{BeforeAndAfterEach, Suite} import org.scalatest.concurrent.Eventually import org.apache.spark.{DebugFilesystem, SparkConf} +import org.apache.spark.internal.config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.optimizer.ConvertToLocalRelation -import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.{SQLConf, StaticSQLConf} /** * Helper trait for SQL test suites where all tests share a single [[TestSparkSession]]. @@ -36,15 +37,18 @@ trait SharedSparkSession with Eventually { self: Suite => protected def sparkConf = { - new SparkConf() + val conf = new SparkConf() .set("spark.hadoop.fs.file.impl", classOf[DebugFilesystem].getName) - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) .set(SQLConf.CODEGEN_FALLBACK.key, "false") // Disable ConvertToLocalRelation for better test coverage. Test cases built on // LocalRelation will exercise the optimization rules better by disabling it as // this rule may potentially block testing of other optimization rules such as // ConstantPropagation etc. .set(SQLConf.OPTIMIZER_EXCLUDED_RULES.key, ConvertToLocalRelation.ruleName) + conf.set( + StaticSQLConf.WAREHOUSE_PATH, + conf.get(StaticSQLConf.WAREHOUSE_PATH) + "/" + getClass.getCanonicalName) } /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala index a239e39d9c5a3..c347caef39a64 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/DataFrameCallbackSuite.scala @@ -48,6 +48,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { df.select("i").collect() df.filter($"i" > 0).count() + sparkContext.listenerBus.waitUntilEmpty(1000) assert(metrics.length == 2) assert(metrics(0)._1 == "collect") @@ -78,6 +79,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { val e = intercept[SparkException](df.select(errorUdf($"i")).collect()) + sparkContext.listenerBus.waitUntilEmpty(1000) assert(metrics.length == 1) assert(metrics(0)._1 == "collect") assert(metrics(0)._2.analyzed.isInstanceOf[Project]) @@ -103,10 +105,16 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { spark.listenerManager.register(listener) val df = Seq(1 -> "a").toDF("i", "j").groupBy("i").count() + df.collect() + // Wait for the first `collect` to be caught by our listener. Otherwise the next `collect` will + // reset the plan metrics. + sparkContext.listenerBus.waitUntilEmpty(1000) df.collect() + Seq(1 -> "a", 2 -> "a").toDF("i", "j").groupBy("i").count().collect() + sparkContext.listenerBus.waitUntilEmpty(1000) assert(metrics.length == 3) assert(metrics(0) === 1) assert(metrics(1) === 1) @@ -142,7 +150,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { def getPeakExecutionMemory(stageId: Int): Long = { val peakMemoryAccumulator = sparkListener.getCompletedStageInfos(stageId).accumulables - .filter(_._2.name == InternalAccumulator.PEAK_EXECUTION_MEMORY) + .filter(_._2.name == Some(InternalAccumulator.PEAK_EXECUTION_MEMORY)) assert(peakMemoryAccumulator.size == 1) peakMemoryAccumulator.head._2.value.get.asInstanceOf[Long] @@ -154,6 +162,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { // For this simple case, the peakExecutionMemory of a stage should be the data size of the // aggregate operator, as we only have one memory consuming operator per stage. + sparkContext.listenerBus.waitUntilEmpty(1000) assert(metrics.length == 2) assert(metrics(0) == topAggDataSize) assert(metrics(1) == bottomAggDataSize) @@ -177,6 +186,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { withTempPath { path => spark.range(10).write.format("json").save(path.getCanonicalPath) + sparkContext.listenerBus.waitUntilEmpty(1000) assert(commands.length == 1) assert(commands.head._1 == "save") assert(commands.head._2.isInstanceOf[InsertIntoHadoopFsRelationCommand]) @@ -187,16 +197,18 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { withTable("tab") { sql("CREATE TABLE tab(i long) using parquet") // adds commands(1) via onSuccess spark.range(10).write.insertInto("tab") + sparkContext.listenerBus.waitUntilEmpty(1000) assert(commands.length == 3) assert(commands(2)._1 == "insertInto") assert(commands(2)._2.isInstanceOf[InsertIntoTable]) assert(commands(2)._2.asInstanceOf[InsertIntoTable].table - .asInstanceOf[UnresolvedRelation].tableIdentifier.table == "tab") + .asInstanceOf[UnresolvedRelation].multipartIdentifier == Seq("tab")) } // exiting withTable adds commands(3) via onSuccess (drops tab) withTable("tab") { spark.range(10).select($"id", $"id" % 5 as "p").write.partitionBy("p").saveAsTable("tab") + sparkContext.listenerBus.waitUntilEmpty(1000) assert(commands.length == 5) assert(commands(4)._1 == "saveAsTable") assert(commands(4)._2.isInstanceOf[CreateTable]) @@ -208,6 +220,7 @@ class DataFrameCallbackSuite extends QueryTest with SharedSQLContext { val e = intercept[AnalysisException] { spark.range(10).select($"id", $"id").write.insertInto("tab") } + sparkContext.listenerBus.waitUntilEmpty(1000) assert(exceptions.length == 1) assert(exceptions.head._1 == "insertInto") assert(exceptions.head._2 == e) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala index 4205e23ae240a..da414f4311e57 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/util/ExecutionListenerManagerSuite.scala @@ -20,26 +20,28 @@ package org.apache.spark.sql.util import java.util.concurrent.atomic.AtomicInteger import org.apache.spark._ +import org.apache.spark.sql.{LocalSparkSession, SparkSession} import org.apache.spark.sql.execution.QueryExecution import org.apache.spark.sql.internal.StaticSQLConf._ -class ExecutionListenerManagerSuite extends SparkFunSuite { +class ExecutionListenerManagerSuite extends SparkFunSuite with LocalSparkSession { import CountingQueryExecutionListener._ test("register query execution listeners using configuration") { val conf = new SparkConf(false) .set(QUERY_EXECUTION_LISTENERS, Seq(classOf[CountingQueryExecutionListener].getName())) + spark = SparkSession.builder().master("local").appName("test").config(conf).getOrCreate() - val mgr = new ExecutionListenerManager(conf) + spark.sql("select 1").collect() + spark.sparkContext.listenerBus.waitUntilEmpty(1000) assert(INSTANCE_COUNT.get() === 1) - mgr.onSuccess(null, null, 42L) assert(CALLBACK_COUNT.get() === 1) - val clone = mgr.clone() + val cloned = spark.cloneSession() + cloned.sql("select 1").collect() + spark.sparkContext.listenerBus.waitUntilEmpty(1000) assert(INSTANCE_COUNT.get() === 1) - - clone.onSuccess(null, null, 42L) assert(CALLBACK_COUNT.get() === 2) } diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java similarity index 100% rename from sql/core/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java rename to sql/core/v1.2.1/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala new file mode 100644 index 0000000000000..995c5ed317de1 --- /dev/null +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -0,0 +1,270 @@ +/* + * 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 org.apache.orc.storage.common.`type`.HiveDecimal +import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} +import org.apache.orc.storage.ql.io.sarg.SearchArgument.Builder +import org.apache.orc.storage.ql.io.sarg.SearchArgumentFactory.newBuilder +import org.apache.orc.storage.serde2.io.HiveDecimalWritable + +import org.apache.spark.SparkException +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types._ + +/** + * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. + * + * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and + * conversion passes through the Filter to make sure we only convert predicates that are known + * to be convertible. + * + * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't + * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite + * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using + * existing simpler ones. + * + * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and + * `startNot()` mutate internal state of the builder instance. This forces us to translate all + * convertible filters with a single builder instance. However, if we try to translate a filter + * before checking whether it can be converted or not, we may end up with a builder whose internal + * state is inconsistent in the case of an inconvertible filter. + * + * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then + * try to convert its children. Say we convert `left` child successfully, but find that `right` + * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent + * now. + * + * The workaround employed here is to trim the Spark filters before trying to convert them. This + * way, we can only do the actual conversion on the part of the Filter that is known to be + * convertible. + * + * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of + * builder methods mentioned above can only be found in test code, where all tested filters are + * known to be convertible. + */ +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 + // Combines all convertible filters using `And` to produce a single conjunction + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + 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 + // following recursive method call `buildSearchArgument`. + buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() + } + } + + def convertibleFilters( + schema: StructType, + dataTypeMap: Map[String, DataType], + filters: Seq[Filter]): Seq[Filter] = { + import org.apache.spark.sql.sources._ + + def convertibleFiltersHelper( + filter: Filter, + canPartialPushDown: Boolean): Option[Filter] = filter match { + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + case And(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) + case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) + case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) + case _ => None + } + + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + case Or(left, right) => + for { + lhs <- convertibleFiltersHelper(left, canPartialPushDown) + rhs <- convertibleFiltersHelper(right, canPartialPushDown) + } yield Or(lhs, rhs) + case Not(pred) => + val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) + childResultOptional.map(Not) + case other => + for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other + } + filters.flatMap { filter => + convertibleFiltersHelper(filter, true) + } + } + + /** + * Get PredicateLeafType which is corresponding to the given DataType. + */ + private def getPredicateLeafType(dataType: DataType) = dataType match { + case BooleanType => PredicateLeaf.Type.BOOLEAN + case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG + case FloatType | DoubleType => PredicateLeaf.Type.FLOAT + case StringType => PredicateLeaf.Type.STRING + case DateType => PredicateLeaf.Type.DATE + case TimestampType => PredicateLeaf.Type.TIMESTAMP + case _: DecimalType => PredicateLeaf.Type.DECIMAL + case _ => throw new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") + } + + /** + * Cast literal values for filters. + * + * We need to cast to long because ORC raises exceptions + * at 'checkLiteralType' of SearchArgumentImpl.java. + */ + private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { + case ByteType | ShortType | IntegerType | LongType => + value.asInstanceOf[Number].longValue + case FloatType | DoubleType => + value.asInstanceOf[Number].doubleValue() + case _: DecimalType => + new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) + case _ => value + } + + /** + * Build a SearchArgument and return the builder so far. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input predicates, which should be fully convertible to SearchArgument. + * @param builder the input SearchArgument.Builder. + * @return the builder so far. + */ + private def buildSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Builder = { + import org.apache.spark.sql.sources._ + + expression match { + case And(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Or(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Not(child) => + buildSearchArgument(dataTypeMap, child, builder.startNot()).end() + + case other => + buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { + throw new SparkException( + "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") + } + } + } + + /** + * Build a SearchArgument for a leaf predicate and return the builder so far. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @param builder the input SearchArgument.Builder. + * @return the builder so far. + */ + private def buildLeafSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Builder] = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + + // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` + // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be + // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). + expression match { + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + Some(builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) + + case _ => None + } + } +} + diff --git a/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala new file mode 100644 index 0000000000000..68503aba22b40 --- /dev/null +++ b/sql/core/v1.2.1/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -0,0 +1,66 @@ +/* + * 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.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} +import org.apache.orc.storage.ql.io.sarg.PredicateLeaf.{Operator => OrcOperator} +import org.apache.orc.storage.serde2.io.{DateWritable, HiveDecimalWritable} + +import org.apache.spark.sql.catalyst.expressions.SpecializedGetters +import org.apache.spark.sql.types.Decimal + +/** + * Various utilities for ORC used to upgrade the built-in Hive. + */ +private[sql] object OrcShimUtils { + + class VectorizedRowBatchWrap(val batch: VectorizedRowBatch) {} + + private[sql] type Operator = OrcOperator + private[sql] type SearchArgument = OrcSearchArgument + + def getSqlDate(value: Any): Date = value.asInstanceOf[DateWritable].get + + def getDecimal(value: Any): Decimal = { + val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() + Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + } + + def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { + if (reuseObj) { + val result = new DateWritable() + (getter, ordinal) => + result.set(getter.getInt(ordinal)) + result + } else { + (getter: SpecializedGetters, ordinal: Int) => + new DateWritable(getter.getInt(ordinal)) + } + } + + def getHiveDecimalWritable(precision: Int, scale: Int): + (SpecializedGetters, Int) => HiveDecimalWritable = { + (getter, ordinal) => + val d = getter.getDecimal(ordinal, precision, scale) + new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala similarity index 84% rename from sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala rename to sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala index ee12f30892436..535c32396b593 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala +++ b/sql/core/v1.2.1/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution.datasources.orc +import java.math.MathContext import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} @@ -24,11 +25,13 @@ import scala.collection.JavaConverters._ import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, SearchArgument} -import org.apache.spark.sql.{Column, DataFrame} +import org.apache.spark.sql.{AnalysisException, Column, DataFrame} import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable import org.apache.spark.sql.test.SharedSQLContext import org.apache.spark.sql.types._ @@ -41,7 +44,7 @@ import org.apache.spark.sql.types._ */ class OrcFilterSuite extends OrcTest with SharedSQLContext { - private def checkFilterPredicate( + protected def checkFilterPredicate( df: DataFrame, predicate: Predicate, checker: (SearchArgument) => Unit): Unit = { @@ -50,24 +53,24 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { .select(output.map(e => Column(e)): _*) .where(Column(predicate)) - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $selectedFilters") - checker(maybeFilter.get) + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcTable: OrcTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(options) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") + checker(maybeFilter.get) + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, filterOperator: PredicateLeaf.Operator) (implicit df: DataFrame): Unit = { def checkComparisonOperator(filter: SearchArgument) = { @@ -77,7 +80,7 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkComparisonOperator) } - private def checkFilterPredicate + protected def checkFilterPredicate (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { def checkLogicalOperator(filter: SearchArgument) = { @@ -86,30 +89,6 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { checkFilterPredicate(df, predicate, checkLogicalOperator) } - private def checkNoFilterPredicate - (predicate: Predicate) - (implicit df: DataFrame): Unit = { - val output = predicate.collect { case a: Attribute => a }.distinct - val query = df - .select(output.map(e => Column(e)): _*) - .where(Column(predicate)) - - var maybeRelation: Option[HadoopFsRelation] = None - val maybeAnalyzedPredicate = query.queryExecution.optimizedPlan.collect { - case PhysicalOperation(_, filters, LogicalRelation(orcRelation: HadoopFsRelation, _, _, _)) => - maybeRelation = Some(orcRelation) - filters - }.flatten.reduceLeftOption(_ && _) - assert(maybeAnalyzedPredicate.isDefined, "No filter is analyzed from the given query") - - val (_, selectedFilters, _) = - DataSourceStrategy.selectFilters(maybeRelation.get, maybeAnalyzedPredicate.toSeq) - assert(selectedFilters.nonEmpty, "No filter is pushed down") - - val maybeFilter = OrcFilters.createFilter(query.schema, selectedFilters) - assert(maybeFilter.isEmpty, s"Could generate filter predicate for $selectedFilters") - } - test("filter pushdown - integer") { withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) @@ -346,15 +325,15 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { } // ArrayType withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => - checkNoFilterPredicate('_1.isNull) + checkNoFilterPredicate('_1.isNull, noneSupported = true) } // BinaryType withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => - checkNoFilterPredicate('_1 <=> 1.b) + checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => - checkNoFilterPredicate('_1.isNotNull) + checkNoFilterPredicate('_1.isNotNull, noneSupported = true) } } @@ -383,17 +362,6 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { )).get.toString } - // Can not remove unsupported `StringContains` predicate since it is under `Or` operator. - assert(OrcFilters.createFilter(schema, Array( - Or( - LessThan("a", 10), - And( - StringContains("b", "prefix"), - GreaterThan("a", 1) - ) - ) - )).isEmpty) - // Safely remove unsupported `StringContains` predicate and push down `LessThan` assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { OrcFilters.createFilter(schema, Array( @@ -418,4 +386,66 @@ class OrcFilterSuite extends OrcTest with SharedSQLContext { )).get.toString } } + + test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { + import org.apache.spark.sql.sources._ + // The `LessThan` should be converted while the `StringContains` shouldn't + val schema = new StructType( + Array( + StructField("a", IntegerType, nullable = true), + StructField("b", StringType, nullable = true))) + + // The predicate `StringContains` predicate is not able to be pushed down. + assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + + " expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + GreaterThan("a", 10), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + + " expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + And( + GreaterThan("a", 10), + StringContains("b", "foobar") + ), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assert(OrcFilters.createFilter(schema, Array( + Or( + StringContains("b", "foobar"), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).isEmpty) + } + + test("SPARK-27160: Fix casting of the DecimalType literal") { + import org.apache.spark.sql.sources._ + val schema = StructType(Array(StructField("a", DecimalType(3, 2)))) + assertResult("leaf-0 = (LESS_THAN a 3.14), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + LessThan( + "a", + new java.math.BigDecimal(3.14, MathContext.DECIMAL64).setScale(2))) + ).get.toString + } + } } + diff --git a/sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java b/sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java new file mode 100644 index 0000000000000..2f1925e69a337 --- /dev/null +++ b/sql/core/v2.3.5/src/main/java/org/apache/spark/sql/execution/datasources/orc/OrcColumnVector.java @@ -0,0 +1,193 @@ +/* + * 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.math.BigDecimal; + +import org.apache.hadoop.hive.ql.exec.vector.*; + +import org.apache.spark.sql.types.DataType; +import org.apache.spark.sql.types.Decimal; +import org.apache.spark.sql.types.TimestampType; +import org.apache.spark.sql.vectorized.ColumnarArray; +import org.apache.spark.sql.vectorized.ColumnarMap; +import org.apache.spark.unsafe.types.UTF8String; + +/** + * A column vector class wrapping Hive's ColumnVector. Because Spark ColumnarBatch only accepts + * Spark's vectorized.ColumnVector, this column vector is used to adapt Hive ColumnVector with + * Spark ColumnarVector. + */ +public class OrcColumnVector extends org.apache.spark.sql.vectorized.ColumnVector { + private ColumnVector baseData; + private LongColumnVector longData; + private DoubleColumnVector doubleData; + private BytesColumnVector bytesData; + private DecimalColumnVector decimalData; + private TimestampColumnVector timestampData; + private final boolean isTimestamp; + + private int batchSize; + + OrcColumnVector(DataType type, ColumnVector vector) { + super(type); + + if (type instanceof TimestampType) { + isTimestamp = true; + } else { + isTimestamp = false; + } + + baseData = vector; + if (vector instanceof LongColumnVector) { + longData = (LongColumnVector) vector; + } else if (vector instanceof DoubleColumnVector) { + doubleData = (DoubleColumnVector) vector; + } else if (vector instanceof BytesColumnVector) { + bytesData = (BytesColumnVector) vector; + } else if (vector instanceof DecimalColumnVector) { + decimalData = (DecimalColumnVector) vector; + } else if (vector instanceof TimestampColumnVector) { + timestampData = (TimestampColumnVector) vector; + } else { + throw new UnsupportedOperationException(); + } + } + + public void setBatchSize(int batchSize) { + this.batchSize = batchSize; + } + + @Override + public void close() { + + } + + @Override + public boolean hasNull() { + return !baseData.noNulls; + } + + @Override + public int numNulls() { + if (baseData.isRepeating) { + if (baseData.isNull[0]) { + return batchSize; + } else { + return 0; + } + } else if (baseData.noNulls) { + return 0; + } else { + int count = 0; + for (int i = 0; i < batchSize; i++) { + if (baseData.isNull[i]) count++; + } + return count; + } + } + + /* A helper method to get the row index in a column. */ + private int getRowIndex(int rowId) { + return baseData.isRepeating ? 0 : rowId; + } + + @Override + public boolean isNullAt(int rowId) { + return baseData.isNull[getRowIndex(rowId)]; + } + + @Override + public boolean getBoolean(int rowId) { + return longData.vector[getRowIndex(rowId)] == 1; + } + + @Override + public byte getByte(int rowId) { + return (byte) longData.vector[getRowIndex(rowId)]; + } + + @Override + public short getShort(int rowId) { + return (short) longData.vector[getRowIndex(rowId)]; + } + + @Override + public int getInt(int rowId) { + return (int) longData.vector[getRowIndex(rowId)]; + } + + @Override + public long getLong(int rowId) { + int index = getRowIndex(rowId); + if (isTimestamp) { + return timestampData.time[index] * 1000 + timestampData.nanos[index] / 1000 % 1000; + } else { + return longData.vector[index]; + } + } + + @Override + public float getFloat(int rowId) { + return (float) doubleData.vector[getRowIndex(rowId)]; + } + + @Override + public double getDouble(int rowId) { + return doubleData.vector[getRowIndex(rowId)]; + } + + @Override + public Decimal getDecimal(int rowId, int precision, int scale) { + if (isNullAt(rowId)) return null; + BigDecimal data = decimalData.vector[getRowIndex(rowId)].getHiveDecimal().bigDecimalValue(); + return Decimal.apply(data, precision, scale); + } + + @Override + public UTF8String getUTF8String(int rowId) { + if (isNullAt(rowId)) return null; + int index = getRowIndex(rowId); + BytesColumnVector col = bytesData; + return UTF8String.fromBytes(col.vector[index], col.start[index], col.length[index]); + } + + @Override + public byte[] getBinary(int rowId) { + if (isNullAt(rowId)) return null; + int index = getRowIndex(rowId); + byte[] binary = new byte[bytesData.length[index]]; + System.arraycopy(bytesData.vector[index], bytesData.start[index], binary, 0, binary.length); + return binary; + } + + @Override + public ColumnarArray getArray(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public ColumnarMap getMap(int rowId) { + throw new UnsupportedOperationException(); + } + + @Override + public org.apache.spark.sql.vectorized.ColumnVector getChild(int ordinal) { + throw new UnsupportedOperationException(); + } +} diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala new file mode 100644 index 0000000000000..948ab44a8c19c --- /dev/null +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilters.scala @@ -0,0 +1,269 @@ +/* + * 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 org.apache.hadoop.hive.common.`type`.HiveDecimal +import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder +import org.apache.hadoop.hive.ql.io.sarg.SearchArgumentFactory.newBuilder +import org.apache.hadoop.hive.serde2.io.HiveDecimalWritable + +import org.apache.spark.SparkException +import org.apache.spark.sql.sources.Filter +import org.apache.spark.sql.types._ + +/** + * Helper object for building ORC `SearchArgument`s, which are used for ORC predicate push-down. + * + * Due to limitation of ORC `SearchArgument` builder, we had to implement separate checking and + * conversion passes through the Filter to make sure we only convert predicates that are known + * to be convertible. + * + * An ORC `SearchArgument` must be built in one pass using a single builder. For example, you can't + * build `a = 1` and `b = 2` first, and then combine them into `a = 1 AND b = 2`. This is quite + * different from the cases in Spark SQL or Parquet, where complex filters can be easily built using + * existing simpler ones. + * + * The annoying part is that, `SearchArgument` builder methods like `startAnd()`, `startOr()`, and + * `startNot()` mutate internal state of the builder instance. This forces us to translate all + * convertible filters with a single builder instance. However, if we try to translate a filter + * before checking whether it can be converted or not, we may end up with a builder whose internal + * state is inconsistent in the case of an inconvertible filter. + * + * For example, to convert an `And` filter with builder `b`, we call `b.startAnd()` first, and then + * try to convert its children. Say we convert `left` child successfully, but find that `right` + * child is inconvertible. Alas, `b.startAnd()` call can't be rolled back, and `b` is inconsistent + * now. + * + * The workaround employed here is to trim the Spark filters before trying to convert them. This + * way, we can only do the actual conversion on the part of the Filter that is known to be + * convertible. + * + * P.S.: Hive seems to use `SearchArgument` together with `ExprNodeGenericFuncDesc` only. Usage of + * builder methods mentioned above can only be found in test code, where all tested filters are + * known to be convertible. + */ +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 + // Combines all convertible filters using `And` to produce a single conjunction + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + 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 + // following recursive method call `buildSearchArgument`. + buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() + } + } + + def convertibleFilters( + schema: StructType, + dataTypeMap: Map[String, DataType], + filters: Seq[Filter]): Seq[Filter] = { + import org.apache.spark.sql.sources._ + + def convertibleFiltersHelper( + filter: Filter, + canPartialPushDown: Boolean): Option[Filter] = filter match { + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + case And(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) + case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) + case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) + case _ => None + } + + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + case Or(left, right) => + for { + lhs <- convertibleFiltersHelper(left, canPartialPushDown) + rhs <- convertibleFiltersHelper(right, canPartialPushDown) + } yield Or(lhs, rhs) + case Not(pred) => + val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) + childResultOptional.map(Not) + case other => + for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other + } + filters.flatMap { filter => + convertibleFiltersHelper(filter, true) + } + } + + /** + * Get PredicateLeafType which is corresponding to the given DataType. + */ + private def getPredicateLeafType(dataType: DataType) = dataType match { + case BooleanType => PredicateLeaf.Type.BOOLEAN + case ByteType | ShortType | IntegerType | LongType => PredicateLeaf.Type.LONG + case FloatType | DoubleType => PredicateLeaf.Type.FLOAT + case StringType => PredicateLeaf.Type.STRING + case DateType => PredicateLeaf.Type.DATE + case TimestampType => PredicateLeaf.Type.TIMESTAMP + case _: DecimalType => PredicateLeaf.Type.DECIMAL + case _ => throw new UnsupportedOperationException(s"DataType: ${dataType.catalogString}") + } + + /** + * Cast literal values for filters. + * + * We need to cast to long because ORC raises exceptions + * at 'checkLiteralType' of SearchArgumentImpl.java. + */ + private def castLiteralValue(value: Any, dataType: DataType): Any = dataType match { + case ByteType | ShortType | IntegerType | LongType => + value.asInstanceOf[Number].longValue + case FloatType | DoubleType => + value.asInstanceOf[Number].doubleValue() + case _: DecimalType => + new HiveDecimalWritable(HiveDecimal.create(value.asInstanceOf[java.math.BigDecimal])) + case _ => value + } + + /** + * Build a SearchArgument and return the builder so far. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input predicates, which should be fully convertible to SearchArgument. + * @param builder the input SearchArgument.Builder. + * @return the builder so far. + */ + private def buildSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Builder = { + import org.apache.spark.sql.sources._ + + expression match { + case And(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Or(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Not(child) => + buildSearchArgument(dataTypeMap, child, builder.startNot()).end() + + case other => + buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { + throw new SparkException( + "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") + } + } + } + + /** + * Build a SearchArgument for a leaf predicate and return the builder so far. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input filter predicates. + * @param builder the input SearchArgument.Builder. + * @return the builder so far. + */ + private def buildLeafSearchArgument( + dataTypeMap: Map[String, DataType], + expression: Filter, + builder: Builder): Option[Builder] = { + def getType(attribute: String): PredicateLeaf.Type = + getPredicateLeafType(dataTypeMap(attribute)) + + import org.apache.spark.sql.sources._ + + // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` + // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be + // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). + expression match { + case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().equals(quotedName, getType(attribute), castedValue).end()) + + case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().nullSafeEquals(quotedName, getType(attribute), castedValue).end()) + + case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThan(quotedName, getType(attribute), castedValue).end()) + + case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startAnd().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + + case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThanEquals(quotedName, getType(attribute), castedValue).end()) + + case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValue = castLiteralValue(value, dataTypeMap(attribute)) + Some(builder.startNot().lessThan(quotedName, getType(attribute), castedValue).end()) + + case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startAnd().isNull(quotedName, getType(attribute)).end()) + + case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + Some(builder.startNot().isNull(quotedName, getType(attribute)).end()) + + case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => + val quotedName = quoteAttributeNameIfNeeded(attribute) + val castedValues = values.map(v => castLiteralValue(v, dataTypeMap(attribute))) + Some(builder.startAnd().in(quotedName, getType(attribute), + castedValues.map(_.asInstanceOf[AnyRef]): _*).end()) + + case _ => None + } + } +} diff --git a/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala new file mode 100644 index 0000000000000..c32f024476e69 --- /dev/null +++ b/sql/core/v2.3.5/src/main/scala/org/apache/spark/sql/execution/datasources/orc/OrcShimUtils.scala @@ -0,0 +1,66 @@ +/* + * 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.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} +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.types.Decimal + +/** + * Various utilities for ORC used to upgrade the built-in Hive. + */ +private[sql] object OrcShimUtils { + + class VectorizedRowBatchWrap(val batch: VectorizedRowBatch) {} + + private[sql] type Operator = OrcOperator + private[sql] type SearchArgument = OrcSearchArgument + + def getSqlDate(value: Any): Date = value.asInstanceOf[DateWritable].get + + def getDecimal(value: Any): Decimal = { + val decimal = value.asInstanceOf[HiveDecimalWritable].getHiveDecimal() + Decimal(decimal.bigDecimalValue, decimal.precision(), decimal.scale()) + } + + def getDateWritable(reuseObj: Boolean): (SpecializedGetters, Int) => DateWritable = { + if (reuseObj) { + val result = new DateWritable() + (getter, ordinal) => + result.set(getter.getInt(ordinal)) + result + } else { + (getter: SpecializedGetters, ordinal: Int) => + new DateWritable(getter.getInt(ordinal)) + } + } + + def getHiveDecimalWritable(precision: Int, scale: Int): + (SpecializedGetters, Int) => HiveDecimalWritable = { + (getter, ordinal) => + val d = getter.getDecimal(ordinal, precision, scale) + new HiveDecimalWritable(HiveDecimal.create(d.toJavaBigDecimal)) + } +} diff --git a/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala new file mode 100644 index 0000000000000..0f19c9e40b933 --- /dev/null +++ b/sql/core/v2.3.5/src/test/scala/org/apache/spark/sql/execution/datasources/orc/OrcFilterSuite.scala @@ -0,0 +1,451 @@ +/* + * 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.math.MathContext +import java.nio.charset.StandardCharsets +import java.sql.{Date, Timestamp} + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.io.sarg.{PredicateLeaf, SearchArgument} + +import org.apache.spark.sql.{AnalysisException, Column, DataFrame} +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} +import org.apache.spark.sql.execution.datasources.v2.DataSourceV2Relation +import org.apache.spark.sql.execution.datasources.v2.orc.OrcTable +import org.apache.spark.sql.test.SharedSQLContext +import org.apache.spark.sql.types._ + +/** + * A test suite that tests Apache ORC filter API based filter pushdown optimization. + * OrcFilterSuite and HiveOrcFilterSuite is logically duplicated to provide the same test coverage. + * The difference are the packages containing 'Predicate' and 'SearchArgument' classes. + * - OrcFilterSuite uses 'org.apache.orc.storage.ql.io.sarg' package. + * - HiveOrcFilterSuite uses 'org.apache.hadoop.hive.ql.io.sarg' package. + */ +class OrcFilterSuite extends OrcTest with SharedSQLContext { + + protected def checkFilterPredicate( + df: DataFrame, + predicate: Predicate, + checker: (SearchArgument) => Unit): Unit = { + val output = predicate.collect { case a: Attribute => a }.distinct + val query = df + .select(output.map(e => Column(e)): _*) + .where(Column(predicate)) + + query.queryExecution.optimizedPlan match { + case PhysicalOperation(_, filters, + DataSourceV2Relation(orcTable: OrcTable, _, options)) => + assert(filters.nonEmpty, "No filter is analyzed from the given query") + val scanBuilder = orcTable.newScanBuilder(options) + scanBuilder.pushFilters(filters.flatMap(DataSourceStrategy.translateFilter).toArray) + val pushedFilters = scanBuilder.pushedFilters() + assert(pushedFilters.nonEmpty, "No filter is pushed down") + val maybeFilter = OrcFilters.createFilter(query.schema, pushedFilters) + assert(maybeFilter.isDefined, s"Couldn't generate filter predicate for $pushedFilters") + checker(maybeFilter.get) + + case _ => + throw new AnalysisException("Can not match OrcTable in the query.") + } + } + + protected def checkFilterPredicate + (predicate: Predicate, filterOperator: PredicateLeaf.Operator) + (implicit df: DataFrame): Unit = { + def checkComparisonOperator(filter: SearchArgument) = { + val operator = filter.getLeaves.asScala + assert(operator.map(_.getOperator).contains(filterOperator)) + } + checkFilterPredicate(df, predicate, checkComparisonOperator) + } + + protected def checkFilterPredicate + (predicate: Predicate, stringExpr: String) + (implicit df: DataFrame): Unit = { + def checkLogicalOperator(filter: SearchArgument) = { + assert(filter.toString == stringExpr) + } + checkFilterPredicate(df, predicate, checkLogicalOperator) + } + + test("filter pushdown - integer") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - long") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toLong)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - float") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toFloat)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - double") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i.toDouble)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === 1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> 1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < 2, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > 3, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= 1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= 4, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(1) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(1) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(2) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(3) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(1) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(4) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - string") { + withOrcDataFrame((1 to 4).map(i => Tuple1(i.toString))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === "1", PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> "1", PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < "2", PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > "3", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= "1", PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= "4", PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal("1") === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal("1") <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal("2") > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal("3") < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("1") >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal("4") <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - boolean") { + withOrcDataFrame((true :: false :: Nil).map(b => Tuple1.apply(Option(b)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === true, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> true, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < true, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= false, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= false, PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(false) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(false) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(false) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(true) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(true) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - decimal") { + withOrcDataFrame((1 to 4).map(i => Tuple1.apply(BigDecimal.valueOf(i)))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === BigDecimal.valueOf(1), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> BigDecimal.valueOf(1), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < BigDecimal.valueOf(2), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > BigDecimal.valueOf(3), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= BigDecimal.valueOf(1), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= BigDecimal.valueOf(4), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(2)) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate( + Literal(BigDecimal.valueOf(3)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(1)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate( + Literal(BigDecimal.valueOf(4)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - timestamp") { + val timeString = "2015-08-20 14:57:00" + val timestamps = (1 to 4).map { i => + val milliseconds = Timestamp.valueOf(timeString).getTime + i * 3600 + new Timestamp(milliseconds) + } + withOrcDataFrame(timestamps.map(Tuple1(_))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === timestamps(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> timestamps(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < timestamps(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > timestamps(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= timestamps(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= timestamps(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(timestamps(0)) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(timestamps(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(timestamps(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(timestamps(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(timestamps(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("filter pushdown - combinations with logical operators") { + withOrcDataFrame((1 to 4).map(i => Tuple1(Option(i)))) { implicit df => + checkFilterPredicate( + '_1.isNotNull, + "leaf-0 = (IS_NULL _1), expr = (not leaf-0)" + ) + checkFilterPredicate( + '_1 =!= 1, + "leaf-0 = (IS_NULL _1), leaf-1 = (EQUALS _1 1), expr = (and (not leaf-0) (not leaf-1))" + ) + checkFilterPredicate( + !('_1 < 4), + "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 4), expr = (and (not leaf-0) (not leaf-1))" + ) + checkFilterPredicate( + '_1 < 2 || '_1 > 3, + "leaf-0 = (LESS_THAN _1 2), leaf-1 = (LESS_THAN_EQUALS _1 3), " + + "expr = (or leaf-0 (not leaf-1))" + ) + checkFilterPredicate( + '_1 < 2 && '_1 > 3, + "leaf-0 = (IS_NULL _1), leaf-1 = (LESS_THAN _1 2), leaf-2 = (LESS_THAN_EQUALS _1 3), " + + "expr = (and (not leaf-0) leaf-1 (not leaf-2))" + ) + } + } + + test("filter pushdown - date") { + val dates = Seq("2017-08-18", "2017-08-19", "2017-08-20", "2017-08-21").map { day => + Date.valueOf(day) + } + withOrcDataFrame(dates.map(Tuple1(_))) { implicit df => + checkFilterPredicate('_1.isNull, PredicateLeaf.Operator.IS_NULL) + + checkFilterPredicate('_1 === dates(0), PredicateLeaf.Operator.EQUALS) + checkFilterPredicate('_1 <=> dates(0), PredicateLeaf.Operator.NULL_SAFE_EQUALS) + + checkFilterPredicate('_1 < dates(1), PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate('_1 > dates(2), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 <= dates(0), PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate('_1 >= dates(3), PredicateLeaf.Operator.LESS_THAN) + + checkFilterPredicate(Literal(dates(0)) === '_1, PredicateLeaf.Operator.EQUALS) + checkFilterPredicate(Literal(dates(0)) <=> '_1, PredicateLeaf.Operator.NULL_SAFE_EQUALS) + checkFilterPredicate(Literal(dates(1)) > '_1, PredicateLeaf.Operator.LESS_THAN) + checkFilterPredicate(Literal(dates(2)) < '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(0)) >= '_1, PredicateLeaf.Operator.LESS_THAN_EQUALS) + checkFilterPredicate(Literal(dates(3)) <= '_1, PredicateLeaf.Operator.LESS_THAN) + } + } + + test("no filter pushdown - non-supported types") { + implicit class IntToBinary(int: Int) { + def b: Array[Byte] = int.toString.getBytes(StandardCharsets.UTF_8) + } + // ArrayType + withOrcDataFrame((1 to 4).map(i => Tuple1(Array(i)))) { implicit df => + checkNoFilterPredicate('_1.isNull, noneSupported = true) + } + // BinaryType + withOrcDataFrame((1 to 4).map(i => Tuple1(i.b))) { implicit df => + checkNoFilterPredicate('_1 <=> 1.b, noneSupported = true) + } + // MapType + withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => + checkNoFilterPredicate('_1.isNotNull, noneSupported = true) + } + } + + test("SPARK-12218 and SPARK-25699 Converting conjunctions into ORC SearchArguments") { + import org.apache.spark.sql.sources._ + // The `LessThan` should be converted while the `StringContains` shouldn't + val schema = new StructType( + Array( + StructField("a", IntegerType, nullable = true), + StructField("b", StringType, nullable = true))) + assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + LessThan("a", 10), + StringContains("b", "prefix") + )).get.toString + } + + // The `LessThan` should be converted while the whole inner `And` shouldn't + assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + LessThan("a", 10), + Not(And( + GreaterThan("a", 1), + StringContains("b", "prefix") + )) + )).get.toString + } + + // Safely remove unsupported `StringContains` predicate and push down `LessThan` + assertResult("leaf-0 = (LESS_THAN a 10), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ) + )).get.toString + } + + // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. + assertResult("leaf-0 = (LESS_THAN a 10), leaf-1 = (LESS_THAN_EQUALS a 1)," + + " expr = (and leaf-0 (not leaf-1))") { + OrcFilters.createFilter(schema, Array( + And( + And( + LessThan("a", 10), + StringContains("b", "prefix") + ), + GreaterThan("a", 1) + ) + )).get.toString + } + } + + test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { + import org.apache.spark.sql.sources._ + // The `LessThan` should be converted while the `StringContains` shouldn't + val schema = new StructType( + Array( + StructField("a", IntegerType, nullable = true), + StructField("b", StringType, nullable = true))) + + // The predicate `StringContains` predicate is not able to be pushed down. + assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + + " expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + GreaterThan("a", 10), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assertResult("leaf-0 = (LESS_THAN_EQUALS a 10), leaf-1 = (LESS_THAN a 1)," + + " expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + And( + GreaterThan("a", 10), + StringContains("b", "foobar") + ), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assert(OrcFilters.createFilter(schema, Array( + Or( + StringContains("b", "foobar"), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).isEmpty) + } + + test("SPARK-27160: Fix casting of the DecimalType literal") { + import org.apache.spark.sql.sources._ + val schema = StructType(Array(StructField("a", DecimalType(3, 2)))) + assertResult("leaf-0 = (LESS_THAN a 3.14), expr = leaf-0") { + OrcFilters.createFilter(schema, Array( + LessThan( + "a", + new java.math.BigDecimal(3.14, MathContext.DECIMAL64).setScale(2))) + ).get.toString + } + } +} + diff --git a/sql/gen-sql-markdown.py b/sql/gen-sql-markdown.py index fa8124b4513ac..e0529f8310613 100644 --- a/sql/gen-sql-markdown.py +++ b/sql/gen-sql-markdown.py @@ -20,7 +20,7 @@ from collections import namedtuple ExpressionInfo = namedtuple( - "ExpressionInfo", "className name usage arguments examples note since") + "ExpressionInfo", "className name usage arguments examples note since deprecated") def _list_function_infos(jvm): @@ -42,7 +42,8 @@ def _list_function_infos(jvm): arguments=jinfo.getArguments().replace("_FUNC_", name), examples=jinfo.getExamples().replace("_FUNC_", name), note=jinfo.getNote(), - since=jinfo.getSince())) + since=jinfo.getSince(), + deprecated=jinfo.getDeprecated())) return sorted(infos, key=lambda i: i.name) @@ -136,6 +137,27 @@ def _make_pretty_note(note): return "**Note:**\n%s\n" % note +def _make_pretty_deprecated(deprecated): + """ + Makes the deprecated description pretty and returns a formatted string if `deprecated` + is not an empty string. Otherwise, returns None. + + Expected input: + + ... + + Expected output: + **Deprecated:** + + ... + + """ + + if deprecated != "": + deprecated = "\n".join(map(lambda n: n[4:], deprecated.split("\n"))) + return "**Deprecated:**\n%s\n" % deprecated + + def generate_sql_markdown(jvm, path): """ Generates a markdown file after listing the function information. The output file @@ -162,6 +184,10 @@ def generate_sql_markdown(jvm, path): **Since:** SINCE + **Deprecated:** + + DEPRECATED +
    """ @@ -174,6 +200,7 @@ def generate_sql_markdown(jvm, path): examples = _make_pretty_examples(info.examples) note = _make_pretty_note(info.note) since = info.since + deprecated = _make_pretty_deprecated(info.deprecated) mdfile.write("### %s\n\n" % name) if usage is not None: @@ -186,6 +213,8 @@ def generate_sql_markdown(jvm, path): mdfile.write(note) if since is not None and since != "": mdfile.write("**Since:** %s\n\n" % since.strip()) + if deprecated is not None: + mdfile.write(deprecated) mdfile.write("
    \n\n") diff --git a/sql/hive-thriftserver/pom.xml b/sql/hive-thriftserver/pom.xml index 55e051c3ed1be..1abc65ad806bb 100644 --- a/sql/hive-thriftserver/pom.xml +++ b/sql/hive-thriftserver/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-hive-thriftserver_2.11 + spark-hive-thriftserver_2.12 jar Spark Project Hive Thrift Server http://spark.apache.org/ @@ -47,6 +47,13 @@ test-jar test + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + test-jar + test + com.google.guava guava @@ -63,6 +70,15 @@ ${hive.group} hive-beeline + + + ${hive.group} + hive-contrib + + + ${hive.group}.hcatalog + hive-hcatalog-core + org.eclipse.jetty jetty-server @@ -120,18 +136,6 @@ org.codehaus.mojo build-helper-maven-plugin - - add-default-sources - generate-sources - - add-source - - - - v${hive.version.short}/src/main/scala - - - add-source generate-sources @@ -140,7 +144,9 @@ - src/gen/ + v${hive.version.short}/src/gen/java + v${hive.version.short}/src/main/java + v${hive.version.short}/src/main/scala diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java index 5a508745414a7..15551da4785f6 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java +++ b/sql/hive-thriftserver/src/main/java/org/apache/hive/service/FilterService.java @@ -71,7 +71,7 @@ public HiveConf getHiveConf() { } @Override - public STATE getServiceState() { + public Service.STATE getServiceState() { return service.getServiceState(); } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 7442c987efc72..b4d1d0d58aad6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -31,6 +31,7 @@ import org.apache.hive.service.server.HiveServer2 import org.apache.spark.SparkContext import org.apache.spark.annotation.DeveloperApi import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils @@ -63,7 +64,7 @@ object HiveThriftServer2 extends Logging { server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) sqlContext.sparkContext.addSparkListener(listener) - uiTab = if (sqlContext.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + uiTab = if (sqlContext.sparkContext.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(sqlContext.sparkContext)) } else { None @@ -71,6 +72,13 @@ object HiveThriftServer2 extends Logging { } def main(args: Array[String]) { + // If the arguments contains "-h" or "--help", print out the usage and exit. + if (args.contains("-h") || args.contains("--help")) { + HiveServer2.main(args) + // The following code should not be reachable. It is added to ensure the main function exits. + return + } + Utils.initDaemon(log) val optionsProcessor = new HiveServer2.ServerOptionsProcessor("HiveThriftServer2") optionsProcessor.parse(args) @@ -94,7 +102,7 @@ object HiveThriftServer2 extends Logging { logInfo("HiveThriftServer2 started") listener = new HiveThriftServer2Listener(server, SparkSQLEnv.sqlContext.conf) SparkSQLEnv.sparkContext.addSparkListener(listener) - uiTab = if (SparkSQLEnv.sparkContext.getConf.getBoolean("spark.ui.enabled", true)) { + uiTab = if (SparkSQLEnv.sparkContext.getConf.get(UI_ENABLED)) { Some(new ThriftServerTab(SparkSQLEnv.sparkContext)) } else { None @@ -129,7 +137,7 @@ object HiveThriftServer2 extends Logging { } private[thriftserver] object ExecutionState extends Enumeration { - val STARTED, COMPILED, FAILED, FINISHED = Value + val STARTED, COMPILED, FAILED, FINISHED, CLOSED = Value type ExecutionState = Value } @@ -139,16 +147,17 @@ object HiveThriftServer2 extends Logging { val startTimestamp: Long, val userName: String) { var finishTimestamp: Long = 0L + var closeTimestamp: Long = 0L var executePlan: String = "" var detail: String = "" var state: ExecutionState.Value = ExecutionState.STARTED val jobId: ArrayBuffer[String] = ArrayBuffer[String]() var groupId: String = "" - def totalTime: Long = { - if (finishTimestamp == 0L) { + def totalTime(endTime: Long): Long = { + if (endTime == 0L) { System.currentTimeMillis - startTimestamp } else { - finishTimestamp - startTimestamp + endTime - startTimestamp } } } @@ -246,6 +255,11 @@ object HiveThriftServer2 extends Logging { trimExecutionIfNecessary() } + def onOperationClosed(id: String): Unit = synchronized { + executionList(id).closeTimestamp = System.currentTimeMillis + executionList(id).state = ExecutionState.CLOSED + } + private def trimExecutionIfNecessary() = { if (executionList.size > retainedStatements) { val toRemove = math.max(retainedStatements / 10, 1) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index 3cfc81b8a9579..2f011c25fe2ce 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -32,10 +32,11 @@ import org.apache.hive.service.cli._ import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession +import org.apache.spark.SparkContext import org.apache.spark.internal.Logging import org.apache.spark.sql.{DataFrame, Row => SparkRow, SQLContext} +import org.apache.spark.sql.execution.HiveResult import org.apache.spark.sql.execution.command.SetCommand -import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ import org.apache.spark.util.{Utils => SparkUtils} @@ -69,11 +70,12 @@ private[hive] class SparkExecuteStatementOperation( } } - def close(): Unit = { + override def close(): Unit = { // RDDs will be cleaned automatically upon garbage collection. logDebug(s"CLOSING $statementId") cleanup(OperationState.CLOSED) sqlContext.sparkContext.clearJobGroup() + HiveThriftServer2.listener.onOperationClosed(statementId) } def addNonNullColumnValue(from: SparkRow, to: ArrayBuffer[Any], ordinal: Int) { @@ -103,16 +105,17 @@ private[hive] class SparkExecuteStatementOperation( case BinaryType => to += from.getAs[Array[Byte]](ordinal) case _: ArrayType | _: StructType | _: MapType | _: UserDefinedType[_] => - val hiveString = HiveUtils.toHiveString((from.get(ordinal), dataTypes(ordinal))) + val hiveString = HiveResult.toHiveString((from.get(ordinal), dataTypes(ordinal))) to += hiveString } } - def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = { + def getNextRowSet(order: FetchOrientation, maxRowsL: Long): RowSet = withSchedulerPool { validateDefaultFetchOrientation(order) assertState(OperationState.FINISHED) setHasResultSet(true) - val resultRowSet: RowSet = RowSetFactory.create(getResultSetSchema, getProtocolVersion) + val resultRowSet: RowSet = + ThriftserverShimUtils.resultRowSet(getResultSetSchema, getProtocolVersion) // Reset iter to header when fetching start from first row if (order.equals(FetchOrientation.FETCH_FIRST)) { @@ -204,12 +207,12 @@ private[hive] class SparkExecuteStatementOperation( case NonFatal(e) => logError(s"Error executing query in background", e) setState(OperationState.ERROR) - throw e + throw new HiveSQLException(e) } } } - private def execute(): Unit = { + private def execute(): Unit = withSchedulerPool { statementId = UUID.randomUUID().toString logInfo(s"Running query '$statement' with $statementId") setState(OperationState.RUNNING) @@ -224,17 +227,14 @@ private[hive] class SparkExecuteStatementOperation( statementId, parentSession.getUsername) sqlContext.sparkContext.setJobGroup(statementId, statement) - val pool = sessionToActivePool.get(parentSession.getSessionHandle) - if (pool != null) { - sqlContext.sparkContext.setLocalProperty("spark.scheduler.pool", pool) - } try { result = sqlContext.sql(statement) logDebug(result.queryExecution.toString()) result.queryExecution.logical match { case SetCommand(Some((SQLConf.THRIFTSERVER_POOL.key, Some(value)))) => sessionToActivePool.put(parentSession.getSessionHandle, value) - logInfo(s"Setting spark.scheduler.pool=$value for future statements in this session.") + logInfo(s"Setting ${SparkContext.SPARK_SCHEDULER_POOL}=$value for future statements " + + "in this session.") case _ => } HiveThriftServer2.listener.onStatementParsed(statementId, result.queryExecution.toString()) @@ -289,6 +289,20 @@ private[hive] class SparkExecuteStatementOperation( sqlContext.sparkContext.cancelJobGroup(statementId) } } + + private def withSchedulerPool[T](body: => T): T = { + val pool = sessionToActivePool.get(parentSession.getSessionHandle) + if (pool != null) { + sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, pool) + } + try { + body + } finally { + if (pool != null) { + sqlContext.sparkContext.setLocalProperty(SparkContext.SPARK_SCHEDULER_POOL, null) + } + } + } } object SparkExecuteStatementOperation { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala new file mode 100644 index 0000000000000..89faff2f6f913 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetColumnsOperation.scala @@ -0,0 +1,186 @@ +/* + * 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.thriftserver + +import java.util.UUID +import java.util.regex.Pattern + +import scala.collection.JavaConverters.seqAsJavaListConverter + +import org.apache.hadoop.hive.ql.security.authorization.plugin.{HiveOperationType, HivePrivilegeObject} +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.GetColumnsOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.TableIdentifier +import org.apache.spark.sql.catalyst.catalog.SessionCatalog +import org.apache.spark.sql.hive.thriftserver.ThriftserverShimUtils.toJavaSQLType +import org.apache.spark.sql.types.StructType +import org.apache.spark.util.{Utils => SparkUtils} + +/** + * Spark's own SparkGetColumnsOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + * @param catalogName catalog name. NULL if not applicable. + * @param schemaName database name, NULL or a concrete database name + * @param tableName table name + * @param columnName column name + */ +private[hive] class SparkGetColumnsOperation( + sqlContext: SQLContext, + parentSession: HiveSession, + catalogName: String, + schemaName: String, + tableName: String, + columnName: String) + extends GetColumnsOperation(parentSession, catalogName, schemaName, tableName, columnName) + with Logging { + + val catalog: SessionCatalog = sqlContext.sessionState.catalog + + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + + override def runInternal(): Unit = { + statementId = UUID.randomUUID().toString + // Do not change cmdStr. It's used for Hive auditing and authorization. + val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName, tablePattern : $tableName" + val logMsg = s"Listing columns '$cmdStr, columnName : $columnName'" + logInfo(s"$logMsg with $statementId") + + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + logMsg, + statementId, + parentSession.getUsername) + + val schemaPattern = convertSchemaPattern(schemaName) + val tablePattern = convertIdentifierPattern(tableName, true) + + var columnPattern: Pattern = null + if (columnName != null) { + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)) + } + + val db2Tabs = catalog.listDatabases(schemaPattern).map { dbName => + (dbName, catalog.listTables(dbName, tablePattern, includeLocalTempViews = false)) + }.toMap + + if (isAuthV2Enabled) { + val privObjs = seqAsJavaListConverter(getPrivObjs(db2Tabs)).asJava + authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr) + } + + try { + // Tables and views + db2Tabs.foreach { + case (dbName, tables) => + catalog.getTablesByName(tables).foreach { catalogTable => + addToRowSet(columnPattern, dbName, catalogTable.identifier.table, catalogTable.schema) + } + } + + // Global temporary views + val globalTempViewDb = catalog.globalTempViewManager.database + val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) + if (databasePattern.matcher(globalTempViewDb).matches()) { + catalog.globalTempViewManager.listViewNames(tablePattern).foreach { globalTempView => + catalog.globalTempViewManager.get(globalTempView).foreach { plan => + addToRowSet(columnPattern, globalTempViewDb, globalTempView, plan.schema) + } + } + } + + // Temporary views + catalog.listLocalTempViews(tablePattern).foreach { localTempView => + catalog.getTempView(localTempView.table).foreach { plan => + addToRowSet(columnPattern, null, localTempView.table, plan.schema) + } + } + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) + throw e + } + HiveThriftServer2.listener.onStatementFinish(statementId) + } + + private def addToRowSet( + columnPattern: Pattern, + dbName: String, + tableName: String, + schema: StructType): Unit = { + schema.foreach { column => + if (columnPattern != null && !columnPattern.matcher(column.name).matches()) { + } else { + val rowData = Array[AnyRef]( + null, // TABLE_CAT + dbName, // TABLE_SCHEM + tableName, // TABLE_NAME + column.name, // COLUMN_NAME + toJavaSQLType(column.dataType.sql).asInstanceOf[AnyRef], // DATA_TYPE + column.dataType.sql, // TYPE_NAME + null, // COLUMN_SIZE + null, // BUFFER_LENGTH, unused + null, // DECIMAL_DIGITS + null, // NUM_PREC_RADIX + (if (column.nullable) 1 else 0).asInstanceOf[AnyRef], // NULLABLE + column.getComment().getOrElse(""), // REMARKS + null, // COLUMN_DEF + null, // SQL_DATA_TYPE + null, // SQL_DATETIME_SUB + null, // CHAR_OCTET_LENGTH + null, // ORDINAL_POSITION + "YES", // IS_NULLABLE + null, // SCOPE_CATALOG + null, // SCOPE_SCHEMA + null, // SCOPE_TABLE + null, // SOURCE_DATA_TYPE + "NO" // IS_AUTO_INCREMENT + ) + rowSet.addRow(rowData) + } + } + } + + private def getPrivObjs(db2Tabs: Map[String, Seq[TableIdentifier]]): Seq[HivePrivilegeObject] = { + db2Tabs.foldLeft(Seq.empty[HivePrivilegeObject])({ + case (i, (dbName, tables)) => i ++ tables.map { tableId => + new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbName, tableId.table) + } + }) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala new file mode 100644 index 0000000000000..87ef154bcc8ab --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetSchemasOperation.scala @@ -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.sql.hive.thriftserver + +import java.util.UUID +import java.util.regex.Pattern + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.GetSchemasOperation +import org.apache.hive.service.cli.operation.MetadataOperation.DEFAULT_HIVE_CATALOG +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.util.{Utils => SparkUtils} + +/** + * Spark's own GetSchemasOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + * @param catalogName catalog name. null if not applicable. + * @param schemaName database name, null or a concrete database name + */ +private[hive] class SparkGetSchemasOperation( + sqlContext: SQLContext, + parentSession: HiveSession, + catalogName: String, + schemaName: String) + extends GetSchemasOperation(parentSession, catalogName, schemaName) with Logging { + + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + + override def runInternal(): Unit = { + statementId = UUID.randomUUID().toString + // Do not change cmdStr. It's used for Hive auditing and authorization. + val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" + val logMsg = s"Listing databases '$cmdStr'" + logInfo(s"$logMsg with $statementId") + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + if (isAuthV2Enabled) { + authorizeMetaGets(HiveOperationType.GET_TABLES, null, cmdStr) + } + + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + logMsg, + statementId, + parentSession.getUsername) + + try { + val schemaPattern = convertSchemaPattern(schemaName) + sqlContext.sessionState.catalog.listDatabases(schemaPattern).foreach { dbName => + rowSet.addRow(Array[AnyRef](dbName, DEFAULT_HIVE_CATALOG)) + } + + val globalTempViewDb = sqlContext.sessionState.catalog.globalTempViewManager.database + val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) + if (databasePattern.matcher(globalTempViewDb).matches()) { + rowSet.addRow(Array[AnyRef](globalTempViewDb, DEFAULT_HIVE_CATALOG)) + } + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) + throw e + } + HiveThriftServer2.listener.onStatementFinish(statementId) + } +} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala new file mode 100644 index 0000000000000..952de42083c42 --- /dev/null +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkGetTablesOperation.scala @@ -0,0 +1,156 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import java.util.{List => JList, UUID} +import java.util.regex.Pattern + +import scala.collection.JavaConverters._ + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils +import org.apache.hive.service.cli._ +import org.apache.hive.service.cli.operation.GetTablesOperation +import org.apache.hive.service.cli.session.HiveSession + +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.catalyst.catalog.CatalogTableType._ +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.util.{Utils => SparkUtils} + +/** + * Spark's own GetTablesOperation + * + * @param sqlContext SQLContext to use + * @param parentSession a HiveSession from SessionManager + * @param catalogName catalog name. null if not applicable + * @param schemaName database name, null or a concrete database name + * @param tableName table name pattern + * @param tableTypes list of allowed table types, e.g. "TABLE", "VIEW" + */ +private[hive] class SparkGetTablesOperation( + sqlContext: SQLContext, + parentSession: HiveSession, + catalogName: String, + schemaName: String, + tableName: String, + tableTypes: JList[String]) + extends GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes) + with Logging{ + + private var statementId: String = _ + + override def close(): Unit = { + super.close() + HiveThriftServer2.listener.onOperationClosed(statementId) + } + + override def runInternal(): Unit = { + statementId = UUID.randomUUID().toString + // Do not change cmdStr. It's used for Hive auditing and authorization. + val cmdStr = s"catalog : $catalogName, schemaPattern : $schemaName" + val tableTypesStr = if (tableTypes == null) "null" else tableTypes.asScala.mkString(",") + val logMsg = s"Listing tables '$cmdStr, tableTypes : $tableTypesStr, tableName : $tableName'" + logInfo(s"$logMsg with $statementId") + setState(OperationState.RUNNING) + // Always use the latest class loader provided by executionHive's state. + val executionHiveClassLoader = sqlContext.sharedState.jarClassLoader + Thread.currentThread().setContextClassLoader(executionHiveClassLoader) + + val catalog = sqlContext.sessionState.catalog + val schemaPattern = convertSchemaPattern(schemaName) + val tablePattern = convertIdentifierPattern(tableName, true) + val matchingDbs = catalog.listDatabases(schemaPattern) + + if (isAuthV2Enabled) { + val privObjs = + HivePrivilegeObjectUtils.getHivePrivDbObjects(seqAsJavaListConverter(matchingDbs).asJava) + authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr) + } + + HiveThriftServer2.listener.onStatementStart( + statementId, + parentSession.getSessionHandle.getSessionId.toString, + logMsg, + statementId, + parentSession.getUsername) + + try { + // Tables and views + matchingDbs.foreach { dbName => + val tables = catalog.listTables(dbName, tablePattern, includeLocalTempViews = false) + catalog.getTablesByName(tables).foreach { table => + val tableType = tableTypeString(table.tableType) + if (tableTypes == null || tableTypes.isEmpty || tableTypes.contains(tableType)) { + addToRowSet(table.database, table.identifier.table, tableType, table.comment) + } + } + } + + // Temporary views and global temporary views + if (tableTypes == null || tableTypes.isEmpty || tableTypes.contains(VIEW.name)) { + val globalTempViewDb = catalog.globalTempViewManager.database + val databasePattern = Pattern.compile(CLIServiceUtils.patternToRegex(schemaName)) + val tempViews = if (databasePattern.matcher(globalTempViewDb).matches()) { + catalog.listTables(globalTempViewDb, tablePattern, includeLocalTempViews = true) + } else { + catalog.listLocalTempViews(tablePattern) + } + tempViews.foreach { view => + addToRowSet(view.database.orNull, view.table, VIEW.name, None) + } + } + setState(OperationState.FINISHED) + } catch { + case e: HiveSQLException => + setState(OperationState.ERROR) + HiveThriftServer2.listener.onStatementError( + statementId, e.getMessage, SparkUtils.exceptionString(e)) + throw e + } + HiveThriftServer2.listener.onStatementFinish(statementId) + } + + private def addToRowSet( + dbName: String, + tableName: String, + tableType: String, + comment: Option[String]): Unit = { + val rowData = Array[AnyRef]( + "", + dbName, + tableName, + tableType, + comment.getOrElse("")) + // Since HIVE-7575(Hive 2.0.0), adds 5 additional columns to the ResultSet of GetTables. + if (HiveUtils.isHive23) { + rowSet.addRow(rowData ++ Array(null, null, null, null, null)) + } else { + rowSet.addRow(rowData) + } + } + + private def tableTypeString(tableType: CatalogTableType): String = tableType match { + case EXTERNAL | MANAGED => "TABLE" + case VIEW => "VIEW" + case t => + throw new IllegalArgumentException(s"Unknown table type is found at showCreateHiveTable: $t") + } +} 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 bb96cea2b0ae1..055822040ca43 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 @@ -19,16 +19,16 @@ package org.apache.spark.sql.hive.thriftserver import java.io._ import java.util.{ArrayList => JArrayList, Locale} +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import jline.console.ConsoleReader import jline.console.history.FileHistory import org.apache.commons.lang3.StringUtils -import org.apache.commons.logging.LogFactory import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor} -import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils} +import org.apache.hadoop.hive.common.HiveInterruptUtils import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.ql.Driver import org.apache.hadoop.hive.ql.exec.Utilities @@ -40,10 +40,10 @@ import org.apache.thrift.transport.TSocket import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.spark.deploy.security.HiveDelegationTokenProvider import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.security.HiveDelegationTokenProvider import org.apache.spark.util.ShutdownHookManager /** @@ -64,16 +64,14 @@ private[hive] object SparkSQLCLIDriver extends Logging { * a command is being processed by the current thread. */ def installSignalHandler() { - HiveInterruptUtils.add(new HiveInterruptCallback { - override def interrupt() { - // Handle remote execution mode - if (SparkSQLEnv.sparkContext != null) { - SparkSQLEnv.sparkContext.cancelAllJobs() - } else { - if (transport != null) { - // Force closing of TCP connection upon session termination - transport.getSocket.close() - } + HiveInterruptUtils.add(() => { + // Handle remote execution mode + if (SparkSQLEnv.sparkContext != null) { + SparkSQLEnv.sparkContext.cancelAllJobs() + } else { + if (transport != null) { + // Force closing of TCP connection upon session termination + transport.getSocket.close() } } }) @@ -207,7 +205,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { reader.setBellEnabled(false) reader.setExpandEvents(false) // reader.setDebug(new PrintWriter(new FileWriter("writer.debug", true))) - CliDriver.getCommandCompleter.foreach((e) => reader.addCompleter(e)) + CliDriver.getCommandCompleter.foreach(reader.addCompleter) val historyDirectory = System.getProperty("user.home") @@ -298,9 +296,7 @@ private[hive] object SparkSQLCLIDriver extends Logging { private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { private val sessionState = SessionState.get().asInstanceOf[CliSessionState] - private val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) - - private val console = new SessionState.LogHelper(LOG) + private val console = ThriftserverShimUtils.getConsole private val isRemoteMode = { SparkSQLCLIDriver.isRemoteMode(sessionState) @@ -343,10 +339,10 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { } if (tokens(0).toLowerCase(Locale.ROOT).equals("source") || cmd_trimmed.startsWith("!") || isRemoteMode) { - val start = System.currentTimeMillis() + val startTimeNs = System.nanoTime() super.processCmd(cmd) - val end = System.currentTimeMillis() - val timeTaken: Double = (end - start) / 1000.0 + val endTimeNs = System.nanoTime() + val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0 console.printInfo(s"Time taken: $timeTaken seconds") 0 } else { @@ -364,13 +360,13 @@ private[hive] class SparkSQLCLIDriver extends CliDriver with Logging { driver.init() val out = sessionState.out val err = sessionState.err - val start: Long = System.currentTimeMillis() + val startTimeNs: Long = System.nanoTime() if (sessionState.getIsVerbose) { out.println(cmd) } val rc = driver.run(cmd) - val end = System.currentTimeMillis() - val timeTaken: Double = (end - start) / 1000.0 + val endTimeNs = System.nanoTime() + val timeTaken: Double = TimeUnit.NANOSECONDS.toMillis(endTimeNs - startTimeNs) / 1000.0 ret = rc.getResponseCode if (ret != 0) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala index 1335e16e35882..c32d908ad1bba 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIService.scala @@ -33,8 +33,10 @@ import org.apache.hive.service.Service.STATE import org.apache.hive.service.auth.HiveAuthFactory import org.apache.hive.service.cli._ import org.apache.hive.service.server.HiveServer2 +import org.slf4j.Logger import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ private[hive] class SparkSQLCLIService(hiveServer: HiveServer2, sqlContext: SQLContext) @@ -112,6 +114,10 @@ private[thriftserver] trait ReflectedCompositeService { this: AbstractService => invoke(classOf[AbstractService], this, "ensureCurrentState", classOf[STATE] -> STATE.NOTINITED) setAncestorField(this, 3, "hiveConf", hiveConf) invoke(classOf[AbstractService], this, "changeState", classOf[STATE] -> STATE.INITED) - getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + if (HiveUtils.isHive23) { + getAncestorField[Logger](this, 3, "LOG").info(s"Service: $getName is inited.") + } else { + getAncestorField[Log](this, 3, "LOG").info(s"Service: $getName is inited.") + } } } diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala index 6775902173444..960fdd11db15d 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLDriver.scala @@ -29,6 +29,7 @@ import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse import org.apache.spark.internal.Logging import org.apache.spark.sql.{AnalysisException, SQLContext} import org.apache.spark.sql.execution.{QueryExecution, SQLExecution} +import org.apache.spark.sql.execution.HiveResult.hiveResultString private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlContext) @@ -61,7 +62,7 @@ private[hive] class SparkSQLDriver(val context: SQLContext = SparkSQLEnv.sqlCont context.sparkContext.setJobDescription(command) val execution = context.sessionState.executePlan(context.sql(command).logicalPlan) hiveResponse = SQLExecution.withNewExecutionId(context.sparkSession, execution) { - execution.hiveResultString() + hiveResultString(execution.executedPlan) } tableSchema = getResultSetSchema(execution) new CommandProcessorResponse(0) diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala index 2958b771f3648..13055e0ae1394 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLSessionManager.scala @@ -24,7 +24,6 @@ import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.SessionHandle import org.apache.hive.service.cli.session.SessionManager -import org.apache.hive.service.cli.thrift.TProtocolVersion import org.apache.hive.service.server.HiveServer2 import org.apache.spark.sql.SQLContext @@ -45,7 +44,7 @@ private[hive] class SparkSQLSessionManager(hiveServer: HiveServer2, sqlContext: } override def openSession( - protocol: TProtocolVersion, + protocol: ThriftserverShimUtils.TProtocolVersion, username: String, passwd: String, ipAddress: String, diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala index bf7c01f60fb5c..44b0908502e99 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/server/SparkSQLOperationManager.scala @@ -17,17 +17,17 @@ package org.apache.spark.sql.hive.thriftserver.server -import java.util.{Map => JMap} +import java.util.{List => JList, Map => JMap} import java.util.concurrent.ConcurrentHashMap import org.apache.hive.service.cli._ -import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, Operation, OperationManager} +import org.apache.hive.service.cli.operation.{ExecuteStatementOperation, GetColumnsOperation, GetSchemasOperation, MetadataOperation, Operation, OperationManager} import org.apache.hive.service.cli.session.HiveSession import org.apache.spark.internal.Logging import org.apache.spark.sql.SQLContext import org.apache.spark.sql.hive.HiveUtils -import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation} +import org.apache.spark.sql.hive.thriftserver.{ReflectionUtils, SparkExecuteStatementOperation, SparkGetColumnsOperation, SparkGetSchemasOperation, SparkGetTablesOperation} import org.apache.spark.sql.internal.SQLConf /** @@ -63,6 +63,51 @@ private[thriftserver] class SparkSQLOperationManager() operation } + override def newGetSchemasOperation( + parentSession: HiveSession, + catalogName: String, + schemaName: String): GetSchemasOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + " initialized or had already closed.") + val operation = new SparkGetSchemasOperation(sqlContext, parentSession, catalogName, schemaName) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetSchemasOperation with session=$parentSession.") + operation + } + + override def newGetTablesOperation( + parentSession: HiveSession, + catalogName: String, + schemaName: String, + tableName: String, + tableTypes: JList[String]): MetadataOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + " initialized or had already closed.") + val operation = new SparkGetTablesOperation(sqlContext, parentSession, + catalogName, schemaName, tableName, tableTypes) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetTablesOperation with session=$parentSession.") + operation + } + + override def newGetColumnsOperation( + parentSession: HiveSession, + catalogName: String, + schemaName: String, + tableName: String, + columnName: String): GetColumnsOperation = synchronized { + val sqlContext = sessionToContexts.get(parentSession.getSessionHandle) + require(sqlContext != null, s"Session handle: ${parentSession.getSessionHandle} has not been" + + s" initialized or had already closed.") + val operation = new SparkGetColumnsOperation(sqlContext, parentSession, + catalogName, schemaName, tableName, columnName) + handleToOperation.put(operation.getHandle, operation) + logDebug(s"Created GetColumnsOperation with session=$parentSession.") + operation + } + def setConfMap(conf: SQLConf, confMap: java.util.Map[String, String]): Unit = { val iterator = confMap.entrySet().iterator() while (iterator.hasNext) { diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 771104ceb8842..1747b5bafc934 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -50,7 +50,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" generateSessionStatsTable(request) ++ generateSQLStatsTable(request) } - UIUtils.headerSparkPage(request, "JDBC/ODBC Server", content, parent, Some(5000)) + UIUtils.headerSparkPage(request, "JDBC/ODBC Server", content, parent) } /** Generate basic stats of the thrift server program */ @@ -70,8 +70,8 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" private def generateSQLStatsTable(request: HttpServletRequest): Seq[Node] = { val numStatement = listener.getExecutionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = listener.getExecutionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -90,7 +90,9 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage("" {info.groupId} {formatDate(info.startTimestamp)} {if (info.finishTimestamp > 0) formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {if (info.closeTimestamp > 0) formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 163eb43aabc72..a45c6e363cbf5 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -39,8 +39,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) /** Render the page */ def render(request: HttpServletRequest): Seq[Node] = { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val parameterId = UIUtils.stripXSS(request.getParameter("id")) + val parameterId = request.getParameter("id") require(parameterId != null && parameterId.nonEmpty, "Missing id parameter") val content = @@ -58,7 +57,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) ++ generateSQLStatsTable(request, sessionStat.sessionId) } - UIUtils.headerSparkPage(request, "JDBC/ODBC Session", content, parent, Some(5000)) + UIUtils.headerSparkPage(request, "JDBC/ODBC Session", content, parent) } /** Generate basic stats of the thrift server program */ @@ -80,8 +79,8 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) .filter(_.sessionId == sessionID) val numStatement = executionList.size val table = if (numStatement > 0) { - val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Duration", - "Statement", "State", "Detail") + val headerRow = Seq("User", "JobID", "GroupID", "Start Time", "Finish Time", "Close Time", + "Execution Time", "Duration", "Statement", "State", "Detail") val dataRows = executionList.sortBy(_.startTimestamp).reverse def generateDataRow(info: ExecutionInfo): Seq[Node] = { @@ -100,7 +99,9 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) {info.groupId} {formatDate(info.startTimestamp)} {formatDate(info.finishTimestamp)} - {formatDurationOption(Some(info.totalTime))} + {formatDate(info.closeTimestamp)} + {formatDurationOption(Some(info.totalTime(info.finishTimestamp)))} + {formatDurationOption(Some(info.totalTime(info.closeTimestamp)))} {info.statement} {info.state} {errorMessageCell(detail)} 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 933fd7369380a..4ccc17cf070c8 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 @@ -31,6 +31,7 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.SparkFunSuite import org.apache.spark.internal.Logging +import org.apache.spark.sql.hive.test.HiveTestUtils import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -200,10 +201,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfterAll with Logging { } test("Commands using SerDe provided in --jars") { - val jarFile = - "../hive/src/test/resources/hive-hcatalog-core-0.13.1.jar" - .split("/") - .mkString(File.separator) + val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath val dataFilePath = Thread.currentThread().getContextClassLoader.getResource("data/files/small_kv.txt") diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala index 5f9ea4d26790b..035b71a37a692 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveCliSessionStateSuite.scala @@ -28,7 +28,7 @@ import org.apache.spark.sql.hive.HiveUtils class HiveCliSessionStateSuite extends SparkFunSuite { def withSessionClear(f: () => Unit): Unit = { - try f finally SessionState.detachSession() + try f() finally SessionState.detachSession() } test("CliSessionState will be reused") { diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala index 70eb28cdd0c64..dd18add53fde8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2Suites.scala @@ -21,7 +21,7 @@ import java.io.{File, FilenameFilter} import java.net.URL import java.nio.charset.StandardCharsets import java.sql.{Date, DriverManager, SQLException, Statement} -import java.util.UUID +import java.util.{Locale, UUID} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -35,7 +35,6 @@ import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.jdbc.HiveDriver import org.apache.hive.service.auth.PlainSaslHelper import org.apache.hive.service.cli.{FetchOrientation, FetchType, GetInfoType} -import org.apache.hive.service.cli.thrift.TCLIService.Client import org.apache.hive.service.cli.thrift.ThriftCLIServiceClient import org.apache.thrift.protocol.TBinaryProtocol import org.apache.thrift.transport.TSocket @@ -44,6 +43,8 @@ import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkException, SparkFunSuite} import org.apache.spark.internal.Logging import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.test.HiveTestUtils +import org.apache.spark.sql.internal.StaticSQLConf.HIVE_THRIFT_SERVER_SINGLESESSION import org.apache.spark.sql.test.ProcessTestUtils.ProcessOutputCapturer import org.apache.spark.util.{ThreadUtils, Utils} @@ -65,7 +66,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val user = System.getProperty("user.name") val transport = PlainSaslHelper.getPlainTransport(user, "anonymous", rawTransport) val protocol = new TBinaryProtocol(transport) - val client = new ThriftCLIServiceClient(new Client(protocol)) + val client = new ThriftCLIServiceClient(new ThriftserverShimUtils.Client(protocol)) transport.open() try f(client) finally transport.close() @@ -280,7 +281,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { var defaultV2: String = null var data: ArrayBuffer[Int] = null - withMultipleConnectionJdbcStatement("test_map")( + withMultipleConnectionJdbcStatement("test_map", "db1.test_map2")( // create table { statement => @@ -295,7 +296,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select * from test_table") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryTableScan")) + assert(plan.getString(1).contains("Scan In-memory table `test_table`")) val rs1 = statement.executeQuery("SELECT key FROM test_table ORDER BY KEY DESC") val buf1 = new collection.mutable.ArrayBuffer[Int]() @@ -381,7 +382,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val plan = statement.executeQuery("explain select key from test_map ORDER BY key DESC") plan.next() plan.next() - assert(plan.getString(1).contains("InMemoryTableScan")) + assert(plan.getString(1).contains("Scan In-memory table `test_table`")) val rs = statement.executeQuery("SELECT key FROM test_map ORDER BY KEY DESC") val buf = new collection.mutable.ArrayBuffer[Int]() @@ -484,10 +485,7 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { withMultipleConnectionJdbcStatement("smallKV", "addJar")( { statement => - val jarFile = - "../hive/src/test/resources/hive-hcatalog-core-0.13.1.jar" - .split("/") - .mkString(File.separator) + val jarFile = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath statement.executeQuery(s"ADD JAR $jarFile") }, @@ -538,7 +536,11 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { conf += resultSet.getString(1) -> resultSet.getString(2) } - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + if (HiveUtils.isHive23) { + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) + } else { + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) + } } } @@ -551,7 +553,11 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { conf += resultSet.getString(1) -> resultSet.getString(2) } - assert(conf.get("spark.sql.hive.version") === Some("1.2.1")) + if (HiveUtils.isHive23) { + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("2.3.5")) + } else { + assert(conf.get(HiveUtils.FAKE_HIVE_VERSION.key) === Some("1.2.1")) + } } } @@ -629,7 +635,11 @@ class HiveThriftBinaryServerSuite extends HiveThriftJdbcTest { val sessionHandle = client.openSession(user, "") val sessionID = sessionHandle.getSessionId - assert(pipeoutFileList(sessionID).length == 1) + if (HiveUtils.isHive23) { + assert(pipeoutFileList(sessionID).length == 2) + } else { + assert(pipeoutFileList(sessionID).length == 1) + } client.closeSession(sessionHandle) @@ -650,7 +660,7 @@ class SingleSessionSuite extends HiveThriftJdbcTest { override def mode: ServerMode.Value = ServerMode.binary override protected def extraConf: Seq[String] = - "--conf spark.sql.hive.thriftServer.singleSession=true" :: Nil + s"--conf ${HIVE_THRIFT_SERVER_SINGLESESSION.key}=true" :: Nil test("share the temporary functions across JDBC connections") { withMultipleConnectionJdbcStatement()( @@ -811,7 +821,28 @@ abstract class HiveThriftJdbcTest extends HiveThriftServer2Test { statements.zip(fs).foreach { case (s, f) => f(s) } } finally { tableNames.foreach { name => - statements(0).execute(s"DROP TABLE IF EXISTS $name") + // TODO: Need a better way to drop the view. + if (name.toUpperCase(Locale.ROOT).startsWith("VIEW")) { + statements(0).execute(s"DROP VIEW IF EXISTS $name") + } else { + statements(0).execute(s"DROP TABLE IF EXISTS $name") + } + } + statements.foreach(_.close()) + connections.foreach(_.close()) + } + } + + def withDatabase(dbNames: String*)(fs: (Statement => Unit)*) { + val user = System.getProperty("user.name") + val connections = fs.map { _ => DriverManager.getConnection(jdbcUri, user, "") } + val statements = connections.map(_.createStatement()) + + try { + statements.zip(fs).foreach { case (s, f) => f(s) } + } finally { + dbNames.foreach { name => + statements(0).execute(s"DROP DATABASE IF EXISTS $name") } statements.foreach(_.close()) connections.foreach(_.close()) diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala index 06e3980662048..13df3fabc4919 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperationSuite.scala @@ -27,8 +27,8 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite { val tableSchema = StructType(Seq(field1, field2)) val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors() assert(columns.size() == 2) - assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) - assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.NULL_TYPE) + assert(columns.get(0).getType().getName == "VOID") + assert(columns.get(1).getType().getName == "VOID") } test("SPARK-20146 Comment should be preserved") { @@ -37,9 +37,9 @@ class SparkExecuteStatementOperationSuite extends SparkFunSuite { val tableSchema = StructType(Seq(field1, field2)) val columns = SparkExecuteStatementOperation.getTableSchema(tableSchema).getColumnDescriptors() assert(columns.size() == 2) - assert(columns.get(0).getType() == org.apache.hive.service.cli.Type.STRING_TYPE) + assert(columns.get(0).getType().getName == "STRING") assert(columns.get(0).getComment() == "comment 1") - assert(columns.get(1).getType() == org.apache.hive.service.cli.Type.INT_TYPE) + assert(columns.get(1).getType().getName == "INT") assert(columns.get(1).getComment() == "") } } diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala new file mode 100644 index 0000000000000..e524861b320f6 --- /dev/null +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/SparkMetadataOperationSuite.scala @@ -0,0 +1,169 @@ +/* + * 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.thriftserver + +import java.sql.ResultSet + +class SparkMetadataOperationSuite extends HiveThriftJdbcTest { + + override def mode: ServerMode.Value = ServerMode.binary + + test("Spark's own GetSchemasOperation(SparkGetSchemasOperation)") { + def checkResult(rs: ResultSet, dbNames: Seq[String]): Unit = { + for (i <- dbNames.indices) { + assert(rs.next()) + assert(rs.getString("TABLE_SCHEM") === dbNames(i)) + } + // Make sure there are no more elements + assert(!rs.next()) + } + + withDatabase("db1", "db2") { statement => + Seq("CREATE DATABASE db1", "CREATE DATABASE db2").foreach(statement.execute) + + val metaData = statement.getConnection.getMetaData + + checkResult(metaData.getSchemas(null, "%"), Seq("db1", "db2", "default", "global_temp")) + checkResult(metaData.getSchemas(null, "db1"), Seq("db1")) + checkResult(metaData.getSchemas(null, "db_not_exist"), Seq.empty) + checkResult(metaData.getSchemas(null, "db*"), Seq("db1", "db2")) + } + } + + test("Spark's own GetTablesOperation(SparkGetTablesOperation)") { + def checkResult(rs: ResultSet, tableNames: Seq[String]): Unit = { + for (i <- tableNames.indices) { + assert(rs.next()) + assert(rs.getString("TABLE_NAME") === tableNames(i)) + } + // Make sure there are no more elements + assert(!rs.next()) + } + + withJdbcStatement("table1", "table2", "view1") { statement => + Seq( + "CREATE TABLE table1(key INT, val STRING)", + "CREATE TABLE table2(key INT, val STRING)", + "CREATE VIEW view1 AS SELECT * FROM table2", + "CREATE OR REPLACE GLOBAL TEMPORARY VIEW view_global_temp_1 AS SELECT 1 AS col1", + "CREATE OR REPLACE TEMPORARY VIEW view_temp_1 AS SELECT 1 as col1" + ).foreach(statement.execute) + + val metaData = statement.getConnection.getMetaData + + checkResult(metaData.getTables(null, "%", "%", null), + Seq("table1", "table2", "view1", "view_global_temp_1", "view_temp_1")) + + checkResult(metaData.getTables(null, "%", "table1", null), Seq("table1")) + + checkResult(metaData.getTables(null, "%", "table_not_exist", null), Seq.empty) + + checkResult(metaData.getTables(null, "%", "%", Array("TABLE")), + Seq("table1", "table2")) + + checkResult(metaData.getTables(null, "%", "%", Array("VIEW")), + Seq("view1", "view_global_temp_1", "view_temp_1")) + + checkResult(metaData.getTables(null, "%", "view_global_temp_1", null), + Seq("view_global_temp_1")) + + checkResult(metaData.getTables(null, "%", "view_temp_1", null), + Seq("view_temp_1")) + + checkResult(metaData.getTables(null, "%", "%", Array("TABLE", "VIEW")), + Seq("table1", "table2", "view1", "view_global_temp_1", "view_temp_1")) + + checkResult(metaData.getTables(null, "%", "table_not_exist", Array("TABLE", "VIEW")), + Seq.empty) + } + } + + test("Spark's own GetColumnsOperation(SparkGetColumnsOperation)") { + def checkResult( + rs: ResultSet, + columns: Seq[(String, String, String, String, String)]) : Unit = { + for (i <- columns.indices) { + assert(rs.next()) + val col = columns(i) + assert(rs.getString("TABLE_NAME") === col._1) + assert(rs.getString("COLUMN_NAME") === col._2) + assert(rs.getString("DATA_TYPE") === col._3) + assert(rs.getString("TYPE_NAME") === col._4) + assert(rs.getString("REMARKS") === col._5) + } + // Make sure there are no more elements + assert(!rs.next()) + } + + withJdbcStatement("table1", "table2", "view1") { statement => + Seq( + "CREATE TABLE table1(key INT comment 'Int column', val STRING comment 'String column')", + "CREATE TABLE table2(key INT, val DECIMAL comment 'Decimal column')", + "CREATE VIEW view1 AS SELECT key FROM table1", + "CREATE OR REPLACE GLOBAL TEMPORARY VIEW view_global_temp_1 AS SELECT 2 AS col2", + "CREATE OR REPLACE TEMPORARY VIEW view_temp_1 AS SELECT 2 as col2" + ).foreach(statement.execute) + + val metaData = statement.getConnection.getMetaData + + checkResult(metaData.getColumns(null, "%", "%", null), + Seq( + ("table1", "key", "4", "INT", "Int column"), + ("table1", "val", "12", "STRING", "String column"), + ("table2", "key", "4", "INT", ""), + ("table2", "val", "3", "DECIMAL(10,0)", "Decimal column"), + ("view1", "key", "4", "INT", "Int column"), + ("view_global_temp_1", "col2", "4", "INT", ""), + ("view_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "%", "table1", null), + Seq( + ("table1", "key", "4", "INT", "Int column"), + ("table1", "val", "12", "STRING", "String column"))) + + checkResult(metaData.getColumns(null, "%", "table1", "key"), + Seq(("table1", "key", "4", "INT", "Int column"))) + + checkResult(metaData.getColumns(null, "%", "view%", null), + Seq( + ("view1", "key", "4", "INT", "Int column"), + ("view_global_temp_1", "col2", "4", "INT", ""), + ("view_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "%", "view_global_temp_1", null), + Seq(("view_global_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "%", "view_temp_1", null), + Seq(("view_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "%", "view_temp_1", "col2"), + Seq(("view_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "default", "%", null), + Seq( + ("table1", "key", "4", "INT", "Int column"), + ("table1", "val", "12", "STRING", "String column"), + ("table2", "key", "4", "INT", ""), + ("table2", "val", "3", "DECIMAL(10,0)", "Decimal column"), + ("view1", "key", "4", "INT", "Int column"), + ("view_temp_1", "col2", "4", "INT", ""))) + + checkResult(metaData.getColumns(null, "%", "table_not_exist", null), Seq.empty) + } + } +} diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala index fef18f147b057..47cf4f104d204 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala @@ -86,12 +86,12 @@ class UISeleniumSuite queries.foreach(statement.execute) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { go to baseURL find(cssSelector("""ul li a[href*="sql"]""")) should not be None } - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (baseURL + "/sql") find(id("sessionstat")) should not be None find(id("sqlstat")) should not be None diff --git a/sql/hive-thriftserver/if/TCLIService.thrift b/sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift similarity index 100% rename from sql/hive-thriftserver/if/TCLIService.thrift rename to sql/hive-thriftserver/v1.2.1/if/TCLIService.thrift diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TArrayTypeEntry.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBinaryColumn.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolColumn.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TBoolValue.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteColumn.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TByteValue.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIService.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCLIServiceConstants.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelDelegationTokenResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCancelOperationResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseOperationResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TCloseSessionResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java similarity index 99% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java index bfe50c7810f73..fc2171dc99e4c 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java +++ b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumn.java @@ -148,7 +148,7 @@ public TColumn() { super(); } - public TColumn(_Fields setField, Object value) { + public TColumn(TColumn._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnDesc.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java similarity index 99% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java index 44da2cdd089d6..8504c6d608d42 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TColumnValue.java @@ -142,7 +142,7 @@ public TColumnValue() { super(); } - public TColumnValue(_Fields setField, Object value) { + public TColumnValue(TColumnValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleColumn.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TDoubleValue.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TExecuteStatementResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchOrientation.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TFetchResultsResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetCatalogsResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetColumnsResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetDelegationTokenResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetFunctionsResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoType.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java similarity index 99% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java index 4fe59b1c51462..fe2a211c46309 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetInfoValue.java @@ -136,7 +136,7 @@ public TGetInfoValue() { super(); } - public TGetInfoValue(_Fields setField, Object value) { + public TGetInfoValue(TGetInfoValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetOperationStatusResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetResultSetMetadataResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetSchemasResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTableTypesResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTablesResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TGetTypeInfoResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/THandleIdentifier.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Column.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI16Value.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Column.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI32Value.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Column.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TI64Value.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TMapTypeEntry.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOpenSessionResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationHandle.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationState.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TOperationType.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TPrimitiveTypeEntry.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TProtocolVersion.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenReq.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRenewDelegationTokenResp.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRow.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TRowSet.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TSessionHandle.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatus.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStatusCode.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringColumn.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStringValue.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TStructTypeEntry.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTableSchema.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeDesc.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java similarity index 99% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java index af7c0b4f15d95..d0d70c1279572 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java +++ b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeEntry.java @@ -136,7 +136,7 @@ public TTypeEntry() { super(); } - public TTypeEntry(_Fields setField, Object value) { + public TTypeEntry(TTypeEntry._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeId.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java similarity index 99% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java index 8c40687a0aab7..a3e3829372276 100644 --- a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java +++ b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifierValue.java @@ -112,7 +112,7 @@ public TTypeQualifierValue() { super(); } - public TTypeQualifierValue(_Fields setField, Object value) { + public TTypeQualifierValue(TTypeQualifierValue._Fields setField, Object value) { super(setField, value); } diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TTypeQualifiers.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUnionTypeEntry.java diff --git a/sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java similarity index 100% rename from sql/hive-thriftserver/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java rename to sql/hive-thriftserver/v1.2.1/src/gen/java/org/apache/hive/service/cli/thrift/TUserDefinedTypeEntry.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java similarity index 95% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java index 9dd0efc03968d..7e557aeccf5b0 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/AbstractService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/AbstractService.java @@ -36,7 +36,7 @@ public abstract class AbstractService implements Service { /** * Service state: initially {@link STATE#NOTINITED}. */ - private STATE state = STATE.NOTINITED; + private Service.STATE state = STATE.NOTINITED; /** * Service name. @@ -70,7 +70,7 @@ public AbstractService(String name) { } @Override - public synchronized STATE getServiceState() { + public synchronized Service.STATE getServiceState() { return state; } @@ -159,7 +159,7 @@ public long getStartTime() { * if the service state is different from * the desired state */ - private void ensureCurrentState(STATE currentState) { + private void ensureCurrentState(Service.STATE currentState) { ServiceOperations.ensureCurrentState(state, currentState); } @@ -173,7 +173,7 @@ private void ensureCurrentState(STATE currentState) { * @param newState * new service state */ - private void changeState(STATE newState) { + private void changeState(Service.STATE newState) { state = newState; // notify listeners for (ServiceStateChangeListener l : listeners) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/CompositeService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CompositeService.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/CookieSigner.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/CookieSigner.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceOperations.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceOperations.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/ServiceUtils.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/ServiceUtils.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/CLIService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/CLIService.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java similarity index 99% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java index adb269aa235ea..26d0f718f383a 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Column.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Column.java @@ -349,7 +349,7 @@ public void addValue(Type type, Object field) { break; case FLOAT_TYPE: nulls.set(size, field == null); - doubleVars()[size] = field == null ? 0 : new Double(field.toString()); + doubleVars()[size] = field == null ? 0 : Double.valueOf(field.toString()); break; case DOUBLE_TYPE: nulls.set(size, field == null); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ColumnValue.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ColumnValue.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/EmbeddedCLIServiceClient.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/FetchOrientation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/FetchOrientation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoType.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoType.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/GetInfoValue.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/GetInfoValue.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Handle.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Handle.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/HiveSQLException.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/HiveSQLException.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/ICLIService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/ICLIService.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationHandle.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationHandle.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationState.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationState.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/OperationType.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/OperationType.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/PatternOrIdentifier.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowBasedSet.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowBasedSet.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSet.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSet.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/RowSetFactory.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/RowSetFactory.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/SessionHandle.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/SessionHandle.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TableSchema.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TableSchema.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/Type.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/Type.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java similarity index 98% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java index 5efb0759383ac..96ba4890075ac 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -49,7 +49,7 @@ */ public class GetColumnsOperation extends MetadataOperation { - private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + protected static final TableSchema RESULT_SET_SCHEMA = new TableSchema() .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, "Catalog name. NULL if not applicable") .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, @@ -109,7 +109,7 @@ public class GetColumnsOperation extends MetadataOperation { private final String tableName; private final String columnName; - private final RowSet rowSet; + protected final RowSet rowSet; protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, String tableName, String columnName) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java similarity index 99% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java index d6f6280f1c398..3516bc2ba242c 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -41,7 +41,7 @@ public class GetSchemasOperation extends MetadataOperation { .addStringColumn("TABLE_SCHEM", "Schema name.") .addStringColumn("TABLE_CATALOG", "Catalog name."); - private RowSet rowSet; + protected RowSet rowSet; protected GetSchemasOperation(HiveSession parentSession, String catalogName, String schemaName) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java similarity index 99% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java index 1a7ca79163d71..2af17a662a296 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -46,7 +46,7 @@ public class GetTablesOperation extends MetadataOperation { private final String schemaName; private final String tableName; private final List tableTypes = new ArrayList(); - private final RowSet rowSet; + protected final RowSet rowSet; private final TableTypeMapping tableTypeMapping; diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/Operation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/Operation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSession.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSession.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/SessionManager.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/session/SessionManager.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java similarity index 97% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java index 6c9efba9e59a5..21b8bf7de75ce 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -94,7 +94,7 @@ public void run() { server = new TThreadPoolServer(sargs); server.setServerEventHandler(serverEventHandler); String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " - + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; + + serverSocket.getServerSocket().getLocalPort() + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); server.serve(); } catch (Throwable t) { diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java index ad7a9a238f8a9..8fce9d9383438 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -344,6 +344,7 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) String ipAddress = getIpAddress(); TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, req.getClient_protocol()); + res.setServerProtocolVersion(protocol); SessionHandle sessionHandle; if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && (userName != null)) { @@ -354,7 +355,6 @@ SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), ipAddress, req.getConfiguration()); } - res.setServerProtocolVersion(protocol); return sessionHandle; } diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java similarity index 96% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java index 341a7fdbb59b8..504e63dbc5e5e 100644 --- a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java +++ b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -19,7 +19,6 @@ package org.apache.hive.service.cli.thrift; import java.util.Arrays; -import java.util.concurrent.ExecutorService; import java.util.concurrent.SynchronousQueue; import java.util.concurrent.ThreadPoolExecutor; import java.util.concurrent.TimeUnit; @@ -65,7 +64,7 @@ public void run() { // Server thread pool // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests String threadPoolName = "HiveServer2-HttpHandler-Pool"; - ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + ThreadPoolExecutor executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), new ThreadFactoryWithGarbageCleanup(threadPoolName)); ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); @@ -87,7 +86,7 @@ public void run() { throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + " Not configured for SSL connection"); } - SslContextFactory sslContextFactory = new SslContextFactory(); + SslContextFactory sslContextFactory = new SslContextFactory.Server(); String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); sslContextFactory.addExcludeProtocols(excludedProtocols); @@ -145,7 +144,7 @@ public void run() { // Finally, start the server httpServer.start(); String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName - + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + + " mode on port " + connector.getLocalPort()+ " path=" + httpPath + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; LOG.info(msg); httpServer.join(); diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/HiveServer2.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/HiveServer2.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java rename to sql/hive-thriftserver/v1.2.1/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java diff --git a/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala new file mode 100644 index 0000000000000..b0702ade2cedc --- /dev/null +++ b/sql/hive-thriftserver/v1.2.1/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -0,0 +1,49 @@ +/* + * 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.thriftserver + +import org.apache.commons.logging.LogFactory +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema, Type} + +/** + * Various utilities for hive-thriftserver used to upgrade the built-in Hive. + */ +private[thriftserver] object ThriftserverShimUtils { + + private[thriftserver] type TProtocolVersion = org.apache.hive.service.cli.thrift.TProtocolVersion + private[thriftserver] type Client = org.apache.hive.service.cli.thrift.TCLIService.Client + private[thriftserver] type TOpenSessionReq = org.apache.hive.service.cli.thrift.TOpenSessionReq + private[thriftserver] type TGetSchemasReq = org.apache.hive.service.cli.thrift.TGetSchemasReq + private[thriftserver] type TGetTablesReq = org.apache.hive.service.cli.thrift.TGetTablesReq + private[thriftserver] type TGetColumnsReq = org.apache.hive.service.cli.thrift.TGetColumnsReq + + private[thriftserver] def getConsole: SessionState.LogHelper = { + val LOG = LogFactory.getLog(classOf[SparkSQLCLIDriver]) + new SessionState.LogHelper(LOG) + } + + private[thriftserver] def resultRowSet( + getResultSetSchema: TableSchema, + getProtocolVersion: TProtocolVersion): RowSet = { + RowSetFactory.create(getResultSetSchema, getProtocolVersion) + } + + private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + +} diff --git a/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift new file mode 100644 index 0000000000000..824b04919073a --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/if/TCLIService.thrift @@ -0,0 +1,1270 @@ +// 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. + +// Coding Conventions for this file: +// +// Structs/Enums/Unions +// * Struct, Enum, and Union names begin with a "T", +// and use a capital letter for each new word, with no underscores. +// * All fields should be declared as either optional or required. +// +// Functions +// * Function names start with a capital letter and have a capital letter for +// each new word, with no underscores. +// * Each function should take exactly one parameter, named TFunctionNameReq, +// and should return either void or TFunctionNameResp. This convention allows +// incremental updates. +// +// Services +// * Service names begin with the letter "T", use a capital letter for each +// new word (with no underscores), and end with the word "Service". + +namespace java org.apache.hive.service.rpc.thrift +namespace cpp apache.hive.service.rpc.thrift + +// List of protocol versions. A new token should be +// added to the end of this list every time a change is made. +enum TProtocolVersion { + HIVE_CLI_SERVICE_PROTOCOL_V1, + + // V2 adds support for asynchronous execution + HIVE_CLI_SERVICE_PROTOCOL_V2 + + // V3 add varchar type, primitive type qualifiers + HIVE_CLI_SERVICE_PROTOCOL_V3 + + // V4 add decimal precision/scale, char type + HIVE_CLI_SERVICE_PROTOCOL_V4 + + // V5 adds error details when GetOperationStatus returns in error state + HIVE_CLI_SERVICE_PROTOCOL_V5 + + // V6 uses binary type for binary payload (was string) and uses columnar result set + HIVE_CLI_SERVICE_PROTOCOL_V6 + + // V7 adds support for delegation token based connection + HIVE_CLI_SERVICE_PROTOCOL_V7 + + // V8 adds support for interval types + HIVE_CLI_SERVICE_PROTOCOL_V8 + + // V9 adds support for serializing ResultSets in SerDe + HIVE_CLI_SERVICE_PROTOCOL_V9 + + // V10 adds support for in place updates via GetOperationStatus + HIVE_CLI_SERVICE_PROTOCOL_V10 +} + +enum TTypeId { + BOOLEAN_TYPE, + TINYINT_TYPE, + SMALLINT_TYPE, + INT_TYPE, + BIGINT_TYPE, + FLOAT_TYPE, + DOUBLE_TYPE, + STRING_TYPE, + TIMESTAMP_TYPE, + BINARY_TYPE, + ARRAY_TYPE, + MAP_TYPE, + STRUCT_TYPE, + UNION_TYPE, + USER_DEFINED_TYPE, + DECIMAL_TYPE, + NULL_TYPE, + DATE_TYPE, + VARCHAR_TYPE, + CHAR_TYPE, + INTERVAL_YEAR_MONTH_TYPE, + INTERVAL_DAY_TIME_TYPE +} + +const set PRIMITIVE_TYPES = [ + TTypeId.BOOLEAN_TYPE, + TTypeId.TINYINT_TYPE, + TTypeId.SMALLINT_TYPE, + TTypeId.INT_TYPE, + TTypeId.BIGINT_TYPE, + TTypeId.FLOAT_TYPE, + TTypeId.DOUBLE_TYPE, + TTypeId.STRING_TYPE, + TTypeId.TIMESTAMP_TYPE, + TTypeId.BINARY_TYPE, + TTypeId.DECIMAL_TYPE, + TTypeId.NULL_TYPE, + TTypeId.DATE_TYPE, + TTypeId.VARCHAR_TYPE, + TTypeId.CHAR_TYPE, + TTypeId.INTERVAL_YEAR_MONTH_TYPE, + TTypeId.INTERVAL_DAY_TIME_TYPE +] + +const set COMPLEX_TYPES = [ + TTypeId.ARRAY_TYPE + TTypeId.MAP_TYPE + TTypeId.STRUCT_TYPE + TTypeId.UNION_TYPE + TTypeId.USER_DEFINED_TYPE +] + +const set COLLECTION_TYPES = [ + TTypeId.ARRAY_TYPE + TTypeId.MAP_TYPE +] + +const map TYPE_NAMES = { + TTypeId.BOOLEAN_TYPE: "BOOLEAN", + TTypeId.TINYINT_TYPE: "TINYINT", + TTypeId.SMALLINT_TYPE: "SMALLINT", + TTypeId.INT_TYPE: "INT", + TTypeId.BIGINT_TYPE: "BIGINT", + TTypeId.FLOAT_TYPE: "FLOAT", + TTypeId.DOUBLE_TYPE: "DOUBLE", + TTypeId.STRING_TYPE: "STRING", + TTypeId.TIMESTAMP_TYPE: "TIMESTAMP", + TTypeId.BINARY_TYPE: "BINARY", + TTypeId.ARRAY_TYPE: "ARRAY", + TTypeId.MAP_TYPE: "MAP", + TTypeId.STRUCT_TYPE: "STRUCT", + TTypeId.UNION_TYPE: "UNIONTYPE", + TTypeId.DECIMAL_TYPE: "DECIMAL", + TTypeId.NULL_TYPE: "NULL" + TTypeId.DATE_TYPE: "DATE" + TTypeId.VARCHAR_TYPE: "VARCHAR" + TTypeId.CHAR_TYPE: "CHAR" + TTypeId.INTERVAL_YEAR_MONTH_TYPE: "INTERVAL_YEAR_MONTH" + TTypeId.INTERVAL_DAY_TIME_TYPE: "INTERVAL_DAY_TIME" +} + +// Thrift does not support recursively defined types or forward declarations, +// which makes it difficult to represent Hive's nested types. +// To get around these limitations TTypeDesc employs a type list that maps +// integer "pointers" to TTypeEntry objects. The following examples show +// how different types are represented using this scheme: +// +// "INT": +// TTypeDesc { +// types = [ +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// } +// ] +// } +// +// "ARRAY": +// TTypeDesc { +// types = [ +// TTypeEntry.array_entry { +// object_type_ptr = 1 +// }, +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// } +// ] +// } +// +// "MAP": +// TTypeDesc { +// types = [ +// TTypeEntry.map_entry { +// key_type_ptr = 1 +// value_type_ptr = 2 +// }, +// TTypeEntry.primitive_entry { +// type = INT_TYPE +// }, +// TTypeEntry.primitive_entry { +// type = STRING_TYPE +// } +// ] +// } + +typedef i32 TTypeEntryPtr + +// Valid TTypeQualifiers key names +const string CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength" + +// Type qualifier key name for decimal +const string PRECISION = "precision" +const string SCALE = "scale" + +union TTypeQualifierValue { + 1: optional i32 i32Value + 2: optional string stringValue +} + +// Type qualifiers for primitive type. +struct TTypeQualifiers { + 1: required map qualifiers +} + +// Type entry for a primitive type. +struct TPrimitiveTypeEntry { + // The primitive type token. This must satisfy the condition + // that type is in the PRIMITIVE_TYPES set. + 1: required TTypeId type + 2: optional TTypeQualifiers typeQualifiers +} + +// Type entry for an ARRAY type. +struct TArrayTypeEntry { + 1: required TTypeEntryPtr objectTypePtr +} + +// Type entry for a MAP type. +struct TMapTypeEntry { + 1: required TTypeEntryPtr keyTypePtr + 2: required TTypeEntryPtr valueTypePtr +} + +// Type entry for a STRUCT type. +struct TStructTypeEntry { + 1: required map nameToTypePtr +} + +// Type entry for a UNIONTYPE type. +struct TUnionTypeEntry { + 1: required map nameToTypePtr +} + +struct TUserDefinedTypeEntry { + // The fully qualified name of the class implementing this type. + 1: required string typeClassName +} + +// We use a union here since Thrift does not support inheritance. +union TTypeEntry { + 1: TPrimitiveTypeEntry primitiveEntry + 2: TArrayTypeEntry arrayEntry + 3: TMapTypeEntry mapEntry + 4: TStructTypeEntry structEntry + 5: TUnionTypeEntry unionEntry + 6: TUserDefinedTypeEntry userDefinedTypeEntry +} + +// Type descriptor for columns. +struct TTypeDesc { + // The "top" type is always the first element of the list. + // If the top type is an ARRAY, MAP, STRUCT, or UNIONTYPE + // type, then subsequent elements represent nested types. + 1: required list types +} + +// A result set column descriptor. +struct TColumnDesc { + // The name of the column + 1: required string columnName + + // The type descriptor for this column + 2: required TTypeDesc typeDesc + + // The ordinal position of this column in the schema + 3: required i32 position + + 4: optional string comment +} + +// Metadata used to describe the schema (column names, types, comments) +// of result sets. +struct TTableSchema { + 1: required list columns +} + +// A Boolean column value. +struct TBoolValue { + // NULL if value is unset. + 1: optional bool value +} + +// A Byte column value. +struct TByteValue { + // NULL if value is unset. + 1: optional byte value +} + +// A signed, 16 bit column value. +struct TI16Value { + // NULL if value is unset + 1: optional i16 value +} + +// A signed, 32 bit column value +struct TI32Value { + // NULL if value is unset + 1: optional i32 value +} + +// A signed 64 bit column value +struct TI64Value { + // NULL if value is unset + 1: optional i64 value +} + +// A floating point 64 bit column value +struct TDoubleValue { + // NULL if value is unset + 1: optional double value +} + +struct TStringValue { + // NULL if value is unset + 1: optional string value +} + +// A single column value in a result set. +// Note that Hive's type system is richer than Thrift's, +// so in some cases we have to map multiple Hive types +// to the same Thrift type. On the client-side this is +// disambiguated by looking at the Schema of the +// result set. +union TColumnValue { + 1: TBoolValue boolVal // BOOLEAN + 2: TByteValue byteVal // TINYINT + 3: TI16Value i16Val // SMALLINT + 4: TI32Value i32Val // INT + 5: TI64Value i64Val // BIGINT, TIMESTAMP + 6: TDoubleValue doubleVal // FLOAT, DOUBLE + 7: TStringValue stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, BINARY, DECIMAL, NULL, INTERVAL_YEAR_MONTH, INTERVAL_DAY_TIME +} + +// Represents a row in a rowset. +struct TRow { + 1: required list colVals +} + +struct TBoolColumn { + 1: required list values + 2: required binary nulls +} + +struct TByteColumn { + 1: required list values + 2: required binary nulls +} + +struct TI16Column { + 1: required list values + 2: required binary nulls +} + +struct TI32Column { + 1: required list values + 2: required binary nulls +} + +struct TI64Column { + 1: required list values + 2: required binary nulls +} + +struct TDoubleColumn { + 1: required list values + 2: required binary nulls +} + +struct TStringColumn { + 1: required list values + 2: required binary nulls +} + +struct TBinaryColumn { + 1: required list values + 2: required binary nulls +} + +// Note that Hive's type system is richer than Thrift's, +// so in some cases we have to map multiple Hive types +// to the same Thrift type. On the client-side this is +// disambiguated by looking at the Schema of the +// result set. +union TColumn { + 1: TBoolColumn boolVal // BOOLEAN + 2: TByteColumn byteVal // TINYINT + 3: TI16Column i16Val // SMALLINT + 4: TI32Column i32Val // INT + 5: TI64Column i64Val // BIGINT, TIMESTAMP + 6: TDoubleColumn doubleVal // FLOAT, DOUBLE + 7: TStringColumn stringVal // STRING, LIST, MAP, STRUCT, UNIONTYPE, DECIMAL, NULL + 8: TBinaryColumn binaryVal // BINARY +} + +// Represents a rowset +struct TRowSet { + // The starting row offset of this rowset. + 1: required i64 startRowOffset + 2: required list rows + 3: optional list columns + 4: optional binary binaryColumns + 5: optional i32 columnCount +} + +// The return status code contained in each response. +enum TStatusCode { + SUCCESS_STATUS, + SUCCESS_WITH_INFO_STATUS, + STILL_EXECUTING_STATUS, + ERROR_STATUS, + INVALID_HANDLE_STATUS +} + +// The return status of a remote request +struct TStatus { + 1: required TStatusCode statusCode + + // If status is SUCCESS_WITH_INFO, info_msgs may be populated with + // additional diagnostic information. + 2: optional list infoMessages + + // If status is ERROR, then the following fields may be set + 3: optional string sqlState // as defined in the ISO/IEF CLI specification + 4: optional i32 errorCode // internal error code + 5: optional string errorMessage +} + +// The state of an operation (i.e. a query or other +// asynchronous operation that generates a result set) +// on the server. +enum TOperationState { + // The operation has been initialized + INITIALIZED_STATE, + + // The operation is running. In this state the result + // set is not available. + RUNNING_STATE, + + // The operation has completed. When an operation is in + // this state its result set may be fetched. + FINISHED_STATE, + + // The operation was canceled by a client + CANCELED_STATE, + + // The operation was closed by a client + CLOSED_STATE, + + // The operation failed due to an error + ERROR_STATE, + + // The operation is in an unrecognized state + UKNOWN_STATE, + + // The operation is in an pending state + PENDING_STATE, + + // The operation is in an timedout state + TIMEDOUT_STATE, +} + +// A string identifier. This is interpreted literally. +typedef string TIdentifier + +// A search pattern. +// +// Valid search pattern characters: +// '_': Any single character. +// '%': Any sequence of zero or more characters. +// '\': Escape character used to include special characters, +// e.g. '_', '%', '\'. If a '\' precedes a non-special +// character it has no special meaning and is interpreted +// literally. +typedef string TPattern + + +// A search pattern or identifier. Used as input +// parameter for many of the catalog functions. +typedef string TPatternOrIdentifier + +struct THandleIdentifier { + // 16 byte globally unique identifier + // This is the public ID of the handle and + // can be used for reporting. + 1: required binary guid, + + // 16 byte secret generated by the server + // and used to verify that the handle is not + // being hijacked by another user. + 2: required binary secret, +} + +// Client-side handle to persistent +// session information on the server-side. +struct TSessionHandle { + 1: required THandleIdentifier sessionId +} + +// The subtype of an OperationHandle. +enum TOperationType { + EXECUTE_STATEMENT, + GET_TYPE_INFO, + GET_CATALOGS, + GET_SCHEMAS, + GET_TABLES, + GET_TABLE_TYPES, + GET_COLUMNS, + GET_FUNCTIONS, + UNKNOWN, +} + +// Client-side reference to a task running +// asynchronously on the server. +struct TOperationHandle { + 1: required THandleIdentifier operationId + 2: required TOperationType operationType + + // If hasResultSet = TRUE, then this operation + // generates a result set that can be fetched. + // Note that the result set may be empty. + // + // If hasResultSet = FALSE, then this operation + // does not generate a result set, and calling + // GetResultSetMetadata or FetchResults against + // this OperationHandle will generate an error. + 3: required bool hasResultSet + + // For operations that don't generate result sets, + // modifiedRowCount is either: + // + // 1) The number of rows that were modified by + // the DML operation (e.g. number of rows inserted, + // number of rows deleted, etc). + // + // 2) 0 for operations that don't modify or add rows. + // + // 3) < 0 if the operation is capable of modifiying rows, + // but Hive is unable to determine how many rows were + // modified. For example, Hive's LOAD DATA command + // doesn't generate row count information because + // Hive doesn't inspect the data as it is loaded. + // + // modifiedRowCount is unset if the operation generates + // a result set. + 4: optional double modifiedRowCount +} + + +// OpenSession() +// +// Open a session (connection) on the server against +// which operations may be executed. +struct TOpenSessionReq { + // The version of the HiveServer2 protocol that the client is using. + 1: required TProtocolVersion client_protocol = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10 + + // Username and password for authentication. + // Depending on the authentication scheme being used, + // this information may instead be provided by a lower + // protocol layer, in which case these fields may be + // left unset. + 2: optional string username + 3: optional string password + + // Configuration overlay which is applied when the session is + // first created. + 4: optional map configuration +} + +struct TOpenSessionResp { + 1: required TStatus status + + // The protocol version that the server is using. + 2: required TProtocolVersion serverProtocolVersion = TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10 + + // Session Handle + 3: optional TSessionHandle sessionHandle + + // The configuration settings for this session. + 4: optional map configuration +} + + +// CloseSession() +// +// Closes the specified session and frees any resources +// currently allocated to that session. Any open +// operations in that session will be canceled. +struct TCloseSessionReq { + 1: required TSessionHandle sessionHandle +} + +struct TCloseSessionResp { + 1: required TStatus status +} + + + +enum TGetInfoType { + CLI_MAX_DRIVER_CONNECTIONS = 0, + CLI_MAX_CONCURRENT_ACTIVITIES = 1, + CLI_DATA_SOURCE_NAME = 2, + CLI_FETCH_DIRECTION = 8, + CLI_SERVER_NAME = 13, + CLI_SEARCH_PATTERN_ESCAPE = 14, + CLI_DBMS_NAME = 17, + CLI_DBMS_VER = 18, + CLI_ACCESSIBLE_TABLES = 19, + CLI_ACCESSIBLE_PROCEDURES = 20, + CLI_CURSOR_COMMIT_BEHAVIOR = 23, + CLI_DATA_SOURCE_READ_ONLY = 25, + CLI_DEFAULT_TXN_ISOLATION = 26, + CLI_IDENTIFIER_CASE = 28, + CLI_IDENTIFIER_QUOTE_CHAR = 29, + CLI_MAX_COLUMN_NAME_LEN = 30, + CLI_MAX_CURSOR_NAME_LEN = 31, + CLI_MAX_SCHEMA_NAME_LEN = 32, + CLI_MAX_CATALOG_NAME_LEN = 34, + CLI_MAX_TABLE_NAME_LEN = 35, + CLI_SCROLL_CONCURRENCY = 43, + CLI_TXN_CAPABLE = 46, + CLI_USER_NAME = 47, + CLI_TXN_ISOLATION_OPTION = 72, + CLI_INTEGRITY = 73, + CLI_GETDATA_EXTENSIONS = 81, + CLI_NULL_COLLATION = 85, + CLI_ALTER_TABLE = 86, + CLI_ORDER_BY_COLUMNS_IN_SELECT = 90, + CLI_SPECIAL_CHARACTERS = 94, + CLI_MAX_COLUMNS_IN_GROUP_BY = 97, + CLI_MAX_COLUMNS_IN_INDEX = 98, + CLI_MAX_COLUMNS_IN_ORDER_BY = 99, + CLI_MAX_COLUMNS_IN_SELECT = 100, + CLI_MAX_COLUMNS_IN_TABLE = 101, + CLI_MAX_INDEX_SIZE = 102, + CLI_MAX_ROW_SIZE = 104, + CLI_MAX_STATEMENT_LEN = 105, + CLI_MAX_TABLES_IN_SELECT = 106, + CLI_MAX_USER_NAME_LEN = 107, + CLI_OJ_CAPABILITIES = 115, + + CLI_XOPEN_CLI_YEAR = 10000, + CLI_CURSOR_SENSITIVITY = 10001, + CLI_DESCRIBE_PARAMETER = 10002, + CLI_CATALOG_NAME = 10003, + CLI_COLLATION_SEQ = 10004, + CLI_MAX_IDENTIFIER_LEN = 10005, +} + +union TGetInfoValue { + 1: string stringValue + 2: i16 smallIntValue + 3: i32 integerBitmask + 4: i32 integerFlag + 5: i32 binaryValue + 6: i64 lenValue +} + +// GetInfo() +// +// This function is based on ODBC's CLIGetInfo() function. +// The function returns general information about the data source +// using the same keys as ODBC. +struct TGetInfoReq { + // The sesssion to run this request against + 1: required TSessionHandle sessionHandle + + 2: required TGetInfoType infoType +} + +struct TGetInfoResp { + 1: required TStatus status + + 2: required TGetInfoValue infoValue +} + + +// ExecuteStatement() +// +// Execute a statement. +// The returned OperationHandle can be used to check on the +// status of the statement, and to fetch results once the +// statement has finished executing. +struct TExecuteStatementReq { + // The session to execute the statement against + 1: required TSessionHandle sessionHandle + + // The statement to be executed (DML, DDL, SET, etc) + 2: required string statement + + // Configuration properties that are overlayed on top of the + // the existing session configuration before this statement + // is executed. These properties apply to this statement + // only and will not affect the subsequent state of the Session. + 3: optional map confOverlay + + // Execute asynchronously when runAsync is true + 4: optional bool runAsync = false + + // The number of seconds after which the query will timeout on the server + 5: optional i64 queryTimeout = 0 +} + +struct TExecuteStatementResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + +// GetTypeInfo() +// +// Get information about types supported by the HiveServer instance. +// The information is returned as a result set which can be fetched +// using the OperationHandle provided in the response. +// +// Refer to the documentation for ODBC's CLIGetTypeInfo function for +// the format of the result set. +struct TGetTypeInfoReq { + // The session to run this request against. + 1: required TSessionHandle sessionHandle +} + +struct TGetTypeInfoResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetCatalogs() +// +// Returns the list of catalogs (databases) +// Results are ordered by TABLE_CATALOG +// +// Resultset columns : +// col1 +// name: TABLE_CAT +// type: STRING +// desc: Catalog name. NULL if not applicable. +// +struct TGetCatalogsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle +} + +struct TGetCatalogsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetSchemas() +// +// Retrieves the schema names available in this database. +// The results are ordered by TABLE_CATALOG and TABLE_SCHEM. +// col1 +// name: TABLE_SCHEM +// type: STRING +// desc: schema name +// col2 +// name: TABLE_CATALOG +// type: STRING +// desc: catalog name +struct TGetSchemasReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog. Must not contain a search pattern. + 2: optional TIdentifier catalogName + + // schema name or pattern + 3: optional TPatternOrIdentifier schemaName +} + +struct TGetSchemasResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetTables() +// +// Returns a list of tables with catalog, schema, and table +// type information. The information is returned as a result +// set which can be fetched using the OperationHandle +// provided in the response. +// Results are ordered by TABLE_TYPE, TABLE_CAT, TABLE_SCHEM, and TABLE_NAME +// +// Result Set Columns: +// +// col1 +// name: TABLE_CAT +// type: STRING +// desc: Catalog name. NULL if not applicable. +// +// col2 +// name: TABLE_SCHEM +// type: STRING +// desc: Schema name. +// +// col3 +// name: TABLE_NAME +// type: STRING +// desc: Table name. +// +// col4 +// name: TABLE_TYPE +// type: STRING +// desc: The table type, e.g. "TABLE", "VIEW", etc. +// +// col5 +// name: REMARKS +// type: STRING +// desc: Comments about the table +// +struct TGetTablesReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog or a search pattern. + 2: optional TPatternOrIdentifier catalogName + + // Name of the schema or a search pattern. + 3: optional TPatternOrIdentifier schemaName + + // Name of the table or a search pattern. + 4: optional TPatternOrIdentifier tableName + + // List of table types to match + // e.g. "TABLE", "VIEW", "SYSTEM TABLE", "GLOBAL TEMPORARY", + // "LOCAL TEMPORARY", "ALIAS", "SYNONYM", etc. + 5: optional list tableTypes +} + +struct TGetTablesResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetTableTypes() +// +// Returns the table types available in this database. +// The results are ordered by table type. +// +// col1 +// name: TABLE_TYPE +// type: STRING +// desc: Table type name. +struct TGetTableTypesReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle +} + +struct TGetTableTypesResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetColumns() +// +// Returns a list of columns in the specified tables. +// The information is returned as a result set which can be fetched +// using the OperationHandle provided in the response. +// Results are ordered by TABLE_CAT, TABLE_SCHEM, TABLE_NAME, +// and ORDINAL_POSITION. +// +// Result Set Columns are the same as those for the ODBC CLIColumns +// function. +// +struct TGetColumnsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog. Must not contain a search pattern. + 2: optional TIdentifier catalogName + + // Schema name or search pattern + 3: optional TPatternOrIdentifier schemaName + + // Table name or search pattern + 4: optional TPatternOrIdentifier tableName + + // Column name or search pattern + 5: optional TPatternOrIdentifier columnName +} + +struct TGetColumnsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + + +// GetFunctions() +// +// Returns a list of functions supported by the data source. The +// behavior of this function matches +// java.sql.DatabaseMetaData.getFunctions() both in terms of +// inputs and outputs. +// +// Result Set Columns: +// +// col1 +// name: FUNCTION_CAT +// type: STRING +// desc: Function catalog (may be null) +// +// col2 +// name: FUNCTION_SCHEM +// type: STRING +// desc: Function schema (may be null) +// +// col3 +// name: FUNCTION_NAME +// type: STRING +// desc: Function name. This is the name used to invoke the function. +// +// col4 +// name: REMARKS +// type: STRING +// desc: Explanatory comment on the function. +// +// col5 +// name: FUNCTION_TYPE +// type: SMALLINT +// desc: Kind of function. One of: +// * functionResultUnknown - Cannot determine if a return value or a table +// will be returned. +// * functionNoTable - Does not a return a table. +// * functionReturnsTable - Returns a table. +// +// col6 +// name: SPECIFIC_NAME +// type: STRING +// desc: The name which uniquely identifies this function within its schema. +// In this case this is the fully qualified class name of the class +// that implements this function. +// +struct TGetFunctionsReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // A catalog name; must match the catalog name as it is stored in the + // database; "" retrieves those without a catalog; null means + // that the catalog name should not be used to narrow the search. + 2: optional TIdentifier catalogName + + // A schema name pattern; must match the schema name as it is stored + // in the database; "" retrieves those without a schema; null means + // that the schema name should not be used to narrow the search. + 3: optional TPatternOrIdentifier schemaName + + // A function name pattern; must match the function name as it is stored + // in the database. + 4: required TPatternOrIdentifier functionName +} + +struct TGetFunctionsResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + +struct TGetPrimaryKeysReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the catalog. + 2: optional TIdentifier catalogName + + // Name of the schema. + 3: optional TIdentifier schemaName + + // Name of the table. + 4: optional TIdentifier tableName +} + +struct TGetPrimaryKeysResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + +struct TGetCrossReferenceReq { + // Session to run this request against + 1: required TSessionHandle sessionHandle + + // Name of the parent catalog. + 2: optional TIdentifier parentCatalogName + + // Name of the parent schema. + 3: optional TIdentifier parentSchemaName + + // Name of the parent table. + 4: optional TIdentifier parentTableName + + // Name of the foreign catalog. + 5: optional TIdentifier foreignCatalogName + + // Name of the foreign schema. + 6: optional TIdentifier foreignSchemaName + + // Name of the foreign table. + 7: optional TIdentifier foreignTableName +} + +struct TGetCrossReferenceResp { + 1: required TStatus status + 2: optional TOperationHandle operationHandle +} + +// GetOperationStatus() +// +// Get the status of an operation running on the server. +struct TGetOperationStatusReq { + // Session to run this request against + 1: required TOperationHandle operationHandle + // optional arguments to get progress information + 2: optional bool getProgressUpdate +} + +struct TGetOperationStatusResp { + 1: required TStatus status + 2: optional TOperationState operationState + + // If operationState is ERROR_STATE, then the following fields may be set + // sqlState as defined in the ISO/IEF CLI specification + 3: optional string sqlState + + // Internal error code + 4: optional i32 errorCode + + // Error message + 5: optional string errorMessage + + // List of statuses of sub tasks + 6: optional string taskStatus + + // When was the operation started + 7: optional i64 operationStarted + + // When was the operation completed + 8: optional i64 operationCompleted + + // If the operation has the result + 9: optional bool hasResultSet + + 10: optional TProgressUpdateResp progressUpdateResponse + +} + + +// CancelOperation() +// +// Cancels processing on the specified operation handle and +// frees any resources which were allocated. +struct TCancelOperationReq { + // Operation to cancel + 1: required TOperationHandle operationHandle +} + +struct TCancelOperationResp { + 1: required TStatus status +} + + +// CloseOperation() +// +// Given an operation in the FINISHED, CANCELED, +// or ERROR states, CloseOperation() will free +// all of the resources which were allocated on +// the server to service the operation. +struct TCloseOperationReq { + 1: required TOperationHandle operationHandle +} + +struct TCloseOperationResp { + 1: required TStatus status +} + + +// GetResultSetMetadata() +// +// Retrieves schema information for the specified operation +struct TGetResultSetMetadataReq { + // Operation for which to fetch result set schema information + 1: required TOperationHandle operationHandle +} + +struct TGetResultSetMetadataResp { + 1: required TStatus status + 2: optional TTableSchema schema +} + + +enum TFetchOrientation { + // Get the next rowset. The fetch offset is ignored. + FETCH_NEXT, + + // Get the previous rowset. The fetch offset is ignored. + // NOT SUPPORTED + FETCH_PRIOR, + + // Return the rowset at the given fetch offset relative + // to the curren rowset. + // NOT SUPPORTED + FETCH_RELATIVE, + + // Return the rowset at the specified fetch offset. + // NOT SUPPORTED + FETCH_ABSOLUTE, + + // Get the first rowset in the result set. + FETCH_FIRST, + + // Get the last rowset in the result set. + // NOT SUPPORTED + FETCH_LAST +} + +// FetchResults() +// +// Fetch rows from the server corresponding to +// a particular OperationHandle. +struct TFetchResultsReq { + // Operation from which to fetch results. + 1: required TOperationHandle operationHandle + + // The fetch orientation. For V1 this must be either + // FETCH_NEXT or FETCH_FIRST. Defaults to FETCH_NEXT. + 2: required TFetchOrientation orientation = TFetchOrientation.FETCH_NEXT + + // Max number of rows that should be returned in + // the rowset. + 3: required i64 maxRows + + // The type of a fetch results request. 0 represents Query output. 1 represents Log + 4: optional i16 fetchType = 0 +} + +struct TFetchResultsResp { + 1: required TStatus status + + // TRUE if there are more rows left to fetch from the server. + 2: optional bool hasMoreRows + + // The rowset. This is optional so that we have the + // option in the future of adding alternate formats for + // representing result set data, e.g. delimited strings, + // binary encoded, etc. + 3: optional TRowSet results +} + +// GetDelegationToken() +// Retrieve delegation token for the current user +struct TGetDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // userid for the proxy user + 2: required string owner + + // designated renewer userid + 3: required string renewer +} + +struct TGetDelegationTokenResp { + // status of the request + 1: required TStatus status + + // delegation token string + 2: optional string delegationToken +} + +// CancelDelegationToken() +// Cancel the given delegation token +struct TCancelDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // delegation token to cancel + 2: required string delegationToken +} + +struct TCancelDelegationTokenResp { + // status of the request + 1: required TStatus status +} + +// RenewDelegationToken() +// Renew the given delegation token +struct TRenewDelegationTokenReq { + // session handle + 1: required TSessionHandle sessionHandle + + // delegation token to renew + 2: required string delegationToken +} + +struct TRenewDelegationTokenResp { + // status of the request + 1: required TStatus status +} + +enum TJobExecutionStatus { + IN_PROGRESS, + COMPLETE, + NOT_AVAILABLE +} + +struct TProgressUpdateResp { + 1: required list headerNames + 2: required list> rows + 3: required double progressedPercentage + 4: required TJobExecutionStatus status + 5: required string footerSummary + 6: required i64 startTime +} + +service TCLIService { + + TOpenSessionResp OpenSession(1:TOpenSessionReq req); + + TCloseSessionResp CloseSession(1:TCloseSessionReq req); + + TGetInfoResp GetInfo(1:TGetInfoReq req); + + TExecuteStatementResp ExecuteStatement(1:TExecuteStatementReq req); + + TGetTypeInfoResp GetTypeInfo(1:TGetTypeInfoReq req); + + TGetCatalogsResp GetCatalogs(1:TGetCatalogsReq req); + + TGetSchemasResp GetSchemas(1:TGetSchemasReq req); + + TGetTablesResp GetTables(1:TGetTablesReq req); + + TGetTableTypesResp GetTableTypes(1:TGetTableTypesReq req); + + TGetColumnsResp GetColumns(1:TGetColumnsReq req); + + TGetFunctionsResp GetFunctions(1:TGetFunctionsReq req); + + TGetPrimaryKeysResp GetPrimaryKeys(1:TGetPrimaryKeysReq req); + + TGetCrossReferenceResp GetCrossReference(1:TGetCrossReferenceReq req); + + TGetOperationStatusResp GetOperationStatus(1:TGetOperationStatusReq req); + + TCancelOperationResp CancelOperation(1:TCancelOperationReq req); + + TCloseOperationResp CloseOperation(1:TCloseOperationReq req); + + TGetResultSetMetadataResp GetResultSetMetadata(1:TGetResultSetMetadataReq req); + + TFetchResultsResp FetchResults(1:TFetchResultsReq req); + + TGetDelegationTokenResp GetDelegationToken(1:TGetDelegationTokenReq req); + + TCancelDelegationTokenResp CancelDelegationToken(1:TCancelDelegationTokenReq req); + + TRenewDelegationTokenResp RenewDelegationToken(1:TRenewDelegationTokenReq req); +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java new file mode 100644 index 0000000000000..358e322632144 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TArrayTypeEntry.java @@ -0,0 +1,387 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TArrayTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TArrayTypeEntry"); + + private static final org.apache.thrift.protocol.TField OBJECT_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("objectTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TArrayTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TArrayTypeEntryTupleSchemeFactory()); + } + + private int objectTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OBJECT_TYPE_PTR((short)1, "objectTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OBJECT_TYPE_PTR + return OBJECT_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __OBJECTTYPEPTR_ISSET_ID = 0; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OBJECT_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("objectTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TArrayTypeEntry.class, metaDataMap); + } + + public TArrayTypeEntry() { + } + + public TArrayTypeEntry( + int objectTypePtr) + { + this(); + this.objectTypePtr = objectTypePtr; + setObjectTypePtrIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TArrayTypeEntry(TArrayTypeEntry other) { + __isset_bitfield = other.__isset_bitfield; + this.objectTypePtr = other.objectTypePtr; + } + + public TArrayTypeEntry deepCopy() { + return new TArrayTypeEntry(this); + } + + @Override + public void clear() { + setObjectTypePtrIsSet(false); + this.objectTypePtr = 0; + } + + public int getObjectTypePtr() { + return this.objectTypePtr; + } + + public void setObjectTypePtr(int objectTypePtr) { + this.objectTypePtr = objectTypePtr; + setObjectTypePtrIsSet(true); + } + + public void unsetObjectTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); + } + + /** Returns true if field objectTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetObjectTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID); + } + + public void setObjectTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OBJECTTYPEPTR_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OBJECT_TYPE_PTR: + if (value == null) { + unsetObjectTypePtr(); + } else { + setObjectTypePtr((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OBJECT_TYPE_PTR: + return getObjectTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OBJECT_TYPE_PTR: + return isSetObjectTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TArrayTypeEntry) + return this.equals((TArrayTypeEntry)that); + return false; + } + + public boolean equals(TArrayTypeEntry that) { + if (that == null) + return false; + + boolean this_present_objectTypePtr = true; + boolean that_present_objectTypePtr = true; + if (this_present_objectTypePtr || that_present_objectTypePtr) { + if (!(this_present_objectTypePtr && that_present_objectTypePtr)) + return false; + if (this.objectTypePtr != that.objectTypePtr) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_objectTypePtr = true; + list.add(present_objectTypePtr); + if (present_objectTypePtr) + list.add(objectTypePtr); + + return list.hashCode(); + } + + @Override + public int compareTo(TArrayTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetObjectTypePtr()).compareTo(other.isSetObjectTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetObjectTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.objectTypePtr, other.objectTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TArrayTypeEntry("); + boolean first = true; + + sb.append("objectTypePtr:"); + sb.append(this.objectTypePtr); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetObjectTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'objectTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TArrayTypeEntryStandardSchemeFactory implements SchemeFactory { + public TArrayTypeEntryStandardScheme getScheme() { + return new TArrayTypeEntryStandardScheme(); + } + } + + private static class TArrayTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OBJECT_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.objectTypePtr = iprot.readI32(); + struct.setObjectTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(OBJECT_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.objectTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TArrayTypeEntryTupleSchemeFactory implements SchemeFactory { + public TArrayTypeEntryTupleScheme getScheme() { + return new TArrayTypeEntryTupleScheme(); + } + } + + private static class TArrayTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.objectTypePtr); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TArrayTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.objectTypePtr = iprot.readI32(); + struct.setObjectTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java new file mode 100644 index 0000000000000..a869cee007c0b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBinaryColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TBinaryColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBinaryColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBinaryColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBinaryColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBinaryColumn.class, metaDataMap); + } + + public TBinaryColumn() { + } + + public TBinaryColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TBinaryColumn(TBinaryColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TBinaryColumn deepCopy() { + return new TBinaryColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(ByteBuffer elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBinaryColumn) + return this.equals((TBinaryColumn)that); + return false; + } + + public boolean equals(TBinaryColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TBinaryColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBinaryColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.values, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBinaryColumnStandardSchemeFactory implements SchemeFactory { + public TBinaryColumnStandardScheme getScheme() { + return new TBinaryColumnStandardScheme(); + } + } + + private static class TBinaryColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBinaryColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list110 = iprot.readListBegin(); + struct.values = new ArrayList(_list110.size); + ByteBuffer _elem111; + for (int _i112 = 0; _i112 < _list110.size; ++_i112) + { + _elem111 = iprot.readBinary(); + struct.values.add(_elem111); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBinaryColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); + for (ByteBuffer _iter113 : struct.values) + { + oprot.writeBinary(_iter113); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBinaryColumnTupleSchemeFactory implements SchemeFactory { + public TBinaryColumnTupleScheme getScheme() { + return new TBinaryColumnTupleScheme(); + } + } + + private static class TBinaryColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (ByteBuffer _iter114 : struct.values) + { + oprot.writeBinary(_iter114); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBinaryColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list115 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.values = new ArrayList(_list115.size); + ByteBuffer _elem116; + for (int _i117 = 0; _i117 < _list115.size; ++_i117) + { + _elem116 = iprot.readBinary(); + struct.values.add(_elem116); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java new file mode 100644 index 0000000000000..9bb636672aa1e --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TBoolColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBoolColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBoolColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolColumn.class, metaDataMap); + } + + public TBoolColumn() { + } + + public TBoolColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TBoolColumn(TBoolColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TBoolColumn deepCopy() { + return new TBoolColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(boolean elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBoolColumn) + return this.equals((TBoolColumn)that); + return false; + } + + public boolean equals(TBoolColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TBoolColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBoolColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBoolColumnStandardSchemeFactory implements SchemeFactory { + public TBoolColumnStandardScheme getScheme() { + return new TBoolColumnStandardScheme(); + } + } + + private static class TBoolColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list54 = iprot.readListBegin(); + struct.values = new ArrayList(_list54.size); + boolean _elem55; + for (int _i56 = 0; _i56 < _list54.size; ++_i56) + { + _elem55 = iprot.readBool(); + struct.values.add(_elem55); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, struct.values.size())); + for (boolean _iter57 : struct.values) + { + oprot.writeBool(_iter57); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBoolColumnTupleSchemeFactory implements SchemeFactory { + public TBoolColumnTupleScheme getScheme() { + return new TBoolColumnTupleScheme(); + } + } + + private static class TBoolColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (boolean _iter58 : struct.values) + { + oprot.writeBool(_iter58); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBoolColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list59 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BOOL, iprot.readI32()); + struct.values = new ArrayList(_list59.size); + boolean _elem60; + for (int _i61 = 0; _i61 < _list59.size; ++_i61) + { + _elem60 = iprot.readBool(); + struct.values.add(_elem60); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java new file mode 100644 index 0000000000000..87b3070a89b11 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TBoolValue.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TBoolValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TBoolValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BOOL, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TBoolValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TBoolValueTupleSchemeFactory()); + } + + private boolean value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TBoolValue.class, metaDataMap); + } + + public TBoolValue() { + } + + /** + * Performs a deep copy on other. + */ + public TBoolValue(TBoolValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TBoolValue deepCopy() { + return new TBoolValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = false; + } + + public boolean isValue() { + return this.value; + } + + public void setValue(boolean value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return isValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TBoolValue) + return this.equals((TBoolValue)that); + return false; + } + + public boolean equals(TBoolValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TBoolValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TBoolValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TBoolValueStandardSchemeFactory implements SchemeFactory { + public TBoolValueStandardScheme getScheme() { + return new TBoolValueStandardScheme(); + } + } + + private static class TBoolValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TBoolValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.value = iprot.readBool(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TBoolValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeBool(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TBoolValueTupleSchemeFactory implements SchemeFactory { + public TBoolValueTupleScheme getScheme() { + return new TBoolValueTupleScheme(); + } + } + + private static class TBoolValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeBool(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TBoolValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readBool(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java new file mode 100644 index 0000000000000..68b3d3c31eb03 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TByteColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TByteColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TByteColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteColumn.class, metaDataMap); + } + + public TByteColumn() { + } + + public TByteColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TByteColumn(TByteColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TByteColumn deepCopy() { + return new TByteColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(byte elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TByteColumn) + return this.equals((TByteColumn)that); + return false; + } + + public boolean equals(TByteColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TByteColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TByteColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TByteColumnStandardSchemeFactory implements SchemeFactory { + public TByteColumnStandardScheme getScheme() { + return new TByteColumnStandardScheme(); + } + } + + private static class TByteColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TByteColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list62 = iprot.readListBegin(); + struct.values = new ArrayList(_list62.size); + byte _elem63; + for (int _i64 = 0; _i64 < _list62.size; ++_i64) + { + _elem63 = iprot.readByte(); + struct.values.add(_elem63); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TByteColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, struct.values.size())); + for (byte _iter65 : struct.values) + { + oprot.writeByte(_iter65); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TByteColumnTupleSchemeFactory implements SchemeFactory { + public TByteColumnTupleScheme getScheme() { + return new TByteColumnTupleScheme(); + } + } + + private static class TByteColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (byte _iter66 : struct.values) + { + oprot.writeByte(_iter66); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TByteColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list67 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.BYTE, iprot.readI32()); + struct.values = new ArrayList(_list67.size); + byte _elem68; + for (int _i69 = 0; _i69 < _list67.size; ++_i69) + { + _elem68 = iprot.readByte(); + struct.values.add(_elem68); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java new file mode 100644 index 0000000000000..a3d5951335fa7 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TByteValue.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TByteValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TByteValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.BYTE, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TByteValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TByteValueTupleSchemeFactory()); + } + + private byte value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BYTE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TByteValue.class, metaDataMap); + } + + public TByteValue() { + } + + /** + * Performs a deep copy on other. + */ + public TByteValue(TByteValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TByteValue deepCopy() { + return new TByteValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public byte getValue() { + return this.value; + } + + public void setValue(byte value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Byte)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TByteValue) + return this.equals((TByteValue)that); + return false; + } + + public boolean equals(TByteValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TByteValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TByteValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TByteValueStandardSchemeFactory implements SchemeFactory { + public TByteValueStandardScheme getScheme() { + return new TByteValueStandardScheme(); + } + } + + private static class TByteValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TByteValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.BYTE) { + struct.value = iprot.readByte(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TByteValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeByte(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TByteValueTupleSchemeFactory implements SchemeFactory { + public TByteValueTupleScheme getScheme() { + return new TByteValueTupleScheme(); + } + } + + private static class TByteValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeByte(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TByteValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readByte(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java new file mode 100644 index 0000000000000..6584c24a0142a --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIService.java @@ -0,0 +1,18138 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCLIService { + + public interface Iface { + + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException; + + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException; + + public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException; + + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException; + + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException; + + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException; + + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException; + + public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException; + + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException; + + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException; + + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException; + + public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException; + + public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException; + + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException; + + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException; + + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException; + + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException; + + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException; + + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException; + + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException; + + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException; + + } + + public interface AsyncIface { + + public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetPrimaryKeys(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetCrossReference(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException; + + } + + public static class Client extends org.apache.thrift.TServiceClient implements Iface { + public static class Factory implements org.apache.thrift.TServiceClientFactory { + public Factory() {} + public Client getClient(org.apache.thrift.protocol.TProtocol prot) { + return new Client(prot); + } + public Client getClient(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + return new Client(iprot, oprot); + } + } + + public Client(org.apache.thrift.protocol.TProtocol prot) + { + super(prot, prot); + } + + public Client(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TProtocol oprot) { + super(iprot, oprot); + } + + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException + { + send_OpenSession(req); + return recv_OpenSession(); + } + + public void send_OpenSession(TOpenSessionReq req) throws org.apache.thrift.TException + { + OpenSession_args args = new OpenSession_args(); + args.setReq(req); + sendBase("OpenSession", args); + } + + public TOpenSessionResp recv_OpenSession() throws org.apache.thrift.TException + { + OpenSession_result result = new OpenSession_result(); + receiveBase(result, "OpenSession"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "OpenSession failed: unknown result"); + } + + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException + { + send_CloseSession(req); + return recv_CloseSession(); + } + + public void send_CloseSession(TCloseSessionReq req) throws org.apache.thrift.TException + { + CloseSession_args args = new CloseSession_args(); + args.setReq(req); + sendBase("CloseSession", args); + } + + public TCloseSessionResp recv_CloseSession() throws org.apache.thrift.TException + { + CloseSession_result result = new CloseSession_result(); + receiveBase(result, "CloseSession"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseSession failed: unknown result"); + } + + public TGetInfoResp GetInfo(TGetInfoReq req) throws org.apache.thrift.TException + { + send_GetInfo(req); + return recv_GetInfo(); + } + + public void send_GetInfo(TGetInfoReq req) throws org.apache.thrift.TException + { + GetInfo_args args = new GetInfo_args(); + args.setReq(req); + sendBase("GetInfo", args); + } + + public TGetInfoResp recv_GetInfo() throws org.apache.thrift.TException + { + GetInfo_result result = new GetInfo_result(); + receiveBase(result, "GetInfo"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetInfo failed: unknown result"); + } + + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException + { + send_ExecuteStatement(req); + return recv_ExecuteStatement(); + } + + public void send_ExecuteStatement(TExecuteStatementReq req) throws org.apache.thrift.TException + { + ExecuteStatement_args args = new ExecuteStatement_args(); + args.setReq(req); + sendBase("ExecuteStatement", args); + } + + public TExecuteStatementResp recv_ExecuteStatement() throws org.apache.thrift.TException + { + ExecuteStatement_result result = new ExecuteStatement_result(); + receiveBase(result, "ExecuteStatement"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "ExecuteStatement failed: unknown result"); + } + + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException + { + send_GetTypeInfo(req); + return recv_GetTypeInfo(); + } + + public void send_GetTypeInfo(TGetTypeInfoReq req) throws org.apache.thrift.TException + { + GetTypeInfo_args args = new GetTypeInfo_args(); + args.setReq(req); + sendBase("GetTypeInfo", args); + } + + public TGetTypeInfoResp recv_GetTypeInfo() throws org.apache.thrift.TException + { + GetTypeInfo_result result = new GetTypeInfo_result(); + receiveBase(result, "GetTypeInfo"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTypeInfo failed: unknown result"); + } + + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException + { + send_GetCatalogs(req); + return recv_GetCatalogs(); + } + + public void send_GetCatalogs(TGetCatalogsReq req) throws org.apache.thrift.TException + { + GetCatalogs_args args = new GetCatalogs_args(); + args.setReq(req); + sendBase("GetCatalogs", args); + } + + public TGetCatalogsResp recv_GetCatalogs() throws org.apache.thrift.TException + { + GetCatalogs_result result = new GetCatalogs_result(); + receiveBase(result, "GetCatalogs"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCatalogs failed: unknown result"); + } + + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException + { + send_GetSchemas(req); + return recv_GetSchemas(); + } + + public void send_GetSchemas(TGetSchemasReq req) throws org.apache.thrift.TException + { + GetSchemas_args args = new GetSchemas_args(); + args.setReq(req); + sendBase("GetSchemas", args); + } + + public TGetSchemasResp recv_GetSchemas() throws org.apache.thrift.TException + { + GetSchemas_result result = new GetSchemas_result(); + receiveBase(result, "GetSchemas"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetSchemas failed: unknown result"); + } + + public TGetTablesResp GetTables(TGetTablesReq req) throws org.apache.thrift.TException + { + send_GetTables(req); + return recv_GetTables(); + } + + public void send_GetTables(TGetTablesReq req) throws org.apache.thrift.TException + { + GetTables_args args = new GetTables_args(); + args.setReq(req); + sendBase("GetTables", args); + } + + public TGetTablesResp recv_GetTables() throws org.apache.thrift.TException + { + GetTables_result result = new GetTables_result(); + receiveBase(result, "GetTables"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTables failed: unknown result"); + } + + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException + { + send_GetTableTypes(req); + return recv_GetTableTypes(); + } + + public void send_GetTableTypes(TGetTableTypesReq req) throws org.apache.thrift.TException + { + GetTableTypes_args args = new GetTableTypes_args(); + args.setReq(req); + sendBase("GetTableTypes", args); + } + + public TGetTableTypesResp recv_GetTableTypes() throws org.apache.thrift.TException + { + GetTableTypes_result result = new GetTableTypes_result(); + receiveBase(result, "GetTableTypes"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetTableTypes failed: unknown result"); + } + + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException + { + send_GetColumns(req); + return recv_GetColumns(); + } + + public void send_GetColumns(TGetColumnsReq req) throws org.apache.thrift.TException + { + GetColumns_args args = new GetColumns_args(); + args.setReq(req); + sendBase("GetColumns", args); + } + + public TGetColumnsResp recv_GetColumns() throws org.apache.thrift.TException + { + GetColumns_result result = new GetColumns_result(); + receiveBase(result, "GetColumns"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetColumns failed: unknown result"); + } + + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException + { + send_GetFunctions(req); + return recv_GetFunctions(); + } + + public void send_GetFunctions(TGetFunctionsReq req) throws org.apache.thrift.TException + { + GetFunctions_args args = new GetFunctions_args(); + args.setReq(req); + sendBase("GetFunctions", args); + } + + public TGetFunctionsResp recv_GetFunctions() throws org.apache.thrift.TException + { + GetFunctions_result result = new GetFunctions_result(); + receiveBase(result, "GetFunctions"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetFunctions failed: unknown result"); + } + + public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException + { + send_GetPrimaryKeys(req); + return recv_GetPrimaryKeys(); + } + + public void send_GetPrimaryKeys(TGetPrimaryKeysReq req) throws org.apache.thrift.TException + { + GetPrimaryKeys_args args = new GetPrimaryKeys_args(); + args.setReq(req); + sendBase("GetPrimaryKeys", args); + } + + public TGetPrimaryKeysResp recv_GetPrimaryKeys() throws org.apache.thrift.TException + { + GetPrimaryKeys_result result = new GetPrimaryKeys_result(); + receiveBase(result, "GetPrimaryKeys"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetPrimaryKeys failed: unknown result"); + } + + public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException + { + send_GetCrossReference(req); + return recv_GetCrossReference(); + } + + public void send_GetCrossReference(TGetCrossReferenceReq req) throws org.apache.thrift.TException + { + GetCrossReference_args args = new GetCrossReference_args(); + args.setReq(req); + sendBase("GetCrossReference", args); + } + + public TGetCrossReferenceResp recv_GetCrossReference() throws org.apache.thrift.TException + { + GetCrossReference_result result = new GetCrossReference_result(); + receiveBase(result, "GetCrossReference"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetCrossReference failed: unknown result"); + } + + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException + { + send_GetOperationStatus(req); + return recv_GetOperationStatus(); + } + + public void send_GetOperationStatus(TGetOperationStatusReq req) throws org.apache.thrift.TException + { + GetOperationStatus_args args = new GetOperationStatus_args(); + args.setReq(req); + sendBase("GetOperationStatus", args); + } + + public TGetOperationStatusResp recv_GetOperationStatus() throws org.apache.thrift.TException + { + GetOperationStatus_result result = new GetOperationStatus_result(); + receiveBase(result, "GetOperationStatus"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetOperationStatus failed: unknown result"); + } + + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException + { + send_CancelOperation(req); + return recv_CancelOperation(); + } + + public void send_CancelOperation(TCancelOperationReq req) throws org.apache.thrift.TException + { + CancelOperation_args args = new CancelOperation_args(); + args.setReq(req); + sendBase("CancelOperation", args); + } + + public TCancelOperationResp recv_CancelOperation() throws org.apache.thrift.TException + { + CancelOperation_result result = new CancelOperation_result(); + receiveBase(result, "CancelOperation"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelOperation failed: unknown result"); + } + + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException + { + send_CloseOperation(req); + return recv_CloseOperation(); + } + + public void send_CloseOperation(TCloseOperationReq req) throws org.apache.thrift.TException + { + CloseOperation_args args = new CloseOperation_args(); + args.setReq(req); + sendBase("CloseOperation", args); + } + + public TCloseOperationResp recv_CloseOperation() throws org.apache.thrift.TException + { + CloseOperation_result result = new CloseOperation_result(); + receiveBase(result, "CloseOperation"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CloseOperation failed: unknown result"); + } + + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException + { + send_GetResultSetMetadata(req); + return recv_GetResultSetMetadata(); + } + + public void send_GetResultSetMetadata(TGetResultSetMetadataReq req) throws org.apache.thrift.TException + { + GetResultSetMetadata_args args = new GetResultSetMetadata_args(); + args.setReq(req); + sendBase("GetResultSetMetadata", args); + } + + public TGetResultSetMetadataResp recv_GetResultSetMetadata() throws org.apache.thrift.TException + { + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + receiveBase(result, "GetResultSetMetadata"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetResultSetMetadata failed: unknown result"); + } + + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException + { + send_FetchResults(req); + return recv_FetchResults(); + } + + public void send_FetchResults(TFetchResultsReq req) throws org.apache.thrift.TException + { + FetchResults_args args = new FetchResults_args(); + args.setReq(req); + sendBase("FetchResults", args); + } + + public TFetchResultsResp recv_FetchResults() throws org.apache.thrift.TException + { + FetchResults_result result = new FetchResults_result(); + receiveBase(result, "FetchResults"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "FetchResults failed: unknown result"); + } + + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException + { + send_GetDelegationToken(req); + return recv_GetDelegationToken(); + } + + public void send_GetDelegationToken(TGetDelegationTokenReq req) throws org.apache.thrift.TException + { + GetDelegationToken_args args = new GetDelegationToken_args(); + args.setReq(req); + sendBase("GetDelegationToken", args); + } + + public TGetDelegationTokenResp recv_GetDelegationToken() throws org.apache.thrift.TException + { + GetDelegationToken_result result = new GetDelegationToken_result(); + receiveBase(result, "GetDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "GetDelegationToken failed: unknown result"); + } + + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException + { + send_CancelDelegationToken(req); + return recv_CancelDelegationToken(); + } + + public void send_CancelDelegationToken(TCancelDelegationTokenReq req) throws org.apache.thrift.TException + { + CancelDelegationToken_args args = new CancelDelegationToken_args(); + args.setReq(req); + sendBase("CancelDelegationToken", args); + } + + public TCancelDelegationTokenResp recv_CancelDelegationToken() throws org.apache.thrift.TException + { + CancelDelegationToken_result result = new CancelDelegationToken_result(); + receiveBase(result, "CancelDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "CancelDelegationToken failed: unknown result"); + } + + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException + { + send_RenewDelegationToken(req); + return recv_RenewDelegationToken(); + } + + public void send_RenewDelegationToken(TRenewDelegationTokenReq req) throws org.apache.thrift.TException + { + RenewDelegationToken_args args = new RenewDelegationToken_args(); + args.setReq(req); + sendBase("RenewDelegationToken", args); + } + + public TRenewDelegationTokenResp recv_RenewDelegationToken() throws org.apache.thrift.TException + { + RenewDelegationToken_result result = new RenewDelegationToken_result(); + receiveBase(result, "RenewDelegationToken"); + if (result.isSetSuccess()) { + return result.success; + } + throw new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.MISSING_RESULT, "RenewDelegationToken failed: unknown result"); + } + + } + public static class AsyncClient extends org.apache.thrift.async.TAsyncClient implements AsyncIface { + public static class Factory implements org.apache.thrift.async.TAsyncClientFactory { + private org.apache.thrift.async.TAsyncClientManager clientManager; + private org.apache.thrift.protocol.TProtocolFactory protocolFactory; + public Factory(org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.protocol.TProtocolFactory protocolFactory) { + this.clientManager = clientManager; + this.protocolFactory = protocolFactory; + } + public AsyncClient getAsyncClient(org.apache.thrift.transport.TNonblockingTransport transport) { + return new AsyncClient(protocolFactory, clientManager, transport); + } + } + + public AsyncClient(org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.async.TAsyncClientManager clientManager, org.apache.thrift.transport.TNonblockingTransport transport) { + super(protocolFactory, clientManager, transport); + } + + public void OpenSession(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + OpenSession_call method_call = new OpenSession_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class OpenSession_call extends org.apache.thrift.async.TAsyncMethodCall { + private TOpenSessionReq req; + public OpenSession_call(TOpenSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("OpenSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); + OpenSession_args args = new OpenSession_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TOpenSessionResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_OpenSession(); + } + } + + public void CloseSession(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CloseSession_call method_call = new CloseSession_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CloseSession_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCloseSessionReq req; + public CloseSession_call(TCloseSessionReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseSession", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CloseSession_args args = new CloseSession_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCloseSessionResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CloseSession(); + } + } + + public void GetInfo(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetInfo_call method_call = new GetInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetInfoReq req; + public GetInfo_call(TGetInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetInfo_args args = new GetInfo_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetInfoResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetInfo(); + } + } + + public void ExecuteStatement(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + ExecuteStatement_call method_call = new ExecuteStatement_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class ExecuteStatement_call extends org.apache.thrift.async.TAsyncMethodCall { + private TExecuteStatementReq req; + public ExecuteStatement_call(TExecuteStatementReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("ExecuteStatement", org.apache.thrift.protocol.TMessageType.CALL, 0)); + ExecuteStatement_args args = new ExecuteStatement_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TExecuteStatementResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_ExecuteStatement(); + } + } + + public void GetTypeInfo(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTypeInfo_call method_call = new GetTypeInfo_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTypeInfo_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTypeInfoReq req; + public GetTypeInfo_call(TGetTypeInfoReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTypeInfo", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTypeInfo_args args = new GetTypeInfo_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTypeInfoResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTypeInfo(); + } + } + + public void GetCatalogs(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetCatalogs_call method_call = new GetCatalogs_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetCatalogs_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetCatalogsReq req; + public GetCatalogs_call(TGetCatalogsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCatalogs", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetCatalogs_args args = new GetCatalogs_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetCatalogsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetCatalogs(); + } + } + + public void GetSchemas(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetSchemas_call method_call = new GetSchemas_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetSchemas_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetSchemasReq req; + public GetSchemas_call(TGetSchemasReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetSchemas", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetSchemas_args args = new GetSchemas_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetSchemasResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetSchemas(); + } + } + + public void GetTables(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTables_call method_call = new GetTables_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTables_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTablesReq req; + public GetTables_call(TGetTablesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTables", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTables_args args = new GetTables_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTablesResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTables(); + } + } + + public void GetTableTypes(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetTableTypes_call method_call = new GetTableTypes_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetTableTypes_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetTableTypesReq req; + public GetTableTypes_call(TGetTableTypesReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetTableTypes", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetTableTypes_args args = new GetTableTypes_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetTableTypesResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetTableTypes(); + } + } + + public void GetColumns(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetColumns_call method_call = new GetColumns_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetColumns_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetColumnsReq req; + public GetColumns_call(TGetColumnsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetColumns", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetColumns_args args = new GetColumns_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetColumnsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetColumns(); + } + } + + public void GetFunctions(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetFunctions_call method_call = new GetFunctions_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetFunctions_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetFunctionsReq req; + public GetFunctions_call(TGetFunctionsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetFunctions", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetFunctions_args args = new GetFunctions_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetFunctionsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetFunctions(); + } + } + + public void GetPrimaryKeys(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetPrimaryKeys_call method_call = new GetPrimaryKeys_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetPrimaryKeys_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetPrimaryKeysReq req; + public GetPrimaryKeys_call(TGetPrimaryKeysReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetPrimaryKeys", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetPrimaryKeys_args args = new GetPrimaryKeys_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetPrimaryKeysResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetPrimaryKeys(); + } + } + + public void GetCrossReference(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetCrossReference_call method_call = new GetCrossReference_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetCrossReference_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetCrossReferenceReq req; + public GetCrossReference_call(TGetCrossReferenceReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetCrossReference", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetCrossReference_args args = new GetCrossReference_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetCrossReferenceResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetCrossReference(); + } + } + + public void GetOperationStatus(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetOperationStatus_call method_call = new GetOperationStatus_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetOperationStatus_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetOperationStatusReq req; + public GetOperationStatus_call(TGetOperationStatusReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetOperationStatus", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetOperationStatus_args args = new GetOperationStatus_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetOperationStatusResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetOperationStatus(); + } + } + + public void CancelOperation(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CancelOperation_call method_call = new CancelOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CancelOperation_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCancelOperationReq req; + public CancelOperation_call(TCancelOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CancelOperation_args args = new CancelOperation_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCancelOperationResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CancelOperation(); + } + } + + public void CloseOperation(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CloseOperation_call method_call = new CloseOperation_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CloseOperation_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCloseOperationReq req; + public CloseOperation_call(TCloseOperationReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CloseOperation", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CloseOperation_args args = new CloseOperation_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCloseOperationResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CloseOperation(); + } + } + + public void GetResultSetMetadata(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetResultSetMetadata_call method_call = new GetResultSetMetadata_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetResultSetMetadata_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetResultSetMetadataReq req; + public GetResultSetMetadata_call(TGetResultSetMetadataReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetResultSetMetadata", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetResultSetMetadata_args args = new GetResultSetMetadata_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetResultSetMetadataResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetResultSetMetadata(); + } + } + + public void FetchResults(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + FetchResults_call method_call = new FetchResults_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class FetchResults_call extends org.apache.thrift.async.TAsyncMethodCall { + private TFetchResultsReq req; + public FetchResults_call(TFetchResultsReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("FetchResults", org.apache.thrift.protocol.TMessageType.CALL, 0)); + FetchResults_args args = new FetchResults_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TFetchResultsResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_FetchResults(); + } + } + + public void GetDelegationToken(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + GetDelegationToken_call method_call = new GetDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class GetDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TGetDelegationTokenReq req; + public GetDelegationToken_call(TGetDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("GetDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + GetDelegationToken_args args = new GetDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TGetDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_GetDelegationToken(); + } + } + + public void CancelDelegationToken(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + CancelDelegationToken_call method_call = new CancelDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class CancelDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TCancelDelegationTokenReq req; + public CancelDelegationToken_call(TCancelDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("CancelDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + CancelDelegationToken_args args = new CancelDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TCancelDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_CancelDelegationToken(); + } + } + + public void RenewDelegationToken(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws org.apache.thrift.TException { + checkReady(); + RenewDelegationToken_call method_call = new RenewDelegationToken_call(req, resultHandler, this, ___protocolFactory, ___transport); + this.___currentMethod = method_call; + ___manager.call(method_call); + } + + public static class RenewDelegationToken_call extends org.apache.thrift.async.TAsyncMethodCall { + private TRenewDelegationTokenReq req; + public RenewDelegationToken_call(TRenewDelegationTokenReq req, org.apache.thrift.async.AsyncMethodCallback resultHandler, org.apache.thrift.async.TAsyncClient client, org.apache.thrift.protocol.TProtocolFactory protocolFactory, org.apache.thrift.transport.TNonblockingTransport transport) throws org.apache.thrift.TException { + super(client, protocolFactory, transport, resultHandler, false); + this.req = req; + } + + public void write_args(org.apache.thrift.protocol.TProtocol prot) throws org.apache.thrift.TException { + prot.writeMessageBegin(new org.apache.thrift.protocol.TMessage("RenewDelegationToken", org.apache.thrift.protocol.TMessageType.CALL, 0)); + RenewDelegationToken_args args = new RenewDelegationToken_args(); + args.setReq(req); + args.write(prot); + prot.writeMessageEnd(); + } + + public TRenewDelegationTokenResp getResult() throws org.apache.thrift.TException { + if (getState() != org.apache.thrift.async.TAsyncMethodCall.State.RESPONSE_READ) { + throw new IllegalStateException("Method call not finished!"); + } + org.apache.thrift.transport.TMemoryInputTransport memoryTransport = new org.apache.thrift.transport.TMemoryInputTransport(getFrameBuffer().array()); + org.apache.thrift.protocol.TProtocol prot = client.getProtocolFactory().getProtocol(memoryTransport); + return (new Client(prot)).recv_RenewDelegationToken(); + } + } + + } + + public static class Processor extends org.apache.thrift.TBaseProcessor implements org.apache.thrift.TProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(Processor.class.getName()); + public Processor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected Processor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("OpenSession", new OpenSession()); + processMap.put("CloseSession", new CloseSession()); + processMap.put("GetInfo", new GetInfo()); + processMap.put("ExecuteStatement", new ExecuteStatement()); + processMap.put("GetTypeInfo", new GetTypeInfo()); + processMap.put("GetCatalogs", new GetCatalogs()); + processMap.put("GetSchemas", new GetSchemas()); + processMap.put("GetTables", new GetTables()); + processMap.put("GetTableTypes", new GetTableTypes()); + processMap.put("GetColumns", new GetColumns()); + processMap.put("GetFunctions", new GetFunctions()); + processMap.put("GetPrimaryKeys", new GetPrimaryKeys()); + processMap.put("GetCrossReference", new GetCrossReference()); + processMap.put("GetOperationStatus", new GetOperationStatus()); + processMap.put("CancelOperation", new CancelOperation()); + processMap.put("CloseOperation", new CloseOperation()); + processMap.put("GetResultSetMetadata", new GetResultSetMetadata()); + processMap.put("FetchResults", new FetchResults()); + processMap.put("GetDelegationToken", new GetDelegationToken()); + processMap.put("CancelDelegationToken", new CancelDelegationToken()); + processMap.put("RenewDelegationToken", new RenewDelegationToken()); + return processMap; + } + + public static class OpenSession extends org.apache.thrift.ProcessFunction { + public OpenSession() { + super("OpenSession"); + } + + public OpenSession_args getEmptyArgsInstance() { + return new OpenSession_args(); + } + + protected boolean isOneway() { + return false; + } + + public OpenSession_result getResult(I iface, OpenSession_args args) throws org.apache.thrift.TException { + OpenSession_result result = new OpenSession_result(); + result.success = iface.OpenSession(args.req); + return result; + } + } + + public static class CloseSession extends org.apache.thrift.ProcessFunction { + public CloseSession() { + super("CloseSession"); + } + + public CloseSession_args getEmptyArgsInstance() { + return new CloseSession_args(); + } + + protected boolean isOneway() { + return false; + } + + public CloseSession_result getResult(I iface, CloseSession_args args) throws org.apache.thrift.TException { + CloseSession_result result = new CloseSession_result(); + result.success = iface.CloseSession(args.req); + return result; + } + } + + public static class GetInfo extends org.apache.thrift.ProcessFunction { + public GetInfo() { + super("GetInfo"); + } + + public GetInfo_args getEmptyArgsInstance() { + return new GetInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetInfo_result getResult(I iface, GetInfo_args args) throws org.apache.thrift.TException { + GetInfo_result result = new GetInfo_result(); + result.success = iface.GetInfo(args.req); + return result; + } + } + + public static class ExecuteStatement extends org.apache.thrift.ProcessFunction { + public ExecuteStatement() { + super("ExecuteStatement"); + } + + public ExecuteStatement_args getEmptyArgsInstance() { + return new ExecuteStatement_args(); + } + + protected boolean isOneway() { + return false; + } + + public ExecuteStatement_result getResult(I iface, ExecuteStatement_args args) throws org.apache.thrift.TException { + ExecuteStatement_result result = new ExecuteStatement_result(); + result.success = iface.ExecuteStatement(args.req); + return result; + } + } + + public static class GetTypeInfo extends org.apache.thrift.ProcessFunction { + public GetTypeInfo() { + super("GetTypeInfo"); + } + + public GetTypeInfo_args getEmptyArgsInstance() { + return new GetTypeInfo_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTypeInfo_result getResult(I iface, GetTypeInfo_args args) throws org.apache.thrift.TException { + GetTypeInfo_result result = new GetTypeInfo_result(); + result.success = iface.GetTypeInfo(args.req); + return result; + } + } + + public static class GetCatalogs extends org.apache.thrift.ProcessFunction { + public GetCatalogs() { + super("GetCatalogs"); + } + + public GetCatalogs_args getEmptyArgsInstance() { + return new GetCatalogs_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetCatalogs_result getResult(I iface, GetCatalogs_args args) throws org.apache.thrift.TException { + GetCatalogs_result result = new GetCatalogs_result(); + result.success = iface.GetCatalogs(args.req); + return result; + } + } + + public static class GetSchemas extends org.apache.thrift.ProcessFunction { + public GetSchemas() { + super("GetSchemas"); + } + + public GetSchemas_args getEmptyArgsInstance() { + return new GetSchemas_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetSchemas_result getResult(I iface, GetSchemas_args args) throws org.apache.thrift.TException { + GetSchemas_result result = new GetSchemas_result(); + result.success = iface.GetSchemas(args.req); + return result; + } + } + + public static class GetTables extends org.apache.thrift.ProcessFunction { + public GetTables() { + super("GetTables"); + } + + public GetTables_args getEmptyArgsInstance() { + return new GetTables_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTables_result getResult(I iface, GetTables_args args) throws org.apache.thrift.TException { + GetTables_result result = new GetTables_result(); + result.success = iface.GetTables(args.req); + return result; + } + } + + public static class GetTableTypes extends org.apache.thrift.ProcessFunction { + public GetTableTypes() { + super("GetTableTypes"); + } + + public GetTableTypes_args getEmptyArgsInstance() { + return new GetTableTypes_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetTableTypes_result getResult(I iface, GetTableTypes_args args) throws org.apache.thrift.TException { + GetTableTypes_result result = new GetTableTypes_result(); + result.success = iface.GetTableTypes(args.req); + return result; + } + } + + public static class GetColumns extends org.apache.thrift.ProcessFunction { + public GetColumns() { + super("GetColumns"); + } + + public GetColumns_args getEmptyArgsInstance() { + return new GetColumns_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetColumns_result getResult(I iface, GetColumns_args args) throws org.apache.thrift.TException { + GetColumns_result result = new GetColumns_result(); + result.success = iface.GetColumns(args.req); + return result; + } + } + + public static class GetFunctions extends org.apache.thrift.ProcessFunction { + public GetFunctions() { + super("GetFunctions"); + } + + public GetFunctions_args getEmptyArgsInstance() { + return new GetFunctions_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetFunctions_result getResult(I iface, GetFunctions_args args) throws org.apache.thrift.TException { + GetFunctions_result result = new GetFunctions_result(); + result.success = iface.GetFunctions(args.req); + return result; + } + } + + public static class GetPrimaryKeys extends org.apache.thrift.ProcessFunction { + public GetPrimaryKeys() { + super("GetPrimaryKeys"); + } + + public GetPrimaryKeys_args getEmptyArgsInstance() { + return new GetPrimaryKeys_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetPrimaryKeys_result getResult(I iface, GetPrimaryKeys_args args) throws org.apache.thrift.TException { + GetPrimaryKeys_result result = new GetPrimaryKeys_result(); + result.success = iface.GetPrimaryKeys(args.req); + return result; + } + } + + public static class GetCrossReference extends org.apache.thrift.ProcessFunction { + public GetCrossReference() { + super("GetCrossReference"); + } + + public GetCrossReference_args getEmptyArgsInstance() { + return new GetCrossReference_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetCrossReference_result getResult(I iface, GetCrossReference_args args) throws org.apache.thrift.TException { + GetCrossReference_result result = new GetCrossReference_result(); + result.success = iface.GetCrossReference(args.req); + return result; + } + } + + public static class GetOperationStatus extends org.apache.thrift.ProcessFunction { + public GetOperationStatus() { + super("GetOperationStatus"); + } + + public GetOperationStatus_args getEmptyArgsInstance() { + return new GetOperationStatus_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetOperationStatus_result getResult(I iface, GetOperationStatus_args args) throws org.apache.thrift.TException { + GetOperationStatus_result result = new GetOperationStatus_result(); + result.success = iface.GetOperationStatus(args.req); + return result; + } + } + + public static class CancelOperation extends org.apache.thrift.ProcessFunction { + public CancelOperation() { + super("CancelOperation"); + } + + public CancelOperation_args getEmptyArgsInstance() { + return new CancelOperation_args(); + } + + protected boolean isOneway() { + return false; + } + + public CancelOperation_result getResult(I iface, CancelOperation_args args) throws org.apache.thrift.TException { + CancelOperation_result result = new CancelOperation_result(); + result.success = iface.CancelOperation(args.req); + return result; + } + } + + public static class CloseOperation extends org.apache.thrift.ProcessFunction { + public CloseOperation() { + super("CloseOperation"); + } + + public CloseOperation_args getEmptyArgsInstance() { + return new CloseOperation_args(); + } + + protected boolean isOneway() { + return false; + } + + public CloseOperation_result getResult(I iface, CloseOperation_args args) throws org.apache.thrift.TException { + CloseOperation_result result = new CloseOperation_result(); + result.success = iface.CloseOperation(args.req); + return result; + } + } + + public static class GetResultSetMetadata extends org.apache.thrift.ProcessFunction { + public GetResultSetMetadata() { + super("GetResultSetMetadata"); + } + + public GetResultSetMetadata_args getEmptyArgsInstance() { + return new GetResultSetMetadata_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetResultSetMetadata_result getResult(I iface, GetResultSetMetadata_args args) throws org.apache.thrift.TException { + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + result.success = iface.GetResultSetMetadata(args.req); + return result; + } + } + + public static class FetchResults extends org.apache.thrift.ProcessFunction { + public FetchResults() { + super("FetchResults"); + } + + public FetchResults_args getEmptyArgsInstance() { + return new FetchResults_args(); + } + + protected boolean isOneway() { + return false; + } + + public FetchResults_result getResult(I iface, FetchResults_args args) throws org.apache.thrift.TException { + FetchResults_result result = new FetchResults_result(); + result.success = iface.FetchResults(args.req); + return result; + } + } + + public static class GetDelegationToken extends org.apache.thrift.ProcessFunction { + public GetDelegationToken() { + super("GetDelegationToken"); + } + + public GetDelegationToken_args getEmptyArgsInstance() { + return new GetDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public GetDelegationToken_result getResult(I iface, GetDelegationToken_args args) throws org.apache.thrift.TException { + GetDelegationToken_result result = new GetDelegationToken_result(); + result.success = iface.GetDelegationToken(args.req); + return result; + } + } + + public static class CancelDelegationToken extends org.apache.thrift.ProcessFunction { + public CancelDelegationToken() { + super("CancelDelegationToken"); + } + + public CancelDelegationToken_args getEmptyArgsInstance() { + return new CancelDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public CancelDelegationToken_result getResult(I iface, CancelDelegationToken_args args) throws org.apache.thrift.TException { + CancelDelegationToken_result result = new CancelDelegationToken_result(); + result.success = iface.CancelDelegationToken(args.req); + return result; + } + } + + public static class RenewDelegationToken extends org.apache.thrift.ProcessFunction { + public RenewDelegationToken() { + super("RenewDelegationToken"); + } + + public RenewDelegationToken_args getEmptyArgsInstance() { + return new RenewDelegationToken_args(); + } + + protected boolean isOneway() { + return false; + } + + public RenewDelegationToken_result getResult(I iface, RenewDelegationToken_args args) throws org.apache.thrift.TException { + RenewDelegationToken_result result = new RenewDelegationToken_result(); + result.success = iface.RenewDelegationToken(args.req); + return result; + } + } + + } + + public static class AsyncProcessor extends org.apache.thrift.TBaseAsyncProcessor { + private static final Logger LOGGER = LoggerFactory.getLogger(AsyncProcessor.class.getName()); + public AsyncProcessor(I iface) { + super(iface, getProcessMap(new HashMap>())); + } + + protected AsyncProcessor(I iface, Map> processMap) { + super(iface, getProcessMap(processMap)); + } + + private static Map> getProcessMap(Map> processMap) { + processMap.put("OpenSession", new OpenSession()); + processMap.put("CloseSession", new CloseSession()); + processMap.put("GetInfo", new GetInfo()); + processMap.put("ExecuteStatement", new ExecuteStatement()); + processMap.put("GetTypeInfo", new GetTypeInfo()); + processMap.put("GetCatalogs", new GetCatalogs()); + processMap.put("GetSchemas", new GetSchemas()); + processMap.put("GetTables", new GetTables()); + processMap.put("GetTableTypes", new GetTableTypes()); + processMap.put("GetColumns", new GetColumns()); + processMap.put("GetFunctions", new GetFunctions()); + processMap.put("GetPrimaryKeys", new GetPrimaryKeys()); + processMap.put("GetCrossReference", new GetCrossReference()); + processMap.put("GetOperationStatus", new GetOperationStatus()); + processMap.put("CancelOperation", new CancelOperation()); + processMap.put("CloseOperation", new CloseOperation()); + processMap.put("GetResultSetMetadata", new GetResultSetMetadata()); + processMap.put("FetchResults", new FetchResults()); + processMap.put("GetDelegationToken", new GetDelegationToken()); + processMap.put("CancelDelegationToken", new CancelDelegationToken()); + processMap.put("RenewDelegationToken", new RenewDelegationToken()); + return processMap; + } + + public static class OpenSession extends org.apache.thrift.AsyncProcessFunction { + public OpenSession() { + super("OpenSession"); + } + + public OpenSession_args getEmptyArgsInstance() { + return new OpenSession_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TOpenSessionResp o) { + OpenSession_result result = new OpenSession_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + OpenSession_result result = new OpenSession_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, OpenSession_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.OpenSession(args.req,resultHandler); + } + } + + public static class CloseSession extends org.apache.thrift.AsyncProcessFunction { + public CloseSession() { + super("CloseSession"); + } + + public CloseSession_args getEmptyArgsInstance() { + return new CloseSession_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TCloseSessionResp o) { + CloseSession_result result = new CloseSession_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + CloseSession_result result = new CloseSession_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, CloseSession_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.CloseSession(args.req,resultHandler); + } + } + + public static class GetInfo extends org.apache.thrift.AsyncProcessFunction { + public GetInfo() { + super("GetInfo"); + } + + public GetInfo_args getEmptyArgsInstance() { + return new GetInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetInfoResp o) { + GetInfo_result result = new GetInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetInfo_result result = new GetInfo_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetInfo(args.req,resultHandler); + } + } + + public static class ExecuteStatement extends org.apache.thrift.AsyncProcessFunction { + public ExecuteStatement() { + super("ExecuteStatement"); + } + + public ExecuteStatement_args getEmptyArgsInstance() { + return new ExecuteStatement_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TExecuteStatementResp o) { + ExecuteStatement_result result = new ExecuteStatement_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + ExecuteStatement_result result = new ExecuteStatement_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, ExecuteStatement_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.ExecuteStatement(args.req,resultHandler); + } + } + + public static class GetTypeInfo extends org.apache.thrift.AsyncProcessFunction { + public GetTypeInfo() { + super("GetTypeInfo"); + } + + public GetTypeInfo_args getEmptyArgsInstance() { + return new GetTypeInfo_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetTypeInfoResp o) { + GetTypeInfo_result result = new GetTypeInfo_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetTypeInfo_result result = new GetTypeInfo_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetTypeInfo_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetTypeInfo(args.req,resultHandler); + } + } + + public static class GetCatalogs extends org.apache.thrift.AsyncProcessFunction { + public GetCatalogs() { + super("GetCatalogs"); + } + + public GetCatalogs_args getEmptyArgsInstance() { + return new GetCatalogs_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetCatalogsResp o) { + GetCatalogs_result result = new GetCatalogs_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetCatalogs_result result = new GetCatalogs_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetCatalogs_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetCatalogs(args.req,resultHandler); + } + } + + public static class GetSchemas extends org.apache.thrift.AsyncProcessFunction { + public GetSchemas() { + super("GetSchemas"); + } + + public GetSchemas_args getEmptyArgsInstance() { + return new GetSchemas_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetSchemasResp o) { + GetSchemas_result result = new GetSchemas_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetSchemas_result result = new GetSchemas_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetSchemas_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetSchemas(args.req,resultHandler); + } + } + + public static class GetTables extends org.apache.thrift.AsyncProcessFunction { + public GetTables() { + super("GetTables"); + } + + public GetTables_args getEmptyArgsInstance() { + return new GetTables_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetTablesResp o) { + GetTables_result result = new GetTables_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetTables_result result = new GetTables_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetTables_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetTables(args.req,resultHandler); + } + } + + public static class GetTableTypes extends org.apache.thrift.AsyncProcessFunction { + public GetTableTypes() { + super("GetTableTypes"); + } + + public GetTableTypes_args getEmptyArgsInstance() { + return new GetTableTypes_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetTableTypesResp o) { + GetTableTypes_result result = new GetTableTypes_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetTableTypes_result result = new GetTableTypes_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetTableTypes_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetTableTypes(args.req,resultHandler); + } + } + + public static class GetColumns extends org.apache.thrift.AsyncProcessFunction { + public GetColumns() { + super("GetColumns"); + } + + public GetColumns_args getEmptyArgsInstance() { + return new GetColumns_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetColumnsResp o) { + GetColumns_result result = new GetColumns_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetColumns_result result = new GetColumns_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetColumns_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetColumns(args.req,resultHandler); + } + } + + public static class GetFunctions extends org.apache.thrift.AsyncProcessFunction { + public GetFunctions() { + super("GetFunctions"); + } + + public GetFunctions_args getEmptyArgsInstance() { + return new GetFunctions_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetFunctionsResp o) { + GetFunctions_result result = new GetFunctions_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetFunctions_result result = new GetFunctions_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetFunctions_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetFunctions(args.req,resultHandler); + } + } + + public static class GetPrimaryKeys extends org.apache.thrift.AsyncProcessFunction { + public GetPrimaryKeys() { + super("GetPrimaryKeys"); + } + + public GetPrimaryKeys_args getEmptyArgsInstance() { + return new GetPrimaryKeys_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetPrimaryKeysResp o) { + GetPrimaryKeys_result result = new GetPrimaryKeys_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetPrimaryKeys_result result = new GetPrimaryKeys_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetPrimaryKeys_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetPrimaryKeys(args.req,resultHandler); + } + } + + public static class GetCrossReference extends org.apache.thrift.AsyncProcessFunction { + public GetCrossReference() { + super("GetCrossReference"); + } + + public GetCrossReference_args getEmptyArgsInstance() { + return new GetCrossReference_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetCrossReferenceResp o) { + GetCrossReference_result result = new GetCrossReference_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetCrossReference_result result = new GetCrossReference_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetCrossReference_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetCrossReference(args.req,resultHandler); + } + } + + public static class GetOperationStatus extends org.apache.thrift.AsyncProcessFunction { + public GetOperationStatus() { + super("GetOperationStatus"); + } + + public GetOperationStatus_args getEmptyArgsInstance() { + return new GetOperationStatus_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetOperationStatusResp o) { + GetOperationStatus_result result = new GetOperationStatus_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetOperationStatus_result result = new GetOperationStatus_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetOperationStatus_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetOperationStatus(args.req,resultHandler); + } + } + + public static class CancelOperation extends org.apache.thrift.AsyncProcessFunction { + public CancelOperation() { + super("CancelOperation"); + } + + public CancelOperation_args getEmptyArgsInstance() { + return new CancelOperation_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TCancelOperationResp o) { + CancelOperation_result result = new CancelOperation_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + CancelOperation_result result = new CancelOperation_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, CancelOperation_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.CancelOperation(args.req,resultHandler); + } + } + + public static class CloseOperation extends org.apache.thrift.AsyncProcessFunction { + public CloseOperation() { + super("CloseOperation"); + } + + public CloseOperation_args getEmptyArgsInstance() { + return new CloseOperation_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TCloseOperationResp o) { + CloseOperation_result result = new CloseOperation_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + CloseOperation_result result = new CloseOperation_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, CloseOperation_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.CloseOperation(args.req,resultHandler); + } + } + + public static class GetResultSetMetadata extends org.apache.thrift.AsyncProcessFunction { + public GetResultSetMetadata() { + super("GetResultSetMetadata"); + } + + public GetResultSetMetadata_args getEmptyArgsInstance() { + return new GetResultSetMetadata_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetResultSetMetadataResp o) { + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetResultSetMetadata_result result = new GetResultSetMetadata_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetResultSetMetadata_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetResultSetMetadata(args.req,resultHandler); + } + } + + public static class FetchResults extends org.apache.thrift.AsyncProcessFunction { + public FetchResults() { + super("FetchResults"); + } + + public FetchResults_args getEmptyArgsInstance() { + return new FetchResults_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TFetchResultsResp o) { + FetchResults_result result = new FetchResults_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + FetchResults_result result = new FetchResults_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, FetchResults_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.FetchResults(args.req,resultHandler); + } + } + + public static class GetDelegationToken extends org.apache.thrift.AsyncProcessFunction { + public GetDelegationToken() { + super("GetDelegationToken"); + } + + public GetDelegationToken_args getEmptyArgsInstance() { + return new GetDelegationToken_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TGetDelegationTokenResp o) { + GetDelegationToken_result result = new GetDelegationToken_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + GetDelegationToken_result result = new GetDelegationToken_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, GetDelegationToken_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.GetDelegationToken(args.req,resultHandler); + } + } + + public static class CancelDelegationToken extends org.apache.thrift.AsyncProcessFunction { + public CancelDelegationToken() { + super("CancelDelegationToken"); + } + + public CancelDelegationToken_args getEmptyArgsInstance() { + return new CancelDelegationToken_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TCancelDelegationTokenResp o) { + CancelDelegationToken_result result = new CancelDelegationToken_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + CancelDelegationToken_result result = new CancelDelegationToken_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, CancelDelegationToken_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.CancelDelegationToken(args.req,resultHandler); + } + } + + public static class RenewDelegationToken extends org.apache.thrift.AsyncProcessFunction { + public RenewDelegationToken() { + super("RenewDelegationToken"); + } + + public RenewDelegationToken_args getEmptyArgsInstance() { + return new RenewDelegationToken_args(); + } + + public AsyncMethodCallback getResultHandler(final AbstractNonblockingServer.AsyncFrameBuffer fb, final int seqid) { + final org.apache.thrift.AsyncProcessFunction fcall = this; + return new AsyncMethodCallback() { + public void onComplete(TRenewDelegationTokenResp o) { + RenewDelegationToken_result result = new RenewDelegationToken_result(); + result.success = o; + try { + fcall.sendResponse(fb,result, org.apache.thrift.protocol.TMessageType.REPLY,seqid); + return; + } catch (Exception e) { + LOGGER.error("Exception writing to internal frame buffer", e); + } + fb.close(); + } + public void onError(Exception e) { + byte msgType = org.apache.thrift.protocol.TMessageType.REPLY; + org.apache.thrift.TBase msg; + RenewDelegationToken_result result = new RenewDelegationToken_result(); + { + msgType = org.apache.thrift.protocol.TMessageType.EXCEPTION; + msg = (org.apache.thrift.TBase)new org.apache.thrift.TApplicationException(org.apache.thrift.TApplicationException.INTERNAL_ERROR, e.getMessage()); + } + try { + fcall.sendResponse(fb,msg,msgType,seqid); + return; + } catch (Exception ex) { + LOGGER.error("Exception writing to internal frame buffer", ex); + } + fb.close(); + } + }; + } + + protected boolean isOneway() { + return false; + } + + public void start(I iface, RenewDelegationToken_args args, org.apache.thrift.async.AsyncMethodCallback resultHandler) throws TException { + iface.RenewDelegationToken(args.req,resultHandler); + } + } + + } + + public static class OpenSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenSession_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenSession_argsTupleSchemeFactory()); + } + + private TOpenSessionReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_args.class, metaDataMap); + } + + public OpenSession_args() { + } + + public OpenSession_args( + TOpenSessionReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public OpenSession_args(OpenSession_args other) { + if (other.isSetReq()) { + this.req = new TOpenSessionReq(other.req); + } + } + + public OpenSession_args deepCopy() { + return new OpenSession_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TOpenSessionReq getReq() { + return this.req; + } + + public void setReq(TOpenSessionReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TOpenSessionReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenSession_args) + return this.equals((OpenSession_args)that); + return false; + } + + public boolean equals(OpenSession_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(OpenSession_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenSession_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenSession_argsStandardSchemeFactory implements SchemeFactory { + public OpenSession_argsStandardScheme getScheme() { + return new OpenSession_argsStandardScheme(); + } + } + + private static class OpenSession_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TOpenSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenSession_argsTupleSchemeFactory implements SchemeFactory { + public OpenSession_argsTupleScheme getScheme() { + return new OpenSession_argsTupleScheme(); + } + } + + private static class OpenSession_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TOpenSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class OpenSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("OpenSession_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new OpenSession_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new OpenSession_resultTupleSchemeFactory()); + } + + private TOpenSessionResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOpenSessionResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(OpenSession_result.class, metaDataMap); + } + + public OpenSession_result() { + } + + public OpenSession_result( + TOpenSessionResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public OpenSession_result(OpenSession_result other) { + if (other.isSetSuccess()) { + this.success = new TOpenSessionResp(other.success); + } + } + + public OpenSession_result deepCopy() { + return new OpenSession_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TOpenSessionResp getSuccess() { + return this.success; + } + + public void setSuccess(TOpenSessionResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TOpenSessionResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof OpenSession_result) + return this.equals((OpenSession_result)that); + return false; + } + + public boolean equals(OpenSession_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(OpenSession_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("OpenSession_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class OpenSession_resultStandardSchemeFactory implements SchemeFactory { + public OpenSession_resultStandardScheme getScheme() { + return new OpenSession_resultStandardScheme(); + } + } + + private static class OpenSession_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, OpenSession_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TOpenSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, OpenSession_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class OpenSession_resultTupleSchemeFactory implements SchemeFactory { + public OpenSession_resultTupleScheme getScheme() { + return new OpenSession_resultTupleScheme(); + } + } + + private static class OpenSession_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, OpenSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TOpenSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CloseSession_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseSession_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseSession_argsTupleSchemeFactory()); + } + + private TCloseSessionReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_args.class, metaDataMap); + } + + public CloseSession_args() { + } + + public CloseSession_args( + TCloseSessionReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CloseSession_args(CloseSession_args other) { + if (other.isSetReq()) { + this.req = new TCloseSessionReq(other.req); + } + } + + public CloseSession_args deepCopy() { + return new CloseSession_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCloseSessionReq getReq() { + return this.req; + } + + public void setReq(TCloseSessionReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCloseSessionReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseSession_args) + return this.equals((CloseSession_args)that); + return false; + } + + public boolean equals(CloseSession_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(CloseSession_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseSession_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseSession_argsStandardSchemeFactory implements SchemeFactory { + public CloseSession_argsStandardScheme getScheme() { + return new CloseSession_argsStandardScheme(); + } + } + + private static class CloseSession_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCloseSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseSession_argsTupleSchemeFactory implements SchemeFactory { + public CloseSession_argsTupleScheme getScheme() { + return new CloseSession_argsTupleScheme(); + } + } + + private static class CloseSession_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCloseSessionReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CloseSession_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseSession_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseSession_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseSession_resultTupleSchemeFactory()); + } + + private TCloseSessionResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseSessionResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseSession_result.class, metaDataMap); + } + + public CloseSession_result() { + } + + public CloseSession_result( + TCloseSessionResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CloseSession_result(CloseSession_result other) { + if (other.isSetSuccess()) { + this.success = new TCloseSessionResp(other.success); + } + } + + public CloseSession_result deepCopy() { + return new CloseSession_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCloseSessionResp getSuccess() { + return this.success; + } + + public void setSuccess(TCloseSessionResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCloseSessionResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseSession_result) + return this.equals((CloseSession_result)that); + return false; + } + + public boolean equals(CloseSession_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(CloseSession_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseSession_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseSession_resultStandardSchemeFactory implements SchemeFactory { + public CloseSession_resultStandardScheme getScheme() { + return new CloseSession_resultStandardScheme(); + } + } + + private static class CloseSession_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseSession_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCloseSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseSession_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseSession_resultTupleSchemeFactory implements SchemeFactory { + public CloseSession_resultTupleScheme getScheme() { + return new CloseSession_resultTupleScheme(); + } + } + + private static class CloseSession_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseSession_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCloseSessionResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetInfo_argsTupleSchemeFactory()); + } + + private TGetInfoReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_args.class, metaDataMap); + } + + public GetInfo_args() { + } + + public GetInfo_args( + TGetInfoReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetInfo_args(GetInfo_args other) { + if (other.isSetReq()) { + this.req = new TGetInfoReq(other.req); + } + } + + public GetInfo_args deepCopy() { + return new GetInfo_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetInfoReq getReq() { + return this.req; + } + + public void setReq(TGetInfoReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetInfoReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetInfo_args) + return this.equals((GetInfo_args)that); + return false; + } + + public boolean equals(GetInfo_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetInfo_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetInfo_argsStandardSchemeFactory implements SchemeFactory { + public GetInfo_argsStandardScheme getScheme() { + return new GetInfo_argsStandardScheme(); + } + } + + private static class GetInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetInfo_argsTupleSchemeFactory implements SchemeFactory { + public GetInfo_argsTupleScheme getScheme() { + return new GetInfo_argsTupleScheme(); + } + } + + private static class GetInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetInfo_resultTupleSchemeFactory()); + } + + private TGetInfoResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetInfo_result.class, metaDataMap); + } + + public GetInfo_result() { + } + + public GetInfo_result( + TGetInfoResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetInfo_result(GetInfo_result other) { + if (other.isSetSuccess()) { + this.success = new TGetInfoResp(other.success); + } + } + + public GetInfo_result deepCopy() { + return new GetInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetInfoResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetInfoResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetInfoResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetInfo_result) + return this.equals((GetInfo_result)that); + return false; + } + + public boolean equals(GetInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetInfo_resultStandardSchemeFactory implements SchemeFactory { + public GetInfo_resultStandardScheme getScheme() { + return new GetInfo_resultStandardScheme(); + } + } + + private static class GetInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetInfo_resultTupleSchemeFactory implements SchemeFactory { + public GetInfo_resultTupleScheme getScheme() { + return new GetInfo_resultTupleScheme(); + } + } + + private static class GetInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class ExecuteStatement_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecuteStatement_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecuteStatement_argsTupleSchemeFactory()); + } + + private TExecuteStatementReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_args.class, metaDataMap); + } + + public ExecuteStatement_args() { + } + + public ExecuteStatement_args( + TExecuteStatementReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public ExecuteStatement_args(ExecuteStatement_args other) { + if (other.isSetReq()) { + this.req = new TExecuteStatementReq(other.req); + } + } + + public ExecuteStatement_args deepCopy() { + return new ExecuteStatement_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TExecuteStatementReq getReq() { + return this.req; + } + + public void setReq(TExecuteStatementReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TExecuteStatementReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecuteStatement_args) + return this.equals((ExecuteStatement_args)that); + return false; + } + + public boolean equals(ExecuteStatement_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(ExecuteStatement_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ExecuteStatement_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ExecuteStatement_argsStandardSchemeFactory implements SchemeFactory { + public ExecuteStatement_argsStandardScheme getScheme() { + return new ExecuteStatement_argsStandardScheme(); + } + } + + private static class ExecuteStatement_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TExecuteStatementReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecuteStatement_argsTupleSchemeFactory implements SchemeFactory { + public ExecuteStatement_argsTupleScheme getScheme() { + return new ExecuteStatement_argsTupleScheme(); + } + } + + private static class ExecuteStatement_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TExecuteStatementReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class ExecuteStatement_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("ExecuteStatement_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new ExecuteStatement_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new ExecuteStatement_resultTupleSchemeFactory()); + } + + private TExecuteStatementResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TExecuteStatementResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(ExecuteStatement_result.class, metaDataMap); + } + + public ExecuteStatement_result() { + } + + public ExecuteStatement_result( + TExecuteStatementResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public ExecuteStatement_result(ExecuteStatement_result other) { + if (other.isSetSuccess()) { + this.success = new TExecuteStatementResp(other.success); + } + } + + public ExecuteStatement_result deepCopy() { + return new ExecuteStatement_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TExecuteStatementResp getSuccess() { + return this.success; + } + + public void setSuccess(TExecuteStatementResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TExecuteStatementResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof ExecuteStatement_result) + return this.equals((ExecuteStatement_result)that); + return false; + } + + public boolean equals(ExecuteStatement_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(ExecuteStatement_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ExecuteStatement_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class ExecuteStatement_resultStandardSchemeFactory implements SchemeFactory { + public ExecuteStatement_resultStandardScheme getScheme() { + return new ExecuteStatement_resultStandardScheme(); + } + } + + private static class ExecuteStatement_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TExecuteStatementResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class ExecuteStatement_resultTupleSchemeFactory implements SchemeFactory { + public ExecuteStatement_resultTupleScheme getScheme() { + return new ExecuteStatement_resultTupleScheme(); + } + } + + private static class ExecuteStatement_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, ExecuteStatement_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TExecuteStatementResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTypeInfo_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTypeInfo_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTypeInfo_argsTupleSchemeFactory()); + } + + private TGetTypeInfoReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_args.class, metaDataMap); + } + + public GetTypeInfo_args() { + } + + public GetTypeInfo_args( + TGetTypeInfoReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTypeInfo_args(GetTypeInfo_args other) { + if (other.isSetReq()) { + this.req = new TGetTypeInfoReq(other.req); + } + } + + public GetTypeInfo_args deepCopy() { + return new GetTypeInfo_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTypeInfoReq getReq() { + return this.req; + } + + public void setReq(TGetTypeInfoReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTypeInfoReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTypeInfo_args) + return this.equals((GetTypeInfo_args)that); + return false; + } + + public boolean equals(GetTypeInfo_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTypeInfo_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTypeInfo_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTypeInfo_argsStandardSchemeFactory implements SchemeFactory { + public GetTypeInfo_argsStandardScheme getScheme() { + return new GetTypeInfo_argsStandardScheme(); + } + } + + private static class GetTypeInfo_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTypeInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTypeInfo_argsTupleSchemeFactory implements SchemeFactory { + public GetTypeInfo_argsTupleScheme getScheme() { + return new GetTypeInfo_argsTupleScheme(); + } + } + + private static class GetTypeInfo_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTypeInfoReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTypeInfo_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTypeInfo_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTypeInfo_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTypeInfo_resultTupleSchemeFactory()); + } + + private TGetTypeInfoResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTypeInfoResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTypeInfo_result.class, metaDataMap); + } + + public GetTypeInfo_result() { + } + + public GetTypeInfo_result( + TGetTypeInfoResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTypeInfo_result(GetTypeInfo_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTypeInfoResp(other.success); + } + } + + public GetTypeInfo_result deepCopy() { + return new GetTypeInfo_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTypeInfoResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTypeInfoResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTypeInfoResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTypeInfo_result) + return this.equals((GetTypeInfo_result)that); + return false; + } + + public boolean equals(GetTypeInfo_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTypeInfo_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTypeInfo_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTypeInfo_resultStandardSchemeFactory implements SchemeFactory { + public GetTypeInfo_resultStandardScheme getScheme() { + return new GetTypeInfo_resultStandardScheme(); + } + } + + private static class GetTypeInfo_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTypeInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTypeInfo_resultTupleSchemeFactory implements SchemeFactory { + public GetTypeInfo_resultTupleScheme getScheme() { + return new GetTypeInfo_resultTupleScheme(); + } + } + + private static class GetTypeInfo_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTypeInfo_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTypeInfoResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetCatalogs_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCatalogs_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCatalogs_argsTupleSchemeFactory()); + } + + private TGetCatalogsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_args.class, metaDataMap); + } + + public GetCatalogs_args() { + } + + public GetCatalogs_args( + TGetCatalogsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetCatalogs_args(GetCatalogs_args other) { + if (other.isSetReq()) { + this.req = new TGetCatalogsReq(other.req); + } + } + + public GetCatalogs_args deepCopy() { + return new GetCatalogs_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetCatalogsReq getReq() { + return this.req; + } + + public void setReq(TGetCatalogsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetCatalogsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCatalogs_args) + return this.equals((GetCatalogs_args)that); + return false; + } + + public boolean equals(GetCatalogs_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetCatalogs_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCatalogs_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCatalogs_argsStandardSchemeFactory implements SchemeFactory { + public GetCatalogs_argsStandardScheme getScheme() { + return new GetCatalogs_argsStandardScheme(); + } + } + + private static class GetCatalogs_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetCatalogsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCatalogs_argsTupleSchemeFactory implements SchemeFactory { + public GetCatalogs_argsTupleScheme getScheme() { + return new GetCatalogs_argsTupleScheme(); + } + } + + private static class GetCatalogs_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetCatalogsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetCatalogs_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCatalogs_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCatalogs_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCatalogs_resultTupleSchemeFactory()); + } + + private TGetCatalogsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCatalogsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCatalogs_result.class, metaDataMap); + } + + public GetCatalogs_result() { + } + + public GetCatalogs_result( + TGetCatalogsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetCatalogs_result(GetCatalogs_result other) { + if (other.isSetSuccess()) { + this.success = new TGetCatalogsResp(other.success); + } + } + + public GetCatalogs_result deepCopy() { + return new GetCatalogs_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetCatalogsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetCatalogsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetCatalogsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCatalogs_result) + return this.equals((GetCatalogs_result)that); + return false; + } + + public boolean equals(GetCatalogs_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetCatalogs_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCatalogs_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCatalogs_resultStandardSchemeFactory implements SchemeFactory { + public GetCatalogs_resultStandardScheme getScheme() { + return new GetCatalogs_resultStandardScheme(); + } + } + + private static class GetCatalogs_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCatalogs_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetCatalogsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCatalogs_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCatalogs_resultTupleSchemeFactory implements SchemeFactory { + public GetCatalogs_resultTupleScheme getScheme() { + return new GetCatalogs_resultTupleScheme(); + } + } + + private static class GetCatalogs_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCatalogs_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetCatalogsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetSchemas_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetSchemas_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetSchemas_argsTupleSchemeFactory()); + } + + private TGetSchemasReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_args.class, metaDataMap); + } + + public GetSchemas_args() { + } + + public GetSchemas_args( + TGetSchemasReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetSchemas_args(GetSchemas_args other) { + if (other.isSetReq()) { + this.req = new TGetSchemasReq(other.req); + } + } + + public GetSchemas_args deepCopy() { + return new GetSchemas_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetSchemasReq getReq() { + return this.req; + } + + public void setReq(TGetSchemasReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetSchemasReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetSchemas_args) + return this.equals((GetSchemas_args)that); + return false; + } + + public boolean equals(GetSchemas_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetSchemas_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetSchemas_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetSchemas_argsStandardSchemeFactory implements SchemeFactory { + public GetSchemas_argsStandardScheme getScheme() { + return new GetSchemas_argsStandardScheme(); + } + } + + private static class GetSchemas_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetSchemasReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetSchemas_argsTupleSchemeFactory implements SchemeFactory { + public GetSchemas_argsTupleScheme getScheme() { + return new GetSchemas_argsTupleScheme(); + } + } + + private static class GetSchemas_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetSchemasReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetSchemas_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetSchemas_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetSchemas_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetSchemas_resultTupleSchemeFactory()); + } + + private TGetSchemasResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetSchemasResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetSchemas_result.class, metaDataMap); + } + + public GetSchemas_result() { + } + + public GetSchemas_result( + TGetSchemasResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetSchemas_result(GetSchemas_result other) { + if (other.isSetSuccess()) { + this.success = new TGetSchemasResp(other.success); + } + } + + public GetSchemas_result deepCopy() { + return new GetSchemas_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetSchemasResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetSchemasResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetSchemasResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetSchemas_result) + return this.equals((GetSchemas_result)that); + return false; + } + + public boolean equals(GetSchemas_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetSchemas_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetSchemas_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetSchemas_resultStandardSchemeFactory implements SchemeFactory { + public GetSchemas_resultStandardScheme getScheme() { + return new GetSchemas_resultStandardScheme(); + } + } + + private static class GetSchemas_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetSchemas_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetSchemasResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetSchemas_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetSchemas_resultTupleSchemeFactory implements SchemeFactory { + public GetSchemas_resultTupleScheme getScheme() { + return new GetSchemas_resultTupleScheme(); + } + } + + private static class GetSchemas_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetSchemas_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetSchemasResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTables_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTables_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTables_argsTupleSchemeFactory()); + } + + private TGetTablesReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_args.class, metaDataMap); + } + + public GetTables_args() { + } + + public GetTables_args( + TGetTablesReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTables_args(GetTables_args other) { + if (other.isSetReq()) { + this.req = new TGetTablesReq(other.req); + } + } + + public GetTables_args deepCopy() { + return new GetTables_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTablesReq getReq() { + return this.req; + } + + public void setReq(TGetTablesReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTablesReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTables_args) + return this.equals((GetTables_args)that); + return false; + } + + public boolean equals(GetTables_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTables_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTables_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTables_argsStandardSchemeFactory implements SchemeFactory { + public GetTables_argsStandardScheme getScheme() { + return new GetTables_argsStandardScheme(); + } + } + + private static class GetTables_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTablesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTables_argsTupleSchemeFactory implements SchemeFactory { + public GetTables_argsTupleScheme getScheme() { + return new GetTables_argsTupleScheme(); + } + } + + private static class GetTables_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTablesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTables_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTables_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTables_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTables_resultTupleSchemeFactory()); + } + + private TGetTablesResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTablesResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTables_result.class, metaDataMap); + } + + public GetTables_result() { + } + + public GetTables_result( + TGetTablesResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTables_result(GetTables_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTablesResp(other.success); + } + } + + public GetTables_result deepCopy() { + return new GetTables_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTablesResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTablesResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTablesResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTables_result) + return this.equals((GetTables_result)that); + return false; + } + + public boolean equals(GetTables_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTables_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTables_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTables_resultStandardSchemeFactory implements SchemeFactory { + public GetTables_resultStandardScheme getScheme() { + return new GetTables_resultStandardScheme(); + } + } + + private static class GetTables_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTables_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTablesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTables_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTables_resultTupleSchemeFactory implements SchemeFactory { + public GetTables_resultTupleScheme getScheme() { + return new GetTables_resultTupleScheme(); + } + } + + private static class GetTables_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTables_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTablesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetTableTypes_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTableTypes_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTableTypes_argsTupleSchemeFactory()); + } + + private TGetTableTypesReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_args.class, metaDataMap); + } + + public GetTableTypes_args() { + } + + public GetTableTypes_args( + TGetTableTypesReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetTableTypes_args(GetTableTypes_args other) { + if (other.isSetReq()) { + this.req = new TGetTableTypesReq(other.req); + } + } + + public GetTableTypes_args deepCopy() { + return new GetTableTypes_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetTableTypesReq getReq() { + return this.req; + } + + public void setReq(TGetTableTypesReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetTableTypesReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTableTypes_args) + return this.equals((GetTableTypes_args)that); + return false; + } + + public boolean equals(GetTableTypes_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTableTypes_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTableTypes_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTableTypes_argsStandardSchemeFactory implements SchemeFactory { + public GetTableTypes_argsStandardScheme getScheme() { + return new GetTableTypes_argsStandardScheme(); + } + } + + private static class GetTableTypes_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetTableTypesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTableTypes_argsTupleSchemeFactory implements SchemeFactory { + public GetTableTypes_argsTupleScheme getScheme() { + return new GetTableTypes_argsTupleScheme(); + } + } + + private static class GetTableTypes_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetTableTypesReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetTableTypes_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetTableTypes_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetTableTypes_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetTableTypes_resultTupleSchemeFactory()); + } + + private TGetTableTypesResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetTableTypesResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetTableTypes_result.class, metaDataMap); + } + + public GetTableTypes_result() { + } + + public GetTableTypes_result( + TGetTableTypesResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetTableTypes_result(GetTableTypes_result other) { + if (other.isSetSuccess()) { + this.success = new TGetTableTypesResp(other.success); + } + } + + public GetTableTypes_result deepCopy() { + return new GetTableTypes_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetTableTypesResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetTableTypesResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetTableTypesResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetTableTypes_result) + return this.equals((GetTableTypes_result)that); + return false; + } + + public boolean equals(GetTableTypes_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetTableTypes_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetTableTypes_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetTableTypes_resultStandardSchemeFactory implements SchemeFactory { + public GetTableTypes_resultStandardScheme getScheme() { + return new GetTableTypes_resultStandardScheme(); + } + } + + private static class GetTableTypes_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetTableTypes_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetTableTypesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetTableTypes_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetTableTypes_resultTupleSchemeFactory implements SchemeFactory { + public GetTableTypes_resultTupleScheme getScheme() { + return new GetTableTypes_resultTupleScheme(); + } + } + + private static class GetTableTypes_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetTableTypes_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetTableTypesResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetColumns_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetColumns_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetColumns_argsTupleSchemeFactory()); + } + + private TGetColumnsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_args.class, metaDataMap); + } + + public GetColumns_args() { + } + + public GetColumns_args( + TGetColumnsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetColumns_args(GetColumns_args other) { + if (other.isSetReq()) { + this.req = new TGetColumnsReq(other.req); + } + } + + public GetColumns_args deepCopy() { + return new GetColumns_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetColumnsReq getReq() { + return this.req; + } + + public void setReq(TGetColumnsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetColumnsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetColumns_args) + return this.equals((GetColumns_args)that); + return false; + } + + public boolean equals(GetColumns_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetColumns_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetColumns_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetColumns_argsStandardSchemeFactory implements SchemeFactory { + public GetColumns_argsStandardScheme getScheme() { + return new GetColumns_argsStandardScheme(); + } + } + + private static class GetColumns_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetColumnsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetColumns_argsTupleSchemeFactory implements SchemeFactory { + public GetColumns_argsTupleScheme getScheme() { + return new GetColumns_argsTupleScheme(); + } + } + + private static class GetColumns_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetColumnsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetColumns_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetColumns_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetColumns_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetColumns_resultTupleSchemeFactory()); + } + + private TGetColumnsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetColumnsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetColumns_result.class, metaDataMap); + } + + public GetColumns_result() { + } + + public GetColumns_result( + TGetColumnsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetColumns_result(GetColumns_result other) { + if (other.isSetSuccess()) { + this.success = new TGetColumnsResp(other.success); + } + } + + public GetColumns_result deepCopy() { + return new GetColumns_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetColumnsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetColumnsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetColumnsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetColumns_result) + return this.equals((GetColumns_result)that); + return false; + } + + public boolean equals(GetColumns_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetColumns_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetColumns_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetColumns_resultStandardSchemeFactory implements SchemeFactory { + public GetColumns_resultStandardScheme getScheme() { + return new GetColumns_resultStandardScheme(); + } + } + + private static class GetColumns_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetColumns_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetColumnsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetColumns_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetColumns_resultTupleSchemeFactory implements SchemeFactory { + public GetColumns_resultTupleScheme getScheme() { + return new GetColumns_resultTupleScheme(); + } + } + + private static class GetColumns_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetColumns_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetColumnsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetFunctions_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetFunctions_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetFunctions_argsTupleSchemeFactory()); + } + + private TGetFunctionsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_args.class, metaDataMap); + } + + public GetFunctions_args() { + } + + public GetFunctions_args( + TGetFunctionsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetFunctions_args(GetFunctions_args other) { + if (other.isSetReq()) { + this.req = new TGetFunctionsReq(other.req); + } + } + + public GetFunctions_args deepCopy() { + return new GetFunctions_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetFunctionsReq getReq() { + return this.req; + } + + public void setReq(TGetFunctionsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetFunctionsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetFunctions_args) + return this.equals((GetFunctions_args)that); + return false; + } + + public boolean equals(GetFunctions_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetFunctions_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetFunctions_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetFunctions_argsStandardSchemeFactory implements SchemeFactory { + public GetFunctions_argsStandardScheme getScheme() { + return new GetFunctions_argsStandardScheme(); + } + } + + private static class GetFunctions_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetFunctionsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetFunctions_argsTupleSchemeFactory implements SchemeFactory { + public GetFunctions_argsTupleScheme getScheme() { + return new GetFunctions_argsTupleScheme(); + } + } + + private static class GetFunctions_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetFunctionsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetFunctions_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetFunctions_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetFunctions_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetFunctions_resultTupleSchemeFactory()); + } + + private TGetFunctionsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetFunctionsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetFunctions_result.class, metaDataMap); + } + + public GetFunctions_result() { + } + + public GetFunctions_result( + TGetFunctionsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetFunctions_result(GetFunctions_result other) { + if (other.isSetSuccess()) { + this.success = new TGetFunctionsResp(other.success); + } + } + + public GetFunctions_result deepCopy() { + return new GetFunctions_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetFunctionsResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetFunctionsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetFunctionsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetFunctions_result) + return this.equals((GetFunctions_result)that); + return false; + } + + public boolean equals(GetFunctions_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetFunctions_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetFunctions_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetFunctions_resultStandardSchemeFactory implements SchemeFactory { + public GetFunctions_resultStandardScheme getScheme() { + return new GetFunctions_resultStandardScheme(); + } + } + + private static class GetFunctions_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetFunctions_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetFunctionsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetFunctions_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetFunctions_resultTupleSchemeFactory implements SchemeFactory { + public GetFunctions_resultTupleScheme getScheme() { + return new GetFunctions_resultTupleScheme(); + } + } + + private static class GetFunctions_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetFunctions_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetFunctionsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetPrimaryKeys_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrimaryKeys_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetPrimaryKeys_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetPrimaryKeys_argsTupleSchemeFactory()); + } + + private TGetPrimaryKeysReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetPrimaryKeysReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPrimaryKeys_args.class, metaDataMap); + } + + public GetPrimaryKeys_args() { + } + + public GetPrimaryKeys_args( + TGetPrimaryKeysReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetPrimaryKeys_args(GetPrimaryKeys_args other) { + if (other.isSetReq()) { + this.req = new TGetPrimaryKeysReq(other.req); + } + } + + public GetPrimaryKeys_args deepCopy() { + return new GetPrimaryKeys_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetPrimaryKeysReq getReq() { + return this.req; + } + + public void setReq(TGetPrimaryKeysReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetPrimaryKeysReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetPrimaryKeys_args) + return this.equals((GetPrimaryKeys_args)that); + return false; + } + + public boolean equals(GetPrimaryKeys_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetPrimaryKeys_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetPrimaryKeys_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetPrimaryKeys_argsStandardSchemeFactory implements SchemeFactory { + public GetPrimaryKeys_argsStandardScheme getScheme() { + return new GetPrimaryKeys_argsStandardScheme(); + } + } + + private static class GetPrimaryKeys_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetPrimaryKeys_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetPrimaryKeysReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetPrimaryKeys_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetPrimaryKeys_argsTupleSchemeFactory implements SchemeFactory { + public GetPrimaryKeys_argsTupleScheme getScheme() { + return new GetPrimaryKeys_argsTupleScheme(); + } + } + + private static class GetPrimaryKeys_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetPrimaryKeys_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetPrimaryKeys_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetPrimaryKeysReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetPrimaryKeys_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetPrimaryKeys_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetPrimaryKeys_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetPrimaryKeys_resultTupleSchemeFactory()); + } + + private TGetPrimaryKeysResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetPrimaryKeysResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetPrimaryKeys_result.class, metaDataMap); + } + + public GetPrimaryKeys_result() { + } + + public GetPrimaryKeys_result( + TGetPrimaryKeysResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetPrimaryKeys_result(GetPrimaryKeys_result other) { + if (other.isSetSuccess()) { + this.success = new TGetPrimaryKeysResp(other.success); + } + } + + public GetPrimaryKeys_result deepCopy() { + return new GetPrimaryKeys_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetPrimaryKeysResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetPrimaryKeysResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetPrimaryKeysResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetPrimaryKeys_result) + return this.equals((GetPrimaryKeys_result)that); + return false; + } + + public boolean equals(GetPrimaryKeys_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetPrimaryKeys_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetPrimaryKeys_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetPrimaryKeys_resultStandardSchemeFactory implements SchemeFactory { + public GetPrimaryKeys_resultStandardScheme getScheme() { + return new GetPrimaryKeys_resultStandardScheme(); + } + } + + private static class GetPrimaryKeys_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetPrimaryKeys_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetPrimaryKeysResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetPrimaryKeys_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetPrimaryKeys_resultTupleSchemeFactory implements SchemeFactory { + public GetPrimaryKeys_resultTupleScheme getScheme() { + return new GetPrimaryKeys_resultTupleScheme(); + } + } + + private static class GetPrimaryKeys_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetPrimaryKeys_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetPrimaryKeys_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetPrimaryKeysResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetCrossReference_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCrossReference_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCrossReference_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCrossReference_argsTupleSchemeFactory()); + } + + private TGetCrossReferenceReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCrossReferenceReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCrossReference_args.class, metaDataMap); + } + + public GetCrossReference_args() { + } + + public GetCrossReference_args( + TGetCrossReferenceReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetCrossReference_args(GetCrossReference_args other) { + if (other.isSetReq()) { + this.req = new TGetCrossReferenceReq(other.req); + } + } + + public GetCrossReference_args deepCopy() { + return new GetCrossReference_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetCrossReferenceReq getReq() { + return this.req; + } + + public void setReq(TGetCrossReferenceReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetCrossReferenceReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCrossReference_args) + return this.equals((GetCrossReference_args)that); + return false; + } + + public boolean equals(GetCrossReference_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetCrossReference_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCrossReference_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCrossReference_argsStandardSchemeFactory implements SchemeFactory { + public GetCrossReference_argsStandardScheme getScheme() { + return new GetCrossReference_argsStandardScheme(); + } + } + + private static class GetCrossReference_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCrossReference_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetCrossReferenceReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCrossReference_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCrossReference_argsTupleSchemeFactory implements SchemeFactory { + public GetCrossReference_argsTupleScheme getScheme() { + return new GetCrossReference_argsTupleScheme(); + } + } + + private static class GetCrossReference_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCrossReference_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCrossReference_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetCrossReferenceReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetCrossReference_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetCrossReference_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetCrossReference_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetCrossReference_resultTupleSchemeFactory()); + } + + private TGetCrossReferenceResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetCrossReferenceResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetCrossReference_result.class, metaDataMap); + } + + public GetCrossReference_result() { + } + + public GetCrossReference_result( + TGetCrossReferenceResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetCrossReference_result(GetCrossReference_result other) { + if (other.isSetSuccess()) { + this.success = new TGetCrossReferenceResp(other.success); + } + } + + public GetCrossReference_result deepCopy() { + return new GetCrossReference_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetCrossReferenceResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetCrossReferenceResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetCrossReferenceResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetCrossReference_result) + return this.equals((GetCrossReference_result)that); + return false; + } + + public boolean equals(GetCrossReference_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetCrossReference_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetCrossReference_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetCrossReference_resultStandardSchemeFactory implements SchemeFactory { + public GetCrossReference_resultStandardScheme getScheme() { + return new GetCrossReference_resultStandardScheme(); + } + } + + private static class GetCrossReference_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetCrossReference_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetCrossReferenceResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetCrossReference_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetCrossReference_resultTupleSchemeFactory implements SchemeFactory { + public GetCrossReference_resultTupleScheme getScheme() { + return new GetCrossReference_resultTupleScheme(); + } + } + + private static class GetCrossReference_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetCrossReference_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetCrossReference_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetCrossReferenceResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetOperationStatus_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOperationStatus_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOperationStatus_argsTupleSchemeFactory()); + } + + private TGetOperationStatusReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_args.class, metaDataMap); + } + + public GetOperationStatus_args() { + } + + public GetOperationStatus_args( + TGetOperationStatusReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetOperationStatus_args(GetOperationStatus_args other) { + if (other.isSetReq()) { + this.req = new TGetOperationStatusReq(other.req); + } + } + + public GetOperationStatus_args deepCopy() { + return new GetOperationStatus_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetOperationStatusReq getReq() { + return this.req; + } + + public void setReq(TGetOperationStatusReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetOperationStatusReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOperationStatus_args) + return this.equals((GetOperationStatus_args)that); + return false; + } + + public boolean equals(GetOperationStatus_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetOperationStatus_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOperationStatus_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOperationStatus_argsStandardSchemeFactory implements SchemeFactory { + public GetOperationStatus_argsStandardScheme getScheme() { + return new GetOperationStatus_argsStandardScheme(); + } + } + + private static class GetOperationStatus_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetOperationStatusReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOperationStatus_argsTupleSchemeFactory implements SchemeFactory { + public GetOperationStatus_argsTupleScheme getScheme() { + return new GetOperationStatus_argsTupleScheme(); + } + } + + private static class GetOperationStatus_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetOperationStatusReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetOperationStatus_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetOperationStatus_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetOperationStatus_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetOperationStatus_resultTupleSchemeFactory()); + } + + private TGetOperationStatusResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetOperationStatusResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetOperationStatus_result.class, metaDataMap); + } + + public GetOperationStatus_result() { + } + + public GetOperationStatus_result( + TGetOperationStatusResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetOperationStatus_result(GetOperationStatus_result other) { + if (other.isSetSuccess()) { + this.success = new TGetOperationStatusResp(other.success); + } + } + + public GetOperationStatus_result deepCopy() { + return new GetOperationStatus_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetOperationStatusResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetOperationStatusResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetOperationStatusResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetOperationStatus_result) + return this.equals((GetOperationStatus_result)that); + return false; + } + + public boolean equals(GetOperationStatus_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetOperationStatus_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetOperationStatus_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetOperationStatus_resultStandardSchemeFactory implements SchemeFactory { + public GetOperationStatus_resultStandardScheme getScheme() { + return new GetOperationStatus_resultStandardScheme(); + } + } + + private static class GetOperationStatus_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetOperationStatusResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetOperationStatus_resultTupleSchemeFactory implements SchemeFactory { + public GetOperationStatus_resultTupleScheme getScheme() { + return new GetOperationStatus_resultTupleScheme(); + } + } + + private static class GetOperationStatus_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetOperationStatus_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetOperationStatusResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CancelOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelOperation_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelOperation_argsTupleSchemeFactory()); + } + + private TCancelOperationReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_args.class, metaDataMap); + } + + public CancelOperation_args() { + } + + public CancelOperation_args( + TCancelOperationReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CancelOperation_args(CancelOperation_args other) { + if (other.isSetReq()) { + this.req = new TCancelOperationReq(other.req); + } + } + + public CancelOperation_args deepCopy() { + return new CancelOperation_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCancelOperationReq getReq() { + return this.req; + } + + public void setReq(TCancelOperationReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCancelOperationReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelOperation_args) + return this.equals((CancelOperation_args)that); + return false; + } + + public boolean equals(CancelOperation_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(CancelOperation_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelOperation_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelOperation_argsStandardSchemeFactory implements SchemeFactory { + public CancelOperation_argsStandardScheme getScheme() { + return new CancelOperation_argsStandardScheme(); + } + } + + private static class CancelOperation_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCancelOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelOperation_argsTupleSchemeFactory implements SchemeFactory { + public CancelOperation_argsTupleScheme getScheme() { + return new CancelOperation_argsTupleScheme(); + } + } + + private static class CancelOperation_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCancelOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CancelOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelOperation_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelOperation_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelOperation_resultTupleSchemeFactory()); + } + + private TCancelOperationResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelOperationResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelOperation_result.class, metaDataMap); + } + + public CancelOperation_result() { + } + + public CancelOperation_result( + TCancelOperationResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CancelOperation_result(CancelOperation_result other) { + if (other.isSetSuccess()) { + this.success = new TCancelOperationResp(other.success); + } + } + + public CancelOperation_result deepCopy() { + return new CancelOperation_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCancelOperationResp getSuccess() { + return this.success; + } + + public void setSuccess(TCancelOperationResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCancelOperationResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelOperation_result) + return this.equals((CancelOperation_result)that); + return false; + } + + public boolean equals(CancelOperation_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(CancelOperation_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelOperation_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelOperation_resultStandardSchemeFactory implements SchemeFactory { + public CancelOperation_resultStandardScheme getScheme() { + return new CancelOperation_resultStandardScheme(); + } + } + + private static class CancelOperation_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelOperation_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCancelOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelOperation_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelOperation_resultTupleSchemeFactory implements SchemeFactory { + public CancelOperation_resultTupleScheme getScheme() { + return new CancelOperation_resultTupleScheme(); + } + } + + private static class CancelOperation_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCancelOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CloseOperation_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseOperation_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseOperation_argsTupleSchemeFactory()); + } + + private TCloseOperationReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_args.class, metaDataMap); + } + + public CloseOperation_args() { + } + + public CloseOperation_args( + TCloseOperationReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CloseOperation_args(CloseOperation_args other) { + if (other.isSetReq()) { + this.req = new TCloseOperationReq(other.req); + } + } + + public CloseOperation_args deepCopy() { + return new CloseOperation_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCloseOperationReq getReq() { + return this.req; + } + + public void setReq(TCloseOperationReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCloseOperationReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseOperation_args) + return this.equals((CloseOperation_args)that); + return false; + } + + public boolean equals(CloseOperation_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(CloseOperation_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseOperation_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseOperation_argsStandardSchemeFactory implements SchemeFactory { + public CloseOperation_argsStandardScheme getScheme() { + return new CloseOperation_argsStandardScheme(); + } + } + + private static class CloseOperation_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCloseOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseOperation_argsTupleSchemeFactory implements SchemeFactory { + public CloseOperation_argsTupleScheme getScheme() { + return new CloseOperation_argsTupleScheme(); + } + } + + private static class CloseOperation_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCloseOperationReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CloseOperation_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CloseOperation_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CloseOperation_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CloseOperation_resultTupleSchemeFactory()); + } + + private TCloseOperationResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCloseOperationResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CloseOperation_result.class, metaDataMap); + } + + public CloseOperation_result() { + } + + public CloseOperation_result( + TCloseOperationResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CloseOperation_result(CloseOperation_result other) { + if (other.isSetSuccess()) { + this.success = new TCloseOperationResp(other.success); + } + } + + public CloseOperation_result deepCopy() { + return new CloseOperation_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCloseOperationResp getSuccess() { + return this.success; + } + + public void setSuccess(TCloseOperationResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCloseOperationResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CloseOperation_result) + return this.equals((CloseOperation_result)that); + return false; + } + + public boolean equals(CloseOperation_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(CloseOperation_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CloseOperation_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CloseOperation_resultStandardSchemeFactory implements SchemeFactory { + public CloseOperation_resultStandardScheme getScheme() { + return new CloseOperation_resultStandardScheme(); + } + } + + private static class CloseOperation_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CloseOperation_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCloseOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CloseOperation_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CloseOperation_resultTupleSchemeFactory implements SchemeFactory { + public CloseOperation_resultTupleScheme getScheme() { + return new CloseOperation_resultTupleScheme(); + } + } + + private static class CloseOperation_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CloseOperation_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCloseOperationResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetResultSetMetadata_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetResultSetMetadata_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetResultSetMetadata_argsTupleSchemeFactory()); + } + + private TGetResultSetMetadataReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_args.class, metaDataMap); + } + + public GetResultSetMetadata_args() { + } + + public GetResultSetMetadata_args( + TGetResultSetMetadataReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetResultSetMetadata_args(GetResultSetMetadata_args other) { + if (other.isSetReq()) { + this.req = new TGetResultSetMetadataReq(other.req); + } + } + + public GetResultSetMetadata_args deepCopy() { + return new GetResultSetMetadata_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetResultSetMetadataReq getReq() { + return this.req; + } + + public void setReq(TGetResultSetMetadataReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetResultSetMetadataReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetResultSetMetadata_args) + return this.equals((GetResultSetMetadata_args)that); + return false; + } + + public boolean equals(GetResultSetMetadata_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetResultSetMetadata_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetResultSetMetadata_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetResultSetMetadata_argsStandardSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_argsStandardScheme getScheme() { + return new GetResultSetMetadata_argsStandardScheme(); + } + } + + private static class GetResultSetMetadata_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetResultSetMetadataReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetResultSetMetadata_argsTupleSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_argsTupleScheme getScheme() { + return new GetResultSetMetadata_argsTupleScheme(); + } + } + + private static class GetResultSetMetadata_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetResultSetMetadataReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetResultSetMetadata_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetResultSetMetadata_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetResultSetMetadata_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetResultSetMetadata_resultTupleSchemeFactory()); + } + + private TGetResultSetMetadataResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetResultSetMetadataResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetResultSetMetadata_result.class, metaDataMap); + } + + public GetResultSetMetadata_result() { + } + + public GetResultSetMetadata_result( + TGetResultSetMetadataResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetResultSetMetadata_result(GetResultSetMetadata_result other) { + if (other.isSetSuccess()) { + this.success = new TGetResultSetMetadataResp(other.success); + } + } + + public GetResultSetMetadata_result deepCopy() { + return new GetResultSetMetadata_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetResultSetMetadataResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetResultSetMetadataResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetResultSetMetadataResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetResultSetMetadata_result) + return this.equals((GetResultSetMetadata_result)that); + return false; + } + + public boolean equals(GetResultSetMetadata_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetResultSetMetadata_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetResultSetMetadata_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetResultSetMetadata_resultStandardSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_resultStandardScheme getScheme() { + return new GetResultSetMetadata_resultStandardScheme(); + } + } + + private static class GetResultSetMetadata_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetResultSetMetadataResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetResultSetMetadata_resultTupleSchemeFactory implements SchemeFactory { + public GetResultSetMetadata_resultTupleScheme getScheme() { + return new GetResultSetMetadata_resultTupleScheme(); + } + } + + private static class GetResultSetMetadata_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetResultSetMetadata_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetResultSetMetadataResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class FetchResults_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new FetchResults_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new FetchResults_argsTupleSchemeFactory()); + } + + private TFetchResultsReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_args.class, metaDataMap); + } + + public FetchResults_args() { + } + + public FetchResults_args( + TFetchResultsReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public FetchResults_args(FetchResults_args other) { + if (other.isSetReq()) { + this.req = new TFetchResultsReq(other.req); + } + } + + public FetchResults_args deepCopy() { + return new FetchResults_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TFetchResultsReq getReq() { + return this.req; + } + + public void setReq(TFetchResultsReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TFetchResultsReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof FetchResults_args) + return this.equals((FetchResults_args)that); + return false; + } + + public boolean equals(FetchResults_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(FetchResults_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("FetchResults_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class FetchResults_argsStandardSchemeFactory implements SchemeFactory { + public FetchResults_argsStandardScheme getScheme() { + return new FetchResults_argsStandardScheme(); + } + } + + private static class FetchResults_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TFetchResultsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class FetchResults_argsTupleSchemeFactory implements SchemeFactory { + public FetchResults_argsTupleScheme getScheme() { + return new FetchResults_argsTupleScheme(); + } + } + + private static class FetchResults_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TFetchResultsReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class FetchResults_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("FetchResults_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new FetchResults_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new FetchResults_resultTupleSchemeFactory()); + } + + private TFetchResultsResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TFetchResultsResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(FetchResults_result.class, metaDataMap); + } + + public FetchResults_result() { + } + + public FetchResults_result( + TFetchResultsResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public FetchResults_result(FetchResults_result other) { + if (other.isSetSuccess()) { + this.success = new TFetchResultsResp(other.success); + } + } + + public FetchResults_result deepCopy() { + return new FetchResults_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TFetchResultsResp getSuccess() { + return this.success; + } + + public void setSuccess(TFetchResultsResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TFetchResultsResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof FetchResults_result) + return this.equals((FetchResults_result)that); + return false; + } + + public boolean equals(FetchResults_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(FetchResults_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("FetchResults_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class FetchResults_resultStandardSchemeFactory implements SchemeFactory { + public FetchResults_resultStandardScheme getScheme() { + return new FetchResults_resultStandardScheme(); + } + } + + private static class FetchResults_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, FetchResults_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TFetchResultsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, FetchResults_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class FetchResults_resultTupleSchemeFactory implements SchemeFactory { + public FetchResults_resultTupleScheme getScheme() { + return new FetchResults_resultTupleScheme(); + } + } + + private static class FetchResults_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, FetchResults_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TFetchResultsResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class GetDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetDelegationToken_argsTupleSchemeFactory()); + } + + private TGetDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_args.class, metaDataMap); + } + + public GetDelegationToken_args() { + } + + public GetDelegationToken_args( + TGetDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public GetDelegationToken_args(GetDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TGetDelegationTokenReq(other.req); + } + } + + public GetDelegationToken_args deepCopy() { + return new GetDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TGetDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TGetDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TGetDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetDelegationToken_args) + return this.equals((GetDelegationToken_args)that); + return false; + } + + public boolean equals(GetDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(GetDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public GetDelegationToken_argsStandardScheme getScheme() { + return new GetDelegationToken_argsStandardScheme(); + } + } + + private static class GetDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TGetDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public GetDelegationToken_argsTupleScheme getScheme() { + return new GetDelegationToken_argsTupleScheme(); + } + } + + private static class GetDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TGetDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class GetDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("GetDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new GetDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new GetDelegationToken_resultTupleSchemeFactory()); + } + + private TGetDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(GetDelegationToken_result.class, metaDataMap); + } + + public GetDelegationToken_result() { + } + + public GetDelegationToken_result( + TGetDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public GetDelegationToken_result(GetDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TGetDelegationTokenResp(other.success); + } + } + + public GetDelegationToken_result deepCopy() { + return new GetDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TGetDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TGetDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TGetDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof GetDelegationToken_result) + return this.equals((GetDelegationToken_result)that); + return false; + } + + public boolean equals(GetDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(GetDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("GetDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class GetDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public GetDelegationToken_resultStandardScheme getScheme() { + return new GetDelegationToken_resultStandardScheme(); + } + } + + private static class GetDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TGetDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class GetDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public GetDelegationToken_resultTupleScheme getScheme() { + return new GetDelegationToken_resultTupleScheme(); + } + } + + private static class GetDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, GetDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TGetDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class CancelDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelDelegationToken_argsTupleSchemeFactory()); + } + + private TCancelDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_args.class, metaDataMap); + } + + public CancelDelegationToken_args() { + } + + public CancelDelegationToken_args( + TCancelDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public CancelDelegationToken_args(CancelDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TCancelDelegationTokenReq(other.req); + } + } + + public CancelDelegationToken_args deepCopy() { + return new CancelDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TCancelDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TCancelDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TCancelDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelDelegationToken_args) + return this.equals((CancelDelegationToken_args)that); + return false; + } + + public boolean equals(CancelDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(CancelDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public CancelDelegationToken_argsStandardScheme getScheme() { + return new CancelDelegationToken_argsStandardScheme(); + } + } + + private static class CancelDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TCancelDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public CancelDelegationToken_argsTupleScheme getScheme() { + return new CancelDelegationToken_argsTupleScheme(); + } + } + + private static class CancelDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TCancelDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class CancelDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("CancelDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new CancelDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new CancelDelegationToken_resultTupleSchemeFactory()); + } + + private TCancelDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TCancelDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(CancelDelegationToken_result.class, metaDataMap); + } + + public CancelDelegationToken_result() { + } + + public CancelDelegationToken_result( + TCancelDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public CancelDelegationToken_result(CancelDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TCancelDelegationTokenResp(other.success); + } + } + + public CancelDelegationToken_result deepCopy() { + return new CancelDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TCancelDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TCancelDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TCancelDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof CancelDelegationToken_result) + return this.equals((CancelDelegationToken_result)that); + return false; + } + + public boolean equals(CancelDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(CancelDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("CancelDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class CancelDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public CancelDelegationToken_resultStandardScheme getScheme() { + return new CancelDelegationToken_resultStandardScheme(); + } + } + + private static class CancelDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TCancelDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class CancelDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public CancelDelegationToken_resultTupleScheme getScheme() { + return new CancelDelegationToken_resultTupleScheme(); + } + } + + private static class CancelDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, CancelDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TCancelDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + + public static class RenewDelegationToken_args implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_args"); + + private static final org.apache.thrift.protocol.TField REQ_FIELD_DESC = new org.apache.thrift.protocol.TField("req", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RenewDelegationToken_argsStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RenewDelegationToken_argsTupleSchemeFactory()); + } + + private TRenewDelegationTokenReq req; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + REQ((short)1, "req"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // REQ + return REQ; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.REQ, new org.apache.thrift.meta_data.FieldMetaData("req", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenReq.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_args.class, metaDataMap); + } + + public RenewDelegationToken_args() { + } + + public RenewDelegationToken_args( + TRenewDelegationTokenReq req) + { + this(); + this.req = req; + } + + /** + * Performs a deep copy on other. + */ + public RenewDelegationToken_args(RenewDelegationToken_args other) { + if (other.isSetReq()) { + this.req = new TRenewDelegationTokenReq(other.req); + } + } + + public RenewDelegationToken_args deepCopy() { + return new RenewDelegationToken_args(this); + } + + @Override + public void clear() { + this.req = null; + } + + public TRenewDelegationTokenReq getReq() { + return this.req; + } + + public void setReq(TRenewDelegationTokenReq req) { + this.req = req; + } + + public void unsetReq() { + this.req = null; + } + + /** Returns true if field req is set (has been assigned a value) and false otherwise */ + public boolean isSetReq() { + return this.req != null; + } + + public void setReqIsSet(boolean value) { + if (!value) { + this.req = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case REQ: + if (value == null) { + unsetReq(); + } else { + setReq((TRenewDelegationTokenReq)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case REQ: + return getReq(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case REQ: + return isSetReq(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof RenewDelegationToken_args) + return this.equals((RenewDelegationToken_args)that); + return false; + } + + public boolean equals(RenewDelegationToken_args that) { + if (that == null) + return false; + + boolean this_present_req = true && this.isSetReq(); + boolean that_present_req = true && that.isSetReq(); + if (this_present_req || that_present_req) { + if (!(this_present_req && that_present_req)) + return false; + if (!this.req.equals(that.req)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_req = true && (isSetReq()); + list.add(present_req); + if (present_req) + list.add(req); + + return list.hashCode(); + } + + @Override + public int compareTo(RenewDelegationToken_args other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetReq()).compareTo(other.isSetReq()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetReq()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.req, other.req); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("RenewDelegationToken_args("); + boolean first = true; + + sb.append("req:"); + if (this.req == null) { + sb.append("null"); + } else { + sb.append(this.req); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (req != null) { + req.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class RenewDelegationToken_argsStandardSchemeFactory implements SchemeFactory { + public RenewDelegationToken_argsStandardScheme getScheme() { + return new RenewDelegationToken_argsStandardScheme(); + } + } + + private static class RenewDelegationToken_argsStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // REQ + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.req = new TRenewDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.req != null) { + oprot.writeFieldBegin(REQ_FIELD_DESC); + struct.req.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class RenewDelegationToken_argsTupleSchemeFactory implements SchemeFactory { + public RenewDelegationToken_argsTupleScheme getScheme() { + return new RenewDelegationToken_argsTupleScheme(); + } + } + + private static class RenewDelegationToken_argsTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetReq()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetReq()) { + struct.req.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_args struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.req = new TRenewDelegationTokenReq(); + struct.req.read(iprot); + struct.setReqIsSet(true); + } + } + } + + } + + public static class RenewDelegationToken_result implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("RenewDelegationToken_result"); + + private static final org.apache.thrift.protocol.TField SUCCESS_FIELD_DESC = new org.apache.thrift.protocol.TField("success", org.apache.thrift.protocol.TType.STRUCT, (short)0); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new RenewDelegationToken_resultStandardSchemeFactory()); + schemes.put(TupleScheme.class, new RenewDelegationToken_resultTupleSchemeFactory()); + } + + private TRenewDelegationTokenResp success; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SUCCESS((short)0, "success"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 0: // SUCCESS + return SUCCESS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SUCCESS, new org.apache.thrift.meta_data.FieldMetaData("success", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRenewDelegationTokenResp.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(RenewDelegationToken_result.class, metaDataMap); + } + + public RenewDelegationToken_result() { + } + + public RenewDelegationToken_result( + TRenewDelegationTokenResp success) + { + this(); + this.success = success; + } + + /** + * Performs a deep copy on other. + */ + public RenewDelegationToken_result(RenewDelegationToken_result other) { + if (other.isSetSuccess()) { + this.success = new TRenewDelegationTokenResp(other.success); + } + } + + public RenewDelegationToken_result deepCopy() { + return new RenewDelegationToken_result(this); + } + + @Override + public void clear() { + this.success = null; + } + + public TRenewDelegationTokenResp getSuccess() { + return this.success; + } + + public void setSuccess(TRenewDelegationTokenResp success) { + this.success = success; + } + + public void unsetSuccess() { + this.success = null; + } + + /** Returns true if field success is set (has been assigned a value) and false otherwise */ + public boolean isSetSuccess() { + return this.success != null; + } + + public void setSuccessIsSet(boolean value) { + if (!value) { + this.success = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SUCCESS: + if (value == null) { + unsetSuccess(); + } else { + setSuccess((TRenewDelegationTokenResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SUCCESS: + return getSuccess(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SUCCESS: + return isSetSuccess(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof RenewDelegationToken_result) + return this.equals((RenewDelegationToken_result)that); + return false; + } + + public boolean equals(RenewDelegationToken_result that) { + if (that == null) + return false; + + boolean this_present_success = true && this.isSetSuccess(); + boolean that_present_success = true && that.isSetSuccess(); + if (this_present_success || that_present_success) { + if (!(this_present_success && that_present_success)) + return false; + if (!this.success.equals(that.success)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_success = true && (isSetSuccess()); + list.add(present_success); + if (present_success) + list.add(success); + + return list.hashCode(); + } + + @Override + public int compareTo(RenewDelegationToken_result other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSuccess()).compareTo(other.isSetSuccess()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSuccess()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.success, other.success); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("RenewDelegationToken_result("); + boolean first = true; + + sb.append("success:"); + if (this.success == null) { + sb.append("null"); + } else { + sb.append(this.success); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + if (success != null) { + success.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class RenewDelegationToken_resultStandardSchemeFactory implements SchemeFactory { + public RenewDelegationToken_resultStandardScheme getScheme() { + return new RenewDelegationToken_resultStandardScheme(); + } + } + + private static class RenewDelegationToken_resultStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 0: // SUCCESS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.success = new TRenewDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.success != null) { + oprot.writeFieldBegin(SUCCESS_FIELD_DESC); + struct.success.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class RenewDelegationToken_resultTupleSchemeFactory implements SchemeFactory { + public RenewDelegationToken_resultTupleScheme getScheme() { + return new RenewDelegationToken_resultTupleScheme(); + } + } + + private static class RenewDelegationToken_resultTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetSuccess()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSuccess()) { + struct.success.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, RenewDelegationToken_result struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.success = new TRenewDelegationTokenResp(); + struct.success.read(iprot); + struct.setSuccessIsSet(true); + } + } + } + + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java new file mode 100644 index 0000000000000..930bed731ed2a --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCLIServiceConstants.java @@ -0,0 +1,106 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TCLIServiceConstants { + + public static final Set PRIMITIVE_TYPES = new HashSet(); + static { + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.BOOLEAN_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.TINYINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.SMALLINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.INT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.BIGINT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.FLOAT_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.DOUBLE_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.STRING_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMP_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.BINARY_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.DECIMAL_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.NULL_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.DATE_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.VARCHAR_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.CHAR_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE); + PRIMITIVE_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE); + } + + public static final Set COMPLEX_TYPES = new HashSet(); + static { + COMPLEX_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.ARRAY_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.MAP_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.STRUCT_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.UNION_TYPE); + COMPLEX_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.USER_DEFINED_TYPE); + } + + public static final Set COLLECTION_TYPES = new HashSet(); + static { + COLLECTION_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.ARRAY_TYPE); + COLLECTION_TYPES.add(org.apache.hive.service.rpc.thrift.TTypeId.MAP_TYPE); + } + + public static final Map TYPE_NAMES = new HashMap(); + static { + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.BOOLEAN_TYPE, "BOOLEAN"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.TINYINT_TYPE, "TINYINT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.SMALLINT_TYPE, "SMALLINT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.INT_TYPE, "INT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.BIGINT_TYPE, "BIGINT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.FLOAT_TYPE, "FLOAT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.DOUBLE_TYPE, "DOUBLE"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.STRING_TYPE, "STRING"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.TIMESTAMP_TYPE, "TIMESTAMP"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.BINARY_TYPE, "BINARY"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.ARRAY_TYPE, "ARRAY"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.MAP_TYPE, "MAP"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.STRUCT_TYPE, "STRUCT"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.UNION_TYPE, "UNIONTYPE"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.DECIMAL_TYPE, "DECIMAL"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.NULL_TYPE, "NULL"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.DATE_TYPE, "DATE"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.VARCHAR_TYPE, "VARCHAR"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.CHAR_TYPE, "CHAR"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_YEAR_MONTH_TYPE, "INTERVAL_YEAR_MONTH"); + TYPE_NAMES.put(org.apache.hive.service.rpc.thrift.TTypeId.INTERVAL_DAY_TIME_TYPE, "INTERVAL_DAY_TIME"); + } + + public static final String CHARACTER_MAXIMUM_LENGTH = "characterMaximumLength"; + + public static final String PRECISION = "precision"; + + public static final String SCALE = "scale"; + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java new file mode 100644 index 0000000000000..a7d4e7de1f60d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenReq.java @@ -0,0 +1,495 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCancelDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String delegationToken; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenReq.class, metaDataMap); + } + + public TCancelDelegationTokenReq() { + } + + public TCancelDelegationTokenReq( + TSessionHandle sessionHandle, + String delegationToken) + { + this(); + this.sessionHandle = sessionHandle; + this.delegationToken = delegationToken; + } + + /** + * Performs a deep copy on other. + */ + public TCancelDelegationTokenReq(TCancelDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TCancelDelegationTokenReq deepCopy() { + return new TCancelDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.delegationToken = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelDelegationTokenReq) + return this.equals((TCancelDelegationTokenReq)that); + return false; + } + + public boolean equals(TCancelDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_delegationToken = true && (isSetDelegationToken()); + list.add(present_delegationToken); + if (present_delegationToken) + list.add(delegationToken); + + return list.hashCode(); + } + + @Override + public int compareTo(TCancelDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(other.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, other.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetDelegationToken()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenReqStandardScheme getScheme() { + return new TCancelDelegationTokenReqStandardScheme(); + } + } + + private static class TCancelDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenReqTupleScheme getScheme() { + return new TCancelDelegationTokenReqTupleScheme(); + } + } + + private static class TCancelDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.delegationToken); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java new file mode 100644 index 0000000000000..611e92ca2af30 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelDelegationTokenResp.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCancelDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelDelegationTokenResp.class, metaDataMap); + } + + public TCancelDelegationTokenResp() { + } + + public TCancelDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCancelDelegationTokenResp(TCancelDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCancelDelegationTokenResp deepCopy() { + return new TCancelDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelDelegationTokenResp) + return this.equals((TCancelDelegationTokenResp)that); + return false; + } + + public boolean equals(TCancelDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + return list.hashCode(); + } + + @Override + public int compareTo(TCancelDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenRespStandardScheme getScheme() { + return new TCancelDelegationTokenRespStandardScheme(); + } + } + + private static class TCancelDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TCancelDelegationTokenRespTupleScheme getScheme() { + return new TCancelDelegationTokenRespTupleScheme(); + } + } + + private static class TCancelDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java new file mode 100644 index 0000000000000..4076c573fafb7 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCancelOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelOperationReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelOperationReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationReq.class, metaDataMap); + } + + public TCancelOperationReq() { + } + + public TCancelOperationReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCancelOperationReq(TCancelOperationReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TCancelOperationReq deepCopy() { + return new TCancelOperationReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelOperationReq) + return this.equals((TCancelOperationReq)that); + return false; + } + + public boolean equals(TCancelOperationReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TCancelOperationReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelOperationReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelOperationReqStandardSchemeFactory implements SchemeFactory { + public TCancelOperationReqStandardScheme getScheme() { + return new TCancelOperationReqStandardScheme(); + } + } + + private static class TCancelOperationReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelOperationReqTupleSchemeFactory implements SchemeFactory { + public TCancelOperationReqTupleScheme getScheme() { + return new TCancelOperationReqTupleScheme(); + } + } + + private static class TCancelOperationReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java new file mode 100644 index 0000000000000..7bcc765c85daa --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCancelOperationResp.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCancelOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCancelOperationResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCancelOperationRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCancelOperationRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCancelOperationResp.class, metaDataMap); + } + + public TCancelOperationResp() { + } + + public TCancelOperationResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCancelOperationResp(TCancelOperationResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCancelOperationResp deepCopy() { + return new TCancelOperationResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCancelOperationResp) + return this.equals((TCancelOperationResp)that); + return false; + } + + public boolean equals(TCancelOperationResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + return list.hashCode(); + } + + @Override + public int compareTo(TCancelOperationResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCancelOperationResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCancelOperationRespStandardSchemeFactory implements SchemeFactory { + public TCancelOperationRespStandardScheme getScheme() { + return new TCancelOperationRespStandardScheme(); + } + } + + private static class TCancelOperationRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCancelOperationResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCancelOperationResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCancelOperationRespTupleSchemeFactory implements SchemeFactory { + public TCancelOperationRespTupleScheme getScheme() { + return new TCancelOperationRespTupleScheme(); + } + } + + private static class TCancelOperationRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCancelOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java new file mode 100644 index 0000000000000..47a6b8329c05b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCloseOperationReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseOperationReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseOperationReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationReq.class, metaDataMap); + } + + public TCloseOperationReq() { + } + + public TCloseOperationReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCloseOperationReq(TCloseOperationReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TCloseOperationReq deepCopy() { + return new TCloseOperationReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseOperationReq) + return this.equals((TCloseOperationReq)that); + return false; + } + + public boolean equals(TCloseOperationReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TCloseOperationReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseOperationReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseOperationReqStandardSchemeFactory implements SchemeFactory { + public TCloseOperationReqStandardScheme getScheme() { + return new TCloseOperationReqStandardScheme(); + } + } + + private static class TCloseOperationReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseOperationReqTupleSchemeFactory implements SchemeFactory { + public TCloseOperationReqTupleScheme getScheme() { + return new TCloseOperationReqTupleScheme(); + } + } + + private static class TCloseOperationReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java new file mode 100644 index 0000000000000..0860a2b1c5bac --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseOperationResp.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCloseOperationResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseOperationResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseOperationRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseOperationRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseOperationResp.class, metaDataMap); + } + + public TCloseOperationResp() { + } + + public TCloseOperationResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCloseOperationResp(TCloseOperationResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCloseOperationResp deepCopy() { + return new TCloseOperationResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseOperationResp) + return this.equals((TCloseOperationResp)that); + return false; + } + + public boolean equals(TCloseOperationResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + return list.hashCode(); + } + + @Override + public int compareTo(TCloseOperationResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseOperationResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseOperationRespStandardSchemeFactory implements SchemeFactory { + public TCloseOperationRespStandardScheme getScheme() { + return new TCloseOperationRespStandardScheme(); + } + } + + private static class TCloseOperationRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseOperationResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseOperationResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseOperationRespTupleSchemeFactory implements SchemeFactory { + public TCloseOperationRespTupleScheme getScheme() { + return new TCloseOperationRespTupleScheme(); + } + } + + private static class TCloseOperationRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseOperationResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java new file mode 100644 index 0000000000000..43ee87f487a67 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCloseSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseSessionReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseSessionReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionReq.class, metaDataMap); + } + + public TCloseSessionReq() { + } + + public TCloseSessionReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TCloseSessionReq(TCloseSessionReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TCloseSessionReq deepCopy() { + return new TCloseSessionReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseSessionReq) + return this.equals((TCloseSessionReq)that); + return false; + } + + public boolean equals(TCloseSessionReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TCloseSessionReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseSessionReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseSessionReqStandardSchemeFactory implements SchemeFactory { + public TCloseSessionReqStandardScheme getScheme() { + return new TCloseSessionReqStandardScheme(); + } + } + + private static class TCloseSessionReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseSessionReqTupleSchemeFactory implements SchemeFactory { + public TCloseSessionReqTupleScheme getScheme() { + return new TCloseSessionReqTupleScheme(); + } + } + + private static class TCloseSessionReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java new file mode 100644 index 0000000000000..38f82ac8d3cd2 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TCloseSessionResp.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TCloseSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TCloseSessionResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TCloseSessionRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TCloseSessionRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TCloseSessionResp.class, metaDataMap); + } + + public TCloseSessionResp() { + } + + public TCloseSessionResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TCloseSessionResp(TCloseSessionResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TCloseSessionResp deepCopy() { + return new TCloseSessionResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TCloseSessionResp) + return this.equals((TCloseSessionResp)that); + return false; + } + + public boolean equals(TCloseSessionResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + return list.hashCode(); + } + + @Override + public int compareTo(TCloseSessionResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TCloseSessionResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TCloseSessionRespStandardSchemeFactory implements SchemeFactory { + public TCloseSessionRespStandardScheme getScheme() { + return new TCloseSessionRespStandardScheme(); + } + } + + private static class TCloseSessionRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TCloseSessionResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TCloseSessionResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TCloseSessionRespTupleSchemeFactory implements SchemeFactory { + public TCloseSessionRespTupleScheme getScheme() { + return new TCloseSessionRespTupleScheme(); + } + } + + private static class TCloseSessionRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TCloseSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java new file mode 100644 index 0000000000000..dd79482200961 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumn.java @@ -0,0 +1,736 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TColumn extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumn"); + private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); + private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); + private static final org.apache.thrift.protocol.TField BINARY_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryVal", org.apache.thrift.protocol.TType.STRUCT, (short)8); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_VAL((short)1, "boolVal"), + BYTE_VAL((short)2, "byteVal"), + I16_VAL((short)3, "i16Val"), + I32_VAL((short)4, "i32Val"), + I64_VAL((short)5, "i64Val"), + DOUBLE_VAL((short)6, "doubleVal"), + STRING_VAL((short)7, "stringVal"), + BINARY_VAL((short)8, "binaryVal"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_VAL + return BOOL_VAL; + case 2: // BYTE_VAL + return BYTE_VAL; + case 3: // I16_VAL + return I16_VAL; + case 4: // I32_VAL + return I32_VAL; + case 5: // I64_VAL + return I64_VAL; + case 6: // DOUBLE_VAL + return DOUBLE_VAL; + case 7: // STRING_VAL + return STRING_VAL; + case 8: // BINARY_VAL + return BINARY_VAL; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolColumn.class))); + tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteColumn.class))); + tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Column.class))); + tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Column.class))); + tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Column.class))); + tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleColumn.class))); + tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringColumn.class))); + tmpMap.put(_Fields.BINARY_VAL, new org.apache.thrift.meta_data.FieldMetaData("binaryVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBinaryColumn.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumn.class, metaDataMap); + } + + public TColumn() { + super(); + } + + public TColumn(TColumn._Fields setField, Object value) { + super(setField, value); + } + + public TColumn(TColumn other) { + super(other); + } + public TColumn deepCopy() { + return new TColumn(this); + } + + public static TColumn boolVal(TBoolColumn value) { + TColumn x = new TColumn(); + x.setBoolVal(value); + return x; + } + + public static TColumn byteVal(TByteColumn value) { + TColumn x = new TColumn(); + x.setByteVal(value); + return x; + } + + public static TColumn i16Val(TI16Column value) { + TColumn x = new TColumn(); + x.setI16Val(value); + return x; + } + + public static TColumn i32Val(TI32Column value) { + TColumn x = new TColumn(); + x.setI32Val(value); + return x; + } + + public static TColumn i64Val(TI64Column value) { + TColumn x = new TColumn(); + x.setI64Val(value); + return x; + } + + public static TColumn doubleVal(TDoubleColumn value) { + TColumn x = new TColumn(); + x.setDoubleVal(value); + return x; + } + + public static TColumn stringVal(TStringColumn value) { + TColumn x = new TColumn(); + x.setStringVal(value); + return x; + } + + public static TColumn binaryVal(TBinaryColumn value) { + TColumn x = new TColumn(); + x.setBinaryVal(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case BOOL_VAL: + if (value instanceof TBoolColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TBoolColumn for field 'boolVal', but got " + value.getClass().getSimpleName()); + case BYTE_VAL: + if (value instanceof TByteColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TByteColumn for field 'byteVal', but got " + value.getClass().getSimpleName()); + case I16_VAL: + if (value instanceof TI16Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI16Column for field 'i16Val', but got " + value.getClass().getSimpleName()); + case I32_VAL: + if (value instanceof TI32Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI32Column for field 'i32Val', but got " + value.getClass().getSimpleName()); + case I64_VAL: + if (value instanceof TI64Column) { + break; + } + throw new ClassCastException("Was expecting value of type TI64Column for field 'i64Val', but got " + value.getClass().getSimpleName()); + case DOUBLE_VAL: + if (value instanceof TDoubleColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TDoubleColumn for field 'doubleVal', but got " + value.getClass().getSimpleName()); + case STRING_VAL: + if (value instanceof TStringColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TStringColumn for field 'stringVal', but got " + value.getClass().getSimpleName()); + case BINARY_VAL: + if (value instanceof TBinaryColumn) { + break; + } + throw new ClassCastException("Was expecting value of type TBinaryColumn for field 'binaryVal', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + if (field.type == BOOL_VAL_FIELD_DESC.type) { + TBoolColumn boolVal; + boolVal = new TBoolColumn(); + boolVal.read(iprot); + return boolVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BYTE_VAL: + if (field.type == BYTE_VAL_FIELD_DESC.type) { + TByteColumn byteVal; + byteVal = new TByteColumn(); + byteVal.read(iprot); + return byteVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I16_VAL: + if (field.type == I16_VAL_FIELD_DESC.type) { + TI16Column i16Val; + i16Val = new TI16Column(); + i16Val.read(iprot); + return i16Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I32_VAL: + if (field.type == I32_VAL_FIELD_DESC.type) { + TI32Column i32Val; + i32Val = new TI32Column(); + i32Val.read(iprot); + return i32Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I64_VAL: + if (field.type == I64_VAL_FIELD_DESC.type) { + TI64Column i64Val; + i64Val = new TI64Column(); + i64Val.read(iprot); + return i64Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DOUBLE_VAL: + if (field.type == DOUBLE_VAL_FIELD_DESC.type) { + TDoubleColumn doubleVal; + doubleVal = new TDoubleColumn(); + doubleVal.read(iprot); + return doubleVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VAL: + if (field.type == STRING_VAL_FIELD_DESC.type) { + TStringColumn stringVal; + stringVal = new TStringColumn(); + stringVal.read(iprot); + return stringVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BINARY_VAL: + if (field.type == BINARY_VAL_FIELD_DESC.type) { + TBinaryColumn binaryVal; + binaryVal = new TBinaryColumn(); + binaryVal.read(iprot); + return binaryVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolColumn boolVal = (TBoolColumn)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteColumn byteVal = (TByteColumn)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Column i16Val = (TI16Column)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Column i32Val = (TI32Column)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Column i64Val = (TI64Column)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleColumn doubleVal = (TDoubleColumn)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringColumn stringVal = (TStringColumn)value_; + stringVal.write(oprot); + return; + case BINARY_VAL: + TBinaryColumn binaryVal = (TBinaryColumn)value_; + binaryVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + TBoolColumn boolVal; + boolVal = new TBoolColumn(); + boolVal.read(iprot); + return boolVal; + case BYTE_VAL: + TByteColumn byteVal; + byteVal = new TByteColumn(); + byteVal.read(iprot); + return byteVal; + case I16_VAL: + TI16Column i16Val; + i16Val = new TI16Column(); + i16Val.read(iprot); + return i16Val; + case I32_VAL: + TI32Column i32Val; + i32Val = new TI32Column(); + i32Val.read(iprot); + return i32Val; + case I64_VAL: + TI64Column i64Val; + i64Val = new TI64Column(); + i64Val.read(iprot); + return i64Val; + case DOUBLE_VAL: + TDoubleColumn doubleVal; + doubleVal = new TDoubleColumn(); + doubleVal.read(iprot); + return doubleVal; + case STRING_VAL: + TStringColumn stringVal; + stringVal = new TStringColumn(); + stringVal.read(iprot); + return stringVal; + case BINARY_VAL: + TBinaryColumn binaryVal; + binaryVal = new TBinaryColumn(); + binaryVal.read(iprot); + return binaryVal; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolColumn boolVal = (TBoolColumn)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteColumn byteVal = (TByteColumn)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Column i16Val = (TI16Column)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Column i32Val = (TI32Column)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Column i64Val = (TI64Column)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleColumn doubleVal = (TDoubleColumn)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringColumn stringVal = (TStringColumn)value_; + stringVal.write(oprot); + return; + case BINARY_VAL: + TBinaryColumn binaryVal = (TBinaryColumn)value_; + binaryVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case BOOL_VAL: + return BOOL_VAL_FIELD_DESC; + case BYTE_VAL: + return BYTE_VAL_FIELD_DESC; + case I16_VAL: + return I16_VAL_FIELD_DESC; + case I32_VAL: + return I32_VAL_FIELD_DESC; + case I64_VAL: + return I64_VAL_FIELD_DESC; + case DOUBLE_VAL: + return DOUBLE_VAL_FIELD_DESC; + case STRING_VAL: + return STRING_VAL_FIELD_DESC; + case BINARY_VAL: + return BINARY_VAL_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TBoolColumn getBoolVal() { + if (getSetField() == _Fields.BOOL_VAL) { + return (TBoolColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBoolVal(TBoolColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOOL_VAL; + value_ = value; + } + + public TByteColumn getByteVal() { + if (getSetField() == _Fields.BYTE_VAL) { + return (TByteColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setByteVal(TByteColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BYTE_VAL; + value_ = value; + } + + public TI16Column getI16Val() { + if (getSetField() == _Fields.I16_VAL) { + return (TI16Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI16Val(TI16Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I16_VAL; + value_ = value; + } + + public TI32Column getI32Val() { + if (getSetField() == _Fields.I32_VAL) { + return (TI32Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Val(TI32Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I32_VAL; + value_ = value; + } + + public TI64Column getI64Val() { + if (getSetField() == _Fields.I64_VAL) { + return (TI64Column)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI64Val(TI64Column value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I64_VAL; + value_ = value; + } + + public TDoubleColumn getDoubleVal() { + if (getSetField() == _Fields.DOUBLE_VAL) { + return (TDoubleColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setDoubleVal(TDoubleColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DOUBLE_VAL; + value_ = value; + } + + public TStringColumn getStringVal() { + if (getSetField() == _Fields.STRING_VAL) { + return (TStringColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringVal(TStringColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VAL; + value_ = value; + } + + public TBinaryColumn getBinaryVal() { + if (getSetField() == _Fields.BINARY_VAL) { + return (TBinaryColumn)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'binaryVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBinaryVal(TBinaryColumn value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BINARY_VAL; + value_ = value; + } + + public boolean isSetBoolVal() { + return setField_ == _Fields.BOOL_VAL; + } + + + public boolean isSetByteVal() { + return setField_ == _Fields.BYTE_VAL; + } + + + public boolean isSetI16Val() { + return setField_ == _Fields.I16_VAL; + } + + + public boolean isSetI32Val() { + return setField_ == _Fields.I32_VAL; + } + + + public boolean isSetI64Val() { + return setField_ == _Fields.I64_VAL; + } + + + public boolean isSetDoubleVal() { + return setField_ == _Fields.DOUBLE_VAL; + } + + + public boolean isSetStringVal() { + return setField_ == _Fields.STRING_VAL; + } + + + public boolean isSetBinaryVal() { + return setField_ == _Fields.BINARY_VAL; + } + + + public boolean equals(Object other) { + if (other instanceof TColumn) { + return equals((TColumn)other); + } else { + return false; + } + } + + public boolean equals(TColumn other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TColumn other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java new file mode 100644 index 0000000000000..31472c8f54b94 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnDesc.java @@ -0,0 +1,704 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TColumnDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnDesc"); + + private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField TYPE_DESC_FIELD_DESC = new org.apache.thrift.protocol.TField("typeDesc", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField POSITION_FIELD_DESC = new org.apache.thrift.protocol.TField("position", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField COMMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("comment", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TColumnDescStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TColumnDescTupleSchemeFactory()); + } + + private String columnName; // required + private TTypeDesc typeDesc; // required + private int position; // required + private String comment; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COLUMN_NAME((short)1, "columnName"), + TYPE_DESC((short)2, "typeDesc"), + POSITION((short)3, "position"), + COMMENT((short)4, "comment"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COLUMN_NAME + return COLUMN_NAME; + case 2: // TYPE_DESC + return TYPE_DESC; + case 3: // POSITION + return POSITION; + case 4: // COMMENT + return COMMENT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __POSITION_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.COMMENT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TYPE_DESC, new org.apache.thrift.meta_data.FieldMetaData("typeDesc", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeDesc.class))); + tmpMap.put(_Fields.POSITION, new org.apache.thrift.meta_data.FieldMetaData("position", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.COMMENT, new org.apache.thrift.meta_data.FieldMetaData("comment", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnDesc.class, metaDataMap); + } + + public TColumnDesc() { + } + + public TColumnDesc( + String columnName, + TTypeDesc typeDesc, + int position) + { + this(); + this.columnName = columnName; + this.typeDesc = typeDesc; + this.position = position; + setPositionIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TColumnDesc(TColumnDesc other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetColumnName()) { + this.columnName = other.columnName; + } + if (other.isSetTypeDesc()) { + this.typeDesc = new TTypeDesc(other.typeDesc); + } + this.position = other.position; + if (other.isSetComment()) { + this.comment = other.comment; + } + } + + public TColumnDesc deepCopy() { + return new TColumnDesc(this); + } + + @Override + public void clear() { + this.columnName = null; + this.typeDesc = null; + setPositionIsSet(false); + this.position = 0; + this.comment = null; + } + + public String getColumnName() { + return this.columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public void unsetColumnName() { + this.columnName = null; + } + + /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ + public boolean isSetColumnName() { + return this.columnName != null; + } + + public void setColumnNameIsSet(boolean value) { + if (!value) { + this.columnName = null; + } + } + + public TTypeDesc getTypeDesc() { + return this.typeDesc; + } + + public void setTypeDesc(TTypeDesc typeDesc) { + this.typeDesc = typeDesc; + } + + public void unsetTypeDesc() { + this.typeDesc = null; + } + + /** Returns true if field typeDesc is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeDesc() { + return this.typeDesc != null; + } + + public void setTypeDescIsSet(boolean value) { + if (!value) { + this.typeDesc = null; + } + } + + public int getPosition() { + return this.position; + } + + public void setPosition(int position) { + this.position = position; + setPositionIsSet(true); + } + + public void unsetPosition() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __POSITION_ISSET_ID); + } + + /** Returns true if field position is set (has been assigned a value) and false otherwise */ + public boolean isSetPosition() { + return EncodingUtils.testBit(__isset_bitfield, __POSITION_ISSET_ID); + } + + public void setPositionIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __POSITION_ISSET_ID, value); + } + + public String getComment() { + return this.comment; + } + + public void setComment(String comment) { + this.comment = comment; + } + + public void unsetComment() { + this.comment = null; + } + + /** Returns true if field comment is set (has been assigned a value) and false otherwise */ + public boolean isSetComment() { + return this.comment != null; + } + + public void setCommentIsSet(boolean value) { + if (!value) { + this.comment = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COLUMN_NAME: + if (value == null) { + unsetColumnName(); + } else { + setColumnName((String)value); + } + break; + + case TYPE_DESC: + if (value == null) { + unsetTypeDesc(); + } else { + setTypeDesc((TTypeDesc)value); + } + break; + + case POSITION: + if (value == null) { + unsetPosition(); + } else { + setPosition((Integer)value); + } + break; + + case COMMENT: + if (value == null) { + unsetComment(); + } else { + setComment((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COLUMN_NAME: + return getColumnName(); + + case TYPE_DESC: + return getTypeDesc(); + + case POSITION: + return getPosition(); + + case COMMENT: + return getComment(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COLUMN_NAME: + return isSetColumnName(); + case TYPE_DESC: + return isSetTypeDesc(); + case POSITION: + return isSetPosition(); + case COMMENT: + return isSetComment(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TColumnDesc) + return this.equals((TColumnDesc)that); + return false; + } + + public boolean equals(TColumnDesc that) { + if (that == null) + return false; + + boolean this_present_columnName = true && this.isSetColumnName(); + boolean that_present_columnName = true && that.isSetColumnName(); + if (this_present_columnName || that_present_columnName) { + if (!(this_present_columnName && that_present_columnName)) + return false; + if (!this.columnName.equals(that.columnName)) + return false; + } + + boolean this_present_typeDesc = true && this.isSetTypeDesc(); + boolean that_present_typeDesc = true && that.isSetTypeDesc(); + if (this_present_typeDesc || that_present_typeDesc) { + if (!(this_present_typeDesc && that_present_typeDesc)) + return false; + if (!this.typeDesc.equals(that.typeDesc)) + return false; + } + + boolean this_present_position = true; + boolean that_present_position = true; + if (this_present_position || that_present_position) { + if (!(this_present_position && that_present_position)) + return false; + if (this.position != that.position) + return false; + } + + boolean this_present_comment = true && this.isSetComment(); + boolean that_present_comment = true && that.isSetComment(); + if (this_present_comment || that_present_comment) { + if (!(this_present_comment && that_present_comment)) + return false; + if (!this.comment.equals(that.comment)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_columnName = true && (isSetColumnName()); + list.add(present_columnName); + if (present_columnName) + list.add(columnName); + + boolean present_typeDesc = true && (isSetTypeDesc()); + list.add(present_typeDesc); + if (present_typeDesc) + list.add(typeDesc); + + boolean present_position = true; + list.add(present_position); + if (present_position) + list.add(position); + + boolean present_comment = true && (isSetComment()); + list.add(present_comment); + if (present_comment) + list.add(comment); + + return list.hashCode(); + } + + @Override + public int compareTo(TColumnDesc other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(other.isSetColumnName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumnName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, other.columnName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTypeDesc()).compareTo(other.isSetTypeDesc()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeDesc()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeDesc, other.typeDesc); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPosition()).compareTo(other.isSetPosition()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPosition()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.position, other.position); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetComment()).compareTo(other.isSetComment()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetComment()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.comment, other.comment); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TColumnDesc("); + boolean first = true; + + sb.append("columnName:"); + if (this.columnName == null) { + sb.append("null"); + } else { + sb.append(this.columnName); + } + first = false; + if (!first) sb.append(", "); + sb.append("typeDesc:"); + if (this.typeDesc == null) { + sb.append("null"); + } else { + sb.append(this.typeDesc); + } + first = false; + if (!first) sb.append(", "); + sb.append("position:"); + sb.append(this.position); + first = false; + if (isSetComment()) { + if (!first) sb.append(", "); + sb.append("comment:"); + if (this.comment == null) { + sb.append("null"); + } else { + sb.append(this.comment); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColumnName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'columnName' is unset! Struct:" + toString()); + } + + if (!isSetTypeDesc()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeDesc' is unset! Struct:" + toString()); + } + + if (!isSetPosition()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'position' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (typeDesc != null) { + typeDesc.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TColumnDescStandardSchemeFactory implements SchemeFactory { + public TColumnDescStandardScheme getScheme() { + return new TColumnDescStandardScheme(); + } + } + + private static class TColumnDescStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TColumnDesc struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COLUMN_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TYPE_DESC + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.typeDesc = new TTypeDesc(); + struct.typeDesc.read(iprot); + struct.setTypeDescIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // POSITION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.position = iprot.readI32(); + struct.setPositionIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // COMMENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.comment = iprot.readString(); + struct.setCommentIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TColumnDesc struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.columnName != null) { + oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); + oprot.writeString(struct.columnName); + oprot.writeFieldEnd(); + } + if (struct.typeDesc != null) { + oprot.writeFieldBegin(TYPE_DESC_FIELD_DESC); + struct.typeDesc.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(POSITION_FIELD_DESC); + oprot.writeI32(struct.position); + oprot.writeFieldEnd(); + if (struct.comment != null) { + if (struct.isSetComment()) { + oprot.writeFieldBegin(COMMENT_FIELD_DESC); + oprot.writeString(struct.comment); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TColumnDescTupleSchemeFactory implements SchemeFactory { + public TColumnDescTupleScheme getScheme() { + return new TColumnDescTupleScheme(); + } + } + + private static class TColumnDescTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.columnName); + struct.typeDesc.write(oprot); + oprot.writeI32(struct.position); + BitSet optionals = new BitSet(); + if (struct.isSetComment()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetComment()) { + oprot.writeString(struct.comment); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TColumnDesc struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + struct.typeDesc = new TTypeDesc(); + struct.typeDesc.read(iprot); + struct.setTypeDescIsSet(true); + struct.position = iprot.readI32(); + struct.setPositionIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.comment = iprot.readString(); + struct.setCommentIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java new file mode 100644 index 0000000000000..d1cc8e919bc0c --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TColumnValue.java @@ -0,0 +1,675 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TColumnValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TColumnValue"); + private static final org.apache.thrift.protocol.TField BOOL_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("boolVal", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField BYTE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("byteVal", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField I16_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i16Val", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField I32_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Val", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField I64_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("i64Val", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField DOUBLE_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("doubleVal", org.apache.thrift.protocol.TType.STRUCT, (short)6); + private static final org.apache.thrift.protocol.TField STRING_VAL_FIELD_DESC = new org.apache.thrift.protocol.TField("stringVal", org.apache.thrift.protocol.TType.STRUCT, (short)7); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + BOOL_VAL((short)1, "boolVal"), + BYTE_VAL((short)2, "byteVal"), + I16_VAL((short)3, "i16Val"), + I32_VAL((short)4, "i32Val"), + I64_VAL((short)5, "i64Val"), + DOUBLE_VAL((short)6, "doubleVal"), + STRING_VAL((short)7, "stringVal"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // BOOL_VAL + return BOOL_VAL; + case 2: // BYTE_VAL + return BYTE_VAL; + case 3: // I16_VAL + return I16_VAL; + case 4: // I32_VAL + return I32_VAL; + case 5: // I64_VAL + return I64_VAL; + case 6: // DOUBLE_VAL + return DOUBLE_VAL; + case 7: // STRING_VAL + return STRING_VAL; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.BOOL_VAL, new org.apache.thrift.meta_data.FieldMetaData("boolVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TBoolValue.class))); + tmpMap.put(_Fields.BYTE_VAL, new org.apache.thrift.meta_data.FieldMetaData("byteVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TByteValue.class))); + tmpMap.put(_Fields.I16_VAL, new org.apache.thrift.meta_data.FieldMetaData("i16Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI16Value.class))); + tmpMap.put(_Fields.I32_VAL, new org.apache.thrift.meta_data.FieldMetaData("i32Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI32Value.class))); + tmpMap.put(_Fields.I64_VAL, new org.apache.thrift.meta_data.FieldMetaData("i64Val", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TI64Value.class))); + tmpMap.put(_Fields.DOUBLE_VAL, new org.apache.thrift.meta_data.FieldMetaData("doubleVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TDoubleValue.class))); + tmpMap.put(_Fields.STRING_VAL, new org.apache.thrift.meta_data.FieldMetaData("stringVal", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStringValue.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TColumnValue.class, metaDataMap); + } + + public TColumnValue() { + super(); + } + + public TColumnValue(TColumnValue._Fields setField, Object value) { + super(setField, value); + } + + public TColumnValue(TColumnValue other) { + super(other); + } + public TColumnValue deepCopy() { + return new TColumnValue(this); + } + + public static TColumnValue boolVal(TBoolValue value) { + TColumnValue x = new TColumnValue(); + x.setBoolVal(value); + return x; + } + + public static TColumnValue byteVal(TByteValue value) { + TColumnValue x = new TColumnValue(); + x.setByteVal(value); + return x; + } + + public static TColumnValue i16Val(TI16Value value) { + TColumnValue x = new TColumnValue(); + x.setI16Val(value); + return x; + } + + public static TColumnValue i32Val(TI32Value value) { + TColumnValue x = new TColumnValue(); + x.setI32Val(value); + return x; + } + + public static TColumnValue i64Val(TI64Value value) { + TColumnValue x = new TColumnValue(); + x.setI64Val(value); + return x; + } + + public static TColumnValue doubleVal(TDoubleValue value) { + TColumnValue x = new TColumnValue(); + x.setDoubleVal(value); + return x; + } + + public static TColumnValue stringVal(TStringValue value) { + TColumnValue x = new TColumnValue(); + x.setStringVal(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case BOOL_VAL: + if (value instanceof TBoolValue) { + break; + } + throw new ClassCastException("Was expecting value of type TBoolValue for field 'boolVal', but got " + value.getClass().getSimpleName()); + case BYTE_VAL: + if (value instanceof TByteValue) { + break; + } + throw new ClassCastException("Was expecting value of type TByteValue for field 'byteVal', but got " + value.getClass().getSimpleName()); + case I16_VAL: + if (value instanceof TI16Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI16Value for field 'i16Val', but got " + value.getClass().getSimpleName()); + case I32_VAL: + if (value instanceof TI32Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI32Value for field 'i32Val', but got " + value.getClass().getSimpleName()); + case I64_VAL: + if (value instanceof TI64Value) { + break; + } + throw new ClassCastException("Was expecting value of type TI64Value for field 'i64Val', but got " + value.getClass().getSimpleName()); + case DOUBLE_VAL: + if (value instanceof TDoubleValue) { + break; + } + throw new ClassCastException("Was expecting value of type TDoubleValue for field 'doubleVal', but got " + value.getClass().getSimpleName()); + case STRING_VAL: + if (value instanceof TStringValue) { + break; + } + throw new ClassCastException("Was expecting value of type TStringValue for field 'stringVal', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + if (field.type == BOOL_VAL_FIELD_DESC.type) { + TBoolValue boolVal; + boolVal = new TBoolValue(); + boolVal.read(iprot); + return boolVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BYTE_VAL: + if (field.type == BYTE_VAL_FIELD_DESC.type) { + TByteValue byteVal; + byteVal = new TByteValue(); + byteVal.read(iprot); + return byteVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I16_VAL: + if (field.type == I16_VAL_FIELD_DESC.type) { + TI16Value i16Val; + i16Val = new TI16Value(); + i16Val.read(iprot); + return i16Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I32_VAL: + if (field.type == I32_VAL_FIELD_DESC.type) { + TI32Value i32Val; + i32Val = new TI32Value(); + i32Val.read(iprot); + return i32Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case I64_VAL: + if (field.type == I64_VAL_FIELD_DESC.type) { + TI64Value i64Val; + i64Val = new TI64Value(); + i64Val.read(iprot); + return i64Val; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case DOUBLE_VAL: + if (field.type == DOUBLE_VAL_FIELD_DESC.type) { + TDoubleValue doubleVal; + doubleVal = new TDoubleValue(); + doubleVal.read(iprot); + return doubleVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VAL: + if (field.type == STRING_VAL_FIELD_DESC.type) { + TStringValue stringVal; + stringVal = new TStringValue(); + stringVal.read(iprot); + return stringVal; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolValue boolVal = (TBoolValue)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteValue byteVal = (TByteValue)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Value i16Val = (TI16Value)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Value i32Val = (TI32Value)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Value i64Val = (TI64Value)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleValue doubleVal = (TDoubleValue)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringValue stringVal = (TStringValue)value_; + stringVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case BOOL_VAL: + TBoolValue boolVal; + boolVal = new TBoolValue(); + boolVal.read(iprot); + return boolVal; + case BYTE_VAL: + TByteValue byteVal; + byteVal = new TByteValue(); + byteVal.read(iprot); + return byteVal; + case I16_VAL: + TI16Value i16Val; + i16Val = new TI16Value(); + i16Val.read(iprot); + return i16Val; + case I32_VAL: + TI32Value i32Val; + i32Val = new TI32Value(); + i32Val.read(iprot); + return i32Val; + case I64_VAL: + TI64Value i64Val; + i64Val = new TI64Value(); + i64Val.read(iprot); + return i64Val; + case DOUBLE_VAL: + TDoubleValue doubleVal; + doubleVal = new TDoubleValue(); + doubleVal.read(iprot); + return doubleVal; + case STRING_VAL: + TStringValue stringVal; + stringVal = new TStringValue(); + stringVal.read(iprot); + return stringVal; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case BOOL_VAL: + TBoolValue boolVal = (TBoolValue)value_; + boolVal.write(oprot); + return; + case BYTE_VAL: + TByteValue byteVal = (TByteValue)value_; + byteVal.write(oprot); + return; + case I16_VAL: + TI16Value i16Val = (TI16Value)value_; + i16Val.write(oprot); + return; + case I32_VAL: + TI32Value i32Val = (TI32Value)value_; + i32Val.write(oprot); + return; + case I64_VAL: + TI64Value i64Val = (TI64Value)value_; + i64Val.write(oprot); + return; + case DOUBLE_VAL: + TDoubleValue doubleVal = (TDoubleValue)value_; + doubleVal.write(oprot); + return; + case STRING_VAL: + TStringValue stringVal = (TStringValue)value_; + stringVal.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case BOOL_VAL: + return BOOL_VAL_FIELD_DESC; + case BYTE_VAL: + return BYTE_VAL_FIELD_DESC; + case I16_VAL: + return I16_VAL_FIELD_DESC; + case I32_VAL: + return I32_VAL_FIELD_DESC; + case I64_VAL: + return I64_VAL_FIELD_DESC; + case DOUBLE_VAL: + return DOUBLE_VAL_FIELD_DESC; + case STRING_VAL: + return STRING_VAL_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TBoolValue getBoolVal() { + if (getSetField() == _Fields.BOOL_VAL) { + return (TBoolValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'boolVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBoolVal(TBoolValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BOOL_VAL; + value_ = value; + } + + public TByteValue getByteVal() { + if (getSetField() == _Fields.BYTE_VAL) { + return (TByteValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'byteVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setByteVal(TByteValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.BYTE_VAL; + value_ = value; + } + + public TI16Value getI16Val() { + if (getSetField() == _Fields.I16_VAL) { + return (TI16Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i16Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI16Val(TI16Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I16_VAL; + value_ = value; + } + + public TI32Value getI32Val() { + if (getSetField() == _Fields.I32_VAL) { + return (TI32Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Val(TI32Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I32_VAL; + value_ = value; + } + + public TI64Value getI64Val() { + if (getSetField() == _Fields.I64_VAL) { + return (TI64Value)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i64Val' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI64Val(TI64Value value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.I64_VAL; + value_ = value; + } + + public TDoubleValue getDoubleVal() { + if (getSetField() == _Fields.DOUBLE_VAL) { + return (TDoubleValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'doubleVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setDoubleVal(TDoubleValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.DOUBLE_VAL; + value_ = value; + } + + public TStringValue getStringVal() { + if (getSetField() == _Fields.STRING_VAL) { + return (TStringValue)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringVal' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringVal(TStringValue value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VAL; + value_ = value; + } + + public boolean isSetBoolVal() { + return setField_ == _Fields.BOOL_VAL; + } + + + public boolean isSetByteVal() { + return setField_ == _Fields.BYTE_VAL; + } + + + public boolean isSetI16Val() { + return setField_ == _Fields.I16_VAL; + } + + + public boolean isSetI32Val() { + return setField_ == _Fields.I32_VAL; + } + + + public boolean isSetI64Val() { + return setField_ == _Fields.I64_VAL; + } + + + public boolean isSetDoubleVal() { + return setField_ == _Fields.DOUBLE_VAL; + } + + + public boolean isSetStringVal() { + return setField_ == _Fields.STRING_VAL; + } + + + public boolean equals(Object other) { + if (other instanceof TColumnValue) { + return equals((TColumnValue)other); + } else { + return false; + } + } + + public boolean equals(TColumnValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TColumnValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java new file mode 100644 index 0000000000000..f93c9b4f0edc3 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TDoubleColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TDoubleColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TDoubleColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleColumn.class, metaDataMap); + } + + public TDoubleColumn() { + } + + public TDoubleColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TDoubleColumn(TDoubleColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TDoubleColumn deepCopy() { + return new TDoubleColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(double elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TDoubleColumn) + return this.equals((TDoubleColumn)that); + return false; + } + + public boolean equals(TDoubleColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TDoubleColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TDoubleColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TDoubleColumnStandardSchemeFactory implements SchemeFactory { + public TDoubleColumnStandardScheme getScheme() { + return new TDoubleColumnStandardScheme(); + } + } + + private static class TDoubleColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list94 = iprot.readListBegin(); + struct.values = new ArrayList(_list94.size); + double _elem95; + for (int _i96 = 0; _i96 < _list94.size; ++_i96) + { + _elem95 = iprot.readDouble(); + struct.values.add(_elem95); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, struct.values.size())); + for (double _iter97 : struct.values) + { + oprot.writeDouble(_iter97); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TDoubleColumnTupleSchemeFactory implements SchemeFactory { + public TDoubleColumnTupleScheme getScheme() { + return new TDoubleColumnTupleScheme(); + } + } + + private static class TDoubleColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (double _iter98 : struct.values) + { + oprot.writeDouble(_iter98); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list99 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.DOUBLE, iprot.readI32()); + struct.values = new ArrayList(_list99.size); + double _elem100; + for (int _i101 = 0; _i101 < _list99.size; ++_i101) + { + _elem100 = iprot.readDouble(); + struct.values.add(_elem100); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java new file mode 100644 index 0000000000000..5700355aad94d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TDoubleValue.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TDoubleValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TDoubleValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.DOUBLE, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TDoubleValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TDoubleValueTupleSchemeFactory()); + } + + private double value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TDoubleValue.class, metaDataMap); + } + + public TDoubleValue() { + } + + /** + * Performs a deep copy on other. + */ + public TDoubleValue(TDoubleValue other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TDoubleValue deepCopy() { + return new TDoubleValue(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0.0; + } + + public double getValue() { + return this.value; + } + + public void setValue(double value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TDoubleValue) + return this.equals((TDoubleValue)that); + return false; + } + + public boolean equals(TDoubleValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TDoubleValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TDoubleValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TDoubleValueStandardSchemeFactory implements SchemeFactory { + public TDoubleValueStandardScheme getScheme() { + return new TDoubleValueStandardScheme(); + } + } + + private static class TDoubleValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TDoubleValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.value = iprot.readDouble(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TDoubleValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeDouble(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TDoubleValueTupleSchemeFactory implements SchemeFactory { + public TDoubleValueTupleScheme getScheme() { + return new TDoubleValueTupleScheme(); + } + } + + private static class TDoubleValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeDouble(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TDoubleValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readDouble(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java new file mode 100644 index 0000000000000..1f73cec61af78 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementReq.java @@ -0,0 +1,863 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TExecuteStatementReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField STATEMENT_FIELD_DESC = new org.apache.thrift.protocol.TField("statement", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField CONF_OVERLAY_FIELD_DESC = new org.apache.thrift.protocol.TField("confOverlay", org.apache.thrift.protocol.TType.MAP, (short)3); + private static final org.apache.thrift.protocol.TField RUN_ASYNC_FIELD_DESC = new org.apache.thrift.protocol.TField("runAsync", org.apache.thrift.protocol.TType.BOOL, (short)4); + private static final org.apache.thrift.protocol.TField QUERY_TIMEOUT_FIELD_DESC = new org.apache.thrift.protocol.TField("queryTimeout", org.apache.thrift.protocol.TType.I64, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TExecuteStatementReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TExecuteStatementReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String statement; // required + private Map confOverlay; // optional + private boolean runAsync; // optional + private long queryTimeout; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + STATEMENT((short)2, "statement"), + CONF_OVERLAY((short)3, "confOverlay"), + RUN_ASYNC((short)4, "runAsync"), + QUERY_TIMEOUT((short)5, "queryTimeout"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // STATEMENT + return STATEMENT; + case 3: // CONF_OVERLAY + return CONF_OVERLAY; + case 4: // RUN_ASYNC + return RUN_ASYNC; + case 5: // QUERY_TIMEOUT + return QUERY_TIMEOUT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __RUNASYNC_ISSET_ID = 0; + private static final int __QUERYTIMEOUT_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.CONF_OVERLAY,_Fields.RUN_ASYNC,_Fields.QUERY_TIMEOUT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.STATEMENT, new org.apache.thrift.meta_data.FieldMetaData("statement", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CONF_OVERLAY, new org.apache.thrift.meta_data.FieldMetaData("confOverlay", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.RUN_ASYNC, new org.apache.thrift.meta_data.FieldMetaData("runAsync", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.QUERY_TIMEOUT, new org.apache.thrift.meta_data.FieldMetaData("queryTimeout", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementReq.class, metaDataMap); + } + + public TExecuteStatementReq() { + this.runAsync = false; + + this.queryTimeout = 0L; + + } + + public TExecuteStatementReq( + TSessionHandle sessionHandle, + String statement) + { + this(); + this.sessionHandle = sessionHandle; + this.statement = statement; + } + + /** + * Performs a deep copy on other. + */ + public TExecuteStatementReq(TExecuteStatementReq other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetStatement()) { + this.statement = other.statement; + } + if (other.isSetConfOverlay()) { + Map __this__confOverlay = new HashMap(other.confOverlay); + this.confOverlay = __this__confOverlay; + } + this.runAsync = other.runAsync; + this.queryTimeout = other.queryTimeout; + } + + public TExecuteStatementReq deepCopy() { + return new TExecuteStatementReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.statement = null; + this.confOverlay = null; + this.runAsync = false; + + this.queryTimeout = 0L; + + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getStatement() { + return this.statement; + } + + public void setStatement(String statement) { + this.statement = statement; + } + + public void unsetStatement() { + this.statement = null; + } + + /** Returns true if field statement is set (has been assigned a value) and false otherwise */ + public boolean isSetStatement() { + return this.statement != null; + } + + public void setStatementIsSet(boolean value) { + if (!value) { + this.statement = null; + } + } + + public int getConfOverlaySize() { + return (this.confOverlay == null) ? 0 : this.confOverlay.size(); + } + + public void putToConfOverlay(String key, String val) { + if (this.confOverlay == null) { + this.confOverlay = new HashMap(); + } + this.confOverlay.put(key, val); + } + + public Map getConfOverlay() { + return this.confOverlay; + } + + public void setConfOverlay(Map confOverlay) { + this.confOverlay = confOverlay; + } + + public void unsetConfOverlay() { + this.confOverlay = null; + } + + /** Returns true if field confOverlay is set (has been assigned a value) and false otherwise */ + public boolean isSetConfOverlay() { + return this.confOverlay != null; + } + + public void setConfOverlayIsSet(boolean value) { + if (!value) { + this.confOverlay = null; + } + } + + public boolean isRunAsync() { + return this.runAsync; + } + + public void setRunAsync(boolean runAsync) { + this.runAsync = runAsync; + setRunAsyncIsSet(true); + } + + public void unsetRunAsync() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __RUNASYNC_ISSET_ID); + } + + /** Returns true if field runAsync is set (has been assigned a value) and false otherwise */ + public boolean isSetRunAsync() { + return EncodingUtils.testBit(__isset_bitfield, __RUNASYNC_ISSET_ID); + } + + public void setRunAsyncIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __RUNASYNC_ISSET_ID, value); + } + + public long getQueryTimeout() { + return this.queryTimeout; + } + + public void setQueryTimeout(long queryTimeout) { + this.queryTimeout = queryTimeout; + setQueryTimeoutIsSet(true); + } + + public void unsetQueryTimeout() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID); + } + + /** Returns true if field queryTimeout is set (has been assigned a value) and false otherwise */ + public boolean isSetQueryTimeout() { + return EncodingUtils.testBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID); + } + + public void setQueryTimeoutIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __QUERYTIMEOUT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case STATEMENT: + if (value == null) { + unsetStatement(); + } else { + setStatement((String)value); + } + break; + + case CONF_OVERLAY: + if (value == null) { + unsetConfOverlay(); + } else { + setConfOverlay((Map)value); + } + break; + + case RUN_ASYNC: + if (value == null) { + unsetRunAsync(); + } else { + setRunAsync((Boolean)value); + } + break; + + case QUERY_TIMEOUT: + if (value == null) { + unsetQueryTimeout(); + } else { + setQueryTimeout((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case STATEMENT: + return getStatement(); + + case CONF_OVERLAY: + return getConfOverlay(); + + case RUN_ASYNC: + return isRunAsync(); + + case QUERY_TIMEOUT: + return getQueryTimeout(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case STATEMENT: + return isSetStatement(); + case CONF_OVERLAY: + return isSetConfOverlay(); + case RUN_ASYNC: + return isSetRunAsync(); + case QUERY_TIMEOUT: + return isSetQueryTimeout(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TExecuteStatementReq) + return this.equals((TExecuteStatementReq)that); + return false; + } + + public boolean equals(TExecuteStatementReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_statement = true && this.isSetStatement(); + boolean that_present_statement = true && that.isSetStatement(); + if (this_present_statement || that_present_statement) { + if (!(this_present_statement && that_present_statement)) + return false; + if (!this.statement.equals(that.statement)) + return false; + } + + boolean this_present_confOverlay = true && this.isSetConfOverlay(); + boolean that_present_confOverlay = true && that.isSetConfOverlay(); + if (this_present_confOverlay || that_present_confOverlay) { + if (!(this_present_confOverlay && that_present_confOverlay)) + return false; + if (!this.confOverlay.equals(that.confOverlay)) + return false; + } + + boolean this_present_runAsync = true && this.isSetRunAsync(); + boolean that_present_runAsync = true && that.isSetRunAsync(); + if (this_present_runAsync || that_present_runAsync) { + if (!(this_present_runAsync && that_present_runAsync)) + return false; + if (this.runAsync != that.runAsync) + return false; + } + + boolean this_present_queryTimeout = true && this.isSetQueryTimeout(); + boolean that_present_queryTimeout = true && that.isSetQueryTimeout(); + if (this_present_queryTimeout || that_present_queryTimeout) { + if (!(this_present_queryTimeout && that_present_queryTimeout)) + return false; + if (this.queryTimeout != that.queryTimeout) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_statement = true && (isSetStatement()); + list.add(present_statement); + if (present_statement) + list.add(statement); + + boolean present_confOverlay = true && (isSetConfOverlay()); + list.add(present_confOverlay); + if (present_confOverlay) + list.add(confOverlay); + + boolean present_runAsync = true && (isSetRunAsync()); + list.add(present_runAsync); + if (present_runAsync) + list.add(runAsync); + + boolean present_queryTimeout = true && (isSetQueryTimeout()); + list.add(present_queryTimeout); + if (present_queryTimeout) + list.add(queryTimeout); + + return list.hashCode(); + } + + @Override + public int compareTo(TExecuteStatementReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatement()).compareTo(other.isSetStatement()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatement()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statement, other.statement); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfOverlay()).compareTo(other.isSetConfOverlay()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfOverlay()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.confOverlay, other.confOverlay); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRunAsync()).compareTo(other.isSetRunAsync()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRunAsync()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.runAsync, other.runAsync); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetQueryTimeout()).compareTo(other.isSetQueryTimeout()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetQueryTimeout()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.queryTimeout, other.queryTimeout); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TExecuteStatementReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("statement:"); + if (this.statement == null) { + sb.append("null"); + } else { + sb.append(this.statement); + } + first = false; + if (isSetConfOverlay()) { + if (!first) sb.append(", "); + sb.append("confOverlay:"); + if (this.confOverlay == null) { + sb.append("null"); + } else { + sb.append(this.confOverlay); + } + first = false; + } + if (isSetRunAsync()) { + if (!first) sb.append(", "); + sb.append("runAsync:"); + sb.append(this.runAsync); + first = false; + } + if (isSetQueryTimeout()) { + if (!first) sb.append(", "); + sb.append("queryTimeout:"); + sb.append(this.queryTimeout); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetStatement()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'statement' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TExecuteStatementReqStandardSchemeFactory implements SchemeFactory { + public TExecuteStatementReqStandardScheme getScheme() { + return new TExecuteStatementReqStandardScheme(); + } + } + + private static class TExecuteStatementReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // STATEMENT + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.statement = iprot.readString(); + struct.setStatementIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // CONF_OVERLAY + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map162 = iprot.readMapBegin(); + struct.confOverlay = new HashMap(2*_map162.size); + String _key163; + String _val164; + for (int _i165 = 0; _i165 < _map162.size; ++_i165) + { + _key163 = iprot.readString(); + _val164 = iprot.readString(); + struct.confOverlay.put(_key163, _val164); + } + iprot.readMapEnd(); + } + struct.setConfOverlayIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // RUN_ASYNC + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.runAsync = iprot.readBool(); + struct.setRunAsyncIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // QUERY_TIMEOUT + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.queryTimeout = iprot.readI64(); + struct.setQueryTimeoutIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.statement != null) { + oprot.writeFieldBegin(STATEMENT_FIELD_DESC); + oprot.writeString(struct.statement); + oprot.writeFieldEnd(); + } + if (struct.confOverlay != null) { + if (struct.isSetConfOverlay()) { + oprot.writeFieldBegin(CONF_OVERLAY_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.confOverlay.size())); + for (Map.Entry _iter166 : struct.confOverlay.entrySet()) + { + oprot.writeString(_iter166.getKey()); + oprot.writeString(_iter166.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.isSetRunAsync()) { + oprot.writeFieldBegin(RUN_ASYNC_FIELD_DESC); + oprot.writeBool(struct.runAsync); + oprot.writeFieldEnd(); + } + if (struct.isSetQueryTimeout()) { + oprot.writeFieldBegin(QUERY_TIMEOUT_FIELD_DESC); + oprot.writeI64(struct.queryTimeout); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TExecuteStatementReqTupleSchemeFactory implements SchemeFactory { + public TExecuteStatementReqTupleScheme getScheme() { + return new TExecuteStatementReqTupleScheme(); + } + } + + private static class TExecuteStatementReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.statement); + BitSet optionals = new BitSet(); + if (struct.isSetConfOverlay()) { + optionals.set(0); + } + if (struct.isSetRunAsync()) { + optionals.set(1); + } + if (struct.isSetQueryTimeout()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetConfOverlay()) { + { + oprot.writeI32(struct.confOverlay.size()); + for (Map.Entry _iter167 : struct.confOverlay.entrySet()) + { + oprot.writeString(_iter167.getKey()); + oprot.writeString(_iter167.getValue()); + } + } + } + if (struct.isSetRunAsync()) { + oprot.writeBool(struct.runAsync); + } + if (struct.isSetQueryTimeout()) { + oprot.writeI64(struct.queryTimeout); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.statement = iprot.readString(); + struct.setStatementIsSet(true); + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TMap _map168 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.confOverlay = new HashMap(2*_map168.size); + String _key169; + String _val170; + for (int _i171 = 0; _i171 < _map168.size; ++_i171) + { + _key169 = iprot.readString(); + _val170 = iprot.readString(); + struct.confOverlay.put(_key169, _val170); + } + } + struct.setConfOverlayIsSet(true); + } + if (incoming.get(1)) { + struct.runAsync = iprot.readBool(); + struct.setRunAsyncIsSet(true); + } + if (incoming.get(2)) { + struct.queryTimeout = iprot.readI64(); + struct.setQueryTimeoutIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java new file mode 100644 index 0000000000000..7101fa5bdb84c --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TExecuteStatementResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TExecuteStatementResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TExecuteStatementResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TExecuteStatementRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TExecuteStatementRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TExecuteStatementResp.class, metaDataMap); + } + + public TExecuteStatementResp() { + } + + public TExecuteStatementResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TExecuteStatementResp(TExecuteStatementResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TExecuteStatementResp deepCopy() { + return new TExecuteStatementResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TExecuteStatementResp) + return this.equals((TExecuteStatementResp)that); + return false; + } + + public boolean equals(TExecuteStatementResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TExecuteStatementResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TExecuteStatementResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TExecuteStatementRespStandardSchemeFactory implements SchemeFactory { + public TExecuteStatementRespStandardScheme getScheme() { + return new TExecuteStatementRespStandardScheme(); + } + } + + private static class TExecuteStatementRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TExecuteStatementRespTupleSchemeFactory implements SchemeFactory { + public TExecuteStatementRespTupleScheme getScheme() { + return new TExecuteStatementRespTupleScheme(); + } + } + + private static class TExecuteStatementRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TExecuteStatementResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java new file mode 100644 index 0000000000000..159be45259434 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchOrientation.java @@ -0,0 +1,57 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TFetchOrientation implements org.apache.thrift.TEnum { + FETCH_NEXT(0), + FETCH_PRIOR(1), + FETCH_RELATIVE(2), + FETCH_ABSOLUTE(3), + FETCH_FIRST(4), + FETCH_LAST(5); + + private final int value; + + private TFetchOrientation(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TFetchOrientation findByValue(int value) { + switch (value) { + case 0: + return FETCH_NEXT; + case 1: + return FETCH_PRIOR; + case 2: + return FETCH_RELATIVE; + case 3: + return FETCH_ABSOLUTE; + case 4: + return FETCH_FIRST; + case 5: + return FETCH_LAST; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java new file mode 100644 index 0000000000000..2c93339d0c68b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsReq.java @@ -0,0 +1,714 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TFetchResultsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ORIENTATION_FIELD_DESC = new org.apache.thrift.protocol.TField("orientation", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField MAX_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("maxRows", org.apache.thrift.protocol.TType.I64, (short)3); + private static final org.apache.thrift.protocol.TField FETCH_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("fetchType", org.apache.thrift.protocol.TType.I16, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TFetchResultsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TFetchResultsReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + private TFetchOrientation orientation; // required + private long maxRows; // required + private short fetchType; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"), + /** + * + * @see TFetchOrientation + */ + ORIENTATION((short)2, "orientation"), + MAX_ROWS((short)3, "maxRows"), + FETCH_TYPE((short)4, "fetchType"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + case 2: // ORIENTATION + return ORIENTATION; + case 3: // MAX_ROWS + return MAX_ROWS; + case 4: // FETCH_TYPE + return FETCH_TYPE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __MAXROWS_ISSET_ID = 0; + private static final int __FETCHTYPE_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.FETCH_TYPE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + tmpMap.put(_Fields.ORIENTATION, new org.apache.thrift.meta_data.FieldMetaData("orientation", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TFetchOrientation.class))); + tmpMap.put(_Fields.MAX_ROWS, new org.apache.thrift.meta_data.FieldMetaData("maxRows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.FETCH_TYPE, new org.apache.thrift.meta_data.FieldMetaData("fetchType", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsReq.class, metaDataMap); + } + + public TFetchResultsReq() { + this.orientation = org.apache.hive.service.rpc.thrift.TFetchOrientation.FETCH_NEXT; + + this.fetchType = (short)0; + + } + + public TFetchResultsReq( + TOperationHandle operationHandle, + TFetchOrientation orientation, + long maxRows) + { + this(); + this.operationHandle = operationHandle; + this.orientation = orientation; + this.maxRows = maxRows; + setMaxRowsIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TFetchResultsReq(TFetchResultsReq other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + if (other.isSetOrientation()) { + this.orientation = other.orientation; + } + this.maxRows = other.maxRows; + this.fetchType = other.fetchType; + } + + public TFetchResultsReq deepCopy() { + return new TFetchResultsReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + this.orientation = org.apache.hive.service.rpc.thrift.TFetchOrientation.FETCH_NEXT; + + setMaxRowsIsSet(false); + this.maxRows = 0; + this.fetchType = (short)0; + + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + /** + * + * @see TFetchOrientation + */ + public TFetchOrientation getOrientation() { + return this.orientation; + } + + /** + * + * @see TFetchOrientation + */ + public void setOrientation(TFetchOrientation orientation) { + this.orientation = orientation; + } + + public void unsetOrientation() { + this.orientation = null; + } + + /** Returns true if field orientation is set (has been assigned a value) and false otherwise */ + public boolean isSetOrientation() { + return this.orientation != null; + } + + public void setOrientationIsSet(boolean value) { + if (!value) { + this.orientation = null; + } + } + + public long getMaxRows() { + return this.maxRows; + } + + public void setMaxRows(long maxRows) { + this.maxRows = maxRows; + setMaxRowsIsSet(true); + } + + public void unsetMaxRows() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MAXROWS_ISSET_ID); + } + + /** Returns true if field maxRows is set (has been assigned a value) and false otherwise */ + public boolean isSetMaxRows() { + return EncodingUtils.testBit(__isset_bitfield, __MAXROWS_ISSET_ID); + } + + public void setMaxRowsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MAXROWS_ISSET_ID, value); + } + + public short getFetchType() { + return this.fetchType; + } + + public void setFetchType(short fetchType) { + this.fetchType = fetchType; + setFetchTypeIsSet(true); + } + + public void unsetFetchType() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); + } + + /** Returns true if field fetchType is set (has been assigned a value) and false otherwise */ + public boolean isSetFetchType() { + return EncodingUtils.testBit(__isset_bitfield, __FETCHTYPE_ISSET_ID); + } + + public void setFetchTypeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __FETCHTYPE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + case ORIENTATION: + if (value == null) { + unsetOrientation(); + } else { + setOrientation((TFetchOrientation)value); + } + break; + + case MAX_ROWS: + if (value == null) { + unsetMaxRows(); + } else { + setMaxRows((Long)value); + } + break; + + case FETCH_TYPE: + if (value == null) { + unsetFetchType(); + } else { + setFetchType((Short)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + case ORIENTATION: + return getOrientation(); + + case MAX_ROWS: + return getMaxRows(); + + case FETCH_TYPE: + return getFetchType(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + case ORIENTATION: + return isSetOrientation(); + case MAX_ROWS: + return isSetMaxRows(); + case FETCH_TYPE: + return isSetFetchType(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TFetchResultsReq) + return this.equals((TFetchResultsReq)that); + return false; + } + + public boolean equals(TFetchResultsReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + boolean this_present_orientation = true && this.isSetOrientation(); + boolean that_present_orientation = true && that.isSetOrientation(); + if (this_present_orientation || that_present_orientation) { + if (!(this_present_orientation && that_present_orientation)) + return false; + if (!this.orientation.equals(that.orientation)) + return false; + } + + boolean this_present_maxRows = true; + boolean that_present_maxRows = true; + if (this_present_maxRows || that_present_maxRows) { + if (!(this_present_maxRows && that_present_maxRows)) + return false; + if (this.maxRows != that.maxRows) + return false; + } + + boolean this_present_fetchType = true && this.isSetFetchType(); + boolean that_present_fetchType = true && that.isSetFetchType(); + if (this_present_fetchType || that_present_fetchType) { + if (!(this_present_fetchType && that_present_fetchType)) + return false; + if (this.fetchType != that.fetchType) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + boolean present_orientation = true && (isSetOrientation()); + list.add(present_orientation); + if (present_orientation) + list.add(orientation.getValue()); + + boolean present_maxRows = true; + list.add(present_maxRows); + if (present_maxRows) + list.add(maxRows); + + boolean present_fetchType = true && (isSetFetchType()); + list.add(present_fetchType); + if (present_fetchType) + list.add(fetchType); + + return list.hashCode(); + } + + @Override + public int compareTo(TFetchResultsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOrientation()).compareTo(other.isSetOrientation()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOrientation()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.orientation, other.orientation); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetMaxRows()).compareTo(other.isSetMaxRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetMaxRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.maxRows, other.maxRows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFetchType()).compareTo(other.isSetFetchType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFetchType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.fetchType, other.fetchType); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TFetchResultsReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("orientation:"); + if (this.orientation == null) { + sb.append("null"); + } else { + sb.append(this.orientation); + } + first = false; + if (!first) sb.append(", "); + sb.append("maxRows:"); + sb.append(this.maxRows); + first = false; + if (isSetFetchType()) { + if (!first) sb.append(", "); + sb.append("fetchType:"); + sb.append(this.fetchType); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + if (!isSetOrientation()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'orientation' is unset! Struct:" + toString()); + } + + if (!isSetMaxRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'maxRows' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TFetchResultsReqStandardSchemeFactory implements SchemeFactory { + public TFetchResultsReqStandardScheme getScheme() { + return new TFetchResultsReqStandardScheme(); + } + } + + private static class TFetchResultsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ORIENTATION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.orientation = org.apache.hive.service.rpc.thrift.TFetchOrientation.findByValue(iprot.readI32()); + struct.setOrientationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // MAX_ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.maxRows = iprot.readI64(); + struct.setMaxRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // FETCH_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.fetchType = iprot.readI16(); + struct.setFetchTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.orientation != null) { + oprot.writeFieldBegin(ORIENTATION_FIELD_DESC); + oprot.writeI32(struct.orientation.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(MAX_ROWS_FIELD_DESC); + oprot.writeI64(struct.maxRows); + oprot.writeFieldEnd(); + if (struct.isSetFetchType()) { + oprot.writeFieldBegin(FETCH_TYPE_FIELD_DESC); + oprot.writeI16(struct.fetchType); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFetchResultsReqTupleSchemeFactory implements SchemeFactory { + public TFetchResultsReqTupleScheme getScheme() { + return new TFetchResultsReqTupleScheme(); + } + } + + private static class TFetchResultsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + oprot.writeI32(struct.orientation.getValue()); + oprot.writeI64(struct.maxRows); + BitSet optionals = new BitSet(); + if (struct.isSetFetchType()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetFetchType()) { + oprot.writeI16(struct.fetchType); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + struct.orientation = org.apache.hive.service.rpc.thrift.TFetchOrientation.findByValue(iprot.readI32()); + struct.setOrientationIsSet(true); + struct.maxRows = iprot.readI64(); + struct.setMaxRowsIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.fetchType = iprot.readI16(); + struct.setFetchTypeIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java new file mode 100644 index 0000000000000..8f86cee3ad468 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TFetchResultsResp.java @@ -0,0 +1,612 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TFetchResultsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TFetchResultsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField HAS_MORE_ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("hasMoreRows", org.apache.thrift.protocol.TType.BOOL, (short)2); + private static final org.apache.thrift.protocol.TField RESULTS_FIELD_DESC = new org.apache.thrift.protocol.TField("results", org.apache.thrift.protocol.TType.STRUCT, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TFetchResultsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TFetchResultsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private boolean hasMoreRows; // optional + private TRowSet results; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + HAS_MORE_ROWS((short)2, "hasMoreRows"), + RESULTS((short)3, "results"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // HAS_MORE_ROWS + return HAS_MORE_ROWS; + case 3: // RESULTS + return RESULTS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __HASMOREROWS_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.HAS_MORE_ROWS,_Fields.RESULTS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.HAS_MORE_ROWS, new org.apache.thrift.meta_data.FieldMetaData("hasMoreRows", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.RESULTS, new org.apache.thrift.meta_data.FieldMetaData("results", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRowSet.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TFetchResultsResp.class, metaDataMap); + } + + public TFetchResultsResp() { + } + + public TFetchResultsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TFetchResultsResp(TFetchResultsResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + this.hasMoreRows = other.hasMoreRows; + if (other.isSetResults()) { + this.results = new TRowSet(other.results); + } + } + + public TFetchResultsResp deepCopy() { + return new TFetchResultsResp(this); + } + + @Override + public void clear() { + this.status = null; + setHasMoreRowsIsSet(false); + this.hasMoreRows = false; + this.results = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public boolean isHasMoreRows() { + return this.hasMoreRows; + } + + public void setHasMoreRows(boolean hasMoreRows) { + this.hasMoreRows = hasMoreRows; + setHasMoreRowsIsSet(true); + } + + public void unsetHasMoreRows() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); + } + + /** Returns true if field hasMoreRows is set (has been assigned a value) and false otherwise */ + public boolean isSetHasMoreRows() { + return EncodingUtils.testBit(__isset_bitfield, __HASMOREROWS_ISSET_ID); + } + + public void setHasMoreRowsIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASMOREROWS_ISSET_ID, value); + } + + public TRowSet getResults() { + return this.results; + } + + public void setResults(TRowSet results) { + this.results = results; + } + + public void unsetResults() { + this.results = null; + } + + /** Returns true if field results is set (has been assigned a value) and false otherwise */ + public boolean isSetResults() { + return this.results != null; + } + + public void setResultsIsSet(boolean value) { + if (!value) { + this.results = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case HAS_MORE_ROWS: + if (value == null) { + unsetHasMoreRows(); + } else { + setHasMoreRows((Boolean)value); + } + break; + + case RESULTS: + if (value == null) { + unsetResults(); + } else { + setResults((TRowSet)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case HAS_MORE_ROWS: + return isHasMoreRows(); + + case RESULTS: + return getResults(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case HAS_MORE_ROWS: + return isSetHasMoreRows(); + case RESULTS: + return isSetResults(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TFetchResultsResp) + return this.equals((TFetchResultsResp)that); + return false; + } + + public boolean equals(TFetchResultsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_hasMoreRows = true && this.isSetHasMoreRows(); + boolean that_present_hasMoreRows = true && that.isSetHasMoreRows(); + if (this_present_hasMoreRows || that_present_hasMoreRows) { + if (!(this_present_hasMoreRows && that_present_hasMoreRows)) + return false; + if (this.hasMoreRows != that.hasMoreRows) + return false; + } + + boolean this_present_results = true && this.isSetResults(); + boolean that_present_results = true && that.isSetResults(); + if (this_present_results || that_present_results) { + if (!(this_present_results && that_present_results)) + return false; + if (!this.results.equals(that.results)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_hasMoreRows = true && (isSetHasMoreRows()); + list.add(present_hasMoreRows); + if (present_hasMoreRows) + list.add(hasMoreRows); + + boolean present_results = true && (isSetResults()); + list.add(present_results); + if (present_results) + list.add(results); + + return list.hashCode(); + } + + @Override + public int compareTo(TFetchResultsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHasMoreRows()).compareTo(other.isSetHasMoreRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHasMoreRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasMoreRows, other.hasMoreRows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetResults()).compareTo(other.isSetResults()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetResults()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.results, other.results); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TFetchResultsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetHasMoreRows()) { + if (!first) sb.append(", "); + sb.append("hasMoreRows:"); + sb.append(this.hasMoreRows); + first = false; + } + if (isSetResults()) { + if (!first) sb.append(", "); + sb.append("results:"); + if (this.results == null) { + sb.append("null"); + } else { + sb.append(this.results); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (results != null) { + results.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TFetchResultsRespStandardSchemeFactory implements SchemeFactory { + public TFetchResultsRespStandardScheme getScheme() { + return new TFetchResultsRespStandardScheme(); + } + } + + private static class TFetchResultsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TFetchResultsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // HAS_MORE_ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.hasMoreRows = iprot.readBool(); + struct.setHasMoreRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // RESULTS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.results = new TRowSet(); + struct.results.read(iprot); + struct.setResultsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TFetchResultsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.isSetHasMoreRows()) { + oprot.writeFieldBegin(HAS_MORE_ROWS_FIELD_DESC); + oprot.writeBool(struct.hasMoreRows); + oprot.writeFieldEnd(); + } + if (struct.results != null) { + if (struct.isSetResults()) { + oprot.writeFieldBegin(RESULTS_FIELD_DESC); + struct.results.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TFetchResultsRespTupleSchemeFactory implements SchemeFactory { + public TFetchResultsRespTupleScheme getScheme() { + return new TFetchResultsRespTupleScheme(); + } + } + + private static class TFetchResultsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetHasMoreRows()) { + optionals.set(0); + } + if (struct.isSetResults()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetHasMoreRows()) { + oprot.writeBool(struct.hasMoreRows); + } + if (struct.isSetResults()) { + struct.results.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TFetchResultsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.hasMoreRows = iprot.readBool(); + struct.setHasMoreRowsIsSet(true); + } + if (incoming.get(1)) { + struct.results = new TRowSet(); + struct.results.read(iprot); + struct.setResultsIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java new file mode 100644 index 0000000000000..b8a2ca6648069 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetCatalogsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCatalogsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCatalogsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsReq.class, metaDataMap); + } + + public TGetCatalogsReq() { + } + + public TGetCatalogsReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetCatalogsReq(TGetCatalogsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetCatalogsReq deepCopy() { + return new TGetCatalogsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCatalogsReq) + return this.equals((TGetCatalogsReq)that); + return false; + } + + public boolean equals(TGetCatalogsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetCatalogsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCatalogsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCatalogsReqStandardSchemeFactory implements SchemeFactory { + public TGetCatalogsReqStandardScheme getScheme() { + return new TGetCatalogsReqStandardScheme(); + } + } + + private static class TGetCatalogsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCatalogsReqTupleSchemeFactory implements SchemeFactory { + public TGetCatalogsReqTupleScheme getScheme() { + return new TGetCatalogsReqTupleScheme(); + } + } + + private static class TGetCatalogsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java new file mode 100644 index 0000000000000..eeeac9a1f9292 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCatalogsResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetCatalogsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCatalogsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCatalogsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCatalogsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCatalogsResp.class, metaDataMap); + } + + public TGetCatalogsResp() { + } + + public TGetCatalogsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetCatalogsResp(TGetCatalogsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetCatalogsResp deepCopy() { + return new TGetCatalogsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCatalogsResp) + return this.equals((TGetCatalogsResp)that); + return false; + } + + public boolean equals(TGetCatalogsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetCatalogsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCatalogsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCatalogsRespStandardSchemeFactory implements SchemeFactory { + public TGetCatalogsRespStandardScheme getScheme() { + return new TGetCatalogsRespStandardScheme(); + } + } + + private static class TGetCatalogsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCatalogsRespTupleSchemeFactory implements SchemeFactory { + public TGetCatalogsRespTupleScheme getScheme() { + return new TGetCatalogsRespTupleScheme(); + } + } + + private static class TGetCatalogsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCatalogsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java new file mode 100644 index 0000000000000..ba80279294957 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsReq.java @@ -0,0 +1,822 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetColumnsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField COLUMN_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("columnName", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetColumnsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetColumnsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String tableName; // optional + private String columnName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + TABLE_NAME((short)4, "tableName"), + COLUMN_NAME((short)5, "columnName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // TABLE_NAME + return TABLE_NAME; + case 5: // COLUMN_NAME + return COLUMN_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.COLUMN_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.COLUMN_NAME, new org.apache.thrift.meta_data.FieldMetaData("columnName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsReq.class, metaDataMap); + } + + public TGetColumnsReq() { + } + + public TGetColumnsReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetColumnsReq(TGetColumnsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetColumnName()) { + this.columnName = other.columnName; + } + } + + public TGetColumnsReq deepCopy() { + return new TGetColumnsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.tableName = null; + this.columnName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public String getColumnName() { + return this.columnName; + } + + public void setColumnName(String columnName) { + this.columnName = columnName; + } + + public void unsetColumnName() { + this.columnName = null; + } + + /** Returns true if field columnName is set (has been assigned a value) and false otherwise */ + public boolean isSetColumnName() { + return this.columnName != null; + } + + public void setColumnNameIsSet(boolean value) { + if (!value) { + this.columnName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case COLUMN_NAME: + if (value == null) { + unsetColumnName(); + } else { + setColumnName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case TABLE_NAME: + return getTableName(); + + case COLUMN_NAME: + return getColumnName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case TABLE_NAME: + return isSetTableName(); + case COLUMN_NAME: + return isSetColumnName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetColumnsReq) + return this.equals((TGetColumnsReq)that); + return false; + } + + public boolean equals(TGetColumnsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + boolean this_present_columnName = true && this.isSetColumnName(); + boolean that_present_columnName = true && that.isSetColumnName(); + if (this_present_columnName || that_present_columnName) { + if (!(this_present_columnName && that_present_columnName)) + return false; + if (!this.columnName.equals(that.columnName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + list.add(present_catalogName); + if (present_catalogName) + list.add(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + list.add(present_schemaName); + if (present_schemaName) + list.add(schemaName); + + boolean present_tableName = true && (isSetTableName()); + list.add(present_tableName); + if (present_tableName) + list.add(tableName); + + boolean present_columnName = true && (isSetColumnName()); + list.add(present_columnName); + if (present_columnName) + list.add(columnName); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetColumnsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetColumnName()).compareTo(other.isSetColumnName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumnName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnName, other.columnName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetColumnsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (isSetTableName()) { + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + } + if (isSetColumnName()) { + if (!first) sb.append(", "); + sb.append("columnName:"); + if (this.columnName == null) { + sb.append("null"); + } else { + sb.append(this.columnName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetColumnsReqStandardSchemeFactory implements SchemeFactory { + public TGetColumnsReqStandardScheme getScheme() { + return new TGetColumnsReqStandardScheme(); + } + } + + private static class TGetColumnsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // COLUMN_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.tableName != null) { + if (struct.isSetTableName()) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + } + if (struct.columnName != null) { + if (struct.isSetColumnName()) { + oprot.writeFieldBegin(COLUMN_NAME_FIELD_DESC); + oprot.writeString(struct.columnName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetColumnsReqTupleSchemeFactory implements SchemeFactory { + public TGetColumnsReqTupleScheme getScheme() { + return new TGetColumnsReqTupleScheme(); + } + } + + private static class TGetColumnsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + if (struct.isSetColumnName()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetColumnName()) { + oprot.writeString(struct.columnName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(3)) { + struct.columnName = iprot.readString(); + struct.setColumnNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java new file mode 100644 index 0000000000000..c68aac9042fc1 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetColumnsResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetColumnsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetColumnsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetColumnsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetColumnsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetColumnsResp.class, metaDataMap); + } + + public TGetColumnsResp() { + } + + public TGetColumnsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetColumnsResp(TGetColumnsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetColumnsResp deepCopy() { + return new TGetColumnsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetColumnsResp) + return this.equals((TGetColumnsResp)that); + return false; + } + + public boolean equals(TGetColumnsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetColumnsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetColumnsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetColumnsRespStandardSchemeFactory implements SchemeFactory { + public TGetColumnsRespStandardScheme getScheme() { + return new TGetColumnsRespStandardScheme(); + } + } + + private static class TGetColumnsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetColumnsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetColumnsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetColumnsRespTupleSchemeFactory implements SchemeFactory { + public TGetColumnsRespTupleScheme getScheme() { + return new TGetColumnsRespTupleScheme(); + } + } + + private static class TGetColumnsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetColumnsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java new file mode 100644 index 0000000000000..972957063b297 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceReq.java @@ -0,0 +1,1034 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetCrossReferenceReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCrossReferenceReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField PARENT_CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parentCatalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PARENT_SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parentSchemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField PARENT_TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("parentTableName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField FOREIGN_CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignCatalogName", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField FOREIGN_SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignSchemaName", org.apache.thrift.protocol.TType.STRING, (short)6); + private static final org.apache.thrift.protocol.TField FOREIGN_TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("foreignTableName", org.apache.thrift.protocol.TType.STRING, (short)7); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCrossReferenceReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCrossReferenceReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String parentCatalogName; // optional + private String parentSchemaName; // optional + private String parentTableName; // optional + private String foreignCatalogName; // optional + private String foreignSchemaName; // optional + private String foreignTableName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + PARENT_CATALOG_NAME((short)2, "parentCatalogName"), + PARENT_SCHEMA_NAME((short)3, "parentSchemaName"), + PARENT_TABLE_NAME((short)4, "parentTableName"), + FOREIGN_CATALOG_NAME((short)5, "foreignCatalogName"), + FOREIGN_SCHEMA_NAME((short)6, "foreignSchemaName"), + FOREIGN_TABLE_NAME((short)7, "foreignTableName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // PARENT_CATALOG_NAME + return PARENT_CATALOG_NAME; + case 3: // PARENT_SCHEMA_NAME + return PARENT_SCHEMA_NAME; + case 4: // PARENT_TABLE_NAME + return PARENT_TABLE_NAME; + case 5: // FOREIGN_CATALOG_NAME + return FOREIGN_CATALOG_NAME; + case 6: // FOREIGN_SCHEMA_NAME + return FOREIGN_SCHEMA_NAME; + case 7: // FOREIGN_TABLE_NAME + return FOREIGN_TABLE_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.PARENT_CATALOG_NAME,_Fields.PARENT_SCHEMA_NAME,_Fields.PARENT_TABLE_NAME,_Fields.FOREIGN_CATALOG_NAME,_Fields.FOREIGN_SCHEMA_NAME,_Fields.FOREIGN_TABLE_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.PARENT_CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("parentCatalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.PARENT_SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("parentSchemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.PARENT_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("parentTableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.FOREIGN_CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreignCatalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.FOREIGN_SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreignSchemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.FOREIGN_TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("foreignTableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCrossReferenceReq.class, metaDataMap); + } + + public TGetCrossReferenceReq() { + } + + public TGetCrossReferenceReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetCrossReferenceReq(TGetCrossReferenceReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetParentCatalogName()) { + this.parentCatalogName = other.parentCatalogName; + } + if (other.isSetParentSchemaName()) { + this.parentSchemaName = other.parentSchemaName; + } + if (other.isSetParentTableName()) { + this.parentTableName = other.parentTableName; + } + if (other.isSetForeignCatalogName()) { + this.foreignCatalogName = other.foreignCatalogName; + } + if (other.isSetForeignSchemaName()) { + this.foreignSchemaName = other.foreignSchemaName; + } + if (other.isSetForeignTableName()) { + this.foreignTableName = other.foreignTableName; + } + } + + public TGetCrossReferenceReq deepCopy() { + return new TGetCrossReferenceReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.parentCatalogName = null; + this.parentSchemaName = null; + this.parentTableName = null; + this.foreignCatalogName = null; + this.foreignSchemaName = null; + this.foreignTableName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getParentCatalogName() { + return this.parentCatalogName; + } + + public void setParentCatalogName(String parentCatalogName) { + this.parentCatalogName = parentCatalogName; + } + + public void unsetParentCatalogName() { + this.parentCatalogName = null; + } + + /** Returns true if field parentCatalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetParentCatalogName() { + return this.parentCatalogName != null; + } + + public void setParentCatalogNameIsSet(boolean value) { + if (!value) { + this.parentCatalogName = null; + } + } + + public String getParentSchemaName() { + return this.parentSchemaName; + } + + public void setParentSchemaName(String parentSchemaName) { + this.parentSchemaName = parentSchemaName; + } + + public void unsetParentSchemaName() { + this.parentSchemaName = null; + } + + /** Returns true if field parentSchemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetParentSchemaName() { + return this.parentSchemaName != null; + } + + public void setParentSchemaNameIsSet(boolean value) { + if (!value) { + this.parentSchemaName = null; + } + } + + public String getParentTableName() { + return this.parentTableName; + } + + public void setParentTableName(String parentTableName) { + this.parentTableName = parentTableName; + } + + public void unsetParentTableName() { + this.parentTableName = null; + } + + /** Returns true if field parentTableName is set (has been assigned a value) and false otherwise */ + public boolean isSetParentTableName() { + return this.parentTableName != null; + } + + public void setParentTableNameIsSet(boolean value) { + if (!value) { + this.parentTableName = null; + } + } + + public String getForeignCatalogName() { + return this.foreignCatalogName; + } + + public void setForeignCatalogName(String foreignCatalogName) { + this.foreignCatalogName = foreignCatalogName; + } + + public void unsetForeignCatalogName() { + this.foreignCatalogName = null; + } + + /** Returns true if field foreignCatalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetForeignCatalogName() { + return this.foreignCatalogName != null; + } + + public void setForeignCatalogNameIsSet(boolean value) { + if (!value) { + this.foreignCatalogName = null; + } + } + + public String getForeignSchemaName() { + return this.foreignSchemaName; + } + + public void setForeignSchemaName(String foreignSchemaName) { + this.foreignSchemaName = foreignSchemaName; + } + + public void unsetForeignSchemaName() { + this.foreignSchemaName = null; + } + + /** Returns true if field foreignSchemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetForeignSchemaName() { + return this.foreignSchemaName != null; + } + + public void setForeignSchemaNameIsSet(boolean value) { + if (!value) { + this.foreignSchemaName = null; + } + } + + public String getForeignTableName() { + return this.foreignTableName; + } + + public void setForeignTableName(String foreignTableName) { + this.foreignTableName = foreignTableName; + } + + public void unsetForeignTableName() { + this.foreignTableName = null; + } + + /** Returns true if field foreignTableName is set (has been assigned a value) and false otherwise */ + public boolean isSetForeignTableName() { + return this.foreignTableName != null; + } + + public void setForeignTableNameIsSet(boolean value) { + if (!value) { + this.foreignTableName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case PARENT_CATALOG_NAME: + if (value == null) { + unsetParentCatalogName(); + } else { + setParentCatalogName((String)value); + } + break; + + case PARENT_SCHEMA_NAME: + if (value == null) { + unsetParentSchemaName(); + } else { + setParentSchemaName((String)value); + } + break; + + case PARENT_TABLE_NAME: + if (value == null) { + unsetParentTableName(); + } else { + setParentTableName((String)value); + } + break; + + case FOREIGN_CATALOG_NAME: + if (value == null) { + unsetForeignCatalogName(); + } else { + setForeignCatalogName((String)value); + } + break; + + case FOREIGN_SCHEMA_NAME: + if (value == null) { + unsetForeignSchemaName(); + } else { + setForeignSchemaName((String)value); + } + break; + + case FOREIGN_TABLE_NAME: + if (value == null) { + unsetForeignTableName(); + } else { + setForeignTableName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case PARENT_CATALOG_NAME: + return getParentCatalogName(); + + case PARENT_SCHEMA_NAME: + return getParentSchemaName(); + + case PARENT_TABLE_NAME: + return getParentTableName(); + + case FOREIGN_CATALOG_NAME: + return getForeignCatalogName(); + + case FOREIGN_SCHEMA_NAME: + return getForeignSchemaName(); + + case FOREIGN_TABLE_NAME: + return getForeignTableName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case PARENT_CATALOG_NAME: + return isSetParentCatalogName(); + case PARENT_SCHEMA_NAME: + return isSetParentSchemaName(); + case PARENT_TABLE_NAME: + return isSetParentTableName(); + case FOREIGN_CATALOG_NAME: + return isSetForeignCatalogName(); + case FOREIGN_SCHEMA_NAME: + return isSetForeignSchemaName(); + case FOREIGN_TABLE_NAME: + return isSetForeignTableName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCrossReferenceReq) + return this.equals((TGetCrossReferenceReq)that); + return false; + } + + public boolean equals(TGetCrossReferenceReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_parentCatalogName = true && this.isSetParentCatalogName(); + boolean that_present_parentCatalogName = true && that.isSetParentCatalogName(); + if (this_present_parentCatalogName || that_present_parentCatalogName) { + if (!(this_present_parentCatalogName && that_present_parentCatalogName)) + return false; + if (!this.parentCatalogName.equals(that.parentCatalogName)) + return false; + } + + boolean this_present_parentSchemaName = true && this.isSetParentSchemaName(); + boolean that_present_parentSchemaName = true && that.isSetParentSchemaName(); + if (this_present_parentSchemaName || that_present_parentSchemaName) { + if (!(this_present_parentSchemaName && that_present_parentSchemaName)) + return false; + if (!this.parentSchemaName.equals(that.parentSchemaName)) + return false; + } + + boolean this_present_parentTableName = true && this.isSetParentTableName(); + boolean that_present_parentTableName = true && that.isSetParentTableName(); + if (this_present_parentTableName || that_present_parentTableName) { + if (!(this_present_parentTableName && that_present_parentTableName)) + return false; + if (!this.parentTableName.equals(that.parentTableName)) + return false; + } + + boolean this_present_foreignCatalogName = true && this.isSetForeignCatalogName(); + boolean that_present_foreignCatalogName = true && that.isSetForeignCatalogName(); + if (this_present_foreignCatalogName || that_present_foreignCatalogName) { + if (!(this_present_foreignCatalogName && that_present_foreignCatalogName)) + return false; + if (!this.foreignCatalogName.equals(that.foreignCatalogName)) + return false; + } + + boolean this_present_foreignSchemaName = true && this.isSetForeignSchemaName(); + boolean that_present_foreignSchemaName = true && that.isSetForeignSchemaName(); + if (this_present_foreignSchemaName || that_present_foreignSchemaName) { + if (!(this_present_foreignSchemaName && that_present_foreignSchemaName)) + return false; + if (!this.foreignSchemaName.equals(that.foreignSchemaName)) + return false; + } + + boolean this_present_foreignTableName = true && this.isSetForeignTableName(); + boolean that_present_foreignTableName = true && that.isSetForeignTableName(); + if (this_present_foreignTableName || that_present_foreignTableName) { + if (!(this_present_foreignTableName && that_present_foreignTableName)) + return false; + if (!this.foreignTableName.equals(that.foreignTableName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_parentCatalogName = true && (isSetParentCatalogName()); + list.add(present_parentCatalogName); + if (present_parentCatalogName) + list.add(parentCatalogName); + + boolean present_parentSchemaName = true && (isSetParentSchemaName()); + list.add(present_parentSchemaName); + if (present_parentSchemaName) + list.add(parentSchemaName); + + boolean present_parentTableName = true && (isSetParentTableName()); + list.add(present_parentTableName); + if (present_parentTableName) + list.add(parentTableName); + + boolean present_foreignCatalogName = true && (isSetForeignCatalogName()); + list.add(present_foreignCatalogName); + if (present_foreignCatalogName) + list.add(foreignCatalogName); + + boolean present_foreignSchemaName = true && (isSetForeignSchemaName()); + list.add(present_foreignSchemaName); + if (present_foreignSchemaName) + list.add(foreignSchemaName); + + boolean present_foreignTableName = true && (isSetForeignTableName()); + list.add(present_foreignTableName); + if (present_foreignTableName) + list.add(foreignTableName); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetCrossReferenceReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetParentCatalogName()).compareTo(other.isSetParentCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetParentCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parentCatalogName, other.parentCatalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetParentSchemaName()).compareTo(other.isSetParentSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetParentSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parentSchemaName, other.parentSchemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetParentTableName()).compareTo(other.isSetParentTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetParentTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.parentTableName, other.parentTableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetForeignCatalogName()).compareTo(other.isSetForeignCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetForeignCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreignCatalogName, other.foreignCatalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetForeignSchemaName()).compareTo(other.isSetForeignSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetForeignSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreignSchemaName, other.foreignSchemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetForeignTableName()).compareTo(other.isSetForeignTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetForeignTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.foreignTableName, other.foreignTableName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCrossReferenceReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetParentCatalogName()) { + if (!first) sb.append(", "); + sb.append("parentCatalogName:"); + if (this.parentCatalogName == null) { + sb.append("null"); + } else { + sb.append(this.parentCatalogName); + } + first = false; + } + if (isSetParentSchemaName()) { + if (!first) sb.append(", "); + sb.append("parentSchemaName:"); + if (this.parentSchemaName == null) { + sb.append("null"); + } else { + sb.append(this.parentSchemaName); + } + first = false; + } + if (isSetParentTableName()) { + if (!first) sb.append(", "); + sb.append("parentTableName:"); + if (this.parentTableName == null) { + sb.append("null"); + } else { + sb.append(this.parentTableName); + } + first = false; + } + if (isSetForeignCatalogName()) { + if (!first) sb.append(", "); + sb.append("foreignCatalogName:"); + if (this.foreignCatalogName == null) { + sb.append("null"); + } else { + sb.append(this.foreignCatalogName); + } + first = false; + } + if (isSetForeignSchemaName()) { + if (!first) sb.append(", "); + sb.append("foreignSchemaName:"); + if (this.foreignSchemaName == null) { + sb.append("null"); + } else { + sb.append(this.foreignSchemaName); + } + first = false; + } + if (isSetForeignTableName()) { + if (!first) sb.append(", "); + sb.append("foreignTableName:"); + if (this.foreignTableName == null) { + sb.append("null"); + } else { + sb.append(this.foreignTableName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCrossReferenceReqStandardSchemeFactory implements SchemeFactory { + public TGetCrossReferenceReqStandardScheme getScheme() { + return new TGetCrossReferenceReqStandardScheme(); + } + } + + private static class TGetCrossReferenceReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCrossReferenceReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // PARENT_CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.parentCatalogName = iprot.readString(); + struct.setParentCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PARENT_SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.parentSchemaName = iprot.readString(); + struct.setParentSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // PARENT_TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.parentTableName = iprot.readString(); + struct.setParentTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // FOREIGN_CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.foreignCatalogName = iprot.readString(); + struct.setForeignCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // FOREIGN_SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.foreignSchemaName = iprot.readString(); + struct.setForeignSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // FOREIGN_TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.foreignTableName = iprot.readString(); + struct.setForeignTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCrossReferenceReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.parentCatalogName != null) { + if (struct.isSetParentCatalogName()) { + oprot.writeFieldBegin(PARENT_CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.parentCatalogName); + oprot.writeFieldEnd(); + } + } + if (struct.parentSchemaName != null) { + if (struct.isSetParentSchemaName()) { + oprot.writeFieldBegin(PARENT_SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.parentSchemaName); + oprot.writeFieldEnd(); + } + } + if (struct.parentTableName != null) { + if (struct.isSetParentTableName()) { + oprot.writeFieldBegin(PARENT_TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.parentTableName); + oprot.writeFieldEnd(); + } + } + if (struct.foreignCatalogName != null) { + if (struct.isSetForeignCatalogName()) { + oprot.writeFieldBegin(FOREIGN_CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.foreignCatalogName); + oprot.writeFieldEnd(); + } + } + if (struct.foreignSchemaName != null) { + if (struct.isSetForeignSchemaName()) { + oprot.writeFieldBegin(FOREIGN_SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.foreignSchemaName); + oprot.writeFieldEnd(); + } + } + if (struct.foreignTableName != null) { + if (struct.isSetForeignTableName()) { + oprot.writeFieldBegin(FOREIGN_TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.foreignTableName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCrossReferenceReqTupleSchemeFactory implements SchemeFactory { + public TGetCrossReferenceReqTupleScheme getScheme() { + return new TGetCrossReferenceReqTupleScheme(); + } + } + + private static class TGetCrossReferenceReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCrossReferenceReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetParentCatalogName()) { + optionals.set(0); + } + if (struct.isSetParentSchemaName()) { + optionals.set(1); + } + if (struct.isSetParentTableName()) { + optionals.set(2); + } + if (struct.isSetForeignCatalogName()) { + optionals.set(3); + } + if (struct.isSetForeignSchemaName()) { + optionals.set(4); + } + if (struct.isSetForeignTableName()) { + optionals.set(5); + } + oprot.writeBitSet(optionals, 6); + if (struct.isSetParentCatalogName()) { + oprot.writeString(struct.parentCatalogName); + } + if (struct.isSetParentSchemaName()) { + oprot.writeString(struct.parentSchemaName); + } + if (struct.isSetParentTableName()) { + oprot.writeString(struct.parentTableName); + } + if (struct.isSetForeignCatalogName()) { + oprot.writeString(struct.foreignCatalogName); + } + if (struct.isSetForeignSchemaName()) { + oprot.writeString(struct.foreignSchemaName); + } + if (struct.isSetForeignTableName()) { + oprot.writeString(struct.foreignTableName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCrossReferenceReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(6); + if (incoming.get(0)) { + struct.parentCatalogName = iprot.readString(); + struct.setParentCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.parentSchemaName = iprot.readString(); + struct.setParentSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.parentTableName = iprot.readString(); + struct.setParentTableNameIsSet(true); + } + if (incoming.get(3)) { + struct.foreignCatalogName = iprot.readString(); + struct.setForeignCatalogNameIsSet(true); + } + if (incoming.get(4)) { + struct.foreignSchemaName = iprot.readString(); + struct.setForeignSchemaNameIsSet(true); + } + if (incoming.get(5)) { + struct.foreignTableName = iprot.readString(); + struct.setForeignTableNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java new file mode 100644 index 0000000000000..1bfe6d192df06 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetCrossReferenceResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetCrossReferenceResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetCrossReferenceResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetCrossReferenceRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetCrossReferenceRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetCrossReferenceResp.class, metaDataMap); + } + + public TGetCrossReferenceResp() { + } + + public TGetCrossReferenceResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetCrossReferenceResp(TGetCrossReferenceResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetCrossReferenceResp deepCopy() { + return new TGetCrossReferenceResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetCrossReferenceResp) + return this.equals((TGetCrossReferenceResp)that); + return false; + } + + public boolean equals(TGetCrossReferenceResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetCrossReferenceResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetCrossReferenceResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetCrossReferenceRespStandardSchemeFactory implements SchemeFactory { + public TGetCrossReferenceRespStandardScheme getScheme() { + return new TGetCrossReferenceRespStandardScheme(); + } + } + + private static class TGetCrossReferenceRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetCrossReferenceResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetCrossReferenceResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetCrossReferenceRespTupleSchemeFactory implements SchemeFactory { + public TGetCrossReferenceRespTupleScheme getScheme() { + return new TGetCrossReferenceRespTupleScheme(); + } + } + + private static class TGetCrossReferenceRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetCrossReferenceResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetCrossReferenceResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java new file mode 100644 index 0000000000000..e3e28c5860522 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenReq.java @@ -0,0 +1,596 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OWNER_FIELD_DESC = new org.apache.thrift.protocol.TField("owner", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField RENEWER_FIELD_DESC = new org.apache.thrift.protocol.TField("renewer", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String owner; // required + private String renewer; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + OWNER((short)2, "owner"), + RENEWER((short)3, "renewer"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // OWNER + return OWNER; + case 3: // RENEWER + return RENEWER; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.OWNER, new org.apache.thrift.meta_data.FieldMetaData("owner", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.RENEWER, new org.apache.thrift.meta_data.FieldMetaData("renewer", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenReq.class, metaDataMap); + } + + public TGetDelegationTokenReq() { + } + + public TGetDelegationTokenReq( + TSessionHandle sessionHandle, + String owner, + String renewer) + { + this(); + this.sessionHandle = sessionHandle; + this.owner = owner; + this.renewer = renewer; + } + + /** + * Performs a deep copy on other. + */ + public TGetDelegationTokenReq(TGetDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetOwner()) { + this.owner = other.owner; + } + if (other.isSetRenewer()) { + this.renewer = other.renewer; + } + } + + public TGetDelegationTokenReq deepCopy() { + return new TGetDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.owner = null; + this.renewer = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getOwner() { + return this.owner; + } + + public void setOwner(String owner) { + this.owner = owner; + } + + public void unsetOwner() { + this.owner = null; + } + + /** Returns true if field owner is set (has been assigned a value) and false otherwise */ + public boolean isSetOwner() { + return this.owner != null; + } + + public void setOwnerIsSet(boolean value) { + if (!value) { + this.owner = null; + } + } + + public String getRenewer() { + return this.renewer; + } + + public void setRenewer(String renewer) { + this.renewer = renewer; + } + + public void unsetRenewer() { + this.renewer = null; + } + + /** Returns true if field renewer is set (has been assigned a value) and false otherwise */ + public boolean isSetRenewer() { + return this.renewer != null; + } + + public void setRenewerIsSet(boolean value) { + if (!value) { + this.renewer = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case OWNER: + if (value == null) { + unsetOwner(); + } else { + setOwner((String)value); + } + break; + + case RENEWER: + if (value == null) { + unsetRenewer(); + } else { + setRenewer((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case OWNER: + return getOwner(); + + case RENEWER: + return getRenewer(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case OWNER: + return isSetOwner(); + case RENEWER: + return isSetRenewer(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetDelegationTokenReq) + return this.equals((TGetDelegationTokenReq)that); + return false; + } + + public boolean equals(TGetDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_owner = true && this.isSetOwner(); + boolean that_present_owner = true && that.isSetOwner(); + if (this_present_owner || that_present_owner) { + if (!(this_present_owner && that_present_owner)) + return false; + if (!this.owner.equals(that.owner)) + return false; + } + + boolean this_present_renewer = true && this.isSetRenewer(); + boolean that_present_renewer = true && that.isSetRenewer(); + if (this_present_renewer || that_present_renewer) { + if (!(this_present_renewer && that_present_renewer)) + return false; + if (!this.renewer.equals(that.renewer)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_owner = true && (isSetOwner()); + list.add(present_owner); + if (present_owner) + list.add(owner); + + boolean present_renewer = true && (isSetRenewer()); + list.add(present_renewer); + if (present_renewer) + list.add(renewer); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOwner()).compareTo(other.isSetOwner()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOwner()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.owner, other.owner); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRenewer()).compareTo(other.isSetRenewer()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRenewer()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.renewer, other.renewer); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("owner:"); + if (this.owner == null) { + sb.append("null"); + } else { + sb.append(this.owner); + } + first = false; + if (!first) sb.append(", "); + sb.append("renewer:"); + if (this.renewer == null) { + sb.append("null"); + } else { + sb.append(this.renewer); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetOwner()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'owner' is unset! Struct:" + toString()); + } + + if (!isSetRenewer()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'renewer' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TGetDelegationTokenReqStandardScheme getScheme() { + return new TGetDelegationTokenReqStandardScheme(); + } + } + + private static class TGetDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OWNER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.owner = iprot.readString(); + struct.setOwnerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // RENEWER + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.renewer = iprot.readString(); + struct.setRenewerIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.owner != null) { + oprot.writeFieldBegin(OWNER_FIELD_DESC); + oprot.writeString(struct.owner); + oprot.writeFieldEnd(); + } + if (struct.renewer != null) { + oprot.writeFieldBegin(RENEWER_FIELD_DESC); + oprot.writeString(struct.renewer); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TGetDelegationTokenReqTupleScheme getScheme() { + return new TGetDelegationTokenReqTupleScheme(); + } + } + + private static class TGetDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.owner); + oprot.writeString(struct.renewer); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.owner = iprot.readString(); + struct.setOwnerIsSet(true); + struct.renewer = iprot.readString(); + struct.setRenewerIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java new file mode 100644 index 0000000000000..6ef2acbbd9435 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetDelegationTokenResp.java @@ -0,0 +1,504 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + private String delegationToken; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.DELEGATION_TOKEN}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetDelegationTokenResp.class, metaDataMap); + } + + public TGetDelegationTokenResp() { + } + + public TGetDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetDelegationTokenResp(TGetDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TGetDelegationTokenResp deepCopy() { + return new TGetDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + this.delegationToken = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetDelegationTokenResp) + return this.equals((TGetDelegationTokenResp)that); + return false; + } + + public boolean equals(TGetDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_delegationToken = true && (isSetDelegationToken()); + list.add(present_delegationToken); + if (present_delegationToken) + list.add(delegationToken); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(other.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, other.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetDelegationToken()) { + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TGetDelegationTokenRespStandardScheme getScheme() { + return new TGetDelegationTokenRespStandardScheme(); + } + } + + private static class TGetDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + if (struct.isSetDelegationToken()) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TGetDelegationTokenRespTupleScheme getScheme() { + return new TGetDelegationTokenRespTupleScheme(); + } + } + + private static class TGetDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetDelegationToken()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetDelegationToken()) { + oprot.writeString(struct.delegationToken); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java new file mode 100644 index 0000000000000..ad4f8a5b031e8 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsReq.java @@ -0,0 +1,711 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetFunctionsReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField FUNCTION_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("functionName", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetFunctionsReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetFunctionsReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String functionName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + FUNCTION_NAME((short)4, "functionName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // FUNCTION_NAME + return FUNCTION_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.FUNCTION_NAME, new org.apache.thrift.meta_data.FieldMetaData("functionName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsReq.class, metaDataMap); + } + + public TGetFunctionsReq() { + } + + public TGetFunctionsReq( + TSessionHandle sessionHandle, + String functionName) + { + this(); + this.sessionHandle = sessionHandle; + this.functionName = functionName; + } + + /** + * Performs a deep copy on other. + */ + public TGetFunctionsReq(TGetFunctionsReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetFunctionName()) { + this.functionName = other.functionName; + } + } + + public TGetFunctionsReq deepCopy() { + return new TGetFunctionsReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.functionName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getFunctionName() { + return this.functionName; + } + + public void setFunctionName(String functionName) { + this.functionName = functionName; + } + + public void unsetFunctionName() { + this.functionName = null; + } + + /** Returns true if field functionName is set (has been assigned a value) and false otherwise */ + public boolean isSetFunctionName() { + return this.functionName != null; + } + + public void setFunctionNameIsSet(boolean value) { + if (!value) { + this.functionName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case FUNCTION_NAME: + if (value == null) { + unsetFunctionName(); + } else { + setFunctionName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case FUNCTION_NAME: + return getFunctionName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case FUNCTION_NAME: + return isSetFunctionName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetFunctionsReq) + return this.equals((TGetFunctionsReq)that); + return false; + } + + public boolean equals(TGetFunctionsReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_functionName = true && this.isSetFunctionName(); + boolean that_present_functionName = true && that.isSetFunctionName(); + if (this_present_functionName || that_present_functionName) { + if (!(this_present_functionName && that_present_functionName)) + return false; + if (!this.functionName.equals(that.functionName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + list.add(present_catalogName); + if (present_catalogName) + list.add(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + list.add(present_schemaName); + if (present_schemaName) + list.add(schemaName); + + boolean present_functionName = true && (isSetFunctionName()); + list.add(present_functionName); + if (present_functionName) + list.add(functionName); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetFunctionsReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFunctionName()).compareTo(other.isSetFunctionName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFunctionName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.functionName, other.functionName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetFunctionsReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (!first) sb.append(", "); + sb.append("functionName:"); + if (this.functionName == null) { + sb.append("null"); + } else { + sb.append(this.functionName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetFunctionName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'functionName' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetFunctionsReqStandardSchemeFactory implements SchemeFactory { + public TGetFunctionsReqStandardScheme getScheme() { + return new TGetFunctionsReqStandardScheme(); + } + } + + private static class TGetFunctionsReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // FUNCTION_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.functionName = iprot.readString(); + struct.setFunctionNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.functionName != null) { + oprot.writeFieldBegin(FUNCTION_NAME_FIELD_DESC); + oprot.writeString(struct.functionName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetFunctionsReqTupleSchemeFactory implements SchemeFactory { + public TGetFunctionsReqTupleScheme getScheme() { + return new TGetFunctionsReqTupleScheme(); + } + } + + private static class TGetFunctionsReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.functionName); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.functionName = iprot.readString(); + struct.setFunctionNameIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java new file mode 100644 index 0000000000000..ead37fb91cc2f --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetFunctionsResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetFunctionsResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetFunctionsResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetFunctionsRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetFunctionsRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetFunctionsResp.class, metaDataMap); + } + + public TGetFunctionsResp() { + } + + public TGetFunctionsResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetFunctionsResp(TGetFunctionsResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetFunctionsResp deepCopy() { + return new TGetFunctionsResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetFunctionsResp) + return this.equals((TGetFunctionsResp)that); + return false; + } + + public boolean equals(TGetFunctionsResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetFunctionsResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetFunctionsResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetFunctionsRespStandardSchemeFactory implements SchemeFactory { + public TGetFunctionsRespStandardScheme getScheme() { + return new TGetFunctionsRespStandardScheme(); + } + } + + private static class TGetFunctionsRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetFunctionsRespTupleSchemeFactory implements SchemeFactory { + public TGetFunctionsRespTupleScheme getScheme() { + return new TGetFunctionsRespTupleScheme(); + } + } + + private static class TGetFunctionsRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetFunctionsResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java new file mode 100644 index 0000000000000..b319b70e5eba5 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoReq.java @@ -0,0 +1,507 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField INFO_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoType", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetInfoReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetInfoReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private TGetInfoType infoType; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + /** + * + * @see TGetInfoType + */ + INFO_TYPE((short)2, "infoType"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // INFO_TYPE + return INFO_TYPE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.INFO_TYPE, new org.apache.thrift.meta_data.FieldMetaData("infoType", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TGetInfoType.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoReq.class, metaDataMap); + } + + public TGetInfoReq() { + } + + public TGetInfoReq( + TSessionHandle sessionHandle, + TGetInfoType infoType) + { + this(); + this.sessionHandle = sessionHandle; + this.infoType = infoType; + } + + /** + * Performs a deep copy on other. + */ + public TGetInfoReq(TGetInfoReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetInfoType()) { + this.infoType = other.infoType; + } + } + + public TGetInfoReq deepCopy() { + return new TGetInfoReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.infoType = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + /** + * + * @see TGetInfoType + */ + public TGetInfoType getInfoType() { + return this.infoType; + } + + /** + * + * @see TGetInfoType + */ + public void setInfoType(TGetInfoType infoType) { + this.infoType = infoType; + } + + public void unsetInfoType() { + this.infoType = null; + } + + /** Returns true if field infoType is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoType() { + return this.infoType != null; + } + + public void setInfoTypeIsSet(boolean value) { + if (!value) { + this.infoType = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case INFO_TYPE: + if (value == null) { + unsetInfoType(); + } else { + setInfoType((TGetInfoType)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case INFO_TYPE: + return getInfoType(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case INFO_TYPE: + return isSetInfoType(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetInfoReq) + return this.equals((TGetInfoReq)that); + return false; + } + + public boolean equals(TGetInfoReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_infoType = true && this.isSetInfoType(); + boolean that_present_infoType = true && that.isSetInfoType(); + if (this_present_infoType || that_present_infoType) { + if (!(this_present_infoType && that_present_infoType)) + return false; + if (!this.infoType.equals(that.infoType)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_infoType = true && (isSetInfoType()); + list.add(present_infoType); + if (present_infoType) + list.add(infoType.getValue()); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetInfoReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoType()).compareTo(other.isSetInfoType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoType, other.infoType); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetInfoReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("infoType:"); + if (this.infoType == null) { + sb.append("null"); + } else { + sb.append(this.infoType); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetInfoType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoType' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetInfoReqStandardSchemeFactory implements SchemeFactory { + public TGetInfoReqStandardScheme getScheme() { + return new TGetInfoReqStandardScheme(); + } + } + + private static class TGetInfoReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.infoType = org.apache.hive.service.rpc.thrift.TGetInfoType.findByValue(iprot.readI32()); + struct.setInfoTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.infoType != null) { + oprot.writeFieldBegin(INFO_TYPE_FIELD_DESC); + oprot.writeI32(struct.infoType.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetInfoReqTupleSchemeFactory implements SchemeFactory { + public TGetInfoReqTupleScheme getScheme() { + return new TGetInfoReqTupleScheme(); + } + } + + private static class TGetInfoReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeI32(struct.infoType.getValue()); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.infoType = org.apache.hive.service.rpc.thrift.TGetInfoType.findByValue(iprot.readI32()); + struct.setInfoTypeIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java new file mode 100644 index 0000000000000..9be810b024987 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoResp.java @@ -0,0 +1,497 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField INFO_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("infoValue", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetInfoRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetInfoRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TGetInfoValue infoValue; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + INFO_VALUE((short)2, "infoValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // INFO_VALUE + return INFO_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.INFO_VALUE, new org.apache.thrift.meta_data.FieldMetaData("infoValue", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TGetInfoValue.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoResp.class, metaDataMap); + } + + public TGetInfoResp() { + } + + public TGetInfoResp( + TStatus status, + TGetInfoValue infoValue) + { + this(); + this.status = status; + this.infoValue = infoValue; + } + + /** + * Performs a deep copy on other. + */ + public TGetInfoResp(TGetInfoResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetInfoValue()) { + this.infoValue = new TGetInfoValue(other.infoValue); + } + } + + public TGetInfoResp deepCopy() { + return new TGetInfoResp(this); + } + + @Override + public void clear() { + this.status = null; + this.infoValue = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TGetInfoValue getInfoValue() { + return this.infoValue; + } + + public void setInfoValue(TGetInfoValue infoValue) { + this.infoValue = infoValue; + } + + public void unsetInfoValue() { + this.infoValue = null; + } + + /** Returns true if field infoValue is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoValue() { + return this.infoValue != null; + } + + public void setInfoValueIsSet(boolean value) { + if (!value) { + this.infoValue = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case INFO_VALUE: + if (value == null) { + unsetInfoValue(); + } else { + setInfoValue((TGetInfoValue)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case INFO_VALUE: + return getInfoValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case INFO_VALUE: + return isSetInfoValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetInfoResp) + return this.equals((TGetInfoResp)that); + return false; + } + + public boolean equals(TGetInfoResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_infoValue = true && this.isSetInfoValue(); + boolean that_present_infoValue = true && that.isSetInfoValue(); + if (this_present_infoValue || that_present_infoValue) { + if (!(this_present_infoValue && that_present_infoValue)) + return false; + if (!this.infoValue.equals(that.infoValue)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_infoValue = true && (isSetInfoValue()); + list.add(present_infoValue); + if (present_infoValue) + list.add(infoValue); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetInfoResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoValue()).compareTo(other.isSetInfoValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoValue, other.infoValue); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetInfoResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("infoValue:"); + if (this.infoValue == null) { + sb.append("null"); + } else { + sb.append(this.infoValue); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetInfoValue()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'infoValue' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetInfoRespStandardSchemeFactory implements SchemeFactory { + public TGetInfoRespStandardScheme getScheme() { + return new TGetInfoRespStandardScheme(); + } + } + + private static class TGetInfoRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetInfoResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.infoValue = new TGetInfoValue(); + struct.infoValue.read(iprot); + struct.setInfoValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetInfoResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.infoValue != null) { + oprot.writeFieldBegin(INFO_VALUE_FIELD_DESC); + struct.infoValue.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetInfoRespTupleSchemeFactory implements SchemeFactory { + public TGetInfoRespTupleScheme getScheme() { + return new TGetInfoRespTupleScheme(); + } + } + + private static class TGetInfoRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + struct.infoValue.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + struct.infoValue = new TGetInfoValue(); + struct.infoValue.read(iprot); + struct.setInfoValueIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java new file mode 100644 index 0000000000000..5b219b62656d7 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoType.java @@ -0,0 +1,180 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TGetInfoType implements org.apache.thrift.TEnum { + CLI_MAX_DRIVER_CONNECTIONS(0), + CLI_MAX_CONCURRENT_ACTIVITIES(1), + CLI_DATA_SOURCE_NAME(2), + CLI_FETCH_DIRECTION(8), + CLI_SERVER_NAME(13), + CLI_SEARCH_PATTERN_ESCAPE(14), + CLI_DBMS_NAME(17), + CLI_DBMS_VER(18), + CLI_ACCESSIBLE_TABLES(19), + CLI_ACCESSIBLE_PROCEDURES(20), + CLI_CURSOR_COMMIT_BEHAVIOR(23), + CLI_DATA_SOURCE_READ_ONLY(25), + CLI_DEFAULT_TXN_ISOLATION(26), + CLI_IDENTIFIER_CASE(28), + CLI_IDENTIFIER_QUOTE_CHAR(29), + CLI_MAX_COLUMN_NAME_LEN(30), + CLI_MAX_CURSOR_NAME_LEN(31), + CLI_MAX_SCHEMA_NAME_LEN(32), + CLI_MAX_CATALOG_NAME_LEN(34), + CLI_MAX_TABLE_NAME_LEN(35), + CLI_SCROLL_CONCURRENCY(43), + CLI_TXN_CAPABLE(46), + CLI_USER_NAME(47), + CLI_TXN_ISOLATION_OPTION(72), + CLI_INTEGRITY(73), + CLI_GETDATA_EXTENSIONS(81), + CLI_NULL_COLLATION(85), + CLI_ALTER_TABLE(86), + CLI_ORDER_BY_COLUMNS_IN_SELECT(90), + CLI_SPECIAL_CHARACTERS(94), + CLI_MAX_COLUMNS_IN_GROUP_BY(97), + CLI_MAX_COLUMNS_IN_INDEX(98), + CLI_MAX_COLUMNS_IN_ORDER_BY(99), + CLI_MAX_COLUMNS_IN_SELECT(100), + CLI_MAX_COLUMNS_IN_TABLE(101), + CLI_MAX_INDEX_SIZE(102), + CLI_MAX_ROW_SIZE(104), + CLI_MAX_STATEMENT_LEN(105), + CLI_MAX_TABLES_IN_SELECT(106), + CLI_MAX_USER_NAME_LEN(107), + CLI_OJ_CAPABILITIES(115), + CLI_XOPEN_CLI_YEAR(10000), + CLI_CURSOR_SENSITIVITY(10001), + CLI_DESCRIBE_PARAMETER(10002), + CLI_CATALOG_NAME(10003), + CLI_COLLATION_SEQ(10004), + CLI_MAX_IDENTIFIER_LEN(10005); + + private final int value; + + private TGetInfoType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TGetInfoType findByValue(int value) { + switch (value) { + case 0: + return CLI_MAX_DRIVER_CONNECTIONS; + case 1: + return CLI_MAX_CONCURRENT_ACTIVITIES; + case 2: + return CLI_DATA_SOURCE_NAME; + case 8: + return CLI_FETCH_DIRECTION; + case 13: + return CLI_SERVER_NAME; + case 14: + return CLI_SEARCH_PATTERN_ESCAPE; + case 17: + return CLI_DBMS_NAME; + case 18: + return CLI_DBMS_VER; + case 19: + return CLI_ACCESSIBLE_TABLES; + case 20: + return CLI_ACCESSIBLE_PROCEDURES; + case 23: + return CLI_CURSOR_COMMIT_BEHAVIOR; + case 25: + return CLI_DATA_SOURCE_READ_ONLY; + case 26: + return CLI_DEFAULT_TXN_ISOLATION; + case 28: + return CLI_IDENTIFIER_CASE; + case 29: + return CLI_IDENTIFIER_QUOTE_CHAR; + case 30: + return CLI_MAX_COLUMN_NAME_LEN; + case 31: + return CLI_MAX_CURSOR_NAME_LEN; + case 32: + return CLI_MAX_SCHEMA_NAME_LEN; + case 34: + return CLI_MAX_CATALOG_NAME_LEN; + case 35: + return CLI_MAX_TABLE_NAME_LEN; + case 43: + return CLI_SCROLL_CONCURRENCY; + case 46: + return CLI_TXN_CAPABLE; + case 47: + return CLI_USER_NAME; + case 72: + return CLI_TXN_ISOLATION_OPTION; + case 73: + return CLI_INTEGRITY; + case 81: + return CLI_GETDATA_EXTENSIONS; + case 85: + return CLI_NULL_COLLATION; + case 86: + return CLI_ALTER_TABLE; + case 90: + return CLI_ORDER_BY_COLUMNS_IN_SELECT; + case 94: + return CLI_SPECIAL_CHARACTERS; + case 97: + return CLI_MAX_COLUMNS_IN_GROUP_BY; + case 98: + return CLI_MAX_COLUMNS_IN_INDEX; + case 99: + return CLI_MAX_COLUMNS_IN_ORDER_BY; + case 100: + return CLI_MAX_COLUMNS_IN_SELECT; + case 101: + return CLI_MAX_COLUMNS_IN_TABLE; + case 102: + return CLI_MAX_INDEX_SIZE; + case 104: + return CLI_MAX_ROW_SIZE; + case 105: + return CLI_MAX_STATEMENT_LEN; + case 106: + return CLI_MAX_TABLES_IN_SELECT; + case 107: + return CLI_MAX_USER_NAME_LEN; + case 115: + return CLI_OJ_CAPABILITIES; + case 10000: + return CLI_XOPEN_CLI_YEAR; + case 10001: + return CLI_CURSOR_SENSITIVITY; + case 10002: + return CLI_DESCRIBE_PARAMETER; + case 10003: + return CLI_CATALOG_NAME; + case 10004: + return CLI_COLLATION_SEQ; + case 10005: + return CLI_MAX_IDENTIFIER_LEN; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java new file mode 100644 index 0000000000000..8e3045a58e5ac --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetInfoValue.java @@ -0,0 +1,597 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TGetInfoValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetInfoValue"); + private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SMALL_INT_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("smallIntValue", org.apache.thrift.protocol.TType.I16, (short)2); + private static final org.apache.thrift.protocol.TField INTEGER_BITMASK_FIELD_DESC = new org.apache.thrift.protocol.TField("integerBitmask", org.apache.thrift.protocol.TType.I32, (short)3); + private static final org.apache.thrift.protocol.TField INTEGER_FLAG_FIELD_DESC = new org.apache.thrift.protocol.TField("integerFlag", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField BINARY_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryValue", org.apache.thrift.protocol.TType.I32, (short)5); + private static final org.apache.thrift.protocol.TField LEN_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("lenValue", org.apache.thrift.protocol.TType.I64, (short)6); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STRING_VALUE((short)1, "stringValue"), + SMALL_INT_VALUE((short)2, "smallIntValue"), + INTEGER_BITMASK((short)3, "integerBitmask"), + INTEGER_FLAG((short)4, "integerFlag"), + BINARY_VALUE((short)5, "binaryValue"), + LEN_VALUE((short)6, "lenValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STRING_VALUE + return STRING_VALUE; + case 2: // SMALL_INT_VALUE + return SMALL_INT_VALUE; + case 3: // INTEGER_BITMASK + return INTEGER_BITMASK; + case 4: // INTEGER_FLAG + return INTEGER_FLAG; + case 5: // BINARY_VALUE + return BINARY_VALUE; + case 6: // LEN_VALUE + return LEN_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.SMALL_INT_VALUE, new org.apache.thrift.meta_data.FieldMetaData("smallIntValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + tmpMap.put(_Fields.INTEGER_BITMASK, new org.apache.thrift.meta_data.FieldMetaData("integerBitmask", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.INTEGER_FLAG, new org.apache.thrift.meta_data.FieldMetaData("integerFlag", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.BINARY_VALUE, new org.apache.thrift.meta_data.FieldMetaData("binaryValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.LEN_VALUE, new org.apache.thrift.meta_data.FieldMetaData("lenValue", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetInfoValue.class, metaDataMap); + } + + public TGetInfoValue() { + super(); + } + + public TGetInfoValue(TGetInfoValue._Fields setField, Object value) { + super(setField, value); + } + + public TGetInfoValue(TGetInfoValue other) { + super(other); + } + public TGetInfoValue deepCopy() { + return new TGetInfoValue(this); + } + + public static TGetInfoValue stringValue(String value) { + TGetInfoValue x = new TGetInfoValue(); + x.setStringValue(value); + return x; + } + + public static TGetInfoValue smallIntValue(short value) { + TGetInfoValue x = new TGetInfoValue(); + x.setSmallIntValue(value); + return x; + } + + public static TGetInfoValue integerBitmask(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setIntegerBitmask(value); + return x; + } + + public static TGetInfoValue integerFlag(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setIntegerFlag(value); + return x; + } + + public static TGetInfoValue binaryValue(int value) { + TGetInfoValue x = new TGetInfoValue(); + x.setBinaryValue(value); + return x; + } + + public static TGetInfoValue lenValue(long value) { + TGetInfoValue x = new TGetInfoValue(); + x.setLenValue(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case STRING_VALUE: + if (value instanceof String) { + break; + } + throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); + case SMALL_INT_VALUE: + if (value instanceof Short) { + break; + } + throw new ClassCastException("Was expecting value of type Short for field 'smallIntValue', but got " + value.getClass().getSimpleName()); + case INTEGER_BITMASK: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'integerBitmask', but got " + value.getClass().getSimpleName()); + case INTEGER_FLAG: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'integerFlag', but got " + value.getClass().getSimpleName()); + case BINARY_VALUE: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'binaryValue', but got " + value.getClass().getSimpleName()); + case LEN_VALUE: + if (value instanceof Long) { + break; + } + throw new ClassCastException("Was expecting value of type Long for field 'lenValue', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case STRING_VALUE: + if (field.type == STRING_VALUE_FIELD_DESC.type) { + String stringValue; + stringValue = iprot.readString(); + return stringValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case SMALL_INT_VALUE: + if (field.type == SMALL_INT_VALUE_FIELD_DESC.type) { + Short smallIntValue; + smallIntValue = iprot.readI16(); + return smallIntValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case INTEGER_BITMASK: + if (field.type == INTEGER_BITMASK_FIELD_DESC.type) { + Integer integerBitmask; + integerBitmask = iprot.readI32(); + return integerBitmask; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case INTEGER_FLAG: + if (field.type == INTEGER_FLAG_FIELD_DESC.type) { + Integer integerFlag; + integerFlag = iprot.readI32(); + return integerFlag; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case BINARY_VALUE: + if (field.type == BINARY_VALUE_FIELD_DESC.type) { + Integer binaryValue; + binaryValue = iprot.readI32(); + return binaryValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case LEN_VALUE: + if (field.type == LEN_VALUE_FIELD_DESC.type) { + Long lenValue; + lenValue = iprot.readI64(); + return lenValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + case SMALL_INT_VALUE: + Short smallIntValue = (Short)value_; + oprot.writeI16(smallIntValue); + return; + case INTEGER_BITMASK: + Integer integerBitmask = (Integer)value_; + oprot.writeI32(integerBitmask); + return; + case INTEGER_FLAG: + Integer integerFlag = (Integer)value_; + oprot.writeI32(integerFlag); + return; + case BINARY_VALUE: + Integer binaryValue = (Integer)value_; + oprot.writeI32(binaryValue); + return; + case LEN_VALUE: + Long lenValue = (Long)value_; + oprot.writeI64(lenValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case STRING_VALUE: + String stringValue; + stringValue = iprot.readString(); + return stringValue; + case SMALL_INT_VALUE: + Short smallIntValue; + smallIntValue = iprot.readI16(); + return smallIntValue; + case INTEGER_BITMASK: + Integer integerBitmask; + integerBitmask = iprot.readI32(); + return integerBitmask; + case INTEGER_FLAG: + Integer integerFlag; + integerFlag = iprot.readI32(); + return integerFlag; + case BINARY_VALUE: + Integer binaryValue; + binaryValue = iprot.readI32(); + return binaryValue; + case LEN_VALUE: + Long lenValue; + lenValue = iprot.readI64(); + return lenValue; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + case SMALL_INT_VALUE: + Short smallIntValue = (Short)value_; + oprot.writeI16(smallIntValue); + return; + case INTEGER_BITMASK: + Integer integerBitmask = (Integer)value_; + oprot.writeI32(integerBitmask); + return; + case INTEGER_FLAG: + Integer integerFlag = (Integer)value_; + oprot.writeI32(integerFlag); + return; + case BINARY_VALUE: + Integer binaryValue = (Integer)value_; + oprot.writeI32(binaryValue); + return; + case LEN_VALUE: + Long lenValue = (Long)value_; + oprot.writeI64(lenValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case STRING_VALUE: + return STRING_VALUE_FIELD_DESC; + case SMALL_INT_VALUE: + return SMALL_INT_VALUE_FIELD_DESC; + case INTEGER_BITMASK: + return INTEGER_BITMASK_FIELD_DESC; + case INTEGER_FLAG: + return INTEGER_FLAG_FIELD_DESC; + case BINARY_VALUE: + return BINARY_VALUE_FIELD_DESC; + case LEN_VALUE: + return LEN_VALUE_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public String getStringValue() { + if (getSetField() == _Fields.STRING_VALUE) { + return (String)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringValue(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VALUE; + value_ = value; + } + + public short getSmallIntValue() { + if (getSetField() == _Fields.SMALL_INT_VALUE) { + return (Short)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'smallIntValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setSmallIntValue(short value) { + setField_ = _Fields.SMALL_INT_VALUE; + value_ = value; + } + + public int getIntegerBitmask() { + if (getSetField() == _Fields.INTEGER_BITMASK) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'integerBitmask' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setIntegerBitmask(int value) { + setField_ = _Fields.INTEGER_BITMASK; + value_ = value; + } + + public int getIntegerFlag() { + if (getSetField() == _Fields.INTEGER_FLAG) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'integerFlag' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setIntegerFlag(int value) { + setField_ = _Fields.INTEGER_FLAG; + value_ = value; + } + + public int getBinaryValue() { + if (getSetField() == _Fields.BINARY_VALUE) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'binaryValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setBinaryValue(int value) { + setField_ = _Fields.BINARY_VALUE; + value_ = value; + } + + public long getLenValue() { + if (getSetField() == _Fields.LEN_VALUE) { + return (Long)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'lenValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setLenValue(long value) { + setField_ = _Fields.LEN_VALUE; + value_ = value; + } + + public boolean isSetStringValue() { + return setField_ == _Fields.STRING_VALUE; + } + + + public boolean isSetSmallIntValue() { + return setField_ == _Fields.SMALL_INT_VALUE; + } + + + public boolean isSetIntegerBitmask() { + return setField_ == _Fields.INTEGER_BITMASK; + } + + + public boolean isSetIntegerFlag() { + return setField_ == _Fields.INTEGER_FLAG; + } + + + public boolean isSetBinaryValue() { + return setField_ == _Fields.BINARY_VALUE; + } + + + public boolean isSetLenValue() { + return setField_ == _Fields.LEN_VALUE; + } + + + public boolean equals(Object other) { + if (other instanceof TGetInfoValue) { + return equals((TGetInfoValue)other); + } else { + return false; + } + } + + public boolean equals(TGetInfoValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TGetInfoValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java new file mode 100644 index 0000000000000..af31ce2b22819 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusReq.java @@ -0,0 +1,501 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetOperationStatusReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField GET_PROGRESS_UPDATE_FIELD_DESC = new org.apache.thrift.protocol.TField("getProgressUpdate", org.apache.thrift.protocol.TType.BOOL, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetOperationStatusReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetOperationStatusReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + private boolean getProgressUpdate; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"), + GET_PROGRESS_UPDATE((short)2, "getProgressUpdate"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + case 2: // GET_PROGRESS_UPDATE + return GET_PROGRESS_UPDATE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __GETPROGRESSUPDATE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.GET_PROGRESS_UPDATE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + tmpMap.put(_Fields.GET_PROGRESS_UPDATE, new org.apache.thrift.meta_data.FieldMetaData("getProgressUpdate", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusReq.class, metaDataMap); + } + + public TGetOperationStatusReq() { + } + + public TGetOperationStatusReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetOperationStatusReq(TGetOperationStatusReq other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + this.getProgressUpdate = other.getProgressUpdate; + } + + public TGetOperationStatusReq deepCopy() { + return new TGetOperationStatusReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + setGetProgressUpdateIsSet(false); + this.getProgressUpdate = false; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public boolean isGetProgressUpdate() { + return this.getProgressUpdate; + } + + public void setGetProgressUpdate(boolean getProgressUpdate) { + this.getProgressUpdate = getProgressUpdate; + setGetProgressUpdateIsSet(true); + } + + public void unsetGetProgressUpdate() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __GETPROGRESSUPDATE_ISSET_ID); + } + + /** Returns true if field getProgressUpdate is set (has been assigned a value) and false otherwise */ + public boolean isSetGetProgressUpdate() { + return EncodingUtils.testBit(__isset_bitfield, __GETPROGRESSUPDATE_ISSET_ID); + } + + public void setGetProgressUpdateIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __GETPROGRESSUPDATE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + case GET_PROGRESS_UPDATE: + if (value == null) { + unsetGetProgressUpdate(); + } else { + setGetProgressUpdate((Boolean)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + case GET_PROGRESS_UPDATE: + return isGetProgressUpdate(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + case GET_PROGRESS_UPDATE: + return isSetGetProgressUpdate(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetOperationStatusReq) + return this.equals((TGetOperationStatusReq)that); + return false; + } + + public boolean equals(TGetOperationStatusReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + boolean this_present_getProgressUpdate = true && this.isSetGetProgressUpdate(); + boolean that_present_getProgressUpdate = true && that.isSetGetProgressUpdate(); + if (this_present_getProgressUpdate || that_present_getProgressUpdate) { + if (!(this_present_getProgressUpdate && that_present_getProgressUpdate)) + return false; + if (this.getProgressUpdate != that.getProgressUpdate) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + boolean present_getProgressUpdate = true && (isSetGetProgressUpdate()); + list.add(present_getProgressUpdate); + if (present_getProgressUpdate) + list.add(getProgressUpdate); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetOperationStatusReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetGetProgressUpdate()).compareTo(other.isSetGetProgressUpdate()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetGetProgressUpdate()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.getProgressUpdate, other.getProgressUpdate); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetOperationStatusReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + if (isSetGetProgressUpdate()) { + if (!first) sb.append(", "); + sb.append("getProgressUpdate:"); + sb.append(this.getProgressUpdate); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetOperationStatusReqStandardSchemeFactory implements SchemeFactory { + public TGetOperationStatusReqStandardScheme getScheme() { + return new TGetOperationStatusReqStandardScheme(); + } + } + + private static class TGetOperationStatusReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // GET_PROGRESS_UPDATE + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.getProgressUpdate = iprot.readBool(); + struct.setGetProgressUpdateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.isSetGetProgressUpdate()) { + oprot.writeFieldBegin(GET_PROGRESS_UPDATE_FIELD_DESC); + oprot.writeBool(struct.getProgressUpdate); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetOperationStatusReqTupleSchemeFactory implements SchemeFactory { + public TGetOperationStatusReqTupleScheme getScheme() { + return new TGetOperationStatusReqTupleScheme(); + } + } + + private static class TGetOperationStatusReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetGetProgressUpdate()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetGetProgressUpdate()) { + oprot.writeBool(struct.getProgressUpdate); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.getProgressUpdate = iprot.readBool(); + struct.setGetProgressUpdateIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java new file mode 100644 index 0000000000000..dbfbb44aa6986 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetOperationStatusResp.java @@ -0,0 +1,1342 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetOperationStatusResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetOperationStatusResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationState", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField TASK_STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("taskStatus", org.apache.thrift.protocol.TType.STRING, (short)6); + private static final org.apache.thrift.protocol.TField OPERATION_STARTED_FIELD_DESC = new org.apache.thrift.protocol.TField("operationStarted", org.apache.thrift.protocol.TType.I64, (short)7); + private static final org.apache.thrift.protocol.TField OPERATION_COMPLETED_FIELD_DESC = new org.apache.thrift.protocol.TField("operationCompleted", org.apache.thrift.protocol.TType.I64, (short)8); + private static final org.apache.thrift.protocol.TField HAS_RESULT_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("hasResultSet", org.apache.thrift.protocol.TType.BOOL, (short)9); + private static final org.apache.thrift.protocol.TField PROGRESS_UPDATE_RESPONSE_FIELD_DESC = new org.apache.thrift.protocol.TField("progressUpdateResponse", org.apache.thrift.protocol.TType.STRUCT, (short)10); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetOperationStatusRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetOperationStatusRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationState operationState; // optional + private String sqlState; // optional + private int errorCode; // optional + private String errorMessage; // optional + private String taskStatus; // optional + private long operationStarted; // optional + private long operationCompleted; // optional + private boolean hasResultSet; // optional + private TProgressUpdateResp progressUpdateResponse; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + /** + * + * @see TOperationState + */ + OPERATION_STATE((short)2, "operationState"), + SQL_STATE((short)3, "sqlState"), + ERROR_CODE((short)4, "errorCode"), + ERROR_MESSAGE((short)5, "errorMessage"), + TASK_STATUS((short)6, "taskStatus"), + OPERATION_STARTED((short)7, "operationStarted"), + OPERATION_COMPLETED((short)8, "operationCompleted"), + HAS_RESULT_SET((short)9, "hasResultSet"), + PROGRESS_UPDATE_RESPONSE((short)10, "progressUpdateResponse"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_STATE + return OPERATION_STATE; + case 3: // SQL_STATE + return SQL_STATE; + case 4: // ERROR_CODE + return ERROR_CODE; + case 5: // ERROR_MESSAGE + return ERROR_MESSAGE; + case 6: // TASK_STATUS + return TASK_STATUS; + case 7: // OPERATION_STARTED + return OPERATION_STARTED; + case 8: // OPERATION_COMPLETED + return OPERATION_COMPLETED; + case 9: // HAS_RESULT_SET + return HAS_RESULT_SET; + case 10: // PROGRESS_UPDATE_RESPONSE + return PROGRESS_UPDATE_RESPONSE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ERRORCODE_ISSET_ID = 0; + private static final int __OPERATIONSTARTED_ISSET_ID = 1; + private static final int __OPERATIONCOMPLETED_ISSET_ID = 2; + private static final int __HASRESULTSET_ISSET_ID = 3; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.OPERATION_STATE,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE,_Fields.TASK_STATUS,_Fields.OPERATION_STARTED,_Fields.OPERATION_COMPLETED,_Fields.HAS_RESULT_SET,_Fields.PROGRESS_UPDATE_RESPONSE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_STATE, new org.apache.thrift.meta_data.FieldMetaData("operationState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationState.class))); + tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.TASK_STATUS, new org.apache.thrift.meta_data.FieldMetaData("taskStatus", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.OPERATION_STARTED, new org.apache.thrift.meta_data.FieldMetaData("operationStarted", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.OPERATION_COMPLETED, new org.apache.thrift.meta_data.FieldMetaData("operationCompleted", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.HAS_RESULT_SET, new org.apache.thrift.meta_data.FieldMetaData("hasResultSet", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.PROGRESS_UPDATE_RESPONSE, new org.apache.thrift.meta_data.FieldMetaData("progressUpdateResponse", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRUCT , "TProgressUpdateResp"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetOperationStatusResp.class, metaDataMap); + } + + public TGetOperationStatusResp() { + } + + public TGetOperationStatusResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetOperationStatusResp(TGetOperationStatusResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationState()) { + this.operationState = other.operationState; + } + if (other.isSetSqlState()) { + this.sqlState = other.sqlState; + } + this.errorCode = other.errorCode; + if (other.isSetErrorMessage()) { + this.errorMessage = other.errorMessage; + } + if (other.isSetTaskStatus()) { + this.taskStatus = other.taskStatus; + } + this.operationStarted = other.operationStarted; + this.operationCompleted = other.operationCompleted; + this.hasResultSet = other.hasResultSet; + if (other.isSetProgressUpdateResponse()) { + this.progressUpdateResponse = other.progressUpdateResponse; + } + } + + public TGetOperationStatusResp deepCopy() { + return new TGetOperationStatusResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationState = null; + this.sqlState = null; + setErrorCodeIsSet(false); + this.errorCode = 0; + this.errorMessage = null; + this.taskStatus = null; + setOperationStartedIsSet(false); + this.operationStarted = 0; + setOperationCompletedIsSet(false); + this.operationCompleted = 0; + setHasResultSetIsSet(false); + this.hasResultSet = false; + this.progressUpdateResponse = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + /** + * + * @see TOperationState + */ + public TOperationState getOperationState() { + return this.operationState; + } + + /** + * + * @see TOperationState + */ + public void setOperationState(TOperationState operationState) { + this.operationState = operationState; + } + + public void unsetOperationState() { + this.operationState = null; + } + + /** Returns true if field operationState is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationState() { + return this.operationState != null; + } + + public void setOperationStateIsSet(boolean value) { + if (!value) { + this.operationState = null; + } + } + + public String getSqlState() { + return this.sqlState; + } + + public void setSqlState(String sqlState) { + this.sqlState = sqlState; + } + + public void unsetSqlState() { + this.sqlState = null; + } + + /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ + public boolean isSetSqlState() { + return this.sqlState != null; + } + + public void setSqlStateIsSet(boolean value) { + if (!value) { + this.sqlState = null; + } + } + + public int getErrorCode() { + return this.errorCode; + } + + public void setErrorCode(int errorCode) { + this.errorCode = errorCode; + setErrorCodeIsSet(true); + } + + public void unsetErrorCode() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorCode() { + return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + public void setErrorCodeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); + } + + public String getErrorMessage() { + return this.errorMessage; + } + + public void setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + } + + public void unsetErrorMessage() { + this.errorMessage = null; + } + + /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorMessage() { + return this.errorMessage != null; + } + + public void setErrorMessageIsSet(boolean value) { + if (!value) { + this.errorMessage = null; + } + } + + public String getTaskStatus() { + return this.taskStatus; + } + + public void setTaskStatus(String taskStatus) { + this.taskStatus = taskStatus; + } + + public void unsetTaskStatus() { + this.taskStatus = null; + } + + /** Returns true if field taskStatus is set (has been assigned a value) and false otherwise */ + public boolean isSetTaskStatus() { + return this.taskStatus != null; + } + + public void setTaskStatusIsSet(boolean value) { + if (!value) { + this.taskStatus = null; + } + } + + public long getOperationStarted() { + return this.operationStarted; + } + + public void setOperationStarted(long operationStarted) { + this.operationStarted = operationStarted; + setOperationStartedIsSet(true); + } + + public void unsetOperationStarted() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OPERATIONSTARTED_ISSET_ID); + } + + /** Returns true if field operationStarted is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationStarted() { + return EncodingUtils.testBit(__isset_bitfield, __OPERATIONSTARTED_ISSET_ID); + } + + public void setOperationStartedIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OPERATIONSTARTED_ISSET_ID, value); + } + + public long getOperationCompleted() { + return this.operationCompleted; + } + + public void setOperationCompleted(long operationCompleted) { + this.operationCompleted = operationCompleted; + setOperationCompletedIsSet(true); + } + + public void unsetOperationCompleted() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __OPERATIONCOMPLETED_ISSET_ID); + } + + /** Returns true if field operationCompleted is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationCompleted() { + return EncodingUtils.testBit(__isset_bitfield, __OPERATIONCOMPLETED_ISSET_ID); + } + + public void setOperationCompletedIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __OPERATIONCOMPLETED_ISSET_ID, value); + } + + public boolean isHasResultSet() { + return this.hasResultSet; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + setHasResultSetIsSet(true); + } + + public void unsetHasResultSet() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + /** Returns true if field hasResultSet is set (has been assigned a value) and false otherwise */ + public boolean isSetHasResultSet() { + return EncodingUtils.testBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + public void setHasResultSetIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASRESULTSET_ISSET_ID, value); + } + + public TProgressUpdateResp getProgressUpdateResponse() { + return this.progressUpdateResponse; + } + + public void setProgressUpdateResponse(TProgressUpdateResp progressUpdateResponse) { + this.progressUpdateResponse = progressUpdateResponse; + } + + public void unsetProgressUpdateResponse() { + this.progressUpdateResponse = null; + } + + /** Returns true if field progressUpdateResponse is set (has been assigned a value) and false otherwise */ + public boolean isSetProgressUpdateResponse() { + return this.progressUpdateResponse != null; + } + + public void setProgressUpdateResponseIsSet(boolean value) { + if (!value) { + this.progressUpdateResponse = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_STATE: + if (value == null) { + unsetOperationState(); + } else { + setOperationState((TOperationState)value); + } + break; + + case SQL_STATE: + if (value == null) { + unsetSqlState(); + } else { + setSqlState((String)value); + } + break; + + case ERROR_CODE: + if (value == null) { + unsetErrorCode(); + } else { + setErrorCode((Integer)value); + } + break; + + case ERROR_MESSAGE: + if (value == null) { + unsetErrorMessage(); + } else { + setErrorMessage((String)value); + } + break; + + case TASK_STATUS: + if (value == null) { + unsetTaskStatus(); + } else { + setTaskStatus((String)value); + } + break; + + case OPERATION_STARTED: + if (value == null) { + unsetOperationStarted(); + } else { + setOperationStarted((Long)value); + } + break; + + case OPERATION_COMPLETED: + if (value == null) { + unsetOperationCompleted(); + } else { + setOperationCompleted((Long)value); + } + break; + + case HAS_RESULT_SET: + if (value == null) { + unsetHasResultSet(); + } else { + setHasResultSet((Boolean)value); + } + break; + + case PROGRESS_UPDATE_RESPONSE: + if (value == null) { + unsetProgressUpdateResponse(); + } else { + setProgressUpdateResponse((TProgressUpdateResp)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_STATE: + return getOperationState(); + + case SQL_STATE: + return getSqlState(); + + case ERROR_CODE: + return getErrorCode(); + + case ERROR_MESSAGE: + return getErrorMessage(); + + case TASK_STATUS: + return getTaskStatus(); + + case OPERATION_STARTED: + return getOperationStarted(); + + case OPERATION_COMPLETED: + return getOperationCompleted(); + + case HAS_RESULT_SET: + return isHasResultSet(); + + case PROGRESS_UPDATE_RESPONSE: + return getProgressUpdateResponse(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_STATE: + return isSetOperationState(); + case SQL_STATE: + return isSetSqlState(); + case ERROR_CODE: + return isSetErrorCode(); + case ERROR_MESSAGE: + return isSetErrorMessage(); + case TASK_STATUS: + return isSetTaskStatus(); + case OPERATION_STARTED: + return isSetOperationStarted(); + case OPERATION_COMPLETED: + return isSetOperationCompleted(); + case HAS_RESULT_SET: + return isSetHasResultSet(); + case PROGRESS_UPDATE_RESPONSE: + return isSetProgressUpdateResponse(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetOperationStatusResp) + return this.equals((TGetOperationStatusResp)that); + return false; + } + + public boolean equals(TGetOperationStatusResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationState = true && this.isSetOperationState(); + boolean that_present_operationState = true && that.isSetOperationState(); + if (this_present_operationState || that_present_operationState) { + if (!(this_present_operationState && that_present_operationState)) + return false; + if (!this.operationState.equals(that.operationState)) + return false; + } + + boolean this_present_sqlState = true && this.isSetSqlState(); + boolean that_present_sqlState = true && that.isSetSqlState(); + if (this_present_sqlState || that_present_sqlState) { + if (!(this_present_sqlState && that_present_sqlState)) + return false; + if (!this.sqlState.equals(that.sqlState)) + return false; + } + + boolean this_present_errorCode = true && this.isSetErrorCode(); + boolean that_present_errorCode = true && that.isSetErrorCode(); + if (this_present_errorCode || that_present_errorCode) { + if (!(this_present_errorCode && that_present_errorCode)) + return false; + if (this.errorCode != that.errorCode) + return false; + } + + boolean this_present_errorMessage = true && this.isSetErrorMessage(); + boolean that_present_errorMessage = true && that.isSetErrorMessage(); + if (this_present_errorMessage || that_present_errorMessage) { + if (!(this_present_errorMessage && that_present_errorMessage)) + return false; + if (!this.errorMessage.equals(that.errorMessage)) + return false; + } + + boolean this_present_taskStatus = true && this.isSetTaskStatus(); + boolean that_present_taskStatus = true && that.isSetTaskStatus(); + if (this_present_taskStatus || that_present_taskStatus) { + if (!(this_present_taskStatus && that_present_taskStatus)) + return false; + if (!this.taskStatus.equals(that.taskStatus)) + return false; + } + + boolean this_present_operationStarted = true && this.isSetOperationStarted(); + boolean that_present_operationStarted = true && that.isSetOperationStarted(); + if (this_present_operationStarted || that_present_operationStarted) { + if (!(this_present_operationStarted && that_present_operationStarted)) + return false; + if (this.operationStarted != that.operationStarted) + return false; + } + + boolean this_present_operationCompleted = true && this.isSetOperationCompleted(); + boolean that_present_operationCompleted = true && that.isSetOperationCompleted(); + if (this_present_operationCompleted || that_present_operationCompleted) { + if (!(this_present_operationCompleted && that_present_operationCompleted)) + return false; + if (this.operationCompleted != that.operationCompleted) + return false; + } + + boolean this_present_hasResultSet = true && this.isSetHasResultSet(); + boolean that_present_hasResultSet = true && that.isSetHasResultSet(); + if (this_present_hasResultSet || that_present_hasResultSet) { + if (!(this_present_hasResultSet && that_present_hasResultSet)) + return false; + if (this.hasResultSet != that.hasResultSet) + return false; + } + + boolean this_present_progressUpdateResponse = true && this.isSetProgressUpdateResponse(); + boolean that_present_progressUpdateResponse = true && that.isSetProgressUpdateResponse(); + if (this_present_progressUpdateResponse || that_present_progressUpdateResponse) { + if (!(this_present_progressUpdateResponse && that_present_progressUpdateResponse)) + return false; + if (!this.progressUpdateResponse.equals(that.progressUpdateResponse)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationState = true && (isSetOperationState()); + list.add(present_operationState); + if (present_operationState) + list.add(operationState.getValue()); + + boolean present_sqlState = true && (isSetSqlState()); + list.add(present_sqlState); + if (present_sqlState) + list.add(sqlState); + + boolean present_errorCode = true && (isSetErrorCode()); + list.add(present_errorCode); + if (present_errorCode) + list.add(errorCode); + + boolean present_errorMessage = true && (isSetErrorMessage()); + list.add(present_errorMessage); + if (present_errorMessage) + list.add(errorMessage); + + boolean present_taskStatus = true && (isSetTaskStatus()); + list.add(present_taskStatus); + if (present_taskStatus) + list.add(taskStatus); + + boolean present_operationStarted = true && (isSetOperationStarted()); + list.add(present_operationStarted); + if (present_operationStarted) + list.add(operationStarted); + + boolean present_operationCompleted = true && (isSetOperationCompleted()); + list.add(present_operationCompleted); + if (present_operationCompleted) + list.add(operationCompleted); + + boolean present_hasResultSet = true && (isSetHasResultSet()); + list.add(present_hasResultSet); + if (present_hasResultSet) + list.add(hasResultSet); + + boolean present_progressUpdateResponse = true && (isSetProgressUpdateResponse()); + list.add(present_progressUpdateResponse); + if (present_progressUpdateResponse) + list.add(progressUpdateResponse); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetOperationStatusResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationState()).compareTo(other.isSetOperationState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationState, other.operationState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(other.isSetSqlState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSqlState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, other.sqlState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(other.isSetErrorCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, other.errorCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(other.isSetErrorMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, other.errorMessage); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTaskStatus()).compareTo(other.isSetTaskStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTaskStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.taskStatus, other.taskStatus); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationStarted()).compareTo(other.isSetOperationStarted()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationStarted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationStarted, other.operationStarted); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationCompleted()).compareTo(other.isSetOperationCompleted()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationCompleted()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationCompleted, other.operationCompleted); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHasResultSet()).compareTo(other.isSetHasResultSet()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHasResultSet()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasResultSet, other.hasResultSet); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetProgressUpdateResponse()).compareTo(other.isSetProgressUpdateResponse()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetProgressUpdateResponse()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.progressUpdateResponse, other.progressUpdateResponse); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetOperationStatusResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationState()) { + if (!first) sb.append(", "); + sb.append("operationState:"); + if (this.operationState == null) { + sb.append("null"); + } else { + sb.append(this.operationState); + } + first = false; + } + if (isSetSqlState()) { + if (!first) sb.append(", "); + sb.append("sqlState:"); + if (this.sqlState == null) { + sb.append("null"); + } else { + sb.append(this.sqlState); + } + first = false; + } + if (isSetErrorCode()) { + if (!first) sb.append(", "); + sb.append("errorCode:"); + sb.append(this.errorCode); + first = false; + } + if (isSetErrorMessage()) { + if (!first) sb.append(", "); + sb.append("errorMessage:"); + if (this.errorMessage == null) { + sb.append("null"); + } else { + sb.append(this.errorMessage); + } + first = false; + } + if (isSetTaskStatus()) { + if (!first) sb.append(", "); + sb.append("taskStatus:"); + if (this.taskStatus == null) { + sb.append("null"); + } else { + sb.append(this.taskStatus); + } + first = false; + } + if (isSetOperationStarted()) { + if (!first) sb.append(", "); + sb.append("operationStarted:"); + sb.append(this.operationStarted); + first = false; + } + if (isSetOperationCompleted()) { + if (!first) sb.append(", "); + sb.append("operationCompleted:"); + sb.append(this.operationCompleted); + first = false; + } + if (isSetHasResultSet()) { + if (!first) sb.append(", "); + sb.append("hasResultSet:"); + sb.append(this.hasResultSet); + first = false; + } + if (isSetProgressUpdateResponse()) { + if (!first) sb.append(", "); + sb.append("progressUpdateResponse:"); + if (this.progressUpdateResponse == null) { + sb.append("null"); + } else { + sb.append(this.progressUpdateResponse); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetOperationStatusRespStandardSchemeFactory implements SchemeFactory { + public TGetOperationStatusRespStandardScheme getScheme() { + return new TGetOperationStatusRespStandardScheme(); + } + } + + private static class TGetOperationStatusRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.operationState = org.apache.hive.service.rpc.thrift.TOperationState.findByValue(iprot.readI32()); + struct.setOperationStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SQL_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // ERROR_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ERROR_MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // TASK_STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.taskStatus = iprot.readString(); + struct.setTaskStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 7: // OPERATION_STARTED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.operationStarted = iprot.readI64(); + struct.setOperationStartedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 8: // OPERATION_COMPLETED + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.operationCompleted = iprot.readI64(); + struct.setOperationCompletedIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 9: // HAS_RESULT_SET + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 10: // PROGRESS_UPDATE_RESPONSE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.progressUpdateResponse = new TProgressUpdateResp(); + struct.progressUpdateResponse.read(iprot); + struct.setProgressUpdateResponseIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationState != null) { + if (struct.isSetOperationState()) { + oprot.writeFieldBegin(OPERATION_STATE_FIELD_DESC); + oprot.writeI32(struct.operationState.getValue()); + oprot.writeFieldEnd(); + } + } + if (struct.sqlState != null) { + if (struct.isSetSqlState()) { + oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); + oprot.writeString(struct.sqlState); + oprot.writeFieldEnd(); + } + } + if (struct.isSetErrorCode()) { + oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); + oprot.writeI32(struct.errorCode); + oprot.writeFieldEnd(); + } + if (struct.errorMessage != null) { + if (struct.isSetErrorMessage()) { + oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); + oprot.writeString(struct.errorMessage); + oprot.writeFieldEnd(); + } + } + if (struct.taskStatus != null) { + if (struct.isSetTaskStatus()) { + oprot.writeFieldBegin(TASK_STATUS_FIELD_DESC); + oprot.writeString(struct.taskStatus); + oprot.writeFieldEnd(); + } + } + if (struct.isSetOperationStarted()) { + oprot.writeFieldBegin(OPERATION_STARTED_FIELD_DESC); + oprot.writeI64(struct.operationStarted); + oprot.writeFieldEnd(); + } + if (struct.isSetOperationCompleted()) { + oprot.writeFieldBegin(OPERATION_COMPLETED_FIELD_DESC); + oprot.writeI64(struct.operationCompleted); + oprot.writeFieldEnd(); + } + if (struct.isSetHasResultSet()) { + oprot.writeFieldBegin(HAS_RESULT_SET_FIELD_DESC); + oprot.writeBool(struct.hasResultSet); + oprot.writeFieldEnd(); + } + if (struct.progressUpdateResponse != null) { + if (struct.isSetProgressUpdateResponse()) { + oprot.writeFieldBegin(PROGRESS_UPDATE_RESPONSE_FIELD_DESC); + struct.progressUpdateResponse.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetOperationStatusRespTupleSchemeFactory implements SchemeFactory { + public TGetOperationStatusRespTupleScheme getScheme() { + return new TGetOperationStatusRespTupleScheme(); + } + } + + private static class TGetOperationStatusRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationState()) { + optionals.set(0); + } + if (struct.isSetSqlState()) { + optionals.set(1); + } + if (struct.isSetErrorCode()) { + optionals.set(2); + } + if (struct.isSetErrorMessage()) { + optionals.set(3); + } + if (struct.isSetTaskStatus()) { + optionals.set(4); + } + if (struct.isSetOperationStarted()) { + optionals.set(5); + } + if (struct.isSetOperationCompleted()) { + optionals.set(6); + } + if (struct.isSetHasResultSet()) { + optionals.set(7); + } + if (struct.isSetProgressUpdateResponse()) { + optionals.set(8); + } + oprot.writeBitSet(optionals, 9); + if (struct.isSetOperationState()) { + oprot.writeI32(struct.operationState.getValue()); + } + if (struct.isSetSqlState()) { + oprot.writeString(struct.sqlState); + } + if (struct.isSetErrorCode()) { + oprot.writeI32(struct.errorCode); + } + if (struct.isSetErrorMessage()) { + oprot.writeString(struct.errorMessage); + } + if (struct.isSetTaskStatus()) { + oprot.writeString(struct.taskStatus); + } + if (struct.isSetOperationStarted()) { + oprot.writeI64(struct.operationStarted); + } + if (struct.isSetOperationCompleted()) { + oprot.writeI64(struct.operationCompleted); + } + if (struct.isSetHasResultSet()) { + oprot.writeBool(struct.hasResultSet); + } + if (struct.isSetProgressUpdateResponse()) { + struct.progressUpdateResponse.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetOperationStatusResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(9); + if (incoming.get(0)) { + struct.operationState = org.apache.hive.service.rpc.thrift.TOperationState.findByValue(iprot.readI32()); + struct.setOperationStateIsSet(true); + } + if (incoming.get(1)) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } + if (incoming.get(2)) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } + if (incoming.get(3)) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } + if (incoming.get(4)) { + struct.taskStatus = iprot.readString(); + struct.setTaskStatusIsSet(true); + } + if (incoming.get(5)) { + struct.operationStarted = iprot.readI64(); + struct.setOperationStartedIsSet(true); + } + if (incoming.get(6)) { + struct.operationCompleted = iprot.readI64(); + struct.setOperationCompletedIsSet(true); + } + if (incoming.get(7)) { + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + } + if (incoming.get(8)) { + struct.progressUpdateResponse = new TProgressUpdateResp(); + struct.progressUpdateResponse.read(iprot); + struct.setProgressUpdateResponseIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java new file mode 100644 index 0000000000000..1bec9b51c72d8 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysReq.java @@ -0,0 +1,716 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetPrimaryKeysReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetPrimaryKeysReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetPrimaryKeysReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetPrimaryKeysReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String tableName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + TABLE_NAME((short)4, "tableName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // TABLE_NAME + return TABLE_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetPrimaryKeysReq.class, metaDataMap); + } + + public TGetPrimaryKeysReq() { + } + + public TGetPrimaryKeysReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetPrimaryKeysReq(TGetPrimaryKeysReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + } + + public TGetPrimaryKeysReq deepCopy() { + return new TGetPrimaryKeysReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.tableName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case TABLE_NAME: + return getTableName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case TABLE_NAME: + return isSetTableName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetPrimaryKeysReq) + return this.equals((TGetPrimaryKeysReq)that); + return false; + } + + public boolean equals(TGetPrimaryKeysReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + list.add(present_catalogName); + if (present_catalogName) + list.add(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + list.add(present_schemaName); + if (present_schemaName) + list.add(schemaName); + + boolean present_tableName = true && (isSetTableName()); + list.add(present_tableName); + if (present_tableName) + list.add(tableName); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetPrimaryKeysReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetPrimaryKeysReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (isSetTableName()) { + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetPrimaryKeysReqStandardSchemeFactory implements SchemeFactory { + public TGetPrimaryKeysReqStandardScheme getScheme() { + return new TGetPrimaryKeysReqStandardScheme(); + } + } + + private static class TGetPrimaryKeysReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetPrimaryKeysReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetPrimaryKeysReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.tableName != null) { + if (struct.isSetTableName()) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetPrimaryKeysReqTupleSchemeFactory implements SchemeFactory { + public TGetPrimaryKeysReqTupleScheme getScheme() { + return new TGetPrimaryKeysReqTupleScheme(); + } + } + + private static class TGetPrimaryKeysReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetPrimaryKeysReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetPrimaryKeysReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java new file mode 100644 index 0000000000000..72d9507fe1031 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetPrimaryKeysResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetPrimaryKeysResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetPrimaryKeysResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetPrimaryKeysRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetPrimaryKeysRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetPrimaryKeysResp.class, metaDataMap); + } + + public TGetPrimaryKeysResp() { + } + + public TGetPrimaryKeysResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetPrimaryKeysResp(TGetPrimaryKeysResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetPrimaryKeysResp deepCopy() { + return new TGetPrimaryKeysResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetPrimaryKeysResp) + return this.equals((TGetPrimaryKeysResp)that); + return false; + } + + public boolean equals(TGetPrimaryKeysResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetPrimaryKeysResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetPrimaryKeysResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetPrimaryKeysRespStandardSchemeFactory implements SchemeFactory { + public TGetPrimaryKeysRespStandardScheme getScheme() { + return new TGetPrimaryKeysRespStandardScheme(); + } + } + + private static class TGetPrimaryKeysRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetPrimaryKeysResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetPrimaryKeysResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetPrimaryKeysRespTupleSchemeFactory implements SchemeFactory { + public TGetPrimaryKeysRespTupleScheme getScheme() { + return new TGetPrimaryKeysRespTupleScheme(); + } + } + + private static class TGetPrimaryKeysRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetPrimaryKeysResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetPrimaryKeysResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java new file mode 100644 index 0000000000000..b94d827de264d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetResultSetMetadataReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataReq"); + + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetResultSetMetadataReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetResultSetMetadataReqTupleSchemeFactory()); + } + + private TOperationHandle operationHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_HANDLE((short)1, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataReq.class, metaDataMap); + } + + public TGetResultSetMetadataReq() { + } + + public TGetResultSetMetadataReq( + TOperationHandle operationHandle) + { + this(); + this.operationHandle = operationHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetResultSetMetadataReq(TGetResultSetMetadataReq other) { + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetResultSetMetadataReq deepCopy() { + return new TGetResultSetMetadataReq(this); + } + + @Override + public void clear() { + this.operationHandle = null; + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetResultSetMetadataReq) + return this.equals((TGetResultSetMetadataReq)that); + return false; + } + + public boolean equals(TGetResultSetMetadataReq that) { + if (that == null) + return false; + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetResultSetMetadataReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetResultSetMetadataReq("); + boolean first = true; + + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetResultSetMetadataReqStandardSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataReqStandardScheme getScheme() { + return new TGetResultSetMetadataReqStandardScheme(); + } + } + + private static class TGetResultSetMetadataReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationHandle != null) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetResultSetMetadataReqTupleSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataReqTupleScheme getScheme() { + return new TGetResultSetMetadataReqTupleScheme(); + } + } + + private static class TGetResultSetMetadataReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java new file mode 100644 index 0000000000000..ae2021ebd5a10 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetResultSetMetadataResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetResultSetMetadataResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetResultSetMetadataResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SCHEMA_FIELD_DESC = new org.apache.thrift.protocol.TField("schema", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetResultSetMetadataRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetResultSetMetadataRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TTableSchema schema; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + SCHEMA((short)2, "schema"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // SCHEMA + return SCHEMA; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.SCHEMA}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.SCHEMA, new org.apache.thrift.meta_data.FieldMetaData("schema", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTableSchema.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetResultSetMetadataResp.class, metaDataMap); + } + + public TGetResultSetMetadataResp() { + } + + public TGetResultSetMetadataResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetResultSetMetadataResp(TGetResultSetMetadataResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetSchema()) { + this.schema = new TTableSchema(other.schema); + } + } + + public TGetResultSetMetadataResp deepCopy() { + return new TGetResultSetMetadataResp(this); + } + + @Override + public void clear() { + this.status = null; + this.schema = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TTableSchema getSchema() { + return this.schema; + } + + public void setSchema(TTableSchema schema) { + this.schema = schema; + } + + public void unsetSchema() { + this.schema = null; + } + + /** Returns true if field schema is set (has been assigned a value) and false otherwise */ + public boolean isSetSchema() { + return this.schema != null; + } + + public void setSchemaIsSet(boolean value) { + if (!value) { + this.schema = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case SCHEMA: + if (value == null) { + unsetSchema(); + } else { + setSchema((TTableSchema)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case SCHEMA: + return getSchema(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case SCHEMA: + return isSetSchema(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetResultSetMetadataResp) + return this.equals((TGetResultSetMetadataResp)that); + return false; + } + + public boolean equals(TGetResultSetMetadataResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_schema = true && this.isSetSchema(); + boolean that_present_schema = true && that.isSetSchema(); + if (this_present_schema || that_present_schema) { + if (!(this_present_schema && that_present_schema)) + return false; + if (!this.schema.equals(that.schema)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_schema = true && (isSetSchema()); + list.add(present_schema); + if (present_schema) + list.add(schema); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetResultSetMetadataResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchema()).compareTo(other.isSetSchema()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchema()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schema, other.schema); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetResultSetMetadataResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetSchema()) { + if (!first) sb.append(", "); + sb.append("schema:"); + if (this.schema == null) { + sb.append("null"); + } else { + sb.append(this.schema); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (schema != null) { + schema.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetResultSetMetadataRespStandardSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataRespStandardScheme getScheme() { + return new TGetResultSetMetadataRespStandardScheme(); + } + } + + private static class TGetResultSetMetadataRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SCHEMA + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.schema = new TTableSchema(); + struct.schema.read(iprot); + struct.setSchemaIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.schema != null) { + if (struct.isSetSchema()) { + oprot.writeFieldBegin(SCHEMA_FIELD_DESC); + struct.schema.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetResultSetMetadataRespTupleSchemeFactory implements SchemeFactory { + public TGetResultSetMetadataRespTupleScheme getScheme() { + return new TGetResultSetMetadataRespTupleScheme(); + } + } + + private static class TGetResultSetMetadataRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetSchema()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetSchema()) { + struct.schema.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetResultSetMetadataResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.schema = new TTableSchema(); + struct.schema.read(iprot); + struct.setSchemaIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java new file mode 100644 index 0000000000000..17eed87ae096f --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasReq.java @@ -0,0 +1,610 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetSchemasReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetSchemasReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetSchemasReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasReq.class, metaDataMap); + } + + public TGetSchemasReq() { + } + + public TGetSchemasReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetSchemasReq(TGetSchemasReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + } + + public TGetSchemasReq deepCopy() { + return new TGetSchemasReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetSchemasReq) + return this.equals((TGetSchemasReq)that); + return false; + } + + public boolean equals(TGetSchemasReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + list.add(present_catalogName); + if (present_catalogName) + list.add(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + list.add(present_schemaName); + if (present_schemaName) + list.add(schemaName); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetSchemasReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetSchemasReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetSchemasReqStandardSchemeFactory implements SchemeFactory { + public TGetSchemasReqStandardScheme getScheme() { + return new TGetSchemasReqStandardScheme(); + } + } + + private static class TGetSchemasReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetSchemasReqTupleSchemeFactory implements SchemeFactory { + public TGetSchemasReqTupleScheme getScheme() { + return new TGetSchemasReqTupleScheme(); + } + } + + private static class TGetSchemasReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java new file mode 100644 index 0000000000000..e5317f7ff5046 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetSchemasResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetSchemasResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetSchemasResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetSchemasRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetSchemasRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetSchemasResp.class, metaDataMap); + } + + public TGetSchemasResp() { + } + + public TGetSchemasResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetSchemasResp(TGetSchemasResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetSchemasResp deepCopy() { + return new TGetSchemasResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetSchemasResp) + return this.equals((TGetSchemasResp)that); + return false; + } + + public boolean equals(TGetSchemasResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetSchemasResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetSchemasResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetSchemasRespStandardSchemeFactory implements SchemeFactory { + public TGetSchemasRespStandardScheme getScheme() { + return new TGetSchemasRespStandardScheme(); + } + } + + private static class TGetSchemasRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetSchemasResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetSchemasResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetSchemasRespTupleSchemeFactory implements SchemeFactory { + public TGetSchemasRespTupleScheme getScheme() { + return new TGetSchemasRespTupleScheme(); + } + } + + private static class TGetSchemasRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetSchemasResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java new file mode 100644 index 0000000000000..c027748a336e6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTableTypesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTableTypesReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTableTypesReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesReq.class, metaDataMap); + } + + public TGetTableTypesReq() { + } + + public TGetTableTypesReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTableTypesReq(TGetTableTypesReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetTableTypesReq deepCopy() { + return new TGetTableTypesReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTableTypesReq) + return this.equals((TGetTableTypesReq)that); + return false; + } + + public boolean equals(TGetTableTypesReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTableTypesReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTableTypesReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTableTypesReqStandardSchemeFactory implements SchemeFactory { + public TGetTableTypesReqStandardScheme getScheme() { + return new TGetTableTypesReqStandardScheme(); + } + } + + private static class TGetTableTypesReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTableTypesReqTupleSchemeFactory implements SchemeFactory { + public TGetTableTypesReqTupleScheme getScheme() { + return new TGetTableTypesReqTupleScheme(); + } + } + + private static class TGetTableTypesReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java new file mode 100644 index 0000000000000..c6ce0d4368fdd --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTableTypesResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTableTypesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTableTypesResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTableTypesRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTableTypesRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTableTypesResp.class, metaDataMap); + } + + public TGetTableTypesResp() { + } + + public TGetTableTypesResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTableTypesResp(TGetTableTypesResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTableTypesResp deepCopy() { + return new TGetTableTypesResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTableTypesResp) + return this.equals((TGetTableTypesResp)that); + return false; + } + + public boolean equals(TGetTableTypesResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTableTypesResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTableTypesResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTableTypesRespStandardSchemeFactory implements SchemeFactory { + public TGetTableTypesRespStandardScheme getScheme() { + return new TGetTableTypesRespStandardScheme(); + } + } + + private static class TGetTableTypesRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTableTypesRespTupleSchemeFactory implements SchemeFactory { + public TGetTableTypesRespTupleScheme getScheme() { + return new TGetTableTypesRespTupleScheme(); + } + } + + private static class TGetTableTypesRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTableTypesResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java new file mode 100644 index 0000000000000..1aa3f946727b6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesReq.java @@ -0,0 +1,871 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTablesReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField CATALOG_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("catalogName", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField SCHEMA_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("schemaName", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField TABLE_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("tableName", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField TABLE_TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("tableTypes", org.apache.thrift.protocol.TType.LIST, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTablesReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTablesReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String catalogName; // optional + private String schemaName; // optional + private String tableName; // optional + private List tableTypes; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + CATALOG_NAME((short)2, "catalogName"), + SCHEMA_NAME((short)3, "schemaName"), + TABLE_NAME((short)4, "tableName"), + TABLE_TYPES((short)5, "tableTypes"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // CATALOG_NAME + return CATALOG_NAME; + case 3: // SCHEMA_NAME + return SCHEMA_NAME; + case 4: // TABLE_NAME + return TABLE_NAME; + case 5: // TABLE_TYPES + return TABLE_TYPES; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.CATALOG_NAME,_Fields.SCHEMA_NAME,_Fields.TABLE_NAME,_Fields.TABLE_TYPES}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CATALOG_NAME, new org.apache.thrift.meta_data.FieldMetaData("catalogName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.SCHEMA_NAME, new org.apache.thrift.meta_data.FieldMetaData("schemaName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_NAME, new org.apache.thrift.meta_data.FieldMetaData("tableName", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , "TPatternOrIdentifier"))); + tmpMap.put(_Fields.TABLE_TYPES, new org.apache.thrift.meta_data.FieldMetaData("tableTypes", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesReq.class, metaDataMap); + } + + public TGetTablesReq() { + } + + public TGetTablesReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTablesReq(TGetTablesReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetCatalogName()) { + this.catalogName = other.catalogName; + } + if (other.isSetSchemaName()) { + this.schemaName = other.schemaName; + } + if (other.isSetTableName()) { + this.tableName = other.tableName; + } + if (other.isSetTableTypes()) { + List __this__tableTypes = new ArrayList(other.tableTypes); + this.tableTypes = __this__tableTypes; + } + } + + public TGetTablesReq deepCopy() { + return new TGetTablesReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.catalogName = null; + this.schemaName = null; + this.tableName = null; + this.tableTypes = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getCatalogName() { + return this.catalogName; + } + + public void setCatalogName(String catalogName) { + this.catalogName = catalogName; + } + + public void unsetCatalogName() { + this.catalogName = null; + } + + /** Returns true if field catalogName is set (has been assigned a value) and false otherwise */ + public boolean isSetCatalogName() { + return this.catalogName != null; + } + + public void setCatalogNameIsSet(boolean value) { + if (!value) { + this.catalogName = null; + } + } + + public String getSchemaName() { + return this.schemaName; + } + + public void setSchemaName(String schemaName) { + this.schemaName = schemaName; + } + + public void unsetSchemaName() { + this.schemaName = null; + } + + /** Returns true if field schemaName is set (has been assigned a value) and false otherwise */ + public boolean isSetSchemaName() { + return this.schemaName != null; + } + + public void setSchemaNameIsSet(boolean value) { + if (!value) { + this.schemaName = null; + } + } + + public String getTableName() { + return this.tableName; + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public void unsetTableName() { + this.tableName = null; + } + + /** Returns true if field tableName is set (has been assigned a value) and false otherwise */ + public boolean isSetTableName() { + return this.tableName != null; + } + + public void setTableNameIsSet(boolean value) { + if (!value) { + this.tableName = null; + } + } + + public int getTableTypesSize() { + return (this.tableTypes == null) ? 0 : this.tableTypes.size(); + } + + public java.util.Iterator getTableTypesIterator() { + return (this.tableTypes == null) ? null : this.tableTypes.iterator(); + } + + public void addToTableTypes(String elem) { + if (this.tableTypes == null) { + this.tableTypes = new ArrayList(); + } + this.tableTypes.add(elem); + } + + public List getTableTypes() { + return this.tableTypes; + } + + public void setTableTypes(List tableTypes) { + this.tableTypes = tableTypes; + } + + public void unsetTableTypes() { + this.tableTypes = null; + } + + /** Returns true if field tableTypes is set (has been assigned a value) and false otherwise */ + public boolean isSetTableTypes() { + return this.tableTypes != null; + } + + public void setTableTypesIsSet(boolean value) { + if (!value) { + this.tableTypes = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CATALOG_NAME: + if (value == null) { + unsetCatalogName(); + } else { + setCatalogName((String)value); + } + break; + + case SCHEMA_NAME: + if (value == null) { + unsetSchemaName(); + } else { + setSchemaName((String)value); + } + break; + + case TABLE_NAME: + if (value == null) { + unsetTableName(); + } else { + setTableName((String)value); + } + break; + + case TABLE_TYPES: + if (value == null) { + unsetTableTypes(); + } else { + setTableTypes((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case CATALOG_NAME: + return getCatalogName(); + + case SCHEMA_NAME: + return getSchemaName(); + + case TABLE_NAME: + return getTableName(); + + case TABLE_TYPES: + return getTableTypes(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case CATALOG_NAME: + return isSetCatalogName(); + case SCHEMA_NAME: + return isSetSchemaName(); + case TABLE_NAME: + return isSetTableName(); + case TABLE_TYPES: + return isSetTableTypes(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTablesReq) + return this.equals((TGetTablesReq)that); + return false; + } + + public boolean equals(TGetTablesReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_catalogName = true && this.isSetCatalogName(); + boolean that_present_catalogName = true && that.isSetCatalogName(); + if (this_present_catalogName || that_present_catalogName) { + if (!(this_present_catalogName && that_present_catalogName)) + return false; + if (!this.catalogName.equals(that.catalogName)) + return false; + } + + boolean this_present_schemaName = true && this.isSetSchemaName(); + boolean that_present_schemaName = true && that.isSetSchemaName(); + if (this_present_schemaName || that_present_schemaName) { + if (!(this_present_schemaName && that_present_schemaName)) + return false; + if (!this.schemaName.equals(that.schemaName)) + return false; + } + + boolean this_present_tableName = true && this.isSetTableName(); + boolean that_present_tableName = true && that.isSetTableName(); + if (this_present_tableName || that_present_tableName) { + if (!(this_present_tableName && that_present_tableName)) + return false; + if (!this.tableName.equals(that.tableName)) + return false; + } + + boolean this_present_tableTypes = true && this.isSetTableTypes(); + boolean that_present_tableTypes = true && that.isSetTableTypes(); + if (this_present_tableTypes || that_present_tableTypes) { + if (!(this_present_tableTypes && that_present_tableTypes)) + return false; + if (!this.tableTypes.equals(that.tableTypes)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_catalogName = true && (isSetCatalogName()); + list.add(present_catalogName); + if (present_catalogName) + list.add(catalogName); + + boolean present_schemaName = true && (isSetSchemaName()); + list.add(present_schemaName); + if (present_schemaName) + list.add(schemaName); + + boolean present_tableName = true && (isSetTableName()); + list.add(present_tableName); + if (present_tableName) + list.add(tableName); + + boolean present_tableTypes = true && (isSetTableTypes()); + list.add(present_tableTypes); + if (present_tableTypes) + list.add(tableTypes); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTablesReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetCatalogName()).compareTo(other.isSetCatalogName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetCatalogName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.catalogName, other.catalogName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSchemaName()).compareTo(other.isSetSchemaName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSchemaName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.schemaName, other.schemaName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableName()).compareTo(other.isSetTableName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableName, other.tableName); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTableTypes()).compareTo(other.isSetTableTypes()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTableTypes()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.tableTypes, other.tableTypes); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTablesReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (isSetCatalogName()) { + if (!first) sb.append(", "); + sb.append("catalogName:"); + if (this.catalogName == null) { + sb.append("null"); + } else { + sb.append(this.catalogName); + } + first = false; + } + if (isSetSchemaName()) { + if (!first) sb.append(", "); + sb.append("schemaName:"); + if (this.schemaName == null) { + sb.append("null"); + } else { + sb.append(this.schemaName); + } + first = false; + } + if (isSetTableName()) { + if (!first) sb.append(", "); + sb.append("tableName:"); + if (this.tableName == null) { + sb.append("null"); + } else { + sb.append(this.tableName); + } + first = false; + } + if (isSetTableTypes()) { + if (!first) sb.append(", "); + sb.append("tableTypes:"); + if (this.tableTypes == null) { + sb.append("null"); + } else { + sb.append(this.tableTypes); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTablesReqStandardSchemeFactory implements SchemeFactory { + public TGetTablesReqStandardScheme getScheme() { + return new TGetTablesReqStandardScheme(); + } + } + + private static class TGetTablesReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // CATALOG_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SCHEMA_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // TABLE_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // TABLE_TYPES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list172 = iprot.readListBegin(); + struct.tableTypes = new ArrayList(_list172.size); + String _elem173; + for (int _i174 = 0; _i174 < _list172.size; ++_i174) + { + _elem173 = iprot.readString(); + struct.tableTypes.add(_elem173); + } + iprot.readListEnd(); + } + struct.setTableTypesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.catalogName != null) { + if (struct.isSetCatalogName()) { + oprot.writeFieldBegin(CATALOG_NAME_FIELD_DESC); + oprot.writeString(struct.catalogName); + oprot.writeFieldEnd(); + } + } + if (struct.schemaName != null) { + if (struct.isSetSchemaName()) { + oprot.writeFieldBegin(SCHEMA_NAME_FIELD_DESC); + oprot.writeString(struct.schemaName); + oprot.writeFieldEnd(); + } + } + if (struct.tableName != null) { + if (struct.isSetTableName()) { + oprot.writeFieldBegin(TABLE_NAME_FIELD_DESC); + oprot.writeString(struct.tableName); + oprot.writeFieldEnd(); + } + } + if (struct.tableTypes != null) { + if (struct.isSetTableTypes()) { + oprot.writeFieldBegin(TABLE_TYPES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.tableTypes.size())); + for (String _iter175 : struct.tableTypes) + { + oprot.writeString(_iter175); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTablesReqTupleSchemeFactory implements SchemeFactory { + public TGetTablesReqTupleScheme getScheme() { + return new TGetTablesReqTupleScheme(); + } + } + + private static class TGetTablesReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetCatalogName()) { + optionals.set(0); + } + if (struct.isSetSchemaName()) { + optionals.set(1); + } + if (struct.isSetTableName()) { + optionals.set(2); + } + if (struct.isSetTableTypes()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetCatalogName()) { + oprot.writeString(struct.catalogName); + } + if (struct.isSetSchemaName()) { + oprot.writeString(struct.schemaName); + } + if (struct.isSetTableName()) { + oprot.writeString(struct.tableName); + } + if (struct.isSetTableTypes()) { + { + oprot.writeI32(struct.tableTypes.size()); + for (String _iter176 : struct.tableTypes) + { + oprot.writeString(_iter176); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + struct.catalogName = iprot.readString(); + struct.setCatalogNameIsSet(true); + } + if (incoming.get(1)) { + struct.schemaName = iprot.readString(); + struct.setSchemaNameIsSet(true); + } + if (incoming.get(2)) { + struct.tableName = iprot.readString(); + struct.setTableNameIsSet(true); + } + if (incoming.get(3)) { + { + org.apache.thrift.protocol.TList _list177 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.tableTypes = new ArrayList(_list177.size); + String _elem178; + for (int _i179 = 0; _i179 < _list177.size; ++_i179) + { + _elem178 = iprot.readString(); + struct.tableTypes.add(_elem178); + } + } + struct.setTableTypesIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java new file mode 100644 index 0000000000000..0b7c3825d35a5 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTablesResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTablesResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTablesResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTablesRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTablesRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTablesResp.class, metaDataMap); + } + + public TGetTablesResp() { + } + + public TGetTablesResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTablesResp(TGetTablesResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTablesResp deepCopy() { + return new TGetTablesResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTablesResp) + return this.equals((TGetTablesResp)that); + return false; + } + + public boolean equals(TGetTablesResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTablesResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTablesResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTablesRespStandardSchemeFactory implements SchemeFactory { + public TGetTablesRespStandardScheme getScheme() { + return new TGetTablesRespStandardScheme(); + } + } + + private static class TGetTablesRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTablesResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTablesResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTablesRespTupleSchemeFactory implements SchemeFactory { + public TGetTablesRespTupleScheme getScheme() { + return new TGetTablesRespTupleScheme(); + } + } + + private static class TGetTablesRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTablesResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java new file mode 100644 index 0000000000000..2e0ec60e4bc3d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoReq.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTypeInfoReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTypeInfoReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTypeInfoReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoReq.class, metaDataMap); + } + + public TGetTypeInfoReq() { + } + + public TGetTypeInfoReq( + TSessionHandle sessionHandle) + { + this(); + this.sessionHandle = sessionHandle; + } + + /** + * Performs a deep copy on other. + */ + public TGetTypeInfoReq(TGetTypeInfoReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + } + + public TGetTypeInfoReq deepCopy() { + return new TGetTypeInfoReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTypeInfoReq) + return this.equals((TGetTypeInfoReq)that); + return false; + } + + public boolean equals(TGetTypeInfoReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTypeInfoReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTypeInfoReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTypeInfoReqStandardSchemeFactory implements SchemeFactory { + public TGetTypeInfoReqStandardScheme getScheme() { + return new TGetTypeInfoReqStandardScheme(); + } + } + + private static class TGetTypeInfoReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTypeInfoReqTupleSchemeFactory implements SchemeFactory { + public TGetTypeInfoReqTupleScheme getScheme() { + return new TGetTypeInfoReqTupleScheme(); + } + } + + private static class TGetTypeInfoReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java new file mode 100644 index 0000000000000..cc2910ef29feb --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TGetTypeInfoResp.java @@ -0,0 +1,509 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TGetTypeInfoResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TGetTypeInfoResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationHandle", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TGetTypeInfoRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TGetTypeInfoRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TOperationHandle operationHandle; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + OPERATION_HANDLE((short)2, "operationHandle"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // OPERATION_HANDLE + return OPERATION_HANDLE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.OPERATION_HANDLE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.OPERATION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("operationHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TOperationHandle.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TGetTypeInfoResp.class, metaDataMap); + } + + public TGetTypeInfoResp() { + } + + public TGetTypeInfoResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TGetTypeInfoResp(TGetTypeInfoResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetOperationHandle()) { + this.operationHandle = new TOperationHandle(other.operationHandle); + } + } + + public TGetTypeInfoResp deepCopy() { + return new TGetTypeInfoResp(this); + } + + @Override + public void clear() { + this.status = null; + this.operationHandle = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public TOperationHandle getOperationHandle() { + return this.operationHandle; + } + + public void setOperationHandle(TOperationHandle operationHandle) { + this.operationHandle = operationHandle; + } + + public void unsetOperationHandle() { + this.operationHandle = null; + } + + /** Returns true if field operationHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationHandle() { + return this.operationHandle != null; + } + + public void setOperationHandleIsSet(boolean value) { + if (!value) { + this.operationHandle = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case OPERATION_HANDLE: + if (value == null) { + unsetOperationHandle(); + } else { + setOperationHandle((TOperationHandle)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case OPERATION_HANDLE: + return getOperationHandle(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case OPERATION_HANDLE: + return isSetOperationHandle(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TGetTypeInfoResp) + return this.equals((TGetTypeInfoResp)that); + return false; + } + + public boolean equals(TGetTypeInfoResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_operationHandle = true && this.isSetOperationHandle(); + boolean that_present_operationHandle = true && that.isSetOperationHandle(); + if (this_present_operationHandle || that_present_operationHandle) { + if (!(this_present_operationHandle && that_present_operationHandle)) + return false; + if (!this.operationHandle.equals(that.operationHandle)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_operationHandle = true && (isSetOperationHandle()); + list.add(present_operationHandle); + if (present_operationHandle) + list.add(operationHandle); + + return list.hashCode(); + } + + @Override + public int compareTo(TGetTypeInfoResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationHandle()).compareTo(other.isSetOperationHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationHandle, other.operationHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TGetTypeInfoResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (isSetOperationHandle()) { + if (!first) sb.append(", "); + sb.append("operationHandle:"); + if (this.operationHandle == null) { + sb.append("null"); + } else { + sb.append(this.operationHandle); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (operationHandle != null) { + operationHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TGetTypeInfoRespStandardSchemeFactory implements SchemeFactory { + public TGetTypeInfoRespStandardScheme getScheme() { + return new TGetTypeInfoRespStandardScheme(); + } + } + + private static class TGetTypeInfoRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationHandle != null) { + if (struct.isSetOperationHandle()) { + oprot.writeFieldBegin(OPERATION_HANDLE_FIELD_DESC); + struct.operationHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TGetTypeInfoRespTupleSchemeFactory implements SchemeFactory { + public TGetTypeInfoRespTupleScheme getScheme() { + return new TGetTypeInfoRespTupleScheme(); + } + } + + private static class TGetTypeInfoRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + BitSet optionals = new BitSet(); + if (struct.isSetOperationHandle()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetOperationHandle()) { + struct.operationHandle.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TGetTypeInfoResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.operationHandle = new TOperationHandle(); + struct.operationHandle.read(iprot); + struct.setOperationHandleIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java new file mode 100644 index 0000000000000..a3879d830000b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/THandleIdentifier.java @@ -0,0 +1,508 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class THandleIdentifier implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("THandleIdentifier"); + + private static final org.apache.thrift.protocol.TField GUID_FIELD_DESC = new org.apache.thrift.protocol.TField("guid", org.apache.thrift.protocol.TType.STRING, (short)1); + private static final org.apache.thrift.protocol.TField SECRET_FIELD_DESC = new org.apache.thrift.protocol.TField("secret", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new THandleIdentifierStandardSchemeFactory()); + schemes.put(TupleScheme.class, new THandleIdentifierTupleSchemeFactory()); + } + + private ByteBuffer guid; // required + private ByteBuffer secret; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + GUID((short)1, "guid"), + SECRET((short)2, "secret"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // GUID + return GUID; + case 2: // SECRET + return SECRET; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.GUID, new org.apache.thrift.meta_data.FieldMetaData("guid", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.SECRET, new org.apache.thrift.meta_data.FieldMetaData("secret", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(THandleIdentifier.class, metaDataMap); + } + + public THandleIdentifier() { + } + + public THandleIdentifier( + ByteBuffer guid, + ByteBuffer secret) + { + this(); + this.guid = org.apache.thrift.TBaseHelper.copyBinary(guid); + this.secret = org.apache.thrift.TBaseHelper.copyBinary(secret); + } + + /** + * Performs a deep copy on other. + */ + public THandleIdentifier(THandleIdentifier other) { + if (other.isSetGuid()) { + this.guid = org.apache.thrift.TBaseHelper.copyBinary(other.guid); + } + if (other.isSetSecret()) { + this.secret = org.apache.thrift.TBaseHelper.copyBinary(other.secret); + } + } + + public THandleIdentifier deepCopy() { + return new THandleIdentifier(this); + } + + @Override + public void clear() { + this.guid = null; + this.secret = null; + } + + public byte[] getGuid() { + setGuid(org.apache.thrift.TBaseHelper.rightSize(guid)); + return guid == null ? null : guid.array(); + } + + public ByteBuffer bufferForGuid() { + return org.apache.thrift.TBaseHelper.copyBinary(guid); + } + + public void setGuid(byte[] guid) { + this.guid = guid == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(guid, guid.length)); + } + + public void setGuid(ByteBuffer guid) { + this.guid = org.apache.thrift.TBaseHelper.copyBinary(guid); + } + + public void unsetGuid() { + this.guid = null; + } + + /** Returns true if field guid is set (has been assigned a value) and false otherwise */ + public boolean isSetGuid() { + return this.guid != null; + } + + public void setGuidIsSet(boolean value) { + if (!value) { + this.guid = null; + } + } + + public byte[] getSecret() { + setSecret(org.apache.thrift.TBaseHelper.rightSize(secret)); + return secret == null ? null : secret.array(); + } + + public ByteBuffer bufferForSecret() { + return org.apache.thrift.TBaseHelper.copyBinary(secret); + } + + public void setSecret(byte[] secret) { + this.secret = secret == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(secret, secret.length)); + } + + public void setSecret(ByteBuffer secret) { + this.secret = org.apache.thrift.TBaseHelper.copyBinary(secret); + } + + public void unsetSecret() { + this.secret = null; + } + + /** Returns true if field secret is set (has been assigned a value) and false otherwise */ + public boolean isSetSecret() { + return this.secret != null; + } + + public void setSecretIsSet(boolean value) { + if (!value) { + this.secret = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case GUID: + if (value == null) { + unsetGuid(); + } else { + setGuid((ByteBuffer)value); + } + break; + + case SECRET: + if (value == null) { + unsetSecret(); + } else { + setSecret((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case GUID: + return getGuid(); + + case SECRET: + return getSecret(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case GUID: + return isSetGuid(); + case SECRET: + return isSetSecret(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof THandleIdentifier) + return this.equals((THandleIdentifier)that); + return false; + } + + public boolean equals(THandleIdentifier that) { + if (that == null) + return false; + + boolean this_present_guid = true && this.isSetGuid(); + boolean that_present_guid = true && that.isSetGuid(); + if (this_present_guid || that_present_guid) { + if (!(this_present_guid && that_present_guid)) + return false; + if (!this.guid.equals(that.guid)) + return false; + } + + boolean this_present_secret = true && this.isSetSecret(); + boolean that_present_secret = true && that.isSetSecret(); + if (this_present_secret || that_present_secret) { + if (!(this_present_secret && that_present_secret)) + return false; + if (!this.secret.equals(that.secret)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_guid = true && (isSetGuid()); + list.add(present_guid); + if (present_guid) + list.add(guid); + + boolean present_secret = true && (isSetSecret()); + list.add(present_secret); + if (present_secret) + list.add(secret); + + return list.hashCode(); + } + + @Override + public int compareTo(THandleIdentifier other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetGuid()).compareTo(other.isSetGuid()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetGuid()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.guid, other.guid); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSecret()).compareTo(other.isSetSecret()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSecret()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.secret, other.secret); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("THandleIdentifier("); + boolean first = true; + + sb.append("guid:"); + if (this.guid == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.guid, sb); + } + first = false; + if (!first) sb.append(", "); + sb.append("secret:"); + if (this.secret == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.secret, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetGuid()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'guid' is unset! Struct:" + toString()); + } + + if (!isSetSecret()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'secret' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class THandleIdentifierStandardSchemeFactory implements SchemeFactory { + public THandleIdentifierStandardScheme getScheme() { + return new THandleIdentifierStandardScheme(); + } + } + + private static class THandleIdentifierStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, THandleIdentifier struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // GUID + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.guid = iprot.readBinary(); + struct.setGuidIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SECRET + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.secret = iprot.readBinary(); + struct.setSecretIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, THandleIdentifier struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.guid != null) { + oprot.writeFieldBegin(GUID_FIELD_DESC); + oprot.writeBinary(struct.guid); + oprot.writeFieldEnd(); + } + if (struct.secret != null) { + oprot.writeFieldBegin(SECRET_FIELD_DESC); + oprot.writeBinary(struct.secret); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class THandleIdentifierTupleSchemeFactory implements SchemeFactory { + public THandleIdentifierTupleScheme getScheme() { + return new THandleIdentifierTupleScheme(); + } + } + + private static class THandleIdentifierTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeBinary(struct.guid); + oprot.writeBinary(struct.secret); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, THandleIdentifier struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.guid = iprot.readBinary(); + struct.setGuidIsSet(true); + struct.secret = iprot.readBinary(); + struct.setSecretIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java new file mode 100644 index 0000000000000..3c44b602b4ff7 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI16Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI16ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI16ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Column.class, metaDataMap); + } + + public TI16Column() { + } + + public TI16Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TI16Column(TI16Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TI16Column deepCopy() { + return new TI16Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(short elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI16Column) + return this.equals((TI16Column)that); + return false; + } + + public boolean equals(TI16Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TI16Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI16Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI16ColumnStandardSchemeFactory implements SchemeFactory { + public TI16ColumnStandardScheme getScheme() { + return new TI16ColumnStandardScheme(); + } + } + + private static class TI16ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list70 = iprot.readListBegin(); + struct.values = new ArrayList(_list70.size); + short _elem71; + for (int _i72 = 0; _i72 < _list70.size; ++_i72) + { + _elem71 = iprot.readI16(); + struct.values.add(_elem71); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, struct.values.size())); + for (short _iter73 : struct.values) + { + oprot.writeI16(_iter73); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI16ColumnTupleSchemeFactory implements SchemeFactory { + public TI16ColumnTupleScheme getScheme() { + return new TI16ColumnTupleScheme(); + } + } + + private static class TI16ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (short _iter74 : struct.values) + { + oprot.writeI16(_iter74); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI16Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list75 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I16, iprot.readI32()); + struct.values = new ArrayList(_list75.size); + short _elem76; + for (int _i77 = 0; _i77 < _list75.size; ++_i77) + { + _elem76 = iprot.readI16(); + struct.values.add(_elem76); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java new file mode 100644 index 0000000000000..29fb4cb85201d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI16Value.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI16Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI16Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I16, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI16ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI16ValueTupleSchemeFactory()); + } + + private short value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I16))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI16Value.class, metaDataMap); + } + + public TI16Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI16Value(TI16Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI16Value deepCopy() { + return new TI16Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public short getValue() { + return this.value; + } + + public void setValue(short value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Short)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI16Value) + return this.equals((TI16Value)that); + return false; + } + + public boolean equals(TI16Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TI16Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI16Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI16ValueStandardSchemeFactory implements SchemeFactory { + public TI16ValueStandardScheme getScheme() { + return new TI16ValueStandardScheme(); + } + } + + private static class TI16ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI16Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I16) { + struct.value = iprot.readI16(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI16Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI16(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI16ValueTupleSchemeFactory implements SchemeFactory { + public TI16ValueTupleScheme getScheme() { + return new TI16ValueTupleScheme(); + } + } + + private static class TI16ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI16(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI16Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI16(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java new file mode 100644 index 0000000000000..9834f1ce8f01b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI32Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI32ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI32ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Column.class, metaDataMap); + } + + public TI32Column() { + } + + public TI32Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TI32Column(TI32Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TI32Column deepCopy() { + return new TI32Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(int elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI32Column) + return this.equals((TI32Column)that); + return false; + } + + public boolean equals(TI32Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TI32Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI32Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI32ColumnStandardSchemeFactory implements SchemeFactory { + public TI32ColumnStandardScheme getScheme() { + return new TI32ColumnStandardScheme(); + } + } + + private static class TI32ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list78 = iprot.readListBegin(); + struct.values = new ArrayList(_list78.size); + int _elem79; + for (int _i80 = 0; _i80 < _list78.size; ++_i80) + { + _elem79 = iprot.readI32(); + struct.values.add(_elem79); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, struct.values.size())); + for (int _iter81 : struct.values) + { + oprot.writeI32(_iter81); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI32ColumnTupleSchemeFactory implements SchemeFactory { + public TI32ColumnTupleScheme getScheme() { + return new TI32ColumnTupleScheme(); + } + } + + private static class TI32ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (int _iter82 : struct.values) + { + oprot.writeI32(_iter82); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI32Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list83 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.values = new ArrayList(_list83.size); + int _elem84; + for (int _i85 = 0; _i85 < _list83.size; ++_i85) + { + _elem84 = iprot.readI32(); + struct.values.add(_elem84); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java new file mode 100644 index 0000000000000..8a69632b2d76e --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI32Value.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI32Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI32Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I32, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI32ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI32ValueTupleSchemeFactory()); + } + + private int value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI32Value.class, metaDataMap); + } + + public TI32Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI32Value(TI32Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI32Value deepCopy() { + return new TI32Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public int getValue() { + return this.value; + } + + public void setValue(int value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI32Value) + return this.equals((TI32Value)that); + return false; + } + + public boolean equals(TI32Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TI32Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI32Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI32ValueStandardSchemeFactory implements SchemeFactory { + public TI32ValueStandardScheme getScheme() { + return new TI32ValueStandardScheme(); + } + } + + private static class TI32ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI32Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.value = iprot.readI32(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI32Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI32(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI32ValueTupleSchemeFactory implements SchemeFactory { + public TI32ValueTupleScheme getScheme() { + return new TI32ValueTupleScheme(); + } + } + + private static class TI32ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI32(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI32Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI32(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java new file mode 100644 index 0000000000000..cd5ef2d7a9ed9 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Column.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI64Column implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Column"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI64ColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI64ColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Column.class, metaDataMap); + } + + public TI64Column() { + } + + public TI64Column( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TI64Column(TI64Column other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TI64Column deepCopy() { + return new TI64Column(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(long elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI64Column) + return this.equals((TI64Column)that); + return false; + } + + public boolean equals(TI64Column that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TI64Column other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI64Column("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI64ColumnStandardSchemeFactory implements SchemeFactory { + public TI64ColumnStandardScheme getScheme() { + return new TI64ColumnStandardScheme(); + } + } + + private static class TI64ColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Column struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list86 = iprot.readListBegin(); + struct.values = new ArrayList(_list86.size); + long _elem87; + for (int _i88 = 0; _i88 < _list86.size; ++_i88) + { + _elem87 = iprot.readI64(); + struct.values.add(_elem87); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Column struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, struct.values.size())); + for (long _iter89 : struct.values) + { + oprot.writeI64(_iter89); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI64ColumnTupleSchemeFactory implements SchemeFactory { + public TI64ColumnTupleScheme getScheme() { + return new TI64ColumnTupleScheme(); + } + } + + private static class TI64ColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (long _iter90 : struct.values) + { + oprot.writeI64(_iter90); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI64Column struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list91 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.I64, iprot.readI32()); + struct.values = new ArrayList(_list91.size); + long _elem92; + for (int _i93 = 0; _i93 < _list91.size; ++_i93) + { + _elem92 = iprot.readI64(); + struct.values.add(_elem92); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java new file mode 100644 index 0000000000000..393c0bd28610d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TI64Value.java @@ -0,0 +1,390 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TI64Value implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TI64Value"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.I64, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TI64ValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TI64ValueTupleSchemeFactory()); + } + + private long value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __VALUE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TI64Value.class, metaDataMap); + } + + public TI64Value() { + } + + /** + * Performs a deep copy on other. + */ + public TI64Value(TI64Value other) { + __isset_bitfield = other.__isset_bitfield; + this.value = other.value; + } + + public TI64Value deepCopy() { + return new TI64Value(this); + } + + @Override + public void clear() { + setValueIsSet(false); + this.value = 0; + } + + public long getValue() { + return this.value; + } + + public void setValue(long value) { + this.value = value; + setValueIsSet(true); + } + + public void unsetValue() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return EncodingUtils.testBit(__isset_bitfield, __VALUE_ISSET_ID); + } + + public void setValueIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUE_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TI64Value) + return this.equals((TI64Value)that); + return false; + } + + public boolean equals(TI64Value that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (this.value != that.value) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TI64Value other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TI64Value("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + sb.append(this.value); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TI64ValueStandardSchemeFactory implements SchemeFactory { + public TI64ValueStandardScheme getScheme() { + return new TI64ValueStandardScheme(); + } + } + + private static class TI64ValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TI64Value struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.value = iprot.readI64(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TI64Value struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeI64(struct.value); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TI64ValueTupleSchemeFactory implements SchemeFactory { + public TI64ValueTupleScheme getScheme() { + return new TI64ValueTupleScheme(); + } + } + + private static class TI64ValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeI64(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TI64Value struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readI64(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java new file mode 100644 index 0000000000000..b39f208c1b878 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TJobExecutionStatus.java @@ -0,0 +1,48 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TJobExecutionStatus implements org.apache.thrift.TEnum { + IN_PROGRESS(0), + COMPLETE(1), + NOT_AVAILABLE(2); + + private final int value; + + private TJobExecutionStatus(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TJobExecutionStatus findByValue(int value) { + switch (value) { + case 0: + return IN_PROGRESS; + case 1: + return COMPLETE; + case 2: + return NOT_AVAILABLE; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java new file mode 100644 index 0000000000000..7ebc15c9432be --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TMapTypeEntry.java @@ -0,0 +1,482 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TMapTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TMapTypeEntry"); + + private static final org.apache.thrift.protocol.TField KEY_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("keyTypePtr", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField VALUE_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("valueTypePtr", org.apache.thrift.protocol.TType.I32, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TMapTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TMapTypeEntryTupleSchemeFactory()); + } + + private int keyTypePtr; // required + private int valueTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + KEY_TYPE_PTR((short)1, "keyTypePtr"), + VALUE_TYPE_PTR((short)2, "valueTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // KEY_TYPE_PTR + return KEY_TYPE_PTR; + case 2: // VALUE_TYPE_PTR + return VALUE_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __KEYTYPEPTR_ISSET_ID = 0; + private static final int __VALUETYPEPTR_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.KEY_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("keyTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + tmpMap.put(_Fields.VALUE_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("valueTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr"))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TMapTypeEntry.class, metaDataMap); + } + + public TMapTypeEntry() { + } + + public TMapTypeEntry( + int keyTypePtr, + int valueTypePtr) + { + this(); + this.keyTypePtr = keyTypePtr; + setKeyTypePtrIsSet(true); + this.valueTypePtr = valueTypePtr; + setValueTypePtrIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TMapTypeEntry(TMapTypeEntry other) { + __isset_bitfield = other.__isset_bitfield; + this.keyTypePtr = other.keyTypePtr; + this.valueTypePtr = other.valueTypePtr; + } + + public TMapTypeEntry deepCopy() { + return new TMapTypeEntry(this); + } + + @Override + public void clear() { + setKeyTypePtrIsSet(false); + this.keyTypePtr = 0; + setValueTypePtrIsSet(false); + this.valueTypePtr = 0; + } + + public int getKeyTypePtr() { + return this.keyTypePtr; + } + + public void setKeyTypePtr(int keyTypePtr) { + this.keyTypePtr = keyTypePtr; + setKeyTypePtrIsSet(true); + } + + public void unsetKeyTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); + } + + /** Returns true if field keyTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetKeyTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID); + } + + public void setKeyTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __KEYTYPEPTR_ISSET_ID, value); + } + + public int getValueTypePtr() { + return this.valueTypePtr; + } + + public void setValueTypePtr(int valueTypePtr) { + this.valueTypePtr = valueTypePtr; + setValueTypePtrIsSet(true); + } + + public void unsetValueTypePtr() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); + } + + /** Returns true if field valueTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetValueTypePtr() { + return EncodingUtils.testBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID); + } + + public void setValueTypePtrIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __VALUETYPEPTR_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case KEY_TYPE_PTR: + if (value == null) { + unsetKeyTypePtr(); + } else { + setKeyTypePtr((Integer)value); + } + break; + + case VALUE_TYPE_PTR: + if (value == null) { + unsetValueTypePtr(); + } else { + setValueTypePtr((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case KEY_TYPE_PTR: + return getKeyTypePtr(); + + case VALUE_TYPE_PTR: + return getValueTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case KEY_TYPE_PTR: + return isSetKeyTypePtr(); + case VALUE_TYPE_PTR: + return isSetValueTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TMapTypeEntry) + return this.equals((TMapTypeEntry)that); + return false; + } + + public boolean equals(TMapTypeEntry that) { + if (that == null) + return false; + + boolean this_present_keyTypePtr = true; + boolean that_present_keyTypePtr = true; + if (this_present_keyTypePtr || that_present_keyTypePtr) { + if (!(this_present_keyTypePtr && that_present_keyTypePtr)) + return false; + if (this.keyTypePtr != that.keyTypePtr) + return false; + } + + boolean this_present_valueTypePtr = true; + boolean that_present_valueTypePtr = true; + if (this_present_valueTypePtr || that_present_valueTypePtr) { + if (!(this_present_valueTypePtr && that_present_valueTypePtr)) + return false; + if (this.valueTypePtr != that.valueTypePtr) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_keyTypePtr = true; + list.add(present_keyTypePtr); + if (present_keyTypePtr) + list.add(keyTypePtr); + + boolean present_valueTypePtr = true; + list.add(present_valueTypePtr); + if (present_valueTypePtr) + list.add(valueTypePtr); + + return list.hashCode(); + } + + @Override + public int compareTo(TMapTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetKeyTypePtr()).compareTo(other.isSetKeyTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetKeyTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.keyTypePtr, other.keyTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetValueTypePtr()).compareTo(other.isSetValueTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValueTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.valueTypePtr, other.valueTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TMapTypeEntry("); + boolean first = true; + + sb.append("keyTypePtr:"); + sb.append(this.keyTypePtr); + first = false; + if (!first) sb.append(", "); + sb.append("valueTypePtr:"); + sb.append(this.valueTypePtr); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetKeyTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'keyTypePtr' is unset! Struct:" + toString()); + } + + if (!isSetValueTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'valueTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TMapTypeEntryStandardSchemeFactory implements SchemeFactory { + public TMapTypeEntryStandardScheme getScheme() { + return new TMapTypeEntryStandardScheme(); + } + } + + private static class TMapTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TMapTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // KEY_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.keyTypePtr = iprot.readI32(); + struct.setKeyTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // VALUE_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.valueTypePtr = iprot.readI32(); + struct.setValueTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TMapTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(KEY_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.keyTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldBegin(VALUE_TYPE_PTR_FIELD_DESC); + oprot.writeI32(struct.valueTypePtr); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TMapTypeEntryTupleSchemeFactory implements SchemeFactory { + public TMapTypeEntryTupleScheme getScheme() { + return new TMapTypeEntryTupleScheme(); + } + } + + private static class TMapTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.keyTypePtr); + oprot.writeI32(struct.valueTypePtr); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TMapTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.keyTypePtr = iprot.readI32(); + struct.setKeyTypePtrIsSet(true); + struct.valueTypePtr = iprot.readI32(); + struct.setValueTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java new file mode 100644 index 0000000000000..e47abbb862cf1 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionReq.java @@ -0,0 +1,778 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TOpenSessionReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionReq"); + + private static final org.apache.thrift.protocol.TField CLIENT_PROTOCOL_FIELD_DESC = new org.apache.thrift.protocol.TField("client_protocol", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField USERNAME_FIELD_DESC = new org.apache.thrift.protocol.TField("username", org.apache.thrift.protocol.TType.STRING, (short)2); + private static final org.apache.thrift.protocol.TField PASSWORD_FIELD_DESC = new org.apache.thrift.protocol.TField("password", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOpenSessionReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOpenSessionReqTupleSchemeFactory()); + } + + private TProtocolVersion client_protocol; // required + private String username; // optional + private String password; // optional + private Map configuration; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TProtocolVersion + */ + CLIENT_PROTOCOL((short)1, "client_protocol"), + USERNAME((short)2, "username"), + PASSWORD((short)3, "password"), + CONFIGURATION((short)4, "configuration"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // CLIENT_PROTOCOL + return CLIENT_PROTOCOL; + case 2: // USERNAME + return USERNAME; + case 3: // PASSWORD + return PASSWORD; + case 4: // CONFIGURATION + return CONFIGURATION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.USERNAME,_Fields.PASSWORD,_Fields.CONFIGURATION}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.CLIENT_PROTOCOL, new org.apache.thrift.meta_data.FieldMetaData("client_protocol", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); + tmpMap.put(_Fields.USERNAME, new org.apache.thrift.meta_data.FieldMetaData("username", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.PASSWORD, new org.apache.thrift.meta_data.FieldMetaData("password", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionReq.class, metaDataMap); + } + + public TOpenSessionReq() { + this.client_protocol = org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10; + + } + + public TOpenSessionReq( + TProtocolVersion client_protocol) + { + this(); + this.client_protocol = client_protocol; + } + + /** + * Performs a deep copy on other. + */ + public TOpenSessionReq(TOpenSessionReq other) { + if (other.isSetClient_protocol()) { + this.client_protocol = other.client_protocol; + } + if (other.isSetUsername()) { + this.username = other.username; + } + if (other.isSetPassword()) { + this.password = other.password; + } + if (other.isSetConfiguration()) { + Map __this__configuration = new HashMap(other.configuration); + this.configuration = __this__configuration; + } + } + + public TOpenSessionReq deepCopy() { + return new TOpenSessionReq(this); + } + + @Override + public void clear() { + this.client_protocol = org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10; + + this.username = null; + this.password = null; + this.configuration = null; + } + + /** + * + * @see TProtocolVersion + */ + public TProtocolVersion getClient_protocol() { + return this.client_protocol; + } + + /** + * + * @see TProtocolVersion + */ + public void setClient_protocol(TProtocolVersion client_protocol) { + this.client_protocol = client_protocol; + } + + public void unsetClient_protocol() { + this.client_protocol = null; + } + + /** Returns true if field client_protocol is set (has been assigned a value) and false otherwise */ + public boolean isSetClient_protocol() { + return this.client_protocol != null; + } + + public void setClient_protocolIsSet(boolean value) { + if (!value) { + this.client_protocol = null; + } + } + + public String getUsername() { + return this.username; + } + + public void setUsername(String username) { + this.username = username; + } + + public void unsetUsername() { + this.username = null; + } + + /** Returns true if field username is set (has been assigned a value) and false otherwise */ + public boolean isSetUsername() { + return this.username != null; + } + + public void setUsernameIsSet(boolean value) { + if (!value) { + this.username = null; + } + } + + public String getPassword() { + return this.password; + } + + public void setPassword(String password) { + this.password = password; + } + + public void unsetPassword() { + this.password = null; + } + + /** Returns true if field password is set (has been assigned a value) and false otherwise */ + public boolean isSetPassword() { + return this.password != null; + } + + public void setPasswordIsSet(boolean value) { + if (!value) { + this.password = null; + } + } + + public int getConfigurationSize() { + return (this.configuration == null) ? 0 : this.configuration.size(); + } + + public void putToConfiguration(String key, String val) { + if (this.configuration == null) { + this.configuration = new HashMap(); + } + this.configuration.put(key, val); + } + + public Map getConfiguration() { + return this.configuration; + } + + public void setConfiguration(Map configuration) { + this.configuration = configuration; + } + + public void unsetConfiguration() { + this.configuration = null; + } + + /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ + public boolean isSetConfiguration() { + return this.configuration != null; + } + + public void setConfigurationIsSet(boolean value) { + if (!value) { + this.configuration = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case CLIENT_PROTOCOL: + if (value == null) { + unsetClient_protocol(); + } else { + setClient_protocol((TProtocolVersion)value); + } + break; + + case USERNAME: + if (value == null) { + unsetUsername(); + } else { + setUsername((String)value); + } + break; + + case PASSWORD: + if (value == null) { + unsetPassword(); + } else { + setPassword((String)value); + } + break; + + case CONFIGURATION: + if (value == null) { + unsetConfiguration(); + } else { + setConfiguration((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case CLIENT_PROTOCOL: + return getClient_protocol(); + + case USERNAME: + return getUsername(); + + case PASSWORD: + return getPassword(); + + case CONFIGURATION: + return getConfiguration(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case CLIENT_PROTOCOL: + return isSetClient_protocol(); + case USERNAME: + return isSetUsername(); + case PASSWORD: + return isSetPassword(); + case CONFIGURATION: + return isSetConfiguration(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOpenSessionReq) + return this.equals((TOpenSessionReq)that); + return false; + } + + public boolean equals(TOpenSessionReq that) { + if (that == null) + return false; + + boolean this_present_client_protocol = true && this.isSetClient_protocol(); + boolean that_present_client_protocol = true && that.isSetClient_protocol(); + if (this_present_client_protocol || that_present_client_protocol) { + if (!(this_present_client_protocol && that_present_client_protocol)) + return false; + if (!this.client_protocol.equals(that.client_protocol)) + return false; + } + + boolean this_present_username = true && this.isSetUsername(); + boolean that_present_username = true && that.isSetUsername(); + if (this_present_username || that_present_username) { + if (!(this_present_username && that_present_username)) + return false; + if (!this.username.equals(that.username)) + return false; + } + + boolean this_present_password = true && this.isSetPassword(); + boolean that_present_password = true && that.isSetPassword(); + if (this_present_password || that_present_password) { + if (!(this_present_password && that_present_password)) + return false; + if (!this.password.equals(that.password)) + return false; + } + + boolean this_present_configuration = true && this.isSetConfiguration(); + boolean that_present_configuration = true && that.isSetConfiguration(); + if (this_present_configuration || that_present_configuration) { + if (!(this_present_configuration && that_present_configuration)) + return false; + if (!this.configuration.equals(that.configuration)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_client_protocol = true && (isSetClient_protocol()); + list.add(present_client_protocol); + if (present_client_protocol) + list.add(client_protocol.getValue()); + + boolean present_username = true && (isSetUsername()); + list.add(present_username); + if (present_username) + list.add(username); + + boolean present_password = true && (isSetPassword()); + list.add(present_password); + if (present_password) + list.add(password); + + boolean present_configuration = true && (isSetConfiguration()); + list.add(present_configuration); + if (present_configuration) + list.add(configuration); + + return list.hashCode(); + } + + @Override + public int compareTo(TOpenSessionReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetClient_protocol()).compareTo(other.isSetClient_protocol()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetClient_protocol()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.client_protocol, other.client_protocol); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetUsername()).compareTo(other.isSetUsername()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetUsername()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.username, other.username); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetPassword()).compareTo(other.isSetPassword()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetPassword()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.password, other.password); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(other.isSetConfiguration()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfiguration()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, other.configuration); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOpenSessionReq("); + boolean first = true; + + sb.append("client_protocol:"); + if (this.client_protocol == null) { + sb.append("null"); + } else { + sb.append(this.client_protocol); + } + first = false; + if (isSetUsername()) { + if (!first) sb.append(", "); + sb.append("username:"); + if (this.username == null) { + sb.append("null"); + } else { + sb.append(this.username); + } + first = false; + } + if (isSetPassword()) { + if (!first) sb.append(", "); + sb.append("password:"); + if (this.password == null) { + sb.append("null"); + } else { + sb.append(this.password); + } + first = false; + } + if (isSetConfiguration()) { + if (!first) sb.append(", "); + sb.append("configuration:"); + if (this.configuration == null) { + sb.append("null"); + } else { + sb.append(this.configuration); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetClient_protocol()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'client_protocol' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOpenSessionReqStandardSchemeFactory implements SchemeFactory { + public TOpenSessionReqStandardScheme getScheme() { + return new TOpenSessionReqStandardScheme(); + } + } + + private static class TOpenSessionReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // CLIENT_PROTOCOL + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.client_protocol = org.apache.hive.service.rpc.thrift.TProtocolVersion.findByValue(iprot.readI32()); + struct.setClient_protocolIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // USERNAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.username = iprot.readString(); + struct.setUsernameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PASSWORD + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.password = iprot.readString(); + struct.setPasswordIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // CONFIGURATION + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map142 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map142.size); + String _key143; + String _val144; + for (int _i145 = 0; _i145 < _map142.size; ++_i145) + { + _key143 = iprot.readString(); + _val144 = iprot.readString(); + struct.configuration.put(_key143, _val144); + } + iprot.readMapEnd(); + } + struct.setConfigurationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.client_protocol != null) { + oprot.writeFieldBegin(CLIENT_PROTOCOL_FIELD_DESC); + oprot.writeI32(struct.client_protocol.getValue()); + oprot.writeFieldEnd(); + } + if (struct.username != null) { + if (struct.isSetUsername()) { + oprot.writeFieldBegin(USERNAME_FIELD_DESC); + oprot.writeString(struct.username); + oprot.writeFieldEnd(); + } + } + if (struct.password != null) { + if (struct.isSetPassword()) { + oprot.writeFieldBegin(PASSWORD_FIELD_DESC); + oprot.writeString(struct.password); + oprot.writeFieldEnd(); + } + } + if (struct.configuration != null) { + if (struct.isSetConfiguration()) { + oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); + for (Map.Entry _iter146 : struct.configuration.entrySet()) + { + oprot.writeString(_iter146.getKey()); + oprot.writeString(_iter146.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOpenSessionReqTupleSchemeFactory implements SchemeFactory { + public TOpenSessionReqTupleScheme getScheme() { + return new TOpenSessionReqTupleScheme(); + } + } + + private static class TOpenSessionReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.client_protocol.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetUsername()) { + optionals.set(0); + } + if (struct.isSetPassword()) { + optionals.set(1); + } + if (struct.isSetConfiguration()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetUsername()) { + oprot.writeString(struct.username); + } + if (struct.isSetPassword()) { + oprot.writeString(struct.password); + } + if (struct.isSetConfiguration()) { + { + oprot.writeI32(struct.configuration.size()); + for (Map.Entry _iter147 : struct.configuration.entrySet()) + { + oprot.writeString(_iter147.getKey()); + oprot.writeString(_iter147.getValue()); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.client_protocol = org.apache.hive.service.rpc.thrift.TProtocolVersion.findByValue(iprot.readI32()); + struct.setClient_protocolIsSet(true); + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + struct.username = iprot.readString(); + struct.setUsernameIsSet(true); + } + if (incoming.get(1)) { + struct.password = iprot.readString(); + struct.setPasswordIsSet(true); + } + if (incoming.get(2)) { + { + org.apache.thrift.protocol.TMap _map148 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map148.size); + String _key149; + String _val150; + for (int _i151 = 0; _i151 < _map148.size; ++_i151) + { + _key149 = iprot.readString(); + _val150 = iprot.readString(); + struct.configuration.put(_key149, _val150); + } + } + struct.setConfigurationIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java new file mode 100644 index 0000000000000..ee1c87bfd76fa --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOpenSessionResp.java @@ -0,0 +1,783 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TOpenSessionResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOpenSessionResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField SERVER_PROTOCOL_VERSION_FIELD_DESC = new org.apache.thrift.protocol.TField("serverProtocolVersion", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField CONFIGURATION_FIELD_DESC = new org.apache.thrift.protocol.TField("configuration", org.apache.thrift.protocol.TType.MAP, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOpenSessionRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOpenSessionRespTupleSchemeFactory()); + } + + private TStatus status; // required + private TProtocolVersion serverProtocolVersion; // required + private TSessionHandle sessionHandle; // optional + private Map configuration; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"), + /** + * + * @see TProtocolVersion + */ + SERVER_PROTOCOL_VERSION((short)2, "serverProtocolVersion"), + SESSION_HANDLE((short)3, "sessionHandle"), + CONFIGURATION((short)4, "configuration"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + case 2: // SERVER_PROTOCOL_VERSION + return SERVER_PROTOCOL_VERSION; + case 3: // SESSION_HANDLE + return SESSION_HANDLE; + case 4: // CONFIGURATION + return CONFIGURATION; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.SESSION_HANDLE,_Fields.CONFIGURATION}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + tmpMap.put(_Fields.SERVER_PROTOCOL_VERSION, new org.apache.thrift.meta_data.FieldMetaData("serverProtocolVersion", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TProtocolVersion.class))); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.CONFIGURATION, new org.apache.thrift.meta_data.FieldMetaData("configuration", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOpenSessionResp.class, metaDataMap); + } + + public TOpenSessionResp() { + this.serverProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10; + + } + + public TOpenSessionResp( + TStatus status, + TProtocolVersion serverProtocolVersion) + { + this(); + this.status = status; + this.serverProtocolVersion = serverProtocolVersion; + } + + /** + * Performs a deep copy on other. + */ + public TOpenSessionResp(TOpenSessionResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + if (other.isSetServerProtocolVersion()) { + this.serverProtocolVersion = other.serverProtocolVersion; + } + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetConfiguration()) { + Map __this__configuration = new HashMap(other.configuration); + this.configuration = __this__configuration; + } + } + + public TOpenSessionResp deepCopy() { + return new TOpenSessionResp(this); + } + + @Override + public void clear() { + this.status = null; + this.serverProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V10; + + this.sessionHandle = null; + this.configuration = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + /** + * + * @see TProtocolVersion + */ + public TProtocolVersion getServerProtocolVersion() { + return this.serverProtocolVersion; + } + + /** + * + * @see TProtocolVersion + */ + public void setServerProtocolVersion(TProtocolVersion serverProtocolVersion) { + this.serverProtocolVersion = serverProtocolVersion; + } + + public void unsetServerProtocolVersion() { + this.serverProtocolVersion = null; + } + + /** Returns true if field serverProtocolVersion is set (has been assigned a value) and false otherwise */ + public boolean isSetServerProtocolVersion() { + return this.serverProtocolVersion != null; + } + + public void setServerProtocolVersionIsSet(boolean value) { + if (!value) { + this.serverProtocolVersion = null; + } + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public int getConfigurationSize() { + return (this.configuration == null) ? 0 : this.configuration.size(); + } + + public void putToConfiguration(String key, String val) { + if (this.configuration == null) { + this.configuration = new HashMap(); + } + this.configuration.put(key, val); + } + + public Map getConfiguration() { + return this.configuration; + } + + public void setConfiguration(Map configuration) { + this.configuration = configuration; + } + + public void unsetConfiguration() { + this.configuration = null; + } + + /** Returns true if field configuration is set (has been assigned a value) and false otherwise */ + public boolean isSetConfiguration() { + return this.configuration != null; + } + + public void setConfigurationIsSet(boolean value) { + if (!value) { + this.configuration = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + case SERVER_PROTOCOL_VERSION: + if (value == null) { + unsetServerProtocolVersion(); + } else { + setServerProtocolVersion((TProtocolVersion)value); + } + break; + + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case CONFIGURATION: + if (value == null) { + unsetConfiguration(); + } else { + setConfiguration((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + case SERVER_PROTOCOL_VERSION: + return getServerProtocolVersion(); + + case SESSION_HANDLE: + return getSessionHandle(); + + case CONFIGURATION: + return getConfiguration(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + case SERVER_PROTOCOL_VERSION: + return isSetServerProtocolVersion(); + case SESSION_HANDLE: + return isSetSessionHandle(); + case CONFIGURATION: + return isSetConfiguration(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOpenSessionResp) + return this.equals((TOpenSessionResp)that); + return false; + } + + public boolean equals(TOpenSessionResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_serverProtocolVersion = true && this.isSetServerProtocolVersion(); + boolean that_present_serverProtocolVersion = true && that.isSetServerProtocolVersion(); + if (this_present_serverProtocolVersion || that_present_serverProtocolVersion) { + if (!(this_present_serverProtocolVersion && that_present_serverProtocolVersion)) + return false; + if (!this.serverProtocolVersion.equals(that.serverProtocolVersion)) + return false; + } + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_configuration = true && this.isSetConfiguration(); + boolean that_present_configuration = true && that.isSetConfiguration(); + if (this_present_configuration || that_present_configuration) { + if (!(this_present_configuration && that_present_configuration)) + return false; + if (!this.configuration.equals(that.configuration)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + boolean present_serverProtocolVersion = true && (isSetServerProtocolVersion()); + list.add(present_serverProtocolVersion); + if (present_serverProtocolVersion) + list.add(serverProtocolVersion.getValue()); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_configuration = true && (isSetConfiguration()); + list.add(present_configuration); + if (present_configuration) + list.add(configuration); + + return list.hashCode(); + } + + @Override + public int compareTo(TOpenSessionResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetServerProtocolVersion()).compareTo(other.isSetServerProtocolVersion()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetServerProtocolVersion()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.serverProtocolVersion, other.serverProtocolVersion); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetConfiguration()).compareTo(other.isSetConfiguration()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetConfiguration()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.configuration, other.configuration); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOpenSessionResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("serverProtocolVersion:"); + if (this.serverProtocolVersion == null) { + sb.append("null"); + } else { + sb.append(this.serverProtocolVersion); + } + first = false; + if (isSetSessionHandle()) { + if (!first) sb.append(", "); + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + } + if (isSetConfiguration()) { + if (!first) sb.append(", "); + sb.append("configuration:"); + if (this.configuration == null) { + sb.append("null"); + } else { + sb.append(this.configuration); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetServerProtocolVersion()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'serverProtocolVersion' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOpenSessionRespStandardSchemeFactory implements SchemeFactory { + public TOpenSessionRespStandardScheme getScheme() { + return new TOpenSessionRespStandardScheme(); + } + } + + private static class TOpenSessionRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOpenSessionResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // SERVER_PROTOCOL_VERSION + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.serverProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion.findByValue(iprot.readI32()); + struct.setServerProtocolVersionIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // CONFIGURATION + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map152 = iprot.readMapBegin(); + struct.configuration = new HashMap(2*_map152.size); + String _key153; + String _val154; + for (int _i155 = 0; _i155 < _map152.size; ++_i155) + { + _key153 = iprot.readString(); + _val154 = iprot.readString(); + struct.configuration.put(_key153, _val154); + } + iprot.readMapEnd(); + } + struct.setConfigurationIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOpenSessionResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.serverProtocolVersion != null) { + oprot.writeFieldBegin(SERVER_PROTOCOL_VERSION_FIELD_DESC); + oprot.writeI32(struct.serverProtocolVersion.getValue()); + oprot.writeFieldEnd(); + } + if (struct.sessionHandle != null) { + if (struct.isSetSessionHandle()) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + } + if (struct.configuration != null) { + if (struct.isSetConfiguration()) { + oprot.writeFieldBegin(CONFIGURATION_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, struct.configuration.size())); + for (Map.Entry _iter156 : struct.configuration.entrySet()) + { + oprot.writeString(_iter156.getKey()); + oprot.writeString(_iter156.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOpenSessionRespTupleSchemeFactory implements SchemeFactory { + public TOpenSessionRespTupleScheme getScheme() { + return new TOpenSessionRespTupleScheme(); + } + } + + private static class TOpenSessionRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + oprot.writeI32(struct.serverProtocolVersion.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetSessionHandle()) { + optionals.set(0); + } + if (struct.isSetConfiguration()) { + optionals.set(1); + } + oprot.writeBitSet(optionals, 2); + if (struct.isSetSessionHandle()) { + struct.sessionHandle.write(oprot); + } + if (struct.isSetConfiguration()) { + { + oprot.writeI32(struct.configuration.size()); + for (Map.Entry _iter157 : struct.configuration.entrySet()) + { + oprot.writeString(_iter157.getKey()); + oprot.writeString(_iter157.getValue()); + } + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOpenSessionResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + struct.serverProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion.findByValue(iprot.readI32()); + struct.setServerProtocolVersionIsSet(true); + BitSet incoming = iprot.readBitSet(2); + if (incoming.get(0)) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } + if (incoming.get(1)) { + { + org.apache.thrift.protocol.TMap _map158 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.configuration = new HashMap(2*_map158.size); + String _key159; + String _val160; + for (int _i161 = 0; _i161 < _map158.size; ++_i161) + { + _key159 = iprot.readString(); + _val160 = iprot.readString(); + struct.configuration.put(_key159, _val160); + } + } + struct.setConfigurationIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java new file mode 100644 index 0000000000000..9eaf2be3ed5ea --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationHandle.java @@ -0,0 +1,709 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TOperationHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TOperationHandle"); + + private static final org.apache.thrift.protocol.TField OPERATION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("operationId", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField OPERATION_TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("operationType", org.apache.thrift.protocol.TType.I32, (short)2); + private static final org.apache.thrift.protocol.TField HAS_RESULT_SET_FIELD_DESC = new org.apache.thrift.protocol.TField("hasResultSet", org.apache.thrift.protocol.TType.BOOL, (short)3); + private static final org.apache.thrift.protocol.TField MODIFIED_ROW_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("modifiedRowCount", org.apache.thrift.protocol.TType.DOUBLE, (short)4); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TOperationHandleStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TOperationHandleTupleSchemeFactory()); + } + + private THandleIdentifier operationId; // required + private TOperationType operationType; // required + private boolean hasResultSet; // required + private double modifiedRowCount; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + OPERATION_ID((short)1, "operationId"), + /** + * + * @see TOperationType + */ + OPERATION_TYPE((short)2, "operationType"), + HAS_RESULT_SET((short)3, "hasResultSet"), + MODIFIED_ROW_COUNT((short)4, "modifiedRowCount"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // OPERATION_ID + return OPERATION_ID; + case 2: // OPERATION_TYPE + return OPERATION_TYPE; + case 3: // HAS_RESULT_SET + return HAS_RESULT_SET; + case 4: // MODIFIED_ROW_COUNT + return MODIFIED_ROW_COUNT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __HASRESULTSET_ISSET_ID = 0; + private static final int __MODIFIEDROWCOUNT_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.MODIFIED_ROW_COUNT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.OPERATION_ID, new org.apache.thrift.meta_data.FieldMetaData("operationId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); + tmpMap.put(_Fields.OPERATION_TYPE, new org.apache.thrift.meta_data.FieldMetaData("operationType", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TOperationType.class))); + tmpMap.put(_Fields.HAS_RESULT_SET, new org.apache.thrift.meta_data.FieldMetaData("hasResultSet", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.BOOL))); + tmpMap.put(_Fields.MODIFIED_ROW_COUNT, new org.apache.thrift.meta_data.FieldMetaData("modifiedRowCount", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TOperationHandle.class, metaDataMap); + } + + public TOperationHandle() { + } + + public TOperationHandle( + THandleIdentifier operationId, + TOperationType operationType, + boolean hasResultSet) + { + this(); + this.operationId = operationId; + this.operationType = operationType; + this.hasResultSet = hasResultSet; + setHasResultSetIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TOperationHandle(TOperationHandle other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetOperationId()) { + this.operationId = new THandleIdentifier(other.operationId); + } + if (other.isSetOperationType()) { + this.operationType = other.operationType; + } + this.hasResultSet = other.hasResultSet; + this.modifiedRowCount = other.modifiedRowCount; + } + + public TOperationHandle deepCopy() { + return new TOperationHandle(this); + } + + @Override + public void clear() { + this.operationId = null; + this.operationType = null; + setHasResultSetIsSet(false); + this.hasResultSet = false; + setModifiedRowCountIsSet(false); + this.modifiedRowCount = 0.0; + } + + public THandleIdentifier getOperationId() { + return this.operationId; + } + + public void setOperationId(THandleIdentifier operationId) { + this.operationId = operationId; + } + + public void unsetOperationId() { + this.operationId = null; + } + + /** Returns true if field operationId is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationId() { + return this.operationId != null; + } + + public void setOperationIdIsSet(boolean value) { + if (!value) { + this.operationId = null; + } + } + + /** + * + * @see TOperationType + */ + public TOperationType getOperationType() { + return this.operationType; + } + + /** + * + * @see TOperationType + */ + public void setOperationType(TOperationType operationType) { + this.operationType = operationType; + } + + public void unsetOperationType() { + this.operationType = null; + } + + /** Returns true if field operationType is set (has been assigned a value) and false otherwise */ + public boolean isSetOperationType() { + return this.operationType != null; + } + + public void setOperationTypeIsSet(boolean value) { + if (!value) { + this.operationType = null; + } + } + + public boolean isHasResultSet() { + return this.hasResultSet; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + setHasResultSetIsSet(true); + } + + public void unsetHasResultSet() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + /** Returns true if field hasResultSet is set (has been assigned a value) and false otherwise */ + public boolean isSetHasResultSet() { + return EncodingUtils.testBit(__isset_bitfield, __HASRESULTSET_ISSET_ID); + } + + public void setHasResultSetIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __HASRESULTSET_ISSET_ID, value); + } + + public double getModifiedRowCount() { + return this.modifiedRowCount; + } + + public void setModifiedRowCount(double modifiedRowCount) { + this.modifiedRowCount = modifiedRowCount; + setModifiedRowCountIsSet(true); + } + + public void unsetModifiedRowCount() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); + } + + /** Returns true if field modifiedRowCount is set (has been assigned a value) and false otherwise */ + public boolean isSetModifiedRowCount() { + return EncodingUtils.testBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID); + } + + public void setModifiedRowCountIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __MODIFIEDROWCOUNT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case OPERATION_ID: + if (value == null) { + unsetOperationId(); + } else { + setOperationId((THandleIdentifier)value); + } + break; + + case OPERATION_TYPE: + if (value == null) { + unsetOperationType(); + } else { + setOperationType((TOperationType)value); + } + break; + + case HAS_RESULT_SET: + if (value == null) { + unsetHasResultSet(); + } else { + setHasResultSet((Boolean)value); + } + break; + + case MODIFIED_ROW_COUNT: + if (value == null) { + unsetModifiedRowCount(); + } else { + setModifiedRowCount((Double)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case OPERATION_ID: + return getOperationId(); + + case OPERATION_TYPE: + return getOperationType(); + + case HAS_RESULT_SET: + return isHasResultSet(); + + case MODIFIED_ROW_COUNT: + return getModifiedRowCount(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case OPERATION_ID: + return isSetOperationId(); + case OPERATION_TYPE: + return isSetOperationType(); + case HAS_RESULT_SET: + return isSetHasResultSet(); + case MODIFIED_ROW_COUNT: + return isSetModifiedRowCount(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TOperationHandle) + return this.equals((TOperationHandle)that); + return false; + } + + public boolean equals(TOperationHandle that) { + if (that == null) + return false; + + boolean this_present_operationId = true && this.isSetOperationId(); + boolean that_present_operationId = true && that.isSetOperationId(); + if (this_present_operationId || that_present_operationId) { + if (!(this_present_operationId && that_present_operationId)) + return false; + if (!this.operationId.equals(that.operationId)) + return false; + } + + boolean this_present_operationType = true && this.isSetOperationType(); + boolean that_present_operationType = true && that.isSetOperationType(); + if (this_present_operationType || that_present_operationType) { + if (!(this_present_operationType && that_present_operationType)) + return false; + if (!this.operationType.equals(that.operationType)) + return false; + } + + boolean this_present_hasResultSet = true; + boolean that_present_hasResultSet = true; + if (this_present_hasResultSet || that_present_hasResultSet) { + if (!(this_present_hasResultSet && that_present_hasResultSet)) + return false; + if (this.hasResultSet != that.hasResultSet) + return false; + } + + boolean this_present_modifiedRowCount = true && this.isSetModifiedRowCount(); + boolean that_present_modifiedRowCount = true && that.isSetModifiedRowCount(); + if (this_present_modifiedRowCount || that_present_modifiedRowCount) { + if (!(this_present_modifiedRowCount && that_present_modifiedRowCount)) + return false; + if (this.modifiedRowCount != that.modifiedRowCount) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_operationId = true && (isSetOperationId()); + list.add(present_operationId); + if (present_operationId) + list.add(operationId); + + boolean present_operationType = true && (isSetOperationType()); + list.add(present_operationType); + if (present_operationType) + list.add(operationType.getValue()); + + boolean present_hasResultSet = true; + list.add(present_hasResultSet); + if (present_hasResultSet) + list.add(hasResultSet); + + boolean present_modifiedRowCount = true && (isSetModifiedRowCount()); + list.add(present_modifiedRowCount); + if (present_modifiedRowCount) + list.add(modifiedRowCount); + + return list.hashCode(); + } + + @Override + public int compareTo(TOperationHandle other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetOperationId()).compareTo(other.isSetOperationId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationId, other.operationId); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetOperationType()).compareTo(other.isSetOperationType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetOperationType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.operationType, other.operationType); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetHasResultSet()).compareTo(other.isSetHasResultSet()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHasResultSet()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.hasResultSet, other.hasResultSet); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetModifiedRowCount()).compareTo(other.isSetModifiedRowCount()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetModifiedRowCount()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.modifiedRowCount, other.modifiedRowCount); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TOperationHandle("); + boolean first = true; + + sb.append("operationId:"); + if (this.operationId == null) { + sb.append("null"); + } else { + sb.append(this.operationId); + } + first = false; + if (!first) sb.append(", "); + sb.append("operationType:"); + if (this.operationType == null) { + sb.append("null"); + } else { + sb.append(this.operationType); + } + first = false; + if (!first) sb.append(", "); + sb.append("hasResultSet:"); + sb.append(this.hasResultSet); + first = false; + if (isSetModifiedRowCount()) { + if (!first) sb.append(", "); + sb.append("modifiedRowCount:"); + sb.append(this.modifiedRowCount); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetOperationId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationId' is unset! Struct:" + toString()); + } + + if (!isSetOperationType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'operationType' is unset! Struct:" + toString()); + } + + if (!isSetHasResultSet()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'hasResultSet' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (operationId != null) { + operationId.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TOperationHandleStandardSchemeFactory implements SchemeFactory { + public TOperationHandleStandardScheme getScheme() { + return new TOperationHandleStandardScheme(); + } + } + + private static class TOperationHandleStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TOperationHandle struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // OPERATION_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.operationId = new THandleIdentifier(); + struct.operationId.read(iprot); + struct.setOperationIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // OPERATION_TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.operationType = org.apache.hive.service.rpc.thrift.TOperationType.findByValue(iprot.readI32()); + struct.setOperationTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // HAS_RESULT_SET + if (schemeField.type == org.apache.thrift.protocol.TType.BOOL) { + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // MODIFIED_ROW_COUNT + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.modifiedRowCount = iprot.readDouble(); + struct.setModifiedRowCountIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TOperationHandle struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.operationId != null) { + oprot.writeFieldBegin(OPERATION_ID_FIELD_DESC); + struct.operationId.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.operationType != null) { + oprot.writeFieldBegin(OPERATION_TYPE_FIELD_DESC); + oprot.writeI32(struct.operationType.getValue()); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(HAS_RESULT_SET_FIELD_DESC); + oprot.writeBool(struct.hasResultSet); + oprot.writeFieldEnd(); + if (struct.isSetModifiedRowCount()) { + oprot.writeFieldBegin(MODIFIED_ROW_COUNT_FIELD_DESC); + oprot.writeDouble(struct.modifiedRowCount); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TOperationHandleTupleSchemeFactory implements SchemeFactory { + public TOperationHandleTupleScheme getScheme() { + return new TOperationHandleTupleScheme(); + } + } + + private static class TOperationHandleTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.operationId.write(oprot); + oprot.writeI32(struct.operationType.getValue()); + oprot.writeBool(struct.hasResultSet); + BitSet optionals = new BitSet(); + if (struct.isSetModifiedRowCount()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetModifiedRowCount()) { + oprot.writeDouble(struct.modifiedRowCount); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TOperationHandle struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.operationId = new THandleIdentifier(); + struct.operationId.read(iprot); + struct.setOperationIdIsSet(true); + struct.operationType = org.apache.hive.service.rpc.thrift.TOperationType.findByValue(iprot.readI32()); + struct.setOperationTypeIsSet(true); + struct.hasResultSet = iprot.readBool(); + struct.setHasResultSetIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.modifiedRowCount = iprot.readDouble(); + struct.setModifiedRowCountIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java new file mode 100644 index 0000000000000..4390b4b887583 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationState.java @@ -0,0 +1,66 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TOperationState implements org.apache.thrift.TEnum { + INITIALIZED_STATE(0), + RUNNING_STATE(1), + FINISHED_STATE(2), + CANCELED_STATE(3), + CLOSED_STATE(4), + ERROR_STATE(5), + UKNOWN_STATE(6), + PENDING_STATE(7), + TIMEDOUT_STATE(8); + + private final int value; + + private TOperationState(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TOperationState findByValue(int value) { + switch (value) { + case 0: + return INITIALIZED_STATE; + case 1: + return RUNNING_STATE; + case 2: + return FINISHED_STATE; + case 3: + return CANCELED_STATE; + case 4: + return CLOSED_STATE; + case 5: + return ERROR_STATE; + case 6: + return UKNOWN_STATE; + case 7: + return PENDING_STATE; + case 8: + return TIMEDOUT_STATE; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java new file mode 100644 index 0000000000000..08002ad1dc8e8 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TOperationType.java @@ -0,0 +1,66 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TOperationType implements org.apache.thrift.TEnum { + EXECUTE_STATEMENT(0), + GET_TYPE_INFO(1), + GET_CATALOGS(2), + GET_SCHEMAS(3), + GET_TABLES(4), + GET_TABLE_TYPES(5), + GET_COLUMNS(6), + GET_FUNCTIONS(7), + UNKNOWN(8); + + private final int value; + + private TOperationType(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TOperationType findByValue(int value) { + switch (value) { + case 0: + return EXECUTE_STATEMENT; + case 1: + return GET_TYPE_INFO; + case 2: + return GET_CATALOGS; + case 3: + return GET_SCHEMAS; + case 4: + return GET_TABLES; + case 5: + return GET_TABLE_TYPES; + case 6: + return GET_COLUMNS; + case 7: + return GET_FUNCTIONS; + case 8: + return UNKNOWN; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java new file mode 100644 index 0000000000000..910c90967f614 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TPrimitiveTypeEntry.java @@ -0,0 +1,516 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TPrimitiveTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TPrimitiveTypeEntry"); + + private static final org.apache.thrift.protocol.TField TYPE_FIELD_DESC = new org.apache.thrift.protocol.TField("type", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField TYPE_QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("typeQualifiers", org.apache.thrift.protocol.TType.STRUCT, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TPrimitiveTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TPrimitiveTypeEntryTupleSchemeFactory()); + } + + private TTypeId type; // required + private TTypeQualifiers typeQualifiers; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TTypeId + */ + TYPE((short)1, "type"), + TYPE_QUALIFIERS((short)2, "typeQualifiers"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE + return TYPE; + case 2: // TYPE_QUALIFIERS + return TYPE_QUALIFIERS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.TYPE_QUALIFIERS}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE, new org.apache.thrift.meta_data.FieldMetaData("type", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TTypeId.class))); + tmpMap.put(_Fields.TYPE_QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("typeQualifiers", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifiers.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TPrimitiveTypeEntry.class, metaDataMap); + } + + public TPrimitiveTypeEntry() { + } + + public TPrimitiveTypeEntry( + TTypeId type) + { + this(); + this.type = type; + } + + /** + * Performs a deep copy on other. + */ + public TPrimitiveTypeEntry(TPrimitiveTypeEntry other) { + if (other.isSetType()) { + this.type = other.type; + } + if (other.isSetTypeQualifiers()) { + this.typeQualifiers = new TTypeQualifiers(other.typeQualifiers); + } + } + + public TPrimitiveTypeEntry deepCopy() { + return new TPrimitiveTypeEntry(this); + } + + @Override + public void clear() { + this.type = null; + this.typeQualifiers = null; + } + + /** + * + * @see TTypeId + */ + public TTypeId getType() { + return this.type; + } + + /** + * + * @see TTypeId + */ + public void setType(TTypeId type) { + this.type = type; + } + + public void unsetType() { + this.type = null; + } + + /** Returns true if field type is set (has been assigned a value) and false otherwise */ + public boolean isSetType() { + return this.type != null; + } + + public void setTypeIsSet(boolean value) { + if (!value) { + this.type = null; + } + } + + public TTypeQualifiers getTypeQualifiers() { + return this.typeQualifiers; + } + + public void setTypeQualifiers(TTypeQualifiers typeQualifiers) { + this.typeQualifiers = typeQualifiers; + } + + public void unsetTypeQualifiers() { + this.typeQualifiers = null; + } + + /** Returns true if field typeQualifiers is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeQualifiers() { + return this.typeQualifiers != null; + } + + public void setTypeQualifiersIsSet(boolean value) { + if (!value) { + this.typeQualifiers = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE: + if (value == null) { + unsetType(); + } else { + setType((TTypeId)value); + } + break; + + case TYPE_QUALIFIERS: + if (value == null) { + unsetTypeQualifiers(); + } else { + setTypeQualifiers((TTypeQualifiers)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE: + return getType(); + + case TYPE_QUALIFIERS: + return getTypeQualifiers(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE: + return isSetType(); + case TYPE_QUALIFIERS: + return isSetTypeQualifiers(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TPrimitiveTypeEntry) + return this.equals((TPrimitiveTypeEntry)that); + return false; + } + + public boolean equals(TPrimitiveTypeEntry that) { + if (that == null) + return false; + + boolean this_present_type = true && this.isSetType(); + boolean that_present_type = true && that.isSetType(); + if (this_present_type || that_present_type) { + if (!(this_present_type && that_present_type)) + return false; + if (!this.type.equals(that.type)) + return false; + } + + boolean this_present_typeQualifiers = true && this.isSetTypeQualifiers(); + boolean that_present_typeQualifiers = true && that.isSetTypeQualifiers(); + if (this_present_typeQualifiers || that_present_typeQualifiers) { + if (!(this_present_typeQualifiers && that_present_typeQualifiers)) + return false; + if (!this.typeQualifiers.equals(that.typeQualifiers)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_type = true && (isSetType()); + list.add(present_type); + if (present_type) + list.add(type.getValue()); + + boolean present_typeQualifiers = true && (isSetTypeQualifiers()); + list.add(present_typeQualifiers); + if (present_typeQualifiers) + list.add(typeQualifiers); + + return list.hashCode(); + } + + @Override + public int compareTo(TPrimitiveTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetType()).compareTo(other.isSetType()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetType()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.type, other.type); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetTypeQualifiers()).compareTo(other.isSetTypeQualifiers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeQualifiers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeQualifiers, other.typeQualifiers); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TPrimitiveTypeEntry("); + boolean first = true; + + sb.append("type:"); + if (this.type == null) { + sb.append("null"); + } else { + sb.append(this.type); + } + first = false; + if (isSetTypeQualifiers()) { + if (!first) sb.append(", "); + sb.append("typeQualifiers:"); + if (this.typeQualifiers == null) { + sb.append("null"); + } else { + sb.append(this.typeQualifiers); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetType()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'type' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (typeQualifiers != null) { + typeQualifiers.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TPrimitiveTypeEntryStandardSchemeFactory implements SchemeFactory { + public TPrimitiveTypeEntryStandardScheme getScheme() { + return new TPrimitiveTypeEntryStandardScheme(); + } + } + + private static class TPrimitiveTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.type = org.apache.hive.service.rpc.thrift.TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // TYPE_QUALIFIERS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.typeQualifiers = new TTypeQualifiers(); + struct.typeQualifiers.read(iprot); + struct.setTypeQualifiersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.type != null) { + oprot.writeFieldBegin(TYPE_FIELD_DESC); + oprot.writeI32(struct.type.getValue()); + oprot.writeFieldEnd(); + } + if (struct.typeQualifiers != null) { + if (struct.isSetTypeQualifiers()) { + oprot.writeFieldBegin(TYPE_QUALIFIERS_FIELD_DESC); + struct.typeQualifiers.write(oprot); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TPrimitiveTypeEntryTupleSchemeFactory implements SchemeFactory { + public TPrimitiveTypeEntryTupleScheme getScheme() { + return new TPrimitiveTypeEntryTupleScheme(); + } + } + + private static class TPrimitiveTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.type.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetTypeQualifiers()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetTypeQualifiers()) { + struct.typeQualifiers.write(oprot); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TPrimitiveTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.type = org.apache.hive.service.rpc.thrift.TTypeId.findByValue(iprot.readI32()); + struct.setTypeIsSet(true); + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.typeQualifiers = new TTypeQualifiers(); + struct.typeQualifiers.read(iprot); + struct.setTypeQualifiersIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java new file mode 100644 index 0000000000000..ecc413aad4cdc --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProgressUpdateResp.java @@ -0,0 +1,1033 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TProgressUpdateResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TProgressUpdateResp"); + + private static final org.apache.thrift.protocol.TField HEADER_NAMES_FIELD_DESC = new org.apache.thrift.protocol.TField("headerNames", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField PROGRESSED_PERCENTAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("progressedPercentage", org.apache.thrift.protocol.TType.DOUBLE, (short)3); + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField FOOTER_SUMMARY_FIELD_DESC = new org.apache.thrift.protocol.TField("footerSummary", org.apache.thrift.protocol.TType.STRING, (short)5); + private static final org.apache.thrift.protocol.TField START_TIME_FIELD_DESC = new org.apache.thrift.protocol.TField("startTime", org.apache.thrift.protocol.TType.I64, (short)6); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TProgressUpdateRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TProgressUpdateRespTupleSchemeFactory()); + } + + private List headerNames; // required + private List> rows; // required + private double progressedPercentage; // required + private TJobExecutionStatus status; // required + private String footerSummary; // required + private long startTime; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + HEADER_NAMES((short)1, "headerNames"), + ROWS((short)2, "rows"), + PROGRESSED_PERCENTAGE((short)3, "progressedPercentage"), + /** + * + * @see TJobExecutionStatus + */ + STATUS((short)4, "status"), + FOOTER_SUMMARY((short)5, "footerSummary"), + START_TIME((short)6, "startTime"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // HEADER_NAMES + return HEADER_NAMES; + case 2: // ROWS + return ROWS; + case 3: // PROGRESSED_PERCENTAGE + return PROGRESSED_PERCENTAGE; + case 4: // STATUS + return STATUS; + case 5: // FOOTER_SUMMARY + return FOOTER_SUMMARY; + case 6: // START_TIME + return START_TIME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __PROGRESSEDPERCENTAGE_ISSET_ID = 0; + private static final int __STARTTIME_ISSET_ID = 1; + private byte __isset_bitfield = 0; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.HEADER_NAMES, new org.apache.thrift.meta_data.FieldMetaData("headerNames", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))))); + tmpMap.put(_Fields.PROGRESSED_PERCENTAGE, new org.apache.thrift.meta_data.FieldMetaData("progressedPercentage", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.DOUBLE))); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TJobExecutionStatus.class))); + tmpMap.put(_Fields.FOOTER_SUMMARY, new org.apache.thrift.meta_data.FieldMetaData("footerSummary", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.START_TIME, new org.apache.thrift.meta_data.FieldMetaData("startTime", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TProgressUpdateResp.class, metaDataMap); + } + + public TProgressUpdateResp() { + } + + public TProgressUpdateResp( + List headerNames, + List> rows, + double progressedPercentage, + TJobExecutionStatus status, + String footerSummary, + long startTime) + { + this(); + this.headerNames = headerNames; + this.rows = rows; + this.progressedPercentage = progressedPercentage; + setProgressedPercentageIsSet(true); + this.status = status; + this.footerSummary = footerSummary; + this.startTime = startTime; + setStartTimeIsSet(true); + } + + /** + * Performs a deep copy on other. + */ + public TProgressUpdateResp(TProgressUpdateResp other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetHeaderNames()) { + List __this__headerNames = new ArrayList(other.headerNames); + this.headerNames = __this__headerNames; + } + if (other.isSetRows()) { + List> __this__rows = new ArrayList>(other.rows.size()); + for (List other_element : other.rows) { + List __this__rows_copy = new ArrayList(other_element); + __this__rows.add(__this__rows_copy); + } + this.rows = __this__rows; + } + this.progressedPercentage = other.progressedPercentage; + if (other.isSetStatus()) { + this.status = other.status; + } + if (other.isSetFooterSummary()) { + this.footerSummary = other.footerSummary; + } + this.startTime = other.startTime; + } + + public TProgressUpdateResp deepCopy() { + return new TProgressUpdateResp(this); + } + + @Override + public void clear() { + this.headerNames = null; + this.rows = null; + setProgressedPercentageIsSet(false); + this.progressedPercentage = 0.0; + this.status = null; + this.footerSummary = null; + setStartTimeIsSet(false); + this.startTime = 0; + } + + public int getHeaderNamesSize() { + return (this.headerNames == null) ? 0 : this.headerNames.size(); + } + + public java.util.Iterator getHeaderNamesIterator() { + return (this.headerNames == null) ? null : this.headerNames.iterator(); + } + + public void addToHeaderNames(String elem) { + if (this.headerNames == null) { + this.headerNames = new ArrayList(); + } + this.headerNames.add(elem); + } + + public List getHeaderNames() { + return this.headerNames; + } + + public void setHeaderNames(List headerNames) { + this.headerNames = headerNames; + } + + public void unsetHeaderNames() { + this.headerNames = null; + } + + /** Returns true if field headerNames is set (has been assigned a value) and false otherwise */ + public boolean isSetHeaderNames() { + return this.headerNames != null; + } + + public void setHeaderNamesIsSet(boolean value) { + if (!value) { + this.headerNames = null; + } + } + + public int getRowsSize() { + return (this.rows == null) ? 0 : this.rows.size(); + } + + public java.util.Iterator> getRowsIterator() { + return (this.rows == null) ? null : this.rows.iterator(); + } + + public void addToRows(List elem) { + if (this.rows == null) { + this.rows = new ArrayList>(); + } + this.rows.add(elem); + } + + public List> getRows() { + return this.rows; + } + + public void setRows(List> rows) { + this.rows = rows; + } + + public void unsetRows() { + this.rows = null; + } + + /** Returns true if field rows is set (has been assigned a value) and false otherwise */ + public boolean isSetRows() { + return this.rows != null; + } + + public void setRowsIsSet(boolean value) { + if (!value) { + this.rows = null; + } + } + + public double getProgressedPercentage() { + return this.progressedPercentage; + } + + public void setProgressedPercentage(double progressedPercentage) { + this.progressedPercentage = progressedPercentage; + setProgressedPercentageIsSet(true); + } + + public void unsetProgressedPercentage() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID); + } + + /** Returns true if field progressedPercentage is set (has been assigned a value) and false otherwise */ + public boolean isSetProgressedPercentage() { + return EncodingUtils.testBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID); + } + + public void setProgressedPercentageIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __PROGRESSEDPERCENTAGE_ISSET_ID, value); + } + + /** + * + * @see TJobExecutionStatus + */ + public TJobExecutionStatus getStatus() { + return this.status; + } + + /** + * + * @see TJobExecutionStatus + */ + public void setStatus(TJobExecutionStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public String getFooterSummary() { + return this.footerSummary; + } + + public void setFooterSummary(String footerSummary) { + this.footerSummary = footerSummary; + } + + public void unsetFooterSummary() { + this.footerSummary = null; + } + + /** Returns true if field footerSummary is set (has been assigned a value) and false otherwise */ + public boolean isSetFooterSummary() { + return this.footerSummary != null; + } + + public void setFooterSummaryIsSet(boolean value) { + if (!value) { + this.footerSummary = null; + } + } + + public long getStartTime() { + return this.startTime; + } + + public void setStartTime(long startTime) { + this.startTime = startTime; + setStartTimeIsSet(true); + } + + public void unsetStartTime() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTTIME_ISSET_ID); + } + + /** Returns true if field startTime is set (has been assigned a value) and false otherwise */ + public boolean isSetStartTime() { + return EncodingUtils.testBit(__isset_bitfield, __STARTTIME_ISSET_ID); + } + + public void setStartTimeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTTIME_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case HEADER_NAMES: + if (value == null) { + unsetHeaderNames(); + } else { + setHeaderNames((List)value); + } + break; + + case ROWS: + if (value == null) { + unsetRows(); + } else { + setRows((List>)value); + } + break; + + case PROGRESSED_PERCENTAGE: + if (value == null) { + unsetProgressedPercentage(); + } else { + setProgressedPercentage((Double)value); + } + break; + + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TJobExecutionStatus)value); + } + break; + + case FOOTER_SUMMARY: + if (value == null) { + unsetFooterSummary(); + } else { + setFooterSummary((String)value); + } + break; + + case START_TIME: + if (value == null) { + unsetStartTime(); + } else { + setStartTime((Long)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case HEADER_NAMES: + return getHeaderNames(); + + case ROWS: + return getRows(); + + case PROGRESSED_PERCENTAGE: + return getProgressedPercentage(); + + case STATUS: + return getStatus(); + + case FOOTER_SUMMARY: + return getFooterSummary(); + + case START_TIME: + return getStartTime(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case HEADER_NAMES: + return isSetHeaderNames(); + case ROWS: + return isSetRows(); + case PROGRESSED_PERCENTAGE: + return isSetProgressedPercentage(); + case STATUS: + return isSetStatus(); + case FOOTER_SUMMARY: + return isSetFooterSummary(); + case START_TIME: + return isSetStartTime(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TProgressUpdateResp) + return this.equals((TProgressUpdateResp)that); + return false; + } + + public boolean equals(TProgressUpdateResp that) { + if (that == null) + return false; + + boolean this_present_headerNames = true && this.isSetHeaderNames(); + boolean that_present_headerNames = true && that.isSetHeaderNames(); + if (this_present_headerNames || that_present_headerNames) { + if (!(this_present_headerNames && that_present_headerNames)) + return false; + if (!this.headerNames.equals(that.headerNames)) + return false; + } + + boolean this_present_rows = true && this.isSetRows(); + boolean that_present_rows = true && that.isSetRows(); + if (this_present_rows || that_present_rows) { + if (!(this_present_rows && that_present_rows)) + return false; + if (!this.rows.equals(that.rows)) + return false; + } + + boolean this_present_progressedPercentage = true; + boolean that_present_progressedPercentage = true; + if (this_present_progressedPercentage || that_present_progressedPercentage) { + if (!(this_present_progressedPercentage && that_present_progressedPercentage)) + return false; + if (this.progressedPercentage != that.progressedPercentage) + return false; + } + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + boolean this_present_footerSummary = true && this.isSetFooterSummary(); + boolean that_present_footerSummary = true && that.isSetFooterSummary(); + if (this_present_footerSummary || that_present_footerSummary) { + if (!(this_present_footerSummary && that_present_footerSummary)) + return false; + if (!this.footerSummary.equals(that.footerSummary)) + return false; + } + + boolean this_present_startTime = true; + boolean that_present_startTime = true; + if (this_present_startTime || that_present_startTime) { + if (!(this_present_startTime && that_present_startTime)) + return false; + if (this.startTime != that.startTime) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_headerNames = true && (isSetHeaderNames()); + list.add(present_headerNames); + if (present_headerNames) + list.add(headerNames); + + boolean present_rows = true && (isSetRows()); + list.add(present_rows); + if (present_rows) + list.add(rows); + + boolean present_progressedPercentage = true; + list.add(present_progressedPercentage); + if (present_progressedPercentage) + list.add(progressedPercentage); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status.getValue()); + + boolean present_footerSummary = true && (isSetFooterSummary()); + list.add(present_footerSummary); + if (present_footerSummary) + list.add(footerSummary); + + boolean present_startTime = true; + list.add(present_startTime); + if (present_startTime) + list.add(startTime); + + return list.hashCode(); + } + + @Override + public int compareTo(TProgressUpdateResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetHeaderNames()).compareTo(other.isSetHeaderNames()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetHeaderNames()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.headerNames, other.headerNames); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetProgressedPercentage()).compareTo(other.isSetProgressedPercentage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetProgressedPercentage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.progressedPercentage, other.progressedPercentage); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetFooterSummary()).compareTo(other.isSetFooterSummary()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetFooterSummary()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.footerSummary, other.footerSummary); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetStartTime()).compareTo(other.isSetStartTime()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStartTime()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startTime, other.startTime); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TProgressUpdateResp("); + boolean first = true; + + sb.append("headerNames:"); + if (this.headerNames == null) { + sb.append("null"); + } else { + sb.append(this.headerNames); + } + first = false; + if (!first) sb.append(", "); + sb.append("rows:"); + if (this.rows == null) { + sb.append("null"); + } else { + sb.append(this.rows); + } + first = false; + if (!first) sb.append(", "); + sb.append("progressedPercentage:"); + sb.append(this.progressedPercentage); + first = false; + if (!first) sb.append(", "); + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + if (!first) sb.append(", "); + sb.append("footerSummary:"); + if (this.footerSummary == null) { + sb.append("null"); + } else { + sb.append(this.footerSummary); + } + first = false; + if (!first) sb.append(", "); + sb.append("startTime:"); + sb.append(this.startTime); + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetHeaderNames()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'headerNames' is unset! Struct:" + toString()); + } + + if (!isSetRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); + } + + if (!isSetProgressedPercentage()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'progressedPercentage' is unset! Struct:" + toString()); + } + + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + if (!isSetFooterSummary()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'footerSummary' is unset! Struct:" + toString()); + } + + if (!isSetStartTime()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'startTime' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TProgressUpdateRespStandardSchemeFactory implements SchemeFactory { + public TProgressUpdateRespStandardScheme getScheme() { + return new TProgressUpdateRespStandardScheme(); + } + } + + private static class TProgressUpdateRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // HEADER_NAMES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list180 = iprot.readListBegin(); + struct.headerNames = new ArrayList(_list180.size); + String _elem181; + for (int _i182 = 0; _i182 < _list180.size; ++_i182) + { + _elem181 = iprot.readString(); + struct.headerNames.add(_elem181); + } + iprot.readListEnd(); + } + struct.setHeaderNamesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list183 = iprot.readListBegin(); + struct.rows = new ArrayList>(_list183.size); + List _elem184; + for (int _i185 = 0; _i185 < _list183.size; ++_i185) + { + { + org.apache.thrift.protocol.TList _list186 = iprot.readListBegin(); + _elem184 = new ArrayList(_list186.size); + String _elem187; + for (int _i188 = 0; _i188 < _list186.size; ++_i188) + { + _elem187 = iprot.readString(); + _elem184.add(_elem187); + } + iprot.readListEnd(); + } + struct.rows.add(_elem184); + } + iprot.readListEnd(); + } + struct.setRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // PROGRESSED_PERCENTAGE + if (schemeField.type == org.apache.thrift.protocol.TType.DOUBLE) { + struct.progressedPercentage = iprot.readDouble(); + struct.setProgressedPercentageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.status = org.apache.hive.service.rpc.thrift.TJobExecutionStatus.findByValue(iprot.readI32()); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // FOOTER_SUMMARY + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.footerSummary = iprot.readString(); + struct.setFooterSummaryIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 6: // START_TIME + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.startTime = iprot.readI64(); + struct.setStartTimeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.headerNames != null) { + oprot.writeFieldBegin(HEADER_NAMES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.headerNames.size())); + for (String _iter189 : struct.headerNames) + { + oprot.writeString(_iter189); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.rows != null) { + oprot.writeFieldBegin(ROWS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, struct.rows.size())); + for (List _iter190 : struct.rows) + { + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, _iter190.size())); + for (String _iter191 : _iter190) + { + oprot.writeString(_iter191); + } + oprot.writeListEnd(); + } + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(PROGRESSED_PERCENTAGE_FIELD_DESC); + oprot.writeDouble(struct.progressedPercentage); + oprot.writeFieldEnd(); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + oprot.writeI32(struct.status.getValue()); + oprot.writeFieldEnd(); + } + if (struct.footerSummary != null) { + oprot.writeFieldBegin(FOOTER_SUMMARY_FIELD_DESC); + oprot.writeString(struct.footerSummary); + oprot.writeFieldEnd(); + } + oprot.writeFieldBegin(START_TIME_FIELD_DESC); + oprot.writeI64(struct.startTime); + oprot.writeFieldEnd(); + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TProgressUpdateRespTupleSchemeFactory implements SchemeFactory { + public TProgressUpdateRespTupleScheme getScheme() { + return new TProgressUpdateRespTupleScheme(); + } + } + + private static class TProgressUpdateRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.headerNames.size()); + for (String _iter192 : struct.headerNames) + { + oprot.writeString(_iter192); + } + } + { + oprot.writeI32(struct.rows.size()); + for (List _iter193 : struct.rows) + { + { + oprot.writeI32(_iter193.size()); + for (String _iter194 : _iter193) + { + oprot.writeString(_iter194); + } + } + } + } + oprot.writeDouble(struct.progressedPercentage); + oprot.writeI32(struct.status.getValue()); + oprot.writeString(struct.footerSummary); + oprot.writeI64(struct.startTime); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TProgressUpdateResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list195 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.headerNames = new ArrayList(_list195.size); + String _elem196; + for (int _i197 = 0; _i197 < _list195.size; ++_i197) + { + _elem196 = iprot.readString(); + struct.headerNames.add(_elem196); + } + } + struct.setHeaderNamesIsSet(true); + { + org.apache.thrift.protocol.TList _list198 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.LIST, iprot.readI32()); + struct.rows = new ArrayList>(_list198.size); + List _elem199; + for (int _i200 = 0; _i200 < _list198.size; ++_i200) + { + { + org.apache.thrift.protocol.TList _list201 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + _elem199 = new ArrayList(_list201.size); + String _elem202; + for (int _i203 = 0; _i203 < _list201.size; ++_i203) + { + _elem202 = iprot.readString(); + _elem199.add(_elem202); + } + } + struct.rows.add(_elem199); + } + } + struct.setRowsIsSet(true); + struct.progressedPercentage = iprot.readDouble(); + struct.setProgressedPercentageIsSet(true); + struct.status = org.apache.hive.service.rpc.thrift.TJobExecutionStatus.findByValue(iprot.readI32()); + struct.setStatusIsSet(true); + struct.footerSummary = iprot.readString(); + struct.setFooterSummaryIsSet(true); + struct.startTime = iprot.readI64(); + struct.setStartTimeIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java new file mode 100644 index 0000000000000..18a782513c500 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TProtocolVersion.java @@ -0,0 +1,69 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TProtocolVersion implements org.apache.thrift.TEnum { + HIVE_CLI_SERVICE_PROTOCOL_V1(0), + HIVE_CLI_SERVICE_PROTOCOL_V2(1), + HIVE_CLI_SERVICE_PROTOCOL_V3(2), + HIVE_CLI_SERVICE_PROTOCOL_V4(3), + HIVE_CLI_SERVICE_PROTOCOL_V5(4), + HIVE_CLI_SERVICE_PROTOCOL_V6(5), + HIVE_CLI_SERVICE_PROTOCOL_V7(6), + HIVE_CLI_SERVICE_PROTOCOL_V8(7), + HIVE_CLI_SERVICE_PROTOCOL_V9(8), + HIVE_CLI_SERVICE_PROTOCOL_V10(9); + + private final int value; + + private TProtocolVersion(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TProtocolVersion findByValue(int value) { + switch (value) { + case 0: + return HIVE_CLI_SERVICE_PROTOCOL_V1; + case 1: + return HIVE_CLI_SERVICE_PROTOCOL_V2; + case 2: + return HIVE_CLI_SERVICE_PROTOCOL_V3; + case 3: + return HIVE_CLI_SERVICE_PROTOCOL_V4; + case 4: + return HIVE_CLI_SERVICE_PROTOCOL_V5; + case 5: + return HIVE_CLI_SERVICE_PROTOCOL_V6; + case 6: + return HIVE_CLI_SERVICE_PROTOCOL_V7; + case 7: + return HIVE_CLI_SERVICE_PROTOCOL_V8; + case 8: + return HIVE_CLI_SERVICE_PROTOCOL_V9; + case 9: + return HIVE_CLI_SERVICE_PROTOCOL_V10; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java new file mode 100644 index 0000000000000..8957ebc8d2fff --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenReq.java @@ -0,0 +1,495 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TRenewDelegationTokenReq implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenReq"); + + private static final org.apache.thrift.protocol.TField SESSION_HANDLE_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionHandle", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField DELEGATION_TOKEN_FIELD_DESC = new org.apache.thrift.protocol.TField("delegationToken", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRenewDelegationTokenReqStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRenewDelegationTokenReqTupleSchemeFactory()); + } + + private TSessionHandle sessionHandle; // required + private String delegationToken; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_HANDLE((short)1, "sessionHandle"), + DELEGATION_TOKEN((short)2, "delegationToken"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_HANDLE + return SESSION_HANDLE; + case 2: // DELEGATION_TOKEN + return DELEGATION_TOKEN; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_HANDLE, new org.apache.thrift.meta_data.FieldMetaData("sessionHandle", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TSessionHandle.class))); + tmpMap.put(_Fields.DELEGATION_TOKEN, new org.apache.thrift.meta_data.FieldMetaData("delegationToken", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenReq.class, metaDataMap); + } + + public TRenewDelegationTokenReq() { + } + + public TRenewDelegationTokenReq( + TSessionHandle sessionHandle, + String delegationToken) + { + this(); + this.sessionHandle = sessionHandle; + this.delegationToken = delegationToken; + } + + /** + * Performs a deep copy on other. + */ + public TRenewDelegationTokenReq(TRenewDelegationTokenReq other) { + if (other.isSetSessionHandle()) { + this.sessionHandle = new TSessionHandle(other.sessionHandle); + } + if (other.isSetDelegationToken()) { + this.delegationToken = other.delegationToken; + } + } + + public TRenewDelegationTokenReq deepCopy() { + return new TRenewDelegationTokenReq(this); + } + + @Override + public void clear() { + this.sessionHandle = null; + this.delegationToken = null; + } + + public TSessionHandle getSessionHandle() { + return this.sessionHandle; + } + + public void setSessionHandle(TSessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public void unsetSessionHandle() { + this.sessionHandle = null; + } + + /** Returns true if field sessionHandle is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionHandle() { + return this.sessionHandle != null; + } + + public void setSessionHandleIsSet(boolean value) { + if (!value) { + this.sessionHandle = null; + } + } + + public String getDelegationToken() { + return this.delegationToken; + } + + public void setDelegationToken(String delegationToken) { + this.delegationToken = delegationToken; + } + + public void unsetDelegationToken() { + this.delegationToken = null; + } + + /** Returns true if field delegationToken is set (has been assigned a value) and false otherwise */ + public boolean isSetDelegationToken() { + return this.delegationToken != null; + } + + public void setDelegationTokenIsSet(boolean value) { + if (!value) { + this.delegationToken = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_HANDLE: + if (value == null) { + unsetSessionHandle(); + } else { + setSessionHandle((TSessionHandle)value); + } + break; + + case DELEGATION_TOKEN: + if (value == null) { + unsetDelegationToken(); + } else { + setDelegationToken((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_HANDLE: + return getSessionHandle(); + + case DELEGATION_TOKEN: + return getDelegationToken(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_HANDLE: + return isSetSessionHandle(); + case DELEGATION_TOKEN: + return isSetDelegationToken(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRenewDelegationTokenReq) + return this.equals((TRenewDelegationTokenReq)that); + return false; + } + + public boolean equals(TRenewDelegationTokenReq that) { + if (that == null) + return false; + + boolean this_present_sessionHandle = true && this.isSetSessionHandle(); + boolean that_present_sessionHandle = true && that.isSetSessionHandle(); + if (this_present_sessionHandle || that_present_sessionHandle) { + if (!(this_present_sessionHandle && that_present_sessionHandle)) + return false; + if (!this.sessionHandle.equals(that.sessionHandle)) + return false; + } + + boolean this_present_delegationToken = true && this.isSetDelegationToken(); + boolean that_present_delegationToken = true && that.isSetDelegationToken(); + if (this_present_delegationToken || that_present_delegationToken) { + if (!(this_present_delegationToken && that_present_delegationToken)) + return false; + if (!this.delegationToken.equals(that.delegationToken)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionHandle = true && (isSetSessionHandle()); + list.add(present_sessionHandle); + if (present_sessionHandle) + list.add(sessionHandle); + + boolean present_delegationToken = true && (isSetDelegationToken()); + list.add(present_delegationToken); + if (present_delegationToken) + list.add(delegationToken); + + return list.hashCode(); + } + + @Override + public int compareTo(TRenewDelegationTokenReq other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionHandle()).compareTo(other.isSetSessionHandle()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionHandle()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionHandle, other.sessionHandle); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetDelegationToken()).compareTo(other.isSetDelegationToken()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetDelegationToken()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.delegationToken, other.delegationToken); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRenewDelegationTokenReq("); + boolean first = true; + + sb.append("sessionHandle:"); + if (this.sessionHandle == null) { + sb.append("null"); + } else { + sb.append(this.sessionHandle); + } + first = false; + if (!first) sb.append(", "); + sb.append("delegationToken:"); + if (this.delegationToken == null) { + sb.append("null"); + } else { + sb.append(this.delegationToken); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionHandle()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionHandle' is unset! Struct:" + toString()); + } + + if (!isSetDelegationToken()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'delegationToken' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionHandle != null) { + sessionHandle.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRenewDelegationTokenReqStandardSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenReqStandardScheme getScheme() { + return new TRenewDelegationTokenReqStandardScheme(); + } + } + + private static class TRenewDelegationTokenReqStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_HANDLE + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // DELEGATION_TOKEN + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionHandle != null) { + oprot.writeFieldBegin(SESSION_HANDLE_FIELD_DESC); + struct.sessionHandle.write(oprot); + oprot.writeFieldEnd(); + } + if (struct.delegationToken != null) { + oprot.writeFieldBegin(DELEGATION_TOKEN_FIELD_DESC); + oprot.writeString(struct.delegationToken); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRenewDelegationTokenReqTupleSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenReqTupleScheme getScheme() { + return new TRenewDelegationTokenReqTupleScheme(); + } + } + + private static class TRenewDelegationTokenReqTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionHandle.write(oprot); + oprot.writeString(struct.delegationToken); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenReq struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionHandle = new TSessionHandle(); + struct.sessionHandle.read(iprot); + struct.setSessionHandleIsSet(true); + struct.delegationToken = iprot.readString(); + struct.setDelegationTokenIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java new file mode 100644 index 0000000000000..6f5004ccc38e4 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRenewDelegationTokenResp.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TRenewDelegationTokenResp implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRenewDelegationTokenResp"); + + private static final org.apache.thrift.protocol.TField STATUS_FIELD_DESC = new org.apache.thrift.protocol.TField("status", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRenewDelegationTokenRespStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRenewDelegationTokenRespTupleSchemeFactory()); + } + + private TStatus status; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + STATUS((short)1, "status"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS + return STATUS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS, new org.apache.thrift.meta_data.FieldMetaData("status", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStatus.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRenewDelegationTokenResp.class, metaDataMap); + } + + public TRenewDelegationTokenResp() { + } + + public TRenewDelegationTokenResp( + TStatus status) + { + this(); + this.status = status; + } + + /** + * Performs a deep copy on other. + */ + public TRenewDelegationTokenResp(TRenewDelegationTokenResp other) { + if (other.isSetStatus()) { + this.status = new TStatus(other.status); + } + } + + public TRenewDelegationTokenResp deepCopy() { + return new TRenewDelegationTokenResp(this); + } + + @Override + public void clear() { + this.status = null; + } + + public TStatus getStatus() { + return this.status; + } + + public void setStatus(TStatus status) { + this.status = status; + } + + public void unsetStatus() { + this.status = null; + } + + /** Returns true if field status is set (has been assigned a value) and false otherwise */ + public boolean isSetStatus() { + return this.status != null; + } + + public void setStatusIsSet(boolean value) { + if (!value) { + this.status = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS: + if (value == null) { + unsetStatus(); + } else { + setStatus((TStatus)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS: + return getStatus(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS: + return isSetStatus(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRenewDelegationTokenResp) + return this.equals((TRenewDelegationTokenResp)that); + return false; + } + + public boolean equals(TRenewDelegationTokenResp that) { + if (that == null) + return false; + + boolean this_present_status = true && this.isSetStatus(); + boolean that_present_status = true && that.isSetStatus(); + if (this_present_status || that_present_status) { + if (!(this_present_status && that_present_status)) + return false; + if (!this.status.equals(that.status)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_status = true && (isSetStatus()); + list.add(present_status); + if (present_status) + list.add(status); + + return list.hashCode(); + } + + @Override + public int compareTo(TRenewDelegationTokenResp other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatus()).compareTo(other.isSetStatus()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatus()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.status, other.status); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRenewDelegationTokenResp("); + boolean first = true; + + sb.append("status:"); + if (this.status == null) { + sb.append("null"); + } else { + sb.append(this.status); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatus()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'status' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (status != null) { + status.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRenewDelegationTokenRespStandardSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenRespStandardScheme getScheme() { + return new TRenewDelegationTokenRespStandardScheme(); + } + } + + private static class TRenewDelegationTokenRespStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.status != null) { + oprot.writeFieldBegin(STATUS_FIELD_DESC); + struct.status.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRenewDelegationTokenRespTupleSchemeFactory implements SchemeFactory { + public TRenewDelegationTokenRespTupleScheme getScheme() { + return new TRenewDelegationTokenRespTupleScheme(); + } + } + + private static class TRenewDelegationTokenRespTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.status.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRenewDelegationTokenResp struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.status = new TStatus(); + struct.status.read(iprot); + struct.setStatusIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java new file mode 100644 index 0000000000000..e95299df97c3a --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRow.java @@ -0,0 +1,443 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TRow implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRow"); + + private static final org.apache.thrift.protocol.TField COL_VALS_FIELD_DESC = new org.apache.thrift.protocol.TField("colVals", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRowStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRowTupleSchemeFactory()); + } + + private List colVals; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COL_VALS((short)1, "colVals"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COL_VALS + return COL_VALS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COL_VALS, new org.apache.thrift.meta_data.FieldMetaData("colVals", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnValue.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRow.class, metaDataMap); + } + + public TRow() { + } + + public TRow( + List colVals) + { + this(); + this.colVals = colVals; + } + + /** + * Performs a deep copy on other. + */ + public TRow(TRow other) { + if (other.isSetColVals()) { + List __this__colVals = new ArrayList(other.colVals.size()); + for (TColumnValue other_element : other.colVals) { + __this__colVals.add(new TColumnValue(other_element)); + } + this.colVals = __this__colVals; + } + } + + public TRow deepCopy() { + return new TRow(this); + } + + @Override + public void clear() { + this.colVals = null; + } + + public int getColValsSize() { + return (this.colVals == null) ? 0 : this.colVals.size(); + } + + public java.util.Iterator getColValsIterator() { + return (this.colVals == null) ? null : this.colVals.iterator(); + } + + public void addToColVals(TColumnValue elem) { + if (this.colVals == null) { + this.colVals = new ArrayList(); + } + this.colVals.add(elem); + } + + public List getColVals() { + return this.colVals; + } + + public void setColVals(List colVals) { + this.colVals = colVals; + } + + public void unsetColVals() { + this.colVals = null; + } + + /** Returns true if field colVals is set (has been assigned a value) and false otherwise */ + public boolean isSetColVals() { + return this.colVals != null; + } + + public void setColValsIsSet(boolean value) { + if (!value) { + this.colVals = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COL_VALS: + if (value == null) { + unsetColVals(); + } else { + setColVals((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COL_VALS: + return getColVals(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COL_VALS: + return isSetColVals(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRow) + return this.equals((TRow)that); + return false; + } + + public boolean equals(TRow that) { + if (that == null) + return false; + + boolean this_present_colVals = true && this.isSetColVals(); + boolean that_present_colVals = true && that.isSetColVals(); + if (this_present_colVals || that_present_colVals) { + if (!(this_present_colVals && that_present_colVals)) + return false; + if (!this.colVals.equals(that.colVals)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_colVals = true && (isSetColVals()); + list.add(present_colVals); + if (present_colVals) + list.add(colVals); + + return list.hashCode(); + } + + @Override + public int compareTo(TRow other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetColVals()).compareTo(other.isSetColVals()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColVals()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.colVals, other.colVals); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRow("); + boolean first = true; + + sb.append("colVals:"); + if (this.colVals == null) { + sb.append("null"); + } else { + sb.append(this.colVals); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColVals()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'colVals' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRowStandardSchemeFactory implements SchemeFactory { + public TRowStandardScheme getScheme() { + return new TRowStandardScheme(); + } + } + + private static class TRowStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRow struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COL_VALS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list46 = iprot.readListBegin(); + struct.colVals = new ArrayList(_list46.size); + TColumnValue _elem47; + for (int _i48 = 0; _i48 < _list46.size; ++_i48) + { + _elem47 = new TColumnValue(); + _elem47.read(iprot); + struct.colVals.add(_elem47); + } + iprot.readListEnd(); + } + struct.setColValsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRow struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.colVals != null) { + oprot.writeFieldBegin(COL_VALS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.colVals.size())); + for (TColumnValue _iter49 : struct.colVals) + { + _iter49.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRowTupleSchemeFactory implements SchemeFactory { + public TRowTupleScheme getScheme() { + return new TRowTupleScheme(); + } + } + + private static class TRowTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.colVals.size()); + for (TColumnValue _iter50 : struct.colVals) + { + _iter50.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRow struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list51 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.colVals = new ArrayList(_list51.size); + TColumnValue _elem52; + for (int _i53 = 0; _i53 < _list51.size; ++_i53) + { + _elem52 = new TColumnValue(); + _elem52.read(iprot); + struct.colVals.add(_elem52); + } + } + struct.setColValsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java new file mode 100644 index 0000000000000..da3d9d3ca8820 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TRowSet.java @@ -0,0 +1,920 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TRowSet implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TRowSet"); + + private static final org.apache.thrift.protocol.TField START_ROW_OFFSET_FIELD_DESC = new org.apache.thrift.protocol.TField("startRowOffset", org.apache.thrift.protocol.TType.I64, (short)1); + private static final org.apache.thrift.protocol.TField ROWS_FIELD_DESC = new org.apache.thrift.protocol.TField("rows", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)3); + private static final org.apache.thrift.protocol.TField BINARY_COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("binaryColumns", org.apache.thrift.protocol.TType.STRING, (short)4); + private static final org.apache.thrift.protocol.TField COLUMN_COUNT_FIELD_DESC = new org.apache.thrift.protocol.TField("columnCount", org.apache.thrift.protocol.TType.I32, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TRowSetStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TRowSetTupleSchemeFactory()); + } + + private long startRowOffset; // required + private List rows; // required + private List columns; // optional + private ByteBuffer binaryColumns; // optional + private int columnCount; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + START_ROW_OFFSET((short)1, "startRowOffset"), + ROWS((short)2, "rows"), + COLUMNS((short)3, "columns"), + BINARY_COLUMNS((short)4, "binaryColumns"), + COLUMN_COUNT((short)5, "columnCount"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // START_ROW_OFFSET + return START_ROW_OFFSET; + case 2: // ROWS + return ROWS; + case 3: // COLUMNS + return COLUMNS; + case 4: // BINARY_COLUMNS + return BINARY_COLUMNS; + case 5: // COLUMN_COUNT + return COLUMN_COUNT; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __STARTROWOFFSET_ISSET_ID = 0; + private static final int __COLUMNCOUNT_ISSET_ID = 1; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.COLUMNS,_Fields.BINARY_COLUMNS,_Fields.COLUMN_COUNT}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.START_ROW_OFFSET, new org.apache.thrift.meta_data.FieldMetaData("startRowOffset", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I64))); + tmpMap.put(_Fields.ROWS, new org.apache.thrift.meta_data.FieldMetaData("rows", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TRow.class)))); + tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumn.class)))); + tmpMap.put(_Fields.BINARY_COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("binaryColumns", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + tmpMap.put(_Fields.COLUMN_COUNT, new org.apache.thrift.meta_data.FieldMetaData("columnCount", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TRowSet.class, metaDataMap); + } + + public TRowSet() { + } + + public TRowSet( + long startRowOffset, + List rows) + { + this(); + this.startRowOffset = startRowOffset; + setStartRowOffsetIsSet(true); + this.rows = rows; + } + + /** + * Performs a deep copy on other. + */ + public TRowSet(TRowSet other) { + __isset_bitfield = other.__isset_bitfield; + this.startRowOffset = other.startRowOffset; + if (other.isSetRows()) { + List __this__rows = new ArrayList(other.rows.size()); + for (TRow other_element : other.rows) { + __this__rows.add(new TRow(other_element)); + } + this.rows = __this__rows; + } + if (other.isSetColumns()) { + List __this__columns = new ArrayList(other.columns.size()); + for (TColumn other_element : other.columns) { + __this__columns.add(new TColumn(other_element)); + } + this.columns = __this__columns; + } + if (other.isSetBinaryColumns()) { + this.binaryColumns = org.apache.thrift.TBaseHelper.copyBinary(other.binaryColumns); + } + this.columnCount = other.columnCount; + } + + public TRowSet deepCopy() { + return new TRowSet(this); + } + + @Override + public void clear() { + setStartRowOffsetIsSet(false); + this.startRowOffset = 0; + this.rows = null; + this.columns = null; + this.binaryColumns = null; + setColumnCountIsSet(false); + this.columnCount = 0; + } + + public long getStartRowOffset() { + return this.startRowOffset; + } + + public void setStartRowOffset(long startRowOffset) { + this.startRowOffset = startRowOffset; + setStartRowOffsetIsSet(true); + } + + public void unsetStartRowOffset() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); + } + + /** Returns true if field startRowOffset is set (has been assigned a value) and false otherwise */ + public boolean isSetStartRowOffset() { + return EncodingUtils.testBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID); + } + + public void setStartRowOffsetIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __STARTROWOFFSET_ISSET_ID, value); + } + + public int getRowsSize() { + return (this.rows == null) ? 0 : this.rows.size(); + } + + public java.util.Iterator getRowsIterator() { + return (this.rows == null) ? null : this.rows.iterator(); + } + + public void addToRows(TRow elem) { + if (this.rows == null) { + this.rows = new ArrayList(); + } + this.rows.add(elem); + } + + public List getRows() { + return this.rows; + } + + public void setRows(List rows) { + this.rows = rows; + } + + public void unsetRows() { + this.rows = null; + } + + /** Returns true if field rows is set (has been assigned a value) and false otherwise */ + public boolean isSetRows() { + return this.rows != null; + } + + public void setRowsIsSet(boolean value) { + if (!value) { + this.rows = null; + } + } + + public int getColumnsSize() { + return (this.columns == null) ? 0 : this.columns.size(); + } + + public java.util.Iterator getColumnsIterator() { + return (this.columns == null) ? null : this.columns.iterator(); + } + + public void addToColumns(TColumn elem) { + if (this.columns == null) { + this.columns = new ArrayList(); + } + this.columns.add(elem); + } + + public List getColumns() { + return this.columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + + public void unsetColumns() { + this.columns = null; + } + + /** Returns true if field columns is set (has been assigned a value) and false otherwise */ + public boolean isSetColumns() { + return this.columns != null; + } + + public void setColumnsIsSet(boolean value) { + if (!value) { + this.columns = null; + } + } + + public byte[] getBinaryColumns() { + setBinaryColumns(org.apache.thrift.TBaseHelper.rightSize(binaryColumns)); + return binaryColumns == null ? null : binaryColumns.array(); + } + + public ByteBuffer bufferForBinaryColumns() { + return org.apache.thrift.TBaseHelper.copyBinary(binaryColumns); + } + + public void setBinaryColumns(byte[] binaryColumns) { + this.binaryColumns = binaryColumns == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(binaryColumns, binaryColumns.length)); + } + + public void setBinaryColumns(ByteBuffer binaryColumns) { + this.binaryColumns = org.apache.thrift.TBaseHelper.copyBinary(binaryColumns); + } + + public void unsetBinaryColumns() { + this.binaryColumns = null; + } + + /** Returns true if field binaryColumns is set (has been assigned a value) and false otherwise */ + public boolean isSetBinaryColumns() { + return this.binaryColumns != null; + } + + public void setBinaryColumnsIsSet(boolean value) { + if (!value) { + this.binaryColumns = null; + } + } + + public int getColumnCount() { + return this.columnCount; + } + + public void setColumnCount(int columnCount) { + this.columnCount = columnCount; + setColumnCountIsSet(true); + } + + public void unsetColumnCount() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __COLUMNCOUNT_ISSET_ID); + } + + /** Returns true if field columnCount is set (has been assigned a value) and false otherwise */ + public boolean isSetColumnCount() { + return EncodingUtils.testBit(__isset_bitfield, __COLUMNCOUNT_ISSET_ID); + } + + public void setColumnCountIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __COLUMNCOUNT_ISSET_ID, value); + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case START_ROW_OFFSET: + if (value == null) { + unsetStartRowOffset(); + } else { + setStartRowOffset((Long)value); + } + break; + + case ROWS: + if (value == null) { + unsetRows(); + } else { + setRows((List)value); + } + break; + + case COLUMNS: + if (value == null) { + unsetColumns(); + } else { + setColumns((List)value); + } + break; + + case BINARY_COLUMNS: + if (value == null) { + unsetBinaryColumns(); + } else { + setBinaryColumns((ByteBuffer)value); + } + break; + + case COLUMN_COUNT: + if (value == null) { + unsetColumnCount(); + } else { + setColumnCount((Integer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case START_ROW_OFFSET: + return getStartRowOffset(); + + case ROWS: + return getRows(); + + case COLUMNS: + return getColumns(); + + case BINARY_COLUMNS: + return getBinaryColumns(); + + case COLUMN_COUNT: + return getColumnCount(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case START_ROW_OFFSET: + return isSetStartRowOffset(); + case ROWS: + return isSetRows(); + case COLUMNS: + return isSetColumns(); + case BINARY_COLUMNS: + return isSetBinaryColumns(); + case COLUMN_COUNT: + return isSetColumnCount(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TRowSet) + return this.equals((TRowSet)that); + return false; + } + + public boolean equals(TRowSet that) { + if (that == null) + return false; + + boolean this_present_startRowOffset = true; + boolean that_present_startRowOffset = true; + if (this_present_startRowOffset || that_present_startRowOffset) { + if (!(this_present_startRowOffset && that_present_startRowOffset)) + return false; + if (this.startRowOffset != that.startRowOffset) + return false; + } + + boolean this_present_rows = true && this.isSetRows(); + boolean that_present_rows = true && that.isSetRows(); + if (this_present_rows || that_present_rows) { + if (!(this_present_rows && that_present_rows)) + return false; + if (!this.rows.equals(that.rows)) + return false; + } + + boolean this_present_columns = true && this.isSetColumns(); + boolean that_present_columns = true && that.isSetColumns(); + if (this_present_columns || that_present_columns) { + if (!(this_present_columns && that_present_columns)) + return false; + if (!this.columns.equals(that.columns)) + return false; + } + + boolean this_present_binaryColumns = true && this.isSetBinaryColumns(); + boolean that_present_binaryColumns = true && that.isSetBinaryColumns(); + if (this_present_binaryColumns || that_present_binaryColumns) { + if (!(this_present_binaryColumns && that_present_binaryColumns)) + return false; + if (!this.binaryColumns.equals(that.binaryColumns)) + return false; + } + + boolean this_present_columnCount = true && this.isSetColumnCount(); + boolean that_present_columnCount = true && that.isSetColumnCount(); + if (this_present_columnCount || that_present_columnCount) { + if (!(this_present_columnCount && that_present_columnCount)) + return false; + if (this.columnCount != that.columnCount) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_startRowOffset = true; + list.add(present_startRowOffset); + if (present_startRowOffset) + list.add(startRowOffset); + + boolean present_rows = true && (isSetRows()); + list.add(present_rows); + if (present_rows) + list.add(rows); + + boolean present_columns = true && (isSetColumns()); + list.add(present_columns); + if (present_columns) + list.add(columns); + + boolean present_binaryColumns = true && (isSetBinaryColumns()); + list.add(present_binaryColumns); + if (present_binaryColumns) + list.add(binaryColumns); + + boolean present_columnCount = true && (isSetColumnCount()); + list.add(present_columnCount); + if (present_columnCount) + list.add(columnCount); + + return list.hashCode(); + } + + @Override + public int compareTo(TRowSet other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStartRowOffset()).compareTo(other.isSetStartRowOffset()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStartRowOffset()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.startRowOffset, other.startRowOffset); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetRows()).compareTo(other.isSetRows()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetRows()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.rows, other.rows); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetBinaryColumns()).compareTo(other.isSetBinaryColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetBinaryColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.binaryColumns, other.binaryColumns); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetColumnCount()).compareTo(other.isSetColumnCount()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumnCount()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columnCount, other.columnCount); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TRowSet("); + boolean first = true; + + sb.append("startRowOffset:"); + sb.append(this.startRowOffset); + first = false; + if (!first) sb.append(", "); + sb.append("rows:"); + if (this.rows == null) { + sb.append("null"); + } else { + sb.append(this.rows); + } + first = false; + if (isSetColumns()) { + if (!first) sb.append(", "); + sb.append("columns:"); + if (this.columns == null) { + sb.append("null"); + } else { + sb.append(this.columns); + } + first = false; + } + if (isSetBinaryColumns()) { + if (!first) sb.append(", "); + sb.append("binaryColumns:"); + if (this.binaryColumns == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.binaryColumns, sb); + } + first = false; + } + if (isSetColumnCount()) { + if (!first) sb.append(", "); + sb.append("columnCount:"); + sb.append(this.columnCount); + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStartRowOffset()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'startRowOffset' is unset! Struct:" + toString()); + } + + if (!isSetRows()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'rows' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TRowSetStandardSchemeFactory implements SchemeFactory { + public TRowSetStandardScheme getScheme() { + return new TRowSetStandardScheme(); + } + } + + private static class TRowSetStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TRowSet struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // START_ROW_OFFSET + if (schemeField.type == org.apache.thrift.protocol.TType.I64) { + struct.startRowOffset = iprot.readI64(); + struct.setStartRowOffsetIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // ROWS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list118 = iprot.readListBegin(); + struct.rows = new ArrayList(_list118.size); + TRow _elem119; + for (int _i120 = 0; _i120 < _list118.size; ++_i120) + { + _elem119 = new TRow(); + _elem119.read(iprot); + struct.rows.add(_elem119); + } + iprot.readListEnd(); + } + struct.setRowsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list121 = iprot.readListBegin(); + struct.columns = new ArrayList(_list121.size); + TColumn _elem122; + for (int _i123 = 0; _i123 < _list121.size; ++_i123) + { + _elem122 = new TColumn(); + _elem122.read(iprot); + struct.columns.add(_elem122); + } + iprot.readListEnd(); + } + struct.setColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // BINARY_COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.binaryColumns = iprot.readBinary(); + struct.setBinaryColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // COLUMN_COUNT + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.columnCount = iprot.readI32(); + struct.setColumnCountIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TRowSet struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + oprot.writeFieldBegin(START_ROW_OFFSET_FIELD_DESC); + oprot.writeI64(struct.startRowOffset); + oprot.writeFieldEnd(); + if (struct.rows != null) { + oprot.writeFieldBegin(ROWS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.rows.size())); + for (TRow _iter124 : struct.rows) + { + _iter124.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.columns != null) { + if (struct.isSetColumns()) { + oprot.writeFieldBegin(COLUMNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); + for (TColumn _iter125 : struct.columns) + { + _iter125.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.binaryColumns != null) { + if (struct.isSetBinaryColumns()) { + oprot.writeFieldBegin(BINARY_COLUMNS_FIELD_DESC); + oprot.writeBinary(struct.binaryColumns); + oprot.writeFieldEnd(); + } + } + if (struct.isSetColumnCount()) { + oprot.writeFieldBegin(COLUMN_COUNT_FIELD_DESC); + oprot.writeI32(struct.columnCount); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TRowSetTupleSchemeFactory implements SchemeFactory { + public TRowSetTupleScheme getScheme() { + return new TRowSetTupleScheme(); + } + } + + private static class TRowSetTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI64(struct.startRowOffset); + { + oprot.writeI32(struct.rows.size()); + for (TRow _iter126 : struct.rows) + { + _iter126.write(oprot); + } + } + BitSet optionals = new BitSet(); + if (struct.isSetColumns()) { + optionals.set(0); + } + if (struct.isSetBinaryColumns()) { + optionals.set(1); + } + if (struct.isSetColumnCount()) { + optionals.set(2); + } + oprot.writeBitSet(optionals, 3); + if (struct.isSetColumns()) { + { + oprot.writeI32(struct.columns.size()); + for (TColumn _iter127 : struct.columns) + { + _iter127.write(oprot); + } + } + } + if (struct.isSetBinaryColumns()) { + oprot.writeBinary(struct.binaryColumns); + } + if (struct.isSetColumnCount()) { + oprot.writeI32(struct.columnCount); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TRowSet struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.startRowOffset = iprot.readI64(); + struct.setStartRowOffsetIsSet(true); + { + org.apache.thrift.protocol.TList _list128 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.rows = new ArrayList(_list128.size); + TRow _elem129; + for (int _i130 = 0; _i130 < _list128.size; ++_i130) + { + _elem129 = new TRow(); + _elem129.read(iprot); + struct.rows.add(_elem129); + } + } + struct.setRowsIsSet(true); + BitSet incoming = iprot.readBitSet(3); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list131 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list131.size); + TColumn _elem132; + for (int _i133 = 0; _i133 < _list131.size; ++_i133) + { + _elem132 = new TColumn(); + _elem132.read(iprot); + struct.columns.add(_elem132); + } + } + struct.setColumnsIsSet(true); + } + if (incoming.get(1)) { + struct.binaryColumns = iprot.readBinary(); + struct.setBinaryColumnsIsSet(true); + } + if (incoming.get(2)) { + struct.columnCount = iprot.readI32(); + struct.setColumnCountIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java new file mode 100644 index 0000000000000..b5cb6e7b15aa6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TSessionHandle.java @@ -0,0 +1,394 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TSessionHandle implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TSessionHandle"); + + private static final org.apache.thrift.protocol.TField SESSION_ID_FIELD_DESC = new org.apache.thrift.protocol.TField("sessionId", org.apache.thrift.protocol.TType.STRUCT, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TSessionHandleStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TSessionHandleTupleSchemeFactory()); + } + + private THandleIdentifier sessionId; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + SESSION_ID((short)1, "sessionId"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // SESSION_ID + return SESSION_ID; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.SESSION_ID, new org.apache.thrift.meta_data.FieldMetaData("sessionId", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, THandleIdentifier.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TSessionHandle.class, metaDataMap); + } + + public TSessionHandle() { + } + + public TSessionHandle( + THandleIdentifier sessionId) + { + this(); + this.sessionId = sessionId; + } + + /** + * Performs a deep copy on other. + */ + public TSessionHandle(TSessionHandle other) { + if (other.isSetSessionId()) { + this.sessionId = new THandleIdentifier(other.sessionId); + } + } + + public TSessionHandle deepCopy() { + return new TSessionHandle(this); + } + + @Override + public void clear() { + this.sessionId = null; + } + + public THandleIdentifier getSessionId() { + return this.sessionId; + } + + public void setSessionId(THandleIdentifier sessionId) { + this.sessionId = sessionId; + } + + public void unsetSessionId() { + this.sessionId = null; + } + + /** Returns true if field sessionId is set (has been assigned a value) and false otherwise */ + public boolean isSetSessionId() { + return this.sessionId != null; + } + + public void setSessionIdIsSet(boolean value) { + if (!value) { + this.sessionId = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case SESSION_ID: + if (value == null) { + unsetSessionId(); + } else { + setSessionId((THandleIdentifier)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case SESSION_ID: + return getSessionId(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case SESSION_ID: + return isSetSessionId(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TSessionHandle) + return this.equals((TSessionHandle)that); + return false; + } + + public boolean equals(TSessionHandle that) { + if (that == null) + return false; + + boolean this_present_sessionId = true && this.isSetSessionId(); + boolean that_present_sessionId = true && that.isSetSessionId(); + if (this_present_sessionId || that_present_sessionId) { + if (!(this_present_sessionId && that_present_sessionId)) + return false; + if (!this.sessionId.equals(that.sessionId)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_sessionId = true && (isSetSessionId()); + list.add(present_sessionId); + if (present_sessionId) + list.add(sessionId); + + return list.hashCode(); + } + + @Override + public int compareTo(TSessionHandle other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetSessionId()).compareTo(other.isSetSessionId()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSessionId()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sessionId, other.sessionId); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TSessionHandle("); + boolean first = true; + + sb.append("sessionId:"); + if (this.sessionId == null) { + sb.append("null"); + } else { + sb.append(this.sessionId); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetSessionId()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'sessionId' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + if (sessionId != null) { + sessionId.validate(); + } + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TSessionHandleStandardSchemeFactory implements SchemeFactory { + public TSessionHandleStandardScheme getScheme() { + return new TSessionHandleStandardScheme(); + } + } + + private static class TSessionHandleStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TSessionHandle struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // SESSION_ID + if (schemeField.type == org.apache.thrift.protocol.TType.STRUCT) { + struct.sessionId = new THandleIdentifier(); + struct.sessionId.read(iprot); + struct.setSessionIdIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TSessionHandle struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.sessionId != null) { + oprot.writeFieldBegin(SESSION_ID_FIELD_DESC); + struct.sessionId.write(oprot); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TSessionHandleTupleSchemeFactory implements SchemeFactory { + public TSessionHandleTupleScheme getScheme() { + return new TSessionHandleTupleScheme(); + } + } + + private static class TSessionHandleTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + struct.sessionId.write(oprot); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TSessionHandle struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.sessionId = new THandleIdentifier(); + struct.sessionId.read(iprot); + struct.setSessionIdIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java new file mode 100644 index 0000000000000..50f4531b0a209 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatus.java @@ -0,0 +1,875 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TStatus implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStatus"); + + private static final org.apache.thrift.protocol.TField STATUS_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("statusCode", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField INFO_MESSAGES_FIELD_DESC = new org.apache.thrift.protocol.TField("infoMessages", org.apache.thrift.protocol.TType.LIST, (short)2); + private static final org.apache.thrift.protocol.TField SQL_STATE_FIELD_DESC = new org.apache.thrift.protocol.TField("sqlState", org.apache.thrift.protocol.TType.STRING, (short)3); + private static final org.apache.thrift.protocol.TField ERROR_CODE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorCode", org.apache.thrift.protocol.TType.I32, (short)4); + private static final org.apache.thrift.protocol.TField ERROR_MESSAGE_FIELD_DESC = new org.apache.thrift.protocol.TField("errorMessage", org.apache.thrift.protocol.TType.STRING, (short)5); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStatusStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStatusTupleSchemeFactory()); + } + + private TStatusCode statusCode; // required + private List infoMessages; // optional + private String sqlState; // optional + private int errorCode; // optional + private String errorMessage; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + /** + * + * @see TStatusCode + */ + STATUS_CODE((short)1, "statusCode"), + INFO_MESSAGES((short)2, "infoMessages"), + SQL_STATE((short)3, "sqlState"), + ERROR_CODE((short)4, "errorCode"), + ERROR_MESSAGE((short)5, "errorMessage"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // STATUS_CODE + return STATUS_CODE; + case 2: // INFO_MESSAGES + return INFO_MESSAGES; + case 3: // SQL_STATE + return SQL_STATE; + case 4: // ERROR_CODE + return ERROR_CODE; + case 5: // ERROR_MESSAGE + return ERROR_MESSAGE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final int __ERRORCODE_ISSET_ID = 0; + private byte __isset_bitfield = 0; + private static final _Fields optionals[] = {_Fields.INFO_MESSAGES,_Fields.SQL_STATE,_Fields.ERROR_CODE,_Fields.ERROR_MESSAGE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.STATUS_CODE, new org.apache.thrift.meta_data.FieldMetaData("statusCode", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.EnumMetaData(org.apache.thrift.protocol.TType.ENUM, TStatusCode.class))); + tmpMap.put(_Fields.INFO_MESSAGES, new org.apache.thrift.meta_data.FieldMetaData("infoMessages", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.SQL_STATE, new org.apache.thrift.meta_data.FieldMetaData("sqlState", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + tmpMap.put(_Fields.ERROR_CODE, new org.apache.thrift.meta_data.FieldMetaData("errorCode", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.ERROR_MESSAGE, new org.apache.thrift.meta_data.FieldMetaData("errorMessage", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStatus.class, metaDataMap); + } + + public TStatus() { + } + + public TStatus( + TStatusCode statusCode) + { + this(); + this.statusCode = statusCode; + } + + /** + * Performs a deep copy on other. + */ + public TStatus(TStatus other) { + __isset_bitfield = other.__isset_bitfield; + if (other.isSetStatusCode()) { + this.statusCode = other.statusCode; + } + if (other.isSetInfoMessages()) { + List __this__infoMessages = new ArrayList(other.infoMessages); + this.infoMessages = __this__infoMessages; + } + if (other.isSetSqlState()) { + this.sqlState = other.sqlState; + } + this.errorCode = other.errorCode; + if (other.isSetErrorMessage()) { + this.errorMessage = other.errorMessage; + } + } + + public TStatus deepCopy() { + return new TStatus(this); + } + + @Override + public void clear() { + this.statusCode = null; + this.infoMessages = null; + this.sqlState = null; + setErrorCodeIsSet(false); + this.errorCode = 0; + this.errorMessage = null; + } + + /** + * + * @see TStatusCode + */ + public TStatusCode getStatusCode() { + return this.statusCode; + } + + /** + * + * @see TStatusCode + */ + public void setStatusCode(TStatusCode statusCode) { + this.statusCode = statusCode; + } + + public void unsetStatusCode() { + this.statusCode = null; + } + + /** Returns true if field statusCode is set (has been assigned a value) and false otherwise */ + public boolean isSetStatusCode() { + return this.statusCode != null; + } + + public void setStatusCodeIsSet(boolean value) { + if (!value) { + this.statusCode = null; + } + } + + public int getInfoMessagesSize() { + return (this.infoMessages == null) ? 0 : this.infoMessages.size(); + } + + public java.util.Iterator getInfoMessagesIterator() { + return (this.infoMessages == null) ? null : this.infoMessages.iterator(); + } + + public void addToInfoMessages(String elem) { + if (this.infoMessages == null) { + this.infoMessages = new ArrayList(); + } + this.infoMessages.add(elem); + } + + public List getInfoMessages() { + return this.infoMessages; + } + + public void setInfoMessages(List infoMessages) { + this.infoMessages = infoMessages; + } + + public void unsetInfoMessages() { + this.infoMessages = null; + } + + /** Returns true if field infoMessages is set (has been assigned a value) and false otherwise */ + public boolean isSetInfoMessages() { + return this.infoMessages != null; + } + + public void setInfoMessagesIsSet(boolean value) { + if (!value) { + this.infoMessages = null; + } + } + + public String getSqlState() { + return this.sqlState; + } + + public void setSqlState(String sqlState) { + this.sqlState = sqlState; + } + + public void unsetSqlState() { + this.sqlState = null; + } + + /** Returns true if field sqlState is set (has been assigned a value) and false otherwise */ + public boolean isSetSqlState() { + return this.sqlState != null; + } + + public void setSqlStateIsSet(boolean value) { + if (!value) { + this.sqlState = null; + } + } + + public int getErrorCode() { + return this.errorCode; + } + + public void setErrorCode(int errorCode) { + this.errorCode = errorCode; + setErrorCodeIsSet(true); + } + + public void unsetErrorCode() { + __isset_bitfield = EncodingUtils.clearBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + /** Returns true if field errorCode is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorCode() { + return EncodingUtils.testBit(__isset_bitfield, __ERRORCODE_ISSET_ID); + } + + public void setErrorCodeIsSet(boolean value) { + __isset_bitfield = EncodingUtils.setBit(__isset_bitfield, __ERRORCODE_ISSET_ID, value); + } + + public String getErrorMessage() { + return this.errorMessage; + } + + public void setErrorMessage(String errorMessage) { + this.errorMessage = errorMessage; + } + + public void unsetErrorMessage() { + this.errorMessage = null; + } + + /** Returns true if field errorMessage is set (has been assigned a value) and false otherwise */ + public boolean isSetErrorMessage() { + return this.errorMessage != null; + } + + public void setErrorMessageIsSet(boolean value) { + if (!value) { + this.errorMessage = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case STATUS_CODE: + if (value == null) { + unsetStatusCode(); + } else { + setStatusCode((TStatusCode)value); + } + break; + + case INFO_MESSAGES: + if (value == null) { + unsetInfoMessages(); + } else { + setInfoMessages((List)value); + } + break; + + case SQL_STATE: + if (value == null) { + unsetSqlState(); + } else { + setSqlState((String)value); + } + break; + + case ERROR_CODE: + if (value == null) { + unsetErrorCode(); + } else { + setErrorCode((Integer)value); + } + break; + + case ERROR_MESSAGE: + if (value == null) { + unsetErrorMessage(); + } else { + setErrorMessage((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case STATUS_CODE: + return getStatusCode(); + + case INFO_MESSAGES: + return getInfoMessages(); + + case SQL_STATE: + return getSqlState(); + + case ERROR_CODE: + return getErrorCode(); + + case ERROR_MESSAGE: + return getErrorMessage(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case STATUS_CODE: + return isSetStatusCode(); + case INFO_MESSAGES: + return isSetInfoMessages(); + case SQL_STATE: + return isSetSqlState(); + case ERROR_CODE: + return isSetErrorCode(); + case ERROR_MESSAGE: + return isSetErrorMessage(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStatus) + return this.equals((TStatus)that); + return false; + } + + public boolean equals(TStatus that) { + if (that == null) + return false; + + boolean this_present_statusCode = true && this.isSetStatusCode(); + boolean that_present_statusCode = true && that.isSetStatusCode(); + if (this_present_statusCode || that_present_statusCode) { + if (!(this_present_statusCode && that_present_statusCode)) + return false; + if (!this.statusCode.equals(that.statusCode)) + return false; + } + + boolean this_present_infoMessages = true && this.isSetInfoMessages(); + boolean that_present_infoMessages = true && that.isSetInfoMessages(); + if (this_present_infoMessages || that_present_infoMessages) { + if (!(this_present_infoMessages && that_present_infoMessages)) + return false; + if (!this.infoMessages.equals(that.infoMessages)) + return false; + } + + boolean this_present_sqlState = true && this.isSetSqlState(); + boolean that_present_sqlState = true && that.isSetSqlState(); + if (this_present_sqlState || that_present_sqlState) { + if (!(this_present_sqlState && that_present_sqlState)) + return false; + if (!this.sqlState.equals(that.sqlState)) + return false; + } + + boolean this_present_errorCode = true && this.isSetErrorCode(); + boolean that_present_errorCode = true && that.isSetErrorCode(); + if (this_present_errorCode || that_present_errorCode) { + if (!(this_present_errorCode && that_present_errorCode)) + return false; + if (this.errorCode != that.errorCode) + return false; + } + + boolean this_present_errorMessage = true && this.isSetErrorMessage(); + boolean that_present_errorMessage = true && that.isSetErrorMessage(); + if (this_present_errorMessage || that_present_errorMessage) { + if (!(this_present_errorMessage && that_present_errorMessage)) + return false; + if (!this.errorMessage.equals(that.errorMessage)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_statusCode = true && (isSetStatusCode()); + list.add(present_statusCode); + if (present_statusCode) + list.add(statusCode.getValue()); + + boolean present_infoMessages = true && (isSetInfoMessages()); + list.add(present_infoMessages); + if (present_infoMessages) + list.add(infoMessages); + + boolean present_sqlState = true && (isSetSqlState()); + list.add(present_sqlState); + if (present_sqlState) + list.add(sqlState); + + boolean present_errorCode = true && (isSetErrorCode()); + list.add(present_errorCode); + if (present_errorCode) + list.add(errorCode); + + boolean present_errorMessage = true && (isSetErrorMessage()); + list.add(present_errorMessage); + if (present_errorMessage) + list.add(errorMessage); + + return list.hashCode(); + } + + @Override + public int compareTo(TStatus other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetStatusCode()).compareTo(other.isSetStatusCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetStatusCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.statusCode, other.statusCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetInfoMessages()).compareTo(other.isSetInfoMessages()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetInfoMessages()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.infoMessages, other.infoMessages); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetSqlState()).compareTo(other.isSetSqlState()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetSqlState()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.sqlState, other.sqlState); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorCode()).compareTo(other.isSetErrorCode()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorCode()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorCode, other.errorCode); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetErrorMessage()).compareTo(other.isSetErrorMessage()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetErrorMessage()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.errorMessage, other.errorMessage); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStatus("); + boolean first = true; + + sb.append("statusCode:"); + if (this.statusCode == null) { + sb.append("null"); + } else { + sb.append(this.statusCode); + } + first = false; + if (isSetInfoMessages()) { + if (!first) sb.append(", "); + sb.append("infoMessages:"); + if (this.infoMessages == null) { + sb.append("null"); + } else { + sb.append(this.infoMessages); + } + first = false; + } + if (isSetSqlState()) { + if (!first) sb.append(", "); + sb.append("sqlState:"); + if (this.sqlState == null) { + sb.append("null"); + } else { + sb.append(this.sqlState); + } + first = false; + } + if (isSetErrorCode()) { + if (!first) sb.append(", "); + sb.append("errorCode:"); + sb.append(this.errorCode); + first = false; + } + if (isSetErrorMessage()) { + if (!first) sb.append(", "); + sb.append("errorMessage:"); + if (this.errorMessage == null) { + sb.append("null"); + } else { + sb.append(this.errorMessage); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetStatusCode()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'statusCode' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + // it doesn't seem like you should have to do this, but java serialization is wacky, and doesn't call the default constructor. + __isset_bitfield = 0; + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStatusStandardSchemeFactory implements SchemeFactory { + public TStatusStandardScheme getScheme() { + return new TStatusStandardScheme(); + } + } + + private static class TStatusStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStatus struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // STATUS_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.statusCode = org.apache.hive.service.rpc.thrift.TStatusCode.findByValue(iprot.readI32()); + struct.setStatusCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // INFO_MESSAGES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list134 = iprot.readListBegin(); + struct.infoMessages = new ArrayList(_list134.size); + String _elem135; + for (int _i136 = 0; _i136 < _list134.size; ++_i136) + { + _elem135 = iprot.readString(); + struct.infoMessages.add(_elem135); + } + iprot.readListEnd(); + } + struct.setInfoMessagesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 3: // SQL_STATE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 4: // ERROR_CODE + if (schemeField.type == org.apache.thrift.protocol.TType.I32) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 5: // ERROR_MESSAGE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStatus struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.statusCode != null) { + oprot.writeFieldBegin(STATUS_CODE_FIELD_DESC); + oprot.writeI32(struct.statusCode.getValue()); + oprot.writeFieldEnd(); + } + if (struct.infoMessages != null) { + if (struct.isSetInfoMessages()) { + oprot.writeFieldBegin(INFO_MESSAGES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.infoMessages.size())); + for (String _iter137 : struct.infoMessages) + { + oprot.writeString(_iter137); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + } + if (struct.sqlState != null) { + if (struct.isSetSqlState()) { + oprot.writeFieldBegin(SQL_STATE_FIELD_DESC); + oprot.writeString(struct.sqlState); + oprot.writeFieldEnd(); + } + } + if (struct.isSetErrorCode()) { + oprot.writeFieldBegin(ERROR_CODE_FIELD_DESC); + oprot.writeI32(struct.errorCode); + oprot.writeFieldEnd(); + } + if (struct.errorMessage != null) { + if (struct.isSetErrorMessage()) { + oprot.writeFieldBegin(ERROR_MESSAGE_FIELD_DESC); + oprot.writeString(struct.errorMessage); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStatusTupleSchemeFactory implements SchemeFactory { + public TStatusTupleScheme getScheme() { + return new TStatusTupleScheme(); + } + } + + private static class TStatusTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeI32(struct.statusCode.getValue()); + BitSet optionals = new BitSet(); + if (struct.isSetInfoMessages()) { + optionals.set(0); + } + if (struct.isSetSqlState()) { + optionals.set(1); + } + if (struct.isSetErrorCode()) { + optionals.set(2); + } + if (struct.isSetErrorMessage()) { + optionals.set(3); + } + oprot.writeBitSet(optionals, 4); + if (struct.isSetInfoMessages()) { + { + oprot.writeI32(struct.infoMessages.size()); + for (String _iter138 : struct.infoMessages) + { + oprot.writeString(_iter138); + } + } + } + if (struct.isSetSqlState()) { + oprot.writeString(struct.sqlState); + } + if (struct.isSetErrorCode()) { + oprot.writeI32(struct.errorCode); + } + if (struct.isSetErrorMessage()) { + oprot.writeString(struct.errorMessage); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStatus struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.statusCode = org.apache.hive.service.rpc.thrift.TStatusCode.findByValue(iprot.readI32()); + struct.setStatusCodeIsSet(true); + BitSet incoming = iprot.readBitSet(4); + if (incoming.get(0)) { + { + org.apache.thrift.protocol.TList _list139 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.infoMessages = new ArrayList(_list139.size); + String _elem140; + for (int _i141 = 0; _i141 < _list139.size; ++_i141) + { + _elem140 = iprot.readString(); + struct.infoMessages.add(_elem140); + } + } + struct.setInfoMessagesIsSet(true); + } + if (incoming.get(1)) { + struct.sqlState = iprot.readString(); + struct.setSqlStateIsSet(true); + } + if (incoming.get(2)) { + struct.errorCode = iprot.readI32(); + struct.setErrorCodeIsSet(true); + } + if (incoming.get(3)) { + struct.errorMessage = iprot.readString(); + struct.setErrorMessageIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java new file mode 100644 index 0000000000000..fbf14184fa9a8 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStatusCode.java @@ -0,0 +1,54 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TStatusCode implements org.apache.thrift.TEnum { + SUCCESS_STATUS(0), + SUCCESS_WITH_INFO_STATUS(1), + STILL_EXECUTING_STATUS(2), + ERROR_STATUS(3), + INVALID_HANDLE_STATUS(4); + + private final int value; + + private TStatusCode(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TStatusCode findByValue(int value) { + switch (value) { + case 0: + return SUCCESS_STATUS; + case 1: + return SUCCESS_WITH_INFO_STATUS; + case 2: + return STILL_EXECUTING_STATUS; + case 3: + return ERROR_STATUS; + case 4: + return INVALID_HANDLE_STATUS; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java new file mode 100644 index 0000000000000..c83a1fd0de3c2 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringColumn.java @@ -0,0 +1,548 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TStringColumn implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringColumn"); + + private static final org.apache.thrift.protocol.TField VALUES_FIELD_DESC = new org.apache.thrift.protocol.TField("values", org.apache.thrift.protocol.TType.LIST, (short)1); + private static final org.apache.thrift.protocol.TField NULLS_FIELD_DESC = new org.apache.thrift.protocol.TField("nulls", org.apache.thrift.protocol.TType.STRING, (short)2); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStringColumnStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStringColumnTupleSchemeFactory()); + } + + private List values; // required + private ByteBuffer nulls; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUES((short)1, "values"), + NULLS((short)2, "nulls"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUES + return VALUES; + case 2: // NULLS + return NULLS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUES, new org.apache.thrift.meta_data.FieldMetaData("values", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING)))); + tmpMap.put(_Fields.NULLS, new org.apache.thrift.meta_data.FieldMetaData("nulls", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING , true))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringColumn.class, metaDataMap); + } + + public TStringColumn() { + } + + public TStringColumn( + List values, + ByteBuffer nulls) + { + this(); + this.values = values; + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + /** + * Performs a deep copy on other. + */ + public TStringColumn(TStringColumn other) { + if (other.isSetValues()) { + List __this__values = new ArrayList(other.values); + this.values = __this__values; + } + if (other.isSetNulls()) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(other.nulls); + } + } + + public TStringColumn deepCopy() { + return new TStringColumn(this); + } + + @Override + public void clear() { + this.values = null; + this.nulls = null; + } + + public int getValuesSize() { + return (this.values == null) ? 0 : this.values.size(); + } + + public java.util.Iterator getValuesIterator() { + return (this.values == null) ? null : this.values.iterator(); + } + + public void addToValues(String elem) { + if (this.values == null) { + this.values = new ArrayList(); + } + this.values.add(elem); + } + + public List getValues() { + return this.values; + } + + public void setValues(List values) { + this.values = values; + } + + public void unsetValues() { + this.values = null; + } + + /** Returns true if field values is set (has been assigned a value) and false otherwise */ + public boolean isSetValues() { + return this.values != null; + } + + public void setValuesIsSet(boolean value) { + if (!value) { + this.values = null; + } + } + + public byte[] getNulls() { + setNulls(org.apache.thrift.TBaseHelper.rightSize(nulls)); + return nulls == null ? null : nulls.array(); + } + + public ByteBuffer bufferForNulls() { + return org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void setNulls(byte[] nulls) { + this.nulls = nulls == null ? (ByteBuffer)null : ByteBuffer.wrap(Arrays.copyOf(nulls, nulls.length)); + } + + public void setNulls(ByteBuffer nulls) { + this.nulls = org.apache.thrift.TBaseHelper.copyBinary(nulls); + } + + public void unsetNulls() { + this.nulls = null; + } + + /** Returns true if field nulls is set (has been assigned a value) and false otherwise */ + public boolean isSetNulls() { + return this.nulls != null; + } + + public void setNullsIsSet(boolean value) { + if (!value) { + this.nulls = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUES: + if (value == null) { + unsetValues(); + } else { + setValues((List)value); + } + break; + + case NULLS: + if (value == null) { + unsetNulls(); + } else { + setNulls((ByteBuffer)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUES: + return getValues(); + + case NULLS: + return getNulls(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUES: + return isSetValues(); + case NULLS: + return isSetNulls(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStringColumn) + return this.equals((TStringColumn)that); + return false; + } + + public boolean equals(TStringColumn that) { + if (that == null) + return false; + + boolean this_present_values = true && this.isSetValues(); + boolean that_present_values = true && that.isSetValues(); + if (this_present_values || that_present_values) { + if (!(this_present_values && that_present_values)) + return false; + if (!this.values.equals(that.values)) + return false; + } + + boolean this_present_nulls = true && this.isSetNulls(); + boolean that_present_nulls = true && that.isSetNulls(); + if (this_present_nulls || that_present_nulls) { + if (!(this_present_nulls && that_present_nulls)) + return false; + if (!this.nulls.equals(that.nulls)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_values = true && (isSetValues()); + list.add(present_values); + if (present_values) + list.add(values); + + boolean present_nulls = true && (isSetNulls()); + list.add(present_nulls); + if (present_nulls) + list.add(nulls); + + return list.hashCode(); + } + + @Override + public int compareTo(TStringColumn other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValues()).compareTo(other.isSetValues()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValues()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.values, other.values); + if (lastComparison != 0) { + return lastComparison; + } + } + lastComparison = Boolean.valueOf(isSetNulls()).compareTo(other.isSetNulls()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNulls()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nulls, other.nulls); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStringColumn("); + boolean first = true; + + sb.append("values:"); + if (this.values == null) { + sb.append("null"); + } else { + sb.append(this.values); + } + first = false; + if (!first) sb.append(", "); + sb.append("nulls:"); + if (this.nulls == null) { + sb.append("null"); + } else { + org.apache.thrift.TBaseHelper.toString(this.nulls, sb); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetValues()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'values' is unset! Struct:" + toString()); + } + + if (!isSetNulls()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nulls' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStringColumnStandardSchemeFactory implements SchemeFactory { + public TStringColumnStandardScheme getScheme() { + return new TStringColumnStandardScheme(); + } + } + + private static class TStringColumnStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStringColumn struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list102 = iprot.readListBegin(); + struct.values = new ArrayList(_list102.size); + String _elem103; + for (int _i104 = 0; _i104 < _list102.size; ++_i104) + { + _elem103 = iprot.readString(); + struct.values.add(_elem103); + } + iprot.readListEnd(); + } + struct.setValuesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + case 2: // NULLS + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStringColumn struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.values != null) { + oprot.writeFieldBegin(VALUES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, struct.values.size())); + for (String _iter105 : struct.values) + { + oprot.writeString(_iter105); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + if (struct.nulls != null) { + oprot.writeFieldBegin(NULLS_FIELD_DESC); + oprot.writeBinary(struct.nulls); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStringColumnTupleSchemeFactory implements SchemeFactory { + public TStringColumnTupleScheme getScheme() { + return new TStringColumnTupleScheme(); + } + } + + private static class TStringColumnTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.values.size()); + for (String _iter106 : struct.values) + { + oprot.writeString(_iter106); + } + } + oprot.writeBinary(struct.nulls); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStringColumn struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list107 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRING, iprot.readI32()); + struct.values = new ArrayList(_list107.size); + String _elem108; + for (int _i109 = 0; _i109 < _list107.size; ++_i109) + { + _elem108 = iprot.readString(); + struct.values.add(_elem108); + } + } + struct.setValuesIsSet(true); + struct.nulls = iprot.readBinary(); + struct.setNullsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java new file mode 100644 index 0000000000000..13874e5516632 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStringValue.java @@ -0,0 +1,393 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TStringValue implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStringValue"); + + private static final org.apache.thrift.protocol.TField VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("value", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStringValueStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStringValueTupleSchemeFactory()); + } + + private String value; // optional + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + VALUE((short)1, "value"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // VALUE + return VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + private static final _Fields optionals[] = {_Fields.VALUE}; + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.VALUE, new org.apache.thrift.meta_data.FieldMetaData("value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStringValue.class, metaDataMap); + } + + public TStringValue() { + } + + /** + * Performs a deep copy on other. + */ + public TStringValue(TStringValue other) { + if (other.isSetValue()) { + this.value = other.value; + } + } + + public TStringValue deepCopy() { + return new TStringValue(this); + } + + @Override + public void clear() { + this.value = null; + } + + public String getValue() { + return this.value; + } + + public void setValue(String value) { + this.value = value; + } + + public void unsetValue() { + this.value = null; + } + + /** Returns true if field value is set (has been assigned a value) and false otherwise */ + public boolean isSetValue() { + return this.value != null; + } + + public void setValueIsSet(boolean value) { + if (!value) { + this.value = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case VALUE: + if (value == null) { + unsetValue(); + } else { + setValue((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case VALUE: + return getValue(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case VALUE: + return isSetValue(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStringValue) + return this.equals((TStringValue)that); + return false; + } + + public boolean equals(TStringValue that) { + if (that == null) + return false; + + boolean this_present_value = true && this.isSetValue(); + boolean that_present_value = true && that.isSetValue(); + if (this_present_value || that_present_value) { + if (!(this_present_value && that_present_value)) + return false; + if (!this.value.equals(that.value)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_value = true && (isSetValue()); + list.add(present_value); + if (present_value) + list.add(value); + + return list.hashCode(); + } + + @Override + public int compareTo(TStringValue other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetValue()).compareTo(other.isSetValue()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetValue()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.value, other.value); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStringValue("); + boolean first = true; + + if (isSetValue()) { + sb.append("value:"); + if (this.value == null) { + sb.append("null"); + } else { + sb.append(this.value); + } + first = false; + } + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStringValueStandardSchemeFactory implements SchemeFactory { + public TStringValueStandardScheme getScheme() { + return new TStringValueStandardScheme(); + } + } + + private static class TStringValueStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStringValue struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // VALUE + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.value = iprot.readString(); + struct.setValueIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStringValue struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.value != null) { + if (struct.isSetValue()) { + oprot.writeFieldBegin(VALUE_FIELD_DESC); + oprot.writeString(struct.value); + oprot.writeFieldEnd(); + } + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStringValueTupleSchemeFactory implements SchemeFactory { + public TStringValueTupleScheme getScheme() { + return new TStringValueTupleScheme(); + } + } + + private static class TStringValueTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + BitSet optionals = new BitSet(); + if (struct.isSetValue()) { + optionals.set(0); + } + oprot.writeBitSet(optionals, 1); + if (struct.isSetValue()) { + oprot.writeString(struct.value); + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStringValue struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + BitSet incoming = iprot.readBitSet(1); + if (incoming.get(0)) { + struct.value = iprot.readString(); + struct.setValueIsSet(true); + } + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java new file mode 100644 index 0000000000000..6c2c4f5dd2ddf --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TStructTypeEntry.java @@ -0,0 +1,452 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TStructTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TStructTypeEntry"); + + private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TStructTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TStructTypeEntryTupleSchemeFactory()); + } + + private Map nameToTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME_TO_TYPE_PTR + return NAME_TO_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TStructTypeEntry.class, metaDataMap); + } + + public TStructTypeEntry() { + } + + public TStructTypeEntry( + Map nameToTypePtr) + { + this(); + this.nameToTypePtr = nameToTypePtr; + } + + /** + * Performs a deep copy on other. + */ + public TStructTypeEntry(TStructTypeEntry other) { + if (other.isSetNameToTypePtr()) { + Map __this__nameToTypePtr = new HashMap(other.nameToTypePtr.size()); + for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { + + String other_element_key = other_element.getKey(); + Integer other_element_value = other_element.getValue(); + + String __this__nameToTypePtr_copy_key = other_element_key; + + Integer __this__nameToTypePtr_copy_value = other_element_value; + + __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); + } + this.nameToTypePtr = __this__nameToTypePtr; + } + } + + public TStructTypeEntry deepCopy() { + return new TStructTypeEntry(this); + } + + @Override + public void clear() { + this.nameToTypePtr = null; + } + + public int getNameToTypePtrSize() { + return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); + } + + public void putToNameToTypePtr(String key, int val) { + if (this.nameToTypePtr == null) { + this.nameToTypePtr = new HashMap(); + } + this.nameToTypePtr.put(key, val); + } + + public Map getNameToTypePtr() { + return this.nameToTypePtr; + } + + public void setNameToTypePtr(Map nameToTypePtr) { + this.nameToTypePtr = nameToTypePtr; + } + + public void unsetNameToTypePtr() { + this.nameToTypePtr = null; + } + + /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetNameToTypePtr() { + return this.nameToTypePtr != null; + } + + public void setNameToTypePtrIsSet(boolean value) { + if (!value) { + this.nameToTypePtr = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME_TO_TYPE_PTR: + if (value == null) { + unsetNameToTypePtr(); + } else { + setNameToTypePtr((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME_TO_TYPE_PTR: + return getNameToTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME_TO_TYPE_PTR: + return isSetNameToTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TStructTypeEntry) + return this.equals((TStructTypeEntry)that); + return false; + } + + public boolean equals(TStructTypeEntry that) { + if (that == null) + return false; + + boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); + boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); + if (this_present_nameToTypePtr || that_present_nameToTypePtr) { + if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) + return false; + if (!this.nameToTypePtr.equals(that.nameToTypePtr)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); + list.add(present_nameToTypePtr); + if (present_nameToTypePtr) + list.add(nameToTypePtr); + + return list.hashCode(); + } + + @Override + public int compareTo(TStructTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(other.isSetNameToTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNameToTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, other.nameToTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TStructTypeEntry("); + boolean first = true; + + sb.append("nameToTypePtr:"); + if (this.nameToTypePtr == null) { + sb.append("null"); + } else { + sb.append(this.nameToTypePtr); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetNameToTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TStructTypeEntryStandardSchemeFactory implements SchemeFactory { + public TStructTypeEntryStandardScheme getScheme() { + return new TStructTypeEntryStandardScheme(); + } + } + + private static class TStructTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TStructTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME_TO_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map10 = iprot.readMapBegin(); + struct.nameToTypePtr = new HashMap(2*_map10.size); + String _key11; + int _val12; + for (int _i13 = 0; _i13 < _map10.size; ++_i13) + { + _key11 = iprot.readString(); + _val12 = iprot.readI32(); + struct.nameToTypePtr.put(_key11, _val12); + } + iprot.readMapEnd(); + } + struct.setNameToTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TStructTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nameToTypePtr != null) { + oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); + for (Map.Entry _iter14 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter14.getKey()); + oprot.writeI32(_iter14.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TStructTypeEntryTupleSchemeFactory implements SchemeFactory { + public TStructTypeEntryTupleScheme getScheme() { + return new TStructTypeEntryTupleScheme(); + } + } + + private static class TStructTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nameToTypePtr.size()); + for (Map.Entry _iter15 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter15.getKey()); + oprot.writeI32(_iter15.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TStructTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map16 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nameToTypePtr = new HashMap(2*_map16.size); + String _key17; + int _val18; + for (int _i19 = 0; _i19 < _map16.size; ++_i19) + { + _key17 = iprot.readString(); + _val18 = iprot.readI32(); + struct.nameToTypePtr.put(_key17, _val18); + } + } + struct.setNameToTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java new file mode 100644 index 0000000000000..007b1603546ac --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTableSchema.java @@ -0,0 +1,443 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TTableSchema implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTableSchema"); + + private static final org.apache.thrift.protocol.TField COLUMNS_FIELD_DESC = new org.apache.thrift.protocol.TField("columns", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTableSchemaStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTableSchemaTupleSchemeFactory()); + } + + private List columns; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + COLUMNS((short)1, "columns"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // COLUMNS + return COLUMNS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.COLUMNS, new org.apache.thrift.meta_data.FieldMetaData("columns", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TColumnDesc.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTableSchema.class, metaDataMap); + } + + public TTableSchema() { + } + + public TTableSchema( + List columns) + { + this(); + this.columns = columns; + } + + /** + * Performs a deep copy on other. + */ + public TTableSchema(TTableSchema other) { + if (other.isSetColumns()) { + List __this__columns = new ArrayList(other.columns.size()); + for (TColumnDesc other_element : other.columns) { + __this__columns.add(new TColumnDesc(other_element)); + } + this.columns = __this__columns; + } + } + + public TTableSchema deepCopy() { + return new TTableSchema(this); + } + + @Override + public void clear() { + this.columns = null; + } + + public int getColumnsSize() { + return (this.columns == null) ? 0 : this.columns.size(); + } + + public java.util.Iterator getColumnsIterator() { + return (this.columns == null) ? null : this.columns.iterator(); + } + + public void addToColumns(TColumnDesc elem) { + if (this.columns == null) { + this.columns = new ArrayList(); + } + this.columns.add(elem); + } + + public List getColumns() { + return this.columns; + } + + public void setColumns(List columns) { + this.columns = columns; + } + + public void unsetColumns() { + this.columns = null; + } + + /** Returns true if field columns is set (has been assigned a value) and false otherwise */ + public boolean isSetColumns() { + return this.columns != null; + } + + public void setColumnsIsSet(boolean value) { + if (!value) { + this.columns = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case COLUMNS: + if (value == null) { + unsetColumns(); + } else { + setColumns((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case COLUMNS: + return getColumns(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case COLUMNS: + return isSetColumns(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTableSchema) + return this.equals((TTableSchema)that); + return false; + } + + public boolean equals(TTableSchema that) { + if (that == null) + return false; + + boolean this_present_columns = true && this.isSetColumns(); + boolean that_present_columns = true && that.isSetColumns(); + if (this_present_columns || that_present_columns) { + if (!(this_present_columns && that_present_columns)) + return false; + if (!this.columns.equals(that.columns)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_columns = true && (isSetColumns()); + list.add(present_columns); + if (present_columns) + list.add(columns); + + return list.hashCode(); + } + + @Override + public int compareTo(TTableSchema other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetColumns()).compareTo(other.isSetColumns()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetColumns()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.columns, other.columns); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTableSchema("); + boolean first = true; + + sb.append("columns:"); + if (this.columns == null) { + sb.append("null"); + } else { + sb.append(this.columns); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetColumns()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'columns' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTableSchemaStandardSchemeFactory implements SchemeFactory { + public TTableSchemaStandardScheme getScheme() { + return new TTableSchemaStandardScheme(); + } + } + + private static class TTableSchemaStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTableSchema struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // COLUMNS + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list38 = iprot.readListBegin(); + struct.columns = new ArrayList(_list38.size); + TColumnDesc _elem39; + for (int _i40 = 0; _i40 < _list38.size; ++_i40) + { + _elem39 = new TColumnDesc(); + _elem39.read(iprot); + struct.columns.add(_elem39); + } + iprot.readListEnd(); + } + struct.setColumnsIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTableSchema struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.columns != null) { + oprot.writeFieldBegin(COLUMNS_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.columns.size())); + for (TColumnDesc _iter41 : struct.columns) + { + _iter41.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTableSchemaTupleSchemeFactory implements SchemeFactory { + public TTableSchemaTupleScheme getScheme() { + return new TTableSchemaTupleScheme(); + } + } + + private static class TTableSchemaTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.columns.size()); + for (TColumnDesc _iter42 : struct.columns) + { + _iter42.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTableSchema struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list43 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.columns = new ArrayList(_list43.size); + TColumnDesc _elem44; + for (int _i45 = 0; _i45 < _list43.size; ++_i45) + { + _elem44 = new TColumnDesc(); + _elem44.read(iprot); + struct.columns.add(_elem44); + } + } + struct.setColumnsIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java new file mode 100644 index 0000000000000..055a14d06a2d6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeDesc.java @@ -0,0 +1,443 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TTypeDesc implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeDesc"); + + private static final org.apache.thrift.protocol.TField TYPES_FIELD_DESC = new org.apache.thrift.protocol.TField("types", org.apache.thrift.protocol.TType.LIST, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTypeDescStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTypeDescTupleSchemeFactory()); + } + + private List types; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TYPES((short)1, "types"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPES + return TYPES; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPES, new org.apache.thrift.meta_data.FieldMetaData("types", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.ListMetaData(org.apache.thrift.protocol.TType.LIST, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeEntry.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeDesc.class, metaDataMap); + } + + public TTypeDesc() { + } + + public TTypeDesc( + List types) + { + this(); + this.types = types; + } + + /** + * Performs a deep copy on other. + */ + public TTypeDesc(TTypeDesc other) { + if (other.isSetTypes()) { + List __this__types = new ArrayList(other.types.size()); + for (TTypeEntry other_element : other.types) { + __this__types.add(new TTypeEntry(other_element)); + } + this.types = __this__types; + } + } + + public TTypeDesc deepCopy() { + return new TTypeDesc(this); + } + + @Override + public void clear() { + this.types = null; + } + + public int getTypesSize() { + return (this.types == null) ? 0 : this.types.size(); + } + + public java.util.Iterator getTypesIterator() { + return (this.types == null) ? null : this.types.iterator(); + } + + public void addToTypes(TTypeEntry elem) { + if (this.types == null) { + this.types = new ArrayList(); + } + this.types.add(elem); + } + + public List getTypes() { + return this.types; + } + + public void setTypes(List types) { + this.types = types; + } + + public void unsetTypes() { + this.types = null; + } + + /** Returns true if field types is set (has been assigned a value) and false otherwise */ + public boolean isSetTypes() { + return this.types != null; + } + + public void setTypesIsSet(boolean value) { + if (!value) { + this.types = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPES: + if (value == null) { + unsetTypes(); + } else { + setTypes((List)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPES: + return getTypes(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPES: + return isSetTypes(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTypeDesc) + return this.equals((TTypeDesc)that); + return false; + } + + public boolean equals(TTypeDesc that) { + if (that == null) + return false; + + boolean this_present_types = true && this.isSetTypes(); + boolean that_present_types = true && that.isSetTypes(); + if (this_present_types || that_present_types) { + if (!(this_present_types && that_present_types)) + return false; + if (!this.types.equals(that.types)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_types = true && (isSetTypes()); + list.add(present_types); + if (present_types) + list.add(types); + + return list.hashCode(); + } + + @Override + public int compareTo(TTypeDesc other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetTypes()).compareTo(other.isSetTypes()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypes()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.types, other.types); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTypeDesc("); + boolean first = true; + + sb.append("types:"); + if (this.types == null) { + sb.append("null"); + } else { + sb.append(this.types); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTypes()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'types' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTypeDescStandardSchemeFactory implements SchemeFactory { + public TTypeDescStandardScheme getScheme() { + return new TTypeDescStandardScheme(); + } + } + + private static class TTypeDescStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeDesc struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPES + if (schemeField.type == org.apache.thrift.protocol.TType.LIST) { + { + org.apache.thrift.protocol.TList _list30 = iprot.readListBegin(); + struct.types = new ArrayList(_list30.size); + TTypeEntry _elem31; + for (int _i32 = 0; _i32 < _list30.size; ++_i32) + { + _elem31 = new TTypeEntry(); + _elem31.read(iprot); + struct.types.add(_elem31); + } + iprot.readListEnd(); + } + struct.setTypesIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeDesc struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.types != null) { + oprot.writeFieldBegin(TYPES_FIELD_DESC); + { + oprot.writeListBegin(new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, struct.types.size())); + for (TTypeEntry _iter33 : struct.types) + { + _iter33.write(oprot); + } + oprot.writeListEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTypeDescTupleSchemeFactory implements SchemeFactory { + public TTypeDescTupleScheme getScheme() { + return new TTypeDescTupleScheme(); + } + } + + private static class TTypeDescTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.types.size()); + for (TTypeEntry _iter34 : struct.types) + { + _iter34.write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTypeDesc struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TList _list35 = new org.apache.thrift.protocol.TList(org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.types = new ArrayList(_list35.size); + TTypeEntry _elem36; + for (int _i37 = 0; _i37 < _list35.size; ++_i37) + { + _elem36 = new TTypeEntry(); + _elem36.read(iprot); + struct.types.add(_elem36); + } + } + struct.setTypesIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java new file mode 100644 index 0000000000000..b609151b8fbee --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeEntry.java @@ -0,0 +1,614 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TTypeEntry extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeEntry"); + private static final org.apache.thrift.protocol.TField PRIMITIVE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("primitiveEntry", org.apache.thrift.protocol.TType.STRUCT, (short)1); + private static final org.apache.thrift.protocol.TField ARRAY_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("arrayEntry", org.apache.thrift.protocol.TType.STRUCT, (short)2); + private static final org.apache.thrift.protocol.TField MAP_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("mapEntry", org.apache.thrift.protocol.TType.STRUCT, (short)3); + private static final org.apache.thrift.protocol.TField STRUCT_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("structEntry", org.apache.thrift.protocol.TType.STRUCT, (short)4); + private static final org.apache.thrift.protocol.TField UNION_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("unionEntry", org.apache.thrift.protocol.TType.STRUCT, (short)5); + private static final org.apache.thrift.protocol.TField USER_DEFINED_TYPE_ENTRY_FIELD_DESC = new org.apache.thrift.protocol.TField("userDefinedTypeEntry", org.apache.thrift.protocol.TType.STRUCT, (short)6); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + PRIMITIVE_ENTRY((short)1, "primitiveEntry"), + ARRAY_ENTRY((short)2, "arrayEntry"), + MAP_ENTRY((short)3, "mapEntry"), + STRUCT_ENTRY((short)4, "structEntry"), + UNION_ENTRY((short)5, "unionEntry"), + USER_DEFINED_TYPE_ENTRY((short)6, "userDefinedTypeEntry"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // PRIMITIVE_ENTRY + return PRIMITIVE_ENTRY; + case 2: // ARRAY_ENTRY + return ARRAY_ENTRY; + case 3: // MAP_ENTRY + return MAP_ENTRY; + case 4: // STRUCT_ENTRY + return STRUCT_ENTRY; + case 5: // UNION_ENTRY + return UNION_ENTRY; + case 6: // USER_DEFINED_TYPE_ENTRY + return USER_DEFINED_TYPE_ENTRY; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.PRIMITIVE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("primitiveEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TPrimitiveTypeEntry.class))); + tmpMap.put(_Fields.ARRAY_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("arrayEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TArrayTypeEntry.class))); + tmpMap.put(_Fields.MAP_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("mapEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TMapTypeEntry.class))); + tmpMap.put(_Fields.STRUCT_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("structEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TStructTypeEntry.class))); + tmpMap.put(_Fields.UNION_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("unionEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUnionTypeEntry.class))); + tmpMap.put(_Fields.USER_DEFINED_TYPE_ENTRY, new org.apache.thrift.meta_data.FieldMetaData("userDefinedTypeEntry", org.apache.thrift.TFieldRequirementType.DEFAULT, + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TUserDefinedTypeEntry.class))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeEntry.class, metaDataMap); + } + + public TTypeEntry() { + super(); + } + + public TTypeEntry(TTypeEntry._Fields setField, Object value) { + super(setField, value); + } + + public TTypeEntry(TTypeEntry other) { + super(other); + } + public TTypeEntry deepCopy() { + return new TTypeEntry(this); + } + + public static TTypeEntry primitiveEntry(TPrimitiveTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setPrimitiveEntry(value); + return x; + } + + public static TTypeEntry arrayEntry(TArrayTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setArrayEntry(value); + return x; + } + + public static TTypeEntry mapEntry(TMapTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setMapEntry(value); + return x; + } + + public static TTypeEntry structEntry(TStructTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setStructEntry(value); + return x; + } + + public static TTypeEntry unionEntry(TUnionTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setUnionEntry(value); + return x; + } + + public static TTypeEntry userDefinedTypeEntry(TUserDefinedTypeEntry value) { + TTypeEntry x = new TTypeEntry(); + x.setUserDefinedTypeEntry(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case PRIMITIVE_ENTRY: + if (value instanceof TPrimitiveTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TPrimitiveTypeEntry for field 'primitiveEntry', but got " + value.getClass().getSimpleName()); + case ARRAY_ENTRY: + if (value instanceof TArrayTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TArrayTypeEntry for field 'arrayEntry', but got " + value.getClass().getSimpleName()); + case MAP_ENTRY: + if (value instanceof TMapTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TMapTypeEntry for field 'mapEntry', but got " + value.getClass().getSimpleName()); + case STRUCT_ENTRY: + if (value instanceof TStructTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TStructTypeEntry for field 'structEntry', but got " + value.getClass().getSimpleName()); + case UNION_ENTRY: + if (value instanceof TUnionTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TUnionTypeEntry for field 'unionEntry', but got " + value.getClass().getSimpleName()); + case USER_DEFINED_TYPE_ENTRY: + if (value instanceof TUserDefinedTypeEntry) { + break; + } + throw new ClassCastException("Was expecting value of type TUserDefinedTypeEntry for field 'userDefinedTypeEntry', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case PRIMITIVE_ENTRY: + if (field.type == PRIMITIVE_ENTRY_FIELD_DESC.type) { + TPrimitiveTypeEntry primitiveEntry; + primitiveEntry = new TPrimitiveTypeEntry(); + primitiveEntry.read(iprot); + return primitiveEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case ARRAY_ENTRY: + if (field.type == ARRAY_ENTRY_FIELD_DESC.type) { + TArrayTypeEntry arrayEntry; + arrayEntry = new TArrayTypeEntry(); + arrayEntry.read(iprot); + return arrayEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case MAP_ENTRY: + if (field.type == MAP_ENTRY_FIELD_DESC.type) { + TMapTypeEntry mapEntry; + mapEntry = new TMapTypeEntry(); + mapEntry.read(iprot); + return mapEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRUCT_ENTRY: + if (field.type == STRUCT_ENTRY_FIELD_DESC.type) { + TStructTypeEntry structEntry; + structEntry = new TStructTypeEntry(); + structEntry.read(iprot); + return structEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case UNION_ENTRY: + if (field.type == UNION_ENTRY_FIELD_DESC.type) { + TUnionTypeEntry unionEntry; + unionEntry = new TUnionTypeEntry(); + unionEntry.read(iprot); + return unionEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case USER_DEFINED_TYPE_ENTRY: + if (field.type == USER_DEFINED_TYPE_ENTRY_FIELD_DESC.type) { + TUserDefinedTypeEntry userDefinedTypeEntry; + userDefinedTypeEntry = new TUserDefinedTypeEntry(); + userDefinedTypeEntry.read(iprot); + return userDefinedTypeEntry; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; + primitiveEntry.write(oprot); + return; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; + arrayEntry.write(oprot); + return; + case MAP_ENTRY: + TMapTypeEntry mapEntry = (TMapTypeEntry)value_; + mapEntry.write(oprot); + return; + case STRUCT_ENTRY: + TStructTypeEntry structEntry = (TStructTypeEntry)value_; + structEntry.write(oprot); + return; + case UNION_ENTRY: + TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; + unionEntry.write(oprot); + return; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; + userDefinedTypeEntry.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry; + primitiveEntry = new TPrimitiveTypeEntry(); + primitiveEntry.read(iprot); + return primitiveEntry; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry; + arrayEntry = new TArrayTypeEntry(); + arrayEntry.read(iprot); + return arrayEntry; + case MAP_ENTRY: + TMapTypeEntry mapEntry; + mapEntry = new TMapTypeEntry(); + mapEntry.read(iprot); + return mapEntry; + case STRUCT_ENTRY: + TStructTypeEntry structEntry; + structEntry = new TStructTypeEntry(); + structEntry.read(iprot); + return structEntry; + case UNION_ENTRY: + TUnionTypeEntry unionEntry; + unionEntry = new TUnionTypeEntry(); + unionEntry.read(iprot); + return unionEntry; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry; + userDefinedTypeEntry = new TUserDefinedTypeEntry(); + userDefinedTypeEntry.read(iprot); + return userDefinedTypeEntry; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case PRIMITIVE_ENTRY: + TPrimitiveTypeEntry primitiveEntry = (TPrimitiveTypeEntry)value_; + primitiveEntry.write(oprot); + return; + case ARRAY_ENTRY: + TArrayTypeEntry arrayEntry = (TArrayTypeEntry)value_; + arrayEntry.write(oprot); + return; + case MAP_ENTRY: + TMapTypeEntry mapEntry = (TMapTypeEntry)value_; + mapEntry.write(oprot); + return; + case STRUCT_ENTRY: + TStructTypeEntry structEntry = (TStructTypeEntry)value_; + structEntry.write(oprot); + return; + case UNION_ENTRY: + TUnionTypeEntry unionEntry = (TUnionTypeEntry)value_; + unionEntry.write(oprot); + return; + case USER_DEFINED_TYPE_ENTRY: + TUserDefinedTypeEntry userDefinedTypeEntry = (TUserDefinedTypeEntry)value_; + userDefinedTypeEntry.write(oprot); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case PRIMITIVE_ENTRY: + return PRIMITIVE_ENTRY_FIELD_DESC; + case ARRAY_ENTRY: + return ARRAY_ENTRY_FIELD_DESC; + case MAP_ENTRY: + return MAP_ENTRY_FIELD_DESC; + case STRUCT_ENTRY: + return STRUCT_ENTRY_FIELD_DESC; + case UNION_ENTRY: + return UNION_ENTRY_FIELD_DESC; + case USER_DEFINED_TYPE_ENTRY: + return USER_DEFINED_TYPE_ENTRY_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public TPrimitiveTypeEntry getPrimitiveEntry() { + if (getSetField() == _Fields.PRIMITIVE_ENTRY) { + return (TPrimitiveTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'primitiveEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setPrimitiveEntry(TPrimitiveTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.PRIMITIVE_ENTRY; + value_ = value; + } + + public TArrayTypeEntry getArrayEntry() { + if (getSetField() == _Fields.ARRAY_ENTRY) { + return (TArrayTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'arrayEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setArrayEntry(TArrayTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.ARRAY_ENTRY; + value_ = value; + } + + public TMapTypeEntry getMapEntry() { + if (getSetField() == _Fields.MAP_ENTRY) { + return (TMapTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'mapEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setMapEntry(TMapTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.MAP_ENTRY; + value_ = value; + } + + public TStructTypeEntry getStructEntry() { + if (getSetField() == _Fields.STRUCT_ENTRY) { + return (TStructTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'structEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStructEntry(TStructTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRUCT_ENTRY; + value_ = value; + } + + public TUnionTypeEntry getUnionEntry() { + if (getSetField() == _Fields.UNION_ENTRY) { + return (TUnionTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'unionEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setUnionEntry(TUnionTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.UNION_ENTRY; + value_ = value; + } + + public TUserDefinedTypeEntry getUserDefinedTypeEntry() { + if (getSetField() == _Fields.USER_DEFINED_TYPE_ENTRY) { + return (TUserDefinedTypeEntry)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'userDefinedTypeEntry' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setUserDefinedTypeEntry(TUserDefinedTypeEntry value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.USER_DEFINED_TYPE_ENTRY; + value_ = value; + } + + public boolean isSetPrimitiveEntry() { + return setField_ == _Fields.PRIMITIVE_ENTRY; + } + + + public boolean isSetArrayEntry() { + return setField_ == _Fields.ARRAY_ENTRY; + } + + + public boolean isSetMapEntry() { + return setField_ == _Fields.MAP_ENTRY; + } + + + public boolean isSetStructEntry() { + return setField_ == _Fields.STRUCT_ENTRY; + } + + + public boolean isSetUnionEntry() { + return setField_ == _Fields.UNION_ENTRY; + } + + + public boolean isSetUserDefinedTypeEntry() { + return setField_ == _Fields.USER_DEFINED_TYPE_ENTRY; + } + + + public boolean equals(Object other) { + if (other instanceof TTypeEntry) { + return equals((TTypeEntry)other); + } else { + return false; + } + } + + public boolean equals(TTypeEntry other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TTypeEntry other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java new file mode 100644 index 0000000000000..a3735ebf3ec07 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeId.java @@ -0,0 +1,105 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + + +import java.util.Map; +import java.util.HashMap; +import org.apache.thrift.TEnum; + +public enum TTypeId implements org.apache.thrift.TEnum { + BOOLEAN_TYPE(0), + TINYINT_TYPE(1), + SMALLINT_TYPE(2), + INT_TYPE(3), + BIGINT_TYPE(4), + FLOAT_TYPE(5), + DOUBLE_TYPE(6), + STRING_TYPE(7), + TIMESTAMP_TYPE(8), + BINARY_TYPE(9), + ARRAY_TYPE(10), + MAP_TYPE(11), + STRUCT_TYPE(12), + UNION_TYPE(13), + USER_DEFINED_TYPE(14), + DECIMAL_TYPE(15), + NULL_TYPE(16), + DATE_TYPE(17), + VARCHAR_TYPE(18), + CHAR_TYPE(19), + INTERVAL_YEAR_MONTH_TYPE(20), + INTERVAL_DAY_TIME_TYPE(21); + + private final int value; + + private TTypeId(int value) { + this.value = value; + } + + /** + * Get the integer value of this enum value, as defined in the Thrift IDL. + */ + public int getValue() { + return value; + } + + /** + * Find a the enum type by its integer value, as defined in the Thrift IDL. + * @return null if the value is not found. + */ + public static TTypeId findByValue(int value) { + switch (value) { + case 0: + return BOOLEAN_TYPE; + case 1: + return TINYINT_TYPE; + case 2: + return SMALLINT_TYPE; + case 3: + return INT_TYPE; + case 4: + return BIGINT_TYPE; + case 5: + return FLOAT_TYPE; + case 6: + return DOUBLE_TYPE; + case 7: + return STRING_TYPE; + case 8: + return TIMESTAMP_TYPE; + case 9: + return BINARY_TYPE; + case 10: + return ARRAY_TYPE; + case 11: + return MAP_TYPE; + case 12: + return STRUCT_TYPE; + case 13: + return UNION_TYPE; + case 14: + return USER_DEFINED_TYPE; + case 15: + return DECIMAL_TYPE; + case 16: + return NULL_TYPE; + case 17: + return DATE_TYPE; + case 18: + return VARCHAR_TYPE; + case 19: + return CHAR_TYPE; + case 20: + return INTERVAL_YEAR_MONTH_TYPE; + case 21: + return INTERVAL_DAY_TIME_TYPE; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java new file mode 100644 index 0000000000000..1720c0e9a72c2 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifierValue.java @@ -0,0 +1,365 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +public class TTypeQualifierValue extends org.apache.thrift.TUnion { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifierValue"); + private static final org.apache.thrift.protocol.TField I32_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("i32Value", org.apache.thrift.protocol.TType.I32, (short)1); + private static final org.apache.thrift.protocol.TField STRING_VALUE_FIELD_DESC = new org.apache.thrift.protocol.TField("stringValue", org.apache.thrift.protocol.TType.STRING, (short)2); + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + I32_VALUE((short)1, "i32Value"), + STRING_VALUE((short)2, "stringValue"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // I32_VALUE + return I32_VALUE; + case 2: // STRING_VALUE + return STRING_VALUE; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.I32_VALUE, new org.apache.thrift.meta_data.FieldMetaData("i32Value", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32))); + tmpMap.put(_Fields.STRING_VALUE, new org.apache.thrift.meta_data.FieldMetaData("stringValue", org.apache.thrift.TFieldRequirementType.OPTIONAL, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifierValue.class, metaDataMap); + } + + public TTypeQualifierValue() { + super(); + } + + public TTypeQualifierValue(TTypeQualifierValue._Fields setField, Object value) { + super(setField, value); + } + + public TTypeQualifierValue(TTypeQualifierValue other) { + super(other); + } + public TTypeQualifierValue deepCopy() { + return new TTypeQualifierValue(this); + } + + public static TTypeQualifierValue i32Value(int value) { + TTypeQualifierValue x = new TTypeQualifierValue(); + x.setI32Value(value); + return x; + } + + public static TTypeQualifierValue stringValue(String value) { + TTypeQualifierValue x = new TTypeQualifierValue(); + x.setStringValue(value); + return x; + } + + + @Override + protected void checkType(_Fields setField, Object value) throws ClassCastException { + switch (setField) { + case I32_VALUE: + if (value instanceof Integer) { + break; + } + throw new ClassCastException("Was expecting value of type Integer for field 'i32Value', but got " + value.getClass().getSimpleName()); + case STRING_VALUE: + if (value instanceof String) { + break; + } + throw new ClassCastException("Was expecting value of type String for field 'stringValue', but got " + value.getClass().getSimpleName()); + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected Object standardSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, org.apache.thrift.protocol.TField field) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(field.id); + if (setField != null) { + switch (setField) { + case I32_VALUE: + if (field.type == I32_VALUE_FIELD_DESC.type) { + Integer i32Value; + i32Value = iprot.readI32(); + return i32Value; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + case STRING_VALUE: + if (field.type == STRING_VALUE_FIELD_DESC.type) { + String stringValue; + stringValue = iprot.readString(); + return stringValue; + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, field.type); + return null; + } + } + + @Override + protected void standardSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case I32_VALUE: + Integer i32Value = (Integer)value_; + oprot.writeI32(i32Value); + return; + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected Object tupleSchemeReadValue(org.apache.thrift.protocol.TProtocol iprot, short fieldID) throws org.apache.thrift.TException { + _Fields setField = _Fields.findByThriftId(fieldID); + if (setField != null) { + switch (setField) { + case I32_VALUE: + Integer i32Value; + i32Value = iprot.readI32(); + return i32Value; + case STRING_VALUE: + String stringValue; + stringValue = iprot.readString(); + return stringValue; + default: + throw new IllegalStateException("setField wasn't null, but didn't match any of the case statements!"); + } + } else { + throw new TProtocolException("Couldn't find a field with field id " + fieldID); + } + } + + @Override + protected void tupleSchemeWriteValue(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + switch (setField_) { + case I32_VALUE: + Integer i32Value = (Integer)value_; + oprot.writeI32(i32Value); + return; + case STRING_VALUE: + String stringValue = (String)value_; + oprot.writeString(stringValue); + return; + default: + throw new IllegalStateException("Cannot write union with unknown field " + setField_); + } + } + + @Override + protected org.apache.thrift.protocol.TField getFieldDesc(_Fields setField) { + switch (setField) { + case I32_VALUE: + return I32_VALUE_FIELD_DESC; + case STRING_VALUE: + return STRING_VALUE_FIELD_DESC; + default: + throw new IllegalArgumentException("Unknown field id " + setField); + } + } + + @Override + protected org.apache.thrift.protocol.TStruct getStructDesc() { + return STRUCT_DESC; + } + + @Override + protected _Fields enumForId(short id) { + return _Fields.findByThriftIdOrThrow(id); + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + + public int getI32Value() { + if (getSetField() == _Fields.I32_VALUE) { + return (Integer)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'i32Value' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setI32Value(int value) { + setField_ = _Fields.I32_VALUE; + value_ = value; + } + + public String getStringValue() { + if (getSetField() == _Fields.STRING_VALUE) { + return (String)getFieldValue(); + } else { + throw new RuntimeException("Cannot get field 'stringValue' because union is currently set to " + getFieldDesc(getSetField()).name); + } + } + + public void setStringValue(String value) { + if (value == null) throw new NullPointerException(); + setField_ = _Fields.STRING_VALUE; + value_ = value; + } + + public boolean isSetI32Value() { + return setField_ == _Fields.I32_VALUE; + } + + + public boolean isSetStringValue() { + return setField_ == _Fields.STRING_VALUE; + } + + + public boolean equals(Object other) { + if (other instanceof TTypeQualifierValue) { + return equals((TTypeQualifierValue)other); + } else { + return false; + } + } + + public boolean equals(TTypeQualifierValue other) { + return other != null && getSetField() == other.getSetField() && getFieldValue().equals(other.getFieldValue()); + } + + @Override + public int compareTo(TTypeQualifierValue other) { + int lastComparison = org.apache.thrift.TBaseHelper.compareTo(getSetField(), other.getSetField()); + if (lastComparison == 0) { + return org.apache.thrift.TBaseHelper.compareTo(getFieldValue(), other.getFieldValue()); + } + return lastComparison; + } + + + @Override + public int hashCode() { + List list = new ArrayList(); + list.add(this.getClass().getName()); + org.apache.thrift.TFieldIdEnum setField = getSetField(); + if (setField != null) { + list.add(setField.getThriftFieldId()); + Object value = getFieldValue(); + if (value instanceof org.apache.thrift.TEnum) { + list.add(((org.apache.thrift.TEnum)getFieldValue()).getValue()); + } else { + list.add(value); + } + } + return list.hashCode(); + } + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java new file mode 100644 index 0000000000000..f46d2ceb79caa --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TTypeQualifiers.java @@ -0,0 +1,454 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TTypeQualifiers implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TTypeQualifiers"); + + private static final org.apache.thrift.protocol.TField QUALIFIERS_FIELD_DESC = new org.apache.thrift.protocol.TField("qualifiers", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TTypeQualifiersStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TTypeQualifiersTupleSchemeFactory()); + } + + private Map qualifiers; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + QUALIFIERS((short)1, "qualifiers"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // QUALIFIERS + return QUALIFIERS; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.QUALIFIERS, new org.apache.thrift.meta_data.FieldMetaData("qualifiers", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.StructMetaData(org.apache.thrift.protocol.TType.STRUCT, TTypeQualifierValue.class)))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TTypeQualifiers.class, metaDataMap); + } + + public TTypeQualifiers() { + } + + public TTypeQualifiers( + Map qualifiers) + { + this(); + this.qualifiers = qualifiers; + } + + /** + * Performs a deep copy on other. + */ + public TTypeQualifiers(TTypeQualifiers other) { + if (other.isSetQualifiers()) { + Map __this__qualifiers = new HashMap(other.qualifiers.size()); + for (Map.Entry other_element : other.qualifiers.entrySet()) { + + String other_element_key = other_element.getKey(); + TTypeQualifierValue other_element_value = other_element.getValue(); + + String __this__qualifiers_copy_key = other_element_key; + + TTypeQualifierValue __this__qualifiers_copy_value = new TTypeQualifierValue(other_element_value); + + __this__qualifiers.put(__this__qualifiers_copy_key, __this__qualifiers_copy_value); + } + this.qualifiers = __this__qualifiers; + } + } + + public TTypeQualifiers deepCopy() { + return new TTypeQualifiers(this); + } + + @Override + public void clear() { + this.qualifiers = null; + } + + public int getQualifiersSize() { + return (this.qualifiers == null) ? 0 : this.qualifiers.size(); + } + + public void putToQualifiers(String key, TTypeQualifierValue val) { + if (this.qualifiers == null) { + this.qualifiers = new HashMap(); + } + this.qualifiers.put(key, val); + } + + public Map getQualifiers() { + return this.qualifiers; + } + + public void setQualifiers(Map qualifiers) { + this.qualifiers = qualifiers; + } + + public void unsetQualifiers() { + this.qualifiers = null; + } + + /** Returns true if field qualifiers is set (has been assigned a value) and false otherwise */ + public boolean isSetQualifiers() { + return this.qualifiers != null; + } + + public void setQualifiersIsSet(boolean value) { + if (!value) { + this.qualifiers = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case QUALIFIERS: + if (value == null) { + unsetQualifiers(); + } else { + setQualifiers((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case QUALIFIERS: + return getQualifiers(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case QUALIFIERS: + return isSetQualifiers(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TTypeQualifiers) + return this.equals((TTypeQualifiers)that); + return false; + } + + public boolean equals(TTypeQualifiers that) { + if (that == null) + return false; + + boolean this_present_qualifiers = true && this.isSetQualifiers(); + boolean that_present_qualifiers = true && that.isSetQualifiers(); + if (this_present_qualifiers || that_present_qualifiers) { + if (!(this_present_qualifiers && that_present_qualifiers)) + return false; + if (!this.qualifiers.equals(that.qualifiers)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_qualifiers = true && (isSetQualifiers()); + list.add(present_qualifiers); + if (present_qualifiers) + list.add(qualifiers); + + return list.hashCode(); + } + + @Override + public int compareTo(TTypeQualifiers other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetQualifiers()).compareTo(other.isSetQualifiers()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetQualifiers()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.qualifiers, other.qualifiers); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TTypeQualifiers("); + boolean first = true; + + sb.append("qualifiers:"); + if (this.qualifiers == null) { + sb.append("null"); + } else { + sb.append(this.qualifiers); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetQualifiers()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'qualifiers' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TTypeQualifiersStandardSchemeFactory implements SchemeFactory { + public TTypeQualifiersStandardScheme getScheme() { + return new TTypeQualifiersStandardScheme(); + } + } + + private static class TTypeQualifiersStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TTypeQualifiers struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // QUALIFIERS + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map0 = iprot.readMapBegin(); + struct.qualifiers = new HashMap(2*_map0.size); + String _key1; + TTypeQualifierValue _val2; + for (int _i3 = 0; _i3 < _map0.size; ++_i3) + { + _key1 = iprot.readString(); + _val2 = new TTypeQualifierValue(); + _val2.read(iprot); + struct.qualifiers.put(_key1, _val2); + } + iprot.readMapEnd(); + } + struct.setQualifiersIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TTypeQualifiers struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.qualifiers != null) { + oprot.writeFieldBegin(QUALIFIERS_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, struct.qualifiers.size())); + for (Map.Entry _iter4 : struct.qualifiers.entrySet()) + { + oprot.writeString(_iter4.getKey()); + _iter4.getValue().write(oprot); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TTypeQualifiersTupleSchemeFactory implements SchemeFactory { + public TTypeQualifiersTupleScheme getScheme() { + return new TTypeQualifiersTupleScheme(); + } + } + + private static class TTypeQualifiersTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.qualifiers.size()); + for (Map.Entry _iter5 : struct.qualifiers.entrySet()) + { + oprot.writeString(_iter5.getKey()); + _iter5.getValue().write(oprot); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TTypeQualifiers struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map6 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.STRUCT, iprot.readI32()); + struct.qualifiers = new HashMap(2*_map6.size); + String _key7; + TTypeQualifierValue _val8; + for (int _i9 = 0; _i9 < _map6.size; ++_i9) + { + _key7 = iprot.readString(); + _val8 = new TTypeQualifierValue(); + _val8.read(iprot); + struct.qualifiers.put(_key7, _val8); + } + } + struct.setQualifiersIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java new file mode 100644 index 0000000000000..d53f74cb8eff1 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUnionTypeEntry.java @@ -0,0 +1,452 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TUnionTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUnionTypeEntry"); + + private static final org.apache.thrift.protocol.TField NAME_TO_TYPE_PTR_FIELD_DESC = new org.apache.thrift.protocol.TField("nameToTypePtr", org.apache.thrift.protocol.TType.MAP, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TUnionTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TUnionTypeEntryTupleSchemeFactory()); + } + + private Map nameToTypePtr; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + NAME_TO_TYPE_PTR((short)1, "nameToTypePtr"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // NAME_TO_TYPE_PTR + return NAME_TO_TYPE_PTR; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.NAME_TO_TYPE_PTR, new org.apache.thrift.meta_data.FieldMetaData("nameToTypePtr", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.MapMetaData(org.apache.thrift.protocol.TType.MAP, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING), + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.I32 , "TTypeEntryPtr")))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUnionTypeEntry.class, metaDataMap); + } + + public TUnionTypeEntry() { + } + + public TUnionTypeEntry( + Map nameToTypePtr) + { + this(); + this.nameToTypePtr = nameToTypePtr; + } + + /** + * Performs a deep copy on other. + */ + public TUnionTypeEntry(TUnionTypeEntry other) { + if (other.isSetNameToTypePtr()) { + Map __this__nameToTypePtr = new HashMap(other.nameToTypePtr.size()); + for (Map.Entry other_element : other.nameToTypePtr.entrySet()) { + + String other_element_key = other_element.getKey(); + Integer other_element_value = other_element.getValue(); + + String __this__nameToTypePtr_copy_key = other_element_key; + + Integer __this__nameToTypePtr_copy_value = other_element_value; + + __this__nameToTypePtr.put(__this__nameToTypePtr_copy_key, __this__nameToTypePtr_copy_value); + } + this.nameToTypePtr = __this__nameToTypePtr; + } + } + + public TUnionTypeEntry deepCopy() { + return new TUnionTypeEntry(this); + } + + @Override + public void clear() { + this.nameToTypePtr = null; + } + + public int getNameToTypePtrSize() { + return (this.nameToTypePtr == null) ? 0 : this.nameToTypePtr.size(); + } + + public void putToNameToTypePtr(String key, int val) { + if (this.nameToTypePtr == null) { + this.nameToTypePtr = new HashMap(); + } + this.nameToTypePtr.put(key, val); + } + + public Map getNameToTypePtr() { + return this.nameToTypePtr; + } + + public void setNameToTypePtr(Map nameToTypePtr) { + this.nameToTypePtr = nameToTypePtr; + } + + public void unsetNameToTypePtr() { + this.nameToTypePtr = null; + } + + /** Returns true if field nameToTypePtr is set (has been assigned a value) and false otherwise */ + public boolean isSetNameToTypePtr() { + return this.nameToTypePtr != null; + } + + public void setNameToTypePtrIsSet(boolean value) { + if (!value) { + this.nameToTypePtr = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case NAME_TO_TYPE_PTR: + if (value == null) { + unsetNameToTypePtr(); + } else { + setNameToTypePtr((Map)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case NAME_TO_TYPE_PTR: + return getNameToTypePtr(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case NAME_TO_TYPE_PTR: + return isSetNameToTypePtr(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TUnionTypeEntry) + return this.equals((TUnionTypeEntry)that); + return false; + } + + public boolean equals(TUnionTypeEntry that) { + if (that == null) + return false; + + boolean this_present_nameToTypePtr = true && this.isSetNameToTypePtr(); + boolean that_present_nameToTypePtr = true && that.isSetNameToTypePtr(); + if (this_present_nameToTypePtr || that_present_nameToTypePtr) { + if (!(this_present_nameToTypePtr && that_present_nameToTypePtr)) + return false; + if (!this.nameToTypePtr.equals(that.nameToTypePtr)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_nameToTypePtr = true && (isSetNameToTypePtr()); + list.add(present_nameToTypePtr); + if (present_nameToTypePtr) + list.add(nameToTypePtr); + + return list.hashCode(); + } + + @Override + public int compareTo(TUnionTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetNameToTypePtr()).compareTo(other.isSetNameToTypePtr()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetNameToTypePtr()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.nameToTypePtr, other.nameToTypePtr); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TUnionTypeEntry("); + boolean first = true; + + sb.append("nameToTypePtr:"); + if (this.nameToTypePtr == null) { + sb.append("null"); + } else { + sb.append(this.nameToTypePtr); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetNameToTypePtr()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'nameToTypePtr' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TUnionTypeEntryStandardSchemeFactory implements SchemeFactory { + public TUnionTypeEntryStandardScheme getScheme() { + return new TUnionTypeEntryStandardScheme(); + } + } + + private static class TUnionTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // NAME_TO_TYPE_PTR + if (schemeField.type == org.apache.thrift.protocol.TType.MAP) { + { + org.apache.thrift.protocol.TMap _map20 = iprot.readMapBegin(); + struct.nameToTypePtr = new HashMap(2*_map20.size); + String _key21; + int _val22; + for (int _i23 = 0; _i23 < _map20.size; ++_i23) + { + _key21 = iprot.readString(); + _val22 = iprot.readI32(); + struct.nameToTypePtr.put(_key21, _val22); + } + iprot.readMapEnd(); + } + struct.setNameToTypePtrIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.nameToTypePtr != null) { + oprot.writeFieldBegin(NAME_TO_TYPE_PTR_FIELD_DESC); + { + oprot.writeMapBegin(new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, struct.nameToTypePtr.size())); + for (Map.Entry _iter24 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter24.getKey()); + oprot.writeI32(_iter24.getValue()); + } + oprot.writeMapEnd(); + } + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TUnionTypeEntryTupleSchemeFactory implements SchemeFactory { + public TUnionTypeEntryTupleScheme getScheme() { + return new TUnionTypeEntryTupleScheme(); + } + } + + private static class TUnionTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + { + oprot.writeI32(struct.nameToTypePtr.size()); + for (Map.Entry _iter25 : struct.nameToTypePtr.entrySet()) + { + oprot.writeString(_iter25.getKey()); + oprot.writeI32(_iter25.getValue()); + } + } + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TUnionTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + { + org.apache.thrift.protocol.TMap _map26 = new org.apache.thrift.protocol.TMap(org.apache.thrift.protocol.TType.STRING, org.apache.thrift.protocol.TType.I32, iprot.readI32()); + struct.nameToTypePtr = new HashMap(2*_map26.size); + String _key27; + int _val28; + for (int _i29 = 0; _i29 < _map26.size; ++_i29) + { + _key27 = iprot.readString(); + _val28 = iprot.readI32(); + struct.nameToTypePtr.put(_key27, _val28); + } + } + struct.setNameToTypePtrIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java new file mode 100644 index 0000000000000..b80c4dd5c6302 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/gen/java/org/apache/hive/service/rpc/thrift/TUserDefinedTypeEntry.java @@ -0,0 +1,389 @@ +/** + * Autogenerated by Thrift Compiler (0.9.3) + * + * DO NOT EDIT UNLESS YOU ARE SURE THAT YOU KNOW WHAT YOU ARE DOING + * @generated + */ +package org.apache.hive.service.rpc.thrift; + +import org.apache.thrift.scheme.IScheme; +import org.apache.thrift.scheme.SchemeFactory; +import org.apache.thrift.scheme.StandardScheme; + +import org.apache.thrift.scheme.TupleScheme; +import org.apache.thrift.protocol.TTupleProtocol; +import org.apache.thrift.protocol.TProtocolException; +import org.apache.thrift.EncodingUtils; +import org.apache.thrift.TException; +import org.apache.thrift.async.AsyncMethodCallback; +import org.apache.thrift.server.AbstractNonblockingServer.*; +import java.util.List; +import java.util.ArrayList; +import java.util.Map; +import java.util.HashMap; +import java.util.EnumMap; +import java.util.Set; +import java.util.HashSet; +import java.util.EnumSet; +import java.util.Collections; +import java.util.BitSet; +import java.nio.ByteBuffer; +import java.util.Arrays; +import javax.annotation.Generated; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +@SuppressWarnings({"cast", "rawtypes", "serial", "unchecked"}) +@Generated(value = "Autogenerated by Thrift Compiler (0.9.3)") +public class TUserDefinedTypeEntry implements org.apache.thrift.TBase, java.io.Serializable, Cloneable, Comparable { + private static final org.apache.thrift.protocol.TStruct STRUCT_DESC = new org.apache.thrift.protocol.TStruct("TUserDefinedTypeEntry"); + + private static final org.apache.thrift.protocol.TField TYPE_CLASS_NAME_FIELD_DESC = new org.apache.thrift.protocol.TField("typeClassName", org.apache.thrift.protocol.TType.STRING, (short)1); + + private static final Map, SchemeFactory> schemes = new HashMap, SchemeFactory>(); + static { + schemes.put(StandardScheme.class, new TUserDefinedTypeEntryStandardSchemeFactory()); + schemes.put(TupleScheme.class, new TUserDefinedTypeEntryTupleSchemeFactory()); + } + + private String typeClassName; // required + + /** The set of fields this struct contains, along with convenience methods for finding and manipulating them. */ + public enum _Fields implements org.apache.thrift.TFieldIdEnum { + TYPE_CLASS_NAME((short)1, "typeClassName"); + + private static final Map byName = new HashMap(); + + static { + for (_Fields field : EnumSet.allOf(_Fields.class)) { + byName.put(field.getFieldName(), field); + } + } + + /** + * Find the _Fields constant that matches fieldId, or null if its not found. + */ + public static _Fields findByThriftId(int fieldId) { + switch(fieldId) { + case 1: // TYPE_CLASS_NAME + return TYPE_CLASS_NAME; + default: + return null; + } + } + + /** + * Find the _Fields constant that matches fieldId, throwing an exception + * if it is not found. + */ + public static _Fields findByThriftIdOrThrow(int fieldId) { + _Fields fields = findByThriftId(fieldId); + if (fields == null) throw new IllegalArgumentException("Field " + fieldId + " doesn't exist!"); + return fields; + } + + /** + * Find the _Fields constant that matches name, or null if its not found. + */ + public static _Fields findByName(String name) { + return byName.get(name); + } + + private final short _thriftId; + private final String _fieldName; + + _Fields(short thriftId, String fieldName) { + _thriftId = thriftId; + _fieldName = fieldName; + } + + public short getThriftFieldId() { + return _thriftId; + } + + public String getFieldName() { + return _fieldName; + } + } + + // isset id assignments + public static final Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> metaDataMap; + static { + Map<_Fields, org.apache.thrift.meta_data.FieldMetaData> tmpMap = new EnumMap<_Fields, org.apache.thrift.meta_data.FieldMetaData>(_Fields.class); + tmpMap.put(_Fields.TYPE_CLASS_NAME, new org.apache.thrift.meta_data.FieldMetaData("typeClassName", org.apache.thrift.TFieldRequirementType.REQUIRED, + new org.apache.thrift.meta_data.FieldValueMetaData(org.apache.thrift.protocol.TType.STRING))); + metaDataMap = Collections.unmodifiableMap(tmpMap); + org.apache.thrift.meta_data.FieldMetaData.addStructMetaDataMap(TUserDefinedTypeEntry.class, metaDataMap); + } + + public TUserDefinedTypeEntry() { + } + + public TUserDefinedTypeEntry( + String typeClassName) + { + this(); + this.typeClassName = typeClassName; + } + + /** + * Performs a deep copy on other. + */ + public TUserDefinedTypeEntry(TUserDefinedTypeEntry other) { + if (other.isSetTypeClassName()) { + this.typeClassName = other.typeClassName; + } + } + + public TUserDefinedTypeEntry deepCopy() { + return new TUserDefinedTypeEntry(this); + } + + @Override + public void clear() { + this.typeClassName = null; + } + + public String getTypeClassName() { + return this.typeClassName; + } + + public void setTypeClassName(String typeClassName) { + this.typeClassName = typeClassName; + } + + public void unsetTypeClassName() { + this.typeClassName = null; + } + + /** Returns true if field typeClassName is set (has been assigned a value) and false otherwise */ + public boolean isSetTypeClassName() { + return this.typeClassName != null; + } + + public void setTypeClassNameIsSet(boolean value) { + if (!value) { + this.typeClassName = null; + } + } + + public void setFieldValue(_Fields field, Object value) { + switch (field) { + case TYPE_CLASS_NAME: + if (value == null) { + unsetTypeClassName(); + } else { + setTypeClassName((String)value); + } + break; + + } + } + + public Object getFieldValue(_Fields field) { + switch (field) { + case TYPE_CLASS_NAME: + return getTypeClassName(); + + } + throw new IllegalStateException(); + } + + /** Returns true if field corresponding to fieldID is set (has been assigned a value) and false otherwise */ + public boolean isSet(_Fields field) { + if (field == null) { + throw new IllegalArgumentException(); + } + + switch (field) { + case TYPE_CLASS_NAME: + return isSetTypeClassName(); + } + throw new IllegalStateException(); + } + + @Override + public boolean equals(Object that) { + if (that == null) + return false; + if (that instanceof TUserDefinedTypeEntry) + return this.equals((TUserDefinedTypeEntry)that); + return false; + } + + public boolean equals(TUserDefinedTypeEntry that) { + if (that == null) + return false; + + boolean this_present_typeClassName = true && this.isSetTypeClassName(); + boolean that_present_typeClassName = true && that.isSetTypeClassName(); + if (this_present_typeClassName || that_present_typeClassName) { + if (!(this_present_typeClassName && that_present_typeClassName)) + return false; + if (!this.typeClassName.equals(that.typeClassName)) + return false; + } + + return true; + } + + @Override + public int hashCode() { + List list = new ArrayList(); + + boolean present_typeClassName = true && (isSetTypeClassName()); + list.add(present_typeClassName); + if (present_typeClassName) + list.add(typeClassName); + + return list.hashCode(); + } + + @Override + public int compareTo(TUserDefinedTypeEntry other) { + if (!getClass().equals(other.getClass())) { + return getClass().getName().compareTo(other.getClass().getName()); + } + + int lastComparison = 0; + + lastComparison = Boolean.valueOf(isSetTypeClassName()).compareTo(other.isSetTypeClassName()); + if (lastComparison != 0) { + return lastComparison; + } + if (isSetTypeClassName()) { + lastComparison = org.apache.thrift.TBaseHelper.compareTo(this.typeClassName, other.typeClassName); + if (lastComparison != 0) { + return lastComparison; + } + } + return 0; + } + + public _Fields fieldForId(int fieldId) { + return _Fields.findByThriftId(fieldId); + } + + public void read(org.apache.thrift.protocol.TProtocol iprot) throws org.apache.thrift.TException { + schemes.get(iprot.getScheme()).getScheme().read(iprot, this); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot) throws org.apache.thrift.TException { + schemes.get(oprot.getScheme()).getScheme().write(oprot, this); + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("TUserDefinedTypeEntry("); + boolean first = true; + + sb.append("typeClassName:"); + if (this.typeClassName == null) { + sb.append("null"); + } else { + sb.append(this.typeClassName); + } + first = false; + sb.append(")"); + return sb.toString(); + } + + public void validate() throws org.apache.thrift.TException { + // check for required fields + if (!isSetTypeClassName()) { + throw new org.apache.thrift.protocol.TProtocolException("Required field 'typeClassName' is unset! Struct:" + toString()); + } + + // check for sub-struct validity + } + + private void writeObject(java.io.ObjectOutputStream out) throws java.io.IOException { + try { + write(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(out))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private void readObject(java.io.ObjectInputStream in) throws java.io.IOException, ClassNotFoundException { + try { + read(new org.apache.thrift.protocol.TCompactProtocol(new org.apache.thrift.transport.TIOStreamTransport(in))); + } catch (org.apache.thrift.TException te) { + throw new java.io.IOException(te); + } + } + + private static class TUserDefinedTypeEntryStandardSchemeFactory implements SchemeFactory { + public TUserDefinedTypeEntryStandardScheme getScheme() { + return new TUserDefinedTypeEntryStandardScheme(); + } + } + + private static class TUserDefinedTypeEntryStandardScheme extends StandardScheme { + + public void read(org.apache.thrift.protocol.TProtocol iprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + org.apache.thrift.protocol.TField schemeField; + iprot.readStructBegin(); + while (true) + { + schemeField = iprot.readFieldBegin(); + if (schemeField.type == org.apache.thrift.protocol.TType.STOP) { + break; + } + switch (schemeField.id) { + case 1: // TYPE_CLASS_NAME + if (schemeField.type == org.apache.thrift.protocol.TType.STRING) { + struct.typeClassName = iprot.readString(); + struct.setTypeClassNameIsSet(true); + } else { + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + break; + default: + org.apache.thrift.protocol.TProtocolUtil.skip(iprot, schemeField.type); + } + iprot.readFieldEnd(); + } + iprot.readStructEnd(); + struct.validate(); + } + + public void write(org.apache.thrift.protocol.TProtocol oprot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + struct.validate(); + + oprot.writeStructBegin(STRUCT_DESC); + if (struct.typeClassName != null) { + oprot.writeFieldBegin(TYPE_CLASS_NAME_FIELD_DESC); + oprot.writeString(struct.typeClassName); + oprot.writeFieldEnd(); + } + oprot.writeFieldStop(); + oprot.writeStructEnd(); + } + + } + + private static class TUserDefinedTypeEntryTupleSchemeFactory implements SchemeFactory { + public TUserDefinedTypeEntryTupleScheme getScheme() { + return new TUserDefinedTypeEntryTupleScheme(); + } + } + + private static class TUserDefinedTypeEntryTupleScheme extends TupleScheme { + + @Override + public void write(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol oprot = (TTupleProtocol) prot; + oprot.writeString(struct.typeClassName); + } + + @Override + public void read(org.apache.thrift.protocol.TProtocol prot, TUserDefinedTypeEntry struct) throws org.apache.thrift.TException { + TTupleProtocol iprot = (TTupleProtocol) prot; + struct.typeClassName = iprot.readString(); + struct.setTypeClassNameIsSet(true); + } + } + +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java new file mode 100644 index 0000000000000..ee9bc07fa9d84 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/AbstractService.java @@ -0,0 +1,184 @@ +/** + * 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.hive.service; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * AbstractService. + * + */ +public abstract class AbstractService implements Service { + + private static final Logger LOG = LoggerFactory.getLogger(AbstractService.class); + + /** + * Service state: initially {@link STATE#NOTINITED}. + */ + private Service.STATE state = STATE.NOTINITED; + + /** + * Service name. + */ + private final String name; + /** + * Service start time. Will be zero until the service is started. + */ + private long startTime; + + /** + * The configuration. Will be null until the service is initialized. + */ + private HiveConf hiveConf; + + /** + * List of state change listeners; it is final to ensure + * that it will never be null. + */ + private final List listeners = + new ArrayList(); + + /** + * Construct the service. + * + * @param name + * service name + */ + public AbstractService(String name) { + this.name = name; + } + + @Override + public synchronized Service.STATE getServiceState() { + return state; + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void init(HiveConf hiveConf) { + ensureCurrentState(STATE.NOTINITED); + this.hiveConf = hiveConf; + changeState(STATE.INITED); + LOG.info("Service:" + getName() + " is inited."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void start() { + startTime = System.currentTimeMillis(); + ensureCurrentState(STATE.INITED); + changeState(STATE.STARTED); + LOG.info("Service:" + getName() + " is started."); + } + + /** + * {@inheritDoc} + * + * @throws IllegalStateException + * if the current service state does not permit + * this action + */ + @Override + public synchronized void stop() { + if (state == STATE.STOPPED || + state == STATE.INITED || + state == STATE.NOTINITED) { + // already stopped, or else it was never + // started (eg another service failing canceled startup) + return; + } + ensureCurrentState(STATE.STARTED); + changeState(STATE.STOPPED); + LOG.info("Service:" + getName() + " is stopped."); + } + + @Override + public synchronized void register(ServiceStateChangeListener l) { + listeners.add(l); + } + + @Override + public synchronized void unregister(ServiceStateChangeListener l) { + listeners.remove(l); + } + + @Override + public String getName() { + return name; + } + + @Override + public synchronized HiveConf getHiveConf() { + return hiveConf; + } + + @Override + public long getStartTime() { + return startTime; + } + + /** + * Verify that a service is in a given state. + * + * @param currentState + * the desired state + * @throws IllegalStateException + * if the service state is different from + * the desired state + */ + private void ensureCurrentState(Service.STATE currentState) { + ServiceOperations.ensureCurrentState(state, currentState); + } + + /** + * Change to a new state and notify all listeners. + * This is a private method that is only invoked from synchronized methods, + * which avoid having to clone the listener list. It does imply that + * the state change listener methods should be short lived, as they + * will delay the state transition. + * + * @param newState + * new service state + */ + private void changeState(Service.STATE newState) { + state = newState; + // notify listeners + for (ServiceStateChangeListener l : listeners) { + l.stateChanged(this); + } + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java new file mode 100644 index 0000000000000..53b355339e35a --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CompositeService.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service; + +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * CompositeService. + * + */ +public class CompositeService extends AbstractService { + + private static final Logger LOG = LoggerFactory.getLogger(CompositeService.class); + + private final List serviceList = new ArrayList(); + + public CompositeService(String name) { + super(name); + } + + public Collection getServices() { + return Collections.unmodifiableList(serviceList); + } + + protected synchronized void addService(Service service) { + serviceList.add(service); + } + + protected synchronized boolean removeService(Service service) { + return serviceList.remove(service); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + for (Service service : serviceList) { + service.init(hiveConf); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + int i = 0; + try { + for (int n = serviceList.size(); i < n; i++) { + Service service = serviceList.get(i); + service.start(); + } + super.start(); + } catch (Throwable e) { + LOG.error("Error starting services " + getName(), e); + // Note that the state of the failed service is still INITED and not + // STARTED. Even though the last service is not started completely, still + // call stop() on all services including failed service to make sure cleanup + // happens. + stop(i); + throw new ServiceException("Failed to Start " + getName(), e); + } + + } + + @Override + public synchronized void stop() { + if (this.getServiceState() == STATE.STOPPED) { + // The base composite-service is already stopped, don't do anything again. + return; + } + if (serviceList.size() > 0) { + stop(serviceList.size() - 1); + } + super.stop(); + } + + private synchronized void stop(int numOfServicesStarted) { + // stop in reserve order of start + for (int i = numOfServicesStarted; i >= 0; i--) { + Service service = serviceList.get(i); + try { + service.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + service.getName(), t); + } + } + } + + /** + * JVM Shutdown hook for CompositeService which will stop the given + * CompositeService gracefully in case of JVM shutdown. + */ + public static class CompositeServiceShutdownHook implements Runnable { + + private final CompositeService compositeService; + + public CompositeServiceShutdownHook(CompositeService compositeService) { + this.compositeService = compositeService; + } + + @Override + public void run() { + try { + // Stop the Composite Service + compositeService.stop(); + } catch (Throwable t) { + LOG.info("Error stopping " + compositeService.getName(), t); + } + } + } + + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java new file mode 100644 index 0000000000000..9c8bd563268bc --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/CookieSigner.java @@ -0,0 +1,108 @@ +/** + * 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.hive.service; + +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; + +import org.apache.commons.codec.binary.Base64; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * The cookie signer generates a signature based on SHA digest + * and appends it to the cookie value generated at the + * server side. It uses SHA digest algorithm to sign and verify signatures. + */ +public class CookieSigner { + private static final String SIGNATURE = "&s="; + private static final String SHA_STRING = "SHA-256"; + private byte[] secretBytes; + private static final Logger LOG = LoggerFactory.getLogger(CookieSigner.class); + + /** + * Constructor + * @param secret Secret Bytes + */ + public CookieSigner(byte[] secret) { + if (secret == null) { + throw new IllegalArgumentException(" NULL Secret Bytes"); + } + this.secretBytes = secret.clone(); + } + + /** + * Sign the cookie given the string token as input. + * @param str Input token + * @return Signed token that can be used to create a cookie + */ + public String signCookie(String str) { + if (str == null || str.isEmpty()) { + throw new IllegalArgumentException("NULL or empty string to sign"); + } + String signature = getSignature(str); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + str + " is " + signature); + } + return str + SIGNATURE + signature; + } + + /** + * Verify a signed string and extracts the original string. + * @param signedStr The already signed string + * @return Raw Value of the string without the signature + */ + public String verifyAndExtract(String signedStr) { + int index = signedStr.lastIndexOf(SIGNATURE); + if (index == -1) { + throw new IllegalArgumentException("Invalid input sign: " + signedStr); + } + String originalSignature = signedStr.substring(index + SIGNATURE.length()); + String rawValue = signedStr.substring(0, index); + String currentSignature = getSignature(rawValue); + + if (LOG.isDebugEnabled()) { + LOG.debug("Signature generated for " + rawValue + " inside verify is " + currentSignature); + } + if (!originalSignature.equals(currentSignature)) { + throw new IllegalArgumentException("Invalid sign, original = " + originalSignature + + " current = " + currentSignature); + } + return rawValue; + } + + /** + * Get the signature of the input string based on SHA digest algorithm. + * @param str Input token + * @return Signed String + */ + private String getSignature(String str) { + try { + MessageDigest md = MessageDigest.getInstance(SHA_STRING); + md.update(str.getBytes()); + md.update(secretBytes); + byte[] digest = md.digest(); + return new Base64(0).encodeToString(digest); + } catch (NoSuchAlgorithmException ex) { + throw new RuntimeException("Invalid SHA digest String: " + SHA_STRING + + " " + ex.getMessage(), ex); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java new file mode 100644 index 0000000000000..2fb2e92942110 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceOperations.java @@ -0,0 +1,141 @@ +/** + * 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.hive.service; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ServiceOperations. + * + */ +public final class ServiceOperations { + private static final Logger LOG = LoggerFactory.getLogger(ServiceOperations.class); + + private ServiceOperations() { + } + + /** + * Verify that a service is in a given state. + * @param state the actual state a service is in + * @param expectedState the desired state + * @throws IllegalStateException if the service state is different from + * the desired state + */ + public static void ensureCurrentState(Service.STATE state, + Service.STATE expectedState) { + if (state != expectedState) { + throw new IllegalStateException("For this operation, the " + + "current service state must be " + + expectedState + + " instead of " + state); + } + } + + /** + * Initialize a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void init(Service service, HiveConf configuration) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.NOTINITED); + service.init(configuration); + } + + /** + * Start a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#INITED} + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + + public static void start(Service service) { + Service.STATE state = service.getServiceState(); + ensureCurrentState(state, Service.STATE.INITED); + service.start(); + } + + /** + * Initialize then start a service. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service that must be in the state + * {@link Service.STATE#NOTINITED} + * @param configuration the configuration to initialize the service with + * @throws RuntimeException on a state change failure + * @throws IllegalStateException if the service is in the wrong state + */ + public static void deploy(Service service, HiveConf configuration) { + init(service, configuration); + start(service); + } + + /** + * Stop a service. + * + * Do nothing if the service is null or not in a state in which it can be/needs to be stopped. + * + * The service state is checked before the operation begins. + * This process is not thread safe. + * @param service a service or null + */ + public static void stop(Service service) { + if (service != null) { + Service.STATE state = service.getServiceState(); + if (state == Service.STATE.STARTED) { + service.stop(); + } + } + } + + /** + * Stop a service; if it is null do nothing. Exceptions are caught and + * logged at warn level. (but not Throwables). This operation is intended to + * be used in cleanup operations + * + * @param service a service; may be null + * @return any exception that was caught; null if none was. + */ + public static Exception stopQuietly(Service service) { + try { + stop(service); + } catch (Exception e) { + LOG.warn("When stopping the service " + service.getName() + + " : " + e, + e); + return e; + } + return null; + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java new file mode 100644 index 0000000000000..608e207c456ea --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/ServiceUtils.java @@ -0,0 +1,70 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hive.service; + +import java.io.IOException; + +import org.slf4j.Logger; + +public class ServiceUtils { + + /* + * Get the index separating the user name from domain name (the user's name up + * to the first '/' or '@'). + * + * @param userName full user name. + * @return index of domain match or -1 if not found + */ + public static int indexOfDomainMatch(String userName) { + if (userName == null) { + return -1; + } + + int idx = userName.indexOf('/'); + int idx2 = userName.indexOf('@'); + int endIdx = Math.min(idx, idx2); // Use the earlier match. + // Unless at least one of '/' or '@' was not found, in + // which case, user the latter match. + if (endIdx == -1) { + endIdx = Math.max(idx, idx2); + } + return endIdx; + } + + /** + * Close the Closeable objects and ignore any {@link IOException} or + * null pointers. Must only be used for cleanup in exception handlers. + * + * @param log the log to record problems to at debug level. Can be null. + * @param closeables the objects to close + */ + public static void cleanup(Logger log, java.io.Closeable... closeables) { + for (java.io.Closeable c : closeables) { + if (c != null) { + try { + c.close(); + } catch(IOException e) { + if (log != null && log.isDebugEnabled()) { + log.debug("Exception in closing " + c, e); + } + } + } + } + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java new file mode 100644 index 0000000000000..0ef8ef28f1162 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HiveAuthFactory.java @@ -0,0 +1,441 @@ +/** + * 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.hive.service.auth; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.Method; +import java.net.InetSocketAddress; +import java.net.UnknownHostException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashMap; +import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; + +import javax.net.ssl.SSLServerSocket; +import javax.security.auth.login.LoginException; +import javax.security.sasl.Sasl; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.HiveMetaStore.HMSHandler; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.DBTokenStore; +import org.apache.hadoop.hive.thrift.HiveDelegationTokenManager; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server.ServerMode; +import org.apache.hadoop.security.SecurityUtil; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.authorize.ProxyUsers; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSSLTransportFactory; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportException; +import org.apache.thrift.transport.TTransportFactory; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class helps in some aspects of authentication. It creates the proper Thrift classes for the + * given configuration as well as helps with authenticating requests. + */ +public class HiveAuthFactory { + private static final Logger LOG = LoggerFactory.getLogger(HiveAuthFactory.class); + + + public enum AuthTypes { + NOSASL("NOSASL"), + NONE("NONE"), + LDAP("LDAP"), + KERBEROS("KERBEROS"), + CUSTOM("CUSTOM"), + PAM("PAM"); + + private final String authType; + + AuthTypes(String authType) { + this.authType = authType; + } + + public String getAuthName() { + return authType; + } + + } + + private HadoopThriftAuthBridge.Server saslServer; + private String authTypeStr; + private final String transportMode; + private final HiveConf conf; + private HiveDelegationTokenManager delegationTokenManager = null; + + public static final String HS2_PROXY_USER = "hive.server2.proxy.user"; + public static final String HS2_CLIENT_TOKEN = "hiveserver2ClientToken"; + + private static Field keytabFile = null; + private static Method getKeytab = null; + static { + Class clz = UserGroupInformation.class; + try { + keytabFile = clz.getDeclaredField("keytabFile"); + keytabFile.setAccessible(true); + } catch (NoSuchFieldException nfe) { + LOG.debug("Cannot find private field \"keytabFile\" in class: " + + UserGroupInformation.class.getCanonicalName(), nfe); + keytabFile = null; + } + + try { + getKeytab = clz.getDeclaredMethod("getKeytab"); + getKeytab.setAccessible(true); + } catch(NoSuchMethodException nme) { + LOG.debug("Cannot find private method \"getKeytab\" in class:" + + UserGroupInformation.class.getCanonicalName(), nme); + getKeytab = null; + } + } + + public HiveAuthFactory(HiveConf conf) throws TTransportException, IOException { + this.conf = conf; + transportMode = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + authTypeStr = conf.getVar(HiveConf.ConfVars.HIVE_SERVER2_AUTHENTICATION); + + // In http mode we use NOSASL as the default auth type + if ("http".equalsIgnoreCase(transportMode)) { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NOSASL.getAuthName(); + } + } else { + if (authTypeStr == null) { + authTypeStr = AuthTypes.NONE.getAuthName(); + } + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + String principal = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + String keytab = conf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + if (needUgiLogin(UserGroupInformation.getCurrentUser(), + SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keytab)) { + saslServer = ShimLoader.getHadoopThriftAuthBridge().createServer(principal, keytab); + } else { + // Using the default constructor to avoid unnecessary UGI login. + saslServer = new HadoopThriftAuthBridge.Server(); + } + + // start delegation token manager + delegationTokenManager = new HiveDelegationTokenManager(); + try { + // rawStore is only necessary for DBTokenStore + Object rawStore = null; + String tokenStoreClass = conf.getVar( + HiveConf.ConfVars.METASTORE_CLUSTER_DELEGATION_TOKEN_STORE_CLS); + + if (tokenStoreClass.equals(DBTokenStore.class.getName())) { + HMSHandler baseHandler = new HiveMetaStore.HMSHandler( + "new db based metaserver", conf, true); + rawStore = baseHandler.getMS(); + } + + delegationTokenManager.startDelegationTokenSecretManager( + conf, rawStore, ServerMode.HIVESERVER2); + saslServer.setSecretManager(delegationTokenManager.getSecretManager()); + } + catch (MetaException|IOException e) { + throw new TTransportException("Failed to start token manager", e); + } + } + } + } + + public Map getSaslProperties() { + Map saslProps = new HashMap(); + SaslQOP saslQOP = SaslQOP.fromString(conf.getVar(ConfVars.HIVE_SERVER2_THRIFT_SASL_QOP)); + saslProps.put(Sasl.QOP, saslQOP.toString()); + saslProps.put(Sasl.SERVER_AUTH, "true"); + return saslProps; + } + + public TTransportFactory getAuthTransFactory() throws LoginException { + TTransportFactory transportFactory; + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + try { + transportFactory = saslServer.createTransportFactory(getSaslProperties()); + } catch (TTransportException e) { + throw new LoginException(e.getMessage()); + } + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NONE.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.LDAP.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.PAM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.NOSASL.getAuthName())) { + transportFactory = new TTransportFactory(); + } else if (authTypeStr.equalsIgnoreCase(AuthTypes.CUSTOM.getAuthName())) { + transportFactory = PlainSaslHelper.getPlainTransportFactory(authTypeStr); + } else { + throw new LoginException("Unsupported authentication type " + authTypeStr); + } + return transportFactory; + } + + /** + * Returns the thrift processor factory for HiveServer2 running in binary mode + * @param service + * @return + * @throws LoginException + */ + public TProcessorFactory getAuthProcFactory(ThriftCLIService service) throws LoginException { + if (authTypeStr.equalsIgnoreCase(AuthTypes.KERBEROS.getAuthName())) { + return KerberosSaslHelper.getKerberosProcessorFactory(saslServer, service); + } else { + return PlainSaslHelper.getPlainProcessorFactory(service); + } + } + + public String getRemoteUser() { + return saslServer == null ? null : saslServer.getRemoteUser(); + } + + public String getIpAddress() { + if (saslServer == null || saslServer.getRemoteAddress() == null) { + return null; + } else { + return saslServer.getRemoteAddress().getHostAddress(); + } + } + + // Perform kerberos login using the hadoop shim API if the configuration is available + public static void loginFromKeytab(HiveConf hiveConf) throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_KERBEROS_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 Kerberos principal or keytab is not correctly configured"); + } else { + UserGroupInformation.loginUserFromKeytab(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + // Perform SPNEGO login using the hadoop shim API if the configuration is available + public static UserGroupInformation loginFromSpnegoKeytabAndReturnUGI(HiveConf hiveConf) + throws IOException { + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + throw new IOException("HiveServer2 SPNEGO principal or keytab is not correctly configured"); + } else { + return UserGroupInformation.loginUserFromKeytabAndReturnUGI(SecurityUtil.getServerPrincipal(principal, "0.0.0.0"), keyTabFile); + } + } + + public static TTransport getSocketTransport(String host, int port, int loginTimeout) { + return new TSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout) + throws TTransportException { + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout); + } + + public static TTransport getSSLSocket(String host, int port, int loginTimeout, + String trustStorePath, String trustStorePassWord) throws TTransportException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setTrustStore(trustStorePath, trustStorePassWord); + params.requireClientAuth(true); + return TSSLTransportFactory.getClientSocket(host, port, loginTimeout, params); + } + + public static TServerSocket getServerSocket(String hiveHost, int portNum) + throws TTransportException { + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + return new TServerSocket(serverAddress); + } + + public static TServerSocket getServerSSLSocket(String hiveHost, int portNum, String keyStorePath, + String keyStorePassWord, List sslVersionBlacklist) throws TTransportException, + UnknownHostException { + TSSLTransportFactory.TSSLTransportParameters params = + new TSSLTransportFactory.TSSLTransportParameters(); + params.setKeyStore(keyStorePath, keyStorePassWord); + InetSocketAddress serverAddress; + if (hiveHost == null || hiveHost.isEmpty()) { + // Wildcard bind + serverAddress = new InetSocketAddress(portNum); + } else { + serverAddress = new InetSocketAddress(hiveHost, portNum); + } + TServerSocket thriftServerSocket = + TSSLTransportFactory.getServerSocket(portNum, 0, serverAddress.getAddress(), params); + if (thriftServerSocket.getServerSocket() instanceof SSLServerSocket) { + List sslVersionBlacklistLocal = new ArrayList(); + for (String sslVersion : sslVersionBlacklist) { + sslVersionBlacklistLocal.add(sslVersion.trim().toLowerCase(Locale.ROOT)); + } + SSLServerSocket sslServerSocket = (SSLServerSocket) thriftServerSocket.getServerSocket(); + List enabledProtocols = new ArrayList(); + for (String protocol : sslServerSocket.getEnabledProtocols()) { + if (sslVersionBlacklistLocal.contains(protocol.toLowerCase(Locale.ROOT))) { + LOG.debug("Disabling SSL Protocol: " + protocol); + } else { + enabledProtocols.add(protocol); + } + } + sslServerSocket.setEnabledProtocols(enabledProtocols.toArray(new String[0])); + LOG.info("SSL Server Socket Enabled Protocols: " + + Arrays.toString(sslServerSocket.getEnabledProtocols())); + } + return thriftServerSocket; + } + + // retrieve delegation token for the given user + public String getDelegationToken(String owner, String renewer, String remoteAddr) + throws HiveSQLException { + if (delegationTokenManager == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + + try { + String tokenStr = delegationTokenManager.getDelegationTokenWithService(owner, renewer, + HS2_CLIENT_TOKEN, remoteAddr); + if (tokenStr == null || tokenStr.isEmpty()) { + throw new HiveSQLException( + "Received empty retrieving delegation token for user " + owner, "08S01"); + } + return tokenStr; + } catch (IOException e) { + throw new HiveSQLException( + "Error retrieving delegation token for user " + owner, "08S01", e); + } catch (InterruptedException e) { + throw new HiveSQLException("delegation token retrieval interrupted", "08S01", e); + } + } + + // cancel given delegation token + public void cancelDelegationToken(String delegationToken) throws HiveSQLException { + if (delegationTokenManager == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + delegationTokenManager.cancelDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error canceling delegation token " + delegationToken, "08S01", e); + } + } + + public void renewDelegationToken(String delegationToken) throws HiveSQLException { + if (delegationTokenManager == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + delegationTokenManager.renewDelegationToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error renewing delegation token " + delegationToken, "08S01", e); + } + } + + public String verifyDelegationToken(String delegationToken) throws HiveSQLException { + if (delegationTokenManager == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + return delegationTokenManager.verifyDelegationToken(delegationToken); + } catch (IOException e) { + String msg = "Error verifying delegation token " + delegationToken; + LOG.error(msg, e); + throw new HiveSQLException(msg, "08S01", e); + } + } + + public String getUserFromToken(String delegationToken) throws HiveSQLException { + if (delegationTokenManager == null) { + throw new HiveSQLException( + "Delegation token only supported over kerberos authentication", "08S01"); + } + try { + return delegationTokenManager.getUserFromToken(delegationToken); + } catch (IOException e) { + throw new HiveSQLException( + "Error extracting user from delegation token " + delegationToken, "08S01", e); + } + } + + public static void verifyProxyAccess(String realUser, String proxyUser, String ipAddress, + HiveConf hiveConf) throws HiveSQLException { + try { + UserGroupInformation sessionUgi; + if (UserGroupInformation.isSecurityEnabled()) { + KerberosNameShim kerbName = ShimLoader.getHadoopShims().getKerberosNameShim(realUser); + sessionUgi = UserGroupInformation.createProxyUser( + kerbName.getServiceName(), UserGroupInformation.getLoginUser()); + } else { + sessionUgi = UserGroupInformation.createRemoteUser(realUser); + } + if (!proxyUser.equalsIgnoreCase(realUser)) { + ProxyUsers.refreshSuperUserGroupsConfiguration(hiveConf); + ProxyUsers.authorize(UserGroupInformation.createProxyUser(proxyUser, sessionUgi), + ipAddress, hiveConf); + } + } catch (IOException e) { + throw new HiveSQLException( + "Failed to validate proxy privilege of " + realUser + " for " + proxyUser, "08S01", e); + } + } + + public static boolean needUgiLogin(UserGroupInformation ugi, String principal, String keytab) { + return null == ugi || !ugi.hasKerberosCredentials() || !ugi.getUserName().equals(principal) || + !Objects.equals(keytab, getKeytabFromUgi()); + } + + private static String getKeytabFromUgi() { + synchronized (UserGroupInformation.class) { + try { + if (keytabFile != null) { + return (String) keytabFile.get(null); + } else if (getKeytab != null) { + return (String) getKeytab.invoke(UserGroupInformation.getCurrentUser()); + } else { + return null; + } + } catch (Exception e) { + LOG.debug("Fail to get keytabFile path via reflection", e); + return null; + } + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java new file mode 100644 index 0000000000000..b0da86c34ee1d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/HttpAuthUtils.java @@ -0,0 +1,189 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import java.security.AccessControlContext; +import java.security.AccessController; +import java.security.PrivilegedExceptionAction; +import java.util.Arrays; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.StringTokenizer; + +import javax.security.auth.Subject; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.http.protocol.BasicHttpContext; +import org.apache.http.protocol.HttpContext; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * Utility functions for HTTP mode authentication. + */ +public final class HttpAuthUtils { + public static final String WWW_AUTHENTICATE = "WWW-Authenticate"; + public static final String AUTHORIZATION = "Authorization"; + public static final String BASIC = "Basic"; + public static final String NEGOTIATE = "Negotiate"; + private static final Logger LOG = LoggerFactory.getLogger(HttpAuthUtils.class); + private static final String COOKIE_ATTR_SEPARATOR = "&"; + private static final String COOKIE_CLIENT_USER_NAME = "cu"; + private static final String COOKIE_CLIENT_RAND_NUMBER = "rn"; + private static final String COOKIE_KEY_VALUE_SEPARATOR = "="; + private static final Set COOKIE_ATTRIBUTES = + new HashSet(Arrays.asList(COOKIE_CLIENT_USER_NAME, COOKIE_CLIENT_RAND_NUMBER)); + + /** + * @return Stringified Base64 encoded kerberosAuthHeader on success + * @throws Exception + */ + public static String getKerberosServiceTicket(String principal, String host, + String serverHttpUrl, boolean assumeSubject) throws Exception { + String serverPrincipal = + ShimLoader.getHadoopThriftAuthBridge().getServerPrincipal(principal, host); + if (assumeSubject) { + // With this option, we're assuming that the external application, + // using the JDBC driver has done a JAAS kerberos login already + AccessControlContext context = AccessController.getContext(); + Subject subject = Subject.getSubject(context); + if (subject == null) { + throw new Exception("The Subject is not set"); + } + return Subject.doAs(subject, new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } else { + // JAAS login from ticket cache to setup the client UserGroupInformation + UserGroupInformation clientUGI = + ShimLoader.getHadoopThriftAuthBridge().getCurrentUGIWithConf("kerberos"); + return clientUGI.doAs(new HttpKerberosClientAction(serverPrincipal, serverHttpUrl)); + } + } + + /** + * Creates and returns a HS2 cookie token. + * @param clientUserName Client User name. + * @return An unsigned cookie token generated from input parameters. + * The final cookie generated is of the following format : + * {@code cu=&rn=&s=} + */ + public static String createCookieToken(String clientUserName) { + StringBuffer sb = new StringBuffer(); + sb.append(COOKIE_CLIENT_USER_NAME).append(COOKIE_KEY_VALUE_SEPARATOR).append(clientUserName) + .append(COOKIE_ATTR_SEPARATOR); + sb.append(COOKIE_CLIENT_RAND_NUMBER).append(COOKIE_KEY_VALUE_SEPARATOR) + .append((new Random(System.currentTimeMillis())).nextLong()); + return sb.toString(); + } + + /** + * Parses a cookie token to retrieve client user name. + * @param tokenStr Token String. + * @return A valid user name if input is of valid format, else returns null. + */ + public static String getUserNameFromCookieToken(String tokenStr) { + Map map = splitCookieToken(tokenStr); + + if (!map.keySet().equals(COOKIE_ATTRIBUTES)) { + LOG.error("Invalid token with missing attributes " + tokenStr); + return null; + } + return map.get(COOKIE_CLIENT_USER_NAME); + } + + /** + * Splits the cookie token into attributes pairs. + * @param str input token. + * @return a map with the attribute pairs of the token if the input is valid. + * Else, returns null. + */ + private static Map splitCookieToken(String tokenStr) { + Map map = new HashMap(); + StringTokenizer st = new StringTokenizer(tokenStr, COOKIE_ATTR_SEPARATOR); + + while (st.hasMoreTokens()) { + String part = st.nextToken(); + int separator = part.indexOf(COOKIE_KEY_VALUE_SEPARATOR); + if (separator == -1) { + LOG.error("Invalid token string " + tokenStr); + return null; + } + String key = part.substring(0, separator); + String value = part.substring(separator + 1); + map.put(key, value); + } + return map; + } + + + private HttpAuthUtils() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + /** + * We'll create an instance of this class within a doAs block so that the client's TGT credentials + * can be read from the Subject + */ + public static class HttpKerberosClientAction implements PrivilegedExceptionAction { + public static final String HTTP_RESPONSE = "HTTP_RESPONSE"; + public static final String SERVER_HTTP_URL = "SERVER_HTTP_URL"; + private final String serverPrincipal; + private final String serverHttpUrl; + private final Base64 base64codec; + private final HttpContext httpContext; + + public HttpKerberosClientAction(String serverPrincipal, String serverHttpUrl) { + this.serverPrincipal = serverPrincipal; + this.serverHttpUrl = serverHttpUrl; + base64codec = new Base64(0); + httpContext = new BasicHttpContext(); + httpContext.setAttribute(SERVER_HTTP_URL, serverHttpUrl); + } + + @Override + public String run() throws Exception { + // This Oid for Kerberos GSS-API mechanism. + Oid mechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + GSSManager manager = GSSManager.getInstance(); + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + // Create a GSSContext for authentication with the service. + // We're passing client credentials as null since we want them to be read from the Subject. + GSSContext gssContext = + manager.createContext(serverName, mechOid, null, GSSContext.DEFAULT_LIFETIME); + gssContext.requestMutualAuth(false); + // Establish context + byte[] inToken = new byte[0]; + byte[] outToken = gssContext.initSecContext(inToken, 0, inToken.length); + gssContext.dispose(); + // Base64 encoded and stringified token for server + return new String(base64codec.encode(outToken)); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java new file mode 100644 index 0000000000000..cb9595009ecf6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/KerberosSaslHelper.java @@ -0,0 +1,111 @@ +/** + * 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.hive.service.auth; + +import java.io.IOException; +import java.util.Map; +import javax.security.sasl.SaslException; + +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge; +import org.apache.hadoop.hive.thrift.HadoopThriftAuthBridge.Server; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.hive.service.rpc.thrift.TCLIService; +import org.apache.hive.service.rpc.thrift.TCLIService.Iface; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TTransport; + +public final class KerberosSaslHelper { + + public static TProcessorFactory getKerberosProcessorFactory(Server saslServer, + ThriftCLIService service) { + return new CLIServiceProcessorFactory(saslServer, service); + } + + public static TTransport getKerberosTransport(String principal, String host, + TTransport underlyingTransport, Map saslProps, boolean assumeSubject) + throws SaslException { + try { + String[] names = principal.split("[/@]"); + if (names.length != 3) { + throw new IllegalArgumentException("Kerberos principal should have 3 parts: " + principal); + } + + if (assumeSubject) { + return createSubjectAssumedTransport(principal, underlyingTransport, saslProps); + } else { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + return authBridge.createClientTransport(principal, host, "KERBEROS", null, + underlyingTransport, saslProps); + } + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + public static TTransport createSubjectAssumedTransport(String principal, + TTransport underlyingTransport, Map saslProps) throws IOException { + String[] names = principal.split("[/@]"); + try { + TTransport saslTransport = + new TSaslClientTransport("GSSAPI", null, names[0], names[1], saslProps, null, + underlyingTransport); + return new TSubjectAssumingTransport(saslTransport); + } catch (SaslException se) { + throw new IOException("Could not instantiate SASL transport", se); + } + } + + public static TTransport getTokenTransport(String tokenStr, String host, + TTransport underlyingTransport, Map saslProps) throws SaslException { + HadoopThriftAuthBridge.Client authBridge = + ShimLoader.getHadoopThriftAuthBridge().createClientWithConf("kerberos"); + + try { + return authBridge.createClientTransport(null, host, "DIGEST", tokenStr, underlyingTransport, + saslProps); + } catch (IOException e) { + throw new SaslException("Failed to open client transport", e); + } + } + + private KerberosSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static class CLIServiceProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + private final Server saslServer; + + CLIServiceProcessorFactory(Server saslServer, ThriftCLIService service) { + super(null); + this.service = service; + this.saslServer = saslServer; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + TProcessor sqlProcessor = new TCLIService.Processor(service); + return saslServer.wrapNonAssumingProcessor(sqlProcessor); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java new file mode 100644 index 0000000000000..80ab3f8b24658 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/PlainSaslHelper.java @@ -0,0 +1,154 @@ +/** + * 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.hive.service.auth; + +import java.io.IOException; +import java.security.Security; +import java.util.HashMap; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.login.LoginException; +import javax.security.sasl.AuthenticationException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.SaslException; + +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.PlainSaslServer.SaslPlainProvider; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.hive.service.rpc.thrift.TCLIService.Iface; +import org.apache.thrift.TProcessor; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TTransport; +import org.apache.thrift.transport.TTransportFactory; + +public final class PlainSaslHelper { + + public static TProcessorFactory getPlainProcessorFactory(ThriftCLIService service) { + return new SQLPlainProcessorFactory(service); + } + + // Register Plain SASL server provider + static { + Security.addProvider(new SaslPlainProvider()); + } + + public static TTransportFactory getPlainTransportFactory(String authTypeStr) + throws LoginException { + TSaslServerTransport.Factory saslFactory = new TSaslServerTransport.Factory(); + try { + saslFactory.addServerDefinition("PLAIN", authTypeStr, null, new HashMap(), + new PlainServerCallbackHandler(authTypeStr)); + } catch (AuthenticationException e) { + throw new LoginException("Error setting callback handler" + e); + } + return saslFactory; + } + + public static TTransport getPlainTransport(String username, String password, + TTransport underlyingTransport) throws SaslException { + return new TSaslClientTransport("PLAIN", null, null, null, new HashMap(), + new PlainCallbackHandler(username, password), underlyingTransport); + } + + private PlainSaslHelper() { + throw new UnsupportedOperationException("Can't initialize class"); + } + + private static final class PlainServerCallbackHandler implements CallbackHandler { + + private final AuthMethods authMethod; + + PlainServerCallbackHandler(String authMethodStr) throws AuthenticationException { + authMethod = AuthMethods.getValidAuthMethod(authMethodStr); + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + String username = null; + String password = null; + AuthorizeCallback ac = null; + + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nc = (NameCallback) callback; + username = nc.getName(); + } else if (callback instanceof PasswordCallback) { + PasswordCallback pc = (PasswordCallback) callback; + password = new String(pc.getPassword()); + } else if (callback instanceof AuthorizeCallback) { + ac = (AuthorizeCallback) callback; + } else { + throw new UnsupportedCallbackException(callback); + } + } + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(username, password); + if (ac != null) { + ac.setAuthorized(true); + } + } + } + + public static class PlainCallbackHandler implements CallbackHandler { + + private final String username; + private final String password; + + public PlainCallbackHandler(String username, String password) { + this.username = username; + this.password = password; + } + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nameCallback = (NameCallback) callback; + nameCallback.setName(username); + } else if (callback instanceof PasswordCallback) { + PasswordCallback passCallback = (PasswordCallback) callback; + passCallback.setPassword(password.toCharArray()); + } else { + throw new UnsupportedCallbackException(callback); + } + } + } + } + + private static final class SQLPlainProcessorFactory extends TProcessorFactory { + + private final ThriftCLIService service; + + SQLPlainProcessorFactory(ThriftCLIService service) { + super(null); + this.service = service; + } + + @Override + public TProcessor getProcessor(TTransport trans) { + return new TSetIpAddressProcessor(service); + } + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java new file mode 100644 index 0000000000000..91a30cdefaecc --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/auth/TSetIpAddressProcessor.java @@ -0,0 +1,114 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.auth; + +import org.apache.hive.service.rpc.thrift.TCLIService; +import org.apache.hive.service.rpc.thrift.TCLIService.Iface; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TSaslClientTransport; +import org.apache.thrift.transport.TSaslServerTransport; +import org.apache.thrift.transport.TSocket; +import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class is responsible for setting the ipAddress for operations executed via HiveServer2. + * + * - IP address is only set for operations that calls listeners with hookContext + * - IP address is only set if the underlying transport mechanism is socket + * + * @see org.apache.hadoop.hive.ql.hooks.ExecuteWithHookContext + */ +public class TSetIpAddressProcessor extends TCLIService.Processor { + + private static final Logger LOGGER = LoggerFactory.getLogger(TSetIpAddressProcessor.class.getName()); + + public TSetIpAddressProcessor(Iface iface) { + super(iface); + } + + @Override + public boolean process(final TProtocol in, final TProtocol out) throws TException { + setIpAddress(in); + setUserName(in); + try { + return super.process(in, out); + } finally { + THREAD_LOCAL_USER_NAME.remove(); + THREAD_LOCAL_IP_ADDRESS.remove(); + } + } + + private void setUserName(final TProtocol in) { + TTransport transport = in.getTransport(); + if (transport instanceof TSaslServerTransport) { + String userName = ((TSaslServerTransport) transport).getSaslServer().getAuthorizationID(); + THREAD_LOCAL_USER_NAME.set(userName); + } + } + + protected void setIpAddress(final TProtocol in) { + TTransport transport = in.getTransport(); + TSocket tSocket = getUnderlyingSocketFromTransport(transport); + if (tSocket == null) { + LOGGER.warn("Unknown Transport, cannot determine ipAddress"); + } else { + THREAD_LOCAL_IP_ADDRESS.set(tSocket.getSocket().getInetAddress().getHostAddress()); + } + } + + private TSocket getUnderlyingSocketFromTransport(TTransport transport) { + while (transport != null) { + if (transport instanceof TSaslServerTransport) { + transport = ((TSaslServerTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSaslClientTransport) { + transport = ((TSaslClientTransport) transport).getUnderlyingTransport(); + } + if (transport instanceof TSocket) { + return (TSocket) transport; + } + } + return null; + } + + private static final ThreadLocal THREAD_LOCAL_IP_ADDRESS = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + private static final ThreadLocal THREAD_LOCAL_USER_NAME = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static String getUserIpAddress() { + return THREAD_LOCAL_IP_ADDRESS.get(); + } + + public static String getUserName() { + return THREAD_LOCAL_USER_NAME.get(); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java new file mode 100644 index 0000000000000..bdc1e6251e560 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/CLIService.java @@ -0,0 +1,573 @@ +/** + * 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.hive.service.cli; + +import java.io.IOException; +import java.util.List; +import java.util.Map; +import java.util.concurrent.CancellationException; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +import javax.security.auth.login.LoginException; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.HiveMetaStoreClient; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * CLIService. + * + */ +public class CLIService extends CompositeService implements ICLIService { + + public static final TProtocolVersion SERVER_VERSION; + + static { + TProtocolVersion[] protocols = TProtocolVersion.values(); + SERVER_VERSION = protocols[protocols.length - 1]; + } + + private final Logger LOG = LoggerFactory.getLogger(CLIService.class.getName()); + + private HiveConf hiveConf; + private SessionManager sessionManager; + private UserGroupInformation serviceUGI; + private UserGroupInformation httpUGI; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public CLIService(HiveServer2 hiveServer2) { + super(CLIService.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + sessionManager = new SessionManager(hiveServer2); + addService(sessionManager); + // If the hadoop cluster is secure, do a kerberos login for the service from the keytab + if (UserGroupInformation.isSecurityEnabled()) { + try { + HiveAuthFactory.loginFromKeytab(hiveConf); + this.serviceUGI = Utils.getUGI(); + } catch (IOException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } catch (LoginException e) { + throw new ServiceException("Unable to login to kerberos with given principal/keytab", e); + } + + // Also try creating a UGI object for the SPNego principal + String principal = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_PRINCIPAL); + String keyTabFile = hiveConf.getVar(ConfVars.HIVE_SERVER2_SPNEGO_KEYTAB); + if (principal.isEmpty() || keyTabFile.isEmpty()) { + LOG.info("SPNego httpUGI not created, spNegoPrincipal: " + principal + + ", ketabFile: " + keyTabFile); + } else { + try { + this.httpUGI = HiveAuthFactory.loginFromSpnegoKeytabAndReturnUGI(hiveConf); + LOG.info("SPNego httpUGI successfully created."); + } catch (IOException e) { + LOG.warn("SPNego httpUGI creation failed: ", e); + } + } + } + // creates connection to HMS and thus *must* occur after kerberos login above + try { + applyAuthorizationConfigPolicy(hiveConf); + } catch (Exception e) { + throw new RuntimeException("Error applying authorization policy on hive configuration: " + + e.getMessage(), e); + } + setupBlockedUdfs(); + super.init(hiveConf); + } + + private void applyAuthorizationConfigPolicy(HiveConf newHiveConf) throws HiveException, + MetaException { + // authorization setup using SessionState should be revisited eventually, as + // authorization and authentication are not session specific settings + SessionState ss = new SessionState(newHiveConf); + ss.setIsHiveServerQuery(true); + SessionState.start(ss); + ss.applyAuthorizationPolicy(); + } + + private void setupBlockedUdfs() { + FunctionRegistry.setupPermissionsForBuiltinUDFs( + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_WHITELIST), + hiveConf.getVar(ConfVars.HIVE_SERVER2_BUILTIN_UDF_BLACKLIST)); + } + + public UserGroupInformation getServiceUGI() { + return this.serviceUGI; + } + + public UserGroupInformation getHttpUGI() { + return this.httpUGI; + } + + @Override + public synchronized void start() { + super.start(); + // Initialize and test a connection to the metastore + IMetaStoreClient metastoreClient = null; + try { + metastoreClient = new HiveMetaStoreClient(hiveConf); + metastoreClient.getDatabases("default"); + } catch (Exception e) { + throw new ServiceException("Unable to connect to MetaStore!", e); + } + finally { + if (metastoreClient != null) { + metastoreClient.close(); + } + } + } + + @Override + public synchronized void stop() { + super.stop(); + } + + /** + * @deprecated Use {@link #openSession(TProtocolVersion, String, String, String, Map)} + */ + @Deprecated + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /** + * @deprecated Use {@link #openSessionWithImpersonation(TProtocolVersion, String, String, String, Map, String)} + */ + @Deprecated + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSessionWithImpersonation()"); + return sessionHandle; + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map configuration) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + public SessionHandle openSessionWithImpersonation(TProtocolVersion protocol, String username, + String password, String ipAddress, Map configuration, String delegationToken) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(protocol, username, password, ipAddress, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, Map configuration) + throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, false, null); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, Map configuration, + String delegationToken) throws HiveSQLException { + SessionHandle sessionHandle = sessionManager.openSession(SERVER_VERSION, username, password, null, configuration, + true, delegationToken); + LOG.debug(sessionHandle + ": openSession()"); + return sessionHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) + throws HiveSQLException { + sessionManager.closeSession(sessionHandle); + LOG.debug(sessionHandle + ": closeSession()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType getInfoType) + throws HiveSQLException { + GetInfoValue infoValue = sessionManager.getSession(sessionHandle) + .getInfo(getInfoType); + LOG.debug(sessionHandle + ": getInfo()"); + return infoValue; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, + * java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgressMonitor(null); + OperationHandle opHandle = session.executeStatement(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatement()"); + return opHandle; + } + + /** + * Execute statement on the server with a timeout. This is a blocking call. + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException { + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgressMonitor(null); + OperationHandle opHandle = session.executeStatement(statement, confOverlay, queryTimeout); + LOG.debug(sessionHandle + ": executeStatement()"); + return opHandle; + } + + /** + * Execute statement asynchronously on the server. This is a non-blocking call + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgressMonitor(null); + OperationHandle opHandle = session.executeStatementAsync(statement, confOverlay); + LOG.debug(sessionHandle + ": executeStatementAsync()"); + return opHandle; + } + + /** + * Execute statement asynchronously on the server with a timeout. This is a non-blocking call + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException { + HiveSession session = sessionManager.getSession(sessionHandle); + // need to reset the monitor, as operation handle is not available down stream, Ideally the + // monitor should be associated with the operation handle. + session.getSessionState().updateProgressMonitor(null); + OperationHandle opHandle = session.executeStatementAsync(statement, confOverlay, queryTimeout); + LOG.debug(sessionHandle + ": executeStatementAsync()"); + return opHandle; + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTypeInfo(); + LOG.debug(sessionHandle + ": getTypeInfo()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getCatalogs(); + LOG.debug(sessionHandle + ": getCatalogs()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getSchemas(catalogName, schemaName); + LOG.debug(sessionHandle + ": getSchemas()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTables(catalogName, schemaName, tableName, tableTypes); + LOG.debug(sessionHandle + ": getTables()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getTableTypes(); + LOG.debug(sessionHandle + ": getTableTypes()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getColumns(catalogName, schemaName, tableName, columnName); + LOG.debug(sessionHandle + ": getColumns()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getFunctions(catalogName, schemaName, functionName); + LOG.debug(sessionHandle + ": getFunctions()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getPrimaryKeys(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getPrimaryKeys(SessionHandle sessionHandle, + String catalog, String schema, String table) throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getPrimaryKeys(catalog, schema, table); + LOG.debug(sessionHandle + ": getPrimaryKeys()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCrossReference(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCrossReference(SessionHandle sessionHandle, + String primaryCatalog, String primarySchema, String primaryTable, String foreignCatalog, + String foreignSchema, String foreignTable) throws HiveSQLException { + OperationHandle opHandle = sessionManager.getSession(sessionHandle) + .getCrossReference(primaryCatalog, primarySchema, primaryTable, + foreignCatalog, + foreignSchema, foreignTable); + LOG.debug(sessionHandle + ": getCrossReference()"); + return opHandle; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + Operation operation = sessionManager.getOperationManager().getOperation(opHandle); + /** + * If this is a background operation run asynchronously, + * we block for a configured duration, before we return + * (duration: HIVE_SERVER2_LONG_POLLING_TIMEOUT). + * However, if the background operation is complete, we return immediately. + */ + if (operation.shouldRunAsync()) { + HiveConf conf = operation.getParentSession().getHiveConf(); + long timeout = HiveConf.getTimeVar(conf, + HiveConf.ConfVars.HIVE_SERVER2_LONG_POLLING_TIMEOUT, TimeUnit.MILLISECONDS); + try { + operation.getBackgroundHandle().get(timeout, TimeUnit.MILLISECONDS); + } catch (TimeoutException e) { + // No Op, return to the caller since long polling timeout has expired + LOG.trace(opHandle + ": Long polling timed out"); + } catch (CancellationException e) { + // The background operation thread was cancelled + LOG.trace(opHandle + ": The background operation was cancelled", e); + } catch (ExecutionException e) { + // The background operation thread was aborted + LOG.warn(opHandle + ": The background operation was aborted", e); + } catch (InterruptedException e) { + // No op, this thread was interrupted + // In this case, the call might return sooner than long polling timeout + } + } + OperationStatus opStatus = operation.getStatus(); + LOG.debug(opHandle + ": getOperationStatus()"); + return opStatus; + } + + public HiveConf getSessionConf(SessionHandle sessionHandle) throws HiveSQLException { + return sessionManager.getSession(sessionHandle).getHiveConf(); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().cancelOperation(opHandle); + LOG.debug(opHandle + ": cancelOperation()"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().closeOperation(opHandle); + LOG.debug(opHandle + ": closeOperation"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + TableSchema tableSchema = sessionManager.getOperationManager() + .getOperation(opHandle).getParentSession().getResultSetMetadata(opHandle); + LOG.debug(opHandle + ": getResultSetMetadata()"); + return tableSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException { + return fetchResults(opHandle, Operation.DEFAULT_FETCH_ORIENTATION, + Operation.DEFAULT_FETCH_MAX_ROWS, FetchType.QUERY_OUTPUT); + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + RowSet rowSet = sessionManager.getOperationManager().getOperation(opHandle) + .getParentSession().fetchResults(opHandle, orientation, maxRows, fetchType); + LOG.debug(opHandle + ": fetchResults()"); + return rowSet; + } + + // obtain delegation token for the give user from metastore + public synchronized String getDelegationTokenFromMetaStore(String owner) + throws HiveSQLException, UnsupportedOperationException, LoginException, IOException { + if (!hiveConf.getBoolVar(HiveConf.ConfVars.METASTORE_USE_THRIFT_SASL) || + !hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ENABLE_DOAS)) { + throw new UnsupportedOperationException( + "delegation token is can only be obtained for a secure remote metastore"); + } + + try { + Hive.closeCurrent(); + return Hive.get(hiveConf).getDelegationToken(owner, owner); + } catch (HiveException e) { + if (e.getCause() instanceof UnsupportedOperationException) { + throw (UnsupportedOperationException)e.getCause(); + } else { + throw new HiveSQLException("Error connect metastore to setup impersonation", e); + } + } + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + String delegationToken = sessionManager.getSession(sessionHandle) + .getDelegationToken(authFactory, owner, renewer); + LOG.info(sessionHandle + ": getDelegationToken()"); + return delegationToken; + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle).cancelDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": cancelDelegationToken()"); + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + sessionManager.getSession(sessionHandle).renewDelegationToken(authFactory, tokenStr); + LOG.info(sessionHandle + ": renewDelegationToken()"); + } + + public SessionManager getSessionManager() { + return sessionManager; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java new file mode 100644 index 0000000000000..55460606f2ad6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnBasedSet.java @@ -0,0 +1,204 @@ +/** + * 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.hive.service.cli; + +import java.io.ByteArrayInputStream; +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.serde2.thrift.ColumnBuffer; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.rpc.thrift.TColumn; +import org.apache.hive.service.rpc.thrift.TRow; +import org.apache.hive.service.rpc.thrift.TRowSet; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TCompactProtocol; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.transport.TIOStreamTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ColumnBasedSet. + */ +public class ColumnBasedSet implements RowSet { + + private long startOffset; + + private final TypeDescriptor[] descriptors; // non-null only for writing (server-side) + private final List columns; + private byte[] blob; + private boolean isBlobBased = false; + public static final Logger LOG = LoggerFactory.getLogger(ColumnBasedSet.class); + + public ColumnBasedSet(TableSchema schema) { + descriptors = schema.toTypeDescriptors(); + columns = new ArrayList(); + for (ColumnDescriptor colDesc : schema.getColumnDescriptors()) { + columns.add(new ColumnBuffer(colDesc.getType())); + } + } + + public ColumnBasedSet(TRowSet tRowSet) throws TException { + descriptors = null; + columns = new ArrayList(); + // Use TCompactProtocol to read serialized TColumns + if (tRowSet.isSetBinaryColumns()) { + TProtocol protocol = + new TCompactProtocol(new TIOStreamTransport(new ByteArrayInputStream( + tRowSet.getBinaryColumns()))); + // Read from the stream using the protocol for each column in final schema + for (int i = 0; i < tRowSet.getColumnCount(); i++) { + TColumn tvalue = new TColumn(); + try { + tvalue.read(protocol); + } catch (TException e) { + LOG.error(e.getMessage(), e); + throw new TException("Error reading column value from the row set blob", e); + } + columns.add(new ColumnBuffer(tvalue)); + } + } + else { + if (tRowSet.getColumns() != null) { + for (TColumn tvalue : tRowSet.getColumns()) { + columns.add(new ColumnBuffer(tvalue)); + } + } + } + startOffset = tRowSet.getStartRowOffset(); + } + + private ColumnBasedSet(TypeDescriptor[] descriptors, List columns, long startOffset) { + this.descriptors = descriptors; + this.columns = columns; + this.startOffset = startOffset; + } + + public ColumnBasedSet(TableSchema schema, boolean isBlobBased) { + this(schema); + this.isBlobBased = isBlobBased; + } + + @Override + public ColumnBasedSet addRow(Object[] fields) { + if (isBlobBased) { + this.blob = (byte[]) fields[0]; + } else { + for (int i = 0; i < fields.length; i++) { + TypeDescriptor descriptor = descriptors[i]; + Object field = fields[i]; + if (field != null && descriptor.getType() == Type.DECIMAL_TYPE) { + int scale = descriptor.getDecimalDigits(); + field = ((HiveDecimal) field).toFormatString(scale); + } + columns.get(i).addValue(descriptor.getType(), field); + } + } + return this; + } + + public List getColumns() { + return columns; + } + + @Override + public int numColumns() { + return columns.size(); + } + + @Override + public int numRows() { + return columns.isEmpty() ? 0 : columns.get(0).size(); + } + + @Override + public ColumnBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + + List subset = new ArrayList(); + for (int i = 0; i < columns.size(); i++) { + subset.add(columns.get(i).extractSubset(numRows)); + } + ColumnBasedSet result = new ColumnBasedSet(descriptors, subset, startOffset); + startOffset += numRows; + return result; + } + + @Override + public long getStartOffset() { + return startOffset; + } + + @Override + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(startOffset, new ArrayList()); + if (isBlobBased) { + tRowSet.setColumns(null); + tRowSet.setBinaryColumns(blob); + tRowSet.setColumnCount(numColumns()); + } else { + for (int i = 0; i < columns.size(); i++) { + tRowSet.addToColumns(columns.get(i).toTColumn()); + } + } + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + private int index; + private final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return index < numRows(); + } + + @Override + public Object[] next() { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + index++; + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + public Object[] fill(int index, Object[] convey) { + for (int i = 0; i < columns.size(); i++) { + convey[i] = columns.get(i).get(index); + } + return convey; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java new file mode 100644 index 0000000000000..d8e61a87e7f62 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnDescriptor.java @@ -0,0 +1,92 @@ +/** + * 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.hive.service.cli; + +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.rpc.thrift.TColumnDesc; + + +/** + * ColumnDescriptor. + * + */ +public class ColumnDescriptor { + private final String name; + private final String comment; + private final TypeDescriptor type; + // ordinal position of this column in the schema + private final int position; + + public ColumnDescriptor(String name, String comment, TypeDescriptor type, int position) { + this.name = name; + this.comment = comment; + this.type = type; + this.position = position; + } + + public ColumnDescriptor(TColumnDesc tColumnDesc) { + name = tColumnDesc.getColumnName(); + comment = tColumnDesc.getComment(); + type = new TypeDescriptor(tColumnDesc.getTypeDesc()); + position = tColumnDesc.getPosition(); + } + + public static ColumnDescriptor newPrimitiveColumnDescriptor(String name, String comment, Type type, int position) { + // Current usage looks like it's only for metadata columns, but if that changes then + // this method may need to require a type qualifiers aruments. + return new ColumnDescriptor(name, comment, new TypeDescriptor(type), position); + } + + public String getName() { + return name; + } + + public String getComment() { + return comment; + } + + public TypeDescriptor getTypeDescriptor() { + return type; + } + + public int getOrdinalPosition() { + return position; + } + + public TColumnDesc toTColumnDesc() { + TColumnDesc tColumnDesc = new TColumnDesc(); + tColumnDesc.setColumnName(name); + tColumnDesc.setComment(comment); + tColumnDesc.setTypeDesc(type.toTTypeDesc()); + tColumnDesc.setPosition(position); + return tColumnDesc; + } + + public Type getType() { + return type.getType(); + } + + public boolean isPrimitive() { + return type.getType().isPrimitiveType(); + } + + public String getTypeName() { + return type.getTypeName(); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java new file mode 100644 index 0000000000000..28149e1e5edb5 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ColumnValue.java @@ -0,0 +1,313 @@ +/** + * 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.hive.service.cli; + +import java.math.BigDecimal; +import java.sql.Date; +import java.sql.Timestamp; + +import org.apache.hadoop.hive.common.type.HiveChar; +import org.apache.hadoop.hive.common.type.HiveDecimal; +import org.apache.hadoop.hive.common.type.HiveIntervalDayTime; +import org.apache.hadoop.hive.common.type.HiveIntervalYearMonth; +import org.apache.hadoop.hive.common.type.HiveVarchar; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.rpc.thrift.TBoolValue; +import org.apache.hive.service.rpc.thrift.TByteValue; +import org.apache.hive.service.rpc.thrift.TColumnValue; +import org.apache.hive.service.rpc.thrift.TDoubleValue; +import org.apache.hive.service.rpc.thrift.TI16Value; +import org.apache.hive.service.rpc.thrift.TI32Value; +import org.apache.hive.service.rpc.thrift.TI64Value; +import org.apache.hive.service.rpc.thrift.TStringValue; + +/** + * Protocols before HIVE_CLI_SERVICE_PROTOCOL_V6 (used by RowBasedSet) + * + */ +public class ColumnValue { + + private static TColumnValue booleanValue(Boolean value) { + TBoolValue tBoolValue = new TBoolValue(); + if (value != null) { + tBoolValue.setValue(value); + } + return TColumnValue.boolVal(tBoolValue); + } + + private static TColumnValue byteValue(Byte value) { + TByteValue tByteValue = new TByteValue(); + if (value != null) { + tByteValue.setValue(value); + } + return TColumnValue.byteVal(tByteValue); + } + + private static TColumnValue shortValue(Short value) { + TI16Value tI16Value = new TI16Value(); + if (value != null) { + tI16Value.setValue(value); + } + return TColumnValue.i16Val(tI16Value); + } + + private static TColumnValue intValue(Integer value) { + TI32Value tI32Value = new TI32Value(); + if (value != null) { + tI32Value.setValue(value); + } + return TColumnValue.i32Val(tI32Value); + } + + private static TColumnValue longValue(Long value) { + TI64Value tI64Value = new TI64Value(); + if (value != null) { + tI64Value.setValue(value); + } + return TColumnValue.i64Val(tI64Value); + } + + private static TColumnValue floatValue(Float value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue doubleValue(Double value) { + TDoubleValue tDoubleValue = new TDoubleValue(); + if (value != null) { + tDoubleValue.setValue(value); + } + return TColumnValue.doubleVal(tDoubleValue); + } + + private static TColumnValue stringValue(String value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveChar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveVarchar value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue dateValue(Date value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return new TColumnValue(TColumnValue.stringVal(tStringValue)); + } + + private static TColumnValue timestampValue(Timestamp value) { + TStringValue tStringValue = new TStringValue(); + if (value != null) { + tStringValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStringValue); + } + + private static TColumnValue stringValue(HiveDecimal value, TypeDescriptor typeDescriptor) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + int scale = typeDescriptor.getDecimalDigits(); + tStrValue.setValue(value.toFormatString(scale)); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalYearMonth value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + private static TColumnValue stringValue(HiveIntervalDayTime value) { + TStringValue tStrValue = new TStringValue(); + if (value != null) { + tStrValue.setValue(value.toString()); + } + return TColumnValue.stringVal(tStrValue); + } + + public static TColumnValue toTColumnValue(TypeDescriptor typeDescriptor, Object value) { + Type type = typeDescriptor.getType(); + + switch (type) { + case BOOLEAN_TYPE: + return booleanValue((Boolean)value); + case TINYINT_TYPE: + return byteValue((Byte)value); + case SMALLINT_TYPE: + return shortValue((Short)value); + case INT_TYPE: + return intValue((Integer)value); + case BIGINT_TYPE: + return longValue((Long)value); + case FLOAT_TYPE: + return floatValue((Float)value); + case DOUBLE_TYPE: + return doubleValue((Double)value); + case STRING_TYPE: + return stringValue((String)value); + case CHAR_TYPE: + return stringValue((HiveChar)value); + case VARCHAR_TYPE: + return stringValue((HiveVarchar)value); + case DATE_TYPE: + return dateValue((Date)value); + case TIMESTAMP_TYPE: + return timestampValue((Timestamp)value); + case INTERVAL_YEAR_MONTH_TYPE: + return stringValue((HiveIntervalYearMonth) value); + case INTERVAL_DAY_TIME_TYPE: + return stringValue((HiveIntervalDayTime) value); + case DECIMAL_TYPE: + return stringValue((HiveDecimal)value, typeDescriptor); + case BINARY_TYPE: + return stringValue((String)value); + case ARRAY_TYPE: + case MAP_TYPE: + case STRUCT_TYPE: + case UNION_TYPE: + case USER_DEFINED_TYPE: + return stringValue((String)value); + case NULL_TYPE: + return stringValue((String)value); + default: + return null; + } + } + + private static Boolean getBooleanValue(TBoolValue tBoolValue) { + if (tBoolValue.isSetValue()) { + return tBoolValue.isValue(); + } + return null; + } + + private static Byte getByteValue(TByteValue tByteValue) { + if (tByteValue.isSetValue()) { + return tByteValue.getValue(); + } + return null; + } + + private static Short getShortValue(TI16Value tI16Value) { + if (tI16Value.isSetValue()) { + return tI16Value.getValue(); + } + return null; + } + + private static Integer getIntegerValue(TI32Value tI32Value) { + if (tI32Value.isSetValue()) { + return tI32Value.getValue(); + } + return null; + } + + private static Long getLongValue(TI64Value tI64Value) { + if (tI64Value.isSetValue()) { + return tI64Value.getValue(); + } + return null; + } + + private static Double getDoubleValue(TDoubleValue tDoubleValue) { + if (tDoubleValue.isSetValue()) { + return tDoubleValue.getValue(); + } + return null; + } + + private static String getStringValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return tStringValue.getValue(); + } + return null; + } + + private static Date getDateValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Date.valueOf(tStringValue.getValue()); + } + return null; + } + + private static Timestamp getTimestampValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return Timestamp.valueOf(tStringValue.getValue()); + } + return null; + } + + private static byte[] getBinaryValue(TStringValue tString) { + if (tString.isSetValue()) { + return tString.getValue().getBytes(); + } + return null; + } + + private static BigDecimal getBigDecimalValue(TStringValue tStringValue) { + if (tStringValue.isSetValue()) { + return new BigDecimal(tStringValue.getValue()); + } + return null; + } + + public static Object toColumnValue(TColumnValue value) { + TColumnValue._Fields field = value.getSetField(); + switch (field) { + case BOOL_VAL: + return getBooleanValue(value.getBoolVal()); + case BYTE_VAL: + return getByteValue(value.getByteVal()); + case I16_VAL: + return getShortValue(value.getI16Val()); + case I32_VAL: + return getIntegerValue(value.getI32Val()); + case I64_VAL: + return getLongValue(value.getI64Val()); + case DOUBLE_VAL: + return getDoubleValue(value.getDoubleVal()); + case STRING_VAL: + return getStringValue(value.getStringVal()); + } + throw new IllegalArgumentException("never"); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java new file mode 100644 index 0000000000000..ced49c9d29764 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/FetchOrientation.java @@ -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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TFetchOrientation; + +/** + * FetchOrientation. + * + */ +public enum FetchOrientation { + FETCH_NEXT(TFetchOrientation.FETCH_NEXT), + FETCH_PRIOR(TFetchOrientation.FETCH_PRIOR), + FETCH_RELATIVE(TFetchOrientation.FETCH_RELATIVE), + FETCH_ABSOLUTE(TFetchOrientation.FETCH_ABSOLUTE), + FETCH_FIRST(TFetchOrientation.FETCH_FIRST), + FETCH_LAST(TFetchOrientation.FETCH_LAST); + + private TFetchOrientation tFetchOrientation; + + FetchOrientation(TFetchOrientation tFetchOrientation) { + this.tFetchOrientation = tFetchOrientation; + } + + public static FetchOrientation getFetchOrientation(TFetchOrientation tFetchOrientation) { + for (FetchOrientation fetchOrientation : values()) { + if (tFetchOrientation.equals(fetchOrientation.toTFetchOrientation())) { + return fetchOrientation; + } + } + // TODO: Should this really default to FETCH_NEXT? + return FETCH_NEXT; + } + + public TFetchOrientation toTFetchOrientation() { + return tFetchOrientation; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java new file mode 100644 index 0000000000000..a64d262a8f301 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoType.java @@ -0,0 +1,96 @@ +/** + * 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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TGetInfoType; + +/** + * GetInfoType. + * + */ +public enum GetInfoType { + CLI_MAX_DRIVER_CONNECTIONS(TGetInfoType.CLI_MAX_DRIVER_CONNECTIONS), + CLI_MAX_CONCURRENT_ACTIVITIES(TGetInfoType.CLI_MAX_CONCURRENT_ACTIVITIES), + CLI_DATA_SOURCE_NAME(TGetInfoType.CLI_DATA_SOURCE_NAME), + CLI_FETCH_DIRECTION(TGetInfoType.CLI_FETCH_DIRECTION), + CLI_SERVER_NAME(TGetInfoType.CLI_SERVER_NAME), + CLI_SEARCH_PATTERN_ESCAPE(TGetInfoType.CLI_SEARCH_PATTERN_ESCAPE), + CLI_DBMS_NAME(TGetInfoType.CLI_DBMS_NAME), + CLI_DBMS_VER(TGetInfoType.CLI_DBMS_VER), + CLI_ACCESSIBLE_TABLES(TGetInfoType.CLI_ACCESSIBLE_TABLES), + CLI_ACCESSIBLE_PROCEDURES(TGetInfoType.CLI_ACCESSIBLE_PROCEDURES), + CLI_CURSOR_COMMIT_BEHAVIOR(TGetInfoType.CLI_CURSOR_COMMIT_BEHAVIOR), + CLI_DATA_SOURCE_READ_ONLY(TGetInfoType.CLI_DATA_SOURCE_READ_ONLY), + CLI_DEFAULT_TXN_ISOLATION(TGetInfoType.CLI_DEFAULT_TXN_ISOLATION), + CLI_IDENTIFIER_CASE(TGetInfoType.CLI_IDENTIFIER_CASE), + CLI_IDENTIFIER_QUOTE_CHAR(TGetInfoType.CLI_IDENTIFIER_QUOTE_CHAR), + CLI_MAX_COLUMN_NAME_LEN(TGetInfoType.CLI_MAX_COLUMN_NAME_LEN), + CLI_MAX_CURSOR_NAME_LEN(TGetInfoType.CLI_MAX_CURSOR_NAME_LEN), + CLI_MAX_SCHEMA_NAME_LEN(TGetInfoType.CLI_MAX_SCHEMA_NAME_LEN), + CLI_MAX_CATALOG_NAME_LEN(TGetInfoType.CLI_MAX_CATALOG_NAME_LEN), + CLI_MAX_TABLE_NAME_LEN(TGetInfoType.CLI_MAX_TABLE_NAME_LEN), + CLI_SCROLL_CONCURRENCY(TGetInfoType.CLI_SCROLL_CONCURRENCY), + CLI_TXN_CAPABLE(TGetInfoType.CLI_TXN_CAPABLE), + CLI_USER_NAME(TGetInfoType.CLI_USER_NAME), + CLI_TXN_ISOLATION_OPTION(TGetInfoType.CLI_TXN_ISOLATION_OPTION), + CLI_INTEGRITY(TGetInfoType.CLI_INTEGRITY), + CLI_GETDATA_EXTENSIONS(TGetInfoType.CLI_GETDATA_EXTENSIONS), + CLI_NULL_COLLATION(TGetInfoType.CLI_NULL_COLLATION), + CLI_ALTER_TABLE(TGetInfoType.CLI_ALTER_TABLE), + CLI_ORDER_BY_COLUMNS_IN_SELECT(TGetInfoType.CLI_ORDER_BY_COLUMNS_IN_SELECT), + CLI_SPECIAL_CHARACTERS(TGetInfoType.CLI_SPECIAL_CHARACTERS), + CLI_MAX_COLUMNS_IN_GROUP_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_GROUP_BY), + CLI_MAX_COLUMNS_IN_INDEX(TGetInfoType.CLI_MAX_COLUMNS_IN_INDEX), + CLI_MAX_COLUMNS_IN_ORDER_BY(TGetInfoType.CLI_MAX_COLUMNS_IN_ORDER_BY), + CLI_MAX_COLUMNS_IN_SELECT(TGetInfoType.CLI_MAX_COLUMNS_IN_SELECT), + CLI_MAX_COLUMNS_IN_TABLE(TGetInfoType.CLI_MAX_COLUMNS_IN_TABLE), + CLI_MAX_INDEX_SIZE(TGetInfoType.CLI_MAX_INDEX_SIZE), + CLI_MAX_ROW_SIZE(TGetInfoType.CLI_MAX_ROW_SIZE), + CLI_MAX_STATEMENT_LEN(TGetInfoType.CLI_MAX_STATEMENT_LEN), + CLI_MAX_TABLES_IN_SELECT(TGetInfoType.CLI_MAX_TABLES_IN_SELECT), + CLI_MAX_USER_NAME_LEN(TGetInfoType.CLI_MAX_USER_NAME_LEN), + CLI_OJ_CAPABILITIES(TGetInfoType.CLI_OJ_CAPABILITIES), + + CLI_XOPEN_CLI_YEAR(TGetInfoType.CLI_XOPEN_CLI_YEAR), + CLI_CURSOR_SENSITIVITY(TGetInfoType.CLI_CURSOR_SENSITIVITY), + CLI_DESCRIBE_PARAMETER(TGetInfoType.CLI_DESCRIBE_PARAMETER), + CLI_CATALOG_NAME(TGetInfoType.CLI_CATALOG_NAME), + CLI_COLLATION_SEQ(TGetInfoType.CLI_COLLATION_SEQ), + CLI_MAX_IDENTIFIER_LEN(TGetInfoType.CLI_MAX_IDENTIFIER_LEN); + + private final TGetInfoType tInfoType; + + GetInfoType(TGetInfoType tInfoType) { + this.tInfoType = tInfoType; + } + + public static GetInfoType getGetInfoType(TGetInfoType tGetInfoType) { + for (GetInfoType infoType : values()) { + if (tGetInfoType.equals(infoType.tInfoType)) { + return infoType; + } + } + throw new IllegalArgumentException("Unrecognized Thrift TGetInfoType value: " + tGetInfoType); + } + + public TGetInfoType toTGetInfoType() { + return tInfoType; + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java new file mode 100644 index 0000000000000..2b2359cc13c0f --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/GetInfoValue.java @@ -0,0 +1,82 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TGetInfoValue; + +/** + * GetInfoValue. + * + */ +public class GetInfoValue { + private String stringValue = null; + private short shortValue; + private int intValue; + private long longValue; + + public GetInfoValue(String stringValue) { + this.stringValue = stringValue; + } + + public GetInfoValue(short shortValue) { + this.shortValue = shortValue; + } + + public GetInfoValue(int intValue) { + this.intValue = intValue; + } + + public GetInfoValue(long longValue) { + this.longValue = longValue; + } + + public GetInfoValue(TGetInfoValue tGetInfoValue) { + switch (tGetInfoValue.getSetField()) { + case STRING_VALUE: + stringValue = tGetInfoValue.getStringValue(); + break; + default: + throw new IllegalArgumentException("Unreconigzed TGetInfoValue"); + } + } + + public TGetInfoValue toTGetInfoValue() { + TGetInfoValue tInfoValue = new TGetInfoValue(); + if (stringValue != null) { + tInfoValue.setStringValue(stringValue); + } + return tInfoValue; + } + + public String getStringValue() { + return stringValue; + } + + public short getShortValue() { + return shortValue; + } + + public int getIntValue() { + return intValue; + } + + public long getLongValue() { + return longValue; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java new file mode 100644 index 0000000000000..64b8bb16387dd --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/Handle.java @@ -0,0 +1,78 @@ +/** + * 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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.THandleIdentifier; + + + + +public abstract class Handle { + + private final HandleIdentifier handleId; + + public Handle() { + handleId = new HandleIdentifier(); + } + + public Handle(HandleIdentifier handleId) { + this.handleId = handleId; + } + + public Handle(THandleIdentifier tHandleIdentifier) { + this.handleId = new HandleIdentifier(tHandleIdentifier); + } + + public HandleIdentifier getHandleIdentifier() { + return handleId; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((handleId == null) ? 0 : handleId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof Handle)) { + return false; + } + Handle other = (Handle) obj; + if (handleId == null) { + if (other.handleId != null) { + return false; + } + } else if (!handleId.equals(other.handleId)) { + return false; + } + return true; + } + + @Override + public abstract String toString(); + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java new file mode 100644 index 0000000000000..b0fae20e6e0e1 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HandleIdentifier.java @@ -0,0 +1,113 @@ +/** + * 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.hive.service.cli; + +import java.nio.ByteBuffer; +import java.util.UUID; + +import org.apache.hive.service.rpc.thrift.THandleIdentifier; + +/** + * HandleIdentifier. + * + */ +public class HandleIdentifier { + private final UUID publicId; + private final UUID secretId; + + public HandleIdentifier() { + publicId = UUID.randomUUID(); + secretId = UUID.randomUUID(); + } + + public HandleIdentifier(UUID publicId, UUID secretId) { + this.publicId = publicId; + this.secretId = secretId; + } + + public HandleIdentifier(THandleIdentifier tHandleId) { + ByteBuffer bb = ByteBuffer.wrap(tHandleId.getGuid()); + this.publicId = new UUID(bb.getLong(), bb.getLong()); + bb = ByteBuffer.wrap(tHandleId.getSecret()); + this.secretId = new UUID(bb.getLong(), bb.getLong()); + } + + public UUID getPublicId() { + return publicId; + } + + public UUID getSecretId() { + return secretId; + } + + public THandleIdentifier toTHandleIdentifier() { + byte[] guid = new byte[16]; + byte[] secret = new byte[16]; + ByteBuffer guidBB = ByteBuffer.wrap(guid); + ByteBuffer secretBB = ByteBuffer.wrap(secret); + guidBB.putLong(publicId.getMostSignificantBits()); + guidBB.putLong(publicId.getLeastSignificantBits()); + secretBB.putLong(secretId.getMostSignificantBits()); + secretBB.putLong(secretId.getLeastSignificantBits()); + return new THandleIdentifier(ByteBuffer.wrap(guid), ByteBuffer.wrap(secret)); + } + + @Override + public int hashCode() { + final int prime = 31; + int result = 1; + result = prime * result + ((publicId == null) ? 0 : publicId.hashCode()); + result = prime * result + ((secretId == null) ? 0 : secretId.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (obj == null) { + return false; + } + if (!(obj instanceof HandleIdentifier)) { + return false; + } + HandleIdentifier other = (HandleIdentifier) obj; + if (publicId == null) { + if (other.publicId != null) { + return false; + } + } else if (!publicId.equals(other.publicId)) { + return false; + } + if (secretId == null) { + if (other.secretId != null) { + return false; + } + } else if (!secretId.equals(other.secretId)) { + return false; + } + return true; + } + + @Override + public String toString() { + return publicId.toString(); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java new file mode 100644 index 0000000000000..c331d5492b04e --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/HiveSQLException.java @@ -0,0 +1,249 @@ +/** + * 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.hive.service.cli; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; + +import org.apache.hive.service.rpc.thrift.TStatus; +import org.apache.hive.service.rpc.thrift.TStatusCode; + +/** + * HiveSQLException. + * + */ +public class HiveSQLException extends SQLException { + + /** + * + */ + private static final long serialVersionUID = -6095254671958748094L; + + /** + * + */ + public HiveSQLException() { + super(); + } + + /** + * @param reason + */ + public HiveSQLException(String reason) { + super(reason); + } + + /** + * @param cause + */ + public HiveSQLException(Throwable cause) { + super(cause); + } + + /** + * @param reason + * @param sqlState + */ + public HiveSQLException(String reason, String sqlState) { + super(reason, sqlState); + } + + /** + * @param reason + * @param cause + */ + public HiveSQLException(String reason, Throwable cause) { + super(reason, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + */ + public HiveSQLException(String reason, String sqlState, int vendorCode) { + super(reason, sqlState, vendorCode); + } + + /** + * @param reason + * @param sqlState + * @param cause + */ + public HiveSQLException(String reason, String sqlState, Throwable cause) { + super(reason, sqlState, cause); + } + + /** + * @param reason + * @param sqlState + * @param vendorCode + * @param cause + */ + public HiveSQLException(String reason, String sqlState, int vendorCode, Throwable cause) { + super(reason, sqlState, vendorCode, cause); + } + + public HiveSQLException(TStatus status) { + // TODO: set correct vendorCode field + super(status.getErrorMessage(), status.getSqlState(), status.getErrorCode()); + if (status.getInfoMessages() != null) { + initCause(toCause(status.getInfoMessages())); + } + } + + /** + * Converts current object to a {@link TStatus} object + * @return a {@link TStatus} object + */ + public TStatus toTStatus() { + // TODO: convert sqlState, etc. + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setSqlState(getSQLState()); + tStatus.setErrorCode(getErrorCode()); + tStatus.setErrorMessage(getMessage()); + tStatus.setInfoMessages(toString(this)); + return tStatus; + } + + /** + * Converts the specified {@link Exception} object into a {@link TStatus} object + * @param e a {@link Exception} object + * @return a {@link TStatus} object + */ + public static TStatus toTStatus(Exception e) { + if (e instanceof HiveSQLException) { + return ((HiveSQLException)e).toTStatus(); + } + TStatus tStatus = new TStatus(TStatusCode.ERROR_STATUS); + tStatus.setErrorMessage(e.getMessage()); + tStatus.setInfoMessages(toString(e)); + return tStatus; + } + + /** + * Converts a {@link Throwable} object into a flattened list of texts including its stack trace + * and the stack traces of the nested causes. + * @param ex a {@link Throwable} object + * @return a flattened list of texts including the {@link Throwable} object's stack trace + * and the stack traces of the nested causes. + */ + public static List toString(Throwable ex) { + return toString(ex, null); + } + + private static List toString(Throwable cause, StackTraceElement[] parent) { + StackTraceElement[] trace = cause.getStackTrace(); + int m = trace.length - 1; + if (parent != null) { + int n = parent.length - 1; + while (m >= 0 && n >= 0 && trace[m].equals(parent[n])) { + m--; + n--; + } + } + List detail = enroll(cause, trace, m); + cause = cause.getCause(); + if (cause != null) { + detail.addAll(toString(cause, trace)); + } + return detail; + } + + private static List enroll(Throwable ex, StackTraceElement[] trace, int max) { + List details = new ArrayList(); + StringBuilder builder = new StringBuilder(); + builder.append('*').append(ex.getClass().getName()).append(':'); + builder.append(ex.getMessage()).append(':'); + builder.append(trace.length).append(':').append(max); + details.add(builder.toString()); + for (int i = 0; i <= max; i++) { + builder.setLength(0); + builder.append(trace[i].getClassName()).append(':'); + builder.append(trace[i].getMethodName()).append(':'); + String fileName = trace[i].getFileName(); + builder.append(fileName == null ? "" : fileName).append(':'); + builder.append(trace[i].getLineNumber()); + details.add(builder.toString()); + } + return details; + } + + /** + * Converts a flattened list of texts including the stack trace and the stack + * traces of the nested causes into a {@link Throwable} object. + * @param details a flattened list of texts including the stack trace and the stack + * traces of the nested causes + * @return a {@link Throwable} object + */ + public static Throwable toCause(List details) { + return toStackTrace(details, null, 0); + } + + private static Throwable toStackTrace(List details, StackTraceElement[] parent, int index) { + String detail = details.get(index++); + if (!detail.startsWith("*")) { + return null; // should not be happened. ignore remaining + } + int i1 = detail.indexOf(':'); + int i3 = detail.lastIndexOf(':'); + int i2 = detail.substring(0, i3).lastIndexOf(':'); + String exceptionClass = detail.substring(1, i1); + String exceptionMessage = detail.substring(i1 + 1, i2); + Throwable ex = newInstance(exceptionClass, exceptionMessage); + + Integer length = Integer.valueOf(detail.substring(i2 + 1, i3)); + Integer unique = Integer.valueOf(detail.substring(i3 + 1)); + + int i = 0; + StackTraceElement[] trace = new StackTraceElement[length]; + for (; i <= unique; i++) { + detail = details.get(index++); + int j1 = detail.indexOf(':'); + int j3 = detail.lastIndexOf(':'); + int j2 = detail.substring(0, j3).lastIndexOf(':'); + String className = detail.substring(0, j1); + String methodName = detail.substring(j1 + 1, j2); + String fileName = detail.substring(j2 + 1, j3); + if (fileName.isEmpty()) { + fileName = null; + } + int lineNumber = Integer.valueOf(detail.substring(j3 + 1)); + trace[i] = new StackTraceElement(className, methodName, fileName, lineNumber); + } + int common = trace.length - i; + if (common > 0) { + System.arraycopy(parent, parent.length - common, trace, trace.length - common, common); + } + if (details.size() > index) { + ex.initCause(toStackTrace(details, trace, index)); + } + ex.setStackTrace(trace); + return ex; + } + + private static Throwable newInstance(String className, String message) { + try { + return (Throwable)Class.forName(className).getConstructor(String.class).newInstance(message); + } catch (Exception e) { + return new RuntimeException(className + ":" + message); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java new file mode 100644 index 0000000000000..3200909477821 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/ICLIService.java @@ -0,0 +1,113 @@ +/** + * 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.hive.service.cli; + +import java.util.List; +import java.util.Map; + + + + +import org.apache.hive.service.auth.HiveAuthFactory; + +public interface ICLIService { + + SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException; + + SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) + throws HiveSQLException; + + void closeSession(SessionHandle sessionHandle) + throws HiveSQLException; + + GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException; + + OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException; + + OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException; + + OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException; + OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException; + + OperationHandle getTypeInfo(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getCatalogs(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getSchemas(SessionHandle sessionHandle, + String catalogName, String schemaName) + throws HiveSQLException; + + OperationHandle getTables(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, List tableTypes) + throws HiveSQLException; + + OperationHandle getTableTypes(SessionHandle sessionHandle) + throws HiveSQLException; + + OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException; + + OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) + throws HiveSQLException; + + OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) + throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) + throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException; + + void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException; + + OperationHandle getPrimaryKeys(SessionHandle sessionHandle, String catalog, + String schema, String table) throws HiveSQLException; + + OperationHandle getCrossReference(SessionHandle sessionHandle, + String primaryCatalog, String primarySchema, String primaryTable, + String foreignCatalog, String foreignSchema, String foreignTable) throws HiveSQLException; +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java new file mode 100644 index 0000000000000..267a6f84872b6 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationHandle.java @@ -0,0 +1,102 @@ +/** + * 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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TOperationHandle; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; + +public class OperationHandle extends Handle { + + private final OperationType opType; + private final TProtocolVersion protocol; + private boolean hasResultSet = false; + + public OperationHandle(OperationType opType, TProtocolVersion protocol) { + super(); + this.opType = opType; + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public OperationHandle(TOperationHandle tOperationHandle) { + this(tOperationHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public OperationHandle(TOperationHandle tOperationHandle, TProtocolVersion protocol) { + super(tOperationHandle.getOperationId()); + this.opType = OperationType.getOperationType(tOperationHandle.getOperationType()); + this.hasResultSet = tOperationHandle.isHasResultSet(); + this.protocol = protocol; + } + + public OperationType getOperationType() { + return opType; + } + + public void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + } + + public boolean hasResultSet() { + return hasResultSet; + } + + public TOperationHandle toTOperationHandle() { + TOperationHandle tOperationHandle = new TOperationHandle(); + tOperationHandle.setOperationId(getHandleIdentifier().toTHandleIdentifier()); + tOperationHandle.setOperationType(opType.toTOperationType()); + tOperationHandle.setHasResultSet(hasResultSet); + return tOperationHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public int hashCode() { + final int prime = 31; + int result = super.hashCode(); + result = prime * result + ((opType == null) ? 0 : opType.hashCode()); + return result; + } + + @Override + public boolean equals(Object obj) { + if (this == obj) { + return true; + } + if (!super.equals(obj)) { + return false; + } + if (!(obj instanceof OperationHandle)) { + return false; + } + OperationHandle other = (OperationHandle) obj; + if (opType != other.opType) { + return false; + } + return true; + } + + @Override + public String toString() { + return "OperationHandle [opType=" + opType + ", getHandleIdentifier()=" + getHandleIdentifier() + + "]"; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java new file mode 100644 index 0000000000000..58e30aceeb54f --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationState.java @@ -0,0 +1,113 @@ +/** + * 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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TOperationState; + +/** + * OperationState. + * + */ +public enum OperationState { + INITIALIZED(TOperationState.INITIALIZED_STATE, false), + RUNNING(TOperationState.RUNNING_STATE, false), + FINISHED(TOperationState.FINISHED_STATE, true), + CANCELED(TOperationState.CANCELED_STATE, true), + CLOSED(TOperationState.CLOSED_STATE, true), + ERROR(TOperationState.ERROR_STATE, true), + UNKNOWN(TOperationState.UKNOWN_STATE, false), + PENDING(TOperationState.PENDING_STATE, false), + TIMEDOUT(TOperationState.TIMEDOUT_STATE, true); + + private final TOperationState tOperationState; + private final boolean terminal; + + OperationState(TOperationState tOperationState, boolean terminal) { + this.tOperationState = tOperationState; + this.terminal = terminal; + } + + // must be sync with TOperationState in order + public static OperationState getOperationState(TOperationState tOperationState) { + return OperationState.values()[tOperationState.getValue()]; + } + + public static void validateTransition(OperationState oldState, + OperationState newState) + throws HiveSQLException { + switch (oldState) { + case INITIALIZED: + switch (newState) { + case PENDING: + case RUNNING: + case CANCELED: + case CLOSED: + case TIMEDOUT: + return; + } + break; + case PENDING: + switch (newState) { + case RUNNING: + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + case TIMEDOUT: + return; + } + break; + case RUNNING: + switch (newState) { + case FINISHED: + case CANCELED: + case ERROR: + case CLOSED: + case TIMEDOUT: + return; + } + break; + case FINISHED: + case CANCELED: + case TIMEDOUT: + case ERROR: + if (OperationState.CLOSED.equals(newState)) { + return; + } + break; + default: + // fall-through + } + throw new HiveSQLException("Illegal Operation state transition " + + "from " + oldState + " to " + newState); + } + + public void validateTransition(OperationState newState) + throws HiveSQLException { + validateTransition(this, newState); + } + + public TOperationState toTOperationState() { + return tOperationState; + } + + public boolean isTerminal() { + return terminal; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java new file mode 100644 index 0000000000000..e712ef0f71044 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/OperationType.java @@ -0,0 +1,58 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TOperationType; + +/** + * OperationType. + * + */ +public enum OperationType { + + UNKNOWN_OPERATION(TOperationType.UNKNOWN), + EXECUTE_STATEMENT(TOperationType.EXECUTE_STATEMENT), + GET_TYPE_INFO(TOperationType.GET_TYPE_INFO), + GET_CATALOGS(TOperationType.GET_CATALOGS), + GET_SCHEMAS(TOperationType.GET_SCHEMAS), + GET_TABLES(TOperationType.GET_TABLES), + GET_TABLE_TYPES(TOperationType.GET_TABLE_TYPES), + GET_COLUMNS(TOperationType.GET_COLUMNS), + GET_FUNCTIONS(TOperationType.GET_FUNCTIONS); + + private TOperationType tOperationType; + + OperationType(TOperationType tOpType) { + this.tOperationType = tOpType; + } + + public static OperationType getOperationType(TOperationType tOperationType) { + // TODO: replace this with a Map? + for (OperationType opType : values()) { + if (tOperationType.equals(opType.tOperationType)) { + return opType; + } + } + return OperationType.UNKNOWN_OPERATION; + } + + public TOperationType toTOperationType() { + return tOperationType; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java new file mode 100644 index 0000000000000..14dcd11bccd13 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowBasedSet.java @@ -0,0 +1,140 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.ArrayList; +import java.util.Iterator; +import java.util.List; + +import org.apache.hive.service.rpc.thrift.TColumnValue; +import org.apache.hive.service.rpc.thrift.TRow; +import org.apache.hive.service.rpc.thrift.TRowSet; + +/** + * RowBasedSet + */ +public class RowBasedSet implements RowSet { + + private long startOffset; + + private final TypeDescriptor[] descriptors; // non-null only for writing (server-side) + private final RemovableList rows; + + public RowBasedSet(TableSchema schema) { + descriptors = schema.toTypeDescriptors(); + rows = new RemovableList(); + } + + public RowBasedSet(TRowSet tRowSet) { + descriptors = null; + rows = new RemovableList(tRowSet.getRows()); + startOffset = tRowSet.getStartRowOffset(); + } + + private RowBasedSet(TypeDescriptor[] descriptors, List rows, long startOffset) { + this.descriptors = descriptors; + this.rows = new RemovableList(rows); + this.startOffset = startOffset; + } + + @Override + public RowBasedSet addRow(Object[] fields) { + TRow tRow = new TRow(); + for (int i = 0; i < fields.length; i++) { + tRow.addToColVals(ColumnValue.toTColumnValue(descriptors[i], fields[i])); + } + rows.add(tRow); + return this; + } + + @Override + public int numColumns() { + return rows.isEmpty() ? 0 : rows.get(0).getColVals().size(); + } + + @Override + public int numRows() { + return rows.size(); + } + + public RowBasedSet extractSubset(int maxRows) { + int numRows = Math.min(numRows(), maxRows); + RowBasedSet result = new RowBasedSet(descriptors, rows.subList(0, numRows), startOffset); + rows.removeRange(0, numRows); + startOffset += numRows; + return result; + } + + public long getStartOffset() { + return startOffset; + } + + public void setStartOffset(long startOffset) { + this.startOffset = startOffset; + } + + public int getSize() { + return rows.size(); + } + + public TRowSet toTRowSet() { + TRowSet tRowSet = new TRowSet(); + tRowSet.setStartRowOffset(startOffset); + tRowSet.setRows(new ArrayList(rows)); + return tRowSet; + } + + @Override + public Iterator iterator() { + return new Iterator() { + + final Iterator iterator = rows.iterator(); + final Object[] convey = new Object[numColumns()]; + + @Override + public boolean hasNext() { + return iterator.hasNext(); + } + + @Override + public Object[] next() { + TRow row = iterator.next(); + List values = row.getColVals(); + for (int i = 0; i < values.size(); i++) { + convey[i] = ColumnValue.toColumnValue(values.get(i)); + } + return convey; + } + + @Override + public void remove() { + throw new UnsupportedOperationException("remove"); + } + }; + } + + private static class RemovableList extends ArrayList { + RemovableList() { super(); } + RemovableList(List rows) { super(rows); } + @Override + public void removeRange(int fromIndex, int toIndex) { + super.removeRange(fromIndex, toIndex); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java new file mode 100644 index 0000000000000..60d79dcadc37b --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSet.java @@ -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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TRowSet; + +public interface RowSet extends Iterable { + + RowSet addRow(Object[] fields); + + RowSet extractSubset(int maxRows); + + int numColumns(); + + int numRows(); + + long getStartOffset(); + + void setStartOffset(long startOffset); + + TRowSet toTRowSet(); +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java new file mode 100644 index 0000000000000..5ed2889e65346 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/RowSetFactory.java @@ -0,0 +1,42 @@ +/** + * 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.hive.service.cli; + +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.apache.hive.service.rpc.thrift.TRowSet; +import org.apache.thrift.TException; + +import static org.apache.hive.service.rpc.thrift.TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V6; + +public class RowSetFactory { + + public static RowSet create(TableSchema schema, TProtocolVersion version, boolean isBlobBased) { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(schema, isBlobBased); + } + return new RowBasedSet(schema); + } + + public static RowSet create(TRowSet results, TProtocolVersion version) throws TException { + if (version.getValue() >= HIVE_CLI_SERVICE_PROTOCOL_V6.getValue()) { + return new ColumnBasedSet(results); + } + return new RowBasedSet(results); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java new file mode 100644 index 0000000000000..5b21242addeb8 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/SessionHandle.java @@ -0,0 +1,72 @@ +/** + * 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.hive.service.cli; + +import java.util.UUID; + +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.apache.hive.service.rpc.thrift.TSessionHandle; + + +/** + * SessionHandle. + * + */ +public class SessionHandle extends Handle { + + private final TProtocolVersion protocol; + + public SessionHandle(TProtocolVersion protocol) { + this.protocol = protocol; + } + + // dummy handle for ThriftCLIService + public SessionHandle(TSessionHandle tSessionHandle) { + this(tSessionHandle, TProtocolVersion.HIVE_CLI_SERVICE_PROTOCOL_V1); + } + + public SessionHandle(TSessionHandle tSessionHandle, TProtocolVersion protocol) { + super(tSessionHandle.getSessionId()); + this.protocol = protocol; + } + + public SessionHandle(HandleIdentifier handleId, TProtocolVersion protocol) { + super(handleId); + this.protocol = protocol; + } + + public UUID getSessionId() { + return getHandleIdentifier().getPublicId(); + } + + public TSessionHandle toTSessionHandle() { + TSessionHandle tSessionHandle = new TSessionHandle(); + tSessionHandle.setSessionId(getHandleIdentifier().toTHandleIdentifier()); + return tSessionHandle; + } + + public TProtocolVersion getProtocolVersion() { + return protocol; + } + + @Override + public String toString() { + return "SessionHandle [" + getHandleIdentifier() + "]"; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java new file mode 100644 index 0000000000000..8d6f9f3ddf9ec --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TableSchema.java @@ -0,0 +1,104 @@ +/** + * 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.hive.service.cli; + +import java.util.ArrayList; +import java.util.List; + +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.rpc.thrift.TColumnDesc; +import org.apache.hive.service.rpc.thrift.TTableSchema; + +/** + * TableSchema. + * + */ +public class TableSchema { + private final List columns = new ArrayList(); + + public TableSchema() { + } + + public TableSchema(int numColumns) { + // TODO: remove this constructor + } + + public TableSchema(TTableSchema tTableSchema) { + for (TColumnDesc tColumnDesc : tTableSchema.getColumns()) { + columns.add(new ColumnDescriptor(tColumnDesc)); + } + } + + public TableSchema(List fieldSchemas) { + int pos = 1; + for (FieldSchema field : fieldSchemas) { + columns.add(new ColumnDescriptor(field.getName(), field.getComment(), + new TypeDescriptor(field.getType()), pos++)); + } + } + + public TableSchema(Schema schema) { + this(schema.getFieldSchemas()); + } + + public List getColumnDescriptors() { + return new ArrayList(columns); + } + + public ColumnDescriptor getColumnDescriptorAt(int pos) { + return columns.get(pos); + } + + public int getSize() { + return columns.size(); + } + + public void clear() { + columns.clear(); + } + + + public TTableSchema toTTableSchema() { + TTableSchema tTableSchema = new TTableSchema(); + for (ColumnDescriptor col : columns) { + tTableSchema.addToColumns(col.toTColumnDesc()); + } + return tTableSchema; + } + + public TypeDescriptor[] toTypeDescriptors() { + TypeDescriptor[] types = new TypeDescriptor[columns.size()]; + for (int i = 0; i < types.length; i++) { + types[i] = columns.get(i).getTypeDescriptor(); + } + return types; + } + + public TableSchema addPrimitiveColumn(String columnName, Type columnType, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, columnType, columns.size() + 1)); + return this; + } + + public TableSchema addStringColumn(String columnName, String columnComment) { + columns.add(ColumnDescriptor.newPrimitiveColumnDescriptor(columnName, columnComment, Type.STRING_TYPE, columns.size() + 1)); + return this; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java new file mode 100644 index 0000000000000..135c957f87849 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeDescriptor.java @@ -0,0 +1,160 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.List; + +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.TypeInfoFactory; +import org.apache.hive.service.rpc.thrift.TPrimitiveTypeEntry; +import org.apache.hive.service.rpc.thrift.TTypeDesc; +import org.apache.hive.service.rpc.thrift.TTypeEntry; + +/** + * TypeDescriptor. + * + */ +public class TypeDescriptor { + + private final Type type; + private String typeName = null; + private TypeQualifiers typeQualifiers = null; + + public TypeDescriptor(Type type) { + this.type = type; + } + + public TypeDescriptor(TTypeDesc tTypeDesc) { + List tTypeEntries = tTypeDesc.getTypes(); + TPrimitiveTypeEntry top = tTypeEntries.get(0).getPrimitiveEntry(); + this.type = Type.getType(top.getType()); + if (top.isSetTypeQualifiers()) { + setTypeQualifiers(TypeQualifiers.fromTTypeQualifiers(top.getTypeQualifiers())); + } + } + + public TypeDescriptor(String typeName) { + this.type = Type.getType(typeName); + if (this.type.isComplexType()) { + this.typeName = typeName; + } else if (this.type.isQualifiedType()) { + PrimitiveTypeInfo pti = TypeInfoFactory.getPrimitiveTypeInfo(typeName); + setTypeQualifiers(TypeQualifiers.fromTypeInfo(pti)); + } + } + + public Type getType() { + return type; + } + + public TTypeDesc toTTypeDesc() { + TPrimitiveTypeEntry primitiveEntry = new TPrimitiveTypeEntry(type.toTType()); + if (getTypeQualifiers() != null) { + primitiveEntry.setTypeQualifiers(getTypeQualifiers().toTTypeQualifiers()); + } + TTypeEntry entry = TTypeEntry.primitiveEntry(primitiveEntry); + + TTypeDesc desc = new TTypeDesc(); + desc.addToTypes(entry); + return desc; + } + + public String getTypeName() { + if (typeName != null) { + return typeName; + } else { + return type.getName(); + } + } + + public TypeQualifiers getTypeQualifiers() { + return typeQualifiers; + } + + public void setTypeQualifiers(TypeQualifiers typeQualifiers) { + this.typeQualifiers = typeQualifiers; + } + + /** + * The column size for this type. + * For numeric data this is the maximum precision. + * For character data this is the length in characters. + * For datetime types this is the length in characters of the String representation + * (assuming the maximum allowed precision of the fractional seconds component). + * For binary data this is the length in bytes. + * Null is returned for data types where the column size is not applicable. + */ + public Integer getColumnSize() { + if (type.isNumericType()) { + return getPrecision(); + } + switch (type) { + case STRING_TYPE: + case BINARY_TYPE: + return Integer.MAX_VALUE; + case CHAR_TYPE: + case VARCHAR_TYPE: + return typeQualifiers.getCharacterMaximumLength(); + case DATE_TYPE: + return 10; + case TIMESTAMP_TYPE: + return 29; + default: + return null; + } + } + + /** + * Maximum precision for numeric types. + * Returns null for non-numeric types. + * @return + */ + public Integer getPrecision() { + if (this.type == Type.DECIMAL_TYPE) { + return typeQualifiers.getPrecision(); + } + return this.type.getMaxPrecision(); + } + + /** + * The number of fractional digits for this type. + * Null is returned for data types where this is not applicable. + */ + public Integer getDecimalDigits() { + switch (this.type) { + case BOOLEAN_TYPE: + case TINYINT_TYPE: + case SMALLINT_TYPE: + case INT_TYPE: + case BIGINT_TYPE: + return 0; + case FLOAT_TYPE: + return 7; + case DOUBLE_TYPE: + return 15; + case DECIMAL_TYPE: + return typeQualifiers.getScale(); + case TIMESTAMP_TYPE: + return 9; + default: + return null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java new file mode 100644 index 0000000000000..bf3c01c2ab001 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/TypeQualifiers.java @@ -0,0 +1,133 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.hadoop.hive.serde2.typeinfo.CharTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.DecimalTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.PrimitiveTypeInfo; +import org.apache.hadoop.hive.serde2.typeinfo.VarcharTypeInfo; +import org.apache.hive.service.rpc.thrift.TCLIServiceConstants; +import org.apache.hive.service.rpc.thrift.TTypeQualifierValue; +import org.apache.hive.service.rpc.thrift.TTypeQualifiers; + +/** + * This class holds type qualifier information for a primitive type, + * such as char/varchar length or decimal precision/scale. + */ +public class TypeQualifiers { + private Integer characterMaximumLength; + private Integer precision; + private Integer scale; + + public TypeQualifiers() {} + + public Integer getCharacterMaximumLength() { + return characterMaximumLength; + } + public void setCharacterMaximumLength(int characterMaximumLength) { + this.characterMaximumLength = characterMaximumLength; + } + + public TTypeQualifiers toTTypeQualifiers() { + TTypeQualifiers ret = null; + + Map qMap = new HashMap(); + if (getCharacterMaximumLength() != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(getCharacterMaximumLength().intValue()); + qMap.put(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH, val); + } + + if (precision != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(precision.intValue()); + qMap.put(TCLIServiceConstants.PRECISION, val); + } + + if (scale != null) { + TTypeQualifierValue val = new TTypeQualifierValue(); + val.setI32Value(scale.intValue()); + qMap.put(TCLIServiceConstants.SCALE, val); + } + + if (qMap.size() > 0) { + ret = new TTypeQualifiers(qMap); + } + + return ret; + } + + public static TypeQualifiers fromTTypeQualifiers(TTypeQualifiers ttq) { + TypeQualifiers ret = null; + if (ttq != null) { + ret = new TypeQualifiers(); + Map tqMap = ttq.getQualifiers(); + + if (tqMap.containsKey(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH)) { + ret.setCharacterMaximumLength( + tqMap.get(TCLIServiceConstants.CHARACTER_MAXIMUM_LENGTH).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.PRECISION)) { + ret.setPrecision(tqMap.get(TCLIServiceConstants.PRECISION).getI32Value()); + } + + if (tqMap.containsKey(TCLIServiceConstants.SCALE)) { + ret.setScale(tqMap.get(TCLIServiceConstants.SCALE).getI32Value()); + } + } + return ret; + } + + public static TypeQualifiers fromTypeInfo(PrimitiveTypeInfo pti) { + TypeQualifiers result = null; + if (pti instanceof VarcharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((VarcharTypeInfo)pti).getLength()); + } else if (pti instanceof CharTypeInfo) { + result = new TypeQualifiers(); + result.setCharacterMaximumLength(((CharTypeInfo)pti).getLength()); + } else if (pti instanceof DecimalTypeInfo) { + result = new TypeQualifiers(); + result.setPrecision(((DecimalTypeInfo)pti).precision()); + result.setScale(((DecimalTypeInfo)pti).scale()); + } + return result; + } + + public Integer getPrecision() { + return precision; + } + + public void setPrecision(Integer precision) { + this.precision = precision; + } + + public Integer getScale() { + return scale; + } + + public void setScale(Integer scale) { + this.scale = scale; + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.java new file mode 100644 index 0000000000000..50091f0230e64 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ClassicTableTypeMapping.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.hive.service.cli.operation; + +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.Set; + +import com.google.common.collect.ArrayListMultimap; +import com.google.common.collect.Iterables; +import com.google.common.collect.Multimap; +import org.apache.hadoop.hive.metastore.TableType; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ClassicTableTypeMapping. + * Classic table type mapping : + * Managed Table to Table + * External Table to Table + * Virtual View to View + */ +public class ClassicTableTypeMapping implements TableTypeMapping { + + private static final Logger LOG = LoggerFactory.getLogger(ClassicTableTypeMapping.class); + + public enum ClassicTableTypes { + TABLE, + VIEW, + MATERIALIZED_VIEW, + } + + private final Map hiveToClientMap = new HashMap(); + private final Multimap clientToHiveMap = ArrayListMultimap.create(); + + public ClassicTableTypeMapping() { + hiveToClientMap.put(TableType.MANAGED_TABLE.name(), ClassicTableTypes.TABLE.name()); + hiveToClientMap.put(TableType.EXTERNAL_TABLE.name(), ClassicTableTypes.TABLE.name()); + hiveToClientMap.put(TableType.VIRTUAL_VIEW.name(), ClassicTableTypes.VIEW.name()); + hiveToClientMap.put(TableType.MATERIALIZED_VIEW.toString(), + ClassicTableTypes.MATERIALIZED_VIEW.toString()); + + clientToHiveMap.putAll(ClassicTableTypes.TABLE.name(), Arrays.asList( + TableType.MANAGED_TABLE.name(), TableType.EXTERNAL_TABLE.name())); + clientToHiveMap.put(ClassicTableTypes.VIEW.name(), TableType.VIRTUAL_VIEW.name()); + clientToHiveMap.put(ClassicTableTypes.MATERIALIZED_VIEW.toString(), + TableType.MATERIALIZED_VIEW.toString()); + } + + @Override + public String[] mapToHiveType(String clientTypeName) { + Collection hiveTableType = clientToHiveMap.get(clientTypeName.toUpperCase()); + if (hiveTableType == null) { + LOG.warn("Not supported client table type " + clientTypeName); + return new String[] {clientTypeName}; + } + return Iterables.toArray(hiveTableType, String.class); + } + + @Override + public String mapToClientType(String hiveTypeName) { + String clientTypeName = hiveToClientMap.get(hiveTypeName); + if (clientTypeName == null) { + LOG.warn("Invalid hive table type " + hiveTypeName); + return hiveTypeName; + } + return clientTypeName; + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (ClassicTableTypes typeNames : ClassicTableTypes.values()) { + typeNameSet.add(typeNames.name()); + } + return typeNameSet; + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java new file mode 100644 index 0000000000000..5b1e21dafe937 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/ExecuteStatementOperation.java @@ -0,0 +1,71 @@ +/** + * 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.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.Map; + +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorFactory; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.session.HiveSession; + +public abstract class ExecuteStatementOperation extends Operation { + protected String statement = null; + + public ExecuteStatementOperation(HiveSession parentSession, String statement, + Map confOverlay, boolean runInBackground) { + super(parentSession, confOverlay, OperationType.EXECUTE_STATEMENT, runInBackground); + this.statement = statement; + } + + public String getStatement() { + return statement; + } + + public static ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, + String statement, Map confOverlay, boolean runAsync, long queryTimeout) + throws HiveSQLException { + String[] tokens = statement.trim().split("\\s+"); + CommandProcessor processor = null; + try { + processor = CommandProcessorFactory.getForHiveCommand(tokens, parentSession.getHiveConf()); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getSQLState(), e); + } + if (processor == null) { + // runAsync, queryTimeout makes sense only for a SQLOperation + return new SQLOperation(parentSession, statement, confOverlay, runAsync, queryTimeout); + } + return new HiveCommandOperation(parentSession, statement, processor, confOverlay); + } + + protected void registerCurrentOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.warn("Failed to get current OperationLog object of Operation: " + + getHandle().getHandleIdentifier()); + isOperationLogEnabled = false; + return; + } + OperationLog.setCurrentOperationLog(operationLog); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java new file mode 100644 index 0000000000000..b85d5c03fedba --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCatalogsOperation.java @@ -0,0 +1,81 @@ +/** + * 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.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetCatalogsOperation. + * + */ +public class GetCatalogsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable."); + + private final RowSet rowSet; + + protected GetCatalogsOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_CATALOGS); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_CATALOGS, null); + } + setState(OperationState.FINISHED); + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } + + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java new file mode 100644 index 0000000000000..c25c742d392b3 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetColumnsOperation.java @@ -0,0 +1,251 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Map.Entry; +import java.util.Set; +import java.util.regex.Pattern; + +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.metastore.api.Table; +import org.apache.hadoop.hive.ql.metadata.TableIterable; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject.HivePrivilegeObjectType; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.cli.ColumnDescriptor; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetColumnsOperation. + * + */ +public class GetColumnsOperation extends MetadataOperation { + + protected static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, + "Catalog name. NULL if not applicable") + .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, + "Schema name") + .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, + "Table name") + .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, + "Column name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL type from java.sql.Types") + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Data source dependent type name, for a UDT the type name is fully qualified") + .addPrimitiveColumn("COLUMN_SIZE", Type.INT_TYPE, + "Column size. For char or date types this is the maximum number of characters," + + " for numeric or decimal types this is precision.") + .addPrimitiveColumn("BUFFER_LENGTH", Type.TINYINT_TYPE, + "Unused") + .addPrimitiveColumn("DECIMAL_DIGITS", Type.INT_TYPE, + "The number of fractional digits") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Radix (typically either 10 or 2)") + .addPrimitiveColumn("NULLABLE", Type.INT_TYPE, + "Is NULL allowed") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Comment describing column (may be null)") + .addPrimitiveColumn("COLUMN_DEF", Type.STRING_TYPE, + "Default value (may be null)") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("CHAR_OCTET_LENGTH", Type.INT_TYPE, + "For char types the maximum number of bytes in the column") + .addPrimitiveColumn("ORDINAL_POSITION", Type.INT_TYPE, + "Index of column in table (starting at 1)") + .addPrimitiveColumn("IS_NULLABLE", Type.STRING_TYPE, + "\"NO\" means column definitely does not allow NULL values; " + + "\"YES\" means the column might allow NULL values. An empty " + + "string means nobody knows.") + .addPrimitiveColumn("SCOPE_CATALOG", Type.STRING_TYPE, + "Catalog of table that is the scope of a reference attribute " + + "(null if DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_SCHEMA", Type.STRING_TYPE, + "Schema of table that is the scope of a reference attribute " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SCOPE_TABLE", Type.STRING_TYPE, + "Table name that this the scope of a reference attribure " + + "(null if the DATA_TYPE isn't REF)") + .addPrimitiveColumn("SOURCE_DATA_TYPE", Type.SMALLINT_TYPE, + "Source type of a distinct type or user-generated Ref type, " + + "SQL type from java.sql.Types (null if DATA_TYPE isn't DISTINCT or user-generated REF)") + .addPrimitiveColumn("IS_AUTO_INCREMENT", Type.STRING_TYPE, + "Indicates whether this column is auto incremented."); + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final String columnName; + + protected final RowSet rowSet; + + protected GetColumnsOperation(HiveSession parentSession, String catalogName, String schemaName, + String tableName, String columnName) { + super(parentSession, OperationType.GET_COLUMNS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + this.columnName = columnName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + String tablePattern = convertIdentifierPattern(tableName, true); + + Pattern columnPattern = null; + if (columnName != null) { + columnPattern = Pattern.compile(convertIdentifierPattern(columnName, false)); + } + + List dbNames = metastoreClient.getDatabases(schemaPattern); + Collections.sort(dbNames); + Map> db2Tabs = new HashMap<>(); + + for (String dbName : dbNames) { + List tableNames = metastoreClient.getTables(dbName, tablePattern); + Collections.sort(tableNames); + db2Tabs.put(dbName, tableNames); + } + + if (isAuthV2Enabled()) { + List privObjs = getPrivObjs(db2Tabs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName + + ", tablePattern : " + tableName; + authorizeMetaGets(HiveOperationType.GET_COLUMNS, privObjs, cmdStr); + } + + int maxBatchSize = SessionState.get().getConf().getIntVar(ConfVars.METASTORE_BATCH_RETRIEVE_MAX); + for (Entry> dbTabs : db2Tabs.entrySet()) { + String dbName = dbTabs.getKey(); + List tableNames = dbTabs.getValue(); + + for (Table table : new TableIterable(metastoreClient, dbName, tableNames, maxBatchSize)) { + + TableSchema schema = new TableSchema(metastoreClient.getSchema(dbName, + table.getTableName())); + List primaryKeys = metastoreClient.getPrimaryKeys(new PrimaryKeysRequest(dbName, table.getTableName())); + Set pkColNames = new HashSet<>(); + for(SQLPrimaryKey key : primaryKeys) { + pkColNames.add(key.getColumn_name().toLowerCase()); + } + for (ColumnDescriptor column : schema.getColumnDescriptors()) { + if (columnPattern != null && !columnPattern.matcher(column.getName()).matches()) { + continue; + } + Object[] rowData = new Object[] { + null, // TABLE_CAT + table.getDbName(), // TABLE_SCHEM + table.getTableName(), // TABLE_NAME + column.getName(), // COLUMN_NAME + column.getType().toJavaSQLType(), // DATA_TYPE + column.getTypeName(), // TYPE_NAME + column.getTypeDescriptor().getColumnSize(), // COLUMN_SIZE + null, // BUFFER_LENGTH, unused + column.getTypeDescriptor().getDecimalDigits(), // DECIMAL_DIGITS + column.getType().getNumPrecRadix(), // NUM_PREC_RADIX + pkColNames.contains(column.getName().toLowerCase()) ? DatabaseMetaData.columnNoNulls + : DatabaseMetaData.columnNullable, // NULLABLE + column.getComment(), // REMARKS + null, // COLUMN_DEF + null, // SQL_DATA_TYPE + null, // SQL_DATETIME_SUB + null, // CHAR_OCTET_LENGTH + column.getOrdinalPosition(), // ORDINAL_POSITION + pkColNames.contains(column.getName().toLowerCase()) ? "NO" : "YES", // IS_NULLABLE + null, // SCOPE_CATALOG + null, // SCOPE_SCHEMA + null, // SCOPE_TABLE + null, // SOURCE_DATA_TYPE + "NO", // IS_AUTO_INCREMENT + }; + rowSet.addRow(rowData); + } + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + + } + + + private List getPrivObjs(Map> db2Tabs) { + List privObjs = new ArrayList<>(); + for (Entry> dbTabs : db2Tabs.entrySet()) { + for (String tabName : dbTabs.getValue()) { + privObjs.add(new HivePrivilegeObject(HivePrivilegeObjectType.TABLE_OR_VIEW, dbTabs.getKey(), + tabName)); + } + } + return privObjs; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java new file mode 100644 index 0000000000000..2dc338127572e --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetCrossReferenceOperation.java @@ -0,0 +1,161 @@ +/** + * 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.hive.service.cli.operation; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.ForeignKeysRequest; +import org.apache.hadoop.hive.metastore.api.SQLForeignKey; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.cli.*; +import org.apache.hive.service.cli.session.HiveSession; + +import java.util.List; + +/** + * GetCrossReferenceOperation. + * + */ +public class GetCrossReferenceOperation extends MetadataOperation { + /** + PKTABLE_CAT String => parent key table catalog (may be null) + PKTABLE_SCHEM String => parent key table schema (may be null) + PKTABLE_NAME String => parent key table name + PKCOLUMN_NAME String => parent key column name + FKTABLE_CAT String => foreign key table catalog (may be null) being exported (may be null) + FKTABLE_SCHEM String => foreign key table schema (may be null) being exported (may be null) + FKTABLE_NAME String => foreign key table name being exported + FKCOLUMN_NAME String => foreign key column name being exported + KEY_SEQ short => sequence number within foreign key( a value of 1 represents the first column of the foreign key, a value of 2 would represent the second column within the foreign key). + UPDATE_RULE short => What happens to foreign key when parent key is updated: + importedNoAction - do not allow update of parent key if it has been imported + importedKeyCascade - change imported key to agree with parent key update + importedKeySetNull - change imported key to NULL if its parent key has been updated + importedKeySetDefault - change imported key to default values if its parent key has been updated + importedKeyRestrict - same as importedKeyNoAction (for ODBC 2.x compatibility) + DELETE_RULE short => What happens to the foreign key when parent key is deleted. + importedKeyNoAction - do not allow delete of parent key if it has been imported + importedKeyCascade - delete rows that import a deleted key + importedKeySetNull - change imported key to NULL if its primary key has been deleted + importedKeyRestrict - same as importedKeyNoAction (for ODBC 2.x compatibility) + importedKeySetDefault - change imported key to default if its parent key has been deleted + FK_NAME String => foreign key name (may be null) + PK_NAME String => parent key name (may be null) + DEFERRABILITY short => can the evaluation of foreign key constraints be deferred until commit + importedKeyInitiallyDeferred - see SQL92 for definition + importedKeyInitiallyImmediate - see SQL92 for definition + importedKeyNotDeferrable - see SQL92 for definition + */ + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("PKTABLE_CAT", Type.STRING_TYPE, + "Parent key table catalog (may be null)") + .addPrimitiveColumn("PKTABLE_SCHEM", Type.STRING_TYPE, + "Parent key table schema (may be null)") + .addPrimitiveColumn("PKTABLE_NAME", Type.STRING_TYPE, + "Parent Key table name") + .addPrimitiveColumn("PKCOLUMN_NAME", Type.STRING_TYPE, + "Parent Key column name") + .addPrimitiveColumn("FKTABLE_CAT", Type.STRING_TYPE, + "Foreign key table catalog (may be null)") + .addPrimitiveColumn("FKTABLE_SCHEM", Type.STRING_TYPE, + "Foreign key table schema (may be null)") + .addPrimitiveColumn("FKTABLE_NAME", Type.STRING_TYPE, + "Foreign Key table name") + .addPrimitiveColumn("FKCOLUMN_NAME", Type.STRING_TYPE, + "Foreign Key column name") + .addPrimitiveColumn("KEQ_SEQ", Type.INT_TYPE, + "Sequence number within primary key") + .addPrimitiveColumn("UPDATE_RULE", Type.INT_TYPE, + "What happens to foreign key when parent key is updated") + .addPrimitiveColumn("DELETE_RULE", Type.INT_TYPE, + "What happens to foreign key when parent key is deleted") + .addPrimitiveColumn("FK_NAME", Type.STRING_TYPE, + "Foreign key name (may be null)") + .addPrimitiveColumn("PK_NAME", Type.STRING_TYPE, + "Primary key name (may be null)") + .addPrimitiveColumn("DEFERRABILITY", Type.INT_TYPE, + "Can the evaluation of foreign key constraints be deferred until commit"); + private final String parentCatalogName; + private final String parentSchemaName; + private final String parentTableName; + private final String foreignCatalogName; + private final String foreignSchemaName; + private final String foreignTableName; + private final RowSet rowSet; + + public GetCrossReferenceOperation(HiveSession parentSession, + String parentCatalogName, String parentSchemaName, String parentTableName, + String foreignCatalog, String foreignSchema, String foreignTable) { + super(parentSession, OperationType.GET_FUNCTIONS); + this.parentCatalogName = parentCatalogName; + this.parentSchemaName = parentSchemaName; + this.parentTableName = parentTableName; + this.foreignCatalogName = foreignCatalog; + this.foreignSchemaName = foreignSchema; + this.foreignTableName = foreignTable; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + ForeignKeysRequest fkReq = new ForeignKeysRequest(parentSchemaName, parentTableName, foreignSchemaName, foreignTableName); + List fks = metastoreClient.getForeignKeys(fkReq); + if (fks == null) { + return; + } + for (SQLForeignKey fk : fks) { + rowSet.addRow(new Object[] {parentCatalogName, + fk.getPktable_db(), fk.getPktable_name(), fk.getPkcolumn_name(), + foreignCatalogName, + fk.getFktable_db(), fk.getFktable_name(), fk.getFkcolumn_name(), + fk.getKey_seq(), fk.getUpdate_rule(), fk.getDelete_rule(), fk.getFk_name(), + fk.getPk_name(), 0}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java new file mode 100644 index 0000000000000..6aa0c41458b53 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetFunctionsOperation.java @@ -0,0 +1,147 @@ +/** + * 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.hive.service.cli.operation; + +import java.sql.DatabaseMetaData; +import java.util.List; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.exec.FunctionInfo; +import org.apache.hadoop.hive.ql.exec.FunctionRegistry; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.cli.CLIServiceUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.thrift.TException; + +/** + * GetFunctionsOperation. + * + */ +public class GetFunctionsOperation extends MetadataOperation { + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("FUNCTION_CAT", Type.STRING_TYPE, + "Function catalog (may be null)") + .addPrimitiveColumn("FUNCTION_SCHEM", Type.STRING_TYPE, + "Function schema (may be null)") + .addPrimitiveColumn("FUNCTION_NAME", Type.STRING_TYPE, + "Function name. This is the name used to invoke the function") + .addPrimitiveColumn("REMARKS", Type.STRING_TYPE, + "Explanatory comment on the function") + .addPrimitiveColumn("FUNCTION_TYPE", Type.INT_TYPE, + "Kind of function.") + .addPrimitiveColumn("SPECIFIC_NAME", Type.STRING_TYPE, + "The name which uniquely identifies this function within its schema"); + + private final String catalogName; + private final String schemaName; + private final String functionName; + + private final RowSet rowSet; + + public GetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + super(parentSession, OperationType.GET_FUNCTIONS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.functionName = functionName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + // get databases for schema pattern + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs; + try { + matchingDbs = metastoreClient.getDatabases(schemaPattern); + } catch (TException e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + // authorize this call on the schema objects + List privObjs = HivePrivilegeObjectUtils + .getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_FUNCTIONS, privObjs, cmdStr); + } + + try { + if ((null == catalogName || "".equals(catalogName)) + && (null == schemaName || "".equals(schemaName))) { + Set functionNames = FunctionRegistry + .getFunctionNames(CLIServiceUtils.patternToRegex(functionName)); + for (String functionName : functionNames) { + FunctionInfo functionInfo = FunctionRegistry.getFunctionInfo(functionName); + Object[] rowData = new Object[] { + null, // FUNCTION_CAT + null, // FUNCTION_SCHEM + functionInfo.getDisplayName(), // FUNCTION_NAME + "", // REMARKS + (functionInfo.isGenericUDTF() ? + DatabaseMetaData.functionReturnsTable + : DatabaseMetaData.functionNoTable), // FUNCTION_TYPE + functionInfo.getClass().getCanonicalName() + }; + rowSet.addRow(rowData); + } + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java new file mode 100644 index 0000000000000..7da6a1a19b6cc --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetPrimaryKeysOperation.java @@ -0,0 +1,115 @@ +/** + * 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.hive.service.cli.operation; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.PrimaryKeysRequest; +import org.apache.hadoop.hive.metastore.api.SQLPrimaryKey; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.cli.*; +import org.apache.hive.service.cli.session.HiveSession; + +import java.util.List; + +/** + * GetPrimaryKeysOperation. + * + */ +public class GetPrimaryKeysOperation extends MetadataOperation { + /** + TABLE_CAT String => table catalog (may be null) + TABLE_SCHEM String => table schema (may be null) + TABLE_NAME String => table name + COLUMN_NAME String => column name + KEY_SEQ short => sequence number within primary key( a value of 1 represents the first column of the primary key, a value of 2 would represent the second column within the primary key). + PK_NAME String => primary key name (may be null) + */ + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TABLE_CAT", Type.STRING_TYPE, + "Table catalog (may be null)") + .addPrimitiveColumn("TABLE_SCHEM", Type.STRING_TYPE, + "Table schema (may be null)") + .addPrimitiveColumn("TABLE_NAME", Type.STRING_TYPE, + "Table name") + .addPrimitiveColumn("COLUMN_NAME", Type.STRING_TYPE, + "Column name") + .addPrimitiveColumn("KEQ_SEQ", Type.INT_TYPE, + "Sequence number within primary key") + .addPrimitiveColumn("PK_NAME", Type.STRING_TYPE, + "Primary key name (may be null)"); + + private final String catalogName; + private final String schemaName; + private final String tableName; + + private final RowSet rowSet; + + public GetPrimaryKeysOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName) { + super(parentSession, OperationType.GET_FUNCTIONS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + PrimaryKeysRequest sqlReq = new PrimaryKeysRequest(schemaName, tableName); + List pks = metastoreClient.getPrimaryKeys(sqlReq); + if (pks == null) { + return; + } + for (SQLPrimaryKey pk : pks) { + rowSet.addRow(new Object[] {catalogName, pk.getTable_db(), + pk.getTable_name(), pk.getColumn_name(), pk.getKey_seq(), pk.getPk_name()}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java new file mode 100644 index 0000000000000..f224e1f1f9474 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetSchemasOperation.java @@ -0,0 +1,96 @@ +/** + * 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.hive.service.cli.operation; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetSchemasOperation. + * + */ +public class GetSchemasOperation extends MetadataOperation { + private final String catalogName; + private final String schemaName; + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_CATALOG", "Catalog name."); + + protected RowSet rowSet; + + protected GetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + super(parentSession, OperationType.GET_SCHEMAS); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_SCHEMAS, null, cmdStr); + } + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + for (String dbName : metastoreClient.getDatabases(schemaPattern)) { + rowSet.addRow(new Object[] {dbName, DEFAULT_HIVE_CATALOG}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java new file mode 100644 index 0000000000000..875af7150fc16 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTableTypesOperation.java @@ -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.hive.service.cli.operation; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.TableType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTableTypesOperation. + * + */ +public class GetTableTypesOperation extends MetadataOperation { + + protected static TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_TYPE", "Table type name."); + + private final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + protected GetTableTypesOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TABLE_TYPES); + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TABLETYPES, null); + } + try { + for (TableType type : TableType.values()) { + rowSet.addRow(new String[] {tableTypeMapping.mapToClientType(type.toString())}); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java new file mode 100644 index 0000000000000..ebcbfbdf15327 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTablesOperation.java @@ -0,0 +1,143 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.TableMeta; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObjectUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTablesOperation. + * + */ +public class GetTablesOperation extends MetadataOperation { + + private final String catalogName; + private final String schemaName; + private final String tableName; + private final List tableTypeList; + protected final RowSet rowSet; + private final TableTypeMapping tableTypeMapping; + + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addStringColumn("TABLE_CAT", "Catalog name. NULL if not applicable.") + .addStringColumn("TABLE_SCHEM", "Schema name.") + .addStringColumn("TABLE_NAME", "Table name.") + .addStringColumn("TABLE_TYPE", "The table type, e.g. \"TABLE\", \"VIEW\", etc.") + .addStringColumn("REMARKS", "Comments about the table.") + .addStringColumn("TYPE_CAT", "The types catalog.") + .addStringColumn("TYPE_SCHEM", "The types schema.") + .addStringColumn("TYPE_NAME", "Type name.") + .addStringColumn("SELF_REFERENCING_COL_NAME", + "Name of the designated \"identifier\" column of a typed table.") + .addStringColumn("REF_GENERATION", + "Specifies how values in SELF_REFERENCING_COL_NAME are created."); + + protected GetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + super(parentSession, OperationType.GET_TABLES); + this.catalogName = catalogName; + this.schemaName = schemaName; + this.tableName = tableName; + String tableMappingStr = getParentSession().getHiveConf() + .getVar(HiveConf.ConfVars.HIVE_SERVER2_TABLE_TYPE_MAPPING); + tableTypeMapping = + TableTypeMappingFactory.getTableTypeMapping(tableMappingStr); + if (tableTypes != null) { + tableTypeList = new ArrayList(); + for (String tableType : tableTypes) { + tableTypeList.addAll(Arrays.asList(tableTypeMapping.mapToHiveType(tableType.trim()))); + } + } else { + tableTypeList = null; + } + this.rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + IMetaStoreClient metastoreClient = getParentSession().getMetaStoreClient(); + String schemaPattern = convertSchemaPattern(schemaName); + List matchingDbs = metastoreClient.getDatabases(schemaPattern); + if(isAuthV2Enabled()){ + List privObjs = HivePrivilegeObjectUtils.getHivePrivDbObjects(matchingDbs); + String cmdStr = "catalog : " + catalogName + ", schemaPattern : " + schemaName; + authorizeMetaGets(HiveOperationType.GET_TABLES, privObjs, cmdStr); + } + + String tablePattern = convertIdentifierPattern(tableName, true); + for (TableMeta tableMeta : + metastoreClient.getTableMeta(schemaPattern, tablePattern, tableTypeList)) { + rowSet.addRow(new Object[] { + DEFAULT_HIVE_CATALOG, + tableMeta.getDbName(), + tableMeta.getTableName(), + tableTypeMapping.mapToClientType(tableMeta.getTableType()), + tableMeta.getComments(), + null, null, null, null, null + }); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java new file mode 100644 index 0000000000000..9612eb145638c --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/GetTypeInfoOperation.java @@ -0,0 +1,142 @@ +/** + * 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.hive.service.cli.operation; + +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.serde2.thrift.Type; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * GetTypeInfoOperation. + * + */ +public class GetTypeInfoOperation extends MetadataOperation { + + private static final TableSchema RESULT_SET_SCHEMA = new TableSchema() + .addPrimitiveColumn("TYPE_NAME", Type.STRING_TYPE, + "Type name") + .addPrimitiveColumn("DATA_TYPE", Type.INT_TYPE, + "SQL data type from java.sql.Types") + .addPrimitiveColumn("PRECISION", Type.INT_TYPE, + "Maximum precision") + .addPrimitiveColumn("LITERAL_PREFIX", Type.STRING_TYPE, + "Prefix used to quote a literal (may be null)") + .addPrimitiveColumn("LITERAL_SUFFIX", Type.STRING_TYPE, + "Suffix used to quote a literal (may be null)") + .addPrimitiveColumn("CREATE_PARAMS", Type.STRING_TYPE, + "Parameters used in creating the type (may be null)") + .addPrimitiveColumn("NULLABLE", Type.SMALLINT_TYPE, + "Can you use NULL for this type") + .addPrimitiveColumn("CASE_SENSITIVE", Type.BOOLEAN_TYPE, + "Is it case sensitive") + .addPrimitiveColumn("SEARCHABLE", Type.SMALLINT_TYPE, + "Can you use \"WHERE\" based on this type") + .addPrimitiveColumn("UNSIGNED_ATTRIBUTE", Type.BOOLEAN_TYPE, + "Is it unsigned") + .addPrimitiveColumn("FIXED_PREC_SCALE", Type.BOOLEAN_TYPE, + "Can it be a money value") + .addPrimitiveColumn("AUTO_INCREMENT", Type.BOOLEAN_TYPE, + "Can it be used for an auto-increment value") + .addPrimitiveColumn("LOCAL_TYPE_NAME", Type.STRING_TYPE, + "Localized version of type name (may be null)") + .addPrimitiveColumn("MINIMUM_SCALE", Type.SMALLINT_TYPE, + "Minimum scale supported") + .addPrimitiveColumn("MAXIMUM_SCALE", Type.SMALLINT_TYPE, + "Maximum scale supported") + .addPrimitiveColumn("SQL_DATA_TYPE", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("SQL_DATETIME_SUB", Type.INT_TYPE, + "Unused") + .addPrimitiveColumn("NUM_PREC_RADIX", Type.INT_TYPE, + "Usually 2 or 10"); + + private final RowSet rowSet; + + protected GetTypeInfoOperation(HiveSession parentSession) { + super(parentSession, OperationType.GET_TYPE_INFO); + rowSet = RowSetFactory.create(RESULT_SET_SCHEMA, getProtocolVersion(), false); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + if (isAuthV2Enabled()) { + authorizeMetaGets(HiveOperationType.GET_TYPEINFO, null); + } + try { + for (Type type : Type.values()) { + Object[] rowData = new Object[] { + type.getName(), // TYPE_NAME + type.toJavaSQLType(), // DATA_TYPE + type.getMaxPrecision(), // PRECISION + type.getLiteralPrefix(), // LITERAL_PREFIX + type.getLiteralSuffix(), // LITERAL_SUFFIX + type.getCreateParams(), // CREATE_PARAMS + type.getNullable(), // NULLABLE + type.isCaseSensitive(), // CASE_SENSITIVE + type.getSearchable(), // SEARCHABLE + type.isUnsignedAttribute(), // UNSIGNED_ATTRIBUTE + type.isFixedPrecScale(), // FIXED_PREC_SCALE + type.isAutoIncrement(), // AUTO_INCREMENT + type.getLocalizedName(), // LOCAL_TYPE_NAME + type.getMinimumScale(), // MINIMUM_SCALE + type.getMaximumScale(), // MAXIMUM_SCALE + null, // SQL_DATA_TYPE, unused + null, // SQL_DATETIME_SUB, unused + type.getNumPrecRadix() //NUM_PREC_RADIX + }; + rowSet.addRow(rowData); + } + setState(OperationState.FINISHED); + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException(e); + } + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + return RESULT_SET_SCHEMA; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + assertState(OperationState.FINISHED); + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + rowSet.setStartOffset(0); + } + return rowSet.extractSubset((int)maxRows); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java new file mode 100644 index 0000000000000..95214f4b6ba0d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveCommandOperation.java @@ -0,0 +1,213 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.FileReader; +import java.io.IOException; +import java.io.PrintStream; +import java.io.UnsupportedEncodingException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.processors.CommandProcessor; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.ServiceUtils; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * Executes a HiveCommand + */ +public class HiveCommandOperation extends ExecuteStatementOperation { + private CommandProcessor commandProcessor; + private TableSchema resultSchema = null; + + /** + * For processors other than Hive queries (Driver), they output to session.out (a temp file) + * first and the fetchOne/fetchN/fetchAll functions get the output from pipeIn. + */ + private BufferedReader resultReader; + + + protected HiveCommandOperation(HiveSession parentSession, String statement, + CommandProcessor commandProcessor, Map confOverlay) { + super(parentSession, statement, confOverlay, false); + this.commandProcessor = commandProcessor; + setupSessionIO(parentSession.getSessionState()); + } + + private void setupSessionIO(SessionState sessionState) { + try { + LOG.info("Putting temp output to file " + sessionState.getTmpOutputFile().toString()); + sessionState.in = null; // hive server's session input stream is not used + // open a per-session file in auto-flush mode for writing temp results + sessionState.out = new PrintStream(new FileOutputStream(sessionState.getTmpOutputFile()), true, "UTF-8"); + // TODO: for hadoop jobs, progress is printed out to session.err, + // we should find a way to feed back job progress to client + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (IOException e) { + LOG.error("Error in creating temp output file ", e); + try { + sessionState.in = null; + sessionState.out = new PrintStream(System.out, true, "UTF-8"); + sessionState.err = new PrintStream(System.err, true, "UTF-8"); + } catch (UnsupportedEncodingException ee) { + LOG.error("Error creating PrintStream", e); + ee.printStackTrace(); + sessionState.out = null; + sessionState.err = null; + } + } + } + + + private void tearDownSessionIO() { + ServiceUtils.cleanup(LOG, + parentSession.getSessionState().out, parentSession.getSessionState().err); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.RUNNING); + try { + String command = getStatement().trim(); + String[] tokens = statement.split("\\s"); + String commandArgs = command.substring(tokens[0].length()).trim(); + + CommandProcessorResponse response = commandProcessor.run(commandArgs); + int returnCode = response.getResponseCode(); + if (returnCode != 0) { + throw toSQLException("Error while processing statement", response); + } + Schema schema = response.getSchema(); + if (schema != null) { + setHasResultSet(true); + resultSchema = new TableSchema(schema); + } else { + setHasResultSet(false); + resultSchema = new TableSchema(); + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + tearDownSessionIO(); + cleanTmpFile(); + cleanupOperationLog(); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getResultSetSchema() + */ + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + return resultSchema; + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.operation.Operation#getNextRowSet(org.apache.hive.service.cli.FetchOrientation, long) + */ + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + if (orientation.equals(FetchOrientation.FETCH_FIRST)) { + resetResultReader(); + } + List rows = readResults((int) maxRows); + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion(), false); + + for (String row : rows) { + rowSet.addRow(new String[] {row}); + } + return rowSet; + } + + /** + * Reads the temporary results for non-Hive (non-Driver) commands to the + * resulting List of strings. + * @param nLines number of lines read at once. If it is <= 0, then read all lines. + */ + private List readResults(int nLines) throws HiveSQLException { + if (resultReader == null) { + SessionState sessionState = getParentSession().getSessionState(); + File tmp = sessionState.getTmpOutputFile(); + try { + resultReader = new BufferedReader(new FileReader(tmp)); + } catch (FileNotFoundException e) { + LOG.error("File " + tmp + " not found. ", e); + throw new HiveSQLException(e); + } + } + List results = new ArrayList(); + + for (int i = 0; i < nLines || nLines <= 0; ++i) { + try { + String line = resultReader.readLine(); + if (line == null) { + // reached the end of the result file + break; + } else { + results.add(line); + } + } catch (IOException e) { + LOG.error("Reading temp results encountered an exception: ", e); + throw new HiveSQLException(e); + } + } + return results; + } + + private void cleanTmpFile() { + resetResultReader(); + SessionState sessionState = getParentSession().getSessionState(); + sessionState.deleteTmpOutputFile(); + sessionState.deleteTmpErrOutputFile(); + } + + private void resetResultReader() { + if (resultReader != null) { + ServiceUtils.cleanup(LOG, resultReader); + resultReader = null; + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java new file mode 100644 index 0000000000000..f8eb50ea3b242 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/HiveTableTypeMapping.java @@ -0,0 +1,51 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.HashSet; +import java.util.Set; + +import org.apache.hadoop.hive.metastore.TableType; + +/** + * HiveTableTypeMapping. + * Default table type mapping + * + */ +public class HiveTableTypeMapping implements TableTypeMapping { + + @Override + public String[] mapToHiveType(String clientTypeName) { + return new String[] {mapToClientType(clientTypeName)}; + } + + @Override + public String mapToClientType(String hiveTypeName) { + return hiveTypeName; + } + + @Override + public Set getTableTypeNames() { + Set typeNameSet = new HashSet(); + for (TableType typeNames : TableType.values()) { + typeNameSet.add(typeNames.name()); + } + return typeNameSet; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java new file mode 100644 index 0000000000000..cd82716134a99 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/MetadataOperation.java @@ -0,0 +1,134 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.operation; + +import java.util.List; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAccessControlException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzContext; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveAuthzPluginException; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HiveOperationType; +import org.apache.hadoop.hive.ql.security.authorization.plugin.HivePrivilegeObject; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; + +/** + * MetadataOperation. + * + */ +public abstract class MetadataOperation extends Operation { + + protected static final String DEFAULT_HIVE_CATALOG = ""; + protected static TableSchema RESULT_SET_SCHEMA; + private static final char SEARCH_STRING_ESCAPE = '\\'; + + protected MetadataOperation(HiveSession parentSession, OperationType opType) { + super(parentSession, opType); + setHasResultSet(true); + } + + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.Operation#close() + */ + @Override + public void close() throws HiveSQLException { + setState(OperationState.CLOSED); + cleanupOperationLog(); + } + + /** + * Convert wildchars and escape sequence from JDBC format to datanucleous/regex + */ + protected String convertIdentifierPattern(final String pattern, boolean datanucleusFormat) { + if (pattern == null) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, datanucleusFormat); + } + } + + /** + * Convert wildchars and escape sequence of schema pattern from JDBC format to datanucleous/regex + * The schema pattern treats empty string also as wildchar + */ + protected String convertSchemaPattern(final String pattern) { + if ((pattern == null) || pattern.isEmpty()) { + return convertPattern("%", true); + } else { + return convertPattern(pattern, true); + } + } + + /** + * Convert a pattern containing JDBC catalog search wildcards into + * Java regex patterns. + * + * @param pattern input which may contain '%' or '_' wildcard characters, or + * these characters escaped using {@link #getSearchStringEscape()}. + * @return replace %/_ with regex search characters, also handle escaped + * characters. + * + * The datanucleus module expects the wildchar as '*'. The columns search on the + * other hand is done locally inside the hive code and that requires the regex wildchar + * format '.*' This is driven by the datanucleusFormat flag. + */ + private String convertPattern(final String pattern, boolean datanucleusFormat) { + String wStr; + if (datanucleusFormat) { + wStr = "*"; + } else { + wStr = ".*"; + } + return pattern + .replaceAll("([^\\\\])%", "$1" + wStr).replaceAll("\\\\%", "%").replaceAll("^%", wStr) + .replaceAll("([^\\\\])_", "$1.").replaceAll("\\\\_", "_").replaceAll("^_", "."); + } + + protected boolean isAuthV2Enabled(){ + SessionState ss = SessionState.get(); + return (ss.isAuthorizationModeV2() && + HiveConf.getBoolVar(ss.getConf(), HiveConf.ConfVars.HIVE_AUTHORIZATION_ENABLED)); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs) + throws HiveSQLException { + authorizeMetaGets(opType, inpObjs, null); + } + + protected void authorizeMetaGets(HiveOperationType opType, List inpObjs, + String cmdString) throws HiveSQLException { + SessionState ss = SessionState.get(); + HiveAuthzContext.Builder ctxBuilder = new HiveAuthzContext.Builder(); + ctxBuilder.setUserIpAddress(ss.getUserIpAddress()); + ctxBuilder.setCommandString(cmdString); + try { + ss.getAuthorizerV2().checkPrivileges(opType, inpObjs, null, + ctxBuilder.build()); + } catch (HiveAuthzPluginException | HiveAccessControlException e) { + throw new HiveSQLException(e.getMessage(), e); + } + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java new file mode 100644 index 0000000000000..788fcdee282ae --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/Operation.java @@ -0,0 +1,344 @@ +/** + * 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.hive.service.cli.operation; + +import java.io.File; +import java.io.FileNotFoundException; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.OperationType; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public abstract class Operation { + protected final HiveSession parentSession; + private OperationState state = OperationState.INITIALIZED; + private final OperationHandle opHandle; + private HiveConf configuration; + public static final Logger LOG = LoggerFactory.getLogger(Operation.class.getName()); + public static final FetchOrientation DEFAULT_FETCH_ORIENTATION = FetchOrientation.FETCH_NEXT; + public static final long DEFAULT_FETCH_MAX_ROWS = 100; + protected boolean hasResultSet; + protected volatile HiveSQLException operationException; + protected final boolean runAsync; + protected volatile Future backgroundHandle; + protected OperationLog operationLog; + protected boolean isOperationLogEnabled; + protected Map confOverlay = new HashMap(); + + private long operationTimeout; + private long lastAccessTime; + + protected final QueryState queryState; + + protected static final EnumSet DEFAULT_FETCH_ORIENTATION_SET = + EnumSet.of(FetchOrientation.FETCH_NEXT,FetchOrientation.FETCH_FIRST); + + protected Operation(HiveSession parentSession, OperationType opType) { + this(parentSession, null, opType); + } + + protected Operation(HiveSession parentSession, Map confOverlay, + OperationType opType) { + this(parentSession, confOverlay, opType, false); + } + + protected Operation(HiveSession parentSession, + Map confOverlay, OperationType opType, boolean runInBackground) { + this.parentSession = parentSession; + this.confOverlay = confOverlay; + this.runAsync = runInBackground; + this.opHandle = new OperationHandle(opType, parentSession.getProtocolVersion()); + lastAccessTime = System.currentTimeMillis(); + operationTimeout = HiveConf.getTimeVar(parentSession.getHiveConf(), + HiveConf.ConfVars.HIVE_SERVER2_IDLE_OPERATION_TIMEOUT, TimeUnit.MILLISECONDS); + queryState = new QueryState(parentSession.getHiveConf(), confOverlay, runInBackground); + } + + public Future getBackgroundHandle() { + return backgroundHandle; + } + + protected void setBackgroundHandle(Future backgroundHandle) { + this.backgroundHandle = backgroundHandle; + } + + public boolean shouldRunAsync() { + return runAsync; + } + + public void setConfiguration(HiveConf configuration) { + this.configuration = new HiveConf(configuration); + } + + public HiveConf getConfiguration() { + return new HiveConf(configuration); + } + + public HiveSession getParentSession() { + return parentSession; + } + + public OperationHandle getHandle() { + return opHandle; + } + + public TProtocolVersion getProtocolVersion() { + return opHandle.getProtocolVersion(); + } + + public OperationType getType() { + return opHandle.getOperationType(); + } + + public OperationStatus getStatus() { + return new OperationStatus(state, operationException); + } + + public boolean hasResultSet() { + return hasResultSet; + } + + protected void setHasResultSet(boolean hasResultSet) { + this.hasResultSet = hasResultSet; + opHandle.setHasResultSet(hasResultSet); + } + + public OperationLog getOperationLog() { + return operationLog; + } + + protected final OperationState setState(OperationState newState) throws HiveSQLException { + state.validateTransition(newState); + this.state = newState; + this.lastAccessTime = System.currentTimeMillis(); + return this.state; + } + + public boolean isTimedOut(long current) { + if (operationTimeout == 0) { + return false; + } + if (operationTimeout > 0) { + // check only when it's in terminal state + return state.isTerminal() && lastAccessTime + operationTimeout <= current; + } + return lastAccessTime + -operationTimeout <= current; + } + + public long getLastAccessTime() { + return lastAccessTime; + } + + public long getOperationTimeout() { + return operationTimeout; + } + + public void setOperationTimeout(long operationTimeout) { + this.operationTimeout = operationTimeout; + } + + protected void setOperationException(HiveSQLException operationException) { + this.operationException = operationException; + } + + protected final void assertState(OperationState state) throws HiveSQLException { + if (this.state != state) { + throw new HiveSQLException("Expected state " + state + ", but found " + this.state); + } + this.lastAccessTime = System.currentTimeMillis(); + } + + public boolean isRunning() { + return OperationState.RUNNING.equals(state); + } + + public boolean isFinished() { + return OperationState.FINISHED.equals(state); + } + + public boolean isCanceled() { + return OperationState.CANCELED.equals(state); + } + + public boolean isFailed() { + return OperationState.ERROR.equals(state); + } + + protected void createOperationLog() { + if (parentSession.isOperationLogEnabled()) { + File operationLogFile = new File(parentSession.getOperationLogSessionDir(), + opHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + + // create log file + try { + if (operationLogFile.exists()) { + LOG.warn("The operation log file should not exist, but it is already there: " + + operationLogFile.getAbsolutePath()); + operationLogFile.delete(); + } + if (!operationLogFile.createNewFile()) { + // the log file already exists and cannot be deleted. + // If it can be read/written, keep its contents and use it. + if (!operationLogFile.canRead() || !operationLogFile.canWrite()) { + LOG.warn("The already existed operation log file cannot be recreated, " + + "and it cannot be read or written: " + operationLogFile.getAbsolutePath()); + isOperationLogEnabled = false; + return; + } + } + } catch (Exception e) { + LOG.warn("Unable to create operation log file: " + operationLogFile.getAbsolutePath(), e); + isOperationLogEnabled = false; + return; + } + + // create OperationLog object with above log file + try { + operationLog = new OperationLog(opHandle.toString(), operationLogFile, parentSession.getHiveConf()); + } catch (FileNotFoundException e) { + LOG.warn("Unable to instantiate OperationLog object for operation: " + + opHandle, e); + isOperationLogEnabled = false; + return; + } + + // register this operationLog to current thread + OperationLog.setCurrentOperationLog(operationLog); + } + } + + protected void unregisterOperationLog() { + if (isOperationLogEnabled) { + OperationLog.removeCurrentOperationLog(); + } + } + + /** + * Invoked before runInternal(). + * Set up some preconditions, or configurations. + */ + protected void beforeRun() { + createOperationLog(); + } + + /** + * Invoked after runInternal(), even if an exception is thrown in runInternal(). + * Clean up resources, which was set up in beforeRun(). + */ + protected void afterRun() { + unregisterOperationLog(); + } + + /** + * Implemented by subclass of Operation class to execute specific behaviors. + * @throws HiveSQLException + */ + protected abstract void runInternal() throws HiveSQLException; + + public void run() throws HiveSQLException { + beforeRun(); + try { + runInternal(); + } finally { + afterRun(); + } + } + + protected void cleanupOperationLog() { + if (isOperationLogEnabled) { + if (operationLog == null) { + LOG.error("Operation [ " + opHandle.getHandleIdentifier() + " ] " + + "logging is enabled, but its OperationLog object cannot be found."); + } else { + operationLog.close(); + } + } + } + + // TODO: make this abstract and implement in subclasses. + public void cancel() throws HiveSQLException { + setState(OperationState.CANCELED); + throw new UnsupportedOperationException("SQLOperation.cancel()"); + } + + public abstract void close() throws HiveSQLException; + + public abstract TableSchema getResultSetSchema() throws HiveSQLException; + + public abstract RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException; + + public RowSet getNextRowSet() throws HiveSQLException { + return getNextRowSet(FetchOrientation.FETCH_NEXT, DEFAULT_FETCH_MAX_ROWS); + } + + /** + * Verify if the given fetch orientation is part of the default orientation types. + * @param orientation + * @throws HiveSQLException + */ + protected void validateDefaultFetchOrientation(FetchOrientation orientation) + throws HiveSQLException { + validateFetchOrientation(orientation, DEFAULT_FETCH_ORIENTATION_SET); + } + + /** + * Verify if the given fetch orientation is part of the supported orientation types. + * @param orientation + * @param supportedOrientations + * @throws HiveSQLException + */ + protected void validateFetchOrientation(FetchOrientation orientation, + EnumSet supportedOrientations) throws HiveSQLException { + if (!supportedOrientations.contains(orientation)) { + throw new HiveSQLException("The fetch type " + orientation.toString() + + " is not supported for this resultset", "HY106"); + } + } + + protected HiveSQLException toSQLException(String prefix, CommandProcessorResponse response) { + HiveSQLException ex = new HiveSQLException(prefix + ": " + response.getErrorMessage(), + response.getSQLState(), response.getResponseCode()); + if (response.getException() != null) { + ex.initCause(response.getException()); + } + return ex; + } + + protected Map getConfOverlay() { + return confOverlay; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java new file mode 100644 index 0000000000000..75edc5763ce44 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/OperationManager.java @@ -0,0 +1,309 @@ +/** + * 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.hive.service.cli.operation; + +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.session.OperationLog; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.OperationStatus; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.log4j.Appender; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * OperationManager. + * + */ +public class OperationManager extends AbstractService { + private final Logger LOG = LoggerFactory.getLogger(OperationManager.class.getName()); + + private final Map handleToOperation = + new HashMap(); + + public OperationManager() { + super(OperationManager.class.getSimpleName()); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + if (hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogCapture(hiveConf.getVar( + HiveConf.ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LEVEL)); + } else { + LOG.debug("Operation level logging is turned off"); + } + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + // TODO + } + + @Override + public synchronized void stop() { + // TODO + super.stop(); + } + + private void initOperationLogCapture(String loggingMode) { + // Register another Appender (with the same layout) that talks to us. + Appender ap = new LogDivertAppender(this, OperationLog.getLoggingLevel(loggingMode)); + org.apache.log4j.Logger.getRootLogger().addAppender(ap); + } + + public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, + String statement, Map confOverlay, boolean runAsync) + throws HiveSQLException { + ExecuteStatementOperation executeStatementOperation = ExecuteStatementOperation + .newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync, 0); + addOperation(executeStatementOperation); + return executeStatementOperation; + } + + public ExecuteStatementOperation newExecuteStatementOperation(HiveSession parentSession, + String statement, Map confOverlay, boolean runAsync, long queryTimeout) + throws HiveSQLException { + return newExecuteStatementOperation(parentSession, statement, confOverlay, runAsync); + } + + public GetTypeInfoOperation newGetTypeInfoOperation(HiveSession parentSession) { + GetTypeInfoOperation operation = new GetTypeInfoOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetCatalogsOperation newGetCatalogsOperation(HiveSession parentSession) { + GetCatalogsOperation operation = new GetCatalogsOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetSchemasOperation newGetSchemasOperation(HiveSession parentSession, + String catalogName, String schemaName) { + GetSchemasOperation operation = new GetSchemasOperation(parentSession, catalogName, schemaName); + addOperation(operation); + return operation; + } + + public MetadataOperation newGetTablesOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, + List tableTypes) { + MetadataOperation operation = + new GetTablesOperation(parentSession, catalogName, schemaName, tableName, tableTypes); + addOperation(operation); + return operation; + } + + public GetTableTypesOperation newGetTableTypesOperation(HiveSession parentSession) { + GetTableTypesOperation operation = new GetTableTypesOperation(parentSession); + addOperation(operation); + return operation; + } + + public GetColumnsOperation newGetColumnsOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName, String columnName) { + GetColumnsOperation operation = new GetColumnsOperation(parentSession, + catalogName, schemaName, tableName, columnName); + addOperation(operation); + return operation; + } + + public GetFunctionsOperation newGetFunctionsOperation(HiveSession parentSession, + String catalogName, String schemaName, String functionName) { + GetFunctionsOperation operation = new GetFunctionsOperation(parentSession, + catalogName, schemaName, functionName); + addOperation(operation); + return operation; + } + + public GetPrimaryKeysOperation newGetPrimaryKeysOperation(HiveSession parentSession, + String catalogName, String schemaName, String tableName) { + GetPrimaryKeysOperation operation = new GetPrimaryKeysOperation(parentSession, + catalogName, schemaName, tableName); + addOperation(operation); + return operation; + } + + public GetCrossReferenceOperation newGetCrossReferenceOperation( + HiveSession session, String primaryCatalog, String primarySchema, + String primaryTable, String foreignCatalog, String foreignSchema, + String foreignTable) { + GetCrossReferenceOperation operation = new GetCrossReferenceOperation(session, + primaryCatalog, primarySchema, primaryTable, foreignCatalog, foreignSchema, + foreignTable); + addOperation(operation); + return operation; + } + + public Operation getOperation(OperationHandle operationHandle) throws HiveSQLException { + Operation operation = getOperationInternal(operationHandle); + if (operation == null) { + throw new HiveSQLException("Invalid OperationHandle: " + operationHandle); + } + return operation; + } + + private synchronized Operation getOperationInternal(OperationHandle operationHandle) { + return handleToOperation.get(operationHandle); + } + + private synchronized Operation removeTimedOutOperation(OperationHandle operationHandle) { + Operation operation = handleToOperation.get(operationHandle); + if (operation != null && operation.isTimedOut(System.currentTimeMillis())) { + handleToOperation.remove(operationHandle); + return operation; + } + return null; + } + + private synchronized void addOperation(Operation operation) { + handleToOperation.put(operation.getHandle(), operation); + } + + private synchronized Operation removeOperation(OperationHandle opHandle) { + return handleToOperation.remove(opHandle); + } + + public OperationStatus getOperationStatus(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getStatus(); + } + + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = getOperation(opHandle); + OperationState opState = operation.getStatus().getState(); + if (opState == OperationState.CANCELED || + opState == OperationState.CLOSED || + opState == OperationState.FINISHED || + opState == OperationState.ERROR || + opState == OperationState.UNKNOWN) { + // Cancel should be a no-op in either cases + LOG.debug(opHandle + ": Operation is already aborted in state - " + opState); + } + else { + LOG.debug(opHandle + ": Attempting to cancel from state - " + opState); + operation.cancel(); + } + } + + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + Operation operation = removeOperation(opHandle); + if (operation == null) { + throw new HiveSQLException("Operation does not exist!"); + } + operation.close(); + } + + public TableSchema getOperationResultSetSchema(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getResultSetSchema(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(); + } + + public RowSet getOperationNextRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + return getOperation(opHandle).getNextRowSet(orientation, maxRows); + } + + public RowSet getOperationLogRowSet(OperationHandle opHandle, + FetchOrientation orientation, long maxRows) + throws HiveSQLException { + // get the OperationLog object from the operation + OperationLog operationLog = getOperation(opHandle).getOperationLog(); + if (operationLog == null) { + throw new HiveSQLException("Couldn't find log associated with operation handle: " + opHandle); + } + + // read logs + List logs; + try { + logs = operationLog.readOperationLog(isFetchFirst(orientation), maxRows); + } catch (SQLException e) { + throw new HiveSQLException(e.getMessage(), e.getCause()); + } + + + // convert logs to RowSet + TableSchema tableSchema = new TableSchema(getLogSchema()); + RowSet rowSet = RowSetFactory.create(tableSchema, + getOperation(opHandle).getProtocolVersion(), false); + for (String log : logs) { + rowSet.addRow(new String[] {log}); + } + + return rowSet; + } + + private boolean isFetchFirst(FetchOrientation fetchOrientation) { + //TODO: Since OperationLog is moved to package o.a.h.h.ql.session, + // we may add a Enum there and map FetchOrientation to it. + if (fetchOrientation.equals(FetchOrientation.FETCH_FIRST)) { + return true; + } + return false; + } + + private Schema getLogSchema() { + Schema schema = new Schema(); + FieldSchema fieldSchema = new FieldSchema(); + fieldSchema.setName("operation_log"); + fieldSchema.setType("string"); + schema.addToFieldSchemas(fieldSchema); + return schema; + } + + public OperationLog getOperationLogByThread() { + return OperationLog.getCurrentOperationLog(); + } + + public List removeExpiredOperations(OperationHandle[] handles) { + List removed = new ArrayList(); + for (OperationHandle handle : handles) { + Operation operation = removeTimedOutOperation(handle); + if (operation != null) { + LOG.warn("Operation " + handle + " is timed-out and will be closed"); + removed.add(operation); + } + } + return removed; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java new file mode 100644 index 0000000000000..7bfd502b60447 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/SQLOperation.java @@ -0,0 +1,459 @@ +/** + * 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.hive.service.cli.operation; + +import java.io.IOException; +import java.io.Serializable; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.sql.SQLException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.Future; +import java.util.concurrent.RejectedExecutionException; + +import org.apache.commons.codec.binary.Base64; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.metastore.api.FieldSchema; +import org.apache.hadoop.hive.metastore.api.Schema; +import org.apache.hadoop.hive.ql.CommandNeedRetryException; +import org.apache.hadoop.hive.ql.Driver; +import org.apache.hadoop.hive.ql.QueryState; +import org.apache.hadoop.hive.ql.exec.ExplainTask; +import org.apache.hadoop.hive.ql.exec.Task; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.serde.serdeConstants; +import org.apache.hadoop.hive.serde2.AbstractSerDe; +import org.apache.hadoop.hive.serde2.SerDeException; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe; +import org.apache.hadoop.hive.serde2.objectinspector.ObjectInspector; +import org.apache.hadoop.hive.serde2.objectinspector.StructField; +import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.io.BytesWritable; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationState; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.RowSetFactory; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.session.HiveSession; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; + +/** + * SQLOperation. + * + */ +public class SQLOperation extends ExecuteStatementOperation { + + private Driver driver = null; + private CommandProcessorResponse response; + private TableSchema resultSchema = null; + private Schema mResultSchema = null; + private AbstractSerDe serde = null; + private boolean fetchStarted = false; + + public SQLOperation(HiveSession parentSession, String statement, Map confOverlay, + boolean runInBackground, long queryTimeout) { + // TODO: call setRemoteUser in ExecuteStatementOperation or higher. + super(parentSession, statement, confOverlay, runInBackground); + } + + /*** + * Compile the query and extract metadata + * @param queryState + * @throws HiveSQLException + */ + public void prepare(QueryState queryState) throws HiveSQLException { + setState(OperationState.RUNNING); + + try { + driver = new Driver(queryState, getParentSession().getUserName()); + + // set the operation handle information in Driver, so that thrift API users + // can use the operation handle they receive, to lookup query information in + // Yarn ATS + String guid64 = Base64.encodeBase64URLSafeString(getHandle().getHandleIdentifier() + .toTHandleIdentifier().getGuid()).trim(); + driver.setOperationId(guid64); + + // In Hive server mode, we are not able to retry in the FetchTask + // case, when calling fetch queries since execute() has returned. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + + response = driver.compileAndRespond(statement); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while compiling statement", response); + } + + mResultSchema = driver.getSchema(); + + // hasResultSet should be true only if the query has a FetchTask + // "explain" is an exception for now + if(driver.getPlan().getFetchTask() != null) { + //Schema has to be set + if (mResultSchema == null || !mResultSchema.isSetFieldSchemas()) { + throw new HiveSQLException("Error compiling query: Schema and FieldSchema " + + "should be set when query plan has a FetchTask"); + } + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + } else { + setHasResultSet(false); + } + // Set hasResultSet true if the plan has ExplainTask + // TODO explain should use a FetchTask for reading + for (Task task: driver.getPlan().getRootTasks()) { + if (task.getClass() == ExplainTask.class) { + resultSchema = new TableSchema(mResultSchema); + setHasResultSet(true); + break; + } + } + } catch (HiveSQLException e) { + setState(OperationState.ERROR); + throw e; + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + } + + private void runQuery(HiveConf sqlOperationConf) throws HiveSQLException { + try { + // In Hive server mode, we are not able to retry in the FetchTask + // case, when calling fetch queries since execute() has returned. + // For now, we disable the test attempts. + driver.setTryCount(Integer.MAX_VALUE); + response = driver.run(); + if (0 != response.getResponseCode()) { + throw toSQLException("Error while processing statement", response); + } + } catch (HiveSQLException e) { + // If the operation was cancelled by another thread, + // Driver#run will return a non-zero response code. + // We will simply return if the operation state is CANCELED, + // otherwise throw an exception + if (getStatus().getState() == OperationState.CANCELED) { + return; + } + else { + setState(OperationState.ERROR); + throw e; + } + } catch (Exception e) { + setState(OperationState.ERROR); + throw new HiveSQLException("Error running query: " + e.toString(), e); + } + setState(OperationState.FINISHED); + } + + @Override + public void runInternal() throws HiveSQLException { + setState(OperationState.PENDING); + final HiveConf opConfig = getConfigForOperation(); + prepare(queryState); + if (!shouldRunAsync()) { + runQuery(opConfig); + } else { + // We'll pass ThreadLocals in the background thread from the foreground (handler) thread + final SessionState parentSessionState = SessionState.get(); + // ThreadLocal Hive object needs to be set in background thread. + // The metastore client in Hive is associated with right user. + final Hive parentHive = getSessionHive(); + // Current UGI will get used by metastore when metsatore is in embedded mode + // So this needs to get passed to the new background thread + final UserGroupInformation currentUGI = getCurrentUGI(opConfig); + // Runnable impl to call runInternal asynchronously, + // from a different thread + Runnable backgroundOperation = new Runnable() { + @Override + public void run() { + PrivilegedExceptionAction doAsAction = new PrivilegedExceptionAction() { + @Override + public Object run() throws HiveSQLException { + Hive.set(parentHive); + SessionState.setCurrentSessionState(parentSessionState); + // Set current OperationLog in this async thread for keeping on saving query log. + registerCurrentOperationLog(); + try { + runQuery(opConfig); + } catch (HiveSQLException e) { + setOperationException(e); + LOG.error("Error running hive query: ", e); + } finally { + unregisterOperationLog(); + } + return null; + } + }; + + try { + currentUGI.doAs(doAsAction); + } catch (Exception e) { + setOperationException(new HiveSQLException(e)); + LOG.error("Error running hive query as user : " + currentUGI.getShortUserName(), e); + } + finally { + /** + * We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup + * when this thread is garbage collected later. + * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() + */ + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + } + } + }; + try { + // This submit blocks if no background threads are available to run this operation + Future backgroundHandle = + getParentSession().getSessionManager().submitBackgroundOperation(backgroundOperation); + setBackgroundHandle(backgroundHandle); + } catch (RejectedExecutionException rejected) { + setState(OperationState.ERROR); + throw new HiveSQLException("The background threadpool cannot accept" + + " new task for execution, please retry the operation", rejected); + } + } + } + + /** + * Returns the current UGI on the stack + * @param opConfig + * @return UserGroupInformation + * @throws HiveSQLException + */ + private UserGroupInformation getCurrentUGI(HiveConf opConfig) throws HiveSQLException { + try { + return Utils.getUGI(); + } catch (Exception e) { + throw new HiveSQLException("Unable to get current user", e); + } + } + + /** + * Returns the ThreadLocal Hive for the current thread + * @return Hive + * @throws HiveSQLException + */ + private Hive getSessionHive() throws HiveSQLException { + try { + return Hive.get(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get ThreadLocal Hive object", e); + } + } + + private void cleanup(OperationState state) throws HiveSQLException { + setState(state); + if (shouldRunAsync()) { + Future backgroundHandle = getBackgroundHandle(); + if (backgroundHandle != null) { + backgroundHandle.cancel(true); + } + } + if (driver != null) { + driver.close(); + driver.destroy(); + } + driver = null; + + SessionState ss = SessionState.get(); + if (ss.getTmpOutputFile() != null) { + ss.getTmpOutputFile().delete(); + } + } + + @Override + public void cancel() throws HiveSQLException { + cleanup(OperationState.CANCELED); + } + + @Override + public void close() throws HiveSQLException { + cleanup(OperationState.CLOSED); + cleanupOperationLog(); + } + + @Override + public TableSchema getResultSetSchema() throws HiveSQLException { + assertState(OperationState.FINISHED); + if (resultSchema == null) { + resultSchema = new TableSchema(driver.getSchema()); + } + return resultSchema; + } + + private final transient List convey = new ArrayList(); + + @Override + public RowSet getNextRowSet(FetchOrientation orientation, long maxRows) throws HiveSQLException { + validateDefaultFetchOrientation(orientation); + assertState(OperationState.FINISHED); + + RowSet rowSet = RowSetFactory.create(resultSchema, getProtocolVersion(), false); + + try { + /* if client is requesting fetch-from-start and its not the first time reading from this operation + * then reset the fetch position to beginning + */ + if (orientation.equals(FetchOrientation.FETCH_FIRST) && fetchStarted) { + driver.resetFetch(); + } + fetchStarted = true; + driver.setMaxRows((int) maxRows); + if (driver.getResults(convey)) { + return decode(convey, rowSet); + } + return rowSet; + } catch (IOException e) { + throw new HiveSQLException(e); + } catch (CommandNeedRetryException e) { + throw new HiveSQLException(e); + } catch (Exception e) { + throw new HiveSQLException(e); + } finally { + convey.clear(); + } + } + + private RowSet decode(List rows, RowSet rowSet) throws Exception { + if (driver.isFetchingTable()) { + return prepareFromRow(rows, rowSet); + } + return decodeFromString(rows, rowSet); + } + + // already encoded to thrift-able object in ThriftFormatter + private RowSet prepareFromRow(List rows, RowSet rowSet) throws Exception { + for (Object row : rows) { + rowSet.addRow((Object[]) row); + } + return rowSet; + } + + private RowSet decodeFromString(List rows, RowSet rowSet) + throws SQLException, SerDeException { + getSerDe(); + StructObjectInspector soi = (StructObjectInspector) serde.getObjectInspector(); + List fieldRefs = soi.getAllStructFieldRefs(); + + Object[] deserializedFields = new Object[fieldRefs.size()]; + Object rowObj; + ObjectInspector fieldOI; + + int protocol = getProtocolVersion().getValue(); + for (Object rowString : rows) { + try { + rowObj = serde.deserialize(new BytesWritable(((String)rowString).getBytes("UTF-8"))); + } catch (UnsupportedEncodingException e) { + throw new SerDeException(e); + } + for (int i = 0; i < fieldRefs.size(); i++) { + StructField fieldRef = fieldRefs.get(i); + fieldOI = fieldRef.getFieldObjectInspector(); + Object fieldData = soi.getStructFieldData(rowObj, fieldRef); + deserializedFields[i] = SerDeUtils.toThriftPayload(fieldData, fieldOI, protocol); + } + rowSet.addRow(deserializedFields); + } + return rowSet; + } + + private AbstractSerDe getSerDe() throws SQLException { + if (serde != null) { + return serde; + } + try { + List fieldSchemas = mResultSchema.getFieldSchemas(); + StringBuilder namesSb = new StringBuilder(); + StringBuilder typesSb = new StringBuilder(); + + if (fieldSchemas != null && !fieldSchemas.isEmpty()) { + for (int pos = 0; pos < fieldSchemas.size(); pos++) { + if (pos != 0) { + namesSb.append(","); + typesSb.append(","); + } + namesSb.append(fieldSchemas.get(pos).getName()); + typesSb.append(fieldSchemas.get(pos).getType()); + } + } + String names = namesSb.toString(); + String types = typesSb.toString(); + + serde = new LazySimpleSerDe(); + Properties props = new Properties(); + if (names.length() > 0) { + LOG.debug("Column names: " + names); + props.setProperty(serdeConstants.LIST_COLUMNS, names); + } + if (types.length() > 0) { + LOG.debug("Column types: " + types); + props.setProperty(serdeConstants.LIST_COLUMN_TYPES, types); + } + SerDeUtils.initializeSerDe(serde, new HiveConf(), props, null); + + } catch (Exception ex) { + ex.printStackTrace(); + throw new SQLException("Could not create ResultSet: " + ex.getMessage(), ex); + } + return serde; + } + + /** + * If there are query specific settings to overlay, then create a copy of config + * There are two cases we need to clone the session config that's being passed to hive driver + * 1. Async query - + * If the client changes a config setting, that shouldn't reflect in the execution already underway + * 2. confOverlay - + * The query specific settings should only be applied to the query config and not session + * @return new configuration + * @throws HiveSQLException + */ + private HiveConf getConfigForOperation() throws HiveSQLException { + HiveConf sqlOperationConf = getParentSession().getHiveConf(); + if (!getConfOverlay().isEmpty() || shouldRunAsync()) { + // clone the parent session config for this query + sqlOperationConf = new HiveConf(sqlOperationConf); + + // apply overlay query specific settings, if any + for (Map.Entry confEntry : getConfOverlay().entrySet()) { + try { + sqlOperationConf.verifyAndSet(confEntry.getKey(), confEntry.getValue()); + } catch (IllegalArgumentException e) { + throw new HiveSQLException("Error applying statement specific settings", e); + } + } + } + return sqlOperationConf; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java new file mode 100644 index 0000000000000..d7671870a47e5 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/operation/TableTypeMapping.java @@ -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.hive.service.cli.operation; + +import java.util.Set; + +import org.apache.hadoop.hive.common.classification.InterfaceAudience; +import org.apache.hadoop.hive.common.classification.InterfaceStability; + +@InterfaceAudience.Public +@InterfaceStability.Evolving +public interface TableTypeMapping { + /** + * Map client's table type name to hive's table type + * @param clientTypeName + * @return + */ + String[] mapToHiveType(String clientTypeName); + + /** + * Map hive's table type name to client's table type + * @param hiveTypeName + * @return + */ + String mapToClientType(String hiveTypeName); + + /** + * Get all the table types of this mapping + * @return + */ + Set getTableTypeNames(); +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java new file mode 100644 index 0000000000000..8bb64350e696c --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSession.java @@ -0,0 +1,204 @@ +/** + * 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.hive.service.cli.session; + +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; + +public interface HiveSession extends HiveSessionBase { + + void open(Map sessionConfMap) throws Exception; + + IMetaStoreClient getMetaStoreClient() throws HiveSQLException; + + /** + * getInfo operation handler + * @param getInfoType + * @return + * @throws HiveSQLException + */ + GetInfoValue getInfo(GetInfoType getInfoType) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatement(String statement, + Map confOverlay) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @param queryTimeout + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatement(String statement, Map confOverlay, + long queryTimeout) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatementAsync(String statement, Map confOverlay) throws HiveSQLException; + + /** + * execute operation handler + * @param statement + * @param confOverlay + * @param queryTimeout + * @return + * @throws HiveSQLException + */ + OperationHandle executeStatementAsync(String statement, Map confOverlay, + long queryTimeout) throws HiveSQLException; + + /** + * getTypeInfo operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTypeInfo() throws HiveSQLException; + + /** + * getCatalogs operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getCatalogs() throws HiveSQLException; + + /** + * getSchemas operation handler + * @param catalogName + * @param schemaName + * @return + * @throws HiveSQLException + */ + OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException; + + /** + * getTables operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param tableTypes + * @return + * @throws HiveSQLException + */ + OperationHandle getTables(String catalogName, String schemaName, + String tableName, List tableTypes) throws HiveSQLException; + + /** + * getTableTypes operation handler + * @return + * @throws HiveSQLException + */ + OperationHandle getTableTypes() throws HiveSQLException ; + + /** + * getColumns operation handler + * @param catalogName + * @param schemaName + * @param tableName + * @param columnName + * @return + * @throws HiveSQLException + */ + OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException; + + /** + * getFunctions operation handler + * @param catalogName + * @param schemaName + * @param functionName + * @return + * @throws HiveSQLException + */ + OperationHandle getFunctions(String catalogName, String schemaName, + String functionName) throws HiveSQLException; + + /** + * getPrimaryKeys operation handler + * @param catalog + * @param schema + * @param table + * @return + * @throws HiveSQLException + */ + OperationHandle getPrimaryKeys(String catalog, String schema, + String table) throws HiveSQLException; + + + /** + * getCrossReference operation handler + * @param primaryCatalog + * @param primarySchema + * @param primaryTable + * @param foreignCatalog + * @param foreignSchema + * @param foreignTable + * @return + * @throws HiveSQLException + */ + OperationHandle getCrossReference(String primaryCatalog, + String primarySchema, String primaryTable, String foreignCatalog, + String foreignSchema, String foreignTable) throws HiveSQLException; + + /** + * close the session + * @throws HiveSQLException + */ + void close() throws HiveSQLException; + + void cancelOperation(OperationHandle opHandle) throws HiveSQLException; + + void closeOperation(OperationHandle opHandle) throws HiveSQLException; + + TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException; + + RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException; + + String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException; + + void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException; + + void closeExpiredOperations(); + + long getNoOperationTime(); +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java new file mode 100644 index 0000000000000..fb6a7ceecab16 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionBase.java @@ -0,0 +1,90 @@ +/** + * 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.hive.service.cli.session; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; + +import java.io.File; + +/** + * Methods that don't need to be executed under a doAs + * context are here. Rest of them in HiveSession interface + */ +public interface HiveSessionBase { + + TProtocolVersion getProtocolVersion(); + + /** + * Set the session manager for the session + * @param sessionManager + */ + void setSessionManager(SessionManager sessionManager); + + /** + * Get the session manager for the session + */ + SessionManager getSessionManager(); + + /** + * Set operation manager for the session + * @param operationManager + */ + void setOperationManager(OperationManager operationManager); + + /** + * Check whether operation logging is enabled and session dir is created successfully + */ + boolean isOperationLogEnabled(); + + /** + * Get the session dir, which is the parent dir of operation logs + * @return a file representing the parent directory of operation logs + */ + File getOperationLogSessionDir(); + + /** + * Set the session dir, which is the parent dir of operation logs + * @param operationLogRootDir the parent dir of the session dir + */ + void setOperationLogSessionDir(File operationLogRootDir); + + SessionHandle getSessionHandle(); + + String getUsername(); + + String getPassword(); + + HiveConf getHiveConf(); + + SessionState getSessionState(); + + String getUserName(); + + void setUserName(String userName); + + String getIpAddress(); + + void setIpAddress(String ipAddress); + + long getLastAccessTime(); +} diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java rename to sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContext.java diff --git a/sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java similarity index 100% rename from sql/hive-thriftserver/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java rename to sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionHookContextImpl.java diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java new file mode 100644 index 0000000000000..14e9c4704c977 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImpl.java @@ -0,0 +1,893 @@ +/** + * 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.hive.service.cli.session; + +import java.io.BufferedReader; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.io.InputStreamReader; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + +import org.apache.commons.io.FileUtils; +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.hive.common.cli.HiveFileProcessor; +import org.apache.hadoop.hive.common.cli.IHiveFileProcessor; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.conf.VariableSubstitution; +import org.apache.hadoop.hive.metastore.IMetaStoreClient; +import org.apache.hadoop.hive.metastore.api.MetaException; +import org.apache.hadoop.hive.ql.exec.Utilities; +import org.apache.hadoop.hive.ql.history.HiveHistory; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.ql.session.SessionState; +import org.apache.hadoop.hive.serde2.SerDeUtils; +import org.apache.hadoop.hive.serde2.thrift.ThriftFormatter; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.common.util.HiveVersionInfo; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.FetchOrientation; +import org.apache.hive.service.cli.FetchType; +import org.apache.hive.service.cli.GetInfoType; +import org.apache.hive.service.cli.GetInfoValue; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.OperationHandle; +import org.apache.hive.service.cli.RowSet; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.TableSchema; +import org.apache.hive.service.cli.operation.ExecuteStatementOperation; +import org.apache.hive.service.cli.operation.GetCatalogsOperation; +import org.apache.hive.service.cli.operation.GetColumnsOperation; +import org.apache.hive.service.cli.operation.GetCrossReferenceOperation; +import org.apache.hive.service.cli.operation.GetFunctionsOperation; +import org.apache.hive.service.cli.operation.GetPrimaryKeysOperation; +import org.apache.hive.service.cli.operation.GetSchemasOperation; +import org.apache.hive.service.cli.operation.GetTableTypesOperation; +import org.apache.hive.service.cli.operation.GetTypeInfoOperation; +import org.apache.hive.service.cli.operation.MetadataOperation; +import org.apache.hive.service.cli.operation.Operation; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.apache.hive.service.server.ThreadWithGarbageCleanup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.hive.conf.SystemVariables.ENV_PREFIX; +import static org.apache.hadoop.hive.conf.SystemVariables.HIVECONF_PREFIX; +import static org.apache.hadoop.hive.conf.SystemVariables.HIVEVAR_PREFIX; +import static org.apache.hadoop.hive.conf.SystemVariables.METACONF_PREFIX; +import static org.apache.hadoop.hive.conf.SystemVariables.SYSTEM_PREFIX; + +/** + * HiveSession + * + */ +public class HiveSessionImpl implements HiveSession { + private final SessionHandle sessionHandle; + private String username; + private final String password; + private HiveConf hiveConf; + private SessionState sessionState; + private String ipAddress; + private static final String FETCH_WORK_SERDE_CLASS = + "org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe"; + private static final Logger LOG = LoggerFactory.getLogger(HiveSessionImpl.class); + private SessionManager sessionManager; + private OperationManager operationManager; + private final Set opHandleSet = new HashSet(); + private boolean isOperationLogEnabled; + private File sessionLogDir; + private volatile long lastAccessTime; + private volatile long lastIdleTime; + + public HiveSessionImpl(TProtocolVersion protocol, String username, String password, + HiveConf serverhiveConf, String ipAddress) { + this.username = username; + this.password = password; + this.sessionHandle = new SessionHandle(protocol); + this.hiveConf = new HiveConf(serverhiveConf); + this.ipAddress = ipAddress; + + try { + // In non-impersonation mode, map scheduler queue to current user + // if fair scheduler is configured. + if (! hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_MAP_FAIR_SCHEDULER_QUEUE)) { + ShimLoader.getHadoopShims().refreshDefaultQueue(hiveConf, username); + } + } catch (IOException e) { + LOG.warn("Error setting scheduler queue: " + e, e); + } + // Set an explicit session name to control the download directory name + hiveConf.set(ConfVars.HIVESESSIONID.varname, + sessionHandle.getHandleIdentifier().toString()); + // Use thrift transportable formatter + hiveConf.set(SerDeUtils.LIST_SINK_OUTPUT_FORMATTER, ThriftFormatter.class.getName()); + hiveConf.setInt(SerDeUtils.LIST_SINK_OUTPUT_PROTOCOL, protocol.getValue()); + } + + @Override + /** + * Opens a new HiveServer2 session for the client connection. + * Creates a new SessionState object that will be associated with this HiveServer2 session. + * When the server executes multiple queries in the same session, + * this SessionState object is reused across multiple queries. + * Note that if doAs is true, this call goes through a proxy object, + * which wraps the method logic in a UserGroupInformation#doAs. + * That's why it is important to create SessionState here rather than in the constructor. + */ + public void open(Map sessionConfMap) throws HiveSQLException { + sessionState = new SessionState(hiveConf, username); + sessionState.setUserIpAddress(ipAddress); + sessionState.setIsHiveServerQuery(true); + SessionState.start(sessionState); + try { + sessionState.loadAuxJars(); + sessionState.loadReloadableAuxJars(); + } catch (IOException e) { + String msg = "Failed to load reloadable jar file path: " + e; + LOG.error(msg, e); + throw new HiveSQLException(msg, e); + } + // Process global init file: .hiverc + processGlobalInitFile(); + if (sessionConfMap != null) { + configureSession(sessionConfMap); + } + lastAccessTime = System.currentTimeMillis(); + lastIdleTime = lastAccessTime; + } + + /** + * It is used for processing hiverc file from HiveServer2 side. + */ + private class GlobalHivercFileProcessor extends HiveFileProcessor { + @Override + protected BufferedReader loadFile(String fileName) throws IOException { + FileInputStream initStream = null; + BufferedReader bufferedReader = null; + initStream = new FileInputStream(fileName); + bufferedReader = new BufferedReader(new InputStreamReader(initStream)); + return bufferedReader; + } + + @Override + protected int processCmd(String cmd) { + int rc = 0; + String cmd_trimed = cmd.trim(); + try { + executeStatementInternal(cmd_trimed, null, false, 0); + } catch (HiveSQLException e) { + rc = -1; + LOG.warn("Failed to execute HQL command in global .hiverc file.", e); + } + return rc; + } + } + + private void processGlobalInitFile() { + IHiveFileProcessor processor = new GlobalHivercFileProcessor(); + + try { + String hiverc = hiveConf.getVar(ConfVars.HIVE_SERVER2_GLOBAL_INIT_FILE_LOCATION); + if (hiverc != null) { + File hivercFile = new File(hiverc); + if (hivercFile.isDirectory()) { + hivercFile = new File(hivercFile, SessionManager.HIVERCFILE); + } + if (hivercFile.isFile()) { + LOG.info("Running global init file: " + hivercFile); + int rc = processor.processFile(hivercFile.getAbsolutePath()); + if (rc != 0) { + LOG.error("Failed on initializing global .hiverc file"); + } + } else { + LOG.debug("Global init file " + hivercFile + " does not exist"); + } + } + } catch (IOException e) { + LOG.warn("Failed on initializing global .hiverc file", e); + } + } + + private void configureSession(Map sessionConfMap) throws HiveSQLException { + SessionState.setCurrentSessionState(sessionState); + for (Map.Entry entry : sessionConfMap.entrySet()) { + String key = entry.getKey(); + if (key.startsWith("set:")) { + try { + setVariable(key.substring(4), entry.getValue()); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } else if (key.startsWith("use:")) { + SessionState.get().setCurrentDatabase(entry.getValue()); + } else { + hiveConf.verifyAndSet(key, entry.getValue()); + } + } + } + + // Copy from org.apache.hadoop.hive.ql.processors.SetProcessor, only change: + // setConf(varname, propName, varvalue, true) when varname.startsWith(HIVECONF_PREFIX) + public static int setVariable(String varname, String varvalue) throws Exception { + SessionState ss = SessionState.get(); + VariableSubstitution substitution = new VariableSubstitution(() -> ss.getHiveVariables()); + if (varvalue.contains("\n")){ + ss.err.println("Warning: Value had a \\n character in it."); + } + varname = varname.trim(); + if (varname.startsWith(ENV_PREFIX)){ + ss.err.println("env:* variables can not be set."); + return 1; + } else if (varname.startsWith(SYSTEM_PREFIX)){ + String propName = varname.substring(SYSTEM_PREFIX.length()); + System.getProperties().setProperty(propName, substitution.substitute(ss.getConf(),varvalue)); + } else if (varname.startsWith(HIVECONF_PREFIX)){ + String propName = varname.substring(HIVECONF_PREFIX.length()); + setConf(varname, propName, varvalue, true); + } else if (varname.startsWith(HIVEVAR_PREFIX)) { + String propName = varname.substring(HIVEVAR_PREFIX.length()); + ss.getHiveVariables().put(propName, substitution.substitute(ss.getConf(),varvalue)); + } else if (varname.startsWith(METACONF_PREFIX)) { + String propName = varname.substring(METACONF_PREFIX.length()); + Hive hive = Hive.get(ss.getConf()); + hive.setMetaConf(propName, substitution.substitute(ss.getConf(), varvalue)); + } else { + setConf(varname, varname, varvalue, true); + } + return 0; + } + + // returns non-null string for validation fail + private static void setConf(String varname, String key, String varvalue, boolean register) + throws IllegalArgumentException { + VariableSubstitution substitution = + new VariableSubstitution(() -> SessionState.get().getHiveVariables()); + HiveConf conf = SessionState.get().getConf(); + String value = substitution.substitute(conf, varvalue); + if (conf.getBoolVar(HiveConf.ConfVars.HIVECONFVALIDATION)) { + HiveConf.ConfVars confVars = HiveConf.getConfVars(key); + if (confVars != null) { + if (!confVars.isType(value)) { + StringBuilder message = new StringBuilder(); + message.append("'SET ").append(varname).append('=').append(varvalue); + message.append("' FAILED because ").append(key).append(" expects "); + message.append(confVars.typeString()).append(" type value."); + throw new IllegalArgumentException(message.toString()); + } + String fail = confVars.validate(value); + if (fail != null) { + StringBuilder message = new StringBuilder(); + message.append("'SET ").append(varname).append('=').append(varvalue); + message.append("' FAILED in validation : ").append(fail).append('.'); + throw new IllegalArgumentException(message.toString()); + } + } else if (key.startsWith("hive.")) { + throw new IllegalArgumentException("hive configuration " + key + " does not exists."); + } + } + conf.verifyAndSet(key, value); + if (register) { + SessionState.get().getOverriddenConfigurations().put(key, value); + } + } + + @Override + public void setOperationLogSessionDir(File operationLogRootDir) { + if (!operationLogRootDir.exists()) { + LOG.warn("The operation log root directory is removed, recreating: " + + operationLogRootDir.getAbsolutePath()); + if (!operationLogRootDir.mkdirs()) { + LOG.warn("Unable to create operation log root directory: " + + operationLogRootDir.getAbsolutePath()); + } + } + if (!operationLogRootDir.canWrite()) { + LOG.warn("The operation log root directory is not writable: " + + operationLogRootDir.getAbsolutePath()); + } + sessionLogDir = new File(operationLogRootDir, sessionHandle.getHandleIdentifier().toString()); + isOperationLogEnabled = true; + if (!sessionLogDir.exists()) { + if (!sessionLogDir.mkdir()) { + LOG.warn("Unable to create operation log session directory: " + + sessionLogDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + if (isOperationLogEnabled) { + LOG.info("Operation log session directory is created: " + sessionLogDir.getAbsolutePath()); + } + } + + @Override + public boolean isOperationLogEnabled() { + return isOperationLogEnabled; + } + + @Override + public File getOperationLogSessionDir() { + return sessionLogDir; + } + + @Override + public TProtocolVersion getProtocolVersion() { + return sessionHandle.getProtocolVersion(); + } + + @Override + public SessionManager getSessionManager() { + return sessionManager; + } + + @Override + public void setSessionManager(SessionManager sessionManager) { + this.sessionManager = sessionManager; + } + + private OperationManager getOperationManager() { + return operationManager; + } + + @Override + public void setOperationManager(OperationManager operationManager) { + this.operationManager = operationManager; + } + + protected synchronized void acquire(boolean userAccess) { + // Need to make sure that the this HiveServer2's session's SessionState is + // stored in the thread local for the handler thread. + SessionState.setCurrentSessionState(sessionState); + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + } + + /** + * 1. We'll remove the ThreadLocal SessionState as this thread might now serve + * other requests. + * 2. We'll cache the ThreadLocal RawStore object for this background thread for an orderly cleanup + * when this thread is garbage collected later. + * @see org.apache.hive.service.server.ThreadWithGarbageCleanup#finalize() + */ + protected synchronized void release(boolean userAccess) { + SessionState.detachSession(); + if (ThreadWithGarbageCleanup.currentThread() instanceof ThreadWithGarbageCleanup) { + ThreadWithGarbageCleanup currentThread = + (ThreadWithGarbageCleanup) ThreadWithGarbageCleanup.currentThread(); + currentThread.cacheThreadLocalRawStore(); + } + if (userAccess) { + lastAccessTime = System.currentTimeMillis(); + } + if (opHandleSet.isEmpty()) { + lastIdleTime = System.currentTimeMillis(); + } else { + lastIdleTime = 0; + } + } + + @Override + public SessionHandle getSessionHandle() { + return sessionHandle; + } + + @Override + public String getUsername() { + return username; + } + + @Override + public String getPassword() { + return password; + } + + @Override + public HiveConf getHiveConf() { + hiveConf.setVar(HiveConf.ConfVars.HIVEFETCHOUTPUTSERDE, FETCH_WORK_SERDE_CLASS); + return hiveConf; + } + + @Override + public IMetaStoreClient getMetaStoreClient() throws HiveSQLException { + try { + return Hive.get(getHiveConf()).getMSC(); + } catch (HiveException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } catch (MetaException e) { + throw new HiveSQLException("Failed to get metastore connection", e); + } + } + + @Override + public GetInfoValue getInfo(GetInfoType getInfoType) + throws HiveSQLException { + acquire(true); + try { + switch (getInfoType) { + case CLI_SERVER_NAME: + return new GetInfoValue("Hive"); + case CLI_DBMS_NAME: + return new GetInfoValue("Apache Hive"); + case CLI_DBMS_VER: + return new GetInfoValue(HiveVersionInfo.getVersion()); + case CLI_MAX_COLUMN_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_SCHEMA_NAME_LEN: + return new GetInfoValue(128); + case CLI_MAX_TABLE_NAME_LEN: + return new GetInfoValue(128); + case CLI_TXN_CAPABLE: + default: + throw new HiveSQLException("Unrecognized GetInfoType value: " + getInfoType.toString()); + } + } finally { + release(true); + } + } + + @Override + public OperationHandle executeStatement(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, false, 0); + } + + @Override + public OperationHandle executeStatement(String statement, Map confOverlay, + long queryTimeout) throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, false, queryTimeout); + } + + @Override + public OperationHandle executeStatementAsync(String statement, Map confOverlay) + throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, true, 0); + } + + @Override + public OperationHandle executeStatementAsync(String statement, Map confOverlay, + long queryTimeout) throws HiveSQLException { + return executeStatementInternal(statement, confOverlay, true, queryTimeout); + } + + private OperationHandle executeStatementInternal(String statement, + Map confOverlay, boolean runAsync, long queryTimeout) throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + ExecuteStatementOperation operation = operationManager + .newExecuteStatementOperation(getSession(), statement, confOverlay, runAsync, queryTimeout); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + // Referring to SQLOperation.java, there is no chance that a HiveSQLException throws and the asyn + // background operation submits to thread pool successfully at the same time. So, Cleanup + // opHandle directly when got HiveSQLException + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTypeInfo() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTypeInfoOperation operation = operationManager.newGetTypeInfoOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getCatalogs() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetCatalogsOperation operation = operationManager.newGetCatalogsOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getSchemas(String catalogName, String schemaName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetSchemasOperation operation = + operationManager.newGetSchemasOperation(getSession(), catalogName, schemaName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTables(String catalogName, String schemaName, String tableName, + List tableTypes) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + MetadataOperation operation = + operationManager.newGetTablesOperation(getSession(), catalogName, schemaName, tableName, tableTypes); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getTableTypes() + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetTableTypesOperation operation = operationManager.newGetTableTypesOperation(getSession()); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getColumns(String catalogName, String schemaName, + String tableName, String columnName) throws HiveSQLException { + acquire(true); + String addedJars = Utilities.getResourceFiles(hiveConf, SessionState.ResourceType.JAR); + if (StringUtils.isNotBlank(addedJars)) { + IMetaStoreClient metastoreClient = getSession().getMetaStoreClient(); + metastoreClient.setHiveAddedJars(addedJars); + } + OperationManager operationManager = getOperationManager(); + GetColumnsOperation operation = operationManager.newGetColumnsOperation(getSession(), + catalogName, schemaName, tableName, columnName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getFunctions(String catalogName, String schemaName, String functionName) + throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetFunctionsOperation operation = operationManager + .newGetFunctionsOperation(getSession(), catalogName, schemaName, functionName); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + // Iterate through the opHandles and close their operations + for (OperationHandle opHandle : opHandleSet) { + operationManager.closeOperation(opHandle); + } + opHandleSet.clear(); + // Cleanup session log directory. + cleanupSessionLogDir(); + // Cleanup pipeout file. + cleanupPipeoutFile(); + HiveHistory hiveHist = sessionState.getHiveHistory(); + if (null != hiveHist) { + hiveHist.closeStream(); + } + try { + sessionState.close(); + } finally { + sessionState = null; + } + } catch (IOException ioe) { + throw new HiveSQLException("Failure to close", ioe); + } finally { + if (sessionState != null) { + try { + sessionState.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + sessionState = null; + } + release(true); + } + } + + private void cleanupPipeoutFile() { + String lScratchDir = hiveConf.getVar(ConfVars.LOCALSCRATCHDIR); + String sessionID = hiveConf.getVar(ConfVars.HIVESESSIONID); + + File[] fileAry = new File(lScratchDir).listFiles( + (dir, name) -> name.startsWith(sessionID) && name.endsWith(".pipeout")); + + for (File file : fileAry) { + try { + FileUtils.forceDelete(file); + } catch (Exception e) { + LOG.error("Failed to cleanup pipeout file: " + file, e); + } + } + } + + private void cleanupSessionLogDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(sessionLogDir); + } catch (Exception e) { + LOG.error("Failed to cleanup session log dir: " + sessionHandle, e); + } + } + } + + @Override + public SessionState getSessionState() { + return sessionState; + } + + @Override + public String getUserName() { + return username; + } + + @Override + public void setUserName(String userName) { + this.username = userName; + } + + @Override + public long getLastAccessTime() { + return lastAccessTime; + } + + @Override + public void closeExpiredOperations() { + OperationHandle[] handles = opHandleSet.toArray(new OperationHandle[opHandleSet.size()]); + if (handles.length > 0) { + List operations = operationManager.removeExpiredOperations(handles); + if (!operations.isEmpty()) { + closeTimedOutOperations(operations); + } + } + } + + @Override + public long getNoOperationTime() { + return lastIdleTime > 0 ? System.currentTimeMillis() - lastIdleTime : 0; + } + + private void closeTimedOutOperations(List operations) { + acquire(false); + try { + for (Operation operation : operations) { + opHandleSet.remove(operation.getHandle()); + try { + operation.close(); + } catch (Exception e) { + LOG.warn("Exception is thrown closing timed-out operation " + operation.getHandle(), e); + } + } + } finally { + release(false); + } + } + + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + sessionManager.getOperationManager().cancelOperation(opHandle); + } finally { + release(true); + } + } + + @Override + public void closeOperation(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + operationManager.closeOperation(opHandle); + opHandleSet.remove(opHandle); + } finally { + release(true); + } + } + + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) throws HiveSQLException { + acquire(true); + try { + return sessionManager.getOperationManager().getOperationResultSetSchema(opHandle); + } finally { + release(true); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, + long maxRows, FetchType fetchType) throws HiveSQLException { + acquire(true); + try { + if (fetchType == FetchType.QUERY_OUTPUT) { + return operationManager.getOperationNextRowSet(opHandle, orientation, maxRows); + } + return operationManager.getOperationLogRowSet(opHandle, orientation, maxRows); + } finally { + release(true); + } + } + + protected HiveSession getSession() { + return this; + } + + @Override + public String getIpAddress() { + return ipAddress; + } + + @Override + public void setIpAddress(String ipAddress) { + this.ipAddress = ipAddress; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, String renewer) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), owner, getIpAddress(), getHiveConf()); + return authFactory.getDelegationToken(owner, renewer, getIpAddress()); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + HiveAuthFactory.verifyProxyAccess(getUsername(), getUserFromToken(authFactory, tokenStr), + getIpAddress(), getHiveConf()); + authFactory.renewDelegationToken(tokenStr); + } + + // extract the real user from the given token string + private String getUserFromToken(HiveAuthFactory authFactory, String tokenStr) throws HiveSQLException { + return authFactory.getUserFromToken(tokenStr); + } + + @Override + public OperationHandle getPrimaryKeys(String catalog, String schema, + String table) throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetPrimaryKeysOperation operation = operationManager + .newGetPrimaryKeysOperation(getSession(), catalog, schema, table); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } + + @Override + public OperationHandle getCrossReference(String primaryCatalog, + String primarySchema, String primaryTable, String foreignCatalog, + String foreignSchema, String foreignTable) throws HiveSQLException { + acquire(true); + + OperationManager operationManager = getOperationManager(); + GetCrossReferenceOperation operation = operationManager + .newGetCrossReferenceOperation(getSession(), primaryCatalog, + primarySchema, primaryTable, foreignCatalog, + foreignSchema, foreignTable); + OperationHandle opHandle = operation.getHandle(); + try { + operation.run(); + opHandleSet.add(opHandle); + return opHandle; + } catch (HiveSQLException e) { + operationManager.closeOperation(opHandle); + throw e; + } finally { + release(true); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java new file mode 100644 index 0000000000000..bf2cf3c07f6b4 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/HiveSessionImplwithUGI.java @@ -0,0 +1,182 @@ +/** + * 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.hive.service.cli.session; + +import java.io.IOException; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.ql.metadata.Hive; +import org.apache.hadoop.hive.ql.metadata.HiveException; +import org.apache.hadoop.hive.shims.Utils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * HiveSessionImplwithUGI. + * HiveSession with connecting user's UGI and delegation token if required + */ +public class HiveSessionImplwithUGI extends HiveSessionImpl { + public static final String HS2TOKEN = "HiveServer2ImpersonationToken"; + + private UserGroupInformation sessionUgi = null; + private String delegationTokenStr = null; + private Hive sessionHive = null; + private HiveSession proxySession = null; + static final Logger LOG = LoggerFactory.getLogger(HiveSessionImplwithUGI.class); + + public HiveSessionImplwithUGI(TProtocolVersion protocol, String username, String password, + HiveConf hiveConf, String ipAddress, String delegationToken) throws HiveSQLException { + super(protocol, username, password, hiveConf, ipAddress); + setSessionUGI(username); + setDelegationToken(delegationToken); + + // create a new metastore connection for this particular user session + Hive.set(null); + try { + sessionHive = Hive.get(getHiveConf()); + } catch (HiveException e) { + throw new HiveSQLException("Failed to setup metastore connection", e); + } + } + + // setup appropriate UGI for the session + public void setSessionUGI(String owner) throws HiveSQLException { + if (owner == null) { + throw new HiveSQLException("No username provided for impersonation"); + } + if (UserGroupInformation.isSecurityEnabled()) { + try { + sessionUgi = UserGroupInformation.createProxyUser( + owner, UserGroupInformation.getLoginUser()); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup proxy user", e); + } + } else { + sessionUgi = UserGroupInformation.createRemoteUser(owner); + } + } + + public UserGroupInformation getSessionUgi() { + return this.sessionUgi; + } + + public String getDelegationToken() { + return this.delegationTokenStr; + } + + @Override + protected synchronized void acquire(boolean userAccess) { + super.acquire(userAccess); + // if we have a metastore connection with impersonation, then set it first + if (sessionHive != null) { + Hive.set(sessionHive); + } + } + + /** + * Close the file systems for the session and remove it from the FileSystem cache. + * Cancel the session's delegation token and close the metastore connection + */ + @Override + public void close() throws HiveSQLException { + try { + acquire(true); + cancelDelegationToken(); + } finally { + try { + super.close(); + } finally { + try { + FileSystem.closeAllForUGI(sessionUgi); + } catch (IOException ioe) { + throw new HiveSQLException("Could not clean up file-system handles for UGI: " + + sessionUgi, ioe); + } + } + } + } + + /** + * Enable delegation token for the session + * save the token string and set the token.signature in hive conf. The metastore client uses + * this token.signature to determine where to use kerberos or delegation token + * @throws HiveException + * @throws IOException + */ + private void setDelegationToken(String delegationTokenStr) throws HiveSQLException { + this.delegationTokenStr = delegationTokenStr; + if (delegationTokenStr != null) { + getHiveConf().set("hive.metastore.token.signature", HS2TOKEN); + try { + Utils.setTokenStr(sessionUgi, delegationTokenStr, HS2TOKEN); + } catch (IOException e) { + throw new HiveSQLException("Couldn't setup delegation token in the ugi", e); + } + } + } + + // If the session has a delegation token obtained from the metastore, then cancel it + private void cancelDelegationToken() throws HiveSQLException { + if (delegationTokenStr != null) { + try { + Hive.get(getHiveConf()).cancelDelegationToken(delegationTokenStr); + } catch (HiveException e) { + throw new HiveSQLException("Couldn't cancel delegation token", e); + } + // close the metastore connection created with this delegation token + Hive.closeCurrent(); + } + } + + @Override + protected HiveSession getSession() { + assert proxySession != null; + + return proxySession; + } + + public void setProxySession(HiveSession proxySession) { + this.proxySession = proxySession; + } + + @Override + public String getDelegationToken(HiveAuthFactory authFactory, String owner, + String renewer) throws HiveSQLException { + return authFactory.getDelegationToken(owner, renewer, getIpAddress()); + } + + @Override + public void cancelDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.cancelDelegationToken(tokenStr); + } + + @Override + public void renewDelegationToken(HiveAuthFactory authFactory, String tokenStr) + throws HiveSQLException { + authFactory.renewDelegationToken(tokenStr); + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java new file mode 100644 index 0000000000000..49221b13bb892 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/session/SessionManager.java @@ -0,0 +1,361 @@ +/** + * 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.hive.service.cli.session; + +import java.io.File; +import java.io.IOException; +import java.util.ArrayList; +import java.util.Date; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.commons.io.FileUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.SessionHandle; +import org.apache.hive.service.cli.operation.OperationManager; +import org.apache.hive.service.rpc.thrift.TProtocolVersion; +import org.apache.hive.service.server.HiveServer2; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * SessionManager. + * + */ +public class SessionManager extends CompositeService { + + private static final Logger LOG = LoggerFactory.getLogger(SessionManager.class); + public static final String HIVERCFILE = ".hiverc"; + private HiveConf hiveConf; + private final Map handleToSession = + new ConcurrentHashMap(); + private final OperationManager operationManager = new OperationManager(); + private ThreadPoolExecutor backgroundOperationPool; + private boolean isOperationLogEnabled; + private File operationLogRootDir; + + private long checkInterval; + private long sessionTimeout; + private boolean checkOperation; + + private volatile boolean shutdown; + // The HiveServer2 instance running this service + private final HiveServer2 hiveServer2; + + public SessionManager(HiveServer2 hiveServer2) { + super(SessionManager.class.getSimpleName()); + this.hiveServer2 = hiveServer2; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + //Create operation log root directory, if operation logging is enabled + if (hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_ENABLED)) { + initOperationLogRootDir(); + } + createBackgroundOperationPool(); + addService(operationManager); + super.init(hiveConf); + } + + private void createBackgroundOperationPool() { + int poolSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_THREADS); + LOG.info("HiveServer2: Background operation thread pool size: " + poolSize); + int poolQueueSize = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_ASYNC_EXEC_WAIT_QUEUE_SIZE); + LOG.info("HiveServer2: Background operation thread wait queue size: " + poolQueueSize); + long keepAliveTime = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_ASYNC_EXEC_KEEPALIVE_TIME, TimeUnit.SECONDS); + LOG.info( + "HiveServer2: Background operation thread keepalive time: " + keepAliveTime + " seconds"); + + // Create a thread pool with #poolSize threads + // Threads terminate when they are idle for more than the keepAliveTime + // A bounded blocking queue is used to queue incoming operations, if #operations > poolSize + String threadPoolName = "HiveServer2-Background-Pool"; + backgroundOperationPool = new ThreadPoolExecutor(poolSize, poolSize, + keepAliveTime, TimeUnit.SECONDS, new LinkedBlockingQueue(poolQueueSize), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + backgroundOperationPool.allowCoreThreadTimeOut(true); + + checkInterval = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_SESSION_CHECK_INTERVAL, TimeUnit.MILLISECONDS); + sessionTimeout = HiveConf.getTimeVar( + hiveConf, ConfVars.HIVE_SERVER2_IDLE_SESSION_TIMEOUT, TimeUnit.MILLISECONDS); + checkOperation = HiveConf.getBoolVar(hiveConf, + ConfVars.HIVE_SERVER2_IDLE_SESSION_CHECK_OPERATION); + } + + private void initOperationLogRootDir() { + operationLogRootDir = new File( + hiveConf.getVar(ConfVars.HIVE_SERVER2_LOGGING_OPERATION_LOG_LOCATION)); + isOperationLogEnabled = true; + + if (operationLogRootDir.exists() && !operationLogRootDir.isDirectory()) { + LOG.warn("The operation log root directory exists, but it is not a directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + + if (!operationLogRootDir.exists()) { + if (!operationLogRootDir.mkdirs()) { + LOG.warn("Unable to create operation log root directory: " + + operationLogRootDir.getAbsolutePath()); + isOperationLogEnabled = false; + } + } + + if (isOperationLogEnabled) { + LOG.info("Operation log root directory is created: " + operationLogRootDir.getAbsolutePath()); + try { + FileUtils.forceDeleteOnExit(operationLogRootDir); + } catch (IOException e) { + LOG.warn("Failed to schedule cleanup HS2 operation logging root dir: " + + operationLogRootDir.getAbsolutePath(), e); + } + } + } + + @Override + public synchronized void start() { + super.start(); + if (checkInterval > 0) { + startTimeoutChecker(); + } + } + + private void startTimeoutChecker() { + final long interval = Math.max(checkInterval, 3000L); // minimum 3 seconds + Runnable timeoutChecker = new Runnable() { + @Override + public void run() { + for (sleepInterval(interval); !shutdown; sleepInterval(interval)) { + long current = System.currentTimeMillis(); + for (HiveSession session : new ArrayList(handleToSession.values())) { + if (sessionTimeout > 0 && session.getLastAccessTime() + sessionTimeout <= current + && (!checkOperation || session.getNoOperationTime() > sessionTimeout)) { + SessionHandle handle = session.getSessionHandle(); + LOG.warn("Session " + handle + " is Timed-out (last access : " + + new Date(session.getLastAccessTime()) + ") and will be closed"); + try { + closeSession(handle); + } catch (HiveSQLException e) { + LOG.warn("Exception is thrown closing session " + handle, e); + } + } else { + session.closeExpiredOperations(); + } + } + } + } + + private void sleepInterval(long interval) { + try { + Thread.sleep(interval); + } catch (InterruptedException e) { + // ignore + } + } + }; + backgroundOperationPool.execute(timeoutChecker); + } + + @Override + public synchronized void stop() { + super.stop(); + shutdown = true; + if (backgroundOperationPool != null) { + backgroundOperationPool.shutdown(); + long timeout = hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT, TimeUnit.SECONDS); + try { + backgroundOperationPool.awaitTermination(timeout, TimeUnit.SECONDS); + } catch (InterruptedException e) { + LOG.warn("HIVE_SERVER2_ASYNC_EXEC_SHUTDOWN_TIMEOUT = " + timeout + + " seconds has been exceeded. RUNNING background operations will be shut down", e); + } + backgroundOperationPool = null; + } + cleanupLoggingRootDir(); + } + + private void cleanupLoggingRootDir() { + if (isOperationLogEnabled) { + try { + FileUtils.forceDelete(operationLogRootDir); + } catch (Exception e) { + LOG.warn("Failed to cleanup root dir of HS2 logging: " + operationLogRootDir + .getAbsolutePath(), e); + } + } + } + + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf) throws HiveSQLException { + return openSession(protocol, username, password, ipAddress, sessionConf, false, null); + } + + /** + * Opens a new session and creates a session handle. + * The username passed to this method is the effective username. + * If withImpersonation is true (==doAs true) we wrap all the calls in HiveSession + * within a UGI.doAs, where UGI corresponds to the effective user. + * + * Please see {@code org.apache.hive.service.cli.thrift.ThriftCLIService.getUserName()} for + * more details. + * + * @param protocol + * @param username + * @param password + * @param ipAddress + * @param sessionConf + * @param withImpersonation + * @param delegationToken + * @return + * @throws HiveSQLException + */ + public SessionHandle openSession(TProtocolVersion protocol, String username, String password, String ipAddress, + Map sessionConf, boolean withImpersonation, String delegationToken) + throws HiveSQLException { + HiveSession session; + // If doAs is set to true for HiveServer2, we will create a proxy object for the session impl. + // Within the proxy object, we wrap the method call in a UserGroupInformation#doAs + if (withImpersonation) { + HiveSessionImplwithUGI sessionWithUGI = new HiveSessionImplwithUGI(protocol, username, password, + hiveConf, ipAddress, delegationToken); + session = HiveSessionProxy.getProxy(sessionWithUGI, sessionWithUGI.getSessionUgi()); + sessionWithUGI.setProxySession(session); + } else { + session = new HiveSessionImpl(protocol, username, password, hiveConf, ipAddress); + } + session.setSessionManager(this); + session.setOperationManager(operationManager); + try { + session.open(sessionConf); + } catch (Exception e) { + try { + session.close(); + } catch (Throwable t) { + LOG.warn("Error closing session", t); + } + session = null; + throw new HiveSQLException("Failed to open new session: " + e, e); + } + if (isOperationLogEnabled) { + session.setOperationLogSessionDir(operationLogRootDir); + } + handleToSession.put(session.getSessionHandle(), session); + return session.getSessionHandle(); + } + + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.remove(sessionHandle); + if (session == null) { + throw new HiveSQLException("Session does not exist!"); + } + session.close(); + } + + public HiveSession getSession(SessionHandle sessionHandle) throws HiveSQLException { + HiveSession session = handleToSession.get(sessionHandle); + if (session == null) { + throw new HiveSQLException("Invalid SessionHandle: " + sessionHandle); + } + return session; + } + + public OperationManager getOperationManager() { + return operationManager; + } + + private static ThreadLocal threadLocalIpAddress = new ThreadLocal() { + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setIpAddress(String ipAddress) { + threadLocalIpAddress.set(ipAddress); + } + + public static void clearIpAddress() { + threadLocalIpAddress.remove(); + } + + public static String getIpAddress() { + return threadLocalIpAddress.get(); + } + + private static ThreadLocal threadLocalUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setUserName(String userName) { + threadLocalUserName.set(userName); + } + + public static void clearUserName() { + threadLocalUserName.remove(); + } + + public static String getUserName() { + return threadLocalUserName.get(); + } + + private static ThreadLocal threadLocalProxyUserName = new ThreadLocal(){ + @Override + protected synchronized String initialValue() { + return null; + } + }; + + public static void setProxyUserName(String userName) { + LOG.debug("setting proxy user name based on query param to: " + userName); + threadLocalProxyUserName.set(userName); + } + + public static String getProxyUserName() { + return threadLocalProxyUserName.get(); + } + + public static void clearProxyUserName() { + threadLocalProxyUserName.remove(); + } + + public Future submitBackgroundOperation(Runnable r) { + return backgroundOperationPool.submit(r); + } + + public int getOpenSessionCount() { + return handleToSession.size(); + } +} + diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java new file mode 100644 index 0000000000000..fc19c65daaf54 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftBinaryCLIService.java @@ -0,0 +1,109 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.common.auth.HiveAuthUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessorFactory; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.server.TThreadPoolServer; +import org.apache.thrift.transport.TServerSocket; +import org.apache.thrift.transport.TTransportFactory; + + +public class ThriftBinaryCLIService extends ThriftCLIService { + + public ThriftBinaryCLIService(CLIService cliService) { + super(cliService, ThriftBinaryCLIService.class.getSimpleName()); + } + + @Override + public void run() { + try { + // Server thread pool + String threadPoolName = "HiveServer2-Handler-Pool"; + ExecutorService executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TTransportFactory transportFactory = hiveAuthFactory.getAuthTransFactory(); + TProcessorFactory processorFactory = hiveAuthFactory.getAuthProcFactory(this); + TServerSocket serverSocket = null; + List sslVersionBlacklist = new ArrayList(); + for (String sslVersion : hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(",")) { + sslVersionBlacklist.add(sslVersion); + } + if (!hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL)) { + serverSocket = HiveAuthUtils.getServerSocket(hiveHost, portNum); + } else { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + serverSocket = HiveAuthUtils.getServerSSLSocket(hiveHost, portNum, keyStorePath, + keyStorePassword, sslVersionBlacklist); + } + + // Server args + int maxMessageSize = hiveConf.getIntVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_MAX_MESSAGE_SIZE); + int requestTimeout = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_TIMEOUT, TimeUnit.SECONDS); + int beBackoffSlotLength = (int) hiveConf.getTimeVar( + HiveConf.ConfVars.HIVE_SERVER2_THRIFT_LOGIN_BEBACKOFF_SLOT_LENGTH, TimeUnit.MILLISECONDS); + TThreadPoolServer.Args sargs = new TThreadPoolServer.Args(serverSocket) + .processorFactory(processorFactory).transportFactory(transportFactory) + .protocolFactory(new TBinaryProtocol.Factory()) + .inputProtocolFactory(new TBinaryProtocol.Factory(true, true, maxMessageSize, maxMessageSize)) + .requestTimeout(requestTimeout).requestTimeoutUnit(TimeUnit.SECONDS) + .beBackoffSlotLength(beBackoffSlotLength).beBackoffSlotLengthUnit(TimeUnit.MILLISECONDS) + .executorService(executorService); + + // TCP Server + server = new TThreadPoolServer(sargs); + server.setServerEventHandler(serverEventHandler); + String msg = "Starting " + ThriftBinaryCLIService.class.getSimpleName() + " on port " + + portNum + " with " + minWorkerThreads + "..." + maxWorkerThreads + " worker threads"; + LOG.info(msg); + server.serve(); + } catch (Throwable t) { + LOG.error( + "Error starting HiveServer2: could not start " + + ThriftBinaryCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java new file mode 100644 index 0000000000000..d41c3b493bb47 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIService.java @@ -0,0 +1,726 @@ +/** + * 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.hive.service.cli.thrift; + +import javax.security.auth.login.LoginException; +import java.io.IOException; +import java.net.InetAddress; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hive.service.AbstractService; +import org.apache.hive.service.ServiceException; +import org.apache.hive.service.ServiceUtils; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.TSetIpAddressProcessor; +import org.apache.hive.service.cli.*; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.hive.service.rpc.thrift.*; +import org.apache.hive.service.server.HiveServer2; +import org.apache.thrift.TException; +import org.apache.thrift.protocol.TProtocol; +import org.apache.thrift.server.ServerContext; +import org.apache.thrift.server.TServer; +import org.apache.thrift.server.TServerEventHandler; +import org.apache.thrift.transport.TTransport; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * ThriftCLIService. + * + */ +public abstract class ThriftCLIService extends AbstractService implements TCLIService.Iface, Runnable { + + public static final Logger LOG = LoggerFactory.getLogger(ThriftCLIService.class.getName()); + + protected CLIService cliService; + private static final TStatus OK_STATUS = new TStatus(TStatusCode.SUCCESS_STATUS); + protected static HiveAuthFactory hiveAuthFactory; + + protected int portNum; + protected InetAddress serverIPAddress; + protected String hiveHost; + protected TServer server; + protected org.eclipse.jetty.server.Server httpServer; + + private boolean isStarted = false; + protected boolean isEmbedded = false; + + protected HiveConf hiveConf; + + protected int minWorkerThreads; + protected int maxWorkerThreads; + protected long workerKeepAliveTime; + + protected TServerEventHandler serverEventHandler; + protected ThreadLocal currentServerContext; + + static class ThriftCLIServerContext implements ServerContext { + private SessionHandle sessionHandle = null; + + public void setSessionHandle(SessionHandle sessionHandle) { + this.sessionHandle = sessionHandle; + } + + public SessionHandle getSessionHandle() { + return sessionHandle; + } + } + + public ThriftCLIService(CLIService service, String serviceName) { + super(serviceName); + this.cliService = service; + currentServerContext = new ThreadLocal(); + serverEventHandler = new TServerEventHandler() { + @Override + public ServerContext createContext( + TProtocol input, TProtocol output) { + return new ThriftCLIServerContext(); + } + + @Override + public void deleteContext(ServerContext serverContext, + TProtocol input, TProtocol output) { + ThriftCLIServerContext context = (ThriftCLIServerContext)serverContext; + SessionHandle sessionHandle = context.getSessionHandle(); + if (sessionHandle != null) { + LOG.info("Session disconnected without closing properly, close it now"); + try { + cliService.closeSession(sessionHandle); + } catch (HiveSQLException e) { + LOG.warn("Failed to close session: " + e, e); + } + } + } + + @Override + public void preServe() { + } + + @Override + public void processContext(ServerContext serverContext, + TTransport input, TTransport output) { + currentServerContext.set(serverContext); + } + }; + } + + @Override + public synchronized void init(HiveConf hiveConf) { + this.hiveConf = hiveConf; + // Initialize common server configs needed in both binary & http modes + String portString; + hiveHost = System.getenv("HIVE_SERVER2_THRIFT_BIND_HOST"); + if (hiveHost == null) { + hiveHost = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_BIND_HOST); + } + try { + if (hiveHost != null && !hiveHost.isEmpty()) { + serverIPAddress = InetAddress.getByName(hiveHost); + } else { + serverIPAddress = InetAddress.getLocalHost(); + } + } catch (UnknownHostException e) { + throw new ServiceException(e); + } + // HTTP mode + if (HiveServer2.isHTTPTransportMode(hiveConf)) { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_WORKER_KEEPALIVE_TIME, + TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_HTTP_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_PORT); + } + } + // Binary mode + else { + workerKeepAliveTime = + hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_WORKER_KEEPALIVE_TIME, TimeUnit.SECONDS); + portString = System.getenv("HIVE_SERVER2_THRIFT_PORT"); + if (portString != null) { + portNum = Integer.valueOf(portString); + } else { + portNum = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_PORT); + } + } + minWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MIN_WORKER_THREADS); + maxWorkerThreads = hiveConf.getIntVar(ConfVars.HIVE_SERVER2_THRIFT_MAX_WORKER_THREADS); + super.init(hiveConf); + } + + @Override + public synchronized void start() { + super.start(); + if (!isStarted && !isEmbedded) { + new Thread(this).start(); + isStarted = true; + } + } + + @Override + public synchronized void stop() { + if (isStarted && !isEmbedded) { + if(server != null) { + server.stop(); + LOG.info("Thrift server has stopped"); + } + if((httpServer != null) && httpServer.isStarted()) { + try { + httpServer.stop(); + LOG.info("Http server has stopped"); + } catch (Exception e) { + LOG.error("Error stopping Http server: ", e); + } + } + isStarted = false; + } + super.stop(); + } + + public int getPortNumber() { + return portNum; + } + + public InetAddress getServerIPAddress() { + return serverIPAddress; + } + + @Override + public TGetDelegationTokenResp GetDelegationToken(TGetDelegationTokenReq req) + throws TException { + TGetDelegationTokenResp resp = new TGetDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + @Override + public TCancelDelegationTokenResp CancelDelegationToken(TCancelDelegationTokenReq req) + throws TException { + TCancelDelegationTokenResp resp = new TCancelDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + @Override + public TRenewDelegationTokenResp RenewDelegationToken(TRenewDelegationTokenReq req) + throws TException { + TRenewDelegationTokenResp resp = new TRenewDelegationTokenResp(); + resp.setStatus(notSupportTokenErrorStatus()); + return resp; + } + + private TStatus notSupportTokenErrorStatus() { + TStatus errorStatus = new TStatus(TStatusCode.ERROR_STATUS); + errorStatus.setErrorMessage("Delegation token is not supported"); + return errorStatus; + } + + @Override + public TOpenSessionResp OpenSession(TOpenSessionReq req) throws TException { + LOG.info("Client protocol version: " + req.getClient_protocol()); + TOpenSessionResp resp = new TOpenSessionResp(); + try { + SessionHandle sessionHandle = getSessionHandle(req, resp); + resp.setSessionHandle(sessionHandle.toTSessionHandle()); + // TODO: set real configuration map + resp.setConfiguration(new HashMap()); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(sessionHandle); + } + } catch (Exception e) { + LOG.warn("Error opening session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + private String getIpAddress() { + String clientIpAddress; + // Http transport mode. + // We set the thread local ip address, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + clientIpAddress = SessionManager.getIpAddress(); + } + else { + // Kerberos + if (isKerberosAuthMode()) { + clientIpAddress = hiveAuthFactory.getIpAddress(); + } + // Except kerberos, NOSASL + else { + clientIpAddress = TSetIpAddressProcessor.getUserIpAddress(); + } + } + LOG.debug("Client's IP Address: " + clientIpAddress); + return clientIpAddress; + } + + /** + * Returns the effective username. + * 1. If hive.server2.allow.user.substitution = false: the username of the connecting user + * 2. If hive.server2.allow.user.substitution = true: the username of the end user, + * that the connecting user is trying to proxy for. + * This includes a check whether the connecting user is allowed to proxy for the end user. + * @param req + * @return + * @throws HiveSQLException + */ + private String getUserName(TOpenSessionReq req) throws HiveSQLException { + String userName = null; + // Kerberos + if (isKerberosAuthMode()) { + userName = hiveAuthFactory.getRemoteUser(); + } + // Except kerberos, NOSASL + if (userName == null) { + userName = TSetIpAddressProcessor.getUserName(); + } + // Http transport mode. + // We set the thread local username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + userName = SessionManager.getUserName(); + } + if (userName == null) { + userName = req.getUsername(); + } + + userName = getShortName(userName); + String effectiveClientUser = getProxyUser(userName, req.getConfiguration(), getIpAddress()); + LOG.debug("Client's username: " + effectiveClientUser); + return effectiveClientUser; + } + + private String getShortName(String userName) { + String ret = null; + if (userName != null) { + int indexOfDomainMatch = ServiceUtils.indexOfDomainMatch(userName); + ret = (indexOfDomainMatch <= 0) ? userName : + userName.substring(0, indexOfDomainMatch); + } + + return ret; + } + + /** + * Create a session handle + * @param req + * @param res + * @return + * @throws HiveSQLException + * @throws LoginException + * @throws IOException + */ + SessionHandle getSessionHandle(TOpenSessionReq req, TOpenSessionResp res) + throws HiveSQLException, LoginException, IOException { + String userName = getUserName(req); + String ipAddress = getIpAddress(); + TProtocolVersion protocol = getMinVersion(CLIService.SERVER_VERSION, + req.getClient_protocol()); + res.setServerProtocolVersion(protocol); + SessionHandle sessionHandle; + if (cliService.getHiveConf().getBoolVar(ConfVars.HIVE_SERVER2_ENABLE_DOAS) && + (userName != null)) { + String delegationTokenStr = getDelegationToken(userName); + sessionHandle = cliService.openSessionWithImpersonation(protocol, userName, + req.getPassword(), ipAddress, req.getConfiguration(), delegationTokenStr); + } else { + sessionHandle = cliService.openSession(protocol, userName, req.getPassword(), + ipAddress, req.getConfiguration()); + } + return sessionHandle; + } + + + private String getDelegationToken(String userName) + throws HiveSQLException, LoginException, IOException { + if (userName == null || !cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString())) { + return null; + } + try { + return cliService.getDelegationTokenFromMetaStore(userName); + } catch (UnsupportedOperationException e) { + // The delegation token is not applicable in the given deployment mode + } + return null; + } + + private TProtocolVersion getMinVersion(TProtocolVersion... versions) { + TProtocolVersion[] values = TProtocolVersion.values(); + int current = values[values.length - 1].getValue(); + for (TProtocolVersion version : versions) { + if (current > version.getValue()) { + current = version.getValue(); + } + } + for (TProtocolVersion version : values) { + if (version.getValue() == current) { + return version; + } + } + throw new IllegalArgumentException("never"); + } + + @Override + public TCloseSessionResp CloseSession(TCloseSessionReq req) throws TException { + TCloseSessionResp resp = new TCloseSessionResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + cliService.closeSession(sessionHandle); + resp.setStatus(OK_STATUS); + ThriftCLIServerContext context = + (ThriftCLIServerContext)currentServerContext.get(); + if (context != null) { + context.setSessionHandle(null); + } + } catch (Exception e) { + LOG.warn("Error closing session: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetInfoResp GetInfo(TGetInfoReq req) throws TException { + TGetInfoResp resp = new TGetInfoResp(); + try { + GetInfoValue getInfoValue = + cliService.getInfo(new SessionHandle(req.getSessionHandle()), + GetInfoType.getGetInfoType(req.getInfoType())); + resp.setInfoValue(getInfoValue.toTGetInfoValue()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TExecuteStatementResp ExecuteStatement(TExecuteStatementReq req) throws TException { + TExecuteStatementResp resp = new TExecuteStatementResp(); + try { + SessionHandle sessionHandle = new SessionHandle(req.getSessionHandle()); + String statement = req.getStatement(); + Map confOverlay = req.getConfOverlay(); + Boolean runAsync = req.isRunAsync(); + long queryTimeout = req.getQueryTimeout(); + OperationHandle operationHandle = runAsync ? + cliService.executeStatementAsync(sessionHandle, statement, confOverlay, queryTimeout) + : cliService.executeStatement(sessionHandle, statement, confOverlay, queryTimeout); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error executing statement: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTypeInfoResp GetTypeInfo(TGetTypeInfoReq req) throws TException { + TGetTypeInfoResp resp = new TGetTypeInfoResp(); + try { + OperationHandle operationHandle = cliService.getTypeInfo(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(operationHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting type info: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetCatalogsResp GetCatalogs(TGetCatalogsReq req) throws TException { + TGetCatalogsResp resp = new TGetCatalogsResp(); + try { + OperationHandle opHandle = cliService.getCatalogs(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting catalogs: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetSchemasResp GetSchemas(TGetSchemasReq req) throws TException { + TGetSchemasResp resp = new TGetSchemasResp(); + try { + OperationHandle opHandle = cliService.getSchemas( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), req.getSchemaName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting schemas: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTablesResp GetTables(TGetTablesReq req) throws TException { + TGetTablesResp resp = new TGetTablesResp(); + try { + OperationHandle opHandle = cliService + .getTables(new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getTableName(), req.getTableTypes()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting tables: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetTableTypesResp GetTableTypes(TGetTableTypesReq req) throws TException { + TGetTableTypesResp resp = new TGetTableTypesResp(); + try { + OperationHandle opHandle = cliService.getTableTypes(new SessionHandle(req.getSessionHandle())); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting table types: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetColumnsResp GetColumns(TGetColumnsReq req) throws TException { + TGetColumnsResp resp = new TGetColumnsResp(); + try { + OperationHandle opHandle = cliService.getColumns( + new SessionHandle(req.getSessionHandle()), + req.getCatalogName(), + req.getSchemaName(), + req.getTableName(), + req.getColumnName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting columns: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetFunctionsResp GetFunctions(TGetFunctionsReq req) throws TException { + TGetFunctionsResp resp = new TGetFunctionsResp(); + try { + OperationHandle opHandle = cliService.getFunctions( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getFunctionName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting functions: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetOperationStatusResp GetOperationStatus(TGetOperationStatusReq req) throws TException { + TGetOperationStatusResp resp = new TGetOperationStatusResp(); + try { + OperationStatus operationStatus = cliService.getOperationStatus( + new OperationHandle(req.getOperationHandle())); + resp.setOperationState(operationStatus.getState().toTOperationState()); + HiveSQLException opException = operationStatus.getOperationException(); + if (opException != null) { + resp.setSqlState(opException.getSQLState()); + resp.setErrorCode(opException.getErrorCode()); + resp.setErrorMessage(opException.getMessage()); + } + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting operation status: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCancelOperationResp CancelOperation(TCancelOperationReq req) throws TException { + TCancelOperationResp resp = new TCancelOperationResp(); + try { + cliService.cancelOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error cancelling operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TCloseOperationResp CloseOperation(TCloseOperationReq req) throws TException { + TCloseOperationResp resp = new TCloseOperationResp(); + try { + cliService.closeOperation(new OperationHandle(req.getOperationHandle())); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error closing operation: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetResultSetMetadataResp GetResultSetMetadata(TGetResultSetMetadataReq req) + throws TException { + TGetResultSetMetadataResp resp = new TGetResultSetMetadataResp(); + try { + TableSchema schema = cliService.getResultSetMetadata(new OperationHandle(req.getOperationHandle())); + resp.setSchema(schema.toTTableSchema()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting result set metadata: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TFetchResultsResp FetchResults(TFetchResultsReq req) throws TException { + TFetchResultsResp resp = new TFetchResultsResp(); + try { + RowSet rowSet = cliService.fetchResults( + new OperationHandle(req.getOperationHandle()), + FetchOrientation.getFetchOrientation(req.getOrientation()), + req.getMaxRows(), + FetchType.getFetchType(req.getFetchType())); + resp.setResults(rowSet.toTRowSet()); + resp.setHasMoreRows(false); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error fetching results: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetPrimaryKeysResp GetPrimaryKeys(TGetPrimaryKeysReq req) + throws TException { + TGetPrimaryKeysResp resp = new TGetPrimaryKeysResp(); + try { + OperationHandle opHandle = cliService.getPrimaryKeys( + new SessionHandle(req.getSessionHandle()), req.getCatalogName(), + req.getSchemaName(), req.getTableName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting functions: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public TGetCrossReferenceResp GetCrossReference(TGetCrossReferenceReq req) + throws TException { + TGetCrossReferenceResp resp = new TGetCrossReferenceResp(); + try { + OperationHandle opHandle = cliService.getCrossReference( + new SessionHandle(req.getSessionHandle()), req.getParentCatalogName(), + req.getParentSchemaName(), req.getParentTableName(), + req.getForeignCatalogName(), req.getForeignSchemaName(), req.getForeignTableName()); + resp.setOperationHandle(opHandle.toTOperationHandle()); + resp.setStatus(OK_STATUS); + } catch (Exception e) { + LOG.warn("Error getting functions: ", e); + resp.setStatus(HiveSQLException.toTStatus(e)); + } + return resp; + } + + @Override + public abstract void run(); + + /** + * If the proxy user name is provided then check privileges to substitute the user. + * @param realUser + * @param sessionConf + * @param ipAddress + * @return + * @throws HiveSQLException + */ + private String getProxyUser(String realUser, Map sessionConf, + String ipAddress) throws HiveSQLException { + String proxyUser = null; + // Http transport mode. + // We set the thread local proxy username, in ThriftHttpServlet. + if (cliService.getHiveConf().getVar( + ConfVars.HIVE_SERVER2_TRANSPORT_MODE).equalsIgnoreCase("http")) { + proxyUser = SessionManager.getProxyUserName(); + LOG.debug("Proxy user from query string: " + proxyUser); + } + + if (proxyUser == null && sessionConf != null && sessionConf.containsKey(HiveAuthFactory.HS2_PROXY_USER)) { + String proxyUserFromThriftBody = sessionConf.get(HiveAuthFactory.HS2_PROXY_USER); + LOG.debug("Proxy user from thrift body: " + proxyUserFromThriftBody); + proxyUser = proxyUserFromThriftBody; + } + + if (proxyUser == null) { + return realUser; + } + + // check whether substitution is allowed + if (!hiveConf.getBoolVar(HiveConf.ConfVars.HIVE_SERVER2_ALLOW_USER_SUBSTITUTION)) { + throw new HiveSQLException("Proxy user substitution is not allowed"); + } + + // If there's no authentication, then directly substitute the user + if (HiveAuthFactory.AuthTypes.NONE.toString() + .equalsIgnoreCase(hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION))) { + return proxyUser; + } + + // Verify proxy user privilege of the realUser for the proxyUser + HiveAuthFactory.verifyProxyAccess(realUser, proxyUser, ipAddress, hiveConf); + LOG.debug("Verified proxy user: " + proxyUser); + return proxyUser; + } + + private boolean isKerberosAuthMode() { + return cliService.getHiveConf().getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION) + .equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java new file mode 100644 index 0000000000000..b13ddf72f77e7 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftCLIServiceClient.java @@ -0,0 +1,493 @@ +/** + * 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.hive.service.cli.thrift; + +import java.util.List; +import java.util.Map; + +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.*; +import org.apache.hive.service.rpc.thrift.*; +import org.apache.thrift.TException; + +/** + * ThriftCLIServiceClient. + * + */ +public class ThriftCLIServiceClient extends CLIServiceClient { + private final TCLIService.Iface cliService; + + public ThriftCLIServiceClient(TCLIService.Iface cliService) { + this.cliService = cliService; + } + + public void checkStatus(TStatus status) throws HiveSQLException { + if (TStatusCode.ERROR_STATUS.equals(status.getStatusCode())) { + throw new HiveSQLException(status); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#openSession(java.lang.String, java.lang.String, java.util.Map) + */ + @Override + public SessionHandle openSession(String username, String password, + Map configuration) + throws HiveSQLException { + try { + TOpenSessionReq req = new TOpenSessionReq(); + req.setUsername(username); + req.setPassword(password); + req.setConfiguration(configuration); + TOpenSessionResp resp = cliService.OpenSession(req); + checkStatus(resp.getStatus()); + return new SessionHandle(resp.getSessionHandle(), resp.getServerProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public SessionHandle openSessionWithImpersonation(String username, String password, + Map configuration, String delegationToken) throws HiveSQLException { + throw new HiveSQLException("open with impersonation operation is not supported in the client"); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeSession(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public void closeSession(SessionHandle sessionHandle) throws HiveSQLException { + try { + TCloseSessionReq req = new TCloseSessionReq(sessionHandle.toTSessionHandle()); + TCloseSessionResp resp = cliService.CloseSession(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getInfo(org.apache.hive.service.cli.SessionHandle, java.util.List) + */ + @Override + public GetInfoValue getInfo(SessionHandle sessionHandle, GetInfoType infoType) + throws HiveSQLException { + try { + // FIXME extract the right info type + TGetInfoReq req = new TGetInfoReq(sessionHandle.toTSessionHandle(), infoType.toTGetInfoType()); + TGetInfoResp resp = cliService.GetInfo(req); + checkStatus(resp.getStatus()); + return new GetInfoValue(resp.getInfoValue()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatement(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, false, 0); + } + + @Override + public OperationHandle executeStatement(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, false, queryTimeout); + } + + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay) throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, true, 0); + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#executeStatementAsync(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.util.Map) + */ + @Override + public OperationHandle executeStatementAsync(SessionHandle sessionHandle, String statement, + Map confOverlay, long queryTimeout) throws HiveSQLException { + return executeStatementInternal(sessionHandle, statement, confOverlay, true, queryTimeout); + } + + private OperationHandle executeStatementInternal(SessionHandle sessionHandle, String statement, + Map confOverlay, boolean isAsync, long queryTimeout) throws HiveSQLException { + try { + TExecuteStatementReq req = + new TExecuteStatementReq(sessionHandle.toTSessionHandle(), statement); + req.setConfOverlay(confOverlay); + req.setRunAsync(isAsync); + req.setQueryTimeout(queryTimeout); + TExecuteStatementResp resp = cliService.ExecuteStatement(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTypeInfo(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTypeInfo(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTypeInfoReq req = new TGetTypeInfoReq(sessionHandle.toTSessionHandle()); + TGetTypeInfoResp resp = cliService.GetTypeInfo(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getCatalogs(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getCatalogs(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetCatalogsReq req = new TGetCatalogsReq(sessionHandle.toTSessionHandle()); + TGetCatalogsResp resp = cliService.GetCatalogs(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getSchemas(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String) + */ + @Override + public OperationHandle getSchemas(SessionHandle sessionHandle, String catalogName, + String schemaName) + throws HiveSQLException { + try { + TGetSchemasReq req = new TGetSchemasReq(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetSchemasResp resp = cliService.GetSchemas(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTables(org.apache.hive.service.cli.SessionHandle, java.lang.String, java.lang.String, java.lang.String, java.util.List) + */ + @Override + public OperationHandle getTables(SessionHandle sessionHandle, String catalogName, + String schemaName, String tableName, List tableTypes) + throws HiveSQLException { + try { + TGetTablesReq req = new TGetTablesReq(sessionHandle.toTSessionHandle()); + req.setTableName(tableName); + req.setTableTypes(tableTypes); + req.setSchemaName(schemaName); + TGetTablesResp resp = cliService.GetTables(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getTableTypes(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getTableTypes(SessionHandle sessionHandle) throws HiveSQLException { + try { + TGetTableTypesReq req = new TGetTableTypesReq(sessionHandle.toTSessionHandle()); + TGetTableTypesResp resp = cliService.GetTableTypes(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getColumns(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getColumns(SessionHandle sessionHandle, + String catalogName, String schemaName, String tableName, String columnName) + throws HiveSQLException { + try { + TGetColumnsReq req = new TGetColumnsReq(); + req.setSessionHandle(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + req.setTableName(tableName); + req.setColumnName(columnName); + TGetColumnsResp resp = cliService.GetColumns(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getFunctions(org.apache.hive.service.cli.SessionHandle) + */ + @Override + public OperationHandle getFunctions(SessionHandle sessionHandle, + String catalogName, String schemaName, String functionName) throws HiveSQLException { + try { + TGetFunctionsReq req = new TGetFunctionsReq(sessionHandle.toTSessionHandle(), functionName); + req.setCatalogName(catalogName); + req.setSchemaName(schemaName); + TGetFunctionsResp resp = cliService.GetFunctions(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getOperationStatus(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public OperationStatus getOperationStatus(OperationHandle opHandle) throws HiveSQLException { + try { + TGetOperationStatusReq req = new TGetOperationStatusReq(opHandle.toTOperationHandle()); + TGetOperationStatusResp resp = cliService.GetOperationStatus(req); + // Checks the status of the RPC call, throws an exception in case of error + checkStatus(resp.getStatus()); + OperationState opState = OperationState.getOperationState(resp.getOperationState()); + HiveSQLException opException = null; + if (opState == OperationState.ERROR) { + opException = new HiveSQLException(resp.getErrorMessage(), resp.getSqlState(), resp.getErrorCode()); + } + return new OperationStatus(opState, opException); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#cancelOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void cancelOperation(OperationHandle opHandle) throws HiveSQLException { + try { + TCancelOperationReq req = new TCancelOperationReq(opHandle.toTOperationHandle()); + TCancelOperationResp resp = cliService.CancelOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#closeOperation(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public void closeOperation(OperationHandle opHandle) + throws HiveSQLException { + try { + TCloseOperationReq req = new TCloseOperationReq(opHandle.toTOperationHandle()); + TCloseOperationResp resp = cliService.CloseOperation(req); + checkStatus(resp.getStatus()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#getResultSetMetadata(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public TableSchema getResultSetMetadata(OperationHandle opHandle) + throws HiveSQLException { + try { + TGetResultSetMetadataReq req = new TGetResultSetMetadataReq(opHandle.toTOperationHandle()); + TGetResultSetMetadataResp resp = cliService.GetResultSetMetadata(req); + checkStatus(resp.getStatus()); + return new TableSchema(resp.getSchema()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public RowSet fetchResults(OperationHandle opHandle, FetchOrientation orientation, long maxRows, + FetchType fetchType) throws HiveSQLException { + try { + TFetchResultsReq req = new TFetchResultsReq(); + req.setOperationHandle(opHandle.toTOperationHandle()); + req.setOrientation(orientation.toTFetchOrientation()); + req.setMaxRows(maxRows); + req.setFetchType(fetchType.toTFetchType()); + TFetchResultsResp resp = cliService.FetchResults(req); + checkStatus(resp.getStatus()); + return RowSetFactory.create(resp.getResults(), opHandle.getProtocolVersion()); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + /* (non-Javadoc) + * @see org.apache.hive.service.cli.ICLIService#fetchResults(org.apache.hive.service.cli.OperationHandle) + */ + @Override + public RowSet fetchResults(OperationHandle opHandle) throws HiveSQLException { + // TODO: set the correct default fetch size + return fetchResults(opHandle, FetchOrientation.FETCH_NEXT, 10000, FetchType.QUERY_OUTPUT); + } + + @Override + public String getDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String owner, String renewer) throws HiveSQLException { + TGetDelegationTokenReq req = new TGetDelegationTokenReq( + sessionHandle.toTSessionHandle(), owner, renewer); + try { + TGetDelegationTokenResp tokenResp = cliService.GetDelegationToken(req); + checkStatus(tokenResp.getStatus()); + return tokenResp.getDelegationToken(); + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public void cancelDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TCancelDelegationTokenReq cancelReq = new TCancelDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TCancelDelegationTokenResp cancelResp = + cliService.CancelDelegationToken(cancelReq); + checkStatus(cancelResp.getStatus()); + return; + } catch (TException e) { + throw new HiveSQLException(e); + } + } + + @Override + public void renewDelegationToken(SessionHandle sessionHandle, HiveAuthFactory authFactory, + String tokenStr) throws HiveSQLException { + TRenewDelegationTokenReq cancelReq = new TRenewDelegationTokenReq( + sessionHandle.toTSessionHandle(), tokenStr); + try { + TRenewDelegationTokenResp renewResp = + cliService.RenewDelegationToken(cancelReq); + checkStatus(renewResp.getStatus()); + return; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public OperationHandle getPrimaryKeys(SessionHandle sessionHandle, + String catalog, String schema, String table) throws HiveSQLException { + try { + TGetPrimaryKeysReq req = new TGetPrimaryKeysReq(sessionHandle.toTSessionHandle()); + req.setCatalogName(catalog); + req.setSchemaName(schema); + req.setTableName(table); + TGetPrimaryKeysResp resp = cliService.GetPrimaryKeys(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } + + @Override + public OperationHandle getCrossReference(SessionHandle sessionHandle, + String primaryCatalog, String primarySchema, String primaryTable, + String foreignCatalog, String foreignSchema, String foreignTable) throws HiveSQLException { + try { + TGetCrossReferenceReq req = new TGetCrossReferenceReq(sessionHandle.toTSessionHandle()); + req.setParentCatalogName(primaryCatalog); + req.setParentSchemaName(primarySchema); + req.setParentTableName(primaryTable); + req.setForeignCatalogName(foreignCatalog); + req.setForeignSchemaName(foreignSchema); + req.setForeignTableName(foreignTable); + TGetCrossReferenceResp resp = cliService.GetCrossReference(req); + checkStatus(resp.getStatus()); + TProtocolVersion protocol = sessionHandle.getProtocolVersion(); + return new OperationHandle(resp.getOperationHandle(), protocol); + } catch (HiveSQLException e) { + throw e; + } catch (Exception e) { + throw new HiveSQLException(e); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java new file mode 100644 index 0000000000000..08626e7eb146d --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpCLIService.java @@ -0,0 +1,183 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hive.service.cli.thrift; + +import java.util.Arrays; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Shell; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.rpc.thrift.TCLIService; +import org.apache.hive.service.rpc.thrift.TCLIService.Iface; +import org.apache.hive.service.server.ThreadFactoryWithGarbageCleanup; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TBinaryProtocol; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.eclipse.jetty.server.AbstractConnectionFactory; +import org.eclipse.jetty.server.ConnectionFactory; +import org.eclipse.jetty.server.HttpConnectionFactory; +import org.eclipse.jetty.server.ServerConnector; +import org.eclipse.jetty.servlet.ServletContextHandler; +import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.util.ssl.SslContextFactory; +import org.eclipse.jetty.util.thread.ExecutorThreadPool; +import org.eclipse.jetty.util.thread.ScheduledExecutorScheduler; + + +public class ThriftHttpCLIService extends ThriftCLIService { + + public ThriftHttpCLIService(CLIService cliService) { + super(cliService, ThriftHttpCLIService.class.getSimpleName()); + } + + /** + * Configure Jetty to serve http requests. Example of a client connection URL: + * http://localhost:10000/servlets/thrifths2/ A gateway may cause actual target URL to differ, + * e.g. http://gateway:port/hive2/servlets/thrifths2/ + */ + @Override + public void run() { + try { + // Server thread pool + // Start with minWorkerThreads, expand till maxWorkerThreads and reject subsequent requests + String threadPoolName = "HiveServer2-HttpHandler-Pool"; + ThreadPoolExecutor executorService = new ThreadPoolExecutor(minWorkerThreads, maxWorkerThreads, + workerKeepAliveTime, TimeUnit.SECONDS, new SynchronousQueue(), + new ThreadFactoryWithGarbageCleanup(threadPoolName)); + ExecutorThreadPool threadPool = new ExecutorThreadPool(executorService); + + // HTTP Server + httpServer = new org.eclipse.jetty.server.Server(threadPool); + + // Connector configs + + ConnectionFactory[] connectionFactories; + boolean useSsl = hiveConf.getBoolVar(ConfVars.HIVE_SERVER2_USE_SSL); + String schemeName = useSsl ? "https" : "http"; + // Change connector if SSL is used + if (useSsl) { + String keyStorePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH).trim(); + String keyStorePassword = ShimLoader.getHadoopShims().getPassword(hiveConf, + HiveConf.ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PASSWORD.varname); + if (keyStorePath.isEmpty()) { + throw new IllegalArgumentException(ConfVars.HIVE_SERVER2_SSL_KEYSTORE_PATH.varname + + " Not configured for SSL connection"); + } + SslContextFactory sslContextFactory = new SslContextFactory.Server(); + String[] excludedProtocols = hiveConf.getVar(ConfVars.HIVE_SSL_PROTOCOL_BLACKLIST).split(","); + LOG.info("HTTP Server SSL: adding excluded protocols: " + Arrays.toString(excludedProtocols)); + sslContextFactory.addExcludeProtocols(excludedProtocols); + LOG.info("HTTP Server SSL: SslContextFactory.getExcludeProtocols = " + + Arrays.toString(sslContextFactory.getExcludeProtocols())); + sslContextFactory.setKeyStorePath(keyStorePath); + sslContextFactory.setKeyStorePassword(keyStorePassword); + connectionFactories = AbstractConnectionFactory.getFactories( + sslContextFactory, new HttpConnectionFactory()); + } else { + connectionFactories = new ConnectionFactory[] { new HttpConnectionFactory() }; + } + ServerConnector connector = new ServerConnector( + httpServer, + null, + // Call this full constructor to set this, which forces daemon threads: + new ScheduledExecutorScheduler("HiveServer2-HttpHandler-JettyScheduler", true), + null, + -1, + -1, + connectionFactories); + + connector.setPort(portNum); + // Linux:yes, Windows:no + connector.setReuseAddress(!Shell.WINDOWS); + int maxIdleTime = (int) hiveConf.getTimeVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_MAX_IDLE_TIME, + TimeUnit.MILLISECONDS); + connector.setIdleTimeout(maxIdleTime); + + httpServer.addConnector(connector); + + // Thrift configs + hiveAuthFactory = new HiveAuthFactory(hiveConf); + TProcessor processor = new TCLIService.Processor(this); + TProtocolFactory protocolFactory = new TBinaryProtocol.Factory(); + // Set during the init phase of HiveServer2 if auth mode is kerberos + // UGI for the hive/_HOST (kerberos) principal + UserGroupInformation serviceUGI = cliService.getServiceUGI(); + // UGI for the http/_HOST (SPNego) principal + UserGroupInformation httpUGI = cliService.getHttpUGI(); + String authType = hiveConf.getVar(ConfVars.HIVE_SERVER2_AUTHENTICATION); + TServlet thriftHttpServlet = new ThriftHttpServlet(processor, protocolFactory, authType, + serviceUGI, httpUGI, hiveAuthFactory); + + // Context handler + final ServletContextHandler context = new ServletContextHandler( + ServletContextHandler.SESSIONS); + context.setContextPath("/"); + String httpPath = getHttpPath(hiveConf + .getVar(HiveConf.ConfVars.HIVE_SERVER2_THRIFT_HTTP_PATH)); + httpServer.setHandler(context); + context.addServlet(new ServletHolder(thriftHttpServlet), httpPath); + + // TODO: check defaults: maxTimeout, keepalive, maxBodySize, bodyRecieveDuration, etc. + // Finally, start the server + httpServer.start(); + String msg = "Started " + ThriftHttpCLIService.class.getSimpleName() + " in " + schemeName + + " mode on port " + portNum + " path=" + httpPath + " with " + minWorkerThreads + "..." + + maxWorkerThreads + " worker threads"; + LOG.info(msg); + httpServer.join(); + } catch (Throwable t) { + LOG.error( + "Error starting HiveServer2: could not start " + + ThriftHttpCLIService.class.getSimpleName(), t); + System.exit(-1); + } + } + + /** + * The config parameter can be like "path", "/path", "/path/", "path/*", "/path1/path2/*" and so on. + * httpPath should end up as "/*", "/path/*" or "/path1/../pathN/*" + * @param httpPath + * @return + */ + private String getHttpPath(String httpPath) { + if(httpPath == null || httpPath.equals("")) { + httpPath = "/*"; + } + else { + if(!httpPath.startsWith("/")) { + httpPath = "/" + httpPath; + } + if(httpPath.endsWith("/")) { + httpPath = httpPath + "*"; + } + if(!httpPath.endsWith("/*")) { + httpPath = httpPath + "/*"; + } + } + return httpPath; + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java new file mode 100644 index 0000000000000..0cfa84d0971b3 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/cli/thrift/ThriftHttpServlet.java @@ -0,0 +1,566 @@ +/** + * 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.hive.service.cli.thrift; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.security.PrivilegedExceptionAction; +import java.util.Map; +import java.util.Random; +import java.util.Set; +import java.util.concurrent.TimeUnit; + +import javax.servlet.ServletException; +import javax.servlet.http.Cookie; +import javax.servlet.http.HttpServletRequest; +import javax.servlet.http.HttpServletResponse; +import javax.ws.rs.core.NewCookie; + +import org.apache.commons.codec.binary.Base64; +import org.apache.commons.codec.binary.StringUtils; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hadoop.hive.conf.HiveConf.ConfVars; +import org.apache.hadoop.hive.shims.HadoopShims.KerberosNameShim; +import org.apache.hadoop.hive.shims.ShimLoader; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hive.service.CookieSigner; +import org.apache.hive.service.auth.AuthenticationProviderFactory; +import org.apache.hive.service.auth.AuthenticationProviderFactory.AuthMethods; +import org.apache.hive.service.auth.HiveAuthFactory; +import org.apache.hive.service.auth.HttpAuthUtils; +import org.apache.hive.service.auth.HttpAuthenticationException; +import org.apache.hive.service.auth.PasswdAuthenticationProvider; +import org.apache.hive.service.cli.HiveSQLException; +import org.apache.hive.service.cli.session.SessionManager; +import org.apache.thrift.TProcessor; +import org.apache.thrift.protocol.TProtocolFactory; +import org.apache.thrift.server.TServlet; +import org.ietf.jgss.GSSContext; +import org.ietf.jgss.GSSCredential; +import org.ietf.jgss.GSSException; +import org.ietf.jgss.GSSManager; +import org.ietf.jgss.GSSName; +import org.ietf.jgss.Oid; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * + * ThriftHttpServlet + * + */ +public class ThriftHttpServlet extends TServlet { + + private static final long serialVersionUID = 1L; + public static final Logger LOG = LoggerFactory.getLogger(ThriftHttpServlet.class.getName()); + private final String authType; + private final UserGroupInformation serviceUGI; + private final UserGroupInformation httpUGI; + private HiveConf hiveConf = new HiveConf(); + + // Class members for cookie based authentication. + private CookieSigner signer; + public static final String AUTH_COOKIE = "hive.server2.auth"; + private static final Random RAN = new Random(); + private boolean isCookieAuthEnabled; + private String cookieDomain; + private String cookiePath; + private int cookieMaxAge; + private boolean isCookieSecure; + private boolean isHttpOnlyCookie; + private final HiveAuthFactory hiveAuthFactory; + private static final String HIVE_DELEGATION_TOKEN_HEADER = "X-Hive-Delegation-Token"; + + public ThriftHttpServlet(TProcessor processor, TProtocolFactory protocolFactory, + String authType, UserGroupInformation serviceUGI, UserGroupInformation httpUGI, + HiveAuthFactory hiveAuthFactory) { + super(processor, protocolFactory); + this.authType = authType; + this.serviceUGI = serviceUGI; + this.httpUGI = httpUGI; + this.hiveAuthFactory = hiveAuthFactory; + this.isCookieAuthEnabled = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_AUTH_ENABLED); + // Initialize the cookie based authentication related variables. + if (isCookieAuthEnabled) { + // Generate the signer with secret. + String secret = Long.toString(RAN.nextLong()); + LOG.debug("Using the random number as the secret for cookie generation " + secret); + this.signer = new CookieSigner(secret.getBytes()); + this.cookieMaxAge = (int) hiveConf.getTimeVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_MAX_AGE, TimeUnit.SECONDS); + this.cookieDomain = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_DOMAIN); + this.cookiePath = hiveConf.getVar(ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_PATH); + this.isCookieSecure = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_SECURE); + this.isHttpOnlyCookie = hiveConf.getBoolVar( + ConfVars.HIVE_SERVER2_THRIFT_HTTP_COOKIE_IS_HTTPONLY); + } + } + + @Override + protected void doPost(HttpServletRequest request, HttpServletResponse response) + throws ServletException, IOException { + String clientUserName = null; + String clientIpAddress; + boolean requireNewCookie = false; + + try { + // If the cookie based authentication is already enabled, parse the + // request and validate the request cookies. + if (isCookieAuthEnabled) { + clientUserName = validateCookie(request); + requireNewCookie = (clientUserName == null); + if (requireNewCookie) { + LOG.info("Could not validate cookie sent, will try to generate a new cookie"); + } + } + // If the cookie based authentication is not enabled or the request does + // not have a valid cookie, use the kerberos or password based authentication + // depending on the server setup. + if (clientUserName == null) { + // For a kerberos setup + if (isKerberosAuthMode(authType)) { + String delegationToken = request.getHeader(HIVE_DELEGATION_TOKEN_HEADER); + // Each http request must have an Authorization header + if ((delegationToken != null) && (!delegationToken.isEmpty())) { + clientUserName = doTokenAuth(request, response); + } else { + clientUserName = doKerberosAuth(request); + } + } + // For password based authentication + else { + clientUserName = doPasswdAuth(request, authType); + } + } + LOG.debug("Client username: " + clientUserName); + + // Set the thread local username to be used for doAs if true + SessionManager.setUserName(clientUserName); + + // find proxy user if any from query param + String doAsQueryParam = getDoAsQueryParam(request.getQueryString()); + if (doAsQueryParam != null) { + SessionManager.setProxyUserName(doAsQueryParam); + } + + clientIpAddress = request.getRemoteAddr(); + LOG.debug("Client IP Address: " + clientIpAddress); + // Set the thread local ip address + SessionManager.setIpAddress(clientIpAddress); + // Generate new cookie and add it to the response + if (requireNewCookie && + !authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + String cookieToken = HttpAuthUtils.createCookieToken(clientUserName); + Cookie hs2Cookie = createCookie(signer.signCookie(cookieToken)); + + if (isHttpOnlyCookie) { + response.setHeader("SET-COOKIE", getHttpOnlyCookieHeader(hs2Cookie)); + } else { + response.addCookie(hs2Cookie); + } + LOG.info("Cookie added for clientUserName " + clientUserName); + } + super.doPost(request, response); + } + catch (HttpAuthenticationException e) { + LOG.error("Error: ", e); + // Send a 401 to the client + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED); + if(isKerberosAuthMode(authType)) { + response.addHeader(HttpAuthUtils.WWW_AUTHENTICATE, HttpAuthUtils.NEGOTIATE); + } + response.getWriter().println("Authentication Error: " + e.getMessage()); + } + finally { + // Clear the thread locals + SessionManager.clearUserName(); + SessionManager.clearIpAddress(); + SessionManager.clearProxyUserName(); + } + } + + /** + * Retrieves the client name from cookieString. If the cookie does not + * correspond to a valid client, the function returns null. + * @param cookies HTTP Request cookies. + * @return Client Username if cookieString has a HS2 Generated cookie that is currently valid. + * Else, returns null. + */ + private String getClientNameFromCookie(Cookie[] cookies) { + // Current Cookie Name, Current Cookie Value + String currName, currValue; + + // Following is the main loop which iterates through all the cookies send by the client. + // The HS2 generated cookies are of the format hive.server2.auth= + // A cookie which is identified as a hiveserver2 generated cookie is validated + // by calling signer.verifyAndExtract(). If the validation passes, send the + // username for which the cookie is validated to the caller. If no client side + // cookie passes the validation, return null to the caller. + for (Cookie currCookie : cookies) { + // Get the cookie name + currName = currCookie.getName(); + if (!currName.equals(AUTH_COOKIE)) { + // Not a HS2 generated cookie, continue. + continue; + } + // If we reached here, we have match for HS2 generated cookie + currValue = currCookie.getValue(); + // Validate the value. + currValue = signer.verifyAndExtract(currValue); + // Retrieve the user name, do the final validation step. + if (currValue != null) { + String userName = HttpAuthUtils.getUserNameFromCookieToken(currValue); + + if (userName == null) { + LOG.warn("Invalid cookie token " + currValue); + continue; + } + //We have found a valid cookie in the client request. + if (LOG.isDebugEnabled()) { + LOG.debug("Validated the cookie for user " + userName); + } + return userName; + } + } + // No valid HS2 generated cookies found, return null + return null; + } + + /** + * Convert cookie array to human readable cookie string + * @param cookies Cookie Array + * @return String containing all the cookies separated by a newline character. + * Each cookie is of the format [key]=[value] + */ + private String toCookieStr(Cookie[] cookies) { + String cookieStr = ""; + + for (Cookie c : cookies) { + cookieStr += c.getName() + "=" + c.getValue() + " ;\n"; + } + return cookieStr; + } + + /** + * Validate the request cookie. This function iterates over the request cookie headers + * and finds a cookie that represents a valid client/server session. If it finds one, it + * returns the client name associated with the session. Else, it returns null. + * @param request The HTTP Servlet Request send by the client + * @return Client Username if the request has valid HS2 cookie, else returns null + * @throws UnsupportedEncodingException + */ + private String validateCookie(HttpServletRequest request) throws UnsupportedEncodingException { + // Find all the valid cookies associated with the request. + Cookie[] cookies = request.getCookies(); + + if (cookies == null) { + if (LOG.isDebugEnabled()) { + LOG.debug("No valid cookies associated with the request " + request); + } + return null; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Received cookies: " + toCookieStr(cookies)); + } + return getClientNameFromCookie(cookies); + } + + /** + * Generate a server side cookie given the cookie value as the input. + * @param str Input string token. + * @return The generated cookie. + * @throws UnsupportedEncodingException + */ + private Cookie createCookie(String str) throws UnsupportedEncodingException { + if (LOG.isDebugEnabled()) { + LOG.debug("Cookie name = " + AUTH_COOKIE + " value = " + str); + } + Cookie cookie = new Cookie(AUTH_COOKIE, str); + + cookie.setMaxAge(cookieMaxAge); + if (cookieDomain != null) { + cookie.setDomain(cookieDomain); + } + if (cookiePath != null) { + cookie.setPath(cookiePath); + } + cookie.setSecure(isCookieSecure); + return cookie; + } + + /** + * Generate httponly cookie from HS2 cookie + * @param cookie HS2 generated cookie + * @return The httponly cookie + */ + private static String getHttpOnlyCookieHeader(Cookie cookie) { + NewCookie newCookie = new NewCookie(cookie.getName(), cookie.getValue(), + cookie.getPath(), cookie.getDomain(), cookie.getVersion(), + cookie.getComment(), cookie.getMaxAge(), cookie.getSecure()); + return newCookie + "; HttpOnly"; + } + + /** + * Do the LDAP/PAM authentication + * @param request + * @param authType + * @throws HttpAuthenticationException + */ + private String doPasswdAuth(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String userName = getUsername(request, authType); + // No-op when authType is NOSASL + if (!authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.NOSASL.toString())) { + try { + AuthMethods authMethod = AuthMethods.getValidAuthMethod(authType); + PasswdAuthenticationProvider provider = + AuthenticationProviderFactory.getAuthenticationProvider(authMethod); + provider.Authenticate(userName, getPassword(request, authType)); + + } catch (Exception e) { + throw new HttpAuthenticationException(e); + } + } + return userName; + } + + private String doTokenAuth(HttpServletRequest request, HttpServletResponse response) + throws HttpAuthenticationException { + String tokenStr = request.getHeader(HIVE_DELEGATION_TOKEN_HEADER); + try { + return hiveAuthFactory.verifyDelegationToken(tokenStr); + } catch (HiveSQLException e) { + throw new HttpAuthenticationException(e); + } + } + + /** + * Do the GSS-API kerberos authentication. + * We already have a logged in subject in the form of serviceUGI, + * which GSS-API will extract information from. + * In case of a SPNego request we use the httpUGI, + * for the authenticating service tickets. + * @param request + * @return + * @throws HttpAuthenticationException + */ + private String doKerberosAuth(HttpServletRequest request) + throws HttpAuthenticationException { + // Try authenticating with the http/_HOST principal + if (httpUGI != null) { + try { + return httpUGI.doAs(new HttpKerberosServerAction(request, httpUGI)); + } catch (Exception e) { + LOG.info("Failed to authenticate with http/_HOST kerberos principal, " + + "trying with hive/_HOST kerberos principal"); + } + } + // Now try with hive/_HOST principal + try { + return serviceUGI.doAs(new HttpKerberosServerAction(request, serviceUGI)); + } catch (Exception e) { + LOG.error("Failed to authenticate with hive/_HOST kerberos principal"); + throw new HttpAuthenticationException(e); + } + + } + + class HttpKerberosServerAction implements PrivilegedExceptionAction { + HttpServletRequest request; + UserGroupInformation serviceUGI; + + HttpKerberosServerAction(HttpServletRequest request, + UserGroupInformation serviceUGI) { + this.request = request; + this.serviceUGI = serviceUGI; + } + + @Override + public String run() throws HttpAuthenticationException { + // Get own Kerberos credentials for accepting connection + GSSManager manager = GSSManager.getInstance(); + GSSContext gssContext = null; + String serverPrincipal = getPrincipalWithoutRealm( + serviceUGI.getUserName()); + try { + // This Oid for Kerberos GSS-API mechanism. + Oid kerberosMechOid = new Oid("1.2.840.113554.1.2.2"); + // Oid for SPNego GSS-API mechanism. + Oid spnegoMechOid = new Oid("1.3.6.1.5.5.2"); + // Oid for kerberos principal name + Oid krb5PrincipalOid = new Oid("1.2.840.113554.1.2.2.1"); + + // GSS name for server + GSSName serverName = manager.createName(serverPrincipal, krb5PrincipalOid); + + // GSS credentials for server + GSSCredential serverCreds = manager.createCredential(serverName, + GSSCredential.DEFAULT_LIFETIME, + new Oid[]{kerberosMechOid, spnegoMechOid}, + GSSCredential.ACCEPT_ONLY); + + // Create a GSS context + gssContext = manager.createContext(serverCreds); + // Get service ticket from the authorization header + String serviceTicketBase64 = getAuthHeader(request, authType); + byte[] inToken = Base64.decodeBase64(serviceTicketBase64.getBytes()); + gssContext.acceptSecContext(inToken, 0, inToken.length); + // Authenticate or deny based on its context completion + if (!gssContext.isEstablished()) { + throw new HttpAuthenticationException("Kerberos authentication failed: " + + "unable to establish context with the service ticket " + + "provided by the client."); + } + else { + return getPrincipalWithoutRealmAndHost(gssContext.getSrcName().toString()); + } + } + catch (GSSException e) { + throw new HttpAuthenticationException("Kerberos authentication failed: ", e); + } + finally { + if (gssContext != null) { + try { + gssContext.dispose(); + } catch (GSSException e) { + // No-op + } + } + } + } + + private String getPrincipalWithoutRealm(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + String serviceName = fullKerberosName.getServiceName(); + String hostName = fullKerberosName.getHostName(); + String principalWithoutRealm = serviceName; + if (hostName != null) { + principalWithoutRealm = serviceName + "/" + hostName; + } + return principalWithoutRealm; + } + + private String getPrincipalWithoutRealmAndHost(String fullPrincipal) + throws HttpAuthenticationException { + KerberosNameShim fullKerberosName; + try { + fullKerberosName = ShimLoader.getHadoopShims().getKerberosNameShim(fullPrincipal); + return fullKerberosName.getShortName(); + } catch (IOException e) { + throw new HttpAuthenticationException(e); + } + } + } + + private String getUsername(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String[] creds = getAuthHeaderTokens(request, authType); + // Username must be present + if (creds[0] == null || creds[0].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[0]; + } + + private String getPassword(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String[] creds = getAuthHeaderTokens(request, authType); + // Password must be present + if (creds[1] == null || creds[1].isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain username."); + } + return creds[1]; + } + + private String[] getAuthHeaderTokens(HttpServletRequest request, + String authType) throws HttpAuthenticationException { + String authHeaderBase64 = getAuthHeader(request, authType); + String authHeaderString = StringUtils.newStringUtf8( + Base64.decodeBase64(authHeaderBase64.getBytes())); + String[] creds = authHeaderString.split(":"); + return creds; + } + + /** + * Returns the base64 encoded auth header payload + * @param request + * @param authType + * @return + * @throws HttpAuthenticationException + */ + private String getAuthHeader(HttpServletRequest request, String authType) + throws HttpAuthenticationException { + String authHeader = request.getHeader(HttpAuthUtils.AUTHORIZATION); + // Each http request must have an Authorization header + if (authHeader == null || authHeader.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client is empty."); + } + + String authHeaderBase64String; + int beginIndex; + if (isKerberosAuthMode(authType)) { + beginIndex = (HttpAuthUtils.NEGOTIATE + " ").length(); + } + else { + beginIndex = (HttpAuthUtils.BASIC + " ").length(); + } + authHeaderBase64String = authHeader.substring(beginIndex); + // Authorization header must have a payload + if (authHeaderBase64String == null || authHeaderBase64String.isEmpty()) { + throw new HttpAuthenticationException("Authorization header received " + + "from the client does not contain any data."); + } + return authHeaderBase64String; + } + + private boolean isKerberosAuthMode(String authType) { + return authType.equalsIgnoreCase(HiveAuthFactory.AuthTypes.KERBEROS.toString()); + } + + private static String getDoAsQueryParam(String queryString) { + if (LOG.isDebugEnabled()) { + LOG.debug("URL query string:" + queryString); + } + if (queryString == null) { + return null; + } + Map params = javax.servlet.http.HttpUtils.parseQueryString( queryString ); + Set keySet = params.keySet(); + for (String key: keySet) { + if (key.equalsIgnoreCase("doAs")) { + return params.get(key)[0]; + } + } + return null; + } + +} + + diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java new file mode 100644 index 0000000000000..ae74641ef6805 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/HiveServer2.java @@ -0,0 +1,296 @@ +/** + * 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.hive.service.server; + +import java.util.Properties; + +import scala.runtime.AbstractFunction0; +import scala.runtime.BoxedUnit; + +import org.apache.commons.cli.GnuParser; +import org.apache.commons.cli.HelpFormatter; +import org.apache.commons.cli.Option; +import org.apache.commons.cli.OptionBuilder; +import org.apache.commons.cli.Options; +import org.apache.commons.cli.ParseException; +import org.apache.hadoop.hive.common.JvmPauseMonitor; +import org.apache.hadoop.hive.common.LogUtils; +import org.apache.hadoop.hive.common.LogUtils.LogInitializationException; +import org.apache.hadoop.hive.conf.HiveConf; +import org.apache.hive.common.util.HiveStringUtils; +import org.apache.hive.service.CompositeService; +import org.apache.hive.service.cli.CLIService; +import org.apache.hive.service.cli.thrift.ThriftBinaryCLIService; +import org.apache.hive.service.cli.thrift.ThriftCLIService; +import org.apache.hive.service.cli.thrift.ThriftHttpCLIService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.spark.util.ShutdownHookManager; + +/** + * HiveServer2. + * + */ +public class HiveServer2 extends CompositeService { + private static final Logger LOG = LoggerFactory.getLogger(HiveServer2.class); + + private CLIService cliService; + private ThriftCLIService thriftCLIService; + + public HiveServer2() { + super(HiveServer2.class.getSimpleName()); + HiveConf.setLoadHiveServer2Config(true); + } + + @Override + public synchronized void init(HiveConf hiveConf) { + cliService = new CLIService(this); + addService(cliService); + if (isHTTPTransportMode(hiveConf)) { + thriftCLIService = new ThriftHttpCLIService(cliService); + } else { + thriftCLIService = new ThriftBinaryCLIService(cliService); + } + addService(thriftCLIService); + super.init(hiveConf); + + // Add a shutdown hook for catching SIGTERM & SIGINT + // this must be higher than the Hadoop Filesystem priority of 10, + // which the default priority is. + // The signature of the callback must match that of a scala () -> Unit + // function + ShutdownHookManager.addShutdownHook( + new AbstractFunction0() { + public BoxedUnit apply() { + try { + LOG.info("Hive Server Shutdown hook invoked"); + stop(); + } catch (Throwable e) { + LOG.warn("Ignoring Exception while stopping Hive Server from shutdown hook", + e); + } + return BoxedUnit.UNIT; + } + }); + } + + public static boolean isHTTPTransportMode(HiveConf hiveConf) { + String transportMode = System.getenv("HIVE_SERVER2_TRANSPORT_MODE"); + if (transportMode == null) { + transportMode = hiveConf.getVar(HiveConf.ConfVars.HIVE_SERVER2_TRANSPORT_MODE); + } + if (transportMode != null && (transportMode.equalsIgnoreCase("http"))) { + return true; + } + return false; + } + + @Override + public synchronized void start() { + super.start(); + } + + @Override + public synchronized void stop() { + LOG.info("Shutting down HiveServer2"); + super.stop(); + } + + private static void startHiveServer2() throws Throwable { + long attempts = 0, maxAttempts = 1; + while (true) { + LOG.info("Starting HiveServer2"); + HiveConf hiveConf = new HiveConf(); + maxAttempts = hiveConf.getLongVar(HiveConf.ConfVars.HIVE_SERVER2_MAX_START_ATTEMPTS); + HiveServer2 server = null; + try { + server = new HiveServer2(); + server.init(hiveConf); + server.start(); + try { + JvmPauseMonitor pauseMonitor = new JvmPauseMonitor(hiveConf); + pauseMonitor.start(); + } catch (Throwable t) { + LOG.warn("Could not initiate the JvmPauseMonitor thread.", t); + } + break; + } catch (Throwable throwable) { + if (server != null) { + try { + server.stop(); + } catch (Throwable t) { + LOG.info("Exception caught when calling stop of HiveServer2 before retrying start", t); + } finally { + server = null; + } + } + if (++attempts >= maxAttempts) { + throw new Error("Max start attempts " + maxAttempts + " exhausted", throwable); + } else { + LOG.warn("Error starting HiveServer2 on attempt " + attempts + + ", will retry in 60 seconds", throwable); + try { + Thread.sleep(60L * 1000L); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + } + } + } + + public static void main(String[] args) { + HiveConf.setLoadHiveServer2Config(true); + try { + ServerOptionsProcessor oproc = new ServerOptionsProcessor("hiveserver2"); + ServerOptionsProcessorResponse oprocResponse = oproc.parse(args); + + // NOTE: It is critical to do this here so that log4j is reinitialized + // before any of the other core hive classes are loaded + String initLog4jMessage = LogUtils.initHiveLog4j(); + LOG.debug(initLog4jMessage); + HiveStringUtils.startupShutdownMessage(HiveServer2.class, args, LOG); + + // Log debug message from "oproc" after log4j initialize properly + LOG.debug(oproc.getDebugMessage().toString()); + + // Call the executor which will execute the appropriate command based on the parsed options + oprocResponse.getServerOptionsExecutor().execute(); + } catch (LogInitializationException e) { + LOG.error("Error initializing log: " + e.getMessage(), e); + System.exit(-1); + } + } + + /** + * ServerOptionsProcessor. + * Process arguments given to HiveServer2 (-hiveconf property=value) + * Set properties in System properties + * Create an appropriate response object, + * which has executor to execute the appropriate command based on the parsed options. + */ + public static class ServerOptionsProcessor { + private final Options options = new Options(); + private org.apache.commons.cli.CommandLine commandLine; + private final String serverName; + private final StringBuilder debugMessage = new StringBuilder(); + + @SuppressWarnings("static-access") + public ServerOptionsProcessor(String serverName) { + this.serverName = serverName; + // -hiveconf x=y + options.addOption(OptionBuilder + .withValueSeparator() + .hasArgs(2) + .withArgName("property=value") + .withLongOpt("hiveconf") + .withDescription("Use value for given property") + .create()); + options.addOption(new Option("H", "help", false, "Print help information")); + } + + public ServerOptionsProcessorResponse parse(String[] argv) { + try { + commandLine = new GnuParser().parse(options, argv); + // Process --hiveconf + // Get hiveconf param values and set the System property values + Properties confProps = commandLine.getOptionProperties("hiveconf"); + for (String propKey : confProps.stringPropertyNames()) { + // save logging message for log4j output latter after log4j initialize properly + debugMessage.append("Setting " + propKey + "=" + confProps.getProperty(propKey) + ";\n"); + System.setProperty(propKey, confProps.getProperty(propKey)); + } + + // Process --help + if (commandLine.hasOption('H')) { + return new ServerOptionsProcessorResponse(new HelpOptionExecutor(serverName, options)); + } + } catch (ParseException e) { + // Error out & exit - we were not able to parse the args successfully + System.err.println("Error starting HiveServer2 with given arguments: "); + System.err.println(e.getMessage()); + System.exit(-1); + } + // Default executor, when no option is specified + return new ServerOptionsProcessorResponse(new StartOptionExecutor()); + } + + StringBuilder getDebugMessage() { + return debugMessage; + } + } + + /** + * The response sent back from {@link ServerOptionsProcessor#parse(String[])} + */ + static class ServerOptionsProcessorResponse { + private final ServerOptionsExecutor serverOptionsExecutor; + + ServerOptionsProcessorResponse(ServerOptionsExecutor serverOptionsExecutor) { + this.serverOptionsExecutor = serverOptionsExecutor; + } + + ServerOptionsExecutor getServerOptionsExecutor() { + return serverOptionsExecutor; + } + } + + /** + * The executor interface for running the appropriate HiveServer2 command based on parsed options + */ + interface ServerOptionsExecutor { + void execute(); + } + + /** + * HelpOptionExecutor: executes the --help option by printing out the usage + */ + static class HelpOptionExecutor implements ServerOptionsExecutor { + private final Options options; + private final String serverName; + + HelpOptionExecutor(String serverName, Options options) { + this.options = options; + this.serverName = serverName; + } + + @Override + public void execute() { + new HelpFormatter().printHelp(serverName, options); + System.exit(0); + } + } + + /** + * StartOptionExecutor: starts HiveServer2. + * This is the default executor, when no option is specified. + */ + static class StartOptionExecutor implements ServerOptionsExecutor { + @Override + public void execute() { + try { + startHiveServer2(); + } catch (Throwable t) { + LOG.error("Error starting HiveServer2", t); + System.exit(-1); + } + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java new file mode 100644 index 0000000000000..9dd1c4d808edb --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/java/org/apache/hive/service/server/ThreadWithGarbageCleanup.java @@ -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.hive.service.server; + +import java.util.Map; + +import org.apache.hadoop.hive.metastore.HiveMetaStore; +import org.apache.hadoop.hive.metastore.RawStore; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * A HiveServer2 thread used to construct new server threads. + * In particular, this thread ensures an orderly cleanup, + * when killed by its corresponding ExecutorService. + */ +public class ThreadWithGarbageCleanup extends Thread { + private static final Logger LOG = LoggerFactory.getLogger(ThreadWithGarbageCleanup.class); + + Map threadRawStoreMap = + ThreadFactoryWithGarbageCleanup.getThreadRawStoreMap(); + + public ThreadWithGarbageCleanup(Runnable runnable) { + super(runnable); + } + + /** + * Add any Thread specific garbage cleanup code here. + * Currently, it shuts down the RawStore object for this thread if it is not null. + */ + @Override + public void finalize() throws Throwable { + cleanRawStore(); + super.finalize(); + } + + private void cleanRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = threadRawStoreMap.get(threadId); + if (threadLocalRawStore != null) { + LOG.debug("RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " will be closed now."); + threadLocalRawStore.shutdown(); + threadRawStoreMap.remove(threadId); + } + } + + /** + * Cache the ThreadLocal RawStore object. Called from the corresponding thread. + */ + public void cacheThreadLocalRawStore() { + Long threadId = this.getId(); + RawStore threadLocalRawStore = HiveMetaStore.HMSHandler.getRawStore(); + if (threadLocalRawStore != null && !threadRawStoreMap.containsKey(threadId)) { + LOG.debug("Adding RawStore: " + threadLocalRawStore + ", for the thread: " + + this.getName() + " to threadRawStoreMap for future cleanup."); + threadRawStoreMap.put(threadId, threadLocalRawStore); + } + } +} diff --git a/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala new file mode 100644 index 0000000000000..75637a81a3ab5 --- /dev/null +++ b/sql/hive-thriftserver/v2.3.5/src/main/scala/org/apache/spark/sql/hive/thriftserver/ThriftserverShimUtils.scala @@ -0,0 +1,50 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.thriftserver + +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde2.thrift.Type +import org.apache.hive.service.cli.{RowSet, RowSetFactory, TableSchema} +import org.slf4j.LoggerFactory + +/** + * Various utilities for hive-thriftserver used to upgrade the built-in Hive. + */ +private[thriftserver] object ThriftserverShimUtils { + + private[thriftserver] type TProtocolVersion = org.apache.hive.service.rpc.thrift.TProtocolVersion + private[thriftserver] type Client = org.apache.hive.service.rpc.thrift.TCLIService.Client + private[thriftserver] type TOpenSessionReq = org.apache.hive.service.rpc.thrift.TOpenSessionReq + private[thriftserver] type TGetSchemasReq = org.apache.hive.service.rpc.thrift.TGetSchemasReq + private[thriftserver] type TGetTablesReq = org.apache.hive.service.rpc.thrift.TGetTablesReq + private[thriftserver] type TGetColumnsReq = org.apache.hive.service.rpc.thrift.TGetColumnsReq + + private[thriftserver] def getConsole: SessionState.LogHelper = { + val LOG = LoggerFactory.getLogger(classOf[SparkSQLCLIDriver]) + new SessionState.LogHelper(LOG) + } + + private[thriftserver] def resultRowSet( + getResultSetSchema: TableSchema, + getProtocolVersion: TProtocolVersion): RowSet = { + RowSetFactory.create(getResultSetSchema, getProtocolVersion, false) + } + + private[thriftserver] def toJavaSQLType(s: String): Int = Type.getType(s).toJavaSQLType + +} diff --git a/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt new file mode 100644 index 0000000000000..f3044da972497 --- /dev/null +++ b/sql/hive/benchmarks/ObjectHashAggregateExecBenchmark-results.txt @@ -0,0 +1,45 @@ +================================================================================================ +Hive UDAF vs Spark AF +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +hive udaf w/o group by 6370 / 6400 0.0 97193.6 1.0X +spark af w/o group by 54 / 63 1.2 820.8 118.4X +hive udaf w/ group by 4492 / 4507 0.0 68539.5 1.4X +spark af w/ group by w/o fallback 58 / 64 1.1 881.7 110.2X +spark af w/ group by w/ fallback 136 / 142 0.5 2075.0 46.8X + + +================================================================================================ +ObjectHashAggregateExec vs SortAggregateExec - typed_count +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sort agg w/ group by 41500 / 41630 2.5 395.8 1.0X +object agg w/ group by w/o fallback 10075 / 10122 10.4 96.1 4.1X +object agg w/ group by w/ fallback 28131 / 28205 3.7 268.3 1.5X +sort agg w/o group by 6182 / 6221 17.0 59.0 6.7X +object agg w/o group by w/o fallback 5435 / 5468 19.3 51.8 7.6X + + +================================================================================================ +ObjectHashAggregateExec vs SortAggregateExec - percentile_approx +================================================================================================ + +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 +Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz +object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +------------------------------------------------------------------------------------------------ +sort agg w/ group by 970 / 1025 2.2 462.5 1.0X +object agg w/ group by w/o fallback 772 / 798 2.7 368.1 1.3X +object agg w/ group by w/ fallback 1013 / 1044 2.1 483.1 1.0X +sort agg w/o group by 751 / 781 2.8 358.0 1.3X +object agg w/o group by w/o fallback 772 / 814 2.7 368.0 1.3X + + diff --git a/sql/hive/benchmarks/OrcReadBenchmark-results.txt b/sql/hive/benchmarks/OrcReadBenchmark-results.txt index c77f966723d71..caa78b9a8f102 100644 --- a/sql/hive/benchmarks/OrcReadBenchmark-results.txt +++ b/sql/hive/benchmarks/OrcReadBenchmark-results.txt @@ -2,172 +2,155 @@ SQL Single Numeric Column Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single TINYINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1630 / 1639 9.7 103.6 1.0X -Native ORC Vectorized 253 / 288 62.2 16.1 6.4X -Native ORC Vectorized with copy 227 / 244 69.2 14.5 7.2X -Hive built-in ORC 1980 / 1991 7.9 125.9 0.8X +Native ORC MR 1725 / 1759 9.1 109.7 1.0X +Native ORC Vectorized 272 / 316 57.8 17.3 6.3X +Hive built-in ORC 1970 / 1987 8.0 125.3 0.9X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single SMALLINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1587 / 1589 9.9 100.9 1.0X -Native ORC Vectorized 227 / 242 69.2 14.5 7.0X -Native ORC Vectorized with copy 228 / 238 69.0 14.5 7.0X -Hive built-in ORC 2323 / 2332 6.8 147.7 0.7X +Native ORC MR 1633 / 1672 9.6 103.8 1.0X +Native ORC Vectorized 238 / 255 66.0 15.1 6.9X +Hive built-in ORC 2293 / 2305 6.9 145.8 0.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single INT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1726 / 1771 9.1 109.7 1.0X -Native ORC Vectorized 309 / 333 50.9 19.7 5.6X -Native ORC Vectorized with copy 313 / 321 50.2 19.9 5.5X -Hive built-in ORC 2668 / 2672 5.9 169.6 0.6X +Native ORC MR 1677 / 1699 9.4 106.6 1.0X +Native ORC Vectorized 325 / 342 48.3 20.7 5.2X +Hive built-in ORC 2561 / 2569 6.1 162.8 0.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single BIGINT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1722 / 1747 9.1 109.5 1.0X -Native ORC Vectorized 395 / 403 39.8 25.1 4.4X -Native ORC Vectorized with copy 399 / 405 39.4 25.4 4.3X -Hive built-in ORC 2767 / 2777 5.7 175.9 0.6X +Native ORC MR 1791 / 1795 8.8 113.9 1.0X +Native ORC Vectorized 400 / 408 39.3 25.4 4.5X +Hive built-in ORC 2713 / 2720 5.8 172.5 0.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single FLOAT Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1797 / 1824 8.8 114.2 1.0X -Native ORC Vectorized 434 / 441 36.2 27.6 4.1X -Native ORC Vectorized with copy 437 / 447 36.0 27.8 4.1X -Hive built-in ORC 2701 / 2710 5.8 171.7 0.7X +Native ORC MR 1791 / 1805 8.8 113.8 1.0X +Native ORC Vectorized 433 / 438 36.3 27.5 4.1X +Hive built-in ORC 2690 / 2803 5.8 171.0 0.7X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz SQL Single DOUBLE Column Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1931 / 2028 8.1 122.8 1.0X -Native ORC Vectorized 542 / 557 29.0 34.5 3.6X -Native ORC Vectorized with copy 550 / 564 28.6 35.0 3.5X -Hive built-in ORC 2816 / 3206 5.6 179.1 0.7X +Native ORC MR 1911 / 1930 8.2 121.5 1.0X +Native ORC Vectorized 543 / 552 29.0 34.5 3.5X +Hive built-in ORC 2967 / 3065 5.3 188.6 0.6X ================================================================================================ Int and String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Int and String Scan: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 4012 / 4068 2.6 382.6 1.0X -Native ORC Vectorized 2337 / 2339 4.5 222.9 1.7X -Native ORC Vectorized with copy 2520 / 2540 4.2 240.3 1.6X -Hive built-in ORC 5503 / 5575 1.9 524.8 0.7X +Native ORC MR 4160 / 4188 2.5 396.7 1.0X +Native ORC Vectorized 2405 / 2406 4.4 229.4 1.7X +Hive built-in ORC 5514 / 5562 1.9 525.9 0.8X ================================================================================================ Partitioned Table Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Partitioned Table: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Data column - Native ORC MR 2020 / 2025 7.8 128.4 1.0X -Data column - Native ORC Vectorized 398 / 409 39.5 25.3 5.1X -Data column - Native ORC Vectorized with copy 406 / 411 38.8 25.8 5.0X -Data column - Hive built-in ORC 2967 / 2969 5.3 188.6 0.7X -Partition column - Native ORC MR 1494 / 1505 10.5 95.0 1.4X -Partition column - Native ORC Vectorized 73 / 82 216.3 4.6 27.8X -Partition column - Native ORC Vectorized with copy 71 / 80 221.4 4.5 28.4X -Partition column - Hive built-in ORC 1932 / 1937 8.1 122.8 1.0X -Both columns - Native ORC MR 2057 / 2071 7.6 130.8 1.0X -Both columns - Native ORC Vectorized 445 / 448 35.4 28.3 4.5X -Both column - Native ORC Vectorized with copy 534 / 539 29.4 34.0 3.8X -Both columns - Hive built-in ORC 2994 / 2994 5.3 190.3 0.7X +Data column - Native ORC MR 1863 / 1867 8.4 118.4 1.0X +Data column - Native ORC Vectorized 411 / 418 38.2 26.2 4.5X +Data column - Hive built-in ORC 3297 / 3308 4.8 209.6 0.6X +Partition column - Native ORC MR 1505 / 1506 10.4 95.7 1.2X +Partition column - Native ORC Vectorized 80 / 93 195.6 5.1 23.2X +Partition column - Hive built-in ORC 1960 / 1979 8.0 124.6 1.0X +Both columns - Native ORC MR 2076 / 2090 7.6 132.0 0.9X +Both columns - Native ORC Vectorized 450 / 463 34.9 28.6 4.1X +Both columns - Hive built-in ORC 3528 / 3548 4.5 224.3 0.5X ================================================================================================ Repeated String Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Repeated String: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1771 / 1785 5.9 168.9 1.0X -Native ORC Vectorized 372 / 375 28.2 35.5 4.8X -Native ORC Vectorized with copy 543 / 576 19.3 51.8 3.3X -Hive built-in ORC 2671 / 2671 3.9 254.7 0.7X +Native ORC MR 1727 / 1733 6.1 164.7 1.0X +Native ORC Vectorized 375 / 379 28.0 35.7 4.6X +Hive built-in ORC 2665 / 2666 3.9 254.2 0.6X ================================================================================================ String with Nulls Scan ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz String with Nulls Scan (0.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 3276 / 3302 3.2 312.5 1.0X -Native ORC Vectorized 1057 / 1080 9.9 100.8 3.1X -Native ORC Vectorized with copy 1420 / 1431 7.4 135.4 2.3X -Hive built-in ORC 5377 / 5407 2.0 512.8 0.6X +Native ORC MR 3324 / 3325 3.2 317.0 1.0X +Native ORC Vectorized 1085 / 1106 9.7 103.4 3.1X +Hive built-in ORC 5272 / 5299 2.0 502.8 0.6X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.5%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +String with Nulls Scan (50.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 3147 / 3147 3.3 300.1 1.0X -Native ORC Vectorized 1305 / 1319 8.0 124.4 2.4X -Native ORC Vectorized with copy 1685 / 1686 6.2 160.7 1.9X -Hive built-in ORC 4077 / 4085 2.6 388.8 0.8X +Native ORC MR 3045 / 3046 3.4 290.4 1.0X +Native ORC Vectorized 1248 / 1260 8.4 119.0 2.4X +Hive built-in ORC 3989 / 3999 2.6 380.4 0.8X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz -String with Nulls Scan (0.95%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative +String with Nulls Scan (95.0%): Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1739 / 1744 6.0 165.8 1.0X -Native ORC Vectorized 500 / 501 21.0 47.7 3.5X -Native ORC Vectorized with copy 618 / 631 17.0 58.9 2.8X -Hive built-in ORC 2411 / 2427 4.3 229.9 0.7X +Native ORC MR 1692 / 1694 6.2 161.3 1.0X +Native ORC Vectorized 471 / 493 22.3 44.9 3.6X +Hive built-in ORC 2398 / 2411 4.4 228.7 0.7X ================================================================================================ Single Column Scan From Wide Columns ================================================================================================ -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 100 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 1348 / 1366 0.8 1285.3 1.0X -Native ORC Vectorized 119 / 134 8.8 113.5 11.3X -Native ORC Vectorized with copy 119 / 148 8.8 113.9 11.3X -Hive built-in ORC 487 / 507 2.2 464.8 2.8X +Native ORC MR 1371 / 1379 0.8 1307.5 1.0X +Native ORC Vectorized 121 / 135 8.6 115.8 11.3X +Hive built-in ORC 521 / 561 2.0 497.1 2.6X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 200 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 2667 / 2837 0.4 2543.6 1.0X -Native ORC Vectorized 203 / 222 5.2 193.4 13.2X -Native ORC Vectorized with copy 217 / 255 4.8 207.0 12.3X -Hive built-in ORC 737 / 741 1.4 702.4 3.6X +Native ORC MR 2711 / 2767 0.4 2585.5 1.0X +Native ORC Vectorized 210 / 232 5.0 200.5 12.9X +Hive built-in ORC 764 / 775 1.4 728.3 3.5X -OpenJDK 64-Bit Server VM 1.8.0_181-b13 on Linux 3.10.0-862.3.2.el7.x86_64 +OpenJDK 64-Bit Server VM 1.8.0_191-b12 on Linux 3.10.0-862.3.2.el7.x86_64 Intel(R) Xeon(R) CPU E5-2670 v2 @ 2.50GHz Single Column Scan from 300 columns: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative ------------------------------------------------------------------------------------------------ -Native ORC MR 3954 / 3956 0.3 3770.4 1.0X -Native ORC Vectorized 348 / 360 3.0 331.7 11.4X -Native ORC Vectorized with copy 349 / 359 3.0 333.2 11.3X -Hive built-in ORC 1057 / 1067 1.0 1008.0 3.7X +Native ORC MR 3979 / 3988 0.3 3794.4 1.0X +Native ORC Vectorized 357 / 366 2.9 340.2 11.2X +Hive built-in ORC 1091 / 1095 1.0 1040.5 3.6X diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index ef22e2abfb53e..d37f0c8573659 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -21,12 +21,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../../pom.xml - spark-hive_2.11 + spark-hive_2.12 jar Spark Project Hive http://spark.apache.org/ @@ -37,7 +37,7 @@ - com.twitter + ${hive.parquet.group} parquet-hadoop-bundle @@ -97,14 +97,20 @@ ${hive.group} hive-exec - + ${hive.classifier} ${hive.group} hive-metastore + + ${hive.group} + hive-contrib + + + ${hive.group}.hcatalog + hive-hcatalog-core + - -da -Xmx3g -XX:ReservedCodeCacheSize=${CodeCacheSize} + -da -Xmx4g -XX:ReservedCodeCacheSize=${CodeCacheSize} diff --git a/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java b/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java index f093637d412f9..8e9362ab8afbc 100644 --- a/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java +++ b/sql/hive/src/main/java/org/apache/hadoop/hive/ql/io/orc/SparkOrcNewRecordReader.java @@ -24,7 +24,6 @@ import org.apache.hadoop.mapreduce.TaskAttemptContext; import java.io.IOException; -import java.util.List; /** * This is based on hive-exec-1.2.1 @@ -42,8 +41,11 @@ public class SparkOrcNewRecordReader extends public SparkOrcNewRecordReader(Reader file, Configuration conf, long offset, long length) throws IOException { - List types = file.getTypes(); - numColumns = (types.size() == 0) ? 0 : types.get(0).getSubtypesCount(); + if (file.getTypes().isEmpty()) { + numColumns = 0; + } else { + numColumns = file.getTypes().get(0).getSubtypesCount(); + } value = new OrcStruct(numColumns); this.reader = OrcInputFormat.createReaderFromFile(file, conf, offset, length); diff --git a/sql/hive/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider new file mode 100644 index 0000000000000..2b0acc0305c49 --- /dev/null +++ b/sql/hive/src/main/resources/META-INF/services/org.apache.spark.security.HadoopDelegationTokenProvider @@ -0,0 +1 @@ +org.apache.spark.sql.hive.security.HiveDelegationTokenProvider 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 445161d5de1c2..d4df35c8ec69c 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 @@ -27,6 +27,7 @@ import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants.DDL_TIME import org.apache.hadoop.hive.ql.metadata.HiveException import org.apache.hadoop.hive.serde.serdeConstants.SERIALIZATION_FORMAT import org.apache.thrift.TException @@ -119,6 +120,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat client.getTable(db, table) } + private[hive] def getRawTablesByNames(db: String, tables: Seq[String]): Seq[CatalogTable] = { + client.getTablesByName(db, tables) + } + /** * If the given table properties contains datasource properties, throw an exception. We will do * this check when create or alter a table, i.e. when we try to write table metadata to Hive @@ -127,7 +132,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat private def verifyTableProperties(table: CatalogTable): Unit = { val invalidKeys = table.properties.keys.filter(_.startsWith(SPARK_SQL_PREFIX)) if (invalidKeys.nonEmpty) { - throw new AnalysisException(s"Cannot persistent ${table.qualifiedName} into hive metastore " + + throw new AnalysisException(s"Cannot persist ${table.qualifiedName} into Hive metastore " + s"as table property keys may not start with '$SPARK_SQL_PREFIX': " + invalidKeys.mkString("[", ", ", "]")) } @@ -701,6 +706,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat restoreTableMetadata(getRawTable(db, table)) } + override def getTablesByName(db: String, tables: Seq[String]): Seq[CatalogTable] = withClient { + getRawTablesByNames(db, tables).map(restoreTableMetadata) + } + /** * Restores table metadata from the table properties. This method is kind of a opposite version * of [[createTable]]. @@ -821,7 +830,8 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat schema = reorderedSchema, partitionColumnNames = partColumnNames, bucketSpec = getBucketSpecFromTableProperties(table), - tracksPartitionsInCatalog = partitionProvider == Some(TABLE_PARTITION_PROVIDER_CATALOG)) + tracksPartitionsInCatalog = partitionProvider == Some(TABLE_PARTITION_PROVIDER_CATALOG), + properties = table.properties.filterKeys(!HIVE_GENERATED_TABLE_PROPERTIES(_))) } override def tableExists(db: String, table: String): Boolean = withClient { @@ -1328,6 +1338,7 @@ object HiveExternalCatalog { val CREATED_SPARK_VERSION = SPARK_SQL_PREFIX + "create.version" + val HIVE_GENERATED_TABLE_PROPERTIES = Set(DDL_TIME) val HIVE_GENERATED_STORAGE_PROPERTIES = Set(SERIALIZATION_FORMAT) // When storing data source tables in hive metastore, we need to set data schema to empty if the 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 4dec2f71b8a50..33b5bcefd853f 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,6 +18,7 @@ package org.apache.spark.sql.hive import java.lang.reflect.{ParameterizedType, Type, WildcardType} +import java.util.concurrent.TimeUnit._ import scala.collection.JavaConverters._ @@ -460,7 +461,7 @@ private[hive] trait HiveInspectors { _ => constant case poi: WritableConstantTimestampObjectInspector => val t = poi.getWritableConstantValue - val constant = t.getSeconds * 1000000L + t.getNanos / 1000L + val constant = SECONDS.toMicros(t.getSeconds) + NANOSECONDS.toMicros(t.getNanos) _ => constant case poi: WritableConstantIntObjectInspector => val constant = poi.getWritableConstantValue.get() @@ -629,7 +630,7 @@ private[hive] trait HiveInspectors { data: Any => { if (data != null) { val t = x.getPrimitiveWritableObject(data) - t.getSeconds * 1000000L + t.getNanos / 1000L + SECONDS.toMicros(t.getSeconds) + NANOSECONDS.toMicros(t.getNanos) } else { null } @@ -846,6 +847,8 @@ private[hive] trait HiveInspectors { ObjectInspectorFactory.getStandardConstantMapObjectInspector(keyOI, valueOI, jmap) } + case Literal(_, dt: StructType) => + toInspector(dt) // We will enumerate all of the possible constant expressions, throw exception if we missed case Literal(_, dt) => sys.error(s"Hive doesn't support the constant type [$dt].") // ideally, we don't test the foldable here(but in optimizer), however, some of the diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index d047953327958..d6b2945b2ea7a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive +import java.util.Locale + import scala.util.control.NonFatal import com.google.common.util.concurrent.Striped @@ -29,6 +31,8 @@ import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode._ import org.apache.spark.sql.types._ @@ -113,7 +117,44 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log } } - def convertToLogicalRelation( + // Return true for Apache ORC and Hive ORC-related configuration names. + // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. + private def isOrcProperty(key: String) = + key.startsWith("orc.") || key.contains(".orc.") + + private def isParquetProperty(key: String) = + key.startsWith("parquet.") || key.contains(".parquet.") + + def convert(relation: HiveTableRelation): LogicalRelation = { + val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + + // Consider table and storage properties. For properties existing in both sides, storage + // properties will supersede table properties. + if (serde.contains("parquet")) { + val options = relation.tableMeta.properties.filterKeys(isParquetProperty) ++ + relation.tableMeta.storage.properties + (ParquetOptions.MERGE_SCHEMA -> + SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING).toString) + convertToLogicalRelation(relation, options, classOf[ParquetFileFormat], "parquet") + } else { + val options = relation.tableMeta.properties.filterKeys(isOrcProperty) ++ + relation.tableMeta.storage.properties + if (SQLConf.get.getConf(SQLConf.ORC_IMPLEMENTATION) == "native") { + convertToLogicalRelation( + relation, + options, + classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat], + "orc") + } else { + convertToLogicalRelation( + relation, + options, + classOf[org.apache.spark.sql.hive.orc.OrcFileFormat], + "orc") + } + } + } + + private def convertToLogicalRelation( relation: HiveTableRelation, options: Map[String, String], fileFormatClass: Class[_ <: FileFormat], @@ -124,7 +165,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log val lazyPruningEnabled = sparkSession.sqlContext.conf.manageFilesourcePartitions val tablePath = new Path(relation.tableMeta.location) - val fileFormat = fileFormatClass.newInstance() + val fileFormat = fileFormatClass.getConstructor().newInstance() val result = if (relation.isPartitioned) { val partitionSchema = relation.tableMeta.partitionSchema @@ -285,8 +326,8 @@ private[hive] object HiveMetastoreCatalog { // Merge missing nullable fields to inferred schema and build a case-insensitive field map. val inferredFields = StructType(inferredSchema ++ missingNullables) .map(f => f.name.toLowerCase -> f).toMap + StructType(metastoreSchema.map(f => f.copy(name = inferredFields(f.name.toLowerCase).name))) // scalastyle:on caselocale - StructType(metastoreSchema.map(f => f.copy(name = inferredFields(f.name).name))) } catch { case NonFatal(_) => val msg = s"""Detected conflicting schemas when merging the schema obtained from the Hive diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala index 405c0c8bfe660..3f0a9f222feb2 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionCatalog.scala @@ -36,6 +36,7 @@ import org.apache.spark.sql.catalyst.parser.ParserInterface import org.apache.spark.sql.hive.HiveShim.HiveFunctionWrapper import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.{DecimalType, DoubleType} +import org.apache.spark.util.Utils private[sql] class HiveSessionCatalog( @@ -142,7 +143,7 @@ private[sql] class HiveSessionCatalog( // Hive is case insensitive. val functionName = funcName.unquotedString.toLowerCase(Locale.ROOT) if (!hiveFunctions.contains(functionName)) { - failFunctionLookup(funcName) + failFunctionLookup(funcName, Some(error)) } // TODO: Remove this fallback path once we implement the list of fallback functions @@ -150,12 +151,12 @@ private[sql] class HiveSessionCatalog( val functionInfo = { try { Option(HiveFunctionRegistry.getFunctionInfo(functionName)).getOrElse( - failFunctionLookup(funcName)) + failFunctionLookup(funcName, Some(error))) } catch { // If HiveFunctionRegistry.getFunctionInfo throws an exception, // we are failing to load a Hive builtin function, which means that // the given function is not a Hive builtin function. - case NonFatal(e) => failFunctionLookup(funcName) + case NonFatal(e) => failFunctionLookup(funcName, Some(e)) } } val className = functionInfo.getFunctionClass.getName diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala index 2882672f327c4..2fa108825982f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveSessionStateBuilder.scala @@ -17,14 +17,16 @@ package org.apache.spark.sql.hive -import org.apache.spark.annotation.{Experimental, InterfaceStability} +import org.apache.spark.annotation.{Experimental, Unstable} import org.apache.spark.sql._ +import org.apache.spark.sql.catalog.v2.CatalogPlugin import org.apache.spark.sql.catalyst.analysis.Analyzer import org.apache.spark.sql.catalyst.catalog.ExternalCatalogWithListener import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution.SparkPlanner import org.apache.spark.sql.execution.datasources._ +import org.apache.spark.sql.execution.datasources.v2.{V2StreamingScanSupportCheck, V2WriteSupportCheck} import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLoader, SessionState} @@ -32,7 +34,7 @@ import org.apache.spark.sql.internal.{BaseSessionStateBuilder, SessionResourceLo * Builder that produces a Hive-aware `SessionState`. */ @Experimental -@InterfaceStability.Unstable +@Unstable class HiveSessionStateBuilder(session: SparkSession, parentState: Option[SessionState] = None) extends BaseSessionStateBuilder(session, parentState) { @@ -71,6 +73,8 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session new ResolveHiveSerdeTable(session) +: new FindDataSourceTable(session) +: new ResolveSQLOnFile(session) +: + new FallBackFileSourceV2(session) +: + DataSourceResolution(conf, this) +: customResolutionRules override val postHocResolutionRules: Seq[Rule[LogicalPlan]] = @@ -85,7 +89,11 @@ class HiveSessionStateBuilder(session: SparkSession, parentState: Option[Session override val extendedCheckRules: Seq[LogicalPlan => Unit] = PreWriteCheck +: PreReadCheck +: + V2WriteSupportCheck +: + V2StreamingScanSupportCheck +: customCheckRules + + override protected def lookupCatalog(name: String): CatalogPlugin = session.catalog(name) } /** 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 11afe1af32809..be4a0c175b6dc 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive import java.io.{InputStream, OutputStream} +import java.lang.reflect.Method import java.rmi.server.UID import scala.collection.JavaConverters._ @@ -28,15 +29,13 @@ import com.google.common.base.Objects import org.apache.avro.Schema import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.hive.ql.exec.{UDF, Utilities} +import org.apache.hadoop.hive.ql.exec.UDF import org.apache.hadoop.hive.ql.plan.{FileSinkDesc, TableDesc} import org.apache.hadoop.hive.ql.udf.generic.GenericUDFMacro import org.apache.hadoop.hive.serde2.ColumnProjectionUtils import org.apache.hadoop.hive.serde2.avro.{AvroGenericRecordWritable, AvroSerdeUtils} import org.apache.hadoop.hive.serde2.objectinspector.primitive.HiveDecimalObjectInspector import org.apache.hadoop.io.Writable -import org.apache.hive.com.esotericsoftware.kryo.Kryo -import org.apache.hive.com.esotericsoftware.kryo.io.{Input, Output} import org.apache.spark.internal.Logging import org.apache.spark.sql.types.Decimal @@ -146,34 +145,60 @@ private[hive] object HiveShim { case _ => false } - @transient - def deserializeObjectByKryo[T: ClassTag]( - kryo: Kryo, - in: InputStream, - clazz: Class[_]): T = { - val inp = new Input(in) - val t: T = kryo.readObject(inp, clazz).asInstanceOf[T] - inp.close() - t - } + private lazy val serUtilClass = + Utils.classForName("org.apache.hadoop.hive.ql.exec.SerializationUtilities") + private lazy val utilClass = Utils.classForName("org.apache.hadoop.hive.ql.exec.Utilities") + private val deserializeMethodName = "deserializeObjectByKryo" + private val serializeMethodName = "serializeObjectByKryo" - @transient - def serializeObjectByKryo( - kryo: Kryo, - plan: Object, - out: OutputStream) { - val output: Output = new Output(out) - kryo.writeObject(output, plan) - output.close() + private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { + val method = klass.getDeclaredMethod(name, args: _*) + method.setAccessible(true) + method } def deserializePlan[UDFType](is: java.io.InputStream, clazz: Class[_]): UDFType = { - deserializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), is, clazz) - .asInstanceOf[UDFType] + if (HiveUtils.isHive23) { + val borrowKryo = serUtilClass.getMethod("borrowKryo") + val kryo = borrowKryo.invoke(serUtilClass) + val deserializeObjectByKryo = findMethod(serUtilClass, deserializeMethodName, + kryo.getClass.getSuperclass, classOf[InputStream], classOf[Class[_]]) + try { + deserializeObjectByKryo.invoke(null, kryo, is, clazz).asInstanceOf[UDFType] + } finally { + serUtilClass.getMethod("releaseKryo", kryo.getClass.getSuperclass).invoke(null, kryo) + } + } else { + val runtimeSerializationKryo = utilClass.getField("runtimeSerializationKryo") + val threadLocalValue = runtimeSerializationKryo.get(utilClass) + val getMethod = threadLocalValue.getClass.getMethod("get") + val kryo = getMethod.invoke(threadLocalValue) + val deserializeObjectByKryo = findMethod(utilClass, deserializeMethodName, + kryo.getClass, classOf[InputStream], classOf[Class[_]]) + deserializeObjectByKryo.invoke(null, kryo, is, clazz).asInstanceOf[UDFType] + } } def serializePlan(function: AnyRef, out: java.io.OutputStream): Unit = { - serializeObjectByKryo(Utilities.runtimeSerializationKryo.get(), function, out) + if (HiveUtils.isHive23) { + val borrowKryo = serUtilClass.getMethod("borrowKryo") + val kryo = borrowKryo.invoke(serUtilClass) + val serializeObjectByKryo = findMethod(serUtilClass, serializeMethodName, + kryo.getClass.getSuperclass, classOf[Object], classOf[OutputStream]) + try { + serializeObjectByKryo.invoke(null, kryo, function, out) + } finally { + serUtilClass.getMethod("releaseKryo", kryo.getClass.getSuperclass).invoke(null, kryo) + } + } else { + val runtimeSerializationKryo = utilClass.getField("runtimeSerializationKryo") + val threadLocalValue = runtimeSerializationKryo.get(utilClass) + val getMethod = threadLocalValue.getClass.getMethod("get") + val kryo = getMethod.invoke(threadLocalValue) + val serializeObjectByKryo = findMethod(utilClass, serializeMethodName, + kryo.getClass, classOf[Object], classOf[OutputStream]) + serializeObjectByKryo.invoke(null, kryo, function, out) + } } def writeExternal(out: java.io.ObjectOutput) { @@ -217,7 +242,7 @@ private[hive] object HiveShim { instance.asInstanceOf[UDFType] } else { val func = Utils.getContextOrSparkClassLoader - .loadClass(functionClassName).newInstance.asInstanceOf[UDFType] + .loadClass(functionClassName).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/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala index 07ee105404311..7b28e4f401aba 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -31,8 +31,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{InsertIntoDir, InsertIntoTab import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.execution._ import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} -import org.apache.spark.sql.execution.datasources.{CreateTable, LogicalRelation} -import org.apache.spark.sql.execution.datasources.parquet.{ParquetFileFormat, ParquetOptions} +import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} @@ -152,15 +151,14 @@ object HiveAnalysis extends Rule[LogicalPlan] { ifPartitionNotExists, query.output.map(_.name)) case CreateTable(tableDesc, mode, None) if DDLUtils.isHiveTable(tableDesc) => - DDLUtils.checkDataColNames(tableDesc) CreateTableCommand(tableDesc, ignoreIfExists = mode == SaveMode.Ignore) - case CreateTable(tableDesc, mode, Some(query)) if DDLUtils.isHiveTable(tableDesc) => - DDLUtils.checkDataColNames(tableDesc) + case CreateTable(tableDesc, mode, Some(query)) + if DDLUtils.isHiveTable(tableDesc) && query.resolved => CreateHiveTableAsSelectCommand(tableDesc, query, query.output.map(_.name), mode) case InsertIntoDir(isLocal, storage, provider, child, overwrite) - if DDLUtils.isHiveTable(provider) => + if DDLUtils.isHiveTable(provider) && child.resolved => val outputPath = new Path(storage.locationUri.get) if (overwrite) DDLUtils.verifyNotReadPath(child, outputPath) @@ -181,49 +179,17 @@ case class RelationConversions( conf: SQLConf, sessionCatalog: HiveSessionCatalog) extends Rule[LogicalPlan] { private def isConvertible(relation: HiveTableRelation): Boolean = { - val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) - serde.contains("parquet") && conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) || - serde.contains("orc") && conf.getConf(HiveUtils.CONVERT_METASTORE_ORC) + isConvertible(relation.tableMeta) } - // Return true for Apache ORC and Hive ORC-related configuration names. - // Note that Spark doesn't support configurations like `hive.merge.orcfile.stripe.level`. - private def isOrcProperty(key: String) = - key.startsWith("orc.") || key.contains(".orc.") - - private def isParquetProperty(key: String) = - key.startsWith("parquet.") || key.contains(".parquet.") - - private def convert(relation: HiveTableRelation): LogicalRelation = { - val serde = relation.tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) - - // Consider table and storage properties. For properties existing in both sides, storage - // properties will supersede table properties. - if (serde.contains("parquet")) { - val options = relation.tableMeta.properties.filterKeys(isParquetProperty) ++ - relation.tableMeta.storage.properties + (ParquetOptions.MERGE_SCHEMA -> - conf.getConf(HiveUtils.CONVERT_METASTORE_PARQUET_WITH_SCHEMA_MERGING).toString) - sessionCatalog.metastoreCatalog - .convertToLogicalRelation(relation, options, classOf[ParquetFileFormat], "parquet") - } else { - val options = relation.tableMeta.properties.filterKeys(isOrcProperty) ++ - relation.tableMeta.storage.properties - if (conf.getConf(SQLConf.ORC_IMPLEMENTATION) == "native") { - sessionCatalog.metastoreCatalog.convertToLogicalRelation( - relation, - options, - classOf[org.apache.spark.sql.execution.datasources.orc.OrcFileFormat], - "orc") - } else { - sessionCatalog.metastoreCatalog.convertToLogicalRelation( - relation, - options, - classOf[org.apache.spark.sql.hive.orc.OrcFileFormat], - "orc") - } - } + private def isConvertible(tableMeta: CatalogTable): Boolean = { + val serde = tableMeta.storage.serde.getOrElse("").toLowerCase(Locale.ROOT) + serde.contains("parquet") && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_PARQUET) || + serde.contains("orc") && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_ORC) } + private val metastoreCatalog = sessionCatalog.metastoreCatalog + override def apply(plan: LogicalPlan): LogicalPlan = { plan resolveOperators { // Write path @@ -231,12 +197,22 @@ case class RelationConversions( // Inserting into partitioned table is not supported in Parquet/Orc data source (yet). if query.resolved && DDLUtils.isHiveTable(r.tableMeta) && !r.isPartitioned && isConvertible(r) => - InsertIntoTable(convert(r), partition, query, overwrite, ifPartitionNotExists) + InsertIntoTable(metastoreCatalog.convert(r), partition, + query, overwrite, ifPartitionNotExists) // Read path case relation: HiveTableRelation if DDLUtils.isHiveTable(relation.tableMeta) && isConvertible(relation) => - convert(relation) + metastoreCatalog.convert(relation) + + // CTAS + case CreateTable(tableDesc, mode, Some(query)) + if DDLUtils.isHiveTable(tableDesc) && tableDesc.partitionColumnNames.isEmpty && + isConvertible(tableDesc) && SQLConf.get.getConf(HiveUtils.CONVERT_METASTORE_CTAS) => + // validation is required to be done here before relation conversion. + DDLUtils.checkDataColNames(tableDesc.copy(schema = query.schema)) + OptimizedCreateHiveTableAsSelectCommand( + tableDesc, query, query.output.map(_.name), mode) } } } 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 cd321d41f43e8..c3ae3d58a351e 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 @@ -19,8 +19,6 @@ package org.apache.spark.sql.hive import java.io.File import java.net.{URL, URLClassLoader} -import java.nio.charset.StandardCharsets -import java.sql.Timestamp import java.util.Locale import java.util.concurrent.TimeUnit @@ -28,13 +26,13 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.HashMap import scala.language.implicitConversions +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.hive.common.`type`.HiveDecimal import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.session.SessionState -import org.apache.hadoop.hive.serde2.io.{DateWritable, TimestampWritable} import org.apache.hadoop.util.VersionInfo +import org.apache.hive.common.util.HiveVersionInfo import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil @@ -57,12 +55,16 @@ private[spark] object HiveUtils extends Logging { sc } + private val hiveVersion = HiveVersionInfo.getVersion + val isHive23: Boolean = hiveVersion.startsWith("2.3") + /** The version of hive used internally by Spark SQL. */ - val builtinHiveVersion: String = "1.2.1" + val builtinHiveVersion: String = if (isHive23) hiveVersion else "1.2.1" val HIVE_METASTORE_VERSION = buildConf("spark.sql.hive.metastore.version") .doc("Version of the Hive metastore. Available options are " + - s"0.12.0 through 2.3.3.") + "0.12.0 through 2.3.5 and " + + "3.0.0 through 3.1.1.") .stringConf .createWithDefault(builtinHiveVersion) @@ -110,6 +112,14 @@ private[spark] object HiveUtils extends Logging { .booleanConf .createWithDefault(true) + val CONVERT_METASTORE_CTAS = buildConf("spark.sql.hive.convertMetastoreCtas") + .doc("When set to true, Spark will try to use built-in data source writer " + + "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") + .booleanConf + .createWithDefault(true) + val HIVE_METASTORE_SHARED_PREFIXES = buildConf("spark.sql.hive.metastore.sharedPrefixes") .doc("A comma separated list of class prefixes that should be loaded using the classloader " + "that is shared between Spark SQL and a specific version of Hive. An example of classes " + @@ -188,7 +198,7 @@ private[spark] object HiveUtils extends Logging { // // Here we enumerate all time `ConfVar`s and convert their values to numeric strings according // to their output time units. - Seq( + val commonTimeVars = Seq( ConfVars.METASTORE_CLIENT_CONNECT_RETRY_DELAY -> TimeUnit.SECONDS, ConfVars.METASTORE_CLIENT_SOCKET_TIMEOUT -> TimeUnit.SECONDS, ConfVars.METASTORE_CLIENT_SOCKET_LIFETIME -> TimeUnit.SECONDS, @@ -201,8 +211,6 @@ private[spark] object HiveUtils extends Logging { ConfVars.METASTORE_AGGREGATE_STATS_CACHE_MAX_READER_WAIT -> TimeUnit.MILLISECONDS, ConfVars.HIVES_AUTO_PROGRESS_TIMEOUT -> TimeUnit.SECONDS, ConfVars.HIVE_LOG_INCREMENTAL_PLAN_PROGRESS_INTERVAL -> TimeUnit.MILLISECONDS, - ConfVars.HIVE_STATS_JDBC_TIMEOUT -> TimeUnit.SECONDS, - ConfVars.HIVE_STATS_RETRIES_WAIT -> TimeUnit.MILLISECONDS, ConfVars.HIVE_LOCK_SLEEP_BETWEEN_RETRIES -> TimeUnit.SECONDS, ConfVars.HIVE_ZOOKEEPER_SESSION_TIMEOUT -> TimeUnit.MILLISECONDS, ConfVars.HIVE_ZOOKEEPER_CONNECTION_BASESLEEPTIME -> TimeUnit.MILLISECONDS, @@ -230,7 +238,18 @@ private[spark] object HiveUtils extends Logging { ConfVars.SPARK_RPC_CLIENT_HANDSHAKE_TIMEOUT -> TimeUnit.MILLISECONDS ).map { case (confVar, unit) => confVar.varname -> HiveConf.getTimeVar(hadoopConf, confVar, unit).toString - }.toMap + } + + // The following configurations were removed by HIVE-12164(Hive 2.0) + val hardcodingTimeVars = Seq( + ("hive.stats.jdbc.timeout", "30s") -> TimeUnit.SECONDS, + ("hive.stats.retries.wait", "3000ms") -> TimeUnit.MILLISECONDS + ).map { case ((key, defaultValue), unit) => + val value = hadoopConf.get(key, defaultValue) + key -> HiveConf.toTime(value, unit, unit).toString + } + + (commonTimeVars ++ hardcodingTimeVars).toMap } /** @@ -252,7 +271,7 @@ private[spark] object HiveUtils extends Logging { /** * Create a [[HiveClient]] used for execution. * - * Currently this must always be Hive 13 as this is the version of Hive that is packaged + * Currently this must always be Hive 1.2.1 as this is the version of Hive that is packaged * with Spark SQL. This copy of the client is used for execution related tasks like * registering temporary functions or ensuring that the ThreadLocal SessionState is * correctly populated. This copy of Hive is *not* used for storing persistent metadata, @@ -320,10 +339,17 @@ private[spark] object HiveUtils extends Logging { val classLoader = Utils.getContextOrSparkClassLoader val jars = allJars(classLoader) - if (jars.length == 0) { - throw new IllegalArgumentException( - "Unable to locate hive jars to connect to metastore. " + - s"Please set ${HIVE_METASTORE_JARS.key}.") + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + // Do nothing. The system classloader is no longer a URLClassLoader in Java 9, + // so it won't match the case in allJars above. It no longer exposes URLs of + // the system classpath + } else { + // Verify at least one jar was found + if (jars.length == 0) { + throw new IllegalArgumentException( + "Unable to locate hive jars to connect to metastore. " + + s"Please set ${HIVE_METASTORE_JARS.key}.") + } } logInfo( @@ -436,52 +462,6 @@ private[spark] object HiveUtils extends Logging { propMap.toMap } - protected val primitiveTypes = - Seq(StringType, IntegerType, LongType, DoubleType, FloatType, BooleanType, ByteType, - ShortType, DateType, TimestampType, BinaryType) - - protected[sql] def toHiveString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "NULL" - case (d: Int, DateType) => new DateWritable(d).toString - case (t: Timestamp, TimestampType) => new TimestampWritable(t).toString - case (bin: Array[Byte], BinaryType) => new String(bin, StandardCharsets.UTF_8) - case (decimal: java.math.BigDecimal, DecimalType()) => - // Hive strips trailing zeros so use its toString - HiveDecimal.create(decimal).toString - case (other, _ : UserDefinedType[_]) => other.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - - /** Hive outputs fields of structs slightly differently than top level attributes. */ - protected def toHiveStructString(a: (Any, DataType)): String = a match { - case (struct: Row, StructType(fields)) => - struct.toSeq.zip(fields).map { - case (v, t) => s""""${t.name}":${toHiveStructString((v, t.dataType))}""" - }.mkString("{", ",", "}") - case (seq: Seq[_], ArrayType(typ, _)) => - seq.map(v => (v, typ)).map(toHiveStructString).mkString("[", ",", "]") - case (map: Map[_, _], MapType(kType, vType, _)) => - map.map { - case (key, value) => - toHiveStructString((key, kType)) + ":" + toHiveStructString((value, vType)) - }.toSeq.sorted.mkString("{", ",", "}") - case (null, _) => "null" - case (s: String, StringType) => "\"" + s + "\"" - case (decimal, DecimalType()) => decimal.toString - case (other, tpe) if primitiveTypes contains tpe => other.toString - } - /** * Infers the schema for Hive serde tables and returns the CatalogTable with the inferred schema. * When the tables are data source tables or the schema already exists, returns the original diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 9443fbb4330a5..3f9925e73705e 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -31,12 +31,13 @@ import org.apache.hadoop.hive.serde2.Deserializer import org.apache.hadoop.hive.serde2.objectinspector.{ObjectInspectorConverters, StructObjectInspector} import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.io.Writable -import org.apache.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} +import org.apache.hadoop.mapred.{FileInputFormat, InputFormat => oldInputClass, JobConf} +import org.apache.hadoop.mapreduce.{InputFormat => newInputClass} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging -import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, NewHadoopRDD, RDD, UnionRDD} import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.CastSupport @@ -93,7 +94,7 @@ class HadoopTableReader( override def makeRDDForTable(hiveTable: HiveTable): RDD[InternalRow] = makeRDDForTable( hiveTable, - Utils.classForName(tableDesc.getSerdeClassName).asInstanceOf[Class[Deserializer]], + Utils.classForName[Deserializer](tableDesc.getSerdeClassName), filterOpt = None) /** @@ -123,16 +124,14 @@ class HadoopTableReader( val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) // logDebug("Table input: %s".format(tablePath)) - val ifc = hiveTable.getInputFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] - val hadoopRDD = createHadoopRdd(localTableDesc, inputPathStr, ifc) + val hadoopRDD = createHadoopRDD(localTableDesc, inputPathStr) val attrsWithIndex = attributes.zipWithIndex val mutableRow = new SpecificInternalRow(attributes.map(_.dataType)) val deserializedHadoopRDD = hadoopRDD.mapPartitions { iter => val hconf = broadcastedHadoopConf.value.value - val deserializer = deserializerClass.newInstance() + val deserializer = deserializerClass.getConstructor().newInstance() deserializer.initialize(hconf, localTableDesc.getProperties) HadoopTableReader.fillObject(iter, deserializer, attrsWithIndex, mutableRow, deserializer) } @@ -164,7 +163,7 @@ class HadoopTableReader( def verifyPartitionPath( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]]): Map[HivePartition, Class[_ <: Deserializer]] = { - if (!sparkSession.sessionState.conf.verifyPartitionPath) { + if (!conf.verifyPartitionPath) { partitionToDeserializer } else { val existPathSet = collection.mutable.Set[String]() @@ -202,8 +201,6 @@ class HadoopTableReader( val partDesc = Utilities.getPartitionDesc(partition) val partPath = partition.getDataLocation val inputPathStr = applyFilterIfNeeded(partPath, filterOpt) - val ifc = partDesc.getInputFileFormatClass - .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] // Get partition field info val partSpec = partDesc.getPartSpec val partProps = partDesc.getProperties @@ -243,9 +240,9 @@ class HadoopTableReader( // Create local references so that the outer object isn't serialized. val localTableDesc = tableDesc - createHadoopRdd(localTableDesc, inputPathStr, ifc).mapPartitions { iter => + createHadoopRDD(localTableDesc, inputPathStr).mapPartitions { iter => val hconf = broadcastedHiveConf.value.value - val deserializer = localDeserializer.newInstance() + val deserializer = localDeserializer.getConstructor().newInstance() // SPARK-13709: For SerDes like AvroSerDe, some essential information (e.g. Avro schema // information) may be defined in table properties. Here we should merge table properties // and partition properties before initializing the deserializer. Note that partition @@ -257,7 +254,7 @@ class HadoopTableReader( } deserializer.initialize(hconf, props) // get the table deserializer - val tableSerDe = localTableDesc.getDeserializerClass.newInstance() + val tableSerDe = localTableDesc.getDeserializerClass.getConstructor().newInstance() tableSerDe.initialize(hconf, localTableDesc.getProperties) // fill the non partition key attributes @@ -288,16 +285,29 @@ class HadoopTableReader( } } + /** + * The entry of creating a RDD. + * [SPARK-26630] Using which HadoopRDD will be decided by the input format of tables. + * The input format of NewHadoopRDD is from `org.apache.hadoop.mapreduce` package while + * the input format of HadoopRDD is from `org.apache.hadoop.mapred` package. + */ + private def createHadoopRDD(localTableDesc: TableDesc, inputPathStr: String): RDD[Writable] = { + val inputFormatClazz = localTableDesc.getInputFileFormatClass + if (classOf[newInputClass[_, _]].isAssignableFrom(inputFormatClazz)) { + createNewHadoopRDD(localTableDesc, inputPathStr) + } else { + createOldHadoopRDD(localTableDesc, inputPathStr) + } + } + /** * Creates a HadoopRDD based on the broadcasted HiveConf and other job properties that will be * applied locally on each slave. */ - private def createHadoopRdd( - tableDesc: TableDesc, - path: String, - inputFormatClass: Class[InputFormat[Writable, Writable]]): RDD[Writable] = { - + private def createOldHadoopRDD(tableDesc: TableDesc, path: String): RDD[Writable] = { val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ + val inputFormatClass = tableDesc.getInputFileFormatClass + .asInstanceOf[Class[oldInputClass[Writable, Writable]]] val rdd = new HadoopRDD( sparkSession.sparkContext, @@ -311,6 +321,29 @@ class HadoopTableReader( // Only take the value (skip the key) because Hive works only with values. rdd.map(_._2) } + + /** + * Creates a NewHadoopRDD based on the broadcasted HiveConf and other job properties that will be + * applied locally on each slave. + */ + private def createNewHadoopRDD(tableDesc: TableDesc, path: String): RDD[Writable] = { + val newJobConf = new JobConf(hadoopConf) + HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc)(newJobConf) + val inputFormatClass = tableDesc.getInputFileFormatClass + .asInstanceOf[Class[newInputClass[Writable, Writable]]] + + val rdd = new NewHadoopRDD( + sparkSession.sparkContext, + inputFormatClass, + classOf[Writable], + classOf[Writable], + newJobConf + ) + + // Only take the value (skip the key) because Hive works only with values. + rdd.map(_._2) + } + } private[hive] object HiveTableUtil { 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 f69717441d615..cb015d7301c19 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 @@ -41,7 +41,7 @@ private[hive] trait HiveClient { /** * Return the associated Hive SessionState of this [[HiveClientImpl]] - * @return [[Any]] not SessionState to avoid linkage error + * @return `Any` not SessionState to avoid linkage error */ def getState: Any @@ -76,7 +76,7 @@ private[hive] trait HiveClient { /** Return whether a table/view with the specified name exists. */ def tableExists(dbName: String, tableName: String): Boolean - /** Returns the specified table, or throws [[NoSuchTableException]]. */ + /** Returns the specified table, or throws `NoSuchTableException`. */ final def getTable(dbName: String, tableName: String): CatalogTable = { getTableOption(dbName, tableName).getOrElse(throw new NoSuchTableException(dbName, tableName)) } @@ -84,6 +84,9 @@ private[hive] trait HiveClient { /** Returns the metadata for the specified table or None if it doesn't exist. */ def getTableOption(dbName: String, tableName: String): Option[CatalogTable] + /** Returns metadata of existing permanent tables/views for given names. */ + def getTablesByName(dbName: String, tableNames: Seq[String]): Seq[CatalogTable] + /** Creates a table with the given metadata. */ def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit @@ -166,7 +169,7 @@ private[hive] trait HiveClient { table: String, newParts: Seq[CatalogTablePartition]): Unit - /** Returns the specified partition, or throws [[NoSuchPartitionException]]. */ + /** Returns the specified partition, or throws `NoSuchPartitionException`. */ final def getPartition( dbName: String, tableName: String, 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 5e9b324a168e0..fc05d93385253 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.client import java.io.{File, PrintStream} import java.lang.{Iterable => JIterable} import java.util.{Locale, Map => JMap} +import java.util.concurrent.TimeUnit._ import scala.collection.JavaConverters._ import scala.collection.mutable @@ -29,14 +30,17 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.common.StatsSetupConst import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars -import org.apache.hadoop.hive.metastore.{TableType => HiveTableType} -import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, FieldSchema, Order} -import org.apache.hadoop.hive.metastore.api.{SerDeInfo, StorageDescriptor} +import org.apache.hadoop.hive.metastore.{IMetaStoreClient, TableType => HiveTableType} +import org.apache.hadoop.hive.metastore.api.{Database => HiveDatabase, Table => MetaStoreApiTable} +import org.apache.hadoop.hive.metastore.api.{FieldSchema, Order, SerDeInfo, StorageDescriptor} import org.apache.hadoop.hive.ql.Driver -import org.apache.hadoop.hive.ql.metadata.{Hive, Partition => HivePartition, Table => HiveTable} +import org.apache.hadoop.hive.ql.metadata.{Hive, HiveException, Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.parse.BaseSemanticAnalyzer.HIVE_COLUMN_ORDER_ASC import org.apache.hadoop.hive.ql.processors._ import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde.serdeConstants +import org.apache.hadoop.hive.serde2.MetadataTypedColumnsetSerDe +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.internal.Logging @@ -51,6 +55,7 @@ import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParseException} import org.apache.spark.sql.execution.QueryExecutionException import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.hive.HiveExternalCatalog.{DATASOURCE_SCHEMA, DATASOURCE_SCHEMA_NUMPARTS, DATASOURCE_SCHEMA_PART_PREFIX} +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.client.HiveClientImpl._ import org.apache.spark.sql.types._ import org.apache.spark.util.{CircularBuffer, Utils} @@ -105,6 +110,8 @@ private[hive] class HiveClientImpl( case hive.v2_1 => new Shim_v2_1() case hive.v2_2 => new Shim_v2_2() case hive.v2_3 => new Shim_v2_3() + case hive.v3_0 => new Shim_v3_0() + case hive.v3_1 => new Shim_v3_1() } // Create an internal session state for this HiveClientImpl. @@ -176,6 +183,8 @@ private[hive] class HiveClientImpl( """.stripMargin) hiveConf.set(k, v) } + // Disable CBO because we removed the Calcite dependency. + hiveConf.setBoolean("hive.cbo.enable", false) val state = new SessionState(hiveConf) if (clientLoader.cachedHive != null) { Hive.set(clientLoader.cachedHive.asInstanceOf[Hive]) @@ -187,7 +196,29 @@ private[hive] class HiveClientImpl( } /** Returns the configuration for the current session. */ - def conf: HiveConf = state.getConf + def conf: HiveConf = if (!HiveUtils.isHive23) { + state.getConf + } else { + val hiveConf = state.getConf + // Hive changed the default of datanucleus.schema.autoCreateAll from true to false + // and hive.metastore.schema.verification from false to true since Hive 2.0. + // For details, see the JIRA HIVE-6113, HIVE-12463 and HIVE-1841. + // isEmbeddedMetaStore should not be true in the production environment. + // We hard-code hive.metastore.schema.verification and datanucleus.schema.autoCreateAll to allow + // bin/spark-shell, bin/spark-sql and sbin/start-thriftserver.sh to automatically create the + // Derby Metastore when running Spark in the non-production environment. + val isEmbeddedMetaStore = { + val msUri = hiveConf.getVar(ConfVars.METASTOREURIS) + val msConnUrl = hiveConf.getVar(ConfVars.METASTORECONNECTURLKEY) + (msUri == null || msUri.trim().isEmpty) && + (msConnUrl != null && msConnUrl.startsWith("jdbc:derby")) + } + if (isEmbeddedMetaStore) { + hiveConf.setBoolean("hive.metastore.schema.verification", false) + hiveConf.setBoolean("datanucleus.schema.autoCreateAll", true) + } + hiveConf + } private val userName = conf.getUser @@ -249,6 +280,10 @@ private[hive] class HiveClientImpl( } } + private def msClient: IMetaStoreClient = { + shim.getMSC(client) + } + /** Return the associated Hive [[SessionState]] of this [[HiveClientImpl]] */ override def getState: SessionState = withHiveState(state) @@ -357,126 +392,145 @@ private[hive] class HiveClientImpl( Option(client.getTable(dbName, tableName, false /* do not throw exception */)) } + private def getRawTablesByName(dbName: String, tableNames: Seq[String]): Seq[HiveTable] = { + try { + msClient.getTableObjectsByName(dbName, tableNames.asJava).asScala + .map(extraFixesForNonView).map(new HiveTable(_)) + } catch { + case ex: Exception => + throw new HiveException(s"Unable to fetch tables of db $dbName", ex); + } + } + override def tableExists(dbName: String, tableName: String): Boolean = withHiveState { getRawTableOption(dbName, tableName).nonEmpty } + override def getTablesByName( + dbName: String, + tableNames: Seq[String]): Seq[CatalogTable] = withHiveState { + getRawTablesByName(dbName, tableNames).map(convertHiveTableToCatalogTable) + } + override def getTableOption( dbName: String, tableName: String): Option[CatalogTable] = withHiveState { logDebug(s"Looking up $dbName.$tableName") - getRawTableOption(dbName, tableName).map { h => - // Note: Hive separates partition columns and the schema, but for us the - // partition columns are part of the schema - val cols = h.getCols.asScala.map(fromHiveColumn) - val partCols = h.getPartCols.asScala.map(fromHiveColumn) - val schema = StructType(cols ++ partCols) - - val bucketSpec = if (h.getNumBuckets > 0) { - val sortColumnOrders = h.getSortCols.asScala - // Currently Spark only supports columns to be sorted in ascending order - // but Hive can support both ascending and descending order. If all the columns - // are sorted in ascending order, only then propagate the sortedness information - // to downstream processing / optimizations in Spark - // TODO: In future we can have Spark support columns sorted in descending order - val allAscendingSorted = sortColumnOrders.forall(_.getOrder == HIVE_COLUMN_ORDER_ASC) - - val sortColumnNames = if (allAscendingSorted) { - sortColumnOrders.map(_.getCol) - } else { - Seq.empty - } - Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala, sortColumnNames)) + getRawTableOption(dbName, tableName).map(convertHiveTableToCatalogTable) + } + + private def convertHiveTableToCatalogTable(h: HiveTable): CatalogTable = { + // Note: Hive separates partition columns and the schema, but for us the + // partition columns are part of the schema + val cols = h.getCols.asScala.map(fromHiveColumn) + val partCols = h.getPartCols.asScala.map(fromHiveColumn) + val schema = StructType(cols ++ partCols) + + val bucketSpec = if (h.getNumBuckets > 0) { + val sortColumnOrders = h.getSortCols.asScala + // Currently Spark only supports columns to be sorted in ascending order + // but Hive can support both ascending and descending order. If all the columns + // are sorted in ascending order, only then propagate the sortedness information + // to downstream processing / optimizations in Spark + // TODO: In future we can have Spark support columns sorted in descending order + val allAscendingSorted = sortColumnOrders.forall(_.getOrder == HIVE_COLUMN_ORDER_ASC) + + val sortColumnNames = if (allAscendingSorted) { + sortColumnOrders.map(_.getCol) } else { - None + Seq.empty } + Option(BucketSpec(h.getNumBuckets, h.getBucketCols.asScala, sortColumnNames)) + } else { + None + } - // Skew spec and storage handler can't be mapped to CatalogTable (yet) - val unsupportedFeatures = ArrayBuffer.empty[String] + // Skew spec and storage handler can't be mapped to CatalogTable (yet) + val unsupportedFeatures = ArrayBuffer.empty[String] - if (!h.getSkewedColNames.isEmpty) { - unsupportedFeatures += "skewed columns" - } + if (!h.getSkewedColNames.isEmpty) { + unsupportedFeatures += "skewed columns" + } - if (h.getStorageHandler != null) { - unsupportedFeatures += "storage handler" - } + if (h.getStorageHandler != null) { + unsupportedFeatures += "storage handler" + } - if (h.getTableType == HiveTableType.VIRTUAL_VIEW && partCols.nonEmpty) { - unsupportedFeatures += "partitioned view" - } + if (h.getTableType == HiveTableType.VIRTUAL_VIEW && partCols.nonEmpty) { + unsupportedFeatures += "partitioned view" + } - val properties = Option(h.getParameters).map(_.asScala.toMap).orNull + val properties = Option(h.getParameters).map(_.asScala.toMap).orNull - // Hive-generated Statistics are also recorded in ignoredProperties - val ignoredProperties = scala.collection.mutable.Map.empty[String, String] - for (key <- HiveStatisticsProperties; value <- properties.get(key)) { - ignoredProperties += key -> value - } + // Hive-generated Statistics are also recorded in ignoredProperties + val ignoredProperties = scala.collection.mutable.Map.empty[String, String] + for (key <- HiveStatisticsProperties; value <- properties.get(key)) { + ignoredProperties += key -> value + } - val excludedTableProperties = HiveStatisticsProperties ++ Set( - // The property value of "comment" is moved to the dedicated field "comment" - "comment", - // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added - // in the function toHiveTable. - "EXTERNAL" - ) + val excludedTableProperties = HiveStatisticsProperties ++ Set( + // The property value of "comment" is moved to the dedicated field "comment" + "comment", + // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added + // in the function toHiveTable. + "EXTERNAL" + ) - val filteredProperties = properties.filterNot { - case (key, _) => excludedTableProperties.contains(key) - } - val comment = properties.get("comment") - - CatalogTable( - identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), - tableType = h.getTableType match { - case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL - case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED - case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIEW - case HiveTableType.INDEX_TABLE => - throw new AnalysisException("Hive index table is not supported.") - }, - schema = schema, - partitionColumnNames = partCols.map(_.name), - // If the table is written by Spark, we will put bucketing information in table properties, - // and will always overwrite the bucket spec in hive metastore by the bucketing information - // in table properties. This means, if we have bucket spec in both hive metastore and - // table properties, we will trust the one in table properties. - bucketSpec = bucketSpec, - owner = Option(h.getOwner).getOrElse(""), - createTime = h.getTTable.getCreateTime.toLong * 1000, - lastAccessTime = h.getLastAccessTime.toLong * 1000, - storage = CatalogStorageFormat( - locationUri = shim.getDataLocation(h).map(CatalogUtils.stringToURI), - // To avoid ClassNotFound exception, we try our best to not get the format class, but get - // the class name directly. However, for non-native tables, there is no interface to get - // the format class name, so we may still throw ClassNotFound in this case. - inputFormat = Option(h.getTTable.getSd.getInputFormat).orElse { - Option(h.getStorageHandler).map(_.getInputFormatClass.getName) - }, - outputFormat = Option(h.getTTable.getSd.getOutputFormat).orElse { - Option(h.getStorageHandler).map(_.getOutputFormatClass.getName) - }, - serde = Option(h.getSerializationLib), - compressed = h.getTTable.getSd.isCompressed, - properties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) - .map(_.asScala.toMap).orNull - ), - // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added - // in the function toHiveTable. - properties = filteredProperties, - stats = readHiveStats(properties), - comment = comment, - // In older versions of Spark(before 2.2.0), we expand the view original text and - // store that into `viewExpandedText`, that should be used in view resolution. - // We get `viewExpandedText` as viewText, and also get `viewOriginalText` in order to - // display the original view text in `DESC [EXTENDED|FORMATTED] table` command for views - // that created by older versions of Spark. - viewOriginalText = Option(h.getViewOriginalText), - viewText = Option(h.getViewExpandedText), - unsupportedFeatures = unsupportedFeatures, - ignoredProperties = ignoredProperties.toMap) + val filteredProperties = properties.filterNot { + case (key, _) => excludedTableProperties.contains(key) } + val comment = properties.get("comment") + + CatalogTable( + identifier = TableIdentifier(h.getTableName, Option(h.getDbName)), + tableType = h.getTableType match { + case HiveTableType.EXTERNAL_TABLE => CatalogTableType.EXTERNAL + case HiveTableType.MANAGED_TABLE => CatalogTableType.MANAGED + case HiveTableType.VIRTUAL_VIEW => CatalogTableType.VIEW + case unsupportedType => + val tableTypeStr = unsupportedType.toString.toLowerCase(Locale.ROOT).replace("_", " ") + throw new AnalysisException(s"Hive $tableTypeStr is not supported.") + }, + schema = schema, + partitionColumnNames = partCols.map(_.name), + // If the table is written by Spark, we will put bucketing information in table properties, + // and will always overwrite the bucket spec in hive metastore by the bucketing information + // in table properties. This means, if we have bucket spec in both hive metastore and + // table properties, we will trust the one in table properties. + bucketSpec = bucketSpec, + owner = Option(h.getOwner).getOrElse(""), + createTime = h.getTTable.getCreateTime.toLong * 1000, + lastAccessTime = h.getLastAccessTime.toLong * 1000, + storage = CatalogStorageFormat( + locationUri = shim.getDataLocation(h).map(CatalogUtils.stringToURI), + // To avoid ClassNotFound exception, we try our best to not get the format class, but get + // the class name directly. However, for non-native tables, there is no interface to get + // the format class name, so we may still throw ClassNotFound in this case. + inputFormat = Option(h.getTTable.getSd.getInputFormat).orElse { + Option(h.getStorageHandler).map(_.getInputFormatClass.getName) + }, + outputFormat = Option(h.getTTable.getSd.getOutputFormat).orElse { + Option(h.getStorageHandler).map(_.getOutputFormatClass.getName) + }, + serde = Option(h.getSerializationLib), + compressed = h.getTTable.getSd.isCompressed, + properties = Option(h.getTTable.getSd.getSerdeInfo.getParameters) + .map(_.asScala.toMap).orNull + ), + // For EXTERNAL_TABLE, the table properties has a particular field "EXTERNAL". This is added + // in the function toHiveTable. + properties = filteredProperties, + stats = readHiveStats(properties), + comment = comment, + // In older versions of Spark(before 2.2.0), we expand the view original text and + // store that into `viewExpandedText`, that should be used in view resolution. + // We get `viewExpandedText` as viewText, and also get `viewOriginalText` in order to + // display the original view text in `DESC [EXTENDED|FORMATTED] table` command for views + // that created by older versions of Spark. + viewOriginalText = Option(h.getViewOriginalText), + viewText = Option(h.getViewExpandedText), + unsupportedFeatures = unsupportedFeatures, + ignoredProperties = ignoredProperties.toMap) } override def createTable(table: CatalogTable, ignoreIfExists: Boolean): Unit = withHiveState { @@ -705,8 +759,20 @@ private[hive] class HiveClientImpl( /** * Execute the command using Hive and return the results as a sequence. Each element * in the sequence is one row. + * Since upgrading the built-in Hive to 2.3, hive-llap-client is needed when + * running MapReduce jobs with `runHive`. + * Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false. */ protected def runHive(cmd: String, maxRows: Int = 1000): Seq[String] = withHiveState { + def closeDriver(driver: Driver): Unit = { + // Since HIVE-18238(Hive 3.0.0), the Driver.close function's return type changed + // and the CommandProcessorFactory.clean function removed. + driver.getClass.getMethod("close").invoke(driver) + if (version != hive.v3_0 && version != hive.v3_1) { + CommandProcessorFactory.clean(conf) + } + } + logDebug(s"Running hiveql '$cmd'") if (cmd.toLowerCase(Locale.ROOT).startsWith("set")) { logDebug(s"Changing config: $cmd") } try { @@ -720,15 +786,13 @@ private[hive] class HiveClientImpl( val response: CommandProcessorResponse = driver.run(cmd) // Throw an exception if there is an error in query processing. if (response.getResponseCode != 0) { - driver.close() - CommandProcessorFactory.clean(conf) + closeDriver(driver) throw new QueryExecutionException(response.getErrorMessage) } driver.setMaxRows(maxRows) val results = shim.getDriverResults(driver) - driver.close() - CommandProcessorFactory.clean(conf) + closeDriver(driver) results case _ => @@ -849,14 +913,31 @@ private[hive] class HiveClientImpl( } def reset(): Unit = withHiveState { - client.getAllTables("default").asScala.foreach { t => + val allTables = client.getAllTables("default") + val (mvs, others) = allTables.asScala.map(t => client.getTable("default", t)) + .partition(_.getTableType.toString.equals("MATERIALIZED_VIEW")) + + // Remove materialized view first, otherwise caused a violation of foreign key constraint. + mvs.foreach { table => + val t = table.getTableName + logDebug(s"Deleting materialized view $t") + client.dropTable("default", t) + } + + others.foreach { table => + val t = table.getTableName logDebug(s"Deleting table $t") - val table = client.getTable("default", t) - client.getIndexes("default", t, 255).asScala.foreach { index => - shim.dropIndex(client, "default", t, index.getIndexName) - } - if (!table.isIndexTable) { - client.dropTable("default", t) + try { + client.getIndexes("default", t, 255).asScala.foreach { index => + shim.dropIndex(client, "default", t, index.getIndexName) + } + if (!table.isIndexTable) { + client.dropTable("default", t) + } + } catch { + case _: NoSuchMethodError => + // HIVE-18448 Hive 3.0 remove index APIs + client.dropTable("default", t) } } client.getAllDatabases.asScala.filterNot(_ == "default").foreach { db => @@ -909,11 +990,10 @@ private[hive] object HiveClientImpl { } private def toInputFormat(name: String) = - Utils.classForName(name).asInstanceOf[Class[_ <: org.apache.hadoop.mapred.InputFormat[_, _]]] + Utils.classForName[org.apache.hadoop.mapred.InputFormat[_, _]](name) private def toOutputFormat(name: String) = - Utils.classForName(name) - .asInstanceOf[Class[_ <: org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]]] + Utils.classForName[org.apache.hadoop.hive.ql.io.HiveOutputFormat[_, _]](name) /** * Converts the native table metadata representation format CatalogTable to Hive's Table. @@ -941,8 +1021,8 @@ private[hive] object HiveClientImpl { hiveTable.setFields(schema.asJava) hiveTable.setPartCols(partCols.asJava) userName.foreach(hiveTable.setOwner) - hiveTable.setCreateTime((table.createTime / 1000).toInt) - hiveTable.setLastAccessTime((table.lastAccessTime / 1000).toInt) + hiveTable.setCreateTime(MILLISECONDS.toSeconds(table.createTime).toInt) + hiveTable.setLastAccessTime(MILLISECONDS.toSeconds(table.lastAccessTime).toInt) table.storage.locationUri.map(CatalogUtils.URIToString).foreach { loc => hiveTable.getTTable.getSd.setLocation(loc)} table.storage.inputFormat.map(toInputFormat).foreach(hiveTable.setInputFormatClass) @@ -988,10 +1068,8 @@ private[hive] object HiveClientImpl { ht: HiveTable): HivePartition = { val tpart = new org.apache.hadoop.hive.metastore.api.Partition val partValues = ht.getPartCols.asScala.map { hc => - p.spec.get(hc.getName).getOrElse { - throw new IllegalArgumentException( - s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}") - } + p.spec.getOrElse(hc.getName, throw new IllegalArgumentException( + s"Partition spec is missing a value for column '${hc.getName}': ${p.spec}")) } val storageDesc = new StorageDescriptor val serdeInfo = new SerDeInfo @@ -1005,8 +1083,8 @@ private[hive] object HiveClientImpl { tpart.setTableName(ht.getTableName) tpart.setValues(partValues.asJava) tpart.setSd(storageDesc) - tpart.setCreateTime((p.createTime / 1000).toInt) - tpart.setLastAccessTime((p.lastAccessTime / 1000).toInt) + tpart.setCreateTime(MILLISECONDS.toSeconds(p.createTime).toInt) + tpart.setLastAccessTime(MILLISECONDS.toSeconds(p.lastAccessTime).toInt) tpart.setParameters(mutable.Map(p.parameters.toSeq: _*).asJava) new HivePartition(ht, tpart) } @@ -1037,6 +1115,40 @@ private[hive] object HiveClientImpl { stats = readHiveStats(properties)) } + /** + * This is the same process copied from the method `getTable()` + * of [[org.apache.hadoop.hive.ql.metadata.Hive]] to do some extra fixes for non-views. + * Methods of extracting multiple [[HiveTable]] like `getRawTablesByName()` + * should invoke this before return. + */ + def extraFixesForNonView(tTable: MetaStoreApiTable): MetaStoreApiTable = { + // For non-views, we need to do some extra fixes + if (!(HiveTableType.VIRTUAL_VIEW.toString == tTable.getTableType)) { + // Fix the non-printable chars + val parameters = tTable.getSd.getParameters + if (parameters != null) { + val sf = parameters.get(serdeConstants.SERIALIZATION_FORMAT) + if (sf != null) { + val b: Array[Char] = sf.toCharArray + if ((b.length == 1) && (b(0) < 10)) { // ^A, ^B, ^C, ^D, \t + parameters.put(serdeConstants.SERIALIZATION_FORMAT, Integer.toString(b(0))) + } + } + } + // Use LazySimpleSerDe for MetadataTypedColumnsetSerDe. + // NOTE: LazySimpleSerDe does not support tables with a single column of col + // of type "array". This happens when the table is created using + // an earlier version of Hive. + if (classOf[MetadataTypedColumnsetSerDe].getName == + tTable.getSd.getSerdeInfo.getSerializationLib && + tTable.getSd.getColsSize > 0 && + tTable.getSd.getCols.get(0).getType.indexOf('<') == -1) { + tTable.getSd.getSerdeInfo.setSerializationLib(classOf[LazySimpleSerDe].getName) + } + } + tTable + } + /** * Reads statistics from Hive. * Note that this statistics could be overridden by Spark's statistics if that's available. 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 bc9d4cd7f4181..586fbbefade46 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 @@ -28,6 +28,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.api.{EnvironmentContext, Function => HiveFunction, FunctionType} import org.apache.hadoop.hive.metastore.api.{MetaException, PrincipalType, ResourceType, ResourceUri} import org.apache.hadoop.hive.ql.Driver @@ -40,6 +41,7 @@ import org.apache.hadoop.hive.serde.serdeConstants import org.apache.spark.internal.Logging import org.apache.spark.sql.AnalysisException +import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.FunctionIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchPermanentFunctionException import org.apache.spark.sql.catalyst.catalog.{CatalogFunction, CatalogTablePartition, CatalogUtils, FunctionResource, FunctionResourceType} @@ -159,6 +161,8 @@ private[client] sealed abstract class Shim { method } + def getMSC(hive: Hive): IMetaStoreClient + protected def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { klass.getMethod(name, args: _*) } @@ -170,6 +174,17 @@ private[client] class Shim_v0_12 extends Shim with Logging { // deletes the underlying data along with metadata protected lazy val deleteDataInDropIndex = JBoolean.TRUE + protected lazy val getMSCMethod = { + // Since getMSC() in Hive 0.12 is private, findMethod() could not work here + val msc = classOf[Hive].getDeclaredMethod("getMSC") + msc.setAccessible(true) + msc + } + + override def getMSC(hive: Hive): IMetaStoreClient = { + getMSCMethod.invoke(hive).asInstanceOf[IMetaStoreClient] + } + private lazy val startMethod = findStaticMethod( classOf[SessionState], @@ -515,7 +530,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { f.className, null, PrincipalType.USER, - (System.currentTimeMillis / 1000).toInt, + TimeUnit.MILLISECONDS.toSeconds(System.currentTimeMillis).toInt, FunctionType.JAVA, resourceUris.asJava) } @@ -678,7 +693,7 @@ private[client] class Shim_v0_13 extends Shim_v0_12 { expr match { case attr: Attribute => Some(attr) case Cast(child @ AtomicType(), dt: AtomicType, _) - if Cast.canSafeCast(child.dataType.asInstanceOf[AtomicType], dt) => unapply(child) + if Cast.canUpCast(child.dataType.asInstanceOf[AtomicType], dt) => unapply(child) case _ => None } } @@ -987,7 +1002,7 @@ private[client] class Shim_v1_2 extends Shim_v1_1 { part: JList[String], deleteData: Boolean, purge: Boolean): Unit = { - val dropOptions = dropOptionsClass.newInstance().asInstanceOf[Object] + val dropOptions = dropOptionsClass.getConstructor().newInstance().asInstanceOf[Object] dropOptionsDeleteData.setBoolean(dropOptions, deleteData) dropOptionsPurge.setBoolean(dropOptions, purge) dropPartitionMethod.invoke(hive, dbName, tableName, part, dropOptions) @@ -1179,3 +1194,130 @@ 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_v3_0 extends Shim_v2_3 { + // Spark supports only non-ACID operations + protected lazy val isAcidIUDoperation = JBoolean.FALSE + + // Writer ID can be 0 for non-ACID operations + protected lazy val writeIdInLoadTableOrPartition: JLong = 0L + + // Statement ID + protected lazy val stmtIdInLoadTableOrPartition: JInteger = 0 + + protected lazy val listBucketingLevel: JInteger = 0 + + private lazy val clazzLoadFileType = getClass.getClassLoader.loadClass( + "org.apache.hadoop.hive.ql.plan.LoadTableDesc$LoadFileType") + + private lazy val loadPartitionMethod = + findMethod( + classOf[Hive], + "loadPartition", + classOf[Path], + classOf[Table], + classOf[JMap[String, String]], + clazzLoadFileType, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + classOf[JLong], + JInteger.TYPE, + JBoolean.TYPE) + private lazy val loadTableMethod = + findMethod( + classOf[Hive], + "loadTable", + classOf[Path], + classOf[String], + clazzLoadFileType, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + JBoolean.TYPE, + classOf[JLong], + JInteger.TYPE, + JBoolean.TYPE) + private lazy val loadDynamicPartitionsMethod = + findMethod( + classOf[Hive], + "loadDynamicPartitions", + classOf[Path], + classOf[String], + classOf[JMap[String, String]], + clazzLoadFileType, + JInteger.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + JLong.TYPE, + JInteger.TYPE, + JBoolean.TYPE, + classOf[AcidUtils.Operation], + JBoolean.TYPE) + + override def loadPartition( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + inheritTableSpecs: Boolean, + isSkewedStoreAsSubdir: Boolean, + isSrcLocal: Boolean): Unit = { + val session = SparkSession.getActiveSession + assert(session.nonEmpty) + val database = session.get.sessionState.catalog.getCurrentDatabase + val table = hive.getTable(database, tableName) + val loadFileType = if (replace) { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL")) + } else { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING")) + } + assert(loadFileType.isDefined) + loadPartitionMethod.invoke(hive, loadPath, table, partSpec, loadFileType.get, + inheritTableSpecs: JBoolean, isSkewedStoreAsSubdir: JBoolean, + isSrcLocal: JBoolean, isAcid, hasFollowingStatsTask, + writeIdInLoadTableOrPartition, stmtIdInLoadTableOrPartition, replace: JBoolean) + } + + override def loadTable( + hive: Hive, + loadPath: Path, + tableName: String, + replace: Boolean, + isSrcLocal: Boolean): Unit = { + val loadFileType = if (replace) { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL")) + } else { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING")) + } + assert(loadFileType.isDefined) + loadTableMethod.invoke(hive, loadPath, tableName, loadFileType.get, isSrcLocal: JBoolean, + isSkewedStoreAsSubdir, isAcidIUDoperation, hasFollowingStatsTask, + writeIdInLoadTableOrPartition, stmtIdInLoadTableOrPartition: JInteger, replace: JBoolean) + } + + override def loadDynamicPartitions( + hive: Hive, + loadPath: Path, + tableName: String, + partSpec: JMap[String, String], + replace: Boolean, + numDP: Int, + listBucketingEnabled: Boolean): Unit = { + val loadFileType = if (replace) { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("REPLACE_ALL")) + } else { + clazzLoadFileType.getEnumConstants.find(_.toString.equalsIgnoreCase("KEEP_EXISTING")) + } + assert(loadFileType.isDefined) + loadDynamicPartitionsMethod.invoke(hive, loadPath, tableName, partSpec, loadFileType.get, + numDP: JInteger, listBucketingLevel, isAcid, writeIdInLoadTableOrPartition, + stmtIdInLoadTableOrPartition, hasFollowingStatsTask, AcidUtils.Operation.NOT_ACID, + replace: JBoolean) + } +} + +private[client] class Shim_v3_1 extends Shim_v3_0 diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala index 31899370454ba..32178800a8ff3 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala @@ -25,8 +25,10 @@ import java.util import scala.util.Try import org.apache.commons.io.{FileUtils, IOUtils} +import org.apache.commons.lang3.{JavaVersion, SystemUtils} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf.ConfVars +import org.apache.hadoop.hive.shims.ShimLoader import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkSubmitUtils @@ -65,7 +67,7 @@ private[hive] object IsolatedClientLoader extends Logging { case e: RuntimeException if e.getMessage.contains("hadoop") => // If the error message contains hadoop, it is probably because the hadoop // version cannot be resolved. - val fallbackVersion = "2.7.3" + val fallbackVersion = "2.7.4" logWarning(s"Failed to resolve Hadoop artifacts for the version $hadoopVersion. We " + s"will change the hadoop version from $hadoopVersion to $fallbackVersion and try " + "again. Hadoop classes will not be shared between Spark and Hive metastore client. " + @@ -93,13 +95,18 @@ private[hive] object IsolatedClientLoader extends Logging { case "12" | "0.12" | "0.12.0" => hive.v12 case "13" | "0.13" | "0.13.0" | "0.13.1" => hive.v13 case "14" | "0.14" | "0.14.0" => hive.v14 - case "1.0" | "1.0.0" => hive.v1_0 - case "1.1" | "1.1.0" => hive.v1_1 + case "1.0" | "1.0.0" | "1.0.1" => hive.v1_0 + case "1.1" | "1.1.0" | "1.1.1" => hive.v1_1 case "1.2" | "1.2.0" | "1.2.1" | "1.2.2" => hive.v1_2 case "2.0" | "2.0.0" | "2.0.1" => hive.v2_0 case "2.1" | "2.1.0" | "2.1.1" => hive.v2_1 case "2.2" | "2.2.0" => hive.v2_2 - case "2.3" | "2.3.0" | "2.3.1" | "2.3.2" | "2.3.3" => hive.v2_3 + case "2.3" | "2.3.0" | "2.3.1" | "2.3.2" | "2.3.3" | "2.3.4" | "2.3.5" => hive.v2_3 + case "3.0" | "3.0.0" => hive.v3_0 + case "3.1" | "3.1.0" | "3.1.1" => hive.v3_1 + case version => + throw new UnsupportedOperationException(s"Unsupported Hive Metastore version ($version). " + + s"Please set ${HiveUtils.HIVE_METASTORE_VERSION.key} with a valid version.") } private def downloadVersion( @@ -153,7 +160,6 @@ private[hive] object IsolatedClientLoader extends Logging { * @param isolationOn When true, custom versions of barrier classes will be constructed. Must be * true unless loading the version of hive that is on Sparks classloader. * @param sharesHadoopClasses When true, we will share Hadoop classes between Spark and - * @param rootClassLoader The system root classloader. Must not know about Hive classes. * @param baseClassLoader The spark classloader that is used to load shared classes. */ private[hive] class IsolatedClientLoader( @@ -164,15 +170,11 @@ private[hive] class IsolatedClientLoader( val config: Map[String, String] = Map.empty, val isolationOn: Boolean = true, val sharesHadoopClasses: Boolean = true, - val rootClassLoader: ClassLoader = ClassLoader.getSystemClassLoader.getParent.getParent, val baseClassLoader: ClassLoader = Thread.currentThread().getContextClassLoader, val sharedPrefixes: Seq[String] = Seq.empty, val barrierPrefixes: Seq[String] = Seq.empty) extends Logging { - // Check to make sure that the root classloader does not know about Hive. - assert(Try(rootClassLoader.loadClass("org.apache.hadoop.hive.conf.HiveConf")).isFailure) - /** All jars used by the hive specific classloader. */ protected def allJars = execJars.toArray @@ -183,13 +185,12 @@ private[hive] class IsolatedClientLoader( name.startsWith("org.slf4j") || name.startsWith("org.apache.log4j") || // log4j1.x name.startsWith("org.apache.logging.log4j") || // log4j2 - name.startsWith("org.apache.derby.") || name.startsWith("org.apache.spark.") || (sharesHadoopClasses && isHadoopClass) || name.startsWith("scala.") || (name.startsWith("com.google") && !name.startsWith("com.google.cloud")) || - name.startsWith("java.lang.") || - name.startsWith("java.net") || + name.startsWith("java.") || + name.startsWith("javax.sql.") || sharedPrefixes.exists(name.startsWith) } @@ -197,6 +198,7 @@ private[hive] class IsolatedClientLoader( protected def isBarrierClass(name: String): Boolean = name.startsWith(classOf[HiveClientImpl].getName) || name.startsWith(classOf[Shim].getName) || + name.startsWith(classOf[ShimLoader].getName) || barrierPrefixes.exists(name.startsWith) protected def classToPath(name: String): String = @@ -211,30 +213,51 @@ private[hive] class IsolatedClientLoader( private[hive] val classLoader: MutableURLClassLoader = { val isolatedClassLoader = if (isolationOn) { - new URLClassLoader(allJars, rootClassLoader) { - override def loadClass(name: String, resolve: Boolean): Class[_] = { - val loaded = findLoadedClass(name) - if (loaded == null) doLoadClass(name, resolve) else loaded - } - def doLoadClass(name: String, resolve: Boolean): Class[_] = { - val classFileName = name.replaceAll("\\.", "/") + ".class" - if (isBarrierClass(name)) { - // For barrier classes, we construct a new copy of the class. - val bytes = IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName)) - logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}") - defineClass(name, bytes, 0, bytes.length) - } else if (!isSharedClass(name)) { - logDebug(s"hive class: $name - ${getResource(classToPath(name))}") - super.loadClass(name, resolve) + if (allJars.isEmpty) { + // See HiveUtils; this is the Java 9+ + builtin mode scenario + baseClassLoader + } else { + val rootClassLoader: ClassLoader = + if (SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_9)) { + // In Java 9, the boot classloader can see few JDK classes. The intended parent + // classloader for delegation is now the platform classloader. + // See http://java9.wtf/class-loading/ + val platformCL = + classOf[ClassLoader].getMethod("getPlatformClassLoader"). + invoke(null).asInstanceOf[ClassLoader] + // Check to make sure that the root classloader does not know about Hive. + assert(Try(platformCL.loadClass("org.apache.hadoop.hive.conf.HiveConf")).isFailure) + platformCL } else { - // For shared classes, we delegate to baseClassLoader, but fall back in case the - // class is not found. - logDebug(s"shared class: $name") - try { - baseClassLoader.loadClass(name) - } catch { - case _: ClassNotFoundException => - super.loadClass(name, resolve) + // The boot classloader is represented by null (the instance itself isn't accessible) + // and before Java 9 can see all JDK classes + null + } + new URLClassLoader(allJars, rootClassLoader) { + override def loadClass(name: String, resolve: Boolean): Class[_] = { + val loaded = findLoadedClass(name) + if (loaded == null) doLoadClass(name, resolve) else loaded + } + def doLoadClass(name: String, resolve: Boolean): Class[_] = { + val classFileName = name.replaceAll("\\.", "/") + ".class" + if (isBarrierClass(name)) { + // For barrier classes, we construct a new copy of the class. + val bytes = IOUtils.toByteArray(baseClassLoader.getResourceAsStream(classFileName)) + logDebug(s"custom defining: $name - ${util.Arrays.hashCode(bytes)}") + defineClass(name, bytes, 0, bytes.length) + } else if (!isSharedClass(name)) { + logDebug(s"hive class: $name - ${getResource(classToPath(name))}") + super.loadClass(name, resolve) + } else { + // For shared classes, we delegate to baseClassLoader, but fall back in case the + // class is not found. + logDebug(s"shared class: $name") + try { + baseClassLoader.loadClass(name) + } catch { + case _: ClassNotFoundException => + super.loadClass(name, resolve) + } } } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala index 25e9886fa6576..31a060fc8f54b 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/package.scala @@ -29,19 +29,20 @@ package object client { case object v12 extends HiveVersion("0.12.0") case object v13 extends HiveVersion("0.13.1") - // Hive 0.14 depends on calcite 0.9.2-incubating-SNAPSHOT which does not exist in - // maven central anymore, so override those with a version that exists. + // Do not need Calcite because we disabled hive.cbo.enable. // - // The other excluded dependencies are also nowhere to be found, so exclude them explicitly. If + // The other excluded dependencies are nowhere to be found, so exclude them explicitly. If // they're needed by the metastore client, users will have to dig them out of somewhere and use // configuration to point Spark at the correct jars. case object v14 extends HiveVersion("0.14.0", - extraDeps = Seq("org.apache.calcite:calcite-core:1.3.0-incubating", - "org.apache.calcite:calcite-avatica:1.3.0-incubating"), - exclusions = Seq("org.pentaho:pentaho-aggdesigner-algorithm")) + exclusions = Seq("org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", + "org.pentaho:pentaho-aggdesigner-algorithm")) - case object v1_0 extends HiveVersion("1.0.0", + case object v1_0 extends HiveVersion("1.0.1", exclusions = Seq("eigenbase:eigenbase-properties", + "org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", "org.pentaho:pentaho-aggdesigner-algorithm", "net.hydromatic:linq4j", "net.hydromatic:quidem")) @@ -49,8 +50,10 @@ package object client { // The curator dependency was added to the exclusions here because it seems to confuse the ivy // library. org.apache.curator:curator is a pom dependency but ivy tries to find the jar for it, // and fails. - case object v1_1 extends HiveVersion("1.1.0", + case object v1_1 extends HiveVersion("1.1.1", exclusions = Seq("eigenbase:eigenbase-properties", + "org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm", "net.hydromatic:linq4j", @@ -58,28 +61,62 @@ package object client { case object v1_2 extends HiveVersion("1.2.2", exclusions = Seq("eigenbase:eigenbase-properties", + "org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm", "net.hydromatic:linq4j", "net.hydromatic:quidem")) case object v2_0 extends HiveVersion("2.0.1", - exclusions = Seq("org.apache.curator:*", + exclusions = Seq("org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", + "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm")) case object v2_1 extends HiveVersion("2.1.1", - exclusions = Seq("org.apache.curator:*", + exclusions = Seq("org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-avatica", + "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm")) case object v2_2 extends HiveVersion("2.2.0", - exclusions = Seq("org.apache.curator:*", + exclusions = Seq("org.apache.calcite:calcite-core", + "org.apache.calcite:calcite-druid", + "org.apache.calcite.avatica:avatica", + "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm")) - case object v2_3 extends HiveVersion("2.3.3", - exclusions = Seq("org.apache.curator:*", + // Since HIVE-14496, Hive materialized view need calcite-core. + // For spark, only VersionsSuite currently creates a hive materialized view for testing. + case object v2_3 extends HiveVersion("2.3.5", + exclusions = Seq("org.apache.calcite:calcite-druid", + "org.apache.calcite.avatica:avatica", + "org.apache.curator:*", + "org.pentaho:pentaho-aggdesigner-algorithm")) + + // Since Hive 3.0, HookUtils uses org.apache.logging.log4j.util.Strings + // Since HIVE-14496, Hive.java uses calcite-core + case object v3_0 extends HiveVersion("3.0.0", + extraDeps = Seq("org.apache.logging.log4j:log4j-api:2.10.0", + "org.apache.derby:derby:10.14.1.0"), + exclusions = Seq("org.apache.calcite:calcite-druid", + "org.apache.calcite.avatica:avatica", + "org.apache.curator:*", + "org.pentaho:pentaho-aggdesigner-algorithm")) + + // Since Hive 3.0, HookUtils uses org.apache.logging.log4j.util.Strings + // Since HIVE-14496, Hive.java uses calcite-core + case object v3_1 extends HiveVersion("3.1.1", + extraDeps = Seq("org.apache.logging.log4j:log4j-api:2.10.0", + "org.apache.derby:derby:10.14.1.0"), + exclusions = Seq("org.apache.calcite:calcite-druid", + "org.apache.calcite.avatica:avatica", + "org.apache.curator:*", "org.pentaho:pentaho-aggdesigner-algorithm")) - val allSupportedHiveVersions = Set(v12, v13, v14, v1_0, v1_1, v1_2, v2_0, v2_1, v2_2, v2_3) + val allSupportedHiveVersions = + Set(v12, v13, v14, v1_0, v1_1, v1_2, v2_0, v2_1, v2_2, v2_3, v3_0, v3_1) } // scalastyle:on diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala index aa573b54a2b62..9f79997e2979a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/CreateHiveTableAsSelectCommand.scala @@ -20,32 +20,27 @@ package org.apache.spark.sql.hive.execution import scala.util.control.NonFatal import org.apache.spark.sql.{AnalysisException, Row, SaveMode, SparkSession} -import org.apache.spark.sql.catalyst.catalog.CatalogTable -import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.catalog.{CatalogTable, SessionCatalog} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan -import org.apache.spark.sql.execution.command.DataWritingCommand +import org.apache.spark.sql.execution.command.{DataWritingCommand, DDLUtils} +import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, InsertIntoHadoopFsRelationCommand, LogicalRelation} +import org.apache.spark.sql.hive.HiveSessionCatalog +import org.apache.spark.util.Utils +trait CreateHiveTableAsSelectBase extends DataWritingCommand { + val tableDesc: CatalogTable + val query: LogicalPlan + val outputColumnNames: Seq[String] + val mode: SaveMode -/** - * Create table and insert the query result into it. - * - * @param tableDesc the Table Describe, which may contain serde, storage handler etc. - * @param query the query whose result will be insert into the new relation - * @param mode SaveMode - */ -case class CreateHiveTableAsSelectCommand( - tableDesc: CatalogTable, - query: LogicalPlan, - outputColumnNames: Seq[String], - mode: SaveMode) - extends DataWritingCommand { - - private val tableIdentifier = tableDesc.identifier + protected val tableIdentifier = tableDesc.identifier override def run(sparkSession: SparkSession, child: SparkPlan): Seq[Row] = { val catalog = sparkSession.sessionState.catalog - if (catalog.tableExists(tableIdentifier)) { + val tableExists = catalog.tableExists(tableIdentifier) + + if (tableExists) { assert(mode != SaveMode.Overwrite, s"Expect the table $tableIdentifier has been dropped when the save mode is Overwrite") @@ -57,13 +52,8 @@ case class CreateHiveTableAsSelectCommand( return Seq.empty } - InsertIntoHiveTable( - tableDesc, - Map.empty, - query, - overwrite = false, - ifPartitionNotExists = false, - outputColumnNames = outputColumnNames).run(sparkSession, child) + val command = getWritingCommand(catalog, tableDesc, tableExists = true) + command.run(sparkSession, child) } else { // TODO ideally, we should get the output data ready first and then // add the relation into catalog, just in case of failure occurs while data @@ -75,15 +65,8 @@ case class CreateHiveTableAsSelectCommand( try { // Read back the metadata of the table which was created just now. val createdTableMeta = catalog.getTableMetadata(tableDesc.identifier) - // For CTAS, there is no static partition values to insert. - val partition = createdTableMeta.partitionColumnNames.map(_ -> None).toMap - InsertIntoHiveTable( - createdTableMeta, - partition, - query, - overwrite = true, - ifPartitionNotExists = false, - outputColumnNames = outputColumnNames).run(sparkSession, child) + val command = getWritingCommand(catalog, createdTableMeta, tableExists = false) + command.run(sparkSession, child) } catch { case NonFatal(e) => // drop the created table. @@ -95,9 +78,99 @@ case class CreateHiveTableAsSelectCommand( Seq.empty[Row] } - override def argString: String = { - s"[Database:${tableDesc.database}}, " + + // Returns `DataWritingCommand` which actually writes data into the table. + def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand + + // A subclass should override this with the Class name of the concrete type expected to be + // returned from `getWritingCommand`. + def writingCommandClassName: String + + override def argString(maxFields: Int): String = { + s"[Database: ${tableDesc.database}, " + s"TableName: ${tableDesc.identifier.table}, " + - s"InsertIntoHiveTable]" + s"${writingCommandClassName}]" } } + +/** + * Create table and insert the query result into it. + * + * @param tableDesc the table description, which may contain serde, storage handler etc. + * @param query the query whose result will be insert into the new relation + * @param mode SaveMode + */ +case class CreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + outputColumnNames: Seq[String], + mode: SaveMode) + extends CreateHiveTableAsSelectBase { + + override def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand = { + // For CTAS, there is no static partition values to insert. + val partition = tableDesc.partitionColumnNames.map(_ -> None).toMap + InsertIntoHiveTable( + tableDesc, + partition, + query, + overwrite = if (tableExists) false else true, + ifPartitionNotExists = false, + outputColumnNames = outputColumnNames) + } + + override def writingCommandClassName: String = + Utils.getSimpleName(classOf[InsertIntoHiveTable]) +} + +/** + * Create table and insert the query result into it. This creates Hive table but inserts + * the query result into it by using data source. + * + * @param tableDesc the table description, which may contain serde, storage handler etc. + * @param query the query whose result will be insert into the new relation + * @param mode SaveMode + */ +case class OptimizedCreateHiveTableAsSelectCommand( + tableDesc: CatalogTable, + query: LogicalPlan, + outputColumnNames: Seq[String], + mode: SaveMode) + extends CreateHiveTableAsSelectBase { + + override def getWritingCommand( + catalog: SessionCatalog, + tableDesc: CatalogTable, + tableExists: Boolean): DataWritingCommand = { + val metastoreCatalog = catalog.asInstanceOf[HiveSessionCatalog].metastoreCatalog + val hiveTable = DDLUtils.readHiveTable(tableDesc) + + val hadoopRelation = metastoreCatalog.convert(hiveTable) match { + case LogicalRelation(t: HadoopFsRelation, _, _, _) => t + case _ => throw new AnalysisException(s"$tableIdentifier should be converted to " + + "HadoopFsRelation.") + } + + InsertIntoHadoopFsRelationCommand( + hadoopRelation.location.rootPaths.head, + Map.empty, // We don't support to convert partitioned table. + false, + Seq.empty, // We don't support to convert partitioned table. + hadoopRelation.bucketSpec, + hadoopRelation.fileFormat, + hadoopRelation.options, + query, + if (tableExists) mode else SaveMode.Overwrite, + Some(tableDesc), + Some(hadoopRelation.location), + query.output.map(_.name)) + } + + override def writingCommandClassName: String = + Utils.getSimpleName(classOf[InsertIntoHadoopFsRelationCommand]) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala index 4a7cd6901923b..27071075b4165 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveFileFormat.scala @@ -30,6 +30,7 @@ import org.apache.hadoop.mapred.{JobConf, Reporter} import org.apache.hadoop.mapreduce.{Job, TaskAttemptContext} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.SPECULATION_ENABLED import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.execution.datasources.{FileFormat, OutputWriter, OutputWriterFactory} @@ -69,7 +70,7 @@ class HiveFileFormat(fileSinkConf: FileSinkDesc) // When speculation is on and output committer class name contains "Direct", we should warn // users that they may loss data if they are using a direct output committer. - val speculationEnabled = sparkSession.sparkContext.conf.getBoolean("spark.speculation", false) + val speculationEnabled = sparkSession.sparkContext.conf.get(SPECULATION_ENABLED) val outputCommitterClass = conf.get("mapred.output.committer.class", "") if (speculationEnabled && outputCommitterClass.contains("Direct")) { val warningMessage = @@ -115,7 +116,8 @@ class HiveOutputWriter( private def tableDesc = fileSinkConf.getTableInfo private val serializer = { - val serializer = tableDesc.getDeserializerClass.newInstance().asInstanceOf[Serializer] + val serializer = tableDesc.getDeserializerClass.getConstructor(). + newInstance().asInstanceOf[Serializer] serializer.initialize(jobConf, tableDesc.getProperties) serializer } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala index b3795b4430404..fa940fe73bd13 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/HiveTableScanExec.scala @@ -120,7 +120,7 @@ case class HiveTableScanExec( HiveShim.appendReadColumns(hiveConf, neededColumnIDs, output.map(_.name)) - val deserializer = tableDesc.getDeserializerClass.newInstance + val deserializer = tableDesc.getDeserializerClass.getConstructor().newInstance() deserializer.initialize(hiveConf, tableDesc.getProperties) // Specifies types and object inspectors of columns to be scanned. @@ -182,7 +182,7 @@ case class HiveTableScanExec( protected override def doExecute(): RDD[InternalRow] = { // Using dummyCallSite, as getCallSite can turn out to be expensive with - // with multiple partitions. + // multiple partitions. val rdd = if (!relation.isPartitioned) { Utils.withDummyCallSite(sqlContext.sparkContext) { hadoopReader.makeRDDForTable(hiveQlTable) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala index 0c694910b06d4..b66c302a7d7ea 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala @@ -17,20 +17,16 @@ package org.apache.spark.sql.hive.execution -import scala.language.existentials - import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.hive.common.FileUtils import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.serde.serdeConstants import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe -import org.apache.hadoop.mapred._ import org.apache.spark.SparkException import org.apache.spark.sql.{Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable} -import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkPlan import org.apache.spark.sql.hive.client.HiveClientImpl @@ -83,23 +79,23 @@ case class InsertIntoHiveDirCommand( ) val hadoopConf = sparkSession.sessionState.newHadoopConf() - val jobConf = new JobConf(hadoopConf) val targetPath = new Path(storage.locationUri.get) - val writeToPath = + val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) + val (writeToPath: Path, fs: FileSystem) = if (isLocal) { - val localFileSystem = FileSystem.getLocal(jobConf) - localFileSystem.makeQualified(targetPath) + val localFileSystem = FileSystem.getLocal(hadoopConf) + (localFileSystem.makeQualified(targetPath), localFileSystem) } else { - val qualifiedPath = FileUtils.makeQualified(targetPath, hadoopConf) - val dfs = qualifiedPath.getFileSystem(jobConf) - if (!dfs.exists(qualifiedPath)) { - dfs.mkdirs(qualifiedPath.getParent) - } - qualifiedPath + val dfs = qualifiedPath.getFileSystem(hadoopConf) + (qualifiedPath, dfs) } + if (!fs.exists(writeToPath)) { + fs.mkdirs(writeToPath) + } - val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, writeToPath) + // The temporary path must be a HDFS path, not a local path. + val tmpPath = getExternalTmpPath(sparkSession, hadoopConf, qualifiedPath) val fileSinkConf = new org.apache.spark.sql.hive.HiveShim.ShimFileSinkDesc( tmpPath.toString, tableDesc, false) @@ -111,15 +107,20 @@ case class InsertIntoHiveDirCommand( fileSinkConf = fileSinkConf, outputLocation = tmpPath.toString) - val fs = writeToPath.getFileSystem(hadoopConf) if (overwrite && fs.exists(writeToPath)) { fs.listStatus(writeToPath).foreach { existFile => if (Option(existFile.getPath) != createdTempDir) fs.delete(existFile.getPath, true) } } - fs.listStatus(tmpPath).foreach { - tmpFile => fs.rename(tmpFile.getPath, writeToPath) + val dfs = tmpPath.getFileSystem(hadoopConf) + dfs.listStatus(tmpPath).foreach { + tmpFile => + if (isLocal) { + dfs.copyToLocalFile(tmpFile.getPath, writeToPath) + } else { + dfs.rename(tmpFile.getPath, writeToPath) + } } } catch { case e: Throwable => diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala index 078968ed0145f..62d3bad63bf58 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/SaveAsHiveFile.scala @@ -83,6 +83,16 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { jobId = java.util.UUID.randomUUID().toString, outputPath = outputLocation) + // SPARK-28054: Hive metastore is not case preserving and keeps partition columns + // with lower cased names, Hive will validate the column names in partition spec and + // the partition paths. Besides lowercasing the column names in the partition spec, + // we also need to lowercase the column names in written partition paths. + // scalastyle:off caselocale + val hiveCompatiblePartitionColumns = partitionAttributes.map { attr => + attr.withName(attr.name.toLowerCase) + } + // scalastyle:on caselocale + FileFormatWriter.write( sparkSession = sparkSession, plan = plan, @@ -91,7 +101,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { outputSpec = FileFormatWriter.OutputSpec(outputLocation, customPartitionLocations, outputColumns), hadoopConf = hadoopConf, - partitionColumns = partitionAttributes, + partitionColumns = hiveCompatiblePartitionColumns, bucketSpec = None, statsTrackers = Seq(basicWriteJobStatsTracker(hadoopConf)), options = Map.empty) @@ -114,7 +124,7 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { // be removed by Hive when Hive is trying to empty the table directory. val hiveVersionsUsingOldExternalTempPath: Set[HiveVersion] = Set(v12, v13, v14, v1_0) val hiveVersionsUsingNewExternalTempPath: Set[HiveVersion] = - Set(v1_1, v1_2, v2_0, v2_1, v2_2, v2_3) + Set(v1_1, v1_2, v2_0, v2_1, v2_2, v2_3, v3_0, v3_1) // Ensure all the supported versions are considered here. assert(hiveVersionsUsingNewExternalTempPath ++ hiveVersionsUsingOldExternalTempPath == @@ -210,12 +220,11 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { stagingDir) } - private def getStagingDir( + private[hive] def getStagingDir( inputPath: Path, hadoopConf: Configuration, stagingDir: String): Path = { - val inputPathUri: URI = inputPath.toUri - val inputPathName: String = inputPathUri.getPath + val inputPathName: String = inputPath.toString val fs: FileSystem = inputPath.getFileSystem(hadoopConf) var stagingPathName: String = if (inputPathName.indexOf(stagingDir) == -1) { @@ -227,8 +236,8 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { // SPARK-20594: This is a walk-around fix to resolve a Hive bug. Hive requires that the // staging directory needs to avoid being deleted when users set hive.exec.stagingdir // under the table directory. - if (FileUtils.isSubDir(new Path(stagingPathName), inputPath, fs) && - !stagingPathName.stripPrefix(inputPathName).stripPrefix(File.separator).startsWith(".")) { + if (isSubDir(new Path(stagingPathName), inputPath, fs) && + !stagingPathName.stripPrefix(inputPathName).stripPrefix("/").startsWith(".")) { logDebug(s"The staging dir '$stagingPathName' should be a child directory starts " + "with '.' to avoid being deleted if we set hive.exec.stagingdir under the table " + "directory.") @@ -253,6 +262,13 @@ private[hive] trait SaveAsHiveFile extends DataWritingCommand { dir } + // HIVE-14259 removed FileUtils.isSubDir(). Adapted it from Hive 1.2's FileUtils.isSubDir(). + private def isSubDir(p1: Path, p2: Path, fs: FileSystem): Boolean = { + val path1 = fs.makeQualified(p1).toString + Path.SEPARATOR + val path2 = fs.makeQualified(p2).toString + Path.SEPARATOR + path1.startsWith(path2) + } + private def executionId: String = { val rand: Random = new Random val format = new SimpleDateFormat("yyyy-MM-dd_HH-mm-ss_SSS", Locale.US) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala index 3328400b214fb..e12f663304e7a 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/ScriptTransformationExec.scala @@ -123,7 +123,7 @@ case class ScriptTransformationExec( var scriptOutputWritable: Writable = null val reusedWritableObject: Writable = if (null != outputSerde) { - outputSerde.getSerializedClass().newInstance + outputSerde.getSerializedClass().getConstructor().newInstance() } else { null } @@ -308,12 +308,15 @@ private class ScriptTransformationWriterThread( } threwException = false } catch { + // SPARK-25158 Exception should not be thrown again, otherwise it will be captured by + // SparkUncaughtExceptionHandler, then Executor will exit because of this Uncaught Exception, + // so pass the exception to `ScriptTransformationExec` is enough. case t: Throwable => // An error occurred while writing input, so kill the child process. According to the // Javadoc this call will not throw an exception: _exception = t proc.destroy() - throw t + logError("Thread-ScriptTransformation-Feed exit cause by: ", t) } finally { try { Utils.tryLogNonFatalError(outputStream.close()) @@ -404,7 +407,8 @@ case class HiveScriptIOSchema ( columnTypes: Seq[DataType], serdeProps: Seq[(String, String)]): AbstractSerDe = { - val serde = Utils.classForName(serdeClassName).newInstance.asInstanceOf[AbstractSerDe] + val serde = Utils.classForName[AbstractSerDe](serdeClassName).getConstructor(). + newInstance() val columnTypesNames = columnTypes.map(_.toTypeInfo.getTypeName()).mkString(",") @@ -424,7 +428,8 @@ case class HiveScriptIOSchema ( inputStream: InputStream, conf: Configuration): Option[RecordReader] = { recordReaderClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordReader] + val instance = Utils.classForName[RecordReader](klass).getConstructor(). + newInstance() val props = new Properties() // Can not use props.putAll(outputSerdeProps.toMap.asJava) in scala-2.12 // See https://github.com/scala/bug/issues/10418 @@ -436,7 +441,8 @@ case class HiveScriptIOSchema ( def recordWriter(outputStream: OutputStream, conf: Configuration): Option[RecordWriter] = { recordWriterClass.map { klass => - val instance = Utils.classForName(klass).newInstance().asInstanceOf[RecordWriter] + val instance = Utils.classForName[RecordWriter](klass).getConstructor(). + newInstance() instance.initialize(outputStream, conf) instance } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala index 68af99ea272a8..d78fc9da9f8a5 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUDFs.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.hive +import java.lang.{Boolean => JBoolean} import java.nio.ByteBuffer import scala.collection.JavaConverters._ @@ -38,7 +39,7 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._ import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback import org.apache.spark.sql.hive.HiveShim._ import org.apache.spark.sql.types._ - +import org.apache.spark.util.Utils private[hive] case class HiveSimpleUDF( name: String, funcWrapper: HiveFunctionWrapper, children: Seq[Expression]) @@ -303,6 +304,13 @@ private[hive] case class HiveGenericUDTF( * - `wrap()`/`wrapperFor()`: from 3 to 1 * - `unwrap()`/`unwrapperFor()`: from 1 to 3 * - `GenericUDAFEvaluator.terminatePartial()`: from 2 to 3 + * + * Note that, Hive UDAF is initialized with aggregate mode, and some specific Hive UDAFs can't + * mix UPDATE and MERGE actions during its life cycle. However, Spark may do UPDATE on a UDAF and + * then do MERGE, in case of hash aggregate falling back to sort aggregate. To work around this + * issue, we track the ability to do MERGE in the Hive UDAF aggregate buffer. If Spark does + * UPDATE then MERGE, we can detect it and re-create the aggregate buffer with a different + * aggregate mode. */ private[hive] case class HiveUDAFFunction( name: String, @@ -311,7 +319,7 @@ private[hive] case class HiveUDAFFunction( isUDAFBridgeRequired: Boolean = false, mutableAggBufferOffset: Int = 0, inputAggBufferOffset: Int = 0) - extends TypedImperativeAggregate[GenericUDAFEvaluator.AggregationBuffer] + extends TypedImperativeAggregate[HiveUDAFBuffer] with HiveInspectors with UserDefinedExpression { @@ -336,43 +344,48 @@ private[hive] case class HiveUDAFFunction( funcWrapper.createFunction[AbstractGenericUDAFResolver]() } - val parameterInfo = new SimpleGenericUDAFParameterInfo(inputInspectors, false, false) - resolver.getEvaluator(parameterInfo) + val clazz = Utils.classForName(classOf[SimpleGenericUDAFParameterInfo].getName) + if (HiveUtils.isHive23) { + val ctor = clazz.getDeclaredConstructor( + classOf[Array[ObjectInspector]], JBoolean.TYPE, JBoolean.TYPE, JBoolean.TYPE) + val args = Array[AnyRef](inputInspectors, JBoolean.FALSE, JBoolean.FALSE, JBoolean.FALSE) + val parameterInfo = ctor.newInstance(args: _*).asInstanceOf[SimpleGenericUDAFParameterInfo] + resolver.getEvaluator(parameterInfo) + } else { + val ctor = clazz.getDeclaredConstructor( + classOf[Array[ObjectInspector]], JBoolean.TYPE, JBoolean.TYPE) + val args = Array[AnyRef](inputInspectors, JBoolean.FALSE, JBoolean.FALSE) + val parameterInfo = ctor.newInstance(args: _*).asInstanceOf[SimpleGenericUDAFParameterInfo] + resolver.getEvaluator(parameterInfo) + } } - // The UDAF evaluator used to consume raw input rows and produce partial aggregation results. - @transient - private lazy val partial1ModeEvaluator = newEvaluator() + private case class HiveEvaluator( + evaluator: GenericUDAFEvaluator, + objectInspector: ObjectInspector) + // The UDAF evaluator used to consume raw input rows and produce partial aggregation results. // Hive `ObjectInspector` used to inspect partial aggregation results. @transient - private val partialResultInspector = partial1ModeEvaluator.init( - GenericUDAFEvaluator.Mode.PARTIAL1, - inputInspectors - ) + private lazy val partial1HiveEvaluator = { + val evaluator = newEvaluator() + HiveEvaluator(evaluator, evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL1, inputInspectors)) + } - // The UDAF evaluator used to merge partial aggregation results. + // The UDAF evaluator used to consume partial aggregation results and produce final results. + // Hive `ObjectInspector` used to inspect final results. @transient - private lazy val partial2ModeEvaluator = { + private lazy val finalHiveEvaluator = { val evaluator = newEvaluator() - evaluator.init(GenericUDAFEvaluator.Mode.PARTIAL2, Array(partialResultInspector)) - evaluator + HiveEvaluator( + evaluator, + evaluator.init(GenericUDAFEvaluator.Mode.FINAL, Array(partial1HiveEvaluator.objectInspector))) } // Spark SQL data type of partial aggregation results @transient - private lazy val partialResultDataType = inspectorToDataType(partialResultInspector) - - // The UDAF evaluator used to compute the final result from a partial aggregation result objects. - @transient - private lazy val finalModeEvaluator = newEvaluator() - - // Hive `ObjectInspector` used to inspect the final aggregation result object. - @transient - private val returnInspector = finalModeEvaluator.init( - GenericUDAFEvaluator.Mode.FINAL, - Array(partialResultInspector) - ) + private lazy val partialResultDataType = + inspectorToDataType(partial1HiveEvaluator.objectInspector) // Wrapper functions used to wrap Spark SQL input arguments into Hive specific format. @transient @@ -381,7 +394,7 @@ private[hive] case class HiveUDAFFunction( // Unwrapper function used to unwrap final aggregation result objects returned by Hive UDAFs into // Spark SQL specific format. @transient - private lazy val resultUnwrapper = unwrapperFor(returnInspector) + private lazy val resultUnwrapper = unwrapperFor(finalHiveEvaluator.objectInspector) @transient private lazy val cached: Array[AnyRef] = new Array[AnyRef](children.length) @@ -391,7 +404,7 @@ private[hive] case class HiveUDAFFunction( override def nullable: Boolean = true - override lazy val dataType: DataType = inspectorToDataType(returnInspector) + override lazy val dataType: DataType = inspectorToDataType(finalHiveEvaluator.objectInspector) override def prettyName: String = name @@ -400,58 +413,107 @@ private[hive] case class HiveUDAFFunction( s"$name($distinct${children.map(_.sql).mkString(", ")})" } - override def createAggregationBuffer(): AggregationBuffer = - partial1ModeEvaluator.getNewAggregationBuffer + // The hive UDAF may create different buffers to handle different inputs: original data or + // aggregate buffer. However, the Spark UDAF framework does not expose this information when + // creating the buffer. Here we return null, and create the buffer in `update` and `merge` + // on demand, so that we can know what input we are dealing with. + override def createAggregationBuffer(): HiveUDAFBuffer = null @transient private lazy val inputProjection = UnsafeProjection.create(children) - override def update(buffer: AggregationBuffer, input: InternalRow): AggregationBuffer = { - partial1ModeEvaluator.iterate( - buffer, wrap(inputProjection(input), inputWrappers, cached, inputDataTypes)) - buffer + override def update(buffer: HiveUDAFBuffer, input: InternalRow): HiveUDAFBuffer = { + // The input is original data, we create buffer with the partial1 evaluator. + val nonNullBuffer = if (buffer == null) { + HiveUDAFBuffer(partial1HiveEvaluator.evaluator.getNewAggregationBuffer, false) + } else { + buffer + } + + assert(!nonNullBuffer.canDoMerge, "can not call `merge` then `update` on a Hive UDAF.") + + partial1HiveEvaluator.evaluator.iterate( + nonNullBuffer.buf, wrap(inputProjection(input), inputWrappers, cached, inputDataTypes)) + nonNullBuffer } - override def merge(buffer: AggregationBuffer, input: AggregationBuffer): AggregationBuffer = { + override def merge(buffer: HiveUDAFBuffer, input: HiveUDAFBuffer): HiveUDAFBuffer = { + // The input is aggregate buffer, we create buffer with the final evaluator. + val nonNullBuffer = if (buffer == null) { + HiveUDAFBuffer(finalHiveEvaluator.evaluator.getNewAggregationBuffer, true) + } else { + buffer + } + + // It's possible that we've called `update` of this Hive UDAF, and some specific Hive UDAF + // implementation can't mix the `update` and `merge` calls during its life cycle. To work + // around it, here we create a fresh buffer with final evaluator, and merge the existing buffer + // to it, and replace the existing buffer with it. + val mergeableBuf = if (!nonNullBuffer.canDoMerge) { + val newBuf = finalHiveEvaluator.evaluator.getNewAggregationBuffer + finalHiveEvaluator.evaluator.merge( + newBuf, partial1HiveEvaluator.evaluator.terminatePartial(nonNullBuffer.buf)) + HiveUDAFBuffer(newBuf, true) + } else { + nonNullBuffer + } + // The 2nd argument of the Hive `GenericUDAFEvaluator.merge()` method is an input aggregation // buffer in the 3rd format mentioned in the ScalaDoc of this class. Originally, Hive converts // this `AggregationBuffer`s into this format before shuffling partial aggregation results, and // calls `GenericUDAFEvaluator.terminatePartial()` to do the conversion. - partial2ModeEvaluator.merge(buffer, partial1ModeEvaluator.terminatePartial(input)) - buffer + finalHiveEvaluator.evaluator.merge( + mergeableBuf.buf, partial1HiveEvaluator.evaluator.terminatePartial(input.buf)) + mergeableBuf } - override def eval(buffer: AggregationBuffer): Any = { - resultUnwrapper(finalModeEvaluator.terminate(buffer)) + override def eval(buffer: HiveUDAFBuffer): Any = { + resultUnwrapper(finalHiveEvaluator.evaluator.terminate( + if (buffer == null) { + finalHiveEvaluator.evaluator.getNewAggregationBuffer + } else { + buffer.buf + } + )) } - override def serialize(buffer: AggregationBuffer): Array[Byte] = { + override def serialize(buffer: HiveUDAFBuffer): Array[Byte] = { // Serializes an `AggregationBuffer` that holds partial aggregation results so that we can // shuffle it for global aggregation later. - aggBufferSerDe.serialize(buffer) + aggBufferSerDe.serialize(if (buffer == null) null else buffer.buf) } - override def deserialize(bytes: Array[Byte]): AggregationBuffer = { + override def deserialize(bytes: Array[Byte]): HiveUDAFBuffer = { // Deserializes an `AggregationBuffer` from the shuffled partial aggregation phase to prepare // for global aggregation by merging multiple partial aggregation results within a single group. - aggBufferSerDe.deserialize(bytes) + HiveUDAFBuffer(aggBufferSerDe.deserialize(bytes), false) } // Helper class used to de/serialize Hive UDAF `AggregationBuffer` objects private class AggregationBufferSerDe { - private val partialResultUnwrapper = unwrapperFor(partialResultInspector) + private val partialResultUnwrapper = unwrapperFor(partial1HiveEvaluator.objectInspector) - private val partialResultWrapper = wrapperFor(partialResultInspector, partialResultDataType) + private val partialResultWrapper = + wrapperFor(partial1HiveEvaluator.objectInspector, partialResultDataType) private val projection = UnsafeProjection.create(Array(partialResultDataType)) private val mutableRow = new GenericInternalRow(1) def serialize(buffer: AggregationBuffer): Array[Byte] = { + // The buffer may be null if there is no input. It's unclear if the hive UDAF accepts null + // buffer, for safety we create an empty buffer here. + val nonNullBuffer = if (buffer == null) { + partial1HiveEvaluator.evaluator.getNewAggregationBuffer + } else { + buffer + } + // `GenericUDAFEvaluator.terminatePartial()` converts an `AggregationBuffer` into an object // that can be inspected by the `ObjectInspector` returned by `GenericUDAFEvaluator.init()`. // Then we can unwrap it to a Spark SQL value. - mutableRow.update(0, partialResultUnwrapper(partial1ModeEvaluator.terminatePartial(buffer))) + mutableRow.update(0, partialResultUnwrapper( + partial1HiveEvaluator.evaluator.terminatePartial(nonNullBuffer))) val unsafeRow = projection(mutableRow) val bytes = ByteBuffer.allocate(unsafeRow.getSizeInBytes) unsafeRow.writeTo(bytes) @@ -463,12 +525,14 @@ private[hive] case class HiveUDAFFunction( // returned by `GenericUDAFEvaluator.terminatePartial()` back to an `AggregationBuffer`. The // workaround here is creating an initial `AggregationBuffer` first and then merge the // deserialized object into the buffer. - val buffer = partial2ModeEvaluator.getNewAggregationBuffer + val buffer = finalHiveEvaluator.evaluator.getNewAggregationBuffer val unsafeRow = new UnsafeRow(1) unsafeRow.pointTo(bytes, bytes.length) val partialResult = unsafeRow.get(0, partialResultDataType) - partial2ModeEvaluator.merge(buffer, partialResultWrapper(partialResult)) + finalHiveEvaluator.evaluator.merge(buffer, partialResultWrapper(partialResult)) buffer } } } + +case class HiveUDAFBuffer(buf: AggregationBuffer, canDoMerge: Boolean) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala index de8085f07db19..7f2eb14956dc1 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileFormat.scala @@ -18,23 +18,31 @@ package org.apache.spark.sql.hive.orc import java.net.URI +import java.nio.charset.StandardCharsets.UTF_8 import java.util.Properties import scala.collection.JavaConverters._ +import scala.util.control.NonFatal +import com.esotericsoftware.kryo.Kryo +import com.esotericsoftware.kryo.io.Output +import org.apache.commons.codec.binary.Base64 import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hadoop.hive.ql.io.orc._ +import org.apache.hadoop.hive.ql.io.sarg.SearchArgument import org.apache.hadoop.hive.serde2.objectinspector.{SettableStructObjectInspector, StructObjectInspector} import org.apache.hadoop.hive.serde2.typeinfo.{StructTypeInfo, TypeInfoUtils} import org.apache.hadoop.io.{NullWritable, Writable} import org.apache.hadoop.mapred.{JobConf, OutputFormat => MapRedOutputFormat, RecordWriter, Reporter} import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit} +import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.orc.OrcConf.COMPRESS -import org.apache.spark.TaskContext +import org.apache.spark.{SPARK_VERSION_SHORT, TaskContext} +import org.apache.spark.internal.Logging +import org.apache.spark.sql.SPARK_VERSION_METADATA_KEY import org.apache.spark.sql.SparkSession import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions._ @@ -59,12 +67,20 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable sparkSession: SparkSession, options: Map[String, String], files: Seq[FileStatus]): Option[StructType] = { - val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles - OrcFileOperator.readSchema( - files.map(_.getPath.toString), - Some(sparkSession.sessionState.newHadoopConf()), - ignoreCorruptFiles - ) + val orcOptions = new OrcOptions(options, sparkSession.sessionState.conf) + if (orcOptions.mergeSchema) { + SchemaMergeUtils.mergeSchemasInParallel( + sparkSession, + files, + OrcFileOperator.readOrcSchemasInParallel) + } else { + val ignoreCorruptFiles = sparkSession.sessionState.conf.ignoreCorruptFiles + OrcFileOperator.readSchema( + files.map(_.getPath.toString), + Some(sparkSession.sessionState.newHadoopConf()), + ignoreCorruptFiles + ) + } } override def prepareWrite( @@ -126,7 +142,7 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable if (sparkSession.sessionState.conf.orcFilterPushDown) { // Sets pushed predicates OrcFilters.createFilter(requiredSchema, filters.toArray).foreach { f => - hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, f.toKryo) + hadoopConf.set(OrcFileFormat.SARG_PUSHDOWN, toKryo(f)) hadoopConf.setBoolean(ConfVars.HIVEOPTINDEXFILTER.varname, true) } } @@ -154,13 +170,12 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable val job = Job.getInstance(conf) FileInputFormat.setInputPaths(job, file.filePath) - val fileSplit = new FileSplit(filePath, file.start, file.length, Array.empty) // Custom OrcRecordReader is used to get // ObjectInspector during recordReader creation itself and can // avoid NameNode call in unwrapOrcStructs per file. // Specifically would be helpful for partitioned datasets. val orcReader = OrcFile.createReader(filePath, OrcFile.readerOptions(conf)) - new SparkOrcNewRecordReader(orcReader, conf, fileSplit.getStart, fileSplit.getLength) + new SparkOrcNewRecordReader(orcReader, conf, file.start, file.length) } val recordsIterator = new RecordReaderIterator[OrcStruct](orcRecordReader) @@ -178,22 +193,30 @@ class OrcFileFormat extends FileFormat with DataSourceRegister with Serializable } } - override def supportDataType(dataType: DataType, isReadPath: Boolean): Boolean = dataType match { + override def supportDataType(dataType: DataType): Boolean = dataType match { case _: AtomicType => true - case st: StructType => st.forall { f => supportDataType(f.dataType, isReadPath) } + case st: StructType => st.forall { f => supportDataType(f.dataType) } - case ArrayType(elementType, _) => supportDataType(elementType, isReadPath) + case ArrayType(elementType, _) => supportDataType(elementType) case MapType(keyType, valueType, _) => - supportDataType(keyType, isReadPath) && supportDataType(valueType, isReadPath) - - case udt: UserDefinedType[_] => supportDataType(udt.sqlType, isReadPath) + supportDataType(keyType) && supportDataType(valueType) - case _: NullType => isReadPath + case udt: UserDefinedType[_] => supportDataType(udt.sqlType) case _ => false } + + // HIVE-11253 moved `toKryo` from `SearchArgument` to `storage-api` module. + // This is copied from Hive 1.2's SearchArgumentImpl.toKryo(). + private def toKryo(sarg: SearchArgument): String = { + val kryo = new Kryo() + val out = new Output(4 * 1024, 10 * 1024 * 1024) + kryo.writeObject(out, sarg) + out.close() + Base64.encodeBase64String(out.toBytes) + } } private[orc] class OrcSerializer(dataSchema: StructType, conf: Configuration) @@ -275,12 +298,14 @@ private[orc] class OrcOutputWriter( override def close(): Unit = { if (recordWriterInstantiated) { + // Hive 1.2.1 ORC initializes its private `writer` field at the first write. + OrcFileFormat.addSparkVersionMetadata(recordWriter) recordWriter.close(Reporter.NULL) } } } -private[orc] object OrcFileFormat extends HiveInspectors { +private[orc] object OrcFileFormat extends HiveInspectors with Logging { // This constant duplicates `OrcInputFormat.SARG_PUSHDOWN`, which is unfortunately not public. private[orc] val SARG_PUSHDOWN = "sarg.pushdown" @@ -340,4 +365,18 @@ private[orc] object OrcFileFormat extends HiveInspectors { val (sortedIDs, sortedNames) = ids.zip(requestedSchema.fieldNames).sorted.unzip HiveShim.appendReadColumns(conf, sortedIDs, sortedNames) } + + /** + * Add a metadata specifying Spark version. + */ + def addSparkVersionMetadata(recordWriter: RecordWriter[NullWritable, Writable]): Unit = { + try { + val writerField = recordWriter.getClass.getDeclaredField("writer") + writerField.setAccessible(true) + val writer = writerField.get(recordWriter).asInstanceOf[Writer] + writer.addUserMetadata(SPARK_VERSION_METADATA_KEY, UTF_8.encode(SPARK_VERSION_SHORT)) + } catch { + case NonFatal(e) => log.warn(e.toString, e) + } + } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala index 713b70f252b6a..1a5f47bf5aa7d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/orc/OrcFileOperator.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql.hive.orc import java.io.IOException import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.hive.ql.io.orc.{OrcFile, Reader} import org.apache.hadoop.hive.serde2.objectinspector.StructObjectInspector @@ -29,6 +29,7 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.sql.catalyst.parser.CatalystSqlParser import org.apache.spark.sql.types.StructType +import org.apache.spark.util.ThreadUtils private[hive] object OrcFileOperator extends Logging { /** @@ -101,6 +102,24 @@ private[hive] object OrcFileOperator extends Logging { } } + /** + * Reads ORC file schemas in multi-threaded manner, using Hive ORC library. + * This is visible for testing. + */ + def readOrcSchemasInParallel( + partFiles: Seq[FileStatus], conf: Configuration, ignoreCorruptFiles: Boolean) + : Seq[StructType] = { + ThreadUtils.parmap(partFiles, "readingOrcSchemas", 8) { currentFile => + val file = currentFile.getPath.toString + getFileReader(file, Some(conf), ignoreCorruptFiles).map(reader => { + val readerInspector = reader.getObjectInspector.asInstanceOf[StructObjectInspector] + val schema = readerInspector.getTypeName + logDebug(s"Reading schema from file $file., got Hive schema string: $schema") + CatalystSqlParser.parseDataType(schema).asInstanceOf[StructType] + }) + }.flatten + } + def getObjectInspector( path: String, conf: Option[Configuration]): Option[StructObjectInspector] = { getFileReader(path, conf).map(_.getObjectInspector.asInstanceOf[StructObjectInspector]) 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 a82576a233acd..cd1bffb6b7ab7 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 @@ -17,12 +17,17 @@ package org.apache.spark.sql.hive.orc +import java.lang.reflect.Method + import org.apache.hadoop.hive.ql.io.sarg.SearchArgument import org.apache.hadoop.hive.ql.io.sarg.SearchArgument.Builder 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.execution.datasources.orc.{OrcFilters => DatasourceOrcFilters} import org.apache.spark.sql.execution.datasources.orc.OrcFilters.buildTree +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.sources._ import org.apache.spark.sql.types._ @@ -57,45 +62,132 @@ import org.apache.spark.sql.types._ * known to be convertible. */ private[orc] object OrcFilters extends Logging { - def createFilter(schema: StructType, filters: Array[Filter]): Option[SearchArgument] = { - val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap - // First, tries to convert each filter individually to see whether it's convertible, and then - // collect all convertible ones to build the final `SearchArgument`. - val convertibleFilters = for { - filter <- filters - _ <- buildSearchArgument(dataTypeMap, filter, newBuilder) - } yield filter + private def findMethod(klass: Class[_], name: String, args: Class[_]*): Method = { + val method = klass.getMethod(name, args: _*) + method.setAccessible(true) + method + } - for { + def createFilter(schema: StructType, filters: Array[Filter]): Option[SearchArgument] = { + if (HiveUtils.isHive23) { + DatasourceOrcFilters.createFilter(schema, filters).asInstanceOf[Option[SearchArgument]] + } else { + val dataTypeMap = schema.map(f => f.name -> f.dataType).toMap // Combines all convertible filters using `And` to produce a single conjunction - conjunction <- buildTree(convertibleFilters) - // Then tries to build a single ORC `SearchArgument` for the conjunction predicate - builder <- buildSearchArgument(dataTypeMap, conjunction, newBuilder) - } yield builder.build() + val conjunctionOptional = buildTree(convertibleFilters(schema, dataTypeMap, filters)) + 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 + // following recursive method call `buildSearchArgument`. + buildSearchArgument(dataTypeMap, conjunction, newBuilder).build() + } + } } + def convertibleFilters( + schema: StructType, + dataTypeMap: Map[String, DataType], + filters: Seq[Filter]): Seq[Filter] = { + import org.apache.spark.sql.sources._ + + def convertibleFiltersHelper( + filter: Filter, + canPartialPushDown: Boolean): Option[Filter] = filter match { + // At here, it is not safe to just convert one side and remove the other side + // if we do not understand what the parent filters are. + // + // Here is an example used to explain the reason. + // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to + // convert b in ('1'). If we only convert a = 2, we will end up with a filter + // NOT(a = 2), which will generate wrong results. + // + // Pushing one side of AND down is only safe to do at the top level or in the child + // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate + // can be safely removed. + case And(left, right) => + val leftResultOptional = convertibleFiltersHelper(left, canPartialPushDown) + val rightResultOptional = convertibleFiltersHelper(right, canPartialPushDown) + (leftResultOptional, rightResultOptional) match { + case (Some(leftResult), Some(rightResult)) => Some(And(leftResult, rightResult)) + case (Some(leftResult), None) if canPartialPushDown => Some(leftResult) + case (None, Some(rightResult)) if canPartialPushDown => Some(rightResult) + case _ => None + } + + // The Or predicate is convertible when both of its children can be pushed down. + // That is to say, if one/both of the children can be partially pushed down, the Or + // predicate can be partially pushed down as well. + // + // Here is an example used to explain the reason. + // Let's say we have + // (a1 AND a2) OR (b1 AND b2), + // a1 and b1 is convertible, while a2 and b2 is not. + // The predicate can be converted as + // (a1 OR b1) AND (a1 OR b2) AND (a2 OR b1) AND (a2 OR b2) + // As per the logical in And predicate, we can push down (a1 OR b1). + case Or(left, right) => + for { + lhs <- convertibleFiltersHelper(left, canPartialPushDown) + rhs <- convertibleFiltersHelper(right, canPartialPushDown) + } yield Or(lhs, rhs) + case Not(pred) => + val childResultOptional = convertibleFiltersHelper(pred, canPartialPushDown = false) + childResultOptional.map(Not) + case other => + for (_ <- buildLeafSearchArgument(dataTypeMap, other, newBuilder())) yield other + } + filters.flatMap { filter => + convertibleFiltersHelper(filter, true) + } + } + + /** + * Build a SearchArgument and return the builder so far. + * + * @param dataTypeMap a map from the attribute name to its data type. + * @param expression the input predicates, which should be fully convertible to SearchArgument. + * @param builder the input SearchArgument.Builder. + * @return the builder so far. + */ private def buildSearchArgument( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder): Option[Builder] = { - createBuilder(dataTypeMap, expression, builder, canPartialPushDownConjuncts = true) + builder: Builder): Builder = { + expression match { + case And(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startAnd()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Or(left, right) => + val lhs = buildSearchArgument(dataTypeMap, left, builder.startOr()) + val rhs = buildSearchArgument(dataTypeMap, right, lhs) + rhs.end() + + case Not(child) => + buildSearchArgument(dataTypeMap, child, builder.startNot()).end() + + case other => + buildLeafSearchArgument(dataTypeMap, other, builder).getOrElse { + throw new SparkException( + "The input filter of OrcFilters.buildSearchArgument should be fully convertible.") + } + } } /** + * Build a SearchArgument for a leaf predicate and return the builder so far. + * * @param dataTypeMap a map from the attribute name to its data type. * @param expression the input filter predicates. * @param builder the input SearchArgument.Builder. - * @param canPartialPushDownConjuncts whether a subset of conjuncts of predicates can be pushed - * down safely. Pushing ONLY one side of AND down is safe to - * do at the top level or none of its ancestors is NOT and OR. * @return the builder so far. */ - private def createBuilder( + private def buildLeafSearchArgument( dataTypeMap: Map[String, DataType], expression: Filter, - builder: Builder, - canPartialPushDownConjuncts: Boolean): Option[Builder] = { + builder: Builder): Option[Builder] = { def isSearchableType(dataType: DataType): Boolean = dataType match { // Only the values in the Spark types below can be recognized by // the `SearchArgumentImpl.BuilderImpl.boxLiteral()` method. @@ -105,86 +197,61 @@ private[orc] object OrcFilters extends Logging { case _ => false } - expression match { - case And(left, right) => - // At here, it is not safe to just convert one side and remove the other side - // if we do not understand what the parent filters are. - // - // Here is an example used to explain the reason. - // Let's say we have NOT(a = 2 AND b in ('1')) and we do not understand how to - // convert b in ('1'). If we only convert a = 2, we will end up with a filter - // NOT(a = 2), which will generate wrong results. - // - // Pushing one side of AND down is only safe to do at the top level or in the child - // AND before hitting NOT or OR conditions, and in this case, the unsupported predicate - // can be safely removed. - val leftBuilderOption = - createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts) - val rightBuilderOption = - createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts) - (leftBuilderOption, rightBuilderOption) match { - case (Some(_), Some(_)) => - for { - lhs <- createBuilder(dataTypeMap, left, - builder.startAnd(), canPartialPushDownConjuncts) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts) - } yield rhs.end() - - case (Some(_), None) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, left, builder, canPartialPushDownConjuncts) - - case (None, Some(_)) if canPartialPushDownConjuncts => - createBuilder(dataTypeMap, right, builder, canPartialPushDownConjuncts) - - case _ => None - } - - case Or(left, right) => - for { - _ <- createBuilder(dataTypeMap, left, newBuilder, canPartialPushDownConjuncts = false) - _ <- createBuilder(dataTypeMap, right, newBuilder, canPartialPushDownConjuncts = false) - lhs <- createBuilder(dataTypeMap, left, - builder.startOr(), canPartialPushDownConjuncts = false) - rhs <- createBuilder(dataTypeMap, right, lhs, canPartialPushDownConjuncts = false) - } yield rhs.end() - - case Not(child) => - for { - _ <- createBuilder(dataTypeMap, child, newBuilder, canPartialPushDownConjuncts = false) - negate <- createBuilder(dataTypeMap, - child, builder.startNot(), canPartialPushDownConjuncts = false) - } yield negate.end() + import org.apache.spark.sql.sources._ + // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` + // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be + // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). + expression match { // NOTE: For all case branches dealing with leaf predicates below, the additional `startAnd()` // call is mandatory. ORC `SearchArgument` builder requires that all leaf predicates must be // wrapped by a "parent" predicate (`And`, `Or`, or `Not`). case EqualTo(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().equals(attribute, value).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "equals", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case EqualNullSafe(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().nullSafeEquals(attribute, value).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "nullSafeEquals", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case LessThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().lessThan(attribute, value).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case LessThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().lessThanEquals(attribute, value).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case GreaterThan(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startNot().lessThanEquals(attribute, value).end()) + val bd = builder.startNot() + val method = findMethod(bd.getClass, "lessThanEquals", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case GreaterThanOrEqual(attribute, value) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startNot().lessThan(attribute, value).end()) + val bd = builder.startNot() + val method = findMethod(bd.getClass, "lessThan", classOf[String], classOf[Object]) + Some(method.invoke(bd, attribute, value.asInstanceOf[AnyRef]).asInstanceOf[Builder].end()) case IsNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().isNull(attribute).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "isNull", classOf[String]) + Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) case IsNotNull(attribute) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startNot().isNull(attribute).end()) + val bd = builder.startNot() + val method = findMethod(bd.getClass, "isNull", classOf[String]) + Some(method.invoke(bd, attribute).asInstanceOf[Builder].end()) case In(attribute, values) if isSearchableType(dataTypeMap(attribute)) => - Some(builder.startAnd().in(attribute, values.map(_.asInstanceOf[AnyRef]): _*).end()) + val bd = builder.startAnd() + val method = findMethod(bd.getClass, "in", classOf[String], classOf[Array[Object]]) + Some(method.invoke(bd, attribute, values.map(_.asInstanceOf[AnyRef])) + .asInstanceOf[Builder].end()) case _ => None } diff --git a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala similarity index 82% rename from core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala rename to sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala index 7249eb85ac7c7..faee405d70cdd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/security/HiveDelegationTokenProvider.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/security/HiveDelegationTokenProvider.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy.security +package org.apache.spark.sql.hive.security import java.lang.reflect.UndeclaredThrowableException import java.security.PrivilegedExceptionAction @@ -34,6 +34,7 @@ import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging import org.apache.spark.internal.config.KEYTAB +import org.apache.spark.security.HadoopDelegationTokenProvider import org.apache.spark.util.Utils private[spark] class HiveDelegationTokenProvider @@ -49,7 +50,7 @@ private[spark] class HiveDelegationTokenProvider new HiveConf(hadoopConf, classOf[HiveConf]) } catch { case NonFatal(e) => - logDebug("Fail to create Hive Configuration", e) + logWarning("Fail to create Hive Configuration", e) hadoopConf case e: NoClassDefFoundError => logWarning(classNotFoundErrorStr) @@ -67,11 +68,17 @@ private[spark] class HiveDelegationTokenProvider // Other modes (such as client with or without keytab, or cluster mode with keytab) do not need // a delegation token, since there's a valid kerberos TGT for the right user available to the // driver, which is the only process that connects to the HMS. - val deployMode = sparkConf.get("spark.submit.deployMode", "client") - UserGroupInformation.isSecurityEnabled && + // + // Note that this means Hive tokens are not re-created periodically by the token manager. + // This is because HMS connections are only performed by the Spark driver, and the driver + // either has a TGT, in which case it does not need tokens, or it has a token created + // elsewhere, in which case it cannot create new ones. The check for an existing token avoids + // printing an exception to the logs in the latter case. + val currentToken = UserGroupInformation.getCurrentUser().getCredentials().getToken(tokenAlias) + currentToken == null && UserGroupInformation.isSecurityEnabled && hiveConf(hadoopConf).getTrimmed("hive.metastore.uris", "").nonEmpty && (SparkHadoopUtil.get.isProxyUser(UserGroupInformation.getCurrentUser()) || - (deployMode == "cluster" && !sparkConf.contains(KEYTAB))) + (!Utils.isClientMode(sparkConf) && !sparkConf.contains(KEYTAB))) } override def obtainDelegationTokens( @@ -98,13 +105,13 @@ private[spark] class HiveDelegationTokenProvider val hive2Token = new Token[DelegationTokenIdentifier]() hive2Token.decodeFromUrlString(tokenStr) logDebug(s"Get Token from hive metastore: ${hive2Token.toString}") - creds.addToken(new Text("hive.server2.delegation.token"), hive2Token) + creds.addToken(tokenAlias, hive2Token) } None } catch { case NonFatal(e) => - logDebug(s"Failed to get token from service $serviceName", e) + logWarning(s"Failed to get token from service $serviceName", e) None case e: NoClassDefFoundError => logWarning(classNotFoundErrorStr) @@ -134,4 +141,6 @@ private[spark] class HiveDelegationTokenProvider case e: UndeclaredThrowableException => throw Option(e.getCause()).getOrElse(e) } } + + private def tokenAlias: Text = new Text("hive.server2.delegation.token") } diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f index 6f1ab55c41ce7..e5461ca8299f8 100644 --- a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f @@ -1,4 +1,4 @@ a ^ b - Bitwise exclusive or Example: > SELECT 3 ^ 5 FROM src LIMIT 1; - 2 + 6 diff --git a/sql/hive/src/test/resources/hive-contrib-0.13.1.jar b/sql/hive/src/test/resources/hive-contrib-0.13.1.jar deleted file mode 100644 index ce0740d9245a7..0000000000000 Binary files a/sql/hive/src/test/resources/hive-contrib-0.13.1.jar and /dev/null differ diff --git a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar b/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar deleted file mode 100644 index 37af9aafad8a4..0000000000000 Binary files a/sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar and /dev/null differ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala index 61f9179042fe4..ae701f266bf45 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/catalyst/ExpressionSQLBuilderSuite.scala @@ -17,12 +17,14 @@ package org.apache.spark.sql.catalyst -import java.sql.Timestamp +import java.time.LocalDateTime import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.util.DateTimeUtils import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.unsafe.types.CalendarInterval class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { @@ -60,8 +62,10 @@ class ExpressionSQLBuilderSuite extends QueryTest with TestHiveSingleton { checkSQL(Literal(Double.NaN), "CAST('NaN' AS DOUBLE)") checkSQL(Literal(BigDecimal("10.0000000").underlying), "10.0000000BD") checkSQL(Literal(Array(0x01, 0xA3).map(_.toByte)), "X'01A3'") - checkSQL( - Literal(Timestamp.valueOf("2016-01-01 00:00:00")), "TIMESTAMP('2016-01-01 00:00:00.0')") + val timestamp = LocalDateTime.of(2016, 1, 1, 0, 0, 0) + .atZone(DateTimeUtils.getZoneId(SQLConf.get.sessionLocalTimeZone)) + .toInstant + checkSQL(Literal(timestamp), "TIMESTAMP('2016-01-01 00:00:00')") // TODO tests for decimals } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala index 3b33785cdfbb2..3226e3a5f318a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/execution/benchmark/ObjectHashAggregateExecBenchmark.scala @@ -21,207 +21,189 @@ import scala.concurrent.duration._ import org.apache.hadoop.hive.ql.udf.generic.GenericUDAFPercentileApprox -import org.apache.spark.benchmark.Benchmark -import org.apache.spark.sql.Column -import org.apache.spark.sql.catalyst.FunctionIdentifier -import org.apache.spark.sql.catalyst.catalog.CatalogFunction +import org.apache.spark.benchmark.{Benchmark, BenchmarkBase} +import org.apache.spark.sql.{Column, SparkSession} import org.apache.spark.sql.catalyst.expressions.Literal import org.apache.spark.sql.catalyst.expressions.aggregate.ApproximatePercentile -import org.apache.spark.sql.hive.HiveSessionCatalog +import org.apache.spark.sql.catalyst.plans.SQLHelper import org.apache.spark.sql.hive.execution.TestingTypedCount -import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types.LongType -class ObjectHashAggregateExecBenchmark extends BenchmarkWithCodegen with TestHiveSingleton { - ignore("Hive UDAF vs Spark AF") { - val N = 2 << 15 +/** + * Benchmark to measure hash based aggregation. + * To run this benchmark: + * {{{ + * 1. without sbt: bin/spark-submit --class + * --jars ,, + * --packages org.spark-project.hive:hive-exec:1.2.1.spark2 + * + * 2. build/sbt "hive/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " + * Results will be written to "benchmarks/ObjectHashAggregateExecBenchmark-results.txt". + * }}} + */ +object ObjectHashAggregateExecBenchmark extends BenchmarkBase with SQLHelper { + + private val spark: SparkSession = TestHive.sparkSession + private val sql = spark.sql _ + import spark.implicits._ + private def hiveUDAFvsSparkAF(N: Int): Unit = { val benchmark = new Benchmark( name = "hive udaf vs spark af", valuesPerIteration = N, minNumIters = 5, warmupTime = 5.seconds, minTime = 10.seconds, - outputPerIteration = true + outputPerIteration = true, + output = output ) - registerHiveFunction("hive_percentile_approx", classOf[GenericUDAFPercentileApprox]) + sql( + s"CREATE TEMPORARY FUNCTION hive_percentile_approx AS '" + + s"${classOf[GenericUDAFPercentileApprox].getName}'" + ) - sparkSession.range(N).createOrReplaceTempView("t") + spark.range(N).createOrReplaceTempView("t") benchmark.addCase("hive udaf w/o group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - sparkSession.sql("SELECT hive_percentile_approx(id, 0.5) FROM t").collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + sql("SELECT hive_percentile_approx(id, 0.5) FROM t").collect() + } } benchmark.addCase("spark af w/o group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - sparkSession.sql("SELECT percentile_approx(id, 0.5) FROM t").collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + sql("SELECT percentile_approx(id, 0.5) FROM t").collect() + } } benchmark.addCase("hive udaf w/ group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - sparkSession.sql( - s"SELECT hive_percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" - ).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + sql( + s"SELECT hive_percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" + ).collect() + } } benchmark.addCase("spark af w/ group by w/o fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - sparkSession.sql( - s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" - ).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + sql(s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)") + .collect() + } } benchmark.addCase("spark af w/ group by w/ fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") - sparkSession.sql( - s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)" - ).collect() + withSQLConf( + SQLConf.USE_OBJECT_HASH_AGG.key -> "true", + SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { + sql(s"SELECT percentile_approx(id, 0.5) FROM t GROUP BY CAST(id / ${N / 4} AS BIGINT)") + .collect() + } } benchmark.run() - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - hive udaf vs spark af: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - hive udaf w/o group by 5326 / 5408 0.0 81264.2 1.0X - spark af w/o group by 93 / 111 0.7 1415.6 57.4X - hive udaf w/ group by 3804 / 3946 0.0 58050.1 1.4X - spark af w/ group by w/o fallback 71 / 90 0.9 1085.7 74.8X - spark af w/ group by w/ fallback 98 / 111 0.7 1501.6 54.1X - */ } - ignore("ObjectHashAggregateExec vs SortAggregateExec - typed_count") { - val N: Long = 1024 * 1024 * 100 - + private def objectHashAggregateExecVsSortAggregateExecUsingTypedCount(N: Int): Unit = { val benchmark = new Benchmark( name = "object agg v.s. sort agg", valuesPerIteration = N, minNumIters = 1, warmupTime = 10.seconds, minTime = 45.seconds, - outputPerIteration = true + outputPerIteration = true, + output = output ) - import sparkSession.implicits._ - def typed_count(column: Column): Column = Column(TestingTypedCount(column.expr).toAggregateExpression()) - val df = sparkSession.range(N) + val df = spark.range(N) benchmark.addCase("sort agg w/ group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } } benchmark.addCase("object agg w/ group by w/o fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } } benchmark.addCase("object agg w/ group by w/ fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") - df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + withSQLConf( + SQLConf.USE_OBJECT_HASH_AGG.key -> "true", + SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { + df.groupBy($"id" < (N / 2)).agg(typed_count($"id")).collect() + } } benchmark.addCase("sort agg w/o group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - df.select(typed_count($"id")).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + df.select(typed_count($"id")).collect() + } } benchmark.addCase("object agg w/o group by w/o fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - df.select(typed_count($"id")).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + df.select(typed_count($"id")).collect() + } } benchmark.run() - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sort agg w/ group by 31251 / 31908 3.4 298.0 1.0X - object agg w/ group by w/o fallback 6903 / 7141 15.2 65.8 4.5X - object agg w/ group by w/ fallback 20945 / 21613 5.0 199.7 1.5X - sort agg w/o group by 4734 / 5463 22.1 45.2 6.6X - object agg w/o group by w/o fallback 4310 / 4529 24.3 41.1 7.3X - */ } - ignore("ObjectHashAggregateExec vs SortAggregateExec - percentile_approx") { - val N = 2 << 20 - + private def objectHashAggregateExecVsSortAggregateExecUsingPercentileApprox(N: Int): Unit = { val benchmark = new Benchmark( name = "object agg v.s. sort agg", valuesPerIteration = N, minNumIters = 5, warmupTime = 15.seconds, minTime = 45.seconds, - outputPerIteration = true + outputPerIteration = true, + output = output ) - import sparkSession.implicits._ - - val df = sparkSession.range(N).coalesce(1) + val df = spark.range(N).coalesce(1) benchmark.addCase("sort agg w/ group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } } benchmark.addCase("object agg w/ group by w/o fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } } benchmark.addCase("object agg w/ group by w/ fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - sparkSession.conf.set(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key, "2") - df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + withSQLConf( + SQLConf.USE_OBJECT_HASH_AGG.key -> "true", + SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "2") { + df.groupBy($"id" / (N / 4) cast LongType).agg(percentile_approx($"id", 0.5)).collect() + } } benchmark.addCase("sort agg w/o group by") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "false") - df.select(percentile_approx($"id", 0.5)).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "false") { + df.select(percentile_approx($"id", 0.5)).collect() + } } benchmark.addCase("object agg w/o group by w/o fallback") { _ => - sparkSession.conf.set(SQLConf.USE_OBJECT_HASH_AGG.key, "true") - df.select(percentile_approx($"id", 0.5)).collect() + withSQLConf(SQLConf.USE_OBJECT_HASH_AGG.key -> "true") { + df.select(percentile_approx($"id", 0.5)).collect() + } } benchmark.run() - - /* - Java HotSpot(TM) 64-Bit Server VM 1.8.0_92-b14 on Mac OS X 10.10.5 - Intel(R) Core(TM) i7-4960HQ CPU @ 2.60GHz - - object agg v.s. sort agg: Best/Avg Time(ms) Rate(M/s) Per Row(ns) Relative - ------------------------------------------------------------------------------------------------ - sort agg w/ group by 3418 / 3530 0.6 1630.0 1.0X - object agg w/ group by w/o fallback 3210 / 3314 0.7 1530.7 1.1X - object agg w/ group by w/ fallback 3419 / 3511 0.6 1630.1 1.0X - sort agg w/o group by 4336 / 4499 0.5 2067.3 0.8X - object agg w/o group by w/o fallback 4271 / 4372 0.5 2036.7 0.8X - */ - } - - private def registerHiveFunction(functionName: String, clazz: Class[_]): Unit = { - val sessionCatalog = sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog] - val functionIdentifier = FunctionIdentifier(functionName, database = None) - val func = CatalogFunction(functionIdentifier, clazz.getName, resources = Nil) - sessionCatalog.registerFunction(func, overrideIfExists = false) } private def percentile_approx( @@ -229,4 +211,18 @@ class ObjectHashAggregateExecBenchmark extends BenchmarkWithCodegen with TestHiv val approxPercentile = new ApproximatePercentile(column.expr, Literal(percentage)) Column(approxPercentile.toAggregateExpression(isDistinct)) } + + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { + runBenchmark("Hive UDAF vs Spark AF") { + hiveUDAFvsSparkAF(2 << 15) + } + + runBenchmark("ObjectHashAggregateExec vs SortAggregateExec - typed_count") { + objectHashAggregateExecVsSortAggregateExecUsingTypedCount(1024 * 1024 * 100) + } + + runBenchmark("ObjectHashAggregateExec vs SortAggregateExec - percentile_approx") { + objectHashAggregateExecVsSortAggregateExecUsingPercentileApprox(2 << 20) + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 569f00c053e5f..fcf0b4591ff84 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.sql.hive import java.io.File import org.apache.spark.sql.{AnalysisException, Dataset, QueryTest, SaveMode} -import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.execution.columnar.InMemoryTableScanExec import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation} @@ -29,6 +28,7 @@ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType import org.apache.spark.storage.RDDBlockId +import org.apache.spark.storage.StorageLevel.{DISK_ONLY, MEMORY_ONLY} import org.apache.spark.util.Utils class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { @@ -97,24 +97,24 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } - test("DROP nonexistant table") { - sql("DROP TABLE IF EXISTS nonexistantTable") + test("DROP nonexistent table") { + sql("DROP TABLE IF EXISTS nonexistentTable") } - test("uncache of nonexistant tables") { - val expectedErrorMsg = "Table or view not found: nonexistantTable" + test("uncache of nonexistent tables") { + val expectedErrorMsg = "Table or view not found: nonexistentTable" // make sure table doesn't exist - var e = intercept[AnalysisException](spark.table("nonexistantTable")).getMessage + var e = intercept[AnalysisException](spark.table("nonexistentTable")).getMessage assert(e.contains(expectedErrorMsg)) e = intercept[AnalysisException] { - spark.catalog.uncacheTable("nonexistantTable") + uncacheTable("nonexistentTable") }.getMessage assert(e.contains(expectedErrorMsg)) e = intercept[AnalysisException] { - sql("UNCACHE TABLE nonexistantTable") + sql("UNCACHE TABLE nonexistentTable") }.getMessage assert(e.contains(expectedErrorMsg)) - sql("UNCACHE TABLE IF EXISTS nonexistantTable") + sql("UNCACHE TABLE IF EXISTS nonexistentTable") } test("no error on uncache of non-cached table") { @@ -122,9 +122,9 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto withTable(tableName) { sql(s"CREATE TABLE $tableName(a INT)") // no error will be reported in the following three ways to uncache a table. - spark.catalog.uncacheTable(tableName) + uncacheTable(tableName) sql("UNCACHE TABLE newTable") - sparkSession.table(tableName).unpersist() + sparkSession.table(tableName).unpersist(blocking = true) } } @@ -351,4 +351,81 @@ class CachedTableSuite extends QueryTest with SQLTestUtils with TestHiveSingleto assert(spark.sharedState.cacheManager.lookupCachedData(samePlan).isDefined) } } + + test("SPARK-27248 refreshTable should recreate cache with same cache name and storage level") { + // This section tests when a table is cached with its qualified name but its is refreshed with + // its unqualified name. + withTempDatabase { db => + withTable(s"$db.cachedTable") { + withCache(s"$db.cachedTable") { + // Create table 'cachedTable' in default db for testing purpose. + sql(s"CREATE TABLE $db.cachedTable AS SELECT 1 AS key") + + // Cache the table 'cachedTable' in temp db with qualified table name, + // and then check whether the table is cached with expected name + sql(s"CACHE TABLE $db.cachedTable OPTIONS('storageLevel' 'MEMORY_ONLY')") + assertCached(sql(s"SELECT * FROM $db.cachedTable"), s"`$db`.`cachedTable`", MEMORY_ONLY) + assert(spark.catalog.isCached(s"$db.cachedTable"), + s"Table '$db.cachedTable' should be cached.") + + // Refresh the table 'cachedTable' in temp db with qualified table name, and then check + // whether the table is still cached with the same name and storage level. + sql(s"REFRESH TABLE $db.cachedTable") + assertCached(sql(s"select * from $db.cachedTable"), s"`$db`.`cachedTable`", MEMORY_ONLY) + assert(spark.catalog.isCached(s"$db.cachedTable"), + s"Table '$db.cachedTable' should be cached after refreshing with its qualified name.") + + // Change the active database to the temp db and refresh the table with unqualified + // table name, and then check whether the table is still cached with the same name and + // storage level. + // Without bug fix 'SPARK-27248', the recreated cache name will be changed to + // 'cachedTable', instead of '$db.cachedTable' + activateDatabase(db) { + sql("REFRESH TABLE cachedTable") + assertCached(sql("SELECT * FROM cachedTable"), s"`$db`.`cachedTable`", MEMORY_ONLY) + assert(spark.catalog.isCached("cachedTable"), + s"Table '$db.cachedTable' should be cached after refreshing with its " + + "unqualified name.") + } + } + } + } + + + // This section tests when a table is cached with its unqualified name but it is refreshed + // with its qualified name. + withTempDatabase { db => + withTable("cachedTable") { + withCache("cachedTable") { + // Create table 'cachedTable' in default db for testing purpose. + sql("CREATE TABLE cachedTable AS SELECT 1 AS key") + + // Cache the table 'cachedTable' in default db without qualified table name , and then + // check whether the table is cached with expected name. + sql("CACHE TABLE cachedTable OPTIONS('storageLevel' 'DISK_ONLY')") + assertCached(sql("SELECT * FROM cachedTable"), "`cachedTable`", DISK_ONLY) + assert(spark.catalog.isCached("cachedTable"), "Table 'cachedTable' should be cached.") + + // Refresh the table 'cachedTable' in default db with unqualified table name, and then + // check whether the table is still cached with the same name. + sql("REFRESH TABLE cachedTable") + assertCached(sql("SELECT * FROM cachedTable"), "`cachedTable`", DISK_ONLY) + assert(spark.catalog.isCached("cachedTable"), + "Table 'cachedTable' should be cached after refreshing with its unqualified name.") + + // Change the active database to the temp db and refresh the table with qualified + // table name, and then check whether the table is still cached with the same name and + // storage level. + // Without bug fix 'SPARK-27248', the recreated cache name will be changed to + // 'default.cachedTable', instead of 'cachedTable' + activateDatabase(db) { + sql("REFRESH TABLE default.cachedTable") + assertCached(sql("SELECT * FROM default.cachedTable"), "`cachedTable`", DISK_ONLY) + assert(spark.catalog.isCached("default.cachedTable"), + "Table 'cachedTable' should be cached after refreshing with its qualified name.") + } + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala index f262ef62be036..a696d6aaff27b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/ClasspathDependenciesSuite.scala @@ -58,11 +58,19 @@ class ClasspathDependenciesSuite extends SparkFunSuite { } test("shaded Protobuf") { - assertLoads("org.apache.hive.com.google.protobuf.ServiceException") + if (HiveUtils.isHive23) { + assertLoads("com.google.protobuf.ServiceException") + } else { + assertLoads("org.apache.hive.com.google.protobuf.ServiceException") + } } test("shaded Kryo") { - assertLoads("org.apache.hive.com.esotericsoftware.kryo.Kryo") + if (HiveUtils.isHive23) { + assertLoads("com.esotericsoftware.kryo.Kryo") + } else { + assertLoads("org.apache.hive.com.esotericsoftware.kryo.Kryo") + } } test("hive-common") { @@ -81,7 +89,12 @@ class ClasspathDependenciesSuite extends SparkFunSuite { } test("parquet-hadoop-bundle") { - assertLoads("parquet.hadoop.ParquetOutputFormat") - assertLoads("parquet.hadoop.ParquetInputFormat") + if (HiveUtils.isHive23) { + assertLoads("org.apache.parquet.hadoop.ParquetOutputFormat") + assertLoads("org.apache.parquet.hadoop.ParquetInputFormat") + } else { + assertLoads("parquet.hadoop.ParquetOutputFormat") + assertLoads("parquet.hadoop.ParquetInputFormat") + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala index 1de258f060943..0a522b6a11c80 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala @@ -113,10 +113,4 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite { catalog.createDatabase(newDb("dbWithNullDesc").copy(description = null), ignoreIfExists = false) assert(catalog.getDatabase("dbWithNullDesc").description == "") } - - test("SPARK-23831: Add org.apache.derby to IsolatedClientLoader") { - val client1 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) - val client2 = HiveUtils.newClientForMetadata(new SparkConf, new Configuration) - assert(!client1.equals(client2)) - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala index fd4985d131885..9bc0be87be5af 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogVersionsSuite.scala @@ -22,13 +22,17 @@ import java.nio.charset.StandardCharsets import java.nio.file.{Files, Paths} import scala.sys.process._ +import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration import org.apache.spark.{SecurityManager, SparkConf, TestUtils} +import org.apache.spark.internal.config.MASTER_REST_SERVER_ENABLED +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog.CatalogTableType +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.util.Utils @@ -69,7 +73,8 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { case _: Exception => None } } - val sites = mirrors.distinct :+ "https://archive.apache.org/dist" + val sites = + mirrors.distinct :+ "https://archive.apache.org/dist" :+ PROCESS_TABLES.releaseMirror logInfo(s"Trying to download Spark $version from $sites") for (site <- sites) { val filename = s"spark-$version-bin-hadoop2.7.tgz" @@ -169,6 +174,10 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { """.stripMargin.getBytes("utf8")) // scalastyle:on line.size.limit + if (PROCESS_TABLES.testingVersions.isEmpty) { + fail("Fail to get the lates Spark versions to test.") + } + PROCESS_TABLES.testingVersions.zipWithIndex.foreach { case (version, index) => val sparkHome = new File(sparkTestingDir, s"spark-$version") if (!sparkHome.exists()) { @@ -178,9 +187,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { val args = Seq( "--name", "prepare testing tables", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--conf", s"spark.sql.test.version.index=$index", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", tempPyFile.getCanonicalPath) @@ -195,9 +206,11 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { "--class", PROCESS_TABLES.getClass.getName.stripSuffix("$"), "--name", "HiveExternalCatalog backward compatibility test", "--master", "local[2]", - "--conf", "spark.ui.enabled=false", - "--conf", "spark.master.rest.enabled=false", - "--conf", s"spark.sql.warehouse.dir=${wareHousePath.getCanonicalPath}", + "--conf", s"${UI_ENABLED.key}=false", + "--conf", s"${MASTER_REST_SERVER_ENABLED.key}=false", + "--conf", s"${HiveUtils.HIVE_METASTORE_VERSION.key}=1.2.1", + "--conf", s"${HiveUtils.HIVE_METASTORE_JARS.key}=maven", + "--conf", s"${WAREHOUSE_PATH.key}=${wareHousePath.getCanonicalPath}", "--driver-java-options", s"-Dderby.system.home=${wareHousePath.getCanonicalPath}", unusedJar.toString) runSparkSubmit(args) @@ -205,8 +218,21 @@ class HiveExternalCatalogVersionsSuite extends SparkSubmitTestUtils { } object PROCESS_TABLES extends QueryTest with SQLTestUtils { + val releaseMirror = "https://dist.apache.org/repos/dist/release" // Tests the latest version of every release line. - val testingVersions = Seq("2.1.3", "2.2.2", "2.3.2") + val testingVersions: Seq[String] = { + import scala.io.Source + try { + Source.fromURL(s"${releaseMirror}/spark").mkString + .split("\n") + .filter(_.contains("""
  • """.r.findFirstMatchIn(_).get.group(1)) + .filter(_ < org.apache.spark.SPARK_VERSION) + } catch { + // do not throw exception during object initialization. + case NonFatal(_) => Nil + } + } protected var spark: SparkSession = _ @@ -260,19 +286,10 @@ object PROCESS_TABLES extends QueryTest with SQLTestUtils { // SPARK-22356: overlapped columns between data and partition schema in data source tables val tbl_with_col_overlap = s"tbl_with_col_overlap_$index" - // For Spark 2.2.0 and 2.1.x, the behavior is different from Spark 2.0, 2.2.1, 2.3+ - if (testingVersions(index).startsWith("2.1") || testingVersions(index) == "2.2.0") { - spark.sql("msck repair table " + tbl_with_col_overlap) - assert(spark.table(tbl_with_col_overlap).columns === Array("i", "j", "p")) - checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) - assert(sql("desc " + tbl_with_col_overlap).select("col_name") - .as[String].collect().mkString(",").contains("i,j,p")) - } else { - assert(spark.table(tbl_with_col_overlap).columns === Array("i", "p", "j")) - checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) - assert(sql("desc " + tbl_with_col_overlap).select("col_name") - .as[String].collect().mkString(",").contains("i,p,j")) - } + assert(spark.table(tbl_with_col_overlap).columns === Array("i", "p", "j")) + checkAnswer(spark.table(tbl_with_col_overlap), Row(1, 1, 1) :: Row(1, 1, 1) :: Nil) + assert(sql("desc " + tbl_with_col_overlap).select("col_name") + .as[String].collect().mkString(",").contains("i,p,j")) } } } 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 688b619cd1bb5..deb0a1085714e 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 @@ -159,10 +159,28 @@ class DataSourceWithHiveMetastoreCatalogSuite "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" )), + "org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat" -> (( + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat", + "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat", + "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe" + )), + "orc" -> (( "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", "org.apache.hadoop.hive.ql.io.orc.OrcSerde" + )), + + "org.apache.spark.sql.hive.orc" -> (( + "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcSerde" + )), + + "org.apache.spark.sql.execution.datasources.orc.OrcFileFormat" -> (( + "org.apache.hadoop.hive.ql.io.orc.OrcInputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat", + "org.apache.hadoop.hive.ql.io.orc.OrcSerde" )) ).foreach { case (provider, (inputFormat, outputFormat, serde)) => test(s"Persist non-partitioned $provider relation into metastore as managed table") { @@ -188,7 +206,13 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === Seq("1.1\t1", "2.1\t2")) + if (HiveUtils.isHive23) { + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.100\t1", "2.100\t2")) + } else { + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.1\t1", "2.1\t2")) + } } } @@ -220,8 +244,13 @@ class DataSourceWithHiveMetastoreCatalogSuite assert(columns.map(_.dataType) === Seq(DecimalType(10, 3), StringType)) checkAnswer(table("t"), testDF) - assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === - Seq("1.1\t1", "2.1\t2")) + if (HiveUtils.isHive23) { + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.100\t1", "2.100\t2")) + } else { + assert(sparkSession.metadataHive.runSqlHive("SELECT * FROM t") === + Seq("1.1\t1", "2.1\t2")) + } } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala index 0d4f040156084..68a0c1213ec20 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetMetastoreSuite.scala @@ -152,7 +152,7 @@ class HiveParquetMetastoreSuite extends ParquetPartitioningTest { } (1 to 10).map(i => (i, s"str$i")).toDF("a", "b").createOrReplaceTempView("jt") - (1 to 10).map(i => Tuple1(Seq(new Integer(i), null))).toDF("a") + (1 to 10).map(i => Tuple1(Seq(Integer.valueOf(i), null))).toDF("a") .createOrReplaceTempView("jt_array") assert(spark.sqlContext.getConf(HiveUtils.CONVERT_METASTORE_PARQUET.key) == "true") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala index e5c9df05d5674..470c6a342b4dd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveParquetSuite.scala @@ -92,4 +92,18 @@ class HiveParquetSuite extends QueryTest with ParquetTest with TestHiveSingleton } } } + + test("SPARK-25271: write empty map into hive parquet table") { + import testImplicits._ + + Seq(Map(1 -> "a"), Map.empty[Int, String]).toDF("m").createOrReplaceTempView("p") + withTempView("p") { + val targetTable = "targetTable" + withTable(targetTable) { + sql(s"CREATE TABLE $targetTable STORED AS PARQUET AS SELECT m FROM p") + checkAnswer(sql(s"SELECT m FROM $targetTable"), + Row(Map(1 -> "a")) :: Row(Map.empty[Int, String]) :: Nil) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala index aa4fc13333c48..590ef949ffbd7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala @@ -264,6 +264,32 @@ class HiveSchemaInferenceSuite StructType(Seq(StructField("lowerCase", BinaryType)))) } + // Parquet schema is subset of metaStore schema and has uppercase field name + assertResult( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = true), + StructField("lowerCase", BinaryType, nullable = true)))) { + + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = true), + StructField("lowerCase", BinaryType, nullable = true))), + + StructType(Seq( + StructField("lowerCase", BinaryType, nullable = true)))) + } + + // Metastore schema contains additional nullable fields. + assert(intercept[Throwable] { + HiveMetastoreCatalog.mergeWithMetastoreSchema( + StructType(Seq( + StructField("UPPERCase", DoubleType, nullable = false), + StructField("lowerCase", BinaryType, nullable = true))), + + StructType(Seq( + StructField("lowerCase", BinaryType, nullable = true)))) + }.getMessage.contains("Detected conflicting schemas")) + // Check that merging missing nullable fields works as expected. assertResult( StructType(Seq( 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 new file mode 100644 index 0000000000000..6e2dcfc04d498 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSharedStateSuite.scala @@ -0,0 +1,66 @@ +/* + * 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.hadoop.hive.conf.HiveConf.ConfVars + +import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.sql.internal.SharedState +import org.apache.spark.sql.internal.StaticSQLConf._ +import org.apache.spark.util.Utils + +class HiveSharedStateSuite extends SparkFunSuite { + + test("initial configs should be passed to SharedState but not SparkContext") { + val conf = new SparkConf().setMaster("local").setAppName("SharedState Test") + val sc = SparkContext.getOrCreate(conf) + val invalidPath = "invalid/path" + val metastorePath = Utils.createTempDir() + val tmpDb = "tmp_db" + + // The initial configs used to generate SharedState, none of these should affect the global + // shared SparkContext's configurations. Especially, all these configs are passed to the cloned + // confs inside SharedState except metastore warehouse dir. + val initialConfigs = Map("spark.foo" -> "bar", + WAREHOUSE_PATH.key -> invalidPath, + ConfVars.METASTOREWAREHOUSE.varname -> invalidPath, + CATALOG_IMPLEMENTATION.key -> "hive", + ConfVars.METASTORECONNECTURLKEY.varname -> + s"jdbc:derby:;databaseName=$metastorePath/metastore_db;create=true", + 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, + "warehouse conf in session options can't affect application wide hadoop conf") + + assert(!state.sparkContext.conf.contains("spark.foo"), + "static spark conf should not be affected by session") + assert(state.externalCatalog.unwrapped.isInstanceOf[HiveExternalCatalog], + "Initial SparkSession options can determine the catalog") + val client = state.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client + assert(client.getConf("spark.foo", "") === "bar", + "session level conf should be passed to catalog") + assert(client.getConf(ConfVars.METASTOREWAREHOUSE.varname, invalidPath) !== invalidPath, + "session level conf should be passed to catalog except warehouse dir") + + assert(state.globalTempViewManager.database === tmpDb) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala new file mode 100644 index 0000000000000..14d07cdf8db08 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShimSuite.scala @@ -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. + */ +package org.apache.spark.sql.hive + +import scala.collection.JavaConverters._ +import scala.language.implicitConversions + +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.hive.serde2.ColumnProjectionUtils + +import org.apache.spark.SparkFunSuite + +class HiveShimSuite extends SparkFunSuite { + + test("appendReadColumns") { + val conf = new Configuration + val ids = Seq(1, 2, 3).map(Int.box) + val names = Seq("a", "b", "c") + val moreIds = Seq(4, 5).map(Int.box) + val moreNames = Seq("d", "e") + + // test when READ_COLUMN_NAMES_CONF_STR is empty + HiveShim.appendReadColumns(conf, ids, names) + if (HiveUtils.isHive23) { + assert(names === ColumnProjectionUtils.getReadColumnNames(conf)) + } else { + assert(names.asJava === ColumnProjectionUtils.getReadColumnNames(conf)) + } + + // test when READ_COLUMN_NAMES_CONF_STR is non-empty + HiveShim.appendReadColumns(conf, moreIds, moreNames) + if (HiveUtils.isHive23) { + assert((names ++ moreNames) === ColumnProjectionUtils.getReadColumnNames(conf)) + } else { + assert((names ++ moreNames).asJava === ColumnProjectionUtils.getReadColumnNames(conf)) + } + } +} 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 new file mode 100644 index 0000000000000..0386dc79804c6 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveShowCreateTableSuite.scala @@ -0,0 +1,198 @@ +/* + * 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.{AnalysisException, ShowCreateTableSuite} +import org.apache.spark.sql.hive.test.TestHiveSingleton + +class HiveShowCreateTableSuite extends ShowCreateTableSuite with TestHiveSingleton { + + test("simple hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("simple external hive table") { + withTempDir { dir => + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |LOCATION '${dir.toURI}' + |TBLPROPERTIES ( + | 'prop1' = 'value1', + | 'prop2' = 'value2' + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + } + + test("partitioned hive table") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |COMMENT 'bla' + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING + |) + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with explicit storage info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' + |COLLECTION ITEMS TERMINATED BY '@' + |MAP KEYS TERMINATED BY '#' + |NULL DEFINED AS 'NaN' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with STORED AS clause") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |STORED AS PARQUET + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive table with serde info") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 ( + | c1 INT COMMENT 'bla', + | c2 STRING + |) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |WITH SERDEPROPERTIES ( + | 'mapkey.delim' = ',', + | 'field.delim' = ',' + |) + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + """.stripMargin + ) + + checkCreateTable("t1") + } + } + + test("hive bucketing is supported") { + withTable("t1") { + sql( + s"""CREATE TABLE t1 (a INT, b STRING) + |CLUSTERED BY (a) + |SORTED BY (b) + |INTO 2 BUCKETS + """.stripMargin + ) + checkCreateTable("t1") + } + } + + test("hive partitioned view is not supported") { + withTable("t1") { + withView("v1") { + sql( + s""" + |CREATE TABLE t1 (c1 INT, c2 STRING) + |PARTITIONED BY ( + | p1 BIGINT COMMENT 'bla', + | p2 STRING ) + """.stripMargin) + + createRawHiveTable( + s""" + |CREATE VIEW v1 + |PARTITIONED ON (p1, p2) + |AS SELECT * from t1 + """.stripMargin + ) + + val cause = intercept[AnalysisException] { + sql("SHOW CREATE TABLE v1") + } + + assert(cause.getMessage.contains(" - partitioned view")) + } + } + } + + test("SPARK-24911: keep quotes for nested fields in hive") { + withTable("t1") { + val createTable = "CREATE TABLE `t1`(`a` STRUCT<`b`: STRING>)" + sql(createTable) + val shownDDL = sql(s"SHOW CREATE TABLE t1") + .head() + .getString(0) + .split("\n") + .head + assert(shownDDL == createTable) + + checkCreateTable("t1") + } + } + + private def createRawHiveTable(ddl: String): Unit = { + hiveContext.sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog] + .client.runSqlHive(ddl) + } +} 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 f839e8979d355..e2ddec3427665 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 @@ -26,12 +26,15 @@ import org.scalatest.{BeforeAndAfterEach, Matchers} import org.apache.spark._ import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.{QueryTest, Row, SparkSession} import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier} import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.execution.command.DDLUtils import org.apache.spark.sql.expressions.Window -import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} +import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHiveContext} +import org.apache.spark.sql.internal.SQLConf.SHUFFLE_PARTITIONS +import org.apache.spark.sql.internal.StaticSQLConf.WAREHOUSE_PATH import org.apache.spark.sql.types.{DecimalType, StructType} import org.apache.spark.tags.ExtendedHiveTest import org.apache.spark.util.{ResetSystemProperties, Utils} @@ -107,8 +110,8 @@ class HiveSparkSubmitSuite val unusedJar = TestUtils.createJarWithClasses(Seq.empty) val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) - val jar3 = TestHive.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath - val jar4 = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").getCanonicalPath + val jar3 = HiveTestUtils.getHiveContribJar.getCanonicalPath + val jar4 = HiveTestUtils.getHiveHcatalogCoreJar.getCanonicalPath val jarsString = Seq(jar1, jar2, jar3, jar4).map(j => j.toString).mkString(",") val args = Seq( "--class", SparkSubmitClassLoaderTest.getClass.getName.stripSuffix("$"), @@ -143,8 +146,9 @@ class HiveSparkSubmitSuite // Before the fix in SPARK-8470, this results in a MissingRequirementError because // the HiveContext code mistakenly overrides the class loader that contains user classes. // For more detail, see sql/hive/src/test/resources/regression-test-SPARK-8489/*scala. + // TODO: revisit for Scala 2.13 support val version = Properties.versionNumberString match { - case v if v.startsWith("2.12") || v.startsWith("2.11") => v.substring(0, 4) + case v if v.startsWith("2.12") => v.substring(0, 4) case x => throw new Exception(s"Unsupported Scala Version: $x") } val jarDir = getTestResourcePath("regression-test-SPARK-8489") @@ -313,7 +317,7 @@ class HiveSparkSubmitSuite "--master", "local-cluster[2,1,1024]", "--conf", "spark.ui.enabled=false", "--conf", "spark.master.rest.enabled=false", - "--jars", TestHive.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath, + "--jars", HiveTestUtils.getHiveContribJar.getCanonicalPath, unusedJar.toString) runSparkSubmit(argsForCreateTable) @@ -335,11 +339,11 @@ object SetMetastoreURLTest extends Logging { val sparkConf = new SparkConf(loadDefaults = true) val builder = SparkSession.builder() .config(sparkConf) - .config("spark.ui.enabled", "false") - .config("spark.sql.hive.metastore.version", "0.13.1") + .config(UI_ENABLED.key, "false") + .config(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") // The issue described in SPARK-16901 only appear when // spark.sql.hive.metastore.jars is not set to builtin. - .config("spark.sql.hive.metastore.jars", "maven") + .config(HiveUtils.HIVE_METASTORE_JARS.key, "maven") .enableHiveSupport() val spark = builder.getOrCreate() @@ -370,7 +374,7 @@ object SetWarehouseLocationTest extends Logging { def main(args: Array[String]): Unit = { TestUtils.configTestLog4j("INFO") - val sparkConf = new SparkConf(loadDefaults = true).set("spark.ui.enabled", "false") + val sparkConf = new SparkConf(loadDefaults = true).set(UI_ENABLED, false) val providedExpectedWarehouseLocation = sparkConf.getOption("spark.sql.test.expectedWarehouseDir") @@ -390,16 +394,16 @@ object SetWarehouseLocationTest extends Logging { // We are expecting that the value of spark.sql.warehouse.dir will override the // value of hive.metastore.warehouse.dir. val session = new TestHiveContext(new SparkContext(sparkConf - .set("spark.sql.warehouse.dir", warehouseLocation.toString) + .set(WAREHOUSE_PATH.key, warehouseLocation.toString) .set("hive.metastore.warehouse.dir", hiveWarehouseLocation.toString))) .sparkSession (session, warehouseLocation.toString) } - if (sparkSession.conf.get("spark.sql.warehouse.dir") != expectedWarehouseLocation) { + if (sparkSession.conf.get(WAREHOUSE_PATH.key) != expectedWarehouseLocation) { throw new Exception( - "spark.sql.warehouse.dir is not set to the expected warehouse location " + + s"${WAREHOUSE_PATH.key} is not set to the expected warehouse location " + s"$expectedWarehouseLocation.") } @@ -449,13 +453,13 @@ object TemporaryHiveUDFTest extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Load a Hive UDF from the jar. logInfo("Registering a temporary Hive UDF provided in a jar.") - val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + val jar = HiveTestUtils.getHiveContribJar.getCanonicalPath hiveContext.sql( s""" |CREATE TEMPORARY FUNCTION example_max @@ -487,13 +491,13 @@ object PermanentHiveUDFTest1 extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Load a Hive UDF from the jar. logInfo("Registering a permanent Hive UDF provided in a jar.") - val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + val jar = HiveTestUtils.getHiveContribJar.getCanonicalPath hiveContext.sql( s""" |CREATE FUNCTION example_max @@ -525,12 +529,12 @@ object PermanentHiveUDFTest2 extends Logging { def main(args: Array[String]) { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Load a Hive UDF from the jar. logInfo("Write the metadata of a permanent Hive UDF into metastore.") - val jar = hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath + val jar = HiveTestUtils.getHiveContribJar.getCanonicalPath val function = CatalogFunction( FunctionIdentifier("example_max"), "org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax", @@ -561,8 +565,8 @@ object SparkSubmitClassLoaderTest extends Logging { TestUtils.configTestLog4j("INFO") val conf = new SparkConf() val hiveWarehouseLocation = Utils.createTempDir() - conf.set("spark.ui.enabled", "false") - conf.set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString) + conf.set(UI_ENABLED, false) + conf.set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i => (i, i))).toDF("i", "j") @@ -640,21 +644,21 @@ object SparkSQLConfTest extends Logging { val conf = new SparkConf() { override def getAll: Array[(String, String)] = { def isMetastoreSetting(conf: String): Boolean = { - conf == "spark.sql.hive.metastore.version" || conf == "spark.sql.hive.metastore.jars" + conf == HiveUtils.HIVE_METASTORE_VERSION.key || conf == HiveUtils.HIVE_METASTORE_JARS.key } // If there is any metastore settings, remove them. val filteredSettings = super.getAll.filterNot(e => isMetastoreSetting(e._1)) // Always add these two metastore settings at the beginning. - ("spark.sql.hive.metastore.version" -> "0.12") +: - ("spark.sql.hive.metastore.jars" -> "maven") +: + (HiveUtils.HIVE_METASTORE_VERSION.key -> "0.12") +: + (HiveUtils.HIVE_METASTORE_JARS.key -> "maven") +: filteredSettings } // For this simple test, we do not really clone this object. override def clone: SparkConf = this } - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) // Run a simple command to make sure all lazy vals in hiveContext get instantiated. @@ -674,10 +678,10 @@ object SPARK_9757 extends QueryTest { val hiveWarehouseLocation = Utils.createTempDir() val sparkContext = new SparkContext( new SparkConf() - .set("spark.sql.hive.metastore.version", "0.13.1") - .set("spark.sql.hive.metastore.jars", "maven") - .set("spark.ui.enabled", "false") - .set("spark.sql.warehouse.dir", hiveWarehouseLocation.toString)) + .set(HiveUtils.HIVE_METASTORE_VERSION.key, "0.13.1") + .set(HiveUtils.HIVE_METASTORE_JARS.key, "maven") + .set(UI_ENABLED, false) + .set(WAREHOUSE_PATH.key, hiveWarehouseLocation.toString)) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -722,8 +726,8 @@ object SPARK_11009 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() - .set("spark.ui.enabled", "false") - .set("spark.sql.shuffle.partitions", "100")) + .set(UI_ENABLED, false) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -753,8 +757,8 @@ object SPARK_14244 extends QueryTest { val sparkContext = new SparkContext( new SparkConf() - .set("spark.ui.enabled", "false") - .set("spark.sql.shuffle.partitions", "100")) + .set(UI_ENABLED, false) + .set(SHUFFLE_PARTITIONS.key, "100")) val hiveContext = new TestHiveContext(sparkContext) spark = hiveContext.sparkSession @@ -774,7 +778,7 @@ object SPARK_14244 extends QueryTest { object SPARK_18360 { def main(args: Array[String]): Unit = { val spark = SparkSession.builder() - .config("spark.ui.enabled", "false") + .config(UI_ENABLED.key, "false") .enableHiveSupport().getOrCreate() val defaultDbLocation = spark.catalog.getDatabase("default").locationUri diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala index f2b75e4b23f02..daf06645abc24 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveUtilsSuite.scala @@ -17,19 +17,25 @@ package org.apache.spark.sql.hive -import java.net.URL - +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.spark.SparkConf import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.execution.HiveResult import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.{ExamplePoint, ExamplePointUDT, SQLTestUtils} -import org.apache.spark.util.{ChildFirstURLClassLoader, MutableURLClassLoader} +import org.apache.spark.util.ChildFirstURLClassLoader class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + private def testFormatTimeVarsForHiveClient(key: String, value: String, expected: Long): Unit = { + val conf = new Configuration + conf.set(key, value) + assert(HiveUtils.formatTimeVarsForHiveClient(conf)(key) === expected.toString) + } + test("newTemporaryConfiguration overwrites listener configurations") { Seq(true, false).foreach { useInMemoryDerby => val conf = HiveUtils.newTemporaryConfiguration(useInMemoryDerby) @@ -63,9 +69,16 @@ class HiveUtilsSuite extends QueryTest with SQLTestUtils with TestHiveSingleton } } - test("toHiveString correctly handles UDTs") { - val point = new ExamplePoint(50.0, 50.0) - val tpe = new ExamplePointUDT() - assert(HiveUtils.toHiveString((point, tpe)) === "(50.0, 50.0)") + test("SPARK-27349: Dealing with TimeVars removed in Hive 2.x") { + // Test default value + val defaultConf = new Configuration + assert(HiveUtils.formatTimeVarsForHiveClient(defaultConf)("hive.stats.jdbc.timeout") === "30") + assert(HiveUtils.formatTimeVarsForHiveClient(defaultConf)("hive.stats.retries.wait") === "3000") + + testFormatTimeVarsForHiveClient("hive.stats.jdbc.timeout", "40s", 40) + testFormatTimeVarsForHiveClient("hive.stats.jdbc.timeout", "1d", 1 * 24 * 60 * 60) + + testFormatTimeVarsForHiveClient("hive.stats.retries.wait", "4000ms", 4000) + testFormatTimeVarsForHiveClient("hive.stats.retries.wait", "1d", 1 * 24 * 60 * 60 * 1000) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala index 5879748d05b2b..70307ed7e830e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.hive import java.io.File -import org.scalatest.BeforeAndAfter +import org.apache.hadoop.fs.Path +import org.scalatest.{BeforeAndAfter, PrivateMethodTester} import org.apache.spark.SparkException import org.apache.spark.sql.{QueryTest, _} import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.InsertIntoTable +import org.apache.spark.sql.hive.execution.InsertIntoHiveTable import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -36,7 +38,7 @@ case class TestData(key: Int, value: String) case class ThreeCloumntable(key: Int, value: String, key1: String) class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter - with SQLTestUtils { + with SQLTestUtils with PrivateMethodTester { import spark.implicits._ override lazy val testData = spark.sparkContext.parallelize( @@ -550,6 +552,32 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + test("SPARK-27552: hive.exec.stagingdir is invalid on Windows OS") { + val conf = spark.sessionState.newHadoopConf() + val inputPath = new Path("/tmp/b/c") + var stagingDir = "tmp/b" + val saveHiveFile = InsertIntoHiveTable(null, Map.empty, null, false, false, null) + val getStagingDir = PrivateMethod[Path]('getStagingDir) + var path = saveHiveFile invokePrivate getStagingDir(inputPath, conf, stagingDir) + assert(path.toString.indexOf("/tmp/b_hive_") != -1) + + stagingDir = "tmp/b/c" + path = saveHiveFile invokePrivate getStagingDir(inputPath, conf, stagingDir) + assert(path.toString.indexOf("/tmp/b/c/.hive-staging_hive_") != -1) + + stagingDir = "d/e" + path = saveHiveFile invokePrivate getStagingDir(inputPath, conf, stagingDir) + assert(path.toString.indexOf("/tmp/b/c/.hive-staging_hive_") != -1) + + stagingDir = ".d/e" + path = saveHiveFile invokePrivate getStagingDir(inputPath, conf, stagingDir) + assert(path.toString.indexOf("/tmp/b/c/.d/e_hive_") != -1) + + stagingDir = "/tmp/c/" + path = saveHiveFile invokePrivate getStagingDir(inputPath, conf, stagingDir) + assert(path.toString.indexOf("/tmp/c_hive_") != -1) + } + test("insert overwrite to dir from hive metastore table") { withTempDir { dir => val path = dir.toURI.getPath @@ -730,6 +758,18 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + + test("insert overwrite to dir from non-existent table") { + withTempDir { dir => + val path = dir.toURI.getPath + + val e = intercept[AnalysisException] { + sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' TABLE notexists") + }.getMessage + assert(e.contains("Table or view not found")) + } + } + test("SPARK-21165: FileFormatWriter should only rely on attributes from analyzed plan") { withSQLConf(("hive.exec.dynamic.partition.mode", "nonstrict")) { withTable("tab1", "tab2") { @@ -752,6 +792,17 @@ class InsertSuite extends QueryTest with TestHiveSingleton with BeforeAndAfter } } + test("SPARK-26307: CTAS - INSERT a partitioned table using Hive serde") { + withTable("tab1") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + val df = Seq(("a", 100)).toDF("part", "id") + df.write.format("hive").partitionBy("part").mode("overwrite").saveAsTable("tab1") + df.write.format("hive").partitionBy("part").mode("append").saveAsTable("tab1") + } + } + } + + Seq("LOCAL", "").foreach { local => Seq(true, false).foreach { caseSensitivity => Seq("orc", "parquet").foreach { format => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index d93215fefb810..ba807fb58fe40 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -907,30 +907,28 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv createDF(10, 19).write.mode(SaveMode.Append).format("orc").saveAsTable("appendOrcToParquet") } assert(e.getMessage.contains( - "The format of the existing table default.appendOrcToParquet is `ParquetFileFormat`. " + - "It doesn't match the specified format `OrcFileFormat`")) + "The format of the existing table default.appendOrcToParquet is `Parquet")) } withTable("appendParquetToJson") { createDF(0, 9).write.format("json").saveAsTable("appendParquetToJson") - val e = intercept[AnalysisException] { + val msg = intercept[AnalysisException] { createDF(10, 19).write.mode(SaveMode.Append).format("parquet") .saveAsTable("appendParquetToJson") - } - assert(e.getMessage.contains( - "The format of the existing table default.appendParquetToJson is `JsonFileFormat`. " + - "It doesn't match the specified format `ParquetFileFormat`")) + }.getMessage + + assert(msg.contains( + "The format of the existing table default.appendParquetToJson is `Json")) } withTable("appendTextToJson") { createDF(0, 9).write.format("json").saveAsTable("appendTextToJson") - val e = intercept[AnalysisException] { + val msg = intercept[AnalysisException] { createDF(10, 19).write.mode(SaveMode.Append).format("text") .saveAsTable("appendTextToJson") - } - assert(e.getMessage.contains( - "The format of the existing table default.appendTextToJson is `JsonFileFormat`. " + - "It doesn't match the specified format `TextFileFormat`")) + }.getMessage + // The format of the existing table can be JsonDataSourceV2 or JsonFileFormat. + assert(msg.contains("The format of the existing table default.appendTextToJson is `Json")) } } @@ -968,6 +966,74 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv } } + test("append a table with file source V2 provider using the v1 file format") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendCSV") { + createDF(0, 9) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2") + .saveAsTable("appendCSV") + createDF(10, 19) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat") + .saveAsTable("appendCSV") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendCSV p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendCSV") { + createDF(0, 9).write.mode(SaveMode.Append).format("csv").saveAsTable("appendCSV") + createDF(10, 19) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat") + .saveAsTable("appendCSV") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendCSV p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + } + + test("append a table with v1 file format provider using file source V2 format") { + def createDF(from: Int, to: Int): DataFrame = { + (from to to).map(i => i -> s"str$i").toDF("c1", "c2") + } + + withTable("appendCSV") { + createDF(0, 9) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat") + .saveAsTable("appendCSV") + createDF(10, 19) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.v2.csv.CSVDataSourceV2") + .saveAsTable("appendCSV") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendCSV p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + + withTable("appendCSV") { + createDF(0, 9) + .write + .mode(SaveMode.Append) + .format("org.apache.spark.sql.execution.datasources.csv.CSVFileFormat") + .saveAsTable("appendCSV") + createDF(10, 19).write.mode(SaveMode.Append).format("csv").saveAsTable("appendCSV") + checkAnswer( + sql("SELECT p.c1, p.c2 FROM appendCSV p WHERE p.c1 > 5"), + (6 to 19).map(i => Row(i, s"str$i"))) + } + } + test("SPARK-8156:create table to specific database by 'use dbname' ") { val df = (1 to 3).map(i => (i, s"val_$i", i * 2)).toDF("a", "b", "c") 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 db2024e8b5d16..f8f95829a4c41 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 @@ -40,6 +40,7 @@ import org.apache.spark.sql.hive.HiveExternalCatalog._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.types._ +import org.apache.spark.util.Utils class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleton { @@ -77,14 +78,14 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto withTempDir { tempDir => // EXTERNAL OpenCSVSerde table pointing to LOCATION val file1 = new File(tempDir + "/data1") - val writer1 = new PrintWriter(file1) - writer1.write("1,2") - writer1.close() + Utils.tryWithResource(new PrintWriter(file1)) { writer => + writer.write("1,2") + } val file2 = new File(tempDir + "/data2") - val writer2 = new PrintWriter(file2) - writer2.write("1,2") - writer2.close() + Utils.tryWithResource(new PrintWriter(file2)) { writer => + writer.write("1,2") + } sql( s""" @@ -100,8 +101,14 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto .asInstanceOf[HiveTableRelation] val properties = relation.tableMeta.ignoredProperties - assert(properties("totalSize").toLong <= 0, "external table totalSize must be <= 0") - assert(properties("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") + if (HiveUtils.isHive23) { + // Since HIVE-6727, Hive fixes table-level stats for external tables are incorrect. + assert(properties("totalSize").toLong == 6) + assert(properties.get("rawDataSize").isEmpty) + } else { + assert(properties("totalSize").toLong <= 0, "external table totalSize must be <= 0") + assert(properties("rawDataSize").toLong <= 0, "external table rawDataSize must be <= 0") + } val sizeInBytes = relation.stats.sizeInBytes assert(sizeInBytes === BigInt(file1.length() + file2.length())) @@ -110,6 +117,41 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto } } + test("Hive serde table with incorrect statistics") { + withTempDir { tempDir => + withTable("t1") { + spark.range(5).write.mode(SaveMode.Overwrite).parquet(tempDir.getCanonicalPath) + val dataSize = getLocalDirSize(tempDir) + spark.sql( + s""" + |CREATE EXTERNAL TABLE t1(id BIGINT) + |ROW FORMAT SERDE 'org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe' + |STORED AS + | INPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat' + | OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat' + |LOCATION '${tempDir.getCanonicalPath}' + |TBLPROPERTIES ( + |'rawDataSize'='-1', 'numFiles'='0', 'totalSize'='0', + |'COLUMN_STATS_ACCURATE'='false', 'numRows'='-1' + |)""".stripMargin) + + spark.sql("REFRESH TABLE t1") + // Before SPARK-19678, sizeInBytes should be equal to dataSize. + // After SPARK-19678, sizeInBytes should be equal to DEFAULT_SIZE_IN_BYTES. + val relation1 = spark.table("t1").queryExecution.analyzed.children.head + assert(relation1.stats.sizeInBytes === spark.sessionState.conf.defaultSizeInBytes) + + spark.sql("REFRESH TABLE t1") + // After SPARK-19678 and enable ENABLE_FALL_BACK_TO_HDFS_FOR_STATS, + // sizeInBytes should be equal to dataSize. + withSQLConf(SQLConf.ENABLE_FALL_BACK_TO_HDFS_FOR_STATS.key -> "true") { + val relation2 = spark.table("t1").queryExecution.analyzed.children.head + assert(relation2.stats.sizeInBytes === dataSize) + } + } + } + } + test("analyze Hive serde tables") { def queryTotalSize(tableName: String): BigInt = spark.table(tableName).queryExecution.analyzed.stats.sizeInBytes @@ -830,17 +872,25 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto val totalSize = extractStatsPropValues(describeResult, "totalSize") assert(totalSize.isDefined && totalSize.get > 0, "totalSize is lost") - // ALTER TABLE SET/UNSET TBLPROPERTIES invalidates some Hive specific statistics, but not - // Spark specific statistics. This is triggered by the Hive alterTable API. val numRows = extractStatsPropValues(describeResult, "numRows") - assert(numRows.isDefined && numRows.get == -1, "numRows is lost") - val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") - assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") - - if (analyzedBySpark) { + if (HiveUtils.isHive23) { + // Since HIVE-15653(Hive 2.3.0), Hive fixs some ALTER TABLE commands drop table stats. + assert(numRows.isDefined && numRows.get == 500) + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + assert(rawDataSize.isDefined && rawDataSize.get == 5312) checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) } else { - checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) + // ALTER TABLE SET/UNSET TBLPROPERTIES invalidates some Hive specific statistics, but not + // Spark specific statistics. This is triggered by the Hive alterTable API. + assert(numRows.isDefined && numRows.get == -1, "numRows is lost") + val rawDataSize = extractStatsPropValues(describeResult, "rawDataSize") + assert(rawDataSize.isDefined && rawDataSize.get == -1, "rawDataSize is lost") + + if (analyzedBySpark) { + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = Some(500)) + } else { + checkTableStats(tabName, hasSizeInBytes = true, expectedRowCounts = None) + } } } } @@ -908,9 +958,9 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto withTempDir { loadPath => // load data command val file = new File(loadPath + "/data") - val writer = new PrintWriter(file) - writer.write("2,xyz") - writer.close() + Utils.tryWithResource(new PrintWriter(file)) { writer => + writer.write("2,xyz") + } sql(s"LOAD DATA INPATH '${loadPath.toURI.toString}' INTO TABLE $table") if (autoUpdate) { val fetched2 = checkTableStats(table, hasSizeInBytes = true, expectedRowCounts = None) @@ -945,14 +995,14 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto withTempPaths(numPaths = 2) { case Seq(dir1, dir2) => val file1 = new File(dir1 + "/data") - val writer1 = new PrintWriter(file1) - writer1.write("1,a") - writer1.close() + Utils.tryWithResource(new PrintWriter(file1)) { writer => + writer.write("1,a") + } val file2 = new File(dir2 + "/data") - val writer2 = new PrintWriter(file2) - writer2.write("1,a") - writer2.close() + Utils.tryWithResource(new PrintWriter(file2)) { writer => + writer.write("1,a") + } // add partition command sql( @@ -1381,4 +1431,26 @@ class StatisticsSuite extends StatisticsCollectionTestBase with TestHiveSingleto assert(catalogStats.rowCount.isEmpty) } } + + test(s"CTAS should update statistics if ${SQLConf.AUTO_SIZE_UPDATE_ENABLED.key} is enabled") { + val tableName = "SPARK_23263" + Seq(false, true).foreach { isConverted => + Seq(false, true).foreach { updateEnabled => + withSQLConf( + SQLConf.AUTO_SIZE_UPDATE_ENABLED.key -> updateEnabled.toString, + HiveUtils.CONVERT_METASTORE_PARQUET.key -> isConverted.toString) { + withTable(tableName) { + sql(s"CREATE TABLE $tableName STORED AS parquet AS SELECT 'a', 'b'") + val catalogTable = getCatalogTable(tableName) + // Hive serde tables always update statistics by Hive metastore + if (!isConverted || updateEnabled) { + assert(catalogTable.stats.nonEmpty) + } else { + assert(catalogTable.stats.isEmpty) + } + } + } + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala index 7a325bf26b4cf..bda711200acdb 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientSuite.scala @@ -19,12 +19,17 @@ package org.apache.spark.sql.hive.client import org.apache.hadoop.conf.Configuration import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.mapred.TextInputFormat import org.scalatest.BeforeAndAfterAll +import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.catalyst.dsl.expressions._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType} +import org.apache.spark.sql.types.{BooleanType, IntegerType, LongType, StructType} +import org.apache.spark.util.Utils // TODO: Refactor this to `HivePartitionFilteringSuite` class HiveClientSuite(version: String) @@ -45,8 +50,24 @@ class HiveClientSuite(version: String) val hadoopConf = new Configuration() hadoopConf.setBoolean(tryDirectSqlKey, tryDirectSql) + hadoopConf.set("hive.metastore.warehouse.dir", Utils.createTempDir().toURI().toString()) val client = buildClient(hadoopConf) - client.runSqlHive("CREATE TABLE test (value INT) PARTITIONED BY (ds INT, h INT, chunk STRING)") + val tableSchema = + new StructType().add("value", "int").add("ds", "int").add("h", "int").add("chunk", "string") + val table = CatalogTable( + identifier = TableIdentifier("test", Some("default")), + tableType = CatalogTableType.MANAGED, + schema = tableSchema, + partitionColumnNames = Seq("ds", "h", "chunk"), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[TextInputFormat].getName), + outputFormat = Some(classOf[HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[LazySimpleSerDe].getName()), + compressed = false, + properties = Map.empty + )) + client.createTable(table, ignoreIfExists = false) val partitions = for { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala index 30592a3f85428..e9eebb4d154fa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveClientVersions.scala @@ -23,5 +23,6 @@ import org.apache.spark.SparkFunSuite private[client] trait HiveClientVersions { protected val versions = - IndexedSeq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3") + IndexedSeq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3", "3.0", + "3.1") } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala index e5963d03f6b52..dd58c302e0197 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/client/HiveVersionSuite.scala @@ -34,10 +34,15 @@ private[client] abstract class HiveVersionSuite(version: String) extends SparkFu // Hive changed the default of datanucleus.schema.autoCreateAll from true to false and // hive.metastore.schema.verification from false to true since 2.0 // For details, see the JIRA HIVE-6113 and HIVE-12463 - if (version == "2.0" || version == "2.1" || version == "2.2" || version == "2.3") { + if (version == "2.0" || version == "2.1" || version == "2.2" || version == "2.3" || + version == "3.0" || version == "3.1") { hadoopConf.set("datanucleus.schema.autoCreateAll", "true") hadoopConf.set("hive.metastore.schema.verification", "false") } + // Since Hive 3.0, HIVE-19310 skipped `ensureDbInit` if `hive.in.test=false`. + if (version == "3.0" || version == "3.1") { + hadoopConf.set("hive.in.test", "true") + } HiveClientBuilder.buildClient( version, hadoopConf, 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 dc96ec416afd8..feb364ec1947a 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 @@ -55,15 +55,6 @@ class VersionsSuite extends SparkFunSuite with Logging { import HiveClientBuilder.buildClient - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally Utils.deleteRecursively(dir) - } - /** * Drops table `tableName` after calling `f`. */ @@ -112,7 +103,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } private val versions = - Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3") + Seq("0.12", "0.13", "0.14", "1.0", "1.1", "1.2", "2.0", "2.1", "2.2", "2.3", "3.0", "3.1") private var client: HiveClient = null @@ -127,10 +118,17 @@ class VersionsSuite extends SparkFunSuite with Logging { // Hive changed the default of datanucleus.schema.autoCreateAll from true to false and // hive.metastore.schema.verification from false to true since 2.0 // For details, see the JIRA HIVE-6113 and HIVE-12463 - if (version == "2.0" || version == "2.1" || version == "2.2" || version == "2.3") { + if (version == "2.0" || version == "2.1" || version == "2.2" || version == "2.3" || + version == "3.0" || version == "3.1") { hadoopConf.set("datanucleus.schema.autoCreateAll", "true") hadoopConf.set("hive.metastore.schema.verification", "false") } + if (version == "3.0" || version == "3.1") { + // Since Hive 3.0, HIVE-19310 skipped `ensureDbInit` if `hive.in.test=false`. + hadoopConf.set("hive.in.test", "true") + // Since HIVE-17626(Hive 3.0.0), need to set hive.query.reexecution.enabled=false. + hadoopConf.set("hive.query.reexecution.enabled", "false") + } client = buildClient(version, hadoopConf, HiveUtils.formatTimeVarsForHiveClient(hadoopConf)) if (versionSpark != null) versionSpark.reset() versionSpark = TestHiveVersion(client) @@ -187,7 +185,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: databaseExists") { - assert(client.databaseExists("default") == true) + assert(client.databaseExists("default")) assert(client.databaseExists("nonexist") == false) } @@ -202,7 +200,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: dropDatabase") { - assert(client.databaseExists("temporary") == true) + assert(client.databaseExists("temporary")) client.dropDatabase("temporary", ignoreIfNotExists = false, cascade = true) assert(client.databaseExists("temporary") == false) } @@ -239,6 +237,33 @@ class VersionsSuite extends SparkFunSuite with Logging { assert(client.getTableOption("default", "src").isDefined) } + test(s"$version: getTablesByName") { + assert(client.getTablesByName("default", Seq("src")).head + == client.getTableOption("default", "src").get) + } + + test(s"$version: getTablesByName when multiple tables") { + assert(client.getTablesByName("default", Seq("src", "temporary")) + .map(_.identifier.table) == Seq("src", "temporary")) + } + + test(s"$version: getTablesByName when some tables do not exist") { + assert(client.getTablesByName("default", Seq("src", "notexist")) + .map(_.identifier.table) == Seq("src")) + } + + test(s"$version: getTablesByName when contains invalid name") { + // scalastyle:off + val name = "砖" + // scalastyle:on + assert(client.getTablesByName("default", Seq("src", name)) + .map(_.identifier.table) == Seq("src")) + } + + test(s"$version: getTablesByName when empty") { + assert(client.getTablesByName("default", Seq.empty).isEmpty) + } + test(s"$version: alterTable(table: CatalogTable)") { val newTable = client.getTable("default", "src").copy(properties = Map("changed" -> "")) client.alterTable(newTable) @@ -327,7 +352,20 @@ class VersionsSuite extends SparkFunSuite with Logging { properties = Map.empty) test(s"$version: sql create partitioned table") { - client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY (key1 INT, key2 INT)") + val table = CatalogTable( + identifier = TableIdentifier("src_part", Some("default")), + tableType = CatalogTableType.MANAGED, + schema = new StructType().add("value", "int").add("key1", "int").add("key2", "int"), + partitionColumnNames = Seq("key1", "key2"), + storage = CatalogStorageFormat( + locationUri = None, + inputFormat = Some(classOf[TextInputFormat].getName), + outputFormat = Some(classOf[HiveIgnoreKeyTextOutputFormat[_, _]].getName), + serde = Some(classOf[LazySimpleSerDe].getName()), + compressed = false, + properties = Map.empty + )) + client.createTable(table, ignoreIfExists = false) } val testPartitionCount = 2 @@ -483,7 +521,7 @@ class VersionsSuite extends SparkFunSuite with Logging { // Hive 0.12 doesn't allow customized permanent functions assert(client.functionExists("default", "func1") == false) } else { - assert(client.functionExists("default", "func1") == true) + assert(client.functionExists("default", "func1")) } } @@ -495,7 +533,7 @@ class VersionsSuite extends SparkFunSuite with Logging { } } else { client.renameFunction("default", "func1", "func2") - assert(client.functionExists("default", "func2") == true) + assert(client.functionExists("default", "func2")) } } @@ -565,9 +603,25 @@ class VersionsSuite extends SparkFunSuite with Logging { } test(s"$version: sql create index and reset") { - client.runSqlHive("CREATE TABLE indexed_table (key INT)") - client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) " + - "as 'COMPACT' WITH DEFERRED REBUILD") + // HIVE-18448 Since Hive 3.0, INDEX is not supported. + if (version != "3.0" && version != "3.1") { + client.runSqlHive("CREATE TABLE indexed_table (key INT)") + client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) " + + "as 'COMPACT' WITH DEFERRED REBUILD") + } + } + + test(s"$version: sql read hive materialized view") { + // HIVE-14249 Since Hive 2.3.0, materialized view is supported. + if (version == "2.3" || version == "3.0" || version == "3.1") { + // Since HIVE-18394(Hive 3.1), "Create Materialized View" should default to rewritable ones + val disableRewrite = if (version == "2.3" || version == "3.0") "" else "DISABLE REWRITE" + client.runSqlHive("CREATE TABLE materialized_view_tbl (c1 INT)") + client.runSqlHive( + s"CREATE MATERIALIZED VIEW mv1 $disableRewrite AS SELECT * FROM materialized_view_tbl") + val e = intercept[AnalysisException](versionSpark.table("mv1").collect()).getMessage + assert(e.contains("Hive materialized view is not supported")) + } } /////////////////////////////////////////////////////////////////////////// @@ -921,9 +975,9 @@ class VersionsSuite extends SparkFunSuite with Logging { |} """.stripMargin val schemaFile = new File(dir, "avroDecimal.avsc") - val writer = new PrintWriter(schemaFile) - writer.write(avroSchema) - writer.close() + Utils.tryWithResource(new PrintWriter(schemaFile)) { writer => + writer.write(avroSchema) + } val schemaPath = schemaFile.toURI.toString val url = Thread.currentThread().getContextClassLoader.getResource("avroDecimal") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala index c65bf7c14c7a5..d06cc1c0a88ac 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/AggregationQuerySuite.scala @@ -594,7 +594,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te | max(distinct value1) |FROM agg2 """.stripMargin), - Row(-60, 70.0, 101.0/9.0, 5.6, 100)) + Row(-60, 70, 101.0/9.0, 5.6, 100)) checkAnswer( spark.sql( @@ -884,7 +884,7 @@ abstract class AggregationQuerySuite extends QueryTest with SQLTestUtils with Te FloatType, DoubleType, DecimalType(25, 5), DecimalType(6, 5), DateType, TimestampType, ArrayType(IntegerType), MapType(StringType, LongType), struct, - new UDT.MyDenseVectorUDT()) + new TestUDT.MyDenseVectorUDT()) // Right now, we will use SortAggregate to handle UDAFs. // UnsafeRow.mutableFieldTypes.asScala.toSeq will trigger SortAggregate to use // UnsafeRow as the aggregation buffer. While, dataTypes will trigger @@ -1028,7 +1028,7 @@ class HashAggregationQueryWithControlledFallbackSuite extends AggregationQuerySu override protected def checkAnswer(actual: => DataFrame, expectedAnswer: Seq[Row]): Unit = { Seq("true", "false").foreach { enableTwoLevelMaps => - withSQLConf("spark.sql.codegen.aggregate.map.twolevel.enabled" -> + withSQLConf(SQLConf.ENABLE_TWOLEVEL_AGG_MAP.key -> enableTwoLevelMaps) { (1 to 3).foreach { fallbackStartsAt => withSQLConf("spark.sql.TungstenAggregate.testFallbackStartsAt" -> diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala index 07d8c5bacb1a2..76134d23d18aa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.execution import org.scalatest.BeforeAndAfterAll import org.apache.spark.{SparkConf, SparkContext, SparkFunSuite} +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.sql.hive.test.TestHiveContext class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { @@ -27,7 +28,7 @@ class ConcurrentHiveSuite extends SparkFunSuite with BeforeAndAfterAll { test("Multiple Hive Instances") { (1 to 10).map { i => val conf = new SparkConf() - conf.set("spark.ui.enabled", "false") + conf.set(UI_ENABLED, false) val ts = new TestHiveContext(new SparkContext("local", s"TestSQLContext$i", conf)) ts.sparkSession.sql("SHOW TABLES").collect() 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 6937e97a47dc6..9147a98c94457 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 @@ -20,11 +20,13 @@ package org.apache.spark.sql.hive.execution import java.io.File import com.google.common.io.Files +import org.apache.hadoop.fs.{FileContext, FsConstants, Path} import org.apache.spark.sql.{AnalysisException, QueryTest, Row, SaveMode} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.NoSuchTableException import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTable, CatalogTableType} +import org.apache.spark.sql.execution.command.LoadDataCommand import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -439,4 +441,11 @@ class HiveCommandSuite extends QueryTest with SQLTestUtils with TestHiveSingleto } } + test("SPARK-25918: LOAD DATA LOCAL INPATH should handle a relative path") { + val localFS = FileContext.getLocalFSFileContext() + val workingDir = localFS.getWorkingDirectory + val r = LoadDataCommand.makeQualified( + FsConstants.LOCAL_FS_URI, workingDir, new Path("kv1.txt")) + assert(r === new Path(s"$workingDir/kv1.txt")) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala index 272e6f51f5002..a4587abbf389d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -31,6 +31,7 @@ import org.apache.spark.sql.Dataset import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.execution.HiveResult.hiveResultString import org.apache.spark.sql.execution.SQLExecution import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveQueryExecution} @@ -171,7 +172,7 @@ abstract class HiveComparisonTest // and does not return it as a query answer. case _: SetCommand => Seq("0") case _: ExplainCommand => answer - case _: DescribeTableCommand | ShowColumnsCommand(_, _) => + case _: DescribeCommandBase | ShowColumnsCommand(_, _) => // Filter out non-deterministic lines and lines which do not have actual results but // can introduce problems because of the way Hive formats these lines. // Then, remove empty lines. Do not sort the results. @@ -345,7 +346,8 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHiveQueryExecution(queryString.replace("../../data", testDataPath)) def getResult(): Seq[String] = { - SQLExecution.withNewExecutionId(query.sparkSession, query)(query.hiveResultString()) + SQLExecution.withNewExecutionId( + query.sparkSession, query)(hiveResultString(query.executedPlan)) } try { (query, prepareAnswer(query, getResult())) } catch { case e: Throwable => @@ -373,7 +375,7 @@ abstract class HiveComparisonTest if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) && (!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) && (!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) && - (!hiveQuery.logical.isInstanceOf[DescribeTableCommand]) && + (!hiveQuery.logical.isInstanceOf[DescribeCommandBase]) && preparedHive != catalyst) { val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala index 6f13143c32cef..90a491d238a8f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala @@ -19,9 +19,6 @@ package org.apache.spark.sql.hive.execution import java.io.File import java.net.URI -import java.util.Date - -import scala.language.existentials import org.apache.hadoop.fs.Path import org.apache.parquet.format.converter.ParquetMetadataConverter.NO_FILTER @@ -261,6 +258,108 @@ class HiveCatalogedDDLSuite extends DDLSuite with TestHiveSingleton with BeforeA assert(err.contains("Cannot recognize hive type string:")) } } + + test("SPARK-26630: table with old input format and without partitioned will use HadoopRDD") { + withTable("table_old", "table_ctas_old") { + sql( + """ + |CREATE TABLE table_old (col1 LONG, col2 STRING, col3 DOUBLE, col4 BOOLEAN) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + """.stripMargin) + sql( + """ + |INSERT INTO table_old + |VALUES (2147483648, 'AAA', 3.14, false), (2147483649, 'BBB', 3.142, true) + """.stripMargin) + checkAnswer( + sql("SELECT col1, col2, col3, col4 FROM table_old"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + + sql("CREATE TABLE table_ctas_old AS SELECT col1, col2, col3, col4 FROM table_old") + checkAnswer( + sql("SELECT col1, col2, col3, col4 from table_ctas_old"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + } + } + + test("SPARK-26630: table with old input format and partitioned will use HadoopRDD") { + withTable("table_pt_old", "table_ctas_pt_old") { + sql( + """ + |CREATE TABLE table_pt_old (col1 LONG, col2 STRING, col3 DOUBLE, col4 BOOLEAN) + |PARTITIONED BY (pt INT) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.mapred.TextInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + """.stripMargin) + sql( + """ + |INSERT INTO table_pt_old PARTITION (pt = 1) + |VALUES (2147483648, 'AAA', 3.14, false), (2147483649, 'BBB', 3.142, true) + """.stripMargin) + checkAnswer( + sql("SELECT col1, col2, col3, col4 FROM table_pt_old WHERE pt = 1"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + + sql("CREATE TABLE table_ctas_pt_old AS SELECT col1, col2, col3, col4 FROM table_pt_old") + checkAnswer( + sql("SELECT col1, col2, col3, col4 from table_ctas_pt_old"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + } + } + + test("SPARK-26630: table with new input format and without partitioned will use NewHadoopRDD") { + withTable("table_new", "table_ctas_new") { + sql( + """ + |CREATE TABLE table_new (col1 LONG, col2 STRING, col3 DOUBLE, col4 BOOLEAN) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.mapreduce.lib.input.TextInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + """.stripMargin) + sql( + """ + |INSERT INTO table_new + |VALUES (2147483648, 'AAA', 3.14, false), (2147483649, 'BBB', 3.142, true) + """.stripMargin) + checkAnswer( + sql("SELECT col1, col2, col3, col4 FROM table_new"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + + sql("CREATE TABLE table_ctas_new AS SELECT col1, col2, col3, col4 FROM table_new") + checkAnswer( + sql("SELECT col1, col2, col3, col4 from table_ctas_new"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + } + } + + test("SPARK-26630: table with new input format and partitioned will use NewHadoopRDD") { + withTable("table_pt_new", "table_ctas_pt_new") { + sql( + """ + |CREATE TABLE table_pt_new (col1 LONG, col2 STRING, col3 DOUBLE, col4 BOOLEAN) + |PARTITIONED BY (pt INT) + |STORED AS + |INPUTFORMAT 'org.apache.hadoop.mapreduce.lib.input.TextInputFormat' + |OUTPUTFORMAT 'org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat' + """.stripMargin) + sql( + """ + |INSERT INTO table_pt_new PARTITION (pt = 1) + |VALUES (2147483648, 'AAA', 3.14, false), (2147483649, 'BBB', 3.142, true) + """.stripMargin) + checkAnswer( + sql("SELECT col1, col2, col3, col4 FROM table_pt_new WHERE pt = 1"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + + sql("CREATE TABLE table_ctas_pt_new AS SELECT col1, col2, col3, col4 FROM table_pt_new") + checkAnswer( + sql("SELECT col1, col2, col3, col4 from table_ctas_pt_new"), + Row(2147483648L, "AAA", 3.14, false) :: Row(2147483649L, "BBB", 3.142, true) :: Nil) + } + } } class HiveDDLSuite @@ -2515,4 +2614,50 @@ class HiveDDLSuite )) } } + + test("Hive CTAS can't create partitioned table by specifying schema") { + val err1 = intercept[ParseException] { + spark.sql( + s""" + |CREATE TABLE t (a int) + |PARTITIONED BY (b string) + |STORED AS parquet + |AS SELECT 1 as a, "a" as b + """.stripMargin) + }.getMessage + assert(err1.contains("Schema may not be specified in a Create Table As Select " + + "(CTAS) statement")) + + val err2 = intercept[ParseException] { + spark.sql( + s""" + |CREATE TABLE t + |PARTITIONED BY (b string) + |STORED AS parquet + |AS SELECT 1 as a, "a" as b + """.stripMargin) + }.getMessage + assert(err2.contains("Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table")) + } + + test("Hive CTAS with dynamic partition") { + Seq("orc", "parquet").foreach { format => + withTable("t") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + spark.sql( + s""" + |CREATE TABLE t + |PARTITIONED BY (b) + |STORED AS $format + |AS SELECT 1 as a, "a" as b + """.stripMargin) + checkAnswer(spark.table("t"), Row(1, "a")) + + assert(spark.sessionState.catalog.getTableMetadata(TableIdentifier("t")) + .partitionColumnNames === Seq("b")) + } + } + } + } } 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 c349a327694bf..d413dfb2b2dc5 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 @@ -20,9 +20,13 @@ package org.apache.spark.sql.hive.execution 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.datasources.InsertIntoHadoopFsRelationCommand +import org.apache.spark.sql.hive.HiveUtils +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils +import org.apache.spark.util.Utils /** * A set of tests that validates support for Hive Explain command. @@ -182,4 +186,25 @@ class HiveExplainSuite extends QueryTest with SQLTestUtils with TestHiveSingleto assert(output.toString.contains(s"Scan hive default.$tableName")) } } + + test("SPARK-26661: Show actual class name of the writing command in CTAS explain") { + Seq(true, false).foreach { convertCTAS => + withSQLConf( + HiveUtils.CONVERT_METASTORE_CTAS.key -> convertCTAS.toString, + HiveUtils.CONVERT_METASTORE_PARQUET.key -> convertCTAS.toString) { + + val df = sql(s"EXPLAIN CREATE TABLE tab1 STORED AS PARQUET AS SELECT * FROM range(2)") + val keywords = if (convertCTAS) { + Seq( + s"Execute ${Utils.getSimpleName(classOf[OptimizedCreateHiveTableAsSelectCommand])}", + Utils.getSimpleName(classOf[InsertIntoHadoopFsRelationCommand])) + } else { + Seq( + s"Execute ${Utils.getSimpleName(classOf[CreateHiveTableAsSelectCommand])}", + Utils.getSimpleName(classOf[InsertIntoHiveTable])) + } + checkKeywordsExist(df, keywords: _*) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a5cff35abf37e..6986963ecc089 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.sql.catalyst.parser.ParseException import org.apache.spark.sql.catalyst.plans.logical.Project import org.apache.spark.sql.execution.joins.BroadcastNestedLoopJoinExec import org.apache.spark.sql.hive._ -import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHive} import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils @@ -816,7 +816,7 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd test("ADD JAR command 2") { // this is a test case from mapjoin_addjar.q - val testJar = TestHive.getHiveFile("hive-hcatalog-core-0.13.1.jar").toURI + val testJar = HiveTestUtils.getHiveHcatalogCoreJar.toURI val testData = TestHive.getHiveFile("data/files/sample.json").toURI sql(s"ADD JAR $testJar") sql( @@ -826,9 +826,9 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd sql("select * from src join t1 on src.key = t1.a") sql("DROP TABLE t1") assert(sql("list jars"). - filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + filter(_.getString(0).contains(HiveTestUtils.getHiveHcatalogCoreJar.getName)).count() > 0) assert(sql("list jar"). - filter(_.getString(0).contains("hive-hcatalog-core-0.13.1.jar")).count() > 0) + filter(_.getString(0).contains(HiveTestUtils.getHiveHcatalogCoreJar.getName)).count() > 0) val testJar2 = TestHive.getHiveFile("TestUDTF.jar").getCanonicalPath sql(s"ADD JAR $testJar2") assert(sql(s"list jar $testJar").count() == 1) @@ -1188,6 +1188,25 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd } } } + + test("SPARK-28054: Unable to insert partitioned table when partition name is upper case") { + withSQLConf("hive.exec.dynamic.partition.mode" -> "nonstrict") { + withTable("spark_28054_test") { + sql("CREATE TABLE spark_28054_test (KEY STRING, VALUE STRING) PARTITIONED BY (DS STRING)") + + sql("INSERT INTO TABLE spark_28054_test PARTITION(DS) SELECT 'k' KEY, 'v' VALUE, '1' DS") + + assertResult(Array(Row("k", "v", "1"))) { + sql("SELECT * from spark_28054_test").collect() + } + + sql("INSERT INTO TABLE spark_28054_test PARTITION(ds) SELECT 'k' key, 'v' value, '2' ds") + assertResult(Array(Row("k", "v", "1"), Row("k", "v", "2"))) { + sql("SELECT * from spark_28054_test").collect() + } + } + } + } } // for SPARK-2180 test diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala new file mode 100644 index 0000000000000..25ff3544185af --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeReadWriteSuite.scala @@ -0,0 +1,185 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.sql.{Date, Timestamp} + +import org.apache.spark.sql.{QueryTest, Row} +import org.apache.spark.sql.hive.HiveUtils.{CONVERT_METASTORE_ORC, CONVERT_METASTORE_PARQUET} +import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf.ORC_IMPLEMENTATION +import org.apache.spark.sql.test.SQLTestUtils + +class HiveSerDeReadWriteSuite extends QueryTest with SQLTestUtils with TestHiveSingleton { + + private var originalConvertMetastoreParquet = CONVERT_METASTORE_PARQUET.defaultValueString + private var originalConvertMetastoreORC = CONVERT_METASTORE_ORC.defaultValueString + private var originalORCImplementation = ORC_IMPLEMENTATION.defaultValueString + + protected override def beforeAll(): Unit = { + super.beforeAll() + originalConvertMetastoreParquet = spark.conf.get(CONVERT_METASTORE_PARQUET.key) + originalConvertMetastoreORC = spark.conf.get(CONVERT_METASTORE_ORC.key) + originalORCImplementation = spark.conf.get(ORC_IMPLEMENTATION) + + spark.conf.set(CONVERT_METASTORE_PARQUET.key, "false") + spark.conf.set(CONVERT_METASTORE_ORC.key, "false") + spark.conf.set(ORC_IMPLEMENTATION.key, "hive") + } + + protected override def afterAll(): Unit = { + spark.conf.set(CONVERT_METASTORE_PARQUET.key, originalConvertMetastoreParquet) + spark.conf.set(CONVERT_METASTORE_ORC.key, originalConvertMetastoreORC) + spark.conf.set(ORC_IMPLEMENTATION.key, originalORCImplementation) + super.afterAll() + } + + private def checkNumericTypes(fileFormat: String, dataType: String, value: Any): Unit = { + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 $dataType) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values(1)") + checkAnswer(spark.table("hive_serde"), Row(1)) + spark.sql(s"INSERT INTO TABLE hive_serde values($value)") + checkAnswer(spark.table("hive_serde"), Seq(Row(1), Row(value))) + } + } + + private def checkDateTimeTypes(fileFormat: String): Unit = { + // TIMESTAMP + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 TIMESTAMP) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('2019-04-11 15:50:00')") + checkAnswer(spark.table("hive_serde"), Row(Timestamp.valueOf("2019-04-11 15:50:00"))) + spark.sql("INSERT INTO TABLE hive_serde values('2019-04-12 15:50:00')") + checkAnswer( + spark.table("hive_serde"), + Seq(Row(Timestamp.valueOf("2019-04-11 15:50:00")), + Row(Timestamp.valueOf("2019-04-12 15:50:00")))) + } + + // DATE + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 DATE) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('2019-04-11')") + checkAnswer(spark.table("hive_serde"), Row(Date.valueOf("2019-04-11"))) + spark.sql("INSERT INTO TABLE hive_serde values('2019-04-12')") + checkAnswer( + spark.table("hive_serde"), + Seq(Row(Date.valueOf("2019-04-11")), Row(Date.valueOf("2019-04-12")))) + } + } + + private def checkStringTypes(fileFormat: String, dataType: String, value: String): Unit = { + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 $dataType) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('s')") + checkAnswer(spark.table("hive_serde"), Row("s")) + spark.sql(s"INSERT INTO TABLE hive_serde values('$value')") + checkAnswer(spark.table("hive_serde"), Seq(Row("s"), Row(value))) + } + } + + private def checkCharTypes(fileFormat: String): Unit = { + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 CHAR(10)) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('s')") + checkAnswer(spark.table("hive_serde"), Row("s" + " " * 9)) + spark.sql(s"INSERT INTO TABLE hive_serde values('s3')") + checkAnswer(spark.table("hive_serde"), Seq(Row("s" + " " * 9), Row("s3" + " " * 8))) + } + } + + private def checkMiscTypes(fileFormat: String): Unit = { + // BOOLEAN + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 BOOLEAN) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values(false)") + checkAnswer(spark.table("hive_serde"), Row(false)) + spark.sql("INSERT INTO TABLE hive_serde values(true)") + checkAnswer(spark.table("hive_serde"), Seq(Row(false), Row(true))) + } + + // BINARY + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 BINARY) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde values('1')") + checkAnswer(spark.table("hive_serde"), Row("1".getBytes)) + spark.sql("INSERT INTO TABLE hive_serde values('2')") + checkAnswer(spark.table("hive_serde"), Seq(Row("1".getBytes), Row("2".getBytes))) + } + } + + private def checkComplexTypes(fileFormat: String): Unit = { + // ARRAY + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 ARRAY ) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde SELECT ARRAY('a','b') FROM (SELECT 1) t") + checkAnswer(spark.table("hive_serde"), Row(Array("a", "b"))) + spark.sql("INSERT INTO TABLE hive_serde SELECT ARRAY('c', 'd')") + checkAnswer(spark.table("hive_serde"), Seq(Row(Array("a", "b")), Row(Array("c", "d")))) + } + // MAP + withTable("hive_serde") { + hiveClient.runSqlHive(s"CREATE TABLE hive_serde (c1 MAP ) STORED AS $fileFormat") + hiveClient.runSqlHive("INSERT INTO TABLE hive_serde SELECT MAP(1, 'a') FROM (SELECT 1) t") + checkAnswer(spark.table("hive_serde"), Row(Map(1 -> "a"))) + spark.sql("INSERT INTO TABLE hive_serde SELECT MAP(2, 'b')") + checkAnswer(spark.table("hive_serde"), Seq(Row(Map(1 -> "a")), Row(Map(2 -> "b")))) + } + + // STRUCT + withTable("hive_serde") { + hiveClient.runSqlHive( + s"CREATE TABLE hive_serde (c1 STRUCT ) STORED AS $fileFormat") + hiveClient.runSqlHive( + "INSERT INTO TABLE hive_serde SELECT NAMED_STRUCT('k', 1) FROM (SELECT 1) t") + checkAnswer(spark.table("hive_serde"), Row(Row(1))) + spark.sql("INSERT INTO TABLE hive_serde SELECT NAMED_STRUCT('k', 2)") + checkAnswer(spark.table("hive_serde"), Seq(Row(Row(1)), Row(Row(2)))) + } + } + + Seq("PARQUET", "ORC").foreach { fileFormat => + test(s"Read/Write Hive $fileFormat serde table") { + // Numeric Types + checkNumericTypes(fileFormat, "TINYINT", 2) + checkNumericTypes(fileFormat, "SMALLINT", 2) + checkNumericTypes(fileFormat, "INT", 2) + checkNumericTypes(fileFormat, "BIGINT", 2) + checkNumericTypes(fileFormat, "FLOAT", 2.1F) + checkNumericTypes(fileFormat, "DOUBLE", 2.1D) + checkNumericTypes(fileFormat, "DECIMAL(9, 2)", 2.1D) + checkNumericTypes(fileFormat, "DECIMAL(18, 2)", 2.1D) + checkNumericTypes(fileFormat, "DECIMAL(38, 2)", 2.1D) + + // Date/Time Types + checkDateTimeTypes(fileFormat) + + // String Types + checkStringTypes(fileFormat, "STRING", "s1") + checkStringTypes(fileFormat, "VARCHAR(10)", "s2") + checkCharTypes(fileFormat) + + // Misc Types + checkMiscTypes(fileFormat) + + // Complex Types + checkComplexTypes(fileFormat) + } + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala index d7752e987cb4b..ed4304b9aa57b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala @@ -21,13 +21,14 @@ import java.net.URI import org.scalatest.BeforeAndAfterAll -import org.apache.spark.sql.{AnalysisException, SaveMode} +import org.apache.spark.sql.{AnalysisException, SaveMode, SparkSession} import org.apache.spark.sql.catalyst.catalog.CatalogTable import org.apache.spark.sql.catalyst.plans.PlanTest import org.apache.spark.sql.execution.command.{CreateTableCommand, DDLUtils} import org.apache.spark.sql.execution.datasources.CreateTable import org.apache.spark.sql.execution.metric.InputOutputMetricsHelper import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf} import org.apache.spark.sql.types.StructType /** @@ -210,4 +211,23 @@ class HiveSerDeSuite extends HiveComparisonTest with PlanTest with BeforeAndAfte val e8 = intercept[IllegalArgumentException](analyzeCreateTable(v8)) assert(e8.getMessage.contains("invalid fileFormat: 'wrong'")) } + + test("SPARK-27555: fall back to hive-site.xml if hive.default.fileformat " + + "is not found in SQLConf ") { + val testSession = SparkSession.getActiveSession.get + try { + testSession.sparkContext.hadoopConfiguration.set("hive.default.fileformat", "parquetfile") + val sqlConf = new SQLConf() + var storageFormat = HiveSerDe.getDefaultStorage(sqlConf) + assert(storageFormat.serde. + contains("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe")) + // should take orc as it is present in sqlConf + sqlConf.setConfString("hive.default.fileformat", "orc") + storageFormat = HiveSerDe.getDefaultStorage(sqlConf) + assert(storageFormat.serde.contains("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) + } + finally { + testSession.sparkContext.hadoopConfiguration.unset("hive.default.fileformat") + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala index fe3deceb08067..b0d615c1acee9 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDAFSuite.scala @@ -31,6 +31,7 @@ import test.org.apache.spark.sql.MyDoubleAvg import org.apache.spark.sql.{AnalysisException, QueryTest, Row} import org.apache.spark.sql.execution.aggregate.ObjectHashAggregateExec import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { @@ -40,6 +41,7 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { super.beforeAll() sql(s"CREATE TEMPORARY FUNCTION mock AS '${classOf[MockUDAF].getName}'") sql(s"CREATE TEMPORARY FUNCTION hive_max AS '${classOf[GenericUDAFMax].getName}'") + sql(s"CREATE TEMPORARY FUNCTION mock2 AS '${classOf[MockUDAF2].getName}'") Seq( (0: Integer) -> "val_0", @@ -92,6 +94,35 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { )) } + test("SPARK-24935: customized Hive UDAF with two aggregation buffers") { + withTempView("v") { + spark.range(100).createTempView("v") + val df = sql("SELECT id % 2, mock2(id) FROM v GROUP BY id % 2") + + val aggs = df.queryExecution.executedPlan.collect { + case agg: ObjectHashAggregateExec => agg + } + + // There should be two aggregate operators, one for partial aggregation, and the other for + // global aggregation. + assert(aggs.length == 2) + + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "1") { + checkAnswer(df, Seq( + Row(0, Row(50, 0)), + Row(1, Row(50, 0)) + )) + } + + withSQLConf(SQLConf.OBJECT_AGG_SORT_BASED_FALLBACK_THRESHOLD.key -> "100") { + checkAnswer(df, Seq( + Row(0, Row(50, 0)), + Row(1, Row(50, 0)) + )) + } + } + } + test("call JAVA UDAF") { withTempView("temp") { withUserDefinedFunction("myDoubleAvg" -> false) { @@ -118,6 +149,16 @@ class HiveUDAFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { } } } + + test("SPARK-27907 HiveUDAF with 0 rows throws NPE") { + withTable("abc") { + sql("create table abc(a int)") + checkAnswer(sql("select histogram_numeric(a,2) from abc"), Row(null)) + sql("insert into abc values (1)") + checkAnswer(sql("select histogram_numeric(a,2) from abc"), Row(Row(1.0, 1.0) :: Nil)) + checkAnswer(sql("select histogram_numeric(a,2) from abc where a=3"), Row(null)) + } + } } /** @@ -127,12 +168,22 @@ class MockUDAF extends AbstractGenericUDAFResolver { override def getEvaluator(info: Array[TypeInfo]): GenericUDAFEvaluator = new MockUDAFEvaluator } +class MockUDAF2 extends AbstractGenericUDAFResolver { + override def getEvaluator(info: Array[TypeInfo]): GenericUDAFEvaluator = new MockUDAFEvaluator2 +} + class MockUDAFBuffer(var nonNullCount: Long, var nullCount: Long) extends GenericUDAFEvaluator.AbstractAggregationBuffer { override def estimate(): Int = JavaDataModel.PRIMITIVES2 * 2 } +class MockUDAFBuffer2(var nonNullCount: Long, var nullCount: Long) + extends GenericUDAFEvaluator.AbstractAggregationBuffer { + + override def estimate(): Int = JavaDataModel.PRIMITIVES2 * 2 +} + class MockUDAFEvaluator extends GenericUDAFEvaluator { private val nonNullCountOI = PrimitiveObjectInspectorFactory.javaLongObjectInspector @@ -184,3 +235,80 @@ class MockUDAFEvaluator extends GenericUDAFEvaluator { override def terminate(agg: AggregationBuffer): AnyRef = terminatePartial(agg) } + +// Same as MockUDAFEvaluator but using two aggregation buffers, one for PARTIAL1 and the other +// for PARTIAL2. +class MockUDAFEvaluator2 extends GenericUDAFEvaluator { + private val nonNullCountOI = PrimitiveObjectInspectorFactory.javaLongObjectInspector + + private val nullCountOI = PrimitiveObjectInspectorFactory.javaLongObjectInspector + private var aggMode: Mode = null + + private val bufferOI = { + val fieldNames = Seq("nonNullCount", "nullCount").asJava + val fieldOIs = Seq(nonNullCountOI: ObjectInspector, nullCountOI: ObjectInspector).asJava + ObjectInspectorFactory.getStandardStructObjectInspector(fieldNames, fieldOIs) + } + + private val nonNullCountField = bufferOI.getStructFieldRef("nonNullCount") + + private val nullCountField = bufferOI.getStructFieldRef("nullCount") + + override def getNewAggregationBuffer: AggregationBuffer = { + // These 2 modes consume original data. + if (aggMode == Mode.PARTIAL1 || aggMode == Mode.COMPLETE) { + new MockUDAFBuffer(0L, 0L) + } else { + new MockUDAFBuffer2(0L, 0L) + } + } + + override def reset(agg: AggregationBuffer): Unit = { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + buffer.nonNullCount = 0L + buffer.nullCount = 0L + } + + override def init(mode: Mode, parameters: Array[ObjectInspector]): ObjectInspector = { + aggMode = mode + bufferOI + } + + override def iterate(agg: AggregationBuffer, parameters: Array[AnyRef]): Unit = { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + if (parameters.head eq null) { + buffer.nullCount += 1L + } else { + buffer.nonNullCount += 1L + } + } + + override def merge(agg: AggregationBuffer, partial: Object): Unit = { + if (partial ne null) { + val nonNullCount = nonNullCountOI.get(bufferOI.getStructFieldData(partial, nonNullCountField)) + val nullCount = nullCountOI.get(bufferOI.getStructFieldData(partial, nullCountField)) + val buffer = agg.asInstanceOf[MockUDAFBuffer2] + buffer.nonNullCount += nonNullCount + buffer.nullCount += nullCount + } + } + + // As this method is called for both states, Partial1 and Partial2, the hack in the method + // to check for class of aggregation buffer was necessary. + override def terminatePartial(agg: AggregationBuffer): AnyRef = { + var result: AnyRef = null + if (agg.getClass.toString.contains("MockUDAFBuffer2")) { + val buffer = agg.asInstanceOf[MockUDAFBuffer2] + result = Array[Object](buffer.nonNullCount: java.lang.Long, buffer.nullCount: java.lang.Long) + } else { + val buffer = agg.asInstanceOf[MockUDAFBuffer] + result = Array[Object](buffer.nonNullCount: java.lang.Long, buffer.nullCount: java.lang.Long) + } + result + } + + override def terminate(agg: AggregationBuffer): AnyRef = { + val buffer = agg.asInstanceOf[MockUDAFBuffer2] + Array[Object](buffer.nonNullCount: java.lang.Long, buffer.nullCount: java.lang.Long) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala index 6198d4963df33..587eab4a24810 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveUDFSuite.scala @@ -454,14 +454,14 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { // EXTERNAL OpenCSVSerde table pointing to LOCATION val file1 = new File(tempDir + "/data1") - val writer1 = new PrintWriter(file1) - writer1.write("1,2") - writer1.close() + Utils.tryWithResource(new PrintWriter(file1)) { writer => + writer.write("1,2") + } val file2 = new File(tempDir + "/data2") - val writer2 = new PrintWriter(file2) - writer2.write("1,2") - writer2.close() + Utils.tryWithResource(new PrintWriter(file2)) { writer => + writer.write("1,2") + } sql( s"""CREATE EXTERNAL TABLE csv_table(page_id INT, impressions INT) @@ -638,6 +638,31 @@ class HiveUDFSuite extends QueryTest with TestHiveSingleton with SQLTestUtils { Row(3) :: Row(3) :: Nil) } } + + test("SPARK-25768 constant argument expecting Hive UDF") { + withTempView("inputTable") { + spark.range(10).createOrReplaceTempView("inputTable") + withUserDefinedFunction("testGenericUDAFPercentileApprox" -> false) { + val numFunc = spark.catalog.listFunctions().count() + sql(s"CREATE FUNCTION testGenericUDAFPercentileApprox AS '" + + s"${classOf[GenericUDAFPercentileApprox].getName}'") + checkAnswer( + sql("SELECT testGenericUDAFPercentileApprox(id, 0.5) FROM inputTable"), + Seq(Row(4.0))) + } + } + } + test("SPARK-28012 Hive UDF supports struct type foldable expression") { + withUserDefinedFunction("testUDFStructType" -> false) { + // Simulate a hive udf that supports struct parameters + sql("CREATE FUNCTION testUDFStructType AS '" + + s"${classOf[GenericUDFArray].getName}'") + checkAnswer( + sql("SELECT testUDFStructType(named_struct('name', 'xx', 'value', 1))[0].value"), + Seq(Row(1))) + } + } + } class TestPair(x: Int, y: Int) extends Writable with Serializable { diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala index eaedac1fa95d8..b20ef035594da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/Hive_2_1_DDLSuite.scala @@ -17,15 +17,12 @@ package org.apache.spark.sql.hive.execution -import scala.language.existentials - import org.apache.hadoop.conf.Configuration import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach} import org.apache.spark.{SparkConf, SparkFunSuite} import org.apache.spark.launcher.SparkLauncher import org.apache.spark.sql.AnalysisException -import org.apache.spark.sql.catalyst.catalog._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.internal.StaticSQLConf._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala index c9309197791bd..2391106cfb253 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ObjectHashAggregateSuite.scala @@ -124,7 +124,7 @@ class ObjectHashAggregateSuite .add("f2", ArrayType(BooleanType), nullable = true), // UDT - new UDT.MyDenseVectorUDT(), + new TestUDT.MyDenseVectorUDT(), // Others StringType, @@ -259,7 +259,7 @@ class ObjectHashAggregateSuite StringType, BinaryType, NullType, BooleanType ) - val udt = new UDT.MyDenseVectorUDT() + val udt = new TestUDT.MyDenseVectorUDT() val fixedLengthTypes = builtinNumericTypes ++ Seq(BooleanType, NullType) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala index 94384185d190a..6b2d0c656b371 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruneFileSourcePartitionsSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.Matchers._ + import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.dsl.expressions._ @@ -25,7 +27,10 @@ import org.apache.spark.sql.catalyst.plans.logical.{Filter, LogicalPlan, Project import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution.datasources.{CatalogFileIndex, HadoopFsRelation, LogicalRelation, PruneFileSourcePartitions} import org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat +import org.apache.spark.sql.execution.joins.BroadcastHashJoinExec +import org.apache.spark.sql.functions.broadcast import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.internal.SQLConf import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types.StructType @@ -91,4 +96,15 @@ class PruneFileSourcePartitionsSuite extends QueryTest with SQLTestUtils with Te assert(size2 < tableStats.get.sizeInBytes) } } + + test("SPARK-26576 Broadcast hint not applied to partitioned table") { + withTable("tbl") { + withSQLConf(SQLConf.AUTO_BROADCASTJOIN_THRESHOLD.key -> "-1") { + spark.range(10).selectExpr("id", "id % 3 as p").write.partitionBy("p").saveAsTable("tbl") + val df = spark.table("tbl") + val qe = df.join(broadcast(df), "p").queryExecution + qe.sparkPlan.collect { case j: BroadcastHashJoinExec => j } should have size 1 + } + } + } } 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 e49aea267026e..fd505d1e4cc30 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import java.io.File +import java.net.URI import java.nio.charset.StandardCharsets import java.sql.{Date, Timestamp} import java.util.{Locale, Set} @@ -25,18 +26,20 @@ import java.util.{Locale, Set} import com.google.common.io.Files import org.apache.hadoop.fs.{FileSystem, Path} -import org.apache.spark.TestUtils +import org.apache.spark.{SparkException, TestUtils} import org.apache.spark.sql._ import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, FunctionRegistry} 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.command.LoadDataCommand import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.{HiveExternalCatalog, HiveUtils} -import org.apache.spark.sql.hive.test.TestHiveSingleton +import org.apache.spark.sql.hive.test.{HiveTestUtils, TestHiveSingleton} import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.StaticSQLConf.GLOBAL_TEMP_DATABASE import org.apache.spark.sql.test.SQLTestUtils import org.apache.spark.sql.types._ import org.apache.spark.unsafe.types.CalendarInterval @@ -71,13 +74,13 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { test("query global temp view") { val df = Seq(1).toDF("i1") df.createGlobalTempView("tbl1") - val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + val global_temp_db = spark.conf.get(GLOBAL_TEMP_DATABASE) checkAnswer(spark.sql(s"select * from ${global_temp_db}.tbl1"), Row(1)) spark.sql(s"drop view ${global_temp_db}.tbl1") } test("non-existent global temp view") { - val global_temp_db = spark.conf.get("spark.sql.globalTempDatabase") + val global_temp_db = spark.conf.get(GLOBAL_TEMP_DATABASE) val message = intercept[AnalysisException] { spark.sql(s"select * from ${global_temp_db}.nonexistentview") }.getMessage @@ -690,8 +693,8 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { |AS SELECT key, value FROM mytable1 """.stripMargin) }.getMessage - assert(e.contains("A Create Table As Select (CTAS) statement is not allowed to " + - "create a partitioned table using Hive's file formats")) + assert(e.contains("Create Partitioned Table As Select cannot specify data type for " + + "the partition columns of the target table")) } } } @@ -1103,7 +1106,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { override def run() { // To make sure this test works, this jar should not be loaded in another place. sql( - s"ADD JAR ${hiveContext.getHiveFile("hive-contrib-0.13.1.jar").getCanonicalPath()}") + s"ADD JAR ${HiveTestUtils.getHiveContribJar.getCanonicalPath}") try { sql( """ @@ -1176,11 +1179,35 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } test("Convert hive interval term into Literal of CalendarIntervalType") { + checkAnswer(sql("select interval '0 0:0:0.1' day to second"), + Row(CalendarInterval.fromString("interval 100 milliseconds"))) checkAnswer(sql("select interval '10-9' year to month"), Row(CalendarInterval.fromString("interval 10 years 9 months"))) + checkAnswer(sql("select interval '20 15:40:32.99899999' day to hour"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours"))) + checkAnswer(sql("select interval '20 15:40:32.99899999' day to minute"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes"))) checkAnswer(sql("select interval '20 15:40:32.99899999' day to second"), Row(CalendarInterval.fromString("interval 2 weeks 6 days 15 hours 40 minutes " + - "32 seconds 99 milliseconds 899 microseconds"))) + "32 seconds 998 milliseconds 999 microseconds"))) + checkAnswer(sql("select interval '15:40:32.99899999' hour to minute"), + Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) + checkAnswer(sql("select interval '15:40.99899999' hour to second"), + Row(CalendarInterval.fromString("interval 15 minutes 40 seconds 998 milliseconds " + + "999 microseconds"))) + checkAnswer(sql("select interval '15:40' hour to second"), + Row(CalendarInterval.fromString("interval 15 hours 40 minutes"))) + checkAnswer(sql("select interval '15:40:32.99899999' hour to second"), + Row(CalendarInterval.fromString("interval 15 hours 40 minutes 32 seconds 998 milliseconds " + + "999 microseconds"))) + checkAnswer(sql("select interval '20 40:32.99899999' minute to second"), + Row(CalendarInterval.fromString("interval 2 weeks 6 days 40 minutes 32 seconds " + + "998 milliseconds 999 microseconds"))) + checkAnswer(sql("select interval '40:32.99899999' minute to second"), + Row(CalendarInterval.fromString("interval 40 minutes 32 seconds 998 milliseconds " + + "999 microseconds"))) + checkAnswer(sql("select interval '40:32' minute to second"), + Row(CalendarInterval.fromString("interval 40 minutes 32 seconds"))) checkAnswer(sql("select interval '30' year"), Row(CalendarInterval.fromString("interval 30 years"))) checkAnswer(sql("select interval '25' month"), @@ -1985,6 +2012,12 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-25738: defaultFs can have a port") { + val defaultURI = new URI("hdfs://fizz.buzz.com:8020") + val r = LoadDataCommand.makeQualified(defaultURI, new Path("/foo/bar"), new Path("/flim/flam")) + assert(r === new Path("hdfs://fizz.buzz.com:8020/flim/flam")) + } + test("Insert overwrite with partition") { withTable("tableWithPartition") { sql( @@ -2155,6 +2188,11 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { }.getMessage assert(m.contains(s"contains invalid character(s)")) + val m1 = intercept[AnalysisException] { + sql(s"CREATE TABLE t21912 STORED AS $source AS SELECT 1 `col$name`") + }.getMessage + assert(m1.contains(s"contains invalid character(s)")) + val m2 = intercept[AnalysisException] { sql(s"CREATE TABLE t21912 USING $source AS SELECT 1 `col$name`") }.getMessage @@ -2268,4 +2306,110 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton { } } + test("SPARK-25271: Hive ctas commands should use data source if it is convertible") { + withTempView("p") { + Seq(1, 2, 3).toDF("id").createOrReplaceTempView("p") + + Seq("orc", "parquet").foreach { format => + Seq(true, false).foreach { isConverted => + withSQLConf( + HiveUtils.CONVERT_METASTORE_ORC.key -> s"$isConverted", + HiveUtils.CONVERT_METASTORE_PARQUET.key -> s"$isConverted") { + Seq(true, false).foreach { isConvertedCtas => + withSQLConf(HiveUtils.CONVERT_METASTORE_CTAS.key -> s"$isConvertedCtas") { + + val targetTable = "targetTable" + withTable(targetTable) { + val df = sql(s"CREATE TABLE $targetTable STORED AS $format AS SELECT id FROM p") + checkAnswer(sql(s"SELECT id FROM $targetTable"), + Row(1) :: Row(2) :: Row(3) :: Nil) + + val ctasDSCommand = df.queryExecution.analyzed.collect { + case _: OptimizedCreateHiveTableAsSelectCommand => true + }.headOption + val ctasCommand = df.queryExecution.analyzed.collect { + case _: CreateHiveTableAsSelectCommand => true + }.headOption + + if (isConverted && isConvertedCtas) { + assert(ctasDSCommand.nonEmpty) + assert(ctasCommand.isEmpty) + } else { + assert(ctasDSCommand.isEmpty) + assert(ctasCommand.nonEmpty) + } + } + } + } + } + } + } + } + } + + test("SPARK-26181 hasMinMaxStats method of ColumnStatsMap is not correct") { + withSQLConf(SQLConf.CBO_ENABLED.key -> "true") { + withTable("all_null") { + sql("create table all_null (attr1 int, attr2 int)") + sql("insert into all_null values (null, null)") + sql("analyze table all_null compute statistics for columns attr1, attr2") + // check if the stats can be calculated without Cast exception. + sql("select * from all_null where attr1 < 1").queryExecution.stringWithStats + sql("select * from all_null where attr1 < attr2").queryExecution.stringWithStats + } + } + } + + test("SPARK-26709: OptimizeMetadataOnlyQuery does not handle empty records correctly") { + Seq(true, false).foreach { enableOptimizeMetadataOnlyQuery => + // This test case is only for file source V1. As the rule OptimizeMetadataOnlyQuery is + // disabled by default, we can skip testing file source v2 in current stage. + withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> enableOptimizeMetadataOnlyQuery.toString, + SQLConf.USE_V1_SOURCE_READER_LIST.key -> "parquet") { + withTable("t") { + sql("CREATE TABLE t (col1 INT, p1 INT) USING PARQUET PARTITIONED BY (p1)") + sql("INSERT INTO TABLE t PARTITION (p1 = 5) SELECT ID FROM range(1, 1)") + if (enableOptimizeMetadataOnlyQuery) { + // The result is wrong if we enable the configuration. + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(5)) + } else { + checkAnswer(sql("SELECT MAX(p1) FROM t"), Row(null)) + } + checkAnswer(sql("SELECT MAX(col1) FROM t"), Row(null)) + } + } + } + } + + test("SPARK-25158: " + + "Executor accidentally exit because ScriptTransformationWriterThread throw Exception") { + withTempView("test") { + val defaultUncaughtExceptionHandler = Thread.getDefaultUncaughtExceptionHandler + try { + val uncaughtExceptionHandler = new TestUncaughtExceptionHandler + Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler) + + // Use a bad udf to generate failed inputs. + val badUDF = org.apache.spark.sql.functions.udf((x: Int) => { + if (x < 1) x + else throw new RuntimeException("Failed to produce data.") + }) + spark + .range(5) + .select(badUDF('id).as("a")) + .createOrReplaceTempView("test") + val scriptFilePath = getTestResourcePath("data") + val e = intercept[SparkException] { + sql( + s"""FROM test SELECT TRANSFORM(a) + |USING 'python $scriptFilePath/scripts/test_transform.py "\t"' + """.stripMargin).collect() + } + assert(e.getMessage.contains("Failed to produce data.")) + assert(uncaughtExceptionHandler.exception.isEmpty) + } finally { + Thread.setDefaultUncaughtExceptionHandler(defaultUncaughtExceptionHandler) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala index 5f73b7170c612..ed3b376f6eda1 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ScriptTransformationSuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.scalatest.BeforeAndAfterEach import org.scalatest.exceptions.TestFailedException import org.apache.spark.{SparkException, TaskContext, TestUtils} @@ -29,7 +30,8 @@ import org.apache.spark.sql.execution.{SparkPlan, SparkPlanTest, UnaryExecNode} import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types.StringType -class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { +class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton with + BeforeAndAfterEach { import spark.implicits._ private val noSerdeIOSchema = HiveScriptIOSchema( @@ -49,6 +51,26 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { outputSerdeClass = Some(classOf[LazySimpleSerDe].getCanonicalName) ) + private var defaultUncaughtExceptionHandler: Thread.UncaughtExceptionHandler = _ + + private val uncaughtExceptionHandler = new TestUncaughtExceptionHandler + + protected override def beforeAll(): Unit = { + super.beforeAll() + defaultUncaughtExceptionHandler = Thread.getDefaultUncaughtExceptionHandler + Thread.setDefaultUncaughtExceptionHandler(uncaughtExceptionHandler) + } + + protected override def afterAll(): Unit = { + super.afterAll() + Thread.setDefaultUncaughtExceptionHandler(defaultUncaughtExceptionHandler) + } + + override protected def afterEach(): Unit = { + super.afterEach() + uncaughtExceptionHandler.cleanStatus() + } + test("cat without SerDe") { assume(TestUtils.testCommandAvailable("/bin/bash")) @@ -63,6 +85,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { ioschema = noSerdeIOSchema ), rowsDf.collect()) + assert(uncaughtExceptionHandler.exception.isEmpty) } test("cat with LazySimpleSerDe") { @@ -79,6 +102,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { ioschema = serdeIOSchema ), rowsDf.collect()) + assert(uncaughtExceptionHandler.exception.isEmpty) } test("script transformation should not swallow errors from upstream operators (no serde)") { @@ -98,6 +122,8 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) + // Before SPARK-25158, uncaughtExceptionHandler will catch IllegalArgumentException + assert(uncaughtExceptionHandler.exception.isEmpty) } test("script transformation should not swallow errors from upstream operators (with serde)") { @@ -117,6 +143,8 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { rowsDf.collect()) } assert(e.getMessage().contains("intentional exception")) + // Before SPARK-25158, uncaughtExceptionHandler will catch IllegalArgumentException + assert(uncaughtExceptionHandler.exception.isEmpty) } test("SPARK-14400 script transformation should fail for bad script command") { @@ -135,6 +163,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { SparkPlanTest.executePlan(plan, hiveContext) } assert(e.getMessage.contains("Subprocess exited with status")) + assert(uncaughtExceptionHandler.exception.isEmpty) } test("SPARK-24339 verify the result after pruning the unused columns") { @@ -154,6 +183,7 @@ class ScriptTransformationSuite extends SparkPlanTest with TestHiveSingleton { ioschema = serdeIOSchema ), rowsDf.select("name").collect()) + assert(uncaughtExceptionHandler.exception.isEmpty) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestUncaughtExceptionHandler.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestUncaughtExceptionHandler.scala new file mode 100644 index 0000000000000..681eb4e255dbc --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/TestUncaughtExceptionHandler.scala @@ -0,0 +1,31 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +class TestUncaughtExceptionHandler extends Thread.UncaughtExceptionHandler { + + @volatile private var _exception: Throwable = _ + + def exception: Option[Throwable] = Option(_exception) + + def cleanStatus(): Unit = _exception = null + + override def uncaughtException(t: Thread, e: Throwable): Unit = { + _exception = e + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala index 5094763b0cd2a..b5e50915c7c89 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcFilterSuite.scala @@ -30,6 +30,7 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning.PhysicalOperation import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, HadoopFsRelation, LogicalRelation} import org.apache.spark.sql.execution.datasources.orc.OrcTest +import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton import org.apache.spark.sql.types._ @@ -76,15 +77,27 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { checkFilterPredicate(df, predicate, checkComparisonOperator) } - private def checkFilterPredicate + private def checkFilterPredicateWithDiffHiveVersion (predicate: Predicate, stringExpr: String) (implicit df: DataFrame): Unit = { def checkLogicalOperator(filter: SearchArgument) = { - assert(filter.toString == stringExpr) + if (HiveUtils.isHive23) { + assert(filter.toString == stringExpr.replace("\n", ", ")) + } else { + assert(filter.toString == stringExpr) + } } checkFilterPredicate(df, predicate, checkLogicalOperator) } + private def assertResultWithDiffHiveVersion(expected : String)(c : scala.Any) = { + if (HiveUtils.isHive23) { + assertResult(expected.replace("\n", ", "))(c) + } else { + assertResult(expected)(c) + } + } + private def checkNoFilterPredicate (predicate: Predicate) (implicit df: DataFrame): Unit = { @@ -295,30 +308,30 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { // such as `and`, `or` or `not`. So, this function uses `SearchArgument.toString()` // to produce string expression and then compare it to given string expression below. // This might have to be changed after Hive version is upgraded. - checkFilterPredicate( + checkFilterPredicateWithDiffHiveVersion( '_1.isNotNull, """leaf-0 = (IS_NULL _1) |expr = (not leaf-0)""".stripMargin.trim ) - checkFilterPredicate( + checkFilterPredicateWithDiffHiveVersion( '_1 =!= 1, """leaf-0 = (IS_NULL _1) |leaf-1 = (EQUALS _1 1) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) - checkFilterPredicate( + checkFilterPredicateWithDiffHiveVersion( !('_1 < 4), """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 4) |expr = (and (not leaf-0) (not leaf-1))""".stripMargin.trim ) - checkFilterPredicate( + checkFilterPredicateWithDiffHiveVersion( '_1 < 2 || '_1 > 3, """leaf-0 = (LESS_THAN _1 2) |leaf-1 = (LESS_THAN_EQUALS _1 3) |expr = (or leaf-0 (not leaf-1))""".stripMargin.trim ) - checkFilterPredicate( + checkFilterPredicateWithDiffHiveVersion( '_1 < 2 && '_1 > 3, """leaf-0 = (IS_NULL _1) |leaf-1 = (LESS_THAN _1 2) @@ -341,9 +354,11 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { checkNoFilterPredicate('_1 <=> 1.b) } // DateType - val stringDate = "2015-01-01" - withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => - checkNoFilterPredicate('_1 === Date.valueOf(stringDate)) + if (!HiveUtils.isHive23) { + val stringDate = "2015-01-01" + withOrcDataFrame(Seq(Tuple1(Date.valueOf(stringDate)))) { implicit df => + checkNoFilterPredicate('_1 === Date.valueOf(stringDate)) + } } // MapType withOrcDataFrame((1 to 4).map(i => Tuple1(Map(i -> i)))) { implicit df => @@ -358,7 +373,7 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { Array( StructField("a", IntegerType, nullable = true), StructField("b", StringType, nullable = true))) - assertResult( + assertResultWithDiffHiveVersion( """leaf-0 = (LESS_THAN a 10) |expr = leaf-0 """.stripMargin.trim @@ -370,7 +385,7 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } // The `LessThan` should be converted while the whole inner `And` shouldn't - assertResult( + assertResultWithDiffHiveVersion( """leaf-0 = (LESS_THAN a 10) |expr = leaf-0 """.stripMargin.trim @@ -384,19 +399,8 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { )).get.toString } - // Can not remove unsupported `StringContains` predicate since it is under `Or` operator. - assert(OrcFilters.createFilter(schema, Array( - Or( - LessThan("a", 10), - And( - StringContains("b", "prefix"), - GreaterThan("a", 1) - ) - ) - )).isEmpty) - // Safely remove unsupported `StringContains` predicate and push down `LessThan` - assertResult( + assertResultWithDiffHiveVersion( """leaf-0 = (LESS_THAN a 10) |expr = leaf-0 """.stripMargin.trim @@ -410,7 +414,7 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { } // Safely remove unsupported `StringContains` predicate, push down `LessThan` and `GreaterThan`. - assertResult( + assertResultWithDiffHiveVersion( """leaf-0 = (LESS_THAN a 10) |leaf-1 = (LESS_THAN_EQUALS a 1) |expr = (and leaf-0 (not leaf-1)) @@ -427,4 +431,53 @@ class HiveOrcFilterSuite extends OrcTest with TestHiveSingleton { )).get.toString } } + + test("SPARK-27699 Converting disjunctions into ORC SearchArguments") { + import org.apache.spark.sql.sources._ + // The `LessThan` should be converted while the `StringContains` shouldn't + val schema = new StructType( + Array( + StructField("a", IntegerType, nullable = true), + StructField("b", StringType, nullable = true))) + + // The predicate `StringContains` predicate is not able to be pushed down. + assertResultWithDiffHiveVersion("leaf-0 = (LESS_THAN_EQUALS a 10)\nleaf-1 = (LESS_THAN a 1)\n" + + "expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + GreaterThan("a", 10), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assertResultWithDiffHiveVersion("leaf-0 = (LESS_THAN_EQUALS a 10)\nleaf-1 = (LESS_THAN a 1)\n" + + "expr = (or (not leaf-0) leaf-1)") { + OrcFilters.createFilter(schema, Array( + Or( + And( + GreaterThan("a", 10), + StringContains("b", "foobar") + ), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).get.toString + } + + assert(OrcFilters.createFilter(schema, Array( + Or( + StringContains("b", "foobar"), + And( + StringContains("b", "prefix"), + LessThan("a", 1) + ) + ) + )).isEmpty) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala index 597b0f56a55e4..94f35b0b3d523 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcQuerySuite.scala @@ -218,4 +218,14 @@ class HiveOrcQuerySuite extends OrcQueryTest with TestHiveSingleton { } } } + + test("SPARK-26437 Can not query decimal type when value is 0") { + assume(HiveUtils.isHive23, "bad test: This bug fixed by HIVE-13083(Hive 2.0.1)") + withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> "false") { + withTable("spark_26437") { + sql("CREATE TABLE spark_26437 STORED AS ORCFILE AS SELECT 0.00 AS c1") + checkAnswer(spark.table("spark_26437"), Seq(Row(0.00))) + } + } + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala index 7fefaf53939bd..3104fb4d8173c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/HiveOrcSourceSuite.scala @@ -21,11 +21,9 @@ import java.io.File import org.apache.spark.sql.{AnalysisException, Row} import org.apache.spark.sql.TestingUDT.{IntervalData, IntervalUDT} -import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.execution.datasources.orc.OrcSuite import org.apache.spark.sql.hive.HiveUtils import org.apache.spark.sql.hive.test.TestHiveSingleton -import org.apache.spark.sql.internal.HiveSerDe import org.apache.spark.sql.types._ import org.apache.spark.util.Utils @@ -67,33 +65,6 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { """.stripMargin) } - test("SPARK-22972: hive orc source") { - val tableName = "normal_orc_as_source_hive" - withTable(tableName) { - sql( - s""" - |CREATE TABLE $tableName - |USING org.apache.spark.sql.hive.orc - |OPTIONS ( - | PATH '${new File(orcTableAsDir.getAbsolutePath).toURI}' - |) - """.stripMargin) - - val tableMetadata = spark.sessionState.catalog.getTableMetadata( - TableIdentifier(tableName)) - assert(tableMetadata.storage.inputFormat == - Option("org.apache.hadoop.hive.ql.io.orc.OrcInputFormat")) - assert(tableMetadata.storage.outputFormat == - Option("org.apache.hadoop.hive.ql.io.orc.OrcOutputFormat")) - assert(tableMetadata.storage.serde == - Option("org.apache.hadoop.hive.ql.io.orc.OrcSerde")) - assert(HiveSerDe.sourceToSerDe("org.apache.spark.sql.hive.orc") - .equals(HiveSerDe.sourceToSerDe("orc"))) - assert(HiveSerDe.sourceToSerDe("org.apache.spark.sql.orc") - .equals(HiveSerDe.sourceToSerDe("orc"))) - } - } - test("SPARK-19459/SPARK-18220: read char/varchar column written by Hive") { val location = Utils.createTempDir() val uri = location.toURI @@ -178,7 +149,12 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { test("Check BloomFilter creation") { Seq(true, false).foreach { convertMetastore => withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { - testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER) // Before ORC-101 + if (HiveUtils.isHive23) { + testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER_UTF8) + } else { + // Before ORC-101 + testBloomFilterCreation(org.apache.orc.OrcProto.Stream.Kind.BLOOM_FILTER) + } } } } @@ -186,8 +162,12 @@ class HiveOrcSourceSuite extends OrcSuite with TestHiveSingleton { test("Enforce direct encoding column-wise selectively") { Seq(true, false).foreach { convertMetastore => withSQLConf(HiveUtils.CONVERT_METASTORE_ORC.key -> s"$convertMetastore") { - testSelectiveDictionaryEncoding(isSelective = false) + testSelectiveDictionaryEncoding(isSelective = false, isHive23 = HiveUtils.isHive23) } } } + + test("SPARK-11412 read and merge orc schemas in parallel") { + testMergeSchemasInParallel(OrcFileOperator.readOrcSchemasInParallel) + } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala index 870ad4818eb28..c03ae144a1595 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/orc/OrcReadBenchmark.scala @@ -32,9 +32,11 @@ import org.apache.spark.sql.types._ * Benchmark to measure ORC read performance. * {{{ * To run this benchmark: - * 1. without sbt: bin/spark-submit --class - * 2. build/sbt "sql/test:runMain " - * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "sql/test:runMain " + * 1. without sbt: bin/spark-submit --class + * --jars ,,,, + * + * 2. build/sbt "hive/test:runMain " + * 3. generate result: SPARK_GENERATE_BENCHMARK_FILES=1 build/sbt "hive/test:runMain " * Results will be written to "benchmarks/OrcReadBenchmark-results.txt". * }}} * @@ -94,12 +96,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } - benchmark.addCase("Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } @@ -131,12 +127,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() } - benchmark.addCase("Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(c1), sum(length(c2)) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(c1), sum(length(c2)) FROM hiveOrcTable").collect() } @@ -166,12 +156,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() } - benchmark.addCase("Data column - Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(id) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Data column - Hive built-in ORC") { _ => spark.sql("SELECT sum(id) FROM hiveOrcTable").collect() } @@ -186,12 +170,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() } - benchmark.addCase("Partition column - Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(p) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Partition column - Hive built-in ORC") { _ => spark.sql("SELECT sum(p) FROM hiveOrcTable").collect() } @@ -206,12 +184,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() } - benchmark.addCase("Both column - Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(p), sum(id) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Both columns - Hive built-in ORC") { _ => spark.sql("SELECT sum(p), sum(id) FROM hiveOrcTable").collect() } @@ -240,12 +212,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() } - benchmark.addCase("Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT sum(length(c1)) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT sum(length(c1)) FROM hiveOrcTable").collect() } @@ -266,8 +232,9 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { s"SELECT IF(RAND(1) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c1, " + s"IF(RAND(2) < $fractionOfNulls, NULL, CAST(id as STRING)) AS c2 FROM t1")) + val percentageOfNulls = fractionOfNulls * 100 val benchmark = - new Benchmark(s"String with Nulls Scan ($fractionOfNulls%)", values, output = output) + new Benchmark(s"String with Nulls Scan ($percentageOfNulls%)", values, output = output) benchmark.addCase("Native ORC MR") { _ => withSQLConf(SQLConf.ORC_VECTORIZED_READER_ENABLED.key -> "false") { @@ -281,13 +248,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() } - benchmark.addCase("Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql("SELECT SUM(LENGTH(c2)) FROM nativeOrcTable " + - "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() - } - } - benchmark.addCase("Hive built-in ORC") { _ => spark.sql("SELECT SUM(LENGTH(c2)) FROM hiveOrcTable " + "WHERE c1 IS NOT NULL AND c2 IS NOT NULL").collect() @@ -321,12 +281,6 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() } - benchmark.addCase("Native ORC Vectorized with copy") { _ => - withSQLConf(SQLConf.ORC_COPY_BATCH_TO_SPARK.key -> "true") { - spark.sql(s"SELECT sum(c$middle) FROM nativeOrcTable").collect() - } - } - benchmark.addCase("Hive built-in ORC") { _ => spark.sql(s"SELECT sum(c$middle) FROM hiveOrcTable").collect() } @@ -336,7 +290,7 @@ object OrcReadBenchmark extends BenchmarkBase with SQLHelper { } } - override def runBenchmarkSuite(): Unit = { + override def runBenchmarkSuite(mainArgs: Array[String]): Unit = { runBenchmark("SQL Single Numeric Column Scan") { Seq(ByteType, ShortType, IntegerType, LongType, FloatType, DoubleType).foreach { dataType => numericScanBenchmark(1024 * 1024 * 15, dataType) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala new file mode 100644 index 0000000000000..ce40cf51746b2 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/security/HiveHadoopDelegationTokenManagerSuite.scala @@ -0,0 +1,102 @@ +/* + * 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.security + +import org.apache.commons.io.IOUtils +import org.apache.hadoop.conf.Configuration + +import org.apache.spark.{SparkConf, SparkFunSuite} +import org.apache.spark.deploy.security.HadoopDelegationTokenManager +import org.apache.spark.util.Utils + +class HiveHadoopDelegationTokenManagerSuite extends SparkFunSuite { + private val hadoopConf = new Configuration() + + test("default configuration") { + val manager = new HadoopDelegationTokenManager(new SparkConf(false), hadoopConf, null) + assert(manager.isProviderLoaded("hive")) + } + + test("using deprecated configurations") { + val sparkConf = new SparkConf(false) + .set("spark.yarn.security.credentials.hive.enabled", "false") + val manager = new HadoopDelegationTokenManager(sparkConf, hadoopConf, null) + assert(!manager.isProviderLoaded("hive")) + } + + test("SPARK-23209: obtain tokens when Hive classes are not available") { + // This test needs a custom class loader to hide Hive classes which are in the classpath. + // Because the manager code loads the Hive provider directly instead of using reflection, we + // need to drive the test through the custom class loader so a new copy that cannot find + // Hive classes is loaded. + val currentLoader = Thread.currentThread().getContextClassLoader() + val noHive = new ClassLoader() { + override def loadClass(name: String, resolve: Boolean): Class[_] = { + if (name.startsWith("org.apache.hive") || name.startsWith("org.apache.hadoop.hive")) { + throw new ClassNotFoundException(name) + } + + val prefixBlacklist = Seq("java", "scala", "com.sun.", "sun.") + if (prefixBlacklist.exists(name.startsWith(_))) { + return currentLoader.loadClass(name) + } + + val found = findLoadedClass(name) + if (found != null) { + return found + } + + val classFileName = name.replaceAll("\\.", "/") + ".class" + val in = currentLoader.getResourceAsStream(classFileName) + if (in != null) { + val bytes = IOUtils.toByteArray(in) + return defineClass(name, bytes, 0, bytes.length) + } + + throw new ClassNotFoundException(name) + } + } + + Utils.withContextClassLoader(noHive) { + val test = noHive.loadClass(NoHiveTest.getClass.getName().stripSuffix("$")) + test.getMethod("runTest").invoke(null) + } + } +} + +/** Test code for SPARK-23209 to avoid using too much reflection above. */ +private object NoHiveTest { + + def runTest(): Unit = { + try { + val manager = new HadoopDelegationTokenManager(new SparkConf(), new Configuration(), null) + assert(manager.isProviderLoaded("hadoopfs")) + assert(manager.isProviderLoaded("hbase")) + require(!manager.isProviderLoaded("hive")) + } catch { + case e: Throwable => + // Throw a better exception in case the test fails, since there may be a lot of nesting. + var cause = e + while (cause.getCause() != null) { + cause = cause.getCause() + } + throw cause + } + } + +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala new file mode 100644 index 0000000000000..7631efedf46af --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/HiveTestUtils.scala @@ -0,0 +1,32 @@ +/* + * 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.test + +import java.io.File + +import org.apache.hadoop.hive.contrib.udaf.example.UDAFExampleMax +import org.apache.hive.hcatalog.data.JsonSerDe + +object HiveTestUtils { + + val getHiveContribJar: File = + new File(classOf[UDAFExampleMax].getProtectionDomain.getCodeSource.getLocation.getPath) + + val getHiveHcatalogCoreJar: File = + new File(classOf[JsonSerDe].getProtectionDomain.getCodeSource.getLocation.getPath) +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala similarity index 91% rename from sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala rename to sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala index 71f15a45d162a..d68a47053f18c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/test/TestHive.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/test/TestHive.scala @@ -33,9 +33,11 @@ import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.internal.Logging -import org.apache.spark.sql.{SparkSession, SQLContext} +import org.apache.spark.internal.config +import org.apache.spark.internal.config.UI._ +import org.apache.spark.sql.{DataFrame, Dataset, SparkSession, SQLContext} import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.catalog.{ExternalCatalog, ExternalCatalogWithListener} +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.execution.{QueryExecution, SQLExecution} @@ -43,7 +45,7 @@ import org.apache.spark.sql.execution.command.CacheTableCommand import org.apache.spark.sql.hive._ import org.apache.spark.sql.hive.client.HiveClient import org.apache.spark.sql.internal.{SessionState, SharedState, SQLConf, WithTestConf} -import org.apache.spark.sql.internal.StaticSQLConf.CATALOG_IMPLEMENTATION +import org.apache.spark.sql.internal.StaticSQLConf.{CATALOG_IMPLEMENTATION, WAREHOUSE_PATH} import org.apache.spark.util.{ShutdownHookManager, Utils} // SPARK-3729: Test key required to check for initialization errors with config. @@ -55,12 +57,15 @@ object TestHive new SparkConf() .set("spark.sql.test", "") .set(SQLConf.CODEGEN_FALLBACK.key, "false") - .set("spark.sql.hive.metastore.barrierPrefixes", + .set(HiveUtils.HIVE_METASTORE_BARRIER_PREFIXES.key, "org.apache.spark.sql.hive.execution.PairSerDe") - .set("spark.sql.warehouse.dir", TestHiveContext.makeWarehouseDir().toURI.getPath) + .set(WAREHOUSE_PATH.key, TestHiveContext.makeWarehouseDir().toURI.getPath) // SPARK-8910 - .set("spark.ui.enabled", "false") - .set("spark.unsafe.exceptionOnMemoryLeak", "true") + .set(UI_ENABLED, false) + .set(config.UNSAFE_EXCEPTION_ON_MEMORY_LEAK, true) + // Hive changed the default of hive.metastore.disallow.incompatible.col.type.changes + // from false to true. For details, see the JIRA HIVE-12320 and HIVE-17764. + .set("spark.hadoop.hive.metastore.disallow.incompatible.col.type.changes", "false") // Disable ConvertToLocalRelation for better test coverage. Test cases built on // LocalRelation will exercise the optimization rules better by disabling it as // this rule may potentially block testing of other optimization rules such as @@ -88,7 +93,11 @@ private[hive] class TestHiveExternalCatalog( private[hive] class TestHiveSharedState( sc: SparkContext, hiveClient: Option[HiveClient] = None) - extends SharedState(sc) { + extends SharedState(sc, initialConfigs = Map.empty[String, String]) { + + // The set of loaded tables should be kept in shared state, since there may be multiple sessions + // created that want to use the same tables. + val loadedTables = new collection.mutable.HashSet[String] override lazy val externalCatalog: ExternalCatalogWithListener = { new ExternalCatalogWithListener(new TestHiveExternalCatalog( @@ -219,6 +228,16 @@ private[hive] class TestHiveSparkSession( sharedState.externalCatalog.unwrapped.asInstanceOf[HiveExternalCatalog].client.newSession() } + /** + * This is a temporary hack to override SparkSession.sql so we can still use the version of + * 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 = { + val plan = sessionState.sqlParser.parsePlan(sqlText) + Dataset.ofRows(self, plan) + } + override def newSession(): TestHiveSparkSession = { new TestHiveSparkSession(sc, Some(sharedState), None, loadTestTables) } @@ -287,7 +306,7 @@ private[hive] class TestHiveSparkSession( protected[hive] implicit class SqlCmd(sql: String) { def cmd: () => Unit = { - () => new TestHiveQueryExecution(sql).hiveResultString(): Unit + () => new TestHiveQueryExecution(sql).executedPlan.executeCollect(): Unit } } @@ -468,14 +487,12 @@ private[hive] class TestHiveSparkSession( hiveQTestUtilTables.foreach(registerTestTable) } - private val loadedTables = new collection.mutable.HashSet[String] - - def getLoadedTables: collection.mutable.HashSet[String] = loadedTables + def getLoadedTables: collection.mutable.HashSet[String] = sharedState.loadedTables def loadTestTable(name: String) { - if (!(loadedTables contains name)) { + if (!sharedState.loadedTables.contains(name)) { // Marks the table as loaded first to prevent infinite mutually recursive table loading. - loadedTables += name + sharedState.loadedTables += name logDebug(s"Loading test table $name") val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) @@ -517,12 +534,12 @@ private[hive] class TestHiveSparkSession( } // Clean out the Hive warehouse between each suite - val warehouseDir = new File(new URI(sparkContext.conf.get("spark.sql.warehouse.dir")).getPath) + val warehouseDir = new File(new URI(sparkContext.conf.get(WAREHOUSE_PATH.key)).getPath) Utils.deleteRecursively(warehouseDir) warehouseDir.mkdir() sharedState.cacheManager.clearCache() - loadedTables.clear() + sharedState.loadedTables.clear() sessionState.catalog.reset() metadataHive.reset() @@ -571,14 +588,14 @@ private[hive] class TestHiveQueryExecution( override lazy val analyzed: LogicalPlan = { val describedTables = logical match { - case CacheTableCommand(tbl, _, _) => tbl.table :: Nil + case CacheTableCommand(tbl, _, _, _) => tbl.table :: Nil case _ => Nil } // Make sure any test tables referenced are loaded. val referencedTables = describedTables ++ - logical.collect { case UnresolvedRelation(tableIdent) => tableIdent.table } + logical.collect { case UnresolvedRelation(ident) => ident.last } val resolver = sparkSession.sessionState.conf.resolver val referencedTestTables = sparkSession.testTables.keys.filter { testTable => referencedTables.exists(resolver(_, testTable)) @@ -586,7 +603,7 @@ private[hive] class TestHiveQueryExecution( logDebug(s"Query references test tables: ${referencedTestTables.mkString(", ")}") referencedTestTables.foreach(sparkSession.loadTestTable) // Proceed with analysis. - sparkSession.sessionState.analyzer.executeAndCheck(logical) + sparkSession.sessionState.analyzer.executeAndCheck(logical, tracker) } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala index 6bd59fde550de..4e9c2e7ee7894 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/HadoopFsRelationTest.scala @@ -38,7 +38,8 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes val dataSourceName: String - protected val parquetDataSourceName: String = "parquet" + protected val parquetDataSourceName: String = + classOf[org.apache.spark.sql.execution.datasources.parquet.ParquetFileFormat].getCanonicalName private def isParquetDataSource: Boolean = dataSourceName == parquetDataSourceName @@ -115,7 +116,7 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes new StructType() .add("f1", FloatType, nullable = true) .add("f2", ArrayType(BooleanType, containsNull = true), nullable = true), - new UDT.MyDenseVectorUDT() + new TestUDT.MyDenseVectorUDT() ).filter(supportsDataType) test(s"test all data types") { @@ -135,46 +136,50 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes logInfo(s"Testing $dataType data type$extraMessage") val extraOptions = Map[String, String]( - "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString + "parquet.enable.dictionary" -> parquetDictionaryEncodingEnabled.toString, + "timestampFormat" -> "yyyy-MM-dd'T'HH:mm:ss.SSSXXXXX" ) withTempPath { file => val path = file.getCanonicalPath - val dataGenerator = RandomDataGenerator.forType( - dataType = dataType, - nullable = true, - new Random(System.nanoTime()) - ).getOrElse { - fail(s"Failed to create data generator for schema $dataType") + val seed = System.nanoTime() + withClue(s"Random data generated with the seed: ${seed}") { + val dataGenerator = RandomDataGenerator.forType( + dataType = dataType, + nullable = true, + new Random(seed) + ).getOrElse { + fail(s"Failed to create data generator for schema $dataType") + } + + // Create a DF for the schema with random data. The index field is used to sort the + // DataFrame. This is a workaround for SPARK-10591. + val schema = new StructType() + .add("index", IntegerType, nullable = false) + .add("col", dataType, nullable = true) + val rdd = + spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) + val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) + + df.write + .mode("overwrite") + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .options(extraOptions) + .save(path) + + val loadedDF = spark + .read + .format(dataSourceName) + .option("dataSchema", df.schema.json) + .schema(df.schema) + .options(extraOptions) + .load(path) + .orderBy("index") + + checkAnswer(loadedDF, df) } - - // Create a DF for the schema with random data. The index field is used to sort the - // DataFrame. This is a workaround for SPARK-10591. - val schema = new StructType() - .add("index", IntegerType, nullable = false) - .add("col", dataType, nullable = true) - val rdd = - spark.sparkContext.parallelize((1 to 10).map(i => Row(i, dataGenerator()))) - val df = spark.createDataFrame(rdd, schema).orderBy("index").coalesce(1) - - df.write - .mode("overwrite") - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .options(extraOptions) - .save(path) - - val loadedDF = spark - .read - .format(dataSourceName) - .option("dataSchema", df.schema.json) - .schema(df.schema) - .options(extraOptions) - .load(path) - .orderBy("index") - - checkAnswer(loadedDF, df) } } } @@ -813,10 +818,12 @@ abstract class HadoopFsRelationTest extends QueryTest with SQLTestUtils with Tes assert(preferredLocations.distinct.length == 2) } - checkLocality() - - withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "0") { + withSQLConf(SQLConf.USE_V1_SOURCE_READER_LIST.key -> dataSourceName) { checkLocality() + + withSQLConf(SQLConf.PARALLEL_PARTITION_DISCOVERY_THRESHOLD.key -> "0") { + checkLocality() + } } } } diff --git a/streaming/pom.xml b/streaming/pom.xml index f9a5029a8e818..1d1ea469f7d18 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -20,12 +20,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-streaming_2.11 + spark-streaming_2.12 streaming diff --git a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js index d004f34ab186c..5b75bc3011b6d 100644 --- a/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js +++ b/streaming/src/main/resources/org/apache/spark/streaming/ui/static/streaming-page.js @@ -129,7 +129,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { svg.append("g") .attr("class", "x axis") .attr("transform", "translate(0," + height + ")") - .call(xAxis) + .call(xAxis); svg.append("g") .attr("class", "y axis") @@ -198,7 +198,7 @@ function drawTimeline(id, data, minX, maxX, minY, maxY, unitY, batchInterval) { lastClickedBatch = null; } lastClickedBatch = d.x; - highlightBatchRow(lastClickedBatch) + highlightBatchRow(lastClickedBatch); lastTimeout = window.setTimeout(function () { lastTimeout = null; if (lastClickedBatch != null) { @@ -261,9 +261,9 @@ function drawHistogram(id, values, minY, maxY, unitY, batchInterval) { svg.append("g") .attr("class", "y axis") - .call(yAxis) + .call(yAxis); - var bar = svg.selectAll(".bar") + svg.selectAll(".bar") .data(data) .enter() .append("g") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 135430f1ef621..54f91ff1c69d5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.{SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.io.CompressionCodec import org.apache.spark.streaming.scheduler.JobGenerator import org.apache.spark.util.Utils @@ -61,7 +62,7 @@ class Checkpoint(ssc: StreamingContext, val checkpointTime: Time) "spark.yarn.principal", "spark.kerberos.keytab", "spark.kerberos.principal", - "spark.ui.filters", + UI_FILTERS.key, "spark.mesos.driver.frameworkId") val newSparkConf = new SparkConf(loadDefaults = false).setAll(sparkConfPairs) @@ -217,7 +218,7 @@ class CheckpointWriter( latestCheckpointTime = checkpointTime } var attempts = 0 - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() val tempFile = new Path(checkpointDir, "temp") // We will do checkpoint when generating a batch and completing a batch. When the processing // time of a batch is greater than the batch interval, checkpointing for completing an old @@ -271,9 +272,9 @@ class CheckpointWriter( } // All done, print success - val finishTime = System.currentTimeMillis() logInfo(s"Checkpoint for time $checkpointTime saved to file '$checkpointFile'" + - s", took ${bytes.length} bytes and ${finishTime - startTime} ms") + s", took ${bytes.length} bytes and " + + s"${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms") jobGenerator.onCheckpointCompletion(checkpointTime, clearCheckpointDataLater) return } catch { @@ -303,14 +304,13 @@ class CheckpointWriter( if (stopped) return executor.shutdown() - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS) if (!terminated) { executor.shutdownNow() } - val endTime = System.currentTimeMillis() logInfo(s"CheckpointWriter executor terminated? $terminated," + - s" waited for ${endTime - startTime} ms.") + s" waited for ${TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs)} ms.") stopped = true } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 122f25b21a0d1..15ebef2b325c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -38,6 +38,7 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.input.FixedLengthBinaryInputFormat import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.UI._ import org.apache.spark.rdd.{RDD, RDDOperationScope} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.SerializationDebugger @@ -188,7 +189,7 @@ class StreamingContext private[streaming] ( private[streaming] val progressListener = new StreamingJobProgressListener(this) private[streaming] val uiTab: Option[StreamingTab] = - if (conf.getBoolean("spark.ui.enabled", true)) { + if (conf.get(UI_ENABLED)) { Some(new StreamingTab(this)) } else { None @@ -409,6 +410,8 @@ class StreamingContext private[streaming] ( * as Text and input format as TextInputFormat). Files must be written to the * monitored directory by "moving" them from another location within the same * file system. File names starting with . are ignored. + * The text files must be encoded as UTF-8. + * * @param directory HDFS directory to monitor for new file */ def textFileStream(directory: String): DStream[String] = withNamedScope("text file stream") { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2ec907c8cfd5f..c3c13df651ccd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -34,7 +34,8 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.{JavaPairRDD, JavaSparkContext, JavaUtils, Optional} import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.function.{FlatMapFunction, Function => JFunction, + Function2 => JFunction2} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ @@ -562,9 +563,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying a flatmap function to the value of each key-value pairs in * 'this' DStream without changing the key. */ - def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairDStream[K, U] = { - import scala.collection.JavaConverters._ - def fn: (V) => Iterable[U] = (x: V) => f.apply(x).asScala + def flatMapValues[U](f: FlatMapFunction[V, U]): JavaPairDStream[K, U] = { + def fn: (V) => Iterator[U] = (x: V) => f.call(x).asScala implicit val cm: ClassTag[U] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] dstream.flatMapValues(fn) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 982e72cffbf3f..d4f03bedc7ed6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -21,6 +21,7 @@ import java.io.{Closeable, InputStream} import java.lang.{Boolean => JBoolean} import java.util.{List => JList, Map => JMap} +import scala.annotation.varargs import scala.collection.JavaConverters._ import scala.reflect.ClassTag @@ -36,7 +37,6 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ -import org.apache.spark.streaming.dstream.DStream import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.streaming.scheduler.StreamingListener @@ -207,6 +207,8 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { * as Text and input format as TextInputFormat). Files must be written to the * monitored directory by "moving" them from another location within the same * file system. File names starting with . are ignored. + * The text files must be encoded as UTF-8. + * * @param directory HDFS directory to monitor for new file */ def textFileStream(directory: String): JavaDStream[String] = { @@ -431,24 +433,23 @@ class JavaStreamingContext(val ssc: StreamingContext) extends Closeable { /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ - def union[T](first: JavaDStream[T], rest: JList[JavaDStream[T]]): JavaDStream[T] = { - val dstreams: Seq[DStream[T]] = (Seq(first) ++ rest.asScala).map(_.dstream) - implicit val cm: ClassTag[T] = first.classTag - ssc.union(dstreams)(cm) + @varargs + def union[T](jdstreams: JavaDStream[T]*): JavaDStream[T] = { + require(jdstreams.nonEmpty, "Union called on no streams") + implicit val cm: ClassTag[T] = jdstreams.head.classTag + ssc.union(jdstreams.map(_.dstream))(cm) } /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ - def union[K, V]( - first: JavaPairDStream[K, V], - rest: JList[JavaPairDStream[K, V]] - ): JavaPairDStream[K, V] = { - val dstreams: Seq[DStream[(K, V)]] = (Seq(first) ++ rest.asScala).map(_.dstream) - implicit val cm: ClassTag[(K, V)] = first.classTag - implicit val kcm: ClassTag[K] = first.kManifest - implicit val vcm: ClassTag[V] = first.vManifest - new JavaPairDStream[K, V](ssc.union(dstreams)(cm))(kcm, vcm) + @varargs + def union[K, V](jdstreams: JavaPairDStream[K, V]*): JavaPairDStream[K, V] = { + require(jdstreams.nonEmpty, "Union called on no streams") + implicit val cm: ClassTag[(K, V)] = jdstreams.head.classTag + implicit val kcm: ClassTag[K] = jdstreams.head.kManifest + implicit val vcm: ClassTag[V] = jdstreams.head.vManifest + new JavaPairDStream[K, V](ssc.union(jdstreams.map(_.dstream))(cm))(kcm, vcm) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 35243373daf9d..41374b5e370f8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -455,7 +455,7 @@ abstract class DStream[T: ClassTag] ( if (unpersistData) { logDebug(s"Unpersisting old RDDs: ${oldRDDs.values.map(_.id).mkString(", ")}") oldRDDs.values.foreach { rdd => - rdd.unpersist(false) + rdd.unpersist() // Explicitly remove blocks of BlockRDD rdd match { case b: BlockRDD[_] => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index e73837eb9602f..b35f7d97233e2 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -39,7 +39,6 @@ class DStreamCheckpointData[T: ClassTag](dstream: DStream[T]) // in that batch's checkpoint data @transient private var timeToOldestCheckpointFileTime = new HashMap[Time, Time] - @transient private var fileSystem: FileSystem = null protected[streaming] def currentCheckpointFiles = data.asInstanceOf[HashMap[Time, String]] /** @@ -80,6 +79,7 @@ class DStreamCheckpointData[T: ClassTag](dstream: DStream[T]) // even after master fails, as the checkpoint data of `time` does not refer to those files val filesToDelete = timeToCheckpointFile.filter(_._1 < lastCheckpointFileTime) logDebug("Files to delete:\n" + filesToDelete.mkString(",")) + var fileSystem: FileSystem = null filesToDelete.foreach { case (time, file) => try { @@ -87,9 +87,12 @@ class DStreamCheckpointData[T: ClassTag](dstream: DStream[T]) if (fileSystem == null) { fileSystem = path.getFileSystem(dstream.ssc.sparkContext.hadoopConfiguration) } - fileSystem.delete(path, true) + if (fileSystem.delete(path, true)) { + logInfo("Deleted checkpoint file '" + file + "' for time " + time) + } else { + logWarning(s"Error deleting old checkpoint file '$file' for time $time") + } timeToCheckpointFile -= time - logInfo("Deleted checkpoint file '" + file + "' for time " + time) } catch { case e: Exception => logWarning("Error deleting old checkpoint file '" + file + "' for time " + time, e) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index fe0f875525660..88bfc88714fa1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -46,7 +46,7 @@ class WindowedDStream[T: ClassTag]( def windowDuration: Duration = _windowDuration - override def dependencies: List[DStream[_]] = List(parent) + override def dependencies: List[DStream[T]] = List(parent) override def slideDuration: Duration = _slideDuration diff --git a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala index 844760ab61d2e..f677c492d561f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/rdd/WriteAheadLogBackedBlockRDD.scala @@ -136,7 +136,7 @@ class WriteAheadLogBackedBlockRDD[T: ClassTag]( // this dummy directory should not already exist otherwise the WAL will try to recover // past events from the directory and throw errors. val nonExistentDirectory = new File( - System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).getAbsolutePath + System.getProperty("java.io.tmpdir"), UUID.randomUUID().toString).toURI.toString writeAheadLog = WriteAheadLogUtils.createLogForReceiver( SparkEnv.get.conf, nonExistentDirectory, hadoopConf) dataRead = writeAheadLog.read(partition.walRecordHandle) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala index 8c3a7977beae3..a0b49e091eb08 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlock.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.receiver import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer -import scala.language.existentials /** Trait representing a received block */ private[streaming] sealed trait ReceivedBlock diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala index 80c07958b41f2..eb70232a7452e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceivedBlockHandler.scala @@ -19,7 +19,6 @@ package org.apache.spark.streaming.receiver import scala.concurrent.{ExecutionContext, Future} import scala.concurrent.duration._ -import scala.language.{existentials, postfixOps} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala index 8717555dea491..e85a3b9009c32 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManager.scala @@ -22,6 +22,7 @@ import scala.util.Random import org.apache.spark.{ExecutorAllocationClient, SparkConf} import org.apache.spark.internal.Logging +import org.apache.spark.internal.config.Streaming._ import org.apache.spark.streaming.util.RecurringTimer import org.apache.spark.util.{Clock, Utils} @@ -55,17 +56,12 @@ private[streaming] class ExecutorAllocationManager( batchDurationMs: Long, clock: Clock) extends StreamingListener with Logging { - import ExecutorAllocationManager._ - - private val scalingIntervalSecs = conf.getTimeAsSeconds( - SCALING_INTERVAL_KEY, - s"${SCALING_INTERVAL_DEFAULT_SECS}s") - private val scalingUpRatio = conf.getDouble(SCALING_UP_RATIO_KEY, SCALING_UP_RATIO_DEFAULT) - private val scalingDownRatio = conf.getDouble(SCALING_DOWN_RATIO_KEY, SCALING_DOWN_RATIO_DEFAULT) - private val minNumExecutors = conf.getInt( - MIN_EXECUTORS_KEY, - math.max(1, receiverTracker.numReceivers)) - private val maxNumExecutors = conf.getInt(MAX_EXECUTORS_KEY, Integer.MAX_VALUE) + private val scalingIntervalSecs = conf.get(STREAMING_DYN_ALLOCATION_SCALING_INTERVAL) + private val scalingUpRatio = conf.get(STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO) + private val scalingDownRatio = conf.get(STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO) + private val minNumExecutors = conf.get(STREAMING_DYN_ALLOCATION_MIN_EXECUTORS) + .getOrElse(math.max(1, receiverTracker.numReceivers())) + private val maxNumExecutors = conf.get(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS) private val timer = new RecurringTimer(clock, scalingIntervalSecs * 1000, _ => manageAllocation(), "streaming-executor-allocation-manager") @@ -150,34 +146,17 @@ private[streaming] class ExecutorAllocationManager( } private def validateSettings(): Unit = { - require( - scalingIntervalSecs > 0, - s"Config $SCALING_INTERVAL_KEY must be more than 0") - - require( - scalingUpRatio > 0, - s"Config $SCALING_UP_RATIO_KEY must be more than 0") - - require( - scalingDownRatio > 0, - s"Config $SCALING_DOWN_RATIO_KEY must be more than 0") - - require( - minNumExecutors > 0, - s"Config $MIN_EXECUTORS_KEY must be more than 0") - - require( - maxNumExecutors > 0, - s"$MAX_EXECUTORS_KEY must be more than 0") - require( scalingUpRatio > scalingDownRatio, - s"Config $SCALING_UP_RATIO_KEY must be more than config $SCALING_DOWN_RATIO_KEY") + s"Config ${STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.key} must be more than config " + + s"${STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.key}") - if (conf.contains(MIN_EXECUTORS_KEY) && conf.contains(MAX_EXECUTORS_KEY)) { + if (conf.contains(STREAMING_DYN_ALLOCATION_MIN_EXECUTORS.key) && + conf.contains(STREAMING_DYN_ALLOCATION_MAX_EXECUTORS.key)) { require( maxNumExecutors >= minNumExecutors, - s"Config $MAX_EXECUTORS_KEY must be more than config $MIN_EXECUTORS_KEY") + s"Config ${STREAMING_DYN_ALLOCATION_MAX_EXECUTORS.key} must be more than config " + + s"${STREAMING_DYN_ALLOCATION_MIN_EXECUTORS.key}") } } @@ -190,23 +169,9 @@ private[streaming] class ExecutorAllocationManager( } private[streaming] object ExecutorAllocationManager extends Logging { - val ENABLED_KEY = "spark.streaming.dynamicAllocation.enabled" - - val SCALING_INTERVAL_KEY = "spark.streaming.dynamicAllocation.scalingInterval" - val SCALING_INTERVAL_DEFAULT_SECS = 60 - - val SCALING_UP_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingUpRatio" - val SCALING_UP_RATIO_DEFAULT = 0.9 - - val SCALING_DOWN_RATIO_KEY = "spark.streaming.dynamicAllocation.scalingDownRatio" - val SCALING_DOWN_RATIO_DEFAULT = 0.3 - - val MIN_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.minExecutors" - - val MAX_EXECUTORS_KEY = "spark.streaming.dynamicAllocation.maxExecutors" def isDynamicAllocationEnabled(conf: SparkConf): Boolean = { - val streamingDynamicAllocationEnabled = conf.getBoolean(ENABLED_KEY, false) + val streamingDynamicAllocationEnabled = Utils.isStreamingDynamicAllocationEnabled(conf) if (Utils.isDynamicAllocationEnabled(conf) && streamingDynamicAllocationEnabled) { throw new IllegalArgumentException( """ @@ -215,8 +180,7 @@ private[streaming] object ExecutorAllocationManager extends Logging { |false to use Dynamic Allocation in streaming. """.stripMargin) } - val testing = conf.getBoolean("spark.streaming.dynamicAllocation.testing", false) - streamingDynamicAllocationEnabled && (!Utils.isLocalMaster(conf) || testing) + streamingDynamicAllocationEnabled } def createIfEnabled( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 8d83dc8a8fc04..709159879d7a9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -17,6 +17,8 @@ package org.apache.spark.streaming.scheduler +import java.util.concurrent.TimeUnit + import scala.util.{Failure, Success, Try} import org.apache.spark.internal.Logging @@ -49,11 +51,11 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.util.SystemClock") try { - Utils.classForName(clockClass).newInstance().asInstanceOf[Clock] + Utils.classForName[Clock](clockClass).getConstructor().newInstance() } catch { case e: ClassNotFoundException if clockClass.startsWith("org.apache.spark.streaming") => val newClockClass = clockClass.replace("org.apache.spark.streaming", "org.apache.spark") - Utils.classForName(newClockClass).newInstance().asInstanceOf[Clock] + Utils.classForName[Clock](newClockClass).getConstructor().newInstance() } } @@ -111,14 +113,15 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") - val timeWhenStopStarted = System.currentTimeMillis() + val timeWhenStopStarted = System.nanoTime() val stopTimeoutMs = conf.getTimeAsMs( "spark.streaming.gracefulStopTimeout", s"${10 * ssc.graph.batchDuration.milliseconds}ms") val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut: Boolean = { - val timedOut = (System.currentTimeMillis() - timeWhenStopStarted) > stopTimeoutMs + val diff = TimeUnit.NANOSECONDS.toMillis((System.nanoTime() - timeWhenStopStarted)) + val timedOut = diff > stopTimeoutMs if (timedOut) { logWarning("Timed out while stopping the job generator (timeout = " + stopTimeoutMs + ")") } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala index cf4324578ea87..a9763cfe04539 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceivedBlockTracker.scala @@ -111,8 +111,12 @@ private[streaming] class ReceivedBlockTracker( */ def allocateBlocksToBatch(batchTime: Time): Unit = synchronized { if (lastAllocatedBatchTime == null || batchTime > lastAllocatedBatchTime) { + // We explicitly create an ArrayBuffer here because at least as of Scala 2.11 and 2.12 + // a mutable.Queue fails serialization with a StackOverflow error if it has more than + // a few thousand elements. So we explicitly allocate a collection for serialization which + // we know doesn't have this issue. (See SPARK-26734). val streamIdToBlocks = streamIds.map { streamId => - (streamId, getReceivedBlockQueue(streamId).clone()) + (streamId, mutable.ArrayBuffer(getReceivedBlockQueue(streamId).clone(): _*)) }.toMap val allocatedBlocks = AllocatedBlocks(streamIdToBlocks) if (writeToLog(BatchAllocationEvent(batchTime, allocatedBlocks))) { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index c74ca1918a81d..551d376fbc1e7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,7 +21,6 @@ import java.util.concurrent.{CountDownLatch, TimeUnit} import scala.collection.mutable.HashMap import scala.concurrent.ExecutionContext -import scala.language.existentials import scala.util.{Failure, Success} import org.apache.spark._ @@ -255,9 +254,7 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false } } - def numReceivers(): Int = { - receiverInputStreams.size - } + def numReceivers(): Int = receiverInputStreams.length /** Register a receiver */ private def registerReceiver( @@ -516,14 +513,12 @@ class ReceiverTracker(ssc: StreamingContext, skipReceiverLaunch: Boolean = false context.reply(successful) case AddBlock(receivedBlockInfo) => if (WriteAheadLogUtils.isBatchingEnabled(ssc.conf, isDriver = true)) { - walBatchingThreadPool.execute(new Runnable { - override def run(): Unit = Utils.tryLogNonFatalError { - if (active) { - context.reply(addBlock(receivedBlockInfo)) - } else { - context.sendFailure( - new IllegalStateException("ReceiverTracker RpcEndpoint already shut down.")) - } + walBatchingThreadPool.execute(() => Utils.tryLogNonFatalError { + if (active) { + context.reply(addBlock(receivedBlockInfo)) + } else { + context.sendFailure( + new IllegalStateException("ReceiverTracker RpcEndpoint already shut down.")) } }) } else { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala index 884d21d0afdd3..3310f3b59a642 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/BatchPage.scala @@ -104,7 +104,7 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } } val lastFailureReason = - sparkJob.stageIds.sorted.reverse.flatMap(getStageData). + sparkJob.stageIds.sorted(Ordering.Int.reverse).flatMap(getStageData). dropWhile(_.failureReason == None).take(1). // get the first info that contains failure flatMap(info => info.failureReason).headOption.getOrElse("") val formattedDuration = duration.map(d => SparkUIUtils.formatDuration(d)).getOrElse("-") @@ -317,12 +317,10 @@ private[ui] class BatchPage(parent: StreamingTab) extends WebUIPage("batch") { } def render(request: HttpServletRequest): Seq[Node] = streamingListener.synchronized { - // stripXSS is called first to remove suspicious characters used in XSS attacks - val batchTime = - Option(SparkUIUtils.stripXSS(request.getParameter("id"))).map(id => Time(id.toLong)) + val batchTime = Option(request.getParameter("id")).map(id => Time(id.toLong)) .getOrElse { - throw new IllegalArgumentException(s"Missing id parameter") - } + throw new IllegalArgumentException(s"Missing id parameter") + } val formattedBatchTime = UIUtils.formatBatchTime(batchTime.milliseconds, streamingListener.batchDuration) 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 4ce661bc1144e..d16611f412034 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 @@ -156,7 +156,7 @@ private[ui] class StreamingPage(parent: StreamingTab) generateStatTable() ++ generateBatchListTables() } - SparkUIUtils.headerSparkPage(request, "Streaming Statistics", content, parent, Some(5000)) + SparkUIUtils.headerSparkPage(request, "Streaming Statistics", content, parent) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala index 71b86d16866ea..31e4c6b59a64a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/BatchedWriteAheadLog.scala @@ -135,18 +135,16 @@ private[util] class BatchedWriteAheadLog(val wrappedLog: WriteAheadLog, conf: Sp /** Start the actual log writer on a separate thread. */ private def startBatchedWriterThread(): Thread = { - val thread = new Thread(new Runnable { - override def run(): Unit = { - while (active.get()) { - try { - flushRecords() - } catch { - case NonFatal(e) => - logWarning("Encountered exception in Batched Writer Thread.", e) - } + val thread = new Thread(() => { + while (active.get()) { + try { + flushRecords() + } catch { + case NonFatal(e) => + logWarning("Encountered exception in Batched Writer Thread.", e) } - logInfo("BatchedWriteAheadLog Writer thread exiting.") } + logInfo("BatchedWriteAheadLog Writer thread exiting.") }, "BatchedWriteAheadLog Writer") thread.setDaemon(true) thread.start() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala index f0161e1465c29..21f3bbe40bfab 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/FileBasedWriteAheadLog.scala @@ -25,7 +25,6 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.collection.parallel.ExecutionContextTaskSupport import scala.concurrent.{Await, ExecutionContext, Future} -import scala.language.postfixOps import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path @@ -142,7 +141,7 @@ private[streaming] class FileBasedWriteAheadLog( CompletionIterator[ByteBuffer, Iterator[ByteBuffer]](reader, () => reader.close()) } if (!closeFileAfterWrite) { - logFilesToRead.iterator.map(readFile).flatten.asJava + logFilesToRead.iterator.flatMap(readFile).asJava } else { // For performance gains, it makes sense to parallelize the recovery if // closeFileAfterWrite = true @@ -190,7 +189,7 @@ private[streaming] class FileBasedWriteAheadLog( if (waitForCompletion) { import scala.concurrent.duration._ // scalastyle:off awaitready - Await.ready(f, 1 second) + Await.ready(f, 1.second) // scalastyle:on awaitready } } catch { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala index a6997359d64d2..8cb68b2be4ecf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/HdfsUtils.scala @@ -21,6 +21,8 @@ import java.io.{FileNotFoundException, IOException} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ +import org.apache.spark.deploy.SparkHadoopUtil + private[streaming] object HdfsUtils { def getOutputStream(path: String, conf: Configuration): FSDataOutputStream = { @@ -37,7 +39,8 @@ private[streaming] object HdfsUtils { throw new IllegalStateException("File exists and there is no append support!") } } else { - dfs.create(dfsPath) + // we dont' want to use hdfs erasure coding, as that lacks support for append and hflush + SparkHadoopUtil.createNonECFile(dfs, dfsPath) } } stream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala index 29cc1fa00ac0f..342f20f47a39e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala @@ -79,7 +79,7 @@ class RateLimitedOutputStream(out: OutputStream, desiredBytesPerSec: Int) } else { // Calculate how much time we should sleep to bring ourselves to the desired rate. val targetTimeInMillis = bytesWrittenSinceSync * 1000 / desiredBytesPerSec - val elapsedTimeInMillis = elapsedNanosecs / 1000000 + val elapsedTimeInMillis = NANOSECONDS.toMillis(elapsedNanosecs) val sleepTimeInMillis = targetTimeInMillis - elapsedTimeInMillis if (sleepTimeInMillis > 0) { logTrace("Natural rate is " + rate + " per second but desired rate is " + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala index 89524cd84ff32..618c036377aee 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/StateMap.scala @@ -70,7 +70,7 @@ private[streaming] object StateMap { /** Implementation of StateMap interface representing an empty map */ private[streaming] class EmptyStateMap[K, S] extends StateMap[K, S] { override def put(key: K, session: S, updateTime: Long): Unit = { - throw new NotImplementedError("put() should not be called on an EmptyStateMap") + throw new UnsupportedOperationException("put() should not be called on an EmptyStateMap") } override def get(key: K): Option[S] = None override def getByTime(threshUpdatedTime: Long): Iterator[(K, S, Long)] = Iterator.empty diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala index 7542e2f5ecf24..b0a4c98fc9e57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/WriteAheadLogUtils.scala @@ -132,8 +132,7 @@ private[streaming] object WriteAheadLogUtils extends Logging { } val wal = classNameOption.map { className => try { - instantiateClass( - Utils.classForName(className).asInstanceOf[Class[_ <: WriteAheadLog]], sparkConf) + instantiateClass(Utils.classForName[WriteAheadLog](className), sparkConf) } catch { case NonFatal(e) => throw new SparkException(s"Could not create a write ahead log of class $className", e) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java index 91560472446a9..e9b35ac89f862 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaReceiverAPISuite.java @@ -39,6 +39,7 @@ import java.net.ConnectException; import java.net.Socket; import java.nio.charset.StandardCharsets; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicLong; public class JavaReceiverAPISuite implements Serializable { @@ -71,15 +72,15 @@ public void testReceiver() throws InterruptedException { }); ssc.start(); - long startTime = System.currentTimeMillis(); - long timeout = 10000; + long startTimeNs = System.nanoTime(); + long timeout = TimeUnit.SECONDS.toNanos(10); Thread.sleep(200); for (int i = 0; i < 6; i++) { server.send(i + "\n"); // \n to make sure these are separate lines Thread.sleep(100); } - while (dataCounter.get() == 0 && System.currentTimeMillis() - startTime < timeout) { + while (dataCounter.get() == 0 && System.nanoTime() - startTimeNs < timeout) { Thread.sleep(100); } ssc.stop(); diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java index 90d1f8c5035b3..b154f0e3ac455 100644 --- a/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java +++ b/streaming/src/test/java/test/org/apache/spark/streaming/Java8APISuite.java @@ -841,7 +841,7 @@ public void testFlatMapValues() { JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream flatMapped = - pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2")); + pairStream.flatMapValues(in -> Arrays.asList(in + "1", in + "2").iterator()); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 2, 2); Assert.assertEquals(expected, result); diff --git a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java index 6c86cacec8279..c7cde5674f547 100644 --- a/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/test/org/apache/spark/streaming/JavaAPISuite.java @@ -1355,7 +1355,7 @@ public void testFlatMapValues() { List out = new ArrayList<>(); out.add(in + "1"); out.add(in + "2"); - return out; + return out.iterator(); }); JavaTestUtils.attachTestOutputStream(flatMapped); diff --git a/streaming/src/test/resources/log4j.properties b/streaming/src/test/resources/log4j.properties index fd51f8faf56b9..afcd5915b7481 100644 --- a/streaming/src/test/resources/log4j.properties +++ b/streaming/src/test/resources/log4j.properties @@ -24,5 +24,5 @@ log4j.appender.file.layout=org.apache.log4j.PatternLayout log4j.appender.file.layout.ConversionPattern=%d{yy/MM/dd HH:mm:ss.SSS} %t %p %c{1}: %m%n # Ignore messages below warning level from Jetty, because it's a bit verbose -log4j.logger.org.spark_project.jetty=WARN +log4j.logger.org.sparkproject.jetty=WARN diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 0a764f61c0cd9..287a43ac689ed 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming import java.util.concurrent.ConcurrentLinkedQueue import scala.collection.mutable -import scala.language.existentials import scala.reflect.ClassTag import org.scalatest.concurrent.Eventually.eventually @@ -656,12 +655,12 @@ class BasicOperationsSuite extends TestSuiteBase { runCleanupTest( conf, - operation _, + operation, numExpectedOutput = cleanupTestInput.size / 2, rememberDuration = Seconds(3)) { operatedStream => eventually(eventuallyTimeout) { - val windowedStream2 = operatedStream.asInstanceOf[WindowedDStream[_]] - val windowedStream1 = windowedStream2.dependencies.head.asInstanceOf[WindowedDStream[_]] + val windowedStream2 = operatedStream.asInstanceOf[WindowedDStream[Int]] + val windowedStream1 = windowedStream2.dependencies.head.asInstanceOf[WindowedDStream[Int]] val mappedStream = windowedStream1.dependencies.head // Checkpoint remember durations diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 19b621f11759d..55fdd4c82ac75 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -162,12 +162,12 @@ trait DStreamCheckpointTester { self: SparkFunSuite => val outputStream = getTestOutputStream[V](ssc.graph.getOutputStreams()) - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { ssc.awaitTerminationOrTimeout(10) assert(batchCounter.getLastCompletedBatchTime === targetBatchTime) } - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { val checkpointFilesOfLatestTime = Checkpoint.getCheckpointFiles(checkpointDir).filter { _.getName.contains(clock.getTimeMillis.toString) } @@ -808,7 +808,8 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester // visible to mutableURLClassLoader val loader = new MutableURLClassLoader( Array(jar), appClassLoader) - assert(loader.loadClass("testClz").newInstance().toString == "testStringValue") + assert(loader.loadClass("testClz").getConstructor().newInstance().toString === + "testStringValue") // create and serialize Array[testClz] // scalastyle:off classforname @@ -911,8 +912,8 @@ class CheckpointSuite extends TestSuiteBase with DStreamCheckpointTester } ssc.start() batchCounter.waitUntilBatchesCompleted(1, 10000) - assert(shouldCheckpointAllMarkedRDDs === true) - assert(rddsCheckpointed === true) + assert(shouldCheckpointAllMarkedRDDs) + assert(rddsCheckpointed) } /** diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 1cf21e8a28033..035ed4aa51bb7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -31,6 +31,7 @@ import org.apache.commons.io.IOUtils import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat +import org.scalatest.Assertions import org.scalatest.BeforeAndAfter import org.scalatest.concurrent.Eventually._ @@ -292,8 +293,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { val textPath = new Path(generatedSubDir, "renamed.txt") write(textPath, "renamed\n") val now = clock.getTimeMillis() - val modTime = now + durationMs / 2 - fs.setTimes(textPath, modTime, modTime) + fs.setTimes(textPath, now, now) val textFilestatus = fs.getFileStatus(existingFile) assert(textFilestatus.getModificationTime < now + durationMs) @@ -334,9 +334,9 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Let the data from the receiver be received val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() while ((!MultiThreadTestReceiver.haveAllThreadsFinished || output.sum < numTotalRecords) && - System.currentTimeMillis() - startTime < 5000) { + System.nanoTime() - startTimeNs < TimeUnit.SECONDS.toNanos(5)) { Thread.sleep(100) clock.advance(batchDuration.milliseconds) } @@ -532,7 +532,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { /** This is a server to test the network input stream */ -class TestServer(portToBind: Int = 0) extends Logging { +class TestServer(portToBind: Int = 0) extends Logging with Assertions { val queue = new ArrayBlockingQueue[String](100) @@ -592,7 +592,7 @@ class TestServer(portToBind: Int = 0) extends Logging { servingThread.start() if (!waitForStart(10000)) { stop() - throw new AssertionError("Timeout: TestServer cannot start in 10 seconds") + fail("Timeout: TestServer cannot start in 10 seconds") } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala index fff2d6fbace3a..cf8dd10571f47 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/MasterFailureTest.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import java.io.{File, IOException} import java.nio.charset.StandardCharsets import java.util.UUID +import java.util.concurrent.TimeUnit import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer @@ -234,12 +235,12 @@ object MasterFailureTest extends Logging { System.clearProperty("spark.streaming.clock") System.clearProperty("spark.driver.port") ssc.start() - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() while (!killed && !isLastOutputGenerated && !isTimedOut) { Thread.sleep(100) - timeRan = System.currentTimeMillis() - startTime + timeRan = System.nanoTime() - startTimeNs isLastOutputGenerated = (output.nonEmpty && output.last == lastExpectedOutput) - isTimedOut = (timeRan + totalTimeRan > maxTimeToRun) + isTimedOut = (timeRan + totalTimeRan > TimeUnit.MILLISECONDS.toNanos(maxTimeToRun)) } } catch { case e: Exception => logError("Error running streaming context", e) @@ -265,7 +266,7 @@ object MasterFailureTest extends Logging { logInfo("New output = " + output.toSeq) logInfo("Merged output = " + mergedOutput) logInfo("Time ran = " + timeRan) - logInfo("Total time ran = " + totalTimeRan) + logInfo("Total time ran = " + TimeUnit.NANOSECONDS.toMillis(totalTimeRan)) if (!isLastOutputGenerated && !isTimedOut) { val sleepTime = Random.nextInt(batchDuration.milliseconds.toInt * 10) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala index fe65353b9d502..c8f424af9af01 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockHandlerSuite.scala @@ -22,7 +22,6 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ -import scala.language.postfixOps import scala.reflect.ClassTag import org.apache.hadoop.conf.Configuration @@ -33,7 +32,7 @@ import org.apache.spark._ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.internal.Logging import org.apache.spark.internal.config._ -import org.apache.spark.memory.StaticMemoryManager +import org.apache.spark.memory.UnifiedMemoryManager import org.apache.spark.network.netty.NettyBlockTransferService import org.apache.spark.rpc.RpcEnv import org.apache.spark.scheduler.LiveListenerBus @@ -90,7 +89,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) blockManagerMaster = new BlockManagerMaster(rpcEnv.setupEndpoint("blockmanager", new BlockManagerMasterEndpoint(rpcEnv, true, conf, - new LiveListenerBus(conf))), conf, true) + new LiveListenerBus(conf), None)), conf, true) storageLevel = StorageLevel.MEMORY_ONLY_SER blockManager = createBlockManager(blockManagerSize, conf) @@ -199,7 +198,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) val cleanupThreshTime = 3000L handler.cleanupOldBlocks(cleanupThreshTime) - eventually(timeout(10000 millis), interval(10 millis)) { + eventually(timeout(10.seconds), interval(10.milliseconds)) { getWriteAheadLogFiles().size should be < preCleanupLogFiles.size } } @@ -214,9 +213,7 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) test("Test Block - isFullyConsumed") { val sparkConf = new SparkConf().set("spark.app.id", "streaming-test") - sparkConf.set("spark.storage.unrollMemoryThreshold", "512") - // spark.storage.unrollFraction set to 0.4 for BlockManager - sparkConf.set("spark.storage.unrollFraction", "0.4") + sparkConf.set(STORAGE_UNROLL_MEMORY_THRESHOLD, 512L) sparkConf.set(IO_ENCRYPTION_ENABLED, enableEncryption) // Block Manager with 12000 * 0.4 = 4800 bytes of free space for unroll @@ -282,10 +279,10 @@ abstract class BaseReceivedBlockHandlerSuite(enableEncryption: Boolean) maxMem: Long, conf: SparkConf, name: String = SparkContext.DRIVER_IDENTIFIER): BlockManager = { - val memManager = new StaticMemoryManager(conf, Long.MaxValue, maxMem, numCores = 1) + val memManager = new UnifiedMemoryManager(conf, maxMem, maxMem / 2, 1) val transfer = new NettyBlockTransferService(conf, securityMgr, "localhost", "localhost", 0, 1) val blockManager = new BlockManager(name, rpcEnv, blockManagerMaster, serializerManager, conf, - memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, 0) + memManager, mapOutputTracker, shuffleManager, transfer, securityMgr, None) memManager.setMemoryStore(blockManager.memoryStore) blockManager.initialize("app-id") blockManagerBuffer += blockManager diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala index fd7e00b1de25f..0b15f00eba499 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceivedBlockTrackerSuite.scala @@ -22,11 +22,11 @@ import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer import scala.concurrent.duration._ -import scala.language.{implicitConversions, postfixOps} +import scala.language.implicitConversions import scala.util.Random import org.apache.hadoop.conf.Configuration -import org.mockito.Matchers.any +import org.mockito.ArgumentMatchers.any import org.mockito.Mockito.{doThrow, reset, spy} import org.scalatest.{BeforeAndAfter, Matchers} import org.scalatest.concurrent.Eventually._ @@ -96,6 +96,27 @@ class ReceivedBlockTrackerSuite receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual blockInfos } + test("block addition, and block to batch allocation with many blocks") { + val receivedBlockTracker = createTracker() + receivedBlockTracker.isWriteAheadLogEnabled should be (true) + + val blockInfos = generateBlockInfos(100000) + blockInfos.map(receivedBlockTracker.addBlock) + receivedBlockTracker.allocateBlocksToBatch(1) + + receivedBlockTracker.getUnallocatedBlocks(streamId) shouldEqual Seq.empty + receivedBlockTracker.hasUnallocatedReceivedBlocks should be (false) + receivedBlockTracker.getBlocksOfBatch(1) shouldEqual Map(streamId -> blockInfos) + receivedBlockTracker.getBlocksOfBatchAndStream(1, streamId) shouldEqual blockInfos + + val expectedWrittenData1 = blockInfos.map(BlockAdditionEvent) :+ + BatchAllocationEvent(1, AllocatedBlocks(Map(streamId -> blockInfos))) + getWrittenLogData() shouldEqual expectedWrittenData1 + getWriteAheadLogFiles() should have size 1 + + receivedBlockTracker.stop() + } + test("recovery with write ahead logs should remove only allocated blocks from received queue") { val manualClock = new ManualClock val batchTime = manualClock.getTimeMillis() @@ -255,7 +276,7 @@ class ReceivedBlockTrackerSuite getWrittenLogData(getWriteAheadLogFiles().last) should contain(createBatchCleanup(batchTime1)) // Verify that at least one log file gets deleted - eventually(timeout(10 seconds), interval(10 millisecond)) { + eventually(timeout(10.seconds), interval(10.millisecond)) { getWriteAheadLogFiles() should not contain oldestLogFile } printLogFiles("After clean") @@ -362,8 +383,8 @@ class ReceivedBlockTrackerSuite } /** Generate blocks infos using random ids */ - def generateBlockInfos(): Seq[ReceivedBlockInfo] = { - List.fill(5)(ReceivedBlockInfo(streamId, Some(0L), None, + def generateBlockInfos(blockCount: Int = 5): Seq[ReceivedBlockInfo] = { + List.fill(blockCount)(ReceivedBlockInfo(streamId, Some(0L), None, BlockManagerBasedStoreResult(StreamBlockId(streamId, math.abs(Random.nextInt)), Some(0L)))) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala index fc6218a33f741..6b664b7a7dfd4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/ReceiverSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming import java.io.File import java.nio.ByteBuffer -import java.util.concurrent.Semaphore +import java.util.concurrent.{Semaphore, TimeUnit} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer @@ -29,6 +29,7 @@ import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ import org.apache.spark.SparkConf +import org.apache.spark.internal.config.UI._ import org.apache.spark.storage.StorageLevel import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.receiver._ @@ -63,7 +64,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { // Verify that the receiver intercept[Exception] { - failAfter(200 millis) { + failAfter(200.milliseconds) { executingThread.join() } } @@ -77,7 +78,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { assert(receiver.isStarted) assert(!receiver.isStopped()) assert(receiver.otherThread.isAlive) - eventually(timeout(100 millis), interval(10 millis)) { + eventually(timeout(100.milliseconds), interval(10.milliseconds)) { assert(receiver.receiving) } @@ -106,12 +107,12 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { // Verify restarting actually stops and starts the receiver receiver.restart("restarting", null, 600) - eventually(timeout(300 millis), interval(10 millis)) { + eventually(timeout(300.milliseconds), interval(10.milliseconds)) { // receiver will be stopped async assert(receiver.isStopped) assert(receiver.onStopCalled) } - eventually(timeout(1000 millis), interval(10 millis)) { + eventually(timeout(1.second), interval(10.milliseconds)) { // receiver will be started async assert(receiver.onStartCalled) assert(executor.isReceiverStarted) @@ -121,7 +122,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { } // Verify that stopping actually stops the thread - failAfter(100 millis) { + failAfter(1.second) { receiver.stop("test") assert(receiver.isStopped) assert(!receiver.otherThread.isAlive) @@ -146,10 +147,10 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { val generatedData = new ArrayBuffer[Int] // Generate blocks - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() blockGenerator.start() var count = 0 - while(System.currentTimeMillis - startTime < waitTime) { + while(System.nanoTime() - startTimeNs < TimeUnit.MILLISECONDS.toNanos(waitTime)) { blockGenerator.addData(count) generatedData += count count += 1 @@ -158,7 +159,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { val recordedBlocks = blockGeneratorListener.arrayBuffers val recordedData = recordedBlocks.flatten - assert(blockGeneratorListener.arrayBuffers.size > 0, "No blocks received") + assert(blockGeneratorListener.arrayBuffers.nonEmpty, "No blocks received") assert(recordedData.toSet === generatedData.toSet, "Received data not same") // recordedData size should be close to the expected rate; use an error margin proportional to @@ -200,7 +201,7 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { val sparkConf = new SparkConf() .setMaster("local[4]") // must be at least 3 as we are going to start 2 receivers .setAppName(framework) - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) .set("spark.streaming.receiver.writeAheadLog.enable", "true") .set("spark.streaming.receiver.writeAheadLog.rollingIntervalSecs", "1") val batchDuration = Milliseconds(500) @@ -244,15 +245,15 @@ class ReceiverSuite extends TestSuiteBase with TimeLimits with Serializable { // Run until sufficient WAL files have been generated and // the first WAL files has been deleted - eventually(timeout(20 seconds), interval(batchDuration.milliseconds millis)) { + eventually(timeout(20.seconds), interval(batchDuration.milliseconds.millis)) { val (logFiles1, logFiles2) = getBothCurrentLogFiles() allLogFiles1 ++= logFiles1 allLogFiles2 ++= logFiles2 - if (allLogFiles1.size > 0) { - assert(!logFiles1.contains(allLogFiles1.toSeq.sorted.head)) + if (allLogFiles1.nonEmpty) { + assert(!logFiles1.contains(allLogFiles1.toSeq.min)) } - if (allLogFiles2.size > 0) { - assert(!logFiles2.contains(allLogFiles2.toSeq.sorted.head)) + if (allLogFiles2.nonEmpty) { + assert(!logFiles2.contains(allLogFiles2.toSeq.min)) } assert(allLogFiles1.size >= 7) assert(allLogFiles2.size >= 7) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala index 484f3733e8423..d0c56ecffcb8a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StateMapSuite.scala @@ -35,7 +35,7 @@ class StateMapSuite extends SparkFunSuite { test("EmptyStateMap") { val map = new EmptyStateMap[Int, Int] - intercept[scala.NotImplementedError] { + intercept[UnsupportedOperationException] { map.put(1, 1, 1) } assert(map.get(1) === None) @@ -165,7 +165,7 @@ class StateMapSuite extends SparkFunSuite { map = map.copy().asInstanceOf[OpenHashMapBasedStateMap[Int, Int]] } assert(map.deltaChainLength > deltaChainThreshold) - assert(map.shouldCompact === true) + assert(map.shouldCompact) val deser_map = testSerialization(map, "Deserialized + compacted map not same as original map") assert(deser_map.deltaChainLength < deltaChainThreshold) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 52c8959351fe7..5cda6f9925455 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -211,8 +211,8 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // Local props set after start should be ignored ssc.sc.setLocalProperty("customPropKey", "value2") - eventually(timeout(10 seconds), interval(10 milliseconds)) { - assert(allFound === true) + eventually(timeout(10.seconds), interval(10.milliseconds)) { + assert(allFound) } // Verify streaming jobs have expected thread-local properties @@ -342,7 +342,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL input.foreachRDD(_ => {}) ssc.start() // Call `ssc.stop` at once so that it's possible that the receiver will miss "StopReceiver" - failAfter(30000 millis) { + failAfter(30.seconds) { ssc.stop(stopSparkContext = true, stopGracefully = true) } } @@ -398,18 +398,18 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL inputStream.map(x => x).register() // test whether start() blocks indefinitely or not - failAfter(2000 millis) { + failAfter(2.seconds) { ssc.start() } // test whether awaitTermination() exits after give amount of time - failAfter(1000 millis) { + failAfter(1.second) { ssc.awaitTerminationOrTimeout(500) } // test whether awaitTermination() does not exit if not time is given val exception = intercept[Exception] { - failAfter(1000 millis) { + failAfter(1.second) { ssc.awaitTermination() throw new Exception("Did not wait for stop") } @@ -418,7 +418,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL var t: Thread = null // test whether wait exits if context is stopped - failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown + failAfter(10.seconds) { // 10 seconds because spark takes a long time to shutdown t = new Thread() { override def run() { Thread.sleep(500) @@ -439,7 +439,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL val inputStream = addInputStream(ssc) inputStream.map(x => x).register() - failAfter(10000 millis) { + failAfter(10.seconds) { ssc.start() ssc.stop() ssc.awaitTermination() @@ -479,13 +479,13 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL ssc.start() // test whether awaitTerminationOrTimeout() return false after give amount of time - failAfter(1000 millis) { + failAfter(1.second) { assert(ssc.awaitTerminationOrTimeout(500) === false) } var t: Thread = null // test whether awaitTerminationOrTimeout() return true if context is stopped - failAfter(10000 millis) { // 10 seconds because spark takes a long time to shutdown + failAfter(10.seconds) { // 10 seconds because spark takes a long time to shutdown t = new Thread() { override def run() { Thread.sleep(500) @@ -493,7 +493,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL } } t.start() - assert(ssc.awaitTerminationOrTimeout(10000) === true) + assert(ssc.awaitTerminationOrTimeout(10000)) } // SparkContext.stop will set SparkEnv.env to null. We need to make sure SparkContext is stopped // before running the next test. Otherwise, it's possible that we set SparkEnv.env to null after @@ -528,7 +528,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getOrCreate should create new context with empty path testGetOrCreate { - ssc = StreamingContext.getOrCreate(emptyPath, creatingFunction _) + ssc = StreamingContext.getOrCreate(emptyPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(newContextCreated, "new context not created") } @@ -537,19 +537,19 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getOrCreate should throw exception with fake checkpoint file and createOnError = false intercept[Exception] { - ssc = StreamingContext.getOrCreate(corruptedCheckpointPath, creatingFunction _) + ssc = StreamingContext.getOrCreate(corruptedCheckpointPath, () => creatingFunction()) } // getOrCreate should throw exception with fake checkpoint file intercept[Exception] { ssc = StreamingContext.getOrCreate( - corruptedCheckpointPath, creatingFunction _, createOnError = false) + corruptedCheckpointPath, () => creatingFunction(), createOnError = false) } // getOrCreate should create new context with fake checkpoint file and createOnError = true testGetOrCreate { ssc = StreamingContext.getOrCreate( - corruptedCheckpointPath, creatingFunction _, createOnError = true) + corruptedCheckpointPath, () => creatingFunction(), createOnError = true) assert(ssc != null, "no context created") assert(newContextCreated, "new context not created") } @@ -558,7 +558,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getOrCreate should recover context with checkpoint path, and recover old configuration testGetOrCreate { - ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _) + ssc = StreamingContext.getOrCreate(checkpointPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(!newContextCreated, "old context not recovered") assert(ssc.conf.get("someKey") === "someValue", "checkpointed config not recovered") @@ -567,7 +567,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL // getOrCreate should recover StreamingContext with existing SparkContext testGetOrCreate { sc = new SparkContext(conf) - ssc = StreamingContext.getOrCreate(checkpointPath, creatingFunction _) + ssc = StreamingContext.getOrCreate(checkpointPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(!newContextCreated, "old context not recovered") assert(!ssc.conf.contains("someKey"), "checkpointed config unexpectedly recovered") @@ -605,7 +605,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL sc = new SparkContext(conf) ssc = StreamingContext.getActiveOrCreate(creatingFunc _) assert(ssc != null, "no context created") - assert(newContextCreated === true, "new context not created") + assert(newContextCreated, "new context not created") assert(StreamingContext.getActive().isEmpty, "new initialized context returned before starting") ssc.start() @@ -669,41 +669,41 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL conf.clone.set("spark.streaming.clock", "org.apache.spark.util.ManualClock"), batchDuration) addInputStream(ssc).register() ssc.start() - val returnedSsc = StreamingContext.getActiveOrCreate(checkpointPath, creatingFunction _) + val returnedSsc = StreamingContext.getActiveOrCreate(checkpointPath, () => creatingFunction()) assert(!newContextCreated, "new context created instead of returning") assert(returnedSsc.eq(ssc), "returned context is not the activated context") } // getActiveOrCreate should create new context with empty path testGetActiveOrCreate { - ssc = StreamingContext.getActiveOrCreate(emptyPath, creatingFunction _) + ssc = StreamingContext.getActiveOrCreate(emptyPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(newContextCreated, "new context not created") } // getActiveOrCreate should throw exception with fake checkpoint file and createOnError = false intercept[Exception] { - ssc = StreamingContext.getOrCreate(corruptedCheckpointPath, creatingFunction _) + ssc = StreamingContext.getOrCreate(corruptedCheckpointPath, () => creatingFunction()) } // getActiveOrCreate should throw exception with fake checkpoint file intercept[Exception] { ssc = StreamingContext.getActiveOrCreate( - corruptedCheckpointPath, creatingFunction _, createOnError = false) + corruptedCheckpointPath, () => creatingFunction(), createOnError = false) } // getActiveOrCreate should create new context with fake // checkpoint file and createOnError = true testGetActiveOrCreate { ssc = StreamingContext.getActiveOrCreate( - corruptedCheckpointPath, creatingFunction _, createOnError = true) + corruptedCheckpointPath, () => creatingFunction(), createOnError = true) assert(ssc != null, "no context created") assert(newContextCreated, "new context not created") } // getActiveOrCreate should recover context with checkpoint path, and recover old configuration testGetActiveOrCreate { - ssc = StreamingContext.getActiveOrCreate(checkpointPath, creatingFunction _) + ssc = StreamingContext.getActiveOrCreate(checkpointPath, () => creatingFunction()) assert(ssc != null, "no context created") assert(!newContextCreated, "old context not recovered") assert(ssc.conf.get("someKey") === "someValue") @@ -781,14 +781,14 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL _ssc.queueStream[Int](Queue(rdd)).register() _ssc } - ssc = StreamingContext.getOrCreate(checkpointDirectory, creatingFunction _) + ssc = StreamingContext.getOrCreate(checkpointDirectory, () => creatingFunction()) ssc.start() - eventually(timeout(10000 millis)) { + eventually(timeout(10.seconds)) { assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) } ssc.stop() val e = intercept[SparkException] { - ssc = StreamingContext.getOrCreate(checkpointDirectory, creatingFunction _) + ssc = StreamingContext.getOrCreate(checkpointDirectory, () => creatingFunction()) } // StreamingContext.validate changes the message, so use "contains" here assert(e.getCause.getMessage.contains("queueStream doesn't support checkpointing. " + @@ -855,7 +855,7 @@ class StreamingContextSuite extends SparkFunSuite with BeforeAndAfter with TimeL ssc.textFileStream(testDirectory).foreachRDD { rdd => rdd.count() } ssc.start() try { - eventually(timeout(30000 millis)) { + eventually(timeout(30.seconds)) { assert(Checkpoint.getCheckpointFiles(checkpointDirectory).size > 1) } } finally { @@ -967,7 +967,7 @@ package object testPackage extends Assertions { } ssc.start() - eventually(timeout(10000 millis), interval(10 millis)) { + eventually(timeout(10.seconds), interval(10.milliseconds)) { assert(rddGenerated && rddCreationSiteCorrect, "RDD creation site was not correct") assert(rddGenerated && foreachCallSiteCorrect, "Call site in foreachRDD was not correct") } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 0f957a1b55706..62fd43302b9d7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -130,7 +130,7 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { ssc.start() try { - eventually(timeout(30 seconds), interval(20 millis)) { + eventually(timeout(30.seconds), interval(20.milliseconds)) { collector.startedReceiverStreamIds.size should equal (1) collector.startedReceiverStreamIds.peek() should equal (0) collector.stoppedReceiverStreamIds.size should equal (1) @@ -157,7 +157,7 @@ class StreamingListenerSuite extends TestSuiteBase with Matchers { ssc.start() try { - eventually(timeout(30 seconds), interval(20 millis)) { + eventually(timeout(30.seconds), interval(20.milliseconds)) { collector.startedOutputOperationIds.asScala.take(3) should be (Seq(0, 1, 2)) collector.completedOutputOperationIds.asScala.take(3) should be (Seq(0, 1, 2)) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index ada494eb897f3..f2ae77896a5d3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -18,7 +18,7 @@ package org.apache.spark.streaming import java.io.{File, IOException, ObjectInputStream} -import java.util.concurrent.ConcurrentLinkedQueue +import java.util.concurrent.{ConcurrentLinkedQueue, TimeUnit} import scala.collection.JavaConverters._ import scala.language.implicitConversions @@ -423,13 +423,13 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { logInfo("Manual clock after advancing = " + clock.getTimeMillis()) // Wait until expected number of output items have been generated - val startTime = System.currentTimeMillis() + val startTimeNs = System.nanoTime() while (output.size < numExpectedOutput && - System.currentTimeMillis() - startTime < maxWaitTimeMillis) { + TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) < maxWaitTimeMillis) { logInfo("output.size = " + output.size + ", numExpectedOutput = " + numExpectedOutput) ssc.awaitTerminationOrTimeout(50) } - val timeTaken = System.currentTimeMillis() - startTime + val timeTaken = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNs) logInfo("Output generated in " + timeTaken + " milliseconds") output.asScala.foreach(x => logInfo("[" + x.mkString(",") + "]")) assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") @@ -557,16 +557,4 @@ trait TestSuiteBase extends SparkFunSuite with BeforeAndAfter with Logging { verifyOutput[W](output.toSeq, expectedOutput, useSet) } } - - /** - * Creates a temporary directory, which is then passed to `f` and will be deleted after `f` - * returns. - * (originally from `SqlTestUtils`.) - * @todo Probably this method should be moved to a more general place - */ - protected def withTempDir(f: File => Unit): Unit = { - val dir = Utils.createTempDir().getCanonicalFile - try f(dir) finally Utils.deleteRecursively(dir) - } - } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala index 957feca2e552d..483a7519873e9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISeleniumSuite.scala @@ -27,6 +27,7 @@ import org.scalatest.selenium.WebBrowser import org.scalatest.time.SpanSugar._ import org.apache.spark._ +import org.apache.spark.internal.config.UI.UI_ENABLED import org.apache.spark.ui.SparkUICssErrorHandler /** @@ -61,7 +62,7 @@ class UISeleniumSuite val conf = new SparkConf() .setMaster("local") .setAppName("test") - .set("spark.ui.enabled", "true") + .set(UI_ENABLED, true) val ssc = new StreamingContext(conf, Seconds(1)) assert(ssc.sc.ui.isDefined, "Spark UI is not started!") ssc @@ -96,12 +97,12 @@ class UISeleniumSuite val sparkUI = ssc.sparkContext.ui.get - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should not be (None) } - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { // check whether streaming page exists go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq @@ -195,12 +196,12 @@ class UISeleniumSuite ssc.stop(false) - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/")) find(cssSelector( """ul li a[href*="streaming"]""")) should be(None) } - eventually(timeout(10 seconds), interval(50 milliseconds)) { + eventually(timeout(10.seconds), interval(50.milliseconds)) { go to (sparkUI.webUrl.stripSuffix("/") + "/streaming") val h3Text = findAll(cssSelector("h3")).map(_.text).toSeq h3Text should not contain("Streaming Statistics") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala index 9b6bc71c7a5b5..58ce3a93251a9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/rdd/MapWithStateRDDSuite.scala @@ -117,7 +117,7 @@ class MapWithStateRDDSuite extends SparkFunSuite with RDDCheckpointTester with B state.remove() None case None => - assert(state.isTimingOut() === true, "State is not timing out when data = None") + assert(state.isTimingOut(), "State is not timing out when data = None") timingOutStates += state.get() None case _ => @@ -153,9 +153,9 @@ class MapWithStateRDDSuite extends SparkFunSuite with RDDCheckpointTester with B // Data present, function should be called irrespective of whether state exists assertRecordUpdate(initStates = Seq(0), data = Seq("noop"), expectedStates = Seq((0, initialTime))) - assert(functionCalled === true) + assert(functionCalled) assertRecordUpdate(initStates = None, data = Some("noop"), expectedStates = None) - assert(functionCalled === true) + assert(functionCalled) // Function called with right state data assertRecordUpdate(initStates = None, data = Seq("get-state"), diff --git a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala index 580f831548cd5..4c0dd0d258c53 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/receiver/BlockGeneratorSuite.scala @@ -66,7 +66,7 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim // Verify start marks the generator active, but does not call the callbacks blockGenerator.start() - assert(blockGenerator.isActive() === true, "block generator active after start()") + assert(blockGenerator.isActive(), "block generator active after start()") assert(blockGenerator.isStopped() === false, "block generator stopped after start()") withClue("callbacks called before adding data") { assert(listener.onAddDataCalled === false) @@ -84,9 +84,9 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim } clock.advance(blockIntervalMs) // advance clock to generate blocks withClue("blocks not generated or pushed") { - eventually(timeout(1 second)) { - assert(listener.onGenerateBlockCalled === true) - assert(listener.onPushBlockCalled === true) + eventually(timeout(1.second)) { + assert(listener.onGenerateBlockCalled) + assert(listener.onPushBlockCalled) } } listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs (data1) @@ -96,11 +96,11 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim val data2 = 11 to 20 val metadata2 = data2.map { _.toString } data2.zip(metadata2).foreach { case (d, m) => blockGenerator.addDataWithCallback(d, m) } - assert(listener.onAddDataCalled === true) + assert(listener.onAddDataCalled) listener.addedData.asScala.toSeq should contain theSameElementsInOrderAs (data2) listener.addedMetadata.asScala.toSeq should contain theSameElementsInOrderAs (metadata2) clock.advance(blockIntervalMs) // advance clock to generate blocks - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { val combined = data1 ++ data2 listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs combined } @@ -112,7 +112,7 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim val combinedMetadata = metadata2 :+ metadata3 listener.addedMetadata.asScala.toSeq should contain theSameElementsInOrderAs (combinedMetadata) clock.advance(blockIntervalMs) // advance clock to generate blocks - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { val combinedData = data1 ++ data2 ++ data3 listener.pushedData.asScala.toSeq should contain theSameElementsInOrderAs (combinedData) } @@ -120,9 +120,9 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim // Stop the block generator by starting the stop on a different thread and // then advancing the manual clock for the stopping to proceed. val thread = stopBlockGenerator(blockGenerator) - eventually(timeout(1 second), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { clock.advance(blockIntervalMs) - assert(blockGenerator.isStopped() === true) + assert(blockGenerator.isStopped()) } thread.join() @@ -148,7 +148,7 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim blockGenerator = new BlockGenerator(listener, 0, conf, clock) require(listener.onGenerateBlockCalled === false) blockGenerator.start() - assert(blockGenerator.isActive() === true, "block generator") + assert(blockGenerator.isActive(), "block generator") assert(blockGenerator.isStopped() === false) val data = 1 to 1000 @@ -160,7 +160,7 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim // - Finally, wait for all blocks to be pushed clock.advance(1) // to make sure that the timer for another interval to complete val thread = stopBlockGenerator(blockGenerator) - eventually(timeout(1 second), interval(10 milliseconds)) { + eventually(timeout(1.second), interval(10.milliseconds)) { assert(blockGenerator.isActive() === false) } assert(blockGenerator.isStopped() === false) @@ -181,7 +181,7 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim // (expected as stop() should never complete) or a SparkException (unexpected as stop() // completed and thread terminated). val exception = intercept[Exception] { - failAfter(200 milliseconds) { + failAfter(200.milliseconds) { thread.join() throw new SparkException( "BlockGenerator.stop() completed before generating timer was stopped") @@ -193,12 +193,12 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim // Verify that the final data is present in the final generated block and // pushed before complete stop assert(blockGenerator.isStopped() === false) // generator has not stopped yet - eventually(timeout(10 seconds), interval(10 milliseconds)) { + eventually(timeout(10.seconds), interval(10.milliseconds)) { // Keep calling `advance` to avoid blocking forever in `clock.waitTillTime` clock.advance(blockIntervalMs) assert(thread.isAlive === false) } - assert(blockGenerator.isStopped() === true) // generator has finally been completely stopped + assert(blockGenerator.isStopped()) // generator has finally been completely stopped assert(listener.pushedData.asScala.toSeq === data, "All data not pushed by stop()") } @@ -213,8 +213,8 @@ class BlockGeneratorSuite extends SparkFunSuite with BeforeAndAfter with TimeLim blockGenerator.start() assert(listener.onErrorCalled === false) blockGenerator.addData(1) - eventually(timeout(1 second), interval(10 milliseconds)) { - assert(listener.onErrorCalled === true) + eventually(timeout(1.second), interval(10.milliseconds)) { + assert(listener.onErrorCalled) } blockGenerator.stop() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala index 8d81b582e4d30..a8b00558b40a7 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ExecutorAllocationManagerSuite.scala @@ -17,14 +17,16 @@ package org.apache.spark.streaming.scheduler -import org.mockito.Matchers.{eq => meq} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{eq => meq} +import org.mockito.Mockito.{never, reset, times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, PrivateMethodTester} import org.scalatest.concurrent.Eventually.{eventually, timeout} import org.scalatest.mockito.MockitoSugar import org.scalatest.time.SpanSugar._ import org.apache.spark.{ExecutorAllocationClient, SparkConf, SparkFunSuite} +import org.apache.spark.internal.config.{DYN_ALLOCATION_ENABLED, DYN_ALLOCATION_TESTING} +import org.apache.spark.internal.config.Streaming._ import org.apache.spark.streaming.{DummyInputDStream, Seconds, StreamingContext} import org.apache.spark.util.{ManualClock, Utils} @@ -32,8 +34,6 @@ import org.apache.spark.util.{ManualClock, Utils} class ExecutorAllocationManagerSuite extends SparkFunSuite with BeforeAndAfter with BeforeAndAfterAll with MockitoSugar with PrivateMethodTester { - import ExecutorAllocationManager._ - private val batchDurationMillis = 1000L private var allocationClient: ExecutorAllocationClient = null private var clock: StreamManualClock = null @@ -57,11 +57,11 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite reset(allocationClient) when(allocationClient.getExecutorIds()).thenReturn(Seq("1", "2")) addBatchProcTime(allocationManager, batchProcTimeMs.toLong) - val advancedTime = SCALING_INTERVAL_DEFAULT_SECS * 1000 + 1 + val advancedTime = STREAMING_DYN_ALLOCATION_SCALING_INTERVAL.defaultValue.get * 1000 + 1 val expectedWaitTime = clock.getTimeMillis() + advancedTime clock.advance(advancedTime) // Make sure ExecutorAllocationManager.manageAllocation is called - eventually(timeout(10 seconds)) { + eventually(timeout(10.seconds)) { assert(clock.isStreamWaitingAt(expectedWaitTime)) } body @@ -100,25 +100,29 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite } // Batch proc time slightly more than the scale up ratio, should increase allocation by 1 - addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT + 1) { + addBatchProcTimeAndVerifyAllocation( + batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.defaultValue.get + 1) { verifyTotalRequestedExecs(Some(3)) verifyKilledExec(None) } // Batch proc time slightly less than the scale up ratio, should not change allocation - addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_UP_RATIO_DEFAULT - 1) { + addBatchProcTimeAndVerifyAllocation( + batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_UP_RATIO.defaultValue.get - 1) { verifyTotalRequestedExecs(None) verifyKilledExec(None) } // Batch proc time slightly more than the scale down ratio, should not change allocation - addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT + 1) { + addBatchProcTimeAndVerifyAllocation( + batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.defaultValue.get + 1) { verifyTotalRequestedExecs(None) verifyKilledExec(None) } // Batch proc time slightly more than the scale down ratio, should not change allocation - addBatchProcTimeAndVerifyAllocation(batchDurationMillis * SCALING_DOWN_RATIO_DEFAULT - 1) { + addBatchProcTimeAndVerifyAllocation( + batchDurationMillis * STREAMING_DYN_ALLOCATION_SCALING_DOWN_RATIO.defaultValue.get - 1) { verifyTotalRequestedExecs(None) verifyKilledExec(Some("2")) } @@ -332,9 +336,9 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite val confWithBothDynamicAllocationEnabled = new SparkConf() .set("spark.streaming.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.enabled", "true") - .set("spark.dynamicAllocation.testing", "true") - require(Utils.isDynamicAllocationEnabled(confWithBothDynamicAllocationEnabled) === true) + .set(DYN_ALLOCATION_ENABLED, true) + .set(DYN_ALLOCATION_TESTING, true) + require(Utils.isDynamicAllocationEnabled(confWithBothDynamicAllocationEnabled)) withStreamingContext(confWithBothDynamicAllocationEnabled) { ssc => intercept[IllegalArgumentException] { ssc.start() @@ -384,7 +388,7 @@ class ExecutorAllocationManagerSuite extends SparkFunSuite } private def withStreamingContext(conf: SparkConf)(body: StreamingContext => Unit): Unit = { - conf.setMaster("myDummyLocalExternalClusterManager") + conf.setMaster("local-cluster[1,1,1024]") .setAppName(this.getClass.getSimpleName) .set("spark.streaming.dynamicAllocation.testing", "true") // to test dynamic allocation diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala index 5f7f7fa5e67f8..f0e502727402e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/JobGeneratorSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.streaming.scheduler import java.util.concurrent.CountDownLatch import scala.concurrent.duration._ -import scala.language.postfixOps import org.scalatest.concurrent.Eventually._ @@ -69,10 +68,10 @@ class JobGeneratorSuite extends TestSuiteBase { val longBatchNumber = 3 // 3rd batch will take a long time val longBatchTime = longBatchNumber * batchDuration.milliseconds - val testTimeout = timeout(10 seconds) + val testTimeout = timeout(10.seconds) val inputStream = ssc.receiverStream(new TestReceiver) - inputStream.foreachRDD((rdd: RDD[Int], time: Time) => { + inputStream.foreachRDD((_: RDD[Int], time: Time) => { if (time.milliseconds == longBatchTime) { while (waitLatch.getCount() > 0) { waitLatch.await() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala index c206d3169d77e..fec20f0429ff0 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/scheduler/ReceiverTrackerSuite.scala @@ -41,7 +41,7 @@ class ReceiverTrackerSuite extends TestSuiteBase { try { // we wait until the Receiver has registered with the tracker, // otherwise our rate update is lost - eventually(timeout(5 seconds)) { + eventually(timeout(5.seconds)) { assert(RateTestReceiver.getActive().nonEmpty) } @@ -49,7 +49,7 @@ class ReceiverTrackerSuite extends TestSuiteBase { // Verify that the rate of the block generator in the receiver get updated val activeReceiver = RateTestReceiver.getActive().get tracker.sendRateUpdate(inputDStream.id, newRateLimit) - eventually(timeout(5 seconds)) { + eventually(timeout(5.seconds)) { assert(activeReceiver.getDefaultBlockGeneratorRateLimit() === newRateLimit, "default block generator did not receive rate update") assert(activeReceiver.getCustomBlockGeneratorRateLimit() === newRateLimit, @@ -76,7 +76,7 @@ class ReceiverTrackerSuite extends TestSuiteBase { output.register() ssc.start() StoppableReceiver.shouldStop = true - eventually(timeout(10 seconds), interval(10 millis)) { + eventually(timeout(10.seconds), interval(10.milliseconds)) { // The receiver is stopped once, so if it's restarted, it should be started twice. assert(startTimes === 2) } @@ -98,7 +98,7 @@ class ReceiverTrackerSuite extends TestSuiteBase { val output = new TestOutputStream(input) output.register() ssc.start() - eventually(timeout(10 seconds), interval(10 millis)) { + eventually(timeout(10.seconds), interval(10.milliseconds)) { // If preferredLocations is set correctly, receiverTaskLocality should be PROCESS_LOCAL assert(receiverTaskLocality === TaskLocality.PROCESS_LOCAL) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala index 4a2549fc0a96d..8d2fa7d515e2f 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/WriteAheadLogSuite.scala @@ -26,13 +26,13 @@ import scala.collection.JavaConverters._ import scala.collection.mutable.ArrayBuffer import scala.concurrent._ import scala.concurrent.duration._ -import scala.language.{implicitConversions, postfixOps} +import scala.language.implicitConversions import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.mockito.ArgumentCaptor -import org.mockito.Matchers.{eq => meq, _} -import org.mockito.Mockito._ +import org.mockito.ArgumentMatchers.{any, anyLong, eq => meq} +import org.mockito.Mockito.{times, verify, when} import org.scalatest.{BeforeAndAfter, BeforeAndAfterEach, PrivateMethodTester} import org.scalatest.concurrent.Eventually import org.scalatest.concurrent.Eventually._ @@ -83,7 +83,7 @@ abstract class CommonWriteAheadLogTests( val logDirectoryPath = new Path(testDir) val fileSystem = HdfsUtils.getFileSystemForPath(logDirectoryPath, hadoopConf) - assert(fileSystem.exists(logDirectoryPath) === true) + assert(fileSystem.exists(logDirectoryPath)) // Read data using manager and verify val readData = readDataUsingWriteAheadLog(testDir, closeFileAfterWrite, allowBatching) @@ -135,7 +135,7 @@ abstract class CommonWriteAheadLogTests( if (waitForCompletion) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) } else { - eventually(Eventually.timeout(1 second), interval(10 milliseconds)) { + eventually(Eventually.timeout(1.second), interval(10.milliseconds)) { assert(getLogFilesInDirectory(testDir).size < logFiles.size) } } @@ -504,7 +504,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // The queue.take() immediately takes the 3, and there is nothing left in the queue at that // moment. Then the promise blocks the writing of 3. The rest get queued. writeAsync(batchedWal, event1, 3L) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(blockingWal.isBlocked) assert(batchedWal.invokePrivate(queueLength()) === 0) } @@ -514,12 +514,12 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // we would like event 5 to be written before event 4 in order to test that they get // sorted before being aggregated writeAsync(batchedWal, event5, 12L) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(blockingWal.isBlocked) assert(batchedWal.invokePrivate(queueLength()) === 3) } writeAsync(batchedWal, event4, 10L) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(walBatchingThreadPool.getActiveCount === 5) assert(batchedWal.invokePrivate(queueLength()) === 4) } @@ -528,7 +528,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val buffer = wrapArrayArrayByte(Array(event1)) val queuedEvents = Set(event2, event3, event4, event5) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(batchedWal.invokePrivate(queueLength()) === 0) verify(wal, times(1)).write(meq(buffer), meq(3L)) // the file name should be the timestamp of the last record, as events should be naturally @@ -559,7 +559,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( // The queue.take() immediately takes the 3, and there is nothing left in the queue at that // moment. Then the promise blocks the writing of 3. The rest get queued. val promise1 = writeAsync(batchedWal, event1, 3L) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(blockingWal.isBlocked) assert(batchedWal.invokePrivate(queueLength()) === 0) } @@ -567,7 +567,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val promise2 = writeAsync(batchedWal, event2, 5L) val promise3 = writeAsync(batchedWal, event3, 8L) - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(walBatchingThreadPool.getActiveCount === 3) assert(blockingWal.isBlocked) assert(batchedWal.invokePrivate(queueLength()) === 2) // event1 is being written @@ -576,7 +576,7 @@ class BatchedWriteAheadLogSuite extends CommonWriteAheadLogTests( val writePromises = Seq(promise1, promise2, promise3) batchedWal.close() - eventually(timeout(1 second)) { + eventually(timeout(1.second)) { assert(writePromises.forall(_.isCompleted)) assert(writePromises.forall(_.future.value.get.isFailure)) // all should have failed } @@ -772,7 +772,7 @@ object WriteAheadLogSuite { override def write(record: ByteBuffer, time: Long): WriteAheadLogRecordHandle = { isWriteCalled = true - eventually(Eventually.timeout(2 second)) { + eventually(Eventually.timeout(2.second)) { assert(!blockWrite) } wal.write(record, time) diff --git a/tools/pom.xml b/tools/pom.xml index 247f5a6df4b08..6286fad403c83 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -19,12 +19,12 @@ 4.0.0 org.apache.spark - spark-parent_2.11 + spark-parent_2.12 3.0.0-SNAPSHOT ../pom.xml - spark-tools_2.11 + spark-tools_2.12 tools